From cb0cddffe9452937033e0e6b1fc0e600d2c787ad Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 6 Sep 2019 11:45:14 +0800 Subject: [PATCH 0001/1387] [SPARK-21870][SQL] Split aggregation code into small functions ## What changes were proposed in this pull request? This pr proposed to split aggregation code into small functions in `HashAggregateExec`. In #18810, we got performance regression if JVMs didn't compile too long functions. I checked and I found the codegen of `HashAggregateExec` frequently goes over the limit when a query has too many aggregate functions (e.g., q66 in TPCDS). The current master places all the generated aggregation code in a single function. In this pr, I modified the code to assign an individual function for each aggregate function (e.g., `SUM` and `AVG`). For example, in a query `SELECT SUM(a), AVG(a) FROM VALUES(1) t(a)`, the proposed code defines two functions for `SUM(a)` and `AVG(a)` as follows; - generated code with this pr (https://gist.github.com/maropu/812990012bc967a78364be0fa793f559): ``` /* 173 */ private void agg_doConsume_0(InternalRow inputadapter_row_0, long agg_expr_0_0, boolean agg_exprIsNull_0_0, double agg_expr_1_0, boolean agg_exprIsNull_1_0, long agg_expr_2_0, boolean agg_exprIsNull_2_0) throws java.io.IOException { /* 174 */ // do aggregate /* 175 */ // common sub-expressions /* 176 */ /* 177 */ // evaluate aggregate functions and update aggregation buffers /* 178 */ agg_doAggregate_sum_0(agg_exprIsNull_0_0, agg_expr_0_0); /* 179 */ agg_doAggregate_avg_0(agg_expr_1_0, agg_exprIsNull_1_0, agg_exprIsNull_2_0, agg_expr_2_0); /* 180 */ /* 181 */ } ... /* 071 */ private void agg_doAggregate_avg_0(double agg_expr_1_0, boolean agg_exprIsNull_1_0, boolean agg_exprIsNull_2_0, long agg_expr_2_0) throws java.io.IOException { /* 072 */ // do aggregate for avg /* 073 */ // evaluate aggregate function /* 074 */ boolean agg_isNull_19 = true; /* 075 */ double agg_value_19 = -1.0; ... /* 114 */ private void agg_doAggregate_sum_0(boolean agg_exprIsNull_0_0, long agg_expr_0_0) throws java.io.IOException { /* 115 */ // do aggregate for sum /* 116 */ // evaluate aggregate function /* 117 */ agg_agg_isNull_11_0 = true; /* 118 */ long agg_value_11 = -1L; ``` - generated code in the current master (https://gist.github.com/maropu/e9d772af2c98d8991a6a5f0af7841760) ``` /* 059 */ private void agg_doConsume_0(InternalRow localtablescan_row_0, int agg_expr_0_0) throws java.io.IOException { /* 060 */ // do aggregate /* 061 */ // common sub-expressions /* 062 */ boolean agg_isNull_4 = false; /* 063 */ long agg_value_4 = -1L; /* 064 */ if (!false) { /* 065 */ agg_value_4 = (long) agg_expr_0_0; /* 066 */ } /* 067 */ // evaluate aggregate function /* 068 */ agg_agg_isNull_7_0 = true; /* 069 */ long agg_value_7 = -1L; /* 070 */ do { /* 071 */ if (!agg_bufIsNull_0) { /* 072 */ agg_agg_isNull_7_0 = false; /* 073 */ agg_value_7 = agg_bufValue_0; /* 074 */ continue; /* 075 */ } /* 076 */ /* 077 */ boolean agg_isNull_9 = false; /* 078 */ long agg_value_9 = -1L; /* 079 */ if (!false) { /* 080 */ agg_value_9 = (long) 0; /* 081 */ } /* 082 */ if (!agg_isNull_9) { /* 083 */ agg_agg_isNull_7_0 = false; /* 084 */ agg_value_7 = agg_value_9; /* 085 */ continue; /* 086 */ } /* 087 */ /* 088 */ } while (false); /* 089 */ /* 090 */ long agg_value_6 = -1L; /* 091 */ /* 092 */ agg_value_6 = agg_value_7 + agg_value_4; /* 093 */ boolean agg_isNull_11 = true; /* 094 */ double agg_value_11 = -1.0; /* 095 */ /* 096 */ if (!agg_bufIsNull_1) { /* 097 */ agg_agg_isNull_13_0 = true; /* 098 */ double agg_value_13 = -1.0; /* 099 */ do { /* 100 */ boolean agg_isNull_14 = agg_isNull_4; /* 101 */ double agg_value_14 = -1.0; /* 102 */ if (!agg_isNull_4) { /* 103 */ agg_value_14 = (double) agg_value_4; /* 104 */ } /* 105 */ if (!agg_isNull_14) { /* 106 */ agg_agg_isNull_13_0 = false; /* 107 */ agg_value_13 = agg_value_14; /* 108 */ continue; /* 109 */ } /* 110 */ /* 111 */ boolean agg_isNull_15 = false; /* 112 */ double agg_value_15 = -1.0; /* 113 */ if (!false) { /* 114 */ agg_value_15 = (double) 0; /* 115 */ } /* 116 */ if (!agg_isNull_15) { /* 117 */ agg_agg_isNull_13_0 = false; /* 118 */ agg_value_13 = agg_value_15; /* 119 */ continue; /* 120 */ } /* 121 */ /* 122 */ } while (false); /* 123 */ /* 124 */ agg_isNull_11 = false; // resultCode could change nullability. /* 125 */ /* 126 */ agg_value_11 = agg_bufValue_1 + agg_value_13; /* 127 */ /* 128 */ } /* 129 */ boolean agg_isNull_17 = false; /* 130 */ long agg_value_17 = -1L; /* 131 */ if (!false && agg_isNull_4) { /* 132 */ agg_isNull_17 = agg_bufIsNull_2; /* 133 */ agg_value_17 = agg_bufValue_2; /* 134 */ } else { /* 135 */ boolean agg_isNull_20 = true; /* 136 */ long agg_value_20 = -1L; /* 137 */ /* 138 */ if (!agg_bufIsNull_2) { /* 139 */ agg_isNull_20 = false; // resultCode could change nullability. /* 140 */ /* 141 */ agg_value_20 = agg_bufValue_2 + 1L; /* 142 */ /* 143 */ } /* 144 */ agg_isNull_17 = agg_isNull_20; /* 145 */ agg_value_17 = agg_value_20; /* 146 */ } /* 147 */ // update aggregation buffer /* 148 */ agg_bufIsNull_0 = false; /* 149 */ agg_bufValue_0 = agg_value_6; /* 150 */ /* 151 */ agg_bufIsNull_1 = agg_isNull_11; /* 152 */ agg_bufValue_1 = agg_value_11; /* 153 */ /* 154 */ agg_bufIsNull_2 = agg_isNull_17; /* 155 */ agg_bufValue_2 = agg_value_17; /* 156 */ /* 157 */ } ``` You can check the previous discussion in https://github.com/apache/spark/pull/19082 ## How was this patch tested? Existing tests Closes #20965 from maropu/SPARK-21870-2. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/dsl/package.scala | 8 +- .../expressions/codegen/CodeGenerator.scala | 51 +++ .../expressions/codegen/javaCode.scala | 5 +- .../expressions/nullExpressions.scala | 12 +- .../apache/spark/sql/internal/SQLConf.scala | 11 + .../aggregate/HashAggregateExec.scala | 309 ++++++++++++++---- .../execution/WholeStageCodegenSuite.scala | 21 ++ 7 files changed, 348 insertions(+), 69 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 796043fff665e..d37d81753f0b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -115,7 +115,13 @@ package object dsl { def getField(fieldName: String): UnresolvedExtractValue = UnresolvedExtractValue(expr, Literal(fieldName)) - def cast(to: DataType): Expression = Cast(expr, to) + def cast(to: DataType): Expression = { + if (expr.resolved && expr.dataType.sameType(to)) { + expr + } else { + Cast(expr, to) + } + } def asc: SortOrder = SortOrder(expr, Ascending) def asc_nullsLast: SortOrder = SortOrder(expr, Ascending, NullsLast, Set.empty) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 95fad412002e2..4c1bfcfdf7f17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1612,6 +1612,48 @@ object CodeGenerator extends Logging { } } + /** + * Extracts all the input variables from references and subexpression elimination states + * for a given `expr`. This result will be used to split the generated code of + * expressions into multiple functions. + */ + def getLocalInputVariableValues( + ctx: CodegenContext, + expr: Expression, + subExprs: Map[Expression, SubExprEliminationState]): Set[VariableValue] = { + val argSet = mutable.Set[VariableValue]() + if (ctx.INPUT_ROW != null) { + argSet += JavaCode.variable(ctx.INPUT_ROW, classOf[InternalRow]) + } + + // Collects local variables from a given `expr` tree + val collectLocalVariable = (ev: ExprValue) => ev match { + case vv: VariableValue => argSet += vv + case _ => + } + + val stack = mutable.Stack[Expression](expr) + while (stack.nonEmpty) { + stack.pop() match { + case e if subExprs.contains(e) => + val SubExprEliminationState(isNull, value) = subExprs(e) + collectLocalVariable(value) + collectLocalVariable(isNull) + + case ref: BoundReference if ctx.currentVars != null && + ctx.currentVars(ref.ordinal) != null => + val ExprCode(_, isNull, value) = ctx.currentVars(ref.ordinal) + collectLocalVariable(value) + collectLocalVariable(isNull) + + case e => + stack.pushAll(e.children) + } + } + + argSet.toSet + } + /** * Returns the name used in accessor and setter for a Java primitive type. */ @@ -1719,6 +1761,15 @@ object CodeGenerator extends Logging { 1 + params.map(paramLengthForExpr).sum } + def calculateParamLengthFromExprValues(params: Seq[ExprValue]): Int = { + def paramLengthForExpr(input: ExprValue): Int = input.javaType match { + case java.lang.Long.TYPE | java.lang.Double.TYPE => 2 + case _ => 1 + } + // Initial value is 1 for `this`. + 1 + params.map(paramLengthForExpr).sum + } + /** * In Java, a method descriptor is valid only if it represents method parameters with a total * length less than a pre-defined constant. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala index 3bb3c602f775b..d9393b9df6bbd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala @@ -143,7 +143,10 @@ trait Block extends TreeNode[Block] with JavaCode { case _ => code.trim } - def length: Int = toString.length + def length: Int = { + // Returns a code length without comments + CodeFormatter.stripExtraNewLinesAndComments(toString).length + } def isEmpty: Boolean = toString.isEmpty 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 293d28e93039a..f54d5f167856c 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 @@ -354,12 +354,14 @@ case class IsNotNull(child: Expression) extends UnaryExpression with Predicate { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = child.genCode(ctx) - val value = eval.isNull match { - case TrueLiteral => FalseLiteral - case FalseLiteral => TrueLiteral - case v => JavaCode.isNullExpression(s"!$v") + val (value, newCode) = eval.isNull match { + case TrueLiteral => (FalseLiteral, EmptyBlock) + case FalseLiteral => (TrueLiteral, EmptyBlock) + case v => + val value = ctx.freshName("value") + (JavaCode.variable(value, BooleanType), code"boolean $value = !$v;") } - ExprCode(code = eval.code, isNull = FalseLiteral, value = value) + ExprCode(code = eval.code + newCode, isNull = FalseLiteral, value = value) } override def sql: String = s"(${child.sql} IS NOT NULL)" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 006bb99e59392..0e7391063ed13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1080,6 +1080,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val CODEGEN_SPLIT_AGGREGATE_FUNC = + buildConf("spark.sql.codegen.aggregate.splitAggregateFunc.enabled") + .internal() + .doc("When true, the code generator would split aggregate code into individual methods " + + "instead of a single big method. This can be used to avoid oversized function that " + + "can miss the opportunity of JIT optimization.") + .booleanConf + .createWithDefault(true) + val MAX_NESTED_VIEW_DEPTH = buildConf("spark.sql.view.maxNestedViewDepth") .internal() @@ -2353,6 +2362,8 @@ class SQLConf extends Serializable with Logging { def cartesianProductExecBufferSpillThreshold: Int = getConf(CARTESIAN_PRODUCT_EXEC_BUFFER_SPILL_THRESHOLD) + def codegenSplitAggregateFunc: Boolean = getConf(SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC) + def maxNestedViewDepth: Int = getConf(SQLConf.MAX_NESTED_VIEW_DEPTH) def starSchemaDetection: Boolean = getConf(STARSCHEMA_DETECTION) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 4a95f76381339..9242583d36717 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.aggregate import java.util.concurrent.TimeUnit._ +import scala.collection.mutable + import org.apache.spark.TaskContext import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.rdd.RDD @@ -174,8 +176,9 @@ case class HashAggregateExec( } } - // The variables used as aggregation buffer. Only used for aggregation without keys. - private var bufVars: Seq[ExprCode] = _ + // The variables are used as aggregation buffers and each aggregate function has one or more + // ExprCode to initialize its buffer slots. Only used for aggregation without keys. + private var bufVars: Seq[Seq[ExprCode]] = _ private def doProduceWithoutKeys(ctx: CodegenContext): String = { val initAgg = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initAgg") @@ -184,27 +187,30 @@ case class HashAggregateExec( // generate variables for aggregation buffer val functions = aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate]) - val initExpr = functions.flatMap(f => f.initialValues) - bufVars = initExpr.map { e => - val isNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "bufIsNull") - val value = ctx.addMutableState(CodeGenerator.javaType(e.dataType), "bufValue") - // The initial expression should not access any column - val ev = e.genCode(ctx) - val initVars = code""" - | $isNull = ${ev.isNull}; - | $value = ${ev.value}; - """.stripMargin - ExprCode( - ev.code + initVars, - JavaCode.isNullGlobal(isNull), - JavaCode.global(value, e.dataType)) + val initExpr = functions.map(f => f.initialValues) + bufVars = initExpr.map { exprs => + exprs.map { e => + val isNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "bufIsNull") + val value = ctx.addMutableState(CodeGenerator.javaType(e.dataType), "bufValue") + // The initial expression should not access any column + val ev = e.genCode(ctx) + val initVars = code""" + |$isNull = ${ev.isNull}; + |$value = ${ev.value}; + """.stripMargin + ExprCode( + ev.code + initVars, + JavaCode.isNullGlobal(isNull), + JavaCode.global(value, e.dataType)) + } } - val initBufVar = evaluateVariables(bufVars) + val flatBufVars = bufVars.flatten + val initBufVar = evaluateVariables(flatBufVars) // generate variables for output val (resultVars, genResult) = if (modes.contains(Final) || modes.contains(Complete)) { // evaluate aggregate results - ctx.currentVars = bufVars + ctx.currentVars = flatBufVars val aggResults = bindReferences( functions.map(_.evaluateExpression), aggregateBufferAttributes).map(_.genCode(ctx)) @@ -218,7 +224,7 @@ case class HashAggregateExec( """.stripMargin) } else if (modes.contains(Partial) || modes.contains(PartialMerge)) { // output the aggregate buffer directly - (bufVars, "") + (flatBufVars, "") } else { // no aggregate function, the result should be literals val resultVars = resultExpressions.map(_.genCode(ctx)) @@ -255,11 +261,85 @@ case class HashAggregateExec( """.stripMargin } + private def isValidParamLength(paramLength: Int): Boolean = { + // This config is only for testing + sqlContext.getConf("spark.sql.HashAggregateExec.validParamLength", null) match { + case null | "" => CodeGenerator.isValidParamLength(paramLength) + case validLength => paramLength <= validLength.toInt + } + } + + // Splits aggregate code into small functions because the most of JVM implementations + // can not compile too long functions. Returns None if we are not able to split the given code. + // + // Note: The difference from `CodeGenerator.splitExpressions` is that we define an individual + // function for each aggregation function (e.g., SUM and AVG). For example, in a query + // `SELECT SUM(a), AVG(a) FROM VALUES(1) t(a)`, we define two functions + // for `SUM(a)` and `AVG(a)`. + private def splitAggregateExpressions( + ctx: CodegenContext, + aggNames: Seq[String], + aggBufferUpdatingExprs: Seq[Seq[Expression]], + aggCodeBlocks: Seq[Block], + subExprs: Map[Expression, SubExprEliminationState]): Option[String] = { + val exprValsInSubExprs = subExprs.flatMap { case (_, s) => s.value :: s.isNull :: Nil } + if (exprValsInSubExprs.exists(_.isInstanceOf[SimpleExprValue])) { + // `SimpleExprValue`s cannot be used as an input variable for split functions, so + // we give up splitting functions if it exists in `subExprs`. + None + } else { + val inputVars = aggBufferUpdatingExprs.map { aggExprsForOneFunc => + val inputVarsForOneFunc = aggExprsForOneFunc.map( + CodeGenerator.getLocalInputVariableValues(ctx, _, subExprs)).reduce(_ ++ _).toSeq + val paramLength = CodeGenerator.calculateParamLengthFromExprValues(inputVarsForOneFunc) + + // Checks if a parameter length for the `aggExprsForOneFunc` does not go over the JVM limit + if (isValidParamLength(paramLength)) { + Some(inputVarsForOneFunc) + } else { + None + } + } + + // Checks if all the aggregate code can be split into pieces. + // If the parameter length of at lease one `aggExprsForOneFunc` goes over the limit, + // we totally give up splitting aggregate code. + if (inputVars.forall(_.isDefined)) { + val splitCodes = inputVars.flatten.zipWithIndex.map { case (args, i) => + val doAggFunc = ctx.freshName(s"doAggregate_${aggNames(i)}") + val argList = args.map(v => s"${v.javaType.getName} ${v.variableName}").mkString(", ") + val doAggFuncName = ctx.addNewFunction(doAggFunc, + s""" + |private void $doAggFunc($argList) throws java.io.IOException { + | ${aggCodeBlocks(i)} + |} + """.stripMargin) + + val inputVariables = args.map(_.variableName).mkString(", ") + s"$doAggFuncName($inputVariables);" + } + Some(splitCodes.mkString("\n").trim) + } else { + val errMsg = "Failed to split aggregate code into small functions because the parameter " + + "length of at least one split function went over the JVM limit: " + + CodeGenerator.MAX_JVM_METHOD_PARAMS_LENGTH + if (Utils.isTesting) { + throw new IllegalStateException(errMsg) + } else { + logInfo(errMsg) + None + } + } + } + } + private def doConsumeWithoutKeys(ctx: CodegenContext, input: Seq[ExprCode]): String = { // only have DeclarativeAggregate val functions = aggregateExpressions.map(_.aggregateFunction.asInstanceOf[DeclarativeAggregate]) val inputAttrs = functions.flatMap(_.aggBufferAttributes) ++ child.output - val updateExpr = aggregateExpressions.flatMap { e => + // To individually generate code for each aggregate function, an element in `updateExprs` holds + // all the expressions for the buffer of an aggregation function. + val updateExprs = aggregateExpressions.map { e => e.mode match { case Partial | Complete => e.aggregateFunction.asInstanceOf[DeclarativeAggregate].updateExpressions @@ -267,28 +347,56 @@ case class HashAggregateExec( e.aggregateFunction.asInstanceOf[DeclarativeAggregate].mergeExpressions } } - ctx.currentVars = bufVars ++ input - val boundUpdateExpr = bindReferences(updateExpr, inputAttrs) - val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + ctx.currentVars = bufVars.flatten ++ input + val boundUpdateExprs = updateExprs.map { updateExprsForOneFunc => + bindReferences(updateExprsForOneFunc, inputAttrs) + } + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExprs.flatten) val effectiveCodes = subExprs.codes.mkString("\n") - val aggVals = ctx.withSubExprEliminationExprs(subExprs.states) { - boundUpdateExpr.map(_.genCode(ctx)) + val bufferEvals = boundUpdateExprs.map { boundUpdateExprsForOneFunc => + ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExprsForOneFunc.map(_.genCode(ctx)) + } } - // aggregate buffer should be updated atomic - val updates = aggVals.zipWithIndex.map { case (ev, i) => - s""" - | ${bufVars(i).isNull} = ${ev.isNull}; - | ${bufVars(i).value} = ${ev.value}; + + val aggNames = functions.map(_.prettyName) + val aggCodeBlocks = bufferEvals.zipWithIndex.map { case (bufferEvalsForOneFunc, i) => + val bufVarsForOneFunc = bufVars(i) + // All the update code for aggregation buffers should be placed in the end + // of each aggregation function code. + val updates = bufferEvalsForOneFunc.zip(bufVarsForOneFunc).map { case (ev, bufVar) => + s""" + |${bufVar.isNull} = ${ev.isNull}; + |${bufVar.value} = ${ev.value}; + """.stripMargin + } + code""" + |// do aggregate for ${aggNames(i)} + |// evaluate aggregate function + |${evaluateVariables(bufferEvalsForOneFunc)} + |// update aggregation buffers + |${updates.mkString("\n").trim} """.stripMargin } + + val codeToEvalAggFunc = if (conf.codegenSplitAggregateFunc && + aggCodeBlocks.map(_.length).sum > conf.methodSplitThreshold) { + val maybeSplitCode = splitAggregateExpressions( + ctx, aggNames, boundUpdateExprs, aggCodeBlocks, subExprs.states) + + maybeSplitCode.getOrElse { + aggCodeBlocks.fold(EmptyBlock)(_ + _).code + } + } else { + aggCodeBlocks.fold(EmptyBlock)(_ + _).code + } + s""" - | // do aggregate - | // common sub-expressions - | $effectiveCodes - | // evaluate aggregate function - | ${evaluateVariables(aggVals)} - | // update aggregation buffer - | ${updates.mkString("\n").trim} + |// do aggregate + |// common sub-expressions + |$effectiveCodes + |// evaluate aggregate functions and update aggregation buffers + |$codeToEvalAggFunc """.stripMargin } @@ -745,8 +853,10 @@ case class HashAggregateExec( val unsafeRowBuffer = ctx.freshName("unsafeRowAggBuffer") val fastRowBuffer = ctx.freshName("fastAggBuffer") - // only have DeclarativeAggregate - val updateExpr = aggregateExpressions.flatMap { e => + // To individually generate code for each aggregate function, an element in `updateExprs` holds + // all the expressions for the buffer of an aggregation function. + val updateExprs = aggregateExpressions.map { e => + // only have DeclarativeAggregate e.mode match { case Partial | Complete => e.aggregateFunction.asInstanceOf[DeclarativeAggregate].updateExpressions @@ -824,25 +934,70 @@ case class HashAggregateExec( // generating input columns, we use `currentVars`. ctx.currentVars = new Array[ExprCode](aggregateBufferAttributes.length) ++ input + val aggNames = aggregateExpressions.map(_.aggregateFunction.prettyName) + // Computes start offsets for each aggregation function code + // in the underlying buffer row. + val bufferStartOffsets = { + val offsets = mutable.ArrayBuffer[Int]() + var curOffset = 0 + updateExprs.foreach { exprsForOneFunc => + offsets += curOffset + curOffset += exprsForOneFunc.length + } + offsets.toArray + } + val updateRowInRegularHashMap: String = { ctx.INPUT_ROW = unsafeRowBuffer - val boundUpdateExpr = bindReferences(updateExpr, inputAttr) - val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val boundUpdateExprs = updateExprs.map { updateExprsForOneFunc => + bindReferences(updateExprsForOneFunc, inputAttr) + } + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExprs.flatten) val effectiveCodes = subExprs.codes.mkString("\n") - val unsafeRowBufferEvals = ctx.withSubExprEliminationExprs(subExprs.states) { - boundUpdateExpr.map(_.genCode(ctx)) + val unsafeRowBufferEvals = boundUpdateExprs.map { boundUpdateExprsForOneFunc => + ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExprsForOneFunc.map(_.genCode(ctx)) + } } - val updateUnsafeRowBuffer = unsafeRowBufferEvals.zipWithIndex.map { case (ev, i) => - val dt = updateExpr(i).dataType - CodeGenerator.updateColumn(unsafeRowBuffer, dt, i, ev, updateExpr(i).nullable) + + val aggCodeBlocks = updateExprs.indices.map { i => + val rowBufferEvalsForOneFunc = unsafeRowBufferEvals(i) + val boundUpdateExprsForOneFunc = boundUpdateExprs(i) + val bufferOffset = bufferStartOffsets(i) + + // All the update code for aggregation buffers should be placed in the end + // of each aggregation function code. + val updateRowBuffers = rowBufferEvalsForOneFunc.zipWithIndex.map { case (ev, j) => + val updateExpr = boundUpdateExprsForOneFunc(j) + val dt = updateExpr.dataType + val nullable = updateExpr.nullable + CodeGenerator.updateColumn(unsafeRowBuffer, dt, bufferOffset + j, ev, nullable) + } + code""" + |// evaluate aggregate function for ${aggNames(i)} + |${evaluateVariables(rowBufferEvalsForOneFunc)} + |// update unsafe row buffer + |${updateRowBuffers.mkString("\n").trim} + """.stripMargin } + + val codeToEvalAggFunc = if (conf.codegenSplitAggregateFunc && + aggCodeBlocks.map(_.length).sum > conf.methodSplitThreshold) { + val maybeSplitCode = splitAggregateExpressions( + ctx, aggNames, boundUpdateExprs, aggCodeBlocks, subExprs.states) + + maybeSplitCode.getOrElse { + aggCodeBlocks.fold(EmptyBlock)(_ + _).code + } + } else { + aggCodeBlocks.fold(EmptyBlock)(_ + _).code + } + s""" |// common sub-expressions |$effectiveCodes - |// evaluate aggregate function - |${evaluateVariables(unsafeRowBufferEvals)} - |// update unsafe row buffer - |${updateUnsafeRowBuffer.mkString("\n").trim} + |// evaluate aggregate functions and update aggregation buffers + |$codeToEvalAggFunc """.stripMargin } @@ -850,16 +1005,48 @@ case class HashAggregateExec( if (isFastHashMapEnabled) { if (isVectorizedHashMapEnabled) { ctx.INPUT_ROW = fastRowBuffer - val boundUpdateExpr = bindReferences(updateExpr, inputAttr) - val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val boundUpdateExprs = updateExprs.map { updateExprsForOneFunc => + bindReferences(updateExprsForOneFunc, inputAttr) + } + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExprs.flatten) val effectiveCodes = subExprs.codes.mkString("\n") - val fastRowEvals = ctx.withSubExprEliminationExprs(subExprs.states) { - boundUpdateExpr.map(_.genCode(ctx)) + val fastRowEvals = boundUpdateExprs.map { boundUpdateExprsForOneFunc => + ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExprsForOneFunc.map(_.genCode(ctx)) + } } - val updateFastRow = fastRowEvals.zipWithIndex.map { case (ev, i) => - val dt = updateExpr(i).dataType - CodeGenerator.updateColumn( - fastRowBuffer, dt, i, ev, updateExpr(i).nullable, isVectorized = true) + + val aggCodeBlocks = fastRowEvals.zipWithIndex.map { case (fastRowEvalsForOneFunc, i) => + val boundUpdateExprsForOneFunc = boundUpdateExprs(i) + val bufferOffset = bufferStartOffsets(i) + // All the update code for aggregation buffers should be placed in the end + // of each aggregation function code. + val updateRowBuffer = fastRowEvalsForOneFunc.zipWithIndex.map { case (ev, j) => + val updateExpr = boundUpdateExprsForOneFunc(j) + val dt = updateExpr.dataType + val nullable = updateExpr.nullable + CodeGenerator.updateColumn(fastRowBuffer, dt, bufferOffset + j, ev, nullable, + isVectorized = true) + } + code""" + |// evaluate aggregate function for ${aggNames(i)} + |${evaluateVariables(fastRowEvalsForOneFunc)} + |// update fast row + |${updateRowBuffer.mkString("\n").trim} + """.stripMargin + } + + + val codeToEvalAggFunc = if (conf.codegenSplitAggregateFunc && + aggCodeBlocks.map(_.length).sum > conf.methodSplitThreshold) { + val maybeSplitCode = splitAggregateExpressions( + ctx, aggNames, boundUpdateExprs, aggCodeBlocks, subExprs.states) + + maybeSplitCode.getOrElse { + aggCodeBlocks.fold(EmptyBlock)(_ + _).code + } + } else { + aggCodeBlocks.fold(EmptyBlock)(_ + _).code } // If vectorized fast hash map is on, we first generate code to update row @@ -869,10 +1056,8 @@ case class HashAggregateExec( |if ($fastRowBuffer != null) { | // common sub-expressions | $effectiveCodes - | // evaluate aggregate function - | ${evaluateVariables(fastRowEvals)} - | // update fast row - | ${updateFastRow.mkString("\n").trim} + | // evaluate aggregate functions and update aggregation buffers + | $codeToEvalAggFunc |} else { | $updateRowInRegularHashMap |} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 0ea16a1a15d66..d8727d5b584f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -398,4 +398,25 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { }.isDefined, "LocalTableScanExec should be within a WholeStageCodegen domain.") } + + test("Give up splitting aggregate code if a parameter length goes over the limit") { + withSQLConf( + SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC.key -> "true", + SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1", + "spark.sql.HashAggregateExec.validParamLength" -> "0") { + withTable("t") { + val expectedErrMsg = "Failed to split aggregate code into small functions" + Seq( + // Test case without keys + "SELECT AVG(v) FROM VALUES(1) t(v)", + // Tet case with keys + "SELECT k, AVG(v) FROM VALUES((1, 1)) t(k, v) GROUP BY k").foreach { query => + val errMsg = intercept[IllegalStateException] { + sql(query).collect + }.getMessage + assert(errMsg.contains(expectedErrMsg)) + } + } + } + } } From 4664a082c2c7ac989e818958c465c72833d3ccfe Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 6 Sep 2019 11:50:45 +0800 Subject: [PATCH 0002/1387] [SPARK-28968][ML] Add HasNumFeatures in the scala side ### What changes were proposed in this pull request? Add HasNumFeatures in the scala side, with `1<<18` as the default value ### Why are the changes needed? HasNumFeatures is already added in the py side, it is reasonable to keep them in sync. I don't find other similar place. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing testsuites Closes #25671 from zhengruifeng/add_HasNumFeatures. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../spark/ml/feature/FeatureHasher.scala | 19 ++------------ .../apache/spark/ml/feature/HashingTF.scala | 20 +++------------ .../ml/param/shared/SharedParamsCodeGen.scala | 7 +++--- .../spark/ml/param/shared/sharedParams.scala | 25 ++++++++++++++++--- project/MimaExcludes.scala | 6 +++++ .../ml/param/_shared_params_code_gen.py | 3 ++- python/pyspark/ml/param/shared.py | 5 ++-- 7 files changed, 43 insertions(+), 42 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala index 0a4f1b98ef67a..61b4d5d54aca4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala @@ -23,7 +23,7 @@ import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators, StringArrayParam} -import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol} +import org.apache.spark.ml.param.shared.{HasInputCols, HasNumFeatures, HasOutputCol} import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF} import org.apache.spark.sql.{DataFrame, Dataset, Row} @@ -83,7 +83,7 @@ import org.apache.spark.util.collection.OpenHashMap */ @Since("2.3.0") class FeatureHasher(@Since("2.3.0") override val uid: String) extends Transformer - with HasInputCols with HasOutputCol with DefaultParamsWritable { + with HasInputCols with HasOutputCol with HasNumFeatures with DefaultParamsWritable { @Since("2.3.0") def this() = this(Identifiable.randomUID("featureHasher")) @@ -99,21 +99,6 @@ class FeatureHasher(@Since("2.3.0") override val uid: String) extends Transforme val categoricalCols = new StringArrayParam(this, "categoricalCols", "numeric columns to treat as categorical") - /** - * Number of features. Should be greater than 0. - * (default = 2^18^) - * @group param - */ - @Since("2.3.0") - val numFeatures = new IntParam(this, "numFeatures", "number of features (> 0)", - ParamValidators.gt(0)) - - setDefault(numFeatures -> (1 << 18)) - - /** @group getParam */ - @Since("2.3.0") - def getNumFeatures: Int = $(numFeatures) - /** @group setParam */ @Since("2.3.0") def setNumFeatures(value: Int): this.type = set(numFeatures, value) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index b8ce9c3169a68..fe9f4f2123da2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared.{HasInputCol, HasNumFeatures, HasOutputCol} import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF} import org.apache.spark.sql.{DataFrame, Dataset} @@ -43,7 +43,8 @@ import org.apache.spark.util.VersionUtils.majorMinorVersion */ @Since("1.2.0") class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + extends Transformer with HasInputCol with HasOutputCol with HasNumFeatures + with DefaultParamsWritable { private var hashFunc: Any => Int = FeatureHasher.murmur3Hash @@ -58,15 +59,6 @@ class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - /** - * Number of features. Should be greater than 0. - * (default = 2^18^) - * @group param - */ - @Since("1.2.0") - val numFeatures = new IntParam(this, "numFeatures", "number of features (> 0)", - ParamValidators.gt(0)) - /** * Binary toggle to control term frequency counts. * If true, all non-zero counts are set to 1. This is useful for discrete probabilistic @@ -79,11 +71,7 @@ class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) "This is useful for discrete probabilistic models that model binary events rather " + "than integer counts") - setDefault(numFeatures -> (1 << 18), binary -> false) - - /** @group getParam */ - @Since("1.2.0") - def getNumFeatures: Int = $(numFeatures) + setDefault(binary -> false) /** @group setParam */ @Since("1.2.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index 1afcf1bf7ba4b..f4abe9aa1c640 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -63,6 +63,8 @@ private[shared] object SharedParamsCodeGen { ParamDesc[Array[String]]("inputCols", "input column names"), ParamDesc[String]("outputCol", "output column name", Some("uid + \"__output\"")), ParamDesc[Array[String]]("outputCols", "output column names"), + ParamDesc[Int]("numFeatures", "Number of features. Should be greater than 0", + Some("262144"), isValid = "ParamValidators.gt(0)"), ParamDesc[Int]("checkpointInterval", "set checkpoint interval (>= 1) or " + "disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed " + "every 10 iterations. Note: this setting will be ignored if the checkpoint directory " + @@ -95,9 +97,8 @@ private[shared] object SharedParamsCodeGen { Some("false"), isExpertParam = true), ParamDesc[String]("loss", "the loss function to be optimized", finalFields = false), ParamDesc[String]("distanceMeasure", "The distance measure. Supported options: 'euclidean'" + - " and 'cosine'", Some("org.apache.spark.mllib.clustering.DistanceMeasure.EUCLIDEAN"), - isValid = "(value: String) => " + - "org.apache.spark.mllib.clustering.DistanceMeasure.validateDistanceMeasure(value)"), + " and 'cosine'", Some("\"euclidean\""), + isValid = "ParamValidators.inArray(Array(\"euclidean\", \"cosine\"))"), ParamDesc[String]("validationIndicatorCol", "name of the column that indicates whether " + "each row is for training or for validation. False indicates training; true indicates " + "validation.") diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 5928a0749f738..9baaf1744a3b7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -274,6 +274,25 @@ trait HasOutputCols extends Params { final def getOutputCols: Array[String] = $(outputCols) } +/** + * Trait for shared param numFeatures (default: 262144). This trait may be changed or + * removed between minor versions. + */ +@DeveloperApi +trait HasNumFeatures extends Params { + + /** + * Param for Number of features. Should be greater than 0. + * @group param + */ + final val numFeatures: IntParam = new IntParam(this, "numFeatures", "Number of features. Should be greater than 0", ParamValidators.gt(0)) + + setDefault(numFeatures, 262144) + + /** @group getParam */ + final def getNumFeatures: Int = $(numFeatures) +} + /** * Trait for shared param checkpointInterval. This trait may be changed or * removed between minor versions. @@ -506,7 +525,7 @@ trait HasLoss extends Params { } /** - * Trait for shared param distanceMeasure (default: org.apache.spark.mllib.clustering.DistanceMeasure.EUCLIDEAN). This trait may be changed or + * Trait for shared param distanceMeasure (default: "euclidean"). This trait may be changed or * removed between minor versions. */ @DeveloperApi @@ -516,9 +535,9 @@ trait HasDistanceMeasure extends Params { * Param for The distance measure. Supported options: 'euclidean' and 'cosine'. * @group param */ - final val distanceMeasure: Param[String] = new Param[String](this, "distanceMeasure", "The distance measure. Supported options: 'euclidean' and 'cosine'", (value: String) => org.apache.spark.mllib.clustering.DistanceMeasure.validateDistanceMeasure(value)) + final val distanceMeasure: Param[String] = new Param[String](this, "distanceMeasure", "The distance measure. Supported options: 'euclidean' and 'cosine'", ParamValidators.inArray(Array("euclidean", "cosine"))) - setDefault(distanceMeasure, org.apache.spark.mllib.clustering.DistanceMeasure.EUCLIDEAN) + setDefault(distanceMeasure, "euclidean") /** @group getParam */ final def getDistanceMeasure: String = $(distanceMeasure) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a1237271046e9..04bec14c42c6f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -200,6 +200,12 @@ object MimaExcludes { ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.tree.HasVarianceImpurity.org$apache$spark$ml$tree$HasVarianceImpurity$_setter_$impurity_="), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.tree.HasVarianceImpurity.org$apache$spark$ml$tree$HasVarianceImpurity$_setter_$impurity_="), + // [SPARK-28968][ML] Add HasNumFeatures in the scala side + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.FeatureHasher.getNumFeatures"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.FeatureHasher.numFeatures"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.HashingTF.getNumFeatures"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.HashingTF.numFeatures"), + // [SPARK-25908][CORE][SQL] Remove old deprecated items in Spark 3 ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.BarrierTaskContext.isRunningLocally"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskContext.isRunningLocally"), diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index ca2e4a0f5bde1..c99ec3f467ac6 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -120,7 +120,8 @@ def get$Name(self): ("inputCols", "input column names.", None, "TypeConverters.toListString"), ("outputCol", "output column name.", "self.uid + '__output'", "TypeConverters.toString"), ("outputCols", "output column names.", None, "TypeConverters.toListString"), - ("numFeatures", "number of features.", None, "TypeConverters.toInt"), + ("numFeatures", "Number of features. Should be greater than 0.", "262144", + "TypeConverters.toInt"), ("checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). " + "E.g. 10 means that the cache will get checkpointed every 10 iterations. Note: " + "this setting will be ignored if the checkpoint directory is not set in the SparkContext.", diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 9527ef6f576b2..771b4bcd9ba02 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -281,13 +281,14 @@ def getOutputCols(self): class HasNumFeatures(Params): """ - Mixin for param numFeatures: number of features. + Mixin for param numFeatures: Number of features. Should be greater than 0. """ - numFeatures = Param(Params._dummy(), "numFeatures", "number of features.", typeConverter=TypeConverters.toInt) + numFeatures = Param(Params._dummy(), "numFeatures", "Number of features. Should be greater than 0.", typeConverter=TypeConverters.toInt) def __init__(self): super(HasNumFeatures, self).__init__() + self._setDefault(numFeatures=262144) def setNumFeatures(self, value): """ From 905b7f7fc7d887da5d7c366b191baf4beea0aadf Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 6 Sep 2019 09:06:39 -0500 Subject: [PATCH 0003/1387] [SPARK-28967][CORE] Include cloned version of "properties" to avoid ConcurrentModificationException ### What changes were proposed in this pull request? This patch fixes the bug which throws ConcurrentModificationException when job with 0 partition is submitted via DAGScheduler. ### Why are the changes needed? Without this patch, structured streaming query throws ConcurrentModificationException, like below stack trace: ``` 19/09/04 09:48:49 ERROR AsyncEventQueue: Listener EventLoggingListener threw an exception java.util.ConcurrentModificationException at java.util.Hashtable$Enumerator.next(Hashtable.java:1387) at scala.collection.convert.Wrappers$JPropertiesWrapper$$anon$6.next(Wrappers.scala:424) at scala.collection.convert.Wrappers$JPropertiesWrapper$$anon$6.next(Wrappers.scala:420) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at scala.collection.AbstractIterable.foreach(Iterable.scala:56) at scala.collection.TraversableLike.map(TraversableLike.scala:237) at scala.collection.TraversableLike.map$(TraversableLike.scala:230) at scala.collection.AbstractTraversable.map(Traversable.scala:108) at org.apache.spark.util.JsonProtocol$.mapToJson(JsonProtocol.scala:514) at org.apache.spark.util.JsonProtocol$.$anonfun$propertiesToJson$1(JsonProtocol.scala:520) at scala.Option.map(Option.scala:163) at org.apache.spark.util.JsonProtocol$.propertiesToJson(JsonProtocol.scala:519) at org.apache.spark.util.JsonProtocol$.jobStartToJson(JsonProtocol.scala:155) at org.apache.spark.util.JsonProtocol$.sparkEventToJson(JsonProtocol.scala:79) at org.apache.spark.scheduler.EventLoggingListener.logEvent(EventLoggingListener.scala:149) at org.apache.spark.scheduler.EventLoggingListener.onJobStart(EventLoggingListener.scala:217) at org.apache.spark.scheduler.SparkListenerBus.doPostEvent(SparkListenerBus.scala:37) at org.apache.spark.scheduler.SparkListenerBus.doPostEvent$(SparkListenerBus.scala:28) at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37) at org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37) at org.apache.spark.util.ListenerBus.postToAll(ListenerBus.scala:99) at org.apache.spark.util.ListenerBus.postToAll$(ListenerBus.scala:84) at org.apache.spark.scheduler.AsyncEventQueue.super$postToAll(AsyncEventQueue.scala:102) at org.apache.spark.scheduler.AsyncEventQueue.$anonfun$dispatch$1(AsyncEventQueue.scala:102) at scala.runtime.java8.JFunction0$mcJ$sp.apply(JFunction0$mcJ$sp.java:23) at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62) at org.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:97) at org.apache.spark.scheduler.AsyncEventQueue$$anon$2.$anonfun$run$1(AsyncEventQueue.scala:93) at org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1319) at org.apache.spark.scheduler.AsyncEventQueue$$anon$2.run(AsyncEventQueue.scala:93) ``` Please refer https://issues.apache.org/jira/browse/SPARK-28967 for detailed reproducer. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Newly added UT. Also manually tested via running simple structured streaming query in spark-shell. Closes #25672 from HeartSaVioR/SPARK-28967. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Sean Owen --- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 35 +++++++++++++++++-- 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b08483267c141..9df59459ca799 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -698,7 +698,7 @@ private[spark] class DAGScheduler( if (partitions.isEmpty) { val time = clock.getTimeMillis() listenerBus.post( - SparkListenerJobStart(jobId, time, Seq[StageInfo](), properties)) + SparkListenerJobStart(jobId, time, Seq[StageInfo](), SerializationUtils.clone(properties))) listenerBus.post( SparkListenerJobEnd(jobId, time, JobSucceeded)) // Return immediately if the job is running 0 tasks diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 2b3423f9a4d40..cd854c379b08a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -19,13 +19,14 @@ package org.apache.spark.scheduler import java.util.Properties import java.util.concurrent.{CountDownLatch, TimeUnit} -import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicLong, AtomicReference} import scala.annotation.meta.param import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.util.control.NonFatal import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} +import org.scalatest.exceptions.TestFailedException import org.scalatest.time.SpanSugar._ import org.apache.spark._ @@ -36,7 +37,7 @@ import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, ThreadUtils, Utils} class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) extends DAGSchedulerEventProcessLoop(dagScheduler) { @@ -788,6 +789,36 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } } + test("SPARK-28967 properties must be cloned before posting to listener bus for 0 partition") { + val properties = new Properties() + val func = (context: TaskContext, it: Iterator[(_)]) => 1 + val resultHandler = (taskIndex: Int, result: Int) => {} + val assertionError = new AtomicReference[TestFailedException]( + new TestFailedException("Listener didn't receive expected JobStart event", 0)) + val listener = new SparkListener() { + override def onJobStart(event: SparkListenerJobStart): Unit = { + try { + assert(event.properties.equals(properties), "Expected same content of properties, " + + s"but got properties with different content. props in caller ${properties} /" + + s" props in event ${event.properties}") + assert(event.properties.ne(properties), "Expected instance with different identity, " + + "but got same instance.") + assertionError.set(null) + } catch { + case e: TestFailedException => assertionError.set(e) + } + } + } + sc.addSparkListener(listener) + + // 0 partition + val testRdd = new MyRDD(sc, 0, Nil) + val waiter = scheduler.submitJob(testRdd, func, Seq.empty, CallSite.empty, + resultHandler, properties) + sc.listenerBus.waitUntilEmpty(1000L) + assert(assertionError.get() === null) + } + // Helper function to validate state when creating tests for task failures private def checkStageId(stageId: Int, attempt: Int, stageAttempt: TaskSet) { assert(stageAttempt.stageId === stageId) From 67b4329fb08fd606461aa1ac9274c4a84d15d70e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 6 Sep 2019 23:36:00 +0900 Subject: [PATCH 0004/1387] [SPARK-28690][SQL] Add `date_part` function for timestamps/dates ## What changes were proposed in this pull request? In the PR, I propose new function `date_part()`. The function is modeled on the traditional Ingres equivalent to the SQL-standard function `extract`: ``` date_part('field', source) ``` and added for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT). The `source` can have `DATE` or `TIMESTAMP` type. Supported string values of `'field'` are: - `millennium` - the current millennium for given date (or a timestamp implicitly casted to a date). For example, years in the 1900s are in the second millennium. The third millennium started _January 1, 2001_. - `century` - the current millennium for given date (or timestamp). The first century starts at 0001-01-01 AD. - `decade` - the current decade for given date (or timestamp). Actually, this is the year field divided by 10. - isoyear` - the ISO 8601 week-numbering year that the date falls in. Each ISO 8601 week-numbering year begins with the Monday of the week containing the 4th of January. - `year`, `month`, `day`, `hour`, `minute`, `second` - `week` - the number of the ISO 8601 week-numbering week of the year. By definition, ISO weeks start on Mondays and the first week of a year contains January 4 of that year. - `quarter` - the quarter of the year (1 - 4) - `dayofweek` - the day of the week for date/timestamp (1 = Sunday, 2 = Monday, ..., 7 = Saturday) - `dow` - the day of the week as Sunday (0) to Saturday (6) - `isodow` - the day of the week as Monday (1) to Sunday (7) - `doy` - the day of the year (1 - 365/366) - `milliseconds` - the seconds field including fractional parts multiplied by 1,000. - `microseconds` - the seconds field including fractional parts multiplied by 1,000,000. - `epoch` - the number of seconds since 1970-01-01 00:00:00 local time in microsecond precision. Here are examples: ```sql spark-sql> select date_part('year', timestamp'2019-08-12 01:00:00.123456'); 2019 spark-sql> select date_part('week', timestamp'2019-08-12 01:00:00.123456'); 33 spark-sql> select date_part('doy', timestamp'2019-08-12 01:00:00.123456'); 224 ``` I changed implementation of `extract` to re-use `date_part()` internally. ## How was this patch tested? Added `date_part.sql` and regenerated results of `extract.sql`. Closes #25410 from MaxGekk/date_part. Lead-authored-by: Maxim Gekk Co-authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/datetimeExpressions.scala | 87 ++++ .../sql/catalyst/parser/AstBuilder.scala | 48 +- .../resources/sql-tests/inputs/date_part.sql | 68 +++ .../sql-tests/inputs/pgSQL/timestamp.sql | 31 +- .../sql-tests/results/date_part.sql.out | 412 ++++++++++++++++++ .../sql-tests/results/extract.sql.out | 126 +++--- .../sql-tests/results/pgSQL/date.sql.out | 52 +-- .../sql-tests/results/pgSQL/timestamp.sql.out | 55 ++- 9 files changed, 727 insertions(+), 153 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/date_part.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/date_part.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 5177f1e55829e..d5728b9027579 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -417,6 +417,7 @@ object FunctionRegistry { expression[TimeWindow]("window"), expression[MakeDate]("make_date"), expression[MakeTimestamp]("make_timestamp"), + expression[DatePart]("date_part"), // collection functions expression[CreateArray]("array"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 1ce493ece18b9..9d43701f03056 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1963,3 +1963,90 @@ case class Epoch(child: Expression, timeZoneId: Option[String] = None) defineCodeGen(ctx, ev, c => s"$dtu.getEpoch($c, $zid)") } } + +object DatePart { + + def parseExtractField( + extractField: String, + source: Expression, + errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { + case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millennium(source) + case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) + case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) + case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) + case "ISOYEAR" => IsoYear(source) + case "QUARTER" | "QTR" => Quarter(source) + case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) + case "WEEK" | "W" | "WEEKS" => WeekOfYear(source) + case "DAY" | "D" | "DAYS" => DayOfMonth(source) + case "DAYOFWEEK" => DayOfWeek(source) + case "DOW" => Subtract(DayOfWeek(source), Literal(1)) + case "ISODOW" => Add(WeekDay(source), Literal(1)) + case "DOY" => DayOfYear(source) + case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) + case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) + case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => + Milliseconds(source) + case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => + Microseconds(source) + case "EPOCH" => Epoch(source) + case _ => errorHandleFunc + } +} + +@ExpressionDescription( + usage = "_FUNC_(field, source) - Extracts a part of the date/timestamp.", + arguments = """ + Arguments: + * field - selects which part of the source should be extracted. Supported string values are: + ["MILLENNIUM", ("MILLENNIA", "MIL", "MILS"), + "CENTURY", ("CENTURIES", "C", "CENT"), + "DECADE", ("DECADES", "DEC", "DECS"), + "YEAR", ("Y", "YEARS", "YR", "YRS"), + "ISOYEAR", + "QUARTER", ("QTR"), + "MONTH", ("MON", "MONS", "MONTHS"), + "WEEK", ("W", "WEEKS"), + "DAY", ("D", "DAYS"), + "DAYOFWEEK", + "DOW", + "ISODOW", + "DOY", + "HOUR", ("H", "HOURS", "HR", "HRS"), + "MINUTE", ("M", "MIN", "MINS", "MINUTES"), + "SECOND", ("S", "SEC", "SECONDS", "SECS"), + "MILLISECONDS", ("MSEC", "MSECS", "MILLISECON", "MSECONDS", "MS"), + "MICROSECONDS", ("USEC", "USECS", "USECONDS", "MICROSECON", "US"), + "EPOCH"] + * source - a date (or timestamp) column from where `field` should be extracted + """, + examples = """ + Examples: + > SELECT _FUNC_('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456'); + 2019 + > SELECT _FUNC_('week', timestamp'2019-08-12 01:00:00.123456'); + 33 + > SELECT _FUNC_('doy', DATE'2019-08-12'); + 224 + """, + since = "3.0.0") +case class DatePart(field: Expression, source: Expression, child: Expression) + extends RuntimeReplaceable { + + def this(field: Expression, source: Expression) { + this(field, source, { + if (!field.foldable) { + throw new AnalysisException("The field parameter needs to be a foldable string value.") + } + val fieldStr = field.eval().asInstanceOf[UTF8String].toString + DatePart.parseExtractField(fieldStr, source, { + throw new AnalysisException(s"Literals of type '$fieldStr' are currently not supported.") + }) + }) + } + + override def flatArguments: Iterator[Any] = Iterator(field, source) + override def sql: String = s"$prettyName(${field.sql}, ${source.sql})" + override def prettyName: String = "date_part" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 27579273f08b6..90a533735aed2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1409,48 +1409,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a Extract expression. */ override def visitExtract(ctx: ExtractContext): Expression = withOrigin(ctx) { - ctx.field.getText.toUpperCase(Locale.ROOT) match { - case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => - Millennium(expression(ctx.source)) - case "CENTURY" | "CENTURIES" | "C" | "CENT" => - Century(expression(ctx.source)) - case "DECADE" | "DECADES" | "DEC" | "DECS" => - Decade(expression(ctx.source)) - case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => - Year(expression(ctx.source)) - case "ISOYEAR" => - IsoYear(expression(ctx.source)) - case "QUARTER" | "QTR" => - Quarter(expression(ctx.source)) - case "MONTH" | "MON" | "MONS" | "MONTHS" => - Month(expression(ctx.source)) - case "WEEK" | "W" | "WEEKS" => - WeekOfYear(expression(ctx.source)) - case "DAY" | "D" | "DAYS" => - DayOfMonth(expression(ctx.source)) - case "DAYOFWEEK" => - DayOfWeek(expression(ctx.source)) - case "DOW" => - Subtract(DayOfWeek(expression(ctx.source)), Literal(1)) - case "ISODOW" => - Add(WeekDay(expression(ctx.source)), Literal(1)) - case "DOY" => - DayOfYear(expression(ctx.source)) - case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => - Hour(expression(ctx.source)) - case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => - Minute(expression(ctx.source)) - case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => - Second(expression(ctx.source)) - case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => - Milliseconds(expression(ctx.source)) - case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => - Microseconds(expression(ctx.source)) - case "EPOCH" => - Epoch(expression(ctx.source)) - case other => - throw new ParseException(s"Literals of type '$other' are currently not supported.", ctx) - } + val fieldStr = ctx.field.getText + val source = expression(ctx.source) + val extractField = DatePart.parseExtractField(fieldStr, source, { + throw new ParseException(s"Literals of type '$fieldStr' are currently not supported.", ctx) + }) + new DatePart(Literal(fieldStr), expression(ctx.source), extractField) } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql new file mode 100644 index 0000000000000..cb3d966281009 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql @@ -0,0 +1,68 @@ +CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c; + +select date_part('millennium', c) from t; +select date_part('millennia', c) from t; +select date_part('mil', c) from t; +select date_part('mils', c) from t; + +select date_part('century', c) from t; +select date_part('centuries', c) from t; +select date_part('c', c) from t; +select date_part('cent', c) from t; + +select date_part('decade', c) from t; +select date_part('decades', c) from t; +select date_part('dec', c) from t; +select date_part('decs', c) from t; + +select date_part('year', c) from t; +select date_part('y', c) from t; +select date_part('years', c) from t; +select date_part('yr', c) from t; +select date_part('yrs', c) from t; + +select date_part('quarter', c) from t; +select date_part('qtr', c) from t; + +select date_part('month', c) from t; +select date_part('mon', c) from t; +select date_part('mons', c) from t; +select date_part('months', c) from t; + +select date_part('week', c) from t; +select date_part('w', c) from t; +select date_part('weeks', c) from t; + +select date_part('day', c) from t; +select date_part('d', c) from t; +select date_part('days', c) from t; + +select date_part('dayofweek', c) from t; + +select date_part('dow', c) from t; + +select date_part('isodow', c) from t; + +select date_part('doy', c) from t; + +select date_part('hour', c) from t; +select date_part('h', c) from t; +select date_part('hours', c) from t; +select date_part('hr', c) from t; +select date_part('hrs', c) from t; + +select date_part('minute', c) from t; +select date_part('m', c) from t; +select date_part('min', c) from t; +select date_part('mins', c) from t; +select date_part('minutes', c) from t; + +select date_part('second', c) from t; +select date_part('s', c) from t; +select date_part('sec', c) from t; +select date_part('seconds', c) from t; +select date_part('secs', c) from t; + +select date_part('not_supported', c) from t; + +select date_part(c, c) from t; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql index 2b974816766bd..65e8d3280e07c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql @@ -187,22 +187,21 @@ SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17 -- WHERE d1 BETWEEN timestamp '1902-01-01' -- AND timestamp '2038-01-01'; --- [SPARK-28420] Date/Time Functions: date_part --- SELECT '' AS "54", d1 as "timestamp", --- date_part( 'year', d1) AS year, date_part( 'month', d1) AS month, --- date_part( 'day', d1) AS day, date_part( 'hour', d1) AS hour, --- date_part( 'minute', d1) AS minute, date_part( 'second', d1) AS second --- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; - --- SELECT '' AS "54", d1 as "timestamp", --- date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, --- date_part( 'usec', d1) AS usec --- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; - --- SELECT '' AS "54", d1 as "timestamp", --- date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, --- date_part( 'dow', d1) AS dow --- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'year', d1) AS `year`, date_part( 'month', d1) AS `month`, + date_part( 'day', d1) AS `day`, date_part( 'hour', d1) AS `hour`, + date_part( 'minute', d1) AS `minute`, date_part( 'second', d1) AS `second` + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, + date_part( 'usec', d1) AS usec + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, + date_part( 'dow', d1) AS dow + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; -- [SPARK-28137] Data Type Formatting Functions -- TO_CHAR() diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out new file mode 100644 index 0000000000000..c59dfdbd3da34 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -0,0 +1,412 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 51 + + +-- !query 0 +CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select date_part('millennium', c) from t +-- !query 1 schema +struct +-- !query 1 output +3 + + +-- !query 2 +select date_part('millennia', c) from t +-- !query 2 schema +struct +-- !query 2 output +3 + + +-- !query 3 +select date_part('mil', c) from t +-- !query 3 schema +struct +-- !query 3 output +3 + + +-- !query 4 +select date_part('mils', c) from t +-- !query 4 schema +struct +-- !query 4 output +3 + + +-- !query 5 +select date_part('century', c) from t +-- !query 5 schema +struct +-- !query 5 output +21 + + +-- !query 6 +select date_part('centuries', c) from t +-- !query 6 schema +struct +-- !query 6 output +21 + + +-- !query 7 +select date_part('c', c) from t +-- !query 7 schema +struct +-- !query 7 output +21 + + +-- !query 8 +select date_part('cent', c) from t +-- !query 8 schema +struct +-- !query 8 output +21 + + +-- !query 9 +select date_part('decade', c) from t +-- !query 9 schema +struct +-- !query 9 output +201 + + +-- !query 10 +select date_part('decades', c) from t +-- !query 10 schema +struct +-- !query 10 output +201 + + +-- !query 11 +select date_part('dec', c) from t +-- !query 11 schema +struct +-- !query 11 output +201 + + +-- !query 12 +select date_part('decs', c) from t +-- !query 12 schema +struct +-- !query 12 output +201 + + +-- !query 13 +select date_part('year', c) from t +-- !query 13 schema +struct +-- !query 13 output +2011 + + +-- !query 14 +select date_part('y', c) from t +-- !query 14 schema +struct +-- !query 14 output +2011 + + +-- !query 15 +select date_part('years', c) from t +-- !query 15 schema +struct +-- !query 15 output +2011 + + +-- !query 16 +select date_part('yr', c) from t +-- !query 16 schema +struct +-- !query 16 output +2011 + + +-- !query 17 +select date_part('yrs', c) from t +-- !query 17 schema +struct +-- !query 17 output +2011 + + +-- !query 18 +select date_part('quarter', c) from t +-- !query 18 schema +struct +-- !query 18 output +2 + + +-- !query 19 +select date_part('qtr', c) from t +-- !query 19 schema +struct +-- !query 19 output +2 + + +-- !query 20 +select date_part('month', c) from t +-- !query 20 schema +struct +-- !query 20 output +5 + + +-- !query 21 +select date_part('mon', c) from t +-- !query 21 schema +struct +-- !query 21 output +5 + + +-- !query 22 +select date_part('mons', c) from t +-- !query 22 schema +struct +-- !query 22 output +5 + + +-- !query 23 +select date_part('months', c) from t +-- !query 23 schema +struct +-- !query 23 output +5 + + +-- !query 24 +select date_part('week', c) from t +-- !query 24 schema +struct +-- !query 24 output +18 + + +-- !query 25 +select date_part('w', c) from t +-- !query 25 schema +struct +-- !query 25 output +18 + + +-- !query 26 +select date_part('weeks', c) from t +-- !query 26 schema +struct +-- !query 26 output +18 + + +-- !query 27 +select date_part('day', c) from t +-- !query 27 schema +struct +-- !query 27 output +6 + + +-- !query 28 +select date_part('d', c) from t +-- !query 28 schema +struct +-- !query 28 output +6 + + +-- !query 29 +select date_part('days', c) from t +-- !query 29 schema +struct +-- !query 29 output +6 + + +-- !query 30 +select date_part('dayofweek', c) from t +-- !query 30 schema +struct +-- !query 30 output +6 + + +-- !query 31 +select date_part('dow', c) from t +-- !query 31 schema +struct +-- !query 31 output +5 + + +-- !query 32 +select date_part('isodow', c) from t +-- !query 32 schema +struct +-- !query 32 output +5 + + +-- !query 33 +select date_part('doy', c) from t +-- !query 33 schema +struct +-- !query 33 output +126 + + +-- !query 34 +select date_part('hour', c) from t +-- !query 34 schema +struct +-- !query 34 output +7 + + +-- !query 35 +select date_part('h', c) from t +-- !query 35 schema +struct +-- !query 35 output +7 + + +-- !query 36 +select date_part('hours', c) from t +-- !query 36 schema +struct +-- !query 36 output +7 + + +-- !query 37 +select date_part('hr', c) from t +-- !query 37 schema +struct +-- !query 37 output +7 + + +-- !query 38 +select date_part('hrs', c) from t +-- !query 38 schema +struct +-- !query 38 output +7 + + +-- !query 39 +select date_part('minute', c) from t +-- !query 39 schema +struct +-- !query 39 output +8 + + +-- !query 40 +select date_part('m', c) from t +-- !query 40 schema +struct +-- !query 40 output +8 + + +-- !query 41 +select date_part('min', c) from t +-- !query 41 schema +struct +-- !query 41 output +8 + + +-- !query 42 +select date_part('mins', c) from t +-- !query 42 schema +struct +-- !query 42 output +8 + + +-- !query 43 +select date_part('minutes', c) from t +-- !query 43 schema +struct +-- !query 43 output +8 + + +-- !query 44 +select date_part('second', c) from t +-- !query 44 schema +struct +-- !query 44 output +9 + + +-- !query 45 +select date_part('s', c) from t +-- !query 45 schema +struct +-- !query 45 output +9 + + +-- !query 46 +select date_part('sec', c) from t +-- !query 46 schema +struct +-- !query 46 output +9 + + +-- !query 47 +select date_part('seconds', c) from t +-- !query 47 schema +struct +-- !query 47 output +9 + + +-- !query 48 +select date_part('secs', c) from t +-- !query 48 schema +struct +-- !query 48 output +9 + + +-- !query 49 +select date_part('not_supported', c) from t +-- !query 49 schema +struct<> +-- !query 49 output +org.apache.spark.sql.AnalysisException +Literals of type 'not_supported' are currently not supported.;; line 1 pos 7 + + +-- !query 50 +select date_part(c, c) from t +-- !query 50 schema +struct<> +-- !query 50 output +org.apache.spark.sql.AnalysisException +The field parameter needs to be a foldable string value.;; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/extract.sql.out b/sql/core/src/test/resources/sql-tests/results/extract.sql.out index b02dfe054344b..e007fa8368ef5 100644 --- a/sql/core/src/test/resources/sql-tests/results/extract.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/extract.sql.out @@ -13,7 +13,7 @@ struct<> -- !query 1 select extract(millennium from c) from t -- !query 1 schema -struct +struct -- !query 1 output 3 @@ -21,7 +21,7 @@ struct -- !query 2 select extract(millennia from c) from t -- !query 2 schema -struct +struct -- !query 2 output 3 @@ -29,7 +29,7 @@ struct -- !query 3 select extract(mil from c) from t -- !query 3 schema -struct +struct -- !query 3 output 3 @@ -37,7 +37,7 @@ struct -- !query 4 select extract(mils from c) from t -- !query 4 schema -struct +struct -- !query 4 output 3 @@ -45,7 +45,7 @@ struct -- !query 5 select extract(century from c) from t -- !query 5 schema -struct +struct -- !query 5 output 21 @@ -53,7 +53,7 @@ struct -- !query 6 select extract(centuries from c) from t -- !query 6 schema -struct +struct -- !query 6 output 21 @@ -61,7 +61,7 @@ struct -- !query 7 select extract(c from c) from t -- !query 7 schema -struct +struct -- !query 7 output 21 @@ -69,7 +69,7 @@ struct -- !query 8 select extract(cent from c) from t -- !query 8 schema -struct +struct -- !query 8 output 21 @@ -77,7 +77,7 @@ struct -- !query 9 select extract(decade from c) from t -- !query 9 schema -struct +struct -- !query 9 output 201 @@ -85,7 +85,7 @@ struct -- !query 10 select extract(decades from c) from t -- !query 10 schema -struct +struct -- !query 10 output 201 @@ -93,7 +93,7 @@ struct -- !query 11 select extract(dec from c) from t -- !query 11 schema -struct +struct -- !query 11 output 201 @@ -101,7 +101,7 @@ struct -- !query 12 select extract(decs from c) from t -- !query 12 schema -struct +struct -- !query 12 output 201 @@ -109,7 +109,7 @@ struct -- !query 13 select extract(year from c) from t -- !query 13 schema -struct +struct -- !query 13 output 2011 @@ -117,7 +117,7 @@ struct -- !query 14 select extract(y from c) from t -- !query 14 schema -struct +struct -- !query 14 output 2011 @@ -125,7 +125,7 @@ struct -- !query 15 select extract(years from c) from t -- !query 15 schema -struct +struct -- !query 15 output 2011 @@ -133,7 +133,7 @@ struct -- !query 16 select extract(yr from c) from t -- !query 16 schema -struct +struct -- !query 16 output 2011 @@ -141,7 +141,7 @@ struct -- !query 17 select extract(yrs from c) from t -- !query 17 schema -struct +struct -- !query 17 output 2011 @@ -149,7 +149,7 @@ struct -- !query 18 select extract(isoyear from c) from t -- !query 18 schema -struct +struct -- !query 18 output 2011 @@ -157,7 +157,7 @@ struct -- !query 19 select extract(quarter from c) from t -- !query 19 schema -struct +struct -- !query 19 output 2 @@ -165,7 +165,7 @@ struct -- !query 20 select extract(qtr from c) from t -- !query 20 schema -struct +struct -- !query 20 output 2 @@ -173,7 +173,7 @@ struct -- !query 21 select extract(month from c) from t -- !query 21 schema -struct +struct -- !query 21 output 5 @@ -181,7 +181,7 @@ struct -- !query 22 select extract(mon from c) from t -- !query 22 schema -struct +struct -- !query 22 output 5 @@ -189,7 +189,7 @@ struct -- !query 23 select extract(mons from c) from t -- !query 23 schema -struct +struct -- !query 23 output 5 @@ -197,7 +197,7 @@ struct -- !query 24 select extract(months from c) from t -- !query 24 schema -struct +struct -- !query 24 output 5 @@ -205,7 +205,7 @@ struct -- !query 25 select extract(week from c) from t -- !query 25 schema -struct +struct -- !query 25 output 18 @@ -213,7 +213,7 @@ struct -- !query 26 select extract(w from c) from t -- !query 26 schema -struct +struct -- !query 26 output 18 @@ -221,7 +221,7 @@ struct -- !query 27 select extract(weeks from c) from t -- !query 27 schema -struct +struct -- !query 27 output 18 @@ -229,7 +229,7 @@ struct -- !query 28 select extract(day from c) from t -- !query 28 schema -struct +struct -- !query 28 output 6 @@ -237,7 +237,7 @@ struct -- !query 29 select extract(d from c) from t -- !query 29 schema -struct +struct -- !query 29 output 6 @@ -245,7 +245,7 @@ struct -- !query 30 select extract(days from c) from t -- !query 30 schema -struct +struct -- !query 30 output 6 @@ -253,7 +253,7 @@ struct -- !query 31 select extract(dayofweek from c) from t -- !query 31 schema -struct +struct -- !query 31 output 6 @@ -261,7 +261,7 @@ struct -- !query 32 select extract(dow from c) from t -- !query 32 schema -struct<(dayofweek(CAST(c AS DATE)) - 1):int> +struct -- !query 32 output 5 @@ -269,7 +269,7 @@ struct<(dayofweek(CAST(c AS DATE)) - 1):int> -- !query 33 select extract(isodow from c) from t -- !query 33 schema -struct<(weekday(CAST(c AS DATE)) + 1):int> +struct -- !query 33 output 5 @@ -277,7 +277,7 @@ struct<(weekday(CAST(c AS DATE)) + 1):int> -- !query 34 select extract(doy from c) from t -- !query 34 schema -struct +struct -- !query 34 output 126 @@ -285,7 +285,7 @@ struct -- !query 35 select extract(hour from c) from t -- !query 35 schema -struct +struct -- !query 35 output 7 @@ -293,7 +293,7 @@ struct -- !query 36 select extract(h from c) from t -- !query 36 schema -struct +struct -- !query 36 output 7 @@ -301,7 +301,7 @@ struct -- !query 37 select extract(hours from c) from t -- !query 37 schema -struct +struct -- !query 37 output 7 @@ -309,7 +309,7 @@ struct -- !query 38 select extract(hr from c) from t -- !query 38 schema -struct +struct -- !query 38 output 7 @@ -317,7 +317,7 @@ struct -- !query 39 select extract(hrs from c) from t -- !query 39 schema -struct +struct -- !query 39 output 7 @@ -325,7 +325,7 @@ struct -- !query 40 select extract(minute from c) from t -- !query 40 schema -struct +struct -- !query 40 output 8 @@ -333,7 +333,7 @@ struct -- !query 41 select extract(m from c) from t -- !query 41 schema -struct +struct -- !query 41 output 8 @@ -341,7 +341,7 @@ struct -- !query 42 select extract(min from c) from t -- !query 42 schema -struct +struct -- !query 42 output 8 @@ -349,7 +349,7 @@ struct -- !query 43 select extract(mins from c) from t -- !query 43 schema -struct +struct -- !query 43 output 8 @@ -357,7 +357,7 @@ struct -- !query 44 select extract(minutes from c) from t -- !query 44 schema -struct +struct -- !query 44 output 8 @@ -365,7 +365,7 @@ struct -- !query 45 select extract(second from c) from t -- !query 45 schema -struct +struct -- !query 45 output 9 @@ -373,7 +373,7 @@ struct -- !query 46 select extract(s from c) from t -- !query 46 schema -struct +struct -- !query 46 output 9 @@ -381,7 +381,7 @@ struct -- !query 47 select extract(sec from c) from t -- !query 47 schema -struct +struct -- !query 47 output 9 @@ -389,7 +389,7 @@ struct -- !query 48 select extract(seconds from c) from t -- !query 48 schema -struct +struct -- !query 48 output 9 @@ -397,7 +397,7 @@ struct -- !query 49 select extract(secs from c) from t -- !query 49 schema -struct +struct -- !query 49 output 9 @@ -405,7 +405,7 @@ struct -- !query 50 select extract(milliseconds from c) from t -- !query 50 schema -struct +struct -- !query 50 output 9123.456 @@ -413,7 +413,7 @@ struct -- !query 51 select extract(msec from c) from t -- !query 51 schema -struct +struct -- !query 51 output 9123.456 @@ -421,7 +421,7 @@ struct -- !query 52 select extract(msecs from c) from t -- !query 52 schema -struct +struct -- !query 52 output 9123.456 @@ -429,7 +429,7 @@ struct -- !query 53 select extract(millisecon from c) from t -- !query 53 schema -struct +struct -- !query 53 output 9123.456 @@ -437,7 +437,7 @@ struct -- !query 54 select extract(mseconds from c) from t -- !query 54 schema -struct +struct -- !query 54 output 9123.456 @@ -445,7 +445,7 @@ struct -- !query 55 select extract(ms from c) from t -- !query 55 schema -struct +struct -- !query 55 output 9123.456 @@ -453,7 +453,7 @@ struct -- !query 56 select extract(microseconds from c) from t -- !query 56 schema -struct +struct -- !query 56 output 9123456 @@ -461,7 +461,7 @@ struct -- !query 57 select extract(usec from c) from t -- !query 57 schema -struct +struct -- !query 57 output 9123456 @@ -469,7 +469,7 @@ struct -- !query 58 select extract(usecs from c) from t -- !query 58 schema -struct +struct -- !query 58 output 9123456 @@ -477,7 +477,7 @@ struct -- !query 59 select extract(useconds from c) from t -- !query 59 schema -struct +struct -- !query 59 output 9123456 @@ -485,7 +485,7 @@ struct -- !query 60 select extract(microsecon from c) from t -- !query 60 schema -struct +struct -- !query 60 output 9123456 @@ -493,7 +493,7 @@ struct -- !query 61 select extract(us from c) from t -- !query 61 schema -struct +struct -- !query 61 output 9123456 @@ -501,7 +501,7 @@ struct -- !query 62 select extract(epoch from c) from t -- !query 62 schema -struct +struct -- !query 62 output 1304665689.123456 @@ -513,7 +513,7 @@ struct<> -- !query 63 output org.apache.spark.sql.catalyst.parser.ParseException -Literals of type 'NOT_SUPPORTED' are currently not supported.(line 1, pos 7) +Literals of type 'not_supported' are currently not supported.(line 1, pos 7) == SQL == select extract(not_supported from c) from t diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index cb2be6d1cd22d..083832007d618 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -504,7 +504,7 @@ struct -- !query 47 SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') -- !query 47 schema -struct +struct -- !query 47 output 0 @@ -512,7 +512,7 @@ struct -- !query 48 SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 48 schema -struct +struct -- !query 48 output 0 @@ -520,7 +520,7 @@ struct -- !query 49 SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) -- !query 49 schema -struct +struct -- !query 49 output -2 @@ -528,7 +528,7 @@ struct -- !query 50 SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) -- !query 50 schema -struct +struct -- !query 50 output -1 @@ -536,7 +536,7 @@ struct -- !query 51 SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 51 schema -struct +struct -- !query 51 output -1 @@ -544,7 +544,7 @@ struct -- !query 52 SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') -- !query 52 schema -struct +struct -- !query 52 output 1 @@ -552,7 +552,7 @@ struct -- !query 53 SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') -- !query 53 schema -struct +struct -- !query 53 output 1 @@ -560,7 +560,7 @@ struct -- !query 54 SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') -- !query 54 schema -struct +struct -- !query 54 output 19 @@ -568,7 +568,7 @@ struct -- !query 55 SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') -- !query 55 schema -struct +struct -- !query 55 output 20 @@ -576,7 +576,7 @@ struct -- !query 56 SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') -- !query 56 schema -struct +struct -- !query 56 output 20 @@ -584,7 +584,7 @@ struct -- !query 57 SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') -- !query 57 schema -struct +struct -- !query 57 output 21 @@ -600,7 +600,7 @@ true -- !query 59 SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 59 schema -struct +struct -- !query 59 output -1 @@ -608,7 +608,7 @@ struct -- !query 60 SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') -- !query 60 schema -struct +struct -- !query 60 output 1 @@ -616,7 +616,7 @@ struct -- !query 61 SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') -- !query 61 schema -struct +struct -- !query 61 output 1 @@ -624,7 +624,7 @@ struct -- !query 62 SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') -- !query 62 schema -struct +struct -- !query 62 output 2 @@ -632,7 +632,7 @@ struct -- !query 63 SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') -- !query 63 schema -struct +struct -- !query 63 output 2 @@ -640,7 +640,7 @@ struct -- !query 64 SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') -- !query 64 schema -struct +struct -- !query 64 output 3 @@ -648,7 +648,7 @@ struct -- !query 65 SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) -- !query 65 schema -struct +struct -- !query 65 output 3 @@ -656,7 +656,7 @@ struct -- !query 66 SELECT EXTRACT(DECADE FROM DATE '1994-12-25') -- !query 66 schema -struct +struct -- !query 66 output 199 @@ -664,7 +664,7 @@ struct -- !query 67 SELECT EXTRACT(DECADE FROM DATE '0010-01-01') -- !query 67 schema -struct +struct -- !query 67 output 1 @@ -672,7 +672,7 @@ struct -- !query 68 SELECT EXTRACT(DECADE FROM DATE '0009-12-31') -- !query 68 schema -struct +struct -- !query 68 output 0 @@ -680,7 +680,7 @@ struct -- !query 69 SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) -- !query 69 schema -struct +struct -- !query 69 output 0 @@ -688,7 +688,7 @@ struct -- !query 70 SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) -- !query 70 schema -struct +struct -- !query 70 output -1 @@ -696,7 +696,7 @@ struct -- !query 71 SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) -- !query 71 schema -struct +struct -- !query 71 output -1 @@ -704,7 +704,7 @@ struct -- !query 72 SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) -- !query 72 schema -struct +struct -- !query 72 output -2 @@ -720,7 +720,7 @@ true -- !query 74 SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 74 schema -struct +struct -- !query 74 output 20 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out index 13a1d09b71b76..75d9ee8d9c797 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 15 +-- Number of queries: 18 -- !query 0 @@ -123,16 +123,59 @@ struct -- !query 13 -SELECT make_timestamp(2014,12,28,6,30,45.887) +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'year', d1) AS `year`, date_part( 'month', d1) AS `month`, + date_part( 'day', d1) AS `day`, date_part( 'hour', d1) AS `hour`, + date_part( 'minute', d1) AS `minute`, date_part( 'second', d1) AS `second` + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' -- !query 13 schema -struct +struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:int> -- !query 13 output -2014-12-28 06:30:45.887 + 1997-01-02 00:00:00 1997 1 2 0 0 0 + 1997-01-02 03:04:05 1997 1 2 3 4 5 + 1997-02-10 17:32:01 1997 2 10 17 32 1 + 2001-09-22 18:19:20 2001 9 22 18 19 20 -- !query 14 -DROP TABLE TIMESTAMP_TBL +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, + date_part( 'usec', d1) AS usec + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' -- !query 14 schema -struct<> +struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> -- !query 14 output + 1997-01-02 00:00:00 1 0 0 + 1997-01-02 03:04:05 1 5000 5000000 + 1997-02-10 17:32:01 1 1000 1000000 + 2001-09-22 18:19:20 3 20000 20000000 + + +-- !query 15 +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, + date_part( 'dow', d1) AS dow + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' +-- !query 15 schema +struct<54:string,timestamp:timestamp,isoyear:int,week:int,dow:int> +-- !query 15 output + 1997-01-02 00:00:00 1997 1 4 + 1997-01-02 03:04:05 1997 1 4 + 1997-02-10 17:32:01 1997 7 1 + 2001-09-22 18:19:20 2001 38 6 + + +-- !query 16 +SELECT make_timestamp(2014,12,28,6,30,45.887) +-- !query 16 schema +struct +-- !query 16 output +2014-12-28 06:30:45.887 + + +-- !query 17 +DROP TABLE TIMESTAMP_TBL +-- !query 17 schema +struct<> +-- !query 17 output From b2f06608b785f577999318c00f2c315f39d90889 Mon Sep 17 00:00:00 2001 From: maryannxue Date: Fri, 6 Sep 2019 12:46:54 -0700 Subject: [PATCH 0005/1387] [SPARK-29002][SQL] Avoid changing SMJ to BHJ if the build side has a high ratio of empty partitions ### What changes were proposed in this pull request? This PR aims to avoid AQE regressions by avoiding changing a sort merge join to a broadcast hash join when the expected build plan has a high ratio of empty partitions, in which case sort merge join can actually perform faster. This PR achieves this by adding an internal join hint in order to let the planner know which side has this high ratio of empty partitions and it should avoid planning it as a build plan of a BHJ. Still, it won't affect the other side if the other side qualifies for a build plan of a BHJ. ### Why are the changes needed? It is a performance improvement for AQE. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added UT. Closes #25703 from maryannxue/aqe-demote-bhj. Authored-by: maryannxue Signed-off-by: Xiao Li --- .../sql/catalyst/plans/logical/hints.scala | 8 +++ .../apache/spark/sql/internal/SQLConf.scala | 12 ++++ .../spark/sql/execution/SparkStrategies.scala | 4 +- .../adaptive/AdaptiveSparkPlanExec.scala | 4 +- .../adaptive/DemoteBroadcastHashJoin.scala | 60 +++++++++++++++++++ .../adaptive/AdaptiveQueryExecSuite.scala | 29 ++++++++- 6 files changed, 114 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala index 20c15947d16cc..f26e5662ee856 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala @@ -159,6 +159,14 @@ case object SHUFFLE_REPLICATE_NL extends JoinStrategyHint { "SHUFFLE_REPLICATE_NL") } +/** + * An internal hint to discourage broadcast hash join, used by adaptive query execution. + */ +case object NO_BROADCAST_HASH extends JoinStrategyHint { + override def displayName: String = "no_broadcast_hash" + override def hintAliases: Set[String] = Set.empty +} + /** * The callback for implementing customized strategies of handling hint errors. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 0e7391063ed13..fb6baf9003325 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -346,6 +346,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = + buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") + .doc("The relation with a non-empty partition ratio lower than this config will not be " + + "considered as the build side of a broadcast-hash join in adaptive execution regardless " + + "of its size.") + .doubleConf + .checkValue(_ >= 0, "The non-empty partition ratio must be positive number.") + .createWithDefault(0.2) + val REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED = buildConf("spark.sql.adaptive.reducePostShufflePartitions.enabled") .doc("When true and adaptive execution is enabled, this enables reducing the number of " + @@ -2093,6 +2102,9 @@ class SQLConf extends Serializable with Logging { def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) + def nonEmptyPartitionRatioForBroadcastJoin: Double = + getConf(NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN) + def reducePostShufflePartitionsEnabled: Boolean = getConf(REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED) def minNumPostShufflePartitions: Int = 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 08b00184ef9d7..85469bf2401d4 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 @@ -291,7 +291,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def createJoinWithoutHint() = { - createBroadcastHashJoin(canBroadcast(left), canBroadcast(right)) + createBroadcastHashJoin( + canBroadcast(left) && !hint.leftHint.exists(_.strategy.contains(NO_BROADCAST_HASH)), + canBroadcast(right) && !hint.rightHint.exists(_.strategy.contains(NO_BROADCAST_HASH))) .orElse { if (!conf.preferSortMergeJoin) { createShuffleHashJoin( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 5d92ddad887bf..524cacc11484c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -73,7 +73,9 @@ case class AdaptiveSparkPlanExec( // The logical plan optimizer for re-optimizing the current logical plan. @transient private val optimizer = new RuleExecutor[LogicalPlan] { // TODO add more optimization rules - override protected def batches: Seq[Batch] = Seq() + override protected def batches: Seq[Batch] = Seq( + Batch("Demote BroadcastHashJoin", Once, DemoteBroadcastHashJoin(conf)) + ) } @transient private val ensureRequirements = EnsureRequirements(conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala new file mode 100644 index 0000000000000..711b49e2d20a9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.adaptive + +import org.apache.spark.MapOutputStatistics +import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, Join, LogicalPlan, NO_BROADCAST_HASH} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf + +/** + * This optimization rule detects a join child that has a high ratio of empty partitions and + * adds a no-broadcast-hash-join hint to avoid it being broadcast. + */ +case class DemoteBroadcastHashJoin(conf: SQLConf) extends Rule[LogicalPlan] { + + private def shouldDemote(plan: LogicalPlan): Boolean = plan match { + case LogicalQueryStage(_, stage: QueryStageExec) + if stage.resultOption.isDefined && + stage.resultOption.get.isInstanceOf[MapOutputStatistics] => + val mapOutputStatistics = stage.resultOption.get.asInstanceOf[MapOutputStatistics] + val partitionCnt = mapOutputStatistics.bytesByPartitionId.length + val nonZeroCnt = mapOutputStatistics.bytesByPartitionId.count(_ > 0) + partitionCnt > 0 && nonZeroCnt > 0 && + (nonZeroCnt * 1.0 / partitionCnt) < conf.nonEmptyPartitionRatioForBroadcastJoin + case _ => false + } + + def apply(plan: LogicalPlan): LogicalPlan = plan.transformDown { + case j @ Join(left, right, _, _, hint) => + var newHint = hint + if (!hint.leftHint.exists(_.strategy.isDefined) && shouldDemote(left)) { + newHint = newHint.copy(leftHint = + Some(hint.leftHint.getOrElse(HintInfo()).copy(strategy = Some(NO_BROADCAST_HASH)))) + } + if (!hint.rightHint.exists(_.strategy.isDefined) && shouldDemote(right)) { + newHint = newHint.copy(rightHint = + Some(hint.rightHint.getOrElse(HintInfo()).copy(strategy = Some(NO_BROADCAST_HASH)))) + } + if (newHint.ne(hint)) { + j.copy(hint = newHint) + } else { + j + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 55e57a244c030..243ef5438ff39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.rule.CoalescedShuffleReaderExec +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.exchange.Exchange -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildRight, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -357,4 +358,30 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSparkSession { assert(smj2.size == 2, origPlan.toString) } } + + test("Avoid changing merge join to broadcast join if too many empty partitions on build plan") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5") { + // `testData` is small enough to be broadcast but has empty partition ratio over the config. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.isEmpty) + } + // It is still possible to broadcast `testData2`. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2000") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + assert(bhj.head.buildSide == BuildRight) + } + } + } } From ff5fa5873e045810f71079f0215b30f210ab5190 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sat, 7 Sep 2019 07:16:36 +0900 Subject: [PATCH 0006/1387] [SPARK-21870][SQL][FOLLOW-UP] Clean up string template formats for generated code in HashAggregateExec ### What changes were proposed in this pull request? This pr cleans up string template formats for generated code in HashAggregateExec. This changes comes from rednaxelafx comment: https://github.com/apache/spark/pull/20965#discussion_r316418729 ### Why are the changes needed? To improve code-readability. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #25714 from maropu/SPARK-21870-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../aggregate/HashAggregateExec.scala | 102 +++++++++--------- 1 file changed, 49 insertions(+), 53 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 9242583d36717..7ead180d869b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -234,30 +234,30 @@ case class HashAggregateExec( val doAgg = ctx.freshName("doAggregateWithoutKey") val doAggFuncName = ctx.addNewFunction(doAgg, s""" - | private void $doAgg() throws java.io.IOException { - | // initialize aggregation buffer - | $initBufVar + |private void $doAgg() throws java.io.IOException { + | // initialize aggregation buffer + | $initBufVar | - | ${child.asInstanceOf[CodegenSupport].produce(ctx, this)} - | } + | ${child.asInstanceOf[CodegenSupport].produce(ctx, this)} + |} """.stripMargin) val numOutput = metricTerm(ctx, "numOutputRows") val aggTime = metricTerm(ctx, "aggTime") val beforeAgg = ctx.freshName("beforeAgg") s""" - | while (!$initAgg) { - | $initAgg = true; - | long $beforeAgg = System.nanoTime(); - | $doAggFuncName(); - | $aggTime.add((System.nanoTime() - $beforeAgg) / $NANOS_PER_MILLIS); + |while (!$initAgg) { + | $initAgg = true; + | long $beforeAgg = System.nanoTime(); + | $doAggFuncName(); + | $aggTime.add((System.nanoTime() - $beforeAgg) / $NANOS_PER_MILLIS); | - | // output the result - | ${genResult.trim} + | // output the result + | ${genResult.trim} | - | $numOutput.add(1); - | ${consume(ctx, resultVars).trim} - | } + | $numOutput.add(1); + | ${consume(ctx, resultVars).trim} + |} """.stripMargin } @@ -581,12 +581,12 @@ case class HashAggregateExec( val evaluateNondeterministicResults = evaluateNondeterministicVariables(output, resultVars, resultExpressions) s""" - $evaluateKeyVars - $evaluateBufferVars - $evaluateAggResults - $evaluateNondeterministicResults - ${consume(ctx, resultVars)} - """ + |$evaluateKeyVars + |$evaluateBufferVars + |$evaluateAggResults + |$evaluateNondeterministicResults + |${consume(ctx, resultVars)} + """.stripMargin } else if (modes.contains(Partial) || modes.contains(PartialMerge)) { // resultExpressions are Attributes of groupingExpressions and aggregateBufferAttributes. assert(resultExpressions.forall(_.isInstanceOf[Attribute])) @@ -613,10 +613,10 @@ case class HashAggregateExec( resultExpressions, inputAttrs).map(_.genCode(ctx)) s""" - $evaluateKeyVars - $evaluateResultBufferVars - ${consume(ctx, resultVars)} - """ + |$evaluateKeyVars + |$evaluateResultBufferVars + |${consume(ctx, resultVars)} + """.stripMargin } else { // generate result based on grouping key ctx.INPUT_ROW = keyTerm @@ -627,18 +627,18 @@ case class HashAggregateExec( val evaluateNondeterministicResults = evaluateNondeterministicVariables(output, resultVars, resultExpressions) s""" - $evaluateNondeterministicResults - ${consume(ctx, resultVars)} - """ + |$evaluateNondeterministicResults + |${consume(ctx, resultVars)} + """.stripMargin } ctx.addNewFunction(funcName, s""" - private void $funcName(UnsafeRow $keyTerm, UnsafeRow $bufferTerm) - throws java.io.IOException { - $numOutput.add(1); - $body - } - """) + |private void $funcName(UnsafeRow $keyTerm, UnsafeRow $bufferTerm) + | throws java.io.IOException { + | $numOutput.add(1); + | $body + |} + """.stripMargin) } /** @@ -829,17 +829,16 @@ case class HashAggregateExec( val aggTime = metricTerm(ctx, "aggTime") val beforeAgg = ctx.freshName("beforeAgg") s""" - if (!$initAgg) { - $initAgg = true; - long $beforeAgg = System.nanoTime(); - $doAggFuncName(); - $aggTime.add((System.nanoTime() - $beforeAgg) / $NANOS_PER_MILLIS); - } - - // output the result - $outputFromFastHashMap - $outputFromRegularHashMap - """ + |if (!$initAgg) { + | $initAgg = true; + | long $beforeAgg = System.nanoTime(); + | $doAggFuncName(); + | $aggTime.add((System.nanoTime() - $beforeAgg) / $NANOS_PER_MILLIS); + |} + |// output the result + |$outputFromFastHashMap + |$outputFromRegularHashMap + """.stripMargin } private def doConsumeWithKeys(ctx: CodegenContext, input: Seq[ExprCode]): String = { @@ -1098,14 +1097,11 @@ case class HashAggregateExec( // continue to do in-memory aggregation and spilling until all the rows had been processed. // Finally, sort the spilled aggregate buffers by key, and merge them together for same key. s""" - $declareRowBuffer - - $findOrInsertHashMap - - $incCounter - - $updateRowInHashMap - """ + |$declareRowBuffer + |$findOrInsertHashMap + |$incCounter + |$updateRowInHashMap + """.stripMargin } override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) From 89aba69378cec141fd99dd8ac79adc9e07d90755 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 6 Sep 2019 15:56:50 -0700 Subject: [PATCH 0007/1387] [SPARK-28935][SQL][DOCS] Document SQL metrics for Details for Query Plan ### What changes were proposed in this pull request? This patch adds the description of common SQL metrics in web ui document. ### Why are the changes needed? The current web ui document describes query plan but does not describe the meaning SQL metrics. For end users, they might not understand the meaning of the metrics. ### Does this PR introduce any user-facing change? No. This is just documentation change. ### How was this patch tested? Built the docs locally. ![image](https://user-images.githubusercontent.com/11567269/64463485-1583d800-d0b9-11e9-9916-141f5c09f009.png) Closes #25658 from viirya/SPARK-28935. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- docs/web-ui.md | 35 +++++++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/docs/web-ui.md b/docs/web-ui.md index 9b22926016c77..72423d9468e83 100644 --- a/docs/web-ui.md +++ b/docs/web-ui.md @@ -363,6 +363,41 @@ number of written shuffle records, total data size, etc. Clicking the 'Details' link on the bottom displays the logical plans and the physical plan, which illustrate how Spark parses, analyzes, optimizes and performs the query. +### SQL metrics + +The metrics of SQL operators are shown in the block of physical operators. The SQL metrics can be useful +when we want to dive into the execution details of each operator. For example, "number of output rows" +can answer how many rows are output after a Filter operator, "shuffle bytes written total" in an Exchange +operator shows the number of bytes written by a shuffle. + +Here is the list of SQL metrics: + + + + + + + + + + + + + + + + + + + + + + + + + + +
SQL metricsMeaningOperators
number of output rows the number of output rows of the operator Aggregate operators, Join operators, Sample, Range, Scan operators, Filter, etc.
data size the size of broadcast/shuffled/collected data of the operator BroadcastExchange, ShuffleExchange, Subquery
time to collect the time spent on collecting data BroadcastExchange, Subquery
scan time the time spent on scanning data ColumnarBatchScan, FileSourceScan
metadata time the time spent on getting metadata like number of partitions, number of files FileSourceScan
shuffle bytes written the number of bytes written CollectLimit, TakeOrderedAndProject, ShuffleExchange
shuffle records written the number of records written CollectLimit, TakeOrderedAndProject, ShuffleExchange
shuffle write time the time spent on shuffle writing CollectLimit, TakeOrderedAndProject, ShuffleExchange
remote blocks read the number of blocks read remotely CollectLimit, TakeOrderedAndProject, ShuffleExchange
remote bytes read the number of bytes read remotely CollectLimit, TakeOrderedAndProject, ShuffleExchange
remote bytes read to disk the number of bytes read from remote to local disk CollectLimit, TakeOrderedAndProject, ShuffleExchange
local blocks read the number of blocks read locally CollectLimit, TakeOrderedAndProject, ShuffleExchange
local bytes read the number of bytes read locally CollectLimit, TakeOrderedAndProject, ShuffleExchange
fetch wait time the time spent on fetching data (local and remote) CollectLimit, TakeOrderedAndProject, ShuffleExchange
records read the number of read records CollectLimit, TakeOrderedAndProject, ShuffleExchange
sort time the time spent on sorting Sort
peak memory the peak memory usage in the operator Sort, HashAggregate
spill size number of bytes spilled to disk from memory in the operator Sort, HashAggregate
time in aggregation build the time spent on aggregation HashAggregate, ObjectHashAggregate
avg hash probe bucket list iters the average bucket list iterations per lookup during aggregation HashAggregate
data size of build side the size of built hash map ShuffledHashJoin
time to build hash map the time spent on building hash map ShuffledHashJoin
## Streaming Tab The web UI includes a Streaming tab if the application uses Spark streaming. This tab displays From 6fb5ef108e002035aa4b72c81d04e1219ed631d5 Mon Sep 17 00:00:00 2001 From: Nicholas Marion Date: Fri, 6 Sep 2019 17:48:53 -0700 Subject: [PATCH 0008/1387] [SPARK-29011][BUILD] Update netty-all from 4.1.30-Final to 4.1.39-Final ### What changes were proposed in this pull request? Upgrade netty-all to latest in the 4.1.x line which is 4.1.39-Final. ### Why are the changes needed? Currency of dependencies. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing unit-tests against master branch. Closes #25712 from n-marion/master. Authored-by: Nicholas Marion Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 2 +- dev/deps/spark-deps-hadoop-3.2 | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 775fb3c0a22e8..96cc76d0f2abb 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -150,7 +150,7 @@ metrics-graphite-3.1.5.jar metrics-json-3.1.5.jar metrics-jvm-3.1.5.jar minlog-1.3.0.jar -netty-all-4.1.30.Final.jar +netty-all-4.1.39.Final.jar objenesis-2.5.1.jar okapi-shade-0.4.2.jar okhttp-3.8.1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index de046634eefbb..a3a5b51226462 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -167,7 +167,7 @@ metrics-json-3.1.5.jar metrics-jvm-3.1.5.jar minlog-1.3.0.jar mssql-jdbc-6.2.1.jre7.jar -netty-all-4.1.30.Final.jar +netty-all-4.1.39.Final.jar nimbus-jose-jwt-4.41.1.jar objenesis-2.5.1.jar okapi-shade-0.4.2.jar diff --git a/pom.xml b/pom.xml index c1b53b8680c9b..6c474f5f7a3e7 100644 --- a/pom.xml +++ b/pom.xml @@ -658,7 +658,7 @@ io.netty netty-all - 4.1.30.Final + 4.1.39.Final org.apache.derby From 723faadf80da91a6e5514fc16b7af3ca4900eda8 Mon Sep 17 00:00:00 2001 From: avk Date: Fri, 6 Sep 2019 17:55:09 -0700 Subject: [PATCH 0009/1387] [SPARK-28912][STREAMING] Fixed MatchError in getCheckpointFiles() ### What changes were proposed in this pull request? This change fixes issue SPARK-28912. ### Why are the changes needed? If checkpoint directory is set to name which matches regex pattern used for checkpoint files then logs are flooded with MatchError exceptions and old checkpoint files are not removed. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually. 1. Start Hadoop in a pseudo-distributed mode. 2. In another terminal run command nc -lk 9999 3. In the Spark shell execute the following statements: ```scala val ssc = new StreamingContext(sc, Seconds(30)) ssc.checkpoint("hdfs://localhost:9000/checkpoint-01") val lines = ssc.socketTextStream("localhost", 9999) val words = lines.flatMap(_.split(" ")) val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) wordCounts.print() ssc.start() ssc.awaitTermination() ``` Closes #25654 from avkgh/SPARK-28912. Authored-by: avk Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/streaming/Checkpoint.scala | 4 ++-- .../spark/streaming/CheckpointSuite.scala | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 54f91ff1c69d5..c66ba2c170675 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -131,8 +131,8 @@ object Checkpoint extends Logging { try { val statuses = fs.listStatus(path) if (statuses != null) { - val paths = statuses.map(_.getPath) - val filtered = paths.filter(p => REGEX.findFirstIn(p.toString).nonEmpty) + val paths = statuses.filterNot(_.isDirectory).map(_.getPath) + val filtered = paths.filter(p => REGEX.findFirstIn(p.getName).nonEmpty) filtered.sortWith(sortFunc) } else { logWarning(s"Listing $path returned null") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 55fdd4c82ac75..ff5e3ffa35d79 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -847,6 +847,23 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester checkpointWriter.stop() } + test("SPARK-28912: Fix MatchError in getCheckpointFiles") { + withTempDir { tempDir => + val fs = FileSystem.get(tempDir.toURI, new Configuration) + val checkpointDir = tempDir.getAbsolutePath + "/checkpoint-01" + + assert(Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)).length === 0) + + // Ignore files whose parent path match. + fs.create(new Path(checkpointDir, "this-is-matched-before-due-to-parent-path")).close() + assert(Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)).length === 0) + + // Ignore directories whose names match. + fs.mkdirs(new Path(checkpointDir, "checkpoint-1000000000")) + assert(Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)).length === 0) + } + } + test("SPARK-6847: stack overflow when updateStateByKey is followed by a checkpointed dstream") { // In this test, there are two updateStateByKey operators. The RDD DAG is as follows: // From a75467432eab2ee38b93906d86416fe9e110e82e Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 8 Sep 2019 10:32:08 +0900 Subject: [PATCH 0010/1387] [SPARK-28000][SQL][TEST] Port comments.sql ## What changes were proposed in this pull request? This PR is to port comments.sql from PostgreSQL regression tests. https://github.com/postgres/postgres/blob/REL_12_BETA3/src/test/regress/sql/comments.sql The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA3/src/test/regress/expected/comments.out When porting the test cases, found one PostgreSQL specific features that do not exist in Spark SQL: [SPARK-28880](https://issues.apache.org/jira/browse/SPARK-28880): ANSI SQL: Bracketed comments ## How was this patch tested? N/A Closes #25588 from wangyum/SPARK-28000. Authored-by: Yuming Wang Signed-off-by: HyukjinKwon --- .../sql-tests/inputs/pgSQL/comments.sql | 48 +++++ .../sql-tests/results/pgSQL/comments.sql.out | 196 ++++++++++++++++++ 2 files changed, 244 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/comments.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/comments.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/comments.sql new file mode 100644 index 0000000000000..6725ce45e72a5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/comments.sql @@ -0,0 +1,48 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- COMMENTS +-- https://github.com/postgres/postgres/blob/REL_12_BETA3/src/test/regress/sql/comments.sql +-- + +SELECT 'trailing' AS first; -- trailing single line +SELECT /* embedded single line */ 'embedded' AS `second`; +SELECT /* both embedded and trailing single line */ 'both' AS third; -- trailing single line + +SELECT 'before multi-line' AS fourth; +-- [SPARK-28880] ANSI SQL: Bracketed comments +/* This is an example of SQL which should not execute: + * select 'multi-line'; + */ +SELECT 'after multi-line' AS fifth; + +-- [SPARK-28880] ANSI SQL: Bracketed comments +-- +-- Nested comments +-- + +/* +SELECT 'trailing' as x1; -- inside block comment +*/ + +/* This block comment surrounds a query which itself has a block comment... +SELECT /* embedded single line */ 'embedded' AS x2; +*/ + +SELECT -- continued after the following block comments... +/* Deeply nested comment. + This includes a single apostrophe to make sure we aren't decoding this part as a string. +SELECT 'deep nest' AS n1; +/* Second level of nesting... +SELECT 'deeper nest' as n2; +/* Third level of nesting... +SELECT 'deepest nest' as n3; +*/ +Hoo boy. Still two deep... +*/ +Now just one deep... +*/ +'deeply nested example' AS sixth; + +/* and this is the end of the file */ diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out new file mode 100644 index 0000000000000..fbd32fc909076 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out @@ -0,0 +1,196 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query 0 +SELECT 'trailing' AS first +-- !query 0 schema +struct +-- !query 0 output +trailing + + +-- !query 1 +SELECT /* embedded single line */ 'embedded' AS `second` +-- !query 1 schema +struct +-- !query 1 output +embedded + + +-- !query 2 +SELECT /* both embedded and trailing single line */ 'both' AS third +-- !query 2 schema +struct +-- !query 2 output +both + + +-- !query 3 +SELECT 'before multi-line' AS fourth +-- !query 3 schema +struct +-- !query 3 output +before multi-line + + +-- !query 4 +/* This is an example of SQL which should not execute: + * select 'multi-line' +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) + +== SQL == +/* This is an example of SQL which should not execute: +^^^ + * select 'multi-line' + + +-- !query 5 +*/ +SELECT 'after multi-line' AS fifth +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.catalyst.parser.ParseException + +extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) + +== SQL == +*/ +^^^ +SELECT 'after multi-line' AS fifth + + +-- !query 6 +/* +SELECT 'trailing' as x1 +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) + +== SQL == +/* +^^^ +SELECT 'trailing' as x1 + + +-- !query 7 +*/ + +/* This block comment surrounds a query which itself has a block comment... +SELECT /* embedded single line */ 'embedded' AS x2 +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) + +== SQL == +*/ +^^^ + +/* This block comment surrounds a query which itself has a block comment... +SELECT /* embedded single line */ 'embedded' AS x2 + + +-- !query 8 +*/ + +SELECT -- continued after the following block comments... +/* Deeply nested comment. + This includes a single apostrophe to make sure we aren't decoding this part as a string. +SELECT 'deep nest' AS n1 +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.catalyst.parser.ParseException + +extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) + +== SQL == +*/ +^^^ + +SELECT -- continued after the following block comments... +/* Deeply nested comment. + This includes a single apostrophe to make sure we aren't decoding this part as a string. +SELECT 'deep nest' AS n1 + + +-- !query 9 +/* Second level of nesting... +SELECT 'deeper nest' as n2 +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) + +== SQL == +/* Second level of nesting... +^^^ +SELECT 'deeper nest' as n2 + + +-- !query 10 +/* Third level of nesting... +SELECT 'deepest nest' as n3 +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) + +== SQL == +/* Third level of nesting... +^^^ +SELECT 'deepest nest' as n3 + + +-- !query 11 +*/ +Hoo boy. Still two deep... +*/ +Now just one deep... +*/ +'deeply nested example' AS sixth +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) + +== SQL == +*/ +^^^ +Hoo boy. Still two deep... +*/ +Now just one deep... +*/ +'deeply nested example' AS sixth + + +-- !query 12 +/* and this is the end of the file */ +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input '' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 37) + +== SQL == +/* and this is the end of the file */ +-------------------------------------^^^ From cb488ecf4165b7bb8c74a0716835ccc566614b6c Mon Sep 17 00:00:00 2001 From: shivusondur Date: Sun, 8 Sep 2019 19:45:19 -0500 Subject: [PATCH 0011/1387] [SPARK-28942][WEBUI] Spark in local mode hostname display localhost in the Host Column of Task Summary Page ### What changes were proposed in this pull request? In spark-shell local mode, in the task page, host name is coming as localhost This PR changes it to show machine IP, as shown in the "spark.driver.host" in the environment page ### Why are the changes needed? To show the proper IP in the task page host column ### Does this PR introduce any user-facing change? It updates the SPARK UI->Task page->Host Column ### How was this patch tested? verfied in spark UI ![image](https://user-images.githubusercontent.com/7912929/64079045-253d9e00-cd00-11e9-8092-26caec4e21dc.png) Closes #25645 from shivusondur/localhost1. Authored-by: shivusondur Signed-off-by: Sean Owen --- .../apache/spark/scheduler/local/LocalSchedulerBackend.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index cbcc5310a59f0..fa09c65c2ba0d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -29,6 +29,7 @@ import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.util.Utils private case class ReviveOffers() @@ -54,7 +55,7 @@ private[spark] class LocalEndpoint( private var freeCores = totalCores val localExecutorId = SparkContext.DRIVER_IDENTIFIER - val localExecutorHostname = "localhost" + val localExecutorHostname = Utils.localCanonicalHostName() private val executor = new Executor( localExecutorId, localExecutorHostname, SparkEnv.get, userClassPath, isLocal = true) From 1f056eb313de792a279c3c7c21b5bccd652c631b Mon Sep 17 00:00:00 2001 From: Kengo Seki Date: Sun, 8 Sep 2019 19:48:53 -0500 Subject: [PATCH 0012/1387] [SPARK-27420][DSTREAMS][KINESIS] KinesisInputDStream should expose a way to configure CloudWatch metrics ## What changes were proposed in this pull request? KinesisInputDStream currently does not provide a way to disable CloudWatch metrics push. Its default level is "DETAILED" which pushes 10s of metrics every 10 seconds. When dealing with multiple streaming jobs this add up pretty quickly, leading to thousands of dollars in cost. To address this problem, this PR adds interfaces for accessing KinesisClientLibConfiguration's `withMetrics` and `withMetricsEnabledDimensions` methods to KinesisInputDStream so that users can configure KCL's metrics levels and dimensions. ## How was this patch tested? By running updated unit tests in KinesisInputDStreamBuilderSuite. In addition, I ran a Streaming job with MetricsLevel.NONE and confirmed: * there's no data point for the "Operation", "Operation, ShardId" and "WorkerIdentifier" dimensions on the AWS management console * there's no DEBUG level message from Amazon KCL, such as "Successfully published xx datums." Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24651 from sekikn/SPARK-27420. Authored-by: Kengo Seki Signed-off-by: Sean Owen --- docs/streaming-kinesis-integration.md | 42 ++++++++++----- .../kinesis/KinesisInputDStream.scala | 54 +++++++++++++++++-- .../streaming/kinesis/KinesisReceiver.scala | 7 ++- .../streaming/kinesis/KinesisUtils.scala | 20 +++++-- .../KinesisInputDStreamBuilderSuite.scala | 17 +++++- 5 files changed, 115 insertions(+), 25 deletions(-) diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 55acec53302e4..26a83e52de2b1 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -64,13 +64,13 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m
- import org.apache.spark.storage.StorageLevel - import org.apache.spark.streaming.kinesis.KinesisInputDStream - import org.apache.spark.streaming.Seconds - import org.apache.spark.streaming.StreamingContext - import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + import org.apache.spark.storage.StorageLevel; + import org.apache.spark.streaming.kinesis.KinesisInputDStream; + import org.apache.spark.streaming.Seconds; + import org.apache.spark.streaming.StreamingContext; + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; - KinesisInputDStream kinesisStream = KinesisInputDStream.builder + KinesisInputDStream kinesisStream = KinesisInputDStream.builder() .streamingContext(streamingContext) .endpointUrl([endpoint URL]) .regionName([region name]) @@ -98,14 +98,21 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m
- You may also provide a "message handler function" that takes a Kinesis `Record` and returns a generic object `T`, in case you would like to use other data included in a `Record` such as partition key. This is currently only supported in Scala and Java. + You may also provide the following settings. These are currently only supported in Scala and Java. + + - A "message handler function" that takes a Kinesis `Record` and returns a generic object `T`, in case you would like to use other data included in a `Record` such as partition key. + + - CloudWatch metrics level and dimensions. See [the AWS documentation about monitoring KCL](https://docs.aws.amazon.com/streams/latest/dev/monitoring-with-kcl.html) for details.
+ import collection.JavaConverters._ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.kinesis.KinesisInputDStream import org.apache.spark.streaming.{Seconds, StreamingContext} import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration + import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel val kinesisStream = KinesisInputDStream.builder .streamingContext(streamingContext) @@ -116,17 +123,22 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m .checkpointAppName([Kinesis app name]) .checkpointInterval([checkpoint interval]) .storageLevel(StorageLevel.MEMORY_AND_DISK_2) + .metricsLevel(MetricsLevel.DETAILED) + .metricsEnabledDimensions(KinesisClientLibConfiguration.DEFAULT_METRICS_ENABLED_DIMENSIONS.asScala.toSet) .buildWithMessageHandler([message handler])
- import org.apache.spark.storage.StorageLevel - import org.apache.spark.streaming.kinesis.KinesisInputDStream - import org.apache.spark.streaming.Seconds - import org.apache.spark.streaming.StreamingContext - import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream - - KinesisInputDStream kinesisStream = KinesisInputDStream.builder + import org.apache.spark.storage.StorageLevel; + import org.apache.spark.streaming.kinesis.KinesisInputDStream; + import org.apache.spark.streaming.Seconds; + import org.apache.spark.streaming.StreamingContext; + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration; + import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; + import scala.collection.JavaConverters; + + KinesisInputDStream kinesisStream = KinesisInputDStream.builder() .streamingContext(streamingContext) .endpointUrl([endpoint URL]) .regionName([region name]) @@ -135,6 +147,8 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m .checkpointAppName([Kinesis app name]) .checkpointInterval([checkpoint interval]) .storageLevel(StorageLevel.MEMORY_AND_DISK_2) + .metricsLevel(MetricsLevel.DETAILED) + .metricsEnabledDimensions(JavaConverters.asScalaSetConverter(KinesisClientLibConfiguration.DEFAULT_METRICS_ENABLED_DIMENSIONS).asScala().toSet()) .buildWithMessageHandler([message handler]);
diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala index 608da0b8bf563..8c3931a1c87fd 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala @@ -19,7 +19,9 @@ package org.apache.spark.streaming.kinesis import scala.reflect.ClassTag -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import collection.JavaConverters._ +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration} +import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel import com.amazonaws.services.kinesis.model.Record import org.apache.spark.rdd.RDD @@ -43,7 +45,9 @@ private[kinesis] class KinesisInputDStream[T: ClassTag]( val messageHandler: Record => T, val kinesisCreds: SparkAWSCredentials, val dynamoDBCreds: Option[SparkAWSCredentials], - val cloudWatchCreds: Option[SparkAWSCredentials] + val cloudWatchCreds: Option[SparkAWSCredentials], + val metricsLevel: MetricsLevel, + val metricsEnabledDimensions: Set[String] ) extends ReceiverInputDStream[T](_ssc) { import KinesisReadConfigurations._ @@ -79,7 +83,8 @@ private[kinesis] class KinesisInputDStream[T: ClassTag]( override def getReceiver(): Receiver[T] = { new KinesisReceiver(streamName, endpointUrl, regionName, initialPosition, checkpointAppName, checkpointInterval, _storageLevel, messageHandler, - kinesisCreds, dynamoDBCreds, cloudWatchCreds) + kinesisCreds, dynamoDBCreds, cloudWatchCreds, + metricsLevel, metricsEnabledDimensions) } } @@ -104,6 +109,8 @@ object KinesisInputDStream { private var kinesisCredsProvider: Option[SparkAWSCredentials] = None private var dynamoDBCredsProvider: Option[SparkAWSCredentials] = None private var cloudWatchCredsProvider: Option[SparkAWSCredentials] = None + private var metricsLevel: Option[MetricsLevel] = None + private var metricsEnabledDimensions: Option[Set[String]] = None /** * Sets the StreamingContext that will be used to construct the Kinesis DStream. This is a @@ -237,6 +244,7 @@ object KinesisInputDStream { * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified. * * @param credentials [[SparkAWSCredentials]] to use for Kinesis authentication + * @return Reference to this [[KinesisInputDStream.Builder]] */ def kinesisCredentials(credentials: SparkAWSCredentials): Builder = { kinesisCredsProvider = Option(credentials) @@ -248,6 +256,7 @@ object KinesisInputDStream { * endpoint. Will use the same credentials used for AWS Kinesis if no custom value is set. * * @param credentials [[SparkAWSCredentials]] to use for DynamoDB authentication + * @return Reference to this [[KinesisInputDStream.Builder]] */ def dynamoDBCredentials(credentials: SparkAWSCredentials): Builder = { dynamoDBCredsProvider = Option(credentials) @@ -259,12 +268,43 @@ object KinesisInputDStream { * endpoint. Will use the same credentials used for AWS Kinesis if no custom value is set. * * @param credentials [[SparkAWSCredentials]] to use for CloudWatch authentication + * @return Reference to this [[KinesisInputDStream.Builder]] */ def cloudWatchCredentials(credentials: SparkAWSCredentials): Builder = { cloudWatchCredsProvider = Option(credentials) this } + /** + * Sets the CloudWatch metrics level. Defaults to + * [[KinesisClientLibConfiguration.DEFAULT_METRICS_LEVEL]] if no custom value is specified. + * + * @param metricsLevel [[MetricsLevel]] to specify the CloudWatch metrics level + * @return Reference to this [[KinesisInputDStream.Builder]] + * @see + * [[https://docs.aws.amazon.com/streams/latest/dev/monitoring-with-kcl.html#metric-levels]] + */ + def metricsLevel(metricsLevel: MetricsLevel): Builder = { + this.metricsLevel = Option(metricsLevel) + this + } + + /** + * Sets the enabled CloudWatch metrics dimensions. Defaults to + * [[KinesisClientLibConfiguration.DEFAULT_METRICS_ENABLED_DIMENSIONS]] + * if no custom value is specified. + * + * @param metricsEnabledDimensions Set[String] to specify which CloudWatch metrics dimensions + * should be enabled + * @return Reference to this [[KinesisInputDStream.Builder]] + * @see + * [[https://docs.aws.amazon.com/streams/latest/dev/monitoring-with-kcl.html#metric-levels]] + */ + def metricsEnabledDimensions(metricsEnabledDimensions: Set[String]): Builder = { + this.metricsEnabledDimensions = Option(metricsEnabledDimensions) + this + } + /** * Create a new instance of [[KinesisInputDStream]] with configured parameters and the provided * message handler. @@ -287,7 +327,9 @@ object KinesisInputDStream { ssc.sc.clean(handler), kinesisCredsProvider.getOrElse(DefaultCredentials), dynamoDBCredsProvider, - cloudWatchCredsProvider) + cloudWatchCredsProvider, + metricsLevel.getOrElse(DEFAULT_METRICS_LEVEL), + metricsEnabledDimensions.getOrElse(DEFAULT_METRICS_ENABLED_DIMENSIONS)) } /** @@ -324,4 +366,8 @@ object KinesisInputDStream { private[kinesis] val DEFAULT_KINESIS_REGION_NAME: String = "us-east-1" private[kinesis] val DEFAULT_INITIAL_POSITION: KinesisInitialPosition = new Latest() private[kinesis] val DEFAULT_STORAGE_LEVEL: StorageLevel = StorageLevel.MEMORY_AND_DISK_2 + private[kinesis] val DEFAULT_METRICS_LEVEL: MetricsLevel = + KinesisClientLibConfiguration.DEFAULT_METRICS_LEVEL + private[kinesis] val DEFAULT_METRICS_ENABLED_DIMENSIONS: Set[String] = + KinesisClientLibConfiguration.DEFAULT_METRICS_ENABLED_DIMENSIONS.asScala.toSet } diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 69c52365b1bf8..9ea7d4081928b 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -25,6 +25,7 @@ import scala.util.control.NonFatal import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor, IRecordProcessorCheckpointer, IRecordProcessorFactory} import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{KinesisClientLibConfiguration, Worker} +import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel import com.amazonaws.services.kinesis.model.Record import org.apache.spark.internal.Logging @@ -92,7 +93,9 @@ private[kinesis] class KinesisReceiver[T]( messageHandler: Record => T, kinesisCreds: SparkAWSCredentials, dynamoDBCreds: Option[SparkAWSCredentials], - cloudWatchCreds: Option[SparkAWSCredentials]) + cloudWatchCreds: Option[SparkAWSCredentials], + metricsLevel: MetricsLevel, + metricsEnabledDimensions: Set[String]) extends Receiver[T](storageLevel) with Logging { receiver => /* @@ -162,6 +165,8 @@ private[kinesis] class KinesisReceiver[T]( .withKinesisEndpoint(endpointUrl) .withTaskBackoffTimeMillis(500) .withRegionName(regionName) + .withMetricsLevel(metricsLevel) + .withMetricsEnabledDimensions(metricsEnabledDimensions.asJava) // Update the Kinesis client lib config with timestamp // if InitialPositionInStream.AT_TIMESTAMP is passed diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala index c60b9896a3473..4e796b2caec87 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala @@ -75,7 +75,9 @@ object KinesisUtils { new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), kinesisAppName, checkpointInterval, storageLevel, - cleanedHandler, DefaultCredentials, None, None) + cleanedHandler, DefaultCredentials, None, None, + KinesisInputDStream.DEFAULT_METRICS_LEVEL, + KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) } } @@ -132,7 +134,9 @@ object KinesisUtils { new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), kinesisAppName, checkpointInterval, storageLevel, - cleanedHandler, kinesisCredsProvider, None, None) + cleanedHandler, kinesisCredsProvider, None, None, + KinesisInputDStream.DEFAULT_METRICS_LEVEL, + KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) } } @@ -202,7 +206,9 @@ object KinesisUtils { new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), kinesisAppName, checkpointInterval, storageLevel, - cleanedHandler, kinesisCredsProvider, None, None) + cleanedHandler, kinesisCredsProvider, None, None, + KinesisInputDStream.DEFAULT_METRICS_LEVEL, + KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) } } @@ -248,7 +254,9 @@ object KinesisUtils { new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName), KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), kinesisAppName, checkpointInterval, storageLevel, - KinesisInputDStream.defaultMessageHandler, DefaultCredentials, None, None) + KinesisInputDStream.defaultMessageHandler, DefaultCredentials, None, None, + KinesisInputDStream.DEFAULT_METRICS_LEVEL, + KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) } } @@ -299,7 +307,9 @@ object KinesisUtils { new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName), KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), kinesisAppName, checkpointInterval, storageLevel, - KinesisInputDStream.defaultMessageHandler, kinesisCredsProvider, None, None) + KinesisInputDStream.defaultMessageHandler, kinesisCredsProvider, None, None, + KinesisInputDStream.DEFAULT_METRICS_LEVEL, + KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala index 361520e292266..25357cb52edea 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.streaming.kinesis import java.util.Calendar -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import collection.JavaConverters._ +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration} +import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel import org.scalatest.BeforeAndAfterEach import org.scalatest.mockito.MockitoSugar @@ -82,6 +84,8 @@ class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterE assert(dstream.kinesisCreds == DefaultCredentials) assert(dstream.dynamoDBCreds == None) assert(dstream.cloudWatchCreds == None) + assert(dstream.metricsLevel == DEFAULT_METRICS_LEVEL) + assert(dstream.metricsEnabledDimensions == DEFAULT_METRICS_ENABLED_DIMENSIONS) } test("should propagate custom non-auth values to KinesisInputDStream") { @@ -94,6 +98,9 @@ class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterE val customKinesisCreds = mock[SparkAWSCredentials] val customDynamoDBCreds = mock[SparkAWSCredentials] val customCloudWatchCreds = mock[SparkAWSCredentials] + val customMetricsLevel = MetricsLevel.NONE + val customMetricsEnabledDimensions = + KinesisClientLibConfiguration.METRICS_ALWAYS_ENABLED_DIMENSIONS.asScala.toSet val dstream = builder .endpointUrl(customEndpointUrl) @@ -105,6 +112,8 @@ class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterE .kinesisCredentials(customKinesisCreds) .dynamoDBCredentials(customDynamoDBCreds) .cloudWatchCredentials(customCloudWatchCreds) + .metricsLevel(customMetricsLevel) + .metricsEnabledDimensions(customMetricsEnabledDimensions) .build() assert(dstream.endpointUrl == customEndpointUrl) assert(dstream.regionName == customRegion) @@ -115,6 +124,8 @@ class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterE assert(dstream.kinesisCreds == customKinesisCreds) assert(dstream.dynamoDBCreds == Option(customDynamoDBCreds)) assert(dstream.cloudWatchCreds == Option(customCloudWatchCreds)) + assert(dstream.metricsLevel == customMetricsLevel) + assert(dstream.metricsEnabledDimensions == customMetricsEnabledDimensions) // Testing with AtTimestamp val cal = Calendar.getInstance() @@ -132,6 +143,8 @@ class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterE .kinesisCredentials(customKinesisCreds) .dynamoDBCredentials(customDynamoDBCreds) .cloudWatchCredentials(customCloudWatchCreds) + .metricsLevel(customMetricsLevel) + .metricsEnabledDimensions(customMetricsEnabledDimensions) .build() assert(dstreamAtTimestamp.endpointUrl == customEndpointUrl) assert(dstreamAtTimestamp.regionName == customRegion) @@ -145,6 +158,8 @@ class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterE assert(dstreamAtTimestamp.kinesisCreds == customKinesisCreds) assert(dstreamAtTimestamp.dynamoDBCreds == Option(customDynamoDBCreds)) assert(dstreamAtTimestamp.cloudWatchCreds == Option(customCloudWatchCreds)) + assert(dstreamAtTimestamp.metricsLevel == customMetricsLevel) + assert(dstreamAtTimestamp.metricsEnabledDimensions == customMetricsEnabledDimensions) } test("old Api should throw UnsupportedOperationExceptionexception with AT_TIMESTAMP") { From 0ed9fae45769d4b06b8cf8128f462f09ff3d9a72 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 8 Sep 2019 20:04:16 -0500 Subject: [PATCH 0013/1387] [SPARK-28886][K8S] Fix the DepsTestsSuite with minikube 1.3.1 ### What changes were proposed in this pull request? Matches the response from minikube service against a regex to extract the URL ### Why are the changes needed? minikube 1.3.1 on OSX has different formatting than expected ### Does this PR introduce any user-facing change? No ### How was this patch tested? Ran the existing integration test run on OSX with minikube 1.3.1 Closes #25599 from holdenk/SPARK-28886-fix-deps-tests-with-minikube-1.3.1. Authored-by: Holden Karau Signed-off-by: Sean Owen --- .../deploy/k8s/integrationtest/DepsTestsSuite.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala index b6f3484a062c8..7181774b9f17e 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala @@ -224,9 +224,20 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => } private def getServiceUrl(serviceName: String): String = { + val fuzzyUrlMatcher = """^(.*?)([a-zA-Z]+://.*?)(\s*)$""".r Eventually.eventually(TIMEOUT, INTERVAL) { // ns is always available either random or provided by the user - Minikube.minikubeServiceAction(serviceName, "-n", kubernetesTestComponents.namespace, "--url") + val rawUrl = Minikube.minikubeServiceAction( + serviceName, "-n", kubernetesTestComponents.namespace, "--url") + val url = rawUrl match { + case fuzzyUrlMatcher(junk, url, extra) => + logDebug(s"Service url matched junk ${junk} - url ${url} - extra ${extra}") + url + case _ => + logWarning(s"Response from minikube ${rawUrl} did not match URL regex") + rawUrl + } + url } } } From c4115793555d35a10c79038ce51dc5f803f3b838 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Mon, 9 Sep 2019 13:30:56 +0800 Subject: [PATCH 0014/1387] [SPARK-28916][SQL] Split subexpression elimination functions code for Generate[Mutable|Unsafe]Projection ### What changes were proposed in this pull request? The PR proposes to split the code for subexpression elimination before inlining the function calls all in the apply method for `Generate[Mutable|Unsafe]Projection`. ### Why are the changes needed? Before this PR, code generation can fail due to the 64KB code size limit if a lot of subexpression elimination functions are generated. The added UT is a reproducer for the issue (thanks to the JIRA reporter and HyukjinKwon for it). ### Does this PR introduce any user-facing change? No. ### How was this patch tested? added UT Closes #25642 from mgaido91/SPARK-28916. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan --- .../expressions/codegen/CodeGenerator.scala | 16 +++++++++++++--- .../codegen/GenerateMutableProjection.scala | 2 +- .../codegen/GenerateUnsafeProjection.scala | 2 +- .../expressions/CodeGenerationSuite.scala | 12 ++++++++++++ 4 files changed, 27 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 4c1bfcfdf7f17..660a1dbaf0aa1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -403,13 +403,14 @@ class CodegenContext { * equivalentExpressions will match the tree containing `col1 + col2` and it will only * be evaluated once. */ - val equivalentExpressions: EquivalentExpressions = new EquivalentExpressions + private val equivalentExpressions: EquivalentExpressions = new EquivalentExpressions // Foreach expression that is participating in subexpression elimination, the state to use. - var subExprEliminationExprs = Map.empty[Expression, SubExprEliminationState] + // Visible for testing. + private[expressions] var subExprEliminationExprs = Map.empty[Expression, SubExprEliminationState] // The collection of sub-expression result resetting methods that need to be called on each row. - val subexprFunctions = mutable.ArrayBuffer.empty[String] + private val subexprFunctions = mutable.ArrayBuffer.empty[String] val outerClassName = "OuterClass" @@ -993,6 +994,15 @@ class CodegenContext { } } + /** + * Returns the code for subexpression elimination after splitting it if necessary. + */ + def subexprFunctionsCode: String = { + // Whole-stage codegen's subexpression elimination is handled in another code path + assert(currentVars == null || subexprFunctions.isEmpty) + splitExpressions(subexprFunctions, "subexprFunc_split", Seq("InternalRow" -> INPUT_ROW)) + } + /** * Perform a function which generates a sequence of ExprCodes with a given mapping between * expressions and common expressions, instead of using the mapping in current context. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 838bd1c679e4d..2e018de07101e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -92,7 +92,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP } // Evaluate all the subexpressions. - val evalSubexpr = ctx.subexprFunctions.mkString("\n") + val evalSubexpr = ctx.subexprFunctionsCode val allProjections = ctx.splitExpressionsWithCurrentInputs(projectionCodes.map(_._1)) val allUpdates = ctx.splitExpressionsWithCurrentInputs(projectionCodes.map(_._2)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index fb1d8a3c8e739..8da7f65bdeee6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -299,7 +299,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro v => s"$v = new $rowWriterClass(${expressions.length}, ${numVarLenFields * 32});") // Evaluate all the subexpression. - val evalSubexpr = ctx.subexprFunctions.mkString("\n") + val evalSubexpr = ctx.subexprFunctionsCode val writeExpressions = writeExpressionsToBuffer( ctx, ctx.INPUT_ROW, exprEvals, exprSchemas, rowWriter, isTopLevel = true) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 4e64313da136b..28d2607e6e43e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -545,6 +545,18 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } assert(appender.seenMessage) } + + test("SPARK-28916: subexrepssion elimination can cause 64kb code limit on UnsafeProjection") { + val numOfExprs = 10000 + val exprs = (0 to numOfExprs).flatMap(colIndex => + Seq(Add(BoundReference(colIndex, DoubleType, true), + BoundReference(numOfExprs + colIndex, DoubleType, true)), + Add(BoundReference(colIndex, DoubleType, true), + BoundReference(numOfExprs + colIndex, DoubleType, true)))) + // these should not fail to compile due to 64K limit + GenerateUnsafeProjection.generate(exprs, true) + GenerateMutableProjection.generate(exprs, true) + } } case class HugeCodeIntExpression(value: Int) extends Expression { From d4eca7c99dd2c079042bba2415ab1775c3bb2361 Mon Sep 17 00:00:00 2001 From: turbofei Date: Mon, 9 Sep 2019 13:50:17 +0800 Subject: [PATCH 0015/1387] [SPARK-29000][SQL] Decimal precision overflow when don't allow precision loss ### What changes were proposed in this pull request? When we set spark.sql.decimalOperations.allowPrecisionLoss to false. For the sql below, the result will overflow and return null. Case a: `select case when 1=2 then 1 else 1.000000000000000000000001 end * 1` Similar with the division operation. This sql below will lost precision. Case b: `select case when 1=2 then 1 else 1.000000000000000000000001 end / 1` Let us check the code of TypeCoercion.scala. https://github.com/apache/spark/blob/a75467432eab2ee38b93906d86416fe9e110e82e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala#L864-L875. For binaryOperator, if the two operands have differnt datatype, rule ImplicitTypeCasts will find a common type and cast both operands to common type. So, for these cases menthioned, their left operand is Decimal(34, 24) and right operand is Literal. Their common type is Decimal(34,24), and Literal(1) will be casted to Decimal(34,24). Then both operands are decimal type and they will be processed by decimalAndDecimal method of DecimalPrecision class. Let's check the relative code. https://github.com/apache/spark/blob/a75467432eab2ee38b93906d86416fe9e110e82e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala#L123-L153 When we don't allow precision loss, the result type of multiply operation in case a is Decimal(38, 38), and that of division operation in case b is Decimal(38, 20). Then the multi operation in case a will overflow and division operation in case b will lost precision. In this PR, we skip to handle the binaryOperator if DecimalType operands are involved and rule `DecimalPrecision` will handle it. ### Why are the changes needed? Data will corrupt without this change. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test. Closes #25701 from turboFei/SPARK-29000. Authored-by: turbofei Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/analysis/TypeCoercion.scala | 4 +++- .../sql/catalyst/analysis/TypeCoercionSuite.scala | 14 ++++++++++++++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 12 ++++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 3125f8cb732db..8ea6e1b0f1808 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -861,7 +861,9 @@ object TypeCoercion { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e - case b @ BinaryOperator(left, right) if left.dataType != right.dataType => + // If DecimalType operands are involved, DecimalPrecision will handle it + case b @ BinaryOperator(left, right) if !left.dataType.isInstanceOf[DecimalType] && + !right.dataType.isInstanceOf[DecimalType] && left.dataType != right.dataType => findTightestCommonType(left.dataType, right.dataType).map { commonType => if (b.inputType.acceptsType(commonType)) { // If the expression accepts the tightest common type, cast to that. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 949bb30d15503..50c38145ae21d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1576,6 +1576,20 @@ class TypeCoercionSuite extends AnalysisTest { SpecifiedWindowFrame(RangeFrame, CurrentRow, UnboundedFollowing)) ) } + + test("SPARK-29000: skip to handle decimals in ImplicitTypeCasts") { + ruleTest(TypeCoercion.ImplicitTypeCasts, + Multiply(CaseWhen(Seq((EqualTo(1, 2), Cast(1, DecimalType(34, 24)))), + Cast(100, DecimalType(34, 24))), Literal(1)), + Multiply(CaseWhen(Seq((EqualTo(1, 2), Cast(1, DecimalType(34, 24)))), + Cast(100, DecimalType(34, 24))), Literal(1))) + + ruleTest(TypeCoercion.ImplicitTypeCasts, + Multiply(CaseWhen(Seq((EqualTo(1, 2), Cast(1, DecimalType(34, 24)))), + Cast(100, DecimalType(34, 24))), Cast(1, IntegerType)), + Multiply(CaseWhen(Seq((EqualTo(1, 2), Cast(1, DecimalType(34, 24)))), + Cast(100, DecimalType(34, 24))), Cast(1, IntegerType))) + } } 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 b8b157e275b61..80c1e24bfa568 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 @@ -3180,6 +3180,18 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { } } + + test("SPARK-29000: arithmetic computation overflow when don't allow decimal precision loss ") { + withSQLConf(SQLConf.DECIMAL_OPERATIONS_ALLOW_PREC_LOSS.key -> "false") { + val df1 = sql("select case when 1=2 then 1 else 100.000000000000000000000000 end * 1") + checkAnswer(df1, Array(Row(100))) + val df2 = sql("select case when 1=2 then 1 else 100.000000000000000000000000 end * " + + "case when 1=2 then 2 else 1 end") + checkAnswer(df2, Array(Row(100))) + val df3 = sql("select case when 1=2 then 1 else 1.000000000000000000000001 end / 10") + checkAnswer(df3, Array(Row(new java.math.BigDecimal("0.100000000000000000000000100")))) + } + } } case class Foo(bar: Option[String]) From 4a3a6b66be999f5941d98e998cfc9b26d4fc5d86 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 8 Sep 2019 23:20:27 -0700 Subject: [PATCH 0016/1387] [SPARK-28637][SQL] Thriftserver support interval type ## What changes were proposed in this pull request? `bin/spark-shell` support query interval value: ```scala scala> spark.sql("SELECT interval 3 months 1 hours AS i").show(false) +-------------------------+ |i | +-------------------------+ |interval 3 months 1 hours| +-------------------------+ ``` But `sbin/start-thriftserver.sh` can't support query interval value: ```sql 0: jdbc:hive2://localhost:10000/default> SELECT interval 3 months 1 hours AS i; Error: java.lang.IllegalArgumentException: Unrecognized type name: interval (state=,code=0) ``` This PR maps `CalendarIntervalType` to `StringType` for `TableSchema` to make Thriftserver support query interval value because we do not support `INTERVAL_YEAR_MONTH` type and `INTERVAL_DAY_TIME`: https://github.com/apache/spark/blob/02c33694c8254f69cb36c71c0876194dccdbc014/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Type.java#L73-L78 [SPARK-27791](https://issues.apache.org/jira/browse/SPARK-27791): Support SQL year-month INTERVAL type [SPARK-27793](https://issues.apache.org/jira/browse/SPARK-27793): Support SQL day-time INTERVAL type ## How was this patch tested? unit tests Closes #25277 from wangyum/Thriftserver-support-interval-type. Authored-by: Yuming Wang Signed-off-by: Xiao Li --- .../SparkExecuteStatementOperation.scala | 9 ++++++++- .../thriftserver/HiveThriftServer2Suites.scala | 15 +++++++++++++++ .../SparkThriftServerProtocolVersionsSuite.scala | 4 ++-- 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 69e85484ccf8e..f246f43435c75 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.execution.HiveResult import org.apache.spark.sql.execution.command.SetCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.{Utils => SparkUtils} private[hive] class SparkExecuteStatementOperation( @@ -103,6 +104,8 @@ private[hive] class SparkExecuteStatementOperation( to += from.getAs[Timestamp](ordinal) case BinaryType => to += from.getAs[Array[Byte]](ordinal) + case CalendarIntervalType => + to += HiveResult.toHiveString((from.getAs[CalendarInterval](ordinal), CalendarIntervalType)) case _: ArrayType | _: StructType | _: MapType | _: UserDefinedType[_] => val hiveString = HiveResult.toHiveString((from.get(ordinal), dataTypes(ordinal))) to += hiveString @@ -331,7 +334,11 @@ private[hive] class SparkExecuteStatementOperation( object SparkExecuteStatementOperation { def getTableSchema(structType: StructType): TableSchema = { val schema = structType.map { field => - val attrTypeString = if (field.dataType == NullType) "void" else field.dataType.catalogString + val attrTypeString = field.dataType match { + case NullType => "void" + case CalendarIntervalType => StringType.catalogString + case other => other.catalogString + } new FieldSchema(field.name, attrTypeString, field.getComment.getOrElse("")) } new TableSchema(schema.asJava) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index b7185db2f2ae7..e504ecd9fefd5 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -662,6 +662,21 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { assert(rs.getBigDecimal(1) === new java.math.BigDecimal("1.000000000000000000")) } } + + test("Support interval type") { + withJdbcStatement() { statement => + val rs = statement.executeQuery("SELECT interval 3 months 1 hours") + assert(rs.next()) + assert(rs.getString(1) === "interval 3 months 1 hours") + } + // Invalid interval value + withJdbcStatement() { statement => + val e = intercept[SQLException] { + statement.executeQuery("SELECT interval 3 months 1 hou") + } + assert(e.getMessage.contains("org.apache.spark.sql.catalyst.parser.ParseException")) + } + } } class SingleSessionSuite extends HiveThriftJdbcTest { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala index f198372a4c998..10ec1ee168303 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -261,10 +261,10 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftJdbcTest { } } - // We do not fully support interval type - ignore(s"$version get interval type") { + test(s"$version get interval type") { testExecuteStatementWithProtocolVersion(version, "SELECT interval '1' year '2' day") { rs => assert(rs.next()) + assert(rs.getString(1) === "interval 1 years 2 days") } } From dadb72028ad4124c14296ece0fa7b2ba2eb532e9 Mon Sep 17 00:00:00 2001 From: colinma Date: Mon, 9 Sep 2019 05:27:53 -0500 Subject: [PATCH 0017/1387] =?UTF-8?q?[SPARK-28340][CORE]=20Noisy=20excepti?= =?UTF-8?q?ons=20when=20tasks=20are=20killed:=20"DiskBloc=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? If a Spark task is killed due to intentional job kills, automated killing of redundant speculative tasks, etc, ClosedByInterruptException occurs if task has unfinished I/O operation with AbstractInterruptibleChannel. A single cancelled task can result in hundreds of stack trace of ClosedByInterruptException being logged. In this PR, stack trace of ClosedByInterruptException won't be logged like Executor.run do for InterruptedException. ### Why are the changes needed? Large numbers of spurious exceptions is confusing to users when they are inspecting Spark logs to diagnose other issues. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #25674 from colinmjj/spark-28340. Authored-by: colinma Signed-off-by: Sean Owen --- .../spark/storage/DiskBlockObjectWriter.scala | 8 +++++++- .../storage/ShuffleBlockFetcherIterator.scala | 19 ++++++++++++++++--- 2 files changed, 23 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index 758621c52495b..1f290356cd2ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -18,7 +18,7 @@ package org.apache.spark.storage import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStream} -import java.nio.channels.FileChannel +import java.nio.channels.{ClosedByInterruptException, FileChannel} import org.apache.spark.internal.Logging import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} @@ -219,6 +219,12 @@ private[spark] class DiskBlockObjectWriter( truncateStream = new FileOutputStream(file, true) truncateStream.getChannel.truncate(committedPosition) } catch { + // ClosedByInterruptException is an excepted exception when kill task, + // don't log the exception stack trace to avoid confusing users. + // See: SPARK-28340 + case ce: ClosedByInterruptException => + logError("Exception occurred while reverting partial writes to file " + + file + ", " + ce.getMessage) case e: Exception => logError("Uncaught exception while reverting partial writes to file " + file, e) } finally { diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index a5b7ee5762c49..8a6c199423506 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.io.{InputStream, IOException} +import java.nio.channels.ClosedByInterruptException import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} import javax.annotation.concurrent.GuardedBy @@ -349,9 +350,16 @@ final class ShuffleBlockFetcherIterator( results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId, buf.size(), buf, false)) } catch { + // If we see an exception, stop immediately. case e: Exception => - // If we see an exception, stop immediately. - logError(s"Error occurred while fetching local blocks", e) + e match { + // ClosedByInterruptException is an excepted exception when kill task, + // don't log the exception stack trace to avoid confusing users. + // See: SPARK-28340 + case ce: ClosedByInterruptException => + logError("Error occurred while fetching local blocks, " + ce.getMessage) + case ex: Exception => logError("Error occurred while fetching local blocks", ex) + } results.put(new FailureFetchResult(blockId, blockManager.blockManagerId, e)) return } @@ -454,7 +462,12 @@ final class ShuffleBlockFetcherIterator( // The exception could only be throwed by local shuffle block case e: IOException => assert(buf.isInstanceOf[FileSegmentManagedBuffer]) - logError("Failed to create input stream from local block", e) + e match { + case ce: ClosedByInterruptException => + logError("Failed to create input stream from local block, " + + ce.getMessage) + case e: IOException => logError("Failed to create input stream from local block", e) + } buf.release() throwFetchFailedException(blockId, address, e) } From abec6d77635279fd696e887f36bf7e132da46839 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 9 Sep 2019 21:14:37 +0800 Subject: [PATCH 0018/1387] [SPARK-28341][SQL] create a public API for V2SessionCatalog ## What changes were proposed in this pull request? The `V2SessionCatalog` has 2 functionalities: 1. work as an adapter: provide v2 APIs and translate calls to the `SessionCatalog`. 2. allow users to extend it, so that they can add hooks to apply custom logic before calling methods of the builtin catalog (session catalog). To leverage the second functionality, users must extend `V2SessionCatalog` which is an internal class. There is no doc to explain this usage. This PR does 2 things: 1. refine the document of the config `spark.sql.catalog.session`. 2. add a public abstract class `CatalogExtension` for users to write implementations. TODOs for followup PRs: 1. discuss if we should allow users to completely overwrite the v2 session catalog with a new one. 2. discuss to change the name of session catalog, so that it's less likely to conflict with existing namespace names. ## How was this patch tested? existing tests Closes #25104 from cloud-fan/session-catalog. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalog/v2/CatalogExtension.java | 38 +++++++ .../v2/DelegatingCatalogExtension.java | 101 ++++++++++++++++++ .../spark/sql/catalog/v2/CatalogManager.scala | 36 +++++-- .../spark/sql/catalog/v2/LookupCatalog.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 36 ++++++- .../apache/spark/sql/internal/SQLConf.scala | 7 +- .../catalog/CatalogManagerSuite.scala | 7 +- .../apache/spark/sql/DataFrameWriter.scala | 14 ++- .../datasources/DataSourceResolution.scala | 19 +--- .../datasources/v2/V2SessionCatalog.scala | 28 ++--- .../internal/BaseSessionStateBuilder.scala | 6 +- .../command/PlanResolutionSuite.scala | 8 +- .../v2/V2SessionCatalogSuite.scala | 13 +-- ...SourceV2DataFrameSessionCatalogSuite.scala | 9 +- .../DataSourceV2SQLSessionCatalogSuite.scala | 2 +- .../sql/sources/v2/DataSourceV2SQLSuite.scala | 7 +- .../v2/utils/TestV2SessionCatalogBase.scala | 5 +- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- 18 files changed, 246 insertions(+), 94 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java new file mode 100644 index 0000000000000..2bf72c1da73de --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java @@ -0,0 +1,38 @@ +/* + * 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.catalog.v2; + +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * An API to extend the Spark built-in session catalog. Implementation can get the built-in session + * catalog from {@link #setDelegateCatalog(TableCatalog)}, implement catalog functions with + * some custom logic and call the built-in session catalog at the end. For example, they can + * implement {@code createTable}, do something else before calling {@code createTable} of the + * built-in session catalog. + */ +@Experimental +public interface CatalogExtension extends TableCatalog { + + /** + * This will be called only once by Spark to pass in the Spark built-in session catalog, after + * {@link #initialize(String, CaseInsensitiveStringMap)} is called. + */ + void setDelegateCatalog(TableCatalog delegate); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java new file mode 100644 index 0000000000000..2d3700a08806f --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java @@ -0,0 +1,101 @@ +/* + * 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.catalog.v2; + +import java.util.Map; + +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * A simple implementation of {@link CatalogExtension}, which implements all the catalog functions + * by calling the built-in session catalog directly. This is created for convenience, so that users + * only need to override some methods where they want to apply custom logic. For example, they can + * override {@code createTable}, do something else before calling {@code super.createTable}. + */ +@Experimental +public abstract class DelegatingCatalogExtension implements CatalogExtension { + + private TableCatalog delegate; + + public final void setDelegateCatalog(TableCatalog delegate) { + this.delegate = delegate; + } + + @Override + public String name() { + return delegate.name(); + } + + @Override + public final void initialize(String name, CaseInsensitiveStringMap options) {} + + @Override + public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException { + return delegate.listTables(namespace); + } + + @Override + public Table loadTable(Identifier ident) throws NoSuchTableException { + return delegate.loadTable(ident); + } + + @Override + public void invalidateTable(Identifier ident) { + delegate.invalidateTable(ident); + } + + @Override + public boolean tableExists(Identifier ident) { + return delegate.tableExists(ident); + } + + @Override + public Table createTable( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { + return delegate.createTable(ident, schema, partitions, properties); + } + + @Override + public Table alterTable( + Identifier ident, + TableChange... changes) throws NoSuchTableException { + return delegate.alterTable(ident, changes); + } + + @Override + public boolean dropTable(Identifier ident) { + return delegate.dropTable(ident); + } + + @Override + public void renameTable( + Identifier oldIdent, + Identifier newIdent) throws NoSuchTableException, TableAlreadyExistsException { + delegate.renameTable(oldIdent, newIdent); + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala index d5a6a61f82579..5bba88dbe76d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala @@ -27,12 +27,16 @@ import org.apache.spark.sql.internal.SQLConf * A thread-safe manager for [[CatalogPlugin]]s. It tracks all the registered catalogs, and allow * the caller to look up a catalog by name. */ -class CatalogManager(conf: SQLConf) extends Logging { +class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends Logging { private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] def catalog(name: String): CatalogPlugin = synchronized { - catalogs.getOrElseUpdate(name, Catalogs.load(name, conf)) + if (name.equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME)) { + v2SessionCatalog + } else { + catalogs.getOrElseUpdate(name, Catalogs.load(name, conf)) + } } def defaultCatalog: Option[CatalogPlugin] = { @@ -47,16 +51,30 @@ class CatalogManager(conf: SQLConf) extends Logging { } } - def v2SessionCatalog: Option[CatalogPlugin] = { - try { - Some(catalog(CatalogManager.SESSION_CATALOG_NAME)) - } catch { - case NonFatal(e) => - logError("Cannot load v2 session catalog", e) - None + private def loadV2SessionCatalog(): CatalogPlugin = { + Catalogs.load(CatalogManager.SESSION_CATALOG_NAME, conf) match { + case extension: CatalogExtension => + extension.setDelegateCatalog(defaultSessionCatalog) + extension + case other => other } } + // If the V2_SESSION_CATALOG config is specified, we try to instantiate the user-specified v2 + // session catalog. Otherwise, return the default session catalog. + def v2SessionCatalog: CatalogPlugin = { + conf.getConf(SQLConf.V2_SESSION_CATALOG).map { customV2SessionCatalog => + try { + catalogs.getOrElseUpdate(CatalogManager.SESSION_CATALOG_NAME, loadV2SessionCatalog()) + } catch { + case NonFatal(_) => + logError( + "Fail to instantiate the custom v2 session catalog: " + customV2SessionCatalog) + defaultSessionCatalog + } + }.getOrElse(defaultSessionCatalog) + } + private def getDefaultNamespace(c: CatalogPlugin) = c match { case c: SupportsNamespaces => c.defaultNamespace() case _ => Array.empty[String] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala index 846810fc18061..2281402740a36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala @@ -45,7 +45,7 @@ trait LookupCatalog extends Logging { * This happens when the source implementation extends the v2 TableProvider API and is not listed * in the fallback configuration, spark.sql.sources.write.useV1SourceList */ - def sessionCatalog: Option[CatalogPlugin] = catalogManager.v2SessionCatalog + def sessionCatalog: CatalogPlugin = catalogManager.v2SessionCatalog /** * Extract catalog plugin and remaining identifier names. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6f17256f8163e..5d130c5af213f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.analysis +import java.util import java.util.Locale import scala.collection.mutable @@ -25,7 +26,7 @@ import scala.util.Random import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog.v2._ -import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform} +import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -45,6 +46,7 @@ import org.apache.spark.sql.internal.SQLConf.{PartitionOverwriteMode, StoreAssig import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.sources.v2.internal.V1Table import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * A trivial [[Analyzer]] with a dummy [[SessionCatalog]] and [[EmptyFunctionRegistry]]. @@ -60,6 +62,24 @@ object SimpleAnalyzer extends Analyzer( }, new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) +object FakeV2SessionCatalog extends TableCatalog { + private def fail() = throw new UnsupportedOperationException + override def listTables(namespace: Array[String]): Array[Identifier] = fail() + override def loadTable(ident: Identifier): Table = { + throw new NoSuchTableException(ident.toString) + } + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = fail() + override def alterTable(ident: Identifier, changes: TableChange*): Table = fail() + override def dropTable(ident: Identifier): Boolean = fail() + override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = fail() + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = fail() + override def name(): String = fail() +} + /** * Provides a way to keep state during the analysis, this enables us to decouple the concerns * of analysis environment from the catalog. @@ -101,15 +121,21 @@ object AnalysisContext { */ class Analyzer( catalog: SessionCatalog, + v2SessionCatalog: TableCatalog, conf: SQLConf, maxIterations: Int) extends RuleExecutor[LogicalPlan] with CheckAnalysis with LookupCatalog { + // Only for tests. def this(catalog: SessionCatalog, conf: SQLConf) = { - this(catalog, conf, conf.optimizerMaxIterations) + this(catalog, FakeV2SessionCatalog, conf, conf.optimizerMaxIterations) + } + + def this(catalog: SessionCatalog, v2SessionCatalog: TableCatalog, conf: SQLConf) = { + this(catalog, v2SessionCatalog, conf, conf.optimizerMaxIterations) } - override val catalogManager: CatalogManager = new CatalogManager(conf) + override val catalogManager: CatalogManager = new CatalogManager(conf, v2SessionCatalog) def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { AnalysisHelper.markInAnalyzer { @@ -954,7 +980,7 @@ class Analyzer( case scala.Right(tableOpt) => tableOpt.map { table => AlterTable( - sessionCatalog.get.asTableCatalog, // table being resolved means this exists + sessionCatalog.asTableCatalog, Identifier.of(tableName.init.toArray, tableName.last), DataSourceV2Relation.create(table), changes @@ -2837,7 +2863,7 @@ class Analyzer( case CatalogObjectIdentifier(Some(v2Catalog), ident) => scala.Left((v2Catalog, ident, loadTable(v2Catalog, ident))) case CatalogObjectIdentifier(None, ident) => - catalogManager.v2SessionCatalog.flatMap(loadTable(_, ident)) match { + loadTable(catalogManager.v2SessionCatalog, ident) match { case Some(_: V1Table) => scala.Right(None) case other => scala.Right(other) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index fb6baf9003325..6c6cca84e19f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1965,9 +1965,12 @@ object SQLConf { .createOptional val V2_SESSION_CATALOG = buildConf("spark.sql.catalog.session") - .doc("Name of the default v2 catalog, used when a catalog is not identified in queries") + .doc("A catalog implementation that will be used in place of the Spark built-in session " + + "catalog for v2 operations. The implementation may extend `CatalogExtension` to be " + + "passed the Spark built-in session catalog, so that it may delegate calls to the " + + "built-in session catalog.") .stringConf - .createWithDefault("org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog") + .createOptional val LEGACY_LOOSE_UPCAST = buildConf("spark.sql.legacy.looseUpcast") .doc("When true, the upcast will be loose and allows string to atomic types.") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala index f7f190136bfce..d2a2ba50ead45 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala @@ -21,6 +21,7 @@ import java.util import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalog.v2.{CatalogManager, NamespaceChange, SupportsNamespaces} +import org.apache.spark.sql.catalyst.analysis.FakeV2SessionCatalog import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -28,7 +29,7 @@ class CatalogManagerSuite extends SparkFunSuite { test("CatalogManager should reflect the changes of default catalog") { val conf = new SQLConf - val catalogManager = new CatalogManager(conf) + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog) assert(catalogManager.currentCatalog.isEmpty) assert(catalogManager.currentNamespace.sameElements(Array("default"))) @@ -42,7 +43,7 @@ class CatalogManagerSuite extends SparkFunSuite { test("CatalogManager should keep the current catalog once set") { val conf = new SQLConf - val catalogManager = new CatalogManager(conf) + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog) assert(catalogManager.currentCatalog.isEmpty) conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) catalogManager.setCurrentCatalog("dummy") @@ -57,7 +58,7 @@ class CatalogManagerSuite extends SparkFunSuite { test("current namespace should be updated when switching current catalog") { val conf = new SQLConf - val catalogManager = new CatalogManager(conf) + val catalogManager = new CatalogManager(conf, FakeV2SessionCatalog) catalogManager.setCurrentNamespace(Array("abc")) assert(catalogManager.currentNamespace.sameElements(Array("abc"))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index c782e5012d8d7..04a611024eb24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -354,15 +354,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - val sessionCatalogOpt = session.sessionState.analyzer.sessionCatalog + val sessionCatalog = session.sessionState.analyzer.sessionCatalog session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { case CatalogObjectIdentifier(Some(catalog), ident) => insertInto(catalog, ident) - case CatalogObjectIdentifier(None, ident) - if canUseV2 && sessionCatalogOpt.isDefined && ident.namespace().length <= 1 => - insertInto(sessionCatalogOpt.get, ident) + case CatalogObjectIdentifier(None, ident) if canUseV2 && ident.namespace().length <= 1 => + insertInto(sessionCatalog, ident) case AsTableIdentifier(tableIdentifier) => insertInto(tableIdentifier) @@ -488,17 +487,16 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - val sessionCatalogOpt = session.sessionState.analyzer.sessionCatalog + val sessionCatalog = session.sessionState.analyzer.sessionCatalog session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { case CatalogObjectIdentifier(Some(catalog), ident) => saveAsTable(catalog.asTableCatalog, ident, modeForDSV2) - case CatalogObjectIdentifier(None, ident) - if canUseV2 && sessionCatalogOpt.isDefined && ident.namespace().length <= 1 => + case CatalogObjectIdentifier(None, ident) if canUseV2 && ident.namespace().length <= 1 => // We pass in the modeForDSV1, as using the V2 session catalog should maintain compatibility // for now. - saveAsTable(sessionCatalogOpt.get.asTableCatalog, ident, modeForDSV1) + saveAsTable(sessionCatalog.asTableCatalog, ident, modeForDSV1) case AsTableIdentifier(tableIdentifier) => saveAsTable(tableIdentifier) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index a37a2cf7f0369..43bee695bd5ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, TableCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogManager, Identifier, LookupCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation} @@ -40,9 +40,6 @@ case class DataSourceResolution( import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ - def v2SessionCatalog: CatalogPlugin = sessionCatalog.getOrElse( - throw new AnalysisException("No v2 session catalog implementation is available")) - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case CreateTableStatement( AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, @@ -64,7 +61,7 @@ case class DataSourceResolution( case _ => // the identifier had no catalog and no default catalog is set, but the source is v2. // use the v2 session catalog, which delegates to the global v1 session catalog - convertCreateTable(v2SessionCatalog.asTableCatalog, identifier, create) + convertCreateTable(sessionCatalog.asTableCatalog, identifier, create) } case CreateTableAsSelectStatement( @@ -87,7 +84,7 @@ case class DataSourceResolution( case _ => // the identifier had no catalog and no default catalog is set, but the source is v2. // use the v2 session catalog, which delegates to the global v1 session catalog - convertCTAS(v2SessionCatalog.asTableCatalog, identifier, create) + convertCTAS(sessionCatalog.asTableCatalog, identifier, create) } case DescribeColumnStatement( @@ -119,19 +116,13 @@ case class DataSourceResolution( case replace: ReplaceTableStatement => // the provider was not a v1 source, convert to a v2 plan val CatalogObjectIdentifier(maybeCatalog, identifier) = replace.tableName - val catalog = maybeCatalog.orElse(sessionCatalog) - .getOrElse(throw new AnalysisException( - s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) - .asTableCatalog + val catalog = maybeCatalog.getOrElse(sessionCatalog).asTableCatalog convertReplaceTable(catalog, identifier, replace) case rtas: ReplaceTableAsSelectStatement => // the provider was not a v1 source, convert to a v2 plan val CatalogObjectIdentifier(maybeCatalog, identifier) = rtas.tableName - val catalog = maybeCatalog.orElse(sessionCatalog) - .getOrElse(throw new AnalysisException( - s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) - .asTableCatalog + val catalog = maybeCatalog.getOrElse(sessionCatalog).asTableCatalog convertRTAS(catalog, identifier, rtas) case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index ebfd7384930fe..75320fb51db3f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -24,15 +24,15 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalog.v2.{Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.{CatalogManager, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange} import org.apache.spark.sql.catalog.v2.NamespaceChange.{RemoveProperty, SetProperty} -import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, FieldReference, IdentityTransform, Transform} +import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, FieldReference, IdentityTransform, LogicalExpressions, Transform} import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogDatabase, CatalogTable, CatalogTableType, CatalogUtils, SessionCatalog} import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.internal.SessionState +import org.apache.spark.sql.internal.{SessionState, SQLConf} import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.sources.v2.internal.V1Table import org.apache.spark.sql.types.StructType @@ -41,25 +41,17 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * A [[TableCatalog]] that translates calls to the v1 SessionCatalog. */ -class V2SessionCatalog(sessionState: SessionState) extends TableCatalog with SupportsNamespaces { +class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) + extends TableCatalog with SupportsNamespaces { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ import V2SessionCatalog._ - def this() = { - this(SparkSession.active.sessionState) - } - override val defaultNamespace: Array[String] = Array("default") - private lazy val catalog: SessionCatalog = sessionState.catalog - - private var _name: String = _ + override def name: String = CatalogManager.SESSION_CATALOG_NAME - override def name: String = _name - - override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { - this._name = name - } + // This class is instantiated by Spark, so `initialize` method will not be called. + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {} override def listTables(namespace: Array[String]): Array[Identifier] = { namespace match { @@ -92,7 +84,7 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog with Sup properties: util.Map[String, String]): Table = { val (partitionColumns, maybeBucketSpec) = V2SessionCatalog.convertTransforms(partitions) - val provider = properties.getOrDefault("provider", sessionState.conf.defaultDataSourceName) + val provider = properties.getOrDefault("provider", conf.defaultDataSourceName) val tableProperties = properties.asScala val location = Option(properties.get(LOCATION_TABLE_PROP)) val storage = DataSource.buildStorageFormatFromOptions(tableProperties.toMap) @@ -108,7 +100,7 @@ class V2SessionCatalog(sessionState: SessionState) extends TableCatalog with Sup partitionColumnNames = partitionColumns, bucketSpec = maybeBucketSpec, properties = tableProperties.toMap, - tracksPartitionsInCatalog = sessionState.conf.manageFilesourcePartitions, + tracksPartitionsInCatalog = conf.manageFilesourcePartitions, comment = Option(properties.get(COMMENT_TABLE_PROP))) try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index db4885aa01bad..20b0143f098c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ColumnarRule, QueryExecution, SparkOptimizer, SparkPlanner, SparkSqlParser} import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.v2.TableCapabilityCheck +import org.apache.spark.sql.execution.datasources.v2.{TableCapabilityCheck, V2SessionCatalog} import org.apache.spark.sql.streaming.StreamingQueryManager import org.apache.spark.sql.util.ExecutionListenerManager @@ -151,6 +151,8 @@ abstract class BaseSessionStateBuilder( catalog } + protected lazy val v2SessionCatalog = new V2SessionCatalog(catalog, conf) + /** * Interface exposed to the user for registering user-defined functions. * @@ -164,7 +166,7 @@ abstract class BaseSessionStateBuilder( * * Note: this depends on the `conf` and `catalog` fields. */ - protected def analyzer: Analyzer = new Analyzer(catalog, conf) { + protected def analyzer: Analyzer = new Analyzer(catalog, v2SessionCatalog, conf) { override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index bba1dc0f697a1..e2cee593af030 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -61,14 +61,12 @@ class PlanResolutionSuite extends AnalysisTest { invocation.getArgument[String](0) match { case "testcat" => testCat - case "session" => - v2SessionCatalog case name => throw new CatalogNotFoundException(s"No such catalog: $name") } }) when(manager.defaultCatalog).thenReturn(Some(testCat)) - when(manager.v2SessionCatalog).thenCallRealMethod() + when(manager.v2SessionCatalog).thenReturn(v2SessionCatalog) manager } @@ -78,14 +76,12 @@ class PlanResolutionSuite extends AnalysisTest { invocation.getArgument[String](0) match { case "testcat" => testCat - case "session" => - v2SessionCatalog case name => throw new CatalogNotFoundException(s"No such catalog: $name") } }) when(manager.defaultCatalog).thenReturn(None) - when(manager.v2SessionCatalog).thenCallRealMethod() + when(manager.v2SessionCatalog).thenReturn(v2SessionCatalog) manager } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index 275bc339b3b5b..a309152b26143 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -46,7 +46,7 @@ class V2SessionCatalogBaseSuite extends SparkFunSuite with SharedSparkSession wi val testIdent: Identifier = Identifier.of(testNs, "test_table") def newCatalog(): V2SessionCatalog = { - val newCatalog = new V2SessionCatalog(spark.sessionState) + val newCatalog = new V2SessionCatalog(spark.sessionState.catalog, spark.sessionState.conf) newCatalog.initialize("test", CaseInsensitiveStringMap.empty()) newCatalog } @@ -58,7 +58,6 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { override protected def beforeAll(): Unit = { super.beforeAll() - // TODO: when there is a public API for v2 catalogs, use that instead val catalog = newCatalog() catalog.createNamespace(Array("db"), emptyProps) catalog.createNamespace(Array("db2"), emptyProps) @@ -82,16 +81,6 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { private val testIdentNew = Identifier.of(testNs, "test_table_new") - test("Catalogs can load the catalog") { - val catalog = newCatalog() - - val conf = new SQLConf - conf.setConfString("spark.sql.catalog.test", catalog.getClass.getName) - - val loaded = Catalogs.load("test", conf) - assert(loaded.getClass == catalog.getClass) - } - test("listTables") { val catalog = newCatalog() val ident1 = Identifier.of(Array("ns"), "test_table_1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala index fee6962501637..629e825ec68dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -21,15 +21,14 @@ import java.util import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, SaveMode} +import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog, TableChange} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.InMemoryTable -import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog -import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG} +import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG import org.apache.spark.sql.sources.v2.utils.TestV2SessionCatalogBase import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType @@ -89,7 +88,7 @@ class DataSourceV2DataFrameSessionCatalogSuite val t1 = "prop_table" withTable(t1) { spark.range(20).write.format(v2Format).option("path", "abc").saveAsTable(t1) - val cat = spark.sessionState.catalogManager.v2SessionCatalog.get.asInstanceOf[TableCatalog] + val cat = spark.sessionState.catalogManager.v2SessionCatalog.asInstanceOf[TableCatalog] val tableInfo = cat.loadTable(Identifier.of(Array.empty, t1)) assert(tableInfo.properties().get("location") === "abc") assert(tableInfo.properties().get("provider") === v2Format) @@ -156,7 +155,7 @@ private[v2] trait SessionCatalogTest[T <: Table, Catalog <: TestV2SessionCatalog override def afterEach(): Unit = { super.afterEach() catalog("session").asInstanceOf[Catalog].clearTables() - spark.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) + spark.conf.unset(V2_SESSION_CATALOG.key) } protected def verifyTable(tableName: String, expected: DataFrame): Unit diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala index cfbafdb65c7c3..8df65c14a8d95 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala @@ -46,7 +46,7 @@ class DataSourceV2SQLSessionCatalogSuite } override def getTableMetadata(tableName: String): Table = { - val v2Catalog = spark.sessionState.catalogManager.v2SessionCatalog.get + val v2Catalog = spark.sessionState.catalogManager.v2SessionCatalog val nameParts = spark.sessionState.sqlParser.parseMultipartIdentifier(tableName) v2Catalog.asInstanceOf[TableCatalog] .loadTable(Identifier.of(Array.empty, nameParts.last)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index b6e7bc5d1a4dc..de5b4692b8ba0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -19,16 +19,14 @@ package org.apache.spark.sql.sources.v2 import scala.collection.JavaConverters._ -import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog} import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog, StagingInMemoryTableCatalog} -import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG import org.apache.spark.sql.sources.v2.internal.V1Table -import org.apache.spark.sql.types.{ArrayType, BooleanType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap class DataSourceV2SQLSuite @@ -512,7 +510,8 @@ class DataSourceV2SQLSuite } test("CreateTableAsSelect: v2 session catalog can load v1 source table") { - spark.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) + // unset this config to use the default v2 session catalog. + spark.conf.unset(V2_SESSION_CATALOG.key) val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala index 28ce6a94b253a..b25eab1546261 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala @@ -22,9 +22,8 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalog.v2.Identifier +import org.apache.spark.sql.catalog.v2.{DelegatingCatalogExtension, Identifier} import org.apache.spark.sql.catalog.v2.expressions.Transform -import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType @@ -33,7 +32,7 @@ import org.apache.spark.sql.types.StructType * for testing DDL as well as write operations (through df.write.saveAsTable, df.write.insertInto * and SQL). */ -private[v2] trait TestV2SessionCatalogBase[T <: Table] extends V2SessionCatalog { +private[v2] abstract class TestV2SessionCatalogBase[T <: Table] extends DelegatingCatalogExtension { protected val tables: util.Map[Identifier, T] = new ConcurrentHashMap[Identifier, T]() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 188aedc3640b8..3fc7908e9c4c1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -67,7 +67,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session /** * A logical query plan `Analyzer` with rules specific to Hive. */ - override protected def analyzer: Analyzer = new Analyzer(catalog, conf) { + override protected def analyzer: Analyzer = new Analyzer(catalog, v2SessionCatalog, conf) { override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new ResolveHiveSerdeTable(session) +: new FindDataSourceTable(session) +: From 3d6b33a49a8daba17973994169ee4a9e2507a6d9 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Mon, 9 Sep 2019 21:20:34 +0800 Subject: [PATCH 0019/1387] [SPARK-28939][SQL] Propagate SQLConf for plans executed by toRdd ### What changes were proposed in this pull request? The PR proposes to create a custom `RDD` which enables to propagate `SQLConf` also in cases not tracked by SQL execution, as it happens when a `Dataset` is converted to and RDD either using `.rdd` or `.queryExecution.toRdd` and then the returned RDD is used to invoke actions on it. In this way, SQL configs are effective also in these cases, while earlier they were ignored. ### Why are the changes needed? Without this patch, all the times `.rdd` or `.queryExecution.toRdd` are used, all the SQL configs set are ignored. An example of a reproducer can be: ``` withSQLConf(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "false") { val df = spark.range(2).selectExpr((0 to 5000).map(i => s"id as field_$i"): _*) df.createOrReplaceTempView("spark64kb") val data = spark.sql("select * from spark64kb limit 10") // Subexpression elimination is used here, despite it should have been disabled data.describe() } ``` ### Does this PR introduce any user-facing change? When a user calls `.queryExecution.toRdd`, a `SQLExecutionRDD` is returned wrapping the `RDD` of the execute. When `.rdd` is used, an additional `SQLExecutionRDD` is present in the hierarchy. ### How was this patch tested? added UT Closes #25643 from mgaido91/SPARK-28939. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 11 +++- .../spark/sql/execution/QueryExecution.scala | 3 +- .../spark/sql/execution/SQLExecutionRDD.scala | 64 +++++++++++++++++++ .../internal/ExecutorSideSQLConfSuite.scala | 46 ++++++++++++- 4 files changed, 119 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6c6cca84e19f8..d9b0a72618c7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -115,7 +115,9 @@ object SQLConf { * Returns the active config object within the current scope. If there is an active SparkSession, * the proper SQLConf associated with the thread's active session is used. If it's called from * tasks in the executor side, a SQLConf will be created from job local properties, which are set - * and propagated from the driver side. + * and propagated from the driver side, unless a `SQLConf` has been set in the scope by + * `withExistingConf` as done for propagating SQLConf for operations performed on RDDs created + * from DataFrames. * * The way this works is a little bit convoluted, due to the fact that config was added initially * only for physical plans (and as a result not in sql/catalyst module). @@ -129,7 +131,12 @@ object SQLConf { */ def get: SQLConf = { if (TaskContext.get != null) { - new ReadOnlySQLConf(TaskContext.get()) + val conf = existingConf.get() + if (conf != null) { + conf + } else { + new ReadOnlySQLConf(TaskContext.get()) + } } else { val isSchedulerEventLoopThread = SparkContext.getActive .map(_.dagScheduler.eventProcessLoop.eventThread) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index e5e86db29fe61..630d062d6577e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -105,7 +105,8 @@ class QueryExecution( * Given QueryExecution is not a public class, end users are discouraged to use this: please * use `Dataset.rdd` instead where conversion will be applied. */ - lazy val toRdd: RDD[InternalRow] = executedPlan.execute() + lazy val toRdd: RDD[InternalRow] = new SQLExecutionRDD( + executedPlan.execute(), sparkSession.sessionState.conf) /** * Prepares a planned [[SparkPlan]] for execution by inserting shuffle operations and internal diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala new file mode 100644 index 0000000000000..7373da33e12ad --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import java.util.Properties + +import scala.collection.JavaConverters._ + +import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.internal.SQLConf + +/** + * It is just a wrapper over `sqlRDD`, which sets and makes effective all the configs from the + * captured `SQLConf`. + * Please notice that this means we may miss configurations set after the creation of this RDD and + * before its execution. + * + * @param sqlRDD the `RDD` generated by the SQL plan + * @param conf the `SQLConf` to apply to the execution of the SQL plan + */ +class SQLExecutionRDD( + var sqlRDD: RDD[InternalRow], @transient conf: SQLConf) extends RDD[InternalRow](sqlRDD) { + private val sqlConfigs = conf.getAllConfs + private lazy val sqlConfExecutorSide = { + val props = new Properties() + props.putAll(sqlConfigs.asJava) + val newConf = new SQLConf() + newConf.setConf(props) + newConf + } + + override val partitioner = firstParent[InternalRow].partitioner + + override def getPartitions: Array[Partition] = firstParent[InternalRow].partitions + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + // If we are in the context of a tracked SQL operation, `SQLExecution.EXECUTION_ID_KEY` is set + // and we have nothing to do here. Otherwise, we use the `SQLConf` captured at the creation of + // this RDD. + if (context.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) == null) { + SQLConf.withExistingConf(sqlConfExecutorSide) { + firstParent[InternalRow].iterator(split, context) + } + } else { + firstParent[InternalRow].iterator(split, context) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index d885348f3774a..94b73ec186379 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -17,8 +17,13 @@ package org.apache.spark.sql.internal -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlan} import org.apache.spark.sql.execution.debug.codegenStringSeq import org.apache.spark.sql.functions.col import org.apache.spark.sql.test.SQLTestUtils @@ -102,4 +107,41 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { } } } + + test("SPARK-28939: propagate SQLConf also in conversions to RDD") { + val confs = Seq("spark.sql.a" -> "x", "spark.sql.b" -> "y") + val physicalPlan = SQLConfAssertPlan(confs) + val dummyQueryExecution = FakeQueryExecution(spark, physicalPlan) + withSQLConf(confs: _*) { + // Force RDD evaluation to trigger asserts + dummyQueryExecution.toRdd.collect() + } + val dummyQueryExecution1 = FakeQueryExecution(spark, physicalPlan) + // Without setting the configs assertions fail + val e = intercept[SparkException](dummyQueryExecution1.toRdd.collect()) + assert(e.getCause.isInstanceOf[NoSuchElementException]) + } +} + +case class SQLConfAssertPlan(confToCheck: Seq[(String, String)]) extends LeafExecNode { + override protected def doExecute(): RDD[InternalRow] = { + sqlContext + .sparkContext + .parallelize(0 until 2, 2) + .mapPartitions { it => + val confs = SQLConf.get + confToCheck.foreach { case (key, expectedValue) => + assert(confs.getConfString(key) == expectedValue) + } + it.map(i => InternalRow.fromSeq(Seq(i))) + } + } + + override def output: Seq[Attribute] = Seq.empty +} + +case class FakeQueryExecution(spark: SparkSession, physicalPlan: SparkPlan) + extends QueryExecution(spark, LocalRelation()) { + override lazy val sparkPlan: SparkPlan = physicalPlan + override lazy val executedPlan: SparkPlan = physicalPlan } From 6378d4bc06cd1bb1a209bd5fb63d10ef52d75eb4 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 9 Sep 2019 10:19:40 -0500 Subject: [PATCH 0020/1387] [SPARK-28980][CORE][SQL][STREAMING][MLLIB] Remove most items deprecated in Spark 2.2.0 or earlier, for Spark 3 ### What changes were proposed in this pull request? - Remove SQLContext.createExternalTable and Catalog.createExternalTable, deprecated in favor of createTable since 2.2.0, plus tests of deprecated methods - Remove HiveContext, deprecated in 2.0.0, in favor of `SparkSession.builder.enableHiveSupport` - Remove deprecated KinesisUtils.createStream methods, plus tests of deprecated methods, deprecate in 2.2.0 - Remove deprecated MLlib (not Spark ML) linear method support, mostly utility constructors and 'train' methods, and associated docs. This includes methods in LinearRegression, LogisticRegression, Lasso, RidgeRegression. These have been deprecated since 2.0.0 - Remove deprecated Pyspark MLlib linear method support, including LogisticRegressionWithSGD, LinearRegressionWithSGD, LassoWithSGD - Remove 'runs' argument in KMeans.train() method, which has been a no-op since 2.0.0 - Remove deprecated ChiSqSelector isSorted protected method - Remove deprecated 'yarn-cluster' and 'yarn-client' master argument in favor of 'yarn' and deploy mode 'cluster', etc Notes: - I was not able to remove deprecated DataFrameReader.json(RDD) in favor of DataFrameReader.json(Dataset); the former was deprecated in 2.2.0, but, it is still needed to support Pyspark's .json() method, which can't use a Dataset. - Looks like SQLContext.createExternalTable was not actually deprecated in Pyspark, but, almost certainly was meant to be? Catalog.createExternalTable was. - I afterwards noted that the toDegrees, toRadians functions were almost removed fully in SPARK-25908, but Felix suggested keeping just the R version as they hadn't been technically deprecated. I'd like to revisit that. Do we really want the inconsistency? I'm not against reverting it again, but then that implies leaving SQLContext.createExternalTable just in Pyspark too, which seems weird. - I *kept* LogisticRegressionWithSGD, LinearRegressionWithSGD, LassoWithSGD, RidgeRegressionWithSGD in Pyspark, though deprecated, as it is hard to remove them (still used by StreamingLogisticRegressionWithSGD?) and they are not fully removed in Scala. Maybe should not have been deprecated. ### Why are the changes needed? Deprecated items are easiest to remove in a major release, so we should do so as much as possible for Spark 3. This does not target items deprecated 'recently' as of Spark 2.3, which is still 18 months old. ### Does this PR introduce any user-facing change? Yes, in that deprecated items are removed from some public APIs. ### How was this patch tested? Existing tests. Closes #25684 from srowen/SPARK-28980. Lead-authored-by: Sean Owen Co-authored-by: HyukjinKwon Signed-off-by: Sean Owen --- R/pkg/R/sparkR.R | 7 +- R/pkg/tests/fulltests/test_sparkR.R | 4 +- .../scala/org/apache/spark/SparkConf.scala | 17 - .../org/apache/spark/deploy/SparkSubmit.scala | 19 - ...HadoopFSDelegationTokenProviderSuite.scala | 5 +- .../scheduler/BlacklistTrackerSuite.scala | 2 +- dev/sparktestsupport/modules.py | 1 - docs/mllib-evaluation-metrics.md | 28 - docs/mllib-feature-extraction.md | 14 - docs/mllib-linear-methods.md | 51 -- docs/sql-migration-guide-upgrade.md | 5 + docs/streaming-kinesis-integration.md | 2 +- docs/streaming-programming-guide.md | 4 +- .../JavaLinearRegressionWithSGDExample.java | 81 --- .../mllib/JavaRegressionMetricsExample.java | 83 --- .../examples/mllib/LinearRegression.scala | 138 ---- .../LinearRegressionWithSGDExample.scala | 65 -- .../spark/examples/mllib/PCAExample.scala | 75 -- .../mllib/RegressionMetricsExample.scala | 74 -- .../streaming/JavaKinesisWordCountASL.java | 23 +- .../streaming/kinesis/KinesisUtils.scala | 642 ------------------ .../kinesis/KinesisUtilsPythonHelper.scala | 93 +++ .../kinesis/JavaKinesisStreamSuite.java | 98 --- .../kinesis/KinesisStreamSuite.scala | 24 +- .../launcher/SparkSubmitCommandBuilder.java | 2 +- .../mllib/api/python/PythonMLLibAPI.scala | 1 - .../classification/LogisticRegression.scala | 106 --- .../spark/mllib/clustering/KMeans.scala | 67 -- .../spark/mllib/feature/ChiSqSelector.scala | 11 - .../apache/spark/mllib/regression/Lasso.scala | 111 --- .../mllib/regression/LinearRegression.scala | 102 --- .../mllib/regression/RidgeRegression.scala | 108 --- .../JavaLogisticRegressionSuite.java | 9 +- .../mllib/clustering/JavaKMeansSuite.java | 4 +- .../mllib/regression/JavaLassoSuite.java | 7 +- .../regression/JavaLinearRegressionSuite.java | 9 +- .../regression/JavaRidgeRegressionSuite.java | 14 +- .../LogisticRegressionSuite.scala | 22 +- .../spark/mllib/clustering/KMeansSuite.scala | 2 +- .../spark/mllib/regression/LassoSuite.scala | 9 +- .../regression/LinearRegressionSuite.scala | 8 +- .../regression/RidgeRegressionSuite.scala | 11 +- project/MimaExcludes.scala | 14 + python/pyspark/__init__.py | 2 +- python/pyspark/ml/tests/test_image.py | 43 +- python/pyspark/mllib/clustering.py | 8 +- python/pyspark/sql/__init__.py | 4 +- python/pyspark/sql/catalog.py | 20 - python/pyspark/sql/context.py | 67 +- python/pyspark/sql/tests/test_appsubmit.py | 97 --- python/pyspark/sql/tests/test_context.py | 22 +- python/pyspark/streaming/kinesis.py | 1 - .../spark/deploy/yarn/ClientArguments.scala | 2 +- sql/README.md | 2 +- .../org/apache/spark/sql/SQLContext.scala | 91 --- .../apache/spark/sql/catalog/Catalog.scala | 102 +-- .../apache/spark/sql/hive/HiveContext.scala | 63 -- .../org/apache/spark/sql/hive/package.scala | 3 - .../hive/JavaMetastoreDataSourcesSuite.java | 54 -- .../spark/sql/hive/CachedTableSuite.scala | 6 +- .../hive/HiveContextCompatibilitySuite.scala | 103 --- .../sql/hive/MetastoreDataSourcesSuite.scala | 8 +- .../spark/sql/hive/MultiDatabaseSuite.scala | 8 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- 64 files changed, 224 insertions(+), 2656 deletions(-) delete mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java delete mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala delete mode 100644 external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala create mode 100644 external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtilsPythonHelper.scala delete mode 100644 external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java delete mode 100644 python/pyspark/sql/tests/test_appsubmit.py delete mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 31b986c326d0c..cdb59093781fb 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -266,11 +266,12 @@ sparkR.sparkContext <- function( #' df <- read.json(path) #' #' sparkR.session("local[2]", "SparkR", "/home/spark") -#' sparkR.session("yarn-client", "SparkR", "/home/spark", -#' list(spark.executor.memory="4g"), +#' sparkR.session("yarn", "SparkR", "/home/spark", +#' list(spark.executor.memory="4g", spark.submit.deployMode="client"), #' c("one.jar", "two.jar", "three.jar"), #' c("com.databricks:spark-avro_2.12:2.0.1")) -#' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g") +#' sparkR.session(spark.master = "yarn", spark.submit.deployMode = "client", +# spark.executor.memory = "4g") #'} #' @note sparkR.session since 2.0.0 sparkR.session <- function( diff --git a/R/pkg/tests/fulltests/test_sparkR.R b/R/pkg/tests/fulltests/test_sparkR.R index f73fc6baeccef..4232f5ec430f6 100644 --- a/R/pkg/tests/fulltests/test_sparkR.R +++ b/R/pkg/tests/fulltests/test_sparkR.R @@ -36,8 +36,8 @@ test_that("sparkCheckInstall", { # "yarn-client, mesos-client" mode, SPARK_HOME was not set sparkHome <- "" - master <- "yarn-client" - deployMode <- "" + master <- "yarn" + deployMode <- "client" expect_error(sparkCheckInstall(sparkHome, master, deployMode)) sparkHome <- "" master <- "" diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 24be54ec91828..72c9000a0a386 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -548,23 +548,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } - if (contains("spark.master") && get("spark.master").startsWith("yarn-")) { - val warning = s"spark.master ${get("spark.master")} is deprecated in Spark 2.0+, please " + - "instead use \"yarn\" with specified deploy mode." - - get("spark.master") match { - case "yarn-cluster" => - logWarning(warning) - set("spark.master", "yarn") - set(SUBMIT_DEPLOY_MODE, "cluster") - case "yarn-client" => - logWarning(warning) - set("spark.master", "yarn") - set(SUBMIT_DEPLOY_MODE, "client") - case _ => // Any other unexpected master will be checked when creating scheduler backend. - } - } - if (contains(SUBMIT_DEPLOY_MODE)) { get(SUBMIT_DEPLOY_MODE) match { case "cluster" | "client" => 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 12a8473b22025..ebc843d9758ee 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -229,10 +229,6 @@ private[spark] class SparkSubmit extends Logging { // Set the cluster manager val clusterManager: Int = args.master match { case "yarn" => YARN - case "yarn-client" | "yarn-cluster" => - logWarning(s"Master ${args.master} is deprecated since 2.0." + - " Please use master \"yarn\" with specified deploy mode instead.") - YARN case m if m.startsWith("spark") => STANDALONE case m if m.startsWith("mesos") => MESOS case m if m.startsWith("k8s") => KUBERNETES @@ -251,22 +247,7 @@ private[spark] class SparkSubmit extends Logging { -1 } - // Because the deprecated way of specifying "yarn-cluster" and "yarn-client" encapsulate both - // the master and deploy mode, we have some logic to infer the master and deploy mode - // from each other if only one is specified, or exit early if they are at odds. if (clusterManager == YARN) { - (args.master, args.deployMode) match { - case ("yarn-cluster", null) => - deployMode = CLUSTER - args.master = "yarn" - case ("yarn-cluster", "client") => - error("Client deploy mode is not compatible with master \"yarn-cluster\"") - case ("yarn-client", "cluster") => - error("Cluster deploy mode is not compatible with master \"yarn-client\"") - case (_, mode) => - args.master = "yarn" - } - // Make sure YARN is included in our build if we're trying to use it if (!Utils.classIsLoadable(YARN_CLUSTER_SUBMIT_CLASS) && !Utils.isTesting) { error( diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala index 1f19884bc24d3..44f38e7043dcd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala @@ -22,14 +22,15 @@ import org.apache.hadoop.fs.Path import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config.STAGING_DIR +import org.apache.spark.internal.config.{STAGING_DIR, SUBMIT_DEPLOY_MODE} class HadoopFSDelegationTokenProviderSuite extends SparkFunSuite with Matchers { test("hadoopFSsToAccess should return defaultFS even if not configured") { val sparkConf = new SparkConf() val defaultFS = "hdfs://localhost:8020" val statingDir = "hdfs://localhost:8021" - sparkConf.set("spark.master", "yarn-client") + sparkConf.setMaster("yarn") + sparkConf.set(SUBMIT_DEPLOY_MODE, "client") sparkConf.set(STAGING_DIR, statingDir) val hadoopConf = new Configuration() hadoopConf.set("fs.defaultFS", defaultFS) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 93a88cc30a20c..6b0928a0d2e68 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -437,7 +437,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M } test("check blacklist configuration invariants") { - val conf = new SparkConf().setMaster("yarn-cluster") + val conf = new SparkConf().setMaster("yarn").set(config.SUBMIT_DEPLOY_MODE, "cluster") Seq( (2, 2), (2, 3) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 0f6dbf2f99a97..d4e6b6749010a 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -362,7 +362,6 @@ def __hash__(self): "pyspark.sql.window", "pyspark.sql.avro.functions", # unittests - "pyspark.sql.tests.test_appsubmit", "pyspark.sql.tests.test_arrow", "pyspark.sql.tests.test_catalog", "pyspark.sql.tests.test_column", diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md index f931fa32ea541..fb2883de6810a 100644 --- a/docs/mllib-evaluation-metrics.md +++ b/docs/mllib-evaluation-metrics.md @@ -577,31 +577,3 @@ variable from a number of independent variables. - -**Examples** - -
-The following code snippets illustrate how to load a sample dataset, train a linear regression algorithm on the data, -and evaluate the performance of the algorithm by several regression metrics. - -
-Refer to the [`RegressionMetrics` Scala docs](api/scala/index.html#org.apache.spark.mllib.evaluation.RegressionMetrics) for details on the API. - -{% include_example scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala %} - -
- -
-Refer to the [`RegressionMetrics` Java docs](api/java/org/apache/spark/mllib/evaluation/RegressionMetrics.html) for details on the API. - -{% include_example java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java %} - -
- -
-Refer to the [`RegressionMetrics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.evaluation.RegressionMetrics) for more details on the API. - -{% include_example python/mllib/regression_metrics_example.py %} - -
-
diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index b7f8ae9d07b0a..33a223ad486af 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -348,17 +348,3 @@ Refer to the [`ElementwiseProduct` Python docs](api/python/pyspark.mllib.html#py A feature transformer that projects vectors to a low-dimensional space using PCA. Details you can read at [dimensionality reduction](mllib-dimensionality-reduction.html). - -### Example - -The following code demonstrates how to compute principal components on a `Vector` -and use them to project the vectors into a low-dimensional space while keeping associated labels -for calculation a [Linear Regression](mllib-linear-methods.html) - -
-
-Refer to the [`PCA` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.PCA) for details on the API. - -{% include_example scala/org/apache/spark/examples/mllib/PCAExample.scala %} -
-
diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 2d3ec4ca24443..801876dbffa79 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -360,57 +360,6 @@ regularization; and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) u regularization. For all of these models, the average loss or training error, $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$, is known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). -**Examples** - -
- -
-The following example demonstrates how to load training data, parse it as an RDD of LabeledPoint. -The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the mean squared error at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -Refer to the [`LinearRegressionWithSGD` Scala docs](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) and [`LinearRegressionModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionModel) for details on the API. - -{% include_example scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala %} - -[`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) can be used in a similar fashion as `LinearRegressionWithSGD`. - -
- -
-All of MLlib's methods use Java-friendly types, so you can import and call them there the same -way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the -Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. The corresponding Java example to -the Scala snippet provided, is presented below: - -Refer to the [`LinearRegressionWithSGD` Java docs](api/java/org/apache/spark/mllib/regression/LinearRegressionWithSGD.html) and [`LinearRegressionModel` Java docs](api/java/org/apache/spark/mllib/regression/LinearRegressionModel.html) for details on the API. - -{% include_example java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java %} -
- -
-The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. -The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the mean squared error at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -Note that the Python API does not yet support model save/load but will in the future. - -Refer to the [`LinearRegressionWithSGD` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.regression.LinearRegressionWithSGD) and [`LinearRegressionModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.regression.LinearRegressionModel) for more details on the API. - -{% include_example python/mllib/linear_regression_with_sgd_example.py %} -
-
- -In order to run the above application, follow the instructions -provided in the [Self-Contained Applications](quick-start.html#self-contained-applications) -section of the Spark -quick-start guide. Be sure to also include *spark-mllib* to your build file as -a dependency. - ### Streaming linear regression When data arrive in a streaming fashion, it is useful to fit regression models online, diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index cc3ef1e757756..5add2fb93937d 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -23,6 +23,11 @@ license: | {:toc} ## Upgrading From Spark SQL 2.4 to 3.0 + + - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. + + - In Spark 3.0, the deprecated `HiveContext` class has been removed. Use `SparkSession.builder.enableHiveSupport()` instead. + - Since Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join. - Since Spark 3.0, we reversed argument order of the trim function from `TRIM(trimStr, str)` to `TRIM(str, trimStr)` to be compatible with other databases. diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 26a83e52de2b1..3389d453c2cbd 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -81,7 +81,7 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m .storageLevel(StorageLevel.MEMORY_AND_DISK_2) .build(); - See the [API docs](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) + See the [API docs](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisInputDStream.html) and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java). Refer to the [Running the Example](#running-the-example) subsection for instructions to run the example.
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index f5abed74bff20..f6b579fbf74d1 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -2488,13 +2488,13 @@ additional effort may be necessary to achieve exactly-once semantics. There are * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), - [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$), + [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisInputDStream), - Java docs * [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html), [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html) * [KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html), - [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) + [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisInputDStream.html) - Python docs * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) * [KafkaUtils](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java deleted file mode 100644 index 324a781c1a44a..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java +++ /dev/null @@ -1,81 +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.examples.mllib; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; - -// $example on$ -import scala.Tuple2; - -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.regression.LinearRegressionModel; -import org.apache.spark.mllib.regression.LinearRegressionWithSGD; -// $example off$ - -/** - * Example for LinearRegressionWithSGD. - */ -public class JavaLinearRegressionWithSGDExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaLinearRegressionWithSGDExample"); - JavaSparkContext sc = new JavaSparkContext(conf); - - // $example on$ - // Load and parse the data - String path = "data/mllib/ridge-data/lpsa.data"; - JavaRDD data = sc.textFile(path); - JavaRDD parsedData = data.map(line -> { - String[] parts = line.split(","); - String[] features = parts[1].split(" "); - double[] v = new double[features.length]; - for (int i = 0; i < features.length - 1; i++) { - v[i] = Double.parseDouble(features[i]); - } - return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v)); - }); - parsedData.cache(); - - // Building the model - int numIterations = 100; - double stepSize = 0.00000001; - LinearRegressionModel model = - LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations, stepSize); - - // Evaluate model on training examples and compute training error - JavaPairRDD valuesAndPreds = parsedData.mapToPair(point -> - new Tuple2<>(model.predict(point.features()), point.label())); - - double MSE = valuesAndPreds.mapToDouble(pair -> { - double diff = pair._1() - pair._2(); - return diff * diff; - }).mean(); - System.out.println("training Mean Squared Error = " + MSE); - - // Save and load model - model.save(sc.sc(), "target/tmp/javaLinearRegressionWithSGDModel"); - LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(), - "target/tmp/javaLinearRegressionWithSGDModel"); - // $example off$ - - sc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java deleted file mode 100644 index 00033b5730a3d..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java +++ /dev/null @@ -1,83 +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.examples.mllib; - -// $example on$ -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.regression.LinearRegressionModel; -import org.apache.spark.mllib.regression.LinearRegressionWithSGD; -import org.apache.spark.mllib.evaluation.RegressionMetrics; -import org.apache.spark.SparkConf; -// $example off$ - -public class JavaRegressionMetricsExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("Java Regression Metrics Example"); - JavaSparkContext sc = new JavaSparkContext(conf); - // $example on$ - // Load and parse the data - String path = "data/mllib/sample_linear_regression_data.txt"; - JavaRDD data = sc.textFile(path); - JavaRDD parsedData = data.map(line -> { - String[] parts = line.split(" "); - double[] v = new double[parts.length - 1]; - for (int i = 1; i < parts.length; i++) { - v[i - 1] = Double.parseDouble(parts[i].split(":")[1]); - } - return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v)); - }); - parsedData.cache(); - - // Building the model - int numIterations = 100; - LinearRegressionModel model = LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), - numIterations); - - // Evaluate model on training examples and compute training error - JavaPairRDD valuesAndPreds = parsedData.mapToPair(point -> - new Tuple2<>(model.predict(point.features()), point.label())); - - // Instantiate metrics object - RegressionMetrics metrics = new RegressionMetrics(valuesAndPreds.rdd()); - - // Squared error - System.out.format("MSE = %f\n", metrics.meanSquaredError()); - System.out.format("RMSE = %f\n", metrics.rootMeanSquaredError()); - - // R-squared - System.out.format("R Squared = %f\n", metrics.r2()); - - // Mean absolute error - System.out.format("MAE = %f\n", metrics.meanAbsoluteError()); - - // Explained variance - System.out.format("Explained Variance = %f\n", metrics.explainedVariance()); - - // Save and load model - model.save(sc.sc(), "target/tmp/LogisticRegressionModel"); - LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(), - "target/tmp/LogisticRegressionModel"); - // $example off$ - - sc.stop(); - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala deleted file mode 100644 index 03222b13ad27d..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ /dev/null @@ -1,138 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.mllib - -import org.apache.log4j.{Level, Logger} -import scopt.OptionParser - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.optimization.{L1Updater, SimpleUpdater, SquaredL2Updater} -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.util.MLUtils - -/** - * An example app for linear regression. Run with - * {{{ - * bin/run-example org.apache.spark.examples.mllib.LinearRegression - * }}} - * A synthetic dataset can be found at `data/mllib/sample_linear_regression_data.txt`. - * If you use it as a template to create your own app, please use `spark-submit` to submit your app. - */ -@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") -object LinearRegression { - - object RegType extends Enumeration { - type RegType = Value - val NONE, L1, L2 = Value - } - - import RegType._ - - case class Params( - input: String = null, - numIterations: Int = 100, - stepSize: Double = 1.0, - regType: RegType = L2, - regParam: Double = 0.01) extends AbstractParams[Params] - - def main(args: Array[String]) { - val defaultParams = Params() - - val parser = new OptionParser[Params]("LinearRegression") { - head("LinearRegression: an example app for linear regression.") - opt[Int]("numIterations") - .text("number of iterations") - .action((x, c) => c.copy(numIterations = x)) - opt[Double]("stepSize") - .text(s"initial step size, default: ${defaultParams.stepSize}") - .action((x, c) => c.copy(stepSize = x)) - opt[String]("regType") - .text(s"regularization type (${RegType.values.mkString(",")}), " + - s"default: ${defaultParams.regType}") - .action((x, c) => c.copy(regType = RegType.withName(x))) - opt[Double]("regParam") - .text(s"regularization parameter, default: ${defaultParams.regParam}") - arg[String]("") - .required() - .text("input paths to labeled examples in LIBSVM format") - .action((x, c) => c.copy(input = x)) - note( - """ - |For example, the following command runs this app on a synthetic dataset: - | - | bin/spark-submit --class org.apache.spark.examples.mllib.LinearRegression \ - | examples/target/scala-*/spark-examples-*.jar \ - | data/mllib/sample_linear_regression_data.txt - """.stripMargin) - } - - parser.parse(args, defaultParams) match { - case Some(params) => run(params) - case _ => sys.exit(1) - } - } - - def run(params: Params): Unit = { - val conf = new SparkConf().setAppName(s"LinearRegression with $params") - val sc = new SparkContext(conf) - - Logger.getRootLogger.setLevel(Level.WARN) - - val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() - - val splits = examples.randomSplit(Array(0.8, 0.2)) - val training = splits(0).cache() - val test = splits(1).cache() - - val numTraining = training.count() - val numTest = test.count() - println(s"Training: $numTraining, test: $numTest.") - - examples.unpersist() - - val updater = params.regType match { - case NONE => new SimpleUpdater() - case L1 => new L1Updater() - case L2 => new SquaredL2Updater() - } - - val algorithm = new LinearRegressionWithSGD() - algorithm.optimizer - .setNumIterations(params.numIterations) - .setStepSize(params.stepSize) - .setUpdater(updater) - .setRegParam(params.regParam) - - val model = algorithm.run(training) - - val prediction = model.predict(test.map(_.features)) - val predictionAndLabel = prediction.zip(test.map(_.label)) - - val loss = predictionAndLabel.map { case (p, l) => - val err = p - l - err * err - }.reduce(_ + _) - val rmse = math.sqrt(loss / numTest) - - println(s"Test RMSE = $rmse.") - - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala deleted file mode 100644 index 449b725d1d173..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala +++ /dev/null @@ -1,65 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.mllib - -import org.apache.spark.{SparkConf, SparkContext} -// $example on$ -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.regression.LinearRegressionModel -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -// $example off$ - -@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") -object LinearRegressionWithSGDExample { - - def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("LinearRegressionWithSGDExample") - val sc = new SparkContext(conf) - - // $example on$ - // Load and parse the data - val data = sc.textFile("data/mllib/ridge-data/lpsa.data") - val parsedData = data.map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) - }.cache() - - // Building the model - val numIterations = 100 - val stepSize = 0.00000001 - val model = LinearRegressionWithSGD.train(parsedData, numIterations, stepSize) - - // Evaluate model on training examples and compute training error - val valuesAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) - } - val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2) }.mean() - println(s"training Mean Squared Error $MSE") - - // Save and load model - model.save(sc, "target/tmp/scalaLinearRegressionWithSGDModel") - val sameModel = LinearRegressionModel.load(sc, "target/tmp/scalaLinearRegressionWithSGDModel") - // $example off$ - - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala deleted file mode 100644 index eff2393cc3abe..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala +++ /dev/null @@ -1,75 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.mllib - -import org.apache.spark.SparkConf -import org.apache.spark.SparkContext -// $example on$ -import org.apache.spark.mllib.feature.PCA -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} -// $example off$ - -@deprecated("Deprecated since LinearRegressionWithSGD is deprecated. Use ml.feature.PCA", "2.0.0") -object PCAExample { - - def main(args: Array[String]): Unit = { - - val conf = new SparkConf().setAppName("PCAExample") - val sc = new SparkContext(conf) - - // $example on$ - val data = sc.textFile("data/mllib/ridge-data/lpsa.data").map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) - }.cache() - - val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) - val training = splits(0).cache() - val test = splits(1) - - val pca = new PCA(training.first().features.size / 2).fit(data.map(_.features)) - val training_pca = training.map(p => p.copy(features = pca.transform(p.features))) - val test_pca = test.map(p => p.copy(features = pca.transform(p.features))) - - val numIterations = 100 - val model = LinearRegressionWithSGD.train(training, numIterations) - val model_pca = LinearRegressionWithSGD.train(training_pca, numIterations) - - val valuesAndPreds = test.map { point => - val score = model.predict(point.features) - (score, point.label) - } - - val valuesAndPreds_pca = test_pca.map { point => - val score = model_pca.predict(point.features) - (score, point.label) - } - - val MSE = valuesAndPreds.map { case (v, p) => math.pow((v - p), 2) }.mean() - val MSE_pca = valuesAndPreds_pca.map { case (v, p) => math.pow((v - p), 2) }.mean() - - println(s"Mean Squared Error = $MSE") - println(s"PCA Mean Squared Error = $MSE_pca") - // $example off$ - - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala deleted file mode 100644 index 76cfb804e18f3..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala +++ /dev/null @@ -1,74 +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. - */ -// scalastyle:off println - -package org.apache.spark.examples.mllib - -// $example on$ -import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} -// $example off$ -import org.apache.spark.sql.SparkSession - -@deprecated("Use ml.regression.LinearRegression and the resulting model summary for metrics", - "2.0.0") -object RegressionMetricsExample { - def main(args: Array[String]): Unit = { - val spark = SparkSession - .builder - .appName("RegressionMetricsExample") - .getOrCreate() - // $example on$ - // Load the data - val data = spark - .read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") - .rdd.map(row => LabeledPoint(row.getDouble(0), row.get(1).asInstanceOf[Vector])) - .cache() - - // Build the model - val numIterations = 100 - val model = LinearRegressionWithSGD.train(data, numIterations) - - // Get predictions - val valuesAndPreds = data.map{ point => - val prediction = model.predict(point.features) - (prediction, point.label) - } - - // Instantiate metrics object - val metrics = new RegressionMetrics(valuesAndPreds) - - // Squared error - println(s"MSE = ${metrics.meanSquaredError}") - println(s"RMSE = ${metrics.rootMeanSquaredError}") - - // R-squared - println(s"R-squared = ${metrics.r2}") - - // Mean absolute error - println(s"MAE = ${metrics.meanAbsoluteError}") - - // Explained variance - println(s"Explained variance = ${metrics.explainedVariance}") - // $example off$ - - spark.stop() - } -} -// scalastyle:on println - diff --git a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index 86c42df9e8435..31ca2fe5c95ff 100644 --- a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -32,13 +32,14 @@ import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.apache.spark.streaming.kinesis.KinesisUtils; +import org.apache.spark.streaming.kinesis.KinesisInitialPositions; +import org.apache.spark.streaming.kinesis.KinesisInputDStream; import scala.Tuple2; +import scala.reflect.ClassTag$; import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; import com.amazonaws.services.kinesis.AmazonKinesisClient; -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; /** * Consumes messages from a Amazon Kinesis streams and does wordcount. @@ -135,11 +136,19 @@ public static void main(String[] args) throws Exception { // Create the Kinesis DStreams List> streamsList = new ArrayList<>(numStreams); for (int i = 0; i < numStreams; i++) { - streamsList.add( - KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, - InitialPositionInStream.LATEST, kinesisCheckpointInterval, - StorageLevel.MEMORY_AND_DISK_2()) - ); + streamsList.add(JavaDStream.fromDStream( + KinesisInputDStream.builder() + .streamingContext(jssc) + .checkpointAppName(kinesisAppName) + .streamName(streamName) + .endpointUrl(endpointUrl) + .regionName(regionName) + .initialPosition(new KinesisInitialPositions.Latest()) + .checkpointInterval(kinesisCheckpointInterval) + .storageLevel(StorageLevel.MEMORY_AND_DISK_2()) + .build(), + ClassTag$.MODULE$.apply(byte[].class) + )); } // Union all the streams if there is more than 1 stream diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala deleted file mode 100644 index 4e796b2caec87..0000000000000 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala +++ /dev/null @@ -1,642 +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.streaming.kinesis - -import scala.reflect.ClassTag - -import com.amazonaws.regions.RegionUtils -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream -import com.amazonaws.services.kinesis.model.Record - -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Duration, StreamingContext} -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -object KinesisUtils { - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param ssc StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param messageHandler A custom message handler that can generate a generic output from a - * Kinesis `Record`, which contains both message data, and metadata. - * - * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets the AWS credentials. - */ - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream[T: ClassTag]( - ssc: StreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel, - messageHandler: Record => T): ReceiverInputDStream[T] = { - val cleanedHandler = ssc.sc.clean(messageHandler) - // Setting scope to override receiver stream's scope of "receiver stream" - ssc.withNamedScope("kinesis stream") { - new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), - KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), - kinesisAppName, checkpointInterval, storageLevel, - cleanedHandler, DefaultCredentials, None, None, - KinesisInputDStream.DEFAULT_METRICS_LEVEL, - KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) - } - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param ssc StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param messageHandler A custom message handler that can generate a generic output from a - * Kinesis `Record`, which contains both message data, and metadata. - * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) - * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) - * - * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - */ - // scalastyle:off - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream[T: ClassTag]( - ssc: StreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel, - messageHandler: Record => T, - awsAccessKeyId: String, - awsSecretKey: String): ReceiverInputDStream[T] = { - // scalastyle:on - val cleanedHandler = ssc.sc.clean(messageHandler) - ssc.withNamedScope("kinesis stream") { - val kinesisCredsProvider = BasicCredentials( - awsAccessKeyId = awsAccessKeyId, - awsSecretKey = awsSecretKey) - new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), - KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), - kinesisAppName, checkpointInterval, storageLevel, - cleanedHandler, kinesisCredsProvider, None, None, - KinesisInputDStream.DEFAULT_METRICS_LEVEL, - KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) - } - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param ssc StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param messageHandler A custom message handler that can generate a generic output from a - * Kinesis `Record`, which contains both message data, and metadata. - * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) - * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) - * @param stsAssumeRoleArn ARN of IAM role to assume when using STS sessions to read from - * Kinesis stream. - * @param stsSessionName Name to uniquely identify STS sessions if multiple principals assume - * the same role. - * @param stsExternalId External ID that can be used to validate against the assumed IAM role's - * trust policy. - * - * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - */ - // scalastyle:off - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream[T: ClassTag]( - ssc: StreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel, - messageHandler: Record => T, - awsAccessKeyId: String, - awsSecretKey: String, - stsAssumeRoleArn: String, - stsSessionName: String, - stsExternalId: String): ReceiverInputDStream[T] = { - // scalastyle:on - val cleanedHandler = ssc.sc.clean(messageHandler) - ssc.withNamedScope("kinesis stream") { - val kinesisCredsProvider = STSCredentials( - stsRoleArn = stsAssumeRoleArn, - stsSessionName = stsSessionName, - stsExternalId = Option(stsExternalId), - longLivedCreds = BasicCredentials( - awsAccessKeyId = awsAccessKeyId, - awsSecretKey = awsSecretKey)) - new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), - KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), - kinesisAppName, checkpointInterval, storageLevel, - cleanedHandler, kinesisCredsProvider, None, None, - KinesisInputDStream.DEFAULT_METRICS_LEVEL, - KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) - } - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param ssc StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * - * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets the AWS credentials. - */ - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream( - ssc: StreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel): ReceiverInputDStream[Array[Byte]] = { - // Setting scope to override receiver stream's scope of "receiver stream" - ssc.withNamedScope("kinesis stream") { - new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName), - KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), - kinesisAppName, checkpointInterval, storageLevel, - KinesisInputDStream.defaultMessageHandler, DefaultCredentials, None, None, - KinesisInputDStream.DEFAULT_METRICS_LEVEL, - KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) - } - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param ssc StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) - * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) - * - * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - */ - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream( - ssc: StreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel, - awsAccessKeyId: String, - awsSecretKey: String): ReceiverInputDStream[Array[Byte]] = { - ssc.withNamedScope("kinesis stream") { - val kinesisCredsProvider = BasicCredentials( - awsAccessKeyId = awsAccessKeyId, - awsSecretKey = awsSecretKey) - new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName), - KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream), - kinesisAppName, checkpointInterval, storageLevel, - KinesisInputDStream.defaultMessageHandler, kinesisCredsProvider, None, None, - KinesisInputDStream.DEFAULT_METRICS_LEVEL, - KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS) - } - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param jssc Java StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param messageHandler A custom message handler that can generate a generic output from a - * Kinesis `Record`, which contains both message data, and metadata. - * @param recordClass Class of the records in DStream - * - * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets the AWS credentials. - */ - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream[T]( - jssc: JavaStreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel, - messageHandler: JFunction[Record, T], - recordClass: Class[T]): JavaReceiverInputDStream[T] = { - implicit val recordCmt: ClassTag[T] = ClassTag(recordClass) - val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_)) - createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, - initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler) - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param jssc Java StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param messageHandler A custom message handler that can generate a generic output from a - * Kinesis `Record`, which contains both message data, and metadata. - * @param recordClass Class of the records in DStream - * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) - * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) - * - * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - */ - // scalastyle:off - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream[T]( - jssc: JavaStreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel, - messageHandler: JFunction[Record, T], - recordClass: Class[T], - awsAccessKeyId: String, - awsSecretKey: String): JavaReceiverInputDStream[T] = { - // scalastyle:on - implicit val recordCmt: ClassTag[T] = ClassTag(recordClass) - val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_)) - createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, - initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler, - awsAccessKeyId, awsSecretKey) - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param jssc Java StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param messageHandler A custom message handler that can generate a generic output from a - * Kinesis `Record`, which contains both message data, and metadata. - * @param recordClass Class of the records in DStream - * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) - * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) - * @param stsAssumeRoleArn ARN of IAM role to assume when using STS sessions to read from - * Kinesis stream. - * @param stsSessionName Name to uniquely identify STS sessions if multiple princpals assume - * the same role. - * @param stsExternalId External ID that can be used to validate against the assumed IAM role's - * trust policy. - * - * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - */ - // scalastyle:off - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream[T]( - jssc: JavaStreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel, - messageHandler: JFunction[Record, T], - recordClass: Class[T], - awsAccessKeyId: String, - awsSecretKey: String, - stsAssumeRoleArn: String, - stsSessionName: String, - stsExternalId: String): JavaReceiverInputDStream[T] = { - // scalastyle:on - implicit val recordCmt: ClassTag[T] = ClassTag(recordClass) - val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_)) - createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, - initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler, - awsAccessKeyId, awsSecretKey, stsAssumeRoleArn, stsSessionName, stsExternalId) - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param jssc Java StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * - * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets the AWS credentials. - */ - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream( - jssc: JavaStreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[Array[Byte]] = { - createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, - initialPositionInStream, checkpointInterval, storageLevel, - KinesisInputDStream.defaultMessageHandler(_)) - } - - /** - * Create an input stream that pulls messages from a Kinesis stream. - * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. - * - * @param jssc Java StreamingContext object - * @param kinesisAppName Kinesis application name used by the Kinesis Client Library - * (KCL) to update DynamoDB - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Name of region used by the Kinesis Client Library (KCL) to update - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects. - * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) - * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) - * - * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - */ - @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") - def createStream( - jssc: JavaStreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointInterval: Duration, - storageLevel: StorageLevel, - awsAccessKeyId: String, - awsSecretKey: String): JavaReceiverInputDStream[Array[Byte]] = { - createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, - initialPositionInStream, checkpointInterval, storageLevel, - KinesisInputDStream.defaultMessageHandler(_), awsAccessKeyId, awsSecretKey) - } - - private def validateRegion(regionName: String): String = { - Option(RegionUtils.getRegion(regionName)).map { _.getName }.getOrElse { - throw new IllegalArgumentException(s"Region name '$regionName' is not valid") - } - } -} - -/** - * This is a helper class that wraps the methods in KinesisUtils into more Python-friendly class and - * function so that it can be easily instantiated and called from Python's KinesisUtils. - */ -private class KinesisUtilsPythonHelper { - - def getInitialPositionInStream(initialPositionInStream: Int): InitialPositionInStream = { - initialPositionInStream match { - case 0 => InitialPositionInStream.LATEST - case 1 => InitialPositionInStream.TRIM_HORIZON - case _ => throw new IllegalArgumentException( - "Illegal InitialPositionInStream. Please use " + - "InitialPositionInStream.LATEST or InitialPositionInStream.TRIM_HORIZON") - } - } - - // scalastyle:off - def createStream( - jssc: JavaStreamingContext, - kinesisAppName: String, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: Int, - checkpointInterval: Duration, - storageLevel: StorageLevel, - awsAccessKeyId: String, - awsSecretKey: String, - stsAssumeRoleArn: String, - stsSessionName: String, - stsExternalId: String): JavaReceiverInputDStream[Array[Byte]] = { - // scalastyle:on - if (!(stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) - && !(stsAssumeRoleArn == null && stsSessionName == null && stsExternalId == null)) { - throw new IllegalArgumentException("stsAssumeRoleArn, stsSessionName, and stsExtenalId " + - "must all be defined or all be null") - } - - if (stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) { - validateAwsCreds(awsAccessKeyId, awsSecretKey) - KinesisUtils.createStream(jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, - getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel, - KinesisInputDStream.defaultMessageHandler(_), awsAccessKeyId, awsSecretKey, - stsAssumeRoleArn, stsSessionName, stsExternalId) - } else { - validateAwsCreds(awsAccessKeyId, awsSecretKey) - if (awsAccessKeyId == null && awsSecretKey == null) { - KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, - getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel) - } else { - KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, - getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel, - awsAccessKeyId, awsSecretKey) - } - } - } - - // Throw IllegalArgumentException unless both values are null or neither are. - private def validateAwsCreds(awsAccessKeyId: String, awsSecretKey: String) { - if (awsAccessKeyId == null && awsSecretKey != null) { - throw new IllegalArgumentException("awsSecretKey is set but awsAccessKeyId is null") - } - if (awsAccessKeyId != null && awsSecretKey == null) { - throw new IllegalArgumentException("awsAccessKeyId is set but awsSecretKey is null") - } - } -} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtilsPythonHelper.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtilsPythonHelper.scala new file mode 100644 index 0000000000000..c89dedd3366d1 --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtilsPythonHelper.scala @@ -0,0 +1,93 @@ +/* + * 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.streaming.kinesis + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} + +/** + * This is a helper class that wraps the methods in KinesisUtils into more Python-friendly class and + * function so that it can be easily instantiated and called from Python's KinesisUtils. + */ +private class KinesisUtilsPythonHelper { + + // scalastyle:off + def createStream( + jssc: JavaStreamingContext, + kinesisAppName: String, + streamName: String, + endpointUrl: String, + regionName: String, + initialPositionInStream: Int, + checkpointInterval: Duration, + storageLevel: StorageLevel, + awsAccessKeyId: String, + awsSecretKey: String, + stsAssumeRoleArn: String, + stsSessionName: String, + stsExternalId: String): JavaReceiverInputDStream[Array[Byte]] = { + // scalastyle:on + if (!(stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) + && !(stsAssumeRoleArn == null && stsSessionName == null && stsExternalId == null)) { + throw new IllegalArgumentException("stsAssumeRoleArn, stsSessionName, and stsExtenalId " + + "must all be defined or all be null") + } + if (awsAccessKeyId == null && awsSecretKey != null) { + throw new IllegalArgumentException("awsSecretKey is set but awsAccessKeyId is null") + } + if (awsAccessKeyId != null && awsSecretKey == null) { + throw new IllegalArgumentException("awsAccessKeyId is set but awsSecretKey is null") + } + + val kinesisInitialPosition = initialPositionInStream match { + case 0 => InitialPositionInStream.LATEST + case 1 => InitialPositionInStream.TRIM_HORIZON + case _ => throw new IllegalArgumentException( + "Illegal InitialPositionInStream. Please use " + + "InitialPositionInStream.LATEST or InitialPositionInStream.TRIM_HORIZON") + } + + val builder = KinesisInputDStream.builder. + streamingContext(jssc). + checkpointAppName(kinesisAppName). + streamName(streamName). + endpointUrl(endpointUrl). + regionName(regionName). + initialPosition(KinesisInitialPositions.fromKinesisInitialPosition(kinesisInitialPosition)). + checkpointInterval(checkpointInterval). + storageLevel(storageLevel) + + if (stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) { + val kinesisCredsProvider = STSCredentials( + stsAssumeRoleArn, stsSessionName, Option(stsExternalId), + BasicCredentials(awsAccessKeyId, awsSecretKey)) + builder. + kinesisCredentials(kinesisCredsProvider). + buildWithMessageHandler(KinesisInputDStream.defaultMessageHandler) + } else { + if (awsAccessKeyId == null && awsSecretKey == null) { + builder.build() + } else { + builder.kinesisCredentials(BasicCredentials(awsAccessKeyId, awsSecretKey)).build() + } + } + } + +} diff --git a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java deleted file mode 100644 index b37b087467926..0000000000000 --- a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java +++ /dev/null @@ -1,98 +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.streaming.kinesis; - -import com.amazonaws.services.kinesis.model.Record; -import org.junit.Test; - -import org.apache.spark.api.java.function.Function; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.Duration; -import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; - -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; - -/** - * Demonstrate the use of the KinesisUtils Java API - */ -public class JavaKinesisStreamSuite extends LocalJavaStreamingContext { - @Test - public void testKinesisStream() { - String dummyEndpointUrl = KinesisTestUtils.defaultEndpointUrl(); - String dummyRegionName = KinesisTestUtils.getRegionNameByEndpoint(dummyEndpointUrl); - - // Tests the API, does not actually test data receiving - JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream", - dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, new Duration(2000), - StorageLevel.MEMORY_AND_DISK_2()); - ssc.stop(); - } - - @Test - public void testAwsCreds() { - String dummyEndpointUrl = KinesisTestUtils.defaultEndpointUrl(); - String dummyRegionName = KinesisTestUtils.getRegionNameByEndpoint(dummyEndpointUrl); - - // Tests the API, does not actually test data receiving - JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream", - dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, new Duration(2000), - StorageLevel.MEMORY_AND_DISK_2(), "fakeAccessKey", "fakeSecretKey"); - ssc.stop(); - } - - private static Function handler = new Function() { - @Override - public String call(Record record) { - return record.getPartitionKey() + "-" + record.getSequenceNumber(); - } - }; - - @Test - public void testCustomHandler() { - // Tests the API, does not actually test data receiving - JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream", - "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST, - new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class); - - ssc.stop(); - } - - @Test - public void testCustomHandlerAwsCreds() { - // Tests the API, does not actually test data receiving - JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream", - "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST, - new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class, - "fakeAccessKey", "fakeSecretKey"); - - ssc.stop(); - } - - @Test - public void testCustomHandlerAwsStsCreds() { - // Tests the API, does not actually test data receiving - JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream", - "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST, - new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class, - "fakeAccessKey", "fakeSecretKey", "fakeSTSRoleArn", "fakeSTSSessionName", - "fakeSTSExternalId"); - - ssc.stop(); - } -} diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index 51ee7fd213de5..ee53fbadf8887 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -21,7 +21,6 @@ import scala.collection.mutable import scala.concurrent.duration._ import scala.util.Random -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream import com.amazonaws.services.kinesis.model.Record import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.Matchers._ @@ -102,20 +101,17 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun } } - test("KinesisUtils API") { - val kinesisStream1 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream", - dummyEndpointUrl, dummyRegionName, - InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2) - val kinesisStream2 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream", - dummyEndpointUrl, dummyRegionName, - InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2, - dummyAWSAccessKey, dummyAWSSecretKey) - } - test("RDD generation") { - val inputStream = KinesisUtils.createStream(ssc, appName, "dummyStream", - dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, Seconds(2), - StorageLevel.MEMORY_AND_DISK_2, dummyAWSAccessKey, dummyAWSSecretKey) + val inputStream = KinesisInputDStream.builder. + streamingContext(ssc). + checkpointAppName(appName). + streamName("dummyStream"). + endpointUrl(dummyEndpointUrl). + regionName(dummyRegionName).initialPosition(new Latest()). + checkpointInterval(Seconds(2)). + storageLevel(StorageLevel.MEMORY_AND_DISK_2). + kinesisCredentials(BasicCredentials(dummyAWSAccessKey, dummyAWSSecretKey)). + build() assert(inputStream.isInstanceOf[KinesisInputDStream[Array[Byte]]]) val kinesisStream = inputStream.asInstanceOf[KinesisInputDStream[Array[Byte]]] diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 3479e0c3422bd..f3805f70ea30d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -392,7 +392,7 @@ boolean isClientMode(Map userProps) { // Default master is "local[*]", so assume client mode in that case return userMaster == null || "client".equals(userDeployMode) || - (!userMaster.equals("yarn-cluster") && userDeployMode == null); + (!userMaster.equals("yarn") && userDeployMode == null); } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 4617073f9decd..f6e51ab7f84f8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -347,7 +347,6 @@ private[python] class PythonMLLibAPI extends Serializable { data: JavaRDD[Vector], k: Int, maxIterations: Int, - runs: Int, initializationMode: String, seed: java.lang.Long, initializationSteps: Int, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index d86aa01c9195a..df888bc3d5d51 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -224,117 +224,11 @@ class LogisticRegressionWithSGD private[mllib] ( .setMiniBatchFraction(miniBatchFraction) override protected val validators = List(DataValidators.binaryLabelValidator) - /** - * Construct a LogisticRegression object with default parameters: {stepSize: 1.0, - * numIterations: 100, regParm: 0.01, miniBatchFraction: 1.0}. - */ - @Since("0.8.0") - @deprecated("Use ml.classification.LogisticRegression or LogisticRegressionWithLBFGS", "2.0.0") - def this() = this(1.0, 100, 0.01, 1.0) - override protected[mllib] def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } } -/** - * Top-level methods for calling Logistic Regression using Stochastic Gradient Descent. - * - * @note Labels used in Logistic Regression should be {0, 1} - */ -@Since("0.8.0") -@deprecated("Use ml.classification.LogisticRegression or LogisticRegressionWithLBFGS", "2.0.0") -object LogisticRegressionWithSGD { - // NOTE(shivaram): We use multiple train methods instead of default arguments to support - // Java programs. - - /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed - * number of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in - * gradient descent are initialized using the initial weights provided. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - * - * @note Labels used in Logistic Regression should be {0, 1} - */ - @Since("1.0.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double, - initialWeights: Vector): LogisticRegressionModel = { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction) - .run(input, initialWeights) - } - - /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed - * number of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. - * - * @note Labels used in Logistic Regression should be {0, 1} - */ - @Since("1.0.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double): LogisticRegressionModel = { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction) - .run(input) - } - - /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed - * number of iterations of gradient descent using the specified step size. We use the entire data - * set to update the gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LogisticRegressionModel which has the weights and offset from training. - * - * @note Labels used in Logistic Regression should be {0, 1} - */ - @Since("1.0.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double): LogisticRegressionModel = { - train(input, numIterations, stepSize, 1.0) - } - - /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed - * number of iterations of gradient descent using a step size of 1.0. We use the entire data set - * to update the gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LogisticRegressionModel which has the weights and offset from training. - * - * @note Labels used in Logistic Regression should be {0, 1} - */ - @Since("1.0.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int): LogisticRegressionModel = { - train(input, numIterations, 1.0, 1.0) - } -} - /** * Train a classification model for Multinomial/Binary Logistic Regression using * Limited-memory BFGS. Standard feature scaling and L2 regularization are used by default. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 4bb79bc69eef4..278d61d916735 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -479,58 +479,6 @@ object KMeans { .run(data) } - /** - * Trains a k-means model using the given set of parameters. - * - * @param data Training points as an `RDD` of `Vector` types. - * @param k Number of clusters to create. - * @param maxIterations Maximum number of iterations allowed. - * @param runs This param has no effect since Spark 2.0.0. - * @param initializationMode The initialization algorithm. This can either be "random" or - * "k-means||". (default: "k-means||") - * @param seed Random seed for cluster initialization. Default is to generate seed based - * on system time. - */ - @Since("1.3.0") - @deprecated("Use train method without 'runs'", "2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - runs: Int, - initializationMode: String, - seed: Long): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setInitializationMode(initializationMode) - .setSeed(seed) - .run(data) - } - - /** - * Trains a k-means model using the given set of parameters. - * - * @param data Training points as an `RDD` of `Vector` types. - * @param k Number of clusters to create. - * @param maxIterations Maximum number of iterations allowed. - * @param runs This param has no effect since Spark 2.0.0. - * @param initializationMode The initialization algorithm. This can either be "random" or - * "k-means||". (default: "k-means||") - */ - @Since("0.8.0") - @deprecated("Use train method without 'runs'", "2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - runs: Int, - initializationMode: String): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setInitializationMode(initializationMode) - .run(data) - } - /** * Trains a k-means model using specified parameters and the default values for unspecified. */ @@ -544,21 +492,6 @@ object KMeans { .run(data) } - /** - * Trains a k-means model using specified parameters and the default values for unspecified. - */ - @Since("0.8.0") - @deprecated("Use train method without 'runs'", "2.1.0") - def train( - data: RDD[Vector], - k: Int, - maxIterations: Int, - runs: Int): KMeansModel = { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .run(data) - } - private[spark] def validateInitMode(initMode: String): Boolean = { initMode match { case KMeans.RANDOM => true diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala index 82f5b279846ba..b771e077b02ac 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala @@ -44,17 +44,6 @@ class ChiSqSelectorModel @Since("1.3.0") ( private val filterIndices = selectedFeatures.sorted - @deprecated("not intended for subclasses to use", "2.1.0") - protected def isSorted(array: Array[Int]): Boolean = { - var i = 1 - val len = array.length - while (i < len) { - if (array(i) < array(i-1)) return false - i += 1 - } - true - } - /** * Applies transformation on a vector. * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index ead9f5b300375..47bb1fa9127a6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Loader, Saveable} -import org.apache.spark.rdd.RDD /** * Regression model trained using Lasso. @@ -99,117 +98,7 @@ class LassoWithSGD private[mllib] ( .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - /** - * Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100, - * regParam: 0.01, miniBatchFraction: 1.0}. - */ - @Since("0.8.0") - @deprecated("Use ml.regression.LinearRegression with elasticNetParam = 1.0. Note the default " + - "regParam is 0.01 for LassoWithSGD, but is 0.0 for LinearRegression.", "2.0.0") - def this() = this(1.0, 100, 0.01, 1.0) - override protected def createModel(weights: Vector, intercept: Double) = { new LassoModel(weights, intercept) } } - -/** - * Top-level methods for calling Lasso. - * - */ -@Since("0.8.0") -@deprecated("Use ml.regression.LinearRegression with elasticNetParam = 1.0. Note the default " + - "regParam is 0.01 for LassoWithSGD, but is 0.0 for LinearRegression.", "2.0.0") -object LassoWithSGD { - - /** - * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. The weights used - * in gradient descent are initialized using the initial weights provided. - * - * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data - * matrix A as well as the corresponding right hand side label y - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size scaling to be used for the iterations of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - * - */ - @Since("1.0.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeights: Vector): LassoModel = { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction) - .run(input, initialWeights) - } - - /** - * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. - * - * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data - * matrix A as well as the corresponding right hand side label y - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double): LassoModel = { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) - } - - /** - * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to - * update the true gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data - * matrix A as well as the corresponding right hand side label y - * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param regParam Regularization parameter. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LassoModel which has the weights and offset from training. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double): LassoModel = { - train(input, numIterations, stepSize, regParam, 1.0) - } - - /** - * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to - * compute the true gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data - * matrix A as well as the corresponding right hand side label y - * @param numIterations Number of iterations of gradient descent to run. - * @return a LassoModel which has the weights and offset from training. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int): LassoModel = { - train(input, numIterations, 1.0, 0.01, 1.0) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index cb08216fbf690..f68ebc17e294d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Loader, Saveable} -import org.apache.spark.rdd.RDD /** * Regression model trained using LinearRegression. @@ -100,109 +99,8 @@ class LinearRegressionWithSGD private[mllib] ( .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - /** - * Construct a LinearRegression object with default parameters: {stepSize: 1.0, - * numIterations: 100, miniBatchFraction: 1.0}. - */ - @Since("0.8.0") - @deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") - def this() = this(1.0, 100, 0.0, 1.0) - override protected[mllib] def createModel(weights: Vector, intercept: Double) = { new LinearRegressionModel(weights, intercept) } } -/** - * Top-level methods for calling LinearRegression. - * - */ -@Since("0.8.0") -@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") -object LinearRegressionWithSGD { - - /** - * Train a Linear Regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. The weights used - * in gradient descent are initialized using the initial weights provided. - * - * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data - * matrix A as well as the corresponding right hand side label y - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - * - */ - @Since("1.0.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double, - initialWeights: Vector): LinearRegressionModel = { - new LinearRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction) - .run(input, initialWeights) - } - - /** - * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. - * - * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data - * matrix A as well as the corresponding right hand side label y - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double): LinearRegressionModel = { - new LinearRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run(input) - } - - /** - * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to - * compute the true gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data - * matrix A as well as the corresponding right hand side label y - * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LinearRegressionModel which has the weights and offset from training. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double): LinearRegressionModel = { - train(input, numIterations, stepSize, 1.0) - } - - /** - * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to - * compute the true gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data - * matrix A as well as the corresponding right hand side label y - * @param numIterations Number of iterations of gradient descent to run. - * @return a LinearRegressionModel which has the weights and offset from training. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int): LinearRegressionModel = { - train(input, numIterations, 1.0, 1.0) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 43c3154dd053b..1c3bdceab1d14 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -24,8 +24,6 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel import org.apache.spark.mllib.util.{Loader, Saveable} -import org.apache.spark.rdd.RDD - /** * Regression model trained using RidgeRegression. @@ -100,113 +98,7 @@ class RidgeRegressionWithSGD private[mllib] ( .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - /** - * Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100, - * regParam: 0.01, miniBatchFraction: 1.0}. - */ - @Since("0.8.0") - @deprecated("Use ml.regression.LinearRegression with elasticNetParam = 0.0. Note the default " + - "regParam is 0.01 for RidgeRegressionWithSGD, but is 0.0 for LinearRegression.", "2.0.0") - def this() = this(1.0, 100, 0.01, 1.0) - override protected def createModel(weights: Vector, intercept: Double) = { new RidgeRegressionModel(weights, intercept) } } - -/** - * Top-level methods for calling RidgeRegression. - * - */ -@Since("0.8.0") -@deprecated("Use ml.regression.LinearRegression with elasticNetParam = 0.0. Note the default " + - "regParam is 0.01 for RidgeRegressionWithSGD, but is 0.0 for LinearRegression.", "2.0.0") -object RidgeRegressionWithSGD { - - /** - * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. The weights used - * in gradient descent are initialized using the initial weights provided. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - * - */ - @Since("1.0.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeights: Vector): RidgeRegressionModel = { - new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run( - input, initialWeights) - } - - /** - * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double): RidgeRegressionModel = { - new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) - } - - /** - * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to - * compute the true gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param regParam Regularization parameter. - * @param numIterations Number of iterations of gradient descent to run. - * @return a RidgeRegressionModel which has the weights and offset from training. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double): RidgeRegressionModel = { - train(input, numIterations, stepSize, regParam, 1.0) - } - - /** - * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to - * compute the true gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @return a RidgeRegressionModel which has the weights and offset from training. - * - */ - @Since("0.8.0") - def train( - input: RDD[LabeledPoint], - numIterations: Int): RidgeRegressionModel = { - train(input, numIterations, 1.0, 0.01, 1.0) - } -} diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index c04e2e69541ba..208a5aaa2bb15 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -50,11 +50,8 @@ public void runLRUsingConstructor() { List validationData = LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); - LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(); + LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(1.0, 100, 1.0, 1.0); lrImpl.setIntercept(true); - lrImpl.optimizer().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100); LogisticRegressionModel model = lrImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -72,8 +69,8 @@ public void runLRUsingStaticMethods() { List validationData = LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); - LogisticRegressionModel model = LogisticRegressionWithSGD.train( - testRDD.rdd(), 100, 1.0, 1.0); + LogisticRegressionModel model = new LogisticRegressionWithSGD(1.0, 100, 0.01, 1.0) + .run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 270e636f82117..a9a8b7f2b88d6 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -42,11 +42,11 @@ public void runKMeansUsingStaticMethods() { Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); JavaRDD data = jsc.parallelize(points, 2); - KMeansModel model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.K_MEANS_PARALLEL()); + KMeansModel model = KMeans.train(data.rdd(), 1, 1, KMeans.K_MEANS_PARALLEL()); assertEquals(1, model.clusterCenters().length); assertEquals(expectedCenter, model.clusterCenters()[0]); - model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.RANDOM()); + model = KMeans.train(data.rdd(), 1, 1, KMeans.RANDOM()); assertEquals(expectedCenter, model.clusterCenters()[0]); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index 1458cc72bc17f..35ad24bc2a84f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -51,10 +51,7 @@ public void runLassoUsingConstructor() { List validationData = LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); - LassoWithSGD lassoSGDImpl = new LassoWithSGD(); - lassoSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20); + LassoWithSGD lassoSGDImpl = new LassoWithSGD(1.0, 20, 0.01, 1.0); LassoModel model = lassoSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -72,7 +69,7 @@ public void runLassoUsingStaticMethods() { List validationData = LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); - LassoModel model = LassoWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); + LassoModel model = new LassoWithSGD(1.0, 100, 0.01, 1.0).run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 86c723aa00746..7e87588c4f0f6 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -33,7 +33,7 @@ private static int validatePrediction( List validationData, LinearRegressionModel model) { int numAccurate = 0; for (LabeledPoint point : validationData) { - Double prediction = model.predict(point.features()); + double prediction = model.predict(point.features()); // A prediction is off if the prediction is more than 0.5 away from expected value. if (Math.abs(prediction - point.label()) <= 0.5) { numAccurate++; @@ -53,7 +53,7 @@ public void runLinearRegressionUsingConstructor() { List validationData = LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); - LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0); linSGDImpl.setIntercept(true); LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); @@ -72,7 +72,8 @@ public void runLinearRegressionUsingStaticMethods() { List validationData = LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); - LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100); + LinearRegressionModel model = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0) + .run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); @@ -85,7 +86,7 @@ public void testPredictJavaRDD() { double[] weights = {10, 10}; JavaRDD testRDD = jsc.parallelize( LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); - LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0); LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); JavaRDD vectors = testRDD.map(LabeledPoint::features); JavaRDD predictions = model.predict(vectors); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index 5a9389c424b44..63441950cd18f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -34,7 +34,7 @@ private static double predictionError(List validationData, RidgeRegressionModel model) { double errorSum = 0; for (LabeledPoint point : validationData) { - Double prediction = model.predict(point.features()); + double prediction = model.predict(point.features()); errorSum += (prediction - point.label()) * (prediction - point.label()); } return errorSum / validationData.size(); @@ -60,11 +60,7 @@ public void runRidgeRegressionUsingConstructor() { new ArrayList<>(data.subList(0, numExamples))); List validationData = data.subList(numExamples, 2 * numExamples); - RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); - ridgeSGDImpl.optimizer() - .setStepSize(1.0) - .setRegParam(0.0) - .setNumIterations(200); + RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(1.0, 200, 0.0, 1.0); RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd()); double unRegularizedErr = predictionError(validationData, model); @@ -85,10 +81,12 @@ public void runRidgeRegressionUsingStaticMethods() { new ArrayList<>(data.subList(0, numExamples))); List validationData = data.subList(numExamples, 2 * numExamples); - RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); + RidgeRegressionModel model = new RidgeRegressionWithSGD(1.0, 200, 0.0, 1.0) + .run(testRDD.rdd()); double unRegularizedErr = predictionError(validationData, model); - model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.1); + model = new RidgeRegressionWithSGD(1.0, 200, 0.1, 1.0) + .run(testRDD.rdd()); double regularizedErr = predictionError(validationData, model); Assert.assertTrue(regularizedErr < unRegularizedErr); diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 5cf4377768516..126f0a23bfcb9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -224,12 +224,8 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegressionWithSGD().setIntercept(true) - lr.optimizer - .setStepSize(10.0) - .setRegParam(0.0) - .setNumIterations(20) - .setConvergenceTol(0.0005) + val lr = new LogisticRegressionWithSGD(10.0, 20, 0.0, 1.0).setIntercept(true) + lr.optimizer.setConvergenceTol(0.0005) val model = lr.run(testRDD) @@ -300,11 +296,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegressionWithSGD().setIntercept(true) - lr.optimizer - .setStepSize(10.0) - .setRegParam(0.0) - .setNumIterations(10) + val lr = new LogisticRegressionWithSGD(10.0, 10, 0.0, 1.0).setIntercept(true) val model = lr.run(testRDD, initialWeights) @@ -335,11 +327,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegressionWithSGD().setIntercept(true) - lr.optimizer. - setStepSize(1.0). - setNumIterations(10). - setRegParam(1.0) + val lr = new LogisticRegressionWithSGD(1.0, 10, 1.0, 1.0).setIntercept(true) val model = lr.run(testRDD, initialWeights) @@ -916,7 +904,7 @@ class LogisticRegressionClusterSuite extends SparkFunSuite with LocalClusterSpar }.cache() // If we serialize data directly in the task closure, the size of the serialized task would be // greater than 1MB and hence Spark would throw an error. - val model = LogisticRegressionWithSGD.train(points, 2) + val model = new LogisticRegressionWithSGD(1.0, 2, 0.0, 1.0).run(points) val predictions = model.predict(points.map(_.features)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index c4bf5b27187f6..149a525a58ff6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -367,7 +367,7 @@ class KMeansClusterSuite extends SparkFunSuite with LocalClusterSparkContext { for (initMode <- Seq(KMeans.RANDOM, KMeans.K_MEANS_PARALLEL)) { // If we serialize data directly in the task closure, the size of the serialized task would be // greater than 1MB and hence Spark would throw an error. - val model = KMeans.train(points, 2, 2, 1, initMode) + val model = KMeans.train(points, 2, 2, initMode) val predictions = model.predict(points).collect() val cost = model.computeCost(points) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index d96103d01e4ab..b4281d66ec1f8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -55,8 +55,7 @@ class LassoSuite extends SparkFunSuite with MLlibTestSparkContext { } val testRDD = sc.parallelize(testData, 2).cache() - val ls = new LassoWithSGD() - ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40) + val ls = new LassoWithSGD(1.0, 40, 0.01, 1.0) val model = ls.run(testRDD) val weight0 = model.weights(0) @@ -99,8 +98,8 @@ class LassoSuite extends SparkFunSuite with MLlibTestSparkContext { val testRDD = sc.parallelize(testData, 2).cache() - val ls = new LassoWithSGD() - ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40).setConvergenceTol(0.0005) + val ls = new LassoWithSGD(1.0, 40, 0.01, 1.0) + ls.optimizer.setConvergenceTol(0.0005) val model = ls.run(testRDD, initialWeights) val weight0 = model.weights(0) @@ -153,7 +152,7 @@ class LassoClusterSuite extends SparkFunSuite with LocalClusterSparkContext { }.cache() // If we serialize data directly in the task closure, the size of the serialized task would be // greater than 1MB and hence Spark would throw an error. - val model = LassoWithSGD.train(points, 2) + val model = new LassoWithSGD(1.0, 2, 0.01, 1.0).run(points) val predictions = model.predict(points.map(_.features)) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 0694079b9df9e..a9765b007b450 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -46,7 +46,7 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { test("linear regression") { val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 100, 42), 2).cache() - val linReg = new LinearRegressionWithSGD().setIntercept(true) + val linReg = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0).setIntercept(true) linReg.optimizer.setNumIterations(1000).setStepSize(1.0) val model = linReg.run(testRDD) @@ -72,7 +72,7 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { test("linear regression without intercept") { val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( 0.0, Array(10.0, 10.0), 100, 42), 2).cache() - val linReg = new LinearRegressionWithSGD().setIntercept(false) + val linReg = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0).setIntercept(false) linReg.optimizer.setNumIterations(1000).setStepSize(1.0) val model = linReg.run(testRDD) @@ -103,7 +103,7 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1)))) LabeledPoint(label, sv) }.cache() - val linReg = new LinearRegressionWithSGD().setIntercept(false) + val linReg = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0).setIntercept(false) linReg.optimizer.setNumIterations(1000).setStepSize(1.0) val model = linReg.run(sparseRDD) @@ -160,7 +160,7 @@ class LinearRegressionClusterSuite extends SparkFunSuite with LocalClusterSparkC }.cache() // If we serialize data directly in the task closure, the size of the serialized task would be // greater than 1MB and hence Spark would throw an error. - val model = LinearRegressionWithSGD.train(points, 2) + val model = new LinearRegressionWithSGD(1.0, 2, 0.0, 1.0).run(points) val predictions = model.predict(points.map(_.features)) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 815be32d2e510..2d6aec184ad9d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -60,18 +60,13 @@ class RidgeRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val validationRDD = sc.parallelize(validationData, 2).cache() // First run without regularization. - val linearReg = new LinearRegressionWithSGD() - linearReg.optimizer.setNumIterations(200) - .setStepSize(1.0) + val linearReg = new LinearRegressionWithSGD(1.0, 200, 0.0, 1.0) val linearModel = linearReg.run(testRDD) val linearErr = predictionError( linearModel.predict(validationRDD.map(_.features)).collect(), validationData) - val ridgeReg = new RidgeRegressionWithSGD() - ridgeReg.optimizer.setNumIterations(200) - .setRegParam(0.1) - .setStepSize(1.0) + val ridgeReg = new RidgeRegressionWithSGD(1.0, 200, 0.1, 1.0) val ridgeModel = ridgeReg.run(testRDD) val ridgeErr = predictionError( ridgeModel.predict(validationRDD.map(_.features)).collect(), validationData) @@ -110,7 +105,7 @@ class RidgeRegressionClusterSuite extends SparkFunSuite with LocalClusterSparkCo }.cache() // If we serialize data directly in the task closure, the size of the serialized task would be // greater than 1MB and hence Spark would throw an error. - val model = RidgeRegressionWithSGD.train(points, 2) + val model = new RidgeRegressionWithSGD(1.0, 2, 0.01, 1.0).run(points) val predictions = model.predict(points.map(_.features)) } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 04bec14c42c6f..097f1d2c2a6e1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,20 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( + // [SPARK-][SQL][CORE][MLLIB] Remove more old deprecated items in Spark 3 + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createExternalTable"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.train"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.clustering.KMeans.train"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.classification.LogisticRegressionWithSGD$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.classification.LogisticRegressionWithSGD.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.ChiSqSelectorModel.isSorted"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.regression.RidgeRegressionWithSGD$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.RidgeRegressionWithSGD.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.LassoWithSGD.this"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.regression.LassoWithSGD$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.regression.LinearRegressionWithSGD$"), + // [SPARK-28486][CORE][PYTHON] Map PythonBroadcast's data file to a BroadcastBlock to avoid delete by GC ProblemFilters.exclude[InaccessibleMethodProblem]("java.lang.Object.finalize"), diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 70c0b27a6aa33..76a5bd0d645ba 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -113,7 +113,7 @@ def wrapper(self, *args, **kwargs): # for back compatibility -from pyspark.sql import SQLContext, HiveContext, Row +from pyspark.sql import SQLContext, Row __all__ = [ "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", diff --git a/python/pyspark/ml/tests/test_image.py b/python/pyspark/ml/tests/test_image.py index 0008b0b670d34..5cc2a815eaa14 100644 --- a/python/pyspark/ml/tests/test_image.py +++ b/python/pyspark/ml/tests/test_image.py @@ -20,7 +20,7 @@ from pyspark.ml.image import ImageSchema from pyspark.testing.mlutils import PySparkTestCase, SparkSessionTestCase -from pyspark.sql import HiveContext, Row +from pyspark.sql import Row from pyspark.testing.utils import QuietTest @@ -67,47 +67,6 @@ def test_read_images(self): lambda: ImageSchema.toImage("a")) -class ImageFileFormatOnHiveContextTest(PySparkTestCase): - - @classmethod - def setUpClass(cls): - super(ImageFileFormatOnHiveContextTest, cls).setUpClass() - cls.hive_available = True - # Note that here we enable Hive's support. - cls.spark = None - try: - cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf() - except py4j.protocol.Py4JError: - cls.tearDownClass() - cls.hive_available = False - except TypeError: - cls.tearDownClass() - cls.hive_available = False - if cls.hive_available: - cls.spark = HiveContext._createForTesting(cls.sc) - - def setUp(self): - if not self.hive_available: - self.skipTest("Hive is not available.") - - @classmethod - def tearDownClass(cls): - super(ImageFileFormatOnHiveContextTest, cls).tearDownClass() - if cls.spark is not None: - cls.spark.sparkSession.stop() - cls.spark = None - - def test_read_images_multiple_times(self): - # This test case is to check if ImageFileFormat tries to - # initiate Hive client multiple times. See SPARK-22651. - data_path = 'data/mllib/images/origin/kittens' - for i in range(2): - self.spark.read.format("image") \ - .option("dropInvalid", True) \ - .option("recursiveFileLookup", True) \ - .load(data_path) - - if __name__ == "__main__": from pyspark.ml.tests.test_image import * diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index f220911993096..e41e5c9cc8e89 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -304,7 +304,7 @@ class KMeans(object): @classmethod @since('0.9.0') - def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||", + def train(cls, rdd, k, maxIterations=100, initializationMode="k-means||", seed=None, initializationSteps=2, epsilon=1e-4, initialModel=None): """ Train a k-means clustering model. @@ -317,8 +317,6 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" :param maxIterations: Maximum number of iterations allowed. (default: 100) - :param runs: - This param has no effect since Spark 2.0.0. :param initializationMode: The initialization algorithm. This can be either "random" or "k-means||". @@ -342,8 +340,6 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" rather than using the random or k-means|| initializationModel. (default: None) """ - if runs != 1: - warnings.warn("The param `runs` has no effect since Spark 2.0.0.") clusterInitialModel = [] if initialModel is not None: if not isinstance(initialModel, KMeansModel): @@ -351,7 +347,7 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" "to be of ") clusterInitialModel = [_convert_to_vector(c) for c in initialModel.clusterCenters] model = callMLlibFunc("trainKMeansModel", rdd.map(_convert_to_vector), k, maxIterations, - runs, initializationMode, seed, initializationSteps, epsilon, + initializationMode, seed, initializationSteps, epsilon, clusterInitialModel) centers = callJavaFunc(rdd.context, model.clusterCenters) return KMeansModel([c.toArray() for c in centers]) diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index c3c06c8124362..9c760e3527be4 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -43,7 +43,7 @@ from pyspark.sql.types import Row -from pyspark.sql.context import SQLContext, HiveContext, UDFRegistration +from pyspark.sql.context import SQLContext, UDFRegistration from pyspark.sql.session import SparkSession from pyspark.sql.column import Column from pyspark.sql.catalog import Catalog @@ -54,7 +54,7 @@ __all__ = [ - 'SparkSession', 'SQLContext', 'HiveContext', 'UDFRegistration', + 'SparkSession', 'SQLContext', 'UDFRegistration', 'DataFrame', 'GroupedData', 'Column', 'Catalog', 'Row', 'DataFrameNaFunctions', 'DataFrameStatFunctions', 'Window', 'WindowSpec', 'DataFrameReader', 'DataFrameWriter' diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py index 974251f63b37a..08cf6ee330785 100644 --- a/python/pyspark/sql/catalog.py +++ b/python/pyspark/sql/catalog.py @@ -138,26 +138,6 @@ def listColumns(self, tableName, dbName=None): isBucket=jcolumn.isBucket())) return columns - @since(2.0) - def createExternalTable(self, tableName, path=None, source=None, schema=None, **options): - """Creates a table based on the dataset in a data source. - - It returns the DataFrame associated with the external table. - - The data source is specified by the ``source`` and a set of ``options``. - If ``source`` is not specified, the default data source configured by - ``spark.sql.sources.default`` will be used. - - Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and - created external table. - - :return: :class:`DataFrame` - """ - warnings.warn( - "createExternalTable is deprecated since Spark 2.2, please use createTable instead.", - DeprecationWarning) - return self.createTable(tableName, path, source, schema, **options) - @since(2.2) def createTable(self, tableName, path=None, source=None, schema=None, **options): """Creates a table based on the dataset in a data source. diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 48a49c583f9c7..e6bd2eb31a844 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -32,7 +32,7 @@ from pyspark.sql.udf import UDFRegistration from pyspark.sql.utils import install_exception_handler -__all__ = ["SQLContext", "HiveContext"] +__all__ = ["SQLContext"] class SQLContext(object): @@ -325,24 +325,6 @@ def dropTempTable(self, tableName): """ self.sparkSession.catalog.dropTempView(tableName) - @since(1.3) - def createExternalTable(self, tableName, path=None, source=None, schema=None, **options): - """Creates an external table based on the dataset in a data source. - - It returns the DataFrame associated with the external table. - - The data source is specified by the ``source`` and a set of ``options``. - If ``source`` is not specified, the default data source configured by - ``spark.sql.sources.default`` will be used. - - Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and - created external table. - - :return: :class:`DataFrame` - """ - return self.sparkSession.catalog.createExternalTable( - tableName, path, source, schema, **options) - @ignore_unicode_prefix @since(1.0) def sql(self, sqlQuery): @@ -466,53 +448,6 @@ def streams(self): return StreamingQueryManager(self._ssql_ctx.streams()) -class HiveContext(SQLContext): - """A variant of Spark SQL that integrates with data stored in Hive. - - Configuration for Hive is read from ``hive-site.xml`` on the classpath. - It supports running both SQL and HiveQL commands. - - :param sparkContext: The SparkContext to wrap. - :param jhiveContext: An optional JVM Scala HiveContext. If set, we do not instantiate a new - :class:`HiveContext` in the JVM, instead we make all calls to this object. - - .. note:: Deprecated in 2.0.0. Use SparkSession.builder.enableHiveSupport().getOrCreate(). - """ - - def __init__(self, sparkContext, jhiveContext=None): - warnings.warn( - "HiveContext is deprecated in Spark 2.0.0. Please use " + - "SparkSession.builder.enableHiveSupport().getOrCreate() instead.", - DeprecationWarning) - if jhiveContext is None: - sparkContext._conf.set("spark.sql.catalogImplementation", "hive") - sparkSession = SparkSession.builder._sparkContext(sparkContext).getOrCreate() - else: - sparkSession = SparkSession(sparkContext, jhiveContext.sparkSession()) - SQLContext.__init__(self, sparkContext, sparkSession, jhiveContext) - - @classmethod - def _createForTesting(cls, sparkContext): - """(Internal use only) Create a new HiveContext for testing. - - All test code that touches HiveContext *must* go through this method. Otherwise, - you may end up launching multiple derby instances and encounter with incredibly - confusing error messages. - """ - jsc = sparkContext._jsc.sc() - jtestHive = sparkContext._jvm.org.apache.spark.sql.hive.test.TestHiveContext(jsc, False) - return cls(sparkContext, jtestHive) - - def refreshTable(self, tableName): - """Invalidate and refresh all the cached the metadata of the given - table. For performance reasons, Spark SQL or the external data source - library it uses might cache certain metadata about a table, such as the - location of blocks. When those change outside of Spark SQL, users should - call this function to invalidate the cache. - """ - self._ssql_ctx.refreshTable(tableName) - - def _test(): import os import doctest diff --git a/python/pyspark/sql/tests/test_appsubmit.py b/python/pyspark/sql/tests/test_appsubmit.py deleted file mode 100644 index 99c0317cc45df..0000000000000 --- a/python/pyspark/sql/tests/test_appsubmit.py +++ /dev/null @@ -1,97 +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. -# - -import os -import subprocess -import tempfile - -import py4j - -from pyspark import SparkContext -from pyspark.tests.test_appsubmit import SparkSubmitTests - - -class HiveSparkSubmitTests(SparkSubmitTests): - - @classmethod - def setUpClass(cls): - # get a SparkContext to check for availability of Hive - sc = SparkContext('local[4]', cls.__name__) - cls.hive_available = True - try: - sc._jvm.org.apache.hadoop.hive.conf.HiveConf() - except py4j.protocol.Py4JError: - cls.hive_available = False - except TypeError: - cls.hive_available = False - finally: - # we don't need this SparkContext for the test - sc.stop() - - def setUp(self): - super(HiveSparkSubmitTests, self).setUp() - if not self.hive_available: - self.skipTest("Hive is not available.") - - def test_hivecontext(self): - # This test checks that HiveContext is using Hive metastore (SPARK-16224). - # It sets a metastore url and checks if there is a derby dir created by - # Hive metastore. If this derby dir exists, HiveContext is using - # Hive metastore. - metastore_path = os.path.join(tempfile.mkdtemp(), "spark16224_metastore_db") - metastore_URL = "jdbc:derby:;databaseName=" + metastore_path + ";create=true" - hive_site_dir = os.path.join(self.programDir, "conf") - hive_site_file = self.createTempFile("hive-site.xml", (""" - | - | - | javax.jdo.option.ConnectionURL - | %s - | - | - """ % metastore_URL).lstrip(), "conf") - script = self.createTempFile("test.py", """ - |import os - | - |from pyspark.conf import SparkConf - |from pyspark.context import SparkContext - |from pyspark.sql import HiveContext - | - |conf = SparkConf() - |sc = SparkContext(conf=conf) - |hive_context = HiveContext(sc) - |print(hive_context.sql("show databases").collect()) - """) - proc = subprocess.Popen( - self.sparkSubmit + ["--master", "local-cluster[1,1,1024]", - "--driver-class-path", hive_site_dir, script], - stdout=subprocess.PIPE) - out, err = proc.communicate() - self.assertEqual(0, proc.returncode) - self.assertIn("default", out.decode('utf-8')) - self.assertTrue(os.path.exists(metastore_path)) - - -if __name__ == "__main__": - import unittest - from pyspark.sql.tests.test_appsubmit import * - - try: - import xmlrunner - testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) - except ImportError: - testRunner = None - unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/test_context.py b/python/pyspark/sql/tests/test_context.py index b2df3921e62a9..d57ebc48cf5e6 100644 --- a/python/pyspark/sql/tests/test_context.py +++ b/python/pyspark/sql/tests/test_context.py @@ -27,7 +27,7 @@ import py4j -from pyspark import HiveContext, Row +from pyspark.sql import Row, SparkSession from pyspark.sql.types import * from pyspark.sql.window import Window from pyspark.testing.utils import ReusedPySparkTestCase @@ -40,15 +40,20 @@ def setUpClass(cls): ReusedPySparkTestCase.setUpClass() cls.tempdir = tempfile.NamedTemporaryFile(delete=False) cls.hive_available = True + cls.spark = None try: cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf() except py4j.protocol.Py4JError: + cls.tearDownClass() cls.hive_available = False except TypeError: + cls.tearDownClass() cls.hive_available = False + if cls.hive_available: + cls.spark = SparkSession.builder.enableHiveSupport().getOrCreate() + os.unlink(cls.tempdir.name) if cls.hive_available: - cls.spark = HiveContext._createForTesting(cls.sc) cls.testData = [Row(key=i, value=str(i)) for i in range(100)] cls.df = cls.sc.parallelize(cls.testData).toDF() @@ -60,13 +65,16 @@ def setUp(self): def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() shutil.rmtree(cls.tempdir.name, ignore_errors=True) + if cls.spark is not None: + cls.spark.stop() + cls.spark = None def test_save_and_load_table(self): df = self.df tmpPath = tempfile.mkdtemp() shutil.rmtree(tmpPath) df.write.saveAsTable("savedJsonTable", "json", "append", path=tmpPath) - actual = self.spark.createExternalTable("externalJsonTable", tmpPath, "json") + actual = self.spark.catalog.createTable("externalJsonTable", tmpPath, "json") self.assertEqual(sorted(df.collect()), sorted(self.spark.sql("SELECT * FROM savedJsonTable").collect())) self.assertEqual(sorted(df.collect()), @@ -76,7 +84,7 @@ def test_save_and_load_table(self): df.write.saveAsTable("savedJsonTable", "json", "overwrite", path=tmpPath) schema = StructType([StructField("value", StringType(), True)]) - actual = self.spark.createExternalTable("externalJsonTable", source="json", + actual = self.spark.catalog.createTable("externalJsonTable", source="json", schema=schema, path=tmpPath, noUse="this options will not be used") self.assertEqual(sorted(df.collect()), @@ -87,11 +95,11 @@ def test_save_and_load_table(self): self.spark.sql("DROP TABLE savedJsonTable") self.spark.sql("DROP TABLE externalJsonTable") - defaultDataSourceName = self.spark.getConf("spark.sql.sources.default", - "org.apache.spark.sql.parquet") + defaultDataSourceName = self.spark.conf.get("spark.sql.sources.default", + "org.apache.spark.sql.parquet") self.spark.sql("SET spark.sql.sources.default=org.apache.spark.sql.json") df.write.saveAsTable("savedJsonTable", path=tmpPath, mode="overwrite") - actual = self.spark.createExternalTable("externalJsonTable", path=tmpPath) + actual = self.spark.catalog.createTable("externalJsonTable", path=tmpPath) self.assertEqual(sorted(df.collect()), sorted(self.spark.sql("SELECT * FROM savedJsonTable").collect())) self.assertEqual(sorted(df.collect()), diff --git a/python/pyspark/streaming/kinesis.py b/python/pyspark/streaming/kinesis.py index 4ed9f2a40c3a7..729ec97505aad 100644 --- a/python/pyspark/streaming/kinesis.py +++ b/python/pyspark/streaming/kinesis.py @@ -80,7 +80,6 @@ def createStream(ssc, kinesisAppName, streamName, endpointUrl, regionName, jduration = ssc._jduration(checkpointInterval) try: - # Use KinesisUtilsPythonHelper to access Scala's KinesisUtils helper = ssc._jvm.org.apache.spark.streaming.kinesis.KinesisUtilsPythonHelper() except TypeError as e: if str(e) == "'JavaPackage' object is not callable": diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 61c027ec4483a..45627e9d3f1f8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -74,7 +74,7 @@ private[spark] class ClientArguments(args: Array[String]) { s""" |Usage: org.apache.spark.deploy.yarn.Client [options] |Options: - | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster + | --jar JAR_PATH Path to your application's JAR file (required in YARN cluster | mode) | --class CLASS_NAME Name of your application's main class (required) | --primary-py-file A main Python file diff --git a/sql/README.md b/sql/README.md index f0ea848a41d09..67e3225e2c275 100644 --- a/sql/README.md +++ b/sql/README.md @@ -6,7 +6,7 @@ This module provides support for executing relational queries expressed in eithe Spark SQL is broken up into four subprojects: - Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions. - Execution (sql/core) - A query planner / execution engine for translating Catalyst's logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files. - - Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allow users to run queries that include Hive UDFs, UDAFs, and UDTFs. + - Hive Support (sql/hive) - Includes extensions that allow users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allow users to run queries that include Hive UDFs, UDAFs, and UDTFs. - HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server. Running `./sql/create-docs.sh` generates SQL documentation for built-in functions under `sql/site`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 45d0bd4122535..2054874e5e07b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -479,97 +479,6 @@ class SQLContext private[sql](val sparkSession: SparkSession) def readStream: DataStreamReader = sparkSession.readStream - /** - * Creates an external table from the given path and returns the corresponding DataFrame. - * It will use the default data source configured by spark.sql.sources.default. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable(tableName: String, path: String): DataFrame = { - sparkSession.catalog.createTable(tableName, path) - } - - /** - * Creates an external table from the given path based on a data source - * and returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - path: String, - source: String): DataFrame = { - sparkSession.catalog.createTable(tableName, path, source) - } - - /** - * Creates an external table from the given path based on a data source and a set of options. - * Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - source: String, - options: java.util.Map[String, String]): DataFrame = { - sparkSession.catalog.createTable(tableName, source, options) - } - - /** - * (Scala-specific) - * Creates an external table from the given path based on a data source and a set of options. - * Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - source: String, - options: Map[String, String]): DataFrame = { - sparkSession.catalog.createTable(tableName, source, options) - } - - /** - * Create an external table from the given path based on a data source, a schema and - * a set of options. Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - source: String, - schema: StructType, - options: java.util.Map[String, String]): DataFrame = { - sparkSession.catalog.createTable(tableName, source, schema, options) - } - - /** - * (Scala-specific) - * Create an external table from the given path based on a data source, a schema and - * a set of options. Then, returns the corresponding DataFrame. - * - * @group ddl_ops - * @since 1.3.0 - */ - @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - source: String, - schema: StructType, - options: Map[String, String]): DataFrame = { - sparkSession.catalog.createTable(tableName, source, schema, options) - } - /** * Registers the given `DataFrame` as a temporary table in the catalog. Temporary tables exist * only during the lifetime of this instance of SQLContext. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 60738e6d4ef9e..318cc629e7a34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalog import scala.collection.JavaConverters._ -import org.apache.spark.annotation.{Evolving, Experimental, Stable} +import org.apache.spark.annotation.Stable import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset} import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel @@ -208,20 +208,6 @@ abstract class Catalog { */ def functionExists(dbName: String, functionName: String): Boolean - /** - * Creates a table from the given path and returns the corresponding DataFrame. - * It will use the default data source configured by spark.sql.sources.default. - * - * @param tableName is either a qualified or unqualified name that designates a table. - * If no database identifier is provided, it refers to a table in - * the current database. - * @since 2.0.0 - */ - @deprecated("use createTable instead.", "2.2.0") - def createExternalTable(tableName: String, path: String): DataFrame = { - createTable(tableName, path) - } - /** * Creates a table from the given path and returns the corresponding DataFrame. * It will use the default data source configured by spark.sql.sources.default. @@ -233,20 +219,6 @@ abstract class Catalog { */ def createTable(tableName: String, path: String): DataFrame - /** - * Creates a table from the given path based on a data source and returns the corresponding - * DataFrame. - * - * @param tableName is either a qualified or unqualified name that designates a table. - * If no database identifier is provided, it refers to a table in - * the current database. - * @since 2.0.0 - */ - @deprecated("use createTable instead.", "2.2.0") - def createExternalTable(tableName: String, path: String, source: String): DataFrame = { - createTable(tableName, path, source) - } - /** * Creates a table from the given path based on a data source and returns the corresponding * DataFrame. @@ -258,23 +230,6 @@ abstract class Catalog { */ def createTable(tableName: String, path: String, source: String): DataFrame - /** - * Creates a table from the given path based on a data source and a set of options. - * Then, returns the corresponding DataFrame. - * - * @param tableName is either a qualified or unqualified name that designates a table. - * If no database identifier is provided, it refers to a table in - * the current database. - * @since 2.0.0 - */ - @deprecated("use createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - source: String, - options: java.util.Map[String, String]): DataFrame = { - createTable(tableName, source, options) - } - /** * Creates a table based on the dataset in a data source and a set of options. * Then, returns the corresponding DataFrame. @@ -291,24 +246,6 @@ abstract class Catalog { createTable(tableName, source, options.asScala.toMap) } - /** - * (Scala-specific) - * Creates a table from the given path based on a data source and a set of options. - * Then, returns the corresponding DataFrame. - * - * @param tableName is either a qualified or unqualified name that designates a table. - * If no database identifier is provided, it refers to a table in - * the current database. - * @since 2.0.0 - */ - @deprecated("use createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - source: String, - options: Map[String, String]): DataFrame = { - createTable(tableName, source, options) - } - /** * (Scala-specific) * Creates a table based on the dataset in a data source and a set of options. @@ -324,24 +261,6 @@ abstract class Catalog { source: String, options: Map[String, String]): DataFrame - /** - * Create a table from the given path based on a data source, a schema and a set of options. - * Then, returns the corresponding DataFrame. - * - * @param tableName is either a qualified or unqualified name that designates a table. - * If no database identifier is provided, it refers to a table in - * the current database. - * @since 2.0.0 - */ - @deprecated("use createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - source: String, - schema: StructType, - options: java.util.Map[String, String]): DataFrame = { - createTable(tableName, source, schema, options) - } - /** * Create a table based on the dataset in a data source, a schema and a set of options. * Then, returns the corresponding DataFrame. @@ -359,25 +278,6 @@ abstract class Catalog { createTable(tableName, source, schema, options.asScala.toMap) } - /** - * (Scala-specific) - * Create a table from the given path based on a data source, a schema and a set of options. - * Then, returns the corresponding DataFrame. - * - * @param tableName is either a qualified or unqualified name that designates a table. - * If no database identifier is provided, it refers to a table in - * the current database. - * @since 2.0.0 - */ - @deprecated("use createTable instead.", "2.2.0") - def createExternalTable( - tableName: String, - source: String, - schema: StructType, - options: Map[String, String]): DataFrame = { - createTable(tableName, source, schema, options) - } - /** * (Scala-specific) * Create a table based on the dataset in a data source, a schema and a set of options. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala deleted file mode 100644 index 02a5117f005e8..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ /dev/null @@ -1,63 +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.sql.hive - -import org.apache.spark.SparkContext -import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.internal.Logging -import org.apache.spark.sql.{SparkSession, SQLContext} - - -/** - * An instance of the Spark SQL execution engine that integrates with data stored in Hive. - * Configuration for Hive is read from hive-site.xml on the classpath. - */ -@deprecated("Use SparkSession.builder.enableHiveSupport instead", "2.0.0") -class HiveContext private[hive](_sparkSession: SparkSession) - extends SQLContext(_sparkSession) with Logging { - - self => - - def this(sc: SparkContext) = { - this(SparkSession.builder().sparkContext(HiveUtils.withHiveExternalCatalog(sc)).getOrCreate()) - } - - def this(sc: JavaSparkContext) = this(sc.sc) - - /** - * Returns a new HiveContext as new session, which will have separated SQLConf, UDF/UDAF, - * temporary tables and SessionState, but sharing the same CacheManager, IsolatedClientLoader - * and Hive client (both of execution and metadata) with existing HiveContext. - */ - override def newSession(): HiveContext = { - new HiveContext(sparkSession.newSession()) - } - - /** - * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, - * Spark SQL or the external data source library it uses might cache certain metadata about a - * table, such as the location of blocks. When those change outside of Spark SQL, users should - * call this function to invalidate the cache. - * - * @since 1.3.0 - */ - def refreshTable(tableName: String): Unit = { - sparkSession.catalog.refreshTable(tableName) - } - -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala index db074361ef03c..14276c9b583f2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala @@ -23,8 +23,5 @@ package org.apache.spark.sql * - Using HiveQL to express queries. * - Reading metadata from the Hive Metastore using HiveSerDes. * - Hive UDFs, UDAs, UDTs - * - * Users that would like access to this functionality should create a - * [[hive.HiveContext HiveContext]] instead of a [[SQLContext]]. */ package object hive diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index 25bd4d0017bd8..5304052b45a41 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -38,9 +38,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.hive.test.TestHive$; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.catalyst.TableIdentifier; import org.apache.spark.util.Utils; @@ -94,57 +91,6 @@ public void tearDown() throws IOException { } } - @Test - public void saveExternalTableAndQueryIt() { - Map options = new HashMap<>(); - options.put("path", path.toString()); - df.write() - .format("org.apache.spark.sql.json") - .mode(SaveMode.Append) - .options(options) - .saveAsTable("javaSavedTable"); - - checkAnswer( - sqlContext.sql("SELECT * FROM javaSavedTable"), - df.collectAsList()); - - Dataset loadedDF = - sqlContext.createExternalTable("externalTable", "org.apache.spark.sql.json", options); - - checkAnswer(loadedDF, df.collectAsList()); - checkAnswer( - sqlContext.sql("SELECT * FROM externalTable"), - df.collectAsList()); - } - - @Test - public void saveExternalTableWithSchemaAndQueryIt() { - Map options = new HashMap<>(); - options.put("path", path.toString()); - df.write() - .format("org.apache.spark.sql.json") - .mode(SaveMode.Append) - .options(options) - .saveAsTable("javaSavedTable"); - - checkAnswer( - sqlContext.sql("SELECT * FROM javaSavedTable"), - df.collectAsList()); - - List fields = new ArrayList<>(); - fields.add(DataTypes.createStructField("b", DataTypes.StringType, true)); - StructType schema = DataTypes.createStructType(fields); - Dataset loadedDF = - sqlContext.createExternalTable("externalTable", "org.apache.spark.sql.json", schema, options); - - checkAnswer( - loadedDF, - sqlContext.sql("SELECT b FROM javaSavedTable").collectAsList()); - checkAnswer( - sqlContext.sql("SELECT * FROM externalTable"), - sqlContext.sql("SELECT b FROM javaSavedTable").collectAsList()); - } - @Test public void saveTableAndQueryIt() { Map options = new HashMap<>(); diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index fcf0b4591ff84..4cbc03d05c9e3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -216,7 +216,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto // Drop the table and create it again. sql("DROP TABLE refreshTable") - sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet") + sparkSession.catalog.createTable("refreshTable", tempPath.toString, "parquet") // It is not cached. assert(!isCached("refreshTable"), "refreshTable should not be cached.") // Refresh the table. REFRESH TABLE command should not make a uncached @@ -237,7 +237,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto tempPath.delete() table("src").write.mode(SaveMode.Overwrite).parquet(tempPath.toString) sql("DROP TABLE IF EXISTS refreshTable") - sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet") + sparkSession.catalog.createTable("refreshTable", tempPath.toString, "parquet") checkAnswer( table("refreshTable"), table("src").collect()) @@ -256,7 +256,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto // Drop the table and create it again. sql("DROP TABLE refreshTable") - sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet") + sparkSession.catalog.createTable("refreshTable", tempPath.toString, "parquet") // It is not cached. assert(!isCached("refreshTable"), "refreshTable should not be cached.") // Refresh the table. REFRESH command should not make a uncached diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala deleted file mode 100644 index a80db765846e9..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala +++ /dev/null @@ -1,103 +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.sql.hive - -import org.scalatest.BeforeAndAfterEach - -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} - - -class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEach { - - override protected val enableAutoThreadAudit = false - private var sc: SparkContext = null - private var hc: HiveContext = null - - override def beforeAll(): Unit = { - super.beforeAll() - sc = SparkContext.getOrCreate(new SparkConf().setMaster("local").setAppName("test")) - HiveUtils.newTemporaryConfiguration(useInMemoryDerby = true).foreach { case (k, v) => - sc.hadoopConfiguration.set(k, v) - } - hc = new HiveContext(sc) - } - - override def afterEach(): Unit = { - try { - hc.sharedState.cacheManager.clearCache() - hc.sessionState.catalog.reset() - } finally { - super.afterEach() - } - } - - override def afterAll(): Unit = { - try { - sc = null - hc = null - } finally { - super.afterAll() - } - } - - test("basic operations") { - val _hc = hc - import _hc.implicits._ - val df1 = (1 to 20).map { i => (i, i) }.toDF("a", "x") - val df2 = (1 to 100).map { i => (i, i % 10, i % 2 == 0) }.toDF("a", "b", "c") - .select($"a", $"b") - .filter($"a" > 10 && $"b" > 6 && $"c") - val df3 = df1.join(df2, "a") - val res = df3.collect() - val expected = Seq((18, 18, 8)).toDF("a", "x", "b").collect() - assert(res.toSeq == expected.toSeq) - df3.createOrReplaceTempView("mai_table") - val df4 = hc.table("mai_table") - val res2 = df4.collect() - assert(res2.toSeq == expected.toSeq) - } - - test("basic DDLs") { - val _hc = hc - import _hc.implicits._ - val databases = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) - assert(databases.toSeq == Seq("default")) - hc.sql("CREATE DATABASE mee_db") - hc.sql("USE mee_db") - val databases2 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) - assert(databases2.toSet == Set("default", "mee_db")) - val df = (1 to 10).map { i => ("bob" + i.toString, i) }.toDF("name", "age") - df.createOrReplaceTempView("mee_table") - hc.sql("CREATE TABLE moo_table (name string, age int)") - hc.sql("INSERT INTO moo_table SELECT * FROM mee_table") - assert( - hc.sql("SELECT * FROM moo_table order by name").collect().toSeq == - df.collect().toSeq.sortBy(_.getString(0))) - val tables = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0)) - assert(tables.toSet == Set("moo_table", "mee_table")) - hc.sql("DROP TABLE moo_table") - hc.sql("DROP TABLE mee_table") - val tables2 = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0)) - assert(tables2.isEmpty) - hc.sql("USE default") - hc.sql("DROP DATABASE mee_db CASCADE") - val databases3 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) - assert(databases3.toSeq == Seq("default")) - } - -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index ba807fb58fe40..41a26344f7c21 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -516,13 +516,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") { - sparkSession.catalog.createExternalTable("createdJsonTable", tempPath.toString) + sparkSession.catalog.createTable("createdJsonTable", tempPath.toString) assert(table("createdJsonTable").schema === df.schema) checkAnswer(sql("SELECT * FROM createdJsonTable"), df) assert( intercept[AnalysisException] { - sparkSession.catalog.createExternalTable("createdJsonTable", jsonFilePath.toString) + sparkSession.catalog.createTable("createdJsonTable", jsonFilePath.toString) }.getMessage.contains("Table createdJsonTable already exists."), "We should complain that createdJsonTable already exists") } @@ -534,7 +534,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // Try to specify the schema. withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source name") { val schema = StructType(StructField("b", StringType, true) :: Nil) - sparkSession.catalog.createExternalTable( + sparkSession.catalog.createTable( "createdJsonTable", "org.apache.spark.sql.json", schema, @@ -553,7 +553,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("path required error") { assert( intercept[AnalysisException] { - sparkSession.catalog.createExternalTable( + sparkSession.catalog.createTable( "createdJsonTable", "org.apache.spark.sql.json", Map.empty[String, String]) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 9060ce2e0eb4b..2d3e462531245 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -66,14 +66,14 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle } } - test(s"createExternalTable() to non-default database - with USE") { + test(s"createTable() to non-default database - with USE") { withTempDatabase { db => activateDatabase(db) { withTempPath { dir => val path = dir.getCanonicalPath df.write.format("parquet").mode(SaveMode.Overwrite).save(path) - spark.catalog.createExternalTable("t", path, "parquet") + spark.catalog.createTable("t", path, "parquet") assert(getTableNames(Option(db)).contains("t")) checkAnswer(spark.table("t"), df) @@ -92,12 +92,12 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle } } - test(s"createExternalTable() to non-default database - without USE") { + test(s"createTable() to non-default database - without USE") { withTempDatabase { db => withTempPath { dir => val path = dir.getCanonicalPath df.write.format("parquet").mode(SaveMode.Overwrite).save(path) - spark.catalog.createExternalTable(s"$db.t", path, "parquet") + spark.catalog.createTable(s"$db.t", path, "parquet") assert(getTableNames(Option(db)).contains("t")) checkAnswer(spark.table(s"$db.t"), df) 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 cd8e2eaa2b4dc..c6e4407c80e83 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 @@ -1749,7 +1749,7 @@ class HiveDDLSuite test("create hive serde table with Catalog") { withTable("t") { withTempDir { dir => - val df = spark.catalog.createExternalTable( + val df = spark.catalog.createTable( "t", "hive", new StructType().add("i", "int"), From c839d09789fa62944a628c04cf4e11057adea53c Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Mon, 9 Sep 2019 13:41:17 -0700 Subject: [PATCH 0021/1387] [SPARK-28773][DOC][SQL] Handling of NULL data in Spark SQL ### What changes were proposed in this pull request? Document ```NULL``` semantics in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on how `NULL` data is handled in various expressions and operators. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2019-09-08 at 11 24 41 PM Screen Shot 2019-09-08 at 11 24 56 PM Screen Shot 2019-09-08 at 11 25 08 PM Screen Shot 2019-09-08 at 11 25 24 PM Screen Shot 2019-09-08 at 11 25 34 PM Screen Shot 2019-09-08 at 11 25 49 PM Screen Shot 2019-09-08 at 11 26 00 PM Screen Shot 2019-09-08 at 11 26 09 PM Screen Shot 2019-09-08 at 11 26 20 PM Screen Shot 2019-09-08 at 11 26 32 PM Screen Shot 2019-09-08 at 11 26 47 PM Screen Shot 2019-09-08 at 11 26 59 PM Screen Shot 2019-09-08 at 11 27 10 PM Screen Shot 2019-09-08 at 11 27 18 PM Screen Shot 2019-09-08 at 11 27 27 PM Screen Shot 2019-09-08 at 11 27 37 PM Screen Shot 2019-09-08 at 11 27 48 PM Screen Shot 2019-09-08 at 11 27 59 PM Screen Shot 2019-09-08 at 11 28 08 PM Screen Shot 2019-09-08 at 11 28 20 PM ### How was this patch tested? Tested using jykyll build --serve Closes #25726 from dilipbiswal/sql-ref-null-data. Lead-authored-by: Dilip Biswal Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- docs/_data/menu-sql.yaml | 2 + docs/sql-ref-null-semantics.md | 703 +++++++++++++++++++++++++++++++++ 2 files changed, 705 insertions(+) create mode 100644 docs/sql-ref-null-semantics.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 717911b5a4645..258762f0d9506 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -78,6 +78,8 @@ subitems: - text: Data Types url: sql-ref-datatypes.html + - text: Null Semantics + url: sql-ref-null-semantics.html - text: NaN Semantics url: sql-ref-nan-semantics.html - text: SQL Syntax diff --git a/docs/sql-ref-null-semantics.md b/docs/sql-ref-null-semantics.md new file mode 100644 index 0000000000000..a67b3993a31c0 --- /dev/null +++ b/docs/sql-ref-null-semantics.md @@ -0,0 +1,703 @@ +--- +layout: global +title: NULL Semantics +displayTitle: NULL Semantics +license: | + 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. +--- + +### Description +A table consists of a set of rows and each row contains a set of columns. +A column is associated with a data type and represents +a specific attribute of an entity (for example, `age` is a column of an +entity called `person`). Sometimes, the value of a column +specific to a row is not known at the time the row comes into existence. +In `SQL`, such values are represnted as `NULL`. This section details the +semantics of `NULL` values handling in various operators, expressions and +other `SQL` constructs. + +1. [Null handling in comparison operators](#comp-operators) +2. [Null handling in Logical operators](#logical-operators) +3. [Null handling in Expressions](#expressions) + 1. [Null handling in null-in-tolerant expressions](#null-in-tolerant) + 2. [Null handling Expressions that can process null value operands](#can-process-null) + 3. [Null handling in built-in aggregate expressions](#built-in-aggregate) +4. [Null handling in WHERE, HAVING and JOIN conditions](#condition-expressions) +5. [Null handling in GROUP BY and DISTINCT](#aggregate-operator) +6. [Null handling in ORDER BY](#order-by) +7. [Null handling in UNION, INTERSECT, EXCEPT](#set-operators) +8. [Null handling in EXISTS and NOT EXISTS subquery](#exists-not-exists) +9. [Null handling in IN and NOT IN subquery](#in-not-in) + + + +The following illustrates the schema layout and data of a table named `person`. The data contains `NULL` values in +the `age` column and this table will be used in various examples in the sections below. +**TABLE: person** + + + + + + + + + +
IdNameAge
100Joe30
200MarryNULL
300Mike18
400Fred50
500AlbertNULL
600Michelle30
700Dan50
+ +### Comparision operators + +Apache spark supports the standard comparison operators such as '>', '>=', '=', '<' and '<='. +The result of these operators is unknown or `NULL` when one of the operarands or both the operands are +unknown or `NULL`. In order to compare the `NULL` values for equality, Spark provides a null-safe +equal operator ('<=>'), which returns `False` when one of the operand is `NULL` and returns 'True` when +both the operands are `NULL`. The following table illustrates the behaviour of comparison operators when +one or both operands are `NULL`: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Left OperandRight Operand>>==<<=<=>
NULLAny valueNULLNULLNULLNULLNULLFalse
Any valueNULLNULLNULLNULLNULLNULLFalse
NULLNULLNULLNULLNULLNULLNULLTrue
+ +### Examples +{% highlight sql %} +-- Normal comparison operators return `NULL` when one of the operand is `NULL`. +SELECT 5 > null AS expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |null | + +-----------------+ + +-- Normal comparison operators return `NULL` when both the operands are `NULL`. +SELECT null = null AS expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |null | + +-----------------+ + +-- Null-safe equal operator return `False` when one of the operand is `NULL` +SELECT 5 <=> null AS expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |false | + +-----------------+ + +-- Null-safe equal operator return `True` when one of the operand is `NULL` +SELECT NULL <=> NULL; + +-----------------+ + |expression_output| + +-----------------+ + |true | + +-----------------+ +{% endhighlight %} + +### Logical operators +Spark supports standard logical operators such as `AND`, `OR` and `NOT`. These operators take `Boolean` expressions +as the arguments and return a `Boolean` value. + +The following tables illustrate the behavior of logical opeators when one or both operands are `NULL`. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Left OperandRight OperandORAND
TrueNULLTrueNULL
FalseNULLNULLFalse
NULLTrueTrueNULL
NULLFalseNULLNULL
NULLNULLNULLNULL
+
+ + + + + + + + + +
operandNOT
NULLNULL
+ +### Examples +{% highlight sql %} +-- Normal comparison operators return `NULL` when one of the operands is `NULL`. +SELECT (true OR null) AS expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |true | + +-----------------+ + +-- Normal comparison operators return `NULL` when both the operands are `NULL`. +SELECT (null OR false) AS expression_output + +-----------------+ + |expression_output| + +-----------------+ + |null | + +-----------------+ + +-- Null-safe equal operator returns `False` when one of the operands is `NULL` +SELECT NOT(null) AS expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |null | + +-----------------+ +{% endhighlight %} + +### Expressions +The comparison operators and logical operators are treated as expressions in +Spark. Other than these two kinds of expressions, Spark supports other form of +expressions such as function expressions, cast expressions, etc. The expressions +in Spark can be broadly classified as : +- Null in-tolerent expressions +- Expressions that can process `NULL` value operands + - The result of these expressions depends on the expression itself. + +#### Null in-tolerant expressions +Null in-tolerant expressions return `NULL` when one or more arguments of +expression are `NULL` and most of the expressions fall in this category. + +##### Examples +{% highlight sql %} +SELECT concat('John', null) as expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |null | + +-----------------+ + +SELECT positive(null) as expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |null | + +-----------------+ + +SELECT to_date(null) as expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |null | + +-----------------+ +{% endhighlight %} + +#### Expressions that can process null value operands. + +This class of expressions are designed to handle `NULL` values. The result of the +expressions depends on the expression itself. As an example, function expression `isnull` +returns a `true` on null input and `false` on non null input where as function `coalesce` +returns the first non `NULL` value in its list of operands. However, `coalesce` returns +`NULL` when all its operands are `NULL`. Below is an incomplete list of expressions of this category. + - COALESCE + - NULLIF + - IFNULL + - NVL + - NVL2 + - ISNAN + - NANVL + - ISNULL + - ISNOTNULL + - ATLEASTNNONNULLS + - IN + + +##### Examples +{% highlight sql %} +SELECT isnull(null) AS expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |true | + +-----------------+ + +-- Returns the first occurence of non `NULL` value. +SELECT coalesce(null, null, 3, null) AS expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |3 | + +-----------------+ + +-- Returns `NULL` as all its operands are `NULL`. +SELECT coalesce(null, null, null, null) AS expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |null | + +-----------------+ + +SELECT isnan(null) as expression_output; + +-----------------+ + |expression_output| + +-----------------+ + |false | + +-----------------+ +{% endhighlight %} + +#### Builtin Aggregate Expressions +Aggregate functions compute a single result by processing a set of input rows. Below are +the rules of how `NULL` values are handled by aggregate functions. +- `NULL` values are ignored from processing by all the aggregate functions. + - Only exception to this rule is COUNT(*) function. +- Some aggregate functions return `NULL` when all input values are `NULL` or the input data set + is empty.
The list of these functions is: + - MAX + - MIN + - SUM + - AVG + - EVERY + - ANY + - SOME + +#### Examples +{% highlight sql %} +-- `count(*)` does not skip `NULL` values. +SELECT count(*) FROM person; + +--------+ + |count(1)| + +--------+ + |7 | + +--------+ + +-- `NULL` values in column `age` are skipped from processing. +SELECT count(age) FROM person; + +----------+ + |count(age)| + +----------+ + |5 | + +----------+ + +-- `count(*)` on an empty input set returns 0. This is unlike the other +-- aggregate functions, such as `max`, which return `NULL`. +SELECT count(*) FROM person where 1 = 0; + +--------+ + |count(1)| + +--------+ + |0 | + +--------+ + +-- `NULL` values are excluded from computation of maximum value. +SELECT max(age) FROM person; + +--------+ + |max(age)| + +--------+ + |50 | + +--------+ + +-- `max` returns `NULL` on an empty input set. +SELECT max(age) FROM person where 1 = 0; + +--------+ + |max(age)| + +--------+ + |null | + +--------+ + +{% endhighlight %} + +### Condition expressions in WHERE, HAVING and JOIN clauses. +`WHERE`, `HAVING` operators filter rows based on the user specified condition. +A `JOIN` operator is used to combine rows from two tables based on a join condition. +For all the three operators, a condition expression is a boolean expression and can return + True, False or Unknown (NULL). They are "satisfied" if the result of the condition is `True`. + +#### Examples +{% highlight sql %} +-- Persons whose age is unknown (`NULL`) are filtered out from the result set. +SELECT * FROM person WHERE age > 0; + +--------+---+ + |name |age| + +--------+---+ + |Michelle|30 | + |Fred |50 | + |Mike |18 | + |Dan |50 | + |Joe |30 | + +--------+---+ + +-- `IS NULL` expression is used in disjunction to select the persons +-- with unknown (`NULL`) records. +SELECT * FROM person WHERE age > 0 OR age IS NULL; + +--------+----+ + |name |age | + +--------+----+ + |Albert |null| + |Michelle|30 | + |Fred |50 | + |Mike |18 | + |Dan |50 | + |Marry |null| + |Joe |30 | + +--------+----+ + +-- Person with unknown(`NULL`) ages are skipped from processing. +SELECT * FROM person GROUP BY age HAVING max(age) > 18; + +---+--------+ + |age|count(1)| + +---+--------+ + |50 |2 | + |30 |2 | + +---+--------+ + +-- A self join case with a join condition `p1.age = p2.age AND p1.name = p2.name`. +-- The persons with unknown age (`NULL`) are filtered out by the join operator. +SELECT * FROM person p1, person p2 +WHERE p1.age = p2.age + AND p1.name = p2.name; + +--------+---+--------+---+ + |name |age|name |age| + +--------+---+--------+---+ + |Michelle|30 |Michelle|30 | + |Fred |50 |Fred |50 | + |Mike |18 |Mike |18 | + |Dan |50 |Dan |50 | + |Joe |30 |Joe |30 | + +--------+---+--------+---+ + +-- The age column from both legs of join are compared using null-safe equal which +-- is why the persons with unknown age (`NULL`) are qualified by the join. +SELECT * FROM person p1, person p2 +WHERE p1.age <=> p2.age + AND p1.name = p2.name; ++--------+----+--------+----+ +| name| age| name| age| ++--------+----+--------+----+ +| Albert|null| Albert|null| +|Michelle| 30|Michelle| 30| +| Fred| 50| Fred| 50| +| Mike| 18| Mike| 18| +| Dan| 50| Dan| 50| +| Marry|null| Marry|null| +| Joe| 30| Joe| 30| ++--------+----+--------+----+ + +{% endhighlight %} + +### Aggregate operator (GROUP BY, DISTINCT) +As discussed in the previous section [comparison operator](sql-ref-null-semantics.html#comparision-operators), +two `NULL` values are not equal. However, for the purpose of grouping and distinct processing, the two or more +values with `NULL data`are grouped together into the same bucket. This behaviour is conformant with SQL +standard and with other enterprise database management systems. + +#### Examples +{% highlight sql %} +-- `NULL` values are put in one bucket in `GROUP BY` processing. +SELECT age, count(*) FROM person GROUP BY age; + +----+--------+ + |age |count(1)| + +----+--------+ + |null|2 | + |50 |2 | + |30 |2 | + |18 |1 | + +----+--------+ + +-- All `NULL` ages are considered one distinct value in `DISTINCT` processing. +SELECT DISTINCT age FROM person; + +----+ + |age | + +----+ + |null| + |50 | + |30 | + |18 | + +----+ + +{% endhighlight %} + +### Sort operator (ORDER BY Clause) +Spark SQL supports null ordering specification in `ORDER BY` clause. Spark processes the `ORDER BY` clause by +placing all the `NULL` values at first or at last depending on the null ordering specification. By default, all +the `NULL` values are placed at first. + +#### Examples +{% highlight sql %} +-- `NULL` values are shown at first and other values +-- are sorted in ascending way. +SELECT age, name FROM person ORDER BY age; + +----+--------+ + |age |name | + +----+--------+ + |null|Marry | + |null|Albert | + |18 |Mike | + |30 |Michelle| + |30 |Joe | + |50 |Fred | + |50 |Dan | + +----+--------+ + +-- Column values other than `NULL` are sorted in ascending +-- way and `NULL` values are shown at the last. +SELECT age, name FROM person ORDER BY age NULLS LAST; + +----+--------+ + |age |name | + +----+--------+ + |18 |Mike | + |30 |Michelle| + |30 |Joe | + |50 |Dan | + |50 |Fred | + |null|Marry | + |null|Albert | + +----+--------+ + +-- Columns other than `NULL` values are sorted in descending +-- and `NULL` values are shown at the last. +SELECT age, name FROM person ORDER BY age DESC NULLS LAST; + +----+--------+ + |age |name | + +----+--------+ + |50 |Fred | + |50 |Dan | + |30 |Michelle| + |30 |Joe | + |18 |Mike | + |null|Marry | + |null|Albert | + +----+--------+ +{% endhighlight %} + +### Set operators (UNION, INTERSECT, EXCEPT) +`NULL` values are compared in a null-safe manner for equality in the context of +set operations. That means when comparing rows, two `NULL` values are considered +equal unlike the regular `EqualTo`(`=`) operator. + +#### Examples +{% highlight sql %} +CREATE VIEW unknown_age SELECT * FROM person WHERE age IS NULL; + +-- Only common rows between two legs of `INTERSECT` are in the +-- result set. The comparison between columns of the row are done +-- in a null-safe manner. +SELECT name, age FROM person +INTERSECT +SELECT name, age from unknown_age; + +------+----+ + |name |age | + +------+----+ + |Albert|null| + |Marry |null| + +------+----+ + +-- `NULL` values from two legs of the `EXCEPT` are not in output. +-- This basically shows that the comparison happens in a null-safe manner. +SELECT age, name FROM person +EXCEPT +SELECT age FROM unknown_age; + +---+--------+ + |age|name | + +---+--------+ + |30 |Joe | + |50 |Fred | + |30 |Michelle| + |18 |Mike | + |50 |Dan | + +---+--------+ + +-- Performs `UNION` operation between two sets of data. +-- The comparison between columns of the row ae done in +-- null-safe manner. +SELECT name, age FROM person +UNION +SELECT name, age FROM unknown_age; + +--------+----+ + |name |age | + +--------+----+ + |Albert |null| + |Joe |30 | + |Michelle|30 | + |Marry |null| + |Fred |50 | + |Mike |18 | + |Dan |50 | + +--------+----+ +{% endhighlight %} + + +### EXISTS/NOT EXISTS Subquery +In Spark, EXISTS and NOT EXISTS expressions are allowed inside a WHERE clause. +These are boolean expressions which return either `TRUE` or +`FALSE`. In otherwords, EXISTS is a membership condition and returns `TRUE` +when the subquery it refers to returns one or more rows. Similary, NOT EXISTS +is a non-membership condition and returns TRUE when no rows or zero rows are +returned from the subquery. + +These two expressions are not affected by presence of NULL in the result of +the subquery. + +#### Examples +{% highlight sql %} +-- Even if subquery produces rows with `NULL` values, the `EXISTS` expression +-- evaluates to `TRUE` as the subquery produces 1 row. +SELECT * FROM person WHERE EXISTS (SELECT null); + +--------+----+ + |name |age | + +--------+----+ + |Albert |null| + |Michelle|30 | + |Fred |50 | + |Mike |18 | + |Dan |50 | + |Marry |null| + |Joe |30 | + +--------+----+ + +-- `NOT EXISTS` expression returns `FALSE`. It returns `TRUE` only when +-- subquery produces no rows. In this case, it returns 1 row. +SELECT * FROM person WHERE NOT EXISTS (SELECT null); + +----+---+ + |name|age| + +----+---+ + +----+---+ + +-- `NOT EXISTS` expression returns `TRUE`. +SELECT * FROM person WHERE NOT EXISTS (SELECT 1 WHERE 1 = 0); + +--------+----+ + |name |age | + +--------+----+ + |Albert |null| + |Michelle|30 | + |Fred |50 | + |Mike |18 | + |Dan |50 | + |Marry |null| + |Joe |30 | + +--------+----+ +{% endhighlight %} + +### IN/NOT IN Subquery +In Spark, `IN` and `NOT IN` expressions are allowed inside a WHERE clause of +a query. Unlike the `EXISTS` expression, `IN` expression can return a `TRUE`, +`FALSE` or `UNKNOWN (NULL)` value. Conceptually a `IN` expression is semantically +equivalent to a set of equality condition separated by a disjunctive operator (`OR`). +For example, c1 IN (1, 2, 3) is semantically equivalent to `(C1 = 1 OR c1 = 2 OR c1 = 3)`. + +As far as handling `NULL` values are concerned, the semantics can be deduced from +the `NULL` value handling in comparison operators(`=`) and logical operators(`OR`). +To summarize, below are the rules for computing the result of an `IN` expression. + +- TRUE is returned when the non-NULL value in question is found in the list +- FALSE is returned when the non-NULL value is not found in the list and the + list does not contain NULL values +- UNKNOWN is returned when the value is `NULL`, or the non-NULL value is not found in the list + and the list contains at least one `NULL` value + +#### Examples +{% highlight sql %} +-- The subquery has only `NULL` value in its result set. Therefore, +-- the result of `IN` predicate is UNKNOWN. +SELECT * FROM person WHERE age IN (SELECT null); + +----+---+ + |name|age| + +----+---+ + +----+---+ + +-- The subquery has `NULL` value in the result set as well as a valid +-- value `50`. Rows with age = 50 are returned. +SELECT * FROM person +WHERE age IN (SELECT age FROM VALUES (50), (null) sub(age)); + +----+---+ + |name|age| + +----+---+ + |Fred|50 | + |Dan |50 | + +----+---+ + +-- Since subquery has `NULL` value in the result set, the `NOT IN` +-- predicate would return UNKNOWN. Hence, no rows are +-- qualified for this query. +SELECT * FROM person +WHERE age NOT IN (SELECT age FROM VALUES (50), (null) sub(age)); + +----+---+ + |name|age| + +----+---+ + +----+---+ + +{% endhighlight %} From 125af78d3232a9d28a4198e72d04e1deaa1acdd1 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 9 Sep 2019 14:28:55 -0700 Subject: [PATCH 0022/1387] [SPARK-28831][DOC][SQL] Document CLEAR CACHE statement in SQL Reference ### What changes were proposed in this pull request? Document CLEAR CACHE statement in SQL Reference ### Why are the changes needed? To complete SQL Reference ### Does this PR introduce any user-facing change? Yes After change: ![image](https://user-images.githubusercontent.com/13592258/64565512-caf89a80-d308-11e9-99ea-88e966d1b1a1.png) ### How was this patch tested? Tested using jykyll build --serve Closes #25541 from huaxingao/spark-28831-n. Authored-by: Huaxin Gao Signed-off-by: Xiao Li --- docs/sql-ref-syntax-aux-cache-clear-cache.md | 18 +++++++++++++++++- docs/sql-ref-syntax-aux-cache-uncache-table.md | 4 ++-- docs/sql-ref-syntax-aux-cache.md | 15 +++++++-------- 3 files changed, 26 insertions(+), 11 deletions(-) diff --git a/docs/sql-ref-syntax-aux-cache-clear-cache.md b/docs/sql-ref-syntax-aux-cache-clear-cache.md index 88d126f0f528e..d8e451a230a71 100644 --- a/docs/sql-ref-syntax-aux-cache-clear-cache.md +++ b/docs/sql-ref-syntax-aux-cache-clear-cache.md @@ -19,4 +19,20 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`CLEAR CACHE` removes the entries and associated data from the in-memory and/or on-disk cache for all cached tables and views. + +### Syntax +{% highlight sql %} +CLEAR CACHE +{% endhighlight %} + +### Examples +{% highlight sql %} +CLEAR CACHE; +{% endhighlight %} + +### Related Statements + * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) + * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) + diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md index 69819fee088da..a6cb4d6807b22 100644 --- a/docs/sql-ref-syntax-aux-cache-uncache-table.md +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -20,7 +20,7 @@ license: | --- ### Description -`UNCACHE TABLE` removes the entries and associated data from the in-memory and/or on-disk cache for a given table. The +`UNCACHE TABLE` removes the entries and associated data from the in-memory and/or on-disk cache for a given table or view. The underlying entries should already have been brought to cache by previous `CACHE TABLE` operation. `UNCACHE TABLE` on a non-existent table throws Exception if `IF EXISTS` is not specified. ### Syntax {% highlight sql %} @@ -29,7 +29,7 @@ UNCACHE TABLE [ IF EXISTS ] table_name ### Parameters
table_name
-
The name of the table to be uncached.
+
The name of the table or view to be uncached.
### Examples {% highlight sql %} diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md index eb0e73d00e848..c3dcb276a7e0f 100644 --- a/docs/sql-ref-syntax-aux-cache.md +++ b/docs/sql-ref-syntax-aux-cache.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Cache +displayTitle: Cache license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -9,9 +9,9 @@ license: | 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. @@ -19,7 +19,6 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +* [CACHE TABLE statement](sql-ref-syntax-aux-cache-cache-table.html) +* [UNCACHE TABLE statement](sql-ref-syntax-aux-cache-uncache-table.html) +* [CLEAR CACHE statement](sql-ref-syntax-aux-cache-clear-cache.html) From 8018ded21794ca3a97204eb7019a8770a663b49d Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 9 Sep 2019 15:36:36 -0700 Subject: [PATCH 0023/1387] [SPARK-28214][STREAMING][TESTS] CheckpointSuite: wait for batch to be fully processed before accessing DStreamCheckpointData ### What changes were proposed in this pull request? This patch fixes the bug regarding accessing `DStreamCheckpointData.currentCheckpointFiles` without guarding which makes the test `basic rdd checkpoints + dstream graph checkpoint recovery` being flaky. There're two possible points to make test failing: 1. checkpoint logic is too slow so that checkpoint cannot be handled within real delay 2. There's multithreads-unsafe point in `DStreamCheckpointData.update`: it clears `currentCheckpointFiles` and adds new checkpointFiles. Race condition can happen between main thread for test and JobGenerator's event loop thread. `lastProcessedBatch` guarantees that all events for given time are processed, as commented: `// last batch whose completion,checkpointing and metadata cleanup has been completed`. That means, if we wait for time for exactly same amount as advanced the time in test (multiply of checkpoint interval as well as batch duration) we can expect nothing will happen in DStreamCheckpointData afterwards unless we advance the clock. This patch applies the observation above. ### Why are the changes needed? The test is reported as flaky as [SPARK-28214](https://issues.apache.org/jira/browse/SPARK-28214), and the test code seems unsafe. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Modified UT. I've added some debug messages and confirmed no method in DStreamCheckpointData is being called between "after waiting lastProcessedBatch" and "advancing clock" even I added huge amount of sleep between twos, which avoids race-condition. I was also able to make existing test artificially failing (not 100% consistently but high likely) via adding sleep between `currentCheckpointFiles.clear()` and `currentCheckpointFiles ++= checkpointFiles` in `DStreamCheckpointData.update`, and confirmed modified test doesn't fail the test multiple times. Closes #25731 from HeartSaVioR/SPARK-28214. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../streaming/scheduler/JobGenerator.scala | 2 +- .../streaming/scheduler/JobScheduler.scala | 3 +- .../spark/streaming/CheckpointSuite.scala | 32 +++++++++++++------ 3 files changed, 26 insertions(+), 11 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index ddeb3d4547c55..77f36436acc1d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -77,7 +77,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { private var eventLoop: EventLoop[JobGeneratorEvent] = null // last batch whose completion,checkpointing and metadata cleanup has been completed - private var lastProcessedBatch: Time = null + @volatile private[streaming] var lastProcessedBatch: Time = null /** Start generation of jobs */ def start(): Unit = synchronized { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 2fa3bf7d5230b..68594e8977cf8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -52,8 +52,9 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private val numConcurrentJobs = ssc.conf.getInt("spark.streaming.concurrentJobs", 1) private val jobExecutor = ThreadUtils.newDaemonFixedThreadPool(numConcurrentJobs, "streaming-job-executor") - private val jobGenerator = new JobGenerator(this) + private[streaming] val jobGenerator = new JobGenerator(this) val clock = jobGenerator.clock + val listenerBus = new StreamingListenerBus(ssc.sparkContext.listenerBus) // These two are created only when scheduler starts. diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index ff5e3ffa35d79..607295730bb9e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -241,8 +241,8 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester val stateStreamCheckpointInterval = Seconds(1) val fs = FileSystem.getLocal(new Configuration()) // this ensure checkpointing occurs at least once - val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2 - val secondNumBatches = firstNumBatches + val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong + val secondNumBatches = firstNumBatches * 2 // Setup the streams val input = (1 to 10).map(_ => Seq("a")).toSeq @@ -258,14 +258,25 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester var ssc = setupStreams(input, operation) var stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head + def waitForCompletionOfBatch(numBatches: Long): Unit = { + eventually(timeout(10.seconds), interval(50.millis)) { + val lastProcessed = ssc.scheduler.jobGenerator.lastProcessedBatch + assert(lastProcessed != null && + lastProcessed >= Time(batchDuration.milliseconds * numBatches)) + } + } + // Run till a time such that at least one RDD in the stream should have been checkpointed, // then check whether some RDD has been checkpointed or not ssc.start() advanceTimeWithRealDelay(ssc, firstNumBatches) + waitForCompletionOfBatch(firstNumBatches) + logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData) - assert(!stateStream.checkpointData.currentCheckpointFiles.isEmpty, + var currCheckpointFiles = stateStream.checkpointData.currentCheckpointFiles + assert(!currCheckpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure") - stateStream.checkpointData.currentCheckpointFiles.foreach { + currCheckpointFiles.foreach { case (time, file) => assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist") @@ -273,8 +284,10 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester // Run till a further time such that previous checkpoint files in the stream would be deleted // and check whether the earlier checkpoint files are deleted - val checkpointFiles = stateStream.checkpointData.currentCheckpointFiles.map(x => new File(x._2)) - advanceTimeWithRealDelay(ssc, secondNumBatches) + currCheckpointFiles = stateStream.checkpointData.currentCheckpointFiles + val checkpointFiles = currCheckpointFiles.map(x => new File(x._2)) + advanceTimeWithRealDelay(ssc, secondNumBatches - firstNumBatches) + waitForCompletionOfBatch(secondNumBatches) checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted")) ssc.stop() @@ -287,14 +300,15 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from first failure") - // Run one batch to generate a new checkpoint file and check whether some RDD // is present in the checkpoint data or not ssc.start() advanceTimeWithRealDelay(ssc, 1) - assert(!stateStream.checkpointData.currentCheckpointFiles.isEmpty, + waitForCompletionOfBatch(secondNumBatches + 1) + currCheckpointFiles = stateStream.checkpointData.currentCheckpointFiles + assert(!currCheckpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure") - stateStream.checkpointData.currentCheckpointFiles.foreach { + currCheckpointFiles.foreach { case (time, file) => assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time + " for state stream before seconds failure does not exist") From e516f7e09e2bfb064376370b00fcf7a0ca91218f Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Mon, 9 Sep 2019 15:41:51 -0700 Subject: [PATCH 0024/1387] [SPARK-28928][SS] Use Kafka delegation token protocol on sources/sinks ### What changes were proposed in this pull request? At the moment there are 3 places where communication protocol with Kafka cluster has to be set when delegation token used: * On delegation token * On source * On sink Most of the time users are using the same protocol on all these places (within one Kafka cluster). It would be better to declare it in one place (delegation token side) and Kafka sources/sinks can take this config over. In this PR I've I've modified the code in a way that Kafka sources/sinks are taking over delegation token side `security.protocol` configuration when the token and the source/sink matches in `bootstrap.servers` configuration. This default configuration can be overwritten on each source/sink independently by using `kafka.security.protocol` configuration. ### Why are the changes needed? The actual configuration's default behavior represents the minority of the use-cases and inconvenient. ### Does this PR introduce any user-facing change? Yes, with this change users need to provide less configuration parameters by default. ### How was this patch tested? Existing + additional unit tests. Closes #25631 from gaborgsomogyi/SPARK-28928. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../structured-streaming-kafka-integration.md | 4 ++- .../kafka010/KafkaDelegationTokenSuite.scala | 2 -- .../spark/kafka010/KafkaConfigUpdater.scala | 1 + .../spark/kafka010/KafkaTokenSparkConf.scala | 4 ++- .../kafka010/KafkaConfigUpdaterSuite.scala | 26 ++++++++++++++++++- 5 files changed, 32 insertions(+), 5 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index c4378b4a02663..8cb2a45c56b21 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -825,7 +825,9 @@ Delegation tokens can be obtained from multiple clusters and ${cluster}spark.kafka.clusters.${cluster}.security.protocol SASL_SSL - Protocol used to communicate with brokers. For further details please see Kafka documentation. Only used to obtain delegation token. + Protocol used to communicate with brokers. For further details please see Kafka documentation. Protocol is applied on all the sources and sinks as default where + bootstrap.servers config matches (for further details please see spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex), + and can be overridden by setting kafka.security.protocol on the source or sink. diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala index 9850a91f34f63..306483825ae3b 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDelegationTokenSuite.scala @@ -82,7 +82,6 @@ class KafkaDelegationTokenSuite extends StreamTest with SharedSparkSession with .format("kafka") .option("checkpointLocation", checkpointDir.getCanonicalPath) .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.security.protocol", SASL_PLAINTEXT.name) .option("topic", topic) .start() @@ -99,7 +98,6 @@ class KafkaDelegationTokenSuite extends StreamTest with SharedSparkSession with val streamingDf = spark.readStream .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.security.protocol", SASL_PLAINTEXT.name) .option("startingOffsets", s"earliest") .option("subscribe", topic) .load() diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala index 0c61045d6d487..209b0d9035e7e 100644 --- a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala @@ -70,6 +70,7 @@ private[spark] case class KafkaConfigUpdater(module: String, kafkaParams: Map[St map.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG).asInstanceOf[String]) clusterConfig.foreach { clusterConf => logDebug("Delegation token detected, using it for login.") + setIfUnset(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, clusterConf.securityProtocol) val jaasParams = KafkaTokenUtil.getTokenJaasParams(clusterConf) set(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) require(clusterConf.tokenMechanism.startsWith("SCRAM"), diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala index e1f3c800a51f8..ed4a6f1e34c55 100644 --- a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala @@ -57,6 +57,7 @@ private [kafka010] object KafkaTokenSparkConf extends Logging { val CLUSTERS_CONFIG_PREFIX = "spark.kafka.clusters." val DEFAULT_TARGET_SERVERS_REGEX = ".*" val DEFAULT_SASL_KERBEROS_SERVICE_NAME = "kafka" + val DEFAULT_SECURITY_PROTOCOL_CONFIG = SASL_SSL.name val DEFAULT_SASL_TOKEN_MECHANISM = "SCRAM-SHA-512" def getClusterConfig(sparkConf: SparkConf, identifier: String): KafkaTokenClusterConf = { @@ -72,7 +73,8 @@ private [kafka010] object KafkaTokenSparkConf extends Logging { s"${configPrefix}auth.${CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG}")), sparkClusterConf.getOrElse(s"target.${CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG}.regex", KafkaTokenSparkConf.DEFAULT_TARGET_SERVERS_REGEX), - sparkClusterConf.getOrElse(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SASL_SSL.name), + sparkClusterConf.getOrElse(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, + DEFAULT_SECURITY_PROTOCOL_CONFIG), sparkClusterConf.getOrElse(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME), sparkClusterConf.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG), diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaConfigUpdaterSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaConfigUpdaterSuite.scala index 7a172892e778c..31815e93353d3 100644 --- a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaConfigUpdaterSuite.scala +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaConfigUpdaterSuite.scala @@ -17,8 +17,11 @@ package org.apache.spark.kafka010 +import java.{util => ju} + import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.config.SaslConfigs +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL} import org.apache.spark.SparkFunSuite @@ -76,6 +79,26 @@ class KafkaConfigUpdaterSuite extends SparkFunSuite with KafkaDelegationTokenTes val params = Map( CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers ) + testWithTokenSetValues(params) { updatedParams => + assert(updatedParams.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) === + KafkaTokenSparkConf.DEFAULT_SECURITY_PROTOCOL_CONFIG) + } + } + + test("setAuthenticationConfigIfNeeded with token should not override user-defined protocol") { + val overrideProtocolName = SASL_PLAINTEXT.name + val params = Map( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers, + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG -> overrideProtocolName + ) + testWithTokenSetValues(params) { updatedParams => + assert(updatedParams.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) === + overrideProtocolName) + } + } + + def testWithTokenSetValues(params: Map[String, String]) + (validate: (ju.Map[String, Object]) => Unit) { setSparkEnv( Map( s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers @@ -87,11 +110,12 @@ class KafkaConfigUpdaterSuite extends SparkFunSuite with KafkaDelegationTokenTes .setAuthenticationConfigIfNeeded() .build() - assert(updatedParams.size() === 3) + assert(updatedParams.size() === 4) assert(updatedParams.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) === bootStrapServers) assert(updatedParams.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) assert(updatedParams.get(SaslConfigs.SASL_MECHANISM) === KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM) + validate(updatedParams) } test("setAuthenticationConfigIfNeeded with invalid mechanism should throw exception") { From bdc1598a4321bbf8ff1bfebb28aaf83000c2e26a Mon Sep 17 00:00:00 2001 From: hongdd Date: Mon, 9 Sep 2019 18:02:52 -0500 Subject: [PATCH 0025/1387] [SPARK-28657][CORE] Fix currentContext Instance failed sometimes ## What changes were proposed in this pull request? Running spark on yarn, I got ``` java.lang.ClassCastException: org.apache.hadoop.ipc.CallerContext$Builder cannot be cast to scala.runtime.Nothing$ ``` Utils.classForName return Class[Nothing], I think it should be defind as Class[_] to resolve this issue ## How was this patch tested? not need Closes #25389 from hddong/SPARK-28657-fix-currentContext-Instance-failed. Lead-authored-by: hongdd Co-authored-by: hongdongdong Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/util/Utils.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 9c1f21fa236ba..5cd937aeb8ea4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -3033,7 +3033,8 @@ private[spark] class CallerContext( if (CallerContext.callerContextSupported) { try { val callerContext = Utils.classForName("org.apache.hadoop.ipc.CallerContext") - val builder = Utils.classForName("org.apache.hadoop.ipc.CallerContext$Builder") + val builder: Class[AnyRef] = + Utils.classForName("org.apache.hadoop.ipc.CallerContext$Builder") val builderInst = builder.getConstructor(classOf[String]).newInstance(context) val hdfsContext = builder.getMethod("build").invoke(builderInst) callerContext.getMethod("setCurrent", callerContext).invoke(null, hdfsContext) From aafce7ebffe1acd8f6022f208beaa9ec6c9f7592 Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Tue, 10 Sep 2019 08:16:18 +0900 Subject: [PATCH 0026/1387] [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array ## What changes were proposed in this pull request? This is a ANSI SQL and feature id is `T312` ``` ::= OVERLAY PLACING FROM [ FOR ] ``` This PR related to https://github.com/apache/spark/pull/24918 and support treat byte array. ref: https://www.postgresql.org/docs/11/functions-binarystring.html ## How was this patch tested? new UT. There are some show of the PR on my production environment. ``` spark-sql> select overlay(encode('Spark SQL', 'utf-8') PLACING encode('_', 'utf-8') FROM 6); Spark_SQL Time taken: 0.285 s spark-sql> select overlay(encode('Spark SQL', 'utf-8') PLACING encode('CORE', 'utf-8') FROM 7); Spark CORE Time taken: 0.202 s spark-sql> select overlay(encode('Spark SQL', 'utf-8') PLACING encode('ANSI ', 'utf-8') FROM 7 FOR 0); Spark ANSI SQL Time taken: 0.165 s spark-sql> select overlay(encode('Spark SQL', 'utf-8') PLACING encode('tructured', 'utf-8') FROM 2 FOR 4); Structured SQL Time taken: 0.141 s ``` Closes #25172 from beliefer/ansi-overlay-byte-array. Lead-authored-by: gengjiaan Co-authored-by: Jiaan Geng Signed-off-by: Takeshi Yamamuro --- .../expressions/stringExpressions.scala | 60 +++++++++++++--- .../expressions/StringExpressionsSuite.scala | 72 ++++++++++++++++++- .../org/apache/spark/sql/functions.scala | 16 ++--- .../spark/sql/StringFunctionsSuite.scala | 33 +++++++-- 4 files changed, 157 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index d7a5fb27a3d56..e4847e9cec3f0 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -472,6 +472,19 @@ object Overlay { builder.append(input.substringSQL(pos + length, Int.MaxValue)) builder.build() } + + def calculate(input: Array[Byte], replace: Array[Byte], pos: Int, len: Int): Array[Byte] = { + // If you specify length, it must be a positive whole number or zero. + // Otherwise it will be ignored. + // The default value for length is the length of replace. + val length = if (len >= 0) { + len + } else { + replace.length + } + ByteArray.concat(ByteArray.subStringSQL(input, 1, pos - 1), + replace, ByteArray.subStringSQL(input, pos + length, Int.MaxValue)) + } } // scalastyle:off line.size.limit @@ -487,6 +500,14 @@ object Overlay { Spark ANSI SQL > SELECT _FUNC_('Spark SQL' PLACING 'tructured' FROM 2 FOR 4); Structured SQL + > SELECT _FUNC_(encode('Spark SQL', 'utf-8') PLACING encode('_', 'utf-8') FROM 6); + Spark_SQL + > SELECT _FUNC_(encode('Spark SQL', 'utf-8') PLACING encode('CORE', 'utf-8') FROM 7); + Spark CORE + > SELECT _FUNC_(encode('Spark SQL', 'utf-8') PLACING encode('ANSI ', 'utf-8') FROM 7 FOR 0); + Spark ANSI SQL + > SELECT _FUNC_(encode('Spark SQL', 'utf-8') PLACING encode('tructured', 'utf-8') FROM 2 FOR 4); + Structured SQL """) // scalastyle:on line.size.limit case class Overlay(input: Expression, replace: Expression, pos: Expression, len: Expression) @@ -496,19 +517,42 @@ case class Overlay(input: Expression, replace: Expression, pos: Expression, len: this(str, replace, pos, Literal.create(-1, IntegerType)) } - override def dataType: DataType = StringType + override def dataType: DataType = input.dataType - override def inputTypes: Seq[AbstractDataType] = - Seq(StringType, StringType, IntegerType, IntegerType) + override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(StringType, BinaryType), + TypeCollection(StringType, BinaryType), IntegerType, IntegerType) override def children: Seq[Expression] = input :: replace :: pos :: len :: Nil + override def checkInputDataTypes(): TypeCheckResult = { + val inputTypeCheck = super.checkInputDataTypes() + if (inputTypeCheck.isSuccess) { + TypeUtils.checkForSameTypeInputExpr( + input.dataType :: replace.dataType :: Nil, s"function $prettyName") + } else { + inputTypeCheck + } + } + + private lazy val replaceFunc = input.dataType match { + case StringType => + (inputEval: Any, replaceEval: Any, posEval: Int, lenEval: Int) => { + Overlay.calculate( + inputEval.asInstanceOf[UTF8String], + replaceEval.asInstanceOf[UTF8String], + posEval, lenEval) + } + case BinaryType => + (inputEval: Any, replaceEval: Any, posEval: Int, lenEval: Int) => { + Overlay.calculate( + inputEval.asInstanceOf[Array[Byte]], + replaceEval.asInstanceOf[Array[Byte]], + posEval, lenEval) + } + } + override def nullSafeEval(inputEval: Any, replaceEval: Any, posEval: Any, lenEval: Any): Any = { - val inputStr = inputEval.asInstanceOf[UTF8String] - val replaceStr = replaceEval.asInstanceOf[UTF8String] - val position = posEval.asInstanceOf[Int] - val length = lenEval.asInstanceOf[Int] - Overlay.calculate(inputStr, replaceStr, position, length) + replaceFunc(inputEval, replaceEval, posEval.asInstanceOf[Int], lenEval.asInstanceOf[Int]) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 1b5acf4b0abcc..4308f98d6969a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -18,6 +18,7 @@ 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._ @@ -428,7 +429,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // scalastyle:on } - test("overlay") { + test("overlay for string") { checkEvaluation(new Overlay(Literal("Spark SQL"), Literal("_"), Literal.create(6, IntegerType)), "Spark_SQL") checkEvaluation(new Overlay(Literal("Spark SQL"), Literal("CORE"), @@ -450,6 +451,75 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(new Overlay(Literal("Spark的SQL"), Literal("_"), Literal.create(6, IntegerType)), "Spark_SQL") // scalastyle:on + // position greater than the length of input string + checkEvaluation(new Overlay(Literal("Spark SQL"), Literal("_"), + Literal.create(10, IntegerType)), "Spark SQL_") + checkEvaluation(Overlay(Literal("Spark SQL"), Literal("_"), + Literal.create(10, IntegerType), Literal.create(4, IntegerType)), "Spark SQL_") + // position is zero + checkEvaluation(new Overlay(Literal("Spark SQL"), Literal("__"), + Literal.create(0, IntegerType)), "__park SQL") + checkEvaluation(Overlay(Literal("Spark SQL"), Literal("__"), + Literal.create(0, IntegerType), Literal.create(4, IntegerType)), "__rk SQL") + // position is negative + checkEvaluation(new Overlay(Literal("Spark SQL"), Literal("__"), + Literal.create(-10, IntegerType)), "__park SQL") + checkEvaluation(Overlay(Literal("Spark SQL"), Literal("__"), + Literal.create(-10, IntegerType), Literal.create(4, IntegerType)), "__rk SQL") + } + + test("overlay for byte array") { + val input = Literal(Array[Byte](1, 2, 3, 4, 5, 6, 7, 8, 9)) + checkEvaluation(new Overlay(input, Literal(Array[Byte](-1)), + Literal.create(6, IntegerType)), Array[Byte](1, 2, 3, 4, 5, -1, 7, 8, 9)) + checkEvaluation(new Overlay(input, Literal(Array[Byte](-1, -1, -1, -1)), + Literal.create(7, IntegerType)), Array[Byte](1, 2, 3, 4, 5, 6, -1, -1, -1, -1)) + checkEvaluation(Overlay(input, Literal(Array[Byte](-1, -1)), Literal.create(7, IntegerType), + Literal.create(0, IntegerType)), Array[Byte](1, 2, 3, 4, 5, 6, -1, -1, 7, 8, 9)) + checkEvaluation(Overlay(input, Literal(Array[Byte](-1, -1, -1, -1, -1)), + Literal.create(2, IntegerType), Literal.create(4, IntegerType)), + Array[Byte](1, -1, -1, -1, -1, -1, 6, 7, 8, 9)) + + val nullInput = Literal.create(null, BinaryType) + checkEvaluation(new Overlay(nullInput, Literal(Array[Byte](-1)), + Literal.create(6, IntegerType)), null) + checkEvaluation(new Overlay(nullInput, Literal(Array[Byte](-1, -1, -1, -1)), + Literal.create(7, IntegerType)), null) + checkEvaluation(Overlay(nullInput, Literal(Array[Byte](-1, -1)), + Literal.create(7, IntegerType), Literal.create(0, IntegerType)), null) + checkEvaluation(Overlay(nullInput, Literal(Array[Byte](-1, -1, -1, -1, -1)), + Literal.create(2, IntegerType), Literal.create(4, IntegerType)), null) + // position greater than the length of input byte array + checkEvaluation(new Overlay(input, Literal(Array[Byte](-1)), + Literal.create(10, IntegerType)), Array[Byte](1, 2, 3, 4, 5, 6, 7, 8, 9, -1)) + checkEvaluation(Overlay(input, Literal(Array[Byte](-1)), Literal.create(10, IntegerType), + Literal.create(4, IntegerType)), Array[Byte](1, 2, 3, 4, 5, 6, 7, 8, 9, -1)) + // position is zero + checkEvaluation(new Overlay(input, Literal(Array[Byte](-1, -1)), + Literal.create(0, IntegerType)), Array[Byte](-1, -1, 2, 3, 4, 5, 6, 7, 8, 9)) + checkEvaluation(Overlay(input, Literal(Array[Byte](-1, -1)), Literal.create(0, IntegerType), + Literal.create(4, IntegerType)), Array[Byte](-1, -1, 4, 5, 6, 7, 8, 9)) + // position is negative + checkEvaluation(new Overlay(input, Literal(Array[Byte](-1, -1)), + Literal.create(-10, IntegerType)), Array[Byte](-1, -1, 2, 3, 4, 5, 6, 7, 8, 9)) + checkEvaluation(Overlay(input, Literal(Array[Byte](-1, -1)), Literal.create(-10, IntegerType), + Literal.create(4, IntegerType)), Array[Byte](-1, -1, 4, 5, 6, 7, 8, 9)) + } + + test("Check Overlay.checkInputDataTypes results") { + assert(new Overlay(Literal("Spark SQL"), Literal("_"), + Literal.create(6, IntegerType)).checkInputDataTypes().isSuccess) + assert(Overlay(Literal("Spark SQL"), Literal("ANSI "), Literal.create(7, IntegerType), + Literal.create(0, IntegerType)).checkInputDataTypes().isSuccess) + assert(new Overlay(Literal.create("Spark SQL".getBytes), Literal.create("_".getBytes), + Literal.create(6, IntegerType)).checkInputDataTypes().isSuccess) + assert(Overlay(Literal.create("Spark SQL".getBytes), Literal.create("ANSI ".getBytes), + Literal.create(7, IntegerType), Literal.create(0, IntegerType)) + .checkInputDataTypes().isSuccess) + assert(new Overlay(Literal.create(1), Literal.create(2), Literal.create(0, IntegerType)) + .checkInputDataTypes().isFailure) + assert(Overlay(Literal("Spark SQL"), Literal.create(2), Literal.create(7, IntegerType), + Literal.create(0, IntegerType)).checkInputDataTypes().isFailure) } test("translate") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 6b8127bab1cb4..395f1b4667b1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2521,25 +2521,25 @@ object functions { } /** - * Overlay the specified portion of `src` with `replaceString`, - * starting from byte position `pos` of `inputString` and proceeding for `len` bytes. + * Overlay the specified portion of `src` with `replace`, + * starting from byte position `pos` of `src` and proceeding for `len` bytes. * * @group string_funcs * @since 3.0.0 */ - def overlay(src: Column, replaceString: String, pos: Int, len: Int): Column = withExpr { - Overlay(src.expr, lit(replaceString).expr, lit(pos).expr, lit(len).expr) + def overlay(src: Column, replace: Column, pos: Column, len: Column): Column = withExpr { + Overlay(src.expr, replace.expr, pos.expr, len.expr) } /** - * Overlay the specified portion of `src` with `replaceString`, - * starting from byte position `pos` of `inputString`. + * Overlay the specified portion of `src` with `replace`, + * starting from byte position `pos` of `src`. * * @group string_funcs * @since 3.0.0 */ - def overlay(src: Column, replaceString: String, pos: Int): Column = withExpr { - new Overlay(src.expr, lit(replaceString).expr, lit(pos).expr) + def overlay(src: Column, replace: Column, pos: Column): Column = withExpr { + new Overlay(src.expr, replace.expr, pos.expr) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 88b3e5ec61f8a..5049df3219959 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -129,18 +129,37 @@ class StringFunctionsSuite extends QueryTest with SharedSparkSession { Row("AQIDBA==", bytes)) } - test("overlay function") { + test("string overlay function") { // scalastyle:off // non ascii characters are not allowed in the code, so we disable the scalastyle here. - val df = Seq(("Spark SQL", "Spark的SQL")).toDF("a", "b") - checkAnswer(df.select(overlay($"a", "_", 6)), Row("Spark_SQL")) - checkAnswer(df.select(overlay($"a", "CORE", 7)), Row("Spark CORE")) - checkAnswer(df.select(overlay($"a", "ANSI ", 7, 0)), Row("Spark ANSI SQL")) - checkAnswer(df.select(overlay($"a", "tructured", 2, 4)), Row("Structured SQL")) - checkAnswer(df.select(overlay($"b", "_", 6)), Row("Spark_SQL")) + val df = Seq(("Spark SQL", "Spark的SQL", "_", "CORE", "ANSI ", "tructured", 6, 7, 0, 2, 4)). + toDF("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k") + checkAnswer(df.select(overlay($"a", $"c", $"g")), Row("Spark_SQL")) + checkAnswer(df.select(overlay($"a", $"d", $"h")), Row("Spark CORE")) + checkAnswer(df.select(overlay($"a", $"e", $"h", $"i")), Row("Spark ANSI SQL")) + checkAnswer(df.select(overlay($"a", $"f", $"j", $"k")), Row("Structured SQL")) + checkAnswer(df.select(overlay($"b", $"c", $"g")), Row("Spark_SQL")) // scalastyle:on } + test("binary overlay function") { + // non ascii characters are not allowed in the code, so we disable the scalastyle here. + val df = Seq(( + Array[Byte](1, 2, 3, 4, 5, 6, 7, 8, 9), + Array[Byte](-1), + Array[Byte](-1, -1, -1, -1), + Array[Byte](-1, -1), + Array[Byte](-1, -1, -1, -1, -1), + 6, 7, 0, 2, 4)).toDF("a", "b", "c", "d", "e", "f", "g", "h", "i", "j") + checkAnswer(df.select(overlay($"a", $"b", $"f")), Row(Array[Byte](1, 2, 3, 4, 5, -1, 7, 8, 9))) + checkAnswer(df.select(overlay($"a", $"c", $"g")), + Row(Array[Byte](1, 2, 3, 4, 5, 6, -1, -1, -1, -1))) + checkAnswer(df.select(overlay($"a", $"d", $"g", $"h")), + Row(Array[Byte](1, 2, 3, 4, 5, 6, -1, -1, 7, 8, 9))) + checkAnswer(df.select(overlay($"a", $"e", $"i", $"j")), + Row(Array[Byte](1, -1, -1, -1, -1, -1, 6, 7, 8, 9))) + } + test("string / binary substring function") { // scalastyle:off // non ascii characters are not allowed in the code, so we disable the scalastyle here. From aa805eca54b4dbf46defc1c50a63227326e5b629 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 9 Sep 2019 19:11:18 -0700 Subject: [PATCH 0027/1387] [SPARK-23265][ML] Update multi-column error handling logic in QuantileDiscretizer ## What changes were proposed in this pull request? SPARK-22799 added more comprehensive error logic for Bucketizer. This PR is to update QuantileDiscretizer match the new error logic in Bucketizer. ## How was this patch tested? Add new unit test. Closes #20442 from huaxingao/spark-23265. Authored-by: Huaxin Gao Signed-off-by: Liang-Chi Hsieh --- .../ml/feature/QuantileDiscretizer.scala | 43 +++++++---- .../ml/feature/QuantileDiscretizerSuite.scala | 76 +++++++++++++++++-- 2 files changed, 96 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 5bfaa3b7f3f52..f7a83cdd41a90 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -167,25 +167,38 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui @Since("2.3.0") def setOutputCols(value: Array[String]): this.type = set(outputCols, value) - private[feature] def getInOutCols: (Array[String], Array[String]) = { - require((isSet(inputCol) && isSet(outputCol) && !isSet(inputCols) && !isSet(outputCols)) || - (!isSet(inputCol) && !isSet(outputCol) && isSet(inputCols) && isSet(outputCols)), - "QuantileDiscretizer only supports setting either inputCol/outputCol or" + - "inputCols/outputCols." - ) + @Since("1.6.0") + override def transformSchema(schema: StructType): StructType = { + ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), + Seq(outputCols)) if (isSet(inputCol)) { - (Array($(inputCol)), Array($(outputCol))) - } else { - require($(inputCols).length == $(outputCols).length, - "inputCols number do not match outputCols") - ($(inputCols), $(outputCols)) + require(!isSet(numBucketsArray), + s"numBucketsArray can't be set for single-column QuantileDiscretizer.") } - } - @Since("1.6.0") - override def transformSchema(schema: StructType): StructType = { - val (inputColNames, outputColNames) = getInOutCols + if (isSet(inputCols)) { + require(getInputCols.length == getOutputCols.length, + s"QuantileDiscretizer $this has mismatched Params " + + s"for multi-column transform. Params (inputCols, outputCols) should have " + + s"equal lengths, but they have different lengths: " + + s"(${getInputCols.length}, ${getOutputCols.length}).") + if (isSet(numBucketsArray)) { + require(getInputCols.length == getNumBucketsArray.length, + s"QuantileDiscretizer $this has mismatched Params " + + s"for multi-column transform. Params (inputCols, outputCols, numBucketsArray) " + + s"should have equal lengths, but they have different lengths: " + + s"(${getInputCols.length}, ${getOutputCols.length}, ${getNumBucketsArray.length}).") + require(!isSet(numBuckets), + s"exactly one of numBuckets, numBucketsArray Params to be set, but both are set." ) + } + } + + val (inputColNames, outputColNames) = if (isSet(inputCols)) { + ($(inputCols).toSeq, $(outputCols).toSeq) + } else { + (Seq($(inputCol)), Seq($(outputCol))) + } val existingFields = schema.fields var outputFields = existingFields inputColNames.zip(outputColNames).foreach { case (inputColName, outputColName) => diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala index ae086d32d6d0b..6f6ab26cbac43 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.ml.feature import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.sql._ @@ -423,33 +424,92 @@ class QuantileDiscretizerSuite extends MLTest with DefaultReadWriteTest { assert(readDiscretizer.hasDefault(readDiscretizer.outputCol)) } - test("Multiple Columns: Both inputCol and inputCols are set") { + test("Multiple Columns: Mismatched sizes of inputCols/outputCols") { val spark = this.spark import spark.implicits._ val discretizer = new QuantileDiscretizer() - .setInputCol("input") - .setOutputCol("result") + .setInputCols(Array("input")) + .setOutputCols(Array("result1", "result2")) .setNumBuckets(3) - .setInputCols(Array("input1", "input2")) val df = sc.parallelize(Array(1.0, 2.0, 3.0, 4.0, 5.0, 6.0)) .map(Tuple1.apply).toDF("input") - // When both inputCol and inputCols are set, we throw Exception. intercept[IllegalArgumentException] { discretizer.fit(df) } } - test("Multiple Columns: Mismatched sizes of inputCols / outputCols") { + test("Multiple Columns: Mismatched sizes of inputCols/numBucketsArray") { val spark = this.spark import spark.implicits._ val discretizer = new QuantileDiscretizer() - .setInputCols(Array("input")) + .setInputCols(Array("input1", "input2")) .setOutputCols(Array("result1", "result2")) - .setNumBuckets(3) + .setNumBucketsArray(Array(2, 5, 10)) + val data1 = Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0) + val data2 = Array(1.0, 2.0, 3.0, 1.0, 1.0, 1.0, 1.0, 3.0, 2.0, 3.0) + val df = data1.zip(data2).toSeq.toDF("input1", "input2") + intercept[IllegalArgumentException] { + discretizer.fit(df) + } + } + + test("Multiple Columns: Set both of numBuckets/numBucketsArray") { + val spark = this.spark + import spark.implicits._ + val discretizer = new QuantileDiscretizer() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("result1", "result2")) + .setNumBucketsArray(Array(2, 5)) + .setNumBuckets(2) + val data1 = Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0) + val data2 = Array(1.0, 2.0, 3.0, 1.0, 1.0, 1.0, 1.0, 3.0, 2.0, 3.0) + val df = data1.zip(data2).toSeq.toDF("input1", "input2") + intercept[IllegalArgumentException] { + discretizer.fit(df) + } + } + + test("Setting numBucketsArray for Single-Column QuantileDiscretizer") { + val spark = this.spark + import spark.implicits._ + val discretizer = new QuantileDiscretizer() + .setInputCol("input") + .setOutputCol("result") + .setNumBucketsArray(Array(2, 5)) val df = sc.parallelize(Array(1.0, 2.0, 3.0, 4.0, 5.0, 6.0)) .map(Tuple1.apply).toDF("input") intercept[IllegalArgumentException] { discretizer.fit(df) } } + + test("Assert exception is thrown if both multi-column and single-column params are set") { + val spark = this.spark + import spark.implicits._ + val df = Seq((0.5, 0.3), (0.5, -0.4)).toDF("feature1", "feature2") + ParamsSuite.testExclusiveParams(new QuantileDiscretizer, df, ("inputCol", "feature1"), + ("inputCols", Array("feature1", "feature2"))) + ParamsSuite.testExclusiveParams(new QuantileDiscretizer, df, ("inputCol", "feature1"), + ("outputCol", "result1"), ("outputCols", Array("result1", "result2"))) + // this should fail because at least one of inputCol and inputCols must be set + ParamsSuite.testExclusiveParams(new QuantileDiscretizer, df, ("outputCol", "feature1")) + } + + test("Setting inputCol without setting outputCol") { + val spark = this.spark + import spark.implicits._ + + val df = sc.parallelize(Array(1.0, 2.0, 3.0, 4.0, 5.0, 6.0)) + .map(Tuple1.apply).toDF("input") + val numBuckets = 2 + val discretizer = new QuantileDiscretizer() + .setInputCol("input") + .setNumBuckets(numBuckets) + val model = discretizer.fit(df) + val result = model.transform(df) + + val observedNumBuckets = result.select(discretizer.getOutputCol).distinct.count + assert(observedNumBuckets === numBuckets, + "Observed number of buckets does not equal expected number of buckets.") + } } From 86fc890d8c4e32df85cefc3373b2f901f8c63097 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 10 Sep 2019 11:22:06 +0900 Subject: [PATCH 0028/1387] [SPARK-28988][SQL][TESTS] Fix invalid tests in CliSuite ### What changes were proposed in this pull request? https://github.com/apache/spark/blob/1f056eb313de792a279c3c7c21b5bccd652c631b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala#L221 is not strong enough. It will success if class not found. https://github.com/apache/spark/blob/1f056eb313de792a279c3c7c21b5bccd652c631b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala#L305 is also incorrect. Whatever the right side value is, it always succeeds. ### Why are the changes needed? Unit tests should failed if the class not found. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exist UTs Closes #25724 from LantaoJin/SPARK-28988. Authored-by: LantaoJin Signed-off-by: HyukjinKwon --- .../apache/spark/sql/hive/thriftserver/CliSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 6e042ac41d9da..a979fe6b89679 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -27,7 +27,7 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax +import org.apache.hadoop.hive.contrib.udf.example.UDFExampleFormat import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite @@ -218,8 +218,8 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { -> "", "INSERT INTO TABLE t1 SELECT key, val FROM sourceTable;" -> "", - "SELECT count(key) FROM t1;" - -> "5", + "SELECT collect_list(array(val)) FROM t1;" + -> """[["val_238"],["val_86"],["val_311"],["val_27"],["val_165"]]""", "DROP TABLE t1;" -> "", "DROP TABLE sourceTable;" @@ -301,8 +301,8 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { runCliWithin( 1.minute, Seq("--conf", s"spark.hadoop.${ConfVars.HIVEAUXJARS}=$hiveContribJar"))( - s"CREATE TEMPORARY FUNCTION example_max AS '${classOf[UDAFExampleMax].getName}';" -> "", - "SELECT example_max(1);" -> "1" + s"CREATE TEMPORARY FUNCTION example_format AS '${classOf[UDFExampleFormat].getName}';" -> "", + "SELECT example_format('%o', 93);" -> "135" ) } } From c2d8ee9c54adf4a425ce41d8743e24dd8be864c3 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 10 Sep 2019 11:01:57 +0800 Subject: [PATCH 0029/1387] [SPARK-28878][SQL][FOLLOWUP] Remove extra project for DSv2 streaming scan ### What changes were proposed in this pull request? Remove the project node if the streaming scan is columnar ### Why are the changes needed? This is a followup of https://github.com/apache/spark/pull/25586. Batch and streaming share the same DS v2 read API so both can support columnar reads. We should apply #25586 to streaming scan as well. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #25727 from cloud-fan/follow. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../datasources/v2/DataSourceV2Strategy.scala | 29 ++++++++++++++----- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 7cad305aefeb8..f629f36642bb2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -155,17 +155,30 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case r: StreamingDataSourceV2Relation if r.startOffset.isDefined && r.endOffset.isDefined => val microBatchStream = r.stream.asInstanceOf[MicroBatchStream] - // ensure there is a projection, which will produce unsafe rows required by some operators - ProjectExec(r.output, - MicroBatchScanExec( - r.output, r.scan, microBatchStream, r.startOffset.get, r.endOffset.get)) :: Nil + val scanExec = MicroBatchScanExec( + r.output, r.scan, microBatchStream, r.startOffset.get, r.endOffset.get) + + val withProjection = if (scanExec.supportsColumnar) { + scanExec + } else { + // Add a Project here to make sure we produce unsafe rows. + ProjectExec(r.output, scanExec) + } + + withProjection :: Nil case r: StreamingDataSourceV2Relation if r.startOffset.isDefined && r.endOffset.isEmpty => val continuousStream = r.stream.asInstanceOf[ContinuousStream] - // ensure there is a projection, which will produce unsafe rows required by some operators - ProjectExec(r.output, - ContinuousScanExec( - r.output, r.scan, continuousStream, r.startOffset.get)) :: Nil + val scanExec = ContinuousScanExec(r.output, r.scan, continuousStream, r.startOffset.get) + + val withProjection = if (scanExec.supportsColumnar) { + scanExec + } else { + // Add a Project here to make sure we produce unsafe rows. + ProjectExec(r.output, scanExec) + } + + withProjection :: Nil case WriteToDataSourceV2(writer, query) => WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil From 580c6266fbe9bfa9c9d21ddfb16482b1b26e4df4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 9 Sep 2019 20:30:49 -0700 Subject: [PATCH 0030/1387] [SPARK-28939][SQL][FOLLOWUP] Fix JDK11 compilation due to ambiguous reference ### What changes were proposed in this pull request? This PR aims to recover the JDK11 compilation with a workaround. For now, the master branch is broken like the following due to a [Scala bug](https://github.com/scala/bug/issues/10418) which is fixed in `2.13.0-RC2`. ``` [ERROR] [Error] /spark/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala:42: ambiguous reference to overloaded definition, both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit and method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit match argument types (java.util.Map[String,String]) ``` - https://github.com/apache/spark/actions (JDK11 build monitoring) ### Why are the changes needed? This workaround recovers JDK11 compilation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manual build with JDK11 because this is JDK11 compilation fix. - Jenkins builds with JDK8 and tests with JDK11. - GitHub action will verify this after merging. Closes #25738 from dongjoon-hyun/SPARK-28939. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala index 7373da33e12ad..0e4dc6491ef04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala @@ -39,7 +39,7 @@ class SQLExecutionRDD( private val sqlConfigs = conf.getAllConfs private lazy val sqlConfExecutorSide = { val props = new Properties() - props.putAll(sqlConfigs.asJava) + sqlConfigs.foreach { case (k, v) => props.setProperty(k, v) } val newConf = new SQLConf() newConf.setConf(props) newConf From 962e330955581aea032ff336a12f23374c39e672 Mon Sep 17 00:00:00 2001 From: sychen Date: Mon, 9 Sep 2019 22:06:19 -0700 Subject: [PATCH 0031/1387] [SPARK-26598][SQL] Fix HiveThriftServer2 cannot be modified hiveconf/hivevar variables ### What changes were proposed in this pull request? The intent to use the --hiveconf/--hivevar parameter is just an initialization value, so setting it once in ```SparkSQLSessionManager#openSession``` is sufficient, and each time the ```SparkExecuteStatementOperation``` setting causes the variable to not be modified. ### Why are the changes needed? It is wrong to set the --hivevar/--hiveconf variable in every ```SparkExecuteStatementOperation```, which prevents variable updates. ### Does this PR introduce any user-facing change? ``` cat < test.sql select '\${a}', '\${b}'; set b=bvalue_MOD_VALUE; set b; EOF beeline -u jdbc:hive2://localhost:10000 --hiveconf a=avalue --hivevar b=bvalue -f test.sql ``` current result: ``` +-----------------+-----------------+--+ | avalue | bvalue | +-----------------+-----------------+--+ | avalue | bvalue | +-----------------+-----------------+--+ +-----------------+-----------------+--+ | key | value | +-----------------+-----------------+--+ | b | bvalue | +-----------------+-----------------+--+ 1 row selected (0.022 seconds) ``` after modification: ``` +-----------------+-----------------+--+ | avalue | bvalue | +-----------------+-----------------+--+ | avalue | bvalue | +-----------------+-----------------+--+ +-----------------+-----------------+--+ | key | value | +-----------------+-----------------+--+ | b | bvalue_MOD_VALUE| +-----------------+-----------------+--+ 1 row selected (0.022 seconds) ``` ### How was this patch tested? modified the existing unit test Closes #25722 from cxzl25/fix_SPARK-26598. Authored-by: sychen Signed-off-by: Yuming Wang --- .../hive/thriftserver/SparkSQLSessionManager.scala | 11 +++++++++++ .../server/SparkSQLOperationManager.scala | 12 ------------ .../hive/thriftserver/HiveThriftServer2Suites.scala | 13 ++++++++++--- 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 13055e0ae1394..8369c09b09e74 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -63,6 +63,9 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: sqlContext.newSession() } ctx.setConf(HiveUtils.FAKE_HIVE_VERSION.key, HiveUtils.builtinHiveVersion) + val hiveSessionState = session.getSessionState + setConfMap(ctx, hiveSessionState.getOverriddenConfigurations) + setConfMap(ctx, hiveSessionState.getHiveVariables) if (sessionConf != null && sessionConf.containsKey("use:database")) { ctx.sql(s"use ${sessionConf.get("use:database")}") } @@ -76,4 +79,12 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: sparkSqlOperationManager.sessionToActivePool.remove(sessionHandle) sparkSqlOperationManager.sessionToContexts.remove(sessionHandle) } + + def setConfMap(conf: SQLContext, confMap: java.util.Map[String, String]): Unit = { + val iterator = confMap.entrySet().iterator() + while (iterator.hasNext) { + val kv = iterator.next() + conf.setConf(kv.getKey, kv.getValue) + } + } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 35f92547e7815..a37894fe908ea 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -28,7 +28,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver._ -import org.apache.spark.sql.internal.SQLConf /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -51,9 +50,6 @@ private[thriftserver] class SparkSQLOperationManager() require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + s" initialized or had already closed.") val conf = sqlContext.sessionState.conf - val hiveSessionState = parentSession.getSessionState - setConfMap(conf, hiveSessionState.getOverriddenConfigurations) - setConfMap(conf, hiveSessionState.getHiveVariables) val runInBackground = async && conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC) val operation = new SparkExecuteStatementOperation(parentSession, statement, confOverlay, runInBackground)(sqlContext, sessionToActivePool) @@ -144,12 +140,4 @@ private[thriftserver] class SparkSQLOperationManager() logDebug(s"Created GetFunctionsOperation with session=$parentSession.") operation } - - def setConfMap(conf: SQLConf, confMap: java.util.Map[String, String]): Unit = { - val iterator = confMap.entrySet().iterator() - while (iterator.hasNext) { - val kv = iterator.next() - conf.setConfString(kv.getKey, kv.getValue) - } - } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index e504ecd9fefd5..3fef23f8eed3e 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -144,10 +144,17 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { def executeTest(hiveList: String): Unit = { hiveList.split(";").foreach{ m => val kv = m.split("=") - // select "${a}"; ---> avalue - val resultSet = statement.executeQuery("select \"${" + kv(0) + "}\"") + val k = kv(0) + val v = kv(1) + val modValue = s"${v}_MOD_VALUE" + // select '${a}'; ---> avalue + val resultSet = statement.executeQuery(s"select '$${$k}'") resultSet.next() - assert(resultSet.getString(1) === kv(1)) + assert(resultSet.getString(1) === v) + statement.executeQuery(s"set $k=$modValue") + val modResultSet = statement.executeQuery(s"select '$${$k}'") + modResultSet.next() + assert(modResultSet.getString(1) === s"$modValue") } } } From ca6f693ef17ccb27a6ef5bdad9141abb2fe0434c Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Tue, 10 Sep 2019 20:47:55 +0900 Subject: [PATCH 0032/1387] [SPARK-28939][SQL][FOLLOWUP] Avoid useless Properties ### What changes were proposed in this pull request? Removes useless `Properties` created according to hvanhovell 's suggestion. ### Why are the changes needed? Avoid useless code. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? existing UTs Closes #25742 from mgaido91/SPARK-28939_followup. Authored-by: Marco Gaido Signed-off-by: Takeshi Yamamuro --- .../org/apache/spark/sql/execution/SQLExecutionRDD.scala | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala index 0e4dc6491ef04..45b9cadc4aeda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecutionRDD.scala @@ -16,10 +16,6 @@ */ package org.apache.spark.sql.execution -import java.util.Properties - -import scala.collection.JavaConverters._ - import org.apache.spark.{Partition, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -38,10 +34,8 @@ class SQLExecutionRDD( var sqlRDD: RDD[InternalRow], @transient conf: SQLConf) extends RDD[InternalRow](sqlRDD) { private val sqlConfigs = conf.getAllConfs private lazy val sqlConfExecutorSide = { - val props = new Properties() - sqlConfigs.foreach { case (k, v) => props.setProperty(k, v) } val newConf = new SQLConf() - newConf.setConf(props) + sqlConfigs.foreach { case (k, v) => newConf.setConfString(k, v) } newConf } From bf43541c928af2788c1e9990faf57e9508f974e9 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 10 Sep 2019 21:23:57 +0800 Subject: [PATCH 0033/1387] [SPARK-28856][SQL] Implement SHOW DATABASES for Data Source V2 Tables ### What changes were proposed in this pull request? Implement the SHOW DATABASES logical and physical plans for data source v2 tables. ### Why are the changes needed? To support `SHOW DATABASES` SQL commands for v2 tables. ### Does this PR introduce any user-facing change? `spark.sql("SHOW DATABASES")` will return namespaces if the default catalog is set: ``` +---------------+ | namespace| +---------------+ | ns1| | ns1.ns1_1| |ns1.ns1_1.ns1_2| +---------------+ ``` ### How was this patch tested? Added unit tests to `DataSourceV2SQLSuite`. Closes #25601 from imback82/show_databases. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 1 + .../spark/sql/catalyst/parser/SqlBase.g4 | 5 ++ .../sql/catalog/v2/CatalogV2Implicits.scala | 8 ++ .../sql/catalyst/parser/AstBuilder.scala | 11 ++- .../plans/logical/basicLogicalOperators.scala | 13 ++- .../logical/sql/ShowNamespacesStatement.scala | 24 +++++ .../sql/catalyst/parser/DDLParserSuite.scala | 17 +++- .../sql/connector/InMemoryTableCatalog.scala | 12 ++- .../datasources/DataSourceResolution.scala | 26 +++++- .../datasources/v2/DataSourceV2Strategy.scala | 5 +- .../datasources/v2/ShowNamespacesExec.scala | 63 +++++++++++++ .../sql/sources/v2/DataSourceV2SQLSuite.scala | 88 ++++++++++++++++++- 12 files changed, 260 insertions(+), 13 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 08be6b62a88e7..16ce35d55ed7b 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -179,6 +179,7 @@ Below is a list of all the keywords in Spark SQL. MONTHreservednon-reservedreserved MONTHSnon-reservednon-reservednon-reserved MSCKnon-reservednon-reservednon-reserved + NAMESPACESnon-reservednon-reservednon-reserved NATURALreservedstrict-non-reservedreserved NOnon-reservednon-reservedreserved NOTreservednon-reservedreserved diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index f16ac6df8cb04..70c0d0e505f6c 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -92,6 +92,8 @@ statement | DROP database (IF EXISTS)? db=errorCapturingIdentifier (RESTRICT | CASCADE)? #dropDatabase | SHOW DATABASES (LIKE? pattern=STRING)? #showDatabases + | SHOW NAMESPACES ((FROM | IN) multipartIdentifier)? + (LIKE? pattern=STRING)? #showNamespaces | createTableHeader ('(' colTypeList ')')? tableProvider ((OPTIONS options=tablePropertyList) | (PARTITIONED BY partitioning=transformList) | @@ -1006,6 +1008,7 @@ ansiNonReserved | MINUTES | MONTHS | MSCK + | NAMESPACES | NO | NULLS | OF @@ -1255,6 +1258,7 @@ nonReserved | MONTH | MONTHS | MSCK + | NAMESPACES | NO | NOT | NULL @@ -1515,6 +1519,7 @@ MINUTES: 'MINUTES'; MONTH: 'MONTH'; MONTHS: 'MONTHS'; MSCK: 'MSCK'; +NAMESPACES: 'NAMESPACES'; NATURAL: 'NATURAL'; NO: 'NO'; NOT: 'NOT' | '!'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala index f512cd5e23c6b..87070fb2ad377 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala @@ -68,6 +68,14 @@ object CatalogV2Implicits { case _ => throw new AnalysisException(s"Cannot use catalog ${plugin.name}: not a TableCatalog") } + + def asNamespaceCatalog: SupportsNamespaces = plugin match { + case namespaceCatalog: SupportsNamespaces => + namespaceCatalog + case _ => + throw new AnalysisException( + s"Cannot use catalog ${plugin.name}: does not support namespaces") + } } implicit class NamespaceHelper(namespace: Array[String]) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 90a533735aed2..25bfaa89017cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowTablesStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -2260,6 +2260,15 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Create a [[ShowNamespacesStatement]] command. + */ + override def visitShowNamespaces(ctx: ShowNamespacesContext): LogicalPlan = withOrigin(ctx) { + ShowNamespacesStatement( + Option(ctx.multipartIdentifier).map(visitMultipartIdentifier), + Option(ctx.pattern).map(string)) + } + /** * Create a table, returning a [[CreateTableStatement]] logical plan. * 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 0be61cf147041..aa613d2649fe3 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.{Identifier, SupportsNamespaces, TableCatalog, TableChange} import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.AliasIdentifier @@ -560,6 +560,17 @@ object OverwritePartitionsDynamic { } } +/** + * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. + */ +case class ShowNamespaces( + catalog: SupportsNamespaces, + namespace: Option[Seq[String]], + pattern: Option[String]) extends Command { + override val output: Seq[Attribute] = Seq( + AttributeReference("namespace", StringType, nullable = false)()) +} + case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { override def children: Seq[LogicalPlan] = Seq(table) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala new file mode 100644 index 0000000000000..95d48a8e80d23 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical.sql + +/** + * A SHOW NAMESPACES statement, as parsed from SQL. + */ +case class ShowNamespacesStatement(namespace: Option[Seq[String]], pattern: Option[String]) + extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 9a0a326ecd5d2..2e5ff8c3ce6dd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransf import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowTablesStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -779,6 +779,21 @@ class DDLParserSuite extends AnalysisTest { ShowTablesStatement(Some(Seq("tbl")), Some("*dog*"))) } + test("show namespaces") { + comparePlans( + parsePlan("SHOW NAMESPACES"), + ShowNamespacesStatement(None, None)) + comparePlans( + parsePlan("SHOW NAMESPACES FROM testcat.ns1.ns2"), + ShowNamespacesStatement(Some(Seq("testcat", "ns1", "ns2")), None)) + comparePlans( + parsePlan("SHOW NAMESPACES IN testcat.ns1.ns2"), + ShowNamespacesStatement(Some(Seq("testcat", "ns1", "ns2")), None)) + comparePlans( + parsePlan("SHOW NAMESPACES IN testcat.ns1 LIKE '*pattern*'"), + ShowNamespacesStatement(Some(Seq("testcat", "ns1")), Some("*pattern*"))) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala index cff09f7550385..8b14ad0d372dd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala @@ -30,12 +30,9 @@ import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -class InMemoryTableCatalog extends TableCatalog with SupportsNamespaces { +class BasicInMemoryTableCatalog extends TableCatalog { import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ - protected val namespaces: util.Map[List[String], Map[String, String]] = - new ConcurrentHashMap[List[String], Map[String, String]]() - protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() @@ -112,6 +109,13 @@ class InMemoryTableCatalog extends TableCatalog with SupportsNamespaces { def clearTables(): Unit = { tables.clear() } +} + +class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamespaces { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + protected val namespaces: util.Map[List[String], Map[String, String]] = + new ConcurrentHashMap[List[String], Map[String, String]]() private def allNamespaces: Seq[Seq[String]] = { (tables.keySet.asScala.map(_.namespace.toSeq) ++ namespaces.keySet.asScala).toSeq.distinct diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index 43bee695bd5ce..68d32059bfe3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -20,15 +20,15 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogManager, Identifier, LookupCatalog, TableCatalog} +import org.apache.spark.sql.catalog.v2.{CatalogManager, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, Filter, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowTables, SubqueryAlias} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowTablesStatement} +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, Filter, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables, SubqueryAlias} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowDatabasesCommand, ShowTablesCommand} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} @@ -169,6 +169,24 @@ case class DataSourceResolution( val aliased = delete.tableAlias.map(SubqueryAlias(_, relation)).getOrElse(relation) DeleteFromTable(aliased, delete.condition) + case ShowNamespacesStatement(None, pattern) => + defaultCatalog match { + case Some(catalog) => + ShowNamespaces(catalog.asNamespaceCatalog, None, pattern) + case None => + throw new AnalysisException("No default v2 catalog is set.") + } + + case ShowNamespacesStatement(Some(namespace), pattern) => + val CatalogNamespace(maybeCatalog, ns) = namespace + maybeCatalog match { + case Some(catalog) => + ShowNamespaces(catalog.asNamespaceCatalog, Some(ns), pattern) + case None => + throw new AnalysisException( + s"No v2 catalog is available for ${namespace.quoted}") + } + case ShowTablesStatement(None, pattern) => defaultCatalog match { case Some(catalog) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index f629f36642bb2..10dcf402ed61d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalog.v2.StagingTableCatalog import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} @@ -291,6 +291,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case AlterTable(catalog, ident, _, changes) => AlterTableExec(catalog, ident, changes) :: Nil + case r: ShowNamespaces => + ShowNamespacesExec(r.output, r.catalog, r.namespace, r.pattern) :: Nil + case r : ShowTables => ShowTablesExec(r.output, r.catalog, r.namespace, r.pattern) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala new file mode 100644 index 0000000000000..f70b943fd2cde --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala @@ -0,0 +1,63 @@ +/* + * 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.v2 + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.NamespaceHelper +import org.apache.spark.sql.catalog.v2.SupportsNamespaces +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.execution.LeafExecNode + +/** + * Physical plan node for showing namespaces. + */ +case class ShowNamespacesExec( + output: Seq[Attribute], + catalog: SupportsNamespaces, + namespace: Option[Seq[String]], + pattern: Option[String]) + extends LeafExecNode { + override protected def doExecute(): RDD[InternalRow] = { + val namespaces = namespace.map { ns => + if (ns.nonEmpty) { + catalog.listNamespaces(ns.toArray) + } else { + catalog.listNamespaces() + } + } + .getOrElse(catalog.listNamespaces()) + + val rows = new ArrayBuffer[InternalRow]() + val encoder = RowEncoder(schema).resolveAndBind() + + namespaces.map(_.quoted).map { ns => + if (pattern.map(StringUtils.filterPattern(Seq(ns), _).nonEmpty).getOrElse(true)) { + rows += encoder + .toRow(new GenericRowWithSchema(Array(ns), schema)) + .copy() + } + } + + sparkContext.parallelize(rows, 1) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index de5b4692b8ba0..4bfbefef44bb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql._ import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog} import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog, StagingInMemoryTableCatalog} +import org.apache.spark.sql.connector.{BasicInMemoryTableCatalog, InMemoryTable, InMemoryTableCatalog, StagingInMemoryTableCatalog} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG import org.apache.spark.sql.sources.v2.internal.V1Table @@ -746,6 +746,92 @@ class DataSourceV2SQLSuite assert(expected === df.collect()) } + test("ShowNamespaces: show root namespaces with default v2 catalog") { + spark.conf.set("spark.sql.default.catalog", "testcat") + + testShowNamespaces("SHOW NAMESPACES", Seq()) + + spark.sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") + spark.sql("CREATE TABLE testcat.ns1.ns1_1.table (id bigint) USING foo") + spark.sql("CREATE TABLE testcat.ns2.table (id bigint) USING foo") + + testShowNamespaces("SHOW NAMESPACES", Seq("ns1", "ns2")) + testShowNamespaces("SHOW NAMESPACES LIKE '*1*'", Seq("ns1")) + } + + test("ShowNamespaces: show namespaces with v2 catalog") { + spark.sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") + spark.sql("CREATE TABLE testcat.ns1.ns1_1.table (id bigint) USING foo") + spark.sql("CREATE TABLE testcat.ns1.ns1_2.table (id bigint) USING foo") + spark.sql("CREATE TABLE testcat.ns2.table (id bigint) USING foo") + spark.sql("CREATE TABLE testcat.ns2.ns2_1.table (id bigint) USING foo") + + // Look up only with catalog name, which should list root namespaces. + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1", "ns2")) + + // Look up sub-namespaces. + testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns1_1", "ns1.ns1_2")) + testShowNamespaces("SHOW NAMESPACES IN testcat.ns1 LIKE '*2*'", Seq("ns1.ns1_2")) + testShowNamespaces("SHOW NAMESPACES IN testcat.ns2", Seq("ns2.ns2_1")) + + // Try to look up namespaces that do not exist. + testShowNamespaces("SHOW NAMESPACES IN testcat.ns3", Seq()) + testShowNamespaces("SHOW NAMESPACES IN testcat.ns1.ns3", Seq()) + } + + test("ShowNamespaces: default v2 catalog is not set") { + spark.sql("CREATE TABLE testcat.ns.table (id bigint) USING foo") + + val exception = intercept[AnalysisException] { + sql("SHOW NAMESPACES") + } + + assert(exception.getMessage.contains("No default v2 catalog is set")) + } + + test("ShowNamespaces: default v2 catalog doesn't support namespace") { + spark.conf.set( + "spark.sql.catalog.testcat_no_namspace", + classOf[BasicInMemoryTableCatalog].getName) + spark.conf.set("spark.sql.default.catalog", "testcat_no_namspace") + + val exception = intercept[AnalysisException] { + sql("SHOW NAMESPACES") + } + + assert(exception.getMessage.contains("does not support namespaces")) + } + + test("ShowNamespaces: v2 catalog doesn't support namespace") { + spark.conf.set( + "spark.sql.catalog.testcat_no_namspace", + classOf[BasicInMemoryTableCatalog].getName) + + val exception = intercept[AnalysisException] { + sql("SHOW NAMESPACES in testcat_no_namspace") + } + + assert(exception.getMessage.contains("does not support namespaces")) + } + + test("ShowNamespaces: no v2 catalog is available") { + val exception = intercept[AnalysisException] { + sql("SHOW NAMESPACES in dummy") + } + + assert(exception.getMessage.contains("No v2 catalog is available")) + } + + private def testShowNamespaces( + sqlText: String, + expected: Seq[String]): Unit = { + val schema = new StructType().add("namespace", StringType, nullable = false) + + val df = spark.sql(sqlText) + assert(df.schema === schema) + assert(df.collect().map(_.getAs[String](0)).sorted === expected.sorted) + } + test("tableCreation: partition column case insensitive resolution") { val testCatalog = catalog("testcat").asTableCatalog val sessionCatalog = catalog("session").asTableCatalog From 7309e021ec6dc9cabec94a02e02c13dd9ef015ba Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Tue, 10 Sep 2019 11:19:55 -0500 Subject: [PATCH 0034/1387] [SPARK-29028][DOCS] Add links to IBM Cloud Object Storage connector in cloud-integration.md ### What changes were proposed in this pull request? Add links to IBM Cloud Storage connector in cloud-integration.md ### Why are the changes needed? This page mentions the connectors to cloud providers. Currently connector to IBM cloud storage is not specified. This PR adds the necessary links for completeness. ### Does this PR introduce any user-facing change? Yes. **Before:** Screen Shot 2019-09-09 at 3 52 44 PM **After.** Screen Shot 2019-09-10 at 8 16 49 AM ### How was this patch tested? Tested using jykyll build --serve Closes #25737 from dilipbiswal/ibm-cloud-storage. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/cloud-integration.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index a8d40fe7456e4..b2a3e77f1ee9d 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -257,4 +257,5 @@ Here is the documentation on the standard connectors both from Apache and the cl * [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon * [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google * [The Azure Blob Filesystem driver (ABFS)](https://docs.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-abfs-driver) +* IBM Cloud Object Storage connector for Apache Spark: [Stocator](https://github.com/CODAIT/stocator), [IBM Object Storage](https://www.ibm.com/cloud/object-storage), [how-to-use-connector](https://developer.ibm.com/code/2018/08/16/installing-running-stocator-apache-spark-ibm-cloud-object-storage). From IBM From 54d3f6e7ec748e04b883ad46c2e1c36a0716f428 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 10 Sep 2019 09:22:50 -0700 Subject: [PATCH 0035/1387] [SPARK-28982][SQL] Implementation Spark's own GetTypeInfoOperation ### What changes were proposed in this pull request? Current Spark Thrift Server return TypeInfo includes 1. INTERVAL_YEAR_MONTH 2. INTERVAL_DAY_TIME 3. UNION 4. USER_DEFINED Spark doesn't support INTERVAL_YEAR_MONTH, INTERVAL_YEAR_MONTH, UNION and won't return USER)DEFINED type. This PR overwrite GetTypeInfoOperation with SparkGetTypeInfoOperation to exclude types which we don't need. In hive-1.2.1 Type class is `org.apache.hive.service.cli.Type` In hive-2.3.x Type class is `org.apache.hadoop.hive.serde2.thrift.Type` Use ThrifrserverShimUtils to fit version problem and exclude types we don't need ### Why are the changes needed? We should return type info of Spark's own type info ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manuel test & Added UT Closes #25694 from AngersZhuuuu/SPARK-28982. Lead-authored-by: angerszhu Co-authored-by: AngersZhuuuu Signed-off-by: Yuming Wang --- .../SparkGetTypeInfoOperation.scala | 103 ++++++++++++++++++ .../server/SparkSQLOperationManager.scala | 11 ++ .../SparkMetadataOperationSuite.scala | 16 +++ .../cli/operation/GetTypeInfoOperation.java | 2 +- .../thriftserver/ThriftserverShimUtils.scala | 9 ++ .../cli/operation/GetTypeInfoOperation.java | 2 +- .../thriftserver/ThriftserverShimUtils.scala | 9 ++ 7 files changed, 150 insertions(+), 2 deletions(-) create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala new file mode 100644 index 0000000000000..7a6a8c59b7216 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.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.sql.hive.thriftserver + +import java.util.UUID + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType +import org.apache.hive.service.cli.{HiveSQLException, OperationState} +import org.apache.hive.service.cli.operation.GetTypeInfoOperation +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.util.{Utils => SparkUtils} + +/** + * Spark's own GetTypeInfoOperation + * + * @param sqlContext SQLContext to use + * @param parentSession a HiveSession from SessionManager + */ +private[hive] class SparkGetTypeInfoOperation( + sqlContext: SQLContext, + parentSession: HiveSession) + extends GetTypeInfoOperation(parentSession) with Logging { + + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + + override def runInternal(): Unit = { + statementId = UUID.randomUUID().toString + val logMsg = "Listing type info" + logInfo(s"$logMsg with $statementId") + setState(OperationState.RUNNING) + // Always use the latest class loader provided by executionHive's state. + val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + Thread.currentThread().setContextClassLoader(executionHiveClassLoader) + + if (isAuthV2Enabled) { + authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null) + } + + HiveThriftServer2.listener.onStatementStart( + statementId, + parentSession.getSessionHandle.getSessionId.toString, + logMsg, + statementId, + parentSession.getUsername) + + try { + ThriftserverShimUtils.supportedType().foreach(typeInfo => { + val rowData = Array[AnyRef]( + typeInfo.getName, // TYPE_NAME + typeInfo.toJavaSQLType.asInstanceOf[AnyRef], // DATA_TYPE + typeInfo.getMaxPrecision.asInstanceOf[AnyRef], // PRECISION + typeInfo.getLiteralPrefix, // LITERAL_PREFIX + typeInfo.getLiteralSuffix, // LITERAL_SUFFIX + typeInfo.getCreateParams, // CREATE_PARAMS + typeInfo.getNullable.asInstanceOf[AnyRef], // NULLABLE + typeInfo.isCaseSensitive.asInstanceOf[AnyRef], // CASE_SENSITIVE + typeInfo.getSearchable.asInstanceOf[AnyRef], // SEARCHABLE + typeInfo.isUnsignedAttribute.asInstanceOf[AnyRef], // UNSIGNED_ATTRIBUTE + typeInfo.isFixedPrecScale.asInstanceOf[AnyRef], // FIXED_PREC_SCALE + typeInfo.isAutoIncrement.asInstanceOf[AnyRef], // AUTO_INCREMENT + typeInfo.getLocalizedName, // LOCAL_TYPE_NAME + typeInfo.getMinimumScale.asInstanceOf[AnyRef], // MINIMUM_SCALE + typeInfo.getMaximumScale.asInstanceOf[AnyRef], // MAXIMUM_SCALE + null, // SQL_DATA_TYPE, unused + null, // SQL_DATETIME_SUB, unused + typeInfo.getNumPrecRadix // NUM_PREC_RADIX + ) + rowSet.addRow(rowData) + }) + setState(OperationState.FINISHED) + } catch { + case e: HiveSQLException => + setState(OperationState.ERROR) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) + throw e + } + HiveThriftServer2.listener.onStatementFinish(statementId) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index a37894fe908ea..3396560f43502 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -140,4 +140,15 @@ private[thriftserver] class SparkSQLOperationManager() logDebug(s"Created GetFunctionsOperation with session=$parentSession.") operation } + + override def newGetTypeInfoOperation( + parentSession: HiveSession): GetTypeInfoOperation = synchronized { + val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) + require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + " initialized or had already closed.") + val operation = new SparkGetTypeInfoOperation(sqlContext, parentSession) + handleToOperation.put(operation.getHandle, operation) + logDebug(s"Created GetTypeInfoOperation with session=$parentSession.") + operation + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index 21870ffd463ec..f7ee3e0a46cd1 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -231,4 +231,20 @@ class SparkMetadataOperationSuite extends HiveThriftJdbcTest { assert(!rs.next()) } } + + test("GetTypeInfo Thrift API") { + def checkResult(rs: ResultSet, typeNames: Seq[String]): Unit = { + for (i <- typeNames.indices) { + assert(rs.next()) + assert(rs.getString("TYPE_NAME") === typeNames(i)) + } + // Make sure there are no more elements + assert(!rs.next()) + } + + withJdbcStatement() { statement => + val metaData = statement.getConnection.getMetaData + checkResult(metaData.getTypeInfo, ThriftserverShimUtils.supportedType().map(_.getName)) + } + } } diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java index 0f72071d7e7d1..3e81f8afbd85f 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -73,7 +73,7 @@ public class GetTypeInfoOperation extends MetadataOperation { .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, "Usually 2 or 10"); - private final RowSet rowSet; + protected final RowSet rowSet; protected GetTypeInfoOperation(HiveSession parentSession) { super(parentSession, OperationType.GET_TYPE_INFO); diff --git a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala index 87c0f8f6a571a..837861a77bf54 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ b/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala @@ -21,6 +21,7 @@ import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema, Type} +import org.apache.hive.service.cli.Type._ import org.apache.hive.service.cli.thrift.TProtocolVersion._ /** @@ -51,6 +52,14 @@ private[thriftserver] object ThriftserverShimUtils { private[thriftserver] def toJavaSQLType(s: String): Int = Type.getType(s).toJavaSQLType + private[thriftserver] def supportedType(): Seq[Type] = { + Seq(NULL_TYPE, BOOLEAN_TYPE, STRING_TYPE, BINARY_TYPE, + TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, + FLOAT_TYPE, DOUBLE_TYPE, DECIMAL_TYPE, + DATE_TYPE, TIMESTAMP_TYPE, + ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE) + } + private[thriftserver] def addToClassPath( loader: ClassLoader, auxJars: Array[String]): ClassLoader = { diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java index 9612eb145638c..0f57a72e2a1ce 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -73,7 +73,7 @@ public class GetTypeInfoOperation extends MetadataOperation { .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, "Usually 2 or 10"); - private final RowSet rowSet; + protected final RowSet rowSet; protected GetTypeInfoOperation(HiveSession parentSession) { super(parentSession, OperationType.GET_TYPE_INFO); diff --git a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala index 124c9937c0fca..cb32ebed0ac11 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ b/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.hive.ql.exec.AddToClassPathAction import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.thrift.Type +import org.apache.hadoop.hive.serde2.thrift.Type._ import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema} import org.apache.hive.service.rpc.thrift.TProtocolVersion._ import org.slf4j.LoggerFactory @@ -56,6 +57,14 @@ private[thriftserver] object ThriftserverShimUtils { private[thriftserver] def toJavaSQLType(s: String): Int = Type.getType(s).toJavaSQLType + private[thriftserver] def supportedType(): Seq[Type] = { + Seq(NULL_TYPE, BOOLEAN_TYPE, STRING_TYPE, BINARY_TYPE, + TINYINT_TYPE, SMALLINT_TYPE, INT_TYPE, BIGINT_TYPE, + FLOAT_TYPE, DOUBLE_TYPE, DECIMAL_TYPE, + DATE_TYPE, TIMESTAMP_TYPE, + ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE) + } + private[thriftserver] def addToClassPath( loader: ClassLoader, auxJars: Array[String]): ClassLoader = { From fa75db2059abaa060c3ab58ef48d66fd3bd39750 Mon Sep 17 00:00:00 2001 From: Mick Jermsurawong Date: Wed, 11 Sep 2019 08:43:40 +0900 Subject: [PATCH 0036/1387] [SPARK-29026][SQL] Improve error message in `schemaFor` in trait without companion object constructor ### What changes were proposed in this pull request? - For trait without companion object constructor, currently the method to get constructor parameters `constructParams` in `ScalaReflection` will throw exception. ``` scala.ScalaReflectionException: is not a term at scala.reflect.api.Symbols$SymbolApi.asTerm(Symbols.scala:211) at scala.reflect.api.Symbols$SymbolApi.asTerm$(Symbols.scala:211) at scala.reflect.internal.Symbols$SymbolContextApiImpl.asTerm(Symbols.scala:106) at org.apache.spark.sql.catalyst.ScalaReflection.getCompanionConstructor(ScalaReflection.scala:909) at org.apache.spark.sql.catalyst.ScalaReflection.constructParams(ScalaReflection.scala:914) at org.apache.spark.sql.catalyst.ScalaReflection.constructParams$(ScalaReflection.scala:912) at org.apache.spark.sql.catalyst.ScalaReflection$.constructParams(ScalaReflection.scala:47) at org.apache.spark.sql.catalyst.ScalaReflection.getConstructorParameters(ScalaReflection.scala:890) at org.apache.spark.sql.catalyst.ScalaReflection.getConstructorParameters$(ScalaReflection.scala:886) at org.apache.spark.sql.catalyst.ScalaReflection$.getConstructorParameters(ScalaReflection.scala:47) ``` - Instead this PR would throw exception: ``` Unable to find constructor for type [XXX]. This could happen if [XXX] is an interface or a trait without companion object constructor UnsupportedOperationException: ``` In the normal usage of ExpressionEncoder, this can happen if the type is interface extending `scala.Product`. Also, since this is a protected method, this could have been other arbitrary types without constructor. ### Why are the changes needed? - The error message ` is not a term` isn't helpful for users to understand the problem. ### Does this PR introduce any user-facing change? - The exception would be thrown instead of runtime exception from the `scala.ScalaReflectionException`. ### How was this patch tested? - Added a unit test to illustrate the `type` where expression encoder will fail and trigger the proposed error message. Closes #25736 from mickjermsurawong-stripe/SPARK-29026. Authored-by: Mick Jermsurawong Signed-off-by: HyukjinKwon --- .../spark/sql/catalyst/ScalaReflection.scala | 13 ++++++++++- .../sql/catalyst/ScalaReflectionSuite.scala | 22 +++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 2a4e84241a941..fbbbda33538d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -906,7 +906,18 @@ trait ScalaReflection extends Logging { * only defines a constructor via `apply` method. */ private def getCompanionConstructor(tpe: Type): Symbol = { - tpe.typeSymbol.asClass.companion.asTerm.typeSignature.member(universe.TermName("apply")) + def throwUnsupportedOperation = { + throw new UnsupportedOperationException(s"Unable to find constructor for $tpe. " + + s"This could happen if $tpe is an interface, or a trait without companion object " + + "constructor.") + } + tpe.typeSymbol.asClass.companion match { + case NoSymbol => throwUnsupportedOperation + case sym => sym.asTerm.typeSignature.member(universe.TermName("apply")) match { + case NoSymbol => throwUnsupportedOperation + case constructorSym => constructorSym + } + } } protected def constructParams(tpe: Type): Seq[Symbol] = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index a2e58c3eaa0bd..f9cd9c3c398f6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -138,6 +138,14 @@ trait ScroogeLikeExample extends Product1[Int] with Serializable { override def hashCode: Int = x } +/** Counter-examples to [[ScroogeLikeExample]] as a trait without a companion object constructor */ +trait TraitProductWithoutCompanion extends Product1[Int] {} + +/** Counter-examples to [[ScroogeLikeExample]] as a trait with no-constructor companion object */ +object TraitProductWithNoConstructorCompanion {} + +trait TraitProductWithNoConstructorCompanion extends Product1[Int] {} + class ScalaReflectionSuite extends SparkFunSuite { import org.apache.spark.sql.catalyst.ScalaReflection._ @@ -404,6 +412,20 @@ class ScalaReflectionSuite extends SparkFunSuite { StructField("x", IntegerType, nullable = false))), nullable = true)) } + test("SPARK-29026: schemaFor for trait without companion object throws exception ") { + val e = intercept[UnsupportedOperationException] { + schemaFor[TraitProductWithoutCompanion] + } + assert(e.getMessage.contains("Unable to find constructor")) + } + + test("SPARK-29026: schemaFor for trait with no-constructor companion throws exception ") { + val e = intercept[UnsupportedOperationException] { + schemaFor[TraitProductWithNoConstructorCompanion] + } + assert(e.getMessage.contains("Unable to find constructor")) + } + test("SPARK-27625: annotated data types") { assert(serializerFor[FooWithAnnotation].dataType == StructType(Seq( StructField("f1", StringType), From 7f36cd2aa5e066a807d498b8c51645b136f08a75 Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 10 Sep 2019 17:30:02 -0700 Subject: [PATCH 0037/1387] [SPARK-28570][CORE][SHUFFLE] Make UnsafeShuffleWriter use the new API ## What changes were proposed in this pull request? Uses the APIs introduced in SPARK-28209 in the UnsafeShuffleWriter. ## How was this patch tested? Since this is just a refactor, existing unit tests should cover the relevant code paths. Micro-benchmarks from the original fork where this code was built show no degradation in performance. Closes #25304 from mccheah/shuffle-writer-refactor-unsafe-writer. Lead-authored-by: mcheah Co-authored-by: mccheah Signed-off-by: Marcelo Vanzin --- .../api/ShuffleExecutorComponents.java | 31 +- .../SingleSpillShuffleMapOutputWriter.java | 36 ++ .../shuffle/sort/UnsafeShuffleWriter.java | 352 +++++++++--------- .../LocalDiskShuffleExecutorComponents.java | 15 + .../io/LocalDiskShuffleMapOutputWriter.java | 24 +- .../LocalDiskSingleSpillMapOutputWriter.java | 55 +++ .../spark/internal/config/package.scala | 2 +- .../shuffle/sort/SortShuffleManager.scala | 4 +- .../sort/UnsafeShuffleWriterSuite.java | 54 ++- 9 files changed, 374 insertions(+), 199 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/shuffle/api/SingleSpillShuffleMapOutputWriter.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 70c112b78911d..804119cd06fa6 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -18,6 +18,7 @@ package org.apache.spark.shuffle.api; import java.io.IOException; +import java.util.Optional; import org.apache.spark.annotation.Private; @@ -39,17 +40,39 @@ public interface ShuffleExecutorComponents { /** * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. - * @param shuffleId Unique identifier for the shuffle the map task is a part of + * + * @param shuffleId Unique identifier for the shuffle the map task is a part of * @param mapId Within the shuffle, the identifier of the map task * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task - * with the same (shuffleId, mapId) pair can be distinguished by the - * different values of mapTaskAttemptId. + * with the same (shuffleId, mapId) pair can be distinguished by the + * different values of mapTaskAttemptId. * @param numPartitions The number of partitions that will be written by the map task. Some of -* these partitions may be empty. + * these partitions may be empty. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, int mapId, long mapTaskAttemptId, int numPartitions) throws IOException; + + /** + * An optional extension for creating a map output writer that can optimize the transfer of a + * single partition file, as the entire result of a map task, to the backing store. + *

+ * Most implementations should return the default {@link Optional#empty()} to indicate that + * they do not support this optimization. This primarily is for backwards-compatibility in + * preserving an optimization in the local disk shuffle storage implementation. + * + * @param shuffleId Unique identifier for the shuffle the map task is a part of + * @param mapId Within the shuffle, the identifier of the map task + * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task + * with the same (shuffleId, mapId) pair can be distinguished by the + * different values of mapTaskAttemptId. + */ + default Optional createSingleFileMapOutputWriter( + int shuffleId, + int mapId, + long mapTaskAttemptId) throws IOException { + return Optional.empty(); + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/SingleSpillShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/SingleSpillShuffleMapOutputWriter.java new file mode 100644 index 0000000000000..cad8dcfda52bc --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/api/SingleSpillShuffleMapOutputWriter.java @@ -0,0 +1,36 @@ +/* + * 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.shuffle.api; + +import java.io.File; +import java.io.IOException; + +import org.apache.spark.annotation.Private; + +/** + * Optional extension for partition writing that is optimized for transferring a single + * file to the backing store. + */ +@Private +public interface SingleSpillShuffleMapOutputWriter { + + /** + * Transfer a file that contains the bytes of all the partitions written by this map task. + */ + void transferMapSpillFile(File mapOutputFile, long[] partitionLengths) throws IOException; +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 9d05f03613ce9..f59bddc993639 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -17,9 +17,12 @@ package org.apache.spark.shuffle.sort; +import java.nio.channels.Channels; +import java.util.Optional; import javax.annotation.Nullable; import java.io.*; import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; import java.util.Iterator; import scala.Option; @@ -31,7 +34,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; -import com.google.common.io.Files; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,8 +43,6 @@ import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.NioBufferedFileInputStream; -import org.apache.commons.io.output.CloseShieldOutputStream; -import org.apache.commons.io.output.CountingOutputStream; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.MapStatus; @@ -50,10 +50,13 @@ import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.serializer.SerializationStream; import org.apache.spark.serializer.SerializerInstance; -import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; +import org.apache.spark.shuffle.api.ShuffleExecutorComponents; +import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; +import org.apache.spark.shuffle.api.ShufflePartitionWriter; +import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter; +import org.apache.spark.shuffle.api.WritableByteChannelWrapper; import org.apache.spark.storage.BlockManager; -import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; @@ -65,15 +68,14 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private static final ClassTag OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object(); @VisibleForTesting - static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024; private final BlockManager blockManager; - private final IndexShuffleBlockResolver shuffleBlockResolver; private final TaskMemoryManager memoryManager; private final SerializerInstance serializer; private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; + private final ShuffleExecutorComponents shuffleExecutorComponents; private final int shuffleId; private final int mapId; private final TaskContext taskContext; @@ -81,7 +83,6 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final boolean transferToEnabled; private final int initialSortBufferSize; private final int inputBufferSizeInBytes; - private final int outputBufferSizeInBytes; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -103,27 +104,15 @@ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream */ private boolean stopping = false; - private class CloseAndFlushShieldOutputStream extends CloseShieldOutputStream { - - CloseAndFlushShieldOutputStream(OutputStream outputStream) { - super(outputStream); - } - - @Override - public void flush() { - // do nothing - } - } - public UnsafeShuffleWriter( BlockManager blockManager, - IndexShuffleBlockResolver shuffleBlockResolver, TaskMemoryManager memoryManager, SerializedShuffleHandle handle, int mapId, TaskContext taskContext, SparkConf sparkConf, - ShuffleWriteMetricsReporter writeMetrics) throws IOException { + ShuffleWriteMetricsReporter writeMetrics, + ShuffleExecutorComponents shuffleExecutorComponents) { final int numPartitions = handle.dependency().partitioner().numPartitions(); if (numPartitions > SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE()) { throw new IllegalArgumentException( @@ -132,7 +121,6 @@ public UnsafeShuffleWriter( " reduce partitions"); } this.blockManager = blockManager; - this.shuffleBlockResolver = shuffleBlockResolver; this.memoryManager = memoryManager; this.mapId = mapId; final ShuffleDependency dep = handle.dependency(); @@ -140,6 +128,7 @@ public UnsafeShuffleWriter( this.serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); this.writeMetrics = writeMetrics; + this.shuffleExecutorComponents = shuffleExecutorComponents; this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); @@ -147,8 +136,6 @@ public UnsafeShuffleWriter( (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()); this.inputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; - this.outputBufferSizeInBytes = - (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); } @@ -231,22 +218,13 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; final long[] partitionLengths; - final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); - final File tmp = Utils.tempFileWith(output); try { - try { - partitionLengths = mergeSpills(spills, tmp); - } finally { - for (SpillInfo spill : spills) { - if (spill.file.exists() && ! spill.file.delete()) { - logger.error("Error while deleting spill file {}", spill.file.getPath()); - } - } - } - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + partitionLengths = mergeSpills(spills); } finally { - if (tmp.exists() && !tmp.delete()) { - logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + for (SpillInfo spill : spills) { + if (spill.file.exists() && !spill.file.delete()) { + logger.error("Error while deleting spill file {}", spill.file.getPath()); + } } } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); @@ -281,137 +259,161 @@ void forceSorterToSpill() throws IOException { * * @return the partition lengths in the merged file. */ - private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { + private long[] mergeSpills(SpillInfo[] spills) throws IOException { + long[] partitionLengths; + if (spills.length == 0) { + final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents + .createMapOutputWriter( + shuffleId, + mapId, + taskContext.taskAttemptId(), + partitioner.numPartitions()); + return mapWriter.commitAllPartitions(); + } else if (spills.length == 1) { + Optional maybeSingleFileWriter = + shuffleExecutorComponents.createSingleFileMapOutputWriter( + shuffleId, mapId, taskContext.taskAttemptId()); + if (maybeSingleFileWriter.isPresent()) { + // Here, we don't need to perform any metrics updates because the bytes written to this + // output file would have already been counted as shuffle bytes written. + partitionLengths = spills[0].partitionLengths; + maybeSingleFileWriter.get().transferMapSpillFile(spills[0].file, partitionLengths); + } else { + partitionLengths = mergeSpillsUsingStandardWriter(spills); + } + } else { + partitionLengths = mergeSpillsUsingStandardWriter(spills); + } + return partitionLengths; + } + + private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOException { + long[] partitionLengths; final boolean compressionEnabled = (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_COMPRESS()); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = - (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_UNDAFE_FAST_MERGE_ENABLE()); + (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FAST_MERGE_ENABLE()); final boolean fastMergeIsSupported = !compressionEnabled || - CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); + CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); + final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents + .createMapOutputWriter( + shuffleId, + mapId, + taskContext.taskAttemptId(), + partitioner.numPartitions()); try { - if (spills.length == 0) { - new FileOutputStream(outputFile).close(); // Create an empty file - return new long[partitioner.numPartitions()]; - } else if (spills.length == 1) { - // Here, we don't need to perform any metrics updates because the bytes written to this - // output file would have already been counted as shuffle bytes written. - Files.move(spills[0].file, outputFile); - return spills[0].partitionLengths; - } else { - final long[] partitionLengths; - // There are multiple spills to merge, so none of these spill files' lengths were counted - // towards our shuffle write count or shuffle write time. If we use the slow merge path, - // then the final output file's size won't necessarily be equal to the sum of the spill - // files' sizes. To guard against this case, we look at the output file's actual size when - // computing shuffle bytes written. - // - // We allow the individual merge methods to report their own IO times since different merge - // strategies use different IO techniques. We count IO during merge towards the shuffle - // shuffle write time, which appears to be consistent with the "not bypassing merge-sort" - // branch in ExternalSorter. - if (fastMergeEnabled && fastMergeIsSupported) { - // Compression is disabled or we are using an IO compression codec that supports - // decompression of concatenated compressed streams, so we can perform a fast spill merge - // that doesn't need to interpret the spilled bytes. - if (transferToEnabled && !encryptionEnabled) { - logger.debug("Using transferTo-based fast merge"); - partitionLengths = mergeSpillsWithTransferTo(spills, outputFile); - } else { - logger.debug("Using fileStream-based fast merge"); - partitionLengths = mergeSpillsWithFileStream(spills, outputFile, null); - } + // There are multiple spills to merge, so none of these spill files' lengths were counted + // towards our shuffle write count or shuffle write time. If we use the slow merge path, + // then the final output file's size won't necessarily be equal to the sum of the spill + // files' sizes. To guard against this case, we look at the output file's actual size when + // computing shuffle bytes written. + // + // We allow the individual merge methods to report their own IO times since different merge + // strategies use different IO techniques. We count IO during merge towards the shuffle + // write time, which appears to be consistent with the "not bypassing merge-sort" branch in + // ExternalSorter. + if (fastMergeEnabled && fastMergeIsSupported) { + // Compression is disabled or we are using an IO compression codec that supports + // decompression of concatenated compressed streams, so we can perform a fast spill merge + // that doesn't need to interpret the spilled bytes. + if (transferToEnabled && !encryptionEnabled) { + logger.debug("Using transferTo-based fast merge"); + mergeSpillsWithTransferTo(spills, mapWriter); } else { - logger.debug("Using slow merge"); - partitionLengths = mergeSpillsWithFileStream(spills, outputFile, compressionCodec); + logger.debug("Using fileStream-based fast merge"); + mergeSpillsWithFileStream(spills, mapWriter, null); } - // When closing an UnsafeShuffleExternalSorter that has already spilled once but also has - // in-memory records, we write out the in-memory records to a file but do not count that - // final write as bytes spilled (instead, it's accounted as shuffle write). The merge needs - // to be counted as shuffle write, but this will lead to double-counting of the final - // SpillInfo's bytes. - writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); - writeMetrics.incBytesWritten(outputFile.length()); - return partitionLengths; + } else { + logger.debug("Using slow merge"); + mergeSpillsWithFileStream(spills, mapWriter, compressionCodec); } - } catch (IOException e) { - if (outputFile.exists() && !outputFile.delete()) { - logger.error("Unable to delete output file {}", outputFile.getPath()); + // When closing an UnsafeShuffleExternalSorter that has already spilled once but also has + // in-memory records, we write out the in-memory records to a file but do not count that + // final write as bytes spilled (instead, it's accounted as shuffle write). The merge needs + // to be counted as shuffle write, but this will lead to double-counting of the final + // SpillInfo's bytes. + writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); + partitionLengths = mapWriter.commitAllPartitions(); + } catch (Exception e) { + try { + mapWriter.abort(e); + } catch (Exception e2) { + logger.warn("Failed to abort writing the map output.", e2); + e.addSuppressed(e2); } throw e; } + return partitionLengths; } /** * Merges spill files using Java FileStreams. This code path is typically slower than * the NIO-based merge, {@link UnsafeShuffleWriter#mergeSpillsWithTransferTo(SpillInfo[], - * File)}, and it's mostly used in cases where the IO compression codec does not support - * concatenation of compressed data, when encryption is enabled, or when users have - * explicitly disabled use of {@code transferTo} in order to work around kernel bugs. + * ShuffleMapOutputWriter)}, and it's mostly used in cases where the IO compression codec + * does not support concatenation of compressed data, when encryption is enabled, or when + * users have explicitly disabled use of {@code transferTo} in order to work around kernel bugs. * This code path might also be faster in cases where individual partition size in a spill * is small and UnsafeShuffleWriter#mergeSpillsWithTransferTo method performs many small * disk ios which is inefficient. In those case, Using large buffers for input and output * files helps reducing the number of disk ios, making the file merging faster. * * @param spills the spills to merge. - * @param outputFile the file to write the merged data to. + * @param mapWriter the map output writer to use for output. * @param compressionCodec the IO compression codec, or null if shuffle compression is disabled. * @return the partition lengths in the merged file. */ - private long[] mergeSpillsWithFileStream( + private void mergeSpillsWithFileStream( SpillInfo[] spills, - File outputFile, + ShuffleMapOutputWriter mapWriter, @Nullable CompressionCodec compressionCodec) throws IOException { - assert (spills.length >= 2); final int numPartitions = partitioner.numPartitions(); - final long[] partitionLengths = new long[numPartitions]; final InputStream[] spillInputStreams = new InputStream[spills.length]; - final OutputStream bos = new BufferedOutputStream( - new FileOutputStream(outputFile), - outputBufferSizeInBytes); - // Use a counting output stream to avoid having to close the underlying file and ask - // the file system for its size after each partition is written. - final CountingOutputStream mergedFileOutputStream = new CountingOutputStream(bos); - boolean threwException = true; try { for (int i = 0; i < spills.length; i++) { spillInputStreams[i] = new NioBufferedFileInputStream( - spills[i].file, - inputBufferSizeInBytes); + spills[i].file, + inputBufferSizeInBytes); } for (int partition = 0; partition < numPartitions; partition++) { - final long initialFileLength = mergedFileOutputStream.getByteCount(); - // Shield the underlying output stream from close() and flush() calls, so that we can close - // the higher level streams to make sure all data is really flushed and internal state is - // cleaned. - OutputStream partitionOutput = new CloseAndFlushShieldOutputStream( - new TimeTrackingOutputStream(writeMetrics, mergedFileOutputStream)); - partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); - if (compressionCodec != null) { - partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); - } - for (int i = 0; i < spills.length; i++) { - final long partitionLengthInSpill = spills[i].partitionLengths[partition]; - if (partitionLengthInSpill > 0) { - InputStream partitionInputStream = new LimitedInputStream(spillInputStreams[i], - partitionLengthInSpill, false); - try { - partitionInputStream = blockManager.serializerManager().wrapForEncryption( - partitionInputStream); - if (compressionCodec != null) { - partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream); + boolean copyThrewException = true; + ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); + OutputStream partitionOutput = writer.openStream(); + try { + partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); + if (compressionCodec != null) { + partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); + } + for (int i = 0; i < spills.length; i++) { + final long partitionLengthInSpill = spills[i].partitionLengths[partition]; + + if (partitionLengthInSpill > 0) { + InputStream partitionInputStream = null; + boolean copySpillThrewException = true; + try { + partitionInputStream = new LimitedInputStream(spillInputStreams[i], + partitionLengthInSpill, false); + partitionInputStream = blockManager.serializerManager().wrapForEncryption( + partitionInputStream); + if (compressionCodec != null) { + partitionInputStream = compressionCodec.compressedInputStream( + partitionInputStream); + } + ByteStreams.copy(partitionInputStream, partitionOutput); + copySpillThrewException = false; + } finally { + Closeables.close(partitionInputStream, copySpillThrewException); } - ByteStreams.copy(partitionInputStream, partitionOutput); - } finally { - partitionInputStream.close(); } } + copyThrewException = false; + } finally { + Closeables.close(partitionOutput, copyThrewException); } - partitionOutput.flush(); - partitionOutput.close(); - partitionLengths[partition] = (mergedFileOutputStream.getByteCount() - initialFileLength); + long numBytesWritten = writer.getNumBytesWritten(); + writeMetrics.incBytesWritten(numBytesWritten); } threwException = false; } finally { @@ -420,9 +422,7 @@ private long[] mergeSpillsWithFileStream( for (InputStream stream : spillInputStreams) { Closeables.close(stream, threwException); } - Closeables.close(mergedFileOutputStream, threwException); } - return partitionLengths; } /** @@ -430,54 +430,46 @@ private long[] mergeSpillsWithFileStream( * This is only safe when the IO compression codec and serializer support concatenation of * serialized streams. * + * @param spills the spills to merge. + * @param mapWriter the map output writer to use for output. * @return the partition lengths in the merged file. */ - private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) throws IOException { - assert (spills.length >= 2); + private void mergeSpillsWithTransferTo( + SpillInfo[] spills, + ShuffleMapOutputWriter mapWriter) throws IOException { final int numPartitions = partitioner.numPartitions(); - final long[] partitionLengths = new long[numPartitions]; final FileChannel[] spillInputChannels = new FileChannel[spills.length]; final long[] spillInputChannelPositions = new long[spills.length]; - FileChannel mergedFileOutputChannel = null; boolean threwException = true; try { for (int i = 0; i < spills.length; i++) { spillInputChannels[i] = new FileInputStream(spills[i].file).getChannel(); } - // This file needs to opened in append mode in order to work around a Linux kernel bug that - // affects transferTo; see SPARK-3948 for more details. - mergedFileOutputChannel = new FileOutputStream(outputFile, true).getChannel(); - - long bytesWrittenToMergedFile = 0; for (int partition = 0; partition < numPartitions; partition++) { - for (int i = 0; i < spills.length; i++) { - final long partitionLengthInSpill = spills[i].partitionLengths[partition]; - final FileChannel spillInputChannel = spillInputChannels[i]; - final long writeStartTime = System.nanoTime(); - Utils.copyFileStreamNIO( - spillInputChannel, - mergedFileOutputChannel, - spillInputChannelPositions[i], - partitionLengthInSpill); - spillInputChannelPositions[i] += partitionLengthInSpill; - writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); - bytesWrittenToMergedFile += partitionLengthInSpill; - partitionLengths[partition] += partitionLengthInSpill; + boolean copyThrewException = true; + ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); + WritableByteChannelWrapper resolvedChannel = writer.openChannelWrapper() + .orElseGet(() -> new StreamFallbackChannelWrapper(openStreamUnchecked(writer))); + try { + for (int i = 0; i < spills.length; i++) { + long partitionLengthInSpill = spills[i].partitionLengths[partition]; + final FileChannel spillInputChannel = spillInputChannels[i]; + final long writeStartTime = System.nanoTime(); + Utils.copyFileStreamNIO( + spillInputChannel, + resolvedChannel.channel(), + spillInputChannelPositions[i], + partitionLengthInSpill); + copyThrewException = false; + spillInputChannelPositions[i] += partitionLengthInSpill; + writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); + } + } finally { + Closeables.close(resolvedChannel, copyThrewException); } - } - // Check the position after transferTo loop to see if it is in the right position and raise an - // exception if it is incorrect. The position will not be increased to the expected length - // after calling transferTo in kernel version 2.6.32. This issue is described at - // https://bugs.openjdk.java.net/browse/JDK-7052359 and SPARK-3948. - if (mergedFileOutputChannel.position() != bytesWrittenToMergedFile) { - throw new IOException( - "Current position " + mergedFileOutputChannel.position() + " does not equal expected " + - "position " + bytesWrittenToMergedFile + " after transferTo. Please check your kernel" + - " version to see if it is 2.6.32, as there is a kernel bug which will lead to " + - "unexpected behavior when using transferTo. You can set spark.file.transferTo=false " + - "to disable this NIO feature." - ); + long numBytes = writer.getNumBytesWritten(); + writeMetrics.incBytesWritten(numBytes); } threwException = false; } finally { @@ -487,9 +479,7 @@ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) th assert(spillInputChannelPositions[i] == spills[i].file.length()); Closeables.close(spillInputChannels[i], threwException); } - Closeables.close(mergedFileOutputChannel, threwException); } - return partitionLengths; } @Override @@ -518,4 +508,30 @@ public Option stop(boolean success) { } } } + + private static OutputStream openStreamUnchecked(ShufflePartitionWriter writer) { + try { + return writer.openStream(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static final class StreamFallbackChannelWrapper implements WritableByteChannelWrapper { + private final WritableByteChannel channel; + + StreamFallbackChannelWrapper(OutputStream fallbackStream) { + this.channel = Channels.newChannel(fallbackStream); + } + + @Override + public WritableByteChannel channel() { + return channel; + } + + @Override + public void close() throws IOException { + channel.close(); + } + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index 02eb710737285..47aa2e39fe29b 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -17,6 +17,8 @@ package org.apache.spark.shuffle.sort.io; +import java.util.Optional; + import com.google.common.annotations.VisibleForTesting; import org.apache.spark.SparkConf; @@ -24,6 +26,7 @@ import org.apache.spark.shuffle.api.ShuffleExecutorComponents; import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter; import org.apache.spark.storage.BlockManager; public class LocalDiskShuffleExecutorComponents implements ShuffleExecutorComponents { @@ -68,4 +71,16 @@ public ShuffleMapOutputWriter createMapOutputWriter( return new LocalDiskShuffleMapOutputWriter( shuffleId, mapId, numPartitions, blockResolver, sparkConf); } + + @Override + public Optional createSingleFileMapOutputWriter( + int shuffleId, + int mapId, + long mapTaskAttemptId) { + if (blockResolver == null) { + throw new IllegalStateException( + "Executor components must be initialized before getting writers."); + } + return Optional.of(new LocalDiskSingleSpillMapOutputWriter(shuffleId, mapId, blockResolver)); + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 7fc19b1270a46..444cdc4270ecd 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -24,8 +24,8 @@ import java.io.OutputStream; import java.nio.channels.FileChannel; import java.nio.channels.WritableByteChannel; - import java.util.Optional; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +54,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { private final int bufferSize; private int lastPartitionId = -1; private long currChannelPosition; + private long bytesWrittenToMergedFile = 0L; private final File outputFile; private File outputTempFile; @@ -97,6 +98,18 @@ public ShufflePartitionWriter getPartitionWriter(int reducePartitionId) throws I @Override public long[] commitAllPartitions() throws IOException { + // Check the position after transferTo loop to see if it is in the right position and raise a + // exception if it is incorrect. The position will not be increased to the expected length + // after calling transferTo in kernel version 2.6.32. This issue is described at + // https://bugs.openjdk.java.net/browse/JDK-7052359 and SPARK-3948. + if (outputFileChannel != null && outputFileChannel.position() != bytesWrittenToMergedFile) { + throw new IOException( + "Current position " + outputFileChannel.position() + " does not equal expected " + + "position " + bytesWrittenToMergedFile + " after transferTo. Please check your " + + " kernel version to see if it is 2.6.32, as there is a kernel bug which will lead " + + "to unexpected behavior when using transferTo. You can set " + + "spark.file.transferTo=false to disable this NIO feature."); + } cleanUp(); File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); @@ -133,11 +146,10 @@ private void initStream() throws IOException { } private void initChannel() throws IOException { - if (outputFileStream == null) { - outputFileStream = new FileOutputStream(outputTempFile, true); - } + // This file needs to opened in append mode in order to work around a Linux kernel bug that + // affects transferTo; see SPARK-3948 for more details. if (outputFileChannel == null) { - outputFileChannel = outputFileStream.getChannel(); + outputFileChannel = new FileOutputStream(outputTempFile, true).getChannel(); } } @@ -227,6 +239,7 @@ public void write(byte[] buf, int pos, int length) throws IOException { public void close() { isClosed = true; partitionLengths[partitionId] = count; + bytesWrittenToMergedFile += count; } private void verifyNotClosed() { @@ -257,6 +270,7 @@ public WritableByteChannel channel() { @Override public void close() throws IOException { partitionLengths[partitionId] = getCount(); + bytesWrittenToMergedFile += partitionLengths[partitionId]; } } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java new file mode 100644 index 0000000000000..6b0a797a61b52 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java @@ -0,0 +1,55 @@ +/* + * 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.shuffle.sort.io; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; + +import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter; +import org.apache.spark.util.Utils; + +public class LocalDiskSingleSpillMapOutputWriter + implements SingleSpillShuffleMapOutputWriter { + + private final int shuffleId; + private final int mapId; + private final IndexShuffleBlockResolver blockResolver; + + public LocalDiskSingleSpillMapOutputWriter( + int shuffleId, + int mapId, + IndexShuffleBlockResolver blockResolver) { + this.shuffleId = shuffleId; + this.mapId = mapId; + this.blockResolver = blockResolver; + } + + @Override + public void transferMapSpillFile( + File mapSpillFile, + long[] partitionLengths) throws IOException { + // The map spill file already has the proper format, and it contains all of the partition data. + // So just transfer it directly to the destination without any merging. + File outputFile = blockResolver.getDataFile(shuffleId, mapId); + File tempFile = Utils.tempFileWith(outputFile); + Files.move(mapSpillFile.toPath(), tempFile.toPath()); + blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tempFile); + } +} 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 b898413ac8d76..158a4b7cfa55a 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 @@ -1026,7 +1026,7 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val SHUFFLE_UNDAFE_FAST_MERGE_ENABLE = + private[spark] val SHUFFLE_UNSAFE_FAST_MERGE_ENABLE = ConfigBuilder("spark.shuffle.unsafe.fastMergeEnabled") .doc("Whether to perform a fast spill merge.") .booleanConf diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 2a99c93b32af4..cbdc2c886dd9f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -140,13 +140,13 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => new UnsafeShuffleWriter( env.blockManager, - shuffleBlockResolver, context.taskMemoryManager(), unsafeShuffleHandle, mapId, context, env.conf, - metrics) + metrics, + shuffleExecutorComponents) case bypassMergeSortHandle: BypassMergeSortShuffleHandle[K @unchecked, V @unchecked] => new BypassMergeSortShuffleWriter( env.blockManager, diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 6b83a984f037c..1022111897a49 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -19,8 +19,10 @@ import java.io.*; import java.nio.ByteBuffer; +import java.nio.file.Files; import java.util.*; +import org.mockito.stubbing.Answer; import scala.Option; import scala.Product2; import scala.Tuple2; @@ -53,6 +55,7 @@ import org.apache.spark.security.CryptoStreamUtils; import org.apache.spark.serializer.*; import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents; import org.apache.spark.storage.*; import org.apache.spark.util.Utils; @@ -65,6 +68,7 @@ public class UnsafeShuffleWriterSuite { + static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; static final int NUM_PARTITITONS = 4; TestMemoryManager memoryManager; TaskMemoryManager taskMemoryManager; @@ -132,14 +136,28 @@ public void setUp() throws IOException { }); when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); - doAnswer(invocationOnMock -> { + + Answer renameTempAnswer = invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; File tmp = (File) invocationOnMock.getArguments()[3]; - mergedOutputFile.delete(); - tmp.renameTo(mergedOutputFile); + if (!mergedOutputFile.delete()) { + throw new RuntimeException("Failed to delete old merged output file."); + } + if (tmp != null) { + Files.move(tmp.toPath(), mergedOutputFile.toPath()); + } else if (!mergedOutputFile.createNewFile()) { + throw new RuntimeException("Failed to create empty merged output file."); + } return null; - }).when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); + }; + + doAnswer(renameTempAnswer) + .when(shuffleBlockResolver) + .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); + + doAnswer(renameTempAnswer) + .when(shuffleBlockResolver) + .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), eq(null)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); @@ -151,21 +169,20 @@ public void setUp() throws IOException { when(taskContext.taskMetrics()).thenReturn(taskMetrics); when(shuffleDep.serializer()).thenReturn(serializer); when(shuffleDep.partitioner()).thenReturn(hashPartitioner); + when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager); } - private UnsafeShuffleWriter createWriter( - boolean transferToEnabled) throws IOException { + private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); - return new UnsafeShuffleWriter<>( + return new UnsafeShuffleWriter( blockManager, - shuffleBlockResolver, - taskMemoryManager, + taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, conf, - taskContext.taskMetrics().shuffleWriteMetrics() - ); + taskContext.taskMetrics().shuffleWriteMetrics(), + new LocalDiskShuffleExecutorComponents(conf, blockManager, shuffleBlockResolver)); } private void assertSpillFilesWereCleanedUp() { @@ -391,7 +408,7 @@ public void mergeSpillsWithFileStreamAndCompressionAndEncryption() throws Except @Test public void mergeSpillsWithCompressionAndEncryptionSlowPath() throws Exception { - conf.set(package$.MODULE$.SHUFFLE_UNDAFE_FAST_MERGE_ENABLE(), false); + conf.set(package$.MODULE$.SHUFFLE_UNSAFE_FAST_MERGE_ENABLE(), false); testMergingSpills(false, LZ4CompressionCodec.class.getName(), true); } @@ -444,10 +461,10 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOn() thro } private void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exception { - memoryManager.limit(UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE * 16); + memoryManager.limit(DEFAULT_INITIAL_SORT_BUFFER_SIZE * 16); final UnsafeShuffleWriter writer = createWriter(false); final ArrayList> dataToWrite = new ArrayList<>(); - for (int i = 0; i < UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { + for (int i = 0; i < DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { dataToWrite.add(new Tuple2<>(i, i)); } writer.write(dataToWrite.iterator()); @@ -516,16 +533,15 @@ public void testPeakMemoryUsed() throws Exception { final long numRecordsPerPage = pageSizeBytes / recordLengthBytes; taskMemoryManager = spy(taskMemoryManager); when(taskMemoryManager.pageSizeBytes()).thenReturn(pageSizeBytes); - final UnsafeShuffleWriter writer = - new UnsafeShuffleWriter<>( + final UnsafeShuffleWriter writer = new UnsafeShuffleWriter( blockManager, - shuffleBlockResolver, taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, conf, - taskContext.taskMetrics().shuffleWriteMetrics()); + taskContext.taskMetrics().shuffleWriteMetrics(), + new LocalDiskShuffleExecutorComponents(conf, blockManager, shuffleBlockResolver)); // Peak memory should be monotonically increasing. More specifically, every time // we allocate a new page it should increase by exactly the size of the page. From 8d1b5ba76651c049c957ad48ae60f2c8656dff49 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 11 Sep 2019 08:12:44 -0500 Subject: [PATCH 0038/1387] [SPARK-28906][BUILD] Fix incorrect information in bin/spark-submit --version ### What changes were proposed in this pull request? This PR allows `bin/spark-submit --version` to show the correct information while the previous versions, which were created by `dev/create-release/do-release-docker.sh`, show incorrect information. There are two root causes to show incorrect information: 1. Did not pass `USER` environment variable to the docker container 1. Did not keep `.git` directory in the work directory ### Why are the changes needed? The information is missing while the previous versions show the correct information. ### Does this PR introduce any user-facing change? Yes, the following is the console output in branch-2.3 ``` $ bin/spark-submit --version Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.3.4 /_/ Using Scala version 2.11.8, OpenJDK 64-Bit Server VM, 1.8.0_212 Branch HEAD Compiled by user ishizaki on 2019-09-02T02:18:10Z Revision 8c6f8150f3c6298ff4e1c7e06028f12d7eaf0210 Url https://gitbox.apache.org/repos/asf/spark.git Type --help for more information. ``` Without this PR, the console output is as follows ``` $ spark-submit --version Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.3.4 /_/ Using Scala version 2.11.8, OpenJDK 64-Bit Server VM, 1.8.0_212 Branch Compiled by user on 2019-08-26T08:29:39Z Revision Url Type --help for more information. ``` ### How was this patch tested? After building the package, I manually executed `bin/spark-submit --version` Closes #25655 from kiszk/SPARK-28906. Authored-by: Kazuaki Ishizaki Signed-off-by: Sean Owen --- dev/create-release/do-release-docker.sh | 1 + dev/create-release/release-build.sh | 3 +-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index c1a122ebfb12e..f643c060eb321 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -127,6 +127,7 @@ GPG_KEY=$GPG_KEY ASF_PASSWORD=$ASF_PASSWORD GPG_PASSPHRASE=$GPG_PASSPHRASE RELEASE_STEP=$RELEASE_STEP +USER=$USER EOF JAVA_VOL= diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index f35bc4f48652b..61951e73f4bab 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -164,7 +164,6 @@ DEST_DIR_NAME="$SPARK_PACKAGE_VERSION" git clean -d -f -x rm .gitignore -rm -rf .git cd .. if [[ "$1" == "package" ]]; then @@ -179,7 +178,7 @@ if [[ "$1" == "package" ]]; then rm -r spark-$SPARK_VERSION/licenses-binary fi - tar cvzf spark-$SPARK_VERSION.tgz spark-$SPARK_VERSION + tar cvzf spark-$SPARK_VERSION.tgz --exclude spark-$SPARK_VERSION/.git spark-$SPARK_VERSION echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour --output spark-$SPARK_VERSION.tgz.asc \ --detach-sig spark-$SPARK_VERSION.tgz echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \ From b425f8ee6599f53f47d7d4a8f0c27f2ba7d2eab9 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 11 Sep 2019 08:22:36 -0500 Subject: [PATCH 0039/1387] [SPARK-27492][DOC][YARN][K8S][CORE] Resource scheduling high level user docs ### What changes were proposed in this pull request? Document the resource scheduling feature - https://issues.apache.org/jira/browse/SPARK-24615 Add general docs, yarn, kubernetes, and standalone cluster specific ones. ### Why are the changes needed? Help users understand the feature ### Does this PR introduce any user-facing change? docs ### How was this patch tested? N/A Closes #25698 from tgravescs/SPARK-27492-gpu-sched-docs. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- docs/configuration.md | 14 +++++++++++++- docs/running-on-kubernetes.md | 11 +++++++++++ docs/running-on-yarn.md | 14 ++++++++++++++ docs/spark-standalone.md | 12 ++++++++++++ 4 files changed, 50 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 9933283cdad87..5cf42d5fe8dd2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -230,7 +230,7 @@ of the most common options to set are: write to STDOUT a JSON string in the format of the ResourceInformation class. This has a name and an array of addresses. For a client-submitted driver in Standalone, discovery script must assign different resource addresses to this driver comparing to workers' and - other dirvers' when spark.resources.coordinate.enable is off. + other drivers' when spark.resources.coordinate.enable is off. @@ -2617,3 +2617,15 @@ Also, you can modify or add configurations at runtime: --conf spark.hadoop.abc.def=xyz \ myApp.jar {% endhighlight %} + +# Custom Resource Scheduling and Configuration Overview + +GPUs and other accelerators have been widely used for accelerating special workloads, e.g., +deep learning and signal processing. Spark now supports requesting and scheduling generic resources, such as GPUs, with a few caveats. The current implementation requires that the resource have addresses that can be allocated by the scheduler. It requires your cluster manager to support and be properly configured with the resources. + +There are configurations available to request resources for the driver: spark.driver.resource.{resourceName}.amount, request resources for the executor(s): spark.executor.resource.{resourceName}.amount and specify the requirements for each task: spark.task.resource.{resourceName}.amount. The spark.driver.resource.{resourceName}.discoveryScript config is required on YARN, Kubernetes and a client side Driver on Spark Standalone. spark.driver.executor.{resourceName}.discoveryScript config is required for YARN and Kubernetes. Kubernetes also requires spark.driver.resource.{resourceName}.vendor and/or spark.executor.resource.{resourceName}.vendor. See the config descriptions above for more information on each. + +Spark will use the configurations specified to first request containers with the corresponding resources from the cluster manager. Once it gets the container, Spark launches an Executor in that container which will discover what resources the container has and the addresses associated with each resource. The Executor will register with the Driver and report back the resources available to that Executor. The Spark scheduler can then schedule tasks to each Executor and assign specific resource addresses based on the resource requirements the user specified. The user can see the resources assigned to a task using the TaskContext.get().resources api. On the driver, the user can see the resources assigned with the SparkContext resources call. It's then up to the user to use the assignedaddresses to do the processing they want or pass those into the ML/AI framework they are using. + +See your cluster manager specific page for requirements and details on each of - [YARN](running-on-yarn.html#resource-allocation-and-configuration-overview), [Kubernetes](running-on-kubernetes.html#resource-allocation-and-configuration-overview) and [Standalone Mode](spark-standalone.html#resource-allocation-and-configuration-overview). It is currently not available with Mesos or local mode. If using local-cluster mode see the Spark Standalone documentation but be aware only a single worker resources file or discovery script can be specified the is shared by all the Workers so you should enable resource coordination (see spark.resources.coordinate.enable). + diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 2d4e5cd65f497..4ef738ed9ef6e 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1266,3 +1266,14 @@ The following affect the driver and executor containers. All other containers in + +### Resource Allocation and Configuration Overview + +Please make sure to have read the Custom Resource Scheduling and Configuration Overview section on the [configuration page](configuration.html). This section only talks about the Kubernetes specific aspects of resource scheduling. + +The user is responsible to properly configuring the Kubernetes cluster to have the resources available and ideally isolate each resource per container so that a resource is not shared between multiple containers. If the resource is not isolated the user is responsible for writing a discovery script so that the resource is not shared between containers. See the Kubernetes documentation for specifics on configuring Kubernetes with [custom resources](https://kubernetes.io/docs/concepts/extend-kubernetes/compute-storage-net/device-plugins/). + +Spark automatically handles translating the Spark configs spark.{driver/executor}.resource.{resourceType} into the kubernetes configs as long as the Kubernetes resource type follows the Kubernetes device plugin format of `vendor-domain/resourcetype`. The user must specify the vendor using the spark.{driver/executor}.resource.{resourceType}.vendor config. The user does not need to explicitly add anything if you are using Pod templates. For reference and an example, you can see the Kubernetes documentation for scheduling [GPUs](https://kubernetes.io/docs/tasks/manage-gpus/scheduling-gpus/). Spark only supports setting the resource limits. + +Kubernetes does not tell Spark the addresses of the resources allocated to each container. For that reason, the user must specify a discovery script that gets run by the executor on startup to discover what resources are available to that executor. You can find an example scripts in `examples/src/main/scripts/getGpusResources.sh`. The script must have execute permissions set and the user should setup permissions to not allow malicious users to modify it. The script should write to STDOUT a JSON string in the format of the ResourceInformation class. This has the resource name and an array of resource addresses available to just that executor. + diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d3d049e6fef70..418db41216cdb 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -542,6 +542,20 @@ For example, suppose you would like to point log url link to Job History Server NOTE: you need to replace `` and `` with actual value. +# Resource Allocation and Configuration Overview + +Please make sure to have read the Custom Resource Scheduling and Configuration Overview section on the [configuration page](configuration.html). This section only talks about the YARN specific aspects of resource scheduling. + +YARN needs to be configured to support any resources the user wants to use with Spark. Resource scheduling on YARN was added in YARN 3.1.0. See the YARN documentation for more information on configuring resources and properly setting up isolation. Ideally the resources are setup isolated so that an executor can only see the resources it was allocated. If you do not have isolation enabled, the user is responsible for creating a discovery script that ensures the resource is not shared between executors. + +YARN currently supports any user defined resource type but has built in types for GPU (yarn.io/gpu) and FPGA (yarn.io/fpga). For that reason, if you are using either of those resources, Spark can translate your request for spark resources into YARN resources and you only have to specify the spark.{driver/executor}.resource. configs. If you are using a resource other then FPGA or GPU, the user is responsible for specifying the configs for both YARN (spark.yarn.{driver/executor}.resource.) and Spark (spark.{driver/executor}.resource.). + +For example, the user wants to request 2 GPUs for each executor. The user can just specify spark.executor.resource.gpu.amount=2 and Spark will handle requesting yarn.io/gpu resource type from YARN. + +If the user has a user defined YARN resource, lets call it `acceleratorX` then the user must specify spark.yarn.executor.resource.acceleratorX.amount=2 and spark.executor.resource.acceleratorX.amount=2. + +YARN does not tell Spark the addresses of the resources allocated to each container. For that reason, the user must specify a discovery script that gets run by the executor on startup to discover what resources are available to that executor. You can find an example scripts in `examples/src/main/scripts/getGpusResources.sh`. The script must have execute permissions set and the user should setup permissions to not allow malicious users to modify it. The script should write to STDOUT a JSON string in the format of the ResourceInformation class. This has the resource name and an array of resource addresses available to just that executor. + # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index bc77469b6664f..1af0befdc70d7 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -340,6 +340,18 @@ SPARK_WORKER_OPTS supports the following system properties: +# Resource Allocation and Configuration Overview + +Please make sure to have read the Custom Resource Scheduling and Configuration Overview section on the [configuration page](configuration.html). This section only talks about the Spark Standalone specific aspects of resource scheduling. + +Spark Standalone has 2 parts, the first is configuring the resources for the Worker, the second is the resource allocation for a specific application. + +The user must configure the Workers to have a set of resources available so that it can assign them out to Executors. The spark.worker.resource.{resourceName}.amount is used to control the amount of each resource the worker has allocated. The user must also specify either spark.worker.resourcesFile or spark.worker.resource.{resourceName}.discoveryScript to specify how the Worker discovers the resources its assigned. See the descriptions above for each of those to see which method works best for your setup. Please take note of spark.resources.coordinate.enable as it indicates whether Spark should handle coordinating resources or if the user has made sure each Worker has separate resources. Also note that if using the resources coordination spark.resources.dir can be used to specify the directory used to do that coordination. + +The second part is running an application on Spark Standalone. The only special case from the standard Spark resource configs is when you are running the Driver in client mode. For a Driver in client mode, the user can specify the resources it uses via spark.driver.resourcesfile or spark.driver.resources.{resourceName}.discoveryScript. If the Driver is running on the same host as other Drivers or Workers there are 2 ways to make sure the they don't use the same resources. The user can either configure spark.resources.coordinate.enable on and give all the Driver/Workers the same set or resources and Spark will handle make sure each Driver/Worker has separate resources, or the user can make sure the resources file or discovery script only returns resources the do not conflict with other Drivers or Workers running on the same node. + +Note, the user does not need to specify a discovery script when submitting an application as the Worker will start each Executor with the resources it allocates to it. + # Connecting an Application to the Cluster To run an application on the Spark cluster, simply pass the `spark://IP:PORT` URL of the master as to the [`SparkContext` From f263909ee1347a4d0f3854f2bf2adc497bc1bcd4 Mon Sep 17 00:00:00 2001 From: "Liu,Linhong" Date: Wed, 11 Sep 2019 21:51:48 +0800 Subject: [PATCH 0040/1387] [SPARK-23243][CORE][FOLLOWUP] Remove todo added by SPARK-23207 ### What changes were proposed in this pull request? PR #22112 fixed the todo added by PR #20393(SPARK-23207). We can remove it now. ### Why are the changes needed? In order not to confuse developers. ### Does this PR introduce any user-facing change? no ### How was this patch tested? no need to test Closes #25755 from LinhongLiu/remove-todo. Authored-by: Liu,Linhong Signed-off-by: Wenchen Fan --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index eafe3b17c2136..53268b169de42 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -430,8 +430,6 @@ abstract class RDD[T: ClassTag]( * * If you are decreasing the number of partitions in this RDD, consider using `coalesce`, * which can avoid performing a shuffle. - * - * TODO Fix the Shuffle+Repartition data loss issue described in SPARK-23207. */ def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = withScope { coalesce(numPartitions, shuffle = true) From 2736efa32d4b11197f74a6542c83ca21a619ac4f Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 11 Sep 2019 10:24:57 -0700 Subject: [PATCH 0041/1387] [SPARK-26989][CORE][TEST] DAGSchedulerSuite: ensure listeners are fully processed before checking recorded values ### What changes were proposed in this pull request? This patch ensures accessing recorded values in listener is always after letting listeners fully process all events. To ensure this, this patch adds new class to hide these values and access with methods which will ensure above condition. Without this guard, two threads are running concurrently - 1) listeners process thread 2) test main thread - and race condition would occur. That's why we also see very odd thing, error message saying condition is met but test failed: ``` - Barrier task failures from the same stage attempt don't trigger multiple stage retries *** FAILED *** ArrayBuffer(0) did not equal List(0) (DAGSchedulerSuite.scala:2656) ``` which means verification failed, and condition is met just before constructing error message. The guard is properly placed in many spots, but missed in some places. This patch enforces that it can't be missed. ### Why are the changes needed? UT fails intermittently and this patch will address the flakyness. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Modified UT. Also made the flaky tests artificially failing via applying 50ms of sleep on each onXXX method. ![Screen Shot 2019-09-07 at 7 44 15 AM](https://user-images.githubusercontent.com/1317309/64465178-1747ad00-d146-11e9-92f6-f4ed4a1f4b08.png) I found 3 methods being failed. (They've marked as X. Just ignore ! as they failed on waiting listener in given timeout and these tests don't deal with these recorded values - it uses other timeout value 1000ms than 10000ms for this listener so affected via side-effect.) When I applied same in this patch all tests marked as X passed. Closes #25706 from HeartSaVioR/SPARK-26989. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../spark/scheduler/DAGSchedulerSuite.scala | 138 +++++++++--------- 1 file changed, 71 insertions(+), 67 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index cd854c379b08a..d6cd57fa6e7b6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -175,31 +175,66 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 - val submittedStageInfos = new HashSet[StageInfo] - val successfulStages = new HashSet[Int] - val failedStages = new ArrayBuffer[Int] - val stageByOrderOfExecution = new ArrayBuffer[Int] - val endedTasks = new HashSet[Long] - val sparkListener = new SparkListener() { + /** + * Listeners which records some information to verify in UTs. Getter-kind methods in this class + * ensures the value is returned after ensuring there's no event to process, as well as the + * value is immutable: prevent showing odd result by race condition. + */ + class EventInfoRecordingListener extends SparkListener { + private val _submittedStageInfos = new HashSet[StageInfo] + private val _successfulStages = new HashSet[Int] + private val _failedStages = new ArrayBuffer[Int] + private val _stageByOrderOfExecution = new ArrayBuffer[Int] + private val _endedTasks = new HashSet[Long] + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - submittedStageInfos += stageSubmitted.stageInfo + _submittedStageInfos += stageSubmitted.stageInfo } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageInfo = stageCompleted.stageInfo - stageByOrderOfExecution += stageInfo.stageId + _stageByOrderOfExecution += stageInfo.stageId if (stageInfo.failureReason.isEmpty) { - successfulStages += stageInfo.stageId + _successfulStages += stageInfo.stageId } else { - failedStages += stageInfo.stageId + _failedStages += stageInfo.stageId } } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - endedTasks += taskEnd.taskInfo.taskId + _endedTasks += taskEnd.taskInfo.taskId + } + + def submittedStageInfos: Set[StageInfo] = { + waitForListeners() + _submittedStageInfos.toSet + } + + def successfulStages: Set[Int] = { + waitForListeners() + _successfulStages.toSet + } + + def failedStages: List[Int] = { + waitForListeners() + _failedStages.toList + } + + def stageByOrderOfExecution: List[Int] = { + waitForListeners() + _stageByOrderOfExecution.toList + } + + def endedTasks: Set[Long] = { + waitForListeners() + _endedTasks.toSet } + + private def waitForListeners(): Unit = sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) } + var sparkListener: EventInfoRecordingListener = null + var mapOutputTracker: MapOutputTrackerMaster = null var broadcastManager: BroadcastManager = null var securityMgr: SecurityManager = null @@ -248,10 +283,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi private def init(testConf: SparkConf): Unit = { sc = new SparkContext("local[2]", "DAGSchedulerSuite", testConf) - submittedStageInfos.clear() - successfulStages.clear() - failedStages.clear() - endedTasks.clear() + sparkListener = new EventInfoRecordingListener failure = null sc.addSparkListener(sparkListener) taskSets.clear() @@ -374,9 +406,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("[SPARK-3353] parent stage should have lower stage id") { - stageByOrderOfExecution.clear() sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + val stageByOrderOfExecution = sparkListener.stageByOrderOfExecution assert(stageByOrderOfExecution.length === 2) assert(stageByOrderOfExecution(0) < stageByOrderOfExecution(1)) } @@ -619,9 +650,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(unserializableRdd, Array(0)) assert(failure.getMessage.startsWith( "Job aborted due to stage failure: Task not serializable:")) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(failedStages.contains(0)) - assert(failedStages.size === 1) + assert(sparkListener.failedStages === Seq(0)) assertDataStructuresEmpty() } @@ -629,9 +658,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(new MyRDD(sc, 1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted due to stage failure: some failure") - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(failedStages.contains(0)) - assert(failedStages.size === 1) + assert(sparkListener.failedStages === Seq(0)) assertDataStructuresEmpty() } @@ -640,9 +667,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val jobId = submit(rdd, Array(0)) cancel(jobId) assert(failure.getMessage === s"Job $jobId cancelled ") - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(failedStages.contains(0)) - assert(failedStages.size === 1) + assert(sparkListener.failedStages === Seq(0)) assertDataStructuresEmpty() } @@ -700,9 +725,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(results === Map(0 -> 42)) assertDataStructuresEmpty() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(failedStages.isEmpty) - assert(successfulStages.contains(0)) + assert(sparkListener.failedStages.isEmpty) + assert(sparkListener.successfulStages.contains(0)) } test("run trivial shuffle") { @@ -1115,8 +1139,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(failedStages.contains(1)) + assert(sparkListener.failedStages.contains(1)) // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( @@ -1124,8 +1147,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), null)) // The SparkListener should not receive redundant failure events. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(failedStages.size == 1) + assert(sparkListener.failedStages.size === 1) } test("Retry all the tasks on a resubmitted attempt of a barrier stage caused by FetchFailure") { @@ -1172,7 +1194,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(0).tasks(1), TaskKilled("test"), null)) - assert(failedStages === Seq(0)) + assert(sparkListener.failedStages === Seq(0)) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq(0, 1))) scheduler.resubmitFailedStages() @@ -1226,11 +1248,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val mapStageId = 0 def countSubmittedMapStageAttempts(): Int = { - submittedStageInfos.count(_.stageId == mapStageId) + sparkListener.submittedStageInfos.count(_.stageId == mapStageId) } // The map stage should have been submitted. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(countSubmittedMapStageAttempts() === 1) complete(taskSets(0), Seq( @@ -1247,12 +1268,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(failedStages.contains(1)) + assert(sparkListener.failedStages.contains(1)) // Trigger resubmission of the failed map stage. runEvent(ResubmitFailedStages) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) // Another attempt for the map stage should have been submitted, resulting in 2 total attempts. assert(countSubmittedMapStageAttempts() === 2) @@ -1269,7 +1288,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // shouldn't effect anything -- our calling it just makes *SURE* it gets called between the // desired event and our check. runEvent(ResubmitFailedStages) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(countSubmittedMapStageAttempts() === 2) } @@ -1287,14 +1305,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0, 1)) def countSubmittedReduceStageAttempts(): Int = { - submittedStageInfos.count(_.stageId == 1) + sparkListener.submittedStageInfos.count(_.stageId == 1) } def countSubmittedMapStageAttempts(): Int = { - submittedStageInfos.count(_.stageId == 0) + sparkListener.submittedStageInfos.count(_.stageId == 0) } // The map stage should have been submitted. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(countSubmittedMapStageAttempts() === 1) // Complete the map stage. @@ -1303,7 +1320,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", 2)))) // The reduce stage should have been submitted. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(countSubmittedReduceStageAttempts() === 1) // The first result task fails, with a fetch failure for the output from the first mapper. @@ -1318,7 +1334,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Because the map stage finished, another attempt for the reduce stage should have been // submitted, resulting in 2 total attempts for each the map and the reduce stage. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(countSubmittedMapStageAttempts() === 2) assert(countSubmittedReduceStageAttempts() === 2) @@ -1348,10 +1363,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(makeCompletionEvent( taskSets(0).tasks(1), Success, 42, Seq.empty, Array.empty, createFakeTaskInfoWithId(1))) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) // verify stage exists assert(scheduler.stageIdToStage.contains(0)) - assert(endedTasks.size == 2) + assert(sparkListener.endedTasks.size === 2) // finish other 2 tasks runEvent(makeCompletionEvent( @@ -1360,8 +1374,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(makeCompletionEvent( taskSets(0).tasks(3), Success, 42, Seq.empty, Array.empty, createFakeTaskInfoWithId(3))) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(endedTasks.size == 4) + assert(sparkListener.endedTasks.size === 4) // verify the stage is done assert(!scheduler.stageIdToStage.contains(0)) @@ -1371,15 +1384,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(makeCompletionEvent( taskSets(0).tasks(3), Success, 42, Seq.empty, Array.empty, createFakeTaskInfoWithId(5))) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(endedTasks.size == 5) + assert(sparkListener.endedTasks.size === 5) // make sure non successful tasks also send out event runEvent(makeCompletionEvent( taskSets(0).tasks(3), UnknownReason, 42, Seq.empty, Array.empty, createFakeTaskInfoWithId(6))) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(endedTasks.size == 6) + assert(sparkListener.endedTasks.size === 6) } test("ignore late map task completions") { @@ -1452,8 +1463,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Listener bus should get told about the map stage failing, but not the reduce stage // (since the reduce stage hasn't been started yet). - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(failedStages.toSet === Set(0)) + assert(sparkListener.failedStages.toSet === Set(0)) assertDataStructuresEmpty() } @@ -1696,9 +1706,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(cancelledStages.toSet === Set(0, 2)) // Make sure the listeners got told about both failed stages. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(successfulStages.isEmpty) - assert(failedStages.toSet === Set(0, 2)) + assert(sparkListener.successfulStages.isEmpty) + assert(sparkListener.failedStages.toSet === Set(0, 2)) assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") @@ -2672,11 +2681,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val mapStageId = 0 def countSubmittedMapStageAttempts(): Int = { - submittedStageInfos.count(_.stageId == mapStageId) + sparkListener.submittedStageInfos.count(_.stageId == mapStageId) } // The map stage should have been submitted. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(countSubmittedMapStageAttempts() === 1) // The first map task fails with TaskKilled. @@ -2684,7 +2692,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(0).tasks(0), TaskKilled("test"), null)) - assert(failedStages === Seq(0)) + assert(sparkListener.failedStages === Seq(0)) // The second map task fails with TaskKilled. runEvent(makeCompletionEvent( @@ -2694,7 +2702,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Trigger resubmission of the failed map stage. runEvent(ResubmitFailedStages) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) // Another attempt for the map stage should have been submitted, resulting in 2 total attempts. assert(countSubmittedMapStageAttempts() === 2) @@ -2708,11 +2715,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val mapStageId = 0 def countSubmittedMapStageAttempts(): Int = { - submittedStageInfos.count(_.stageId == mapStageId) + sparkListener.submittedStageInfos.count(_.stageId == mapStageId) } // The map stage should have been submitted. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(countSubmittedMapStageAttempts() === 1) // The first map task fails with TaskKilled. @@ -2720,11 +2726,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(0).tasks(0), TaskKilled("test"), null)) - assert(failedStages === Seq(0)) + assert(sparkListener.failedStages === Seq(0)) // Trigger resubmission of the failed map stage. runEvent(ResubmitFailedStages) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) // Another attempt for the map stage should have been submitted, resulting in 2 total attempts. assert(countSubmittedMapStageAttempts() === 2) @@ -2737,7 +2742,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second map task failure doesn't trigger stage retry. runEvent(ResubmitFailedStages) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(countSubmittedMapStageAttempts() === 2) } From b62ef8f7935ae5c9a4a5e7e8a17aa5d7375c85b1 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 11 Sep 2019 10:29:13 -0700 Subject: [PATCH 0042/1387] [SPARK-29007][STREAMING][MLLIB][TESTS] Enforce not leaking SparkContext in tests which creates new StreamingContext with new SparkContext ### What changes were proposed in this pull request? This patch enforces tests to prevent leaking newly created SparkContext while is created via initializing StreamingContext. Leaking SparkContext in test would make most of following tests being failed as well, so this patch applies defensive programming, trying its best to ensure SparkContext is cleaned up. ### Why are the changes needed? We got some case in CI build where SparkContext is being leaked and other tests are affected by leaked SparkContext. Ideally we should isolate the environment among tests if possible. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Modified UTs. Closes #25709 from HeartSaVioR/SPARK-29007. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- external/kafka-0-10/pom.xml | 7 ++ .../kafka010/DirectKafkaStreamSuite.scala | 24 +++--- .../kinesis/KinesisStreamSuite.scala | 37 ++++----- mllib/pom.xml | 7 ++ .../StreamingLogisticRegressionSuite.scala | 16 ++-- .../clustering/StreamingKMeansSuite.scala | 13 +-- .../StreamingLinearRegressionSuite.scala | 16 ++-- .../spark/streaming/CheckpointSuite.scala | 18 ++-- .../spark/streaming/DStreamClosureSuite.scala | 16 +--- .../spark/streaming/DStreamScopeSuite.scala | 23 ++--- .../streaming/LocalStreamingContext.scala | 83 +++++++++++++++++++ .../spark/streaming/MapWithStateSuite.scala | 30 +++---- .../streaming/ReceiverInputDStreamSuite.scala | 16 +--- .../streaming/StreamingContextSuite.scala | 52 +++++------- .../streaming/StreamingListenerSuite.scala | 11 +-- .../spark/streaming/TestSuiteBase.scala | 30 ++++--- .../streaming/WindowOperationsSuite.scala | 19 +++-- .../ExecutorAllocationManagerSuite.scala | 19 ++--- .../scheduler/InputInfoTrackerSuite.scala | 22 ++--- .../scheduler/RateControllerSuite.scala | 6 +- .../StreamingJobProgressListenerSuite.scala | 16 ++-- 21 files changed, 240 insertions(+), 241 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/LocalStreamingContext.scala diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 397de87d3cdff..d11569d709b23 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -45,6 +45,13 @@ ${project.version} provided + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + org.apache.spark spark-core_${scala.binary.version} diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index 4d3e476e7cc58..26b41e68efb49 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.streaming.kafka010 import java.io.File -import java.lang.{ Long => JLong } -import java.util.{ Arrays, HashMap => JHashMap, Map => JMap, UUID } +import java.lang.{Long => JLong} +import java.util.{Arrays, HashMap => JHashMap, Map => JMap, UUID} import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicLong @@ -31,13 +31,12 @@ import scala.util.Random import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.StringDeserializer -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.concurrent.Eventually import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} +import org.apache.spark.streaming.{LocalStreamingContext, Milliseconds, StreamingContext, Time} import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.scheduler.rate.RateEstimator @@ -45,8 +44,7 @@ import org.apache.spark.util.Utils class DirectKafkaStreamSuite extends SparkFunSuite - with BeforeAndAfter - with BeforeAndAfterAll + with LocalStreamingContext with Eventually with Logging { val sparkConf = new SparkConf() @@ -56,7 +54,6 @@ class DirectKafkaStreamSuite // Otherwise the poll timeout defaults to 2 minutes and causes test cases to run longer. .set("spark.streaming.kafka.consumer.poll.ms", "10000") - private var ssc: StreamingContext = _ private var testDir: File = _ private var kafkaTestUtils: KafkaTestUtils = _ @@ -78,12 +75,13 @@ class DirectKafkaStreamSuite } } - after { - if (ssc != null) { - ssc.stop(stopSparkContext = true) - } - if (testDir != null) { - Utils.deleteRecursively(testDir) + override def afterEach(): Unit = { + try { + if (testDir != null) { + Utils.deleteRecursively(testDir) + } + } finally { + super.afterEach() } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index ee53fbadf8887..eee62d25e62bb 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming._ +import org.apache.spark.streaming.{LocalStreamingContext, _} import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.kinesis.KinesisInitialPositions.Latest import org.apache.spark.streaming.kinesis.KinesisReadConfigurations._ @@ -40,7 +40,7 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo import org.apache.spark.util.Utils abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFunSuite - with Eventually with BeforeAndAfter with BeforeAndAfterAll { + with LocalStreamingContext with Eventually with BeforeAndAfter with BeforeAndAfterAll { // This is the name that KCL will use to save metadata to DynamoDB private val appName = s"KinesisStreamSuite-${math.abs(Random.nextLong())}" @@ -53,15 +53,9 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun private val dummyAWSSecretKey = "dummySecretKey" private var testUtils: KinesisTestUtils = null - private var ssc: StreamingContext = null private var sc: SparkContext = null override def beforeAll(): Unit = { - val conf = new SparkConf() - .setMaster("local[4]") - .setAppName("KinesisStreamSuite") // Setting Spark app name to Kinesis app name - sc = new SparkContext(conf) - runIfTestsEnabled("Prepare KinesisTestUtils") { testUtils = new KPLBasedKinesisTestUtils() testUtils.createStream() @@ -70,12 +64,6 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun override def afterAll(): Unit = { try { - if (ssc != null) { - ssc.stop() - } - if (sc != null) { - sc.stop() - } if (testUtils != null) { // Delete the Kinesis stream as well as the DynamoDB table generated by // Kinesis Client Library when consuming the stream @@ -87,17 +75,22 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun } } - before { + override def beforeEach(): Unit = { + super.beforeEach() + val conf = new SparkConf() + .setMaster("local[4]") + .setAppName("KinesisStreamSuite") // Setting Spark app name to Kinesis app name + sc = new SparkContext(conf) ssc = new StreamingContext(sc, batchDuration) } - after { - if (ssc != null) { - ssc.stop(stopSparkContext = false) - ssc = null - } - if (testUtils != null) { - testUtils.deleteDynamoDBTable(appName) + override def afterEach(): Unit = { + try { + if (testUtils != null) { + testUtils.deleteDynamoDBTable(appName) + } + } finally { + super.afterEach() } } diff --git a/mllib/pom.xml b/mllib/pom.xml index 11769ef548d7c..2d2119683e7ad 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -55,6 +55,13 @@ spark-streaming_${scala.binary.version} ${project.version} + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + org.apache.spark spark-sql_${scala.binary.version} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala index 5f797a60f09e6..7349e0319324a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala @@ -23,23 +23,17 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.streaming.{LocalStreamingContext, TestSuiteBase} import org.apache.spark.streaming.dstream.DStream -class StreamingLogisticRegressionSuite extends SparkFunSuite with TestSuiteBase { +class StreamingLogisticRegressionSuite + extends SparkFunSuite + with LocalStreamingContext + with TestSuiteBase { // use longer wait time to ensure job completion override def maxWaitTimeMillis: Int = 30000 - var ssc: StreamingContext = _ - - override def afterFunction() { - super.afterFunction() - if (ssc != null) { - ssc.stop() - } - } - // Test if we can accurately learn B for Y = logistic(BX) on streaming data test("parameter accuracy") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala index a1ac10c06c697..415ac87275390 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala @@ -20,23 +20,14 @@ package org.apache.spark.mllib.clustering import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.streaming.{LocalStreamingContext, TestSuiteBase} import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.random.XORShiftRandom -class StreamingKMeansSuite extends SparkFunSuite with TestSuiteBase { +class StreamingKMeansSuite extends SparkFunSuite with LocalStreamingContext with TestSuiteBase { override def maxWaitTimeMillis: Int = 30000 - var ssc: StreamingContext = _ - - override def afterFunction() { - super.afterFunction() - if (ssc != null) { - ssc.stop() - } - } - test("accuracy for single center and equivalence to grand average") { // set parameters val numBatches = 10 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index eaeaa3fc1e68d..5b94f7ea4a075 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -22,23 +22,17 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LinearDataGenerator -import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.streaming.{LocalStreamingContext, TestSuiteBase} import org.apache.spark.streaming.dstream.DStream -class StreamingLinearRegressionSuite extends SparkFunSuite with TestSuiteBase { +class StreamingLinearRegressionSuite + extends SparkFunSuite + with LocalStreamingContext + with TestSuiteBase { // use longer wait time to ensure job completion override def maxWaitTimeMillis: Int = 20000 - var ssc: StreamingContext = _ - - override def afterFunction() { - super.afterFunction() - if (ssc != null) { - ssc.stop() - } - } - // Assert that two values are equal within tolerance epsilon def assertEqual(v1: Double, v2: Double, epsilon: Double) { def errorMessage = v1.toString + " did not equal " + v2.toString diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 607295730bb9e..e6f4f04d465eb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -39,8 +39,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.rdd.RDD import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.scheduler._ -import org.apache.spark.util.{Clock, ManualClock, MutableURLClassLoader, ResetSystemProperties, - Utils} +import org.apache.spark.util.{Clock, ManualClock, MutableURLClassLoader, ResetSystemProperties, Utils} /** * A input stream that records the times of restore() invoked @@ -206,24 +205,21 @@ trait DStreamCheckpointTester { self: SparkFunSuite => * the checkpointing of a DStream's RDDs as well as the checkpointing of * the whole DStream graph. */ -class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester +class CheckpointSuite extends TestSuiteBase with LocalStreamingContext with DStreamCheckpointTester with ResetSystemProperties { - var ssc: StreamingContext = null - override def batchDuration: Duration = Milliseconds(500) - override def beforeFunction() { - super.beforeFunction() + override def beforeEach(): Unit = { + super.beforeEach() Utils.deleteRecursively(new File(checkpointDir)) } - override def afterFunction() { + override def afterEach(): Unit = { try { - if (ssc != null) { ssc.stop() } Utils.deleteRecursively(new File(checkpointDir)) } finally { - super.afterFunction() + super.afterEach() } } @@ -255,7 +251,7 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester .checkpoint(stateStreamCheckpointInterval) .map(t => (t._1, t._2)) } - var ssc = setupStreams(input, operation) + ssc = setupStreams(input, operation) var stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head def waitForCompletionOfBatch(numBatches: Long): Unit = { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala index 2ab600ab817e0..0576bf560f30e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala @@ -29,24 +29,14 @@ import org.apache.spark.util.ReturnStatementInClosureException /** * Test that closures passed to DStream operations are actually cleaned. */ -class DStreamClosureSuite extends SparkFunSuite with BeforeAndAfterAll { - private var ssc: StreamingContext = null +class DStreamClosureSuite extends SparkFunSuite with LocalStreamingContext with BeforeAndAfterAll { + override protected def beforeEach(): Unit = { + super.beforeEach() - override def beforeAll(): Unit = { - super.beforeAll() val sc = new SparkContext("local", "test") ssc = new StreamingContext(sc, Seconds(1)) } - override def afterAll(): Unit = { - try { - ssc.stop(stopSparkContext = true) - ssc = null - } finally { - super.afterAll() - } - } - test("user provided closures are actually cleaned") { val dstream = new DummyInputDStream(ssc) val pairDstream = dstream.map { i => (i, i) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala index 94f1bcebc3a39..1bb41164d362e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala @@ -30,28 +30,29 @@ import org.apache.spark.util.ManualClock /** * Tests whether scope information is passed from DStream operations to RDDs correctly. */ -class DStreamScopeSuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterAll { - private var ssc: StreamingContext = null - private val batchDuration: Duration = Seconds(1) +class DStreamScopeSuite + extends SparkFunSuite + with LocalStreamingContext { + + override def beforeEach(): Unit = { + super.beforeEach() - override def beforeAll(): Unit = { - super.beforeAll() val conf = new SparkConf().setMaster("local").setAppName("test") conf.set("spark.streaming.clock", classOf[ManualClock].getName()) + val batchDuration: Duration = Seconds(1) ssc = new StreamingContext(new SparkContext(conf), batchDuration) + + assertPropertiesNotSet() } - override def afterAll(): Unit = { + override def afterEach(): Unit = { try { - ssc.stop(stopSparkContext = true) + assertPropertiesNotSet() } finally { - super.afterAll() + super.afterEach() } } - before { assertPropertiesNotSet() } - after { assertPropertiesNotSet() } - test("dstream without scope") { val dummyStream = new DummyDStream(ssc) dummyStream.initialize(Time(0)) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/LocalStreamingContext.scala b/streaming/src/test/scala/org/apache/spark/streaming/LocalStreamingContext.scala new file mode 100644 index 0000000000000..2008c1cb9d61c --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/LocalStreamingContext.scala @@ -0,0 +1,83 @@ +/* + * 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.streaming + +import org.scalatest.{BeforeAndAfterEach, Suite} + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging + +/** + * Manages a local `ssc` `StreamingContext` variable, correctly stopping it after each test. + * Note that it also stops active SparkContext if `stopSparkContext` is set to true (default). + * In most cases you may want to leave it, to isolate environment for SparkContext in each test. + */ +trait LocalStreamingContext extends BeforeAndAfterEach { self: Suite => + + @transient var ssc: StreamingContext = _ + @transient var stopSparkContext: Boolean = true + + override def afterEach() { + try { + resetStreamingContext() + } finally { + super.afterEach() + } + } + + def resetStreamingContext(): Unit = { + LocalStreamingContext.stop(ssc, stopSparkContext) + ssc = null + } +} + +object LocalStreamingContext extends Logging { + def stop(ssc: StreamingContext, stopSparkContext: Boolean): Unit = { + try { + if (ssc != null) { + ssc.stop(stopSparkContext = stopSparkContext) + } + } finally { + if (stopSparkContext) { + ensureNoActiveSparkContext() + } + } + } + + /** + * Clean up active SparkContext: try to stop first if there's an active SparkContext. + * If it fails to stop, log warning message and clear active SparkContext to avoid + * interfere between tests. + */ + def ensureNoActiveSparkContext(): Unit = { + // if SparkContext is still active, try to clean up + SparkContext.getActive match { + case Some(sc) => + try { + sc.stop() + } catch { + case e: Throwable => + logError("Exception trying to stop SparkContext, clear active SparkContext...", e) + SparkContext.clearActiveContext() + throw e + } + case _ => + } + } + +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala index 06c0c2aa97ee1..14796c4741b3b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala @@ -23,46 +23,36 @@ import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.PrivateMethodTester._ import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.streaming.dstream.{DStream, InternalMapWithStateDStream, MapWithStateDStream, MapWithStateDStreamImpl} import org.apache.spark.util.{ManualClock, Utils} -class MapWithStateSuite extends SparkFunSuite - with DStreamCheckpointTester with BeforeAndAfterAll with BeforeAndAfter { +class MapWithStateSuite extends SparkFunSuite with LocalStreamingContext + with DStreamCheckpointTester { private var sc: SparkContext = null protected var checkpointDir: File = null protected val batchDuration = Seconds(1) - before { - StreamingContext.getActive().foreach { _.stop(stopSparkContext = false) } - checkpointDir = Utils.createTempDir(namePrefix = "checkpoint") - } + override def beforeEach(): Unit = { + super.beforeEach() - after { - StreamingContext.getActive().foreach { _.stop(stopSparkContext = false) } - if (checkpointDir != null) { - Utils.deleteRecursively(checkpointDir) - } - } - - override def beforeAll(): Unit = { - super.beforeAll() val conf = new SparkConf().setMaster("local").setAppName("MapWithStateSuite") conf.set("spark.streaming.clock", classOf[ManualClock].getName()) sc = new SparkContext(conf) + + checkpointDir = Utils.createTempDir(namePrefix = "checkpoint") } - override def afterAll(): Unit = { + override def afterEach(): Unit = { try { - if (sc != null) { - sc.stop() + if (checkpointDir != null) { + Utils.deleteRecursively(checkpointDir) } } finally { - super.afterAll() + super.afterEach() } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala index 0349e11224cfc..5e2ce25c7c441 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.streaming import scala.util.Random -import org.scalatest.BeforeAndAfterAll - import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.rdd.BlockRDD import org.apache.spark.storage.{StorageLevel, StreamBlockId} @@ -30,15 +28,9 @@ import org.apache.spark.streaming.receiver.{BlockManagerBasedStoreResult, Receiv import org.apache.spark.streaming.scheduler.ReceivedBlockInfo import org.apache.spark.streaming.util.{WriteAheadLogRecordHandle, WriteAheadLogUtils} -class ReceiverInputDStreamSuite extends TestSuiteBase with BeforeAndAfterAll { - - override def afterAll(): Unit = { - try { - StreamingContext.getActive().foreach(_.stop()) - } finally { - super.afterAll() - } - } +class ReceiverInputDStreamSuite + extends TestSuiteBase + with LocalStreamingContext { testWithoutWAL("createBlockRDD creates empty BlockRDD when no block info") { receiverStream => val rdd = receiverStream.createBlockRDD(Time(0), Seq.empty) @@ -127,7 +119,7 @@ class ReceiverInputDStreamSuite extends TestSuiteBase with BeforeAndAfterAll { conf.setMaster("local[4]").setAppName("ReceiverInputDStreamSuite") conf.set(WriteAheadLogUtils.RECEIVER_WAL_ENABLE_CONF_KEY, enableWAL.toString) require(WriteAheadLogUtils.enableReceiverLog(conf) === enableWAL) - val ssc = new StreamingContext(conf, Seconds(1)) + ssc = new StreamingContext(conf, Seconds(1)) val receiverStream = new ReceiverInputDStream[Int](ssc) { override def getReceiver(): Receiver[Int] = null } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index c4424b3cff877..ea4c1d4e21d67 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Queue import org.apache.commons.io.FileUtils -import org.scalatest.{Assertions, BeforeAndAfter, PrivateMethodTester} +import org.scalatest.{Assertions, PrivateMethodTester} import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.Eventually._ import org.scalatest.exceptions.TestFailedDueToTimeoutException @@ -44,7 +44,11 @@ import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.util.{ManualClock, Utils} -class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeLimits with Logging { +class StreamingContextSuite + extends SparkFunSuite + with LocalStreamingContext + with TimeLimits + with Logging { // Necessary to make ScalaTest 3.x interrupt a thread on the JVM like ScalaTest 2.2.x implicit val signaler: Signaler = ThreadSignaler @@ -56,20 +60,6 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL val envPair = "key" -> "value" val conf = new SparkConf().setMaster(master).setAppName(appName) - var sc: SparkContext = null - var ssc: StreamingContext = null - - after { - if (ssc != null) { - ssc.stop() - ssc = null - } - if (sc != null) { - sc.stop() - sc = null - } - } - test("from no conf constructor") { ssc = new StreamingContext(master, appName, batchDuration) assert(ssc.sparkContext.conf.get("spark.master") === master) @@ -95,7 +85,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL } test("from existing SparkContext") { - sc = new SparkContext(master, appName) + val sc = new SparkContext(master, appName) ssc = new StreamingContext(sc, batchDuration) } @@ -272,7 +262,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // Explicitly do not stop SparkContext ssc = new StreamingContext(conf, batchDuration) - sc = ssc.sparkContext + var sc = ssc.sparkContext addInputStream(ssc).register() ssc.start() ssc.stop(stopSparkContext = false) @@ -306,7 +296,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) conf.set("spark.dummyTimeConfig", "3600s") - sc = new SparkContext(conf) + val sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) @@ -338,7 +328,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // This is not a deterministic unit. But if this unit test is flaky, then there is definitely // something wrong. See SPARK-5681 val conf = new SparkConf().setMaster(master).setAppName(appName) - sc = new SparkContext(conf) + val sc = new SparkContext(conf) ssc = new StreamingContext(sc, Milliseconds(100)) val input = ssc.receiverStream(new TestReceiver) input.foreachRDD(_ => {}) @@ -352,7 +342,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL test("stop slow receiver gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) conf.set("spark.streaming.gracefulStopTimeout", "20000s") - sc = new SparkContext(conf) + val sc = new SparkContext(conf) logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 @@ -591,7 +581,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // getOrCreate should recover StreamingContext with existing SparkContext testGetOrCreate { - sc = new SparkContext(conf) + val sc = new SparkContext(conf) ssc = StreamingContext.getOrCreate(checkpointPath, () => creatingFunction()) assert(ssc != null, "no context created") assert(!newContextCreated, "old context not recovered") @@ -603,7 +593,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL require(StreamingContext.getActive().isEmpty, "context exists from before") var newContextCreated = false - def creatingFunc(): StreamingContext = { + def creatingFunc(sc: SparkContext)(): StreamingContext = { newContextCreated = true val newSsc = new StreamingContext(sc, batchDuration) val input = addInputStream(newSsc) @@ -627,8 +617,8 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // getActiveOrCreate should create new context and getActive should return it only // after starting the context testGetActiveOrCreate { - sc = new SparkContext(conf) - ssc = StreamingContext.getActiveOrCreate(creatingFunc _) + val sc = new SparkContext(conf) + ssc = StreamingContext.getActiveOrCreate(creatingFunc(sc)) assert(ssc != null, "no context created") assert(newContextCreated, "new context not created") assert(StreamingContext.getActive().isEmpty, @@ -636,25 +626,25 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL ssc.start() assert(StreamingContext.getActive() === Some(ssc), "active context not returned") - assert(StreamingContext.getActiveOrCreate(creatingFunc _) === ssc, + assert(StreamingContext.getActiveOrCreate(creatingFunc(sc)) === ssc, "active context not returned") ssc.stop() assert(StreamingContext.getActive().isEmpty, "inactive context returned") - assert(StreamingContext.getActiveOrCreate(creatingFunc _) !== ssc, + assert(StreamingContext.getActiveOrCreate(creatingFunc(sc)) !== ssc, "inactive context returned") } // getActiveOrCreate and getActive should return independently created context after activating testGetActiveOrCreate { - sc = new SparkContext(conf) - ssc = creatingFunc() // Create + val sc = new SparkContext(conf) + ssc = creatingFunc(sc) // Create assert(StreamingContext.getActive().isEmpty, "new initialized context returned before starting") ssc.start() assert(StreamingContext.getActive() === Some(ssc), "active context not returned") - assert(StreamingContext.getActiveOrCreate(creatingFunc _) === ssc, + assert(StreamingContext.getActiveOrCreate(creatingFunc(sc)) === ssc, "active context not returned") ssc.stop() assert(StreamingContext.getActive().isEmpty, @@ -736,7 +726,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL } test("multiple streaming contexts") { - sc = new SparkContext( + val sc = new SparkContext( conf.clone.set("spark.streaming.clock", "org.apache.spark.util.ManualClock")) ssc = new StreamingContext(sc, Seconds(1)) val input = addInputStream(ssc) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 62fd43302b9d7..9cd5d8c0e95e1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -36,20 +36,11 @@ import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.streaming.scheduler._ -class StreamingListenerSuite extends TestSuiteBase with Matchers { +class StreamingListenerSuite extends TestSuiteBase with LocalStreamingContext with Matchers { val input = (1 to 4).map(Seq(_)).toSeq val operation = (d: DStream[Int]) => d.map(x => x) - var ssc: StreamingContext = _ - - override def afterFunction() { - super.afterFunction() - if (ssc != null) { - ssc.stop() - } - } - // To make sure that the processing start and end times in collected // information are different for successive batches override def batchDuration: Duration = Milliseconds(100) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index f2ae77896a5d3..6f0475c11940a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag -import org.scalatest.BeforeAndAfter +import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually.timeout import org.scalatest.concurrent.PatienceConfiguration import org.scalatest.time.{Seconds => ScalaTestSeconds, Span} @@ -211,7 +211,7 @@ class BatchCounter(ssc: StreamingContext) { * This is the base trait for Spark Streaming testsuites. This provides basic functionality * to run user-defined set of input on user-defined stream operations, and verify the output. */ -trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { +trait TestSuiteBase extends SparkFunSuite with BeforeAndAfterEach with Logging { // Name of the framework for Spark context def framework: String = this.getClass.getSimpleName @@ -250,7 +250,7 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { val eventuallyTimeout: PatienceConfiguration.Timeout = timeout(Span(10, ScalaTestSeconds)) // Default before function for any streaming test suite. Override this - // if you want to add your stuff to "before" (i.e., don't call before { } ) + // if you want to add your stuff to "beforeEach" def beforeFunction() { if (useManualClock) { logInfo("Using manual clock") @@ -262,13 +262,24 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { } // Default after function for any streaming test suite. Override this - // if you want to add your stuff to "after" (i.e., don't call after { } ) + // if you want to add your stuff to "afterEach" def afterFunction() { System.clearProperty("spark.streaming.clock") } - before(beforeFunction) - after(afterFunction) + override def beforeEach(): Unit = { + super.beforeEach() + beforeFunction() + } + + override def afterEach(): Unit = { + try { + afterFunction() + } finally { + super.afterEach() + } + + } /** * Run a block of code with the given StreamingContext and automatically @@ -278,12 +289,7 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { try { block(ssc) } finally { - try { - ssc.stop(stopSparkContext = true) - } catch { - case e: Exception => - logError("Error stopping StreamingContext", e) - } + LocalStreamingContext.stop(ssc, stopSparkContext = true) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index c7d085ec0799b..f580b49221654 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -146,15 +146,16 @@ class WindowOperationsSuite extends TestSuiteBase { test("window - persistence level") { val input = Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)) - val ssc = new StreamingContext(conf, batchDuration) - val inputStream = new TestInputStream[Int](ssc, input, 1) - val windowStream1 = inputStream.window(batchDuration * 2) - assert(windowStream1.storageLevel === StorageLevel.NONE) - assert(inputStream.storageLevel === StorageLevel.MEMORY_ONLY_SER) - windowStream1.persist(StorageLevel.MEMORY_ONLY) - assert(windowStream1.storageLevel === StorageLevel.NONE) - assert(inputStream.storageLevel === StorageLevel.MEMORY_ONLY) - ssc.stop() + + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val inputStream = new TestInputStream[Int](ssc, input, 1) + val windowStream1 = inputStream.window(batchDuration * 2) + assert(windowStream1.storageLevel === StorageLevel.NONE) + assert(inputStream.storageLevel === StorageLevel.MEMORY_ONLY_SER) + windowStream1.persist(StorageLevel.MEMORY_ONLY) + assert(windowStream1.storageLevel === StorageLevel.NONE) + assert(inputStream.storageLevel === StorageLevel.MEMORY_ONLY) + } } // Testing naive reduceByKeyAndWindow (without invertible function) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala index a8b00558b40a7..a3026b23c8375 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala @@ -19,26 +19,25 @@ package org.apache.spark.streaming.scheduler import org.mockito.ArgumentMatchers.{eq => meq} import org.mockito.Mockito.{never, reset, times, verify, when} -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, PrivateMethodTester} +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.Eventually.{eventually, timeout} import org.scalatest.mockito.MockitoSugar import org.scalatest.time.SpanSugar._ -import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkFunSuite} +import org.apache.spark.{ExecutorAllocationClient, SparkConf} import org.apache.spark.internal.config.{DYN_ALLOCATION_ENABLED, DYN_ALLOCATION_TESTING} import org.apache.spark.internal.config.Streaming._ -import org.apache.spark.streaming.{DummyInputDStream, Seconds, StreamingContext} +import org.apache.spark.streaming.{DummyInputDStream, Seconds, StreamingContext, TestSuiteBase} import org.apache.spark.util.{ManualClock, Utils} - -class ExecutorAllocationManagerSuite extends SparkFunSuite - with BeforeAndAfter with BeforeAndAfterAll with MockitoSugar with PrivateMethodTester { +class ExecutorAllocationManagerSuite extends TestSuiteBase + with MockitoSugar with PrivateMethodTester { private val batchDurationMillis = 1000L private var allocationClient: ExecutorAllocationClient = null private var clock: StreamManualClock = null - before { + override def beforeEach(): Unit = { allocationClient = mock[ExecutorAllocationClient] clock = new StreamManualClock() } @@ -392,13 +391,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite .setAppName(this.getClass.getSimpleName) .set("spark.streaming.dynamicAllocation.testing", "true") // to test dynamic allocation - var ssc: StreamingContext = null - try { - ssc = new StreamingContext(conf, Seconds(1)) + withStreamingContext(new StreamingContext(conf, Seconds(1))) { ssc => new DummyInputDStream(ssc).foreachRDD(_ => { }) body(ssc) - } finally { - if (ssc != null) ssc.stop() } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala index a7e365649d3e8..cc393425ca6f0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala @@ -17,27 +17,15 @@ package org.apache.spark.streaming.scheduler -import org.scalatest.BeforeAndAfter - import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.streaming.{Duration, StreamingContext, Time} - -class InputInfoTrackerSuite extends SparkFunSuite with BeforeAndAfter { +import org.apache.spark.streaming.{Duration, LocalStreamingContext, StreamingContext, Time} - private var ssc: StreamingContext = _ +class InputInfoTrackerSuite extends SparkFunSuite with LocalStreamingContext { - before { + override def beforeEach(): Unit = { + super.beforeEach() val conf = new SparkConf().setMaster("local[2]").setAppName("DirectStreamTacker") - if (ssc == null) { - ssc = new StreamingContext(conf, Duration(1000)) - } - } - - after { - if (ssc != null) { - ssc.stop() - ssc = null - } + ssc = new StreamingContext(conf, Duration(1000)) } test("test report and get InputInfo from InputInfoTracker") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala index 37ca0ce2f6a30..b5a45fc317d0e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala @@ -30,8 +30,7 @@ class RateControllerSuite extends TestSuiteBase { override def batchDuration: Duration = Milliseconds(50) test("RateController - rate controller publishes updates after batches complete") { - val ssc = new StreamingContext(conf, batchDuration) - withStreamingContext(ssc) { ssc => + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => val dstream = new RateTestInputDStream(ssc) dstream.register() ssc.start() @@ -43,8 +42,7 @@ class RateControllerSuite extends TestSuiteBase { } test("ReceiverRateController - published rates reach receivers") { - val ssc = new StreamingContext(conf, batchDuration) - withStreamingContext(ssc) { ssc => + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => val estimator = new ConstantEstimator(100) val dstream = new RateTestInputDStream(ssc) { override val rateController = diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala index 56b400850fdd4..10f92f9386173 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -22,24 +22,18 @@ import java.util.Properties import org.scalatest.Matchers import org.apache.spark.scheduler.SparkListenerJobStart -import org.apache.spark.streaming._ +import org.apache.spark.streaming.{LocalStreamingContext, _} import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.scheduler._ -class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { +class StreamingJobProgressListenerSuite + extends TestSuiteBase + with LocalStreamingContext + with Matchers { val input = (1 to 4).map(Seq(_)).toSeq val operation = (d: DStream[Int]) => d.map(x => x) - var ssc: StreamingContext = _ - - override def afterFunction() { - super.afterFunction() - if (ssc != null) { - ssc.stop() - } - } - private def createJobStart( batchTime: Time, outputOpId: Int, jobId: Int): SparkListenerJobStart = { val properties = new Properties() From c18f849d7667012597fe747448845d80a367b1bb Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 11 Sep 2019 13:31:43 -0700 Subject: [PATCH 0043/1387] [SPARK-24663][STREAMING][TESTS] StreamingContextSuite: Wait until slow receiver has been initialized, but with hard timeout ### What changes were proposed in this pull request? This patch fixes the flaky test failure from StreamingContextSuite "stop slow receiver gracefully", via putting flag whether initializing slow receiver is completed, and wait for such flag to be true. As receiver should be submitted via job and initialized in executor, 500ms might not be enough for covering all cases. ### Why are the changes needed? We got some reports for test failure on this test. Please refer [SPARK-24663](https://issues.apache.org/jira/browse/SPARK-24663) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Modified UT. I've artificially made delay on handling job submission via adding below code in `DAGScheduler.submitJob`: ``` if (rdd != null && rdd.name != null && rdd.name.startsWith("Receiver")) { println(s"Receiver Job! rdd name: ${rdd.name}") Thread.sleep(1000) } ``` and the test "stop slow receiver gracefully" failed on current master and passed on the patch. Closes #25725 from HeartSaVioR/SPARK-24663. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../spark/streaming/StreamingContextSuite.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index ea4c1d4e21d67..6cce280250eec 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -346,7 +346,6 @@ class StreamingContextSuite logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 - SlowTestReceiver.receivedAllRecords = false // Create test receiver that sleeps in onStop() val totalNumRecords = 15 val recordsPerSecond = 1 @@ -358,6 +357,9 @@ class StreamingContextSuite } ssc.start() ssc.awaitTerminationOrTimeout(500) + eventually(timeout(10.seconds), interval(10.millis)) { + assert(SlowTestReceiver.initialized) + } ssc.stop(stopSparkContext = false, stopGracefully = true) logInfo("Running count = " + runningCount) assert(runningCount > 0) @@ -949,6 +951,7 @@ class SlowTestReceiver(totalRecords: Int, recordsPerSecond: Int) extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging { var receivingThreadOption: Option[Thread] = None + @volatile var receivedAllRecords = false def onStart() { val thread = new Thread() { @@ -958,17 +961,18 @@ class SlowTestReceiver(totalRecords: Int, recordsPerSecond: Int) Thread.sleep(1000 / recordsPerSecond) store(i) } - SlowTestReceiver.receivedAllRecords = true + receivedAllRecords = true logInfo(s"Received all $totalRecords records") } } receivingThreadOption = Some(thread) thread.start() + SlowTestReceiver.initialized = true } def onStop() { // Simulate slow receiver by waiting for all records to be produced - while (!SlowTestReceiver.receivedAllRecords) { + while (!receivedAllRecords) { Thread.sleep(100) } // no clean to be done, the receiving thread should stop on it own @@ -976,7 +980,7 @@ class SlowTestReceiver(totalRecords: Int, recordsPerSecond: Int) } object SlowTestReceiver { - var receivedAllRecords = false + var initialized = false } /** Streaming application for testing DStream and RDD creation sites */ From 7ce0f2b4993f5b2893ff4b13569d7e7aec7834be Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 12 Sep 2019 08:52:25 +0900 Subject: [PATCH 0044/1387] [SPARK-29041][PYTHON] Allows createDataFrame to accept bytes as binary type ### What changes were proposed in this pull request? This PR proposes to allow `bytes` as an acceptable type for binary type for `createDataFrame`. ### Why are the changes needed? `bytes` is a standard type for binary in Python. This should be respected in PySpark side. ### Does this PR introduce any user-facing change? Yes, _when specified type is binary_, we will allow `bytes` as a binary type. Previously this was not allowed in both Python 2 and Python 3 as below: ```python spark.createDataFrame([[b"abcd"]], "col binary") ``` in Python 3 ``` Traceback (most recent call last): File "", line 1, in File "/.../spark/python/pyspark/sql/session.py", line 787, in createDataFrame rdd, schema = self._createFromLocal(map(prepare, data), schema) File "/.../spark/python/pyspark/sql/session.py", line 442, in _createFromLocal data = list(data) File "/.../spark/python/pyspark/sql/session.py", line 769, in prepare verify_func(obj) File "/.../forked/spark/python/pyspark/sql/types.py", line 1403, in verify verify_value(obj) File "/.../spark/python/pyspark/sql/types.py", line 1384, in verify_struct verifier(v) File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify verify_value(obj) File "/.../spark/python/pyspark/sql/types.py", line 1397, in verify_default verify_acceptable_types(obj) File "/.../spark/python/pyspark/sql/types.py", line 1282, in verify_acceptable_types % (dataType, obj, type(obj)))) TypeError: field col: BinaryType can not accept object b'abcd' in type ``` in Python 2: ``` Traceback (most recent call last): File "", line 1, in File "/.../spark/python/pyspark/sql/session.py", line 787, in createDataFrame rdd, schema = self._createFromLocal(map(prepare, data), schema) File "/.../spark/python/pyspark/sql/session.py", line 442, in _createFromLocal data = list(data) File "/.../spark/python/pyspark/sql/session.py", line 769, in prepare verify_func(obj) File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify verify_value(obj) File "/.../spark/python/pyspark/sql/types.py", line 1384, in verify_struct verifier(v) File "/.../spark/python/pyspark/sql/types.py", line 1403, in verify verify_value(obj) File "/.../spark/python/pyspark/sql/types.py", line 1397, in verify_default verify_acceptable_types(obj) File "/.../spark/python/pyspark/sql/types.py", line 1282, in verify_acceptable_types % (dataType, obj, type(obj)))) TypeError: field col: BinaryType can not accept object 'abcd' in type ``` So, it won't break anything. ### How was this patch tested? Unittests were added and also manually tested as below. ```bash ./run-tests --python-executables=python2,python3 --testnames "pyspark.sql.tests.test_serde" ``` Closes #25749 from HyukjinKwon/SPARK-29041. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/tests/test_serde.py | 4 ++++ python/pyspark/sql/types.py | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_serde.py b/python/pyspark/sql/tests/test_serde.py index ea2a686cddaa2..052a5b2835245 100644 --- a/python/pyspark/sql/tests/test_serde.py +++ b/python/pyspark/sql/tests/test_serde.py @@ -132,6 +132,10 @@ def test_int_array_serialization(self): df = self.spark.createDataFrame(data, "array") self.assertEqual(len(list(filter(lambda r: None in r.value, df.collect()))), 0) + def test_bytes_as_binary_type(self): + df = self.spark.createDataFrame([[b"abcd"]], "col binary") + self.assertEqual(df.first().col, bytearray(b'abcd')) + if __name__ == "__main__": import unittest diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 0c7f4ce3ddc67..81fdd41435694 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1190,7 +1190,7 @@ def convert_struct(obj): DoubleType: (float,), DecimalType: (decimal.Decimal,), StringType: (str, unicode), - BinaryType: (bytearray,), + BinaryType: (bytearray, bytes), DateType: (datetime.date, datetime.datetime), TimestampType: (datetime.datetime,), ArrayType: (list, tuple, array), From eec728a0d4f75c1acdcaebec761129f3ed3422c2 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 12 Sep 2019 09:24:36 +0900 Subject: [PATCH 0045/1387] [SPARK-29057][SQL] remove InsertIntoTable ### What changes were proposed in this pull request? Remove `InsertIntoTable` and replace it's usage by `InsertIntoStatement` ### Why are the changes needed? `InsertIntoTable` and `InsertIntoStatement` are almost identical (except some namings). It doesn't make sense to keep 2 identical plans. After the removal of `InsertIntoTable`, the analysis process becomes: 1. parser creates `InsertIntoStatement` 2. v2 rule `ResolveInsertInto` converts `InsertIntoStatement` to v2 commands. 3. v1 rules like `DataSourceAnalysis` and `HiveAnalysis` convert `InsertIntoStatement` to v1 commands. ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing tests Closes #25763 from cloud-fan/remove. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon --- .../sql/catalyst/analysis/Analyzer.scala | 4 +-- .../sql/catalyst/analysis/CheckAnalysis.scala | 9 +++-- .../plans/logical/basicLogicalOperators.scala | 36 ------------------- .../apache/spark/sql/DataFrameWriter.scala | 9 ++--- .../datasources/DataSourceStrategy.scala | 11 +++--- .../datasources/FallBackFileSourceV2.scala | 7 ++-- .../sql/execution/datasources/rules.scala | 25 ++++++------- .../sql/util/DataFrameCallbackSuite.scala | 7 ++-- .../spark/sql/hive/HiveStrategies.scala | 15 ++++---- .../apache/spark/sql/hive/InsertSuite.scala | 1 - 10 files changed, 46 insertions(+), 78 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5d130c5af213f..73b9c368e313a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -743,7 +743,7 @@ class Analyzer( } def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case i @ InsertIntoTable(u @ UnresolvedRelation(AsTableIdentifier(ident)), _, child, _, _) + case i @ InsertIntoStatement(u @ UnresolvedRelation(AsTableIdentifier(ident)), _, child, _, _) if child.resolved => EliminateSubqueryAliases(lookupTableFromCatalog(ident, u)) match { case v: View => @@ -794,7 +794,7 @@ class Analyzer( case scala.Right(Some(v2Table: Table)) => resolveV2Insert(i, v2Table) case _ => - InsertIntoTable(i.table, i.partitionSpec, i.query, i.overwrite, i.ifPartitionNotExists) + i } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 30e0117423f75..8ad31342fc9b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.expressions._ @@ -26,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.AlterTableStatement +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableStatement, InsertIntoStatement} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -94,6 +93,9 @@ trait CheckAnalysis extends PredicateHelper { case u: UnresolvedRelation => u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") + case InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) => + failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") + case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. // If the arguments of the higher-order functions are resolved but the type check fails, @@ -491,9 +493,6 @@ trait CheckAnalysis extends PredicateHelper { throw new IllegalStateException( "Internal error: logical hint operator should have been removed during analysis") - case InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => - failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") - case f @ Filter(condition, _) if PlanHelper.specialExpressionsInUnsupportedOperator(f).nonEmpty => val invalidExprSqls = PlanHelper.specialExpressionsInUnsupportedOperator(f).map(_.sql) 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 aa613d2649fe3..9806273f379c5 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 @@ -639,42 +639,6 @@ case class ShowTables( AttributeReference("tableName", StringType, nullable = false)()) } -/** - * Insert some data into a table. Note that this plan is unresolved and has to be replaced by the - * concrete implementations during analysis. - * - * @param table the logical plan representing the table. In the future this should be a - * [[org.apache.spark.sql.catalyst.catalog.CatalogTable]] once we converge Hive tables - * and data source tables. - * @param partition a map from the partition key to the partition value (optional). If the partition - * value is optional, dynamic partition insert will be performed. - * As an example, `INSERT INTO tbl PARTITION (a=1, b=2) AS ...` would have - * Map('a' -> Some('1'), 'b' -> Some('2')), - * and `INSERT INTO tbl PARTITION (a=1, b) AS ...` - * would have Map('a' -> Some('1'), 'b' -> None). - * @param query the logical plan representing data to write to. - * @param overwrite overwrite existing table or partitions. - * @param ifPartitionNotExists If true, only write if the partition does not exist. - * Only valid for static partitions. - */ -case class InsertIntoTable( - table: LogicalPlan, - partition: Map[String, Option[String]], - query: LogicalPlan, - overwrite: Boolean, - ifPartitionNotExists: Boolean) - extends LogicalPlan { - // IF NOT EXISTS is only valid in INSERT OVERWRITE - assert(overwrite || !ifPartitionNotExists) - // IF NOT EXISTS is only valid in static partitions - assert(partition.values.forall(_.nonEmpty) || !ifPartitionNotExists) - - // We don't want `table` in children as sometimes we don't want to transform it. - override def children: Seq[LogicalPlan] = query :: Nil - override def output: Seq[Attribute] = Seq.empty - override lazy val resolved: Boolean = false -} - /** * Insert query result into a directory. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 04a611024eb24..4489aece505d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.util.{Locale, Properties, UUID} +import java.util.{Locale, Properties} import scala.collection.JavaConverters._ @@ -28,7 +28,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, InsertIntoTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} +import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils @@ -408,9 +409,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def insertInto(tableIdent: TableIdentifier): Unit = { runCommand(df.sparkSession, "insertInto") { - InsertIntoTable( + InsertIntoStatement( table = UnresolvedRelation(tableIdent), - partition = Map.empty[String, Option[String]], + partitionSpec = Map.empty[String, Option[String]], query = df.logicalPlan, overwrite = modeForDSV1 == SaveMode.Overwrite, ifPartitionNotExists = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 4dcf5c52ce83d..1a92a2bb7156c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -34,7 +34,8 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ @@ -140,7 +141,7 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast if query.resolved && DDLUtils.isDatasourceTable(tableDesc) => CreateDataSourceTableAsSelectCommand(tableDesc, mode, query, query.output.map(_.name)) - case InsertIntoTable(l @ LogicalRelation(_: InsertableRelation, _, _, _), + case InsertIntoStatement(l @ LogicalRelation(_: InsertableRelation, _, _, _), parts, query, overwrite, false) if parts.isEmpty => InsertIntoDataSourceCommand(l, query, overwrite) @@ -152,7 +153,7 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast InsertIntoDataSourceDirCommand(storage, provider.get, query, overwrite) - case i @ InsertIntoTable( + case i @ InsertIntoStatement( l @ LogicalRelation(t: HadoopFsRelation, _, table, _), parts, query, overwrite, _) => // If the InsertIntoTable command is for a partitioned HadoopFsRelation and // the user has specified static partitions, we add a Project operator on top of the query @@ -241,11 +242,11 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] } override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoTable(UnresolvedCatalogRelation(tableMeta), _, _, _, _) + case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta), _, _, _, _) if DDLUtils.isDatasourceTable(tableMeta) => i.copy(table = readDataSourceTable(tableMeta)) - case i @ InsertIntoTable(UnresolvedCatalogRelation(tableMeta), _, _, _, _) => + case i @ InsertIntoStatement(UnresolvedCatalogRelation(tableMeta), _, _, _, _) => i.copy(table = DDLUtils.readHiveTable(tableMeta)) case UnresolvedCatalogRelation(tableMeta) if DDLUtils.isDatasourceTable(tableMeta) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala index 813af8203c2c5..70e78a9ecc859 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution.datasources import scala.collection.JavaConverters._ import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2, FileTable} /** - * Replace the File source V2 table in [[InsertIntoTable]] to V1 [[FileFormat]]. + * Replace the File source V2 table in [[InsertIntoStatement]] to V1 [[FileFormat]]. * E.g, with temporary view `t` using [[FileDataSourceV2]], inserting into view `t` fails * since there is no corresponding physical plan. * This is a temporary hack for making current data source V2 work. It should be @@ -33,7 +34,7 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, File */ class FallBackFileSourceV2(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoTable(d @ DataSourceV2Relation(table: FileTable, _, _), _, _, _, _) => + case i @ InsertIntoStatement(d @ DataSourceV2Relation(table: FileTable, _, _), _, _, _, _) => val v1FileFormat = table.fallbackFileFormat.newInstance() val relation = HadoopFsRelation( table.fileIndex, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index b507ef1c509dd..615e8a5f8a69a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -23,15 +23,16 @@ import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} +import org.apache.spark.sql.catalyst.expressions.{Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy import org.apache.spark.sql.sources.InsertableRelation -import org.apache.spark.sql.types.{ArrayType, AtomicType, StructField, StructType} +import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.util.SchemaUtils /** @@ -377,19 +378,19 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi } /** - * Preprocess the [[InsertIntoTable]] plan. Throws exception if the number of columns mismatch, or - * specified partition columns are different from the existing partition columns in the target + * Preprocess the [[InsertIntoStatement]] plan. Throws exception if the number of columns mismatch, + * or specified partition columns are different from the existing partition columns in the target * table. It also does data type casting and field renaming, to make sure that the columns to be * inserted have the correct data type and fields have the correct names. */ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { private def preprocess( - insert: InsertIntoTable, + insert: InsertIntoStatement, tblName: String, - partColNames: Seq[String]): InsertIntoTable = { + partColNames: Seq[String]): InsertIntoStatement = { val normalizedPartSpec = PartitioningUtils.normalizePartitionSpec( - insert.partition, partColNames, tblName, conf.resolver) + insert.partitionSpec, partColNames, tblName, conf.resolver) val staticPartCols = normalizedPartSpec.filter(_._2.isDefined).keySet val expectedColumns = insert.table.output.filterNot(a => staticPartCols.contains(a.name)) @@ -417,16 +418,16 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { """.stripMargin) } - insert.copy(query = newQuery, partition = normalizedPartSpec) + insert.copy(query = newQuery, partitionSpec = normalizedPartSpec) } else { // All partition columns are dynamic because the InsertIntoTable command does // not explicitly specify partitioning columns. - insert.copy(query = newQuery, partition = partColNames.map(_ -> None).toMap) + insert.copy(query = newQuery, partitionSpec = partColNames.map(_ -> None).toMap) } } def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoTable(table, _, query, _, _) if table.resolved && query.resolved => + case i @ InsertIntoStatement(table, _, query, _, _) if table.resolved && query.resolved => table match { case relation: HiveTableRelation => val metadata = relation.tableMeta @@ -503,7 +504,7 @@ object PreWriteCheck extends (LogicalPlan => Unit) { def apply(plan: LogicalPlan): Unit = { plan.foreach { - case InsertIntoTable(l @ LogicalRelation(relation, _, _, _), partition, query, _, _) => + case InsertIntoStatement(l @ LogicalRelation(relation, _, _, _), partition, query, _, _) => // Get all input data source relations of the query. val srcRelations = query.collect { case LogicalRelation(src, _, _, _) => src @@ -525,7 +526,7 @@ object PreWriteCheck extends (LogicalPlan => Unit) { case _ => failAnalysis(s"$relation does not allow insertion.") } - case InsertIntoTable(t, _, _, _, _) + case InsertIntoStatement(t, _, _, _, _) if !t.isInstanceOf[LeafNode] || t.isInstanceOf[Range] || t.isInstanceOf[OneRowRelation] || diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index a8e1a44f3d5d2..18a82f0f84283 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -22,7 +22,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.sql.{functions, AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoTable, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.execution.datasources.json.JsonFileFormat @@ -200,8 +201,8 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { sparkContext.listenerBus.waitUntilEmpty(1000) assert(commands.length == 3) assert(commands(2)._1 == "insertInto") - assert(commands(2)._2.isInstanceOf[InsertIntoTable]) - assert(commands(2)._2.asInstanceOf[InsertIntoTable].table + assert(commands(2)._2.isInstanceOf[InsertIntoStatement]) + assert(commands(2)._2.asInstanceOf[InsertIntoStatement].table .asInstanceOf[UnresolvedRelation].multipartIdentifier == Seq("tab")) } // exiting withTable adds commands(3) via onSuccess (drops tab) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 18feb98519fbe..56df50827f00d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -26,8 +26,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, - ScriptTransformation, Statistics} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, LogicalPlan, ScriptTransformation, Statistics} +import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} @@ -143,9 +143,9 @@ class DetermineTableStats(session: SparkSession) extends Rule[LogicalPlan] { if DDLUtils.isHiveTable(relation.tableMeta) && relation.tableMeta.stats.isEmpty => hiveTableWithStats(relation) - // handles InsertIntoTable specially as the table in InsertIntoTable is not added in its + // handles InsertIntoStatement specially as the table in InsertIntoStatement is not added in its // children, hence not matched directly by previous HiveTableRelation case. - case i @ InsertIntoTable(relation: HiveTableRelation, _, _, _, _) + case i @ InsertIntoStatement(relation: HiveTableRelation, _, _, _, _) if DDLUtils.isHiveTable(relation.tableMeta) && relation.tableMeta.stats.isEmpty => i.copy(table = hiveTableWithStats(relation)) } @@ -159,7 +159,7 @@ class DetermineTableStats(session: SparkSession) extends Rule[LogicalPlan] { */ object HiveAnalysis extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case InsertIntoTable(r: HiveTableRelation, partSpec, query, overwrite, ifPartitionNotExists) + case InsertIntoStatement(r: HiveTableRelation, partSpec, query, overwrite, ifPartitionNotExists) if DDLUtils.isHiveTable(r.tableMeta) => InsertIntoHiveTable(r.tableMeta, partSpec, query, overwrite, ifPartitionNotExists, query.output.map(_.name)) @@ -207,11 +207,12 @@ case class RelationConversions( override def apply(plan: LogicalPlan): LogicalPlan = { plan resolveOperators { // Write path - case InsertIntoTable(r: HiveTableRelation, partition, query, overwrite, ifPartitionNotExists) + case InsertIntoStatement( + r: HiveTableRelation, partition, query, overwrite, ifPartitionNotExists) if query.resolved && DDLUtils.isHiveTable(r.tableMeta) && (!r.isPartitioned || SQLConf.get.getConf(HiveUtils.CONVERT_INSERTING_PARTITIONED_TABLE)) && isConvertible(r) => - InsertIntoTable(metastoreCatalog.convert(r), partition, + InsertIntoStatement(metastoreCatalog.convert(r), partition, query, overwrite, ifPartitionNotExists) // Read path diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 73f5bbd88624e..2add537eb7933 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -25,7 +25,6 @@ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf From 8f632d70455156010f0e87288541304ad2164a52 Mon Sep 17 00:00:00 2001 From: dengziming Date: Thu, 12 Sep 2019 09:30:03 +0900 Subject: [PATCH 0046/1387] [MINOR][DOCS] Fix few typos in the java docs JIRA :https://issues.apache.org/jira/browse/SPARK-29050 'a hdfs' change into 'an hdfs' 'an unique' change into 'a unique' 'an url' change into 'a url' 'a error' change into 'an error' Closes #25756 from dengziming/feature_fix_typos. Authored-by: dengziming Signed-off-by: HyukjinKwon --- R/pkg/R/context.R | 4 ++-- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 2 +- .../main/scala/org/apache/spark/metrics/MetricsSystem.scala | 2 +- .../main/scala/org/apache/spark/storage/BlockManagerId.scala | 2 +- .../test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- docs/spark-standalone.md | 2 +- .../apache/spark/streaming/kinesis/KinesisCheckpointer.scala | 2 +- python/pyspark/context.py | 2 +- .../streaming/state/HDFSBackedStateStoreProvider.scala | 4 ++-- .../apache/spark/sql/streaming/StreamingQueryListener.scala | 4 ++-- .../main/scala/org/apache/spark/sql/streaming/progress.scala | 2 +- .../resources/ql/src/test/queries/clientpositive/load_fs2.q | 2 +- .../org/apache/spark/streaming/dstream/InputDStream.scala | 4 ++-- 13 files changed, 17 insertions(+), 17 deletions(-) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 51ae2d2954a9a..93ba1307043a3 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -301,7 +301,7 @@ broadcastRDD <- function(sc, object) { #' Set the checkpoint directory #' #' Set the directory under which RDDs are going to be checkpointed. The -#' directory must be a HDFS path if running on a cluster. +#' directory must be an HDFS path if running on a cluster. #' #' @param sc Spark Context to use #' @param dirName Directory path @@ -446,7 +446,7 @@ setLogLevel <- function(level) { #' Set checkpoint directory #' #' Set the directory under which SparkDataFrame are going to be checkpointed. The directory must be -#' a HDFS path if running on a cluster. +#' an HDFS path if running on a cluster. #' #' @rdname setCheckpointDir #' @param directory Directory path to checkpoint to diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 330c2f6e6117e..34851282082a8 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -609,7 +609,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { /** * Set the directory under which RDDs are going to be checkpointed. The directory must - * be a HDFS path if running on a cluster. + * be an HDFS path if running on a cluster. */ def setCheckpointDir(dir: String) { sc.setCheckpointDir(dir) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index c96640a6fab3f..b5524442a8f99 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -124,7 +124,7 @@ private[spark] class MetricsSystem private ( * If either ID is not available, this defaults to just using . * * @param source Metric source to be named by this method. - * @return An unique metric name for each combination of + * @return A unique metric name for each combination of * application, executor/driver and metric source. */ private[spark] def buildRegistryName(source: Source): String = { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index d188bdd912e5e..49e32d04d450a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils /** * :: DeveloperApi :: - * This class represent an unique identifier for a BlockManager. + * This class represent a unique identifier for a BlockManager. * * The first 2 constructors of this class are made private to ensure that BlockManagerId objects * can be created only using the apply method in the companion object. This allows de-duplication 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 385f549aa1ad9..d44f48095a8a1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -600,7 +600,7 @@ class SparkSubmitSuite } // TODO(SPARK-9603): Building a package is flaky on Jenkins Maven builds. - // See https://gist.github.com/shivaram/3a2fecce60768a603dac for a error log + // See https://gist.github.com/shivaram/3a2fecce60768a603dac for an error log ignore("correctly builds R packages included in a jar with --packages") { assume(RUtils.isRInstalled, "R isn't installed on this machine.") assume(RUtils.isSparkRInstalled, "SparkR is not installed in this build.") diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 1af0befdc70d7..1264951a2f270 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -432,7 +432,7 @@ In addition, detailed log output for each job is also written to the work direct # Running Alongside Hadoop -You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the same machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on). +You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the same machines. To access Hadoop data from Spark, just use an hdfs:// URL (typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on). # Configuring Ports for Network Security diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala index 5fb83b26f8382..11e949536f2b6 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala @@ -68,7 +68,7 @@ private[kinesis] class KinesisCheckpointer( if (checkpointer != null) { try { // We must call `checkpoint()` with no parameter to finish reading shards. - // See an URL below for details: + // See a URL below for details: // https://forums.aws.amazon.com/thread.jspa?threadID=244218 KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(), 4, 100) } catch { diff --git a/python/pyspark/context.py b/python/pyspark/context.py index d689d1d034bc7..4d140f91f032e 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -950,7 +950,7 @@ def addPyFile(self, path): def setCheckpointDir(self, dirName): """ Set the directory under which RDDs are going to be checkpointed. The - directory must be a HDFS path if running on a cluster. + directory must be an HDFS path if running on a cluster. """ self._jsc.sc().setCheckpointDir(dirName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 6ee54b948a7d4..05c651f9951b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -43,7 +43,7 @@ import org.apache.spark.util.{SizeEstimator, Utils} /** * An implementation of [[StateStoreProvider]] and [[StateStore]] in which all the data is backed - * by files in a HDFS-compatible file system. All updates to the store has to be done in sets + * by files in an HDFS-compatible file system. All updates to the store has to be done in sets * transactionally, and each set of updates increments the store's version. These versions can * be used to re-execute the updates (by retries in RDD operations) on the correct version of * the store, and regenerate the store version. @@ -79,7 +79,7 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit // java.util.ConcurrentModificationException type MapType = java.util.concurrent.ConcurrentHashMap[UnsafeRow, UnsafeRow] - /** Implementation of [[StateStore]] API which is backed by a HDFS-compatible file system */ + /** Implementation of [[StateStore]] API which is backed by an HDFS-compatible file system */ class HDFSBackedStateStore(val version: Long, mapToUpdate: MapType) extends StateStore { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 916d6a0365965..cc81cf6cfafec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -79,7 +79,7 @@ object StreamingQueryListener { /** * Event representing the start of a query - * @param id An unique query id that persists across restarts. See `StreamingQuery.id()`. + * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`. * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`. * @param name User-specified name of the query, null if not specified. * @since 2.1.0 @@ -101,7 +101,7 @@ object StreamingQueryListener { /** * Event representing that termination of a query. * - * @param id An unique query id that persists across restarts. See `StreamingQuery.id()`. + * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`. * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`. * @param exception The exception message of the query if the query was terminated * with an exception. Otherwise, it will be `None`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index 0b3945cbd1323..e2fea8c9dd6ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -74,7 +74,7 @@ class StateOperatorProgress private[sql]( * a trigger. Each event relates to processing done for a single trigger of the streaming * query. Events are emitted even when no new data is available to be processed. * - * @param id An unique query id that persists across restarts. See `StreamingQuery.id()`. + * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`. * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`. * @param name User-specified name of the query, null if not specified. * @param timestamp Beginning time of the trigger in ISO8601 format, i.e. UTC timestamps. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q index a75758a0728d5..f92cf24dede8b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q @@ -1,4 +1,4 @@ --- HIVE-3300 [jira] LOAD DATA INPATH fails if a hdfs file with same name is added to table +-- HIVE-3300 [jira] LOAD DATA INPATH fails if an hdfs file with same name is added to table -- 'loader' table is used only for uploading kv1.txt to HDFS (!hdfs -put is not working on minMRDriver) create table result (key string, value string); diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 6495c91247047..5a75b77659960 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.Utils * Input streams that can generate RDDs from new data by running a service/thread only on * the driver node (that is, without running a receiver on worker nodes), can be * implemented by directly inheriting this InputDStream. For example, - * FileInputDStream, a subclass of InputDStream, monitors a HDFS directory from the driver for + * FileInputDStream, a subclass of InputDStream, monitors an HDFS directory from the driver for * new files and generates RDDs with the new files. For implementing input streams * that requires running a receiver on the worker nodes, use * [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] as the parent class. @@ -48,7 +48,7 @@ abstract class InputDStream[T: ClassTag](_ssc: StreamingContext) ssc.graph.addInputStream(this) - /** This is an unique identifier for the input stream. */ + /** This is a unique identifier for the input stream. */ val id = ssc.getNewInputStreamId() // Keep track of the freshest rate for this stream using the rateEstimator From 850833fa177ec1f265e143fc383e40ec2c8341a6 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 12 Sep 2019 11:16:33 +0900 Subject: [PATCH 0047/1387] [SPARK-29046][SQL] Fix NPE in SQLConf.get when active SparkContext is stopping # What changes were proposed in this pull request? This patch fixes the bug regarding NPE in SQLConf.get, which is only possible when SparkContext._dagScheduler is null due to stopping SparkContext. The logic doesn't seem to consider active SparkContext could be in progress of stopping. Note that it can't be encountered easily as `SparkContext.stop()` blocks the main thread, but there're many cases which SQLConf.get is accessed concurrently while SparkContext.stop() is executing - users run another threads, or listener is accessing SQLConf.get after dagScheduler is set to null (this is the case what I encountered.) ### Why are the changes needed? The bug brings NPE. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added new UT to verify NPE doesn't occur. Without patch, the test fails with throwing NPE. Closes #25753 from HeartSaVioR/SPARK-29046. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../apache/spark/sql/internal/SQLConf.scala | 3 ++- .../spark/sql/internal/SQLConfSuite.scala | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d9b0a72618c7e..4f3e39ad49afe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -139,7 +139,8 @@ object SQLConf { } } else { val isSchedulerEventLoopThread = SparkContext.getActive - .map(_.dagScheduler.eventProcessLoop.eventThread) + .flatMap { sc => Option(sc.dagScheduler) } + .map(_.eventProcessLoop.eventThread) .exists(_.getId == Thread.currentThread().getId) if (isSchedulerEventLoopThread) { // DAGScheduler event loop thread does not have an active SparkSession, the `confGetter` diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 1dfbca64f5778..e698ac032cd82 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.internal import org.apache.hadoop.fs.Path +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} import org.apache.spark.sql._ import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext} @@ -320,4 +321,22 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { assert(e2.getMessage.contains("spark.sql.shuffle.partitions")) } + test("SPARK-29046: SQLConf.get shouldn't throw NPE when active SparkContext is stopping") { + // Logically, there's only one case SQLConf.get throws NPE: there's active SparkContext, + // but SparkContext is stopping - especially it sets dagScheduler as null. + + val oldSparkContext = SparkContext.getActive + Utils.tryWithSafeFinally { + // this is necessary to set new SparkContext as active: it cleans up active SparkContext + oldSparkContext.foreach(_ => SparkContext.clearActiveContext()) + + val conf = new SparkConf().setAppName("test").setMaster("local") + LocalSparkContext.withSpark(new SparkContext(conf)) { sc => + sc.dagScheduler = null + SQLConf.get + } + } { + oldSparkContext.orElse(Some(null)).foreach(SparkContext.setActiveContext) + } + } } From 6768431c974b20eb6c0aed01bf6794ed40239e17 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Wed, 11 Sep 2019 23:05:03 -0700 Subject: [PATCH 0048/1387] [SPARK-29045][SQL][TESTS] Drop table to avoid test failure in SQLMetricsSuite ### What changes were proposed in this pull request? In method `SQLMetricsTestUtils.testMetricsDynamicPartition()`, there is a CREATE TABLE sentence without `withTable` block. It causes test failure if use same table name in other unit tests. ### Why are the changes needed? To avoid "table already exists" in tests. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exist UT Closes #25752 from LantaoJin/SPARK-29045. Authored-by: LantaoJin Signed-off-by: Yuming Wang --- .../metric/SQLMetricsTestUtils.scala | 46 ++++++++++--------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index 8f26c04307adc..5ab9b6f5fc2d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -115,29 +115,31 @@ trait SQLMetricsTestUtils extends SQLTestUtils { provider: String, dataFormat: String, tableName: String): Unit = { - withTempPath { dir => - spark.sql( - s""" - |CREATE TABLE $tableName(a int, b int) - |USING $provider - |PARTITIONED BY(a) - |LOCATION '${dir.toURI}' - """.stripMargin) - val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) - assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) - - val df = spark.range(start = 0, end = 40, step = 1, numPartitions = 1) - .selectExpr("id a", "id b") - - // 40 files, 80 rows, 40 dynamic partitions. - verifyWriteDataMetrics(Seq(40, 40, 80)) { - df.union(df).repartition(2, $"a") - .write - .format(dataFormat) - .mode("overwrite") - .insertInto(tableName) + withTable(tableName) { + withTempPath { dir => + spark.sql( + s""" + |CREATE TABLE $tableName(a int, b int) + |USING $provider + |PARTITIONED BY(a) + |LOCATION '${dir.toURI}' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + val df = spark.range(start = 0, end = 40, step = 1, numPartitions = 1) + .selectExpr("id a", "id b") + + // 40 files, 80 rows, 40 dynamic partitions. + verifyWriteDataMetrics(Seq(40, 40, 80)) { + df.union(df).repartition(2, $"a") + .write + .format(dataFormat) + .mode("overwrite") + .insertInto(tableName) + } + assert(TestUtils.recursiveList(dir).count(_.getName.startsWith("part-")) == 40) } - assert(TestUtils.recursiveList(dir).count(_.getName.startsWith("part-")) == 40) } } From 7e6142591f3bc865806b86c7a7b90be008a319df Mon Sep 17 00:00:00 2001 From: sandeep katta Date: Thu, 12 Sep 2019 03:47:30 -0700 Subject: [PATCH 0049/1387] [SPARK-28840][SQL] conf.getClassLoader in SparkSQLCLIDriver should be avoided as it returns the UDFClassLoader which is created by Hive ### What changes were proposed in this pull request? Spark loads the jars to custom class loader which is returned by `getSubmitClassLoader` . [Spark code](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L337) **In 1.2.1.spark2 version of Hive** `HiveConf.getClassLoader` returns same the class loader which is set by the spark **In Hive 2.3.5** `HiveConf.getClassLoader` returns the UDFClassLoader which is created by Hive. Because of this spark cannot find the jars as class loader got changed [Hive code](https://github.com/apache/hive/blob/rel/release-2.3.5/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java#L395) ### Why are the changes needed? Before creating `CliSessionState` object save the current class loader object in some reference. After SessionState.start() reset back class Loader to the one which saved earlier. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added Test case and also Manually tested **Before Fix** ![b4Fix](https://user-images.githubusercontent.com/35216143/63442838-6789f400-c451-11e9-9529-ccf4ea9621b9.png) **After Fix** ![afterFix](https://user-images.githubusercontent.com/35216143/63442860-707ac580-c451-11e9-8012-2b70934d55f3.png) Closes #25542 from sandeep-katta/jarIssue. Lead-authored-by: sandeep katta Co-authored-by: angerszhu Signed-off-by: Yuming Wang --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 29 ++++++++++--------- .../sql/hive/thriftserver/CliSuite.scala | 27 +++++++++++++++++ .../thriftserver/ThriftserverShimUtils.scala | 7 ----- .../thriftserver/ThriftserverShimUtils.scala | 12 -------- 4 files changed, 42 insertions(+), 33 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index b9614d49eadbd..9f554b200f775 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -111,6 +111,11 @@ private[hive] object SparkSQLCLIDriver extends Logging { // Set all properties specified via command line. val conf: HiveConf = sessionState.getConf + // Hive 2.0.0 onwards HiveConf.getClassLoader returns the UDFClassLoader (created by Hive). + // Because of this spark cannot find the jars as class loader got changed + // Hive changed the class loader because of HIVE-11878, so it is required to use old + // classLoader as sparks loaded all the jars in this classLoader + conf.setClassLoader(Thread.currentThread().getContextClassLoader) sessionState.cmdProperties.entrySet().asScala.foreach { item => val key = item.getKey.toString val value = item.getValue.toString @@ -133,20 +138,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { // Clean up after we exit ShutdownHookManager.addShutdownHook { () => SparkSQLEnv.stop() } - val remoteMode = isRemoteMode(sessionState) - // "-h" option has been passed, so connect to Hive thrift server. - if (!remoteMode) { - // Hadoop-20 and above - we need to augment classpath using hiveconf - // components. - // See also: code in ExecDriver.java - var loader = conf.getClassLoader - val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) - if (StringUtils.isNotBlank(auxJars)) { - loader = ThriftserverShimUtils.addToClassPath(loader, StringUtils.split(auxJars, ",")) - } - conf.setClassLoader(loader) - Thread.currentThread().setContextClassLoader(loader) - } else { + if (isRemoteMode(sessionState)) { // Hive 1.2 + not supported in CLI throw new RuntimeException("Remote operations not supported") } @@ -164,6 +156,15 @@ private[hive] object SparkSQLCLIDriver extends Logging { val cli = new SparkSQLCLIDriver cli.setHiveVariables(oproc.getHiveVariables) + // In SparkSQL CLI, we may want to use jars augmented by hiveconf + // hive.aux.jars.path, here we add jars augmented by hiveconf to + // Spark's SessionResourceLoader to obtain these jars. + val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) + if (StringUtils.isNotBlank(auxJars)) { + val resourceLoader = SparkSQLEnv.sqlContext.sessionState.resourceLoader + StringUtils.split(auxJars, ",").foreach(resourceLoader.addJar(_)) + } + // TODO work around for set the log output to console, because the HiveContext // will set the output into an invalid buffer. sessionState.in = System.in diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index a979fe6b89679..6d45041e12821 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -305,4 +305,31 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { "SELECT example_format('%o', 93);" -> "135" ) } + + test("SPARK-28840 test --jars command") { + val jarFile = new File("../../sql/hive/src/test/resources/SPARK-21101-1.0.jar").getCanonicalPath + runCliWithin( + 1.minute, + Seq("--jars", s"$jarFile"))( + "CREATE TEMPORARY FUNCTION testjar AS" + + " 'org.apache.spark.sql.hive.execution.UDTFStack';" -> "", + "SELECT testjar(1,'TEST-SPARK-TEST-jar', 28840);" -> "TEST-SPARK-TEST-jar\t28840" + ) + } + + test("SPARK-28840 test --jars and hive.aux.jars.path command") { + val jarFile = new File("../../sql/hive/src/test/resources/SPARK-21101-1.0.jar").getCanonicalPath + val hiveContribJar = HiveTestUtils.getHiveContribJar.getCanonicalPath + runCliWithin( + 1.minute, + Seq("--jars", s"$jarFile", "--conf", + s"spark.hadoop.${ConfVars.HIVEAUXJARS}=$hiveContribJar"))( + "CREATE TEMPORARY FUNCTION testjar AS" + + " 'org.apache.spark.sql.hive.execution.UDTFStack';" -> "", + "SELECT testjar(1,'TEST-SPARK-TEST-jar', 28840);" -> "TEST-SPARK-TEST-jar\t28840", + "CREATE TEMPORARY FUNCTION example_max AS " + + "'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax';" -> "", + "SELECT concat_ws(',', 'First', example_max(1234321), 'Third');" -> "First,1234321,Third" + ) + } } diff --git a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala index 837861a77bf54..fbfc698ecb4bf 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ b/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.commons.logging.LogFactory -import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema, Type} import org.apache.hive.service.cli.Type._ @@ -60,12 +59,6 @@ private[thriftserver] object ThriftserverShimUtils { ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE) } - private[thriftserver] def addToClassPath( - loader: ClassLoader, - auxJars: Array[String]): ClassLoader = { - Utilities.addToClassPath(loader, auxJars) - } - private[thriftserver] val testedProtocolVersions = Seq( HIVE_CLI_SERVICE_PROTOCOL_V1, HIVE_CLI_SERVICE_PROTOCOL_V2, diff --git a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala index cb32ebed0ac11..850382fe2bfd7 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala +++ b/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala @@ -17,11 +17,6 @@ package org.apache.spark.sql.hive.thriftserver -import java.security.AccessController - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.hive.ql.exec.AddToClassPathAction import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.thrift.Type import org.apache.hadoop.hive.serde2.thrift.Type._ @@ -65,13 +60,6 @@ private[thriftserver] object ThriftserverShimUtils { ARRAY_TYPE, MAP_TYPE, STRUCT_TYPE) } - private[thriftserver] def addToClassPath( - loader: ClassLoader, - auxJars: Array[String]): ClassLoader = { - val addAction = new AddToClassPathAction(loader, auxJars.toList.asJava) - AccessController.doPrivileged(addAction) - } - private[thriftserver] val testedProtocolVersions = Seq( HIVE_CLI_SERVICE_PROTOCOL_V1, HIVE_CLI_SERVICE_PROTOCOL_V2, From 053dd858d38e6107bc71e0aa3a4954291b74f8c8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 12 Sep 2019 19:59:34 +0800 Subject: [PATCH 0050/1387] [SPARK-28998][SQL] reorganize the packages of DS v2 interfaces/classes ### What changes were proposed in this pull request? reorganize the packages of DS v2 interfaces/classes: 1. `org.spark.sql.connector.catalog`: put `TableCatalog`, `Table` and other related interfaces/classes 2. `org.spark.sql.connector.expression`: put `Expression`, `Transform` and other related interfaces/classes 3. `org.spark.sql.connector.read`: put `ScanBuilder`, `Scan` and other related interfaces/classes 4. `org.spark.sql.connector.write`: put `WriteBuilder`, `BatchWrite` and other related interfaces/classes ### Why are the changes needed? Data Source V2 has evolved a lot. It's a bit weird that `Expression` is in `org.spark.sql.catalog.v2` and `Table` is in `org.spark.sql.sources.v2`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing tests Closes #25700 from cloud-fan/package. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../spark/sql/v2/avro/AvroDataSourceV2.scala | 2 +- .../v2/avro/AvroPartitionReaderFactory.scala | 2 +- .../apache/spark/sql/v2/avro/AvroScan.scala | 2 +- .../spark/sql/v2/avro/AvroScanBuilder.scala | 2 +- .../apache/spark/sql/v2/avro/AvroTable.scala | 2 +- .../apache/spark/sql/kafka010/KafkaBatch.scala | 3 +-- .../kafka010/KafkaBatchPartitionReader.scala | 3 +-- .../spark/sql/kafka010/KafkaBatchWrite.scala | 2 +- .../sql/kafka010/KafkaContinuousStream.scala | 4 ++-- .../spark/sql/kafka010/KafkaDataWriter.scala | 2 +- .../sql/kafka010/KafkaMicroBatchStream.scala | 4 ++-- .../spark/sql/kafka010/KafkaSourceOffset.scala | 2 +- .../sql/kafka010/KafkaSourceProvider.scala | 12 ++++++------ .../sql/kafka010/KafkaStreamingWrite.scala | 4 ++-- .../kafka010/KafkaMicroBatchSourceSuite.scala | 2 +- .../kafka010/KafkaSourceProviderSuite.scala | 2 +- .../sql/tests/test_pandas_udf_scalar.py | 2 +- python/pyspark/sql/tests/test_udf.py | 2 +- .../catalog}/CatalogExtension.java | 2 +- .../catalog}/CatalogPlugin.java | 2 +- .../v2 => connector/catalog}/Catalogs.java | 2 +- .../catalog}/DelegatingCatalogExtension.java | 5 ++--- .../v2 => connector/catalog}/Identifier.java | 2 +- .../catalog}/IdentifierImpl.java | 2 +- .../catalog}/NamespaceChange.java | 5 ++++- .../catalog}/SessionConfigSupport.java | 2 +- .../v2 => connector/catalog}/StagedTable.java | 9 +++++---- .../catalog}/StagingTableCatalog.java | 12 ++++++------ .../catalog}/SupportsDelete.java | 4 +++- .../catalog}/SupportsNamespaces.java | 4 +++- .../v2 => connector/catalog}/SupportsRead.java | 8 +++++--- .../catalog}/SupportsWrite.java | 8 +++++--- .../v2 => connector/catalog}/Table.java | 4 ++-- .../catalog}/TableCapability.java | 10 +++++----- .../v2 => connector/catalog}/TableCatalog.java | 7 ++++--- .../v2 => connector/catalog}/TableChange.java | 4 +++- .../catalog}/TableProvider.java | 2 +- .../expressions/Expression.java | 2 +- .../expressions/Expressions.java | 2 +- .../v2 => connector}/expressions/Literal.java | 2 +- .../expressions/NamedReference.java | 2 +- .../expressions/Transform.java | 2 +- .../v2/reader => connector/read}/Batch.java | 2 +- .../read}/InputPartition.java | 2 +- .../read}/PartitionReader.java | 2 +- .../read}/PartitionReaderFactory.java | 2 +- .../v2/reader => connector/read}/Scan.java | 10 +++++----- .../reader => connector/read}/ScanBuilder.java | 2 +- .../reader => connector/read}/Statistics.java | 2 +- .../read}/SupportsPushDownFilters.java | 2 +- .../read}/SupportsPushDownRequiredColumns.java | 2 +- .../read}/SupportsReportPartitioning.java | 4 ++-- .../read}/SupportsReportStatistics.java | 2 +- .../partitioning/ClusteredDistribution.java | 4 ++-- .../read}/partitioning/Distribution.java | 4 ++-- .../read}/partitioning/Partitioning.java | 6 +++--- .../streaming/ContinuousPartitionReader.java | 4 ++-- .../ContinuousPartitionReaderFactory.java | 10 +++++----- .../read}/streaming/ContinuousStream.java | 6 +++--- .../read}/streaming/MicroBatchStream.java | 10 +++++----- .../read}/streaming/Offset.java | 2 +- .../read}/streaming/PartitionOffset.java | 2 +- .../read}/streaming/SparkDataStream.java | 2 +- .../writer => connector/write}/BatchWrite.java | 2 +- .../writer => connector/write}/DataWriter.java | 2 +- .../write}/DataWriterFactory.java | 2 +- .../write}/SupportsDynamicOverwrite.java | 2 +- .../write}/SupportsOverwrite.java | 2 +- .../write}/SupportsTruncate.java | 2 +- .../write}/WriteBuilder.java | 8 ++++---- .../write}/WriterCommitMessage.java | 4 ++-- .../streaming/StreamingDataWriterFactory.java | 4 ++-- .../write}/streaming/StreamingWrite.java | 6 +++--- .../analysis/AlreadyExistException.scala | 4 ++-- .../spark/sql/catalyst/analysis/Analyzer.scala | 10 ++++------ .../CannotReplaceMissingTableException.scala | 2 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 4 ++-- .../analysis/NoSuchItemException.scala | 4 ++-- .../sql/catalyst/analysis/unresolved.scala | 2 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 9 ++++----- .../plans/logical/basicLogicalOperators.scala | 7 +++---- .../logical/sql/CreateTableStatement.scala | 2 +- .../logical/sql/ReplaceTableStatement.scala | 2 +- .../catalog}/CatalogManager.scala | 5 +++-- .../catalog}/CatalogNotFoundException.scala | 2 +- .../catalog}/CatalogV2Implicits.scala | 6 +++--- .../sql/connector/catalog}/CatalogV2Util.scala | 10 ++++------ .../catalog}/LookupCatalog.scala | 5 ++--- .../spark/sql/connector/catalog}/V1Table.scala | 7 +++---- .../expressions/expressions.scala | 4 ++-- .../datasources/v2/DataSourceV2Implicits.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 8 ++++---- .../apache/spark/sql/util/SchemaUtils.scala | 2 +- .../catalog}/CatalogLoadingSuite.java | 2 +- ...reateTablePartitioningValidationSuite.scala | 4 ++-- .../sql/catalyst/parser/DDLParserSuite.scala | 2 +- .../spark/sql/connector/InMemoryTable.scala | 8 ++++---- .../sql/connector/InMemoryTableCatalog.scala | 10 ++++------ .../StagingInMemoryTableCatalog.scala | 11 +++++------ .../catalog/CatalogManagerSuite.scala | 3 +-- .../catalog}/LookupCatalogSuite.scala | 9 ++++----- .../catalog}/TableCatalogSuite.scala | 2 +- .../expressions/TransformExtractorSuite.scala | 2 +- .../spark/sql/execution/streaming/Offset.java | 4 ++-- .../org/apache/spark/sql/DataFrameReader.scala | 4 ++-- .../org/apache/spark/sql/DataFrameWriter.scala | 14 ++++++-------- .../write}/V1WriteBuilder.scala | 4 ++-- .../sql/execution/datasources/DataSource.scala | 2 +- .../datasources/DataSourceResolution.scala | 12 ++++++------ .../datasources/FileFormatDataWriter.scala | 2 +- .../execution/datasources/FilePartition.scala | 2 +- .../execution/datasources/FileScanRDD.scala | 5 +---- .../datasources/noop/NoopDataSource.scala | 6 +++--- .../sql/execution/datasources/rules.scala | 2 +- .../datasources/v2/AlterTableExec.scala | 2 +- .../datasources/v2/BatchScanExec.scala | 2 +- .../datasources/v2/ContinuousScanExec.scala | 4 ++-- .../datasources/v2/CreateTableExec.scala | 6 +++--- .../v2/DataSourcePartitioning.scala | 2 +- .../datasources/v2/DataSourceRDD.scala | 2 +- .../v2/DataSourceV2ScanExecBase.scala | 2 +- .../datasources/v2/DataSourceV2Strategy.scala | 10 +++------- .../datasources/v2/DataSourceV2Utils.scala | 2 +- .../datasources/v2/DeleteFromTableExec.scala | 2 +- .../datasources/v2/DescribeTableExec.scala | 2 +- .../datasources/v2/DropTableExec.scala | 2 +- .../datasources/v2/EmptyPartitionReader.scala | 2 +- .../datasources/v2/FileBatchWrite.scala | 3 +-- .../datasources/v2/FileDataSourceV2.scala | 2 +- .../datasources/v2/FilePartitionReader.scala | 2 +- .../v2/FilePartitionReaderFactory.scala | 5 ++--- .../execution/datasources/v2/FileScan.scala | 2 +- .../datasources/v2/FileScanBuilder.scala | 2 +- .../execution/datasources/v2/FileTable.scala | 8 ++++---- .../datasources/v2/FileWriteBuilder.scala | 2 +- .../datasources/v2/FileWriterFactory.scala | 3 +-- .../datasources/v2/MicroBatchScanExec.scala | 4 ++-- .../v2/PartitionReaderFromIterator.scala | 2 +- .../PartitionReaderWithPartitionValues.scala | 2 +- .../datasources/v2/PartitionRecordReader.scala | 2 +- .../datasources/v2/ReplaceTableExec.scala | 5 ++--- .../datasources/v2/ShowNamespacesExec.scala | 5 +++-- .../datasources/v2/ShowTablesExec.scala | 4 ++-- .../datasources/v2/TableCapabilityCheck.scala | 2 +- .../datasources/v2/V1FallbackWriters.scala | 10 ++++------ .../datasources/v2/V2SessionCatalog.scala | 14 +++++--------- .../v2/WriteToDataSourceV2Exec.scala | 13 ++++++------- .../datasources/v2/csv/CSVDataSourceV2.scala | 2 +- .../v2/csv/CSVPartitionReaderFactory.scala | 2 +- .../execution/datasources/v2/csv/CSVScan.scala | 6 +++--- .../datasources/v2/csv/CSVScanBuilder.scala | 2 +- .../datasources/v2/csv/CSVTable.scala | 2 +- .../datasources/v2/json/JsonDataSourceV2.scala | 2 +- .../v2/json/JsonPartitionReaderFactory.scala | 2 +- .../datasources/v2/json/JsonScan.scala | 4 ++-- .../datasources/v2/json/JsonScanBuilder.scala | 2 +- .../datasources/v2/json/JsonTable.scala | 2 +- .../datasources/v2/orc/OrcDataSourceV2.scala | 2 +- .../v2/orc/OrcPartitionReaderFactory.scala | 6 ++---- .../execution/datasources/v2/orc/OrcScan.scala | 2 +- .../datasources/v2/orc/OrcScanBuilder.scala | 2 +- .../datasources/v2/orc/OrcTable.scala | 2 +- .../v2/parquet/ParquetDataSourceV2.scala | 2 +- .../ParquetPartitionReaderFactory.scala | 2 +- .../datasources/v2/parquet/ParquetScan.scala | 3 +-- .../v2/parquet/ParquetScanBuilder.scala | 2 +- .../datasources/v2/parquet/ParquetTable.scala | 2 +- .../datasources/v2/text/TextDataSourceV2.scala | 2 +- .../v2/text/TextPartitionReaderFactory.scala | 2 +- .../datasources/v2/text/TextScan.scala | 2 +- .../datasources/v2/text/TextScanBuilder.scala | 2 +- .../datasources/v2/text/TextTable.scala | 2 +- .../streaming/ContinuousRecordEndpoint.scala | 2 +- .../streaming/MicroBatchExecution.scala | 4 ++-- .../sql/execution/streaming/OffsetSeq.scala | 2 +- .../sql/execution/streaming/OffsetSeqLog.scala | 2 +- .../execution/streaming/ProgressReporter.scala | 4 ++-- .../execution/streaming/RateStreamOffset.scala | 4 ++-- .../spark/sql/execution/streaming/Sink.scala | 2 +- .../spark/sql/execution/streaming/Source.scala | 3 +-- .../execution/streaming/StreamExecution.scala | 10 +++++----- .../execution/streaming/StreamProgress.scala | 3 +-- .../streaming/StreamingRelation.scala | 4 ++-- .../sql/execution/streaming/console.scala | 6 +++--- .../continuous/ContinuousDataSourceRDD.scala | 4 ++-- .../continuous/ContinuousExecution.scala | 7 +++---- .../ContinuousQueuedDataReader.scala | 2 +- .../ContinuousRateStreamSource.scala | 4 ++-- .../ContinuousTextSocketSource.scala | 4 ++-- .../continuous/ContinuousWriteRDD.scala | 4 ++-- .../continuous/EpochCoordinator.scala | 6 +++--- .../WriteToContinuousDataSource.scala | 2 +- .../WriteToContinuousDataSourceExec.scala | 2 +- .../spark/sql/execution/streaming/memory.scala | 6 +++--- .../streaming/sources/ConsoleWrite.scala | 4 ++-- .../sources/ContinuousMemoryStream.scala | 4 ++-- .../streaming/sources/ForeachWriterTable.scala | 6 +++--- .../streaming/sources/MicroBatchWrite.scala | 4 ++-- .../sources/PackedRowWriterFactory.scala | 4 ++-- .../sources/RateControlMicroBatchStream.scala | 2 +- .../sources/RateStreamMicroBatchStream.scala | 4 ++-- .../streaming/sources/RateStreamProvider.scala | 6 +++--- .../sources/TextSocketMicroBatchStream.scala | 4 ++-- .../sources/TextSocketSourceProvider.scala | 6 +++--- .../sources/WriteToMicroBatchDataSource.scala | 2 +- .../execution/streaming/sources/memory.scala | 6 +++--- .../spark/sql/internal/SessionState.scala | 2 +- .../spark/sql/streaming/DataStreamReader.scala | 4 ++-- .../spark/sql/streaming/DataStreamWriter.scala | 4 ++-- .../sql/streaming/StreamingQueryManager.scala | 2 +- .../JavaAdvancedDataSourceV2.java | 8 ++++---- .../JavaColumnarDataSourceV2.java | 11 +++++++---- .../JavaPartitionAwareDataSource.java | 18 +++++++++--------- .../JavaRangeInputPartition.java | 4 ++-- .../JavaReportStatisticsDataSource.java | 14 +++++++------- .../JavaSchemaRequiredDataSource.java | 9 +++++---- .../v2 => connector}/JavaSimpleBatchTable.java | 8 ++++---- .../JavaSimpleDataSourceV2.java | 9 +++++---- .../JavaSimpleReaderFactory.java | 8 ++++---- .../JavaSimpleScanBuilder.java | 10 +++++----- .../v2 => connector}/AlterTableTests.scala | 3 ++- ...aSourceV2DataFrameSessionCatalogSuite.scala | 13 ++++--------- .../DataSourceV2DataFrameSuite.scala | 3 +-- .../DataSourceV2SQLSessionCatalogSuite.scala | 6 ++---- .../DataSourceV2SQLSuite.scala | 8 +++----- .../v2 => connector}/DataSourceV2Suite.scala | 11 ++++++----- .../DataSourceV2UtilsSuite.scala | 3 ++- .../FileDataSourceV2FallBackSuite.scala | 7 ++++--- .../v2 => connector}/InsertIntoTests.scala | 4 ++-- .../SimpleWritableDataSource.scala | 11 ++++++----- .../TableCapabilityCheckSuite.scala | 7 ++++--- .../TestV2SessionCatalogBase.scala | 9 ++++----- .../V1WriteFallbackSuite.scala | 12 +++++------- .../command/PlanResolutionSuite.scala | 5 ++--- .../datasources/v2/FileTableSuite.scala | 4 ++-- .../datasources/v2/V2SessionCatalogSuite.scala | 9 ++++----- .../sources/RateStreamProviderSuite.scala | 2 +- .../sources/TextSocketStreamSuite.scala | 2 +- .../spark/sql/streaming/StreamTest.scala | 2 +- .../StreamingQueryListenerSuite.scala | 2 +- .../sql/streaming/StreamingQuerySuite.scala | 4 ++-- .../ContinuousQueuedDataReaderSuite.scala | 4 ++-- .../continuous/EpochCoordinatorSuite.scala | 6 +++--- .../sources/StreamingDataSourceV2Suite.scala | 12 ++++++------ 244 files changed, 522 insertions(+), 551 deletions(-) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/CatalogExtension.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/CatalogPlugin.java (98%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/Catalogs.java (98%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/DelegatingCatalogExtension.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/Identifier.java (96%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/IdentifierImpl.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/NamespaceChange.java (96%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2 => connector/catalog}/SessionConfigSupport.java (96%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2 => connector/catalog}/StagedTable.java (90%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/StagingTableCatalog.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2 => connector/catalog}/SupportsDelete.java (94%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/SupportsNamespaces.java (98%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2 => connector/catalog}/SupportsRead.java (87%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2 => connector/catalog}/SupportsWrite.java (85%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2 => connector/catalog}/Table.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2 => connector/catalog}/TableCapability.java (89%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/TableCatalog.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/TableChange.java (99%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2 => connector/catalog}/TableProvider.java (98%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector}/expressions/Expression.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector}/expressions/Expressions.java (99%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector}/expressions/Literal.java (96%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector}/expressions/NamedReference.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{catalog/v2 => connector}/expressions/Transform.java (96%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/Batch.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/InputPartition.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/PartitionReader.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/PartitionReaderFactory.java (98%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/Scan.java (93%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/ScanBuilder.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/Statistics.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/SupportsPushDownFilters.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/SupportsPushDownRequiredColumns.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/SupportsReportPartitioning.java (91%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/SupportsReportStatistics.java (96%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/partitioning/ClusteredDistribution.java (91%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/partitioning/Distribution.java (93%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/partitioning/Partitioning.java (90%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/streaming/ContinuousPartitionReader.java (92%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/streaming/ContinuousPartitionReaderFactory.java (81%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/streaming/ContinuousStream.java (94%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/streaming/MicroBatchStream.java (86%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/streaming/Offset.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/streaming/PartitionOffset.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/reader => connector/read}/streaming/SparkDataStream.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/BatchWrite.java (98%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/DataWriter.java (99%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/DataWriterFactory.java (98%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/SupportsDynamicOverwrite.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/SupportsOverwrite.java (97%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/SupportsTruncate.java (96%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/WriteBuilder.java (93%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/WriterCommitMessage.java (92%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/streaming/StreamingDataWriterFactory.java (95%) rename sql/catalyst/src/main/java/org/apache/spark/sql/{sources/v2/writer => connector/write}/streaming/StreamingWrite.java (95%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalog/v2 => connector/catalog}/CatalogManager.scala (97%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalog/v2 => connector/catalog}/CatalogNotFoundException.scala (95%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalog/v2 => connector/catalog}/CatalogV2Implicits.scala (94%) rename sql/catalyst/src/main/{java/org/apache/spark/sql/catalog/v2/utils => scala/org/apache/spark/sql/connector/catalog}/CatalogV2Util.scala (94%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalog/v2 => connector/catalog}/LookupCatalog.scala (97%) rename sql/catalyst/src/main/{java/org/apache/spark/sql/sources/v2/internal => scala/org/apache/spark/sql/connector/catalog}/V1Table.scala (91%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/{catalog/v2 => connector}/expressions/expressions.scala (98%) rename sql/catalyst/src/test/java/org/apache/spark/sql/{catalog/v2 => connector/catalog}/CatalogLoadingSuite.java (99%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/{catalyst => connector}/catalog/CatalogManagerSuite.scala (96%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/{catalyst/catalog/v2 => connector/catalog}/LookupCatalogSuite.scala (94%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/{catalog/v2 => connector/catalog}/TableCatalogSuite.scala (99%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/{catalog/v2 => connector}/expressions/TransformExtractorSuite.scala (98%) rename sql/core/src/main/scala/org/apache/spark/sql/{sources/v2/writer => connector/write}/V1WriteBuilder.scala (94%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaAdvancedDataSourceV2.java (96%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaColumnarDataSourceV2.java (90%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaPartitionAwareDataSource.java (86%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaRangeInputPartition.java (90%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaReportStatisticsDataSource.java (82%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaSchemaRequiredDataSource.java (87%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaSimpleBatchTable.java (86%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaSimpleDataSourceV2.java (84%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaSimpleReaderFactory.java (86%) rename sql/core/src/test/java/test/org/apache/spark/sql/{sources/v2 => connector}/JavaSimpleScanBuilder.java (81%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/AlterTableTests.scala (99%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/DataSourceV2DataFrameSessionCatalogSuite.scala (94%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/DataSourceV2DataFrameSuite.scala (97%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/DataSourceV2SQLSessionCatalogSuite.scala (89%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/DataSourceV2SQLSuite.scala (99%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/DataSourceV2Suite.scala (98%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/DataSourceV2UtilsSuite.scala (95%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/FileDataSourceV2FallBackSuite.scala (96%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/InsertIntoTests.scala (99%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/SimpleWritableDataSource.scala (94%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/TableCapabilityCheckSuite.scala (96%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2/utils => connector}/TestV2SessionCatalogBase.scala (88%) rename sql/core/src/test/scala/org/apache/spark/sql/{sources/v2 => connector}/V1WriteFallbackSuite.scala (92%) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala index 3171f1e08b4fc..c6f52d676422c 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.v2.avro import org.apache.spark.sql.avro.AvroFileFormat +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala index 243af7da47003..0397d15aed924 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala @@ -31,10 +31,10 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory, PartitionReaderWithPartitionValues} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala index 6ec351080a118..e1268ac2ce581 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala @@ -21,9 +21,9 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScan -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala index 815da2bd92d44..e36c71ef4b1f7 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.v2.avro import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala index a781624aa61aa..765e5727d944a 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala @@ -22,9 +22,9 @@ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.AvroUtils +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala index 700414167f3ef..b958035b39222 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala @@ -23,8 +23,7 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory} - +import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory} private[kafka010] class KafkaBatch( strategy: ConsumerStrategy, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala index 53b0b3c46854e..d4aa6774be66c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala @@ -22,8 +22,7 @@ import java.{util => ju} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.sources.v2.reader._ - +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} /** A [[InputPartition]] for reading Kafka data in a batch based streaming query. */ private[kafka010] case class KafkaBatchInputPartition( diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala index 47ec07ae128d2..8e29e38b2a644 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery -import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.types.StructType /** diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala index a9c1181a01c51..1e8da4bc0f7cc 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala @@ -27,9 +27,9 @@ import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ import org.apache.spark.sql.util.CaseInsensitiveStringMap /** diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala index 884773452b2a5..3f8d3d2da5797 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala @@ -21,7 +21,7 @@ import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} /** * Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index 9cd16c8e16249..52d91abc86218 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -26,10 +26,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchStream import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.UninterruptibleThread diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index 90d70439c5329..b9674a30aee39 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition +import org.apache.spark.sql.connector.read.streaming.PartitionOffset import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} -import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index c3f0be4be96e2..53a6919e2d4bc 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -30,14 +30,13 @@ import org.apache.spark.internal.Logging import org.apache.spark.kafka010.KafkaConfigUpdater import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.read.{Batch, Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} +import org.apache.spark.sql.connector.write.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.sources.v2.reader.{Batch, Scan, ScanBuilder} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -367,6 +366,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister override def schema(): StructType = KafkaOffsetReader.kafkaSchema override def capabilities(): ju.Set[TableCapability] = { + import TableCapability._ // ACCEPT_ANY_SCHEMA is needed because of the following reasons: // * Kafka writer validates the schema instead of the SQL analyzer (the schema is fixed) // * Read schema differs from write schema (please see Kafka integration guide) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala index 6dd1d2984a96e..2b50b771e694e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType /** diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index ae8a6886b2b4d..9fe100431c01a 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -35,6 +35,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.read.streaming.SparkDataStream import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ @@ -42,7 +43,6 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.kafka010.KafkaSourceProvider._ -import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSparkSession diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala index 8e6de88865e06..f7b00b31ebba0 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.mockito.Mockito.{mock, when} import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} -import org.apache.spark.sql.sources.v2.reader.Scan +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.util.CaseInsensitiveStringMap class KafkaSourceProviderSuite extends SparkFunSuite { diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index d254508e5d35b..d0eeaa5c62182 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -1110,7 +1110,7 @@ def test_datasource_with_udf(self): .format("org.apache.spark.sql.sources.SimpleScanSource") \ .option('from', 0).option('to', 1).load().toDF('i') datasource_v2_df = self.spark.read \ - .format("org.apache.spark.sql.sources.v2.SimpleDataSourceV2") \ + .format("org.apache.spark.sql.connector.SimpleDataSourceV2") \ .load().toDF('i', 'j') c1 = pandas_udf(lambda x: x + 1, 'int')(lit(1)) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 4a0a3760a556c..c274dc73225b6 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -552,7 +552,7 @@ def test_datasource_with_udf(self): .format("org.apache.spark.sql.sources.SimpleScanSource") \ .option('from', 0).option('to', 1).load().toDF('i') datasource_v2_df = self.spark.read \ - .format("org.apache.spark.sql.sources.v2.SimpleDataSourceV2") \ + .format("org.apache.spark.sql.connector.SimpleDataSourceV2") \ .load().toDF('i', 'j') c1 = udf(lambda x: x + 1, 'int')(lit(1)) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java index 2bf72c1da73de..32ba24ff0256f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogExtension.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogExtension.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.util.CaseInsensitiveStringMap; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java index 5d4995a05d233..615aab3ef3259 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.internal.SQLConf; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Catalogs.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Catalogs.java index f471a4e71cf43..b6a896c52d535 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Catalogs.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.SparkException; import org.apache.spark.annotation.Private; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java index 2d3700a08806f..d7f0ee15f840e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/DelegatingCatalogExtension.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import java.util.Map; import org.apache.spark.annotation.Experimental; -import org.apache.spark.sql.catalog.v2.expressions.Transform; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Identifier.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Identifier.java index 3e697c1945bfc..c3ac9c20db3b0 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Identifier.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java index 34f3882c9c412..56d13ef742cea 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import com.google.common.base.Preconditions; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/NamespaceChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/NamespaceChange.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/NamespaceChange.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/NamespaceChange.java index 6f5895bcc3806..1db6877c8d0be 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/NamespaceChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/NamespaceChange.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; + +import org.apache.spark.annotation.Experimental; /** * NamespaceChange subclasses represent requested changes to a namespace. These are passed to @@ -29,6 +31,7 @@ * ) * */ +@Experimental public interface NamespaceChange { /** * Create a NamespaceChange for setting a namespace property. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SessionConfigSupport.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SessionConfigSupport.java index d27fbfdd14617..fc076ee00a623 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SessionConfigSupport.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java similarity index 90% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java index b2baa93b146a5..9fd70cc97700d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagedTable.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import java.util.Map; -import org.apache.spark.sql.catalog.v2.Identifier; -import org.apache.spark.sql.catalog.v2.StagingTableCatalog; -import org.apache.spark.sql.catalog.v2.expressions.Transform; + +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -37,6 +37,7 @@ * at which point implementations are expected to commit the table's metadata into the metastore * along with the data that was written by the writes from the write builder this table created. */ +@Experimental public interface StagedTable extends Table { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java index fc055e91a6acf..ca9160aa2f168 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/StagingTableCatalog.java @@ -15,18 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import java.util.Map; -import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.sources.v2.StagedTable; -import org.apache.spark.sql.sources.v2.SupportsWrite; -import org.apache.spark.sql.sources.v2.writer.BatchWrite; -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.WriterCommitMessage; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -50,6 +49,7 @@ * {@link StagedTable#commitStagedChanges()} is called, at which point the staged table can * complete both the data write and the metadata swap operation atomically. */ +@Experimental public interface StagingTableCatalog extends TableCatalog { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsDelete.java similarity index 94% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsDelete.java index 8650a0ef1d4ba..80aa57ca18775 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsDelete.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsDelete.java @@ -15,14 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; +import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.sources.Filter; /** * A mix-in interface for {@link Table} delete support. Data sources can implement this * interface to provide the ability to delete data from tables that matches filter expressions. */ +@Experimental public interface SupportsDelete { /** * Delete data from a data source table that matches filter expressions. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/SupportsNamespaces.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java index 12c2e511f33f8..6845f9294a6de 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/SupportsNamespaces.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; +import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; @@ -35,6 +36,7 @@ * drop a namespace. Implementations are allowed to discover the existence of objects or namespaces * without throwing {@link NoSuchNamespaceException} when no namespace is found. */ +@Experimental public interface SupportsNamespaces extends CatalogPlugin { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRead.java similarity index 87% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRead.java index 826fa2f8a0720..cdff1a38ff05f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRead.java @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; -import org.apache.spark.sql.sources.v2.reader.Scan; -import org.apache.spark.sql.sources.v2.reader.ScanBuilder; +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** @@ -26,6 +27,7 @@ * {@link #newScanBuilder(CaseInsensitiveStringMap)} that is used to create a scan for batch, * micro-batch, or continuous processing. */ +@Experimental public interface SupportsRead extends Table { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsWrite.java similarity index 85% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsWrite.java index c52e54569dc0c..5b648468c84ad 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsWrite.java @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; -import org.apache.spark.sql.sources.v2.writer.BatchWrite; -import org.apache.spark.sql.sources.v2.writer.WriteBuilder; +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.WriteBuilder; import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** @@ -26,6 +27,7 @@ * {@link #newWriteBuilder(CaseInsensitiveStringMap)} that is used to create a write * for batch or streaming. */ +@Experimental public interface SupportsWrite extends Table { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java index 21e1e8c533f3b..a493736c77277 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/Table.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import java.util.Collections; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java similarity index 89% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java index eccf2892b039b..9765118a8dbf7 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCapability.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Experimental; @@ -24,7 +24,7 @@ *

* Tables use {@link Table#capabilities()} to return a set of capabilities. Each capability signals * to Spark that the table supports a feature identified by the capability. For example, returning - * {@code BATCH_READ} allows Spark to read from the table using a batch scan. + * {@link #BATCH_READ} allows Spark to read from the table using a batch scan. */ @Experimental public enum TableCapability { @@ -66,7 +66,7 @@ public enum TableCapability { *

* Truncating a table removes all existing rows. *

- * See {@code org.apache.spark.sql.sources.v2.writer.SupportsTruncate}. + * See {@link org.apache.spark.sql.connector.write.SupportsTruncate}. */ TRUNCATE, @@ -74,7 +74,7 @@ public enum TableCapability { * Signals that the table can replace existing data that matches a filter with appended data in * a write operation. *

- * See {@code org.apache.spark.sql.sources.v2.writer.SupportsOverwrite}. + * See {@link org.apache.spark.sql.connector.write.SupportsOverwrite}. */ OVERWRITE_BY_FILTER, @@ -82,7 +82,7 @@ public enum TableCapability { * Signals that the table can dynamically replace existing data partitions with appended data in * a write operation. *

- * See {@code org.apache.spark.sql.sources.v2.writer.SupportsDynamicOverwrite}. + * See {@link org.apache.spark.sql.connector.write.SupportsDynamicOverwrite}. */ OVERWRITE_DYNAMIC, diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index 4775b58edf049..b72f7845a6ea5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; -import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; -import org.apache.spark.sql.sources.v2.Table; import org.apache.spark.sql.types.StructType; import java.util.Map; @@ -35,6 +35,7 @@ * table schema when updating, renaming, or dropping existing columns when catalyst analysis is case * insensitive. */ +@Experimental public interface TableCatalog extends CatalogPlugin { /** * List the tables in a namespace from the catalog. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java similarity index 99% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java index 7eef6aea88120..88ab51c1d70b3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; +import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.types.DataType; /** @@ -32,6 +33,7 @@ * ) * */ +@Experimental public interface TableChange { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java index 1d37ff042bd33..e9fd87d0e2d40 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.StructType; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expression.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expression.java index 1e2aca9556df4..807731ac22948 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expression.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expressions.java similarity index 99% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expressions.java index 7b264e7480e17..bf2a1bab7bf4a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Expressions.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.types.DataType; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Literal.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Literal.java index e41bcf9000c52..942744bf6a5d3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Literal.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; import org.apache.spark.sql.types.DataType; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NamedReference.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NamedReference.java index c71ffbe70651f..e6e5fa59c2499 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NamedReference.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Transform.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Transform.java index c85e0c412f1ab..edac6df8a4d3e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/Transform.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions; +package org.apache.spark.sql.connector.expressions; import org.apache.spark.annotation.Experimental; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Batch.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Batch.java index 28d80b7a5bc3f..09592598a82f9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Batch.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/InputPartition.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/InputPartition.java index 413349782efa2..bedf30803f73f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/InputPartition.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import java.io.Serializable; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java index 2945925959538..254ee56501b9a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import java.io.Closeable; import java.io.IOException; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReaderFactory.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReaderFactory.java index 97f4a473953fc..7fef69ce2a5b8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/PartitionReaderFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import java.io.Serializable; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java similarity index 93% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java index c3964e2176d4f..c1584a58c117f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Scan.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousStream; -import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchStream; +import org.apache.spark.sql.connector.read.streaming.ContinuousStream; +import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableCapability; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; /** * A logical representation of a data source scan. This interface is used to provide logical diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java index d4bc1ff977132..ca2cd59b2e442 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Statistics.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Statistics.java index a0b194a41f585..42778b4e28a1a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/Statistics.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import java.util.OptionalLong; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownFilters.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownFilters.java index f10fd884daabe..bee9e5508ca66 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownFilters.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.Filter; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownRequiredColumns.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownRequiredColumns.java index 862bd14bffede..97143686d3efc 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownRequiredColumns.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.StructType; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportPartitioning.java similarity index 91% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportPartitioning.java index 4ce97bc5e76b5..5bbc191730391 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportPartitioning.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; +import org.apache.spark.sql.connector.read.partitioning.Partitioning; /** * A mix in interface for {@link Scan}. Data sources can implement this interface to diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java index d7364af69e89c..3cd5ef60dc533 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsReportStatistics.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader; +package org.apache.spark.sql.connector.read; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/ClusteredDistribution.java similarity index 91% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/ClusteredDistribution.java index 1cdc02f5736b1..23d3c6b563d1c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/ClusteredDistribution.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.partitioning; +package org.apache.spark.sql.connector.read.partitioning; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReader; /** * A concrete implementation of {@link Distribution}. Represents a distribution where records that diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Distribution.java similarity index 93% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Distribution.java index 02b0e68974919..fd397c760f05a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Distribution.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.partitioning; +package org.apache.spark.sql.connector.read.partitioning; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReader; /** * An interface to represent data distribution requirement, which specifies how the records should diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java similarity index 90% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java index c7370eb3d38af..f0b9410ee6e61 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Partitioning.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.partitioning; +package org.apache.spark.sql.connector.read.partitioning; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.SupportsReportPartitioning; /** * An interface to represent the output data partitioning for a data source, which is returned by diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java similarity index 92% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java index c7f6fce6e81af..8bd5273bb7d8e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReader; /** * A variation on {@link PartitionReader} for use with continuous streaming processing. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java similarity index 81% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java index 41195befe5e57..962864da4aad8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; import org.apache.spark.sql.vectorized.ColumnarBatch; /** * A variation on {@link PartitionReaderFactory} that returns {@link ContinuousPartitionReader} - * instead of {@link org.apache.spark.sql.sources.v2.reader.PartitionReader}. It's used for - * continuous streaming processing. + * instead of {@link PartitionReader}. It's used for continuous streaming processing. */ @Evolving public interface ContinuousPartitionReaderFactory extends PartitionReaderFactory { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java similarity index 94% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java index fff5b95a4de14..ee01a2553ae7a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.Scan; /** * A {@link SparkDataStream} for streaming queries with continuous mode. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java similarity index 86% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java index 330f07ba4f2f8..ceab0f75734d3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; -import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.Scan; /** * A {@link SparkDataStream} for streaming queries with micro-batch mode. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java index 1d34fdd1c28ab..400de2a659746 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java index 2c97d924a0629..35ad3bbde5cbf 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import java.io.Serializable; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java index 2068a84fc6bb1..1ba0c25ef4466 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.reader.streaming; +package org.apache.spark.sql.connector.read.streaming; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java index 91297759971b5..37c5539d2518f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import org.apache.spark.annotation.Evolving; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java similarity index 99% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java index 11228ad1ea672..eefe784dede43 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import java.io.IOException; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java similarity index 98% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java index bf2db9059b088..bcf8d8a59e5e5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import java.io.Serializable; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsDynamicOverwrite.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsDynamicOverwrite.java index 8058964b662bd..90e668d1935b1 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsDynamicOverwrite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; /** * Write builder trait for tables that support dynamic partition overwrite. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsOverwrite.java similarity index 97% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsOverwrite.java index b443b3c3aeb4a..6063a155ee209 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsOverwrite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import org.apache.spark.sql.sources.AlwaysTrue$; import org.apache.spark.sql.sources.Filter; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsTruncate.java similarity index 96% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsTruncate.java index 69c2ba5e01a49..15a68375032d2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/SupportsTruncate.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; /** * Write builder trait for tables that support truncation. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java similarity index 93% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java index 158066d402922..f26304e8db7b6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriteBuilder.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableCapability; -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; +import org.apache.spark.sql.connector.write.streaming.StreamingWrite; import org.apache.spark.sql.types.StructType; /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriterCommitMessage.java similarity index 92% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriterCommitMessage.java index 23e8580c404d4..823f955bffb53 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/WriterCommitMessage.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer; +package org.apache.spark.sql.connector.write; import java.io.Serializable; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite; +import org.apache.spark.sql.connector.write.streaming.StreamingWrite; /** * A commit message returned by {@link DataWriter#commit()} and will be sent back to the driver side diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java index af2f03c9d4192..daaa18d5bc4e7 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer.streaming; +package org.apache.spark.sql.connector.write.streaming; import java.io.Serializable; import org.apache.spark.TaskContext; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.writer.DataWriter; +import org.apache.spark.sql.connector.write.DataWriter; /** * A factory of {@link DataWriter} returned by diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java index 5617f1cdc0efc..0821b34891654 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer.streaming; +package org.apache.spark.sql.connector.write.streaming; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.sources.v2.writer.DataWriter; -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.WriterCommitMessage; /** * An interface that defines how to write the data to data source in streaming queries. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala index f5e9a146bf359..7e5d56a7d1196 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ -import org.apache.spark.sql.catalog.v2.Identifier import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.connector.catalog.Identifier /** * Thrown by a catalog when an item already exists. The analyzer will rethrow the exception diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 73b9c368e313a..8e6be32bcf70e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -25,8 +25,6 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2._ -import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -40,11 +38,11 @@ import org.apache.spark.sql.catalyst.plans.logical.sql._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.{PartitionOverwriteMode, StoreAssignmentPolicy} -import org.apache.spark.sql.sources.v2.Table -import org.apache.spark.sql.sources.v2.internal.V1Table import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -925,7 +923,7 @@ class Analyzer( * the table identifier does not include a catalog. */ object ResolveAlterTable extends Rule[LogicalPlan] { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case alter @ AlterTableAddColumnsStatement(tableName, cols) => val changes = cols.map { col => @@ -2855,7 +2853,7 @@ class Analyzer( private def lookupV2Relation( identifier: Seq[String] ): Either[(CatalogPlugin, Identifier, Option[Table]), Option[Table]] = { - import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ identifier match { case AsTemporaryViewIdentifier(ti) if catalog.isTemporaryTable(ti) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala index 3036f7c21093f..123d8aff25fb9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.Identifier +import org.apache.spark.sql.connector.catalog.Identifier class CannotReplaceMissingTableException( tableIdentifier: Identifier, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 8ad31342fc9b5..4a19806bd80ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -26,6 +25,7 @@ import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableStatement, InsertIntoStatement} +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -34,7 +34,7 @@ import org.apache.spark.sql.types._ */ trait CheckAnalysis extends PredicateHelper { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ /** * Override to provide additional checks for correct analysis. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index 7ac8ae61ed537..9b5b059908c00 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ -import org.apache.spark.sql.catalog.v2.Identifier import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.connector.catalog.Identifier /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 9e0e0d528a968..40f25fb7f9728 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -42,7 +42,7 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str */ case class UnresolvedRelation( multipartIdentifier: Seq[String]) extends LeafNode with NamedRelation { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ /** Returns a `.` separated name for this relation. */ def tableName: String = multipartIdentifier.quoted diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 25bfaa89017cb..8556ff4775530 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -28,8 +28,6 @@ import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2 -import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat} @@ -40,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} +import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -2177,7 +2176,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitTransformList(ctx: TransformListContext): Seq[Transform] = withOrigin(ctx) { def getFieldReference( ctx: ApplyTransformContext, - arg: v2.expressions.Expression): FieldReference = { + arg: V2Expression): FieldReference = { lazy val name: String = ctx.identifier.getText arg match { case ref: FieldReference => @@ -2190,7 +2189,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging def getSingleFieldReference( ctx: ApplyTransformContext, - arguments: Seq[v2.expressions.Expression]): FieldReference = { + arguments: Seq[V2Expression]): FieldReference = { lazy val name: String = ctx.identifier.getText if (arguments.size > 1) { throw new ParseException(s"Too many arguments for transform $name", ctx) @@ -2247,7 +2246,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Parse an argument to a transform. An argument may be a field reference (qualified name) or * a value literal. */ - override def visitTransformArgument(ctx: TransformArgumentContext): v2.expressions.Expression = { + override def visitTransformArgument(ctx: TransformArgumentContext): V2Expression = { withOrigin(ctx) { val reference = Option(ctx.qualifiedName) .map(typedVisit[Seq[String]]) 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 9806273f379c5..017f31760b584 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 @@ -17,18 +17,17 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalog.v2.{Identifier, SupportsNamespaces, TableCatalog, TableChange} -import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, ColumnChange} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.AliasIdentifier import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} -import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala index 190711303e32d..551a0ce44b4d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.catalyst.plans.logical.sql -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala index 2808892b089b9..b9767ff9b889a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala @@ -17,10 +17,10 @@ package org.apache.spark.sql.catalyst.plans.logical.sql -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala similarity index 97% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index 5bba88dbe76d0..3cf8cec84d0c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import scala.collection.mutable import scala.util.control.NonFatal @@ -27,6 +27,7 @@ import org.apache.spark.sql.internal.SQLConf * A thread-safe manager for [[CatalogPlugin]]s. It tracks all the registered catalogs, and allow * the caller to look up a catalog by name. */ +private[sql] class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends Logging { private val catalogs = mutable.HashMap.empty[String, CatalogPlugin] @@ -114,6 +115,6 @@ class CatalogManager(conf: SQLConf, defaultSessionCatalog: TableCatalog) extends } } -object CatalogManager { +private[sql] object CatalogManager { val SESSION_CATALOG_NAME: String = "session" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogNotFoundException.scala similarity index 95% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogNotFoundException.scala index 86de1c9285b73..d376b98afa415 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogNotFoundException.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import org.apache.spark.SparkException import org.apache.spark.annotation.Experimental diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala similarity index 94% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 87070fb2ad377..7d0acceec37da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.connector.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} import org.apache.spark.sql.types.StructType /** * Conversion helpers for working with v2 [[CatalogPlugin]]. */ -object CatalogV2Implicits { +private[sql] object CatalogV2Implicits { implicit class PartitionTypeHelper(partitionType: StructType) { def asTransforms: Array[Transform] = partitionType.names.map(LogicalExpressions.identity) } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala similarity index 94% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index d5079202c8fee..a61a55b222858 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -15,21 +15,19 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.utils +package org.apache.spark.sql.connector.catalog import java.util import java.util.Collections import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, NamespaceChange, TableChange} -import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RemoveProperty, RenameColumn, SetProperty, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException} -import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} -object CatalogV2Util { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ +private[sql] object CatalogV2Util { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ /** * Apply properties changes to a map and return the result. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala similarity index 97% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 2281402740a36..baea8204d9be4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging @@ -24,8 +24,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier /** * A trait to encapsulate catalog lookup function and helpful extractors. */ -@Experimental -trait LookupCatalog extends Logging { +private[sql] trait LookupCatalog extends Logging { protected val catalogManager: CatalogManager diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/V1Table.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala similarity index 91% rename from sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/V1Table.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala index ddfc77d0cc0ac..616c3cf696396 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/internal/V1Table.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/V1Table.scala @@ -15,23 +15,22 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.internal +package org.apache.spark.sql.connector.catalog import java.util import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.catalog.v2.expressions.{LogicalExpressions, Transform} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.connector.expressions.{LogicalExpressions, Transform} import org.apache.spark.sql.types.StructType /** * An implementation of catalog v2 `Table` to expose v1 table metadata. */ -case class V1Table(v1Table: CatalogTable) extends Table { +private[sql] case class V1Table(v1Table: CatalogTable) extends Table { implicit class IdentifierHelper(identifier: TableIdentifier) { def quoted: String = { identifier.database match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/expressions/expressions.scala similarity index 98% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/connector/expressions/expressions.scala index bceea147dddd5..2dd0f34c6c051 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/expressions/expressions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions +package org.apache.spark.sql.connector.expressions import org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -253,7 +253,7 @@ private[sql] final case class LiteralValue[T](value: T, dataType: DataType) exte } private[sql] final case class FieldReference(parts: Seq[String]) extends NamedReference { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.MultipartIdentifierHelper + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper override def fieldNames: Array[String] = parts.toArray override def describe: String = parts.quoted override def toString: String = describe diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index 2d59c42ee8684..74842e762b401 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.sources.v2.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} object DataSourceV2Implicits { implicit class TableHelper(table: Table) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 9ae3dbbc45502..7da502fc298ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -21,10 +21,10 @@ import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelat import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{Statistics => V2Statistics, _} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream} -import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.connector.catalog.{Table, TableCapability} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, Statistics => V2Statistics, SupportsReportStatistics} +import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.util.CaseInsensitiveStringMap /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala index d15440632f324..27b5eec27281d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/SchemaUtils.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.util import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.expressions._ import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, NamedTransform, Transform} import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java similarity index 99% rename from sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java rename to sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java index 6bd9192816992..37f60511cd60a 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2; +package org.apache.spark.sql.connector.catalog; import org.apache.spark.SparkException; import org.apache.spark.sql.internal.SQLConf; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala index c4e7194f7d651..79e6245528dea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.LogicalExpressions import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LeafNode} import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.connector.expressions.LogicalExpressions import org.apache.spark.sql.types.{DoubleType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 2e5ff8c3ce6dd..2cb01d4e8aa5c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} +import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 0dea1e3a68dc8..04ae27df94535 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -22,12 +22,12 @@ import java.util import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.catalog.v2.expressions.{IdentityTransform, Transform} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.{IdentityTransform, Transform} +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.sources.{And, EqualTo, Filter, IsNotNull} -import org.apache.spark.sql.sources.v2.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala index 8b14ad0d372dd..8724a38d08d1f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala @@ -22,16 +22,14 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalog.v2.{Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange} -import org.apache.spark.sql.catalog.v2.expressions.Transform -import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class BasicInMemoryTableCatalog extends TableCatalog { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() @@ -112,7 +110,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { } class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamespaces { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ protected val namespaces: util.Map[List[String], Map[String, String]] = new ConcurrentHashMap[List[String], Map[String, String]]() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/StagingInMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/StagingInMemoryTableCatalog.scala index e1b0e6136de95..513ea67b1fe95 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/StagingInMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/StagingInMemoryTableCatalog.scala @@ -19,18 +19,17 @@ package org.apache.spark.sql.connector import java.util -import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, TableAlreadyExistsException} -import org.apache.spark.sql.sources.v2.{StagedTable, SupportsRead, SupportsWrite, TableCapability} -import org.apache.spark.sql.sources.v2.reader.ScanBuilder -import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class StagingInMemoryTableCatalog extends InMemoryTableCatalog with StagingTableCatalog { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ import InMemoryTableCatalog._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override def stageCreate( ident: Identifier, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala similarity index 96% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index d2a2ba50ead45..58e2f410bf205 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.catalog +package org.apache.spark.sql.connector.catalog import java.util import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalog.v2.{CatalogManager, NamespaceChange, SupportsNamespaces} import org.apache.spark.sql.catalyst.analysis.FakeV2SessionCatalog import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala similarity index 94% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index 229ed22b507a3..187551dd90db7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.catalyst.catalog.v2 +package org.apache.spark.sql.connector.catalog import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, when} @@ -23,12 +23,11 @@ import org.scalatest.Inside import org.scalatest.Matchers._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.util.CaseInsensitiveStringMap -private case class TestCatalogPlugin(override val name: String) extends CatalogPlugin { +private case class DummyCatalogPlugin(override val name: String) extends CatalogPlugin { override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = Unit } @@ -36,7 +35,7 @@ private case class TestCatalogPlugin(override val name: String) extends CatalogP class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ - private val catalogs = Seq("prod", "test").map(x => x -> TestCatalogPlugin(x)).toMap + private val catalogs = Seq("prod", "test").map(x => x -> DummyCatalogPlugin(x)).toMap override val catalogManager: CatalogManager = { val manager = mock(classOf[CatalogManager]) @@ -128,7 +127,7 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ - private val catalogs = Seq("prod", "test").map(x => x -> TestCatalogPlugin(x)).toMap + private val catalogs = Seq("prod", "test").map(x => x -> DummyCatalogPlugin(x)).toMap override val catalogManager: CatalogManager = { val manager = mock(classOf[CatalogManager]) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala similarity index 99% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala index 3904790a5e5c9..265a940d66a65 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2 +package org.apache.spark.sql.connector.catalog import java.util import java.util.Collections diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/expressions/TransformExtractorSuite.scala similarity index 98% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/connector/expressions/TransformExtractorSuite.scala index c0a5dada19dba..fbd6a886d011b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/expressions/TransformExtractorSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalog.v2.expressions +package org.apache.spark.sql.connector.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java b/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java index 7c167dc012329..707812e5af61c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java @@ -18,10 +18,10 @@ package org.apache.spark.sql.execution.streaming; /** - * This class is an alias of {@link org.apache.spark.sql.sources.v2.reader.streaming.Offset}. It's + * This class is an alias of {@link org.apache.spark.sql.connector.read.streaming.Offset}. It's * internal and deprecated. New streaming data source implementations should use data source v2 API, * which will be supported in the long term. * * This class will be removed in a future release. */ -public abstract class Offset extends org.apache.spark.sql.sources.v2.reader.streaming.Offset {} +public abstract class Offset extends org.apache.spark.sql.connector.read.streaming.Offset {} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index f901005ad4fcf..b9cc25817d2f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -32,14 +32,14 @@ import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, Univocit import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} import org.apache.spark.sql.catalyst.util.FailureSafeParser +import org.apache.spark.sql.connector.catalog.SupportsRead +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils} -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 4489aece505d8..58acfb836b305 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -22,8 +22,6 @@ import java.util.{Locale, Properties} import scala.collection.JavaConverters._ import org.apache.spark.annotation.Stable -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ @@ -31,15 +29,15 @@ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsWrite, TableCatalog, TableProvider, V1Table} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, LiteralValue, Transform} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, DataSourceUtils, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.sources.v2.internal.V1Table import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -341,7 +339,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def insertInto(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ assertNotBucketed("insertInto") @@ -373,7 +371,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def insertInto(catalog: CatalogPlugin, ident: Identifier): Unit = { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val table = catalog.asTableCatalog.loadTable(ident) match { case _: V1Table => @@ -484,7 +482,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ def saveAsTable(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/v2/writer/V1WriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala similarity index 94% rename from sql/core/src/main/scala/org/apache/spark/sql/sources/v2/writer/V1WriteBuilder.scala rename to sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala index 2a88555e29276..e738ad1ede446 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/v2/writer/V1WriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.writer +package org.apache.spark.sql.connector.write import org.apache.spark.annotation.{Experimental, Unstable} +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.sources.InsertableRelation -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite /** * A trait that should be implemented by V1 DataSources that would like to leverage the DataSource diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 9376f08351791..0f5f1591623af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.catalog.TableProvider import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.DataWritingCommand import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat @@ -46,7 +47,6 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ -import org.apache.spark.sql.sources.v2.TableProvider import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{CalendarIntervalType, StructField, StructType} import org.apache.spark.sql.util.SchemaUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index 68d32059bfe3e..b9b8e069d5237 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -20,15 +20,15 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogManager, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation} -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, Filter, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables, SubqueryAlias} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables, SubqueryAlias} import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowDatabasesCommand, ShowTablesCommand} +import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, LookupCatalog, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} @@ -38,7 +38,7 @@ case class DataSourceResolution( catalogManager: CatalogManager) extends Rule[LogicalPlan] with CastSupport with LookupCatalog { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case CreateTableStatement( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index 2595cc6371bc2..c1ebc98fb1ddd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} import org.apache.spark.sql.types.StringType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala index 14bee173cc116..b4fc94e097aa8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.Partition import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.sources.v2.reader.InputPartition +import org.apache.spark.sql.connector.read.InputPartition /** * A collection of file blocks that should be read as a single task diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 9e98b0bbfabc9..542c996a5342d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -19,17 +19,14 @@ package org.apache.spark.sql.execution.datasources import java.io.{FileNotFoundException, IOException} -import scala.collection.mutable - import org.apache.parquet.io.ParquetDecodingException -import org.apache.spark.{Partition => RDDPartition, TaskContext, TaskKilledException} +import org.apache.spark.{Partition => RDDPartition, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.sources.v2.reader.InputPartition import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.NextIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala index e4f9e49c4dd28..3f4f29c3e135a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -22,10 +22,10 @@ import java.util import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 615e8a5f8a69a..c92c68095db8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} -import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala index a3fa82b12e938..4525f2608b710 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.SparkException import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog, TableChange} import org.apache.spark.sql.execution.LeafExecNode /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala index 0f98d9486bbbf..e4e7887017a1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan} /** * Physical plan node for scanning a batch of data from a data source v2. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala index f54ff608a53e3..dc95d157e40fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector.read.{InputPartition, Scan} +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, Offset} import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, Offset} /** * Physical plan node for scanning data from a streaming data source with continuous mode. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala index f35758bf08c67..600ed6f7c98ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.types.StructType @@ -35,7 +35,7 @@ case class CreateTableExec( partitioning: Seq[Transform], tableProperties: Map[String, String], ignoreIfExists: Boolean) extends LeafExecNode { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override protected def doExecute(): RDD[InternalRow] = { if (!catalog.tableExists(identifier)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourcePartitioning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourcePartitioning.scala index 33079d5912506..9211ec25525fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourcePartitioning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourcePartitioning.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Expression} import org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.sources.v2.reader.partitioning.{ClusteredDistribution, Partitioning} +import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Partitioning} /** * An adapter from public data source partitioning to catalyst internal `Partitioning`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala index f62f7349d1da7..71560f60f531c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} class DataSourceRDDPartition(val index: Int, val inputPartition: InputPartition) extends Partition with Serializable diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala index 74fc5432ea82c..211f61279ddd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.expressions.AttributeMap import org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan, SupportsReportPartitioning} import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, Scan, SupportsReportPartitioning} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 10dcf402ed61d..22100c7d3d593 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -17,24 +17,20 @@ package org.apache.spark.sql.execution.datasources.v2 -import java.util.UUID - import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} -import org.apache.spark.sql.catalog.v2.StagingTableCatalog import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables} +import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} import org.apache.spark.sql.sources -import org.apache.spark.sql.sources.v2.TableCapability -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} -import org.apache.spark.sql.sources.v2.writer.V1WriteBuilder import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Strategy extends Strategy with PredicateHelper { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala index 30897d86f8179..52294ae2cb851 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util.regex.Pattern import org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, TableProvider} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.{SessionConfigSupport, TableProvider} private[sql] object DataSourceV2Utils extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala index a5840571fff23..b91cbca8d2e70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DeleteFromTableExec.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.SupportsDelete import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.SupportsDelete import org.apache.spark.sql.util.CaseInsensitiveStringMap case class DeleteFromTableExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 640bdfb8cba54..e696efe950f98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -24,8 +24,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.LeafExecNode -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType case class DescribeTableExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala index d325e0205f9d8..9164b211ed0e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.execution.LeafExecNode /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala index b177d15e1fe32..711bd41e1db24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import java.io.IOException -import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReader /** * A [[PartitionReader]] with empty output. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala index db31927fa73bb..e7d9a247533c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala @@ -20,10 +20,9 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult} import org.apache.spark.sql.execution.datasources.FileFormatWriter.processStats -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.util.SerializableConfiguration class FileBatchWrite( job: Job, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index ac786bbaac6d7..e0091293d1669 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -20,9 +20,9 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.catalog.TableProvider import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.TableProvider import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala index 836eae88e4da7..1b55c181c7efc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -22,10 +22,10 @@ import org.apache.parquet.io.ParquetDecodingException import org.apache.spark.internal.Logging import org.apache.spark.rdd.InputFileBlockHolder +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader class FilePartitionReader[T](readers: Iterator[PartitionedFileReader[T]]) extends PartitionReader[T] with Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala index 5a19412c90334..c1d91736a8b8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala @@ -17,9 +17,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, PartitioningUtils} -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} import org.apache.spark.sql.vectorized.ColumnarBatch abstract class FilePartitionReaderFactory extends PartitionReaderFactory { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 0438bd0430da1..55104a2b21deb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -25,10 +25,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.connector.read.{Batch, InputPartition, Scan, Statistics, SupportsReportStatistics} import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala index 3b236be90e6ff..21bc14c577bdc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownRequiredColumns} import org.apache.spark.sql.execution.datasources.{PartitioningAwareFileIndex, PartitioningUtils} -import org.apache.spark.sql.sources.v2.reader.{ScanBuilder, SupportsPushDownRequiredColumns} import org.apache.spark.sql.types.StructType abstract class FileScanBuilder( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala index 4483f5b1dd30c..5329e09916bd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -23,11 +23,11 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFileIndex} -import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.util.SchemaUtils @@ -39,7 +39,7 @@ abstract class FileTable( userSpecifiedSchema: Option[StructType]) extends Table with SupportsRead with SupportsWrite { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ lazy val fileIndex: PartitioningAwareFileIndex = { val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala index eacc4cb3ac4a9..65a2c61ba090f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala @@ -30,10 +30,10 @@ import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.connector.write.{BatchWrite, WriteBuilder} import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, DataSource, OutputWriterFactory, WriteJobDescription} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.util.SchemaUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala index eb573b317142a..1f25fed3000b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala @@ -23,9 +23,8 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory} import org.apache.spark.sql.execution.datasources.{DynamicPartitionDataWriter, SingleDirectoryDataWriter, WriteJobDescription} -import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} -import org.apache.spark.util.SerializableConfiguration case class FileWriterFactory ( description: WriteJobDescription, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala index a9b0f5bce1b09..bca28e3cacb62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, Scan} -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} /** * Physical plan node for scanning a micro-batch of data from a data source. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala index f9dfcf448a3ea..0d9aa5b42a6ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReader class PartitionReaderFromIterator[InternalRow]( iter: Iterator[InternalRow]) extends PartitionReader[InternalRow] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala index 072465b56857d..7bca98e54efa7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala index baa8cb6b24659..8e524a986aa06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.hadoop.mapreduce.RecordReader -import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.connector.read.PartitionReader class PartitionRecordReader[T]( private[this] var rowReader: RecordReader[_, T]) extends PartitionReader[T] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index 35d86ee2abbbb..a56ccd6a6a9f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -20,13 +20,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException} import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.LeafExecNode -import org.apache.spark.sql.sources.v2.StagedTable import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala index f70b943fd2cde..49f55be52de16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.NamespaceHelper -import org.apache.spark.sql.catalog.v2.SupportsNamespaces import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper +import org.apache.spark.sql.connector.catalog.SupportsNamespaces import org.apache.spark.sql.execution.LeafExecNode /** @@ -37,6 +37,7 @@ case class ShowNamespacesExec( namespace: Option[Seq[String]], pattern: Option[String]) extends LeafExecNode { + override protected def doExecute(): RDD[InternalRow] = { val namespaces = namespace.map { ns => if (ns.nonEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index c652f28a5e760..e352bc6759623 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.NamespaceHelper -import org.apache.spark.sql.catalog.v2.TableCatalog import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.execution.LeafExecNode /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala index 660b6e763e056..51993b1e9ad60 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.BooleanType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala index 2f05ff3a7c2e1..bf67e972976b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala @@ -19,18 +19,16 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util.UUID -import scala.collection.JavaConverters._ - import org.apache.spark.SparkException import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Dataset, SaveMode} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.SupportsWrite +import org.apache.spark.sql.connector.write.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.sources.{AlwaysTrue, CreatableRelationProvider, Filter, InsertableRelation} -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table} -import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.sources.{AlwaysTrue, Filter, InsertableRelation} import org.apache.spark.sql.util.CaseInsensitiveStringMap /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 75320fb51db3f..525d9c4e541e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -23,18 +23,14 @@ import java.util import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalog.v2.{CatalogManager, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange} -import org.apache.spark.sql.catalog.v2.NamespaceChange.{RemoveProperty, SetProperty} -import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, FieldReference, IdentityTransform, LogicalExpressions, Transform} -import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogDatabase, CatalogTable, CatalogTableType, CatalogUtils, SessionCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.NamespaceChange.RemoveProperty +import org.apache.spark.sql.connector.expressions.{BucketTransform, FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.internal.{SessionState, SQLConf} -import org.apache.spark.sql.sources.v2.Table -import org.apache.spark.sql.sources.v2.internal.V1Table +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -43,7 +39,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap */ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) extends TableCatalog with SupportsNamespaces { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import V2SessionCatalog._ override val defaultNamespace: Array[String] = Array("default") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 0131d72ebc97a..9f4392da6ab4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -26,16 +26,15 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.executor.CommitDeniedException import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} -import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, SupportsWrite, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.sources.{AlwaysTrue, Filter} -import org.apache.spark.sql.sources.v2.{StagedTable, SupportsWrite} -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{LongAccumulator, Utils} @@ -69,7 +68,7 @@ case class CreateTableAsSelectExec( writeOptions: CaseInsensitiveStringMap, ifNotExists: Boolean) extends V2TableWriteExec with SupportsV1Write { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper override protected def doExecute(): RDD[InternalRow] = { if (catalog.tableExists(ident)) { @@ -158,7 +157,7 @@ case class ReplaceTableAsSelectExec( writeOptions: CaseInsensitiveStringMap, orCreate: Boolean) extends V2TableWriteExec with SupportsV1Write { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper override protected def doExecute(): RDD[InternalRow] = { // Note that this operation is potentially unsafe, but these are the strict semantics of @@ -470,7 +469,7 @@ object DataWritingSparkTask extends Logging { } private[v2] trait AtomicTableWriteExec extends V2TableWriteExec with SupportsV1Write { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper protected def writeToStagedTable( stagedTable: StagedTable, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala index 045f41e670ad3..1f99d4282f6da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.csv +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index 828594ffb10af..a20b0f1560a1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index 3cbcfca01a9c3..5125de9313a4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -23,11 +23,11 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.csv.CSVOptions import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.csv.{CSVDataSource, MultiLineCSVDataSource} +import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala index 28c5b3d81a3d5..8b486d0344506 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala index 8170661a70172..04beee0e3b0f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala @@ -22,10 +22,10 @@ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{AtomicType, DataType, StructType, UserDefinedType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala index 610bd4c1b9d85..7a0949e586cd8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.json +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.v2._ -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala index e5b7ae0bd228a..698423948f916 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2.json import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptionsInRead} +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.json.JsonDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala index 5c41bbd931982..a64b78d3c8305 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.JSONOptionsInRead import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.json.JsonDataSource -import org.apache.spark.sql.execution.datasources.v2.{FileScan, TextBasedFileScan} -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala index bb3c0366bdc2f..be53b1b1676f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.execution.datasources.v2.json import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala index bbdd3ae69222a..9bb615528fc5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala @@ -22,10 +22,10 @@ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.json.JSONOptionsInRead +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.json.JsonDataSource import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala index 1ea80d2ba5fbc..8665af33b976a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.orc +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.v2._ -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index ec923797e2691..3d6206128cabd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -29,13 +29,11 @@ import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.JoinedRow -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningUtils} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} +import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcUtils} import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index a4fb03405d162..40784516a6f34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -20,10 +20,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScan import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 458b98c627be4..8d1d4ec45915b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -22,11 +22,11 @@ import scala.collection.JavaConverters._ import org.apache.orc.mapreduce.OrcInputFormat import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.orc.OrcFilters import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala index 3fe433861a3c4..f2e4b88e9f1ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala @@ -21,10 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.orc.OrcUtils import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala index 0b6d5a960374b..8cb6186c12ff3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.parquet +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2._ -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala index a0f19c3dd2eb4..ded145f7de94e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala @@ -33,12 +33,12 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} import org.apache.spark.sql.execution.datasources.{PartitionedFile, RecordReaderIterator} import org.apache.spark.sql.execution.datasources.parquet._ import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala index a67aa3b92ce82..7e6ea41cf0b8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala @@ -21,13 +21,12 @@ import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetInputFormat import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetWriteSupport} import org.apache.spark.sql.execution.datasources.v2.FileScan -import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala index 4b8b434af88e6..87db00077e798 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.execution.datasources.v2.parquet import scala.collection.JavaConverters._ import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.parquet.{ParquetFilters, SparkToParquetSchemaConverter} import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.reader.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala index dce851dbcd336..2ad64b1aa5244 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala @@ -21,10 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala index f6aa1e9c898b9..049c717effa26 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2.text +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 -import org.apache.spark.sql.sources.v2.Table import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala index 8788887111880..0cd184da6ef8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala @@ -21,11 +21,11 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter +import org.apache.spark.sql.connector.read.PartitionReader import org.apache.spark.sql.execution.datasources.{HadoopFileLinesReader, HadoopFileWholeTextReader, PartitionedFile} import org.apache.spark.sql.execution.datasources.text.TextOptions import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.PartitionReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala index 89b0511442d4a..a2c42db59d7fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala @@ -21,10 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.text.TextOptions import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala index fbe5e1688b836..b2b518c12b01a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources.v2.text import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala index b8cb61a6c646e..87bfa84985e5a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.v2.text import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{DataType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala index c9c2ebc875f28..7eb9768e483dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ContinuousRecordEndpoint.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.SparkEnv import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset +import org.apache.spark.sql.connector.read.streaming.PartitionOffset case class ContinuousRecordPartitionOffset(partitionId: Int, offset: Int) extends PartitionOffset case class GetRecord(offset: ContinuousRecordPartitionOffset) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index e7eb2cb558cdb..5fe1f92e396c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -24,12 +24,12 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, Project} import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamWriterCommitProgress, WriteToDataSourceV2Exec} import org.apache.spark.sql.execution.streaming.sources.{RateControlMicroBatchStream, WriteToMicroBatchDataSource} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala index b6fa2e9dc3612..357c049aa18fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala @@ -22,9 +22,9 @@ import org.json4s.jackson.Serialization import org.apache.spark.internal.Logging import org.apache.spark.sql.RuntimeConfig +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, StreamingAggregationStateManager} import org.apache.spark.sql.internal.SQLConf.{FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION, _} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala index b40426aff0e79..f6543c3e4c4ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala @@ -24,7 +24,7 @@ import java.nio.charset.StandardCharsets._ import scala.io.{Source => IOSource} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} /** * This class is used to log offsets to persistent files in HDFS. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 6cb75083d0c0b..204c6a4febfd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -27,10 +27,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, SparkDataStream} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.{MicroBatchScanExec, StreamingDataSourceV2Relation, StreamWriterCommitProgress} -import org.apache.spark.sql.sources.v2.Table -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, SparkDataStream} import org.apache.spark.sql.streaming._ import org.apache.spark.sql.streaming.StreamingQueryListener.QueryProgressEvent import org.apache.spark.util.Clock diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala index 02fed50485b94..84f0961e4af12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/RateStreamOffset.scala @@ -20,10 +20,10 @@ package org.apache.spark.sql.execution.streaming import org.json4s.DefaultFormats import org.json4s.jackson.Serialization -import org.apache.spark.sql.sources.v2 +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} case class RateStreamOffset(partitionToValueAndRunTimeMs: Map[Int, ValueRunTimeMsPair]) - extends v2.reader.streaming.Offset { + extends OffsetV2 { implicit val defaultFormats: DefaultFormats = DefaultFormats override val json = Serialization.write(partitionToValueAndRunTimeMs) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala index 190325fb7ec25..36c7796ec4399 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming import java.util import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{Table, TableCapability} import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala index 7f66d0b055cc3..6d51d7dc44171 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} -import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 7c1f6ca42c1f2..debf1fad57308 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -22,7 +22,7 @@ import java.nio.channels.ClosedByInterruptException import java.util.UUID import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} import java.util.concurrent.atomic.AtomicReference -import java.util.concurrent.locks.{Condition, ReentrantLock} +import java.util.concurrent.locks.ReentrantLock import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MutableMap} @@ -36,14 +36,14 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} +import org.apache.spark.sql.connector.write.SupportsTruncate +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.command.StreamingExplainCommand import org.apache.spark.sql.execution.datasources.v2.StreamWriterCommitProgress import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} -import org.apache.spark.sql.sources.v2.writer.SupportsTruncate -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala index 8783eaa0e68b3..c6634f51d0382 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.execution.streaming import scala.collection.{immutable, GenTraversableOnce} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} - +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} /** * A helper class that looks like a Map[Source, Offset]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala index 142b6e7d18068..5858c54ce554a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala @@ -23,10 +23,10 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.connector.catalog.{Table, TableProvider} +import org.apache.spark.sql.connector.read.streaming.SparkDataStream import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.sources.v2.{Table, TableProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream import org.apache.spark.sql.util.CaseInsensitiveStringMap object StreamingRelation { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala index 9ae39c79c5156..20eb7ae5a6d96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -22,11 +22,11 @@ import java.util import scala.collection.JavaConverters._ import org.apache.spark.sql._ +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.write.{SupportsTruncate, WriteBuilder} +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.sources.ConsoleWrite import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister} -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.writer.{SupportsTruncate, WriteBuilder} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala index b68f67e0b22d9..5ee27c71aa731 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.streaming.continuous import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousPartitionReaderFactory +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReaderFactory import org.apache.spark.sql.types.StructType import org.apache.spark.util.NextIterator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index f6d156ded7663..8c7371e75b539 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -28,12 +28,11 @@ import org.apache.spark.SparkEnv import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, TableCapability} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, Offset => OffsetV2, PartitionOffset} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming.{StreamingRelationV2, _} -import org.apache.spark.sql.sources.v2 -import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, TableCapability} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock @@ -340,7 +339,7 @@ class ContinuousExecution( val offset = sources(0).deserializeOffset(offsetLog.get(epoch).get.offsets(0).get.json) committedOffsets ++= Seq(sources(0) -> offset) - sources(0).commit(offset.asInstanceOf[v2.reader.streaming.Offset]) + sources(0).commit(offset.asInstanceOf[OffsetV2]) } else { return } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala index 65c5fc63c2f46..dff2fa69e42fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala @@ -26,7 +26,7 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeProjection -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, PartitionOffset} +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, PartitionOffset} import org.apache.spark.sql.types.StructType import org.apache.spark.util.ThreadUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index e1b7a8fc283d3..e66a1fe48a2e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -22,9 +22,9 @@ import org.json4s.jackson.Serialization import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} import org.apache.spark.sql.execution.streaming.{RateStreamOffset, ValueRunTimeMsPair} -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ case class RateStreamPartitionOffset( partition: Int, currentValue: Long, currentTimeMs: Long) extends PartitionOffset diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala index 2263b42870a65..aaeb13defd34b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala @@ -32,10 +32,10 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} import org.apache.spark.sql.execution.streaming.{Offset => _, _} import org.apache.spark.sql.execution.streaming.sources.TextSocketReader -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.RpcUtils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala index a08411d746abe..6fcd6ba64f65d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.streaming.continuous import org.apache.spark.{Partition, SparkEnv, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.DataWriter -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory +import org.apache.spark.sql.connector.write.DataWriter +import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory import org.apache.spark.util.Utils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala index decf524f7167c..dbddab2e9acdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala @@ -23,9 +23,9 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, PartitionOffset} +import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.util.RpcUtils private[continuous] sealed trait EpochCoordinatorMessage extends Serializable diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala index 54f484c4adae3..cecb2843fc3b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.streaming.continuous import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite +import org.apache.spark.sql.connector.write.streaming.StreamingWrite /** * The logical plan for writing data in a continuous stream. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala index 2f3af6a6544c4..d4e522562e914 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala @@ -24,9 +24,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.streaming.StreamExecution -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite /** * The physical plan for writing data into a continuous processing [[StreamingWrite]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index df149552dfb30..26cf1fb8def56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -31,10 +31,10 @@ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala index dbe242784986d..6afb811a4d998 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} +import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala index 41eaf84b7f9ea..8f89ccd698763 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala @@ -29,9 +29,9 @@ import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.{Encoder, SQLContext} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} import org.apache.spark.sql.execution.streaming.{Offset => _, _} -import org.apache.spark.sql.sources.v2.reader.InputPartition -import org.apache.spark.sql.sources.v2.reader.streaming._ import org.apache.spark.util.RpcUtils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala index 838c7d497e35b..bae7fa7d07356 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala @@ -26,10 +26,10 @@ import org.apache.spark.sql.{ForeachWriter, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.write.{DataWriter, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.python.PythonForeachWriter -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.writer.{DataWriter, SupportsTruncate, WriteBuilder, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala index f3951897ea747..5f12832cd2550 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} /** * A [[BatchWrite]] used to hook V2 stream writers into a microbatch plan. It implements diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala index fd4cb444ce580..53f56edc27681 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala @@ -21,8 +21,8 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory /** * A simple [[DataWriterFactory]] whose tasks just pack rows into the commit message for delivery diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala index 6a66f52c8f732..fb46f76682688 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming.sources -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} // A special `MicroBatchStream` that can get latestOffset with a start offset. trait RateControlMicroBatchStream extends MicroBatchStream { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala index 156ba95ab9733..eb6baf698a5b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala @@ -27,9 +27,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{ManualClock, SystemClock} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala index f61e9dbecd4ea..3f7b0377f1eab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala @@ -23,11 +23,11 @@ import scala.collection.JavaConverters._ import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousStream import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{Scan, ScanBuilder} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala index 25e9af2bc2927..97a6576832515 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala @@ -28,9 +28,9 @@ import scala.collection.mutable.ListBuffer import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.execution.streaming.LongOffset -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} import org.apache.spark.unsafe.types.UTF8String /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala index 0f807e235661a..fae3cb765c0c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala @@ -26,11 +26,11 @@ import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging import org.apache.spark.sql._ +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousStream import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{Scan, ScanBuilder} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala index a3f58fa966fe8..ef1115e6d9e01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2 -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite /** * The logical plan for writing data to a micro-batch stream. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala index de8d00d4ac348..51ab5ce3578af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala @@ -32,10 +32,10 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.streaming.Sink -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index a83a0f51ecf11..92e255ae8aaa8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -24,12 +24,12 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Unstable import org.apache.spark.sql._ -import org.apache.spark.sql.catalog.v2.CatalogManager import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution._ import org.apache.spark.sql.streaming.StreamingQueryManager import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListener} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 23a84cbd0dc02..4a6516d325ddd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -24,13 +24,13 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.connector.catalog.{SupportsRead, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 36104d7a70443..74170b1b5d77e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -25,13 +25,13 @@ import org.apache.spark.annotation.Evolving import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes +import org.apache.spark.sql.connector.catalog.{SupportsWrite, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources._ -import org.apache.spark.sql.sources.v2.{SupportsWrite, TableProvider} -import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.util.CaseInsensitiveStringMap /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 976595616bd28..9abe38dfda0be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -31,12 +31,12 @@ import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS -import org.apache.spark.sql.sources.v2.{SupportsWrite, Table} import org.apache.spark.util.{Clock, SystemClock, Utils} /** diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2.java similarity index 96% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2.java index 255a9f887878b..9386ab51d64f0 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.io.IOException; import java.util.*; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.*; import org.apache.spark.sql.sources.Filter; import org.apache.spark.sql.sources.GreaterThan; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaColumnarDataSourceV2.java similarity index 90% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaColumnarDataSourceV2.java index 699859cfaebe1..76da45e182b3c 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaColumnarDataSourceV2.java @@ -15,15 +15,18 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.io.IOException; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.apache.spark.sql.vectorized.ColumnVector; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaPartitionAwareDataSource.java similarity index 86% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaPartitionAwareDataSource.java index 391af5a306a16..fbbc457b2945d 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaPartitionAwareDataSource.java @@ -15,21 +15,21 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.io.IOException; import java.util.Arrays; -import org.apache.spark.sql.catalog.v2.expressions.Expressions; -import org.apache.spark.sql.catalog.v2.expressions.Transform; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; -import org.apache.spark.sql.sources.v2.reader.partitioning.ClusteredDistribution; -import org.apache.spark.sql.sources.v2.reader.partitioning.Distribution; -import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; +import org.apache.spark.sql.connector.expressions.Expressions; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.*; +import org.apache.spark.sql.connector.read.partitioning.ClusteredDistribution; +import org.apache.spark.sql.connector.read.partitioning.Distribution; +import org.apache.spark.sql.connector.read.partitioning.Partitioning; import org.apache.spark.sql.util.CaseInsensitiveStringMap; public class JavaPartitionAwareDataSource implements TableProvider { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaRangeInputPartition.java similarity index 90% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaRangeInputPartition.java index 438f489a3eea7..d612441201e64 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaRangeInputPartition.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; -import org.apache.spark.sql.sources.v2.reader.InputPartition; +import org.apache.spark.sql.connector.read.InputPartition; class JavaRangeInputPartition implements InputPartition { int start; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java similarity index 82% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java index f3755e18b58d5..49438fe668d56 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.util.OptionalLong; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.ScanBuilder; -import org.apache.spark.sql.sources.v2.reader.Statistics; -import org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.SupportsReportStatistics; import org.apache.spark.sql.util.CaseInsensitiveStringMap; public class JavaReportStatisticsDataSource implements TableProvider { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSchemaRequiredDataSource.java similarity index 87% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSchemaRequiredDataSource.java index 3800a94f88898..2181887ae54e2 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSchemaRequiredDataSource.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java similarity index 86% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java index 64663d5db4bed..97b00477e1764 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import java.util.Arrays; import java.util.HashSet; import java.util.Set; -import org.apache.spark.sql.sources.v2.SupportsRead; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableCapability; +import org.apache.spark.sql.connector.catalog.SupportsRead; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; import org.apache.spark.sql.types.StructType; abstract class JavaSimpleBatchTable implements Table, SupportsRead { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleDataSourceV2.java similarity index 84% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleDataSourceV2.java index 7474f36c97f75..8b6d71b986ff7 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleDataSourceV2.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; -import org.apache.spark.sql.sources.v2.Table; -import org.apache.spark.sql.sources.v2.TableProvider; -import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.util.CaseInsensitiveStringMap; public class JavaSimpleDataSourceV2 implements TableProvider { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleReaderFactory.java similarity index 86% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleReaderFactory.java index 740279033c416..0c702031a939b 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleReaderFactory.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; class JavaSimpleReaderFactory implements PartitionReaderFactory { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java similarity index 81% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java rename to sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java index 217e66950d146..7cbba00420928 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package test.org.apache.spark.sql.sources.v2; +package test.org.apache.spark.sql.connector; -import org.apache.spark.sql.sources.v2.reader.Batch; -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; -import org.apache.spark.sql.sources.v2.reader.Scan; -import org.apache.spark.sql.sources.v2.reader.ScanBuilder; +import org.apache.spark.sql.connector.read.Batch; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; abstract class JavaSimpleScanBuilder implements ScanBuilder, Scan, Batch { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/AlterTableTests.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 4b7ee384b4c10..03cefa0d2e77a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala similarity index 94% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index 629e825ec68dd..207ece83759ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -15,21 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.util import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog, TableChange} -import org.apache.spark.sql.catalog.v2.expressions.Transform -import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.connector.InMemoryTable +import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG -import org.apache.spark.sql.sources.v2.utils.TestV2SessionCatalogBase import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -38,8 +35,6 @@ class DataSourceV2DataFrameSessionCatalogSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = false) with SessionCatalogTest[InMemoryTable, InMemoryTableSessionCatalog] { - import testImplicits._ - override protected def doInsert(tableName: String, insert: DataFrame, mode: SaveMode): Unit = { val dfw = insert.write.format(v2Format) if (mode != null) { @@ -135,7 +130,7 @@ class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable } } -private[v2] trait SessionCatalogTest[T <: Table, Catalog <: TestV2SessionCatalogBase[T]] +private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2SessionCatalogBase[T]] extends QueryTest with SharedSparkSession with BeforeAndAfter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala similarity index 97% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index abccb5cec6752..cd811bb7afb51 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import org.apache.spark.sql.{DataFrame, Row, SaveMode} -import org.apache.spark.sql.connector.InMemoryTableCatalog class DataSourceV2DataFrameSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = false) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala similarity index 89% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala index 8df65c14a8d95..e1a5dbe3351ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import org.apache.spark.sql.{DataFrame, SaveMode} -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} -import org.apache.spark.sql.connector.InMemoryTable -import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} class DataSourceV2SQLSessionCatalogSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 4bfbefef44bb2..ef484ce6ac362 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -15,17 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import org.apache.spark.sql._ -import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableCatalog} import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.connector.{BasicInMemoryTableCatalog, InMemoryTable, InMemoryTableCatalog, StagingInMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG -import org.apache.spark.sql.sources.v2.internal.V1Table import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -33,7 +31,7 @@ class DataSourceV2SQLSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = true) with AlterTableTests { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ private val v2Source = classOf[FakeV2Provider].getName override protected val v2Format = v2Source diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 8f7dbe8d13c39..00a8b430d33c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.io.File import java.util @@ -23,19 +23,20 @@ import java.util.OptionalLong import scala.collection.JavaConverters._ -import test.org.apache.spark.sql.sources.v2._ +import test.org.apache.spark.sql.connector._ import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.functions._ import org.apache.spark.sql.sources.{Filter, GreaterThan} -import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.partitioning.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2UtilsSuite.scala similarity index 95% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2UtilsSuite.scala index 0b1e3b5fb076d..01fcced5b12a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2UtilsSuite.scala @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.connector.catalog.SessionConfigSupport import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala similarity index 96% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala index 26f941244f5cc..cd804adfa2133 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -14,21 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.{FileSourceScanExec, QueryExecution} import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.ScanBuilder -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.{CaseInsensitiveStringMap, QueryExecutionListener} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/InsertIntoTests.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index 5b5382e5ca931..0fd6cf1b6746c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import org.scalatest.BeforeAndAfter @@ -175,7 +175,7 @@ abstract class InsertIntoTests( } } -private[v2] trait InsertIntoSQLOnlyTests +trait InsertIntoSQLOnlyTests extends QueryTest with SharedSparkSession with BeforeAndAfter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala similarity index 94% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala index c9d2f1eef24bb..22d3750022c57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.io.{BufferedReader, InputStreamReader, IOException} import java.util @@ -27,9 +27,10 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, ScanBuilder} +import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -190,7 +191,7 @@ class CSVReaderFactory(conf: SerializableConfiguration) } } -private[v2] object SimpleCounter { +private[connector] object SimpleCounter { private var count: Int = 0 def increaseCounter: Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala similarity index 96% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TableCapabilityCheckSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index ab47836001704..39f4085a9baf9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.util @@ -24,12 +24,13 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.analysis.{AnalysisSuite, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LeafNode, OverwriteByExpression, OverwritePartitionsDynamic, Union} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.catalog.{Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, TableCapabilityCheck} import org.apache.spark.sql.execution.streaming.{Offset, Source, StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_WRITE, CONTINUOUS_READ, MICRO_BATCH_READ, OVERWRITE_BY_FILTER, OVERWRITE_DYNAMIC, TRUNCATE} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala similarity index 88% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala index b25eab1546261..d03294cb4067b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/utils/TestV2SessionCatalogBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2.utils +package org.apache.spark.sql.connector import java.util import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalog.v2.{DelegatingCatalogExtension, Identifier} -import org.apache.spark.sql.catalog.v2.expressions.Transform -import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType /** @@ -32,7 +31,7 @@ import org.apache.spark.sql.types.StructType * for testing DDL as well as write operations (through df.write.saveAsTable, df.write.insertInto * and SQL). */ -private[v2] abstract class TestV2SessionCatalogBase[T <: Table] extends DelegatingCatalogExtension { +private[connector] trait TestV2SessionCatalogBase[T <: Table] extends DelegatingCatalogExtension { protected val tables: util.Map[Identifier, T] = new ConcurrentHashMap[Identifier, T]() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala similarity index 92% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V1WriteFallbackSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index 9002775bce211..7cd6ba21b56ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.connector import java.util @@ -24,13 +24,11 @@ import scala.collection.mutable import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession} -import org.apache.spark.sql.catalog.v2.expressions.{FieldReference, IdentityTransform, Transform} -import org.apache.spark.sql.connector.InMemoryTable -import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode} +import org.apache.spark.sql.{DataFrame, QueryTest, Row, SaveMode, SparkSession} +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} +import org.apache.spark.sql.connector.write.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} import org.apache.spark.sql.sources.{DataSourceRegister, Filter, InsertableRelation} -import org.apache.spark.sql.sources.v2.utils.TestV2SessionCatalogBase -import org.apache.spark.sql.sources.v2.writer.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index e2cee593af030..7110f13e2ead9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -25,16 +25,15 @@ import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalog.v2.{CatalogManager, CatalogNotFoundException, Identifier, TableCatalog} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} -import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.{InMemoryTableCatalog, InMemoryTableProvider} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, TableCatalog} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSourceResolution} import org.apache.spark.sql.internal.SQLConf.DEFAULT_V2_CATALOG -import org.apache.spark.sql.sources.v2.InMemoryTableProvider import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala index ad0dfadacca15..7f4bbcf97b534 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala @@ -21,10 +21,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.{QueryTest, SparkSession} +import org.apache.spark.sql.connector.read.ScanBuilder +import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.text.TextFileFormat -import org.apache.spark.sql.sources.v2.reader.ScanBuilder -import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index a309152b26143..6bd6e0d89a677 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -24,17 +24,16 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter -import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalog.v2.{Catalogs, Identifier, NamespaceChange, TableChange} import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.connector.catalog.{Catalogs, Identifier, NamespaceChange, TableChange} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class V2SessionCatalogBaseSuite extends SparkFunSuite with SharedSparkSession with BeforeAndAfter { +abstract class V2SessionCatalogBaseSuite extends SharedSparkSession with BeforeAndAfter { val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] val schema: StructType = new StructType() @@ -54,7 +53,7 @@ class V2SessionCatalogBaseSuite extends SparkFunSuite with SharedSparkSession wi class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override protected def beforeAll(): Unit = { super.beforeAll() @@ -752,7 +751,7 @@ class V2SessionCatalogTableSuite extends V2SessionCatalogBaseSuite { class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { - import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ def checkMetadata( expected: scala.collection.Map[String, String], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala index ef88598fcb11b..6440e69e2ec23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala @@ -24,12 +24,12 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ManualClock diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala index e1284ea03267e..b1e6459838151 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala @@ -29,12 +29,12 @@ import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 7914a713f0baa..e54a5370d351d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -37,12 +37,12 @@ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, Ro import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.AllTuples import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, EpochCoordinatorRef, IncrementAndGetEpoch} import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.execution.streaming.state.StateStore -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.{Clock, SystemClock, Utils} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index d96404863a255..d84dc62a696b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -30,9 +30,9 @@ import org.scalatest.concurrent.Waiters.Waiter import org.apache.spark.SparkException import org.apache.spark.scheduler._ import org.apache.spark.sql.{Encoder, SparkSession} +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.util.JsonProtocol diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 3ad893f871c94..4a674b647917a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -35,13 +35,13 @@ import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid} +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{MemorySink, TestForeachWriter} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.InputPartition -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} import org.apache.spark.sql.streaming.util.{BlockingSource, MockSourceProvider, StreamManualClock} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala index bad22590807a7..5d6050dbda0fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala @@ -26,9 +26,9 @@ import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousStream, PartitionOffset} +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, ContinuousStream, PartitionOffset} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types.{DataType, IntegerType, StructType} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala index e3498db4194e8..48998f5d6c5e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala @@ -26,11 +26,11 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.LocalSparkSession +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, PartitionOffset} +import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.internal.SQLConf.CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.test.TestSparkSession class EpochCoordinatorSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index 4db605ee1b238..e9d148c38e6cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -23,16 +23,16 @@ import java.util.Collections import scala.collection.JavaConverters._ import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, SupportsRead, SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, MicroBatchStream, Offset, PartitionOffset} +import org.apache.spark.sql.connector.write.{WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RateStreamOffset, Sink, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.TableCapability._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ -import org.apache.spark.sql.sources.v2.writer.{WriteBuilder, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery, StreamTest, Trigger} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap From 8e9fafbb21a26028710df1831fc16b41855f0b4a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 12 Sep 2019 21:32:35 +0900 Subject: [PATCH 0051/1387] [SPARK-29065][SQL][TEST] Extend `EXTRACT` benchmark ### What changes were proposed in this pull request? In the PR, I propose to extend `ExtractBenchmark` and add new ones for: - `EXTRACT` and `DATE` as input column - the `DATE_PART` function and `DATE`/`TIMESTAMP` input column ### Why are the changes needed? The `EXTRACT` expression is rebased on the `DATE_PART` expression by the PR https://github.com/apache/spark/pull/25410 where some of sub-expressions take `DATE` column as the input (`Millennium`, `Year` and etc.) but others require `TIMESTAMP` column (`Hour`, `Minute`). Separate benchmarks for `DATE` should exclude overhead of implicit conversions `DATE` <-> `TIMESTAMP`. ### Does this PR introduce any user-facing change? No, it doesn't. ### How was this patch tested? - Regenerated results of `ExtractBenchmark` Closes #25772 from MaxGekk/date_part-benchmark. Authored-by: Maxim Gekk Signed-off-by: Takeshi Yamamuro --- .../benchmarks/ExtractBenchmark-results.txt | 243 +++++++----------- .../benchmark/ExtractBenchmark.scala | 95 ++++--- 2 files changed, 161 insertions(+), 177 deletions(-) diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 7ee185e9adc59..31ad7878ee643 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -1,145 +1,100 @@ -================================================================================================ -Extract -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -cast to timestamp wholestage off 407 432 36 24.6 40.7 1.0X -cast to timestamp wholestage on 348 396 80 28.7 34.8 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -MILLENNIUM of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -MILLENNIUM of timestamp wholestage off 1407 1408 2 7.1 140.7 1.0X -MILLENNIUM of timestamp wholestage on 1334 1380 81 7.5 133.4 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -CENTURY of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -CENTURY of timestamp wholestage off 1362 1364 3 7.3 136.2 1.0X -CENTURY of timestamp wholestage on 1334 1342 8 7.5 133.4 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -DECADE of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -DECADE of timestamp wholestage off 1226 1229 4 8.2 122.6 1.0X -DECADE of timestamp wholestage on 1218 1225 8 8.2 121.8 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -YEAR of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -YEAR of timestamp wholestage off 1207 1210 4 8.3 120.7 1.0X -YEAR of timestamp wholestage on 1201 1216 17 8.3 120.1 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -ISOYEAR of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -ISOYEAR of timestamp wholestage off 1442 1446 6 6.9 144.2 1.0X -ISOYEAR of timestamp wholestage on 1315 1336 18 7.6 131.5 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -QUARTER of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -QUARTER of timestamp wholestage off 1443 1454 16 6.9 144.3 1.0X -QUARTER of timestamp wholestage on 1429 1442 9 7.0 142.9 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -MONTH of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -MONTH of timestamp wholestage off 1196 1200 5 8.4 119.6 1.0X -MONTH of timestamp wholestage on 1192 1204 10 8.4 119.2 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -WEEK of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -WEEK of timestamp wholestage off 2103 2104 2 4.8 210.3 1.0X -WEEK of timestamp wholestage on 1798 1804 8 5.6 179.8 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -DAY of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -DAY of timestamp wholestage off 1211 1228 23 8.3 121.1 1.0X -DAY of timestamp wholestage on 1204 1212 6 8.3 120.4 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -DAYOFWEEK of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -DAYOFWEEK of timestamp wholestage off 1387 1389 3 7.2 138.7 1.0X -DAYOFWEEK of timestamp wholestage on 1353 1360 8 7.4 135.3 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -DOW of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -DOW of timestamp wholestage off 1373 1373 0 7.3 137.3 1.0X -DOW of timestamp wholestage on 1361 1372 15 7.3 136.1 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -ISODOW of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -ISODOW of timestamp wholestage off 1311 1366 77 7.6 131.1 1.0X -ISODOW of timestamp wholestage on 1307 1314 6 7.7 130.7 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -DOY of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -DOY of timestamp wholestage off 1241 1243 2 8.1 124.1 1.0X -DOY of timestamp wholestage on 1229 1239 9 8.1 122.9 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -HOUR of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -HOUR of timestamp wholestage off 353 358 8 28.3 35.3 1.0X -HOUR of timestamp wholestage on 358 365 5 27.9 35.8 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -MINUTE of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -MINUTE of timestamp wholestage off 353 354 2 28.3 35.3 1.0X -MINUTE of timestamp wholestage on 362 368 9 27.6 36.2 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SECOND of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -SECOND of timestamp wholestage off 341 350 13 29.3 34.1 1.0X -SECOND of timestamp wholestage on 362 368 7 27.6 36.2 0.9X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -MILLISECONDS of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -MILLISECONDS of timestamp wholestage off 36785 36808 32 0.3 3678.5 1.0X -MILLISECONDS of timestamp wholestage on 36644 36760 72 0.3 3664.4 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -MICROSECONDS of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -MICROSECONDS of timestamp wholestage off 446 447 0 22.4 44.6 1.0X -MICROSECONDS of timestamp wholestage on 458 463 4 21.8 45.8 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -EPOCH of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -EPOCH of timestamp wholestage off 29807 29811 5 0.3 2980.7 1.0X -EPOCH of timestamp wholestage on 29843 29930 64 0.3 2984.3 1.0X - +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to timestamp 264 281 25 37.9 26.4 1.0X +MILLENNIUM of timestamp 1187 1198 16 8.4 118.7 0.2X +CENTURY of timestamp 1122 1124 2 8.9 112.2 0.2X +DECADE of timestamp 1041 1049 7 9.6 104.1 0.3X +YEAR of timestamp 1027 1035 6 9.7 102.7 0.3X +ISOYEAR of timestamp 1155 1171 27 8.7 115.5 0.2X +QUARTER of timestamp 1181 1192 10 8.5 118.1 0.2X +MONTH of timestamp 1023 1030 7 9.8 102.3 0.3X +WEEK of timestamp 1511 1517 5 6.6 151.1 0.2X +DAY of timestamp 1010 1016 6 9.9 101.0 0.3X +DAYOFWEEK of timestamp 1127 1129 4 8.9 112.7 0.2X +DOW of timestamp 1123 1130 6 8.9 112.3 0.2X +ISODOW of timestamp 1099 1105 6 9.1 109.9 0.2X +DOY of timestamp 1029 1030 1 9.7 102.9 0.3X +HOUR of timestamp 415 417 1 24.1 41.5 0.6X +MINUTE of timestamp 409 418 13 24.4 40.9 0.6X +SECOND of timestamp 408 413 8 24.5 40.8 0.6X +MILLISECONDS of timestamp 28956 29040 73 0.3 2895.6 0.0X +MICROSECONDS of timestamp 504 519 13 19.8 50.4 0.5X +EPOCH of timestamp 23543 23566 28 0.4 2354.3 0.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to date 853 856 6 11.7 85.3 1.0X +MILLENNIUM of date 1081 1088 6 9.2 108.1 0.8X +CENTURY of date 1087 1090 4 9.2 108.7 0.8X +DECADE of date 1018 1019 1 9.8 101.8 0.8X +YEAR of date 996 1006 9 10.0 99.6 0.9X +ISOYEAR of date 1133 1147 21 8.8 113.3 0.8X +QUARTER of date 1246 1254 10 8.0 124.6 0.7X +MONTH of date 998 1002 4 10.0 99.8 0.9X +WEEK of date 1483 1490 7 6.7 148.3 0.6X +DAY of date 992 998 5 10.1 99.2 0.9X +DAYOFWEEK of date 1121 1128 7 8.9 112.1 0.8X +DOW of date 1118 1126 8 8.9 111.8 0.8X +ISODOW of date 1093 1103 9 9.1 109.3 0.8X +DOY of date 1026 1032 5 9.7 102.6 0.8X +HOUR of date 1707 1726 24 5.9 170.7 0.5X +MINUTE of date 1710 1731 19 5.8 171.0 0.5X +SECOND of date 1701 1720 19 5.9 170.1 0.5X +MILLISECONDS of date 2256 2272 19 4.4 225.6 0.4X +MICROSECONDS of date 1801 1810 11 5.6 180.1 0.5X +EPOCH of date 24848 24860 17 0.4 2484.8 0.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to timestamp 227 253 39 44.0 22.7 1.0X +MILLENNIUM of timestamp 1121 1127 9 8.9 112.1 0.2X +CENTURY of timestamp 1084 1094 17 9.2 108.4 0.2X +DECADE of timestamp 1015 1037 35 9.8 101.5 0.2X +YEAR of timestamp 1011 1013 4 9.9 101.1 0.2X +ISOYEAR of timestamp 1121 1126 7 8.9 112.1 0.2X +QUARTER of timestamp 1243 1247 4 8.0 124.3 0.2X +MONTH of timestamp 1000 1009 14 10.0 100.0 0.2X +WEEK of timestamp 1481 1492 11 6.8 148.1 0.2X +DAY of timestamp 999 1000 1 10.0 99.9 0.2X +DAYOFWEEK of timestamp 1114 1132 20 9.0 111.4 0.2X +DOW of timestamp 1135 1147 17 8.8 113.5 0.2X +ISODOW of timestamp 1070 1079 10 9.3 107.0 0.2X +DOY of timestamp 1018 1027 8 9.8 101.8 0.2X +HOUR of timestamp 413 419 9 24.2 41.3 0.6X +MINUTE of timestamp 411 414 3 24.3 41.1 0.6X +SECOND of timestamp 410 413 3 24.4 41.0 0.6X +MILLISECONDS of timestamp 29225 29264 52 0.3 2922.5 0.0X +MICROSECONDS of timestamp 507 512 7 19.7 50.7 0.4X +EPOCH of timestamp 23565 23608 56 0.4 2356.5 0.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to date 848 852 3 11.8 84.8 1.0X +MILLENNIUM of date 1083 1086 3 9.2 108.3 0.8X +CENTURY of date 1081 1084 5 9.3 108.1 0.8X +DECADE of date 1010 1014 4 9.9 101.0 0.8X +YEAR of date 992 1000 9 10.1 99.2 0.9X +ISOYEAR of date 1111 1116 6 9.0 111.1 0.8X +QUARTER of date 1230 1249 17 8.1 123.0 0.7X +MONTH of date 996 1008 11 10.0 99.6 0.9X +WEEK of date 1487 1516 28 6.7 148.7 0.6X +DAY of date 993 999 6 10.1 99.3 0.9X +DAYOFWEEK of date 1113 1118 5 9.0 111.3 0.8X +DOW of date 1113 1116 2 9.0 111.3 0.8X +ISODOW of date 1069 1072 3 9.4 106.9 0.8X +DOY of date 1027 1028 1 9.7 102.7 0.8X +HOUR of date 1707 1710 3 5.9 170.7 0.5X +MINUTE of date 1704 1705 2 5.9 170.4 0.5X +SECOND of date 1701 1705 4 5.9 170.1 0.5X +MILLISECONDS of date 2229 2238 9 4.5 222.9 0.4X +MICROSECONDS of date 1801 1808 12 5.6 180.1 0.5X +EPOCH of date 24783 24817 31 0.4 2478.3 0.0X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index dbbad43efa08c..b9086aa5cb378 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -19,6 +19,11 @@ package org.apache.spark.sql.execution.benchmark import java.time.Instant +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf + /** * Synthetic benchmark for the extract function. * To run this benchmark: @@ -31,52 +36,76 @@ import java.time.Instant * Results will be written to "benchmarks/ExtractBenchmark-results.txt". * }}} */ -object ExtractBenchmark extends SqlBasedBenchmark { +object ExtractBenchmark extends BenchmarkBase with SQLHelper { + private val spark: SparkSession = SparkSession.builder() + .master("local[1]") + .appName(this.getClass.getCanonicalName) + .getOrCreate() + private def doBenchmark(cardinality: Long, exprs: String*): Unit = { val sinceSecond = Instant.parse("2010-01-01T00:00:00Z").getEpochSecond - spark - .range(sinceSecond, sinceSecond + cardinality, 1, 1) - .selectExpr(exprs: _*) - .write - .format("noop") - .save() + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + spark + .range(sinceSecond, sinceSecond + cardinality, 1, 1) + .selectExpr(exprs: _*) + .write + .format("noop") + .save() + } } - private def run(cardinality: Long, name: String, exprs: String*): Unit = { - codegenBenchmark(name, cardinality) { + private def run( + benchmark: Benchmark, + cardinality: Long, + name: String, + exprs: String*): Unit = { + benchmark.addCase(name, numIters = 3) { _ => doBenchmark(cardinality, exprs: _*) } } - private def run(cardinality: Long, field: String): Unit = { - codegenBenchmark(s"$field of timestamp", cardinality) { - doBenchmark(cardinality, s"EXTRACT($field FROM (cast(id as timestamp)))") + private def castExpr(from: String): String = from match { + case "timestamp" => s"cast(id as timestamp)" + case "date" => s"cast(cast(id as timestamp) as date)" + case other => throw new IllegalArgumentException( + s"Unsupported column type $other. Valid column types are 'timestamp' and 'date'") + } + + private def run( + benchmark: Benchmark, + func: String, + cardinality: Long, + field: String, + from: String): Unit = { + val expr = func match { + case "extract" => s"EXTRACT($field FROM ${castExpr(from)})" + case "date_part" => s"DATE_PART('$field', ${castExpr(from)})" + case other => throw new IllegalArgumentException( + s"Unsupported function '$other'. Valid functions are 'extract' and 'date_part'.") + } + benchmark.addCase(s"$field of $from", numIters = 3) { _ => + doBenchmark(cardinality, expr) } } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val N = 10000000L - runBenchmark("Extract") { - run(N, "cast to timestamp", "cast(id as timestamp)") - run(N, "MILLENNIUM") - run(N, "CENTURY") - run(N, "DECADE") - run(N, "YEAR") - run(N, "ISOYEAR") - run(N, "QUARTER") - run(N, "MONTH") - run(N, "WEEK") - run(N, "DAY") - run(N, "DAYOFWEEK") - run(N, "DOW") - run(N, "ISODOW") - run(N, "DOY") - run(N, "HOUR") - run(N, "MINUTE") - run(N, "SECOND") - run(N, "MILLISECONDS") - run(N, "MICROSECONDS") - run(N, "EPOCH") + val fields = Seq( + "MILLENNIUM", "CENTURY", "DECADE", "YEAR", + "ISOYEAR", "QUARTER", "MONTH", "WEEK", + "DAY", "DAYOFWEEK", "DOW", "ISODOW", + "DOY", "HOUR", "MINUTE", "SECOND", + "MILLISECONDS", "MICROSECONDS", "EPOCH") + + Seq("extract", "date_part").foreach { func => + Seq("timestamp", "date").foreach { dateType => + val benchmark = new Benchmark(s"Invoke $func for $dateType", N, output = output) + + run(benchmark, N, s"cast to $dateType", castExpr(dateType)) + fields.foreach(run(benchmark, func, N, _, dateType)) + + benchmark.run() + } } } } From c56a012bc839cd2f92c2be41faea91d1acfba4eb Mon Sep 17 00:00:00 2001 From: maryannxue Date: Thu, 12 Sep 2019 21:49:21 +0800 Subject: [PATCH 0052/1387] [SPARK-29060][SQL] Add tree traversal helper for adaptive spark plans ### What changes were proposed in this pull request? This PR adds a utility class `AdaptiveSparkPlanHelper` which provides methods related to tree traversal of an `AdaptiveSparkPlanExec` plan. Unlike their counterparts in `TreeNode` or `QueryPlan`, these methods traverse down leaf nodes of adaptive plans, i.e., `AdaptiveSparkPlanExec` and `QueryStageExec`. ### Why are the changes needed? This utility class can greatly simplify tree traversal code for adaptive spark plans. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Refined `AdaptiveQueryExecSuite` with the help of the new utility methods. Closes #25764 from maryannxue/aqe-utils. Authored-by: maryannxue Signed-off-by: Wenchen Fan --- .../adaptive/AdaptiveSparkPlanHelper.scala | 130 ++++++++++++++++++ .../adaptive/AdaptiveQueryExecSuite.scala | 38 +++-- 2 files changed, 147 insertions(+), 21 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala new file mode 100644 index 0000000000000..0ec8710e4db43 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala @@ -0,0 +1,130 @@ +/* + * 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.adaptive + +import org.apache.spark.sql.execution.SparkPlan + +/** + * This class provides utility methods related to tree traversal of an [[AdaptiveSparkPlanExec]] + * plan. Unlike their counterparts in [[org.apache.spark.sql.catalyst.trees.TreeNode]] or + * [[org.apache.spark.sql.catalyst.plans.QueryPlan]], these methods traverse down leaf nodes of + * adaptive plans, i.e., [[AdaptiveSparkPlanExec]] and [[QueryStageExec]]. + */ +trait AdaptiveSparkPlanHelper { + + /** + * Find the first [[SparkPlan]] that satisfies the condition specified by `f`. + * The condition is recursively applied to this node and all of its children (pre-order). + */ + def find(p: SparkPlan)(f: SparkPlan => Boolean): Option[SparkPlan] = if (f(p)) { + Some(p) + } else { + allChildren(p).foldLeft(Option.empty[SparkPlan]) { (l, r) => l.orElse(find(r)(f)) } + } + + /** + * Runs the given function on this node and then recursively on children. + * @param f the function to be applied to each node in the tree. + */ + def foreach(p: SparkPlan)(f: SparkPlan => Unit): Unit = { + f(p) + allChildren(p).foreach(foreach(_)(f)) + } + + /** + * Runs the given function recursively on children then on this node. + * @param f the function to be applied to each node in the tree. + */ + def foreachUp(p: SparkPlan)(f: SparkPlan => Unit): Unit = { + allChildren(p).foreach(foreachUp(_)(f)) + f(p) + } + + /** + * Returns a Seq containing the result of applying the given function to each + * node in this tree in a preorder traversal. + * @param f the function to be applied. + */ + def map[A](p: SparkPlan)(f: SparkPlan => A): Seq[A] = { + val ret = new collection.mutable.ArrayBuffer[A]() + foreach(p)(ret += f(_)) + ret + } + + /** + * Returns a Seq by applying a function to all nodes in this tree and using the elements of the + * resulting collections. + */ + def flatMap[A](p: SparkPlan)(f: SparkPlan => TraversableOnce[A]): Seq[A] = { + val ret = new collection.mutable.ArrayBuffer[A]() + foreach(p)(ret ++= f(_)) + ret + } + + /** + * Returns a Seq containing the result of applying a partial function to all elements in this + * tree on which the function is defined. + */ + def collect[B](p: SparkPlan)(pf: PartialFunction[SparkPlan, B]): Seq[B] = { + val ret = new collection.mutable.ArrayBuffer[B]() + val lifted = pf.lift + foreach(p)(node => lifted(node).foreach(ret.+=)) + ret + } + + /** + * Returns a Seq containing the leaves in this tree. + */ + def collectLeaves(p: SparkPlan): Seq[SparkPlan] = { + collect(p) { case plan if allChildren(plan).isEmpty => plan } + } + + /** + * Finds and returns the first [[SparkPlan]] of the tree for which the given partial function + * is defined (pre-order), and applies the partial function to it. + */ + def collectFirst[B](p: SparkPlan)(pf: PartialFunction[SparkPlan, B]): Option[B] = { + val lifted = pf.lift + lifted(p).orElse { + allChildren(p).foldLeft(Option.empty[B]) { (l, r) => l.orElse(collectFirst(r)(pf)) } + } + } + + /** + * Returns a sequence containing the result of applying a partial function to all elements in this + * plan, also considering all the plans in its (nested) subqueries + */ + def collectInPlanAndSubqueries[B](p: SparkPlan)(f: PartialFunction[SparkPlan, B]): Seq[B] = { + (p +: subqueriesAll(p)).flatMap(collect(_)(f)) + } + + /** + * Returns a sequence containing the subqueries in this plan, also including the (nested) + * subquries in its children + */ + def subqueriesAll(p: SparkPlan): Seq[SparkPlan] = { + val subqueries = flatMap(p)(_.subqueries) + subqueries ++ subqueries.flatMap(subqueriesAll) + } + + private def allChildren(p: SparkPlan): Seq[SparkPlan] = p match { + case a: AdaptiveSparkPlanExec => Seq(a.executedPlan) + case s: QueryStageExec => Seq(s.plan) + case _ => p.children + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 243ef5438ff39..37b106c3ea530 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -20,13 +20,16 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.rule.CoalescedShuffleReaderExec -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildRight, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -class AdaptiveQueryExecSuite extends QueryTest with SharedSparkSession { +class AdaptiveQueryExecSuite + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { + import testImplicits._ setupTestData() @@ -51,34 +54,27 @@ class AdaptiveQueryExecSuite extends QueryTest with SharedSparkSession { } private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { - plan.collect { - case j: BroadcastHashJoinExec => Seq(j) - case s: QueryStageExec => findTopLevelBroadcastHashJoin(s.plan) - }.flatten + collect(plan) { + case j: BroadcastHashJoinExec => j + } } private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { - plan.collect { - case j: SortMergeJoinExec => Seq(j) - case s: QueryStageExec => findTopLevelSortMergeJoin(s.plan) - }.flatten + collect(plan) { + case j: SortMergeJoinExec => j + } } private def findReusedExchange(plan: SparkPlan): Seq[ReusedQueryStageExec] = { - plan.collect { - case e: ReusedQueryStageExec => Seq(e) - case a: AdaptiveSparkPlanExec => findReusedExchange(a.executedPlan) - case s: QueryStageExec => findReusedExchange(s.plan) - case p: SparkPlan => p.subqueries.flatMap(findReusedExchange) - }.flatten + collectInPlanAndSubqueries(plan) { + case e: ReusedQueryStageExec => e + } } private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { - plan.collect { - case e: ReusedSubqueryExec => Seq(e) - case s: QueryStageExec => findReusedSubquery(s.plan) - case p: SparkPlan => p.subqueries.flatMap(findReusedSubquery) - }.flatten + collectInPlanAndSubqueries(plan) { + case e: ReusedSubqueryExec => e + } } test("Change merge join to broadcast join") { From 5631a96367d2576e1e0f95d7ae529468da8f5fa8 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Thu, 12 Sep 2019 17:23:08 -0700 Subject: [PATCH 0053/1387] [SPARK-29048] Improve performance on Column.isInCollection() with a large size collection ### What changes were proposed in this pull request? The `Column.isInCollection()` with a large size collection will generate an expression with large size children expressions. This make analyzer and optimizer take a long time to run. In this PR, in `isInCollection()` function, directly generate `InSet` expression, avoid generating too many children expressions. ### Why are the changes needed? `Column.isInCollection()` with a large size collection sometimes become a bottleneck when running sql. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually benchmark it in spark-shell: ``` def testExplainTime(collectionSize: Int) = { val df = spark.range(10).withColumn("id2", col("id") + 1) val list = Range(0, collectionSize).toList val startTime = System.currentTimeMillis() df.where(col("id").isInCollection(list)).where(col("id2").isInCollection(list)).explain() val elapsedTime = System.currentTimeMillis() - startTime println(s"cost time: ${elapsedTime}ms") } ``` Then test on collection size 5, 10, 100, 1000, 10000, test result is: collection size | explain time (before) | explain time (after) ------ | ------ | ------ 5 | 26ms | 29ms 10 | 30ms | 48ms 100 | 104ms | 50ms 1000 | 1202ms | 58ms 10000 | 10012ms | 523ms Closes #25754 from WeichenXu123/improve_in_collection. Lead-authored-by: WeichenXu Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- .../scala/org/apache/spark/sql/Column.scala | 10 ++++- .../spark/sql/ColumnExpressionSuite.scala | 45 ++++++++++++------- 2 files changed, 37 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index b0de3c85aaef8..7b903a3f7f148 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ private[sql] object Column { @@ -808,7 +809,14 @@ class Column(val expr: Expression) extends Logging { * @group expr_ops * @since 2.4.0 */ - def isInCollection(values: scala.collection.Iterable[_]): Column = isin(values.toSeq: _*) + def isInCollection(values: scala.collection.Iterable[_]): Column = withExpr { + val hSet = values.toSet[Any] + if (hSet.size > SQLConf.get.optimizerInSetConversionThreshold) { + InSet(expr, hSet) + } else { + In(expr, values.toSeq.map(lit(_).expr)) + } + } /** * A boolean expression that is evaluated to true if the value of this expression is contained diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index a52c6d503d147..c346c8946a972 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} import org.scalatest.Matchers._ -import org.apache.spark.sql.catalyst.expressions.NamedExpression +import org.apache.spark.sql.catalyst.expressions.{In, InSet, NamedExpression} import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -454,25 +454,36 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("isInCollection: Scala Collection") { val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") - // Test with different types of collections - checkAnswer(df.filter($"a".isInCollection(Seq(3, 1))), - df.collect().toSeq.filter(r => r.getInt(0) == 3 || r.getInt(0) == 1)) - checkAnswer(df.filter($"a".isInCollection(Seq(1, 2).toSet)), - df.collect().toSeq.filter(r => r.getInt(0) == 1 || r.getInt(0) == 2)) - checkAnswer(df.filter($"a".isInCollection(Seq(3, 2).toArray)), - df.collect().toSeq.filter(r => r.getInt(0) == 3 || r.getInt(0) == 2)) - checkAnswer(df.filter($"a".isInCollection(Seq(3, 1).toList)), - df.collect().toSeq.filter(r => r.getInt(0) == 3 || r.getInt(0) == 1)) - val df2 = Seq((1, Seq(1)), (2, Seq(2)), (3, Seq(3))).toDF("a", "b") + Seq(1, 2).foreach { conf => + withSQLConf(SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> conf.toString) { + if (conf <= 1) { + assert($"a".isInCollection(Seq(3, 1)).expr.isInstanceOf[InSet], "Expect expr to be InSet") + } else { + assert($"a".isInCollection(Seq(3, 1)).expr.isInstanceOf[In], "Expect expr to be In") + } - val e = intercept[AnalysisException] { - df2.filter($"a".isInCollection(Seq($"b"))) - } - Seq("cannot resolve", "due to data type mismatch: Arguments must be same type but were") - .foreach { s => - assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) + // Test with different types of collections + checkAnswer(df.filter($"a".isInCollection(Seq(3, 1))), + df.collect().toSeq.filter(r => r.getInt(0) == 3 || r.getInt(0) == 1)) + checkAnswer(df.filter($"a".isInCollection(Seq(1, 2).toSet)), + df.collect().toSeq.filter(r => r.getInt(0) == 1 || r.getInt(0) == 2)) + checkAnswer(df.filter($"a".isInCollection(Seq(3, 2).toArray)), + df.collect().toSeq.filter(r => r.getInt(0) == 3 || r.getInt(0) == 2)) + checkAnswer(df.filter($"a".isInCollection(Seq(3, 1).toList)), + df.collect().toSeq.filter(r => r.getInt(0) == 3 || r.getInt(0) == 1)) + + val df2 = Seq((1, Seq(1)), (2, Seq(2)), (3, Seq(3))).toDF("a", "b") + + val e = intercept[AnalysisException] { + df2.filter($"a".isInCollection(Seq($"b"))) + } + Seq("cannot resolve", + "due to data type mismatch: Arguments must be same type but were").foreach { s => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) + } } + } } test("&&") { From ee630312707f5bf809fe85d971a4c45a57d6a283 Mon Sep 17 00:00:00 2001 From: Kevin Yu Date: Thu, 12 Sep 2019 23:00:42 -0700 Subject: [PATCH 0054/1387] [SPARK-28828][DOC] Document REFRESH TABLE command ### What changes were proposed in this pull request? Document REFRESH TABLE statement in the SQL Reference Guide. ### Why are the changes needed? Currently there is no documentation in the SPARK SQL to describe how to use this command, it is to address this issue. ### Does this PR introduce any user-facing change? Yes. #### Before: There is no documentation for this. #### After: Screen Shot 2019-09-12 at 11 39 21 AM ### How was this patch tested? Using jykll build --serve Closes #25549 from kevinyu98/spark-28828-refreshTable. Authored-by: Kevin Yu Signed-off-by: Xiao Li --- docs/_data/menu-sql.yaml | 2 + docs/sql-ref-syntax-aux-refresh-table.md | 58 ++++++++++++++++++++++++ 2 files changed, 60 insertions(+) create mode 100644 docs/sql-ref-syntax-aux-refresh-table.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 258762f0d9506..d03616163c964 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -172,6 +172,8 @@ url: sql-ref-syntax-aux-cache-uncache-table.html - text: CLEAR CACHE url: sql-ref-syntax-aux-cache-clear-cache.html + - text: REFRESH TABLE + url: sql-ref-syntax-aux-refresh-table.html - text: Describe Commands url: sql-ref-syntax-aux-describe.html subitems: diff --git a/docs/sql-ref-syntax-aux-refresh-table.md b/docs/sql-ref-syntax-aux-refresh-table.md new file mode 100644 index 0000000000000..262382a467073 --- /dev/null +++ b/docs/sql-ref-syntax-aux-refresh-table.md @@ -0,0 +1,58 @@ +--- +layout: global +title: REFRESH TABLE +displayTitle: REFRESH TABLE +license: | + 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. +--- + +### Description +`REFRESH TABLE` statement invalidates the cached entries, which include data +and metadata of the given table or view. The invalidated cache is populated in +lazy manner when the cached table or the query associated with it is executed again. + +### Syntax +{% highlight sql %} +REFRESH [TABLE] tableIdentifier +{% endhighlight %} + +### Parameters +

+
tableIdentifier
+
+ Specifies a table name, which is either a qualified or unqualified name that designates a table/view. If no database identifier is provided, it refers to a temporary view or a table/view in the current database.

+ Syntax: + + [database_name.]table_name + +
+
+ +### Examples +{% highlight sql %} +-- The cached entries of the table will be refreshed +-- The table is resolved from the current database as the table name is unqualified. +REFRESH TABLE tbl1; + +-- The cached entries of the view will be refreshed or invalidated +-- The view is resolved from tempDB database, as the view name is qualified. +REFRESH TABLE tempDB.view1; +{% endhighlight %} + +### Related Statements +- [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) +- [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) +- [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) \ No newline at end of file From b83304fb01aaceabd8e296f4b6de2b86235e1c9b Mon Sep 17 00:00:00 2001 From: sandeep katta Date: Thu, 12 Sep 2019 23:10:50 -0700 Subject: [PATCH 0055/1387] [SPARK-28796][DOC] Document DROP DATABASE statement in SQL Reference ### What changes were proposed in this pull request? Document DROP DATABASE statement in SQL Reference ### Why are the changes needed? Currently from spark there is no complete sql guide is present, so it is better to document all the sql commands, this jira is sub part of this task. ### Does this PR introduce any user-facing change? Yes, Before there was no documentation about drop database syntax After Fix ![image](https://user-images.githubusercontent.com/35216143/64787097-977a7200-d58d-11e9-911c-d2ff6f3ccff5.png) ![image](https://user-images.githubusercontent.com/35216143/64787122-a6612480-d58d-11e9-978c-9455baff007f.png) ### How was this patch tested? tested with jenkyll build Closes #25554 from sandeep-katta/dropDbDoc. Authored-by: sandeep katta Signed-off-by: Xiao Li --- docs/sql-ref-syntax-ddl-drop-database.md | 60 +++++++++++++++++++++++- 1 file changed, 59 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-drop-database.md b/docs/sql-ref-syntax-ddl-drop-database.md index cd900a7e393db..f3cdbf91a8d2a 100644 --- a/docs/sql-ref-syntax-ddl-drop-database.md +++ b/docs/sql-ref-syntax-ddl-drop-database.md @@ -19,4 +19,62 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +Drop a database and delete the directory associated with the database from the file system. An +exception will be thrown if the database does not exist in the system. + +### Syntax + +{% highlight sql %} +DROP (DATABASE|SCHEMA) [IF EXISTS] dbname [RESTRICT|CASCADE]; +{% endhighlight %} + + +### Parameters + +
+
DATABASE|SCHEMA
+
`DATABASE` and `SCHEMA` mean the same thing, either of them can be used.
+
+ +
+
IF EXISTS
+
If specified, no exception is thrown when the database does not exist.
+
+ +
+
RESTRICT
+
If specified, will restrict dropping a non-empty database and is enabled by default.
+
+ +
+
CASCADE
+
If specified, will drop all the associated tables and functions.
+
+ +### Example +{% highlight sql %} +-- Create `inventory_db` Database +CREATE DATABASE inventory_db COMMENT 'This database is used to maintain Inventory'; + +-- Drop the database and it's tables +DROP DATABASE inventory_db CASCADE; ++---------+ +| Result | ++---------+ ++---------+ + +-- Drop the database using IF EXISTS +DROP DATABASE IF EXISTS inventory_db CASCADE; ++---------+ +| Result | ++---------+ ++---------+ + +{% endhighlight %} + +### Related statements +- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) +- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) +- [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) \ No newline at end of file From d59980783e34bbcae0f362a8c0c8ccdfd682197e Mon Sep 17 00:00:00 2001 From: aman_omer Date: Thu, 12 Sep 2019 23:44:23 -0700 Subject: [PATCH 0056/1387] [SPARK-28795][DOC][SQL] Document CREATE VIEW statement in SQL Reference ### What changes were proposed in this pull request? Added document for CREATE VIEW command. ### Why are the changes needed? As a reference to syntax and examples of CREATE VIEW command. ### How was this patch tested? Documentation update. Verified manually. Closes #25543 from amanomer/spark-28795. Lead-authored-by: aman_omer Co-authored-by: Xiao Li Co-authored-by: Aman Omer Signed-off-by: Xiao Li --- docs/sql-ref-syntax-ddl-create-view.md | 62 +++++++++++++++++++++++++- 1 file changed, 61 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md index eff7df91f59c5..c7ca28ea5b62f 100644 --- a/docs/sql-ref-syntax-ddl-create-view.md +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -19,4 +19,64 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +Views are based on the result-set of an `SQL` query. `CREATE VIEW` constructs +a virtual table that has no physical data therefore other operations like +`ALTER VIEW` and `DROP VIEW` only change metadata. + +### Syntax +{% highlight sql %} +CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name + create_view_clauses + AS query; +{% endhighlight %} + +### Parameters +
+
OR REPLACE
+
If a view of same name already exists, it will be replaced.
+
+
+
[GLOBAL] TEMPORARY
+
TEMPORARY views are session-scoped and will be dropped when session ends + because it skips persisting the definition in the underlying metastore, if any. + GLOBAL TEMPORARY views are tied to a system preserved temporary database `global_temp`.
+
+
+
IF NOT EXISTS
+
Creates a view if it does not exists.
+
+
+
create_view_clauses
+
These clauses are optional and order insensitive. It can be of following formats. +
    +
  • [(column_name [COMMENT column_comment], ...) ] to specify column-level comments.
  • +
  • [COMMENT view_comment] to specify view-level comments.
  • +
  • [TBLPROPERTIES (property_name = property_value, ...)] to add metadata key-value pairs.
  • +
+
+
+
+
query
+
A SELECT statement that constructs the view from base tables or other views.
+
+ +### Examples +{% highlight sql %} +-- Create or replace view for `experienced_employee` with comments. +CREATE OR REPLACE VIEW experienced_employee + (ID COMMENT 'Unique identification number', Name) + COMMENT 'View for experienced employees' + AS SELECT id, name FROM all_employee + WHERE working_years > 5; + +-- Create a global temporary view `subscribed_movies` if it does not exist. +CREATE GLOBAL TEMPORARY VIEW IF NOT EXISTS subscribed_movies + AS SELECT mo.member_id, mb.full_name, mo.movie_title + FROM movies AS mo INNER JOIN members AS mb + ON mo.member_id = mb.id; +{% endhighlight %} + +### Related Statements +- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.md) +- [DROP VIEW](sql-ref-syntax-ddl-drop-view.md) From fcf9b41b491a7ed990f3563d3c1446d431e38262 Mon Sep 17 00:00:00 2001 From: Juliusz Sompolski Date: Fri, 13 Sep 2019 09:13:57 -0700 Subject: [PATCH 0057/1387] [SPARK-29056] ThriftServerSessionPage displays 1970/01/01 finish and close time when unset ### What changes were proposed in this pull request? ThriftServerSessionPage displays timestamp 0 (1970/01/01) instead of nothing if query finish time and close time are not set. ![image](https://user-images.githubusercontent.com/25019163/64711118-6d578000-d4b9-11e9-9b11-2e3616319a98.png) Change it to display nothing, like ThriftServerPage. ### Why are the changes needed? Obvious bug. ### Does this PR introduce any user-facing change? Finish time and Close time will be displayed correctly on ThriftServerSessionPage in JDBC/ODBC Spark UI. ### How was this patch tested? Manual test. Closes #25762 from juliuszsompolski/SPARK-29056. Authored-by: Juliusz Sompolski Signed-off-by: Yuming Wang --- .../sql/hive/thriftserver/ui/ThriftServerSessionPage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 81df1304085e8..912b658eb36b9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -98,8 +98,8 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) {info.groupId} {formatDate(info.startTimestamp)} - {formatDate(info.finishTimestamp)} - {formatDate(info.closeTimestamp)} + {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} + {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} From 77e9b58d4f6dd06b1d2598845b1ba9657424bf9f Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 13 Sep 2019 12:29:19 -0500 Subject: [PATCH 0058/1387] [SPARK-28969][PYTHON][ML] OneVsRestParams parity between scala and python ### What changes were proposed in this pull request? Follow the scala ```OneVsRestParams``` implementation, move ```setClassifier``` from ```OneVsRestParams``` to ```OneVsRest``` in Pyspark ### Why are the changes needed? 1. Maintain the parity between scala and python code. 2. ```Classifier``` can only be set in the estimator. ### Does this PR introduce any user-facing change? Yes. Previous behavior: ```OneVsRestModel``` has method ```setClassifier``` Current behavior: ```setClassifier``` is removed from ```OneVsRestModel```. ```classifier``` can only be set in ```OneVsRest```. ### How was this patch tested? Use existing tests Closes #25715 from huaxingao/spark-28969. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- python/pyspark/ml/classification.py | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 3ae2338aa77e6..b6c28d05ea273 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1872,15 +1872,6 @@ class OneVsRestParams(HasFeaturesCol, HasLabelCol, HasWeightCol, HasPredictionCo classifier = Param(Params._dummy(), "classifier", "base binary classifier") - @since("2.0.0") - def setClassifier(self, value): - """ - Sets the value of :py:attr:`classifier`. - - .. note:: Only LogisticRegression and NaiveBayes are supported now. - """ - return self._set(classifier=value) - @since("2.0.0") def getClassifier(self): """ @@ -1959,6 +1950,13 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre kwargs = self._input_kwargs return self._set(**kwargs) + @since("2.0.0") + def setClassifier(self, value): + """ + Sets the value of :py:attr:`classifier`. + """ + return self._set(classifier=value) + def _fit(self, dataset): labelCol = self.getLabelCol() featuresCol = self.getFeaturesCol() @@ -2212,7 +2210,8 @@ def _from_java(cls, java_stage): classifier = JavaParams._from_java(java_stage.getClassifier()) models = [JavaParams._from_java(model) for model in java_stage.models()] py_stage = cls(models=models).setPredictionCol(predictionCol).setLabelCol(labelCol)\ - .setFeaturesCol(featuresCol).setClassifier(classifier) + .setFeaturesCol(featuresCol) + py_stage._set(classifier=classifier) py_stage._resetUid(java_stage.uid()) return py_stage From 1675d5114e57b770265dccb2bae7c8a0586fbf29 Mon Sep 17 00:00:00 2001 From: Lee Dongjin Date: Fri, 13 Sep 2019 12:31:28 -0500 Subject: [PATCH 0059/1387] [SPARK-23539][SS] Add support for Kafka headers in Structured Streaming ## What changes were proposed in this pull request? This update adds support for Kafka Headers functionality in Structured Streaming. ## How was this patch tested? With following unit tests: - KafkaRelationSuite: "default starting and ending offsets with headers" (new) - KafkaSinkSuite: "batch - write to kafka" (updated) Closes #22282 from dongjinleekr/feature/SPARK-23539. Lead-authored-by: Lee Dongjin Co-authored-by: Jungtaek Lim Signed-off-by: Sean Owen --- .../structured-streaming-kafka-integration.md | 55 +++++++++++ .../spark/sql/kafka010/KafkaBatch.scala | 5 +- .../kafka010/KafkaBatchPartitionReader.scala | 13 ++- .../sql/kafka010/KafkaContinuousStream.scala | 30 +++--- .../sql/kafka010/KafkaMicroBatchStream.scala | 9 +- .../sql/kafka010/KafkaOffsetReader.scala | 14 --- .../kafka010/KafkaRecordToRowConverter.scala | 93 +++++++++++++++++++ .../KafkaRecordToUnsafeRowConverter.scala | 54 ----------- .../spark/sql/kafka010/KafkaRelation.scala | 24 +++-- .../spark/sql/kafka010/KafkaSource.scala | 41 ++++---- .../sql/kafka010/KafkaSourceProvider.scala | 22 +++-- .../spark/sql/kafka010/KafkaWriteTask.scala | 39 +++++++- .../spark/sql/kafka010/KafkaWriter.scala | 15 ++- .../sql/kafka010/KafkaDataConsumerSuite.scala | 28 ++++-- .../sql/kafka010/KafkaRelationSuite.scala | 33 ++++++- .../spark/sql/kafka010/KafkaSinkSuite.scala | 48 +++++++++- .../spark/sql/kafka010/KafkaTestUtils.scala | 35 +++++-- 17 files changed, 402 insertions(+), 156 deletions(-) create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToRowConverter.scala delete mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToUnsafeRowConverter.scala diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 8cb2a45c56b21..ef4cdc2608f45 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -27,6 +27,8 @@ For Scala/Java applications using SBT/Maven project definitions, link your appli artifactId = spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION_SHORT}} +Please note that to use the headers functionality, your Kafka client version should be version 0.11.0.0 or up. + For Python applications, you need to add this above library and its dependencies when deploying your application. See the [Deploying](#deploying) subsection below. @@ -50,6 +52,17 @@ val df = spark df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .as[(String, String)] +// Subscribe to 1 topic, with headers +val df = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .option("includeHeaders", "true") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)", "headers") + .as[(String, String, Map)] + // Subscribe to multiple topics val df = spark .readStream @@ -84,6 +97,16 @@ Dataset df = spark .load(); df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); +// Subscribe to 1 topic, with headers +Dataset df = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .option("includeHeaders", "true") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)", "headers"); + // Subscribe to multiple topics Dataset df = spark .readStream() @@ -116,6 +139,16 @@ df = spark \ .load() df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +# Subscribe to 1 topic, with headers +val df = spark \ + .readStream \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribe", "topic1") \ + .option("includeHeaders", "true") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)", "headers") + # Subscribe to multiple topics df = spark \ .readStream \ @@ -286,6 +319,10 @@ Each row in the source has the following schema: timestampType int + + headers (optional) + array + The following options must be set for the Kafka source @@ -425,6 +462,13 @@ The following configurations are optional: issues, set the Kafka consumer session timeout (by setting option "kafka.session.timeout.ms") to be very small. When this is set, option "groupIdPrefix" will be ignored. + + includeHeaders + boolean + false + streaming and batch + Whether to include the Kafka headers in the row. + ### Consumer Caching @@ -522,6 +566,10 @@ The Dataframe being written to Kafka should have the following columns in schema value (required) string or binary + + headers (optional) + array + topic (*optional) string @@ -559,6 +607,13 @@ The following configurations are optional: Sets the topic that all rows will be written to in Kafka. This option overrides any topic column that may exist in the data. + + includeHeaders + boolean + false + streaming and batch + Whether to include the Kafka headers in the row. + ### Creating a Kafka Sink for Streaming Queries diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala index b958035b39222..667c383681917 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala @@ -31,7 +31,8 @@ private[kafka010] class KafkaBatch( specifiedKafkaParams: Map[String, String], failOnDataLoss: Boolean, startingOffsets: KafkaOffsetRangeLimit, - endingOffsets: KafkaOffsetRangeLimit) + endingOffsets: KafkaOffsetRangeLimit, + includeHeaders: Boolean) extends Batch with Logging { assert(startingOffsets != LatestOffsetRangeLimit, "Starting offset not allowed to be set to latest offsets.") @@ -90,7 +91,7 @@ private[kafka010] class KafkaBatch( KafkaSourceProvider.kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId) offsetRanges.map { range => new KafkaBatchInputPartition( - range, executorKafkaParams, pollTimeoutMs, failOnDataLoss) + range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, includeHeaders) }.toArray } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala index d4aa6774be66c..645b68b0c407a 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala @@ -29,13 +29,14 @@ private[kafka010] case class KafkaBatchInputPartition( offsetRange: KafkaOffsetRange, executorKafkaParams: ju.Map[String, Object], pollTimeoutMs: Long, - failOnDataLoss: Boolean) extends InputPartition + failOnDataLoss: Boolean, + includeHeaders: Boolean) extends InputPartition private[kafka010] object KafkaBatchReaderFactory extends PartitionReaderFactory { override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { val p = partition.asInstanceOf[KafkaBatchInputPartition] KafkaBatchPartitionReader(p.offsetRange, p.executorKafkaParams, p.pollTimeoutMs, - p.failOnDataLoss) + p.failOnDataLoss, p.includeHeaders) } } @@ -44,12 +45,14 @@ private case class KafkaBatchPartitionReader( offsetRange: KafkaOffsetRange, executorKafkaParams: ju.Map[String, Object], pollTimeoutMs: Long, - failOnDataLoss: Boolean) extends PartitionReader[InternalRow] with Logging { + failOnDataLoss: Boolean, + includeHeaders: Boolean) extends PartitionReader[InternalRow] with Logging { private val consumer = KafkaDataConsumer.acquire(offsetRange.topicPartition, executorKafkaParams) private val rangeToRead = resolveRange(offsetRange) - private val converter = new KafkaRecordToUnsafeRowConverter + private val unsafeRowProjector = new KafkaRecordToRowConverter() + .toUnsafeRowProjector(includeHeaders) private var nextOffset = rangeToRead.fromOffset private var nextRow: UnsafeRow = _ @@ -58,7 +61,7 @@ private case class KafkaBatchPartitionReader( if (nextOffset < rangeToRead.untilOffset) { val record = consumer.get(nextOffset, rangeToRead.untilOffset, pollTimeoutMs, failOnDataLoss) if (record != null) { - nextRow = converter.toUnsafeRow(record) + nextRow = unsafeRowProjector(record) nextOffset = record.offset + 1 true } else { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala index 1e8da4bc0f7cc..9e7b7d6db2038 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReader, ContinuousPartitionReaderFactory, ContinuousStream, Offset, PartitionOffset} -import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} +import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.sql.util.CaseInsensitiveStringMap /** @@ -56,6 +56,7 @@ class KafkaContinuousStream( private[kafka010] val pollTimeoutMs = options.getLong(KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, 512) + private val includeHeaders = options.getBoolean(INCLUDE_HEADERS, false) // Initialized when creating reader factories. If this diverges from the partitions at the latest // offsets, we need to reconfigure. @@ -88,7 +89,7 @@ class KafkaContinuousStream( if (deletedPartitions.nonEmpty) { val message = if ( offsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { - s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + s"$deletedPartitions are gone. ${CUSTOM_GROUP_ID_ERROR_MESSAGE}" } else { s"$deletedPartitions are gone. Some data may have been missed." } @@ -102,7 +103,7 @@ class KafkaContinuousStream( startOffsets.toSeq.map { case (topicPartition, start) => KafkaContinuousInputPartition( - topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss) + topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss, includeHeaders) }.toArray } @@ -153,19 +154,22 @@ class KafkaContinuousStream( * @param pollTimeoutMs The timeout for Kafka consumer polling. * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets * are skipped. + * @param includeHeaders Flag indicating whether to include Kafka records' headers. */ case class KafkaContinuousInputPartition( - topicPartition: TopicPartition, - startOffset: Long, - kafkaParams: ju.Map[String, Object], - pollTimeoutMs: Long, - failOnDataLoss: Boolean) extends InputPartition + topicPartition: TopicPartition, + startOffset: Long, + kafkaParams: ju.Map[String, Object], + pollTimeoutMs: Long, + failOnDataLoss: Boolean, + includeHeaders: Boolean) extends InputPartition object KafkaContinuousReaderFactory extends ContinuousPartitionReaderFactory { override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = { val p = partition.asInstanceOf[KafkaContinuousInputPartition] new KafkaContinuousPartitionReader( - p.topicPartition, p.startOffset, p.kafkaParams, p.pollTimeoutMs, p.failOnDataLoss) + p.topicPartition, p.startOffset, p.kafkaParams, p.pollTimeoutMs, + p.failOnDataLoss, p.includeHeaders) } } @@ -184,9 +188,11 @@ class KafkaContinuousPartitionReader( startOffset: Long, kafkaParams: ju.Map[String, Object], pollTimeoutMs: Long, - failOnDataLoss: Boolean) extends ContinuousPartitionReader[InternalRow] { + failOnDataLoss: Boolean, + includeHeaders: Boolean) extends ContinuousPartitionReader[InternalRow] { private val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams) - private val converter = new KafkaRecordToUnsafeRowConverter + private val unsafeRowProjector = new KafkaRecordToRowConverter() + .toUnsafeRowProjector(includeHeaders) private var nextKafkaOffset = startOffset private var currentRecord: ConsumerRecord[Array[Byte], Array[Byte]] = _ @@ -225,7 +231,7 @@ class KafkaContinuousPartitionReader( } override def get(): UnsafeRow = { - converter.toUnsafeRow(currentRecord) + unsafeRowProjector(currentRecord) } override def getOffset(): KafkaSourcePartitionOffset = { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index 52d91abc86218..6ea6efe5d1b74 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchStream -import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} +import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.UninterruptibleThread @@ -64,6 +64,8 @@ private[kafka010] class KafkaMicroBatchStream( private[kafka010] val maxOffsetsPerTrigger = Option(options.get( KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER)).map(_.toLong) + private val includeHeaders = options.getBoolean(INCLUDE_HEADERS, false) + private val rangeCalculator = KafkaOffsetRangeCalculator(options) private var endPartitionOffsets: KafkaSourceOffset = _ @@ -112,7 +114,7 @@ private[kafka010] class KafkaMicroBatchStream( if (deletedPartitions.nonEmpty) { val message = if (kafkaOffsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { - s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + s"$deletedPartitions are gone. ${CUSTOM_GROUP_ID_ERROR_MESSAGE}" } else { s"$deletedPartitions are gone. Some data may have been missed." } @@ -146,7 +148,8 @@ private[kafka010] class KafkaMicroBatchStream( // Generate factories based on the offset ranges offsetRanges.map { range => - KafkaBatchInputPartition(range, executorKafkaParams, pollTimeoutMs, failOnDataLoss) + KafkaBatchInputPartition(range, executorKafkaParams, pollTimeoutMs, + failOnDataLoss, includeHeaders) }.toArray } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index f3effd5300a79..20f2ce11d4afb 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -31,7 +31,6 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.types._ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} /** @@ -421,16 +420,3 @@ private[kafka010] class KafkaOffsetReader( _consumer = null // will automatically get reinitialized again } } - -private[kafka010] object KafkaOffsetReader { - - def kafkaSchema: StructType = StructType(Seq( - StructField("key", BinaryType), - StructField("value", BinaryType), - StructField("topic", StringType), - StructField("partition", IntegerType), - StructField("offset", LongType), - StructField("timestamp", TimestampType), - StructField("timestampType", IntegerType) - )) -} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToRowConverter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToRowConverter.scala new file mode 100644 index 0000000000000..aed099c142bc3 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToRowConverter.scala @@ -0,0 +1,93 @@ +/* + * 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.kafka010 + +import java.sql.Timestamp + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer.ConsumerRecord + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** A simple class for converting Kafka ConsumerRecord to InternalRow/UnsafeRow */ +private[kafka010] class KafkaRecordToRowConverter { + import KafkaRecordToRowConverter._ + + private val toUnsafeRowWithoutHeaders = UnsafeProjection.create(schemaWithoutHeaders) + private val toUnsafeRowWithHeaders = UnsafeProjection.create(schemaWithHeaders) + + val toInternalRowWithoutHeaders: Record => InternalRow = + (cr: Record) => InternalRow( + cr.key, cr.value, UTF8String.fromString(cr.topic), cr.partition, cr.offset, + DateTimeUtils.fromJavaTimestamp(new Timestamp(cr.timestamp)), cr.timestampType.id + ) + + val toInternalRowWithHeaders: Record => InternalRow = + (cr: Record) => InternalRow( + cr.key, cr.value, UTF8String.fromString(cr.topic), cr.partition, cr.offset, + DateTimeUtils.fromJavaTimestamp(new Timestamp(cr.timestamp)), cr.timestampType.id, + if (cr.headers.iterator().hasNext) { + new GenericArrayData(cr.headers.iterator().asScala + .map(header => + InternalRow(UTF8String.fromString(header.key()), header.value()) + ).toArray) + } else { + null + } + ) + + def toUnsafeRowWithoutHeadersProjector: Record => UnsafeRow = + (cr: Record) => toUnsafeRowWithoutHeaders(toInternalRowWithoutHeaders(cr)) + + def toUnsafeRowWithHeadersProjector: Record => UnsafeRow = + (cr: Record) => toUnsafeRowWithHeaders(toInternalRowWithHeaders(cr)) + + def toUnsafeRowProjector(includeHeaders: Boolean): Record => UnsafeRow = { + if (includeHeaders) toUnsafeRowWithHeadersProjector else toUnsafeRowWithoutHeadersProjector + } +} + +private[kafka010] object KafkaRecordToRowConverter { + type Record = ConsumerRecord[Array[Byte], Array[Byte]] + + val headersType = ArrayType(StructType(Array( + StructField("key", StringType), + StructField("value", BinaryType)))) + + private val schemaWithoutHeaders = new StructType(Array( + StructField("key", BinaryType), + StructField("value", BinaryType), + StructField("topic", StringType), + StructField("partition", IntegerType), + StructField("offset", LongType), + StructField("timestamp", TimestampType), + StructField("timestampType", IntegerType) + )) + + private val schemaWithHeaders = + new StructType(schemaWithoutHeaders.fields :+ StructField("headers", headersType)) + + def kafkaSchema(includeHeaders: Boolean): StructType = { + if (includeHeaders) schemaWithHeaders else schemaWithoutHeaders + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToUnsafeRowConverter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToUnsafeRowConverter.scala deleted file mode 100644 index 306ef10b775a9..0000000000000 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToUnsafeRowConverter.scala +++ /dev/null @@ -1,54 +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.sql.kafka010 - -import org.apache.kafka.clients.consumer.ConsumerRecord - -import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.unsafe.types.UTF8String - -/** A simple class for converting Kafka ConsumerRecord to UnsafeRow */ -private[kafka010] class KafkaRecordToUnsafeRowConverter { - private val rowWriter = new UnsafeRowWriter(7) - - def toUnsafeRow(record: ConsumerRecord[Array[Byte], Array[Byte]]): UnsafeRow = { - rowWriter.reset() - rowWriter.zeroOutNullBytes() - - if (record.key == null) { - rowWriter.setNullAt(0) - } else { - rowWriter.write(0, record.key) - } - if (record.value == null) { - rowWriter.setNullAt(1) - } else { - rowWriter.write(1, record.value) - } - rowWriter.write(2, UTF8String.fromString(record.topic)) - rowWriter.write(3, record.partition) - rowWriter.write(4, record.offset) - rowWriter.write( - 5, - DateTimeUtils.fromJavaTimestamp(new java.sql.Timestamp(record.timestamp))) - rowWriter.write(6, record.timestampType.id) - rowWriter.getRow() - } -} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index dc7087821b10c..886f6b0fe0a5e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -24,10 +24,9 @@ import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types.StructType -import org.apache.spark.unsafe.types.UTF8String private[kafka010] class KafkaRelation( @@ -36,6 +35,7 @@ private[kafka010] class KafkaRelation( sourceOptions: CaseInsensitiveMap[String], specifiedKafkaParams: Map[String, String], failOnDataLoss: Boolean, + includeHeaders: Boolean, startingOffsets: KafkaOffsetRangeLimit, endingOffsets: KafkaOffsetRangeLimit) extends BaseRelation with TableScan with Logging { @@ -49,7 +49,9 @@ private[kafka010] class KafkaRelation( (sqlContext.sparkContext.conf.get(NETWORK_TIMEOUT) * 1000L).toString ).toLong - override def schema: StructType = KafkaOffsetReader.kafkaSchema + private val converter = new KafkaRecordToRowConverter() + + override def schema: StructType = KafkaRecordToRowConverter.kafkaSchema(includeHeaders) override def buildScan(): RDD[Row] = { // Each running query should use its own group id. Otherwise, the query may be only assigned @@ -100,18 +102,14 @@ private[kafka010] class KafkaRelation( // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. val executorKafkaParams = KafkaSourceProvider.kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId) + val toInternalRow = if (includeHeaders) { + converter.toInternalRowWithHeaders + } else { + converter.toInternalRowWithoutHeaders + } val rdd = new KafkaSourceRDD( sqlContext.sparkContext, executorKafkaParams, offsetRanges, - pollTimeoutMs, failOnDataLoss).map { cr => - InternalRow( - cr.key, - cr.value, - UTF8String.fromString(cr.topic), - cr.partition, - cr.offset, - DateTimeUtils.fromJavaTimestamp(new java.sql.Timestamp(cr.timestamp)), - cr.timestampType.id) - } + pollTimeoutMs, failOnDataLoss).map(toInternalRow) sqlContext.internalCreateDataFrame(rdd.setName("kafka"), schema).rdd } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index d1a35ec53bc94..29944dc3fbf1e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -31,12 +31,11 @@ import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.kafka010.KafkaSource._ -import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} +import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String /** * A [[Source]] that reads data from Kafka using the following design. @@ -84,13 +83,15 @@ private[kafka010] class KafkaSource( private val sc = sqlContext.sparkContext - private val pollTimeoutMs = sourceOptions.getOrElse( - KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, - (sc.conf.get(NETWORK_TIMEOUT) * 1000L).toString - ).toLong + private val pollTimeoutMs = + sourceOptions.getOrElse(CONSUMER_POLL_TIMEOUT, (sc.conf.get(NETWORK_TIMEOUT) * 1000L).toString) + .toLong private val maxOffsetsPerTrigger = - sourceOptions.get(KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER).map(_.toLong) + sourceOptions.get(MAX_OFFSET_PER_TRIGGER).map(_.toLong) + + private val includeHeaders = + sourceOptions.getOrElse(INCLUDE_HEADERS, "false").toBoolean /** * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only @@ -113,7 +114,9 @@ private[kafka010] class KafkaSource( private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None - override def schema: StructType = KafkaOffsetReader.kafkaSchema + private val converter = new KafkaRecordToRowConverter() + + override def schema: StructType = KafkaRecordToRowConverter.kafkaSchema(includeHeaders) /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { @@ -223,7 +226,7 @@ private[kafka010] class KafkaSource( val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) if (deletedPartitions.nonEmpty) { val message = if (kafkaReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { - s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + s"$deletedPartitions are gone. ${CUSTOM_GROUP_ID_ERROR_MESSAGE}" } else { s"$deletedPartitions are gone. Some data may have been missed." } @@ -267,16 +270,14 @@ private[kafka010] class KafkaSource( }.toArray // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. - val rdd = new KafkaSourceRDD( - sc, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss).map { cr => - InternalRow( - cr.key, - cr.value, - UTF8String.fromString(cr.topic), - cr.partition, - cr.offset, - DateTimeUtils.fromJavaTimestamp(new java.sql.Timestamp(cr.timestamp)), - cr.timestampType.id) + val rdd = if (includeHeaders) { + new KafkaSourceRDD( + sc, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss) + .map(converter.toInternalRowWithHeaders) + } else { + new KafkaSourceRDD( + sc, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss) + .map(converter.toInternalRowWithoutHeaders) } logInfo("GetBatch generating RDD of offset range: " + diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 53a6919e2d4bc..a7f8db35d7cf9 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -69,7 +69,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val caseInsensitiveParameters = CaseInsensitiveMap(parameters) validateStreamOptions(caseInsensitiveParameters) require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") - (shortName(), KafkaOffsetReader.kafkaSchema) + val includeHeaders = caseInsensitiveParameters.getOrElse(INCLUDE_HEADERS, "false").toBoolean + (shortName(), KafkaRecordToRowConverter.kafkaSchema(includeHeaders)) } override def createSource( @@ -107,7 +108,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } override def getTable(options: CaseInsensitiveStringMap): KafkaTable = { - new KafkaTable + val includeHeaders = options.getBoolean(INCLUDE_HEADERS, false) + new KafkaTable(includeHeaders) } /** @@ -131,12 +133,15 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister caseInsensitiveParameters, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) assert(endingRelationOffsets != EarliestOffsetRangeLimit) + val includeHeaders = caseInsensitiveParameters.getOrElse(INCLUDE_HEADERS, "false").toBoolean + new KafkaRelation( sqlContext, strategy(caseInsensitiveParameters), sourceOptions = caseInsensitiveParameters, specifiedKafkaParams = specifiedKafkaParams, failOnDataLoss = failOnDataLoss(caseInsensitiveParameters), + includeHeaders = includeHeaders, startingOffsets = startingRelationOffsets, endingOffsets = endingRelationOffsets) } @@ -359,11 +364,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } } - class KafkaTable extends Table with SupportsRead with SupportsWrite { + class KafkaTable(includeHeaders: Boolean) extends Table with SupportsRead with SupportsWrite { override def name(): String = "KafkaTable" - override def schema(): StructType = KafkaOffsetReader.kafkaSchema + override def schema(): StructType = KafkaRecordToRowConverter.kafkaSchema(includeHeaders) override def capabilities(): ju.Set[TableCapability] = { import TableCapability._ @@ -403,8 +408,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } class KafkaScan(options: CaseInsensitiveStringMap) extends Scan { + val includeHeaders = options.getBoolean(INCLUDE_HEADERS, false) - override def readSchema(): StructType = KafkaOffsetReader.kafkaSchema + override def readSchema(): StructType = { + KafkaRecordToRowConverter.kafkaSchema(includeHeaders) + } override def toBatch(): Batch = { val caseInsensitiveOptions = CaseInsensitiveMap(options.asScala.toMap) @@ -423,7 +431,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister specifiedKafkaParams, failOnDataLoss(caseInsensitiveOptions), startingRelationOffsets, - endingRelationOffsets) + endingRelationOffsets, + includeHeaders) } override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = { @@ -498,6 +507,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { private[kafka010] val FETCH_OFFSET_RETRY_INTERVAL_MS = "fetchoffset.retryintervalms" private[kafka010] val CONSUMER_POLL_TIMEOUT = "kafkaconsumer.polltimeoutms" private val GROUP_ID_PREFIX = "groupidprefix" + private[kafka010] val INCLUDE_HEADERS = "includeheaders" val TOPIC_OPTION_KEY = "topic" diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index 041fac7717635..b423ddc959c1b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -19,9 +19,13 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} +import scala.collection.JavaConverters._ + import org.apache.kafka.clients.producer.{Callback, KafkaProducer, ProducerRecord, RecordMetadata} +import org.apache.kafka.common.header.Header +import org.apache.kafka.common.header.internals.RecordHeader -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} import org.apache.spark.sql.types.{BinaryType, StringType} @@ -88,7 +92,17 @@ private[kafka010] abstract class KafkaRowWriter( throw new NullPointerException(s"null topic present in the data. Use the " + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a default topic.") } - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, key, value) + val record = if (projectedRow.isNullAt(3)) { + new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, null, key, value) + } else { + val headerArray = projectedRow.getArray(3) + val headers = (0 until headerArray.numElements()).map { i => + val struct = headerArray.getStruct(i, 2) + new RecordHeader(struct.getUTF8String(0).toString, struct.getBinary(1)) + .asInstanceOf[Header] + } + new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, null, key, value, headers.asJava) + } producer.send(record, callback) } @@ -131,9 +145,26 @@ private[kafka010] abstract class KafkaRowWriter( throw new IllegalStateException(s"${KafkaWriter.VALUE_ATTRIBUTE_NAME} " + s"attribute unsupported type ${t.catalogString}") } + val headersExpression = inputSchema + .find(_.name == KafkaWriter.HEADERS_ATTRIBUTE_NAME).getOrElse( + Literal(CatalystTypeConverters.convertToCatalyst(null), + KafkaRecordToRowConverter.headersType) + ) + headersExpression.dataType match { + case KafkaRecordToRowConverter.headersType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.HEADERS_ATTRIBUTE_NAME} " + + s"attribute unsupported type ${t.catalogString}") + } UnsafeProjection.create( - Seq(topicExpression, Cast(keyExpression, BinaryType), - Cast(valueExpression, BinaryType)), inputSchema) + Seq( + topicExpression, + Cast(keyExpression, BinaryType), + Cast(valueExpression, BinaryType), + headersExpression + ), + inputSchema + ) } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index e1a9191cc5a84..bbb060356f730 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -21,9 +21,10 @@ import java.{util => ju} import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} -import org.apache.spark.sql.types.{BinaryType, StringType} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.types.{BinaryType, MapType, StringType} import org.apache.spark.util.Utils /** @@ -39,6 +40,7 @@ private[kafka010] object KafkaWriter extends Logging { val TOPIC_ATTRIBUTE_NAME: String = "topic" val KEY_ATTRIBUTE_NAME: String = "key" val VALUE_ATTRIBUTE_NAME: String = "value" + val HEADERS_ATTRIBUTE_NAME: String = "headers" override def toString: String = "KafkaWriter" @@ -75,6 +77,15 @@ private[kafka010] object KafkaWriter extends Logging { throw new AnalysisException(s"$VALUE_ATTRIBUTE_NAME attribute type " + s"must be a ${StringType.catalogString} or ${BinaryType.catalogString}") } + schema.find(_.name == HEADERS_ATTRIBUTE_NAME).getOrElse( + Literal(CatalystTypeConverters.convertToCatalyst(null), + KafkaRecordToRowConverter.headersType) + ).dataType match { + case KafkaRecordToRowConverter.headersType => // good + case _ => + throw new AnalysisException(s"$HEADERS_ATTRIBUTE_NAME attribute type " + + s"must be a ${KafkaRecordToRowConverter.headersType.catalogString}") + } } def write( diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala index 80f9a1b410d2c..d97f627fbac08 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.kafka010 +import java.nio.charset.StandardCharsets import java.util.concurrent.{Executors, TimeUnit} import scala.collection.JavaConverters._ @@ -91,7 +92,7 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester test("new KafkaDataConsumer instance in case of Task retry") { try { val kafkaParams = getKafkaParams() - val key = new CacheKey(groupId, topicPartition) + val key = CacheKey(groupId, topicPartition) val context1 = new TaskContextImpl(0, 0, 0, 0, 0, null, null, null) TaskContext.setTaskContext(context1) @@ -137,7 +138,8 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester } test("SPARK-23623: concurrent use of KafkaDataConsumer") { - val data: immutable.IndexedSeq[String] = prepareTestTopicHavingTestMessages(topic) + val data: immutable.IndexedSeq[(String, Seq[(String, Array[Byte])])] = + prepareTestTopicHavingTestMessages(topic) val topicPartition = new TopicPartition(topic, 0) val kafkaParams = getKafkaParams() @@ -157,10 +159,22 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester try { val range = consumer.getAvailableOffsetRange() val rcvd = range.earliest until range.latest map { offset => - val bytes = consumer.get(offset, Long.MaxValue, 10000, failOnDataLoss = false).value() - new String(bytes) + val record = consumer.get(offset, Long.MaxValue, 10000, failOnDataLoss = false) + val value = new String(record.value(), StandardCharsets.UTF_8) + val headers = record.headers().toArray.map(header => (header.key(), header.value())).toSeq + (value, headers) + } + data.zip(rcvd).foreach { case (expected, actual) => + // value + assert(expected._1 === actual._1) + // headers + expected._2.zip(actual._2).foreach { case (l, r) => + // header key + assert(l._1 === r._1) + // header value + assert(l._2 === r._2) + } } - assert(rcvd == data) } catch { case e: Throwable => error = e @@ -307,9 +321,9 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester } private def prepareTestTopicHavingTestMessages(topic: String) = { - val data = (1 to 1000).map(_.toString) + val data = (1 to 1000).map(i => (i.toString, Seq[(String, Array[Byte])]())) testUtils.createTopic(topic, 1) - testUtils.sendMessages(topic, data.toArray) + testUtils.sendMessages(topic, data.toArray, None) data } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index b4e1b78c7db4e..3c88609bcb45d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.kafka010 +import java.nio.charset.StandardCharsets.UTF_8 import java.util.Locale import java.util.concurrent.atomic.AtomicInteger @@ -70,7 +71,8 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession protected def createDF( topic: String, withOptions: Map[String, String] = Map.empty[String, String], - brokerAddress: Option[String] = None) = { + brokerAddress: Option[String] = None, + includeHeaders: Boolean = false) = { val df = spark .read .format("kafka") @@ -80,7 +82,13 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession withOptions.foreach { case (key, value) => df.option(key, value) } - df.load().selectExpr("CAST(value AS STRING)") + if (includeHeaders) { + df.option("includeHeaders", "true") + df.load() + .selectExpr("CAST(value AS STRING)", "headers") + } else { + df.load().selectExpr("CAST(value AS STRING)") + } } test("explicit earliest to latest offsets") { @@ -147,6 +155,27 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession checkAnswer(df, (0 to 30).map(_.toString).toDF) } + test("default starting and ending offsets with headers") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessage( + topic, ("1", Seq()), Some(0) + ) + testUtils.sendMessage( + topic, ("2", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))), Some(1) + ) + testUtils.sendMessage( + topic, ("3", Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8)))), Some(2) + ) + + // Implicit offset values, should default to earliest and latest + val df = createDF(topic, includeHeaders = true) + // Test that we default to "earliest" and "latest" + checkAnswer(df, Seq(("1", null), + ("2", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))), + ("3", Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8))))).toDF) + } + test("reuse same dataframe in query") { // This test ensures that we do not cache the Kafka Consumer in KafkaRelation val topic = newTopic() diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 84ad41610cccd..fdda13b1bf6ec 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.kafka010 +import java.nio.charset.StandardCharsets.UTF_8 import java.util.Locale import java.util.concurrent.atomic.AtomicInteger @@ -32,7 +33,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{BinaryType, DataType} +import org.apache.spark.sql.types.{BinaryType, DataType, StringType, StructField, StructType} abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with KafkaTest { protected var testUtils: KafkaTestUtils = _ @@ -59,13 +60,14 @@ abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with protected def newTopic(): String = s"topic-${topicId.getAndIncrement()}" - protected def createKafkaReader(topic: String): DataFrame = { + protected def createKafkaReader(topic: String, includeHeaders: Boolean = false): DataFrame = { spark.read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("startingOffsets", "earliest") .option("endingOffsets", "latest") .option("subscribe", topic) + .option("includeHeaders", includeHeaders.toString) .load() } } @@ -368,15 +370,51 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { test("batch - write to kafka") { val topic = newTopic() testUtils.createTopic(topic) - val df = Seq("1", "2", "3", "4", "5").map(v => (topic, v)).toDF("topic", "value") + val data = Seq( + Row(topic, "1", Seq( + Row("a", "b".getBytes(UTF_8)) + )), + Row(topic, "2", Seq( + Row("c", "d".getBytes(UTF_8)), + Row("e", "f".getBytes(UTF_8)) + )), + Row(topic, "3", Seq( + Row("g", "h".getBytes(UTF_8)), + Row("g", "i".getBytes(UTF_8)) + )), + Row(topic, "4", null), + Row(topic, "5", Seq( + Row("j", "k".getBytes(UTF_8)), + Row("j", "l".getBytes(UTF_8)), + Row("m", "n".getBytes(UTF_8)) + )) + ) + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), + StructType(Seq(StructField("topic", StringType), StructField("value", StringType), + StructField("headers", KafkaRecordToRowConverter.headersType))) + ) + df.write .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("topic", topic) .save() checkAnswer( - createKafkaReader(topic).selectExpr("CAST(value as STRING) value"), - Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) + createKafkaReader(topic, includeHeaders = true).selectExpr( + "CAST(value as STRING) value", "headers" + ), + Row("1", Seq(Row("a", "b".getBytes(UTF_8)))) :: + Row("2", Seq(Row("c", "d".getBytes(UTF_8)), Row("e", "f".getBytes(UTF_8)))) :: + Row("3", Seq(Row("g", "h".getBytes(UTF_8)), Row("g", "i".getBytes(UTF_8)))) :: + Row("4", null) :: + Row("5", Seq( + Row("j", "k".getBytes(UTF_8)), + Row("j", "l".getBytes(UTF_8)), + Row("m", "n".getBytes(UTF_8)))) :: + Nil + ) } test("batch - null topic field value, and no topic option") { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index d7cb30f530396..f7114129a3cdc 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -41,6 +41,8 @@ import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.SaslConfigs +import org.apache.kafka.common.header.Header +import org.apache.kafka.common.header.internals.RecordHeader import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol.{PLAINTEXT, SASL_PLAINTEXT} import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} @@ -369,17 +371,36 @@ class KafkaTestUtils( topic: String, messages: Array[String], partition: Option[Int]): Seq[(String, RecordMetadata)] = { + sendMessages(topic, messages.map(m => (m, Seq())), partition) + } + + /** Send record to the Kafka broker with headers using specified partition */ + def sendMessage(topic: String, + record: (String, Seq[(String, Array[Byte])]), + partition: Option[Int]): Seq[(String, RecordMetadata)] = { + sendMessages(topic, Array(record).toSeq, partition) + } + + /** Send the array of records to the Kafka broker with headers using specified partition */ + def sendMessages(topic: String, + records: Seq[(String, Seq[(String, Array[Byte])])], + partition: Option[Int]): Seq[(String, RecordMetadata)] = { producer = new KafkaProducer[String, String](producerConfiguration) val offsets = try { - messages.map { m => + records.map { case (value, header) => + val headers = header.map { case (k, v) => + new RecordHeader(k, v).asInstanceOf[Header] + } val record = partition match { - case Some(p) => new ProducerRecord[String, String](topic, p, null, m) - case None => new ProducerRecord[String, String](topic, m) + case Some(p) => + new ProducerRecord[String, String](topic, p, null, value, headers.asJava) + case None => + new ProducerRecord[String, String](topic, null, null, value, headers.asJava) } - val metadata = - producer.send(record).get(10, TimeUnit.SECONDS) - logInfo(s"\tSent $m to partition ${metadata.partition}, offset ${metadata.offset}") - (m, metadata) + val metadata = producer.send(record).get(10, TimeUnit.SECONDS) + logInfo(s"\tSent ($value, $header) to partition ${metadata.partition}," + + " offset ${metadata.offset}") + (value, metadata) } } finally { if (producer != null) { From e63098b28766ffe22aba4767584d5cfdea4f78a0 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 13 Sep 2019 11:54:46 -0700 Subject: [PATCH 0060/1387] [SPARK-29007][MLLIB][FOLLOWUP] Remove duplicated dependency ### What changes were proposed in this pull request? This removes the duplicated dependency which is added by [SPARK-29007](https://github.com/apache/spark/blame/b62ef8f7935ae5c9a4a5e7e8a17aa5d7375c85b1/mllib/pom.xml#L58-L64). ### Why are the changes needed? Maven complains this kind of duplications. We had better be safe in the future Maven versions. ``` $ cd mllib $ mvn clean package -DskipTests [INFO] Scanning for projects... [WARNING] [WARNING] Some problems were encountered while building the effective model for org.apache.spark:spark-mllib_2.12:jar:3.0.0-SNAPSHOT [WARNING] 'dependencies.dependency.(groupId:artifactId:type:classifier)' must be unique: org.apache.spark:spark-streaming_${scala.binary.version}:test-jar -> duplicate declaration of version ${project.version} line 119, column 17 [WARNING] [WARNING] It is highly recommended to fix these problems because they threaten the stability of your build. [WARNING] [WARNING] For this reason, future Maven versions might no longer support building such malformed projects. [WARNING] ... ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manual check since this is a warning. ``` $ cd mllib $ mvn clean package -DskipTests ``` Closes #25783 from dongjoon-hyun/SPARK-29007. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- mllib/pom.xml | 7 ------- 1 file changed, 7 deletions(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index 2d2119683e7ad..11769ef548d7c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -55,13 +55,6 @@ spark-streaming_${scala.binary.version} ${project.version}
- - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - test-jar - test - org.apache.spark spark-sql_${scala.binary.version} From c610de69526d65f3b679cfd81ab7e1a5791ff37f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 13 Sep 2019 14:07:00 -0700 Subject: [PATCH 0061/1387] [SPARK-29042][CORE] Sampling-based RDD with unordered input should be INDETERMINATE ### What changes were proposed in this pull request? We already have found and fixed the correctness issue before when RDD output is INDETERMINATE. One missing part is sampling-based RDD. This kind of RDDs is order sensitive to its input. A sampling-based RDD with unordered input, should be INDETERMINATE. ### Why are the changes needed? A sampling-based RDD with unordered input is just like MapPartitionsRDD with isOrderSensitive parameter as true. The RDD output can be different after a rerun. It is a problem in ML applications. In ML, sample is used to prepare training data. ML algorithm fits the model based on the sampled data. If rerun tasks of sample produce different output during model fitting, ML results will be unreliable and also buggy. Each sample is random output, but once you sampled, the output should be determinate. ### Does this PR introduce any user-facing change? Previously, a sampling-based RDD can possibly come with different output after a rerun. After this patch, sampling-based RDD is INDETERMINATE. For an INDETERMINATE map stage, currently Spark scheduler will re-try all the tasks of the failed stage. ### How was this patch tested? Added test. Closes #25751 from viirya/sample-order-sensitive. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- .../apache/spark/rdd/PairRDDFunctions.scala | 4 +-- .../spark/rdd/PartitionwiseSampledRDD.scala | 8 ++++++ .../main/scala/org/apache/spark/rdd/RDD.scala | 25 ++++++++++++++++++- .../spark/scheduler/DAGSchedulerSuite.scala | 12 +++++++++ .../apache/spark/graphx/impl/GraphImpl.scala | 19 +++++++------- 5 files changed, 56 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index e23133682360f..1e39e10856877 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -261,7 +261,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } else { StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, false, seed) } - self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true, isOrderSensitive = true) } /** @@ -291,7 +291,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } else { StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, true, seed) } - self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true, isOrderSensitive = true) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index 15691a8fc8eaa..c8cdaa60e4335 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -67,4 +67,12 @@ private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( thisSampler.setSeed(split.seed) thisSampler.sample(firstParent[T].iterator(split.prev, context)) } + + override protected def getOutputDeterministicLevel = { + if (prev.outputDeterministicLevel == DeterministicLevel.UNORDERED) { + DeterministicLevel.INDETERMINATE + } else { + super.getOutputDeterministicLevel + } + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 53268b169de42..08fc309d5238e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -555,7 +555,7 @@ abstract class RDD[T: ClassTag]( val sampler = new BernoulliCellSampler[T](lb, ub) sampler.setSeed(seed + index) sampler.sample(partition) - }, preservesPartitioning = true) + }, isOrderSensitive = true, preservesPartitioning = true) } /** @@ -868,6 +868,29 @@ abstract class RDD[T: ClassTag]( preservesPartitioning) } + /** + * Return a new RDD by applying a function to each partition of this RDD, while tracking the index + * of the original partition. + * + * `preservesPartitioning` indicates whether the input function preserves the partitioner, which + * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. + * + * `isOrderSensitive` indicates whether the function is order-sensitive. If it is order + * sensitive, it may return totally different result when the input order + * is changed. Mostly stateful functions are order-sensitive. + */ + private[spark] def mapPartitionsWithIndex[U: ClassTag]( + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean, + isOrderSensitive: Boolean): RDD[U] = withScope { + val cleanedF = sc.clean(f) + new MapPartitionsRDD( + this, + (_: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(index, iter), + preservesPartitioning, + isOrderSensitive = isOrderSensitive) + } + /** * Zips this RDD with another one, returning key-value pairs with the first element in each RDD, * second element in each RDD, etc. Assumes that the two RDDs have the *same number of diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index d6cd57fa6e7b6..7cb7eceec615b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2783,6 +2783,18 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi .contains("Spark cannot rollback the ShuffleMapStage 1")) } + test("SPARK-29042: Sampled RDD with unordered input should be indeterminate") { + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = false) + + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) + val shuffleMapRdd2 = new MyRDD(sc, 2, List(shuffleDep1), tracker = mapOutputTracker) + + assert(shuffleMapRdd2.outputDeterministicLevel == DeterministicLevel.UNORDERED) + + val sampledRdd = shuffleMapRdd2.sample(true, 0.3, 1000L) + assert(sampledRdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) + } + private def assertResultStageFailToRollback(mapRdd: MyRDD): Unit = { val shuffleDep = new ShuffleDependency(mapRdd, new HashPartitioner(2)) val shuffleId = shuffleDep.shuffleId diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 0a97ab492600d..8564597f4f135 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -103,15 +103,16 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( (part, (e.srcId, e.dstId, e.attr)) } .partitionBy(new HashPartitioner(numPartitions)) - .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED, VD]()(edTag, vdTag) - iter.foreach { message => - val data = message._2 - builder.add(data._1, data._2, data._3) - } - val edgePartition = builder.toEdgePartition - Iterator((pid, edgePartition)) - }, preservesPartitioning = true)).cache() + .mapPartitionsWithIndex( + { (pid: Int, iter: Iterator[(PartitionID, (VertexId, VertexId, ED))]) => + val builder = new EdgePartitionBuilder[ED, VD]()(edTag, vdTag) + iter.foreach { message => + val data = message._2 + builder.add(data._1, data._2, data._3) + } + val edgePartition = builder.toEdgePartition + Iterator((pid, edgePartition)) + }, preservesPartitioning = true)).cache() GraphImpl.fromExistingRDDs(vertices.withEdges(newEdges), newEdges) } From bbfaadb280a80b511a98d18881641c6d9851dd51 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 13 Sep 2019 21:28:27 +0000 Subject: [PATCH 0062/1387] [SPARK-29064][CORE] Add PrometheusResource to export Executor metrics ### What changes were proposed in this pull request? At Apache Spark 3.0.0, [SPARK-23429](https://github.com/apache/spark/pull/21221) added the ability to collect executor metrics via heartbeats and to expose it as a REST API. This PR aims to extend it to support `Prometheus` format additionally. ### Why are the changes needed? Prometheus.io is a CNCF project used widely with K8s. - https://github.com/prometheus/prometheus ### Does this PR introduce any user-facing change? Yes. New web interfaces are added along with the existing JSON API. | | JSON End Point | Prometheus End Point | | ------- | ------------------------------------ | --------------------------------- | | Driver | /api/v1/applications/{id}/executors/ | /metrics/executors/prometheus/ | ### How was this patch tested? Manually connect to the new end-points with `curl` and compare with JSON. **SETUP** ``` $ sbin/start-master.sh $ sbin/start-slave.sh spark://`hostname`:7077 $ bin/spark-shell --master spark://`hostname`:7077 --conf spark.ui.prometheus.enabled=true ``` **JSON (existing after SPARK-23429)** ``` $ curl -s http://localhost:4040/api/v1/applications/app-20190911204823-0000/executors [ { "id" : "driver", "hostPort" : "localhost:52615", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, "diskUsed" : 0, "totalCores" : 0, "maxTasks" : 0, "activeTasks" : 0, "failedTasks" : 0, "completedTasks" : 0, "totalTasks" : 0, "totalDuration" : 0, "totalGCTime" : 0, "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 384093388, "addTime" : "2019-09-12T03:48:23.875GMT", "executorLogs" : { }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, "usedOffHeapStorageMemory" : 0, "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 0 }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { "JVMHeapMemory" : 229995952, "JVMOffHeapMemory" : 145872280, "OnHeapExecutionMemory" : 0, "OffHeapExecutionMemory" : 0, "OnHeapStorageMemory" : 0, "OffHeapStorageMemory" : 0, "OnHeapUnifiedMemory" : 0, "OffHeapUnifiedMemory" : 0, "DirectPoolMemory" : 75891, "MappedPoolMemory" : 0, "ProcessTreeJVMVMemory" : 0, "ProcessTreeJVMRSSMemory" : 0, "ProcessTreePythonVMemory" : 0, "ProcessTreePythonRSSMemory" : 0, "ProcessTreeOtherVMemory" : 0, "ProcessTreeOtherRSSMemory" : 0, "MinorGCCount" : 8, "MinorGCTime" : 82, "MajorGCCount" : 3, "MajorGCTime" : 128 }, "attributes" : { }, "resources" : { } }, { "id" : "0", "hostPort" : "127.0.0.1:52619", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, "diskUsed" : 0, "totalCores" : 16, "maxTasks" : 16, "activeTasks" : 0, "failedTasks" : 0, "completedTasks" : 0, "totalTasks" : 0, "totalDuration" : 0, "totalGCTime" : 0, "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 384093388, "addTime" : "2019-09-12T03:48:25.907GMT", "executorLogs" : { "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20190911204823-0000&executorId=0&logType=stdout", "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20190911204823-0000&executorId=0&logType=stderr" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, "usedOffHeapStorageMemory" : 0, "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 0 }, "blacklistedInStages" : [ ], "attributes" : { }, "resources" : { } } ] ``` **Prometheus** ``` $ curl -s http://localhost:4040/metrics/executors/prometheus metrics_app_20190911204823_0000_driver_executor_rddBlocks_Count 0 metrics_app_20190911204823_0000_driver_executor_memoryUsed_Count 0 metrics_app_20190911204823_0000_driver_executor_diskUsed_Count 0 metrics_app_20190911204823_0000_driver_executor_totalCores_Count 0 metrics_app_20190911204823_0000_driver_executor_maxTasks_Count 0 metrics_app_20190911204823_0000_driver_executor_activeTasks_Count 0 metrics_app_20190911204823_0000_driver_executor_failedTasks_Count 0 metrics_app_20190911204823_0000_driver_executor_completedTasks_Count 0 metrics_app_20190911204823_0000_driver_executor_totalTasks_Count 0 metrics_app_20190911204823_0000_driver_executor_totalDuration_Value 0 metrics_app_20190911204823_0000_driver_executor_totalGCTime_Value 0 metrics_app_20190911204823_0000_driver_executor_totalInputBytes_Count 0 metrics_app_20190911204823_0000_driver_executor_totalShuffleRead_Count 0 metrics_app_20190911204823_0000_driver_executor_totalShuffleWrite_Count 0 metrics_app_20190911204823_0000_driver_executor_maxMemory_Count 384093388 metrics_app_20190911204823_0000_driver_executor_usedOnHeapStorageMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_usedOffHeapStorageMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_totalOnHeapStorageMemory_Count 384093388 metrics_app_20190911204823_0000_driver_executor_totalOffHeapStorageMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_JVMHeapMemory_Count 230406336 metrics_app_20190911204823_0000_driver_executor_JVMOffHeapMemory_Count 146132592 metrics_app_20190911204823_0000_driver_executor_OnHeapExecutionMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_OffHeapExecutionMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_OnHeapStorageMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_OffHeapStorageMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_OnHeapUnifiedMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_OffHeapUnifiedMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_DirectPoolMemory_Count 97049 metrics_app_20190911204823_0000_driver_executor_MappedPoolMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_ProcessTreeJVMVMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_ProcessTreeJVMRSSMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_ProcessTreePythonVMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_ProcessTreePythonRSSMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_ProcessTreeOtherVMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_ProcessTreeOtherRSSMemory_Count 0 metrics_app_20190911204823_0000_driver_executor_MinorGCCount_Count 8 metrics_app_20190911204823_0000_driver_executor_MinorGCTime_Count 82 metrics_app_20190911204823_0000_driver_executor_MajorGCCount_Count 3 metrics_app_20190911204823_0000_driver_executor_MajorGCTime_Count 128 metrics_app_20190911204823_0000_0_executor_rddBlocks_Count 0 metrics_app_20190911204823_0000_0_executor_memoryUsed_Count 0 metrics_app_20190911204823_0000_0_executor_diskUsed_Count 0 metrics_app_20190911204823_0000_0_executor_totalCores_Count 16 metrics_app_20190911204823_0000_0_executor_maxTasks_Count 16 metrics_app_20190911204823_0000_0_executor_activeTasks_Count 0 metrics_app_20190911204823_0000_0_executor_failedTasks_Count 0 metrics_app_20190911204823_0000_0_executor_completedTasks_Count 0 metrics_app_20190911204823_0000_0_executor_totalTasks_Count 0 metrics_app_20190911204823_0000_0_executor_totalDuration_Value 0 metrics_app_20190911204823_0000_0_executor_totalGCTime_Value 0 metrics_app_20190911204823_0000_0_executor_totalInputBytes_Count 0 metrics_app_20190911204823_0000_0_executor_totalShuffleRead_Count 0 metrics_app_20190911204823_0000_0_executor_totalShuffleWrite_Count 0 metrics_app_20190911204823_0000_0_executor_maxMemory_Count 384093388 metrics_app_20190911204823_0000_0_executor_usedOnHeapStorageMemory_Count 0 metrics_app_20190911204823_0000_0_executor_usedOffHeapStorageMemory_Count 0 metrics_app_20190911204823_0000_0_executor_totalOnHeapStorageMemory_Count 384093388 metrics_app_20190911204823_0000_0_executor_totalOffHeapStorageMemory_Count 0 ``` Closes #25770 from dongjoon-hyun/SPARK-29064. Authored-by: Dongjoon Hyun Signed-off-by: DB Tsai --- .../org/apache/spark/internal/config/UI.scala | 7 ++ .../status/api/v1/PrometheusResource.scala | 110 ++++++++++++++++++ .../scala/org/apache/spark/ui/SparkUI.scala | 3 + 3 files changed, 120 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/status/api/v1/PrometheusResource.scala diff --git a/core/src/main/scala/org/apache/spark/internal/config/UI.scala b/core/src/main/scala/org/apache/spark/internal/config/UI.scala index a11970ec73d88..1a8268161160b 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/UI.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/UI.scala @@ -81,6 +81,13 @@ private[spark] object UI { .booleanConf .createWithDefault(true) + val UI_PROMETHEUS_ENABLED = ConfigBuilder("spark.ui.prometheus.enabled") + .internal() + .doc("Expose executor metrics at /metrics/executors/prometheus. " + + "For master/worker/driver metrics, you need to configure `conf/metrics.properties`.") + .booleanConf + .createWithDefault(false) + val UI_X_XSS_PROTECTION = ConfigBuilder("spark.ui.xXssProtection") .doc("Value for HTTP X-XSS-Protection response header") .stringConf diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/PrometheusResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/PrometheusResource.scala new file mode 100644 index 0000000000000..6e52e213bda8e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/PrometheusResource.scala @@ -0,0 +1,110 @@ +/* + * 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.status.api.v1 + +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.glassfish.jersey.server.ServerProperties +import org.glassfish.jersey.servlet.ServletContainer + +import org.apache.spark.ui.SparkUI + +/** + * This aims to expose Executor metrics like REST API which is documented in + * + * https://spark.apache.org/docs/3.0.0/monitoring.html#executor-metrics + * + * Note that this is based on ExecutorSummary which is different from ExecutorSource. + */ +@Path("/executors") +private[v1] class PrometheusResource extends ApiRequestContext { + @GET + @Path("prometheus") + @Produces(Array(MediaType.TEXT_PLAIN)) + def executors(): String = { + val sb = new StringBuilder + val store = uiRoot.asInstanceOf[SparkUI].store + val appId = store.applicationInfo.id.replaceAll("[^a-zA-Z0-9]", "_") + store.executorList(true).foreach { executor => + val prefix = s"metrics_${appId}_${executor.id}_executor_" + sb.append(s"${prefix}rddBlocks_Count ${executor.rddBlocks}\n") + sb.append(s"${prefix}memoryUsed_Count ${executor.memoryUsed}\n") + sb.append(s"${prefix}diskUsed_Count ${executor.diskUsed}\n") + sb.append(s"${prefix}totalCores_Count ${executor.totalCores}\n") + sb.append(s"${prefix}maxTasks_Count ${executor.maxTasks}\n") + sb.append(s"${prefix}activeTasks_Count ${executor.activeTasks}\n") + sb.append(s"${prefix}failedTasks_Count ${executor.failedTasks}\n") + sb.append(s"${prefix}completedTasks_Count ${executor.completedTasks}\n") + sb.append(s"${prefix}totalTasks_Count ${executor.totalTasks}\n") + sb.append(s"${prefix}totalDuration_Value ${executor.totalDuration}\n") + sb.append(s"${prefix}totalGCTime_Value ${executor.totalGCTime}\n") + sb.append(s"${prefix}totalInputBytes_Count ${executor.totalInputBytes}\n") + sb.append(s"${prefix}totalShuffleRead_Count ${executor.totalShuffleRead}\n") + sb.append(s"${prefix}totalShuffleWrite_Count ${executor.totalShuffleWrite}\n") + sb.append(s"${prefix}maxMemory_Count ${executor.maxMemory}\n") + executor.executorLogs.foreach { case (k, v) => } + executor.memoryMetrics.foreach { m => + sb.append(s"${prefix}usedOnHeapStorageMemory_Count ${m.usedOnHeapStorageMemory}\n") + sb.append(s"${prefix}usedOffHeapStorageMemory_Count ${m.usedOffHeapStorageMemory}\n") + sb.append(s"${prefix}totalOnHeapStorageMemory_Count ${m.totalOnHeapStorageMemory}\n") + sb.append(s"${prefix}totalOffHeapStorageMemory_Count ${m.totalOffHeapStorageMemory}\n") + } + executor.peakMemoryMetrics.foreach { m => + val names = Array( + "JVMHeapMemory", + "JVMOffHeapMemory", + "OnHeapExecutionMemory", + "OffHeapExecutionMemory", + "OnHeapStorageMemory", + "OffHeapStorageMemory", + "OnHeapUnifiedMemory", + "OffHeapUnifiedMemory", + "DirectPoolMemory", + "MappedPoolMemory", + "ProcessTreeJVMVMemory", + "ProcessTreeJVMRSSMemory", + "ProcessTreePythonVMemory", + "ProcessTreePythonRSSMemory", + "ProcessTreeOtherVMemory", + "ProcessTreeOtherRSSMemory", + "MinorGCCount", + "MinorGCTime", + "MajorGCCount", + "MajorGCTime" + ) + names.foreach { name => + sb.append(s"$prefix${name}_Count ${m.getMetricValue(name)}\n") + } + } + } + sb.toString + } +} + +private[spark] object PrometheusResource { + def getServletHandler(uiRoot: UIRoot): ServletContextHandler = { + val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) + jerseyContext.setContextPath("/metrics") + val holder: ServletHolder = new ServletHolder(classOf[ServletContainer]) + holder.setInitParameter(ServerProperties.PROVIDER_PACKAGES, "org.apache.spark.status.api.v1") + UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot) + jerseyContext.addServlet(holder, "/*") + jerseyContext + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 6fb8e458a789c..05e9c7f859b2e 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -66,6 +66,9 @@ private[spark] class SparkUI private ( addStaticHandler(SparkUI.STATIC_RESOURCE_DIR) attachHandler(createRedirectHandler("/", "/jobs/", basePath = basePath)) attachHandler(ApiRootResource.getServletHandler(this)) + if (sc.map(_.conf.get(UI_PROMETHEUS_ENABLED)).getOrElse(false)) { + attachHandler(PrometheusResource.getServletHandler(this)) + } // These should be POST only, but, the YARN AM proxy won't proxy POSTs attachHandler(createRedirectHandler( From 8f057a961289fa5c4c4863fb36ce22b1c3789de6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 13 Sep 2019 21:31:21 +0000 Subject: [PATCH 0063/1387] [SPARK-29032][CORE] Add PrometheusServlet to monitor Master/Worker/Driver ### What changes were proposed in this pull request? This PR aims to simplify `Prometheus` support by adding `PrometheusServlet`. The main use cases are `K8s` and `Spark Standalone` cluster environments. ### Why are the changes needed? Prometheus.io is a CNCF project used widely with K8s. - https://github.com/prometheus/prometheus For `Master/Worker/Driver`, `Spark JMX Sink` and `Prometheus JMX Converter` combination is used in many cases. One way to achieve that is having the followings. **JMX Sink (conf/metrics.properties)** ``` *.sink.jmx.class=org.apache.spark.metrics.sink.JmxSink ``` **JMX Converter(conf/spark-env.sh)** - https://repo1.maven.org/maven2/io/prometheus/jmx/jmx_prometheus_javaagent/0.12.0/jmx_prometheus_javaagent-0.12.0.jar ``` export SPARK_DAEMON_JAVA_OPTS= "-javaagent:${PWD}/jmx_prometheus_javaagent-${JMX_PROMETHEUS_VERSION}.jar= ${PORT_AGENT}:jmx_prometheus.yaml" ``` This agent approach requires `PORT_AGENT` additionally. Instead, this PR natively support `Prometheus` format exporting with reusing REST API port for the better UX. ### Does this PR introduce any user-facing change? Yes. New web interfaces are added along with the existing JSON API. | | JSON End Point | Prometheus End Point | | ------- | ------------------------------------------- | ---------------------------------- | | Master | /metrics/master/json/ | /metrics/master/prometheus/ | | Master | /metrics/applications/json/ | /metrics/applications/prometheus/ | | Worker | /metrics/json/ | /metrics/prometheus/ | | Driver | /metrics/json/ | /metrics/prometheus/ | ### How was this patch tested? Manually connect the new end-points with `curl`. **Setup (Master/Worker/Driver)** Add the followings at `conf/metrics.properties` (`conf/metrics.properties.template` has these examples) ``` *.sink.prometheusServlet.class=org.apache.spark.metrics.sink.PrometheusServlet *.sink.prometheusServlet.path=/metrics/prometheus master.sink.prometheusServlet.path=/metrics/master/prometheus applications.sink.prometheusServlet.path=/metrics/applications/prometheus ``` ``` $ sbin/start-master.sh $ sbin/start-slave.sh spark://`hostname`:7077 $ bin/spark-shell --master spark://`hostname`:7077 ``` ``` $ curl -s http://localhost:8080/metrics/master/json/ | jq { "version": "3.1.3", "gauges": { "master.aliveWorkers": { "value": 1 }, "master.apps": { "value": 1 }, "master.waitingApps": { "value": 0 }, "master.workers": { "value": 1 } }, ... $ curl -s http://localhost:8080/metrics/master/prometheus/ | grep master metrics_master_aliveWorkers_Value 1 metrics_master_apps_Value 1 metrics_master_waitingApps_Value 0 metrics_master_workers_Value 1 ``` ``` $ curl -s http://localhost:8080/metrics/applications/json/ | jq { "version": "3.1.3", "gauges": { "application.Spark shell.1568261490667.cores": { "value": 16 }, "application.Spark shell.1568261490667.runtime_ms": { "value": 108966 }, "application.Spark shell.1568261490667.status": { "value": "RUNNING" } }, ... $ curl -s http://localhost:8080/metrics/applications/prometheus/ | grep application metrics_application_Spark_shell_1568261490667_cores_Value 16 metrics_application_Spark_shell_1568261490667_runtime_ms_Value 143174 ``` ``` $ curl -s http://localhost:8081/metrics/json/ | jq { "version": "3.1.3", "gauges": { "worker.coresFree": { "value": 0 }, "worker.coresUsed": { "value": 16 }, "worker.executors": { "value": 1 }, "worker.memFree_MB": { "value": 30720 }, "worker.memUsed_MB": { "value": 1024 } }, ... $ curl -s http://localhost:8081/metrics/prometheus/ | grep worker metrics_worker_coresFree_Value 0 metrics_worker_coresUsed_Value 16 metrics_worker_executors_Value 1 metrics_worker_memFree_MB_Value 30720 metrics_worker_memUsed_MB_Value 1024 ``` ``` $ curl -s http://localhost:4040/metrics/json/ | jq { "version": "3.1.3", "gauges": { "app-20190911211130-0000.driver.BlockManager.disk.diskSpaceUsed_MB": { "value": 0 }, "app-20190911211130-0000.driver.BlockManager.memory.maxMem_MB": { "value": 732 }, "app-20190911211130-0000.driver.BlockManager.memory.maxOffHeapMem_MB": { "value": 0 }, "app-20190911211130-0000.driver.BlockManager.memory.maxOnHeapMem_MB": { "value": 732 }, ... $ curl -s http://localhost:4040/metrics/prometheus/ | head -n5 metrics_app_20190911211130_0000_driver_BlockManager_disk_diskSpaceUsed_MB_Value 0 metrics_app_20190911211130_0000_driver_BlockManager_memory_maxMem_MB_Value 732 metrics_app_20190911211130_0000_driver_BlockManager_memory_maxOffHeapMem_MB_Value 0 metrics_app_20190911211130_0000_driver_BlockManager_memory_maxOnHeapMem_MB_Value 732 metrics_app_20190911211130_0000_driver_BlockManager_memory_memUsed_MB_Value 0 ``` Closes #25769 from dongjoon-hyun/SPARK-29032-2. Authored-by: Dongjoon Hyun Signed-off-by: DB Tsai --- conf/metrics.properties.template | 17 ++- .../apache/spark/metrics/MetricsSystem.scala | 12 +- .../metrics/sink/PrometheusServlet.scala | 121 ++++++++++++++++++ 3 files changed, 147 insertions(+), 3 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/metrics/sink/PrometheusServlet.scala diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index da0b06d295252..f52d33fd64223 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -113,6 +113,15 @@ # /metrics/applications/json # App information # /metrics/master/json # Master information +# org.apache.spark.metrics.sink.PrometheusServlet +# Name: Default: Description: +# path VARIES* Path prefix from the web server root +# +# * Default path is /metrics/prometheus for all instances except the master. The +# master has two paths: +# /metrics/applications/prometheus # App information +# /metrics/master/prometheus # Master information + # org.apache.spark.metrics.sink.GraphiteSink # Name: Default: Description: # host NONE Hostname of the Graphite server, must be set @@ -192,4 +201,10 @@ #driver.source.jvm.class=org.apache.spark.metrics.source.JvmSource -#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource \ No newline at end of file +#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource + +# Example configuration for PrometheusServlet +#*.sink.prometheusServlet.class=org.apache.spark.metrics.sink.PrometheusServlet +#*.sink.prometheusServlet.path=/metrics/prometheus +#master.sink.prometheusServlet.path=/metrics/master/prometheus +#applications.sink.prometheusServlet.path=/metrics/applications/prometheus diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index b5524442a8f99..f4dd8a349ad79 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -28,7 +28,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.metrics.sink.{MetricsServlet, Sink} +import org.apache.spark.metrics.sink.{MetricsServlet, PrometheusServlet, Sink} import org.apache.spark.metrics.source.{Source, StaticSources} import org.apache.spark.util.Utils @@ -83,13 +83,15 @@ private[spark] class MetricsSystem private ( // Treat MetricsServlet as a special sink as it should be exposed to add handlers to web ui private var metricsServlet: Option[MetricsServlet] = None + private var prometheusServlet: Option[PrometheusServlet] = None /** * Get any UI handlers used by this metrics system; can only be called after start(). */ def getServletHandlers: Array[ServletContextHandler] = { require(running, "Can only call getServletHandlers on a running MetricsSystem") - metricsServlet.map(_.getHandlers(conf)).getOrElse(Array()) + metricsServlet.map(_.getHandlers(conf)).getOrElse(Array()) ++ + prometheusServlet.map(_.getHandlers(conf)).getOrElse(Array()) } metricsConfig.initialize() @@ -201,6 +203,12 @@ private[spark] class MetricsSystem private ( classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) .newInstance(kv._2, registry, securityMgr) metricsServlet = Some(servlet) + } else if (kv._1 == "prometheusServlet") { + val servlet = Utils.classForName[PrometheusServlet](classPath) + .getConstructor( + classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) + .newInstance(kv._2, registry, securityMgr) + prometheusServlet = Some(servlet) } else { val sink = Utils.classForName[Sink](classPath) .getConstructor( diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusServlet.scala new file mode 100644 index 0000000000000..5b67754b657be --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusServlet.scala @@ -0,0 +1,121 @@ +/* + * 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.metrics.sink + +import java.util.Properties +import javax.servlet.http.HttpServletRequest + +import com.codahale.metrics.MetricRegistry +import org.eclipse.jetty.servlet.ServletContextHandler + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.ui.JettyUtils._ + +/** + * This exposes the metrics of the given registry with Prometheus format. + * + * The output is consistent with /metrics/json result in terms of item ordering + * and with the previous result of Spark JMX Sink + Prometheus JMX Converter combination + * in terms of key string format. + */ +private[spark] class PrometheusServlet( + val property: Properties, + val registry: MetricRegistry, + securityMgr: SecurityManager) + extends Sink { + + val SERVLET_KEY_PATH = "path" + + val servletPath = property.getProperty(SERVLET_KEY_PATH) + + def getHandlers(conf: SparkConf): Array[ServletContextHandler] = { + Array[ServletContextHandler]( + createServletHandler(servletPath, + new ServletParams(request => getMetricsSnapshot(request), "text/plain"), conf) + ) + } + + def getMetricsSnapshot(request: HttpServletRequest): String = { + import scala.collection.JavaConverters._ + + val sb = new StringBuilder() + registry.getGauges.asScala.foreach { case (k, v) => + if (!v.getValue.isInstanceOf[String]) { + sb.append(s"${normalizeKey(k)}Value ${v.getValue}\n") + } + } + registry.getCounters.asScala.foreach { case (k, v) => + sb.append(s"${normalizeKey(k)}Count ${v.getCount}\n") + } + registry.getHistograms.asScala.foreach { case (k, h) => + val snapshot = h.getSnapshot + val prefix = normalizeKey(k) + sb.append(s"${prefix}Count ${h.getCount}\n") + sb.append(s"${prefix}Max ${snapshot.getMax}\n") + sb.append(s"${prefix}Mean ${snapshot.getMean}\n") + sb.append(s"${prefix}Min ${snapshot.getMin}\n") + sb.append(s"${prefix}50thPercentile ${snapshot.getMedian}\n") + sb.append(s"${prefix}75thPercentile ${snapshot.get75thPercentile}\n") + sb.append(s"${prefix}95thPercentile ${snapshot.get95thPercentile}\n") + sb.append(s"${prefix}98thPercentile ${snapshot.get98thPercentile}\n") + sb.append(s"${prefix}99thPercentile ${snapshot.get99thPercentile}\n") + sb.append(s"${prefix}999thPercentile ${snapshot.get999thPercentile}\n") + sb.append(s"${prefix}StdDev ${snapshot.getStdDev}\n") + } + registry.getMeters.entrySet.iterator.asScala.foreach { kv => + val prefix = normalizeKey(kv.getKey) + val meter = kv.getValue + sb.append(s"${prefix}Count ${meter.getCount}\n") + sb.append(s"${prefix}MeanRate ${meter.getMeanRate}\n") + sb.append(s"${prefix}OneMinuteRate ${meter.getOneMinuteRate}\n") + sb.append(s"${prefix}FiveMinuteRate ${meter.getFiveMinuteRate}\n") + sb.append(s"${prefix}FifteenMinuteRate ${meter.getFifteenMinuteRate}\n") + } + registry.getTimers.entrySet.iterator.asScala.foreach { kv => + val prefix = normalizeKey(kv.getKey) + val timer = kv.getValue + val snapshot = timer.getSnapshot + sb.append(s"${prefix}Count ${timer.getCount}\n") + sb.append(s"${prefix}Max ${snapshot.getMax}\n") + sb.append(s"${prefix}Mean ${snapshot.getMax}\n") + sb.append(s"${prefix}Min ${snapshot.getMin}\n") + sb.append(s"${prefix}50thPercentile ${snapshot.getMedian}\n") + sb.append(s"${prefix}75thPercentile ${snapshot.get75thPercentile}\n") + sb.append(s"${prefix}95thPercentile ${snapshot.get95thPercentile}\n") + sb.append(s"${prefix}98thPercentile ${snapshot.get98thPercentile}\n") + sb.append(s"${prefix}99thPercentile ${snapshot.get99thPercentile}\n") + sb.append(s"${prefix}999thPercentile ${snapshot.get999thPercentile}\n") + sb.append(s"${prefix}StdDev ${snapshot.getStdDev}\n") + sb.append(s"${prefix}FifteenMinuteRate ${timer.getFifteenMinuteRate}\n") + sb.append(s"${prefix}FiveMinuteRate ${timer.getFiveMinuteRate}\n") + sb.append(s"${prefix}OneMinuteRate ${timer.getOneMinuteRate}\n") + sb.append(s"${prefix}MeanRate ${timer.getMeanRate}\n") + } + sb.toString() + } + + private def normalizeKey(key: String): String = { + s"metrics_${key.replaceAll("[^a-zA-Z0-9]", "_")}_" + } + + override def start() { } + + override def stop() { } + + override def report() { } +} From 8174238d5521795bc7c7956b38cc6c604a97c153 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 13 Sep 2019 14:35:02 -0700 Subject: [PATCH 0064/1387] [SPARK-29075][BUILD] Add enforcer rule to ban duplicated pom dependency ### What changes were proposed in this pull request? This PR aims to add a new enforcer rule to ban duplicated pom dependency during build stage. ### Why are the changes needed? This will help us by preventing the extra effort like the followings. ``` e63098b287 [SPARK-29007][MLLIB][FOLLOWUP] Remove duplicated dependency 39e044e3d8 [MINOR][BUILD] Remove duplicate test-jar:test spark-sql dependency from Hive module d8fefab4d8 [HOTFIX][BUILD][TEST-MAVEN] Remove duplicate dependency e9445b187e [SPARK-6866][Build] Remove duplicated dependency in launcher/pom.xml ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually. If we have something like e63098b287, it will fail at building phase at PR like the following. ``` [WARNING] Rule 0: org.apache.maven.plugins.enforcer.BanDuplicatePomDependencyVersions failed with message: Found 1 duplicate dependency declaration in this project: - dependencies.dependency[org.apache.spark:spark-streaming_${scala.binary.version}:test-jar] ( 2 times ) ... [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [ERROR] Failed to execute goal org.apache.maven.plugins:maven-enforcer-plugin:3.0.0-M2:enforce (enforce-no-duplicate-dependencies) on project spark-mllib_2.12: Some Enforcer rules have failed. Look above for specific messages explaining why the rule failed. -> [Help 1] ``` Closes #25784 from dongjoon-hyun/SPARK-29075. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/pom.xml b/pom.xml index 6c474f5f7a3e7..f67918af4dc90 100644 --- a/pom.xml +++ b/pom.xml @@ -2287,6 +2287,17 @@ + + enforce-no-duplicate-dependencies + + enforce + + + + + + + From 703fb2b054651aa479c721ddd76845c1df52066c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 13 Sep 2019 21:50:06 +0000 Subject: [PATCH 0065/1387] [SPARK-29079][INFRA] Enable GitHub Action on PR ### What changes were proposed in this pull request? This PR enables GitHub Action on PRs. ### Why are the changes needed? So far, we detect JDK11 compilation error after merging. This PR aims to prevent JDK11 compilation error at PR stage. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manual. See the GitHub Action on this PR. Closes #25786 from dongjoon-hyun/SPARK-29079. Authored-by: Dongjoon Hyun Signed-off-by: DB Tsai --- .github/workflows/master.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index a9f757c3e2413..a4233d113a561 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -4,6 +4,9 @@ on: push: branches: - master + pull_request: + branches: + - master jobs: build: From 174d9104cf3915f1098bcf0fdbe4fec5dbddbadb Mon Sep 17 00:00:00 2001 From: Shanyu Zhao Date: Fri, 13 Sep 2019 21:09:17 -0700 Subject: [PATCH 0066/1387] [SPARK-29003][CORE] Add `start` method to ApplicationHistoryProvider to avoid deadlock on startup ### What changes were proposed in this pull request? During Spark History Server startup, there are two things happening simultaneously that call into `java.nio.file.FileSystems.getDefault()` and we sometime hit [JDK-8194653](https://bugs.openjdk.java.net/browse/JDK-8194653). 1) start jetty server 2) start ApplicationHistoryProvider (which reads files from HDFS) We should do these two things sequentially instead of in parallel. We introduce a start() method in ApplicationHistoryProvider (and its subclass FsHistoryProvider), and we do initialize inside the start() method instead of the constructor. In HistoryServer, we explicitly call provider.start() after we call bind() which starts the Jetty server. ### Why are the changes needed? It is a bug that occasionally starting Spark History Server results in process hang due to deadlock among threads. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? I stress tested this PR with a bash script to stop and start Spark History Server more than 1000 times, it worked fine. Previously I can only do the stop/start loop less than 10 times before I hit the deadlock issue. Closes #25705 from shanyu/shanyu-29003. Authored-by: Shanyu Zhao Signed-off-by: Dongjoon Hyun --- .../spark/deploy/history/ApplicationHistoryProvider.scala | 6 ++++++ .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 6 +++++- .../org/apache/spark/deploy/history/HistoryServer.scala | 1 + .../apache/spark/deploy/history/HistoryServerSuite.scala | 2 ++ 4 files changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index f1c06205bf04c..472b52957ed7f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -114,6 +114,12 @@ private[history] abstract class ApplicationHistoryProvider { */ def stop(): Unit = { } + /** + * Called when the server is starting up. Implement this function to init the provider and start + * background threads. With this function we can start provider later after it is created. + */ + def start(): Unit = { } + /** * Returns configuration data to be shown in the History Server home page. * diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 5f9b18ce01279..dce9581be2905 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -200,7 +200,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - val initThread = initialize() + var initThread: Thread = null private[history] def initialize(): Thread = { if (!isFsInSafeMode()) { @@ -384,6 +384,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) Map("Event log directory" -> logDir.toString) ++ safeMode } + override def start(): Unit = { + initThread = initialize() + } + override def stop(): Unit = { try { if (initThread != null && initThread.isAlive()) { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 878f0cb632c5a..9beed89d8ed25 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -297,6 +297,7 @@ object HistoryServer extends Logging { val server = new HistoryServer(conf, provider, securityManager, port) server.bind() + provider.start() ShutdownHookManager.addShutdownHook { () => server.stop() } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index dbc1938ed469a..71a127bd4b9f6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -94,6 +94,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers server = new HistoryServer(conf, provider, securityManager, 18080) server.initialize() server.bind() + provider.start() port = server.boundPort } @@ -451,6 +452,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers server = new HistoryServer(myConf, provider, securityManager, 0) server.initialize() server.bind() + provider.start() val port = server.boundPort val metrics = server.cacheMetrics From 13b77e52d2315824b63758887b12b84360aa911e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 14 Sep 2019 00:09:45 -0700 Subject: [PATCH 0067/1387] Revert "[SPARK-29046][SQL] Fix NPE in SQLConf.get when active SparkContext is stopping" This reverts commit 850833fa177ec1f265e143fc383e40ec2c8341a6. --- .../apache/spark/sql/internal/SQLConf.scala | 3 +-- .../spark/sql/internal/SQLConfSuite.scala | 19 ------------------- 2 files changed, 1 insertion(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4f3e39ad49afe..d9b0a72618c7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -139,8 +139,7 @@ object SQLConf { } } else { val isSchedulerEventLoopThread = SparkContext.getActive - .flatMap { sc => Option(sc.dagScheduler) } - .map(_.eventProcessLoop.eventThread) + .map(_.dagScheduler.eventProcessLoop.eventThread) .exists(_.getId == Thread.currentThread().getId) if (isSchedulerEventLoopThread) { // DAGScheduler event loop thread does not have an active SparkSession, the `confGetter` diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index e698ac032cd82..1dfbca64f5778 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.internal import org.apache.hadoop.fs.Path -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} import org.apache.spark.sql._ import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext} @@ -321,22 +320,4 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { assert(e2.getMessage.contains("spark.sql.shuffle.partitions")) } - test("SPARK-29046: SQLConf.get shouldn't throw NPE when active SparkContext is stopping") { - // Logically, there's only one case SQLConf.get throws NPE: there's active SparkContext, - // but SparkContext is stopping - especially it sets dagScheduler as null. - - val oldSparkContext = SparkContext.getActive - Utils.tryWithSafeFinally { - // this is necessary to set new SparkContext as active: it cleans up active SparkContext - oldSparkContext.foreach(_ => SparkContext.clearActiveContext()) - - val conf = new SparkConf().setAppName("test").setMaster("local") - LocalSparkContext.withSpark(new SparkContext(conf)) { sc => - sc.dagScheduler = null - SQLConf.get - } - } { - oldSparkContext.orElse(Some(null)).foreach(SparkContext.setActiveContext) - } - } } From d334fee502fa85cfc6ef1b7f193fa3a0ff0b244e Mon Sep 17 00:00:00 2001 From: Pablo Langa Date: Sat, 14 Sep 2019 10:18:52 -0700 Subject: [PATCH 0068/1387] [SPARK-28373][DOCS][WEBUI] JDBC/ODBC Server Tab ### What changes were proposed in this pull request? New documentation to explain in detail JDBC/ODBC server tab. New images are included to better explanation. ![image](https://user-images.githubusercontent.com/12819544/64735402-c4287e00-d4e8-11e9-9366-c8ac0fbfc058.png) ![image](https://user-images.githubusercontent.com/12819544/64735429-cee31300-d4e8-11e9-83f1-0b662037e194.png) ### Does this PR introduce any user-facing change? Only documentation ### How was this patch tested? I have generated it using "jekyll build" to ensure that it's ok Closes #25718 from planga82/SPARK-28373_JDBCServerPage. Lead-authored-by: Pablo Langa Co-authored-by: Unknown Co-authored-by: Pablo Signed-off-by: Xiao Li --- docs/img/JDBCServer1.png | Bin 0 -> 14763 bytes docs/img/JDBCServer2.png | Bin 0 -> 45084 bytes docs/img/JDBCServer3.png | Bin 0 -> 108360 bytes docs/web-ui.md | 41 +++++++++++++++++++++++++++++++++++++++ 4 files changed, 41 insertions(+) create mode 100644 docs/img/JDBCServer1.png create mode 100644 docs/img/JDBCServer2.png create mode 100644 docs/img/JDBCServer3.png diff --git a/docs/img/JDBCServer1.png b/docs/img/JDBCServer1.png new file mode 100644 index 0000000000000000000000000000000000000000..c568b199353ae9cdd2021c2e61bb030cfc51d1f0 GIT binary patch literal 14763 zcmbumWl$zTvo6RCE`#gf?(PnQySuyVi|YV`yE`xL?#|%u?(Xgmm$~1GvwLIz?AeWu z?&!>_&Z_9@$jW^B$xwM&u^%u$VL(7Yen^N5D}sQ0v;MmGhl2RJ8W!GsekuM~2noqc z2ni9`JKC9ASet-=P{ex2@{0Eg;PmS$NvTl5n<63&%V*Mrg>hJ-g~=1n!BlXiMEeNF z2a)vmE7;R=Jk_W>+dC}GcdH_1K3fBrxSl0a$4SH9{^A0Od3u2s-x9n5uQ&5?u z1dh6uGi?}_CzeUxW}OAbW^~q$Ii&@N>mx}a5(X9T% zS0bZ2(qw=C5qApt8nC&p`8*6wPl*iqi$3a>5$=zxpXj(lXEbh+!*?*CA9crm)tq0d z+>8}1tq0hhEhxq!t;_}}o$K!pav>&CHJF*b8~3gEh)zn_yY=tuR=RfM-Whx#7AI8h zpg1_+iNQy7AKhKeJncTSEFK;~y*eDrFtA|Ozm^5UP)bY~4jog`#LAa|j`Q3*Ky8V%@usj!?x)SZOvY-~(yoj`;fO$?k&j0s#V zoXiQtBxK~({NOP_KnOr2gawt|R?ahB)s+Wd2Kl8lvm9u33Np5$caV?3XA2)%Z{7M*r7_Xzlmk0^!nEBLSj6 zEg}?{|91oVDTI*A<*!cNz7VO>WJLdZJ^@3j`Bl9Bb$c^IMEZXi`rprf|GyjhRSo{n zDwx*SkYN82{xkmnU^4$@5KJouRI)O{E+{@8Wuif=hC-NDn+Oq#4?EyWdinwzL}M%FlSzi z`5M;9GXR~)IBXBF9aKQiR4nmkd2%ozj+mqA5@c{f`_N?%fQ%d`FX1R%K~qDkl-ZiK z@zi*qTboJ~d_$~0UkTYIR&=<-6`7fcQ@Xp{3OatWY&hl{e0hTnwR@SRe>#!3){HA_ ze8wH8Of__s9+?R`kKI_PrA2=>-j|y1|Ec?Lbw5@7-=POK=L&AeN}k%1sj82>!ko`* zz$%ZVIo;vo5Nnb(cF`U7EF?GJRnp~K&fN|o?flNkK|{IrNp^V$0FrSGbPECV!NE!@ zVorEC+CyXDf;{3AZIZ@;qf<`GAhRo~!$i%`&<{l^YPN$*8hIABc5m~qbnJ6|4wmiw zN^(4+iIWX4{X8jmqlPO9C@4~XA=S4R5h?P+!{CGLk|1$JLhdoyW1j#B(vlPF=oYXc z<@|v`6f7(eFVRbBch7hkp%-L68#MkV>SQZURh9JV>e^Ju-@~Xqmf7VyLBNA z8k876J(PPkX5iusp*!4?n76OfvbX2mlcLnc0<(R%KnuP;er{|Jjw#y@rWKPnJhnF8 zjt9Q+M}T9%3!F9^IL0~@18b62TM`rR6$xWW_JZ7==3kN|)sM{W7AUzE^xo8^3Ss!! zPl6!VlyNy2D?Y-A;v+1!iV#&N^0tODA4( zA7MXEpQ&8%pQdMs?7#nOFacUJ(gW%ro)drNOLyC+^4n@Q{S|xs)sb!gGWZ4Rql(MZ*zHjsk zY7#Ccb57ZuQ;9Z=rJ*DlNEQ?tX|$A*KVTgc|JdPZ&?lb;ch?=YtOs@`v38VQGkWHD zc!#KP=fUbCeF|>w(=5@8yCq_^ta@-LQmu|nB&_%-VXhi65Ep_o_?{cSC{Jvh?Ox*T z9ftZiU0wH#uS2ub<7+jc;aVT$#UOM>|&Qg>6VjbER3G%-oKITWWDHVtfw~_*9Cfs3^WZioVDj%) zk(H}V()YPo9eCVwQ`1hV&xMfMa<^7BiWH**uz-Qw}Nrx#Sde0{Se2U0Pk~Tv)IvNq5C|Yekxi;jGGt zL;6~JGZ73)mrmQdHE093FJ7+UbX+*CxFk~r&ej3eEuK-cg9EktB4(vS%SlId5#H5U==_UJ4ZBIFl~-F_1tJ@O(b5(f*Xqgr;H3#k%;x79i3Q_DS1Jr9o^2 z)}GQs)GhgzcKt=~FGn1N$}_^Z8Ae%ejG?&c24*vTN2LgUS>wBSw`Vtmx&iAQk5&?8 zS>Yu;iTp@PI@-+oSi>}$gqSv9mx?SFe*ZJjto9_fFz>h^F@O56=!$Ii0j2AO$v{NV zKCeUFm?uL_az(fnygzXXKJtq&0RKG>UU`gyoc4+A@dp#2%~^}{AS-?XF??lnp_e8P zzi=3c?D(kDO|9F-#&?mbv2!;hZ(DGq@*3Kba%I=L80+X4I^1GdL#MlU5Y3wK z)U>y??oM=kAe;n!>T3F6?bF-p*!M`2A%2)fUF0U;&j2fnAoUfMd_z)hnz4vKE1^)Y z)FHK_%LeK2a(ArZs&-kFe>pubZchyLGSCl8-!Rcif-Z+ZL{IhtNOe)Lft(cqonAR!qj9TT`emXhnZwHMQ4%0ws7r-$ zD%M-_8KMGnHSO-CevxNow>r83$+uSy^(|~!-t?@nOD70O8Ozw+sU2A7dHJBMW!yXa z>tPU))Mhn;<4O;2jww$OVsz^sQ~(bSD+uXyQ#e>Jx#mgh$^t3j3$01LUdw3`;z{~N zXl>vIe351_8V&F$!(=?C!4pYVXKL&t7hZSAWN~KYbu@Ppz`}!j#|kRaaUqKddct1x z-X84x-z+IDiV(m&) zg`&xZLLu7@z?LvI&9IU=R@}SnjVOIjRKu=NQBtqq@}}Ka0in=#vnOLCve-EF9Z$jm zNt@@I95C>Z?H%4t(-xB&o7B_sQOf7jMPp4&O7e(@Wp~W#z(TX^ z-eR0sUcfnNa$TJG!}lj^-+w|j72|Nya+VOH@rQ+;)GXSj9=Djd=+9n)0-}Z!ulRl4 zz)YRu0{Yn`cUsCrF6x~Ppu(?mkw5SA1tJ@y#s_$$oMeTjoVv+2RzI^PWrzrw60ltj zp?I5urvRQyxphGGARK1S7(yP4%LF}ZGvzB3Wh@1yXD$=xKQsUj+GA>Q4{|}i5>O3A z;?GLi@R=)!b@ zn%WYNA*Lxd+pRYmuQ4N1pVt}aVXicL@`qr!?GeZv(k`pcY=0UcY(d<~Bm znPB4Cz3N6U5+KL!S9KQ~@}-$=;;d9tws2|&^WPR58PJff+895dRkvKAGk;A)Tmo$Y zB$oS1eu{YRHN{j<2HA=<0S}dG`a`icE3>n{fmJ|8dK#@uPz{ey!uN;3TN?U%hXZEf z;TkHtXYec~WVAODK&i?QM4Mr4)r2d!Osu+!Q-4t|HQzeRXKqOAn4ey9^;?Q*l}jsX zH?T}@_PmG;vxr0?#(?BGk!aADy{2}=VO>zJi>+ck48@58-k9nFr*)`Jm6p-dJuA<3 z<}V0b0-E{Eu0^N>E2vFAn!^qR{Wo4o`5H$?+#H^X{*?Avh>^B*OlB{nHaebZ zzPdEAHE5|>j4kXV|ed{EVak;6F&M|giOORONSGzmzwtW%l&D~ z#~Wv1wq6rsu8SJ|`GQh^13+6Ma1mp|0wYa^x?xbvEI2w|$!m2v;XdZ35brdwg^2 z9)_&<)DnqOx_ndSA_gtYN1J!@;$O5@XBeSZr~JrdY?)3SNp)LF5}KNtF+7LTQ7-*k z2e~#~WO^Xw$AeFdZUbqgqfiy$^@fiPsvzG&f8nu=4tI|#- z{S5HyiPSsN+4ueGtL5>`z<%{ptBnywFws~rHESi&_nu)N58h>aYUE0XlHg^i{}t$p zB_z7N-h2M$L3SHXO!k2F+8;ibqd@9?ezw)UVWiRdCtnu9boWX@I)g8m z)GxgSQSVAb!@G26Rq>YO%bQo?YmTNOjst#{o^G9IHax&AS`}wH6kzf0txddYz(y>M z=EtJ2vc0=d0R?5QJ|lg#;8*_^px8)4TIdPiU2Q?F=vImL8KxNm-T*Myi_DZGR?!{9`B?|vireL=l%$}O0 z8ST(36%KhK5MzGRFgc1WX7<>6xnUdny`FFN5aAank4_pJX9uYTHN!_ zvkkT)rdriBsgshMztmv4r}H+|`$agrr-3kbGVDi6N~tsf15o}-C5B$cj?w4R(f>ZG zIN}2~MXZg-rh2)eu}1?SV@ZxNOYL-lf>K?Rx9dMY^Dl35g{^$Ks__379-+pRxH;+L z*vKX4YmQ4UX{HI5`u`zET7!-rUgJvD3@Y@&e+*B~G5H*@#;d}5+6<&M$@fL}Zd>S) zq)!5z`WQ$JEbhyU-evdyrT3f#3Q5VG`9^Pr^0jk${8qy^6T4rXy^E)JKSr|{`^{$6 z+$hGJy|K!ak#p1SWFrrwv7_?<_AzqIF>cdDL zIl2ZG>wUP5(^w>p$tr*n9mUj~n}G=pT49^S%TlftoRv$?U;qpu%uIM-`;iqmEk`Z! zd~?7l8t$@5XP9~Xi)?(sT}#gqNAA+t*F|I6(kTn^@6o_B&2~rB)=WW&_UBuqH@7Hc zRAFG`#@D{)wVEQx8dQ-U&f>+De%U$ya&}R`adF_(;e(|)ragd}`CvM2jSRT#GTg)O zrZA?$)6?$W3kM1h_w!)34Cx)FM}Z6JbduUoRT5t*95nqhu|%uQFP zcWNLkvG2OnVmkR!!{@GN@-FM86B|o^t+j~df<=(s;6iiaD2>p#&;`$Qc?ecMw|s-Y zZl%&&?KoN8as#>9ftN=sqcy_Frd5`vUd@3`sVB#JPvp7_-g9q%ES$yHHIBm{X@6HbQrd4cqBU4RJ` zvh}S!lWUN@+Gml{b}gEaP$F<`4mq7^-u`?lJBZ zFG$$`g{J)nTKhk2{73jdp}n5}ISluq=wH;>Gdvy1I5_)hmZkB2z#xPpX3rq;qBR1W zzMfSH+be8@4E1pe#(j;hRfLcq5Y25XE6dU1K3%v7_R)-3L5wK~=XhhfGVd!x@>%Kl zffg4f_qXf->Y>Qz!z{lZ3SJgl9GjlK!q+9Ref-UH$HWJ3J@mj#)5AM(HgS|iLuAV`Y zydqAOr296jgN<$5!AZ^YBVByAm3jUvWQcE|{uN~Ykv<tH#JhT!wxEsU%l^no)lqFF;2uzh)(U zTji|vrJl{))p+04o;VerH8`|W$~Scm1C36OG@8)>cvLQTZ?E&u77bw)!msra3%qah z)ImaAO|3*HzD8HSKYpZSW%NAVM>E+y9iXe>sSUK0;*D(W!=-y9-w^u}A1%p#Uy(S& zJ#=P3*25pkw|0xgoor{(sxvR_!Pxl4}65FVNG@t=9K$O5zIArekfP2_BDvMgUltjFcM7_ejJlCml9gc1;C0%5|=Z< zplXLBx|{2<{Aceb$1~xaEAB$J5w)(q*Zl};MMIn5XiUpS*z5Yi)6zqgC{y3%+*|S; ze{i-jm$qM0wWz-tzvRXo0rH|`8kTc1u?g=kZONZ1Np(rpeE4abj%TwK$Y4|N!%3{E z=HR@lkWWU8xn7K;n#4AD%;jHcVvFLWjiN=hgEmA{QPWi4GeWgnl!7(TGOl~KmfG^J z2C+b!c5XKujeqly&gEoJAIPY$ol9AfAsy0!B>@vNp;E}v%4q%RDUe!aQ$i_X(d359 z@2&QYnDUjt)4ZCTZ+Fsa#huSN0ci8RTa9ojt_55*h)H=kc^MtGhnn;^68z@4;q{t? z*Wufbf|Zxq91p^nJr3urp5{(g`64}ol1&`=b=Gd9J8QD1gsto20%=`0nc2UQdzcbe zCshsQp1M&3xa-9$9_kN{A^L%v3~btNavjrkshk^ukBiAnwr(+|T<_+%dE}S-28LKX zZ$9}s3W#OVBnNmB`v%D{uo=O3g}Me<5F;@4DIDn+Kf0db4wG~C%RxN2F-CBXcJjny zwB&13BEEvoEVPQK02$LITlJQoCUHb=SCYS?nOM$uWUq0g)N&O4wX?9IN2>o8y8Vpp zW?Tv3I5R-AoSPQAG(knGl2R;4b2mj2p^~q-lfnz`04*?$vdafR&Kf3R4v5GO_k$rm6xBonjdKB-Db?m(NqIOn$%A=tD=7%%I!ki(QV zoD0y_Wfai|W~R%N?lXDHio}1nEF*QtSE#nHJw zpZeD1IH0a#7u4nq;`s|kp{EROW8y`8*v{gJKadN3-eTo1ULFJt{8@*X@DOg+uC%*R zq&WruoWY%6Dhtb4$ORu|@K(c_XOisn%RAQTDpboDZZC6kikBC=n9<6P;}MPAU5?=@ z<=Ckb8tV@RpQWfk1I}oJTGVi!^U|$t5z=^mD0iz!!0ACV_IIblCV-9n@HoS3h9KQD z9km%=Py6m93uEFn9Vb$%2KX317Sy;gjBb0}U(uvPefcIIhlMlE-V#-2bL#@4hc~;k z_FKF_lkXsxRTe}$B|!vAeYr8xl$zR2(#$mDCL~m-#p!cJrPE%G)<5!Gs!CHxH%(So zl=_OaV&Bl`$dI(#iSlOlB*oXhcJ`hzno}jM)YKfWtiRj>q|ALd_D4*K_@h`z4g4|B zz3;ILwx;S8zrfSS(d%&(UHM?WrHr3)Nq)ROd?U!*^Ri){ytNfo zzB?A_kJ!8^Bi5*Bw?TpGgz1JLK3kbI0-AAP(Ch5n^kc8VcXUtNBy)BGFl;a?jN-)L z6^cW_MPm=u-){{MNJ`{Ol=VRdsK6yNM`2X#yO$Xr+sOH%NOd8!GbM>1OHwnnDt8R( zf3~Hl$LuXFP?ep}B~oDoRR76~8)Z&YdsNAelY zDnD;W7Jw&b92U(TzwanHe<~t$NQ9$iMdRW20pHK_a)_K`Mg(~mR~c(tKU5d9yl<>%3pUJT>5Da`I`3iR=OvmOhnc+Rr)Ga>;xp9fO$uC zM@;04K+|HewzFTJ_VC2)83pp=Su{9{=%)$2(&x9>vCGPRM|Fh>B02-L#sX-#CJ9Cw zs?oum50z_bDSWGy^aT4Fh`Vp9*>5i2tX>vpM6C|U(8Xb7=8ACbH%N2kwd8ZvKg!aX z;w_!tNkb|n!C}T27{iRjCY@6})pN=G!@E6{Eef!z=aLpuhU47c6~Dhmn^1>DCNqqO zh}!TB)IOgvpu(U9nVMqiWeHgcNzsg%kN{i8>ZmH`n82{e)YC@XvgJWU>L2>3Q880B z4=}q&s`JYmIK!UY6te}6{XJJ3k(4Y5te9{560Y>kA5WetImDL`R7%qJ`M{fK>2J#N zJWu&H+5EQfM-2w?QY0bq>A51ft7cOoDYOc;C zs1y5>7W;e)6I=Jy6Y2&e{qLIOb47nx&K3Jb@}QwP!i|}zuc`+~_unE$?v=WaWI|Rs z(!K)D z4}yqJn8d1hA&HsCB3L+2r3?a#729ZPle^c#DCIgLLe!~t1n!X?{;bW%X0aOx_azmY z>HzZR)O+CH`@I8kn39&nS;*}x~yP2uWWv(!pSj42fxm*spK(KvR7tD<1^ z|HT;4F2f#`2_8V{xljkE6ShC0vqj9_3HZ2TXJmp*IcZhY{oepWN+cb|bhdguNlK5D zniD^Nhk#-)x!Uu_AFTh){@Nte*f||e7%v^COXNn~_1eA?`qOyS)+9<;*J`^}nxOlS z*c=oW#JumpNm)xpy?NOh5n5e=dSnfMv}i#(nZj9Wbp3Z*5kXc;R2B!S}MVTWhNkGPz^(Xl!rDEWP{`M57b?ZBfh&9D3 zIsKtaOO}(v0y!rMGZDm85!HFyGE%!2o*;tmH5FY#c50csb_FcogGccXJ%IH+MZ%-tV2;(KE#h+2QahP9g>-`HpQ(}HUC?}5K5}nA!;*_|S>0Qt{iy$GE z>=cW0+A8_+8dt{~&P8e0h9?&Oc^9W{%)8kTVpX!e48V%Ev6gX;j{bD<-zZGl?hx66Cdu62mBJQ zq!N8BWrP1)}*w@3;AU740NVt(A(m z6TA7ij~k{hN@%3G>F4Cm=0{YV3(kpNtr0QbC%cAfa0?{V@>6YPm$T}88nR}B4lN~U zDf3?;O(nj#HY93lYQy8ZTEliPy2r6hx3}?ib5^>I1>%EZEMqmYZ;zx>mJ)=xICJth zc&>D_e|b(Q_*cRF{R!epB#?WP{;^OHiiH1hTK->U!T-f}@qd3Liv7ERNB4{edk-Yi z@s6vxKiS^UM^6@MdrEd|E(VQ!y&S*!MdEkRJ~|Kp9~uDL`D;SXTy(WYt;>V^lh*E3 z38lN#h4+b<>yfz)XVBWnd@<|JiAr$Xu-11CiHR4oz3nkK!UR{+J5i;8gd>|c&+guM zbYkMU6Zw8qEh=v7w=LX??ckqN{^^ygt}-J@5(n2)FZ3$z{!B3FTjY6*hh8^PFD-bZ zt&t~FIKFGHQL~6~?z*=*^^>>oN4j-8GC%y0H(p2#;Y_2&(GjO&wKuy*g}430F)j4I z{wud?iU2?!VMNvxhgO$7O83Epw?-b`$b@f+49tf@Hw1HO8+$*qEQ(ZLP!=(dU{9;2 z5S8Yz08_3gCw1u}w)B0KX{ka`VQ=uv9d$*q6Q09e6-F`4XWc$U?*ikiI+5e>MOSS% zIbOyS>{mgq*DKoFozd4eM@&jw{t=~Xpp*I2 zcNQRgrZt@8y8GeMos?x`wl>9!ipP;SK1$u&i%Wyan`*JJ`cxrzOtZ+h+8QH6Kl`ke zLHu;6Nc4qdZl{2t)9a4q_M;pttbnAhfE9JZYP7pCh9{5ZxOUXjP$G@Dgxoun@ucwE zT^cyOiZ-{o&05Xr;&pV61`*m^I|A}d2A4BtIET5957@{|GU|>FGT%uqY5FqFhULeP z7i96}M)^LO1`|TK_Mv*MxVAU;*(-IR`l?ij*R(s1SS{_6P-=*y>8G+n)`yzd8_YyqaOz`bCrRh*{4q7lFt~L~-SJy~ zvuOhR9bBI6WuBvK-Mn9k!(~Oxg!xlH-gb8sU)Y@Tc1Pwplj55_DP(13ozieJF2^ih z>j|vep5)nppwYAUsB3Es{!v-BbAbE8%XXy|kJuTaCFTx#oAxz(!1xIUD^j`n>TiR# z4P*2Nb2J^x&%YPO(9R=BqH|~cJH-6cO*z=_A8ruzvJ9>QVqaM@OHs$;>=P}y;U-#P z&Zpaeo65k#OwfwX>#oiTDT2QqVXSc6ub#dQc<=;+3bwZBKUxaJr8BCq)D z$T;3nq(W`8-GkxiIs*WX-Xt2z*wNW*Tdwrp;KSX=%!9h~7YIk%-QRm#B^6g#4vJ>k zKTLg$T*>gI4!3JJ?sV}5=Pm@@STn~>Y_zab`iowfy{KY^j}{!9zOwNz^2DR*dTqUu zPTX$dQX@`xbLWrhuY|SpL%@Tbjr7LRElD}*jwz&|EZd}QWf?)_gGBSuZWD){d_45# zoeaX_RZf=;lmeKgfAPD~t;NC`ufnMAzz+B8d*~e>4bJ-LvhpsfNQP}oWf4$9h+K`D zYOIO=I668GYBQsLnSk4Aku2&!&E|dYLOi1viqfVdu8;>y#{d>7FS5-gbRBFQ12MVX z2)og^XA1||+auPbw!V))N6n1lJ|Soig<|X7bo@-Je^E^RjGcJ2WMA`j1IKis z+9bnUFHjeyh|#1Oa)7;w(Ru2UhpkK~8d|1WCBV-_>CFtGpe8ZGOsj~=;2m6X)tBwZPFu#cuRd{z+`$@ydjN{L-YCtd_VecB121YeIB`%8 z;AX)b=R#{B6r#DCtU>%l+K#T|`qr~NCQ#3d*@u4xjr&=vzUru&mksQx?zJp+j`W#bSH!ZZ6{5kI|3JKjA}=!tEcDZ z`+C4~bC=V*1ro(0F?`QSnis6SW;1bB(Z=Cq41?CA%4pfgX(_kbH<8zdsqYhu)}=dI z(@{hQ)HhQnMz>z>&q3+hN^A}=;m*7D{($)?OI@A94y4#~~rpnM}{X2Q>pvDGW%-ZclgLbv`3@oMR`-Z|y>#iNTY(DoN>XzFl+N8m!vd76kyRuALrf zzG5NdHQs09*UvRM^~-Gy+1r>9;x`8vrrEe=5BJ%g0Ze=hH6paaCG`V3O^9n!^EMNB zUK?YQbiVDXV>5Sq(hLM0Sb*U!CGtEv3y>w);2uUWPcem-SH$oaj-67^vNOd`_iZ+~ zl&Opr%i_ZN_EO2~xfrCfLcN z)IF2C+o&cN+OR@i4yc|D{VBnvI6aYxG-Q(0hTf?Hb%ln06(GlsXt;C3>FEQTjZU14!6inY$6?7@$^vW7s-hXO+re^SXKcWvoaCZ;mLR@^l@(J%3>am_gbAbsuzQN>rDn(VyNLz9ldNNu%M{bOLf@+660o6 zNDN|f6lX7PlGaQlpZ(kIsF4>9!skewxCb#IJNtY{hX^i26@RM1ZD6|%ol8i3wB>G7 z$+yNrrs9=Qi`#G%C1r$;soo4X#qiE_Y$Gc$OwOqWUjW-Z-%#Kldj0I;oK(dGZZQ!g zLxF9;h3c4PtsXjnWl$YWNhm`x5Sdg#j-|*~PvDOM1Ef0l}f>HfmRBo{*Xs$ zg+*U5)Kb~?KG*vZUYW`NPMfXt2xFLMy+1#J^#HjzL2`~~w|f`$E-|rKm5%sVbe6i~1fPgFt z6YMREZRK;K-oeCHX4XN1Q%v5>aO+V~V!o;Os#m-1pVU-^tR5G~jmh@Y&Ir5*5AkWB zLTo<`Wg#=>=>BiXKwU2y2mJDDn#7hnvRP_BnQNAwJTxGCbI}*hSnn>noWz2)7U|vV zll@#!+T7x{eY`7E=h5g+cOg+C$I0T%LQS-u+E9qH|II%8u+aN%u4RHwAC)~l(roki z+O3{0Bac>u37#&=ASAXXqKAtTl^8&5Ou5(0ENp5X6ew{(K!7zo<@pNkO~#jWl)z#8 z2r#`Q_XS8sh@CH|c2|CkT`YzCVAXc8I*PQ@qNhq9zR}KzT;d9JC{yujvP7a~HZ?p! zPZ(0`r?A8Bu$T6I zA~MY;5k21FlOH~M=Ch5SY?h>}B!P(ylng=Ww5hH6C@GX>(5H@SFneT0! zF4tT+i?+4CvnEk^&Al_d@g>PN*9>1id6pAXD`S9JZAa}E`=av0&q6F7aDSMbi>>fZ zJ6zR8h+Dj1fo$3UdGmTb@kftmbh>U(W49|vrXso4G#3zM#v(cFy1d7p#=Ge?nR}nf zoLj9;p&VD6#8gR24iflz(5PEY%T~%$tH$)sbhp+VZ=m8h;n?_QL64`v-r{ymGx#`p zIH>2sWO+DQ-;lV^A)p71IZ;0P0JqE%fWBgtu=cr6-edXWaM1$@sO>;PYkTiQvKtp% zGmEw{l3e63=6`fOP@Y^p*aIbyiv4J0Q_Sw){@z&hIgR5Y$e3%)iT(8_9%w5V`IEI+ zw-*mk0JBH7T(3eD+6*b@rdCLqS4eBL{xOS#se<86=?|-xU4?*NEAuC-rTnyH5Dcg30cSml%6GZTLH%`W#Mv*ba&Y;5 z%v?R;PK~|gj)D0t`7WUtWH}wG$7l{SWwAgfT{D5wO8TU1lArX-h!dYXSFvQw#J4(5 z%TxCDt{)G=!*hFoUnDQ93sLik2ax{E@ccL3-Q+O!3Rz-As5}7%oYGN>%A>*Ts6O#5<8u6zx2+|07EmNo?frxapOijS3akrj zxcfj~HKm)SL-whsW!eD2rXKyYzfFpLAY^7rE+ENvquxJJ!p;-U;`K-_=osu$H2>7Q zmJLnS(eY}H)bCE3f0KV<E=Ng7a<$y&_`IN*@v-@tpfW0@eniZ2m~gdQa%p zD_v;2Jebwed&H~{60f>0c7NP&M|Z3A*?EV}?yNv$=7NE&WVW13zr=qVpQ|c{T`y#!uhVHC3LxrNq$G$ zFi@!&_V9%|v~NAn!PT|MEUHa7!al1docPtH7Ep9x zb~{t4EI&W^IXNC-rJ66`GrL$OR%TkLt<@65T}2{KYJZ=6uT8s>kVF?4YvAWh$(uX5 zxuqj|va|J2Mq%7SZ%u2y8f1VipZ5(pN)l&ar3~BXcG-m=G8Gk2}L8l>6$5C9-W6#F!dYh+f^v zw6y6ms(Z7JnhJDX`jcZY#F?G61?q+wbc1~}Fm>O3 z9R{EdDft-<7qrQo9m#bm2TL`TM9Z-v>vBx}hK#nLqN21!v4M%|iT2s|zaQp!e6r?H z|L(_{L=ZwNm6y3QIS&7(37<$4r;5@dzlgtMO7N>_GI>sS9z{C? z9!pP}0Nk*vBQ!QENbr7B^kIN_ejTYV#>T`N`6vbKY{639_#oVli z@<7eGJUMR9Yz>i(9Ei0v7fkf<8B1GGuUkmeldNpSR|@7ao5*)@E0!)<_5$<%xoP|i zN$T-}2VBqkKpICeUdh<=@c%DQf%gI*kl!<6jR8%DTn!GMGt&#!gh%SwsE z!T$C50V+#*-GbsMt>pp-$3p$D1>Unn#Qk+6va5`|1oAG*TRbWfdaixG*GM z##){E2@wIqAPc`K$c@mU?>Xd2h^~(fUHrdWGInG1m;YHKrInQU&l=6Lva#BKR>YG( z{QrHI%yoacC|;1P)Sn6`=4W5uPkJ>-@*-qCvAG#AP)U;=QJU<7QnMptDH17bfDe7? zbYT@%*T4zD^ziHrGfu-6?!UgHQO{FZ6uYO9Lg5h)J--37x*4nC`o8Wk*k=So5m$Na zk?LJF1+u^2_LZel{L+U-8lfdPQ^Su*e$XCWTwncDY)<>Hv^s5 zCD56kIH1FdG45=>_jadJK=Dtm6o%mCv|Ros*`1SWQ@plkKVge?S*DQ0>h7x(#Q?`^ zuTfU+$TSf}qmAyiIQcJHMB?@J9(u2xDW7rXCd5(UZUM6w#@@Ire?iw0y5a;3qpo z;79(L)I*trfv7#bm!dqr>YbQ)`KeI;d!a)(POBZmu1}W?5EG5h?B!|9u#ac+@?{C1 zEK272dSQK*7`*OzP=)rixHwRjCUHA`L40>eOICFAr_MpOfyU6UG?n)LhmHDLowM=I zA``lX7$wzzH3+Sh@tnI4HSB3qxGGR1%YR&}BZ~{I>xVNxu9L!{CWn_@uy+v1`J1cp z;|@O4(=I8AIX7qx*V&Xi#bdMgBOR%maD~L*H~GVzO@Ec%@*oBuLWe27!97}!~ALx7}6^({)al*L2YH|Am z2q(5uE3h|fmip7(7&SZ0KDZ#OQ#WE5_h|sk1}l8Va)8Sb8?uVVZ`t<(V&F|mubwWt zInIVen;6o_0~=Q(E|-V>0#2~dO2<~Uu!$f!vQjRt$1GGZ>iH)bGo$yD+4ncObU*Oz zQ0bm$(rO-kI+Nyuh$7C$AN{)#^#5^_^F!KXS2H7QUVPRl#pS=gOCWf@^&FYpUzqaG zGw|ZBw^7%oa?R(-=EuADvczl`1{ZK zc2E~(I_v~IFfwVVC5)8OrKJt-moCt7PZ(?vH-mLo;rW7-v?MpQWSQ{&Ms z$Bae-pMyt)q=sdE;#davQ8=Ld}LWM8bUe)?Rs4Es{w7nqV zg%=I7GZn*lyx7XV|IO8KW5V*gTcu|}K6yEs2TUIt61FC7s^{(K!+4%!c3kJqS-uw$ zuuRf)@46y(ci<#;jH{|JIMK&MbiW~XvgzYQtQWm9wO;o@QQzq-=& z_rOR@AQq_-C8lv^RzGUqse_!p3t^c z2Pg^V*Fwmc$nPQ`sR&JdgucB5gG;|ewKqL&BYUSxqx2kmZPG4dp-n}v;ud8qwNDyi z@&?aaQHP8^C>0#rvd~jX+If7R9cU4xU_$#@Cq8uOFy!!IC&~=nyM^1| z-(SssE(y@VV$zRP-=v6U{qV45Pyon+)XZExxgqlCgNt$P)b=puoL%icMJg}F>))?Q94Adys`w&EzC zVs_bX{IQBTpBaRg&lzPrl!$c2+SV-8ecClbhgN6)ug&`5Zc3u-e z51W+pb_;gwnj?JQvOklg?fcmk&`?hn5QVcrwb_}0T`JG;rJu$1{MS{5P>5Yi3rFkz z5IV3s>jxV>`~Fx&#xYJ1X@H64SORe)3@o_dI@2AvMckXc>(pG5 zwr_?O{AFQNJVJM5v`snay;>WM9AX({l)N~)96rWmnbKu>v%F{ThzOV_6D}q#-w$M| zg10y0y+FghE&iSs^kpF^K$B00ih};E@~yjPRt6Vg^CIkKvtd3%CcPmz7}-kQ=^_%K zGlr&2#L8vY)gDt>i7N&9PnHihVwrJ+7mK~Kfts^M3pE|k-U*|xJKX9)IQOP4Ezxv3 zpBo-i3r!gmb8vBr6v+4Pm}6U1eb0t(NoZeWmFMPDx9)w91U-mHpqZ(~gWvYT?8cv6 zd5e(!Mu$V3T7CF&{Z7bMyPjfec{aZ3#?r9G=c^I0^vuhX@*cd4kEgw21+puj34R$2 zw8o1JSt9kx%d{oL{#Q%dJEZMS6V68Ch`4-7()9VdbLMR7*~`Ofr#%_OE3I6PrfQ*J zytYJ<*?wu0o$=`_bi5!3WC=HYdT%ZWbRgAi_+{8Te&s9V%sz{(PM^jXd{z`-#A>+v zPWHP&FWI3&k+RFZ*(vX!eAZa%)XyvJB;Wo#^leN-Ox=n0Mkh^Ugri17I+0k?p?)tH zV*Qp+Z_zriz5kB9=s4Kj=M|8iu9n{qAV6~fVaG9qH+t(;L5>Op z7QhU(vCPjfKW>nHb3*-WT9nbd>J zA@kZA=+pH^%QzWNHp=XiGHqDYQs&t+^l&{i7`HW(ZJcMhdo~Gkmb=01gg^YdBc<^G zL{89|R7wX$_{9vhNGi@`CC5M2oz2N<*<;+xc!w?9q)6)WsZD@a^V=VngNyNeK*(}s z=^r{p)SXz$1I%1f*be>^7@)=1M9nN@F}|BI;*vF8OGeN`QXfPsg`2VG*rlU(2zLCn z0Lv6GK~>H08zLnhfBmy$dEIGaU&y2~xowFFClagArOlr&xr*o%EP79tBOA~6hw5!` z`PNqQ0;Apddbj^PJ~!EOoIw1fQq-mhwWfZ`V%c9vo068Yyo8P199&BTf*|0u3E5@x zo1J}Ae!hqiof`PWma8`e6Tshd_#DTH*9e69)9pm!*yO}^BC{3ij%|yd+jg=C^KN6` zVFi5cON-;DQ2RX_*h;!eyjg;+{&}HDJ_0^A4~pCoOrcP&FXWakjmx_+B9*mcMc$z> zv}5*Z%dpmgFh197dx7<%(}{X5#qLUaKLYPS+8)SB&p$B5U1|S;r?{ z=W$hV?@m(yNUr1cSJ5g)LV!>Vq4MH=lBvHJcl^5Q1*em~IDBC=ctA8d4I*^g|F{3& zKljSwPF%h9ySJRz4f2P-!C5@{&qh2KO?3Cp= z0O>0Q8}%H3Y&Whi$4YZG_@p>d7~jvIZ+O!@M4z_HJF>$J{@r=rRb8InmxswA&^5T~ zK!8knF#OH^#cZ>2(MBTeo;*0~xl);O!Qa0&-=}MtlDXc@@>zjJw+_-&eQYAKIK6H0 z4so22r-5%Sd?Y=(aQ@piPJn;R;$7ct;PCp%GK6fEYguJ`6G|Q323jgke05z2g5czb z&1S9C%Tdi3QW^G#{ z_dQDq>u?B)ZXIui7Z>?L%^J{t3B-)$D-h#Zsr6V1j@0z|uEiXPC`L z2*B5t%zlUKUDw{ajh*ir6CEoT%+0p#1jGY88a|~i-MdCJ%7l1p!Al?Nr0ZTaQsCLD zB?J?&m3$IyyN@+B6=&ng2nf0Ty7?;!=|db$>l#CvTXb=Fa&Homh+AcozdD3sUe+ZqK3bn z>NKmEZslU$V3I`YN$k(%7>8nIc5&_mMOy)5T-cuakm$uLBIo%SCzAY6` zn-Dr)ziRr;SLPT@jdAJedrj-vN7O~un8<8_n-2F4mc7_FL}9*pxLP*&XV_w#db8X4 zmgzE6yp=gLk?o1^mT7MS(Z`>)!e90)%}_n zj^5*5japH$wMqmrH)#1q;sbU#{YNqVWy^hNQ}?B#j&8KSFxYe)A(yeW*EOwLIdCVS zOj*m=`%>?EUhYUr-q8*7VwnqXk#uQI0W3qh59IcfUzl20q@0l*MgIX~$sII`u_WS7 zrn<=d&(1`Q1gP_*<8?vm`Pq1H*-Fq1FRPBVi!BM~kYESGSzP}jpIR#%%M`I-*1;sFVWUZ|!zIlG7?Z9l`i`;s z`8j>W#pnh~RAk%$P^gamP#STfX>8FU#0!Wid^=F1K<=eyn(fj6jJ z&rreJvPKT9Ok`9G`%F>_M4T+(BR)3dd|KWGIOO5Pv6F!l1*#ox24l{O-C0v!qt<6c zz;!D(*NXo*7X2vh%H(SI$*He{S^ITN$BQh-G2II|LU2471!t~xd9o1HU8#t;dyRCqHEaS)Rcg<5v zy>ZrVoseh*?qHSU`{MM==5{%54WW5AO3a){)Xv#6bIEmp@>H75W7x0u@VjTEAGtJrC81VhCUm-ot-+MO5hET*`E;Up zOqzI2Yd7++A($X(dJoET;by_c@jpy+_#(*egCM%J)7yq;`{lXL?qr}**A>n<-~aT% z#j`J*vZ@S`PvQfrlZfcC;h1>M*8~S4Dy%8v+bJ79;?(3)Tq%)MLry-kBDLx9I-Ymu znr(>1pT8#~)HCvJ5sk(Z1KX6Dwvt+f>kUzc$a-EYGZJH_3dz?pmTPoov0X~OU7M-@VBY!)aV*ra z^`1jJa2}Qv9FewdN$0b;>Tua zomuDY-v?9A!6bgw!e&2AiIaJCIUR2p#yV+$?zk1Qqcf9OZUY03y1xSPnzUUYGfR$* zM(n@hum;sTju5^<#0&ay6Ji&7|8*?%vaZ`9+l79Z5V-v=YsjEH+plIcvq(0}XMBTe zzV%ne1;6h&{rVHT32e8cl0Okh0tIu*_s1J(;lOZ+b{F*hpXY_CjiOcNuORmpxJAuO zw3j1zu)>0@lk@q6L|U=?`y$2r+p&FJI9sRHhl4)u&ecbg@anE--Bsr);|ma>i<6K| z$l2l60k~ouBid)5^gYD5Hi=^rnp|d<0&ziRGoAzl{5^Oh?~?e-Ucz}Wh|3gzHeGK^ zQte%37jO)K?0Ef2HY_;7p-^|W-`RMAB_7GaP_DM%*=dU^X*!TAheId8Bq8zuI^dZ@G_uUz$nRPxm^*lQdoCJP0T6uE>KMZIK9WZ$|VQrUyG(;zn%_Yohe=J zwuOoIo1?W6LZ)zw7FolGF2-(#4#y&s`t`)gnbC01J!2jcHcyYp19p^GSxp6YkYF$;Y68J}I+1xP@y_v}p)|L4e_Ez8$p$$h{= z+|L?1oSRA4J^|JZr}x`$&nVE* zkx{1`bkw0opyaI&;KwA^3H4yiH@IfSg|=sTV87U?ol~e7SpzX|PC(bk_S*Rx=}D_@ z=aubuCO|GDb$bq@p_!y^TN0M@*7yhfb6)GQ^~eKZ4x$DKf#}WZtfL~PZqcOk05fjh zpcYl0`k_m=Xjyj5ar@^~ktd#=srzo=?i#}^=nj;(6ks+wle(d|cr}ssQX+0w6p5RfV(kypJPYDYj#Bm!eDt#x_3I{ml9iuK|VB0k{yt1+K4g^-ByXE zaDPredV2L0g&!v_Lx-}FN#a#E^=4P=K+|jY5D@lSJ5jG$-4u3ik2`q}Wopm+r#zYg zPLzN0&Fjs?J%^Hjjf`#2`45As+sx%kOc+U`t4r-}j6>9V4BB0^eHMajYTPzrUAf#j z9-r}hCP+rNKa@o&KuEY&{l*}DmB}Peq>~-cTcP!nG`bYN&=8%LhPxh#*i$pBnH&1S zfhhrt(U}~82&71+?YTJqo$<*+^{J3Maze73mB{&F*(9qq9jJw=}rnY7j{qt zuMZzS-?zDvv0|%OZy}qV6ggsj>h1_)px7HJNUn_U^81PH+#xte7X9dC#TBJCmL4-g#|+?>Vpb9%nbs?`>tj)|cwT zFwnb&(nL8<5h}8G>k;%-< z<_Lmb4ZLEI$}0+Dklq0;E8D?_rDeWU_OjkMN3@Z^HWi ztB^7w{u?LjpEDyiJR5PiQ8|%5{d zJc|i|on%hU^5{(KDJLQ6fM$Jd{C7QF%5&Z@o*J#2%k2vePZ++8`zAgy&TxU3!L;9~ zu(55$r$q;D6H-Qfef7b8zD@&^H^>te<=p0ceYkvw4oBnMbEJOnEi%G=8^JEVxoV~V zc5WQ+*wGlr(s3QGVcU=+7qm<#CZ~{*+M3*n(p9ARbey^Be9?gio+zQZxb+3Oy~$ku zU>ik({Cj-V`eH^ToZ-q)YG29z42T%6d-b7&D)HNA?oa z3Tra>NO2}>02kBuyNdV46QoTBj+iOT_!9h0{${w!2YB$VmIerDjvTHZ^ShMaQcwh} z40SWf=ofg0BdhZ-yyjpv=Z|SwQn=8f>G?E~o2NEjvfR;2^N!%pa{r8FTRy20cBeN= zAU{Z?FpyezojpjVqlVtHO%`pC{p>55LdDPNE*AjE?ZE-Q{a_`Itg{vrwt`d3ghxEwgTuve`!rcbK|7oVrmM{etSu zvXoBF#g2368ZXgzKz9@e8LS39UjXODjXyd7?K!XiU`hTnoX+_NME{uZwWCRL*ko*7cQtnrKKc9(rsCg}>o9BfhbmrT zLlIN5`c$l>P*002rS8{C4Raw#5d-No60yg0Qa0|O+Vg35)Kq56|BO^S-&)?nh4>XL za@xp@L4YcppyQI%{`oZAhlf2o_U?^1o=*8!4}NK7`B#4w8AN{0Jw%E&#ycYpUrJhR z48I?X${%cG8dtiF`Y+5B896L62w>1Kr;a?f6962TlIfRxR%r>RFHHnh>2@SX=}kGl zEFQbE+$RfEn4|u1D+|nMk19_?^lF*=|ePZVg_|Vo)z@1=%aB z{de)#T0Dz*5%+*2JVL;+e7*su(YGfOvzLjS)wYNd1NXs8{o$U#MazYsyt9%HrW?s< zb6z}^R7(F*R6wz1vP(GiHilMuR$Gg$`Xgof9;$pi9^)UQUgv;nm$uQ2lq7;-e~!ui zpGmNOn17n(QiF&~!bVq1-etAw@s$Ge=Z6QSc3q6@W{Q!O7|CAYfN`bAtux#Uzs+uA zp^b}M4&aUt)5<^mrp6CMjs#}2KOMiWZf=U)V?Hu<-{0%UMR1?d;h^oe%bDK(Aa}hG zNQ~#5Qlq98}o&6ocED4yYU#(%@NinX@bXgv(<}jH4ANoxh!H62XI9WPdpc&*PmI*L)k5m$aM zvt)Y1qBHb%AphUq5cw>Y%n7hAb|0D@3B5buHfLU$O0#_Rw=Ry%={}L&51X8?k_HoL zi^s`TDH2#7I&??qTJI%f+77uMFxum8#}^`yVD?fS-1Ju%PMSK#`EAX z0Kd(T=4|DqKU{XX9PGp+{4+U76-fW5@lG171^-H*(HX100hs)u8vms|(%=eKF_Eht$ZEuU(xjWzR6k^GY5TxNm;stW6BR z;dq)o%#dFjxf0IDvR9}F`edz|yl)J_Xy7~*Pw8dosoT0aIK1_)9j0FOTdB`?W)zJP zRv)oNluEF6CAmgX7lQKp3cqp)?SEb}`3Ou7@v{lnGwoeyc`j3AIMO>*W&@X)0>1uf!*npJZ-8&flyVXoI z3fAcBJ%w(ID4#sjV*p6UGpWVS1KxYCl?evRizqE-G#rMQ0gs1R?C&gg6Jk*4@VOp+ zWxK5z0Oc}|V{ycFZVZjh9>to0xRXdr9b~)Qo@S@x?y~jA{`>gK;Aq>8C9gGmz8A6+{{rQTrg(&N%sH!dFq%|I z{e0!OlIPhhUr%%gwd_c#uH^f)3WXwwLEqcDeg9WGH#p^mlwE9jmJvzF7DsNzB732E zn%H9P^o}^bt5Y04eh-Y&4O3aoxVi0pol3{roQo7LsdSJA*B3j!3HMrylk|04;ZETC zYC5}ppNje9$%u#QCe4sZx{Kn6u{?@^t&Z)GR-VqZ=? zJDbld|5*v*sjyjgM`!eD>BP}*hFf4vGSqZ*c6%D5pMEoWDqp5d+V)pBxY-0rWoQ=9 z09w=9uJt}mTX6>^+UZwQf_UXX&B zH*S&Szi^CiJ&$LU3hb)?M2eH0{{b-6T_4^eSl4pC)F3!@__k<_6(A5#EqBV9Sn`p$ z{CYrqg+RVQJBB-6g zPtT4|GyjSIduhqVyP0C?w`DRD-#CwdT?MuA<&Grv0q2nV#dX@!Xj%C_$U6$*$j0~L zF{p(4=FwGYI?A4*FE~zqKpU*|_Cb1u7CCn@#A+WuIB9_E?n2L&^F>@8wHT6T!!$F# zV4nW+7~_0*_~(i-b9Zedca?g2$TCJ~+@qRAY2rexG`Fi;h+0qox&!#zbb!Q91l}t1 zOh$9|7bQZ79d9X%OcBz+aC%=@CS!to7N^{FX8v3VXV8#+W>LMc{U*oi*am@b`mFLR zhF-bEx7+l~Aa7Kl`)Lc$biCeUN=x_odSjF7J7&pKs$c)U!-$6nhBpm^^?M1mVtkhz zM)FD2FaLIW><0@PtAuH}lCh-y1!wNszXPNW8O3V;icCobr^^j7j@0-if77XpT==u+ zMtZYLNQs}0HEX^rjx`^PG`uQ6_$RY^ff@c=yO=li1OYLaanZ@6WTWjpCz*PvDfE>n zCwXN7lTlBv6}yl?ESsHmd-%$5zi%sy>cj6_6B&H$x3jCc9FU6N*wg?2xGtIFl6qmpPx4u+toD_f&@wnmHmc8U=g4I27 zAF}mlGm9`8DTX2R2Zk)4W-~)Yt}bA_z&7iJi(}{SY~6L=s{cYgX?;y`bI{!_yMInj zcss@m`G!Zvb?59SMjHZKQ`=f)T?|Ll*(sMLA1W~OX|{}ygSi7^q8G08I3 z7&+PwO1}NlU1E9b!hQMv3G)Lj+a6NQ!>CI48JsuQL>w9cdj^DBqn`n6uQQQ6-x2lA zAbdfr`^TM4O=e^254p$6H|NhOVKM2A&SThBS&Ww(m;Rh12iN0MDC@&bn+e8ay15H? z&f7U92x+K;&R0v^-&r0~ga|*3=s*#XVy)z{EpQ!yyEhhf$12+2Kr^ORuF<;@V3(s|uNvaxPudlH`Iu+o%T080o{1Z0G{9!E zLp3Ez$pUWIDx=*ksNd;EEAX;u;z*zY(^j(&OpP4HxRkocShBU1t(U{=ksRYhTdsl6 z{n8qJ6aP^0pfAxNbh4y3lR{_814Kx`#ls8t*e5C06}bsAaiKD4L)~k4iw_~yxEX4v z)@;1LIsa48#0)-dfHAs6SK2<9aU41v7C#CMI5vJTzw)T$#aN4P?OATLxOhQ}V-{^h zdh6KM8J|yYyJHPqv|6nqs=yEV?{g~FKR|fT>oxUN7?bpEWJUh)eZ07&eJVpZ~DSFTn!Q_)nod77ayS@XW_pv2goAEeo(0u-kN=QhkQYsgfl9Dp{DJ`&sIYtB6g3DWqztvB)h)IteD(R<_hrg@g>p8ljQRKw_&{OF)=Y)+uIQw&sKiD z@@sU*WJN6wn_Vr9{~2p4D%Q_tqpAJ<{WK5WDVzWjQqq{BqN2(4|0J-HG$@VB&ng>c zsl`#sz=`nXD#G=D_@UQN3|Go){Tp$={^tMXFeC-k`t_Bok4F5o)2L(;-x*l`KHKPO z^*n}U4vZE{*8J1ja&>i8@G>sTS|;2T>Mm6O#B}Wt$&5X3mQ;Mz()o3xl`S{@u6tXq zlt4~d(LcWdyezbh|kaOgRA78!Ib+U7JvHjkbl_<`Z`x8>J@Br`_@oN z({bW^<~IkAAFIFt>1yMYfXYbFIGI3Hn-11{9P+sNooJ6dXHIB2;zjz@^>s@=Ltu+e zhtX*6O^MykaN=N-wr9NY2g1wQ4y}t97s|t$x_}}ISE-<<31Myo_gGi4pv z-lyb>5-=XCiMP&pic9M;BRP5Upe*y;qvh%UkJ( zl7zm)Y&Md_Z1=5Sr1o-z_xi;jKM2p92ZG1E>O;iJFp7hPoSL3ry&wP3ZOew?_{h+O z>L{5|mHxa!IB(ad3ydWlIE)zxyc1fRN7(tT}i@;K8KSSpcLs{vLpv48(*~_2}KF?$s+taBhnrbfA+UpMl@s>XZuloG;&f!pQ zcBfV8Cn9>)bXD<5hfd@osK+F3z@?-Rda0V0ealcRFIAwXPlg^o;*%?n60qc$noQ)P^0We2VMQ3WDdQh z-{~$MYGKJy8Y}>kF1OEpL-H1jS#0(s=ccU6xtRHPKb#Nl15wro?~``58t#a;!v)^@ zX;1rUx;%*;JqF#o83bE|Oj)_uy|Pi`iuq#HB2W)QYwNSJ0bQ8rO6gp*V0~h3#Pj${kKwCf_EN zx=Osp`g}#`Zo_&W>T+GwRXe}1;HM+vtJ=GeY z{Fr8{zT|SVpV)sPq2^8c#lbv-B#ADCR6&MK6JsG^b)OzjZV@; z_}%NAvLUx$Z|^;HqUPZ<=TTu27{s7smr=ppEJ^4@{Wa=q$ei4oG|>vEq^y3#+`3_b zZ#~VDfPqC^aC{QtQ0xS5PD*uB&jF#V-WmG~H4ew~X4o0i_=r!SF_cMRvz7kutr&dx5^^u;63EZP) zZUEme5m5)t@rJm=_N;QuCQ$#q!8G+EwOdvig3e%0wcRPfl;zK6|+A$^OO&lcpo`dF!jS$b>uIvItNDF3;<3eTyU?rW`DOJ zhBu=C8x#&HkXg84`nw+Dglu2=by$pv0PXd1jHjV^bGL()rW!4{u@a%@1rI?1W1N8Q z0FCjhidcPuJ{$PAjvZd{cDu-chjL~H@co9H;p0)9sntSYVR|K*-V=t9K&+`j7mt+^ z4MxTklj*nySd)uosz9|Psj58{_#H7~rbUGxtau`GVW996;kk@A1$184#GI9!xpY>g z=OGjnBearku`_bkjql+r&|K>J4L5$;t9;$#-2`K?%&pKwh}}~p#n&sl={ErhI%HZ9 ze1i#*+F1sa>n87J5$^TflfOw6_f3bk;KB5QmrHogyRPv^6ruaiQ1Ts#tjiznFncux zx0q_`E|ZsPwsb$fy!C0~iYqS<1YP;5R>i1;rE(Tmm>I5-7s>Q57er@qTT|nf$!Ax7 z_{F47hnXUOdpJQqotdYY@IBu_j=1LzkcyDx#I{8nkx3$pyv%2fDk+7Q$NMzDTt^*A zcEkfem-`oAvg1+TFH`Ml^_kC3+AV4RjE9X>tsj~7dWa*t^n%G{yM6?xDAaWpVl4v> zHM!g{Uax~u{_(C&jpmxw)SW>wLRg)T+ZkULxqGTXwBmA9Bb2ISRJlb4_u0A4aBX?A z(~PjL*tTcL&sup6$;|$=c<>JfleR~0VDOYBB;QHT^Ha_LgM!r|6IK}l?eH;gOx?{# zUAQ?5POo{lsowun<%!K+zwB$-6vMWDruDwdj;?SrEm{Ij6O5AZiWY)q$SdA) z;s9~IPx?RHmlzH`*qV;SMz7J0x%JVo| zqOYxxvnt-WIqA%G29!HjiL+FJd|Gr?F&TVpMWSqqIE z=w?2UK4G!GfQiX@D)9z*FZra){@zB>IRevwI`$-PHzO|UKm}^|t5n$f~xN-3> zWPCT+>t|M!_>3nt&B6j1}v;Qwvh%4W6_ zhU5;`AJq6TiS4oVS3=+r*-uR)|rg;2)sXtYMzP|s>A1iUJzwOzUnXDLn{vuOn zsz#Spn>nCwN{wm+;mpD^D>?HO1|{#C1uBVOnk=%tt<(#k1Ns&H<;))HpZD`8_xemc zi)O?HuhaT#<=jX>FcOw8{T1(f<$B_wYT%0i7=n4d^G-0vP6S%LuCA-V7MR`0Z-?Ooo+xyVWTAtQe6|sv3;<35QmIeAQcS)SPY$2V4c?)QnKI;I z5%%=tl4+5#1y_r&%RIk8L1*|ZvdZ*6&-<6xZ)$K{*0Wu&-XDrCtbtFyw6Ep$XtD2& zi(2;_Y)g2PhDz#0(PcN5g}^TX+F77Q7qxvjF6k)%CF!XTuxcOhVGiAML9<}c84TBi z?IVQqDoQz`$!p}qZu3}8tMm1&$HlKruaaDvCv?!F6jMgZ(;~MsG~IV>GoarMUlnCp zM8PvuzAZOW1*v^=KJRmR+?#v5>HkjlC%-S)%S#2(=`}vK5qYQ0DkGU+oI(~Zw3|Ql zjSe~-T)XIUun50oA<;_0=An80R|lhMkssW>M6UZ;Giq!VyP&OGVxs#=eduN9_ETC>bYAB2B3v6oI%}ghwI9#nlEG8!pA1C=oFt1j{z@8 zTI9o|+B5sTv16*MegQVuh6+-Nmdgs6S!lQM}@vFj}nCJKg=$q8#e z@n2bVWKQqB6KvUt>9fVV;!i8dNg$FPHw(U(&6g;Sgf;pX7YH-A#S_<&lj<~|~rCY$l*l$GId^?1n7Drx+!` zmC@S{E!T-Sq*{xF$9G|gR}RFTev5_<;Ybvat$1LgqY*oMtp}`e>Rixs$}37k1^=Ye zx4tOag5vxocVup&zo|4OC{|lcpAxO5C^OcoltT@>ym@#I?20@s=9>G3NJY%zr+u6V zNk^}?q%wicAC?HmDuSdBX{IvH0M-w7c{`B(m(np2|_6w>py&Pz_p zbAD4Ml~5{)=qoUGsD z3c<2Fn-gnQs1$O!gH2YmncPlH)WVh@MU}5a&aU!MTyIvLP4`VV{)m2xaj7Uh%(B(W#wY{E1=tE>x)_V6>rgA z$}suwmwMy)i<(^7*73rJrzo``ixVUqqKOc*m*sw5iIO3jGeFKqe-CMdXznwGGnG8g z_wuGJKKfZLHwjpgjwPu;6r{#78erOwuVJWamknYi$m4eVJB{}~`bGLMkGlT()emYi zG{yW%1z+EeYFzdzux#4O(jSxv#_@-ii0!hS?;S5TXm8u=NjGNc5u zLC&3k6%lrmx&9`03OB~FX|Ba(kyb{H%8cIiJpg1xoKfa!gpUNQX=3b5QD5}m0WN>mCTtrhrEtgD zT8bN3l^W6c81{aLVo$r8APfh5bC@*kdUi#wF3V?BfU-Mtj3&@y6pJx=j#Z3w-Wr&zf|u-wMz?I6S5c(>vTblU6~;oXh2m9pwJ3!XPT+fjb#SVf33< zQ5zpLvdwYDht#jiL$U3RBoh4w^Z&Wl%yrGBYt)+6{YmNPs=#b>) zuaMzF+nsx?PQ7j$aIn~MQlVtJe2;AJwQn69`1_W@!k=j59k2>FOiq4u+DzNvGg7VT z<73AX0cr12ZASr5hs1?J2AkV2)^1<5)A^TF1-GN+0=A*%eAC}YeBz5-yK}6-HZr}O zydsd&=^Y=mhM9?}4(=S*7CaB^_)~5Zrx9-@oEqf~n}rs%%;>1tUQ(qe*i89s(1q$f z$8bQCG*ioErX2juw>8X^5f8YK8x3`Rz73x4K3xm~{QrKl7vMDia~;*xEYmUXe@y1R zIHLFmoF$hXfVb{*!Qu9xp_V@O>+8F#Dfd!cZFZX%*GJAzVf(xPi?_FoitAgxMMH2W z!9BQpaEIXT5Zs-{T|#i};O_3yxVzK1yE_DTc>K=)oO{n5JNMG+GEvR zRjcN#B}1?b3gP!#6QR)p5}_xumU$VfE-L!Gco({r2R}wdjU~54TJO(2dJhu_+^`PU ztWQ`zUfWnl8XvWF%(}Es++7rbUNpb2M+IN?wsZhq_i&b5&AEJExCY+WJ!{GJ z&TuXfd|N}SQr>=>@C7SAy=}uTHE}9Wg5wW-YdZvjzUdJ_j5aH0liA~2?~P^6g%ssP z-x!P7cB0Yc=n-`PxP z8(op3*_DnKzqeq5xIVB@Oi4!M{=I9Ofwo0Xdt|63h#7Hl@VEl#s8vJx#rra_hAHF7 z$?9J0_t%SRk1@}4O01lMEZN)O^HX?kTRo>36cp6v8N-AI znnysq8;Pkd9L}o-&kMs z>({4YrT_IBa3p=&2WMxm>W$`i0EKL{cXbE|2<7#lco&lVa>V~_$2KwT2_S7%n(069 z|6$3H6FD>O|Bdk{jISx}ho99ZD5T9^MsbYp*ou#t@2nC!I~jg!t<+O&WXdE&Vv2mfE7oFx!+$45T{j+4I0qd|8# z95`d+s?>*vhqk|#yj!a>FinQ(4{wMqv-kajjS#x9V+X3cqp;wS8MYU$S~>=ApJLrk zWPRCH1)=Ftps1AZSlGq$RIwu)?vIEClg$RdZxvWN`J4GpO%qo|!yRy+S+Vml2wU%T zix%@EZV+ULpImI|LGwBTCo5?-V>1Sk_e2y$%$(?52?pN1rZD{qSq>ztZo z+kf053loHCIA&EYP$Q|K*= z*fc-H1f1Csae;c^km8&va$PlZP@1?JETW&$xa1oUAaS8cVe6+idQnTt$8C+f=oRM( znozM{7Nf0%H&95ws-hcjp^`{`Hr2p;cv7rTXY)jU3qt%Noyg;-GlBK+^t8RQPc~rJ z62_S1Qu5U=V`Jc^zynghbk0irf@xu^YB&{^Mg#0vWy4UqRP0zhcnd|CI=Bf&F43J! z60Z<2UVi&UZ%MMxY7Qx_(q10s@Y=5O&{W~@inSI3I`+1CDmCx4dlK@*Z; zFMl0vcoiP4*YDigF{{6n;!QkS0TIa@hCSXW@XiMl!qjZ4KHxWud;Y!~%lA77ZkA3% z+wNS~pAgec9Xu4dyVT8c{r>kh$90A%QfEhD74*cj$LQ5bDAWowa4FfkZoGI*oJN!7 z#(ciyW{tNJtP!Qrw{qLEBS>+ufw^51E4W+?DzuxN)2|F$Ldfd+D3+V ztHJyc_MYw(mF7<`Q+-$oX$&sqs((sNPoIQbwRx^(-oqd6uk&5Kp@>i6sCYP!JawNe zIu_B!h@&fQ@Rfs3`1z^b>3-e$71%N@{ZcpWt=qx`<$Jh^jk@`woTtN^^*ccV{~|&- z3zp;=jVh<(D*0Eik8^jP5#^COvIq^606=oRS@#c%fhBZZ(H4J-qKP783;yh;%d0(3lNk}iP;0Qe$%oT&dJ}|W=YYRLgtBI zOt4CJI$R;#VUF&^i}F`?Hb+(s5OzQ1dDsXvMf4j-I5~|-CO(=6cBoXQOQF^tzhbeP z^tgm0zo%q`G%(i(^z1VY){;p>+*riMQqNnmw6fOA@N%GJkKpgllr49j@~u5Hi25QG zo@yE|@82+X4a1XZxy~5y&Ew19KHXA;qp~vU|5Xx@SI`GH?=Aq+ZhaAPuhz75JSU7t z{UOV<=DxgkT*_+t=+iopTp9TBbGlgS8AzlH{~jk;Y)PNsKL^=Qesq(85#5$DL(WvN$QZWa@$pxYz@FlEK|~L>rPK-E zY87;=;(kLXWqKO?oR@lnkGqjah+Kwej^@qk$i|taTfoH@V3GyQ^~AI#NT$*5ocp26NLSm?B}|J`OepoaJJrzXxK z-l&$!anJq@01BZsk_qUX1>bAVv?~~r9`6RiHjOjep|xts;Q&H{5@|D_Vcc?dm#mO* zyttNZg5J=+s1BE8oQK8vyyV9AS0B@kg_lkHGpEJqf7WunaW@WtWTeiYne7< zR?jd^-#Tn~fD!$y&w_>!wm;#nP1)d*V*)n=iwGRtv}^SS&xC5V88n=wJ^aL2)U`<(R_B;ow@)^NcP~^!I4vflO73*Sb5sBUY&Yslc&(Q1p(7s@540+*lUa8i!tR zf@G)|M;=|#nMyO{iJug{M-!UqgpZp0ooHYvc%6&8MWx`Gxdpqq2i%LQG?_a`y?ad? z?p5eIJiK%lbwyKj2gFA_e9!~T7ZTp@a9JNCzok~;#SipOWsIx|=4j>E+dYeY@&s3` zg_~x@~%+*byx|0c)~MpoLGBi_PHyGbxb-#f1Yj-X8&is?X2OQ z862KSthe5b093%#U9kHzSlZSKyZti6^tyg6v5kCwf#l%?2<6@NPAB1;uK^2XeDtjr z0HPaGydLrSyR_pI5dsa-f*DV&Z*f*lyxd!QAna?D^8FoFN@_lekv7)X4qVaJa3Zee zg#5YcN{of|y<<_x(qe<(nOKENW#Jf5s*lo>`J%Y-T`N!~ji9>4E^r3=Lz9y>-aG(| zy^%#W31A?5cFxg#g>fdXqajQDW_+q~Z>Z<=LKO%lz7eDPqWQA-S%fyyeC7x-eU&q8 z`~HB0(pWJ0>O+7uh9kDN=#&~*T47nL4l%sC zXz$O8zEE?$3GhC0OkL9V9^GqT)9ifyR_0m4#b*wAFO_!0Y{A6wp_)9sv5bCwf#;Br z=yWa=o$t;`Bx5skTsQob=GWQvxYK!MT7&qJrU|}cL0xVrRr;9IE}0Qa)|E_0NNv>T zD|#-4ZXb~mG6tgEcMeJgL>hcE{%qz zn;oM5eBPtk%8cQ-p5CL*Z67^5HH&DTFHkWC(3{vb4>jmTn1i2$?piRntET4MS#!w) z;(CwT9^t2i-bCb)=#4|vep*o78FI-GmslvxQ&yDoOg5IYAKQdy>N;o$FeIm0j=+G4 z;kVo-M7$5oQajd;*4iwsGnBwtKPYn+i)j)goP(bE4Z0XGv;2OhP@$LFXP3f1bHIRT z9w{5S4HJU;4BZzdcU9(}qSx!y*A*n*`h%{-`1^@(CO)hF`b&Xn$$ z%yNPtAfrU=Vl_sigqoon1sjSDvWd&@SmX5$*;uuKcmqr`_n$O62vR>V(}w1Lu#fAq zf$NRpfCvr4I&d<DyKZgl8V8pDPe z{^;M$Ga%7cZl35LEo@qP`iN!|KVtCd+E-?Ey)ZdC^L-4!0#HD%-En~<2_OwrH=VvF zxq3orNaxJ4nTb_iLX3fZKH;7#sA23os> zs8(z)qf*g!SwF(MxFfU9LKtnAa(NSys~;8KUaaZR%R;Vp%x6Aa${^av1Gur!cfHkf zYs1|Xaxt=MczF*S2C2&FYxuI-s+!pnAP5AQT2++Ua1&m`7_0De%CC{?OJNQM`$$Io z?3lG%CVP!xp~T3U+b!`{<4DHN$CS=Fo~6J2Bk0So5J)r$h)Q?GZ;iQYz+tfhvS>-lC$K=hj4hY^}PNETaZ7r zdCmtASkV>HAc|td*?ir(-+WiC8NLN$^fd$(&ly>%sHubSW?EK7-?#{pvGUuy zb5y2O&vqkpm)+Km-Ew2%5YTmK^ha<Aay83aFx(I)g!l18F^+g%j+y}Eu7fwEo< z;j*Rfz1{{Vto=UJ&sKb6QSV7_OHfk>mC%fTJ7BlFxvjn^#XUCDa|bq?YA^L(>>NWh z-5<%tT&=o<*KMnPFDd;FoTBEypiXX9F^|a02rt4LM?4wq7Zfu+NiG)$P2IqQB$x5khVdE;Kmv?1 zxl+T_XLri(8nxP}W)9YuEpTu;pzm%?DkwA9+twl0jGaJ8hBQ;9JrGL4AmlXCs}1P6 zQ(^0exrS+UoS|Z)_bK* z<7{y#eCA#Fi+c{gX@;31x&T#KFO|We@ptl9-Cr*3HnmVTAjE8fHU!Qx|!5hdQ3 zhvrFIk7r|J1A^ilKeJ(3;G^IuiYggZ2D#G@CS9-!T+q>k(?$61wZ-<$aP2r*po8%N zK2?U9PW*ap^3b!s_%m7Pc-XbTIoRJ))8e?WVJw_Ym&HhPrOCNy3i0aQ^BgBgid5R# zH_OyUQvqTSvgo!J+?E2JFW+M-6Ax0Ryzr=;F25F+KgK&f7daM74f>wdBCAsWaPWNs zrZcSr^;{~t|He-aaWS59DI`>JI-WkLcT`~akC32qm#Te|k>pWNpt#6WY-{6r)`%83#QYemJ=9WO2FT{b;SLDL0Pty(uQ>U#e(i#(7zE zqE12(zdzKsh;Tgs@jye!=%o6JEc6ss@pGt6e#I=Idg&rMuTSeM z+=0~+EG4)%^XpH)W?EOu>(e*C%=cMz!Id<|$E1wkjL;%;c<@1Euzud0$!3e_No6tg z&Pa+57J0YA;kF0B7%`6Qez|mYqfT7P{ zF=KxXe)LHBxk^|h`%I%akP$!b<^jhvb~nny<(z-nMRbaHMZ@q$8vz)HnFXvz7S8eM zJ@Q{}6lCw6S8XtRdVJ+i0_P;C%=>2xyM^n!iS8uSZxlyrN@7L0txbhbp~qm1{u(~O z`BU&#PcDQwL7LF8$H%SA6-K3yo9wvBcs4d_-xPkUE!8{6W|+=!+H;&V_7IAUkTWw+ zU9O#AQlT-g-!C(h5Ia=ZPC}17fB8WQ6_|_9{Zyl1!66um_=~3S`=MUgI7LSCjFQYm zdn5H_D}qXW>ioshw!_Ba`ceWNc+N4!^>nkJY3rvqQ?OG`B9%bZViql>|2H3m1$8~6 z4_&-E2TwxN;xonurVN6CrBUzAl<8p{-r!=u&r&y9AQ0X{FVTYr-e%o3X>027anIN- z976hMgj&xUEm^jemGBqsrW`5ow5>Y}0GVL@c<^~43POA4H&Oc0g>e0VUL+8C^V$Pj zii52mJQD_9r-Vw`+rK1$$y`M>9Xhh8w1@&xsi3P!(x;}eT5SGhdB>hsIRzwgM{Rr! zj$94t?dp%bA4Yv?PqIi+47}R_ZXN$waxP;>tWSX~@Ny|^SQ)=z*x>cRJZQ;ML<=1$ zIj>@gEg_CF8v8Tw5)6IZbNi4lMe)Y>Bk`McfsX7DM*&p({8_F>;X7sq&uu;FB!^Lz_UY0cseiSGJdF<&hDm8^T#OF|55S8w$c0RsOIRnKb|B1^n$Ipw zQKfJj&?dzevR=N6$m*nm3|v3Ol8rlYsy4a==aS+AoeU)INGWd!nJ&z|!#v@6Qaf94t=r=Htt_M zb`+xnDur&J&gRGp?DS{5Y)nn>^a~r_O!=g{S&oOZGG#5=+B^5E7f|r%KvbEpO#bCk z?jE@7H}6{7a}t^|X`ikL_X;1_{1pM>6IE>!aI%(|kD?a`X;he)O5xJUj5IB7XP_yJ z3-&ZC_M>NYzVc4j9;>s?rD6`bHds`@^Dh%g=8HG7(%;vPskMMLXnrhKwnonL)EbiNol!_d!}X|5q=fBy*bOx|0s zoYnF={`|n8IrW`;8yizh(PVwN4woG}=S)2u4x5Pd7wlif&~OU64*9)wqLwTZq(9Yn z@_lUCJ}HYqm-}xR?_Q?YV?y`M+^z7PhqXw%nJKsK3KvMVhLsbBWUc|}Tq~@bX84D( zJOmp*oo`rIMK-slmO+s;r8q2Z%pyjQGXfEZ8QNW_-AW$KbOHH-jJ7d!`o>8JK=ynj zTcGvk1vw|A({yNzcQ*NfjqHRE-|FYKiLp#NB?2KXwp4v8b*!!KP*%iQDPp71NQ1g} z-cBbW*jmaCmmKG6NnmgUg?=REcX6`uX+J{VM|jU{*cvOfg2v8{GB3;_0hE zIfj}VP+t$n4j2xTqltGp7A@1^Q4PB`Zt4{~?}94=GfP!^L;Na8?i5%Gj`W?2j04ik zOj;RWb2O|CT;!Y`?PZSZex5INvnTL~i+m3pS`u`m^LZI>{joToXZ=%Al_b2ik<>9o z2wU1S8O>qGAyM9C@%0D6D*vz?Z?^J6juSGnx$D{1WEfPb&*rtb#-G&@_iO}rA(O&) z{tnF4w^Mq0wS4l}H}_8oj0kcz;?{)=7L!_B%C*8&9WSo%+GZSNA4tLY5n-xe)IE&j z4(wVB$gcc(%~lj1-6PYWR%}0}gS`s(fgfoN-16gBQNeZ!XaypwT()Dkel{~5RpENV zTM%IzF>QON5V=Yix?060ah#i~ZOs2AKZq<>sOKfk3Uq<#>IxIpplPN1J;te}1p$+= zb%yk_CzqShTnv8#n@OI(nNTQ=OzP&j>C{PhN~1_dc#s&VLZh+7em3ti8DiO-<%`{n zn;AVH?>a7lD`CBJwnF0x)&uV6u0wg%5 zJ4_h*g-1bCjKwvfr92)z&;<79zuKBgSuS{69w`|E6V)2g!#eLiT6Kc^qW;4L05K}N z&x=VDg!liYj!$I+Fe~HwLE4&=l#(?_aHVw@i?ldymL1X;(E9?j3mGQ~_fU91w$Qj{rE z4l6wx9VKUbd`xq#Rrbsq}Tm=P0=^X zLEIP4$32&iuF6nowtTNG&vHzX6YVsry4Os6OQxYlNLGwQ*pr2Ca-=f$=|v}!NLK0^ zzN??O*yCi?)>BF{yI*}zin@NW&&hA30W!jxmgv{T#0+4= z^ENA~z9#>A#+8XXCosE3@iR(avj=&OwTHCOyVzSa;X)(s=}d6*EE(c`O?caPgH&)w zEA=|7DM7%Op8rd_{%-ZOvxAWfhaA=0QC-8Jn0b1THsROv5|^c!N<(c$fF3|AEPNj= zpAcE2FE`*PvP*gXfXeLV*vE5+hfRE_7`lB`>T#-29UNLrFFara+rg#0^Sazepw*qI z?eHtJx>rWi1%y_?n*?6l#KpIA21stNi=8ga=eQJb@}BlEXz$!w@J8IhWpIIVFesVMxyK3yLd0G zuIXd=2nAtSzB&AN-*UF>nIn(#IrMk~A>l2D^60^z{8K4JWw_XhLZJz%j_MOG$2U`v zi(llr#l;S-GK^LW)fJLCQu+v{)gaiU%hXm;Bgrr(CB#{u>Jq)pN&>}-%(76;hEI7M zfqJlHO<%IcDptB9|H|a}@>8usa8d;Agb*`tS`ip7o7k zy6XB~#sDS(vQdhoNC} z)!rI#*y%br%8N*t9Nv1vsV9R()ugI}JDSL&_X68ihUT8!Wr{1w#sxB)&Od1L$(>%D znYC{z7p;}y#%qu|_=Ef6#@_Fsf`X&7n^nBU+cVUwuq|f&d8Bw#*zPg>m16j~xU;yQ zF|F>e+4Vxla~P4=>wf_Tn08ey(rffGJ#dO#iIrBw#oFF+MMg|kpE6bWE6E>f%w{!x z7d_%a9|qI4xdP_8F^^W4@YR}#eMwAb0?G}f>et6vxNDQSJ7LjqF~c`fJ#2s_q6ZDo zO67!7Ee$DRJHA^Jumc=s?{0#Q;8OUr6YWk{fxX!`8ik4K*8|;$BIuTj%B+!C31r-< zP9(~SdAU(e%;@h&bD@1_Y$;b>pS~ywzi|8DQdOi)))ca5AAq~rUp#LNKeoib&b5c} zwuBz>zQRXE?oLG!jy*i$q15B=l76%%eWYP1HbPkhW|7>`c9EhFAFHkmUJk z#)P9E)v-%ysQ*XRHm&znS`=4rfi|x)I`T&z?9B0Pv58Bp{=yS_$mJ!q1eWcVVa${2 zYnneduJJ)`XQpVUFx(~TnlwNi#3Z`n-G9%IS2JBm-LckTc9pN}{d#ExgH&f~dZ7WU zMUWb8?7J&lh-x{k6D9KY`(n!d&PIUISj6stFh$K;Omv1&$8IW^7T``$`dc^U(U?5W z%tttH*rE6evq1e`OZM2#`!3!Z)P2LxlQ<=py0fVlU#rVI)zc3|L+7U~WXEZ#7Gx{( z7G0pov8d^;(D4cWAl=zbJOQQl2n6ehhvZ*Djy^!&@g##ny3sDtGLMTxeLh(Pqsu`C zt@I-=bs|X3dXWCI(_7|OOgs0jAe>K^ZqbgL_NL1yfFHuylM-ynUs*Dsv+B*IW0Cdd zTG%}FybvG|hM|bp21&H_#p|vHpfrBUWypY{Z zyg_S;_pi1kv$*ytJgGqEv4^v_xq^f>pG}C%7%UkY)F5`hcGiHaFoTH;rD4pBep&4u z=bCA*qw1`OGdisCnhlle&u++;J|ZQsIRAx-=Dmd?5V9==bg^rZ=Kj{4j@@}k{P(dv zvcYirH=XR~^%Kn$GR(-Cfu66ip^a*t?)W}J7@a37QmZ}zW5ByQ4ox;+XYQwiKb3Y2 zI!g^F5D7)lDqZ85G=$+Zhl_GBwjJO<>zK*t+Dhp zhO$Mv+zJiWzi~WwIWXA~Xz}Ur78r(eY3GX_L7npyS}pewc}cpt`lOo8%a{GI>&|Qw zcFcyq+oMbi({j59rlHu_*mTb574r^+exdlNlCJpioyRL^>*Y#0sN;XeZFr;I ziIuazbT*R1HpW#B_hPOXvlXvs>LZ{-Jf6KVQRAf(&?J`5yDfaV^*<>a-PAVP_KR1k zQL-p>-f=H{Poq*bJRII@yUr}=PU4uSy>=^2Ny8L-j0+l30RD)8fj90OCwhpg0# ze;w!f8XU$S$sSFIQuq6ZBTKMaikQv{-2!|K7I%*#PkyFJwnNMBr83*}OO~buklfA< zS|~LOqQmeTEjOXe%GKr=MHt>#I)?>?dCJP3bj&ZRs#WvHVfV2Z-(j>=0(++m_@c}i z4J2{Vk$42(xUgla`z@9U!2~g&?5)VSF}vm1;nD46mS_BBhA{FagMbw;xkodHjZS`) zL>5=ZE9wy60JP7l|4`aW^gozj?p|K+bf~jvpsX+BetE|Ly2r`UAK6WZ@OH5-1A7#zo~Qokmx+3p~U}x7`NftlAjemu(60dzAyRN zko#ck_Zg3v;8akbBJ|GfCT8 zV_Fy|H`68eqdNDcX(_hU;g9}oRRqsAy~RC~+z4WTn?qHRtPn?3Y5Kwo<=yiR2KO`` zI8OUFOSKBN`7E<85=MJhcugMlS8u&km+&G0Rnq0Rp7UdrPTLmWdKm}4ma=tMR`aW@ zcun_rV=&vV2!w>7h^7b*w-+stj97W_AA^{EYC+$ zh2=41CRv^Q;EZ3+xOMtdadZ)e85s%Aw!*cSUlD0#hX8q>Z_3x)li*8G$}jX|c?!d9 zmhpv(#uFcWHp6+*^5iJi1&Gxu#uf9-`wP8Zid}aag~1pyk;c>Wu+Cg&!)t|iXt$x2 zok7MwM>stZZSh6Q7{*;L{>3mQje-L(aDmM;Z~FU ziMC>Yi?S)ef&ZWR*2hcg{|^z;Pi8TX+0zDoPcsmLPJ?&Ag&JH0Ek{BdD@HOeOS~m+ z>x8+;{kL}z7esga6S9QEYGhH%==MeY7SAx6wu$ri?<2Qqa%q z*CO19Qt_3B()ZQ;MVONqiL)(ltNn2wmQFpS0#tGPrZ9f?k6I3`J99-0WdEltufHxe zaLWRrR5dN^>Q*wVtY>fMDPE@zjt5m=6F(I@Y1)}=zD#{{#c(Y&HNbePd#AO~@)cKz zkS{n?n_N6~J<*A=vm9kNQ>*fBAo#Z1-M6hdYn77Mqx^@bL= zK=&|-1C}<4ff+ew{8b5E`sN9FxV;6nJXY*y&|w7fi+(un+yzu=&)Bo zy!%P?3xeK#+1iO?wp)^KvW;D3xkk-6MTI6haLCx1MVoaI@jtJBt-%jyU)%DZm1Hke zZ`zTD{bIbM>Eu_nP#^f;pmT{LW$=%=8_L5)%(gb;!nSe%eOMh*W3KdKG_1J5r%M&} zG%S|R+0oWcKXu}H_;D5JKgcN>5O~$kAZR2reqUZHH zX2F(eV$1(Uo0d23+blvldM~~`Jr-xDKH;MfH)c$ciTQ>ld-P1y9U#N{r|9PSmgOEI zT}N|cATceiYMbSVfan^Md*^6)8X-}+=|4_O7^{8s0fTMaDo!IjW>(S&np^Tn3-yY; zYTaY3f^m4c+Szc)J%`7>vtBI9~HHIVk&?Jo@Bj}dHig#Is&a#G6X8U078 z8=nz%X{VO#h6xm?RC4Qw$v>+{gUbmvBjuAKb2$d3iNzW_bL~MMEX-B_f{&hK?iMpRulyofOiHT{9{=LLq z$`4iCd|a>VSh4!Yq?k#HQ+*BRuH;bcV<%>uf=`zogToYnn~5>74(9blA#5Ma|Kw8` zMd3t8i@EYSQ8PwlvK=m3tRPoC%@~>-THI-PES0j735H>Ot^9JD!d@w^rm88#VO3dM zWlkJA-~N1)p#!dFZy^avj3fxu$e!%Eq9U*oFz)NoH!dmPN5yR0Pk}JeBDuB1sfC|5 z(yjk`x&DaJKOSAX&m%85maYK*gL>b6g&GIYH$i*Z|F`tt>uYGE?B~3H%=mMum4*F= zuqP=(n}4YJZ2nzg_0-a;K;+VSi!2GM&<=HPqKP~cD(Lc7jZjZ5ohFzY+8O?(iT_U! zNf_V!F_$mTA7Y|3wx-m}yGveQ)T0iHKr#oZK>o1o_Z$mVHC@ZJaU?Z@j6)5slnXcf zd(1oX-#HS-)qYjq#y|N?iaup=Zq2w>V#rKiYg)eqTB=NQ=49{4=8Jq;IMw&Ksoet$ z^k3rZed%NxaXOEv#|q?d>710;5@kZ@h^v)`+|<&TW+l2v3n-z z<%CX~P+MSPz@-l>;Pt4C#0IvkxvMQo$6)<8NV=Qvs(XH117uBe_h0b}U)q*|0$*Up zbk9;&_Lf#QDjrJHI2GK2Ca(u2{#m-$ig3J2wILNNJRE#wJ-XaFb&ZWkcin0W6eg5q z?bRX7zT0k%hw7pp-WzjWnqjFjUJUye-Z6Qov{JP5V3Rp(C_Z)2s$UvulWBjd)QQ;f z7aknL)JRS_(k?`Se)RcMnxU;s4+G|9*|-qyL)R{|Spi`6T`|Oa3p83#h}d8aS}f z(7tj1x9-6Gq^-Suc|35{M z|KIlmcakxCdXplb$nHm`qml$Zpt9M!??5A*d#AR}8Iv8gK9CZrRI-x&^fXsos8;4qTeZF6~_`Q(N!RLNc>mNc8q%RAW_#BXQgr z`<|-t8Z1aoWkSFd_%IQJn>R6~v(eooaAvW2B6^lOe#riSEiYzO!~d|k zQjRWKM7)z@k~Y*pZ@XK@QD3rPioP|$;HAaQiZ(Ey$NtgW$m7HPdYY4v?0dN`U$LD_ zE+T*c$QG33tb#EC1rN_)VhPuRpeoe*lg6!xuxEHl&qoL9VIcRVk073!9c6&U8sKVk;1f6Es1PoBTr=)2$or4H=hy0fDF`oCs0(C;pz-%%`qJl%1FF%;V!@G$El z)BI3Eq_hN&<-Yjs3?W15qUO&1j%bqQ0j_N0Bf-m#`E(q9r{8c6ik$&(vEF`lX$*4g z{(C5TcdQZE?G#HZ!qTz8RY7v{$`d`Y4``E-jzi+z#Er>|0@v|pU`K0OOL$>4D(6z9D zy}?1u;yNSe+*~e`@Nn|NQOGoo|DgPja>B0}#ZPYDlW%WC>pY=Cj2&MM<+dp^Tl5-d zwV9;8>}k4q&Rlkd_zwj`pU%FGkY#BSJ4iq`lEJf-ajO?@o^mJMNJZbTrm>+&HmJJcLbo5-e;xHkHxdJ1>8#ln^3g~y-|y}$ zL*}ESCG9L-%2~%}tVpv}2VT#L#j!VA!Rf&_R7xbxq%$F~5Du?I)XuL;I#2_F!GVG( z^|oMw1L#%nDvj&^&`LZy{P?bK1GkSVq0xX`(sP?hgwcj6^xXN?5!W)@wC@S6GU0A_ z_AhNZ;6P})*sjM?$Qg!>QoaJi8=Y366GC+}-fKOB(- zeusD)EriZ3%tfWS$eCvm?($tq{TrKg&!iNt8b?LbmuHV{bo_3d_rOj!f&Zi^*9KA( z;d1v6UnDq?O2_Jf2uhcRaD>tAi%XqOl*yRy8yP0H7?2Bsq}7^A^2k*^mng13h&l#C z{0E9R5_a4kA1H5*l!*R$sPaNOsP%&7p=K+Ca`ca5Mu88dqx8{8I`Z>U=g0gaRcAe4 zdF_LMSyQ6)YX~e7*wO0r!anINky`A@krl*EGuACd>wi-ceV{4+imrZUvT>JEr_8}@ zb@N6K*@#b7WuRZfC(k{*^OQFF@KKyE7fc=84`|ioYvAH6iZn^_6Mx{^8%eqOR<;fx z5v9~ATKt=}5<#jTqP(k7E-}d`1f8%`7i1f&kTx%TVvX{ON&Bbj-Rn=lhePmu&ssw{T;QNDV) z7K#PHI3Mz6U0XIL(hPT>5?+%qSUoDY4ogICT=Pb(EExVJ+WDsVFOc+>MARqt_Qsz) zkE1#DGkK|`rA^K*3bLFnC547i>-hTpND+qOF9(E>cRmqNy%%S#5k)Ur-hB?I>%HD> zV@po$9dd9(N@AMhXuWO{E1bbAfd0!;b~be=0*cO%M#HaM>Tk6_f`6fl1yidC1=E`j z5t#x%7X&m+*44=^Q_7`1Ka&-RB0zT>UcU|>-16}ODaYHX4l>idCJubT%}fn*4jvmG zW7CO0j%wOlpm&md`hUUbakgtH@lGH5T;4J~ny=pw!Uk-ZNb*iN_VBZmzfkxWRsjpW zZlxcf=X0-%7s5K4QuQZR38+7$WyW(`(l=fgxW`b>+xSbFhN3Gm9hR7|Bh8|f;E|#c z>+1(!Z){r3;@z(B+OJjD=FyozBGl}hG;|0O-!ZE*sjDq(g4z)M#m<8^{#L;Dm!FmG z0|g)9zcrPL)(-Aat(jinNjh8^-pEsTJ?_ClSJ;O9q-7^=Q6QAnTC%i$a&J!pRRoA$ zyGoWJ?y(rm!s?R4FT=J8LPz3K!GNQ!i!pWiHYp+VM_%Ph!7Gosk?CL@VX=*hP-chO zO#B#b+R}2ljt3T2y`cRV<(OivYxRc)B|q*DL^t!#knJg#j=BxAZVV-9pfJ8iF)5DY zho*2aD+h{Q#6gmxAWu>_Y1uULn8G~s%ca3{$7k=9xn=Z~%kN%j_rc}XNE_xm1(YqF zNI2&WcBjf!`O`(kFNXSAJ89MhD5_>K7mR%3i&S^E9D1%nYmb~AZg`!okIan~oLf9p zxuhZ~sw(4&{0q&|$a2|CS%SFTD>;mJo&Ypi^NfVJ;NP9~C1sJgie=c4RDz;;%!1x! zjr(T@X@C4)n2Y?!kUQ-s+hGBy9|sppI6E6}ObZTIW;3ac!7jBa$fYR(W=O|SJG-84 zh@N3`ynMHTCIyFwl+_3Bj4nhT#XY`!4zDOkO0PI~KXiZl_=H(;C%@N?nm@;)7yK?i zfGbu(^rI(uc;t*93})S)g7fzIUGjdam|5YS&aNG+$ZW_;+Z3ch(lqNayrAF*IOvqY z9=a-4n7|^jfUp(fZDDsLdmJ)-c9}g|;#eZxKs;ndW zp6a&zLur^Q*>j>pI^>UadW2N#iEmpI{R$RqF3_j^b@n78RBCzH93ZGL5p8?8>g9a- z&^)I4gfMW#V%$3AaM-}QlT2f){r7EOB=j*}9#BEbZHMK9cW>yMj@+Nxl^6#3&y!OH zSA)-e25T!1m%*77yx|AD4cSV0;&m1qAuNc13|r5Yw+#!@!L^X;u3BT}_Wg~i0lIJp zIFh%LNTmvxe(uaDLF`Gt?w=fQHL44zVl1s>iNl-lup9k&FG-Ev&JOLya-ki(bUP)9P&3!&p#i1H^R+F2SDzP7NW>P z8l6l#gD#Z$54pS@|P*G-QXq3Z+KfO+CPzp>D7lKG18=|sLIi%8F6c@r~dMgq^dw5*WHRb^Z4#vh@vq#yE913?Tv3urfo_bPSBbh;8Y#H zHwwgtWdq2jRj$==q@`Nbs8@~+-&?aSm;GTVn_#1rnYa(YM|;nL!_$?^y#}O+aTa1! z60rZdKNL;J*UEBnAzZ3sjV5-)RP|72RpXF<^~l_3LqTUH3sx_S<+h=ry|EqjV1s7k zRsJ@z);GFAKw#E;-mP_uYJO z)>uW9eH2T7n{MYL>!p{{c$m;a&T_fx&Ql#IW~WG^8oxMEnv#$zlvM_ECiMl3?FI z{(Fp6tCx3hRi-mI&`jrH@_1CDb1}$joqhJWQ)0tlzqj#e1BhtArcZup zRVaWEC~zOv+U5w~$PwbyY~_Wbp`e`-MC$a)%;s6yAdXw5#c-nL-RZ<$js#JU_-88l zgD>-So|hYCC9)=Uh$YXAd4B>}WVW7iJdiv~(SZWfF`1B5wMQKOPb(SkI%$03bsofJ zjZ}sM9emx1kXFb1dWa0!P#ifMT}35%MLo3;hj^}gVZ~S1@v2-(XJKz`vjQ*CBkCNA zv0b%CD}Vj0zg0Lpswjr~f$|GLn&)pYWGxz%e*xDScOXYEt^mK~zB~Q(BC~1l4tl%* zv-0;pZXK*s9(O%RmCLamHXX3kPMh_CEgd17T7tx#hNg!{I1!J{&0! z)3y5`hx73h=>-GNL)CC$IrYC{siNtHlcf*~bHy#unRVdym)1DJnNgG*g5tqH>}d}# z?UWCeLML;PRCrHL6#F_3>V?L7axEQ8IBr5`nsFg{I!&&3tQ^Wqt+%AzXHQc!IuRob zu|yYlxmMt)2ou7J!g8a%1)s3#GDU8+{g{vcE)t8wb}LHA{=U5mI@?W!irad!a@Io{ zZveE;Z+^|*&Ln%~kTA93GB8N3g0vgS5C;!@d;1HkuAJQ;1=SK*p!0HjM>}uA^nk$D zaHPD;!p;fsuu#E0%}gey8{mePOk>ZGX4@{vbGoUkug+)b&P6y%ykk91tfAtMEL%EP zf|$-9CgLFLPwY7^qK60nPj_Dx6-T>m3&9&pAh?APoB+Wcg1ftW<3WRafZ(pd-QC?K zxVuZ^(71Ct|G)P+d!MoI!yV&3-Ph_KUsa8&wd$K|&AFy~P8oJgiqg-6MqlPSvY&Xf zK51zyjkTL^{4u_by(j^~>3A%2R&(ObHurwc^~9oQgy`EV51l-%&_IWw%w{_|X50M| z(2VH`xf7&<1*)-P*QG=3WDG6GKNRc8>4!qw8O!cnAP@(@cVG#7pPCyXb|MG#qU$kXwcSCqJ ze{;l}$EW{L3(n&5&aKVY2hHHho?7cvS_zs02jTaQm>Cf`Gb+^d@f+Wx6%UEtt)ZRL zs*Du&iqq=*qGhT$8#l^{=BbJ%cIZIq`W@;*`T}s9!a&k@K3GmDO(IVcaQ;-D3MmI%5JzW0wktr(>6DS_)OHM+?FTffohTc{iKAgtwEM$rb?F0R185)v z;f13kF@NY;`}0pA3qtbB8RqxZ9-&O-msLFeJW{cNt4tHf*Yubv0XcktvDnh*+4Ibl zXYm#3@K6OSp=_%t0l(P!SS({e@U%I#iMzwdRa0!ugGS)NmGq@Au7n^NY{CcmPWeS~ z?@tk4luTGJU(~3wiZtz+#P!a7TCJOaffTvV{NA=G|4U4 zVvmn4?c&4w&R6)1_+f=aZFZNCIZR^l<=Yxf?mT6*jp&6@V_gys9O`BKo?1=aZCBeN z%ua5+Hga1NbGZhM_~UM?W=AE00qCqp${#5?n^wH;RJX5MRLr;si=FOS2p*W~o{t~P zz=C?3yH+OK>4JXUy@w zb1r4{Z1{pkN-P}lx{wZN1BcS#F;TUDb>V*cddRw)uP=YNcJI-y&rLb?S)?n&u~6-> zdr2a)L2KxP_++6x&{q(i8a}}1=f>8p)l0PUt%3p7gSSyP{oaUfclss#1vH)L#b~4f zeB4ToG!e_MV@gmpIH3OyySI9CJ30(sG2k2MmMVS|`0@JzLf8XW2B`fhj=nuc=ELcx zVX5DDW9Bo{11^{2N9^0U=wmt+x)&QRJwDH=UWcP`jQbVXmY{8FYNOax7Np zS~3A|HGs04N2RapGjkLc@n9I0yTDk9sL2{?Y@q6g-o4qIkSvecUh#~V6{pew|y zy{7EW4~0KbzEdL=a&jnXpcjOzo}UBzDyKN-Z9EpqBLhRHG)ymY_N;#w>2E!26x?T1 zHRriDat&EqCm~~Hrv!jgI-V2f9;)6hk?@XZBL`fCw9B+X?Fqt(g(GY)yDc-Lz`RLX znGx{MIY_lS5|vD7<57`hAonI`Y4o7(!Uy}=hDJ}!xAId22I(6^1vp~*sg#tZBY>Q76rqH7`H z%Jp7t$Q?i&afLN}K+uB6)G_6QQ;76>G7Je2!Drj)=(T!_;)J>}y0 z5r(q;svo*4w-(RCW6PCILwX=8!BlC_w&Q_z!3v&n z-}+EWXJ8dsl5-OrX1ixc%16;O$zW5D2|Y)`oj`{(;ZyK>>TtD`~Hcd!o*rAp6RhpMJjX&WR$sm2$oNFD4*un86wWCg-v}(|Vb8KnDB*_#IKuB8Wi~{j-sg8z#l=w=fbP ziWqvBj6KkR8ra?5`2oH}UL5$^ODes9GvkqASzsKgx zXU+F@UxlJvPwB%`KU-KX?t90FU-M}8&X=NJ5=LtkOn~~eu9zE?)*35qG^{;$HKry^ zqiigiV%qfD9{~t&R+Z&5JN16WV;x6(A@H!$tgzbuz;5U=mD)c2(}Lh z3WB*`s@6#LB_8AWg*-%cND6ha!GM=)q3K-Mm5zP$prD;vm=fR5?7f8hB>7e)lwYG; z3G_3xsWZLF&A%y%j<66^ZfmRc=BP)A6T>ZZ*tezZS2i!TEIUak!B;s-H4^N|LdhU| zLB$qFw4IbvlXmy|p%8^5;iBYivn$Q)Yt3w!l|i}S&APPYu)~9%d6?^@GC)WyKyLie zJ2D_lU@IFg@kt*CEcnEq1ZWrEL?w=Pb1*ZwLMd z@S~2{eM6I@5~p(hSvFR~$eR7WoG7w_e%Zi z7at0}m(qGUez;6L8z~)N4Jc!j$Ca97OGZ%6?g?SNR`d6!GFRmuXkAJxKok^d3pM7> zbMLlgqAEhdu3|g@*LCV+>q+Gqb|Fg1@#fF=$nNs*#d)xUiV6{Fm<#RWwfMv(x`amx zgpzHm-(@^eeO0iF>ZQ*M~nE$>6YWGMdK>Lh@ zP2yfk(A&&ok6~9>9*DO=q0T9jk>9ofno>1>c8nFJ)}Kn6061b*F<3_Bq_=CxG}s00 zKIO;pIQ{DOzzBQ0fAE9+nns8%*S>n@hA!Fl)3)$u9g$t-?ad=egvY$;B{eA76aJT6Ax2^J7jI4viJJQ4eO5i~>4mgjL!z?rD43ks89}V-ax(UJPc24Q9QGwBGGg5x z$GgEdc6D;I9g!$46iklUzQhjgv{>)!kgXry-UH?0+0Fl78q{WnML0of!9Obnk@miOG*<|K?VdY-pmE*lRUZ2s# zMW|V?+2mX85?;?hIpde=+oE?ZDc8HDSw)cgl+_@2X{1Xe11JtPWbM}LsUu54Axg|H zqnxS0XoQB};rAYaa3zI*Z|I^)zgQrP5!joc=JS3DOQumqmh>~&QWu$bEabC^-1Z_bh{xF5Ax6XqvWj zZo&*WAk<-u4R0!sO3>vYdYRLsh=P|96jy(NSw;mUZ7nhyp{m)eAWkD-6iSj(>;d#i zU0-oVxUBl>KKO?b!Ua*36&+QF=7k>1)hd-DS*5l3<^GPGU{n|J8c7V(%F711k1Gtt zM0Dv_zA|g(k_R(M!@~q#8Bd{_0*JzLa|sx8@xJsB)sYb(rSkj|6IWi4S(aUI`AMB& z4Z?>|l+q8C0+PllYxcD9&5I~Cm|raEXOCqDY+3a;cZ#W{*Y!!tX-Ol&R+q+JFJ@l` zVo`H-7CLtrzHi zVJ;r*j~zuvV9kYMV0VT>H%V)!jbtp|Ylq93Tp=-UUOyKCgy$}Esz}prBTQq>>+O%l zd3qX*V|8UQ0`+n%dQnP*MzrcU&Q(97wJXHy+Muy`6~lXfzk(&vdkE4@ZXau}m?<$w zSxg89!gzCKL7a@8IN1hY7Gb{48iL5&te{_>m@@Id?Vsh@9K+NLxyLT1Xd%LxVn4g4 zm3&RFbympwaSzT`BX(fZ-Bl*lA2V4@nV!)x^yY|dujePSp2t(4p%_b-+IYr1pXt$U z+z_mR!gBYD+`jDPQIy7xAc_PonP7dk+<%>`w;5jUD@Aw4U^3<)@^p}VILY=}EwM(| z+^g7}9{-M+XtQxquHn(FkR|fEcQY~wc$oT8S98DdJieHgYJHYiVAXXSjANdq6zyO| z5#CH3RQ(g)Q61m-k2{?TYOzOq&`>HxI(WoajqXhyV zoFy{mQexKFQj{=jO}nAq$c832A}Ik?>?SD`Ra)Ix)R6@3I3kB|Wu!VjzAnoZV2{1l zN98TYoEBQKr}BSCEBMJsBZ0%v+mRGVv8`5ZegaS}t$fvDy3{D%WjC3Slfs={yF7g( zcQ;Y%jLg*F)e};)=o2vWS+8SPzZT!lWbfRh9C)fy zkHbZO_TvYM3a!6G|J&^fYQV!#*Y_0!nf}7l&R)sw>vuo&HqtkotR$Ov!a;PD<2 z)wf=;)o^?MJi7r)-_iUUn&#UGW)x!Nah>3i$uoYT)JS)vR7kAo-+TE-%#P z5l5n1hVs9?*fQpm{ndLs-*Rc!9&z43&SYTfEC>Zhi?qyEsj;5Asq;}DQ#=uAno;`c zN3`5{_r7{^f7$?ZlbJPpfbz|AZNzb57Y`9?QAyym^+R+_TI5NWX=lMIiWg+2C)y2+ zY(NBh24Hup0;x8cI7kvC2%;>$6v^dP`KpwMp~uF|tlx)Jyu*#5(SqTh?SPxAceM-w zyOuR#Zg@~G{7=~z*f`#On9s6O^65#Gu1+-Kcj~}>kHt#-sVded zo}mS@LI;}}vN`>elhFdsK=;mj;zQBAxv7>Sm`m@Un|Uk*vXE=IanzsWoCaIv`rEO26)YCUS6o3P4$S0j8!~}mrfRsJY+BJHzc##P>^`eWttbTXOgZZO|+yajdOq(tcVHztqV6|!41(-jiJ6kRSL z=C>{P5(~)kc6K)fxfEY+&&;YRvmv+e9S`lNak8JPotrnvdO&QOw{prxJXZPx>Mk`h z(F?|nNU;(|+qGkYQT*X_D#v-tP9Rb*@Avq#kJ;2t$z@5N6WCnad)uKe#HJ(e4J18u zJ?|+{Pe*{FaOkh_JZ#qZ3}PkFL}l zq|U`)2iz&J5ehc&Dht)cV=L(DLF&V>D2-S{dY|LTCm7<_rV;nAI(t<%9^|!dQ$01jWKyM|dk)BRMi~jz&8QC>84}yZk_a8c#9bV{JRO%Q6Jxww#gk5X#Fop& zokW<)y|=8vy|&u#$Sy14yTZ0>ZdCOdr)1Dqk z*f|dy!plrzm6h4@lhhN2y3N7&v)VJy6Jts{n_=FvG!9b6#Ew8RK8{8Bnhq$`9U>`VC(^Y~3(2I%a~A zM9;&m`lV{tKc{>r|0mV&58v%oUSGNFq<4iZ?IoHa*S=W#@r&MsR~Vu8KS-2+xTM1O z#$HYTBHGyUCHsqU|Iah_!u&4^A;SMsb)XYMCb0g?g2Wtu4^4g%B#o@_=8+t{^RMML zhz7iatk}!%%^Oqu1Lvr{fBYasIq)#iN>_okm4<8FS^;3V^>F8~C*+-qxyt66HN<9M ze1Zs%@HbuW*CXjs&G@_K9FlivnUc-@Y9OKm{z~jKH%@4M2^pjF{7-Y_>O=FEW{Y%T zXT&$1$enS0apB;RNY+?iKw1TN4|mYz_do~7~Ifn|{3kLGi$ujyD0<+xke~J^< zh5uq#o(B5`d`r~Pk6`h-!PZV>Zj^KhpP57jZ!DA2O>;Qbp7@v z!r8C;6ydWu>)KEKdUI`DXP|g+XKV($N_~H-Sq3Ze4{6TU*{8oA_^oifjlNfP;HJO7 zd6t@Q3qy@25aU`4Uu(&AN*7K8EYaHZg70mNt+=rzz1^JUu{`%nztgZ

P`Gq`WJy zmFT^R##o0{zxsxcdm$xeOAxrW0Y+XtA8y4%2XSikqfb(DSBoX4MoOGq`j)`%Ae8A< zp|BK?F+W9O_P~{QrLX;)-nR#3$qh08B%_5q9dyzjyv zFUA%?MI{aEoh-+Xe&Fc@ySr*Q&;&dWry-rc=0jpDyAx`VV$D|^@mSd&+%Ki%q@6_v zym+ZoiB-NGm(Ocb+H5;Yb)({Zc(=GOB9Cb-hnRDHrkk7dap^ELiPX;EdQ6nUl{di_ zV0qnE+64*u;n(C8i@<`x$p@ z!PNEn*ZjN=Ii7rFbTWb~PVD|w-#Z#PUcs~g97rEuon>QabDG?)6dho2R@kpiq?`72 z&hkpzEB?rbZzOgJo2L^)Br~|Y21OcF@~DENM5Tuuu7`;YWJ>(d$M)FC=}j-Rm_HJWc?bHq^M$Wa^Zj?NexTr`c$-}eW(@?h|kJlaiAa; zITocb$L^f@G zc|+(8@e&&qJXM$Y8zr<2-WMvJ2x^oQEJ-paNGi03k)tGtQ1R2avv^Otyf41)<$LPx zkQee?6f5Kb-nTpzK^nyP>y5F)|>r=3=s+Qo+eXadEuSLPAvLsNAVh`S5zb z{+;nan*NSyX>miPMbGjIX#-@8F&c85E!6^?V~@vk$7!7wwN&q9V9 zsng}|0>9{YL&L{C5CBDb$T8LJWA^V$qT$D941u|{7DDi6pF?w|zl%jQT;B~lR~x`r zcf%yV%j}6_Z+oK0fF;KCk2YsTt={cFw<~Du>GR%>qpf*1zB@}G=2$L{ zPRZYJ-vdaBGm?r_lpy7L0UrUN$8qhfN=?T0JU$Q>k#yIEdQ9g_a|XJX$XStX_affl zxLADJR{$T+mKsw}Z;$KOrL=9-JgTx~lXHxvW(@u?m0N5>BdsnU_($0My`tw@thvLpCBoA=P~2 z+8;c^PZBazS;l5&i|&Th5yx&QXBdbT;?Z`WPRi5ZBLZNnIH&4d!Y3nBa;uD;xq_D5 zp2%?zd28BIX}?V3uS*5`1N`Rf0xiF~<;W&;i~(WXDr}Q0>{znOMi7AIjAvu^bsyWyIDI^oV@s$ z?s|NG4OimKZ0mV0dS4LoO2z-WH?9$u6v}|bqI|~eFGDm2amd#FZuOFko~|P>EwTQk zsH}rf8S)Z>9Q+{LE<5y8KW!v-ko`j8`;LUatFz&bXv#?Tg3i1_N_qUowNZ+Q`UaQv z$YCAE)Ytj(H3tFvZ7hw(_?*c{Bg5$K%l_E{$MHM>o2Meq(2DuqDjqFrRu@Kxr{)j= zIm#7YR+r?_RC*OQKxuB0wGZkeS?urMLQQ?_PPw{1@Rr5Vp?wut_MA0uz0)w?FCmDT zES?so;~y%ga(gDLp&7?hkMAloW2g1MWK#awbE>U+!2d0*HZq z37=Lz7H=dINO`yw9r-;uqKM2Bew~b$*J&P!-(vQndBlDlBuTXDJHwMaq20A&D$cCj zLYdY|7-E0>O@#}|ocwXL14B-|Y zU){I2rC#M}r{~Epf#3OxAhPx9&zR%zFOE>f^H&ABs$Bl)rVd1mKocgbNhHnF!8fVr zOduwp6JZABUbG=ojgcH7vtlal%mR#~<`cEPRw`cQJ8$}Q^qBYB0DyQs4V5OaI%dlC zC=s^aWG6P+#CS91Q~f><9>&2>jJ<(SUDuICmw2TBAI?M*OIamn2^t&)rCU2Joj)uaDOrc_i=)F~%*+|jS8h@5CyG|8P?(d9mb@ z{>vc%vX4KUK?ka6J^G`ZBkF@|R0E&B`)U#(SDK|2X>rGWj$D{Ow_2&fcL2~cpBBV= z#32y3yXm@}lJ-9oK{4>1rj9vi$zfDZ`cCW%v>&PjkRu!7aX-FU>-p#kx<{O1rvU#> zR2%B??R?9ycU2Sy-H&51P2Unv6A`gO%p=`X| z9{AP5*{_NLM?-CXDnv48opjJzxM0y>F zUAv%#3eVfzHe_>%362(y;8OEHfNAgRdOf|;AIa)}Yo`8B7!Q&V{`wD(?Z3OsxNM03 zk@WuaXGC!4U&8hOKCEqi>&{VR< z8xde3D*8=YRP>9zqn)XRwFwjyb&Pk6fK<;{f0c-rKQ!G?bMTgM$2h{cbCW zdz*o?m(l`2K{1m3ecrj{3AwyY!a7OIi^J}~!G0uvr|^fp|7{A_NnF!O)Xv7n#MTK) z)X~Ji$;9}JtA&&K7fEUPZyLVHI8abuprplwRo#|P*PNX&HSh;O7mP!cU;QW=TRzHG zetqzV<71#m<@9UmPR z1RONEX14KJu_E?-75it9=+DaR1x* z+8JWu?EeLugH}qo!QcNbKA#^{-{!xjbrrkq--i2EJbv5Dh&4M?fHrjuU zkok3alA9+%(_1Uyv1&ax54X*&Bl!f%wUp}HZj~@x*Qm-%unT*k4uK*HN^7WEwn0hP zsF2(36f5UAOPTGvtAq01%AFVn>_F5?56ldF)vlEpXcSk)(4=xj@M{?3f{NY4RInt$;I|5vh_pFU zGm#lj?sSsVrz4!Z4+90}1x$mMTSblI7CIkLP*7xQb%Z}L#-HB6q{t;{30Vu`I57tQ z*N|d2y|2fw+?ojAx54$M4bMIAE7a`AQrQ%ox7O?lKsIaQaQ=8!oyuZs{l|4jp|sh$ zheVy0)not5zPJza`Hjiqo%|_2dqfu&2Bg5du_m8b-$}n`O%sg2uV8$ zc~|T=27oe=oU`Bou!2=uMcT23zcTO0j41O8{u)clZlg^;eUip_Y6xzEjeH<=>9jd8_0W5^oX2a(kTZnFXCsNVat^vQ^i$I4r*w9#t4H1EtgAkBa7 zOs8~XC?CEoV|Xl35(jce2ihf$=4yRs)7cmxCiBv)d|^*2#h%_RmnzB3pvmNKHN>qo z!V~Qu|8c)$waS4W&8v7+km)ns)-hi_Vj4YLBiHNj$*iR*mRRDPLg|r}T)#b47DB|Q zL#@6J-I9^E*mA>dJe1ES@{c*m&TGhgZ{92Yv!o5~L|15^Ua)fx6$+M>^KC zF;1F|P)Wn9|C+z`EbceH>`!>@I>n)9kF05ge^6Yj4R*>PwJ40`j6bR!Esx&UXEY)G zqyeJwJ|Bu+*zOIXvY|vYCy_THo$AO;xqQQyX-wr7DnVI;EDrm#j~xnBk~E#48vzeq zN85~}vn{nP_k-aAVWN23j-C=$02TPhT!G_c>gdVUco|Q3c`A<&1p2JedeE+Du~o|h7`cyZB_0g#FwLqVXn)#;{fXH}f#jbl< zMKmTG3)-Z?K;g-m3vY;yDM6FssFUvJ)f!})Pb6ls2BW-~!%YaBj`;6%5IKU0xtRLG z)JV$!Cj1EMRD+4hyA!>go5)FwFb2r7BH6o*%%)x$OdymnNHfL!@_=KO%Oc3aLQFU)wc3AFb{d;&q~bs@k5rC6){C=tRbRhddo! zidartTS%dyjSwJtuyJp@<5Pgf`Td6^B~(^C-!rCMZEO8zpG!44Skho_6uq>;&u6AW z??qzbTsglm-kro+O!zFK9o5)+i4!wA&aK_qNI1%nf%1>Gpv$+l=npsy1(@aKJ3Rvt zRT4|hOb`4Pe`Of05#u-RrGC)x#4zA`&ka~`v*hsS`;gxBA@G+QuFkq4EP>RqULc-L z^A4MXAbui5Dxuf3oH#Y~!uP<(TQEFF&U__m4`{G^F6f)Wwm;J$_sr!Xu996WcbOrw z!C6P0N{8%R3BhqC?s$0Eyaw|!rjk5N<55QHJa$(sg#F`G#3b!>KOyPN2c?>{6)icBVDt&W zPUlKX5I~EFmIiR|_w#AUiHw0Y`CmVyWUh8TV$m-4u8Z0?ofIsA|>5?^gy3R4}l*R^V-e9E(N}e!a_y>oTME%HupgoXmK2(i(K^- z`kWSH@h_Z}A05sgC}I zkouW7_71C;MpHS$mSmC9t>5o-WADGT$6P~k(sijb_c>6OkHh{OJmjxGm4KwLut@vb z60EgX5|(eTw=e@TTS?D5STK53}%@Rq>n!_4L8YkUvK2Wtu% z(F74K$CU}w74thlzG;V_JiIKXC@3hds?6GVwprNW?S?)clGNya?*buq>5D7Pyc#`2~tdV4a}n3Zo&?@`cy z3J>eU42JCQ5zuOeao;I?-XF2f;)_i&NGN(74VVMeG=uhbWsQp;kZ$3rdK;S({1!tahyV}#6_>-D5pLYXd=IQomZW#kzEdS@IV9ypG$xb_%w zshcdkGc4p-Gzq01Uh$LchYtMav?#R775l`RtLi__io7{^szSd5Xjk(tPDMf{UMA>l znBu(xktt-$C>^miLM7i%)pk9pyl#32YLSBeE8Kl@`g&W~3+#Qh^9*JEIe!-it8)Gx ziq{4ySJgXzZGu#xs0=P5M>^)8KYvm|6b#1Kqpy}STSfB;rTN3+CU}x%H535bm+z2? z1t#Rl1bN!~1g<1ix-D5g zBd?Muty6k&<&y9irZ7`ElNc8eOO9u5H+EpS$h4y`Rb~ox zts{+S`do4{x))E{7<0i7uo)=C1igo|v%jjCqyOF-%drw=D6bb^5qOhrT6}OJdW=~e zlyahgv3)AOmMxtrq}hP7#WxbQ8Q*82$wKDR**50GYzly)kp;@CR0Qyn{VV?Othj*& zUW-YHTX&Dnh?bK{@(n7=M(2plrjbQtAml~c@?4`7!BdywvBg{6!RypLBtOli;*T%& zJStXq=*`9~bix1HMF#JH;Us7WW%bQ8Uff^ zIJP&c?RkAmr=!_7dgkgdteNzY)-Uwm2={KPSpR4RG0&_oy^)_kXrn)ZfX`W5a;JYW95yv`M*@i zo$4p7f2bI9#b4k4A<+KME}gFTNt~~>42-35Redj8wB{67`-cOu_2eb%e#0tMCOv+` zzR+8R2Aj8x(Jc*Wwaqh$&-Hxs@KCB$v#PS8VFAq(p+GTXL;hF#b{IuXo{b_Z(B5h zJOdK>U=CZ&2>M!&C*u)vhX|>ZSJ+e7l=BTRl$&-*`pmE=tib!swpZvb1pK1kn7OH> zy^VXiUIKta6 zw+JBOr?btPjmTgR9DLVzF*Ww9T9(MSEwh;RUsB_qM7h$ znVg?f;*e(^`PyK6f9=DEl{kKHCA|AXg|bqX91~AO9?n1sqjavy@2w@Wxxzkw>z&;RO6(6o7(IpS2%5Cz85~ zMO-(~GQz7=xgcyH{Srw_em! zS+L0zj>!9Ht^HG^=fPyb-bK2HVbdhW%%v8Z%pIwgeY+=ehr>&X*XW&vQ)AG46*v$n zNY&W_SLp-NMR44=+JlPH)y9~rITb--{PC(rUBE{szMo??M1<{DKeZ3hykOZ2?z=BN ztBDBLB7oOX+p5hJ-eA{k)2EVBqPaxk`SspS5Ro$vHxj{0f6Qt!=1 z2+6+J3k8*KOm`T3EPV&S6+FS5H|k(B9CF{zwGy7lKdRcAc-TKnZoVaYVe@t(lL9I+Pb};-m2lp9OPqk9-4p zY*cO76Efm70PPZl@0s*^2=+xjUa-b9rhk&OQQB&=q-jm01 z<#*><2Q`};TNu}Rwp5;5ma!+I0Y!#@y5Mc{OB;Xf#ub~hF2Ue*JD0xAODIjp-S$e} z0gv6V=!pSRk5PvYk?eR7T1V%CQ;{Y?*bo9Zg72`-l*C7`SZ2p(pLJ)@+c5v6do$Bn zO865`9HPco=uKJmzUcPHYN@84j zn+v!`~<4&9AAs<()_(c3L^ks};m3o7uRrW-gwpUy(*) zGiqHZ%k>1iX$S^Y*5e-1<|_-mVbuFfK%hiA=YBIqec3!ZtdC%^P+%9V(O5!)?( zz=_oLQ>;aOs`i|x1O&USGxCiEvKx!~7ZJ3Zk(jmEvdiGViYK2GKsTAbce(x3sF5{5 zQlb0(i`8)z>}BmLKn>_+^j${Vx&)&1iIa7aLetg3wv zYonVpE+%niKi2F>MBQ#RKTv-RH z5%7#ENi!7JVP(5|Yj&W=SB)htdn#o;awB)PkavTcUqI89N`b%B=!z1=33fEzv2?k~ z9A8Y|8d;<&{F)V0Ck^U#sO(j|AF3bFSR=d9cU6G3A0r^Dv;t2!wpctBoKxX<{J@a_ zrrm}>8+1`b;4-#8zVb7-W%;!ZnRd}+PvM)|GslPhWlHRo?msqAP|ynXAX1;y;wbS3 zpC!dTEpfWT+^f6%++<**-W-H&W1B~vxhOY?rB6A|)7ODvWTRtx_M5Tx{976X$p=_$ z;74C1ZR>wRg)GplZ+QsvPIos)C9`pv#t^aJlpbFjk+@axPhmI0!q4X*iEv2DkwtM} za+jE1PT5pvPGsk?2LQ<8O5o_Q(|5SVRwzwYbw(zMIRP7Wn73h6+gJU69<1TABvZcV zx+(FVVw1Z7dd9{gj1#|=YF36BdE7jY?NE_gW|N%auf#H?)tU|i@v*~T5bSrhDQq!- zib{~3iHRxpx5W0M!~B9kDua(9xinqS?P@CLKp~xc@dFL3*%LVsO75>jc2#n4-)j3; zQ(E^5+bik_Rr}?#C6+uPHtnPRaShCT`(H$5FrHtIIwS$%tF@=v2_4JeYHv5);jy7e zZFtG@Ebh^e->8OEMG__0TE3kH^EDb@^53%%+PHN5 z(UWP!tT;HDscnQ%DmDl2iFf5GBo6cJy-srCWAHSZ9bO1V?f5+r)1b>V5P~-J^pwXs zCSKnoyN~y?_8K24CHwf{P8O}qF6^3 z6=aCa#TF%PKJWM{F%Dok4|&JCWiz#fGl8w#cz8(vh+|MJ!bWx7vEB*E&pFuKHjrX~fX`ouZC8*$@&16m@{-KT! zKMC=7oC@6rtWkZh~22T+>$(!lTKc zKtA#8l+D-X1rtTwgng?YDZAQBeoqJu!Y$x(Z@L+(`Xi7iWfI+mucCB)g?K%IxS0*)$7bc0#@EU6y2k&bMsRN5jW+a1m z(M64)60~fbl{wmX1p;52%g>3A$R^Y@EAOLY5_f`(5Qzq=v2q@!q`n^W zZ%rIDz@n@D(Oy`)NpAd=fnW;y?Ip>p@#@bR_u~GU1)AL|j&k=Je@q{o$R?zqH!P&` ziKCCM?`MiSkU%g+9svA-p^!VK7eSPl|aQabp+U@7*Cs5zl9DOH2}b@Yv2xa{T6 zWM*;f+Xo3}Lw8~#I;(}G8^j@^b<%W4F+zcn-T0MjI&vq5&`T!7uaM*+ZDtb(w&{*A z8=?QC6x=`bGrL+~x7&({Gcytmyq&%=a8t<3kH5#Vm~JH@f|G|l)}0gRjZefK)HqDQ zF|Ji@qw>L7IAUh^L%yr9X{<&OAD9^`v2Q~l{e(S!Z6yd=ZzuYcBwrNE_6N^Q)!Cc4 zrvG{LnGhBZ5x&dPfmAC{WcERyuAMr48|1EbOobk)FAZrJl&)AFuQ@me3~`e~>z6lcNdUw6TT2adY4R2YZL=5oE=+4M> z-l;&j0M=>wnk?ar#8-nb)zO!|X~*m1i^L}|>_YZ+d;e&6+CK($N6>sDtT&A*#3v4D_Pa$+QY113Z!4YZ zG=$->`JLXSOO=6e zgyW7@$B4@&8~~%0@5q6|Lk(VS8ny!@a4;jD7;Lkh%=!n1Pog>v6klYexzO=`R4dB# zCl}F7wIk(at72vhZK;JCCJ{rSy1B7Y`6I@fGRaVCV0=by%mYwJwyW=ci&G)M8lb+| zxium=)$YJAIReEv3~6U@x(h`C$--5ho6g|pHIe_=aMdTKs!~*~ zhtCsdM{#H7w%Seb-~vvfZ1r4Xi?##LN0R^M%a4eyndU^L`skJ|y~1RrJS97$r`I&a zzP7KC&4ClCFcML3n~MB`$KAt2W>**MK}`mZj;-)6_{7*(~-x$p&oYxSpnWtnG!nHg5Y2#%@W)X0>2ohQpSy7=8=u z!UJnki&F0N&P&bf(irvbxvlDP+YB>e4>x~y;8tcs^c3*Pqo=@`AxHC%NTAuHUHFuC zc`GyaL?5ML)&%{DW&GHQE#~OPx+b5mKflXj97VXmzX$Sm-yq? z7tTxQO}zIm4b3`TlezPu-tgg3sr8wwe-o=9lY8OS^R)_hA$)HP9*@X96z_#o(2KDG zXHaS3;npb`&Gf@(m6KgAcoSQTQQpeQL)EpFOZsoi=BrZ+58x+*ohWs+aKpc6Xobu^ zuj~E%>VH4V=^@tJ`t9ggf-j4qKfyeC086qDWYo)0{i137G?^69rkBVC5%{RMKZwo7 zgikdv>v?nGqkDr7F_$~ks?7$erEP>WXb?xKypDym)Q7XSmwnEJ#&^BpQ1~Ub1|&P} zC(RKb9~gnGaOx*iY)G~U{5_kPg2E}2xpzGc-l93yT?0m0NrYr*5ihSRJkK=y5Di_| z4tmJ|T;KkFma}j?6Mm%o)?eXb>4+;N<4U+rH)o(jbZ6dzIzN3N4(&SLWp@Gk0HlA zpHIDtq&z3o?zxk-yIab9oaC)vtK5oih-T`?j!zwR4_&&D#Fc(^=6TIs&M2Vyn zVFa@R?+C9F@B7Ij6I6_NrWd-YS*(s6_jSzx!PTL_h=!XX8C5=}ZSu~!^?=)QVgNSr zrlr(_!^q-~FM^^T(_egalvQnH#|T)uDmRr1xO6)ld$8-{l*T&D`eGI_kNAVnO)vl+ zJ=7jZkT6)LA-*Nr22Wc)yEX{6Cp|KyNcpHqH!Ccy>QK;02EZD0Bz^5`A@4Ai{fH}>XdT4E+=j6B%6*I5 z@Z&Au@Y;Aa61wvvNDfFkkf(zi+-RnGG)$>P82b81HcW~Na4rJgbHUUSV7VxvbwS9LvD*-g( zE5bI^Tb3U3?ZEr9dBdt7$rp}4!ck6QlL{=B=rzkvOqO1{3jH{?IDaDfmb^8e?t& z9149A*@T)cW=Jc3mMVNM?vNi8^jji&Dp=xw7e@IBX} zEOr(EB-!+KW@8F@K(!%6O~hb_8IhroIE$n?xnvu8J_L`MrGa|6p35D5_qXa+ zEjK2z=~A}Q12K26_*L91h5Nx4+lsr!DMVU)4<@7N%n^pK!xB-*1$kgQ=jb~ZMUo9{ z(QZu=8|=7YNqhiYVwC(u}6G3rFn7FAiJ=QU31&X#z&; zz?1FD{I7xy2_7}jjc66faVzRy#*2Qq%swTIm?_#vJg;WeMEe^=cSv*LVTVa-4)qdf z-3uj->kUukM_sVJ@F=~-3n&;_X5cC*H(_M<^C!R#mOGm$njPyvX1A}l^pJiNf!^IF z#2DoFrT)1(UbZ)1?q2DJ77M|@n*6!oqqi%)+4%T;qq}%GMma_P@#uv``e?6i0h{$n zLkI?+;?-|G8G@W29S%c`Lcib-dfg9sZMhHfRMnVtnnq(ac2CB6YTB+{z!#`Xu{&0Q zu8e-ao^_U%qL*+K9Cy8}+ugcY(5DtJIx>2QEk|z|uOTJnEgJg8GX6Nb!T(#3!kMvU z6lu1|A;Qx1|GHK~D7pD`^tC+Yr>-~k`P_lScKzRK2@*Y8T@xbA5M&8?QRLI2&qqXr zBS%w@8}s7P6njo&z}~2qZ5(U%U~*`>dyh|ir?G-~{;uh3E5_g>!bjuRO3#-s&)?x^ zB3|fqE2G)aoXKPl)vE7GW%h$1D@$fn8uOR5MLd_E1}^Y=rD9{`81sYgV@R@@@jH(? zkfiV)y`?g#oO{LNFr;T%y33(#^*(vj9G(?NIb7RC^B~Ki6MuZEguX^9)3B zT;qIm?ltzncJB<7b(uW(?c34B9!@c1F83?rVp&+JHf+ncR(@%%}mfi z+QX5qFBE#7yeL8Eer$XIXCr|jMF8si@qmqMwZi*c3-4agK0n$a@x#CVr_1qbhxhKw~1qB@+yY z$RMS0K&%W5Xw5Ni-2m?BSBxwikPV+*RIizaJotpJrgV3>i@u-E4fb^y8qR>lV+rmD zUR)8pP8yZF#z6Ev*IxX(P5ro0mcS$AK)mU!6;rx-R49abuw@7-x^r8HNFhb5h{Xq3 zp<8DY|CSFyct12Nxs#dyyUG=6mENNHjg)_<>o-8?b+76&V2-E!0kf8_3ccn!3tb zuQ8|>Bx+19Wol;z8gAM(L-zHz-Z~ULU?x8U5~b$cVQQrWSxSC7lkOU2l`jEtHGH7H zPeSuARlc0!#R3fvw|g5i4#$*+`_`kE7SZk-TX$n^S(fCy5OYHoxL>CL!; zx~;p{L{SUue&w$5XBP>H>U=I=j$pt5Q%tRs>{CSYXsf*Jkv_YifyRCHjvYSS^e>$8 zZd+kPt~WcpDrE}S7YftDoi(D`ipqIk5|Cu3^?ek%19m<`_MHwJv_rZ>zwd0*jNj1k zkv|o2ne%zCw4Z!5DS&cYjQ?)ZG15qq)T0UfF1^*f$67%qPzswJ$#0R@nBe3BzxSq8 z${~r1?;~e}GG>$+mVMtwY%fnI8!QFYYc;=7am+p7LSgLiOia) zm3V`yP4lmd5m0_2Jv#>-*N9Bb@y(SG;|K4$Pus?w9L)5+vRmj!JC3%G)Re^D!F@XM z@GwTSIbtlnBTwIIO1U^%1l=K;{}bU7wfx%vzKZ~Kbo=;1RrS6|cACEa@vSAiv8ib*krO!|wR4>Z-bZuyYpl~yr}3Vi=KD8^4Q?eS za+9c6_5$skBM&7VR0%e`yg=4Mca+|TainQnNY}E0$>bEa7KHAHZn6I0b0ItcfuI8x zK=b`{f$QTJH1~~{Km+8iC%voN7OMs(J_*vd^Pd1b>>NM_^;{uoWO7_DJnln*y6LT| zC9z?u76fo@c-a!vq)O|aX=s*_F=_eE)m65~XzKN4V}G7gq)m}`FLceCOzWfDy13$< zSPgE|dK=2H4aano)aM-0SG-apsO$_}Muh|8DXkU$_Z7V8L$)UB5$$QDk!cPBqP6fR z{26o)FQzWd%RhL_q;y6>P zuU792GPzHtjMt7QY`~2CppUrucFd)Lzh-}h#S9R+3Cptw@(|x1oBmiM=EI$_Wp=g7 z-B~@hDDAL|u9ESV;Z0#4l*%_Yb$O{tWD%J8MsmwC|FGu0CEj^^wsXX?Pw;qNesIus zU+GGFJM3qA*yH`Y-h?5GRQc9G$i!H1Gnf4u^Nx`EDEDGelHilFnzT++z`O_Pb0$zYpjWv+5uKu^y16+sSuJGko77!#Pgoa7gwPW^eX*v56Hmpa za%tdr=BX4KY9AYD6Y|-KDs!cowPG|<7F#oL544;c;_?10nUDNCN=4ogmfrOA8Jauk zDdtENLjr;edF%T>__mf4BY64?-V}L(Q(IGN#U4-y1=tBqyC#pgRg2}8ss2V_F?PeT zrN5Z0Iv|m58nV%cyQKN2Z9yu<`Li_WC%<<56!SKu_Rqx#@HHn&Ho! zY-BU5gY4dny^X>pCNum@CzGK3&%?#K=x^_jt=i$HU)wG{eNwj*3_C|(Rhdxz$pFAj zaEdmINrgnzkZ55lfA&`#!yh=Yz8^CX04D29qpN^~-KOK|lc3@XdUHIk$V!VDN=}=_ zo+a3jA&qhYvV({`^<~JsmrmG?@jlaw#b(AY?uD17sxwrxi^aGib7x+pwDIt);gu+($21vI;xrvjDSDxLdm0|j^)Yx$!#<)Zx|9F^37qNQ{Y<8Q!?K20AG)^1C332~gnBtkeDBK5Cxl(I% zm=yFRv&C={udNK+pw6~Zz4~wyzDOfuTA3F?3|wr7S7V#$053R#PsoEcy}=E8@lTIgdvxe!VpWCVc@mTlb0{zGe;AEc>(M)N66l0wu9si$fLOL6w`T zA~LSTJ-_jkS|U=F=M5HnT`;Acf7Ow}Zq^*bYZc#^&UTVUnbwBART$gb7H*B~rh^1& z91=7R#!ifLySe(LkL{DlMhdE*M$ZoFw{(6}VMuz5$B-9X-r{yooY+N|i+KXmU*}-K zzQyWs;cA{vu zdjhN*$~^qI(jRF&)yI1C+(xCQ%O=enGf{z{_*}VuAAfkFH7srm`M)7)Vh%GmM&XP8g~5oq(gys4S_+n`H7h_J zJfJp8(>k@y;r{1MUX1_gME|X*zm05xzdg|Z8sy6=B>%$T{`;l4x0=(6_2$jdOlbB$ zpB&r@IGWD$G+U_-v*f#J;`4rTstq=CawpEd%jXXL`_*kj%F_S3X zu469_NM%4^qWakF)o`Qt0Ie}UT_WKC@4;^Hgsc6)4ch{L$>!M9d3{+)V?$F^w#o56;m{@6E+{6j83QXhmy?!gx%*SBKo)-1u)bCd-IAGTdJ7^zob|3zZ0DYYz-YGp=^%~_kkSmb9LGxT&gEl6UVS>WIqyoS3%6FmzX(p|um$tARNyHRMr9%mZygKE+0(XuQqLMoe#NMbz@5Qu`CaO)S7Ny?(mgXO14d(*`g{OJ^69c25?v*WfO4_ zPW9!6hY86Sa0_RT7-{_>}k z?{7)|hpKW&)Q9-v;f5wd!MSsMtefBKmje`7D%)2#wmZY!L_!@m+-=sQbi@*aO=ICF zHxz2E0&h*E&K5bOWu)YRKf&o{J%4ZX%vcXUYS2`-XZKUYZA-W2v*&dB2@@Ip^98#abZ(caCuenpVQn|*6Ilg_{>Y1Z zDy6c|Q{xzKhG_k)^1}%OK8VeX@=qU0tIy?`RL=NGrR+uM9!!XrZUy6Lo9-`zYmF*H z1tv$Ghlmh_9t=-=d*<;YZRWEHwbJicuMO5OOzd)CDrT)tX;x_D&W2CyrNae z%>23&I;D<C26hZQI!zQnl3)8(ZALa79k_%A6&j{4eTzGx1Tg6YRXHp zTue}eU&Qzt`SY?Jg=o>BpgalC8nzX`q!1#onfDD#9`qKgm1sU|V^s4T{ITb(`P-E{ z;cjP4YF_}7!F%ei%5@2^)?od^FWgBg(LJHU##;=o{!k_C*ru*9|Jk@dtQAII06NhtisKPVkstGkji4Vr%sj!s6`;bI<3a@ z+|8WholO>J7PC&v7{r=DzszQJhfazx{1l8-L6M+`t*cZ>EZnG$H%%*QC_)jyg!0i- z5f=Fk7e+rmPTVf!;eRq!xT?|4n8nw^YlyS3#c?s^n=N1}z&l{(!WbM^gb9ru0P&f+ zqNyD*1fEqrn;(S0|FL7~wUBWjP`(Jh&5SU<^4mZy9slic#8gRQ3J;-8w~i~ONKH9mosTWB75cMC;Jh%I+wC`rmw zLZ|Z$fTgha_Lz-LhG;&0MG|R|zobvLWtQJF9O`g#0&D6W@R>KNW1rGMazx*JcTwag zoZ5Y=mM6kp7dYQGkm@FPFI+j$R<_L=>z9R$b8-*d`QHy}(X-_voM<*u@yL4w5DP7Y z=N&En!4xjRuBO8rXWHnkqnAI_q`(={(H&B5BrKEF(h}O4#1JpVARkZh7pt)GLFR4} z6r@6L6x{91Ra^94iYXqFf6Mhgm+d?>>pU$3cb0%BNd1UgYzHlY2jAXqvdSzJ9Lz#9 zJvXbUjs$(!*s>+SnI?B9Wh^g8!wIpQKky!Q@H}g8HC(++U!eCKv+K^$TIegOGaa4x z`BYPuqkUvs%5i2aR=fQPttn+kIMBG+lo`K7_8Bm)hrT~=6FsCmWDZ`XfeR6)b})NL{8{) zF#_ivUgYnQy3NM1e2(ctw`%Mx@8Ka|2R;c5T%AWyYYbvkOy3^udxo#%Up|1N$`ogW zo}1e2K4Z0|UY#t#o~GBNf-^a&kBmdj5nM?W`GA@8k?5c?rpDl5sXV+QZhk8c98#A3 zHNxgtWS+FQ%tW%C_W04fe!irfE%J@4UDs<(w#L33H>73z)VOxZb`p&zGFFBi? zNd!7pln{yjEP0$_{-Rm%x8Qf<)OPj zJJ2~WL~|@lbKIkrEU!Cvwmv6opeVHpsjIU6lbte&DQ~fVKalHhIb3CRb`5OXuPaHp zxc1JRxb_lS0C+h!=T#7sOt1|rJD5PP)9`7hr#YF7`pxjJ-6)i@nqOSt);}N5SxIjZ zfC%G-E6&Em+p1@*p0*#;k5?1R>sU`0TWSh%+UD2xw<5b5Iq$aQl|Ilno$zkK3lpFBH0W&?9i3>sUt4l;N@hD;iZz!5YDOD0fNpPf>OAj=h;u0ffW zw}D+WqcMHre^&;MzPR(jMl$wsyp!<7{09rLkAn?!Ba0pl!(!Y!S z1b#%nQ{k!xPhtmR^Mq||(&$+5q{(}tRMiIskX-}!8Gn1%MpSKF4Ah!^8JNi2dpHIr z6XB1<6VP)kpbtzf0UFIYgY^}3wzr4I5E%d=OYL0{>7YI+I^K%qpi%#C({NN$K6*|Vu`RtL%22Pq3bZ0Z_KYpUhG9o+0 z7_Yj1FhZZSq2YJz`f16~AuPk~6VR4#eCAlxRr9W|s%AoVwh2~i_)ANDd)alB7G200 zXX3ewJAg5aSIC~Zl5n2gblP|y4zV$qSjCTwx}NYPJQ-hktisBiPlD$oa6)bKgY)uWmR<5?=DVx-} zt+B*c#PfGuI?YZYy6ucqXifhqypQ=Jn@(Q$!-LNmSmbt5HH61KB!k%>FOd-Z{FmXWRSk zPSP^;U>wO6fLt7cWr@BCC% zsUC$mIw6fRQc{&?BmmUFVtv@{3Pul zrjsLa8%)T+$wK~Cg`Ilv(#gGx8KKy;Xdhd{nm4x^><*wId6Si|kO}{y$z$D^J?-4> zvAVP%USkdv9&ICFKkcE~ErucnQxTq0(XfcCZ;F@Ioq^^atkJ#d68dJ*F!)oPefe+! zw`rYguqrBS*7$ed<5&)>#VZT`^CLlens+*}hUr|mvEwfAl>>jg0^c0mfy`coTa zX&h|4R*Bebf2zF@cN%Gg(O>bK?yAK6F=Sz?Ho#Qg{0I{1X{qId6q%rI3Gb@r(Kk7( zD7K$9sBVBkUC3pccr`OO+`UnV@Y<+lt8Khmm4@Mls~wNWK1(TB8+KJsylct<0nE&; zL4a~a?D9?iD_YgC#YXtd1f1K`m{&596ptJxG;PR)!b|U(? zm-;e=Z{9l!V>%ev0|Z+$*T9_h7vo#Z1rB9EVJ@`QDmFNlQz7uOh^R$^hNh9?pa=wJ zC!f^`d_FknhM+%G^5n@~JiK^>5IKHUUxI*nVTF&N;|^9XRlJJCqs8D>pK_+`pQ6%@ z68ZRP_I-^XF)VmVNxT@cEk7UP6Hn4(Cb_WWlt9t_&kvTdgFu5VPWTQlgBeu5z$UnK&Y-sck71%E4JErNqLcSp4IfBJPpa+ zU=r6C8lZX>7y2;-M4ybw0b_{Aj5WxODNo}rx?yu)^UwTnr`zNNa;ZNCH^jpS z22{kFl^bce)NcyXzRh$5XDeBvwo+@p6!9a%gUZ?;jlc6_aM< z{Lt_`5iKUq8L=qasA&++*HoC6}yZJq->HI^ZWk95RT=~bs z%TALnm7S3M&UREEdeg-P*lN31$$D)`7Ppscr~uSgwG5pD>2g zl@cMR!~^|0x|oHiSo)aYerT68lPEFUer0@&j@%+hAyXJ9z>JQzd?H!EkMv;(iM^$D zHs?^d@CS&^jj8=$!bB`#CG|zQ+I!l?T(;w+rmk|isz`kd9Jd}m(zIyFY6;haY_zs$ zge&qi_squJR+OGUw?jT(68Rq3p26C5TaLm>L6oo4jnZ+Z`(pu5;UQio!V8=U%16oNamid6@9=9$AqXP*e`rZ0xi4n$+JWI zx1r+%Dcz{rm>v@B2iF_WOC43;o5sN`bN3M_m)RmC11Z5v=KT65f%pgIPEJ0M$Y{nJ(xtCMU$=pEDB7$OrDV6 z5t22?l%|>6kycY2jNPHc7G*W=3!1&4&N6goqL<+P`Bv0Do z+q3Wu%g5P0N5#a{~CM8}0!opwLn!37jj%FL+F%cNbUx@N52I@ZjrpkxMa zT`~p4lqFVIdOh4r5{Cy^bQ#PMY{=2s7p< zR!Kk}cI#UK;j%L5DGWhBGUY?RD_Z40-v;?tYWALG-bP#97VcDSGN!o3@^_QAFy~3p z-(G6mgnZSh>4J=)7GCn*Ry4sd>3GQ%_(B8j0>NHH~j`?PP0;v4Fo%p05v% z&Wqml`u(rwL(f@ZNc=KRF6)n61tQ-GY+n%{-=b1wnB04zbE9K{@7oyr8W}@N%3{`N z=F73Cye)KHrvHrDnC8dX{>z`oXO>Z_w$`It%-OulCKQ&;?1lyriA_vx0f)!NS=KaL#0vT)P|Q?B+^tu`}+G`UbESWXgH(| zNaPlA4=j2V<@*RoDbHs$bj6H@^TSzfbVW>wdZt)Ir$ws8(SC#)wdjfiv%HEPHh}`7 zT1B2#lmo0$SaV3YqGrGN1Q8&>H|xS6yhi{|JT3zl<7=Dn`&wSMbZ0EoSu0cJmnhou zBVpd%lJW7m90c|6K!*2?9yBX@hNWI-BC^mghn%{yxpW252B{y>qu(6xO@DpVASl4w zna?gPw8p~-9uMA*rExp6KIxOn8LEdHcRvu^t?v4hR^B`4FL5NX_LBxH8E92W@iIGv ziPPp;vS*m7R&q;>#Z*`oDJdmDj-XRS-Cy-%}Gz>1|y3WjgTnmd{)$|l5geNuU(iim0$ll3`1 zZi+CK4xO^Tu>hoUKmD-+u)5sYPL>K*RwC|dKd1fchk7-dIoroq>PXmXonV0zLFvei zN5C?IULKG8#%mfI1u;?m$cH@ME|b(89ZbrHtbV;=w`ELNO%=U8>J6zI3U^fx_=N0W z!=WRNvJLARpc-`x?Gtyp)-w}TkJ=?qS1;9zG?|et?Z-D1R-Ykw1(+DC+`lV0i9A{& z-Vdhg4iKTgY2V&38EbZePOS9~53QlCOlEcWNu;dd6Fo_?5imX$w|Lxh$I4Q-{(#5* zExYr2Owhm{!(V|EAr3UchMT7%)2##hn#A(5=PIGAoQ{L(OTR((v+FWzK@GQb*EY9| z>GS5X+ym0>j?Ex?)roBw(;dDAK1S`u|6|h#|6|Z1cmMu4ITmCgckf~1D|A{4z8{M5 z6B{MPFkBkJ;gQLZnaGY2ikPla=yjER@_c1zTdt~jVPe}4skXsGhVT(bKbb4C9X-ae zFj2fWZ7F}UUlbv}ShJAp+B6{)q=ao>eT~k3l-~D%asPP|TOCD@Mo4)jsgH1nxoI}T z0|^~HL?b^+( zGuUw?)Zt`+8d_3WhzBgz>@WcCc8v>$zQvN`whRhSJ#kg0DI8>^PP&pmK1b1kFJo8j z8Z(ruYw|vr@p(|IW1IkyV}mVXR4tv;Gs1|T?dFZ1(cg}GkB<4CLhlc2mlj=jr|QW3 zZ?4ztB<9M=OfSBK7r*0FoSJT-^?vj;oP$!`Co+c}Qlp)j( z)9YZ#4cCdfbg&oWv^{3sCB2dau{XpyF{rl@_YqXbn6|Y@dENez2yL$kk%1RQ85RVSjXOcs2``>BO5_FHfE&W)}9C#B1EX& zSTZBWQwIQj#gh*L3Rts*+w=V^j%T^}W9Uj^4HRkezD_Ot#sIp-ZTp2IjAVYD%Qk|g zpfg7omY7t6ay*7x4k5-8)9Wb$ti;Ghbq!VEmbL>IxEP7wOaGP$WpQ~}*V&$h*{@QO zzsy#iro-KTb5>bdu9r2|&1ufwa;Jize9eJrO)rAGsB#j9xS)bA>}%RXtHo7p@EWIw zp2Ui7Y}7^5*%~Pyb)3HNG>PEQ&&QV7sFk}qMXAj#u|Io>A3s8*383t5I@%{M;Si>C z_eNE{e6{xdj13CaWa)`*#!EIjYy5QTnG7y;#a}TZ`^a!yP0@q*3trP>@&{1%7g5(p&|?Z>gpg$Lb## zJduFm2JjylW4Kb%qy)|F13?K&p*u4bkO;e`&5OcY zO$uu*C9}Da>_f2q@H3o~88!4_faBTE-xEn!ly?WW5j^X>V=P%r(q?jpn-Mh#Qbu`Mymn zT~Dl1^VuFay{i>XWa?Ga2HtCF?kW%YczK<5L%(x=Uq2w038>{GBt%$MTdrbEat0H# zAD|c=&>ZaDhO#bET}4GuO`QG&DopQPG((!8+?hK`r|*PpX~@ysu{@w+o$x5|1;hJ3 z4z;+mz9fP=U7|pW1gV=jfM8ZkECQdhYUq8%6_(6sLsvwg8`5d#yJVL{N!Dp2iUD*Cy>)_rDxr~F_kh5h@LJpNwKQ+xXYlYdCaR_EG$F+eHD z<8@XzGXUYGSl1LG3@ehy�8eLZy6AUxo!pZrApE7GkgkJL3_;*QNt5L}-fj9r6g zq-8wap?D1(<^%`4;WjnA3}&@lLrGF)4vzb=1WS`PdwHVQH44a9n#qm2XHUHTd)!Gs ztTts8*5V>fu1g{22>8V=*^`k0l~QvDA_XNpJ-R1++yB#(1;v5-krNYRK34!yj+$C+ z0Cz@JSl(9Wn4KEQG+Zdd-5bedlklsB_+2om;h%)~$;r{B%+peqR&NYJsTE{%QF!4F zc$&62gOHMhur4oNYlemwT0+E%`Sw+BIv9g#=gMPC7vo89e21KL83;p=Ab*6S5s^0} z_OlYfL;e^wGReSqK@i+aLx`t-)ro2AWaYcU>#0DtD1`1OUXQwbeW`gIm%!nC3*(Wr z)RBG60l{pYDe>&FzeSAs0-W+aGD*LL<~a9^LPvtnOYS?|2ShNhE)o7cU`x%FsLyU6 zkj;{jEr<&wP)bUobpTtC&770n@gI;G8^}z5En??I%1Hi|-PLUC#YeZ-5J^m>R8E_I zp<{dde!ni(IzU#B2XM38LfAI$l`I;>0prOM^lqIU+Bxoc0~kC%XZz$fksf*jOJpms zYINMYZfYsD{Fs+d##j|XMlmtFIYrv!U;*ZoMNJfU_szXde__8Q)T>jko6NG%eM!zA zs#UjC8=Gf@NFl;6L3EA7XnM_#{&lzB^|T-=YDNtdbisJ0KYng3(Uu)QakS&&4E`;l$#fKt&dNy6cA816wl`*RZ0CYn}2-ryiKZD2?6EJ9Pd%9^( ze$amsj;L$voZ_7+c45Og9j%$8yC9I$?_IIGE-%mM&4}lEpwgA|eG_pBl`ES_mw#5m z-ks7ntl4>u91j78YGZeP0=uf{_^vIPCKA`e?!xV%@gB3XSa_OTDbMW`&(DOWJ+y^f zCw!&BkE?{c(_k21elXG>fL-&WzM(^fbs{T3j$;=kPcU{UlsuyS8uPMPOmrEJNIR8B z4)w0U!9VrQ;FQ6701ka~IIXQwHF}dLDrt;XJW}vQWaY|Ss%rTCcE_=vCUK^L-GD1^uDSaNUc*V z!A8>S^{f*F!$%WzfNE%f26k%&noyC~RmC-^Qk2b00YZWliDY3`v5NwBRdj+qvy;wv z==ON#8Q-Hsjc~=8F})5vfn4Lf;5nK=*z@thGI7&3t@frfUXWqi7ONvDm(Y~HZV*iC zhjo?@;K>v%@8IpYtJ)qt-831xx>qj{7ZQ*Xe*Dv$T*}6@Masz0D}y;2$!ucJq$wF= z8t(B}m2zzHa1D9aM2{NJ6TpU}U_WAwlV<}vn8gn-5Uf^efT*9lFk0ALD zjFzUW7j9>A`%&>XG69M-zGyX^96qu+l&wr+I3A_%*sZ-Pu0%kk=-24kY0sRL-2*?B zng))xoN5NBI4IIIu$=)ykBmd1*BzuJwzpqg*%ggIuNmdo7lw38ZKN6tPRMu$4A7Np zt`aMKmv{UMI&>*pO{b5wnvB`&>o1v&V;xP>L1Bg}T>MdDVNfJIeYin7W%hy3;h`9D(`pLd`fzK-S*Hr~#?ZM&${kq*Lb`E;B-90XTA2h$t7 zD2IN?#lO1i#pR^9^xXx+dCsGm{OOKXTf3psn8R8Iv{-H0D~Z2@K-W;4r`Q2Oi94GP zDQkb7PZ)xk1~zAE_u@lTuzmLPEfo#F>}WH;KCxo_TOoCyCr+u_bo9cz?68s!{vXUWX>egtSRK)@ z12S|kIm*!ZJjp3G@iXO1zovmPGbwGA-!_TVT7%UVtI>{4PJPAxXd42k;Srs+}8SIva1Uuf(e0};#ul2 zdflBGayL;AIetTIgP=Q*KT6$wPlA<3k1*ztyj)p?kXGsKd7fgA`9Xu{KxkQtw-vyW@Z}F35CK{#f#xPow z=%1AAhm$C$K6N=jblV}=Apd&A0qg$hAX+>pm|;h&r`#S;0JZ|NQss zyzkw*hGY}>X-n3?h0L5gjyMlH-CWbAmt`)qmsHX~lF0auo69-+ndXxTK%A;4T~JQY zx_Jy*k?}obTjAF5S?zpI`+zdCpGsRaFi8mVE5D^to}45*-$d5P2;;H6+iR7#-=VwJ z6;Dpcaf-h7%%5hP7@t=B<`Jmi{mzKCNVZ5{u2BmF8w3c@H8fPp-$E^8(EL~7{`1o! zNXYx+Z+ZW72&hLXO4$D^%l~oIw+bf)9Qj|*=09n*xbWfsc^v;eZKo?gzWlEzenP~+ zE;la0OAjLdYgoM9v}xDzA9&#Zq?DchIT#4E`>-P?sIEQcW;03&NhZBF3WdE_}7HUoU>JGU5TD9n6JBfJ1@4v z)n0X~uyL^MZt>)3M@c_WS#9E6X9>DoW$7R3`Ce3bewFHwKGZ_Wvm^fv&|r0#p?djm z79iM8f&`=SIa8o0%R}74r5O-(;ed6tgZj`@V4QVsDE+Rgl9&UY66)4aDBcpCzDuv* zWogj+nYz2T=Pwtg`Br@{SgJGobD+ef&jc`z9+WEC>;kQQa#4Rd6=mk@=f@W~)fO3Z zGF1_7xh?(j5*&ez594dj~a6YDu$p8zId$-#>ov?_!Xcwi_{teGr2u9y-vn}O`8)8E>Se_yf zH0x!>o~H~r260~Md&+41Kotf!dhDK$Uhu68zwr)!dX;B^u7cYv`CA=r-oBhap zC(1RqXbYbEzPn|@!HTvrF<^gc!>9UNq~l|$=^j7x)HRfQpAd1$XHDFb>V4;kHu059 zrMg$^pJA>k){cv!)Px?vbJ=@G%{b6&vB3&5yKt6uC`3JVi`KglGAQ(HQ?)82Ark{s zj1WV~blT{2u!t-5PH<_&3wWkAsCB%WK-Nyz;KW!=*=3i4usE>z*&e48kSsH3!1?*o-h(+eNJ<{F8*Zh9D=m5Np+-%CRRh$pEXZb;Bvc!na zm?hH^J%usQV&pD!83KVJveUC1{c7{VlDsbK6u*L6c>iVoqp!$c)_xB(QmeK98pe-~ zi!0=sf&tfRWej=V%Q9aI4!07CEV6`|#N$Xx{|8MX81`QGVhC$Hxb98!#wAe_j>UuC zZk_&DRPF2YFS}{sm)H@52^dWpk|lpFcZya23=rVa5(|OQPgW`xcM%&Y>Vi9nFILUW zt?g2ZwLP^eR%iDjws?OL0X&%N0sL>xGdhDXMYIO-Q$AIBAZ!D_?Du2yyAzZDEse&} zvmeL90SXe;X%A}^JLgJ!674TCDFt?^TLbsg(q(03=~7j=52Y#pVG4Nm0a2_bN6CY= zL>L>}*@`Fi>Q9?dNKz(LW^RG)U6b(`G#G1S?DJ_q$lD!@{1|Xzlg6L{rqHOCNrDMKV|$zBD33 zm&zP@=;em`Sh3UTI|^Zms_M#Aa`UyZ#X6rm^~$~+y&Tlo8*lmB`JvpUbf%Gne1_x2 zb8-(x0>Wl*gCgZ?YPx$XAqa#R1zoJcP-B06Y+`Go6id1psfB;v#g8!A*5>Ws2WzE6 zZVYZ@a;?SVg>v14jE6p`}vcW=|^)}q)wV&#xV8DRR>h-s=l&`Mo zu66bRwYTPXyF`(i)p|5%`55cTlW2I1o&9GtC%^^?l9ge8Msrv=TVDg$<|K8n_CRH_ zoQ>{xsI}#J1lsQAl&t`))au?v1m#v^EyDYMxh+3gTYFaF7Nt2xV0XDVW_SsO> zQbu9%0%A=knRddb4&Y3*oJcQG$opRL^8E9SAL=V7Zf7v~N8lsXG71VDuHkAXO);#b zrtktaIi>XUbZZL^@06ZLVP~}|oUb0j-uIrMYvFi|SmRk|sbhRC!P3nA<%N}%b!$DJ z0&&r(6G%GFIvQTVmHzV%mKgd>@YS%gxdA7p)a*Nw67yC@?djy-I+(xK@#N^%{F2ew znP5=s6FPP(G@avv*l@yJFm<+DvSe6PBzNjyjpuQ~G2?WeC;s|HTJ-#_5`LK(QC3kz zPnNehg-sU8jRjnoYFC!fEY1%$k>VCHClrMMj4BViw0AlChtZru#0xS9kwq*$^*9@> zgmfd#-9YHIYsR&wce#0nHD!i4&rum3EVU>nrHy4P7H#01D5)ClV$1esC!8Y8%ilXC z*I$^O&RXV(z1d9c0Zi(Q%aI^2;z&>ORQq`;PZNEO83o4T{Q30B;k(%C$!eUO$!u?s z7Bg&5o+8Ja<79TgF8ISxL4(#UyFZ=eL^1|`>LXH7Ry#uj;}Gc$Gs|v6 zvge$CCC)>M*afoJ1^!X)cjUDAIrL+KPUi~gnMErWqp8fTq-#g@6t-|=7Dht5BtR9K zE)|@Cs+H5fPYikY7`- zS(hOkrYmzHA*(%wPS2r?x{vQtPrZ}4U@pPnP&u-ke2Iy@eYD`?J=20$%D?Iv;=b@I3N& zxZp!=LP*#5+vp5}CvKLG?rBY0A2gkdJA`YGc&q$(tdWSi_aS3nK|%^6h0xZa67_o% z_LW4-*H#|TVL_9Jyt0z3gg?PMH?;wwlVy(dHN2WNu0ZjTA zQ)_J2_GFcGUDc#VHm%N-F^%7Q{Qb*~iuW&{U_L_H7TQ%f>Op^01l2w77`~v6C7TAX z@C3|I>{x5n2h|x#UEL1X`b*vz3Y@TIiKuIN1;1^MEq@i}k0!JFjm8KW7so516AL}T zFVIJCZEB6y{z`A={;G_@RPiY|rvDZ23Kn45=5QI7yt#jk*)^K7HP_SH6C$B=5gY8! zf5>QjpfdP|c71Xn89AM1ASC(o4B+2<4yBw`OF+ZH1(Tf+XlL5=-J=n!zss!V(ey(% zhz9X-Qp3SQGt2DXa)r1ily9Ys=5eskW5K4a3c4VgdkoIW@ zw--9zKd(N`Gxd%pWURa;5G2xO!SGhdA+m z1i3$7(;7Tldn8_k>W;2W%opN}uwzAyLoi>>jN@ z?F0XyR&m_O`4d)w>bW~6^(Z3Si{e>waTJKF(o;P?t~hZma%Mn!TwzC86m{(N=W8b& z`PK)v3qj$w6RxujF&2VCC)dQSOE%pFHK=%neAj6Qv8z?rMJR|**<_m?W7=ul zu*dI5-@(43EV;T2#KP3Dv*k4540wPn+x}|khT9f`SNvAp2%Bl#mj{_oGIc8I?5Py? z=KUvAbiN`w$%q9ve^3u4^Zi&Fk7tmW$av?xZ{9~?#4@S%KQv)czs(}G$Fr8hk0M8P zF$r1K#YSVycvGdsbiCh;#v6TH8yn96#ha^f^37F+gZV{ zo_LIV@@$VA@1gFzt&BH?RuVv4!TIDdpf6Hw&1<5EgCaH>^(J3xs#ICj0IBpmByIo- zCx9fnTu90fUKpX~_Dp~)?QNI=zA)ub-0>_$;xzU-Mq5~f1BQ4XZ;H``0wgl1%Q!9BuRp$IM4Ux+?|1emm>q!2 zmBC9~c?7j@yQ1@y2EThusjTa!-#cX!a{Gn&^CbxnTofRpd|HhJ zbY(*-D@G9 zCAs@d>z(TMUnb+oD54h9;kfyHzIGGafT!^079+Z=+)xx?Tc``guxJwR6wK+pQt}H3E|jAo?2nUr9Qh&M-h-K3iM(3 zHQeUeeLEv|7Ps#~U~vPFrN)lsdLnT!AEya6E=8XrcWmQq&e-T+lM>moxMTyU^_m~Q zj|YO#&-_U`i*U*P)spOTBB}}usdEc&RPHG`@5H>rbkE*!c&?jQN5u)#12BUmP@>Y& z`wVS~KLz_*m)>GxoobRaud-Ti27fRz8nL+=9)+wd#Oz<7R@c^{>afhP%&#ZaX(O-i z(9jdKA}h!_bcuLk%zrVSj5rLV^zdkvB(Zx2!em`TJOXajKR_lQfj;=L#Y>Bf)e;m! zy3!zNy;jZl$^qXM8BNq8_RAg+n7hM5b%Y>7>~x&{e9=hpUDAIJG;~IuxfXH3(n0Ph zxUTAx!()HRIP|~hV)YDT3o+~IAwX&V^vPLr%}5V#5Nm8HhP>dkenyPua>7WtTq*k% z1Uam7(GHCcCLc=SOBWsW`3g)hsFO_es7Dm4k_66U2TBD6mRG5x7QIMZo1cpYlsh(= zY|gCE|IP@;klt|a{V(!pes^#n81xQD#MC6x!1OgdTGXeXzrz*dcdU7!LQVP9)_f<# z;qV&~$h&7$Gym`01BF`j{o3~4VBP|;LJ8y38Y>hjq~Y<3xdH0|8&Z?;s+4&3##e2O zOf;#(!PgR%I%0>^#{RyJQ8x}Uhh3t!r;RheljocoL&?`1Z55K-DQi6k=hv0WS+?V3x|2 zbgqD?G3~*IG?!o_)Zg|=-P80{30_qEOVH55V5Zhvwzp*^SHKsvWg9#+oShxUs(U*Q zPgwVjwS^K*B%K;xrfscUpUkpI1CqrOvLLB8i+LQCH~_LGAY0`$35Dci7^foQlqt!s z=HON{yM73UK{VsL_)$biET`$#!5kyuo`_f@%)02m@Fb5fZ~EKr@yr-}~#tQG1@)a0em*aHOW zK?ah<1hwfW3ej7!&muHl+QKtFVaXo}8h79lfe!eICpWMo;sBFg{#=PlWm zm_R@sZS@?J6ZjEvu?t_NAk&s|DV;SnSyyfrDJ_<$#QZsa5_{2xb{0KI0t%w+WFS=4 zxC0PBZuSIpTNjntnsn#Yj7^z@Q4nk|~6Z4rg8gtK?EUs$YTm_;!q6x@V?1IIOZ zq;ts9*ROZ=B46U-rWlH0SGu@a_J{6?>2DiOl#AZ|ZG*{7u927yJG5%XV3|K+#u|)v z9u*_Rc`E-xfkqjbpaytIw}RX>XI z@PUdDwNOHKIADY$a^~`dOcLiMp$`hI#>E2njpa^)OO~wT$2hs?QhxrVo!rzdE=Qx? zG#oCUWmY(}Mc04l+;)6;4x5aQHp;UkzM{~aT`H5?eNo5i+h|G#)Z>-p%$}!9m_R*% zU}mz|YG9-0l35EaTEU4dBTtgePEdu2t$xJzSCwyCs3fmnqQ<4&ZnZlnp_+w_}H@-YtHSs%=L zzRwhts_lk0gMyg46v%%+AK=3wx0eau6QOnc2;VB#GE&#juyJrOIEy)4UtvowJO^sF zzWG*!yae&4#rJj2WWE}A4GA(5+KT^G1eKOd!`HXU@>V*9n*1+&wD8`he11c?`R3<# zEPs&!)kI6J6nC?3Xx0@OzIn~4R}j(j>wA4wWwpcCt{Hrp--W>seemI!Q4ZKng#H@= zBqD7W_J+nSTZjW)xlIO#kMI$5omjNPg5>6zaSCr@5z*zEB1G)iYKO5v2BuNOq8Xw} z2(kJYo-7!pZYQ=*rqx=1yDLfzAVLqQVR^!Vlak0)7=5}dH%^musm^UWQuPl_~=K|qf2Zb^{{{H3O+6XCEae8DL>ter-Xxt~*AVd<-+s~+{x zppd>+7u8p!DN~jPJV>^*`Y*bLZ@N5K(W8+W0>szL4du6ld24^tO$dI_h#!7@J<~6C z%JU~U-Ae1NAwLru`;nbiTLA{mA_0;4jiuz3_&q(%k4hp~g>np+5DdmscHb7Bs(h~+ zGZ1W-^wZk%^RQf(WbQ#AdB|KI`pYsYsnd>3>q>yTM1bUZiFwIsbAH;=^XWuUos}WV zvkV2nSDU!~X2SzjBwN8-kf?Jjf!G>(V{qRXBH!K-_kNi^W|~dn`5Eo%DPI0u^B!%I zQkCnRT3Ja4FA6;_s(t@mkNf`t;EcJZ@-2skJGT1H0Cs^u=c=+Y#?4qq^gdfw{Caok zeH26V1|K#T6v&+55ZVul$kf28GHBIyloDq(1>>t83E%;~Bsy)$9WU&Lla=zl>8lp$ z!5nq*$Vr*sO}|tX+Cm4b0}b~G5Ic-V>)rcFwx{%Fq52y<~ln(hQ!P*lkXcUsgMCc0^tY=s$#n7vETtf`*k4D)w!0t^!!*FQr7%H z{akx@v%h-99ri^-9}FjA{LWw?@kvR8v(I@Lay;=uSyFmd6FiFN+=;>9&SyMOk@_2{ zz831XiagDaC`Xe+mJo1CMQR&LRB3BT0{^<$@VKWv?hRCw53i*~aZ+=d3zTHhsEuYd z%3-JEULGr8*^ITznHkfoD~cDrry}p8oRuKQHEh)QHYLva9W_z%DTDzoHzcxE@~WOl zeRwT-5$2`|6(4xAEu>nN3}apJ(ui=(gGbGv1D!{6 zKtfSg%x|ow$Is!5Vy}a6>3`ELCxVp%q``j)m@(e!vyDvl-{?|c;WXyj&jm(=-%g6{ z%3q0)7>tz8zFUm5IaY4=DI{F%tE`MD93md+!{^B7F$oiDmb^BJ!ek#T=G>ce7ut>9 zD3tt(Ino%<@<>mEhs^Dps`1>$$qi%4t`vX?H0SEh?D$inr~f|9K0Nkd(9`%_UmNoe zLT9x9*O%ZCh*$n;Q!y zwq%ndy*rPJBTQT1m=xH)*%iOl?CM-t{zS`IEZ`=^KuxXHdmRk&r98MDLc6+28hWKa z)cx%Z-rQzTD18)iiryTk4MGo&Wtg*^`DI}%6SO16(p1~myW+Dxf{B?w@Ptq%3^zAP zgbRKpj0lT1f00Gxe7R!2Yz-V7w>wAVzLtYPrn!-ndgm}|oNG!?#SfetoCs5vomf&L zfkfOL16tQEq`G)h4k`odYr41|-utHxmvz!Y8G1UOlFl@)UR1`478B4OuBu(@4q?_8 zSsxijzt8gW#sN*IEL33J(DniXCa$6S5W_d^Uztu2lUNz|o3xGbmAaV^vSxz7ghsyWf2knPDh`5Z|ykAqMpA7XHXxb;| z@WxiLDquwPShW)jdY6xYyzL&;YO5Nh6U_gm6=(Rh)f=Vj}4 z^#rMK683d^Vs$f;>3CvSzWbRlXhhU2!>qnM&#tHgIW5r^_eHIHl_^<;XD@itHl9t6 zYSWyxNoeyQ!HpkF!{H)wq=Vh(P6*&$h#>g*CaVt9N;p-$(MPAI>c#2CQ|X%wvY|um z$wF9Bme)jM0y#o!(uIKl1nz9^@iff@M@%!t)t6j8c!Gy~`|GJ6=q61c{PacaV+^ z>bj?UKv%%vHwsR}r<|*dk!0BK#WzSriLdsCMd?k(s%vQhb8jlF4Gflari3o##k0d4 z4(;#`KKEiYGm{-e3j4L@`;ex*h4n`3CGm5J5!#wz2jg?1HkqzSpNSY3XDSn_E@Z9D zwFBAN9Nc4iDd&e%PT2UtOQ|xsu871x9%d$XRFw76c&=^H14nGjC-%{x{!*_&e@%At zQ2xtgzv(xuZQm8)X1rD9&Zo-Qd0UkNVlR|fB~^!m(d&|@XVAZR7J-7naOM4so}dxA zF~7jTw&vOP9}9CDY7XH%dT)l?pFAvrB)r?Y_s5;4)=VJ3Mq>li&_EbV!C7 z+%g=Y)e$3tOONX;T05Z>*CUat`w4n_x|>;f9}uj{HJ{oO3yPCemtpGdAjTlq_N{n; z>Zu})JE+9={;x>GtJxW3ftlIbz@Pe&O$7b2uaP0!72Xis-WeM1>U@UJR~g+|Mo^PD z`OsK$I1Iyl(G(}P^3Kv4Cw~(=rSy-_T$nE?%j7)@zs$Ff-#pR_7BZ$X`lWS>U6~&E z@lpwBmV;7x6itr^`hp5^npxs&YnxSiG^H+d)P@1H4mvYi!kGZ)MKZWEa4E#9xAQ}F zlUDx|Eb^RAo{}qq)1(q)=qkfB6BpZ>H^E{kOfp(;@`1*{HsB;uyx7 zAPl7|D5jPwe2=oadP%=I`+)Eedr0^zKvOK|8WUN4gPdzS_vjnLQPL_H_V?s5Bx&D< zrnVM;P8Ldbv@9F^14O(~fq}z`{yX^pRGTL^KdN??Qk~o;$z?(moBcPC{1pF%GLRfr zigSM?gUs`1YE2wK*^)C+5b56%xq*@Y$T>Bt#v6>dpb7)`+57p`JpJFx#j>$v9r!)U zWUCqxPuLK2MF;=o47iC5ItPq+LFx>h3OCW1ZtxSV?;rH+6WV#1wMbL_^~7y4hb|B9 zkLn0dcfb9(BtXddH;K?=l)clBjrEE2W8v8cG0E#E&|e^-$B0+3rEyMW&aw5UfTRg+ z0%Fl;%Npj_1m}x^uv}bjen;N_hh+fr*)kLQ5BH#;O`7y?7U2J8_5TY4@gGpaw<^Bb z?z_F&7C%>0i~4igMG0Q|XPpy3Mn>HVc&IC^T8uYDR=ygMDG`lHSB=v9N6#ix7R9U`{cR@7z!O(_ z6Ft8CKiPkMJ^dnzeo_|dr0MKdt#Bz5lhs)6e{(nt0sZY~kOJwhQxU6i^<8d9s#@p9 zk(!81MJgIithN+Qpp(|C41ys*SdvmR_|oAWTpBqMsqE=T0ORGKpAuo$1_QjwU6$pa zdWOtevOSY;9pz22;K=_fG&mkNEqm|EbQ_TzHP8NREj4TF#Ey74>N|yl{y;>%P+(po zba%@$WF;_GmT=#T+|@&7&m*9|FeZZzhm@@v5I=3l+P^oAVG~HzyZ{`{c589&fSK#k=)-zm~E- z3vuY#=7Mlp0pRXZXm*;v^iu>>5N|!;dE#UPG?yOo?rfw&06A>$#&Sww9>^Zgw^8d- z8eL&_s&W7i%pgO!T({Fg*00=_KzK5xatH&er}Ahfuo{fb*k{|voE`J~>D!b^PT$ZL zw2?YX5yvYp`@5-V+CjIUi6)Gpv@J0Ym~-egHch`rG#VZ%4l(#u?< zf9{Ocb5GdJukKD~6I_%{UI$yZyEf^M?hieMmqv{JTL^>K@`I|4zLNCxId#+69rtT!d)Y9HaZJ9;~PoUE<-q+U~9IkdTc?lwW>k{NE1x3<)L}Ez75V7 zD$axOYj>o8xu( zU}a2tF*qO13!_?i{tsnu8Qj*ktnDUoVrGb$nVBgu#LUbTGc!}n%*@QpG05zgnL%c@ z%uHA7?7jA0U)}T7t@~G1b4nW0>>kfM-tLY_$@;W8{S|2tG&i+}L@1lJJX(ME;}q|T z)rVSwD#}pgCJX_uj4v3-d9MGyQcczFV=&K+AKTrQ7LbPdGiIo`0FS>JJLXA&4vQ%y zL1KA`XCjw?Rqd|ur=jzjHxd&-#Xr)&?2=RRl<=G9Lv@_HjrKs|GC}}L5Iy1_aURJl7Dk@S_jU& zQfp`lWrJFx@ASk=m1QC4)Tq?cBh}q#5&7T&ZZ-PMKp_TefsSXGb%08#?Ve-Ya6hQt z&5CEFjMYOtsQE@X;E;;uC6C!-9DZE#IHUYB=r z6U+;?8>1)Rv%A+HbZSIGJ~mPPEbjTOtwk}0^;7 zA&y<9?9H9EGIt6n+=EZnZW{cda9vW;AEm5ha)va7nXQGxKQ1FUMYnsWs9WMn7S_x* z4^ZhJ%9(}6TW9&E-P}@^U=0q4pjuiBZ_WKI4pzjF(syEUY$NBr?>p)?TUi8(Cef&r zPS;Y~RO`jf%SEquq7UwjD%myU#BVu9*`pAAFnKrRXC1_}l3X&pNk1EG7e~$e`v(d9 z@U3W6;5HA7tOM<&%yWRSSAn`s`bhgdYs<)sn%kUO*XC*31@oZMr2BS_k=7`1P8qj3 zZ2+HD5T{y7f)Mv@l<-y|binphUv=ZC`xN+9sYf%2VI#zVJ>@`v!5v&7)kGOEVnU!% z76G`ZF?^*#)!VvnRK6T}m&)?Z!WumD3UaubH+lIg=%K^a-^1@)cc0Q#ska=^R^h40 z5pg8gkkeCLbX)&ITyAM=&ky_o|1Nc9*75JXBJ^b)lQ%_z%Q0w=x74a0_e(kj zND&!n?0AZU9HtyuVrma+d7{7*b)xf6Bbm-zmN-0JxI*lw#O6`qrDZK_f{K`^P@*>k ztBPo325le7G?%I}{VHIYL<*zbo(97~OMQMM*;iVJ+1_BqmHevE`6lp(IG94! z1Z!}eoaELx1Lkv9d)#e1Cz@fX#FYNCF$53dwf?|hT_W7dNVU{&x|250g5b{wiMmv1 z06R1Up90|4_j6Ol+ zwMn{w!<0)zIp0Q8DDchwo6{Qo-FVK#DZWAJeC~pocsSDbkDt=PWI3=>h%?7HE=y9t(1Vbmp?1# zjzZ|}=}4QK;3GPL$W?u!z2XLu3;mLu`X?$Rqy`6JIMy|&d|~gjhJ6x>oc=2&k^^N+f`IZy8>^nOBDiH= zN`fHuU(Ij*Em1=aw3vX0DzhGCd$N!(g-e|8^KJ(|f_gH6_^k{)x&S zQ=@4K3F1I?j>G#^#7vfXPNiJNxUa!Vmr$Gy+1`-)WTM|Vn!N5I+G3zZxW%6P$4AuA z-wyt~Jhv1pm0m|Mv;*7b8MSI%FS7N(KJjoM~j|r_?TV;SA9RKuzUel9}8Mhvay|9nXVC zeyw6)b+tm}Ks@5Dl@?zbDUaHWwWnTIMSVpWWIE@CYesYst#HmC%D1x^PZzzfw-q4I z8v!O`!{MUcTMA}2X==B>hs_vJB=DjS&%Xq-doZ3lFn2Hw2Wklc z-&X%vyyUIaB1T!_8Hun$;jIm#m~n=}1dJ5fJZs+@ewMa~5yqfgiM}O&57lT@d);kn znfLqE8U+##%cX8CJrX~V7%F4y4)C9#IK`b!d;IttPjX7$zF+%?SzqdN6zkneF#MB} z=#U=e3>T(^P>~%wj+*<}puqT1JMn=l6uK2xEP=5sbA1<_H4mb3k_H+M2GgcFgKkS~ zh{H{mV4P&!=;8EK*w+A;%Oh7-Y!yL9cPgpAWO5kojcnUs4@bSC!UzRw54s%jfs5rv z=NZLOYqF3rz(L9IL@FQ-C&e2cA@YRzXFe{BrN@qD0kQC+`b@R!p7c;AtkQARGYqAX z9AvxlKvy`Lb;)}E@1f5t%jGNsnn?ie1mCk?E2dOYc&GL@oC&!seQ7+kYXGfDISW%s^xZcqwVH+m&*gjRI*lIGMEEDQzZ+ZG`4zn0JAM}DqG^$7yLOaf^4-~Z zv+X}n7mVAS(kXb|a<`dB{0C%6s6=nsjH7CL{lx|t62IOA z08Fj{M;JYzI(=$^U)7ZPMI` zd!s*#GO6ww_o^dmHZ?_B0~=PM${TLW{z{r#Yqp%1=XPERFh8p;>4VSR;wz4fVkQfe zbm*HhO&#v=BV)i>>rJo4aD!YY*ca|n;2-aG>Hqr#kOW~6Ub%{k+L%jv>(SxUt3aGA zT+wo|wQy%%?jbnYfA&Vm+@q09n zo8P{AE*6F|0T|}(ZS7)tdUnS8I&ahU>tv>}qvan9050GKZ!6u1yB3}6ZEQo+l9+>Y z%$a=7;!(xDDoKHudiqrh9m`9DIr@FOXWEjA9}p~>V4KpGEtAT6;|(zYxv~4>L(2Cz zpB>wsOsD%kB{F%q-Z>BZk6vD0O1xfgG^7nX!^*d6V!%pVUv0$ycV#dAr%|HS&Xc9v zUa)c$n(VQ+O9%%_OQ+X21`2I;>`U_p+8&Yl82M$g7`IO?)&S;#suctOqR_x+MlJCN zgJNe#(Te%)6)$L7hM$>=VLTJP23h8XO+&gT;tje=7l=x_^)@it%j`iQnXIXm)qnFc zN>+6TSsxYR_>mZ^7(JqM2Gh@p;W_77l!^ZkJ-sZwkN#U+<82sMSD=3t&e3S~YGAgY z2TEpYeYOPg>Gh!$_|ld1Xtr~fZ2Tj`_-sw1@VYpSW3TofuEh#6-jzSBJ4G64tOHL* zN9Iu`f;Z^X93H7tlAU{|t-qT3r$*p{2c}}ns+^W-HV}8NaoSTG<0srj4^sD$-oQK{ zR!ObaeZ*d1)BuOi0ELHgS*G?bG>8@fkt$WAa#honuR-cA_cakGn|<7iC71*WgjXtR z`*_Q}sRCJMXWOY|1SmeKIZ%y73NrMXVwgNgdVOuFm16HWe6oe==L0VS%TYp;%G`4& z$f_+BGVeg|hrMeZi}OiDke0U zWV~g9OypB|xjo=MmR|A$=H0^6y&y##oD$>6gFKnL-c<$8ZUwXE^LHm3HuB(s(w%L`b92i(^m>s3UUOZfD1VfJXQh(Cqn+8C&ux{ zI*2^^IPmul>&D&YVhnryxHF|k@YEuLL1TCi7$1l;D4BN7d*N3)athkN&hRW{yvAfx zKI`g&jtYM@zB|(zjd}?!3sgK>-rIGN_zghpqiGt@;l9c_y50Yj)Xf>1ORBj;b|FyV zH>p0A816rAeJ0TVK1!lYSQkp{EI7pl(KO{ss7JP$28{fiXDa%O$5oCDXh5#+ENo zV)VQ6igyunD7}N+KD+0OIV;69=K32a%?f{uoydeySNOr(%Ds5tb2B!(s!&vygw=1Q z_o~%P&U;V?y;u-~|E4^fw}@57#JqkeyNlG1?4B)yvIsN@SB>Y2Q3pGSDyfQ|gI_rq z&xNpAF=z_%=bWp;9d=-QG%6Fyr&?Z`J^2NBFbA;^l8x8x8a>Cdx}nNuv3xaivd=~g zi>LKf>d>la2OUj-Up~;`$Qh5`9kDUmM8hgc{^lrDzEWWbq@gNTHzcGk9hbRXHC9M_ z^L?n>UL_$rk*zZpBViAAD*4p}i;EmDrH7ENGAyUwowVpfSe7PZZZ+N@xnWLq zHj`v?3k&eb3r`O}(^OKe0xs$;(j;icz<=tzXjhG@1~t_!_>B~z(?41##O;&c?6APGSqv|_f8z4GqlU!nxa?GjyX(uMPOLvg?8Gr}a>5kbMmHoeWMXRU z-UY9kH1&cTOGb=zu`FWDt|0dfZU#Z?g?GiL8NZGLc=FGGBU}H__afGTL7uZ7-6lwJ zak7u0ZT|s21A_knzN1;=@u>(2^x`vf`=`nWH&pc>{^>T1MLhBFIQ)*+I3o6GD8G zpV2a)OHtk8t-hNR#eAptKm-nVGt=8qzp=02M9p?Me+oVuPfOoZVG7`?vsF>uFg90exDD`u1OqG7m$mqWW$Yba!(W!^WW~6;Mv;ux4aoNjVcgIE zvB7uAxek7_Ly&nQ>X_t}sMI(ah z#C_KEA)Gljwfp+w*iC#FVx3nbM*1{7?p^}9g=Cf=G6zdS7r z?{x5P14m!~1Y`u1Ya2>RGu?a^5s1Gb?e3!7akWjP|roFYqGE}?9Z<<&$BH|Y=z>Q7F z^Gmn%Yrn)1J1uJXxw>i91RrbpM7Sa8B0=1PG%`NkkSF z5&5ieUl2Bt@J+E_hlGTrUM5q3r!AX3r9dHc?`cqw`?->31t%w$b}{p5V6x7BABBL- z57jkJCE}II&PJ4emdi`5X?rzqwH2@61B{4HTQ`0df@qaq_HKs<2=<4|05uuH2-Np~ zkTet(lCk+xJ(>UL>h-w1E$DzLSLT1GHm2(g6czOu!!T=Z``;rUG1|HE|GDq`J=e>}sB1Akq7+od_jA%jAj5i1Enr z$QFGhfOq0B0H#iiMVt-F8A;mP|M1K_J1g7c2*l6v8Qry4Vg4aJfZ~XqhhPNaoLI8N zWjU0?!1v7(-@)53)+n6C3QlaFh~#LpGvf}=GUlBo0eplh7jYYKK(M^gw?q3f89gy) zaRG6XuyQfGJAn>p3G8}pu|L0VhCfSn@A(lf3x$${j}Ks_RbF0xyK*wCysR1!ZMXNCCeJ?v2SKzu4g0 zOmZaw0}_xW@<$ZT);>ozX@9+|7aR_|g-A)6SiP=a{@}!XoLGd%cA>TSy-FBcI#2pw zF%3MGXv`j38J0jsv#h2?ThwmsD(;q zg=bCT_hsm!?p1ZeJv}{3%gYjEz*r>mY(>VvInJcJx9=D=$XJ&h7#V zWfIAl_kUI*mP_&z)7mtbx}f6e;_kUmAQi$@h^M{zajqv~OvLo_%{~vJHl312N0N?ZUwG)cc9*SbJdt>^ zySI925vYn!TxoMM5|}VdsFM{3UV+vlcPPYN? zXchL-As*U0rqs!T=dSrcmQzar3YCNetK$i8pg6nGZwRQYMw^a{bVTzz#5XJZ1_^!VK&z-ln#;ao^F*^9 ze<>pzDJKN>UN9gan~RV)lVGJEKt4rimESa>xhf?;!{^`olQqrH1`9TE@_M>>70zS)q;L0(9ETP+FA-QR1lZoXeSu(3ff6kLVH z=8pDWnCa;Jxvtfn-lH0RygBnw9tgi@hTFXTY!9$!I}NH>yXeC%b_D;F6R##)j-&$C z?)2rR%Q1B=Epj?Bg`EY{#qY9Uodr>l$@;d;^_cG~Q#0t@w%fs;<(K{ZSd)hizaMFCm`(aG*-&s3_=V zx%2vS;C^-W7taa9iDrGLHG}g>f=Z;9cR+Vs!~HW2cQWDPhJMIkIt+G2l?Jl zb`^Aek;K@F_TX%U9FYwM_s*{E^Y400%PiU0C%=~8f}u{2poc8`vAnidS!QRUYTgm@ z54n*B$6%-Q0tZE|MAY!_vjiR<3SVW+<1xYJ1;fh;_?+{NTh{Tas#5t9r)pF$=CClY z4GZSg*`zvaVLh4y3fVkdY;Jv8nA@Rxmlz~VUyEYy_>MBbgMd)t>S2761}97EV}x?PMM23$h1inJM&mLqhr#Trd%QLA;5 zD*X%sfI|EKmJ`Xa-s6vZDfVl4bJC$zLVvzbW|MK8$-qfrXmj@`zotkqafnt)?jGy$ zKGK4{QiH&qmVNWR6jyhN=>x^L^)zmqoyz zwrJvNmd1_BC&X6&e>C#sFgFU_uqSKl66qcJ{ZA9(O0jR3Y z+Js&<@~NV(agDCisL$}Ks2=I!y2WhoHWk+Ij7HOhoL=kb^_DdaRLVaJhs;sVn8Hye z_@%0EkI_@>PQlVvGzzjt`S?U>3{LKN`|Kp1-NRSss0n`mKLz78xy9iKa*iK8g(A#W zvq=WV*2xC=IU7se7phTqVDY)>m8wG#v^ywDCjZA8n$&YHKd=acy!^f>qk)<;MvpxE zv#iZhfTArlNY+JNV@0v8519v@KW4FOl@TBhcWJDdv(II5W^Xoya<6jqgE5eB|KOvw zxw2Bk!gSC?A8QOq5iEGRQnxmhCRQtD`27~r`Z1O&j{A)I*UhSpIq~55Np3Wq5W|j% zIAkyKl?4coC~A83qGDqTu(z1KjirKsyTye@jxLU9&wShYIti^6fIiXkvW`xq+7IQ zz+DbFO3Z0tmScSlkP_1|8@T*CSJTv8+5%Qb0+k|?e& zGUvG_ux&5ZcUNGTNAL$bR#46H=Y|trW;?0*%;VovmcR6l zWst~^x!^*M#O_us@y!=3VK=B3vwyo027ApXar;!*otFaMY;M;rKG_g%AAI~vM-R1p zRw$b_^?jQZ<;(d}=WK|S&Z9ccHuCW3`d>32pVAMOBXCj!D2k~TdfxC^jDE<>sF?x~ z(6sxAbX)!9=gY#qK2&EfihO=v-nNz5ku|!7%!^I+5wBU}P%geXp^(UUzVM_AxLQ4nJoPAvrOcrM<(G_@y(59<} z#%t88Q(U>v*XuZ5d8(mn=y=3TZ?QHHV?ETUGuh6P=FPL)bxhhs9z z2I0_ScUWNyOJ*PVDm6P2GPU}~0nRbU(Dx=`d>c#X(V8Y{|=CrI^A6xvjjU`CvgqC`_e_c9VDLEK=$fc0| z%fs0?z^-+3b#uIfemc7tQ>Sbi7RocX}1<8M3wrY*2j$&(u=^K~a{BtieKGIQv3*-@kM zC;JBDF6_4LV}rpFpUGs_=vo_`M*9H`b9ZM!#gf|=@m7dx-NWQH-xtGx`+A2hJd?@n z@f!`eN23%UO}|ZRIQ2%>hoB-s(wDKX$rD-Qr^^j#th7}x_dF=+(jOwA#wZ(m)j6JW z_Afxgi5&tlWwbd!V#NgXg9y`HG}c9Hla_N5yaEnpppq|a)C#-*)~KO<769M(9ikFf zGv3c0L?`7vB#Mfude*b%4U^$|+T&PDU&5d4NChKsK9$(J=-NQJKA^NI>zl1?E)8LX z5i7F!dp0qv6|#okmnGLvu;EE&(`xppa=k7jNk{oKqZ3^A7WwJL0JB&JvjGUqd8PF* zIOb$=M%S0cob8%5wHwK2%s|ZCpX<4We|TY=hDGSh(>7iwrR zc2KjzEfye!fP2g@_)##z^gN8pW{(w{^ zp*p^~z@5E~tYU^}HjA|=LcJ@hP%xtbUaF^TtXQuK0IPuK5(zwcSJQF5w^Q9kxZ4Tu zH7}nY89e3~41#Rkv2JkQC_fQ8c_S!`sr*0{yoIOjioY1N%H?6l55#y1aVO;fkM68m z3mQ()Rw67>{7bvc{uE>_kJc;bFGsE0>2!Sf0U}(n{FE|~5bzjpME1X!F@K+vzV5IF zP}(iH5LLWi9UN8%HOlHvhQ@7R|KNO((UNs4iF0DOm3UGj8Nkz;ZrnC4yJeblg@7VV z8pzu>T?XZ>o*dK-cOZerBTOWz{#U1E`p%WZ3ZpGMN_q`PfGI*@wz;NvL}a9hu~KI5 zH}P(SNIs|GGG-W>uZ+9HJ%CXQkd-6R)7U*|0xrclZ=Amv~7e9AnA~|Ok*3zXTSP;4F zj3NE@xL;LfRy?x*@?3v+xs$B3mi_rJx@N1yU7>a2 zdLKNP-YR49S8Gzj!)rW)+QI&fMB-_&iV!D@!;-+&t)~C|B|hC*u}RD`nG2tj$^J-o zqI6|+`}xQH7Mj?$oeC@zZ$k(%nwYKoao#0BGwn#Vp1QMah!C9^MU0VWAe`fU=Js5V z3Vf}=V5LHaVs%lZJlo<0&~Re&He>mWH9Fp33Y%jWIf_{b8LAaqqZ^Z~2Cl zrmexDdvUi3yZ=e)Zu#8}&*QcCeI&x0XVo zD#n#A|HN?Xm_(u?#KN^$i(;x8K~Hso?}bhrAJs83qC57~jvV2>3thYs@OuFkil-H2 z3XZJlb@bWGit>1~gtx{^E`-#g&5&gmsjJ$1xGT8P!JanMJlOz2@3}oxvekzFWCvtm z6~V+5y9vDR(J)6&4%O|w;hXDWn2*IkXlsBLmxd|NU%|jrM-6o$X78X+j)k7Ly-UNz zyT*eeEHi^U)&2XOTj8nYe+y~4H6Yu z-kL=~8M#^l?o(umkzURZFCr|DJsW$}DIasoC7F#8o2?QwBfob<1O}_in zpNh*BblDA*qdT?QIYE`9`CV3;fXs6moZW{`H%vZKw;IfYc#D}?Z>HC$=6v-fdXj`3 zN(qSNu0g^gP0TeYBolc-PU#w(fs=EyE8fPzwpx@gCr(9rAvIG}YRj(&Kr9Q-9S~@u zR(D=ppc5ZxwNX$d^}F96-(blZE3mztR~V|z0L>?R6zMHvdd#-!>wX?;K&O}bg%Nut zyMAU30-Y~4miM+h04 zq!~P8IKqu7Gp~8(B3FEZ+rcWGOVAiqWpNaDw|TWiE_iFrrfxpKd>y$x?oog`KEEza#edNO+V3UQDb4^qeq%uX3*)#x!Ld>r=0N z?EIC(2IG5Hpe1fvGY{%P+&Go=CzLwo)bw;?aRThL3< zN5!8|jmXe2Of*X~r)spx6!b<9Pm|H^=y?3a1#vbQ1RO_LO}=qy>RbC0uShS(WGtj5 zFggD%jZB*Uc(yBBfWLrNI}LUr1{xtuEQq?v)XL!ciXo9%DUXFlQfIN8inBVF=c};7 zwMu*O6K?z9UYTOAUq@2l6em3qmB`ld2Qx=k>8TU%Ldm=jV+_oC^)@C$fT`~OKy1C z5-ab%JQ5@o%9hk)K?Lim6gM6-qp#nCpzQLqlng}i}b=ub8J7ld5}T0S5#>y;_=S!W`jNOmWE7T0B5L{k7sn+f=T$T^=OLM!_my3*={i zIs3fGn^1@-!g?vz!a^6lfjpciF6@g=w_0^0) zl{bsY+}?~E;k(^3(m5s%-9G~{&b3#WIZmE> z{iFHM;2?>WU>Y0@$e8a#{5Bg424^M-pv|Vg3mU$C|8VuSe#{L@(mi{fUOXTr z@e=@7y6u}D;?RPL{CdYXt`8JP1R)Eh^ZM3d&MQ@PZsoDiYn|vn93-?VXXq*au6FqO zY(s>RJa+<&I}7lbJyhpzWXCHznTdr|f$N-hh2`(UkgOPw^Ey^dpC^Cwo>Uwt0iQvV zXBFf)9QIp3)pYq6x8NhZ^+hI`XN6hSB!B|8NmIXm1yimo@{+aPu~dokrI8A_|i2%!Ag(`HU!cy?#rW8G5ch= z3NtLu1DcQCM=6!#0mrE)`$+BVKb%sgJ0?XWC7e^#NF37z!|)T70g|Rd33_)s1db`G zpOTk~qY-E2rB`DOzpYKQuWb_*Sgx@0ynd{pm3V;oqLZEQ!g5R4^=6J>Jdb=Izo$xg z2bI0v*Y?{LBLWJc|E8tLsbxuGvKV{O*lS(vK}Jv9spQd#!NBjT7yBGk#6+N%{VU@& zF>+C5P}J>18Gb}%O3JHG*eZ7dEDB0`mLQs7XB$5hGGZ=I?KK>4T`HJNRkT2@W6{+M0rz~)P`mHbUuO?cm_u~DOmcF&<=*Iw+D)RA z`acyzY=kWppu5MS4EUQb$8&~SH`?)Xe97#mR4l~a(;?Dk2E$Y^Eb^;nV+Y6asllEa z2CipE$Kan{h_jkHu)@?<86IzJxkO1iw~H-pIFKoz-zxvAO@a23l>TGzbwV%VeI6S1 zBdVczWKGJ7y~4lBU7U5J>cZt_^)NFmTHSpVl8!PF|SJ zb=Qrq!!91ImZn`PR{!NoM5<}4$w>9Nep3CrB7jrX$jU6(e59EQ5{Hey=x=OZw{!k);B0$kOlPQo>s&Gc? zSr-z;wGE$e;xT)E>8|%~NlX+OGogoTkLtroX~u=4`tYe!@wXw8zCxR=Zi~ft*+LDr%wQBh-tdH`s{? zn}gnTVpfT^ZJN)NGBwER7C6vlJ0WM}%MD>7Wlg-tx+?#Qg%Wp|DqB{E3bePalG4)s zOTT$KzseTYmBFsVMTtirK(51C2Z8-xyQH2>yPe4;+FCz~$fT=npM!QppJvNOM&r5d z`voD!Db18*mqnKF~w32s4h0$}xuWF)7g9~hPK#eCe( zT!0tntd%Qb^{|YXJXuH$Z%A&~ktAF?n`AwcFYoLbC`(MiO!lGrtx?qV(p-bD_KH-0 zuqoh!|3GV*KC!n*SkW2RPE34IJw10oJ-?b{>>jqrPmdGJd0wJC@l2frhmFeiOv74@ zUi>|XGX;j7v+i5iFqe=x2EJ(I#IH|`!*8ogOS%0&j8hNtfHtxiW@9zRv+HM)(qr4d zh=U(at7!$O#qFgqO14Rq?kaF(I5>Y|kZ^A?r{+YbY=*EW*wNo~kQ)3dDhsD&X8yJ@ zhPknjlvbZdGr2g`Pl0p_q4F|{X=1XLX)ZRXKpK$&@894{T)VBGtTO&KP0*A?KTu^w zt@@h{(myJ#>56T=q}@a_<)EO1&!nTH;yLXV-i5v@#Tr#`bni9x)rYx^H{CfKVus-} zPzd#AqH>2-=x;AT7Yuy5ebO{cp<@1CA>9U^AwBx1m>81Tn?;sI<1=+%NbV z1{!#@sI6{f^kGG#WnD!Tw3Ho)ehovo=I`K2f+1JxXhw+xdv^uUir8f@_s|G@RGfJ* zVslXux#VuvqHoruXTM*5>uY_~O3T|I@SW({Wo+|(sr|f`s+z&C47_nw9fKW=QI=nPJgj~$&>VJEk*c~2I<`|L#V_cD!7qfr#`-PiuveLk@438 zlLBFbcFnV!=xF zy_k@D;IfPj=kgK>L2B|*kl%XSbp<|$OY=G58D((r5-^OlBew>#;>6aYbi(`Ls7r}r zp5+1FNkgy$|CU?xsqD5BJq>o+`4ur(6TthM#X=7Ua0GEw32T)0i2TN>J3B&rGQ?OS zB5;MJw|R#E?ji3;Yip(df!Ov+p;msz4cAEbH*A!YTHfV7)AlgOkH$OwqrBiT>#xAa z=p51;qY2xODdz=L$GHIYTjGha!pjS9k(W-h!$_*xcT67;a`y#fb!Lp`eeReIf3P*6 z(Zd#?qT%yl)MSygT^2xR9Dl%b7`l7dZ%_$y{HJ4H>XD)4lXW8_sb#fvb|ljq-|hMG zq!$CzQ>1G`;Sx|vtc&bcI+|Q&tpiJ*r|T1|qM>o#OQB-`N7hjw?lsSh=VF2DWS%XZ zd6NZ{4B~t2@pHlM-9wEZ3`H98%WXzr!?#Qx7Gh?be6|59ON^Bm3!%JwG66!eN7uD1 z2&-3qoS~lA(&Rp5`bn<(L(^X~G`;&pb+#%vo}c7taFbFL9}%ECQ8MX1VmO?$o2ZJ;IAcf~C`NG1&u2bb6Nbok z^fot)i?MHBX&i{5JbX}z2Ydm-N~X>t7=5wTO>iS8=YhW+?H`dXP(`!;DN8-H0H0kS ziSLMWwFuIAdBiq+Idvf#kM5tUFkP0D`qc$r6fxwy&nKrT@y;q7&zeFn*8`}wr>C;l zQAl~;wQr73Y*f6sh=poc)RA?;c?rupe%_8mjCB1OobT2(Y$p6o4Y6{{X&RE)zJ z8YyAT<^AH}tIH!lrB+?b1Qcc7)lOX_?s3fg@qQI4}u zBT`MgRk%@8xO~)`YjJZ|;~ku2M4&ab;@oiqkRk8N(Wo^eTs1`vv6b<3#rG~|Nf~7i}tpK6~J>al=FkUOhem$yG=0VSGbWDx65^lF6<-p|dOS9>^#-+(>e! zoUrA~u=f>pT_b~NbM6Na8TUEZtCT>k7T3>f@&@4~k{>;lOQ#+5BP*EIbyo5Ty`wj} zPfn4a^Ib zji|X>P%FXYe~-GV!t9FJ;4*2fyE`G)MF&>RQL`##zTWV$y2JLn?jo=aqjDO^sSx5P z&lS^naGK{)u&g9nabpnTHz+?N=4=X&=>MLz>~TdM$%|$Z`jyYj*&w-KD_LP{5-$mq zvW!Eb-LWG859@TvEEAf4|6@C|XP;}3wwS5W76Uj)aH?ToE-{9qz?1L`Nl5rvk&OCh<<65tBSJM`fn+rCo{RrF((~>ic^Ea=rN2 z$O$#F%VtNoJxpHC#Iu%%&OPH*P(QbjlBeFwGyEUnEukNL@7L)*@MddL)j4qDRA}Mx zdsHZnA&zR>C!p38?fdnSjSP#8=#sh8+aYmnCpKGY2Yqu3=nmqSD-DoHCouz062#l1 z`9jr_)VG&LqQS4pyBrG0&jUX1g8fsynU#z+_Ltw;czRnK2O)CQG>F+E8`GI#$I3Q( zvN6R}>vmK>=MwK;uq>?6eVgSlo@w&=QmlNgp0fCyvWcbl8 zA4<7Vd4rgzj4KBF!F^N{#r57-J8WKU)lC27;bY+{nF3<7EG7fLbE&w@$ZGo7WWWTcM zpX8`W>)3=}6F?(;4{&om8|05>LKC&Ma-Kij*Sr0M%BmTwUa#b4y6*ZMM_^##aV1D2 zO8dRPmQ&#T*_#X3WZoFlvWtH)N4&(rx3 z=Wl}7{ei;&hqkkfigVf4H3@+P3GN=;-8F&W9^BpC9g^VIxVyW%d*kj7jk~+u&RXm2 zz3&<4{JCS?KRw3RFuv}pud8ZS&1b%Ea407>S{B2w1A@V1HqUK7Y&LzyW|b4Dtj8?Q zYn8@lENL22_rEW*;F4 zGHxDk`^#O8OUtWI$dLMs>VH)8TZkh_WLWoI%nNcEfmY!G1cv-ql&e^y)WeW_EIR?w?P3kC$l`QpmO6 z-W?=s$X{GTciF%yl=k+|Vw&CH^B?eDzB6KM)9(h$C`EW{ki2kIUsc1*(MkH)uG8kV ztB^x#EU~g}Jzw_W!G+ApGSw}|5lBiHP^oe;FUz{BdT6}VlLs6ve2(f6iYmZ&z$gLxixSC{BU4tR=NtEf{Y6!gAcg~k z?l1N~0OxG{l9cp@>Mf+VT3>nDCv+o=xO*^eOvb{*Yc(*wljd*Js01RzJ(7($Jxspu zxz|GNKWgKkOxPY*t5ZwjtjBw)q2%BA9_QbMFL8d@NfPk(oLb2ND|JP6a3duDNF`uX zFbN@<+}NCBpE8(OpI*p<**)jRUGub05?yvn$qQsKoQwWd`$;)xL<6CPKRqlwqT$tC zk5g~F?u<84=oeQRx%BSk_QSRPb!X#OC+_frUKbMp66$*9!692rg0*(zYb88-K3dUC zY}FC8!YER$JPsO@f6+P@qQ(;!3n}UE)Q2M$cWVt;i1~=u2nr@dwWaB#Jn^{{vNwo1 z*SqY-8(rQ4H%6y#4C>Hk1KQc_FUnG4w|@J%%<}Q?v=!V=>VpVzWyskz#LaKhh8rKb zn20DKRYr)HMnehlO_1_fa>VlT!Frc6>p}IEovee^*|9*w5QXS*O#zhBxpH6B4+v7(WF}D}Z<^^5O`c9L{?;b!IQ64>hZz6R!TS!uf)V zjyR(uUmjoLmmbj~54|5mE3P&9hXxb$3QcPFT&LjJDyUgTY{>9O@Ud$2DQhUkGlzkn z!e}L@I{QMAFB)b7j&a0m9>jq*vetcHI$K`_5yqS|msC}n!_xyLs*H9u*XsI2s> zbbt74BD+T3(9=okfS#v)iw>p1?6!oT?|ut(YX+>4nNK_PxvD481TDD)^c&$vkM^5J z11|IBpis-%NK^>?d*wlidLXQRE{u9SzlL2a9uR#S4jqYuqZ%~6x}-|U%u)YaaqX6J zbh$PQjMJ6An1-wuwmBO`R@;?MRKc_(IkdEi83hIKd!KG<3F+-vd6-!|z`>#VR%UQU zK%{{KB26xNlOhhV z+ZECYr?<~l++NFR!|gD$%Hz5#ck}n&lL#WpCEtRBJ({p`b1ZQ%zo)0Q3|&njC_W7o zxlbMPc&hU>8#WK?Hai&3Y8bezrri=Gz)UB}*Hz%=tg+OInS0gDJZS?J@wU3K?$lhL zH+uK61{5mUwUiI`n6@+SrOBErExo=dyS)xkC#-%D2I`utp)9M@(-O`kGK=g##e}F@ zTN8+dj*5HiiOi&(1bX;NVI`c5PvMi@FB*_YioQz88N!Pg`xmN~vA`KS0(VbpwfmlG zEbGWZ6Ks0&`tiDn4H(Ca#iK6Su&d1&=}OV4D$cA3%SXODTbc^f6S0p)P>tgBo7_lO z4QPbmtnM;aX_X;%(7o#D|0I(J-fdkcV%HWQga|i1UT16v26+ll2|Fb!I-kP1(#JIA%CzsOstx z^@X56QM;7h`XiFyT4f!^>m2;JUdH;J_Jh~S&S%nVSIGXlVj$a*U;OC#m4@^C4oUM% z3r8IkUX#SEiJR*j7sElauyjtg|C7Qow34nSeIJJw{dXYAjx>l83Ip^02=3z1I9BEEa^afOOsTHq7x{cm+>!+vtBIPWmm@cZ7v2P7T&OjqrdW` z`G@_p85ZxN4OI0nZRWiuu_*93zN9=&8JT#^9h<&oG=~ZX+0=OtXQHy2PWY{EawG_^ z)EtR~W-}xHQ;Cuhb9=)DcvFf$8|gSTqd-gCV$G(Td{Vai8h42m*>U^wyCNS7GI=L1 zjmC@N%bLzv-@}&rcrgDJcoqG%Zz8JN^veB!qZ~(Ch_oIQ<`&P*k^%heUb*W;Lf*t} zl;}q32+d@v?ewms?py97BbU;ro89z+*}fTy_#yi z7YXh;Vm~Aug%|QWx_?z@f@@)z|66U$|I2kqMOc7^g->=>+UK#J=k_AH`OBc26MU@3 zhooZ>a7|CdG~c4wU5M8LPJWE8*b0BfmO_!na2yLtFxV~0=Ib!i9_cCgtN4kDpk#qU zr$6B7s=vi^E-mWnw6OJHq;9)Cd)R3j9}P*y5>%&HIp9Tm$x1>TE%V5=oYeb`i=N;H zhlfj7tAw9%Uk-a&g&e8qtouqvVEnm*M$MPBKsWmzd1tn(Y4f{jJBteo8xxn9r(WBS z{CF&{Zbz8fo#wzY~P7GE9J6L>23H6E^6RPYghgIfvwLfKb z@tl`_aec;|Z3qOZ>NA0CpvE>_(s(r^oI3Zd~xw_*50bCeY1R+n8adlJx?aE3dPYN zqRR}PuGMvX+(H3BZXlWX zxWJaC>lwr?4W^v*qVW6M_3ze&c+QVw;NHY@)I8NPJ|SM?gbrUEJ8IS8XJ~~UL|q}0 zBQ8Yhchl8*m3y>?SNXc+6bVFyfAm`1UxF9~JHV=4se%$6m)A5++#-rnXX*^r$O%2? zWO6n4D;lLPz7XU02lO&|+0CqIc9|xS7@nMV-THP^ISRs9k74zN3|H**lK)+(cW*34 ztmwJ;-eobOh5>({y$jpI0q!!vw7XG{&9&K$LB#;T+8xN zOynx%8N!g%NrZRv5(?rF;Pd(}+$(!t>X#%V)m0gnn%3nH)oCF5*@v(e;zMX|3g?Sn}v52lC{&(q zVc+eT>9kK2Xi(l+YecpI=A=;9=$G&ybQqR)2IIO`YN&rR4f)%rQ_fwVuB{g2-p%nbzKFs# z7d$%29f}4cx77aIzQ;A|+trzDl4wZZ!wFMOP%L86$rWW>z_WA!%Wc0p8e2oA>CHPw z7v`O=Bpw}4w6%FWQGaf{p$3zQW9J35!EoWCCHPa#@I)HSrYL}c5Gb*olYh+zPbtYd z%v2s3+#_H-en|GlfV2MM5L2Ms>AStvl`1z!bw7Fgn4P%#p5J|yt}PGFd~MHI6|$Cy z%|dcb4Ot*!o=oO`{Q%qn;I zKHrY1J|7;azw~qrTvocdV$a#3DajAy(OYDHoa}5PccsAoOl$X)49W+e>Sc`{>Z?pg zA-pI%n2)RCr8FKL#cnB>O;|!NbHHZ@TUqdKMHC2ty-OX0M30Y;ZJ+t=dTZk$;_FNM zCej#CRBUay@TqU~RALcM;B26k4tw8c)P+4}L51Vvn@w3WuiYt>A2@}m5LJ^l_avTm?>LvK9Qi_rQFsL8XV&=&7LV$8=1oBS*U!3wth}?B5|L}cOyAu-|zL~ z`xSfyBhUki5!e@xs>5!o7+pH&85KXLWYWby~l<7vSpp-IZoGJ%GH2 zu!X{3K@!c=LUFVV>-@-EtNRv%Hg3|)@$NYNk%SKKq_X5dY`8pT(eBFW(&>qQyMM{~ zTK$SM%d?@(AkZt#_?dGxrGTl~a7Q5TrXy{s@NIWJ@zqN!6`H{H%sCIu#p+r}DbPcDo*xRxNwU@+-}tR1Dh#mcZ~$mTOi`i(VutZ6A6-Bo%P(9M>6%AOz9?i@eIW6>0cpRg^iCu+$AGGt*`g6)ZTsh$_SL(F^w)#lzzD5$n{6S4|jBZ+yok zDObMX@ZRQoW@jV{o%>|Yw@JSSh2_2cxE5xwWEQAs$Kvh`k(h?B?F{LiPsv+_13R6J zcVA{$q*v-ID;d!qLP&e*>kQQYjAL`OpH=eF)h1y{{wOyu%Q~LT(0y&68{Gr8jt_eL z;V>7r7DEczrtfd^l`|Z1*!?Ayg;d}A&16n=q*5A3!>~&T7mNR_93+S_C3g(>sf)vx zry_WGY~g%*+%FyiC;hd{KLBa`qg6WY2JViQ-ISIFXoOa5`{r40{L*xGIY{M4k|?i1_si!g+?PVLTX*>thWt>1`bwXq3|3H|r0o zdQWIV|BSSBR1Tz_#z?+5o`Gn>SxZ%W00(!^eYQ<6ZXMxANcEZ-6S|ku&!5)+xRnyy z-9Cm-7x-6@SufKUj%fc$XhsRPwXA6b%+ z)le}4l=y7r-h$=-W${-uK6#C28YT&T_60ushjCWx`5AD4oPfj5IYStbaiF_Wz|}gv z@8zV=wk(7@+2OQ?sIEi547jct^sS0QIV#65N*W~WkgOPV+=382Q_9}jP~TR;iD|Lq z$sfTMzj`(!42bY@c4$%M@V1x0qnaz=_dkdIFBZT;{Pg&FD-AHU-k>2$sz?2K-0RM+ zq0g}%Utj>pw9|ORpX|1W+MpqTX-a2l!ESOmIu)kDgP25~?EHf<=D~c5dV# zWKmi+0C2O(l{v8PJNuEHrK<>n9LvCZ^+Uft157jWV?{zkP@yfFghMsO>fPd~9Q6z@ z+-%msbo7BkvHf5kdYz?Z;jAIEiavsqA@Vcz;Z^uiM7l#w!{Q`_$3 zn#3_Xfs$U{mfyB@IE?v`yM4fs!K0LQDX%vGamJ!D+!*VIJ?;OES0s$16wT{{fy!F%O zTPY4*pfWKcP*w3v7O};0kUvH|E`j?cl~O;eH<9uAHkHEUh2N1X2A`2ULx{n&^G39WRP}ka&S6E07$=4OWi2-Kocupa) z;WZfJf{D3eEmS71o3rI=uEk2YOn@iiyFO1QB74)#a4}=y$64*P&b<+uP9!v@Sj(j} zb>&5Q9HGw1H)OR&5-*FLcVUCenH(pu#vwKz^Gsf<4C}2W_RcPIMy#K|*&+7E9^xNG zA6(PR^hE$)j%JXpF)SrmEPU_CGZnbW`At?x_$yy1!E5=mieWrb5W^r*0gHjLmeF5P_jkoa6Q>#@A=ly?M^yKIZX(<3dMiFRJd5p-!d(cu~X$Aq$b@&}jA49$n5%e&|1 z3u>WDzFd_nF3n&_e|Fqa3<3iQe6O6u_jE}ued(2vlE;^q`p33pl9+QO~!?-KBlwnB9!>BChBfAfRG|op0 zT3gwC$=O+=Use}#iLxRi&&ADJ-0+3w%vZ6LRMzE{*s(++_@ve9@2D4)r(KRIWzKZi zC+;f(@87Bu7gu+)R@h;edFd`z)|m&pLm162`M7TvM$h!dYB?gd58ojwzwXM)<$7bu zh8?o?u`=m+xy`Jsc2*Y67xLR6;vU0wvIy`7NbB_$_gU^Mu=kxQ5jMr+^9eONnBq$Y zB6K<4*n6`|x$%i)REBedvk<(&RWt_odJnwGs+6PvsvWoI1SZFscO%erM%mJUZ6|Sw zKieiPJJw=;*oIRBEP+}A4aV+VT`Twbih&{_9e$Z5i4$x^hfgd^b4IH!Y*i(zk@<9B zUu8ukbgU5}084xR=dBoa=UTE)Ftd7gS5(dKm0_In-^D9B*MHbJ9xdpNtL!Eoyt7x+ zxG?sBkp230x3atKtPt+Vsyk8?!*g-ZI;4X_lT+~cIGc&lCURyFBuRsV zUi4e*v*=-iX^_|CF-k)Pt+l*tchE>$#Cw?jNG38Gj6TM}p8n6(>hg@Cnj1gS9F+CL*WN!3hRV7!QoY*B`-tUb?T=X&HmLw%w2KMoLE} z0|ZFX3fHQ*px95hbme%s_ipwkl8zPc3L`0sfR*ySVH8knoXqgSznSzAHaF6kl;4n`j>mC1EFroLdkD%0Q= z{)TZ&i{OSrG00{d2vthwSaHjS)_7^AgCmdkJujdJ!6sAGZov=uD|yHGdBr|HjzS38 zqtiqm;$s%|wH1Qb`jJ2|vL~6E+-G;&X%yluK)Z>M>ym3L^>uC><8B|(RN#*p1Xy_$ zL}<9f7!ckcn62EeTd1gSZ%jV=d6a4R^$yq`pdpE2p95l4UN9D)uSQQ)Ak}f6p+-;U z9UECNA6|CUtoKZc_8dbiANzCWX8P_0r6^q0Oq?r#ZRmH4wdDz7LuvZ!)2Q2=S zw<7(?l=F9)&vjS5+L+6nfod+Pagq$`rKadWa8a0AiP8R4cr9HFK3&AZLOY9(CzKJL z2Rn=7;Wo(}8uDggo$>Z)*~V^#1`IXd&y1}-L(&RJU!H`5gQ=I5KSRPVDas#Rn6{Ks zvltDUBN^1b`<@TDBUBW$U9rba`B+a|0;7Dus6;Km41)tyYtRE%nGDI|=5MCze6f@I zHnB+spXLyFsNkcgSjKz63qI&M2+w)1K#iy!Vxz-m9*#V84E#WQ*vqgh98K2Q{s9tp z(r}XikKlFiV7i~H)dj&T;aGPY&^PljmYgoh$sUDm?P?WFxrGNf$}(k5VCTPNCAu&? zQ?i<8qX*wu6v@C{Sh)sHzdaH5BU%~#=x0;Rs7qFpaU-?Xv0GVtTCNSqaB<@%2@_~{ zi?nS1qV>z2m98UIC@_5@Yeb$Nmxr+h6u1tNAAV(GCct4mn>>2Beof-f#UCd^0^+x) z2`^-~r;y~yv-#dG&kf3Ad*wZvN69`z!}jr9`OK`8E#21oT>FHW&V%J}uPKkygmw=L zN^if-0UwNB=nyvmLm4D6aYw=-3mWt0*1-g9z)=A`ijkIoh|iHquU$+V|1Vx%reF*J zfRmeh%5m3Dd*v=u0esj6BY)AsR-@(tGOF&K^xgP0GFr;`En}Y+gWoLkv*N8gRJBs%+a%wYa29vQRNk5k{w>o`YmTV`XQsCeH;6|yPy8Rm@llu zz9O4iAekV(-WoCclY9fhp}?KOW6JT^7ecUh7i}zOV{%c$uzVr8-UuXW-0D3ryRl!9 z-D*Uyk^b99YfD{De-h%b|{(}$HL~4AJQdq^Gv;?(^KY3>QaB6$L-yR8&enu{oc9o4Y)aq*Z zy~eYC%(HB$7o0>KCYlv|6z=vIc}~hBQ>N_eMs=L*WYRhhjZw$3dz?S5V`=^_SmV*K zMlPC*8cWHtTVAYl=czY;A#zNz=o#8??UT$L`=_Qm9F`K|MBT6{)o+$kpokEd&lJq! zO1XdIa(mLk6Fq(o#~v|86@squhB{Cm{q=N}6Ib>{kJ#_qQGW3UF1#&M+1FaX+U0EY zR5-i{e=)zSou}CP*`IJmv3z>%!hZRqSd~0o+dc94%l9biSr0;q+`56k8LsH<)qT+q zeB)B-@6S3S4XmTmq#}Ay2;S%JU?cv{7BGJS*RU%D^L)Jkj1GF-Fa{#vgN&1)In}9N zXB&OQp8iOrMuIiH-jK>Rb@0@=;;F78>;$CxAO&QNZIreq8YX$t4y67;TwFf#IPdEm zLC=3x%f^JOA1#Q>i>8$wzqlGHvg~+7Q&L6pu}JzTGfcw6>T3o8FWeLd76Wzm-AJ$}^6a=kx`U0hX3 zlz(HjbfL~QbIRTm5>^rUH4u8Y@p%X;Twc&u^FLS<#X`Z^?~qVw9sS(b5X18ad&;wZ zsd{=Dw_%qOudGvdRvZ`W4EXH?4Nw#?KZ+V_)??xl=n7Yx-#54J+>dROTNJzu6@AR| z3s~Qrc^1ZdE1|t*bI^??iGLEs1hGG1ar7t_zDKCB1#l$c@qi4=J83(hnwNd-)yjl0 z(SP0@pYRUmZOl$r`n0tkz|Q34Xd!77SZ_!~hD@}4Ol%v8Sm9fegQQ%WfsS!FQuE&A zn!K>nYFQ;@@enPY6lij~W9YC`E9?0w7~3#s?>uSAuGtzvO&i1`%ByN8@omR=Ju0}m z_EZt;O}OTw9bdw9fUUKlg0xceBMv$n1nUY{BVbfCgY?KRuXfzvIf2D0;Ws;~H&F*U ze4Ip{PZ1kRKdIf!|BVfC_VqKB{e5J3sQpl5J@Ns=eVvq*I?*B2xaL+KYN?j$wr}hH zUtmr&$-lrH6e~3o95i^t;nI7GP*MKv^N}@2>|CF7sc!}3XrGsc^#4#p#f>cD{>~0Hy4Q_2RUZA4e152X-Oi!@3BH znD-bynBAw?O`AE>(3&B6|6V7XwdOImh=tVp?JG}SG-Mo$j=ssW=5BG#@S-CV_0IZF zu34UiE6@P7^GXN=s~?^=FgXRvbSWPmc4R=M^CyKj=8Z#6KfnH4y+h$!{6ce+89F#~ zv9C{Pr@MM)8p0uV(JgH`264;44OIBN|9WXKItOt%Q=DXr>pVs-U4&Gd+}BTMx^PSq zA)iMUAENvGlJOqr_ZWT8l|HT7x*saRjb4umIC^M{!;dP6uLN~FeDyt?atGEPM_HC% z`rT&yq8r6YwD(@5E)O^)A`f$roU^A_c|jw1`~Y<{-+R4@p`7k1!y5JB_zn9kQi`zd zH8vOB<hY#M=yW)#7*cDwa)@f`zNk>BeS1Y|HFV86Rn36!T+br|zyIOW(A7hgGLNb<^C{So6<}!l=Dfr6PGqyMe z4o1RukVl;FGJTebvK@`l%2$4hOZrG*F=dfN+r~jF&|Gz_iWsIIKGtStbhZnTXw&aY ztXgFC?$XL0CB>MT7efp&cUn+atYTak^bGbI(}<6@J7B8_P}rp!=na2!wOC`fKbtC< zWM&pkFq+7!bF4ETyjB8#BH_0!bZRi3Ekej`|t+44NRqv z(Mn&YRBsb@;+$!tC2&p^X}^P~C*0W{W}IPJa^M|g$89ejVeE(l(-yhzmSS#FCU|i5 zhL^QGQa8upSC05ik~8^xAs4lem?8_i2drtq-=@`;27{U8nXvY#gWGCo43c+xf2P7~ zGKHFQFnBwrev2IoBaTaEw$tPB`cbayp}F^wW!ZtKEZ>5WTGjt-b#-629Ab&dq$=dV z;Uj1p=NGYtp1E`C@HN0pAW`&x1A%hhh1~Vd3ilmz;oIWmi$sk^Gtas+dPSdL(!`)N zI1&_)-(d0I5TmX&Ha)5<>Z7=PA0R~*%wlQY36blM(J@OM8wy|6-7 zz=MY0+fZdtXH^dnUW;Zu1Iz0Mz66CsSN{Tqlg%#e=vfGu1o(ZAWbyG58Mk^``)(g5 z=qq3USX<6qaP{xS;XD3bFC*# zzkU^ak1M%{4|;3X6}Aas<~4ICiOsf)#GW25->H*d%s0wN{4mVUWX7X_pE>{EM2c5S zuBKN&1&%PA&_w$lyw&w(sP+gz@}3&Zrv>viwTiRnt7 z1M_sDUU~)bX$Y@b10fg)2ztZumnoR`8hZa85|T=}H1Yc-%OK@Nto@hVR&hKsfFKxI zcw*nm{erXQXJ|37+wQoimchoeNqvSN-bTh67O9xx^b)c44ki>Au3j$g2@mD_Iid8V)+02pklgv1Tg?c!>Z0v%k_|`uBZg-w60C z1Qb>&rl+(8@TK~iQIqC!?o~QYmiBX`>~Z>quhgHxl{vHUMn2wAji}wXYbUDLnGkjT zjd`4WF@NZTc7*?O${MHC0Mqap8HDT{zyP#Kd#&aj3y0}rvK(u_;)omm z6gU5tbZ=(;JQOChiAA;x0Jml`*kA!;@^=%DCs(Bt@lM8b_oY_EyT<<|QB4E&<1bXh zlZ9FG)1Mx{C<7;aH0GeU_7BCknALu+u%Cr~6bB5Mb=>SA4|oV+%ucM$(`f@gQ0#r$&6Shn8etiLecKlifV`Y2W3IR&K3Cb<cgy5(&EJU`(T@umI|UQ~nL?ywK&R!J}j`6r4A`Z4=j# z_Ji)RtUKV(#(0PWQcAMxZAa{UTDlWBC411kiLzL%G9!!W7$2(25>oY|G3b~6Yq%KF z{ynMnHVByom>HAhFQSSPACAE336sh*gjX^up&+BjADit)9Wv_287E!_k)*r~KJ^f# zh8lhh8`QZFb)Qk*YPjk)F;zs*9_;4BTdDrp?sSFZsf*x+`|5W$?ITcx`R}~W8Dfb) zih+}r}D{@qb_B$WvyAYD!g?rvaw8!O7VFy?xz%*snY@5>`)eA_9+R2R@^i75V6#<2cZNz1<~8Ga=GZ@V{au77^GIfJzS48fkn5Ch)C zURsPBi$VBsDzNU4;i(~*%{R2u9SO-AN$2&Y$0cE`rWJ?8WDr>}ewTS1=`W?vFX=vK zkgXnQR~|is$DLpYKHk2J-}ztSz301}{Mt@Vac%kkB%5P%OXj5nS6i{B=3*8`{41^4 z0uS)vfc;iXF^Qv>Fk0tw2XgB1xnp9oA~WL)|221n3W9@^sSi@We*HqDQI;ZpNHPSw zb2TCTSo`npTwt>*i{ka|IoH4D(Z6552mFV975FLs|GzGnIB;y1Rr{)|oB93U1~B0D zY7w1A`M8FT=4!tF+nW=(IoQPM*gGztsBHc+8RL{#A(avQ#1%kB4*ut}S(}WQmH&`5 z|Kd0B-<-Ste_TI?PxQSAt28*Mj_^jzE)kvP-Rk{?Z&JKnJ0x1d#cc%pj9nDXl* z2Y(N#W$?T&I<&nNJ>>UNo!L1NuZ99p(Dg7b6}I-(Wbsm|SP)Ozz)YqfX8jt-#XVT zEl9Y(4bQuV}6p6~}=?9}`fD~gS@v*Jf-Uh*fJA3NP)T=e{e9E=zw1vQS$#U)Vk8FDP| za*E0I`2cn&VK5?&i)<{G7d93u*@Oyg_ROETR3srJHRiJCllzP4DL@xdhQn20Ec7)7 zTu^tg4roW}FwCf_m=2?dT)k)`Olxdco?Ty-X{f3aUjQr!ZBUh9;y$qc`arx&UYOY+VKBF{Ip0DAB5i!dDTwE-96Jb)ew>Cl>Qu z=C>R;=Q?bRPXV%V+YW3IoiLh_XU1&kq*Mze3nSTTW;T?BkAIk!;Xex6Ug^~vd0^pa zYpdUCaUgrSTsN4Mz2d@5aeg+2x81UCf2CbM8m^yhW8&iuwzIhD$Oe0 zw~j^*Rug!s-ZWzw)e3k;*gy9JbcIwu=UZQFEp$CJermzu857XZKtO05mYprplY;j) zps?d@LWT2;*lN)Gkaz0JY^E2f-i|oWY&>NKd#es?M-9A4S4|JB;Zyb4^NJ^$iwh%) zXEuc*$>+~pp>*=Nv!tlBS@@1H2*rABndCJBzWe#|LJakk`hxN#(hlX&MN7Pa^&_W% zL0^QFwLNO2gYl*6EqSfpuQNNqgH-;~aCj*f(ia@fw_1{DPxOqWK(|Ed?LtwRCryxB zG5aGF(d#RfcT2i2RNjxs%kocTbfF}(=tk6+zEIyXPIYbXE>hi9V!T5+Hd}lsXvm#_ z=ac5%CtKoQ`Bj8Tc{Rr(0SZl#=KYaI^gnMIYTNAf~tn-}x&+Wr7kNa0}ONu8OqnkZ}0#Bo7a>QFrOUce7VuU>iXWD~} z1UoS6r!sfi%QaG}uTNqn0NwTAYrES4`uGT|&Z9B|?6%Xzh*b!%zEhcq9qh~M_#|^) z*?bl}&)#`_QzJ%mSN1m+&KSy|kd2QNOqcH%wBNi=hz>8$Rz8jE5Y<|?=)qel-KIp3 zuj71D;yu^1pJ!(+&l#iL->YJ9aDEzcOy(H_uKV!&zV1oDef+re0_wZDmZ6CpDC67O zpIdonVPS!{WMZ6C7}Qk|h%e-vZm?X)-&pUQZC6t+#ft{pIW26>dQh8|pazFp8+7|Q zpqhaTCxgHle$6RWslz#^E(TJ1Ic#qt1~WCCJJGr<%@ni{%4NbFF0mLl3x5`wo;9tR z23E{m`=+3=$$BXTY%=@6)DiqSMmFE1sg-+lR3Phc8Ed3#;X-ggt)@GvUv zHUEp#&%rBRe!39&IhNH_XjKtbE=P=p-#^Cg@sW~>oO9^JZdg_iPN1mP{p2}jqWDVqrDM5^vbu5 z4btFnhT#T;`@@ju*X=Vq5s^XOoqNpmTl(6*KWoo%&yf(*PT%Yt1zo2u1Rw4Ku~bZf zg;9Wk2*M3@>hE}b5vBIBB6UT-nyQ%MtVGXC3}8)M)`n4f-KDAyXDLIXE%+&rk^#Lr zc4}eaBVfZHl!hNJ7d$s86MaOIJdCWbooRLblZPxJ^?hX7c*0!X^Y$asgnK5X_gUef zmF?S0&SgkyX2BQJ-lkNH**QU7Vs^5{M_C9xhVdyM|}EMPrQZi>@+~<<$dKOg)ILL)MKHxkz`%L^Z&BB)mrW2sTW& z^hiGDYzh9J4^Ks@lVN&yx*Bz9?%v#^>Z8W%fUL$D1%@~@em`5=M}}IvO5oDauX?_G zP)fS^_2k?uj>m(7VMD`KRQjg%bSaHNO{uP1_AWEP_m)O9Ojt zrA=RC-~3F>cfXpp8hjxRLX(4^fLdq!OuJ!i*L1Kmv@q)HTmQMVrYzdb;sLvi z6ryW_)%y)O9}n&^gt3~Dw|Z%kXaU7uTT`HVHg%ZM%OP@Ykpgz;RNQ@26s`jMSf0a! zOKXVOSihIx0+SvE%v_CX(=(>}Oi`G>Ka>E6WyI4`T^OxT!si-9R5;UK{*Aek=vuth zsAdduvAiEr5^TBs&;C+))-iNM`9{_OO-&Cb0$YMfTUo%5kk`}wj8{{$m1YNkyW2t4 zfbuSfdBVihX6BC+A1&$_Qf^FkIvF%FvLyJ&R5en@BQ!LoJ@>PuUhayta_IU7D7{kI z8aDQfPs6M&M7E2*YF`eTx5_>@N4&UCoc4@Zt|yWU0ivEMuFphQSF57f*u&K?hNT3E z6&BN;AAhn>aK~q_FoQ<@KPX-yZW^d6t2_~(ig!?#2CJS7<=wSTuNan?b;+s)p3x3a?*iRB(mM}t{oI!U}sr#U#6bLj9+ z5V1^m)+Q^1_XU!P1F_P0kr;f_UXXv%-CH?imf43*K1(j-KgrvT&+h@Fxz8*NyOs}} zLb*cbQ)ZsNxHY=--hec1f1zjo_^0CV1|0vj(H?wmbpzM1LX?K%s*$k3_Z>g&fF5zjV{ z>8oQ7j|8?we0vqi$R^7*?F~S_TtXWzR+Ea#7*eP+q|NF|vLSg`G>(7PhQ5oEj`U`B zR<=5l_*_I9D5D75PtT4v>QX@CKTIp@KFbhkCN|I=d4#2JPW&ZYR~hk+bfMcfje_28 zpoHw=sGs@l8dt7Z6j=J&`YRZDY4QuN1PVXXM)zXNJ-l!reBT(+ovx7e4YX_1a;TR- zG-$GZ#-#rTzvjmN!_vrY+d!XhO(9Nm>Vr1o?8|*D#i<2)txHeNC_<$3e{oClBetGMCY3dn$soqT}5dKVE zCN_t%|4bXubYMU$*pgohLeG30>ho;IpN7<#sG!pLkQ|#K~&FxMpWvy6A@0v4im4 zMkk_!ca<_K*VbkkW=LkCD+;xv#v%y&7wTT|Nl&T$+xgGgo&r=u_QbTk8GD%j8iPoOgkrx z9QIAsh-^Gw7O}m3)Kc>yaeTZ9B_`W_6sD}xLHf}@brnOV0ya}u-03xd`QJOJN0%1m zl=m14RhUd3YZM-YVfMZ(i_mAF}%N?OmC&SrU^NCd+cV6Br^9gj*8lFnV zIRF3x&H?`M=sQ{Rw|vQnvi>ITD^iOV)2rnHvV+AC?)`ni`_r!><14?`Tl4o{U|wio z)_S+ezRXX-I>@yEqxU30w~Sj6PfNI zp*~dWB)GJ0M`VTrAgKNiaqk$P*|)5Vb~?7Lj&0j`V>{^>9otFAwr$(CZL4G3zUlv7 zYu|PDIp^d3-SKU_bIy0pp&B)+o_Z>F>eRvZ00nL=UCn8Sb7{)jAbG?HE%LLOQW0C= zk>aqCWX5wq#zXdP5N3?bg^oAIU_E!VJK=0Jr{ip`BNB*(`L7(Nx$>Eg^I8Wqc7 z34u9hI~Zcz)JF-->cU+@2p4Yw!#g{R!r1ow84-B1B9WUPOdutxXK1(_L1;QTf|6x# zJVeu2e80ZB(sk&1$=_ffKR*14hp$`$&*4lQTWdvh^>t+)0)1Tn;)eVpYsC0tO(Tbt z#rBbFGBo6#VFt8=5Y|kwh&)ChiYXkv;!)X*qZlvX8*D|Xnldf_SspsXFeJM}Mi$QN zT!$S2KRQ!e=UY``I5hZ$v4b75JKJhX*#>Q&8!W>+f{0rBZ@F9wnzGKlN5v^7l2MNLU5N$>(xa2Lxf~t6ZxRc|!2((^i2VI42L36Idg5)hM?h{Q z+89Slq1ErtnEvs^q?9u(^n$AB#H+^3vUl75O=DYGl%El~0-%A^SKU|ZRqSi?1(kkO zW%{X1ji?TgXmtGHCQP3(410={urSYXM4zS2AnDV1;P@S#fmOJIg;#HgU zsT-O>#cq0GSuFQ=vXg;%o(Ov$l<(_UZzu3cF6B?V+F?uVWhd1mNz7OzGa2P0-G#2q zwArKTSN=Spu&aF0>Mic8ss~&SiNe0jHWa3pz2d)9oy$Hu(AhVQnK4$vlE@KCDt(2e zPoQzn(h`ekBFbiRHJ&Kv=sh>3|8!Qfyek}TY5T>t6o9gWJRjrdytE)0gl#G$sKAX> zBH!3ahr%dKDSt&Nlihwkbp-lscvhB-U>jaB)CT~>m%&D}^?-TL0GNSIRC9?rWCh+vV3cwM%e3RT2va;}8QVt7c8mq9DJ|6{nSu_n3 zr|auy8SO5Px|kpPqJQ>R!lTvOhONgI<5BZwx=GbABxh7WYd#$a@4+66DnmdZMV0VZ z(_SuV|9J0f2UcYK{L)|v-{5bWEo8kc6$CB$(CzcRqoBA)sWdnU5rCTNOsLG(9}n+tu^cex{CAG+ zoKKg&pG%6h#OKQbHw~JC;r&r~d3EzVW`>t+%ugr~tu%uB1hd47p#y=*>kWREOC44I z%%}Ne3|x+%ji<@F^{gLL=~kjj4YmP-od65h=1q=$ESy>C@rmRSW9S=8x^>>EHGz>- zXsOE`Gf}sTz}mYc3M4Vt_#xc&^uk|UiL#4RE62nn@7U$CzE1qPl`iLxW3aJp6MlXw z_nqE?BM3O?aAq+V0SSiuay=c&Otu-Hz)-5uRJ}2OZE7F}ZT4O9g>7%#2uZSyp}Ubh zTe8OZ{YXjS7QVwx8Sh1Y*Y=x0)ybTbuykl{n600{UE|`E`sKmS(mt8ajWVy%FXkl` z%l%RnnsRd!>n0?a>NM})@275Z>XZVx+M%M8_q7$H!qYjTCaTnq^M^}~72yca0S}}s zlPw)z(M4;CuDIyWn&Qx#L+vYdDLn41ZI{GX^WH#pCR2mqTk4?@Mg%o1o40vqHD!kfKMI;RL ziuWu%Xlc|X@!x#`{y*EM!kQoRkK+pfRgv>dF>ge*FW5^;R3RO!O@|+A0lqNgxZBJ`nxb+IB`Wg$Dn$q zV*hM9{Hve~WD6}@qOR~a4C}r z-zVJJ@)h_&I9t_GHDQXDKy`N-NrY&BQc7CgqkUPtHYJc+X^Px2Vf}GL1*JP?r)Z& zhUjrlCiYqbeE54Q6D4t@rg;;)~qQU%|z$J2#;!KTn&h ztpH9aK#CR2gWTmtE%I!gR@spD-WyGkyf#o>XaZ> z<&mG}HfUdr7D2w<3>PI~q)`YFWf%|h`ZF>%9$@y|kioT+f5UB-;)|9os>HcJ>>FmNWH+sv@*UeFqSql8ZbldtozHzfvs`CPxv zS8lB{^(Nbk!nfghVr-(tHM+(CCC*PRJoC&ZD{{8H*2$+c1iZY8XBZ zc#D2wiH(X=nZ?I+lnEe?hkY_canZde6DP!(>WhZJV2dX~v5aPC^nst)Kc3!1Chb`% z4g9$cqY0Y*Oo@&4%>)3yRHVyD0z;O{>dS>*eN&;`0{lTv(XggM%_nA?n65)g01~MM z>2!1eGJ=wj&+uR~RXOf|y4yZl$tlpH;E1|^-gRY*v^~Z)mKsW1!I3Oo#Cc16#Z5MurT>0)e`eYdevf?m8C^$E!uGI3 zS2fk%c|RXmZw0Cq=+s|y4>T##FoA&)cMHL2!VYX~3=wjT$?F$=_Z!!bn_aCLKCIDc zkg;Q{EPOm#0?8GZia#ZaPF$c~s+*rux&oHX`vr6E4=rNIOLvnU33sw zj=!*o4)}6805>Dh7F2OC z%8LOVBzLu8Ucj@X_ zjUd(!*_v8Jb?rwQ92+B}V6@n~oZVbD3auo(9=cOlclYfVZEqG(nFr>18ZL*S1(n5;?Ab-v>uylM>Ez*2Y9YtsMxe2@RvOywl3y3HdC^#jxsgq(65$kM5#o4(mIYRfHF!@ubD;(qB$SsdSDUZ* z8}I%Ed8xbKP!FZXVRbwk3$uSRym34=$;cvG1m!}ZNWp5oDSV#Jg^^N_kxQ1vq_4{H zr1J}2&d;dWXs)T4_?od>;NmP>BRCq-57FMD8Wn+{Q9stN?3NHjX%jnctA`Rf(feu- zRu)NlTa*>7$qTos3gU8GlBq^~RzAX$Qcfn4@z`OkuD5j|Aj&tkqMAlofa%$H^SjH1 zu4df#gqQ0uFKyl(k`^&LDcftU$v+auSdwC)Ew+;c#L5~7o4`D-hFnq7K?~{)=~3^eZAe!KO6Z?XZsL#@*kwcFXzA83E~Y1@TTL#hRWA8g8}u-On0iaR`2(7 zyGQ96g?UW)0`4qqEvu7Bg1m*;{@~slFNnZDt!jIWQ-=|8Tz$th9A6dV3l-E?i9`A= zrP>>`EG488OD%S-GEBc04+ZKw9~=sYTYNO_;}cOd_v;SGO1eC%4}N)rc;tsGnFqth zXI?vzk6kencGWq%WNibsQ6{a4vy{QQyRp~P;lM7sDF&y$ubbj&$AtmAh7+N=@v_m{ zl-N_r_l3{_n?r%`@c`%fEf8cR?by`@=<93R-{1f+tzU8m^FN8ZVxjs|Ht_ifa=ADvd(;uJ+xw%nwGy2RM3v0x;I! zbYGyQ2x|DDWK%h87y5!$6y!R>wiC2|-;U%%bNMT z1S`;vyl~9{>c};r?^RyeFa5v%TK7MS_L8yn$ddLsRf6}zON>PB@&sK7(Ti{7v&6*m z)bo$wN!WaZk&x?S5-TGVk5`#bPevyvH?DZ@V;k=!Z|B;u{YOBtyl{Rh?hVJn)_EQY z^CTX6Sy??y^6BB1hELE_Axz9AjmVL1Y>UH@QL2P}nFz1Q`nHS{2jtI3N~gXBgtOxs zGys1rfEhy7l}Z`SOyvyj%5DUr(+~&e6(8%noAt zQ+#;HWFUOyY58<1jj)ctyhJjTGsob1c${Q#*Bt7Gf$cz&x-QAwwXKKkN^U2bNz}J8 z{;HvFuY2ERj=pO>b9k1RpX9xAe|$sp_WNLAr1d{Q<*13~1-yo$%a7z2>3kHt$p6#~ zRZ(EBRhl)xKKbGJ3(lO(?)Ef&^W)kDAWFYF{@wo80|a#!|LTu41=fAd$aD{)b7mYl z#eLAKL4ps}_505+NjTk;-3%5mdf=tCwU;J|@-03O$jvPHAo15w9^~_MjBhA4+;|<>N4Fi1N+viZfLgB6 z>%Zde#2=5IPf998NOyfv;p@4$#uXED$^+`5n0Y;ZkO}~#^eHd9w&9A@w=%Inzb|)a zzc+9bu%yqGdAO>VB07$f#9QnLrjX;Syzmgy$`8)@Nv1TM;8a0L($Bj01)wq|+rJUW zGh$9vRT^Jxno>Gh%5eNJEFvo-!e(-dCc$FFtc~gd0g^7A&M`OhRT=NQw8gPyc)U() ziJUIgwxP`P(HrBy;j?8oAUf=DKiJk9h6tkIQsZhA4XmMx6C+3>i7PDmp+PHzR`dh2 z;DzJb4n@KzUYO(Sl{d?KbnPt|Z=^Ik>afnDE2zWNMN)x+kY>tSHq`b(8)5H|(n<*E z?S9X3toA}BeiE-NWJ?hSt2=evNnYHX?O`x8X9uBMVU=aq<0FG z%fC)9JIN&W(B2>KrYg4{E&|`?3iRmqnt7VjB@z^E(^sswfa5%KyLJB z=;gZx6)qMDP{x$-r)3Z%=7jvwT&7hN{-^DIsg!o#FnP#+9-hJW=OTKmIcRKPNPu%C zf?i6CgBtk^3fn2UN8#lkt9U8TkCSgwZf&y&2%DfamS@6=f5(a<)tu^}hNYdeveGD~ z9}E#iKESPknaOwc-i~rI|7HV+{2F-VXyHP>{_J36h5Q~P^X3FHDz2W~ zm|8@6AjOzOXoYD>wgcT-AEW5tF7I{PGwVDjajB(ch0C`R1-HVbH;W|uEsYmYSt1{$ zG6B=v>BjT810o({Z@HjSiE7(lu$?v4Pky|PH`Z)WzEZe3r-f-BQ1H&RTk{rm$?&sMN9O7s9`jS%n$2qPwu6EVJ_eb^t`Q4S&niHdZ+wMTU|k55+4N)S_sS`rC&xQrL%jbY3# zg%z=^W6o^D8xnsCx%(SG4lJyxa(B7Jsr?|>VGeWV=I`PstTGz$Zuy`&HVqJ1##_`5 zCrJ?&jI`>eBYj*jrmPU&`RRFsXBf(TpI2Il+fZ*+$M>EFTU%zn?8rnhWU_a#^q%4} zqDsUvpoAnXCr6x3d~))u-KtRGN{IU!;QixWd7bDktw^gariD5Td^$H*QnOUzyBe1@ zqRu9`i+TyAF{L;tbA8!#tGnmycecOYLNuC<@d1B@d1@jNU)qAwB!cG`(nO2q-{Z5H zV1rXD4Gx%8OBUN&5;Pi1OhQh2RAPdg-g>j8V66Kn=~@*Zvv08j;K8)}Qf1yhu$8o5 zyG{pTy?Z(qFwnl(@{Ijv)tvFB%2N_BwuaKa_;f8UQrv@>ELb7wRy+76b>4=B7|Vc%9j1bB*33{GJ@Zs zNUPk&Y4?YE4tOHRbaKd0bkB-^HA|9g83xTijijiXBPTc6Kv}Aci7R93sR$HLLQ1Bz{erN}MHLTI9%$`C-(8o;B@A3E z!jHjl&+4i0tmT;-A4nr&I#tYF;x-%%6~-W;&ka{KR?4 zlZ4xOG`EIa!W2tWTrnBOnu6BVtD}U`;mvr=q|~49s{>F=YIVnAYFCFSq1uh*KY6}(&vz}Fahk`OV`|q#aY^fL$*S2y zHyr)0v?eZMbdK1+*!cQ3pXJF{9ZBQzjCVE~APwRarfC#_un*P4E$@-B-uj~(>;Vs2 zlXP%(03?{2;+5DDNiI(`>cwoqi^Bu`%i?0$F;1>)tmVYEuM=L$(iURxQo%=jL3lb# z5+K7On!=I3@??1U9&di>;{{bIYi-IhrG?kW6pcFf zZT7W8EHf315Y}5a7HW;imowPG5>f&~)hH|{&ITiAR`c&q@=c^(Wz+N^(r*;;0`eKv??)yJcmpzapy~P zVV~$<389d|kl3AA&&!SVv=(9n6uE2WutuK%8x&_9-*3C*KA~8^8Sa$W zXolxI$7zUHO7|UKpP46&C{ix_ee3+v3K9x!i^3 zytGGu+@JK$NplmN)chWBDKwRs6NkmK!v+21Q zaQs;$#4+^AFn|5CwCx9k_b>49>1nyqM4DIWPPGMWZ&9xEoXN}y2fAC&d|huqrr@f% z+XQ6ZRW?@n@L^JYC4NVkb}Z&dhID6&TX1mjx%8KVM+{LuGIw2=BV=zalPo-sa$S6-}Y5LDtFxlAdPW{7lrV@3m>xts63F)_*#`vn-8l#f|9WCUYUFWC_|YU z@(W$3XgX5t&kW;@2k>N;Dk3le5#dhc`j5DV*4kt=A+J1}30kR?tv$Xv_<-z~-iv#- z<)7{BKpo>;3GKDR+_o{#uiHSA&8~A}lQ2 zi3dB88m`40d<uRBwTm;2-ERhnyp zUvK=MWsffs0|FKCe=E;ZL|}p<|MPBP;g8#3AG!bSZp4^RZv7u$efU4i9$(bg-#;Hf zCQpYq8dQ0r8%Yx-8%;KbbKJ;y*#2z*AARpw#$D+QK7Pv&D53bYfA`ohoA2KzhvWls z3!y{(EznaeB97S9 zXA{TidS1uV#fb-V?i;Dj(Zsv8qp3>Er#fuxtg>&-oSiEsLzZdOe^|Q{IG8IHAM18i zly43_psrQbj%Py979DHM=0geQzEu3@H*Qn{i%W|j&!LW=_mQ6pMVw2ul#s8Mynb5P z2HH)uixXUbcps_hx*X_`TDsh!JX6=?*xob7d6J)H{PV%jQm&w7;1=iWe#uPfaW&^X z)pEyu&Brg4-d9HyWGdUc!EE(MG>_jJsjBQ=ccDE~u0>Ic4fqL3Ni0F$m#5B6Y=ng| zZo?-?$L~H8i2cWa#qb@(zXLLP+)0Dq9N$#;o(tUaJKBCoS^c7M$nuhcNF0Eq6N-e2 z)57uR)y(v|)-Ht2g$g-x#%TW*ShOJ2c;bOCW_eqp7bm!KB>Br>wJ%n!)cA+YouNoA&szER)r=LQIF2)u7gUrTZ^x{ zBo_0YVr$#mu^|+olEDfgd+X9FNeP^W0;QcxR+1glo9cV_4UEzuyc9g zr|IUfM78}rqRu<^IZah9?-kPOq7$;QPB_9QwC@$~K|KA)fW}cLs|m%h7d8oV%>@ND z`=gu-MHq_J1s)Zw4~-T#CgyQ z#8TD}lx@@$wEc!fz=2;Blr^Y;T)-m5ah=c*i9M$#!fku^Hy0ktK4r7|N z*STJdQ&GXJ78C!V7+XP$Wmi1_oo0^hmfy*3SFJ@ZPM0T#7hN%-69iA$Bl5$rQZDW_ z?5K48&#yI|1&9y%T4uvhC0qUBaxva05nMo8p5|gxA(LuYU;1G?^zH^WQp+nu4<@F} z?$PovR+?zVlTjgl3`Gj2RMphQpV(_UsS8zjdjo@U><8VI>x#RYdHw;a3m(}*skQ^9 zM2To!;RUn_ihF`^bXyJ+0g?Y3)YVC6HUQNEqN!)0Q1>Ckf8HiiM$RPr%XBsR$!ORooB;@n@ZLZF!6QIVISOJve#4_`ow5q9NBn{;`y4KrZ(?H>am#sP;qvCiU z(t0nLp^V<1{?OoF%OmlAlZyz=;o;;G+2CA%BWPH!&aovo$xuBgminiDD08V~93mX7qA z#>T;xb8*(qsZ8)J+soDyi$9&n?{?$)==%?rBhvrg3-NK{paUbGH1xlY(cEcaJ$G#s(m;bNPF_(L|dWqcd+BXn6^ z57IU9?@mG{_J4sZLZK44w8`!^KHD^Y;uSBBnYCeZO=f}c2QQB`&lHi*S`&S!MphRl z!I5zr+lbQ^e}R@!{4zJwVPW>p2E8!SF^B4D5P!i^o5tBJ1cfz_*0_EmPF|da*f>zE zT@)ik+gmKcOdj9q4PE-+tS!K2Y{^^`^5_;xw2hvG1WjKYPpn|GvD@3AH4r83wonPi zud&gx{r9HFfIvxj6D0p1n5oMDz)T4ar;HINIvM=7l;=;uHXE=rZ|r4zGG1d&M7*NzZ#QT z^6Vg?s4&f%aHX@9DId02YvctFgYU+mpUzkFOs*L!oF>R0K&z-;XUUOw9?NmTi&MpQ z2}E*)B5>OQG|AMlT(8HO;ZVg?b%^sFsjjFHs%A*NK{iQ-$l@8Jb^(TWFvWz$7TZ|T zwuF{cnUlft!&C!(=?4J?UdViK`L=dWG=V~huj;YSbUrlImDdbT)aL&5D|(l6jH)_` z$Qq-)7p|4;uOQ2f*5y{IeX|ZgxVgI2dn63@Y^OE$FL9Er4Nd;Z`xdGeolZKQGbmpj zRIEhrv1iAig)cU1yNk-z2}go}ClHcsNs*#V2TKjYp<2qU3!@7s4(Y)Lf+8mUNLs^% zIZg+BLyA!ba1JOs=h+TD?K7MWNP3LHUQDMWflMRh{ipX4L=nP}qaO*Sz@4PZt<`5| zK%#2DvJuaQQi|@6kM)8=XvOcSrmak&@91zze#rmWIb+C7YbxrpSBEmgmd|-tGnPFS z=0Fur=Mha-M-Qy0Dw}kpIqu~zQt&_j#RHa@-!X|}6ZR(~A_lkp zs0CAb*QO`~MtU!GIF{!l!q$lGn&HgLX}SD3CVMq+N8xQ#+@Twk4lHR;f3elNtjyWl4 z@~;1@FKKg>{&F%dRwlmXVjXns20^=IPl^F#p%{X*VJj#3Cb;A^d}k zave1E11p@jbNXX~3^@Z&1fa5xv`sWYtzkawON$9ISPfTWWkYkijcV_5e)P`>Ytf`>}Xm}1+zf~DM+wnY=A3z8NZ8tj1U zr~;&}e}Sg=StD%sjRR zWT)8>z3tB|%jcV|kVX+u7#84=5Y%W4&lZb84lSx42054!PI#)!|rT!O156il`W>t6B z=As>yIRi2zJliXRvKc-0>?2QpGHze1a$dosyzi@Oj)aUr(bqu;W3Q*D*Hf*&!M;(q zN(X>$jBleCD(LegAvew=5E45~u6aHQ9bPOWbujo@)|B3p2(MA3jt`ncm1iTxu>hv) zBH72@HiUeL?nlozduZTs8tSK&JydN@t9@Z5WUq(km%n;a<9@LKg;pY7FYOtckIx~8 zqaiP_$}{7tGfCucr79YGI++|jxFsFwPwPm|8%ajQ`deO#{aa}&5(&_e*zxN!{iOmN z>kt3#u_O-f*&qrlO;dVjWcxG})cpt8{}2dW0m+DzH&KN7qR1X+NKq&!`>t+ZP?2L& zKw?8l4y@>4z;t-DiHESj0F{m~i3$4=Dt0`<@nlChH8K!~ygcNa((I9>4jM7EcsxF? z2Q>ApH)-qdFW>UiCW(q_Xniy-Wp4}86_WjH76sbz)Hb+02<@4#pC2$B99%fpeQFVS z&~&>WdJl%aG4-~isZt^EBWXCj`p2R#l~81jZY%y&?HkR#2m}Kqkpz0eo(3!Op4KK~ zR0aB`@hBPxT`n>`BeU4!hNXuw?GPWMqJ+UFM%8Fm&i_yfi zYCi($R6Fn48y*};f39)?7Rk@8LocO+ntZON;OQ!U^*S+}Yj~2gUX<1RvGV=&+jiH^ z@Jea$YPORsZL%y)b*f}ZehLo@l9Wpl5&l@}?D3L!F9^M=SB%KSjN#OT489Q;WhBpA z;v^yEqeAf0Ep6yOi87$&qx(4B@t`h@vx=f^32Al^zNAR?ro?p@OGcBG-gFQz6=Mw7 zzGPOoXm+n*xaX&%q@&ksNqPK)p&%r)vkkTLLuo_)c6OIAuvR| zVKyjGpcu;it;%$y)LT8FldW1CsN!EM+*;#5S9m{v@i1TL|HTbR4i)cE-}}P?{*-nw zR*H~7jO`D4UW%UmePntBEUl5&<;F!0fY^DQELczj&%2}xC7bMao7Sl!H~jcb+K(GPjKfv!+)HT3qZ+tP6LUlF!vmT+eF8VxA#VC z)q&z;k47V#1N97_CVs9h2@M2tl}6fkm6j$G5o)nOM&qcB7tp=agFuq}!Ck#Q5kX9y z={U7>aSJgKm^{NsuqiLvqlUHNx&r^N2lT{bf-k@}O=lTH{2Pg^jD^7?rX$ABd){wV zX6WSGyjWzr`Qz>b6yjhCzKis`zd@ElKJ%ItGF`QD-iBV!o2Z9;7t9r@FA%~GGxp`< z!<(xH{Ia>K$U&Vo-nkL*)WSuq12qXI4c{)m#;IYjI*$G>VM)k!F72!BMc? zYvDfWzjvt{e4(c}ZubW%gy++zs$r%kH)1{xa;dWuVzq@5BKGF5Xd!$L(so(CLL`*l z37QMvvW^6~COxL@q~jrfqB%{di6QObX5NcMRy_w*vYyt)gz4p$F8Ws~;X#Ikx7uob zCAOZDoCMv4>X<+CM)}ppt7TdesnqRHmWk7CDGhQ(Uk~Kovg?UWr_p74>b2)h8};~& z9~ST;!ZkaiV5u)4mg8>tGrYbB6j)mN>xt<=`$kizW0}#-mnf3GRy8Kjzu71>w*LKC zwY4UcKq+=&5UN{MR%MA~8@q=eJqw!>#fobZHYbb2hq-V0-7%iIJTol9{rlMK?sPVUaeyxXHUnEN}}tn;Nv;_4?D9 z1pCnq6I~4Br{@>MT<)#!)3ioQzd9#x%NP<^lM_NkMaB2;ozr(hy>(HSe|(arD^=k2 z);?F-E{n-4S%3F+i7XFvV-=9=vN<7Bsdf8m*EqlHhjj?o{CGebh=f9;284G!>d564 z*N8HQf)2ou>%xdo)ZV_=h$Dhf1LE7KIMVaZ7~Ky$oG;d04<|Q1)EMp?OA}8DEFcyrR5vt>p4At&6n=iL zuv3O2c{%zq7HTpTs5D4~S}wduq*M%9hBjw{bTvlDaQj*+t*n7mj zBxfU!reCg+UWl!E;f2Zu(_UZV<3GxQH8?4qQK=~6Xu?R%SO>lx6u|sikn4)N=cMfO z=1ywX{#>a@SmkLVd+@s+r7@DX2RFJ(M%e=&5*VP=!^r-}5aFwx& z^=c=yf}6x$iH-T%CAz$}X0TO{ZNRE68y*&^E7Qz1SE#43PI}({fghIk59{F5I)G5= zFWpxb?1m)1G3*0z^$QAXMrZd3QJ5)mR{*3C+bxP5#O^S=Dr*3n$zme#*&2If75@8p ztR*~&jYGO`7t|P|YSzG<(%OAKLYh}+8PHqTL~1VTWD);j>|=!?0r`hO zORKmS&*oFz>x3(TgzYzg;iQ2&;1He-@*6ciu5N>@E!f9uWNjg3HmQ*6g4Ek<_7z<* zhYJc3j^uaUjT&@V6|W)bEd=revH_E02xI#sr?*uX>YrIU6F{Fs?bAv|(e8XwOXGV< zl0YCG^|_lzE8!(+*a6~|6RG=TH^K3; zGn>mFy2W?Z5oQ}AfdY7F(@&5@aNnf_G9^D+=45~@MHmsfjKJ~MfeQNd^Mh>XzY_D+ zf&u*h(rfN8VKt)|bW@{`M|#Q{fNNUdu2qva10fG8lYt|kp$(3;xnDnYMdRNHF$G>RhR9Z9tYh6f@)6p{6UwQOu(&&Gd zK&SI{X6Ar;8|_7BaDUhDHK1oK0W|3?jFc^8(;W0);Tb`!-(ow*T9aJ>y^21X)OPVG zqBZT7Iq=T)GnzFPi&lV1+Ef4h6s!nLf~xsl>ERBh;yym!*kCqi!fd~Yvn%RZf3J6dse!*nhKqe$GD_4veueD8Ni*{3cf#|Rdlmm7Ow&2?2y z1C!Dhalg{<>=SD}J+Su2id<|0ug0@QJ&$J|ZT*KC7NAK)dTBYz z6b2X|?+#Ve6p(1h$%8Fgp!6WgtKL?a4ME()!u3XaAL755GHWS>kr46x8g)gmoUlgA z9mkF`KA|+>_+FCo(E19_hg_}qed>YQWLoC8KDu>1rZR3iUN<_-V)$~X1JA{aa++BnKWk+czXUq*)< z`GM$sDWd73l#P~Pp>#Cblr0oO^~UT5tNL9C@_KDJo#g}NeKE%4_vdN}wU<`f_Pc~0 z0+z(lY1S2dmgkeEh=9N! zLnuNViw{r&O4^>t9+%Jp2kjKmYIlm6mS?l|o@J&vNWak!spo`|EmQu!+hF{6FpZ%P{z#?fB^U zI5&tD+x=G_<$pe1{eLk0{(n*DLzZ~Ua!7H53o|>cQO6w4xZA|RlG;!|9N~J#8B7!8 zefA7nWlCR-;JF&>cX#C09qLYGSE?OyJ-u~~+RbZlpjt|c(32Rn=H(4pT_~~TP%)r@XI*S_9v@&kY;Aq=TaXNI6-E2{O_}n~fk&b=y?G zhY!cKoCAF4CiNJ`l1|z(ZD{bSt9OFSaJeWkO*+6%oBU8sYUd0#fQ%bQo)MdH{4VDg zmy@tZFPg{U82vc5FQn1};g(fuiy{^A3(uQn?ld>?qww(}s0_@_({_p;rim_$jn|~PrJpix9GG}U*9s{wPeB|s zUir)p*B(E&kly>#TKrbGO|8SU8^9be+tZ;50D$aZ^Z!)^D_937p$$g1cT*_x@Kb`s zCpp2w7Jbal$>VWaz?ycP#-bI=m!j`zwM$$pDAO}36?An944^$XHM9;Bswjj>uFv|t zYU4%nXsJ6>#}eSr|7Tq`K$T9bGPXUC!Y8V0Wi zTMyhaL}^@wJI!xR8*BL=)S8;V>Yq5?(}F({k(yuzH3prUKNvF_MLq%uZr+hCcs}p) z;;D}AoFaL=y0Gy^sl3~pev)dzpxT0Iqq1G^CHYby>kLM9DmL#wg7m>c0LW zWIvdT8S8bgYK3dIefLr7r)OqSmYjzt8MCybOqF&Qa-TQvGa%vCatAJ=(5c}1?Zw%m zeb$*;ZQgIz+u+%8%9$~H9s7KEGsdZ7ecAK9z~S4@`{XCVkQ%owY7$cnm-qc1R3|*s z(&G2VapTFzZw?@KrWZaufNC_-cwi=^q$cq`JKZoR_bMJuU6uU2 z-_zBdhYW~9R8fh&LvTFCnXSXO=$die0yQX-kGov!?ND~%G#QLX5V25I5Z#<0g}U&2 z&Ir>mf3mt`c_RXZrD(CFK2r1&;u3W7CJZxzYz>vX(YR>=HSL4kZua;d=zqAku^KP< znA*x29^t2qd$Sq#*4v^J4@w=H-fiA37P%w9!wk{25%t=qm3xe);OCj7bhJEQTkCYm z4LT}pGF!ohda-EZnIfjEhdy;4mJI785y_g|&CcmhdsS z)>7{c#*pnjDToH$6b;+K)6Q>^ybL%L#agT9jcF$LYsrXfCW%RFW~(kwn$Oq1A|+67 zU2yXOaF8(k zijU7FzRnl}F_7-NKmM68^{~~D?6^&2W{~viH(G!&s9qJ2emR%E>RQ*~4pV7m{L%X6 z(m*K&hiMl!#Q^E%4*Txzf~Zmm+;I?|Umg$|?4GpT=)kb$?Tp+6YhX)1;lj1ObjXzb z3Z`Fnr{A3(6VJ>>O8A5nj-_b)E$aMw>mgG7Q0&t*lM8Lw%RSe{%C( z?ZyB-w3sh~-F%F_&cn)}KJ){N=4ch}K-$*G<^Q4V9it=dgRsw;a3-ACnAkQawr$(C zCbn&BVyk1@={V`wHa5?TeRt2y;-V0KA&{2eUpK`x#xGF zH8vN4ohH4b&+o2|@;R6eCuMfHd<54!%a*kny?ZMa;RA>sVH8s)N!_g@ZH z51sw^w%zO{cG9So&7`n*DqBef4#dS97@}i6>c03E3HVGgg~uyj^G4;i-RSoc^aI!| z#c|!YGoFjjb^V)Rj>KB$G6<0kT!KsYVscmmpk&N30HXlz-su{{0S=+arfoderk-2P z9INE(#mAhlU6(VZm_;GMH%@sWJCoM-Bl$yAv5+~eP6ZvveFppUd=x_7Ury*owuePr zi&-Z&KEIMyo&(dqe?(qdTDT`}eI_b>|ALt3%~|wu1l*a+Gl&CQcxR_2vzYyxo6hdt zh_rmC-x5bJgrHs-y0nbW&d)8@4Jr8bWPD;3L-{kwE=cS`Lr^`991{v6Kp)@P0X9ee zRb9k*36r($ZXkRhHA}vNfzh8p4K#8p4Ic8NRjmX0Bg1fSQ^A}*Hg$D%l6P}ftV`~^ zUyx{78qKTeTy2{?&PE(9*MyQAUqjgF{syPbVZAn`ooHo>i>n($9G5guf}f`ni%OMT5J5FdPK5 zQHtezF{3vp&goq~$Koi&YYwKVqtHzh*it2X7!!H(KYVqy=M$5Hg0GY{aDs(VJC2nS zLYex5wS5YN zLget-hcvND{&RoyV~y^h_%zKmsj04~K;0(A}j;OnnPmI&m>QeUO@tp4WfJt=~Lejr6};+mZuB7>m9B zO-b=xO&axmq~$HDk7`V4riM!~GI-W_{C!ShT&s`uC;{n$2!XyiuzG^CExOL9yI4l( z>vUp{r33-lVE)rm6!}HPIqYI(TsO&9Lceb1vt=2)4>9n^JVZd6b5LU0)@WBM9(P{+#nj^Q z?fq~g+f7V*cI?{do{k*n`qP1FHy5nfrp+DingQPJtA!61x5njI@D6Elw+F?e;>A{y zi3GQ@t)I(n5)G5plNLf7NST~GO@4-O>oJh_R<984ers8xteH&uQ=EMgp32_a^w4R` zrC*DBxD-1ixc0sGrynGE6_cFC?^DC;{BFy$ps72^b-aqyF0sy(BGPiwMd!yTQ<-K( z2znO*zZ1Q6jVp9@W1YG*mF3!s7P3j5DYi0g6eL4zSo>W+%JKRk8bG1N5a>?(!-G>S zHOdKskWyDNRWVVY9$e>24tE$#p01%wxtn=ocP(pJje1T;ysDc57(U;YPY}xW zT3Skbd+_Y4tdir=oo<6}Lu7sf%K3wKX#aV5Ke5CjpX}OvH0G%v{#c{JuPg6a;K=H0VpKD-Z zuM5j=#|KhO0!BH`Iu=Tu*ypA9g@OxMvbxf|#d7~8ZvEWz6u;?bJ5zC9QErg2vM)FH z4mTCHwz{uD@(rvPsV-Omx({EuDS}<)KQFQERP?aW`iE^=B7gEy)w<)8T!>ut;N=kNP6#4!ggV0=-1Uu(bVje=)Q=}yu)k`J zzDGVlbY36V`~b8@lj2{rc62bCxz0WCp9-7v0Nmj-ru*%N>W=Rqy&q&sj_~A4s@&iG zlQUW=?3$kzO_$oM&Te&sKn;@+*ccAl zIxcazbdkXJS3<3j6?=mdCfR{rHN4FD%PBsDO&P9{uU)zaBE@72?5f`#o~k`H$hAL#q#tf|btF^dk8WV@*Cx@RafT zzt=3OE5%xLzmL5wdAI{?s6Y?qNA)+~4(ag?Eue`X;IhZ##UX?Q`k+TT^heFNMg@$%4|%F(^Pt47@JX-juUpq{%6zho<W!L>v0#K>oS21wi$a~!+!=BNuAe6(b4omcxRqCsW^V*X6DtMam$4#+CQ z;B1qef*0Sx%MB`ayJdGDs-xstS~#1jBBo%M>i56wnG`OtHJs2tq5r=nPDh_`h&e$uE`Ce+^Qpz*ws;`z$5PpzWH*DsyokL< zSJU_B`#T!zxC`~ujj2aQ?HpDJIX|ruTfV{at!AdCO=fhsjQ+OPM!N-B%+_pr&-a1Z z`BKjM5^~MUYw49df}))`s&kfgDL!*T03=9J zp5j-z?#K7~?u;)-^p5_Ys5lSkj68V(9_l&!!W>EoN{#4;0kSKarh_d`rg zz@s02+|1X<5wm70dyIOZ?S7yB59|0Db#OOY`gEFgNWL3IHT6LMSIi$NMj+<`f z;!nyELv4*qO}9+ThbEwal?*dR*fvroJb|OzQ%Jr(NvZ8VmZh5rkmidWbk`V&PE3*l zjP!p58lZ7CJ!pW)%y~?dLhmUE1SU|WRB}cDE90S!uI`(uc^E1POWkglVmS_-mcJc0o%RygT;fq#^5lj9nXYl893Dw&> zxAtjSLXqFKiZfcV4VF$QWSghw=e*V$@Y#!_4R2MRJQqoowSy_oNP}gJ_QB=}Pv~{C zwY8O@Q_WCDO&-JfCb0p`DNRa+NJ@ za%bXk40I}4U1*ds^#_+pX=c+FikhzwX^(wf!o!@Y|1sq6;7k6( zM$_|NmCpHd@D~9lABC*Dwqw*-${0sn?@ zSucUum5*Y2Gv0Ut_zb#@#N(=}1o?C4h6ccn*XU%g9~1tdILn^q1|z`N!tXdyBhkWo zjWrNG*fFm!sUR9hp5_Bxjev?`7J&OX>2KM!CNEq?5aoQN$T|2=Y=6}EfHu7Nz4c&E zir6}22eOR6j4`6y-x{uh~bMMqqTPvU9XlYntf1YzDbld-G2#wo+lADQENqaax{J&cjQBY>*4E7zG$mb*#ha`FPuee!OI6`*UE#--{bkJcoX=suS(w0-lx!mQ;uXGT79dTJ z*t@8J=9Fn)Fosjl=MX)O`djPGc@LA-bApGD=+&oA3Gk{7A$Am|Vm8 z3Rp9(uW0Rrrf4#PU4%hSF2*xuWPLbp`IbxGX7VFN=QdV{y>eDi(eJx|@FjIESD zOa>aQfZYm0d_GCtDcMGff3X@lqE%&SqMR;Vdd;3egQ9e9WyqlX~T*4#`LGY-pDltKD6)Z_p#z~J{RNFngJ$%|AmYZyO7 zvrv>?meB&RHA;Wpr0nw}7#tQt0lbsZui|DPo+!P?zKI(5#@vhr(|U4tug5TnH7ix>Zo) zbLVLsP8^dr*v3?PH?pObt%Oswnxh*1Fe_@(KwFhGMWX|@X1Y^Bi7J?6h3}TE%|dNn ziT)E=mzzaR;vHRkO*65y%$_=Lw7c(Y%>9$_?Ck7IdI*OP9~!M~ZHz9fv01)G^;t0t z7Z9C#!aYo$Wtd+M5QaZgShcf3jEOA_-mD@i*uX zDt!Ey*9$5q>dW)42cHvG2IA&@}%?9!NR zCRve|Cocna-ZxfB`7efQrhiGkdVo+gzz;P!1N6U1(~4S)$O z6QJlVkUh_Z?@^ibS5AKL{B?$|rkW_{u0`p{@|r(tnOjkFDN^l+T4))6RlsLvj*!Ym z03Z4I$&XENPZ1TybFe~^`E`s}pbCxMDnC{Zd+T)o5TYk`+BZ&(6pO4PvKqA4x7%%I@euU0IR(eItDsZu8c@FW(`XX_!z z(OjB3GgG6O3!smnKYeqo>H4)`tF4YWNDv!ePj@Qld!R#u_PBXR$AsNIo)y6f^eCj( z`@m8i?bZ4k5pW5X(YFte%XMq2u6C%A-LJ*RuN1{-#MF}|doyiRQt*5`) z$X~``@)a+z#c5+tGII_4mm#~ zx2G=TO4CZANm`#$1@fdF%3pJk{4L`JmGd8S)cGrTqlso|i(4gjx6DUg=|nY*^G!f` zGEO~65gveF5~1Y{X3f6s@MJzLni^P}i;mLS*w+GbTKCuZ{PZ;nG1>xZ;hIaUcl#D> ztD{`2$DtN54{Ic9Joh^_-&BF}s2zQe z(26CJUo4Z&9CJ$Rx@Vb9wz^s$&LnT9EiV?ka8RHvEuHt)dh8K*}eH%($Bw6 z4wg;);^|sdI=aWsw>c=$C0152N(l!(R-`S{g64nuTA)XtnUSK>rL?A!C0U=1u^i5& zpK>!0sl^uoKteCdk;C}|hwO4$T~>eStZx8YXl|lDf0Edu#HFlmut@PL2glHcjY z)yAy!;33a(S4OjWedW~VK#6$%fg`fXtKIf=34~A zA;ZE6+1hOl3DP&8Wxet1lWm8~f_m4o@oqU1{Rp>IJ=Z1zDGKxg$h;IKKJ&j-5VU z2y$#lt$Q`qTXD-!d+0BFL*kb$?({JlEiJGbe>8jEd06N2$qF1syajC&Xv|Lp5)3PU z4P#5aV&+R_b8j-KnTE1}BP0rw{V{zCj8if*{kWc329I^e-|pH><@sC_ z1PQ)mOlVa@Z&=y(+nhoU0aZ%I3!)n(DgLewP@NmQ<{PG&aLZgN1%Hid+XVyyERjDEhoqGk^&b_FtbeJ(96QN&LxPFyK! zt;mn`h5Q|iG}>Mc0#eh>_=xtlK+Rw>l(S~IXhD&}e|#K}m~Zcdb4+)NBunsv6nWhY z4QQmBg2aN5Waea|N5U9(8$F(wP}W$dE`!$C$cMy)^URyd3S${w4z;~CdelFUYHPk1 zXP|=-hK6HF3XcIDDz40gk0wU`%Qz^}pgbJ%lrL_8}J3i5BK!d|Lv|6@q*e{uYJZx;|jupvwL)sS$iZ^ao*HsDj(Y$18#QfZl(-K)l`+tPi?tnA9goQuA&Z% zUHt<1(pzYNA|JP86|5Ys=sEWzQTwC&1YCm<0j{!}t>$IV)2l zy#!xqgW&mv=WxB0AkD~VkyT0(olbgPpHj!d7r2zmx#^1iClYLg$mxaW3mM-FyyK>G zfs@-^YHH1oR>;!+PpL;23=f_U6#b~W-#lK=RM!(&1W4&=nCwd#1s?@O=Q>#waiY7o zy7iyKTdTuEMzb6UCsOPmAIq2c-7q8eq_bFUObGI&INiOM%Dg50TwI zm!Hk(m@XT+ziebyNpTmxDzmL0qsI9+@;o))fBOoI{>2l$%1%uKqK+--FeE#;tSX0Z zPgL{ba$SRU*x77Zp_jOncb0)&AZ47n39y5qEISJo?k-9tToeh1B3-K~nqiI_np!-6AJwB#0)?X z0b=!*ls55QFAxVa#Y$!^;b-Yu>fp#*7j0B#*xLAxiI`(u>IW`{i-%XfVS=RA@YPK56&gC9t)VRRLYhc4 zwi1Jc3^CAG{Uv1>7N#KI#i_;2#b%&`3y;ngOrb3PX=G#}9rej)Dzhqa-X2? z0Cl3uv(!iK)bK2G4%p^(5j)Ua)}OoOqbJ`g>7blus6w6c%pHO$<99($xV!!6H4Wk13MpKq6IT5q0*2!$DJEv3G%Kt70EAmTx^k;VA#$>n-(TU zjc~a%oOmD|{%*@RJnX8Ex*?21?Rjs^=^`wc7eE8Z{DoNCjSOKkCB6x0ilcjc(yD7h5+E4-m z#UesZy(cIB=a}4OTfU^cM9MSikI&Ja`23B&pu$wPFactO;1TR9AzsN zkT5due%Jk;4pWXVXDVHu;vWSuxxr&6d5@->c)2s@7JaQEoIsz%0%c#X#!GGvu~5HZ zkf-2_H>Fg^KwE41pn|`WDYGjed73hU%!+iKmMONDQ5zYyMFv8r%a!mijQ2mxyf-RT zGqy0|s$l}v8(8021&8gVy`pFKnKuEVWn~f&i31FkFnNm`JY4?+x?Lm4!i_1t4R~<>f}4N=i^lQzdWES8S(BQzWhHQIvx;xEODdLo+ zrbk9XmhaT@7RCWv>Bb+>HM;&c=NXjNz#XTwQwPES(_ke2aPMDQCRFa#_fl%Lz#Gn~ zI-gM!K|`Y-U&g!D`S|df%>AZ6ZO`1P#HdLWey&q16NTDTx;zL{g)#TBEE~KY@uh#B zgzOIfwS`DLh=0IEi<)TT4@bB9?HVr}^7l4YL;2b1d(Y{M}NjA|x^TShf#L` zLPC%qSg>dGmz#b=ZR)lXz!I|1{Mu5%xAxP`W}8%OX|_%>+aB^cTg`6#tb|e#(P|`j zVxP=!i-vIm9cHOtfzKmolQXa+b(mo=`zZYyCuK&gQ=PjSYW)qR;c@lZqKNl|XZ(^yXSIO_~v$0@AZRmd#QQ^dY zjLj*&ERaFLqA0nvfAeaKH~_AvH!@KXsGg7M%o=F}=OE8|UrZT5b_NGRn2%#*rb5VG z-An=XM%v3W_zHthsqe<-$SZEnQsfjkd$)@Ze>7NpEcnmpu-`9%k|mKcFM%M8G?$w+hX z9Tkj$5-t1{{`o-ZY{&F_@*e4y-BzSL&WA>0UyS1yig$=d*BXdv>#~dZ)|8XJOR^(N z<}}mM_d50sso;+_<{^63o_4h-hsrwhf3%zxOSI?%*VVHQ?OVn7_S0%+le^F)5Sx3)?XnN~2 z4Lsv>3G{ljyBWTQkGkXudk3D(08FQh+ZqYMz`+xJ@k5c33cE*_edk*CT8~68d&fEL zub~129$OtO=<$7CcU?M=!H3w$*!2y{cP}2&qcS=ZVIRSMhQsY&2W|oReYipcrFPV* zAJ_OV@8!f5yN=xUbfqtn+OsbJwN z>TN-Of(Dp);e7oW7DYH<2hTkG(HD$)54|t7nZ5)=t?~;1hz-C5o2DDOHN@5H^c-p^ zdu$aTw+iyT0a$XfWDoCthoih$_{&5^kTPXIJa2^)E1A@*3GLt?pkR^VQZ4;kUx>rzmT`l#(^ zs$H9f3PNZN5l5r?r_8O>p*3;YAES&9@CJb$J8zcFT1&(b7QEgpS(m55M#6`4F-8-| zpQ1j8BA)@8zQ-9K-_2D0XmawI(=XQ3+K^!M4ywgBfDO;pN#E1oV+4h`$t}&+dW%J0 zI!Z4W!w+qTCxY{(b6EYkl=>9XQHwN(HKZ@n2)P@2wX2-eimt;Hr0Mxn z0nDK@{9DsBbg7PcAeMIeQk@NR4I3DAlapDyH)Tsq1mlI3skF(hzU^z2RvrTcUOe8J zy-GpyWm#{hq0AM`nKZ3D33R&uhryUo6)kYJ2;YVZcev>p;vxh`AuoX$1!bv=lGhy{ zl-^%(dBQ{|D3Dl*xtL|fSYbQW(pZcxolY ze~MYK8b>Z0+EnMtlq@*+8r|`f&}09(&HK^lF{Zz@mKgujru%L?ILZu4YRGwrSXsbI zB}%iO7fY3%{1-w*Mm-+(&GI_k0svc3d#c8Q*9J`HqZex&DmB*rriOuNebtHQIXVLa z4Rg043gLj&Ugu;Y>Zh%6s91?K(ml$m;;`sXziO~)S$HSDp0%$D*V zd#LR(X+y2emEp`Z{yHGa349%VBSQ|a8?qEuC=#v85tvfShM(D7@XO6P!kFki)z`+R zcS|kgDj(?5Pdn&e?BA?={a=&W4DS|a7TQS_;jiUv{d25}|Ax)8!e3Os^do6A(du=o z@WKJc@d3shrwJ>@xNOFnVWgI0pgm*AY%!L*IjN4dGKKIB4GWYK&TVWxnVTx1!|Riy zTgF6ddgMWB^3O@{WMy*Ur$atS9RAvt)8EV<{2wnK#PNq?H(;CF)eN z$@WhSl5L|4SN}?3V)}`%D!e9|$x%JR+hr#;KfR?ytV!aN)nD#&%UF`Bktjvi z_cW$D*$j-LHr%YT&h==8s_Hq*Ui6m|=B2SZcZ8JVtV^Y8h0^@2O6cl_(yxJATv1`c z>(5qm&%+WHG4RT5S`5#__NEi~qJ_ozW%X!z+Sy3bIBz+H<{~ojLoqyh3yG^31f^P2 zW1vGu=P~D@W+y@x(P)u$Pq_D4(!qNCVlB=DuX{F0GOv>ff5YSDSFf|gd>x&N(97+- z*-vBw@=0^JETikJflEGl1Ig$QAx;iQzDXRqAtE`(jC&gyU_al0NML}(i^)>){1m@R zMnSelXZtsP&yT)XqZSlQtjKZ|#+97)oU#)fDin?H7|AUZD_Em)`r%P=|Lhx1&nnC zGs}yZowmOwZItakJ0bLvaEDV^X-6Rk245NKEDD3TQA*VJC}kI`nX#6uU(B2msS4R% zikHEx`3!R_k}ijXE(w@ztfedmG8CI<<=dB^vmxoX_;A?iHbX6BycOT*yf#1ntpJ(d z`FhIegvfP_C(brm9+-BhJF4%_bpoiR=1yXz=KVfMbY*q^#6qEjF;j{*@ikzo(36K4 z$I%}@A)WTOa)A%4nzV{RU7%oRE5*A}abQWRZ1+%#%FR zCz4J@!LH8d=lW#$St-<^^iz|)^V_+Zos_tX5r_glMCy>>?$2M;+L~rv&W-~XVWl$;nGtqYb;?|_wk!EC% z-6DI-D;Qg0Nw}ru&GCacA-PT>yGzhTCa&)G-lD2?b#mtb_(yv)K=Q$U z4eQHs@Un?Q7lhl7oH(rlH*1b;_ZOn(Qs}ViUcIiEx4y6WSFZ18iU@{z*l)5k^o}9A z?HC`m42%>PivI;Y5cp;zeavKHF~=Iswj4!$xK+v0gnbBc+>3G-i@0#B^e z*~>^I;6Iq7tu4{-$rcLQy^9T$LdBr&!AFdxc@k}t}hVFJ0Y<1|3Cza z2xC7Ff;yOOu(|#>pxF%a+zk%-Vz1x}y=G=+@jEQxkGRa?Q;9cy8PTRq*yuWh!+Y0& zEDfSV7@`WxD-0#f(PuVwl}r9PMy`&$s(RJYIu(Xr$ihh}Fv#g!7M=?KqSfN*#IEA5 zh{C>d%`(0x$#sL!G|kYyUvzCt(;z^=Dny!ptZ*+Pu?zbjZxdWzm51^{|MG@`b4U!E;i@d(h}xrD<3kW>}^RJzdeg@f&qE2Am( zb4)q&@&Be$+qfpj9K4r799{Jru6BtB4jP4n)9djZzReD(yLP;?KS3jix2<}1BtGLN zhi7FkESdH(slPLiRRC5c360NQvP1L5Ttn&aGiIski)+Wxx8H=%v1HGO@Yk1~L5oD^ zdls6PvtKB}I2C1#+80J^zC0eiv8?w2Vw|K1sY&eb@6!9_X3o-t<8|c;Q8obSU??kh zl4RF2;B;L+9DE$5$K=V|<$LKfj{9{LLh{*h{h4e9pVwB8rn81Cbg=l8thMAhLiwxw z#&|?GvSCquvys7;4Va?>Y=2^$0L9u zNcX?J0GgZ;4xtV6UZY5K+B`!g;U=woE)Wehox;DP{`1{CcjKa{o*8dFq;pW7+T*o- zp1V!DAPoRUL_QQfW-XZ<3|OX;Wp*ahoA$rGvc-T-#rPYH*M#6|3yPQV5pX%;o*L{f zwKdf>uVf~7|G**D)p;+jM(T`iHW(TCQGNUMKjDPUcte~_aX;@`kp0216ogSL=6(34+lT5a-bi z$n?QZOZ+jbn<@Vj`{G-BDqrh;?wHM~FU_Wpzs$Iofu}J~q3TKK1tS2~{$q#6qusN1 zzvuhXl%>c7z(HF5`La~Z8KPm{g$BA{jnQiSyX(iEbilY07WbOkCscPahuj_MW$hwv zmwiBCCA7Y$BTr#!vCe~;7x`}I)%W7zw}~ksE&Iz1*xa9VG+os~zoUFWVzr!|ocN)# z4j*@_C5x~oCN;;n;nJT4oI{rw>j=C;*=IIbix#Repbp$&tmOXy1In24cjy+THdFi? zm1<$#FV`8C*zXTps_rKDY%1Ah-+cxs7E5@|CQ58#r6^D%rU_9;dDBORUBuqV!r`E{ zmQXC6t@fDGw|RD149h0$S&s6E1JQ z;-@BKpiF>8?x#wcA-g5XqR=vfSAYLrRRFb<*=}MX9^D2&vcVP}ZhYqS6-GO|;fA0_ z>_2$!D};8Irn<2+6}S-NQUT{%`zc&Eqh-a}L>wQQAIz8Xw^x=(5=c9xwxk)u8N_mf(7nO)~_qFbB!<;G2UV z>*_LX0Q!fb#>lAw@~4z=T!UM_&hZF=B=1+O-) z3FYNV`gPcRfg$TK#H^j?et~PqHIk5pB`Iv_{mIa6Daq-j&Z<>4=ic2(p=7ZqVr>J3 zojkMqHC!tH+LN@FN&0Swh$Y@Kq{Mwlu z-oZjTuTanX`_0$e32x__wEByP#ERBU;QD~*LA)20a$D>t_U3c1G}2l&8O}%4o8dvt{d#$Z zY=;j3GTL$%3)aY`jg1zyUr0y_M9L5hZE|&{55I}{G)F>MosKl#*h}ED#wB_J6^TYX0)p60-6)RrhR(rWg?%8GLzB3MQdklTU(vD#m1l=Ej;-xv^yRuz+GY{Q5O2ayasoo((Opm;S+Y+8X$buroAJm6S3L6I=9!qBXrEivAe>oe%47!L% zZ`TinZPrIl(s{3RSd$F4bhVm6x1r3wUO!(ZVjTw`u9ynwj2o$Qw^JHDkDw~WsJnZT z6rw356*8iHQ*Bio-<4H%@QRy84@ZA$;((_KQTmR&JX2t57O0knZF!VnE|5)lh7OBi~dy?~#-AwP-n z9iky4mr8XK>(y(RjYcir66PPWUb?!qO$nECA?Uq91}E(}TSTRQ8u;!iuW|HZ zVs^oX)42`N+g1@k!BmeXUbehE708Ob;af}iGXww4YY&%iJ0%q;mZhKP=)v)poPTH& zU3asm#j)MZ(Spo`H*mgHCu^~jj>dKgKWa!o78<^C#zMTR7SD$M)SoQnZ&i&XS$lRl z!EMeko*Lw~t1u@oNO2^|qn#E@BF!x&*k463zPlRsAVfQJUHhmAW;Ktv)gz1lytyKP-o9dhvrb@>S;<;avH zLaTja`RDWJr^(9`9qKxJFduts)cr!jW11&JTB)8hsQ~`JM}|%AZlf2~Wqt0C0Y!kW zZ`K|J_ciCh0SPXF$y*uin}sM;^R$iR6s_>{gXEKa^OJfT(j z8d3O@Cd{GU3?a-URadEAuEz7(2r9rbnHQXEbIf~K72-pl)H{zqh6+@4X=Hl;e6*!v zgD561fb9-zO-?(9iu z!L{+^7u)Q_}fNk=n;4t&SwlzE^+p5X9x$YkBwK$*Si} zU*5W5;?O`c=y2xj8(}7y`ZP|^nzk{O-$B%G`7>+ul^4G_73v$|n_0bxagC0t+*wiXH=5le1 zF2`>7o`?p9F^)G-jGY;J>3`JTZztUOfjO19L-d?B)8$*Q9VX4Fmc4o3_n%D$LYF0I z%t(}oL%IdSX%jJTUhDzk@TGnS$q(!tEngot4-&Fu z*ULtlWx3RbA^JamPUciIs}5g=Jk$G-D0Qdj4AAg-WN*?BaXpV?0fh>Yplkczuv6~j zIUYDORern?ubw+v)sQv3THz}Mexr_suM0)@6B=+b?aML*BcAdxbj*HVR>kG;GVK7S zeL;7_39ALB%IBR!DRr0@)3Jm>Z_aktt`0RmVNUi3^3W?yo^SaVcS%tj>g69xi4F(^|l02S~lUMyB^*q~$KL zvfil^2Lhze1wu8V&y$ryd)W=v7|te8H&wFIzZYk`V6LXp-R?ZTb|+k=N7{IJQfOvPm@;|i{N*%^-Kqn zJghm8Yg#vg9U2C5`{Om^44>=u&T+q9nExTRM5Kg(*wcB}HOoPaGqbpce^{X-{}0eV zYPd5}apI9cUB-r|e8rdP_F&TKeKL0n2i{uRy@Ef|5DveYLzXPt&>%L7!0Mk$2YaL< zau~PSIF~_dxDRbPA1Xys9m2*5BuApdW`J+s3wz5JoyaOcGW9QY8kff8%J%im7 z-DtT2v!eSQhSkooxJ%A~|EWJ;xa@0WKkZo6uMz%9!+*=`;HL*<+rt{HX5=C{Sznei z>Uho0u?A1z6MKDcpE|NC(<=stGdoX z$BnC|gzb#i&F7cb-TmXlePjoIe*P$4wsFF{x34L0v~vm5e;uE4b^rwsFB&Tu=Fm6V z4M65vTu3JyXVS2D8q726FJ4I@bL{G70!bKk8k0inDu0!FUVU>upxlbLG{qM~fNsIVz*jKHtrT5nFn=;L*u;Nt^)&*sQ|p_@cbLW5aOW)9t47_%)}X$KH67fRvvoZq z9w|Nx$BneJE~z1H)ow!w2yKZzDEYg6gRHGL4ky#gLKXJfJt~n@yMx(7E~Au%qRJ_m z=GP0?_vO=EFS~MIVC+{R;M?%NxnmmGt5^n3pzeh&zMD0xR~pVIGAmv^8EjYW1ciJPQko6MU0L?~5cDdX`l zD&&l_2sYK$#%+UvD5-H~B&ju8tU)b9lRmghX=@F<6!qigM6b_bpqW?&1C+9>GStwl<9GT_9N~1=oLAqN?gc)nqM&l zJB4MA>cxzS$ktiOm6Z1ccL@(C@9{6*8Yu3rbU1^*;#Esgbve|V^?$}=DrZyW)EWt@ zadp*YOx%*n*^T3AK_+NiL?lys__B8}zpJb6R-b2$Xid4uF_fNOPsBTr;m}ANw41Q| ziQf8bEo&$~$nZT=Sj`GsJH0uR(6b^E)6hhT>Xg*K0$ieO(5JNc zbw_rxz9L?O-EfO`Gs|I%^{%i!p5>Fq$NyE@bw)M0bZacPA|1oJ@szZ*q9g_N}~tjp-EMAQAB%!IJ4CW z<>HiWc;T=}G$gA1N`LS+{KbXgJ2WX@)w z8(;3b*4KV`j$gpJQc)Gcn3z}$C>JSzxULIMxf__^Z>up^4Vdvev9o6vblqqok-w7c z+PLBSCc@HgI>V_l1>?U(}6QLXi>Y&hiR zvviN3Z`*@@U4jI1~Gy z<}XX9abfXAC4jC)2I{rqN5pbY@haZxZ`@El^Ng6_@eXw}!h|k;@iRz(F2;w#PR-|x zUE!mhhAg`1&%HtvlhT^s65}o?%DxHm;j8==58`bMx3Ecdns*dNOiM(XS*J5n!1qQU z`?h|AeWgC>tnO${?YOWh9(QhG!qt`^0_4XH;Fvdd)*ZYeA+mq>6nZ`O306hjYOMWU z=`@+mU)c9{SwS$vA3L25W|t4NJ*ztpKozDmS1R@((Gcge8)Zh9fKK>*9p>*;x`+{mT&zQTw+rQXR84RrLO*2yYxQ;><@kVTclk^=;XH=H8|89%E;~J-UnQdq zcN}P;cKLzRYS_hv?duKm+ncecWu&;d@qG?qdWR`sSxWD%Y)LZk3N94eTKl&RF>Z6J zY-`^40wx&{PAUCDB2!1Jp z1s^h$ZEkS8T&^20;8dJd z8VI2mmbcJ0oO&VJ{>(xl(^u64gCDK8)?CsR?4mI;1{K;)%EQ^sH0(5?%wTO|^21As zx`xA5SHAUwS)7~zjvI%&-+uf>^1NVBSd-aaFdtCHvRSbkZ3;+7K-COfy?DqKqrnXh zl$DUGtX>V4&88U`jd)a&Yo`>TmIPY*gGQ368eP7(DsvR;uYXvVYee#{_}$kmMy|Jz zy-SP~P62iM$#r<5AL&<+1+ekQYHkl6WT8BA(uvPqM@I*AGb|mpmEK+`z!~hSZh8vP z)bxls3sk_*%eQ5S3n^+F`Vq@r1fBPbnne`CmNzsyg&yO;hNJEci^L|SlN^BbwUK1t zQa(VURBAdc@nO>Q`;g=fA1fo|6I&-DL=E(Ej5RlZ&yRBCjAfY;_1-qMY{U^gpdOYI^V6Dp z__e5z{T;oay}pn4ItK_FUsdoOJ6IJ zftEWp)8pOgDVDf0rY&%u`HrKA+g4{*NoibQCrPdfV}vXatLH^iL;9_%&^Z#YWOtsO zWT99^O7fVR_na!M7gfZEbN!yMrM7T;Y15aes}eQg!0q?|UmjoweX_F2Iz&yt37 z3lZKV!~pB;JGrV>%T^~Z41K#`I~hEHbn|E6yc-HvA(L=7cUxmW@Ue1N5Fm}CnK1(lY9jp?s*^NE@q_~H5HoTe91ceq^zoF2h>?zsI zfaPI{1q7k|(>5k4?TMH74m~ZU0}_>-GVa2y?0Ba2=FPswF_8#)s)uEY*XFVfi@z{0 zlI}*4rJ2_MMO0}uhM#3UwdW@$M>h_o)(WKS_U)9fmW;+vsYp1|sH^&sL3y)UL4xzz z4NW_wuNQiiqQ3{8IP-uri!pdOGJ^Mxa)pgC=;lqvh)j1T?I3OTSO{!TH0l9;SW?Q1ALv0}`yC4y)KPr9xu1rv-sbC-!*g+LO97@RP~Wpnp?+M zcS1vf!%-@YWZipN3wja>3LgTbn{{07?nl{Kk^j0B+?+x{62EZ-ud=iKmP?<9_cpN7 zP)PA7UiVmmamWwq!Y>WLo46zt%c(QdC}EGv12}FbwC9HIchZx>Tq;e!92nnhmtAaOZ0UqX$Ns+d8!FeH zZb~a7JMJ}@szjR~=5#!i9BuK9KGojKq@wcKpQ+KbLabi546Vw*3e39ljXR&n21Shb zf-`V#o{5fV|Au1@n)liF$8VDK4kEnoV+ZKH(6ziBOhz3(fwge~^}C%+0IQp%U>X9Sd|-X2b=ZsVH)> zruwT0D{nuC=7`M>e!@cyPHx{ENkjHVT#DgwyF1ndjB#XY1D3pCRCoN{<@|QOle6Y8 z1AWzCQ>9U)p9& z%6JIpl6y7JoY5bHr51gD3d}pZ-bK3oV?^;$IR#=RRr+?r|CA%nZUSB5_eXE7nVFgm zAdIa&opnvr6BRDiK3M6-cOtu77`}b1q2|Pc6myDzs;8KjHFZvN%}mv%bEMnlo5i}u zNj4y{=Bnq-P-%|i7v?b$dLQn*w)H!5wg6l!?odKn{no8TIY?N)crC+o2J>j|GCL${az|Mjtz3MDog9$L0g)T~iN>CF8AP@1ozs-m zpE~|>`TNK-ALS+X_ongmO^jA62b=C;4l~-?+8^)lwxU2N^J{7EHMB=dMI|O4^)-tX z{kfX949$vO(rF4Z`K9PIq`YDFhib*7R+ivN>dg6&J&JUV?&!fPt9J^OVref^FfkTy zk|%wHH(o4_nAp>~-BX_s(iwrQ9XFw$qZ=}-5{btd2CWD-wxgkXH?Nz%+?9MfbtsQV z_G0YwzFhDR4bfLL0p9M0X>0MWUA3UrsZ5Da;14TrNp+6wfhThu+W^SJ{6LCuN++-!$WQW6K=nJ4N=EpJE1V+j$ zCVwUz@k`{&5&^y|vV7HI*$7u~%WE=nb=BN5*k&vY07QR=`%)k^>f@<9w8eQaqG7VA z1A?HVaIF{X-}#cQTGBz$nfy^y(_Ep<6OgN-#a=qvw-*YVG|F`UG9INz$1k{}4%kPl z&mWk*No=fjnN1nawn_}HmKdU1durqnyWX|uZEl|u>%uvfil)QI+Pb|>(8V2ht8^tv z*YM$$rm7pK>_|{>erMZrF_X=wiQ1=z8Pgt#bN~3fCyh%A*FNF1xt0TL-QR5PkZ9;f zOW^TuvwD#i&$j|=PeE4o#iO7%3yY1E*2`k*R=96pDgLq7DIv93ZdW#Bf#l4dV>4n0 zq(rff=wB12L|e~I4-0yXsvi)FP^>(qkM3K(qUw%w;uVXXBGd6lN3Z*bB0-B8Z&vqa zTxyQ?IsQcMTy-CG!eV)%E1cAlPi;NhQ?^V*Kv==XK6YAUfQM2v;SDbDjze-J^JnGm zRq%M%WrVbkXs@~o%-UAL>W}ZnDz;~j2z4TH6A82ia;9$eDFf06(*liWG!GSFp<>51 zISdHcR_-W*PMAS#@E~8$MEJ+}Ho0i3!vor0*|=?TEW4<`=9!qSb^ERJW|8v9xxC%4 zVO%Dau8(ff0W1eL*D8EAh4M-D4dqj{l#$2>Kt4J=eAJMUA{%x7zk%m;RrZ0R{DW8j zFSz<}3APa2( z>F6BxsZHJ8S|Rt_K_ca;OM;V>)(DI0C(kD*?Zs~V3qB|1Xv@oAoGW6rk-_vyycZ-o z2yy_8`&h*iU&XP@{|h$%`9xWN_iqfn|M@S6|6d6GA2qeaJOsoGti6EdZLqOK@0QWc JLLKWD{{`hMvT6VT literal 0 HcmV?d00001 diff --git a/docs/web-ui.md b/docs/web-ui.md index 72423d9468e83..e6025370e6796 100644 --- a/docs/web-ui.md +++ b/docs/web-ui.md @@ -404,3 +404,44 @@ The web UI includes a Streaming tab if the application uses Spark streaming. Thi scheduling delay and processing time for each micro-batch in the data stream, which can be useful for troubleshooting the streaming application. +## JDBC/ODBC Server Tab +We can see this tab when Spark is running as a [distributed SQL engine](sql-distributed-sql-engine.html). It shows information about sessions and submitted SQL operations. + +The first section of the page displays general information about the JDBC/ODBC server: start time and uptime. + +

+ JDBC/ODBC Header +

+ +The second section contains information about active and finished sessions. +* **User** and **IP** of the connection. +* **Session id** link to access to session info. +* **Start time**, **finish time** and **duration** of the session. +* **Total execute** is the number of operations submitted in this session. + +

+ JDBC/ODBC sessions +

+ +The third section has the SQL statistics of the submitted operations. +* **User** that submit the operation. +* **Job id** link to [jobs tab](web-ui.html#jobs-tab). +* **Group id** of the query that group all jobs together. An application can cancel all running jobs using this group id. +* **Start time** of the operation. +* **Finish time** of the execution, before fetching the results. +* **Close time** of the operation after fetching the results. +* **Execution time** is the difference between finish time and start time. +* **Duration time** is the difference between close time and start time. +* **Statement** is the operation being executed. +* **State** of the process. + * _Started_, first state, when the process begins. + * _Compiled_, execution plan generated. + * _Failed_, final state when the execution failed or finished with error. + * _Canceled_, final state when the execution is canceled. + * _Finished_ processing and waiting to fetch results. + * _Closed_, final state when client closed the statement. +* **Detail** of the execution plan with parsed logical plan, analyzed logical plan, optimized logical plan and physical plan or errors in the the SQL statement. + +

+ JDBC/ODBC SQL Statistics +

From 956f6e988cf83cf68f6cea214b3d9045920bca55 Mon Sep 17 00:00:00 2001 From: Andy Zhang Date: Sun, 15 Sep 2019 00:17:11 -0700 Subject: [PATCH 0069/1387] [SPARK-29080][CORE][SPARKR] Support R file extension case-insensitively ### What changes were proposed in this pull request? Make r file extension check case insensitive for spark-submit. ### Why are the changes needed? spark-submit does not accept `.r` files as R scripts. Some codebases have r files that end with lowercase file extensions. It is inconvenient to use spark-submit with lowercase extension R files. The error is not very clear (https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala#L232). ``` $ ./bin/spark-submit examples/src/main/r/dataframe.r Exception in thread "main" org.apache.spark.SparkException: Cannot load main class from JAR file:/Users/dongjoon/APACHE/spark-release/spark-2.4.4-bin-hadoop2.7/examples/src/main/r/dataframe.r ``` ### Does this PR introduce any user-facing change? Yes. spark-submit can now be used to run R scripts with `.r` file extension. ### How was this patch tested? Manual. ``` $ mv examples/src/main/r/dataframe.R examples/src/main/r/dataframe.r $ ./bin/spark-submit examples/src/main/r/dataframe.r ``` Closes #25778 from Loquats/r-case. Authored-by: Andy Zhang Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../org/apache/spark/launcher/SparkSubmitCommandBuilder.java | 2 +- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 ++- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 3 ++- 4 files changed, 6 insertions(+), 4 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 ebc843d9758ee..b776ec8f81e06 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -1028,7 +1028,7 @@ object SparkSubmit extends CommandLineUtils with Logging { * Return whether the given primary resource requires running R. */ private[deploy] def isR(res: String): Boolean = { - res != null && res.endsWith(".R") || res == SPARKR_SHELL + res != null && (res.endsWith(".R") || res.endsWith(".r")) || res == SPARKR_SHELL } private[deploy] def isInternal(res: String): Boolean = { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index f3805f70ea30d..f603e100fd2eb 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -348,7 +348,7 @@ private List buildPySparkShellCommand(Map env) throws IO } private List buildSparkRCommand(Map env) throws IOException { - if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".R")) { + if (!appArgs.isEmpty() && (appArgs.get(0).endsWith(".R") || appArgs.get(0).endsWith(".r"))) { System.err.println( "Running R applications through 'sparkR' is not supported as of Spark 2.0.\n" + "Use ./bin/spark-submit "); 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 ae9486d7e44cf..4d884d54501ad 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 @@ -703,7 +703,8 @@ private[spark] class ApplicationMaster( // of files to add to PYTHONPATH, which Client.scala already handles, so it's empty. userArgs = Seq(args.primaryPyFile, "") ++ userArgs } - if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { + if (args.primaryRFile != null && + (args.primaryRFile.endsWith(".R") || args.primaryRFile.endsWith(".r"))) { // TODO(davies): add R dependencies here } 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 9be3e7bbbfa67..79e63d77a40f2 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 @@ -973,7 +973,8 @@ private[spark] class Client( } else { Utils.classForName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName } - if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { + if (args.primaryRFile != null && + (args.primaryRFile.endsWith(".R") || args.primaryRFile.endsWith(".r"))) { args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs } val userArgs = args.userArgs.flatMap { arg => From 8c0e961f6c5fe0da3b36a6fe642c12f88ac34d0f Mon Sep 17 00:00:00 2001 From: David Lewis Date: Sun, 15 Sep 2019 00:28:32 -0700 Subject: [PATCH 0070/1387] [SPARK-29081][CORE] Replace calls to SerializationUtils.clone on properties with a faster implementation Replace use of `SerializationUtils.clone` with new `Utils.cloneProperties` method Add benchmark + results showing dramatic speed up for effectively equivalent functionality. ### What changes were proposed in this pull request? While I am not sure that SerializationUtils.clone is a performance issue in production, I am sure that it is overkill for the task it is doing (providing a distinct copy of a `Properties` object). This PR provides a benchmark showing the dramatic improvement over the clone operation and replaces uses of `SerializationUtils.clone` on `Properties` with the more specialized `Utils.cloneProperties`. ### Does this PR introduce any user-facing change? Strings are immutable so there is no reason to serialize and deserialize them, it just creates extra garbage. The only functionality that would be changed is the unsupported insertion of non-String objects into the spark local properties. ### How was this patch tested? 1. Pass the Jenkins with the existing tests. 2. Since this is a performance improvement PR, manually run the benchmark. Closes #25787 from databricks-david-lewis/SPARK-29081. Authored-by: David Lewis Signed-off-by: Dongjoon Hyun --- .../PropertiesCloneBenchmark-results.txt | 40 ++++++++++ .../scala/org/apache/spark/SparkContext.scala | 3 +- .../apache/spark/scheduler/DAGScheduler.scala | 10 +-- .../scala/org/apache/spark/util/Utils.scala | 7 ++ .../apache/spark/benchmark/Benchmark.scala | 4 +- .../spark/util/PropertiesCloneBenchmark.scala | 74 +++++++++++++++++++ .../spark/util/ResetSystemProperties.scala | 5 +- .../spark/streaming/StreamingContext.scala | 3 +- .../streaming/scheduler/JobScheduler.scala | 6 +- 9 files changed, 134 insertions(+), 18 deletions(-) create mode 100644 core/benchmarks/PropertiesCloneBenchmark-results.txt create mode 100644 core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala diff --git a/core/benchmarks/PropertiesCloneBenchmark-results.txt b/core/benchmarks/PropertiesCloneBenchmark-results.txt new file mode 100644 index 0000000000000..00c9561648be0 --- /dev/null +++ b/core/benchmarks/PropertiesCloneBenchmark-results.txt @@ -0,0 +1,40 @@ +================================================================================================ +Properties Cloning +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 +Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +Empty Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 0 0 0 0.2 4184.0 1.0X +Utils.cloneProperties 0 0 0 55.6 18.0 232.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 +Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +System Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 0 0 0 0.0 107612.0 1.0X +Utils.cloneProperties 0 0 0 1.0 962.0 111.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 +Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +Small Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 0 0 0 0.0 330210.0 1.0X +Utils.cloneProperties 0 0 0 0.9 1082.0 305.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 +Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +Medium Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 1 2 0 0.0 1336301.0 1.0X +Utils.cloneProperties 0 0 0 0.2 5456.0 244.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14.6 +Intel(R) Core(TM) i9-8950HK CPU @ 2.90GHz +Large Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 3 3 0 0.0 2634336.0 1.0X +Utils.cloneProperties 0 0 0 0.1 10822.0 243.4X + + diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 396d712bd739c..44c59e2a56c28 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -31,7 +31,6 @@ import scala.reflect.{classTag, ClassTag} import scala.util.control.NonFatal import com.google.common.collect.MapMaker -import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} @@ -346,7 +345,7 @@ class SparkContext(config: SparkConf) extends Logging { override protected def childValue(parent: Properties): Properties = { // Note: make a clone such that changes in the parent properties aren't reflected in // the those of the children threads, which has confusing semantics (SPARK-10563). - SerializationUtils.clone(parent) + Utils.cloneProperties(parent) } override protected def initialValue(): Properties = new Properties() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 9df59459ca799..894234f70e05a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -29,8 +29,6 @@ import scala.collection.mutable.{HashMap, HashSet, ListBuffer} import scala.concurrent.duration._ import scala.util.control.NonFatal -import org.apache.commons.lang3.SerializationUtils - import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} @@ -698,7 +696,7 @@ private[spark] class DAGScheduler( if (partitions.isEmpty) { val time = clock.getTimeMillis() listenerBus.post( - SparkListenerJobStart(jobId, time, Seq[StageInfo](), SerializationUtils.clone(properties))) + SparkListenerJobStart(jobId, time, Seq[StageInfo](), Utils.cloneProperties(properties))) listenerBus.post( SparkListenerJobEnd(jobId, time, JobSucceeded)) // Return immediately if the job is running 0 tasks @@ -710,7 +708,7 @@ private[spark] class DAGScheduler( val waiter = new JobWaiter[U](this, jobId, partitions.size, resultHandler) eventProcessLoop.post(JobSubmitted( jobId, rdd, func2, partitions.toArray, callSite, waiter, - SerializationUtils.clone(properties))) + Utils.cloneProperties(properties))) waiter } @@ -782,7 +780,7 @@ private[spark] class DAGScheduler( val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] eventProcessLoop.post(JobSubmitted( jobId, rdd, func2, rdd.partitions.indices.toArray, callSite, listener, - SerializationUtils.clone(properties))) + Utils.cloneProperties(properties))) listener.awaitResult() // Will throw an exception if the job fails } @@ -819,7 +817,7 @@ private[spark] class DAGScheduler( this, jobId, 1, (_: Int, r: MapOutputStatistics) => callback(r)) eventProcessLoop.post(MapStageSubmitted( - jobId, dependency, callSite, waiter, SerializationUtils.clone(properties))) + jobId, dependency, callSite, waiter, Utils.cloneProperties(properties))) waiter } 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 5cd937aeb8ea4..c47a23edde726 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2950,6 +2950,13 @@ private[spark] object Utils extends Logging { val codec = codecFactory.getCodec(path) codec == null || codec.isInstanceOf[SplittableCompressionCodec] } + + /** Create a new properties object with the same values as `props` */ + def cloneProperties(props: Properties): Properties = { + val resultProps = new Properties() + props.forEach((k, v) => resultProps.put(k, v)) + resultProps + } } private[util] object CallerContext extends Logging { diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index 73f9d0e2bc0e1..022fcbb25b0af 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -141,12 +141,14 @@ private[spark] class Benchmark( val minIters = if (overrideNumIters != 0) overrideNumIters else minNumIters val minDuration = if (overrideNumIters != 0) 0 else minTime.toNanos val runTimes = ArrayBuffer[Long]() + var totalTime = 0L var i = 0 - while (i < minIters || runTimes.sum < minDuration) { + while (i < minIters || totalTime < minDuration) { val timer = new Benchmark.Timer(i) f(timer) val runTime = timer.totalTime() runTimes += runTime + totalTime += runTime if (outputPerIteration) { // scalastyle:off diff --git a/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala b/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala new file mode 100644 index 0000000000000..0726886c70fe6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.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.util + +import java.util.Properties + +import scala.util.Random + +import org.apache.commons.lang.SerializationUtils + +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} + + +/** + * Benchmark for Kryo Unsafe vs safe Serialization. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "core/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " + * Results will be written to "benchmarks/PropertiesCloneBenchmark-results.txt". + * }}} + */ +object PropertiesCloneBenchmark extends BenchmarkBase { + /** + * Benchmark various cases of cloning properties objects + */ + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("Properties Cloning") { + def compareSerialization(name: String, props: Properties): Unit = { + val benchmark = new Benchmark(name, 1, output = output) + benchmark.addCase("SerializationUtils.clone") { _ => + SerializationUtils.clone(props) + } + benchmark.addCase("Utils.cloneProperties") { _ => + Utils.cloneProperties(props) + } + benchmark.run() + } + compareSerialization("Empty Properties", new Properties) + compareSerialization("System Properties", System.getProperties) + compareSerialization("Small Properties", makeRandomProps(10, 40, 100)) + compareSerialization("Medium Properties", makeRandomProps(50, 40, 100)) + compareSerialization("Large Properties", makeRandomProps(100, 40, 100)) + } + } + + def makeRandomProps(numProperties: Int, keySize: Int, valueSize: Int): Properties = { + val props = new Properties + for (_ <- 1 to numProperties) { + props.put( + Random.alphanumeric.take(keySize), + Random.alphanumeric.take(valueSize) + ) + } + props + } +} diff --git a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala index 75e4504850679..0b1796540abbb 100644 --- a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala +++ b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala @@ -19,7 +19,6 @@ package org.apache.spark.util import java.util.Properties -import org.apache.commons.lang3.SerializationUtils import org.scalatest.{BeforeAndAfterEach, Suite} /** @@ -43,11 +42,11 @@ private[spark] trait ResetSystemProperties extends BeforeAndAfterEach { this: Su var oldProperties: Properties = null override def beforeEach(): Unit = { - // we need SerializationUtils.clone instead of `new Properties(System.getProperties())` because + // we need Utils.cloneProperties instead of `new Properties(System.getProperties())` because // the later way of creating a copy does not copy the properties but it initializes a new // Properties object with the given properties as defaults. They are not recognized at all // by standard Scala wrapper over Java Properties then. - oldProperties = SerializationUtils.clone(System.getProperties) + oldProperties = Utils.cloneProperties(System.getProperties) super.beforeEach() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 589dd877c8c97..21ffa78297ee3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -26,7 +26,6 @@ import scala.collection.mutable.Queue import scala.reflect.ClassTag import scala.util.control.NonFatal -import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} @@ -586,7 +585,7 @@ class StreamingContext private[streaming] ( sparkContext.setCallSite(startSite.get) sparkContext.clearJobGroup() sparkContext.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false") - savedProperties.set(SerializationUtils.clone(sparkContext.localProperties.get())) + savedProperties.set(Utils.cloneProperties(sparkContext.localProperties.get())) scheduler.start() } state = StreamingContextState.ACTIVE diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 68594e8977cf8..2388ca8059807 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -22,8 +22,6 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ import scala.util.Failure -import org.apache.commons.lang3.SerializationUtils - import org.apache.spark.ExecutorAllocationClient import org.apache.spark.internal.Logging import org.apache.spark.internal.io.SparkHadoopWriterUtils @@ -31,7 +29,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.api.python.PythonDStream import org.apache.spark.streaming.ui.UIUtils -import org.apache.spark.util.{EventLoop, ThreadUtils} +import org.apache.spark.util.{EventLoop, ThreadUtils, Utils} private[scheduler] sealed trait JobSchedulerEvent @@ -231,7 +229,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { def run() { val oldProps = ssc.sparkContext.getLocalProperties try { - ssc.sparkContext.setLocalProperties(SerializationUtils.clone(ssc.savedProperties.get())) + ssc.sparkContext.setLocalProperties(Utils.cloneProperties(ssc.savedProperties.get())) val formattedTime = UIUtils.formatBatchTime( job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" From 729b3180bcbaa5289cb9a5848a3cce9010e75515 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 15 Sep 2019 10:15:49 -0700 Subject: [PATCH 0071/1387] [SPARK-29087][CORE][STREAMING] Use DelegatingServletContextHandler to avoid CCE ### What changes were proposed in this pull request? [SPARK-27122](https://github.com/apache/spark/pull/24088) fixes `ClassCastException` at `yarn` module by introducing `DelegatingServletContextHandler`. Initially, this was discovered with JDK9+, but the class path issues affected JDK8 environment, too. After [SPARK-28709](https://github.com/apache/spark/pull/25439), I also hit the similar issue at `streaming` module. This PR aims to fix `streaming` module by adding `getContextPath` to `DelegatingServletContextHandler` and using it. ### Why are the changes needed? Currently, when we test `streaming` module independently, it fails like the following. ``` $ build/mvn test -pl streaming ... UISeleniumSuite: - attaching and detaching a Streaming tab *** FAILED *** java.lang.ClassCastException: org.sparkproject.jetty.servlet.ServletContextHandler cannot be cast to org.eclipse.jetty.servlet.ServletContextHandler ... Tests: succeeded 337, failed 1, canceled 0, ignored 1, pending 0 *** 1 TEST FAILED *** [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the modified tests. And do the following manually. Since you can observe this when you run `streaming` module test only (instead of running all), you need to install the changed `core` module and use it. ``` $ java -version openjdk version "1.8.0_222" OpenJDK Runtime Environment (AdoptOpenJDK)(build 1.8.0_222-b10) OpenJDK 64-Bit Server VM (AdoptOpenJDK)(build 25.222-b10, mixed mode) $ build/mvn install -DskipTests $ build/mvn test -pl streaming ``` Closes #25791 from dongjoon-hyun/SPARK-29087. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/ui/WebUI.scala | 4 ++++ .../scala/org/apache/spark/streaming/UISeleniumSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 1fe822a0e3b57..39d64056ad430 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -236,4 +236,8 @@ private[spark] class DelegatingServletContextHandler(handler: ServletContextHand def filterCount(): Int = { handler.getServletHandler.getFilters.length } + + def getContextPath(): String = { + handler.getContextPath + } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 1d34221fde4f4..3d477ca3d0a9d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -97,7 +97,7 @@ class UISeleniumSuite val sparkUI = ssc.sparkContext.ui.get - sparkUI.getHandlers.count(_.getContextPath.contains("/streaming")) should be (5) + sparkUI.getDelegatingHandlers.count(_.getContextPath.contains("/streaming")) should be (5) eventually(timeout(10.seconds), interval(50.milliseconds)) { go to (sparkUI.webUrl.stripSuffix("/")) @@ -198,7 +198,7 @@ class UISeleniumSuite ssc.stop(false) - sparkUI.getHandlers.count(_.getContextPath.contains("/streaming")) should be (0) + sparkUI.getDelegatingHandlers.count(_.getContextPath.contains("/streaming")) should be (0) eventually(timeout(10.seconds), interval(50.milliseconds)) { go to (sparkUI.webUrl.stripSuffix("/")) From 1b7afc0c986ed8e5431df351f51434424460f4b3 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Sep 2019 11:02:15 -0700 Subject: [PATCH 0072/1387] [SPARK-28471][SQL][DOC][FOLLOWUP] Fix year patterns in the comments of date-time expressions ### What changes were proposed in this pull request? In the PR, I propose to fix comments of date-time expressions, and replace the `yyyy` pattern by `uuuu` when the implementation supposes the former one. ### Why are the changes needed? To make comments consistent to implementations. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running Scala Style checker. Closes #25796 from MaxGekk/year-pattern-uuuu-followup. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 2 +- .../apache/spark/sql/catalyst/util/TimestampFormatter.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 9d43701f03056..eafdca2bfc53e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -647,7 +647,7 @@ case class ToUnixTimestamp( * Converts time string with given pattern to Unix time stamp (in seconds), returns null if fail. * See [https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html]. * Note that hive Language Manual says it returns 0 if fail, but in fact it returns null. - * If the second parameter is missing, use "yyyy-MM-dd HH:mm:ss". + * If the second parameter is missing, use "uuuu-MM-dd HH:mm:ss". * If no parameters provided, the first parameter will be current_timestamp. * If the first parameter is a Date or Timestamp instead of String, we will ignore the * second parameter. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index b23cec64568df..3dfcf1a33eb56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -67,7 +67,7 @@ class Iso8601TimestampFormatter( } /** - * The formatter parses/formats timestamps according to the pattern `yyyy-MM-dd HH:mm:ss.[..fff..]` + * The formatter parses/formats timestamps according to the pattern `uuuu-MM-dd HH:mm:ss.[..fff..]` * where `[..fff..]` is a fraction of second up to microsecond resolution. The formatter does not * output trailing zeros in the fraction. For example, the timestamp `2019-03-05 15:00:01.123400` is * formatted as the string `2019-03-05 15:00:01.1234`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 395f1b4667b1c..84e0eaff2d428 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2634,8 +2634,8 @@ object functions { * See [[java.time.format.DateTimeFormatter]] for valid date and time format patterns * * @param dateExpr A date, timestamp or string. If a string, the data must be in a format that - * can be cast to a timestamp, such as `yyyy-MM-dd` or `yyyy-MM-dd HH:mm:ss.SSSS` - * @param format A pattern `dd.MM.yyyy` would return a string like `18.03.1993` + * can be cast to a timestamp, such as `uuuu-MM-dd` or `uuuu-MM-dd HH:mm:ss.SSSS` + * @param format A pattern `dd.MM.uuuu` would return a string like `18.03.1993` * @return A string, or null if `dateExpr` was a string that could not be cast to a timestamp * @note Use specialized functions like [[year]] whenever possible as they benefit from a * specialized implementation. From 61e5aebce3925e7c512899939688e0eee4ac8a06 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sun, 15 Sep 2019 11:04:56 -0700 Subject: [PATCH 0073/1387] [SPARK-29046][SQL] Fix NPE in SQLConf.get when active SparkContext is stopping ### What changes were proposed in this pull request? This patch fixes the bug regarding NPE in SQLConf.get, which is only possible when SparkContext._dagScheduler is null due to stopping SparkContext. The logic doesn't seem to consider active SparkContext could be in progress of stopping. Note that it can't be encountered easily as SparkContext.stop() blocks the main thread, but there're many cases which SQLConf.get is accessed concurrently while SparkContext.stop() is executing - users run another threads, or listener is accessing SQLConf.get after dagScheduler is set to null (this is the case what I encountered.) ### Why are the changes needed? The bug brings NPE. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Previous patch #25753 was tested with new UT, and due to disruption with other tests in concurrent test run, the test is excluded in this patch. Closes #25790 from HeartSaVioR/SPARK-29046-v2. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d9b0a72618c7e..4f3e39ad49afe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -139,7 +139,8 @@ object SQLConf { } } else { val isSchedulerEventLoopThread = SparkContext.getActive - .map(_.dagScheduler.eventProcessLoop.eventThread) + .flatMap { sc => Option(sc.dagScheduler) } + .map(_.eventProcessLoop.eventThread) .exists(_.getId == Thread.currentThread().getId) if (isSchedulerEventLoopThread) { // DAGScheduler event loop thread does not have an active SparkSession, the `confGetter` From b91648cfd0d7ab7014a137cdb61d8dbb3611438c Mon Sep 17 00:00:00 2001 From: "changchun.wang" <251922566@qq.com> Date: Sun, 15 Sep 2019 11:11:38 -0700 Subject: [PATCH 0074/1387] [SPARK-28856][FOLLOW-UP][SQL][TEST] Add the `namespaces` keyword to TableIdentifierParserSuite ### What changes were proposed in this pull request? This PR add the `namespaces` keyword to `TableIdentifierParserSuite`. ### Why are the changes needed? Improve the test. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #25758 from highmoutain/3.0bugfix. Authored-by: changchun.wang <251922566@qq.com> Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/TableIdentifierParserSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 246d0cf01e9ba..81f0bd024a80e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -443,6 +443,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "month", "months", "msck", + "namespaces", "natural", "no", "not", From 7d4eb38bbcc887fb61ba7344df3f77a046ad77f8 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sun, 15 Sep 2019 11:17:30 -0700 Subject: [PATCH 0075/1387] [SPARK-29052][DOCS][ML][PYTHON][CORE][R][SQL][SS] Create a Migration Guide tap in Spark documentation MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Currently, there is no migration section for PySpark, SparkCore and Structured Streaming. It is difficult for users to know what to do when they upgrade. This PR proposes to create create a "Migration Guide" tap at Spark documentation. ![Screen Shot 2019-09-11 at 7 02 05 PM](https://user-images.githubusercontent.com/6477701/64688126-ad712f80-d4c6-11e9-8672-9a2c56c05bf8.png) ![Screen Shot 2019-09-11 at 7 27 15 PM](https://user-images.githubusercontent.com/6477701/64689915-389ff480-d4ca-11e9-8c54-7f46095d0d23.png) This page will contain migration guides for Spark SQL, PySpark, SparkR, MLlib, Structured Streaming and Core. Basically it is a refactoring. There are some new information added, which I will leave a comment inlined for easier review. 1. **MLlib** Merge [ml-guide.html#migration-guide](https://spark.apache.org/docs/latest/ml-guide.html#migration-guide) and [ml-migration-guides.html](https://spark.apache.org/docs/latest/ml-migration-guides.html) ``` 'docs/ml-guide.md' ↓ Merge new/old migration guides 'docs/ml-migration-guide.md' ``` 2. **PySpark** Extract PySpark specific items from https://spark.apache.org/docs/latest/sql-migration-guide-upgrade.html ``` 'docs/sql-migration-guide-upgrade.md' ↓ Extract PySpark specific items 'docs/pyspark-migration-guide.md' ``` 3. **SparkR** Move [sparkr.html#migration-guide](https://spark.apache.org/docs/latest/sparkr.html#migration-guide) into a separate file, and extract from [sql-migration-guide-upgrade.html](https://spark.apache.org/docs/latest/sql-migration-guide-upgrade.html) ``` 'docs/sparkr.md' 'docs/sql-migration-guide-upgrade.md' Move migration guide section ↘ ↙ Extract SparkR specific items docs/sparkr-migration-guide.md ``` 4. **Core** Newly created at `'docs/core-migration-guide.md'`. I skimmed resolved JIRAs at 3.0.0 and found some items to note. 5. **Structured Streaming** Newly created at `'docs/ss-migration-guide.md'`. I skimmed resolved JIRAs at 3.0.0 and found some items to note. 6. **SQL** Merged [sql-migration-guide-upgrade.html](https://spark.apache.org/docs/latest/sql-migration-guide-upgrade.html) and [sql-migration-guide-hive-compatibility.html](https://spark.apache.org/docs/latest/sql-migration-guide-hive-compatibility.html) ``` 'docs/sql-migration-guide-hive-compatibility.md' 'docs/sql-migration-guide-upgrade.md' Move Hive compatibility section ↘ ↙ Left over after filtering PySpark and SparkR items 'docs/sql-migration-guide.md' ``` ### Why are the changes needed? In order for users in production to effectively migrate to higher versions, and detect behaviour or breaking changes before upgrading and/or migrating. ### Does this PR introduce any user-facing change? Yes, this changes Spark's documentation at https://spark.apache.org/docs/latest/index.html. ### How was this patch tested? Manually build the doc. This can be verified as below: ```bash cd docs SKIP_API=1 jekyll build open _site/index.html ``` Closes #25757 from HyukjinKwon/migration-doc. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- docs/_data/menu-migration.yaml | 12 + docs/_data/menu-sql.yaml | 10 +- .../_includes/nav-left-wrapper-migration.html | 6 + docs/_layouts/global.html | 7 +- docs/core-migration-guide.md | 32 + docs/index.md | 1 + docs/migration-guide.md | 30 + docs/ml-guide.md | 65 +- ...ration-guides.md => ml-migration-guide.md} | 96 +- docs/pyspark-migration-guide.md | 120 +++ docs/sparkr-migration-guide.md | 77 ++ docs/sparkr.md | 46 +- .../sql-migration-guide-hive-compatibility.md | 167 ---- docs/sql-migration-guide-upgrade.md | 834 ---------------- docs/sql-migration-guide.md | 910 +++++++++++++++++- ...gration-guides.md => sql-migration-old.md} | 7 +- docs/ss-migration-guide.md | 32 + 17 files changed, 1295 insertions(+), 1157 deletions(-) create mode 100644 docs/_data/menu-migration.yaml create mode 100644 docs/_includes/nav-left-wrapper-migration.html create mode 100644 docs/core-migration-guide.md create mode 100644 docs/migration-guide.md rename docs/{ml-migration-guides.md => ml-migration-guide.md} (85%) create mode 100644 docs/pyspark-migration-guide.md create mode 100644 docs/sparkr-migration-guide.md delete mode 100644 docs/sql-migration-guide-hive-compatibility.md delete mode 100644 docs/sql-migration-guide-upgrade.md rename docs/{mllib-migration-guides.md => sql-migration-old.md} (73%) create mode 100644 docs/ss-migration-guide.md diff --git a/docs/_data/menu-migration.yaml b/docs/_data/menu-migration.yaml new file mode 100644 index 0000000000000..1d8b311dd64fb --- /dev/null +++ b/docs/_data/menu-migration.yaml @@ -0,0 +1,12 @@ +- text: Spark Core + url: core-migration-guide.html +- text: SQL, Datasets and DataFrame + url: sql-migration-guide.html +- text: Structured Streaming + url: ss-migration-guide.html +- text: MLlib (Machine Learning) + url: ml-migration-guide.html +- text: PySpark (Python on Spark) + url: pyspark-migration-guide.html +- text: SparkR (R on Spark) + url: sparkr-migration-guide.html diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index d03616163c964..07ca150a1523f 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -64,15 +64,7 @@ - text: Usage Notes url: sql-pyspark-pandas-with-arrow.html#usage-notes - text: Migration Guide - url: sql-migration-guide.html - subitems: - - text: Spark SQL Upgrading Guide - url: sql-migration-guide-upgrade.html - - text: Compatibility with Apache Hive - url: sql-migration-guide-hive-compatibility.html - - text: SQL Reserved/Non-Reserved Keywords - url: sql-reserved-and-non-reserved-keywords.html - + url: sql-migration-old.html - text: SQL Reference url: sql-ref.html subitems: diff --git a/docs/_includes/nav-left-wrapper-migration.html b/docs/_includes/nav-left-wrapper-migration.html new file mode 100644 index 0000000000000..4318a324a9475 --- /dev/null +++ b/docs/_includes/nav-left-wrapper-migration.html @@ -0,0 +1,6 @@ +
+
+

Migration Guide

+ {% include nav-left.html nav=include.nav-migration %} +
+
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 8ea15dc71d541..d5fb18bfb06c0 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -112,6 +112,7 @@
  • Job Scheduling
  • Security
  • Hardware Provisioning
  • +
  • Migration Guide
  • Building Spark
  • Contributing to Spark
  • @@ -126,8 +127,10 @@
    - {% if page.url contains "/ml" or page.url contains "/sql" %} - {% if page.url contains "/ml" %} + {% if page.url contains "/ml" or page.url contains "/sql" or page.url contains "migration-guide.html" %} + {% if page.url contains "migration-guide.html" %} + {% include nav-left-wrapper-migration.html nav-migration=site.data.menu-migration %} + {% elsif page.url contains "/ml" %} {% include nav-left-wrapper-ml.html nav-mllib=site.data.menu-mllib nav-ml=site.data.menu-ml %} {% else %} {% include nav-left-wrapper-sql.html nav-sql=site.data.menu-sql %} diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md new file mode 100644 index 0000000000000..1c47dfb27124c --- /dev/null +++ b/docs/core-migration-guide.md @@ -0,0 +1,32 @@ +--- +layout: global +title: "Migration Guide: Spark Core" +displayTitle: "Migration Guide: Spark Core" +license: | + 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. +--- + +* Table of contents +{:toc} + +## Upgrading from Core 2.4 to 3.0 + +- In Spark 3.0, deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. + +- In Spark 3.0, deprecated method `shuffleBytesWritten`, `shuffleWriteTime` and `shuffleRecordsWritten` in `ShuffleWriteMetrics` have been removed. Instead, use `bytesWritten`, `writeTime ` and `recordsWritten` respectively. + +- In Spark 3.0, deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. + diff --git a/docs/index.md b/docs/index.md index 4217918a87462..6cfbbaed20028 100644 --- a/docs/index.md +++ b/docs/index.md @@ -146,6 +146,7 @@ options for deployment: * Integration with other storage systems: * [Cloud Infrastructures](cloud-integration.html) * [OpenStack Swift](storage-openstack-swift.html) +* [Migration Guide](migration-guide.html): Migration guides for Spark components * [Building Spark](building-spark.html): build Spark using the Maven system * [Contributing to Spark](https://spark.apache.org/contributing.html) * [Third Party Projects](https://spark.apache.org/third-party-projects.html): related third party Spark projects diff --git a/docs/migration-guide.md b/docs/migration-guide.md new file mode 100644 index 0000000000000..9ca0ada37a2fe --- /dev/null +++ b/docs/migration-guide.md @@ -0,0 +1,30 @@ +--- +layout: global +title: Migration Guide +displayTitle: Migration Guide +license: | + 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. +--- + +This page documents sections of the migration guide for each component in order +for users to migrate effectively. + +* [Spark Core](core-migration-guide.html) +* [SQL, Datasets, and DataFrame](sql-migration-guide.html) +* [Structured Streaming](ss-migration-guide.html) +* [MLlib (Machine Learning)](ml-migration-guide.html) +* [PySpark (Python on Spark)](pyspark-migration-guide.html) +* [SparkR (R on Spark)](sparkr-migration-guide.html) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 4661d6cd87c04..7b4fa4f651e64 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -113,68 +113,7 @@ transforming multiple columns. * Robust linear regression with Huber loss ([SPARK-3181](https://issues.apache.org/jira/browse/SPARK-3181)). -# Migration guide +# Migration Guide -MLlib is under active development. -The APIs marked `Experimental`/`DeveloperApi` may change in future releases, -and the migration guide below will explain all changes between releases. +The migration guide is now archived [on this page](ml-migration-guide.html). -## From 2.4 to 3.0 - -### Breaking changes - -* `OneHotEncoder` which is deprecated in 2.3, is removed in 3.0 and `OneHotEncoderEstimator` is now renamed to `OneHotEncoder`. - -### Changes of behavior - -* [SPARK-11215](https://issues.apache.org/jira/browse/SPARK-11215): - In Spark 2.4 and previous versions, when specifying `frequencyDesc` or `frequencyAsc` as - `stringOrderType` param in `StringIndexer`, in case of equal frequency, the order of - strings is undefined. Since Spark 3.0, the strings with equal frequency are further - sorted by alphabet. And since Spark 3.0, `StringIndexer` supports encoding multiple - columns. - -## From 2.2 to 2.3 - -### Breaking changes - -* The class and trait hierarchy for logistic regression model summaries was changed to be cleaner -and better accommodate the addition of the multi-class summary. This is a breaking change for user -code that casts a `LogisticRegressionTrainingSummary` to a -`BinaryLogisticRegressionTrainingSummary`. Users should instead use the `model.binarySummary` -method. See [SPARK-17139](https://issues.apache.org/jira/browse/SPARK-17139) for more detail -(_note_ this is an `Experimental` API). This _does not_ affect the Python `summary` method, which -will still work correctly for both multinomial and binary cases. - -### Deprecations and changes of behavior - -**Deprecations** - -* `OneHotEncoder` has been deprecated and will be removed in `3.0`. It has been replaced by the -new [`OneHotEncoderEstimator`](ml-features.html#onehotencoderestimator) -(see [SPARK-13030](https://issues.apache.org/jira/browse/SPARK-13030)). **Note** that -`OneHotEncoderEstimator` will be renamed to `OneHotEncoder` in `3.0` (but -`OneHotEncoderEstimator` will be kept as an alias). - -**Changes of behavior** - -* [SPARK-21027](https://issues.apache.org/jira/browse/SPARK-21027): - The default parallelism used in `OneVsRest` is now set to 1 (i.e. serial). In `2.2` and - earlier versions, the level of parallelism was set to the default threadpool size in Scala. -* [SPARK-22156](https://issues.apache.org/jira/browse/SPARK-22156): - The learning rate update for `Word2Vec` was incorrect when `numIterations` was set greater than - `1`. This will cause training results to be different between `2.3` and earlier versions. -* [SPARK-21681](https://issues.apache.org/jira/browse/SPARK-21681): - Fixed an edge case bug in multinomial logistic regression that resulted in incorrect coefficients - when some features had zero variance. -* [SPARK-16957](https://issues.apache.org/jira/browse/SPARK-16957): - Tree algorithms now use mid-points for split values. This may change results from model training. -* [SPARK-14657](https://issues.apache.org/jira/browse/SPARK-14657): - Fixed an issue where the features generated by `RFormula` without an intercept were inconsistent - with the output in R. This may change results from model training in this scenario. - -## Previous Spark versions - -Earlier migration guides are archived [on this page](ml-migration-guides.html). - ---- diff --git a/docs/ml-migration-guides.md b/docs/ml-migration-guide.md similarity index 85% rename from docs/ml-migration-guides.md rename to docs/ml-migration-guide.md index 99edd9bd69efa..9e8cd3e07b1ee 100644 --- a/docs/ml-migration-guides.md +++ b/docs/ml-migration-guide.md @@ -1,8 +1,7 @@ --- layout: global -title: Old Migration Guides - MLlib -displayTitle: Old Migration Guides - MLlib -description: MLlib migration guides from before Spark SPARK_VERSION_SHORT +title: "Migration Guide: MLlib (Machine Learning)" +displayTitle: "Migration Guide: MLlib (Machine Learning)" license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -20,15 +19,80 @@ license: | limitations under the License. --- -The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). +* Table of contents +{:toc} -## From 2.1 to 2.2 +Note that this migration guide describes the items specific to MLlib. +Many items of SQL migration can be applied when migrating MLlib to higher versions for DataFrame-based APIs. +Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). + +## Upgrading from MLlib 2.4 to 3.0 + +### Breaking changes +{:.no_toc} + +* `OneHotEncoder` which is deprecated in 2.3, is removed in 3.0 and `OneHotEncoderEstimator` is now renamed to `OneHotEncoder`. + +### Changes of behavior +{:.no_toc} + +* [SPARK-11215](https://issues.apache.org/jira/browse/SPARK-11215): + In Spark 2.4 and previous versions, when specifying `frequencyDesc` or `frequencyAsc` as + `stringOrderType` param in `StringIndexer`, in case of equal frequency, the order of + strings is undefined. Since Spark 3.0, the strings with equal frequency are further + sorted by alphabet. And since Spark 3.0, `StringIndexer` supports encoding multiple + columns. + +## Upgrading from MLlib 2.2 to 2.3 + +### Breaking changes +{:.no_toc} + +* The class and trait hierarchy for logistic regression model summaries was changed to be cleaner +and better accommodate the addition of the multi-class summary. This is a breaking change for user +code that casts a `LogisticRegressionTrainingSummary` to a +`BinaryLogisticRegressionTrainingSummary`. Users should instead use the `model.binarySummary` +method. See [SPARK-17139](https://issues.apache.org/jira/browse/SPARK-17139) for more detail +(_note_ this is an `Experimental` API). This _does not_ affect the Python `summary` method, which +will still work correctly for both multinomial and binary cases. + +### Deprecations and changes of behavior +{:.no_toc} + +**Deprecations** + +* `OneHotEncoder` has been deprecated and will be removed in `3.0`. It has been replaced by the +new [`OneHotEncoderEstimator`](ml-features.html#onehotencoderestimator) +(see [SPARK-13030](https://issues.apache.org/jira/browse/SPARK-13030)). **Note** that +`OneHotEncoderEstimator` will be renamed to `OneHotEncoder` in `3.0` (but +`OneHotEncoderEstimator` will be kept as an alias). + +**Changes of behavior** + +* [SPARK-21027](https://issues.apache.org/jira/browse/SPARK-21027): + The default parallelism used in `OneVsRest` is now set to 1 (i.e. serial). In `2.2` and + earlier versions, the level of parallelism was set to the default threadpool size in Scala. +* [SPARK-22156](https://issues.apache.org/jira/browse/SPARK-22156): + The learning rate update for `Word2Vec` was incorrect when `numIterations` was set greater than + `1`. This will cause training results to be different between `2.3` and earlier versions. +* [SPARK-21681](https://issues.apache.org/jira/browse/SPARK-21681): + Fixed an edge case bug in multinomial logistic regression that resulted in incorrect coefficients + when some features had zero variance. +* [SPARK-16957](https://issues.apache.org/jira/browse/SPARK-16957): + Tree algorithms now use mid-points for split values. This may change results from model training. +* [SPARK-14657](https://issues.apache.org/jira/browse/SPARK-14657): + Fixed an issue where the features generated by `RFormula` without an intercept were inconsistent + with the output in R. This may change results from model training in this scenario. + +## Upgrading from MLlib 2.1 to 2.2 ### Breaking changes +{:.no_toc} There are no breaking changes. ### Deprecations and changes of behavior +{:.no_toc} **Deprecations** @@ -45,9 +109,10 @@ There are no deprecations. `StringIndexer` now handles `NULL` values in the same way as unseen values. Previously an exception would always be thrown regardless of the setting of the `handleInvalid` parameter. -## From 2.0 to 2.1 +## Upgrading from MLlib 2.0 to 2.1 ### Breaking changes +{:.no_toc} **Deprecated methods removed** @@ -59,6 +124,7 @@ There are no deprecations. * `validateParams` in `Evaluator` ### Deprecations and changes of behavior +{:.no_toc} **Deprecations** @@ -74,9 +140,10 @@ There are no deprecations. * [SPARK-17389](https://issues.apache.org/jira/browse/SPARK-17389): `KMeans` reduces the default number of steps from 5 to 2 for the k-means|| initialization mode. -## From 1.6 to 2.0 +## Upgrading from MLlib 1.6 to 2.0 ### Breaking changes +{:.no_toc} There were several breaking changes in Spark 2.0, which are outlined below. @@ -171,6 +238,7 @@ Several deprecated methods were removed in the `spark.mllib` and `spark.ml` pack A full list of breaking changes can be found at [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810). ### Deprecations and changes of behavior +{:.no_toc} **Deprecations** @@ -221,7 +289,7 @@ Changes of behavior in the `spark.mllib` and `spark.ml` packages include: `QuantileDiscretizer` now uses `spark.sql.DataFrameStatFunctions.approxQuantile` to find splits (previously used custom sampling logic). The output buckets will differ for same input data and params. -## From 1.5 to 1.6 +## Upgrading from MLlib 1.5 to 1.6 There are no breaking API changes in the `spark.mllib` or `spark.ml` packages, but there are deprecations and changes of behavior. @@ -248,7 +316,7 @@ Changes of behavior: tokenizing. Now, it converts to lowercase by default, with an option not to. This matches the behavior of the simpler `Tokenizer` transformer. -## From 1.4 to 1.5 +## Upgrading from MLlib 1.4 to 1.5 In the `spark.mllib` package, there are no breaking API changes but several behavior changes: @@ -267,7 +335,7 @@ In the `spark.ml` package, there exists one breaking API change and one behavior * [SPARK-10097](https://issues.apache.org/jira/browse/SPARK-10097): `Evaluator.isLargerBetter` is added to indicate metric ordering. Metrics like RMSE no longer flip signs as in 1.4. -## From 1.3 to 1.4 +## Upgrading from MLlib 1.3 to 1.4 In the `spark.mllib` package, there were several breaking changes, but all in `DeveloperApi` or `Experimental` APIs: @@ -286,7 +354,7 @@ Since the `spark.ml` API was an alpha component in Spark 1.3, we do not list all However, since 1.4 `spark.ml` is no longer an alpha component, we will provide details on any API changes for future releases. -## From 1.2 to 1.3 +## Upgrading from MLlib 1.2 to 1.3 In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental. @@ -313,7 +381,7 @@ Other changes were in `LogisticRegression`: * The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). * In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. -## From 1.1 to 1.2 +## Upgrading from MLlib 1.1 to 1.2 The only API changes in MLlib v1.2 are in [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), @@ -339,7 +407,7 @@ The tree `Node` now includes more information, including the probability of the Examples in the Spark distribution and examples in the [Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. -## From 1.0 to 1.1 +## Upgrading from MLlib 1.0 to 1.1 The only API changes in MLlib v1.1 are in [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), @@ -365,7 +433,7 @@ simple `String` types. Examples of the new recommended `trainClassifier` and `trainRegressor` are given in the [Decision Trees Guide](mllib-decision-tree.html#examples). -## From 0.9 to 1.0 +## Upgrading from MLlib 0.9 to 1.0 In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few breaking changes. If your data is sparse, please store it in a sparse format instead of dense to diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md new file mode 100644 index 0000000000000..889941c37bf43 --- /dev/null +++ b/docs/pyspark-migration-guide.md @@ -0,0 +1,120 @@ +--- +layout: global +title: "Migration Guide: PySpark (Python on Spark)" +displayTitle: "Migration Guide: PySpark (Python on Spark)" +license: | + 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. +--- + +* Table of contents +{:toc} + +Note that this migration guide describes the items specific to PySpark. +Many items of SQL migration can be applied when migrating PySpark to higher versions. +Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). + +## Upgrading from PySpark 2.4 to 3.0 + + - Since Spark 3.0, PySpark requires a Pandas version of 0.23.2 or higher to use Pandas related functionality, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. + + - Since Spark 3.0, PySpark requires a PyArrow version of 0.12.1 or higher to use PyArrow related functionality, such as `pandas_udf`, `toPandas` and `createDataFrame` with "spark.sql.execution.arrow.enabled=true", etc. + + - In PySpark, when creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 3.0, the builder comes to not update the configurations. This is the same behavior as Java/Scala API in 2.3 and above. If you want to update them, you need to update them prior to creating a `SparkSession`. + + - In PySpark, when Arrow optimization is enabled, if Arrow version is higher than 0.11.0, Arrow can perform safe type conversion when converting Pandas.Series to Arrow array during serialization. Arrow will raise errors when detecting unsafe type conversion like overflow. Setting `spark.sql.execution.pandas.arrowSafeTypeConversion` to true can enable it. The default setting is false. PySpark's behavior for Arrow versions is illustrated in the table below: + + + + + + + + + + + + + + + + + + + + + +
    + PyArrow version + + Integer Overflow + + Floating Point Truncation +
    + version < 0.11.0 + + Raise error + + Silently allows +
    + version > 0.11.0, arrowSafeTypeConversion=false + + Silent overflow + + Silently allows +
    + version > 0.11.0, arrowSafeTypeConversion=true + + Raise error + + Raise error +
    + + - Since Spark 3.0, `createDataFrame(..., verifySchema=True)` validates `LongType` as well in PySpark. Previously, `LongType` was not verified and resulted in `None` in case the value overflows. To restore this behavior, `verifySchema` can be set to `False` to disable the validation. + +## Upgrading from PySpark 2.3 to 2.4 + + - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. + +## Upgrading from PySpark 2.3.0 to 2.3.1 and above + + - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. + +## Upgrading from PySpark 2.2 to 2.3 + + - 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. + + - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. + + - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. + +## Upgrading from PySpark 1.4 to 1.5 + + - Resolution of strings to columns in Python now supports using dots (`.`) to qualify the column or + access nested values. For example `df['table.column.nestedField']`. However, this means that if + your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + + - DataFrame.withColumn method in PySpark supports adding a new column or replacing existing columns of the same name. + + +## Upgrading from PySpark 1.0-1.2 to 1.3 + +#### Python DataTypes No Longer Singletons +{:.no_toc} + +When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of +referencing a singleton. diff --git a/docs/sparkr-migration-guide.md b/docs/sparkr-migration-guide.md new file mode 100644 index 0000000000000..6fbc4c03aefc1 --- /dev/null +++ b/docs/sparkr-migration-guide.md @@ -0,0 +1,77 @@ +--- +layout: global +title: "Migration Guide: SparkR (R on Spark)" +displayTitle: "Migration Guide: SparkR (R on Spark)" +license: | + 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. +--- + +* Table of contents +{:toc} + +Note that this migration guide describes the items specific to SparkR. +Many items of SQL migration can be applied when migrating SparkR to higher versions. +Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). + +## Upgrading from SparkR 2.4 to 3.0 + + - The deprecated methods `sparkR.init`, `sparkRSQL.init`, `sparkRHive.init` have been removed. Use `sparkR.session` instead. + - The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `registerTempTable`, `createExternalTable`, and `dropTempTable` have been removed. Use `read.parquet`, `write.parquet`, `read.json`, `createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead. + +## Upgrading from SparkR 2.3 to 2.4 + + - Previously, we don't check the validity of the size of the last layer in `spark.mlp`. For example, if the training data only has two labels, a `layers` param like `c(1, 3)` doesn't cause an error previously, now it does. + +## Upgrading from SparkR 2.3 to 2.3.1 and above + + - In SparkR 2.3.0 and earlier, the `start` parameter of `substr` method was wrongly subtracted by one and considered as 0-based. This can lead to inconsistent substring results and also does not match with the behaviour with `substr` in R. In version 2.3.1 and later, it has been fixed so the `start` parameter of `substr` method is now 1-based. As an example, `substr(lit('abcdef'), 2, 4))` would result to `abc` in SparkR 2.3.0, and the result would be `bcd` in SparkR 2.3.1. + +## Upgrading from SparkR 2.2 to 2.3 + + - The `stringsAsFactors` parameter was previously ignored with `collect`, for example, in `collect(createDataFrame(iris), stringsAsFactors = TRUE))`. It has been corrected. + - For `summary`, option for statistics to compute has been added. Its output is changed from that from `describe`. + - A warning can be raised if versions of SparkR package and the Spark JVM do not match. + +## Upgrading from SparkR 2.1 to 2.2 + + - A `numPartitions` parameter has been added to `createDataFrame` and `as.DataFrame`. When splitting the data, the partition position calculation has been made to match the one in Scala. + - The method `createExternalTable` has been deprecated to be replaced by `createTable`. Either methods can be called to create external or managed table. Additional catalog methods have also been added. + - By default, derby.log is now saved to `tempdir()`. This will be created when instantiating the SparkSession with `enableHiveSupport` set to `TRUE`. + - `spark.lda` was not setting the optimizer correctly. It has been corrected. + - Several model summary outputs are updated to have `coefficients` as `matrix`. This includes `spark.logit`, `spark.kmeans`, `spark.glm`. Model summary outputs for `spark.gaussianMixture` have added log-likelihood as `loglik`. + +## Upgrading from SparkR 2.0 to 3.1 + + - `join` no longer performs Cartesian Product by default, use `crossJoin` instead. + + +## Upgrading from SparkR 1.6 to 2.0 + + - The method `table` has been removed and replaced by `tableToDF`. + - The class `DataFrame` has been renamed to `SparkDataFrame` to avoid name conflicts. + - Spark's `SQLContext` and `HiveContext` have been deprecated to be replaced by `SparkSession`. Instead of `sparkR.init()`, call `sparkR.session()` in its place to instantiate the SparkSession. Once that is done, that currently active SparkSession will be used for SparkDataFrame operations. + - The parameter `sparkExecutorEnv` is not supported by `sparkR.session`. To set environment for the executors, set Spark config properties with the prefix "spark.executorEnv.VAR_NAME", for example, "spark.executorEnv.PATH" + - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable`. + - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. + - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. + - The `sc` SparkContext parameter is no longer required for these functions: `setJobGroup`, `clearJobGroup`, `cancelJobGroup` + +## Upgrading from SparkR 1.5 to 1.6 + + - Before Spark 1.6.0, the default mode for writes was `append`. It was changed in Spark 1.6.0 to `error` to match the Scala API. + - SparkSQL converts `NA` in R to `null` and vice-versa. + - Since 1.6.1, withColumn method in SparkR supports adding a new column to or replacing existing columns + of the same name of a DataFrame. diff --git a/docs/sparkr.md b/docs/sparkr.md index 7431d025aa629..c36d1a23d0d2e 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -748,49 +748,5 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma # Migration Guide -## Upgrading From SparkR 1.5.x to 1.6.x +The migration guide is now archived [on this page](sparkr-migration-guide.html). - - Before Spark 1.6.0, the default mode for writes was `append`. It was changed in Spark 1.6.0 to `error` to match the Scala API. - - SparkSQL converts `NA` in R to `null` and vice-versa. - -## Upgrading From SparkR 1.6.x to 2.0 - - - The method `table` has been removed and replaced by `tableToDF`. - - The class `DataFrame` has been renamed to `SparkDataFrame` to avoid name conflicts. - - Spark's `SQLContext` and `HiveContext` have been deprecated to be replaced by `SparkSession`. Instead of `sparkR.init()`, call `sparkR.session()` in its place to instantiate the SparkSession. Once that is done, that currently active SparkSession will be used for SparkDataFrame operations. - - The parameter `sparkExecutorEnv` is not supported by `sparkR.session`. To set environment for the executors, set Spark config properties with the prefix "spark.executorEnv.VAR_NAME", for example, "spark.executorEnv.PATH" - - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable`. - - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. - - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. - - The `sc` SparkContext parameter is no longer required for these functions: `setJobGroup`, `clearJobGroup`, `cancelJobGroup` - -## Upgrading to SparkR 2.1.0 - - - `join` no longer performs Cartesian Product by default, use `crossJoin` instead. - -## Upgrading to SparkR 2.2.0 - - - A `numPartitions` parameter has been added to `createDataFrame` and `as.DataFrame`. When splitting the data, the partition position calculation has been made to match the one in Scala. - - The method `createExternalTable` has been deprecated to be replaced by `createTable`. Either methods can be called to create external or managed table. Additional catalog methods have also been added. - - By default, derby.log is now saved to `tempdir()`. This will be created when instantiating the SparkSession with `enableHiveSupport` set to `TRUE`. - - `spark.lda` was not setting the optimizer correctly. It has been corrected. - - Several model summary outputs are updated to have `coefficients` as `matrix`. This includes `spark.logit`, `spark.kmeans`, `spark.glm`. Model summary outputs for `spark.gaussianMixture` have added log-likelihood as `loglik`. - -## Upgrading to SparkR 2.3.0 - - - The `stringsAsFactors` parameter was previously ignored with `collect`, for example, in `collect(createDataFrame(iris), stringsAsFactors = TRUE))`. It has been corrected. - - For `summary`, option for statistics to compute has been added. Its output is changed from that from `describe`. - - A warning can be raised if versions of SparkR package and the Spark JVM do not match. - -## Upgrading to SparkR 2.3.1 and above - - - In SparkR 2.3.0 and earlier, the `start` parameter of `substr` method was wrongly subtracted by one and considered as 0-based. This can lead to inconsistent substring results and also does not match with the behaviour with `substr` in R. In version 2.3.1 and later, it has been fixed so the `start` parameter of `substr` method is now 1-based. As an example, `substr(lit('abcdef'), 2, 4))` would result to `abc` in SparkR 2.3.0, and the result would be `bcd` in SparkR 2.3.1. - -## Upgrading to SparkR 2.4.0 - - - Previously, we don't check the validity of the size of the last layer in `spark.mlp`. For example, if the training data only has two labels, a `layers` param like `c(1, 3)` doesn't cause an error previously, now it does. - -## Upgrading to SparkR 3.0.0 - - - The deprecated methods `sparkR.init`, `sparkRSQL.init`, `sparkRHive.init` have been removed. Use `sparkR.session` instead. - - The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `registerTempTable`, `createExternalTable`, and `dropTempTable` have been removed. Use `read.parquet`, `write.parquet`, `read.json`, `createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead. diff --git a/docs/sql-migration-guide-hive-compatibility.md b/docs/sql-migration-guide-hive-compatibility.md deleted file mode 100644 index d4b4fdf19d926..0000000000000 --- a/docs/sql-migration-guide-hive-compatibility.md +++ /dev/null @@ -1,167 +0,0 @@ ---- -layout: global -title: Compatibility with Apache Hive -displayTitle: Compatibility with Apache Hive -license: | - 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. ---- - -* Table of contents -{:toc} - -Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. -Currently, Hive SerDes and UDFs are based on Hive 1.2.1, -and Spark SQL can be connected to different versions of Hive Metastore -(from 0.12.0 to 2.3.6 and 3.0.0 to 3.1.2. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)). - -#### Deploying in Existing Hive Warehouses - -The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive -installations. You do not need to modify your existing Hive Metastore or change the data placement -or partitioning of your tables. - -### Supported Hive Features - -Spark SQL supports the vast majority of Hive features, such as: - -* Hive query statements, including: - * `SELECT` - * `GROUP BY` - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` -* All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathematical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) -* User defined functions (UDF) -* User defined aggregation functions (UDAF) -* User defined serialization formats (SerDes) -* Window functions -* Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` -* Unions -* Sub-queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` -* Sampling -* Explain -* Partitioned tables including dynamic partition insertion -* View - * If column aliases are not specified in view definition queries, both Spark and Hive will - generate alias names, but in different ways. In order for Spark to be able to read views created - by Hive, users should explicitly specify column aliases in view definition queries. As an - example, Spark cannot read `v1` created as below by Hive. - - ``` - CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 FROM (SELECT 1 c) t1) t2; - ``` - - Instead, you should create `v1` as below with column aliases explicitly specified. - - ``` - CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 AS inc_c FROM (SELECT 1 c) t1) t2; - ``` - -* All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` -* Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `DATE` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` - -### Unsupported Hive Functionality - -Below is a list of Hive features that we don't support yet. Most of these features are rarely used -in Hive deployments. - -**Major Hive Features** - -* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL - doesn't support buckets yet. - - -**Esoteric Hive Features** - -* `UNION` type -* Unique join -* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment and only supports populating the sizeInBytes field of the hive metastore. - -**Hive Input/Output Formats** - -* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. -* Hadoop archive - -**Hive Optimizations** - -A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -less important due to Spark SQL's in-memory computational model. Others are slotted for future -releases of Spark SQL. - -* Block-level bitmap indexes and virtual columns (used to build indexes) -* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you - need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". -* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still - launches tasks to compute the result. -* Skew data flag: Spark SQL does not follow the skew data flags in Hive. -* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. -* Merge multiple small files for query results: if the result output contains multiple small files, - Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS - metadata. Spark SQL does not support that. - -**Hive UDF/UDTF/UDAF** - -Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: - -* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically - include additional resources required by this UDF. -* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses - a deprecated interface `initialize(ObjectInspector[])` only. -* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize - functions with `MapredContext`, which is inapplicable to Spark. -* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. - Spark SQL does not call this function when tasks finish. -* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. - Spark SQL currently does not support the reuse of aggregation. -* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating - an aggregate over a fixed window. - -### Incompatible Hive UDF - -Below are the scenarios in which Hive and Spark generate different results: - -* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. -* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. -* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md deleted file mode 100644 index 5add2fb93937d..0000000000000 --- a/docs/sql-migration-guide-upgrade.md +++ /dev/null @@ -1,834 +0,0 @@ ---- -layout: global -title: Spark SQL Upgrading Guide -displayTitle: Spark SQL Upgrading Guide -license: | - 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. ---- - -* Table of contents -{:toc} - -## Upgrading From Spark SQL 2.4 to 3.0 - - - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. - - - In Spark 3.0, the deprecated `HiveContext` class has been removed. Use `SparkSession.builder.enableHiveSupport()` instead. - - - Since Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join. - - - Since Spark 3.0, we reversed argument order of the trim function from `TRIM(trimStr, str)` to `TRIM(str, trimStr)` to be compatible with other databases. - - - Since Spark 3.0, PySpark requires a Pandas version of 0.23.2 or higher to use Pandas related functionality, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. - - - Since Spark 3.0, PySpark requires a PyArrow version of 0.12.1 or higher to use PyArrow related functionality, such as `pandas_udf`, `toPandas` and `createDataFrame` with "spark.sql.execution.arrow.enabled=true", etc. - - - In Spark version 2.4 and earlier, SQL queries such as `FROM ` or `FROM
    UNION ALL FROM
    ` are supported by accident. In hive-style `FROM
    SELECT `, the `SELECT` clause is not negligible. Neither Hive nor Presto support this syntax. Therefore we will treat these queries as invalid since Spark 3.0. - - - Since Spark 3.0, the Dataset and DataFrame API `unionAll` is not deprecated any more. It is an alias for `union`. - - - In PySpark, when creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 3.0, the builder comes to not update the configurations. This is the same behavior as Java/Scala API in 2.3 and above. If you want to update them, you need to update them prior to creating a `SparkSession`. - - - In Spark version 2.4 and earlier, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType` and `DoubleType`, it fails on empty strings and throws exceptions. Since Spark 3.0, we disallow empty strings and will throw exceptions for data types except for `StringType` and `BinaryType`. - - - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. - - - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. - - - In Spark version 2.4 and earlier, users can create map values with map type key via built-in function like `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can still read map values with map type key from data source or Java/Scala collections, though they are not very useful. - - - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a grouped dataset with key attribute wrongly named as "value", if the key is non-struct type, e.g. int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries weird. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behaviour is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`. - - - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal to 0.0, but -0.0 and 0.0 are considered as different values when used in aggregate grouping keys, window partition keys and join keys. Since Spark 3.0, this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and earlier. - - - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be undefined. - - - In Spark version 2.4 and earlier, partition column value is converted as null if it can't be casted to corresponding user provided schema. Since 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. - - - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. - - - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. - - - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. - - - Refreshing a cached table would trigger a table uncache operation and then a table cache (lazily) operation. In Spark version 2.4 and earlier, the cache name and storage level are not preserved before the uncache operation. Therefore, the cache name and storage level could be changed unexpectedly. Since Spark 3.0, cache name and storage level will be first preserved for cache recreation. It helps to maintain a consistent cache behavior upon table refreshing. - - - Since Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. - - - In PySpark, when Arrow optimization is enabled, if Arrow version is higher than 0.11.0, Arrow can perform safe type conversion when converting Pandas.Series to Arrow array during serialization. Arrow will raise errors when detecting unsafe type conversion like overflow. Setting `spark.sql.execution.pandas.arrowSafeTypeConversion` to true can enable it. The default setting is false. PySpark's behavior for Arrow versions is illustrated in the table below: -
    - - - - - - - - - - - - - - - - - - - - -
    - PyArrow version - - Integer Overflow - - Floating Point Truncation -
    - version < 0.11.0 - - Raise error - - Silently allows -
    - version > 0.11.0, arrowSafeTypeConversion=false - - Silent overflow - - Silently allows -
    - version > 0.11.0, arrowSafeTypeConversion=true - - Raise error - - Raise error -
    - - - In Spark version 2.4 and earlier, if `org.apache.spark.sql.functions.udf(Any, DataType)` gets a Scala closure with primitive-type argument, the returned UDF will return null if the input values is null. Since Spark 3.0, the UDF will return the default value of the Java type if the input value is null. For example, `val f = udf((x: Int) => x, IntegerType)`, `f($"x")` will return null in Spark 2.4 and earlier if column `x` is null, and return 0 in Spark 3.0. This behavior change is introduced because Spark 3.0 is built with Scala 2.12 by default. - - - Since Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and etc. Spark 3.0 uses Java 8 API classes from the java.time packages that based on ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and earlier, those operations are performed by using the hybrid calendar (Julian + Gregorian, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: - - - CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. - - - The `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. - - - The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. - - - the JDBC options `lowerBound` and `upperBound` are converted to TimestampType/DateType values in the same way as casting strings to TimestampType/DateType values. The conversion is based on Proleptic Gregorian calendar, and time zone defined by the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone. - - - Formatting of `TIMESTAMP` and `DATE` literals. - - - In Spark version 2.4 and earlier, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. Since Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`. - - - In Spark version 2.4 and earlier, the `current_timestamp` function returns a timestamp with millisecond resolution only. Since Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution. - - - In Spark version 2.4 and earlier, when reading a Hive Serde table with Spark native data sources(parquet/orc), Spark will infer the actual file schema and update the table schema in metastore. Since Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, please set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`. - - - Since Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion uses the default time zone of the Java virtual machine. - - - In Spark version 2.4, when a spark session is created via `cloneSession()`, the newly created spark session inherits its configuration from its parent `SparkContext` even though the same configuration may exist with a different value in its parent spark session. Since Spark 3.0, the configurations of a parent `SparkSession` have a higher precedence over the parent `SparkContext`. The old behavior can be restored by setting `spark.sql.legacy.sessionInitWithConfigDefaults` to `true`. - - - Since Spark 3.0, parquet logical type `TIMESTAMP_MICROS` is used by default while saving `TIMESTAMP` columns. In Spark version 2.4 and earlier, `TIMESTAMP` columns are saved as `INT96` in parquet files. To set `INT96` to `spark.sql.parquet.outputTimestampType` restores the previous behavior. - - - Since Spark 3.0, if `hive.default.fileformat` is not found in `Spark SQL configuration` then it will fallback to hive-site.xml present in the `Hadoop configuration` of `SparkContext`. - - - Since Spark 3.0, Spark will cast `String` to `Date/TimeStamp` in binary comparisons with dates/timestamps. The previous behaviour of casting `Date/Timestamp` to `String` can be restored by setting `spark.sql.legacy.typeCoercion.datetimeToString` to `true`. - - - Since Spark 3.0, when Avro files are written with user provided schema, the fields will be matched by field names between catalyst schema and avro schema instead of positions. - - - Since Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark will throw runtime NPE if any of the records contains null. - - - Since Spark 3.0, we use a new protocol for fetching shuffle blocks, for external shuffle service users, we need to upgrade the server correspondingly. Otherwise, we'll get the error message `UnsupportedOperationException: Unexpected message: FetchShuffleBlocks`. If it is hard to upgrade the shuffle service right now, you can still use the old protocol by setting `spark.shuffle.useOldFetchProtocol` to `true`. - - - Since Spark 3.0, a higher-order function `exists` follows the three-valued boolean logic, i.e., if the `predicate` returns any `null`s and no `true` is obtained, then `exists` will return `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` will be `null`. The previous behaviour can be restored by setting `spark.sql.legacy.arrayExistsFollowsThreeValuedLogic` to `false`. - - - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. - - - Since Spark 3.0, `createDataFrame(..., verifySchema=True)` validates `LongType` as well in PySpark. Previously, `LongType` was not verified and resulted in `None` in case the value overflows. To restore this behavior, `verifySchema` can be set to `False` to disable the validation. - - - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. - - - Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. - - - Since Spark 3.0, 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and earlier, 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases. - - - The result of `java.lang.Math`'s `log`, `log1p`, `exp`, `expm1`, and `pow` may vary across platforms. In Spark 3.0, the result of the equivalent SQL functions (including related SQL functions like `LOG10`) return values consistent with `java.lang.StrictMath`. In virtually all cases this makes no difference in the return value, and the difference is very small, but may not exactly match `java.lang.Math` on x86 platforms in cases like, for example, `log(3.0)`, whose value varies between `Math.log()` and `StrictMath.log()`. - - - Since Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`. - - - Since Spark 3.0, `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in case insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behaviour change is illustrated in the table below: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Operation - - Result prior to Spark 3.0 - - Result starting Spark 3.0 -
    - CAST('infinity' AS DOUBLE)
    - CAST('+infinity' AS DOUBLE)
    - CAST('inf' AS DOUBLE)
    - CAST('+inf' AS DOUBLE)
    -
    - NULL - - Double.PositiveInfinity -
    - CAST('-infinity' AS DOUBLE)
    - CAST('-inf' AS DOUBLE)
    -
    - NULL - - Double.NegativeInfinity -
    - CAST('infinity' AS FLOAT)
    - CAST('+infinity' AS FLOAT)
    - CAST('inf' AS FLOAT)
    - CAST('+inf' AS FLOAT)
    -
    - NULL - - Float.PositiveInfinity -
    - CAST('-infinity' AS FLOAT)
    - CAST('-inf' AS FLOAT)
    -
    - NULL - - Float.NegativeInfinity -
    - CAST('nan' AS DOUBLE) - - NULL - - Double.NaN -
    - CAST('nan' AS FLOAT) - - NULL - - Float.NaN -
    - -## Upgrading from Spark SQL 2.4 to 2.4.1 - - - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was - inconsistently interpreted as both seconds and milliseconds in Spark 2.4.0 in different parts of the code. - Unitless values are now consistently interpreted as milliseconds. Applications that set values like "30" - need to specify a value with units like "30s" now, to avoid being interpreted as milliseconds; otherwise, - the extremely short interval that results will likely cause applications to fail. - - - When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution. In Spark 3.0, the up cast is stricter and turning String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` will fail during analysis. - -## Upgrading From Spark SQL 2.3 to 2.4 - - - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Query - - Spark 2.3 or Prior - - Spark 2.4 - - Remarks -
    - SELECT array_contains(array(1), 1.34D); - - true - - false - - In Spark 2.4, left and right parameters are promoted to array type of double type and double type respectively. -
    - SELECT array_contains(array(1), '1'); - - true - - AnalysisException is thrown. - - Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. -
    - SELECT array_contains(array(1), 'anystring'); - - null - - AnalysisException is thrown. - - Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. -
    - - - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. - - - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. - - - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. - - - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. - - - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. ORC files created by native ORC writer cannot be read by some old Apache Hive releases. Use `spark.sql.orc.impl=hive` to create the files shared with Hive 2.1.1 and older. - - - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. - - - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. - - - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. - - - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. - - - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. - - - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. - - - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - - - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. - - - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. - - - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. - - - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. - - - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. - - - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. - - - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - - - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - - - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. - - - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. - - - In version 2.3 and earlier, when reading from a Parquet data source table, Spark always returns null for any column whose column names in Hive metastore schema and Parquet schema are in different letter cases, no matter whether `spark.sql.caseSensitive` is set to `true` or `false`. Since 2.4, when `spark.sql.caseSensitive` is set to `false`, Spark does case insensitive column name resolution between Hive metastore schema and Parquet schema, so even column names are in different letter cases, Spark returns corresponding column values. An exception is thrown if there is ambiguity, i.e. more than one Parquet column is matched. This change also applies to Parquet Hive tables when `spark.sql.hive.convertMetastoreParquet` is set to `true`. - -## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above - - - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. - -## Upgrading From Spark SQL 2.2 to 2.3 - - - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. - - - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. - - - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. - - - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - InputA \ InputB - - NullType - - IntegerType - - LongType - - DecimalType(38,0)* - - DoubleType - - DateType - - TimestampType - - StringType -
    - NullType - NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    - IntegerType - IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    - LongType - LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DecimalType(38,0)* - DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DoubleType - DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    - DateType - DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    - TimestampType - TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    - StringType - StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    - - 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. - - - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. - - - 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](sql-performance-tuning.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). - - - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. - - - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. - - - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes - - - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). - - - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. - - - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. - - - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. - - - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. - - - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. - -## Upgrading From Spark SQL 2.1 to 2.2 - - - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. - - - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). - - - Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions. - -## Upgrading From Spark SQL 2.0 to 2.1 - - - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. - - - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. - - - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. - - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. - - - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. - - - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. - -## Upgrading From Spark SQL 1.6 to 2.0 - - - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and - - `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. - - - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for - `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed - transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., - `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in - Python and R is not a language feature, the concept of Dataset does not apply to these languages’ - APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the - single-node data frame notion in these languages. - - - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` - - - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` - - - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` - - - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. - - - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` - in order to prevent accidental dropping the existing data in the user-provided locations. - That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. - Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. - Note that this is different from the Hive behavior. - - - As a result, `DROP TABLE` statements on those tables will not remove the data. - - - `spark.sql.parquet.cacheMetadata` is no longer used. - See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. - -## Upgrading From Spark SQL 1.5 to 1.6 - - - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC - connection owns a copy of their own SQL configuration and temporary function registry. Cached - tables are still shared though. If you prefer to run the Thrift server in the old single-session - mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add - this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: - - {% highlight bash %} - ./sbin/start-thriftserver.sh \ - --conf spark.sql.hive.thriftServer.singleSession=true \ - ... - {% endhighlight %} - - - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns - of the same name of a DataFrame. - - - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This - change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType - from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for - details. - -## Upgrading From Spark SQL 1.4 to 1.5 - - - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with - code generation for expression evaluation. These features can both be disabled by setting - `spark.sql.tungsten.enabled` to `false`. - - - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting - `spark.sql.parquet.mergeSchema` to `true`. - - - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or - access nested values. For example `df['table.column.nestedField']`. However, this means that if - your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). - - - In-memory columnar storage partition pruning is on by default. It can be disabled by setting - `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. - - - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum - precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now - used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. - - - Timestamps are now stored at a precision of 1us, rather than 1ns - - - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains - unchanged. - - - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). - - - JSON data source will not automatically load new files that are created by other applications - (i.e. files that are not inserted to the dataset through Spark SQL). - For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), - users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method - to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate - the DataFrame and the new DataFrame will include new files. - - - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. - -## Upgrading from Spark SQL 1.3 to 1.4 - -#### DataFrame data reader/writer interface - -Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) -and writing data out (`DataFrame.write`), -and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). - -See the API docs for `SQLContext.read` ( - Scala, - Java, - Python -) and `DataFrame.write` ( - Scala, - Java, - Python -) more information. - - -#### DataFrame.groupBy retains grouping columns - -Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the -grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. - -
    -
    -{% highlight scala %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg($"department", max("age"), sum("expense")) - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")) - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    -{% highlight java %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(col("department"), max("age"), sum("expense")); - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")); - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false"); - -{% endhighlight %} -
    - -
    -{% highlight python %} - -import pyspark.sql.functions as func - -# In 1.3.x, in order for the grouping column "department" to show up, -# it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) - -# In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(func.max("age"), func.sum("expense")) - -# Revert to 1.3.x behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    - - -#### Behavior change on DataFrame.withColumn - -Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added -as a new column with its specified name in the result DataFrame even if there may be any existing -columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different -name from names of all existing columns or replacing existing columns of the same name. - -Note that this change is only for Scala API, not for PySpark and SparkR. - - -## Upgrading from Spark SQL 1.0-1.2 to 1.3 - -In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the -available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other -releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked -as unstable (i.e., DeveloperAPI or Experimental). - -#### Rename of SchemaRDD to DataFrame - -The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has -been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD -directly, but instead provide most of the functionality that RDDs provide though their own -implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. - -In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for -some use cases. It is still recommended that users update their code to use `DataFrame` instead. -Java and Python users will need to update their code. - -#### Unification of the Java and Scala APIs - -Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) -that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users -of either language should use `SQLContext` and `DataFrame`. In general these classes try to -use types that are usable from both languages (i.e. `Array` instead of language-specific collections). -In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading -is used instead. - -Additionally, the Java specific types API has been removed. Users of both Scala and Java should -use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. - - -#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) - -Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought -all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit -conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. -Users should now write `import sqlContext.implicits._`. - -Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., -case classes or tuples) with a method `toDF`, instead of applying automatically. - -When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import -`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: -`import org.apache.spark.sql.functions._`. - -#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) - -Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users -should instead import the classes in `org.apache.spark.sql.types` - -#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) - -Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been -moved into the udf object in `SQLContext`. - -
    -
    -{% highlight scala %} - -sqlContext.udf.register("strLen", (s: String) => s.length()) - -{% endhighlight %} -
    - -
    -{% highlight java %} - -sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); - -{% endhighlight %} -
    - -
    - -Python UDF registration is unchanged. - -#### Python DataTypes No Longer Singletons - -When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of -referencing a singleton. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 4c23147106b65..71e533eef5fe6 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -1,7 +1,7 @@ --- layout: global -title: Migration Guide -displayTitle: Migration Guide +title: "Migration Guide: SQL, Datasets and DataFrame" +displayTitle: "Migration Guide: SQL, Datasets and DataFrame" license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,20 +19,892 @@ license: | limitations under the License. --- -* [Spark SQL Upgrading Guide](sql-migration-guide-upgrade.html) - * [Upgrading From Spark SQL 2.4 to 3.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-24-to-30) - * [Upgrading From Spark SQL 2.3 to 2.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-23-to-24) - * [Upgrading From Spark SQL 2.3.0 to 2.3.1 and above](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-230-to-231-and-above) - * [Upgrading From Spark SQL 2.2 to 2.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-22-to-23) - * [Upgrading From Spark SQL 2.1 to 2.2](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-21-to-22) - * [Upgrading From Spark SQL 2.0 to 2.1](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-20-to-21) - * [Upgrading From Spark SQL 1.6 to 2.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-16-to-20) - * [Upgrading From Spark SQL 1.5 to 1.6](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-15-to-16) - * [Upgrading From Spark SQL 1.4 to 1.5](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-14-to-15) - * [Upgrading from Spark SQL 1.3 to 1.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-13-to-14) - * [Upgrading from Spark SQL 1.0-1.2 to 1.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-10-12-to-13) -* [Compatibility with Apache Hive](sql-migration-guide-hive-compatibility.html) - * [Deploying in Existing Hive Warehouses](sql-migration-guide-hive-compatibility.html#deploying-in-existing-hive-warehouses) - * [Supported Hive Features](sql-migration-guide-hive-compatibility.html#supported-hive-features) - * [Unsupported Hive Functionality](sql-migration-guide-hive-compatibility.html#unsupported-hive-functionality) - * [Incompatible Hive UDF](sql-migration-guide-hive-compatibility.html#incompatible-hive-udf) +* Table of contents +{:toc} + +## Upgrading from Spark SQL 2.4 to 3.0 + + - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. + + - In Spark 3.0, the deprecated `HiveContext` class has been removed. Use `SparkSession.builder.enableHiveSupport()` instead. + + - Since Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join. + + - Since Spark 3.0, we reversed argument order of the trim function from `TRIM(trimStr, str)` to `TRIM(str, trimStr)` to be compatible with other databases. + + - In Spark version 2.4 and earlier, SQL queries such as `FROM ` or `FROM
    UNION ALL FROM
    ` are supported by accident. In hive-style `FROM
    SELECT `, the `SELECT` clause is not negligible. Neither Hive nor Presto support this syntax. Therefore we will treat these queries as invalid since Spark 3.0. + + - Since Spark 3.0, the Dataset and DataFrame API `unionAll` is not deprecated any more. It is an alias for `union`. + + - In Spark version 2.4 and earlier, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType` and `DoubleType`, it fails on empty strings and throws exceptions. Since Spark 3.0, we disallow empty strings and will throw exceptions for data types except for `StringType` and `BinaryType`. + + - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. + + - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. + + - In Spark version 2.4 and earlier, users can create map values with map type key via built-in function like `CreateMap`, `MapFromArrays`, etc. Since Spark 3.0, it's not allowed to create map values with map type key with these built-in functions. Users can still read map values with map type key from data source or Java/Scala collections, though they are not very useful. + + - In Spark version 2.4 and earlier, `Dataset.groupByKey` results to a grouped dataset with key attribute wrongly named as "value", if the key is non-struct type, e.g. int, string, array, etc. This is counterintuitive and makes the schema of aggregation queries weird. For example, the schema of `ds.groupByKey(...).count()` is `(value, count)`. Since Spark 3.0, we name the grouping attribute to "key". The old behaviour is preserved under a newly added configuration `spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue` with a default value of `false`. + + - In Spark version 2.4 and earlier, float/double -0.0 is semantically equal to 0.0, but -0.0 and 0.0 are considered as different values when used in aggregate grouping keys, window partition keys and join keys. Since Spark 3.0, this bug is fixed. For example, `Seq(-0.0, 0.0).toDF("d").groupBy("d").count()` returns `[(0.0, 2)]` in Spark 3.0, and `[(0.0, 1), (-0.0, 1)]` in Spark 2.4 and earlier. + + - In Spark version 2.4 and earlier, users can create a map with duplicated keys via built-in functions like `CreateMap`, `StringToMap`, etc. The behavior of map with duplicated keys is undefined, e.g. map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. Since Spark 3.0, these built-in functions will remove duplicated map keys with last wins policy. Users may still read map values with duplicated keys from data sources which do not enforce it (e.g. Parquet), the behavior will be undefined. + + - In Spark version 2.4 and earlier, partition column value is converted as null if it can't be casted to corresponding user provided schema. Since 3.0, partition column value is validated with user provided schema. An exception is thrown if the validation fails. You can disable such validation by setting `spark.sql.sources.validatePartitionColumns` to `false`. + + - In Spark version 2.4 and earlier, the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. Since 3.0, the command fails if a `SparkConf` key is used. You can disable such a check by setting `spark.sql.legacy.setCommandRejectsSparkCoreConfs` to `false`. + + - In Spark version 2.4 and earlier, CSV datasource converts a malformed CSV string to a row with all `null`s in the PERMISSIVE mode. Since Spark 3.0, the returned row can contain non-`null` fields if some of CSV column values were parsed and converted to desired types successfully. + + - In Spark version 2.4 and earlier, JSON datasource and JSON functions like `from_json` convert a bad JSON record to a row with all `null`s in the PERMISSIVE mode when specified schema is `StructType`. Since Spark 3.0, the returned row can contain non-`null` fields if some of JSON column values were parsed and converted to desired types successfully. + + - Refreshing a cached table would trigger a table uncache operation and then a table cache (lazily) operation. In Spark version 2.4 and earlier, the cache name and storage level are not preserved before the uncache operation. Therefore, the cache name and storage level could be changed unexpectedly. Since Spark 3.0, cache name and storage level will be first preserved for cache recreation. It helps to maintain a consistent cache behavior upon table refreshing. + + - Since Spark 3.0, JSON datasource and JSON function `schema_of_json` infer TimestampType from string values if they match to the pattern defined by the JSON option `timestampFormat`. Set JSON option `inferTimestamp` to `false` to disable such type inferring. + + - In Spark version 2.4 and earlier, if `org.apache.spark.sql.functions.udf(Any, DataType)` gets a Scala closure with primitive-type argument, the returned UDF will return null if the input values is null. Since Spark 3.0, the UDF will return the default value of the Java type if the input value is null. For example, `val f = udf((x: Int) => x, IntegerType)`, `f($"x")` will return null in Spark 2.4 and earlier if column `x` is null, and return 0 in Spark 3.0. This behavior change is introduced because Spark 3.0 is built with Scala 2.12 by default. + + - Since Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and etc. Spark 3.0 uses Java 8 API classes from the java.time packages that based on ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and earlier, those operations are performed by using the hybrid calendar (Julian + Gregorian, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: + + - CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. + + - The `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. + + - The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. + + - the JDBC options `lowerBound` and `upperBound` are converted to TimestampType/DateType values in the same way as casting strings to TimestampType/DateType values. The conversion is based on Proleptic Gregorian calendar, and time zone defined by the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone. + + - Formatting of `TIMESTAMP` and `DATE` literals. + + - In Spark version 2.4 and earlier, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. Since Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`. + + - In Spark version 2.4 and earlier, the `current_timestamp` function returns a timestamp with millisecond resolution only. Since Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution. + + - In Spark version 2.4 and earlier, when reading a Hive Serde table with Spark native data sources(parquet/orc), Spark will infer the actual file schema and update the table schema in metastore. Since Spark 3.0, Spark doesn't infer the schema anymore. This should not cause any problems to end users, but if it does, please set `spark.sql.hive.caseSensitiveInferenceMode` to `INFER_AND_SAVE`. + + - Since Spark 3.0, `TIMESTAMP` literals are converted to strings using the SQL config `spark.sql.session.timeZone`. In Spark version 2.4 and earlier, the conversion uses the default time zone of the Java virtual machine. + + - In Spark version 2.4, when a spark session is created via `cloneSession()`, the newly created spark session inherits its configuration from its parent `SparkContext` even though the same configuration may exist with a different value in its parent spark session. Since Spark 3.0, the configurations of a parent `SparkSession` have a higher precedence over the parent `SparkContext`. The old behavior can be restored by setting `spark.sql.legacy.sessionInitWithConfigDefaults` to `true`. + + - Since Spark 3.0, parquet logical type `TIMESTAMP_MICROS` is used by default while saving `TIMESTAMP` columns. In Spark version 2.4 and earlier, `TIMESTAMP` columns are saved as `INT96` in parquet files. To set `INT96` to `spark.sql.parquet.outputTimestampType` restores the previous behavior. + + - Since Spark 3.0, if `hive.default.fileformat` is not found in `Spark SQL configuration` then it will fallback to hive-site.xml present in the `Hadoop configuration` of `SparkContext`. + + - Since Spark 3.0, Spark will cast `String` to `Date/TimeStamp` in binary comparisons with dates/timestamps. The previous behaviour of casting `Date/Timestamp` to `String` can be restored by setting `spark.sql.legacy.typeCoercion.datetimeToString` to `true`. + + - Since Spark 3.0, when Avro files are written with user provided schema, the fields will be matched by field names between catalyst schema and avro schema instead of positions. + + - Since Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark will throw runtime NPE if any of the records contains null. + + - Since Spark 3.0, we use a new protocol for fetching shuffle blocks, for external shuffle service users, we need to upgrade the server correspondingly. Otherwise, we'll get the error message `UnsupportedOperationException: Unexpected message: FetchShuffleBlocks`. If it is hard to upgrade the shuffle service right now, you can still use the old protocol by setting `spark.shuffle.useOldFetchProtocol` to `true`. + + - Since Spark 3.0, a higher-order function `exists` follows the three-valued boolean logic, i.e., if the `predicate` returns any `null`s and no `true` is obtained, then `exists` will return `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` will be `null`. The previous behaviour can be restored by setting `spark.sql.legacy.arrayExistsFollowsThreeValuedLogic` to `false`. + + - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. + + - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. + + - Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. + + - Since Spark 3.0, 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and earlier, 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases. + + - The result of `java.lang.Math`'s `log`, `log1p`, `exp`, `expm1`, and `pow` may vary across platforms. In Spark 3.0, the result of the equivalent SQL functions (including related SQL functions like `LOG10`) return values consistent with `java.lang.StrictMath`. In virtually all cases this makes no difference in the return value, and the difference is very small, but may not exactly match `java.lang.Math` on x86 platforms in cases like, for example, `log(3.0)`, whose value varies between `Math.log()` and `StrictMath.log()`. + + - Since Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`. + + - Since Spark 3.0, `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in case insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behaviour change is illustrated in the table below: +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Operation + + Result prior to Spark 3.0 + + Result starting Spark 3.0 +
    + CAST('infinity' AS DOUBLE)
    + CAST('+infinity' AS DOUBLE)
    + CAST('inf' AS DOUBLE)
    + CAST('+inf' AS DOUBLE)
    +
    + NULL + + Double.PositiveInfinity +
    + CAST('-infinity' AS DOUBLE)
    + CAST('-inf' AS DOUBLE)
    +
    + NULL + + Double.NegativeInfinity +
    + CAST('infinity' AS FLOAT)
    + CAST('+infinity' AS FLOAT)
    + CAST('inf' AS FLOAT)
    + CAST('+inf' AS FLOAT)
    +
    + NULL + + Float.PositiveInfinity +
    + CAST('-infinity' AS FLOAT)
    + CAST('-inf' AS FLOAT)
    +
    + NULL + + Float.NegativeInfinity +
    + CAST('nan' AS DOUBLE) + + NULL + + Double.NaN +
    + CAST('nan' AS FLOAT) + + NULL + + Float.NaN +
    + +## Upgrading from Spark SQL 2.4 to 2.4.1 + + - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was + inconsistently interpreted as both seconds and milliseconds in Spark 2.4.0 in different parts of the code. + Unitless values are now consistently interpreted as milliseconds. Applications that set values like "30" + need to specify a value with units like "30s" now, to avoid being interpreted as milliseconds; otherwise, + the extremely short interval that results will likely cause applications to fail. + + - When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution. In Spark 3.0, the up cast is stricter and turning String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` will fail during analysis. + +## Upgrading from Spark SQL 2.3 to 2.4 + + - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Query + + Spark 2.3 or Prior + + Spark 2.4 + + Remarks +
    + SELECT array_contains(array(1), 1.34D); + + true + + false + + In Spark 2.4, left and right parameters are promoted to array type of double type and double type respectively. +
    + SELECT array_contains(array(1), '1'); + + true + + AnalysisException is thrown. + + Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. +
    + SELECT array_contains(array(1), 'anystring'); + + null + + AnalysisException is thrown. + + Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. +
    + + - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. + + - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. + + - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. + + - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. + + - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. ORC files created by native ORC writer cannot be read by some old Apache Hive releases. Use `spark.sql.orc.impl=hive` to create the files shared with Hive 2.1.1 and older. + + - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. + + - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. + + - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. + + - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. + + - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. + + - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. + + - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. + + - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. + + - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. + + - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. + + - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. + + - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. + + - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. + + - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. + + - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. + + - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. + + - In version 2.3 and earlier, when reading from a Parquet data source table, Spark always returns null for any column whose column names in Hive metastore schema and Parquet schema are in different letter cases, no matter whether `spark.sql.caseSensitive` is set to `true` or `false`. Since 2.4, when `spark.sql.caseSensitive` is set to `false`, Spark does case insensitive column name resolution between Hive metastore schema and Parquet schema, so even column names are in different letter cases, Spark returns corresponding column values. An exception is thrown if there is ambiguity, i.e. more than one Parquet column is matched. This change also applies to Parquet Hive tables when `spark.sql.hive.convertMetastoreParquet` is set to `true`. + +## Upgrading from Spark SQL 2.2 to 2.3 + + - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. + + - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. + + - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. + + - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + InputA \ InputB + + NullType + + IntegerType + + LongType + + DecimalType(38,0)* + + DoubleType + + DateType + + TimestampType + + StringType +
    + NullType + NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    + IntegerType + IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    + LongType + LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DecimalType(38,0)* + DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DoubleType + DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    + DateType + DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    + TimestampType + TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    + StringType + StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    + + 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. + + - 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](sql-performance-tuning.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). + + - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. + + - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes + + - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). + + - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. + + - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. + + - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. + + - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. + +## Upgrading from Spark SQL 2.1 to 2.2 + + - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. + + - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). + + - Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions. + +## Upgrading from Spark SQL 2.0 to 2.1 + + - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + + - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. + + - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. + - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. + + - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. + + - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. + +## Upgrading from Spark SQL 1.6 to 2.0 + + - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and + + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. + + - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for + `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed + transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., + `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in + Python and R is not a language feature, the concept of Dataset does not apply to these languages’ + APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the + single-node data frame notion in these languages. + + - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` + + - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` + + - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` + + - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. + + - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` + in order to prevent accidental dropping the existing data in the user-provided locations. + That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. + Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. + Note that this is different from the Hive behavior. + + - As a result, `DROP TABLE` statements on those tables will not remove the data. + + - `spark.sql.parquet.cacheMetadata` is no longer used. + See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. + +## Upgrading from Spark SQL 1.5 to 1.6 + + - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC + connection owns a copy of their own SQL configuration and temporary function registry. Cached + tables are still shared though. If you prefer to run the Thrift server in the old single-session + mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add + this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: + + {% highlight bash %} + ./sbin/start-thriftserver.sh \ + --conf spark.sql.hive.thriftServer.singleSession=true \ + ... + {% endhighlight %} + + - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This + change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType + from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for + details. + +## Upgrading from Spark SQL 1.4 to 1.5 + + - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with + code generation for expression evaluation. These features can both be disabled by setting + `spark.sql.tungsten.enabled` to `false`. + + - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting + `spark.sql.parquet.mergeSchema` to `true`. + + - In-memory columnar storage partition pruning is on by default. It can be disabled by setting + `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. + + - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum + precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now + used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. + + - Timestamps are now stored at a precision of 1us, rather than 1ns + + - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains + unchanged. + + - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). + + - JSON data source will not automatically load new files that are created by other applications + (i.e. files that are not inserted to the dataset through Spark SQL). + For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), + users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method + to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate + the DataFrame and the new DataFrame will include new files. + +## Upgrading from Spark SQL 1.3 to 1.4 + +#### DataFrame data reader/writer interface +{:.no_toc} + +Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) +and writing data out (`DataFrame.write`), +and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). + +See the API docs for `SQLContext.read` ( + Scala, + Java, + Python +) and `DataFrame.write` ( + Scala, + Java, + Python +) more information. + + +#### DataFrame.groupBy retains grouping columns +{:.no_toc} + +Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the +grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. + +
    +
    +{% highlight scala %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg($"department", max("age"), sum("expense")) + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")) + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    +{% highlight java %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(col("department"), max("age"), sum("expense")); + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")); + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false"); + +{% endhighlight %} +
    + +
    +{% highlight python %} + +import pyspark.sql.functions as func + +# In 1.3.x, in order for the grouping column "department" to show up, +# it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) + +# In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(func.max("age"), func.sum("expense")) + +# Revert to 1.3.x behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    + + +#### Behavior change on DataFrame.withColumn +{:.no_toc} + +Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added +as a new column with its specified name in the result DataFrame even if there may be any existing +columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different +name from names of all existing columns or replacing existing columns of the same name. + +Note that this change is only for Scala API, not for PySpark and SparkR. + + +## Upgrading from Spark SQL 1.0-1.2 to 1.3 + +In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the +available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other +releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked +as unstable (i.e., DeveloperAPI or Experimental). + +#### Rename of SchemaRDD to DataFrame +{:.no_toc} + +The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has +been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD +directly, but instead provide most of the functionality that RDDs provide though their own +implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. + +In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for +some use cases. It is still recommended that users update their code to use `DataFrame` instead. +Java and Python users will need to update their code. + +#### Unification of the Java and Scala APIs +{:.no_toc} + +Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) +that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users +of either language should use `SQLContext` and `DataFrame`. In general these classes try to +use types that are usable from both languages (i.e. `Array` instead of language-specific collections). +In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading +is used instead. + +Additionally, the Java specific types API has been removed. Users of both Scala and Java should +use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. + + +#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) +{:.no_toc} + +Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought +all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit +conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. +Users should now write `import sqlContext.implicits._`. + +Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., +case classes or tuples) with a method `toDF`, instead of applying automatically. + +When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import +`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: +`import org.apache.spark.sql.functions._`. + +#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) +{:.no_toc} + +Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users +should instead import the classes in `org.apache.spark.sql.types` + +#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) +{:.no_toc} + +Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been +moved into the udf object in `SQLContext`. + +
    +
    +{% highlight scala %} + +sqlContext.udf.register("strLen", (s: String) => s.length()) + +{% endhighlight %} +
    + +
    +{% highlight java %} + +sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); + +{% endhighlight %} +
    + +
    + +Python UDF registration is unchanged. + + + +## Compatibility with Apache Hive + +Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. +Currently, Hive SerDes and UDFs are based on Hive 1.2.1, +and Spark SQL can be connected to different versions of Hive Metastore +(from 0.12.0 to 2.3.6 and 3.0.0 to 3.1.2. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)). + +#### Deploying in Existing Hive Warehouses +{:.no_toc} + +The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +### Supported Hive Features +{:.no_toc} + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY` + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathematical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDes) +* Window functions +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub-queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables including dynamic partition insertion +* View + * If column aliases are not specified in view definition queries, both Spark and Hive will + generate alias names, but in different ways. In order for Spark to be able to read views created + by Hive, users should explicitly specify column aliases in view definition queries. As an + example, Spark cannot read `v1` created as below by Hive. + + ``` + CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 FROM (SELECT 1 c) t1) t2; + ``` + + Instead, you should create `v1` as below with column aliases explicitly specified. + + ``` + CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 AS inc_c FROM (SELECT 1 c) t1) t2; + ``` + +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `DATE` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +### Unsupported Hive Functionality +{:.no_toc} + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + + +**Esoteric Hive Features** + +* `UNION` type +* Unique join +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment and only supports populating the sizeInBytes field of the hive metastore. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +less important due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block-level bitmap indexes and virtual columns (used to build indexes) +* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you + need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". +* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +**Hive UDF/UDTF/UDAF** + +Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: + +* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically + include additional resources required by this UDF. +* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses + a deprecated interface `initialize(ObjectInspector[])` only. +* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize + functions with `MapredContext`, which is inapplicable to Spark. +* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. + Spark SQL does not call this function when tasks finish. +* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. + Spark SQL currently does not support the reuse of aggregation. +* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating + an aggregate over a fixed window. + +### Incompatible Hive UDF +{:.no_toc} + +Below are the scenarios in which Hive and Spark generate different results: + +* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. +* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. +* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. diff --git a/docs/mllib-migration-guides.md b/docs/sql-migration-old.md similarity index 73% rename from docs/mllib-migration-guides.md rename to docs/sql-migration-old.md index b746b96e19f07..e100820f6d664 100644 --- a/docs/mllib-migration-guides.md +++ b/docs/sql-migration-old.md @@ -1,7 +1,7 @@ --- layout: global -title: Old Migration Guides - MLlib -displayTitle: Old Migration Guides - MLlib +title: Migration Guide +displayTitle: Migration Guide license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,6 +19,5 @@ license: | limitations under the License. --- -The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). +The migration guide is now archived [on this page](sql-migration-guide.html). -Past migration guides are now stored at [ml-migration-guides.html](ml-migration-guides.html). diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md new file mode 100644 index 0000000000000..b0fd8a8325dff --- /dev/null +++ b/docs/ss-migration-guide.md @@ -0,0 +1,32 @@ +--- +layout: global +title: "Migration Guide: Structured Streaming" +displayTitle: "Migration Guide: Structured Streaming" +license: | + 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. +--- + +* Table of contents +{:toc} + +Note that this migration guide describes the items specific to Structured Streaming. +Many items of SQL migration can be applied when migrating Structured Streaming to higher versions. +Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). + +## Upgrading from Structured Streaming 2.4 to 3.0 + +- In Spark 3.0, Structured Streaming forces the source schema into nullable when file-based datasources such as text, json, csv, parquet and orc are used via `spark.readStream(...)`. Previously, it respected the nullability in source schema; however, it caused issues tricky to debug with NPE. To restore the previous behavior, set `spark.sql.streaming.fileSource.schema.forceNullable` to `false`. + From 1b99d0cca4b4fb6d193091f92c46c916b70cd84e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 16 Sep 2019 09:46:34 +0900 Subject: [PATCH 0076/1387] [SPARK-29069][SQL] ResolveInsertInto should not do table lookup ### What changes were proposed in this pull request? It's more clear to only do table lookup in `ResolveTables` rule (for v2 tables) and `ResolveRelations` rule (for v1 tables). `ResolveInsertInto` should only resolve the `InsertIntoStatement` with resolved relations. ### Why are the changes needed? to make the code simpler ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #25774 from cloud-fan/simplify. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon --- .../sql/catalyst/analysis/Analyzer.scala | 58 +++++++++---------- 1 file changed, 27 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8e6be32bcf70e..413c75d2581d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -671,6 +671,15 @@ class Analyzer( case scala.Right(tableOpt) => tableOpt } v2TableOpt.map(DataSourceV2Relation.create).getOrElse(u) + + case i @ InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) if i.query.resolved => + val v2TableOpt = lookupV2Relation(u.multipartIdentifier) match { + case scala.Left((_, _, tableOpt)) => tableOpt + case scala.Right(tableOpt) => tableOpt + } + v2TableOpt.map(DataSourceV2Relation.create).map { v2Relation => + i.copy(table = v2Relation) + }.getOrElse(i) } } @@ -785,41 +794,28 @@ class Analyzer( object ResolveInsertInto extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) if i.query.resolved => - lookupV2Relation(u.multipartIdentifier) match { - case scala.Left((_, _, Some(v2Table: Table))) => - resolveV2Insert(i, v2Table) - case scala.Right(Some(v2Table: Table)) => - resolveV2Insert(i, v2Table) - case _ => - i + case i @ InsertIntoStatement(r: DataSourceV2Relation, _, _, _, _) if i.query.resolved => + // ifPartitionNotExists is append with validation, but validation is not supported + if (i.ifPartitionNotExists) { + throw new AnalysisException( + s"Cannot write, IF NOT EXISTS is not supported for table: ${r.table.name}") } - } - - private def resolveV2Insert(i: InsertIntoStatement, table: Table): LogicalPlan = { - val relation = DataSourceV2Relation.create(table) - // ifPartitionNotExists is append with validation, but validation is not supported - if (i.ifPartitionNotExists) { - throw new AnalysisException( - s"Cannot write, IF NOT EXISTS is not supported for table: ${relation.table.name}") - } - val partCols = partitionColumnNames(relation.table) - validatePartitionSpec(partCols, i.partitionSpec) + val partCols = partitionColumnNames(r.table) + validatePartitionSpec(partCols, i.partitionSpec) - val staticPartitions = i.partitionSpec.filter(_._2.isDefined).mapValues(_.get) - val query = addStaticPartitionColumns(relation, i.query, staticPartitions) - val dynamicPartitionOverwrite = partCols.size > staticPartitions.size && - conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC + val staticPartitions = i.partitionSpec.filter(_._2.isDefined).mapValues(_.get) + val query = addStaticPartitionColumns(r, i.query, staticPartitions) + val dynamicPartitionOverwrite = partCols.size > staticPartitions.size && + conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC - if (!i.overwrite) { - AppendData.byPosition(relation, query) - } else if (dynamicPartitionOverwrite) { - OverwritePartitionsDynamic.byPosition(relation, query) - } else { - OverwriteByExpression.byPosition( - relation, query, staticDeleteExpression(relation, staticPartitions)) - } + if (!i.overwrite) { + AppendData.byPosition(r, query) + } else if (dynamicPartitionOverwrite) { + OverwritePartitionsDynamic.byPosition(r, query) + } else { + OverwriteByExpression.byPosition(r, query, staticDeleteExpression(r, staticPartitions)) + } } private def partitionColumnNames(table: Table): Seq[String] = { From 471a3eff514480cfcbda79bde9294408cc8eb125 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 16 Sep 2019 00:13:07 -0700 Subject: [PATCH 0077/1387] [SPARK-28932][BUILD][FOLLOWUP] Switch to scala-library compile dependency for JDK11 ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/25638 to switch `scala-library` from `test` dependency to `compile` dependency in `network-common` module. ### Why are the changes needed? Previously, we added `scala-library` as a test dependency to resolve the followings, but it was insufficient to resolve. This PR aims to switch it to compile dependency. ``` $ java -version openjdk version "11.0.3" 2019-04-16 OpenJDK Runtime Environment AdoptOpenJDK (build 11.0.3+7) OpenJDK 64-Bit Server VM AdoptOpenJDK (build 11.0.3+7, mixed mode) $ mvn clean install -pl common/network-common -DskipTests ... [INFO] --- scala-maven-plugin:4.2.0:doc-jar (attach-scaladocs) spark-network-common_2.12 --- error: fatal error: object scala in compiler mirror not found. one error found [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually, run the following on JDK11. ``` $ mvn clean install -pl common/network-common -DskipTests ``` Closes #25800 from dongjoon-hyun/SPARK-28932-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- common/network-common/pom.xml | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index c107af9ceb415..2ee17800c10e4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -35,6 +35,12 @@ + + + org.scala-lang + scala-library + + io.netty @@ -87,13 +93,6 @@ - - - org.scala-lang - scala-library - ${scala.version} - test - log4j log4j From 6297287dfa6e9d30141728c931ed58c8c4966851 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 16 Sep 2019 21:48:07 +0800 Subject: [PATCH 0078/1387] [SPARK-29061][SQL] Prints bytecode statistics in debugCodegen ### What changes were proposed in this pull request? This pr proposes to print bytecode statistics (max class bytecode size, max method bytecode size, max constant pool size, and # of inner classes) for generated classes in debug prints, `debugCodegen`. Since these metrics are critical for codegen framework developments, I think its worth printing there. This pr intends to enable `debugCodegen` to print these metrics as following; ``` scala> sql("SELECT sum(v) FROM VALUES(1) t(v)").debugCodegen Found 2 WholeStageCodegen subtrees. == Subtree 1 / 2 (maxClassCodeSize:2693; maxMethodCodeSize:124; maxConstantPoolSize:130(0.20% used); numInnerClasses:0) == ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ *(1) HashAggregate(keys=[], functions=[partial_sum(cast(v#0 as bigint))], output=[sum#5L]) +- *(1) LocalTableScan [v#0] Generated code: /* 001 */ public Object generate(Object[] references) { /* 002 */ return new GeneratedIteratorForCodegenStage1(references); /* 003 */ } ... ``` ### Why are the changes needed? For efficient developments ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested Closes #25766 from maropu/PrintBytecodeStats. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../expressions/codegen/CodeGenerator.scala | 56 ++++++++++++------- .../spark/sql/execution/QueryExecution.scala | 3 +- .../sql/execution/WholeStageCodegenExec.scala | 6 +- .../spark/sql/execution/debug/package.scala | 30 +++++++--- .../execution/WholeStageCodegenSuite.scala | 6 +- .../sql/execution/debug/DebuggingSuite.scala | 45 ++++++++++++++- .../internal/ExecutorSideSQLConfSuite.scala | 2 +- 7 files changed, 112 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 660a1dbaf0aa1..89277706233d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1211,6 +1211,15 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } } +/** + * Java bytecode statistics of a compiled class by Janino. + */ +case class ByteCodeStats(maxMethodCodeSize: Int, maxConstPoolSize: Int, numInnerClasses: Int) + +object ByteCodeStats { + val UNAVAILABLE = ByteCodeStats(-1, -1, -1) +} + object CodeGenerator extends Logging { // This is the default value of HugeMethodLimit in the OpenJDK HotSpot JVM, @@ -1220,6 +1229,9 @@ object CodeGenerator extends Logging { // The max valid length of method parameters in JVM. final val MAX_JVM_METHOD_PARAMS_LENGTH = 255 + // The max number of constant pool entries in JVM. + final val MAX_JVM_CONSTANT_POOL_SIZE = 65535 + // This is the threshold over which the methods in an inner class are grouped in a single // method which is going to be called by the outer class instead of the many small ones final val MERGE_SPLIT_METHODS_THRESHOLD = 3 @@ -1242,9 +1254,9 @@ object CodeGenerator extends Logging { /** * Compile the Java source code into a Java class, using Janino. * - * @return a pair of a generated class and the max bytecode size of generated functions. + * @return a pair of a generated class and the bytecode statistics of generated functions. */ - def compile(code: CodeAndComment): (GeneratedClass, Int) = try { + def compile(code: CodeAndComment): (GeneratedClass, ByteCodeStats) = try { cache.get(code) } catch { // Cache.get() may wrap the original exception. See the following URL @@ -1257,7 +1269,7 @@ object CodeGenerator extends Logging { /** * Compile the Java source code into a Java class, using Janino. */ - private[this] def doCompile(code: CodeAndComment): (GeneratedClass, Int) = { + private[this] def doCompile(code: CodeAndComment): (GeneratedClass, ByteCodeStats) = { val evaluator = new ClassBodyEvaluator() // A special classloader used to wrap the actual parent classloader of @@ -1296,7 +1308,7 @@ object CodeGenerator extends Logging { s"\n${CodeFormatter.format(code)}" }) - val maxCodeSize = try { + val codeStats = try { evaluator.cook("generated.java", code.body) updateAndGetCompilationStats(evaluator) } catch { @@ -1314,14 +1326,15 @@ object CodeGenerator extends Logging { throw new CompileException(msg, e.getLocation) } - (evaluator.getClazz().getConstructor().newInstance().asInstanceOf[GeneratedClass], maxCodeSize) + (evaluator.getClazz().getConstructor().newInstance().asInstanceOf[GeneratedClass], codeStats) } /** - * Returns the max bytecode size of the generated functions by inspecting janino private fields. + * Returns the bytecode statistics (max method bytecode size, max constant pool size, and + * # of inner classes) of generated classes by inspecting Janino classes. * Also, this method updates the metrics information. */ - private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): Int = { + private def updateAndGetCompilationStats(evaluator: ClassBodyEvaluator): ByteCodeStats = { // First retrieve the generated classes. val classes = { val resultField = classOf[SimpleCompiler].getDeclaredField("result") @@ -1336,11 +1349,13 @@ object CodeGenerator extends Logging { val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute") val codeAttrField = codeAttr.getDeclaredField("code") codeAttrField.setAccessible(true) - val codeSizes = classes.flatMap { case (_, classBytes) => - CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classBytes.length) + val codeStats = classes.map { case (_, classBytes) => + val classCodeSize = classBytes.length + CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classCodeSize) try { val cf = new ClassFile(new ByteArrayInputStream(classBytes)) - val stats = cf.methodInfos.asScala.flatMap { method => + val constPoolSize = cf.getConstantPoolSize + val methodCodeSizes = cf.methodInfos.asScala.flatMap { method => method.getAttributes().filter(_.getClass eq codeAttr).map { a => val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize) @@ -1353,19 +1368,20 @@ object CodeGenerator extends Logging { byteCodeSize } } - Some(stats) + (methodCodeSizes.max, constPoolSize) } catch { case NonFatal(e) => logWarning("Error calculating stats of compiled class.", e) - None + (-1, -1) } - }.flatten - - if (codeSizes.nonEmpty) { - codeSizes.max - } else { - 0 } + + val (maxMethodSizes, constPoolSize) = codeStats.unzip + ByteCodeStats( + maxMethodCodeSize = maxMethodSizes.max, + maxConstPoolSize = constPoolSize.max, + // Minus 2 for `GeneratedClass` and an outer-most generated class + numInnerClasses = classes.size - 2) } /** @@ -1380,8 +1396,8 @@ object CodeGenerator extends Logging { private val cache = CacheBuilder.newBuilder() .maximumSize(SQLConf.get.codegenCacheMaxEntries) .build( - new CacheLoader[CodeAndComment, (GeneratedClass, Int)]() { - override def load(code: CodeAndComment): (GeneratedClass, Int) = { + new CacheLoader[CodeAndComment, (GeneratedClass, ByteCodeStats)]() { + override def load(code: CodeAndComment): (GeneratedClass, ByteCodeStats) = { val startTime = System.nanoTime() val result = doCompile(code) val endTime = System.nanoTime() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 630d062d6577e..f294a56c60e26 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -25,6 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.catalyst.expressions.codegen.ByteCodeStats import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule @@ -213,7 +214,7 @@ class QueryExecution( * * @return Sequence of WholeStageCodegen subtrees and corresponding codegen */ - def codegenToSeq(): Seq[(String, String)] = { + def codegenToSeq(): Seq[(String, String, ByteCodeStats)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index ce9a6ea319d5f..f723fcfac6d06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -688,7 +688,7 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def doExecute(): RDD[InternalRow] = { val (ctx, cleanedSource) = doCodeGen() // try to compile and fallback if it failed - val (_, maxCodeSize) = try { + val (_, compiledCodeStats) = try { CodeGenerator.compile(cleanedSource) } catch { case NonFatal(_) if !Utils.isTesting && sqlContext.conf.codegenFallback => @@ -698,9 +698,9 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) } // Check if compiled code has a too large function - if (maxCodeSize > sqlContext.conf.hugeMethodLimit) { + if (compiledCodeStats.maxMethodCodeSize > sqlContext.conf.hugeMethodLimit) { logInfo(s"Found too long generated codes and JIT optimization might not work: " + - s"the bytecode size ($maxCodeSize) is above the limit " + + s"the bytecode size (${compiledCodeStats.maxMethodCodeSize}) is above the limit " + s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " + s"for this plan (id=$codegenStageId). To avoid this, you can raise the limit " + s"`${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}`:\n$treeString") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 03adeaaa66569..6a57ef2cafe23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import java.util.Collections import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging @@ -27,7 +28,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeFormatter, CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat @@ -81,11 +82,20 @@ package object debug { def writeCodegen(append: String => Unit, plan: SparkPlan): Unit = { val codegenSeq = codegenStringSeq(plan) append(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") - for (((subtree, code), i) <- codegenSeq.zipWithIndex) { - append(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n") + for (((subtree, code, codeStats), i) <- codegenSeq.zipWithIndex) { + val usedConstPoolRatio = if (codeStats.maxConstPoolSize > 0) { + val rt = 100.0 * codeStats.maxConstPoolSize / CodeGenerator.MAX_JVM_CONSTANT_POOL_SIZE + "(%.2f%% used)".format(rt) + } else { + "" + } + val codeStatsStr = s"maxMethodCodeSize:${codeStats.maxMethodCodeSize}; " + + s"maxConstantPoolSize:${codeStats.maxConstPoolSize}$usedConstPoolRatio; " + + s"numInnerClasses:${codeStats.numInnerClasses}" + append(s"== Subtree ${i + 1} / ${codegenSeq.size} ($codeStatsStr) ==\n") append(subtree) append("\nGenerated code:\n") - append(s"${code}\n") + append(s"$code\n") } } @@ -95,7 +105,7 @@ package object debug { * @param plan the query plan for codegen * @return Sequence of WholeStageCodegen subtrees and corresponding codegen */ - def codegenStringSeq(plan: SparkPlan): Seq[(String, String)] = { + def codegenStringSeq(plan: SparkPlan): Seq[(String, String, ByteCodeStats)] = { val codegenSubtrees = new collection.mutable.HashSet[WholeStageCodegenExec]() plan transform { case s: WholeStageCodegenExec => @@ -105,7 +115,13 @@ package object debug { } codegenSubtrees.toSeq.map { subtree => val (_, source) = subtree.doCodeGen() - (subtree.toString, CodeFormatter.format(source)) + val codeStats = try { + CodeGenerator.compile(source)._2 + } catch { + case NonFatal(_) => + ByteCodeStats.UNAVAILABLE + } + (subtree.toString, CodeFormatter.format(source), codeStats) } } @@ -130,7 +146,7 @@ package object debug { * @param query the streaming query for codegen * @return Sequence of WholeStageCodegen subtrees and corresponding codegen */ - def codegenStringSeq(query: StreamingQuery): Seq[(String, String)] = { + def codegenStringSeq(query: StreamingQuery): Seq[(String, String, ByteCodeStats)] = { val w = asStreamExecution(query) if (w.lastExecution != null) { codegenStringSeq(w.lastExecution.executedPlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index d8727d5b584f1..6dfb6c85dc9e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeGenerator} +import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec @@ -213,10 +213,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { ignore("SPARK-21871 check if we can get large code size when compiling too long functions") { val codeWithShortFunctions = genGroupByCode(3) - val (_, maxCodeSize1) = CodeGenerator.compile(codeWithShortFunctions) + val (_, ByteCodeStats(maxCodeSize1, _, _)) = CodeGenerator.compile(codeWithShortFunctions) assert(maxCodeSize1 < SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) val codeWithLongFunctions = genGroupByCode(50) - val (_, maxCodeSize2) = CodeGenerator.compile(codeWithLongFunctions) + val (_, ByteCodeStats(maxCodeSize2, _, _)) = CodeGenerator.compile(codeWithLongFunctions) assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index 7a8da7e7669a4..9a48c1ea0f318 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -19,9 +19,15 @@ package org.apache.spark.sql.execution.debug import java.io.ByteArrayOutputStream +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.TestData +import org.apache.spark.sql.types.StructType class DebuggingSuite extends SharedSparkSession { @@ -46,7 +52,7 @@ class DebuggingSuite extends SharedSparkSession { val res = codegenStringSeq(spark.range(10).groupBy(col("id") * 2).count() .queryExecution.executedPlan) assert(res.length == 2) - assert(res.forall{ case (subtree, code) => + assert(res.forall{ case (subtree, code, _) => subtree.contains("Range") && code.contains("Object[]")}) } @@ -90,4 +96,41 @@ class DebuggingSuite extends SharedSparkSession { | id LongType: {} |""".stripMargin)) } + + case class DummyCodeGeneratorPlan(useInnerClass: Boolean) + extends CodegenSupport with LeafExecNode { + override def output: Seq[Attribute] = StructType.fromDDL("d int").toAttributes + override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(spark.sparkContext.emptyRDD[InternalRow]) + override protected def doExecute(): RDD[InternalRow] = sys.error("Not used") + override protected def doProduce(ctx: CodegenContext): String = { + if (useInnerClass) { + val innerClassName = ctx.freshName("innerClass") + ctx.addInnerClass( + s""" + |public class $innerClassName { + | public $innerClassName() {} + |} + """.stripMargin) + } + "" + } + } + + test("Prints bytecode statistics in debugCodegen") { + Seq(true, false).foreach { useInnerClass => + val plan = WholeStageCodegenExec(DummyCodeGeneratorPlan(useInnerClass))(codegenStageId = 0) + + val genCodes = codegenStringSeq(plan) + assert(genCodes.length == 1) + val (_, _, codeStats) = genCodes.head + val expectedNumInnerClasses = if (useInnerClass) 1 else 0 + assert(codeStats.maxMethodCodeSize > 0 && codeStats.maxConstPoolSize > 0 && + codeStats.numInnerClasses == expectedNumInnerClasses) + + val debugCodegenStr = codegenString(plan) + assert(debugCodegenStr.contains("maxMethodCodeSize:")) + assert(debugCodegenStr.contains("maxConstantPoolSize:")) + assert(debugCodegenStr.contains("numInnerClasses:")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index 94b73ec186379..c0238069afcc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -100,7 +100,7 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { val res = codegenStringSeq(spark.range(10).groupBy(col("id") * 2).count() .queryExecution.executedPlan) assert(res.length == 2) - assert(res.forall { case (_, code) => + assert(res.forall { case (_, code, _) => (code.contains("* Codegend pipeline") == flag) && (code.contains("// input[") == flag) }) From 67751e26940a16ab6f9950ae66a46b7cb901c102 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 16 Sep 2019 09:08:25 -0500 Subject: [PATCH 0079/1387] [SPARK-29072][CORE] Put back usage of TimeTrackingOutputStream for UnsafeShuffleWriter and SortShuffleWriter ### What changes were proposed in this pull request? The previous refactors of the shuffle writers using the shuffle writer plugin resulted in shuffle write metric updates - particularly write times - being lost in particular situations. This patch restores the lost metric updates. ### Why are the changes needed? This fixes a regression. I'm pretty sure that without this, the Spark UI will lose shuffle write time information. ### Does this PR introduce any user-facing change? No change from Spark 2.4. Without this, there would be a user-facing bug in Spark 3.0. ### How was this patch tested? Existing unit tests. Closes #25780 from mccheah/fix-write-metrics. Authored-by: mcheah Signed-off-by: Imran Rashid --- .../spark/shuffle/sort/UnsafeShuffleWriter.java | 2 ++ .../shuffle/ShufflePartitionPairsWriter.scala | 15 ++++++++++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index f59bddc993639..4d11abd36985e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -57,6 +57,7 @@ import org.apache.spark.shuffle.api.SingleSpillShuffleMapOutputWriter; import org.apache.spark.shuffle.api.WritableByteChannelWrapper; import org.apache.spark.storage.BlockManager; +import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; @@ -382,6 +383,7 @@ private void mergeSpillsWithFileStream( ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); OutputStream partitionOutput = writer.openStream(); try { + partitionOutput = new TimeTrackingOutputStream(writeMetrics, partitionOutput); partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); if (compressionCodec != null) { partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala index a988c5e126a76..e0affb858c359 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShufflePartitionPairsWriter.scala @@ -21,7 +21,7 @@ import java.io.{Closeable, IOException, OutputStream} import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.api.ShufflePartitionWriter -import org.apache.spark.storage.BlockId +import org.apache.spark.storage.{BlockId, TimeTrackingOutputStream} import org.apache.spark.util.Utils import org.apache.spark.util.collection.PairsWriter @@ -39,6 +39,7 @@ private[spark] class ShufflePartitionPairsWriter( private var isClosed = false private var partitionStream: OutputStream = _ + private var timeTrackingStream: OutputStream = _ private var wrappedStream: OutputStream = _ private var objOut: SerializationStream = _ private var numRecordsWritten = 0 @@ -59,7 +60,8 @@ private[spark] class ShufflePartitionPairsWriter( private def open(): Unit = { try { partitionStream = partitionWriter.openStream - wrappedStream = serializerManager.wrapStream(blockId, partitionStream) + timeTrackingStream = new TimeTrackingOutputStream(writeMetrics, partitionStream) + wrappedStream = serializerManager.wrapStream(blockId, timeTrackingStream) objOut = serializerInstance.serializeStream(wrappedStream) } catch { case e: Exception => @@ -78,6 +80,7 @@ private[spark] class ShufflePartitionPairsWriter( // Setting these to null will prevent the underlying streams from being closed twice // just in case any stream's close() implementation is not idempotent. wrappedStream = null + timeTrackingStream = null partitionStream = null } { // Normally closing objOut would close the inner streams as well, but just in case there @@ -86,9 +89,15 @@ private[spark] class ShufflePartitionPairsWriter( wrappedStream = closeIfNonNull(wrappedStream) // Same as above - if wrappedStream closes then assume it closes underlying // partitionStream and don't close again in the finally + timeTrackingStream = null partitionStream = null } { - partitionStream = closeIfNonNull(partitionStream) + Utils.tryWithSafeFinally { + timeTrackingStream = closeIfNonNull(timeTrackingStream) + partitionStream = null + } { + partitionStream = closeIfNonNull(partitionStream) + } } } updateBytesWritten() From 5881871ca5156ef0e83c9503d5eac288320147c3 Mon Sep 17 00:00:00 2001 From: hongdd Date: Mon, 16 Sep 2019 11:07:50 -0700 Subject: [PATCH 0080/1387] [SPARK-26929][SQL] fix table owner use user instead of principal when create table through spark-sql or beeline MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …create table through spark-sql or beeline ## What changes were proposed in this pull request? fix table owner use user instead of principal when create table through spark-sql private val userName = conf.getUser will get ugi's userName which is principal info, and i copy the source code into HiveClientImpl, and use ugi.getShortUserName() instead of ugi.getUserName(). The owner display correctly. ## How was this patch tested? 1. create a table in kerberos cluster 2. use "desc formatted tbName" check owner Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23952 from hddong/SPARK-26929-fix-table-owner. Lead-authored-by: hongdd Co-authored-by: hongdongdong Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 5b2eeb2cf34c0..ae9eca823d00c 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 @@ -42,6 +42,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging @@ -221,7 +222,7 @@ private[hive] class HiveClientImpl( hiveConf } - private val userName = conf.getUser + private val userName = UserGroupInformation.getCurrentUser.getShortUserName override def getConf(key: String, defaultValue: String): String = { conf.get(key, defaultValue) From 88c8d5eed2bf26ec4cc6ef68d9bdabbcb7ba1b83 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 16 Sep 2019 15:22:04 -0500 Subject: [PATCH 0081/1387] [SPARK-23539][SS][FOLLOWUP][TESTS] Add UT to ensure existing query doesn't break with default conf of includeHeaders ### What changes were proposed in this pull request? This patch adds new UT to ensure existing query (before Spark 3.0.0) with checkpoint doesn't break with default configuration of "includeHeaders" being introduced via SPARK-23539. This patch also modifies existing test which checks type of columns to also check headers column as well. ### Why are the changes needed? The patch adds missing tests which guarantees backward compatibility of the change of SPARK-23539. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? UT passed. Closes #25792 from HeartSaVioR/SPARK-23539-FOLLOWUP. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Sean Owen --- .../commits/0 | 2 + .../metadata | 1 + .../offsets/0 | 3 + .../sources/0/0 | Bin 0 -> 93 bytes .../state/0/0/1.delta | Bin 0 -> 182 bytes .../state/0/1/1.delta | Bin 0 -> 160 bytes .../state/0/2/1.delta | Bin 0 -> 256 bytes .../state/0/3/1.delta | Bin 0 -> 244 bytes .../state/0/4/1.delta | Bin 0 -> 240 bytes .../kafka010/KafkaMicroBatchSourceSuite.scala | 80 +++++++++++++++++- 10 files changed, 84 insertions(+), 2 deletions(-) create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/commits/0 create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/metadata create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/offsets/0 create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/sources/0/0 create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/0/1.delta create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/1/1.delta create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/2/1.delta create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/3/1.delta create mode 100644 external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/4/1.delta diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/commits/0 b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/metadata b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/metadata new file mode 100644 index 0000000000000..f1b5ab7aa17f0 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/metadata @@ -0,0 +1 @@ +{"id":"fc415a71-f0a2-4c3c-aeaf-f9e258c3f726"} \ No newline at end of file diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/offsets/0 b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/offsets/0 new file mode 100644 index 0000000000000..5dbadea57acbe --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1568508285207,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +{"spark-test-topic-2b8619f5-d3c4-4c2d-b5d1-8d9d9458aa62":{"2":3,"4":3,"1":3,"3":3,"0":3}} \ No newline at end of file diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/sources/0/0 b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/sources/0/0 new file mode 100644 index 0000000000000000000000000000000000000000..8cf9f8e009ce87d894533fb440d991ffcba255a3 GIT binary patch literal 93 zcmW;6OA3G>5CG8D9b?uFIR4OXqfSAzNK)E}cTaqaw|ulVC^@`0)}vIcPree$=q&lu hAgiQNC&?HM6FT344Q)*g(%|uI6Ig`w!oD7&ZU^ literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/0/1.delta b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/0/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..5815bbdcc2467001ef9551fd51e414e2f8abc282 GIT binary patch literal 182 zcmeZ?GI7euPtI2DVPIe|VPs%%e&oZF1Y|dGFfcMq+7SHXRL>+<24;Q+ekKOCuhynO zDF+~C0^$lFE-pwc%GNDOEiTb5$uG!E)-_79Ff+7FGu2HoPBzgsNj6H+O)^a})U`;l zOtCaEwMa}fGXm-}G)QJp6=39MU}a)poxz}=5kG^0o5hBKe+Gk@s9_5O&kTkDhW|j| M19hM%P!k*g086JS#{d8T literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/1/1.delta b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/1/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..e1a065b2b1c78c8d842a402fbe6933db404c55a6 GIT binary patch literal 160 zcmeZ?GI7euPtH~?V_;yI1H@{)x-3aRx`Bg%kztZY-;Yy0lUNy8_!)Sa7=)RG7=lfK zk`4k4AM`4KjN*dCqHNug)Z!A|lKg_qWL={q3o}E@G*jIa<75+ElVqb5-6Yc#LtTp$ q%M?o!Q;WnzGb5lrBUT0`CI%$|Mh*rRCWZip|3Kgab&4oZ9u5GT{Ui7Q literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/2/1.delta b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/2/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..cce14294e00443261aa2c3a9cf70ba51fd0d9478 GIT binary patch literal 256 zcmeZ?GI7euPtI1o#=yXk!o+<21b4cUM2=%CLx9o z)}}yN2Owq!;tC)xE=VlO)-6dbF3~N?FUU;RHA=EDGqg-I)lD%@HqkXnHcHV=GEFhm zwMel{u{1HYNK7;{0_xK>WMg1tVvrXoVPIgL!Ju@bYzBiMi#7v469YTT3v^lNQD4cg_vO~Sb<@~lW!*&+4Mmwn1Ry#Cm8kE3>kT5 SFa$9C2Ld0cS4Dw#!vO#$CpRqs literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/3/1.delta b/external/kafka-0-10-sql/src/test/resources/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/state/0/3/1.delta new file mode 100644 index 0000000000000000000000000000000000000000..57063019503bc7c09e5adedab5929420eb2f8525 GIT binary patch literal 244 zcmeZ?GI7euPtI07#lXOj!oLlObjeG3@kGki3I=`yMom^dMxZ2*zJtvS25y!OAeE*JXBe587aE zO@WdQ0t_GYDu9gQg2bY1-ICPe65W#gg3M%Hqa+J6L(4Q%-4x?w6J3*JqZHjF(-cEp zixkTgOA}Lz#6&YApgu!Z1|}v3B>_eb1`Z|$))@>+6`V5|_*EDf7?~JYBp6s`Ffbfo z;Ge;uX{c*x7y)AO8VVf$ZotT>#lR1ACQ!gm-e?Acu>ON{GZ?sePB8KtFzSgKon!S literal 0 HcmV?d00001 diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 9fe100431c01a..609cf3ce4bd75 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -28,12 +28,13 @@ import scala.collection.JavaConverters._ import scala.io.Source import scala.util.Random +import org.apache.commons.io.FileUtils import org.apache.kafka.clients.producer.{ProducerRecord, RecordMetadata} import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession} +import org.apache.spark.sql.{Dataset, ForeachWriter, Row, SparkSession} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.read.streaming.SparkDataStream import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation @@ -47,6 +48,7 @@ import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils abstract class KafkaSourceTest extends StreamTest with SharedSparkSession with KafkaTest { @@ -1162,6 +1164,62 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { intercept[IllegalArgumentException] { test(minPartitions = "-1", 1, true) } } + test("default config of includeHeader doesn't break existing query from Spark 2.4") { + import testImplicits._ + + // This topic name is migrated from Spark 2.4.3 test run + val topic = "spark-test-topic-2b8619f5-d3c4-4c2d-b5d1-8d9d9458aa62" + // create same topic and messages as test run + testUtils.createTopic(topic, partitions = 5, overwrite = true) + testUtils.sendMessages(topic, Array(-20, -21, -22).map(_.toString), Some(0)) + testUtils.sendMessages(topic, Array(-10, -11, -12).map(_.toString), Some(1)) + testUtils.sendMessages(topic, Array(0, 1, 2).map(_.toString), Some(2)) + testUtils.sendMessages(topic, Array(10, 11, 12).map(_.toString), Some(3)) + testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + (31 to 35).map { num => + (num - 31, (num.toString, Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8))))) + }.foreach { rec => testUtils.sendMessage(topic, rec._2, Some(rec._1)) } + + val kafka = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", topic) + .option("startingOffsets", "earliest") + .load() + + val query = kafka.dropDuplicates() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + .map(kv => kv._2.toInt + 1) + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-2.4.3-kafka-include-headers-default/").toURI + + val checkpointDir = Utils.createTempDir().getCanonicalFile + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(new File(resourceUri), checkpointDir) + + testStream(query)( + StartStream(checkpointLocation = checkpointDir.getAbsolutePath), + /* + Note: The checkpoint was generated using the following input in Spark version 2.4.3 + testUtils.createTopic(topic, partitions = 5, overwrite = true) + + testUtils.sendMessages(topic, Array(-20, -21, -22).map(_.toString), Some(0)) + testUtils.sendMessages(topic, Array(-10, -11, -12).map(_.toString), Some(1)) + testUtils.sendMessages(topic, Array(0, 1, 2).map(_.toString), Some(2)) + testUtils.sendMessages(topic, Array(10, 11, 12).map(_.toString), Some(3)) + testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) + */ + makeSureGetOffsetCalled, + CheckNewAnswer(32, 33, 34, 35, 36) + ) + } } abstract class KafkaSourceSuiteBase extends KafkaSourceTest { @@ -1414,7 +1472,9 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { val now = System.currentTimeMillis() val topic = newTopic() testUtils.createTopic(newTopic(), partitions = 1) - testUtils.sendMessages(topic, Array(1).map(_.toString)) + testUtils.sendMessage( + topic, ("1", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))), None + ) val kafka = spark .readStream @@ -1423,6 +1483,7 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { .option("kafka.metadata.max.age.ms", "1") .option("startingOffsets", s"earliest") .option("subscribe", topic) + .option("includeHeaders", "true") .load() val query = kafka @@ -1445,6 +1506,21 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { // producer. So here we just use a low bound to make sure the internal conversion works. assert(row.getAs[java.sql.Timestamp]("timestamp").getTime >= now, s"Unexpected results: $row") assert(row.getAs[Int]("timestampType") === 0, s"Unexpected results: $row") + + def checkHeader(row: Row, expected: Seq[(String, Array[Byte])]): Unit = { + // array> + val headers = row.getList[Row](row.fieldIndex("headers")).asScala + assert(headers.length === expected.length) + + (0 until expected.length).foreach { idx => + val key = headers(idx).getAs[String]("key") + val value = headers(idx).getAs[Array[Byte]]("value") + assert(key === expected(idx)._1) + assert(value === expected(idx)._2) + } + } + + checkHeader(row, Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))) query.stop() } From 95073fb62b646c3e8394941c5835a396b9d48c0f Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 17 Sep 2019 11:09:55 +0900 Subject: [PATCH 0082/1387] [SPARK-29008][SQL] Define an individual method for each common subexpression in HashAggregateExec ### What changes were proposed in this pull request? This pr proposes to define an individual method for each common subexpression in HashAggregateExec. In the current master, the common subexpr elimination code in HashAggregateExec is expanded in a single method; https://github.com/apache/spark/blob/4664a082c2c7ac989e818958c465c72833d3ccfe/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L397 The method size can be too big for JIT compilation, so I believe splitting it is beneficial for performance. For example, in a query `SELECT SUM(a + b), AVG(a + b + c) FROM VALUES (1, 1, 1) t(a, b, c)`, the current master generates; ``` /* 098 */ private void agg_doConsume_0(InternalRow localtablescan_row_0, int agg_expr_0_0, int agg_expr_1_0, int agg_expr_2_0) throws java.io.IOException { /* 099 */ // do aggregate /* 100 */ // common sub-expressions /* 101 */ int agg_value_6 = -1; /* 102 */ /* 103 */ agg_value_6 = agg_expr_0_0 + agg_expr_1_0; /* 104 */ /* 105 */ int agg_value_5 = -1; /* 106 */ /* 107 */ agg_value_5 = agg_value_6 + agg_expr_2_0; /* 108 */ boolean agg_isNull_4 = false; /* 109 */ long agg_value_4 = -1L; /* 110 */ if (!false) { /* 111 */ agg_value_4 = (long) agg_value_5; /* 112 */ } /* 113 */ int agg_value_10 = -1; /* 114 */ /* 115 */ agg_value_10 = agg_expr_0_0 + agg_expr_1_0; /* 116 */ // evaluate aggregate functions and update aggregation buffers /* 117 */ agg_doAggregate_sum_0(agg_value_10); /* 118 */ agg_doAggregate_avg_0(agg_value_4, agg_isNull_4); /* 119 */ /* 120 */ } ``` On the other hand, this pr generates; ``` /* 121 */ private void agg_doConsume_0(InternalRow localtablescan_row_0, int agg_expr_0_0, int agg_expr_1_0, int agg_expr_2_0) throws java.io.IOException { /* 122 */ // do aggregate /* 123 */ // common sub-expressions /* 124 */ long agg_subExprValue_0 = agg_subExpr_0(agg_expr_2_0, agg_expr_0_0, agg_expr_1_0); /* 125 */ int agg_subExprValue_1 = agg_subExpr_1(agg_expr_0_0, agg_expr_1_0); /* 126 */ // evaluate aggregate functions and update aggregation buffers /* 127 */ agg_doAggregate_sum_0(agg_subExprValue_1); /* 128 */ agg_doAggregate_avg_0(agg_subExprValue_0); /* 129 */ /* 130 */ } ``` I run some micro benchmarks for this pr; ``` (base) maropu~:$system_profiler SPHardwareDataType Hardware: Hardware Overview: Processor Name: Intel Core i5 Processor Speed: 2 GHz Number of Processors: 1 Total Number of Cores: 2 L2 Cache (per Core): 256 KB L3 Cache: 4 MB Memory: 8 GB (base) maropu~:$java -version java version "1.8.0_181" Java(TM) SE Runtime Environment (build 1.8.0_181-b13) Java HotSpot(TM) 64-Bit Server VM (build 25.181-b13, mixed mode) (base) maropu~:$ /bin/spark-shell --master=local[1] --conf spark.driver.memory=8g --conf spark.sql.shurtitions=1 -v val numCols = 40 val colExprs = "id AS key" +: (0 until numCols).map { i => s"id AS _c$i" } spark.range(3000000).selectExpr(colExprs: _*).createOrReplaceTempView("t") val aggExprs = (2 until numCols).map { i => (0 until i).map(d => s"_c$d") .mkString("AVG(", " + ", ")") } // Drops the time of a first run then pick that of a second run timer { sql(s"SELECT ${aggExprs.mkString(", ")} FROM t").write.format("noop").save() } // the master maxCodeGen: 12957 Elapsed time: 36.309858661s // this pr maxCodeGen=4184 Elapsed time: 2.399490285s ``` ### Why are the changes needed? To avoid the too-long-function issue in JVMs. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added tests in `WholeStageCodegenSuite` Closes #25710 from maropu/SplitSubexpr. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../expressions/codegen/CodeGenerator.scala | 78 ++++++++++++++++--- .../aggregate/HashAggregateExec.scala | 10 +-- .../execution/WholeStageCodegenSuite.scala | 25 +++++- 3 files changed, 93 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 89277706233d7..1d010ff5abfcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -112,7 +112,7 @@ private[codegen] case class NewFunctionSpec( * A context for codegen, tracking a list of objects that could be passed into generated Java * function. */ -class CodegenContext { +class CodegenContext extends Logging { import CodeGenerator._ @@ -1038,13 +1038,67 @@ class CodegenContext { // Get all the expressions that appear at least twice and set up the state for subexpression // elimination. val commonExprs = equivalentExpressions.getAllEquivalentExprs.filter(_.size > 1) - val codes = commonExprs.map { e => - val expr = e.head - // Generate the code for this expression tree. - val eval = expr.genCode(this) - val state = SubExprEliminationState(eval.isNull, eval.value) - e.foreach(localSubExprEliminationExprs.put(_, state)) - eval.code.toString + val commonExprVals = commonExprs.map(_.head.genCode(this)) + + lazy val nonSplitExprCode = { + commonExprs.zip(commonExprVals).map { case (exprs, eval) => + // Generate the code for this expression tree. + val state = SubExprEliminationState(eval.isNull, eval.value) + exprs.foreach(localSubExprEliminationExprs.put(_, state)) + eval.code.toString + } + } + + val codes = if (commonExprVals.map(_.code.length).sum > SQLConf.get.methodSplitThreshold) { + if (commonExprs.map(calculateParamLength).forall(isValidParamLength)) { + commonExprs.zipWithIndex.map { case (exprs, i) => + val expr = exprs.head + val eval = commonExprVals(i) + + val isNullLiteral = eval.isNull match { + case TrueLiteral | FalseLiteral => true + case _ => false + } + val (isNull, isNullEvalCode) = if (!isNullLiteral) { + val v = addMutableState(JAVA_BOOLEAN, "subExprIsNull") + (JavaCode.isNullGlobal(v), s"$v = ${eval.isNull};") + } else { + (eval.isNull, "") + } + + // Generate the code for this expression tree and wrap it in a function. + val fnName = freshName("subExpr") + val inputVars = getLocalInputVariableValues(this, expr).toSeq + val argList = inputVars.map(v => s"${v.javaType.getName} ${v.variableName}") + val returnType = javaType(expr.dataType) + val fn = + s""" + |private $returnType $fnName(${argList.mkString(", ")}) { + | ${eval.code} + | $isNullEvalCode + | return ${eval.value}; + |} + """.stripMargin + + val value = freshName("subExprValue") + val state = SubExprEliminationState(isNull, JavaCode.variable(value, expr.dataType)) + exprs.foreach(localSubExprEliminationExprs.put(_, state)) + val inputVariables = inputVars.map(_.variableName).mkString(", ") + s"$returnType $value = ${addNewFunction(fnName, fn)}($inputVariables);" + } + } else { + val errMsg = "Failed to split subexpression code into small functions because the " + + "parameter length of at least one split function went over the JVM limit: " + + MAX_JVM_METHOD_PARAMS_LENGTH + if (Utils.isTesting) { + throw new IllegalStateException(errMsg) + } else { + logInfo(errMsg) + nonSplitExprCode + } + } + } else { + nonSplitExprCode } SubExprCodes(codes, localSubExprEliminationExprs.toMap) } @@ -1646,7 +1700,7 @@ object CodeGenerator extends Logging { def getLocalInputVariableValues( ctx: CodegenContext, expr: Expression, - subExprs: Map[Expression, SubExprEliminationState]): Set[VariableValue] = { + subExprs: Map[Expression, SubExprEliminationState] = Map.empty): Set[VariableValue] = { val argSet = mutable.Set[VariableValue]() if (ctx.INPUT_ROW != null) { argSet += JavaCode.variable(ctx.INPUT_ROW, classOf[InternalRow]) @@ -1801,6 +1855,10 @@ object CodeGenerator extends Logging { * length less than a pre-defined constant. */ def isValidParamLength(paramLength: Int): Boolean = { - paramLength <= MAX_JVM_METHOD_PARAMS_LENGTH + // This config is only for testing + SQLConf.get.getConfString("spark.sql.CodeGenerator.validParamLength", null) match { + case null | "" => paramLength <= MAX_JVM_METHOD_PARAMS_LENGTH + case validLength => paramLength <= validLength.toInt + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 7ead180d869b2..2d187e3c9ebe5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -261,14 +261,6 @@ case class HashAggregateExec( """.stripMargin } - private def isValidParamLength(paramLength: Int): Boolean = { - // This config is only for testing - sqlContext.getConf("spark.sql.HashAggregateExec.validParamLength", null) match { - case null | "" => CodeGenerator.isValidParamLength(paramLength) - case validLength => paramLength <= validLength.toInt - } - } - // Splits aggregate code into small functions because the most of JVM implementations // can not compile too long functions. Returns None if we are not able to split the given code. // @@ -294,7 +286,7 @@ case class HashAggregateExec( val paramLength = CodeGenerator.calculateParamLengthFromExprValues(inputVarsForOneFunc) // Checks if a parameter length for the `aggExprsForOneFunc` does not go over the JVM limit - if (isValidParamLength(paramLength)) { + if (CodeGenerator.isValidParamLength(paramLength)) { Some(inputVarsForOneFunc) } else { None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 6dfb6c85dc9e0..afe9eb5c151d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -403,7 +403,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { withSQLConf( SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC.key -> "true", SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1", - "spark.sql.HashAggregateExec.validParamLength" -> "0") { + "spark.sql.CodeGenerator.validParamLength" -> "0") { withTable("t") { val expectedErrMsg = "Failed to split aggregate code into small functions" Seq( @@ -419,4 +419,27 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { } } } + + test("Give up splitting subexpression code if a parameter length goes over the limit") { + withSQLConf( + SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC.key -> "false", + SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1", + "spark.sql.CodeGenerator.validParamLength" -> "0") { + withTable("t") { + val expectedErrMsg = "Failed to split subexpression code into small functions" + Seq( + // Test case without keys + "SELECT AVG(a + b), SUM(a + b + c) FROM VALUES((1, 1, 1)) t(a, b, c)", + // Tet case with keys + "SELECT k, AVG(a + b), SUM(a + b + c) FROM VALUES((1, 1, 1, 1)) t(k, a, b, c) " + + "GROUP BY k").foreach { query => + val e = intercept[Exception] { + sql(query).collect + }.getCause + assert(e.isInstanceOf[IllegalStateException]) + assert(e.getMessage.contains(expectedErrMsg)) + } + } + } + } } From dffd92e9779021fa7df2ec962c9cd07e0dbfc2f3 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 17 Sep 2019 11:06:10 +0800 Subject: [PATCH 0083/1387] [SPARK-29100][SQL] Fix compilation error in codegen with switch from InSet expression ### What changes were proposed in this pull request? When InSet generates Java switch-based code, if the input set is empty, we don't generate switch condition, but a simple expression that is default case of original switch. ### Why are the changes needed? SPARK-26205 adds an optimization to InSet that generates Java switch condition for certain cases. When the given set is empty, it is possibly that codegen causes compilation error: ``` [info] - SPARK-29100: InSet with empty input set *** FAILED *** (58 milliseconds) [info] Code generation of input[0, int, true] INSET () failed: [info] org.codehaus.janino.InternalCompilerException: failed to compile: org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass" in "generated.java": Compiling "apply(java.lang.Object _i)"; apply(java.lang.Object _i): Operand stack inconsistent at offset 45: Previous size 0, now 1 [info] org.codehaus.janino.InternalCompilerException: failed to compile: org.codehaus.janino.InternalCompilerException: Compiling "GeneratedClass" in "generated.java": Compiling "apply(java.lang.Object _i)"; apply(java.lang.Object _i): Operand stack inconsistent at offset 45: Previous size 0, now 1 [info] at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.org$apache$spark$sql$catalyst$expressions$codegen$CodeGenerator$$doCompile(CodeGenerator.scala:1308) [info] at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1386) [info] at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$$anon$1.load(CodeGenerator.scala:1383) ``` ### Does this PR introduce any user-facing change? Yes. Previously, when users have InSet against an empty set, generated code causes compilation error. This patch fixed it. ### How was this patch tested? Unit test added. Closes #25806 from viirya/SPARK-29100. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/predicates.scala | 18 +++++++++++++----- .../catalyst/expressions/PredicateSuite.scala | 6 ++++++ 2 files changed, 19 insertions(+), 5 deletions(-) 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 7ce113120e4c6..4c0998412f729 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 @@ -457,17 +457,25 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with break; """) + val switchCode = if (caseBranches.size > 0) { + code""" + switch (${valueGen.value}) { + ${caseBranches.mkString("\n")} + default: + ${ev.isNull} = $hasNull; + } + """ + } else { + s"${ev.isNull} = $hasNull;" + } + ev.copy(code = code""" ${valueGen.code} ${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = ${valueGen.isNull}; ${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = false; if (!${valueGen.isNull}) { - switch (${valueGen.value}) { - ${caseBranches.mkString("\n")} - default: - ${ev.isNull} = $hasNull; - } + $switchCode } """) } 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 7bff277c793ea..b05e49ea33015 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 @@ -560,4 +560,10 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { assert(msg.contains("argument 1 requires boolean type")) } } + + test("SPARK-29100: InSet with empty input set") { + val row = create_row(1) + val inSet = InSet(BoundReference(0, IntegerType, true), Set.empty) + checkEvaluation(inSet, false, row) + } } From 4d27a259087258492d0a66ca1ace7ef584c72a6f Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Tue, 17 Sep 2019 11:52:20 +0800 Subject: [PATCH 0084/1387] [SPARK-22797][ML][PYTHON] Bucketizer support multi-column ### What changes were proposed in this pull request? Bucketizer support multi-column in the python side ### Why are the changes needed? Bucketizer should support multi-column like the scala side. ### Does this PR introduce any user-facing change? yes, this PR add new Python API ### How was this patch tested? added testsuites Closes #25801 from zhengruifeng/20542_py. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- python/pyspark/ml/feature.py | 102 ++++++++++++++++++++------ python/pyspark/ml/param/__init__.py | 10 +++ python/pyspark/ml/tests/test_param.py | 9 +++ 3 files changed, 98 insertions(+), 23 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 9ab4e4d68691d..5f55e48ce9f45 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -327,26 +327,34 @@ class BucketedRandomProjectionLSHModel(LSHModel, JavaMLReadable, JavaMLWritable) @inherit_doc -class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, - JavaMLReadable, JavaMLWritable): +class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, + HasHandleInvalid, JavaMLReadable, JavaMLWritable): """ - Maps a column of continuous features to a column of feature buckets. - - >>> values = [(0.1,), (0.4,), (1.2,), (1.5,), (float("nan"),), (float("nan"),)] - >>> df = spark.createDataFrame(values, ["values"]) + Maps a column of continuous features to a column of feature buckets. Since 3.0.0, + :py:class:`Bucketizer` can map multiple columns at once by setting the :py:attr:`inputCols` + parameter. Note that when both the :py:attr:`inputCol` and :py:attr:`inputCols` parameters + are set, an Exception will be thrown. The :py:attr:`splits` parameter is only used for single + column usage, and :py:attr:`splitsArray` is for multiple columns. + + >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")), + ... (float("nan"), 1.0), (float("nan"), 0.0)] + >>> df = spark.createDataFrame(values, ["values1", "values2"]) >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], - ... inputCol="values", outputCol="buckets") + ... inputCol="values1", outputCol="buckets") >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df).collect() - >>> len(bucketed) - 6 - >>> bucketed[0].buckets - 0.0 - >>> bucketed[1].buckets - 0.0 - >>> bucketed[2].buckets - 1.0 - >>> bucketed[3].buckets - 2.0 + >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df.select("values1")) + >>> bucketed.show(truncate=False) + +-------+-------+ + |values1|buckets| + +-------+-------+ + |0.1 |0.0 | + |0.4 |0.0 | + |1.2 |1.0 | + |1.5 |2.0 | + |NaN |3.0 | + |NaN |3.0 | + +-------+-------+ + ... >>> bucketizer.setParams(outputCol="b").transform(df).head().b 0.0 >>> bucketizerPath = temp_path + "/bucketizer" @@ -357,6 +365,22 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, >>> bucketed = bucketizer.setHandleInvalid("skip").transform(df).collect() >>> len(bucketed) 4 + >>> bucketizer2 = Bucketizer(splitsArray= + ... [[-float("inf"), 0.5, 1.4, float("inf")], [-float("inf"), 0.5, float("inf")]], + ... inputCols=["values1", "values2"], outputCols=["buckets1", "buckets2"]) + >>> bucketed2 = bucketizer2.setHandleInvalid("keep").transform(df) + >>> bucketed2.show(truncate=False) + +-------+-------+--------+--------+ + |values1|values2|buckets1|buckets2| + +-------+-------+--------+--------+ + |0.1 |0.0 |0.0 |0.0 | + |0.4 |1.0 |0.0 |1.0 | + |1.2 |1.3 |1.0 |1.0 | + |1.5 |NaN |2.0 |2.0 | + |NaN |1.0 |3.0 |1.0 | + |NaN |0.0 |3.0 |0.0 | + +-------+-------+--------+--------+ + ... .. versionadded:: 1.4.0 """ @@ -374,14 +398,30 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasHandleInvalid, handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries " "containing NaN values. Values outside the splits will always be treated " "as errors. Options are 'skip' (filter out rows with invalid values), " + - "'error' (throw an error), or 'keep' (keep invalid values in a special " + - "additional bucket).", + "'error' (throw an error), or 'keep' (keep invalid values in a " + + "special additional bucket). Note that in the multiple column " + + "case, the invalid handling is applied to all columns. That said " + + "for 'error' it will throw an error if any invalids are found in " + + "any column, for 'skip' it will skip rows with any invalids in " + + "any columns, etc.", typeConverter=TypeConverters.toString) + splitsArray = Param(Params._dummy(), "splitsArray", "The array of split points for mapping " + + "continuous features into buckets for multiple columns. For each input " + + "column, with n+1 splits, there are n buckets. A bucket defined by " + + "splits x,y holds values in the range [x,y) except the last bucket, " + + "which also includes y. The splits should be of length >= 3 and " + + "strictly increasing. Values at -inf, inf must be explicitly provided " + + "to cover all Double values; otherwise, values outside the splits " + + "specified will be treated as errors.", + typeConverter=TypeConverters.toListListFloat) + @keyword_only - def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error"): + def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + splitsArray=None, inputCols=None, outputCols=None): """ - __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error") + __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", \ + splitsArray=None, inputCols=None, outputCols=None) """ super(Bucketizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Bucketizer", self.uid) @@ -391,9 +431,11 @@ def __init__(self, splits=None, inputCol=None, outputCol=None, handleInvalid="er @keyword_only @since("1.4.0") - def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error"): + def setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", + splitsArray=None, inputCols=None, outputCols=None): """ - setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error") + setParams(self, splits=None, inputCol=None, outputCol=None, handleInvalid="error", \ + splitsArray=None, inputCols=None, outputCols=None) Sets params for this Bucketizer. """ kwargs = self._input_kwargs @@ -413,6 +455,20 @@ def getSplits(self): """ return self.getOrDefault(self.splits) + @since("3.0.0") + def setSplitsArray(self, value): + """ + Sets the value of :py:attr:`splitsArray`. + """ + return self._set(splitsArray=value) + + @since("3.0.0") + def getSplitsArray(self): + """ + Gets the array of split points or its default value. + """ + return self.getOrDefault(self.splitsArray) + class _CountVectorizerParams(JavaParams, HasInputCol, HasOutputCol): """ diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 043c25cf9feb4..5b6b70292f099 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -134,6 +134,16 @@ def toListFloat(value): return [float(v) for v in value] raise TypeError("Could not convert %s to list of floats" % value) + @staticmethod + def toListListFloat(value): + """ + Convert a value to list of list of floats, if possible. + """ + if TypeConverters._can_convert_to_list(value): + value = TypeConverters.toList(value) + return [TypeConverters.toListFloat(v) for v in value] + raise TypeError("Could not convert %s to list of list of floats" % value) + @staticmethod def toListInt(value): """ diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index cbeac0b1319c8..72b146054ca44 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -87,6 +87,15 @@ def test_list_float(self): self.assertTrue(all([type(v) == float for v in b.getSplits()])) self.assertRaises(TypeError, lambda: Bucketizer(splits=["a", 1.0])) + def test_list_list_float(self): + b = Bucketizer(splitsArray=[[-0.1, 0.5, 3], [-5, 1.5]]) + self.assertEqual(b.getSplitsArray(), [[-0.1, 0.5, 3.0], [-5.0, 1.5]]) + self.assertTrue(all([type(v) == list for v in b.getSplitsArray()])) + self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[0]])) + self.assertTrue(all([type(v) == float for v in b.getSplitsArray()[1]])) + self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=["a", 1.0])) + self.assertRaises(TypeError, lambda: Bucketizer(splitsArray=[[-5, 1.5], ["a", 1.0]])) + def test_list_string(self): for labels in [np.array(['a', u'b']), ['a', u'b'], np.array(['a', 'b'])]: idx_to_string = IndexToString(labels=labels) From c8628354b7d2e6116b2a6eb3bdb2fc957c91fd03 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 17 Sep 2019 14:04:23 +0800 Subject: [PATCH 0085/1387] [SPARK-28996][SQL][TESTS] Add tests regarding username of HiveClient ### What changes were proposed in this pull request? This patch proposes to add new tests to test the username of HiveClient to prevent changing the semantic unintentionally. The owner of Hive table has been changed back-and-forth, principal -> username -> principal, and looks like the change is not intentional. (Please refer [SPARK-28996](https://issues.apache.org/jira/browse/SPARK-28996) for more details.) This patch intends to prevent this. This patch also renames previous HiveClientSuite(s) to HivePartitionFilteringSuite(s) as it was commented as TODO, as well as previous tests are too narrowed to test only partition filtering. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Newly added UTs. Closes #25696 from HeartSaVioR/SPARK-28996. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Wenchen Fan --- .../spark/sql/hive/client/HiveClient.scala | 2 + .../sql/hive/client/HiveClientImpl.scala | 2 +- .../hive/client/HiveClientUserNameSuite.scala | 63 +++++++++++++++++++ .../client/HiveClientUserNameSuites.scala | 28 +++++++++ ...cala => HivePartitionFilteringSuite.scala} | 3 +- ...ala => HivePartitionFilteringSuites.scala} | 4 +- 6 files changed, 97 insertions(+), 5 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuites.scala rename sql/hive/src/test/scala/org/apache/spark/sql/hive/client/{HiveClientSuite.scala => HivePartitionFilteringSuite.scala} (99%) rename sql/hive/src/test/scala/org/apache/spark/sql/hive/client/{HiveClientSuites.scala => HivePartitionFilteringSuites.scala} (87%) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index cb015d7301c19..51b1778ec653f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -292,4 +292,6 @@ private[hive] trait HiveClient { /** Used for testing only. Removes all metadata from this instance of Hive. */ def reset(): Unit + /** Returns the user name which is used as owner for Hive table. */ + def userName: String } 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 ae9eca823d00c..96e61bd542806 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 @@ -222,7 +222,7 @@ private[hive] class HiveClientImpl( hiveConf } - private val userName = UserGroupInformation.getCurrentUser.getShortUserName + override val userName = UserGroupInformation.getCurrentUser.getShortUserName override def getConf(key: String, defaultValue: String): String = { conf.get(key, defaultValue) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuite.scala new file mode 100644 index 0000000000000..77956f4fe69da --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuite.scala @@ -0,0 +1,63 @@ +/* + * 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.hive.client + +import java.security.PrivilegedExceptionAction + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.UserGroupInformation +import org.scalatest.{BeforeAndAfterAll, PrivateMethodTester} + +import org.apache.spark.util.Utils + +class HiveClientUserNameSuite(version: String) extends HiveVersionSuite(version) { + + test("username of HiveClient - no UGI") { + // Assuming we're not faking System username + assert(getUserNameFromHiveClient === System.getProperty("user.name")) + } + + test("username of HiveClient - UGI") { + val ugi = UserGroupInformation.createUserForTesting( + "fakeprincipal@EXAMPLE.COM", Array.empty) + ugi.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + assert(getUserNameFromHiveClient === ugi.getShortUserName) + } + }) + } + + test("username of HiveClient - Proxy user") { + val ugi = UserGroupInformation.createUserForTesting( + "fakeprincipal@EXAMPLE.COM", Array.empty) + val proxyUgi = UserGroupInformation.createProxyUserForTesting( + "proxyprincipal@EXAMPLE.COM", ugi, Array.empty) + proxyUgi.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + assert(getUserNameFromHiveClient === proxyUgi.getShortUserName) + } + }) + } + + private def getUserNameFromHiveClient: String = { + val hadoopConf = new Configuration() + hadoopConf.set("hive.metastore.warehouse.dir", Utils.createTempDir().toURI().toString()) + val client = buildClient(hadoopConf) + client.userName + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuites.scala new file mode 100644 index 0000000000000..e076c01c08980 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientUserNameSuites.scala @@ -0,0 +1,28 @@ +/* + * 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.hive.client + +import scala.collection.immutable.IndexedSeq + +import org.scalatest.Suite + +class HiveClientUserNameSuites extends Suite with HiveClientVersions { + override def nestedSuites: IndexedSeq[Suite] = { + versions.map(new HiveClientUserNameSuite(_)) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala similarity index 99% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala index bda711200acdb..5f4ee7d7f1c0f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala @@ -31,8 +31,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType, StructType} import org.apache.spark.util.Utils -// TODO: Refactor this to `HivePartitionFilteringSuite` -class HiveClientSuite(version: String) +class HivePartitionFilteringSuite(version: String) extends HiveVersionSuite(version) with BeforeAndAfterAll { private val tryDirectSqlKey = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuites.scala similarity index 87% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuites.scala index de1be2115b2d8..a43e778b13b92 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuites.scala @@ -21,9 +21,9 @@ import scala.collection.immutable.IndexedSeq import org.scalatest.Suite -class HiveClientSuites extends Suite with HiveClientVersions { +class HivePartitionFilteringSuites extends Suite with HiveClientVersions { override def nestedSuites: IndexedSeq[Suite] = { // Hive 0.12 does not provide the partition filtering API we call - versions.filterNot(_ == "0.12").map(new HiveClientSuite(_)) + versions.filterNot(_ == "0.12").map(new HivePartitionFilteringSuite(_)) } } From db996ccad91bbd7db412b1363641820784ce77bc Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Sep 2019 16:00:10 +0900 Subject: [PATCH 0086/1387] [SPARK-29074][SQL] Optimize `date_format` for foldable `fmt` ### What changes were proposed in this pull request? In the PR, I propose to create an instance of `TimestampFormatter` only once at the initialization, and reuse it inside of `nullSafeEval()` and `doGenCode()` in the case when the `fmt` parameter is foldable. ### Why are the changes needed? The changes improve performance of the `date_format()` function. Before: ``` format date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ format date wholestage off 7180 / 7181 1.4 718.0 1.0X format date wholestage on 7051 / 7194 1.4 705.1 1.0X ``` After: ``` format date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ format date wholestage off 4787 / 4839 2.1 478.7 1.0X format date wholestage on 4736 / 4802 2.1 473.6 1.0X ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? By existing test suites `DateExpressionsSuite` and `DateFunctionsSuite`. Closes #25782 from MaxGekk/date_format-foldable. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../expressions/datetimeExpressions.scala | 32 ++++++++++++++----- .../benchmarks/DateTimeBenchmark-results.txt | 4 +-- 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index eafdca2bfc53e..597be5dc97b43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -592,19 +592,35 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) + @transient private lazy val formatter: Option[TimestampFormatter] = { + if (right.foldable) { + Option(right.eval()).map(format => TimestampFormatter(format.toString, zoneId)) + } else None + } + override protected def nullSafeEval(timestamp: Any, format: Any): Any = { - val df = TimestampFormatter(format.toString, zoneId) - UTF8String.fromString(df.format(timestamp.asInstanceOf[Long])) + val tf = if (formatter.isEmpty) { + TimestampFormatter(format.toString, zoneId) + } else { + formatter.get + } + UTF8String.fromString(tf.format(timestamp.asInstanceOf[Long])) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tf = TimestampFormatter.getClass.getName.stripSuffix("$") - val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) - val locale = ctx.addReferenceObj("locale", Locale.US) - defineCodeGen(ctx, ev, (timestamp, format) => { - s"""UTF8String.fromString($tf$$.MODULE$$.apply($format.toString(), $zid, $locale) + formatter.map { tf => + val timestampFormatter = ctx.addReferenceObj("timestampFormatter", tf) + defineCodeGen(ctx, ev, (timestamp, _) => { + s"""UTF8String.fromString($timestampFormatter.format($timestamp))""" + }) + }.getOrElse { + val tf = TimestampFormatter.getClass.getName.stripSuffix("$") + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + defineCodeGen(ctx, ev, (timestamp, format) => { + s"""UTF8String.fromString($tf$$.MODULE$$.apply($format.toString(), $zid) .format($timestamp))""" - }) + }) + } } override def prettyName: String = "date_format" diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 1a58b05a2abba..7d562544dd498 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -168,8 +168,8 @@ Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz format date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -format date wholestage off 7180 / 7181 1.4 718.0 1.0X -format date wholestage on 7051 / 7194 1.4 705.1 1.0X +format date wholestage off 4787 / 4839 2.1 478.7 1.0X +format date wholestage on 4736 / 4802 2.1 473.6 1.0X ================================================================================================ From 79b10a1aab9be6abdf749ad94c88234ace8ba34a Mon Sep 17 00:00:00 2001 From: iRakson Date: Tue, 17 Sep 2019 00:53:12 -0700 Subject: [PATCH 0087/1387] [SPARK-28929][CORE] Spark Logging level should be INFO instead of DEBUG in Executor Plugin API ### What changes were proposed in this pull request? Log levels in Executor.scala are changed from DEBUG to INFO. ### Why are the changes needed? Logging level DEBUG is too low here. These messages are simple acknowledgement for successful loading and initialization of plugins. So its better to keep them in INFO level. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested. Closes #25634 from iRakson/ExecutorPlugin. Authored-by: iRakson Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/executor/Executor.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c337d24381286..a32226581b973 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -139,7 +139,7 @@ private[spark] class Executor( private val executorPlugins: Seq[ExecutorPlugin] = { val pluginNames = conf.get(EXECUTOR_PLUGINS) if (pluginNames.nonEmpty) { - logDebug(s"Initializing the following plugins: ${pluginNames.mkString(", ")}") + logInfo(s"Initializing the following plugins: ${pluginNames.mkString(", ")}") // Plugins need to load using a class loader that includes the executor's user classpath val pluginList: Seq[ExecutorPlugin] = @@ -147,12 +147,12 @@ private[spark] class Executor( val plugins = Utils.loadExtensions(classOf[ExecutorPlugin], pluginNames, conf) plugins.foreach { plugin => plugin.init() - logDebug(s"Successfully loaded plugin " + plugin.getClass().getCanonicalName()) + logInfo(s"Successfully loaded plugin " + plugin.getClass().getCanonicalName()) } plugins } - logDebug("Finished initializing plugins") + logInfo("Finished initializing plugins") pluginList } else { Nil From 104b9b6f8c93f341bda043852aa61ea2a1d2e21b Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Tue, 17 Sep 2019 19:08:09 +0800 Subject: [PATCH 0088/1387] [SPARK-28483][FOLLOW-UP] Fix flaky test in BarrierTaskContextSuite ### What changes were proposed in this pull request? I fix the test "barrier task killed" which is flaky: * Split interrupt/no interrupt test into separate sparkContext. Prevent them to influence each other. * only check exception on partiton-0. partition-1 is hang on sleep which may throw other exception. ### Why are the changes needed? Make test robust. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #25799 from WeichenXu123/oss_fix_barrier_test. Authored-by: WeichenXu Signed-off-by: WeichenXu --- .../scheduler/BarrierTaskContextSuite.scala | 72 ++++++++----------- 1 file changed, 30 insertions(+), 42 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index 8d5f04ac7651a..fc8ac38479932 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -26,13 +26,18 @@ import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { - test("global sync by barrier() call") { + def initLocalClusterSparkContext(): Unit = { val conf = new SparkConf() // Init local cluster here so each barrier task runs in a separated process, thus `barrier()` // call is actually useful. .setMaster("local-cluster[4, 1, 1024]") .setAppName("test-cluster") + .set(TEST_NO_STAGE_RETRY, true) sc = new SparkContext(conf) + } + + test("global sync by barrier() call") { + initLocalClusterSparkContext() val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -48,10 +53,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } test("support multiple barrier() call within a single task") { - val conf = new SparkConf() - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + initLocalClusterSparkContext() val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -77,12 +79,8 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } test("throw exception on barrier() call timeout") { - val conf = new SparkConf() - .set("spark.barrier.sync.timeout", "1") - .set(TEST_NO_STAGE_RETRY, true) - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + initLocalClusterSparkContext() + sc.conf.set("spark.barrier.sync.timeout", "1") val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -102,12 +100,8 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } test("throw exception if barrier() call doesn't happen on every task") { - val conf = new SparkConf() - .set("spark.barrier.sync.timeout", "1") - .set(TEST_NO_STAGE_RETRY, true) - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + initLocalClusterSparkContext() + sc.conf.set("spark.barrier.sync.timeout", "1") val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -125,12 +119,8 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } test("throw exception if the number of barrier() calls are not the same on every task") { - val conf = new SparkConf() - .set("spark.barrier.sync.timeout", "1") - .set(TEST_NO_STAGE_RETRY, true) - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + initLocalClusterSparkContext() + sc.conf.set("spark.barrier.sync.timeout", "1") val rdd = sc.makeRDD(1 to 10, 4) val rdd2 = rdd.barrier().mapPartitions { it => val context = BarrierTaskContext.get() @@ -156,10 +146,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { assert(error.contains("within 1 second(s)")) } - - def testBarrierTaskKilled(sc: SparkContext, interruptOnCancel: Boolean): Unit = { - sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, interruptOnCancel.toString) - + def testBarrierTaskKilled(interruptOnKill: Boolean): Unit = { withTempDir { dir => val killedFlagFile = "barrier.task.killed" val rdd = sc.makeRDD(Seq(0, 1), 2) @@ -181,12 +168,15 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { val listener = new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { - new Thread { - override def run: Unit = { - Thread.sleep(1000) - sc.killTaskAttempt(taskStart.taskInfo.taskId, interruptThread = false) - } - }.start() + val partitionId = taskStart.taskInfo.index + if (partitionId == 0) { + new Thread { + override def run: Unit = { + Thread.sleep(1000) + sc.killTaskAttempt(taskStart.taskInfo.taskId, interruptThread = interruptOnKill) + } + }.start() + } } } sc.addSparkListener(listener) @@ -201,15 +191,13 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { } } - test("barrier task killed") { - val conf = new SparkConf() - .set("spark.barrier.sync.timeout", "1") - .set(TEST_NO_STAGE_RETRY, true) - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) + test("barrier task killed, no interrupt") { + initLocalClusterSparkContext() + testBarrierTaskKilled(interruptOnKill = false) + } - testBarrierTaskKilled(sc, true) - testBarrierTaskKilled(sc, false) + test("barrier task killed, interrupt") { + initLocalClusterSparkContext() + testBarrierTaskKilled(interruptOnKill = true) } } From 34915b22ab174a45c563ccdcd5035299f3ccc56c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 17 Sep 2019 20:21:25 +0900 Subject: [PATCH 0089/1387] [SPARK-29104][CORE][TESTS] Fix PipedRDDSuite to use `eventually` to check thread termination ### What changes were proposed in this pull request? `PipedRDD` will invoke `stdinWriterThread.interrupt()` at task completion, and `obj.wait` will get `InterruptedException`. However, there exists a possibility which the thread termination gets delayed because the thread starts from `obj.wait()` with that exception. To prevent test flakiness, we need to use `eventually`. Also, This PR fixes the typo in code comment and variable name. ### Why are the changes needed? ``` - stdin writer thread should be exited when task is finished *** FAILED *** Some(Thread[stdin writer for List(cat),5,]) was not empty (PipedRDDSuite.scala:107) ``` - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/6867/testReport/junit/org.apache.spark.rdd/PipedRDDSuite/stdin_writer_thread_should_be_exited_when_task_is_finished/ ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manual. We can reproduce the same failure like Jenkins if we catch `InterruptedException` and sleep longer than the `eventually` timeout inside the test code. The following is the example to reproduce it. ```scala val nums = sc.makeRDD(Array(1, 2, 3, 4), 1).map { x => try { obj.synchronized { obj.wait() // make the thread waits here. } } catch { case ie: InterruptedException => Thread.sleep(15000) throw ie } x } ``` Closes #25808 from dongjoon-hyun/SPARK-29104. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- .../org/apache/spark/rdd/PipedRDDSuite.scala | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 69739a2e58481..7d419579a36d0 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -21,16 +21,18 @@ import java.io.File import scala.collection.JavaConverters._ import scala.collection.Map +import scala.concurrent.duration._ import scala.io.Codec import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.{FileSplit, JobConf, TextInputFormat} +import org.scalatest.concurrent.Eventually import org.apache.spark._ import org.apache.spark.util.Utils -class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { +class PipedRDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { val envCommand = if (Utils.isWindows) { "cmd.exe /C set" } else { @@ -100,11 +102,16 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { assert(result.collect().length === 0) - // collect stderr writer threads - val stderrWriterThread = Thread.getAllStackTraces.keySet().asScala - .find { _.getName.startsWith(PipedRDD.STDIN_WRITER_THREAD_PREFIX) } - - assert(stderrWriterThread.isEmpty) + // SPARK-29104 PipedRDD will invoke `stdinWriterThread.interrupt()` at task completion, + // and `obj.wait` will get InterruptedException. However, there exists a possibility + // which the thread termination gets delayed because the thread starts from `obj.wait()` + // with that exception. To prevent test flakiness, we need to use `eventually`. + eventually(timeout(10.seconds), interval(1.second)) { + // collect stdin writer threads + val stdinWriterThread = Thread.getAllStackTraces.keySet().asScala + .find { _.getName.startsWith(PipedRDD.STDIN_WRITER_THREAD_PREFIX) } + assert(stdinWriterThread.isEmpty) + } } test("advanced pipe") { From 3fc52b5557b4608d8f0ce26d11c1ca3e24c157a2 Mon Sep 17 00:00:00 2001 From: xy_xin Date: Wed, 18 Sep 2019 01:14:14 +0800 Subject: [PATCH 0090/1387] [SPARK-28950][SQL] Refine the code of DELETE ### What changes were proposed in this pull request? This pr refines the code of DELETE, including, 1, make `whereClause` to be optional, in which case DELETE will delete all of the data of a table; 2, add more test cases; 3, some other refines. This is a following-up of SPARK-28351. ### Why are the changes needed? An optional where clause in DELETE respects the SQL standard. ### Does this PR introduce any user-facing change? Yes. But since this is a non-released feature, this change does not have any end-user affects. ### How was this patch tested? New case is added. Closes #25652 from xianyinxin/SPARK-28950. Authored-by: xy_xin Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 14 +++++++-- .../plans/logical/basicLogicalOperators.scala | 2 +- .../logical/sql/DeleteFromStatement.scala | 2 +- .../sql/catalyst/parser/DDLParserSuite.scala | 29 +++++++++++++++++-- .../datasources/v2/DataSourceV2Strategy.scala | 13 +++++---- .../sql/connector/DataSourceV2SQLSuite.scala | 27 +++++++++++++++-- 7 files changed, 73 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 70c0d0e505f6c..20be8e539cf58 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -216,7 +216,7 @@ statement | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration - | DELETE FROM multipartIdentifier tableAlias whereClause #deleteFromTable + | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable | unsupportedHiveNativeCommands .*? #failNativeCommand ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 8556ff4775530..9335be5b239b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -343,12 +343,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) val tableAlias = if (ctx.tableAlias() != null) { val ident = ctx.tableAlias().strictIdentifier() - if (ident != null) { Some(ident.getText) } else { None } + // We do not allow columns aliases after table alias. + if (ctx.tableAlias().identifierList() != null) { + throw new ParseException("Columns aliases is not allowed in DELETE.", + ctx.tableAlias().identifierList()) + } + if (ident != null) Some(ident.getText) else None + } else { + None + } + val predicate = if (ctx.whereClause() != null) { + Some(expression(ctx.whereClause().booleanExpression())) } else { None } - DeleteFromStatement(tableId, tableAlias, expression(ctx.whereClause().booleanExpression())) + DeleteFromStatement(tableId, tableAlias, predicate) } /** 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 017f31760b584..2b6378e7c7268 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 @@ -579,7 +579,7 @@ case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Comm case class DeleteFromTable( child: LogicalPlan, - condition: Expression) extends Command { + condition: Option[Expression]) extends Command { override def children: Seq[LogicalPlan] = child :: Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala index 21e24127eee31..035bc09d31871 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala @@ -23,5 +23,5 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan case class DeleteFromStatement( tableName: Seq[String], tableAlias: Option[String], - condition: Expression) + condition: Option[Expression]) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 2cb01d4e8aa5c..be781081b0112 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -20,10 +20,11 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedRelation, UnresolvedStar} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -764,6 +765,30 @@ class DDLParserSuite extends AnalysisTest { assert(exc.getMessage.contains("INSERT INTO ... IF NOT EXISTS")) } + test("delete from table: delete all") { + parseCompare("DELETE FROM testcat.ns1.ns2.tbl", + DeleteFromStatement( + Seq("testcat", "ns1", "ns2", "tbl"), + None, + None)) + } + + test("delete from table: with alias and where clause") { + parseCompare("DELETE FROM testcat.ns1.ns2.tbl AS t WHERE t.a = 2", + DeleteFromStatement( + Seq("testcat", "ns1", "ns2", "tbl"), + Some("t"), + Some(EqualTo(UnresolvedAttribute("t.a"), Literal(2))))) + } + + test("delete from table: columns aliases is not allowed") { + val exc = intercept[ParseException] { + parsePlan("DELETE FROM testcat.ns1.ns2.tbl AS t(a,b,c,d) WHERE d = 2") + } + + assert(exc.getMessage.contains("Columns aliases is not allowed in DELETE.")) + } + test("show tables") { comparePlans( parsePlan("SHOW TABLES"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 22100c7d3d593..6285159dad9b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -251,16 +251,17 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { OverwritePartitionsDynamicExec(r.table.asWritable, r.options, planLater(query)) :: Nil case DeleteFromTable(r: DataSourceV2Relation, condition) => - if (SubqueryExpression.hasSubquery(condition)) { + if (condition.exists(SubqueryExpression.hasSubquery)) { throw new AnalysisException( s"Delete by condition with subquery is not supported: $condition") } // fail if any filter cannot be converted. correctness depends on removing all matching data. - val filters = splitConjunctivePredicates(condition).map { - f => DataSourceStrategy.translateFilter(f).getOrElse( - throw new AnalysisException(s"Exec delete failed:" + - s" cannot translate expression to source filter: $f")) - }.toArray + val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, r.output) + .flatMap(splitConjunctivePredicates(_).map { + f => DataSourceStrategy.translateFilter(f).getOrElse( + throw new AnalysisException(s"Exec update failed:" + + s" cannot translate expression to source filter: $f")) + }).toArray DeleteFromTableExec(r.table.asDeletable, filters) :: Nil case WriteToContinuousDataSource(writer, query) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index ef484ce6ac362..9eb8e5b3b73fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -992,7 +992,17 @@ class DataSourceV2SQLSuite } } - test("DeleteFrom: basic") { + test("DeleteFrom: basic - delete all") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") + sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") + sql(s"DELETE FROM $t") + checkAnswer(spark.table(t), Seq()) + } + } + + test("DeleteFrom: basic - delete with where clause") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") @@ -1003,12 +1013,23 @@ class DataSourceV2SQLSuite } } - test("DeleteFrom: alias") { + test("DeleteFrom: delete from aliased target table") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") + sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") + sql(s"DELETE FROM $t AS tbl WHERE tbl.id = 2") + checkAnswer(spark.table(t), Seq( + Row(3, "c", 3))) + } + } + + test("DeleteFrom: normalize attribute names") { val t = "testcat.ns1.ns2.tbl" withTable(t) { sql(s"CREATE TABLE $t (id bigint, data string, p int) USING foo PARTITIONED BY (id, p)") sql(s"INSERT INTO $t VALUES (2L, 'a', 2), (2L, 'b', 3), (3L, 'c', 3)") - sql(s"DELETE FROM $t tbl WHERE tbl.id = 2") + sql(s"DELETE FROM $t AS tbl WHERE tbl.ID = 2") checkAnswer(spark.table(t), Seq( Row(3, "c", 3))) } From c6ca66113f8d988a61683ee0e98038b74b7db24f Mon Sep 17 00:00:00 2001 From: sharangk Date: Tue, 17 Sep 2019 14:36:56 -0700 Subject: [PATCH 0091/1387] [SPARK-28814][SQL][DOC] Document SET/RESET in SQL Reference ### What changes were proposed in this pull request? Document SET/REST statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. #### Before: There was no documentation for this. #### After: **SET** ![image](https://user-images.githubusercontent.com/29914590/65037551-94a3c680-d96b-11e9-9d59-9f7af5185e06.png) ![image](https://user-images.githubusercontent.com/29914590/64858792-fb607180-d645-11e9-8a53-8cf87a166fc1.png) **RESET** ![image](https://user-images.githubusercontent.com/29914590/64859019-b12bc000-d646-11e9-8cb4-73dc21830067.png) ### How was this patch tested? Manual Review and Tested using jykyll build --serve Closes #25606 from sharangk/resetDoc. Authored-by: sharangk Signed-off-by: Xiao Li --- docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 18 +++++++- docs/sql-ref-syntax-aux-conf-mgmt-set.md | 49 +++++++++++++++++++++- 2 files changed, 65 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index ad2d7f9a83316..8ee61514ee4ef 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -19,4 +19,20 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +Reset all the properties specific to the current session to their default values. After RESET command, executing SET command will output empty. + +### Syntax +{% highlight sql %} +RESET +{% endhighlight %} + + +### Examples +{% highlight sql %} +-- Reset all the properties specific to the current session to their default values. +RESET; +{% endhighlight %} + +### Related Statements +- [SET](sql-ref-syntax-aux-conf-mgmt-set.html) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-set.md b/docs/sql-ref-syntax-aux-conf-mgmt-set.md index c38d68dbb4f1d..f05dde3f567ee 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-set.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-set.md @@ -19,4 +19,51 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +The SET command sets a property, returns the value of an existing property or returns all SQLConf properties with value and meaning. + +### Syntax +{% highlight sql %} +SET +SET [ -v ] +SET property_key[ = property_value ] +{% endhighlight %} + +### Parameters +
    +
    -v
    +
    Outputs the key, value and meaning of existing SQLConf properties.
    +
    + +
    +
    property_key
    +
    Returns the value of specified property key.
    +
    + +
    +
    property_key=property_value
    +
    Sets the value for a given property key. If an old value exists for a given property key, then it gets overridden by the new value.
    +
    + +### Examples +{% highlight sql %} +-- Set a property. +SET spark.sql.variable.substitute=false; + +-- List all SQLConf properties with value and meaning. +SET -v; + +-- List all SQLConf properties with value for current session. +SET; + +-- List the value of specified property key. +SET spark.sql.variable.substitute; + +--------------------------------+--------+ + | key | value | + +--------------------------------+--------+ + | spark.sql.variable.substitute | false | + +--------------------------------+--------+ +{% endhighlight %} + +### Related Statements +- [RESET](sql-ref-syntax-aux-conf-mgmt-reset.html) From dd32476a8250e82df554683e195c355459d10a5d Mon Sep 17 00:00:00 2001 From: sharangk Date: Tue, 17 Sep 2019 14:40:08 -0700 Subject: [PATCH 0092/1387] [SPARK-28792][SQL][DOC] Document CREATE DATABASE statement in SQL Reference ### What changes were proposed in this pull request? Document CREATE DATABASE statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. ### Before: There was no documentation for this. ### After: ![image](https://user-images.githubusercontent.com/29914590/65037831-290e2900-d96c-11e9-8563-92e5379c3ad1.png) ![image](https://user-images.githubusercontent.com/29914590/64858915-55f9cd80-d646-11e9-91a9-16c52b1daa56.png) ### How was this patch tested? Manual Review and Tested using jykyll build --serve Closes #25595 from sharangk/createDbDoc. Lead-authored-by: sharangk Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- docs/sql-ref-syntax-ddl-create-database.md | 59 +++++++++++++++++++++- 1 file changed, 58 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-create-database.md b/docs/sql-ref-syntax-ddl-create-database.md index bbcd34a6d6853..ed0bbf629b027 100644 --- a/docs/sql-ref-syntax-ddl-create-database.md +++ b/docs/sql-ref-syntax-ddl-create-database.md @@ -19,4 +19,61 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +Creates a database with the specified name. If database with the same name already exists, an exception will be thrown. + +### Syntax +{% highlight sql %} +CREATE {DATABASE | SCHEMA} [ IF NOT EXISTS ] database_name + [ COMMENT database_comment ] + [ LOCATION database_directory ] + [ WITH DBPROPERTIES (property_name=property_value [ , ...]) ] +{% endhighlight %} + +### Parameters +
    +
    database_name
    +
    Specifies the name of the database to be created.
    + +
    IF NOT EXISTS
    +
    Creates a database with the given name if it doesn't exists. If a database with the same name already exists, nothing will happen.
    + +
    database_directory
    +
    Path of the file system in which the specified database is to be created. If the specified path does not exist in the underlying file system, this command creates a directory with the path. If the location is not specified, the database will be created in the default warehouse directory, whose path is configured by the static configuration spark.sql.warehouse.dir.
    + +
    database_comment
    +
    Specifies the description for the database.
    + +
    WITH DBPROPERTIES (property_name=property_value [ , ...])
    +
    Specifies the properties for the database in key-value pairs.
    +
    + +### Examples +{% highlight sql %} +-- Create database `customer_db`. This throws exception if database with name customer_db +-- already exists. +CREATE DATABASE customer_db; + +-- Create database `customer_db` only if database with same name doesn't exist. +CREATE DATABASE IF NOT EXISTS customer_db; + +-- Create database `customer_db` only if database with same name doesn't exist with +-- `Comments`,`Specific Location` and `Database properties`. +CREATE DATABASE IF NOT EXISTS customer_db COMMENT 'This is customer database' LOCATION '/user' + WITH DBPROPERTIES (ID=001, Name='John'); + +-- Verify that properties are set. +DESCRIBE DATABASE EXTENDED customer_db; + +----------------------------+-----------------------------+ + | database_description_item | database_description_value | + +----------------------------+-----------------------------+ + | Database Name | customer_db | + | Description | This is customer database | + | Location | hdfs://hacluster/user | + | Properties | ((ID,001), (Name,John)) | + +----------------------------+-----------------------------+ +{% endhighlight %} + +### Related Statements +- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) +- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) From 02db706090c11c3ffc056be3ca1baf3affe5788a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Sep 2019 15:09:16 -0700 Subject: [PATCH 0093/1387] [SPARK-29115][SQL][TEST] Add benchmarks for make_date() and make_timestamp() ### What changes were proposed in this pull request? Added new benchmarks for `make_date()` and `make_timestamp()` to detect performance issues, and figure out functions speed on foldable arguments. - `make_date()` is benchmarked on fully foldable arguments. - `make_timestamp()` is benchmarked on corner case `60.0`, foldable time fields and foldable date. ### Why are the changes needed? To find out inputs where `make_date()` and `make_timestamp()` have performance problems. This should be useful in the future optimizations of the functions and users apps. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the benchmark and manually checking generated dates/timestamps. Closes #25813 from MaxGekk/make_datetime-benchmark. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../MakeDateTimeBenchmark-results.txt | 22 ++++ .../benchmark/MakeDateTimeBenchmark.scala | 120 ++++++++++++++++++ 2 files changed, 142 insertions(+) create mode 100644 sql/core/benchmarks/MakeDateTimeBenchmark-results.txt create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt new file mode 100644 index 0000000000000..a3c89d643e912 --- /dev/null +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt @@ -0,0 +1,22 @@ +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +prepare make_date() 2149 2289 196 46.5 21.5 1.0X +make_date(2019, 9, 16) 1829 1868 58 54.7 18.3 1.2X +make_date(*, *, *) 3180 3339 139 31.4 31.8 0.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +prepare make_timestamp() 2950 3025 96 0.3 2950.3 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 45 47 1 22.1 45.2 65.3X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 42 42 1 24.0 41.7 70.8X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 41 42 1 24.2 41.3 71.4X +make_timestamp(*, *, *, 3, 4, 50.123456) 252 256 7 4.0 251.5 11.7X +make_timestamp(*, *, *, *, *, 0) 225 227 3 4.5 224.6 13.1X +make_timestamp(*, *, *, *, *, 60.0) 230 233 2 4.3 230.4 12.8X +make_timestamp(2019, 1, 2, *, *, *) 3078 3118 35 0.3 3078.5 1.0X +make_timestamp(*, *, *, *, *, *) 3092 3109 17 0.3 3092.4 1.0X + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala new file mode 100644 index 0000000000000..7f7908544693f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.internal.SQLConf + +/** + * Synthetic benchmark for the make_date() and make_timestamp() functions. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "sql/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/MakeDateTimeBenchmark-results.txt". + * }}} + */ +object MakeDateTimeBenchmark extends SqlBasedBenchmark { + + private def doBenchmark(cardinality: Long, exprs: String*): Unit = { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + spark + .range(0, cardinality, 1, 1) + .selectExpr(exprs: _*) + .write + .format("noop") + .save() + } + } + + private def run(benchmark: Benchmark, cardinality: Long, name: String, exprs: String*): Unit = { + benchmark.addCase(name, numIters = 3) { _ => doBenchmark(cardinality, exprs: _*) } + } + + private val ymdExprs = Seq("(2000 + (id % 30))", "((id % 12) + 1)", "((id % 27) + 1)") + + private def benchmarkMakeDate(cardinality: Long): Unit = { + val benchmark = new Benchmark("make_date()", cardinality, output = output) + val args = ymdExprs + + run(benchmark, cardinality, "prepare make_date()", args: _*) + val foldableExpr = "make_date(2019, 9, 16)" + run(benchmark, cardinality, foldableExpr, foldableExpr) + run( + benchmark, + cardinality, + "make_date(*, *, *)", + "make_date" + args.mkString("(", ",", ")")) + + benchmark.run() + } + + private def benchmarkMakeTimestamp(cardinality: Long): Unit = { + val benchmark = new Benchmark("make_timestamp()", cardinality, output = output) + val hmExprs = Seq("id % 24", "id % 60") + val hmsExprs = hmExprs ++ Seq("cast((id % 60000000) / 1000000.0 as decimal(8, 6))") + val args = ymdExprs ++ hmsExprs + + run( + benchmark, + cardinality, + "prepare make_timestamp()", + args: _*) + var foldableExpr = "make_timestamp(2019, 1, 2, 3, 4, 50.123456)" + run(benchmark, cardinality, foldableExpr, foldableExpr) + foldableExpr = "make_timestamp(2019, 1, 2, 3, 4, 60.000000)" + run(benchmark, cardinality, foldableExpr, foldableExpr) + foldableExpr = "make_timestamp(2019, 12, 31, 23, 59, 60.00)" + run(benchmark, cardinality, foldableExpr, foldableExpr) + run( + benchmark, + cardinality, + "make_timestamp(*, *, *, 3, 4, 50.123456)", + s"make_timestamp(${ymdExprs.mkString(",")}, 3, 4, 50.123456)") + run( + benchmark, + cardinality, + "make_timestamp(*, *, *, *, *, 0)", + s"make_timestamp(" + (ymdExprs ++ hmExprs).mkString(", ") + ", 0)") + run( + benchmark, + cardinality, + "make_timestamp(*, *, *, *, *, 60.0)", + s"make_timestamp(" + (ymdExprs ++ hmExprs).mkString(", ") + ", 60.0)") + run( + benchmark, + cardinality, + "make_timestamp(2019, 1, 2, *, *, *)", + s"make_timestamp(2019, 1, 2, ${hmsExprs.mkString(",")})") + run( + benchmark, + cardinality, + "make_timestamp(*, *, *, *, *, *)", + s"make_timestamp" + args.mkString("(", ", ", ")")) + + benchmark.run() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + benchmarkMakeDate(100000000L) + benchmarkMakeTimestamp(1000000L) + } +} From 71e7516132147b200db65a202b8cf74108958430 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 17 Sep 2019 15:30:18 -0700 Subject: [PATCH 0094/1387] [SPARK-29027][TESTS] KafkaDelegationTokenSuite fix when loopback canonical host name differs from localhost ### What changes were proposed in this pull request? `KafkaDelegationTokenSuite` fails on different platforms with the following problem: ``` 19/09/11 11:07:42.690 pool-1-thread-1-SendThread(localhost:44965) DEBUG ZooKeeperSaslClient: creating sasl client: Client=zkclient/localhostEXAMPLE.COM;service=zookeeper;serviceHostname=localhost.localdomain ... NIOServerCxn.Factory:localhost/127.0.0.1:0: Zookeeper Server failed to create a SaslServer to interact with a client during session initiation: javax.security.sasl.SaslException: Failure to initialize security context [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos credentails)] at com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:125) at com.sun.security.sasl.gsskerb.FactoryImpl.createSaslServer(FactoryImpl.java:85) at javax.security.sasl.Sasl.createSaslServer(Sasl.java:524) at org.apache.zookeeper.util.SecurityUtils$2.run(SecurityUtils.java:233) at org.apache.zookeeper.util.SecurityUtils$2.run(SecurityUtils.java:229) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.zookeeper.util.SecurityUtils.createSaslServer(SecurityUtils.java:228) at org.apache.zookeeper.server.ZooKeeperSaslServer.createSaslServer(ZooKeeperSaslServer.java:44) at org.apache.zookeeper.server.ZooKeeperSaslServer.(ZooKeeperSaslServer.java:38) at org.apache.zookeeper.server.NIOServerCnxn.(NIOServerCnxn.java:100) at org.apache.zookeeper.server.NIOServerCnxnFactory.createConnection(NIOServerCnxnFactory.java:186) at org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:227) at java.lang.Thread.run(Thread.java:748) Caused by: GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos credentails) at sun.security.jgss.krb5.Krb5AcceptCredential.getInstance(Krb5AcceptCredential.java:87) at sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:127) at sun.security.jgss.GSSManagerImpl.getCredentialElement(GSSManagerImpl.java:193) at sun.security.jgss.GSSCredentialImpl.add(GSSCredentialImpl.java:427) at sun.security.jgss.GSSCredentialImpl.(GSSCredentialImpl.java:62) at sun.security.jgss.GSSManagerImpl.createCredential(GSSManagerImpl.java:154) at com.sun.security.sasl.gsskerb.GssKrb5Server.(GssKrb5Server.java:108) ... 13 more NIOServerCxn.Factory:localhost/127.0.0.1:0: Client attempting to establish new session at /127.0.0.1:33742 SyncThread:0: Creating new log file: log.1 SyncThread:0: Established session 0x100003736ae0000 with negotiated timeout 10000 for client /127.0.0.1:33742 pool-1-thread-1-SendThread(localhost:35625): Session establishment complete on server localhost/127.0.0.1:35625, sessionid = 0x100003736ae0000, negotiated timeout = 10000 pool-1-thread-1-SendThread(localhost:35625): ClientCnxn:sendSaslPacket:length=0 pool-1-thread-1-SendThread(localhost:35625): saslClient.evaluateChallenge(len=0) pool-1-thread-1-EventThread: zookeeper state changed (SyncConnected) NioProcessor-1: No server entry found for kerberos principal name zookeeper/localhost.localdomainEXAMPLE.COM NioProcessor-1: No server entry found for kerberos principal name zookeeper/localhost.localdomainEXAMPLE.COM NioProcessor-1: Server not found in Kerberos database (7) NioProcessor-1: Server not found in Kerberos database (7) ``` The problem reproducible if the `localhost` and `localhost.localdomain` order exhanged: ``` [systestgsomogyi-build spark]$ cat /etc/hosts 127.0.0.1 localhost.localdomain localhost localhost4 localhost4.localdomain4 ::1 localhost.localdomain localhost localhost6 localhost6.localdomain6 ``` The main problem is that `ZkClient` connects to the canonical loopback address (which is not necessarily `localhost`). ### Why are the changes needed? `KafkaDelegationTokenSuite` failed in some environments. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing unit tests on different platforms. Closes #25803 from gaborgsomogyi/SPARK-29027. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../spark/sql/kafka010/KafkaTestUtils.scala | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index f7114129a3cdc..ee3eb43967069 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.kafka010 import java.io.{File, IOException} import java.lang.{Integer => JInt} -import java.net.InetSocketAddress +import java.net.{InetAddress, InetSocketAddress} import java.nio.charset.StandardCharsets import java.util.{Collections, Map => JMap, Properties, UUID} import java.util.concurrent.TimeUnit @@ -68,10 +68,13 @@ class KafkaTestUtils( private val JAVA_AUTH_CONFIG = "java.security.auth.login.config" + private val localCanonicalHostName = InetAddress.getLoopbackAddress().getCanonicalHostName() + logInfo(s"Local host name is $localCanonicalHostName") + private var kdc: MiniKdc = _ // Zookeeper related configurations - private val zkHost = "localhost" + private val zkHost = localCanonicalHostName private var zkPort: Int = 0 private val zkConnectionTimeout = 60000 private val zkSessionTimeout = 10000 @@ -80,12 +83,12 @@ class KafkaTestUtils( private var zkUtils: ZkUtils = _ // Kafka broker related configurations - private val brokerHost = "localhost" + private val brokerHost = localCanonicalHostName private var brokerPort = 0 private var brokerConf: KafkaConfig = _ private val brokerServiceName = "kafka" - private val clientUser = "client/localhost" + private val clientUser = s"client/$localCanonicalHostName" private var clientKeytabFile: File = _ // Kafka broker server @@ -139,17 +142,17 @@ class KafkaTestUtils( assert(kdcReady, "KDC should be set up beforehand") val baseDir = Utils.createTempDir() - val zkServerUser = "zookeeper/localhost" + val zkServerUser = s"zookeeper/$localCanonicalHostName" val zkServerKeytabFile = new File(baseDir, "zookeeper.keytab") kdc.createPrincipal(zkServerKeytabFile, zkServerUser) logDebug(s"Created keytab file: ${zkServerKeytabFile.getAbsolutePath()}") - val zkClientUser = "zkclient/localhost" + val zkClientUser = s"zkclient/$localCanonicalHostName" val zkClientKeytabFile = new File(baseDir, "zkclient.keytab") kdc.createPrincipal(zkClientKeytabFile, zkClientUser) logDebug(s"Created keytab file: ${zkClientKeytabFile.getAbsolutePath()}") - val kafkaServerUser = "kafka/localhost" + val kafkaServerUser = s"kafka/$localCanonicalHostName" val kafkaServerKeytabFile = new File(baseDir, "kafka.keytab") kdc.createPrincipal(kafkaServerKeytabFile, kafkaServerUser) logDebug(s"Created keytab file: ${kafkaServerKeytabFile.getAbsolutePath()}") From 197732e1f4b514a323bccb3f561d1502faeafd0a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 17 Sep 2019 16:53:21 -0700 Subject: [PATCH 0095/1387] [SPARK-29125][INFRA] Add Hadoop 2.7 combination to GitHub Action ### What changes were proposed in this pull request? Until now, we are testing JDK8/11 with Hadoop-3.2. This PR aims to extend the test coverage for JDK8/Hadoop-2.7. ### Why are the changes needed? This will prevent Hadoop 2.7 compile/package issues at PR stage. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? GitHub Action on this PR shows all three combinations now. And, this is irrelevant to Jenkins test. Closes #25824 from dongjoon-hyun/SPARK-29125. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index a4233d113a561..dfed834c1d08e 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -15,7 +15,11 @@ jobs: strategy: matrix: java: [ '1.8', '11' ] - name: Build Spark with JDK ${{ matrix.java }} + hadoop: [ 'hadoop-2.7', 'hadoop-3.2' ] + exclude: + - java: '11' + hadoop: 'hadoop-2.7' + name: Build Spark with JDK ${{ matrix.java }} and ${{ matrix.hadoop }} steps: - uses: actions/checkout@master @@ -27,4 +31,4 @@ jobs: run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" - ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-3.2 -Phadoop-cloud -Djava.version=${{ matrix.java }} package + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} package From 05988b256e8d6d07517a9b0d037649fa8c3fc65d Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Tue, 17 Sep 2019 17:13:50 -0700 Subject: [PATCH 0096/1387] [SPARK-27463][PYTHON] Support Dataframe Cogroup via Pandas UDFs ### What changes were proposed in this pull request? Adds a new cogroup Pandas UDF. This allows two grouped dataframes to be cogrouped together and apply a (pandas.DataFrame, pandas.DataFrame) -> pandas.DataFrame UDF to each cogroup. **Example usage** ``` from pyspark.sql.functions import pandas_udf, PandasUDFType df1 = spark.createDataFrame( [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], ("time", "id", "v1")) df2 = spark.createDataFrame( [(20000101, 1, "x"), (20000101, 2, "y")], ("time", "id", "v2")) pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP) def asof_join(l, r): return pd.merge_asof(l, r, on="time", by="id") df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() ``` +--------+---+---+---+ | time| id| v1| v2| +--------+---+---+---+ |20000101| 1|1.0| x| |20000102| 1|3.0| x| |20000101| 2|2.0| y| |20000102| 2|4.0| y| +--------+---+---+---+ ### How was this patch tested? Added unit test test_pandas_udf_cogrouped_map Closes #24981 from d80tb7/SPARK-27463-poc-arrow-stream. Authored-by: Chris Martin Signed-off-by: Bryan Cutler --- .../spark/api/python/PythonRunner.scala | 2 + python/pyspark/rdd.py | 1 + python/pyspark/serializers.py | 26 ++ python/pyspark/sql/cogroup.py | 98 ++++++ python/pyspark/sql/functions.py | 5 +- python/pyspark/sql/group.py | 12 +- .../tests/test_pandas_udf_cogrouped_map.py | 280 ++++++++++++++++++ python/pyspark/sql/udf.py | 19 ++ python/pyspark/worker.py | 98 +++++- .../sql/catalyst/analysis/Analyzer.scala | 6 + .../logical/pythonLogicalOperators.scala | 19 +- .../spark/sql/RelationalGroupedDataset.scala | 46 ++- .../spark/sql/execution/SparkStrategies.scala | 3 + .../execution/python/ArrowPythonRunner.scala | 76 +---- .../python/BaseArrowPythonRunner.scala | 112 +++++++ .../python/BasePandasGroupExec.scala | 137 +++++++++ .../python/CogroupedArrowPythonRunner.scala | 113 +++++++ .../python/FlatMapCoGroupsInPandasExec.scala | 97 ++++++ .../python/FlatMapGroupsInPandasExec.scala | 98 +----- 19 files changed, 1070 insertions(+), 178 deletions(-) create mode 100644 python/pyspark/sql/cogroup.py create mode 100644 python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index d2a10df7acbd3..277804ec41d98 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -48,6 +48,7 @@ private[spark] object PythonEvalType { val SQL_WINDOW_AGG_PANDAS_UDF = 203 val SQL_SCALAR_PANDAS_ITER_UDF = 204 val SQL_MAP_PANDAS_ITER_UDF = 205 + val SQL_COGROUPED_MAP_PANDAS_UDF = 206 def toString(pythonEvalType: Int): String = pythonEvalType match { case NON_UDF => "NON_UDF" @@ -58,6 +59,7 @@ private[spark] object PythonEvalType { case SQL_WINDOW_AGG_PANDAS_UDF => "SQL_WINDOW_AGG_PANDAS_UDF" case SQL_SCALAR_PANDAS_ITER_UDF => "SQL_SCALAR_PANDAS_ITER_UDF" case SQL_MAP_PANDAS_ITER_UDF => "SQL_MAP_PANDAS_ITER_UDF" + case SQL_COGROUPED_MAP_PANDAS_UDF => "SQL_COGROUPED_MAP_PANDAS_UDF" } } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 16c226f02e633..be0244b7d13e2 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -75,6 +75,7 @@ class PythonEvalType(object): SQL_WINDOW_AGG_PANDAS_UDF = 203 SQL_SCALAR_PANDAS_ITER_UDF = 204 SQL_MAP_PANDAS_ITER_UDF = 205 + SQL_COGROUPED_MAP_PANDAS_UDF = 206 def portable_hash(x): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 00f6081a3b14f..bceb92cb274ae 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -401,6 +401,32 @@ def __repr__(self): return "ArrowStreamPandasUDFSerializer" +class CogroupUDFSerializer(ArrowStreamPandasUDFSerializer): + + def load_stream(self, stream): + """ + Deserialize Cogrouped ArrowRecordBatches to a tuple of Arrow tables and yield as two + lists of pandas.Series. + """ + import pyarrow as pa + dataframes_in_group = None + + while dataframes_in_group is None or dataframes_in_group > 0: + dataframes_in_group = read_int(stream) + + if dataframes_in_group == 2: + batch1 = [batch for batch in ArrowStreamSerializer.load_stream(self, stream)] + batch2 = [batch for batch in ArrowStreamSerializer.load_stream(self, stream)] + yield ( + [self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch1).itercolumns()], + [self.arrow_to_pandas(c) for c in pa.Table.from_batches(batch2).itercolumns()] + ) + + elif dataframes_in_group != 0: + raise ValueError( + 'Invalid number of pandas.DataFrames in group {0}'.format(dataframes_in_group)) + + class BatchedSerializer(Serializer): """ diff --git a/python/pyspark/sql/cogroup.py b/python/pyspark/sql/cogroup.py new file mode 100644 index 0000000000000..9b725e4bafe79 --- /dev/null +++ b/python/pyspark/sql/cogroup.py @@ -0,0 +1,98 @@ +# +# 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 pyspark import since +from pyspark.rdd import PythonEvalType +from pyspark.sql.column import Column +from pyspark.sql.dataframe import DataFrame + + +class CoGroupedData(object): + """ + A logical grouping of two :class:`GroupedData`, + created by :func:`GroupedData.cogroup`. + + .. note:: Experimental + + .. versionadded:: 3.0 + """ + + def __init__(self, gd1, gd2): + self._gd1 = gd1 + self._gd2 = gd2 + self.sql_ctx = gd1.sql_ctx + + @since(3.0) + def apply(self, udf): + """ + Applies a function to each cogroup using a pandas udf and returns the result + as a `DataFrame`. + + The user-defined function should take two `pandas.DataFrame` and return another + `pandas.DataFrame`. For each side of the cogroup, all columns are passed together + as a `pandas.DataFrame` to the user-function and the returned `pandas.DataFrame` + are combined as a :class:`DataFrame`. + + The returned `pandas.DataFrame` can be of arbitrary length and its schema must match the + returnType of the pandas udf. + + .. note:: This function requires a full shuffle. All the data of a cogroup will be loaded + into memory, so the user should be aware of the potential OOM risk if data is skewed + and certain groups are too large to fit in memory. + + .. note:: Experimental + + :param udf: a cogrouped map user-defined function returned by + :func:`pyspark.sql.functions.pandas_udf`. + + >>> from pyspark.sql.functions import pandas_udf, PandasUDFType + >>> df1 = spark.createDataFrame( + ... [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], + ... ("time", "id", "v1")) + >>> df2 = spark.createDataFrame( + ... [(20000101, 1, "x"), (20000101, 2, "y")], + ... ("time", "id", "v2")) + >>> @pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP) + ... def asof_join(l, r): + ... return pd.merge_asof(l, r, on="time", by="id") + >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() + +--------+---+---+---+ + | time| id| v1| v2| + +--------+---+---+---+ + |20000101| 1|1.0| x| + |20000102| 1|3.0| x| + |20000101| 2|2.0| y| + |20000102| 2|4.0| y| + +--------+---+---+---+ + + .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` + + """ + # Columns are special because hasattr always return True + if isinstance(udf, Column) or not hasattr(udf, 'func') \ + or udf.evalType != PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + raise ValueError("Invalid udf: the udf argument must be a pandas_udf of type " + "COGROUPED_MAP.") + all_cols = self._extract_cols(self._gd1) + self._extract_cols(self._gd2) + udf_column = udf(*all_cols) + jdf = self._gd1._jgd.flatMapCoGroupsInPandas(self._gd2._jgd, udf_column._jc.expr()) + return DataFrame(jdf, self.sql_ctx) + + @staticmethod + def _extract_cols(gd): + df = gd._df + return [df[col] for col in df.columns] diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index c7ff2882ed95a..d96c264aa7398 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2814,6 +2814,8 @@ class PandasUDFType(object): GROUPED_MAP = PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF + COGROUPED_MAP = PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF + GROUPED_AGG = PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF MAP_ITER = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF @@ -3320,7 +3322,8 @@ def pandas_udf(f=None, returnType=None, functionType=None): PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, - PythonEvalType.SQL_MAP_PANDAS_ITER_UDF]: + PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]: raise ValueError("Invalid functionType: " "functionType must be one the values from PandasUDFType") diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index ec90ba905ef66..fcad64142485e 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -22,6 +22,7 @@ from pyspark.sql.column import Column, _to_seq from pyspark.sql.dataframe import DataFrame from pyspark.sql.types import * +from pyspark.sql.cogroup import CoGroupedData __all__ = ["GroupedData"] @@ -218,6 +219,15 @@ def pivot(self, pivot_col, values=None): jgd = self._jgd.pivot(pivot_col, values) return GroupedData(jgd, self._df) + @since(3.0) + def cogroup(self, other): + """ + Cogroups this group with another group so that we can run cogrouped operations. + + See :class:`CoGroupedData` for the operations that can be run. + """ + return CoGroupedData(self, other) + @since(2.3) def apply(self, udf): """ @@ -232,7 +242,7 @@ def apply(self, udf): The returned `pandas.DataFrame` can be of arbitrary length and its schema must match the returnType of the pandas udf. - .. note:: This function requires a full shuffle. all the data of a group will be loaded + .. note:: This function requires a full shuffle. All the data of a group will be loaded into memory, so the user should be aware of the potential OOM risk if data is skewed and certain groups are too large to fit in memory. diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py new file mode 100644 index 0000000000000..7f3f7fa3168a7 --- /dev/null +++ b/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py @@ -0,0 +1,280 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import unittest +import sys + +from pyspark.sql.functions import array, explode, col, lit, udf, sum, pandas_udf, PandasUDFType +from pyspark.sql.types import DoubleType, StructType, StructField +from pyspark.testing.sqlutils import ReusedSQLTestCase, have_pandas, have_pyarrow, \ + pandas_requirement_message, pyarrow_requirement_message +from pyspark.testing.utils import QuietTest + +if have_pandas: + import pandas as pd + from pandas.util.testing import assert_frame_equal, assert_series_equal + +if have_pyarrow: + import pyarrow as pa + + +""" +Tests below use pd.DataFrame.assign that will infer mixed types (unicode/str) for column names +from kwargs w/ Python 2, so need to set check_column_type=False and avoid this check +""" +if sys.version < '3': + _check_column_type = False +else: + _check_column_type = True + + +@unittest.skipIf( + not have_pandas or not have_pyarrow, + pandas_requirement_message or pyarrow_requirement_message) +class CoGroupedMapPandasUDFTests(ReusedSQLTestCase): + + @property + def data1(self): + return self.spark.range(10).toDF('id') \ + .withColumn("ks", array([lit(i) for i in range(20, 30)])) \ + .withColumn("k", explode(col('ks')))\ + .withColumn("v", col('k') * 10)\ + .drop('ks') + + @property + def data2(self): + return self.spark.range(10).toDF('id') \ + .withColumn("ks", array([lit(i) for i in range(20, 30)])) \ + .withColumn("k", explode(col('ks'))) \ + .withColumn("v2", col('k') * 100) \ + .drop('ks') + + def test_simple(self): + self._test_merge(self.data1, self.data2) + + def test_left_group_empty(self): + left = self.data1.where(col("id") % 2 == 0) + self._test_merge(left, self.data2) + + def test_right_group_empty(self): + right = self.data2.where(col("id") % 2 == 0) + self._test_merge(self.data1, right) + + def test_different_schemas(self): + right = self.data2.withColumn('v3', lit('a')) + self._test_merge(self.data1, right, 'id long, k int, v int, v2 int, v3 string') + + def test_complex_group_by(self): + left = pd.DataFrame.from_dict({ + 'id': [1, 2, 3], + 'k': [5, 6, 7], + 'v': [9, 10, 11] + }) + + right = pd.DataFrame.from_dict({ + 'id': [11, 12, 13], + 'k': [5, 6, 7], + 'v2': [90, 100, 110] + }) + + left_gdf = self.spark\ + .createDataFrame(left)\ + .groupby(col('id') % 2 == 0) + + right_gdf = self.spark \ + .createDataFrame(right) \ + .groupby(col('id') % 2 == 0) + + @pandas_udf('k long, v long, v2 long', PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l[['k', 'v']], r[['k', 'v2']], on=['k']) + + result = left_gdf \ + .cogroup(right_gdf) \ + .apply(merge_pandas) \ + .sort(['k']) \ + .toPandas() + + expected = pd.DataFrame.from_dict({ + 'k': [5, 6, 7], + 'v': [9, 10, 11], + 'v2': [90, 100, 110] + }) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + def test_empty_group_by(self): + left = self.data1 + right = self.data2 + + @pandas_udf('id long, k int, v int, v2 int', PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l, r, on=['id', 'k']) + + result = left.groupby().cogroup(right.groupby())\ + .apply(merge_pandas) \ + .sort(['id', 'k']) \ + .toPandas() + + left = left.toPandas() + right = right.toPandas() + + expected = pd \ + .merge(left, right, on=['id', 'k']) \ + .sort_values(by=['id', 'k']) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + def test_mixed_scalar_udfs_followed_by_cogrouby_apply(self): + df = self.spark.range(0, 10).toDF('v1') + df = df.withColumn('v2', udf(lambda x: x + 1, 'int')(df['v1'])) \ + .withColumn('v3', pandas_udf(lambda x: x + 2, 'int')(df['v1'])) + + result = df.groupby().cogroup(df.groupby())\ + .apply(pandas_udf(lambda x, y: pd.DataFrame([(x.sum().sum(), y.sum().sum())]), + 'sum1 int, sum2 int', + PandasUDFType.COGROUPED_MAP)).collect() + + self.assertEquals(result[0]['sum1'], 165) + self.assertEquals(result[0]['sum2'], 165) + + def test_with_key_left(self): + self._test_with_key(self.data1, self.data1, isLeft=True) + + def test_with_key_right(self): + self._test_with_key(self.data1, self.data1, isLeft=False) + + def test_with_key_left_group_empty(self): + left = self.data1.where(col("id") % 2 == 0) + self._test_with_key(left, self.data1, isLeft=True) + + def test_with_key_right_group_empty(self): + right = self.data1.where(col("id") % 2 == 0) + self._test_with_key(self.data1, right, isLeft=False) + + def test_with_key_complex(self): + + @pandas_udf('id long, k int, v int, key boolean', PandasUDFType.COGROUPED_MAP) + def left_assign_key(key, l, _): + return l.assign(key=key[0]) + + result = self.data1 \ + .groupby(col('id') % 2 == 0)\ + .cogroup(self.data2.groupby(col('id') % 2 == 0)) \ + .apply(left_assign_key) \ + .sort(['id', 'k']) \ + .toPandas() + + expected = self.data1.toPandas() + expected = expected.assign(key=expected.id % 2 == 0) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + def test_wrong_return_type(self): + with QuietTest(self.sc): + with self.assertRaisesRegexp( + NotImplementedError, + 'Invalid returnType.*cogrouped map Pandas UDF.*MapType'): + pandas_udf( + lambda l, r: l, + 'id long, v map', + PandasUDFType.COGROUPED_MAP) + + def test_wrong_args(self): + # Test that we get a sensible exception invalid values passed to apply + left = self.data1 + right = self.data2 + with QuietTest(self.sc): + # Function rather than a udf + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(lambda l, r: l) + + # Udf missing return type + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(udf(lambda l, r: l, DoubleType())) + + # Pass in expression rather than udf + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id')).apply(left.v + 1) + + # Zero arg function + with self.assertRaisesRegexp(ValueError, 'Invalid function'): + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda: 1, StructType([StructField("d", DoubleType())]))) + + # Udf without PandasUDFType + with self.assertRaisesRegexp(ValueError, 'Invalid udf'): + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda x, y: x, DoubleType())) + + # Udf with incorrect PandasUDFType + with self.assertRaisesRegexp(ValueError, 'Invalid udf.*COGROUPED_MAP'): + left.groupby('id').cogroup(right.groupby('id'))\ + .apply(pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) + + @staticmethod + def _test_with_key(left, right, isLeft): + + @pandas_udf('id long, k int, v int, key long', PandasUDFType.COGROUPED_MAP) + def right_assign_key(key, l, r): + return l.assign(key=key[0]) if isLeft else r.assign(key=key[0]) + + result = left \ + .groupby('id') \ + .cogroup(right.groupby('id')) \ + .apply(right_assign_key) \ + .toPandas() + + expected = left.toPandas() if isLeft else right.toPandas() + expected = expected.assign(key=expected.id) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + @staticmethod + def _test_merge(left, right, output_schema='id long, k int, v int, v2 int'): + + @pandas_udf(output_schema, PandasUDFType.COGROUPED_MAP) + def merge_pandas(l, r): + return pd.merge(l, r, on=['id', 'k']) + + result = left \ + .groupby('id') \ + .cogroup(right.groupby('id')) \ + .apply(merge_pandas)\ + .sort(['id', 'k']) \ + .toPandas() + + left = left.toPandas() + right = right.toPandas() + + expected = pd \ + .merge(left, right, on=['id', 'k']) \ + .sort_values(by=['id', 'k']) + + assert_frame_equal(expected, result, check_column_type=_check_column_type) + + +if __name__ == "__main__": + from pyspark.sql.tests.test_pandas_udf_cogrouped_map import * + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 188ec2634974a..c4d7c1ed205f1 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -42,6 +42,7 @@ def _create_udf(f, returnType, evalType): if evalType in (PythonEvalType.SQL_SCALAR_PANDAS_UDF, PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, PythonEvalType.SQL_MAP_PANDAS_ITER_UDF): @@ -65,6 +66,13 @@ def _create_udf(f, returnType, evalType): "Invalid function: pandas_udfs with function type GROUPED_MAP " "must take either one argument (data) or two arguments (key, data).") + if evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF \ + and len(argspec.args) not in (2, 3): + raise ValueError( + "Invalid function: pandas_udfs with function type COGROUPED_MAP " + "must take either two arguments (left, right) " + "or three arguments (key, left, right).") + # Set the name of the UserDefinedFunction object to be the name of function f udf_obj = UserDefinedFunction( f, returnType=returnType, name=None, evalType=evalType, deterministic=True) @@ -147,6 +155,17 @@ def returnType(self): else: raise TypeError("Invalid returnType for map iterator Pandas " "UDFs: returnType must be a StructType.") + elif self.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + if isinstance(self._returnType_placeholder, StructType): + try: + to_arrow_type(self._returnType_placeholder) + except TypeError: + raise NotImplementedError( + "Invalid returnType with cogrouped map Pandas UDFs: " + "%s is not supported" % str(self._returnType_placeholder)) + else: + raise TypeError("Invalid returnType for cogrouped map Pandas " + "UDFs: returnType must be a StructType.") elif self.evalType == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: try: # StructType is not yet allowed as a return type, explicitly check here to fail fast diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 7f38c27360ed9..086202de2c68b 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -39,7 +39,7 @@ from pyspark.rdd import PythonEvalType from pyspark.serializers import write_with_length, write_int, read_long, read_bool, \ write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ - BatchedSerializer, ArrowStreamPandasUDFSerializer + BatchedSerializer, ArrowStreamPandasUDFSerializer, CogroupUDFSerializer from pyspark.sql.types import to_arrow_type, StructType from pyspark.util import _get_argspec, fail_on_stopiteration from pyspark import shuffle @@ -121,6 +121,33 @@ def verify_result_type(result): map(verify_result_type, f(*iterator))) +def wrap_cogrouped_map_pandas_udf(f, return_type, argspec): + + def wrapped(left_key_series, left_value_series, right_key_series, right_value_series): + import pandas as pd + + left_df = pd.concat(left_value_series, axis=1) + right_df = pd.concat(right_value_series, axis=1) + + if len(argspec.args) == 2: + result = f(left_df, right_df) + elif len(argspec.args) == 3: + key_series = left_key_series if not left_df.empty else right_key_series + key = tuple(s[0] for s in key_series) + result = f(key, left_df, right_df) + if not isinstance(result, pd.DataFrame): + raise TypeError("Return type of the user-defined function should be " + "pandas.DataFrame, but is {}".format(type(result))) + if not len(result.columns) == len(return_type): + raise RuntimeError( + "Number of columns of the returned pandas.DataFrame " + "doesn't match specified schema. " + "Expected: {} Actual: {}".format(len(return_type), len(result.columns))) + return result + + return lambda kl, vl, kr, vr: [(wrapped(kl, vl, kr, vr), to_arrow_type(return_type))] + + def wrap_grouped_map_pandas_udf(f, return_type, argspec): def wrapped(key_series, value_series): @@ -244,6 +271,9 @@ def read_single_udf(pickleSer, infile, eval_type, runner_conf, udf_index): elif eval_type == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF: argspec = _get_argspec(chained_func) # signature was lost when wrapping it return arg_offsets, wrap_grouped_map_pandas_udf(func, return_type, argspec) + elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + argspec = _get_argspec(chained_func) # signature was lost when wrapping it + return arg_offsets, wrap_cogrouped_map_pandas_udf(func, return_type, argspec) elif eval_type == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: return arg_offsets, wrap_grouped_agg_pandas_udf(func, return_type) elif eval_type == PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF: @@ -258,6 +288,7 @@ def read_udfs(pickleSer, infile, eval_type): runner_conf = {} if eval_type in (PythonEvalType.SQL_SCALAR_PANDAS_UDF, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, @@ -280,13 +311,16 @@ def read_udfs(pickleSer, infile, eval_type): "spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName", "true")\ .lower() == "true" - # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of - # pandas Series. See SPARK-27240. - df_for_struct = (eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF or - eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF or - eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF) - ser = ArrowStreamPandasUDFSerializer(timezone, safecheck, assign_cols_by_name, - df_for_struct) + if eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + ser = CogroupUDFSerializer(timezone, safecheck, assign_cols_by_name) + else: + # Scalar Pandas UDF handles struct type arguments as pandas DataFrames instead of + # pandas Series. See SPARK-27240. + df_for_struct = (eval_type == PythonEvalType.SQL_SCALAR_PANDAS_UDF or + eval_type == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF or + eval_type == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF) + ser = ArrowStreamPandasUDFSerializer(timezone, safecheck, assign_cols_by_name, + df_for_struct) else: ser = BatchedSerializer(PickleSerializer(), 100) @@ -343,6 +377,32 @@ def map_batch(batch): # profiling is not supported for UDF return func, None, ser, ser + def extract_key_value_indexes(grouped_arg_offsets): + """ + Helper function to extract the key and value indexes from arg_offsets for the grouped and + cogrouped pandas udfs. See BasePandasGroupExec.resolveArgOffsets for equivalent scala code. + + :param grouped_arg_offsets: List containing the key and value indexes of columns of the + DataFrames to be passed to the udf. It consists of n repeating groups where n is the + number of DataFrames. Each group has the following format: + group[0]: length of group + group[1]: length of key indexes + group[2.. group[1] +2]: key attributes + group[group[1] +3 group[0]]: value attributes + """ + parsed = [] + idx = 0 + while idx < len(grouped_arg_offsets): + offsets_len = grouped_arg_offsets[idx] + idx += 1 + offsets = grouped_arg_offsets[idx: idx + offsets_len] + split_index = offsets[0] + 1 + offset_keys = offsets[1: split_index] + offset_values = offsets[split_index:] + parsed.append([offset_keys, offset_values]) + idx += offsets_len + return parsed + udfs = {} call_udf = [] mapper_str = "" @@ -359,10 +419,24 @@ def map_batch(batch): arg_offsets, udf = read_single_udf( pickleSer, infile, eval_type, runner_conf, udf_index=0) udfs['f'] = udf - split_offset = arg_offsets[0] + 1 - arg0 = ["a[%d]" % o for o in arg_offsets[1: split_offset]] - arg1 = ["a[%d]" % o for o in arg_offsets[split_offset:]] - mapper_str = "lambda a: f([%s], [%s])" % (", ".join(arg0), ", ".join(arg1)) + parsed_offsets = extract_key_value_indexes(arg_offsets) + keys = ["a[%d]" % (o,) for o in parsed_offsets[0][0]] + vals = ["a[%d]" % (o, ) for o in parsed_offsets[0][1]] + mapper_str = "lambda a: f([%s], [%s])" % (", ".join(keys), ", ".join(vals)) + elif eval_type == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: + # We assume there is only one UDF here because cogrouped map doesn't + # support combining multiple UDFs. + assert num_udfs == 1 + arg_offsets, udf = read_single_udf( + pickleSer, infile, eval_type, runner_conf, udf_index=0) + udfs['f'] = udf + parsed_offsets = extract_key_value_indexes(arg_offsets) + df1_keys = ["a[0][%d]" % (o, ) for o in parsed_offsets[0][0]] + df1_vals = ["a[0][%d]" % (o, ) for o in parsed_offsets[0][1]] + df2_keys = ["a[1][%d]" % (o, ) for o in parsed_offsets[1][0]] + df2_vals = ["a[1][%d]" % (o, ) for o in parsed_offsets[1][1]] + mapper_str = "lambda a: f([%s], [%s], [%s], [%s])" % ( + ", ".join(df1_keys), ", ".join(df1_vals), ", ".join(df2_keys), ", ".join(df2_vals)) else: # Create function like this: # lambda a: (f0(a[0]), f1(a[1], a[2]), f2(a[3])) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 413c75d2581d1..0a13a34720846 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1191,6 +1191,12 @@ class Analyzer( // To resolve duplicate expression IDs for Join and Intersect case j @ Join(left, right, _, _, _) if !j.duplicateResolved => j.copy(right = dedupRight(left, right)) + case f @ FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, _, _, left, right) => + val leftRes = leftAttributes + .map(x => resolveExpressionBottomUp(x, left).asInstanceOf[Attribute]) + val rightRes = rightAttributes + .map(x => resolveExpressionBottomUp(x, right).asInstanceOf[Attribute]) + f.copy(leftAttributes = leftRes, rightAttributes = rightRes) // intersect/except will be rewritten to join at the begininng of optimizer. Here we need to // deduplicate the right side plan, so that we won't produce an invalid self-join later. case i @ Intersect(left, right, _) if !i.duplicateResolved => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index dc2185194d84e..c4f741cd2cec8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF} /** - * FlatMap groups using an udf: pandas.Dataframe -> pandas.DataFrame. + * FlatMap groups using a udf: pandas.Dataframe -> pandas.DataFrame. * This is used by DataFrame.groupby().apply(). */ case class FlatMapGroupsInPandas( @@ -40,7 +40,7 @@ case class FlatMapGroupsInPandas( } /** - * Map partitions using an udf: iter(pandas.Dataframe) -> iter(pandas.DataFrame). + * Map partitions using a udf: iter(pandas.Dataframe) -> iter(pandas.DataFrame). * This is used by DataFrame.mapInPandas() */ case class MapInPandas( @@ -51,6 +51,21 @@ case class MapInPandas( override val producedAttributes = AttributeSet(output) } +/** + * Flatmap cogroups using a udf: pandas.Dataframe, pandas.Dataframe -> pandas.Dataframe + * This is used by DataFrame.groupby().cogroup().apply(). + */ +case class FlatMapCoGroupsInPandas( + leftAttributes: Seq[Attribute], + rightAttributes: Seq[Attribute], + functionExpr: Expression, + output: Seq[Attribute], + left: LogicalPlan, + right: LogicalPlan) extends BinaryNode { + + override val producedAttributes = AttributeSet(output) +} + trait BaseEvalPython extends UnaryNode { def udfs: Seq[PythonUDF] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index e85636d82a62c..f6d13be0e89be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -47,8 +47,8 @@ import org.apache.spark.sql.types.{NumericType, StructType} */ @Stable class RelationalGroupedDataset protected[sql]( - df: DataFrame, - groupingExprs: Seq[Expression], + val df: DataFrame, + val groupingExprs: Seq[Expression], groupType: RelationalGroupedDataset.GroupType) { private[this] def toDF(aggExprs: Seq[Expression]): DataFrame = { @@ -523,6 +523,48 @@ class RelationalGroupedDataset protected[sql]( Dataset.ofRows(df.sparkSession, plan) } + /** + * Applies a vectorized python user-defined function to each cogrouped data. + * The user-defined function defines a transformation: + * `pandas.DataFrame`, `pandas.DataFrame` -> `pandas.DataFrame`. + * For each group in the cogrouped data, all elements in the group are passed as a + * `pandas.DataFrame` and the results for all cogroups are combined into a new [[DataFrame]]. + * + * This function uses Apache Arrow as serialization format between Java executors and Python + * workers. + */ + private[sql] def flatMapCoGroupsInPandas( + r: RelationalGroupedDataset, + expr: PythonUDF): DataFrame = { + require(expr.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + "Must pass a cogrouped map udf") + require(expr.dataType.isInstanceOf[StructType], + s"The returnType of the udf must be a ${StructType.simpleString}") + + val leftGroupingNamedExpressions = groupingExprs.map { + case ne: NamedExpression => ne + case other => Alias(other, other.toString)() + } + + val rightGroupingNamedExpressions = r.groupingExprs.map { + case ne: NamedExpression => ne + case other => Alias(other, other.toString)() + } + + val leftAttributes = leftGroupingNamedExpressions.map(_.toAttribute) + val rightAttributes = rightGroupingNamedExpressions.map(_.toAttribute) + + val leftChild = df.logicalPlan + val rightChild = r.df.logicalPlan + + val left = Project(leftGroupingNamedExpressions ++ leftChild.output, leftChild) + val right = Project(rightGroupingNamedExpressions ++ rightChild.output, rightChild) + + val output = expr.dataType.asInstanceOf[StructType].toAttributes + val plan = FlatMapCoGroupsInPandas(leftAttributes, rightAttributes, expr, output, left, right) + Dataset.ofRows(df.sparkSession, plan) + } + override def toString: String = { val builder = new StringBuilder builder.append("RelationalGroupedDataset: [grouping expressions: [") 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 85469bf2401d4..a2f45898d273f 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 @@ -682,6 +682,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { f, p, b, is, ot, planLater(child)) :: Nil case logical.FlatMapGroupsInPandas(grouping, func, output, child) => execution.python.FlatMapGroupsInPandasExec(grouping, func, output, planLater(child)) :: Nil + case logical.FlatMapCoGroupsInPandas(leftGroup, rightGroup, func, output, left, right) => + execution.python.FlatMapCoGroupsInPandasExec( + leftGroup, rightGroup, func, output, planLater(left), planLater(right)) :: Nil case logical.MapInPandas(func, output, child) => execution.python.MapInPandasExec(func, output, planLater(child)) :: Nil case logical.MapElements(f, _, _, objAttr, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 5101f7e871af2..fcf68467460bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -19,12 +19,9 @@ package org.apache.spark.sql.execution.python import java.io._ import java.net._ -import java.util.concurrent.atomic.AtomicBoolean - -import scala.collection.JavaConverters._ import org.apache.arrow.vector.VectorSchemaRoot -import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} +import org.apache.arrow.vector.ipc.ArrowStreamWriter import org.apache.spark._ import org.apache.spark.api.python._ @@ -33,7 +30,6 @@ import org.apache.spark.sql.execution.arrow.ArrowWriter import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} import org.apache.spark.util.Utils /** @@ -46,7 +42,7 @@ class ArrowPythonRunner( schema: StructType, timeZoneId: String, conf: Map[String, String]) - extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch]( + extends BaseArrowPythonRunner[Iterator[InternalRow]]( funcs, evalType, argOffsets) { override val bufferSize: Int = SQLConf.get.pandasUDFBufferSize @@ -119,72 +115,4 @@ class ArrowPythonRunner( } } - protected override def newReaderIterator( - stream: DataInputStream, - writerThread: WriterThread, - startTime: Long, - env: SparkEnv, - worker: Socket, - releasedOrClosed: AtomicBoolean, - context: TaskContext): Iterator[ColumnarBatch] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { - - private val allocator = ArrowUtils.rootAllocator.newChildAllocator( - s"stdin reader for $pythonExec", 0, Long.MaxValue) - - private var reader: ArrowStreamReader = _ - private var root: VectorSchemaRoot = _ - private var schema: StructType = _ - private var vectors: Array[ColumnVector] = _ - - context.addTaskCompletionListener[Unit] { _ => - if (reader != null) { - reader.close(false) - } - allocator.close() - } - - private var batchLoaded = true - - protected override def read(): ColumnarBatch = { - if (writerThread.exception.isDefined) { - throw writerThread.exception.get - } - try { - if (reader != null && batchLoaded) { - batchLoaded = reader.loadNextBatch() - if (batchLoaded) { - val batch = new ColumnarBatch(vectors) - batch.setNumRows(root.getRowCount) - batch - } else { - reader.close(false) - allocator.close() - // Reach end of stream. Call `read()` again to read control data. - read() - } - } else { - stream.readInt() match { - case SpecialLengths.START_ARROW_STREAM => - reader = new ArrowStreamReader(stream, allocator) - root = reader.getVectorSchemaRoot() - schema = ArrowUtils.fromArrowSchema(root.getSchema()) - vectors = root.getFieldVectors().asScala.map { vector => - new ArrowColumnVector(vector) - }.toArray[ColumnVector] - read() - case SpecialLengths.TIMING_DATA => - handleTimingData() - read() - case SpecialLengths.PYTHON_EXCEPTION_THROWN => - throw handlePythonException() - case SpecialLengths.END_OF_DATA_SECTION => - handleEndOfDataSection() - null - } - } - } catch handleException - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala new file mode 100644 index 0000000000000..0cee7d2f96c22 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BaseArrowPythonRunner.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import java.io._ +import java.net._ +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ + +import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.vector.ipc.ArrowStreamReader + +import org.apache.spark._ +import org.apache.spark.api.python._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} + +/** + * Common functionality for a udf runner that exchanges data with Python worker via Arrow stream. + */ +abstract class BaseArrowPythonRunner[T]( + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]]) + extends BasePythonRunner[T, ColumnarBatch](funcs, evalType, argOffsets) { + + protected override def newReaderIterator( + stream: DataInputStream, + writerThread: WriterThread, + startTime: Long, + env: SparkEnv, + worker: Socket, + releasedOrClosed: AtomicBoolean, + context: TaskContext): Iterator[ColumnarBatch] = { + + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { + + private val allocator = ArrowUtils.rootAllocator.newChildAllocator( + s"stdin reader for $pythonExec", 0, Long.MaxValue) + + private var reader: ArrowStreamReader = _ + private var root: VectorSchemaRoot = _ + private var schema: StructType = _ + private var vectors: Array[ColumnVector] = _ + + context.addTaskCompletionListener[Unit] { _ => + if (reader != null) { + reader.close(false) + } + allocator.close() + } + + private var batchLoaded = true + + protected override def read(): ColumnarBatch = { + if (writerThread.exception.isDefined) { + throw writerThread.exception.get + } + try { + if (reader != null && batchLoaded) { + batchLoaded = reader.loadNextBatch() + if (batchLoaded) { + val batch = new ColumnarBatch(vectors) + batch.setNumRows(root.getRowCount) + batch + } else { + reader.close(false) + allocator.close() + // Reach end of stream. Call `read()` again to read control data. + read() + } + } else { + stream.readInt() match { + case SpecialLengths.START_ARROW_STREAM => + reader = new ArrowStreamReader(stream, allocator) + root = reader.getVectorSchemaRoot() + schema = ArrowUtils.fromArrowSchema(root.getSchema()) + vectors = root.getFieldVectors().asScala.map { vector => + new ArrowColumnVector(vector) + }.toArray[ColumnVector] + read() + case SpecialLengths.TIMING_DATA => + handleTimingData() + read() + case SpecialLengths.PYTHON_EXCEPTION_THROWN => + throw handlePythonException() + case SpecialLengths.END_OF_DATA_SECTION => + handleEndOfDataSection() + null + } + } + } catch handleException + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala new file mode 100644 index 0000000000000..477c288ad1211 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BasePandasGroupExec.scala @@ -0,0 +1,137 @@ +/* + * 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.python + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.TaskContext +import org.apache.spark.api.python.{BasePythonRunner, ChainedPythonFunctions} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF, UnsafeProjection} +import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan} +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + +/** + * Base functionality for plans which execute grouped python udfs. + */ +abstract class BasePandasGroupExec( + func: Expression, + output: Seq[Attribute]) + extends SparkPlan { + + protected val sessionLocalTimeZone = conf.sessionLocalTimeZone + + protected val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) + + protected val pandasFunction = func.asInstanceOf[PythonUDF].func + + protected val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + + override def producedAttributes: AttributeSet = AttributeSet(output) + + /** + * passes the data to the python runner and coverts the resulting + * columnarbatch into internal rows. + */ + protected def executePython[T]( + data: Iterator[T], + runner: BasePythonRunner[T, ColumnarBatch]): Iterator[InternalRow] = { + + val context = TaskContext.get() + val columnarBatchIter = runner.compute(data, context.partitionId(), context) + val unsafeProj = UnsafeProjection.create(output, output) + + columnarBatchIter.flatMap { batch => + // UDF returns a StructType column in ColumnarBatch, select the children here + val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] + val outputVectors = output.indices.map(structVector.getChild) + val flattenedBatch = new ColumnarBatch(outputVectors.toArray) + flattenedBatch.setNumRows(batch.numRows()) + flattenedBatch.rowIterator.asScala + }.map(unsafeProj) + } + + /** + * groups according to grouping attributes and then projects into the deduplicated schema + */ + protected def groupAndProject( + input: Iterator[InternalRow], + groupingAttributes: Seq[Attribute], + inputSchema: Seq[Attribute], + dedupSchema: Seq[Attribute]): Iterator[(InternalRow, Iterator[InternalRow])] = { + val groupedIter = GroupedIterator(input, groupingAttributes, inputSchema) + val dedupProj = UnsafeProjection.create(dedupSchema, inputSchema) + groupedIter.map { + case (k, groupedRowIter) => (k, groupedRowIter.map(dedupProj)) + } + } + + /** + * Returns a the deduplicated attributes of the spark plan and the arg offsets of the + * keys and values. + * + * The deduplicated attributes are needed because the spark plan may contain an attribute + * twice; once in the key and once in the value. For any such attribute we need to + * deduplicate. + * + * The arg offsets are used to distinguish grouping grouping attributes and data attributes + * as following: + * + * argOffsets[0] is the length of the argOffsets array + * + * argOffsets[1] is the length of grouping attribute + * argOffsets[2 .. argOffsets[0]+2] is the arg offsets for grouping attributes + * + * argOffsets[argOffsets[0]+2 .. ] is the arg offsets for data attributes + */ + protected def resolveArgOffsets( + child: SparkPlan, groupingAttributes: Seq[Attribute]): (Seq[Attribute], Array[Int]) = { + + val dataAttributes = child.output.drop(groupingAttributes.length) + val groupingIndicesInData = groupingAttributes.map { attribute => + dataAttributes.indexWhere(attribute.semanticEquals) + } + + val groupingArgOffsets = new ArrayBuffer[Int] + val nonDupGroupingAttributes = new ArrayBuffer[Attribute] + val nonDupGroupingSize = groupingIndicesInData.count(_ == -1) + + groupingAttributes.zip(groupingIndicesInData).foreach { + case (attribute, index) => + if (index == -1) { + groupingArgOffsets += nonDupGroupingAttributes.length + nonDupGroupingAttributes += attribute + } else { + groupingArgOffsets += index + nonDupGroupingSize + } + } + + val dataArgOffsets = nonDupGroupingAttributes.length until + (nonDupGroupingAttributes.length + dataAttributes.length) + + val argOffsetsLength = groupingAttributes.length + dataArgOffsets.length + 1 + val argOffsets = Array(argOffsetsLength, + groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets + + // Attributes after deduplication + val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes + (dedupAttributes, argOffsets) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.scala new file mode 100644 index 0000000000000..8ea9881c575a1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/CogroupedArrowPythonRunner.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.python + +import java.io._ +import java.net._ + +import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.vector.ipc.ArrowStreamWriter + +import org.apache.spark._ +import org.apache.spark.api.python._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.arrow.ArrowWriter +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.util.Utils + + +/** + * Python UDF Runner for cogrouped udfs. Although the data is exchanged with the python + * worker via arrow, we cannot use `ArrowPythonRunner` as we need to send more than one + * dataframe. + */ +class CogroupedArrowPythonRunner( + funcs: Seq[ChainedPythonFunctions], + evalType: Int, + argOffsets: Array[Array[Int]], + leftSchema: StructType, + rightSchema: StructType, + timeZoneId: String, + conf: Map[String, String]) + extends BaseArrowPythonRunner[(Iterator[InternalRow], Iterator[InternalRow])]( + funcs, evalType, argOffsets) { + + protected def newWriterThread( + env: SparkEnv, + worker: Socket, + inputIterator: Iterator[(Iterator[InternalRow], Iterator[InternalRow])], + partitionIndex: Int, + context: TaskContext): WriterThread = { + + new WriterThread(env, worker, inputIterator, partitionIndex, context) { + + protected override def writeCommand(dataOut: DataOutputStream): Unit = { + + // Write config for the worker as a number of key -> value pairs of strings + dataOut.writeInt(conf.size) + for ((k, v) <- conf) { + PythonRDD.writeUTF(k, dataOut) + PythonRDD.writeUTF(v, dataOut) + } + + PythonUDFRunner.writeUDFs(dataOut, funcs, argOffsets) + } + + protected override def writeIteratorToStream(dataOut: DataOutputStream): Unit = { + // For each we first send the number of dataframes in each group then send + // first df, then send second df. End of data is marked by sending 0. + while (inputIterator.hasNext) { + dataOut.writeInt(2) + val (nextLeft, nextRight) = inputIterator.next() + writeGroup(nextLeft, leftSchema, dataOut, "left") + writeGroup(nextRight, rightSchema, dataOut, "right") + } + dataOut.writeInt(0) + } + + def writeGroup( + group: Iterator[InternalRow], + schema: StructType, + dataOut: DataOutputStream, + name: String) = { + val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) + val allocator = ArrowUtils.rootAllocator.newChildAllocator( + s"stdout writer for $pythonExec ($name)", 0, Long.MaxValue) + val root = VectorSchemaRoot.create(arrowSchema, allocator) + + Utils.tryWithSafeFinally { + val writer = new ArrowStreamWriter(root, null, dataOut) + val arrowWriter = ArrowWriter.create(root) + writer.start() + + while (group.hasNext) { + arrowWriter.write(group.next()) + } + arrowWriter.finish() + writer.writeBatch() + writer.end() + }{ + root.close() + allocator.close() + } + } + } + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala new file mode 100644 index 0000000000000..cc83e0cecdc33 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapCoGroupsInPandasExec.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import org.apache.spark.api.python.PythonEvalType +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.execution.{BinaryExecNode, CoGroupedIterator, SparkPlan} +import org.apache.spark.sql.types.StructType + + +/** + * Physical node for [[org.apache.spark.sql.catalyst.plans.logical.FlatMapCoGroupsInPandas]] + * + * The input dataframes are first Cogrouped. Rows from each side of the cogroup are passed to the + * Python worker via Arrow. As each side of the cogroup may have a different schema we send every + * group in its own Arrow stream. + * The Python worker turns the resulting record batches to `pandas.DataFrame`s, invokes the + * user-defined function, and passes the resulting `pandas.DataFrame` + * as an Arrow record batch. Finally, each record batch is turned to + * Iterator[InternalRow] using ColumnarBatch. + * + * Note on memory usage: + * Both the Python worker and the Java executor need to have enough memory to + * hold the largest cogroup. The memory on the Java side is used to construct the + * record batches (off heap memory). The memory on the Python side is used for + * holding the `pandas.DataFrame`. It's possible to further split one group into + * multiple record batches to reduce the memory footprint on the Java side, this + * is left as future work. + */ +case class FlatMapCoGroupsInPandasExec( + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + func: Expression, + output: Seq[Attribute], + left: SparkPlan, + right: SparkPlan) + extends BasePandasGroupExec(func, output) with BinaryExecNode { + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution: Seq[Distribution] = { + val leftDist = if (leftGroup.isEmpty) AllTuples else ClusteredDistribution(leftGroup) + val rightDist = if (rightGroup.isEmpty) AllTuples else ClusteredDistribution(rightGroup) + leftDist :: rightDist :: Nil + } + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + leftGroup + .map(SortOrder(_, Ascending)) :: rightGroup.map(SortOrder(_, Ascending)) :: Nil + } + + override protected def doExecute(): RDD[InternalRow] = { + + val (leftDedup, leftArgOffsets) = resolveArgOffsets(left, leftGroup) + val (rightDedup, rightArgOffsets) = resolveArgOffsets(right, rightGroup) + + // Map cogrouped rows to ArrowPythonRunner results, Only execute if partition is not empty + left.execute().zipPartitions(right.execute()) { (leftData, rightData) => + if (leftData.isEmpty && rightData.isEmpty) Iterator.empty else { + + val leftGrouped = groupAndProject(leftData, leftGroup, left.output, leftDedup) + val rightGrouped = groupAndProject(rightData, rightGroup, right.output, rightDedup) + val data = new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup) + .map { case (_, l, r) => (l, r) } + + val runner = new CogroupedArrowPythonRunner( + chainedFunc, + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + Array(leftArgOffsets ++ rightArgOffsets), + StructType.fromAttributes(leftDedup), + StructType.fromAttributes(rightDedup), + sessionLocalTimeZone, + pythonRunnerConf) + + executePython(data, runner) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index 267698d1bca50..22a0d1e09b12e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -17,19 +17,14 @@ package org.apache.spark.sql.execution.python -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.TaskContext -import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.api.python.PythonEvalType import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} -import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.ArrowUtils -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + /** * Physical node for [[org.apache.spark.sql.catalyst.plans.logical.FlatMapGroupsInPandas]] @@ -53,14 +48,10 @@ case class FlatMapGroupsInPandasExec( func: Expression, output: Seq[Attribute], child: SparkPlan) - extends UnaryExecNode { - - private val pandasFunction = func.asInstanceOf[PythonUDF].func + extends BasePandasGroupExec(func, output) with UnaryExecNode { override def outputPartitioning: Partitioning = child.outputPartitioning - override def producedAttributes: AttributeSet = AttributeSet(output) - override def requiredChildDistribution: Seq[Distribution] = { if (groupingAttributes.isEmpty) { AllTuples :: Nil @@ -75,88 +66,23 @@ case class FlatMapGroupsInPandasExec( override protected def doExecute(): RDD[InternalRow] = { val inputRDD = child.execute() - val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) - val sessionLocalTimeZone = conf.sessionLocalTimeZone - val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) - - // Deduplicate the grouping attributes. - // If a grouping attribute also appears in data attributes, then we don't need to send the - // grouping attribute to Python worker. If a grouping attribute is not in data attributes, - // then we need to send this grouping attribute to python worker. - // - // We use argOffsets to distinguish grouping attributes and data attributes as following: - // - // argOffsets[0] is the length of grouping attributes - // argOffsets[1 .. argOffsets[0]+1] is the arg offsets for grouping attributes - // argOffsets[argOffsets[0]+1 .. ] is the arg offsets for data attributes - - val dataAttributes = child.output.drop(groupingAttributes.length) - val groupingIndicesInData = groupingAttributes.map { attribute => - dataAttributes.indexWhere(attribute.semanticEquals) - } - - val groupingArgOffsets = new ArrayBuffer[Int] - val nonDupGroupingAttributes = new ArrayBuffer[Attribute] - val nonDupGroupingSize = groupingIndicesInData.count(_ == -1) - - // Non duplicate grouping attributes are added to nonDupGroupingAttributes and - // their offsets are 0, 1, 2 ... - // Duplicate grouping attributes are NOT added to nonDupGroupingAttributes and - // their offsets are n + index, where n is the total number of non duplicate grouping - // attributes and index is the index in the data attributes that the grouping attribute - // is a duplicate of. - - groupingAttributes.zip(groupingIndicesInData).foreach { - case (attribute, index) => - if (index == -1) { - groupingArgOffsets += nonDupGroupingAttributes.length - nonDupGroupingAttributes += attribute - } else { - groupingArgOffsets += index + nonDupGroupingSize - } - } - - val dataArgOffsets = nonDupGroupingAttributes.length until - (nonDupGroupingAttributes.length + dataAttributes.length) - - val argOffsets = Array(Array(groupingAttributes.length) ++ groupingArgOffsets ++ dataArgOffsets) - - // Attributes after deduplication - val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes - val dedupSchema = StructType.fromAttributes(dedupAttributes) + val (dedupAttributes, argOffsets) = resolveArgOffsets(child, groupingAttributes) // Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { - val grouped = if (groupingAttributes.isEmpty) { - Iterator(iter) - } else { - val groupedIter = GroupedIterator(iter, groupingAttributes, child.output) - val dedupProj = UnsafeProjection.create(dedupAttributes, child.output) - groupedIter.map { - case (_, groupedRowIter) => groupedRowIter.map(dedupProj) - } - } - val context = TaskContext.get() + val data = groupAndProject(iter, groupingAttributes, child.output, dedupAttributes) + .map{case(_, x) => x} - val columnarBatchIter = new ArrowPythonRunner( + val runner = new ArrowPythonRunner( chainedFunc, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, - argOffsets, - dedupSchema, + Array(argOffsets), + StructType.fromAttributes(dedupAttributes), sessionLocalTimeZone, - pythonRunnerConf).compute(grouped, context.partitionId(), context) - - val unsafeProj = UnsafeProjection.create(output, output) + pythonRunnerConf) - columnarBatchIter.flatMap { batch => - // Grouped Map UDF returns a StructType column in ColumnarBatch, select the children here - val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] - val outputVectors = output.indices.map(structVector.getChild) - val flattenedBatch = new ColumnarBatch(outputVectors.toArray) - flattenedBatch.setNumRows(batch.numRows()) - flattenedBatch.rowIterator.asScala - }.map(unsafeProj) + executePython(data, runner) }} } } From 3ece8ee15775307bded572ac391aeed10be3c9aa Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 18 Sep 2019 10:33:03 +0900 Subject: [PATCH 0097/1387] [SPARK-29124][CORE] Use MurmurHash3 `bytesHash(data, seed)` instead of `bytesHash(data)` ### What changes were proposed in this pull request? This PR changes `bytesHash(data)` API invocation with the underlaying `byteHash(data, arraySeed)` invocation. ```scala def bytesHash(data: Array[Byte]): Int = bytesHash(data, arraySeed) ``` ### Why are the changes needed? The original API is changed between Scala versions by the following commit. From Scala 2.12.9, the semantic of the function is changed. If we use the underlying form, we are safe during Scala version migration. - https://github.com/scala/scala/commit/846ee2b1a47014c69ebd2352d91d467be74918b5#diff-ac889f851e109fc4387cd738d52ce177 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This is a kind of refactoring. Pass the Jenkins with the existing tests. Closes #25821 from dongjoon-hyun/SPARK-SCALA-HASH. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- .../scala/org/apache/spark/util/random/XORShiftRandom.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index af09e50a157ae..11aa9da83b9ed 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -60,7 +60,7 @@ private[spark] object XORShiftRandom { /** Hash seeds to have 0/1 bits throughout. */ private[random] def hashSeed(seed: Long): Long = { val bytes = ByteBuffer.allocate(java.lang.Long.BYTES).putLong(seed).array() - val lowBits = MurmurHash3.bytesHash(bytes) + val lowBits = MurmurHash3.bytesHash(bytes, MurmurHash3.arraySeed) val highBits = MurmurHash3.bytesHash(bytes, lowBits) (highBits.toLong << 32) | (lowBits.toLong & 0xFFFFFFFFL) } From 4559a82a1de289093064490ef2d39c3c535fb3d4 Mon Sep 17 00:00:00 2001 From: s71955 Date: Wed, 18 Sep 2019 12:54:44 +0900 Subject: [PATCH 0098/1387] [SPARK-28930][SQL] Last Access Time value shall display 'UNKNOWN' in all clients **What changes were proposed in this pull request?** Issue 1 : modifications not required as these are different formats for the same info. In the case of a Spark DataFrame, null is correct. Issue 2 mentioned in JIRA Spark SQL "desc formatted tablename" is not showing the header # col_name,data_type,comment , seems to be the header has been removed knowingly as part of SPARK-20954. Issue 3: Corrected the Last Access time, the value shall display 'UNKNOWN' as currently system wont support the last access time evaluation, since hive was setting Last access time as '0' in metastore even though spark CatalogTable last access time value set as -1. this will make the validation logic of LasAccessTime where spark sets 'UNKNOWN' value if last access time value set as -1 (means not evaluated). **Does this PR introduce any user-facing change?** No **How was this patch tested?** Locally and corrected a ut. Attaching the test report below ![SPARK-28930](https://user-images.githubusercontent.com/12999161/64484908-83a1d980-d236-11e9-8062-9facf3003e5e.PNG) Closes #25720 from sujith71955/master_describe_info. Authored-by: s71955 Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/catalyst/catalog/interface.scala | 7 +++++-- .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index ce8c23ac6dceb..01b21feab0dd9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -117,7 +117,7 @@ case class CatalogTablePartition( } map.put("Created Time", new Date(createTime).toString) val lastAccess = { - if (-1 == lastAccessTime) "UNKNOWN" else new Date(lastAccessTime).toString + if (lastAccessTime <= 0) "UNKNOWN" else new Date(lastAccessTime).toString } map.put("Last Access", lastAccess) stats.foreach(s => map.put("Partition Statistics", s.simpleString)) @@ -320,12 +320,15 @@ case class CatalogTable( val map = new mutable.LinkedHashMap[String, String]() val tableProperties = properties.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") val partitionColumns = partitionColumnNames.map(quoteIdentifier).mkString("[", ", ", "]") + val lastAccess = { + if (lastAccessTime <= 0) "UNKNOWN" else new Date(lastAccessTime).toString + } identifier.database.foreach(map.put("Database", _)) map.put("Table", identifier.table) if (owner != null && owner.nonEmpty) map.put("Owner", owner) map.put("Created Time", new Date(createTime).toString) - map.put("Last Access", new Date(lastAccessTime).toString) + map.put("Last Access", lastAccess) map.put("Created By", "Spark " + createVersion) map.put("Type", tableType.name) provider.foreach(map.put("Provider", _)) 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 c6e4407c80e83..d63d741943bc7 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 @@ -2435,7 +2435,7 @@ class HiveDDLSuite .select("data_type") // check if the last access time doesnt have the default date of year // 1970 as its a wrong access time - assert(!(desc.first.toString.contains("1970"))) + assert((desc.first.toString.contains("UNKNOWN"))) } } From eef5e6d348e7025f48e8f69026254e07efe80894 Mon Sep 17 00:00:00 2001 From: turbofei Date: Wed, 18 Sep 2019 13:12:18 +0900 Subject: [PATCH 0099/1387] [SPARK-29113][DOC] Fix some annotation errors and remove meaningless annotations in project ### What changes were proposed in this pull request? In this PR, I fix some annotation errors and remove meaningless annotations in project. ### Why are the changes needed? There are some annotation errors and meaningless annotations in project. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Verified manually. Closes #25809 from turboFei/SPARK-29113. Authored-by: turbofei Signed-off-by: HyukjinKwon --- .../org/apache/spark/io/NioBufferedFileInputStream.java | 1 - .../main/java/org/apache/spark/memory/MemoryConsumer.java | 1 - .../apache/spark/shuffle/sort/ShuffleExternalSorter.java | 1 - .../util/collection/unsafe/sort/UnsafeExternalSorter.java | 2 -- .../org/apache/spark/deploy/history/ApplicationCache.scala | 7 +++---- .../org/apache/spark/scheduler/TaskSetBlacklist.scala | 1 - .../main/scala/org/apache/spark/storage/BlockManager.scala | 1 - .../scheduler/cluster/mesos/MesosSchedulerUtils.scala | 1 - .../org/apache/spark/sql/execution/ExplainUtils.scala | 4 ++-- 9 files changed, 5 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java index 92bf0ecc1b5cb..a1e29a8c873da 100644 --- a/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java +++ b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java @@ -51,7 +51,6 @@ public NioBufferedFileInputStream(File file) throws IOException { /** * Checks weather data is left to be read from the input stream. * @return true if data is left, false otherwise - * @throws IOException */ private boolean refill() throws IOException { if (!byteBuffer.hasRemaining()) { diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 4bfd2d358f36f..7b68b399e6e84 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -78,7 +78,6 @@ public void spill() throws IOException { * @param size the amount of memory should be released * @param trigger the MemoryConsumer that trigger this spilling * @return the amount of released memory in bytes - * @throws IOException */ public abstract long spill(long size, MemoryConsumer trigger) throws IOException; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 024756087bf7f..833744f4777ce 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -423,7 +423,6 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p * * @return metadata for the spill files written by this sorter. If no records were ever inserted * into this sorter, then this will return an empty array. - * @throws IOException */ public SpillInfo[] closeAndGetSpills() throws IOException { if (inMemSorter != null) { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 1b206c11d9a8e..55e4e609c3c7b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -447,8 +447,6 @@ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, /** * Merges another UnsafeExternalSorters into this one, the other one will be emptied. - * - * @throws IOException */ public void merge(UnsafeExternalSorter other) throws IOException { other.spill(); diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index 8c63fa65b40fd..fb2a67c2ab103 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -209,9 +209,8 @@ private[history] class ApplicationCache( /** * Register a filter for the web UI which checks for updates to the given app/attempt - * @param ui Spark UI to attach filters to - * @param appId application ID - * @param attemptId attempt ID + * @param key consisted of appId and attemptId + * @param loadedUI Spark UI to attach filters to */ private def registerFilter(key: CacheKey, loadedUI: LoadedAppUI): Unit = { require(loadedUI != null) @@ -231,7 +230,7 @@ private[history] class ApplicationCache( /** * An entry in the cache. * - * @param ui Spark UI + * @param loadedUI Spark UI * @param completed Flag to indicated that the application has completed (and so * does not need refreshing). */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala index b680979a466a5..4df2889089ee9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -69,7 +69,6 @@ private[scheduler] class TaskSetBlacklist( /** * Get the most recent failure reason of this TaskSet. - * @return */ def getLatestFailureReason: String = { latestFailureReason diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 4b71dc1fff345..0dfadb657b770 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -853,7 +853,6 @@ private[spark] class BlockManager( * @param bufferTransformer this transformer expected to open the file if the block is backed by a * file by this it is guaranteed the whole content can be loaded * @tparam T result type - * @return */ private[spark] def getRemoteBlock[T]( blockId: BlockId, diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 06993712035ff..f60b3b8db194c 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -285,7 +285,6 @@ trait MesosSchedulerUtils extends Logging { * The attribute values are the mesos attribute types and they are * * @param offerAttributes the attributes offered - * @return */ protected def toAttributeMap(offerAttributes: JList[Attribute]) : Map[String, GeneratedMessageV3] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala index 18a7f9822dcbc..fc384fe117caf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainUtils.scala @@ -41,7 +41,7 @@ object ExplainUtils { * * @param plan Input query plan to process * @param append function used to append the explain output - * @param startOperationID The start value of operation id. The subsequent operations will + * @param startOperatorID The start value of operation id. The subsequent operations will * be assigned higher value. * * @return The last generated operation id for this input plan. This is to ensure we @@ -125,7 +125,7 @@ object ExplainUtils { * appear in the explain output. * 2. operator identifier starts at startOperatorID + 1 * @param plan Input query plan to process - * @param startOperationID The start value of operation id. The subsequent operations will + * @param startOperatorID The start value of operation id. The subsequent operations will * be assigned higher value. * @param operatorIDs A output parameter that contains a map of operator id and query plan. This * is used by caller to print the detail portion of the plan. From 0b6775e6e9aea091e16d885639b16768650c513c Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Wed, 18 Sep 2019 14:11:39 +0800 Subject: [PATCH 0100/1387] [SPARK-29112][YARN] Expose more details when ApplicationMaster reporter faces a fatal exception ### What changes were proposed in this pull request? In `ApplicationMaster.Reporter` thread, fatal exception information is swallowed. It's better to expose it. We found our thrift server was shutdown due to a fatal exception but no useful information from log. > 19/09/16 06:59:54,498 INFO [Reporter] yarn.ApplicationMaster:54 : Final app status: FAILED, exitCode: 12, (reason: Exception was thrown 1 time(s) from Reporter thread.) 19/09/16 06:59:54,500 ERROR [Driver] thriftserver.HiveThriftServer2:91 : Error starting HiveThriftServer2 java.lang.InterruptedException: sleep interrupted at java.lang.Thread.sleep(Native Method) at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2$.main(HiveThriftServer2.scala:160) at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.main(HiveThriftServer2.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$4.run(ApplicationMaster.scala:708) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual test Closes #25810 from LantaoJin/SPARK-29112. Authored-by: LantaoJin Signed-off-by: jerryshao --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 4d884d54501ad..68035e4321e01 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 @@ -578,7 +578,11 @@ private[spark] class ApplicationMaster( e.getMessage) case e: Throwable => failureCount += 1 - if (!NonFatal(e) || failureCount >= reporterMaxFailures) { + if (!NonFatal(e)) { + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_REPORTER_FAILURE, + "Fatal exception: " + StringUtils.stringifyException(e)) + } else if (failureCount >= reporterMaxFailures) { finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " + s"$failureCount time(s) from Reporter thread.") From 3da2786dc6910fd1fd71541b191ca29da2f17f96 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 18 Sep 2019 16:59:17 +0800 Subject: [PATCH 0101/1387] [SPARK-29096][SQL] The exact math method should be called only when there is a corresponding function in Math ### What changes were proposed in this pull request? 1. After https://github.com/apache/spark/pull/21599, if the option "spark.sql.failOnIntegralTypeOverflow" is enabled, all the Binary Arithmetic operator will used the exact version function. However, only `Add`/`Substract`/`Multiply` has a corresponding exact function in java.lang.Math . When the option "spark.sql.failOnIntegralTypeOverflow" is enabled, a runtime exception "BinaryArithmetics must override either exactMathMethod or genCode" is thrown if the other Binary Arithmetic operators are used, such as "Divide", "Remainder". The exact math method should be called only when there is a corresponding function in `java.lang.Math` 2. Revise the log output of casting to `Int`/`Short` 3. Enable `spark.sql.failOnIntegralTypeOverflow` for pgSQL tests in `SQLQueryTestSuite`. ### Why are the changes needed? 1. Fix the bugs of https://github.com/apache/spark/pull/21599 2. The test case of pgSQL intends to check the overflow of integer/long type. We should enable `spark.sql.failOnIntegralTypeOverflow`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test. Closes #25804 from gengliangwang/enableIntegerOverflowInSQLTest. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/Cast.scala | 18 +- .../sql/catalyst/expressions/arithmetic.scala | 17 +- .../org/apache/spark/sql/types/Decimal.scala | 2 +- .../org/apache/spark/sql/types/numerics.scala | 6 +- .../resources/sql-tests/inputs/pgSQL/int4.sql | 5 - .../sql-tests/results/pgSQL/float4.sql.out | 10 +- .../sql-tests/results/pgSQL/float8.sql.out | 5 +- .../sql-tests/results/pgSQL/int4.sql.out | 276 ++++++++---------- .../sql-tests/results/pgSQL/int8.sql.out | 50 ++-- .../apache/spark/sql/SQLQueryTestSuite.scala | 13 + 10 files changed, 193 insertions(+), 209 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 baa98171e265f..d1943f02f85e5 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 @@ -537,7 +537,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if (longValue == longValue.toShort) { longValue.toShort } else { - throw new ArithmeticException(s"Casting $t to short causes overflow.") + throw new ArithmeticException(s"Casting $t to short causes overflow") } }) case TimestampType => @@ -548,12 +548,12 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String x.exactNumeric.asInstanceOf[Numeric[Any]].toInt(b) } catch { case _: ArithmeticException => - throw new ArithmeticException(s"Casting $b to short causes overflow.") + throw new ArithmeticException(s"Casting $b to short causes overflow") } if (intValue == intValue.toShort) { intValue.toShort } else { - throw new ArithmeticException(s"Casting $b to short causes overflow.") + throw new ArithmeticException(s"Casting $b to short causes overflow") } case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort @@ -578,7 +578,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if (longValue == longValue.toByte) { longValue.toByte } else { - throw new ArithmeticException(s"Casting $t to byte causes overflow.") + throw new ArithmeticException(s"Casting $t to byte causes overflow") } }) case TimestampType => @@ -589,12 +589,12 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String x.exactNumeric.asInstanceOf[Numeric[Any]].toInt(b) } catch { case _: ArithmeticException => - throw new ArithmeticException(s"Casting $b to byte causes overflow.") + throw new ArithmeticException(s"Casting $b to byte causes overflow") } if (intValue == intValue.toByte) { intValue.toByte } else { - throw new ArithmeticException(s"Casting $b to byte causes overflow.") + throw new ArithmeticException(s"Casting $b to byte causes overflow") } case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte @@ -1275,7 +1275,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if ($longValue == ($integralType) $longValue) { $evPrim = ($integralType) $longValue; } else { - throw new ArithmeticException("Casting $c to $integralType causes overflow"); + throw new ArithmeticException("Casting " + $c + " to $integralType causes overflow"); } """ } else { @@ -1300,7 +1300,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if ($c == ($integralType) $c) { $evPrim = ($integralType) $c; } else { - throw new ArithmeticException("Casting $c to $integralType causes overflow"); + throw new ArithmeticException("Casting " + $c + " to $integralType causes overflow"); } """ } @@ -1335,7 +1335,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if ($mathClass.floor($c) <= $max && $mathClass.ceil($c) >= $min) { $evPrim = ($integralType) $c; } else { - throw new ArithmeticException("Casting $c to $integralType causes overflow"); + throw new ArithmeticException("Casting " + $c + " to $integralType causes overflow"); } """ } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index a13929bb772f9..2ed82d99fe2bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -150,9 +150,10 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { def calendarIntervalMethod: String = sys.error("BinaryArithmetics must override either calendarIntervalMethod or genCode") - /** Name of the function for the exact version of this expression in [[Math]]. */ - def exactMathMethod: String = - sys.error("BinaryArithmetics must override either exactMathMethod or genCode") + // Name of the function for the exact version of this expression in [[Math]]. + // If the option "spark.sql.failOnIntegralTypeOverflow" is enabled and there is corresponding + // function in [[Math]], the exact function will be called instead of evaluation with [[symbol]]. + def exactMathMethod: Option[String] = None override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = dataType match { case _: DecimalType => @@ -182,9 +183,9 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { }) case IntegerType | LongType => nullSafeCodeGen(ctx, ev, (eval1, eval2) => { - val operation = if (checkOverflow) { + val operation = if (checkOverflow && exactMathMethod.isDefined) { val mathClass = classOf[Math].getName - s"$mathClass.$exactMathMethod($eval1, $eval2)" + s"$mathClass.${exactMathMethod.get}($eval1, $eval2)" } else { s"$eval1 $symbol $eval2" } @@ -235,7 +236,7 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic { } } - override def exactMathMethod: String = "addExact" + override def exactMathMethod: Option[String] = Some("addExact") } @ExpressionDescription( @@ -265,7 +266,7 @@ case class Subtract(left: Expression, right: Expression) extends BinaryArithmeti } } - override def exactMathMethod: String = "subtractExact" + override def exactMathMethod: Option[String] = Some("subtractExact") } @ExpressionDescription( @@ -286,7 +287,7 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti protected override def nullSafeEval(input1: Any, input2: Any): Any = numeric.times(input1, input2) - override def exactMathMethod: String = "multiplyExact" + override def exactMathMethod: Option[String] = Some("multiplyExact") } // Common base trait for Divide and Remainder, since these two classes are almost identical diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 6445820f1237b..3136669f64803 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -239,7 +239,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { def toByte: Byte = toLong.toByte private def overflowException(dataType: String) = - throw new ArithmeticException(s"Casting $this to $dataType causes overflow.") + throw new ArithmeticException(s"Casting $this to $dataType causes overflow") /** * @return the Byte value that is equal to the rounded decimal. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala index 9ff55b7f6e6ec..518255ecc42cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala @@ -114,13 +114,13 @@ object LongExactNumeric extends LongIsIntegral with Ordering.LongOrdering { if (x == x.toInt) { x.toInt } else { - throw new ArithmeticException(s"Casting $x to int causes overflow.") + throw new ArithmeticException(s"Casting $x to int causes overflow") } } object FloatExactNumeric extends FloatIsFractional with Ordering.FloatOrdering { private def overflowException(x: Float, dataType: String) = - throw new ArithmeticException(s"Casting $x to $dataType causes overflow.") + throw new ArithmeticException(s"Casting $x to $dataType causes overflow") private val intUpperBound = Int.MaxValue.toFloat private val intLowerBound = Int.MinValue.toFloat @@ -152,7 +152,7 @@ object FloatExactNumeric extends FloatIsFractional with Ordering.FloatOrdering { object DoubleExactNumeric extends DoubleIsFractional with Ordering.DoubleOrdering { private def overflowException(x: Double, dataType: String) = - throw new ArithmeticException(s"Casting $x to $dataType causes overflow.") + throw new ArithmeticException(s"Casting $x to $dataType causes overflow") private val intUpperBound = Int.MaxValue.toDouble private val intLowerBound = Int.MinValue.toDouble diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql index 1012db72e1873..1c2320ff7fad6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql @@ -33,11 +33,6 @@ INSERT INTO INT4_TBL VALUES ('-2147483647'); -- INSERT INTO INT4_TBL(f1) VALUES ('123 5'); -- INSERT INTO INT4_TBL(f1) VALUES (''); --- We cannot test this when failOnOverFlow=true here --- because exception happens in the executors and the --- output stacktrace cannot have an exact match -set spark.sql.arithmeticOperations.failOnOverFlow=false; - SELECT '' AS five, * FROM INT4_TBL; SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0'); diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out index 6e47cff91a7d5..c205f70a65f2d 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out @@ -334,9 +334,10 @@ struct -- !query 37 SELECT int(float('-2147483900')) -- !query 37 schema -struct +struct<> -- !query 37 output --2147483648 +java.lang.ArithmeticException +Casting -2.1474839E9 to int causes overflow -- !query 38 @@ -366,9 +367,10 @@ struct -- !query 41 SELECT bigint(float('-9223380000000000000')) -- !query 41 schema -struct +struct<> -- !query 41 output --9223372036854775808 +java.lang.ArithmeticException +Casting -9.22338E18 to int causes overflow -- !query 42 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out index b4ea3c1ad1cab..9a785670fa950 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out @@ -826,9 +826,10 @@ struct -- !query 93 SELECT bigint(double('-9223372036854780000')) -- !query 93 schema -struct +struct<> -- !query 93 output --9223372036854775808 +java.lang.ArithmeticException +Casting -9.22337203685478E18 to long causes overflow -- !query 94 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out index 879b3c626ec1b..52221d58392f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 54 +-- Number of queries: 53 -- !query 0 @@ -51,27 +51,30 @@ struct<> -- !query 6 -set spark.sql.arithmeticOperations.failOnOverFlow=false +SELECT '' AS five, * FROM INT4_TBL -- !query 6 schema -struct +struct -- !query 6 output -spark.sql.arithmeticOperations.failOnOverFlow false + -123456 + -2147483647 + 0 + 123456 + 2147483647 -- !query 7 -SELECT '' AS five, * FROM INT4_TBL +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0') -- !query 7 schema -struct +struct -- !query 7 output -123456 -2147483647 - 0 123456 2147483647 -- !query 8 -SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0') +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0') -- !query 8 schema struct -- !query 8 output @@ -82,18 +85,15 @@ struct -- !query 9 -SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0') +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0') -- !query 9 schema -struct +struct -- !query 9 output - -123456 - -2147483647 - 123456 - 2147483647 + 0 -- !query 10 -SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0') +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0') -- !query 10 schema struct -- !query 10 output @@ -101,15 +101,16 @@ struct -- !query 11 -SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0') +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0') -- !query 11 schema -struct +struct -- !query 11 output - 0 + -123456 + -2147483647 -- !query 12 -SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0') +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0') -- !query 12 schema struct -- !query 12 output @@ -118,16 +119,17 @@ struct -- !query 13 -SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0') -- !query 13 schema -struct +struct -- !query 13 output -123456 -2147483647 + 0 -- !query 14 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0') -- !query 14 schema struct -- !query 14 output @@ -137,17 +139,16 @@ struct -- !query 15 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0') +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0') -- !query 15 schema -struct +struct -- !query 15 output - -123456 - -2147483647 - 0 + 123456 + 2147483647 -- !query 16 -SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0') +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0') -- !query 16 schema struct -- !query 16 output @@ -156,16 +157,17 @@ struct -- !query 17 -SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0') -- !query 17 schema -struct +struct -- !query 17 output + 0 123456 2147483647 -- !query 18 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0') -- !query 18 schema struct -- !query 18 output @@ -175,81 +177,75 @@ struct -- !query 19 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0') +SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') -- !query 19 schema -struct +struct -- !query 19 output - 0 - 123456 2147483647 -- !query 20 -SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0') -- !query 20 schema -struct +struct -- !query 20 output - 2147483647 + -123456 + 0 + 123456 -- !query 21 -SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0') +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i -- !query 21 schema -struct +struct<> -- !query 21 output - -123456 - 0 - 123456 +java.lang.ArithmeticException +integer overflow -- !query 22 SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 -- !query 22 schema struct -- !query 22 output -123456 -246912 - -2147483647 2 0 0 123456 246912 - 2147483647 -2 -- !query 23 -SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i -WHERE abs(f1) < 1073741824 +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i -- !query 23 schema -struct +struct<> -- !query 23 output - -123456 -246912 - 0 0 - 123456 246912 +java.lang.ArithmeticException +integer overflow -- !query 24 SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 -- !query 24 schema struct -- !query 24 output -123456 -246912 - -2147483647 2 0 0 123456 246912 - 2147483647 -2 -- !query 25 -SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i -WHERE abs(f1) < 1073741824 +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i -- !query 25 schema -struct +struct<> -- !query 25 output - -123456 -246912 - 0 0 - 123456 246912 +java.lang.ArithmeticException +integer overflow -- !query 26 SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 -- !query 26 schema struct -- !query 26 output @@ -257,23 +253,20 @@ struct -2147483647 -2147483645 0 2 123456 123458 - 2147483647 -2147483647 -- !query 27 -SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i -WHERE f1 < 2147483646 +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i -- !query 27 schema -struct +struct<> -- !query 27 output - -123456 -123454 - -2147483647 -2147483645 - 0 2 - 123456 123458 +java.lang.ArithmeticException +integer overflow -- !query 28 SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 -- !query 28 schema struct -- !query 28 output @@ -281,71 +274,64 @@ struct -2147483647 -2147483645 0 2 123456 123458 - 2147483647 -2147483647 -- !query 29 -SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i -WHERE f1 < 2147483646 +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i -- !query 29 schema -struct +struct<> -- !query 29 output - -123456 -123454 - -2147483647 -2147483645 - 0 2 - 123456 123458 +java.lang.ArithmeticException +integer overflow -- !query 30 SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 -- !query 30 schema struct -- !query 30 output -123456 -123458 - -2147483647 2147483647 0 -2 123456 123454 2147483647 2147483645 -- !query 31 -SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i -WHERE f1 > -2147483647 +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i -- !query 31 schema -struct +struct<> -- !query 31 output - -123456 -123458 - 0 -2 - 123456 123454 - 2147483647 2147483645 +java.lang.ArithmeticException +integer overflow -- !query 32 SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 -- !query 32 schema struct -- !query 32 output -123456 -123458 - -2147483647 2147483647 0 -2 123456 123454 2147483647 2147483645 -- !query 33 -SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i -WHERE f1 > -2147483647 +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i -- !query 33 schema struct -- !query 33 output - -123456 -123458 - 0 -2 - 123456 123454 - 2147483647 2147483645 + -123456 -61728 + -2147483647 -1073741823 + 0 0 + 123456 61728 + 2147483647 1073741823 -- !query 34 -SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i -- !query 34 schema struct -- !query 34 output @@ -357,51 +343,47 @@ struct -- !query 35 -SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i +SELECT -2+3 AS one -- !query 35 schema -struct +struct -- !query 35 output - -123456 -61728 - -2147483647 -1073741823 - 0 0 - 123456 61728 - 2147483647 1073741823 +1 -- !query 36 -SELECT -2+3 AS one +SELECT 4-2 AS two -- !query 36 schema -struct +struct -- !query 36 output -1 +2 -- !query 37 -SELECT 4-2 AS two +SELECT 2- -1 AS three -- !query 37 schema -struct +struct -- !query 37 output -2 +3 -- !query 38 -SELECT 2- -1 AS three +SELECT 2 - -2 AS four -- !query 38 schema -struct +struct -- !query 38 output -3 +4 -- !query 39 -SELECT 2 - -2 AS four +SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true -- !query 39 schema -struct +struct -- !query 39 output -4 +true -- !query 40 -SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true +SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true -- !query 40 schema struct -- !query 40 output @@ -409,7 +391,7 @@ true -- !query 41 -SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true +SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true -- !query 41 schema struct -- !query 41 output @@ -417,78 +399,70 @@ true -- !query 42 -SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true +SELECT int('1000') < int('999') AS `false` -- !query 42 schema -struct +struct -- !query 42 output -true +false -- !query 43 -SELECT int('1000') < int('999') AS `false` +SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten -- !query 43 schema -struct +struct -- !query 43 output -false +10 -- !query 44 -SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten +SELECT 2 + 2 / 2 AS three -- !query 44 schema -struct +struct -- !query 44 output -10 +3 -- !query 45 -SELECT 2 + 2 / 2 AS three +SELECT (2 + 2) / 2 AS two -- !query 45 schema -struct +struct -- !query 45 output -3 +2 -- !query 46 -SELECT (2 + 2) / 2 AS two +SELECT string(shiftleft(int(-1), 31)) -- !query 46 schema -struct +struct -- !query 46 output -2 +-2147483648 -- !query 47 -SELECT string(shiftleft(int(-1), 31)) +SELECT string(int(shiftleft(int(-1), 31))+1) -- !query 47 schema -struct +struct -- !query 47 output --2147483648 +-2147483647 -- !query 48 -SELECT string(int(shiftleft(int(-1), 31))+1) +SELECT int(-2147483648) % int(-1) -- !query 48 schema -struct +struct<(CAST(-2147483648 AS INT) % CAST(-1 AS INT)):int> -- !query 48 output --2147483647 +0 -- !query 49 -SELECT int(-2147483648) % int(-1) +SELECT int(-2147483648) % smallint(-1) -- !query 49 schema -struct<(CAST(-2147483648 AS INT) % CAST(-1 AS INT)):int> +struct<(CAST(-2147483648 AS INT) % CAST(CAST(-1 AS SMALLINT) AS INT)):int> -- !query 49 output 0 -- !query 50 -SELECT int(-2147483648) % smallint(-1) --- !query 50 schema -struct<(CAST(-2147483648 AS INT) % CAST(CAST(-1 AS SMALLINT) AS INT)):int> --- !query 50 output -0 - - --- !query 51 SELECT x, int(x) AS int4_value FROM (VALUES double(-2.5), double(-1.5), @@ -497,9 +471,9 @@ FROM (VALUES double(-2.5), double(0.5), double(1.5), double(2.5)) t(x) --- !query 51 schema +-- !query 50 schema struct --- !query 51 output +-- !query 50 output -0.5 0 -1.5 -1 -2.5 -2 @@ -509,7 +483,7 @@ struct 2.5 2 --- !query 52 +-- !query 51 SELECT x, int(x) AS int4_value FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), @@ -518,9 +492,9 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(0.5 as decimal(38, 18)), cast(1.5 as decimal(38, 18)), cast(2.5 as decimal(38, 18))) t(x) --- !query 52 schema +-- !query 51 schema struct --- !query 52 output +-- !query 51 output -0.5 0 -1.5 -1 -2.5 -2 @@ -530,9 +504,9 @@ struct 2.5 2 --- !query 53 +-- !query 52 DROP TABLE INT4_TBL --- !query 53 schema +-- !query 52 schema struct<> --- !query 53 output +-- !query 52 output diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out index fc9f1474eb26c..20bb5dbda645f 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out @@ -389,13 +389,10 @@ struct -- !query 40 SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL -- !query 40 schema -struct +struct<> -- !query 40 output - 123 456 56088 - 123 4567890123456789 561850485185185047 - 4567890123456789 -4567890123456789 -4868582358072306617 - 4567890123456789 123 561850485185185047 - 4567890123456789 4567890123456789 4868582358072306617 +java.lang.ArithmeticException +long overflow -- !query 41 @@ -604,12 +601,10 @@ struct -- !query 60 SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 -- !query 60 schema -struct +struct<> -- !query 60 output --869367531 --869367531 --869367531 -123 +java.lang.ArithmeticException +Casting 4567890123456789 to int causes overflow -- !query 61 @@ -623,12 +618,10 @@ struct -- !query 62 SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 -- !query 62 schema -struct +struct<> -- !query 62 output --32491 --32491 --32491 -123 +java.lang.ArithmeticException +Casting 4567890123456789 to short causes overflow -- !query 63 @@ -662,9 +655,10 @@ struct -- !query 66 SELECT CAST(double('922337203685477580700.0') AS bigint) -- !query 66 schema -struct +struct<> -- !query 66 output -9223372036854775807 +java.lang.ArithmeticException +Casting 9.223372036854776E20 to long causes overflow -- !query 67 @@ -728,17 +722,19 @@ struct -- !query 72 SELECT string(int(shiftleft(bigint(-1), 63))+1) -- !query 72 schema -struct +struct<> -- !query 72 output -1 +java.lang.ArithmeticException +Casting -9223372036854775808 to int causes overflow -- !query 73 SELECT bigint((-9223372036854775808)) * bigint((-1)) -- !query 73 schema -struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(-1 AS BIGINT)):bigint> +struct<> -- !query 73 output --9223372036854775808 +java.lang.ArithmeticException +long overflow -- !query 74 @@ -760,9 +756,10 @@ struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(-1 AS BIGINT)):bigint> -- !query 76 SELECT bigint((-9223372036854775808)) * int((-1)) -- !query 76 schema -struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +struct<> -- !query 76 output --9223372036854775808 +java.lang.ArithmeticException +long overflow -- !query 77 @@ -784,9 +781,10 @@ struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(CAST(-1 AS INT) AS BIGINT)): -- !query 79 SELECT bigint((-9223372036854775808)) * smallint((-1)) -- !query 79 schema -struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +struct<> -- !query 79 output --9223372036854775808 +java.lang.ArithmeticException +long overflow -- !query 80 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 28ca0edaef871..cc92d6556387d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -22,12 +22,14 @@ import java.util.{Locale, TimeZone} import scala.util.control.NonFatal +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -308,6 +310,11 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) localSparkSession.conf.set(SQLConf.ANSI_SQL_PARSER.key, true) localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) + localSparkSession.conf.set(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key, true) + // Propagate the SQL conf FAIL_ON_INTEGRAL_TYPE_OVERFLOW to executor. + // TODO: remove this after SPARK-29122 is resolved. + localSparkSession.sparkContext.setLocalProperty( + SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key, "true") case _ => } @@ -413,6 +420,12 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // with a generic pattern "###". val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage (StructType(Seq.empty), Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x"))) + case s: SparkException if s.getCause != null => + // For a runtime exception, it is hard to match because its message contains + // information of stage, task ID, etc. + // To make result matching simpler, here we match the cause of the exception if it exists. + val cause = s.getCause + (StructType(Seq.empty), Seq(cause.getClass.getName, cause.getMessage)) case NonFatal(e) => // If there is an exception, put the exception class followed by the message. (StructType(Seq.empty), Seq(e.getClass.getName, e.getMessage)) From b48ef7a9fb1dec720d0f94ce7066fda106ab7bfe Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Wed, 18 Sep 2019 08:44:44 -0500 Subject: [PATCH 0102/1387] [SPARK-28799][DOC] Documentation for Truncate command ### What changes were proposed in this pull request? Document TRUNCATE statement in SQL Reference Guide. ### Why are the changes needed? Adding documentation for SQL reference. ### Does this PR introduce any user-facing change? yes Before: There was no documentation for this. After. ![image (4)](https://user-images.githubusercontent.com/51401130/64956929-5e057780-d8a9-11e9-89a3-2d02c942b9ad.png) ![image (5)](https://user-images.githubusercontent.com/51401130/64956942-61006800-d8a9-11e9-9767-6164eabfdc2c.png) ### How was this patch tested? Used jekyll build and serve to verify. Closes #25557 from PavithraRamachandran/truncate_doc. Lead-authored-by: Pavithra Ramachandran Co-authored-by: pavithra Signed-off-by: Sean Owen --- docs/sql-ref-syntax-ddl-truncate-table.md | 66 ++++++++++++++++++++++- 1 file changed, 65 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md index 2704259391e94..4b4094ab708e5 100644 --- a/docs/sql-ref-syntax-ddl-truncate-table.md +++ b/docs/sql-ref-syntax-ddl-truncate-table.md @@ -19,4 +19,68 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +The `TRUNCATE TABLE` statement removes all the rows from a table or partition(s). The table must not be a view +or an external/temporary table. In order to truncate multiple partitions at once, the user can specify the partitions +in `partition_spec`. If no `partition_spec` is specified it will remove all partitions in the table. + +### Syntax +{% highlight sql %} +TRUNCATE TABLE table_name [PARTITION partition_spec]; +{% endhighlight %} + +### Parameters +
    +
    table_name
    +
    The name of an existing table.
    +
    + +
    +
    PARTITION ( partition_spec :[ partition_column = partition_col_value, partition_column = partition_col_value, ...] )
    +
    Specifies one or more partition column and value pairs. The partition value is optional.
    +
    + + +### Examples +{% highlight sql %} + +--Create table Student with partition +CREATE TABLE Student ( name String, rollno INT) PARTITIONED BY (age int); + +SELECT * from Student; ++-------+---------+------+--+ +| name | rollno | age | ++-------+---------+------+--+ +| ABC | 1 | 10 | +| DEF | 2 | 10 | +| XYZ | 3 | 12 | ++-------+---------+------+--+ + +-- Removes all rows from the table in the partion specified +TRUNCATE TABLE Student partition(age=10); + +--After truncate execution, records belonging to partition age=10 are removed +SELECT * from Student; ++-------+---------+------+--+ +| name | rollno | age | ++-------+---------+------+--+ +| XYZ | 3 | 12 | ++-------+---------+------+--+ + +-- Removes all rows from the table from all partitions +TRUNCATE TABLE Student; + +SELECT * from Student; ++-------+---------+------+--+ +| name | rollno | age | ++-------+---------+------+--+ ++-------+---------+------+--+ +No rows selected + +{% endhighlight %} + + +### Related Statements +- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) +- [ALTER TABLE](sql-ref-syntax-ddl-alter-tabley.html) + From 600a2a4ae585a47c822efe42f4fcccb048d37864 Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Wed, 18 Sep 2019 09:11:15 -0500 Subject: [PATCH 0103/1387] [SPARK-28972][DOCS] Updating unit description in configurations, to maintain consistency ### What changes were proposed in this pull request? Updating unit description in configurations, inorder to maintain consistency across configurations. ### Why are the changes needed? the description does not mention about suffix that can be mentioned while configuring this value. For better user understanding ### Does this PR introduce any user-facing change? yes. Doc description ### How was this patch tested? generated document and checked. ![Screenshot from 2019-09-05 11-09-17](https://user-images.githubusercontent.com/51401130/64314853-07a55880-cfce-11e9-8af0-6416a50b0188.png) Closes #25689 from PavithraRamachandran/heapsize_config. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- .../apache/spark/internal/config/package.scala | 3 ++- docs/configuration.md | 18 ++++++++++-------- 2 files changed, 12 insertions(+), 9 deletions(-) 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 158a4b7cfa55a..6011901d1d53d 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 @@ -243,7 +243,8 @@ package object config { .createWithDefault(false) private[spark] val MEMORY_OFFHEAP_SIZE = ConfigBuilder("spark.memory.offHeap.size") - .doc("The absolute amount of memory in bytes which can be used for off-heap allocation. " + + .doc("The absolute amount of memory which can be used for off-heap allocation, " + + " in bytes unless otherwise specified. " + "This setting has no impact on heap memory usage, so if your executors' total memory " + "consumption must fit within some hard limit then be sure to shrink your JVM heap size " + "accordingly. This must be set to a positive value when spark.memory.offHeap.enabled=true.") diff --git a/docs/configuration.md b/docs/configuration.md index 5cf42d5fe8dd2..e0b7386ac9bcc 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -866,7 +866,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.service.index.cache.size 100m - Cache entries limited to the specified memory footprint in bytes. + Cache entries limited to the specified memory footprint, in bytes unless otherwise specified. @@ -1207,16 +1207,18 @@ Apart from these, the following properties are also available, and may be useful spark.io.compression.lz4.blockSize 32k - Block size in bytes used in LZ4 compression, in the case when LZ4 compression codec + Block size used in LZ4 compression, in the case when LZ4 compression codec is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used. + Default unit is bytes, unless otherwise specified. spark.io.compression.snappy.blockSize 32k - Block size in bytes used in Snappy compression, in the case when Snappy compression codec - is used. Lowering this block size will also lower shuffle memory usage when Snappy is used. + Block size in Snappy compression, in the case when Snappy compression codec is used. + Lowering this block size will also lower shuffle memory usage when Snappy is used. + Default unit is bytes, unless otherwise specified. @@ -1384,7 +1386,7 @@ Apart from these, the following properties are also available, and may be useful spark.memory.offHeap.size 0 - The absolute amount of memory in bytes which can be used for off-heap allocation. + The absolute amount of memory which can be used for off-heap allocation, in bytes unless otherwise specified. This setting has no impact on heap memory usage, so if your executors' total memory consumption must fit within some hard limit then be sure to shrink your JVM heap size accordingly. This must be set to a positive value when spark.memory.offHeap.enabled=true. @@ -1568,9 +1570,9 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryMapThreshold 2m - Size in bytes of a block above which Spark memory maps when reading a block from disk. - This prevents Spark from memory mapping very small blocks. In general, memory - mapping has high overhead for blocks close to or below the page size of the operating system. + Size of a block above which Spark memory maps when reading a block from disk. Default unit is bytes, + unless specified otherwise. This prevents Spark from memory mapping very small blocks. In general, + memory mapping has high overhead for blocks close to or below the page size of the operating system. From 12e1583093bbd8f6b7146969f93bbff09d583561 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 18 Sep 2019 09:22:13 -0500 Subject: [PATCH 0104/1387] [SPARK-28927][ML] Rethrow block mismatch exception in ALS when input data is nondeterministic ### What changes were proposed in this pull request? Fitting ALS model can be failed due to nondeterministic input data. Currently the failure is thrown by an ArrayIndexOutOfBoundsException which is not explainable for end users what is wrong in fitting. This patch catches this exception and rethrows a more explainable one, when the input data is nondeterministic. Because we may not exactly know the output deterministic level of RDDs produced by user code, this patch also adds a note to Scala/Python/R ALS document about the training data deterministic level. ### Why are the changes needed? ArrayIndexOutOfBoundsException was observed during fitting ALS model. It was caused by mismatching between in/out user/item blocks during computing ratings. If the training RDD output is nondeterministic, when fetch failure is happened, rerun part of training RDD can produce inconsistent user/item blocks. This patch is needed to notify users ALS fitting on nondeterministic input. ### Does this PR introduce any user-facing change? Yes. When fitting ALS model on nondeterministic input data, previously if rerun happens, users would see ArrayIndexOutOfBoundsException caused by mismatch between In/Out user/item blocks. After this patch, a SparkException with more clear message will be thrown, and original ArrayIndexOutOfBoundsException is wrapped. ### How was this patch tested? Tested on development cluster. Closes #25789 from viirya/als-indeterminate-input. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Sean Owen --- R/pkg/R/mllib_recommendation.R | 6 ++++ .../apache/spark/ml/recommendation/ALS.scala | 32 +++++++++++++++++-- .../spark/mllib/recommendation/ALS.scala | 7 ++++ python/pyspark/ml/recommendation.py | 7 ++++ 4 files changed, 49 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/mllib_recommendation.R b/R/pkg/R/mllib_recommendation.R index 9a77b07462585..d238ff93ed245 100644 --- a/R/pkg/R/mllib_recommendation.R +++ b/R/pkg/R/mllib_recommendation.R @@ -82,6 +82,12 @@ setClass("ALSModel", representation(jobj = "jobj")) #' statsS <- summary(modelS) #' } #' @note spark.als since 2.1.0 +#' @note the input rating dataframe to the ALS implementation should be deterministic. +#' Nondeterministic data can cause failure during fitting ALS model. For example, +#' an order-sensitive operation like sampling after a repartition makes dataframe output +#' nondeterministic, like \code{sample(repartition(df, 2L), FALSE, 0.5, 1618L)}. +#' Checkpointing sampled dataframe or adding a sort before sampling can help make the +#' dataframe deterministic. setMethod("spark.als", signature(data = "SparkDataFrame"), function(data, ratingCol = "rating", userCol = "user", itemCol = "item", rank = 10, regParam = 0.1, maxIter = 10, nonnegative = FALSE, diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index fb7334d41ba44..33097f322444b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats import org.json4s.JsonDSL._ -import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext} +import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext, SparkException} import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} @@ -42,7 +42,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.mllib.optimization.NNLS -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -564,6 +564,13 @@ object ALSModel extends MLReadable[ALSModel] { * r is greater than 0 and 0 if r is less than or equal to 0. The ratings then act as 'confidence' * values related to strength of indicated user * preferences rather than explicit ratings given to items. + * + * Note: the input rating dataset to the ALS implementation should be deterministic. + * Nondeterministic data can cause failure during fitting ALS model. + * For example, an order-sensitive operation like sampling after a repartition makes dataset + * output nondeterministic, like `dataset.repartition(2).sample(false, 0.5, 1618)`. + * Checkpointing sampled dataset or adding a sort before sampling can help make the dataset + * deterministic. */ @Since("1.3.0") class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] with ALSParams @@ -1666,6 +1673,13 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { } } val merged = srcOut.groupByKey(new ALSPartitioner(dstInBlocks.partitions.length)) + + // SPARK-28927: Nondeterministic RDDs causes inconsistent in/out blocks in case of rerun. + // It can cause runtime error when matching in/out user/item blocks. + val isBlockRDDNondeterministic = + dstInBlocks.outputDeterministicLevel == DeterministicLevel.INDETERMINATE || + srcOutBlocks.outputDeterministicLevel == DeterministicLevel.INDETERMINATE + dstInBlocks.join(merged).mapValues { case (InBlock(dstIds, srcPtrs, srcEncodedIndices, ratings), srcFactors) => val sortedSrcFactors = new Array[FactorBlock](numSrcBlocks) @@ -1686,7 +1700,19 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { val encoded = srcEncodedIndices(i) val blockId = srcEncoder.blockId(encoded) val localIndex = srcEncoder.localIndex(encoded) - val srcFactor = sortedSrcFactors(blockId)(localIndex) + var srcFactor: Array[Float] = null + try { + srcFactor = sortedSrcFactors(blockId)(localIndex) + } catch { + case a: ArrayIndexOutOfBoundsException if isBlockRDDNondeterministic => + val errMsg = "A failure detected when matching In/Out blocks of users/items. " + + "Because at least one In/Out block RDD is found to be nondeterministic now, " + + "the issue is probably caused by nondeterministic input data. You can try to " + + "checkpoint training data to make it deterministic. If you do `repartition` + " + + "`sample` or `randomSplit`, you can also try to sort it before `sample` or " + + "`randomSplit` to make it deterministic." + throw new SparkException(errMsg, a) + } val rating = ratings(i) if (implicitPrefs) { // Extension to the original paper to handle rating < 0. confidence is a function diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 12870f819b147..f3f15ba0d0f2c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -62,6 +62,13 @@ case class Rating @Since("0.8.0") ( * r > 0 and 0 if r <= 0. The ratings then act as 'confidence' values related to strength of * indicated user * preferences rather than explicit ratings given to items. + * + * Note: the input rating RDD to the ALS implementation should be deterministic. + * Nondeterministic data can cause failure during fitting ALS model. + * For example, an order-sensitive operation like sampling after a repartition makes RDD + * output nondeterministic, like `rdd.repartition(2).sample(false, 0.5, 1618)`. + * Checkpointing sampled RDD or adding a sort before sampling can help make the RDD + * deterministic. */ @Since("0.8.0") class ALS private ( diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index bf2716485df9c..676662da8c316 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -67,6 +67,13 @@ class ALS(JavaEstimator, HasCheckpointInterval, HasMaxIter, HasPredictionCol, Ha indicated user preferences rather than explicit ratings given to items. + .. note:: the input rating dataframe to the ALS implementation should be deterministic. + Nondeterministic data can cause failure during fitting ALS model. + For example, an order-sensitive operation like sampling after a repartition makes + dataframe output nondeterministic, like `df.repartition(2).sample(False, 0.5, 1618)`. + Checkpointing sampled dataframe or adding a sort before sampling can help make the + dataframe deterministic. + >>> df = spark.createDataFrame( ... [(0, 0, 4.0), (0, 1, 2.0), (1, 1, 3.0), (1, 2, 4.0), (2, 1, 1.0), (2, 2, 5.0)], ... ["user", "item", "rating"]) From c2734ab1fcc523507174c4f9e7368c1e7a2368e8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Sep 2019 23:30:59 +0900 Subject: [PATCH 0105/1387] [SPARK-29012][SQL] Support special timestamp values ### What changes were proposed in this pull request? Supported special string values for `TIMESTAMP` type. They are simply notational shorthands that will be converted to ordinary timestamp values when read. The following string values are supported: - `epoch [zoneId]` - `1970-01-01 00:00:00+00 (Unix system time zero)` - `today [zoneId]` - midnight today. - `yesterday [zoneId]` -midnight yesterday - `tomorrow [zoneId]` - midnight tomorrow - `now` - current query start time. For example: ```sql spark-sql> SELECT timestamp 'tomorrow'; 2019-09-07 00:00:00 ``` ### Why are the changes needed? To maintain feature parity with PostgreSQL, see [8.5.1.4. Special Values](https://www.postgresql.org/docs/12/datatype-datetime.html) ### Does this PR introduce any user-facing change? Previously, the parser fails on the special values with the error: ```sql spark-sql> select timestamp 'today'; Error in query: Cannot parse the TIMESTAMP value: today(line 1, pos 7) ``` After the changes, the special values are converted to appropriate dates: ```sql spark-sql> select timestamp 'today'; 2019-09-06 00:00:00 ``` ### How was this patch tested? - Added tests to `TimestampFormatterSuite` to check parsing special values from regular strings. - Tests in `DateTimeUtilsSuite` check parsing those values from `UTF8String` - Uncommented tests in `timestamp.sql` Closes #25716 from MaxGekk/timestamp-special-values. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../expressions/datetimeExpressions.scala | 4 +- .../sql/catalyst/util/DateTimeUtils.scala | 67 ++++++ .../catalyst/util/TimestampFormatter.scala | 21 +- .../catalyst/util/DateTimeUtilsSuite.scala | 36 +++- .../sql/util/TimestampFormatterSuite.scala | 28 ++- .../sql-tests/inputs/pgSQL/timestamp.sql | 29 +-- .../sql-tests/results/pgSQL/timestamp.sql.out | 190 ++++++++++++++---- .../apache/spark/sql/CsvFunctionsSuite.scala | 10 + .../apache/spark/sql/JsonFunctionsSuite.scala | 10 + 9 files changed, 317 insertions(+), 78 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 597be5dc97b43..89a6d23b1d73d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -98,9 +98,7 @@ case class CurrentTimestamp() extends LeafExpression with CodegenFallback { override def dataType: DataType = TimestampType - override def eval(input: InternalRow): Any = { - instantToMicros(Instant.now()) - } + override def eval(input: InternalRow): Any = currentTimestamp() override def prettyName: String = "current_timestamp" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 65a9bee5eaedd..a82471aae652d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.catalyst.util +import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.time._ import java.time.temporal.{ChronoField, ChronoUnit, IsoFields} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit._ +import scala.util.Try import scala.util.control.NonFatal import org.apache.spark.sql.types.Decimal @@ -218,6 +220,8 @@ object DateTimeUtils { var i = 0 var currentSegmentValue = 0 val bytes = s.trim.getBytes + val specialTimestamp = convertSpecialTimestamp(bytes, timeZoneId) + if (specialTimestamp.isDefined) return specialTimestamp var j = 0 var digitsMilli = 0 var justTime = false @@ -848,4 +852,67 @@ object DateTimeUtils { val sinceEpoch = BigDecimal(timestamp) / MICROS_PER_SECOND + offset new Decimal().set(sinceEpoch, 20, 6) } + + def currentTimestamp(): SQLTimestamp = instantToMicros(Instant.now()) + + private def today(zoneId: ZoneId): ZonedDateTime = { + Instant.now().atZone(zoneId).`with`(LocalTime.MIDNIGHT) + } + + private val specialValueRe = """(\p{Alpha}+)\p{Blank}*(.*)""".r + + /** + * Extracts special values from an input string ignoring case. + * @param input - a trimmed string + * @param zoneId - zone identifier used to get the current date. + * @return some special value in lower case or None. + */ + private def extractSpecialValue(input: String, zoneId: ZoneId): Option[String] = { + def isValid(value: String, timeZoneId: String): Boolean = { + // Special value can be without any time zone + if (timeZoneId.isEmpty) return true + // "now" must not have the time zone field + if (value.compareToIgnoreCase("now") == 0) return false + // If the time zone field presents in the input, it must be resolvable + try { + getZoneId(timeZoneId) + true + } catch { + case NonFatal(_) => false + } + } + + assert(input.trim.length == input.length) + if (input.length < 3 || !input(0).isLetter) return None + input match { + case specialValueRe(v, z) if isValid(v, z) => Some(v.toLowerCase(Locale.US)) + case _ => None + } + } + + /** + * Converts notational shorthands that are converted to ordinary timestamps. + * @param input - a trimmed string + * @param zoneId - zone identifier used to get the current date. + * @return some of microseconds since the epoch if the conversion completed + * successfully otherwise None. + */ + def convertSpecialTimestamp(input: String, zoneId: ZoneId): Option[SQLTimestamp] = { + extractSpecialValue(input, zoneId).flatMap { + case "epoch" => Some(0) + case "now" => Some(currentTimestamp()) + case "today" => Some(instantToMicros(today(zoneId).toInstant)) + case "tomorrow" => Some(instantToMicros(today(zoneId).plusDays(1).toInstant)) + case "yesterday" => Some(instantToMicros(today(zoneId).minusDays(1).toInstant)) + case _ => None + } + } + + private def convertSpecialTimestamp(bytes: Array[Byte], zoneId: ZoneId): Option[SQLTimestamp] = { + if (bytes.length > 0 && Character.isAlphabetic(bytes(0))) { + convertSpecialTimestamp(new String(bytes, StandardCharsets.UTF_8), zoneId) + } else { + None + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 3dfcf1a33eb56..5be4807083fa3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -22,9 +22,11 @@ import java.time._ import java.time.format.DateTimeParseException import java.time.temporal.ChronoField.MICRO_OF_SECOND import java.time.temporal.TemporalQueries -import java.util.{Locale, TimeZone} +import java.util.Locale import java.util.concurrent.TimeUnit.SECONDS +import DateTimeUtils.convertSpecialTimestamp + sealed trait TimestampFormatter extends Serializable { /** * Parses a timestamp in a string and converts it to microseconds. @@ -50,14 +52,17 @@ class Iso8601TimestampFormatter( protected lazy val formatter = getOrCreateFormatter(pattern, locale) override def parse(s: String): Long = { - val parsed = formatter.parse(s) - val parsedZoneId = parsed.query(TemporalQueries.zone()) - val timeZoneId = if (parsedZoneId == null) zoneId else parsedZoneId - val zonedDateTime = toZonedDateTime(parsed, timeZoneId) - val epochSeconds = zonedDateTime.toEpochSecond - val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND) + val specialDate = convertSpecialTimestamp(s.trim, zoneId) + specialDate.getOrElse { + val parsed = formatter.parse(s) + val parsedZoneId = parsed.query(TemporalQueries.zone()) + val timeZoneId = if (parsedZoneId == null) zoneId else parsedZoneId + val zonedDateTime = toZonedDateTime(parsed, timeZoneId) + val epochSeconds = zonedDateTime.toEpochSecond + val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND) - Math.addExact(SECONDS.toMicros(epochSeconds), microsOfSecond) + Math.addExact(SECONDS.toMicros(epochSeconds), microsOfSecond) + } } override def format(us: Long): String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 056337205ae7e..31fefd613f9c8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -19,16 +19,18 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.ZoneId +import java.time.{LocalDateTime, LocalTime, ZoneId} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit +import org.scalatest.Matchers + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.unsafe.types.UTF8String -class DateTimeUtilsSuite extends SparkFunSuite { +class DateTimeUtilsSuite extends SparkFunSuite with Matchers { val TimeZonePST = TimeZone.getTimeZone("PST") private def defaultZoneId = ZoneId.systemDefault() @@ -142,10 +144,14 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(stringToDate(UTF8String.fromString("1999 08")).isEmpty) } + private def toTimestamp(str: String, zoneId: ZoneId): Option[SQLTimestamp] = { + stringToTimestamp(UTF8String.fromString(str), zoneId) + } + test("string to timestamp") { for (tz <- ALL_TIMEZONES) { def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = { - assert(stringToTimestamp(UTF8String.fromString(str), tz.toZoneId) === expected) + assert(toTimestamp(str, tz.toZoneId) === expected) } checkStringToTimestamp("1969-12-31 16:00:00", Option(date(1969, 12, 31, 16, tz = tz))) @@ -271,8 +277,8 @@ class DateTimeUtilsSuite extends SparkFunSuite { UTF8String.fromString("2015-02-29 00:00:00"), defaultZoneId).isEmpty) assert(stringToTimestamp( UTF8String.fromString("2015-04-31 00:00:00"), defaultZoneId).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-02-29"), defaultZoneId).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-04-31"), defaultZoneId).isEmpty) + assert(toTimestamp("2015-02-29", defaultZoneId).isEmpty) + assert(toTimestamp("2015-04-31", defaultZoneId).isEmpty) } test("hours") { @@ -456,8 +462,7 @@ class DateTimeUtilsSuite extends SparkFunSuite { timezone: TimeZone = DateTimeUtils.defaultTimeZone()): Unit = { val truncated = DateTimeUtils.truncTimestamp(inputTS, level, timezone) - val expectedTS = - DateTimeUtils.stringToTimestamp(UTF8String.fromString(expected), defaultZoneId) + val expectedTS = toTimestamp(expected, defaultZoneId) assert(truncated === expectedTS.get) } @@ -564,4 +569,21 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(DateTimeUtils.toMillis(-9223372036844776001L) === -9223372036844777L) assert(DateTimeUtils.toMillis(-157700927876544L) === -157700927877L) } + + test("special timestamp values") { + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + val tolerance = TimeUnit.SECONDS.toMicros(30) + + assert(toTimestamp("Epoch", zoneId).get === 0) + val now = instantToMicros(LocalDateTime.now(zoneId).atZone(zoneId).toInstant) + toTimestamp("NOW", zoneId).get should be (now +- tolerance) + assert(toTimestamp("now UTC", zoneId) === None) + val today = instantToMicros(LocalDateTime.now(zoneId) + .`with`(LocalTime.MIDNIGHT) + .atZone(zoneId).toInstant) + toTimestamp(" Yesterday", zoneId).get should be (today - MICROS_PER_DAY +- tolerance) + toTimestamp("Today ", zoneId).get should be (today +- tolerance) + toTimestamp(" tomorrow CET ", zoneId).get should be (today + MICROS_PER_DAY +- tolerance) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index c223639a47294..170daa6277c49 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -17,14 +17,18 @@ package org.apache.spark.sql.util -import java.time.{LocalDateTime, ZoneId, ZoneOffset} +import java.time.{LocalDateTime, LocalTime, ZoneOffset} import java.util.concurrent.TimeUnit +import org.scalatest.Matchers + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, instantToMicros, MICROS_PER_DAY} +import org.apache.spark.sql.internal.SQLConf -class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { +class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers { test("parsing timestamps using time zones") { val localDate = "2018-12-02T10:11:12.001234" @@ -131,4 +135,24 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { val micros = DateTimeUtils.instantToMicros(instant) assert(TimestampFormatter(ZoneOffset.UTC).format(micros) === "-0099-01-01 00:00:00") } + + test("special timestamp values") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val zoneId = getZoneId(timeZone) + val formatter = TimestampFormatter(zoneId) + val tolerance = TimeUnit.SECONDS.toMicros(30) + + assert(formatter.parse("EPOCH") === 0) + val now = instantToMicros(LocalDateTime.now(zoneId).atZone(zoneId).toInstant) + formatter.parse("now") should be (now +- tolerance) + val today = instantToMicros(LocalDateTime.now(zoneId) + .`with`(LocalTime.MIDNIGHT) + .atZone(zoneId).toInstant) + formatter.parse("yesterday CET") should be (today - MICROS_PER_DAY +- tolerance) + formatter.parse(" TODAY ") should be (today +- tolerance) + formatter.parse("Tomorrow ") should be (today + MICROS_PER_DAY +- tolerance) + } + } + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql index 65e8d3280e07c..92bbe14dc1e5f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql @@ -7,7 +7,6 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; --- [SPARK-28141] Timestamp type can not accept special values -- Test shorthand input values -- We can't just "select" the results since they aren't constants; test for -- equality instead. We can do that by running the test inside a transaction @@ -17,22 +16,24 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; -- block is entered exactly at local midnight; then 'now' and 'today' have -- the same values and the counts will come out different. --- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +INSERT INTO TIMESTAMP_TBL VALUES ('now'); -- SELECT pg_sleep(0.1); -- BEGIN; --- INSERT INTO TIMESTAMP_TBL VALUES ('now'); --- INSERT INTO TIMESTAMP_TBL VALUES ('today'); --- INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); --- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); +INSERT INTO TIMESTAMP_TBL VALUES ('now'); +INSERT INTO TIMESTAMP_TBL VALUES ('today'); +INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); -- time zone should be ignored by this data type --- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); --- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow zulu'); - --- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; --- SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; --- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday'; +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); +-- [SPARK-29024] Ignore case while resolving time zones +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow Zulu'); + +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; +SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday'; +-- [SPARK-29025] Support seconds precision by the timestamp type -- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; -- COMMIT; @@ -48,12 +49,12 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; -- SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; -- COMMIT; --- TRUNCATE TIMESTAMP_TBL; +TRUNCATE TABLE TIMESTAMP_TBL; -- Special values -- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); -- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); --- INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); -- [SPARK-27923] Spark SQL insert there obsolete special values to NULL -- Obsolete special values -- INSERT INTO TIMESTAMP_TBL VALUES ('invalid'); diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out index 75d9ee8d9c797..db17ee46c6829 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 18 +-- Number of queries: 30 -- !query 0 @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +INSERT INTO TIMESTAMP_TBL VALUES ('now') -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +INSERT INTO TIMESTAMP_TBL VALUES ('now') -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +INSERT INTO TIMESTAMP_TBL VALUES ('today') -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +INSERT INTO TIMESTAMP_TBL VALUES ('yesterday') -- !query 4 schema struct<> -- !query 4 output @@ -43,139 +43,241 @@ struct<> -- !query 5 -SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow') -- !query 5 schema -struct<64:string,d1:timestamp> +struct<> -- !query 5 output + + + +-- !query 6 +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow Zulu') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today' +-- !query 8 schema +struct +-- !query 8 output +1 + + +-- !query 9 +SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow' +-- !query 9 schema +struct +-- !query 9 output +3 + + +-- !query 10 +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday' +-- !query 10 schema +struct +-- !query 10 output +1 + + +-- !query 11 +TRUNCATE TABLE TIMESTAMP_TBL +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO TIMESTAMP_TBL VALUES ('epoch') +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +-- !query 17 schema +struct<64:string,d1:timestamp> +-- !query 17 output + 1969-12-31 16:00:00 1997-01-02 00:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 6 +-- !query 18 SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL WHERE d1 > timestamp '1997-01-02' --- !query 6 schema +-- !query 18 schema struct<48:string,d1:timestamp> --- !query 6 output +-- !query 18 output 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 7 +-- !query 19 SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL WHERE d1 < timestamp '1997-01-02' --- !query 7 schema +-- !query 19 schema struct<15:string,d1:timestamp> --- !query 7 output +-- !query 19 output + 1969-12-31 16:00:00 - --- !query 8 +-- !query 20 SELECT '' AS one, d1 FROM TIMESTAMP_TBL WHERE d1 = timestamp '1997-01-02' --- !query 8 schema +-- !query 20 schema struct --- !query 8 output +-- !query 20 output 1997-01-02 00:00:00 --- !query 9 +-- !query 21 SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL WHERE d1 != timestamp '1997-01-02' --- !query 9 schema +-- !query 21 schema struct<63:string,d1:timestamp> --- !query 9 output +-- !query 21 output + 1969-12-31 16:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 10 +-- !query 22 SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL WHERE d1 <= timestamp '1997-01-02' --- !query 10 schema +-- !query 22 schema struct<16:string,d1:timestamp> --- !query 10 output +-- !query 22 output + 1969-12-31 16:00:00 1997-01-02 00:00:00 --- !query 11 +-- !query 23 SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL WHERE d1 >= timestamp '1997-01-02' --- !query 11 schema +-- !query 23 schema struct<49:string,d1:timestamp> --- !query 11 output +-- !query 23 output 1997-01-02 00:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 12 +-- !query 24 SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc --- !query 12 schema +-- !query 24 schema struct --- !query 12 output +-- !query 24 output 2004-02-23 00:00:00 --- !query 13 +-- !query 25 SELECT '' AS `54`, d1 as `timestamp`, date_part( 'year', d1) AS `year`, date_part( 'month', d1) AS `month`, date_part( 'day', d1) AS `day`, date_part( 'hour', d1) AS `hour`, date_part( 'minute', d1) AS `minute`, date_part( 'second', d1) AS `second` FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' --- !query 13 schema +-- !query 25 schema struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:int> --- !query 13 output +-- !query 25 output + 1969-12-31 16:00:00 1969 12 31 16 0 0 1997-01-02 00:00:00 1997 1 2 0 0 0 1997-01-02 03:04:05 1997 1 2 3 4 5 1997-02-10 17:32:01 1997 2 10 17 32 1 2001-09-22 18:19:20 2001 9 22 18 19 20 --- !query 14 +-- !query 26 SELECT '' AS `54`, d1 as `timestamp`, date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, date_part( 'usec', d1) AS usec FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' --- !query 14 schema +-- !query 26 schema struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> --- !query 14 output +-- !query 26 output + 1969-12-31 16:00:00 4 0 0 1997-01-02 00:00:00 1 0 0 1997-01-02 03:04:05 1 5000 5000000 1997-02-10 17:32:01 1 1000 1000000 2001-09-22 18:19:20 3 20000 20000000 --- !query 15 +-- !query 27 SELECT '' AS `54`, d1 as `timestamp`, date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, date_part( 'dow', d1) AS dow FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' --- !query 15 schema +-- !query 27 schema struct<54:string,timestamp:timestamp,isoyear:int,week:int,dow:int> --- !query 15 output +-- !query 27 output + 1969-12-31 16:00:00 1970 1 3 1997-01-02 00:00:00 1997 1 4 1997-01-02 03:04:05 1997 1 4 1997-02-10 17:32:01 1997 7 1 2001-09-22 18:19:20 2001 38 6 --- !query 16 +-- !query 28 SELECT make_timestamp(2014,12,28,6,30,45.887) --- !query 16 schema +-- !query 28 schema struct --- !query 16 output +-- !query 28 output 2014-12-28 06:30:45.887 --- !query 17 +-- !query 29 DROP TABLE TIMESTAMP_TBL --- !query 17 schema +-- !query 29 schema struct<> --- !query 17 output +-- !query 29 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index 52cf91cfade51..1094d7d23e5ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import java.sql.Timestamp import java.text.SimpleDateFormat import java.util.Locale @@ -181,4 +182,13 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df, Row(Row(java.sql.Timestamp.valueOf("2018-11-06 18:00:00.0")))) } } + + test("special timestamp values") { + Seq("now", "today", "epoch", "tomorrow", "yesterday").foreach { specialValue => + val input = Seq(specialValue).toDS() + val readback = input.select(from_csv($"value", lit("t timestamp"), + Map.empty[String, String].asJava)).collect() + assert(readback(0).getAs[Row](0).getAs[Timestamp](0).getTime >= 0) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 92a4acc130be5..c61c8109ee8e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import java.sql.Timestamp import java.text.SimpleDateFormat import java.util.Locale @@ -608,4 +609,13 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df, Row(Row(java.sql.Timestamp.valueOf("2018-11-06 18:00:00.0")))) } } + + test("special timestamp values") { + Seq("now", "today", "epoch", "tomorrow", "yesterday").foreach { specialValue => + val input = Seq(s"""{"t": "$specialValue"}""").toDS() + val readback = input.select(from_json($"value", lit("t timestamp"), + Map.empty[String, String].asJava)).collect() + assert(readback(0).getAs[Row](0).getAs[Timestamp](0).getTime >= 0) + } + } } From 203bf9e5698df83a2c7b77ed8bad824784818064 Mon Sep 17 00:00:00 2001 From: Xianjin YE Date: Wed, 18 Sep 2019 23:32:10 +0900 Subject: [PATCH 0106/1387] [SPARK-19926][PYSPARK] make captured exception from JVM side user friendly MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? The str of `CapaturedException` is now returned by str(self.desc) rather than repr(self.desc), which is more user-friendly. It also handles unicode under python2 specially. ### Why are the changes needed? This is an improvement, and makes exception more human readable in python side. ### Does this PR introduce any user-facing change? Before this pr, select `中文字段` throws exception something likes below: ``` Traceback (most recent call last): File "/Users/advancedxy/code_workspace/github/spark/python/pyspark/sql/tests/test_utils.py", line 34, in test_capture_user_friendly_exception raise e AnalysisException: u"cannot resolve '`\u4e2d\u6587\u5b57\u6bb5`' given input columns: []; line 1 pos 7;\n'Project ['\u4e2d\u6587\u5b57\u6bb5]\n+- OneRowRelation\n" ``` after this pr: ``` Traceback (most recent call last): File "/Users/advancedxy/code_workspace/github/spark/python/pyspark/sql/tests/test_utils.py", line 34, in test_capture_user_friendly_exception raise e AnalysisException: cannot resolve '`中文字段`' given input columns: []; line 1 pos 7; 'Project ['中文字段] +- OneRowRelation ``` ### How was this patch Add a new test to verify unicode are correctly converted and manual checks for thrown exceptions. This pr's credits should go to uncleGen and is based on https://github.com/apache/spark/pull/17267 Closes #25814 from advancedxy/python_exception_19926_and_21045. Authored-by: Xianjin YE Signed-off-by: HyukjinKwon --- python/pyspark/sql/tests/test_utils.py | 7 +++++++ python/pyspark/sql/utils.py | 8 +++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_utils.py b/python/pyspark/sql/tests/test_utils.py index fdadd5631cb73..072ea08085fba 100644 --- a/python/pyspark/sql/tests/test_utils.py +++ b/python/pyspark/sql/tests/test_utils.py @@ -1,3 +1,4 @@ +# -*- encoding: utf-8 -*- # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -26,6 +27,12 @@ def test_capture_analysis_exception(self): self.assertRaises(AnalysisException, lambda: self.spark.sql("select abc")) self.assertRaises(AnalysisException, lambda: self.df.selectExpr("a + b")) + def test_capture_user_friendly_exception(self): + try: + self.spark.sql("select `中文字段`") + except AnalysisException as e: + self.assertRegexpMatches(str(e), "cannot resolve '`中文字段`'") + def test_capture_parse_exception(self): self.assertRaises(ParseException, lambda: self.spark.sql("abc")) diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index c30cc1482750a..996b7dd59ce9f 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -16,6 +16,7 @@ # import py4j +import sys class CapturedException(Exception): @@ -25,7 +26,12 @@ def __init__(self, desc, stackTrace, cause=None): self.cause = convert_exception(cause) if cause is not None else None def __str__(self): - return repr(self.desc) + desc = self.desc + # encode unicode instance for python2 for human readable description + if sys.version_info.major < 3 and isinstance(desc, unicode): + return str(desc.encode('utf-8')) + else: + return str(desc) class AnalysisException(CapturedException): From 376e17c082f0cb668a51f9b358ae66eace28ea9f Mon Sep 17 00:00:00 2001 From: sandeep katta Date: Wed, 18 Sep 2019 23:33:13 +0900 Subject: [PATCH 0107/1387] [SPARK-29101][SQL] Fix count API for csv file when DROPMALFORMED mode is selected ### What changes were proposed in this pull request? #DataSet fruit,color,price,quantity apple,red,1,3 banana,yellow,2,4 orange,orange,3,5 xxx This PR aims to fix the below ``` scala> spark.conf.set("spark.sql.csv.parser.columnPruning.enabled", false) scala> spark.read.option("header", "true").option("mode", "DROPMALFORMED").csv("fruit.csv").count res1: Long = 4 ``` This is caused by the issue [SPARK-24645](https://issues.apache.org/jira/browse/SPARK-24645). SPARK-24645 issue can also be solved by [SPARK-25387](https://issues.apache.org/jira/browse/SPARK-25387) ### Why are the changes needed? SPARK-24645 caused this regression, so reverted the code as it can also be solved by SPARK-25387 ### Does this PR introduce any user-facing change? No, ### How was this patch tested? Added UT, and also tested the bug SPARK-24645 **SPARK-24645 regression** ![image](https://user-images.githubusercontent.com/35216143/65067957-4c08ff00-d9a5-11e9-8d43-a4a23a61e8b8.png) Closes #25820 from sandeep-katta/SPARK-29101. Authored-by: sandeep katta Signed-off-by: HyukjinKwon --- .../spark/sql/catalyst/csv/UnivocityParser.scala | 7 ++++--- .../src/test/resources/test-data/malformedRow.csv | 5 +++++ .../sql/execution/datasources/csv/CSVSuite.scala | 14 ++++++++++++++ 3 files changed, 23 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/test/resources/test-data/malformedRow.csv diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 8456b7d218ead..39a08ec06c6a0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -188,12 +188,13 @@ class UnivocityParser( } } - private val doParse = if (requiredSchema.nonEmpty) { - (input: String) => convert(tokenizer.parseLine(input)) - } else { + private val doParse = if (options.columnPruning && requiredSchema.isEmpty) { // If `columnPruning` enabled and partition attributes scanned only, // `schema` gets empty. (_: String) => InternalRow.empty + } else { + // parse if the columnPruning is disabled or requiredSchema is nonEmpty + (input: String) => convert(tokenizer.parseLine(input)) } /** diff --git a/sql/core/src/test/resources/test-data/malformedRow.csv b/sql/core/src/test/resources/test-data/malformedRow.csv new file mode 100644 index 0000000000000..8cfb3eefb982c --- /dev/null +++ b/sql/core/src/test/resources/test-data/malformedRow.csv @@ -0,0 +1,5 @@ +fruit,color,price,quantity +apple,red,1,3 +banana,yellow,2,4 +orange,orange,3,5 +malformedrow diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 5afd019c11a16..835fa4ef166b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -66,6 +66,7 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { private val unescapedQuotesFile = "test-data/unescaped-quotes.csv" private val valueMalformedFile = "test-data/value-malformed.csv" private val badAfterGoodFile = "test-data/bad_after_good.csv" + private val malformedRowFile = "test-data/malformedRow.csv" /** Verifies data and schema. */ private def verifyCars( @@ -2109,4 +2110,17 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { "expect the TextParsingException truncate the error content to be 1000 length.") } } + + test("SPARK-29101 test count with DROPMALFORMED mode") { + Seq((true, 4), (false, 3)).foreach { case (csvColumnPruning, expectedCount) => + withSQLConf(SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> csvColumnPruning.toString) { + val count = spark.read + .option("header", "true") + .option("mode", "DROPMALFORMED") + .csv(testFile(malformedRowFile)) + .count() + assert(expectedCount == count) + } + } + } } From d74fc6bb82f5a7cb8047955bd5fad10b11125e43 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Wed, 18 Sep 2019 09:41:02 -0500 Subject: [PATCH 0108/1387] [SPARK-29118][ML] Avoid redundant computation in transform of GMM & GLR ### What changes were proposed in this pull request? 1,GMM: obtaining the prediction (double) from its probabilty prediction(vector) 2,GLR: obtaining the prediction (double) from its link prediction(double) ### Why are the changes needed? it avoid predict twice ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #25815 from zhengruifeng/gmm_transform_opt. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../spark/ml/clustering/GaussianMixture.scala | 36 ++++++++++--------- .../GeneralizedLinearRegression.scala | 34 +++++++++--------- 2 files changed, 38 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 86caa1247e77f..979eb5e5448a8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -33,8 +33,8 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.functions.udf +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.storage.StorageLevel @@ -111,28 +111,32 @@ class GaussianMixtureModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - var predictionColNames = Seq.empty[String] - var predictionColumns = Seq.empty[Column] - - if ($(predictionCol).nonEmpty) { - val predUDF = udf((vector: Vector) => predict(vector)) - predictionColNames :+= $(predictionCol) - predictionColumns :+= predUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) - } + val vectorCol = DatasetUtils.columnToVector(dataset, $(featuresCol)) + var outputData = dataset + var numColsOutput = 0 if ($(probabilityCol).nonEmpty) { val probUDF = udf((vector: Vector) => predictProbability(vector)) - predictionColNames :+= $(probabilityCol) - predictionColumns :+= probUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) + outputData = outputData.withColumn($(probabilityCol), probUDF(vectorCol)) + numColsOutput += 1 + } + + if ($(predictionCol).nonEmpty) { + if ($(probabilityCol).nonEmpty) { + val predUDF = udf((vector: Vector) => vector.argmax) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(probabilityCol)))) + } else { + val predUDF = udf((vector: Vector) => predict(vector)) + outputData = outputData.withColumn($(predictionCol), predUDF(vectorCol)) + } + numColsOutput += 1 } - if (predictionColNames.nonEmpty) { - dataset.withColumns(predictionColNames, predictionColumns) - } else { + if (numColsOutput == 0) { this.logWarning(s"$uid: GaussianMixtureModel.transform() does nothing" + " because no output columns were set.") - dataset.toDF() } + outputData.toDF } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index a226ca49e6deb..4dc0c247ce331 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -1036,31 +1036,33 @@ class GeneralizedLinearRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - var predictionColNames = Seq.empty[String] - var predictionColumns = Seq.empty[Column] - val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) + var outputData = dataset + var numColsOutput = 0 - if ($(predictionCol).nonEmpty) { - val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } - predictionColNames :+= $(predictionCol) - predictionColumns :+= predictUDF(col($(featuresCol)), offset) + if (hasLinkPredictionCol) { + val predLinkUDF = udf((features: Vector, offset: Double) => predictLink(features, offset)) + outputData = outputData + .withColumn($(linkPredictionCol), predLinkUDF(col($(featuresCol)), offset)) + numColsOutput += 1 } - if (hasLinkPredictionCol) { - val predictLinkUDF = - udf { (features: Vector, offset: Double) => predictLink(features, offset) } - predictionColNames :+= $(linkPredictionCol) - predictionColumns :+= predictLinkUDF(col($(featuresCol)), offset) + if ($(predictionCol).nonEmpty) { + if (hasLinkPredictionCol) { + val predUDF = udf((eta: Double) => familyAndLink.fitted(eta)) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(linkPredictionCol)))) + } else { + val predUDF = udf((features: Vector, offset: Double) => predict(features, offset)) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(featuresCol)), offset)) + } + numColsOutput += 1 } - if (predictionColNames.nonEmpty) { - dataset.withColumns(predictionColNames, predictionColumns) - } else { + if (numColsOutput == 0) { this.logWarning(s"$uid: GeneralizedLinearRegressionModel.transform() does nothing" + " because no output columns were set.") - dataset.toDF() } + outputData.toDF } /** From 276aaaae8d404975f8701089e9f4dfecd16e0d9f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 18 Sep 2019 09:11:55 -0700 Subject: [PATCH 0109/1387] [SPARK-29105][CORE] Keep driver log file size up to date in HDFS HDFS doesn't update the file size reported by the NM if you just keep writing to the file; this makes the SHS believe the file is inactive, and so it may delete it after the configured max age for log files. This change uses hsync to keep the log file as up to date as possible when using HDFS. It also disables erasure coding by default for these logs, since hsync (& friends) does not work with EC. Tested with a SHS configured to aggressively clean up logs; verified a spark-shell session kept updating the log, which was not deleted by the SHS. Closes #25819 from vanzin/SPARK-29105. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../apache/spark/deploy/SparkHadoopUtil.scala | 54 ++++++++++--------- .../spark/internal/config/package.scala | 5 ++ .../scheduler/EventLoggingListener.scala | 8 +-- .../spark/util/logging/DriverLogger.scala | 16 +++++- docs/configuration.md | 10 ++++ .../spark/streaming/util/HdfsUtils.scala | 2 +- 6 files changed, 61 insertions(+), 34 deletions(-) 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 11420bb985520..8ba1ebf92a05a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -454,35 +454,39 @@ private[spark] object SparkHadoopUtil { // scalastyle:off line.size.limit /** - * Create a path that uses replication instead of erasure coding (ec), regardless of the default - * configuration in hdfs for the given path. This can be helpful as hdfs ec doesn't support - * hflush(), hsync(), or append() + * Create a file on the given file system, optionally making sure erasure coding is disabled. + * + * Disabling EC can be helpful as HDFS EC doesn't support hflush(), hsync(), or append(). * https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations */ // scalastyle:on line.size.limit - def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = { - try { - // Use reflection as this uses APIs only available in Hadoop 3 - val builderMethod = fs.getClass().getMethod("createFile", classOf[Path]) - // the builder api does not resolve relative paths, nor does it create parent dirs, while - // the old api does. - if (!fs.mkdirs(path.getParent())) { - throw new IOException(s"Failed to create parents of $path") + def createFile(fs: FileSystem, path: Path, allowEC: Boolean): FSDataOutputStream = { + if (allowEC) { + fs.create(path) + } else { + try { + // Use reflection as this uses APIs only available in Hadoop 3 + val builderMethod = fs.getClass().getMethod("createFile", classOf[Path]) + // the builder api does not resolve relative paths, nor does it create parent dirs, while + // the old api does. + if (!fs.mkdirs(path.getParent())) { + throw new IOException(s"Failed to create parents of $path") + } + val qualifiedPath = fs.makeQualified(path) + val builder = builderMethod.invoke(fs, qualifiedPath) + val builderCls = builder.getClass() + // this may throw a NoSuchMethodException if the path is not on hdfs + val replicateMethod = builderCls.getMethod("replicate") + val buildMethod = builderCls.getMethod("build") + val b2 = replicateMethod.invoke(builder) + buildMethod.invoke(b2).asInstanceOf[FSDataOutputStream] + } catch { + case _: NoSuchMethodException => + // No createFile() method, we're using an older hdfs client, which doesn't give us control + // over EC vs. replication. Older hdfs doesn't have EC anyway, so just create a file with + // old apis. + fs.create(path) } - val qualifiedPath = fs.makeQualified(path) - val builder = builderMethod.invoke(fs, qualifiedPath) - val builderCls = builder.getClass() - // this may throw a NoSuchMethodException if the path is not on hdfs - val replicateMethod = builderCls.getMethod("replicate") - val buildMethod = builderCls.getMethod("build") - val b2 = replicateMethod.invoke(builder) - buildMethod.invoke(b2).asInstanceOf[FSDataOutputStream] - } catch { - case _: NoSuchMethodException => - // No createFile() method, we're using an older hdfs client, which doesn't give us control - // over EC vs. replication. Older hdfs doesn't have EC anyway, so just create a file with - // old apis. - fs.create(path) } } 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 6011901d1d53d..828b70dfe92e9 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 @@ -106,6 +106,11 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val DRIVER_LOG_ALLOW_EC = + ConfigBuilder("spark.driver.log.allowErasureCoding") + .booleanConf + .createWithDefault(false) + private[spark] val EVENT_LOG_ENABLED = ConfigBuilder("spark.eventLog.enabled") .booleanConf .createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 48eb2da3015f8..bee784dcdfe33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -67,7 +67,6 @@ private[spark] class EventLoggingListener( private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) - private val shouldAllowECLogs = sparkConf.get(EVENT_LOG_ALLOW_EC) private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) private val testing = sparkConf.get(EVENT_LOG_TESTING) private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt @@ -121,11 +120,8 @@ private[spark] class EventLoggingListener( if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { new FileOutputStream(uri.getPath) } else { - hadoopDataStream = Some(if (shouldAllowECLogs) { - fileSystem.create(path) - } else { - SparkHadoopUtil.createNonECFile(fileSystem, path) - }) + hadoopDataStream = Some( + SparkHadoopUtil.createFile(fileSystem, path, sparkConf.get(EVENT_LOG_ALLOW_EC))) hadoopDataStream.get } diff --git a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala index c4540433bce97..4c1b49762ace3 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala @@ -18,15 +18,18 @@ package org.apache.spark.util.logging import java.io._ +import java.util.EnumSet import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.hdfs.client.HdfsDataOutputStream import org.apache.log4j.{FileAppender => Log4jFileAppender, _} import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils @@ -111,7 +114,8 @@ private[spark] class DriverLogger(conf: SparkConf) extends Logging { + DriverLogger.DRIVER_LOG_FILE_SUFFIX).getAbsolutePath() try { inStream = new BufferedInputStream(new FileInputStream(localLogFile)) - outputStream = fileSystem.create(new Path(dfsLogFile), true) + outputStream = SparkHadoopUtil.createFile(fileSystem, new Path(dfsLogFile), + conf.get(DRIVER_LOG_ALLOW_EC)) fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS) } catch { case e: Exception => @@ -131,12 +135,20 @@ private[spark] class DriverLogger(conf: SparkConf) extends Logging { } try { var remaining = inStream.available() + val hadData = remaining > 0 while (remaining > 0) { val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE)) outputStream.write(tmpBuffer, 0, read) remaining -= read } - outputStream.hflush() + if (hadData) { + outputStream match { + case hdfsStream: HdfsDataOutputStream => + hdfsStream.hsync(EnumSet.allOf(classOf[HdfsDataOutputStream.SyncFlag])) + case other => + other.hflush() + } + } } catch { case e: Exception => logError("Failed writing driver logs to dfs", e) } diff --git a/docs/configuration.md b/docs/configuration.md index e0b7386ac9bcc..e4a1fd322952b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -411,6 +411,16 @@ of the most common options to set are: use the default layout. + + spark.driver.log.allowErasureCoding + false + + Whether to allow driver logs to use erasure coding. On HDFS, erasure coded files will not + update as quickly as regular replicated files, so they make take longer to reflect changes + written by the application. Note that even if this is true, Spark will still not force the + file to use erasure coding, it will simply use file system defaults. + + Apart from these, the following properties are also available, and may be useful in some situations: diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index 8cb68b2be4ecf..e5df2508268b8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -40,7 +40,7 @@ private[streaming] object HdfsUtils { } } else { // we dont' want to use hdfs erasure coding, as that lacks support for append and hflush - SparkHadoopUtil.createNonECFile(dfs, dfsPath) + SparkHadoopUtil.createFile(dfs, dfsPath, false) } } stream From ee94b5d7019f8ec181d42e953cb8b5190186fe30 Mon Sep 17 00:00:00 2001 From: John Zhuge Date: Wed, 18 Sep 2019 09:27:11 -0700 Subject: [PATCH 0110/1387] [SPARK-29030][SQL] Simplify lookupV2Relation ## What changes were proposed in this pull request? Simplify the return type for `lookupV2Relation` which makes the 3 callers more straightforward. ## How was this patch tested? Existing unit tests. Closes #25735 from jzhuge/lookupv2relation. Authored-by: John Zhuge Signed-off-by: Burak Yavuz --- .../sql/catalyst/analysis/Analyzer.scala | 87 +++++++------------ .../catalog/CatalogV2Implicits.scala | 6 +- 2 files changed, 37 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0a13a34720846..76e59fa785426 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical.sql._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, Identifier, LookupCatalog, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf @@ -666,20 +666,13 @@ class Analyzer( object ResolveTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case u: UnresolvedRelation => - val v2TableOpt = lookupV2Relation(u.multipartIdentifier) match { - case scala.Left((_, _, tableOpt)) => tableOpt - case scala.Right(tableOpt) => tableOpt - } - v2TableOpt.map(DataSourceV2Relation.create).getOrElse(u) + lookupV2Relation(u.multipartIdentifier) + .getOrElse(u) case i @ InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) if i.query.resolved => - val v2TableOpt = lookupV2Relation(u.multipartIdentifier) match { - case scala.Left((_, _, tableOpt)) => tableOpt - case scala.Right(tableOpt) => tableOpt - } - v2TableOpt.map(DataSourceV2Relation.create).map { v2Relation => - i.copy(table = v2Relation) - }.getOrElse(i) + lookupV2Relation(u.multipartIdentifier) + .map(v2Relation => i.copy(table = v2Relation)) + .getOrElse(i) } } @@ -963,26 +956,13 @@ class Analyzer( private def resolveV2Alter( tableName: Seq[String], changes: Seq[TableChange]): Option[AlterTable] = { - lookupV2Relation(tableName) match { - case scala.Left((v2Catalog, ident, tableOpt)) => - Some(AlterTable( - v2Catalog.asTableCatalog, - ident, - tableOpt.map(DataSourceV2Relation.create).getOrElse(UnresolvedRelation(tableName)), - changes - )) - case scala.Right(tableOpt) => - tableOpt.map { table => - AlterTable( - sessionCatalog.asTableCatalog, - Identifier.of(tableName.init.toArray, tableName.last), - DataSourceV2Relation.create(table), - changes - ) - } + lookupV2RelationAndCatalog(tableName).map { + case (relation, catalog, ident) => + AlterTable(catalog.asTableCatalog, ident, relation, changes) } } } + /** * Resolve DESCRIBE TABLE statements that use a DSv2 catalog. * @@ -2840,36 +2820,35 @@ class Analyzer( /** * Performs the lookup of DataSourceV2 Tables. The order of resolution is: - * 1. Check if this relation is a temporary table - * 2. Check if it has a catalog identifier. Here we try to load the table. If we find the table, - * we can return the table. The result returned by an explicit catalog will be returned on - * the Left projection of the Either. - * 3. Try resolving the relation using the V2SessionCatalog if that is defined. If the - * V2SessionCatalog returns a V1 table definition (UnresolvedTable), then we return a `None` - * on the right side so that we can fallback to the V1 code paths. - * The basic idea is, if a value is returned on the Left, it means a v2 catalog is defined and - * must be used to resolve the table. If a value is returned on the right, then we can try - * creating a V2 relation if a V2 Table is defined. If it isn't defined, then we should defer - * to V1 code paths. + * 1. Check if this relation is a temporary table. + * 2. Check if it has a catalog identifier. Here we try to load the table. + * If we find the table, return the v2 relation and catalog. + * 3. Try resolving the relation using the V2SessionCatalog if that is defined. + * If the V2SessionCatalog returns a V1 table definition, + * return `None` so that we can fallback to the V1 code paths. + * If the V2SessionCatalog returns a V2 table, return the v2 relation and V2SessionCatalog. */ - private def lookupV2Relation( - identifier: Seq[String] - ): Either[(CatalogPlugin, Identifier, Option[Table]), Option[Table]] = { - import org.apache.spark.sql.connector.catalog.CatalogV2Util._ - + private def lookupV2RelationAndCatalog( + identifier: Seq[String]): Option[(DataSourceV2Relation, CatalogPlugin, Identifier)] = identifier match { - case AsTemporaryViewIdentifier(ti) if catalog.isTemporaryTable(ti) => - scala.Right(None) + case AsTemporaryViewIdentifier(ti) if catalog.isTemporaryTable(ti) => None case CatalogObjectIdentifier(Some(v2Catalog), ident) => - scala.Left((v2Catalog, ident, loadTable(v2Catalog, ident))) + CatalogV2Util.loadTable(v2Catalog, ident) match { + case Some(table) => Some((DataSourceV2Relation.create(table), v2Catalog, ident)) + case None => None + } case CatalogObjectIdentifier(None, ident) => - loadTable(catalogManager.v2SessionCatalog, ident) match { - case Some(_: V1Table) => scala.Right(None) - case other => scala.Right(other) + CatalogV2Util.loadTable(catalogManager.v2SessionCatalog, ident) match { + case Some(_: V1Table) => None + case Some(table) => + Some((DataSourceV2Relation.create(table), catalogManager.v2SessionCatalog, ident)) + case None => None } - case _ => scala.Right(None) + case _ => None } - } + + private def lookupV2Relation(identifier: Seq[String]): Option[DataSourceV2Relation] = + lookupV2RelationAndCatalog(identifier).map(_._1) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 7d0acceec37da..031e0586a5f0f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -92,8 +92,10 @@ private[sql] object CatalogV2Implicits { } } - implicit class MultipartIdentifierHelper(namespace: Seq[String]) { - def quoted: String = namespace.map(quote).mkString(".") + implicit class MultipartIdentifierHelper(parts: Seq[String]) { + def quoted: String = parts.map(quote).mkString(".") + + def asIdentifier: Identifier = Identifier.of(parts.init.toArray, parts.last) } private def quote(part: String): String = { From dfb0a8bb048d43f8fd1fb05b1027bd2fc7438dbc Mon Sep 17 00:00:00 2001 From: Owen O'Malley Date: Wed, 18 Sep 2019 09:32:43 -0700 Subject: [PATCH 0111/1387] [SPARK-28208][BUILD][SQL] Upgrade to ORC 1.5.6 including closing the ORC readers ## What changes were proposed in this pull request? It upgrades ORC from 1.5.5 to 1.5.6 and adds closes the ORC readers when they aren't used to create RecordReaders. ## How was this patch tested? The changed unit tests were run. Closes #25006 from omalley/spark-28208. Lead-authored-by: Owen O'Malley Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 6 ++--- dev/deps/spark-deps-hadoop-3.2 | 6 ++--- pom.xml | 2 +- .../datasources/orc/OrcFileFormat.scala | 1 + .../execution/datasources/orc/OrcUtils.scala | 2 ++ .../v2/orc/OrcPartitionReaderFactory.scala | 2 ++ .../datasources/orc/OrcQuerySuite.scala | 24 ++++++++++++++----- .../datasources/orc/OrcSourceSuite.scala | 1 + .../sql/hive/execution/HiveDDLSuite.scala | 1 + 9 files changed, 32 insertions(+), 13 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 96cc76d0f2abb..f305be976e593 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -156,9 +156,9 @@ okapi-shade-0.4.2.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.5-nohive.jar -orc-mapreduce-1.5.5-nohive.jar -orc-shims-1.5.5.jar +orc-core-1.5.6-nohive.jar +orc-mapreduce-1.5.6-nohive.jar +orc-shims-1.5.6.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index a3a5b51226462..ae6fc0a0bac8e 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -175,9 +175,9 @@ okhttp-2.7.5.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.5-nohive.jar -orc-mapreduce-1.5.5-nohive.jar -orc-shims-1.5.5.jar +orc-core-1.5.6-nohive.jar +orc-mapreduce-1.5.6-nohive.jar +orc-shims-1.5.6.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index f67918af4dc90..ec6107d292f4b 100644 --- a/pom.xml +++ b/pom.xml @@ -139,7 +139,7 @@ 2.3.0 10.12.1.1 1.10.1 - 1.5.5 + 1.5.6 nohive com.twitter 1.6.0 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 f7c12598da209..59ee63aeab269 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 @@ -184,6 +184,7 @@ class OrcFileFormat val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, requiredSchema, reader, conf) + reader.close() if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty 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 index 12d4244e19812..f3c92f32a3694 100644 --- 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 @@ -64,6 +64,7 @@ object OrcUtils extends Logging { try { val reader = OrcFile.createReader(file, readerOptions) val schema = reader.getSchema + reader.close() if (schema.getFieldNames.size == 0) { None } else { @@ -162,6 +163,7 @@ object OrcUtils extends Logging { if (matchedOrcFields.size > 1) { // Need to fail if there is ambiguity, i.e. more than one field is matched. val matchedOrcFieldsString = matchedOrcFields.mkString("[", ", ", "]") + reader.close() throw new RuntimeException(s"""Found duplicate field(s) "$requiredFieldName": """ + s"$matchedOrcFieldsString in case-insensitive mode") } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 3d6206128cabd..7780ce45e7c69 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -78,6 +78,7 @@ case class OrcPartitionReaderFactory( val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, readDataSchema, reader, conf) + reader.close() if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader[InternalRow] @@ -123,6 +124,7 @@ case class OrcPartitionReaderFactory( val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( isCaseSensitive, dataSchema, readDataSchema, reader, conf) + reader.close() if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index edc1822887f9f..c334d0116ebbb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -194,7 +194,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("ZLIB" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("ZLIB" === reader.getCompressionKind.name) + reader.close() } // `compression` overrides `orc.compress`. @@ -209,7 +211,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("ZLIB" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("ZLIB" === reader.getCompressionKind.name) + reader.close() } } @@ -225,7 +229,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("ZLIB" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("ZLIB" === reader.getCompressionKind.name) + reader.close() } withTempPath { file => @@ -238,7 +244,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("SNAPPY" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("SNAPPY" === reader.getCompressionKind.name) + reader.close() } withTempPath { file => @@ -251,7 +259,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("NONE" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("NONE" === reader.getCompressionKind.name) + reader.close() } } @@ -635,7 +645,9 @@ class OrcQuerySuite extends OrcQueryTest with SharedSparkSession { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - assert("LZO" === OrcFile.createReader(orcFilePath, conf).getCompressionKind.name) + val reader = OrcFile.createReader(orcFilePath, conf) + assert("LZO" === reader.getCompressionKind.name) + reader.close() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 0d904a09c07e8..70d15bb195d53 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -374,6 +374,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { val readerOptions = OrcFile.readerOptions(new Configuration()) val reader = OrcFile.createReader(orcFilePath, readerOptions) val version = UTF_8.decode(reader.getMetadataValue(SPARK_VERSION_METADATA_KEY)).toString + reader.close() assert(version === SPARK_VERSION_SHORT) } } 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 d63d741943bc7..487e888d588de 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 @@ -2370,6 +2370,7 @@ class HiveDDLSuite assert(reader.getCompressionKind.name === "ZLIB") assert(reader.getCompressionSize == 1001) assert(reader.getRowIndexStride == 2002) + reader.close() } } } From cd481773c3577da5df3e1c0eb3b0a8c3c327684a Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Wed, 18 Sep 2019 10:32:10 -0700 Subject: [PATCH 0112/1387] [SPARK-28091][CORE] Extend Spark metrics system with user-defined metrics using executor plugins MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? This proposes to improve Spark instrumentation by adding a hook for user-defined metrics, extending Spark’s Dropwizard/Codahale metrics system. The original motivation of this work was to add instrumentation for S3 filesystem access metrics by Spark job. Currently, [[ExecutorSource]] instruments HDFS and local filesystem metrics. Rather than extending the code there, we proposes with this JIRA to add a metrics plugin system which is of more flexible and general use. Context: The Spark metrics system provides a large variety of metrics, see also , useful to monitor and troubleshoot Spark workloads. A typical workflow is to sink the metrics to a storage system and build dashboards on top of that. Highlights: - The metric plugin system makes it easy to implement instrumentation for S3 access by Spark jobs. - The metrics plugin system allows for easy extensions of how Spark collects HDFS-related workload metrics. This is currently done using the Hadoop Filesystem GetAllStatistics method, which is deprecated in recent versions of Hadoop. Recent versions of Hadoop Filesystem recommend using method GetGlobalStorageStatistics, which also provides several additional metrics. GetGlobalStorageStatistics is not available in Hadoop 2.7 (had been introduced in Hadoop 2.8). Using a metric plugin for Spark would allow an easy way to “opt in” using such new API calls for those deploying suitable Hadoop versions. - We also have the use case of adding Hadoop filesystem monitoring for a custom Hadoop compliant filesystem in use in our organization (EOS using the XRootD protocol). The metrics plugin infrastructure makes this easy to do. Others may have similar use cases. - More generally, this method makes it straightforward to plug in Filesystem and other metrics to the Spark monitoring system. Future work on plugin implementation can address extending monitoring to measure usage of external resources (OS, filesystem, network, accelerator cards, etc), that maybe would not normally be considered general enough for inclusion in Apache Spark code, but that can be nevertheless useful for specialized use cases, tests or troubleshooting. Implementation: The proposed implementation extends and modifies the work on Executor Plugin of SPARK-24918. Additionally, this is related to recent work on extending Spark executor metrics, such as SPARK-25228. As discussed during the review, the implementaiton of this feature modifies the Developer API for Executor Plugins, such that the new version is incompatible with the original version in Spark 2.4. ## How was this patch tested? This modifies existing tests for ExecutorPluginSuite to adapt them to the API changes. In addition, the new funtionality for registering pluginMetrics has been manually tested running Spark on YARN and K8S clusters, in particular for monitoring S3 and for extending HDFS instrumentation with the Hadoop Filesystem “GetGlobalStorageStatistics” metrics. Executor metric plugin example and code used for testing are available, for example at: https://github.com/cerndb/SparkExecutorPlugins Closes #24901 from LucaCanali/executorMetricsPlugin. Authored-by: Luca Canali Signed-off-by: Marcelo Vanzin --- .../java/org/apache/spark/ExecutorPlugin.java | 7 ++- .../apache/spark/ExecutorPluginContext.java | 50 +++++++++++++++++++ .../org/apache/spark/executor/Executor.scala | 10 +++- .../spark/executor/ExecutorPluginSource.scala | 30 +++++++++++ .../org/apache/spark/ExecutorPluginSuite.java | 46 +++++++++++++++-- docs/monitoring.md | 5 ++ project/MimaExcludes.scala | 3 ++ 7 files changed, 144 insertions(+), 7 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/ExecutorPluginContext.java create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala diff --git a/core/src/main/java/org/apache/spark/ExecutorPlugin.java b/core/src/main/java/org/apache/spark/ExecutorPlugin.java index f86520c81df33..b25c46266247e 100644 --- a/core/src/main/java/org/apache/spark/ExecutorPlugin.java +++ b/core/src/main/java/org/apache/spark/ExecutorPlugin.java @@ -40,12 +40,15 @@ public interface ExecutorPlugin { * Initialize the executor plugin. * *

    Each executor will, during its initialization, invoke this method on each - * plugin provided in the spark.executor.plugins configuration.

    + * plugin provided in the spark.executor.plugins configuration. The Spark executor + * will wait on the completion of the execution of the init method.

    * *

    Plugins should create threads in their implementation of this method for * any polling, blocking, or intensive computation.

    + * + * @param pluginContext Context information for the executor where the plugin is running. */ - default void init() {} + default void init(ExecutorPluginContext pluginContext) {} /** * Clean up and terminate this plugin. diff --git a/core/src/main/java/org/apache/spark/ExecutorPluginContext.java b/core/src/main/java/org/apache/spark/ExecutorPluginContext.java new file mode 100644 index 0000000000000..8f018732b8217 --- /dev/null +++ b/core/src/main/java/org/apache/spark/ExecutorPluginContext.java @@ -0,0 +1,50 @@ +/* + * 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; + +import com.codahale.metrics.MetricRegistry; +import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.annotation.Private; + +/** + * Encapsulates information about the executor when initializing {@link ExecutorPlugin} instances. + */ +@DeveloperApi +public class ExecutorPluginContext { + + public final MetricRegistry metricRegistry; + public final SparkConf sparkConf; + public final String executorId; + public final String executorHostName; + public final boolean isLocal; + + @Private + public ExecutorPluginContext( + MetricRegistry registry, + SparkConf conf, + String id, + String hostName, + boolean local) { + metricRegistry = registry; + sparkConf = conf; + executorId = id; + executorHostName = hostName; + isLocal = local; + } + +} diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index a32226581b973..12bc227b341a3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -146,8 +146,14 @@ private[spark] class Executor( Utils.withContextClassLoader(replClassLoader) { val plugins = Utils.loadExtensions(classOf[ExecutorPlugin], pluginNames, conf) plugins.foreach { plugin => - plugin.init() - logInfo(s"Successfully loaded plugin " + plugin.getClass().getCanonicalName()) + val pluginSource = new ExecutorPluginSource(plugin.getClass().getSimpleName()) + val pluginContext = new ExecutorPluginContext(pluginSource.metricRegistry, conf, + executorId, executorHostname, isLocal) + plugin.init(pluginContext) + logInfo("Successfully loaded plugin " + plugin.getClass().getCanonicalName()) + if (pluginSource.metricRegistry.getNames.size() > 0) { + env.metricsSystem.registerSource(pluginSource) + } } plugins } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala new file mode 100644 index 0000000000000..5625e953c5e67 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.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.executor + +import com.codahale.metrics.MetricRegistry + +import org.apache.spark.metrics.source.Source + +private[spark] +class ExecutorPluginSource(name: String) extends Source { + + override val metricRegistry = new MetricRegistry() + + override val sourceName = name +} diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java index 80cd70282a51d..ade13f02bde73 100644 --- a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java +++ b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java @@ -17,6 +17,8 @@ package org.apache.spark; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.MetricRegistry; import org.apache.spark.api.java.JavaSparkContext; import org.junit.After; @@ -30,6 +32,7 @@ public class ExecutorPluginSuite { private static final String testBadPluginName = TestBadShutdownPlugin.class.getName(); private static final String testPluginName = TestExecutorPlugin.class.getName(); private static final String testSecondPluginName = TestSecondPlugin.class.getName(); + private static final String testMetricsPluginName = TestMetricsPlugin.class.getName(); // Static value modified by testing plugins to ensure plugins loaded correctly. public static int numSuccessfulPlugins = 0; @@ -37,6 +40,10 @@ public class ExecutorPluginSuite { // Static value modified by testing plugins to verify plugins shut down properly. public static int numSuccessfulTerminations = 0; + // Static values modified by testing plugins to ensure metrics have been registered correctly. + public static MetricRegistry testMetricRegistry; + public static String gaugeName; + private JavaSparkContext sc; @Before @@ -107,8 +114,21 @@ public void testPluginShutdownWithException() { assertEquals(2, numSuccessfulTerminations); } + @Test + public void testPluginMetrics() { + // Verify that a custom metric is registered with the Spark metrics system + gaugeName = "test42"; + SparkConf conf = initializeSparkConf(testMetricsPluginName); + sc = new JavaSparkContext(conf); + assertEquals(1, numSuccessfulPlugins); + assertEquals(gaugeName, testMetricRegistry.getGauges().firstKey()); + sc.stop(); + sc = null; + assertEquals(1, numSuccessfulTerminations); + } + public static class TestExecutorPlugin implements ExecutorPlugin { - public void init() { + public void init(ExecutorPluginContext pluginContext) { ExecutorPluginSuite.numSuccessfulPlugins++; } @@ -118,7 +138,7 @@ public void shutdown() { } public static class TestSecondPlugin implements ExecutorPlugin { - public void init() { + public void init(ExecutorPluginContext pluginContext) { ExecutorPluginSuite.numSuccessfulPlugins++; } @@ -128,7 +148,7 @@ public void shutdown() { } public static class TestBadShutdownPlugin implements ExecutorPlugin { - public void init() { + public void init(ExecutorPluginContext pluginContext) { ExecutorPluginSuite.numSuccessfulPlugins++; } @@ -136,4 +156,24 @@ public void shutdown() { throw new RuntimeException("This plugin will fail to cleanly shut down"); } } + + public static class TestMetricsPlugin implements ExecutorPlugin { + public void init(ExecutorPluginContext myContext) { + MetricRegistry metricRegistry = myContext.metricRegistry; + // Registers a dummy metrics gauge for testing + String gaugeName = ExecutorPluginSuite.gaugeName; + metricRegistry.register(MetricRegistry.name(gaugeName), new Gauge() { + @Override + public Integer getValue() { + return 42; + } + }); + ExecutorPluginSuite.testMetricRegistry = metricRegistry; + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + ExecutorPluginSuite.numSuccessfulTerminations++; + } + } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 8c81916d4f7d0..a45a41dc78cc3 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -1059,6 +1059,11 @@ when running in local mode. - hiveClientCalls.count - sourceCodeSize (histogram) +- namespace= + - Optional namespace(s). Metrics in this namespace are defined by user-supplied code, and + configured using the Spark executor plugin infrastructure. + See also the configuration parameter `spark.executor.plugins` + ### Source = JVM Source Notes: - Activate this source by setting the relevant `metrics.properties` file entry or the diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 097f1d2c2a6e1..38738112849f7 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,9 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( + // [SPARK-28091[CORE] Extend Spark metrics system with user-defined metrics using executor plugins + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExecutorPlugin.init"), + // [SPARK-][SQL][CORE][MLLIB] Remove more old deprecated items in Spark 3 ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createExternalTable"), From b4b2e958ce2d7142eb4ffb56401acf1739039555 Mon Sep 17 00:00:00 2001 From: bartosz25 Date: Wed, 18 Sep 2019 10:51:11 -0700 Subject: [PATCH 0113/1387] [MINOR][SS][DOCS] Adapt multiple watermark policy comment to the reality ### What changes were proposed in this pull request? Previous comment was true for Apache Spark 2.3.0. The 2.4.0 release brought multiple watermark policy and therefore stating that the 'min' is always chosen is misleading. This PR updates the comments about multiple watermark policy. They aren't true anymore since in case of multiple watermarks, we can configure which one will be applied to the query. This change was brought with Apache Spark 2.4.0 release. ### Why are the changes needed? It introduces some confusion about the real execution of the commented code. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? The tests weren't added because the change is only about the documentation level. I affirm that the contribution is my original work and that I license the work to the project under the project's open source license. Closes #25832 from bartosz25/fix_comments_multiple_watermark_policy. Authored-by: bartosz25 Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/streaming/WatermarkTracker.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala index 76ab1284633b1..b0f8cf9cd1846 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/WatermarkTracker.scala @@ -63,7 +63,7 @@ case object MinWatermark extends MultipleWatermarkPolicy { } /** - * Policy to choose the *min* of the operator watermark values as the global watermark value. So the + * Policy to choose the *max* of the operator watermark values as the global watermark value. So the * global watermark will advance if any of the individual operator watermarks has advanced. * In other words, in a streaming query with multiple input streams and watermarks defined on all * of them, the global watermark will advance as fast as the fastest input. So if there is watermark @@ -108,10 +108,9 @@ case class WatermarkTracker(policy: MultipleWatermarkPolicy) extends Logging { } } - // Update the global watermark to the minimum of all watermark nodes. - // This is the safest option, because only the global watermark is fault-tolerant. Making - // it the minimum of all individual watermarks guarantees it will never advance past where - // any individual watermark operator would be if it were in a plan by itself. + // Update the global watermark accordingly to the chosen policy. To find all available policies + // and their semantics, please check the comments of + // `org.apache.spark.sql.execution.streaming.MultipleWatermarkPolicy` implementations. val chosenGlobalWatermark = policy.chooseGlobalWatermark(operatorToWatermarkMap.values.toSeq) if (chosenGlobalWatermark > globalWatermarkMs) { logInfo(s"Updating event-time watermark from $globalWatermarkMs to $chosenGlobalWatermark ms") From db9e0fda6bdfc3946a5011772b6b0180596ea331 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 18 Sep 2019 12:16:06 -0700 Subject: [PATCH 0114/1387] [SPARK-22796][PYTHON][ML] Add multiple columns support to PySpark QuantileDiscretizer ### What changes were proposed in this pull request? Add multiple columns support to PySpark QuantileDiscretizer ### Why are the changes needed? Multiple columns support for QuantileDiscretizer was in scala side a while ago. We need to add multiple columns support to python too. ### Does this PR introduce any user-facing change? Yes. New Python is added ### How was this patch tested? Add doctest Closes #25812 from huaxingao/spark-22796. Authored-by: Huaxin Gao Signed-off-by: Liang-Chi Hsieh --- python/pyspark/ml/feature.py | 121 +++++++++++++++++++++++++++-------- 1 file changed, 95 insertions(+), 26 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 5f55e48ce9f45..94bdd24f0f887 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1959,17 +1959,22 @@ def getDegree(self): @inherit_doc -class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasHandleInvalid, - JavaMLReadable, JavaMLWritable): +class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, + HasHandleInvalid, JavaMLReadable, JavaMLWritable): """ - `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned - categorical features. The number of bins can be set using the :py:attr:`numBuckets` parameter. - It is possible that the number of buckets used will be less than this value, for example, if - there are too few distinct values of the input to create enough distinct quantiles. + :py:class:`QuantileDiscretizer` takes a column with continuous features and outputs a column + with binned categorical features. The number of bins can be set using the :py:attr:`numBuckets` + parameter. It is possible that the number of buckets used will be less than this value, for + example, if there are too few distinct values of the input to create enough distinct quantiles. + Since 3.0.0, :py:class:`QuantileDiscretizer` can map multiple columns at once by setting the + :py:attr:`inputCols` parameter. If both of the :py:attr:`inputCol` and :py:attr:`inputCols` + parameters are set, an Exception will be thrown. To specify the number of buckets for each + column, the :py:attr:`numBucketsArray` parameter can be set, or if the number of buckets + should be the same across columns, :py:attr:`numBuckets` can be set as a convenience. NaN handling: Note also that - QuantileDiscretizer will raise an error when it finds NaN values in the dataset, but the user - can also choose to either keep or remove NaN values within the dataset by setting + :py:class:`QuantileDiscretizer` will raise an error when it finds NaN values in the dataset, + but the user can also choose to either keep or remove NaN values within the dataset by setting :py:attr:`handleInvalid` parameter. If the user chooses to keep NaN values, they will be handled specially and placed into their own bucket, for example, if 4 buckets are used, then non-NaN data will be put into buckets[0-3], but NaNs will be counted in a special bucket[4]. @@ -1981,29 +1986,61 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasHandleInv The lower and upper bin bounds will be `-Infinity` and `+Infinity`, covering all real values. >>> values = [(0.1,), (0.4,), (1.2,), (1.5,), (float("nan"),), (float("nan"),)] - >>> df = spark.createDataFrame(values, ["values"]) - >>> qds = QuantileDiscretizer(numBuckets=2, + >>> df1 = spark.createDataFrame(values, ["values"]) + >>> qds1 = QuantileDiscretizer(numBuckets=2, ... inputCol="values", outputCol="buckets", relativeError=0.01, handleInvalid="error") - >>> qds.getRelativeError() + >>> qds1.getRelativeError() 0.01 - >>> bucketizer = qds.fit(df) - >>> qds.setHandleInvalid("keep").fit(df).transform(df).count() + >>> bucketizer = qds1.fit(df1) + >>> qds1.setHandleInvalid("keep").fit(df1).transform(df1).count() 6 - >>> qds.setHandleInvalid("skip").fit(df).transform(df).count() + >>> qds1.setHandleInvalid("skip").fit(df1).transform(df1).count() 4 >>> splits = bucketizer.getSplits() >>> splits[0] -inf >>> print("%2.1f" % round(splits[1], 1)) 0.4 - >>> bucketed = bucketizer.transform(df).head() + >>> bucketed = bucketizer.transform(df1).head() >>> bucketed.buckets 0.0 >>> quantileDiscretizerPath = temp_path + "/quantile-discretizer" - >>> qds.save(quantileDiscretizerPath) + >>> qds1.save(quantileDiscretizerPath) >>> loadedQds = QuantileDiscretizer.load(quantileDiscretizerPath) - >>> loadedQds.getNumBuckets() == qds.getNumBuckets() + >>> loadedQds.getNumBuckets() == qds1.getNumBuckets() True + >>> inputs = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, 1.5), + ... (float("nan"), float("nan")), (float("nan"), float("nan"))] + >>> df2 = spark.createDataFrame(inputs, ["input1", "input2"]) + >>> qds2 = QuantileDiscretizer(relativeError=0.01, handleInvalid="error", numBuckets=2, + ... inputCols=["input1", "input2"], outputCols=["output1", "output2"]) + >>> qds2.getRelativeError() + 0.01 + >>> qds2.setHandleInvalid("keep").fit(df2).transform(df2).show() + +------+------+-------+-------+ + |input1|input2|output1|output2| + +------+------+-------+-------+ + | 0.1| 0.0| 0.0| 0.0| + | 0.4| 1.0| 1.0| 1.0| + | 1.2| 1.3| 1.0| 1.0| + | 1.5| 1.5| 1.0| 1.0| + | NaN| NaN| 2.0| 2.0| + | NaN| NaN| 2.0| 2.0| + +------+------+-------+-------+ + ... + >>> qds3 = QuantileDiscretizer(relativeError=0.01, handleInvalid="error", + ... numBucketsArray=[5, 10], inputCols=["input1", "input2"], + ... outputCols=["output1", "output2"]) + >>> qds3.setHandleInvalid("skip").fit(df2).transform(df2).show() + +------+------+-------+-------+ + |input1|input2|output1|output2| + +------+------+-------+-------+ + | 0.1| 0.0| 1.0| 1.0| + | 0.4| 1.0| 2.0| 2.0| + | 1.2| 1.3| 3.0| 3.0| + | 1.5| 1.5| 4.0| 4.0| + +------+------+-------+-------+ + ... .. versionadded:: 2.0.0 """ @@ -2021,15 +2058,26 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasHandleInv handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. " + "Options are skip (filter out rows with invalid values), " + "error (throw an error), or keep (keep invalid values in a special " + - "additional bucket).", + "additional bucket). Note that in the multiple columns " + + "case, the invalid handling is applied to all columns. That said " + + "for 'error' it will throw an error if any invalids are found in " + + "any columns, for 'skip' it will skip rows with any invalids in " + + "any columns, etc.", typeConverter=TypeConverters.toString) + numBucketsArray = Param(Params._dummy(), "numBucketsArray", "Array of number of buckets " + + "(quantiles, or categories) into which data points are grouped. " + + "This is for multiple columns input. If transforming multiple " + + "columns and numBucketsArray is not set, but numBuckets is set, " + + "then numBuckets will be applied across all columns.", + typeConverter=TypeConverters.toListInt) + @keyword_only def __init__(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001, - handleInvalid="error"): + handleInvalid="error", numBucketsArray=None, inputCols=None, outputCols=None): """ __init__(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001, \ - handleInvalid="error") + handleInvalid="error", numBucketsArray=None, inputCols=None, outputCols=None) """ super(QuantileDiscretizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.QuantileDiscretizer", @@ -2041,10 +2089,10 @@ def __init__(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0. @keyword_only @since("2.0.0") def setParams(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001, - handleInvalid="error"): + handleInvalid="error", numBucketsArray=None, inputCols=None, outputCols=None): """ setParams(self, numBuckets=2, inputCol=None, outputCol=None, relativeError=0.001, \ - handleInvalid="error") + handleInvalid="error", numBucketsArray=None, inputCols=None, outputCols=None) Set the params for the QuantileDiscretizer """ kwargs = self._input_kwargs @@ -2064,6 +2112,20 @@ def getNumBuckets(self): """ return self.getOrDefault(self.numBuckets) + @since("3.0.0") + def setNumBucketsArray(self, value): + """ + Sets the value of :py:attr:`numBucketsArray`. + """ + return self._set(numBucketsArray=value) + + @since("3.0.0") + def getNumBucketsArray(self): + """ + Gets the value of numBucketsArray or its default value. + """ + return self.getOrDefault(self.numBucketsArray) + @since("2.0.0") def setRelativeError(self, value): """ @@ -2082,10 +2144,17 @@ def _create_model(self, java_model): """ Private method to convert the java_model to a Python model. """ - return Bucketizer(splits=list(java_model.getSplits()), - inputCol=self.getInputCol(), - outputCol=self.getOutputCol(), - handleInvalid=self.getHandleInvalid()) + if (self.isSet(self.inputCol)): + return Bucketizer(splits=list(java_model.getSplits()), + inputCol=self.getInputCol(), + outputCol=self.getOutputCol(), + handleInvalid=self.getHandleInvalid()) + else: + splitsArrayList = [list(x) for x in list(java_model.getSplitsArray())] + return Bucketizer(splitsArray=splitsArrayList, + inputCols=self.getInputCols(), + outputCols=self.getOutputCols(), + handleInvalid=self.getHandleInvalid()) @inherit_doc From f32f16fd68f51a94d6adb2d01a9b2e557885e656 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 18 Sep 2019 13:30:00 -0700 Subject: [PATCH 0115/1387] [SPARK-29082][CORE] Skip delegation token generation if no credentials are available This situation can happen when an external system (e.g. Oozie) generates delegation tokens for a Spark application. The Spark driver will then run against secured services, have proper credentials (the tokens), but no kerberos credentials. So trying to do things that requires a kerberos credential fails. Instead, if no kerberos credentials are detected, just skip the whole delegation token code. Tested with an application that simulates Oozie; fails before the fix, passes with the fix. Also with other DT-related tests to make sure other functionality keeps working. Closes #25805 from vanzin/SPARK-29082. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../HadoopDelegationTokenManager.scala | 22 +++++++---- .../CoarseGrainedSchedulerBackend.scala | 10 +++-- .../HadoopDelegationTokenManagerSuite.scala | 39 ++++++++++++++++++- 3 files changed, 59 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index 759d857d56e0e..f769ce468e49c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -140,13 +140,21 @@ private[spark] class HadoopDelegationTokenManager( * @param creds Credentials object where to store the delegation tokens. */ def obtainDelegationTokens(creds: Credentials): Unit = { - val freshUGI = doLogin() - freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { - override def run(): Unit = { - val (newTokens, _) = obtainDelegationTokens() - creds.addAll(newTokens) - } - }) + val currentUser = UserGroupInformation.getCurrentUser() + val hasKerberosCreds = principal != null || + Option(currentUser.getRealUser()).getOrElse(currentUser).hasKerberosCredentials() + + // Delegation tokens can only be obtained if the real user has Kerberos credentials, so + // skip creation when those are not available. + if (hasKerberosCreds) { + val freshUGI = doLogin() + freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val (newTokens, _) = obtainDelegationTokens() + creds.addAll(newTokens) + } + }) + } } /** 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 d81070c362ba6..c9b408bed1163 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 @@ -427,12 +427,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val ugi = UserGroupInformation.getCurrentUser() val tokens = if (dtm.renewalEnabled) { dtm.start() - } else if (ugi.hasKerberosCredentials() || SparkHadoopUtil.get.isProxyUser(ugi)) { + } else { val creds = ugi.getCredentials() dtm.obtainDelegationTokens(creds) - SparkHadoopUtil.get.serialize(creds) - } else { - null + if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { + SparkHadoopUtil.get.serialize(creds) + } else { + null + } } if (tokens != null) { updateDelegationTokens(tokens) diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index 70174f7ff939a..bf53386da6304 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -17,10 +17,14 @@ package org.apache.spark.deploy.security +import java.security.PrivilegedExceptionAction + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.Credentials +import org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.security.HadoopDelegationTokenProvider private class ExceptionThrowingDelegationTokenProvider extends HadoopDelegationTokenProvider { @@ -69,4 +73,37 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { assert(!manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) } + + test("SPARK-29082: do not fail if current user does not have credentials") { + // SparkHadoopUtil overrides the UGI configuration during initialization. That normally + // happens early in the Spark application, but here it may affect the test depending on + // how it's run, so force its initialization. + SparkHadoopUtil.get + + val krbConf = new Configuration() + krbConf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos") + + UserGroupInformation.setConfiguration(krbConf) + try { + val manager = new HadoopDelegationTokenManager(new SparkConf(false), krbConf, null) + val testImpl = new PrivilegedExceptionAction[Unit] { + override def run(): Unit = { + assert(UserGroupInformation.isSecurityEnabled()) + val creds = new Credentials() + manager.obtainDelegationTokens(creds) + assert(creds.numberOfTokens() === 0) + assert(creds.numberOfSecretKeys() === 0) + } + } + + val realUser = UserGroupInformation.createUserForTesting("realUser", Array.empty) + realUser.doAs(testImpl) + + val proxyUser = UserGroupInformation.createProxyUserForTesting("proxyUser", realUser, + Array.empty) + proxyUser.doAs(testImpl) + } finally { + UserGroupInformation.reset() + } + } } From 8c3f27ceb43d7cfb78bda7f39e2758fb26f660d4 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 18 Sep 2019 13:30:36 -0700 Subject: [PATCH 0116/1387] [SPARK-28683][BUILD] Upgrade Scala to 2.12.10 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? This PR upgrade Scala to **2.12.10**. Release notes: - Fix regression in large string interpolations with non-String typed splices - Revert "Generate shallower ASTs in pattern translation" - Fix regression in classpath when JARs have 'a.b' entries beside 'a/b' - Faster compiler: 5–10% faster since 2.12.8 - Improved compatibility with JDK 11, 12, and 13 - Experimental support for build pipelining and outline type checking More details: https://github.com/scala/scala/releases/tag/v2.12.10 https://github.com/scala/scala/releases/tag/v2.12.9 ## How was this patch tested? Existing tests Closes #25404 from wangyum/SPARK-28683. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 6 +++--- dev/deps/spark-deps-hadoop-3.2 | 6 +++--- docs/_config.yml | 2 +- external/docker/spark-test/base/Dockerfile | 2 +- pom.xml | 2 +- project/SparkBuild.scala | 2 +- .../org/apache/spark/sql/catalyst/ScalaReflection.scala | 2 +- 7 files changed, 11 insertions(+), 11 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index f305be976e593..cae2614128b1a 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -172,10 +172,10 @@ parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar -scala-compiler-2.12.8.jar -scala-library-2.12.8.jar +scala-compiler-2.12.10.jar +scala-library-2.12.10.jar scala-parser-combinators_2.12-1.1.0.jar -scala-reflect-2.12.8.jar +scala-reflect-2.12.10.jar scala-xml_2.12-1.2.0.jar shapeless_2.12-2.3.2.jar shims-0.7.45.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index ae6fc0a0bac8e..21df5a70e7f8d 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -191,10 +191,10 @@ protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar re2j-1.1.jar -scala-compiler-2.12.8.jar -scala-library-2.12.8.jar +scala-compiler-2.12.10.jar +scala-library-2.12.10.jar scala-parser-combinators_2.12-1.1.0.jar -scala-reflect-2.12.8.jar +scala-reflect-2.12.10.jar scala-xml_2.12-1.2.0.jar shapeless_2.12-2.3.2.jar shims-0.7.45.jar diff --git a/docs/_config.yml b/docs/_config.yml index 146c90fcff6e5..57b8d716ee55c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -17,7 +17,7 @@ include: SPARK_VERSION: 3.0.0-SNAPSHOT SPARK_VERSION_SHORT: 3.0.0 SCALA_BINARY_VERSION: "2.12" -SCALA_VERSION: "2.12.8" +SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/external/docker/spark-test/base/Dockerfile b/external/docker/spark-test/base/Dockerfile index c1fd630d0b665..5bec5d3f16548 100644 --- a/external/docker/spark-test/base/Dockerfile +++ b/external/docker/spark-test/base/Dockerfile @@ -25,7 +25,7 @@ RUN apt-get update && \ apt-get install -y less openjdk-8-jre-headless iproute2 vim-tiny sudo openssh-server && \ rm -rf /var/lib/apt/lists/* -ENV SCALA_VERSION 2.12.8 +ENV SCALA_VERSION 2.12.10 ENV CDH_VERSION cdh4 ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark diff --git a/pom.xml b/pom.xml index ec6107d292f4b..a78a5beb814f5 100644 --- a/pom.xml +++ b/pom.xml @@ -164,7 +164,7 @@ 3.4.1 3.2.2 - 2.12.8 + 2.12.10 2.12 --diff --test diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 25c2fb4af5c34..d984f5dacf1af 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -223,7 +223,7 @@ object SparkBuild extends PomBuild { .map(file), incOptions := incOptions.value.withNameHashing(true), publishMavenStyle := true, - unidocGenjavadocVersion := "0.13", + unidocGenjavadocVersion := "0.14", // Override SBT's default resolvers: resolvers := Seq( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index fbbbda33538d1..ceb3155500385 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -72,7 +72,7 @@ object ScalaReflection extends ScalaReflection { /** * Synchronize to prevent concurrent usage of `<:<` operator. * This operator is not thread safe in any current version of scala; i.e. - * (2.11.12, 2.12.8, 2.13.0-M5). + * (2.11.12, 2.12.10, 2.13.0-M5). * * See https://github.com/scala/bug/issues/10766 */ From a6a663c4379390217443bc5b6f75873fb1c38c73 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Sep 2019 17:52:23 -0700 Subject: [PATCH 0117/1387] [SPARK-29141][SQL][TEST] Use SqlBasedBenchmark in SQL benchmarks ### What changes were proposed in this pull request? Refactored SQL-related benchmark and made them depend on `SqlBasedBenchmark`. In particular, creation of Spark session are moved into `override def getSparkSession: SparkSession`. ### Why are the changes needed? This should simplify maintenance of SQL-based benchmarks by reducing the number of dependencies. In the future, it should be easier to refactor & extend all SQL benchmarks by changing only one trait. Finally, all SQL-based benchmarks will look uniformly. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the modified benchmarks. Closes #25828 from MaxGekk/sql-benchmarks-refactoring. Lead-authored-by: Maxim Gekk Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../benchmarks/AvroReadBenchmark-results.txt | 124 ++--- .../benchmark/AvroReadBenchmark.scala | 3 +- .../DataSourceReadBenchmark-results.txt | 428 +++++++++--------- .../benchmarks/ExtractBenchmark-results.txt | 176 +++---- .../PrimitiveArrayBenchmark-results.txt | 13 +- .../benchmark/DataSourceReadBenchmark.scala | 38 +- .../benchmark/ExtractBenchmark.scala | 10 +- .../benchmark/FilterPushdownBenchmark.scala | 31 +- .../benchmark/PrimitiveArrayBenchmark.scala | 23 +- ...jectHashAggregateExecBenchmark-results.txt | 48 +- .../benchmarks/OrcReadBenchmark-results.txt | 192 ++++---- .../ObjectHashAggregateExecBenchmark.scala | 8 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 33 +- 13 files changed, 566 insertions(+), 561 deletions(-) diff --git a/external/avro/benchmarks/AvroReadBenchmark-results.txt b/external/avro/benchmarks/AvroReadBenchmark-results.txt index 7900fea453b10..8a12d9d15d915 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-results.txt @@ -2,121 +2,121 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2774 / 2815 5.7 176.4 1.0X +SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2958 3072 161 5.3 188.1 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2761 / 2777 5.7 175.5 1.0X +SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2856 2866 15 5.5 181.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2783 / 2870 5.7 176.9 1.0X +SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2883 2890 10 5.5 183.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 3256 / 3266 4.8 207.0 1.0X +SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 3336 3360 34 4.7 212.1 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2841 / 2867 5.5 180.6 1.0X +SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2728 2760 45 5.8 173.5 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum 2981 / 2996 5.3 189.5 1.0X +SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum 2826 2833 9 5.6 179.7 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of columns 4781 / 4783 2.2 456.0 1.0X +Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of columns 4682 4704 30 2.2 446.6 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Data column 3372 / 3386 4.7 214.4 1.0X -Partition column 3035 / 3064 5.2 193.0 1.1X -Both columns 3445 / 3461 4.6 219.1 1.0X +Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Data column 3206 3265 84 4.9 203.8 1.0X +Partition column 3314 3316 3 4.7 210.7 1.0X +Both columns 3353 3367 20 4.7 213.2 1.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of string length 3395 / 3401 3.1 323.8 1.0X +Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of string length 3364 3367 4 3.1 320.8 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of string length 5580 / 5624 1.9 532.2 1.0X +String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of string length 5403 5410 10 1.9 515.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of string length 4622 / 4623 2.3 440.8 1.0X +String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of string length 4524 4526 2 2.3 431.5 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of string length 3238 / 3241 3.2 308.8 1.0X +String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of string length 3121 3129 11 3.4 297.7 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of single column 5472 / 5484 0.2 5218.8 1.0X +Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of single column 4537 4541 6 0.2 4326.9 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of single column 10680 / 10701 0.1 10185.1 1.0X +Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of single column 8768 8776 11 0.1 8361.8 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Sum of single column 16143 / 16238 0.1 15394.9 1.0X +Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Sum of single column 13042 13064 31 0.1 12437.6 1.0X diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala index f2f7d650066fb..a16126ae24246 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala @@ -22,7 +22,6 @@ import scala.util.Random import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.types._ /** @@ -36,7 +35,7 @@ import org.apache.spark.sql.types._ * Results will be written to "benchmarks/AvroReadBenchmark-results.txt". * }}} */ -object AvroReadBenchmark extends SqlBasedBenchmark with SQLHelper { +object AvroReadBenchmark extends SqlBasedBenchmark { def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) } diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt index f547f61654b5f..efb0601cc5d65 100644 --- a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt +++ b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt @@ -2,251 +2,251 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 26366 / 26562 0.6 1676.3 1.0X -SQL Json 8709 / 8724 1.8 553.7 3.0X -SQL Parquet Vectorized 166 / 187 94.8 10.5 159.0X -SQL Parquet MR 1706 / 1720 9.2 108.4 15.5X -SQL ORC Vectorized 167 / 174 94.2 10.6 157.9X -SQL ORC MR 1433 / 1465 11.0 91.1 18.4X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 200 / 207 78.7 12.7 1.0X -ParquetReader Vectorized -> Row 117 / 119 134.7 7.4 1.7X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 26489 / 26547 0.6 1684.1 1.0X -SQL Json 8990 / 8998 1.7 571.5 2.9X -SQL Parquet Vectorized 209 / 221 75.1 13.3 126.5X -SQL Parquet MR 1949 / 1949 8.1 123.9 13.6X -SQL ORC Vectorized 221 / 228 71.3 14.0 120.1X -SQL ORC MR 1527 / 1549 10.3 97.1 17.3X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 286 / 296 54.9 18.2 1.0X -ParquetReader Vectorized -> Row 249 / 253 63.1 15.8 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 27701 / 27744 0.6 1761.2 1.0X -SQL Json 9703 / 9733 1.6 616.9 2.9X -SQL Parquet Vectorized 176 / 182 89.2 11.2 157.0X -SQL Parquet MR 2164 / 2173 7.3 137.6 12.8X -SQL ORC Vectorized 307 / 314 51.2 19.5 90.2X -SQL ORC MR 1690 / 1700 9.3 107.4 16.4X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 259 / 277 60.7 16.5 1.0X -ParquetReader Vectorized -> Row 261 / 265 60.3 16.6 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 34813 / 34900 0.5 2213.3 1.0X -SQL Json 12570 / 12617 1.3 799.2 2.8X -SQL Parquet Vectorized 270 / 308 58.2 17.2 128.9X -SQL Parquet MR 2427 / 2431 6.5 154.3 14.3X -SQL ORC Vectorized 388 / 398 40.6 24.6 89.8X -SQL ORC MR 1819 / 1851 8.6 115.7 19.1X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 372 / 379 42.3 23.7 1.0X -ParquetReader Vectorized -> Row 357 / 368 44.1 22.7 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 28753 / 28781 0.5 1828.0 1.0X -SQL Json 12039 / 12215 1.3 765.4 2.4X -SQL Parquet Vectorized 170 / 177 92.4 10.8 169.0X -SQL Parquet MR 2184 / 2196 7.2 138.9 13.2X -SQL ORC Vectorized 432 / 440 36.4 27.5 66.5X -SQL ORC MR 1812 / 1833 8.7 115.2 15.9X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 253 / 260 62.2 16.1 1.0X -ParquetReader Vectorized -> Row 256 / 257 61.6 16.2 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 36177 / 36188 0.4 2300.1 1.0X -SQL Json 18895 / 18898 0.8 1201.3 1.9X -SQL Parquet Vectorized 267 / 276 58.9 17.0 135.6X -SQL Parquet MR 2355 / 2363 6.7 149.7 15.4X -SQL ORC Vectorized 543 / 546 29.0 34.5 66.6X -SQL ORC MR 2246 / 2258 7.0 142.8 16.1X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 353 / 367 44.6 22.4 1.0X -ParquetReader Vectorized -> Row 351 / 357 44.7 22.3 1.0X +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 23939 24126 265 0.7 1522.0 1.0X +SQL Json 8908 9008 142 1.8 566.4 2.7X +SQL Parquet Vectorized 192 229 36 82.1 12.2 125.0X +SQL Parquet MR 2356 2363 10 6.7 149.8 10.2X +SQL ORC Vectorized 329 347 25 47.9 20.9 72.9X +SQL ORC MR 1711 1747 50 9.2 108.8 14.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 194 197 4 81.1 12.3 1.0X +ParquetReader Vectorized -> Row 97 102 13 162.3 6.2 2.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 24603 24607 6 0.6 1564.2 1.0X +SQL Json 9587 9652 92 1.6 609.5 2.6X +SQL Parquet Vectorized 227 241 13 69.4 14.4 108.6X +SQL Parquet MR 2432 2441 12 6.5 154.6 10.1X +SQL ORC Vectorized 320 327 8 49.2 20.3 76.9X +SQL ORC MR 1889 1921 46 8.3 120.1 13.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 290 294 8 54.3 18.4 1.0X +ParquetReader Vectorized -> Row 252 256 5 62.4 16.0 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 26742 26743 1 0.6 1700.2 1.0X +SQL Json 10855 10855 0 1.4 690.1 2.5X +SQL Parquet Vectorized 195 202 7 80.8 12.4 137.3X +SQL Parquet MR 2805 2806 0 5.6 178.4 9.5X +SQL ORC Vectorized 376 383 5 41.8 23.9 71.1X +SQL ORC MR 2021 2092 102 7.8 128.5 13.2X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 248 253 5 63.4 15.8 1.0X +ParquetReader Vectorized -> Row 249 251 2 63.1 15.9 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 34841 34855 20 0.5 2215.1 1.0X +SQL Json 14121 14133 18 1.1 897.8 2.5X +SQL Parquet Vectorized 288 303 17 54.7 18.3 121.2X +SQL Parquet MR 3178 3197 27 4.9 202.0 11.0X +SQL ORC Vectorized 465 476 8 33.8 29.6 74.9X +SQL ORC MR 2255 2260 6 7.0 143.4 15.4X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 344 354 11 45.8 21.8 1.0X +ParquetReader Vectorized -> Row 383 385 3 41.1 24.3 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 29336 29563 322 0.5 1865.1 1.0X +SQL Json 13452 13544 130 1.2 855.3 2.2X +SQL Parquet Vectorized 186 200 22 84.8 11.8 158.1X +SQL Parquet MR 2752 2815 90 5.7 175.0 10.7X +SQL ORC Vectorized 460 465 6 34.2 29.3 63.7X +SQL ORC MR 2054 2072 26 7.7 130.6 14.3X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 244 246 4 64.6 15.5 1.0X +ParquetReader Vectorized -> Row 247 250 4 63.7 15.7 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 37812 37897 120 0.4 2404.0 1.0X +SQL Json 19499 19509 15 0.8 1239.7 1.9X +SQL Parquet Vectorized 284 292 10 55.4 18.1 133.2X +SQL Parquet MR 3236 3248 17 4.9 205.7 11.7X +SQL ORC Vectorized 542 558 18 29.0 34.4 69.8X +SQL ORC MR 2273 2298 36 6.9 144.5 16.6X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet Reader Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +ParquetReader Vectorized 342 352 13 46.0 21.7 1.0X +ParquetReader Vectorized -> Row 341 344 3 46.1 21.7 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 21130 / 21246 0.5 2015.1 1.0X -SQL Json 12145 / 12174 0.9 1158.2 1.7X -SQL Parquet Vectorized 2363 / 2377 4.4 225.3 8.9X -SQL Parquet MR 4555 / 4557 2.3 434.4 4.6X -SQL ORC Vectorized 2361 / 2388 4.4 225.1 9.0X -SQL ORC MR 4186 / 4209 2.5 399.2 5.0X +Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 26777 26806 41 0.4 2553.7 1.0X +SQL Json 13894 14071 251 0.8 1325.0 1.9X +SQL Parquet Vectorized 2351 2404 75 4.5 224.2 11.4X +SQL Parquet MR 5198 5219 29 2.0 495.8 5.2X +SQL ORC Vectorized 2434 2435 1 4.3 232.1 11.0X +SQL ORC MR 4281 4345 91 2.4 408.3 6.3X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 11693 / 11729 0.9 1115.1 1.0X -SQL Json 7025 / 7025 1.5 669.9 1.7X -SQL Parquet Vectorized 803 / 821 13.1 76.6 14.6X -SQL Parquet MR 1776 / 1790 5.9 169.4 6.6X -SQL ORC Vectorized 491 / 494 21.4 46.8 23.8X -SQL ORC MR 2050 / 2063 5.1 195.5 5.7X +Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 15779 16507 1029 0.7 1504.8 1.0X +SQL Json 7866 7877 14 1.3 750.2 2.0X +SQL Parquet Vectorized 820 826 5 12.8 78.2 19.2X +SQL Parquet MR 2646 2658 17 4.0 252.4 6.0X +SQL ORC Vectorized 638 644 7 16.4 60.9 24.7X +SQL ORC MR 2205 2222 25 4.8 210.3 7.2X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Data column - CSV 30965 / 31041 0.5 1968.7 1.0X -Data column - Json 12876 / 12882 1.2 818.6 2.4X -Data column - Parquet Vectorized 277 / 282 56.7 17.6 111.6X -Data column - Parquet MR 3398 / 3402 4.6 216.0 9.1X -Data column - ORC Vectorized 399 / 407 39.4 25.4 77.5X -Data column - ORC MR 2583 / 2589 6.1 164.2 12.0X -Partition column - CSV 7403 / 7427 2.1 470.7 4.2X -Partition column - Json 5587 / 5625 2.8 355.2 5.5X -Partition column - Parquet Vectorized 71 / 78 222.6 4.5 438.3X -Partition column - Parquet MR 1798 / 1808 8.7 114.3 17.2X -Partition column - ORC Vectorized 72 / 75 219.0 4.6 431.2X -Partition column - ORC MR 1772 / 1778 8.9 112.6 17.5X -Both columns - CSV 30211 / 30212 0.5 1920.7 1.0X -Both columns - Json 13382 / 13391 1.2 850.8 2.3X -Both columns - Parquet Vectorized 321 / 333 49.0 20.4 96.4X -Both columns - Parquet MR 3656 / 3661 4.3 232.4 8.5X -Both columns - ORC Vectorized 443 / 448 35.5 28.2 69.9X -Both columns - ORC MR 2626 / 2633 6.0 167.0 11.8X +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Data column - CSV 38142 38183 58 0.4 2425.0 1.0X +Data column - Json 14664 14667 4 1.1 932.3 2.6X +Data column - Parquet Vectorized 304 318 13 51.8 19.3 125.7X +Data column - Parquet MR 3378 3384 8 4.7 214.8 11.3X +Data column - ORC Vectorized 475 481 7 33.1 30.2 80.3X +Data column - ORC MR 2324 2356 46 6.8 147.7 16.4X +Partition column - CSV 14680 14742 88 1.1 933.3 2.6X +Partition column - Json 11200 11251 73 1.4 712.1 3.4X +Partition column - Parquet Vectorized 102 111 14 154.7 6.5 375.1X +Partition column - Parquet MR 1477 1483 9 10.7 93.9 25.8X +Partition column - ORC Vectorized 100 112 18 157.4 6.4 381.6X +Partition column - ORC MR 1675 1685 15 9.4 106.5 22.8X +Both columns - CSV 41925 41929 6 0.4 2665.5 0.9X +Both columns - Json 15409 15422 18 1.0 979.7 2.5X +Both columns - Parquet Vectorized 351 358 10 44.8 22.3 108.7X +Both columns - Parquet MR 3719 3720 2 4.2 236.4 10.3X +Both columns - ORC Vectorized 609 630 23 25.8 38.7 62.6X +Both columns - ORC MR 2959 2959 1 5.3 188.1 12.9X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13918 / 13979 0.8 1327.3 1.0X -SQL Json 10068 / 10068 1.0 960.1 1.4X -SQL Parquet Vectorized 1563 / 1564 6.7 149.0 8.9X -SQL Parquet MR 3835 / 3836 2.7 365.8 3.6X -ParquetReader Vectorized 1115 / 1118 9.4 106.4 12.5X -SQL ORC Vectorized 1172 / 1208 8.9 111.8 11.9X -SQL ORC MR 3708 / 3711 2.8 353.6 3.8X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13972 / 14043 0.8 1332.5 1.0X -SQL Json 7436 / 7469 1.4 709.1 1.9X -SQL Parquet Vectorized 1103 / 1112 9.5 105.2 12.7X -SQL Parquet MR 2841 / 2847 3.7 271.0 4.9X -ParquetReader Vectorized 992 / 1012 10.6 94.6 14.1X -SQL ORC Vectorized 1275 / 1349 8.2 121.6 11.0X -SQL ORC MR 3244 / 3259 3.2 309.3 4.3X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 11228 / 11244 0.9 1070.8 1.0X -SQL Json 5200 / 5247 2.0 495.9 2.2X -SQL Parquet Vectorized 238 / 242 44.1 22.7 47.2X -SQL Parquet MR 1730 / 1734 6.1 165.0 6.5X -ParquetReader Vectorized 237 / 238 44.3 22.6 47.4X -SQL ORC Vectorized 459 / 462 22.8 43.8 24.4X -SQL ORC MR 1767 / 1783 5.9 168.5 6.4X +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 19510 19709 282 0.5 1860.6 1.0X +SQL Json 11816 11822 8 0.9 1126.9 1.7X +SQL Parquet Vectorized 1535 1548 18 6.8 146.4 12.7X +SQL Parquet MR 5491 5514 33 1.9 523.6 3.6X +ParquetReader Vectorized 1126 1129 5 9.3 107.4 17.3X +SQL ORC Vectorized 1200 1215 21 8.7 114.5 16.3X +SQL ORC MR 3901 3904 4 2.7 372.1 5.0X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 21439 21457 26 0.5 2044.6 1.0X +SQL Json 9653 9669 22 1.1 920.6 2.2X +SQL Parquet Vectorized 1126 1131 8 9.3 107.4 19.0X +SQL Parquet MR 3947 3961 19 2.7 376.4 5.4X +ParquetReader Vectorized 998 1023 36 10.5 95.2 21.5X +SQL ORC Vectorized 1274 1277 4 8.2 121.5 16.8X +SQL ORC MR 3424 3425 1 3.1 326.5 6.3X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 17885 17893 11 0.6 1705.7 1.0X +SQL Json 5201 5210 13 2.0 496.0 3.4X +SQL Parquet Vectorized 261 267 6 40.2 24.9 68.6X +SQL Parquet MR 2841 2853 18 3.7 270.9 6.3X +ParquetReader Vectorized 244 246 3 43.1 23.2 73.4X +SQL ORC Vectorized 465 468 1 22.5 44.4 38.4X +SQL ORC MR 1904 1945 58 5.5 181.6 9.4X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 3322 / 3356 0.3 3167.9 1.0X -SQL Json 2808 / 2843 0.4 2678.2 1.2X -SQL Parquet Vectorized 56 / 63 18.9 52.9 59.8X -SQL Parquet MR 215 / 219 4.9 205.4 15.4X -SQL ORC Vectorized 64 / 76 16.4 60.9 52.0X -SQL ORC MR 314 / 316 3.3 299.6 10.6X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 50 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 7978 / 7989 0.1 7608.5 1.0X -SQL Json 10294 / 10325 0.1 9816.9 0.8X -SQL Parquet Vectorized 72 / 85 14.5 69.0 110.3X -SQL Parquet MR 237 / 241 4.4 226.4 33.6X -SQL ORC Vectorized 82 / 92 12.7 78.5 97.0X -SQL ORC MR 900 / 909 1.2 858.5 8.9X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13489 / 13508 0.1 12864.3 1.0X -SQL Json 18813 / 18827 0.1 17941.4 0.7X -SQL Parquet Vectorized 107 / 111 9.8 101.8 126.3X -SQL Parquet MR 275 / 286 3.8 262.3 49.0X -SQL ORC Vectorized 107 / 115 9.8 101.7 126.4X -SQL ORC MR 1659 / 1664 0.6 1582.3 8.1X +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 3841 3861 28 0.3 3663.1 1.0X +SQL Json 3780 3787 10 0.3 3604.6 1.0X +SQL Parquet Vectorized 83 90 10 12.7 79.0 46.4X +SQL Parquet MR 291 303 18 3.6 277.9 13.2X +SQL ORC Vectorized 93 106 20 11.3 88.8 41.2X +SQL ORC MR 217 224 10 4.8 206.6 17.7X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 50 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 8896 8971 106 0.1 8483.9 1.0X +SQL Json 14731 14773 59 0.1 14048.2 0.6X +SQL Parquet Vectorized 120 146 26 8.8 114.0 74.4X +SQL Parquet MR 330 363 33 3.2 314.4 27.0X +SQL ORC Vectorized 122 130 11 8.6 115.9 73.2X +SQL ORC MR 248 254 9 4.2 237.0 35.8X + +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SQL CSV 14771 14817 65 0.1 14086.3 1.0X +SQL Json 29677 29787 157 0.0 28302.0 0.5X +SQL Parquet Vectorized 182 191 13 5.8 173.8 81.1X +SQL Parquet MR 1209 1213 5 0.9 1153.1 12.2X +SQL ORC Vectorized 165 176 17 6.3 157.7 89.3X +SQL ORC MR 809 813 4 1.3 771.4 18.3X diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 31ad7878ee643..6c4bdfed703ff 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -1,100 +1,100 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 264 281 25 37.9 26.4 1.0X -MILLENNIUM of timestamp 1187 1198 16 8.4 118.7 0.2X -CENTURY of timestamp 1122 1124 2 8.9 112.2 0.2X -DECADE of timestamp 1041 1049 7 9.6 104.1 0.3X -YEAR of timestamp 1027 1035 6 9.7 102.7 0.3X -ISOYEAR of timestamp 1155 1171 27 8.7 115.5 0.2X -QUARTER of timestamp 1181 1192 10 8.5 118.1 0.2X -MONTH of timestamp 1023 1030 7 9.8 102.3 0.3X -WEEK of timestamp 1511 1517 5 6.6 151.1 0.2X -DAY of timestamp 1010 1016 6 9.9 101.0 0.3X -DAYOFWEEK of timestamp 1127 1129 4 8.9 112.7 0.2X -DOW of timestamp 1123 1130 6 8.9 112.3 0.2X -ISODOW of timestamp 1099 1105 6 9.1 109.9 0.2X -DOY of timestamp 1029 1030 1 9.7 102.9 0.3X -HOUR of timestamp 415 417 1 24.1 41.5 0.6X -MINUTE of timestamp 409 418 13 24.4 40.9 0.6X -SECOND of timestamp 408 413 8 24.5 40.8 0.6X -MILLISECONDS of timestamp 28956 29040 73 0.3 2895.6 0.0X -MICROSECONDS of timestamp 504 519 13 19.8 50.4 0.5X -EPOCH of timestamp 23543 23566 28 0.4 2354.3 0.0X +cast to timestamp 397 428 45 25.2 39.7 1.0X +MILLENNIUM of timestamp 1480 1546 67 6.8 148.0 0.3X +CENTURY of timestamp 1368 1384 17 7.3 136.8 0.3X +DECADE of timestamp 1281 1344 57 7.8 128.1 0.3X +YEAR of timestamp 1238 1244 5 8.1 123.8 0.3X +ISOYEAR of timestamp 1379 1455 122 7.2 137.9 0.3X +QUARTER of timestamp 1442 1456 15 6.9 144.2 0.3X +MONTH of timestamp 1213 1217 3 8.2 121.3 0.3X +WEEK of timestamp 1927 1947 22 5.2 192.7 0.2X +DAY of timestamp 1306 1320 16 7.7 130.6 0.3X +DAYOFWEEK of timestamp 1394 1402 11 7.2 139.4 0.3X +DOW of timestamp 1367 1374 6 7.3 136.7 0.3X +ISODOW of timestamp 1317 1321 5 7.6 131.7 0.3X +DOY of timestamp 1223 1238 14 8.2 122.3 0.3X +HOUR of timestamp 361 362 2 27.7 36.1 1.1X +MINUTE of timestamp 354 362 10 28.3 35.4 1.1X +SECOND of timestamp 362 365 4 27.6 36.2 1.1X +MILLISECONDS of timestamp 36723 36761 63 0.3 3672.3 0.0X +MICROSECONDS of timestamp 469 490 29 21.3 46.9 0.8X +EPOCH of timestamp 30137 30181 38 0.3 3013.7 0.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 853 856 6 11.7 85.3 1.0X -MILLENNIUM of date 1081 1088 6 9.2 108.1 0.8X -CENTURY of date 1087 1090 4 9.2 108.7 0.8X -DECADE of date 1018 1019 1 9.8 101.8 0.8X -YEAR of date 996 1006 9 10.0 99.6 0.9X -ISOYEAR of date 1133 1147 21 8.8 113.3 0.8X -QUARTER of date 1246 1254 10 8.0 124.6 0.7X -MONTH of date 998 1002 4 10.0 99.8 0.9X -WEEK of date 1483 1490 7 6.7 148.3 0.6X -DAY of date 992 998 5 10.1 99.2 0.9X -DAYOFWEEK of date 1121 1128 7 8.9 112.1 0.8X -DOW of date 1118 1126 8 8.9 111.8 0.8X -ISODOW of date 1093 1103 9 9.1 109.3 0.8X -DOY of date 1026 1032 5 9.7 102.6 0.8X -HOUR of date 1707 1726 24 5.9 170.7 0.5X -MINUTE of date 1710 1731 19 5.8 171.0 0.5X -SECOND of date 1701 1720 19 5.9 170.1 0.5X -MILLISECONDS of date 2256 2272 19 4.4 225.6 0.4X -MICROSECONDS of date 1801 1810 11 5.6 180.1 0.5X -EPOCH of date 24848 24860 17 0.4 2484.8 0.0X +cast to date 1010 1022 11 9.9 101.0 1.0X +MILLENNIUM of date 1300 1311 18 7.7 130.0 0.8X +CENTURY of date 1304 1306 2 7.7 130.4 0.8X +DECADE of date 1199 1205 10 8.3 119.9 0.8X +YEAR of date 1191 1194 4 8.4 119.1 0.8X +ISOYEAR of date 1451 1456 9 6.9 145.1 0.7X +QUARTER of date 1494 1501 10 6.7 149.4 0.7X +MONTH of date 1189 1191 3 8.4 118.9 0.8X +WEEK of date 1893 1958 111 5.3 189.3 0.5X +DAY of date 1282 1285 3 7.8 128.2 0.8X +DAYOFWEEK of date 1374 1386 17 7.3 137.4 0.7X +DOW of date 1348 1351 3 7.4 134.8 0.7X +ISODOW of date 1292 1297 5 7.7 129.2 0.8X +DOY of date 1213 1216 3 8.2 121.3 0.8X +HOUR of date 1450 1458 9 6.9 145.0 0.7X +MINUTE of date 1445 1452 9 6.9 144.5 0.7X +SECOND of date 1448 1458 8 6.9 144.8 0.7X +MILLISECONDS of date 2094 2103 11 4.8 209.4 0.5X +MICROSECONDS of date 1562 1573 19 6.4 156.2 0.6X +EPOCH of date 31000 31047 68 0.3 3100.0 0.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 227 253 39 44.0 22.7 1.0X -MILLENNIUM of timestamp 1121 1127 9 8.9 112.1 0.2X -CENTURY of timestamp 1084 1094 17 9.2 108.4 0.2X -DECADE of timestamp 1015 1037 35 9.8 101.5 0.2X -YEAR of timestamp 1011 1013 4 9.9 101.1 0.2X -ISOYEAR of timestamp 1121 1126 7 8.9 112.1 0.2X -QUARTER of timestamp 1243 1247 4 8.0 124.3 0.2X -MONTH of timestamp 1000 1009 14 10.0 100.0 0.2X -WEEK of timestamp 1481 1492 11 6.8 148.1 0.2X -DAY of timestamp 999 1000 1 10.0 99.9 0.2X -DAYOFWEEK of timestamp 1114 1132 20 9.0 111.4 0.2X -DOW of timestamp 1135 1147 17 8.8 113.5 0.2X -ISODOW of timestamp 1070 1079 10 9.3 107.0 0.2X -DOY of timestamp 1018 1027 8 9.8 101.8 0.2X -HOUR of timestamp 413 419 9 24.2 41.3 0.6X -MINUTE of timestamp 411 414 3 24.3 41.1 0.6X -SECOND of timestamp 410 413 3 24.4 41.0 0.6X -MILLISECONDS of timestamp 29225 29264 52 0.3 2922.5 0.0X -MICROSECONDS of timestamp 507 512 7 19.7 50.7 0.4X -EPOCH of timestamp 23565 23608 56 0.4 2356.5 0.0X +cast to timestamp 327 333 10 30.6 32.7 1.0X +MILLENNIUM of timestamp 1292 1296 4 7.7 129.2 0.3X +CENTURY of timestamp 1301 1305 6 7.7 130.1 0.3X +DECADE of timestamp 1200 1204 6 8.3 120.0 0.3X +YEAR of timestamp 1185 1193 8 8.4 118.5 0.3X +ISOYEAR of timestamp 1449 1469 18 6.9 144.9 0.2X +QUARTER of timestamp 1497 1505 7 6.7 149.7 0.2X +MONTH of timestamp 1185 1188 3 8.4 118.5 0.3X +WEEK of timestamp 1901 1909 7 5.3 190.1 0.2X +DAY of timestamp 1278 1282 4 7.8 127.8 0.3X +DAYOFWEEK of timestamp 1371 1376 5 7.3 137.1 0.2X +DOW of timestamp 1361 1372 17 7.3 136.1 0.2X +ISODOW of timestamp 1299 1306 9 7.7 129.9 0.3X +DOY of timestamp 1216 1219 4 8.2 121.6 0.3X +HOUR of timestamp 352 356 5 28.4 35.2 0.9X +MINUTE of timestamp 350 369 17 28.6 35.0 0.9X +SECOND of timestamp 351 364 19 28.5 35.1 0.9X +MILLISECONDS of timestamp 36989 37022 52 0.3 3698.9 0.0X +MICROSECONDS of timestamp 473 476 2 21.1 47.3 0.7X +EPOCH of timestamp 29890 29908 27 0.3 2989.0 0.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 848 852 3 11.8 84.8 1.0X -MILLENNIUM of date 1083 1086 3 9.2 108.3 0.8X -CENTURY of date 1081 1084 5 9.3 108.1 0.8X -DECADE of date 1010 1014 4 9.9 101.0 0.8X -YEAR of date 992 1000 9 10.1 99.2 0.9X -ISOYEAR of date 1111 1116 6 9.0 111.1 0.8X -QUARTER of date 1230 1249 17 8.1 123.0 0.7X -MONTH of date 996 1008 11 10.0 99.6 0.9X -WEEK of date 1487 1516 28 6.7 148.7 0.6X -DAY of date 993 999 6 10.1 99.3 0.9X -DAYOFWEEK of date 1113 1118 5 9.0 111.3 0.8X -DOW of date 1113 1116 2 9.0 111.3 0.8X -ISODOW of date 1069 1072 3 9.4 106.9 0.8X -DOY of date 1027 1028 1 9.7 102.7 0.8X -HOUR of date 1707 1710 3 5.9 170.7 0.5X -MINUTE of date 1704 1705 2 5.9 170.4 0.5X -SECOND of date 1701 1705 4 5.9 170.1 0.5X -MILLISECONDS of date 2229 2238 9 4.5 222.9 0.4X -MICROSECONDS of date 1801 1808 12 5.6 180.1 0.5X -EPOCH of date 24783 24817 31 0.4 2478.3 0.0X +cast to date 1005 1006 1 9.9 100.5 1.0X +MILLENNIUM of date 1295 1300 5 7.7 129.5 0.8X +CENTURY of date 1297 1298 1 7.7 129.7 0.8X +DECADE of date 1198 1208 13 8.3 119.8 0.8X +YEAR of date 1184 1193 13 8.4 118.4 0.8X +ISOYEAR of date 1445 1460 13 6.9 144.5 0.7X +QUARTER of date 1495 1500 4 6.7 149.5 0.7X +MONTH of date 1176 1179 3 8.5 117.6 0.9X +WEEK of date 1893 1904 15 5.3 189.3 0.5X +DAY of date 1275 1283 8 7.8 127.5 0.8X +DAYOFWEEK of date 1369 1373 4 7.3 136.9 0.7X +DOW of date 1353 1354 2 7.4 135.3 0.7X +ISODOW of date 1290 1290 1 7.8 129.0 0.8X +DOY of date 1208 1212 4 8.3 120.8 0.8X +HOUR of date 1446 1449 2 6.9 144.6 0.7X +MINUTE of date 1441 1442 1 6.9 144.1 0.7X +SECOND of date 1443 1450 8 6.9 144.3 0.7X +MILLISECONDS of date 2087 2089 3 4.8 208.7 0.5X +MICROSECONDS of date 1557 1570 21 6.4 155.7 0.6X +EPOCH of date 30980 31001 32 0.3 3098.0 0.0X diff --git a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt index b06b5c092b61a..5fa461fecd4d2 100644 --- a/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt +++ b/sql/core/benchmarks/PrimitiveArrayBenchmark-results.txt @@ -2,12 +2,11 @@ Write primitive arrays in dataset ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.13.6 -Intel(R) Core(TM) i7-6820HQ CPU @ 2.70GHz - -Write an array in Dataset: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Int 437 / 529 19.2 52.1 1.0X -Double 638 / 670 13.1 76.1 0.7X +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Write an array in Dataset: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Int 543 605 67 15.4 64.7 1.0X +Double 737 776 36 11.4 87.9 0.7X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala index bd2470ee20660..df122977fe5f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala @@ -22,11 +22,10 @@ import scala.collection.JavaConverters._ import scala.util.Random import org.apache.spark.SparkConf -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.execution.datasources.parquet.{SpecificParquetRecordReaderBase, VectorizedParquetRecordReader} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -44,21 +43,26 @@ import org.apache.spark.sql.vectorized.ColumnVector * Results will be written to "benchmarks/DataSourceReadBenchmark-results.txt". * }}} */ -object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { - val conf = new SparkConf() - .setAppName("DataSourceReadBenchmark") - // Since `spark.master` always exists, overrides this value - .set("spark.master", "local[1]") - .setIfMissing("spark.driver.memory", "3g") - .setIfMissing("spark.executor.memory", "3g") - .setIfMissing(UI_ENABLED, false) - - val spark = SparkSession.builder.config(conf).getOrCreate() - - // Set default configs. Individual cases will change them if necessary. - spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") - spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") +object DataSourceReadBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + .setAppName("DataSourceReadBenchmark") + // Since `spark.master` always exists, overrides this value + .set("spark.master", "local[1]") + .setIfMissing("spark.driver.memory", "3g") + .setIfMissing("spark.executor.memory", "3g") + .setIfMissing(UI_ENABLED, false) + + val sparkSession = SparkSession.builder.config(conf).getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + sparkSession.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + + sparkSession + } def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index b9086aa5cb378..a109b11b2d6d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -19,9 +19,7 @@ package org.apache.spark.sql.execution.benchmark import java.time.Instant -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.internal.SQLConf /** @@ -36,11 +34,7 @@ import org.apache.spark.sql.internal.SQLConf * Results will be written to "benchmarks/ExtractBenchmark-results.txt". * }}} */ -object ExtractBenchmark extends BenchmarkBase with SQLHelper { - private val spark: SparkSession = SparkSession.builder() - .master("local[1]") - .appName(this.getClass.getCanonicalName) - .getOrCreate() +object ExtractBenchmark extends SqlBasedBenchmark { private def doBenchmark(cardinality: Long, exprs: String*): Unit = { val sinceSecond = Instant.parse("2010-01-01T00:00:00Z").getEpochSecond diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index b040243717137..50ba50176c7f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -22,10 +22,9 @@ import java.io.File import scala.util.Random import org.apache.spark.SparkConf -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.functions.monotonically_increasing_id import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType @@ -41,17 +40,21 @@ import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType * Results will be written to "benchmarks/FilterPushdownBenchmark-results.txt". * }}} */ -object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { - - private val conf = new SparkConf() - .setAppName(this.getClass.getSimpleName) - // Since `spark.master` always exists, overrides this value - .set("spark.master", "local[1]") - .setIfMissing("spark.driver.memory", "3g") - .setIfMissing("spark.executor.memory", "3g") - .setIfMissing(UI_ENABLED, false) - .setIfMissing("orc.compression", "snappy") - .setIfMissing("spark.sql.parquet.compression.codec", "snappy") +object FilterPushdownBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + .setAppName(this.getClass.getSimpleName) + // Since `spark.master` always exists, overrides this value + .set("spark.master", "local[1]") + .setIfMissing("spark.driver.memory", "3g") + .setIfMissing("spark.executor.memory", "3g") + .setIfMissing(UI_ENABLED, false) + .setIfMissing("orc.compression", "snappy") + .setIfMissing("spark.sql.parquet.compression.codec", "snappy") + + SparkSession.builder().config(conf).getOrCreate() + } private val numRows = 1024 * 1024 * 15 private val width = 5 @@ -59,8 +62,6 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { // For Parquet/ORC, we will use the same value for block size and compression size private val blockSize = org.apache.parquet.hadoop.ParquetWriter.DEFAULT_PAGE_SIZE - private val spark = SparkSession.builder().config(conf).getOrCreate() - def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala index 8b1c422e63a3f..e07921bf3aa74 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.benchmark -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.SparkSession /** @@ -28,13 +28,16 @@ import org.apache.spark.sql.SparkSession * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/PrimitiveArrayBenchmark-results.txt". */ -object PrimitiveArrayBenchmark extends BenchmarkBase { - lazy val sparkSession = SparkSession.builder - .master("local[1]") - .appName("microbenchmark") - .config("spark.sql.shuffle.partitions", 1) - .config("spark.sql.autoBroadcastJoinThreshold", 1) - .getOrCreate() +object PrimitiveArrayBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + SparkSession.builder + .master("local[1]") + .appName("microbenchmark") + .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.autoBroadcastJoinThreshold", 1) + .getOrCreate() + } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Write primitive arrays in dataset") { @@ -43,11 +46,11 @@ object PrimitiveArrayBenchmark extends BenchmarkBase { } def writeDatasetArray(iters: Int): Unit = { - import sparkSession.implicits._ + import spark.implicits._ val count = 1024 * 1024 * 2 - val sc = sparkSession.sparkContext + val sc = spark.sparkContext val primitiveIntArray = Array.fill[Int](count)(65535) val dsInt = sc.parallelize(Seq(primitiveIntArray), 1).toDS dsInt.count // force to build dataset diff --git a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt index f3044da972497..0c394a340333a 100644 --- a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt +++ b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt @@ -2,44 +2,44 @@ Hive UDAF vs Spark AF ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -hive udaf vs spark af: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -hive udaf w/o group by 6370 / 6400 0.0 97193.6 1.0X -spark af w/o group by 54 / 63 1.2 820.8 118.4X -hive udaf w/ group by 4492 / 4507 0.0 68539.5 1.4X -spark af w/ group by w/o fallback 58 / 64 1.1 881.7 110.2X -spark af w/ group by w/ fallback 136 / 142 0.5 2075.0 46.8X +hive udaf vs spark af: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +hive udaf w/o group by 6741 6759 22 0.0 102864.5 1.0X +spark af w/o group by 56 66 9 1.2 851.6 120.8X +hive udaf w/ group by 4610 4642 25 0.0 70350.3 1.5X +spark af w/ group by w/o fallback 60 67 8 1.1 916.7 112.2X +spark af w/ group by w/ fallback 135 144 9 0.5 2065.6 49.8X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - typed_count ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sort agg w/ group by 41500 / 41630 2.5 395.8 1.0X -object agg w/ group by w/o fallback 10075 / 10122 10.4 96.1 4.1X -object agg w/ group by w/ fallback 28131 / 28205 3.7 268.3 1.5X -sort agg w/o group by 6182 / 6221 17.0 59.0 6.7X -object agg w/o group by w/o fallback 5435 / 5468 19.3 51.8 7.6X +object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort agg w/ group by 41568 41894 461 2.5 396.4 1.0X +object agg w/ group by w/o fallback 10314 10494 149 10.2 98.4 4.0X +object agg w/ group by w/ fallback 26720 26951 326 3.9 254.8 1.6X +sort agg w/o group by 6638 6681 38 15.8 63.3 6.3X +object agg w/o group by w/o fallback 5665 5706 30 18.5 54.0 7.3X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - percentile_approx ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sort agg w/ group by 970 / 1025 2.2 462.5 1.0X -object agg w/ group by w/o fallback 772 / 798 2.7 368.1 1.3X -object agg w/ group by w/ fallback 1013 / 1044 2.1 483.1 1.0X -sort agg w/o group by 751 / 781 2.8 358.0 1.3X -object agg w/o group by w/o fallback 772 / 814 2.7 368.0 1.3X +object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort agg w/ group by 794 862 33 2.6 378.8 1.0X +object agg w/ group by w/o fallback 605 622 10 3.5 288.5 1.3X +object agg w/ group by w/ fallback 840 860 15 2.5 400.5 0.9X +sort agg w/o group by 555 570 12 3.8 264.6 1.4X +object agg w/o group by w/o fallback 544 562 12 3.9 259.6 1.5X diff --git a/sql/hive/benchmarks/OrcReadBenchmark-results.txt b/sql/hive/benchmarks/OrcReadBenchmark-results.txt index caa78b9a8f102..c47cf27bf617a 100644 --- a/sql/hive/benchmarks/OrcReadBenchmark-results.txt +++ b/sql/hive/benchmarks/OrcReadBenchmark-results.txt @@ -2,155 +2,155 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1725 / 1759 9.1 109.7 1.0X -Native ORC Vectorized 272 / 316 57.8 17.3 6.3X -Hive built-in ORC 1970 / 1987 8.0 125.3 0.9X +SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1843 1958 162 8.5 117.2 1.0X +Native ORC Vectorized 321 355 31 48.9 20.4 5.7X +Hive built-in ORC 2143 2175 44 7.3 136.3 0.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1633 / 1672 9.6 103.8 1.0X -Native ORC Vectorized 238 / 255 66.0 15.1 6.9X -Hive built-in ORC 2293 / 2305 6.9 145.8 0.7X +SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1987 2020 47 7.9 126.3 1.0X +Native ORC Vectorized 276 299 25 57.0 17.6 7.2X +Hive built-in ORC 2350 2357 10 6.7 149.4 0.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1677 / 1699 9.4 106.6 1.0X -Native ORC Vectorized 325 / 342 48.3 20.7 5.2X -Hive built-in ORC 2561 / 2569 6.1 162.8 0.7X +SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2092 2115 32 7.5 133.0 1.0X +Native ORC Vectorized 360 373 18 43.6 22.9 5.8X +Hive built-in ORC 2550 2557 9 6.2 162.2 0.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1791 / 1795 8.8 113.9 1.0X -Native ORC Vectorized 400 / 408 39.3 25.4 4.5X -Hive built-in ORC 2713 / 2720 5.8 172.5 0.7X +SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2173 2188 21 7.2 138.2 1.0X +Native ORC Vectorized 435 448 14 36.2 27.7 5.0X +Hive built-in ORC 2683 2690 10 5.9 170.6 0.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1791 / 1805 8.8 113.8 1.0X -Native ORC Vectorized 433 / 438 36.3 27.5 4.1X -Hive built-in ORC 2690 / 2803 5.8 171.0 0.7X +SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2233 2323 127 7.0 142.0 1.0X +Native ORC Vectorized 475 483 13 33.1 30.2 4.7X +Hive built-in ORC 2605 2610 6 6.0 165.7 0.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1911 / 1930 8.2 121.5 1.0X -Native ORC Vectorized 543 / 552 29.0 34.5 3.5X -Hive built-in ORC 2967 / 3065 5.3 188.6 0.6X +SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2367 2384 24 6.6 150.5 1.0X +Native ORC Vectorized 600 641 69 26.2 38.1 3.9X +Hive built-in ORC 2860 2877 24 5.5 181.9 0.8X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 4160 / 4188 2.5 396.7 1.0X -Native ORC Vectorized 2405 / 2406 4.4 229.4 1.7X -Hive built-in ORC 5514 / 5562 1.9 525.9 0.8X +Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 4253 4330 108 2.5 405.6 1.0X +Native ORC Vectorized 2295 2301 8 4.6 218.9 1.9X +Hive built-in ORC 5364 5465 144 2.0 511.5 0.8X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Data column - Native ORC MR 1863 / 1867 8.4 118.4 1.0X -Data column - Native ORC Vectorized 411 / 418 38.2 26.2 4.5X -Data column - Hive built-in ORC 3297 / 3308 4.8 209.6 0.6X -Partition column - Native ORC MR 1505 / 1506 10.4 95.7 1.2X -Partition column - Native ORC Vectorized 80 / 93 195.6 5.1 23.2X -Partition column - Hive built-in ORC 1960 / 1979 8.0 124.6 1.0X -Both columns - Native ORC MR 2076 / 2090 7.6 132.0 0.9X -Both columns - Native ORC Vectorized 450 / 463 34.9 28.6 4.1X -Both columns - Hive built-in ORC 3528 / 3548 4.5 224.3 0.5X +Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Data column - Native ORC MR 2443 2448 6 6.4 155.3 1.0X +Data column - Native ORC Vectorized 446 473 44 35.3 28.3 5.5X +Data column - Hive built-in ORC 2868 2877 12 5.5 182.4 0.9X +Partition column - Native ORC MR 1623 1656 47 9.7 103.2 1.5X +Partition column - Native ORC Vectorized 112 121 14 140.8 7.1 21.9X +Partition column - Hive built-in ORC 1846 1850 5 8.5 117.4 1.3X +Both columns - Native ORC MR 2610 2635 36 6.0 165.9 0.9X +Both columns - Native ORC Vectorized 492 508 19 32.0 31.3 5.0X +Both columns - Hive built-in ORC 2969 2973 4 5.3 188.8 0.8X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1727 / 1733 6.1 164.7 1.0X -Native ORC Vectorized 375 / 379 28.0 35.7 4.6X -Hive built-in ORC 2665 / 2666 3.9 254.2 0.6X +Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2056 2064 11 5.1 196.1 1.0X +Native ORC Vectorized 415 421 7 25.3 39.6 5.0X +Hive built-in ORC 2710 2722 17 3.9 258.4 0.8X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 3324 / 3325 3.2 317.0 1.0X -Native ORC Vectorized 1085 / 1106 9.7 103.4 3.1X -Hive built-in ORC 5272 / 5299 2.0 502.8 0.6X +String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 3655 3674 27 2.9 348.6 1.0X +Native ORC Vectorized 1166 1167 1 9.0 111.2 3.1X +Hive built-in ORC 5268 5305 52 2.0 502.4 0.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 3045 / 3046 3.4 290.4 1.0X -Native ORC Vectorized 1248 / 1260 8.4 119.0 2.4X -Hive built-in ORC 3989 / 3999 2.6 380.4 0.8X +String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 3447 3467 27 3.0 328.8 1.0X +Native ORC Vectorized 1222 1223 1 8.6 116.6 2.8X +Hive built-in ORC 3947 3959 18 2.7 376.4 0.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1692 / 1694 6.2 161.3 1.0X -Native ORC Vectorized 471 / 493 22.3 44.9 3.6X -Hive built-in ORC 2398 / 2411 4.4 228.7 0.7X +String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1912 1917 6 5.5 182.4 1.0X +Native ORC Vectorized 477 484 5 22.0 45.5 4.0X +Hive built-in ORC 2374 2386 17 4.4 226.4 0.8X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 1371 / 1379 0.8 1307.5 1.0X -Native ORC Vectorized 121 / 135 8.6 115.8 11.3X -Hive built-in ORC 521 / 561 2.0 497.1 2.6X +Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 290 350 102 3.6 276.1 1.0X +Native ORC Vectorized 155 166 15 6.7 148.2 1.9X +Hive built-in ORC 520 531 8 2.0 495.8 0.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 2711 / 2767 0.4 2585.5 1.0X -Native ORC Vectorized 210 / 232 5.0 200.5 12.9X -Hive built-in ORC 764 / 775 1.4 728.3 3.5X +Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 365 406 73 2.9 347.9 1.0X +Native ORC Vectorized 232 246 20 4.5 221.6 1.6X +Hive built-in ORC 794 864 62 1.3 757.6 0.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Native ORC MR 3979 / 3988 0.3 3794.4 1.0X -Native ORC Vectorized 357 / 366 2.9 340.2 11.2X -Hive built-in ORC 1091 / 1095 1.0 1040.5 3.6X +Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 501 544 40 2.1 477.6 1.0X +Native ORC Vectorized 365 386 33 2.9 348.0 1.4X +Hive built-in ORC 1153 1153 0 0.9 1099.8 0.4X diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala index 3226e3a5f318a..c475c7b21ab95 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala @@ -21,11 +21,10 @@ import scala.concurrent.duration._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentileApprox -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile -import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.hive.execution.TestingTypedCount import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.internal.SQLConf @@ -44,9 +43,10 @@ import org.apache.spark.sql.types.LongType * Results will be written to "benchmarks/ObjectHashAggregateExecBenchmark-results.txt". * }}} */ -object ObjectHashAggregateExecBenchmark extends BenchmarkBase with SQLHelper { +object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = TestHive.sparkSession - private val spark: SparkSession = TestHive.sparkSession private val sql = spark.sql _ import spark.implicits._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index c03ae144a1595..f28b9be60d3cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -22,9 +22,9 @@ import java.io.File import scala.util.Random import org.apache.spark.SparkConf -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -43,18 +43,23 @@ import org.apache.spark.sql.types._ * This is in `sql/hive` module in order to compare `sql/core` and `sql/hive` ORC data sources. */ // scalastyle:off line.size.limit -object OrcReadBenchmark extends BenchmarkBase with SQLHelper { - val conf = new SparkConf() - conf.set("orc.compression", "snappy") - - private val spark = SparkSession.builder() - .master("local[1]") - .appName("OrcReadBenchmark") - .config(conf) - .getOrCreate() - - // Set default configs. Individual cases will change them if necessary. - spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") +object OrcReadBenchmark extends SqlBasedBenchmark { + + override def getSparkSession: SparkSession = { + val conf = new SparkConf() + conf.set("orc.compression", "snappy") + + val sparkSession = SparkSession.builder() + .master("local[1]") + .appName("OrcReadBenchmark") + .config(conf) + .getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + sparkSession.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") + + sparkSession + } def withTempTable(tableNames: String*)(f: => Unit): Unit = { try f finally tableNames.foreach(spark.catalog.dropTempView) From b917a6593dc969b9b766259eb8cbbd6e90e0dc53 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 18 Sep 2019 22:30:28 -0700 Subject: [PATCH 0118/1387] [SPARK-28989][SQL] Add a SQLConf `spark.sql.ansi.enabled` ### What changes were proposed in this pull request? Currently, there are new configurations for compatibility with ANSI SQL: * `spark.sql.parser.ansi.enabled` * `spark.sql.decimalOperations.nullOnOverflow` * `spark.sql.failOnIntegralTypeOverflow` This PR is to add new configuration `spark.sql.ansi.enabled` and remove the 3 options above. When the configuration is true, Spark tries to conform to the ANSI SQL specification. It will be disabled by default. ### Why are the changes needed? Make it simple and straightforward. ### Does this PR introduce any user-facing change? The new features for ANSI compatibility will be set via one configuration `spark.sql.ansi.enabled`. ### How was this patch tested? Existing unit tests. Closes #25693 from gengliangwang/ansiEnabled. Lead-authored-by: Gengliang Wang Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- docs/sql-keywords.md | 8 ++-- .../sql/catalyst/CatalystTypeConverters.scala | 2 +- .../sql/catalyst/SerializerBuildHelper.scala | 2 +- .../catalyst/analysis/DecimalPrecision.scala | 2 +- .../sql/catalyst/encoders/RowEncoder.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 8 ++-- .../catalyst/expressions/aggregate/Sum.scala | 2 +- .../sql/catalyst/expressions/arithmetic.scala | 4 +- .../expressions/decimalExpressions.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../sql/catalyst/parser/ParseDriver.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 41 +++++-------------- .../encoders/ExpressionEncoderSuite.scala | 8 ++-- .../catalyst/encoders/RowEncoderSuite.scala | 4 +- .../ArithmeticExpressionSuite.scala | 24 +++++------ .../sql/catalyst/expressions/CastSuite.scala | 12 +++--- .../expressions/DecimalExpressionSuite.scala | 4 +- .../catalyst/expressions/ScalaUDFSuite.scala | 4 +- .../parser/ExpressionParserSuite.scala | 10 ++--- .../parser/TableIdentifierParserSuite.scala | 2 +- .../sql-tests/inputs/ansi/interval.sql | 4 +- .../inputs/decimalArithmeticOperations.sql | 2 +- .../resources/sql-tests/inputs/pgSQL/text.sql | 6 +-- .../sql-tests/results/ansi/interval.sql.out | 8 ++-- .../decimalArithmeticOperations.sql.out | 4 +- .../sql-tests/results/pgSQL/text.sql.out | 8 ++-- .../org/apache/spark/sql/DataFrameSuite.scala | 6 +-- .../apache/spark/sql/SQLQueryTestSuite.scala | 8 ++-- .../ThriftServerQueryTestSuite.scala | 2 +- 29 files changed, 86 insertions(+), 109 deletions(-) diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 16ce35d55ed7b..c55e589aa8c3f 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,15 +19,15 @@ license: | limitations under the License. --- -When `spark.sql.parser.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.ansi.enabled` is true, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.parser.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.parser.ansi.enabled=true`. +When `spark.sql.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when `spark.sql.ansi.enabled=true`. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. -By default `spark.sql.parser.ansi.enabled` is false. +By default `spark.sql.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 488252aa0c7b5..34d2f45e715e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -344,7 +344,7 @@ object CatalystTypeConverters { private class DecimalConverter(dataType: DecimalType) extends CatalystTypeConverter[Any, JavaBigDecimal, Decimal] { - private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + private val nullOnOverflow = !SQLConf.get.ansiEnabled override def toCatalystImpl(scalaValue: Any): Decimal = { val decimal = scalaValue match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala index 75c278e781140..026ff6f2983fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala @@ -26,7 +26,7 @@ import org.apache.spark.unsafe.types.UTF8String object SerializerBuildHelper { - private def nullOnOverflow: Boolean = SQLConf.get.decimalOperationsNullOnOverflow + private def nullOnOverflow: Boolean = !SQLConf.get.ansiEnabled def createSerializerForBoolean(inputObject: Expression): Expression = { Invoke(inputObject, "booleanValue", BooleanType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala index 856c2ed828002..a64befecb68d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala @@ -82,7 +82,7 @@ object DecimalPrecision extends TypeCoercionRule { PromotePrecision(Cast(e, dataType)) } - private def nullOnOverflow: Boolean = SQLConf.get.decimalOperationsNullOnOverflow + private def nullOnOverflow: Boolean = !SQLConf.get.ansiEnabled override protected def coerceTypes(plan: LogicalPlan): LogicalPlan = plan resolveOperators { // fix decimal precision for expressions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index afe8a23f8f150..765018f07d87a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -114,7 +114,7 @@ object RowEncoder { d, "fromDecimal", inputObject :: Nil, - returnNullable = false), d, SQLConf.get.decimalOperationsNullOnOverflow) + returnNullable = false), d, !SQLConf.get.ansiEnabled) case StringType => createSerializerForString(inputObject) 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 d1943f02f85e5..5001b4350d778 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 @@ -289,7 +289,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private lazy val dateFormatter = DateFormatter() private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) - private val failOnIntegralTypeOverflow = SQLConf.get.failOnIntegralTypeOverflow + private val failOnIntegralTypeOverflow = SQLConf.get.ansiEnabled // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { @@ -600,13 +600,13 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } - private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + private val nullOnOverflow = !SQLConf.get.ansiEnabled /** * Change the precision / scale in a given decimal to those set in `decimalType` (if any), * modifying `value` in-place and returning it if successful. If an overflow occurs, it * either returns null or throws an exception according to the value set for - * `spark.sql.decimalOperations.nullOnOverflow`. + * `spark.sql.ansi.enabled`. * * NOTE: this modifies `value` in-place, so don't call it on external data. */ @@ -625,7 +625,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String /** * Create new `Decimal` with precision and scale given in `decimalType` (if any). - * If overflow occurs, if `spark.sql.decimalOperations.nullOnOverflow` is true, null is returned; + * If overflow occurs, if `spark.sql.ansi.enabled` is false, null is returned; * otherwise, an `ArithmeticException` is thrown. */ private[this] def toPrecision(value: Decimal, decimalType: DecimalType): Decimal = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala index d04fe9249d064..c2ab8adfaef67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala @@ -91,7 +91,7 @@ case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCast } override lazy val evaluateExpression: Expression = resultType match { - case d: DecimalType => CheckOverflow(sum, d, SQLConf.get.decimalOperationsNullOnOverflow) + case d: DecimalType => CheckOverflow(sum, d, !SQLConf.get.ansiEnabled) case _ => sum } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 2ed82d99fe2bb..e4276e33acbd2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -35,7 +35,7 @@ import org.apache.spark.unsafe.types.CalendarInterval """) case class UnaryMinus(child: Expression) extends UnaryExpression with ExpectsInputTypes with NullIntolerant { - private val checkOverflow = SQLConf.get.failOnIntegralTypeOverflow + private val checkOverflow = SQLConf.get.ansiEnabled override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection.NumericAndInterval) @@ -136,7 +136,7 @@ case class Abs(child: Expression) abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { - protected val checkOverflow = SQLConf.get.failOnIntegralTypeOverflow + protected val checkOverflow = SQLConf.get.ansiEnabled override def dataType: DataType = left.dataType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala index b5b712cda8ea3..7b2489e682661 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala @@ -47,7 +47,7 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { */ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends UnaryExpression { - private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + private val nullOnOverflow = !SQLConf.get.ansiEnabled override def dataType: DataType = DecimalType(precision, scale) override def nullable: Boolean = child.nullable || nullOnOverflow diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 9335be5b239b6..c334d09b691ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1363,7 +1363,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } override def visitCurrentDatetime(ctx: CurrentDatetimeContext): Expression = withOrigin(ctx) { - if (conf.ansiParserEnabled) { + if (conf.ansiEnabled) { ctx.name.getType match { case SqlBaseParser.CURRENT_DATE => CurrentDate() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 932e795f1d0bc..85998e33140d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -92,7 +92,7 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced - lexer.ansi = SQLConf.get.ansiParserEnabled + lexer.ansi = SQLConf.get.ansiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -100,7 +100,7 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced - parser.ansi = SQLConf.get.ansiParserEnabled + parser.ansi = SQLConf.get.ansiEnabled try { try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4f3e39ad49afe..0ec661fc16c88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -411,12 +411,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val ANSI_SQL_PARSER = - buildConf("spark.sql.parser.ansi.enabled") - .doc("When true, tries to conform to ANSI SQL syntax.") - .booleanConf - .createWithDefault(false) - val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals") .internal() .doc("When true, string literals (including regex patterns) remain escaped in our SQL " + @@ -1557,16 +1551,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val DECIMAL_OPERATIONS_NULL_ON_OVERFLOW = - buildConf("spark.sql.decimalOperations.nullOnOverflow") - .internal() - .doc("When true (default), if an overflow on a decimal occurs, then NULL is returned. " + - "Spark's older versions and Hive behave in this way. If turned to false, SQL ANSI 2011 " + - "specification will be followed instead: an arithmetic exception is thrown, as most " + - "of the SQL databases do.") - .booleanConf - .createWithDefault(true) - val LITERAL_PICK_MINIMUM_PRECISION = buildConf("spark.sql.legacy.literal.pickMinimumPrecision") .internal() @@ -1723,6 +1707,14 @@ object SQLConf { .checkValues(StoreAssignmentPolicy.values.map(_.toString)) .createOptional + val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") + .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + + "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + + "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + + "the SQL parser.") + .booleanConf + .createWithDefault(false) + val SORT_BEFORE_REPARTITION = buildConf("spark.sql.execution.sortBeforeRepartition") .internal() @@ -1886,15 +1878,6 @@ object SQLConf { .booleanConf .createWithDefault(false) - val FAIL_ON_INTEGRAL_TYPE_OVERFLOW = - buildConf("spark.sql.failOnIntegralTypeOverflow") - .doc("If it is set to true, all operations on integral fields throw an " + - "exception if an overflow occurs. If it is false (default), in case of overflow a wrong " + - "result is returned.") - .internal() - .booleanConf - .createWithDefault(false) - val LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE = buildConf("spark.sql.legacy.parser.havingWithoutGroupByAsWhere") .internal() @@ -2195,8 +2178,6 @@ class SQLConf extends Serializable with Logging { def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED) - def ansiParserEnabled: Boolean = getConf(ANSI_SQL_PARSER) - def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS) def fileCompressionFactor: Double = getConf(FILE_COMPRESSION_FACTOR) @@ -2418,10 +2399,6 @@ class SQLConf extends Serializable with Logging { def decimalOperationsAllowPrecisionLoss: Boolean = getConf(DECIMAL_OPERATIONS_ALLOW_PREC_LOSS) - def decimalOperationsNullOnOverflow: Boolean = getConf(DECIMAL_OPERATIONS_NULL_ON_OVERFLOW) - - def failOnIntegralTypeOverflow: Boolean = getConf(FAIL_ON_INTEGRAL_TYPE_OVERFLOW) - def literalPickMinimumPrecision: Boolean = getConf(LITERAL_PICK_MINIMUM_PRECISION) def continuousStreamingEpochBacklogQueueSize: Int = @@ -2454,6 +2431,8 @@ class SQLConf extends Serializable with Logging { def storeAssignmentPolicy: Option[StoreAssignmentPolicy.Value] = getConf(STORE_ASSIGNMENT_POLICY).map(StoreAssignmentPolicy.withName) + def ansiEnabled: Boolean = getConf(ANSI_ENABLED) + def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) def serializerNestedSchemaPruningEnabled: Boolean = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 9380c7e3f5f72..d2acfb9686700 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -432,16 +432,16 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes } private def testOverflowingBigNumeric[T: TypeTag](bigNumeric: T, testName: String): Unit = { - Seq(true, false).foreach { allowNullOnOverflow => + Seq(true, false).foreach { ansiEnabled => testAndVerifyNotLeakingReflectionObjects( - s"overflowing $testName, allowNullOnOverflow=$allowNullOnOverflow") { + s"overflowing $testName, ansiEnabled=$ansiEnabled") { withSQLConf( - SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> allowNullOnOverflow.toString + SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString ) { // Need to construct Encoder here rather than implicitly resolving it // so that SQLConf changes are respected. val encoder = ExpressionEncoder[T]() - if (allowNullOnOverflow) { + if (!ansiEnabled) { val convertedBack = encoder.resolveAndBind().fromRow(encoder.toRow(bigNumeric)) assert(convertedBack === null) } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 5d21e4a2a83ca..1a1cab823d4f3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -169,7 +169,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } private def testDecimalOverflow(schema: StructType, row: Row): Unit = { - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { val encoder = RowEncoder(schema).resolveAndBind() intercept[Exception] { encoder.toRow(row) @@ -182,7 +182,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } } - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val encoder = RowEncoder(schema).resolveAndBind() assert(encoder.fromRow(encoder.toRow(row)).get(0) == null) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 729e2f529b97f..ad8b1a1673679 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -61,7 +61,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Add(positiveLongLit, negativeLongLit), -1L) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Add, tpe, tpe) } @@ -80,7 +80,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(UnaryMinus(Literal(Int.MinValue)), Int.MinValue) checkEvaluation(UnaryMinus(Literal(Short.MinValue)), Short.MinValue) checkEvaluation(UnaryMinus(Literal(Byte.MinValue)), Byte.MinValue) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkExceptionInExpression[ArithmeticException]( UnaryMinus(Literal(Long.MinValue)), "overflow") checkExceptionInExpression[ArithmeticException]( @@ -122,7 +122,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Subtract(positiveLongLit, negativeLongLit), positiveLong - negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Subtract, tpe, tpe) } @@ -144,7 +144,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Multiply(positiveLongLit, negativeLongLit), positiveLong * negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Multiply, tpe, tpe) } @@ -445,12 +445,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minLongLiteral, minLongLiteral) val e5 = Subtract(minLongLiteral, maxLongLiteral) val e6 = Multiply(minLongLiteral, minLongLiteral) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Long.MinValue) checkEvaluation(e2, Long.MinValue) checkEvaluation(e3, -2L) @@ -469,12 +469,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minIntLiteral, minIntLiteral) val e5 = Subtract(minIntLiteral, maxIntLiteral) val e6 = Multiply(minIntLiteral, minIntLiteral) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Int.MinValue) checkEvaluation(e2, Int.MinValue) checkEvaluation(e3, -2) @@ -493,12 +493,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minShortLiteral, minShortLiteral) val e5 = Subtract(minShortLiteral, maxShortLiteral) val e6 = Multiply(minShortLiteral, minShortLiteral) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Short.MinValue) checkEvaluation(e2, Short.MinValue) checkEvaluation(e3, (-2).toShort) @@ -517,12 +517,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minByteLiteral, minByteLiteral) val e5 = Subtract(minByteLiteral, maxByteLiteral) val e6 = Multiply(minByteLiteral, minByteLiteral) - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Byte.MinValue) checkEvaluation(e2, Byte.MinValue) checkEvaluation(e3, (-2).toByte) 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 7a781295a7957..ffb14e2838687 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 @@ -1044,14 +1044,14 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-28470: Cast should honor nullOnOverflow property") { - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(Cast(Literal("134.12"), DecimalType(3, 2)), null) checkEvaluation( Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), null) checkEvaluation(Cast(Literal(BigDecimal(134.12)), DecimalType(3, 2)), null) checkEvaluation(Cast(Literal(134.12), DecimalType(3, 2)), null) } - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkExceptionInExpression[ArithmeticException]( Cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( @@ -1117,7 +1117,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Cast to byte with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { testIntMaxAndMin(ByteType) Seq(Byte.MaxValue + 1, Byte.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ByteType), "overflow") @@ -1142,7 +1142,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Cast to short with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { testIntMaxAndMin(ShortType) Seq(Short.MaxValue + 1, Short.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ShortType), "overflow") @@ -1167,7 +1167,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Cast to int with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { testIntMaxAndMin(IntegerType) testLongMaxAndMin(IntegerType) @@ -1184,7 +1184,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Cast to long with option FAIL_ON_INTEGER_OVERFLOW enabled") { - withSQLConf(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { testLongMaxAndMin(LongType) Seq(Long.MaxValue, 0, Long.MinValue).foreach { value => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala index fc5e8dc5ee7f1..36bc3db580400 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala @@ -32,7 +32,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("MakeDecimal") { - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) @@ -41,7 +41,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { evaluateWithoutCodegen(overflowExpr, null) checkEvaluationWithUnsafeProjection(overflowExpr, null) } - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index 981ef57c051fd..c5ffc381b58e2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -57,7 +57,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-28369: honor nullOnOverflow config for ScalaUDF") { - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, @@ -69,7 +69,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } assert(e2.getCause.isInstanceOf[ArithmeticException]) } - withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index ee89980d3eedd..5da2bf059758d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -605,7 +605,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual(s"interval $intervalValue", expected) // SPARK-23264 Support interval values without INTERVAL clauses if ANSI SQL enabled - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } } @@ -682,12 +682,12 @@ class ExpressionParserSuite extends AnalysisTest { test("SPARK-23264 Interval Compatibility tests") { def checkIntervals(intervalValue: String, expected: Literal): Unit = { - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } // Compatibility tests: If ANSI SQL disabled, `intervalValue` should be parsed as an alias - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val aliases = defaultParser.parseExpression(intervalValue).collect { case a @ Alias(_: Literal, name) if intervalUnits.exists { unit => name.startsWith(unit) } => a @@ -785,12 +785,12 @@ class ExpressionParserSuite extends AnalysisTest { } test("current date/timestamp braceless expressions") { - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual("current_date", CurrentDate()) assertEqual("current_timestamp", CurrentTimestamp()) } - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { assertEqual("current_date", UnresolvedAttribute.quoted("current_date")) assertEqual("current_timestamp", UnresolvedAttribute.quoted("current_timestamp")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 81f0bd024a80e..a9216174804d0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -658,7 +658,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { } test("table identifier - reserved/non-reserved keywords if ANSI mode enabled") { - withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { reservedKeywordsInAnsiMode.foreach { keyword => val errMsg = intercept[ParseException] { parseTableIdentifier(keyword) diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql index f2f4b02c8634b..4a831855a091a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql @@ -1,5 +1,5 @@ -- Turns on ANSI mode -SET spark.sql.parser.ansi.enabled=true; +SET spark.sql.ansi.enabled=true; select '1' second, @@ -185,4 +185,4 @@ select date '2012-01-01' + interval (a + 1) day from t; select date '2012-01-01' + (a + 1) day from t; -- Turns off ANSI mode -SET spark.sql.parser.ansi.enabled=false; +SET spark.sql.ansi.enabled=false; diff --git a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql index 35f2be46cd130..a946123f709c7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql @@ -84,7 +84,7 @@ select 123456789123456789.1234567890 * 1.123456789123456789; select 12345678912345.123456789123 / 0.000000012345678; -- throw an exception instead of returning NULL, according to SQL ANSI 2011 -set spark.sql.decimalOperations.nullOnOverflow=false; +set spark.sql.ansi.enabled=true; -- test operations between decimals and constants select id, a*10, b/10 from decimals_test order by id; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql index 04d3acc145e95..7abf903bc6bee 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/text.sql @@ -45,10 +45,10 @@ select concat_ws('',10,20,null,30); select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); -- [SPARK-28036] Built-in udf left/right has inconsistent behavior --- [SPARK-28479] Parser error when enabling ANSI mode -set spark.sql.parser.ansi.enabled=false; +-- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode +set spark.sql.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; -set spark.sql.parser.ansi.enabled=true; +set spark.sql.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 13f72614f5778..43ad3c3f539f1 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -3,11 +3,11 @@ -- !query 0 -SET spark.sql.parser.ansi.enabled=true +SET spark.sql.ansi.enabled=true -- !query 0 schema struct -- !query 0 output -spark.sql.parser.ansi.enabled true +spark.sql.ansi.enabled true -- !query 1 @@ -432,8 +432,8 @@ select date '2012-01-01' + (a + 1) day from t -- !query 34 -SET spark.sql.parser.ansi.enabled=false +SET spark.sql.ansi.enabled=false -- !query 34 schema struct -- !query 34 output -spark.sql.parser.ansi.enabled false +spark.sql.ansi.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index 217233bfad378..00e139d90f488 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -328,11 +328,11 @@ NULL -- !query 39 -set spark.sql.decimalOperations.nullOnOverflow=false +set spark.sql.ansi.enabled=true -- !query 39 schema struct -- !query 39 output -spark.sql.decimalOperations.nullOnOverflow false +spark.sql.ansi.enabled true -- !query 40 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out index 352b0232e8945..2e1d639974ec6 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/text.sql.out @@ -151,11 +151,11 @@ edcba -- !query 18 -set spark.sql.parser.ansi.enabled=false +set spark.sql.ansi.enabled=false -- !query 18 schema struct -- !query 18 output -spark.sql.parser.ansi.enabled false +spark.sql.ansi.enabled false -- !query 19 @@ -177,11 +177,11 @@ struct -- !query 20 -set spark.sql.parser.ansi.enabled=true +set spark.sql.ansi.enabled=true -- !query 20 schema struct -- !query 20 output -spark.sql.parser.ansi.enabled true +spark.sql.ansi.enabled true -- !query 21 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 b4ddfecaee469..6a2eac9e7e21c 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 @@ -161,10 +161,10 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { DecimalData(BigDecimal("1"* 20 + ".123"), BigDecimal("1"* 20 + ".123")) :: DecimalData(BigDecimal("9"* 20 + ".123"), BigDecimal("9"* 20 + ".123")) :: Nil).toDF() - Seq(true, false).foreach { nullOnOverflow => - withSQLConf((SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key, nullOnOverflow.toString)) { + Seq(true, false).foreach { ansiEnabled => + withSQLConf((SQLConf.ANSI_ENABLED.key, ansiEnabled.toString)) { val structDf = largeDecimals.select("a").agg(sum("a")) - if (nullOnOverflow) { + if (!ansiEnabled) { checkAnswer(structDf, Row(null)) } else { val e = intercept[SparkException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index cc92d6556387d..abecaa7a9b16a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -308,13 +308,11 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("vol", (s: String) => s) // PostgreSQL enabled cartesian product by default. localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) - localSparkSession.conf.set(SQLConf.ANSI_SQL_PARSER.key, true) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) - localSparkSession.conf.set(SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key, true) - // Propagate the SQL conf FAIL_ON_INTEGRAL_TYPE_OVERFLOW to executor. + // Propagate the SQL conf ANSI_ENABLED to executor. // TODO: remove this after SPARK-29122 is resolved. - localSparkSession.sparkContext.setLocalProperty( - SQLConf.FAIL_ON_INTEGRAL_TYPE_OVERFLOW.key, "true") + localSparkSession.sparkContext.setLocalProperty(SQLConf.ANSI_ENABLED.key, "true") case _ => } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 1f7b3feae47b5..381b8f2324ca6 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -110,7 +110,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { case _: PgSQLTest => // PostgreSQL enabled cartesian product by default. statement.execute(s"SET ${SQLConf.CROSS_JOINS_ENABLED.key} = true") - statement.execute(s"SET ${SQLConf.ANSI_SQL_PARSER.key} = true") + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.PREFER_INTEGRAL_DIVISION.key} = true") case _ => } From 3bf43fb60d6f8aba23eaa1e43405024725b50f22 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 Sep 2019 00:24:15 -0700 Subject: [PATCH 0119/1387] [SPARK-29159][BUILD] Increase ReservedCodeCacheSize to 1G ### What changes were proposed in this pull request? This PR aims to increase the JVM CodeCacheSize from 0.5G to 1G. ### Why are the changes needed? After upgrading to `Scala 2.12.10`, the following is observed during building. ``` 2019-09-18T20:49:23.5030586Z OpenJDK 64-Bit Server VM warning: CodeCache is full. Compiler has been disabled. 2019-09-18T20:49:23.5032920Z OpenJDK 64-Bit Server VM warning: Try increasing the code cache size using -XX:ReservedCodeCacheSize= 2019-09-18T20:49:23.5034959Z CodeCache: size=524288Kb used=521399Kb max_used=521423Kb free=2888Kb 2019-09-18T20:49:23.5035472Z bounds [0x00007fa62c000000, 0x00007fa64c000000, 0x00007fa64c000000] 2019-09-18T20:49:23.5035781Z total_blobs=156549 nmethods=155863 adapters=592 2019-09-18T20:49:23.5036090Z compilation: disabled (not enough contiguous free space left) ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually check the Jenkins or GitHub Action build log (which should not have the above). Closes #25836 from dongjoon-hyun/SPARK-CODE-CACHE-1G. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 2 +- build/mvn | 2 +- dev/appveyor-install-dependencies.ps1 | 2 +- dev/make-distribution.sh | 2 +- docs/building-spark.md | 4 ++-- pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index dfed834c1d08e..99dd1d04a60d1 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -29,6 +29,6 @@ jobs: version: ${{ matrix.java }} - name: Build with Maven run: | - export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} package diff --git a/build/mvn b/build/mvn index f68377b3ddc71..3628be9880253 100755 --- a/build/mvn +++ b/build/mvn @@ -22,7 +22,7 @@ _DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" # Preserve the calling directory _CALLING_DIR="$(pwd)" # Options used during compilation -_COMPILE_JVM_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m" +_COMPILE_JVM_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g" # Installs any application tarball given a URL, the expected tarball name, # and, optionally, a checkable binary path to determine if the binary has diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index d33a107cc86a5..e0976e66db29f 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -90,7 +90,7 @@ Invoke-Expression "7z.exe x maven.zip" # add maven to environment variables $env:PATH = "$tools\apache-maven-$mavenVer\bin;" + $env:PATH $env:M2_HOME = "$tools\apache-maven-$mavenVer" -$env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=512m" +$env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=1g" Pop-Location diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index a550af93feecd..cd18b6870e07c 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -160,7 +160,7 @@ fi # Build uber fat JAR cd "$SPARK_HOME" -export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=512m}" +export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=1g}" # Store the command as an array because $MVN variable might have spaces in it. # Normal quoting tricks don't work. diff --git a/docs/building-spark.md b/docs/building-spark.md index 37f898645da68..9fd8b20d2dde6 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -34,7 +34,7 @@ Spark requires Scala 2.12; support for Scala 2.11 was removed in Spark 3.0.0. You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`: - export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m" + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g" (The `ReservedCodeCacheSize` setting is optional but recommended.) If you don't add these parameters to `MAVEN_OPTS`, you may see errors and warnings like the following: @@ -160,7 +160,7 @@ prompt. Configure the JVM options for SBT in `.jvmopts` at the project root, for example: -Xmx2g - -XX:ReservedCodeCacheSize=512m + -XX:ReservedCodeCacheSize=1g For the meanings of these two options, please carefully read the [Setting up Maven's Memory Usage section](https://spark.apache.org/docs/latest/building-spark.html#setting-up-mavens-memory-usage). diff --git a/pom.xml b/pom.xml index a78a5beb814f5..2f29ad07f1d90 100644 --- a/pom.xml +++ b/pom.xml @@ -240,7 +240,7 @@ --> ${session.executionRootDirectory} - 512m + 1g diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index cd3e8b234c189..84d99115a93ac 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -141,7 +141,7 @@ ${project.build.directory}/surefire-reports . SparkTestSuite.txt - -ea -Xmx4g -XX:ReservedCodeCacheSize=512m ${extraScalaTestArgs} + -ea -Xmx4g -XX:ReservedCodeCacheSize=1g ${extraScalaTestArgs} file:src/test/resources/log4j.properties From e97b55d32285052a1f76cca35377c4b21eb2e7d7 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 19 Sep 2019 08:17:25 -0500 Subject: [PATCH 0120/1387] [SPARK-28985][PYTHON][ML] Add common classes (JavaPredictor/JavaClassificationModel/JavaProbabilisticClassifier) in PYTHON ### What changes were proposed in this pull request? Add some common classes in Python to make it have the same structure as Scala 1. Scala has ClassifierParams/Classifier/ClassificationModel: ``` trait ClassifierParams extends PredictorParams with HasRawPredictionCol abstract class Classifier extends Predictor with ClassifierParams { def setRawPredictionCol } abstract class ClassificationModel extends PredictionModel with ClassifierParams { def setRawPredictionCol } ``` This PR makes Python has the following: ``` class JavaClassifierParams(HasRawPredictionCol, JavaPredictorParams): pass class JavaClassifier(JavaPredictor, JavaClassifierParams): def setRawPredictionCol class JavaClassificationModel(JavaPredictionModel, JavaClassifierParams): def setRawPredictionCol ``` 2. Scala has ProbabilisticClassifierParams/ProbabilisticClassifier/ProbabilisticClassificationModel: ``` trait ProbabilisticClassifierParams extends ClassifierParams with HasProbabilityCol with HasThresholds abstract class ProbabilisticClassifier extends Classifier with ProbabilisticClassifierParams { def setProbabilityCol def setThresholds } abstract class ProbabilisticClassificationModel extends ClassificationModel with ProbabilisticClassifierParams { def setProbabilityCol def setThresholds } ``` This PR makes Python have the following: ``` class JavaProbabilisticClassifierParams(HasProbabilityCol, HasThresholds, JavaClassifierParams): pass class JavaProbabilisticClassifier(JavaClassifier, JavaProbabilisticClassifierParams): def setProbabilityCol def setThresholds class JavaProbabilisticClassificationModel(JavaClassificationModel, JavaProbabilisticClassifierParams): def setProbabilityCol def setThresholds ``` 3. Scala has PredictorParams/Predictor/PredictionModel: ``` trait PredictorParams extends Params with HasLabelCol with HasFeaturesCol with HasPredictionCol abstract class Predictor extends Estimator with PredictorParams { def setLabelCol def setFeaturesCol def setPredictionCol } abstract class PredictionModel extends Model with PredictorParams { def setFeaturesCol def setPredictionCol def numFeatures def predict } ``` This PR makes Python have the following: ``` class JavaPredictorParams(HasLabelCol, HasFeaturesCol, HasPredictionCol): pass class JavaPredictor(JavaEstimator, JavaPredictorParams): def setLabelCol def setFeaturesCol def setPredictionCol class JavaPredictionModel(JavaModel, JavaPredictorParams): def setFeaturesCol def setPredictionCol def numFeatures def predict ``` ### Why are the changes needed? Have parity between Python and Scala ML ### Does this PR introduce any user-facing change? Yes. Add the following changes: ``` LinearSVCModel - get/setFeatureCol - get/setPredictionCol - get/setLabelCol - get/setRawPredictionCol - predict ``` ``` LogisticRegressionModel DecisionTreeClassificationModel RandomForestClassificationModel GBTClassificationModel NaiveBayesModel MultilayerPerceptronClassificationModel - get/setFeatureCol - get/setPredictionCol - get/setLabelCol - get/setRawPredictionCol - get/setProbabilityCol - predict ``` ``` LinearRegressionModel IsotonicRegressionModel DecisionTreeRegressionModel RandomForestRegressionModel GBTRegressionModel AFTSurvivalRegressionModel GeneralizedLinearRegressionModel - get/setFeatureCol - get/setPredictionCol - get/setLabelCol - predict ``` ### How was this patch tested? Add a few doc tests. Closes #25776 from huaxingao/spark-28985. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- python/pyspark/ml/classification.py | 212 ++++++++++++++++++++------ python/pyspark/ml/regression.py | 74 +++++---- python/pyspark/ml/tests/test_param.py | 3 +- python/pyspark/ml/util.py | 16 -- python/pyspark/ml/wrapper.py | 74 +++++++++ 5 files changed, 291 insertions(+), 88 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index b6c28d05ea273..e2e313c7f9252 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -26,8 +26,8 @@ DecisionTreeRegressionModel, GBTParams, HasVarianceImpurity, RandomForestParams, \ TreeEnsembleModel from pyspark.ml.util import * -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams -from pyspark.ml.wrapper import JavaWrapper +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, \ + JavaPredictor, JavaPredictorParams, JavaPredictionModel, JavaWrapper from pyspark.ml.common import inherit_doc, _java2py, _py2java from pyspark.ml.linalg import Vectors from pyspark.sql import DataFrame @@ -47,14 +47,43 @@ 'OneVsRest', 'OneVsRestModel'] +class JavaClassifierParams(HasRawPredictionCol, JavaPredictorParams): + """ + (Private) Java Classifier Params for classification tasks. + """ + pass + + @inherit_doc -class JavaClassificationModel(JavaPredictionModel): +class JavaClassifier(JavaPredictor, JavaClassifierParams): """ - (Private) Java Model produced by a ``Classifier``. + Java Classifier for classification tasks. + Classes are indexed {0, 1, ..., numClasses - 1}. + """ + + @since("3.0.0") + def setRawPredictionCol(self, value): + """ + Sets the value of :py:attr:`rawPredictionCol`. + """ + return self._set(rawPredictionCol=value) + + +@inherit_doc +class JavaClassificationModel(JavaPredictionModel, JavaClassifierParams): + """ + Java Model produced by a ``Classifier``. Classes are indexed {0, 1, ..., numClasses - 1}. To be mixed in with class:`pyspark.ml.JavaModel` """ + @since("3.0.0") + def setRawPredictionCol(self, value): + """ + Sets the value of :py:attr:`rawPredictionCol`. + """ + return self._set(rawPredictionCol=value) + @property @since("2.1.0") def numClasses(self): @@ -64,10 +93,60 @@ def numClasses(self): return self._call_java("numClasses") +class JavaProbabilisticClassifierParams(HasProbabilityCol, HasThresholds, JavaClassifierParams): + """ + (Private) Java Probabilistic Classifier Params for classification tasks. + """ + pass + + @inherit_doc -class LinearSVC(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, - HasRegParam, HasTol, HasRawPredictionCol, HasFitIntercept, HasStandardization, - HasWeightCol, HasAggregationDepth, HasThreshold, JavaMLWritable, JavaMLReadable): +class JavaProbabilisticClassifier(JavaClassifier, JavaProbabilisticClassifierParams): + """ + Java Probabilistic Classifier for classification tasks. + """ + + @since("3.0.0") + def setProbabilityCol(self, value): + """ + Sets the value of :py:attr:`probabilityCol`. + """ + return self._set(probabilityCol=value) + + @since("3.0.0") + def setThresholds(self, value): + """ + Sets the value of :py:attr:`thresholds`. + """ + return self._set(thresholds=value) + + +@inherit_doc +class JavaProbabilisticClassificationModel(JavaClassificationModel, + JavaProbabilisticClassifierParams): + """ + Java Model produced by a ``ProbabilisticClassifier``. + """ + + @since("3.0.0") + def setProbabilityCol(self, value): + """ + Sets the value of :py:attr:`probabilityCol`. + """ + return self._set(probabilityCol=value) + + @since("3.0.0") + def setThresholds(self, value): + """ + Sets the value of :py:attr:`thresholds`. + """ + return self._set(thresholds=value) + + +@inherit_doc +class LinearSVC(JavaClassifier, HasMaxIter, HasRegParam, HasTol, + HasFitIntercept, HasStandardization, HasWeightCol, HasAggregationDepth, + HasThreshold, JavaMLWritable, JavaMLReadable): """ `Linear SVM Classifier `_ @@ -81,6 +160,10 @@ class LinearSVC(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, Ha ... Row(label=0.0, features=Vectors.dense(1.0, 2.0, 3.0))]).toDF() >>> svm = LinearSVC(maxIter=5, regParam=0.01) >>> model = svm.fit(df) + >>> model.setPredictionCol("newPrediction") + LinearSVC... + >>> model.getPredictionCol() + 'newPrediction' >>> model.coefficients DenseVector([0.0, -0.2792, -0.1833]) >>> model.intercept @@ -90,8 +173,10 @@ class LinearSVC(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, Ha >>> model.numFeatures 3 >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0, -1.0, -1.0))]).toDF() + >>> model.predict(test0.head().features) + 1.0 >>> result = model.transform(test0).head() - >>> result.prediction + >>> result.newPrediction 1.0 >>> result.rawPrediction DenseVector([-1.4831, 1.4831]) @@ -156,7 +241,7 @@ def _create_model(self, java_model): return LinearSVCModel(java_model) -class LinearSVCModel(JavaModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable): +class LinearSVCModel(JavaClassificationModel, JavaMLWritable, JavaMLReadable): """ Model fitted by LinearSVC. @@ -181,8 +266,7 @@ def intercept(self): @inherit_doc -class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, - HasRegParam, HasTol, HasProbabilityCol, HasRawPredictionCol, +class LogisticRegression(JavaProbabilisticClassifier, HasMaxIter, HasRegParam, HasTol, HasElasticNetParam, HasFitIntercept, HasStandardization, HasThresholds, HasWeightCol, HasAggregationDepth, JavaMLWritable, JavaMLReadable): """ @@ -198,6 +282,12 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti ... Row(label=0.0, weight=4.0, features=Vectors.dense(3.0, 3.0))]).toDF() >>> blor = LogisticRegression(regParam=0.01, weightCol="weight") >>> blorModel = blor.fit(bdf) + >>> blorModel.setFeaturesCol("features") + LogisticRegressionModel... + >>> blorModel.setProbabilityCol("newProbability") + LogisticRegressionModel... + >>> blorModel.getProbabilityCol() + 'newProbability' >>> blorModel.coefficients DenseVector([-1.080..., -0.646...]) >>> blorModel.intercept @@ -211,10 +301,12 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti >>> mlorModel.interceptVector DenseVector([0.04..., -0.42..., 0.37...]) >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0, 1.0))]).toDF() + >>> blorModel.predict(test0.head().features) + 1.0 >>> result = blorModel.transform(test0).head() >>> result.prediction 1.0 - >>> result.probability + >>> result.newProbability DenseVector([0.02..., 0.97...]) >>> result.rawPrediction DenseVector([-3.54..., 3.54...]) @@ -481,7 +573,7 @@ def getUpperBoundsOnIntercepts(self): return self.getOrDefault(self.upperBoundsOnIntercepts) -class LogisticRegressionModel(JavaModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable, +class LogisticRegressionModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by LogisticRegression. @@ -872,8 +964,7 @@ def getImpurity(self): @inherit_doc -class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeightCol, - HasPredictionCol, HasProbabilityCol, HasRawPredictionCol, +class DecisionTreeClassifier(JavaProbabilisticClassifier, HasWeightCol, DecisionTreeParams, TreeClassifierParams, HasCheckpointInterval, HasSeed, JavaMLWritable, JavaMLReadable): """ @@ -892,6 +983,10 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeig >>> td = si_model.transform(df) >>> dt = DecisionTreeClassifier(maxDepth=2, labelCol="indexed", leafCol="leafId") >>> model = dt.fit(td) + >>> model.getLabelCol() + 'indexed' + >>> model.setFeaturesCol("features") + DecisionTreeClassificationModel... >>> model.numNodes 3 >>> model.depth @@ -905,6 +1000,8 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeig >>> print(model.toDebugString) DecisionTreeClassificationModel (uid=...) of depth 1 with 3 nodes... >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -1031,8 +1128,8 @@ def setImpurity(self, value): @inherit_doc -class DecisionTreeClassificationModel(DecisionTreeModel, JavaClassificationModel, JavaMLWritable, - JavaMLReadable): +class DecisionTreeClassificationModel(DecisionTreeModel, JavaProbabilisticClassificationModel, + JavaMLWritable, JavaMLReadable): """ Model fitted by DecisionTreeClassifier. @@ -1062,9 +1159,8 @@ def featureImportances(self): @inherit_doc -class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasSeed, - HasRawPredictionCol, HasProbabilityCol, - RandomForestParams, TreeClassifierParams, HasCheckpointInterval, +class RandomForestClassifier(JavaProbabilisticClassifier, HasSeed, RandomForestParams, + TreeClassifierParams, HasCheckpointInterval, JavaMLWritable, JavaMLReadable): """ `Random Forest `_ @@ -1085,17 +1181,27 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred >>> rf = RandomForestClassifier(numTrees=3, maxDepth=2, labelCol="indexed", seed=42, ... leafCol="leafId") >>> model = rf.fit(td) + >>> model.getLabelCol() + 'indexed' + >>> model.setFeaturesCol("features") + RandomForestClassificationModel... + >>> model.setRawPredictionCol("newRawPrediction") + RandomForestClassificationModel... + >>> model.getRawPredictionCol() + 'newRawPrediction' >>> model.featureImportances SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 1.0, 1.0]) True >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 >>> numpy.argmax(result.probability) 0 - >>> numpy.argmax(result.rawPrediction) + >>> numpy.argmax(result.newRawPrediction) 0 >>> result.leafId DenseVector([0.0, 0.0, 0.0]) @@ -1231,8 +1337,8 @@ def setFeatureSubsetStrategy(self, value): return self._set(featureSubsetStrategy=value) -class RandomForestClassificationModel(TreeEnsembleModel, JavaClassificationModel, JavaMLWritable, - JavaMLReadable): +class RandomForestClassificationModel(TreeEnsembleModel, JavaProbabilisticClassificationModel, + JavaMLWritable, JavaMLReadable): """ Model fitted by RandomForestClassifier. @@ -1284,9 +1390,8 @@ def getLossType(self): @inherit_doc -class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - GBTClassifierParams, HasCheckpointInterval, HasSeed, JavaMLWritable, - JavaMLReadable): +class GBTClassifier(JavaProbabilisticClassifier, GBTClassifierParams, HasCheckpointInterval, + HasSeed, JavaMLWritable, JavaMLReadable): """ `Gradient-Boosted Trees (GBTs) `_ learning algorithm for classification. @@ -1318,11 +1423,21 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol >>> gbt.getFeatureSubsetStrategy() 'all' >>> model = gbt.fit(td) + >>> model.getLabelCol() + 'indexed' + >>> model.setFeaturesCol("features") + GBTClassificationModel... + >>> model.setThresholds([0.3, 0.7]) + GBTClassificationModel... + >>> model.getThresholds() + [0.3, 0.7] >>> model.featureImportances SparseVector(1, {0: 1.0}) >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1]) True >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -1485,8 +1600,8 @@ def setValidationIndicatorCol(self, value): return self._set(validationIndicatorCol=value) -class GBTClassificationModel(TreeEnsembleModel, JavaClassificationModel, JavaMLWritable, - JavaMLReadable): +class GBTClassificationModel(TreeEnsembleModel, JavaProbabilisticClassificationModel, + JavaMLWritable, JavaMLReadable): """ Model fitted by GBTClassifier. @@ -1527,8 +1642,8 @@ def evaluateEachIteration(self, dataset): @inherit_doc -class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasProbabilityCol, - HasRawPredictionCol, HasThresholds, HasWeightCol, JavaMLWritable, JavaMLReadable): +class NaiveBayes(JavaProbabilisticClassifier, HasThresholds, HasWeightCol, + JavaMLWritable, JavaMLReadable): """ Naive Bayes Classifiers. It supports both Multinomial and Bernoulli NB. `Multinomial NB @@ -1547,11 +1662,19 @@ class NaiveBayes(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, H ... Row(label=1.0, weight=1.0, features=Vectors.dense([1.0, 0.0]))]) >>> nb = NaiveBayes(smoothing=1.0, modelType="multinomial", weightCol="weight") >>> model = nb.fit(df) + >>> model.setFeaturesCol("features") + NaiveBayes_... + >>> model.setLabelCol("newLabel") + NaiveBayes_... + >>> model.getLabelCol() + 'newLabel' >>> model.pi DenseVector([-0.81..., -0.58...]) >>> model.theta DenseMatrix(2, 2, [-0.91..., -0.51..., -0.40..., -1.09...], 1) >>> test0 = sc.parallelize([Row(features=Vectors.dense([1.0, 0.0]))]).toDF() + >>> model.predict(test0.head().features) + 1.0 >>> result = model.transform(test0).head() >>> result.prediction 1.0 @@ -1651,7 +1774,7 @@ def getModelType(self): return self.getOrDefault(self.modelType) -class NaiveBayesModel(JavaModel, JavaClassificationModel, JavaMLWritable, JavaMLReadable): +class NaiveBayesModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable): """ Model fitted by NaiveBayes. @@ -1676,10 +1799,8 @@ def theta(self): @inherit_doc -class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasMaxIter, HasTol, HasSeed, HasStepSize, HasSolver, - JavaMLWritable, JavaMLReadable, HasProbabilityCol, - HasRawPredictionCol): +class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, HasMaxIter, HasTol, HasSeed, + HasStepSize, HasSolver, JavaMLWritable, JavaMLReadable): """ Classifier trainer based on the Multilayer Perceptron. Each layer has sigmoid activation function, output layer has softmax. @@ -1694,6 +1815,8 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, ... (0.0, Vectors.dense([1.0, 1.0]))], ["label", "features"]) >>> mlp = MultilayerPerceptronClassifier(maxIter=100, layers=[2, 2, 2], blockSize=1, seed=123) >>> model = mlp.fit(df) + >>> model.setFeaturesCol("features") + MultilayerPerceptronClassifier... >>> model.layers [2, 2, 2] >>> model.weights.size @@ -1701,6 +1824,8 @@ class MultilayerPerceptronClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, >>> testDF = spark.createDataFrame([ ... (Vectors.dense([1.0, 0.0]),), ... (Vectors.dense([0.0, 0.0]),)], ["features"]) + >>> model.predict(testDF.head().features) + 1.0 >>> model.transform(testDF).select("features", "prediction").show() +---------+----------+ | features|prediction| @@ -1839,7 +1964,7 @@ def getInitialWeights(self): return self.getOrDefault(self.initialWeights) -class MultilayerPerceptronClassificationModel(JavaModel, JavaClassificationModel, JavaMLWritable, +class MultilayerPerceptronClassificationModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable): """ Model fitted by MultilayerPerceptronClassifier. @@ -1864,8 +1989,7 @@ def weights(self): return self._call_java("weights") -class OneVsRestParams(HasFeaturesCol, HasLabelCol, HasWeightCol, HasPredictionCol, - HasRawPredictionCol): +class OneVsRestParams(JavaClassifierParams, HasWeightCol): """ Parameters for OneVsRest and OneVsRestModel. """ @@ -1897,6 +2021,8 @@ class OneVsRest(Estimator, OneVsRestParams, HasParallelism, JavaMLReadable, Java >>> ovr = OneVsRest(classifier=lr) >>> ovr.getRawPredictionCol() 'rawPrediction' + >>> ovr.setPredictionCol("newPrediction") + OneVsRest... >>> model = ovr.fit(df) >>> model.models[0].coefficients DenseVector([0.5..., -1.0..., 3.4..., 4.2...]) @@ -1907,21 +2033,21 @@ class OneVsRest(Estimator, OneVsRestParams, HasParallelism, JavaMLReadable, Java >>> [x.intercept for x in model.models] [-2.7..., -2.5..., -1.3...] >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0, 0.0, 1.0, 1.0))]).toDF() - >>> model.transform(test0).head().prediction + >>> model.transform(test0).head().newPrediction 0.0 >>> test1 = sc.parallelize([Row(features=Vectors.sparse(4, [0], [1.0]))]).toDF() - >>> model.transform(test1).head().prediction + >>> model.transform(test1).head().newPrediction 2.0 >>> test2 = sc.parallelize([Row(features=Vectors.dense(0.5, 0.4, 0.3, 0.2))]).toDF() - >>> model.transform(test2).head().prediction + >>> model.transform(test2).head().newPrediction 0.0 >>> model_path = temp_path + "/ovr_model" >>> model.save(model_path) >>> model2 = OneVsRestModel.load(model_path) - >>> model2.transform(test0).head().prediction + >>> model2.transform(test0).head().newPrediction 0.0 >>> model.transform(test2).columns - ['features', 'rawPrediction', 'prediction'] + ['features', 'rawPrediction', 'newPrediction'] .. versionadded:: 2.0.0 """ diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 2d1d1272c17f8..f2bcc662030c6 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -20,7 +20,8 @@ from pyspark import since, keyword_only from pyspark.ml.param.shared import * from pyspark.ml.util import * -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaWrapper +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, \ + JavaPredictor, JavaPredictionModel, JavaWrapper from pyspark.ml.common import inherit_doc from pyspark.sql import DataFrame @@ -37,10 +38,9 @@ @inherit_doc -class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, - HasRegParam, HasTol, HasElasticNetParam, HasFitIntercept, - HasStandardization, HasSolver, HasWeightCol, HasAggregationDepth, HasLoss, - JavaMLWritable, JavaMLReadable): +class LinearRegression(JavaPredictor, HasMaxIter, HasRegParam, HasTol, HasElasticNetParam, + HasFitIntercept, HasStandardization, HasSolver, HasWeightCol, + HasAggregationDepth, HasLoss, JavaMLWritable, JavaMLReadable): """ Linear regression. @@ -66,15 +66,21 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction ... (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) >>> lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight") >>> model = lr.fit(df) + >>> model.setFeaturesCol("features") + LinearRegression... + >>> model.setPredictionCol("newPrediction") + LinearRegression... >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) - >>> abs(model.transform(test0).head().prediction - (-1.0)) < 0.001 + >>> abs(model.predict(test0.head().features) - (-1.0)) < 0.001 + True + >>> abs(model.transform(test0).head().newPrediction - (-1.0)) < 0.001 True >>> abs(model.coefficients[0] - 1.0) < 0.001 True >>> abs(model.intercept - 0.0) < 0.001 True >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) - >>> abs(model.transform(test1).head().prediction - 1.0) < 0.001 + >>> abs(model.transform(test1).head().newPrediction - 1.0) < 0.001 True >>> lr.setParams("vector") Traceback (most recent call last): @@ -161,7 +167,7 @@ def getEpsilon(self): return self.getOrDefault(self.epsilon) -class LinearRegressionModel(JavaModel, JavaPredictionModel, GeneralJavaMLWritable, JavaMLReadable, +class LinearRegressionModel(JavaPredictionModel, GeneralJavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by :class:`LinearRegression`. @@ -460,8 +466,7 @@ def totalIterations(self): @inherit_doc -class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasWeightCol, JavaMLWritable, JavaMLReadable): +class IsotonicRegression(JavaPredictor, HasWeightCol, JavaMLWritable, JavaMLReadable): """ Currently implemented using parallelized pool adjacent violators algorithm. Only univariate (single feature) algorithm supported. @@ -472,6 +477,12 @@ class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> ir = IsotonicRegression() >>> model = ir.fit(df) + >>> model.setFeaturesCol("features") + IsotonicRegression... + >>> model.setLabelCol("newLabel") + IsotonicRegression... + >>> model.getLabelCol() + 'newLabel' >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 @@ -555,7 +566,7 @@ def getFeatureIndex(self): return self.getOrDefault(self.featureIndex) -class IsotonicRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): +class IsotonicRegressionModel(JavaPredictionModel, JavaMLWritable, JavaMLReadable): """ Model fitted by :class:`IsotonicRegression`. @@ -804,8 +815,7 @@ def getLossType(self): @inherit_doc -class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeightCol, - HasPredictionCol, DecisionTreeParams, TreeRegressorParams, +class DecisionTreeRegressor(JavaPredictor, HasWeightCol, DecisionTreeParams, TreeRegressorParams, HasCheckpointInterval, HasSeed, JavaMLWritable, JavaMLReadable, HasVarianceCol): """ @@ -828,6 +838,8 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasWeigh >>> model.numFeatures 1 >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -950,7 +962,7 @@ def setImpurity(self, value): @inherit_doc -class DecisionTreeModel(JavaModel, JavaPredictionModel): +class DecisionTreeModel(JavaPredictionModel): """ Abstraction for Decision Tree models. @@ -1052,9 +1064,8 @@ def featureImportances(self): @inherit_doc -class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasSeed, - RandomForestParams, TreeRegressorParams, HasCheckpointInterval, - JavaMLWritable, JavaMLReadable): +class RandomForestRegressor(JavaPredictor, HasSeed, RandomForestParams, TreeRegressorParams, + HasCheckpointInterval, JavaMLWritable, JavaMLReadable): """ `Random Forest `_ learning algorithm for regression. @@ -1072,6 +1083,8 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi >>> allclose(model.treeWeights, [1.0, 1.0]) True >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -1240,9 +1253,8 @@ def featureImportances(self): @inherit_doc -class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - GBTRegressorParams, HasCheckpointInterval, HasSeed, JavaMLWritable, - JavaMLReadable): +class GBTRegressor(JavaPredictor, GBTRegressorParams, HasCheckpointInterval, HasSeed, + JavaMLWritable, JavaMLReadable): """ `Gradient-Boosted Trees (GBTs) `_ learning algorithm for regression. @@ -1259,6 +1271,8 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, >>> print(gbt.getFeatureSubsetStrategy()) all >>> model = gbt.fit(df) + >>> model.setFeaturesCol("features") + GBTRegressionModel... >>> model.featureImportances SparseVector(1, {0: 1.0}) >>> model.numFeatures @@ -1266,6 +1280,8 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1]) True >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) + >>> model.predict(test0.head().features) + 0.0 >>> result = model.transform(test0).head() >>> result.prediction 0.0 @@ -1465,9 +1481,8 @@ def evaluateEachIteration(self, dataset, loss): @inherit_doc -class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasFitIntercept, HasMaxIter, HasTol, HasAggregationDepth, - JavaMLWritable, JavaMLReadable): +class AFTSurvivalRegression(JavaPredictor, HasFitIntercept, HasMaxIter, HasTol, + HasAggregationDepth, JavaMLWritable, JavaMLReadable): """ Accelerated Failure Time (AFT) Model Survival Regression @@ -1482,6 +1497,8 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi ... (1e-40, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) >>> aftsr = AFTSurvivalRegression() >>> model = aftsr.fit(df) + >>> model.setFeaturesCol("features") + AFTSurvivalRegression... >>> model.predict(Vectors.dense(6.3)) 1.0 >>> model.predictQuantiles(Vectors.dense(6.3)) @@ -1607,7 +1624,7 @@ def getQuantilesCol(self): return self.getOrDefault(self.quantilesCol) -class AFTSurvivalRegressionModel(JavaModel, JavaMLWritable, JavaMLReadable): +class AFTSurvivalRegressionModel(JavaPredictionModel, JavaMLWritable, JavaMLReadable): """ Model fitted by :class:`AFTSurvivalRegression`. @@ -1654,9 +1671,8 @@ def predict(self, features): @inherit_doc -class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, HasPredictionCol, - HasFitIntercept, HasMaxIter, HasTol, HasRegParam, HasWeightCol, - HasSolver, JavaMLWritable, JavaMLReadable): +class GeneralizedLinearRegression(JavaPredictor, HasFitIntercept, HasMaxIter, HasTol, HasRegParam, + HasWeightCol, HasSolver, JavaMLWritable, JavaMLReadable): """ Generalized Linear Regression. @@ -1686,6 +1702,8 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha ... (2.0, Vectors.dense(1.0, 1.0)),], ["label", "features"]) >>> glr = GeneralizedLinearRegression(family="gaussian", link="identity", linkPredictionCol="p") >>> model = glr.fit(df) + >>> model.setFeaturesCol("features") + GeneralizedLinearRegression... >>> transformed = model.transform(df) >>> abs(transformed.head().prediction - 1.5) < 0.001 True @@ -1861,7 +1879,7 @@ def getOffsetCol(self): return self.getOrDefault(self.offsetCol) -class GeneralizedLinearRegressionModel(JavaModel, JavaPredictionModel, JavaMLWritable, +class GeneralizedLinearRegressionModel(JavaPredictionModel, JavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by :class:`GeneralizedLinearRegression`. diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index 72b146054ca44..f12310c7d8eba 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -352,7 +352,8 @@ def test_java_params(self): for module in modules: for name, cls in inspect.getmembers(module, inspect.isclass): if not name.endswith('Model') and not name.endswith('Params') \ - and issubclass(cls, JavaParams) and not inspect.isabstract(cls): + and issubclass(cls, JavaParams) and not inspect.isabstract(cls) \ + and not name.startswith('Java'): # NOTE: disable check_params_exist until there is parity with Scala API check_params(self, cls(), check_params_exist=False) diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index debaf38e0896e..81b785e71f302 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -342,22 +342,6 @@ def read(cls): return JavaMLReader(cls) -@inherit_doc -class JavaPredictionModel(): - """ - (Private) Java Model for prediction tasks (regression and classification). - To be mixed in with class:`pyspark.ml.JavaModel` - """ - - @property - @since("2.1.0") - def numFeatures(self): - """ - Returns the number of features the model was trained on. If unknown, returns -1 - """ - return self._call_java("numFeatures") - - @inherit_doc class DefaultParamsWritable(MLWritable): """ diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 9bb1262a54500..3137db7181c07 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -20,10 +20,12 @@ if sys.version >= '3': xrange = range +from pyspark import since from pyspark import SparkContext from pyspark.sql import DataFrame from pyspark.ml import Estimator, Transformer, Model from pyspark.ml.param import Params +from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol from pyspark.ml.util import _jvm from pyspark.ml.common import inherit_doc, _java2py, _py2java @@ -361,3 +363,75 @@ def __init__(self, java_model=None): self._create_params_from_java() self._resetUid(java_model.uid()) + + +@inherit_doc +class JavaPredictorParams(HasLabelCol, HasFeaturesCol, HasPredictionCol): + """ + (Private) Trait for parameters for prediction (regression and classification) + """ + pass + + +@inherit_doc +class JavaPredictor(JavaEstimator, JavaPredictorParams): + """ + (Private) Java Estimator for prediction tasks (regression and classification). + """ + + @since("3.0.0") + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + +@inherit_doc +class JavaPredictionModel(JavaModel, JavaPredictorParams): + """ + (Private) Java Model for prediction tasks (regression and classification). + """ + + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @property + @since("2.1.0") + def numFeatures(self): + """ + Returns the number of features the model was trained on. If unknown, returns -1 + """ + return self._call_java("numFeatures") + + @since("3.0.0") + def predict(self, value): + """ + Predict label for the given features. + """ + return self._call_java("predict", value) From c5d8a51f3b956c226d38f8fdc9ec644839e8b2f2 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 19 Sep 2019 11:37:42 -0700 Subject: [PATCH 0121/1387] [MINOR][BUILD] Fix about 15 misc build warnings ### What changes were proposed in this pull request? This addresses about 15 miscellaneous warnings that appear in the current build. ### Why are the changes needed? No functional changes, it just slightly reduces the amount of extra warning output. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests, run manually. Closes #25852 from srowen/BuildWarnings. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/util/kvstore/InMemoryStore.java | 2 +- .../java/org/apache/spark/util/kvstore/KVTypeInfo.java | 6 +++--- .../spark/shuffle/sort/UnsafeShuffleWriterSuite.java | 6 +++--- .../spark/deploy/history/HistoryServerSuite.scala | 4 +--- .../scala/org/apache/spark/util/KeyLockSuite.scala | 2 +- .../apache/spark/repl/ExecutorClassLoaderSuite.scala | 3 ++- .../spark/sql/catalyst/trees/TreeNodeSuite.scala | 10 +++++----- 7 files changed, 16 insertions(+), 17 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java index 6af45aec3c7b2..b33c53871c32f 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java @@ -252,7 +252,7 @@ private static Predicate getPredicate( return (value) -> set.contains(indexValueForEntity(getter, value)); } else { - HashSet set = new HashSet<>(values.size()); + HashSet> set = new HashSet<>(values.size()); for (Object key : values) { set.add(asKey(key)); } diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java index b8c5fab8709ed..d2a26982d8703 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java @@ -124,7 +124,7 @@ interface Accessor { Object get(Object instance) throws ReflectiveOperationException; - Class getType(); + Class getType(); } private class FieldAccessor implements Accessor { @@ -141,7 +141,7 @@ public Object get(Object instance) throws ReflectiveOperationException { } @Override - public Class getType() { + public Class getType() { return field.getType(); } } @@ -160,7 +160,7 @@ public Object get(Object instance) throws ReflectiveOperationException { } @Override - public Class getType() { + public Class getType() { return method.getReturnType(); } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 1022111897a49..4f660a13f7400 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -172,9 +172,9 @@ public void setUp() throws IOException { when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager); } - private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { + private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); - return new UnsafeShuffleWriter( + return new UnsafeShuffleWriter<>( blockManager, taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), @@ -533,7 +533,7 @@ public void testPeakMemoryUsed() throws Exception { final long numRecordsPerPage = pageSizeBytes / recordLengthBytes; taskMemoryManager = spy(taskMemoryManager); when(taskMemoryManager.pageSizeBytes()).thenReturn(pageSizeBytes); - final UnsafeShuffleWriter writer = new UnsafeShuffleWriter( + final UnsafeShuffleWriter writer = new UnsafeShuffleWriter<>( blockManager, taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 71a127bd4b9f6..e3cfec90ee81b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -26,7 +26,6 @@ import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpSe import scala.collection.JavaConverters._ import scala.concurrent.duration._ -import com.gargoylesoftware.htmlunit.BrowserVersion import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.{FileUtils, IOUtils} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -365,8 +364,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers contextHandler.addServlet(holder, "/") server.attachHandler(contextHandler) - implicit val webDriver: WebDriver = - new HtmlUnitDriver(BrowserVersion.INTERNET_EXPLORER_11, true) + implicit val webDriver: WebDriver = new HtmlUnitDriver(true) try { val url = s"http://localhost:$port" diff --git a/core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala b/core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala index 2169a0e4d442f..6888e492a8d33 100644 --- a/core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/KeyLockSuite.scala @@ -49,7 +49,7 @@ class KeyLockSuite extends SparkFunSuite with TimeLimits { @volatile var e: Throwable = null val threads = (0 until numThreads).map { i => new Thread() { - override def run(): Unit = try { + override def run(): Unit = { latch.await(foreverMs, TimeUnit.MILLISECONDS) keyLock.withLock(keys(i)) { var cur = numThreadsHoldingLock.get() diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 1d2e8acff9a3a..6c694804e451b 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -272,7 +272,8 @@ class ExecutorClassLoaderSuite assert(e.getMessage.contains("ThisIsAClassName")) // RemoteClassLoaderError must not be LinkageError nor ClassNotFoundException. Otherwise, // JVM will cache it and doesn't retry to load a class. - assert(!e.isInstanceOf[LinkageError] && !e.isInstanceOf[ClassNotFoundException]) + assert(!(classOf[LinkageError].isAssignableFrom(e.getClass))) + assert(!(classOf[ClassNotFoundException].isAssignableFrom(e.getClass))) } finally { rpcEnv.shutdown() rpcEnv.awaitTermination() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index fbaa5527a7058..828706743c5b5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -680,12 +680,12 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { } test("clone") { - def assertDifferentInstance(before: AnyRef, after: AnyRef): Unit = { + def assertDifferentInstance[T <: TreeNode[T]](before: TreeNode[T], after: TreeNode[T]): Unit = { assert(before.ne(after) && before == after) - before.asInstanceOf[TreeNode[_]].children.zip( - after.asInstanceOf[TreeNode[_]].children).foreach { - case (beforeChild: AnyRef, afterChild: AnyRef) => - assertDifferentInstance(beforeChild, afterChild) + before.children.zip(after.children).foreach { case (beforeChild, afterChild) => + assertDifferentInstance( + beforeChild.asInstanceOf[TreeNode[T]], + afterChild.asInstanceOf[TreeNode[T]]) } } From eee2e026bb4019020b492c54ba51a97df42b13c2 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 19 Sep 2019 11:47:47 -0700 Subject: [PATCH 0122/1387] [SPARK-29165][SQL][TEST] Set log level of log generated code as ERROR in case of compile error on generated code in UT ### What changes were proposed in this pull request? This patch proposes to change the log level of logging generated code in case of compile error being occurred in UT. This would help to investigate compilation issue of generated code easier, as currently we got exception message of line number but there's no generated code being logged actually (as in most cases of UT the threshold of log level is at least WARN). ### Why are the changes needed? This would help investigating issue on compilation error for generated code in UT. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #25835 from HeartSaVioR/MINOR-always-log-generated-code-on-fail-to-compile-in-unit-testing. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../expressions/codegen/CodeGenerator.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 1d010ff5abfcf..68ddec9fc8d00 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1369,20 +1369,27 @@ object CodeGenerator extends Logging { case e: InternalCompilerException => val msg = s"failed to compile: $e" logError(msg, e) - val maxLines = SQLConf.get.loggingMaxLinesForCodegen - logInfo(s"\n${CodeFormatter.format(code, maxLines)}") + logGeneratedCode(code) throw new InternalCompilerException(msg, e) case e: CompileException => val msg = s"failed to compile: $e" logError(msg, e) - val maxLines = SQLConf.get.loggingMaxLinesForCodegen - logInfo(s"\n${CodeFormatter.format(code, maxLines)}") + logGeneratedCode(code) throw new CompileException(msg, e.getLocation) } (evaluator.getClazz().getConstructor().newInstance().asInstanceOf[GeneratedClass], codeStats) } + private def logGeneratedCode(code: CodeAndComment): Unit = { + val maxLines = SQLConf.get.loggingMaxLinesForCodegen + if (Utils.isTesting) { + logError(s"\n${CodeFormatter.format(code, maxLines)}") + } else { + logInfo(s"\n${CodeFormatter.format(code, maxLines)}") + } + } + /** * Returns the bytecode statistics (max method bytecode size, max constant pool size, and * # of inner classes) of generated classes by inspecting Janino classes. From d3eb4c94cc0d927c7bab310b05e1bb9427a7d67a Mon Sep 17 00:00:00 2001 From: shivusondur Date: Thu, 19 Sep 2019 13:04:17 -0700 Subject: [PATCH 0123/1387] [SPARK-28822][DOC][SQL] Document USE DATABASE in SQL Reference ### What changes were proposed in this pull request? Added document reference for USE databse sql command ### Why are the changes needed? For USE database command usage ### Does this PR introduce any user-facing change? It is adding the USE database sql command refernce information in the doc ### How was this patch tested? Attached the test snap ![image](https://user-images.githubusercontent.com/7912929/65170499-7242a380-da66-11e9-819c-76df62c86c5a.png) Closes #25572 from shivusondur/jiraUSEDaBa1. Lead-authored-by: shivusondur Co-authored-by: Xiao Li Signed-off-by: Xiao Li --- docs/_data/menu-sql.yaml | 2 + docs/sql-ref-syntax-qry-select-usedb.md | 60 +++++++++++++++++++++++++ 2 files changed, 62 insertions(+) create mode 100644 docs/sql-ref-syntax-qry-select-usedb.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 07ca150a1523f..edcdad4ee7db5 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -133,6 +133,8 @@ url: sql-ref-syntax-qry-select-limit.html - text: Set operations url: sql-ref-syntax-qry-select-setops.html + - text: USE database + url: sql-ref-syntax-qry-select-usedb.html - text: Common Table Expression(CTE) url: sql-ref-syntax-qry-select-cte.html - text: Subqueries diff --git a/docs/sql-ref-syntax-qry-select-usedb.md b/docs/sql-ref-syntax-qry-select-usedb.md new file mode 100644 index 0000000000000..92ac91ac51769 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-usedb.md @@ -0,0 +1,60 @@ +--- +layout: global +title: USE Database +displayTitle: USE Database +license: | + 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. +--- + +### Description +`USE` statement is used to set the current database. After the current database is set, +the unqualified database artifacts such as tables, functions and views that are +referenced by SQLs are resolved from the current database. +The default database name is 'default'. + +### Syntax +{% highlight sql %} +USE database_name +{% endhighlight %} + +### Parameter + +
    +
    database_name
    +
    + Name of the database will be used. If the database does not exist, an exception will be thrown. +
    +
    + +### Example +{% highlight sql %} +-- Use the 'userdb' which exists. +USE userdb; ++---------+--+ +| Result | ++---------+--+ ++---------+--+ + +-- Use the 'userdb1' which doesn't exist +USE userdb1; +Error: org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database 'userdb1' not found;(state=,code=0) +{% endhighlight %} + +### Related statements. +- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) +- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) +- [CREATE TABLE ](sql-ref-syntax-ddl-create-table.html) + From 2c775f418f5fae4dbf3adfbb5ea99cd030918d41 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 19 Sep 2019 13:32:09 -0700 Subject: [PATCH 0124/1387] [SPARK-28612][SQL] Add DataFrameWriterV2 API ## What changes were proposed in this pull request? This adds a new write API as proposed in the [SPIP to standardize logical plans](https://issues.apache.org/jira/browse/SPARK-23521). This new API: * Uses clear verbs to execute writes, like `append`, `overwrite`, `create`, and `replace` that correspond to the new logical plans. * Only creates v2 logical plans so the behavior is always consistent. * Does not allow table configuration options for operations that cannot change table configuration. For example, `partitionedBy` can only be called when the writer executes `create` or `replace`. Here are a few example uses of the new API: ```scala df.writeTo("catalog.db.table").append() df.writeTo("catalog.db.table").overwrite($"date" === "2019-06-01") df.writeTo("catalog.db.table").overwritePartitions() df.writeTo("catalog.db.table").asParquet.create() df.writeTo("catalog.db.table").partitionedBy(days($"ts")).createOrReplace() df.writeTo("catalog.db.table").using("abc").replace() ``` ## How was this patch tested? Added `DataFrameWriterV2Suite` that tests the new write API. Existing tests for v2 plans. Closes #25681 from rdblue/SPARK-28612-add-data-frame-writer-v2. Authored-by: Ryan Blue Signed-off-by: Burak Yavuz --- .../expressions/PartitionTransforms.scala | 77 +++ .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../plans/logical/basicLogicalOperators.scala | 47 +- .../v2/DataSourceV2Implicits.scala | 9 + .../spark/sql/connector/InMemoryTable.scala | 5 +- .../apache/spark/sql/DataFrameWriter.scala | 11 +- .../apache/spark/sql/DataFrameWriterV2.scala | 367 +++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 28 + .../datasources/v2/DataSourceV2Strategy.scala | 17 +- .../datasources/v2/TableCapabilityCheck.scala | 6 +- .../org/apache/spark/sql/functions.scala | 58 ++ .../spark/sql/JavaDataFrameWriterV2Suite.java | 112 ++++ .../spark/sql/DataFrameWriterV2Suite.scala | 507 ++++++++++++++++++ 13 files changed, 1217 insertions(+), 33 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala new file mode 100644 index 0000000000000..e48fd8adaef09 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.types.{DataType, IntegerType} + +/** + * Base class for expressions that are converted to v2 partition transforms. + * + * Subclasses represent abstract transform functions with concrete implementations that are + * determined by data source implementations. Because the concrete implementation is not known, + * these expressions are [[Unevaluable]]. + * + * These expressions are used to pass transformations from the DataFrame API: + * + * {{{ + * df.writeTo("catalog.db.table").partitionedBy($"category", days($"timestamp")).create() + * }}} + */ +abstract class PartitionTransformExpression extends Expression with Unevaluable { + override def nullable: Boolean = true +} + +/** + * Expression for the v2 partition transform years. + */ +case class Years(child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(child) +} + +/** + * Expression for the v2 partition transform months. + */ +case class Months(child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(child) +} + +/** + * Expression for the v2 partition transform days. + */ +case class Days(child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(child) +} + +/** + * Expression for the v2 partition transform hours. + */ +case class Hours(child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(child) +} + +/** + * Expression for the v2 partition transform bucket. + */ +case class Bucket(numBuckets: Literal, child: Expression) extends PartitionTransformExpression { + override def dataType: DataType = IntegerType + override def children: Seq[Expression] = Seq(numBuckets, child) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 76e59fa785426..7c81185388d02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2564,7 +2564,7 @@ class Analyzer( */ object ResolveOutputRelation extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { - case append @ AppendData(table, query, isByName) + case append @ AppendData(table, query, _, isByName) if table.resolved && query.resolved && !append.outputResolved => val projection = TableOutputResolver.resolveOutputColumns( @@ -2576,7 +2576,7 @@ class Analyzer( append } - case overwrite @ OverwriteByExpression(table, _, query, isByName) + case overwrite @ OverwriteByExpression(table, _, query, _, isByName) if table.resolved && query.resolved && !overwrite.outputResolved => val projection = TableOutputResolver.resolveOutputColumns( @@ -2588,7 +2588,7 @@ class Analyzer( overwrite } - case overwrite @ OverwritePartitionsDynamic(table, query, isByName) + case overwrite @ OverwritePartitionsDynamic(table, query, _, isByName) if table.resolved && query.resolved && !overwrite.outputResolved => val projection = TableOutputResolver.resolveOutputColumns( 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 2b6378e7c7268..9356d7707104e 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 @@ -488,7 +488,7 @@ case class ReplaceTableAsSelect( override def tableSchema: StructType = query.schema override def children: Seq[LogicalPlan] = Seq(query) - override lazy val resolved: Boolean = { + override lazy val resolved: Boolean = childrenResolved && { // the table schema is created from the query schema, so the only resolution needed is to check // that the columns referenced by the table's partitioning exist in the query schema val references = partitioning.flatMap(_.references).toSet @@ -506,15 +506,22 @@ case class ReplaceTableAsSelect( case class AppendData( table: NamedRelation, query: LogicalPlan, + writeOptions: Map[String, String], isByName: Boolean) extends V2WriteCommand object AppendData { - def byName(table: NamedRelation, df: LogicalPlan): AppendData = { - new AppendData(table, df, isByName = true) + def byName( + table: NamedRelation, + df: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): AppendData = { + new AppendData(table, df, writeOptions, isByName = true) } - def byPosition(table: NamedRelation, query: LogicalPlan): AppendData = { - new AppendData(table, query, isByName = false) + def byPosition( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): AppendData = { + new AppendData(table, query, writeOptions, isByName = false) } } @@ -525,19 +532,26 @@ case class OverwriteByExpression( table: NamedRelation, deleteExpr: Expression, query: LogicalPlan, + writeOptions: Map[String, String], isByName: Boolean) extends V2WriteCommand { override lazy val resolved: Boolean = outputResolved && deleteExpr.resolved } object OverwriteByExpression { def byName( - table: NamedRelation, df: LogicalPlan, deleteExpr: Expression): OverwriteByExpression = { - OverwriteByExpression(table, deleteExpr, df, isByName = true) + table: NamedRelation, + df: LogicalPlan, + deleteExpr: Expression, + writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, df, writeOptions, isByName = true) } def byPosition( - table: NamedRelation, query: LogicalPlan, deleteExpr: Expression): OverwriteByExpression = { - OverwriteByExpression(table, deleteExpr, query, isByName = false) + table: NamedRelation, + query: LogicalPlan, + deleteExpr: Expression, + writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, query, writeOptions, isByName = false) } } @@ -547,15 +561,22 @@ object OverwriteByExpression { case class OverwritePartitionsDynamic( table: NamedRelation, query: LogicalPlan, + writeOptions: Map[String, String], isByName: Boolean) extends V2WriteCommand object OverwritePartitionsDynamic { - def byName(table: NamedRelation, df: LogicalPlan): OverwritePartitionsDynamic = { - OverwritePartitionsDynamic(table, df, isByName = true) + def byName( + table: NamedRelation, + df: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, df, writeOptions, isByName = true) } - def byPosition(table: NamedRelation, query: LogicalPlan): OverwritePartitionsDynamic = { - OverwritePartitionsDynamic(table, query, isByName = false) + def byPosition( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, query, writeOptions, isByName = false) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index 74842e762b401..86ef867eca547 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql.execution.datasources.v2 +import scala.collection.JavaConverters._ + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Implicits { implicit class TableHelper(table: Table) { @@ -53,4 +56,10 @@ object DataSourceV2Implicits { def supportsAny(capabilities: TableCapability*): Boolean = capabilities.exists(supports) } + + implicit class OptionsHelper(options: Map[String, String]) { + def asOptions: CaseInsensitiveStringMap = { + new CaseInsensitiveStringMap(options.asJava) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 04ae27df94535..414f9d5834868 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -41,8 +41,11 @@ class InMemoryTable( override val properties: util.Map[String, String]) extends Table with SupportsRead with SupportsWrite with SupportsDelete { + private val allowUnsupportedTransforms = + properties.getOrDefault("allow-unsupported-transforms", "false").toBoolean + partitioning.foreach { t => - if (!t.isInstanceOf[IdentityTransform]) { + if (!t.isInstanceOf[IdentityTransform] && !allowUnsupportedTransforms) { throw new IllegalArgumentException(s"Transform $t must be IdentityTransform") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 58acfb836b305..289a976c6db9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -271,13 +271,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { modeForDSV2 match { case SaveMode.Append => runCommand(df.sparkSession, "save") { - AppendData.byName(relation, df.logicalPlan) + AppendData.byName(relation, df.logicalPlan, extraOptions.toMap) } case SaveMode.Overwrite if table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER) => // truncate the table runCommand(df.sparkSession, "save") { - OverwriteByExpression.byName(relation, df.logicalPlan, Literal(true)) + OverwriteByExpression.byName( + relation, df.logicalPlan, Literal(true), extraOptions.toMap) } case other => @@ -382,7 +383,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val command = modeForDSV2 match { case SaveMode.Append => - AppendData.byPosition(table, df.logicalPlan) + AppendData.byPosition(table, df.logicalPlan, extraOptions.toMap) case SaveMode.Overwrite => val conf = df.sparkSession.sessionState.conf @@ -390,9 +391,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { conf.partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC if (dynamicPartitionOverwrite) { - OverwritePartitionsDynamic.byPosition(table, df.logicalPlan) + OverwritePartitionsDynamic.byPosition(table, df.logicalPlan, extraOptions.toMap) } else { - OverwriteByExpression.byPosition(table, df.logicalPlan, Literal(true)) + OverwriteByExpression.byPosition(table, df.logicalPlan, Literal(true), extraOptions.toMap) } case other => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala new file mode 100644 index 0000000000000..d3ab33919a7bb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -0,0 +1,367 @@ +/* + * 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 + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Bucket, Days, Hours, Literal, Months, Years} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} +import org.apache.spark.sql.connector.expressions.{LogicalExpressions, Transform} +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.types.IntegerType + +/** + * Interface used to write a [[org.apache.spark.sql.Dataset]] to external storage using the v2 API. + * + * @since 3.0.0 + */ +@Experimental +final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) + extends CreateTableWriter[T] { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ + import df.sparkSession.sessionState.analyzer.CatalogObjectIdentifier + + private val df: DataFrame = ds.toDF() + + private val sparkSession = ds.sparkSession + + private val catalogManager = sparkSession.sessionState.analyzer.catalogManager + + private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) + + private val (catalog, identifier) = { + val CatalogObjectIdentifier(maybeCatalog, identifier) = tableName + val catalog = maybeCatalog.orElse(catalogManager.currentCatalog.map(catalogManager.catalog)) + .getOrElse(throw new AnalysisException( + s"No catalog specified for table ${identifier.quoted} and no default v2 catalog is set")) + .asTableCatalog + + (catalog, identifier) + } + + private val logicalPlan = df.queryExecution.logical + + private var provider: Option[String] = None + + private val options = new mutable.HashMap[String, String]() + + private val properties = new mutable.HashMap[String, String]() + + private var partitioning: Option[Seq[Transform]] = None + + override def using(provider: String): CreateTableWriter[T] = { + this.provider = Some(provider) + this + } + + override def option(key: String, value: String): DataFrameWriterV2[T] = { + this.options.put(key, value) + this + } + + override def options(options: scala.collection.Map[String, String]): DataFrameWriterV2[T] = { + options.foreach { + case (key, value) => + this.options.put(key, value) + } + this + } + + override def options(options: java.util.Map[String, String]): DataFrameWriterV2[T] = { + this.options(options.asScala) + this + } + + override def tableProperty(property: String, value: String): DataFrameWriterV2[T] = { + this.properties.put(property, value) + this + } + + @scala.annotation.varargs + override def partitionedBy(column: Column, columns: Column*): CreateTableWriter[T] = { + val asTransforms = (column +: columns).map(_.expr).map { + case Years(attr: Attribute) => + LogicalExpressions.years(attr.name) + case Months(attr: Attribute) => + LogicalExpressions.months(attr.name) + case Days(attr: Attribute) => + LogicalExpressions.days(attr.name) + case Hours(attr: Attribute) => + LogicalExpressions.hours(attr.name) + case Bucket(Literal(numBuckets: Int, IntegerType), attr: Attribute) => + LogicalExpressions.bucket(numBuckets, attr.name) + case attr: Attribute => + LogicalExpressions.identity(attr.name) + case expr => + throw new AnalysisException(s"Invalid partition transformation: ${expr.sql}") + } + + this.partitioning = Some(asTransforms) + this + } + + override def create(): Unit = { + // create and replace could alternatively create ParsedPlan statements, like + // `CreateTableFromDataFrameStatement(UnresolvedRelation(tableName), ...)`, to keep the catalog + // resolution logic in the analyzer. + runCommand("create") { + CreateTableAsSelect( + catalog, + identifier, + partitioning.getOrElse(Seq.empty), + logicalPlan, + properties = provider.map(p => properties + ("provider" -> p)).getOrElse(properties).toMap, + writeOptions = options.toMap, + ignoreIfExists = false) + } + } + + override def replace(): Unit = { + internalReplace(orCreate = false) + } + + override def createOrReplace(): Unit = { + internalReplace(orCreate = true) + } + + + /** + * Append the contents of the data frame to the output table. + * + * If the output table does not exist, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. The data frame will be + * validated to ensure it is compatible with the existing table. + * + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + */ + @throws(classOf[NoSuchTableException]) + def append(): Unit = { + val append = loadTable(catalog, identifier) match { + case Some(t) => + AppendData.byName(DataSourceV2Relation.create(t), logicalPlan, options.toMap) + case _ => + throw new NoSuchTableException(identifier) + } + + runCommand("append")(append) + } + + /** + * Overwrite rows matching the given filter condition with the contents of the data frame in + * the output table. + * + * If the output table does not exist, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. + * The data frame will be validated to ensure it is compatible with the existing table. + * + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + */ + @throws(classOf[NoSuchTableException]) + def overwrite(condition: Column): Unit = { + val overwrite = loadTable(catalog, identifier) match { + case Some(t) => + OverwriteByExpression.byName( + DataSourceV2Relation.create(t), logicalPlan, condition.expr, options.toMap) + case _ => + throw new NoSuchTableException(identifier) + } + + runCommand("overwrite")(overwrite) + } + + /** + * Overwrite all partition for which the data frame contains at least one row with the contents + * of the data frame in the output table. + * + * This operation is equivalent to Hive's `INSERT OVERWRITE ... PARTITION`, which replaces + * partitions dynamically depending on the contents of the data frame. + * + * If the output table does not exist, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.NoSuchTableException]]. The data frame will be + * validated to ensure it is compatible with the existing table. + * + * @throws org.apache.spark.sql.catalyst.analysis.NoSuchTableException If the table does not exist + */ + @throws(classOf[NoSuchTableException]) + def overwritePartitions(): Unit = { + val dynamicOverwrite = loadTable(catalog, identifier) match { + case Some(t) => + OverwritePartitionsDynamic.byName( + DataSourceV2Relation.create(t), logicalPlan, options.toMap) + case _ => + throw new NoSuchTableException(identifier) + } + + runCommand("overwritePartitions")(dynamicOverwrite) + } + + /** + * Wrap an action to track the QueryExecution and time cost, then report to the user-registered + * callback functions. + */ + private def runCommand(name: String)(command: LogicalPlan): Unit = { + val qe = sparkSession.sessionState.executePlan(command) + // call `QueryExecution.toRDD` to trigger the execution of commands. + SQLExecution.withNewExecutionId(sparkSession, qe, Some(name))(qe.toRdd) + } + + private def internalReplace(orCreate: Boolean): Unit = { + runCommand("replace") { + ReplaceTableAsSelect( + catalog, + identifier, + partitioning.getOrElse(Seq.empty), + logicalPlan, + properties = provider.map(p => properties + ("provider" -> p)).getOrElse(properties).toMap, + writeOptions = options.toMap, + orCreate = orCreate) + } + } +} + +/** + * Configuration methods common to create/replace operations and insert/overwrite operations. + * @tparam R builder type to return + */ +trait WriteConfigMethods[R] { + /** + * Add a write option. + * + * @since 3.0.0 + */ + def option(key: String, value: String): R + + /** + * Add a boolean output option. + * + * @since 3.0.0 + */ + def option(key: String, value: Boolean): R = option(key, value.toString) + + /** + * Add a long output option. + * + * @since 3.0.0 + */ + def option(key: String, value: Long): R = option(key, value.toString) + + /** + * Add a double output option. + * + * @since 3.0.0 + */ + def option(key: String, value: Double): R = option(key, value.toString) + + /** + * Add write options from a Scala Map. + * + * @since 3.0.0 + */ + def options(options: scala.collection.Map[String, String]): R + + /** + * Add write options from a Java Map. + * + * @since 3.0.0 + */ + def options(options: java.util.Map[String, String]): R +} + +/** + * Trait to restrict calls to create and replace operations. + */ +trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] { + /** + * Create a new table from the contents of the data frame. + * + * The new table's schema, partition layout, properties, and other configuration will be + * based on the configuration set on this writer. + * + * If the output table exists, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException]]. + * + * @throws org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException + * If the table already exists + */ + @throws(classOf[TableAlreadyExistsException]) + def create(): Unit + + /** + * Replace an existing table with the contents of the data frame. + * + * The existing table's schema, partition layout, properties, and other configuration will be + * replaced with the contents of the data frame and the configuration set on this writer. + * + * If the output table does not exist, this operation will fail with + * [[org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException]]. + * + * @throws org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException + * If the table already exists + */ + @throws(classOf[CannotReplaceMissingTableException]) + def replace(): Unit + + /** + * Create a new table or replace an existing table with the contents of the data frame. + * + * The output table's schema, partition layout, properties, and other configuration will be based + * on the contents of the data frame and the configuration set on this writer. If the table + * exists, its configuration and data will be replaced. + */ + def createOrReplace(): Unit + + /** + * Partition the output table created by `create`, `createOrReplace`, or `replace` using + * the given columns or transforms. + * + * When specified, the table data will be stored by these values for efficient reads. + * + * For example, when a table is partitioned by day, it may be stored in a directory layout like: + *
      + *
    • `table/day=2019-06-01/`
    • + *
    • `table/day=2019-06-02/`
    • + *
    + * + * Partitioning is one of the most widely used techniques to optimize physical data layout. + * It provides a coarse-grained index for skipping unnecessary data reads when queries have + * predicates on the partitioned columns. In order for partitioning to work well, the number + * of distinct values in each column should typically be less than tens of thousands. + * + * @since 3.0.0 + */ + def partitionedBy(column: Column, columns: Column*): CreateTableWriter[T] + + /** + * Specifies a provider for the underlying output data source. Spark's default catalog supports + * "parquet", "json", etc. + * + * @since 3.0.0 + */ + def using(provider: String): CreateTableWriter[T] + + /** + * Add a table property. + */ + def tableProperty(property: String, value: String): CreateTableWriter[T] +} 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 88fe7a3f380ab..a2f5f03d797a2 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 @@ -3120,6 +3120,34 @@ class Dataset[T] private[sql]( new DataFrameWriter[T](this) } + /** + * Create a write configuration builder for v2 sources. + * + * This builder is used to configure and execute write operations. For example, to append to an + * existing table, run: + * + * {{{ + * df.writeTo("catalog.db.table").append() + * }}} + * + * This can also be used to create or replace existing tables: + * + * {{{ + * df.writeTo("catalog.db.table").partitionedBy($"col").createOrReplace() + * }}} + * + * @group basic + * @since 3.0.0 + */ + def writeTo(table: String): DataFrameWriterV2[T] = { + // TODO: streaming could be adapted to use this interface + if (isStreaming) { + logicalPlan.failAnalysis( + "'writeTo' can not be called on streaming Dataset/DataFrame") + } + new DataFrameWriterV2[T](table, this) + } + /** * Interface for saving the content of the streaming Dataset out into external storage. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 6285159dad9b0..6387333d06880 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -226,15 +226,15 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { orCreate = orCreate) :: Nil } - case AppendData(r: DataSourceV2Relation, query, _) => + case AppendData(r: DataSourceV2Relation, query, writeOptions, _) => r.table.asWritable match { case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) => - AppendDataExecV1(v1, r.options, query) :: Nil + AppendDataExecV1(v1, writeOptions.asOptions, query) :: Nil case v2 => - AppendDataExec(v2, r.options, planLater(query)) :: Nil + AppendDataExec(v2, writeOptions.asOptions, planLater(query)) :: Nil } - case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, _) => + case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, writeOptions, _) => // fail if any filter cannot be converted. correctness depends on removing all matching data. val filters = splitConjunctivePredicates(deleteExpr).map { filter => DataSourceStrategy.translateFilter(deleteExpr).getOrElse( @@ -242,13 +242,14 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { }.toArray r.table.asWritable match { case v1 if v1.supports(TableCapability.V1_BATCH_WRITE) => - OverwriteByExpressionExecV1(v1, filters, r.options, query) :: Nil + OverwriteByExpressionExecV1(v1, filters, writeOptions.asOptions, query) :: Nil case v2 => - OverwriteByExpressionExec(v2, filters, r.options, planLater(query)) :: Nil + OverwriteByExpressionExec(v2, filters, writeOptions.asOptions, planLater(query)) :: Nil } - case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _) => - OverwritePartitionsDynamicExec(r.table.asWritable, r.options, planLater(query)) :: Nil + case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, writeOptions, _) => + OverwritePartitionsDynamicExec( + r.table.asWritable, writeOptions.asOptions, planLater(query)) :: Nil case DeleteFromTable(r: DataSourceV2Relation, condition) => if (condition.exists(SubqueryExpression.hasSubquery)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala index 51993b1e9ad60..45ca3dfb9cb93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala @@ -44,14 +44,14 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) { // TODO: check STREAMING_WRITE capability. It's not doable now because we don't have a // a logical plan for streaming write. - case AppendData(r: DataSourceV2Relation, _, _) if !r.table.supports(BATCH_WRITE) => + case AppendData(r: DataSourceV2Relation, _, _, _) if !r.table.supports(BATCH_WRITE) => failAnalysis(s"Table ${r.table.name()} does not support append in batch mode.") - case OverwritePartitionsDynamic(r: DataSourceV2Relation, _, _) + case OverwritePartitionsDynamic(r: DataSourceV2Relation, _, _, _) if !r.table.supports(BATCH_WRITE) || !r.table.supports(OVERWRITE_DYNAMIC) => failAnalysis(s"Table ${r.table.name()} does not support dynamic overwrite in batch mode.") - case OverwriteByExpression(r: DataSourceV2Relation, expr, _, _) => + case OverwriteByExpression(r: DataSourceV2Relation, expr, _, _, _) => expr match { case Literal(true, BooleanType) => if (!r.table.supports(BATCH_WRITE) || diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 84e0eaff2d428..a141b5e6423aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -69,6 +69,7 @@ import org.apache.spark.util.Utils * @groupname window_funcs Window functions * @groupname string_funcs String functions * @groupname collection_funcs Collection functions + * @groupname partition_transforms Partition transform functions * @groupname Ungrouped Support functions for DataFrames * @since 1.3.0 */ @@ -3942,6 +3943,63 @@ object functions { */ def to_csv(e: Column): Column = to_csv(e, Map.empty[String, String].asJava) + /** + * A transform for timestamps and dates to partition data into years. + * + * @group partition_transforms + * @since 3.0.0 + */ + def years(e: Column): Column = withExpr { Years(e.expr) } + + /** + * A transform for timestamps and dates to partition data into months. + * + * @group partition_transforms + * @since 3.0.0 + */ + def months(e: Column): Column = withExpr { Months(e.expr) } + + /** + * A transform for timestamps and dates to partition data into days. + * + * @group partition_transforms + * @since 3.0.0 + */ + def days(e: Column): Column = withExpr { Days(e.expr) } + + /** + * A transform for timestamps to partition data into hours. + * + * @group partition_transforms + * @since 3.0.0 + */ + def hours(e: Column): Column = withExpr { Hours(e.expr) } + + /** + * A transform for any type that partitions by a hash of the input column. + * + * @group partition_transforms + * @since 3.0.0 + */ + def bucket(numBuckets: Column, e: Column): Column = withExpr { + numBuckets.expr match { + case lit @ Literal(_, IntegerType) => + Bucket(lit, e.expr) + case _ => + throw new AnalysisException(s"Invalid number of buckets: bucket($numBuckets, $e)") + } + } + + /** + * A transform for any type that partitions by a hash of the input column. + * + * @group partition_transforms + * @since 3.0.0 + */ + def bucket(numBuckets: Int, e: Column): Column = withExpr { + Bucket(Literal(numBuckets), e.expr) + } + // scalastyle:off line.size.limit // scalastyle:off parameter.number diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java new file mode 100644 index 0000000000000..72f2e8f185637 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameWriterV2Suite.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.connector.InMemoryTableCatalog; +import org.apache.spark.sql.test.TestSparkSession; +import org.apache.spark.sql.types.StructType; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.apache.spark.sql.functions.*; + +public class JavaDataFrameWriterV2Suite { + private static StructType schema = new StructType().add("s", "string"); + private SparkSession spark = null; + + public Dataset df() { + return spark.read().schema(schema).text(); + } + + @Before + public void createTestTable() { + this.spark = new TestSparkSession(); + spark.conf().set("spark.sql.catalog.testcat", InMemoryTableCatalog.class.getName()); + spark.sql("CREATE TABLE testcat.t (s string) USING foo"); + } + + @After + public void dropTestTable() { + spark.sql("DROP TABLE testcat.t"); + spark.stop(); + } + + @Test + public void testAppendAPI() throws NoSuchTableException { + df().writeTo("testcat.t").append(); + df().writeTo("testcat.t").option("property", "value").append(); + df().writeTo("testcat.t").tableProperty("property", "value").append(); + } + + @Test + public void testOverwritePartitionsAPI() throws NoSuchTableException { + df().writeTo("testcat.t").overwritePartitions(); + df().writeTo("testcat.t").option("property", "value").overwritePartitions(); + df().writeTo("testcat.t").tableProperty("property", "value").overwritePartitions(); + } + + @Test + public void testOverwriteAPI() throws NoSuchTableException { + df().writeTo("testcat.t").overwrite(lit(true)); + df().writeTo("testcat.t").option("property", "value").overwrite(lit(true)); + df().writeTo("testcat.t").tableProperty("property", "value").overwrite(lit(true)); + } + + @Test + public void testCreateAPI() throws TableAlreadyExistsException { + df().writeTo("testcat.t2").create(); + spark.sql("DROP TABLE testcat.t2"); + + df().writeTo("testcat.t2").option("property", "value").create(); + spark.sql("DROP TABLE testcat.t2"); + + df().writeTo("testcat.t2").tableProperty("property", "value").create(); + spark.sql("DROP TABLE testcat.t2"); + + df().writeTo("testcat.t2").using("v2format").create(); + spark.sql("DROP TABLE testcat.t2"); + + df().writeTo("testcat.t2").partitionedBy(col("s")).create(); + spark.sql("DROP TABLE testcat.t2"); + } + + @Test + public void testReplaceAPI() throws CannotReplaceMissingTableException { + df().writeTo("testcat.t").replace(); + df().writeTo("testcat.t").option("property", "value").replace(); + df().writeTo("testcat.t").tableProperty("property", "value").replace(); + df().writeTo("testcat.t").using("v2format").replace(); + df().writeTo("testcat.t").partitionedBy(col("s")).replace(); + } + + @Test + public void testCreateOrReplaceAPI() { + df().writeTo("testcat.t").createOrReplace(); + df().writeTo("testcat.t").option("property", "value").createOrReplace(); + df().writeTo("testcat.t").tableProperty("property", "value").createOrReplace(); + df().writeTo("testcat.t").using("v2format").createOrReplace(); + df().writeTo("testcat.t").partitionedBy(col("s")).createOrReplace(); + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala new file mode 100644 index 0000000000000..ce0a5f21fd7ec --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -0,0 +1,507 @@ +/* + * 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 + +import scala.collection.JavaConverters._ + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} + +class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with BeforeAndAfter { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.functions._ + import testImplicits._ + + private def catalog(name: String): TableCatalog = { + spark.sessionState.catalogManager.catalog(name).asTableCatalog + } + + before { + spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName) + + val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") + df.createOrReplaceTempView("source") + val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data") + df2.createOrReplaceTempView("source2") + } + + after { + spark.sessionState.catalogManager.reset() + spark.sessionState.conf.clear() + } + + test("Append: basic append") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"), Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + } + + test("Append: by name not position") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + val exc = intercept[AnalysisException] { + spark.table("source").withColumnRenamed("data", "d").writeTo("testcat.table_name").append() + } + + assert(exc.getMessage.contains("Cannot find data for output column")) + assert(exc.getMessage.contains("'data'")) + + checkAnswer( + spark.table("testcat.table_name"), + Seq()) + } + + test("Append: fail if table does not exist") { + val exc = intercept[NoSuchTableException] { + spark.table("source").writeTo("testcat.table_name").append() + } + + assert(exc.getMessage.contains("table_name")) + } + + test("Overwrite: overwrite by expression: true") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").writeTo("testcat.table_name").overwrite(lit(true)) + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + } + + test("Overwrite: overwrite by expression: id = 3") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").writeTo("testcat.table_name").overwrite($"id" === 3) + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + } + + test("Overwrite: by name not position") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + val exc = intercept[AnalysisException] { + spark.table("source").withColumnRenamed("data", "d") + .writeTo("testcat.table_name").overwrite(lit(true)) + } + + assert(exc.getMessage.contains("Cannot find data for output column")) + assert(exc.getMessage.contains("'data'")) + + checkAnswer( + spark.table("testcat.table_name"), + Seq()) + } + + test("Overwrite: fail if table does not exist") { + val exc = intercept[NoSuchTableException] { + spark.table("source").writeTo("testcat.table_name").overwrite(lit(true)) + } + + assert(exc.getMessage.contains("table_name")) + } + + test("OverwritePartitions: overwrite conflicting partitions") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").withColumn("id", $"id" - 2) + .writeTo("testcat.table_name").overwritePartitions() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "d"), Row(3L, "e"), Row(4L, "f"))) + } + + test("OverwritePartitions: overwrite all rows if not partitioned") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + spark.table("source").writeTo("testcat.table_name").append() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + spark.table("source2").writeTo("testcat.table_name").overwritePartitions() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + } + + test("OverwritePartitions: by name not position") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + checkAnswer(spark.table("testcat.table_name"), Seq.empty) + + val exc = intercept[AnalysisException] { + spark.table("source").withColumnRenamed("data", "d") + .writeTo("testcat.table_name").overwritePartitions() + } + + assert(exc.getMessage.contains("Cannot find data for output column")) + assert(exc.getMessage.contains("'data'")) + + checkAnswer( + spark.table("testcat.table_name"), + Seq()) + } + + test("OverwritePartitions: fail if table does not exist") { + val exc = intercept[NoSuchTableException] { + spark.table("source").writeTo("testcat.table_name").overwritePartitions() + } + + assert(exc.getMessage.contains("table_name")) + } + + test("Create: basic behavior") { + spark.table("source").writeTo("testcat.table_name").create() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning.isEmpty) + assert(table.properties.isEmpty) + } + + test("Create: with using") { + spark.table("source").writeTo("testcat.table_name").using("foo").create() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning.isEmpty) + assert(table.properties === Map("provider" -> "foo").asJava) + } + + test("Create: with property") { + spark.table("source").writeTo("testcat.table_name").tableProperty("prop", "value").create() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning.isEmpty) + assert(table.properties === Map("prop" -> "value").asJava) + } + + test("Create: identity partitioned table") { + spark.table("source").writeTo("testcat.table_name").partitionedBy($"id").create() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(table.properties.isEmpty) + } + + test("Create: partitioned by years(ts)") { + spark.table("source") + .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(years($"ts")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(YearsTransform(FieldReference("ts")))) + } + + test("Create: partitioned by months(ts)") { + spark.table("source") + .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(months($"ts")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(MonthsTransform(FieldReference("ts")))) + } + + test("Create: partitioned by days(ts)") { + spark.table("source") + .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(days($"ts")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(DaysTransform(FieldReference("ts")))) + } + + test("Create: partitioned by hours(ts)") { + spark.table("source") + .withColumn("ts", lit("2019-06-01 10:00:00.000000").cast("timestamp")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(hours($"ts")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(HoursTransform(FieldReference("ts")))) + } + + test("Create: partitioned by bucket(4, id)") { + spark.table("source") + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(bucket(4, $"id")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === + Seq(BucketTransform(LiteralValue(4, IntegerType), Seq(FieldReference("id"))))) + } + + test("Create: fail if table already exists") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + + val exc = intercept[TableAlreadyExistsException] { + spark.table("source").writeTo("testcat.table_name").create() + } + + assert(exc.getMessage.contains("table_name")) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // table should not have been changed + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(table.properties === Map("provider" -> "foo").asJava) + } + + test("Replace: basic behavior") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + spark.sql("INSERT INTO TABLE testcat.table_name SELECT * FROM source") + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the initial table + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(table.properties === Map("provider" -> "foo").asJava) + + spark.table("source2") + .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) + .writeTo("testcat.table_name").replace() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d", "even"), Row(5L, "e", "odd"), Row(6L, "f", "even"))) + + val replaced = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the replacement table + assert(replaced.name === "testcat.table_name") + assert(replaced.schema === new StructType() + .add("id", LongType) + .add("data", StringType) + .add("even_or_odd", StringType)) + assert(replaced.partitioning.isEmpty) + assert(replaced.properties.isEmpty) + } + + test("Replace: partitioned table") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + spark.sql("INSERT INTO TABLE testcat.table_name SELECT * FROM source") + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the initial table + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning.isEmpty) + assert(table.properties === Map("provider" -> "foo").asJava) + + spark.table("source2") + .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) + .writeTo("testcat.table_name").partitionedBy($"id").replace() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d", "even"), Row(5L, "e", "odd"), Row(6L, "f", "even"))) + + val replaced = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the replacement table + assert(replaced.name === "testcat.table_name") + assert(replaced.schema === new StructType() + .add("id", LongType) + .add("data", StringType) + .add("even_or_odd", StringType)) + assert(replaced.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(replaced.properties.isEmpty) + } + + test("Replace: fail if table does not exist") { + val exc = intercept[CannotReplaceMissingTableException] { + spark.table("source").writeTo("testcat.table_name").replace() + } + + assert(exc.getMessage.contains("table_name")) + } + + test("CreateOrReplace: table does not exist") { + spark.table("source2").writeTo("testcat.table_name").createOrReplace() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d"), Row(5L, "e"), Row(6L, "f"))) + + val replaced = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the replacement table + assert(replaced.name === "testcat.table_name") + assert(replaced.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(replaced.partitioning.isEmpty) + assert(replaced.properties.isEmpty) + } + + test("CreateOrReplace: table exists") { + spark.sql( + "CREATE TABLE testcat.table_name (id bigint, data string) USING foo PARTITIONED BY (id)") + spark.sql("INSERT INTO TABLE testcat.table_name SELECT * FROM source") + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(1L, "a"), Row(2L, "b"), Row(3L, "c"))) + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the initial table + assert(table.name === "testcat.table_name") + assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) + assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) + assert(table.properties === Map("provider" -> "foo").asJava) + + spark.table("source2") + .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) + .writeTo("testcat.table_name").createOrReplace() + + checkAnswer( + spark.table("testcat.table_name"), + Seq(Row(4L, "d", "even"), Row(5L, "e", "odd"), Row(6L, "f", "even"))) + + val replaced = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + // validate the replacement table + assert(replaced.name === "testcat.table_name") + assert(replaced.schema === new StructType() + .add("id", LongType) + .add("data", StringType) + .add("even_or_odd", StringType)) + assert(replaced.partitioning.isEmpty) + assert(replaced.properties.isEmpty) + } +} From 5b478416f8e3fe2f015af1b6c8faa7fe9f15c05d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 Sep 2019 15:33:12 -0700 Subject: [PATCH 0125/1387] [SPARK-28208][SQL][FOLLOWUP] Use `tryWithResource` pattern ### What changes were proposed in this pull request? This PR aims to use `tryWithResource` for ORC file. ### Why are the changes needed? This is a follow-up to address https://github.com/apache/spark/pull/25006#discussion_r298788206 . ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #25842 from dongjoon-hyun/SPARK-28208. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../datasources/orc/OrcFileFormat.scala | 13 ++++--- .../execution/datasources/orc/OrcUtils.scala | 8 ++--- .../v2/orc/OrcPartitionReaderFactory.scala | 22 ++++++------ .../datasources/orc/OrcQuerySuite.scala | 36 +++++++++---------- .../datasources/orc/OrcSourceSuite.scala | 8 ++--- .../sql/hive/execution/HiveDDLSuite.scala | 10 +++--- 6 files changed, 48 insertions(+), 49 deletions(-) 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 59ee63aeab269..12c81a1dcab56 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 @@ -38,10 +38,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.{SerializableConfiguration, Utils} private[sql] object OrcFileFormat { private def checkFieldName(name: String): Unit = { @@ -180,11 +179,11 @@ class OrcFileFormat val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(filePath, readerOptions) - - val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, requiredSchema, reader, conf) - reader.close() + val requestedColIdsOrEmptyFile = + Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => + OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, requiredSchema, reader, conf) + } if (requestedColIdsOrEmptyFile.isEmpty) { Iterator.empty 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 index f3c92f32a3694..eea9b2a8f9613 100644 --- 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 @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.execution.datasources.SchemaMergeUtils import org.apache.spark.sql.types._ -import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} +import org.apache.spark.util.{ThreadUtils, Utils} object OrcUtils extends Logging { @@ -62,9 +62,9 @@ object OrcUtils extends Logging { val fs = file.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) try { - val reader = OrcFile.createReader(file, readerOptions) - val schema = reader.getSchema - reader.close() + val schema = Utils.tryWithResource(OrcFile.createReader(file, readerOptions)) { reader => + reader.getSchema + } if (schema.getFieldNames.size == 0) { None } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala index 7780ce45e7c69..03d58fdcb7f67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{AtomicType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.{SerializableConfiguration, Utils} /** * A factory used to create Orc readers. @@ -74,11 +74,11 @@ case class OrcPartitionReaderFactory( val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(filePath, readerOptions) - - val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, readDataSchema, reader, conf) - reader.close() + val requestedColIdsOrEmptyFile = + Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => + OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readDataSchema, reader, conf) + } if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader[InternalRow] @@ -120,11 +120,11 @@ case class OrcPartitionReaderFactory( val fs = filePath.getFileSystem(conf) val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) - val reader = OrcFile.createReader(filePath, readerOptions) - - val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( - isCaseSensitive, dataSchema, readDataSchema, reader, conf) - reader.close() + val requestedColIdsOrEmptyFile = + Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions)) { reader => + OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readDataSchema, reader, conf) + } if (requestedColIdsOrEmptyFile.isEmpty) { new EmptyPartitionReader diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index c334d0116ebbb..b8bf4b16fe53c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -194,9 +194,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("ZLIB" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("ZLIB" === reader.getCompressionKind.name) + } } // `compression` overrides `orc.compress`. @@ -211,9 +211,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("ZLIB" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("ZLIB" === reader.getCompressionKind.name) + } } } @@ -229,9 +229,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("ZLIB" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("ZLIB" === reader.getCompressionKind.name) + } } withTempPath { file => @@ -244,9 +244,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("SNAPPY" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("SNAPPY" === reader.getCompressionKind.name) + } } withTempPath { file => @@ -259,9 +259,9 @@ abstract class OrcQueryTest extends OrcTest { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("NONE" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("NONE" === reader.getCompressionKind.name) + } } } @@ -645,9 +645,9 @@ class OrcQuerySuite extends OrcQueryTest with SharedSparkSession { val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath) val conf = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, conf) - assert("LZO" === reader.getCompressionKind.name) - reader.close() + Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader => + assert("LZO" === reader.getCompressionKind.name) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 70d15bb195d53..a2d96dd0468b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -372,10 +372,10 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { val orcFilePath = new Path(partFiles.head.getAbsolutePath) val readerOptions = OrcFile.readerOptions(new Configuration()) - val reader = OrcFile.createReader(orcFilePath, readerOptions) - val version = UTF_8.decode(reader.getMetadataValue(SPARK_VERSION_METADATA_KEY)).toString - reader.close() - assert(version === SPARK_VERSION_SHORT) + Utils.tryWithResource(OrcFile.createReader(orcFilePath, readerOptions)) { reader => + val version = UTF_8.decode(reader.getMetadataValue(SPARK_VERSION_METADATA_KEY)).toString + assert(version === SPARK_VERSION_SHORT) + } } } 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 487e888d588de..4253fe2e1edcb 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 @@ -2366,11 +2366,11 @@ class HiveDDLSuite checkAnswer(spark.table("t"), Row(1)) val maybeFile = path.listFiles().find(_.getName.startsWith("part")) - val reader = getReader(maybeFile.head.getCanonicalPath) - assert(reader.getCompressionKind.name === "ZLIB") - assert(reader.getCompressionSize == 1001) - assert(reader.getRowIndexStride == 2002) - reader.close() + Utils.tryWithResource(getReader(maybeFile.head.getCanonicalPath)) { reader => + assert(reader.getCompressionKind.name === "ZLIB") + assert(reader.getCompressionSize == 1001) + assert(reader.getRowIndexStride == 2002) + } } } } From 76ebf2241a3f2149de13d6c89adcb86325b06004 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 19 Sep 2019 17:54:42 -0700 Subject: [PATCH 0126/1387] Revert "[SPARK-29082][CORE] Skip delegation token generation if no credentials are available" This reverts commit f32f16fd68f51a94d6adb2d01a9b2e557885e656. --- .../HadoopDelegationTokenManager.scala | 22 ++++------- .../CoarseGrainedSchedulerBackend.scala | 10 ++--- .../HadoopDelegationTokenManagerSuite.scala | 39 +------------------ 3 files changed, 12 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index f769ce468e49c..759d857d56e0e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -140,21 +140,13 @@ private[spark] class HadoopDelegationTokenManager( * @param creds Credentials object where to store the delegation tokens. */ def obtainDelegationTokens(creds: Credentials): Unit = { - val currentUser = UserGroupInformation.getCurrentUser() - val hasKerberosCreds = principal != null || - Option(currentUser.getRealUser()).getOrElse(currentUser).hasKerberosCredentials() - - // Delegation tokens can only be obtained if the real user has Kerberos credentials, so - // skip creation when those are not available. - if (hasKerberosCreds) { - val freshUGI = doLogin() - freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { - override def run(): Unit = { - val (newTokens, _) = obtainDelegationTokens() - creds.addAll(newTokens) - } - }) - } + val freshUGI = doLogin() + freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val (newTokens, _) = obtainDelegationTokens() + creds.addAll(newTokens) + } + }) } /** 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 c9b408bed1163..d81070c362ba6 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 @@ -427,14 +427,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val ugi = UserGroupInformation.getCurrentUser() val tokens = if (dtm.renewalEnabled) { dtm.start() - } else { + } else if (ugi.hasKerberosCredentials() || SparkHadoopUtil.get.isProxyUser(ugi)) { val creds = ugi.getCredentials() dtm.obtainDelegationTokens(creds) - if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { - SparkHadoopUtil.get.serialize(creds) - } else { - null - } + SparkHadoopUtil.get.serialize(creds) + } else { + null } if (tokens != null) { updateDelegationTokens(tokens) diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index bf53386da6304..70174f7ff939a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -17,14 +17,10 @@ package org.apache.spark.deploy.security -import java.security.PrivilegedExceptionAction - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION -import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.Credentials import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.security.HadoopDelegationTokenProvider private class ExceptionThrowingDelegationTokenProvider extends HadoopDelegationTokenProvider { @@ -73,37 +69,4 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { assert(!manager.isProviderLoaded("hadoopfs")) assert(manager.isProviderLoaded("hbase")) } - - test("SPARK-29082: do not fail if current user does not have credentials") { - // SparkHadoopUtil overrides the UGI configuration during initialization. That normally - // happens early in the Spark application, but here it may affect the test depending on - // how it's run, so force its initialization. - SparkHadoopUtil.get - - val krbConf = new Configuration() - krbConf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos") - - UserGroupInformation.setConfiguration(krbConf) - try { - val manager = new HadoopDelegationTokenManager(new SparkConf(false), krbConf, null) - val testImpl = new PrivilegedExceptionAction[Unit] { - override def run(): Unit = { - assert(UserGroupInformation.isSecurityEnabled()) - val creds = new Credentials() - manager.obtainDelegationTokens(creds) - assert(creds.numberOfTokens() === 0) - assert(creds.numberOfSecretKeys() === 0) - } - } - - val realUser = UserGroupInformation.createUserForTesting("realUser", Array.empty) - realUser.doAs(testImpl) - - val proxyUser = UserGroupInformation.createProxyUserForTesting("proxyUser", realUser, - Array.empty) - proxyUser.doAs(testImpl) - } finally { - UserGroupInformation.reset() - } - } } From bd05339171db00c2f2dd89702f9500ed6e1e321a Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 20 Sep 2019 14:39:24 +0900 Subject: [PATCH 0127/1387] [SPARK-29158][SQL] Expose SerializableConfiguration for DataSource V2 developers ### What changes were proposed in this pull request? Currently the SerializableConfiguration, which makes the Hadoop configuration serializable is private. This makes it public, with a developer annotation. ### Why are the changes needed? Many data source depend on the Hadoop configuration which may have specific components on the driver. Inside of Spark's own DataSourceV2 implementations this is frequently used (Parquet, Json, Orc, etc.) ### Does this PR introduce any user-facing change? This provides a new developer API. ### How was this patch tested? No new tests are added as this only exposes a previously developed & thoroughly used + tested component. Closes #25838 from holdenk/SPARK-29158-expose-serializableconfiguration-for-dsv2. Authored-by: Holden Karau Signed-off-by: HyukjinKwon --- .../util/SerializableConfigurationSuite.java | 28 +++++++++++++++++++ .../util/SerializableConfiguration.scala | 8 +++++- 2 files changed, 35 insertions(+), 1 deletion(-) create mode 100644 core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java diff --git a/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java b/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java new file mode 100644 index 0000000000000..1352653028271 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java @@ -0,0 +1,28 @@ +/* + * 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.util; + +/** + * This test ensures that the API we've exposed for SerializableConfiguration is usable + * from Java. It does not test any of the serialization it's self. + */ +class SerializableConfigurationSuite { + public SerializableConfiguration compileTest() { + SerializableConfiguration scs = new SerializableConfiguration(null); + return scs; + } +} diff --git a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala index 3354a923273ff..52b309abd77f2 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala @@ -20,7 +20,13 @@ import java.io.{ObjectInputStream, ObjectOutputStream} import org.apache.hadoop.conf.Configuration -private[spark] +import org.apache.spark.annotation.{DeveloperApi, Unstable} + +/** + * Helper wrapper to serialize a Hadoop configuration. Intended for use when implementing + * DataSourceV2 readers & writers which depend on the Hadoop configuration from the driver node. + */ +@DeveloperApi @Unstable class SerializableConfiguration(@transient var value: Configuration) extends Serializable { private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { out.defaultWriteObject() From 5e92301723464d0876b5a7eec59c15fed0c5b98c Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 19 Sep 2019 23:11:54 -0700 Subject: [PATCH 0128/1387] [SPARK-29161][CORE][SQL][STREAMING] Unify default wait time for waitUntilEmpty ### What changes were proposed in this pull request? This is a follow-up of the [review comment](https://github.com/apache/spark/pull/25706#discussion_r321923311). This patch unifies the default wait time to be 10 seconds as it would fit most of UTs (as they have smaller timeouts) and doesn't bring additional latency since it will return if the condition is met. This patch doesn't touch the one which waits 100000 milliseconds (100 seconds), to not break anything unintentionally, though I'd rather questionable that we really need to wait for 100 seconds. ### Why are the changes needed? It simplifies the test code and get rid of various heuristic values on timeout. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? CI build will test the patch, as it would be the best environment to test the patch (builds are running there). Closes #25837 from HeartSaVioR/MINOR-unify-default-wait-time-for-wait-until-empty. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/TestUtils.scala | 2 +- .../spark/scheduler/LiveListenerBus.scala | 11 +++++++ .../org/apache/spark/AccumulatorSuite.scala | 2 +- .../ExecutorAllocationManagerSuite.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 2 +- .../apache/spark/SparkContextInfoSuite.scala | 2 +- .../org/apache/spark/SparkContextSuite.scala | 2 +- .../spark/deploy/LogUrlsStandaloneSuite.scala | 7 ++--- .../metrics/InputOutputMetricsSuite.scala | 8 ++--- .../spark/scheduler/DAGSchedulerSuite.scala | 21 ++++++------- .../spark/scheduler/SparkListenerSuite.scala | 31 +++++++++---------- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../spark/sql/FileBasedDataSourceSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 4 +-- .../apache/spark/sql/SessionStateSuite.scala | 6 ++-- .../scala/org/apache/spark/sql/UDFSuite.scala | 6 ++-- .../FileDataSourceV2FallBackSuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 2 +- .../ui/SQLAppStatusListenerSuite.scala | 4 +-- .../StreamingQueryListenerSuite.scala | 6 ++-- .../StreamingQueryListenersConfSuite.scala | 2 +- .../sql/test/DataFrameReaderWriterSuite.scala | 10 +++--- .../sql/util/DataFrameCallbackSuite.scala | 18 +++++------ .../util/ExecutionListenerManagerSuite.scala | 4 +-- .../streaming/StreamingListenerSuite.scala | 2 +- 26 files changed, 82 insertions(+), 80 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 41ae3ae3b758a..9aeb128fcc491 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -264,7 +264,7 @@ private[spark] object TestUtils { try { body(listener) } finally { - sc.listenerBus.waitUntilEmpty(TimeUnit.SECONDS.toMillis(10)) + sc.listenerBus.waitUntilEmpty() sc.listenerBus.removeListener(listener) } } 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 302ebd30da228..bbbddd86cad39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -186,6 +186,17 @@ private[spark] class LiveListenerBus(conf: SparkConf) { metricsSystem.registerSource(metrics) } + /** + * For testing only. Wait until there are no more events in the queue, or until the default + * wait time has elapsed. Throw `TimeoutException` if the specified time elapsed before the queue + * emptied. + * Exposed for testing. + */ + @throws(classOf[TimeoutException]) + private[spark] def waitUntilEmpty(): Unit = { + waitUntilEmpty(TimeUnit.SECONDS.toMillis(10)) + } + /** * For testing only. Wait until there are no more events in the queue, or until the specified * time has elapsed. Throw `TimeoutException` if the specified time elapsed before the queue diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 435665d8a1ce2..a75cf3f0381df 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -126,7 +126,7 @@ private[spark] object AccumulatorSuite { sc.addSparkListener(listener) testBody // wait until all events have been processed before proceeding to assert things - sc.listenerBus.waitUntilEmpty(10 * 1000) + sc.listenerBus.waitUntilEmpty() val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) val isSet = accums.exists { a => a.name == Some(PEAK_EXECUTION_MEMORY) && a.value.exists(_.asInstanceOf[Long] > 0L) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 07fb323cfc355..460714f204a3a 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -64,7 +64,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private def post(event: SparkListenerEvent): Unit = { listenerBus.post(event) - listenerBus.waitUntilEmpty(1000) + listenerBus.waitUntilEmpty() } test("initialize dynamic allocation in SparkContext") { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 923c9c90447fd..4c9d7e9faeb2a 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -498,7 +498,7 @@ object ShuffleSuite { job - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() AggregatedShuffleMetrics(recordsWritten, recordsRead, bytesWritten, bytesRead) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 536b4aec75623..09510edcb4499 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -63,7 +63,7 @@ class SparkContextInfoSuite extends SparkFunSuite with LocalSparkContext { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(sc.getRDDStorageInfo.length === 0) rdd.collect() - sc.listenerBus.waitUntilEmpty(10000) + sc.listenerBus.waitUntilEmpty() eventually(timeout(10.seconds), interval(100.milliseconds)) { assert(sc.getRDDStorageInfo.length === 1) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 786f55c96a3e8..eca49e5efdbbb 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -716,7 +716,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } x }.collect() - sc.listenerBus.waitUntilEmpty(10000) + sc.listenerBus.waitUntilEmpty() // As executors will send the metrics of running tasks via heartbeat, we can use this to check // whether there is any running task. eventually(timeout(10.seconds)) { diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index cbdf1755b0c5b..d953211c13b1d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -29,9 +29,6 @@ import org.apache.spark.util.SparkConfWithEnv class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext { - /** Length of time to wait while draining listener events. */ - private val WAIT_TIMEOUT_MILLIS = 10000 - test("verify that correct log urls get propagated from workers") { sc = new SparkContext("local-cluster[2,1,1024]", "test") @@ -41,7 +38,7 @@ class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext { // Trigger a job so that executors get added sc.parallelize(1 to 100, 4).map(_.toString).count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) // Browse to each URL to check that it's valid @@ -61,7 +58,7 @@ class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext { // Trigger a job so that executors get added sc.parallelize(1 to 100, 4).map(_.toString).count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] assert(listeners.size === 1) val listener = listeners(0) diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index c7bd0c905d027..f9511045ee89e 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -182,7 +182,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext .reduceByKey(_ + _) .saveAsTextFile(tmpFile.toURI.toString) - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() assert(inputRead == numRecords) assert(outputWritten == numBuckets) @@ -243,7 +243,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext val taskMetrics = new ArrayBuffer[Long]() // Avoid receiving earlier taskEnd events - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() sc.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { @@ -253,7 +253,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext job - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() taskMetrics.sum } @@ -293,7 +293,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext try { rdd.saveAsTextFile(outPath.toString) - sc.listenerBus.waitUntilEmpty(500) + sc.listenerBus.waitUntilEmpty() assert(taskBytesWritten.length == 2) val outFiles = fs.listStatus(outPath).filter(_.getPath.getName != "_SUCCESS") taskBytesWritten.zip(outFiles).foreach { case (bytes, fileStatus) => diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 7cb7eceec615b..b86d24deb15fc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -172,9 +172,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def applicationAttemptId(): Option[String] = None } - /** Length of time to wait while draining listener events. */ - val WAIT_TIMEOUT_MILLIS = 10000 - /** * Listeners which records some information to verify in UTs. Getter-kind methods in this class * ensures the value is returned after ensuring there's no event to process, as well as the @@ -230,7 +227,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi _endedTasks.toSet } - private def waitForListeners(): Unit = sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + private def waitForListeners(): Unit = sc.listenerBus.waitUntilEmpty() } var sparkListener: EventInfoRecordingListener = null @@ -839,7 +836,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val testRdd = new MyRDD(sc, 0, Nil) val waiter = scheduler.submitJob(testRdd, func, Seq.empty, CallSite.empty, resultHandler, properties) - sc.listenerBus.waitUntilEmpty(1000L) + sc.listenerBus.waitUntilEmpty() assert(assertionError.get() === null) } @@ -957,7 +954,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi completeNextResultStageWithSuccess(1, 1) // Confirm job finished successfully - sc.listenerBus.waitUntilEmpty(1000) + sc.listenerBus.waitUntilEmpty() assert(ended) assert(results === (0 until parts).map { idx => idx -> 42 }.toMap) assertDataStructuresEmpty() @@ -994,7 +991,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } else { // Stage should have been aborted and removed from running stages assertDataStructuresEmpty() - sc.listenerBus.waitUntilEmpty(1000) + sc.listenerBus.waitUntilEmpty() assert(ended) jobResult match { case JobFailed(reason) => @@ -1116,7 +1113,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi completeNextResultStageWithSuccess(2, 1) assertDataStructuresEmpty() - sc.listenerBus.waitUntilEmpty(1000) + sc.listenerBus.waitUntilEmpty() assert(ended) assert(results === Map(0 -> 42)) } @@ -1175,7 +1172,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete the result stage. completeNextResultStageWithSuccess(1, 1) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() assertDataStructuresEmpty() } @@ -1204,7 +1201,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete the result stage. completeNextResultStageWithSuccess(1, 0) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() assertDataStructuresEmpty() } @@ -1230,7 +1227,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi null)) // Assert the stage has been cancelled. - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() assert(failure.getMessage.startsWith("Job aborted due to stage failure: Could not recover " + "from a failed barrier ResultStage.")) } @@ -2668,7 +2665,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi sc.parallelize(1 to tasks, tasks).foreach { _ => accum.add(1L) } - sc.listenerBus.waitUntilEmpty(1000) + sc.listenerBus.waitUntilEmpty() assert(foundCount.get() === tasks) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 8903e1054f53d..cda6fbad5419f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -38,9 +38,6 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match import LiveListenerBus._ - /** Length of time to wait while draining listener events. */ - val WAIT_TIMEOUT_MILLIS = 10000 - val jobCompletionTime = 1421191296660L private val mockSparkContext: SparkContext = Mockito.mock(classOf[SparkContext]) @@ -65,7 +62,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match sc.listenerBus.addToSharedQueue(listener) sc.listenerBus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() sc.stop() assert(listener.sparkExSeen) @@ -97,7 +94,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // Starting listener bus should flush all buffered events bus.start(mockSparkContext, mockMetricsSystem) Mockito.verify(mockMetricsSystem).registerSource(bus.metrics) - bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + bus.waitUntilEmpty() assert(counter.count === 5) assert(sharedQueueSize(bus) === 0) assert(eventProcessingTimeCount(bus) === 5) @@ -223,7 +220,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match rdd2.setName("Target RDD") rdd2.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {1} val (stageInfo, taskInfoMetrics) = listener.stageInfos.head @@ -248,7 +245,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match rdd3.setName("Trois") rdd1.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {1} val stageInfo1 = listener.stageInfos.keys.find(_.stageId == 0).get stageInfo1.rddInfos.size should be {1} // ParallelCollectionRDD @@ -257,7 +254,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match listener.stageInfos.clear() rdd2.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {1} val stageInfo2 = listener.stageInfos.keys.find(_.stageId == 1).get stageInfo2.rddInfos.size should be {3} @@ -266,7 +263,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match listener.stageInfos.clear() rdd3.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {2} // Shuffle map stage + result stage val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 3).get stageInfo3.rddInfos.size should be {1} // ShuffledRDD @@ -282,7 +279,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val rdd2 = rdd1.map(_.toString) sc.runJob(rdd2, (items: Iterator[String]) => items.size, Seq(0, 1)) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be {1} val (stageInfo, _) = listener.stageInfos.head @@ -310,7 +307,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val numSlices = 16 val d = sc.parallelize(0 to 10000, numSlices).map(w) d.count() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be (1) val d2 = d.map { i => w(i) -> i * 2 }.setName("shuffle input 1") @@ -321,7 +318,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match d4.setName("A Cogroup") d4.collectAsMap() - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() listener.stageInfos.size should be (4) listener.stageInfos.foreach { case (stageInfo, taskInfoMetrics) => /** @@ -372,7 +369,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match .reduce { case (x, y) => x } assert(result === 1.to(maxRpcMessageSize).toArray) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) assert(listener.startedGettingResultTasks.contains(TASK_INDEX)) @@ -388,7 +385,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.listenerBus.waitUntilEmpty() val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) assert(listener.startedGettingResultTasks.isEmpty) @@ -443,7 +440,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // Post events to all listeners, and wait until the queue is drained (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } - bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + bus.waitUntilEmpty() // The exception should be caught, and the event should be propagated to other listeners assert(jobCounter1.count === 5) @@ -513,7 +510,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // after we post one event, both interrupting listeners should get removed, and the // event log queue should be removed bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) - bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + bus.waitUntilEmpty() assert(bus.activeQueues() === Set(SHARED_QUEUE, APP_STATUS_QUEUE)) assert(bus.findListenersByClass[BasicJobCounter]().size === 2) assert(bus.findListenersByClass[InterruptingListener]().size === 0) @@ -522,7 +519,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match // posting more events should be fine, they'll just get processed from the OK queue. (0 until 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } - bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + bus.waitUntilEmpty() assert(counter1.count === 6) assert(counter2.count === 6) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 6e1ee6da9200d..edafdaa54945a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -849,7 +849,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi sparkContext.addSparkListener(jobListener) try { val result = f - sparkContext.listenerBus.waitUntilEmpty(10000L) + sparkContext.listenerBus.waitUntilEmpty() assert(numJobTrigered === 0) result } finally { 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 6a2eac9e7e21c..2306c3ff039f1 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 @@ -2114,7 +2114,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { val df = spark.read.json(path.getCanonicalPath) assert(df.columns === Array("i", "p")) - spark.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.listenerBus.waitUntilEmpty() assert(numJobs == 1) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 23848d90dc53d..e4c10d1990f25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -488,7 +488,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { sparkContext.addSparkListener(bytesReadListener) try { spark.read.csv(path).limit(1).collect() - sparkContext.listenerBus.waitUntilEmpty(1000L) + sparkContext.listenerBus.waitUntilEmpty() assert(bytesReads.sum === 7860) } finally { sparkContext.removeSparkListener(bytesReadListener) 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 80c1e24bfa568..28a027690db04 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 @@ -2609,14 +2609,14 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { } // Make sure no spurious job starts are pending in the listener bus. - sparkContext.listenerBus.waitUntilEmpty(500) + sparkContext.listenerBus.waitUntilEmpty() sparkContext.addSparkListener(listener) try { // Execute the command. sql("show databases").head() // Make sure we have seen all events triggered by DataFrame.show() - sparkContext.listenerBus.waitUntilEmpty(500) + sparkContext.listenerBus.waitUntilEmpty() } finally { sparkContext.removeSparkListener(listener) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala index 1d461a03fd1f6..31957a99e15af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala @@ -158,7 +158,7 @@ class SessionStateSuite extends SparkFunSuite { assert(forkedSession ne activeSession) assert(forkedSession.listenerManager ne activeSession.listenerManager) runCollectQueryOn(forkedSession) - activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) + activeSession.sparkContext.listenerBus.waitUntilEmpty() assert(collectorA.commands.length == 1) // forked should callback to A assert(collectorA.commands(0) == "collect") @@ -166,14 +166,14 @@ class SessionStateSuite extends SparkFunSuite { // => changes to forked do not affect original forkedSession.listenerManager.register(collectorB) runCollectQueryOn(activeSession) - activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) + activeSession.sparkContext.listenerBus.waitUntilEmpty() assert(collectorB.commands.isEmpty) // original should not callback to B assert(collectorA.commands.length == 2) // original should still callback to A assert(collectorA.commands(1) == "collect") // <= changes to original do not affect forked activeSession.listenerManager.register(collectorC) runCollectQueryOn(forkedSession) - activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) + activeSession.sparkContext.listenerBus.waitUntilEmpty() assert(collectorC.commands.isEmpty) // forked should not callback to C assert(collectorA.commands.length == 3) // forked should still callback to A assert(collectorB.commands.length == 1) // forked should still callback to B diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 2a034bcdc3f00..268fb65144e72 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -360,13 +360,13 @@ class UDFSuite extends QueryTest with SharedSparkSession { .withColumn("b", udf1($"a", lit(10))) df.cache() df.write.saveAsTable("t") - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(numTotalCachedHit == 1, "expected to be cached in saveAsTable") df.write.insertInto("t") - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(numTotalCachedHit == 2, "expected to be cached in insertInto") df.write.save(path.getCanonicalPath) - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(numTotalCachedHit == 3, "expected to be cached in save for native") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala index cd804adfa2133..2b3340527a4e2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -173,7 +173,7 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { withTempPath { path => val inputData = spark.range(10) inputData.write.format(format).save(path.getCanonicalPath) - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(commands.length == 1) assert(commands.head._1 == "save") assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index e7f1c42d7d7c5..ccefb53fc4b39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -388,7 +388,7 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { // Assume the execution plan is // PhysicalRDD(nodeId = 0) data.write.format("json").save(file.getAbsolutePath) - sparkContext.listenerBus.waitUntilEmpty(10000) + sparkContext.listenerBus.waitUntilEmpty() val executionIds = currentExecutionIds().diff(previousExecutionIds) assert(executionIds.size === 1) val executionId = executionIds.head diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 90966d2efec23..88864ccec7523 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -480,7 +480,7 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils // At the beginning of this test case, there should be no live data in the listener. assert(listener.noLiveData()) spark.sparkContext.parallelize(1 to 10).foreach(i => ()) - spark.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.listenerBus.waitUntilEmpty() // Listener should ignore the non-SQL stages, as the stage data are only removed when SQL // execution ends, which will not be triggered for non-SQL jobs. assert(listener.noLiveData()) @@ -673,7 +673,7 @@ class SQLAppStatusListenerMemoryLeakSuite extends SparkFunSuite { case e: SparkException => // This is expected for a failed job } } - sc.listenerBus.waitUntilEmpty(10000) + sc.listenerBus.waitUntilEmpty() val statusStore = spark.sharedState.statusStore assert(statusStore.executionsCount() <= 50) assert(statusStore.planGraphCount() <= 50) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index d84dc62a696b9..4c58cb85c4d36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -49,7 +49,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(spark.streams.active.isEmpty) assert(spark.streams.listListeners().isEmpty) // Make sure we don't leak any events to the next test - spark.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.listenerBus.waitUntilEmpty() } testQuietly("single listener, check trigger events are generated correctly") { @@ -320,7 +320,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { q.recentProgress.size > 1 && q.recentProgress.size <= 11 } testStream(input.toDS)(actions: _*) - spark.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.listenerBus.waitUntilEmpty() // 11 is the max value of the possible numbers of events. assert(numProgressEvent > 1 && numProgressEvent <= 11) } finally { @@ -343,7 +343,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { AddData(mem, 1, 2, 3), CheckAnswer(1, 2, 3) ) - session.sparkContext.listenerBus.waitUntilEmpty(5000) + session.sparkContext.listenerBus.waitUntilEmpty() } def assertEventsCollected(collector: EventCollector): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala index 7801d968e901d..d538d93b845b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala @@ -39,7 +39,7 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { StopStream ) - spark.sparkContext.listenerBus.waitUntilEmpty(5000) + spark.sparkContext.listenerBus.waitUntilEmpty() assert(TestListener.queryStartedEvent != null) assert(TestListener.queryTerminatedEvent != null) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index b98626a34cc29..01a03e484c90d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -277,7 +277,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with .format(classOf[NoopDataSource].getName) .mode(SaveMode.Append) .save() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(plan.isInstanceOf[AppendData]) // overwrite mode creates `OverwriteByExpression` @@ -285,21 +285,21 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with .format(classOf[NoopDataSource].getName) .mode(SaveMode.Overwrite) .save() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(plan.isInstanceOf[OverwriteByExpression]) // By default the save mode is `ErrorIfExists` for data source v2. spark.range(10).write .format(classOf[NoopDataSource].getName) .save() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(plan.isInstanceOf[AppendData]) spark.range(10).write .format(classOf[NoopDataSource].getName) .mode("default") .save() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(plan.isInstanceOf[AppendData]) } finally { spark.listenerManager.unregister(listener) @@ -1058,7 +1058,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with checkDatasetUnorderly( spark.read.parquet(dir.getCanonicalPath).as[(Long, Long)], 0L -> 0L, 1L -> 1L, 2L -> 2L) - sparkContext.listenerBus.waitUntilEmpty(10000) + sparkContext.listenerBus.waitUntilEmpty() assert(jobDescriptions.asScala.toList.exists( _.contains("Listing leaf files and directories for 3 paths"))) } finally { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index 18a82f0f84283..e96f05384c879 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -49,7 +49,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { df.select("i").collect() df.filter($"i" > 0).count() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(metrics.length == 2) assert(metrics(0)._1 == "collect") @@ -80,7 +80,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException](df.select(errorUdf($"i")).collect()) - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(metrics.length == 1) assert(metrics(0)._1 == "collect") assert(metrics(0)._2.analyzed.isInstanceOf[Project]) @@ -110,12 +110,12 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { df.collect() // Wait for the first `collect` to be caught by our listener. Otherwise the next `collect` will // reset the plan metrics. - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() df.collect() Seq(1 -> "a", 2 -> "a").toDF("i", "j").groupBy("i").count().collect() - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(metrics.length == 3) assert(metrics(0) === 1) assert(metrics(1) === 1) @@ -163,7 +163,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { // For this simple case, the peakExecutionMemory of a stage should be the data size of the // aggregate operator, as we only have one memory consuming operator per stage. - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(metrics.length == 2) assert(metrics(0) == topAggDataSize) assert(metrics(1) == bottomAggDataSize) @@ -187,7 +187,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { withTempPath { path => spark.range(10).write.format("json").save(path.getCanonicalPath) - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(commands.length == 1) assert(commands.head._1 == "save") assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) @@ -198,7 +198,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { withTable("tab") { sql("CREATE TABLE tab(i long) using parquet") // adds commands(1) via onSuccess spark.range(10).write.insertInto("tab") - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(commands.length == 3) assert(commands(2)._1 == "insertInto") assert(commands(2)._2.isInstanceOf[InsertIntoStatement]) @@ -209,7 +209,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { withTable("tab") { spark.range(10).select($"id", $"id" % 5 as "p").write.partitionBy("p").saveAsTable("tab") - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(commands.length == 5) assert(commands(4)._1 == "saveAsTable") assert(commands(4)._2.isInstanceOf[CreateTable]) @@ -221,7 +221,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { val e = intercept[AnalysisException] { spark.range(10).select($"id", $"id").write.insertInto("tab") } - sparkContext.listenerBus.waitUntilEmpty(1000) + sparkContext.listenerBus.waitUntilEmpty() assert(errors.length == 1) assert(errors.head._1 == "insertInto") assert(errors.head._2 == e) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala index 79819e7655414..2fd6cb220ea3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala @@ -34,13 +34,13 @@ class ExecutionListenerManagerSuite extends SparkFunSuite with LocalSparkSession spark = SparkSession.builder().master("local").appName("test").config(conf).getOrCreate() spark.sql("select 1").collect() - spark.sparkContext.listenerBus.waitUntilEmpty(1000) + spark.sparkContext.listenerBus.waitUntilEmpty() assert(INSTANCE_COUNT.get() === 1) assert(CALLBACK_COUNT.get() === 1) val cloned = spark.cloneSession() cloned.sql("select 1").collect() - spark.sparkContext.listenerBus.waitUntilEmpty(1000) + spark.sparkContext.listenerBus.waitUntilEmpty() assert(INSTANCE_COUNT.get() === 1) assert(CALLBACK_COUNT.get() === 2) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 9cd5d8c0e95e1..01dbe75a9eae8 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -227,7 +227,7 @@ class StreamingListenerSuite extends TestSuiteBase with LocalStreamingContext wi // Post a Streaming event after stopping StreamingContext val receiverInfoStopped = ReceiverInfo(0, "test", false, "localhost", "0") ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(receiverInfoStopped)) - ssc.sparkContext.listenerBus.waitUntilEmpty(1000) + ssc.sparkContext.listenerBus.waitUntilEmpty() // The StreamingListener should not receive any event verifyNoMoreInteractions(streamingListener) } From ec8a1a8e8869b000a8001126ca013b753acddf6a Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 20 Sep 2019 21:41:09 +0900 Subject: [PATCH 0129/1387] [SPARK-29122][SQL] Propagate all the SQL conf to executors in SQLQueryTestSuite ### What changes were proposed in this pull request? This pr is to propagate all the SQL configurations to executors in `SQLQueryTestSuite`. When the propagation enabled in the tests, a potential bug below becomes apparent; ``` CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet; .... select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4): QueryOutput(select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4),struct<>,java.lang.IllegalArgumentException [info] requirement failed: MutableProjection cannot use UnsafeRow for output data types: decimal(38,0)) (SQLQueryTestSuite.scala:380) ``` The root culprit is that `InterpretedMutableProjection` has incorrect validation in the interpreter mode: `validExprs.forall { case (e, _) => UnsafeRow.isFixedLength(e.dataType) }`. This validation should be the same with the condition (`isMutable`) in `HashAggregate.supportsAggregate`: https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L1126 ### Why are the changes needed? Bug fixes. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added tests in `AggregationQuerySuite` Closes #25831 from maropu/SPARK-29122. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../expressions/InterpretedMutableProjection.scala | 6 +++--- .../sql/catalyst/expressions/windowExpressions.scala | 2 +- .../org/apache/spark/sql/SQLQueryTestSuite.scala | 8 ++++---- .../sql/hive/execution/AggregationQuerySuite.scala | 12 +++++++++++- 4 files changed, 19 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala index 5c8aa4e2e9d83..a2daec0b1ade1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala @@ -50,11 +50,11 @@ class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mutable def currentValue: InternalRow = mutableRow override def target(row: InternalRow): MutableProjection = { - // If `mutableRow` is `UnsafeRow`, `MutableProjection` accepts fixed-length types only + // If `mutableRow` is `UnsafeRow`, `MutableProjection` accepts mutable types only require(!row.isInstanceOf[UnsafeRow] || - validExprs.forall { case (e, _) => UnsafeRow.isFixedLength(e.dataType) }, + validExprs.forall { case (e, _) => UnsafeRow.isMutable(e.dataType) }, "MutableProjection cannot use UnsafeRow for output data types: " + - validExprs.map(_._1.dataType).filterNot(UnsafeRow.isFixedLength) + validExprs.map(_._1.dataType).filterNot(UnsafeRow.isMutable) .map(_.catalogString).mkString(", ")) mutableRow = row this diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index 0b674d025d1ac..74df1e7b8e8c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -606,7 +606,7 @@ case class NTile(buckets: Expression) extends RowNumberLike with SizeBasedWindow zero, zero, zero, - (n / buckets).cast(IntegerType), + (n.cast(DecimalType.IntDecimal) / buckets.cast(DecimalType.IntDecimal)).cast(IntegerType), (n % buckets).cast(IntegerType) ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index abecaa7a9b16a..f32575794f946 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -310,9 +310,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) - // Propagate the SQL conf ANSI_ENABLED to executor. - // TODO: remove this after SPARK-29122 is resolved. - localSparkSession.sparkContext.setLocalProperty(SQLConf.ANSI_ENABLED.key, "true") + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } @@ -406,7 +404,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { val df = session.sql(sql) val schema = df.schema // Get answer, but also get rid of the #1234 expression ids that show up in explain plans - val answer = hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) + val answer = SQLExecution.withNewExecutionId(session, df.queryExecution, Some(sql)) { + hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) + } // If the output is not pre-sorted, sort it. if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index d06cc1c0a88ac..8c7e5bf5ac1d4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -24,7 +24,7 @@ import test.org.apache.spark.sql.MyDoubleAvg import test.org.apache.spark.sql.MyDoubleSum import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.{CodegenObjectFactoryMode, UnsafeRow} import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -1018,6 +1018,16 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te val agg2 = agg1.groupBy($"text").agg(sum($"avg_res")) checkAnswer(agg2, Row("a", BigDecimal("11.9999999994857142860000"))) } + + test("SPARK-29122: hash-based aggregates for unfixed-length decimals in the interpreter mode") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString) { + withTempView("t") { + spark.range(3).selectExpr("CAST(id AS decimal(38, 0)) a").createOrReplaceTempView("t") + checkAnswer(sql("SELECT SUM(a) FROM t"), Row(java.math.BigDecimal.valueOf(3))) + } + } + } } From eb7ee6834dc7936b11d85daa7770c10d23842058 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 20 Sep 2019 22:04:32 +0800 Subject: [PATCH 0130/1387] [SPARK-29062][SQL] Add V1_BATCH_WRITE to the TableCapabilityChecks ### What changes were proposed in this pull request? Currently the checks in the Analyzer require that V2 Tables have BATCH_WRITE defined for all tables that have V1 Write fallbacks. This is confusing as these tables may not have the V2 writer interface implemented yet. This PR adds this table capability to these checks. In addition, this allows V2 tables to leverage the V1 APIs for DataFrameWriter.save if they do extend the V1_BATCH_WRITE capability. This way, these tables can continue to receive partitioning information and also perform checks for the existence of tables, and support all SaveModes. ### Why are the changes needed? Partitioned saves through DataFrame.write are otherwise broken for V2 tables that support the V1 write API. ### Does this PR introduce any user-facing change? No ### How was this patch tested? V1WriteFallbackSuite Closes #25767 from brkyvz/bwcheck. Authored-by: Burak Yavuz Signed-off-by: Wenchen Fan --- .../apache/spark/sql/DataFrameWriter.scala | 9 +- .../datasources/v2/TableCapabilityCheck.scala | 14 +- .../connector/TableCapabilityCheckSuite.scala | 28 ++-- .../sql/connector/V1WriteFallbackSuite.scala | 130 ++++++++++++++++-- 4 files changed, 153 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 289a976c6db9e..edf0963e71e81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -253,11 +253,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val maybeV2Provider = lookupV2Provider() if (maybeV2Provider.isDefined) { - if (partitioningColumns.nonEmpty) { - throw new AnalysisException( - "Cannot write data to TableProvider implementation if partition columns are specified.") - } - val provider = maybeV2Provider.get val sessionOptions = DataSourceV2Utils.extractSessionConfigs( provider, df.sparkSession.sessionState.conf) @@ -267,6 +262,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ provider.getTable(dsOptions) match { case table: SupportsWrite if table.supports(BATCH_WRITE) => + if (partitioningColumns.nonEmpty) { + throw new AnalysisException("Cannot write data to TableProvider implementation " + + "if partition columns are specified.") + } lazy val relation = DataSourceV2Relation.create(table, dsOptions) modeForDSV2 match { case SaveMode.Append => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala index 45ca3dfb9cb93..509a5f7139cca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TableCapabilityCheck.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.types.BooleanType @@ -32,6 +33,10 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) { private def failAnalysis(msg: String): Unit = throw new AnalysisException(msg) + private def supportsBatchWrite(table: Table): Boolean = { + table.supportsAny(BATCH_WRITE, V1_BATCH_WRITE) + } + override def apply(plan: LogicalPlan): Unit = { plan foreach { case r: DataSourceV2Relation if !r.table.supports(BATCH_READ) => @@ -43,8 +48,7 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) { // TODO: check STREAMING_WRITE capability. It's not doable now because we don't have a // a logical plan for streaming write. - - case AppendData(r: DataSourceV2Relation, _, _, _) if !r.table.supports(BATCH_WRITE) => + case AppendData(r: DataSourceV2Relation, _, _, _) if !supportsBatchWrite(r.table) => failAnalysis(s"Table ${r.table.name()} does not support append in batch mode.") case OverwritePartitionsDynamic(r: DataSourceV2Relation, _, _, _) @@ -54,13 +58,13 @@ object TableCapabilityCheck extends (LogicalPlan => Unit) { case OverwriteByExpression(r: DataSourceV2Relation, expr, _, _, _) => expr match { case Literal(true, BooleanType) => - if (!r.table.supports(BATCH_WRITE) || - !r.table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER)) { + if (!supportsBatchWrite(r.table) || + !r.table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER)) { failAnalysis( s"Table ${r.table.name()} does not support truncate in batch mode.") } case _ => - if (!r.table.supports(BATCH_WRITE) || !r.table.supports(OVERWRITE_BY_FILTER)) { + if (!supportsBatchWrite(r.table) || !r.table.supports(OVERWRITE_BY_FILTER)) { failAnalysis(s"Table ${r.table.name()} does not support " + "overwrite by filter in batch mode.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 39f4085a9baf9..ce6d56cf84df1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -98,16 +98,19 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { } test("AppendData: check correct capabilities") { - val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(BATCH_WRITE), CaseInsensitiveStringMap.empty), - TestRelation) + Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => + val plan = AppendData.byName( + DataSourceV2Relation.create(CapabilityTable(write), CaseInsensitiveStringMap.empty), + TestRelation) - TableCapabilityCheck.apply(plan) + TableCapabilityCheck.apply(plan) + } } test("Truncate: check missing capabilities") { Seq(CapabilityTable(), CapabilityTable(BATCH_WRITE), + CapabilityTable(V1_BATCH_WRITE), CapabilityTable(TRUNCATE), CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => @@ -125,7 +128,9 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("Truncate: check correct capabilities") { Seq(CapabilityTable(BATCH_WRITE, TRUNCATE), - CapabilityTable(BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => + CapabilityTable(V1_BATCH_WRITE, TRUNCATE), + CapabilityTable(BATCH_WRITE, OVERWRITE_BY_FILTER), + CapabilityTable(V1_BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, @@ -137,6 +142,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("OverwriteByExpression: check missing capabilities") { Seq(CapabilityTable(), + CapabilityTable(V1_BATCH_WRITE), CapabilityTable(BATCH_WRITE), CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => @@ -153,12 +159,14 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { } test("OverwriteByExpression: check correct capabilities") { - val table = CapabilityTable(BATCH_WRITE, OVERWRITE_BY_FILTER) - val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, - EqualTo(AttributeReference("x", LongType)(), Literal(5))) + Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => + val table = CapabilityTable(write, OVERWRITE_BY_FILTER) + val plan = OverwriteByExpression.byName( + DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + EqualTo(AttributeReference("x", LongType)(), Literal(5))) - TableCapabilityCheck.apply(plan) + TableCapabilityCheck.apply(plan) + } } test("OverwritePartitionsDynamic: check missing capabilities") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index 7cd6ba21b56ec..de843ba4375d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -24,11 +24,12 @@ import scala.collection.mutable import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.{DataFrame, QueryTest, Row, SaveMode, SparkSession} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.connector.write.{SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} -import org.apache.spark.sql.sources.{DataSourceRegister, Filter, InsertableRelation} +import org.apache.spark.sql.execution.datasources.{DataSource, DataSourceUtils} +import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -52,7 +53,11 @@ class V1WriteFallbackSuite extends QueryTest with SharedSparkSession with Before test("append fallback") { val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") df.write.mode("append").option("name", "t1").format(v2Format).save() + checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df) + assert(InMemoryV1Provider.tables("t1").schema === df.schema.asNullable) + assert(InMemoryV1Provider.tables("t1").partitioning.isEmpty) + df.write.mode("append").option("name", "t1").format(v2Format).save() checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df.union(df)) } @@ -65,6 +70,59 @@ class V1WriteFallbackSuite extends QueryTest with SharedSparkSession with Before df2.write.mode("overwrite").option("name", "t1").format(v2Format).save() checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df2) } + + SaveMode.values().foreach { mode => + test(s"save: new table creations with partitioning for table - mode: $mode") { + val format = classOf[InMemoryV1Provider].getName + val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") + df.write.mode(mode).option("name", "t1").format(format).partitionBy("a").save() + + checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df) + assert(InMemoryV1Provider.tables("t1").schema === df.schema.asNullable) + assert(InMemoryV1Provider.tables("t1").partitioning.sameElements( + Array(IdentityTransform(FieldReference(Seq("a")))))) + } + } + + test("save: default mode is ErrorIfExists") { + val format = classOf[InMemoryV1Provider].getName + val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") + + df.write.option("name", "t1").format(format).partitionBy("a").save() + // default is ErrorIfExists, and since a table already exists we throw an exception + val e = intercept[AnalysisException] { + df.write.option("name", "t1").format(format).partitionBy("a").save() + } + assert(e.getMessage.contains("already exists")) + } + + test("save: Ignore mode") { + val format = classOf[InMemoryV1Provider].getName + val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") + + df.write.option("name", "t1").format(format).partitionBy("a").save() + // no-op + df.write.option("name", "t1").format(format).mode("ignore").partitionBy("a").save() + + checkAnswer(InMemoryV1Provider.getTableData(spark, "t1"), df) + } + + test("save: tables can perform schema and partitioning checks if they already exist") { + val format = classOf[InMemoryV1Provider].getName + val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") + + df.write.option("name", "t1").format(format).partitionBy("a").save() + val e2 = intercept[IllegalArgumentException] { + df.write.mode("append").option("name", "t1").format(format).partitionBy("b").save() + } + assert(e2.getMessage.contains("partitioning")) + + val e3 = intercept[IllegalArgumentException] { + Seq((1, "x")).toDF("c", "d").write.mode("append").option("name", "t1").format(format) + .save() + } + assert(e3.getMessage.contains("schema")) + } } class V1WriteFallbackSessionCatalogSuite @@ -114,26 +172,83 @@ private object InMemoryV1Provider { } } -class InMemoryV1Provider extends TableProvider with DataSourceRegister { +class InMemoryV1Provider + extends TableProvider + with DataSourceRegister + with CreatableRelationProvider { override def getTable(options: CaseInsensitiveStringMap): Table = { - InMemoryV1Provider.tables.getOrElseUpdate(options.get("name"), { + + InMemoryV1Provider.tables.getOrElse(options.get("name"), { new InMemoryTableWithV1Fallback( "InMemoryTableWithV1Fallback", - new StructType().add("a", IntegerType).add("b", StringType), - Array(IdentityTransform(FieldReference(Seq("a")))), + new StructType(), + Array.empty, options.asCaseSensitiveMap() ) }) } override def shortName(): String = "in-memory" + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + val _sqlContext = sqlContext + + val partitioning = parameters.get(DataSourceUtils.PARTITIONING_COLUMNS_KEY).map { value => + DataSourceUtils.decodePartitioningColumns(value).map { partitioningColumn => + IdentityTransform(FieldReference(partitioningColumn)) + } + }.getOrElse(Nil) + + val tableName = parameters("name") + val tableOpt = InMemoryV1Provider.tables.get(tableName) + val table = tableOpt.getOrElse(new InMemoryTableWithV1Fallback( + "InMemoryTableWithV1Fallback", + data.schema.asNullable, + partitioning.toArray, + Map.empty[String, String].asJava + )) + if (tableOpt.isEmpty) { + InMemoryV1Provider.tables.put(tableName, table) + } else { + if (data.schema.asNullable != table.schema) { + throw new IllegalArgumentException("Wrong schema provided") + } + if (!partitioning.sameElements(table.partitioning)) { + throw new IllegalArgumentException("Wrong partitioning provided") + } + } + + def getRelation: BaseRelation = new BaseRelation { + override def sqlContext: SQLContext = _sqlContext + override def schema: StructType = table.schema + } + + if (mode == SaveMode.ErrorIfExists && tableOpt.isDefined) { + throw new AnalysisException("Table already exists") + } else if (mode == SaveMode.Ignore && tableOpt.isDefined) { + // do nothing + return getRelation + } + val writer = table.newWriteBuilder(new CaseInsensitiveStringMap(parameters.asJava)) + if (mode == SaveMode.Overwrite) { + writer.asInstanceOf[SupportsTruncate].truncate() + } + writer.asInstanceOf[V1WriteBuilder].buildForV1Write().insert(data, overwrite = false) + getRelation + } } class InMemoryTableWithV1Fallback( override val name: String, override val schema: StructType, override val partitioning: Array[Transform], - override val properties: util.Map[String, String]) extends Table with SupportsWrite { + override val properties: util.Map[String, String]) + extends Table + with SupportsWrite { partitioning.foreach { t => if (!t.isInstanceOf[IdentityTransform]) { @@ -142,7 +257,6 @@ class InMemoryTableWithV1Fallback( } override def capabilities: util.Set[TableCapability] = Set( - TableCapability.BATCH_WRITE, TableCapability.V1_BATCH_WRITE, TableCapability.OVERWRITE_BY_FILTER, TableCapability.TRUNCATE).asJava From a23ad25ba43ec0974986b3ffdb4df671c8c8b9ae Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 20 Sep 2019 08:52:30 -0700 Subject: [PATCH 0131/1387] [SPARK-29158][SQL][FOLLOW-UP] Create an actual test case under `src/test` and minor documentation correction ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/25838 and proposes to create an actual test case under `src/test`. Previously, compile only test existed at `src/main`. Also, just changed the wordings in `SerializableConfiguration` just only to describe what it does (remove other words). ### Why are the changes needed? Tests codes should better exist in `src/test` not `src/main`. Also, it should better test a basic functionality. ### Does this PR introduce any user-facing change? No except minor doc change. ### How was this patch tested? Unit test was added. Closes #25867 from HyukjinKwon/SPARK-29158. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../util/SerializableConfigurationSuite.java | 28 ---------- .../util/SerializableConfiguration.scala | 5 +- .../util/SerializableConfigurationSuite.java | 55 +++++++++++++++++++ 3 files changed, 58 insertions(+), 30 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java create mode 100644 core/src/test/java/org/apache/spark/util/SerializableConfigurationSuite.java diff --git a/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java b/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java deleted file mode 100644 index 1352653028271..0000000000000 --- a/core/src/main/java/org/apache/spark/util/SerializableConfigurationSuite.java +++ /dev/null @@ -1,28 +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.util; - -/** - * This test ensures that the API we've exposed for SerializableConfiguration is usable - * from Java. It does not test any of the serialization it's self. - */ -class SerializableConfigurationSuite { - public SerializableConfiguration compileTest() { - SerializableConfiguration scs = new SerializableConfiguration(null); - return scs; - } -} diff --git a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala index 52b309abd77f2..42d7f71404594 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala @@ -23,8 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Unstable} /** - * Helper wrapper to serialize a Hadoop configuration. Intended for use when implementing - * DataSourceV2 readers & writers which depend on the Hadoop configuration from the driver node. + * Hadoop configuration but serializable. Use `value` to access the Hadoop configuration. + * + * @param value Hadoop configuration */ @DeveloperApi @Unstable class SerializableConfiguration(@transient var value: Configuration) extends Serializable { diff --git a/core/src/test/java/org/apache/spark/util/SerializableConfigurationSuite.java b/core/src/test/java/org/apache/spark/util/SerializableConfigurationSuite.java new file mode 100644 index 0000000000000..0944d681599a1 --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/SerializableConfigurationSuite.java @@ -0,0 +1,55 @@ +/* + * 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.util; + +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import static org.junit.Assert.assertEquals; + + +public class SerializableConfigurationSuite { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "SerializableConfigurationSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void testSerializableConfiguration() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + Configuration hadoopConfiguration = new Configuration(false); + hadoopConfiguration.set("test.property", "value"); + SerializableConfiguration scs = new SerializableConfiguration(hadoopConfiguration); + SerializableConfiguration actual = rdd.map(val -> scs).collect().get(0); + assertEquals(actual.value().get("test.property"), "value"); + } +} From 9e234a5434f874a9548123375df5bb7a1fee5bc5 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 20 Sep 2019 08:54:34 -0700 Subject: [PATCH 0132/1387] [MINOR][INFRA] Use java-version instead of version for GitHub Action ### What changes were proposed in this pull request? This PR use `java-version` instead of `version` for GitHub Action. More details: https://github.com/actions/setup-java/commit/204b974cf476e9709b6fab0c59007578676321c5 https://github.com/actions/setup-java/commit/ac25aeee3a8ad80e5e24d12610e451338577534f ### Why are the changes needed? The `version` property will not be supported after October 1, 2019. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #25866 from wangyum/java-version. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 99dd1d04a60d1..350f3405bc4c7 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -26,7 +26,7 @@ jobs: - name: Set up JDK ${{ matrix.java }} uses: actions/setup-java@v1 with: - version: ${{ matrix.java }} + java-version: ${{ matrix.java }} - name: Build with Maven run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" From 27d0c3f9131835cd7f992841dbcda890a7b67ec8 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 20 Sep 2019 08:57:47 -0700 Subject: [PATCH 0133/1387] [SPARK-29139][CORE][TESTS] Increase timeout to wait for executor(s) to be up in SparkContextSuite ### What changes were proposed in this pull request? This patch proposes to increase timeout to wait for executor(s) to be up in SparkContextSuite, as we observed these tests failed due to wait timeout. ### Why are the changes needed? There's some case that CI build is extremely slow which requires 3x or more time to pass the test. (https://issues.apache.org/jira/browse/SPARK-29139?focusedCommentId=16934034&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16934034) Allocating higher timeout wouldn't bring additional latency, as the code checks the condition with sleeping 10 ms per loop iteration. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A, as the case is not likely to be occurred frequently. Closes #25864 from HeartSaVioR/SPARK-29139. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- core/src/test/scala/org/apache/spark/SparkContextSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index eca49e5efdbbb..a7fb7ea27ff26 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -761,7 +761,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) // Ensure all executors has started - TestUtils.waitUntilExecutorsUp(sc, 1, 10000) + TestUtils.waitUntilExecutorsUp(sc, 1, 60000) assert(sc.resources.size === 1) assert(sc.resources.get(GPU).get.addresses === Array("5", "6")) assert(sc.resources.get(GPU).get.name === "gpu") @@ -790,7 +790,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) // Ensure all executors has started - TestUtils.waitUntilExecutorsUp(sc, 1, 10000) + TestUtils.waitUntilExecutorsUp(sc, 1, 60000) // driver gpu resources file should take precedence over the script assert(sc.resources.size === 1) assert(sc.resources.get(GPU).get.addresses === Array("0", "1", "8")) From 42050c3f4f21adaa14808e474a0db69f62671935 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 20 Sep 2019 09:59:31 -0700 Subject: [PATCH 0134/1387] [SPARK-27659][PYTHON] Allow PySpark to prefetch during toLocalIterator ### What changes were proposed in this pull request? This PR allows Python toLocalIterator to prefetch the next partition while the first partition is being collected. The PR also adds a demo micro bench mark in the examples directory, we may wish to keep this or not. ### Why are the changes needed? In https://issues.apache.org/jira/browse/SPARK-23961 / 5e79ae3b40b76e3473288830ab958fc4834dcb33 we changed PySpark to only pull one partition at a time. This is memory efficient, but if partitions take time to compute this can mean we're spending more time blocking. ### Does this PR introduce any user-facing change? A new param is added to toLocalIterator ### How was this patch tested? New unit test inside of `test_rdd.py` checks the time that the elements are evaluated at. Another test that the results remain the same are added to `test_dataframe.py`. I also ran a micro benchmark in the examples directory `prefetch.py` which shows an improvement of ~40% in this specific use case. > > 19/08/16 17:11:36 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable > Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties > Setting default log level to "WARN". > To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). > Running timers: > > [Stage 32:> (0 + 1) / 1] > Results: > > Prefetch time: > > 100.228110831 > > > Regular time: > > 188.341721614 > > > Closes #25515 from holdenk/SPARK-27659-allow-pyspark-tolocalitr-to-prefetch. Authored-by: Holden Karau Signed-off-by: Holden Karau --- .../apache/spark/api/python/PythonRDD.scala | 21 ++++++++++++++---- python/pyspark/rdd.py | 10 +++++++-- python/pyspark/sql/dataframe.py | 8 +++++-- python/pyspark/sql/tests/test_dataframe.py | 6 +++++ python/pyspark/tests/test_rdd.py | 22 +++++++++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 4 ++-- 6 files changed, 61 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 4d76ff76e6752..7cbfb71beea37 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,6 +24,7 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.duration.Duration import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration @@ -179,15 +180,22 @@ private[spark] object PythonRDD extends Logging { * data collected from this job, the secret for authentication, and a socket auth * server object that can be used to join the JVM serving thread in Python. */ - def toLocalIteratorAndServe[T](rdd: RDD[T]): Array[Any] = { + def toLocalIteratorAndServe[T](rdd: RDD[T], prefetchPartitions: Boolean = false): Array[Any] = { val handleFunc = (sock: Socket) => { val out = new DataOutputStream(sock.getOutputStream) val in = new DataInputStream(sock.getInputStream) Utils.tryWithSafeFinallyAndFailureCallbacks(block = { // Collects a partition on each iteration val collectPartitionIter = rdd.partitions.indices.iterator.map { i => - rdd.sparkContext.runJob(rdd, (iter: Iterator[Any]) => iter.toArray, Seq(i)).head + var result: Array[Any] = null + rdd.sparkContext.submitJob( + rdd, + (iter: Iterator[Any]) => iter.toArray, + Seq(i), // The partition we are evaluating + (_, res: Array[Any]) => result = res, + result) } + val prefetchIter = collectPartitionIter.buffered // Write data until iteration is complete, client stops iteration, or error occurs var complete = false @@ -196,10 +204,15 @@ private[spark] object PythonRDD extends Logging { // Read request for data, value of zero will stop iteration or non-zero to continue if (in.readInt() == 0) { complete = true - } else if (collectPartitionIter.hasNext) { + } else if (prefetchIter.hasNext) { // Client requested more data, attempt to collect the next partition - val partitionArray = collectPartitionIter.next() + val partitionFuture = prefetchIter.next() + // Cause the next job to be submitted if prefetchPartitions is enabled. + if (prefetchPartitions) { + prefetchIter.headOption + } + val partitionArray = ThreadUtils.awaitResult(partitionFuture, Duration.Inf) // Send response there is a partition to read out.writeInt(1) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index be0244b7d13e2..1edffaa4ca168 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2437,17 +2437,23 @@ def countApproxDistinct(self, relativeSD=0.05): hashRDD = self.map(lambda x: portable_hash(x) & 0xFFFFFFFF) return hashRDD._to_java_object_rdd().countApproxDistinct(relativeSD) - def toLocalIterator(self): + def toLocalIterator(self, prefetchPartitions=False): """ Return an iterator that contains all of the elements in this RDD. The iterator will consume as much memory as the largest partition in this RDD. + With prefetch it may consume up to the memory of the 2 largest partitions. + + :param prefetchPartitions: If Spark should pre-fetch the next partition + before it is needed. >>> rdd = sc.parallelize(range(10)) >>> [x for x in rdd.toLocalIterator()] [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] """ with SCCallSiteSync(self.context) as css: - sock_info = self.ctx._jvm.PythonRDD.toLocalIteratorAndServe(self._jrdd.rdd()) + sock_info = self.ctx._jvm.PythonRDD.toLocalIteratorAndServe( + self._jrdd.rdd(), + prefetchPartitions) return _local_iterator_from_socket(sock_info, self._jrdd_deserializer) def barrier(self): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 398471234d2b7..03b37fa7d0d9b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -520,16 +520,20 @@ def collect(self): @ignore_unicode_prefix @since(2.0) - def toLocalIterator(self): + def toLocalIterator(self, prefetchPartitions=False): """ Returns an iterator that contains all of the rows in this :class:`DataFrame`. The iterator will consume as much memory as the largest partition in this DataFrame. + With prefetch it may consume up to the memory of the 2 largest partitions. + + :param prefetchPartitions: If Spark should pre-fetch the next partition + before it is needed. >>> list(df.toLocalIterator()) [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] """ with SCCallSiteSync(self._sc) as css: - sock_info = self._jdf.toPythonIterator() + sock_info = self._jdf.toPythonIterator(prefetchPartitions) return _local_iterator_from_socket(sock_info, BatchedSerializer(PickleSerializer())) @ignore_unicode_prefix diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index bc4ee8814f7af..90a5415c15c27 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -690,6 +690,12 @@ def test_to_local_iterator(self): expected = df.collect() self.assertEqual(expected, list(it)) + def test_to_local_iterator_prefetch(self): + df = self.spark.range(8, numPartitions=4) + expected = df.collect() + it = df.toLocalIterator(prefetchPartitions=True) + self.assertEqual(expected, list(it)) + def test_to_local_iterator_not_fully_consumed(self): # SPARK-23961: toLocalIterator throws exception when not fully consumed # Create a DataFrame large enough so that write to socket will eventually block diff --git a/python/pyspark/tests/test_rdd.py b/python/pyspark/tests/test_rdd.py index bff080362085f..e7a7971dfc9a6 100644 --- a/python/pyspark/tests/test_rdd.py +++ b/python/pyspark/tests/test_rdd.py @@ -14,11 +14,13 @@ # See the License for the specific language governing permissions and # limitations under the License. # +from datetime import datetime, timedelta import hashlib import os import random import sys import tempfile +import time from glob import glob from py4j.protocol import Py4JJavaError @@ -68,6 +70,26 @@ def test_to_localiterator(self): it2 = rdd2.toLocalIterator() self.assertEqual([1, 2, 3], sorted(it2)) + def test_to_localiterator_prefetch(self): + # Test that we fetch the next partition in parallel + # We do this by returning the current time and: + # reading the first elem, waiting, and reading the second elem + # If not in parallel then these would be at different times + # But since they are being computed in parallel we see the time + # is "close enough" to the same. + rdd = self.sc.parallelize(range(2), 2) + times1 = rdd.map(lambda x: datetime.now()) + times2 = rdd.map(lambda x: datetime.now()) + times_iter_prefetch = times1.toLocalIterator(prefetchPartitions=True) + times_iter = times2.toLocalIterator(prefetchPartitions=False) + times_prefetch_head = next(times_iter_prefetch) + times_head = next(times_iter) + time.sleep(2) + times_next = next(times_iter) + times_prefetch_next = next(times_iter_prefetch) + self.assertTrue(times_next - times_head >= timedelta(seconds=2)) + self.assertTrue(times_prefetch_next - times_prefetch_head < timedelta(seconds=1)) + def test_save_as_textfile_with_unicode(self): # Regression test for SPARK-970 x = u"\u00A1Hola, mundo!" 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 a2f5f03d797a2..9a2d80030ee66 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 @@ -3356,9 +3356,9 @@ class Dataset[T] private[sql]( } } - private[sql] def toPythonIterator(): Array[Any] = { + private[sql] def toPythonIterator(prefetchPartitions: Boolean = false): Array[Any] = { withNewExecutionId { - PythonRDD.toLocalIteratorAndServe(javaToPython.rdd) + PythonRDD.toLocalIteratorAndServe(javaToPython.rdd, prefetchPartitions) } } From 4080c4beeb9cb27027145a37799ee8599ee51aab Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 20 Sep 2019 10:08:16 -0700 Subject: [PATCH 0135/1387] [SPARK-28937][SPARK-28936][KUBERNETES] Reduce test flakyness ### What changes were proposed in this pull request? Switch from using a Thread sleep for waiting for commands to finish to just waiting for the command to finish with a watcher & improve the error messages in the SecretsTestsSuite. ### Why are the changes needed? Currently some of the Spark Kubernetes tests have race conditions with command execution, and the frequent use of eventually makes debugging test failures difficult. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests pass after removal of thread.sleep Closes #25765 from holdenk/SPARK-28937SPARK-28936-improve-kubernetes-integration-tests. Authored-by: Holden Karau Signed-off-by: Holden Karau --- .../integrationtest/SecretsTestsSuite.scala | 51 +++++++++++++------ .../deploy/k8s/integrationtest/Utils.scala | 40 +++++++++++++-- 2 files changed, 72 insertions(+), 19 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala index cd61ea1040f35..54a9dbf07816c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.deploy.k8s.integrationtest +import java.util.Locale + import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{Pod, SecretBuilder} @@ -57,11 +59,17 @@ private[spark] trait SecretsTestsSuite { k8sSuite: KubernetesSuite => createTestSecret() sparkAppConf .set(s"spark.kubernetes.driver.secrets.$ENV_SECRET_NAME", SECRET_MOUNT_PATH) - .set(s"spark.kubernetes.driver.secretKeyRef.USERNAME", s"$ENV_SECRET_NAME:username") - .set(s"spark.kubernetes.driver.secretKeyRef.PASSWORD", s"$ENV_SECRET_NAME:password") + .set( + s"spark.kubernetes.driver.secretKeyRef.${ENV_SECRET_KEY_1_CAP}", + s"$ENV_SECRET_NAME:${ENV_SECRET_KEY_1}") + .set( + s"spark.kubernetes.driver.secretKeyRef.${ENV_SECRET_KEY_2_CAP}", + s"$ENV_SECRET_NAME:${ENV_SECRET_KEY_2}") .set(s"spark.kubernetes.executor.secrets.$ENV_SECRET_NAME", SECRET_MOUNT_PATH) - .set(s"spark.kubernetes.executor.secretKeyRef.USERNAME", s"$ENV_SECRET_NAME:username") - .set(s"spark.kubernetes.executor.secretKeyRef.PASSWORD", s"$ENV_SECRET_NAME:password") + .set(s"spark.kubernetes.executor.secretKeyRef.${ENV_SECRET_KEY_1_CAP}", + s"${ENV_SECRET_NAME}:$ENV_SECRET_KEY_1") + .set(s"spark.kubernetes.executor.secretKeyRef.${ENV_SECRET_KEY_2_CAP}", + s"${ENV_SECRET_NAME}:$ENV_SECRET_KEY_2") try { runSparkPiAndVerifyCompletion( driverPodChecker = (driverPod: Pod) => { @@ -81,19 +89,30 @@ private[spark] trait SecretsTestsSuite { k8sSuite: KubernetesSuite => } private def checkSecrets(pod: Pod): Unit = { - Eventually.eventually(TIMEOUT, INTERVAL) { - implicit val podName: String = pod.getMetadata.getName - implicit val components: KubernetesTestComponents = kubernetesTestComponents + logDebug(s"Checking secrets for ${pod}") + // Wait for the pod to become ready & have secrets provisioned + implicit val podName: String = pod.getMetadata.getName + implicit val components: KubernetesTestComponents = kubernetesTestComponents + val env = Eventually.eventually(TIMEOUT, INTERVAL) { + logDebug(s"Checking env of ${pod.getMetadata().getName()} ....") val env = Utils.executeCommand("env") - assert(env.toString.contains(ENV_SECRET_VALUE_1)) - assert(env.toString.contains(ENV_SECRET_VALUE_2)) - val fileUsernameContents = Utils - .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_1") - val filePasswordContents = Utils - .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_2") - assert(fileUsernameContents.toString.trim.equals(ENV_SECRET_VALUE_1)) - assert(filePasswordContents.toString.trim.equals(ENV_SECRET_VALUE_2)) + assert(!env.isEmpty) + env } + env.toString should include (s"${ENV_SECRET_KEY_1_CAP}=$ENV_SECRET_VALUE_1") + env.toString should include (s"${ENV_SECRET_KEY_2_CAP}=$ENV_SECRET_VALUE_2") + + // Make sure our secret files are mounted correctly + val files = Utils.executeCommand("ls", s"$SECRET_MOUNT_PATH") + files should include (ENV_SECRET_KEY_1) + files should include (ENV_SECRET_KEY_2) + // Validate the contents + val fileUsernameContents = Utils + .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_1") + fileUsernameContents.toString.trim should equal(ENV_SECRET_VALUE_1) + val filePasswordContents = Utils + .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_2") + filePasswordContents.toString.trim should equal(ENV_SECRET_VALUE_2) } } @@ -102,6 +121,8 @@ private[spark] object SecretsTestsSuite { val SECRET_MOUNT_PATH = "/etc/secret" val ENV_SECRET_KEY_1 = "username" val ENV_SECRET_KEY_2 = "password" + val ENV_SECRET_KEY_1_CAP = ENV_SECRET_KEY_1.toUpperCase(Locale.ROOT) + val ENV_SECRET_KEY_2_CAP = ENV_SECRET_KEY_2.toUpperCase(Locale.ROOT) val ENV_SECRET_VALUE_1 = "secretusername" val ENV_SECRET_VALUE_2 = "secretpassword" } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala index a687a1bca1800..9f85805b9d315 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala @@ -18,9 +18,12 @@ package org.apache.spark.deploy.k8s.integrationtest import java.io.{Closeable, File, PrintWriter} import java.nio.file.{Files, Path} +import java.util.concurrent.CountDownLatch import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.client.dsl.ExecListener +import okhttp3.Response import org.apache.commons.io.output.ByteArrayOutputStream import org.apache.spark.{SPARK_VERSION, SparkException} @@ -45,20 +48,49 @@ object Utils extends Logging { implicit podName: String, kubernetesTestComponents: KubernetesTestComponents): String = { val out = new ByteArrayOutputStream() - val watch = kubernetesTestComponents + val pod = kubernetesTestComponents .kubernetesClient .pods() .withName(podName) + // Avoid timing issues by looking for open/close + class ReadyListener extends ExecListener { + val openLatch: CountDownLatch = new CountDownLatch(1) + val closeLatch: CountDownLatch = new CountDownLatch(1) + + override def onOpen(response: Response) { + openLatch.countDown() + } + + override def onClose(a: Int, b: String) { + closeLatch.countDown() + } + + override def onFailure(e: Throwable, r: Response) { + } + + def waitForInputStreamToConnect(): Unit = { + openLatch.await() + } + + def waitForClose(): Unit = { + closeLatch.await() + } + } + val listener = new ReadyListener() + val watch = pod .readingInput(System.in) .writingOutput(out) .writingError(System.err) .withTTY() + .usingListener(listener) .exec(cmd.toArray: _*) - // wait to get some result back - Thread.sleep(1000) + // under load sometimes the stdout isn't connected by the time we try to read from it. + listener.waitForInputStreamToConnect() + listener.waitForClose() watch.close() out.flush() - out.toString() + val result = out.toString() + result } def createTempFile(contents: String, hostPath: String): String = { From 8c8016a152a7f1dce96d33994a3b541c0bb06f00 Mon Sep 17 00:00:00 2001 From: Xianjin YE Date: Sat, 21 Sep 2019 08:09:19 +0900 Subject: [PATCH 0136/1387] [SPARK-21045][PYTHON] Allow non-ascii string as an exception message from python execution in Python 2 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR allows non-ascii string as an exception message in Python 2 by explicitly en/decoding in case of `str` in Python 2. ### Why are the changes needed? Previously PySpark will hang when the `UnicodeDecodeError` occurs and the real exception cannot be passed to the JVM side. See the reproducer as below: ```python def f(): raise Exception("中") spark = SparkSession.builder.master('local').getOrCreate() spark.sparkContext.parallelize([1]).map(lambda x: f()).count() ``` ### Does this PR introduce any user-facing change? User may not observe hanging for the similar cases. ### How was this patch tested? Added a new test and manually checking. This pr is based on #18324, credits should also go to dataknocker. To make lint-python happy for python3, it also includes a followup fix for #25814 Closes #25847 from advancedxy/python_exception_19926_and_21045. Authored-by: Xianjin YE Signed-off-by: HyukjinKwon --- python/pyspark/sql/utils.py | 3 +++ python/pyspark/tests/test_worker.py | 15 +++++++++++++++ python/pyspark/worker.py | 9 ++++++++- 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 996b7dd59ce9f..83afafdd8b138 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -18,6 +18,9 @@ import py4j import sys +if sys.version_info.major >= 3: + unicode = str + class CapturedException(Exception): def __init__(self, desc, stackTrace, cause=None): diff --git a/python/pyspark/tests/test_worker.py b/python/pyspark/tests/test_worker.py index 18fde17f4a064..ccbe21f3a6f38 100644 --- a/python/pyspark/tests/test_worker.py +++ b/python/pyspark/tests/test_worker.py @@ -1,3 +1,4 @@ +# -*- encoding: utf-8 -*- # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -150,6 +151,20 @@ def test_with_different_versions_of_python(self): finally: self.sc.pythonVer = version + def test_python_exception_non_hanging(self): + # SPARK-21045: exceptions with no ascii encoding shall not hanging PySpark. + try: + def f(): + raise Exception("exception with 中 and \xd6\xd0") + + self.sc.parallelize([1]).map(lambda x: f()).count() + except Py4JJavaError as e: + if sys.version_info.major < 3: + # we have to use unicode here to avoid UnicodeDecodeError + self.assertRegexpMatches(unicode(e).encode("utf-8"), "exception with 中") + else: + self.assertRegexpMatches(str(e), "exception with 中") + class WorkerReuseTest(PySparkTestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 086202de2c68b..698193d6bdd87 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -598,8 +598,15 @@ def process(): process() except Exception: try: + exc_info = traceback.format_exc() + if isinstance(exc_info, bytes): + # exc_info may contains other encoding bytes, replace the invalid bytes and convert + # it back to utf-8 again + exc_info = exc_info.decode("utf-8", "replace").encode("utf-8") + else: + exc_info = exc_info.encode("utf-8") write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) - write_with_length(traceback.format_exc().encode("utf-8"), outfile) + write_with_length(exc_info, outfile) except IOError: # JVM close the socket pass From abc88deeed164bf48eff1d1504e141c1cf5afade Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sat, 21 Sep 2019 08:26:30 +0900 Subject: [PATCH 0137/1387] [SPARK-29063][SQL] Modify fillValue approach to support joined dataframe ### What changes were proposed in this pull request? Modify the approach in `DataFrameNaFunctions.fillValue`, the new one uses `df.withColumns` which only address the columns need to be filled. After this change, there are no more ambiguous fileds detected for joined dataframe. ### Why are the changes needed? Before this change, when you have a joined table that has the same field name from both original table, fillna will fail even if you specify a subset that does not include the 'ambiguous' fields. ``` scala> val df1 = Seq(("f1-1", "f2", null), ("f1-2", null, null), ("f1-3", "f2", "f3-1"), ("f1-4", "f2", "f3-1")).toDF("f1", "f2", "f3") scala> val df2 = Seq(("f1-1", null, null), ("f1-2", "f2", null), ("f1-3", "f2", "f4-1")).toDF("f1", "f2", "f4") scala> val df_join = df1.alias("df1").join(df2.alias("df2"), Seq("f1"), joinType="left_outer") scala> df_join.na.fill("", cols=Seq("f4")) org.apache.spark.sql.AnalysisException: Reference 'f2' is ambiguous, could be: df1.f2, df2.f2.; ``` ### Does this PR introduce any user-facing change? Yes, fillna operation will pass and give the right answer for a joined table. ### How was this patch tested? Local test and newly added UT. Closes #25768 from xuanyuanking/SPARK-29063. Lead-authored-by: Yuanjian Li Co-authored-by: Xiao Li Signed-off-by: HyukjinKwon --- .../spark/sql/DataFrameNaFunctions.scala | 10 ++---- .../spark/sql/DataFrameNaFunctionsSuite.scala | 35 +++++++++++++++++++ 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 53e9f810d7c85..6dd21f114c902 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -488,7 +488,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { } val columnEquals = df.sparkSession.sessionState.analyzer.resolver - val projections = df.schema.fields.map { f => + val filledColumns = df.schema.fields.filter { f => val typeMatches = (targetType, f.dataType) match { case (NumericType, dt) => dt.isInstanceOf[NumericType] case (StringType, dt) => dt == StringType @@ -497,12 +497,8 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { throw new IllegalArgumentException(s"$targetType is not matched at fillValue") } // Only fill if the column is part of the cols list. - if (typeMatches && cols.exists(col => columnEquals(f.name, col))) { - fillCol[T](f, value) - } else { - df.col(f.name) - } + typeMatches && cols.exists(col => columnEquals(f.name, col)) } - df.select(projections : _*) + df.withColumns(filledColumns.map(_.name), filledColumns.map(fillCol[T](_, value))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index aeee4577d3483..75642a0bd9325 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -231,6 +231,41 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { } } + def createDFsWithSameFieldsName(): (DataFrame, DataFrame) = { + val df1 = Seq( + ("f1-1", "f2", null), + ("f1-2", null, null), + ("f1-3", "f2", "f3-1"), + ("f1-4", "f2", "f3-1") + ).toDF("f1", "f2", "f3") + val df2 = Seq( + ("f1-1", null, null), + ("f1-2", "f2", null), + ("f1-3", "f2", "f4-1") + ).toDF("f1", "f2", "f4") + (df1, df2) + } + + test("fill unambiguous field for join operation") { + val (df1, df2) = createDFsWithSameFieldsName() + val joined_df = df1.join(df2, Seq("f1"), joinType = "left_outer") + checkAnswer(joined_df.na.fill("", cols = Seq("f4")), + Row("f1-1", "f2", null, null, "") :: + Row("f1-2", null, null, "f2", "") :: + Row("f1-3", "f2", "f3-1", "f2", "f4-1") :: + Row("f1-4", "f2", "f3-1", null, "") :: Nil) + } + + test("fill ambiguous field for join operation") { + val (df1, df2) = createDFsWithSameFieldsName() + val joined_df = df1.join(df2, Seq("f1"), joinType = "left_outer") + + val message = intercept[AnalysisException] { + joined_df.na.fill("", cols = Seq("f2")) + }.getMessage + assert(message.contains("Reference 'f2' is ambiguous")) + } + test("replace") { val input = createDF() From 4a89fa1cd1d3f25abc02be83a90653f465a452aa Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 21 Sep 2019 08:40:49 +0900 Subject: [PATCH 0138/1387] [SPARK-29196][DOCS] Add JDK11 support to the document ### What changes were proposed in this pull request? This PRs add Java 11 version to the document. ### Why are the changes needed? Apache Spark 3.0.0 starts to support JDK11 officially. ### Does this PR introduce any user-facing change? Yes. ![jdk11](https://user-images.githubusercontent.com/9700541/65364063-39204580-dbc4-11e9-982b-fc1552be2ec5.png) ### How was this patch tested? Manually. Doc generation. Closes #25875 from dongjoon-hyun/SPARK-29196. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- docs/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/index.md b/docs/index.md index 6cfbbaed20028..edb1c421fb794 100644 --- a/docs/index.md +++ b/docs/index.md @@ -46,7 +46,7 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy locally on one machine --- all you need is to have `java` installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. -Spark runs on Java 8, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. +Spark runs on Java 8/11, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. Python 2 support is deprecated as of Spark 3.0.0. R prior to version 3.4 support is deprecated as of Spark 3.0.0. For the Scala API, Spark {{site.SPARK_VERSION}} From c764dd6dd725647d3dca2493875429513f706e86 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 20 Sep 2019 19:22:46 -0500 Subject: [PATCH 0139/1387] [SPARK-29144][ML] Binarizer handle sparse vectors incorrectly with negative threshold ### What changes were proposed in this pull request? if threshold<0, convert implict 0 to 1, althought this will break sparsity ### Why are the changes needed? if `threshold<0`, current impl deal with sparse vector incorrectly. See JIRA [SPARK-29144](https://issues.apache.org/jira/browse/SPARK-29144) and [Scikit-Learn's Binarizer](https://scikit-learn.org/stable/modules/generated/sklearn.preprocessing.Binarizer.html) ('Threshold may not be less than 0 for operations on sparse matrices.') for details. ### Does this PR introduce any user-facing change? no ### How was this patch tested? added testsuite Closes #25829 from zhengruifeng/binarizer_throw_exception_sparse_vector. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../apache/spark/ml/feature/Binarizer.scala | 48 +++++++++++-------- .../spark/ml/feature/BinarizerSuite.scala | 14 ++++++ 2 files changed, 43 insertions(+), 19 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 2b0862c60fdf7..c4daf64dfc5f0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -75,30 +75,40 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) val schema = dataset.schema val inputType = schema($(inputCol)).dataType val td = $(threshold) + val metadata = outputSchema($(outputCol)).metadata - val binarizerDouble = udf { in: Double => if (in > td) 1.0 else 0.0 } - val binarizerVector = udf { (data: Vector) => - val indices = ArrayBuilder.make[Int] - val values = ArrayBuilder.make[Double] - - data.foreachActive { (index, value) => - if (value > td) { - indices += index - values += 1.0 + val binarizerUDF = inputType match { + case DoubleType => + udf { in: Double => if (in > td) 1.0 else 0.0 } + + case _: VectorUDT if td >= 0 => + udf { vector: Vector => + val indices = ArrayBuilder.make[Int] + val values = ArrayBuilder.make[Double] + vector.foreachActive { (index, value) => + if (value > td) { + indices += index + values += 1.0 + } + } + Vectors.sparse(vector.size, indices.result(), values.result()).compressed } - } - Vectors.sparse(data.size, indices.result(), values.result()).compressed + case _: VectorUDT if td < 0 => + this.logWarning(s"Binarization operations on sparse dataset with negative threshold " + + s"$td will build a dense output, so take care when applying to sparse input.") + udf { vector: Vector => + val values = Array.fill(vector.size)(1.0) + vector.foreachActive { (index, value) => + if (value <= td) { + values(index) = 0.0 + } + } + Vectors.dense(values).compressed + } } - val metadata = outputSchema($(outputCol)).metadata - - inputType match { - case DoubleType => - dataset.select(col("*"), binarizerDouble(col($(inputCol))).as($(outputCol), metadata)) - case _: VectorUDT => - dataset.select(col("*"), binarizerVector(col($(inputCol))).as($(outputCol), metadata)) - } + dataset.withColumn($(outputCol), binarizerUDF(col($(inputCol))), metadata) } @Since("1.4.0") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala index 05d4a6ee2dabf..91bec50fb904f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala @@ -101,6 +101,20 @@ class BinarizerSuite extends MLTest with DefaultReadWriteTest { } } + test("Binarizer should support sparse vector with negative threshold") { + val data = Seq( + (Vectors.sparse(3, Array(1), Array(0.5)), Vectors.dense(Array(1.0, 1.0, 1.0))), + (Vectors.dense(Array(0.0, 0.5, 0.0)), Vectors.dense(Array(1.0, 1.0, 1.0)))) + val df = data.toDF("feature", "expected") + val binarizer = new Binarizer() + .setInputCol("feature") + .setOutputCol("binarized_feature") + .setThreshold(-0.5) + binarizer.transform(df).select("binarized_feature", "expected").collect().foreach { + case Row(x: Vector, y: Vector) => + assert(x == y, "The feature value is not correct after binarization.") + } + } test("read/write") { val t = new Binarizer() From ff3a737c759b21d166053a0fbb6ad539accb8e49 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 20 Sep 2019 19:41:25 -0700 Subject: [PATCH 0140/1387] [SPARK-29192][TESTS] Extend BenchmarkBase to write JDK9+ results separately ### What changes were proposed in this pull request? This PR aims to extend the existing benchmarks to save JDK9+ result separately. All `core` module benchmark test results are added. I'll run the other test suites in another PR. After regenerating all results, we will check JDK11 performance regressions. ### Why are the changes needed? From Apache Spark 3.0, we support both JDK8 and JDK11. We need to have a way to find the performance regression. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually run the benchmark. Closes #25873 from dongjoon-hyun/SPARK-JDK11-PERF. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../CoalescedRDDBenchmark-jdk11-results.txt | 40 +++++++++++++++++ .../KryoBenchmark-jdk11-results.txt | 28 ++++++++++++ .../KryoSerializerBenchmark-jdk11-results.txt | 12 +++++ ...PropertiesCloneBenchmark-jdk11-results.txt | 40 +++++++++++++++++ .../XORShiftRandomBenchmark-jdk11-results.txt | 44 +++++++++++++++++++ .../spark/benchmark/BenchmarkBase.scala | 5 ++- 6 files changed, 168 insertions(+), 1 deletion(-) create mode 100644 core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/KryoBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt diff --git a/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt b/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..f43e083060e34 --- /dev/null +++ b/core/benchmarks/CoalescedRDDBenchmark-jdk11-results.txt @@ -0,0 +1,40 @@ +================================================================================================ +Coalesced RDD , large scale +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Coalesced RDD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Coalesce Num Partitions: 100 Num Hosts: 1 455 623 151 0.2 4547.4 1.0X +Coalesce Num Partitions: 100 Num Hosts: 5 340 409 65 0.3 3397.1 1.3X +Coalesce Num Partitions: 100 Num Hosts: 10 292 363 95 0.3 2923.3 1.6X +Coalesce Num Partitions: 100 Num Hosts: 20 307 320 11 0.3 3069.8 1.5X +Coalesce Num Partitions: 100 Num Hosts: 40 333 368 55 0.3 3329.1 1.4X +Coalesce Num Partitions: 100 Num Hosts: 80 286 338 63 0.3 2862.5 1.6X +Coalesce Num Partitions: 500 Num Hosts: 1 769 837 59 0.1 7693.5 0.6X +Coalesce Num Partitions: 500 Num Hosts: 5 427 461 31 0.2 4268.5 1.1X +Coalesce Num Partitions: 500 Num Hosts: 10 372 389 27 0.3 3722.2 1.2X +Coalesce Num Partitions: 500 Num Hosts: 20 347 365 31 0.3 3468.5 1.3X +Coalesce Num Partitions: 500 Num Hosts: 40 335 336 1 0.3 3347.3 1.4X +Coalesce Num Partitions: 500 Num Hosts: 80 329 360 49 0.3 3294.5 1.4X +Coalesce Num Partitions: 1000 Num Hosts: 1 1254 1292 47 0.1 12538.6 0.4X +Coalesce Num Partitions: 1000 Num Hosts: 5 518 553 47 0.2 5177.0 0.9X +Coalesce Num Partitions: 1000 Num Hosts: 10 394 432 42 0.3 3937.3 1.2X +Coalesce Num Partitions: 1000 Num Hosts: 20 341 381 44 0.3 3414.4 1.3X +Coalesce Num Partitions: 1000 Num Hosts: 40 313 358 48 0.3 3134.9 1.5X +Coalesce Num Partitions: 1000 Num Hosts: 80 335 360 38 0.3 3347.0 1.4X +Coalesce Num Partitions: 5000 Num Hosts: 1 3937 4066 156 0.0 39375.0 0.1X +Coalesce Num Partitions: 5000 Num Hosts: 5 1413 1453 40 0.1 14133.4 0.3X +Coalesce Num Partitions: 5000 Num Hosts: 10 826 861 49 0.1 8255.2 0.6X +Coalesce Num Partitions: 5000 Num Hosts: 20 542 609 58 0.2 5423.3 0.8X +Coalesce Num Partitions: 5000 Num Hosts: 40 410 470 64 0.2 4101.0 1.1X +Coalesce Num Partitions: 5000 Num Hosts: 80 352 427 69 0.3 3515.3 1.3X +Coalesce Num Partitions: 10000 Num Hosts: 1 7101 7151 54 0.0 71007.4 0.1X +Coalesce Num Partitions: 10000 Num Hosts: 5 2540 2582 59 0.0 25396.2 0.2X +Coalesce Num Partitions: 10000 Num Hosts: 10 1378 1432 48 0.1 13781.4 0.3X +Coalesce Num Partitions: 10000 Num Hosts: 20 829 867 66 0.1 8286.8 0.5X +Coalesce Num Partitions: 10000 Num Hosts: 40 573 630 49 0.2 5730.2 0.8X +Coalesce Num Partitions: 10000 Num Hosts: 80 438 449 9 0.2 4382.5 1.0X + + diff --git a/core/benchmarks/KryoBenchmark-jdk11-results.txt b/core/benchmarks/KryoBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..0afa06519d2f4 --- /dev/null +++ b/core/benchmarks/KryoBenchmark-jdk11-results.txt @@ -0,0 +1,28 @@ +================================================================================================ +Benchmark Kryo Unsafe vs safe Serialization +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Benchmark Kryo Unsafe vs safe Serialization: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +basicTypes: Int with unsafe:true 324 329 2 3.1 324.0 1.0X +basicTypes: Long with unsafe:true 353 355 1 2.8 353.0 0.9X +basicTypes: Float with unsafe:true 336 338 1 3.0 336.4 1.0X +basicTypes: Double with unsafe:true 347 347 1 2.9 346.5 0.9X +Array: Int with unsafe:true 4 4 0 265.4 3.8 86.0X +Array: Long with unsafe:true 6 7 0 157.3 6.4 51.0X +Array: Float with unsafe:true 4 4 0 268.8 3.7 87.1X +Array: Double with unsafe:true 6 7 0 157.5 6.3 51.0X +Map of string->Double with unsafe:true 52 52 1 19.3 51.8 6.3X +basicTypes: Int with unsafe:false 357 358 1 2.8 357.2 0.9X +basicTypes: Long with unsafe:false 387 388 0 2.6 387.4 0.8X +basicTypes: Float with unsafe:false 356 357 1 2.8 356.0 0.9X +basicTypes: Double with unsafe:false 371 372 1 2.7 371.0 0.9X +Array: Int with unsafe:false 24 24 0 41.3 24.2 13.4X +Array: Long with unsafe:false 37 38 0 26.8 37.4 8.7X +Array: Float with unsafe:false 11 11 0 94.9 10.5 30.8X +Array: Double with unsafe:false 18 18 0 55.2 18.1 17.9X +Map of string->Double with unsafe:false 55 55 0 18.2 55.1 5.9X + + diff --git a/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt b/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..0e6d32649840f --- /dev/null +++ b/core/benchmarks/KryoSerializerBenchmark-jdk11-results.txt @@ -0,0 +1,12 @@ +================================================================================================ +Benchmark KryoPool vs old"pool of 1" implementation +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Benchmark KryoPool vs old"pool of 1" implementation: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +KryoPool:true 6524 9149 NaN 0.0 13047076.5 1.0X +KryoPool:false 12855 16469 663 0.0 25709170.6 0.5X + + diff --git a/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt b/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..1cd08a7748109 --- /dev/null +++ b/core/benchmarks/PropertiesCloneBenchmark-jdk11-results.txt @@ -0,0 +1,40 @@ +================================================================================================ +Properties Cloning +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Empty Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 0 0 0 0.1 13755.0 1.0X +Utils.cloneProperties 0 0 0 3.5 285.0 48.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +System Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 0 0 0 0.0 191892.0 1.0X +Utils.cloneProperties 0 0 0 0.2 6027.0 31.8X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Small Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 1 1 0 0.0 721334.0 1.0X +Utils.cloneProperties 0 0 0 0.2 5237.0 137.7X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Medium Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 3 3 0 0.0 3006554.0 1.0X +Utils.cloneProperties 0 0 0 0.0 27042.0 111.2X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Large Properties: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +SerializationUtils.clone 6 6 0 0.0 5864452.0 1.0X +Utils.cloneProperties 0 0 0 0.0 53760.0 109.1X + + diff --git a/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt b/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..3b6091cc740ac --- /dev/null +++ b/core/benchmarks/XORShiftRandomBenchmark-jdk11-results.txt @@ -0,0 +1,44 @@ +================================================================================================ +Pseudo random +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextInt: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +java.util.Random 1357 1358 0 73.7 13.6 1.0X +XORShiftRandom 228 228 0 438.0 2.3 5.9X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextLong: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +java.util.Random 2718 2719 1 36.8 27.2 1.0X +XORShiftRandom 632 633 0 158.1 6.3 4.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextDouble: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +java.util.Random 2722 2723 0 36.7 27.2 1.0X +XORShiftRandom 632 632 0 158.3 6.3 4.3X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextGaussian: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +java.util.Random 6979 6979 1 14.3 69.8 1.0X +XORShiftRandom 5183 5183 0 19.3 51.8 1.3X + + +================================================================================================ +hash seed +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash seed: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +XORShiftRandom.hashSeed 38 39 1 263.9 3.8 1.0X + + diff --git a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala index a6666db4e95c3..55e34b32fe0d4 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala @@ -21,6 +21,7 @@ import java.io.{File, FileOutputStream, OutputStream} /** * A base class for generate benchmark results to a file. + * For JDK9+, JDK major version number is added to the file names to distingush the results. */ abstract class BenchmarkBase { var output: Option[OutputStream] = None @@ -43,7 +44,9 @@ abstract class BenchmarkBase { def main(args: Array[String]): Unit = { val regenerateBenchmarkFiles: Boolean = System.getenv("SPARK_GENERATE_BENCHMARK_FILES") == "1" if (regenerateBenchmarkFiles) { - val resultFileName = s"${this.getClass.getSimpleName.replace("$", "")}-results.txt" + val version = System.getProperty("java.version").split("\\D+")(0).toInt + val jdkString = if (version > 8) s"-jdk$version" else "" + val resultFileName = s"${this.getClass.getSimpleName.replace("$", "")}$jdkString-results.txt" val file = new File(s"benchmarks/$resultFileName") if (!file.exists()) { file.createNewFile() From 252b6cf3c926724ba5c4ddc3659df2ad20bb4734 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 20 Sep 2019 20:28:56 -0700 Subject: [PATCH 0141/1387] [SPARK-29187][SQL] Return null from `date_part()` for the null `field` ### What changes were proposed in this pull request? In the PR, I propose to change behavior of the `date_part()` function in handling `null` field, and make it the same as PostgreSQL has. If `field` parameter is `null`, the function should return `null` of the `double` type as PostgreSQL does: ```sql # select date_part(null, date '2019-09-20'); date_part ----------- (1 row) # select pg_typeof(date_part(null, date '2019-09-20')); pg_typeof ------------------ double precision (1 row) ``` ### Why are the changes needed? The `date_part()` function was added to maintain feature parity with PostgreSQL but current behavior of the function is different in handling null as `field`. ### Does this PR introduce any user-facing change? Yes. Before: ```sql spark-sql> select date_part(null, date'2019-09-20'); Error in query: null; line 1 pos 7 ``` After: ```sql spark-sql> select date_part(null, date'2019-09-20'); NULL ``` ### How was this patch tested? Add new tests to `DateFunctionsSuite for 2 cases: - `field` = `null`, `source` = a date literal - `field` = `null`, `source` = a date column Closes #25865 from MaxGekk/date_part-null. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../catalyst/expressions/datetimeExpressions.scala | 13 +++++++++---- .../test/resources/sql-tests/inputs/date_part.sql | 2 ++ .../resources/sql-tests/results/date_part.sql.out | 10 +++++++++- .../org/apache/spark/sql/DateFunctionsSuite.scala | 10 ++++++++++ 4 files changed, 30 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 89a6d23b1d73d..592b9de83d9a0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -2053,10 +2053,15 @@ case class DatePart(field: Expression, source: Expression, child: Expression) if (!field.foldable) { throw new AnalysisException("The field parameter needs to be a foldable string value.") } - val fieldStr = field.eval().asInstanceOf[UTF8String].toString - DatePart.parseExtractField(fieldStr, source, { - throw new AnalysisException(s"Literals of type '$fieldStr' are currently not supported.") - }) + val fieldEval = field.eval() + if (fieldEval == null) { + Literal(null, DoubleType) + } else { + val fieldStr = fieldEval.asInstanceOf[UTF8String].toString + DatePart.parseExtractField(fieldStr, source, { + throw new AnalysisException(s"Literals of type '$fieldStr' are currently not supported.") + }) + } }) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql index cb3d966281009..fd0fb50f71460 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql @@ -66,3 +66,5 @@ select date_part('secs', c) from t; select date_part('not_supported', c) from t; select date_part(c, c) from t; + +select date_part(null, c) from t; diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index c59dfdbd3da34..776786850e9da 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 51 +-- Number of queries: 52 -- !query 0 @@ -410,3 +410,11 @@ struct<> -- !query 50 output org.apache.spark.sql.AnalysisException The field parameter needs to be a foldable string value.;; line 1 pos 7 + + +-- !query 51 +select date_part(null, c) from t +-- !query 51 schema +struct +-- !query 51 output +NULL diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 2fef05f97e57c..99189a96b2995 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} import org.apache.spark.unsafe.types.CalendarInterval class DateFunctionsSuite extends QueryTest with SharedSparkSession { @@ -796,4 +797,13 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { Seq(Row(Instant.parse(timestamp)))) } } + + test("handling null field by date_part") { + val input = Seq(Date.valueOf("2019-09-20")).toDF("d") + Seq("date_part(null, d)", "date_part(null, date'2019-09-20')").foreach { expr => + val df = input.selectExpr(expr) + assert(df.schema.headOption.get.dataType == DoubleType) + checkAnswer(df, Row(null)) + } + } } From a9ae262cf279bc607cb842204c717257c259d82b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 20 Sep 2019 20:31:26 -0700 Subject: [PATCH 0142/1387] [SPARK-28772][BUILD][MLLIB] Update breeze to 1.0 ### What changes were proposed in this pull request? Update breeze dependency to 1.0. ### Why are the changes needed? Breeze 1.0 supports Scala 2.13 and has a few bug fixes. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #25874 from srowen/SPARK-28772. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- LICENSE-binary | 15 +- dev/deps/spark-deps-hadoop-2.7 | 20 +- dev/deps/spark-deps-hadoop-3.2 | 20 +- licenses-binary/LICENSE-JLargeArrays.txt | 23 ++ licenses-binary/LICENSE-JTransforms.txt | 23 ++ licenses-binary/LICENSE-jtransforms.html | 388 ------------------ .../org/apache/spark/ml/feature/DCT.scala | 2 +- .../apache/spark/ml/feature/JavaDCTSuite.java | 2 +- .../apache/spark/ml/feature/DCTSuite.scala | 2 +- pom.xml | 8 +- 10 files changed, 84 insertions(+), 419 deletions(-) create mode 100644 licenses-binary/LICENSE-JLargeArrays.txt create mode 100644 licenses-binary/LICENSE-JTransforms.txt delete mode 100644 licenses-binary/LICENSE-jtransforms.html diff --git a/LICENSE-binary b/LICENSE-binary index ba20eea118687..16da39145d050 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -396,6 +396,8 @@ org.codehaus.janino:commons-compiler org.codehaus.janino:janino jline:jline org.jodd:jodd-core +com.github.wendykierp:JTransforms +pl.edu.icm:JLargeArrays BSD 3-Clause @@ -433,8 +435,12 @@ is distributed under the 3-Clause BSD license. MIT License ----------- -org.spire-math:spire-macros_2.12 -org.spire-math:spire_2.12 +org.typelevel:spire_2.12 +org.typelevel:spire-macros_2.12 +org.typelevel:spire-platform_2.12 +org.typelevel:spire-util_2.12 +org.typelevel:algebra_2.12:jar +org.typelevel:cats-kernel_2.12 org.typelevel:machinist_2.12 net.razorvine:pyrolite org.slf4j:jcl-over-slf4j @@ -496,11 +502,6 @@ Eclipse Public License (EPL) 2.0 jakarta.annotation:jakarta-annotation-api https://projects.eclipse.org/projects/ee4j.ca jakarta.ws.rs:jakarta.ws.rs-api https://github.com/eclipse-ee4j/jaxrs-api -Mozilla Public License (MPL) 1.1 --------------------------------- - -com.github.rwl:jtransforms https://sourceforge.net/projects/jtransforms/ - Python Software Foundation License ---------------------------------- diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index cae2614128b1a..2da4c9e44b29e 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -1,8 +1,11 @@ +JLargeArrays-1.5.jar +JTransforms-3.1.jar JavaEWAH-0.3.2.jar RoaringBitmap-0.7.45.jar ST4-4.0.4.jar activation-1.1.1.jar aircompressor-0.10.jar +algebra_2.12-2.0.0-M2.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.1.jar @@ -22,8 +25,9 @@ avro-1.8.2.jar avro-ipc-1.8.2.jar avro-mapred-1.8.2-hadoop2.jar bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-0.13.2.jar -breeze_2.12-0.13.2.jar +breeze-macros_2.12-1.0.jar +breeze_2.12-1.0.jar +cats-kernel_2.12-2.0.0-M4.jar chill-java-0.9.3.jar chill_2.12-0.9.3.jar commons-beanutils-1.9.3.jar @@ -130,7 +134,6 @@ json4s-scalap_2.12-3.6.6.jar jsp-api-2.1.jar jsr305-3.0.0.jar jta-1.1.jar -jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kryo-shaded-4.0.2.jar kubernetes-client-4.4.2.jar @@ -142,7 +145,7 @@ libthrift-0.12.0.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar lz4-java-1.6.0.jar -machinist_2.12-0.6.1.jar +machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar metrics-core-3.1.5.jar @@ -172,20 +175,23 @@ parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar +scala-collection-compat_2.12-2.1.1.jar scala-compiler-2.12.10.jar scala-library-2.12.10.jar scala-parser-combinators_2.12-1.1.0.jar scala-reflect-2.12.10.jar scala-xml_2.12-1.2.0.jar -shapeless_2.12-2.3.2.jar +shapeless_2.12-2.3.3.jar shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.23.jar snappy-0.2.jar snappy-java-1.1.7.3.jar -spire-macros_2.12-0.13.0.jar -spire_2.12-0.13.0.jar +spire-macros_2.12-0.17.0-M1.jar +spire-platform_2.12-0.17.0-M1.jar +spire-util_2.12-0.17.0-M1.jar +spire_2.12-0.17.0-M1.jar stax-api-1.0-2.jar stax-api-1.0.1.jar stream-2.9.6.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 21df5a70e7f8d..2226baeadfba1 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -1,9 +1,12 @@ +JLargeArrays-1.5.jar +JTransforms-3.1.jar JavaEWAH-0.3.2.jar RoaringBitmap-0.7.45.jar ST4-4.0.4.jar accessors-smart-1.2.jar activation-1.1.1.jar aircompressor-0.10.jar +algebra_2.12-2.0.0-M2.jar antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.1.jar @@ -20,8 +23,9 @@ avro-1.8.2.jar avro-ipc-1.8.2.jar avro-mapred-1.8.2-hadoop2.jar bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-0.13.2.jar -breeze_2.12-0.13.2.jar +breeze-macros_2.12-1.0.jar +breeze_2.12-1.0.jar +cats-kernel_2.12-2.0.0-M4.jar chill-java-0.9.3.jar chill_2.12-0.9.3.jar commons-beanutils-1.9.3.jar @@ -132,7 +136,6 @@ json4s-scalap_2.12-3.6.6.jar jsp-api-2.1.jar jsr305-3.0.0.jar jta-1.1.jar -jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kerb-admin-1.0.1.jar kerb-client-1.0.1.jar @@ -158,7 +161,7 @@ libthrift-0.12.0.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar lz4-java-1.6.0.jar -machinist_2.12-0.6.1.jar +machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar metrics-core-3.1.5.jar @@ -191,20 +194,23 @@ protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.30.jar re2j-1.1.jar +scala-collection-compat_2.12-2.1.1.jar scala-compiler-2.12.10.jar scala-library-2.12.10.jar scala-parser-combinators_2.12-1.1.0.jar scala-reflect-2.12.10.jar scala-xml_2.12-1.2.0.jar -shapeless_2.12-2.3.2.jar +shapeless_2.12-2.3.3.jar shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar snakeyaml-1.23.jar snappy-0.2.jar snappy-java-1.1.7.3.jar -spire-macros_2.12-0.13.0.jar -spire_2.12-0.13.0.jar +spire-macros_2.12-0.17.0-M1.jar +spire-platform_2.12-0.17.0-M1.jar +spire-util_2.12-0.17.0-M1.jar +spire_2.12-0.17.0-M1.jar stax-api-1.0.1.jar stax2-api-3.1.4.jar stream-2.9.6.jar diff --git a/licenses-binary/LICENSE-JLargeArrays.txt b/licenses-binary/LICENSE-JLargeArrays.txt new file mode 100644 index 0000000000000..304e724556984 --- /dev/null +++ b/licenses-binary/LICENSE-JLargeArrays.txt @@ -0,0 +1,23 @@ +JLargeArrays +Copyright (C) 2013 onward University of Warsaw, ICM +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/licenses-binary/LICENSE-JTransforms.txt b/licenses-binary/LICENSE-JTransforms.txt new file mode 100644 index 0000000000000..2f0589f76da7d --- /dev/null +++ b/licenses-binary/LICENSE-JTransforms.txt @@ -0,0 +1,23 @@ +JTransforms +Copyright (c) 2007 onward, Piotr Wendykier +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/licenses-binary/LICENSE-jtransforms.html b/licenses-binary/LICENSE-jtransforms.html deleted file mode 100644 index 351c17412357b..0000000000000 --- a/licenses-binary/LICENSE-jtransforms.html +++ /dev/null @@ -1,388 +0,0 @@ - - -Mozilla Public License version 1.1 - - - - -

    Mozilla Public License Version 1.1

    -

    1. Definitions.

    -
    -
    1.0.1. "Commercial Use" -
    means distribution or otherwise making the Covered Code available to a third party. -
    1.1. "Contributor" -
    means each entity that creates or contributes to the creation of Modifications. -
    1.2. "Contributor Version" -
    means the combination of the Original Code, prior Modifications used by a Contributor, - and the Modifications made by that particular Contributor. -
    1.3. "Covered Code" -
    means the Original Code or Modifications or the combination of the Original Code and - Modifications, in each case including portions thereof. -
    1.4. "Electronic Distribution Mechanism" -
    means a mechanism generally accepted in the software development community for the - electronic transfer of data. -
    1.5. "Executable" -
    means Covered Code in any form other than Source Code. -
    1.6. "Initial Developer" -
    means the individual or entity identified as the Initial Developer in the Source Code - notice required by Exhibit A. -
    1.7. "Larger Work" -
    means a work which combines Covered Code or portions thereof with code not governed - by the terms of this License. -
    1.8. "License" -
    means this document. -
    1.8.1. "Licensable" -
    means having the right to grant, to the maximum extent possible, whether at the - time of the initial grant or subsequently acquired, any and all of the rights - conveyed herein. -
    1.9. "Modifications" -
    -

    means any addition to or deletion from the substance or structure of either the - Original Code or any previous Modifications. When Covered Code is released as a - series of files, a Modification is: -

      -
    1. Any addition to or deletion from the contents of a file - containing Original Code or previous Modifications. -
    2. Any new file that contains any part of the Original Code or - previous Modifications. -
    -
    1.10. "Original Code" -
    means Source Code of computer software code which is described in the Source Code - notice required by Exhibit A as Original Code, and which, - at the time of its release under this License is not already Covered Code governed - by this License. -
    1.10.1. "Patent Claims" -
    means any patent claim(s), now owned or hereafter acquired, including without - limitation, method, process, and apparatus claims, in any patent Licensable by - grantor. -
    1.11. "Source Code" -
    means the preferred form of the Covered Code for making modifications to it, - including all modules it contains, plus any associated interface definition files, - scripts used to control compilation and installation of an Executable, or source - code differential comparisons against either the Original Code or another well known, - available Covered Code of the Contributor's choice. The Source Code can be in a - compressed or archival form, provided the appropriate decompression or de-archiving - software is widely available for no charge. -
    1.12. "You" (or "Your") -
    means an individual or a legal entity exercising rights under, and complying with - all of the terms of, this License or a future version of this License issued under - Section 6.1. For legal entities, "You" includes any entity - which controls, is controlled by, or is under common control with You. For purposes of - this definition, "control" means (a) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or otherwise, or (b) - ownership of more than fifty percent (50%) of the outstanding shares or beneficial - ownership of such entity. -
    -

    2. Source Code License.

    -

    2.1. The Initial Developer Grant.

    -

    The Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive - license, subject to third party intellectual property claims: -

      -
    1. under intellectual property rights (other than patent or - trademark) Licensable by Initial Developer to use, reproduce, modify, display, perform, - sublicense and distribute the Original Code (or portions thereof) with or without - Modifications, and/or as part of a Larger Work; and -
    2. under Patents Claims infringed by the making, using or selling - of Original Code, to make, have made, use, practice, sell, and offer for sale, and/or - otherwise dispose of the Original Code (or portions thereof). -
    3. the licenses granted in this Section 2.1 - (a) and (b) are effective on - the date Initial Developer first distributes Original Code under the terms of this - License. -
    4. Notwithstanding Section 2.1 (b) - above, no patent license is granted: 1) for code that You delete from the Original Code; - 2) separate from the Original Code; or 3) for infringements caused by: i) the - modification of the Original Code or ii) the combination of the Original Code with other - software or devices. -
    -

    2.2. Contributor Grant.

    -

    Subject to third party intellectual property claims, each Contributor hereby grants You - a world-wide, royalty-free, non-exclusive license -

      -
    1. under intellectual property rights (other than patent or trademark) - Licensable by Contributor, to use, reproduce, modify, display, perform, sublicense and - distribute the Modifications created by such Contributor (or portions thereof) either on - an unmodified basis, with other Modifications, as Covered Code and/or as part of a Larger - Work; and -
    2. under Patent Claims infringed by the making, using, or selling of - Modifications made by that Contributor either alone and/or in combination with its - Contributor Version (or portions of such combination), to make, use, sell, offer for - sale, have made, and/or otherwise dispose of: 1) Modifications made by that Contributor - (or portions thereof); and 2) the combination of Modifications made by that Contributor - with its Contributor Version (or portions of such combination). -
    3. the licenses granted in Sections 2.2 - (a) and 2.2 (b) are effective - on the date Contributor first makes Commercial Use of the Covered Code. -
    4. Notwithstanding Section 2.2 (b) - above, no patent license is granted: 1) for any code that Contributor has deleted from - the Contributor Version; 2) separate from the Contributor Version; 3) for infringements - caused by: i) third party modifications of Contributor Version or ii) the combination of - Modifications made by that Contributor with other software (except as part of the - Contributor Version) or other devices; or 4) under Patent Claims infringed by Covered Code - in the absence of Modifications made by that Contributor. -
    -

    3. Distribution Obligations.

    -

    3.1. Application of License.

    -

    The Modifications which You create or to which You contribute are governed by the terms - of this License, including without limitation Section 2.2. The - Source Code version of Covered Code may be distributed only under the terms of this License - or a future version of this License released under Section 6.1, - and You must include a copy of this License with every copy of the Source Code You - distribute. You may not offer or impose any terms on any Source Code version that alters or - restricts the applicable version of this License or the recipients' rights hereunder. - However, You may include an additional document offering the additional rights described in - Section 3.5. -

    3.2. Availability of Source Code.

    -

    Any Modification which You create or to which You contribute must be made available in - Source Code form under the terms of this License either on the same media as an Executable - version or via an accepted Electronic Distribution Mechanism to anyone to whom you made an - Executable version available; and if made available via Electronic Distribution Mechanism, - must remain available for at least twelve (12) months after the date it initially became - available, or at least six (6) months after a subsequent version of that particular - Modification has been made available to such recipients. You are responsible for ensuring - that the Source Code version remains available even if the Electronic Distribution - Mechanism is maintained by a third party. -

    3.3. Description of Modifications.

    -

    You must cause all Covered Code to which You contribute to contain a file documenting the - changes You made to create that Covered Code and the date of any change. You must include a - prominent statement that the Modification is derived, directly or indirectly, from Original - Code provided by the Initial Developer and including the name of the Initial Developer in - (a) the Source Code, and (b) in any notice in an Executable version or related documentation - in which You describe the origin or ownership of the Covered Code. -

    3.4. Intellectual Property Matters

    -

    (a) Third Party Claims

    -

    If Contributor has knowledge that a license under a third party's intellectual property - rights is required to exercise the rights granted by such Contributor under Sections - 2.1 or 2.2, Contributor must include a - text file with the Source Code distribution titled "LEGAL" which describes the claim and the - party making the claim in sufficient detail that a recipient will know whom to contact. If - Contributor obtains such knowledge after the Modification is made available as described in - Section 3.2, Contributor shall promptly modify the LEGAL file in - all copies Contributor makes available thereafter and shall take other steps (such as - notifying appropriate mailing lists or newsgroups) reasonably calculated to inform those who - received the Covered Code that new knowledge has been obtained. -

    (b) Contributor APIs

    -

    If Contributor's Modifications include an application programming interface and Contributor - has knowledge of patent licenses which are reasonably necessary to implement that - API, Contributor must also include this information in the - legal file. -

    (c) Representations.

    -

    Contributor represents that, except as disclosed pursuant to Section 3.4 - (a) above, Contributor believes that Contributor's Modifications - are Contributor's original creation(s) and/or Contributor has sufficient rights to grant the - rights conveyed by this License. -

    3.5. Required Notices.

    -

    You must duplicate the notice in Exhibit A in each file of the - Source Code. If it is not possible to put such notice in a particular Source Code file due to - its structure, then You must include such notice in a location (such as a relevant directory) - where a user would be likely to look for such a notice. If You created one or more - Modification(s) You may add your name as a Contributor to the notice described in - Exhibit A. You must also duplicate this License in any documentation - for the Source Code where You describe recipients' rights or ownership rights relating to - Covered Code. You may choose to offer, and to charge a fee for, warranty, support, indemnity - or liability obligations to one or more recipients of Covered Code. However, You may do so - only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You - must make it absolutely clear than any such warranty, support, indemnity or liability - obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer - and every Contributor for any liability incurred by the Initial Developer or such Contributor - as a result of warranty, support, indemnity or liability terms You offer. -

    3.6. Distribution of Executable Versions.

    -

    You may distribute Covered Code in Executable form only if the requirements of Sections - 3.1, 3.2, - 3.3, 3.4 and - 3.5 have been met for that Covered Code, and if You include a - notice stating that the Source Code version of the Covered Code is available under the terms - of this License, including a description of how and where You have fulfilled the obligations - of Section 3.2. The notice must be conspicuously included in any - notice in an Executable version, related documentation or collateral in which You describe - recipients' rights relating to the Covered Code. You may distribute the Executable version of - Covered Code or ownership rights under a license of Your choice, which may contain terms - different from this License, provided that You are in compliance with the terms of this - License and that the license for the Executable version does not attempt to limit or alter the - recipient's rights in the Source Code version from the rights set forth in this License. If - You distribute the Executable version under a different license You must make it absolutely - clear that any terms which differ from this License are offered by You alone, not by the - Initial Developer or any Contributor. You hereby agree to indemnify the Initial Developer and - every Contributor for any liability incurred by the Initial Developer or such Contributor as - a result of any such terms You offer. -

    3.7. Larger Works.

    -

    You may create a Larger Work by combining Covered Code with other code not governed by the - terms of this License and distribute the Larger Work as a single product. In such a case, - You must make sure the requirements of this License are fulfilled for the Covered Code. -

    4. Inability to Comply Due to Statute or Regulation.

    -

    If it is impossible for You to comply with any of the terms of this License with respect to - some or all of the Covered Code due to statute, judicial order, or regulation then You must: - (a) comply with the terms of this License to the maximum extent possible; and (b) describe - the limitations and the code they affect. Such description must be included in the - legal file described in Section - 3.4 and must be included with all distributions of the Source Code. - Except to the extent prohibited by statute or regulation, such description must be - sufficiently detailed for a recipient of ordinary skill to be able to understand it. -

    5. Application of this License.

    -

    This License applies to code to which the Initial Developer has attached the notice in - Exhibit A and to related Covered Code. -

    6. Versions of the License.

    -

    6.1. New Versions

    -

    Netscape Communications Corporation ("Netscape") may publish revised and/or new versions - of the License from time to time. Each version will be given a distinguishing version number. -

    6.2. Effect of New Versions

    -

    Once Covered Code has been published under a particular version of the License, You may - always continue to use it under the terms of that version. You may also choose to use such - Covered Code under the terms of any subsequent version of the License published by Netscape. - No one other than Netscape has the right to modify the terms applicable to Covered Code - created under this License. -

    6.3. Derivative Works

    -

    If You create or use a modified version of this License (which you may only do in order to - apply it to code which is not already Covered Code governed by this License), You must (a) - rename Your license so that the phrases "Mozilla", "MOZILLAPL", "MOZPL", "Netscape", "MPL", - "NPL" or any confusingly similar phrase do not appear in your license (except to note that - your license differs from this License) and (b) otherwise make it clear that Your version of - the license contains terms which differ from the Mozilla Public License and Netscape Public - License. (Filling in the name of the Initial Developer, Original Code or Contributor in the - notice described in Exhibit A shall not of themselves be deemed to - be modifications of this License.) -

    7. Disclaimer of warranty

    -

    Covered code is provided under this license on an "as is" - basis, without warranty of any kind, either expressed or implied, including, without - limitation, warranties that the covered code is free of defects, merchantable, fit for a - particular purpose or non-infringing. The entire risk as to the quality and performance of - the covered code is with you. Should any covered code prove defective in any respect, you - (not the initial developer or any other contributor) assume the cost of any necessary - servicing, repair or correction. This disclaimer of warranty constitutes an essential part - of this license. No use of any covered code is authorized hereunder except under this - disclaimer. -

    8. Termination

    -

    8.1. This License and the rights granted hereunder will terminate - automatically if You fail to comply with terms herein and fail to cure such breach - within 30 days of becoming aware of the breach. All sublicenses to the Covered Code which - are properly granted shall survive any termination of this License. Provisions which, by - their nature, must remain in effect beyond the termination of this License shall survive. -

    8.2. If You initiate litigation by asserting a patent infringement - claim (excluding declatory judgment actions) against Initial Developer or a Contributor - (the Initial Developer or Contributor against whom You file such action is referred to - as "Participant") alleging that: -

      -
    1. such Participant's Contributor Version directly or indirectly - infringes any patent, then any and all rights granted by such Participant to You under - Sections 2.1 and/or 2.2 of this - License shall, upon 60 days notice from Participant terminate prospectively, unless if - within 60 days after receipt of notice You either: (i) agree in writing to pay - Participant a mutually agreeable reasonable royalty for Your past and future use of - Modifications made by such Participant, or (ii) withdraw Your litigation claim with - respect to the Contributor Version against such Participant. If within 60 days of - notice, a reasonable royalty and payment arrangement are not mutually agreed upon in - writing by the parties or the litigation claim is not withdrawn, the rights granted by - Participant to You under Sections 2.1 and/or - 2.2 automatically terminate at the expiration of the 60 day - notice period specified above. -
    2. any software, hardware, or device, other than such Participant's - Contributor Version, directly or indirectly infringes any patent, then any rights - granted to You by such Participant under Sections 2.1(b) - and 2.2(b) are revoked effective as of the date You first - made, used, sold, distributed, or had made, Modifications made by that Participant. -
    -

    8.3. If You assert a patent infringement claim against Participant - alleging that such Participant's Contributor Version directly or indirectly infringes - any patent where such claim is resolved (such as by license or settlement) prior to the - initiation of patent infringement litigation, then the reasonable value of the licenses - granted by such Participant under Sections 2.1 or - 2.2 shall be taken into account in determining the amount or - value of any payment or license. -

    8.4. In the event of termination under Sections - 8.1 or 8.2 above, all end user - license agreements (excluding distributors and resellers) which have been validly - granted by You or any distributor hereunder prior to termination shall survive - termination. -

    9. Limitation of liability

    -

    Under no circumstances and under no legal theory, whether - tort (including negligence), contract, or otherwise, shall you, the initial developer, - any other contributor, or any distributor of covered code, or any supplier of any of - such parties, be liable to any person for any indirect, special, incidental, or - consequential damages of any character including, without limitation, damages for loss - of goodwill, work stoppage, computer failure or malfunction, or any and all other - commercial damages or losses, even if such party shall have been informed of the - possibility of such damages. This limitation of liability shall not apply to liability - for death or personal injury resulting from such party's negligence to the extent - applicable law prohibits such limitation. Some jurisdictions do not allow the exclusion - or limitation of incidental or consequential damages, so this exclusion and limitation - may not apply to you. -

    10. U.S. government end users

    -

    The Covered Code is a "commercial item," as that term is defined in 48 - C.F.R. 2.101 (Oct. 1995), consisting of - "commercial computer software" and "commercial computer software documentation," as such - terms are used in 48 C.F.R. 12.212 (Sept. - 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. - 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users - acquire Covered Code with only those rights set forth herein. -

    11. Miscellaneous

    -

    This License represents the complete agreement concerning subject matter hereof. If - any provision of this License is held to be unenforceable, such provision shall be - reformed only to the extent necessary to make it enforceable. This License shall be - governed by California law provisions (except to the extent applicable law, if any, - provides otherwise), excluding its conflict-of-law provisions. With respect to - disputes in which at least one party is a citizen of, or an entity chartered or - registered to do business in the United States of America, any litigation relating to - this License shall be subject to the jurisdiction of the Federal Courts of the - Northern District of California, with venue lying in Santa Clara County, California, - with the losing party responsible for costs, including without limitation, court - costs and reasonable attorneys' fees and expenses. The application of the United - Nations Convention on Contracts for the International Sale of Goods is expressly - excluded. Any law or regulation which provides that the language of a contract - shall be construed against the drafter shall not apply to this License. -

    12. Responsibility for claims

    -

    As between Initial Developer and the Contributors, each party is responsible for - claims and damages arising, directly or indirectly, out of its utilization of rights - under this License and You agree to work with Initial Developer and Contributors to - distribute such responsibility on an equitable basis. Nothing herein is intended or - shall be deemed to constitute any admission of liability. -

    13. Multiple-licensed code

    -

    Initial Developer may designate portions of the Covered Code as - "Multiple-Licensed". "Multiple-Licensed" means that the Initial Developer permits - you to utilize portions of the Covered Code under Your choice of the MPL - or the alternative licenses, if any, specified by the Initial Developer in the file - described in Exhibit A. -

    Exhibit A - Mozilla Public License.

    -
    "The contents of this file are subject to the Mozilla Public License
    -Version 1.1 (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.mozilla.org/MPL/
    -
    -Software distributed under the License is distributed on an "AS IS"
    -basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See the
    -License for the specific language governing rights and limitations
    -under the License.
    -
    -The Original Code is JTransforms.
    -
    -The Initial Developer of the Original Code is
    -Piotr Wendykier, Emory University.
    -Portions created by the Initial Developer are Copyright (C) 2007-2009
    -the Initial Developer. All Rights Reserved.
    -
    -Alternatively, the contents of this file may be used under the terms of
    -either the GNU General Public License Version 2 or later (the "GPL"), or
    -the GNU Lesser General Public License Version 2.1 or later (the "LGPL"),
    -in which case the provisions of the GPL or the LGPL are applicable instead
    -of those above. If you wish to allow use of your version of this file only
    -under the terms of either the GPL or the LGPL, and not to allow others to
    -use your version of this file under the terms of the MPL, indicate your
    -decision by deleting the provisions above and replace them with the notice
    -and other provisions required by the GPL or the LGPL. If you do not delete
    -the provisions above, a recipient may use your version of this file under
    -the terms of any one of the MPL, the GPL or the LGPL.
    -

    NOTE: The text of this Exhibit A may differ slightly from the text of - the notices in the Source Code files of the Original Code. You should - use the text of this Exhibit A rather than the text found in the - Original Code Source Code for Your Modifications. - -

    \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index 32d98151bdcff..84d6a536ccca8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import edu.emory.mathcs.jtransforms.dct._ +import org.jtransforms.dct._ import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index b7956b6fd3e9a..69952f0b64ac2 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -20,7 +20,7 @@ import java.util.Arrays; import java.util.List; -import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D; +import org.jtransforms.dct.DoubleDCT_1D; import org.junit.Assert; import org.junit.Test; diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala index 985e396000d05..079dabb3665be 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D +import org.jtransforms.dct.DoubleDCT_1D import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} diff --git a/pom.xml b/pom.xml index 2f29ad07f1d90..2396c5168b166 100644 --- a/pom.xml +++ b/pom.xml @@ -786,14 +786,8 @@ org.scalanlp breeze_${scala.binary.version} - 0.13.2 + 1.0 - - - junit - junit - org.apache.commons commons-math3 From f7cc6958084111cc6180c572519082ec235eb189 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 21 Sep 2019 16:29:23 +0900 Subject: [PATCH 0143/1387] [SPARK-29140][SQL] Handle parameters having "array" of javaType properly in splitAggregateExpressions ### What changes were proposed in this pull request? This patch fixes the issue brought by [SPARK-21870](http://issues.apache.org/jira/browse/SPARK-21870): when generating code for parameter type, it doesn't consider array type in javaType. At least we have one, Spark should generate code for BinaryType as `byte[]`, but Spark create the code for BinaryType as `[B` and generated code fails compilation. Below is the generated code which failed compilation (Line 380): ``` /* 380 */ private void agg_doAggregate_count_0([B agg_expr_1_1, boolean agg_exprIsNull_1_1, org.apache.spark.sql.catalyst.InternalRow agg_unsafeRowAggBuffer_1) throws java.io.IOException { /* 381 */ // evaluate aggregate function for count /* 382 */ boolean agg_isNull_26 = false; /* 383 */ long agg_value_28 = -1L; /* 384 */ if (!false && agg_exprIsNull_1_1) { /* 385 */ long agg_value_31 = agg_unsafeRowAggBuffer_1.getLong(1); /* 386 */ agg_isNull_26 = false; /* 387 */ agg_value_28 = agg_value_31; /* 388 */ } else { /* 389 */ long agg_value_33 = agg_unsafeRowAggBuffer_1.getLong(1); /* 390 */ /* 391 */ long agg_value_32 = -1L; /* 392 */ /* 393 */ agg_value_32 = agg_value_33 + 1L; /* 394 */ agg_isNull_26 = false; /* 395 */ agg_value_28 = agg_value_32; /* 396 */ } /* 397 */ // update unsafe row buffer /* 398 */ agg_unsafeRowAggBuffer_1.setLong(1, agg_value_28); /* 399 */ } ``` There wasn't any test for HashAggregateExec specifically testing this, but randomized test in ObjectHashAggregateSuite could encounter this and that's why ObjectHashAggregateSuite is flaky. ### Why are the changes needed? Without the fix, generated code from HashAggregateExec may fail compilation. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added new UT. Without the fix, newly added UT fails. Closes #25830 from HeartSaVioR/SPARK-29140. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Takeshi Yamamuro --- .../expressions/codegen/CodeGenerator.scala | 8 ++++++++ .../execution/aggregate/HashAggregateExec.scala | 4 +++- .../hive/execution/AggregationQuerySuite.scala | 15 +++++++++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 68ddec9fc8d00..946fc7f421ad6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1811,6 +1811,14 @@ object CodeGenerator extends Logging { def boxedType(dt: DataType): String = boxedType(javaType(dt)) + def typeName(clazz: Class[_]): String = { + if (clazz.isArray) { + typeName(clazz.getComponentType) + "[]" + } else { + clazz.getName + } + } + /** * Returns the representation of default value for a given Java Type. * @param jt the string name of the Java type diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 2d187e3c9ebe5..5dc5b822919be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -299,7 +299,9 @@ case class HashAggregateExec( if (inputVars.forall(_.isDefined)) { val splitCodes = inputVars.flatten.zipWithIndex.map { case (args, i) => val doAggFunc = ctx.freshName(s"doAggregate_${aggNames(i)}") - val argList = args.map(v => s"${v.javaType.getName} ${v.variableName}").mkString(", ") + val argList = args.map { v => + s"${CodeGenerator.typeName(v.javaType)} ${v.variableName}" + }.mkString(", ") val doAggFuncName = ctx.addNewFunction(doAggFunc, s""" |private void $doAggFunc($argList) throws java.io.IOException { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 8c7e5bf5ac1d4..4a3277f5a7e49 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -1028,6 +1028,21 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te } } } + + test("SPARK-29140: HashAggregateExec aggregating binary type doesn't break codegen compilation") { + val schema = new StructType().add("id", IntegerType, nullable = false) + .add("c1", BinaryType, nullable = true) + + withSQLConf( + SQLConf.CODEGEN_SPLIT_AGGREGATE_FUNC.key -> "true", + SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1") { + val emptyRows = spark.sparkContext.parallelize(Seq.empty[Row], 1) + val aggDf = spark.createDataFrame(emptyRows, schema) + .groupBy($"id" % 10 as "group") + .agg(countDistinct($"c1")) + checkAnswer(aggDf, Seq.empty[Row]) + } + } } From 076186e8815ba6381f14941e05a6ec15348369ba Mon Sep 17 00:00:00 2001 From: colinma Date: Sat, 21 Sep 2019 07:31:39 -0500 Subject: [PATCH 0144/1387] [SPARK-19147][CORE] Gracefully handle error in task after executor is stopped ### What changes were proposed in this pull request? TransportClientFactory.createClient() is called by task and TransportClientFactory.close() is called by executor. When stop the executor, close() will set workerGroup = null, NPE will occur in createClient which generate many exception in log. For exception occurs after close(), treated it as an expected Exception and transform it to InterruptedException which can be processed by Executor. ### Why are the changes needed? The change can reduce the exception stack trace in log file, and user won't be confused by these excepted exception. ### Does this PR introduce any user-facing change? N/A ### How was this patch tested? New tests are added in TransportClientFactorySuite and ExecutorSuite Closes #25759 from colinmjj/spark-19147. Authored-by: colinma Signed-off-by: Sean Owen --- .../spark/network/client/TransportClientFactory.java | 3 +-- .../apache/spark/network/TransportClientFactorySuite.java | 7 +++++++ core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../main/scala/org/apache/spark/executor/Executor.scala | 5 +++++ 4 files changed, 14 insertions(+), 3 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 53835d8304866..c9ef9f918ffd1 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -293,9 +293,8 @@ public void close() { } connectionPool.clear(); - if (workerGroup != null) { + if (workerGroup != null && !workerGroup.isShuttingDown()) { workerGroup.shutdownGracefully(); - workerGroup = null; } } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 2aec4a33bbe43..9b76981c31c57 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -217,4 +217,11 @@ public Iterable> getAll() { assertFalse(c1.isActive()); } } + + @Test(expected = IOException.class) + public void closeFactoryBeforeCreateClient() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + factory.close(); + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + } } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 419f0ab065150..14895b39b24bc 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -70,7 +70,7 @@ class SparkEnv ( val outputCommitCoordinator: OutputCommitCoordinator, val conf: SparkConf) extends Logging { - private[spark] var isStopped = false + @volatile private[spark] var isStopped = false private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 12bc227b341a3..d2eb01d20e9a3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -629,6 +629,11 @@ private[spark] class Executor( setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(reason)) + case t: Throwable if env.isStopped => + // Log the expected exception after executor.stop without stack traces + // see: SPARK-19147 + logError(s"Exception in $taskName (TID $taskId): ${t.getMessage}") + case t: Throwable => // Attempt to exit cleanly by informing the driver of our failure. // If anything goes wrong (or this was a fatal exception), we will delegate to From 93ac4e1b2df5ca1dde29010320c6faead537a7b1 Mon Sep 17 00:00:00 2001 From: aman_omer Date: Sat, 21 Sep 2019 07:34:04 -0500 Subject: [PATCH 0145/1387] [SPARK-29053][WEBUI] Sort does not work on some columns ### What changes were proposed in this pull request? Setting custom sort key for duration and execution time column. ### Why are the changes needed? Sorting on duration and execution time columns consider time as a string after converting into readable form which is the reason for wrong sort results as mentioned in [SPARK-29053](https://issues.apache.org/jira/browse/SPARK-29053). ### Does this PR introduce any user-facing change? No ### How was this patch tested? Test manually. Screenshots are attached. After patch: **Duration** ![Duration](https://user-images.githubusercontent.com/40591404/65339861-93cc9800-dbea-11e9-95e6-63b107a5a372.png) **Execution time** ![Execution Time](https://user-images.githubusercontent.com/40591404/65339870-97601f00-dbea-11e9-9d1d-690c59bc1bde.png) Closes #25855 from amanomer/SPARK29053. Authored-by: aman_omer Signed-off-by: Sean Owen --- .../sql/hive/thriftserver/ui/ThriftServerPage.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 261e8fc912eb9..1a7db1b58e5bd 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -91,8 +91,10 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" {formatDate(info.startTimestamp)} {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} - {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} - {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} + + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} {info.state} {errorMessageCell(detail)} @@ -157,7 +159,8 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" {session.sessionId} {formatDate(session.startTimestamp)} {if (session.finishTimestamp > 0) formatDate(session.finishTimestamp)} - {formatDurationOption(Some(session.totalTime))} + + {formatDurationOption(Some(session.totalTime))} {session.totalExecution.toString} } From 81b6f11a3a4f0f3e0fe807646618220c0c71f869 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 21 Sep 2019 23:59:37 +0900 Subject: [PATCH 0146/1387] [SPARK-29160][CORE] Use UTF-8 explicitly for reading/writing event log file ### What changes were proposed in this pull request? Credit to vanzin as he found and commented on this while reviewing #25670 - [comment](https://github.com/apache/spark/pull/25670#discussion_r325383512). This patch proposes to specify UTF-8 explicitly while reading/writer event log file. ### Why are the changes needed? The event log file is being read/written as default character set of JVM process which may open the chance to bring some problems on reading event log files from another machines. Spark's de facto standard character set is UTF-8, so it should be explicitly set to. ### Does this PR introduce any user-facing change? Yes, if end users have been running Spark process with different default charset than "UTF-8", especially their driver JVM processes. No otherwise. ### How was this patch tested? Existing UTs, as ReplayListenerSuite contains "end-to-end" event logging/reading tests (both uncompressed/compressed). Closes #25845 from HeartSaVioR/SPARK-29160. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../apache/spark/scheduler/EventLoggingListener.scala | 2 +- .../org/apache/spark/scheduler/ReplayListenerBus.scala | 4 ++-- .../apache/spark/scheduler/ReplayListenerSuite.scala | 10 +++++++--- docs/core-migration-guide.md | 1 + 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index bee784dcdfe33..3e8318e1921cb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -131,7 +131,7 @@ private[spark] class EventLoggingListener( EventLoggingListener.initEventLog(bstream, testing, loggedEvents) fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) - writer = Some(new PrintWriter(bstream)) + writer = Some(new PrintWriter(new OutputStreamWriter(bstream, StandardCharsets.UTF_8))) logInfo("Logging events to %s".format(logPath)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 226c23733c870..699042dd967bc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.io.{EOFException, InputStream, IOException} -import scala.io.Source +import scala.io.{Codec, Source} import com.fasterxml.jackson.core.JsonParseException import com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException @@ -54,7 +54,7 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { sourceName: String, maybeTruncated: Boolean = false, eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { - val lines = Source.fromInputStream(logData).getLines() + val lines = Source.fromInputStream(logData)(Codec.UTF8).getLines() replay(lines, sourceName, maybeTruncated, eventsFilter) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index d65b5cbfc094e..cda537159a573 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI +import java.nio.charset.StandardCharsets import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer @@ -52,10 +53,11 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp test("Simple replay") { val logFilePath = getFilePath(testDir, "events.txt") val fstream = fileSystem.create(logFilePath) + val fwriter = new OutputStreamWriter(fstream, StandardCharsets.UTF_8) val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) - Utils.tryWithResource(new PrintWriter(fstream)) { writer => + Utils.tryWithResource(new PrintWriter(fwriter)) { writer => // scalastyle:off println writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) @@ -88,7 +90,8 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp val buffered = new ByteArrayOutputStream val codec = new LZ4CompressionCodec(new SparkConf()) val compstream = codec.compressedOutputStream(buffered) - Utils.tryWithResource(new PrintWriter(compstream)) { writer => + val cwriter = new OutputStreamWriter(compstream, StandardCharsets.UTF_8) + Utils.tryWithResource(new PrintWriter(cwriter)) { writer => val applicationStart = SparkListenerApplicationStart("AppStarts", None, 125L, "Mickey", None) @@ -134,10 +137,11 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp test("Replay incompatible event log") { val logFilePath = getFilePath(testDir, "incompatible.txt") val fstream = fileSystem.create(logFilePath) + val fwriter = new OutputStreamWriter(fstream, StandardCharsets.UTF_8) val applicationStart = SparkListenerApplicationStart("Incompatible App", None, 125L, "UserUsingIncompatibleVersion", None) val applicationEnd = SparkListenerApplicationEnd(1000L) - Utils.tryWithResource(new PrintWriter(fstream)) { writer => + Utils.tryWithResource(new PrintWriter(fwriter)) { writer => // scalastyle:off println writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println("""{"Event":"UnrecognizedEventOnlyForTest","Timestamp":1477593059313}""") diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 1c47dfb27124c..2d4d91dab075e 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -30,3 +30,4 @@ license: | - In Spark 3.0, deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. +- In Spark 3.0, event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. \ No newline at end of file From 3e2649287d98b9692fdd1025e1bf8fe8d3de768e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 21 Sep 2019 08:13:00 -0700 Subject: [PATCH 0147/1387] [SPARK-29199][INFRA] Add linters and license/dependency checkers to GitHub Action ### What changes were proposed in this pull request? This PR aims to add linters and license/dependency checkers to GitHub Action. This excludes `lint-r` intentionally because https://github.com/actions/setup-r is not ready. We can add that later when it becomes available. ### Why are the changes needed? This will help the PR reviews. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? See the GitHub Action result on this PR. Closes #25879 from dongjoon-hyun/SPARK-29199. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 350f3405bc4c7..e2eb0683b6e59 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -32,3 +32,29 @@ jobs: export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} package + + + lint: + runs-on: ubuntu-latest + name: Linters + steps: + - uses: actions/checkout@master + - uses: actions/setup-java@v1 + with: + java-version: '11' + - uses: actions/setup-python@v1 + with: + python-version: '3.x' + architecture: 'x64' + - name: Scala + run: ./dev/lint-scala + - name: Java + run: ./dev/lint-java + - name: Python + run: | + pip install flake8 sphinx numpy + ./dev/lint-python + - name: License + run: ./dev/check-license + - name: Dependencies + run: ./dev/test-dependencies.sh From c7c6b642dccb3fbd5f82b81cff7f88a776976c7d Mon Sep 17 00:00:00 2001 From: Patrick Pisciuneri Date: Sat, 21 Sep 2019 14:26:54 -0500 Subject: [PATCH 0148/1387] [SPARK-29121][ML][MLLIB] Support for dot product operation on Vector(s) ### What changes were proposed in this pull request? Support for dot product with: - `ml.linalg.Vector` - `ml.linalg.Vectors` - `mllib.linalg.Vector` - `mllib.linalg.Vectors` ### Why are the changes needed? Dot product is useful for feature engineering and scoring. BLAS routines are already there, just a wrapper is needed. ### Does this PR introduce any user-facing change? No user facing changes, just some new functionality. ### How was this patch tested? Tests were written and added to the appropriate `VectorSuites` classes. They can be quickly run with: ``` sbt "mllib-local/testOnly org.apache.spark.ml.linalg.VectorsSuite" sbt "mllib/testOnly org.apache.spark.mllib.linalg.VectorsSuite" ``` Closes #25818 from phpisciuneri/SPARK-29121. Authored-by: Patrick Pisciuneri Signed-off-by: Sean Owen --- .../org/apache/spark/ml/linalg/Vectors.scala | 8 +++++++ .../apache/spark/ml/linalg/VectorsSuite.scala | 23 +++++++++++++++++++ .../apache/spark/mllib/linalg/Vectors.scala | 8 +++++++ .../spark/mllib/linalg/VectorsSuite.scala | 23 +++++++++++++++++++ 4 files changed, 62 insertions(+) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index 6e43d60bd03a3..f437d66cddb54 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -178,6 +178,14 @@ sealed trait Vector extends Serializable { */ @Since("2.0.0") def argmax: Int + + /** + * Calculate the dot product of this vector with another. + * + * If `size` does not match an [[IllegalArgumentException]] is thrown. + */ + @Since("3.0.0") + def dot(v: Vector): Double = BLAS.dot(this, v) } /** diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala index 0a316f57f811b..c97dc2c3c06f8 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala @@ -380,4 +380,27 @@ class VectorsSuite extends SparkMLFunSuite { Vectors.sparse(-1, Array((1, 2.0))) } } + + test("dot product only supports vectors of same size") { + val vSize4 = Vectors.dense(arr) + val vSize1 = Vectors.zeros(1) + intercept[IllegalArgumentException]{ vSize1.dot(vSize4) } + } + + test("dense vector dot product") { + val dv = Vectors.dense(arr) + assert(dv.dot(dv) === 0.26) + } + + test("sparse vector dot product") { + val sv = Vectors.sparse(n, indices, values) + assert(sv.dot(sv) === 0.26) + } + + test("mixed sparse and dense vector dot product") { + val sv = Vectors.sparse(n, indices, values) + val dv = Vectors.dense(arr) + assert(sv.dot(dv) === 0.26) + assert(dv.dot(sv) === 0.26) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index b754fad0c1796..83a519326df75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -204,6 +204,14 @@ sealed trait Vector extends Serializable { */ @Since("2.0.0") def asML: newlinalg.Vector + + /** + * Calculate the dot product of this vector with another. + * + * If `size` does not match an [[IllegalArgumentException]] is thrown. + */ + @Since("3.0.0") + def dot(v: Vector): Double = BLAS.dot(this, v) } /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index fee0b02bf8ed8..b2163b518dbd1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -510,4 +510,27 @@ class VectorsSuite extends SparkFunSuite with Logging { Vectors.sparse(-1, Array((1, 2.0))) } } + + test("dot product only supports vectors of same size") { + val vSize4 = Vectors.dense(arr) + val vSize1 = Vectors.zeros(1) + intercept[IllegalArgumentException]{ vSize1.dot(vSize4) } + } + + test("dense vector dot product") { + val dv = Vectors.dense(arr) + assert(dv.dot(dv) === 0.26) + } + + test("sparse vector dot product") { + val sv = Vectors.sparse(n, indices, values) + assert(sv.dot(sv) === 0.26) + } + + test("mixed sparse and dense vector dot product") { + val sv = Vectors.sparse(n, indices, values) + val dv = Vectors.dense(arr) + assert(sv.dot(dv) === 0.26) + assert(dv.dot(sv) === 0.26) + } } From 3be5741029c9ad60b56686eb6a33dce8eb8189d8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 21 Sep 2019 21:11:31 -0700 Subject: [PATCH 0149/1387] [SPARK-29190][SQL] Optimize `extract`/`date_part` for the milliseconds `field` ### What changes were proposed in this pull request? Changed the `DateTimeUtils.getMilliseconds()` by avoiding the decimal division, and replacing it by setting scale and precision while converting microseconds to the decimal type. ### Why are the changes needed? This improves performance of `extract` and `date_part()` by more than **50 times**: Before: ``` Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast to timestamp 397 428 45 25.2 39.7 1.0X MILLISECONDS of timestamp 36723 36761 63 0.3 3672.3 0.0X ``` After: ``` Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast to timestamp 278 284 6 36.0 27.8 1.0X MILLISECONDS of timestamp 592 606 13 16.9 59.2 0.5X ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suite - `DateExpressionsSuite` Closes #25871 from MaxGekk/optimize-epoch-millis. Lead-authored-by: Maxim Gekk Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/util/DateTimeUtils.scala | 3 +- .../ExtractBenchmark-jdk11-results.txt | 100 +++++++++++ .../benchmarks/ExtractBenchmark-results.txt | 160 +++++++++--------- 3 files changed, 181 insertions(+), 82 deletions(-) create mode 100644 sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index a82471aae652d..7757ea51536c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -465,8 +465,7 @@ object DateTimeUtils { * is expressed in microseconds since the epoch. */ def getMilliseconds(timestamp: SQLTimestamp, timeZone: TimeZone): Decimal = { - val micros = Decimal(getMicroseconds(timestamp, timeZone)) - (micros / Decimal(MICROS_PER_MILLIS)).toPrecision(8, 3) + Decimal(getMicroseconds(timestamp, timeZone), 8, 3) } /** diff --git a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..9ee9176bea508 --- /dev/null +++ b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt @@ -0,0 +1,100 @@ +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to timestamp 438 548 128 22.8 43.8 1.0X +MILLENNIUM of timestamp 1343 1453 139 7.4 134.3 0.3X +CENTURY of timestamp 1287 1305 16 7.8 128.7 0.3X +DECADE of timestamp 1253 1258 7 8.0 125.3 0.3X +YEAR of timestamp 1224 1247 24 8.2 122.4 0.4X +ISOYEAR of timestamp 1356 1383 35 7.4 135.6 0.3X +QUARTER of timestamp 1386 1395 8 7.2 138.6 0.3X +MONTH of timestamp 1215 1227 11 8.2 121.5 0.4X +WEEK of timestamp 1711 1720 9 5.8 171.1 0.3X +DAY of timestamp 1227 1251 37 8.1 122.7 0.4X +DAYOFWEEK of timestamp 1386 1392 11 7.2 138.6 0.3X +DOW of timestamp 1405 1426 34 7.1 140.5 0.3X +ISODOW of timestamp 1344 1363 30 7.4 134.4 0.3X +DOY of timestamp 1249 1251 3 8.0 124.9 0.4X +HOUR of timestamp 766 773 9 13.1 76.6 0.6X +MINUTE of timestamp 761 774 22 13.1 76.1 0.6X +SECOND of timestamp 627 638 11 16.0 62.7 0.7X +MILLISECONDS of timestamp 700 704 4 14.3 70.0 0.6X +MICROSECONDS of timestamp 615 627 10 16.3 61.5 0.7X +EPOCH of timestamp 28897 28929 29 0.3 2889.7 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to date 1078 1081 4 9.3 107.8 1.0X +MILLENNIUM of date 1232 1244 16 8.1 123.2 0.9X +CENTURY of date 1233 1234 1 8.1 123.3 0.9X +DECADE of date 1210 1212 3 8.3 121.0 0.9X +YEAR of date 1201 1212 9 8.3 120.1 0.9X +ISOYEAR of date 1468 1474 5 6.8 146.8 0.7X +QUARTER of date 1474 1482 11 6.8 147.4 0.7X +MONTH of date 1211 1215 4 8.3 121.1 0.9X +WEEK of date 1684 1685 2 5.9 168.4 0.6X +DAY of date 1208 1214 6 8.3 120.8 0.9X +DAYOFWEEK of date 1374 1387 23 7.3 137.4 0.8X +DOW of date 1396 1404 11 7.2 139.6 0.8X +ISODOW of date 1320 1322 3 7.6 132.0 0.8X +DOY of date 1243 1258 13 8.0 124.3 0.9X +HOUR of date 1997 2018 29 5.0 199.7 0.5X +MINUTE of date 2021 2039 26 4.9 202.1 0.5X +SECOND of date 1862 1878 22 5.4 186.2 0.6X +MILLISECONDS of date 1998 2015 16 5.0 199.8 0.5X +MICROSECONDS of date 1893 1901 7 5.3 189.3 0.6X +EPOCH of date 30353 30376 41 0.3 3035.3 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to timestamp 384 389 4 26.1 38.4 1.0X +MILLENNIUM of timestamp 1237 1244 6 8.1 123.7 0.3X +CENTURY of timestamp 1236 1244 7 8.1 123.6 0.3X +DECADE of timestamp 1204 1210 9 8.3 120.4 0.3X +YEAR of timestamp 1197 1207 16 8.4 119.7 0.3X +ISOYEAR of timestamp 1466 1470 4 6.8 146.6 0.3X +QUARTER of timestamp 1500 1505 6 6.7 150.0 0.3X +MONTH of timestamp 1190 1218 25 8.4 119.0 0.3X +WEEK of timestamp 1681 1710 25 5.9 168.1 0.2X +DAY of timestamp 1201 1206 7 8.3 120.1 0.3X +DAYOFWEEK of timestamp 1376 1390 13 7.3 137.6 0.3X +DOW of timestamp 1399 1409 17 7.1 139.9 0.3X +ISODOW of timestamp 1347 1354 8 7.4 134.7 0.3X +DOY of timestamp 1257 1263 6 8.0 125.7 0.3X +HOUR of timestamp 749 753 5 13.4 74.9 0.5X +MINUTE of timestamp 746 749 4 13.4 74.6 0.5X +SECOND of timestamp 626 637 15 16.0 62.6 0.6X +MILLISECONDS of timestamp 695 724 25 14.4 69.5 0.6X +MICROSECONDS of timestamp 611 629 27 16.4 61.1 0.6X +EPOCH of timestamp 28908 28938 31 0.3 2890.8 0.0X + +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to date 1076 1083 6 9.3 107.6 1.0X +MILLENNIUM of date 1230 1236 7 8.1 123.0 0.9X +CENTURY of date 1245 1250 5 8.0 124.5 0.9X +DECADE of date 1206 1211 8 8.3 120.6 0.9X +YEAR of date 1194 1201 6 8.4 119.4 0.9X +ISOYEAR of date 1461 1471 9 6.8 146.1 0.7X +QUARTER of date 1496 1500 7 6.7 149.6 0.7X +MONTH of date 1192 1195 4 8.4 119.2 0.9X +WEEK of date 1682 1687 6 5.9 168.2 0.6X +DAY of date 1199 1207 14 8.3 119.9 0.9X +DAYOFWEEK of date 1372 1383 19 7.3 137.2 0.8X +DOW of date 1384 1393 14 7.2 138.4 0.8X +ISODOW of date 1327 1338 10 7.5 132.7 0.8X +DOY of date 1243 1247 7 8.0 124.3 0.9X +HOUR of date 2001 2010 10 5.0 200.1 0.5X +MINUTE of date 2046 2053 9 4.9 204.6 0.5X +SECOND of date 1859 1863 4 5.4 185.9 0.6X +MILLISECONDS of date 2000 2013 16 5.0 200.0 0.5X +MICROSECONDS of date 1856 1857 1 5.4 185.6 0.6X +EPOCH of date 30365 30388 29 0.3 3036.5 0.0X + diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 6c4bdfed703ff..4894db4c522f6 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -2,99 +2,99 @@ OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 397 428 45 25.2 39.7 1.0X -MILLENNIUM of timestamp 1480 1546 67 6.8 148.0 0.3X -CENTURY of timestamp 1368 1384 17 7.3 136.8 0.3X -DECADE of timestamp 1281 1344 57 7.8 128.1 0.3X -YEAR of timestamp 1238 1244 5 8.1 123.8 0.3X -ISOYEAR of timestamp 1379 1455 122 7.2 137.9 0.3X -QUARTER of timestamp 1442 1456 15 6.9 144.2 0.3X -MONTH of timestamp 1213 1217 3 8.2 121.3 0.3X -WEEK of timestamp 1927 1947 22 5.2 192.7 0.2X -DAY of timestamp 1306 1320 16 7.7 130.6 0.3X -DAYOFWEEK of timestamp 1394 1402 11 7.2 139.4 0.3X -DOW of timestamp 1367 1374 6 7.3 136.7 0.3X -ISODOW of timestamp 1317 1321 5 7.6 131.7 0.3X -DOY of timestamp 1223 1238 14 8.2 122.3 0.3X -HOUR of timestamp 361 362 2 27.7 36.1 1.1X -MINUTE of timestamp 354 362 10 28.3 35.4 1.1X -SECOND of timestamp 362 365 4 27.6 36.2 1.1X -MILLISECONDS of timestamp 36723 36761 63 0.3 3672.3 0.0X -MICROSECONDS of timestamp 469 490 29 21.3 46.9 0.8X -EPOCH of timestamp 30137 30181 38 0.3 3013.7 0.0X +cast to timestamp 375 411 52 26.7 37.5 1.0X +MILLENNIUM of timestamp 1389 1410 34 7.2 138.9 0.3X +CENTURY of timestamp 1327 1345 25 7.5 132.7 0.3X +DECADE of timestamp 1214 1257 59 8.2 121.4 0.3X +YEAR of timestamp 1185 1192 6 8.4 118.5 0.3X +ISOYEAR of timestamp 1297 1371 93 7.7 129.7 0.3X +QUARTER of timestamp 1375 1395 32 7.3 137.5 0.3X +MONTH of timestamp 1179 1191 17 8.5 117.9 0.3X +WEEK of timestamp 1760 1778 21 5.7 176.0 0.2X +DAY of timestamp 1177 1185 8 8.5 117.7 0.3X +DAYOFWEEK of timestamp 1330 1331 1 7.5 133.0 0.3X +DOW of timestamp 1335 1362 43 7.5 133.5 0.3X +ISODOW of timestamp 1277 1282 8 7.8 127.7 0.3X +DOY of timestamp 1195 1208 16 8.4 119.5 0.3X +HOUR of timestamp 335 342 6 29.8 33.5 1.1X +MINUTE of timestamp 322 330 7 31.1 32.2 1.2X +SECOND of timestamp 324 330 8 30.9 32.4 1.2X +MILLISECONDS of timestamp 543 550 6 18.4 54.3 0.7X +MICROSECONDS of timestamp 426 431 8 23.5 42.6 0.9X +EPOCH of timestamp 29807 29831 29 0.3 2980.7 0.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1010 1022 11 9.9 101.0 1.0X -MILLENNIUM of date 1300 1311 18 7.7 130.0 0.8X -CENTURY of date 1304 1306 2 7.7 130.4 0.8X -DECADE of date 1199 1205 10 8.3 119.9 0.8X -YEAR of date 1191 1194 4 8.4 119.1 0.8X -ISOYEAR of date 1451 1456 9 6.9 145.1 0.7X -QUARTER of date 1494 1501 10 6.7 149.4 0.7X -MONTH of date 1189 1191 3 8.4 118.9 0.8X -WEEK of date 1893 1958 111 5.3 189.3 0.5X -DAY of date 1282 1285 3 7.8 128.2 0.8X -DAYOFWEEK of date 1374 1386 17 7.3 137.4 0.7X -DOW of date 1348 1351 3 7.4 134.8 0.7X -ISODOW of date 1292 1297 5 7.7 129.2 0.8X -DOY of date 1213 1216 3 8.2 121.3 0.8X -HOUR of date 1450 1458 9 6.9 145.0 0.7X -MINUTE of date 1445 1452 9 6.9 144.5 0.7X -SECOND of date 1448 1458 8 6.9 144.8 0.7X -MILLISECONDS of date 2094 2103 11 4.8 209.4 0.5X -MICROSECONDS of date 1562 1573 19 6.4 156.2 0.6X -EPOCH of date 31000 31047 68 0.3 3100.0 0.0X +cast to date 976 983 7 10.2 97.6 1.0X +MILLENNIUM of date 1265 1271 7 7.9 126.5 0.8X +CENTURY of date 1273 1285 20 7.9 127.3 0.8X +DECADE of date 1158 1166 8 8.6 115.8 0.8X +YEAR of date 1165 1177 19 8.6 116.5 0.8X +ISOYEAR of date 1394 1399 5 7.2 139.4 0.7X +QUARTER of date 1463 1464 1 6.8 146.3 0.7X +MONTH of date 1153 1156 3 8.7 115.3 0.8X +WEEK of date 1743 1748 9 5.7 174.3 0.6X +DAY of date 1145 1150 5 8.7 114.5 0.9X +DAYOFWEEK of date 1315 1316 2 7.6 131.5 0.7X +DOW of date 1315 1325 14 7.6 131.5 0.7X +ISODOW of date 1267 1269 2 7.9 126.7 0.8X +DOY of date 1193 1203 17 8.4 119.3 0.8X +HOUR of date 1419 1428 15 7.0 141.9 0.7X +MINUTE of date 1416 1423 6 7.1 141.6 0.7X +SECOND of date 1412 1416 6 7.1 141.2 0.7X +MILLISECONDS of date 1671 1676 9 6.0 167.1 0.6X +MICROSECONDS of date 1524 1528 4 6.6 152.4 0.6X +EPOCH of date 30922 30959 54 0.3 3092.2 0.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 327 333 10 30.6 32.7 1.0X -MILLENNIUM of timestamp 1292 1296 4 7.7 129.2 0.3X -CENTURY of timestamp 1301 1305 6 7.7 130.1 0.3X -DECADE of timestamp 1200 1204 6 8.3 120.0 0.3X -YEAR of timestamp 1185 1193 8 8.4 118.5 0.3X -ISOYEAR of timestamp 1449 1469 18 6.9 144.9 0.2X -QUARTER of timestamp 1497 1505 7 6.7 149.7 0.2X -MONTH of timestamp 1185 1188 3 8.4 118.5 0.3X -WEEK of timestamp 1901 1909 7 5.3 190.1 0.2X -DAY of timestamp 1278 1282 4 7.8 127.8 0.3X -DAYOFWEEK of timestamp 1371 1376 5 7.3 137.1 0.2X -DOW of timestamp 1361 1372 17 7.3 136.1 0.2X -ISODOW of timestamp 1299 1306 9 7.7 129.9 0.3X -DOY of timestamp 1216 1219 4 8.2 121.6 0.3X -HOUR of timestamp 352 356 5 28.4 35.2 0.9X -MINUTE of timestamp 350 369 17 28.6 35.0 0.9X -SECOND of timestamp 351 364 19 28.5 35.1 0.9X -MILLISECONDS of timestamp 36989 37022 52 0.3 3698.9 0.0X -MICROSECONDS of timestamp 473 476 2 21.1 47.3 0.7X -EPOCH of timestamp 29890 29908 27 0.3 2989.0 0.0X +cast to timestamp 292 296 7 34.3 29.2 1.0X +MILLENNIUM of timestamp 1263 1274 10 7.9 126.3 0.2X +CENTURY of timestamp 1271 1275 4 7.9 127.1 0.2X +DECADE of timestamp 1154 1157 3 8.7 115.4 0.3X +YEAR of timestamp 1151 1157 10 8.7 115.1 0.3X +ISOYEAR of timestamp 1392 1393 1 7.2 139.2 0.2X +QUARTER of timestamp 1463 1476 12 6.8 146.3 0.2X +MONTH of timestamp 1157 1173 20 8.6 115.7 0.3X +WEEK of timestamp 1742 1749 9 5.7 174.2 0.2X +DAY of timestamp 1140 1145 5 8.8 114.0 0.3X +DAYOFWEEK of timestamp 1312 1317 5 7.6 131.2 0.2X +DOW of timestamp 1318 1321 3 7.6 131.8 0.2X +ISODOW of timestamp 1268 1270 2 7.9 126.8 0.2X +DOY of timestamp 1194 1197 3 8.4 119.4 0.2X +HOUR of timestamp 327 330 4 30.6 32.7 0.9X +MINUTE of timestamp 320 326 9 31.3 32.0 0.9X +SECOND of timestamp 320 329 16 31.2 32.0 0.9X +MILLISECONDS of timestamp 540 544 7 18.5 54.0 0.5X +MICROSECONDS of timestamp 431 438 11 23.2 43.1 0.7X +EPOCH of timestamp 29802 29824 32 0.3 2980.2 0.0X OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1005 1006 1 9.9 100.5 1.0X -MILLENNIUM of date 1295 1300 5 7.7 129.5 0.8X -CENTURY of date 1297 1298 1 7.7 129.7 0.8X -DECADE of date 1198 1208 13 8.3 119.8 0.8X -YEAR of date 1184 1193 13 8.4 118.4 0.8X -ISOYEAR of date 1445 1460 13 6.9 144.5 0.7X -QUARTER of date 1495 1500 4 6.7 149.5 0.7X -MONTH of date 1176 1179 3 8.5 117.6 0.9X -WEEK of date 1893 1904 15 5.3 189.3 0.5X -DAY of date 1275 1283 8 7.8 127.5 0.8X -DAYOFWEEK of date 1369 1373 4 7.3 136.9 0.7X -DOW of date 1353 1354 2 7.4 135.3 0.7X -ISODOW of date 1290 1290 1 7.8 129.0 0.8X -DOY of date 1208 1212 4 8.3 120.8 0.8X -HOUR of date 1446 1449 2 6.9 144.6 0.7X -MINUTE of date 1441 1442 1 6.9 144.1 0.7X -SECOND of date 1443 1450 8 6.9 144.3 0.7X -MILLISECONDS of date 2087 2089 3 4.8 208.7 0.5X -MICROSECONDS of date 1557 1570 21 6.4 155.7 0.6X -EPOCH of date 30980 31001 32 0.3 3098.0 0.0X +cast to date 976 981 8 10.2 97.6 1.0X +MILLENNIUM of date 1264 1268 7 7.9 126.4 0.8X +CENTURY of date 1267 1275 14 7.9 126.7 0.8X +DECADE of date 1150 1153 4 8.7 115.0 0.8X +YEAR of date 1150 1154 6 8.7 115.0 0.8X +ISOYEAR of date 1390 1393 3 7.2 139.0 0.7X +QUARTER of date 1459 1462 3 6.9 145.9 0.7X +MONTH of date 1149 1150 1 8.7 114.9 0.8X +WEEK of date 1740 1741 1 5.7 174.0 0.6X +DAY of date 1145 1153 14 8.7 114.5 0.9X +DAYOFWEEK of date 1311 1314 3 7.6 131.1 0.7X +DOW of date 1318 1320 3 7.6 131.8 0.7X +ISODOW of date 1264 1266 2 7.9 126.4 0.8X +DOY of date 1187 1189 3 8.4 118.7 0.8X +HOUR of date 1418 1419 1 7.1 141.8 0.7X +MINUTE of date 1410 1412 2 7.1 141.0 0.7X +SECOND of date 1414 1415 1 7.1 141.4 0.7X +MILLISECONDS of date 1669 1676 5 6.0 166.9 0.6X +MICROSECONDS of date 1519 1525 6 6.6 151.9 0.6X +EPOCH of date 30900 30950 56 0.3 3090.0 0.0X From 89bad267d4b8cf451d1906aa57764d97e1a565f4 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 22 Sep 2019 16:59:59 +0900 Subject: [PATCH 0150/1387] [SPARK-29200][SQL] Optimize `extract`/`date_part` for epoch ### What changes were proposed in this pull request? Refactoring of the `DateTimeUtils.getEpoch()` function by avoiding decimal operations that are pretty expensive, and converting the final result to the decimal type at the end. ### Why are the changes needed? The changes improve performance of the `getEpoch()` method at least up to **20 times**. Before: ``` Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast to timestamp 256 277 33 39.0 25.6 1.0X EPOCH of timestamp 23455 23550 131 0.4 2345.5 0.0X ``` After: ``` Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast to timestamp 255 294 34 39.2 25.5 1.0X EPOCH of timestamp 1049 1054 9 9.5 104.9 0.2X ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test from `DateExpressionSuite`. Closes #25881 from MaxGekk/optimize-extract-epoch. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../sql/catalyst/util/DateTimeUtils.scala | 7 +- .../benchmarks/ExtractBenchmark-results.txt | 176 +++++++++--------- 2 files changed, 92 insertions(+), 91 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 7757ea51536c9..ffadd95052513 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -847,9 +847,10 @@ object DateTimeUtils { * since 1970-01-01 00:00:00 local time. */ def getEpoch(timestamp: SQLTimestamp, zoneId: ZoneId): Decimal = { - val offset = zoneId.getRules.getOffset(microsToInstant(timestamp)).getTotalSeconds - val sinceEpoch = BigDecimal(timestamp) / MICROS_PER_SECOND + offset - new Decimal().set(sinceEpoch, 20, 6) + val offset = SECONDS.toMicros( + zoneId.getRules.getOffset(microsToInstant(timestamp)).getTotalSeconds) + val sinceEpoch = timestamp + offset + Decimal(sinceEpoch, 20, 6) } def currentTimestamp(): SQLTimestamp = instantToMicros(Instant.now()) diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 4894db4c522f6..2e1df667bbe8c 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -1,100 +1,100 @@ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 375 411 52 26.7 37.5 1.0X -MILLENNIUM of timestamp 1389 1410 34 7.2 138.9 0.3X -CENTURY of timestamp 1327 1345 25 7.5 132.7 0.3X -DECADE of timestamp 1214 1257 59 8.2 121.4 0.3X -YEAR of timestamp 1185 1192 6 8.4 118.5 0.3X -ISOYEAR of timestamp 1297 1371 93 7.7 129.7 0.3X -QUARTER of timestamp 1375 1395 32 7.3 137.5 0.3X -MONTH of timestamp 1179 1191 17 8.5 117.9 0.3X -WEEK of timestamp 1760 1778 21 5.7 176.0 0.2X -DAY of timestamp 1177 1185 8 8.5 117.7 0.3X -DAYOFWEEK of timestamp 1330 1331 1 7.5 133.0 0.3X -DOW of timestamp 1335 1362 43 7.5 133.5 0.3X -ISODOW of timestamp 1277 1282 8 7.8 127.7 0.3X -DOY of timestamp 1195 1208 16 8.4 119.5 0.3X -HOUR of timestamp 335 342 6 29.8 33.5 1.1X -MINUTE of timestamp 322 330 7 31.1 32.2 1.2X -SECOND of timestamp 324 330 8 30.9 32.4 1.2X -MILLISECONDS of timestamp 543 550 6 18.4 54.3 0.7X -MICROSECONDS of timestamp 426 431 8 23.5 42.6 0.9X -EPOCH of timestamp 29807 29831 29 0.3 2980.7 0.0X +cast to timestamp 254 275 35 39.4 25.4 1.0X +MILLENNIUM of timestamp 1149 1159 9 8.7 114.9 0.2X +CENTURY of timestamp 1102 1115 16 9.1 110.2 0.2X +DECADE of timestamp 1024 1036 11 9.8 102.4 0.2X +YEAR of timestamp 1000 1004 5 10.0 100.0 0.3X +ISOYEAR of timestamp 1090 1101 11 9.2 109.0 0.2X +QUARTER of timestamp 1169 1178 7 8.6 116.9 0.2X +MONTH of timestamp 981 984 4 10.2 98.1 0.3X +WEEK of timestamp 1372 1388 14 7.3 137.2 0.2X +DAY of timestamp 994 1000 7 10.1 99.4 0.3X +DAYOFWEEK of timestamp 1102 1108 6 9.1 110.2 0.2X +DOW of timestamp 1102 1105 3 9.1 110.2 0.2X +ISODOW of timestamp 1063 1078 18 9.4 106.3 0.2X +DOY of timestamp 1015 1021 5 9.9 101.5 0.2X +HOUR of timestamp 385 390 5 26.0 38.5 0.7X +MINUTE of timestamp 387 391 7 25.8 38.7 0.7X +SECOND of timestamp 381 382 1 26.2 38.1 0.7X +MILLISECONDS of timestamp 584 588 4 17.1 58.4 0.4X +MICROSECONDS of timestamp 482 500 16 20.8 48.2 0.5X +EPOCH of timestamp 957 961 6 10.4 95.7 0.3X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 976 983 7 10.2 97.6 1.0X -MILLENNIUM of date 1265 1271 7 7.9 126.5 0.8X -CENTURY of date 1273 1285 20 7.9 127.3 0.8X -DECADE of date 1158 1166 8 8.6 115.8 0.8X -YEAR of date 1165 1177 19 8.6 116.5 0.8X -ISOYEAR of date 1394 1399 5 7.2 139.4 0.7X -QUARTER of date 1463 1464 1 6.8 146.3 0.7X -MONTH of date 1153 1156 3 8.7 115.3 0.8X -WEEK of date 1743 1748 9 5.7 174.3 0.6X -DAY of date 1145 1150 5 8.7 114.5 0.9X -DAYOFWEEK of date 1315 1316 2 7.6 131.5 0.7X -DOW of date 1315 1325 14 7.6 131.5 0.7X -ISODOW of date 1267 1269 2 7.9 126.7 0.8X -DOY of date 1193 1203 17 8.4 119.3 0.8X -HOUR of date 1419 1428 15 7.0 141.9 0.7X -MINUTE of date 1416 1423 6 7.1 141.6 0.7X -SECOND of date 1412 1416 6 7.1 141.2 0.7X -MILLISECONDS of date 1671 1676 9 6.0 167.1 0.6X -MICROSECONDS of date 1524 1528 4 6.6 152.4 0.6X -EPOCH of date 30922 30959 54 0.3 3092.2 0.0X +cast to date 834 842 12 12.0 83.4 1.0X +MILLENNIUM of date 1078 1088 16 9.3 107.8 0.8X +CENTURY of date 1063 1067 4 9.4 106.3 0.8X +DECADE of date 989 992 3 10.1 98.9 0.8X +YEAR of date 975 976 1 10.3 97.5 0.9X +ISOYEAR of date 1177 1186 9 8.5 117.7 0.7X +QUARTER of date 1219 1222 4 8.2 121.9 0.7X +MONTH of date 982 992 10 10.2 98.2 0.8X +WEEK of date 1360 1364 6 7.4 136.0 0.6X +DAY of date 973 980 7 10.3 97.3 0.9X +DAYOFWEEK of date 1100 1104 7 9.1 110.0 0.8X +DOW of date 1091 1096 4 9.2 109.1 0.8X +ISODOW of date 1053 1057 6 9.5 105.3 0.8X +DOY of date 1006 1012 4 9.9 100.6 0.8X +HOUR of date 1683 1688 6 5.9 168.3 0.5X +MINUTE of date 1686 1691 5 5.9 168.6 0.5X +SECOND of date 1706 1714 7 5.9 170.6 0.5X +MILLISECONDS of date 1881 1887 6 5.3 188.1 0.4X +MICROSECONDS of date 1767 1778 16 5.7 176.7 0.5X +EPOCH of date 2274 2281 7 4.4 227.4 0.4X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 292 296 7 34.3 29.2 1.0X -MILLENNIUM of timestamp 1263 1274 10 7.9 126.3 0.2X -CENTURY of timestamp 1271 1275 4 7.9 127.1 0.2X -DECADE of timestamp 1154 1157 3 8.7 115.4 0.3X -YEAR of timestamp 1151 1157 10 8.7 115.1 0.3X -ISOYEAR of timestamp 1392 1393 1 7.2 139.2 0.2X -QUARTER of timestamp 1463 1476 12 6.8 146.3 0.2X -MONTH of timestamp 1157 1173 20 8.6 115.7 0.3X -WEEK of timestamp 1742 1749 9 5.7 174.2 0.2X -DAY of timestamp 1140 1145 5 8.8 114.0 0.3X -DAYOFWEEK of timestamp 1312 1317 5 7.6 131.2 0.2X -DOW of timestamp 1318 1321 3 7.6 131.8 0.2X -ISODOW of timestamp 1268 1270 2 7.9 126.8 0.2X -DOY of timestamp 1194 1197 3 8.4 119.4 0.2X -HOUR of timestamp 327 330 4 30.6 32.7 0.9X -MINUTE of timestamp 320 326 9 31.3 32.0 0.9X -SECOND of timestamp 320 329 16 31.2 32.0 0.9X -MILLISECONDS of timestamp 540 544 7 18.5 54.0 0.5X -MICROSECONDS of timestamp 431 438 11 23.2 43.1 0.7X -EPOCH of timestamp 29802 29824 32 0.3 2980.2 0.0X +cast to timestamp 212 214 3 47.2 21.2 1.0X +MILLENNIUM of timestamp 1082 1092 10 9.2 108.2 0.2X +CENTURY of timestamp 1087 1089 2 9.2 108.7 0.2X +DECADE of timestamp 986 1001 21 10.1 98.6 0.2X +YEAR of timestamp 983 985 2 10.2 98.3 0.2X +ISOYEAR of timestamp 1160 1168 12 8.6 116.0 0.2X +QUARTER of timestamp 1219 1228 9 8.2 121.9 0.2X +MONTH of timestamp 973 979 5 10.3 97.3 0.2X +WEEK of timestamp 1348 1357 8 7.4 134.8 0.2X +DAY of timestamp 969 974 5 10.3 96.9 0.2X +DAYOFWEEK of timestamp 1093 1096 3 9.1 109.3 0.2X +DOW of timestamp 1089 1096 7 9.2 108.9 0.2X +ISODOW of timestamp 1048 1056 7 9.5 104.8 0.2X +DOY of timestamp 1007 1011 5 9.9 100.7 0.2X +HOUR of timestamp 385 387 3 26.0 38.5 0.6X +MINUTE of timestamp 382 384 2 26.2 38.2 0.6X +SECOND of timestamp 377 378 1 26.5 37.7 0.6X +MILLISECONDS of timestamp 572 574 2 17.5 57.2 0.4X +MICROSECONDS of timestamp 473 490 18 21.1 47.3 0.4X +EPOCH of timestamp 937 947 11 10.7 93.7 0.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.6 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 976 981 8 10.2 97.6 1.0X -MILLENNIUM of date 1264 1268 7 7.9 126.4 0.8X -CENTURY of date 1267 1275 14 7.9 126.7 0.8X -DECADE of date 1150 1153 4 8.7 115.0 0.8X -YEAR of date 1150 1154 6 8.7 115.0 0.8X -ISOYEAR of date 1390 1393 3 7.2 139.0 0.7X -QUARTER of date 1459 1462 3 6.9 145.9 0.7X -MONTH of date 1149 1150 1 8.7 114.9 0.8X -WEEK of date 1740 1741 1 5.7 174.0 0.6X -DAY of date 1145 1153 14 8.7 114.5 0.9X -DAYOFWEEK of date 1311 1314 3 7.6 131.1 0.7X -DOW of date 1318 1320 3 7.6 131.8 0.7X -ISODOW of date 1264 1266 2 7.9 126.4 0.8X -DOY of date 1187 1189 3 8.4 118.7 0.8X -HOUR of date 1418 1419 1 7.1 141.8 0.7X -MINUTE of date 1410 1412 2 7.1 141.0 0.7X -SECOND of date 1414 1415 1 7.1 141.4 0.7X -MILLISECONDS of date 1669 1676 5 6.0 166.9 0.6X -MICROSECONDS of date 1519 1525 6 6.6 151.9 0.6X -EPOCH of date 30900 30950 56 0.3 3090.0 0.0X +cast to date 835 836 1 12.0 83.5 1.0X +MILLENNIUM of date 1062 1070 10 9.4 106.2 0.8X +CENTURY of date 1060 1068 12 9.4 106.0 0.8X +DECADE of date 984 992 8 10.2 98.4 0.8X +YEAR of date 971 973 3 10.3 97.1 0.9X +ISOYEAR of date 1158 1175 22 8.6 115.8 0.7X +QUARTER of date 1239 1239 0 8.1 123.9 0.7X +MONTH of date 972 987 23 10.3 97.2 0.9X +WEEK of date 1346 1350 5 7.4 134.6 0.6X +DAY of date 970 971 1 10.3 97.0 0.9X +DAYOFWEEK of date 1099 1107 10 9.1 109.9 0.8X +DOW of date 1089 1091 2 9.2 108.9 0.8X +ISODOW of date 1053 1062 13 9.5 105.3 0.8X +DOY of date 1008 1010 4 9.9 100.8 0.8X +HOUR of date 1697 1703 7 5.9 169.7 0.5X +MINUTE of date 1697 1699 3 5.9 169.7 0.5X +SECOND of date 1682 1692 8 5.9 168.2 0.5X +MILLISECONDS of date 1884 1890 7 5.3 188.4 0.4X +MICROSECONDS of date 1765 1770 7 5.7 176.5 0.5X +EPOCH of date 2273 2282 8 4.4 227.3 0.4X From e2c47876e9bd7faacb196baf517f549d58391002 Mon Sep 17 00:00:00 2001 From: madianjun Date: Sun, 22 Sep 2019 17:22:37 +0900 Subject: [PATCH 0151/1387] [CORE][MINOR] Correct a log message in DAGScheduler ### What changes were proposed in this pull request? Correct a word in a log message. ### Why are the changes needed? Log message will be more clearly. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Test is not needed. Closes #25880 from mdianjun/fix-a-word. Authored-by: madianjun Signed-off-by: HyukjinKwon --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 894234f70e05a..f9d2df8185ec7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1521,8 +1521,8 @@ private[spark] class DAGScheduler( markStageAsFinished(failedStage, errorMessage = Some(failureMessage), willRetry = !shouldAbortStage) } else { - logDebug(s"Received fetch failure from $task, but its from $failedStage which is no " + - s"longer running") + logDebug(s"Received fetch failure from $task, but it's from $failedStage which is no " + + "longer running") } if (mapStage.rdd.isBarrier()) { From 051e691029c456fc2db5f229485d3fb8f5d0e84c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 22 Sep 2019 17:31:33 +0900 Subject: [PATCH 0152/1387] [SPARK-28141][SQL] Support special date values ### What changes were proposed in this pull request? Supported special string values for `DATE` type. They are simply notational shorthands that will be converted to ordinary date values when read. The following string values are supported: - `epoch [zoneId]` - `1970-01-01` - `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone`. - `yesterday [zoneId]` - the current date -1 - `tomorrow [zoneId]` - the current date + 1 - `now` - the date of running the current query. It has the same notion as `today`. For example: ```sql spark-sql> SELECT date 'tomorrow' - date 'yesterday'; 2 ``` ### Why are the changes needed? To maintain feature parity with PostgreSQL, see [8.5.1.4. Special Values](https://www.postgresql.org/docs/12/datatype-datetime.html) ### Does this PR introduce any user-facing change? Previously, the parser fails on the special values with the error: ```sql spark-sql> select date 'today'; Error in query: Cannot parse the DATE value: today(line 1, pos 7) ``` After the changes, the special values are converted to appropriate dates: ```sql spark-sql> select date 'today'; 2019-09-06 ``` ### How was this patch tested? - Added tests to `DateFormatterSuite` to check parsing special values from regular strings. - Tests in `DateTimeUtilsSuite` check parsing those values from `UTF8String` - Uncommented tests in `date.sql` Closes #25708 from MaxGekk/datetime-special-values. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../sql/catalyst/catalog/interface.scala | 4 +- .../sql/catalyst/csv/UnivocityGenerator.scala | 5 +- .../sql/catalyst/csv/UnivocityParser.scala | 5 +- .../spark/sql/catalyst/expressions/Cast.scala | 51 +-- .../expressions/datetimeExpressions.scala | 4 +- .../sql/catalyst/expressions/literals.scala | 4 +- .../sql/catalyst/json/JacksonGenerator.scala | 5 +- .../sql/catalyst/json/JacksonParser.scala | 5 +- .../sql/catalyst/parser/AstBuilder.scala | 3 +- .../sql/catalyst/util/DateFormatter.scala | 22 +- .../sql/catalyst/util/DateTimeUtils.scala | 31 +- .../expressions/HashExpressionsSuite.scala | 2 +- .../catalyst/util/DateTimeUtilsSuite.scala | 62 ++-- .../sql/catalyst/util/UnsafeArraySuite.scala | 6 +- .../spark/sql/util/DateFormatterSuite.scala | 31 +- .../spark/sql/execution/HiveResult.scala | 6 +- .../datasources/PartitioningUtils.scala | 4 +- .../datasources/jdbc/JDBCRelation.scala | 6 +- .../resources/sql-tests/inputs/pgSQL/date.sql | 17 +- .../sql-tests/results/pgSQL/date.sql.out | 328 +++++++++++------- .../apache/spark/sql/CsvFunctionsSuite.scala | 11 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 11 +- .../ParquetPartitionDiscoverySuite.scala | 2 +- 23 files changed, 400 insertions(+), 225 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 01b21feab0dd9..f653bf41c1624 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -491,7 +491,7 @@ object CatalogColumnStat extends Logging { dataType match { case BooleanType => s.toBoolean case DateType if version == 1 => DateTimeUtils.fromJavaDate(java.sql.Date.valueOf(s)) - case DateType => DateFormatter().parse(s) + case DateType => DateFormatter(ZoneOffset.UTC).parse(s) case TimestampType if version == 1 => DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s)) case TimestampType => getTimestampFormatter().parse(s) @@ -516,7 +516,7 @@ object CatalogColumnStat extends Logging { */ def toExternalString(v: Any, colName: String, dataType: DataType): String = { val externalValue = dataType match { - case DateType => DateFormatter().format(v.asInstanceOf[Int]) + case DateType => DateFormatter(ZoneOffset.UTC).format(v.asInstanceOf[Int]) case TimestampType => getTimestampFormatter().format(v.asInstanceOf[Long]) case BooleanType | _: IntegralType | FloatType | DoubleType => v case _: DecimalType => v.asInstanceOf[Decimal].toJavaBigDecimal diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala index 9ca94501f5c58..05cb91d10868e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala @@ -45,7 +45,10 @@ class UnivocityGenerator( options.timestampFormat, options.zoneId, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private val dateFormatter = DateFormatter( + options.dateFormat, + options.zoneId, + options.locale) private def makeConverter(dataType: DataType): ValueConverter = dataType match { case DateType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 39a08ec06c6a0..661525a65294d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -78,7 +78,10 @@ class UnivocityParser( options.timestampFormat, options.zoneId, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private val dateFormatter = DateFormatter( + options.dateFormat, + options.zoneId, + options.locale) // Retrieve the raw record string. private def getCurrentInput: UTF8String = { 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 5001b4350d778..118f261de775d 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 @@ -106,7 +106,7 @@ object Cast { * * Cast.castToTimestamp */ def needsTimeZone(from: DataType, to: DataType): Boolean = (from, to) match { - case (StringType, TimestampType) => true + case (StringType, TimestampType | DateType) => true case (DateType, TimestampType) => true case (TimestampType, StringType) => true case (TimestampType, DateType) => true @@ -287,7 +287,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // [[func]] assumes the input is no longer null because eval already does the null check. @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) - private lazy val dateFormatter = DateFormatter() + private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) private val failOnIntegralTypeOverflow = SQLConf.get.ansiEnabled @@ -469,7 +469,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => DateTimeUtils.stringToDate(s).orNull) + buildCast[UTF8String](_, s => DateTimeUtils.stringToDate(s, zoneId).orNull) case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. @@ -1056,28 +1056,35 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castToDateCode( from: DataType, - ctx: CodegenContext): CastFunction = from match { - case StringType => - val intOpt = ctx.freshVariable("intOpt", classOf[Option[Integer]]) - (c, evPrim, evNull) => code""" - scala.Option $intOpt = - org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToDate($c); - if ($intOpt.isDefined()) { - $evPrim = ((Integer) $intOpt.get()).intValue(); - } else { - $evNull = true; - } - """ - case TimestampType => + ctx: CodegenContext): CastFunction = { + def getZoneId() = { val zoneIdClass = classOf[ZoneId] - val zid = JavaCode.global( + JavaCode.global( ctx.addReferenceObj("zoneId", zoneId, zoneIdClass.getName), zoneIdClass) - (c, evPrim, evNull) => - code"""$evPrim = - org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToEpochDays($c, $zid);""" - case _ => - (c, evPrim, evNull) => code"$evNull = true;" + } + from match { + case StringType => + val intOpt = ctx.freshVariable("intOpt", classOf[Option[Integer]]) + val zid = getZoneId() + (c, evPrim, evNull) => + code""" + scala.Option $intOpt = + org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToDate($c, $zid); + if ($intOpt.isDefined()) { + $evPrim = ((Integer) $intOpt.get()).intValue(); + } else { + $evNull = true; + } + """ + case TimestampType => + val zid = getZoneId() + (c, evPrim, evNull) => + code"""$evPrim = + org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToEpochDays($c, $zid);""" + case _ => + (c, evPrim, evNull) => code"$evNull = true;" + } } private[this] def changePrecision(d: ExprValue, decimalType: DecimalType, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 592b9de83d9a0..0098226b3258c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -76,9 +76,7 @@ case class CurrentDate(timeZoneId: Option[String] = None) override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) - override def eval(input: InternalRow): Any = { - localDateToDays(LocalDate.now(zoneId)) - } + override def eval(input: InternalRow): Any = currentDate(zoneId) override def prettyName: String = "current_date" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 9cef3ecadc543..4793b5942a79e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -371,7 +371,9 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { case _ => v + "D" } case (v: Decimal, t: DecimalType) => v + "BD" - case (v: Int, DateType) => s"DATE '${DateFormatter().format(v)}'" + case (v: Int, DateType) => + val formatter = DateFormatter(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + s"DATE '${formatter.format(v)}'" case (v: Long, TimestampType) => val formatter = TimestampFormatter.getFractionFormatter( DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index 3378040d1b640..3ee7e484690d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -81,7 +81,10 @@ private[sql] class JacksonGenerator( options.timestampFormat, options.zoneId, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private val dateFormatter = DateFormatter( + options.dateFormat, + options.zoneId, + options.locale) private def makeWriter(dataType: DataType): ValueWriter = dataType match { case NullType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 19bc5bf3b29e3..b534b5a3d2d64 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -59,7 +59,10 @@ class JacksonParser( options.timestampFormat, options.zoneId, options.locale) - private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private val dateFormatter = DateFormatter( + options.dateFormat, + options.zoneId, + options.locale) /** * Create a converter which converts the JSON documents held by the `JsonParser` diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c334d09b691ea..74c485e091cd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1734,7 +1734,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } try { valueType match { - case "DATE" => toLiteral(stringToDate, DateType) + case "DATE" => + toLiteral(stringToDate(_, getZoneId(SQLConf.get.sessionLocalTimeZone)), DateType) case "TIMESTAMP" => val zoneId = getZoneId(SQLConf.get.sessionLocalTimeZone) toLiteral(stringToTimestamp(_, zoneId), TimestampType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 4940aa83a3017..7f982b019c8d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.catalyst.util -import java.time.LocalDate +import java.time.{LocalDate, ZoneId} import java.util.Locale +import DateTimeUtils.{convertSpecialDate, localDateToDays} + sealed trait DateFormatter extends Serializable { def parse(s: String): Int // returns days since epoch def format(days: Int): String @@ -27,14 +29,18 @@ sealed trait DateFormatter extends Serializable { class Iso8601DateFormatter( pattern: String, + zoneId: ZoneId, locale: Locale) extends DateFormatter with DateTimeFormatterHelper { @transient private lazy val formatter = getOrCreateFormatter(pattern, locale) override def parse(s: String): Int = { - val localDate = LocalDate.parse(s, formatter) - DateTimeUtils.localDateToDays(localDate) + val specialDate = convertSpecialDate(s.trim, zoneId) + specialDate.getOrElse { + val localDate = LocalDate.parse(s, formatter) + localDateToDays(localDate) + } } override def format(days: Int): String = { @@ -46,11 +52,13 @@ object DateFormatter { val defaultPattern: String = "uuuu-MM-dd" val defaultLocale: Locale = Locale.US - def apply(format: String, locale: Locale): DateFormatter = { - new Iso8601DateFormatter(format, locale) + def apply(format: String, zoneId: ZoneId, locale: Locale): DateFormatter = { + new Iso8601DateFormatter(format, zoneId, locale) } - def apply(format: String): DateFormatter = apply(format, defaultLocale) + def apply(format: String, zoneId: ZoneId): DateFormatter = { + apply(format, zoneId, defaultLocale) + } - def apply(): DateFormatter = apply(defaultPattern) + def apply(zoneId: ZoneId): DateFormatter = apply(defaultPattern, zoneId) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index ffadd95052513..3cf310a0a9683 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -24,7 +24,6 @@ import java.time.temporal.{ChronoField, ChronoUnit, IsoFields} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit._ -import scala.util.Try import scala.util.control.NonFatal import org.apache.spark.sql.types.Decimal @@ -379,7 +378,7 @@ object DateTimeUtils { * `yyyy-[m]m-[d]d *` * `yyyy-[m]m-[d]dT*` */ - def stringToDate(s: UTF8String): Option[SQLDate] = { + def stringToDate(s: UTF8String, zoneId: ZoneId): Option[SQLDate] = { if (s == null) { return None } @@ -387,6 +386,8 @@ object DateTimeUtils { var i = 0 var currentSegmentValue = 0 val bytes = s.trim.getBytes + val specialDate = convertSpecialDate(bytes, zoneId) + if (specialDate.isDefined) return specialDate var j = 0 while (j < bytes.length && (i < 3 && !(bytes(j) == ' ' || bytes(j) == 'T'))) { val b = bytes(j) @@ -855,6 +856,8 @@ object DateTimeUtils { def currentTimestamp(): SQLTimestamp = instantToMicros(Instant.now()) + def currentDate(zoneId: ZoneId): SQLDate = localDateToDays(LocalDate.now(zoneId)) + private def today(zoneId: ZoneId): ZonedDateTime = { Instant.now().atZone(zoneId).`with`(LocalTime.MIDNIGHT) } @@ -915,4 +918,28 @@ object DateTimeUtils { None } } + + /** + * Converts notational shorthands that are converted to ordinary dates. + * @param input - a trimmed string + * @param zoneId - zone identifier used to get the current date. + * @return some of days since the epoch if the conversion completed successfully otherwise None. + */ + def convertSpecialDate(input: String, zoneId: ZoneId): Option[SQLDate] = { + extractSpecialValue(input, zoneId).flatMap { + case "epoch" => Some(0) + case "now" | "today" => Some(currentDate(zoneId)) + case "tomorrow" => Some(Math.addExact(currentDate(zoneId), 1)) + case "yesterday" => Some(Math.subtractExact(currentDate(zoneId), 1)) + case _ => None + } + } + + private def convertSpecialDate(bytes: Array[Byte], zoneId: ZoneId): Option[SQLDate] = { + if (bytes.length > 0 && Character.isAlphabetic(bytes(0))) { + convertSpecialDate(new String(bytes, StandardCharsets.UTF_8), zoneId) + } else { + None + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index b5cfaf8f4b0fd..f90c98be0b3fd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -174,7 +174,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("hive-hash for date type") { def checkHiveHashForDateType(dateString: String, expected: Long): Unit = { checkHiveHash( - DateTimeUtils.stringToDate(UTF8String.fromString(dateString)).get, + DateTimeUtils.stringToDate(UTF8String.fromString(dateString), ZoneOffset.UTC).get, DateType, expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 31fefd613f9c8..1da8efe4ef42c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{LocalDateTime, LocalTime, ZoneId} +import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneId, ZoneOffset} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit @@ -120,28 +120,32 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { checkFromToJavaDate(new Date(df2.parse("1776-07-04 18:30:00 UTC").getTime)) } + private def toDate(s: String, zoneId: ZoneId = ZoneOffset.UTC): Option[SQLDate] = { + stringToDate(UTF8String.fromString(s), zoneId) + } + test("string to date") { - assert(stringToDate(UTF8String.fromString("2015-01-28")).get === days(2015, 1, 28)) - assert(stringToDate(UTF8String.fromString("2015")).get === days(2015, 1, 1)) - assert(stringToDate(UTF8String.fromString("0001")).get === days(1, 1, 1)) - assert(stringToDate(UTF8String.fromString("2015-03")).get === days(2015, 3, 1)) + assert(toDate("2015-01-28").get === days(2015, 1, 28)) + assert(toDate("2015").get === days(2015, 1, 1)) + assert(toDate("0001").get === days(1, 1, 1)) + assert(toDate("2015-03").get === days(2015, 3, 1)) Seq("2015-03-18", "2015-03-18 ", " 2015-03-18", " 2015-03-18 ", "2015-03-18 123142", "2015-03-18T123123", "2015-03-18T").foreach { s => - assert(stringToDate(UTF8String.fromString(s)).get === days(2015, 3, 18)) + assert(toDate(s).get === days(2015, 3, 18)) } - assert(stringToDate(UTF8String.fromString("2015-03-18X")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015/03/18")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015.03.18")).isEmpty) - assert(stringToDate(UTF8String.fromString("20150318")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015-031-8")).isEmpty) - assert(stringToDate(UTF8String.fromString("02015-03-18")).isEmpty) - assert(stringToDate(UTF8String.fromString("015-03-18")).isEmpty) - assert(stringToDate(UTF8String.fromString("015")).isEmpty) - assert(stringToDate(UTF8String.fromString("02015")).isEmpty) - assert(stringToDate(UTF8String.fromString("1999 08 01")).isEmpty) - assert(stringToDate(UTF8String.fromString("1999-08 01")).isEmpty) - assert(stringToDate(UTF8String.fromString("1999 08")).isEmpty) + assert(toDate("2015-03-18X").isEmpty) + assert(toDate("2015/03/18").isEmpty) + assert(toDate("2015.03.18").isEmpty) + assert(toDate("20150318").isEmpty) + assert(toDate("2015-031-8").isEmpty) + assert(toDate("02015-03-18").isEmpty) + assert(toDate("015-03-18").isEmpty) + assert(toDate("015").isEmpty) + assert(toDate("02015").isEmpty) + assert(toDate("1999 08 01").isEmpty) + assert(toDate("1999-08 01").isEmpty) + assert(toDate("1999 08").isEmpty) } private def toTimestamp(str: String, zoneId: ZoneId): Option[SQLTimestamp] = { @@ -264,12 +268,10 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { test("SPARK-15379: special invalid date string") { // Test stringToDate - assert(stringToDate( - UTF8String.fromString("2015-02-29 00:00:00")).isEmpty) - assert(stringToDate( - UTF8String.fromString("2015-04-31 00:00:00")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015-02-29")).isEmpty) - assert(stringToDate(UTF8String.fromString("2015-04-31")).isEmpty) + assert(toDate("2015-02-29 00:00:00").isEmpty) + assert(toDate("2015-04-31 00:00:00").isEmpty) + assert(toDate("2015-02-29").isEmpty) + assert(toDate("2015-04-31").isEmpty) // Test stringToTimestamp @@ -586,4 +588,16 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { toTimestamp(" tomorrow CET ", zoneId).get should be (today + MICROS_PER_DAY +- tolerance) } } + + test("special date values") { + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + assert(toDate("epoch", zoneId).get === 0) + val today = localDateToDays(LocalDate.now(zoneId)) + assert(toDate("YESTERDAY", zoneId).get === today - 1) + assert(toDate(" Now ", zoneId).get === today) + assert(toDate("now UTC", zoneId) === None) // "now" does not accept time zones + assert(toDate("today", zoneId).get === today) + assert(toDate("tomorrow CET ", zoneId).get === today + 1) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 0b9e023b0b450..41adf845a6fab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.util -import java.time.ZoneId +import java.time.{ZoneId, ZoneOffset} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -38,8 +38,8 @@ class UnsafeArraySuite extends SparkFunSuite { val doubleArray = Array(1.1, 2.2, 3.3) val stringArray = Array("1", "10", "100") val dateArray = Array( - DateTimeUtils.stringToDate(UTF8String.fromString("1970-1-1")).get, - DateTimeUtils.stringToDate(UTF8String.fromString("2016-7-26")).get) + DateTimeUtils.stringToDate(UTF8String.fromString("1970-1-1"), ZoneOffset.UTC).get, + DateTimeUtils.stringToDate(UTF8String.fromString("2016-7-26"), ZoneOffset.UTC).get) private def defaultZoneId = ZoneId.systemDefault() val timestampArray = Array( DateTimeUtils.stringToTimestamp( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala index 1f0eff2e5b114..291d40a9e84d3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala @@ -17,18 +17,19 @@ package org.apache.spark.sql.util -import java.time.LocalDate +import java.time.{LocalDate, ZoneOffset} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, localDateToDays} import org.apache.spark.sql.internal.SQLConf class DateFormatterSuite extends SparkFunSuite with SQLHelper { test("parsing dates") { DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter() + val formatter = DateFormatter(getZoneId(timeZone)) val daysSinceEpoch = formatter.parse("2018-12-02") assert(daysSinceEpoch === 17867) } @@ -38,7 +39,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { test("format dates") { DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter() + val formatter = DateFormatter(getZoneId(timeZone)) val date = formatter.format(17867) assert(date === "2018-12-02") } @@ -58,7 +59,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { "5010-11-17").foreach { date => DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter() + val formatter = DateFormatter(getZoneId(timeZone)) val days = formatter.parse(date) val formatted = formatter.format(days) assert(date === formatted) @@ -81,7 +82,7 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { 1110657).foreach { days => DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { - val formatter = DateFormatter() + val formatter = DateFormatter(getZoneId(timeZone)) val date = formatter.format(days) val parsed = formatter.parse(date) assert(days === parsed) @@ -91,13 +92,29 @@ class DateFormatterSuite extends SparkFunSuite with SQLHelper { } test("parsing date without explicit day") { - val formatter = DateFormatter("yyyy MMM") + val formatter = DateFormatter("yyyy MMM", ZoneOffset.UTC) val daysSinceEpoch = formatter.parse("2018 Dec") assert(daysSinceEpoch === LocalDate.of(2018, 12, 1).toEpochDay) } test("formatting negative years with default pattern") { val epochDays = LocalDate.of(-99, 1, 1).toEpochDay.toInt - assert(DateFormatter().format(epochDays) === "-0099-01-01") + assert(DateFormatter(ZoneOffset.UTC).format(epochDays) === "-0099-01-01") + } + + test("special date values") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val zoneId = getZoneId(timeZone) + val formatter = DateFormatter(zoneId) + + assert(formatter.parse("EPOCH") === 0) + val today = localDateToDays(LocalDate.now(zoneId)) + assert(formatter.parse("Yesterday") === today - 1) + assert(formatter.parse("now") === today) + assert(formatter.parse("today ") === today) + assert(formatter.parse("tomorrow UTC") === today + 1) + } + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index eec8d70b5adf0..75abac4cfd1da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -77,9 +77,9 @@ object HiveResult { TimestampType, BinaryType) - private lazy val dateFormatter = DateFormatter() - private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter( - DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + private lazy val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) + private lazy val dateFormatter = DateFormatter(zoneId) + private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) /** Hive outputs fields of structs slightly differently than top level attributes. */ private def toHiveStructString(a: (Any, DataType)): String = a match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 1e47d53b7e976..fdad43b23c5aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -130,7 +130,7 @@ object PartitioningUtils { Map.empty[String, String] } - val dateFormatter = DateFormatter() + val dateFormatter = DateFormatter(zoneId) val timestampFormatter = TimestampFormatter(timestampPartitionPattern, zoneId) // First, we need to parse every partition's path and see if we can find partition values. val (partitionValues, optDiscoveredBasePaths) = paths.map { path => @@ -492,7 +492,7 @@ object PartitioningUtils { // We need to check that we can cast the raw string since we later can use Cast to get // the partition values with the right DataType (see // org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning) - val dateValue = Cast(Literal(raw), DateType).eval() + val dateValue = Cast(Literal(raw), DateType, Some(zoneId.getId)).eval() // Disallow DateType if the cast returned null require(dateValue != null) Literal.create(dateValue, DateType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 3cd5cb1647923..f5a474ddf3904 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -186,7 +186,7 @@ private[sql] object JDBCRelation extends Logging { } columnType match { case _: NumericType => value.toLong - case DateType => parse(stringToDate).toLong + case DateType => parse(stringToDate(_, getZoneId(timeZoneId))).toLong case TimestampType => parse(stringToTimestamp(_, getZoneId(timeZoneId))) } } @@ -197,7 +197,9 @@ private[sql] object JDBCRelation extends Logging { timeZoneId: String): String = { def dateTimeToString(): String = { val dateTimeStr = columnType match { - case DateType => DateFormatter().format(value.toInt) + case DateType => + val dateFormatter = DateFormatter(DateTimeUtils.getZoneId(timeZoneId)) + dateFormatter.format(value.toInt) case TimestampType => val timestampFormatter = TimestampFormatter.getFractionFormatter( DateTimeUtils.getZoneId(timeZoneId)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index b9a6b998e52fe..d3cd46e4e6b89 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -208,20 +208,19 @@ SELECT date '5874898-01-01'; -- out of range SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; --- [SPARK-28141] Date type can not accept special values --- SELECT f1 - date 'epoch' AS "Days From Epoch" FROM DATE_TBL; +SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL; --- SELECT date 'yesterday' - date 'today' AS "One day"; +SELECT date 'yesterday' - date 'today' AS `One day`; --- SELECT date 'today' - date 'tomorrow' AS "One day"; +SELECT date 'today' - date 'tomorrow' AS `One day`; --- SELECT date 'yesterday' - date 'tomorrow' AS "Two days"; +SELECT date 'yesterday' - date 'tomorrow' AS `Two days`; --- SELECT date 'tomorrow' - date 'today' AS "One day"; +SELECT date 'tomorrow' - date 'today' AS `One day`; --- SELECT date 'today' - date 'yesterday' AS "One day"; +SELECT date 'today' - date 'yesterday' AS `One day`; --- SELECT date 'tomorrow' - date 'yesterday' AS "Two days"; +SELECT date 'tomorrow' - date 'yesterday' AS `Two days`; -- [SPARK-28017] Enhance date EXTRACT -- @@ -290,7 +289,7 @@ SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- 0011-01-01 BC --- [SPARK-28141] Date type can not accept special values +-- [SPARK-29006] Support special date/timestamp values `infinity`/`-infinity` -- -- test infinity -- diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index 083832007d618..29fcf61bd5b78 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 91 +-- Number of queries: 98 -- !query 0 @@ -502,352 +502,422 @@ struct -- !query 47 -SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') +SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL -- !query 47 schema -struct +struct -- !query 47 output -0 +-4585 +-4650 +11048 +11049 +11050 +24934 +25300 +25667 +9554 +9555 +9556 +9557 +9920 +9921 +9922 -- !query 48 -SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') +SELECT date 'yesterday' - date 'today' AS `One day` -- !query 48 schema -struct +struct -- !query 48 output -0 +-1 -- !query 49 -SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) +SELECT date 'today' - date 'tomorrow' AS `One day` -- !query 49 schema -struct +struct -- !query 49 output --2 +-1 -- !query 50 -SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) +SELECT date 'yesterday' - date 'tomorrow' AS `Two days` -- !query 50 schema -struct +struct -- !query 50 output --1 +-2 -- !query 51 -SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) +SELECT date 'tomorrow' - date 'today' AS `One day` -- !query 51 schema -struct +struct -- !query 51 output --1 +1 -- !query 52 -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') +SELECT date 'today' - date 'yesterday' AS `One day` -- !query 52 schema -struct +struct -- !query 52 output 1 -- !query 53 -SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') +SELECT date 'tomorrow' - date 'yesterday' AS `Two days` -- !query 53 schema -struct +struct -- !query 53 output -1 +2 -- !query 54 -SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') +SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') -- !query 54 schema -struct +struct -- !query 54 output -19 +0 -- !query 55 -SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') +SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 55 schema -struct +struct -- !query 55 output -20 +0 -- !query 56 -SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') +SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) -- !query 56 schema -struct +struct -- !query 56 output -20 +-2 -- !query 57 -SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') +SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) -- !query 57 schema -struct +struct -- !query 57 output -21 +-1 -- !query 58 -SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True +SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 58 schema -struct +struct -- !query 58 output -true +-1 -- !query 59 -SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') -- !query 59 schema -struct +struct -- !query 59 output --1 +1 -- !query 60 -SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') +SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') -- !query 60 schema -struct +struct -- !query 60 output 1 -- !query 61 -SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') +SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') -- !query 61 schema -struct +struct -- !query 61 output -1 +19 -- !query 62 -SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') +SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') -- !query 62 schema -struct +struct -- !query 62 output -2 +20 -- !query 63 -SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') +SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') -- !query 63 schema -struct +struct -- !query 63 output -2 +20 -- !query 64 -SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') +SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') -- !query 64 schema -struct +struct -- !query 64 output -3 +21 -- !query 65 -SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) +SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True -- !query 65 schema -struct +struct -- !query 65 output -3 +true -- !query 66 -SELECT EXTRACT(DECADE FROM DATE '1994-12-25') +SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) -- !query 66 schema -struct +struct -- !query 66 output -199 +-1 -- !query 67 -SELECT EXTRACT(DECADE FROM DATE '0010-01-01') +SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') -- !query 67 schema -struct +struct -- !query 67 output 1 -- !query 68 -SELECT EXTRACT(DECADE FROM DATE '0009-12-31') +SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') -- !query 68 schema -struct +struct -- !query 68 output -0 +1 -- !query 69 -SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') -- !query 69 schema -struct +struct -- !query 69 output -0 +2 -- !query 70 -SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') -- !query 70 schema -struct +struct -- !query 70 output --1 +2 -- !query 71 -SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') -- !query 71 schema -struct +struct -- !query 71 output --1 +3 -- !query 72 -SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) -- !query 72 schema -struct +struct -- !query 72 output --2 +3 -- !query 73 -SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True +SELECT EXTRACT(DECADE FROM DATE '1994-12-25') -- !query 73 schema -struct +struct -- !query 73 output -true +199 -- !query 74 -SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(DECADE FROM DATE '0010-01-01') -- !query 74 schema -struct +struct -- !query 74 output -20 +1 -- !query 75 -SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(DECADE FROM DATE '0009-12-31') -- !query 75 schema -struct +struct -- !query 75 output -1001-01-01 00:07:02 +0 -- !query 76 -SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') +SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) -- !query 76 schema -struct +struct -- !query 76 output -1001-01-01 00:07:02 +0 -- !query 77 -SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') +SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) -- !query 77 schema -struct +struct -- !query 77 output -1901-01-01 00:00:00 +-1 -- !query 78 -SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') +SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) -- !query 78 schema -struct +struct -- !query 78 output -1901-01-01 00:00:00 +-1 -- !query 79 -SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') +SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) -- !query 79 schema -struct +struct -- !query 79 output -2001-01-01 00:00:00 +-2 -- !query 80 -SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') +SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True -- !query 80 schema -struct +struct -- !query 80 output -0001-01-01 00:07:02 +true -- !query 81 -SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) +SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 81 schema -struct +struct -- !query 81 output --0099-01-01 00:07:02 +20 -- !query 82 -SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') +SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 82 schema -struct +struct -- !query 82 output -1990-01-01 00:00:00 +1001-01-01 00:07:02 -- !query 83 -SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') +SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') -- !query 83 schema -struct +struct -- !query 83 output -0000-01-01 00:07:02 +1001-01-01 00:07:02 -- !query 84 -SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 84 schema -struct +struct -- !query 84 output --0010-01-01 00:07:02 +1901-01-01 00:00:00 -- !query 85 -select make_date(2013, 7, 15) +SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') -- !query 85 schema -struct +struct -- !query 85 output -2013-07-15 +1901-01-01 00:00:00 -- !query 86 -select make_date(-44, 3, 15) +SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') -- !query 86 schema -struct +struct -- !query 86 output --0044-03-15 +2001-01-01 00:00:00 -- !query 87 -select make_date(2013, 2, 30) +SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') -- !query 87 schema -struct +struct -- !query 87 output -NULL +0001-01-01 00:07:02 -- !query 88 -select make_date(2013, 13, 1) +SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) -- !query 88 schema -struct +struct -- !query 88 output -NULL +-0099-01-01 00:07:02 -- !query 89 -select make_date(2013, 11, -1) +SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') -- !query 89 schema -struct +struct -- !query 89 output -NULL +1990-01-01 00:00:00 -- !query 90 -DROP TABLE DATE_TBL +SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') -- !query 90 schema -struct<> +struct -- !query 90 output +0000-01-01 00:07:02 + + +-- !query 91 +SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) +-- !query 91 schema +struct +-- !query 91 output +-0010-01-01 00:07:02 + + +-- !query 92 +select make_date(2013, 7, 15) +-- !query 92 schema +struct +-- !query 92 output +2013-07-15 + + +-- !query 93 +select make_date(-44, 3, 15) +-- !query 93 schema +struct +-- !query 93 output +-0044-03-15 + + +-- !query 94 +select make_date(2013, 2, 30) +-- !query 94 schema +struct +-- !query 94 output +NULL + + +-- !query 95 +select make_date(2013, 13, 1) +-- !query 95 schema +struct +-- !query 95 output +NULL + + +-- !query 96 +select make_date(2013, 11, -1) +-- !query 96 schema +struct +-- !query 96 output +NULL + + +-- !query 97 +DROP TABLE DATE_TBL +-- !query 97 schema +struct<> +-- !query 97 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index 1094d7d23e5ea..d34e50518348a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.Locale @@ -191,4 +191,13 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { assert(readback(0).getAs[Row](0).getAs[Timestamp](0).getTime >= 0) } } + + test("special date values") { + Seq("now", "today", "epoch", "tomorrow", "yesterday").foreach { specialValue => + val input = Seq(specialValue).toDS() + val readback = input.select(from_csv($"value", lit("d date"), + Map.empty[String, String].asJava)).collect() + assert(readback(0).getAs[Row](0).getAs[Date](0).getTime >= 0) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index c61c8109ee8e6..e55d2bbe00e63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.Locale @@ -618,4 +618,13 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { assert(readback(0).getAs[Row](0).getAs[Timestamp](0).getTime >= 0) } } + + test("special date values") { + Seq("now", "today", "epoch", "tomorrow", "yesterday").foreach { specialValue => + val input = Seq(s"""{"d": "$specialValue"}""").toDS() + val readback = input.select(from_json($"value", lit("d date"), + Map.empty[String, String].asJava)).collect() + assert(readback(0).getAs[Row](0).getAs[Date](0).getTime >= 0) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 0a85e3cdeaf1d..138336dc7e333 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -58,7 +58,7 @@ abstract class ParquetPartitionDiscoverySuite val defaultPartitionName = ExternalCatalogUtils.DEFAULT_PARTITION_NAME val timeZoneId = ZoneId.systemDefault() - val df = DateFormatter() + val df = DateFormatter(timeZoneId) val tf = TimestampFormatter(timestampPartitionPattern, timeZoneId) protected override def beforeAll(): Unit = { From a838dbd2f97389fda096093cbe6832a07fab4a0d Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sun, 22 Sep 2019 21:39:30 +0900 Subject: [PATCH 0153/1387] [SPARK-27463][PYTHON][FOLLOW-UP] Run the tests of Cogrouped pandas UDF ### What changes were proposed in this pull request? This is a followup for https://github.com/apache/spark/pull/24981 Seems we mistakenly didn't added `test_pandas_udf_cogrouped_map` into `modules.py`. So we don't have official test results against that PR. ``` ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf_grouped_agg ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf_grouped_map ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf_scalar ... Starting test(python3.6): pyspark.sql.tests.test_pandas_udf_window Finished test(python3.6): pyspark.sql.tests.test_pandas_udf (21s) ... Finished test(python3.6): pyspark.sql.tests.test_pandas_udf_grouped_map (49s) ... Finished test(python3.6): pyspark.sql.tests.test_pandas_udf_window (58s) ... Finished test(python3.6): pyspark.sql.tests.test_pandas_udf_scalar (82s) ... Finished test(python3.6): pyspark.sql.tests.test_pandas_udf_grouped_agg (105s) ... ``` If tests fail, we should revert that PR. ### Why are the changes needed? Relevant tests should be ran. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Jenkins tests. Closes #25890 from HyukjinKwon/SPARK-28840. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/sparktestsupport/modules.py | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index d4e6b6749010a..c7ea065b28ed8 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -372,6 +372,7 @@ def __hash__(self): "pyspark.sql.tests.test_functions", "pyspark.sql.tests.test_group", "pyspark.sql.tests.test_pandas_udf", + "pyspark.sql.tests.test_pandas_udf_cogrouped_map", "pyspark.sql.tests.test_pandas_udf_grouped_agg", "pyspark.sql.tests.test_pandas_udf_grouped_map", "pyspark.sql.tests.test_pandas_udf_scalar", From fe4bee8fd825f8b123569ded6e50e567f5cf9e2c Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sun, 22 Sep 2019 11:17:47 -0700 Subject: [PATCH 0154/1387] [SPARK-29162][SQL] Simplify NOT(IsNull(x)) and NOT(IsNotNull(x)) ### What changes were proposed in this pull request? Rewrite ``` NOT isnull(x) -> isnotnull(x) NOT isnotnull(x) -> isnull(x) ``` ### Why are the changes needed? Make LogicalPlan more readable and useful for query canonicalization. Make same condition equal when judge query canonicalization equal ### Does this PR introduce any user-facing change? NO ### How was this patch tested? Newly added UTs. Closes #25878 from AngersZhuuuu/SPARK-29162. Authored-by: angerszhu Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/optimizer/expressions.scala | 3 +++ .../sql/catalyst/optimizer/BooleanSimplificationSuite.scala | 5 +++++ 2 files changed, 8 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 39709529c00d3..0a6737ba42118 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -373,6 +373,9 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case Not(a And b) => Or(Not(a), Not(b)) case Not(Not(e)) => e + + case Not(IsNull(e)) => IsNotNull(e) + case Not(IsNotNull(e)) => IsNull(e) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index a0de5f6930958..a8b8417754b00 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -239,6 +239,11 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with checkCondition(!'f || 'e, testRelationWithData.where(!'f || 'e).analyze) } + test("simplify NOT(IsNull(x)) and NOT(IsNotNull(x))") { + checkCondition(Not(IsNotNull('b)), IsNull('b)) + checkCondition(Not(IsNull('b)), IsNotNull('b)) + } + protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { val correctAnswer = Project(Alias(e2, "out")() :: Nil, OneRowRelation()).analyze val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, OneRowRelation()).analyze) From 76bc9db7493a53fc89d9549ac366226701491bc5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 22 Sep 2019 13:53:21 -0700 Subject: [PATCH 0155/1387] [SPARK-29191][TESTS][SQL] Add tag ExtendedSQLTest for SQLQueryTestSuite ### What changes were proposed in this pull request? This PR aims to add tag `ExtendedSQLTest` for `SQLQueryTestSuite`. This doesn't affect our Jenkins test coverage. Instead, this tag gives us an ability to parallelize them by splitting this test suite and the other suites. ### Why are the changes needed? `SQLQueryTestSuite` takes 45 mins alone because it has many SQL scripts to run. time ### Does this PR introduce any user-facing change? No. ### How was this patch tested? ``` build/sbt "sql/test-only *.SQLQueryTestSuite" -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest ... [info] SQLQueryTestSuite: [info] ScalaTest [info] Run completed in 3 seconds, 147 milliseconds. [info] Total number of tests run: 0 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0 [info] No tests were executed. [info] Passed: Total 0, Failed 0, Errors 0, Passed 0 [success] Total time: 22 s, completed Sep 20, 2019 12:23:13 PM ``` Closes #25872 from dongjoon-hyun/SPARK-29191. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/tags/ExtendedSQLTest.java | 30 +++++++++++++++++++ .../apache/spark/sql/SQLQueryTestSuite.scala | 2 ++ 2 files changed, 32 insertions(+) create mode 100644 common/tags/src/test/java/org/apache/spark/tags/ExtendedSQLTest.java diff --git a/common/tags/src/test/java/org/apache/spark/tags/ExtendedSQLTest.java b/common/tags/src/test/java/org/apache/spark/tags/ExtendedSQLTest.java new file mode 100644 index 0000000000000..1c0fff1b4045d --- /dev/null +++ b/common/tags/src/test/java/org/apache/spark/tags/ExtendedSQLTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.tags; + +import org.scalatest.TagAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface ExtendedSQLTest { } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index f32575794f946..1a41dd95a5700 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCo import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType +import org.apache.spark.tags.ExtendedSQLTest /** * End-to-end test cases for SQL queries. @@ -104,6 +105,7 @@ import org.apache.spark.sql.types.StructType * Therefore, UDF test cases should have single input and output files but executed by three * different types of UDFs. See 'udf/udf-inner-join.sql' as an example. */ +@ExtendedSQLTest class SQLQueryTestSuite extends QueryTest with SharedSparkSession { import IntegratedUDFTestUtils._ From 51d350942865302e33480c83324704a54a24a494 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 22 Sep 2019 14:12:06 -0700 Subject: [PATCH 0156/1387] [SPARK-28599][SQL] Fix `Execution Time` and `Duration` column sorting for ThriftServerSessionPage ### What changes were proposed in this pull request? This PR add support sorting `Execution Time` and `Duration` columns for `ThriftServerSessionPage`. ### Why are the changes needed? Previously, it's not sorted correctly. ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? Manually do the following and test sorting on those columns in the Spark Thrift Server Session Page. ``` $ sbin/start-thriftserver.sh $ bin/beeline -u jdbc:hive2://localhost:10000 0: jdbc:hive2://localhost:10000> create table t(a int); +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (0.521 seconds) 0: jdbc:hive2://localhost:10000> select * from t; +----+--+ | a | +----+--+ +----+--+ No rows selected (0.772 seconds) 0: jdbc:hive2://localhost:10000> show databases; +---------------+--+ | databaseName | +---------------+--+ | default | +---------------+--+ 1 row selected (0.249 seconds) ``` **Sorted by `Execution Time` column**: ![image](https://user-images.githubusercontent.com/5399861/65387476-53038900-dd7a-11e9-885c-fca80287f550.png) **Sorted by `Duration` column**: ![image](https://user-images.githubusercontent.com/5399861/65387481-6e6e9400-dd7a-11e9-9318-f917247efaa8.png) Closes #25892 from wangyum/SPARK-28599. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../hive/thriftserver/ui/ThriftServerSessionPage.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 912b658eb36b9..b752634aef956 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -100,8 +100,12 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) {formatDate(info.startTimestamp)} {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} - {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} - {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} + + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + + + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} + {info.statement} {info.state} {errorMessageCell(detail)} From 7a2ea58e782cde4c9b7ec887c6eeb642659d8614 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 22 Sep 2019 14:47:42 -0700 Subject: [PATCH 0157/1387] [SPARK-29084][SQL][TESTS] Check method bytecode size in BenchmarkQueryTest ### What changes were proposed in this pull request? This pr proposes to check method bytecode size in `BenchmarkQueryTest`. This metric is critical for performance numbers. ### Why are the changes needed? For performance checks ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #25788 from maropu/CheckMethodSize. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/BenchmarkQueryTest.scala | 11 ++++++++--- .../scala/org/apache/spark/sql/TPCDSQuerySuite.scala | 10 ++++++++-- .../execution/LogicalPlanTagInSparkPlanSuite.scala | 5 +++-- 3 files changed, 19 insertions(+), 7 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala index 3fcb9892800b6..9bb741bee4e40 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import org.apache.spark.internal.config.Tests.IS_TESTING -import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator} +import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeFormatter, CodeGenerator} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.test.SharedSparkSession @@ -48,7 +48,7 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession { RuleExecutor.resetMetrics() } - protected def checkGeneratedCode(plan: SparkPlan): Unit = { + protected def checkGeneratedCode(plan: SparkPlan, checkMethodCodeSize: Boolean = true): Unit = { val codegenSubtrees = new collection.mutable.HashSet[WholeStageCodegenExec]() plan foreach { case s: WholeStageCodegenExec => @@ -57,7 +57,7 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession { } codegenSubtrees.toSeq.foreach { subtree => val code = subtree.doCodeGen()._2 - try { + val (_, ByteCodeStats(maxMethodCodeSize, _, _)) = try { // Just check the generated code can be properly compiled CodeGenerator.compile(code) } catch { @@ -72,6 +72,11 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession { """.stripMargin throw new Exception(msg, e) } + + assert(!checkMethodCodeSize || + maxMethodCodeSize <= CodeGenerator.DEFAULT_JVM_HUGE_METHOD_LIMIT, + s"too long generated codes found in the WholeStageCodegenExec subtree (id=${subtree.id}) " + + s"and JIT optimization might not work:\n${subtree.treeString}") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala index a668434a68aff..11e66e878c149 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala @@ -82,13 +82,19 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSSchema { "q3", "q7", "q10", "q19", "q27", "q34", "q42", "q43", "q46", "q52", "q53", "q55", "q59", "q63", "q65", "q68", "q73", "q79", "q89", "q98", "ss_max") + // List up the known queries having too large code in a generated function. + // A JIRA file for `modified-q3` is as follows; + // [SPARK-29128] Split predicate code in OR expressions + val blackListForMethodCodeSizeCheck = Set("modified-q3") + modifiedTPCDSQueries.foreach { name => val queryString = resourceToString(s"tpcds-modifiedQueries/$name.sql", classLoader = Thread.currentThread().getContextClassLoader) - test(s"modified-$name") { + val testName = s"modified-$name" + test(testName) { // check the plans can be properly generated val plan = sql(queryString).queryExecution.executedPlan - checkGeneratedCode(plan) + checkGeneratedCode(plan, !blackListForMethodCodeSizeCheck.contains(testName)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala index aa83b9b11dcfc..44af422b90837 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala @@ -32,8 +32,9 @@ import org.apache.spark.sql.execution.window.WindowExec class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite { - override protected def checkGeneratedCode(plan: SparkPlan): Unit = { - super.checkGeneratedCode(plan) + override protected def checkGeneratedCode( + plan: SparkPlan, checkMethodCodeSize: Boolean = true): Unit = { + super.checkGeneratedCode(plan, checkMethodCodeSize) checkLogicalPlanTag(plan) } From f725d472f51fb80c6ce1882ec283ff69bafb0de4 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 23 Sep 2019 16:16:52 +0800 Subject: [PATCH 0158/1387] [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files After the newly added shuffle block fetching protocol in #24565, we can keep this work by extending the FetchShuffleBlocks message. ### What changes were proposed in this pull request? In this patch, we achieve the indeterminate shuffle rerun by reusing the task attempt id(unique id within an application) in shuffle id, so that each shuffle write attempt has a different file name. For the indeterministic stage, when the stage resubmits, we'll clear all existing map status and rerun all partitions. All changes are summarized as follows: - Change the mapId to mapTaskAttemptId in shuffle related id. - Record the mapTaskAttemptId in MapStatus. - Still keep mapId in ShuffleFetcherIterator for fetch failed scenario. - Add the determinate flag in Stage and use it in DAGScheduler and the cleaning work for the intermediate stage. ### Why are the changes needed? This is a follow-up work for #22112's future improvment[1]: `Currently we can't rollback and rerun a shuffle map stage, and just fail.` Spark will rerun a finished shuffle write stage while meeting fetch failures, currently, the rerun shuffle map stage will only resubmit the task for missing partitions and reuse the output of other partitions. This logic is fine in most scenarios, but for indeterministic operations(like repartition), multiple shuffle write attempts may write different data, only rerun the missing partition will lead a correctness bug. So for the shuffle map stage of indeterministic operations, we need to support rolling back the shuffle map stage and re-generate the shuffle files. ### Does this PR introduce any user-facing change? Yes, after this PR, the indeterminate stage rerun will be accepted by rerunning the whole stage. The original behavior is aborting the stage and fail the job. ### How was this patch tested? - UT: Add UT for all changing code and newly added function. - Manual Test: Also providing a manual test to verify the effect. ``` import scala.sys.process._ import org.apache.spark.TaskContext val determinateStage0 = sc.parallelize(0 until 1000 * 1000 * 100, 10) val indeterminateStage1 = determinateStage0.repartition(200) val indeterminateStage2 = indeterminateStage1.repartition(200) val indeterminateStage3 = indeterminateStage2.repartition(100) val indeterminateStage4 = indeterminateStage3.repartition(300) val fetchFailIndeterminateStage4 = indeterminateStage4.map { x => if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId == 190 && TaskContext.get.stageAttemptNumber == 0) { throw new Exception("pkill -f -n java".!!) } x } val indeterminateStage5 = fetchFailIndeterminateStage4.repartition(200) val finalStage6 = indeterminateStage5.repartition(100).collect().distinct.length ``` It's a simple job with multi indeterminate stage, it will get a wrong answer while using old Spark version like 2.2/2.3, and will be killed after #22112. With this fix, the job can retry all indeterminate stage as below screenshot and get the right result. ![image](https://user-images.githubusercontent.com/4833765/63948434-3477de00-caab-11e9-9ed1-75abfe6d16bd.png) Closes #25620 from xuanyuanking/SPARK-25341-8.27. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../spark/network/protocol/Encoders.java | 23 ++ .../network/shuffle/ExternalBlockHandler.java | 60 +++-- .../shuffle/ExternalShuffleBlockResolver.java | 4 +- .../shuffle/OneForOneBlockFetcher.java | 27 +- .../shuffle/protocol/FetchShuffleBlocks.java | 10 +- .../shuffle/BlockTransferMessagesSuite.java | 2 +- .../shuffle/ExternalBlockHandlerSuite.java | 2 +- .../shuffle/OneForOneBlockFetcherSuite.java | 4 +- .../api/ShuffleExecutorComponents.java | 16 +- .../shuffle/api/ShuffleMapOutputWriter.java | 2 +- .../sort/BypassMergeSortShuffleWriter.java | 15 +- .../shuffle/sort/UnsafeShuffleWriter.java | 22 +- .../LocalDiskShuffleExecutorComponents.java | 6 +- .../io/LocalDiskShuffleMapOutputWriter.java | 4 +- .../LocalDiskSingleSpillMapOutputWriter.java | 4 +- .../scala/org/apache/spark/Dependency.scala | 2 +- .../org/apache/spark/MapOutputTracker.scala | 91 ++++--- .../org/apache/spark/TaskEndReason.scala | 7 +- .../spark/internal/config/package.scala | 8 + .../apache/spark/scheduler/DAGScheduler.scala | 39 ++- .../apache/spark/scheduler/MapStatus.scala | 49 +++- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../org/apache/spark/scheduler/Stage.scala | 6 +- .../spark/shuffle/BaseShuffleHandle.scala | 1 - .../shuffle/BlockStoreShuffleReader.scala | 3 +- .../spark/shuffle/FetchFailedException.scala | 14 +- .../shuffle/IndexShuffleBlockResolver.scala | 8 +- .../apache/spark/shuffle/ShuffleManager.scala | 3 +- .../spark/shuffle/ShuffleWriteProcessor.scala | 3 +- .../shuffle/sort/SortShuffleManager.scala | 32 ++- .../shuffle/sort/SortShuffleWriter.scala | 6 +- .../org/apache/spark/storage/BlockId.scala | 12 +- .../storage/ShuffleBlockFetcherIterator.scala | 102 +++++--- .../org/apache/spark/util/JsonProtocol.scala | 6 +- .../util/collection/ExternalSorter.scala | 2 +- .../sort/UnsafeShuffleWriterSuite.java | 16 +- .../spark/InternalAccumulatorSuite.scala | 1 + .../apache/spark/MapOutputTrackerSuite.scala | 46 ++-- .../scala/org/apache/spark/ShuffleSuite.scala | 2 +- .../org/apache/spark/SparkContextSuite.scala | 2 +- .../apache/spark/executor/ExecutorSuite.scala | 3 +- .../spark/scheduler/DAGSchedulerSuite.scala | 241 +++++++++++++----- .../spark/scheduler/MapStatusSuite.scala | 12 +- .../OutputCommitCoordinatorSuite.scala | 2 +- .../scheduler/SchedulerIntegrationSuite.scala | 2 +- .../spark/scheduler/TaskContextSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 4 +- .../serializer/KryoSerializerSuite.scala | 5 +- .../BlockStoreShuffleReaderSuite.scala | 7 +- .../BypassMergeSortShuffleWriterSuite.scala | 17 +- .../shuffle/sort/SortShuffleWriterSuite.scala | 2 +- ...LocalDiskShuffleMapOutputWriterSuite.scala | 6 +- .../ShuffleBlockFetcherIteratorSuite.scala | 54 ++-- .../org/apache/spark/ui/UISeleniumSuite.scala | 6 +- .../apache/spark/util/JsonProtocolSuite.scala | 9 +- project/MimaExcludes.scala | 27 +- 56 files changed, 672 insertions(+), 391 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 736059fdd1f57..490915f6de4b3 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -112,4 +112,27 @@ public static int[] decode(ByteBuf buf) { return ints; } } + + /** Long integer arrays are encoded with their length followed by long integers. */ + public static class LongArrays { + public static int encodedLength(long[] longs) { + return 4 + 8 * longs.length; + } + + public static void encode(ByteBuf buf, long[] longs) { + buf.writeInt(longs.length); + for (long i : longs) { + buf.writeLong(i); + } + } + + public static long[] decode(ByteBuf buf) { + int numLongs = buf.readInt(); + long[] longs = new long[numLongs]; + for (int i = 0; i < longs.length; i ++) { + longs[i] = buf.readLong(); + } + return longs; + } + } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 037e5cf7e5222..2d7a72315cf23 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -106,7 +106,7 @@ protected void handleMessage( numBlockIds += ids.length; } streamId = streamManager.registerStream(client.getClientId(), - new ManagedBufferIterator(msg, numBlockIds), client.getChannel()); + new ShuffleManagedBufferIterator(msg), client.getChannel()); } else { // For the compatibility with the old version, still keep the support for OpenBlocks. OpenBlocks msg = (OpenBlocks) msgObj; @@ -299,21 +299,6 @@ private int[] shuffleMapIdAndReduceIds(String[] blockIds, int shuffleId) { return mapIdAndReduceIds; } - ManagedBufferIterator(FetchShuffleBlocks msg, int numBlockIds) { - final int[] mapIdAndReduceIds = new int[2 * numBlockIds]; - int idx = 0; - for (int i = 0; i < msg.mapIds.length; i++) { - for (int reduceId : msg.reduceIds[i]) { - mapIdAndReduceIds[idx++] = msg.mapIds[i]; - mapIdAndReduceIds[idx++] = reduceId; - } - } - assert(idx == 2 * numBlockIds); - size = mapIdAndReduceIds.length; - blockDataForIndexFn = index -> blockManager.getBlockData(msg.appId, msg.execId, - msg.shuffleId, mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); - } - @Override public boolean hasNext() { return index < size; @@ -328,6 +313,49 @@ public ManagedBuffer next() { } } + private class ShuffleManagedBufferIterator implements Iterator { + + private int mapIdx = 0; + private int reduceIdx = 0; + + private final String appId; + private final String execId; + private final int shuffleId; + private final long[] mapIds; + private final int[][] reduceIds; + + ShuffleManagedBufferIterator(FetchShuffleBlocks msg) { + appId = msg.appId; + execId = msg.execId; + shuffleId = msg.shuffleId; + mapIds = msg.mapIds; + reduceIds = msg.reduceIds; + } + + @Override + public boolean hasNext() { + // mapIds.length must equal to reduceIds.length, and the passed in FetchShuffleBlocks + // must have non-empty mapIds and reduceIds, see the checking logic in + // OneForOneBlockFetcher. + assert(mapIds.length != 0 && mapIds.length == reduceIds.length); + return mapIdx < mapIds.length && reduceIdx < reduceIds[mapIdx].length; + } + + @Override + public ManagedBuffer next() { + final ManagedBuffer block = blockManager.getBlockData( + appId, execId, shuffleId, mapIds[mapIdx], reduceIds[mapIdx][reduceIdx]); + if (reduceIdx < reduceIds[mapIdx].length - 1) { + reduceIdx += 1; + } else { + reduceIdx = 0; + mapIdx += 1; + } + metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); + return block; + } + } + @Override public void channelActive(TransportClient client) { metrics.activeConnections.inc(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 50f16fc700f12..8b0d1e145a813 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -172,7 +172,7 @@ public ManagedBuffer getBlockData( String appId, String execId, int shuffleId, - int mapId, + long mapId, int reduceId) { ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); if (executor == null) { @@ -296,7 +296,7 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) { * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) { + ExecutorShuffleInfo executor, int shuffleId, long mapId, int reduceId) { File indexFile = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, "shuffle_" + shuffleId + "_" + mapId + "_0.index"); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index cc11e92067375..52854c86be3e6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -24,6 +24,8 @@ import java.util.HashMap; import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import org.apache.commons.lang3.tuple.ImmutableTriple; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -111,21 +113,21 @@ private boolean isShuffleBlocks(String[] blockIds) { */ private FetchShuffleBlocks createFetchShuffleBlocksMsg( String appId, String execId, String[] blockIds) { - int shuffleId = splitBlockId(blockIds[0])[0]; - HashMap> mapIdToReduceIds = new HashMap<>(); + int shuffleId = splitBlockId(blockIds[0]).left; + HashMap> mapIdToReduceIds = new HashMap<>(); for (String blockId : blockIds) { - int[] blockIdParts = splitBlockId(blockId); - if (blockIdParts[0] != shuffleId) { + ImmutableTriple blockIdParts = splitBlockId(blockId); + if (blockIdParts.left != shuffleId) { throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockId); } - int mapId = blockIdParts[1]; + long mapId = blockIdParts.middle; if (!mapIdToReduceIds.containsKey(mapId)) { mapIdToReduceIds.put(mapId, new ArrayList<>()); } - mapIdToReduceIds.get(mapId).add(blockIdParts[2]); + mapIdToReduceIds.get(mapId).add(blockIdParts.right); } - int[] mapIds = Ints.toArray(mapIdToReduceIds.keySet()); + long[] mapIds = Longs.toArray(mapIdToReduceIds.keySet()); int[][] reduceIdArr = new int[mapIds.length][]; for (int i = 0; i < mapIds.length; i++) { reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); @@ -134,17 +136,16 @@ private FetchShuffleBlocks createFetchShuffleBlocksMsg( } /** Split the shuffleBlockId and return shuffleId, mapId and reduceId. */ - private int[] splitBlockId(String blockId) { + private ImmutableTriple splitBlockId(String blockId) { String[] blockIdParts = blockId.split("_"); if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException( "Unexpected shuffle block id format: " + blockId); } - return new int[] { - Integer.parseInt(blockIdParts[1]), - Integer.parseInt(blockIdParts[2]), - Integer.parseInt(blockIdParts[3]) - }; + return new ImmutableTriple<>( + Integer.parseInt(blockIdParts[1]), + Long.parseLong(blockIdParts[2]), + Integer.parseInt(blockIdParts[3])); } /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java index 466eeb3e048a8..faa960d414bcc 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -34,14 +34,14 @@ public class FetchShuffleBlocks extends BlockTransferMessage { public final int shuffleId; // The length of mapIds must equal to reduceIds.size(), for the i-th mapId in mapIds, // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map id. - public final int[] mapIds; + public final long[] mapIds; public final int[][] reduceIds; public FetchShuffleBlocks( String appId, String execId, int shuffleId, - int[] mapIds, + long[] mapIds, int[][] reduceIds) { this.appId = appId; this.execId = execId; @@ -98,7 +98,7 @@ public int encodedLength() { return Encoders.Strings.encodedLength(appId) + Encoders.Strings.encodedLength(execId) + 4 /* encoded length of shuffleId */ - + Encoders.IntArrays.encodedLength(mapIds) + + Encoders.LongArrays.encodedLength(mapIds) + 4 /* encoded length of reduceIds.size() */ + encodedLengthOfReduceIds; } @@ -108,7 +108,7 @@ public void encode(ByteBuf buf) { Encoders.Strings.encode(buf, appId); Encoders.Strings.encode(buf, execId); buf.writeInt(shuffleId); - Encoders.IntArrays.encode(buf, mapIds); + Encoders.LongArrays.encode(buf, mapIds); buf.writeInt(reduceIds.length); for (int[] ids: reduceIds) { Encoders.IntArrays.encode(buf, ids); @@ -119,7 +119,7 @@ public static FetchShuffleBlocks decode(ByteBuf buf) { String appId = Encoders.Strings.decode(buf); String execId = Encoders.Strings.decode(buf); int shuffleId = buf.readInt(); - int[] mapIds = Encoders.IntArrays.decode(buf); + long[] mapIds = Encoders.LongArrays.decode(buf); int reduceIdsSize = buf.readInt(); int[][] reduceIds = new int[reduceIdsSize][]; for (int i = 0; i < reduceIdsSize; i++) { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java index 649c471dc1679..ba40f4a45ac8f 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java @@ -29,7 +29,7 @@ public class BlockTransferMessagesSuite { public void serializeOpenShuffleBlocks() { checkSerializeDeserialize(new OpenBlocks("app-1", "exec-2", new String[] { "b1", "b2" })); checkSerializeDeserialize(new FetchShuffleBlocks( - "app-1", "exec-2", 0, new int[] {0, 1}, + "app-1", "exec-2", 0, new long[] {0, 1}, new int[][] {{ 0, 1 }, { 0, 1, 2 }})); checkSerializeDeserialize(new RegisterExecutor("app-1", "exec-2", new ExecutorShuffleInfo( new String[] { "/local1", "/local2" }, 32, "MyShuffleManager"))); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index 9c623a70424b6..6a5d04b6f417b 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -101,7 +101,7 @@ public void testFetchShuffleBlocks() { when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(blockMarkers[1]); FetchShuffleBlocks fetchShuffleBlocks = new FetchShuffleBlocks( - "app0", "exec1", 0, new int[] { 0 }, new int[][] {{ 0, 1 }}); + "app0", "exec1", 0, new long[] { 0 }, new int[][] {{ 0, 1 }}); checkOpenBlocksReceive(fetchShuffleBlocks, blockMarkers); verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 66633cc7a3595..26a11672b8068 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -64,7 +64,7 @@ public void testFetchOne() { BlockFetchingListener listener = fetchBlocks( blocks, blockIds, - new FetchShuffleBlocks("app-id", "exec-id", 0, new int[] { 0 }, new int[][] {{ 0 }}), + new FetchShuffleBlocks("app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0 }}), conf); verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); @@ -100,7 +100,7 @@ public void testFetchThreeShuffleBlocks() { BlockFetchingListener listener = fetchBlocks( blocks, blockIds, - new FetchShuffleBlocks("app-id", "exec-id", 0, new int[] { 0 }, new int[][] {{ 0, 1, 2 }}), + new FetchShuffleBlocks("app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0, 1, 2 }}), conf); for (int i = 0; i < 3; i ++) { diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 804119cd06fa6..d30f3dad3c940 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -42,17 +42,13 @@ public interface ShuffleExecutorComponents { * partitioned bytes written by that map task. * * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId Within the shuffle, the identifier of the map task - * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task - * with the same (shuffleId, mapId) pair can be distinguished by the - * different values of mapTaskAttemptId. + * @param mapId An ID of the map task. The ID is unique within this Spark application. * @param numPartitions The number of partitions that will be written by the map task. Some of * these partitions may be empty. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId, + long mapId, int numPartitions) throws IOException; /** @@ -64,15 +60,11 @@ ShuffleMapOutputWriter createMapOutputWriter( * preserving an optimization in the local disk shuffle storage implementation. * * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId Within the shuffle, the identifier of the map task - * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task - * with the same (shuffleId, mapId) pair can be distinguished by the - * different values of mapTaskAttemptId. + * @param mapId An ID of the map task. The ID is unique within this Spark application. */ default Optional createSingleFileMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId) throws IOException { + long mapId) throws IOException { return Optional.empty(); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 7fac00b7fbc3f..21abe9a57cd25 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -39,7 +39,7 @@ public interface ShuffleMapOutputWriter { * for the same partition within any given map task. The partition identifier will be in the * range of precisely 0 (inclusive) to numPartitions (exclusive), where numPartitions was * provided upon the creation of this map output writer via - * {@link ShuffleExecutorComponents#createMapOutputWriter(int, int, long, int)}. + * {@link ShuffleExecutorComponents#createMapOutputWriter(int, long, int)}. *

    * Calls to this method will be invoked with monotonically increasing reducePartitionIds; each * call to this method will be called with a reducePartitionId that is strictly greater than diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index f75e932860f90..dc157eaa3b253 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -85,8 +85,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; - private final int mapId; - private final long mapTaskAttemptId; + private final long mapId; private final Serializer serializer; private final ShuffleExecutorComponents shuffleExecutorComponents; @@ -106,8 +105,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BypassMergeSortShuffleWriter( BlockManager blockManager, BypassMergeSortShuffleHandle handle, - int mapId, - long mapTaskAttemptId, + long mapId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, ShuffleExecutorComponents shuffleExecutorComponents) { @@ -117,7 +115,6 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); this.mapId = mapId; - this.mapTaskAttemptId = mapTaskAttemptId; this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); @@ -130,11 +127,12 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions); + .createMapOutputWriter(shuffleId, mapId, numPartitions); try { if (!records.hasNext()) { partitionLengths = mapOutputWriter.commitAllPartitions(); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapId); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -167,7 +165,8 @@ public void write(Iterator> records) throws IOException { } partitionLengths = writePartitionedData(mapOutputWriter); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapId); } catch (Exception e) { try { mapOutputWriter.abort(e); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 4d11abd36985e..d09282e61a9c7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -78,7 +78,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final ShuffleWriteMetricsReporter writeMetrics; private final ShuffleExecutorComponents shuffleExecutorComponents; private final int shuffleId; - private final int mapId; + private final long mapId; private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; @@ -109,7 +109,7 @@ public UnsafeShuffleWriter( BlockManager blockManager, TaskMemoryManager memoryManager, SerializedShuffleHandle handle, - int mapId, + long mapId, TaskContext taskContext, SparkConf sparkConf, ShuffleWriteMetricsReporter writeMetrics, @@ -228,7 +228,8 @@ void closeAndWriteOutput() throws IOException { } } } - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapId); } @VisibleForTesting @@ -264,16 +265,11 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { long[] partitionLengths; if (spills.length == 0) { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents - .createMapOutputWriter( - shuffleId, - mapId, - taskContext.taskAttemptId(), - partitioner.numPartitions()); + .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); return mapWriter.commitAllPartitions(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = - shuffleExecutorComponents.createSingleFileMapOutputWriter( - shuffleId, mapId, taskContext.taskAttemptId()); + shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapId); if (maybeSingleFileWriter.isPresent()) { // Here, we don't need to perform any metrics updates because the bytes written to this // output file would have already been counted as shuffle bytes written. @@ -298,11 +294,7 @@ private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOExcep CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents - .createMapOutputWriter( - shuffleId, - mapId, - taskContext.taskAttemptId(), - partitioner.numPartitions()); + .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); try { // There are multiple spills to merge, so none of these spill files' lengths were counted // towards our shuffle write count or shuffle write time. If we use the slow merge path, diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index 47aa2e39fe29b..a0c7d3c248d48 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -61,8 +61,7 @@ public void initializeExecutor(String appId, String execId) { @Override public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId, + long mapId, int numPartitions) { if (blockResolver == null) { throw new IllegalStateException( @@ -75,8 +74,7 @@ public ShuffleMapOutputWriter createMapOutputWriter( @Override public Optional createSingleFileMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId) { + long mapId) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 444cdc4270ecd..a6529fd76188a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -48,7 +48,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { LoggerFactory.getLogger(LocalDiskShuffleMapOutputWriter.class); private final int shuffleId; - private final int mapId; + private final long mapId; private final IndexShuffleBlockResolver blockResolver; private final long[] partitionLengths; private final int bufferSize; @@ -64,7 +64,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { public LocalDiskShuffleMapOutputWriter( int shuffleId, - int mapId, + long mapId, int numPartitions, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java index 6b0a797a61b52..c8b41992a8919 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java @@ -29,12 +29,12 @@ public class LocalDiskSingleSpillMapOutputWriter implements SingleSpillShuffleMapOutputWriter { private final int shuffleId; - private final int mapId; + private final long mapId; private final IndexShuffleBlockResolver blockResolver; public LocalDiskSingleSpillMapOutputWriter( int shuffleId, - int mapId, + long mapId, IndexShuffleBlockResolver blockResolver) { this.shuffleId = shuffleId; this.mapId = mapId; diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index fb051a8c0db8e..f0ac9acd90156 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -93,7 +93,7 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val shuffleId: Int = _rdd.context.newShuffleId() val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle( - shuffleId, _rdd.partitions.length, this) + shuffleId, this) _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d878fc527791a..53329f0a937bd 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -41,7 +41,7 @@ import org.apache.spark.util._ * Helper class used by the [[MapOutputTrackerMaster]] to perform bookkeeping for a single * ShuffleMapStage. * - * This class maintains a mapping from mapIds to `MapStatus`. It also maintains a cache of + * This class maintains a mapping from map index to `MapStatus`. It also maintains a cache of * serialized map statuses in order to speed up tasks' requests for map output statuses. * * All public methods of this class are thread-safe. @@ -88,12 +88,12 @@ private class ShuffleStatus(numPartitions: Int) { * Register a map output. If there is already a registered location for the map output then it * will be replaced by the new location. */ - def addMapOutput(mapId: Int, status: MapStatus): Unit = synchronized { - if (mapStatuses(mapId) == null) { + def addMapOutput(mapIndex: Int, status: MapStatus): Unit = synchronized { + if (mapStatuses(mapIndex) == null) { _numAvailableOutputs += 1 invalidateSerializedMapOutputStatusCache() } - mapStatuses(mapId) = status + mapStatuses(mapIndex) = status } /** @@ -101,10 +101,10 @@ private class ShuffleStatus(numPartitions: Int) { * This is a no-op if there is no registered map output or if the registered output is from a * different block manager. */ - def removeMapOutput(mapId: Int, bmAddress: BlockManagerId): Unit = synchronized { - if (mapStatuses(mapId) != null && mapStatuses(mapId).location == bmAddress) { + def removeMapOutput(mapIndex: Int, bmAddress: BlockManagerId): Unit = synchronized { + if (mapStatuses(mapIndex) != null && mapStatuses(mapIndex).location == bmAddress) { _numAvailableOutputs -= 1 - mapStatuses(mapId) = null + mapStatuses(mapIndex) = null invalidateSerializedMapOutputStatusCache() } } @@ -131,10 +131,10 @@ private class ShuffleStatus(numPartitions: Int) { * remove outputs which are served by an external shuffle server (if one exists). */ def removeOutputsByFilter(f: (BlockManagerId) => Boolean): Unit = synchronized { - for (mapId <- 0 until mapStatuses.length) { - if (mapStatuses(mapId) != null && f(mapStatuses(mapId).location)) { + for (mapIndex <- 0 until mapStatuses.length) { + if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) { _numAvailableOutputs -= 1 - mapStatuses(mapId) = null + mapStatuses(mapIndex) = null invalidateSerializedMapOutputStatusCache() } } @@ -282,8 +282,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // For testing def getMapSizesByExecutorId(shuffleId: Int, reduceId: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { - getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1) + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1, useOldFetchProtocol = false) } /** @@ -292,11 +292,15 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * endPartition is excluded from the range). * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block id, shuffle block size) tuples - * describing the shuffle blocks that are stored at that block manager. + * and the second item is a sequence of (shuffle block id, shuffle block size, map index) + * tuples describing the shuffle blocks that are stored at that block manager. */ - def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + useOldFetchProtocol: Boolean) + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] /** * Deletes map output status information for the specified shuffle stage. @@ -418,15 +422,15 @@ private[spark] class MapOutputTrackerMaster( } } - def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) { - shuffleStatuses(shuffleId).addMapOutput(mapId, status) + def registerMapOutput(shuffleId: Int, mapIndex: Int, status: MapStatus) { + shuffleStatuses(shuffleId).addMapOutput(mapIndex, status) } /** Unregister map output information of the given shuffle, mapper and block manager */ - def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) { + def unregisterMapOutput(shuffleId: Int, mapIndex: Int, bmAddress: BlockManagerId) { shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => - shuffleStatus.removeMapOutput(mapId, bmAddress) + shuffleStatus.removeMapOutput(mapIndex, bmAddress) incrementEpoch() case None => throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID") @@ -645,13 +649,18 @@ private[spark] class MapOutputTrackerMaster( // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. // This method is only called in local-mode. - def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + useOldFetchProtocol: Boolean) + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => shuffleStatus.withMapStatuses { statuses => - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses, useOldFetchProtocol) } case None => Iterator.empty @@ -685,12 +694,17 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr private val fetchingLock = new KeyLock[Int] // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. - override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + override def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + useOldFetchProtocol: Boolean) + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) try { - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses, useOldFetchProtocol) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -832,19 +846,21 @@ private[spark] object MapOutputTracker extends Logging { * @param shuffleId Identifier for the shuffle * @param startPartition Start of map output partition ID range (included in range) * @param endPartition End of map output partition ID range (excluded from range) - * @param statuses List of map statuses, indexed by map ID. + * @param statuses List of map statuses, indexed by map partition index. + * @param useOldFetchProtocol Whether to use the old shuffle fetch protocol. * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block ID, shuffle block size) tuples - * describing the shuffle blocks that are stored at that block manager. + * and the second item is a sequence of (shuffle block id, shuffle block size, map index) + * tuples describing the shuffle blocks that are stored at that block manager. */ def convertMapStatuses( shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + statuses: Array[MapStatus], + useOldFetchProtocol: Boolean): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { assert (statuses != null) - val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long)]] - for ((status, mapId) <- statuses.iterator.zipWithIndex) { + val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] + for ((status, mapIndex) <- statuses.iterator.zipWithIndex) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) @@ -853,8 +869,15 @@ private[spark] object MapOutputTracker extends Logging { for (part <- startPartition until endPartition) { val size = status.getSizeForBlock(part) if (size != 0) { - splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, mapId, part), size)) + if (useOldFetchProtocol) { + // While we use the old shuffle fetch protocol, we use mapIndex as mapId in the + // ShuffleBlockId. + splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += + ((ShuffleBlockId(shuffleId, mapIndex, part), size, mapIndex)) + } else { + splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += + ((ShuffleBlockId(shuffleId, status.mapTaskId, part), size, mapIndex)) + } } } } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 19f71a1dec296..b13028f868072 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -83,14 +83,15 @@ case object Resubmitted extends TaskFailedReason { case class FetchFailed( bmAddress: BlockManagerId, // Note that bmAddress can be null shuffleId: Int, - mapId: Int, + mapId: Long, + mapIndex: Int, reduceId: Int, message: String) extends TaskFailedReason { override def toErrorString: String = { val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString - s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId, " + - s"message=\n$message\n)" + s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapIndex=$mapIndex, " + + s"mapId=$mapId, reduceId=$reduceId, message=\n$message\n)" } /** 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 828b70dfe92e9..d142d22929728 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 @@ -1053,6 +1053,14 @@ package object config { .checkValue(v => v > 0, "The value should be a positive integer.") .createWithDefault(2000) + private[spark] val SHUFFLE_USE_OLD_FETCH_PROTOCOL = + ConfigBuilder("spark.shuffle.useOldFetchProtocol") + .doc("Whether to use the old protocol while doing the shuffle block fetching. " + + "It is only enabled while we need the compatibility in the scenario of new Spark " + + "version job fetching shuffle blocks from old version external shuffle service.") + .booleanConf + .createWithDefault(false) + private[spark] val MEMORY_MAP_LIMIT_FOR_TESTS = ConfigBuilder("spark.storage.memoryMapLimitForTests") .internal() diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f9d2df8185ec7..ee3e9cc03b966 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1103,7 +1103,16 @@ private[spark] class DAGScheduler( private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") - // First figure out the indexes of partition ids to compute. + // Before find missing partition, do the intermediate state clean work first. + // The operation here can make sure for the partially completed intermediate stage, + // `findMissingPartitions()` returns all partitions every time. + stage match { + case sms: ShuffleMapStage if stage.isIndeterminate && !sms.isAvailable => + mapOutputTracker.unregisterAllMapOutput(sms.shuffleDep.shuffleId) + case _ => + } + + // Figure out the indexes of partition ids to compute. val partitionsToCompute: Seq[Int] = stage.findMissingPartitions() // Use the scheduling pool, job group, description, etc. from an ActiveJob associated @@ -1498,7 +1507,7 @@ private[spark] class DAGScheduler( } } - case FetchFailed(bmAddress, shuffleId, mapId, _, failureMessage) => + case FetchFailed(bmAddress, shuffleId, _, mapIndex, _, failureMessage) => val failedStage = stageIdToStage(task.stageId) val mapStage = shuffleIdToMapStage(shuffleId) @@ -1529,9 +1538,9 @@ private[spark] class DAGScheduler( // Mark all the map as broken in the map stage, to ensure retry all the tasks on // resubmitted stage attempt. mapOutputTracker.unregisterAllMapOutput(shuffleId) - } else if (mapId != -1) { + } else if (mapIndex != -1) { // Mark the map whose fetch failed as broken in the map stage - mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + mapOutputTracker.unregisterMapOutput(shuffleId, mapIndex, bmAddress) } if (failedStage.rdd.isBarrier()) { @@ -1573,7 +1582,7 @@ private[spark] class DAGScheduler( // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is // guaranteed to be determinate, so the input data of the reducers will not change // even if the map tasks are re-tried. - if (mapStage.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) { + if (mapStage.isIndeterminate) { // It's a little tricky to find all the succeeding stages of `mapStage`, because // each stage only know its parents not children. Here we traverse the stages from // the leaf nodes (the result stages of active jobs), and rollback all the stages @@ -1601,15 +1610,22 @@ private[spark] class DAGScheduler( activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) + // The stages will be rolled back after checking + val rollingBackStages = HashSet[Stage](mapStage) stagesToRollback.foreach { case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { - // TODO: support to rollback shuffle files. - // Currently the shuffle writing is "first write wins", so we can't re-run a - // shuffle map stage and overwrite existing shuffle files. We have to finish - // SPARK-8029 first. - abortStage(mapStage, generateErrorMessage(mapStage), None) + if (sc.getConf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { + val reason = "A shuffle map stage with indeterminate output was failed " + + "and retried. However, Spark can only do this while using the new " + + "shuffle block fetching protocol. Please check the config " + + "'spark.shuffle.useOldFetchProtocol', see more detail in " + + "SPARK-27665 and SPARK-25341." + abortStage(mapStage, reason, None) + } else { + rollingBackStages += mapStage + } } case resultStage: ResultStage if resultStage.activeJob.isDefined => @@ -1621,6 +1637,9 @@ private[spark] class DAGScheduler( case _ => } + logInfo(s"The shuffle map stage $mapStage with indeterminate output was failed, " + + s"we will roll back and rerun below stages which include itself and all its " + + s"indeterminate child stages: $rollingBackStages") } // We expect one executor failure to trigger many FetchFailures in rapid succession, diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 64f0a060a247c..c9d37c985d211 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -43,6 +43,11 @@ private[spark] sealed trait MapStatus { * necessary for correctness, since block fetchers are allowed to skip zero-size blocks. */ def getSizeForBlock(reduceId: Int): Long + + /** + * The unique ID of this shuffle map task, we use taskContext.taskAttemptId to fill this. + */ + def mapTaskId: Long } @@ -56,11 +61,14 @@ private[spark] object MapStatus { .map(_.conf.get(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS)) .getOrElse(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS.defaultValue.get) - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { + def apply( + loc: BlockManagerId, + uncompressedSizes: Array[Long], + mapTaskId: Long): MapStatus = { if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { - HighlyCompressedMapStatus(loc, uncompressedSizes) + HighlyCompressedMapStatus(loc, uncompressedSizes, mapTaskId) } else { - new CompressedMapStatus(loc, uncompressedSizes) + new CompressedMapStatus(loc, uncompressedSizes, mapTaskId) } } @@ -100,16 +108,19 @@ private[spark] object MapStatus { * * @param loc location where the task is being executed. * @param compressedSizes size of the blocks, indexed by reduce partition id. + * @param _mapTaskId unique task id for the task */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, - private[this] var compressedSizes: Array[Byte]) + private[this] var compressedSizes: Array[Byte], + private[this] var _mapTaskId: Long) extends MapStatus with Externalizable { - protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only + // For deserialization only + protected def this() = this(null, null.asInstanceOf[Array[Byte]], -1) - def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { - this(loc, uncompressedSizes.map(MapStatus.compressSize)) + def this(loc: BlockManagerId, uncompressedSizes: Array[Long], mapTaskId: Long) { + this(loc, uncompressedSizes.map(MapStatus.compressSize), mapTaskId) } override def location: BlockManagerId = loc @@ -118,10 +129,13 @@ private[spark] class CompressedMapStatus( MapStatus.decompressSize(compressedSizes(reduceId)) } + override def mapTaskId: Long = _mapTaskId + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) + out.writeLong(_mapTaskId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -129,6 +143,7 @@ private[spark] class CompressedMapStatus( val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) + _mapTaskId = in.readLong() } } @@ -142,20 +157,23 @@ private[spark] class CompressedMapStatus( * @param emptyBlocks a bitmap tracking which blocks are empty * @param avgSize average size of the non-empty and non-huge blocks * @param hugeBlockSizes sizes of huge blocks by their reduceId. + * @param _mapTaskId unique task id for the task */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, private[this] var numNonEmptyBlocks: Int, private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, - private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte]) + private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte], + private[this] var _mapTaskId: Long) extends MapStatus with Externalizable { // loc could be null when the default constructor is called during deserialization - require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 || numNonEmptyBlocks == 0, + require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 + || numNonEmptyBlocks == 0 || _mapTaskId > 0, "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, -1, null, -1, null) // For deserialization only + protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only override def location: BlockManagerId = loc @@ -171,6 +189,8 @@ private[spark] class HighlyCompressedMapStatus private ( } } + override def mapTaskId: Long = _mapTaskId + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) emptyBlocks.writeExternal(out) @@ -180,6 +200,7 @@ private[spark] class HighlyCompressedMapStatus private ( out.writeInt(kv._1) out.writeByte(kv._2) } + out.writeLong(_mapTaskId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -195,11 +216,15 @@ private[spark] class HighlyCompressedMapStatus private ( hugeBlockSizesImpl(block) = size } hugeBlockSizes = hugeBlockSizesImpl + _mapTaskId = in.readLong() } } private[spark] object HighlyCompressedMapStatus { - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { + def apply( + loc: BlockManagerId, + uncompressedSizes: Array[Long], + mapTaskId: Long): HighlyCompressedMapStatus = { // We must keep track of which blocks are empty so that we don't report a zero-sized // block as being non-empty (or vice-versa) when using the average block size. var i = 0 @@ -240,6 +265,6 @@ private[spark] object HighlyCompressedMapStatus { emptyBlocks.trim() emptyBlocks.runOptimize() new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizes) + hugeBlockSizes, mapTaskId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 710f5eb211dde..06e5d8ab0302a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -91,7 +91,7 @@ private[spark] class ShuffleMapTask( val rdd = rddAndDep._1 val dep = rddAndDep._2 - dep.shuffleWriterProcessor.write(rdd, dep, partitionId, context, partition) + dep.shuffleWriterProcessor.write(rdd, dep, context, partition) } override def preferredLocations: Seq[TaskLocation] = preferredLocs diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 26cca334d3bd5..a9f72eae71368 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.HashSet import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.util.CallSite /** @@ -116,4 +116,8 @@ private[scheduler] abstract class Stage( /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ def findMissingPartitions(): Seq[Int] + + def isIndeterminate: Boolean = { + rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE + } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala index 04e4cf88d7063..6fe183c078089 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala @@ -24,6 +24,5 @@ import org.apache.spark.ShuffleDependency */ private[spark] class BaseShuffleHandle[K, V, C]( shuffleId: Int, - val numMaps: Int, val dependency: ShuffleDependency[K, V, C]) extends ShuffleHandle(shuffleId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 4329824b1b627..8a0e84d901c2f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -47,7 +47,8 @@ private[spark] class BlockStoreShuffleReader[K, C]( context, blockManager.blockStoreClient, blockManager, - mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), + mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition, + SparkEnv.get.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)), serializerManager.wrapStream, // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index 265a8acfa8d61..6509a04dc4893 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -35,7 +35,8 @@ import org.apache.spark.util.Utils private[spark] class FetchFailedException( bmAddress: BlockManagerId, shuffleId: Int, - mapId: Int, + mapId: Long, + mapIndex: Int, reduceId: Int, message: String, cause: Throwable = null) @@ -44,10 +45,11 @@ private[spark] class FetchFailedException( def this( bmAddress: BlockManagerId, shuffleId: Int, - mapId: Int, + mapTaskId: Long, + mapIndex: Int, reduceId: Int, cause: Throwable) { - this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) + this(bmAddress, shuffleId, mapTaskId, mapIndex, reduceId, cause.getMessage, cause) } // SPARK-19276. We set the fetch failure in the task context, so that even if there is user-code @@ -56,8 +58,8 @@ private[spark] class FetchFailedException( // because the TaskContext is not defined in some test cases. Option(TaskContext.get()).map(_.setFetchFailed(this)) - def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, - Utils.exceptionString(this)) + def toTaskFailedReason: TaskFailedReason = FetchFailed( + bmAddress, shuffleId, mapId, mapIndex, reduceId, Utils.exceptionString(this)) } /** @@ -67,4 +69,4 @@ private[spark] class MetadataFetchFailedException( shuffleId: Int, reduceId: Int, message: String) - extends FetchFailedException(null, shuffleId, -1, reduceId, message) + extends FetchFailedException(null, shuffleId, -1L, -1, reduceId, message) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index d3f1c7ec1bbee..332164a7be3e7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -51,18 +51,18 @@ private[spark] class IndexShuffleBlockResolver( private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - def getDataFile(shuffleId: Int, mapId: Int): File = { + def getDataFile(shuffleId: Int, mapId: Long): File = { blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } - private def getIndexFile(shuffleId: Int, mapId: Int): File = { + private def getIndexFile(shuffleId: Int, mapId: Long): File = { blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } /** * Remove data file and index file that contain the output data from one map. */ - def removeDataByMap(shuffleId: Int, mapId: Int): Unit = { + def removeDataByMap(shuffleId: Int, mapId: Long): Unit = { var file = getDataFile(shuffleId, mapId) if (file.exists()) { if (!file.delete()) { @@ -135,7 +135,7 @@ private[spark] class IndexShuffleBlockResolver( */ def writeIndexFileAndCommit( shuffleId: Int, - mapId: Int, + mapId: Long, lengths: Array[Long], dataTmp: File): Unit = { val indexFile = getIndexFile(shuffleId, mapId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 18a743fbfa6fc..a717ef242ea7c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -34,13 +34,12 @@ private[spark] trait ShuffleManager { */ def registerShuffle[K, V, C]( shuffleId: Int, - numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle /** Get a writer for a given partition. Called on executors by map tasks. */ def getWriter[K, V]( handle: ShuffleHandle, - mapId: Int, + mapId: Long, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 5b0c7e9f2b0b4..f222200a7816c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -44,7 +44,6 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { def write( rdd: RDD[_], dep: ShuffleDependency[_, _, _], - partitionId: Int, context: TaskContext, partition: Partition): MapStatus = { var writer: ShuffleWriter[Any, Any] = null @@ -52,7 +51,7 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any]( dep.shuffleHandle, - partitionId, + context.taskAttemptId(), context, createMetricsReporter(context)) writer.write( diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index cbdc2c886dd9f..d96bcb3d073df 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -24,6 +24,7 @@ import org.apache.spark.internal.{config, Logging} import org.apache.spark.shuffle._ import org.apache.spark.shuffle.api.{ShuffleDataIO, ShuffleExecutorComponents} import org.apache.spark.util.Utils +import org.apache.spark.util.collection.OpenHashSet /** * In sort-based shuffle, incoming records are sorted according to their target partition ids, then @@ -79,9 +80,9 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager } /** - * A mapping from shuffle ids to the number of mappers producing output for those shuffles. + * A mapping from shuffle ids to the task ids of mappers producing output for those shuffles. */ - private[this] val numMapsForShuffle = new ConcurrentHashMap[Int, Int]() + private[this] val taskIdMapsForShuffle = new ConcurrentHashMap[Int, OpenHashSet[Long]]() private lazy val shuffleExecutorComponents = loadShuffleExecutorComponents(conf) @@ -92,7 +93,6 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager */ override def registerShuffle[K, V, C]( shuffleId: Int, - numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { if (SortShuffleWriter.shouldBypassMergeSort(conf, dependency)) { // If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't @@ -101,14 +101,14 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager // together the spilled files, which would happen with the normal code path. The downside is // having multiple files open at a time and thus more memory allocated to buffers. new BypassMergeSortShuffleHandle[K, V]( - shuffleId, numMaps, dependency.asInstanceOf[ShuffleDependency[K, V, V]]) + shuffleId, dependency.asInstanceOf[ShuffleDependency[K, V, V]]) } else if (SortShuffleManager.canUseSerializedShuffle(dependency)) { // Otherwise, try to buffer map outputs in a serialized form, since this is more efficient: new SerializedShuffleHandle[K, V]( - shuffleId, numMaps, dependency.asInstanceOf[ShuffleDependency[K, V, V]]) + shuffleId, dependency.asInstanceOf[ShuffleDependency[K, V, V]]) } else { // Otherwise, buffer map outputs in a deserialized form: - new BaseShuffleHandle(shuffleId, numMaps, dependency) + new BaseShuffleHandle(shuffleId, dependency) } } @@ -130,11 +130,12 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Get a writer for a given partition. Called on executors by map tasks. */ override def getWriter[K, V]( handle: ShuffleHandle, - mapId: Int, + mapId: Long, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - numMapsForShuffle.putIfAbsent( - handle.shuffleId, handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps) + val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent( + handle.shuffleId, _ => new OpenHashSet[Long](16)) + mapTaskIds.synchronized { mapTaskIds.add(context.taskAttemptId()) } val env = SparkEnv.get handle match { case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => @@ -152,7 +153,6 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager env.blockManager, bypassMergeSortHandle, mapId, - context.taskAttemptId(), env.conf, metrics, shuffleExecutorComponents) @@ -164,9 +164,9 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - Option(numMapsForShuffle.remove(shuffleId)).foreach { numMaps => - (0 until numMaps).foreach { mapId => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId) + Option(taskIdMapsForShuffle.remove(shuffleId)).foreach { mapTaskIds => + mapTaskIds.iterator.foreach { mapTaskId => + shuffleBlockResolver.removeDataByMap(shuffleId, mapTaskId) } } true @@ -231,9 +231,8 @@ private[spark] object SortShuffleManager extends Logging { */ private[spark] class SerializedShuffleHandle[K, V]( shuffleId: Int, - numMaps: Int, dependency: ShuffleDependency[K, V, V]) - extends BaseShuffleHandle(shuffleId, numMaps, dependency) { + extends BaseShuffleHandle(shuffleId, dependency) { } /** @@ -242,7 +241,6 @@ private[spark] class SerializedShuffleHandle[K, V]( */ private[spark] class BypassMergeSortShuffleHandle[K, V]( shuffleId: Int, - numMaps: Int, dependency: ShuffleDependency[K, V, V]) - extends BaseShuffleHandle(shuffleId, numMaps, dependency) { + extends BaseShuffleHandle(shuffleId, dependency) { } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index a781b16252432..a391bdf2db44e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( shuffleBlockResolver: IndexShuffleBlockResolver, handle: BaseShuffleHandle[K, V, C], - mapId: Int, + mapId: Long, context: TaskContext, shuffleExecutorComponents: ShuffleExecutorComponents) extends ShuffleWriter[K, V] with Logging { @@ -65,10 +65,10 @@ private[spark] class SortShuffleWriter[K, V, C]( // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( - dep.shuffleId, mapId, context.taskAttemptId(), dep.partitioner.numPartitions) + dep.shuffleId, mapId, dep.partitioner.numPartitions) sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) val partitionLengths = mapOutputWriter.commitAllPartitions() - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 7ac2c71c18eb3..9c5b7f64e7abe 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -52,17 +52,17 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { // Format of the shuffle block ids (including data and index) should be kept in sync with // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData(). @DeveloperApi -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +case class ShuffleBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } @DeveloperApi -case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +case class ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" } @DeveloperApi -case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } @@ -117,11 +117,11 @@ object BlockId { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => - ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + ShuffleBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case SHUFFLE_DATA(shuffleId, mapId, reduceId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + ShuffleDataBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 8a6c199423506..5fce358fae37f 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -49,9 +49,10 @@ import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} * @param shuffleClient [[BlockStoreClient]] for fetching remote blocks * @param blockManager [[BlockManager]] for reading local blocks * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. - * For each block we also require the size (in bytes as a long field) in - * order to throttle the memory usage. Note that zero-sized blocks are - * already excluded, which happened in + * For each block we also require two info: 1. the size (in bytes as a long + * field) in order to throttle the memory usage; 2. the mapIndex for this + * block, which indicate the index in the map stage. + * Note that zero-sized blocks are already excluded, which happened in * [[org.apache.spark.MapOutputTracker.convertMapStatuses]]. * @param streamWrapper A function to wrap the returned input stream. * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. @@ -67,7 +68,7 @@ final class ShuffleBlockFetcherIterator( context: TaskContext, shuffleClient: BlockStoreClient, blockManager: BlockManager, - blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])], + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], streamWrapper: (BlockId, InputStream) => InputStream, maxBytesInFlight: Long, maxReqsInFlight: Int, @@ -97,7 +98,7 @@ final class ShuffleBlockFetcherIterator( private[this] val startTimeNs = System.nanoTime() /** Local blocks to fetch, excluding zero-sized blocks. */ - private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[BlockId]() + private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]() /** Remote blocks to fetch, excluding zero-sized blocks. */ private[this] val remoteBlocks = new HashSet[BlockId]() @@ -199,7 +200,7 @@ final class ShuffleBlockFetcherIterator( while (iter.hasNext) { val result = iter.next() result match { - case SuccessFetchResult(_, address, _, buf, _) => + case SuccessFetchResult(_, _, address, _, buf, _) => if (address != blockManager.blockManagerId) { shuffleMetrics.incRemoteBytesRead(buf.size) if (buf.isInstanceOf[FileSegmentManagedBuffer]) { @@ -224,10 +225,12 @@ final class ShuffleBlockFetcherIterator( bytesInFlight += req.size reqsInFlight += 1 - // so we can look up the size of each blockID - val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap - val remainingBlocks = new HashSet[String]() ++= sizeMap.keys - val blockIds = req.blocks.map(_._1.toString) + // so we can look up the block info of each blockID + val infoMap = req.blocks.map { + case FetchBlockInfo(blockId, size, mapIndex) => (blockId.toString, (size, mapIndex)) + }.toMap + val remainingBlocks = new HashSet[String]() ++= infoMap.keys + val blockIds = req.blocks.map(_.blockId.toString) val address = req.address val blockFetchingListener = new BlockFetchingListener { @@ -240,8 +243,8 @@ final class ShuffleBlockFetcherIterator( // This needs to be released after use. buf.retain() remainingBlocks -= blockId - results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf, - remainingBlocks.isEmpty)) + results.put(new SuccessFetchResult(BlockId(blockId), infoMap(blockId)._2, + address, infoMap(blockId)._1, buf, remainingBlocks.isEmpty)) logDebug("remainingBlocks: " + remainingBlocks) } } @@ -250,7 +253,7 @@ final class ShuffleBlockFetcherIterator( override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) - results.put(new FailureFetchResult(BlockId(blockId), address, e)) + results.put(new FailureFetchResult(BlockId(blockId), infoMap(blockId)._2, address, e)) } } @@ -283,28 +286,28 @@ final class ShuffleBlockFetcherIterator( for ((address, blockInfos) <- blocksByAddress) { if (address.executorId == blockManager.blockManagerId.executorId) { blockInfos.find(_._2 <= 0) match { - case Some((blockId, size)) if size < 0 => + case Some((blockId, size, _)) if size < 0 => throw new BlockException(blockId, "Negative block size " + size) - case Some((blockId, size)) if size == 0 => + case Some((blockId, size, _)) if size == 0 => throw new BlockException(blockId, "Zero-sized blocks should be excluded.") case None => // do nothing. } - localBlocks ++= blockInfos.map(_._1) + localBlocks ++= blockInfos.map(info => (info._1, info._3)) localBlockBytes += blockInfos.map(_._2).sum numBlocksToFetch += localBlocks.size } else { val iterator = blockInfos.iterator var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(BlockId, Long)] + var curBlocks = new ArrayBuffer[FetchBlockInfo] while (iterator.hasNext) { - val (blockId, size) = iterator.next() + val (blockId, size, mapIndex) = iterator.next() remoteBlockBytes += size if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } else if (size == 0) { throw new BlockException(blockId, "Zero-sized blocks should be excluded.") } else { - curBlocks += ((blockId, size)) + curBlocks += FetchBlockInfo(blockId, size, mapIndex) remoteBlocks += blockId numBlocksToFetch += 1 curRequestSize += size @@ -315,7 +318,7 @@ final class ShuffleBlockFetcherIterator( remoteRequests += new FetchRequest(address, curBlocks) logDebug(s"Creating fetch request of $curRequestSize at $address " + s"with ${curBlocks.size} blocks") - curBlocks = new ArrayBuffer[(BlockId, Long)] + curBlocks = new ArrayBuffer[FetchBlockInfo] curRequestSize = 0 } } @@ -341,13 +344,13 @@ final class ShuffleBlockFetcherIterator( logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}") val iter = localBlocks.iterator while (iter.hasNext) { - val blockId = iter.next() + val (blockId, mapIndex) = iter.next() try { val buf = blockManager.getBlockData(blockId) shuffleMetrics.incLocalBlocksFetched(1) shuffleMetrics.incLocalBytesRead(buf.size) buf.retain() - results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId, + results.put(new SuccessFetchResult(blockId, mapIndex, blockManager.blockManagerId, buf.size(), buf, false)) } catch { // If we see an exception, stop immediately. @@ -360,7 +363,7 @@ final class ShuffleBlockFetcherIterator( logError("Error occurred while fetching local blocks, " + ce.getMessage) case ex: Exception => logError("Error occurred while fetching local blocks", ex) } - results.put(new FailureFetchResult(blockId, blockManager.blockManagerId, e)) + results.put(new FailureFetchResult(blockId, mapIndex, blockManager.blockManagerId, e)) return } } @@ -420,7 +423,7 @@ final class ShuffleBlockFetcherIterator( shuffleMetrics.incFetchWaitTime(fetchWaitTime) result match { - case r @ SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) => + case r @ SuccessFetchResult(blockId, mapIndex, address, size, buf, isNetworkReqDone) => if (address != blockManager.blockManagerId) { numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 shuffleMetrics.incRemoteBytesRead(buf.size) @@ -429,7 +432,7 @@ final class ShuffleBlockFetcherIterator( } shuffleMetrics.incRemoteBlocksFetched(1) } - if (!localBlocks.contains(blockId)) { + if (!localBlocks.contains((blockId, mapIndex))) { bytesInFlight -= size } if (isNetworkReqDone) { @@ -453,7 +456,7 @@ final class ShuffleBlockFetcherIterator( // since the last call. val msg = s"Received a zero-size buffer for block $blockId from $address " + s"(expectedApproxSize = $size, isNetworkReqDone=$isNetworkReqDone)" - throwFetchFailedException(blockId, address, new IOException(msg)) + throwFetchFailedException(blockId, mapIndex, address, new IOException(msg)) } val in = try { @@ -469,7 +472,7 @@ final class ShuffleBlockFetcherIterator( case e: IOException => logError("Failed to create input stream from local block", e) } buf.release() - throwFetchFailedException(blockId, address, e) + throwFetchFailedException(blockId, mapIndex, address, e) } try { input = streamWrapper(blockId, in) @@ -487,11 +490,12 @@ final class ShuffleBlockFetcherIterator( buf.release() if (buf.isInstanceOf[FileSegmentManagedBuffer] || corruptedBlocks.contains(blockId)) { - throwFetchFailedException(blockId, address, e) + throwFetchFailedException(blockId, mapIndex, address, e) } else { logWarning(s"got an corrupted block $blockId from $address, fetch again", e) corruptedBlocks += blockId - fetchRequests += FetchRequest(address, Array((blockId, size))) + fetchRequests += FetchRequest( + address, Array(FetchBlockInfo(blockId, size, mapIndex))) result = null } } finally { @@ -503,8 +507,8 @@ final class ShuffleBlockFetcherIterator( } } - case FailureFetchResult(blockId, address, e) => - throwFetchFailedException(blockId, address, e) + case FailureFetchResult(blockId, mapIndex, address, e) => + throwFetchFailedException(blockId, mapIndex, address, e) } // Send fetch requests up to maxBytesInFlight @@ -517,6 +521,7 @@ final class ShuffleBlockFetcherIterator( input, this, currentResult.blockId, + currentResult.mapIndex, currentResult.address, detectCorrupt && streamCompressedOrEncrypted)) } @@ -583,11 +588,12 @@ final class ShuffleBlockFetcherIterator( private[storage] def throwFetchFailedException( blockId: BlockId, + mapIndex: Int, address: BlockManagerId, e: Throwable) = { blockId match { case ShuffleBlockId(shufId, mapId, reduceId) => - throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, e) + throw new FetchFailedException(address, shufId, mapId, mapIndex, reduceId, e) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block", e) @@ -604,6 +610,7 @@ private class BufferReleasingInputStream( private[storage] val delegate: InputStream, private val iterator: ShuffleBlockFetcherIterator, private val blockId: BlockId, + private val mapIndex: Int, private val address: BlockManagerId, private val detectCorruption: Boolean) extends InputStream { @@ -615,7 +622,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, address, e) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) } } @@ -637,7 +644,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, address, e) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) } } @@ -649,7 +656,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, address, e) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) } } @@ -659,7 +666,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, address, e) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) } } @@ -690,14 +697,25 @@ private class ShuffleFetchCompletionListener(var data: ShuffleBlockFetcherIterat private[storage] object ShuffleBlockFetcherIterator { + /** + * The block information to fetch used in FetchRequest. + * @param blockId block id + * @param size estimated size of the block. Note that this is NOT the exact bytes. + * Size of remote block is used to calculate bytesInFlight. + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage. + */ + private[storage] case class FetchBlockInfo( + blockId: BlockId, + size: Long, + mapIndex: Int) + /** * A request to fetch blocks from a remote BlockManager. * @param address remote BlockManager to fetch from. - * @param blocks Sequence of tuple, where the first element is the block id, - * and the second element is the estimated size, used to calculate bytesInFlight. + * @param blocks Sequence of the information for blocks to fetch from the same address. */ - case class FetchRequest(address: BlockManagerId, blocks: Seq[(BlockId, Long)]) { - val size = blocks.map(_._2).sum + case class FetchRequest(address: BlockManagerId, blocks: Seq[FetchBlockInfo]) { + val size = blocks.map(_.size).sum } /** @@ -711,6 +729,7 @@ object ShuffleBlockFetcherIterator { /** * Result of a fetch from a remote block successfully. * @param blockId block id + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage. * @param address BlockManager that the block was fetched from. * @param size estimated size of the block. Note that this is NOT the exact bytes. * Size of remote block is used to calculate bytesInFlight. @@ -719,6 +738,7 @@ object ShuffleBlockFetcherIterator { */ private[storage] case class SuccessFetchResult( blockId: BlockId, + mapIndex: Int, address: BlockManagerId, size: Long, buf: ManagedBuffer, @@ -730,11 +750,13 @@ object ShuffleBlockFetcherIterator { /** * Result of a fetch from a remote block unsuccessfully. * @param blockId block id + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage * @param address BlockManager that the block was attempted to be fetched from * @param e the failure exception */ private[storage] case class FailureFetchResult( blockId: BlockId, + mapIndex: Int, address: BlockManagerId, e: Throwable) extends FetchResult diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 73ef80980e73f..353590d201bc5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -420,6 +420,7 @@ private[spark] object JsonProtocol { ("Block Manager Address" -> blockManagerAddress) ~ ("Shuffle ID" -> fetchFailed.shuffleId) ~ ("Map ID" -> fetchFailed.mapId) ~ + ("Map Index" -> fetchFailed.mapIndex) ~ ("Reduce ID" -> fetchFailed.reduceId) ~ ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => @@ -974,10 +975,11 @@ private[spark] object JsonProtocol { case `fetchFailed` => val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") val shuffleId = (json \ "Shuffle ID").extract[Int] - val mapId = (json \ "Map ID").extract[Int] + val mapId = (json \ "Map ID").extract[Long] + val mapIndex = (json \ "Map Index").extract[Int] val reduceId = (json \ "Reduce ID").extract[Int] val message = jsonOption(json \ "Message").map(_.extract[String]) - new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId, + new FetchFailed(blockManagerAddress, shuffleId, mapId, mapIndex, reduceId, message.getOrElse("Unknown reason")) case `exceptionFailure` => val className = (json \ "Class Name").extract[String] diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 7a822e137e556..6fecfbaca8416 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -727,7 +727,7 @@ private[spark] class ExternalSorter[K, V, C]( */ def writePartitionedMapOutput( shuffleId: Int, - mapId: Int, + mapId: Long, mapOutputWriter: ShuffleMapOutputWriter): Unit = { var nextPartitionId = 0 if (spills.isEmpty) { diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 4f660a13f7400..10e6936eb3799 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -135,7 +135,7 @@ public void setUp() throws IOException { ); }); - when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); + when(shuffleBlockResolver.getDataFile(anyInt(), anyLong())).thenReturn(mergedOutputFile); Answer renameTempAnswer = invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; @@ -153,11 +153,11 @@ public void setUp() throws IOException { doAnswer(renameTempAnswer) .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); + .writeIndexFileAndCommit(anyInt(), anyLong(), any(long[].class), any(File.class)); doAnswer(renameTempAnswer) .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), eq(null)); + .writeIndexFileAndCommit(anyInt(), anyLong(), any(long[].class), eq(null)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); @@ -176,9 +176,9 @@ private UnsafeShuffleWriter createWriter(boolean transferToEnabl conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); return new UnsafeShuffleWriter<>( blockManager, - taskMemoryManager, - new SerializedShuffleHandle<>(0, 1, shuffleDep), - 0, // map id + taskMemoryManager, + new SerializedShuffleHandle<>(0, shuffleDep), + 0L, // map id taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), @@ -536,8 +536,8 @@ public void testPeakMemoryUsed() throws Exception { final UnsafeShuffleWriter writer = new UnsafeShuffleWriter<>( blockManager, taskMemoryManager, - new SerializedShuffleHandle<>(0, 1, shuffleDep), - 0, // map id + new SerializedShuffleHandle<>(0, shuffleDep), + 0L, // map id taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index e7eef8ec5150c..8433a6f52ac7a 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -142,6 +142,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { sid, taskContext.partitionId(), taskContext.partitionId(), + taskContext.partitionId(), "simulated fetch failure") } else { iter diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d86975964b558..da2ba2165bb0c 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -64,14 +64,15 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(1000L, 10000L))) + Array(1000L, 10000L), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(10000L, 1000L))) + Array(10000L, 1000L), 6)) val statuses = tracker.getMapSizesByExecutorId(10, 0) assert(statuses.toSet === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), - (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) - .toSet) + Seq((BlockManagerId("a", "hostA", 1000), + ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000, 0))), + (BlockManagerId("b", "hostB", 1000), + ArrayBuffer((ShuffleBlockId(10, 6, 0), size10000, 1)))).toSet) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() rpcEnv.shutdown() @@ -86,9 +87,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(compressedSize1000, compressedSize10000))) + Array(compressedSize1000, compressedSize10000), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(compressedSize10000, compressedSize1000))) + Array(compressedSize10000, compressedSize1000), 6)) assert(tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0).nonEmpty) assert(0 == tracker.getNumCachedSerializedBroadcast) @@ -109,9 +110,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(compressedSize1000, compressedSize1000, compressedSize1000))) + Array(compressedSize1000, compressedSize1000, compressedSize1000), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(compressedSize10000, compressedSize1000, compressedSize1000))) + Array(compressedSize10000, compressedSize1000, compressedSize1000), 6)) assert(0 == tracker.getNumCachedSerializedBroadcast) // As if we had two simultaneous fetch failures @@ -147,10 +148,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("a", "hostA", 1000), Array(1000L))) + BlockManagerId("a", "hostA", 1000), Array(1000L), 5)) slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByExecutorId(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) + Seq((BlockManagerId("a", "hostA", 1000), + ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000, 0))))) assert(0 == masterTracker.getNumCachedSerializedBroadcast) val masterTrackerEpochBeforeLossOfMapOutput = masterTracker.getEpoch @@ -184,7 +186,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { // Message size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) + BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0), 5)) val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) when(rpcCallContext.senderAddress).thenReturn(senderAddress) @@ -218,11 +220,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { // on hostB with output size 3 tracker.registerShuffle(10, 3) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(2L))) + Array(2L), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(2L))) + Array(2L), 6)) tracker.registerMapOutput(10, 2, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(3L))) + Array(3L), 7)) // When the threshold is 50%, only host A should be returned as a preferred location // as it has 4 out of 7 bytes of output. @@ -262,7 +264,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.registerShuffle(20, 100) (0 until 100).foreach { i => masterTracker.registerMapOutput(20, i, new CompressedMapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0), 5)) } val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) @@ -311,16 +313,18 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(size0, size1000, size0, size10000))) + Array(size0, size1000, size0, size10000), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(size10000, size0, size1000, size0))) + Array(size10000, size0, size1000, size0), 6)) assert(tracker.containsShuffle(10)) - assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq === + assert(tracker.getMapSizesByExecutorId(10, 0, 4, false).toSeq === Seq( (BlockManagerId("a", "hostA", 1000), - Seq((ShuffleBlockId(10, 0, 1), size1000), (ShuffleBlockId(10, 0, 3), size10000))), + Seq((ShuffleBlockId(10, 5, 1), size1000, 0), + (ShuffleBlockId(10, 5, 3), size10000, 0))), (BlockManagerId("b", "hostB", 1000), - Seq((ShuffleBlockId(10, 1, 0), size10000), (ShuffleBlockId(10, 1, 2), size1000))) + Seq((ShuffleBlockId(10, 6, 0), size10000, 1), + (ShuffleBlockId(10, 6, 2), size1000, 1))) ) ) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 4c9d7e9faeb2a..aa13f1e16203b 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -360,7 +360,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val metricsSystem = sc.env.metricsSystem val shuffleMapRdd = new MyRDD(sc, 1, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) - val shuffleHandle = manager.registerShuffle(0, 1, shuffleDep) + val shuffleHandle = manager.registerShuffle(0, shuffleDep) mapTrackerMaster.registerShuffle(0, 1) // first attempt -- its successful diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index a7fb7ea27ff26..8df66673b9619 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -705,7 +705,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu if (context.stageAttemptNumber == 0) { if (context.partitionId == 0) { // Make the first task in the first stage attempt fail. - throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 0, 0, 0, + throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 0, 0L, 0, 0, new java.io.IOException("fake")) } else { // Make the second task in the first stage attempt sleep to generate a zombie task diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index ac7e4b51ebc2b..3faab52d6510c 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -528,7 +528,8 @@ class FetchFailureThrowingRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { throw new FetchFailedException( bmAddress = BlockManagerId("1", "hostA", 1234), shuffleId = 0, - mapId = 0, + mapId = 0L, + mapIndex = 0, reduceId = 0, message = "fake fetch failure" ) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index b86d24deb15fc..e41d10709b0f4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -484,18 +484,22 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // map stage1 completes successfully, with one task on each executor complete(taskSets(0), Seq( (Success, - MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))), + MapStatus( + BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 5)), (Success, - MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))), - (Success, makeMapStatus("hostB", 1)) + MapStatus( + BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 6)), + (Success, makeMapStatus("hostB", 1, mapTaskId = 7)) )) // map stage2 completes successfully, with one task on each executor complete(taskSets(1), Seq( (Success, - MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))), + MapStatus( + BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 8)), (Success, - MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))), - (Success, makeMapStatus("hostB", 1)) + MapStatus( + BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 9)), + (Success, makeMapStatus("hostB", 1, mapTaskId = 10)) )) // make sure our test setup is correct val initialMapStatus1 = mapOutputTracker.shuffleStatuses(firstShuffleId).mapStatuses @@ -503,16 +507,19 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(initialMapStatus1.count(_ != null) === 3) assert(initialMapStatus1.map{_.location.executorId}.toSet === Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + assert(initialMapStatus1.map{_.mapTaskId}.toSet === Set(5, 6, 7)) val initialMapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get assert(initialMapStatus2.count(_ != null) === 3) assert(initialMapStatus2.map{_.location.executorId}.toSet === Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + assert(initialMapStatus2.map{_.mapTaskId}.toSet === Set(8, 9, 10)) // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( - (FetchFailed(BlockManagerId("exec-hostA2", "hostA", 12345), firstShuffleId, 0, 0, "ignored"), + (FetchFailed(BlockManagerId("exec-hostA2", "hostA", 12345), + firstShuffleId, 0L, 0, 0, "ignored"), null) )) @@ -754,7 +761,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null))) // this will get called // blockManagerMaster.removeExecutor("exec-hostA") // ask the scheduler to try it again @@ -901,7 +908,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) => - (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0L, 0, idx, "ignored"), null) }.toSeq) } @@ -1134,14 +1141,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) assert(sparkListener.failedStages.contains(1)) // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1L, 1, 1, "ignored"), null)) // The SparkListener should not receive redundant failure events. assert(sparkListener.failedStages.size === 1) @@ -1161,7 +1168,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq(0, 1))) @@ -1263,7 +1270,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) assert(sparkListener.failedStages.contains(1)) @@ -1276,7 +1283,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1L, 1, 1, "ignored"), null)) // Another ResubmitFailedStages event should not result in another attempt for the map @@ -1322,7 +1329,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) // Trigger resubmission of the failed map stage and finish the re-started map task. @@ -1337,7 +1344,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // A late FetchFailed arrives from the second task in the original reduce stage. runEvent(makeCompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1L, 1, 1, "ignored"), null)) // Running ResubmitFailedStages shouldn't result in any more attempts for the map stage, because @@ -1532,7 +1539,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(null, firstShuffleId, 2, 0, "Fetch failed"), + FetchFailed(null, firstShuffleId, 2L, 2, 0, "Fetch failed"), null)) // so we resubmit stage 0, which completes happily @@ -1791,7 +1798,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // lets say there is a fetch failure in this task set, which makes us go back and // run stage 0, attempt 1 complete(taskSets(1), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), + shuffleDep1.shuffleId, 0L, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // stage 0, attempt 1 should have the properties of job2 @@ -1872,7 +1880,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), + shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again @@ -1903,7 +1912,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", 1)))) // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), + shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. @@ -2264,7 +2274,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0, 1)) complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null))) // Ask the scheduler to try it again; TaskSet 2 will rerun the map task that we couldn't fetch // from, then TaskSet 3 will run the reduce stage scheduler.resubmitFailedStages() @@ -2323,7 +2333,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1) complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", rdd2.partitions.length)), - (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0L, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() assert(listener2.results.size === 0) // Second stage listener should not have a result yet @@ -2349,7 +2359,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(4).stageId === 2) complete(taskSets(4), Seq( (Success, 52), - (FetchFailed(makeBlockManagerId("hostD"), dep2.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostD"), dep2.shuffleId, 0L, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // TaskSet 5 will rerun stage 1's lost task, then TaskSet 6 will rerun stage 2 @@ -2387,7 +2397,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1) complete(taskSets(1), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)), - (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0L, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // Stage1 listener should not have a result yet assert(listener2.results.size === 0) @@ -2522,7 +2532,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi rdd1.map { case (x, _) if (x == 1) => throw new FetchFailedException( - BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0L, 0, 0, "test") case (x, _) => x }.count() } @@ -2535,7 +2545,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi rdd1.map { case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) => throw new FetchFailedException( - BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0L, 0, 0, "test") } } @@ -2589,7 +2599,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1 && taskSets(1).stageAttemptId === 0) runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleIdA, 0, 0, + FetchFailed(makeBlockManagerId("hostA"), shuffleIdA, 0L, 0, 0, "Fetch failure of task: stageId=1, stageAttempt=0, partitionId=0"), result = null)) @@ -2742,7 +2752,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(countSubmittedMapStageAttempts() === 2) } - test("SPARK-23207: retry all the succeeding stages when the map stage is indeterminate") { + private def constructIndeterminateStageFetchFailed(): (Int, Int) = { val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) @@ -2770,14 +2780,140 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first task of the final stage failed with fetch failure runEvent(makeCompletionEvent( taskSets(2).tasks(0), - FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0L, 0, 0, "ignored"), null)) + (shuffleId1, shuffleId2) + } + + test("SPARK-25341: abort stage while using old fetch protocol") { + // reset the test context with using old fetch protocol + afterEach() + val conf = new SparkConf() + conf.set(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL.key, "true") + init(conf) + // Construct the scenario of indeterminate stage fetch failed. + constructIndeterminateStageFetchFailed() + // The job should fail because Spark can't rollback the shuffle map stage while + // using old protocol. + assert(failure != null && failure.getMessage.contains( + "Spark can only do this while using the new shuffle block fetching protocol")) + } + + test("SPARK-25341: retry all the succeeding stages when the map stage is indeterminate") { + val (shuffleId1, shuffleId2) = constructIndeterminateStageFetchFailed() + + // Check status for all failedStages + val failedStages = scheduler.failedStages.toSeq + assert(failedStages.map(_.id) == Seq(1, 2)) + // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. + assert(failedStages.collect { + case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage + }.head.findMissingPartitions() == Seq(0)) + // The result stage is still waiting for its 2 tasks to complete + assert(failedStages.collect { + case stage: ResultStage => stage + }.head.findMissingPartitions() == Seq(0, 1)) + + scheduler.resubmitFailedStages() + + // The first task of the `shuffleMapRdd2` failed with fetch failure + runEvent(makeCompletionEvent( + taskSets(3).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), + null)) + + val newFailedStages = scheduler.failedStages.toSeq + assert(newFailedStages.map(_.id) == Seq(0, 1)) + + scheduler.resubmitFailedStages() + + // First shuffle map stage resubmitted and reran all tasks. + assert(taskSets(4).stageId == 0) + assert(taskSets(4).stageAttemptId == 1) + assert(taskSets(4).tasks.length == 2) + + // Finish all stage. + complete(taskSets(4), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + + complete(taskSets(5), Seq( + (Success, makeMapStatus("hostC", 2)), + (Success, makeMapStatus("hostD", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) + + complete(taskSets(6), Seq((Success, 11), (Success, 12))) + + // Job successful ended. + assert(results === Map(0 -> 11, 1 -> 12)) + results.clear() + assertDataStructuresEmpty() + } + + test("SPARK-25341: continuous indeterminate stage roll back") { + // shuffleMapRdd1/2/3 are all indeterminate. + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) + val shuffleId1 = shuffleDep1.shuffleId + + val shuffleMapRdd2 = new MyRDD( + sc, 2, List(shuffleDep1), tracker = mapOutputTracker, indeterminate = true) + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new HashPartitioner(2)) + val shuffleId2 = shuffleDep2.shuffleId + + val shuffleMapRdd3 = new MyRDD( + sc, 2, List(shuffleDep2), tracker = mapOutputTracker, indeterminate = true) + val shuffleDep3 = new ShuffleDependency(shuffleMapRdd3, new HashPartitioner(2)) + val shuffleId3 = shuffleDep3.shuffleId + val finalRdd = new MyRDD(sc, 2, List(shuffleDep3), tracker = mapOutputTracker) + + submit(finalRdd, Array(0, 1), properties = new Properties()) + + // Finish the first 2 shuffle map stages. + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + + complete(taskSets(1), Seq( + (Success, makeMapStatus("hostB", 2)), + (Success, makeMapStatus("hostD", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) + + // Executor lost on hostB, both of stage 0 and 1 should be reran. + runEvent(makeCompletionEvent( + taskSets(2).tasks(0), + FetchFailed(makeBlockManagerId("hostB"), shuffleId2, 0L, 0, 0, "ignored"), + null)) + mapOutputTracker.removeOutputsOnHost("hostB") + + assert(scheduler.failedStages.toSeq.map(_.id) == Seq(1, 2)) + scheduler.resubmitFailedStages() + + def checkAndCompleteRetryStage( + taskSetIndex: Int, + stageId: Int, + shuffleId: Int): Unit = { + assert(taskSets(taskSetIndex).stageId == stageId) + assert(taskSets(taskSetIndex).stageAttemptId == 1) + assert(taskSets(taskSetIndex).tasks.length == 2) + complete(taskSets(taskSetIndex), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) + } + + // Check all indeterminate stage roll back. + checkAndCompleteRetryStage(3, 0, shuffleId1) + checkAndCompleteRetryStage(4, 1, shuffleId2) + checkAndCompleteRetryStage(5, 2, shuffleId3) - // The second shuffle map stage need to rerun, the job will abort for the indeterminate - // stage rerun. - // TODO: After we support re-generate shuffle file(SPARK-25341), this test will be extended. - assert(failure != null && failure.getMessage - .contains("Spark cannot rollback the ShuffleMapStage 1")) + // Result stage success, all job ended. + complete(taskSets(6), Seq((Success, 11), (Success, 12))) + assert(results === Map(0 -> 11, 1 -> 12)) + results.clear() + assertDataStructuresEmpty() } test("SPARK-29042: Sampled RDD with unordered input should be indeterminate") { @@ -2810,7 +2946,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( taskSets.last.tasks(1), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) // The job should fail because Spark can't rollback the result stage. @@ -2853,7 +2989,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( taskSets.last.tasks(1), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) assert(failure == null, "job should not fail") @@ -2900,33 +3036,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(latch.await(10, TimeUnit.SECONDS)) } - test("SPARK-28699: abort stage if parent stage is indeterminate stage") { - val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) - - val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) - val shuffleId = shuffleDep.shuffleId - val finalRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) - - submit(finalRdd, Array(0, 1)) - - // Finish the first shuffle map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) - assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) - - runEvent(makeCompletionEvent( - taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), - null)) - - // Shuffle blocks of "hostA" is lost, so first task of the `shuffleMapRdd` needs to retry. - // The result stage is still waiting for its 2 tasks to complete. - // Because of shuffleMapRdd is indeterminate, this job will be abort. - assert(failure != null && failure.getMessage - .contains("Spark cannot rollback the ShuffleMapStage 0")) - } - test("Completions in zombie tasksets update status of non-zombie taskset") { val parts = 4 val shuffleMapRdd = new MyRDD(sc, parts, Nil) @@ -2943,7 +3052,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second task of the shuffle map stage failed with FetchFailed. runEvent(makeCompletionEvent( taskSets(0).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleDep.shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleDep.shuffleId, 0L, 0, 0, "ignored"), null)) scheduler.resubmitFailedStages() @@ -3033,8 +3142,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } object DAGSchedulerSuite { - def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus = - MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes)) + def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2, mapTaskId: Long = -1): MapStatus = + MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes), mapTaskId) def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index c1e7fb9a1db16..700d9ebd76c0c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -61,7 +61,7 @@ class MapStatusSuite extends SparkFunSuite { stddev <- Seq(0.0, 0.01, 0.5, 1.0) ) { val sizes = Array.fill[Long](numSizes)(abs(round(Random.nextGaussian() * stddev)) + mean) - val status = MapStatus(BlockManagerId("a", "b", 10), sizes) + val status = MapStatus(BlockManagerId("a", "b", 10), sizes, -1) val status1 = compressAndDecompressMapStatus(status) for (i <- 0 until numSizes) { if (sizes(i) != 0) { @@ -75,7 +75,7 @@ class MapStatusSuite extends SparkFunSuite { test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { val sizes = Array.fill[Long](2001)(150L) - val status = MapStatus(null, sizes) + val status = MapStatus(null, sizes, -1) assert(status.isInstanceOf[HighlyCompressedMapStatus]) assert(status.getSizeForBlock(10) === 150L) assert(status.getSizeForBlock(50) === 150L) @@ -87,10 +87,12 @@ class MapStatusSuite extends SparkFunSuite { val sizes = Array.tabulate[Long](3000) { i => i.toLong } val avg = sizes.sum / sizes.count(_ != 0) val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes) + val mapTaskAttemptId = 5 + val status = MapStatus(loc, sizes, mapTaskAttemptId) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) + assert(status1.mapTaskId == mapTaskAttemptId) for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { @@ -109,7 +111,7 @@ class MapStatusSuite extends SparkFunSuite { val smallBlockSizes = sizes.filter(n => n > 0 && n < threshold) val avg = smallBlockSizes.sum / smallBlockSizes.length val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes) + val status = MapStatus(loc, sizes, 5) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) @@ -165,7 +167,7 @@ class MapStatusSuite extends SparkFunSuite { SparkEnv.set(env) // Value of element in sizes is equal to the corresponding index. val sizes = (0L to 2000L).toArray - val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes) + val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes, 5) val arrayStream = new ByteArrayOutputStream(102400) val objectOutputStream = new ObjectOutputStream(arrayStream) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index d6964063c118e..6f80c7c0fe817 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -254,7 +254,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { .reduceByKey { case (_, _) => val ctx = TaskContext.get() if (ctx.stageAttemptNumber() == 0) { - throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 1, 1, 1, + throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 1, 1L, 1, 1, new Exception("Failure for test.")) } else { ctx.stageId() diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 96706536fe53c..4f737c9499ad6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -621,7 +621,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor backend.taskSuccess(taskDescription, DAGSchedulerSuite.makeMapStatus("hostA", 10)) case (1, 0, 0) => val fetchFailed = FetchFailed( - DAGSchedulerSuite.makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored") + DAGSchedulerSuite.makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored") backend.taskFailed(taskDescription, fetchFailed) case (1, _, partition) => backend.taskSuccess(taskDescription, 42 + partition) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index c16b552d20891..394a2a9fbf7cb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -176,7 +176,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark if (stageAttemptNumber < 2) { // Throw FetchFailedException to explicitly trigger stage resubmission. A normal exception // will only trigger task resubmission in the same stage. - throw new FetchFailedException(null, 0, 0, 0, "Fake") + throw new FetchFailedException(null, 0, 0L, 0, 0, "Fake") } Seq(stageAttemptNumber).iterator }.collect() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index fedfa083e8d8f..5b1cb08aa4813 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1262,7 +1262,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // now fail those tasks tsmSpy.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0, 0, "ignored")) + FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0L, 0, 0, "ignored")) tsmSpy.handleFailedTask(taskDescs(1).taskId, TaskState.FAILED, ExecutorLostFailure(taskDescs(1).executorId, exitCausedByApp = false, reason = None)) tsmSpy.handleFailedTask(taskDescs(2).taskId, TaskState.FAILED, @@ -1302,7 +1302,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Fail the task with fetch failure tsm.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0, 0, "ignored")) + FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0L, 0, 0, "ignored")) assert(blacklistTracker.isNodeBlacklisted("host1")) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 2442670b6d3f0..43d7d12a3caed 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -350,8 +350,11 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val ser = new KryoSerializer(conf).newInstance() val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) + var mapTaskId = 0 Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) + mapTaskId += 1 + ser.serialize(HighlyCompressedMapStatus( + BlockManagerId("exec-1", "host", 1234), blockSizes, mapTaskId)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 6d2ef17a7a790..d0cbb30fe0232 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -102,12 +102,13 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Make a mocked MapOutputTracker for the shuffle reader to use to determine what // shuffle data to read. val mapOutputTracker = mock(classOf[MapOutputTracker]) - when(mapOutputTracker.getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1)).thenReturn { + when(mapOutputTracker.getMapSizesByExecutorId( + shuffleId, reduceId, reduceId + 1, useOldFetchProtocol = false)).thenReturn { // Test a scenario where all data is local, to avoid creating a bunch of additional mocks // for the code to read data over the network. val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) - (shuffleBlockId, byteOutputStream.size().toLong) + (shuffleBlockId, byteOutputStream.size().toLong, mapId) } Seq((localBlockManagerId, shuffleBlockIdsAndSizes)).toIterator } @@ -118,7 +119,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext when(dependency.serializer).thenReturn(serializer) when(dependency.aggregator).thenReturn(None) when(dependency.keyOrdering).thenReturn(None) - new BaseShuffleHandle(shuffleId, numMaps, dependency) + new BaseShuffleHandle(shuffleId, dependency) } val serializerManager = new SerializerManager( diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index b9f81fa0d0a06..f8474022867f4 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, anyInt} +import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mockito._ import org.scalatest.BeforeAndAfterEach @@ -65,7 +65,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte taskMetrics = new TaskMetrics shuffleHandle = new BypassMergeSortShuffleHandle[Int, Int]( shuffleId = 0, - numMaps = 2, dependency = dependency ) val memoryManager = new TestMemoryManager(conf) @@ -78,7 +77,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) when(blockResolver.writeIndexFileAndCommit( - anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]))) + anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[File]))) .thenAnswer { invocationOnMock => val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] if (tmp != null) { @@ -139,8 +138,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0, // MapId - 0L, // MapTaskAttemptId + 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, shuffleExecutorComponents) @@ -166,8 +164,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0, // MapId - 0L, + 0L, // MapId transferConf, taskContext.taskMetrics().shuffleWriteMetrics, shuffleExecutorComponents) @@ -202,8 +199,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0, // MapId - 0L, + 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, shuffleExecutorComponents) @@ -224,8 +220,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0, // MapId - 0L, + 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, shuffleExecutorComponents) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala index 0dd6040808f9e..4c5694fcf0305 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -57,7 +57,7 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with when(dependency.serializer).thenReturn(serializer) when(dependency.aggregator).thenReturn(None) when(dependency.keyOrdering).thenReturn(None) - new BaseShuffleHandle(shuffleId, numMaps = numMaps, dependency) + new BaseShuffleHandle(shuffleId, dependency) } shuffleExecutorComponents = new LocalDiskShuffleExecutorComponents( conf, blockManager, shuffleBlockResolver) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index 5156cc2cc47a6..f92455912f510 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -23,7 +23,7 @@ import java.nio.file.Files import java.util.Arrays import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, anyInt} +import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mock import org.mockito.Mockito.when import org.mockito.MockitoAnnotations @@ -73,9 +73,9 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA conf = new SparkConf() .set("spark.app.id", "example.spark.app") .set("spark.shuffle.unsafe.file.output.buffer", "16k") - when(blockResolver.getDataFile(anyInt, anyInt)).thenReturn(mergedOutputFile) + when(blockResolver.getDataFile(anyInt, anyLong)).thenReturn(mergedOutputFile) when(blockResolver.writeIndexFileAndCommit( - anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]))) + anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[File]))) .thenAnswer { invocationOnMock => partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index ed402440e74f1..e5a615c2c2cbb 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -98,9 +98,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = createMockTransfer(remoteBlocks) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (localBmId, localBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq), - (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (localBmId, localBlocks.keys.map(blockId => (blockId, 1L, 0)).toSeq), + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1L, 1)).toSeq) ).toIterator val taskContext = TaskContext.empty() @@ -179,8 +179,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT } }) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)).toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -247,8 +247,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT } }) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)) + .toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -336,8 +337,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT } }) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)).toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -389,8 +390,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val corruptBuffer1 = mockCorruptBuffer(streamLength, 0) val blockManagerId1 = BlockManagerId("remote-client-1", "remote-client-1", 1) val shuffleBlockId1 = ShuffleBlockId(0, 1, 0) - val blockLengths1 = Seq[Tuple2[BlockId, Long]]( - shuffleBlockId1 -> corruptBuffer1.size() + val blockLengths1 = Seq[Tuple3[BlockId, Long, Int]]( + (shuffleBlockId1, corruptBuffer1.size(), 1) ) val streamNotCorruptTill = 8 * 1024 @@ -398,13 +399,13 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val corruptBuffer2 = mockCorruptBuffer(streamLength, streamNotCorruptTill) val blockManagerId2 = BlockManagerId("remote-client-2", "remote-client-2", 2) val shuffleBlockId2 = ShuffleBlockId(0, 2, 0) - val blockLengths2 = Seq[Tuple2[BlockId, Long]]( - shuffleBlockId2 -> corruptBuffer2.size() + val blockLengths2 = Seq[Tuple3[BlockId, Long, Int]]( + (shuffleBlockId2, corruptBuffer2.size(), 2) ) val transfer = createMockTransfer( Map(shuffleBlockId1 -> corruptBuffer1, shuffleBlockId2 -> corruptBuffer2)) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( (blockManagerId1, blockLengths1), (blockManagerId2, blockLengths2) ).toIterator @@ -465,11 +466,11 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val localBmId = BlockManagerId("test-client", "test-client", 1) doReturn(localBmId).when(blockManager).blockManagerId doReturn(managedBuffer).when(blockManager).getBlockData(ShuffleBlockId(0, 0, 0)) - val localBlockLengths = Seq[Tuple2[BlockId, Long]]( - ShuffleBlockId(0, 0, 0) -> 10000 + val localBlockLengths = Seq[Tuple3[BlockId, Long, Int]]( + (ShuffleBlockId(0, 0, 0), 10000, 0) ) val transfer = createMockTransfer(Map(ShuffleBlockId(0, 0, 0) -> managedBuffer)) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( (localBmId, localBlockLengths) ).toIterator @@ -531,8 +532,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT } }) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)) + .toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -591,7 +593,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT }) def fetchShuffleBlock( - blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])]): Unit = { + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])]): Unit = { // Set `maxBytesInFlight` and `maxReqsInFlight` to `Int.MaxValue`, so that during the // construction of `ShuffleBlockFetcherIterator`, all requests to fetch remote shuffle blocks // are issued. The `maxReqSizeShuffleToMem` is hard-coded as 200 here. @@ -611,15 +613,15 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT taskContext.taskMetrics.createTempShuffleReadMetrics()) } - val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq)).toIterator + val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L, 0)).toSeq)).toIterator fetchShuffleBlock(blocksByAddress1) // `maxReqSizeShuffleToMem` is 200, which is greater than the block size 100, so don't fetch // shuffle block to disk. assert(tempFileManager == null) - val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq)).toIterator + val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L, 0)).toSeq)).toIterator fetchShuffleBlock(blocksByAddress2) // `maxReqSizeShuffleToMem` is 200, which is smaller than the block size 300, so fetch // shuffle block to disk. @@ -640,8 +642,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = createMockTransfer(blocks.mapValues(_ => createMockManagedBuffer(0))) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)) val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 1913b8d425519..580af086ba9da 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -316,10 +316,12 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val env = SparkEnv.get val bmAddress = env.blockManager.blockManagerId val shuffleId = shuffleHandle.shuffleId - val mapId = 0 + val mapId = 0L + val mapIndex = 0 val reduceId = taskContext.partitionId() val message = "Simulated fetch failure" - throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message) + throw new FetchFailedException( + bmAddress, shuffleId, mapId, mapIndex, reduceId, message) } else { x } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index e781c5f71faf4..54625a93679fb 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -179,7 +179,7 @@ class JsonProtocolSuite extends SparkFunSuite { testJobResult(jobFailed) // TaskEndReason - val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, 18, 19, "Some exception") val fetchMetadataFailed = new MetadataFetchFailedException(17, 19, "metadata Fetch failed exception").toTaskFailedReason @@ -296,12 +296,12 @@ class JsonProtocolSuite extends SparkFunSuite { test("FetchFailed backwards compatibility") { // FetchFailed in Spark 1.1.0 does not have a "Message" property. - val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, 18, 19, "ignored") val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) .removeField({ _._1 == "Message" }) - val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, - "Unknown reason") + val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, + 18, 19, "Unknown reason") assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } @@ -732,6 +732,7 @@ private[spark] object JsonProtocolSuite extends Assertions { case (r1: FetchFailed, r2: FetchFailed) => assert(r1.shuffleId === r2.shuffleId) assert(r1.mapId === r2.mapId) + assert(r1.mapIndex === r2.mapIndex) assert(r1.reduceId === r2.reduceId) assert(r1.bmAddress === r2.bmAddress) assert(r1.message === r2.message) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 38738112849f7..78897d2863c22 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -420,7 +420,32 @@ object MimaExcludes { // [SPARK-25382][SQL][PYSPARK] Remove ImageSchema.readImages in 3.0 ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.image.ImageSchema.readImages"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.image.ImageSchema.readImages") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.image.ImageSchema.readImages"), + + // [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.shuffle.sort.UnsafeShuffleWriter.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.copy$default$2"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.copy$default$2"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleBlockId.copy$default$2"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.apply"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.apply"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.apply"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.mapId"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.FetchFailed$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.apply"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.copy$default$5"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.copy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.copy$default$3"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.this") ) // Exclude rules for 2.4.x From 655356e8257c8d1b551a0b4686a0ef324ddda8a1 Mon Sep 17 00:00:00 2001 From: xy_xin Date: Mon, 23 Sep 2019 19:25:56 +0800 Subject: [PATCH 0159/1387] [SPARK-28892][SQL] support UPDATE in the parser and add the corresponding logical plan ### What changes were proposed in this pull request? This PR supports UPDATE in the parser and add the corresponding logical plan. The SQL syntax is a standard UPDATE statement: ``` UPDATE tableName tableAlias SET colName=value [, colName=value]+ WHERE predicate? ``` ### Why are the changes needed? With this change, we can start to implement UPDATE in builtin sources and think about how to design the update API in DS v2. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? New test cases added. Closes #25626 from xianyinxin/SPARK-28892. Authored-by: xy_xin Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 1 + .../spark/sql/catalyst/parser/SqlBase.g4 | 12 +++++ .../sql/catalyst/parser/AstBuilder.scala | 32 +++++++++++++- .../logical/sql/UpdateTableStatement.scala | 27 ++++++++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 44 ++++++++++++++++++- .../datasources/DataSourceResolution.scala | 5 ++- .../sql-tests/results/pgSQL/comments.sql.out | 18 ++++---- .../sql/connector/DataSourceV2SQLSuite.scala | 25 +++++++++++ 8 files changed, 152 insertions(+), 12 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UpdateTableStatement.scala diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index c55e589aa8c3f..4ea1754618cd4 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -280,6 +280,7 @@ Below is a list of all the keywords in Spark SQL. UNKNOWNreservednon-reservedreserved UNLOCKnon-reservednon-reservednon-reserved UNSETnon-reservednon-reservednon-reserved + UPDATEnon-reservednon-reservedreserved USEnon-reservednon-reservednon-reserved USERreservednon-reservedreserved USINGreservedstrict-non-reservedreserved diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 20be8e539cf58..642e140668761 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -217,6 +217,7 @@ statement | SET .*? #setConfiguration | RESET #resetConfiguration | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable + | UPDATE multipartIdentifier tableAlias setClause whereClause? #updateTable | unsupportedHiveNativeCommands .*? #failNativeCommand ; @@ -476,6 +477,14 @@ selectClause : SELECT (hints+=hint)* setQuantifier? namedExpressionSeq ; +setClause + : SET assign (',' assign)* + ; + +assign + : key=multipartIdentifier EQ value=expression + ; + whereClause : WHERE booleanExpression ; @@ -1085,6 +1094,7 @@ ansiNonReserved | UNCACHE | UNLOCK | UNSET + | UPDATE | USE | VALUES | VIEW @@ -1355,6 +1365,7 @@ nonReserved | UNKNOWN | UNLOCK | UNSET + | UPDATE | USE | USER | VALUES @@ -1622,6 +1633,7 @@ UNIQUE: 'UNIQUE'; UNKNOWN: 'UNKNOWN'; UNLOCK: 'UNLOCK'; UNSET: 'UNSET'; +UPDATE: 'UPDATE'; USE: 'USE'; USER: 'USER'; USING: 'USING'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 74c485e091cd8..ae036a831e8ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement, UpdateTableStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf @@ -361,6 +361,36 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging DeleteFromStatement(tableId, tableAlias, predicate) } + override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) { + val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) + val tableAlias = if (ctx.tableAlias() != null) { + val ident = ctx.tableAlias().strictIdentifier() + // We do not allow columns aliases after table alias. + if (ctx.tableAlias().identifierList() != null) { + throw new ParseException("Columns aliases is not allowed in UPDATE.", + ctx.tableAlias().identifierList()) + } + if (ident != null) Some(ident.getText) else None + } else { + None + } + val (attrs, values) = ctx.setClause().assign().asScala.map { + kv => visitMultipartIdentifier(kv.key) -> expression(kv.value) + }.unzip + val predicate = if (ctx.whereClause() != null) { + Some(expression(ctx.whereClause().booleanExpression())) + } else { + None + } + + UpdateTableStatement( + tableId, + tableAlias, + attrs, + values, + predicate) + } + /** * Create a partition specification map. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UpdateTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UpdateTableStatement.scala new file mode 100644 index 0000000000000..954374c15b932 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UpdateTableStatement.scala @@ -0,0 +1,27 @@ +/* + * 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.plans.logical.sql + +import org.apache.spark.sql.catalyst.expressions.Expression + +case class UpdateTableStatement( + tableName: Seq[String], + tableAlias: Option[String], + attrs: Seq[Seq[String]], + values: Seq[Expression], + condition: Option[Expression]) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index be781081b0112..1dfc9b58504a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement, UpdateTableStatement} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -789,6 +789,48 @@ class DDLParserSuite extends AnalysisTest { assert(exc.getMessage.contains("Columns aliases is not allowed in DELETE.")) } + test("update table: basic") { + parseCompare( + """ + |UPDATE testcat.ns1.ns2.tbl + |SET t.a='Robert', t.b=32 + """.stripMargin, + UpdateTableStatement( + Seq("testcat", "ns1", "ns2", "tbl"), + None, + Seq(Seq("t", "a"), Seq("t", "b")), + Seq(Literal("Robert"), Literal(32)), + None)) + } + + test("update table: with alias and where clause") { + parseCompare( + """ + |UPDATE testcat.ns1.ns2.tbl AS t + |SET t.a='Robert', t.b=32 + |WHERE t.c=2 + """.stripMargin, + UpdateTableStatement( + Seq("testcat", "ns1", "ns2", "tbl"), + Some("t"), + Seq(Seq("t", "a"), Seq("t", "b")), + Seq(Literal("Robert"), Literal(32)), + Some(EqualTo(UnresolvedAttribute("t.c"), Literal(2))))) + } + + test("update table: columns aliases is not allowed") { + val exc = intercept[ParseException] { + parsePlan( + """ + |UPDATE testcat.ns1.ns2.tbl AS t(a,b,c,d) + |SET b='Robert', c=32 + |WHERE d=2 + """.stripMargin) + } + + assert(exc.getMessage.contains("Columns aliases is not allowed in UPDATE.")) + } + test("show tables") { comparePlans( parsePlan("SHOW TABLES"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index b9b8e069d5237..dfa5a68661289 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{CastSupport, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DeleteFromTable, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect, ShowNamespaces, ShowTables, SubqueryAlias} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement, UpdateTableStatement} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, LookupCatalog, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform @@ -187,6 +187,9 @@ case class DataSourceResolution( s"No v2 catalog is available for ${namespace.quoted}") } + case update: UpdateTableStatement => + throw new AnalysisException(s"Update table is not supported temporarily.") + case ShowTablesStatement(None, pattern) => defaultCatalog match { case Some(catalog) => diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out index fbd32fc909076..5e8383e794579 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/comments.sql.out @@ -42,7 +42,7 @@ struct<> -- !query 4 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == /* This is an example of SQL which should not execute: @@ -58,7 +58,7 @@ struct<> -- !query 5 output org.apache.spark.sql.catalyst.parser.ParseException -extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == */ @@ -74,7 +74,7 @@ struct<> -- !query 6 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == /* @@ -92,7 +92,7 @@ struct<> -- !query 7 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == */ @@ -114,7 +114,7 @@ struct<> -- !query 8 output org.apache.spark.sql.catalyst.parser.ParseException -extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == */ @@ -134,7 +134,7 @@ struct<> -- !query 9 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == /* Second level of nesting... @@ -150,7 +150,7 @@ struct<> -- !query 10 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == /* Third level of nesting... @@ -170,7 +170,7 @@ struct<> -- !query 11 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) == SQL == */ @@ -189,7 +189,7 @@ struct<> -- !query 12 output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'USE', 'VALUES', 'WITH'}(line 1, pos 37) +mismatched input '' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 37) == SQL == /* and this is the end of the file */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 9eb8e5b3b73fc..7fde00fe9a419 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1049,6 +1049,31 @@ class DataSourceV2SQLSuite } } + test("Update: basic - update all") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql( + s""" + |CREATE TABLE $t (id bigint, name string, age int, p int) + |USING foo + |PARTITIONED BY (id, p) + """.stripMargin) + sql( + s""" + |INSERT INTO $t + |VALUES (1L, 'Herry', 26, 1), + |(2L, 'Jack', 31, 2), + |(3L, 'Lisa', 28, 3), + |(4L, 'Frank', 33, 3) + """.stripMargin) + } + val errMsg = "Update table is not supported temporarily" + testCreateAnalysisError( + s"UPDATE $t SET name='Robert', age=32", + errMsg + ) + } + private def testCreateAnalysisError(sqlStatement: String, expectedError: String): Unit = { val errMsg = intercept[AnalysisException] { sql(sqlStatement) From c08bc3728141db139273dc56039430ecb70e6abb Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 23 Sep 2019 19:46:01 +0800 Subject: [PATCH 0160/1387] [SPARK-29177][CORE] fix zombie tasks after stage abort ### What changes were proposed in this pull request? Do task handling even the task exceeds maxResultSize configured. More details are in the jira description https://issues.apache.org/jira/browse/SPARK-29177 . ### Why are the changes needed? Without this patch, the zombie tasks will prevent yarn from recycle those containers running these tasks, which will affect other applications. ### Does this PR introduce any user-facing change? No ### How was this patch tested? unit test and production test with a very large `SELECT` in spark thriftserver. Closes #25850 from adrian-wang/zombie. Authored-by: Daoyuan Wang Signed-off-by: Wenchen Fan --- .../spark/scheduler/TaskResultGetter.scala | 6 ++++ .../scheduler/TaskResultGetterSuite.scala | 36 +++++++++++++++++++ 2 files changed, 42 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 9b7f901c55e00..670742c65602e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -64,6 +64,9 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul val (result, size) = serializer.get().deserialize[TaskResult[_]](serializedData) match { case directResult: DirectTaskResult[_] => if (!taskSetManager.canFetchMoreResults(serializedData.limit())) { + // kill the task so that it will not become zombie task + scheduler.handleFailedTask(taskSetManager, tid, TaskState.KILLED, TaskKilled( + "Tasks result size has exceeded maxResultSize")) return } // deserialize "value" without holding any lock so that it won't block other threads. @@ -75,6 +78,9 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul if (!taskSetManager.canFetchMoreResults(size)) { // dropped by executor if size is larger than maxResultSize sparkEnv.blockManager.master.removeBlock(blockId) + // kill the task so that it will not become zombie task + scheduler.handleFailedTask(taskSetManager, tid, TaskState.KILLED, TaskKilled( + "Tasks result size has exceeded maxResultSize")) return } logDebug("Fetching indirect task result for TID %s".format(tid)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index ae464352da440..0d9f2b2df53dc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -33,6 +33,7 @@ import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.TaskState.TaskState import org.apache.spark.TestUtils.JavaSourceFromString import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.storage.TaskResultBlockId @@ -78,6 +79,16 @@ private class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: Task } } +private class DummyTaskSchedulerImpl(sc: SparkContext) + extends TaskSchedulerImpl(sc, 1, true) { + override def handleFailedTask( + taskSetManager: TaskSetManager, + tid: Long, + taskState: TaskState, + reason: TaskFailedReason): Unit = { + // do nothing + } +} /** * A [[TaskResultGetter]] that stores the [[DirectTaskResult]]s it receives from executors @@ -130,6 +141,31 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local "Expect result to be removed from the block manager.") } + test("handling total size of results larger than maxResultSize") { + sc = new SparkContext("local", "test", conf) + val scheduler = new DummyTaskSchedulerImpl(sc) + val spyScheduler = spy(scheduler) + val resultGetter = new TaskResultGetter(sc.env, spyScheduler) + scheduler.taskResultGetter = resultGetter + val myTsm = new TaskSetManager(spyScheduler, FakeTask.createTaskSet(2), 1) { + // always returns false + override def canFetchMoreResults(size: Long): Boolean = false + } + val indirectTaskResult = IndirectTaskResult(TaskResultBlockId(0), 0) + val directTaskResult = new DirectTaskResult(ByteBuffer.allocate(0), Nil, Array()) + val ser = sc.env.closureSerializer.newInstance() + val serializedIndirect = ser.serialize(indirectTaskResult) + val serializedDirect = ser.serialize(directTaskResult) + resultGetter.enqueueSuccessfulTask(myTsm, 0, serializedDirect) + resultGetter.enqueueSuccessfulTask(myTsm, 1, serializedIndirect) + eventually(timeout(1.second)) { + verify(spyScheduler, times(1)).handleFailedTask( + myTsm, 0, TaskState.KILLED, TaskKilled("Tasks result size has exceeded maxResultSize")) + verify(spyScheduler, times(1)).handleFailedTask( + myTsm, 1, TaskState.KILLED, TaskKilled("Tasks result size has exceeded maxResultSize")) + } + } + test("task retried if result missing from block manager") { // Set the maximum number of task failures to > 0, so that the task set isn't aborted // after the result is missing. From d22768a6be42bdd8af147112a01ec0910d8d0931 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 23 Sep 2019 05:47:25 -0700 Subject: [PATCH 0161/1387] [SPARK-29036][SQL] SparkThriftServer cancel job after execute() thread interrupted ### What changes were proposed in this pull request? Discuss in https://github.com/apache/spark/pull/25611 If cancel() and close() is called very quickly after the query is started, then they may both call cleanup() before Spark Jobs are started. Then sqlContext.sparkContext.cancelJobGroup(statementId) does nothing. But then the execute thread can start the jobs, and only then get interrupted and exit through here. But then it will exit here, and no-one will cancel these jobs and they will keep running even though this execution has exited. So when execute() was interrupted by `cancel()`, when get into catch block, we should call canJobGroup again to make sure the job was canceled. ### Why are the changes needed? ### Does this PR introduce any user-facing change? NO ### How was this patch tested? MT Closes #25743 from AngersZhuuuu/SPARK-29036. Authored-by: angerszhu Signed-off-by: Yuming Wang --- .../hive/thriftserver/SparkExecuteStatementOperation.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index f246f43435c75..ce3cbc3a2fc41 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -267,6 +267,13 @@ private[hive] class SparkExecuteStatementOperation( // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. case e: Throwable => + // When cancel() or close() is called very quickly after the query is started, + // then they may both call cleanup() before Spark Jobs are started. But before background + // task interrupted, it may have start some spark job, so we need to cancel again to + // make sure job was cancelled when background thread was interrupted + if (statementId != null) { + sqlContext.sparkContext.cancelJobGroup(statementId) + } val currentState = getStatus().getState() if (currentState.isTerminal) { // This may happen if the execution was cancelled, and then closed from another thread. From 0c40b94ae57b994cfc5c848baa0fc2629ba378c3 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 23 Sep 2019 08:38:40 -0700 Subject: [PATCH 0162/1387] [SPARK-29203][SQL][TESTS] Reduce shuffle partitions in SQLQueryTestSuite ### What changes were proposed in this pull request? This PR reduce shuffle partitions from 200 to 4 in `SQLQueryTestSuite` to reduce testing time. ### Why are the changes needed? Reduce testing time. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested in my local: Before: ``` ... [info] - subquery/in-subquery/in-joins.sql (6 minutes, 19 seconds) [info] - subquery/in-subquery/not-in-joins.sql (2 minutes, 17 seconds) [info] - subquery/scalar-subquery/scalar-subquery-predicate.sql (45 seconds, 763 milliseconds) ... Run completed in 1 hour, 22 minutes. ``` After: ``` ... [info] - subquery/in-subquery/in-joins.sql (1 minute, 12 seconds) [info] - subquery/in-subquery/not-in-joins.sql (27 seconds, 541 milliseconds) [info] - subquery/scalar-subquery/scalar-subquery-predicate.sql (17 seconds, 360 milliseconds) ... Run completed in 47 minutes. ``` Closes #25891 from wangyum/SPARK-29203. Authored-by: Yuming Wang Signed-off-by: Yuming Wang --- .../sql-tests/inputs/subquery/in-subquery/in-joins.sql | 2 +- .../inputs/subquery/in-subquery/not-in-joins.sql | 2 +- .../results/subquery/in-subquery/in-joins.sql.out | 2 +- .../results/subquery/in-subquery/not-in-joins.sql.out | 2 +- .../scala/org/apache/spark/sql/SQLQueryTestSuite.scala | 10 +++++----- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql index 22f3eafd6a02d..f06be5fc6aa58 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -83,7 +83,7 @@ GROUP BY t1a, t3a, t3b, t3c -ORDER BY t1a DESC, t3b DESC; +ORDER BY t1a DESC, t3b DESC, t3c ASC; -- TC 01.03 SELECT Count(DISTINCT(t1a)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql index 4f8ca8bfb27c1..bebc18a61894b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql @@ -128,7 +128,7 @@ GROUP BY t1b, HAVING t1d NOT IN (SELECT t2d FROM t2 WHERE t1d = t2d) -ORDER BY t1b DESC; +ORDER BY t1b DESC, t1d ASC; -- TC 01.05 SELECT COUNT(DISTINCT(t1a)), diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out index ab6a11a2b7efa..7089fbd25e28c 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out @@ -102,7 +102,7 @@ GROUP BY t1a, t3a, t3b, t3c -ORDER BY t1a DESC, t3b DESC +ORDER BY t1a DESC, t3b DESC, t3c ASC -- !query 4 schema struct -- !query 4 output diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out index bae5d00cc8632..498378bfa5c37 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out @@ -169,7 +169,7 @@ GROUP BY t1b, HAVING t1d NOT IN (SELECT t2d FROM t2 WHERE t1d = t2d) -ORDER BY t1b DESC +ORDER BY t1b DESC, t1d ASC -- !query 6 schema struct -- !query 6 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 1a41dd95a5700..71e881a0c6cc7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -22,7 +22,7 @@ import java.util.{Locale, TimeZone} import scala.util.control.NonFatal -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} @@ -135,6 +135,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { private val notIncludedMsg = "[not included in comparison]" private val clsName = this.getClass.getCanonicalName + override def sparkConf: SparkConf = super.sparkConf + // Fewer shuffle partitions to speed up testing. + .set(SQLConf.SHUFFLE_PARTITIONS, 4) + /** List of test cases to ignore, in lower cases. */ protected def blackList: Set[String] = Set( "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. @@ -293,10 +297,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { testCase match { case udfTestCase: UDFTest => - // In Python UDF tests, the number of shuffle partitions matters considerably in - // the testing time because it requires to fork and communicate between external - // processes. - localSparkSession.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, 4) registerTestUDF(udfTestCase.udf, localSparkSession) case _ => } From d50f6e6344f9966c872e7ea9639af752a28ba23e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 24 Sep 2019 00:13:38 +0800 Subject: [PATCH 0163/1387] [SPARK-25903][CORE] TimerTask should be synchronized on ContextBarrierState ### What changes were proposed in this pull request? BarrierCoordinator sets up a TimerTask for a round of global sync. Currently the run method is synchronized on the created TimerTask. But to be synchronized with handleRequest, it should be synchronized on the ContextBarrierState object, not TimerTask object. ### Why are the changes needed? ContextBarrierState.handleRequest and TimerTask.run both access the internal status of a ContextBarrierState object. If TimerTask doesn't be synchronized on the same ContextBarrierState object, when the timer task is triggered, handleRequest still accepts new request and modify requesters field in the ContextBarrierState object. It makes the behavior inconsistency. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Test locally Closes #25897 from viirya/SPARK-25903. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/BarrierCoordinator.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala index 9f59295059d30..4e417679ca663 100644 --- a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala @@ -107,9 +107,9 @@ private[spark] class BarrierCoordinator( private var timerTask: TimerTask = null // Init a TimerTask for a barrier() call. - private def initTimerTask(): Unit = { + private def initTimerTask(state: ContextBarrierState): Unit = { timerTask = new TimerTask { - override def run(): Unit = synchronized { + override def run(): Unit = state.synchronized { // Timeout current barrier() call, fail all the sync requests. requesters.foreach(_.sendFailure(new SparkException("The coordinator didn't get all " + s"barrier sync requests for barrier epoch $barrierEpoch from $barrierId within " + @@ -148,7 +148,7 @@ private[spark] class BarrierCoordinator( // If this is the first sync message received for a barrier() call, start timer to ensure // we may timeout for the sync. if (requesters.isEmpty) { - initTimerTask() + initTimerTask(this) timer.schedule(timerTask, timeoutInSecs * 1000) } // Add the requester to array of RPCCallContexts pending for reply. From c38f459059f0b38e2c7f0b5bcb633d96ace42a50 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 23 Sep 2019 09:19:04 -0700 Subject: [PATCH 0164/1387] [SPARK-29016][BUILD] Update LICENSE and NOTICE for Hive 2.3 ### What changes were proposed in this pull request? This PR update LICENSE and NOTICE for Hive 2.3. Hive 2.3 newly added jars: ``` dropwizard-metrics-hadoop-metrics2-reporter.jar HikariCP-2.5.1.jar hive-common-2.3.6.jar hive-llap-common-2.3.6.jar hive-serde-2.3.6.jar hive-service-rpc-2.3.6.jar hive-shims-0.23-2.3.6.jar hive-shims-2.3.6.jar hive-shims-common-2.3.6.jar hive-shims-scheduler-2.3.6.jar hive-storage-api-2.6.0.jar hive-vector-code-gen-2.3.6.jar javax.jdo-3.2.0-m3.jar json-1.8.jar transaction-api-1.1.jar velocity-1.5.jar ``` ### Why are the changes needed? We will distribute a binary release based on Hadoop 3.2 / Hive 2.3 in future. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #25896 from wangyum/SPARK-29016. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- LICENSE-binary | 15 ++++ NOTICE-binary | 231 ++++++++++++++++++++++++++++++++++++++++++++++++- 2 files changed, 245 insertions(+), 1 deletion(-) diff --git a/LICENSE-binary b/LICENSE-binary index 16da39145d050..327c8b13fc370 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -369,6 +369,20 @@ org.eclipse.jetty:jetty-webapp org.eclipse.jetty:jetty-xml org.scala-lang.modules:scala-xml_2.12 org.opencypher:okapi-shade +com.github.joshelser:dropwizard-metrics-hadoop-metrics2-reporter +com.zaxxer.HikariCP +org.apache.hive:hive-common +org.apache.hive:hive-llap-common +org.apache.hive:hive-serde +org.apache.hive:hive-service-rpc +org.apache.hive:hive-shims-0.23 +org.apache.hive:hive-shims +org.apache.hive:hive-shims-scheduler +org.apache.hive:hive-storage-api +org.apache.hive:hive-vector-code-gen +org.datanucleus:javax.jdo +com.tdunning:json +org.apache.velocity:velocity core/src/main/java/org/apache/spark/util/collection/TimSort.java core/src/main/resources/org/apache/spark/ui/static/bootstrap* @@ -464,6 +478,7 @@ Common Development and Distribution License (CDDL) 1.0 javax.activation:activation http://www.oracle.com/technetwork/java/javase/tech/index-jsp-138795.html javax.xml.stream:stax-api https://jcp.org/en/jsr/detail?id=173 +javax.transaction:javax.transaction-api Common Development and Distribution License (CDDL) 1.1 diff --git a/NOTICE-binary b/NOTICE-binary index f93e088a9a731..4dfcca1023f57 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -1135,4 +1135,233 @@ Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and -limitations under the License. \ No newline at end of file +limitations under the License. + +dropwizard-metrics-hadoop-metrics2-reporter +Copyright 2016 Josh Elser + +Hive Common +Copyright 2019 The Apache Software Foundation + +Hive Llap Common +Copyright 2019 The Apache Software Foundation + +Hive Serde +Copyright 2019 The Apache Software Foundation + +Hive Service RPC +Copyright 2019 The Apache Software Foundation + +Hive Shims 0.23 +Copyright 2019 The Apache Software Foundation + +Hive Shims Common +Copyright 2019 The Apache Software Foundation + +Hive Shims Scheduler +Copyright 2019 The Apache Software Foundation + +Hive Storage API +Copyright 2018 The Apache Software Foundation + +Hive Vector-Code-Gen Utilities +Copyright 2019 The Apache Software Foundation + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + Copyright 2015-2015 DataNucleus + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +Apache Velocity + +Copyright (C) 2000-2007 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). From 4513f1c0dc450e9249d43fdad618fdcaf8d399b6 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 23 Sep 2019 19:25:36 -0500 Subject: [PATCH 0165/1387] [SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) ## What changes were proposed in this pull request? This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp. The new option would be optional of course, and take preference over existing offset options. ## How was this patch tested? New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server. Running query below ``` val df = spark.read.format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1") .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""") .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""") .load().selectExpr("CAST(value AS STRING)") df.show() ``` with below records (one string which number part remarks when they're put after such timestamp) in topic `spark_26848_test_v1` ``` hello1 1549669142193 world1 1549669142193 hellow1 1549669240965 world1 1549669240965 hello1 1549669265676 world1 1549669265676 ``` topic `spark_26848_test_2_v1` ``` hello2 1549669142193 world2 1549669142193 hello2 1549669240965 world2 1549669240965 hello2 1549669265676 world2 1549669265676 ``` the result of `df.show()` follows: ``` +--------------------+ | value| +--------------------+ |world1 1549669240965| |world1 1549669142193| |world2 1549669240965| |hello2 1549669240965| |hellow1 154966924...| |hello2 1549669265676| |hello1 1549669142193| |world2 1549669265676| +--------------------+ ``` Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive. Closes #23747 from HeartSaVioR/SPARK-26848. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Sean Owen --- .../structured-streaming-kafka-integration.md | 40 ++++ .../apache/spark/sql/kafka010/JsonUtils.scala | 24 ++- .../spark/sql/kafka010/KafkaBatch.scala | 4 +- .../sql/kafka010/KafkaContinuousStream.scala | 2 + .../sql/kafka010/KafkaMicroBatchStream.scala | 2 + .../sql/kafka010/KafkaOffsetRangeLimit.scala | 7 + .../sql/kafka010/KafkaOffsetReader.scala | 194 +++++++++++------ .../spark/sql/kafka010/KafkaRelation.scala | 4 +- .../spark/sql/kafka010/KafkaSource.scala | 2 + .../sql/kafka010/KafkaSourceProvider.scala | 53 +++-- .../sql/kafka010/KafkaDataConsumerSuite.scala | 5 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 119 ++++++++++- .../sql/kafka010/KafkaRelationSuite.scala | 198 +++++++++++++++++- .../spark/sql/kafka010/KafkaTestUtils.scala | 59 ++---- .../spark/sql/kafka010/RecordBuilder.scala | 74 +++++++ 15 files changed, 644 insertions(+), 143 deletions(-) create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/RecordBuilder.scala diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index ef4cdc2608f45..89732d309aa27 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -362,6 +362,27 @@ The following configurations are optional: + + + + + + + + + + + + + +
    Optionvaluedefaultquery typemeaning
    startingOffsetsByTimestampjson string + """ {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """ + none (the value of startingOffsets will apply)streaming and batchThe start point of timestamp when a query is started, a json string specifying a starting timestamp for + each TopicPartition. The returned offset for each partition is the earliest offset whose timestamp is greater than or + equal to the given timestamp in the corresponding partition. If the matched offset doesn't exist, + the query will fail immediately to prevent unintended read from such partition. (This is a kind of limitation as of now, and will be addressed in near future.)

    +

    + Spark simply passes the timestamp information to KafkaConsumer.offsetsForTimes, and doesn't interpret or reason about the value.

    + For more details on KafkaConsumer.offsetsForTimes, please refer javadoc for details.

    + Also the meaning of timestamp here can be vary according to Kafka configuration (log.message.timestamp.type): please refer Kafka documentation for further details.

    + Note: This option requires Kafka 0.10.1.0 or higher.

    + Note2: startingOffsetsByTimestamp takes precedence over startingOffsets.

    + Note3: For streaming queries, this only applies when a new query is started, and that resuming will + always pick up from where the query left off. Newly discovered partitions during a query will start at + earliest.

    startingOffsets "earliest", "latest" (streaming only), or json string @@ -377,6 +398,25 @@ The following configurations are optional: always pick up from where the query left off. Newly discovered partitions during a query will start at earliest.
    endingOffsetsByTimestampjson string + """ {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """ + latestbatch queryThe end point when a batch query is ended, a json string specifying an ending timesamp for each TopicPartition. + The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to + the given timestamp in the corresponding partition. If the matched offset doesn't exist, the offset will + be set to latest.

    +

    + Spark simply passes the timestamp information to KafkaConsumer.offsetsForTimes, and doesn't interpret or reason about the value.

    + For more details on KafkaConsumer.offsetsForTimes, please refer javadoc for details.

    + Also the meaning of timestamp here can be vary according to Kafka configuration (log.message.timestamp.type): please refer Kafka documentation for further details.

    + Note: This option requires Kafka 0.10.1.0 or higher.

    + Note2: endingOffsetsByTimestamp takes precedence over endingOffsets. +

    endingOffsets latest or json string diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala index 868edb5dcdc0c..6dd5af2389a81 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala @@ -68,7 +68,7 @@ private object JsonUtils { partOffsets.map { case (part, offset) => new TopicPartition(topic, part) -> offset } - }.toMap + } } catch { case NonFatal(x) => throw new IllegalArgumentException( @@ -76,12 +76,27 @@ private object JsonUtils { } } + def partitionTimestamps(str: String): Map[TopicPartition, Long] = { + try { + Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partTimestamps) => + partTimestamps.map { case (part, timestamp) => + new TopicPartition(topic, part) -> timestamp + } + } + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA": {"0": 123456789, "1": 123456789}, + |"topicB": {"0": 123456789, "1": 123456789}}, got $str""".stripMargin) + } + } + /** * Write per-TopicPartition offsets as json string */ def partitionOffsets(partitionOffsets: Map[TopicPartition, Long]): String = { val result = new HashMap[String, HashMap[Int, Long]]() - implicit val ordering = new Ordering[TopicPartition] { + implicit val order = new Ordering[TopicPartition] { override def compare(x: TopicPartition, y: TopicPartition): Int = { Ordering.Tuple2[String, Int].compare((x.topic, x.partition), (y.topic, y.partition)) } @@ -95,4 +110,9 @@ private object JsonUtils { } Serialization.write(result) } + + def partitionTimestamps(topicTimestamps: Map[TopicPartition, Long]): String = { + // For now it's same as partitionOffsets + partitionOffsets(topicTimestamps) + } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala index 667c383681917..3006770f306c0 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala @@ -59,8 +59,8 @@ private[kafka010] class KafkaBatch( // Leverage the KafkaReader to obtain the relevant partition offsets val (fromPartitionOffsets, untilPartitionOffsets) = { try { - (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets), - kafkaOffsetReader.fetchPartitionOffsets(endingOffsets)) + (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets, isStartingOffsets = true), + kafkaOffsetReader.fetchPartitionOffsets(endingOffsets, isStartingOffsets = false)) } finally { kafkaOffsetReader.close() } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala index 9e7b7d6db2038..0603ae39ba622 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala @@ -69,6 +69,8 @@ class KafkaContinuousStream( case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) + case SpecificTimestampRangeLimit(p) => offsetReader.fetchSpecificTimestampBasedOffsets(p, + failsOnNoMatchingOffset = true) } logInfo(s"Initial offsets: $offsets") offsets diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index 6ea6efe5d1b74..01f6ba4445162 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -192,6 +192,8 @@ private[kafka010] class KafkaMicroBatchStream( KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss) + case SpecificTimestampRangeLimit(p) => + kafkaOffsetReader.fetchSpecificTimestampBasedOffsets(p, failsOnNoMatchingOffset = true) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala index 80a026f4f5d73..d64b5d4f7e9e8 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala @@ -42,6 +42,13 @@ private[kafka010] case object LatestOffsetRangeLimit extends KafkaOffsetRangeLim private[kafka010] case class SpecificOffsetRangeLimit( partitionOffsets: Map[TopicPartition, Long]) extends KafkaOffsetRangeLimit +/** + * Represents the desire to bind to earliest offset which timestamp for the offset is equal or + * greater than specific timestamp. + */ +private[kafka010] case class SpecificTimestampRangeLimit( + topicTimestamps: Map[TopicPartition, Long]) extends KafkaOffsetRangeLimit + private[kafka010] object KafkaOffsetRangeLimit { /** * Used to denote offset range limits that are resolved via Kafka diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 20f2ce11d4afb..0179f4dd822f1 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -26,7 +26,7 @@ import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration import scala.util.control.NonFatal -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetAndTimestamp} import org.apache.kafka.common.TopicPartition import org.apache.spark.internal.Logging @@ -126,12 +126,14 @@ private[kafka010] class KafkaOffsetReader( * Fetch the partition offsets for the topic partitions that are indicated * in the [[ConsumerStrategy]] and [[KafkaOffsetRangeLimit]]. */ - def fetchPartitionOffsets(offsetRangeLimit: KafkaOffsetRangeLimit): Map[TopicPartition, Long] = { + def fetchPartitionOffsets( + offsetRangeLimit: KafkaOffsetRangeLimit, + isStartingOffsets: Boolean): Map[TopicPartition, Long] = { def validateTopicPartitions(partitions: Set[TopicPartition], partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { assert(partitions == partitionOffsets.keySet, "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + - "Use -1 for latest, -2 for earliest, if you don't care.\n" + + "Use -1 for latest, -2 for earliest.\n" + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions}") logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") partitionOffsets @@ -147,6 +149,9 @@ private[kafka010] class KafkaOffsetReader( }.toMap case SpecificOffsetRangeLimit(partitionOffsets) => validateTopicPartitions(partitions, partitionOffsets) + case SpecificTimestampRangeLimit(partitionTimestamps) => + fetchSpecificTimestampBasedOffsets(partitionTimestamps, + failsOnNoMatchingOffset = isStartingOffsets).partitionToOffsets } } @@ -161,23 +166,83 @@ private[kafka010] class KafkaOffsetReader( def fetchSpecificOffsets( partitionOffsets: Map[TopicPartition, Long], reportDataLoss: String => Unit): KafkaSourceOffset = { - val fetched = runUninterruptibly { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() + val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { partitions => + assert(partitions.asScala == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + } - // Call `position` to wait until the potential offset request triggered by `poll(0)` is - // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by - // `poll(0)` may reset offsets that should have been set by another request. - partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) + val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => + partitionOffsets + } + + val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { fetched => + partitionOffsets.foreach { + case (tp, off) if off != KafkaOffsetRangeLimit.LATEST && + off != KafkaOffsetRangeLimit.EARLIEST => + if (fetched(tp) != off) { + reportDataLoss( + s"startingOffsets for $tp was $off but consumer reset to ${fetched(tp)}") + } + case _ => + // no real way to check that beginning or end is reasonable + } + } + + fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets, + fnAssertFetchedOffsets) + } + + def fetchSpecificTimestampBasedOffsets( + partitionTimestamps: Map[TopicPartition, Long], + failsOnNoMatchingOffset: Boolean): KafkaSourceOffset = { + val fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit = { partitions => + assert(partitions.asScala == partitionTimestamps.keySet, + "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " + + s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps") + } - consumer.pause(partitions) - assert(partitions.asScala == partitionOffsets.keySet, - "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + - "Use -1 for latest, -2 for earliest, if you don't care.\n" + - s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => { + val converted = partitionTimestamps.map { case (tp, timestamp) => + tp -> java.lang.Long.valueOf(timestamp) + }.asJava + + val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] = + consumer.offsetsForTimes(converted) + + offsetForTime.asScala.map { case (tp, offsetAndTimestamp) => + if (failsOnNoMatchingOffset) { + assert(offsetAndTimestamp != null, "No offset matched from request of " + + s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.") + } + + if (offsetAndTimestamp == null) { + tp -> KafkaOffsetRangeLimit.LATEST + } else { + tp -> offsetAndTimestamp.offset() + } + }.toMap + } + } + + val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { _ => } + + fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets, + fnAssertFetchedOffsets) + } + + private def fetchSpecificOffsets0( + fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit, + fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long], + fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit): KafkaSourceOffset = { + val fetched = partitionsAssignedToConsumer { + partitions => { + fnAssertParametersWithPartitions(partitions) + + val partitionOffsets = fnRetrievePartitionOffsets(partitions) partitionOffsets.foreach { case (tp, KafkaOffsetRangeLimit.LATEST) => @@ -186,22 +251,15 @@ private[kafka010] class KafkaOffsetReader( consumer.seekToBeginning(ju.Arrays.asList(tp)) case (tp, off) => consumer.seek(tp, off) } + partitionOffsets.map { case (tp, _) => tp -> consumer.position(tp) } } } - partitionOffsets.foreach { - case (tp, off) if off != KafkaOffsetRangeLimit.LATEST && - off != KafkaOffsetRangeLimit.EARLIEST => - if (fetched(tp) != off) { - reportDataLoss( - s"startingOffsets for $tp was $off but consumer reset to ${fetched(tp)}") - } - case _ => - // no real way to check that beginning or end is reasonable - } + fnAssertFetchedOffsets(fetched) + KafkaSourceOffset(fetched) } @@ -209,20 +267,15 @@ private[kafka010] class KafkaOffsetReader( * Fetch the earliest offsets for the topic partitions that are indicated * in the [[ConsumerStrategy]]. */ - def fetchEarliestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") + def fetchEarliestOffsets(): Map[TopicPartition, Long] = partitionsAssignedToConsumer( + partitions => { + logDebug("Seeking to the beginning") consumer.seekToBeginning(partitions) val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap logDebug(s"Got earliest offsets for partition : $partitionOffsets") partitionOffsets - } - } + }, fetchingEarliestOffset = true) /** * Fetch the latest offsets for the topic partitions that are indicated @@ -239,19 +292,9 @@ private[kafka010] class KafkaOffsetReader( * distinguish this with KAFKA-7703, so we just return whatever we get from Kafka after retrying. */ def fetchLatestOffsets( - knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap = runUninterruptibly { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - - // Call `position` to wait until the potential offset request triggered by `poll(0)` is - // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by - // `poll(0)` may reset offsets that should have been set by another request. - partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) - - consumer.pause(partitions) - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") + knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap = + partitionsAssignedToConsumer { partitions => { + logDebug("Seeking to the end.") if (knownOffsets.isEmpty) { consumer.seekToEnd(partitions) @@ -315,25 +358,40 @@ private[kafka010] class KafkaOffsetReader( if (newPartitions.isEmpty) { Map.empty[TopicPartition, Long] } else { - runUninterruptibly { - withRetriesWithoutInterrupt { - // Poll to get the latest assigned partitions - consumer.poll(0) - val partitions = consumer.assignment() - consumer.pause(partitions) - logDebug(s"\tPartitions assigned to consumer: $partitions") - - // Get the earliest offset of each partition - consumer.seekToBeginning(partitions) - val partitionOffsets = newPartitions.filter { p => - // When deleting topics happen at the same time, some partitions may not be in - // `partitions`. So we need to ignore them - partitions.contains(p) - }.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") - partitionOffsets - } + partitionsAssignedToConsumer(partitions => { + // Get the earliest offset of each partition + consumer.seekToBeginning(partitions) + val partitionOffsets = newPartitions.filter { p => + // When deleting topics happen at the same time, some partitions may not be in + // `partitions`. So we need to ignore them + partitions.contains(p) + }.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") + partitionOffsets + }, fetchingEarliestOffset = true) + } + } + + private def partitionsAssignedToConsumer( + body: ju.Set[TopicPartition] => Map[TopicPartition, Long], + fetchingEarliestOffset: Boolean = false) + : Map[TopicPartition, Long] = runUninterruptibly { + + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + + if (!fetchingEarliestOffset) { + // Call `position` to wait until the potential offset request triggered by `poll(0)` is + // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by + // `poll(0)` may reset offsets that should have been set by another request. + partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) } + + consumer.pause(partitions) + logDebug(s"Partitions assigned to consumer: $partitions.") + body(partitions) } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index 886f6b0fe0a5e..61479c992039b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -68,8 +68,8 @@ private[kafka010] class KafkaRelation( // Leverage the KafkaReader to obtain the relevant partition offsets val (fromPartitionOffsets, untilPartitionOffsets) = { try { - (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets), - kafkaOffsetReader.fetchPartitionOffsets(endingOffsets)) + (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets, isStartingOffsets = true), + kafkaOffsetReader.fetchPartitionOffsets(endingOffsets, isStartingOffsets = false)) } finally { kafkaOffsetReader.close() } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 29944dc3fbf1e..e1392b6215d3a 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -105,6 +105,8 @@ private[kafka010] class KafkaSource( case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaReader.fetchSpecificOffsets(p, reportDataLoss) + case SpecificTimestampRangeLimit(p) => + kafkaReader.fetchSpecificTimestampBasedOffsets(p, failsOnNoMatchingOffset = true) } metadataLog.add(0, offsets) logInfo(s"Initial offsets: $offsets") diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index a7f8db35d7cf9..c15f08d78741d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -89,7 +89,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveParameters, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( strategy(caseInsensitiveParameters), @@ -126,11 +127,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveParameters) val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParameters, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) + caseInsensitiveParameters, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) assert(startingRelationOffsets != LatestOffsetRangeLimit) val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveParameters, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveParameters, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) assert(endingRelationOffsets != EarliestOffsetRangeLimit) val includeHeaders = caseInsensitiveParameters.getOrElse(INCLUDE_HEADERS, "false").toBoolean @@ -321,13 +324,17 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister // Stream specific options params.get(ENDING_OFFSETS_OPTION_KEY).map(_ => throw new IllegalArgumentException("ending offset not valid in streaming queries")) + params.get(ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY).map(_ => + throw new IllegalArgumentException("ending timestamp not valid in streaming queries")) + validateGeneralOptions(params) } private def validateBatchOptions(params: CaseInsensitiveMap[String]) = { // Batch specific options KafkaSourceProvider.getKafkaOffsetRangeLimit( - params, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match { + params, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, STARTING_OFFSETS_OPTION_KEY, + EarliestOffsetRangeLimit) match { case EarliestOffsetRangeLimit => // good to go case LatestOffsetRangeLimit => throw new IllegalArgumentException("starting offset can't be latest " + @@ -339,10 +346,12 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister "be latest for batch queries on Kafka") case _ => // ignore } + case _: SpecificTimestampRangeLimit => // good to go } KafkaSourceProvider.getKafkaOffsetRangeLimit( - params, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match { + params, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, ENDING_OFFSETS_OPTION_KEY, + LatestOffsetRangeLimit) match { case EarliestOffsetRangeLimit => throw new IllegalArgumentException("ending offset can't be earliest " + "for batch queries on Kafka") @@ -354,6 +363,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister "earliest for batch queries on Kafka") case _ => // ignore } + case _: SpecificTimestampRangeLimit => // good to go } validateGeneralOptions(params) @@ -420,10 +430,12 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) + caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveOptions, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveOptions, ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) new KafkaBatch( strategy(caseInsensitiveOptions), @@ -446,7 +458,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( strategy(caseInsensitiveOptions), @@ -474,7 +487,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( - caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + caseInsensitiveOptions, STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY, + STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) val kafkaOffsetReader = new KafkaOffsetReader( strategy(caseInsensitiveOptions), @@ -500,6 +514,8 @@ private[kafka010] object KafkaSourceProvider extends Logging { private val STRATEGY_OPTION_KEYS = Set(SUBSCRIBE, SUBSCRIBE_PATTERN, ASSIGN) private[kafka010] val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" + private[kafka010] val STARTING_OFFSETS_BY_TIMESTAMP_OPTION_KEY = "startingoffsetsbytimestamp" + private[kafka010] val ENDING_OFFSETS_BY_TIMESTAMP_OPTION_KEY = "endingoffsetsbytimestamp" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" private[kafka010] val MIN_PARTITIONS_OPTION_KEY = "minpartitions" private[kafka010] val MAX_OFFSET_PER_TRIGGER = "maxoffsetspertrigger" @@ -543,15 +559,20 @@ private[kafka010] object KafkaSourceProvider extends Logging { def getKafkaOffsetRangeLimit( params: CaseInsensitiveMap[String], + offsetByTimestampOptionKey: String, offsetOptionKey: String, defaultOffsets: KafkaOffsetRangeLimit): KafkaOffsetRangeLimit = { - params.get(offsetOptionKey).map(_.trim) match { - case Some(offset) if offset.toLowerCase(Locale.ROOT) == "latest" => - LatestOffsetRangeLimit - case Some(offset) if offset.toLowerCase(Locale.ROOT) == "earliest" => - EarliestOffsetRangeLimit - case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json)) - case None => defaultOffsets + params.get(offsetByTimestampOptionKey).map(_.trim) match { + case Some(json) => SpecificTimestampRangeLimit(JsonUtils.partitionTimestamps(json)) + case None => + params.get(offsetOptionKey).map(_.trim) match { + case Some(offset) if offset.toLowerCase(Locale.ROOT) == "latest" => + LatestOffsetRangeLimit + case Some(offset) if offset.toLowerCase(Locale.ROOT) == "earliest" => + EarliestOffsetRangeLimit + case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json)) + case None => defaultOffsets + } } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala index d97f627fbac08..a790ed064c3d5 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala @@ -323,7 +323,10 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester private def prepareTestTopicHavingTestMessages(topic: String) = { val data = (1 to 1000).map(i => (i.toString, Seq[(String, Array[Byte])]())) testUtils.createTopic(topic, 1) - testUtils.sendMessages(topic, data.toArray, None) + val messages = data.map { case (value, hdrs) => + new RecordBuilder(topic, value).headers(hdrs).build() + } + testUtils.sendMessages(messages) data } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 609cf3ce4bd75..26136203b09a6 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -1178,9 +1178,10 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) require(testUtils.getLatestOffsets(Set(topic)).size === 5) + val headers = Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8))) (31 to 35).map { num => - (num - 31, (num.toString, Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8))))) - }.foreach { rec => testUtils.sendMessage(topic, rec._2, Some(rec._1)) } + new RecordBuilder(topic, num.toString).partition(num - 31).headers(headers).build() + }.foreach { rec => testUtils.sendMessage(rec) } val kafka = spark .readStream @@ -1277,6 +1278,16 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { "failOnDataLoss" -> failOnDataLoss.toString) } + test(s"assign from specific timestamps (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromSpecificTimestamps( + topic, + failOnDataLoss = failOnDataLoss, + addPartitions = false, + "assign" -> assignString(topic, 0 to 4), + "failOnDataLoss" -> failOnDataLoss.toString) + } + test(s"subscribing topic by name from latest offsets (failOnDataLoss: $failOnDataLoss)") { val topic = newTopic() testFromLatestOffsets( @@ -1300,6 +1311,12 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { testFromSpecificOffsets(topic, failOnDataLoss = failOnDataLoss, "subscribe" -> topic) } + test(s"subscribing topic by name from specific timestamps (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromSpecificTimestamps(topic, failOnDataLoss = failOnDataLoss, addPartitions = true, + "subscribe" -> topic) + } + test(s"subscribing topic by pattern from latest offsets (failOnDataLoss: $failOnDataLoss)") { val topicPrefix = newTopic() val topic = topicPrefix + "-suffix" @@ -1328,6 +1345,17 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { failOnDataLoss = failOnDataLoss, "subscribePattern" -> s"$topicPrefix-.*") } + + test(s"subscribing topic by pattern from specific timestamps " + + s"(failOnDataLoss: $failOnDataLoss)") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromSpecificTimestamps( + topic, + failOnDataLoss = failOnDataLoss, + addPartitions = true, + "subscribePattern" -> s"$topicPrefix-.*") + } } test("bad source options") { @@ -1347,6 +1375,9 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { // Specifying an ending offset testBadOptions("endingOffsets" -> "latest")("Ending offset not valid in streaming queries") + testBadOptions("subscribe" -> "t", "endingOffsetsByTimestamp" -> "{\"t\": {\"0\": 1000}}")( + "Ending timestamp not valid in streaming queries") + // No strategy specified testBadOptions()("options must be specified", "subscribe", "subscribePattern") @@ -1395,7 +1426,8 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { (STARTING_OFFSETS_OPTION_KEY, """{"topic-A":{"0":23}}""", SpecificOffsetRangeLimit(Map(new TopicPartition("topic-A", 0) -> 23))))) { val offset = getKafkaOffsetRangeLimit( - CaseInsensitiveMap[String](Map(optionKey -> optionValue)), optionKey, answer) + CaseInsensitiveMap[String](Map(optionKey -> optionValue)), "dummy", optionKey, + answer) assert(offset === answer) } @@ -1403,7 +1435,7 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { (STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit), (ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit))) { val offset = getKafkaOffsetRangeLimit( - CaseInsensitiveMap[String](Map.empty), optionKey, answer) + CaseInsensitiveMap[String](Map.empty), "dummy", optionKey, answer) assert(offset === answer) } } @@ -1468,12 +1500,89 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { ) } + private def testFromSpecificTimestamps( + topic: String, + failOnDataLoss: Boolean, + addPartitions: Boolean, + options: (String, String)*): Unit = { + def sendMessages(topic: String, msgs: Seq[String], part: Int, ts: Long): Unit = { + val records = msgs.map { msg => + new RecordBuilder(topic, msg).partition(part).timestamp(ts).build() + } + testUtils.sendMessages(records) + } + + testUtils.createTopic(topic, partitions = 5) + + val firstTimestamp = System.currentTimeMillis() - 5000 + sendMessages(topic, Array(-20).map(_.toString), 0, firstTimestamp) + sendMessages(topic, Array(-10).map(_.toString), 1, firstTimestamp) + sendMessages(topic, Array(0, 1).map(_.toString), 2, firstTimestamp) + sendMessages(topic, Array(10, 11).map(_.toString), 3, firstTimestamp) + sendMessages(topic, Array(20, 21, 22).map(_.toString), 4, firstTimestamp) + + val secondTimestamp = firstTimestamp + 1000 + sendMessages(topic, Array(-21, -22).map(_.toString), 0, secondTimestamp) + sendMessages(topic, Array(-11, -12).map(_.toString), 1, secondTimestamp) + sendMessages(topic, Array(2).map(_.toString), 2, secondTimestamp) + sendMessages(topic, Array(12).map(_.toString), 3, secondTimestamp) + // no data after second timestamp for partition 4 + + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + // we intentionally starts from second timestamp, + // except for partition 4 - it starts from first timestamp + val startPartitionTimestamps: Map[TopicPartition, Long] = Map( + (0 to 3).map(new TopicPartition(topic, _) -> secondTimestamp): _* + ) ++ Map(new TopicPartition(topic, 4) -> firstTimestamp) + val startingTimestamps = JsonUtils.partitionTimestamps(startPartitionTimestamps) + + val reader = spark + .readStream + .format("kafka") + .option("startingOffsetsByTimestamp", startingTimestamps) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("failOnDataLoss", failOnDataLoss.toString) + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + + testStream(mapped)( + makeSureGetOffsetCalled, + Execute { q => + val partitions = (0 to 4).map(new TopicPartition(topic, _)) + // wait to reach the last offset in every partition + q.awaitOffset( + 0, KafkaSourceOffset(partitions.map(tp => tp -> 3L).toMap), streamingTimeout.toMillis) + }, + CheckAnswer(-21, -22, -11, -12, 2, 12, 20, 21, 22), + StopStream, + StartStream(), + CheckAnswer(-21, -22, -11, -12, 2, 12, 20, 21, 22), // Should get the data back on recovery + StopStream, + AddKafkaData(Set(topic), 30, 31, 32), // Add data when stream is stopped + StartStream(), + CheckAnswer(-21, -22, -11, -12, 2, 12, 20, 21, 22, 30, 31, 32), // Should get the added data + AssertOnQuery("Add partitions") { query: StreamExecution => + if (addPartitions) setTopicPartitions(topic, 10, query) + true + }, + AddKafkaData(Set(topic), 40, 41, 42, 43, 44)(ensureDataInMultiplePartition = true), + CheckAnswer(-21, -22, -11, -12, 2, 12, 20, 21, 22, 30, 31, 32, 40, 41, 42, 43, 44), + StopStream + ) + } + test("Kafka column types") { val now = System.currentTimeMillis() val topic = newTopic() testUtils.createTopic(newTopic(), partitions = 1) testUtils.sendMessage( - topic, ("1", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))), None + new RecordBuilder(topic, "1") + .headers(Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))).build() ) val kafka = spark diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 3c88609bcb45d..556eab4b56387 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets.UTF_8 import java.util.Locale import java.util.concurrent.atomic.AtomicInteger +import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.util.Random @@ -28,11 +29,11 @@ import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkConf -import org.apache.spark.sql.QueryTest +import org.apache.spark.SparkException +import org.apache.spark.sql.{DataFrameReader, QueryTest} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils @@ -159,13 +160,15 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession val topic = newTopic() testUtils.createTopic(topic, partitions = 3) testUtils.sendMessage( - topic, ("1", Seq()), Some(0) + new RecordBuilder(topic, "1").headers(Seq()).partition(0).build() ) testUtils.sendMessage( - topic, ("2", Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))), Some(1) + new RecordBuilder(topic, "2").headers( + Seq(("a", "b".getBytes(UTF_8)), ("c", "d".getBytes(UTF_8)))).partition(1).build() ) testUtils.sendMessage( - topic, ("3", Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8)))), Some(2) + new RecordBuilder(topic, "3").headers( + Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8)))).partition(2).build() ) // Implicit offset values, should default to earliest and latest @@ -176,6 +179,191 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession ("3", Seq(("e", "f".getBytes(UTF_8)), ("e", "g".getBytes(UTF_8))))).toDF) } + test("timestamp provided for starting and ending") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // timestamp both presented: starting "first" ending "finalized" + verifyTimestampRelatedQueryResult({ df => + val startPartitionTimestamps: Map[TopicPartition, Long] = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(1)): _*) + val startingTimestamps = JsonUtils.partitionTimestamps(startPartitionTimestamps) + + val endPartitionTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(2)): _*) + val endingTimestamps = JsonUtils.partitionTimestamps(endPartitionTimestamps) + + df.option("startingOffsetsByTimestamp", startingTimestamps) + .option("endingOffsetsByTimestamp", endingTimestamps) + }, topic, 10 to 19) + } + + test("timestamp provided for starting, offset provided for ending") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // starting only presented as "first", and ending presented as endingOffsets + verifyTimestampRelatedQueryResult({ df => + val startTopicTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps.head): _*) + val startingTimestamps = JsonUtils.partitionTimestamps(startTopicTimestamps) + + val endPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -1L, // -1 => latest + new TopicPartition(topic, 1) -> -1L, + new TopicPartition(topic, 2) -> 1L // explicit offset - take only first one + ) + val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) + + // so we here expect full of records from partition 0 and 1, and only the first record + // from partition 2 which is "2" + + df.option("startingOffsetsByTimestamp", startingTimestamps) + .option("endingOffsets", endingOffsets) + }, topic, (0 to 29).filterNot(_ % 3 == 2) ++ Seq(2)) + } + + test("timestamp provided for ending, offset provided for starting") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // ending only presented as "third", and starting presented as startingOffsets + verifyTimestampRelatedQueryResult({ df => + val startPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -2L, // -2 => earliest + new TopicPartition(topic, 1) -> -2L, + new TopicPartition(topic, 2) -> 0L // explicit earliest + ) + val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) + + val endTopicTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(2)): _*) + val endingTimestamps = JsonUtils.partitionTimestamps(endTopicTimestamps) + + df.option("startingOffsets", startingOffsets) + .option("endingOffsetsByTimestamp", endingTimestamps) + }, topic, 0 to 19) + } + + test("timestamp provided for starting, ending not provided") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // starting only presented as "second", and ending not presented + verifyTimestampRelatedQueryResult({ df => + val startTopicTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(1)): _*) + val startingTimestamps = JsonUtils.partitionTimestamps(startTopicTimestamps) + + df.option("startingOffsetsByTimestamp", startingTimestamps) + }, topic, 10 to 29) + } + + test("timestamp provided for ending, starting not provided") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // ending only presented as "third", and starting not presented + verifyTimestampRelatedQueryResult({ df => + val endTopicTimestamps = Map( + (0 to 2).map(new TopicPartition(topic, _) -> timestamps(2)): _*) + val endingTimestamps = JsonUtils.partitionTimestamps(endTopicTimestamps) + + df.option("endingOffsetsByTimestamp", endingTimestamps) + }, topic, 0 to 19) + } + + test("no matched offset for timestamp - startingOffsets") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + val e = intercept[SparkException] { + verifyTimestampRelatedQueryResult({ df => + // partition 2 will make query fail + val startTopicTimestamps = Map( + (0 to 1).map(new TopicPartition(topic, _) -> timestamps(1)): _*) ++ + Map(new TopicPartition(topic, 2) -> Long.MaxValue) + + val startingTimestamps = JsonUtils.partitionTimestamps(startTopicTimestamps) + + df.option("startingOffsetsByTimestamp", startingTimestamps) + }, topic, Seq.empty) + } + + @tailrec + def assertionErrorInExceptionChain(e: Throwable): Boolean = { + if (e.isInstanceOf[AssertionError]) { + true + } else if (e.getCause == null) { + false + } else { + assertionErrorInExceptionChain(e.getCause) + } + } + + assert(assertionErrorInExceptionChain(e), + "Cannot find expected AssertionError in chained exceptions") + } + + test("no matched offset for timestamp - endingOffsets") { + val (topic, timestamps) = prepareTimestampRelatedUnitTest + + // the query will run fine, since we allow no matching offset for timestamp + // if it's endingOffsets + // for partition 0 and 1, it only takes records between first and second timestamp + // for partition 2, it will take all records + verifyTimestampRelatedQueryResult({ df => + val endTopicTimestamps = Map( + (0 to 1).map(new TopicPartition(topic, _) -> timestamps(1)): _*) ++ + Map(new TopicPartition(topic, 2) -> Long.MaxValue) + + val endingTimestamps = JsonUtils.partitionTimestamps(endTopicTimestamps) + + df.option("endingOffsetsByTimestamp", endingTimestamps) + }, topic, (0 to 9) ++ (10 to 29).filter(_ % 3 == 2)) + } + + private def prepareTimestampRelatedUnitTest: (String, Seq[Long]) = { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + + def sendMessages(topic: String, msgs: Array[String], part: Int, ts: Long): Unit = { + val records = msgs.map { msg => + new RecordBuilder(topic, msg).partition(part).timestamp(ts).build() + } + testUtils.sendMessages(records) + } + + val firstTimestamp = System.currentTimeMillis() - 5000 + (0 to 2).foreach { partNum => + sendMessages(topic, (0 to 9).filter(_ % 3 == partNum) + .map(_.toString).toArray, partNum, firstTimestamp) + } + + val secondTimestamp = firstTimestamp + 1000 + (0 to 2).foreach { partNum => + sendMessages(topic, (10 to 19).filter(_ % 3 == partNum) + .map(_.toString).toArray, partNum, secondTimestamp) + } + + val thirdTimestamp = secondTimestamp + 1000 + (0 to 2).foreach { partNum => + sendMessages(topic, (20 to 29).filter(_ % 3 == partNum) + .map(_.toString).toArray, partNum, thirdTimestamp) + } + + val finalizedTimestamp = thirdTimestamp + 1000 + + (topic, Seq(firstTimestamp, secondTimestamp, thirdTimestamp, finalizedTimestamp)) + } + + private def verifyTimestampRelatedQueryResult( + optionFn: DataFrameReader => DataFrameReader, + topic: String, + expectation: Seq[Int]): Unit = { + val df = spark.read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + + val df2 = optionFn(df).load().selectExpr("CAST(value AS STRING)") + checkAnswer(df2, expectation.map(_.toString).toDF) + } + test("reuse same dataframe in query") { // This test ensures that we do not cache the Kafka Consumer in KafkaRelation val topic = newTopic() diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index ee3eb43967069..6453b215ff750 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -353,57 +353,33 @@ class KafkaTestUtils( } } - /** Java-friendly function for sending messages to the Kafka broker */ - def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { - sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) + def sendMessages(topic: String, msgs: Array[String]): Seq[(String, RecordMetadata)] = { + sendMessages(topic, msgs, None) } - /** Send the messages to the Kafka broker */ - def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { - val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray - sendMessages(topic, messages) - } - - /** Send the array of messages to the Kafka broker */ - def sendMessages(topic: String, messages: Array[String]): Seq[(String, RecordMetadata)] = { - sendMessages(topic, messages, None) - } - - /** Send the array of messages to the Kafka broker using specified partition */ def sendMessages( topic: String, - messages: Array[String], - partition: Option[Int]): Seq[(String, RecordMetadata)] = { - sendMessages(topic, messages.map(m => (m, Seq())), partition) + msgs: Array[String], + part: Option[Int]): Seq[(String, RecordMetadata)] = { + val records = msgs.map { msg => + val builder = new RecordBuilder(topic, msg) + part.foreach { p => builder.partition(p) } + builder.build() + } + sendMessages(records) } - /** Send record to the Kafka broker with headers using specified partition */ - def sendMessage(topic: String, - record: (String, Seq[(String, Array[Byte])]), - partition: Option[Int]): Seq[(String, RecordMetadata)] = { - sendMessages(topic, Array(record).toSeq, partition) + def sendMessage(msg: ProducerRecord[String, String]): Seq[(String, RecordMetadata)] = { + sendMessages(Array(msg)) } - /** Send the array of records to the Kafka broker with headers using specified partition */ - def sendMessages(topic: String, - records: Seq[(String, Seq[(String, Array[Byte])])], - partition: Option[Int]): Seq[(String, RecordMetadata)] = { + def sendMessages(msgs: Seq[ProducerRecord[String, String]]): Seq[(String, RecordMetadata)] = { producer = new KafkaProducer[String, String](producerConfiguration) val offsets = try { - records.map { case (value, header) => - val headers = header.map { case (k, v) => - new RecordHeader(k, v).asInstanceOf[Header] - } - val record = partition match { - case Some(p) => - new ProducerRecord[String, String](topic, p, null, value, headers.asJava) - case None => - new ProducerRecord[String, String](topic, null, null, value, headers.asJava) - } - val metadata = producer.send(record).get(10, TimeUnit.SECONDS) - logInfo(s"\tSent ($value, $header) to partition ${metadata.partition}," + - " offset ${metadata.offset}") - (value, metadata) + msgs.map { msg => + val metadata = producer.send(msg).get(10, TimeUnit.SECONDS) + logInfo(s"\tSent ($msg) to partition ${metadata.partition}, offset ${metadata.offset}") + (msg.value(), metadata) } } finally { if (producer != null) { @@ -658,4 +634,3 @@ class KafkaTestUtils( } } } - diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/RecordBuilder.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/RecordBuilder.scala new file mode 100644 index 0000000000000..ef07798442e56 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/RecordBuilder.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.sql.kafka010 + +import java.lang.{Integer => JInt, Long => JLong} + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.header.Header +import org.apache.kafka.common.header.internals.RecordHeader + +class RecordBuilder(topic: String, value: String) { + var _partition: Option[JInt] = None + var _timestamp: Option[JLong] = None + var _key: Option[String] = None + var _headers: Option[Seq[(String, Array[Byte])]] = None + + def partition(part: JInt): RecordBuilder = { + _partition = Some(part) + this + } + + def partition(part: Int): RecordBuilder = { + _partition = Some(part.intValue()) + this + } + + def timestamp(ts: JLong): RecordBuilder = { + _timestamp = Some(ts) + this + } + + def timestamp(ts: Long): RecordBuilder = { + _timestamp = Some(ts.longValue()) + this + } + + def key(k: String): RecordBuilder = { + _key = Some(k) + this + } + + def headers(hdrs: Seq[(String, Array[Byte])]): RecordBuilder = { + _headers = Some(hdrs) + this + } + + def build(): ProducerRecord[String, String] = { + val part = _partition.orNull + val ts = _timestamp.orNull + val k = _key.orNull + val hdrs = _headers.map { h => + h.map { case (k, v) => new RecordHeader(k, v).asInstanceOf[Header] } + }.map(_.asJava).orNull + + new ProducerRecord[String, String](topic, part, ts, k, value, hdrs) + } +} From da7e5c4ffb50c3d6c3c6cfe82328e4b8c858016c Mon Sep 17 00:00:00 2001 From: windpiger Date: Tue, 24 Sep 2019 14:48:47 +0800 Subject: [PATCH 0166/1387] [SPARK-19917][SQL] qualified partition path stored in catalog ## What changes were proposed in this pull request? partition path should be qualified to store in catalog. There are some scenes: 1. ALTER TABLE t PARTITION(b=1) SET LOCATION '/path/x' should be qualified: file:/path/x **Hive 2.0.0 does not support for location without schema here.** ``` FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. {0} is not absolute or has no scheme information. Please specify a complete absolute uri with scheme information. ``` 2. ALTER TABLE t PARTITION(b=1) SET LOCATION 'x' should be qualified: file:/tablelocation/x **Hive 2.0.0 does not support for relative location here.** 3. ALTER TABLE t ADD PARTITION(b=1) LOCATION '/path/x' should be qualified: file:/path/x **the same with Hive 2.0.0** 4. ALTER TABLE t ADD PARTITION(b=1) LOCATION 'x' should be qualified: file:/tablelocation/x **the same with Hive 2.0.0** Currently only ALTER TABLE t ADD PARTITION(b=1) LOCATION for hive serde table has the expected qualified path. we should make other scenes to be consist with it. Another change is for alter table location. ## How was this patch tested? add / modify existing TestCases Closes #17254 from windpiger/qualifiedPartitionPath. Authored-by: windpiger Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/SessionCatalog.scala | 35 ++++++++++- .../sql/execution/command/DDLSuite.scala | 59 +++++++++++++++---- .../spark/sql/sources/PathOptionSuite.scala | 3 +- 3 files changed, 83 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 2a6124a4079a1..4d2dee5da383a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -353,9 +353,20 @@ class SessionCatalog( val db = formatDatabaseName(tableDefinition.identifier.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableDefinition.identifier.table) val tableIdentifier = TableIdentifier(table, Some(db)) - val newTableDefinition = tableDefinition.copy(identifier = tableIdentifier) requireDbExists(db) requireTableExists(tableIdentifier) + val newTableDefinition = if (tableDefinition.storage.locationUri.isDefined + && !tableDefinition.storage.locationUri.get.isAbsolute) { + // make the location of the table qualified. + val qualifiedTableLocation = + makeQualifiedPath(tableDefinition.storage.locationUri.get) + tableDefinition.copy( + storage = tableDefinition.storage.copy(locationUri = Some(qualifiedTableLocation)), + identifier = tableIdentifier) + } else { + tableDefinition.copy(identifier = tableIdentifier) + } + externalCatalog.alterTable(newTableDefinition) } @@ -882,7 +893,8 @@ class SessionCatalog( requireTableExists(TableIdentifier(table, Option(db))) requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) requireNonEmptyValueInPartitionSpec(parts.map(_.spec)) - externalCatalog.createPartitions(db, table, parts, ignoreIfExists) + externalCatalog.createPartitions( + db, table, partitionWithQualifiedPath(tableName, parts), ignoreIfExists) } /** @@ -942,7 +954,7 @@ class SessionCatalog( requireTableExists(TableIdentifier(table, Option(db))) requireExactMatchedPartitionSpec(parts.map(_.spec), getTableMetadata(tableName)) requireNonEmptyValueInPartitionSpec(parts.map(_.spec)) - externalCatalog.alterPartitions(db, table, parts) + externalCatalog.alterPartitions(db, table, partitionWithQualifiedPath(tableName, parts)) } /** @@ -1064,6 +1076,23 @@ class SessionCatalog( } } + /** + * Make the partition path qualified. + * If the partition path is relative, e.g. 'paris', it will be qualified with + * parent path using table location, e.g. 'file:/warehouse/table/paris' + */ + private def partitionWithQualifiedPath( + tableIdentifier: TableIdentifier, + parts: Seq[CatalogTablePartition]): Seq[CatalogTablePartition] = { + lazy val tbl = getTableMetadata(tableIdentifier) + parts.map { part => + if (part.storage.locationUri.isDefined && !part.storage.locationUri.get.isAbsolute) { + val partPath = new Path(new Path(tbl.location), new Path(part.storage.locationUri.get)) + val qualifiedPartPath = makeQualifiedPath(CatalogUtils.stringToURI(partPath.toString)) + part.copy(storage = part.storage.copy(locationUri = Some(qualifiedPartPath))) + } else part + } + } // ---------------------------------------------------------------------------- // Functions // ---------------------------------------------------------------------------- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index fd1da2011f28e..ccc1ad76f2b1c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1372,7 +1372,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { // if (isUsingHiveMetastore) { // assert(storageFormat.properties.get("path") === expected) // } - assert(storageFormat.locationUri.map(_.getPath) === Some(expected.getPath)) + assert(storageFormat.locationUri === + Some(makeQualifiedPath(CatalogUtils.URIToString(expected)))) } // set table location sql("ALTER TABLE dbx.tab1 SET LOCATION '/path/to/your/lovely/heart'") @@ -1386,7 +1387,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { verifyLocation(new URI("/swanky/steak/place")) // set table partition location without explicitly specifying database sql("ALTER TABLE tab1 PARTITION (a='1', b='2') SET LOCATION 'vienna'") - verifyLocation(new URI("vienna"), Some(partSpec)) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("tab1")) + val viennaPartPath = new Path(new Path(table. location), "vienna") + verifyLocation(CatalogUtils.stringToURI(viennaPartPath.toString), Some(partSpec)) // table to alter does not exist intercept[AnalysisException] { sql("ALTER TABLE dbx.does_not_exist SET LOCATION '/mister/spark'") @@ -1550,13 +1553,11 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { "PARTITION (a='2', b='6') LOCATION 'paris' PARTITION (a='3', b='7')") assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part2, part3)) assert(catalog.getPartition(tableIdent, part1).storage.locationUri.isDefined) - val partitionLocation = if (isUsingHiveMetastore) { - val tableLocation = catalog.getTableMetadata(tableIdent).storage.locationUri - assert(tableLocation.isDefined) - makeQualifiedPath(new Path(tableLocation.get.toString, "paris").toString) - } else { - new URI("paris") - } + + val tableLocation = catalog.getTableMetadata(tableIdent).storage.locationUri + assert(tableLocation.isDefined) + val partitionLocation = makeQualifiedPath( + new Path(tableLocation.get.toString, "paris").toString) assert(catalog.getPartition(tableIdent, part2).storage.locationUri == Option(partitionLocation)) assert(catalog.getPartition(tableIdent, part3).storage.locationUri.isDefined) @@ -2138,7 +2139,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { spark.sessionState.catalog.refreshTable(TableIdentifier("t")) val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) - assert(table1.location == newDir) + assert(table1.location == makeQualifiedPath(newDir.toString)) assert(!newDirFile.exists) spark.sql("INSERT INTO TABLE t SELECT 'c', 1") @@ -2501,6 +2502,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { assert(table.location.toString.startsWith("file:/")) } + withTempDir { dir => + assert(!dir.getAbsolutePath.startsWith("file:/")) + spark.sql(s"ALTER TABLE t SET LOCATION '$dir'") + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location.toString.startsWith("file:/")) + } + withTempDir { dir => assert(!dir.getAbsolutePath.startsWith("file:/")) // The parser does not recognize the backslashes on Windows as they are. @@ -2519,6 +2527,37 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } + test("the qualified path of a partition is stored in the catalog") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a STRING, b STRING) + |USING ${dataSource} PARTITIONED BY(b) LOCATION '$dir' + """.stripMargin) + spark.sql("INSERT INTO TABLE t PARTITION(b=1) SELECT 2") + val part = spark.sessionState.catalog.getPartition(TableIdentifier("t"), Map("b" -> "1")) + assert(part.storage.locationUri.contains( + makeQualifiedPath(new File(dir, "b=1").getAbsolutePath))) + assert(part.storage.locationUri.get.toString.startsWith("file:/")) + } + withTempDir { dir => + spark.sql(s"ALTER TABLE t PARTITION(b=1) SET LOCATION '$dir'") + + val part = spark.sessionState.catalog.getPartition(TableIdentifier("t"), Map("b" -> "1")) + assert(part.storage.locationUri.contains(makeQualifiedPath(dir.getAbsolutePath))) + assert(part.storage.locationUri.get.toString.startsWith("file:/")) + } + + withTempDir { dir => + spark.sql(s"ALTER TABLE t ADD PARTITION(b=2) LOCATION '$dir'") + val part = spark.sessionState.catalog.getPartition(TableIdentifier("t"), Map("b" -> "2")) + assert(part.storage.locationUri.contains(makeQualifiedPath(dir.getAbsolutePath))) + assert(part.storage.locationUri.get.toString.startsWith("file:/")) + } + } + } + protected def testAddColumn(provider: String): Unit = { withTable("t1") { sql(s"CREATE TABLE t1 (c1 int) USING $provider") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala index 87dce376a09dd..9b26a5659df49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala @@ -123,7 +123,8 @@ class PathOptionSuite extends DataSourceTest with SharedSparkSession { |USING ${classOf[TestOptionsSource].getCanonicalName} |OPTIONS (PATH '/tmp/path')""".stripMargin) sql("ALTER TABLE src SET LOCATION '/tmp/path2'") - assert(getPathOption("src").map(makeQualifiedPath) == Some(makeQualifiedPath("/tmp/path2"))) + assert(getPathOption("src") == + Some(CatalogUtils.URIToString(makeQualifiedPath("/tmp/path2")))) } withTable("src", "src2") { From 7c8596823a7dff5c05e5f61ff19e4b5179d38815 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 23 Sep 2019 23:57:08 -0700 Subject: [PATCH 0167/1387] [SPARK-29218][WEBUI] Increase `Show Additional Metrics` checkbox width in StagePage ### What changes were proposed in this pull request? Modified widths of some checkboxes in StagePage. ### Why are the changes needed? When we increase the font size of the browsers or the default font size is big, the labels of checkbox of `Show Additional Metrics` in `StagePage` are wrapped like as follows. ![before-modified1](https://user-images.githubusercontent.com/4736016/65449180-634c5e80-de75-11e9-9f27-88f4cc1313b7.png) ![before-modified2](https://user-images.githubusercontent.com/4736016/65449182-63e4f500-de75-11e9-96b8-46e92a61f40c.png) ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? Run the following and visit the `Stage Detail` page. Then, increase the font size. ``` $ bin/spark-shell ... scala> spark.range(100000).groupBy("id").count.collect ``` Closes #25905 from sarutak/adjust-checkbox-width. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/ui/static/stagepage.js | 2 +- .../org/apache/spark/ui/static/webui.css | 18 +++++++++++------- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 3ef1a76fd7202..b28c981da20a5 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -286,7 +286,7 @@ $(document).ready(function () { " Show Additional Metrics" + "" + "
    + - Since Spark 3.0, special values are supported in conversion from strings to dates and timestamps. Those values are simply notational shorthands that will be converted to ordinary date or timestamp values when read. The following string values are supported for dates: + - `epoch [zoneId]` - 1970-01-01 + - `today [zoneId]` - the current date in the time zone specified by `spark.sql.session.timeZone` + - `yesterday [zoneId]` - the current date - 1 + - `tomorrow [zoneId]` - the current date + 1 + - `now` - the date of running the current query. It has the same notion as today + For example `SELECT date 'tomorrow' - date 'yesterday';` should output `2`. Here are special timestamp values: + - `epoch [zoneId]` - 1970-01-01 00:00:00+00 (Unix system time zero) + - `today [zoneId]` - midnight today + - `yesterday [zoneId]` - midnight yesterday + - `tomorrow [zoneId]` - midnight tomorrow + - `now` - current query start time + For example `SELECT timestamp 'tomorrow';`. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was From fda0e6e48d00a1ba8e9d41d7670b3ad3c6951492 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 27 Sep 2019 11:04:55 -0700 Subject: [PATCH 0201/1387] [SPARK-29240][PYTHON] Pass Py4J column instance to support PySpark column in element_at function ### What changes were proposed in this pull request? This PR makes `element_at` in PySpark able to take PySpark `Column` instances. ### Why are the changes needed? To match with Scala side. Seems it was intended but not working correctly as a bug. ### Does this PR introduce any user-facing change? Yes. See below: ```python from pyspark.sql import functions as F x = spark.createDataFrame([([1,2,3],1),([4,5,6],2),([7,8,9],3)],['list','num']) x.withColumn('aa',F.element_at('list',x.num.cast('int'))).show() ``` Before: ``` Traceback (most recent call last): File "", line 1, in File "/.../spark/python/pyspark/sql/functions.py", line 2059, in element_at return Column(sc._jvm.functions.element_at(_to_java_column(col), extraction)) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1277, in __call__ File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1241, in _build_args File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1228, in _get_args File "/.../forked/spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_collections.py", line 500, in convert File "/.../spark/python/pyspark/sql/column.py", line 344, in __iter__ raise TypeError("Column is not iterable") TypeError: Column is not iterable ``` After: ``` +---------+---+---+ | list|num| aa| +---------+---+---+ |[1, 2, 3]| 1| 1| |[4, 5, 6]| 2| 5| |[7, 8, 9]| 3| 9| +---------+---+---+ ``` ### How was this patch tested? Manually tested against literal, Python native types, and PySpark column. Closes #25950 from HyukjinKwon/SPARK-29240. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/functions.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index deebe0efcc335..95b78175d5561 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2052,11 +2052,12 @@ def element_at(col, extraction): [Row(element_at(data, 1)=u'a'), Row(element_at(data, 1)=None)] >>> df = spark.createDataFrame([({"a": 1.0, "b": 2.0},), ({},)], ['data']) - >>> df.select(element_at(df.data, "a")).collect() + >>> df.select(element_at(df.data, lit("a"))).collect() [Row(element_at(data, a)=1.0), Row(element_at(data, a)=None)] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.element_at(_to_java_column(col), extraction)) + return Column(sc._jvm.functions.element_at( + _to_java_column(col), lit(extraction)._jc)) # noqa: F821 'lit' is dynamically defined. @since(2.4) From 420abb457df0f422f73bab19a6ed6d7c6bab3173 Mon Sep 17 00:00:00 2001 From: Juliusz Sompolski Date: Fri, 27 Sep 2019 11:18:32 -0700 Subject: [PATCH 0202/1387] [SPARK-29263][SCHEDULER] Update `availableSlots` in `resourceOffers()` before checking available slots for barrier taskSet ### What changes were proposed in this pull request? availableSlots are computed before the for loop looping over all TaskSets in resourceOffers. But the number of slots changes in every iteration, as in every iteration these slots are taken. The number of available slots checked by a barrier task set has therefore to be recomputed in every iteration from availableCpus. ### Why are the changes needed? Bugfix. This could make resourceOffer attempt to start a barrier task set, even though it has not enough slots available. That would then be caught by the `require` in line 519, which will throw an exception, which will get caught and ignored by Dispatcher's MessageLoop, so nothing terrible would happen, but the exception would prevent resourceOffers from considering further TaskSets. Note that launching the barrier TaskSet can still fail if other requirements are not satisfied, and still can be rolled-back by throwing exception in this `require`. Handling it more gracefully remains a TODO in SPARK-24818, but this fix at least should resolve the situation when it's unable to launch because of insufficient slots. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added UT Closes #23375 Closes #25946 from juliuszsompolski/SPARK-29263. Authored-by: Juliusz Sompolski Signed-off-by: Xingbo Jiang --- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../org/apache/spark/scheduler/FakeTask.scala | 36 +++++++++---- .../scheduler/TaskSchedulerImplSuite.scala | 51 ++++++++++++++----- 3 files changed, 65 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 1496dff31a4dc..016c56cc34639 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -430,7 +430,6 @@ private[spark] class TaskSchedulerImpl( val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK)) val availableResources = shuffledOffers.map(_.resources).toArray val availableCpus = shuffledOffers.map(o => o.cores).toArray - val availableSlots = shuffledOffers.map(o => o.cores / CPUS_PER_TASK).sum val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( @@ -444,6 +443,7 @@ private[spark] class TaskSchedulerImpl( // of locality levels so that it gets a chance to launch local tasks on all of them. // NOTE: the preferredLocality order: PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY for (taskSet <- sortedTaskSets) { + val availableSlots = availableCpus.map(c => c / CPUS_PER_TASK).sum // Skip the barrier taskSet if the available slots are less than the number of pending tasks. if (taskSet.isBarrier && availableSlots < taskSet.numTasks) { // Skip the launch process. diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index b29d32f7b35c5..abc8841ac03da 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -42,15 +42,23 @@ object FakeTask { * locations for each task (given as varargs) if this sequence is not empty. */ def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, stageAttemptId = 0, prefLocs: _*) + createTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, prefLocs: _*) } - def createTaskSet(numTasks: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, stageId = 0, stageAttemptId, prefLocs: _*) + def createTaskSet( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + prefLocs: Seq[TaskLocation]*): TaskSet = { + createTaskSet(numTasks, stageId, stageAttemptId, priority = 0, prefLocs: _*) } - def createTaskSet(numTasks: Int, stageId: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): - TaskSet = { + def createTaskSet( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + priority: Int, + prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") } @@ -65,6 +73,15 @@ object FakeTask { stageId: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { + createShuffleMapTaskSet(numTasks, stageId, stageAttemptId, priority = 0, prefLocs: _*) + } + + def createShuffleMapTaskSet( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + priority: Int, + prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") } @@ -74,17 +91,18 @@ object FakeTask { }, prefLocs(i), new Properties, SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array()) } - new TaskSet(tasks, stageId, stageAttemptId, priority = 0, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null) } def createBarrierTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createBarrierTaskSet(numTasks, stageId = 0, stageAttempId = 0, prefLocs: _*) + createBarrierTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, prefLocs: _*) } def createBarrierTaskSet( numTasks: Int, stageId: Int, - stageAttempId: Int, + stageAttemptId: Int, + priority: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") @@ -92,6 +110,6 @@ object FakeTask { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil, isBarrier = true) } - new TaskSet(tasks, stageId, stageAttempId, priority = 0, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index cac6285e58417..3bff6c145ed51 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -228,19 +228,19 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B taskScheduler.taskSetManagerForAttempt(taskset.stageId, taskset.stageAttemptId).get.isZombie } - val attempt1 = FakeTask.createTaskSet(1, 0) + val attempt1 = FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 0) taskScheduler.submitTasks(attempt1) // The first submitted taskset is active assert(!isTasksetZombie(attempt1)) - val attempt2 = FakeTask.createTaskSet(1, 1) + val attempt2 = FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 1) taskScheduler.submitTasks(attempt2) // The first submitted taskset is zombie now assert(isTasksetZombie(attempt1)) // The newly submitted taskset is active assert(!isTasksetZombie(attempt2)) - val attempt3 = FakeTask.createTaskSet(1, 2) + val attempt3 = FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 2) taskScheduler.submitTasks(attempt3) // The first submitted taskset remains zombie assert(isTasksetZombie(attempt1)) @@ -255,7 +255,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val numFreeCores = 1 val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores)) - val attempt1 = FakeTask.createTaskSet(10) + val attempt1 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 0) // submit attempt 1, offer some resources, some tasks get scheduled taskScheduler.submitTasks(attempt1) @@ -271,7 +271,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions2.length) // if we schedule another attempt for the same stage, it should get scheduled - val attempt2 = FakeTask.createTaskSet(10, 1) + val attempt2 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 1) // submit attempt 2, offer some resources, some tasks get scheduled taskScheduler.submitTasks(attempt2) @@ -287,7 +287,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B val numFreeCores = 10 val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores)) - val attempt1 = FakeTask.createTaskSet(10) + val attempt1 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 0) // submit attempt 1, offer some resources, some tasks get scheduled taskScheduler.submitTasks(attempt1) @@ -303,7 +303,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions2.length) // submit attempt 2 - val attempt2 = FakeTask.createTaskSet(10, 1) + val attempt2 = FakeTask.createTaskSet(10, stageId = 0, stageAttemptId = 1) taskScheduler.submitTasks(attempt2) // attempt 1 finished (this can happen even if it was marked zombie earlier -- all tasks were @@ -497,7 +497,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("abort stage when all executors are blacklisted and we cannot acquire new executor") { taskScheduler = setupSchedulerWithMockTaskSetBlacklist() - val taskSet = FakeTask.createTaskSet(numTasks = 10, stageAttemptId = 0) + val taskSet = FakeTask.createTaskSet(numTasks = 10) taskScheduler.submitTasks(taskSet) val tsm = stageToMockTaskSetManager(0) @@ -539,7 +539,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0") // We have only 1 task remaining with 1 executor - val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0) + val taskSet = FakeTask.createTaskSet(numTasks = 1) taskScheduler.submitTasks(taskSet) val tsm = stageToMockTaskSetManager(0) @@ -571,7 +571,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10") // We have only 1 task remaining with 1 executor - val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0) + val taskSet = FakeTask.createTaskSet(numTasks = 1) taskScheduler.submitTasks(taskSet) val tsm = stageToMockTaskSetManager(0) @@ -910,7 +910,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("SPARK-16106 locality levels updated if executor added to existing host") { val taskScheduler = setupScheduler() - taskScheduler.submitTasks(FakeTask.createTaskSet(2, 0, + taskScheduler.submitTasks(FakeTask.createTaskSet(2, stageId = 0, stageAttemptId = 0, (0 until 2).map { _ => Seq(TaskLocation("host0", "executor2")) }: _* )) @@ -948,7 +948,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("scheduler checks for executors that can be expired from blacklist") { taskScheduler = setupScheduler() - taskScheduler.submitTasks(FakeTask.createTaskSet(1, 0)) + taskScheduler.submitTasks(FakeTask.createTaskSet(1, stageId = 0, stageAttemptId = 0)) taskScheduler.resourceOffers(IndexedSeq( new WorkerOffer("executor0", "host0", 1) )).flatten @@ -1154,6 +1154,29 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(3 === taskDescriptions.length) } + test("SPARK-29263: barrier TaskSet can't schedule when higher prio taskset takes the slots") { + val taskCpus = 2 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", + config.CPUS_PER_TASK.key -> taskCpus.toString) + + val numFreeCores = 3 + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625")), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627")), + new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629"))) + val barrier = FakeTask.createBarrierTaskSet(3, stageId = 0, stageAttemptId = 0, priority = 1) + val highPrio = FakeTask.createTaskSet(1, stageId = 1, stageAttemptId = 0, priority = 0) + + // submit highPrio and barrier taskSet + taskScheduler.submitTasks(highPrio) + taskScheduler.submitTasks(barrier) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + // it schedules the highPrio task first, and then will not have enough slots to schedule + // the barrier taskset + assert(1 === taskDescriptions.length) + } + test("cancelTasks shall kill all the running tasks and fail the stage") { val taskScheduler = setupScheduler() @@ -1169,7 +1192,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } }) - val attempt1 = FakeTask.createTaskSet(10, 0) + val attempt1 = FakeTask.createTaskSet(10) taskScheduler.submitTasks(attempt1) val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 1), @@ -1200,7 +1223,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } }) - val attempt1 = FakeTask.createTaskSet(10, 0) + val attempt1 = FakeTask.createTaskSet(10) taskScheduler.submitTasks(attempt1) val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 1), From 233c214a752771f5d8ca9fb2aea93cf1776a552d Mon Sep 17 00:00:00 2001 From: Jeff Evans Date: Fri, 27 Sep 2019 11:32:22 -0700 Subject: [PATCH 0203/1387] [SPARK-29070][CORE] Make SparkLauncher log full spark-submit command line Log the full spark-submit command in SparkSubmit#launchApplication Adding .python-version (pyenv file) to RAT exclusion list ### What changes were proposed in this pull request? Original motivation [here](http://apache-spark-user-list.1001560.n3.nabble.com/Is-it-possible-to-obtain-the-full-command-to-be-invoked-by-SparkLauncher-td35144.html), expanded in the [Jira](https://issues.apache.org/jira/browse/SPARK-29070).. In essence, we want to be able to log the full `spark-submit` command being constructed by `SparkLauncher` ### Why are the changes needed? Currently, it is not possible to directly obtain this information from the `SparkLauncher` instance, which makes debugging and customer support more difficult. ### Does this PR introduce any user-facing change? No ### How was this patch tested? `core` `sbt` tests were executed. The `SparkLauncherSuite` (where I added assertions to an existing test) was also checked. Within that, `testSparkLauncherGetError` is failing, but that appears not to have been caused by this change (failing for me even on the parent commit of c18f849d7667012597fe747448845d80a367b1bb). Closes #25777 from jeff303/SPARK-29070. Authored-by: Jeff Evans Signed-off-by: Marcelo Vanzin --- .../java/org/apache/spark/launcher/SparkLauncher.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index 84940d96b563f..32844104c1deb 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -26,8 +26,11 @@ import java.util.Map; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; +import java.util.logging.Level; +import java.util.logging.Logger; import static org.apache.spark.launcher.CommandBuilderUtils.*; +import static org.apache.spark.launcher.CommandBuilderUtils.join; /** * Launcher for Spark applications. @@ -38,6 +41,8 @@ */ public class SparkLauncher extends AbstractLauncher { + private static final Logger LOG = Logger.getLogger(SparkLauncher.class.getName()); + /** The Spark master. */ public static final String SPARK_MASTER = "spark.master"; @@ -363,6 +368,9 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr String loggerName = getLoggerName(); ProcessBuilder pb = createBuilder(); + if (LOG.isLoggable(Level.FINE)) { + LOG.fine(String.format("Launching Spark application:%n%s", join(" ", pb.command()))); + } boolean outputToLog = outputStream == null; boolean errorToLog = !redirectErrorStream && errorStream == null; From d72f39897b00d0bbd7a4db9de281a1256fcf908d Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 27 Sep 2019 12:35:26 -0700 Subject: [PATCH 0204/1387] [SPARK-27254][SS] Cleanup complete but invalid output files in ManifestFileCommitProtocol if job is aborted ## What changes were proposed in this pull request? SPARK-27210 enables ManifestFileCommitProtocol to clean up incomplete output files in task level if task is aborted. This patch extends the area of cleaning up, proposes ManifestFileCommitProtocol to clean up complete but invalid output files in job level if job aborts. Please note that this works as 'best-effort', not kind of guarantee, as we have in HadoopMapReduceCommitProtocol. ## How was this patch tested? Added UT. Closes #24186 from HeartSaVioR/SPARK-27254. Lead-authored-by: Jungtaek Lim (HeartSaVioR) Co-authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Shixiong Zhu --- .../ManifestFileCommitProtocol.scala | 37 +++++++++- .../sql/streaming/FileStreamSinkSuite.scala | 74 +++++++++++++++++++ 2 files changed, 109 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index 916bd2ddbc818..f6cc8116c6c4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.streaming +import java.io.IOException import java.util.UUID import scala.collection.mutable.ArrayBuffer @@ -43,6 +44,8 @@ class ManifestFileCommitProtocol(jobId: String, path: String) @transient private var fileLog: FileStreamSinkLog = _ private var batchId: Long = _ + @transient private var pendingCommitFiles: ArrayBuffer[Path] = _ + /** * Sets up the manifest log output and the batch id for this job. * Must be called before any other function. @@ -54,13 +57,21 @@ class ManifestFileCommitProtocol(jobId: String, path: String) override def setupJob(jobContext: JobContext): Unit = { require(fileLog != null, "setupManifestOptions must be called before this function") - // Do nothing + pendingCommitFiles = new ArrayBuffer[Path] } override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { require(fileLog != null, "setupManifestOptions must be called before this function") val fileStatuses = taskCommits.flatMap(_.obj.asInstanceOf[Seq[SinkFileStatus]]).toArray + // We shouldn't remove the files if they're written to the metadata: + // `fileLog.add(batchId, fileStatuses)` could fail AFTER writing files to the metadata + // as well as there could be race + // so for the safety we clean up the list before calling anything incurs exception. + // The case is uncommon and we do best effort instead of guarantee, so the simplicity of + // logic here would be OK, and safe for dealing with unexpected situations. + pendingCommitFiles.clear() + if (fileLog.add(batchId, fileStatuses)) { logInfo(s"Committed batch $batchId") } else { @@ -70,7 +81,29 @@ class ManifestFileCommitProtocol(jobId: String, path: String) override def abortJob(jobContext: JobContext): Unit = { require(fileLog != null, "setupManifestOptions must be called before this function") - // Do nothing + // Best effort cleanup of complete files from failed job. + // Since the file has UUID in its filename, we are safe to try deleting them + // as the file will not conflict with file with another attempt on the same task. + if (pendingCommitFiles.nonEmpty) { + pendingCommitFiles.foreach { path => + try { + val fs = path.getFileSystem(jobContext.getConfiguration) + // this is to make sure the file can be seen from driver as well + if (fs.exists(path)) { + fs.delete(path, false) + } + } catch { + case e: IOException => + logWarning(s"Fail to remove temporary file $path, continue removing next.", e) + } + } + pendingCommitFiles.clear() + } + } + + override def onTaskCommit(taskCommit: TaskCommitMessage): Unit = { + pendingCommitFiles ++= taskCommit.obj.asInstanceOf[Seq[SinkFileStatus]] + .map(_.toFileStatus.getPath) } override def setupTask(taskContext: TaskAttemptContext): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 7d343bb58ea3f..edeb416be4d33 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -22,10 +22,13 @@ import java.nio.file.Files import java.util.Locale import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.JobContext import org.apache.spark.SparkConf +import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{AnalysisException, DataFrame} import org.apache.spark.sql.execution.DataSourceScanExec @@ -473,6 +476,77 @@ abstract class FileStreamSinkSuite extends StreamTest { assert(outputFiles.toList.isEmpty, "Incomplete files should be cleaned up.") } } + + testQuietly("cleanup complete but invalid output for aborted job") { + withSQLConf(("spark.sql.streaming.commitProtocolClass", + classOf[PendingCommitFilesTrackingManifestFileCommitProtocol].getCanonicalName)) { + withTempDir { tempDir => + val checkpointDir = new File(tempDir, "chk") + val outputDir = new File(tempDir, "output @#output") + val inputData = MemoryStream[Int] + inputData.addData(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + val q = inputData.toDS() + .repartition(10) + .map { value => + // we intend task failure after some tasks succeeds + if (value == 5) { + // put some delay to let other task commits before this task fails + Thread.sleep(100) + value / 0 + } else { + value + } + } + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format("parquet") + .start(outputDir.getCanonicalPath) + + intercept[StreamingQueryException] { + try { + q.processAllAvailable() + } finally { + q.stop() + } + } + + import PendingCommitFilesTrackingManifestFileCommitProtocol._ + val outputFileNames = Files.walk(outputDir.toPath).iterator().asScala + .filter(_.toString.endsWith(".parquet")) + .map(_.getFileName.toString) + .toSet + val trackingFileNames = tracking.map(new Path(_).getName).toSet + + // there would be possible to have race condition: + // - some tasks complete while abortJob is being called + // we can't delete complete files for these tasks (it's OK since this is a best effort) + assert(outputFileNames.intersect(trackingFileNames).isEmpty, + "abortJob should clean up files reported as successful.") + } + } + } +} + +object PendingCommitFilesTrackingManifestFileCommitProtocol { + val tracking: ArrayBuffer[String] = new ArrayBuffer[String]() + + def cleanPendingCommitFiles(): Unit = tracking.clear() + def addPendingCommitFiles(paths: Seq[String]): Unit = tracking ++= paths +} + +class PendingCommitFilesTrackingManifestFileCommitProtocol(jobId: String, path: String) + extends ManifestFileCommitProtocol(jobId, path) { + import PendingCommitFilesTrackingManifestFileCommitProtocol._ + + override def setupJob(jobContext: JobContext): Unit = { + super.setupJob(jobContext) + cleanPendingCommitFiles() + } + + override def onTaskCommit(taskCommit: FileCommitProtocol.TaskCommitMessage): Unit = { + super.onTaskCommit(taskCommit) + addPendingCommitFiles(taskCommit.obj.asInstanceOf[Seq[SinkFileStatus]].map(_.path)) + } } class FileStreamSinkV1Suite extends FileStreamSinkSuite { From ece42131765de2c79a841672ecd9a848e34bc55e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 29 Sep 2019 02:57:55 +0900 Subject: [PATCH 0205/1387] [SPARK-21914][FOLLOWUP][TEST-HADOOP3.2][TEST-JAVA11] Clone SparkSession per each function example ### What changes were proposed in this pull request? In the PR, I propose to clone Spark session per-each expression example. Examples can modify SQL settings, and can influence on each other if they run in the same Spark session in parallel. ### Why are the changes needed? This should fix test failures like [this](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-3.2-jdk-11/478/testReport/junit/org.apache.spark.sql/SQLQuerySuite/check_outputs_of_expression_examples/) checking of the `Like` example: ``` org.apache.spark.sql.AnalysisException: the pattern '\%SystemDrive\%\Users%' is invalid, the escape character is not allowed to precede 'U'; at org.apache.spark.sql.catalyst.util.StringUtils$.fail$1(StringUtils.scala:48) at org.apache.spark.sql.catalyst.util.StringUtils$.escapeLikeRegex(StringUtils.scala:57) at org.apache.spark.sql.catalyst.expressions.Like.escape(regexpExpressions.scala:108) ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `check outputs of expression examples` in `org.apache.spark.sql.SQLQuerySuite` Closes #25956 from MaxGekk/fix-expr-examples-checks. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 4a5201735fbda..46ef7943739ca 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 @@ -169,7 +169,9 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { spark.sessionState.functionRegistry.listFunction().par.foreach { funcId => - val info = spark.sessionState.catalog.lookupFunctionInfo(funcId) + // Examples can change settings. We clone the session to prevent tests clashing. + val clonedSpark = spark.cloneSession() + val info = clonedSpark.sessionState.catalog.lookupFunctionInfo(funcId) val className = info.getClassName if (!ignoreSet.contains(className)) { withClue(s"Function '${info.getName}', Expression class '$className'") { @@ -177,7 +179,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { checkExampleSyntax(example) example.split(" > ").toList.foreach(_ match { case exampleRe(sql, output) => - val df = spark.sql(sql) + val df = clonedSpark.sql(sql) val actual = unindentAndTrim( hiveResultString(df.queryExecution.executedPlan).mkString("\n")) val expected = unindentAndTrim(output) From 94946e4836503abb340831e476283cba47322b91 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sun, 29 Sep 2019 03:05:49 +0900 Subject: [PATCH 0206/1387] [SPARK-29281][SQL] Correct example of Like/RLike to test the origin intention correctly ### What changes were proposed in this pull request? This patch fixes examples of Like/RLike to test its origin intention correctly. The example doesn't consider the default value of spark.sql.parser.escapedStringLiterals: it's false by default. Please take a look at current example of Like: https://github.com/apache/spark/blob/d72f39897b00d0bbd7a4db9de281a1256fcf908d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala#L97-L106 If spark.sql.parser.escapedStringLiterals=false, then it should fail as there's `\U` in pattern (spark.sql.parser.escapedStringLiterals=false by default) but it doesn't fail. ``` The escape character is '\'. If an escape character precedes a special symbol or another escape character, the following character is matched literally. It is invalid to escape any other character. ``` For the query ``` SET spark.sql.parser.escapedStringLiterals=false; SELECT '%SystemDrive%\Users\John' like '\%SystemDrive\%\Users%'; ``` SQL parser removes single `\` (not sure that is intended) so the expressions of Like are constructed as following (I've printed out expression of left and right for Like/RLike): > LIKE - left `%SystemDrive%UsersJohn` / right `\%SystemDrive\%Users%` which are no longer having origin intention (see left). Below query tests the origin intention: ``` SET spark.sql.parser.escapedStringLiterals=false; SELECT '%SystemDrive%\\Users\\John' like '\%SystemDrive\%\\\\Users%'; ``` > LIKE - left `%SystemDrive%\Users\John` / right `\%SystemDrive\%\\Users%` Note that `\\\\` is needed in pattern as `StringUtils.escapeLikeRegex` requires `\\` to represent normal character of `\`. Same for RLIKE: ``` SET spark.sql.parser.escapedStringLiterals=true; SELECT '%SystemDrive%\Users\John' rlike '%SystemDrive%\\Users.*'; ``` > RLIKE - left `%SystemDrive%\Users\John` / right `%SystemDrive%\\Users.*` which is OK, but ``` SET spark.sql.parser.escapedStringLiterals=false; SELECT '%SystemDrive%\Users\John' rlike '%SystemDrive%\Users.*'; ``` > RLIKE - left `%SystemDrive%UsersJohn` / right `%SystemDrive%Users.*` which no longer haves origin intention. Below query tests the origin intention: ``` SET spark.sql.parser.escapedStringLiterals=true; SELECT '%SystemDrive%\\Users\\John' rlike '%SystemDrive%\\\\Users.*'; ``` > RLIKE - left `%SystemDrive%\Users\John` / right `%SystemDrive%\\Users.*` ### Why are the changes needed? Because the example doesn't test the origin intention. Spark is now running automated tests from these examples, so now it's not only documentation issue but also test issue. ### Does this PR introduce any user-facing change? No, as it only corrects documentation. ### How was this patch tested? Added debug log (like above) and ran queries from `spark-sql`. Closes #25957 from HeartSaVioR/SPARK-29281. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../catalyst/expressions/regexpExpressions.scala | 13 ++++++++++--- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 1 + 2 files changed, 11 insertions(+), 3 deletions(-) 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 79fe05f8087a1..f52e903e75094 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 @@ -65,7 +65,7 @@ abstract class StringRegexExpression extends BinaryExpression override def sql: String = s"${left.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}" } - +// scalastyle:off line.contains.tab /** * Simple RegEx pattern matching function */ @@ -96,13 +96,20 @@ abstract class StringRegexExpression extends BinaryExpression """, examples = """ Examples: - > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\Users%'; + > SET spark.sql.parser.escapedStringLiterals=true; + spark.sql.parser.escapedStringLiterals true + > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\Users%'; + true + > SET spark.sql.parser.escapedStringLiterals=false; + spark.sql.parser.escapedStringLiterals false + > SELECT '%SystemDrive%\\Users\\John' _FUNC_ '\%SystemDrive\%\\\\Users%'; true """, note = """ Use RLIKE to match with standard regular expressions. """, since = "1.0.0") +// scalastyle:on line.contains.tab case class Like(left: Expression, right: Expression) extends StringRegexExpression { override def escape(v: String): String = StringUtils.escapeLikeRegex(v) @@ -177,7 +184,7 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi true > SET spark.sql.parser.escapedStringLiterals=false; spark.sql.parser.escapedStringLiterals false - > SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\Users.*'; + > SELECT '%SystemDrive%\\Users\\John' _FUNC_ '%SystemDrive%\\\\Users.*'; true """, note = """ 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 46ef7943739ca..f9fd5f64f036e 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 @@ -130,6 +130,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { // Examples demonstrate alternative names, see SPARK-20749 "org.apache.spark.sql.catalyst.expressions.Length", // Uses settings without _FUNC_ in `SET spark.sql.parser.escapedStringLiterals=true` + "org.apache.spark.sql.catalyst.expressions.Like", "org.apache.spark.sql.catalyst.expressions.RLike") spark.sessionState.functionRegistry.listFunction().foreach { funcId => val info = spark.sessionState.catalog.lookupFunctionInfo(funcId) From 2409320d8f542d3f32846e2bd0e23bd4dfba3113 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 29 Sep 2019 08:51:47 +0900 Subject: [PATCH 0207/1387] [SPARK-29237][SQL][FOLLOWUP] Ignore `SET` commands in expression examples while checking the _FUNC_ pattern ### What changes were proposed in this pull request? The `SET` commands do not contain the `_FUNC_` pattern a priori. In the PR, I propose filter out such commands in the `using _FUNC_ instead of function names in examples` test. ### Why are the changes needed? After the merge of https://github.com/apache/spark/pull/25942, examples will require particular settings. Currently, the whole expression example has to be ignored which is so much. It makes sense to ignore only `SET` commands in expression examples. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the `using _FUNC_ instead of function names in examples` test. Closes #25958 from MaxGekk/dont-check-_FUNC_-in-set. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 f9fd5f64f036e..0a9408ff6d5ea 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 @@ -119,6 +119,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { test("using _FUNC_ instead of function names in examples") { val exampleRe = "(>.*;)".r + val setStmtRe = "(?i)^(>\\s+set\\s+).+".r val ignoreSet = Set( // Examples for CaseWhen show simpler syntax: // `CASE WHEN ... THEN ... WHEN ... THEN ... END` @@ -128,17 +129,16 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { // _FUNC_ is replaced by `%` which causes a parsing error on `SELECT %(2, 1.8)` "org.apache.spark.sql.catalyst.expressions.Remainder", // Examples demonstrate alternative names, see SPARK-20749 - "org.apache.spark.sql.catalyst.expressions.Length", - // Uses settings without _FUNC_ in `SET spark.sql.parser.escapedStringLiterals=true` - "org.apache.spark.sql.catalyst.expressions.Like", - "org.apache.spark.sql.catalyst.expressions.RLike") + "org.apache.spark.sql.catalyst.expressions.Length") spark.sessionState.functionRegistry.listFunction().foreach { funcId => val info = spark.sessionState.catalog.lookupFunctionInfo(funcId) val className = info.getClassName withClue(s"Expression class '$className'") { val exprExamples = info.getOriginalExamples if (!exprExamples.isEmpty && !ignoreSet.contains(className)) { - assert(exampleRe.findAllIn(exprExamples).toSet.forall(_.contains("_FUNC_"))) + assert(exampleRe.findAllIn(exprExamples).toIterable + .filter(setStmtRe.findFirstIn(_).isEmpty) // Ignore SET commands + .forall(_.contains("_FUNC_"))) } } } From 8167714cab93a5c06c23f92c9077fe8b9677ab28 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 28 Sep 2019 16:55:49 -0700 Subject: [PATCH 0208/1387] [SPARK-27831][FOLLOW-UP][SQL][TEST] Should not use maven to add Hive test jars ### What changes were proposed in this pull request? This PR moves Hive test jars(`hive-contrib-*.jar` and `hive-hcatalog-core-*.jar`) from maven dependency to local file. ### Why are the changes needed? `--jars` can't be tested since `hive-contrib-*.jar` and `hive-hcatalog-core-*.jar` are already in classpath. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test Closes #25690 from wangyum/SPARK-27831-revert. Authored-by: Yuming Wang Signed-off-by: Yuming Wang --- pom.xml | 69 ------------------- sql/hive-thriftserver/pom.xml | 9 --- .../sql/hive/thriftserver/CliSuite.scala | 12 ++-- .../HiveThriftServer2Suites.scala | 4 +- sql/hive/pom.xml | 8 --- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 14 ++-- .../sql/hive/execution/HiveQuerySuite.scala | 8 +-- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- .../spark/sql/hive/test/HiveTestUtils.scala | 32 --------- .../apache/spark/sql/hive/test/TestHive.scala | 21 ++++++ 10 files changed, 42 insertions(+), 139 deletions(-) delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/test/HiveTestUtils.scala diff --git a/pom.xml b/pom.xml index 2ea699371b44f..e24e32f213de5 100644 --- a/pom.xml +++ b/pom.xml @@ -1996,75 +1996,6 @@ - - ${hive.group} - hive-contrib - ${hive.version} - test - - - ${hive.group} - hive-exec - - - ${hive.group} - hive-serde - - - ${hive.group} - hive-shims - - - commons-codec - commons-codec - - - org.slf4j - slf4j-api - - - - - ${hive.group}.hcatalog - hive-hcatalog-core - ${hive.version} - test - - - ${hive.group} - hive-exec - - - ${hive.group} - hive-metastore - - - ${hive.group} - hive-cli - - - ${hive.group} - hive-common - - - com.google.guava - guava - - - org.slf4j - slf4j-api - - - org.codehaus.jackson - jackson-mapper-asl - - - org.apache.hadoop - * - - - - org.apache.orc orc-core diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 5b1352adddd89..9dd927084298a 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -77,15 +77,6 @@ ${hive.group} hive-beeline - - - ${hive.group} - hive-contrib - - - ${hive.group}.hcatalog - hive-hcatalog-core - org.eclipse.jetty jetty-server diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 6d45041e12821..db33d1d4a07dd 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -27,12 +27,11 @@ import scala.concurrent.Promise import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.contrib.udf.example.UDFExampleFormat import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.test.HiveTestUtils +import org.apache.spark.sql.hive.test.HiveTestJars import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} @@ -202,7 +201,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { } test("Commands using SerDe provided in --jars") { - val jarFile = HiveTestUtils.getHiveHcatalogCoreJar.getCanonicalPath + val jarFile = HiveTestJars.getHiveHcatalogCoreJar().getCanonicalPath val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") @@ -297,11 +296,12 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { } test("Support hive.aux.jars.path") { - val hiveContribJar = HiveTestUtils.getHiveContribJar.getCanonicalPath + val hiveContribJar = HiveTestJars.getHiveContribJar().getCanonicalPath runCliWithin( 1.minute, Seq("--conf", s"spark.hadoop.${ConfVars.HIVEAUXJARS}=$hiveContribJar"))( - s"CREATE TEMPORARY FUNCTION example_format AS '${classOf[UDFExampleFormat].getName}';" -> "", + "CREATE TEMPORARY FUNCTION example_format AS " + + "'org.apache.hadoop.hive.contrib.udf.example.UDFExampleFormat';" -> "", "SELECT example_format('%o', 93);" -> "135" ) } @@ -319,7 +319,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { test("SPARK-28840 test --jars and hive.aux.jars.path command") { val jarFile = new File("../../sql/hive/src/test/resources/SPARK-21101-1.0.jar").getCanonicalPath - val hiveContribJar = HiveTestUtils.getHiveContribJar.getCanonicalPath + val hiveContribJar = HiveTestJars.getHiveContribJar().getCanonicalPath runCliWithin( 1.minute, Seq("--jars", s"$jarFile", "--conf", diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 3fef23f8eed3e..bc62cfaf62272 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -43,7 +43,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.HiveUtils -import org.apache.spark.sql.hive.test.HiveTestUtils +import org.apache.spark.sql.hive.test.HiveTestJars import org.apache.spark.sql.internal.StaticSQLConf.HIVE_THRIFT_SERVER_SINGLESESSION import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} @@ -492,7 +492,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { withMultipleConnectionJdbcStatement("smallKV", "addJar")( { statement => - val jarFile = HiveTestUtils.getHiveHcatalogCoreJar.getCanonicalPath + val jarFile = HiveTestJars.getHiveHcatalogCoreJar().getCanonicalPath statement.executeQuery(s"ADD JAR $jarFile") }, diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d37f0c8573659..f627227aa0380 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -103,14 +103,6 @@ ${hive.group} hive-metastore - - ${hive.group} - hive-contrib - - - ${hive.group}.hcatalog - hive-hcatalog-core - Yes, new TaskContext function to access from Java ### How was this patch tested? new unit test Closes #26083 from tgravescs/SPARK-29417. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Xiangrui Meng --- .../main/scala/org/apache/spark/BarrierTaskContext.scala | 5 +++++ core/src/main/scala/org/apache/spark/TaskContext.scala | 8 ++++++++ .../src/main/scala/org/apache/spark/TaskContextImpl.scala | 5 +++++ .../org/apache/spark/JavaTaskContextCompileCheck.java | 5 +++++ project/MimaExcludes.scala | 3 +++ 5 files changed, 26 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index 5afd8a5d866b2..3d369802f3023 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.util.{Properties, Timer, TimerTask} +import scala.collection.JavaConverters._ import scala.concurrent.TimeoutException import scala.concurrent.duration._ @@ -211,6 +212,10 @@ class BarrierTaskContext private[spark] ( override def resources(): Map[String, ResourceInformation] = taskContext.resources() + override def resourcesJMap(): java.util.Map[String, ResourceInformation] = { + resources().asJava + } + override private[spark] def killTaskIfInterrupted(): Unit = taskContext.killTaskIfInterrupted() override private[spark] def getKillReason(): Option[String] = taskContext.getKillReason() diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 2299c54e2624b..fd41facf95c76 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -185,6 +185,14 @@ abstract class TaskContext extends Serializable { @Evolving def resources(): Map[String, ResourceInformation] + /** + * (java-specific) Resources allocated to the task. The key is the resource name and the value + * is information about the resource. Please refer to + * [[org.apache.spark.resource.ResourceInformation]] for specifics. + */ + @Evolving + def resourcesJMap(): java.util.Map[String, ResourceInformation] + @DeveloperApi def taskMetrics(): TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 516fb95593324..08a58a029528b 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -20,6 +20,7 @@ package org.apache.spark import java.util.Properties import javax.annotation.concurrent.GuardedBy +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.executor.TaskMetrics @@ -101,6 +102,10 @@ private[spark] class TaskContextImpl( this } + override def resourcesJMap(): java.util.Map[String, ResourceInformation] = { + resources.asJava + } + @GuardedBy("this") private[spark] override def markTaskFailed(error: Throwable): Unit = synchronized { if (failed) return diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java index 62a0b85915efc..5ce7937c03de2 100644 --- a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java +++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java @@ -17,7 +17,10 @@ package test.org.apache.spark; +import java.util.Map; + import org.apache.spark.TaskContext; +import org.apache.spark.resource.ResourceInformation; import org.apache.spark.util.TaskCompletionListener; import org.apache.spark.util.TaskFailureListener; @@ -40,7 +43,9 @@ public static void test() { tc.stageId(); tc.stageAttemptNumber(); tc.taskAttemptId(); + // this returns a scala Map, so make sure the JMap version give a java type back tc.resources(); + Map resources = tc.resourcesJMap(); tc.taskMetrics(); tc.taskMemoryManager(); tc.getLocalProperties(); diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bc607b5722f32..53a5a5029f8c3 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -59,6 +59,9 @@ object MimaExcludes { // [SPARK-27366][CORE] Support GPU Resources in Spark job scheduling ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.resources"), + // [SPARK-29417][CORE] Resource Scheduling - add TaskContext.resource java api + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.resourcesJMap"), + // [SPARK-27410][MLLIB] Remove deprecated / no-op mllib.KMeans getRuns, setRuns ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.getRuns"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.setRuns"), From bfa09cf049b7996ba0ed89e9b949edc2ab7fb3d6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 14 Oct 2019 14:09:21 -0700 Subject: [PATCH 0321/1387] [SPARK-29463][SQL] move v2 commands to a new file ### What changes were proposed in this pull request? move the v2 command logical plans from `basicLogicalOperators.scala` to a new file `v2Commands.scala` ### Why are the changes needed? As we keep adding v2 commands, the `basicLogicalOperators.scala` grows bigger and bigger. It's better to have a separated file for them. ### Does this PR introduce any user-facing change? no ### How was this patch tested? not needed Closes #26111 from cloud-fan/command. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../plans/logical/basicLogicalOperators.scala | 305 ---------------- .../catalyst/plans/logical/v2Commands.scala | 342 ++++++++++++++++++ 2 files changed, 342 insertions(+), 305 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala 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 d66371dd89e2d..d58c607cb87ee 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 @@ -382,298 +382,6 @@ case class Join( } } -/** - * Base trait for DataSourceV2 write commands - */ -trait V2WriteCommand extends Command { - def table: NamedRelation - def query: LogicalPlan - - override def children: Seq[LogicalPlan] = Seq(query) - - override lazy val resolved: Boolean = outputResolved - - def outputResolved: Boolean = { - // If the table doesn't require schema match, we don't need to resolve the output columns. - table.skipSchemaResolution || { - table.resolved && query.resolved && query.output.size == table.output.size && - query.output.zip(table.output).forall { - case (inAttr, outAttr) => - // names and types must match, nullability must be compatible - inAttr.name == outAttr.name && - DataType.equalsIgnoreCompatibleNullability(outAttr.dataType, inAttr.dataType) && - (outAttr.nullable || !inAttr.nullable) - } - } - } -} - -/** - * Create a new table with a v2 catalog. - */ -case class CreateV2Table( - catalog: TableCatalog, - tableName: Identifier, - tableSchema: StructType, - partitioning: Seq[Transform], - properties: Map[String, String], - ignoreIfExists: Boolean) extends Command with V2CreateTablePlan { - override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { - this.copy(partitioning = rewritten) - } -} - -/** - * Create a new table from a select query with a v2 catalog. - */ -case class CreateTableAsSelect( - catalog: TableCatalog, - tableName: Identifier, - partitioning: Seq[Transform], - query: LogicalPlan, - properties: Map[String, String], - writeOptions: Map[String, String], - ignoreIfExists: Boolean) extends Command with V2CreateTablePlan { - - override def tableSchema: StructType = query.schema - override def children: Seq[LogicalPlan] = Seq(query) - - override lazy val resolved: Boolean = childrenResolved && { - // the table schema is created from the query schema, so the only resolution needed is to check - // that the columns referenced by the table's partitioning exist in the query schema - val references = partitioning.flatMap(_.references).toSet - references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) - } - - override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { - this.copy(partitioning = rewritten) - } -} - -/** - * Replace a table with a v2 catalog. - * - * If the table does not exist, and orCreate is true, then it will be created. - * If the table does not exist, and orCreate is false, then an exception will be thrown. - * - * The persisted table will have no contents as a result of this operation. - */ -case class ReplaceTable( - catalog: TableCatalog, - tableName: Identifier, - tableSchema: StructType, - partitioning: Seq[Transform], - properties: Map[String, String], - orCreate: Boolean) extends Command with V2CreateTablePlan { - override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { - this.copy(partitioning = rewritten) - } -} - -/** - * Replaces a table from a select query with a v2 catalog. - * - * If the table does not exist, and orCreate is true, then it will be created. - * If the table does not exist, and orCreate is false, then an exception will be thrown. - */ -case class ReplaceTableAsSelect( - catalog: TableCatalog, - tableName: Identifier, - partitioning: Seq[Transform], - query: LogicalPlan, - properties: Map[String, String], - writeOptions: Map[String, String], - orCreate: Boolean) extends Command with V2CreateTablePlan { - - override def tableSchema: StructType = query.schema - override def children: Seq[LogicalPlan] = Seq(query) - - override lazy val resolved: Boolean = childrenResolved && { - // the table schema is created from the query schema, so the only resolution needed is to check - // that the columns referenced by the table's partitioning exist in the query schema - val references = partitioning.flatMap(_.references).toSet - references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) - } - - override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { - this.copy(partitioning = rewritten) - } -} - -/** - * Append data to an existing table. - */ -case class AppendData( - table: NamedRelation, - query: LogicalPlan, - writeOptions: Map[String, String], - isByName: Boolean) extends V2WriteCommand - -object AppendData { - def byName( - table: NamedRelation, - df: LogicalPlan, - writeOptions: Map[String, String] = Map.empty): AppendData = { - new AppendData(table, df, writeOptions, isByName = true) - } - - def byPosition( - table: NamedRelation, - query: LogicalPlan, - writeOptions: Map[String, String] = Map.empty): AppendData = { - new AppendData(table, query, writeOptions, isByName = false) - } -} - -/** - * Overwrite data matching a filter in an existing table. - */ -case class OverwriteByExpression( - table: NamedRelation, - deleteExpr: Expression, - query: LogicalPlan, - writeOptions: Map[String, String], - isByName: Boolean) extends V2WriteCommand { - override lazy val resolved: Boolean = outputResolved && deleteExpr.resolved -} - -object OverwriteByExpression { - def byName( - table: NamedRelation, - df: LogicalPlan, - deleteExpr: Expression, - writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { - OverwriteByExpression(table, deleteExpr, df, writeOptions, isByName = true) - } - - def byPosition( - table: NamedRelation, - query: LogicalPlan, - deleteExpr: Expression, - writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { - OverwriteByExpression(table, deleteExpr, query, writeOptions, isByName = false) - } -} - -/** - * Dynamically overwrite partitions in an existing table. - */ -case class OverwritePartitionsDynamic( - table: NamedRelation, - query: LogicalPlan, - writeOptions: Map[String, String], - isByName: Boolean) extends V2WriteCommand - -object OverwritePartitionsDynamic { - def byName( - table: NamedRelation, - df: LogicalPlan, - writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { - OverwritePartitionsDynamic(table, df, writeOptions, isByName = true) - } - - def byPosition( - table: NamedRelation, - query: LogicalPlan, - writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { - OverwritePartitionsDynamic(table, query, writeOptions, isByName = false) - } -} - -/** - * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. - */ -case class ShowNamespaces( - catalog: SupportsNamespaces, - namespace: Option[Seq[String]], - pattern: Option[String]) extends Command { - override val output: Seq[Attribute] = Seq( - AttributeReference("namespace", StringType, nullable = false)()) -} - -case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { - - override def children: Seq[LogicalPlan] = Seq(table) - - override val output = DescribeTableSchema.describeTableAttributes() -} - -case class DeleteFromTable( - table: LogicalPlan, - condition: Option[Expression]) extends Command with SupportsSubquery { - override def children: Seq[LogicalPlan] = table :: Nil -} - -case class UpdateTable( - table: LogicalPlan, - columns: Seq[Expression], - values: Seq[Expression], - condition: Option[Expression]) extends Command with SupportsSubquery { - override def children: Seq[LogicalPlan] = table :: Nil -} - -/** - * Drop a table. - */ -case class DropTable( - catalog: TableCatalog, - ident: Identifier, - ifExists: Boolean) extends Command - -/** - * Alter a table. - */ -case class AlterTable( - catalog: TableCatalog, - ident: Identifier, - table: NamedRelation, - changes: Seq[TableChange]) extends Command { - - override def children: Seq[LogicalPlan] = Seq(table) - - override lazy val resolved: Boolean = childrenResolved && { - changes.forall { - case add: AddColumn => - add.fieldNames match { - case Array(_) => - // a top-level field can always be added - true - case _ => - // the parent field must exist - table.schema.findNestedField(add.fieldNames.init, includeCollections = true).isDefined - } - - case colChange: ColumnChange => - // the column that will be changed must exist - table.schema.findNestedField(colChange.fieldNames, includeCollections = true).isDefined - - case _ => - // property changes require no resolution checks - true - } - } -} - -/** - * The logical plan of the SHOW TABLE command that works for v2 catalogs. - */ -case class ShowTables( - catalog: TableCatalog, - namespace: Seq[String], - pattern: Option[String]) extends Command { - override val output: Seq[Attribute] = Seq( - AttributeReference("namespace", StringType, nullable = false)(), - AttributeReference("tableName", StringType, nullable = false)()) -} - -/** - * The logical plan of the USE/USE NAMESPACE command that works for v2 catalogs. - */ -case class SetCatalogAndNamespace( - catalogManager: CatalogManager, - catalogName: Option[String], - namespace: Option[Seq[String]]) extends Command - /** * Insert query result into a directory. * @@ -1253,16 +961,3 @@ case class Deduplicate( * This is used to whitelist such commands in the subquery-related checks. */ trait SupportsSubquery extends LogicalPlan - -/** A trait used for logical plan nodes that create or replace V2 table definitions. */ -trait V2CreateTablePlan extends LogicalPlan { - def tableName: Identifier - def partitioning: Seq[Transform] - def tableSchema: StructType - - /** - * Creates a copy of this node with the new partitoning transforms. This method is used to - * rewrite the partition transforms normalized according to the table schema. - */ - def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala new file mode 100644 index 0000000000000..f89dfb1ec47d8 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -0,0 +1,342 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.analysis.NamedRelation +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import org.apache.spark.sql.catalyst.plans.DescribeTableSchema +import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, SupportsNamespaces, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.{DataType, StringType, StructType} + +/** + * Base trait for DataSourceV2 write commands + */ +trait V2WriteCommand extends Command { + def table: NamedRelation + def query: LogicalPlan + + override def children: Seq[LogicalPlan] = Seq(query) + + override lazy val resolved: Boolean = outputResolved + + def outputResolved: Boolean = { + // If the table doesn't require schema match, we don't need to resolve the output columns. + table.skipSchemaResolution || { + table.resolved && query.resolved && query.output.size == table.output.size && + query.output.zip(table.output).forall { + case (inAttr, outAttr) => + // names and types must match, nullability must be compatible + inAttr.name == outAttr.name && + DataType.equalsIgnoreCompatibleNullability(outAttr.dataType, inAttr.dataType) && + (outAttr.nullable || !inAttr.nullable) + } + } + } +} + +/** + * Append data to an existing table. + */ +case class AppendData( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String], + isByName: Boolean) extends V2WriteCommand + +object AppendData { + def byName( + table: NamedRelation, + df: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): AppendData = { + new AppendData(table, df, writeOptions, isByName = true) + } + + def byPosition( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): AppendData = { + new AppendData(table, query, writeOptions, isByName = false) + } +} + +/** + * Overwrite data matching a filter in an existing table. + */ +case class OverwriteByExpression( + table: NamedRelation, + deleteExpr: Expression, + query: LogicalPlan, + writeOptions: Map[String, String], + isByName: Boolean) extends V2WriteCommand { + override lazy val resolved: Boolean = outputResolved && deleteExpr.resolved +} + +object OverwriteByExpression { + def byName( + table: NamedRelation, + df: LogicalPlan, + deleteExpr: Expression, + writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, df, writeOptions, isByName = true) + } + + def byPosition( + table: NamedRelation, + query: LogicalPlan, + deleteExpr: Expression, + writeOptions: Map[String, String] = Map.empty): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, query, writeOptions, isByName = false) + } +} + +/** + * Dynamically overwrite partitions in an existing table. + */ +case class OverwritePartitionsDynamic( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String], + isByName: Boolean) extends V2WriteCommand + +object OverwritePartitionsDynamic { + def byName( + table: NamedRelation, + df: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, df, writeOptions, isByName = true) + } + + def byPosition( + table: NamedRelation, + query: LogicalPlan, + writeOptions: Map[String, String] = Map.empty): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, query, writeOptions, isByName = false) + } +} + + +/** A trait used for logical plan nodes that create or replace V2 table definitions. */ +trait V2CreateTablePlan extends LogicalPlan { + def tableName: Identifier + def partitioning: Seq[Transform] + def tableSchema: StructType + + /** + * Creates a copy of this node with the new partitioning transforms. This method is used to + * rewrite the partition transforms normalized according to the table schema. + */ + def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan +} + +/** + * Create a new table with a v2 catalog. + */ +case class CreateV2Table( + catalog: TableCatalog, + tableName: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + properties: Map[String, String], + ignoreIfExists: Boolean) extends Command with V2CreateTablePlan { + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} + +/** + * Create a new table from a select query with a v2 catalog. + */ +case class CreateTableAsSelect( + catalog: TableCatalog, + tableName: Identifier, + partitioning: Seq[Transform], + query: LogicalPlan, + properties: Map[String, String], + writeOptions: Map[String, String], + ignoreIfExists: Boolean) extends Command with V2CreateTablePlan { + + override def tableSchema: StructType = query.schema + override def children: Seq[LogicalPlan] = Seq(query) + + override lazy val resolved: Boolean = childrenResolved && { + // the table schema is created from the query schema, so the only resolution needed is to check + // that the columns referenced by the table's partitioning exist in the query schema + val references = partitioning.flatMap(_.references).toSet + references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) + } + + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} + +/** + * Replace a table with a v2 catalog. + * + * If the table does not exist, and orCreate is true, then it will be created. + * If the table does not exist, and orCreate is false, then an exception will be thrown. + * + * The persisted table will have no contents as a result of this operation. + */ +case class ReplaceTable( + catalog: TableCatalog, + tableName: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + properties: Map[String, String], + orCreate: Boolean) extends Command with V2CreateTablePlan { + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} + +/** + * Replaces a table from a select query with a v2 catalog. + * + * If the table does not exist, and orCreate is true, then it will be created. + * If the table does not exist, and orCreate is false, then an exception will be thrown. + */ +case class ReplaceTableAsSelect( + catalog: TableCatalog, + tableName: Identifier, + partitioning: Seq[Transform], + query: LogicalPlan, + properties: Map[String, String], + writeOptions: Map[String, String], + orCreate: Boolean) extends Command with V2CreateTablePlan { + + override def tableSchema: StructType = query.schema + override def children: Seq[LogicalPlan] = Seq(query) + + override lazy val resolved: Boolean = childrenResolved && { + // the table schema is created from the query schema, so the only resolution needed is to check + // that the columns referenced by the table's partitioning exist in the query schema + val references = partitioning.flatMap(_.references).toSet + references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) + } + + override def withPartitioning(rewritten: Seq[Transform]): V2CreateTablePlan = { + this.copy(partitioning = rewritten) + } +} + + +/** + * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. + */ +case class ShowNamespaces( + catalog: SupportsNamespaces, + namespace: Option[Seq[String]], + pattern: Option[String]) extends Command { + override val output: Seq[Attribute] = Seq( + AttributeReference("namespace", StringType, nullable = false)()) +} + +/** + * The logical plan of the DESCRIBE TABLE command that works for v2 tables. + */ +case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { + + override def children: Seq[LogicalPlan] = Seq(table) + + override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() +} + +/** + * The logical plan of the DELETE FROM command that works for v2 tables. + */ +case class DeleteFromTable( + table: LogicalPlan, + condition: Option[Expression]) extends Command with SupportsSubquery { + override def children: Seq[LogicalPlan] = table :: Nil +} + +/** + * The logical plan of the UPDATE TABLE command that works for v2 tables. + */ +case class UpdateTable( + table: LogicalPlan, + columns: Seq[Expression], + values: Seq[Expression], + condition: Option[Expression]) extends Command with SupportsSubquery { + override def children: Seq[LogicalPlan] = table :: Nil +} + +/** + * The logical plan of the DROP TABLE command that works for v2 tables. + */ +case class DropTable( + catalog: TableCatalog, + ident: Identifier, + ifExists: Boolean) extends Command + +/** + * The logical plan of the ALTER TABLE command that works for v2 tables. + */ +case class AlterTable( + catalog: TableCatalog, + ident: Identifier, + table: NamedRelation, + changes: Seq[TableChange]) extends Command { + + override def children: Seq[LogicalPlan] = Seq(table) + + override lazy val resolved: Boolean = childrenResolved && { + changes.forall { + case add: AddColumn => + add.fieldNames match { + case Array(_) => + // a top-level field can always be added + true + case _ => + // the parent field must exist + table.schema.findNestedField(add.fieldNames.init, includeCollections = true).isDefined + } + + case colChange: ColumnChange => + // the column that will be changed must exist + table.schema.findNestedField(colChange.fieldNames, includeCollections = true).isDefined + + case _ => + // property changes require no resolution checks + true + } + } +} + +/** + * The logical plan of the SHOW TABLE command that works for v2 catalogs. + */ +case class ShowTables( + catalog: TableCatalog, + namespace: Seq[String], + pattern: Option[String]) extends Command { + override val output: Seq[Attribute] = Seq( + AttributeReference("namespace", StringType, nullable = false)(), + AttributeReference("tableName", StringType, nullable = false)()) +} + +/** + * The logical plan of the USE/USE NAMESPACE command that works for v2 catalogs. + */ +case class SetCatalogAndNamespace( + catalogManager: CatalogManager, + catalogName: Option[String], + namespace: Option[Seq[String]]) extends Command From ff9fcd501c4035312b0123dcc0c82ad3c44715f9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 14 Oct 2019 15:15:32 -0700 Subject: [PATCH 0322/1387] Revert "[SPARK-29107][SQL][TESTS] Port window.sql (Part 1)" This reverts commit 81915dacc43f3eceb2434b2115d685c415adbc9b. --- .../sql-tests/inputs/pgSQL/window_part1.sql | 348 ------- .../results/pgSQL/window_part1.sql.out | 902 ------------------ 2 files changed, 1250 deletions(-) delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/pgSQL/window_part1.sql delete mode 100644 sql/core/src/test/resources/sql-tests/results/pgSQL/window_part1.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/window_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/window_part1.sql deleted file mode 100644 index 2c868d7c38de9..0000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/window_part1.sql +++ /dev/null @@ -1,348 +0,0 @@ --- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group --- --- Window Functions Testing --- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L1-L319 - -CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; - -CREATE TABLE empsalary ( - depname string, - empno integer, - salary int, - enroll_date date -) USING parquet; - -INSERT INTO empsalary VALUES -('develop', 10, 5200, '2007-08-01'), -('sales', 1, 5000, '2006-10-01'), -('personnel', 5, 3500, '2007-12-10'), -('sales', 4, 4800, '2007-08-08'), -('personnel', 2, 3900, '2006-12-23'), -('develop', 7, 4200, '2008-01-01'), -('develop', 9, 4500, '2008-01-01'), -('sales', 3, 4800, '2007-08-01'), -('develop', 8, 6000, '2006-10-01'), -('develop', 11, 5200, '2007-08-15'); - -SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname) FROM empsalary ORDER BY depname, salary; - -SELECT depname, empno, salary, rank() OVER (PARTITION BY depname ORDER BY salary) FROM empsalary; - --- with GROUP BY -SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 -GROUP BY four, ten ORDER BY four, ten; - -SELECT depname, empno, salary, sum(salary) OVER w FROM empsalary WINDOW w AS (PARTITION BY depname); - --- [SPARK-28064] Order by does not accept a call to rank() --- SELECT depname, empno, salary, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary) ORDER BY rank() OVER w; - --- empty window specification -SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10; - -SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS (); - --- no window operation -SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten); - --- cumulative aggregate -SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10; - -SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10; - --- [SPARK-28065] ntile does not accept NULL as input --- SELECT ntile(NULL) OVER (ORDER BY ten, four), ten, four FROM tenk1 LIMIT 2; - -SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - --- [SPARK-28068] `lag` second argument must be a literal in Spark --- SELECT lag(ten, four) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - --- [SPARK-28068] `lag` second argument must be a literal in Spark --- SELECT lag(ten, four, 0) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - --- last returns the last row of the frame, which is CURRENT ROW in ORDER BY window. -SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; - -SELECT last(ten) OVER (PARTITION BY four), ten, four FROM -(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s -ORDER BY four, ten; - --- [SPARK-27951] ANSI SQL: NTH_VALUE function --- SELECT nth_value(ten, four + 1) OVER (PARTITION BY four), ten, four --- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s; - -SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum -FROM tenk1 GROUP BY ten, two; - -SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10; - -SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + - sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum - FROM tenk1 WHERE unique2 < 10; - --- opexpr with different windows evaluation. -SELECT * FROM( - SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + - sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, - count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, - sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum - FROM tenk1 -)sub WHERE total <> fourcount + twosum; - -SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10; - -SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum -FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten); - --- more than one window with GROUP BY -SELECT sum(salary), - row_number() OVER (ORDER BY depname), - sum(sum(salary)) OVER (ORDER BY depname DESC) -FROM empsalary GROUP BY depname; - --- identical windows with different names -SELECT sum(salary) OVER w1, count(*) OVER w2 -FROM empsalary WINDOW w1 AS (ORDER BY salary), w2 AS (ORDER BY salary); - --- subplan --- [SPARK-28379] Correlated scalar subqueries must be aggregated --- SELECT lead(ten, (SELECT two FROM tenk1 WHERE s.unique2 = unique2)) OVER (PARTITION BY four ORDER BY ten) --- FROM tenk1 s WHERE unique2 < 10; - --- empty table -SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s; - --- mixture of agg/wfunc in the same window -SELECT sum(salary) OVER w, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary DESC); - --- strict aggs --- Temporarily turns off the ANSI mode because of compatibility issues between --- keywords related to date (in this case, year) -SET spark.sql.parser.ansi.enabled=false; -SELECT empno, depname, salary, bonus, depadj, MIN(bonus) OVER (ORDER BY empno), MAX(depadj) OVER () FROM( -SELECT *, - CASE WHEN enroll_date < '2008-01-01' THEN 2008 - extract(year FROM enroll_date) END * 500 AS bonus, - CASE WHEN - AVG(salary) OVER (PARTITION BY depname) < salary - THEN 200 END AS depadj FROM empsalary - )s; -SET spark.sql.parser.ansi.enabled=true; - -create temporary view int4_tbl as select * from values - (0), - (123456), - (-123456), - (2147483647), - (-2147483647) - as int4_tbl(f1); - --- window function over ungrouped agg over empty row set (bug before 9.1) -SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42; - --- window function with ORDER BY an expression involving aggregates (9.1 bug) -select ten, - sum(unique1) + sum(unique2) as res, - rank() over (order by sum(unique1) + sum(unique2)) as rank -from tenk1 -group by ten order by ten; - --- window and aggregate with GROUP BY expression (9.2 bug) --- explain --- select first(max(x)) over (), y --- from (select unique1 as x, ten+four as y from tenk1) ss --- group by y; - --- test non-default frame specifications -SELECT four, ten, -sum(ten) over (partition by four order by ten), -last(ten) over (partition by four order by ten) -FROM (select distinct ten, four from tenk1) ss; - -SELECT four, ten, -sum(ten) over (partition by four order by ten range between unbounded preceding and current row), -last(ten) over (partition by four order by ten range between unbounded preceding and current row) -FROM (select distinct ten, four from tenk1) ss; - -SELECT four, ten, -sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), -last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) -FROM (select distinct ten, four from tenk1) ss; - --- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift --- SELECT four, ten/4 as two, --- sum(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row), --- last(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row) --- FROM (select distinct ten, four from tenk1) ss; - --- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift --- SELECT four, ten/4 as two, --- sum(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row), --- last(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row) --- FROM (select distinct ten, four from tenk1) ss; - -SELECT sum(unique1) over (order by four range between current row and unbounded following), -unique1, four -FROM tenk1 WHERE unique1 < 10; - -SELECT sum(unique1) over (rows between current row and unbounded following), -unique1, four -FROM tenk1 WHERE unique1 < 10; - -SELECT sum(unique1) over (rows between 2 preceding and 2 following), -unique1, four -FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude no others), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude current row), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude group), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude ties), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude group), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude group), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), --- unique1, four --- FROM tenk1 WHERE unique1 < 10; - -SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), -unique1, four -FROM tenk1 WHERE unique1 < 10; - -SELECT sum(unique1) over (rows between 1 following and 3 following), -unique1, four -FROM tenk1 WHERE unique1 < 10; - -SELECT sum(unique1) over (rows between unbounded preceding and 1 following), -unique1, four -FROM tenk1 WHERE unique1 < 10; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT sum(unique1) over (w range between current row and unbounded following), --- unique1, four --- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude current row), --- unique1, four --- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude group), --- unique1, four --- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); - --- [SPARK-28428] Spark `exclude` always expecting `()` --- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude ties), --- unique1, four --- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); - --- [SPARK-27951] ANSI SQL: NTH_VALUE function --- SELECT first_value(unique1) over w, --- nth_value(unique1, 2) over w AS nth_2, --- last_value(unique1) over w, unique1, four --- FROM tenk1 WHERE unique1 < 10 --- WINDOW w AS (order by four range between current row and unbounded following); - --- [SPARK-28501] Frame bound value must be a literal. --- SELECT sum(unique1) over --- (order by unique1 --- rows (SELECT unique1 FROM tenk1 ORDER BY unique1 LIMIT 1) + 1 PRECEDING), --- unique1 --- FROM tenk1 WHERE unique1 < 10; - -CREATE TEMP VIEW v_window AS -SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows -FROM range(1, 11) i; - -SELECT * FROM v_window; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- CREATE OR REPLACE TEMP VIEW v_window AS --- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following --- exclude current row) as sum_rows FROM range(1, 10) i; - --- SELECT * FROM v_window; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- CREATE OR REPLACE TEMP VIEW v_window AS --- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following --- exclude group) as sum_rows FROM range(1, 10) i; --- SELECT * FROM v_window; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- CREATE OR REPLACE TEMP VIEW v_window AS --- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following --- exclude ties) as sum_rows FROM generate_series(1, 10) i; - --- [SPARK-28428] Spark `exclude` always expecting `()` --- CREATE OR REPLACE TEMP VIEW v_window AS --- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following --- exclude no others) as sum_rows FROM generate_series(1, 10) i; --- SELECT * FROM v_window; - --- [SPARK-28648] Adds support to `groups` unit type in window clauses --- CREATE OR REPLACE TEMP VIEW v_window AS --- SELECT i.id, sum(i.id) over (order by i.id groups between 1 preceding and 1 following) as sum_rows FROM range(1, 11) i; --- SELECT * FROM v_window; - -DROP VIEW v_window; -DROP TABLE empsalary; -DROP VIEW tenk2; -DROP VIEW int4_tbl; diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/window_part1.sql.out deleted file mode 100644 index e75df1e0315d7..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/window_part1.sql.out +++ /dev/null @@ -1,902 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- Number of queries: 55 - - --- !query 0 -CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 --- !query 0 schema -struct<> --- !query 0 output - - - --- !query 1 -CREATE TABLE empsalary ( - depname string, - empno integer, - salary int, - enroll_date date -) USING parquet --- !query 1 schema -struct<> --- !query 1 output - - - --- !query 2 -INSERT INTO empsalary VALUES -('develop', 10, 5200, '2007-08-01'), -('sales', 1, 5000, '2006-10-01'), -('personnel', 5, 3500, '2007-12-10'), -('sales', 4, 4800, '2007-08-08'), -('personnel', 2, 3900, '2006-12-23'), -('develop', 7, 4200, '2008-01-01'), -('develop', 9, 4500, '2008-01-01'), -('sales', 3, 4800, '2007-08-01'), -('develop', 8, 6000, '2006-10-01'), -('develop', 11, 5200, '2007-08-15') --- !query 2 schema -struct<> --- !query 2 output - - - --- !query 3 -SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname) FROM empsalary ORDER BY depname, salary --- !query 3 schema -struct --- !query 3 output -develop 7 4200 25100 -develop 9 4500 25100 -develop 10 5200 25100 -develop 11 5200 25100 -develop 8 6000 25100 -personnel 5 3500 7400 -personnel 2 3900 7400 -sales 4 4800 14600 -sales 3 4800 14600 -sales 1 5000 14600 - - --- !query 4 -SELECT depname, empno, salary, rank() OVER (PARTITION BY depname ORDER BY salary) FROM empsalary --- !query 4 schema -struct --- !query 4 output -develop 10 5200 3 -develop 11 5200 3 -develop 7 4200 1 -develop 8 6000 5 -develop 9 4500 2 -personnel 2 3900 2 -personnel 5 3500 1 -sales 1 5000 3 -sales 3 4800 1 -sales 4 4800 1 - - --- !query 5 -SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 -GROUP BY four, ten ORDER BY four, ten --- !query 5 schema -struct --- !query 5 output -0 0 0 0.0 -0 2 0 2.0 -0 4 0 4.0 -0 6 0 6.0 -0 8 0 8.0 -1 1 2500 1.0 -1 3 2500 3.0 -1 5 2500 5.0 -1 7 2500 7.0 -1 9 2500 9.0 -2 0 5000 0.0 -2 2 5000 2.0 -2 4 5000 4.0 -2 6 5000 6.0 -2 8 5000 8.0 -3 1 7500 1.0 -3 3 7500 3.0 -3 5 7500 5.0 -3 7 7500 7.0 -3 9 7500 9.0 - - --- !query 6 -SELECT depname, empno, salary, sum(salary) OVER w FROM empsalary WINDOW w AS (PARTITION BY depname) --- !query 6 schema -struct --- !query 6 output -develop 10 5200 25100 -develop 11 5200 25100 -develop 7 4200 25100 -develop 8 6000 25100 -develop 9 4500 25100 -personnel 2 3900 7400 -personnel 5 3500 7400 -sales 1 5000 14600 -sales 3 4800 14600 -sales 4 4800 14600 - - --- !query 7 -SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10 --- !query 7 schema -struct --- !query 7 output -10 -10 -10 -10 -10 -10 -10 -10 -10 -10 - - --- !query 8 -SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS () --- !query 8 schema -struct --- !query 8 output -10 -10 -10 -10 -10 -10 -10 -10 -10 -10 - - --- !query 9 -SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten) --- !query 9 schema -struct --- !query 9 output - - - --- !query 10 -SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10 --- !query 10 schema -struct --- !query 10 output -0 0 0 -0 0 0 -0 4 0 -1 7 1 -1 9 1 -2 0 2 -3 1 3 -3 3 3 -4 1 1 -5 1 1 - - --- !query 11 -SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10 --- !query 11 schema -struct --- !query 11 output -1 -10 -2 -3 -4 -5 -6 -7 -8 -9 - - --- !query 12 -SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10 --- !query 12 schema -struct --- !query 12 output -1 0 0 -1 0 0 -1 0 2 -1 1 1 -1 1 1 -1 1 3 -2 3 3 -3 4 0 -3 7 1 -4 9 1 - - --- !query 13 -SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 13 schema -struct --- !query 13 output -1 0 0 -1 0 0 -1 0 2 -1 1 1 -1 1 1 -1 1 3 -2 3 3 -2 4 0 -2 7 1 -3 9 1 - - --- !query 14 -SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 14 schema -struct --- !query 14 output -0.0 0 0 -0.0 0 0 -0.0 0 2 -0.0 1 1 -0.0 1 1 -0.0 1 3 -0.6666666666666666 7 1 -1.0 3 3 -1.0 4 0 -1.0 9 1 - - --- !query 15 -SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 15 schema -struct --- !query 15 output -0.5 1 1 -0.5 1 1 -0.5 1 3 -0.6666666666666666 0 0 -0.6666666666666666 0 0 -0.75 7 1 -1.0 0 2 -1.0 3 3 -1.0 4 0 -1.0 9 1 - - --- !query 16 -SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 16 schema -struct --- !query 16 output -1 0 0 -1 0 0 -1 0 2 -1 1 1 -2 1 1 -2 1 3 -2 3 3 -3 4 0 -3 7 1 -3 9 1 - - --- !query 17 -SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 17 schema -struct --- !query 17 output -0 0 0 -0 4 0 -1 1 1 -1 3 3 -1 7 1 -7 9 1 -NULL 0 0 -NULL 0 2 -NULL 1 1 -NULL 1 3 - - --- !query 18 -SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 18 schema -struct --- !query 18 output -0 0 0 -1 1 1 -3 1 3 -4 0 0 -7 1 1 -9 7 1 -NULL 0 2 -NULL 3 3 -NULL 4 0 -NULL 9 1 - - --- !query 19 -SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 19 schema -struct --- !query 19 output -0 0 0 -14 1 1 -18 7 1 -2 1 1 -6 1 3 -8 0 0 -NULL 0 2 -NULL 3 3 -NULL 4 0 -NULL 9 1 - - --- !query 20 -SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 20 schema -struct --- !query 20 output --1 0 2 --1 3 3 --1 4 0 --1 9 1 -0 0 0 -14 1 1 -18 7 1 -2 1 1 -6 1 3 -8 0 0 - - --- !query 21 -SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 21 schema -struct --- !query 21 output -0 0 0 -0 0 0 -0 0 2 -0 4 0 -1 1 1 -1 1 1 -1 1 3 -1 3 3 -1 7 1 -1 9 1 - - --- !query 22 -SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 22 schema -struct --- !query 22 output -0 4 0 -1 1 1 -1 1 1 -1 1 3 -1 7 1 -1 9 1 -2 0 0 -2 0 0 -2 0 2 -3 3 3 - - --- !query 23 -SELECT last(ten) OVER (PARTITION BY four), ten, four FROM -(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s -ORDER BY four, ten --- !query 23 schema -struct --- !query 23 output -4 0 0 -4 0 0 -4 4 0 -9 1 1 -9 1 1 -9 7 1 -9 9 1 -0 0 2 -3 1 3 -3 3 3 - - --- !query 24 -SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum -FROM tenk1 GROUP BY ten, two --- !query 24 schema -struct --- !query 24 output -0 0 45000 45000 -1 1 46000 46000 -2 0 47000 92000 -3 1 48000 94000 -4 0 49000 141000 -5 1 50000 144000 -6 0 51000 192000 -7 1 52000 196000 -8 0 53000 245000 -9 1 54000 250000 - - --- !query 25 -SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10 --- !query 25 schema -struct --- !query 25 output -2 3 -2 3 -4 1 -4 1 -4 1 -4 1 - - --- !query 26 -SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + - sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum - FROM tenk1 WHERE unique2 < 10 --- !query 26 schema -struct --- !query 26 output -136 -22 -22 -24 -24 -51 -82 -87 -92 -92 - - --- !query 27 -SELECT * FROM( - SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + - sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, - count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, - sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum - FROM tenk1 -)sub WHERE total <> fourcount + twosum --- !query 27 schema -struct --- !query 27 output - - - --- !query 28 -SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 --- !query 28 schema -struct --- !query 28 output -0.0 -0.0 -0.0 -1.0 -1.0 -1.0 -1.0 -2.0 -3.0 -3.0 - - --- !query 29 -SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum -FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten) --- !query 29 schema -struct --- !query 29 output -0 0 45000 45000 -1 1 46000 46000 -2 0 47000 92000 -3 1 48000 94000 -4 0 49000 141000 -5 1 50000 144000 -6 0 51000 192000 -7 1 52000 196000 -8 0 53000 245000 -9 1 54000 250000 - - --- !query 30 -SELECT sum(salary), - row_number() OVER (ORDER BY depname), - sum(sum(salary)) OVER (ORDER BY depname DESC) -FROM empsalary GROUP BY depname --- !query 30 schema -struct --- !query 30 output -14600 3 14600 -25100 1 47100 -7400 2 22000 - - --- !query 31 -SELECT sum(salary) OVER w1, count(*) OVER w2 -FROM empsalary WINDOW w1 AS (ORDER BY salary), w2 AS (ORDER BY salary) --- !query 31 schema -struct --- !query 31 output -11600 3 -16100 4 -25700 6 -25700 6 -30700 7 -3500 1 -41100 9 -41100 9 -47100 10 -7400 2 - - --- !query 32 -SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s --- !query 32 schema -struct --- !query 32 output - - - --- !query 33 -SELECT sum(salary) OVER w, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary DESC) --- !query 33 schema -struct --- !query 33 output -14600 2 -14600 2 -16400 2 -16400 2 -20900 4 -25100 5 -3900 1 -5000 1 -6000 1 -7400 2 - - --- !query 34 -SET spark.sql.parser.ansi.enabled=false --- !query 34 schema -struct --- !query 34 output -spark.sql.parser.ansi.enabled false - - --- !query 35 -SELECT empno, depname, salary, bonus, depadj, MIN(bonus) OVER (ORDER BY empno), MAX(depadj) OVER () FROM( -SELECT *, - CASE WHEN enroll_date < '2008-01-01' THEN 2008 - extract(year FROM enroll_date) END * 500 AS bonus, - CASE WHEN - AVG(salary) OVER (PARTITION BY depname) < salary - THEN 200 END AS depadj FROM empsalary - )s --- !query 35 schema -struct --- !query 35 output -1 sales 5000 1000 200 1000 200 -10 develop 5200 500 200 500 200 -11 develop 5200 500 200 500 200 -2 personnel 3900 1000 200 1000 200 -3 sales 4800 500 NULL 500 200 -4 sales 4800 500 NULL 500 200 -5 personnel 3500 500 NULL 500 200 -7 develop 4200 NULL NULL 500 200 -8 develop 6000 1000 200 500 200 -9 develop 4500 NULL NULL 500 200 - - --- !query 36 -SET spark.sql.parser.ansi.enabled=true --- !query 36 schema -struct --- !query 36 output -spark.sql.parser.ansi.enabled true - - --- !query 37 -create temporary view int4_tbl as select * from values - (0), - (123456), - (-123456), - (2147483647), - (-2147483647) - as int4_tbl(f1) --- !query 37 schema -struct<> --- !query 37 output - - - --- !query 38 -SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42 --- !query 38 schema -struct --- !query 38 output -0 - - --- !query 39 -select ten, - sum(unique1) + sum(unique2) as res, - rank() over (order by sum(unique1) + sum(unique2)) as rank -from tenk1 -group by ten order by ten --- !query 39 schema -struct --- !query 39 output -0 9976146 4 -1 10114187 9 -2 10059554 8 -3 9878541 1 -4 9881005 2 -5 9981670 5 -6 9947099 3 -7 10120309 10 -8 9991305 6 -9 10040184 7 - - --- !query 40 -SELECT four, ten, -sum(ten) over (partition by four order by ten), -last(ten) over (partition by four order by ten) -FROM (select distinct ten, four from tenk1) ss --- !query 40 schema -struct --- !query 40 output -0 0 0 0 -0 2 2 2 -0 4 6 4 -0 6 12 6 -0 8 20 8 -1 1 1 1 -1 3 4 3 -1 5 9 5 -1 7 16 7 -1 9 25 9 -2 0 0 0 -2 2 2 2 -2 4 6 4 -2 6 12 6 -2 8 20 8 -3 1 1 1 -3 3 4 3 -3 5 9 5 -3 7 16 7 -3 9 25 9 - - --- !query 41 -SELECT four, ten, -sum(ten) over (partition by four order by ten range between unbounded preceding and current row), -last(ten) over (partition by four order by ten range between unbounded preceding and current row) -FROM (select distinct ten, four from tenk1) ss --- !query 41 schema -struct --- !query 41 output -0 0 0 0 -0 2 2 2 -0 4 6 4 -0 6 12 6 -0 8 20 8 -1 1 1 1 -1 3 4 3 -1 5 9 5 -1 7 16 7 -1 9 25 9 -2 0 0 0 -2 2 2 2 -2 4 6 4 -2 6 12 6 -2 8 20 8 -3 1 1 1 -3 3 4 3 -3 5 9 5 -3 7 16 7 -3 9 25 9 - - --- !query 42 -SELECT four, ten, -sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), -last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) -FROM (select distinct ten, four from tenk1) ss --- !query 42 schema -struct --- !query 42 output -0 0 20 8 -0 2 20 8 -0 4 20 8 -0 6 20 8 -0 8 20 8 -1 1 25 9 -1 3 25 9 -1 5 25 9 -1 7 25 9 -1 9 25 9 -2 0 20 8 -2 2 20 8 -2 4 20 8 -2 6 20 8 -2 8 20 8 -3 1 25 9 -3 3 25 9 -3 5 25 9 -3 7 25 9 -3 9 25 9 - - --- !query 43 -SELECT sum(unique1) over (order by four range between current row and unbounded following), -unique1, four -FROM tenk1 WHERE unique1 < 10 --- !query 43 schema -struct --- !query 43 output -10 3 3 -10 7 3 -18 2 2 -18 6 2 -33 1 1 -33 5 1 -33 9 1 -45 0 0 -45 4 0 -45 8 0 - - --- !query 44 -SELECT sum(unique1) over (rows between current row and unbounded following), -unique1, four -FROM tenk1 WHERE unique1 < 10 --- !query 44 schema -struct --- !query 44 output -0 0 0 -10 3 3 -15 5 1 -23 8 0 -32 9 1 -38 6 2 -39 1 1 -41 2 2 -45 4 0 -7 7 3 - - --- !query 45 -SELECT sum(unique1) over (rows between 2 preceding and 2 following), -unique1, four -FROM tenk1 WHERE unique1 < 10 --- !query 45 schema -struct --- !query 45 output -10 0 0 -13 2 2 -15 7 3 -22 1 1 -23 3 3 -26 6 2 -29 9 1 -31 8 0 -32 5 1 -7 4 0 - - --- !query 46 -SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), -unique1, four -FROM tenk1 WHERE unique1 < 10 --- !query 46 schema -struct --- !query 46 output -10 0 0 -13 3 3 -15 8 0 -17 5 1 -3 6 2 -4 2 2 -6 1 1 -7 9 1 -8 7 3 -NULL 4 0 - - --- !query 47 -SELECT sum(unique1) over (rows between 1 following and 3 following), -unique1, four -FROM tenk1 WHERE unique1 < 10 --- !query 47 schema -struct --- !query 47 output -0 7 3 -10 5 1 -15 8 0 -16 2 2 -16 9 1 -22 6 2 -23 1 1 -7 3 3 -9 4 0 -NULL 0 0 - - --- !query 48 -SELECT sum(unique1) over (rows between unbounded preceding and 1 following), -unique1, four -FROM tenk1 WHERE unique1 < 10 --- !query 48 schema -struct --- !query 48 output -13 1 1 -22 6 2 -30 9 1 -35 8 0 -38 5 1 -45 0 0 -45 3 3 -45 7 3 -6 4 0 -7 2 2 - - --- !query 49 -CREATE TEMP VIEW v_window AS -SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows -FROM range(1, 11) i --- !query 49 schema -struct<> --- !query 49 output - - - --- !query 50 -SELECT * FROM v_window --- !query 50 schema -struct --- !query 50 output -1 3 -10 19 -2 6 -3 9 -4 12 -5 15 -6 18 -7 21 -8 24 -9 27 - - --- !query 51 -DROP VIEW v_window --- !query 51 schema -struct<> --- !query 51 output - - - --- !query 52 -DROP TABLE empsalary --- !query 52 schema -struct<> --- !query 52 output - - - --- !query 53 -DROP VIEW tenk2 --- !query 53 schema -struct<> --- !query 53 output - - - --- !query 54 -DROP VIEW int4_tbl --- !query 54 schema -struct<> --- !query 54 output - From 1f1443ebb2a52d311e599880c4065977ec1ee9dc Mon Sep 17 00:00:00 2001 From: herman Date: Tue, 15 Oct 2019 00:24:17 +0200 Subject: [PATCH 0323/1387] [SPARK-29347][SQL] Add JSON serialization for external Rows ### What changes were proposed in this pull request? This PR adds JSON serialization for Spark external Rows. ### Why are the changes needed? This is to be used for observable metrics where the `StreamingQueryProgress` contains a map of observed metrics rows which needs to be serialized in some cases. ### Does this PR introduce any user-facing change? Yes, a user can call `toJson` on rows returned when collecting a DataFrame to the driver. ### How was this patch tested? Added a new test suite: `RowJsonSuite` that should test this. Closes #26013 from hvanhovell/SPARK-29347. Authored-by: herman Signed-off-by: herman --- .../main/scala/org/apache/spark/sql/Row.scala | 108 +++++++++++++- .../org/apache/spark/sql/RowJsonSuite.scala | 140 ++++++++++++++++++ 2 files changed, 246 insertions(+), 2 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 494387ecfe0c1..3f9d07520e05f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -17,12 +17,24 @@ package org.apache.spark.sql +import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} +import java.util.{Base64, TimeZone} + import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.util.hashing.MurmurHash3 -import org.apache.spark.annotation.Stable +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.annotation.{Private, Stable, Unstable} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType, Decimal, MapType, StringType, StructType, UserDefinedType} /** * @since 1.3.0 @@ -501,4 +513,96 @@ trait Row extends Serializable { private def getAnyValAs[T <: AnyVal](i: Int): T = if (isNullAt(i)) throw new NullPointerException(s"Value at index $i is null") else getAs[T](i) + + /** + * The compact JSON representation of this row. + * @since 3.0 + */ + @Unstable + def json: String = compact(jsonValue) + + /** + * The pretty (i.e. indented) JSON representation of this row. + * @since 3.0 + */ + @Unstable + def prettyJson: String = pretty(render(jsonValue)) + + /** + * JSON representation of the row. + * + * Note that this only supports the data types that are also supported by + * [[org.apache.spark.sql.catalyst.encoders.RowEncoder]]. + * + * @return the JSON representation of the row. + */ + private[sql] def jsonValue: JValue = { + require(schema != null, "JSON serialization requires a non-null schema.") + + lazy val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) + lazy val dateFormatter = DateFormatter.apply(zoneId) + lazy val timestampFormatter = TimestampFormatter(zoneId) + + // Convert an iterator of values to a json array + def iteratorToJsonArray(iterator: Iterator[_], elementType: DataType): JArray = { + JArray(iterator.map(toJson(_, elementType)).toList) + } + + // Convert a value to json. + def toJson(value: Any, dataType: DataType): JValue = (value, dataType) match { + case (null, _) => JNull + case (b: Boolean, _) => JBool(b) + case (b: Byte, _) => JLong(b) + case (s: Short, _) => JLong(s) + case (i: Int, _) => JLong(i) + case (l: Long, _) => JLong(l) + case (f: Float, _) => JDouble(f) + case (d: Double, _) => JDouble(d) + case (d: BigDecimal, _) => JDecimal(d) + case (d: java.math.BigDecimal, _) => JDecimal(d) + case (d: Decimal, _) => JDecimal(d.toBigDecimal) + case (s: String, _) => JString(s) + case (b: Array[Byte], BinaryType) => + JString(Base64.getEncoder.encodeToString(b)) + case (d: LocalDate, _) => + JString(dateFormatter.format(DateTimeUtils.localDateToDays(d))) + case (d: Date, _) => + JString(dateFormatter.format(DateTimeUtils.fromJavaDate(d))) + case (i: Instant, _) => + JString(timestampFormatter.format(DateTimeUtils.instantToMicros(i))) + case (t: Timestamp, _) => + JString(timestampFormatter.format(DateTimeUtils.fromJavaTimestamp(t))) + case (a: Array[_], ArrayType(elementType, _)) => + iteratorToJsonArray(a.iterator, elementType) + case (s: Seq[_], ArrayType(elementType, _)) => + iteratorToJsonArray(s.iterator, elementType) + case (m: Map[String @unchecked, _], MapType(StringType, valueType, _)) => + new JObject(m.toList.sortBy(_._1).map { + case (k, v) => k -> toJson(v, valueType) + }) + case (m: Map[_, _], MapType(keyType, valueType, _)) => + new JArray(m.iterator.map { + case (k, v) => + new JObject("key" -> toJson(k, keyType) :: "value" -> toJson(v, valueType) :: Nil) + }.toList) + case (r: Row, _) => r.jsonValue + case (v: Any, udt: UserDefinedType[Any @unchecked]) => + val dataType = udt.sqlType + toJson(CatalystTypeConverters.convertToScala(udt.serialize(v), dataType), dataType) + case _ => + throw new IllegalArgumentException(s"Failed to convert value $value " + + s"(class of ${value.getClass}}) with the type of $dataType to JSON.") + } + + // Convert the row fields to json + var n = 0 + var elements = new mutable.ListBuffer[JField] + val len = length + while (n < len) { + val field = schema(n) + elements += (field.name -> toJson(apply(n), field.dataType)) + n += 1 + } + new JObject(elements.toList) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala new file mode 100644 index 0000000000000..ac18b0f79b5f3 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala @@ -0,0 +1,140 @@ +/* + * 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 + +import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} + +import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JLong, JNull, JObject, JString, JValue} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.encoders.{ExamplePoint, ExamplePointUDT} +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * Test suite for [[Row]] JSON serialization. + */ +class RowJsonSuite extends SparkFunSuite { + private val schema = new StructType() + .add("c1", "string") + .add("c2", IntegerType) + + private def testJson(name: String, value: Any, dt: DataType, expected: JValue): Unit = { + test(name) { + val row = new GenericRowWithSchema(Array(value), new StructType().add("a", dt)) + assert(row.jsonValue === JObject("a" -> expected)) + } + } + + private def testJson(value: Any, dt: DataType, expected: JValue): Unit = { + testJson(s"$dt $value", value, dt, expected) + } + + // Nulls + private def testJsonNull(dt: DataType, expected: JValue): Unit = { + testJson(null, dt, JNull) + } + testJsonNull(IntegerType, JNull) + testJsonNull(FloatType, JNull) + testJsonNull(ArrayType(DoubleType, containsNull = true), JNull) + + // Primitives + testJson(true, BooleanType, JBool(true)) + testJson(false, BooleanType, JBool(false)) + testJson(23.toByte, ByteType, JLong(23)) + testJson(-126.toByte, ByteType, JLong(-126)) + testJson(20281.toShort, ShortType, JLong(20281)) + testJson(-8752.toShort, ShortType, JLong(-8752)) + testJson(1078231987, IntegerType, JLong(1078231987)) + testJson(-10, IntegerType, JLong(-10)) + testJson(139289832109874199L, LongType, JLong(139289832109874199L)) + testJson(-7873748239973488L, LongType, JLong(-7873748239973488L)) + testJson(10.232e10f, FloatType, JDouble(10.232e10f)) + testJson(9.7e-13f, FloatType, JDouble(9.7e-13f)) + testJson(3.891e98d, DoubleType, JDouble(3.891e98d)) + testJson(-7.8e5d, DoubleType, JDouble(-7.8e5d)) + testJson(BigDecimal("1092.88"), DecimalType(10, 2), JDecimal(BigDecimal("1092.88"))) + testJson(Decimal("782.0003"), DecimalType(7, 4), JDecimal(BigDecimal("782.0003"))) + testJson(new java.math.BigDecimal("-77.89"), DecimalType(4, 2), JDecimal(BigDecimal("-77.89"))) + testJson("hello world", StringType, JString("hello world")) + testJson("BinaryType", Array('a'.toByte, 'b'.toByte), BinaryType, JString("YWI=")) + testJson(Date.valueOf("2019-04-22"), DateType, JString("2019-04-22")) + testJson(LocalDate.of(2018, 5, 14), DateType, JString("2018-05-14")) + testJson( + Timestamp.valueOf("2017-01-06 10:22:03.00"), + TimestampType, + JString("2017-01-06 10:22:03")) + testJson( + Timestamp.valueOf("2017-05-30 10:22:03.00").toInstant, + TimestampType, + JString("2017-05-30 10:22:03")) + + // Complex types + testJson( + "ArrayType(LongType,true)", + Array(1L, null, 77L), + ArrayType(LongType, containsNull = true), + JArray(JLong(1L) :: JNull :: JLong(77L) :: Nil)) + + testJson( + Seq(1, -2, 3), + ArrayType(IntegerType, containsNull = false), + JArray(JLong(1) :: JLong(-2) :: JLong(3) :: Nil)) + + testJson( + Map("a" -> "b", "c" -> "d", "e" -> null), + MapType(StringType, StringType, valueContainsNull = true), + JObject("a" -> JString("b"), "c" -> JString("d"), "e" -> JNull)) + + testJson( + Map(1 -> "b", 2 -> "d", 3 -> null), + MapType(IntegerType, StringType, valueContainsNull = true), + JArray( + JObject("key" -> JLong(1), "value" -> JString("b")) :: + JObject("key" -> JLong(2), "value" -> JString("d")) :: + JObject("key" -> JLong(3), "value" -> JNull) :: Nil)) + + testJson( + new GenericRowWithSchema(Array("1", 2), schema), + schema, + JObject("c1" -> JString("1"), "c2" -> JLong(2))) + + testJson( + "UDT", + new ExamplePoint(3.4d, 8.98d), + new ExamplePointUDT, + JArray(JDouble(3.4d) :: JDouble(8.98d) :: Nil)) + + test("no schema") { + val e = intercept[IllegalArgumentException] { + Row("a").jsonValue + } + assert(e.getMessage.contains("requires a non-null schema")) + } + + test("unsupported type") { + val e = intercept[IllegalArgumentException] { + val row = new GenericRowWithSchema( + Array((1, 2)), + new StructType().add("a", ObjectType(classOf[(Int, Int)]))) + row.jsonValue + } + assert(e.getMessage.contains("Failed to convert value")) + } +} From 9407fba0375675d6ee6461253f3b8230e8d67509 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 15 Oct 2019 10:18:58 +0800 Subject: [PATCH 0324/1387] [SPARK-29412][SQL] refine the document of v2 session catalog config ### What changes were proposed in this pull request? Refine the document of v2 session catalog config, to clearly explain what it is, when it should be used and how to implement it. ### Why are the changes needed? Make this config more understandable ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass the Jenkins with the newly updated test cases. Closes #26071 from cloud-fan/config. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../connector/catalog/CatalogManager.scala | 8 ++++---- .../apache/spark/sql/internal/SQLConf.scala | 14 ++++++++----- ...SourceV2DataFrameSessionCatalogSuite.scala | 9 +++++---- .../sql/connector/DataSourceV2SQLSuite.scala | 20 ++++++++++--------- .../command/PlanResolutionSuite.scala | 6 +++--- 5 files changed, 32 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index be14b17701276..14ccfd5bfcc9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -74,10 +74,10 @@ class CatalogManager( } } - // If the V2_SESSION_CATALOG config is specified, we try to instantiate the user-specified v2 - // session catalog. Otherwise, return the default session catalog. + // If the V2_SESSION_CATALOG_IMPLEMENTATION config is specified, we try to instantiate the + // user-specified v2 session catalog. Otherwise, return the default session catalog. def v2SessionCatalog: CatalogPlugin = { - conf.getConf(SQLConf.V2_SESSION_CATALOG).map { customV2SessionCatalog => + conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { customV2SessionCatalog => try { catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) } catch { @@ -146,5 +146,5 @@ class CatalogManager( } private[sql] object CatalogManager { - val SESSION_CATALOG_NAME: String = "session" + val SESSION_CATALOG_NAME: String = "spark_catalog" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index eebf4b6dfd396..bcb3153a3ca48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator import org.apache.spark.sql.catalyst.plans.logical.HintErrorHandler +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils @@ -1981,11 +1982,14 @@ object SQLConf { .stringConf .createOptional - val V2_SESSION_CATALOG = buildConf("spark.sql.catalog.session") - .doc("A catalog implementation that will be used in place of the Spark built-in session " + - "catalog for v2 operations. The implementation may extend `CatalogExtension` to be " + - "passed the Spark built-in session catalog, so that it may delegate calls to the " + - "built-in session catalog.") + val V2_SESSION_CATALOG_IMPLEMENTATION = + buildConf(s"spark.sql.catalog.$SESSION_CATALOG_NAME") + .doc("A catalog implementation that will be used as the v2 interface to Spark's built-in " + + s"v1 catalog: $SESSION_CATALOG_NAME. This catalog shares its identifier namespace with " + + s"the $SESSION_CATALOG_NAME and must be consistent with it; for example, if a table can " + + s"be loaded by the $SESSION_CATALOG_NAME, this catalog must also return the table " + + s"metadata. To delegate operations to the $SESSION_CATALOG_NAME, implementations can " + + "extend 'CatalogExtension'.") .stringConf .createOptional diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index 207ece83759ed..e27575cecde25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -25,8 +25,9 @@ import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG +import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -144,13 +145,13 @@ private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2Sessio protected val catalogClassName: String = classOf[InMemoryTableSessionCatalog].getName before { - spark.conf.set(V2_SESSION_CATALOG.key, catalogClassName) + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION.key, catalogClassName) } override def afterEach(): Unit = { super.afterEach() - catalog("session").asInstanceOf[Catalog].clearTables() - spark.conf.unset(V2_SESSION_CATALOG.key) + catalog(SESSION_CATALOG_NAME).asInstanceOf[Catalog].clearTables() + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) } protected def verifyTable(tableName: String, expected: DataFrame): Unit diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 570d1f2f24ccc..3b42c2374f006 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -22,8 +22,9 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.catalog._ +import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG +import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.sources.SimpleScanSource import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -67,7 +68,8 @@ class DataSourceV2SQLSuite spark.conf.set( "spark.sql.catalog.testcat_atomic", classOf[StagingInMemoryTableCatalog].getName) spark.conf.set("spark.sql.catalog.testcat2", classOf[InMemoryTableCatalog].getName) - spark.conf.set(V2_SESSION_CATALOG.key, classOf[InMemoryTableSessionCatalog].getName) + spark.conf.set( + V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[InMemoryTableSessionCatalog].getName) val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -149,7 +151,7 @@ class DataSourceV2SQLSuite test("CreateTable: use v2 plan and session catalog when provider is v2") { spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING $v2Source") - val testCatalog = catalog("session").asTableCatalog + val testCatalog = catalog(SESSION_CATALOG_NAME).asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "default.table_name") @@ -408,7 +410,7 @@ class DataSourceV2SQLSuite test("CreateTableAsSelect: use v2 plan and session catalog when provider is v2") { spark.sql(s"CREATE TABLE table_name USING $v2Source AS SELECT id, data FROM source") - val testCatalog = catalog("session").asTableCatalog + val testCatalog = catalog(SESSION_CATALOG_NAME).asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "default.table_name") @@ -510,7 +512,7 @@ class DataSourceV2SQLSuite test("CreateTableAsSelect: v2 session catalog can load v1 source table") { // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG.key) + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -520,7 +522,7 @@ class DataSourceV2SQLSuite checkAnswer(sql(s"TABLE default.table_name"), spark.table("source")) // The fact that the following line doesn't throw an exception means, the session catalog // can load the table. - val t = catalog("session").asTableCatalog + val t = catalog(SESSION_CATALOG_NAME).asTableCatalog .loadTable(Identifier.of(Array.empty, "table_name")) assert(t.isInstanceOf[V1Table], "V1 table wasn't returned as an unresolved table") } @@ -848,7 +850,7 @@ class DataSourceV2SQLSuite val catalogManager = spark.sessionState.catalogManager // Validate the initial current catalog and namespace. - assert(catalogManager.currentCatalog.name() == "session") + assert(catalogManager.currentCatalog.name() == SESSION_CATALOG_NAME) assert(catalogManager.currentNamespace === Array("default")) // The following implicitly creates namespaces. @@ -885,7 +887,7 @@ class DataSourceV2SQLSuite test("Use: set v2 catalog as a current catalog") { val catalogManager = spark.sessionState.catalogManager - assert(catalogManager.currentCatalog.name() == "session") + assert(catalogManager.currentCatalog.name() == SESSION_CATALOG_NAME) sql("USE testcat") assert(catalogManager.currentCatalog.name() == "testcat") @@ -907,7 +909,7 @@ class DataSourceV2SQLSuite test("tableCreation: partition column case insensitive resolution") { val testCatalog = catalog("testcat").asTableCatalog - val sessionCatalog = catalog("session").asTableCatalog + val sessionCatalog = catalog(SESSION_CATALOG_NAME).asTableCatalog def checkPartitioning(cat: TableCatalog, partition: String): Unit = { val table = cat.loadTable(Identifier.of(Array.empty, "tbl")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 0f4fe656dd20a..674efa9b8ba42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -77,7 +77,7 @@ class PlanResolutionSuite extends AnalysisTest { throw new NoSuchTableException(name) } }) - when(newCatalog.name()).thenReturn("session") + when(newCatalog.name()).thenReturn(CatalogManager.SESSION_CATALOG_NAME) newCatalog } @@ -477,7 +477,7 @@ class PlanResolutionSuite extends AnalysisTest { parseAndResolve(sql) match { case create: CreateV2Table => - assert(create.catalog.name == "session") + assert(create.catalog.name == CatalogManager.SESSION_CATALOG_NAME) assert(create.tableName == Identifier.of(Array("mydb"), "page_view")) assert(create.tableSchema == new StructType() .add("id", LongType) @@ -581,7 +581,7 @@ class PlanResolutionSuite extends AnalysisTest { parseAndResolve(sql) match { case ctas: CreateTableAsSelect => - assert(ctas.catalog.name == "session") + assert(ctas.catalog.name == CatalogManager.SESSION_CATALOG_NAME) assert(ctas.tableName == Identifier.of(Array("mydb"), "page_view")) assert(ctas.properties == expectedProperties) assert(ctas.writeOptions.isEmpty) From 857f109c47b26a38f5d114a94f94c516177db3fc Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 14 Oct 2019 19:38:04 -0700 Subject: [PATCH 0325/1387] [SPARK-10614][CORE] Add monotonic time to Clock interface This change adds a new method to the Clock interface that returns the time from a monotonic time source, so that code that needs that feature can also mock the Clock in tests. The original getTimeMillis and waitTillTime methods are unchanged, since streaming code that uses the Clock interface seems to rely on wall clock semantics, not monotonic clock. So, in a way, this doesn't directly address the problem raised in the bug, that waitTillTime can be affected by drift, but then the places being modified to use the new API don't really rely on that API. The dynamic allocation code was modified to use the new time source, since they shouldn't be basing their decisions on wall clock time. For a longer discussion on how monotonic clocks work on Linux/x64, the following blog post (and links within) shed a lot of light on the safety of `System.nanoTime()`: http://btorpey.github.io/blog/2014/02/18/clock-sources-in-linux/ Tested with unit test and also running apps with dynamic allocation on. Closes #26058 from vanzin/SPARK-10614. Authored-by: Marcelo Vanzin Signed-off-by: Dongjoon Hyun --- .../spark/ExecutorAllocationManager.scala | 6 +-- .../scheduler/dynalloc/ExecutorMonitor.scala | 17 ++++---- .../scala/org/apache/spark/util/Clock.scala | 42 +++++++++++++++++-- .../org/apache/spark/util/ManualClock.scala | 16 ++++--- .../ExecutorAllocationManagerSuite.scala | 10 +++-- .../dynalloc/ExecutorMonitorSuite.scala | 24 +++++------ 6 files changed, 79 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 5114cf70e3f26..7415302fc529a 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -288,7 +288,7 @@ private[spark] class ExecutorAllocationManager( } // Update executor target number only after initializing flag is unset - updateAndSyncNumExecutorsTarget(clock.getTimeMillis()) + updateAndSyncNumExecutorsTarget(clock.nanoTime()) if (executorIdsToBeRemoved.nonEmpty) { removeExecutors(executorIdsToBeRemoved) } @@ -336,7 +336,7 @@ private[spark] class ExecutorAllocationManager( val delta = addExecutors(maxNeeded) logDebug(s"Starting timer to add more executors (to " + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") - addTime = now + (sustainedSchedulerBacklogTimeoutS * 1000) + addTime = now + TimeUnit.SECONDS.toNanos(sustainedSchedulerBacklogTimeoutS) delta } else { 0 @@ -481,7 +481,7 @@ private[spark] class ExecutorAllocationManager( if (addTime == NOT_SET) { logDebug(s"Starting timer to add executors because pending tasks " + s"are building up (to expire in $schedulerBacklogTimeoutS seconds)") - addTime = clock.getTimeMillis + schedulerBacklogTimeoutS * 1000 + addTime = clock.nanoTime() + TimeUnit.SECONDS.toNanos(schedulerBacklogTimeoutS) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index aa901d6568b26..3dfd1eac8cc9c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -39,11 +39,12 @@ private[spark] class ExecutorMonitor( listenerBus: LiveListenerBus, clock: Clock) extends SparkListener with CleanerListener with Logging { - private val idleTimeoutMs = TimeUnit.SECONDS.toMillis( + private val idleTimeoutNs = TimeUnit.SECONDS.toNanos( conf.get(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT)) - private val storageTimeoutMs = TimeUnit.SECONDS.toMillis( + private val storageTimeoutNs = TimeUnit.SECONDS.toNanos( conf.get(DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT)) - private val shuffleTimeoutMs = conf.get(DYN_ALLOCATION_SHUFFLE_TIMEOUT) + private val shuffleTimeoutNs = TimeUnit.MILLISECONDS.toNanos( + conf.get(DYN_ALLOCATION_SHUFFLE_TIMEOUT)) private val fetchFromShuffleSvcEnabled = conf.get(SHUFFLE_SERVICE_ENABLED) && conf.get(SHUFFLE_SERVICE_FETCH_RDD_ENABLED) @@ -100,7 +101,7 @@ private[spark] class ExecutorMonitor( * Should only be called from the EAM thread. */ def timedOutExecutors(): Seq[String] = { - val now = clock.getTimeMillis() + val now = clock.nanoTime() if (now >= nextTimeout.get()) { // Temporarily set the next timeout at Long.MaxValue. This ensures that after // scanning all executors below, we know when the next timeout for non-timed out @@ -437,7 +438,7 @@ private[spark] class ExecutorMonitor( def updateRunningTasks(delta: Int): Unit = { runningTasks = math.max(0, runningTasks + delta) - idleStart = if (runningTasks == 0) clock.getTimeMillis() else -1L + idleStart = if (runningTasks == 0) clock.nanoTime() else -1L updateTimeout() } @@ -445,15 +446,15 @@ private[spark] class ExecutorMonitor( val oldDeadline = timeoutAt val newDeadline = if (idleStart >= 0) { val timeout = if (cachedBlocks.nonEmpty || (shuffleIds != null && shuffleIds.nonEmpty)) { - val _cacheTimeout = if (cachedBlocks.nonEmpty) storageTimeoutMs else Long.MaxValue + val _cacheTimeout = if (cachedBlocks.nonEmpty) storageTimeoutNs else Long.MaxValue val _shuffleTimeout = if (shuffleIds != null && shuffleIds.nonEmpty) { - shuffleTimeoutMs + shuffleTimeoutNs } else { Long.MaxValue } math.min(_cacheTimeout, _shuffleTimeout) } else { - idleTimeoutMs + idleTimeoutNs } val deadline = idleStart + timeout if (deadline >= 0) deadline else Long.MaxValue diff --git a/core/src/main/scala/org/apache/spark/util/Clock.scala b/core/src/main/scala/org/apache/spark/util/Clock.scala index e92ed11bd165b..d2674d4f47224 100644 --- a/core/src/main/scala/org/apache/spark/util/Clock.scala +++ b/core/src/main/scala/org/apache/spark/util/Clock.scala @@ -21,7 +21,37 @@ package org.apache.spark.util * An interface to represent clocks, so that they can be mocked out in unit tests. */ private[spark] trait Clock { + /** @return Current system time, in ms. */ def getTimeMillis(): Long + + // scalastyle:off line.size.limit + /** + * Current value of high resolution time source, in ns. + * + * This method abstracts the call to the JRE's `System.nanoTime()` call. As with that method, the + * value here is not guaranteed to be monotonically increasing, but rather a higher resolution + * time source for use in the calculation of time intervals. The characteristics of the values + * returned may very from JVM to JVM (or even the same JVM running on different OSes or CPUs), but + * in general it should be preferred over [[getTimeMillis()]] when calculating time differences. + * + * Specifically for Linux on x64 architecture, the following links provide useful information + * about the characteristics of the value returned: + * + * http://btorpey.github.io/blog/2014/02/18/clock-sources-in-linux/ + * https://stackoverflow.com/questions/10921210/cpu-tsc-fetch-operation-especially-in-multicore-multi-processor-environment + * + * TL;DR: on modern (2.6.32+) Linux kernels with modern (AMD K8+) CPUs, the values returned by + * `System.nanoTime()` are consistent across CPU cores *and* packages, and provide always + * increasing values (although it may not be completely monotonic when the the system clock is + * adjusted by NTP daemons using time slew). + */ + // scalastyle:on line.size.limit + def nanoTime(): Long + + /** + * Wait until the wall clock reaches at least the given time. Note this may not actually wait for + * the actual difference between the current and target times, since the wall clock may drift. + */ def waitTillTime(targetTime: Long): Long } @@ -36,15 +66,19 @@ private[spark] class SystemClock extends Clock { * @return the same time (milliseconds since the epoch) * as is reported by `System.currentTimeMillis()` */ - def getTimeMillis(): Long = System.currentTimeMillis() + override def getTimeMillis(): Long = System.currentTimeMillis() + + /** + * @return value reported by `System.nanoTime()`. + */ + override def nanoTime(): Long = System.nanoTime() /** * @param targetTime block until the current time is at least this value * @return current system time when wait has completed */ - def waitTillTime(targetTime: Long): Long = { - var currentTime = 0L - currentTime = System.currentTimeMillis() + override def waitTillTime(targetTime: Long): Long = { + var currentTime = System.currentTimeMillis() var waitTime = targetTime - currentTime if (waitTime <= 0) { diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala index e7a65d74a440e..36d6820eba239 100644 --- a/core/src/main/scala/org/apache/spark/util/ManualClock.scala +++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala @@ -17,11 +17,16 @@ package org.apache.spark.util +import java.util.concurrent.TimeUnit + /** * A `Clock` whose time can be manually set and modified. Its reported time does not change * as time elapses, but only as its time is modified by callers. This is mainly useful for * testing. * + * For this implementation, `getTimeMillis()` and `nanoTime()` always return the same value + * (adjusted for the correct unit). + * * @param time initial time (in milliseconds since the epoch) */ private[spark] class ManualClock(private var time: Long) extends Clock { @@ -31,10 +36,11 @@ private[spark] class ManualClock(private var time: Long) extends Clock { */ def this() = this(0L) - def getTimeMillis(): Long = - synchronized { - time - } + override def getTimeMillis(): Long = synchronized { + time + } + + override def nanoTime(): Long = TimeUnit.MILLISECONDS.toNanos(getTimeMillis()) /** * @param timeToSet new time (in milliseconds) that the clock should represent @@ -56,7 +62,7 @@ private[spark] class ManualClock(private var time: Long) extends Clock { * @param targetTime block until the clock time is set or advanced to at least this time * @return current time reported by the clock when waiting finishes */ - def waitTillTime(targetTime: Long): Long = synchronized { + override def waitTillTime(targetTime: Long): Long = synchronized { while (time < targetTime) { wait(10) } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 04c00f11816ee..6ae1f197cf744 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark +import java.util.concurrent.TimeUnit + import scala.collection.mutable import org.mockito.ArgumentMatchers.{any, eq => meq} @@ -541,7 +543,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(addTime(manager) === NOT_SET) onSchedulerBacklogged(manager) val firstAddTime = addTime(manager) - assert(firstAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) + assert(firstAddTime === clock.nanoTime() + TimeUnit.SECONDS.toNanos(schedulerBacklogTimeout)) clock.advance(100L) onSchedulerBacklogged(manager) assert(addTime(manager) === firstAddTime) // timer is already started @@ -555,7 +557,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(addTime(manager) === NOT_SET) onSchedulerBacklogged(manager) val secondAddTime = addTime(manager) - assert(secondAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) + assert(secondAddTime === clock.nanoTime() + TimeUnit.SECONDS.toNanos(schedulerBacklogTimeout)) clock.advance(100L) onSchedulerBacklogged(manager) assert(addTime(manager) === secondAddTime) // timer is already started @@ -936,7 +938,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(SparkListenerStageSubmitted(createStageInfo(0, 2))) clock.advance(1000) - manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.getTimeMillis()) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) assert(numExecutorsTarget(manager) === 2) val taskInfo0 = createTaskInfo(0, 0, "executor-1") post(SparkListenerTaskStart(0, 0, taskInfo0)) @@ -952,7 +954,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(maxNumExecutorsNeeded(manager) === 1) assert(numExecutorsTarget(manager) === 2) clock.advance(1000) - manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.getTimeMillis()) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) assert(numExecutorsTarget(manager) === 1) verify(client, never).killExecutors(any(), any(), any(), any()) diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index d3feb35537b34..1397cb7b39004 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -33,9 +33,9 @@ import org.apache.spark.util.ManualClock class ExecutorMonitorSuite extends SparkFunSuite { - private val idleTimeoutMs = TimeUnit.SECONDS.toMillis(60L) - private val storageTimeoutMs = TimeUnit.SECONDS.toMillis(120L) - private val shuffleTimeoutMs = TimeUnit.SECONDS.toMillis(240L) + private val idleTimeoutNs = TimeUnit.SECONDS.toNanos(60L) + private val storageTimeoutNs = TimeUnit.SECONDS.toNanos(120L) + private val shuffleTimeoutNs = TimeUnit.SECONDS.toNanos(240L) private val conf = new SparkConf() .set(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "60s") @@ -111,8 +111,8 @@ class ExecutorMonitorSuite extends SparkFunSuite { monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("1", 1), new ExecutorMetrics, null)) assert(monitor.isExecutorIdle("1")) - assert(monitor.timedOutExecutors(clock.getTimeMillis()).isEmpty) - assert(monitor.timedOutExecutors(clock.getTimeMillis() + idleTimeoutMs + 1) === Seq("1")) + assert(monitor.timedOutExecutors(clock.nanoTime()).isEmpty) + assert(monitor.timedOutExecutors(clock.nanoTime() + idleTimeoutNs + 1) === Seq("1")) } test("use appropriate time out depending on whether blocks are stored") { @@ -166,7 +166,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { // originally went idle. clock.setTime(idleDeadline) monitor.onUnpersistRDD(SparkListenerUnpersistRDD(2)) - assert(monitor.timedOutExecutors(clock.getTimeMillis()) === Seq("1")) + assert(monitor.timedOutExecutors(clock.nanoTime()) === Seq("1")) } test("handle timeouts correctly with multiple executors") { @@ -186,7 +186,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { // start exec 3 at 60s (should idle timeout at 120s, exec 1 should time out) clock.setTime(TimeUnit.SECONDS.toMillis(60)) monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", null)) - assert(monitor.timedOutExecutors(clock.getTimeMillis()) === Seq("1")) + assert(monitor.timedOutExecutors(clock.nanoTime()) === Seq("1")) // store block on exec 3 (should now idle time out at 180s) monitor.onBlockUpdated(rddUpdate(1, 0, "3")) @@ -196,11 +196,11 @@ class ExecutorMonitorSuite extends SparkFunSuite { // advance to 140s, remove block from exec 3 (time out immediately) clock.setTime(TimeUnit.SECONDS.toMillis(140)) monitor.onBlockUpdated(rddUpdate(1, 0, "3", level = StorageLevel.NONE)) - assert(monitor.timedOutExecutors(clock.getTimeMillis()).toSet === Set("1", "3")) + assert(monitor.timedOutExecutors(clock.nanoTime()).toSet === Set("1", "3")) // advance to 150s, now exec 2 should time out clock.setTime(TimeUnit.SECONDS.toMillis(150)) - assert(monitor.timedOutExecutors(clock.getTimeMillis()).toSet === Set("1", "2", "3")) + assert(monitor.timedOutExecutors(clock.nanoTime()).toSet === Set("1", "2", "3")) } test("SPARK-27677: don't track blocks stored on disk when using shuffle service") { @@ -410,9 +410,9 @@ class ExecutorMonitorSuite extends SparkFunSuite { assert(monitor.timedOutExecutors(idleDeadline).isEmpty) } - private def idleDeadline: Long = clock.getTimeMillis() + idleTimeoutMs + 1 - private def storageDeadline: Long = clock.getTimeMillis() + storageTimeoutMs + 1 - private def shuffleDeadline: Long = clock.getTimeMillis() + shuffleTimeoutMs + 1 + private def idleDeadline: Long = clock.nanoTime() + idleTimeoutNs + 1 + private def storageDeadline: Long = clock.nanoTime() + storageTimeoutNs + 1 + private def shuffleDeadline: Long = clock.nanoTime() + shuffleTimeoutNs + 1 private def stageInfo(id: Int, shuffleId: Int = -1): StageInfo = { new StageInfo(id, 0, s"stage$id", 1, Nil, Nil, "", From a988aaf3fa38e89a05ab644ce79ddb14af56859d Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 15 Oct 2019 12:42:42 +0800 Subject: [PATCH 0326/1387] [SPARK-29454][SQL] Reduce unsafeProjection times when read Parquet file use non-vectorized mode ### What changes were proposed in this pull request? There will be 2 times unsafeProjection convert operation When we read a Parquet data file use non-vectorized mode: 1. `ParquetGroupConverter` call unsafeProjection function to covert `SpecificInternalRow` to `UnsafeRow` every times when read Parquet data file use `ParquetRecordReader`. 2. `ParquetFileFormat` will call unsafeProjection function to covert this `UnsafeRow` to another `UnsafeRow` again when partitionSchema is not empty in DataSourceV1 branch, and `PartitionReaderWithPartitionValues` will always do this convert operation in DataSourceV2 branch. In this pr, remove `unsafeProjection` convert operation in `ParquetGroupConverter` and change `ParquetRecordReader` to produce `SpecificInternalRow` instead of `UnsafeRow`. ### Why are the changes needed? The first time convert in `ParquetGroupConverter` is redundant and `ParquetRecordReader` return a `InternalRow(SpecificInternalRow)` is enough. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit Test Closes #26106 from LuciferYang/spark-parquet-unsafe-projection. Authored-by: yangjie01 Signed-off-by: Wenchen Fan --- .../parquet/ParquetFileFormat.scala | 20 ++++++++----------- .../parquet/ParquetReadSupport.scala | 10 +++++----- .../parquet/ParquetRecordMaterializer.scala | 6 +++--- .../parquet/ParquetRowConverter.scala | 6 ++---- .../ParquetPartitionReaderFactory.scala | 11 +++++----- 5 files changed, 23 insertions(+), 30 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 815b62dfbf898..fb5daa4bfae3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -328,32 +328,28 @@ class ParquetFileFormat iter.asInstanceOf[Iterator[InternalRow]] } else { logDebug(s"Falling back to parquet-mr") - // ParquetRecordReader returns UnsafeRow + // ParquetRecordReader returns InternalRow val readSupport = new ParquetReadSupport(convertTz, enableVectorizedReader = false) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[UnsafeRow](readSupport, parquetFilter) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](readSupport) + new ParquetRecordReader[InternalRow](readSupport) } - val iter = new RecordReaderIterator(reader) + val iter = new RecordReaderIterator[InternalRow](reader) // SPARK-23457 Register a task completion listener before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) reader.initialize(split, hadoopAttemptContext) val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) - // This is a horrible erasure hack... if we type the iterator above, then it actually check - // the type in next() and we get a class cast exception. If we make that function return - // Object, then we can defer the cast until later! if (partitionSchema.length == 0) { // There is no partition columns - iter.asInstanceOf[Iterator[InternalRow]] + iter.map(unsafeProjection) } else { - iter.asInstanceOf[Iterator[InternalRow]] - .map(d => appendPartitionColumns(joinedRow(d, file.partitionValues))) + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 2c7231d2c3e0a..69c8bad5f1c83 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -29,13 +29,13 @@ import org.apache.parquet.schema._ import org.apache.parquet.schema.Type.Repetition import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** * A Parquet [[ReadSupport]] implementation for reading Parquet records as Catalyst - * [[UnsafeRow]]s. + * [[InternalRow]]s. * * The API interface of [[ReadSupport]] is a little bit over complicated because of historical * reasons. In older versions of parquet-mr (say 1.6.0rc3 and prior), [[ReadSupport]] need to be @@ -51,7 +51,7 @@ import org.apache.spark.sql.types._ */ class ParquetReadSupport(val convertTz: Option[TimeZone], enableVectorizedReader: Boolean) - extends ReadSupport[UnsafeRow] with Logging { + extends ReadSupport[InternalRow] with Logging { private var catalystRequestedSchema: StructType = _ def this() { @@ -114,13 +114,13 @@ class ParquetReadSupport(val convertTz: Option[TimeZone], /** * Called on executor side after [[init()]], before instantiating actual Parquet record readers. * Responsible for instantiating [[RecordMaterializer]], which is used for converting Parquet - * records to Catalyst [[UnsafeRow]]s. + * records to Catalyst [[InternalRow]]s. */ override def prepareForRead( conf: Configuration, keyValueMetaData: JMap[String, String], fileSchema: MessageType, - readContext: ReadContext): RecordMaterializer[UnsafeRow] = { + readContext: ReadContext): RecordMaterializer[InternalRow] = { val parquetRequestedSchema = readContext.getRequestedSchema new ParquetRecordMaterializer( parquetRequestedSchema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala index b2459dd0e8bba..3098a332d3027 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala @@ -22,7 +22,7 @@ import java.util.TimeZone import org.apache.parquet.io.api.{GroupConverter, RecordMaterializer} import org.apache.parquet.schema.MessageType -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.StructType /** @@ -37,12 +37,12 @@ private[parquet] class ParquetRecordMaterializer( catalystSchema: StructType, schemaConverter: ParquetToSparkSchemaConverter, convertTz: Option[TimeZone]) - extends RecordMaterializer[UnsafeRow] { + extends RecordMaterializer[InternalRow] { private val rootConverter = new ParquetRowConverter(schemaConverter, parquetSchema, catalystSchema, convertTz, NoopUpdater) - override def getCurrentRecord: UnsafeRow = rootConverter.currentRecord + override def getCurrentRecord: InternalRow = rootConverter.currentRecord override def getRootConverter: GroupConverter = rootConverter } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index b772b6b77d1ce..ff5c724375c33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -173,12 +173,10 @@ private[parquet] class ParquetRowConverter( private val currentRow = new SpecificInternalRow(catalystType.map(_.dataType)) - private val unsafeProjection = UnsafeProjection.create(catalystType) - /** - * The [[UnsafeRow]] converted from an entire Parquet record. + * The [[InternalRow]] converted from an entire Parquet record. */ - def currentRecord: UnsafeRow = unsafeProjection(currentRow) + def currentRecord: InternalRow = currentRow // Converters for each field. private val fieldConverters: Array[Converter with HasParentContainerUpdater] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala index ded145f7de94e..b2fc724057eba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala @@ -31,7 +31,6 @@ import org.apache.spark.TaskContext import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} import org.apache.spark.sql.execution.datasources.{PartitionedFile, RecordReaderIterator} @@ -176,7 +175,7 @@ case class ParquetPartitionReaderFactory( reader } - private def createRowBaseReader(file: PartitionedFile): RecordReader[Void, UnsafeRow] = { + private def createRowBaseReader(file: PartitionedFile): RecordReader[Void, InternalRow] = { buildReaderBase(file, createRowBaseParquetReader) } @@ -185,16 +184,16 @@ case class ParquetPartitionReaderFactory( partitionValues: InternalRow, hadoopAttemptContext: TaskAttemptContextImpl, pushed: Option[FilterPredicate], - convertTz: Option[TimeZone]): RecordReader[Void, UnsafeRow] = { + convertTz: Option[TimeZone]): RecordReader[Void, InternalRow] = { logDebug(s"Falling back to parquet-mr") val taskContext = Option(TaskContext.get()) - // ParquetRecordReader returns UnsafeRow + // ParquetRecordReader returns InternalRow val readSupport = new ParquetReadSupport(convertTz, enableVectorizedReader = false) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[UnsafeRow](readSupport, parquetFilter) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](readSupport) + new ParquetRecordReader[InternalRow](readSupport) } val iter = new RecordReaderIterator(reader) // SPARK-23457 Register a task completion listener before `initialization`. From 8915966bf4f45c69c234fbe49d4304c3d7faeb67 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 15 Oct 2019 15:05:49 +0200 Subject: [PATCH 0327/1387] [SPARK-29473][SQL] move statement logical plans to a new file ### What changes were proposed in this pull request? move the statement logical plans that were created for v2 commands to a new file `statements.scala`, under the same package of `v2Commands.scala`. This PR also includes some minor cleanups: 1. remove `private[sql]` from `ParsedStatement` as it's in the private package. 2. remove unnecessary override of `output` and `children`. 3. add missing classdoc. ### Why are the changes needed? Similar to https://github.com/apache/spark/pull/26111 , this is to better organize the logical plans of data source v2. It's a bit weird to put the statements in the package `org.apache.spark.sql.catalyst.plans.logical.sql` as `sql` is not a good sub-package name in Spark SQL. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26125 from cloud-fan/statement. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon --- .../sql/catalyst/analysis/Analyzer.scala | 1 - .../sql/catalyst/analysis/CheckAnalysis.scala | 1 - .../catalyst/analysis/ResolveCatalogs.scala | 1 - .../spark/sql/catalyst/dsl/package.scala | 1 - .../sql/catalyst/parser/AstBuilder.scala | 1 - .../logical/sql/AlterTableStatements.scala | 78 ----- .../logical/sql/AlterViewStatements.scala | 33 -- .../logical/sql/CreateTableStatement.scala | 58 ---- .../logical/sql/DeleteFromStatement.scala | 27 -- .../logical/sql/DescribeColumnStatement.scala | 23 -- .../logical/sql/DescribeTableStatement.scala | 25 -- .../logical/sql/DropTableStatement.scala | 34 -- .../plans/logical/sql/DropViewStatement.scala | 33 -- .../logical/sql/InsertIntoStatement.scala | 50 --- .../plans/logical/sql/ParsedStatement.scala | 49 --- .../logical/sql/ReplaceTableStatement.scala | 60 ---- .../logical/sql/ShowNamespacesStatement.scala | 24 -- .../logical/sql/ShowTablesStatement.scala | 24 -- .../logical/sql/UpdateTableStatement.scala | 27 -- .../plans/logical/sql/UseStatement.scala | 23 -- .../catalyst/plans/logical/statements.scala | 294 ++++++++++++++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 3 +- .../sql/catalyst/parser/PlanParserSuite.scala | 1 - .../apache/spark/sql/DataFrameWriter.scala | 3 +- .../analysis/ResolveSessionCatalog.scala | 5 +- .../datasources/DataSourceStrategy.scala | 3 +- .../datasources/FallBackFileSourceV2.scala | 3 +- .../sql/execution/datasources/rules.scala | 1 - .../apache/spark/sql/SQLQueryTestSuite.scala | 1 - .../sql/execution/SparkSqlParserSuite.scala | 3 +- .../sql/util/DataFrameCallbackSuite.scala | 3 +- .../spark/sql/hive/HiveStrategies.scala | 3 +- .../hive/execution/HiveComparisonTest.scala | 1 - 33 files changed, 303 insertions(+), 594 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterTableStatements.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/InsertIntoStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ParsedStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowTablesStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UpdateTableStatement.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UseStatement.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b4d159eab4508..10e833c8a0756 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index e053d73c59d46..6a5d938f0fdc6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 981ef16459408..5689446785441 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index d37d81753f0b8..c1cef013eab1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql._ import org.apache.spark.sql.types._ /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index a731b994dd5a4..d138ff401944f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -36,7 +36,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement, UpdateTableStatement, UseStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterTableStatements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterTableStatements.scala deleted file mode 100644 index 9d7dec9ae0ce0..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterTableStatements.scala +++ /dev/null @@ -1,78 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.types.DataType - -/** - * Column data as parsed by ALTER TABLE ... ADD COLUMNS. - */ -case class QualifiedColType(name: Seq[String], dataType: DataType, comment: Option[String]) - -/** - * ALTER TABLE ... ADD COLUMNS command, as parsed from SQL. - */ -case class AlterTableAddColumnsStatement( - tableName: Seq[String], - columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement - -/** - * ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL. - */ -case class AlterTableAlterColumnStatement( - tableName: Seq[String], - column: Seq[String], - dataType: Option[DataType], - comment: Option[String]) extends ParsedStatement - -/** - * ALTER TABLE ... RENAME COLUMN command, as parsed from SQL. - */ -case class AlterTableRenameColumnStatement( - tableName: Seq[String], - column: Seq[String], - newName: String) extends ParsedStatement - -/** - * ALTER TABLE ... DROP COLUMNS command, as parsed from SQL. - */ -case class AlterTableDropColumnsStatement( - tableName: Seq[String], - columnsToDrop: Seq[Seq[String]]) extends ParsedStatement - -/** - * ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterTableSetPropertiesStatement( - tableName: Seq[String], - properties: Map[String, String]) extends ParsedStatement - -/** - * ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterTableUnsetPropertiesStatement( - tableName: Seq[String], - propertyKeys: Seq[String], - ifExists: Boolean) extends ParsedStatement - -/** - * ALTER TABLE ... SET LOCATION command, as parsed from SQL. - */ -case class AlterTableSetLocationStatement( - tableName: Seq[String], - location: String) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala deleted file mode 100644 index bba7f12c94e50..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala +++ /dev/null @@ -1,33 +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.sql.catalyst.plans.logical.sql - -/** - * ALTER VIEW ... SET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterViewSetPropertiesStatement( - viewName: Seq[String], - properties: Map[String, String]) extends ParsedStatement - -/** - * ALTER VIEW ... UNSET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterViewUnsetPropertiesStatement( - viewName: Seq[String], - propertyKeys: Seq[String], - ifExists: Boolean) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala deleted file mode 100644 index 551a0ce44b4d5..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala +++ /dev/null @@ -1,58 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.types.StructType - -/** - * A CREATE TABLE command, as parsed from SQL. - * - * This is a metadata-only command and is not used to write data to the created table. - */ -case class CreateTableStatement( - tableName: Seq[String], - tableSchema: StructType, - partitioning: Seq[Transform], - bucketSpec: Option[BucketSpec], - properties: Map[String, String], - provider: String, - options: Map[String, String], - location: Option[String], - comment: Option[String], - ifNotExists: Boolean) extends ParsedStatement - -/** - * A CREATE TABLE AS SELECT command, as parsed from SQL. - */ -case class CreateTableAsSelectStatement( - tableName: Seq[String], - asSelect: LogicalPlan, - partitioning: Seq[Transform], - bucketSpec: Option[BucketSpec], - properties: Map[String, String], - provider: String, - options: Map[String, String], - location: Option[String], - comment: Option[String], - ifNotExists: Boolean) extends ParsedStatement { - - override def children: Seq[LogicalPlan] = Seq(asSelect) -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala deleted file mode 100644 index 035bc09d31871..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DeleteFromStatement.scala +++ /dev/null @@ -1,27 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -case class DeleteFromStatement( - tableName: Seq[String], - tableAlias: Option[String], - condition: Option[Expression]) - extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala deleted file mode 100644 index 06b9f3e9a9a3e..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeColumnStatement.scala +++ /dev/null @@ -1,23 +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.sql.catalyst.plans.logical.sql - -case class DescribeColumnStatement( - tableName: Seq[String], - colNameParts: Seq[String], - isExtended: Boolean) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala deleted file mode 100644 index 02604b4ac5ac1..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DescribeTableStatement.scala +++ /dev/null @@ -1,25 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec - -case class DescribeTableStatement( - tableName: Seq[String], - partitionSpec: TablePartitionSpec, - isExtended: Boolean) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala deleted file mode 100644 index d41e8a5010257..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala +++ /dev/null @@ -1,34 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -/** - * A DROP TABLE statement, as parsed from SQL. - */ -case class DropTableStatement( - tableName: Seq[String], - ifExists: Boolean, - purge: Boolean) extends ParsedStatement { - - override def output: Seq[Attribute] = Seq.empty - - override def children: Seq[LogicalPlan] = Seq.empty -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala deleted file mode 100644 index 523158788e834..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala +++ /dev/null @@ -1,33 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -/** - * A DROP VIEW statement, as parsed from SQL. - */ -case class DropViewStatement( - viewName: Seq[String], - ifExists: Boolean) extends ParsedStatement { - - override def output: Seq[Attribute] = Seq.empty - - override def children: Seq[LogicalPlan] = Seq.empty -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/InsertIntoStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/InsertIntoStatement.scala deleted file mode 100644 index c4210eabe26a3..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/InsertIntoStatement.scala +++ /dev/null @@ -1,50 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -/** - * An INSERT INTO statement, as parsed from SQL. - * - * @param table the logical plan representing the table. - * @param query the logical plan representing data to write to. - * @param overwrite overwrite existing table or partitions. - * @param partitionSpec a map from the partition key to the partition value (optional). - * If the value is missing, dynamic partition insert will be performed. - * As an example, `INSERT INTO tbl PARTITION (a=1, b=2) AS` would have - * Map('a' -> Some('1'), 'b' -> Some('2')), - * and `INSERT INTO tbl PARTITION (a=1, b) AS ...` - * would have Map('a' -> Some('1'), 'b' -> None). - * @param ifPartitionNotExists If true, only write if the partition does not exist. - * Only valid for static partitions. - */ -case class InsertIntoStatement( - table: LogicalPlan, - partitionSpec: Map[String, Option[String]], - query: LogicalPlan, - overwrite: Boolean, - ifPartitionNotExists: Boolean) extends ParsedStatement { - - require(overwrite || !ifPartitionNotExists, - "IF NOT EXISTS is only valid in INSERT OVERWRITE") - require(partitionSpec.values.forall(_.nonEmpty) || !ifPartitionNotExists, - "IF NOT EXISTS is only valid with static partitions") - - override def children: Seq[LogicalPlan] = query :: Nil -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ParsedStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ParsedStatement.scala deleted file mode 100644 index 23fc009fecdc2..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ParsedStatement.scala +++ /dev/null @@ -1,49 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -/** - * A logical plan node that contains exactly what was parsed from SQL. - * - * This is used to hold information parsed from SQL when there are multiple implementations of a - * query or command. For example, CREATE TABLE may be implemented by different nodes for v1 and v2. - * Instead of parsing directly to a v1 CreateTable that keeps metadata in CatalogTable, and then - * converting that v1 metadata to the v2 equivalent, the sql [[CreateTableStatement]] plan is - * produced by the parser and converted once into both implementations. - * - * Parsed logical plans are not resolved because they must be converted to concrete logical plans. - * - * Parsed logical plans are located in Catalyst so that as much SQL parsing logic as possible is be - * kept in a [[org.apache.spark.sql.catalyst.parser.AbstractSqlParser]]. - */ -private[sql] abstract class ParsedStatement extends LogicalPlan { - // Redact properties and options when parsed nodes are used by generic methods like toString - override def productIterator: Iterator[Any] = super.productIterator.map { - case mapArg: Map[_, _] => conf.redactOptions(mapArg) - case other => other - } - - override def output: Seq[Attribute] = Seq.empty - - override def children: Seq[LogicalPlan] = Seq.empty - - final override lazy val resolved = false -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala deleted file mode 100644 index b9767ff9b889a..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala +++ /dev/null @@ -1,60 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.types.StructType - -/** - * A REPLACE TABLE command, as parsed from SQL. - * - * If the table exists prior to running this command, executing this statement - * will replace the table's metadata and clear the underlying rows from the table. - */ -case class ReplaceTableStatement( - tableName: Seq[String], - tableSchema: StructType, - partitioning: Seq[Transform], - bucketSpec: Option[BucketSpec], - properties: Map[String, String], - provider: String, - options: Map[String, String], - location: Option[String], - comment: Option[String], - orCreate: Boolean) extends ParsedStatement - -/** - * A REPLACE TABLE AS SELECT command, as parsed from SQL. - */ -case class ReplaceTableAsSelectStatement( - tableName: Seq[String], - asSelect: LogicalPlan, - partitioning: Seq[Transform], - bucketSpec: Option[BucketSpec], - properties: Map[String, String], - provider: String, - options: Map[String, String], - location: Option[String], - comment: Option[String], - orCreate: Boolean) extends ParsedStatement { - - override def children: Seq[LogicalPlan] = Seq(asSelect) -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala deleted file mode 100644 index 95d48a8e80d23..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowNamespacesStatement.scala +++ /dev/null @@ -1,24 +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.sql.catalyst.plans.logical.sql - -/** - * A SHOW NAMESPACES statement, as parsed from SQL. - */ -case class ShowNamespacesStatement(namespace: Option[Seq[String]], pattern: Option[String]) - extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowTablesStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowTablesStatement.scala deleted file mode 100644 index d75c4085a974b..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ShowTablesStatement.scala +++ /dev/null @@ -1,24 +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.sql.catalyst.plans.logical.sql - -/** - * A SHOW TABLES statement, as parsed from SQL. - */ -case class ShowTablesStatement(namespace: Option[Seq[String]], pattern: Option[String]) - extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UpdateTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UpdateTableStatement.scala deleted file mode 100644 index 84b6d3d5d0b83..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UpdateTableStatement.scala +++ /dev/null @@ -1,27 +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.sql.catalyst.plans.logical.sql - -import org.apache.spark.sql.catalyst.expressions.Expression - -case class UpdateTableStatement( - tableName: Seq[String], - tableAlias: Option[String], - columns: Seq[Seq[String]], - values: Seq[Expression], - condition: Option[Expression]) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UseStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UseStatement.scala deleted file mode 100644 index 1fd76a252d826..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/UseStatement.scala +++ /dev/null @@ -1,23 +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.sql.catalyst.plans.logical.sql - -/** - * A USE statement, as parsed from SQL. - */ -case class UseStatement(isNamespaceSet: Boolean, nameParts: Seq[String]) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala new file mode 100644 index 0000000000000..757a8bfe219f3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -0,0 +1,294 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.{DataType, StructType} + +/** + * A logical plan node that contains exactly what was parsed from SQL. + * + * This is used to hold information parsed from SQL when there are multiple implementations of a + * query or command. For example, CREATE TABLE may be implemented by different nodes for v1 and v2. + * Instead of parsing directly to a v1 CreateTable that keeps metadata in CatalogTable, and then + * converting that v1 metadata to the v2 equivalent, the sql [[CreateTableStatement]] plan is + * produced by the parser and converted once into both implementations. + * + * Parsed logical plans are not resolved because they must be converted to concrete logical plans. + * + * Parsed logical plans are located in Catalyst so that as much SQL parsing logic as possible is be + * kept in a [[org.apache.spark.sql.catalyst.parser.AbstractSqlParser]]. + */ +abstract class ParsedStatement extends LogicalPlan { + // Redact properties and options when parsed nodes are used by generic methods like toString + override def productIterator: Iterator[Any] = super.productIterator.map { + case mapArg: Map[_, _] => conf.redactOptions(mapArg) + case other => other + } + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.empty + + final override lazy val resolved = false +} + +/** + * A CREATE TABLE command, as parsed from SQL. + * + * This is a metadata-only command and is not used to write data to the created table. + */ +case class CreateTableStatement( + tableName: Seq[String], + tableSchema: StructType, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + ifNotExists: Boolean) extends ParsedStatement + +/** + * A CREATE TABLE AS SELECT command, as parsed from SQL. + */ +case class CreateTableAsSelectStatement( + tableName: Seq[String], + asSelect: LogicalPlan, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + ifNotExists: Boolean) extends ParsedStatement { + + override def children: Seq[LogicalPlan] = Seq(asSelect) +} + +/** + * A REPLACE TABLE command, as parsed from SQL. + * + * If the table exists prior to running this command, executing this statement + * will replace the table's metadata and clear the underlying rows from the table. + */ +case class ReplaceTableStatement( + tableName: Seq[String], + tableSchema: StructType, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + orCreate: Boolean) extends ParsedStatement + +/** + * A REPLACE TABLE AS SELECT command, as parsed from SQL. + */ +case class ReplaceTableAsSelectStatement( + tableName: Seq[String], + asSelect: LogicalPlan, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + orCreate: Boolean) extends ParsedStatement { + + override def children: Seq[LogicalPlan] = Seq(asSelect) +} + + +/** + * Column data as parsed by ALTER TABLE ... ADD COLUMNS. + */ +case class QualifiedColType(name: Seq[String], dataType: DataType, comment: Option[String]) + +/** + * ALTER TABLE ... ADD COLUMNS command, as parsed from SQL. + */ +case class AlterTableAddColumnsStatement( + tableName: Seq[String], + columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement + +/** + * ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL. + */ +case class AlterTableAlterColumnStatement( + tableName: Seq[String], + column: Seq[String], + dataType: Option[DataType], + comment: Option[String]) extends ParsedStatement + +/** + * ALTER TABLE ... RENAME COLUMN command, as parsed from SQL. + */ +case class AlterTableRenameColumnStatement( + tableName: Seq[String], + column: Seq[String], + newName: String) extends ParsedStatement + +/** + * ALTER TABLE ... DROP COLUMNS command, as parsed from SQL. + */ +case class AlterTableDropColumnsStatement( + tableName: Seq[String], + columnsToDrop: Seq[Seq[String]]) extends ParsedStatement + +/** + * ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterTableSetPropertiesStatement( + tableName: Seq[String], + properties: Map[String, String]) extends ParsedStatement + +/** + * ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterTableUnsetPropertiesStatement( + tableName: Seq[String], + propertyKeys: Seq[String], + ifExists: Boolean) extends ParsedStatement + +/** + * ALTER TABLE ... SET LOCATION command, as parsed from SQL. + */ +case class AlterTableSetLocationStatement( + tableName: Seq[String], + location: String) extends ParsedStatement + +/** + * ALTER VIEW ... SET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterViewSetPropertiesStatement( + viewName: Seq[String], + properties: Map[String, String]) extends ParsedStatement + +/** + * ALTER VIEW ... UNSET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterViewUnsetPropertiesStatement( + viewName: Seq[String], + propertyKeys: Seq[String], + ifExists: Boolean) extends ParsedStatement + + +/** + * A DROP TABLE statement, as parsed from SQL. + */ +case class DropTableStatement( + tableName: Seq[String], + ifExists: Boolean, + purge: Boolean) extends ParsedStatement + +/** + * A DROP VIEW statement, as parsed from SQL. + */ +case class DropViewStatement( + viewName: Seq[String], + ifExists: Boolean) extends ParsedStatement + +/** + * A DESCRIBE TABLE tbl_name statement, as parsed from SQL. + */ +case class DescribeTableStatement( + tableName: Seq[String], + partitionSpec: TablePartitionSpec, + isExtended: Boolean) extends ParsedStatement + +/** + * A DESCRIBE TABLE tbl_name col_name statement, as parsed from SQL. + */ +case class DescribeColumnStatement( + tableName: Seq[String], + colNameParts: Seq[String], + isExtended: Boolean) extends ParsedStatement + +/** + * A DELETE FROM statement, as parsed from SQL. + */ +case class DeleteFromStatement( + tableName: Seq[String], + tableAlias: Option[String], + condition: Option[Expression]) extends ParsedStatement + +/** + * A UPDATE tbl_name statement, as parsed from SQL. + */ +case class UpdateTableStatement( + tableName: Seq[String], + tableAlias: Option[String], + columns: Seq[Seq[String]], + values: Seq[Expression], + condition: Option[Expression]) extends ParsedStatement + +/** + * An INSERT INTO statement, as parsed from SQL. + * + * @param table the logical plan representing the table. + * @param query the logical plan representing data to write to. + * @param overwrite overwrite existing table or partitions. + * @param partitionSpec a map from the partition key to the partition value (optional). + * If the value is missing, dynamic partition insert will be performed. + * As an example, `INSERT INTO tbl PARTITION (a=1, b=2) AS` would have + * Map('a' -> Some('1'), 'b' -> Some('2')), + * and `INSERT INTO tbl PARTITION (a=1, b) AS ...` + * would have Map('a' -> Some('1'), 'b' -> None). + * @param ifPartitionNotExists If true, only write if the partition does not exist. + * Only valid for static partitions. + */ +case class InsertIntoStatement( + table: LogicalPlan, + partitionSpec: Map[String, Option[String]], + query: LogicalPlan, + overwrite: Boolean, + ifPartitionNotExists: Boolean) extends ParsedStatement { + + require(overwrite || !ifPartitionNotExists, + "IF NOT EXISTS is only valid in INSERT OVERWRITE") + require(partitionSpec.values.forall(_.nonEmpty) || !ifPartitionNotExists, + "IF NOT EXISTS is only valid with static partitions") + + override def children: Seq[LogicalPlan] = query :: Nil +} + +/** + * A SHOW TABLES statement, as parsed from SQL. + */ +case class ShowTablesStatement(namespace: Option[Seq[String]], pattern: Option[String]) + extends ParsedStatement + +/** + * A SHOW NAMESPACES statement, as parsed from SQL. + */ +case class ShowNamespacesStatement(namespace: Option[Seq[String]], pattern: Option[String]) + extends ParsedStatement + +/** + * A USE statement, as parsed from SQL. + */ +case class UseStatement(isNamespaceSet: Boolean, nameParts: Seq[String]) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index df1bbe9554f42..d9e50ef09fdd5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -23,8 +23,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DeleteFromStatement, DescribeColumnStatement, DescribeTableStatement, DropTableStatement, DropViewStatement, InsertIntoStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement, ShowNamespacesStatement, ShowTablesStatement, UpdateTableStatement} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 61f8c3b99149a..8b940a7aa2c3b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, Un import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index a7c4eef4e19f7..3d04a0616e96b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -26,8 +26,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, NoSuchTableException, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, InsertIntoStatement, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsWrite, TableCatalog, TableProvider, V1Table} import org.apache.spark.sql.connector.catalog.TableCapability._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index e23b617af48b3..a96533dac97ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -21,13 +21,12 @@ import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, Table, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2} +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 1a92a2bb7156c..211642d78aabe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -34,8 +34,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala index 70e78a9ecc859..5f6c3e8f7eeed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.execution.datasources import scala.collection.JavaConverters._ import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2, FileTable} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index c92c68095db8f..58e9f8941887f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.expressions.{FieldReference, RewritableTransform} import org.apache.spark.sql.execution.command.DDLUtils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index c74fa2da42afa..127deea1cecc7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -25,7 +25,6 @@ import scala.util.control.NonFatal import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} import org.apache.spark.sql.execution.HiveResult.hiveResultString diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index b751fb7c50438..9da160888618f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -22,8 +22,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} -import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} +import org.apache.spark.sql.catalyst.plans.logical.{DescribeColumnStatement, DescribeTableStatement, LogicalPlan, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index e96f05384c879..083b40d0680aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -22,8 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.sql.{functions, AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.execution.datasources.json.JsonFileFormat diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 56df50827f00d..33ca1889e944d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -26,8 +26,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, LogicalPlan, ScriptTransformation, Statistics} -import org.apache.spark.sql.catalyst.plans.logical.sql.InsertIntoStatement +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, ScriptTransformation, Statistics} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index f627974a62144..8bf2e19b929e7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -30,7 +30,6 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{DescribeColumnStatement, DescribeTableStatement} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.SQLExecution From 9ac4b2dbc55a1114fea313483e5543d83c352213 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 15 Oct 2019 21:51:15 +0800 Subject: [PATCH 0328/1387] [SPARK-28560][SQL] Optimize shuffle reader to local shuffle reader when smj converted to bhj in adaptive execution ## What changes were proposed in this pull request? Implement a rule in the new adaptive execution framework introduced in [SPARK-23128](https://issues.apache.org/jira/browse/SPARK-23128). This rule is used to optimize the shuffle reader to local shuffle reader when smj is converted to bhj in adaptive execution. ## How was this patch tested? Existing tests Closes #25295 from JkSelf/localShuffleOptimization. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../org/apache/spark/MapOutputTracker.scala | 91 +++++++++++- .../shuffle/BlockStoreShuffleReader.scala | 19 ++- .../apache/spark/shuffle/ShuffleManager.scala | 13 ++ .../shuffle/sort/SortShuffleManager.scala | 21 +++ .../apache/spark/sql/internal/SQLConf.scala | 8 ++ .../spark/sql/execution/SparkPlanInfo.scala | 3 +- .../adaptive/AdaptiveSparkPlanExec.scala | 1 + .../adaptive/AdaptiveSparkPlanHelper.scala | 1 + .../adaptive/LocalShuffledRowRDD.scala | 98 +++++++++++++ .../adaptive/OptimizeLocalShuffleReader.scala | 132 ++++++++++++++++++ .../exchange/ShuffleExchangeExec.scala | 5 + .../adaptive/AdaptiveQueryExecSuite.scala | 46 ++++-- 12 files changed, 424 insertions(+), 14 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 3b0062efeff01..ef4c421cbf829 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -33,7 +33,7 @@ import org.apache.spark.broadcast.{Broadcast, BroadcastManager} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} -import org.apache.spark.scheduler.MapStatus +import org.apache.spark.scheduler.{ExecutorCacheTaskLocation, MapStatus} import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} import org.apache.spark.util._ @@ -337,6 +337,21 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging endPartition: Int) : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] + /** + * Called from executors to get the server URIs and output sizes for each shuffle block that + * needs to be read from a given range of map output partitions (startPartition is included but + * endPartition is excluded from the range) and a given mapId. + * + * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, + * and the second item is a sequence of (shuffle block id, shuffle block size, map index) + * tuples describing the shuffle blocks that are stored at that block manager. + */ + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + mapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] + /** * Deletes map output status information for the specified shuffle stage. */ @@ -668,6 +683,31 @@ private[spark] class MapOutputTrackerMaster( None } + /** + * Return the location where the Mapper ran. The locations each includes both a host and an + * executor id on that host. + * + * @param dep shuffle dependency object + * @param mapId the map id + * @return a sequence of locations where task runs. + */ + def getMapLocation(dep: ShuffleDependency[_, _, _], mapId: Int): Seq[String] = + { + val shuffleStatus = shuffleStatuses.get(dep.shuffleId).orNull + if (shuffleStatus != null) { + shuffleStatus.withMapStatuses { statuses => + if (mapId >= 0 && mapId < statuses.length) { + Seq( ExecutorCacheTaskLocation(statuses(mapId).location.host, + statuses(mapId).location.executorId).toString) + } else { + Nil + } + } + } else { + Nil + } + } + def incrementEpoch(): Unit = { epochLock.synchronized { epoch += 1 @@ -701,6 +741,29 @@ private[spark] class MapOutputTrackerMaster( } } + override def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + mapId: Int) + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + logDebug(s"Fetching outputs for shuffle $shuffleId, mapId $mapId" + + s"partitions $startPartition-$endPartition") + shuffleStatuses.get(shuffleId) match { + case Some (shuffleStatus) => + shuffleStatus.withMapStatuses { statuses => + MapOutputTracker.convertMapStatuses( + shuffleId, + startPartition, + endPartition, + statuses, + Some(mapId)) + } + case None => + Iterator.empty + } + } + override def stop(): Unit = { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() @@ -746,6 +809,25 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } + override def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + mapId: Int) : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + logDebug(s"Fetching outputs for shuffle $shuffleId, mapId $mapId" + + s"partitions $startPartition-$endPartition") + val statuses = getStatuses(shuffleId) + try { + MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, + statuses, Some(mapId)) + } catch { + case e: MetadataFetchFailedException => + // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: + mapStatuses.clear() + throw e + } + } + /** * Get or fetch the array of MapStatuses for a given shuffle ID. NOTE: clients MUST synchronize * on this array when reading it, because on the driver, we may be changing it in place. @@ -888,10 +970,12 @@ private[spark] object MapOutputTracker extends Logging { shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + statuses: Array[MapStatus], + mapId : Option[Int] = None): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] - for ((status, mapIndex) <- statuses.iterator.zipWithIndex) { + val iter = statuses.iterator.zipWithIndex + for ((status, mapIndex) <- mapId.map(id => iter.filter(_._2 == id)).getOrElse(iter)) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) @@ -906,6 +990,7 @@ private[spark] object MapOutputTracker extends Logging { } } } + splitsByAddress.iterator } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 4329824b1b627..242442ac9d8f2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -36,18 +36,33 @@ private[spark] class BlockStoreShuffleReader[K, C]( readMetrics: ShuffleReadMetricsReporter, serializerManager: SerializerManager = SparkEnv.get.serializerManager, blockManager: BlockManager = SparkEnv.get.blockManager, - mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker) + mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, + mapId: Option[Int] = None) extends ShuffleReader[K, C] with Logging { private val dep = handle.dependency /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { + val blocksByAddress = mapId match { + case (Some(mapId)) => mapOutputTracker.getMapSizesByExecutorId( + handle.shuffleId, + startPartition, + endPartition, + mapId) + case (None) => mapOutputTracker.getMapSizesByExecutorId( + handle.shuffleId, + startPartition, + endPartition) + case (_) => throw new IllegalArgumentException( + "mapId should be both set or unset") + } + val wrappedStreams = new ShuffleBlockFetcherIterator( context, blockManager.blockStoreClient, blockManager, - mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), + blocksByAddress, serializerManager.wrapStream, // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index a717ef242ea7c..0041dca507c0f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -54,6 +54,19 @@ private[spark] trait ShuffleManager { context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] + /** + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to + * read from mapId. + * Called on executors by reduce tasks. + */ + def getMapReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter, + mapId: Int): ShuffleReader[K, C] + /** * Remove a shuffle's metadata from the ShuffleManager. * @return true if the metadata removed successfully, otherwise false. diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index d96bcb3d073df..b21ce9ce0fc71 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -127,6 +127,27 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager startPartition, endPartition, context, metrics) } + /** + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to + * read from mapId. + * Called on executors by reduce tasks. + */ + override def getMapReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter, + mapId: Int): ShuffleReader[K, C] = { + new BlockStoreShuffleReader( + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], + startPartition, + endPartition, + context, + metrics, + mapId = Some(mapId)) + } + /** Get a writer for a given partition. Called on executors by map tasks. */ override def getWriter[K, V]( handle: ShuffleHandle, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index bcb3153a3ca48..f00a4b545ee3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -394,6 +394,14 @@ object SQLConf { "must be a positive integer.") .createOptional + val OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED = + buildConf("spark.sql.adaptive.optimizedLocalShuffleReader.enabled") + .doc("When true and adaptive execution is enabled, this enables the optimization of" + + " converting the shuffle reader to local shuffle reader for the shuffle exchange" + + " of the broadcast hash join in probe side.") + .booleanConf + .createWithDefault(true) + val SUBEXPRESSION_ELIMINATION_ENABLED = buildConf("spark.sql.subexpressionElimination.enabled") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 8c7752c4bb742..459311df22d23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, LocalShuffleReaderExec, QueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo import org.apache.spark.sql.internal.SQLConf @@ -56,6 +56,7 @@ private[execution] object SparkPlanInfo { case ReusedSubqueryExec(child) => child :: Nil case a: AdaptiveSparkPlanExec => a.executedPlan :: Nil case stage: QueryStageExec => stage.plan :: Nil + case localReader: LocalShuffleReaderExec => localReader.child :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 524cacc11484c..f45e3560b2cf1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -84,6 +84,7 @@ case class AdaptiveSparkPlanExec( // plan should reach a final status of query stages (i.e., no more addition or removal of // Exchange nodes) after running these rules. private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( + OptimizeLocalShuffleReader(conf), ensureRequirements ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala index 0ec8710e4db43..94e66b0c3a430 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala @@ -125,6 +125,7 @@ trait AdaptiveSparkPlanHelper { private def allChildren(p: SparkPlan): Seq[SparkPlan] = p match { case a: AdaptiveSparkPlanExec => Seq(a.executedPlan) case s: QueryStageExec => Seq(s.plan) + case l: LocalShuffleReaderExec => Seq(l.child) case _ => p.children } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala new file mode 100644 index 0000000000000..9ad1ebaf6f376 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -0,0 +1,98 @@ +/* + * 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.adaptive + +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} + + +/** + * The [[Partition]] used by [[LocalShuffledRowRDD]]. A pre-shuffle partition + * (identified by `preShufflePartitionIndex`) contains a range of post-shuffle partitions + * (`startPostShufflePartitionIndex` to `endPostShufflePartitionIndex - 1`, inclusive). + */ +private final class LocalShuffleRowRDDPartition( + val preShufflePartitionIndex: Int) extends Partition { + override val index: Int = preShufflePartitionIndex +} + +/** + * This is a specialized version of [[org.apache.spark.sql.execution.ShuffledRowRDD]]. This is used + * in Spark SQL adaptive execution when a shuffle join is converted to broadcast join at runtime + * because the map output of one input table is small enough for broadcast. This RDD represents the + * data of another input table of the join that reads from shuffle. Each partition of the RDD reads + * the whole data from just one mapper output locally. So actually there is no data transferred + * from the network. + + * This RDD takes a [[ShuffleDependency]] (`dependency`). + * + * The `dependency` has the parent RDD of this RDD, which represents the dataset before shuffle + * (i.e. map output). Elements of this RDD are (partitionId, Row) pairs. + * Partition ids should be in the range [0, numPartitions - 1]. + * `dependency.partitioner.numPartitions` is the number of pre-shuffle partitions. (i.e. the number + * of partitions of the map output). The post-shuffle partition number is the same to the parent + * RDD's partition number. + */ +class LocalShuffledRowRDD( + var dependency: ShuffleDependency[Int, InternalRow, InternalRow], + metrics: Map[String, SQLMetric]) + extends RDD[InternalRow](dependency.rdd.context, Nil) { + + private[this] val numReducers = dependency.partitioner.numPartitions + private[this] val numMappers = dependency.rdd.partitions.length + + override def getDependencies: Seq[Dependency[_]] = List(dependency) + + override def getPartitions: Array[Partition] = { + + Array.tabulate[Partition](numMappers) { i => + new LocalShuffleRowRDDPartition(i) + } + } + + override def getPreferredLocations(partition: Partition): Seq[String] = { + val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + tracker.getMapLocation(dependency, partition.index) + } + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val localRowPartition = split.asInstanceOf[LocalShuffleRowRDDPartition] + val mapId = localRowPartition.index + val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() + // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, + // as well as the `tempMetrics` for basic shuffle metrics. + val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) + + val reader = SparkEnv.get.shuffleManager.getMapReader( + dependency.shuffleHandle, + 0, + numReducers, + context, + sqlMetricsReporter, + mapId) + reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) + } + + override def clearDependencies() { + super.clearDependencies() + dependency = null + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala new file mode 100644 index 0000000000000..308e65e793d8f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -0,0 +1,132 @@ +/* + * 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.adaptive + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchangeExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight} +import org.apache.spark.sql.internal.SQLConf + +case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { + + def canUseLocalShuffleReaderLeft(join: BroadcastHashJoinExec): Boolean = { + join.buildSide == BuildRight && ShuffleQueryStageExec.isShuffleQueryStageExec(join.left) + } + + def canUseLocalShuffleReaderRight(join: BroadcastHashJoinExec): Boolean = { + join.buildSide == BuildLeft && ShuffleQueryStageExec.isShuffleQueryStageExec(join.right) + } + + override def apply(plan: SparkPlan): SparkPlan = { + if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { + return plan + } + + val optimizedPlan = plan.transformDown { + case join: BroadcastHashJoinExec if canUseLocalShuffleReaderRight(join) => + val localReader = LocalShuffleReaderExec(join.right.asInstanceOf[QueryStageExec]) + join.copy(right = localReader) + case join: BroadcastHashJoinExec if canUseLocalShuffleReaderLeft(join) => + val localReader = LocalShuffleReaderExec(join.left.asInstanceOf[QueryStageExec]) + join.copy(left = localReader) + } + + def numExchanges(plan: SparkPlan): Int = { + plan.collect { + case e: ShuffleExchangeExec => e + }.length + } + + val numExchangeBefore = numExchanges(EnsureRequirements(conf).apply(plan)) + val numExchangeAfter = numExchanges(EnsureRequirements(conf).apply(optimizedPlan)) + + if (numExchangeAfter > numExchangeBefore) { + logWarning("OptimizeLocalShuffleReader rule is not applied due" + + " to additional shuffles will be introduced.") + plan + } else { + optimizedPlan + } + } +} + +case class LocalShuffleReaderExec(child: QueryStageExec) extends LeafExecNode { + + override def output: Seq[Attribute] = child.output + + override def doCanonicalize(): SparkPlan = child.canonicalized + + override def outputPartitioning: Partitioning = { + + def tryReserveChildPartitioning(stage: ShuffleQueryStageExec): Partitioning = { + val initialPartitioning = stage.plan.child.outputPartitioning + if (initialPartitioning.isInstanceOf[UnknownPartitioning]) { + UnknownPartitioning(stage.plan.shuffleDependency.rdd.partitions.length) + } else { + initialPartitioning + } + } + + child match { + case stage: ShuffleQueryStageExec => + tryReserveChildPartitioning(stage) + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => + tryReserveChildPartitioning(stage) + } + } + + private var cachedShuffleRDD: RDD[InternalRow] = null + + override protected def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = child match { + case stage: ShuffleQueryStageExec => + stage.plan.createLocalShuffleRDD() + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => + stage.plan.createLocalShuffleRDD() + } + } + cachedShuffleRDD + } + + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + append: String => Unit, + verbose: Boolean, + prefix: String = "", + addSuffix: Boolean = false, + maxFields: Int, + printNodeId: Boolean): Unit = { + super.generateTreeString(depth, + lastChildren, + append, + verbose, + prefix, + addSuffix, + maxFields, + printNodeId) + child.generateTreeString( + depth + 1, lastChildren :+ true, append, verbose, "", false, maxFields, printNodeId) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 2f4c5734469f8..2f94c522712b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Uns import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.LocalShuffledRowRDD import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -82,6 +83,10 @@ case class ShuffleExchangeExec( new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) } + def createLocalShuffleRDD(): LocalShuffledRowRDD = { + new LocalShuffledRowRDD(shuffleDependency, readMetrics) + } + /** * Caches the created ShuffleRowRDD so we can reuse that. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 37b106c3ea530..cd0bf726da9aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -77,6 +77,13 @@ class AdaptiveQueryExecSuite } } + private def checkNumLocalShuffleReaders(plan: SparkPlan, expected: Int): Unit = { + val localReaders = plan.collect { + case reader: LocalShuffleReaderExec => reader + } + assert(localReaders.length === expected) + } + test("Change merge join to broadcast join") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", @@ -87,6 +94,7 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -103,14 +111,7 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - val shuffleReaders = adaptivePlan.collect { - case reader: CoalescedShuffleReaderExec => reader - } - assert(shuffleReaders.length === 1) - // The pre-shuffle partition size is [0, 72, 0, 72, 126] - shuffleReaders.foreach { reader => - assert(reader.outputPartitioning.numPartitions === 2) - } + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -125,6 +126,7 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -139,6 +141,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -160,6 +164,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) + + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -183,6 +189,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) + + checkNumLocalShuffleReaders(adaptivePlan, 0) } } @@ -206,6 +214,7 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) + checkNumLocalShuffleReaders(adaptivePlan, 0) } } @@ -355,6 +364,27 @@ class AdaptiveQueryExecSuite } } + test("Change merge join to broadcast join without local shuffle reader") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |SELECT * FROM testData t1 join testData2 t2 + |ON t1.key = t2.a join testData3 t3 on t2.a = t3.a + |where t1.value = 1 + """.stripMargin + ) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 2) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + // additional shuffle exchange introduced, so revert OptimizeLocalShuffleReader rule. + checkNumLocalShuffleReaders(adaptivePlan, 0) + } + } + test("Avoid changing merge join to broadcast join if too many empty partitions on build plan") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", From 2e28622d8aeb9ce2460e803bb7d994196bcc0253 Mon Sep 17 00:00:00 2001 From: Yifei Huang Date: Tue, 15 Oct 2019 12:26:49 -0500 Subject: [PATCH 0329/1387] [SPARK-28211][CORE][SHUFFLE] Propose Shuffle Driver Components API ### What changes were proposed in this pull request? This is the next step of the Spark-25299 work of proposing a new Shuffle storage API. This patch includes the components of the plugin that hook into the driver, including driver shuffle initialization, application cleanup, and shuffle cleanup. ### How was this patch tested? Existing unit tests, plus an additional test for testing the interactions between the driver and executor initialization. Closes #25823 from yifeih/yh/upstream/driver-lifecycle. Lead-authored-by: Yifei Huang Co-authored-by: mccheah Signed-off-by: Imran Rashid --- .../spark/shuffle/api/ShuffleDataIO.java | 6 ++ .../shuffle/api/ShuffleDriverComponents.java | 64 +++++++++++++ .../api/ShuffleExecutorComponents.java | 12 ++- .../sort/io/LocalDiskShuffleDataIO.java | 8 +- .../io/LocalDiskShuffleDriverComponents.java | 49 ++++++++++ .../LocalDiskShuffleExecutorComponents.java | 7 +- .../org/apache/spark/ContextCleaner.scala | 8 +- .../scala/org/apache/spark/Dependency.scala | 1 + .../scala/org/apache/spark/SparkContext.scala | 17 +++- .../spark/shuffle/ShuffleDataIOUtils.scala | 42 +++++++++ .../shuffle/sort/SortShuffleManager.scala | 15 +-- .../spark/InternalAccumulatorSuite.scala | 3 +- .../ShuffleDriverComponentsSuite.scala | 94 +++++++++++++++++++ 13 files changed, 308 insertions(+), 18 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDriverComponents.java create mode 100644 core/src/main/scala/org/apache/spark/shuffle/ShuffleDataIOUtils.scala create mode 100644 core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java index e9e50ecc11e52..e4554bda8acab 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java @@ -46,4 +46,10 @@ public interface ShuffleDataIO { * are only invoked on the executors. */ ShuffleExecutorComponents executor(); + + /** + * Called once on driver process to bootstrap the shuffle metadata modules that + * are maintained by the driver. + */ + ShuffleDriverComponents driver(); } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java new file mode 100644 index 0000000000000..b4cec17b85b32 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDriverComponents.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.api; + +import java.util.Map; + +import org.apache.spark.annotation.Private; + +/** + * :: Private :: + * An interface for building shuffle support modules for the Driver. + */ +@Private +public interface ShuffleDriverComponents { + + /** + * Called once in the driver to bootstrap this module that is specific to this application. + * This method is called before submitting executor requests to the cluster manager. + * + * This method should prepare the module with its shuffle components i.e. registering against + * an external file servers or shuffle services, or creating tables in a shuffle + * storage data database. + * + * @return additional SparkConf settings necessary for initializing the executor components. + * This would include configurations that cannot be statically set on the application, like + * the host:port of external services for shuffle storage. + */ + Map initializeApplication(); + + /** + * Called once at the end of the Spark application to clean up any existing shuffle state. + */ + void cleanupApplication(); + + /** + * Called once per shuffle id when the shuffle id is first generated for a shuffle stage. + * + * @param shuffleId The unique identifier for the shuffle stage. + */ + default void registerShuffle(int shuffleId) {} + + /** + * Removes shuffle data associated with the given shuffle. + * + * @param shuffleId The unique identifier for the shuffle stage. + * @param blocking Whether this call should block on the deletion of the data. + */ + default void removeShuffle(int shuffleId, boolean blocking) {} +} diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index d30f3dad3c940..30ca177545789 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -18,6 +18,7 @@ package org.apache.spark.shuffle.api; import java.io.IOException; +import java.util.Map; import java.util.Optional; import org.apache.spark.annotation.Private; @@ -34,21 +35,26 @@ public interface ShuffleExecutorComponents { /** * Called once per executor to bootstrap this module with state that is specific to * that executor, specifically the application ID and executor ID. + * + * @param appId The Spark application id + * @param execId The unique identifier of the executor being initialized + * @param extraConfigs Extra configs that were returned by + * {@link ShuffleDriverComponents#initializeApplication()} */ - void initializeExecutor(String appId, String execId); + void initializeExecutor(String appId, String execId, Map extraConfigs); /** * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. * * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId An ID of the map task. The ID is unique within this Spark application. + * @param mapTaskId An ID of the map task. The ID is unique within this Spark application. * @param numPartitions The number of partitions that will be written by the map task. Some of * these partitions may be empty. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - long mapId, + long mapTaskId, int numPartitions) throws IOException; /** diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java index cabcb171ac23a..50eb2f1813714 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java @@ -18,8 +18,9 @@ package org.apache.spark.shuffle.sort.io; import org.apache.spark.SparkConf; -import org.apache.spark.shuffle.api.ShuffleExecutorComponents; import org.apache.spark.shuffle.api.ShuffleDataIO; +import org.apache.spark.shuffle.api.ShuffleDriverComponents; +import org.apache.spark.shuffle.api.ShuffleExecutorComponents; /** * Implementation of the {@link ShuffleDataIO} plugin system that replicates the local shuffle @@ -37,4 +38,9 @@ public LocalDiskShuffleDataIO(SparkConf sparkConf) { public ShuffleExecutorComponents executor() { return new LocalDiskShuffleExecutorComponents(sparkConf); } + + @Override + public ShuffleDriverComponents driver() { + return new LocalDiskShuffleDriverComponents(); + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDriverComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDriverComponents.java new file mode 100644 index 0000000000000..92b4b318c552d --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDriverComponents.java @@ -0,0 +1,49 @@ +/* + * 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.shuffle.sort.io; + +import java.util.Collections; +import java.util.Map; + +import org.apache.spark.SparkEnv; +import org.apache.spark.shuffle.api.ShuffleDriverComponents; +import org.apache.spark.storage.BlockManagerMaster; + +public class LocalDiskShuffleDriverComponents implements ShuffleDriverComponents { + + private BlockManagerMaster blockManagerMaster; + + @Override + public Map initializeApplication() { + blockManagerMaster = SparkEnv.get().blockManager().master(); + return Collections.emptyMap(); + } + + @Override + public void cleanupApplication() { + // nothing to clean up + } + + @Override + public void removeShuffle(int shuffleId, boolean blocking) { + if (blockManagerMaster == null) { + throw new IllegalStateException("Driver components must be initialized before using"); + } + blockManagerMaster.removeShuffle(shuffleId, blocking); + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index a0c7d3c248d48..eb4d9d9abc8e3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.sort.io; +import java.util.Map; import java.util.Optional; import com.google.common.annotations.VisibleForTesting; @@ -50,7 +51,7 @@ public LocalDiskShuffleExecutorComponents( } @Override - public void initializeExecutor(String appId, String execId) { + public void initializeExecutor(String appId, String execId, Map extraConfigs) { blockManager = SparkEnv.get().blockManager(); if (blockManager == null) { throw new IllegalStateException("No blockManager available from the SparkEnv."); @@ -61,14 +62,14 @@ public void initializeExecutor(String appId, String execId) { @Override public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - long mapId, + long mapTaskId, int numPartitions) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, blockResolver, sparkConf); + shuffleId, mapTaskId, numPartitions, blockResolver, sparkConf); } @Override diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index dfbd7d1c6f058..9506c36bf9c8c 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -27,6 +27,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} +import org.apache.spark.shuffle.api.ShuffleDriverComponents import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, ThreadUtils, Utils} /** @@ -58,7 +59,9 @@ private class CleanupTaskWeakReference( * to be processed when the associated object goes out of scope of the application. Actual * cleanup is performed in a separate daemon thread. */ -private[spark] class ContextCleaner(sc: SparkContext) extends Logging { +private[spark] class ContextCleaner( + sc: SparkContext, + shuffleDriverComponents: ShuffleDriverComponents) extends Logging { /** * A buffer to ensure that `CleanupTaskWeakReference`s are not garbage collected as long as they @@ -221,7 +224,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { try { logDebug("Cleaning shuffle " + shuffleId) mapOutputTrackerMaster.unregisterShuffle(shuffleId) - blockManagerMaster.removeShuffle(shuffleId, blocking) + shuffleDriverComponents.removeShuffle(shuffleId, blocking) listeners.asScala.foreach(_.shuffleCleaned(shuffleId)) logDebug("Cleaned shuffle " + shuffleId) } catch { @@ -269,7 +272,6 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } - private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] } diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index f0ac9acd90156..ba8e4d69ba755 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -96,6 +96,7 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( shuffleId, this) _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) + _rdd.sparkContext.shuffleDriverComponents.registerShuffle(shuffleId) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4792c0a5b664b..2db880976c3a1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -58,6 +58,8 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.scheduler.local.LocalSchedulerBackend +import org.apache.spark.shuffle.ShuffleDataIOUtils +import org.apache.spark.shuffle.api.ShuffleDriverComponents import org.apache.spark.status.{AppStatusSource, AppStatusStore} import org.apache.spark.status.api.v1.ThreadStackTrace import org.apache.spark.storage._ @@ -217,6 +219,7 @@ class SparkContext(config: SparkConf) extends Logging { private var _statusStore: AppStatusStore = _ private var _heartbeater: Heartbeater = _ private var _resources: scala.collection.immutable.Map[String, ResourceInformation] = _ + private var _shuffleDriverComponents: ShuffleDriverComponents = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -319,6 +322,8 @@ class SparkContext(config: SparkConf) extends Logging { _dagScheduler = ds } + private[spark] def shuffleDriverComponents: ShuffleDriverComponents = _shuffleDriverComponents + /** * A unique identifier for the Spark application. * Its format depends on the scheduler implementation. @@ -524,6 +529,11 @@ class SparkContext(config: SparkConf) extends Logging { executorEnvs ++= _conf.getExecutorEnv executorEnvs("SPARK_USER") = sparkUser + _shuffleDriverComponents = ShuffleDataIOUtils.loadShuffleDataIO(config).driver() + _shuffleDriverComponents.initializeApplication().asScala.foreach { case (k, v) => + _conf.set(ShuffleDataIOUtils.SHUFFLE_SPARK_CONF_PREFIX + k, v) + } + // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) _heartbeatReceiver = env.rpcEnv.setupEndpoint( @@ -576,7 +586,7 @@ class SparkContext(config: SparkConf) extends Logging { _cleaner = if (_conf.get(CLEANER_REFERENCE_TRACKING)) { - Some(new ContextCleaner(this)) + Some(new ContextCleaner(this, _shuffleDriverComponents)) } else { None } @@ -1975,6 +1985,11 @@ class SparkContext(config: SparkConf) extends Logging { } _heartbeater = null } + if (_shuffleDriverComponents != null) { + Utils.tryLogNonFatalError { + _shuffleDriverComponents.cleanupApplication() + } + } if (env != null && _heartbeatReceiver != null) { Utils.tryLogNonFatalError { env.rpcEnv.stop(_heartbeatReceiver) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleDataIOUtils.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleDataIOUtils.scala new file mode 100644 index 0000000000000..e9507a7584ba3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleDataIOUtils.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.SHUFFLE_IO_PLUGIN_CLASS +import org.apache.spark.shuffle.api.ShuffleDataIO +import org.apache.spark.util.Utils + +private[spark] object ShuffleDataIOUtils { + + /** + * The prefix of spark config keys that are passed from the driver to the executor. + */ + val SHUFFLE_SPARK_CONF_PREFIX = "spark.shuffle.plugin.__config__." + + def loadShuffleDataIO(conf: SparkConf): ShuffleDataIO = { + val configuredPluginClass = conf.get(SHUFFLE_IO_PLUGIN_CLASS) + val maybeIO = Utils.loadExtensions( + classOf[ShuffleDataIO], Seq(configuredPluginClass), conf) + require(maybeIO.nonEmpty, s"A valid shuffle plugin must be specified by config " + + s"${SHUFFLE_IO_PLUGIN_CLASS.key}, but $configuredPluginClass resulted in zero valid " + + s"plugins.") + maybeIO.head + } + +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index b21ce9ce0fc71..5adfd71083ac7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -19,6 +19,8 @@ package org.apache.spark.shuffle.sort import java.util.concurrent.ConcurrentHashMap +import scala.collection.JavaConverters._ + import org.apache.spark._ import org.apache.spark.internal.{config, Logging} import org.apache.spark.shuffle._ @@ -236,12 +238,13 @@ private[spark] object SortShuffleManager extends Logging { } private def loadShuffleExecutorComponents(conf: SparkConf): ShuffleExecutorComponents = { - val configuredPluginClasses = conf.get(config.SHUFFLE_IO_PLUGIN_CLASS) - val maybeIO = Utils.loadExtensions( - classOf[ShuffleDataIO], Seq(configuredPluginClasses), conf) - require(maybeIO.size == 1, s"Failed to load plugins of type $configuredPluginClasses") - val executorComponents = maybeIO.head.executor() - executorComponents.initializeExecutor(conf.getAppId, SparkEnv.get.executorId) + val executorComponents = ShuffleDataIOUtils.loadShuffleDataIO(conf).executor() + val extraConfigs = conf.getAllWithPrefix(ShuffleDataIOUtils.SHUFFLE_SPARK_CONF_PREFIX) + .toMap + executorComponents.initializeExecutor( + conf.getAppId, + SparkEnv.get.executorId, + extraConfigs.asJava) executorComponents } } diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index 8433a6f52ac7a..b982626ef0f7a 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -211,7 +211,8 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { /** * A special [[ContextCleaner]] that saves the IDs of the accumulators registered for cleanup. */ - private class SaveAccumContextCleaner(sc: SparkContext) extends ContextCleaner(sc) { + private class SaveAccumContextCleaner(sc: SparkContext) extends + ContextCleaner(sc, null) { private val accumsRegistered = new ArrayBuffer[Long] override def registerAccumulatorForCleanup(a: AccumulatorV2[_, _]): Unit = { diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala new file mode 100644 index 0000000000000..d8657ecdff676 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import java.util.{Map => JMap} +import java.util.concurrent.atomic.AtomicBoolean + +import com.google.common.collect.ImmutableMap +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.config.SHUFFLE_IO_PLUGIN_CLASS +import org.apache.spark.shuffle.api.{ShuffleDataIO, ShuffleDriverComponents, ShuffleExecutorComponents, ShuffleMapOutputWriter} +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleDataIO + +class ShuffleDriverComponentsSuite + extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach { + + test("test serialization of shuffle initialization conf to executors") { + val testConf = new SparkConf() + .setAppName("testing") + .set(ShuffleDataIOUtils.SHUFFLE_SPARK_CONF_PREFIX + "test-plugin-key", "user-set-value") + .set(ShuffleDataIOUtils.SHUFFLE_SPARK_CONF_PREFIX + "test-user-key", "user-set-value") + .setMaster("local-cluster[2,1,1024]") + .set(SHUFFLE_IO_PLUGIN_CLASS, "org.apache.spark.shuffle.TestShuffleDataIO") + + sc = new SparkContext(testConf) + + val out = sc.parallelize(Seq((1, "one"), (2, "two"), (3, "three")), 3) + .groupByKey() + .foreach { _ => + if (!TestShuffleExecutorComponentsInitialized.initialized.get()) { + throw new RuntimeException("TestShuffleExecutorComponents wasn't initialized") + } + } + } +} + +class TestShuffleDataIO(sparkConf: SparkConf) extends ShuffleDataIO { + private val delegate = new LocalDiskShuffleDataIO(sparkConf) + + override def driver(): ShuffleDriverComponents = new TestShuffleDriverComponents() + + override def executor(): ShuffleExecutorComponents = + new TestShuffleExecutorComponentsInitialized(delegate.executor()) +} + +class TestShuffleDriverComponents extends ShuffleDriverComponents { + override def initializeApplication(): JMap[String, String] = { + ImmutableMap.of("test-plugin-key", "plugin-set-value") + } + + override def cleanupApplication(): Unit = {} +} + +object TestShuffleExecutorComponentsInitialized { + val initialized = new AtomicBoolean(false) +} + +class TestShuffleExecutorComponentsInitialized(delegate: ShuffleExecutorComponents) + extends ShuffleExecutorComponents { + + override def initializeExecutor( + appId: String, + execId: String, + extraConfigs: JMap[String, String]): Unit = { + delegate.initializeExecutor(appId, execId, extraConfigs) + assert(extraConfigs.get("test-plugin-key") == "plugin-set-value", extraConfigs) + assert(extraConfigs.get("test-user-key") == "user-set-value") + TestShuffleExecutorComponentsInitialized.initialized.set(true) + } + + override def createMapOutputWriter( + shuffleId: Int, + mapTaskId: Long, + numPartitions: Int): ShuffleMapOutputWriter = { + delegate.createMapOutputWriter(shuffleId, mapTaskId, numPartitions) + } +} From 322ec0ba9ba75708cfe679368a43655de7b0e4f9 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Tue, 15 Oct 2019 10:41:37 -0700 Subject: [PATCH 0330/1387] [SPARK-28885][SQL] Follow ANSI store assignment rules in table insertion by default ### What changes were proposed in this pull request? When inserting a value into a column with the different data type, Spark performs type coercion. Currently, we support 3 policies for the store assignment rules: ANSI, legacy and strict, which can be set via the option "spark.sql.storeAssignmentPolicy": 1. ANSI: Spark performs the type coercion as per ANSI SQL. In practice, the behavior is mostly the same as PostgreSQL. It disallows certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean`. It will throw a runtime exception if the value is out-of-range(overflow). 2. Legacy: Spark allows the type coercion as long as it is a valid `Cast`, which is very loose. E.g., converting either `string` to `int` or `double` to `boolean` is allowed. It is the current behavior in Spark 2.x for compatibility with Hive. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of Byte type, the result is 1. 3. Strict: Spark doesn't allow any possible precision loss or data truncation in store assignment, e.g., converting either `double` to `int` or `decimal` to `double` is allowed. The rules are originally for Dataset encoder. As far as I know, no mainstream DBMS is using this policy by default. Currently, the V1 data source uses "Legacy" policy by default, while V2 uses "Strict". This proposal is to use "ANSI" policy by default for both V1 and V2 in Spark 3.0. ### Why are the changes needed? Following the ANSI SQL standard is most reasonable among the 3 policies. ### Does this PR introduce any user-facing change? Yes. The default store assignment policy is ANSI for both V1 and V2 data sources. ### How was this patch tested? Unit test Closes #26107 from gengliangwang/ansiPolicyAsDefault. Authored-by: Gengliang Wang Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 1 + .../sql/catalyst/analysis/Analyzer.scala | 18 +++++++------- .../analysis/TableOutputResolver.scala | 11 ++++----- .../spark/sql/catalyst/expressions/Cast.scala | 1 + .../apache/spark/sql/internal/SQLConf.scala | 6 ++--- .../org/apache/spark/sql/types/DataType.scala | 2 ++ .../DataTypeWriteCompatibilitySuite.scala | 24 ++++++++++++------- .../sql/execution/datasources/rules.scala | 10 ++------ .../apache/spark/sql/SQLQueryTestSuite.scala | 14 ++++++++++- .../sql/execution/command/DDLSuite.scala | 4 ++-- .../datasources/orc/OrcSourceSuite.scala | 4 +++- .../parquet/ParquetQuerySuite.scala | 16 ++++++------- .../spark/sql/sources/InsertSuite.scala | 5 +++- .../ThriftServerQueryTestSuite.scala | 11 +++++++++ .../execution/HiveCompatibilitySuite.scala | 3 +++ .../spark/sql/hive/client/VersionsSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 4 ++-- .../execution/HiveSerDeReadWriteSuite.scala | 8 ++++--- .../sql/hive/orc/HiveOrcQuerySuite.scala | 5 +++- 19 files changed, 92 insertions(+), 57 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 8c5721340a30c..f250fec7d0689 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -23,6 +23,7 @@ license: | {:toc} ## Upgrading from Spark SQL 2.4 to 3.0 + - Since Spark 3.0, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean` are disallowed. A runtime exception will be thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and earlier, type conversions during table insertion are allowed as long as they are valid `Cast`. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of byte type, the result is 1. The behavior is controlled by the option `spark.sql.storeAssignmentPolicy`, with a default value as "ANSI". Setting the option as "Legacy" restores the previous behavior. - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 10e833c8a0756..46eca4a1d4800 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2502,9 +2502,9 @@ class Analyzer( override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { case append @ AppendData(table, query, _, isByName) if table.resolved && query.resolved && !append.outputResolved => + validateStoreAssignmentPolicy() val projection = - TableOutputResolver.resolveOutputColumns( - table.name, table.output, query, isByName, conf, storeAssignmentPolicy) + TableOutputResolver.resolveOutputColumns(table.name, table.output, query, isByName, conf) if (projection != query) { append.copy(query = projection) @@ -2514,9 +2514,9 @@ class Analyzer( case overwrite @ OverwriteByExpression(table, _, query, _, isByName) if table.resolved && query.resolved && !overwrite.outputResolved => + validateStoreAssignmentPolicy() val projection = - TableOutputResolver.resolveOutputColumns( - table.name, table.output, query, isByName, conf, storeAssignmentPolicy) + TableOutputResolver.resolveOutputColumns(table.name, table.output, query, isByName, conf) if (projection != query) { overwrite.copy(query = projection) @@ -2526,9 +2526,9 @@ class Analyzer( case overwrite @ OverwritePartitionsDynamic(table, query, _, isByName) if table.resolved && query.resolved && !overwrite.outputResolved => + validateStoreAssignmentPolicy() val projection = - TableOutputResolver.resolveOutputColumns( - table.name, table.output, query, isByName, conf, storeAssignmentPolicy) + TableOutputResolver.resolveOutputColumns(table.name, table.output, query, isByName, conf) if (projection != query) { overwrite.copy(query = projection) @@ -2538,16 +2538,14 @@ class Analyzer( } } - private def storeAssignmentPolicy: StoreAssignmentPolicy.Value = { - val policy = conf.storeAssignmentPolicy.getOrElse(StoreAssignmentPolicy.STRICT) + private def validateStoreAssignmentPolicy(): Unit = { // SPARK-28730: LEGACY store assignment policy is disallowed in data source v2. - if (policy == StoreAssignmentPolicy.LEGACY) { + if (conf.storeAssignmentPolicy == StoreAssignmentPolicy.LEGACY) { val configKey = SQLConf.STORE_ASSIGNMENT_POLICY.key throw new AnalysisException(s""" |"LEGACY" store assignment policy is disallowed in Spark data source V2. |Please set the configuration $configKey to other values.""".stripMargin) } - policy } private def commonNaturalJoinProcessing( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala index e5d25547d4d55..4f33ca99c02db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala @@ -32,8 +32,7 @@ object TableOutputResolver { expected: Seq[Attribute], query: LogicalPlan, byName: Boolean, - conf: SQLConf, - storeAssignmentPolicy: StoreAssignmentPolicy.Value): LogicalPlan = { + conf: SQLConf): LogicalPlan = { if (expected.size < query.output.size) { throw new AnalysisException( @@ -47,8 +46,7 @@ object TableOutputResolver { expected.flatMap { tableAttr => query.resolve(Seq(tableAttr.name), conf.resolver) match { case Some(queryExpr) => - checkField( - tableAttr, queryExpr, byName, conf, storeAssignmentPolicy, err => errors += err) + checkField(tableAttr, queryExpr, byName, conf, err => errors += err) case None => errors += s"Cannot find data for output column '${tableAttr.name}'" None @@ -66,8 +64,7 @@ object TableOutputResolver { query.output.zip(expected).flatMap { case (queryExpr, tableAttr) => - checkField( - tableAttr, queryExpr, byName, conf, storeAssignmentPolicy, err => errors += err) + checkField(tableAttr, queryExpr, byName, conf, err => errors += err) } } @@ -88,9 +85,9 @@ object TableOutputResolver { queryExpr: NamedExpression, byName: Boolean, conf: SQLConf, - storeAssignmentPolicy: StoreAssignmentPolicy.Value, addError: String => Unit): Option[NamedExpression] = { + val storeAssignmentPolicy = conf.storeAssignmentPolicy lazy val outputField = if (tableAttr.dataType.sameType(queryExpr.dataType) && tableAttr.name == queryExpr.name && tableAttr.metadata == queryExpr.metadata) { 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 42b471f20ff91..d71f300dd26dd 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 @@ -165,6 +165,7 @@ object Cast { */ def canANSIStoreAssign(from: DataType, to: DataType): Boolean = (from, to) match { case _ if from == to => true + case (NullType, _) => true case (_: NumericType, _: NumericType) => true case (_: AtomicType, StringType) => true case (_: CalendarIntervalType, StringType) => true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f00a4b545ee3e..42e3beca2ad59 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1740,7 +1740,7 @@ object SQLConf { .stringConf .transform(_.toUpperCase(Locale.ROOT)) .checkValues(StoreAssignmentPolicy.values.map(_.toString)) - .createOptional + .createWithDefault(StoreAssignmentPolicy.ANSI.toString) val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + @@ -2473,8 +2473,8 @@ class SQLConf extends Serializable with Logging { def partitionOverwriteMode: PartitionOverwriteMode.Value = PartitionOverwriteMode.withName(getConf(PARTITION_OVERWRITE_MODE)) - def storeAssignmentPolicy: Option[StoreAssignmentPolicy.Value] = - getConf(STORE_ASSIGNMENT_POLICY).map(StoreAssignmentPolicy.withName) + def storeAssignmentPolicy: StoreAssignmentPolicy.Value = + StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY)) def ansiEnabled: Boolean = getConf(ANSI_ENABLED) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 3a10a56f6937f..ad1d6b62ef3a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -456,6 +456,8 @@ object DataType { true } + case (_: NullType, _) if storeAssignmentPolicy == ANSI => true + case (w: AtomicType, r: AtomicType) if storeAssignmentPolicy == ANSI => if (!Cast.canANSIStoreAssign(w, r)) { addError(s"Cannot safely cast '$context': $w to $r") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala index 9d6827194f004..c47332f5d9fcb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala @@ -76,6 +76,14 @@ class StrictDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBa assert(err.contains("Cannot safely cast")) } } + + test("Check NullType is incompatible with all other types") { + allNonNullTypes.foreach { t => + assertSingleError(NullType, t, "nulls", s"Should not allow writing None to type $t") { err => + assert(err.contains(s"incompatible with $t")) + } + } + } } class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBaseSuite { @@ -145,6 +153,12 @@ class ANSIDataTypeWriteCompatibilitySuite extends DataTypeWriteCompatibilityBase assert(err.contains("Cannot safely cast 'timestampToLong': TimestampType to LongType")) } } + + test("Check NullType is compatible with all other types") { + allNonNullTypes.foreach { t => + assertAllowed(NullType, t, "nulls", s"Should allow writing None to type $t") + } + } } abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { @@ -175,17 +189,9 @@ abstract class DataTypeWriteCompatibilityBaseSuite extends SparkFunSuite { private val nestedContainerTypes = Seq(ArrayType(point2, containsNull = false), MapType(StringType, point3, valueContainsNull = false)) - private val allNonNullTypes = Seq( + protected val allNonNullTypes = Seq( atomicTypes, simpleContainerTypes, nestedContainerTypes, Seq(CalendarIntervalType)).flatten - test("Check NullType is incompatible with all other types") { - allNonNullTypes.foreach { t => - assertSingleError(NullType, t, "nulls", s"Should not allow writing None to type $t") { err => - assert(err.contains(s"incompatible with $t")) - } - } - } - test("Check each type with itself") { allNonNullTypes.foreach { t => assertAllowed(t, t, "t", s"Should allow writing type to itself $t") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 58e9f8941887f..95343e2872def 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -189,14 +189,11 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi query } - // SPARK-28730: for V1 data source, we use the "LEGACY" as default store assignment policy. - // TODO: use ANSI store assignment policy by default in SPARK-28495. - val storeAssignmentPolicy = conf.storeAssignmentPolicy.getOrElse(StoreAssignmentPolicy.LEGACY) c.copy( tableDesc = existingTable, query = Some(TableOutputResolver.resolveOutputColumns( tableDesc.qualifiedName, existingTable.schema.toAttributes, newQuery, - byName = true, conf, storeAssignmentPolicy))) + byName = true, conf))) // Here we normalize partition, bucket and sort column names, w.r.t. the case sensitivity // config, and do various checks: @@ -402,11 +399,8 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { s"including ${staticPartCols.size} partition column(s) having constant value(s).") } - // SPARK-28730: for V1 data source, we use the "LEGACY" as default store assignment policy. - // TODO: use ANSI store assignment policy by default in SPARK-28495. - val storeAssignmentPolicy = conf.storeAssignmentPolicy.getOrElse(StoreAssignmentPolicy.LEGACY) val newQuery = TableOutputResolver.resolveOutputColumns( - tblName, expectedColumns, insert.query, byName = false, conf, storeAssignmentPolicy) + tblName, expectedColumns, insert.query, byName = false, conf) if (normalizedPartSpec.nonEmpty) { if (normalizedPartSpec.size != partColNames.length) { throw new AnalysisException( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 127deea1cecc7..075f6920b1a85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -142,7 +142,19 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { /** List of test cases to ignore, in lower cases. */ protected def blackList: Set[String] = Set( - "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. + "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. + // SPARK-28885 String value is not allowed to be stored as numeric type with + // ANSI store assignment policy. + "postgreSQL/numeric.sql", + "postgreSQL/int2.sql", + "postgreSQL/int4.sql", + "postgreSQL/int8.sql", + "postgreSQL/float4.sql", + "postgreSQL/float8.sql", + // SPARK-28885 String value is not allowed to be stored as date/timestamp type with + // ANSI store assignment policy. + "postgreSQL/date.sql", + "postgreSQL/timestamp.sql" ) // Create all the test cases. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 70b1db8e5f0d2..16348096013ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -150,9 +150,9 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { Seq(3 -> "c").toDF("i", "j").write.mode("append").saveAsTable("t") checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Nil) - Seq("c" -> 3).toDF("i", "j").write.mode("append").saveAsTable("t") + Seq(3.5 -> 3).toDF("i", "j").write.mode("append").saveAsTable("t") checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") - :: Row(null, "3") :: Nil) + :: Row(3, "3") :: Nil) Seq(4 -> "d").toDF("i", "j").write.saveAsTable("t1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 55b361d5ac994..1e27593584786 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -346,7 +346,9 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } - test("SPARK-23340 Empty float/double array columns raise EOFException") { + // SPARK-28885 String value is not allowed to be stored as numeric type with + // ANSI store assignment policy. + ignore("SPARK-23340 Empty float/double array columns raise EOFException") { Seq(Seq(Array.empty[Float]).toDF(), Seq(Array.empty[Double]).toDF()).foreach { df => withTempPath { path => df.write.format("orc").save(path.getCanonicalPath) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 88b94281d88ee..f38973f7dffd1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -162,9 +162,9 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS test("SPARK-10634 timestamp written and read as INT64 - truncation") { withTable("ts") { sql("create table ts (c1 int, c2 timestamp) using parquet") - sql("insert into ts values (1, '2016-01-01 10:11:12.123456')") + sql("insert into ts values (1, timestamp'2016-01-01 10:11:12.123456')") sql("insert into ts values (2, null)") - sql("insert into ts values (3, '1965-01-01 10:11:12.123456')") + sql("insert into ts values (3, timestamp'1965-01-01 10:11:12.123456')") val expected = Seq( (1, "2016-01-01 10:11:12.123456"), (2, null), @@ -177,13 +177,13 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS withTable("ts") { withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "true") { sql("create table ts (c1 int, c2 timestamp) using parquet") - sql("insert into ts values (1, '2016-01-01 10:11:12.123456')") + sql("insert into ts values (1, timestamp'2016-01-01 10:11:12.123456')") sql("insert into ts values (2, null)") - sql("insert into ts values (3, '1965-01-01 10:11:12.125456')") - sql("insert into ts values (4, '1965-01-01 10:11:12.125')") - sql("insert into ts values (5, '1965-01-01 10:11:12.1')") - sql("insert into ts values (6, '1965-01-01 10:11:12.123456789')") - sql("insert into ts values (7, '0001-01-01 00:00:00.000000')") + sql("insert into ts values (3, timestamp'1965-01-01 10:11:12.125456')") + sql("insert into ts values (4, timestamp'1965-01-01 10:11:12.125')") + sql("insert into ts values (5, timestamp'1965-01-01 10:11:12.1')") + sql("insert into ts values (6, timestamp'1965-01-01 10:11:12.123456789')") + sql("insert into ts values (7, timestamp'0001-01-01 00:00:00.000000')") val expected = Seq( (1, "2016-01-01 10:11:12.123"), (2, null), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 5e853e666be64..9e33b8aaec5d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -729,7 +729,10 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { spark.sessionState.catalog.createTable(newTable, false) sql("INSERT INTO TABLE test_table SELECT 1, 'a'") - sql("INSERT INTO TABLE test_table SELECT 2, null") + val msg = intercept[AnalysisException] { + sql("INSERT INTO TABLE test_table SELECT 2, null") + }.getMessage + assert(msg.contains("Cannot write nullable values to non-null column 's'")) } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 36fcde35982cc..799f00a28fd49 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -92,6 +92,17 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { "date.sql", // SPARK-28620 "postgreSQL/float4.sql", + // SPARK-28885 String value is not allowed to be stored as numeric type with + // ANSI store assignment policy. + "postgreSQL/numeric.sql", + "postgreSQL/int2.sql", + "postgreSQL/int4.sql", + "postgreSQL/int8.sql", + "postgreSQL/float8.sql", + // SPARK-28885 String value is not allowed to be stored as date/timestamp type with + // ANSI store assignment policy. + "postgreSQL/date.sql", + "postgreSQL/timestamp.sql", // SPARK-28636 "decimalArithmeticOperations.sql", "literals.sql", diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 7a9f5c67fc693..36c19c680d165 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy /** * Runs the test cases that are included in the hive distribution. @@ -59,6 +60,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true) // Ensures that cross joins are enabled so that we can test them TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) + // Ensures that the table insertion behaivor is consistent with Hive + TestHive.setConf(SQLConf.STORE_ASSIGNMENT_POLICY, StoreAssignmentPolicy.LEGACY.toString) // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests // (timestamp_*) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 1c82c7e86faab..ac31557b94197 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -908,7 +908,7 @@ class VersionsSuite extends SparkFunSuite with Logging { """.stripMargin ) - val errorMsg = "data type mismatch: cannot cast decimal(2,1) to binary" + val errorMsg = "Cannot safely cast 'f0': DecimalType(2,1) to BinaryType" if (isPartitioned) { val insertStmt = s"INSERT OVERWRITE TABLE $tableName partition (ds='a') SELECT 1.3" 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 4253fe2e1edcb..6d12310714164 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 @@ -1828,10 +1828,10 @@ class HiveDDLSuite .write.format("hive").mode("append").saveAsTable("t") checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Nil) - Seq("c" -> 3).toDF("i", "j") + Seq(3.5 -> 3).toDF("i", "j") .write.format("hive").mode("append").saveAsTable("t") checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") - :: Row(null, "3") :: Nil) + :: Row(3, "3") :: Nil) Seq(4 -> "d").toDF("i", "j").write.saveAsTable("t1") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeReadWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeReadWriteSuite.scala index 25ff3544185af..f8ba7bf2c1a62 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeReadWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeReadWriteSuite.scala @@ -65,7 +65,7 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 TIMESTAMP) STORED AS $fileFormat") hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values('2019-04-11 15:50:00')") checkAnswer(spark.table("hive_serde"), Row(Timestamp.valueOf("2019-04-11 15:50:00"))) - spark.sql("INSERT INTO TABLE hive_serde values('2019-04-12 15:50:00')") + spark.sql("INSERT INTO TABLE hive_serde values(TIMESTAMP('2019-04-12 15:50:00'))") checkAnswer( spark.table("hive_serde"), Seq(Row(Timestamp.valueOf("2019-04-11 15:50:00")), @@ -77,7 +77,7 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 DATE) STORED AS $fileFormat") hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values('2019-04-11')") checkAnswer(spark.table("hive_serde"), Row(Date.valueOf("2019-04-11"))) - spark.sql("INSERT INTO TABLE hive_serde values('2019-04-12')") + spark.sql("INSERT INTO TABLE hive_serde values(TIMESTAMP('2019-04-12'))") checkAnswer( spark.table("hive_serde"), Seq(Row(Date.valueOf("2019-04-11")), Row(Date.valueOf("2019-04-12")))) @@ -119,7 +119,7 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 BINARY) STORED AS $fileFormat") hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values('1')") checkAnswer(spark.table("hive_serde"), Row("1".getBytes)) - spark.sql("INSERT INTO TABLE hive_serde values('2')") + spark.sql("INSERT INTO TABLE hive_serde values(BINARY('2'))") checkAnswer(spark.table("hive_serde"), Seq(Row("1".getBytes), Row("2".getBytes))) } } @@ -168,6 +168,8 @@ class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveS checkNumericTypes(fileFormat, "DECIMAL(38, 2)", 2.1D) // Date/Time Types + // SPARK-28885 String value is not allowed to be stored as date/timestamp type with + // ANSI store assignment policy. checkDateTimeTypes(fileFormat) // String Types diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala index 00333397e1fbb..3c545c577f16d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala @@ -210,7 +210,10 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { } } - test("SPARK-23340 Empty float/double array columns raise EOFException") { + // SPARK-28885 String value is not allowed to be stored as numeric type with + // ANSI store assignment policy. + // TODO: re-enable the test case when SPARK-29462 is fixed. + ignore("SPARK-23340 Empty float/double array columns raise EOFException") { withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false") { withTable("spark_23340") { sql("CREATE TABLE spark_23340(a array, b array) STORED AS ORC") From 4ecbdbb6a7bd3908da32c82832e886b4f9f9e596 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 15 Oct 2019 10:45:18 -0700 Subject: [PATCH 0331/1387] [SPARK-29182][CORE] Cache preferred locations of checkpointed RDD ### What changes were proposed in this pull request? This proposes to add a Spark config to control the caching behavior of ReliableCheckpointRDD.getPreferredLocations. If it is enabled, getPreferredLocations will only compute preferred locations once and cache it for later usage. The drawback of caching the preferred locations is that when the cached locations are outdated, and lose data locality. It was documented in config document. To mitigate this, this patch also adds a config to set up expire time (default is 60 mins) for the cache. If time expires, the cache will be invalid and it needs to query updated location info. This adds a test case. Looks like the most suitable test suite is CheckpointCompressionSuite. So this renames CheckpointCompressionSuite to CheckpointStorageSuite and put the test case into. ### Why are the changes needed? One Spark job in our cluster fits many ALS models in parallel. The fitting goes well, but in next when we union all factors, the union operation is very slow. By looking into the driver stack dump, looks like the driver spends a lot of time on computing preferred locations. As we checkpoint training data before fitting ALS, the time is spent on ReliableCheckpointRDD.getPreferredLocations. In this method, it will call DFS interface to query file status and block locations. As we have big number of partitions derived from the checkpointed RDD, the union will spend a lot of time on querying the same information. It reduces the time on huge union from few hours to dozens of minutes. This issue is not limited to ALS so this change is not specified to ALS. Actually it is common usage to checkpoint data in Spark, to increase reliability and cut RDD linage. Spark operations on the checkpointed data, will be beneficial. ### Does this PR introduce any user-facing change? Yes. This adds a Spark config users can use to control the cache behavior of preferred locations of checkpointed RDD. ### How was this patch tested? Unit test added and manual test on development cluster. Closes #25856 from viirya/cache-checkpoint-preferredloc. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- .../spark/internal/config/package.scala | 11 ++++++ .../spark/rdd/ReliableCheckpointRDD.scala | 36 ++++++++++++++++--- .../org/apache/spark/CheckpointSuite.scala | 26 +++++++++++++- 3 files changed, 67 insertions(+), 6 deletions(-) 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 d142d22929728..79a6dc159e001 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 @@ -783,6 +783,17 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME = + ConfigBuilder("spark.rdd.checkpoint.cachePreferredLocsExpireTime") + .internal() + .doc("Expire time in minutes for caching preferred locations of checkpointed RDD." + + "Caching preferred locations can relieve query loading to DFS and save the query " + + "time. The drawback is that the cached locations can be possibly outdated and " + + "lose data locality. If this config is not specified or is 0, it will not cache.") + .timeConf(TimeUnit.MINUTES) + .checkValue(_ > 0, "The expire time for caching preferred locations cannot be non-positive.") + .createOptional + private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD = ConfigBuilder("spark.shuffle.accurateBlockThreshold") .doc("Threshold in bytes above which the size of shuffle blocks in " + diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index 2caf9761b4432..a5c07c07e8f2b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -20,15 +20,17 @@ package org.apache.spark.rdd import java.io.{FileNotFoundException, IOException} import java.util.concurrent.TimeUnit +import scala.collection.mutable import scala.reflect.ClassTag import scala.util.control.NonFatal +import com.google.common.cache.{CacheBuilder, CacheLoader} import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{BUFFER_SIZE, CHECKPOINT_COMPRESS} +import org.apache.spark.internal.config.{BUFFER_SIZE, CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME, CHECKPOINT_COMPRESS} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -82,16 +84,40 @@ private[spark] class ReliableCheckpointRDD[T: ClassTag]( Array.tabulate(inputFiles.length)(i => new CheckpointRDDPartition(i)) } - /** - * Return the locations of the checkpoint file associated with the given partition. - */ - protected override def getPreferredLocations(split: Partition): Seq[String] = { + // Cache of preferred locations of checkpointed files. + @transient private[spark] lazy val cachedPreferredLocations = CacheBuilder.newBuilder() + .expireAfterWrite( + SparkEnv.get.conf.get(CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME).get, + TimeUnit.MINUTES) + .build( + new CacheLoader[Partition, Seq[String]]() { + override def load(split: Partition): Seq[String] = { + getPartitionBlockLocations(split) + } + }) + + // Returns the block locations of given partition on file system. + private def getPartitionBlockLocations(split: Partition): Seq[String] = { val status = fs.getFileStatus( new Path(checkpointPath, ReliableCheckpointRDD.checkpointFileName(split.index))) val locations = fs.getFileBlockLocations(status, 0, status.getLen) locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost") } + private lazy val cachedExpireTime = + SparkEnv.get.conf.get(CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME) + + /** + * Return the locations of the checkpoint file associated with the given partition. + */ + protected override def getPreferredLocations(split: Partition): Seq[String] = { + if (cachedExpireTime.isDefined && cachedExpireTime.get > 0) { + cachedPreferredLocations.get(split) + } else { + getPartitionBlockLocations(split) + } + } + /** * Read the content of the checkpoint file associated with the given partition. */ diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 3a43f1a033da1..6a108a55045ee 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import com.google.common.io.ByteStreams import org.apache.hadoop.fs.Path +import org.apache.spark.internal.config.CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.rdd._ @@ -584,7 +585,7 @@ object CheckpointSuite { } } -class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { +class CheckpointStorageSuite extends SparkFunSuite with LocalSparkContext { test("checkpoint compression") { withTempDir { checkpointDir => @@ -618,4 +619,27 @@ class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { assert(rdd.collect().toSeq === (1 to 20)) } } + + test("cache checkpoint preferred location") { + withTempDir { checkpointDir => + val conf = new SparkConf() + .set(CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME.key, "10") + .set(UI_ENABLED.key, "false") + sc = new SparkContext("local", "test", conf) + sc.setCheckpointDir(checkpointDir.toString) + val rdd = sc.makeRDD(1 to 20, numSlices = 1) + rdd.checkpoint() + assert(rdd.collect().toSeq === (1 to 20)) + + // Verify that RDD is checkpointed + assert(rdd.firstParent.isInstanceOf[ReliableCheckpointRDD[_]]) + val checkpointedRDD = rdd.firstParent.asInstanceOf[ReliableCheckpointRDD[_]] + val partiton = checkpointedRDD.partitions(0) + assert(!checkpointedRDD.cachedPreferredLocations.asMap.containsKey(partiton)) + + val preferredLoc = checkpointedRDD.preferredLocations(partiton) + assert(checkpointedRDD.cachedPreferredLocations.asMap.containsKey(partiton)) + assert(preferredLoc == checkpointedRDD.cachedPreferredLocations.get(partiton)) + } + } } From 39d53d3e7482ef37a51240f03883284e7bb25942 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 15 Oct 2019 11:55:52 -0700 Subject: [PATCH 0332/1387] [SPARK-29470][BUILD] Update plugins to latest versions ### What changes were proposed in this pull request? This PR updates plugins to latest versions. ### Why are the changes needed? This brings bug fixes like the following. - https://issues.apache.org/jira/projects/MCOMPILER/versions/12343484 (maven-compiler-plugin) - https://issues.apache.org/jira/projects/MJAVADOC/versions/12345060 (maven-javadoc-plugin) - https://issues.apache.org/jira/projects/MCHECKSTYLE/versions/12342397 (maven-checkstyle-plugin) - https://checkstyle.sourceforge.io/releasenotes.html#Release_8.25 (checkstyle) - https://checkstyle.sourceforge.io/releasenotes.html#Release_8.24 (checkstyle) ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins building and testing with the existing code. Closes #26117 from dongjoon-hyun/SPARK-29470. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../resources/org/apache/spark/log4j-defaults.properties | 3 ++- dev/checkstyle.xml | 9 +++++---- pom.xml | 8 ++++---- project/plugins.sbt | 2 +- 4 files changed, 12 insertions(+), 10 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index eb12848900b58..71652d0e9f5e8 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -33,7 +33,8 @@ log4j.logger.org.sparkproject.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO -# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support +# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs +# in SparkSQL with Hive support log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml index 00700a7791b31..32ce96f5034d8 100644 --- a/dev/checkstyle.xml +++ b/dev/checkstyle.xml @@ -64,6 +64,11 @@ + + + + + @@ -91,10 +96,6 @@ - - - - diff --git a/pom.xml b/pom.xml index bc8595a1f4ef7..c0a9cd285140a 100644 --- a/pom.xml +++ b/pom.xml @@ -2295,7 +2295,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.8.0 + 3.8.1 ${java.version} ${java.version} @@ -2458,7 +2458,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 3.0.1 + 3.1.1 -Xdoclint:all @@ -2739,7 +2739,7 @@ org.apache.maven.plugins maven-checkstyle-plugin - 3.0.0 + 3.1.0 false true @@ -2759,7 +2759,7 @@ com.puppycrawl.tools checkstyle - 8.23 + 8.25 diff --git a/project/plugins.sbt b/project/plugins.sbt index 86349497ede81..d1fe59a47217c 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -18,7 +18,7 @@ addSbtPlugin("com.etsy" % "sbt-checkstyle-plugin" % "3.1.1") // sbt-checkstyle-plugin uses an old version of checkstyle. Match it to Maven's. -libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "8.23" +libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "8.25" // checkstyle uses guava 23.0. libraryDependencies += "com.google.guava" % "guava" % "23.0" From 02c5b4f76337cc3901b8741887292bb4478931f3 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 15 Oct 2019 12:34:39 -0700 Subject: [PATCH 0333/1387] [SPARK-28947][K8S] Status logging not happens at an interval for liveness ### What changes were proposed in this pull request? This pr invoke the start method of `LoggingPodStatusWatcherImpl` for status logging at intervals. ### Why are the changes needed? This pr invoke the start method of `LoggingPodStatusWatcherImpl` is declared but never called ### Does this PR introduce any user-facing change? no ### How was this patch tested? manually test Closes #25648 from yaooqinn/SPARK-28947. Authored-by: Kent Yao Signed-off-by: Marcelo Vanzin --- .../submit/KubernetesClientApplication.scala | 25 ++------ .../k8s/submit/LoggingPodStatusWatcher.scala | 61 ++++++++----------- .../spark/deploy/k8s/submit/ClientSuite.scala | 5 +- 3 files changed, 33 insertions(+), 58 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 11bbad9c480a1..8e5532d70b5da 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -86,15 +86,12 @@ private[spark] object ClientArguments { * @param builder Responsible for building the base driver pod based on a composition of * implemented features. * @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 watcher a watcher that monitors and logs the application status */ private[spark] class Client( conf: KubernetesDriverConf, builder: KubernetesDriverBuilder, kubernetesClient: KubernetesClient, - waitForAppCompletion: Boolean, watcher: LoggingPodStatusWatcher) extends Logging { def run(): Unit = { @@ -124,10 +121,11 @@ private[spark] class Client( .endVolume() .endSpec() .build() + val driverPodName = resolvedDriverPod.getMetadata.getName Utils.tryWithResource( kubernetesClient .pods() - .withName(resolvedDriverPod.getMetadata.getName) + .withName(driverPodName) .watch(watcher)) { _ => val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) try { @@ -141,16 +139,8 @@ private[spark] class Client( throw e } - val sId = s"${Option(conf.namespace).map(_ + ":").getOrElse("")}" + - s"${resolvedDriverPod.getMetadata.getName}" - if (waitForAppCompletion) { - logInfo(s"Waiting for application ${conf.appName} with submission ID ${sId} to finish...") - watcher.awaitCompletion() - logInfo(s"Application ${conf.appName} with submission ID ${sId} finished.") - } else { - logInfo(s"Deployed Spark application ${conf.appName} with " + - s"submission ID ${sId} into Kubernetes.") - } + val sId = Seq(conf.namespace, driverPodName).mkString(":") + watcher.watchOrStop(sId) } } @@ -199,13 +189,11 @@ private[spark] class KubernetesClientApplication extends SparkApplication { } private def run(clientArguments: ClientArguments, sparkConf: SparkConf): Unit = { - val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // 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 waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, kubernetesAppId, @@ -215,9 +203,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { // The master URL has been checked for validity already in SparkSubmit. // We just need to get rid of the "k8s://" prefix here. val master = KubernetesUtils.parseMasterUrl(sparkConf.get("spark.master")) - val loggingInterval = if (waitForAppCompletion) Some(sparkConf.get(REPORT_INTERVAL)) else None - - val watcher = new LoggingPodStatusWatcherImpl(kubernetesAppId, loggingInterval) + val watcher = new LoggingPodStatusWatcherImpl(kubernetesConf) Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, @@ -231,7 +217,6 @@ private[spark] class KubernetesClientApplication extends SparkApplication { kubernetesConf, new KubernetesDriverBuilder(), kubernetesClient, - waitForAppCompletion, watcher) client.run() } 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 f16d1f3be7a6c..ce3c80c0f85b1 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 @@ -16,49 +16,36 @@ */ package org.apache.spark.deploy.k8s.submit -import java.util.concurrent.{CountDownLatch, TimeUnit} - import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.KubernetesDriverConf import org.apache.spark.deploy.k8s.KubernetesUtils._ import org.apache.spark.internal.Logging -import org.apache.spark.util.ThreadUtils private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { - def awaitCompletion(): Unit + def watchOrStop(submissionId: String): 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. + * @param conf kubernetes driver conf. */ -private[k8s] class LoggingPodStatusWatcherImpl( - appId: String, - maybeLoggingInterval: Option[Long]) +private[k8s] class LoggingPodStatusWatcherImpl(conf: KubernetesDriverConf) 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 = () => logShortStatus() + private val appId = conf.appId + + private var podCompleted = false private var pod = Option.empty[Pod] private def phase: String = pod.map(_.getStatus.getPhase).getOrElse("unknown") - def start(): Unit = { - maybeLoggingInterval.foreach { interval => - scheduler.scheduleAtFixedRate(logRunnable, 0, interval, TimeUnit.MILLISECONDS) - } - } - override def eventReceived(action: Action, pod: Pod): Unit = { this.pod = Option(pod) action match { @@ -78,11 +65,7 @@ private[k8s] class LoggingPodStatusWatcherImpl( closeWatch() } - private def logShortStatus() = { - logInfo(s"Application status for $appId (phase: $phase)") - } - - private def logLongStatus() = { + private def logLongStatus(): Unit = { logInfo("State changed, new state: " + pod.map(formatPodState).getOrElse("unknown")) } @@ -90,15 +73,25 @@ private[k8s] class LoggingPodStatusWatcherImpl( phase == "Succeeded" || phase == "Failed" } - private def closeWatch(): Unit = { - podCompletedFuture.countDown() - scheduler.shutdown() + private def closeWatch(): Unit = synchronized { + podCompleted = true + this.notifyAll() } - 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.")) + override def watchOrStop(sId: String): Unit = if (conf.get(WAIT_FOR_APP_COMPLETION)) { + logInfo(s"Waiting for application ${conf.appName} with submission ID $sId to finish...") + val interval = conf.get(REPORT_INTERVAL) + synchronized { + while (!podCompleted) { + wait(interval) + logInfo(s"Application status for $appId (phase: $phase)") + } + } + logInfo( + pod.map { p => s"Container final statuses:\n\n${containersDescription(p)}" } + .getOrElse("No containers were found in the driver pod.")) + logInfo(s"Application ${conf.appName} with submission ID $sId finished") + } else { + logInfo(s"Deployed Spark application ${conf.appName} with submission ID $sId into Kubernetes") } } 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 index 2cc7f8e21bad0..5d49ac0bbaafa 100644 --- 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 @@ -146,7 +146,6 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { kconf, driverBuilder, kubernetesClient, - false, loggingPodStatusWatcher) submissionClient.run() verify(podOperations).create(FULL_EXPECTED_POD) @@ -157,7 +156,6 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { kconf, driverBuilder, kubernetesClient, - false, loggingPodStatusWatcher) submissionClient.run() val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues @@ -181,9 +179,8 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { kconf, driverBuilder, kubernetesClient, - true, loggingPodStatusWatcher) submissionClient.run() - verify(loggingPodStatusWatcher).awaitCompletion() + verify(loggingPodStatusWatcher).watchOrStop(kconf.namespace + ":driver") } } From 95de93b24e3afe499daab6f64ebe4fb4d740db02 Mon Sep 17 00:00:00 2001 From: Jeff Evans Date: Tue, 15 Oct 2019 15:44:51 -0500 Subject: [PATCH 0334/1387] [SPARK-24540][SQL] Support for multiple character delimiter in Spark CSV read Updating univocity-parsers version to 2.8.3, which adds support for multiple character delimiters Moving univocity-parsers version to spark-parent pom dependencyManagement section Adding new utility method to build multi-char delimiter string, which delegates to existing one Adding tests for multiple character delimited CSV ### What changes were proposed in this pull request? Adds support for parsing CSV data using multiple-character delimiters. Existing logic for converting the input delimiter string to characters was kept and invoked in a loop. Project dependencies were updated to remove redundant declaration of `univocity-parsers` version, and also to change that version to the latest. ### Why are the changes needed? It is quite common for people to have delimited data, where the delimiter is not a single character, but rather a sequence of characters. Currently, it is difficult to handle such data in Spark (typically needs pre-processing). ### Does this PR introduce any user-facing change? Yes. Specifying the "delimiter" option for the DataFrame read, and providing more than one character, will no longer result in an exception. Instead, it will be converted as before and passed to the underlying library (Univocity), which has accepted multiple character delimiters since 2.8.0. ### How was this patch tested? The `CSVSuite` tests were confirmed passing (including new methods), and `sbt` tests for `sql` were executed. Closes #26027 from jeff303/SPARK-24540. Authored-by: Jeff Evans Signed-off-by: Sean Owen --- dev/deps/spark-deps-hadoop-2.7 | 2 +- dev/deps/spark-deps-hadoop-3.2 | 2 +- pom.xml | 5 ++ python/pyspark/sql/readwriter.py | 6 +-- python/pyspark/sql/streaming.py | 4 +- sql/catalyst/pom.xml | 1 - .../spark/sql/catalyst/csv/CSVExprUtils.scala | 46 +++++++++++++++++++ .../spark/sql/catalyst/csv/CSVOptions.scala | 2 +- .../sql/catalyst/csv/CSVExprUtilsSuite.scala | 38 +++++++++++++++ sql/core/pom.xml | 1 - .../apache/spark/sql/DataFrameReader.scala | 4 +- .../test-data/cars-multichar-delim-crazy.csv | 4 ++ .../test-data/cars-multichar-delim.csv | 4 ++ .../execution/datasources/csv/CSVSuite.scala | 45 ++++++++++++++++++ 14 files changed, 152 insertions(+), 12 deletions(-) create mode 100644 sql/core/src/test/resources/test-data/cars-multichar-delim-crazy.csv create mode 100644 sql/core/src/test/resources/test-data/cars-multichar-delim.csv diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 72392546be83c..defe85e52cb32 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -198,7 +198,7 @@ stax-api-1.0.1.jar stream-2.9.6.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar -univocity-parsers-2.7.3.jar +univocity-parsers-2.8.3.jar validation-api-2.0.1.Final.jar xbean-asm7-shaded-4.14.jar xercesImpl-2.9.1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 982534f874d60..386e111792c49 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -231,7 +231,7 @@ stream-2.9.6.jar super-csv-2.2.0.jar token-provider-1.0.1.jar transaction-api-1.1.jar -univocity-parsers-2.7.3.jar +univocity-parsers-2.8.3.jar validation-api-2.0.1.Final.jar velocity-1.5.jar woodstox-core-5.0.3.jar diff --git a/pom.xml b/pom.xml index c0a9cd285140a..333a7d1b408e5 100644 --- a/pom.xml +++ b/pom.xml @@ -2180,6 +2180,11 @@ + + com.univocity + univocity-parsers + 2.8.3 + diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index e51ff9bad0746..f92face2d0573 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -360,8 +360,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non or RDD of Strings storing CSV rows. :param schema: an optional :class:`pyspark.sql.types.StructType` for the input schema or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``). - :param sep: sets a single character as a separator for each field and value. - If None is set, it uses the default value, ``,``. + :param sep: sets a separator (one or more characters) for each field and value. If None is + set, it uses the default value, ``,``. :param encoding: decodes the CSV files by the given encoding type. If None is set, it uses the default value, ``UTF-8``. :param quote: sets a single character used for escaping quoted values where the @@ -890,7 +890,7 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No :param compression: compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). - :param sep: sets a single character as a separator for each field and value. If None is + :param sep: sets a separator (one or more characters) for each field and value. If None is set, it uses the default value, ``,``. :param quote: sets a single character used for escaping quoted values where the separator can be part of the value. If None is set, it uses the default diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 7faa8662ebf03..c3ceaea211d80 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -596,8 +596,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param path: string, or list of strings, for input path(s). :param schema: an optional :class:`pyspark.sql.types.StructType` for the input schema or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``). - :param sep: sets a single character as a separator for each field and value. - If None is set, it uses the default value, ``,``. + :param sep: sets a separator (one or more characters) for each field and value. If None is + set, it uses the default value, ``,``. :param encoding: decodes the CSV files by the given encoding type. If None is set, it uses the default value, ``UTF-8``. :param quote: sets a single character used for escaping quoted values where the diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 33a22d190fa70..bcebb225dfaca 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -111,7 +111,6 @@ com.univocity univocity-parsers - 2.7.3 jar diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala index bbe27831f01df..3e83c1dcb4758 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.csv +import org.apache.commons.lang3.StringUtils + object CSVExprUtils { /** * Filter ignorable rows for CSV iterator (lines empty and starting with `comment`). @@ -79,4 +81,48 @@ object CSVExprUtils { throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") } } + + /** + * Helper method that converts string representation of a character sequence to actual + * delimiter characters. The input is processed in "chunks", and each chunk is converted + * by calling [[CSVExprUtils.toChar()]]. A chunk is either: + *

      + *
    • a backslash followed by another character
    • + *
    • a non-backslash character by itself
    • + *
    + * , in that order of precedence. The result of the converting all chunks is returned as + * a [[String]]. + * + *

    Examples: + *
    • `\t` will result in a single tab character as the separator (same as before) + *
    • `|||` will result in a sequence of three pipe characters as the separator + *
    • `\\` will result in a single backslash as the separator (same as before) + *
    • `\.` will result in an error (since a dot is not a character that needs escaped) + *
    • `\\.` will result in a backslash, then dot, as the separator character sequence + *
    • `.\t.` will result in a dot, then tab, then dot as the separator character sequence + *
    • + *
    + * + * @param str the string representing the sequence of separator characters + * @return a [[String]] representing the multi-character delimiter + * @throws IllegalArgumentException if any of the individual input chunks are illegal + */ + def toDelimiterStr(str: String): String = { + var idx = 0 + + var delimiter = "" + + while (idx < str.length()) { + // if the current character is a backslash, check it plus the next char + // in order to use existing escape logic + val readAhead = if (str(idx) == '\\') 2 else 1 + // get the chunk of 1 or 2 input characters to convert to a single delimiter char + val chunk = StringUtils.substring(str, idx, idx + readAhead) + delimiter += toChar(chunk) + // advance the counter by the length of input chunk processed + idx += chunk.length() + } + + delimiter.mkString("") + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 8a91c71f1f145..5e40d74e54f11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -95,7 +95,7 @@ class CSVOptions( } } - val delimiter = CSVExprUtils.toChar( + val delimiter = CSVExprUtils.toDelimiterStr( parameters.getOrElse("sep", parameters.getOrElse("delimiter", ","))) val parseMode: ParseMode = parameters.get("mode").map(ParseMode.fromString).getOrElse(PermissiveMode) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index 838ac42184fa5..4696a578f2f91 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.csv +import org.scalatest.prop.TableDrivenPropertyChecks._ + import org.apache.spark.SparkFunSuite class CSVExprUtilsSuite extends SparkFunSuite { @@ -58,4 +60,40 @@ class CSVExprUtilsSuite extends SparkFunSuite { } assert(exception.getMessage.contains("Delimiter cannot be empty string")) } + + val testCases = Table( + ("input", "separatorStr", "expectedErrorMsg"), + // normal tab + ("""\t""", Some("\t"), None), + // backslash, then tab + ("""\\t""", Some("""\t"""), None), + // invalid special character (dot) + ("""\.""", None, Some("Unsupported special character for delimiter")), + // backslash, then dot + ("""\\.""", Some("""\."""), None), + // nothing special, just straight conversion + ("""foo""", Some("foo"), None), + // tab in the middle of some other letters + ("""ba\tr""", Some("ba\tr"), None), + // null character, expressed in Unicode literal syntax + ("""\u0000""", Some("\u0000"), None), + // and specified directly + ("\0", Some("\u0000"), None) + ) + + test("should correctly produce separator strings, or exceptions, from input") { + forAll(testCases) { (input, separatorStr, expectedErrorMsg) => + try { + val separator = CSVExprUtils.toDelimiterStr(input) + assert(separatorStr.isDefined) + assert(expectedErrorMsg.isEmpty) + assert(separator.equals(separatorStr.get)) + } catch { + case e: IllegalArgumentException => + assert(separatorStr.isEmpty) + assert(expectedErrorMsg.isDefined) + assert(e.getMessage.contains(expectedErrorMsg.get)) + } + } + } } diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 461d6f9ed8a47..3a8b7d22397ff 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -38,7 +38,6 @@ com.univocity univocity-parsers - 2.7.3 jar diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b9cc25817d2f3..8570e4640feea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -566,8 +566,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * * You can set the following CSV-specific options to deal with CSV files: *
      - *
    • `sep` (default `,`): sets a single character as a separator for each - * field and value.
    • + *
    • `sep` (default `,`): sets a separator for each field and value. This separator can be one + * or more characters.
    • *
    • `encoding` (default `UTF-8`): decodes the CSV files by the given encoding * type.
    • *
    • `quote` (default `"`): sets a single character used for escaping quoted values where diff --git a/sql/core/src/test/resources/test-data/cars-multichar-delim-crazy.csv b/sql/core/src/test/resources/test-data/cars-multichar-delim-crazy.csv new file mode 100644 index 0000000000000..cabb50e9608e6 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-multichar-delim-crazy.csv @@ -0,0 +1,4 @@ +year_/-\_make_/-\_model_/-\_comment_/-\_blank +'2012'_/-\_'Tesla'_/-\_'S'_/-\_'No comment'_/-\_ +1997_/-\_Ford_/-\_E350_/-\_'Go get one now they are going fast'_/-\_ +2015_/-\_Chevy_/-\_Volt diff --git a/sql/core/src/test/resources/test-data/cars-multichar-delim.csv b/sql/core/src/test/resources/test-data/cars-multichar-delim.csv new file mode 100644 index 0000000000000..4309edbf04418 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-multichar-delim.csv @@ -0,0 +1,4 @@ +year, make, model, comment, blank +'2012', 'Tesla', 'S', No comment, +1997, Ford, E350, 'Go get one now they are going fast', +2015, Chevy, Volt diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 0d8e6007e7b98..48a8c92aad759 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -50,6 +50,8 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { private val carsFile8859 = "test-data/cars_iso-8859-1.csv" private val carsTsvFile = "test-data/cars.tsv" private val carsAltFile = "test-data/cars-alternative.csv" + private val carsMultiCharDelimitedFile = "test-data/cars-multichar-delim.csv" + private val carsMultiCharCrazyDelimitedFile = "test-data/cars-multichar-delim-crazy.csv" private val carsUnbalancedQuotesFile = "test-data/cars-unbalanced-quotes.csv" private val carsNullFile = "test-data/cars-null.csv" private val carsEmptyValueFile = "test-data/cars-empty-value.csv" @@ -188,6 +190,49 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { verifyCars(cars, withHeader = true) } + test("test with tab delimiter and double quote") { + val cars = spark.read + .options(Map("quote" -> "\"", "delimiter" -> """\t""", "header" -> "true")) + .csv(testFile(carsTsvFile)) + + verifyCars(cars, numFields = 6, withHeader = true, checkHeader = false) + } + + test("SPARK-24540: test with multiple character delimiter (comma space)") { + val cars = spark.read + .options(Map("quote" -> "\'", "delimiter" -> ", ", "header" -> "true")) + .csv(testFile(carsMultiCharDelimitedFile)) + + verifyCars(cars, withHeader = true) + } + + test("SPARK-24540: test with multiple (crazy) character delimiter") { + val cars = spark.read + .options(Map("quote" -> "\'", "delimiter" -> """_/-\\_""", "header" -> "true")) + .csv(testFile(carsMultiCharCrazyDelimitedFile)) + + verifyCars(cars, withHeader = true) + + // check all the other columns, besides year (which is covered by verifyCars) + val otherCols = cars.select("make", "model", "comment", "blank").collect() + val expectedOtherColVals = Seq( + ("Tesla", "S", "No comment", null), + ("Ford", "E350", "Go get one now they are going fast", null), + ("Chevy", "Volt", null, null) + ) + + expectedOtherColVals.zipWithIndex.foreach { case (values, index) => + val actualRow = otherCols(index) + values match { + case (make, model, comment, blank) => + assert(make == actualRow.getString(0)) + assert(model == actualRow.getString(1)) + assert(comment == actualRow.getString(2)) + assert(blank == actualRow.getString(3)) + } + } + } + test("parse unescaped quotes with maxCharsPerColumn") { val rows = spark.read .format("csv") From 51f10ed90f6b28c58fa1e576c8ceaa22e8c5f5ba Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 16 Oct 2019 11:19:16 +0800 Subject: [PATCH 0335/1387] [SPARK-28560][SQL][FOLLOWUP] code cleanup for local shuffle reader ### What changes were proposed in this pull request? A followup of https://github.com/apache/spark/pull/25295 This PR proposes a few code cleanups: 1. rename the special `getMapSizesByExecutorId` to `getMapSizesByMapIndex` 2. rename the parameter `mapId` to `mapIndex` as that's really a mapper index. 3. `BlockStoreShuffleReader` should take `blocksByAddress` directly instead of a map id. 4. rename `getMapReader` to `getReaderForOneMapper` to be more clearer. ### Why are the changes needed? make code easier to understand ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26128 from cloud-fan/followup. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../org/apache/spark/MapOutputTracker.scala | 34 +++++++++---------- .../shuffle/BlockStoreShuffleReader.scala | 22 ++---------- .../apache/spark/shuffle/ShuffleManager.scala | 11 +++--- .../shuffle/sort/SortShuffleManager.scala | 25 +++++--------- .../BlockStoreShuffleReaderSuite.scala | 8 ++--- .../adaptive/LocalShuffledRowRDD.scala | 14 ++++---- 6 files changed, 45 insertions(+), 69 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ef4c421cbf829..24da855633db3 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -340,17 +340,17 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** * Called from executors to get the server URIs and output sizes for each shuffle block that * needs to be read from a given range of map output partitions (startPartition is included but - * endPartition is excluded from the range) and a given mapId. + * endPartition is excluded from the range) and is produced by a specific mapper. * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. */ - def getMapSizesByExecutorId( + def getMapSizesByMapIndex( shuffleId: Int, + mapIndex: Int, startPartition: Int, - endPartition: Int, - mapId: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] /** * Deletes map output status information for the specified shuffle stage. @@ -741,13 +741,12 @@ private[spark] class MapOutputTrackerMaster( } } - override def getMapSizesByExecutorId( + override def getMapSizesByMapIndex( shuffleId: Int, + mapIndex: Int, startPartition: Int, - endPartition: Int, - mapId: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mapId $mapId" + + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + logDebug(s"Fetching outputs for shuffle $shuffleId, mapIndex $mapIndex" + s"partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => @@ -757,7 +756,7 @@ private[spark] class MapOutputTrackerMaster( startPartition, endPartition, statuses, - Some(mapId)) + Some(mapIndex)) } case None => Iterator.empty @@ -809,17 +808,17 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } - override def getMapSizesByExecutorId( + override def getMapSizesByMapIndex( shuffleId: Int, + mapIndex: Int, startPartition: Int, - endPartition: Int, - mapId: Int) : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mapId $mapId" + + endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + logDebug(s"Fetching outputs for shuffle $shuffleId, mapIndex $mapIndex" + s"partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) try { MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, - statuses, Some(mapId)) + statuses, Some(mapIndex)) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -962,6 +961,7 @@ private[spark] object MapOutputTracker extends Logging { * @param startPartition Start of map output partition ID range (included in range) * @param endPartition End of map output partition ID range (excluded from range) * @param statuses List of map statuses, indexed by map partition index. + * @param mapIndex When specified, only shuffle blocks from this mapper will be processed. * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. @@ -971,11 +971,11 @@ private[spark] object MapOutputTracker extends Logging { startPartition: Int, endPartition: Int, statuses: Array[MapStatus], - mapId : Option[Int] = None): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + mapIndex : Option[Int] = None): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] val iter = statuses.iterator.zipWithIndex - for ((status, mapIndex) <- mapId.map(id => iter.filter(_._2 == id)).getOrElse(iter)) { + for ((status, mapIndex) <- mapIndex.map(index => iter.filter(_._2 == index)).getOrElse(iter)) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 242442ac9d8f2..3737102a1abab 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -20,7 +20,7 @@ package org.apache.spark.shuffle import org.apache.spark._ import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer.SerializerManager -import org.apache.spark.storage.{BlockManager, ShuffleBlockFetcherIterator} +import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter @@ -30,34 +30,18 @@ import org.apache.spark.util.collection.ExternalSorter */ private[spark] class BlockStoreShuffleReader[K, C]( handle: BaseShuffleHandle[K, _, C], - startPartition: Int, - endPartition: Int, + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], context: TaskContext, readMetrics: ShuffleReadMetricsReporter, serializerManager: SerializerManager = SparkEnv.get.serializerManager, blockManager: BlockManager = SparkEnv.get.blockManager, - mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, - mapId: Option[Int] = None) + mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker) extends ShuffleReader[K, C] with Logging { private val dep = handle.dependency /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - val blocksByAddress = mapId match { - case (Some(mapId)) => mapOutputTracker.getMapSizesByExecutorId( - handle.shuffleId, - startPartition, - endPartition, - mapId) - case (None) => mapOutputTracker.getMapSizesByExecutorId( - handle.shuffleId, - startPartition, - endPartition) - case (_) => throw new IllegalArgumentException( - "mapId should be both set or unset") - } - val wrappedStreams = new ShuffleBlockFetcherIterator( context, blockManager.blockStoreClient, diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 0041dca507c0f..01aa43eb9763d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -55,17 +55,16 @@ private[spark] trait ShuffleManager { metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] /** - * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to - * read from mapId. - * Called on executors by reduce tasks. + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) + * that are produced by one specific mapper. Called on executors by reduce tasks. */ - def getMapReader[K, C]( + def getReaderForOneMapper[K, C]( handle: ShuffleHandle, + mapIndex: Int, startPartition: Int, endPartition: Int, context: TaskContext, - metrics: ShuffleReadMetricsReporter, - mapId: Int): ShuffleReader[K, C] + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] /** * Remove a shuffle's metadata from the ShuffleManager. diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 5adfd71083ac7..a3529378a4d64 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -124,30 +124,23 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( + handle.shuffleId, startPartition, endPartition) new BlockStoreShuffleReader( - handle.asInstanceOf[BaseShuffleHandle[K, _, C]], - startPartition, endPartition, context, metrics) + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics) } - /** - * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to - * read from mapId. - * Called on executors by reduce tasks. - */ - override def getMapReader[K, C]( + override def getReaderForOneMapper[K, C]( handle: ShuffleHandle, + mapIndex: Int, startPartition: Int, endPartition: Int, context: TaskContext, - metrics: ShuffleReadMetricsReporter, - mapId: Int): ShuffleReader[K, C] = { + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByMapIndex( + handle.shuffleId, mapIndex, startPartition, endPartition) new BlockStoreShuffleReader( - handle.asInstanceOf[BaseShuffleHandle[K, _, C]], - startPartition, - endPartition, - context, - metrics, - mapId = Some(mapId)) + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics) } /** Get a writer for a given partition. Called on executors by map tasks. */ diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 74ec8abb22ad4..3f9536e224ded 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -130,15 +130,15 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext val taskContext = TaskContext.empty() val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() + val blocksByAddress = mapOutputTracker.getMapSizesByExecutorId( + shuffleId, reduceId, reduceId + 1) val shuffleReader = new BlockStoreShuffleReader( shuffleHandle, - reduceId, - reduceId + 1, + blocksByAddress, taskContext, metrics, serializerManager, - blockManager, - mapOutputTracker) + blockManager) assert(shuffleReader.read().length === keyValuePairsPerMap * numMaps) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala index 9ad1ebaf6f376..5fccb5ce65783 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsRe * (identified by `preShufflePartitionIndex`) contains a range of post-shuffle partitions * (`startPostShufflePartitionIndex` to `endPostShufflePartitionIndex - 1`, inclusive). */ -private final class LocalShuffleRowRDDPartition( +private final class LocalShuffledRowRDDPartition( val preShufflePartitionIndex: Int) extends Partition { override val index: Int = preShufflePartitionIndex } @@ -63,7 +63,7 @@ class LocalShuffledRowRDD( override def getPartitions: Array[Partition] = { Array.tabulate[Partition](numMappers) { i => - new LocalShuffleRowRDDPartition(i) + new LocalShuffledRowRDDPartition(i) } } @@ -73,20 +73,20 @@ class LocalShuffledRowRDD( } override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { - val localRowPartition = split.asInstanceOf[LocalShuffleRowRDDPartition] - val mapId = localRowPartition.index + val localRowPartition = split.asInstanceOf[LocalShuffledRowRDDPartition] + val mapIndex = localRowPartition.index val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, // as well as the `tempMetrics` for basic shuffle metrics. val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) - val reader = SparkEnv.get.shuffleManager.getMapReader( + val reader = SparkEnv.get.shuffleManager.getReaderForOneMapper( dependency.shuffleHandle, + mapIndex, 0, numReducers, context, - sqlMetricsReporter, - mapId) + sqlMetricsReporter) reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) } From e00344edc121f07270519e3c46451104be07c96e Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 15 Oct 2019 21:08:15 -0700 Subject: [PATCH 0336/1387] [SPARK-29423][SS] lazily initialize StreamingQueryManager in SessionState ### What changes were proposed in this pull request? This PR makes `SessionState` lazily initialize `StreamingQueryManager` to avoid constructing `StreamingQueryManager` for each session when connecting to ThriftServer. ### Why are the changes needed? Reduce memory usage. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test 1. Start thriftserver: ``` build/sbt clean package -Phive -Phadoop-3.2 -Phive-thriftserver export SPARK_PREPEND_CLASSES=true sbin/start-thriftserver.sh ``` 2. Open a session: ``` bin/beeline -u jdbc:hive2://localhost:10000 ``` 3. Check `StreamingQueryManager` instance: ``` jcmd | grep HiveThriftServer2 | awk -F ' ' '{print $1}' | xargs jmap -histo | grep StreamingQueryManager ``` **Before this PR**: ``` [rootspark-3267648 spark]# jcmd | grep HiveThriftServer2 | awk -F ' ' '{print $1}' | xargs jmap -histo | grep StreamingQueryManager 1954: 2 96 org.apache.spark.sql.streaming.StreamingQueryManager ``` **After this PR**: ``` [rootspark-3267648 spark]# jcmd | grep HiveThriftServer2 | awk -F ' ' '{print $1}' | xargs jmap -histo | grep StreamingQueryManager [rootspark-3267648 spark]# ``` Closes #26089 from wangyum/SPARK-29423. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../spark/sql/internal/BaseSessionStateBuilder.scala | 2 +- .../org/apache/spark/sql/internal/SessionState.scala | 9 +++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 17ef9019b520d..91c693ab34c8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -316,7 +316,7 @@ abstract class BaseSessionStateBuilder( () => analyzer, () => optimizer, planner, - streamingQueryManager, + () => streamingQueryManager, listenerManager, () => resourceLoader, createQueryExecution, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 92e255ae8aaa8..abd1250628539 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -49,7 +49,8 @@ import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListen * unresolved attributes and relations. * @param optimizerBuilder a function to create the logical query plan optimizer. * @param planner Planner that converts optimized logical plans to physical plans. - * @param streamingQueryManager Interface to start and stop streaming queries. + * @param streamingQueryManagerBuilder A function to create a streaming query manager to + * start and stop streaming queries. * @param listenerManager Interface to register custom [[QueryExecutionListener]]s. * @param resourceLoaderBuilder a function to create a session shared resource loader to load JARs, * files, etc. @@ -67,7 +68,7 @@ private[sql] class SessionState( analyzerBuilder: () => Analyzer, optimizerBuilder: () => Optimizer, val planner: SparkPlanner, - val streamingQueryManager: StreamingQueryManager, + val streamingQueryManagerBuilder: () => StreamingQueryManager, val listenerManager: ExecutionListenerManager, resourceLoaderBuilder: () => SessionResourceLoader, createQueryExecution: LogicalPlan => QueryExecution, @@ -83,6 +84,10 @@ private[sql] class SessionState( lazy val resourceLoader: SessionResourceLoader = resourceLoaderBuilder() + // The streamingQueryManager is lazy to avoid creating a StreamingQueryManager for each session + // when connecting to ThriftServer. + lazy val streamingQueryManager: StreamingQueryManager = streamingQueryManagerBuilder() + def catalogManager: CatalogManager = analyzer.catalogManager def newHadoopConf(): Configuration = SessionState.newHadoopConf( From 93e71e60e6a2cd0a5752faf43bf42719a6056951 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 16 Oct 2019 13:11:07 +0800 Subject: [PATCH 0337/1387] [SPARK-29469][SHUFFLE] Avoid retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed ### What changes were proposed in this pull request? When ExternalBlockStoreClient was closed, retries from RetryingBlockFetcher will cause NPE. This proposes to skip retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed. ### Why are the changes needed? When ExternalBlockStoreClient was closed, retries from RetryingBlockFetcher will cause NPE: ``` 2019-10-14 20:06:16 ERROR RetryingBlockFetcher:143 - Exception while beginning fetch of 2 outstanding blocks (after 3 retries) java.lang.NullPointerException at org.apache.spark.network.shuffle.ExternalShuffleClient.lambda$fetchBlocks$0(ExternalShuffleClient.java:100) at org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:141) at org.apache.spark.network.shuffle.RetryingBlockFetcher.lambda$initiateRetry$0(RetryingBlockFetcher.java:169) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) ``` It was happened after BlockManager and ExternalBlockStoreClient was closed due to previous errors. In this cases, RetryingBlockFetcher does not need to retry. This NPE is harmless for job execution, but is a source of misleading when looking at log. Especially for end-users. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #26115 from viirya/SPARK-29469. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../network/shuffle/ExternalBlockStoreClient.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index b8e52c8621fb6..85d278138c2b4 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -53,7 +53,7 @@ public class ExternalBlockStoreClient extends BlockStoreClient { private final SecretKeyHolder secretKeyHolder; private final long registrationTimeoutMs; - protected TransportClientFactory clientFactory; + protected volatile TransportClientFactory clientFactory; protected String appId; /** @@ -102,9 +102,14 @@ public void fetchBlocks( try { RetryingBlockFetcher.BlockFetchStarter blockFetchStarter = (blockIds1, listener1) -> { - TransportClient client = clientFactory.createClient(host, port); - new OneForOneBlockFetcher(client, appId, execId, - blockIds1, listener1, conf, downloadFileManager).start(); + // Unless this client is closed. + if (clientFactory != null) { + TransportClient client = clientFactory.createClient(host, port); + new OneForOneBlockFetcher(client, appId, execId, + blockIds1, listener1, conf, downloadFileManager).start(); + } else { + logger.info("This clientFactory was closed. Skipping further block fetch retries."); + } }; int maxRetries = conf.maxIORetries(); From 57edb4258254fa582f8aae6bfd8bed1069e8155c Mon Sep 17 00:00:00 2001 From: prasha2 Date: Tue, 15 Oct 2019 22:22:37 -0700 Subject: [PATCH 0338/1387] [SPARK-27259][CORE] Allow setting -1 as length for FileBlock ### What changes were proposed in this pull request? This PR aims to update the validation check on `length` from `length >= 0` to `length >= -1` in order to allow set `-1` to keep the default value. ### Why are the changes needed? At Apache Spark 2.2.0, [SPARK-18702](https://github.com/apache/spark/pull/16133/files#diff-2c5519b1cf4308d77d6f12212971544fR27-R38) adds `class FileBlock` with the default `length` value, `-1`, initially. There is no way to set `filePath` only while keeping `length` is `-1`. ```scala def set(filePath: String, startOffset: Long, length: Long): Unit = { require(filePath != null, "filePath cannot be null") require(startOffset >= 0, s"startOffset ($startOffset) cannot be negative") require(length >= 0, s"length ($length) cannot be negative") inputBlock.set(new FileBlock(UTF8String.fromString(filePath), startOffset, length)) } ``` For compressed files (like GZ), the size of split can be set to -1. This was allowed till Spark 2.1 but regressed starting with spark 2.2.x. Please note that split length of -1 also means the length was unknown - a valid scenario. Thus, split length of -1 should be acceptable like pre Spark 2.2. ### Does this PR introduce any user-facing change? No ### How was this patch tested? This is updating the corner case on the requirement check. Manually check the code. Closes #26123 from praneetsharma/fix-SPARK-27259. Authored-by: prasha2 Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala b/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala index bfe8152d4dee2..1beb085db27d9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala +++ b/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala @@ -76,7 +76,7 @@ private[spark] object InputFileBlockHolder { def set(filePath: String, startOffset: Long, length: Long): Unit = { require(filePath != null, "filePath cannot be null") require(startOffset >= 0, s"startOffset ($startOffset) cannot be negative") - require(length >= 0, s"length ($length) cannot be negative") + require(length >= -1, s"length ($length) cannot be smaller than -1") inputBlock.get().set(new FileBlock(UTF8String.fromString(filePath), startOffset, length)) } From eb8c420edb48315ae4d2ea76c3af51c87f33d810 Mon Sep 17 00:00:00 2001 From: Juliusz Sompolski Date: Tue, 15 Oct 2019 23:22:19 -0700 Subject: [PATCH 0339/1387] [SPARK-29349][SQL] Support FETCH_PRIOR in Thriftserver fetch request ### What changes were proposed in this pull request? Support FETCH_PRIOR fetching in Thriftserver, and report correct fetch start offset it TFetchResultsResp.results.startRowOffset The semantics of FETCH_PRIOR are as follow: Assuming the previous fetch returned a block of rows from offsets [10, 20) * calling FETCH_PRIOR(maxRows=5) will scroll back and return rows [5, 10) * calling FETCH_PRIOR(maxRows=10) again, will scroll back, but can't go earlier than 0. It will nevertheless return 10 rows, returning rows [0, 10) (overlapping with the previous fetch) * calling FETCH_PRIOR(maxRows=4) again will again return rows starting from offset 0 - [0, 4) * calling FETCH_NEXT(maxRows=6) after that will move the cursor forward and return rows [4, 10) ##### Client/server backwards/forwards compatibility: Old driver with new server: * Drivers that don't support FETCH_PRIOR will not attempt to use it * Field TFetchResultsResp.results.startRowOffset was not set, old drivers don't depend on it. New driver with old server * Using an older thriftserver with FETCH_PRIOR will make the thriftserver return unsupported operation error. The driver can then recognize that it's an old server. * Older thriftserver will return TFetchResultsResp.results.startRowOffset=0. If the client driver receives 0, it can know that it can not rely on it as correct offset. If the client driver intentionally wants to fetch from 0, it can use FETCH_FIRST. ### Why are the changes needed? It's intended to be used to recover after connection errors. If a client lost connection during fetching (e.g. of rows [10, 20)), and wants to reconnect and continue, it could not know whether the request got lost before reaching the server, or on the response back. When it issued another FETCH_NEXT(10) request after reconnecting, because TFetchResultsResp.results.startRowOffset was not set, it could not know if the server will return rows [10,20) (because the previous request didn't reach it) or rows [20, 30) (because it returned data from the previous request but the connection got broken on the way back). Now, with TFetchResultsResp.results.startRowOffset the client can know after reconnecting which rows it is getting, and use FETCH_PRIOR to scroll back if a fetch block was lost in transmission. Driver should always use FETCH_PRIOR after a broken connection. * If the Thriftserver returns unsuported operation error, the driver knows that it's an old server that doesn't support it. The driver then must error the query, as it will also not support returning the correct startRowOffset, so the driver cannot reliably guarantee if it hadn't lost any rows on the fetch cursor. * If the driver gets a response to FETCH_PRIOR, it should also have a correctly set startRowOffset, which the driver can use to position itself back where it left off before the connection broke. * If FETCH_NEXT was used after a broken connection on the first fetch, and returned with an startRowOffset=0, then the client driver can't know if it's 0 because it's the older server version, or if it's genuinely 0. Better to call FETCH_PRIOR, as scrolling back may anyway be possibly required after a broken connection. This way it is implemented in a backwards/forwards compatible way, and doesn't require bumping the protocol version. FETCH_ABSOLUTE might have been better, but that would require a bigger protocol change, as there is currently no field to specify the requested absolute offset. ### Does this PR introduce any user-facing change? ODBC/JDBC drivers connecting to Thriftserver may now implement using the FETCH_PRIOR fetch order to scroll back in query results, and check TFetchResultsResp.results.startRowOffset if their cursor position is consistent after connection errors. ### How was this patch tested? Added tests to HiveThriftServer2Suites Closes #26014 from juliuszsompolski/SPARK-29349. Authored-by: Juliusz Sompolski Signed-off-by: Yuming Wang --- .../SparkExecuteStatementOperation.scala | 37 +++++++- .../HiveThriftServer2Suites.scala | 89 ++++++++++++++++++- .../v1.2.1/if/TCLIService.thrift | 5 +- .../hive/service/cli/operation/Operation.java | 5 +- .../v2.3.5/if/TCLIService.thrift | 5 +- .../hive/service/cli/operation/Operation.java | 5 +- 6 files changed, 134 insertions(+), 12 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 9ca6c39d016ba..90a428defd76d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -57,7 +57,8 @@ private[hive] class SparkExecuteStatementOperation( // This is only used when `spark.sql.thriftServer.incrementalCollect` is set to `false`. // In case of `true`, this will be `None` and FETCH_FIRST will trigger re-execution. private var resultList: Option[Array[SparkRow]] = _ - + private var previousFetchEndOffset: Long = 0 + private var previousFetchStartOffset: Long = 0 private var iter: Iterator[SparkRow] = _ private var dataTypes: Array[DataType] = _ private var statementId: String = _ @@ -113,14 +114,18 @@ private[hive] class SparkExecuteStatementOperation( } def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = withSchedulerPool { + log.info(s"Received getNextRowSet request order=${order} and maxRowsL=${maxRowsL} " + + s"with ${statementId}") validateDefaultFetchOrientation(order) assertState(OperationState.FINISHED) setHasResultSet(true) val resultRowSet: RowSet = ThriftserverShimUtils.resultRowSet(getResultSetSchema, getProtocolVersion) - // Reset iter to header when fetching start from first row - if (order.equals(FetchOrientation.FETCH_FIRST)) { + // Reset iter when FETCH_FIRST or FETCH_PRIOR + if ((order.equals(FetchOrientation.FETCH_FIRST) || + order.equals(FetchOrientation.FETCH_PRIOR)) && previousFetchEndOffset != 0) { + // Reset the iterator to the beginning of the query. iter = if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { resultList = None result.toLocalIterator.asScala @@ -132,6 +137,28 @@ private[hive] class SparkExecuteStatementOperation( } } + var resultOffset = { + if (order.equals(FetchOrientation.FETCH_FIRST)) { + logInfo(s"FETCH_FIRST request with $statementId. Resetting to resultOffset=0") + 0 + } else if (order.equals(FetchOrientation.FETCH_PRIOR)) { + // TODO: FETCH_PRIOR should be handled more efficiently than rewinding to beginning and + // reiterating. + val targetOffset = math.max(previousFetchStartOffset - maxRowsL, 0) + logInfo(s"FETCH_PRIOR request with $statementId. Resetting to resultOffset=$targetOffset") + var off = 0 + while (off < targetOffset && iter.hasNext) { + iter.next() + off += 1 + } + off + } else { // FETCH_NEXT + previousFetchEndOffset + } + } + + resultRowSet.setStartOffset(resultOffset) + previousFetchStartOffset = resultOffset if (!iter.hasNext) { resultRowSet } else { @@ -152,7 +179,11 @@ private[hive] class SparkExecuteStatementOperation( } resultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]]) curRow += 1 + resultOffset += 1 } + previousFetchEndOffset = resultOffset + log.info(s"Returning result set with ${curRow} rows from offsets " + + s"[$previousFetchStartOffset, $previousFetchEndOffset) with $statementId") resultRowSet } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 8a5526ea780ef..3c8d25d935ec0 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -23,6 +23,7 @@ import java.nio.charset.StandardCharsets import java.sql.{Date, DriverManager, SQLException, Statement} import java.util.{Locale, UUID} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future, Promise} @@ -34,7 +35,7 @@ import com.google.common.io.Files import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.jdbc.HiveDriver import org.apache.hive.service.auth.PlainSaslHelper -import org.apache.hive.service.cli.{FetchOrientation, FetchType, GetInfoType} +import org.apache.hive.service.cli.{FetchOrientation, FetchType, GetInfoType, RowSet} import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket @@ -684,6 +685,92 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { assert(e.getMessage.contains("org.apache.spark.sql.catalyst.parser.ParseException")) } } + + test("ThriftCLIService FetchResults FETCH_FIRST, FETCH_NEXT, FETCH_PRIOR") { + def checkResult(rows: RowSet, start: Long, end: Long): Unit = { + assert(rows.getStartOffset() == start) + assert(rows.numRows() == end - start) + rows.iterator.asScala.zip((start until end).iterator).foreach { case (row, v) => + assert(row(0).asInstanceOf[Long] === v) + } + } + + withCLIServiceClient { client => + val user = System.getProperty("user.name") + val sessionHandle = client.openSession(user, "") + + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] + val operationHandle = client.executeStatement( + sessionHandle, + "SELECT * FROM range(10)", + confOverlay) // 10 rows result with sequence 0, 1, 2, ..., 9 + var rows: RowSet = null + + // Fetch 5 rows with FETCH_NEXT + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 5, FetchType.QUERY_OUTPUT) + checkResult(rows, 0, 5) // fetched [0, 5) + + // Fetch another 2 rows with FETCH_NEXT + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 2, FetchType.QUERY_OUTPUT) + checkResult(rows, 5, 7) // fetched [5, 7) + + // FETCH_PRIOR 3 rows + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_PRIOR, 3, FetchType.QUERY_OUTPUT) + checkResult(rows, 2, 5) // fetched [2, 5) + + // FETCH_PRIOR again will scroll back to 0, and then the returned result + // may overlap the results of previous FETCH_PRIOR + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_PRIOR, 3, FetchType.QUERY_OUTPUT) + checkResult(rows, 0, 3) // fetched [0, 3) + + // FETCH_PRIOR again will stay at 0 + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_PRIOR, 4, FetchType.QUERY_OUTPUT) + checkResult(rows, 0, 4) // fetched [0, 4) + + // FETCH_NEXT will continue moving forward from offset 4 + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 10, FetchType.QUERY_OUTPUT) + checkResult(rows, 4, 10) // fetched [4, 10) until the end of results + + // FETCH_NEXT is at end of results + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 5, FetchType.QUERY_OUTPUT) + checkResult(rows, 10, 10) // fetched empty [10, 10) (at end of results) + + // FETCH_NEXT is at end of results again + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 2, FetchType.QUERY_OUTPUT) + checkResult(rows, 10, 10) // fetched empty [10, 10) (at end of results) + + // FETCH_PRIOR 1 rows yet again + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_PRIOR, 1, FetchType.QUERY_OUTPUT) + checkResult(rows, 9, 10) // fetched [9, 10) + + // FETCH_NEXT will return 0 yet again + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 5, FetchType.QUERY_OUTPUT) + checkResult(rows, 10, 10) // fetched empty [10, 10) (at end of results) + + // FETCH_FIRST results from first row + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_FIRST, 3, FetchType.QUERY_OUTPUT) + checkResult(rows, 0, 3) // fetch [0, 3) + + // Fetch till the end rows with FETCH_NEXT" + rows = client.fetchResults( + operationHandle, FetchOrientation.FETCH_NEXT, 1000, FetchType.QUERY_OUTPUT) + checkResult(rows, 3, 10) // fetched [3, 10) + + client.closeOperation(operationHandle) + client.closeSession(sessionHandle) + } + } } class SingleSessionSuite extends HiveThriftJdbcTest { diff --git a/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift b/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift index 7cd6fa37cec37..225e319737811 100644 --- a/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift +++ b/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift @@ -1028,7 +1028,6 @@ enum TFetchOrientation { FETCH_NEXT, // Get the previous rowset. The fetch offset is ignored. - // NOT SUPPORTED FETCH_PRIOR, // Return the rowset at the given fetch offset relative @@ -1056,8 +1055,8 @@ struct TFetchResultsReq { // Operation from which to fetch results. 1: required TOperationHandle operationHandle - // The fetch orientation. For V1 this must be either - // FETCH_NEXT or FETCH_FIRST. Defaults to FETCH_NEXT. + // The fetch orientation. This must be either + // FETCH_NEXT, FETCH_PRIOR or FETCH_FIRST. Defaults to FETCH_NEXT. 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT // Max number of rows that should be returned in diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java index 19153b654b08a..51bb28748d9e2 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -58,7 +58,10 @@ public abstract class Operation { private long lastAccessTime; protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = - EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST); + EnumSet.of( + FetchOrientation.FETCH_NEXT, + FetchOrientation.FETCH_FIRST, + FetchOrientation.FETCH_PRIOR); protected Operation(HiveSession parentSession, OperationType opType, boolean runInBackground) { this.parentSession = parentSession; diff --git a/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift b/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift index 824b04919073a..9026cd25df5b3 100644 --- a/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift +++ b/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift @@ -1105,7 +1105,6 @@ enum TFetchOrientation { FETCH_NEXT, // Get the previous rowset. The fetch offset is ignored. - // NOT SUPPORTED FETCH_PRIOR, // Return the rowset at the given fetch offset relative @@ -1133,8 +1132,8 @@ struct TFetchResultsReq { // Operation from which to fetch results. 1: required TOperationHandle operationHandle - // The fetch orientation. For V1 this must be either - // FETCH_NEXT or FETCH_FIRST. Defaults to FETCH_NEXT. + // The fetch orientation. This must be either + // FETCH_NEXT, FETCH_PRIOR or FETCH_FIRST. Defaults to FETCH_NEXT. 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT // Max number of rows that should be returned in diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java index 788fcdee282ae..f26c715add987 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -64,7 +64,10 @@ public abstract class Operation { protected final QueryState queryState; protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = - EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST); + EnumSet.of( + FetchOrientation.FETCH_NEXT, + FetchOrientation.FETCH_FIRST, + FetchOrientation.FETCH_PRIOR); protected Operation(HiveSession parentSession, OperationType opType) { this(parentSession, null, opType); From 9dacdd38b35429a47b2ab1686abeaa77136bf781 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Wed, 16 Oct 2019 18:32:07 +0800 Subject: [PATCH 0340/1387] [SPARK-23578][ML][PYSPARK] Binarizer support multi-column ### What changes were proposed in this pull request? Binarizer support multi-column by extending `HasInputCols`/`HasOutputCols`/`HasThreshold`/`HasThresholds` ### Why are the changes needed? similar algs in `ml.feature` already support multi-column, like `Bucketizer`/`StringIndexer`/`QuantileDiscretizer` ### Does this PR introduce any user-facing change? yes, add setter/getter of `thresholds`/`inputCols`/`outputCols` ### How was this patch tested? added suites Closes #26064 from zhengruifeng/binarizer_multicols. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../apache/spark/ml/feature/Binarizer.scala | 171 +++++++++++++----- .../apache/spark/ml/feature/Bucketizer.scala | 4 +- .../ml/feature/QuantileDiscretizer.scala | 12 +- .../spark/ml/feature/StringIndexer.scala | 1 - .../ml/param/shared/SharedParamsCodeGen.scala | 2 +- .../spark/ml/param/shared/sharedParams.scala | 2 +- .../spark/ml/feature/BinarizerSuite.scala | 126 +++++++++++++ python/pyspark/ml/feature.py | 46 ++++- python/pyspark/ml/tests/test_feature.py | 5 +- 9 files changed, 296 insertions(+), 73 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index c4daf64dfc5f0..ec4d45b65317f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.BinaryAttribute import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ @@ -32,10 +32,17 @@ import org.apache.spark.sql.types._ /** * Binarize a column of continuous features given a threshold. + * + * Since 3.0.0, + * `Binarize` can map multiple columns at once by setting the `inputCols` parameter. Note that + * when both the `inputCol` and `inputCols` parameters are set, an Exception will be thrown. The + * `threshold` parameter is used for single column usage, and `thresholds` is for multiple + * columns. */ @Since("1.4.0") final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + extends Transformer with HasThreshold with HasThresholds with HasInputCol with HasOutputCol + with HasInputCols with HasOutputCols with DefaultParamsWritable { @Since("1.4.0") def this() = this(Identifiable.randomUID("binarizer")) @@ -48,19 +55,32 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) * @group param */ @Since("1.4.0") - val threshold: DoubleParam = + override val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold used to binarize continuous features") - /** @group getParam */ - @Since("1.4.0") - def getThreshold: Double = $(threshold) - /** @group setParam */ @Since("1.4.0") def setThreshold(value: Double): this.type = set(threshold, value) setDefault(threshold -> 0.0) + /** + * Array of threshold used to binarize continuous features. + * This is for multiple columns input. If transforming multiple columns and thresholds is + * not set, but threshold is set, then threshold will be applied across all columns. + * + * @group param + */ + @Since("3.0.0") + override val thresholds: DoubleArrayParam = new DoubleArrayParam(this, "thresholds", "Array of " + + "threshold used to binarize continuous features. This is for multiple columns input. " + + "If transforming multiple columns and thresholds is not set, but threshold is set, " + + "then threshold will be applied across all columns.") + + /** @group setParam */ + @Since("3.0.0") + def setThresholds(value: Array[Double]): this.type = set(thresholds, value) + /** @group setParam */ @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) @@ -69,66 +89,117 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ + @Since("3.0.0") + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(value: Array[String]): this.type = set(outputCols, value) + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema, logging = true) - val schema = dataset.schema - val inputType = schema($(inputCol)).dataType - val td = $(threshold) - val metadata = outputSchema($(outputCol)).metadata - - val binarizerUDF = inputType match { - case DoubleType => - udf { in: Double => if (in > td) 1.0 else 0.0 } - - case _: VectorUDT if td >= 0 => - udf { vector: Vector => - val indices = ArrayBuilder.make[Int] - val values = ArrayBuilder.make[Double] - vector.foreachActive { (index, value) => - if (value > td) { - indices += index - values += 1.0 + + val (inputColNames, outputColNames, tds) = + if (isSet(inputCols)) { + if (isSet(thresholds)) { + ($(inputCols).toSeq, $(outputCols).toSeq, $(thresholds).toSeq) + } else { + ($(inputCols).toSeq, $(outputCols).toSeq, Seq.fill($(inputCols).length)($(threshold))) + } + } else { + (Seq($(inputCol)), Seq($(outputCol)), Seq($(threshold))) + } + + val ouputCols = inputColNames.zip(tds).map { case (inputColName, td) => + val binarizerUDF = dataset.schema(inputColName).dataType match { + case DoubleType => + udf { in: Double => if (in > td) 1.0 else 0.0 } + + case _: VectorUDT if td >= 0 => + udf { vector: Vector => + val indices = ArrayBuilder.make[Int] + val values = ArrayBuilder.make[Double] + vector.foreachActive { (index, value) => + if (value > td) { + indices += index + values += 1.0 + } } + Vectors.sparse(vector.size, indices.result(), values.result()).compressed } - Vectors.sparse(vector.size, indices.result(), values.result()).compressed - } - case _: VectorUDT if td < 0 => - this.logWarning(s"Binarization operations on sparse dataset with negative threshold " + - s"$td will build a dense output, so take care when applying to sparse input.") - udf { vector: Vector => - val values = Array.fill(vector.size)(1.0) - vector.foreachActive { (index, value) => - if (value <= td) { - values(index) = 0.0 + case _: VectorUDT if td < 0 => + this.logWarning(s"Binarization operations on sparse dataset with negative threshold " + + s"$td will build a dense output, so take care when applying to sparse input.") + udf { vector: Vector => + val values = Array.fill(vector.size)(1.0) + vector.foreachActive { (index, value) => + if (value <= td) { + values(index) = 0.0 + } } + Vectors.dense(values).compressed } - Vectors.dense(values).compressed - } + } + + binarizerUDF(col(inputColName)) } - dataset.withColumn($(outputCol), binarizerUDF(col($(inputCol))), metadata) + val ouputMetadata = outputColNames.map(outputSchema(_).metadata) + dataset.withColumns(outputColNames, ouputCols, ouputMetadata) } @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - val outputColName = $(outputCol) - - val outCol: StructField = inputType match { - case DoubleType => - BinaryAttribute.defaultAttr.withName(outputColName).toStructField() - case _: VectorUDT => - StructField(outputColName, new VectorUDT) - case _ => - throw new IllegalArgumentException(s"Data type $inputType is not supported.") + ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), + Seq(outputCols)) + + if (isSet(inputCol)) { + require(!isSet(thresholds), + s"thresholds can't be set for single-column Binarizer.") + } + + if (isSet(inputCols)) { + require(getInputCols.length == getOutputCols.length, + s"Binarizer $this has mismatched Params " + + s"for multi-column transform. Params (inputCols, outputCols) should have " + + s"equal lengths, but they have different lengths: " + + s"(${getInputCols.length}, ${getOutputCols.length}).") + if (isSet(thresholds)) { + require(getInputCols.length == getThresholds.length, + s"Binarizer $this has mismatched Params " + + s"for multi-column transform. Params (inputCols, outputCols, thresholds) " + + s"should have equal lengths, but they have different lengths: " + + s"(${getInputCols.length}, ${getOutputCols.length}, ${getThresholds.length}).") + require(!isSet(threshold), + s"exactly one of threshold, thresholds Params to be set, but both are set." ) + } + } + + val (inputColNames, outputColNames) = if (isSet(inputCols)) { + ($(inputCols).toSeq, $(outputCols).toSeq) + } else { + (Seq($(inputCol)), Seq($(outputCol))) } - if (schema.fieldNames.contains(outputColName)) { - throw new IllegalArgumentException(s"Output column $outputColName already exists.") + var outputFields = schema.fields + inputColNames.zip(outputColNames).foreach { case (inputColName, outputColName) => + require(!schema.fieldNames.contains(outputColName), + s"Output column $outputColName already exists.") + val inputType = schema(inputColName).dataType + val outputField = inputType match { + case DoubleType => + BinaryAttribute.defaultAttr.withName(outputColName).toStructField() + case _: VectorUDT => + StructField(outputColName, new VectorUDT) + case _ => + throw new IllegalArgumentException(s"Data type $inputType is not supported.") + } + outputFields :+= outputField } - StructType(schema.fields :+ outCol) + StructType(outputFields) } @Since("1.4.1") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index 16073d5fc1b6b..8533ed5ce5727 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -194,7 +194,7 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String if (isSet(inputCols)) { require(getInputCols.length == getOutputCols.length && getInputCols.length == getSplitsArray.length, s"Bucketizer $this has mismatched Params " + - s"for multi-column transform. Params (inputCols, outputCols, splitsArray) should have " + + s"for multi-column transform. Params (inputCols, outputCols, splitsArray) should have " + s"equal lengths, but they have different lengths: " + s"(${getInputCols.length}, ${getOutputCols.length}, ${getSplitsArray.length}).") @@ -283,7 +283,7 @@ object Bucketizer extends DefaultParamsReadable[Bucketizer] { val insertPos = -idx - 1 if (insertPos == 0 || insertPos == splits.length) { throw new SparkException(s"Feature value $feature out of Bucketizer bounds" + - s" [${splits.head}, ${splits.last}]. Check your features, or loosen " + + s" [${splits.head}, ${splits.last}]. Check your features, or loosen " + s"the lower/upper bound constraints.") } else { insertPos - 1 diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index f7a83cdd41a90..aa4ab5903f711 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -180,13 +180,13 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui if (isSet(inputCols)) { require(getInputCols.length == getOutputCols.length, s"QuantileDiscretizer $this has mismatched Params " + - s"for multi-column transform. Params (inputCols, outputCols) should have " + + s"for multi-column transform. Params (inputCols, outputCols) should have " + s"equal lengths, but they have different lengths: " + s"(${getInputCols.length}, ${getOutputCols.length}).") if (isSet(numBucketsArray)) { require(getInputCols.length == getNumBucketsArray.length, s"QuantileDiscretizer $this has mismatched Params " + - s"for multi-column transform. Params (inputCols, outputCols, numBucketsArray) " + + s"for multi-column transform. Params (inputCols, outputCols, numBucketsArray) " + s"should have equal lengths, but they have different lengths: " + s"(${getInputCols.length}, ${getOutputCols.length}, ${getNumBucketsArray.length}).") require(!isSet(numBuckets), @@ -199,12 +199,12 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui } else { (Seq($(inputCol)), Seq($(outputCol))) } - val existingFields = schema.fields - var outputFields = existingFields + + var outputFields = schema.fields inputColNames.zip(outputColNames).foreach { case (inputColName, outputColName) => SchemaUtils.checkNumericType(schema, inputColName) - require(existingFields.forall(_.name != outputColName), - s"Output column ${outputColName} already exists.") + require(!schema.fieldNames.contains(outputColName), + s"Output column $outputColName already exists.") val attr = NominalAttribute.defaultAttr.withName(outputColName) outputFields :+= attr.toStructField() } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index eab90a32a3f4b..2ce5acf1fea13 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions.{If, Literal} import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ -import org.apache.spark.storage.StorageLevel import org.apache.spark.util.ThreadUtils import org.apache.spark.util.VersionUtils.majorMinorVersion import org.apache.spark.util.collection.OpenHashMap diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index f4abe9aa1c640..7ac9a288d285d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -58,7 +58,7 @@ private[shared] object SharedParamsCodeGen { " The class with largest value p/t is predicted, where p is the original probability" + " of that class and t is the class's threshold", isValid = "(t: Array[Double]) => t.forall(_ >= 0) && t.count(_ == 0) <= 1", - finalMethods = false), + finalMethods = false, finalFields = false), ParamDesc[String]("inputCol", "input column name"), ParamDesc[Array[String]]("inputCols", "input column names"), ParamDesc[String]("outputCol", "output column name", Some("uid + \"__output\"")), diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 9baaf1744a3b7..6eeeb57e08fb0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -198,7 +198,7 @@ trait HasThresholds extends Params { * Param for Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values > 0 excepting that at most one value may be 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class's threshold. * @group param */ - final val thresholds: DoubleArrayParam = new DoubleArrayParam(this, "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values > 0 excepting that at most one value may be 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class's threshold", (t: Array[Double]) => t.forall(_ >= 0) && t.count(_ == 0) <= 1) + val thresholds: DoubleArrayParam = new DoubleArrayParam(this, "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values > 0 excepting that at most one value may be 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class's threshold", (t: Array[Double]) => t.forall(_ >= 0) && t.count(_ == 0) <= 1) /** @group getParam */ def getThresholds: Array[Double] = $(thresholds) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala index 91bec50fb904f..9baad52db00b3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala @@ -122,5 +122,131 @@ class BinarizerSuite extends MLTest with DefaultReadWriteTest { .setOutputCol("myOutputCol") .setThreshold(0.1) testDefaultReadWrite(t) + + val t2 = new Binarizer() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("result1", "result2", "result3")) + .setThresholds(Array(30.0, 30.0, 30.0)) + testDefaultReadWrite(t2) + } + + test("Multiple Columns: Test thresholds") { + val thresholds = Array(10.0, -0.5, 0.0) + + val data1 = Seq(5.0, 11.0) + val expected1 = Seq(0.0, 1.0) + val data2 = Seq(Vectors.sparse(3, Array(1), Array(0.5)), + Vectors.dense(Array(0.0, 0.5, 0.0))) + val expected2 = Seq(Vectors.dense(Array(1.0, 1.0, 1.0)), + Vectors.dense(Array(1.0, 1.0, 1.0))) + val data3 = Seq(0.0, 1.0) + val expected3 = Seq(0.0, 1.0) + + val df = Seq(0, 1).map { idx => + (data1(idx), data2(idx), data3(idx), expected1(idx), expected2(idx), expected3(idx)) + }.toDF("input1", "input2", "input3", "expected1", "expected2", "expected3") + + val binarizer = new Binarizer() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("result1", "result2", "result3")) + .setThresholds(thresholds) + + binarizer.transform(df) + .select("result1", "expected1", "result2", "expected2", "result3", "expected3") + .collect().foreach { + case Row(r1: Double, e1: Double, r2: Vector, e2: Vector, r3: Double, e3: Double) => + assert(r1 === e1, + s"The result value is not correct after bucketing. Expected $e1 but found $r1") + assert(r2 === e2, + s"The result value is not correct after bucketing. Expected $e2 but found $r2") + assert(r3 === e3, + s"The result value is not correct after bucketing. Expected $e3 but found $r3") + } + } + + test("Multiple Columns: Comparing setting threshold with setting thresholds " + + "explicitly with identical values") { + val data1 = Array.range(1, 21, 1).map(_.toDouble) + val data2 = Array.range(1, 40, 2).map(_.toDouble) + val data3 = Array.range(1, 60, 3).map(_.toDouble) + val df = (0 until 20).map { idx => + (data1(idx), data2(idx), data3(idx)) + }.toDF("input1", "input2", "input3") + + val binarizerSingleThreshold = new Binarizer() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("result1", "result2", "result3")) + .setThreshold(30.0) + + val df2 = binarizerSingleThreshold.transform(df) + + val binarizerMultiThreshold = new Binarizer() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("expected1", "expected2", "expected3")) + .setThresholds(Array(30.0, 30.0, 30.0)) + + binarizerMultiThreshold.transform(df2) + .select("result1", "expected1", "result2", "expected2", "result3", "expected3") + .collect().foreach { + case Row(r1: Double, e1: Double, r2: Double, e2: Double, r3: Double, e3: Double) => + assert(r1 === e1, + s"The result value is not correct after bucketing. Expected $e1 but found $r1") + assert(r2 === e2, + s"The result value is not correct after bucketing. Expected $e2 but found $r2") + assert(r3 === e3, + s"The result value is not correct after bucketing. Expected $e3 but found $r3") + } + } + + test("Multiple Columns: Mismatched sizes of inputCols/outputCols") { + val binarizer = new Binarizer() + .setInputCols(Array("input")) + .setOutputCols(Array("result1", "result2")) + .setThreshold(1.0) + val df = sc.parallelize(Array(1.0, 2.0, 3.0, 4.0, 5.0, 6.0)) + .map(Tuple1.apply).toDF("input") + intercept[IllegalArgumentException] { + binarizer.transform(df).count() + } + } + + test("Multiple Columns: Mismatched sizes of inputCols/thresholds") { + val binarizer = new Binarizer() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("result1", "result2")) + .setThresholds(Array(1.0, 2.0, 3.0)) + val data1 = Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0) + val data2 = Array(1.0, 2.0, 3.0, 1.0, 1.0, 1.0, 1.0, 3.0, 2.0, 3.0) + val df = data1.zip(data2).toSeq.toDF("input1", "input2") + intercept[IllegalArgumentException] { + binarizer.transform(df).count() + } + } + + test("Multiple Columns: Mismatched sizes of inputCol/thresholds") { + val binarizer = new Binarizer() + .setInputCol("input1") + .setOutputCol("result1") + .setThresholds(Array(1.0, 2.0)) + val data1 = Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0) + val data2 = Array(1.0, 2.0, 3.0, 1.0, 1.0, 1.0, 1.0, 3.0, 2.0, 3.0) + val df = data1.zip(data2).toSeq.toDF("input1", "input2") + intercept[IllegalArgumentException] { + binarizer.transform(df).count() + } + } + + test("Multiple Columns: Set both of threshold/thresholds") { + val binarizer = new Binarizer() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("result1", "result2")) + .setThresholds(Array(1.0, 2.0)) + .setThreshold(1.0) + val data1 = Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0) + val data2 = Array(1.0, 2.0, 3.0, 1.0, 1.0, 1.0, 1.0, 3.0, 2.0, 3.0) + val df = data1.zip(data2).toSeq.toDF("input1", "input2") + intercept[IllegalArgumentException] { + binarizer.transform(df).count() + } } } diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index dc717165b7a4d..a0883f1d54fed 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -65,9 +65,14 @@ @inherit_doc -class Binarizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): +class Binarizer(JavaTransformer, HasThreshold, HasThresholds, HasInputCol, HasOutputCol, + HasInputCols, HasOutputCols, JavaMLReadable, JavaMLWritable): """ - Binarize a column of continuous features given a threshold. + Binarize a column of continuous features given a threshold. Since 3.0.0, + :py:class:`Binarize` can map multiple columns at once by setting the :py:attr:`inputCols` + parameter. Note that when both the :py:attr:`inputCol` and :py:attr:`inputCols` parameters + are set, an Exception will be thrown. The :py:attr:`threshold` parameter is used for + single column usage, and :py:attr:`thresholds` is for multiple columns. >>> df = spark.createDataFrame([(0.5,)], ["values"]) >>> binarizer = Binarizer(threshold=1.0, inputCol="values", outputCol="features") @@ -83,6 +88,17 @@ class Binarizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java >>> loadedBinarizer = Binarizer.load(binarizerPath) >>> loadedBinarizer.getThreshold() == binarizer.getThreshold() True + >>> df2 = spark.createDataFrame([(0.5, 0.3)], ["values1", "values2"]) + >>> binarizer2 = Binarizer(thresholds=[0.0, 1.0]) + >>> binarizer2.setInputCols(["values1", "values2"]).setOutputCols(["output1", "output2"]) + Binarizer... + >>> binarizer2.transform(df2).show() + +-------+-------+-------+-------+ + |values1|values2|output1|output2| + +-------+-------+-------+-------+ + | 0.5| 0.3| 1.0| 0.0| + +-------+-------+-------+-------+ + ... .. versionadded:: 1.4.0 """ @@ -92,11 +108,19 @@ class Binarizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java "The features greater than the threshold will be binarized to 1.0. " + "The features equal to or less than the threshold will be binarized to 0.0", typeConverter=TypeConverters.toFloat) + thresholds = Param(Params._dummy(), "thresholds", + "Param for array of threshold used to binarize continuous features. " + + "This is for multiple columns input. If transforming multiple columns " + + "and thresholds is not set, but threshold is set, then threshold will " + + "be applied across all columns.", + typeConverter=TypeConverters.toListFloat) @keyword_only - def __init__(self, threshold=0.0, inputCol=None, outputCol=None): + def __init__(self, threshold=0.0, inputCol=None, outputCol=None, thresholds=None, + inputCols=None, outputCols=None): """ - __init__(self, threshold=0.0, inputCol=None, outputCol=None) + __init__(self, threshold=0.0, inputCol=None, outputCol=None, thresholds=None, \ + inputCols=None, outputCols=None) """ super(Binarizer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Binarizer", self.uid) @@ -106,9 +130,11 @@ def __init__(self, threshold=0.0, inputCol=None, outputCol=None): @keyword_only @since("1.4.0") - def setParams(self, threshold=0.0, inputCol=None, outputCol=None): + def setParams(self, threshold=0.0, inputCol=None, outputCol=None, thresholds=None, + inputCols=None, outputCols=None): """ - setParams(self, threshold=0.0, inputCol=None, outputCol=None) + setParams(self, threshold=0.0, inputCol=None, outputCol=None, thresholds=None, \ + inputCols=None, outputCols=None) Sets params for this Binarizer. """ kwargs = self._input_kwargs @@ -121,12 +147,12 @@ def setThreshold(self, value): """ return self._set(threshold=value) - @since("1.4.0") - def getThreshold(self): + @since("3.0.0") + def setThresholds(self, value): """ - Gets the value of threshold or its default value. + Sets the value of :py:attr:`thresholds`. """ - return self.getOrDefault(self.threshold) + return self._set(thresholds=value) class _LSHParams(HasInputCol, HasOutputCol): diff --git a/python/pyspark/ml/tests/test_feature.py b/python/pyspark/ml/tests/test_feature.py index 6b0d1dc9d0624..4c6bfa696b110 100644 --- a/python/pyspark/ml/tests/test_feature.py +++ b/python/pyspark/ml/tests/test_feature.py @@ -34,12 +34,13 @@ class FeatureTests(SparkSessionTestCase): def test_binarizer(self): b0 = Binarizer() - self.assertListEqual(b0.params, [b0.inputCol, b0.outputCol, b0.threshold]) + self.assertListEqual(b0.params, [b0.inputCol, b0.inputCols, b0.outputCol, + b0.outputCols, b0.threshold, b0.thresholds]) self.assertTrue(all([~b0.isSet(p) for p in b0.params])) self.assertTrue(b0.hasDefault(b0.threshold)) self.assertEqual(b0.getThreshold(), 0.0) b0.setParams(inputCol="input", outputCol="output").setThreshold(1.0) - self.assertTrue(all([b0.isSet(p) for p in b0.params])) + self.assertTrue(not all([b0.isSet(p) for p in b0.params])) self.assertEqual(b0.getThreshold(), 1.0) self.assertEqual(b0.getInputCol(), "input") self.assertEqual(b0.getOutputCol(), "output") From 5a482e72091c8db940408905e8c044f7f5d7814f Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Wed, 16 Oct 2019 21:06:13 +0800 Subject: [PATCH 0341/1387] [SPARK-29468][SQL] Change Literal.sql to be correct for floats ### What changes were proposed in this pull request? Change Literal.sql to output CAST('fpValue' AS FLOAT) instead of CAST(fpValue AS FLOAT) as the SQL for a floating point literal. ### Why are the changes needed? The old version doesn't work for very small floating point numbers; the value will fail to parse if it doesn't fit in a DECIMAL(38). This doesn't apply to doubles because they have special literal syntax. ### Does this PR introduce any user-facing change? Not really. ### How was this patch tested? New unit tests. Closes #26114 from jose-torres/fpliteral. Authored-by: Jose Torres Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/literals.scala | 22 ++++++++++- .../ExpressionSQLBuilderSuite.scala | 38 +++++++++++++++---- 2 files changed, 52 insertions(+), 8 deletions(-) rename sql/{hive/src/test/scala/org/apache/spark/sql/catalyst => catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions}/ExpressionSQLBuilderSuite.scala (78%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index f03174babcd9d..f3c129934314c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -223,6 +223,26 @@ object NonNullLiteral { } } +/** + * Extractor for retrieving Float literals. + */ +object FloatLiteral { + def unapply(a: Any): Option[Float] = a match { + case Literal(a: Float, FloatType) => Some(a) + case _ => None + } +} + +/** + * Extractor for retrieving Double literals. + */ +object DoubleLiteral { + def unapply(a: Any): Option[Double] = a match { + case Literal(a: Double, DoubleType) => Some(a) + case _ => None + } +} + /** * Extractor for retrieving Int literals. */ @@ -370,7 +390,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { case _ if v.isNaN => "'NaN'" case Float.PositiveInfinity => "'Infinity'" case Float.NegativeInfinity => "'-Infinity'" - case _ => v + case _ => s"'$v'" } s"CAST($castedValue AS ${FloatType.sql})" case (v: Double, DoubleType) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala similarity index 78% rename from sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala index ae701f266bf45..442a23218c010 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala @@ -15,19 +15,21 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst +package org.apache.spark.sql.catalyst.expressions import java.time.LocalDateTime -import org.apache.spark.sql.QueryTest +import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{FloatType, TimestampType} import org.apache.spark.unsafe.types.CalendarInterval -class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { +class ExpressionSQLBuilderSuite extends SparkFunSuite { + import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._ + protected def checkSQL(e: Expression, expectedSQL: String): Unit = { val actualSQL = e.sql try { @@ -42,6 +44,24 @@ class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { |$cause """.stripMargin) } + + // For literals, check that the SQL evaluates to the same value. Excludes timestamp type which + // currently doesn't have a round-trippable format. + if (e.isInstanceOf[Literal] && e.dataType != TimestampType) { + val roundTrippedValue = parseExpression(actualSQL).eval() + + e match { + // NaNs don't compare equal so we need special checks. + case FloatLiteral(f) if f.isNaN => + assert(roundTrippedValue.isInstanceOf[Float]) + assert(roundTrippedValue.asInstanceOf[Float].isNaN) + case DoubleLiteral(d) if d.isNaN => + assert(roundTrippedValue.isInstanceOf[Double]) + assert(roundTrippedValue.asInstanceOf[Double].isNaN) + case lit: Literal => + assert(lit.value === roundTrippedValue) + } + } } test("literal") { @@ -52,20 +72,24 @@ class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { checkSQL(Literal(2: Short), "2S") checkSQL(Literal(4: Int), "4") checkSQL(Literal(8: Long), "8L") - checkSQL(Literal(1.5F), "CAST(1.5 AS FLOAT)") + checkSQL(Literal(1.5F), "CAST('1.5' AS FLOAT)") checkSQL(Literal(Float.PositiveInfinity), "CAST('Infinity' AS FLOAT)") checkSQL(Literal(Float.NegativeInfinity), "CAST('-Infinity' AS FLOAT)") checkSQL(Literal(Float.NaN), "CAST('NaN' AS FLOAT)") + checkSQL(Literal(Float.MinPositiveValue), "CAST('1.4E-45' AS FLOAT)") checkSQL(Literal(2.5D), "2.5D") checkSQL(Literal(Double.PositiveInfinity), "CAST('Infinity' AS DOUBLE)") checkSQL(Literal(Double.NegativeInfinity), "CAST('-Infinity' AS DOUBLE)") checkSQL(Literal(Double.NaN), "CAST('NaN' AS DOUBLE)") + checkSQL(Literal(Double.MinPositiveValue), "4.9E-324D") checkSQL(Literal(BigDecimal("10.0000000").underlying), "10.0000000BD") checkSQL(Literal(Array(0x01, 0xA3).map(_.toByte)), "X'01A3'") - val timestamp = LocalDateTime.of(2016, 1, 1, 0, 0, 0) + + // Nanos are truncated, but micros should not be + val timestamp = LocalDateTime.of(2016, 1, 1, 0, 0, 0, 987654321) .atZone(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) .toInstant - checkSQL(Literal(timestamp), "TIMESTAMP('2016-01-01 00:00:00')") + checkSQL(Literal(timestamp), "TIMESTAMP('2016-01-01 00:00:00.987654')") // TODO tests for decimals } From d11cbf2e367317e6a2bedfcd71819ddf485c39c5 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 16 Oct 2019 06:26:01 -0700 Subject: [PATCH 0342/1387] [SPARK-29364][SQL] Return an interval from date subtract according to SQL standard ### What changes were proposed in this pull request? Proposed new expression `SubtractDates` which is used in `date1` - `date2`. It has the `INTERVAL` type, and returns the interval from `date1` (inclusive) and `date2` (exclusive). For example: ```sql > select date'tomorrow' - date'yesterday'; interval 2 days ``` Closes #26034 ### Why are the changes needed? - To conform the SQL standard which states the result type of `date operand 1` - `date operand 2` must be the interval type. See [4.5.3 Operations involving datetimes and intervals](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt). - Improve Spark SQL UX and allow mixing date and timestamp in subtractions. For example: `select timestamp'now' + (date'2019-10-01' - date'2019-09-15')` ### Does this PR introduce any user-facing change? Before the query below returns number of days: ```sql spark-sql> select date'2019-10-05' - date'2018-09-01'; 399 ``` After it returns an interval: ```sql spark-sql> select date'2019-10-05' - date'2018-09-01'; interval 1 years 1 months 4 days ``` ### How was this patch tested? - by new tests in `DateExpressionsSuite` and `TypeCoercionSuite`. - by existing tests in `date.sql` Closes #26112 from MaxGekk/date-subtract. Authored-by: Maxim Gekk Signed-off-by: Yuming Wang --- .../sql/catalyst/analysis/TypeCoercion.scala | 15 ++++++---- .../expressions/datetimeExpressions.scala | 24 ++++++++++++++- .../sql/catalyst/util/DateTimeUtils.scala | 18 ++++++++++- .../catalyst/analysis/TypeCoercionSuite.scala | 8 ++--- .../expressions/DateExpressionsSuite.scala | 30 +++++++++++++++---- .../sql-tests/results/datetime.sql.out | 8 ++--- 6 files changed, 82 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 0983810c9ad1a..c451eb2b877da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -828,8 +828,9 @@ object TypeCoercion { /** * 1. Turns Add/Subtract of DateType/TimestampType/StringType and CalendarIntervalType * to TimeAdd/TimeSub. - * 2. Turns Add/Subtract of DateType/IntegerType and IntegerType/DateType - * to DateAdd/DateSub/DateDiff. + * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType + * and TimestampType/IntegerType/DateType to DateAdd/DateSub/SubtractDates and + * to SubtractTimestamps. */ object DateTimeOperations extends Rule[LogicalPlan] { @@ -849,12 +850,14 @@ object TypeCoercion { case Add(l @ DateType(), r @ IntegerType()) => DateAdd(l, r) case Add(l @ IntegerType(), r @ DateType()) => DateAdd(r, l) case Subtract(l @ DateType(), r @ IntegerType()) => DateSub(l, r) - case Subtract(l @ DateType(), r @ DateType()) => DateDiff(l, r) - case Subtract(l @ TimestampType(), r @ TimestampType()) => TimestampDiff(l, r) + case Subtract(l @ DateType(), r @ DateType()) => + if (SQLConf.get.usePostgreSQLDialect) DateDiff(l, r) else SubtractDates(l, r) + case Subtract(l @ TimestampType(), r @ TimestampType()) => + SubtractTimestamps(l, r) case Subtract(l @ TimestampType(), r @ DateType()) => - TimestampDiff(l, Cast(r, TimestampType)) + SubtractTimestamps(l, Cast(r, TimestampType)) case Subtract(l @ DateType(), r @ TimestampType()) => - TimestampDiff(Cast(l, TimestampType), r) + SubtractTimestamps(Cast(l, TimestampType), r) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 5aea884ad5003..cddd8c9bd61b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -2099,7 +2099,7 @@ case class DatePart(field: Expression, source: Expression, child: Expression) * is set to 0 and the `microseconds` field is initialized to the microsecond difference * between the given timestamps. */ -case class TimestampDiff(endTimestamp: Expression, startTimestamp: Expression) +case class SubtractTimestamps(endTimestamp: Expression, startTimestamp: Expression) extends BinaryExpression with ImplicitCastInputTypes { override def left: Expression = endTimestamp @@ -2116,3 +2116,25 @@ case class TimestampDiff(endTimestamp: Expression, startTimestamp: Expression) s"new org.apache.spark.unsafe.types.CalendarInterval(0, $end - $start)") } } + +/** + * Returns the interval from the `left` date (inclusive) to the `right` date (exclusive). + */ +case class SubtractDates(left: Expression, right: Expression) + extends BinaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType, DateType) + override def dataType: DataType = CalendarIntervalType + + override def nullSafeEval(leftDays: Any, rightDays: Any): Any = { + DateTimeUtils.subtractDates(leftDays.asInstanceOf[Int], rightDays.asInstanceOf[Int]) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, (leftDays, rightDays) => { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + s"$dtu.subtractDates($leftDays, $rightDays)" + }) + } +} + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 34e8012106bbe..088876921dccd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -27,7 +27,7 @@ import java.util.concurrent.TimeUnit._ import scala.util.control.NonFatal import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** * Helper functions for converting between internal and external date and time representations. @@ -950,4 +950,20 @@ object DateTimeUtils { None } } + + /** + * Subtracts two dates. + * @param endDate - the end date, exclusive + * @param startDate - the start date, inclusive + * @return an interval between two dates. The interval can be negative + * if the end date is before the start date. + */ + def subtractDates(endDate: SQLDate, startDate: SQLDate): CalendarInterval = { + val period = Period.between( + LocalDate.ofEpochDay(startDate), + LocalDate.ofEpochDay(endDate)) + val months = period.getMonths + 12 * period.getYears + val microseconds = period.getDays * MICROS_PER_DAY + new CalendarInterval(months, microseconds) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index f60e0f2bfee6a..4f9e4ec0201dd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1430,13 +1430,13 @@ class TypeCoercionSuite extends AnalysisTest { ruleTest(dateTimeOperations, Add(date, intValue), DateAdd(date, intValue)) ruleTest(dateTimeOperations, Add(intValue, date), DateAdd(date, intValue)) ruleTest(dateTimeOperations, Subtract(date, intValue), DateSub(date, intValue)) - ruleTest(dateTimeOperations, Subtract(date, date), DateDiff(date, date)) + ruleTest(dateTimeOperations, Subtract(date, date), SubtractDates(date, date)) ruleTest(dateTimeOperations, Subtract(timestamp, timestamp), - TimestampDiff(timestamp, timestamp)) + SubtractTimestamps(timestamp, timestamp)) ruleTest(dateTimeOperations, Subtract(timestamp, date), - TimestampDiff(timestamp, Cast(date, TimestampType))) + SubtractTimestamps(timestamp, Cast(date, TimestampType))) ruleTest(dateTimeOperations, Subtract(date, timestamp), - TimestampDiff(Cast(date, TimestampType), timestamp)) + SubtractTimestamps(Cast(date, TimestampType), timestamp)) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 8680a15ee1cd7..e893e863b3675 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{Instant, LocalDateTime, ZoneId, ZoneOffset} +import java.time.{Instant, LocalDate, LocalDateTime, ZoneId, ZoneOffset} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit._ @@ -1072,19 +1072,39 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("timestamps difference") { val end = Instant.parse("2019-10-04T11:04:01.123456Z") - checkEvaluation(TimestampDiff(Literal(end), Literal(end)), + checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), new CalendarInterval(0, 0)) - checkEvaluation(TimestampDiff(Literal(end), Literal(Instant.EPOCH)), + checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), CalendarInterval.fromString("interval 18173 days " + "11 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds")) - checkEvaluation(TimestampDiff(Literal(Instant.EPOCH), Literal(end)), + checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), CalendarInterval.fromString("interval -18173 days " + "-11 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds")) checkEvaluation( - TimestampDiff( + SubtractTimestamps( Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), Literal(Instant.parse("0001-01-01T00:00:00Z"))), CalendarInterval.fromString("interval 521722 weeks 4 days " + "23 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds")) } + + test("subtract dates") { + val end = LocalDate.of(2019, 10, 5) + checkEvaluation(SubtractDates(Literal(end), Literal(end)), + new CalendarInterval(0, 0)) + checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), + CalendarInterval.fromString("interval 1 days")) + checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), + CalendarInterval.fromString("interval -1 days")) + val epochDate = Literal(LocalDate.ofEpochDay(0)) + checkEvaluation(SubtractDates(Literal(end), epochDate), + CalendarInterval.fromString("interval 49 years 9 months 4 days")) + checkEvaluation(SubtractDates(epochDate, Literal(end)), + CalendarInterval.fromString("interval -49 years -9 months -4 days")) + checkEvaluation( + SubtractDates( + Literal(LocalDate.of(10000, 1, 1)), + Literal(LocalDate.of(1, 1, 1))), + CalendarInterval.fromString("interval 9999 years")) + } } diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index c3c131d22d0fb..0f4036cad6125 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -126,15 +126,15 @@ struct -- !query 14 select date '2001-10-01' - date '2001-09-28' -- !query 14 schema -struct +struct -- !query 14 output -3 +interval 3 days -- !query 15 select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' -- !query 15 schema -struct +struct -- !query 15 output interval 12 weeks 2 days 14 hours 48 minutes 47 seconds 654 milliseconds 322 microseconds @@ -142,6 +142,6 @@ interval 12 weeks 2 days 14 hours 48 minutes 47 seconds 654 milliseconds 322 mic -- !query 16 select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' -- !query 16 schema -struct +struct -- !query 16 output interval -12 weeks -2 days -14 hours -48 minutes -47 seconds -654 milliseconds -322 microseconds From 6d4cc7b855be670114eea3b4d23816d2c9c67e02 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 16 Oct 2019 22:43:47 +0800 Subject: [PATCH 0343/1387] [SPARK-27880][SQL] Add bool_and for every and bool_or for any as function aliases ### What changes were proposed in this pull request? bool_or(x) <=> any/some(x) <=> max(x) bool_and(x) <=> every(x) <=> min(x) Args: x: boolean ### Why are the changes needed? PostgreSQL, Presto and Vertica, etc also support this feature: ### Does this PR introduce any user-facing change? add new functions support ### How was this patch tested? add ut Closes #26126 from yaooqinn/SPARK-27880. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 4 +- .../aggregate/UnevaluableAggs.scala | 16 -- .../catalyst/optimizer/finishAnalysis.scala | 1 - .../ExpressionTypeCheckingSuite.scala | 1 - .../resources/sql-tests/inputs/group-by.sql | 18 ++- .../inputs/postgreSQL/aggregates_part2.sql | 68 ++++----- .../udf/postgreSQL/udf-aggregates_part2.sql | 68 ++++----- .../sql-tests/results/group-by.sql.out | 142 ++++++++++++------ .../postgreSQL/aggregates_part2.sql.out | 140 ++++++++++++----- .../postgreSQL/udf-aggregates_part2.sql.out | 140 ++++++++++++----- .../results/udf/udf-group-by.sql.out | 12 +- .../org/apache/spark/sql/ExplainSuite.scala | 2 +- 12 files changed, 382 insertions(+), 230 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index d5728b9027579..8cf0928c2afd9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -314,8 +314,10 @@ object FunctionRegistry { expression[CollectSet]("collect_set"), expression[CountMinSketchAgg]("count_min_sketch"), expression[EveryAgg]("every"), + expression[EveryAgg]("bool_and"), expression[AnyAgg]("any"), - expression[SomeAgg]("some"), + expression[AnyAgg]("some"), + expression[AnyAgg]("bool_or"), // string functions expression[Ascii]("ascii"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala index 4562fbcff5f3d..a8220ec641d6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala @@ -71,19 +71,3 @@ case class EveryAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { case class AnyAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { override def nodeName: String = "Any" } - -@ExpressionDescription( - usage = "_FUNC_(expr) - Returns true if at least one value of `expr` is true.", - examples = """ - Examples: - > SELECT _FUNC_(col) FROM VALUES (true), (false), (false) AS tab(col); - true - > SELECT _FUNC_(col) FROM VALUES (NULL), (true), (false) AS tab(col); - true - > SELECT _FUNC_(col) FROM VALUES (false), (false), (NULL) AS tab(col); - false - """, - since = "3.0.0") -case class SomeAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = "Some" -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 9d7564175314a..70277526cba8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -47,7 +47,6 @@ object ReplaceExpressions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case e: RuntimeReplaceable => e.child case CountIf(predicate) => Count(new NullIf(predicate, Literal.FalseLiteral)) - case SomeAgg(arg) => Max(arg) case AnyAgg(arg) => Max(arg) case EveryAgg(arg) => Min(arg) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 4440ac9e281c4..ed11bce5d12b4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -146,7 +146,6 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertSuccess(Min('arrayField)) assertSuccess(new EveryAgg('booleanField)) assertSuccess(new AnyAgg('booleanField)) - assertSuccess(new SomeAgg('booleanField)) assertError(Min('mapField), "min does not support ordering on type") assertError(Max('mapField), "max does not support ordering on type") diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 66bc90914e0d4..fcde225676cb9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -90,16 +90,16 @@ CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (5, null), (5, true), (5, false) AS test_agg(k, v); -- empty table -SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0; +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0; -- all null values -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4; +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4; -- aggregates are null Filtering -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5; +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5; -- group by -SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k; +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k; -- having SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false; @@ -137,10 +137,18 @@ SELECT any(1L); -- input type checking String SELECT every("true"); --- every/some/any aggregates are supported as windows expression. +-- input type checking Decimal +SELECT bool_and(1.0); + +-- input type checking double +SELECT bool_or(1.0D); + +-- every/some/any aggregates/bool_and/bool_or are supported as windows expression. SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; -- Having referencing aggregate expressions is ok. SELECT count(*) FROM test_agg HAVING count(*) > 1L; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql index 47f9d2f373069..144a3bc2a38fc 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql @@ -114,50 +114,40 @@ SELECT NOT (FALSE OR FALSE) AS `t`; -- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) --- CREATE TEMPORARY TABLE bool_test( --- b1 BOOL, --- b2 BOOL, --- b3 BOOL, --- b4 BOOL); +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4); -- empty case --- SELECT --- BOOL_AND(b1) AS "n", --- BOOL_OR(b3) AS "n" --- FROM bool_test; - --- COPY bool_test FROM STDIN NULL 'null'; --- TRUE null FALSE null --- FALSE TRUE null null --- null TRUE FALSE null --- \. +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0; --- SELECT --- BOOL_AND(b1) AS "f", --- BOOL_AND(b2) AS "t", --- BOOL_AND(b3) AS "f", --- BOOL_AND(b4) AS "n", --- BOOL_AND(NOT b2) AS "f", --- BOOL_AND(NOT b3) AS "t" --- FROM bool_test; +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test; --- SELECT --- EVERY(b1) AS "f", --- EVERY(b2) AS "t", --- EVERY(b3) AS "f", --- EVERY(b4) AS "n", --- EVERY(NOT b2) AS "f", --- EVERY(NOT b3) AS "t" --- FROM bool_test; +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test; --- SELECT --- BOOL_OR(b1) AS "t", --- BOOL_OR(b2) AS "t", --- BOOL_OR(b3) AS "f", --- BOOL_OR(b4) AS "n", --- BOOL_OR(NOT b2) AS "f", --- BOOL_OR(NOT b3) AS "t" --- FROM bool_test; +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test; -- -- Test cases that should be optimized into indexscans instead of diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql index a86bb0b47487b..d82fcd90a22af 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql @@ -116,50 +116,40 @@ SELECT NOT (FALSE OR FALSE) AS `t`; -- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) --- CREATE TEMPORARY TABLE bool_test( --- b1 BOOL, --- b2 BOOL, --- b3 BOOL, --- b4 BOOL); +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4); -- empty case --- SELECT --- BOOL_AND(b1) AS "n", --- BOOL_OR(b3) AS "n" --- FROM bool_test; - --- COPY bool_test FROM STDIN NULL 'null'; --- TRUE null FALSE null --- FALSE TRUE null null --- null TRUE FALSE null --- \. +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0; --- SELECT --- BOOL_AND(b1) AS "f", --- BOOL_AND(b2) AS "t", --- BOOL_AND(b3) AS "f", --- BOOL_AND(b4) AS "n", --- BOOL_AND(NOT b2) AS "f", --- BOOL_AND(NOT b3) AS "t" --- FROM bool_test; +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test; --- SELECT --- EVERY(b1) AS "f", --- EVERY(b2) AS "t", --- EVERY(b3) AS "f", --- EVERY(b4) AS "n", --- EVERY(NOT b2) AS "f", --- EVERY(NOT b3) AS "t" --- FROM bool_test; +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test; --- SELECT --- BOOL_OR(b1) AS "t", --- BOOL_OR(b2) AS "t", --- BOOL_OR(b3) AS "f", --- BOOL_OR(b4) AS "n", --- BOOL_OR(NOT b2) AS "f", --- BOOL_OR(NOT b3) AS "t" --- FROM bool_test; +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test; -- -- Test cases that should be optimized into indexscans instead of diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 3a5df254f2cd9..545aa238dd756 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 52 +-- Number of queries: 56 -- !query 0 @@ -291,39 +291,39 @@ struct<> -- !query 31 -SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0 +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output -NULL NULL NULL +NULL NULL NULL NULL NULL -- !query 32 -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4 +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output -NULL NULL NULL +NULL NULL NULL NULL NULL -- !query 33 -SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5 +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output -false true true +false true true false true -- !query 34 -SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k -- !query 34 schema -struct +struct -- !query 34 output -1 false true true -2 true true true -3 false false false -4 NULL NULL NULL -5 false true true +1 false true true false true +2 true true true true true +3 false false false false false +4 NULL NULL NULL NULL NULL +5 false true true false true -- !query 35 @@ -389,7 +389,7 @@ SELECT some(1S) struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve 'some(1S)' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 +cannot resolve 'any(1S)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 @@ -411,10 +411,28 @@ cannot resolve 'every('true')' due to data type mismatch: Input to function 'eve -- !query 43 -SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +SELECT bool_and(1.0) -- !query 43 schema -struct +struct<> -- !query 43 output +org.apache.spark.sql.AnalysisException +cannot resolve 'every(1.0BD)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [decimal(2,1)].; line 1 pos 7 + + +-- !query 44 +SELECT bool_or(1.0D) +-- !query 44 schema +struct<> +-- !query 44 output +org.apache.spark.sql.AnalysisException +cannot resolve 'any(1.0D)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [double].; line 1 pos 7 + + +-- !query 45 +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 45 schema +struct +-- !query 45 output 1 false false 1 true false 2 true true @@ -427,11 +445,11 @@ struct --- !query 44 output +-- !query 46 schema +struct +-- !query 46 output 1 false false 1 true true 2 true true @@ -444,11 +462,11 @@ struct --- !query 45 output +-- !query 47 output 1 false false 1 true true 2 true true @@ -461,37 +479,71 @@ struct +-- !query 48 output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query 49 +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 49 schema +struct +-- !query 49 output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query 50 SELECT count(*) FROM test_agg HAVING count(*) > 1L --- !query 46 schema +-- !query 50 schema struct --- !query 46 output +-- !query 50 output 10 --- !query 47 +-- !query 51 SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true --- !query 47 schema +-- !query 51 schema struct --- !query 47 output +-- !query 51 output 1 true 2 true 5 true --- !query 48 +-- !query 52 SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L --- !query 48 schema +-- !query 52 schema struct --- !query 48 output +-- !query 52 output 10 --- !query 49 +-- !query 53 SELECT count(*) FROM test_agg WHERE count(*) > 1L --- !query 49 schema +-- !query 53 schema struct<> --- !query 49 output +-- !query 53 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -499,11 +551,11 @@ Expression in where clause: [(count(1) > 1L)] Invalid expressions: [count(1)]; --- !query 50 +-- !query 54 SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L --- !query 50 schema +-- !query 54 schema struct<> --- !query 50 output +-- !query 54 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -511,11 +563,11 @@ Expression in where clause: [((count(1) + 1L) > 1L)] Invalid expressions: [count(1)]; --- !query 51 +-- !query 55 SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 --- !query 51 schema +-- !query 55 schema struct<> --- !query 51 output +-- !query 55 output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out index 2b5371a657196..d5362809c8049 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 16 +-- Number of queries: 21 -- !query 0 @@ -51,106 +51,170 @@ true false true false true true true true true -- !query 3 -select min(unique1) from tenk1 +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) -- !query 3 schema -struct +struct<> -- !query 3 output -0 + -- !query 4 -select max(unique1) from tenk1 +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 -- !query 4 schema -struct +struct -- !query 4 output -9999 +NULL NULL -- !query 5 -select max(unique1) from tenk1 where unique1 < 42 +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test -- !query 5 schema -struct +struct -- !query 5 output -41 +false true false NULL false true -- !query 6 -select max(unique1) from tenk1 where unique1 > 42 +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test -- !query 6 schema -struct +struct -- !query 6 output -9999 +false true false NULL false true -- !query 7 -select max(unique1) from tenk1 where unique1 > 42000 +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test -- !query 7 schema -struct +struct -- !query 7 output -NULL +true true false NULL false true -- !query 8 -select max(tenthous) from tenk1 where thousand = 33 +select min(unique1) from tenk1 -- !query 8 schema -struct +struct -- !query 8 output -9033 +0 -- !query 9 -select min(tenthous) from tenk1 where thousand = 33 +select max(unique1) from tenk1 -- !query 9 schema -struct +struct -- !query 9 output -33 +9999 -- !query 10 -select distinct max(unique2) from tenk1 +select max(unique1) from tenk1 where unique1 < 42 -- !query 10 schema -struct +struct -- !query 10 output -9999 +41 -- !query 11 -select max(unique2) from tenk1 order by 1 +select max(unique1) from tenk1 where unique1 > 42 -- !query 11 schema -struct +struct -- !query 11 output 9999 -- !query 12 -select max(unique2) from tenk1 order by max(unique2) +select max(unique1) from tenk1 where unique1 > 42000 -- !query 12 schema -struct +struct -- !query 12 output -9999 +NULL -- !query 13 -select max(unique2) from tenk1 order by max(unique2)+1 +select max(tenthous) from tenk1 where thousand = 33 -- !query 13 schema -struct +struct -- !query 13 output -9999 +9033 -- !query 14 -select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +select min(tenthous) from tenk1 where thousand = 33 -- !query 14 schema -struct +struct -- !query 14 output +33 + + +-- !query 15 +select distinct max(unique2) from tenk1 +-- !query 15 schema +struct +-- !query 15 output +9999 + + +-- !query 16 +select max(unique2) from tenk1 order by 1 +-- !query 16 schema +struct +-- !query 16 output +9999 + + +-- !query 17 +select max(unique2) from tenk1 order by max(unique2) +-- !query 17 schema +struct +-- !query 17 output +9999 + + +-- !query 18 +select max(unique2) from tenk1 order by max(unique2)+1 +-- !query 18 schema +struct +-- !query 18 output +9999 + + +-- !query 19 +select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query 19 schema +struct +-- !query 19 output 9999 3 9999 2 9999 1 --- !query 15 +-- !query 20 select max(100) from tenk1 --- !query 15 schema +-- !query 20 schema struct --- !query 15 output +-- !query 20 output 100 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out index ad2f1bdf77d7a..9efc7f65fdfc2 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 16 +-- Number of queries: 21 -- !query 0 @@ -51,106 +51,170 @@ true false true false true true true true true -- !query 3 -select min(udf(unique1)) from tenk1 +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) -- !query 3 schema -struct +struct<> -- !query 3 output -0 + -- !query 4 -select udf(max(unique1)) from tenk1 +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 -- !query 4 schema -struct +struct -- !query 4 output -9999 +NULL NULL -- !query 5 -select max(unique1) from tenk1 where udf(unique1) < 42 +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test -- !query 5 schema -struct +struct -- !query 5 output -41 +false true false NULL false true -- !query 6 -select max(unique1) from tenk1 where unique1 > udf(42) +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test -- !query 6 schema -struct +struct -- !query 6 output -9999 +false true false NULL false true -- !query 7 -select max(unique1) from tenk1 where udf(unique1) > 42000 +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test -- !query 7 schema -struct +struct -- !query 7 output -NULL +true true false NULL false true -- !query 8 -select max(tenthous) from tenk1 where udf(thousand) = 33 +select min(udf(unique1)) from tenk1 -- !query 8 schema -struct +struct -- !query 8 output -9033 +0 -- !query 9 -select min(tenthous) from tenk1 where udf(thousand) = 33 +select udf(max(unique1)) from tenk1 -- !query 9 schema -struct +struct -- !query 9 output -33 +9999 -- !query 10 -select distinct max(udf(unique2)) from tenk1 +select max(unique1) from tenk1 where udf(unique1) < 42 -- !query 10 schema -struct +struct -- !query 10 output -9999 +41 -- !query 11 -select max(unique2) from tenk1 order by udf(1) +select max(unique1) from tenk1 where unique1 > udf(42) -- !query 11 schema -struct +struct -- !query 11 output 9999 -- !query 12 -select max(unique2) from tenk1 order by max(udf(unique2)) +select max(unique1) from tenk1 where udf(unique1) > 42000 -- !query 12 schema -struct +struct -- !query 12 output -9999 +NULL -- !query 13 -select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 +select max(tenthous) from tenk1 where udf(thousand) = 33 -- !query 13 schema -struct +struct -- !query 13 output -9999 +9033 -- !query 14 -select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +select min(tenthous) from tenk1 where udf(thousand) = 33 -- !query 14 schema -struct +struct -- !query 14 output +33 + + +-- !query 15 +select distinct max(udf(unique2)) from tenk1 +-- !query 15 schema +struct +-- !query 15 output +9999 + + +-- !query 16 +select max(unique2) from tenk1 order by udf(1) +-- !query 16 schema +struct +-- !query 16 output +9999 + + +-- !query 17 +select max(unique2) from tenk1 order by max(udf(unique2)) +-- !query 17 schema +struct +-- !query 17 output +9999 + + +-- !query 18 +select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 +-- !query 18 schema +struct +-- !query 18 output +9999 + + +-- !query 19 +select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query 19 schema +struct +-- !query 19 output 9999 3 9999 2 9999 1 --- !query 15 +-- !query 20 select udf(max(100)) from tenk1 --- !query 15 schema +-- !query 20 schema struct --- !query 15 output +-- !query 20 output 100 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index febe47b5ba84e..b762d18fb8390 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -293,7 +293,7 @@ struct<> -- !query 31 SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output NULL NULL NULL @@ -301,7 +301,7 @@ NULL NULL NULL -- !query 32 SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output NULL NULL NULL @@ -309,7 +309,7 @@ NULL NULL NULL -- !query 33 SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output false true true @@ -317,7 +317,7 @@ false true true -- !query 34 SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) -- !query 34 schema -struct +struct -- !query 34 output 1 false true true 2 true true true @@ -389,7 +389,7 @@ SELECT some(udf(1S)) struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 +cannot resolve 'any(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 @@ -430,7 +430,7 @@ struct +struct -- !query 44 output 1 false false 1 true true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index 37183556d792a..630cf22fb20d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -95,7 +95,7 @@ class ExplainSuite extends QueryTest with SharedSparkSession { // plan should show the rewritten aggregate expression. val df = sql("SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k") checkKeywordsExistsInExplain(df, - "Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS some(v)#x, " + + "Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS any(v)#x, " + "max(v#x) AS any(v)#x]") } } From 8eb8f7478cac5fcfdec5bf5d76fa09cbf8e8d4bf Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 16 Oct 2019 15:38:54 -0700 Subject: [PATCH 0344/1387] [SPARK-29483][BUILD] Bump Jackson to 2.10.0 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Release blog: https://medium.com/cowtowncoder/jackson-2-10-features-cd880674d8a2 Fixes the following CVE's: https://www.cvedetails.com/cve/CVE-2019-16942/ https://www.cvedetails.com/cve/CVE-2019-16943/ Looking back, there were 3 major goals for this minor release: - Resolve the growing problem of “endless CVE patches”, a stream of fixes for reported CVEs related to “Polymorphic Deserialization” problem (described in “On Jackson CVEs… ”) that resulted in security tools forcing Jackson upgrades. 2.10 now includes “Safe Default Typing” that is hoped to resolve this problem. - Evolve 2.x API towards 3.0, based on changes that were done in master, within limits of 2.x API backwards-compatibility requirements. - Add JDK support for versions beyond Java 8: specifically add“module-info.class” for JDK9+, defining proper module definitions for Jackson components Full changelog: https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.10 Improved Scala 2.13 support: https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.10#scala ### Why are the changes needed? Patches CVE's reported by the vulnerability scanner. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Ran `mvn clean install -DskipTests` locally. Closes #26131 from Fokko/SPARK-29483. Authored-by: Fokko Driesprong Signed-off-by: Dongjoon Hyun --- LICENSE-binary | 1 + dev/deps/spark-deps-hadoop-2.7 | 17 +++++++++-------- dev/deps/spark-deps-hadoop-3.2 | 17 +++++++++-------- pom.xml | 5 ++--- 4 files changed, 21 insertions(+), 19 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index fa413ba19465c..7865d9df63144 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -544,6 +544,7 @@ Eclipse Distribution License (EDL) 1.0 -------------------------------------- org.glassfish.jaxb:jaxb-runtime +jakarta.activation:jakarta.activation-api jakarta.xml.bind:jakarta.xml.bind-api com.sun.istack:istack-commons-runtime diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index defe85e52cb32..81e32f13155f1 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -89,17 +89,18 @@ httpclient-4.5.6.jar httpcore-4.4.10.jar istack-commons-runtime-3.0.8.jar ivy-2.4.0.jar -jackson-annotations-2.9.10.jar -jackson-core-2.9.10.jar +jackson-annotations-2.10.0.jar +jackson-core-2.10.0.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.9.10.jar -jackson-dataformat-yaml-2.9.10.jar +jackson-databind-2.10.0.jar +jackson-dataformat-yaml-2.10.0.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.9.10.jar -jackson-module-paranamer-2.9.10.jar -jackson-module-scala_2.12-2.9.10.jar +jackson-module-jaxb-annotations-2.10.0.jar +jackson-module-paranamer-2.10.0.jar +jackson-module-scala_2.12-2.10.0.jar jackson-xc-1.9.13.jar +jakarta.activation-api-1.2.1.jar jakarta.annotation-api-1.3.4.jar jakarta.inject-2.5.0.jar jakarta.ws.rs-api-2.1.5.jar @@ -186,7 +187,7 @@ shapeless_2.12-2.3.3.jar shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar -snakeyaml-1.23.jar +snakeyaml-1.24.jar snappy-0.2.jar snappy-java-1.1.7.3.jar spire-macros_2.12-0.17.0-M1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 386e111792c49..e28418d8bee1f 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -102,17 +102,18 @@ httpclient-4.5.6.jar httpcore-4.4.10.jar istack-commons-runtime-3.0.8.jar ivy-2.4.0.jar -jackson-annotations-2.9.10.jar -jackson-core-2.9.10.jar +jackson-annotations-2.10.0.jar +jackson-core-2.10.0.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.9.10.jar -jackson-dataformat-yaml-2.9.10.jar +jackson-databind-2.10.0.jar +jackson-dataformat-yaml-2.10.0.jar jackson-jaxrs-base-2.9.5.jar jackson-jaxrs-json-provider-2.9.5.jar jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.9.10.jar -jackson-module-paranamer-2.9.10.jar -jackson-module-scala_2.12-2.9.10.jar +jackson-module-jaxb-annotations-2.10.0.jar +jackson-module-paranamer-2.10.0.jar +jackson-module-scala_2.12-2.10.0.jar +jakarta.activation-api-1.2.1.jar jakarta.annotation-api-1.3.4.jar jakarta.inject-2.5.0.jar jakarta.ws.rs-api-2.1.5.jar @@ -219,7 +220,7 @@ shapeless_2.12-2.3.3.jar shims-0.7.45.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar -snakeyaml-1.23.jar +snakeyaml-1.24.jar snappy-java-1.1.7.3.jar spire-macros_2.12-0.17.0-M1.jar spire-platform_2.12-0.17.0-M1.jar diff --git a/pom.xml b/pom.xml index 333a7d1b408e5..20a63d41f5bf5 100644 --- a/pom.xml +++ b/pom.xml @@ -170,8 +170,7 @@ true 1.9.13 - 2.9.10 - 2.9.10 + 2.10.0 1.1.7.3 1.1.2 1.10 @@ -698,7 +697,7 @@ com.fasterxml.jackson.core jackson-databind - ${fasterxml.jackson.databind.version} + ${fasterxml.jackson.version} com.fasterxml.jackson.core From fda4070ea934cac081162f70d9ea7fe2e9a07cd4 Mon Sep 17 00:00:00 2001 From: lajin Date: Wed, 16 Oct 2019 19:51:56 -0700 Subject: [PATCH 0345/1387] [SPARK-29283][SQL] Error message is hidden when query from JDBC, especially enabled adaptive execution ### What changes were proposed in this pull request? When adaptive execution is enabled, the Spark users who connected from JDBC always get adaptive execution error whatever the under root cause is. It's very confused. We have to check the driver log to find out why. ```shell 0: jdbc:hive2://localhost:10000> SELECT * FROM testData join testData2 ON key = v; SELECT * FROM testData join testData2 ON key = v; Error: Error running query: org.apache.spark.SparkException: Adaptive execution failed due to stage materialization failures. (state=,code=0) 0: jdbc:hive2://localhost:10000> ``` For example, a job queried from JDBC failed due to HDFS missing block. User still get the error message `Adaptive execution failed due to stage materialization failures`. The easiest way to reproduce is changing the code of `AdaptiveSparkPlanExec`, to let it throws out an exception when it faces `StageSuccess`. ```scala case class AdaptiveSparkPlanExec( events.drainTo(rem) (Seq(nextMsg) ++ rem.asScala).foreach { case StageSuccess(stage, res) => // stage.resultOption = Some(res) val ex = new SparkException("Wrapper Exception", new IllegalArgumentException("Root cause is IllegalArgumentException for Test")) errors.append( new SparkException(s"Failed to materialize query stage: ${stage.treeString}", ex)) case StageFailure(stage, ex) => errors.append( new SparkException(s"Failed to materialize query stage: ${stage.treeString}", ex)) ``` ### Why are the changes needed? To make the error message more user-friend and more useful for query from JDBC. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually test query: ```shell 0: jdbc:hive2://localhost:10000> CREATE TEMPORARY VIEW testData (key, value) AS SELECT explode(array(1, 2, 3, 4)), cast(substring(rand(), 3, 4) as string); CREATE TEMPORARY VIEW testData (key, value) AS SELECT explode(array(1, 2, 3, 4)), cast(substring(rand(), 3, 4) as string); +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (0.225 seconds) 0: jdbc:hive2://localhost:10000> CREATE TEMPORARY VIEW testData2 (k, v) AS SELECT explode(array(1, 1, 2, 2)), cast(substring(rand(), 3, 4) as int); CREATE TEMPORARY VIEW testData2 (k, v) AS SELECT explode(array(1, 1, 2, 2)), cast(substring(rand(), 3, 4) as int); +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (0.043 seconds) ``` Before: ```shell 0: jdbc:hive2://localhost:10000> SELECT * FROM testData join testData2 ON key = v; SELECT * FROM testData join testData2 ON key = v; Error: Error running query: org.apache.spark.SparkException: Adaptive execution failed due to stage materialization failures. (state=,code=0) 0: jdbc:hive2://localhost:10000> ``` After: ```shell 0: jdbc:hive2://localhost:10000> SELECT * FROM testData join testData2 ON key = v; SELECT * FROM testData join testData2 ON key = v; Error: Error running query: java.lang.IllegalArgumentException: Root cause is IllegalArgumentException for Test (state=,code=0) 0: jdbc:hive2://localhost:10000> ``` Closes #25960 from LantaoJin/SPARK-29283. Authored-by: lajin Signed-off-by: Yuming Wang --- .../SparkExecuteStatementOperation.scala | 17 +++++++++++------ .../SparkGetCatalogsOperation.scala | 18 ++++++++++++++---- .../SparkGetColumnsOperation.scala | 18 ++++++++++++++---- .../SparkGetFunctionsOperation.scala | 18 ++++++++++++++---- .../SparkGetSchemasOperation.scala | 18 ++++++++++++++---- .../SparkGetTableTypesOperation.scala | 18 ++++++++++++++---- .../SparkGetTablesOperation.scala | 19 ++++++++++++++----- .../SparkGetTypeInfoOperation.scala | 18 ++++++++++++++---- 8 files changed, 109 insertions(+), 35 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 90a428defd76d..68197a9de8566 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -26,6 +26,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.shims.Utils import org.apache.hive.service.cli._ @@ -312,12 +313,16 @@ private[hive] class SparkExecuteStatementOperation( } else { logError(s"Error executing query with $statementId, currentState $currentState, ", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - if (e.isInstanceOf[HiveSQLException]) { - throw e.asInstanceOf[HiveSQLException] - } else { - throw new HiveSQLException("Error running query: " + e.toString, e) + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error running query: " + root.toString, root) } } } finally { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index cde99fd35bd59..6c8a5b00992da 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.GetCatalogsOperation @@ -68,11 +69,20 @@ private[hive] class SparkGetCatalogsOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get catalogs operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting catalogs: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 89faff2f6f913..f845a2285b9a3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -22,6 +22,7 @@ import java.util.regex.Pattern import scala.collection.JavaConverters.seqAsJavaListConverter +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObject} import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType import org.apache.hive.service.cli._ @@ -129,11 +130,20 @@ private[hive] class SparkGetColumnsOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get columns operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting columns: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index 462e57300e82b..1cdd8918421bb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -22,6 +22,7 @@ import java.util.UUID import scala.collection.JavaConverters.seqAsJavaListConverter +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils} import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetFunctionsOperation @@ -104,11 +105,20 @@ private[hive] class SparkGetFunctionsOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get functions operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting functions: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 87ef154bcc8ab..928610a6bcff9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID import java.util.regex.Pattern +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetSchemasOperation @@ -87,11 +88,20 @@ private[hive] class SparkGetSchemasOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get schemas operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting schemas: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index 8f2257f77d2a0..ec03f1e148e69 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.GetTableTypesOperation @@ -74,11 +75,20 @@ private[hive] class SparkGetTableTypesOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get table types operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting table types: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 6441dc50f49fe..bf9cf7ad46d95 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -22,6 +22,7 @@ import java.util.regex.Pattern import scala.collection.JavaConverters._ +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils import org.apache.hive.service.cli._ @@ -30,7 +31,6 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.util.{Utils => SparkUtils} @@ -119,11 +119,20 @@ private[hive] class SparkGetTablesOperation( } setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get tables operation with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting tables: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index 7a6a8c59b7216..0d263b09d57d3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.util.UUID +import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli.{HiveSQLException, OperationState} import org.apache.hive.service.cli.operation.GetTypeInfoOperation @@ -92,11 +93,20 @@ private[hive] class SparkGetTypeInfoOperation( }) setState(OperationState.FINISHED) } catch { - case e: HiveSQLException => + case e: Throwable => + logError(s"Error executing get type info with $statementId", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, SparkUtils.exceptionString(e)) - throw e + e match { + case hiveException: HiveSQLException => + HiveThriftServer2.listener.onStatementError( + statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) + throw hiveException + case _ => + val root = ExceptionUtils.getRootCause(e) + HiveThriftServer2.listener.onStatementError( + statementId, root.getMessage, SparkUtils.exceptionString(root)) + throw new HiveSQLException("Error getting type info: " + root.toString, root) + } } HiveThriftServer2.listener.onStatementFinish(statementId) } From 239ee3f5611889683e4a432a544bd790633680f1 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 17 Oct 2019 14:47:56 +0800 Subject: [PATCH 0346/1387] [SPARK-9853][CORE] Optimize shuffle fetch of continuous partition IDs This PR takes over #19788. After we split the shuffle fetch protocol from `OpenBlock` in #24565, this optimization can be extended in the new shuffle protocol. Credit to yucai, closes #19788. ### What changes were proposed in this pull request? This PR adds the support for continuous shuffle block fetching in batch: - Shuffle client changes: - Add new feature tag `spark.shuffle.fetchContinuousBlocksInBatch`, implement the decision logic in `BlockStoreShuffleReader`. - Merge the continuous shuffle block ids in batch if needed in ShuffleBlockFetcherIterator. - Shuffle server changes: - Add support in `ExternalBlockHandler` for the external shuffle service side. - Make `ShuffleBlockResolver.getBlockData` accept getting block data by range. - Protocol changes: - Add new block id type `ShuffleBlockBatchId` represent continuous shuffle block ids. - Extend `FetchShuffleBlocks` and `OneForOneBlockFetcher`. - After the new shuffle fetch protocol completed in #24565, the backward compatibility for external shuffle service can be controlled by `spark.shuffle.useOldFetchProtocol`. ### Why are the changes needed? In adaptive execution, one reducer may fetch multiple continuous shuffle blocks from one map output file. However, as the original approach, each reducer needs to fetch those 10 reducer blocks one by one. This way needs many IO and impacts performance. This PR is to support fetching those continuous shuffle blocks in one IO (batch way). See below example: The shuffle block is stored like below: ![image](https://user-images.githubusercontent.com/2989575/51654634-c37fbd80-1fd3-11e9-935e-5652863676c3.png) The ShuffleId format is s"shuffle_$shuffleId_$mapId_$reduceId", referring to BlockId.scala. In adaptive execution, one reducer may want to read output for reducer 5 to 14, whose block Ids are from shuffle_0_x_5 to shuffle_0_x_14. Before this PR, Spark needs 10 disk IOs + 10 network IOs for each output file. After this PR, Spark only needs 1 disk IO and 1 network IO. This way can reduce IO dramatically. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Add new UT. Integrate test with setting `spark.sql.adaptive.enabled=true`. Closes #26040 from xuanyuanking/SPARK-9853. Lead-authored-by: Yuanjian Li Co-authored-by: yucai Signed-off-by: Wenchen Fan --- .../network/shuffle/ExternalBlockHandler.java | 28 +++-- .../shuffle/ExternalShuffleBlockResolver.java | 24 +++- .../shuffle/OneForOneBlockFetcher.java | 35 +++--- .../shuffle/ShuffleIndexInformation.java | 11 +- .../shuffle/protocol/FetchShuffleBlocks.java | 24 +++- .../shuffle/BlockTransferMessagesSuite.java | 5 +- .../shuffle/ExternalBlockHandlerSuite.java | 25 ++++- .../ExternalShuffleBlockResolverSuite.java | 7 ++ .../shuffle/OneForOneBlockFetcherSuite.java | 22 +++- .../network/netty/NettyBlockRpcServer.scala | 27 ++++- .../spark/serializer/SerializerManager.scala | 1 + .../shuffle/BlockStoreShuffleReader.scala | 31 +++++- .../shuffle/IndexShuffleBlockResolver.scala | 27 +++-- .../spark/shuffle/ShuffleBlockResolver.scala | 4 +- .../shuffle/sort/SortShuffleManager.scala | 24 +++- .../org/apache/spark/storage/BlockId.scala | 17 ++- .../apache/spark/storage/BlockManager.scala | 2 +- .../storage/ShuffleBlockFetcherIterator.scala | 71 ++++++++++-- .../BlockStoreShuffleReaderSuite.scala | 3 +- .../apache/spark/storage/BlockIdSuite.scala | 14 +++ .../ShuffleBlockFetcherIteratorSuite.scala | 103 ++++++++++++++++-- .../apache/spark/sql/internal/SQLConf.scala | 13 +++ .../spark/sql/execution/ShuffledRowRDD.scala | 7 ++ 23 files changed, 443 insertions(+), 82 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 2d7a72315cf23..b886fce9be21a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -102,8 +102,12 @@ protected void handleMessage( FetchShuffleBlocks msg = (FetchShuffleBlocks) msgObj; checkAuth(client, msg.appId); numBlockIds = 0; - for (int[] ids: msg.reduceIds) { - numBlockIds += ids.length; + if (msg.batchFetchEnabled) { + numBlockIds = msg.mapIds.length; + } else { + for (int[] ids: msg.reduceIds) { + numBlockIds += ids.length; + } } streamId = streamManager.registerStream(client.getClientId(), new ShuffleManagedBufferIterator(msg), client.getChannel()); @@ -323,6 +327,7 @@ private class ShuffleManagedBufferIterator implements Iterator { private final int shuffleId; private final long[] mapIds; private final int[][] reduceIds; + private final boolean batchFetchEnabled; ShuffleManagedBufferIterator(FetchShuffleBlocks msg) { appId = msg.appId; @@ -330,6 +335,7 @@ private class ShuffleManagedBufferIterator implements Iterator { shuffleId = msg.shuffleId; mapIds = msg.mapIds; reduceIds = msg.reduceIds; + batchFetchEnabled = msg.batchFetchEnabled; } @Override @@ -343,12 +349,20 @@ public boolean hasNext() { @Override public ManagedBuffer next() { - final ManagedBuffer block = blockManager.getBlockData( - appId, execId, shuffleId, mapIds[mapIdx], reduceIds[mapIdx][reduceIdx]); - if (reduceIdx < reduceIds[mapIdx].length - 1) { - reduceIdx += 1; + ManagedBuffer block; + if (!batchFetchEnabled) { + block = blockManager.getBlockData( + appId, execId, shuffleId, mapIds[mapIdx], reduceIds[mapIdx][reduceIdx]); + if (reduceIdx < reduceIds[mapIdx].length - 1) { + reduceIdx += 1; + } else { + reduceIdx = 0; + mapIdx += 1; + } } else { - reduceIdx = 0; + assert(reduceIds[mapIdx].length == 2); + block = blockManager.getContinuousBlocksData(appId, execId, shuffleId, mapIds[mapIdx], + reduceIds[mapIdx][0], reduceIds[mapIdx][1]); mapIdx += 1; } metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 8b0d1e145a813..beca5d6e5a788 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -165,8 +165,7 @@ public void registerExecutor( } /** - * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions - * about how the hash and sort based shuffles store their data. + * Obtains a FileSegmentManagedBuffer from a single block (shuffleId, mapId, reduceId). */ public ManagedBuffer getBlockData( String appId, @@ -174,12 +173,26 @@ public ManagedBuffer getBlockData( int shuffleId, long mapId, int reduceId) { + return getContinuousBlocksData(appId, execId, shuffleId, mapId, reduceId, reduceId + 1); + } + + /** + * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, [startReduceId, endReduceId)). + * We make assumptions about how the hash and sort based shuffles store their data. + */ + public ManagedBuffer getContinuousBlocksData( + String appId, + String execId, + int shuffleId, + long mapId, + int startReduceId, + int endReduceId) { ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); if (executor == null) { throw new RuntimeException( String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); } - return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); + return getSortBasedShuffleBlockData(executor, shuffleId, mapId, startReduceId, endReduceId); } public ManagedBuffer getRddBlockData( @@ -296,13 +309,14 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) { * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, long mapId, int reduceId) { + ExecutorShuffleInfo executor, int shuffleId, long mapId, int startReduceId, int endReduceId) { File indexFile = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, "shuffle_" + shuffleId + "_" + mapId + "_0.index"); try { ShuffleIndexInformation shuffleIndexInformation = shuffleIndexCache.get(indexFile); - ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(reduceId); + ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex( + startReduceId, endReduceId); return new FileSegmentManagedBuffer( conf, ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 52854c86be3e6..ab373a7f03d9d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -25,7 +25,6 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; -import org.apache.commons.lang3.tuple.ImmutableTriple; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -113,39 +112,47 @@ private boolean isShuffleBlocks(String[] blockIds) { */ private FetchShuffleBlocks createFetchShuffleBlocksMsg( String appId, String execId, String[] blockIds) { - int shuffleId = splitBlockId(blockIds[0]).left; + String[] firstBlock = splitBlockId(blockIds[0]); + int shuffleId = Integer.parseInt(firstBlock[1]); + boolean batchFetchEnabled = firstBlock.length == 5; + HashMap> mapIdToReduceIds = new HashMap<>(); for (String blockId : blockIds) { - ImmutableTriple blockIdParts = splitBlockId(blockId); - if (blockIdParts.left != shuffleId) { + String[] blockIdParts = splitBlockId(blockId); + if (Integer.parseInt(blockIdParts[1]) != shuffleId) { throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockId); } - long mapId = blockIdParts.middle; + long mapId = Long.parseLong(blockIdParts[2]); if (!mapIdToReduceIds.containsKey(mapId)) { mapIdToReduceIds.put(mapId, new ArrayList<>()); } - mapIdToReduceIds.get(mapId).add(blockIdParts.right); + mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[3])); + if (batchFetchEnabled) { + // When we read continuous shuffle blocks in batch, we will reuse reduceIds in + // FetchShuffleBlocks to store the start and end reduce id for range + // [startReduceId, endReduceId). + assert(blockIdParts.length == 5); + mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[4])); + } } long[] mapIds = Longs.toArray(mapIdToReduceIds.keySet()); int[][] reduceIdArr = new int[mapIds.length][]; for (int i = 0; i < mapIds.length; i++) { reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); } - return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIdArr); + return new FetchShuffleBlocks( + appId, execId, shuffleId, mapIds, reduceIdArr, batchFetchEnabled); } - /** Split the shuffleBlockId and return shuffleId, mapId and reduceId. */ - private ImmutableTriple splitBlockId(String blockId) { + /** Split the shuffleBlockId and return shuffleId, mapId and reduceIds. */ + private String[] splitBlockId(String blockId) { String[] blockIdParts = blockId.split("_"); - if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { + if (blockIdParts.length < 4 || blockIdParts.length > 5 || !blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException( "Unexpected shuffle block id format: " + blockId); } - return new ImmutableTriple<>( - Integer.parseInt(blockIdParts[1]), - Long.parseLong(blockIdParts[2]), - Integer.parseInt(blockIdParts[3])); + return blockIdParts; } /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java index 371149bef3974..b65aacfcc4b9e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java @@ -54,8 +54,15 @@ public int getSize() { * Get index offset for a particular reducer. */ public ShuffleIndexRecord getIndex(int reduceId) { - long offset = offsets.get(reduceId); - long nextOffset = offsets.get(reduceId + 1); + return getIndex(reduceId, reduceId + 1); + } + + /** + * Get index offset for the reducer range of [startReduceId, endReduceId). + */ + public ShuffleIndexRecord getIndex(int startReduceId, int endReduceId) { + long offset = offsets.get(startReduceId); + long nextOffset = offsets.get(endReduceId); return new ShuffleIndexRecord(offset, nextOffset - offset); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java index faa960d414bcc..c0f307af042e2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -35,20 +35,32 @@ public class FetchShuffleBlocks extends BlockTransferMessage { // The length of mapIds must equal to reduceIds.size(), for the i-th mapId in mapIds, // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map id. public final long[] mapIds; + // When batchFetchEnabled=true, reduceIds[i] contains 2 elements: startReduceId (inclusive) and + // endReduceId (exclusive) for the mapper mapIds[i]. + // When batchFetchEnabled=false, reduceIds[i] contains all the reduce IDs that mapper mapIds[i] + // needs to fetch. public final int[][] reduceIds; + public final boolean batchFetchEnabled; public FetchShuffleBlocks( String appId, String execId, int shuffleId, long[] mapIds, - int[][] reduceIds) { + int[][] reduceIds, + boolean batchFetchEnabled) { this.appId = appId; this.execId = execId; this.shuffleId = shuffleId; this.mapIds = mapIds; this.reduceIds = reduceIds; assert(mapIds.length == reduceIds.length); + this.batchFetchEnabled = batchFetchEnabled; + if (batchFetchEnabled) { + for (int[] ids: reduceIds) { + assert(ids.length == 2); + } + } } @Override @@ -62,6 +74,7 @@ public String toString() { .add("shuffleId", shuffleId) .add("mapIds", Arrays.toString(mapIds)) .add("reduceIds", Arrays.deepToString(reduceIds)) + .add("batchFetchEnabled", batchFetchEnabled) .toString(); } @@ -73,6 +86,7 @@ public boolean equals(Object o) { FetchShuffleBlocks that = (FetchShuffleBlocks) o; if (shuffleId != that.shuffleId) return false; + if (batchFetchEnabled != that.batchFetchEnabled) return false; if (!appId.equals(that.appId)) return false; if (!execId.equals(that.execId)) return false; if (!Arrays.equals(mapIds, that.mapIds)) return false; @@ -86,6 +100,7 @@ public int hashCode() { result = 31 * result + shuffleId; result = 31 * result + Arrays.hashCode(mapIds); result = 31 * result + Arrays.deepHashCode(reduceIds); + result = 31 * result + (batchFetchEnabled ? 1 : 0); return result; } @@ -100,7 +115,8 @@ public int encodedLength() { + 4 /* encoded length of shuffleId */ + Encoders.LongArrays.encodedLength(mapIds) + 4 /* encoded length of reduceIds.size() */ - + encodedLengthOfReduceIds; + + encodedLengthOfReduceIds + + 1; /* encoded length of batchFetchEnabled */ } @Override @@ -113,6 +129,7 @@ public void encode(ByteBuf buf) { for (int[] ids: reduceIds) { Encoders.IntArrays.encode(buf, ids); } + buf.writeBoolean(batchFetchEnabled); } public static FetchShuffleBlocks decode(ByteBuf buf) { @@ -125,6 +142,7 @@ public static FetchShuffleBlocks decode(ByteBuf buf) { for (int i = 0; i < reduceIdsSize; i++) { reduceIds[i] = Encoders.IntArrays.decode(buf); } - return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIds); + boolean batchFetchEnabled = buf.readBoolean(); + return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIds, batchFetchEnabled); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java index ba40f4a45ac8f..fd2c67a3a270a 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java @@ -30,7 +30,10 @@ public void serializeOpenShuffleBlocks() { checkSerializeDeserialize(new OpenBlocks("app-1", "exec-2", new String[] { "b1", "b2" })); checkSerializeDeserialize(new FetchShuffleBlocks( "app-1", "exec-2", 0, new long[] {0, 1}, - new int[][] {{ 0, 1 }, { 0, 1, 2 }})); + new int[][] {{ 0, 1 }, { 0, 1, 2 }}, false)); + checkSerializeDeserialize(new FetchShuffleBlocks( + "app-1", "exec-2", 0, new long[] {0, 1}, + new int[][] {{ 0, 1 }, { 0, 2 }}, true)); checkSerializeDeserialize(new RegisterExecutor("app-1", "exec-2", new ExecutorShuffleInfo( new String[] { "/local1", "/local2" }, 32, "MyShuffleManager"))); checkSerializeDeserialize(new UploadBlock("app-1", "exec-2", "block-3", new byte[] { 1, 2 }, diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index 6a5d04b6f417b..455351fcf767c 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -101,7 +101,7 @@ public void testFetchShuffleBlocks() { when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(blockMarkers[1]); FetchShuffleBlocks fetchShuffleBlocks = new FetchShuffleBlocks( - "app0", "exec1", 0, new long[] { 0 }, new int[][] {{ 0, 1 }}); + "app0", "exec1", 0, new long[] { 0 }, new int[][] {{ 0, 1 }}, false); checkOpenBlocksReceive(fetchShuffleBlocks, blockMarkers); verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); @@ -109,6 +109,22 @@ public void testFetchShuffleBlocks() { verifyOpenBlockLatencyMetrics(); } + @Test + public void testFetchShuffleBlocksInBatch() { + ManagedBuffer[] batchBlockMarkers = { + new NioManagedBuffer(ByteBuffer.wrap(new byte[10])) + }; + when(blockResolver.getContinuousBlocksData( + "app0", "exec1", 0, 0, 0, 1)).thenReturn(batchBlockMarkers[0]); + + FetchShuffleBlocks fetchShuffleBlocks = new FetchShuffleBlocks( + "app0", "exec1", 0, new long[] { 0 }, new int[][] {{ 0, 1 }}, true); + checkOpenBlocksReceive(fetchShuffleBlocks, batchBlockMarkers); + + verify(blockResolver, times(1)).getContinuousBlocksData("app0", "exec1", 0, 0, 0, 1); + verifyOpenBlockLatencyMetrics(); + } + @Test public void testOpenDiskPersistedRDDBlocks() { when(blockResolver.getRddBlockData("app0", "exec1", 0, 0)).thenReturn(blockMarkers[0]); @@ -154,7 +170,7 @@ private void checkOpenBlocksReceive(BlockTransferMessage msg, ManagedBuffer[] bl StreamHandle handle = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); - assertEquals(2, handle.numChunks); + assertEquals(blockMarkers.length, handle.numChunks); @SuppressWarnings("unchecked") ArgumentCaptor> stream = (ArgumentCaptor>) @@ -162,8 +178,9 @@ private void checkOpenBlocksReceive(BlockTransferMessage msg, ManagedBuffer[] bl verify(streamManager, times(1)).registerStream(anyString(), stream.capture(), any()); Iterator buffers = stream.getValue(); - assertEquals(blockMarkers[0], buffers.next()); - assertEquals(blockMarkers[1], buffers.next()); + for (ManagedBuffer blockMarker : blockMarkers) { + assertEquals(blockMarker, buffers.next()); + } assertFalse(buffers.hasNext()); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 09eb699be305a..09b31430b1eb9 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -111,6 +111,13 @@ public void testSortShuffleBlocks() throws IOException { CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); assertEquals(sortBlock1, block1); } + + try (InputStream blocksStream = resolver.getContinuousBlocksData( + "app0", "exec0", 0, 0, 0, 2).createInputStream()) { + String blocks = + CharStreams.toString(new InputStreamReader(blocksStream, StandardCharsets.UTF_8)); + assertEquals(sortBlock0 + sortBlock1, blocks); + } } @Test diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 26a11672b8068..285eedb39c65c 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -64,7 +64,7 @@ public void testFetchOne() { BlockFetchingListener listener = fetchBlocks( blocks, blockIds, - new FetchShuffleBlocks("app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0 }}), + new FetchShuffleBlocks("app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0 }}, false), conf); verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); @@ -100,7 +100,8 @@ public void testFetchThreeShuffleBlocks() { BlockFetchingListener listener = fetchBlocks( blocks, blockIds, - new FetchShuffleBlocks("app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0, 1, 2 }}), + new FetchShuffleBlocks( + "app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0, 1, 2 }}, false), conf); for (int i = 0; i < 3; i ++) { @@ -109,6 +110,23 @@ public void testFetchThreeShuffleBlocks() { } } + @Test + public void testBatchFetchThreeShuffleBlocks() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0_3", new NioManagedBuffer(ByteBuffer.wrap(new byte[58]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks( + "app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0, 3 }}, true), + conf); + + verify(listener, times(1)).onBlockFetchSuccess( + "shuffle_0_0_0_3", blocks.get("shuffle_0_0_0_3")); + } + @Test public void testFetchThree() { LinkedHashMap blocks = Maps.newLinkedHashMap(); diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index b2ab31488e4c1..3a41c5f73c0ac 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -29,7 +29,7 @@ import org.apache.spark.network.client.{RpcResponseCallback, StreamCallbackWithI import org.apache.spark.network.server.{OneForOneStreamManager, RpcHandler, StreamManager} import org.apache.spark.network.shuffle.protocol._ import org.apache.spark.serializer.Serializer -import org.apache.spark.storage.{BlockId, ShuffleBlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, ShuffleBlockBatchId, ShuffleBlockId, StorageLevel} /** * Serves requests to open blocks by simply registering one chunk per block requested. @@ -65,12 +65,29 @@ class NettyBlockRpcServer( case fetchShuffleBlocks: FetchShuffleBlocks => val blocks = fetchShuffleBlocks.mapIds.zipWithIndex.flatMap { case (mapId, index) => - fetchShuffleBlocks.reduceIds.apply(index).map { reduceId => - blockManager.getBlockData( - ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapId, reduceId)) + if (!fetchShuffleBlocks.batchFetchEnabled) { + fetchShuffleBlocks.reduceIds(index).map { reduceId => + blockManager.getBlockData( + ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapId, reduceId)) + } + } else { + val startAndEndId = fetchShuffleBlocks.reduceIds(index) + if (startAndEndId.length != 2) { + throw new IllegalStateException(s"Invalid shuffle fetch request when batch mode " + + s"is enabled: $fetchShuffleBlocks") + } + Array(blockManager.getBlockData( + ShuffleBlockBatchId( + fetchShuffleBlocks.shuffleId, mapId, startAndEndId(0), startAndEndId(1)))) } } - val numBlockIds = fetchShuffleBlocks.reduceIds.map(_.length).sum + + val numBlockIds = if (fetchShuffleBlocks.batchFetchEnabled) { + fetchShuffleBlocks.mapIds.length + } else { + fetchShuffleBlocks.reduceIds.map(_.length).sum + } + val streamId = streamManager.registerStream(appId, blocks.iterator.asJava, client.getChannel) logTrace(s"Registered streamId $streamId with $numBlockIds buffers") diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 3e3c387911d36..623db9d00ab53 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -114,6 +114,7 @@ private[spark] class SerializerManager( case _: RDDBlockId => compressRdds case _: TempLocalBlockId => compressShuffleSpill case _: TempShuffleBlockId => compressShuffle + case _: ShuffleBlockBatchId => compressShuffle case _ => false } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 3737102a1abab..14080f8822f9e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -19,14 +19,14 @@ package org.apache.spark.shuffle import org.apache.spark._ import org.apache.spark.internal.{config, Logging} +import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.SerializerManager import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter /** - * Fetches and reads the partitions in range [startPartition, endPartition) from a shuffle by - * requesting them from other nodes' block stores. + * Fetches and reads the blocks from a shuffle by requesting them from other nodes' block stores. */ private[spark] class BlockStoreShuffleReader[K, C]( handle: BaseShuffleHandle[K, _, C], @@ -35,11 +35,33 @@ private[spark] class BlockStoreShuffleReader[K, C]( readMetrics: ShuffleReadMetricsReporter, serializerManager: SerializerManager = SparkEnv.get.serializerManager, blockManager: BlockManager = SparkEnv.get.blockManager, - mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker) + mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, + shouldBatchFetch: Boolean) extends ShuffleReader[K, C] with Logging { private val dep = handle.dependency + private def fetchContinuousBlocksInBatch: Boolean = { + val conf = SparkEnv.get.conf + val serializerRelocatable = dep.serializer.supportsRelocationOfSerializedObjects + val compressed = conf.get(config.SHUFFLE_COMPRESS) + val codecConcatenation = if (compressed) { + CompressionCodec.supportsConcatenationOfSerializedStreams(CompressionCodec.createCodec(conf)) + } else { + true + } + + val doBatchFetch = shouldBatchFetch && serializerRelocatable && + (!compressed || codecConcatenation) + if (shouldBatchFetch && !doBatchFetch) { + logDebug("The feature tag of continuous shuffle block fetching is set to true, but " + + "we can not enable the feature because other conditions are not satisfied. " + + s"Shuffle compress: $compressed, serializer relocatable: $serializerRelocatable, " + + s"codec concatenation: $codecConcatenation.") + } + doBatchFetch + } + /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { val wrappedStreams = new ShuffleBlockFetcherIterator( @@ -55,7 +77,8 @@ private[spark] class BlockStoreShuffleReader[K, C]( SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM), SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT), SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT_MEMORY), - readMetrics).toCompletionIterator + readMetrics, + fetchContinuousBlocksInBatch).toCompletionIterator val serializerInstance = dep.serializer.newInstance() diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 332164a7be3e7..8b3993e21f077 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -190,10 +190,18 @@ private[spark] class IndexShuffleBlockResolver( } } - override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { + override def getBlockData(blockId: BlockId): ManagedBuffer = { + val (shuffleId, mapId, startReduceId, endReduceId) = blockId match { + case id: ShuffleBlockId => + (id.shuffleId, id.mapId, id.reduceId, id.reduceId + 1) + case batchId: ShuffleBlockBatchId => + (batchId.shuffleId, batchId.mapId, batchId.startReduceId, batchId.endReduceId) + case _ => + throw new IllegalArgumentException("unexpected shuffle block id format: " + blockId) + } // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index - val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) + val indexFile = getIndexFile(shuffleId, mapId) // SPARK-22982: if this FileInputStream's position is seeked forward by another piece of code // which is incorrectly using our file descriptor then this code will fetch the wrong offsets @@ -202,22 +210,23 @@ private[spark] class IndexShuffleBlockResolver( // class of issue from re-occurring in the future which is why they are left here even though // SPARK-22982 is fixed. val channel = Files.newByteChannel(indexFile.toPath) - channel.position(blockId.reduceId * 8L) + channel.position(startReduceId * 8L) val in = new DataInputStream(Channels.newInputStream(channel)) try { - val offset = in.readLong() - val nextOffset = in.readLong() + val startOffset = in.readLong() + channel.position(endReduceId * 8L) + val endOffset = in.readLong() val actualPosition = channel.position() - val expectedPosition = blockId.reduceId * 8L + 16 + val expectedPosition = endReduceId * 8L + 8 if (actualPosition != expectedPosition) { throw new Exception(s"SPARK-22982: Incorrect channel position after index file reads: " + s"expected $expectedPosition but actual position was $actualPosition.") } new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapId), - offset, - nextOffset - offset) + getDataFile(shuffleId, mapId), + startOffset, + endOffset - startOffset) } finally { in.close() } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala index d1ecbc1bf0178..c50789658d61a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala @@ -18,7 +18,7 @@ package org.apache.spark.shuffle import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.storage.BlockId private[spark] /** @@ -34,7 +34,7 @@ trait ShuffleBlockResolver { * Retrieve the data for the specified block. If the data for that block is not available, * throws an unspecified exception. */ - def getBlockData(blockId: ShuffleBlockId): ManagedBuffer + def getBlockData(blockId: BlockId): ManagedBuffer def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index a3529378a4d64..3cd04de0f741c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -127,7 +127,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId( handle.shuffleId, startPartition, endPartition) new BlockStoreShuffleReader( - handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics) + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, + shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) } override def getReaderForOneMapper[K, C]( @@ -140,7 +141,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByMapIndex( handle.shuffleId, mapIndex, startPartition, endPartition) new BlockStoreShuffleReader( - handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics) + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, + shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) } /** Get a writer for a given partition. Called on executors by map tasks. */ @@ -201,10 +203,26 @@ private[spark] object SortShuffleManager extends Logging { * The maximum number of shuffle output partitions that SortShuffleManager supports when * buffering map outputs in a serialized form. This is an extreme defensive programming measure, * since it's extremely unlikely that a single shuffle produces over 16 million output partitions. - * */ + */ val MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE = PackedRecordPointer.MAXIMUM_PARTITION_ID + 1 + /** + * The local property key for continuous shuffle block fetching feature. + */ + val FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED_KEY = + "__fetch_continuous_blocks_in_batch_enabled" + + /** + * Helper method for determining whether a shuffle reader should fetch the continuous blocks + * in batch. + */ + def canUseBatchFetch(startPartition: Int, endPartition: Int, context: TaskContext): Boolean = { + val fetchMultiPartitions = endPartition - startPartition > 1 + fetchMultiPartitions && + context.getLocalProperty(FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED_KEY) == "true" + } + /** * Helper method for determining whether a shuffle should use an optimized serialized shuffle * path or whether it should fall back to the original path that operates on deserialized objects. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 9c5b7f64e7abe..68ed3aa5b062f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -38,7 +38,7 @@ sealed abstract class BlockId { // convenience methods def asRDDId: Option[RDDBlockId] = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None def isRDD: Boolean = isInstanceOf[RDDBlockId] - def isShuffle: Boolean = isInstanceOf[ShuffleBlockId] + def isShuffle: Boolean = isInstanceOf[ShuffleBlockId] || isInstanceOf[ShuffleBlockBatchId] def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId] override def toString: String = name @@ -56,6 +56,18 @@ case class ShuffleBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends Bl override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } +// The batch id of continuous shuffle blocks of same mapId in range [startReduceId, endReduceId). +@DeveloperApi +case class ShuffleBlockBatchId( + shuffleId: Int, + mapId: Long, + startReduceId: Int, + endReduceId: Int) extends BlockId { + override def name: String = { + "shuffle_" + shuffleId + "_" + mapId + "_" + startReduceId + "_" + endReduceId + } +} + @DeveloperApi case class ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" @@ -104,6 +116,7 @@ class UnrecognizedBlockId(name: String) object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r + val SHUFFLE_BATCH = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r @@ -118,6 +131,8 @@ object BlockId { RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => ShuffleBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) + case SHUFFLE_BATCH(shuffleId, mapId, startReduceId, endReduceId) => + ShuffleBlockBatchId(shuffleId.toInt, mapId.toLong, startReduceId.toInt, endReduceId.toInt) case SHUFFLE_DATA(shuffleId, mapId, reduceId) => ShuffleDataBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a7dfc20d15ebc..c869a7078a1e9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -548,7 +548,7 @@ private[spark] class BlockManager( */ override def getBlockData(blockId: BlockId): ManagedBuffer = { if (blockId.isShuffle) { - shuffleManager.shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) + shuffleManager.shuffleBlockResolver.getBlockData(blockId) } else { getLocalBytes(blockId) match { case Some(blockData) => diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index dce5ebaebbae5..f8aa97267cf1b 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -62,6 +62,8 @@ import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} * @param maxReqSizeShuffleToMem max size (in bytes) of a request that can be shuffled to memory. * @param detectCorrupt whether to detect any corruption in fetched blocks. * @param shuffleMetrics used to report shuffle metrics. + * @param doBatchFetch fetch continuous shuffle blocks from same executor in batch if the server + * side supports. */ private[spark] final class ShuffleBlockFetcherIterator( @@ -76,7 +78,8 @@ final class ShuffleBlockFetcherIterator( maxReqSizeShuffleToMem: Long, detectCorrupt: Boolean, detectCorruptUseExtraMemory: Boolean, - shuffleMetrics: ShuffleReadMetricsReporter) + shuffleMetrics: ShuffleReadMetricsReporter, + doBatchFetch: Boolean) extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging { import ShuffleBlockFetcherIterator._ @@ -292,9 +295,10 @@ final class ShuffleBlockFetcherIterator( throw new BlockException(blockId, "Zero-sized blocks should be excluded.") case None => // do nothing. } - localBlocks ++= blockInfos.map(info => (info._1, info._3)) - localBlockBytes += blockInfos.map(_._2).sum - numBlocksToFetch += localBlocks.size + val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded( + blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)).to[ArrayBuffer]) + localBlocks ++= mergedBlockInfos.map(info => (info.blockId, info.mapIndex)) + localBlockBytes += mergedBlockInfos.map(_.size).sum } else { val iterator = blockInfos.iterator var curRequestSize = 0L @@ -308,23 +312,25 @@ final class ShuffleBlockFetcherIterator( throw new BlockException(blockId, "Zero-sized blocks should be excluded.") } else { curBlocks += FetchBlockInfo(blockId, size, mapIndex) - remoteBlocks += blockId - numBlocksToFetch += 1 curRequestSize += size } if (curRequestSize >= targetRequestSize || curBlocks.size >= maxBlocksInFlightPerAddress) { // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) + val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) + remoteBlocks ++= mergedBlocks.map(_.blockId) + remoteRequests += new FetchRequest(address, mergedBlocks) logDebug(s"Creating fetch request of $curRequestSize at $address " - + s"with ${curBlocks.size} blocks") + + s"with ${mergedBlocks.size} blocks") curBlocks = new ArrayBuffer[FetchBlockInfo] curRequestSize = 0 } } // Add in the final request if (curBlocks.nonEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) + val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) + remoteBlocks ++= mergedBlocks.map(_.blockId) + remoteRequests += new FetchRequest(address, mergedBlocks) } } } @@ -335,6 +341,51 @@ final class ShuffleBlockFetcherIterator( remoteRequests } + private[this] def mergeContinuousShuffleBlockIdsIfNeeded( + blocks: ArrayBuffer[FetchBlockInfo]): ArrayBuffer[FetchBlockInfo] = { + + def mergeFetchBlockInfo(toBeMerged: ArrayBuffer[FetchBlockInfo]): FetchBlockInfo = { + val startBlockId = toBeMerged.head.blockId.asInstanceOf[ShuffleBlockId] + FetchBlockInfo( + ShuffleBlockBatchId( + startBlockId.shuffleId, + startBlockId.mapId, + startBlockId.reduceId, + toBeMerged.last.blockId.asInstanceOf[ShuffleBlockId].reduceId + 1), + toBeMerged.map(_.size).sum, + toBeMerged.head.mapIndex) + } + + val result = if (doBatchFetch) { + var curBlocks = new ArrayBuffer[FetchBlockInfo] + val mergedBlockInfo = new ArrayBuffer[FetchBlockInfo] + val iter = blocks.iterator + + while (iter.hasNext) { + val info = iter.next() + val curBlockId = info.blockId.asInstanceOf[ShuffleBlockId] + if (curBlocks.isEmpty) { + curBlocks += info + } else { + if (curBlockId.mapId != curBlocks.head.blockId.asInstanceOf[ShuffleBlockId].mapId) { + mergedBlockInfo += mergeFetchBlockInfo(curBlocks) + curBlocks.clear() + } + curBlocks += info + } + } + if (curBlocks.nonEmpty) { + mergedBlockInfo += mergeFetchBlockInfo(curBlocks) + } + mergedBlockInfo + } else { + blocks + } + // update metrics + numBlocksToFetch += result.size + result + } + /** * Fetch the local blocks while we are fetching remote blocks. This is ok because * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we @@ -594,6 +645,8 @@ final class ShuffleBlockFetcherIterator( blockId match { case ShuffleBlockId(shufId, mapId, reduceId) => throw new FetchFailedException(address, shufId, mapId, mapIndex, reduceId, e) + case ShuffleBlockBatchId(shuffleId, mapId, startReduceId, _) => + throw new FetchFailedException(address, shuffleId, mapId, mapIndex, startReduceId, e) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block", e) diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 3f9536e224ded..67adf5fa5e185 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -138,7 +138,8 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext taskContext, metrics, serializerManager, - blockManager) + blockManager, + shouldBatchFetch = false) assert(shuffleReader.read().length === keyValuePairsPerMap * numMaps) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index 0f3767c4f8c84..ef7b13875540f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -64,6 +64,20 @@ class BlockIdSuite extends SparkFunSuite { assertSame(id, BlockId(id.toString)) } + test("shuffle batch") { + val id = ShuffleBlockBatchId(1, 2, 3, 4) + assertSame(id, ShuffleBlockBatchId(1, 2, 3, 4)) + assertDifferent(id, ShuffleBlockBatchId(2, 2, 3, 4)) + assert(id.name === "shuffle_1_2_3_4") + assert(id.asRDDId === None) + assert(id.shuffleId === 1) + assert(id.mapId === 2) + assert(id.startReduceId === 3) + assert(id.endReduceId === 4) + assert(id.isShuffle) + assertSame(id, BlockId(id.toString)) + } + test("shuffle data") { val id = ShuffleDataBlockId(4, 5, 6) assertSame(id, ShuffleDataBlockId(4, 5, 6)) diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 6f7469a9c2b40..85b1a865603ad 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -117,7 +117,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, true, false, - metrics) + metrics, + false) // 3 local blocks fetched in initialization verify(blockManager, times(3)).getBlockData(any()) @@ -148,6 +149,82 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any(), any()) } + test("fetch continuous blocks in batch successful 3 local reads + 2 remote reads") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure blockManager.getBlockData would return the merged block + val localBlocks = Seq[BlockId]( + ShuffleBlockId(0, 0, 0), + ShuffleBlockId(0, 0, 1), + ShuffleBlockId(0, 0, 2)) + val mergedLocalBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockBatchId(0, 0, 0, 3) -> createMockManagedBuffer()) + mergedLocalBlocks.foreach { case (blockId, buf) => + doReturn(buf).when(blockManager).getBlockData(meq(blockId)) + } + + // Make sure remote blocks would return the merged block + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val remoteBlocks = Seq[BlockId]( + ShuffleBlockId(0, 3, 0), + ShuffleBlockId(0, 3, 1)) + val mergedRemoteBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockBatchId(0, 3, 0, 2) -> createMockManagedBuffer()) + val transfer = createMockTransfer(mergedRemoteBlocks) + + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (localBmId, localBlocks.map(blockId => (blockId, 1L, 0))), + (remoteBmId, remoteBlocks.map(blockId => (blockId, 1L, 1))) + ).toIterator + + val taskContext = TaskContext.empty() + val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() + val iterator = new ShuffleBlockFetcherIterator( + taskContext, + transfer, + blockManager, + blocksByAddress, + (_, in) => in, + 48 * 1024 * 1024, + Int.MaxValue, + Int.MaxValue, + Int.MaxValue, + true, + false, + metrics, + true) + + // 3 local blocks batch fetched in initialization + verify(blockManager, times(1)).getBlockData(any()) + + for (i <- 0 until 2) { + assert(iterator.hasNext, s"iterator should have 2 elements but actually has $i elements") + val (blockId, inputStream) = iterator.next() + + // Make sure we release buffers when a wrapped input stream is closed. + val mockBuf = mergedLocalBlocks.getOrElse(blockId, mergedRemoteBlocks(blockId)) + // Note: ShuffleBlockFetcherIterator wraps input streams in a BufferReleasingInputStream + val wrappedInputStream = inputStream.asInstanceOf[BufferReleasingInputStream] + verify(mockBuf, times(0)).release() + val delegateAccess = PrivateMethod[InputStream]('delegate) + + verify(wrappedInputStream.invokePrivate(delegateAccess()), times(0)).close() + wrappedInputStream.close() + verify(mockBuf, times(1)).release() + verify(wrappedInputStream.invokePrivate(delegateAccess()), times(1)).close() + wrappedInputStream.close() // close should be idempotent + verify(mockBuf, times(1)).release() + verify(wrappedInputStream.invokePrivate(delegateAccess()), times(1)).close() + } + + // 2 remote blocks batch fetched + // (but from the same block manager so one call to fetchBlocks) + verify(blockManager, times(1)).getBlockData(any()) + verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any(), any()) + } + test("release current unexhausted buffer in case the task completes early") { val blockManager = mock(classOf[BlockManager]) val localBmId = BlockManagerId("test-client", "test-client", 1) @@ -195,7 +272,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, true, false, - taskContext.taskMetrics.createTempShuffleReadMetrics()) + taskContext.taskMetrics.createTempShuffleReadMetrics(), + false) verify(blocks(ShuffleBlockId(0, 0, 0)), times(0)).release() iterator.next()._2.close() // close() first block's input stream @@ -264,7 +342,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, true, false, - taskContext.taskMetrics.createTempShuffleReadMetrics()) + taskContext.taskMetrics.createTempShuffleReadMetrics(), + false) // Continue only after the mock calls onBlockFetchFailure sem.acquire() @@ -353,7 +432,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, true, true, - taskContext.taskMetrics.createTempShuffleReadMetrics()) + taskContext.taskMetrics.createTempShuffleReadMetrics(), + false) // Continue only after the mock calls onBlockFetchFailure sem.acquire() @@ -423,7 +503,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, true, true, - taskContext.taskMetrics.createTempShuffleReadMetrics()) + taskContext.taskMetrics.createTempShuffleReadMetrics(), + false) // We'll get back the block which has corruption after maxBytesInFlight/3 because the other // block will detect corruption on first fetch, and then get added to the queue again for @@ -487,7 +568,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, true, true, - taskContext.taskMetrics.createTempShuffleReadMetrics()) + taskContext.taskMetrics.createTempShuffleReadMetrics(), + false) val (id, st) = iterator.next() // Check that the test setup is correct -- make sure we have a concatenated stream. assert (st.asInstanceOf[BufferReleasingInputStream].delegate.isInstanceOf[SequenceInputStream]) @@ -549,7 +631,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, true, false, - taskContext.taskMetrics.createTempShuffleReadMetrics()) + taskContext.taskMetrics.createTempShuffleReadMetrics(), + false) // Continue only after the mock calls onBlockFetchFailure sem.acquire() @@ -610,7 +693,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT maxReqSizeShuffleToMem = 200, detectCorrupt = true, false, - taskContext.taskMetrics.createTempShuffleReadMetrics()) + taskContext.taskMetrics.createTempShuffleReadMetrics(), + false) } val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( @@ -658,7 +742,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, true, false, - taskContext.taskMetrics.createTempShuffleReadMetrics()) + taskContext.taskMetrics.createTempShuffleReadMetrics(), + false) // All blocks fetched return zero length and should trigger a receive-side error: val e = intercept[FetchFailedException] { iterator.next() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 42e3beca2ad59..e2c1308cdc60f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -355,6 +355,16 @@ object SQLConf { .bytesConf(ByteUnit.BYTE) .createWithDefault(64 * 1024 * 1024) + + val FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED = + buildConf("spark.sql.adaptive.fetchShuffleBlocksInBatch.enabled") + .doc("Whether to fetch the continuous shuffle blocks in batch. Instead of fetching blocks " + + "one by one, fetching continuous shuffle blocks for the same map task in batch can " + + "reduce IO and improve performance. Note, this feature also depends on a relocatable " + + "serializer and the concatenation support codec in use.") + .booleanConf + .createWithDefault(true) + val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") .doc("When true, enable adaptive query execution.") .booleanConf @@ -2141,6 +2151,9 @@ class SQLConf extends Serializable with Logging { def targetPostShuffleInputSize: Long = getConf(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) + def fetchShuffleBlocksInBatchEnabled: Boolean = + getConf(FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED) + def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) def nonEmptyPartitionRatioForBroadcastJoin: Double = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index f5b0e761161de..4c19f95796d04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -21,8 +21,10 @@ import java.util.Arrays import org.apache.spark._ import org.apache.spark.rdd.RDD +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} +import org.apache.spark.sql.internal.SQLConf /** * The [[Partition]] used by [[ShuffledRowRDD]]. A post-shuffle partition @@ -117,6 +119,11 @@ class ShuffledRowRDD( specifiedPartitionStartIndices: Option[Array[Int]] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { + if (SQLConf.get.fetchShuffleBlocksInBatchEnabled) { + dependency.rdd.context.setLocalProperty( + SortShuffleManager.FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED_KEY, "true") + } + private[this] val numPreShufflePartitions = dependency.partitioner.numPartitions private[this] val partitionStartIndices: Array[Int] = specifiedPartitionStartIndices match { From 4b902d3b455e9769e8808fe795eb76871fa5134a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 17 Oct 2019 20:22:38 +0800 Subject: [PATCH 0347/1387] [SPARK-29491][SQL] Add bit_count function support MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? BIT_COUNT(N) - Returns the number of bits that are set in the argument N as an unsigned 64-bit integer, or NULL if the argument is NULL ### Why are the changes needed? Supported by MySQL,Microsoft SQL Server ,etc. ### Does this PR introduce any user-facing change? add a built-in function ### How was this patch tested? add uts Closes #26139 from yaooqinn/SPARK-29491. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/bitwiseExpressions.scala | 32 ++++ .../resources/sql-tests/inputs/bitwise.sql | 39 +++++ .../sql-tests/results/bitwise.sql.out | 164 ++++++++++++++++++ 4 files changed, 236 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/bitwise.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/bitwise.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 8cf0928c2afd9..addeca507c0f7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -524,6 +524,7 @@ object FunctionRegistry { expression[BitwiseNot]("~"), expression[BitwiseOr]("|"), expression[BitwiseXor]("^"), + expression[BitwiseCount]("bit_count"), // json expression[StructsToJson]("to_json"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala index c766bd8e56bb6..56f170cd77073 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala @@ -154,3 +154,35 @@ case class BitwiseNot(child: Expression) extends UnaryExpression with ExpectsInp override def sql: String = s"~${child.sql}" } + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the number of bits that are set in the argument expr as an" + + " unsigned 64-bit integer, or NULL if the argument is NULL.", + examples = """ + Examples: + > SELECT _FUNC_(0); + 0 + """) +case class BitwiseCount(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(IntegralType, BooleanType)) + + override def dataType: DataType = IntegerType + + override def toString: String = s"bit_count($child)" + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = child.dataType match { + case BooleanType => defineCodeGen(ctx, ev, c => s"if ($c) 1 else 0") + case _ => defineCodeGen(ctx, ev, c => s"java.lang.Long.bitCount($c)") + } + + protected override def nullSafeEval(input: Any): Any = child.dataType match { + case BooleanType => if (input.asInstanceOf[Boolean]) 1 else 0 + case ByteType => java.lang.Long.bitCount(input.asInstanceOf[Byte]) + case ShortType => java.lang.Long.bitCount(input.asInstanceOf[Short]) + case IntegerType => java.lang.Long.bitCount(input.asInstanceOf[Int]) + case LongType => java.lang.Long.bitCount(input.asInstanceOf[Long]) + } + + override def sql: String = s"bit_count(${child.sql})" +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql b/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql new file mode 100644 index 0000000000000..993eecf0f89b6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql @@ -0,0 +1,39 @@ +-- test cases for bitwise functions + +-- null +select bit_count(null); + +-- boolean +select bit_count(true); +select bit_count(false); + +-- byte/tinyint +select bit_count(cast(1 as tinyint)); +select bit_count(cast(2 as tinyint)); +select bit_count(cast(3 as tinyint)); + +-- short/smallint +select bit_count(1S); +select bit_count(2S); +select bit_count(3S); + +-- int +select bit_count(1); +select bit_count(2); +select bit_count(3); + +-- long/bigint +select bit_count(1L); +select bit_count(2L); +select bit_count(3L); + +-- negative num +select bit_count(-1L); + +-- edge value +select bit_count(9223372036854775807L); +select bit_count(-9223372036854775808L); + +-- other illegal arguments +select bit_count("bit count"); +select bit_count('a'); diff --git a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out new file mode 100644 index 0000000000000..7cbd26e87bd2b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out @@ -0,0 +1,164 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 20 + + +-- !query 0 +select bit_count(null) +-- !query 0 schema +struct +-- !query 0 output +NULL + + +-- !query 1 +select bit_count(true) +-- !query 1 schema +struct +-- !query 1 output +1 + + +-- !query 2 +select bit_count(false) +-- !query 2 schema +struct +-- !query 2 output +0 + + +-- !query 3 +select bit_count(cast(1 as tinyint)) +-- !query 3 schema +struct +-- !query 3 output +1 + + +-- !query 4 +select bit_count(cast(2 as tinyint)) +-- !query 4 schema +struct +-- !query 4 output +1 + + +-- !query 5 +select bit_count(cast(3 as tinyint)) +-- !query 5 schema +struct +-- !query 5 output +2 + + +-- !query 6 +select bit_count(1S) +-- !query 6 schema +struct +-- !query 6 output +1 + + +-- !query 7 +select bit_count(2S) +-- !query 7 schema +struct +-- !query 7 output +1 + + +-- !query 8 +select bit_count(3S) +-- !query 8 schema +struct +-- !query 8 output +2 + + +-- !query 9 +select bit_count(1) +-- !query 9 schema +struct +-- !query 9 output +1 + + +-- !query 10 +select bit_count(2) +-- !query 10 schema +struct +-- !query 10 output +1 + + +-- !query 11 +select bit_count(3) +-- !query 11 schema +struct +-- !query 11 output +2 + + +-- !query 12 +select bit_count(1L) +-- !query 12 schema +struct +-- !query 12 output +1 + + +-- !query 13 +select bit_count(2L) +-- !query 13 schema +struct +-- !query 13 output +1 + + +-- !query 14 +select bit_count(3L) +-- !query 14 schema +struct +-- !query 14 output +2 + + +-- !query 15 +select bit_count(-1L) +-- !query 15 schema +struct +-- !query 15 output +64 + + +-- !query 16 +select bit_count(9223372036854775807L) +-- !query 16 schema +struct +-- !query 16 output +63 + + +-- !query 17 +select bit_count(-9223372036854775808L) +-- !query 17 schema +struct +-- !query 17 output +1 + + +-- !query 18 +select bit_count("bit count") +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +cannot resolve 'bit_count('bit count')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''bit count'' is of string type.; line 1 pos 7 + + +-- !query 19 +select bit_count('a') +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +cannot resolve 'bit_count('a')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''a'' is of string type.; line 1 pos 7 From dc0bc7a6eb87142aaa1e8751d934f8c77a0e1208 Mon Sep 17 00:00:00 2001 From: Jiajia Li Date: Thu, 17 Oct 2019 07:22:01 -0700 Subject: [PATCH 0348/1387] [MINOR][DOCS] Fix some typos ### What changes were proposed in this pull request? This PR proposes a few typos: 1. Sparks => Spark's 2. parallize => parallelize 3. doesnt => doesn't Closes #26140 from plusplusjiajia/fix-typos. Authored-by: Jiajia Li Signed-off-by: Sean Owen --- docs/running-on-kubernetes.md | 2 +- .../org/apache/spark/examples/AccumulatorMetricsTest.scala | 2 +- .../sql/catalyst/expressions/UnsafeRowConverterSuite.scala | 2 +- .../org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala | 2 +- .../org/apache/spark/sql/hive/client/IsolatedClientLoader.scala | 2 +- .../test/queries/clientpositive/infer_bucket_sort_list_bucket.q | 2 +- .../org/apache/spark/sql/hive/execution/HiveDDLSuite.scala | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 4ef738ed9ef6e..a035db2fe152d 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -324,7 +324,7 @@ If no volume is set as local storage, Spark uses temporary scratch space to spil `emptyDir` volumes use the nodes backing storage for ephemeral storage by default, this behaviour may not be appropriate for some compute environments. For example if you have diskless nodes with remote storage mounted over a network, having lots of executors doing IO to this remote storage may actually degrade performance. -In this case it may be desirable to set `spark.kubernetes.local.dirs.tmpfs=true` in your configuration which will cause the `emptyDir` volumes to be configured as `tmpfs` i.e. RAM backed volumes. When configured like this Sparks local storage usage will count towards your pods memory usage therefore you may wish to increase your memory requests by increasing the value of `spark.kubernetes.memoryOverheadFactor` as appropriate. +In this case it may be desirable to set `spark.kubernetes.local.dirs.tmpfs=true` in your configuration which will cause the `emptyDir` volumes to be configured as `tmpfs` i.e. RAM backed volumes. When configured like this Spark's local storage usage will count towards your pods memory usage therefore you may wish to increase your memory requests by increasing the value of `spark.kubernetes.memoryOverheadFactor` as appropriate. ## Introspection and Debugging diff --git a/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala b/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala index 36da10568989d..a0e8d63133b74 100644 --- a/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.SparkSession * This example shows how to register accumulators against the accumulator source. * A simple RDD is created, and during the map, the accumulators are incremented. * - * The only argument, numElem, sets the number elements in the collection to parallize. + * The only argument, numElem, sets the number elements in the collection to parallelize. * * The result is output to stdout in the driver with the values of the accumulators. * For the long accumulator, it should equal numElem the double accumulator should be diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 69523fa81bc65..323a3a901689f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -207,7 +207,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB rowWithNoNullColumns.getDecimal(11, 38, 18)) for (i <- fieldTypes.indices) { - // Cann't call setNullAt() on DecimalType + // Can't call setNullAt() on DecimalType if (i == 11) { setToNullAfterCreation.setDecimal(11, null, 38) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala index 241001a857c8f..c6e7f3978469d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala @@ -133,7 +133,7 @@ class SparkPlanGraphNodeWrapper( val cluster: SparkPlanGraphClusterWrapper) { def toSparkPlanGraphNode(): SparkPlanGraphNode = { - assert(node == null ^ cluster == null, "One and only of of nore or cluster must be set.") + assert(node == null ^ cluster == null, "Exactly one of node, cluster values to be set.") if (node != null) node else cluster.toSparkPlanGraphCluster() } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index be50cb0ded5b5..5da7b70cfc7aa 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -162,7 +162,7 @@ private[hive] object IsolatedClientLoader extends Logging { * @param execJars A collection of jar files that must include hive and hadoop. * @param config A set of options that will be added to the HiveConf of the constructed client. * @param isolationOn When true, custom versions of barrier classes will be constructed. Must be - * true unless loading the version of hive that is on Sparks classloader. + * true unless loading the version of hive that is on Spark's classloader. * @param sharesHadoopClasses When true, we will share Hadoop classes between Spark and * @param baseClassLoader The spark classloader that is used to load shared classes. */ diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q index 7d3c0dc7d5859..c0a7a368a0768 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q @@ -20,7 +20,7 @@ SELECT key, count(*) FROM src GROUP BY key; DESC FORMATTED list_bucketing_table PARTITION (part = '1'); --- create a table skewed on a key which doesnt exist in the data +-- create a table skewed on a key which doesn't exist in the data CREATE TABLE list_bucketing_table2 (key STRING, value STRING) PARTITIONED BY (part STRING) SKEWED BY (key) ON ("abc") 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 6d12310714164..f34c96efdd2ee 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 @@ -2434,7 +2434,7 @@ class HiveDDLSuite "CREATE TABLE IF NOT EXISTS t1 (c1_int INT, c2_string STRING, c3_float FLOAT)") val desc = sql("DESC FORMATTED t1").filter($"col_name".startsWith("Last Access")) .select("data_type") - // check if the last access time doesnt have the default date of year + // check if the last access time doesn't have the default date of year // 1970 as its a wrong access time assert((desc.first.toString.contains("UNKNOWN"))) } From f800fa383131559c4e841bf062c9775d09190935 Mon Sep 17 00:00:00 2001 From: maruilei Date: Thu, 17 Oct 2019 07:24:13 -0700 Subject: [PATCH 0349/1387] [SPARK-29436][K8S] Support executor for selecting scheduler through scheduler name in the case of k8s multi-scheduler scenario ### What changes were proposed in this pull request? Support executor for selecting scheduler through scheduler name in the case of k8s multi-scheduler scenario. ### Why are the changes needed? If there is no such function, spark can not support the case of k8s multi-scheduler scenario. ### Does this PR introduce any user-facing change? Yes, users can add scheduler name through configuration. ### How was this patch tested? Manually tested with spark + k8s cluster Closes #26088 from merrily01/SPARK-29436. Authored-by: maruilei Signed-off-by: Sean Owen --- .../src/main/scala/org/apache/spark/deploy/k8s/Config.scala | 6 ++++++ .../deploy/k8s/features/BasicExecutorFeatureStep.scala | 3 +++ 2 files changed, 9 insertions(+) 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 54afe92e81567..819ea4a7b13d0 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 @@ -142,6 +142,12 @@ private[spark] object Config extends Logging { .stringConf .createOptional + val KUBERNETES_EXECUTOR_SCHEDULER_NAME = + ConfigBuilder("spark.kubernetes.executor.scheduler.name") + .doc("Specify the scheduler name for each executor pod") + .stringConf + .createOptional + val KUBERNETES_EXECUTOR_REQUEST_CORES = ConfigBuilder("spark.kubernetes.executor.request.cores") .doc("Specify the cpu request for each executor pod") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index d6487556a371e..d88bd5858bc94 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -216,6 +216,9 @@ private[spark] class BasicExecutorFeatureStep( .endSpec() .build() + kubernetesConf.get(KUBERNETES_EXECUTOR_SCHEDULER_NAME) + .foreach(executorPod.getSpec.setSchedulerName) + SparkPod(executorPod, containerWithLimitCores) } } From 2f0a38cb50e3e8b4b72219c7b2b8b15d51f6b931 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 17 Oct 2019 13:14:32 -0500 Subject: [PATCH 0350/1387] [SPARK-29398][CORE] Support dedicated thread pools for RPC endpoints The current RPC backend in Spark supports single- and multi-threaded message delivery to endpoints, but they all share the same underlying thread pool. So an RPC endpoint that blocks a dispatcher thread can negatively affect other endpoints. This can be more pronounced with configurations that limit the number of RPC dispatch threads based on configuration and / or running environment. And exposing the RPC layer to other code (for example with something like SPARK-29396) could make it easy to affect normal Spark operation with a badly written RPC handler. This change adds a new RPC endpoint type that tells the RPC env to create dedicated dispatch threads, so that those effects are minimised. Other endpoints will still need CPU to process their messages, but they won't be able to actively block the dispatch thread of these isolated endpoints. As part of the change, I've changed the most important Spark endpoints (the driver, executor and block manager endpoints) to be isolated from others. This means a couple of extra threads are created on the driver and executor for these endpoints. Tested with existing unit tests, which hammer the RPC system extensively, and also by running applications on a cluster (with a prototype of SPARK-29396). Closes #26059 from vanzin/SPARK-29398. Authored-by: Marcelo Vanzin Signed-off-by: Imran Rashid --- .../CoarseGrainedExecutorBackend.scala | 2 +- .../org/apache/spark/rpc/RpcEndpoint.scala | 16 ++ .../apache/spark/rpc/netty/Dispatcher.scala | 130 ++++-------- .../org/apache/spark/rpc/netty/Inbox.scala | 6 +- .../apache/spark/rpc/netty/MessageLoop.scala | 194 ++++++++++++++++++ .../CoarseGrainedSchedulerBackend.scala | 2 +- .../storage/BlockManagerMasterEndpoint.scala | 4 +- .../storage/BlockManagerSlaveEndpoint.scala | 4 +- .../org/apache/spark/rpc/RpcEnvSuite.scala | 35 +++- .../apache/spark/rpc/netty/InboxSuite.scala | 23 +-- 10 files changed, 296 insertions(+), 120 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala 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 fbf2dc73ea075..b4bca1e9401e2 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -51,7 +51,7 @@ private[spark] class CoarseGrainedExecutorBackend( userClassPath: Seq[URL], env: SparkEnv, resourcesFileOpt: Option[String]) - extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging { + extends IsolatedRpcEndpoint with ExecutorBackend with Logging { private implicit val formats = DefaultFormats diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala index 97eed540b8f59..4728759e7fb0d 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala @@ -146,3 +146,19 @@ private[spark] trait RpcEndpoint { * [[ThreadSafeRpcEndpoint]] for different messages. */ private[spark] trait ThreadSafeRpcEndpoint extends RpcEndpoint + +/** + * An endpoint that uses a dedicated thread pool for delivering messages. + */ +private[spark] trait IsolatedRpcEndpoint extends RpcEndpoint { + + /** + * How many threads to use for delivering messages. By default, use a single thread. + * + * Note that requesting more than one thread means that the endpoint should be able to handle + * messages arriving from many threads at once, and all the things that entails (including + * messages being delivered to the endpoint out of order). + */ + def threadCount(): Int = 1 + +} diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index 2f923d7902b05..27c943da88105 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -17,20 +17,16 @@ package org.apache.spark.rpc.netty -import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, CountDownLatch} import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.concurrent.Promise -import scala.util.control.NonFatal -import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.SparkException import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.EXECUTOR_ID -import org.apache.spark.internal.config.Network.RPC_NETTY_DISPATCHER_NUM_THREADS import org.apache.spark.network.client.RpcResponseCallback import org.apache.spark.rpc._ -import org.apache.spark.util.ThreadUtils /** * A message dispatcher, responsible for routing RPC messages to the appropriate endpoint(s). @@ -40,20 +36,23 @@ import org.apache.spark.util.ThreadUtils */ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) extends Logging { - private class EndpointData( - val name: String, - val endpoint: RpcEndpoint, - val ref: NettyRpcEndpointRef) { - val inbox = new Inbox(ref, endpoint) - } - - private val endpoints: ConcurrentMap[String, EndpointData] = - new ConcurrentHashMap[String, EndpointData] + private val endpoints: ConcurrentMap[String, MessageLoop] = + new ConcurrentHashMap[String, MessageLoop] private val endpointRefs: ConcurrentMap[RpcEndpoint, RpcEndpointRef] = new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef] - // Track the receivers whose inboxes may contain messages. - private val receivers = new LinkedBlockingQueue[EndpointData] + private val shutdownLatch = new CountDownLatch(1) + private lazy val sharedLoop = new SharedMessageLoop(nettyEnv.conf, this, numUsableCores) + + private def getMessageLoop(name: String, endpoint: RpcEndpoint): MessageLoop = { + endpoint match { + case e: IsolatedRpcEndpoint => + new DedicatedMessageLoop(name, e, this) + case _ => + sharedLoop.register(name, endpoint) + sharedLoop + } + } /** * True if the dispatcher has been stopped. Once stopped, all messages posted will be bounced @@ -69,13 +68,11 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte if (stopped) { throw new IllegalStateException("RpcEnv has been stopped") } - if (endpoints.putIfAbsent(name, new EndpointData(name, endpoint, endpointRef)) != null) { + if (endpoints.putIfAbsent(name, getMessageLoop(name, endpoint)) != null) { throw new IllegalArgumentException(s"There is already an RpcEndpoint called $name") } - val data = endpoints.get(name) - endpointRefs.put(data.endpoint, data.ref) - receivers.offer(data) // for the OnStart message } + endpointRefs.put(endpoint, endpointRef) endpointRef } @@ -85,10 +82,9 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte // Should be idempotent private def unregisterRpcEndpoint(name: String): Unit = { - val data = endpoints.remove(name) - if (data != null) { - data.inbox.stop() - receivers.offer(data) // for the OnStop message + val loop = endpoints.remove(name) + if (loop != null) { + loop.unregister(name) } // Don't clean `endpointRefs` here because it's possible that some messages are being processed // now and they can use `getRpcEndpointRef`. So `endpointRefs` will be cleaned in Inbox via @@ -155,14 +151,13 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte message: InboxMessage, callbackIfStopped: (Exception) => Unit): Unit = { val error = synchronized { - val data = endpoints.get(endpointName) + val loop = endpoints.get(endpointName) if (stopped) { Some(new RpcEnvStoppedException()) - } else if (data == null) { + } else if (loop == null) { Some(new SparkException(s"Could not find $endpointName.")) } else { - data.inbox.post(message) - receivers.offer(data) + loop.post(endpointName, message) None } } @@ -177,15 +172,23 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte } stopped = true } - // Stop all endpoints. This will queue all endpoints for processing by the message loops. - endpoints.keySet().asScala.foreach(unregisterRpcEndpoint) - // Enqueue a message that tells the message loops to stop. - receivers.offer(PoisonPill) - threadpool.shutdown() + var stopSharedLoop = false + endpoints.asScala.foreach { case (name, loop) => + unregisterRpcEndpoint(name) + if (!loop.isInstanceOf[SharedMessageLoop]) { + loop.stop() + } else { + stopSharedLoop = true + } + } + if (stopSharedLoop) { + sharedLoop.stop() + } + shutdownLatch.countDown() } def awaitTermination(): Unit = { - threadpool.awaitTermination(Long.MaxValue, TimeUnit.MILLISECONDS) + shutdownLatch.await() } /** @@ -194,61 +197,4 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte def verify(name: String): Boolean = { endpoints.containsKey(name) } - - private def getNumOfThreads(conf: SparkConf): Int = { - val availableCores = - if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() - - val modNumThreads = conf.get(RPC_NETTY_DISPATCHER_NUM_THREADS) - .getOrElse(math.max(2, availableCores)) - - conf.get(EXECUTOR_ID).map { id => - val role = if (id == SparkContext.DRIVER_IDENTIFIER) "driver" else "executor" - conf.getInt(s"spark.$role.rpc.netty.dispatcher.numThreads", modNumThreads) - }.getOrElse(modNumThreads) - } - - /** Thread pool used for dispatching messages. */ - private val threadpool: ThreadPoolExecutor = { - val numThreads = getNumOfThreads(nettyEnv.conf) - val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "dispatcher-event-loop") - for (i <- 0 until numThreads) { - pool.execute(new MessageLoop) - } - pool - } - - /** Message loop used for dispatching messages. */ - private class MessageLoop extends Runnable { - override def run(): Unit = { - try { - while (true) { - try { - val data = receivers.take() - if (data == PoisonPill) { - // Put PoisonPill back so that other MessageLoops can see it. - receivers.offer(PoisonPill) - return - } - data.inbox.process(Dispatcher.this) - } catch { - case NonFatal(e) => logError(e.getMessage, e) - } - } - } catch { - case _: InterruptedException => // exit - case t: Throwable => - try { - // Re-submit a MessageLoop so that Dispatcher will still work if - // UncaughtExceptionHandler decides to not kill JVM. - threadpool.execute(new MessageLoop) - } finally { - throw t - } - } - } - } - - /** A poison endpoint that indicates MessageLoop should exit its message loop. */ - private val PoisonPill = new EndpointData(null, null, null) } diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index 44d2622a42f58..2ed03f7430c32 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -54,9 +54,7 @@ private[netty] case class RemoteProcessConnectionError(cause: Throwable, remoteA /** * An inbox that stores messages for an [[RpcEndpoint]] and posts messages to it thread-safely. */ -private[netty] class Inbox( - val endpointRef: NettyRpcEndpointRef, - val endpoint: RpcEndpoint) +private[netty] class Inbox(val endpointName: String, val endpoint: RpcEndpoint) extends Logging { inbox => // Give this an alias so we can use it more clearly in closures. @@ -195,7 +193,7 @@ private[netty] class Inbox( * Exposed for testing. */ protected def onDrop(message: InboxMessage): Unit = { - logWarning(s"Drop $message because $endpointRef is stopped") + logWarning(s"Drop $message because endpoint $endpointName is stopped") } /** diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala b/core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala new file mode 100644 index 0000000000000..c985c72f2adce --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/netty/MessageLoop.scala @@ -0,0 +1,194 @@ +/* + * 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.rpc.netty + +import java.util.concurrent._ + +import scala.util.control.NonFatal + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.EXECUTOR_ID +import org.apache.spark.internal.config.Network._ +import org.apache.spark.rpc.{IsolatedRpcEndpoint, RpcEndpoint} +import org.apache.spark.util.ThreadUtils + +/** + * A message loop used by [[Dispatcher]] to deliver messages to endpoints. + */ +private sealed abstract class MessageLoop(dispatcher: Dispatcher) extends Logging { + + // List of inboxes with pending messages, to be processed by the message loop. + private val active = new LinkedBlockingQueue[Inbox]() + + // Message loop task; should be run in all threads of the message loop's pool. + protected val receiveLoopRunnable = new Runnable() { + override def run(): Unit = receiveLoop() + } + + protected val threadpool: ExecutorService + + private var stopped = false + + def post(endpointName: String, message: InboxMessage): Unit + + def unregister(name: String): Unit + + def stop(): Unit = { + synchronized { + if (!stopped) { + setActive(MessageLoop.PoisonPill) + threadpool.shutdown() + stopped = true + } + } + threadpool.awaitTermination(Long.MaxValue, TimeUnit.MILLISECONDS) + } + + protected final def setActive(inbox: Inbox): Unit = active.offer(inbox) + + private def receiveLoop(): Unit = { + try { + while (true) { + try { + val inbox = active.take() + if (inbox == MessageLoop.PoisonPill) { + // Put PoisonPill back so that other threads can see it. + setActive(MessageLoop.PoisonPill) + return + } + inbox.process(dispatcher) + } catch { + case NonFatal(e) => logError(e.getMessage, e) + } + } + } catch { + case _: InterruptedException => // exit + case t: Throwable => + try { + // Re-submit a receive task so that message delivery will still work if + // UncaughtExceptionHandler decides to not kill JVM. + threadpool.execute(receiveLoopRunnable) + } finally { + throw t + } + } + } +} + +private object MessageLoop { + /** A poison inbox that indicates the message loop should stop processing messages. */ + val PoisonPill = new Inbox(null, null) +} + +/** + * A message loop that serves multiple RPC endpoints, using a shared thread pool. + */ +private class SharedMessageLoop( + conf: SparkConf, + dispatcher: Dispatcher, + numUsableCores: Int) + extends MessageLoop(dispatcher) { + + private val endpoints = new ConcurrentHashMap[String, Inbox]() + + private def getNumOfThreads(conf: SparkConf): Int = { + val availableCores = + if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() + + val modNumThreads = conf.get(RPC_NETTY_DISPATCHER_NUM_THREADS) + .getOrElse(math.max(2, availableCores)) + + conf.get(EXECUTOR_ID).map { id => + val role = if (id == SparkContext.DRIVER_IDENTIFIER) "driver" else "executor" + conf.getInt(s"spark.$role.rpc.netty.dispatcher.numThreads", modNumThreads) + }.getOrElse(modNumThreads) + } + + /** Thread pool used for dispatching messages. */ + override protected val threadpool: ThreadPoolExecutor = { + val numThreads = getNumOfThreads(conf) + val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "dispatcher-event-loop") + for (i <- 0 until numThreads) { + pool.execute(receiveLoopRunnable) + } + pool + } + + override def post(endpointName: String, message: InboxMessage): Unit = { + val inbox = endpoints.get(endpointName) + inbox.post(message) + setActive(inbox) + } + + override def unregister(name: String): Unit = { + val inbox = endpoints.remove(name) + if (inbox != null) { + inbox.stop() + // Mark active to handle the OnStop message. + setActive(inbox) + } + } + + def register(name: String, endpoint: RpcEndpoint): Unit = { + val inbox = new Inbox(name, endpoint) + endpoints.put(name, inbox) + // Mark active to handle the OnStart message. + setActive(inbox) + } +} + +/** + * A message loop that is dedicated to a single RPC endpoint. + */ +private class DedicatedMessageLoop( + name: String, + endpoint: IsolatedRpcEndpoint, + dispatcher: Dispatcher) + extends MessageLoop(dispatcher) { + + private val inbox = new Inbox(name, endpoint) + + override protected val threadpool = if (endpoint.threadCount() > 1) { + ThreadUtils.newDaemonCachedThreadPool(s"dispatcher-$name", endpoint.threadCount()) + } else { + ThreadUtils.newDaemonSingleThreadExecutor(s"dispatcher-$name") + } + + (1 to endpoint.threadCount()).foreach { _ => + threadpool.submit(receiveLoopRunnable) + } + + // Mark active to handle the OnStart message. + setActive(inbox) + + override def post(endpointName: String, message: InboxMessage): Unit = { + require(endpointName == name) + inbox.post(message) + setActive(inbox) + } + + override def unregister(endpointName: String): Unit = synchronized { + require(endpointName == name) + inbox.stop() + // Mark active to handle the OnStop message. + setActive(inbox) + setActive(MessageLoop.PoisonPill) + threadpool.shutdown() + } +} 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 4958389ae4257..6e990d1335897 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 @@ -111,7 +111,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp private val reviveThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") - class DriverEndpoint extends ThreadSafeRpcEndpoint with Logging { + class DriverEndpoint extends IsolatedRpcEndpoint with Logging { override val rpcEnv: RpcEnv = CoarseGrainedSchedulerBackend.this.rpcEnv diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index faf6f713c838f..02d0e1a834909 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -30,7 +30,7 @@ import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.{config, Logging} import org.apache.spark.network.shuffle.ExternalBlockStoreClient -import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} +import org.apache.spark.rpc.{IsolatedRpcEndpoint, RpcCallContext, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils} @@ -46,7 +46,7 @@ class BlockManagerMasterEndpoint( conf: SparkConf, listenerBus: LiveListenerBus, externalBlockStoreClient: Option[ExternalBlockStoreClient]) - extends ThreadSafeRpcEndpoint with Logging { + extends IsolatedRpcEndpoint with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala index f90595ab924b4..29e21142ce449 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala @@ -21,7 +21,7 @@ import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{MapOutputTracker, SparkEnv} import org.apache.spark.internal.Logging -import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} +import org.apache.spark.rpc.{IsolatedRpcEndpoint, RpcCallContext, RpcEnv} import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{ThreadUtils, Utils} @@ -34,7 +34,7 @@ class BlockManagerSlaveEndpoint( override val rpcEnv: RpcEnv, blockManager: BlockManager, mapOutputTracker: MapOutputTracker) - extends ThreadSafeRpcEndpoint with Logging { + extends IsolatedRpcEndpoint with Logging { private val asyncThreadPool = ThreadUtils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool", 100) diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 5929fbf85a1f4..c10f2c244e133 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -36,7 +36,6 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SecurityManager, SparkConf, SparkEnv, SparkException, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.config._ -import org.apache.spark.internal.config.Network import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -954,6 +953,40 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { verify(endpoint, never()).onDisconnected(any()) verify(endpoint, never()).onNetworkError(any(), any()) } + + test("isolated endpoints") { + val latch = new CountDownLatch(1) + val singleThreadedEnv = createRpcEnv( + new SparkConf().set(Network.RPC_NETTY_DISPATCHER_NUM_THREADS, 1), "singleThread", 0) + try { + val blockingEndpoint = singleThreadedEnv.setupEndpoint("blocking", new IsolatedRpcEndpoint { + override val rpcEnv: RpcEnv = singleThreadedEnv + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => + latch.await() + context.reply(m) + } + }) + + val nonBlockingEndpoint = singleThreadedEnv.setupEndpoint("non-blocking", new RpcEndpoint { + override val rpcEnv: RpcEnv = singleThreadedEnv + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply(m) + } + }) + + val to = new RpcTimeout(5.seconds, "test-timeout") + val blockingFuture = blockingEndpoint.ask[String]("hi", to) + assert(nonBlockingEndpoint.askSync[String]("hello", to) === "hello") + latch.countDown() + assert(ThreadUtils.awaitResult(blockingFuture, 5.seconds) === "hi") + } finally { + latch.countDown() + singleThreadedEnv.shutdown() + } + } } class UnserializableClass diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala index e5539566e4b6f..c74c728b3e3f3 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala @@ -29,12 +29,9 @@ class InboxSuite extends SparkFunSuite { test("post") { val endpoint = new TestRpcEndpoint - val endpointRef = mock(classOf[NettyRpcEndpointRef]) - when(endpointRef.name).thenReturn("hello") - val dispatcher = mock(classOf[Dispatcher]) - val inbox = new Inbox(endpointRef, endpoint) + val inbox = new Inbox("name", endpoint) val message = OneWayMessage(null, "hi") inbox.post(message) inbox.process(dispatcher) @@ -51,10 +48,9 @@ class InboxSuite extends SparkFunSuite { test("post: with reply") { val endpoint = new TestRpcEndpoint - val endpointRef = mock(classOf[NettyRpcEndpointRef]) val dispatcher = mock(classOf[Dispatcher]) - val inbox = new Inbox(endpointRef, endpoint) + val inbox = new Inbox("name", endpoint) val message = RpcMessage(null, "hi", null) inbox.post(message) inbox.process(dispatcher) @@ -65,13 +61,10 @@ class InboxSuite extends SparkFunSuite { test("post: multiple threads") { val endpoint = new TestRpcEndpoint - val endpointRef = mock(classOf[NettyRpcEndpointRef]) - when(endpointRef.name).thenReturn("hello") - val dispatcher = mock(classOf[Dispatcher]) val numDroppedMessages = new AtomicInteger(0) - val inbox = new Inbox(endpointRef, endpoint) { + val inbox = new Inbox("name", endpoint) { override def onDrop(message: InboxMessage): Unit = { numDroppedMessages.incrementAndGet() } @@ -107,12 +100,10 @@ class InboxSuite extends SparkFunSuite { test("post: Associated") { val endpoint = new TestRpcEndpoint - val endpointRef = mock(classOf[NettyRpcEndpointRef]) val dispatcher = mock(classOf[Dispatcher]) - val remoteAddress = RpcAddress("localhost", 11111) - val inbox = new Inbox(endpointRef, endpoint) + val inbox = new Inbox("name", endpoint) inbox.post(RemoteProcessConnected(remoteAddress)) inbox.process(dispatcher) @@ -121,12 +112,11 @@ class InboxSuite extends SparkFunSuite { test("post: Disassociated") { val endpoint = new TestRpcEndpoint - val endpointRef = mock(classOf[NettyRpcEndpointRef]) val dispatcher = mock(classOf[Dispatcher]) val remoteAddress = RpcAddress("localhost", 11111) - val inbox = new Inbox(endpointRef, endpoint) + val inbox = new Inbox("name", endpoint) inbox.post(RemoteProcessDisconnected(remoteAddress)) inbox.process(dispatcher) @@ -135,13 +125,12 @@ class InboxSuite extends SparkFunSuite { test("post: AssociationError") { val endpoint = new TestRpcEndpoint - val endpointRef = mock(classOf[NettyRpcEndpointRef]) val dispatcher = mock(classOf[Dispatcher]) val remoteAddress = RpcAddress("localhost", 11111) val cause = new RuntimeException("Oops") - val inbox = new Inbox(endpointRef, endpoint) + val inbox = new Inbox("name", endpoint) inbox.post(RemoteProcessConnectionError(cause, remoteAddress)) inbox.process(dispatcher) From 100fc58da54e026cda87832a10e2d06eaeccdf87 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 17 Oct 2019 11:15:25 -0700 Subject: [PATCH 0351/1387] [SPARK-28869][CORE] Roll over event log files ### What changes were proposed in this pull request? This patch is a part of [SPARK-28594](https://issues.apache.org/jira/browse/SPARK-28594) and design doc for SPARK-28594 is linked here: https://docs.google.com/document/d/12bdCC4nA58uveRxpeo8k7kGOI2NRTXmXyBOweSi4YcY/edit?usp=sharing This patch proposes adding new feature to event logging, rolling event log files via configured file size. Previously event logging is done with single file and related codebase (`EventLoggingListener`/`FsHistoryProvider`) is tightly coupled with it. This patch adds layer on both reader (`EventLogFileReader`) and writer (`EventLogFileWriter`) to decouple implementation details between "handling events" and "how to read/write events from/to file". This patch adds two properties, `spark.eventLog.rollLog` and `spark.eventLog.rollLog.maxFileSize` which provides configurable behavior of rolling log. The feature is disabled by default, as we only expect huge event log for huge/long-running application. For other cases single event log file would be sufficient and still simpler. ### Why are the changes needed? This is a part of SPARK-28594 which addresses event log growing infinitely for long-running application. This patch itself also provides some option for the situation where event log file gets huge and consume their storage. End users may give up replaying their events and want to delete the event log file, but given application is still running and writing the file, it's not safe to delete the file. End users will be able to delete some of old files after applying rolling over event log. ### Does this PR introduce any user-facing change? No, as the new feature is turned off by default. ### How was this patch tested? Added unit tests, as well as basic manual tests. Basic manual tests - ran SHS, ran structured streaming query with roll event log enabled, verified split files are generated as well as SHS can load these files, with handling app status as incomplete/complete. Closes #25670 from HeartSaVioR/SPARK-28869. Lead-authored-by: Jungtaek Lim (HeartSaVioR) Co-authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../deploy/history/EventLogFileReaders.scala | 264 +++++++++++ .../deploy/history/EventLogFileWriters.scala | 415 ++++++++++++++++++ .../deploy/history/FsHistoryProvider.scala | 195 ++++---- .../spark/internal/config/package.scala | 15 + .../scheduler/EventLoggingListener.scala | 222 ++-------- .../spark/scheduler/ReplayListenerBus.scala | 11 +- .../spark/deploy/SparkSubmitSuite.scala | 4 +- .../history/EventLogFileReadersSuite.scala | 345 +++++++++++++++ .../history/EventLogFileWritersSuite.scala | 378 ++++++++++++++++ .../deploy/history/EventLogTestHelper.scala | 59 +++ .../history/FsHistoryProviderSuite.scala | 81 +++- .../scheduler/EventLoggingListenerSuite.scala | 108 +---- .../spark/scheduler/ReplayListenerSuite.scala | 17 +- docs/configuration.md | 15 + 14 files changed, 1719 insertions(+), 410 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala new file mode 100644 index 0000000000000..c8956ed3d423d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -0,0 +1,264 @@ +/* + * 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.history + +import java.io.{BufferedInputStream, InputStream} +import java.util.concurrent.ConcurrentHashMap +import java.util.zip.{ZipEntry, ZipOutputStream} + +import com.google.common.io.ByteStreams +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.hdfs.DFSInputStream + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.history.EventLogFileWriter.codecName +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils + +/** The base class of reader which will read the information of event log file(s). */ +abstract class EventLogFileReader( + protected val fileSystem: FileSystem, + val rootPath: Path) { + + protected def fileSizeForDFS(path: Path): Option[Long] = { + Utils.tryWithResource(fileSystem.open(path)) { in => + in.getWrappedStream match { + case dfsIn: DFSInputStream => Some(dfsIn.getFileLength) + case _ => None + } + } + } + + protected def addFileAsZipEntry( + zipStream: ZipOutputStream, + path: Path, + entryName: String): Unit = { + Utils.tryWithResource(fileSystem.open(path, 1 * 1024 * 1024)) { inputStream => + zipStream.putNextEntry(new ZipEntry(entryName)) + ByteStreams.copy(inputStream, zipStream) + zipStream.closeEntry() + } + } + + /** Returns the last index of event log files. None for single event log file. */ + def lastIndex: Option[Long] + + /** + * Returns the size of file for the last index of event log files. Returns its size for + * single event log file. + */ + def fileSizeForLastIndex: Long + + /** Returns whether the application is completed. */ + def completed: Boolean + + /** + * Returns the size of file for the last index (itself for single event log file) of event log + * files, only when underlying input stream is DFSInputStream. Otherwise returns None. + */ + def fileSizeForLastIndexForDFS: Option[Long] + + /** + * Returns the modification time for the last index (itself for single event log file) + * of event log files. + */ + def modificationTime: Long + + /** + * This method compresses the files passed in, and writes the compressed data out into the + * ZipOutputStream passed in. Each file is written as a new ZipEntry with its name being + * the name of the file being compressed. + */ + def zipEventLogFiles(zipStream: ZipOutputStream): Unit + + /** Returns all available event log files. */ + def listEventLogFiles: Seq[FileStatus] + + /** Returns the short compression name if being used. None if it's uncompressed. */ + def compressionCodec: Option[String] + + /** Returns the size of all event log files. */ + def totalSize: Long +} + +object EventLogFileReader { + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new ConcurrentHashMap[String, CompressionCodec]() + + def apply( + fs: FileSystem, + path: Path, + lastIndex: Option[Long]): EventLogFileReader = { + lastIndex match { + case Some(_) => new RollingEventLogFilesFileReader(fs, path) + case None => new SingleFileEventLogFileReader(fs, path) + } + } + + def apply(fs: FileSystem, path: Path): Option[EventLogFileReader] = { + apply(fs, fs.getFileStatus(path)) + } + + def apply(fs: FileSystem, status: FileStatus): Option[EventLogFileReader] = { + if (isSingleEventLog(status)) { + Some(new SingleFileEventLogFileReader(fs, status.getPath)) + } else if (isRollingEventLogs(status)) { + Some(new RollingEventLogFilesFileReader(fs, status.getPath)) + } else { + None + } + } + + /** + * Opens an event log file and returns an input stream that contains the event data. + * + * @return input stream that holds one JSON record per line. + */ + def openEventLog(log: Path, fs: FileSystem): InputStream = { + val in = new BufferedInputStream(fs.open(log)) + try { + val codec = codecName(log).map { c => + codecMap.computeIfAbsent(c, CompressionCodec.createCodec(new SparkConf, _)) + } + codec.map(_.compressedContinuousInputStream(in)).getOrElse(in) + } catch { + case e: Throwable => + in.close() + throw e + } + } + + private def isSingleEventLog(status: FileStatus): Boolean = { + !status.isDirectory && + // FsHistoryProvider used to generate a hidden file which can't be read. Accidentally + // reading a garbage file is safe, but we would log an error which can be scary to + // the end-user. + !status.getPath.getName.startsWith(".") + } + + private def isRollingEventLogs(status: FileStatus): Boolean = { + RollingEventLogFilesWriter.isEventLogDir(status) + } +} + +/** + * The reader which will read the information of single event log file. + * + * This reader gets the status of event log file only once when required; + * It may not give "live" status of file that could be changing concurrently, and + * FileNotFoundException could occur if the log file is renamed before getting the + * status of log file. + */ +class SingleFileEventLogFileReader( + fs: FileSystem, + path: Path) extends EventLogFileReader(fs, path) { + private lazy val status = fileSystem.getFileStatus(rootPath) + + override def lastIndex: Option[Long] = None + + override def fileSizeForLastIndex: Long = status.getLen + + override def completed: Boolean = !rootPath.getName.endsWith(EventLogFileWriter.IN_PROGRESS) + + override def fileSizeForLastIndexForDFS: Option[Long] = { + if (completed) { + Some(fileSizeForLastIndex) + } else { + fileSizeForDFS(rootPath) + } + } + + override def modificationTime: Long = status.getModificationTime + + override def zipEventLogFiles(zipStream: ZipOutputStream): Unit = { + addFileAsZipEntry(zipStream, rootPath, rootPath.getName) + } + + override def listEventLogFiles: Seq[FileStatus] = Seq(status) + + override def compressionCodec: Option[String] = EventLogFileWriter.codecName(rootPath) + + override def totalSize: Long = fileSizeForLastIndex +} + +/** + * The reader which will read the information of rolled multiple event log files. + * + * This reader lists the files only once; if caller would like to play with updated list, + * it needs to create another reader instance. + */ +class RollingEventLogFilesFileReader( + fs: FileSystem, + path: Path) extends EventLogFileReader(fs, path) { + import RollingEventLogFilesWriter._ + + private lazy val files: Seq[FileStatus] = { + val ret = fs.listStatus(rootPath).toSeq + require(ret.exists(isEventLogFile), "Log directory must contain at least one event log file!") + require(ret.exists(isAppStatusFile), "Log directory must contain an appstatus file!") + ret + } + + private lazy val appStatusFile = files.find(isAppStatusFile).get + + private lazy val eventLogFiles: Seq[FileStatus] = { + val eventLogFiles = files.filter(isEventLogFile).sortBy { status => + getIndex(status.getPath.getName) + } + val indices = eventLogFiles.map { file => getIndex(file.getPath.getName) }.sorted + require((indices.head to indices.last) == indices, "Found missing event log file, expected" + + s" indices: ${(indices.head to indices.last)}, actual: ${indices}") + eventLogFiles + } + + override def lastIndex: Option[Long] = Some(getIndex(lastEventLogFile.getPath.getName)) + + override def fileSizeForLastIndex: Long = lastEventLogFile.getLen + + override def completed: Boolean = { + !appStatusFile.getPath.getName.endsWith(EventLogFileWriter.IN_PROGRESS) + } + + override def fileSizeForLastIndexForDFS: Option[Long] = { + if (completed) { + Some(fileSizeForLastIndex) + } else { + fileSizeForDFS(lastEventLogFile.getPath) + } + } + + override def modificationTime: Long = lastEventLogFile.getModificationTime + + override def zipEventLogFiles(zipStream: ZipOutputStream): Unit = { + val dirEntryName = rootPath.getName + "/" + zipStream.putNextEntry(new ZipEntry(dirEntryName)) + files.foreach { file => + addFileAsZipEntry(zipStream, file.getPath, dirEntryName + file.getPath.getName) + } + } + + override def listEventLogFiles: Seq[FileStatus] = eventLogFiles + + override def compressionCodec: Option[String] = { + EventLogFileWriter.codecName(eventLogFiles.head.getPath) + } + + override def totalSize: Long = eventLogFiles.map(_.getLen).sum + + private def lastEventLogFile: FileStatus = eventLogFiles.last +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala new file mode 100644 index 0000000000000..3fa5ef94892aa --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -0,0 +1,415 @@ +/* + * 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.history + +import java.io._ +import java.net.URI +import java.nio.charset.StandardCharsets + +import org.apache.commons.compress.utils.CountingOutputStream +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils + +/** + * The base class of writer which will write event logs into file. + * + * The following configurable parameters are available to tune the behavior of writing: + * spark.eventLog.compress - Whether to compress logged events + * spark.eventLog.compression.codec - The codec to compress logged events + * spark.eventLog.overwrite - Whether to overwrite any existing files + * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams + * + * Note that descendant classes can maintain its own parameters: refer the javadoc of each class + * for more details. + * + * NOTE: CountingOutputStream being returned by "initLogFile" counts "non-compressed" bytes. + */ +abstract class EventLogFileWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration) extends Logging { + + protected val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) + protected val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) + protected val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt + protected val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) + protected val compressionCodec = + if (shouldCompress) { + Some(CompressionCodec.createCodec(sparkConf, sparkConf.get(EVENT_LOG_COMPRESSION_CODEC))) + } else { + None + } + + private[history] val compressionCodecName = compressionCodec.map { c => + CompressionCodec.getShortName(c.getClass.getName) + } + + // Only defined if the file system scheme is not local + protected var hadoopDataStream: Option[FSDataOutputStream] = None + protected var writer: Option[PrintWriter] = None + + protected def requireLogBaseDirAsDirectory(): Unit = { + if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDirectory) { + throw new IllegalArgumentException(s"Log directory $logBaseDir is not a directory.") + } + } + + protected def initLogFile(path: Path)(fnSetupWriter: OutputStream => PrintWriter): Unit = { + if (shouldOverwrite && fileSystem.delete(path, true)) { + logWarning(s"Event log $path already exists. Overwriting...") + } + + val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme + val isDefaultLocal = defaultFs == null || defaultFs == "file" + val uri = path.toUri + + // The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + // Therefore, for local files, use FileOutputStream instead. + val dstream = + if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { + new FileOutputStream(uri.getPath) + } else { + hadoopDataStream = Some( + SparkHadoopUtil.createFile(fileSystem, path, sparkConf.get(EVENT_LOG_ALLOW_EC))) + hadoopDataStream.get + } + + try { + val cstream = compressionCodec.map(_.compressedContinuousOutputStream(dstream)) + .getOrElse(dstream) + val bstream = new BufferedOutputStream(cstream, outputBufferSize) + fileSystem.setPermission(path, EventLogFileWriter.LOG_FILE_PERMISSIONS) + logInfo(s"Logging events to $path") + writer = Some(fnSetupWriter(bstream)) + } catch { + case e: Exception => + dstream.close() + throw e + } + } + + protected def writeJson(json: String, flushLogger: Boolean = false): Unit = { + // scalastyle:off println + writer.foreach(_.println(json)) + // scalastyle:on println + if (flushLogger) { + writer.foreach(_.flush()) + hadoopDataStream.foreach(_.hflush()) + } + } + + protected def closeWriter(): Unit = { + writer.foreach(_.close()) + } + + protected def renameFile(src: Path, dest: Path, overwrite: Boolean): Unit = { + if (fileSystem.exists(dest)) { + if (overwrite) { + logWarning(s"Event log $dest already exists. Overwriting...") + if (!fileSystem.delete(dest, true)) { + logWarning(s"Error deleting $dest") + } + } else { + throw new IOException(s"Target log file already exists ($dest)") + } + } + fileSystem.rename(src, dest) + // touch file to ensure modtime is current across those filesystems where rename() + // does not set it but support setTimes() instead; it's a no-op on most object stores + try { + fileSystem.setTimes(dest, System.currentTimeMillis(), -1) + } catch { + case e: Exception => logDebug(s"failed to set time of $dest", e) + } + } + + /** initialize writer for event logging */ + def start(): Unit + + /** writes JSON format of event to file */ + def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit + + /** stops writer - indicating the application has been completed */ + def stop(): Unit + + /** returns representative path of log. for tests only. */ + def logPath: String +} + +object EventLogFileWriter { + // Suffix applied to the names of files still being written by applications. + val IN_PROGRESS = ".inprogress" + + val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) + + def apply( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + if (sparkConf.get(EVENT_LOG_ENABLE_ROLLING)) { + new RollingEventLogFilesWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } else { + new SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + } + + def nameForAppAndAttempt(appId: String, appAttemptId: Option[String]): String = { + val base = Utils.sanitizeDirName(appId) + if (appAttemptId.isDefined) { + base + "_" + Utils.sanitizeDirName(appAttemptId.get) + } else { + base + } + } + + def codecName(log: Path): Option[String] = { + // Compression codec is encoded as an extension, e.g. app_123.lzf + // Since we sanitize the app ID to not include periods, it is safe to split on it + val logName = log.getName.stripSuffix(IN_PROGRESS) + logName.split("\\.").tail.lastOption + } +} + +/** + * The writer to write event logs into single file. + */ +class SingleEventLogFileWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration) + extends EventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) { + + override val logPath: String = SingleEventLogFileWriter.getLogPath(logBaseDir, appId, + appAttemptId, compressionCodecName) + + private val inProgressPath = logPath + EventLogFileWriter.IN_PROGRESS + + override def start(): Unit = { + requireLogBaseDirAsDirectory() + + initLogFile(new Path(inProgressPath)) { os => + new PrintWriter(new OutputStreamWriter(os, StandardCharsets.UTF_8)) + } + } + + override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { + writeJson(eventJson, flushLogger) + } + + /** + * Stop logging events. The event log file will be renamed so that it loses the + * ".inprogress" suffix. + */ + override def stop(): Unit = { + closeWriter() + renameFile(new Path(inProgressPath), new Path(logPath), shouldOverwrite) + } +} + +object SingleEventLogFileWriter { + /** + * Return a file-system-safe path to the log file for the given application. + * + * Note that because we currently only create a single log file for each application, + * we must encode all the information needed to parse this event log in the file name + * instead of within the file itself. Otherwise, if the file is compressed, for instance, + * we won't know which codec to use to decompress the metadata needed to open the file in + * the first place. + * + * The log file name will identify the compression codec used for the contents, if any. + * For example, app_123 for an uncompressed log, app_123.lzf for an LZF-compressed log. + * + * @param logBaseDir Directory where the log file will be written. + * @param appId A unique app ID. + * @param appAttemptId A unique attempt id of appId. May be the empty string. + * @param compressionCodecName Name to identify the codec used to compress the contents + * of the log, or None if compression is not enabled. + * @return A path which consists of file-system-safe characters. + */ + def getLogPath( + logBaseDir: URI, + appId: String, + appAttemptId: Option[String], + compressionCodecName: Option[String] = None): String = { + val codec = compressionCodecName.map("." + _).getOrElse("") + new Path(logBaseDir).toString.stripSuffix("/") + "/" + + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) + codec + } +} + +/** + * The writer to write event logs into multiple log files, rolled over via configured size. + * + * The class creates one directory per application, and stores event log files as well as + * metadata files. The name of directory and files in the directory would follow: + * + * - The name of directory: eventlog_v2_appId(_[appAttemptId]) + * - The prefix of name on event files: events_[index]_[appId](_[appAttemptId])(.[codec]) + * - "index" would be monotonically increasing value (say, sequence) + * - The name of metadata (app. status) file name: appstatus_[appId](_[appAttemptId])(.inprogress) + * + * The writer will roll over the event log file when configured size is reached. Note that the + * writer doesn't check the size on file being open for write: the writer tracks the count of bytes + * written before compression is applied. + * + * For metadata files, the class will leverage zero-byte file, as it provides minimized cost. + */ +class RollingEventLogFilesWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration) + extends EventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) { + + import RollingEventLogFilesWriter._ + + private val eventFileMaxLength = sparkConf.get(EVENT_LOG_ROLLING_MAX_FILE_SIZE) + + private val logDirForAppPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) + + private var countingOutputStream: Option[CountingOutputStream] = None + + // index and event log path will be updated soon in rollEventLogFile, which `start` will call + private var index: Long = 0L + private var currentEventLogFilePath: Path = _ + + override def start(): Unit = { + requireLogBaseDirAsDirectory() + + if (fileSystem.exists(logDirForAppPath) && shouldOverwrite) { + fileSystem.delete(logDirForAppPath, true) + } + + if (fileSystem.exists(logDirForAppPath)) { + throw new IOException(s"Target log directory already exists ($logDirForAppPath)") + } + + fileSystem.mkdirs(logDirForAppPath, EventLogFileWriter.LOG_FILE_PERMISSIONS) + createAppStatusFile(inProgress = true) + rollEventLogFile() + } + + override def writeEvent(eventJson: String, flushLogger: Boolean = false): Unit = { + writer.foreach { w => + val currentLen = countingOutputStream.get.getBytesWritten + if (currentLen + eventJson.length > eventFileMaxLength) { + rollEventLogFile() + } + } + + writeJson(eventJson, flushLogger) + } + + private def rollEventLogFile(): Unit = { + closeWriter() + + index += 1 + currentEventLogFilePath = getEventLogFilePath(logDirForAppPath, appId, appAttemptId, index, + compressionCodecName) + + initLogFile(currentEventLogFilePath) { os => + countingOutputStream = Some(new CountingOutputStream(os)) + new PrintWriter( + new OutputStreamWriter(countingOutputStream.get, StandardCharsets.UTF_8)) + } + } + + override def stop(): Unit = { + closeWriter() + val appStatusPathIncomplete = getAppStatusFilePath(logDirForAppPath, appId, appAttemptId, + inProgress = true) + val appStatusPathComplete = getAppStatusFilePath(logDirForAppPath, appId, appAttemptId, + inProgress = false) + renameFile(appStatusPathIncomplete, appStatusPathComplete, overwrite = true) + } + + override def logPath: String = logDirForAppPath.toString + + private def createAppStatusFile(inProgress: Boolean): Unit = { + val appStatusPath = getAppStatusFilePath(logDirForAppPath, appId, appAttemptId, inProgress) + val outputStream = fileSystem.create(appStatusPath) + // we intentionally create zero-byte file to minimize the cost + outputStream.close() + } +} + +object RollingEventLogFilesWriter { + private[history] val EVENT_LOG_DIR_NAME_PREFIX = "eventlog_v2_" + private[history] val EVENT_LOG_FILE_NAME_PREFIX = "events_" + private[history] val APPSTATUS_FILE_NAME_PREFIX = "appstatus_" + + def getAppEventLogDirPath(logBaseDir: URI, appId: String, appAttemptId: Option[String]): Path = + new Path(new Path(logBaseDir), EVENT_LOG_DIR_NAME_PREFIX + + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId)) + + def getAppStatusFilePath( + appLogDir: Path, + appId: String, + appAttemptId: Option[String], + inProgress: Boolean): Path = { + val base = APPSTATUS_FILE_NAME_PREFIX + + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) + val name = if (inProgress) base + EventLogFileWriter.IN_PROGRESS else base + new Path(appLogDir, name) + } + + def getEventLogFilePath( + appLogDir: Path, + appId: String, + appAttemptId: Option[String], + index: Long, + codecName: Option[String]): Path = { + val base = s"${EVENT_LOG_FILE_NAME_PREFIX}${index}_" + + EventLogFileWriter.nameForAppAndAttempt(appId, appAttemptId) + val codec = codecName.map("." + _).getOrElse("") + new Path(appLogDir, base + codec) + } + + def isEventLogDir(status: FileStatus): Boolean = { + status.isDirectory && status.getPath.getName.startsWith(EVENT_LOG_DIR_NAME_PREFIX) + } + + def isEventLogFile(status: FileStatus): Boolean = { + status.isFile && status.getPath.getName.startsWith(EVENT_LOG_FILE_NAME_PREFIX) + } + + def isAppStatusFile(status: FileStatus): Boolean = { + status.isFile && status.getPath.getName.startsWith(APPSTATUS_FILE_NAME_PREFIX) + } + + def getIndex(eventLogFileName: String): Long = { + require(eventLogFileName.startsWith(EVENT_LOG_FILE_NAME_PREFIX), "Not an event log file!") + val index = eventLogFileName.stripPrefix(EVENT_LOG_FILE_NAME_PREFIX).split("_")(0) + index.toLong + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index dce9581be2905..0b7e695efa870 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -439,27 +439,27 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logDebug(s"Scanning $logDir with lastScanTime==$lastScanTime") val updated = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) - .filter { entry => - !entry.isDirectory() && - // FsHistoryProvider used to generate a hidden file which can't be read. Accidentally - // reading a garbage file is safe, but we would log an error which can be scary to - // the end-user. - !entry.getPath().getName().startsWith(".") && - !isBlacklisted(entry.getPath) - } - .filter { entry => + .filter { entry => !isBlacklisted(entry.getPath) } + .flatMap { entry => EventLogFileReader(fs, entry) } + .filter { reader => try { - val info = listing.read(classOf[LogInfo], entry.getPath().toString()) + val info = listing.read(classOf[LogInfo], reader.rootPath.toString()) if (info.appId.isDefined) { // If the SHS view has a valid application, update the time the file was last seen so // that the entry is not deleted from the SHS listing. Also update the file size, in // case the code below decides we don't need to parse the log. - listing.write(info.copy(lastProcessed = newLastScanTime, fileSize = entry.getLen())) + listing.write(info.copy(lastProcessed = newLastScanTime, + fileSize = reader.fileSizeForLastIndex, + lastIndex = reader.lastIndex, + isComplete = reader.completed)) } - if (shouldReloadLog(info, entry)) { - if (info.appId.isDefined && fastInProgressParsing) { + if (shouldReloadLog(info, reader)) { + // ignore fastInProgressParsing when the status of application is changed from + // in-progress to completed, which is needed for rolling event log. + if (info.appId.isDefined && (info.isComplete == reader.completed) && + fastInProgressParsing) { // When fast in-progress parsing is on, we don't need to re-parse when the // size changes, but we do need to invalidate any existing UIs. // Also, we need to update the `lastUpdated time` to display the updated time in @@ -472,6 +472,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attempt.info.copy(lastUpdated = new Date(newLastScanTime)), attempt.logPath, attempt.fileSize, + attempt.lastIndex, attempt.adminAcls, attempt.viewAcls, attempt.adminAclsGroups, @@ -497,24 +498,25 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // If the file is currently not being tracked by the SHS, add an entry for it and try // to parse it. This will allow the cleaner code to detect the file as stale later on // if it was not possible to parse it. - listing.write(LogInfo(entry.getPath().toString(), newLastScanTime, LogType.EventLogs, - None, None, entry.getLen())) - entry.getLen() > 0 + listing.write(LogInfo(reader.rootPath.toString(), newLastScanTime, LogType.EventLogs, + None, None, reader.fileSizeForLastIndex, reader.lastIndex, + reader.completed)) + reader.fileSizeForLastIndex > 0 } } .sortWith { case (entry1, entry2) => - entry1.getModificationTime() > entry2.getModificationTime() + entry1.modificationTime > entry2.modificationTime } if (updated.nonEmpty) { - logDebug(s"New/updated attempts found: ${updated.size} ${updated.map(_.getPath)}") + logDebug(s"New/updated attempts found: ${updated.size} ${updated.map(_.rootPath)}") } val tasks = updated.flatMap { entry => try { val task: Future[Unit] = replayExecutor.submit( () => mergeApplicationListing(entry, newLastScanTime, true)) - Some(task -> entry.getPath) + Some(task -> entry.rootPath) } catch { // let the iteration over the updated entries break, since an exception on // replayExecutor.submit (..) indicates the ExecutorService is unable @@ -574,22 +576,26 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - private[history] def shouldReloadLog(info: LogInfo, entry: FileStatus): Boolean = { - var result = info.fileSize < entry.getLen - if (!result && info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { - try { - result = Utils.tryWithResource(fs.open(entry.getPath)) { in => - in.getWrappedStream match { - case dfsIn: DFSInputStream => info.fileSize < dfsIn.getFileLength - case _ => false - } + private[history] def shouldReloadLog(info: LogInfo, reader: EventLogFileReader): Boolean = { + if (info.isComplete != reader.completed) { + true + } else { + var result = if (info.lastIndex.isDefined) { + require(reader.lastIndex.isDefined) + info.lastIndex.get < reader.lastIndex.get || info.fileSize < reader.fileSizeForLastIndex + } else { + info.fileSize < reader.fileSizeForLastIndex + } + if (!result && !reader.completed) { + try { + result = reader.fileSizeForLastIndexForDFS.exists(info.fileSize < _) + } catch { + case e: Exception => + logDebug(s"Failed to check the length for the file : ${info.logPath}", e) } - } catch { - case e: Exception => - logDebug(s"Failed to check the length for the file : ${info.logPath}", e) } + result } - result } private def cleanAppData(appId: String, attemptId: Option[String], logPath: String): Unit = { @@ -636,23 +642,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attemptId: Option[String], zipStream: ZipOutputStream): Unit = { - /** - * This method compresses the files passed in, and writes the compressed data out into the - * [[OutputStream]] passed in. Each file is written as a new [[ZipEntry]] with its name being - * the name of the file being compressed. - */ - def zipFileToStream(file: Path, entryName: String, outputStream: ZipOutputStream): Unit = { - val fs = file.getFileSystem(hadoopConf) - val inputStream = fs.open(file, 1 * 1024 * 1024) // 1MB Buffer - try { - outputStream.putNextEntry(new ZipEntry(entryName)) - ByteStreams.copy(inputStream, outputStream) - outputStream.closeEntry() - } finally { - inputStream.close() - } - } - val app = try { load(appId) } catch { @@ -665,9 +654,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attemptId .map { id => app.attempts.filter(_.info.attemptId == Some(id)) } .getOrElse(app.attempts) - .map(_.logPath) - .foreach { log => - zipFileToStream(new Path(logDir, log), log, zipStream) + .foreach { attempt => + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastIndex) + reader.zipEventLogFiles(zipStream) } } finally { zipStream.close() @@ -678,7 +668,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * Replay the given log file, saving the application in the listing db. */ protected def mergeApplicationListing( - fileStatus: FileStatus, + reader: EventLogFileReader, scanTime: Long, enableOptimizations: Boolean): Unit = { val eventsFilter: ReplayEventsFilter = { eventString => @@ -688,8 +678,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) eventString.startsWith(ENV_UPDATE_EVENT_PREFIX) } - val logPath = fileStatus.getPath() - val appCompleted = isCompleted(logPath.getName()) + val logPath = reader.rootPath + val appCompleted = reader.completed val reparseChunkSize = conf.get(END_EVENT_REPARSE_CHUNK_SIZE) // Enable halt support in listener if: @@ -699,13 +689,12 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) ((!appCompleted && fastInProgressParsing) || reparseChunkSize > 0) val bus = new ReplayListenerBus() - val listener = new AppListingListener(fileStatus, clock, shouldHalt) + val listener = new AppListingListener(reader, clock, shouldHalt) bus.addListener(listener) logInfo(s"Parsing $logPath for listing data...") - Utils.tryWithResource(EventLoggingListener.openEventLog(logPath, fs)) { in => - bus.replay(in, logPath.toString, !appCompleted, eventsFilter) - } + val logFiles = reader.listEventLogFiles + parseAppEventLogs(logFiles, bus, !appCompleted, eventsFilter) // If enabled above, the listing listener will halt parsing when there's enough information to // create a listing entry. When the app is completed, or fast parsing is disabled, we still need @@ -727,8 +716,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // current position is, since the replay listener bus buffers data internally. val lookForEndEvent = shouldHalt && (appCompleted || !fastInProgressParsing) if (lookForEndEvent && listener.applicationInfo.isDefined) { - Utils.tryWithResource(EventLoggingListener.openEventLog(logPath, fs)) { in => - val target = fileStatus.getLen() - reparseChunkSize + val lastFile = logFiles.last + Utils.tryWithResource(EventLogFileReader.openEventLog(lastFile.getPath, fs)) { in => + val target = lastFile.getLen - reparseChunkSize if (target > 0) { logInfo(s"Looking for end event; skipping $target bytes from $logPath...") var skipped = 0L @@ -745,7 +735,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) source.next() } - bus.replay(source, logPath.toString, !appCompleted, eventsFilter) + bus.replay(source, lastFile.getPath.toString, !appCompleted, eventsFilter) } } @@ -758,12 +748,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) invalidateUI(app.info.id, app.attempts.head.info.attemptId) addListing(app) listing.write(LogInfo(logPath.toString(), scanTime, LogType.EventLogs, Some(app.info.id), - app.attempts.head.info.attemptId, fileStatus.getLen())) + app.attempts.head.info.attemptId, reader.fileSizeForLastIndex, + reader.lastIndex, reader.completed)) // For a finished log, remove the corresponding "in progress" entry from the listing DB if // the file is really gone. - if (appCompleted) { - val inProgressLog = logPath.toString() + EventLoggingListener.IN_PROGRESS + // The logic is only valid for single event log, as root path doesn't change for + // rolled event logs. + if (appCompleted && reader.lastIndex.isEmpty) { + val inProgressLog = logPath.toString() + EventLogFileWriter.IN_PROGRESS try { // Fetch the entry first to avoid an RPC when it's already removed. listing.read(classOf[LogInfo], inProgressLog) @@ -780,14 +773,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // mean the end event is before the configured threshold, so call the method again to // re-parse the whole log. logInfo(s"Reparsing $logPath since end event was not found.") - mergeApplicationListing(fileStatus, scanTime, false) + mergeApplicationListing(reader, scanTime, enableOptimizations = false) case _ => // If the app hasn't written down its app ID to the logs, still record the entry in the // listing db, with an empty ID. This will make the log eligible for deletion if the app // does not make progress after the configured max log age. listing.write( - LogInfo(logPath.toString(), scanTime, LogType.EventLogs, None, None, fileStatus.getLen())) + LogInfo(logPath.toString(), scanTime, LogType.EventLogs, None, None, + reader.fileSizeForLastIndex, reader.lastIndex, reader.completed)) } } @@ -922,7 +916,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) case e: NoSuchElementException => // For every new driver log file discovered, create a new entry in listing listing.write(LogInfo(f.getPath().toString(), currentTime, LogType.DriverLogs, None, - None, f.getLen())) + None, f.getLen(), None, false)) false } if (deleteFile) { @@ -953,7 +947,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) */ private def rebuildAppStore( store: KVStore, - eventLog: FileStatus, + reader: EventLogFileReader, lastUpdated: Long): Unit = { // Disable async updates, since they cause higher memory usage, and it's ok to take longer // to parse the event logs in the SHS. @@ -970,13 +964,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } replayBus.addListener(listener) try { - val path = eventLog.getPath() - logInfo(s"Parsing $path to re-build UI...") - Utils.tryWithResource(EventLoggingListener.openEventLog(path, fs)) { in => - replayBus.replay(in, path.toString(), maybeTruncated = !isCompleted(path.toString())) - } + logInfo(s"Parsing ${reader.rootPath} to re-build UI...") + parseAppEventLogs(reader.listEventLogFiles, replayBus, !reader.completed) trackingStore.close(false) - logInfo(s"Finished parsing $path") + logInfo(s"Finished parsing ${reader.rootPath}") } catch { case e: Exception => Utils.tryLogNonFatalError { @@ -986,6 +977,23 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + private def parseAppEventLogs( + logFiles: Seq[FileStatus], + replayBus: ReplayListenerBus, + maybeTruncated: Boolean, + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { + // stop replaying next log files if ReplayListenerBus indicates some error or halt + var continueReplay = true + logFiles.foreach { file => + if (continueReplay) { + Utils.tryWithResource(EventLogFileReader.openEventLog(file.getPath, fs)) { in => + continueReplay = replayBus.replay(in, file.getPath.toString, + maybeTruncated = maybeTruncated, eventsFilter = eventsFilter) + } + } + } + } + /** * Checks whether HDFS is in safe mode. * @@ -1067,15 +1075,15 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // At this point the disk data either does not exist or was deleted because it failed to // load, so the event log needs to be replayed. - val status = fs.getFileStatus(new Path(logDir, attempt.logPath)) - val isCompressed = EventLoggingListener.codecName(status.getPath()).flatMap { name => - Try(CompressionCodec.getShortName(name)).toOption - }.isDefined + + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastIndex) + val isCompressed = reader.compressionCodec.isDefined logInfo(s"Leasing disk manager space for app $appId / ${attempt.info.attemptId}...") - val lease = dm.lease(status.getLen(), isCompressed) + val lease = dm.lease(reader.totalSize, isCompressed) val newStorePath = try { Utils.tryWithResource(KVUtils.open(lease.tmpPath, metadata)) { store => - rebuildAppStore(store, status, attempt.info.lastUpdated.getTime()) + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) } lease.commit(appId, attempt.info.attemptId) } catch { @@ -1089,8 +1097,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private def createInMemoryStore(attempt: AttemptInfoWrapper): KVStore = { val store = new InMemoryStore() - val status = fs.getFileStatus(new Path(logDir, attempt.logPath)) - rebuildAppStore(store, status, attempt.info.lastUpdated.getTime()) + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastIndex) + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) store } @@ -1120,11 +1129,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } deleted } - - private def isCompleted(name: String): Boolean = { - !name.endsWith(EventLoggingListener.IN_PROGRESS) - } - } private[history] object FsHistoryProvider { @@ -1165,12 +1169,15 @@ private[history] case class LogInfo( logType: LogType.Value, appId: Option[String], attemptId: Option[String], - fileSize: Long) + fileSize: Long, + lastIndex: Option[Long], + isComplete: Boolean) private[history] class AttemptInfoWrapper( val info: ApplicationAttemptInfo, val logPath: String, val fileSize: Long, + val lastIndex: Option[Long], val adminAcls: Option[String], val viewAcls: Option[String], val adminAclsGroups: Option[String], @@ -1194,12 +1201,13 @@ private[history] class ApplicationInfoWrapper( } private[history] class AppListingListener( - log: FileStatus, + reader: EventLogFileReader, clock: Clock, haltEnabled: Boolean) extends SparkListener { private val app = new MutableApplicationInfo() - private val attempt = new MutableAttemptInfo(log.getPath().getName(), log.getLen()) + private val attempt = new MutableAttemptInfo(reader.rootPath.getName(), + reader.fileSizeForLastIndex, reader.lastIndex) private var gotEnvUpdate = false private var halted = false @@ -1218,7 +1226,7 @@ private[history] class AppListingListener( override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { attempt.endTime = new Date(event.time) - attempt.lastUpdated = new Date(log.getModificationTime()) + attempt.lastUpdated = new Date(reader.modificationTime) attempt.duration = event.time - attempt.startTime.getTime() attempt.completed = true } @@ -1284,7 +1292,7 @@ private[history] class AppListingListener( } - private class MutableAttemptInfo(logPath: String, fileSize: Long) { + private class MutableAttemptInfo(logPath: String, fileSize: Long, lastIndex: Option[Long]) { var attemptId: Option[String] = None var startTime = new Date(-1) var endTime = new Date(-1) @@ -1313,6 +1321,7 @@ private[history] class AppListingListener( apiInfo, logPath, fileSize, + lastIndex, adminAcls, viewAcls, adminAclsGroups, 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 79a6dc159e001..36211dc2ed4f8 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 @@ -179,6 +179,21 @@ package object config { private[spark] val EVENT_LOG_CALLSITE_LONG_FORM = ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false) + private[spark] val EVENT_LOG_ENABLE_ROLLING = + ConfigBuilder("spark.eventLog.rolling.enabled") + .doc("Whether rolling over event log files is enabled. If set to true, it cuts down " + + "each event log file to the configured size.") + .booleanConf + .createWithDefault(false) + + private[spark] val EVENT_LOG_ROLLING_MAX_FILE_SIZE = + ConfigBuilder("spark.eventLog.rolling.maxFileSize") + .doc("The max size of event log file to be rolled over.") + .bytesConf(ByteUnit.BYTE) + .checkValue(_ >= ByteUnit.MiB.toBytes(10), "Max file size of event log should be " + + "configured to be at least 10 MiB.") + .createWithDefaultString("128m") + private[spark] val EXECUTOR_ID = ConfigBuilder("spark.executor.id").stringConf.createOptional diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index a0a4428dc7f55..683cf3e918ee2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,24 +17,20 @@ package org.apache.spark.scheduler -import java.io._ import java.net.URI -import java.nio.charset.StandardCharsets -import scala.collection.mutable.{ArrayBuffer, Map} +import scala.collection.mutable import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} -import org.apache.hadoop.fs.permission.FsPermission import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogFileWriter import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} /** @@ -42,13 +38,12 @@ import org.apache.spark.util.{JsonProtocol, Utils} * * Event logging is specified by the following configurable parameters: * spark.eventLog.enabled - Whether event logging is enabled. - * spark.eventLog.logBlockUpdates.enabled - Whether to log block updates - * spark.eventLog.compress - Whether to compress logged events - * spark.eventLog.compression.codec - The codec to compress logged events - * spark.eventLog.overwrite - Whether to overwrite any existing files. * spark.eventLog.dir - Path to the directory in which events are logged. - * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams + * spark.eventLog.logBlockUpdates.enabled - Whether to log block updates * spark.eventLog.logStageExecutorMetrics.enabled - Whether to log stage executor metrics + * + * Event log file writer maintains its own parameters: refer the doc of [[EventLogFileWriter]] + * and its descendant for more details. */ private[spark] class EventLoggingListener( appId: String, @@ -64,93 +59,43 @@ private[spark] class EventLoggingListener( this(appId, appAttemptId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) - private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) - private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) - private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) - private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) - private val testing = sparkConf.get(EVENT_LOG_TESTING) - private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt - private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) - private val compressionCodec = - if (shouldCompress) { - Some(CompressionCodec.createCodec(sparkConf, sparkConf.get(EVENT_LOG_COMPRESSION_CODEC))) - } else { - None - } - // Visible for tests only. - private[scheduler] val compressionCodecName = compressionCodec.map { c => - CompressionCodec.getShortName(c.getClass.getName) - } - - // Only defined if the file system scheme is not local - private var hadoopDataStream: Option[FSDataOutputStream] = None - - private var writer: Option[PrintWriter] = None + // For testing. + private[scheduler] val logWriter: EventLogFileWriter = + EventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) // For testing. Keep track of all JSON serialized events that have been logged. - private[scheduler] val loggedEvents = new ArrayBuffer[JValue] + private[scheduler] val loggedEvents = new mutable.ArrayBuffer[JValue] - // Visible for tests only. - private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) + private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) + private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) + private val testing = sparkConf.get(EVENT_LOG_TESTING) // map of (stageId, stageAttempt) to executor metric peaks per executor/driver for the stage - private val liveStageExecutorMetrics = Map.empty[(Int, Int), Map[String, ExecutorMetrics]] + private val liveStageExecutorMetrics = + mutable.HashMap.empty[(Int, Int), mutable.HashMap[String, ExecutorMetrics]] /** * Creates the log file in the configured log directory. */ def start(): Unit = { - if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDirectory) { - throw new IllegalArgumentException(s"Log directory $logBaseDir is not a directory.") - } - - val workingPath = logPath + IN_PROGRESS - val path = new Path(workingPath) - val uri = path.toUri - val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme - val isDefaultLocal = defaultFs == null || defaultFs == "file" - - if (shouldOverwrite && fileSystem.delete(path, true)) { - logWarning(s"Event log $path already exists. Overwriting...") - } - - /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). - * Therefore, for local files, use FileOutputStream instead. */ - val dstream = - if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { - new FileOutputStream(uri.getPath) - } else { - hadoopDataStream = Some( - SparkHadoopUtil.createFile(fileSystem, path, sparkConf.get(EVENT_LOG_ALLOW_EC))) - hadoopDataStream.get - } + logWriter.start() + initEventLog() + } - try { - val cstream = compressionCodec.map(_.compressedContinuousOutputStream(dstream)) - .getOrElse(dstream) - val bstream = new BufferedOutputStream(cstream, outputBufferSize) - - EventLoggingListener.initEventLog(bstream, testing, loggedEvents) - fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) - writer = Some(new PrintWriter(new OutputStreamWriter(bstream, StandardCharsets.UTF_8))) - logInfo("Logging events to %s".format(logPath)) - } catch { - case e: Exception => - dstream.close() - throw e + private def initEventLog(): Unit = { + val metadata = SparkListenerLogStart(SPARK_VERSION) + val eventJson = JsonProtocol.logStartToJson(metadata) + val metadataJson = compact(eventJson) + logWriter.writeEvent(metadataJson, flushLogger = true) + if (testing && loggedEvents != null) { + loggedEvents += eventJson } } /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false): Unit = { val eventJson = JsonProtocol.sparkEventToJson(event) - // scalastyle:off println - writer.foreach(_.println(compact(render(eventJson)))) - // scalastyle:on println - if (flushLogger) { - writer.foreach(_.flush()) - hadoopDataStream.foreach(_.hflush()) - } + logWriter.writeEvent(compact(render(eventJson)), flushLogger) if (testing) { loggedEvents += eventJson } @@ -162,7 +107,7 @@ private[spark] class EventLoggingListener( if (shouldLogStageExecutorMetrics) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), - Map.empty[String, ExecutorMetrics]) + mutable.HashMap.empty[String, ExecutorMetrics]) } } @@ -296,32 +241,9 @@ private[spark] class EventLoggingListener( } } - /** - * Stop logging events. The event log file will be renamed so that it loses the - * ".inprogress" suffix. - */ + /** Stop logging events. */ def stop(): Unit = { - writer.foreach(_.close()) - - val target = new Path(logPath) - if (fileSystem.exists(target)) { - if (shouldOverwrite) { - logWarning(s"Event log $target already exists. Overwriting...") - if (!fileSystem.delete(target, true)) { - logWarning(s"Error deleting $target") - } - } else { - throw new IOException("Target log file already exists (%s)".format(logPath)) - } - } - fileSystem.rename(new Path(logPath + IN_PROGRESS), target) - // touch file to ensure modtime is current across those filesystems where rename() - // does not set it, -and which support setTimes(); it's a no-op on most object stores - try { - fileSystem.setTimes(target, System.currentTimeMillis(), -1) - } catch { - case e: Exception => logDebug(s"failed to set time of $target", e) - } + logWriter.stop() } private[spark] def redactEvent( @@ -342,93 +264,7 @@ private[spark] class EventLoggingListener( } private[spark] object EventLoggingListener extends Logging { - // Suffix applied to the names of files still being written by applications. - val IN_PROGRESS = ".inprogress" val DEFAULT_LOG_DIR = "/tmp/spark-events" // Dummy stage key used by driver in executor metrics updates val DRIVER_STAGE_KEY = (-1, -1) - - private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) - - // A cache for compression codecs to avoid creating the same codec many times - private val codecMap = Map.empty[String, CompressionCodec] - - /** - * Write metadata about an event log to the given stream. - * The metadata is encoded in the first line of the event log as JSON. - * - * @param logStream Raw output stream to the event log file. - */ - def initEventLog( - logStream: OutputStream, - testing: Boolean, - loggedEvents: ArrayBuffer[JValue]): Unit = { - val metadata = SparkListenerLogStart(SPARK_VERSION) - val eventJson = JsonProtocol.logStartToJson(metadata) - val metadataJson = compact(eventJson) + "\n" - logStream.write(metadataJson.getBytes(StandardCharsets.UTF_8)) - if (testing && loggedEvents != null) { - loggedEvents += eventJson - } - } - - /** - * Return a file-system-safe path to the log file for the given application. - * - * Note that because we currently only create a single log file for each application, - * we must encode all the information needed to parse this event log in the file name - * instead of within the file itself. Otherwise, if the file is compressed, for instance, - * we won't know which codec to use to decompress the metadata needed to open the file in - * the first place. - * - * The log file name will identify the compression codec used for the contents, if any. - * For example, app_123 for an uncompressed log, app_123.lzf for an LZF-compressed log. - * - * @param logBaseDir Directory where the log file will be written. - * @param appId A unique app ID. - * @param appAttemptId A unique attempt id of appId. May be the empty string. - * @param compressionCodecName Name to identify the codec used to compress the contents - * of the log, or None if compression is not enabled. - * @return A path which consists of file-system-safe characters. - */ - def getLogPath( - logBaseDir: URI, - appId: String, - appAttemptId: Option[String], - compressionCodecName: Option[String] = None): String = { - val base = new Path(logBaseDir).toString.stripSuffix("/") + "/" + Utils.sanitizeDirName(appId) - val codec = compressionCodecName.map("." + _).getOrElse("") - if (appAttemptId.isDefined) { - base + "_" + Utils.sanitizeDirName(appAttemptId.get) + codec - } else { - base + codec - } - } - - /** - * Opens an event log file and returns an input stream that contains the event data. - * - * @return input stream that holds one JSON record per line. - */ - def openEventLog(log: Path, fs: FileSystem): InputStream = { - val in = new BufferedInputStream(fs.open(log)) - try { - val codec = codecName(log).map { c => - codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) - } - codec.map(_.compressedContinuousInputStream(in)).getOrElse(in) - } catch { - case e: Throwable => - in.close() - throw e - } - } - - def codecName(log: Path): Option[String] = { - // Compression codec is encoded as an extension, e.g. app_123.lzf - // Since we sanitize the app ID to not include periods, it is safe to split on it - val logName = log.getName.stripSuffix(IN_PROGRESS) - logName.split("\\.").tail.lastOption - } - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 699042dd967bc..60b6fe7a60915 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -48,12 +48,14 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { * @param eventsFilter Filter function to select JSON event strings in the log data stream that * should be parsed and replayed. When not specified, all event strings in the log data * are parsed and replayed. + * @return whether it succeeds to replay the log file entirely without error including + * HaltReplayException. false otherwise. */ def replay( logData: InputStream, sourceName: String, maybeTruncated: Boolean = false, - eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Boolean = { val lines = Source.fromInputStream(logData)(Codec.UTF8).getLines() replay(lines, sourceName, maybeTruncated, eventsFilter) } @@ -66,7 +68,7 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { lines: Iterator[String], sourceName: String, maybeTruncated: Boolean, - eventsFilter: ReplayEventsFilter): Unit = { + eventsFilter: ReplayEventsFilter): Boolean = { var currentLine: String = null var lineNumber: Int = 0 val unrecognizedEvents = new scala.collection.mutable.HashSet[String] @@ -114,15 +116,18 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { } } } + true } catch { case e: HaltReplayException => // Just stop replay. - case _: EOFException if maybeTruncated => + false + case _: EOFException if maybeTruncated => false case ioe: IOException => throw ioe case e: Exception => logError(s"Exception parsing Spark event log: $sourceName", e) logError(s"Malformed line #$lineNumber: $currentLine\n") + false } } 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 5b81671edb149..9244ab407cb68 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -39,11 +39,11 @@ import org.apache.spark.TestUtils.JavaSourceFromString import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate +import org.apache.spark.deploy.history.EventLogFileReader import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.scheduler.EventLoggingListener import org.apache.spark.util.{CommandLineUtils, ResetSystemProperties, Utils} trait TestPrematureExit { @@ -535,7 +535,7 @@ class SparkSubmitSuite unusedJar.toString) runSparkSubmit(args) val listStatus = fileSystem.listStatus(testDirPath) - val logData = EventLoggingListener.openEventLog(listStatus.last.getPath, fileSystem) + val logData = EventLogFileReader.openEventLog(listStatus.last.getPath, fileSystem) Source.fromInputStream(logData).getLines().foreach { line => assert(!line.contains("secret_password")) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala new file mode 100644 index 0000000000000..a2ce4acdaaf37 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileReadersSuite.scala @@ -0,0 +1,345 @@ +/* + * 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.history + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} +import java.net.URI +import java.nio.charset.StandardCharsets +import java.util.zip.{ZipInputStream, ZipOutputStream} + +import com.google.common.io.{ByteStreams, Files} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogTestHelper._ +import org.apache.spark.deploy.history.RollingEventLogFilesWriter._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils + + +abstract class EventLogFileReadersSuite extends SparkFunSuite with LocalSparkContext + with BeforeAndAfter with Logging { + + protected val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.conf) + protected var testDir: File = _ + protected var testDirPath: Path = _ + + before { + testDir = Utils.createTempDir(namePrefix = s"event log") + testDirPath = new Path(testDir.getAbsolutePath()) + } + + after { + Utils.deleteRecursively(testDir) + } + + test("Retrieve EventLogFileReader correctly") { + def assertInstanceOfEventLogReader( + expectedClazz: Option[Class[_ <: EventLogFileReader]], + actual: Option[EventLogFileReader]): Unit = { + if (expectedClazz.isEmpty) { + assert(actual.isEmpty, s"Expected no EventLogFileReader instance but was " + + s"${actual.map(_.getClass).getOrElse("")}") + } else { + assert(actual.isDefined, s"Expected an EventLogFileReader instance but was empty") + assert(expectedClazz.get.isAssignableFrom(actual.get.getClass), + s"Expected ${expectedClazz.get} but was ${actual.get.getClass}") + } + } + + def testCreateEventLogReaderWithPath( + path: Path, + isFile: Boolean, + expectedClazz: Option[Class[_ <: EventLogFileReader]]): Unit = { + if (isFile) { + Utils.tryWithResource(fileSystem.create(path)) { is => + is.writeInt(10) + } + } else { + fileSystem.mkdirs(path) + } + + val reader = EventLogFileReader(fileSystem, path) + assertInstanceOfEventLogReader(expectedClazz, reader) + val reader2 = EventLogFileReader(fileSystem, + fileSystem.getFileStatus(path)) + assertInstanceOfEventLogReader(expectedClazz, reader2) + } + + // path with no last index - single event log + val reader1 = EventLogFileReader(fileSystem, new Path(testDirPath, "aaa"), + None) + assertInstanceOfEventLogReader(Some(classOf[SingleFileEventLogFileReader]), Some(reader1)) + + // path with last index - rolling event log + val reader2 = EventLogFileReader(fileSystem, + new Path(testDirPath, s"${EVENT_LOG_DIR_NAME_PREFIX}aaa"), Some(3)) + assertInstanceOfEventLogReader(Some(classOf[RollingEventLogFilesFileReader]), Some(reader2)) + + // path - file (both path and FileStatus) + val eventLogFile = new Path(testDirPath, "bbb") + testCreateEventLogReaderWithPath(eventLogFile, isFile = true, + Some(classOf[SingleFileEventLogFileReader])) + + // path - file starting with "." + val invalidEventLogFile = new Path(testDirPath, ".bbb") + testCreateEventLogReaderWithPath(invalidEventLogFile, isFile = true, None) + + // path - directory with "eventlog_v2_" prefix + val eventLogDir = new Path(testDirPath, s"${EVENT_LOG_DIR_NAME_PREFIX}ccc") + testCreateEventLogReaderWithPath(eventLogDir, isFile = false, + Some(classOf[RollingEventLogFilesFileReader])) + + // path - directory with no "eventlog_v2_" prefix + val invalidEventLogDir = new Path(testDirPath, "ccc") + testCreateEventLogReaderWithPath(invalidEventLogDir, isFile = false, None) + } + + val allCodecs = Seq(None) ++ + CompressionCodec.ALL_COMPRESSION_CODECS.map { c => Some(CompressionCodec.getShortName(c)) } + + allCodecs.foreach { codecShortName => + test(s"get information, list event log files, zip log files - with codec $codecShortName") { + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, codecShortName) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, hadoopConf) + writer.start() + + // The test for writing events into EventLogFileWriter is covered to its own test suite. + val dummyData = Seq("dummy1", "dummy2", "dummy3") + dummyData.foreach(writer.writeEvent(_, flushLogger = true)) + + val logPathIncompleted = getCurrentLogPath(writer.logPath, isCompleted = false) + val readerOpt = EventLogFileReader(fileSystem, new Path(logPathIncompleted)) + assertAppropriateReader(readerOpt) + val reader = readerOpt.get + + verifyReader(reader, new Path(logPathIncompleted), codecShortName, isCompleted = false) + + writer.stop() + + val logPathCompleted = getCurrentLogPath(writer.logPath, isCompleted = true) + val readerOpt2 = EventLogFileReader(fileSystem, new Path(logPathCompleted)) + assertAppropriateReader(readerOpt2) + val reader2 = readerOpt2.get + + verifyReader(reader2, new Path(logPathCompleted), codecShortName, isCompleted = true) + } + } + + protected def createWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter + + protected def getCurrentLogPath(logPath: String, isCompleted: Boolean): String + + protected def assertAppropriateReader(actualReader: Option[EventLogFileReader]): Unit + + protected def verifyReader( + reader: EventLogFileReader, + logPath: Path, + compressionCodecShortName: Option[String], + isCompleted: Boolean): Unit +} + +class SingleFileEventLogFileReaderSuite extends EventLogFileReadersSuite { + override protected def createWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + new SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + + override protected def assertAppropriateReader(actualReader: Option[EventLogFileReader]): Unit = { + assert(actualReader.isDefined, s"Expected an EventLogReader instance but was empty") + assert(actualReader.get.isInstanceOf[SingleFileEventLogFileReader], + s"Expected SingleFileEventLogReader but was ${actualReader.get.getClass}") + } + + override protected def getCurrentLogPath(logPath: String, isCompleted: Boolean): String = { + if (!isCompleted) logPath + EventLogFileWriter.IN_PROGRESS else logPath + } + + override protected def verifyReader( + reader: EventLogFileReader, + logPath: Path, + compressionCodecShortName: Option[String], + isCompleted: Boolean): Unit = { + val status = fileSystem.getFileStatus(logPath) + + assert(status.isFile) + assert(reader.rootPath === fileSystem.makeQualified(logPath)) + assert(reader.lastIndex.isEmpty) + assert(reader.fileSizeForLastIndex === status.getLen) + assert(reader.completed === isCompleted) + assert(reader.modificationTime === status.getModificationTime) + assert(reader.listEventLogFiles.length === 1) + assert(reader.listEventLogFiles.map(_.getPath.toUri.getPath) === + Seq(logPath.toUri.getPath)) + assert(reader.compressionCodec === compressionCodecShortName) + assert(reader.totalSize === status.getLen) + + val underlyingStream = new ByteArrayOutputStream() + Utils.tryWithResource(new ZipOutputStream(underlyingStream)) { os => + reader.zipEventLogFiles(os) + } + + Utils.tryWithResource(new ZipInputStream( + new ByteArrayInputStream(underlyingStream.toByteArray))) { is => + + var entry = is.getNextEntry + assert(entry != null) + val actual = new String(ByteStreams.toByteArray(is), StandardCharsets.UTF_8) + val expected = Files.toString(new File(logPath.toString), StandardCharsets.UTF_8) + assert(actual === expected) + assert(is.getNextEntry === null) + } + } +} + +class RollingEventLogFilesReaderSuite extends EventLogFileReadersSuite { + allCodecs.foreach { codecShortName => + test(s"rolling event log files - codec $codecShortName") { + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, codecShortName) + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + conf.set(EVENT_LOG_ROLLING_MAX_FILE_SIZE.key, "10m") + + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + + writer.start() + + // write log more than 20m (intended to roll over to 3 files) + val dummyStr = "dummy" * 1024 + writeTestEvents(writer, dummyStr, 1024 * 1024 * 20) + + val logPathIncompleted = getCurrentLogPath(writer.logPath, isCompleted = false) + val readerOpt = EventLogFileReader(fileSystem, + new Path(logPathIncompleted)) + verifyReader(readerOpt.get, new Path(logPathIncompleted), codecShortName, isCompleted = false) + assert(readerOpt.get.listEventLogFiles.length === 3) + + writer.stop() + + val logPathCompleted = getCurrentLogPath(writer.logPath, isCompleted = true) + val readerOpt2 = EventLogFileReader(fileSystem, new Path(logPathCompleted)) + verifyReader(readerOpt2.get, new Path(logPathCompleted), codecShortName, isCompleted = true) + assert(readerOpt2.get.listEventLogFiles.length === 3) + } + } + + override protected def createWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + new RollingEventLogFilesWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + + override protected def assertAppropriateReader(actualReader: Option[EventLogFileReader]): Unit = { + assert(actualReader.isDefined, s"Expected an EventLogReader instance but was empty") + assert(actualReader.get.isInstanceOf[RollingEventLogFilesFileReader], + s"Expected RollingEventLogFilesReader but was ${actualReader.get.getClass}") + } + + override protected def getCurrentLogPath(logPath: String, isCompleted: Boolean): String = logPath + + override protected def verifyReader( + reader: EventLogFileReader, + logPath: Path, + compressionCodecShortName: Option[String], + isCompleted: Boolean): Unit = { + import RollingEventLogFilesWriter._ + + val status = fileSystem.getFileStatus(logPath) + assert(status.isDirectory) + + val statusInDir = fileSystem.listStatus(logPath) + val eventFiles = statusInDir.filter(isEventLogFile).sortBy { s => getIndex(s.getPath.getName) } + assert(eventFiles.nonEmpty) + val lastEventFile = eventFiles.last + val allLen = eventFiles.map(_.getLen).sum + + assert(reader.rootPath === fileSystem.makeQualified(logPath)) + assert(reader.lastIndex === Some(getIndex(lastEventFile.getPath.getName))) + assert(reader.fileSizeForLastIndex === lastEventFile.getLen) + assert(reader.completed === isCompleted) + assert(reader.modificationTime === lastEventFile.getModificationTime) + assert(reader.listEventLogFiles.length === eventFiles.length) + assert(reader.listEventLogFiles.map(_.getPath) === eventFiles.map(_.getPath)) + assert(reader.compressionCodec === compressionCodecShortName) + assert(reader.totalSize === allLen) + + val underlyingStream = new ByteArrayOutputStream() + Utils.tryWithResource(new ZipOutputStream(underlyingStream)) { os => + reader.zipEventLogFiles(os) + } + + Utils.tryWithResource(new ZipInputStream( + new ByteArrayInputStream(underlyingStream.toByteArray))) { is => + + val entry = is.getNextEntry + assert(entry != null) + + // directory + assert(entry.getName === logPath.getName + "/") + + val allFileNames = fileSystem.listStatus(logPath).map(_.getPath.getName).toSet + + var count = 0 + var noMoreEntry = false + while (!noMoreEntry) { + val entry = is.getNextEntry + if (entry == null) { + noMoreEntry = true + } else { + count += 1 + + assert(entry.getName.startsWith(logPath.getName + "/")) + val fileName = entry.getName.stripPrefix(logPath.getName + "/") + assert(allFileNames.contains(fileName)) + + val actual = new String(ByteStreams.toByteArray(is), StandardCharsets.UTF_8) + val expected = Files.toString(new File(logPath.toString, fileName), + StandardCharsets.UTF_8) + assert(actual === expected) + } + } + + assert(count === allFileNames.size) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala new file mode 100644 index 0000000000000..c4b40884eebf5 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala @@ -0,0 +1,378 @@ +/* + * 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.history + +import java.io.{File, FileOutputStream, IOException} +import java.net.URI + +import scala.collection.mutable +import scala.io.Source + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogTestHelper._ +import org.apache.spark.internal.config._ +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils + + +abstract class EventLogFileWritersSuite extends SparkFunSuite with LocalSparkContext + with BeforeAndAfter { + + protected val fileSystem = Utils.getHadoopFileSystem("/", + SparkHadoopUtil.get.newConfiguration(new SparkConf())) + protected var testDir: File = _ + protected var testDirPath: Path = _ + + before { + testDir = Utils.createTempDir(namePrefix = s"event log") + testDirPath = new Path(testDir.getAbsolutePath()) + } + + after { + Utils.deleteRecursively(testDir) + } + + test("create EventLogFileWriter with enable/disable rolling") { + def buildWriterAndVerify(conf: SparkConf, expectedClazz: Class[_]): Unit = { + val writer = EventLogFileWriter( + getUniqueApplicationId, None, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + val writerClazz = writer.getClass + assert(expectedClazz === writerClazz) + } + + val conf = new SparkConf + conf.set(EVENT_LOG_ENABLED, true) + conf.set(EVENT_LOG_DIR, testDir.toString) + + // default config + buildWriterAndVerify(conf, classOf[SingleEventLogFileWriter]) + + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + buildWriterAndVerify(conf, classOf[RollingEventLogFilesWriter]) + + conf.set(EVENT_LOG_ENABLE_ROLLING, false) + buildWriterAndVerify(conf, classOf[SingleEventLogFileWriter]) + } + + val allCodecs = Seq(None) ++ + CompressionCodec.ALL_COMPRESSION_CODECS.map(c => Some(CompressionCodec.getShortName(c))) + + allCodecs.foreach { codecShortName => + test(s"initialize, write, stop - with codec $codecShortName") { + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, codecShortName) + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + + writer.start() + + // snappy stream throws exception on empty stream, so we should provide some data to test. + val dummyData = Seq("dummy1", "dummy2", "dummy3") + dummyData.foreach(writer.writeEvent(_, flushLogger = true)) + + writer.stop() + + verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, codecShortName, dummyData) + } + } + + test("spark.eventLog.compression.codec overrides spark.io.compression.codec") { + val conf = new SparkConf + conf.set(EVENT_LOG_COMPRESS, true) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + + val appId = "test" + val appAttemptId = None + + // The default value is `spark.io.compression.codec`. + val writer = createWriter(appId, appAttemptId, testDirPath.toUri, conf, hadoopConf) + assert(writer.compressionCodecName.contains("lz4")) + + // `spark.eventLog.compression.codec` overrides `spark.io.compression.codec`. + conf.set(EVENT_LOG_COMPRESSION_CODEC, "zstd") + val writer2 = createWriter(appId, appAttemptId, testDirPath.toUri, conf, hadoopConf) + assert(writer2.compressionCodecName.contains("zstd")) + } + + protected def readLinesFromEventLogFile(log: Path, fs: FileSystem): List[String] = { + val logDataStream = EventLogFileReader.openEventLog(log, fs) + try { + Source.fromInputStream(logDataStream).getLines().toList + } finally { + logDataStream.close() + } + } + + protected def createWriter( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter + + /** + * This should be called with "closed" event log file; No guarantee on reading event log file + * which is being written, especially the file is compressed. SHS also does the best it can. + */ + protected def verifyWriteEventLogFile( + appId: String, + appAttemptId : Option[String], + logBaseDir: URI, + compressionCodecShortName: Option[String], + expectedLines: Seq[String] = Seq.empty): Unit +} + +class SingleEventLogFileWriterSuite extends EventLogFileWritersSuite { + + test("Log overwriting") { + val appId = "test" + val appAttemptId = None + val logUri = SingleEventLogFileWriter.getLogPath(testDir.toURI, appId, appAttemptId) + + val conf = getLoggingConf(testDirPath) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val writer = createWriter(appId, appAttemptId, testDir.toURI, conf, hadoopConf) + + val logPath = new Path(logUri).toUri.getPath + writer.start() + + val dummyData = Seq("dummy1", "dummy2", "dummy3") + dummyData.foreach(writer.writeEvent(_, flushLogger = true)) + + // Create file before writing the event log + new FileOutputStream(new File(logPath)).close() + // Expected IOException, since we haven't enabled log overwrite. + intercept[IOException] { writer.stop() } + + // Try again, but enable overwriting. + conf.set(EVENT_LOG_OVERWRITE, true) + val writer2 = createWriter(appId, appAttemptId, testDir.toURI, conf, hadoopConf) + writer2.start() + dummyData.foreach(writer2.writeEvent(_, flushLogger = true)) + writer2.stop() + } + + test("Event log name") { + val baseDirUri = Utils.resolveURI("/base-dir") + // without compression + assert(s"${baseDirUri.toString}/app1" === SingleEventLogFileWriter.getLogPath( + baseDirUri, "app1", None, None)) + // with compression + assert(s"${baseDirUri.toString}/app1.lzf" === + SingleEventLogFileWriter.getLogPath(baseDirUri, "app1", None, Some("lzf"))) + // illegal characters in app ID + assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1" === + SingleEventLogFileWriter.getLogPath(baseDirUri, + "a fine:mind$dollar{bills}.1", None, None)) + // illegal characters in app ID with compression + assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1.lz4" === + SingleEventLogFileWriter.getLogPath(baseDirUri, + "a fine:mind$dollar{bills}.1", None, Some("lz4"))) + } + + override protected def createWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + new SingleEventLogFileWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + + override protected def verifyWriteEventLogFile( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + compressionCodecShortName: Option[String], + expectedLines: Seq[String]): Unit = { + // read single event log file + val logPath = SingleEventLogFileWriter.getLogPath(logBaseDir, appId, appAttemptId, + compressionCodecShortName) + + val finalLogPath = new Path(logPath) + assert(fileSystem.exists(finalLogPath) && fileSystem.isFile(finalLogPath)) + assert(expectedLines === readLinesFromEventLogFile(finalLogPath, fileSystem)) + } +} + +class RollingEventLogFilesWriterSuite extends EventLogFileWritersSuite { + import RollingEventLogFilesWriter._ + + test("Event log names") { + val baseDirUri = Utils.resolveURI("/base-dir") + val appId = "app1" + val appAttemptId = None + + // happy case with app ID + val logDir = RollingEventLogFilesWriter.getAppEventLogDirPath(baseDirUri, appId, None) + assert(s"${baseDirUri.toString}/${EVENT_LOG_DIR_NAME_PREFIX}${appId}" === logDir.toString) + + // appstatus: inprogress or completed + assert(s"$logDir/${APPSTATUS_FILE_NAME_PREFIX}${appId}${EventLogFileWriter.IN_PROGRESS}" === + RollingEventLogFilesWriter.getAppStatusFilePath(logDir, appId, appAttemptId, + inProgress = true).toString) + assert(s"$logDir/${APPSTATUS_FILE_NAME_PREFIX}${appId}" === + RollingEventLogFilesWriter.getAppStatusFilePath(logDir, appId, appAttemptId, + inProgress = false).toString) + + // without compression + assert(s"$logDir/${EVENT_LOG_FILE_NAME_PREFIX}1_${appId}" === + RollingEventLogFilesWriter.getEventLogFilePath(logDir, appId, appAttemptId, 1, None).toString) + + // with compression + assert(s"$logDir/${EVENT_LOG_FILE_NAME_PREFIX}1_${appId}.lzf" === + RollingEventLogFilesWriter.getEventLogFilePath(logDir, appId, appAttemptId, + 1, Some("lzf")).toString) + + // illegal characters in app ID + assert(s"${baseDirUri.toString}/${EVENT_LOG_DIR_NAME_PREFIX}a-fine-mind_dollar_bills__1" === + RollingEventLogFilesWriter.getAppEventLogDirPath(baseDirUri, + "a fine:mind$dollar{bills}.1", None).toString) + } + + test("Log overwriting") { + val appId = "test" + val appAttemptId = None + val logDirPath = RollingEventLogFilesWriter.getAppEventLogDirPath(testDir.toURI, appId, + appAttemptId) + + val conf = getLoggingConf(testDirPath) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val writer = createWriter(appId, appAttemptId, testDir.toURI, conf, hadoopConf) + + val logPath = logDirPath.toUri.getPath + + // Create file before writing the event log directory + // it doesn't matter whether the existing one is file or directory + new FileOutputStream(new File(logPath)).close() + + // Expected IOException, since we haven't enabled log overwrite. + // Note that the place IOException is thrown is different from single event log file. + intercept[IOException] { writer.start() } + + // Try again, but enable overwriting. + conf.set(EVENT_LOG_OVERWRITE, true) + + val writer2 = createWriter(appId, appAttemptId, testDir.toURI, conf, hadoopConf) + writer2.start() + val dummyData = Seq("dummy1", "dummy2", "dummy3") + dummyData.foreach(writer2.writeEvent(_, flushLogger = true)) + writer2.stop() + } + + allCodecs.foreach { codecShortName => + test(s"rolling event log files - codec $codecShortName") { + def assertEventLogFilesIndex( + eventLogFiles: Seq[FileStatus], + expectedLastIndex: Int, + expectedMaxSizeBytes: Long): Unit = { + assert(eventLogFiles.forall(f => f.getLen <= expectedMaxSizeBytes)) + assert((1 to expectedLastIndex) === + eventLogFiles.map(f => getIndex(f.getPath.getName))) + } + + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, codecShortName) + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + conf.set(EVENT_LOG_ROLLING_MAX_FILE_SIZE.key, "10m") + + val writer = createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + + writer.start() + + // write log more than 20m (intended to roll over to 3 files) + val dummyStr = "dummy" * 1024 + val expectedLines = writeTestEvents(writer, dummyStr, 1024 * 1024 * 21) + + val logDirPath = getAppEventLogDirPath(testDirPath.toUri, appId, attemptId) + + val eventLogFiles = listEventLogFiles(logDirPath) + assertEventLogFilesIndex(eventLogFiles, 3, 1024 * 1024 * 10) + + writer.stop() + + val eventLogFiles2 = listEventLogFiles(logDirPath) + assertEventLogFilesIndex(eventLogFiles2, 3, 1024 * 1024 * 10) + + verifyWriteEventLogFile(appId, attemptId, testDirPath.toUri, + codecShortName, expectedLines) + } + } + + test(s"rolling event log files - the max size of event log file size less than lower limit") { + val appId = getUniqueApplicationId + val attemptId = None + + val conf = getLoggingConf(testDirPath, None) + conf.set(EVENT_LOG_ENABLE_ROLLING, true) + conf.set(EVENT_LOG_ROLLING_MAX_FILE_SIZE.key, "9m") + + val e = intercept[IllegalArgumentException] { + createWriter(appId, attemptId, testDirPath.toUri, conf, + SparkHadoopUtil.get.newConfiguration(conf)) + } + assert(e.getMessage.contains("should be configured to be at least")) + } + + override protected def createWriter( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + sparkConf: SparkConf, + hadoopConf: Configuration): EventLogFileWriter = { + new RollingEventLogFilesWriter(appId, appAttemptId, logBaseDir, sparkConf, hadoopConf) + } + + override protected def verifyWriteEventLogFile( + appId: String, + appAttemptId: Option[String], + logBaseDir: URI, + compressionCodecShortName: Option[String], + expectedLines: Seq[String]): Unit = { + val logDirPath = getAppEventLogDirPath(logBaseDir, appId, appAttemptId) + + assert(fileSystem.exists(logDirPath) && fileSystem.isDirectory(logDirPath)) + + val appStatusFile = getAppStatusFilePath(logDirPath, appId, appAttemptId, inProgress = false) + assert(fileSystem.exists(appStatusFile) && fileSystem.isFile(appStatusFile)) + + val eventLogFiles = listEventLogFiles(logDirPath) + val allLines = mutable.ArrayBuffer[String]() + eventLogFiles.foreach { file => + allLines.appendAll(readLinesFromEventLogFile(file.getPath, fileSystem)) + } + + assert(expectedLines === allLines) + } + + private def listEventLogFiles(logDirPath: Path): Seq[FileStatus] = { + fileSystem.listStatus(logDirPath).filter(isEventLogFile) + .sortBy { fs => getIndex(fs.getPath.getName) } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala new file mode 100644 index 0000000000000..55eddce3968c2 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala @@ -0,0 +1,59 @@ +/* + * 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.history + +import java.nio.charset.StandardCharsets + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config._ + +object EventLogTestHelper { + def getUniqueApplicationId: String = "test-" + System.currentTimeMillis + + /** + * Get a SparkConf with event logging enabled. It doesn't enable rolling event logs, so caller + * should set it manually. + */ + def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { + val conf = new SparkConf + conf.set(EVENT_LOG_ENABLED, true) + conf.set(EVENT_LOG_BLOCK_UPDATES, true) + conf.set(EVENT_LOG_TESTING, true) + conf.set(EVENT_LOG_DIR, logDir.toString) + compressionCodec.foreach { codec => + conf.set(EVENT_LOG_COMPRESS, true) + conf.set(EVENT_LOG_COMPRESSION_CODEC, codec) + } + conf.set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) + conf + } + + def writeTestEvents( + writer: EventLogFileWriter, + eventStr: String, + desiredSize: Long): Seq[String] = { + val stringLen = eventStr.getBytes(StandardCharsets.UTF_8).length + val repeatCount = Math.floor(desiredSize / stringLen).toInt + (0 until repeatCount).map { _ => + writer.writeEvent(eventStr, flushLogger = true) + eventStr + } + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 1d465ba37364b..281e6935de375 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -47,6 +47,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.KVUtils.KVStoreScalaSerializer import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils} import org.apache.spark.util.logging.DriverLogger @@ -74,8 +75,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { appAttemptId: Option[String], inProgress: Boolean, codec: Option[String] = None): File = { - val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" - val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId, appAttemptId, codec) + val ip = if (inProgress) EventLogFileWriter.IN_PROGRESS else "" + val logUri = SingleEventLogFileWriter.getLogPath(testDir.toURI, appId, appAttemptId, codec) val logPath = new Path(logUri).toUri.getPath + ip new File(logPath) } @@ -161,10 +162,10 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { class TestFsHistoryProvider extends FsHistoryProvider(createTestConf()) { var mergeApplicationListingCall = 0 override protected def mergeApplicationListing( - fileStatus: FileStatus, + reader: EventLogFileReader, lastSeen: Long, enableSkipToEnd: Boolean): Unit = { - super.mergeApplicationListing(fileStatus, lastSeen, enableSkipToEnd) + super.mergeApplicationListing(reader, lastSeen, enableSkipToEnd) mergeApplicationListingCall += 1 } } @@ -199,13 +200,13 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { ) updateAndCheck(provider) { list => list.size should be (1) - provider.getAttempt("app1", None).logPath should endWith(EventLoggingListener.IN_PROGRESS) + provider.getAttempt("app1", None).logPath should endWith(EventLogFileWriter.IN_PROGRESS) } logFile1.renameTo(newLogFile("app1", None, inProgress = false)) updateAndCheck(provider) { list => list.size should be (1) - provider.getAttempt("app1", None).logPath should not endWith(EventLoggingListener.IN_PROGRESS) + provider.getAttempt("app1", None).logPath should not endWith(EventLogFileWriter.IN_PROGRESS) } } @@ -1161,29 +1162,45 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { when(mockedFs.open(path)).thenReturn(in) when(in.getWrappedStream).thenReturn(dfsIn) when(dfsIn.getFileLength).thenReturn(200) + // FileStatus.getLen is more than logInfo fileSize var fileStatus = new FileStatus(200, false, 0, 0, 0, path) + when(mockedFs.getFileStatus(path)).thenReturn(fileStatus) var logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100) - assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) + Some("attemptId"), 100, None, false) + var reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(mockedProvider.shouldReloadLog(logInfo, reader.get)) fileStatus = new FileStatus() fileStatus.setPath(path) + when(mockedFs.getFileStatus(path)).thenReturn(fileStatus) // DFSInputStream.getFileLength is more than logInfo fileSize logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100) - assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) + Some("attemptId"), 100, None, false) + reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(mockedProvider.shouldReloadLog(logInfo, reader.get)) + // DFSInputStream.getFileLength is equal to logInfo fileSize logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 200) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + Some("attemptId"), 200, None, false) + reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) + // in.getWrappedStream returns other than DFSInputStream val bin = mock(classOf[BufferedInputStream]) when(in.getWrappedStream).thenReturn(bin) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) + // fs.open throws exception when(mockedFs.open(path)).thenThrow(new IOException("Throwing intentionally")) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + reader = EventLogFileReader(mockedFs, path) + assert(reader.isDefined) + assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) } test("log cleaner with the maximum number of log files") { @@ -1236,6 +1253,36 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } + test("backwards compatibility with LogInfo from Spark 2.4") { + case class LogInfoV24( + logPath: String, + lastProcessed: Long, + appId: Option[String], + attemptId: Option[String], + fileSize: Long) + + val oldObj = LogInfoV24("dummy", System.currentTimeMillis(), Some("hello"), + Some("attempt1"), 100) + + val serializer = new KVStoreScalaSerializer() + val serializedOldObj = serializer.serialize(oldObj) + val deserializedOldObj = serializer.deserialize(serializedOldObj, classOf[LogInfo]) + assert(deserializedOldObj.logPath === oldObj.logPath) + assert(deserializedOldObj.lastProcessed === oldObj.lastProcessed) + assert(deserializedOldObj.appId === oldObj.appId) + assert(deserializedOldObj.attemptId === oldObj.attemptId) + assert(deserializedOldObj.fileSize === oldObj.fileSize) + + // SPARK-25118: added logType: LogType.Value - expected 'null' on old format + assert(deserializedOldObj.logType === null) + + // SPARK-28869: added lastIndex: Option[Long], isComplete: Boolean - expected 'None' and + // 'false' on old format. The default value for isComplete is wrong value for completed app, + // but the value will be corrected once checkForLogs is called. + assert(deserializedOldObj.lastIndex === None) + assert(deserializedOldObj.isComplete === false) + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: @@ -1256,7 +1303,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { val fstream = new FileOutputStream(file) val cstream = codec.map(_.compressedContinuousOutputStream(fstream)).getOrElse(fstream) val bstream = new BufferedOutputStream(cstream) - EventLoggingListener.initEventLog(bstream, false, null) + + val metadata = SparkListenerLogStart(org.apache.spark.SPARK_VERSION) + val eventJson = JsonProtocol.logStartToJson(metadata) + val metadataJson = compact(eventJson) + "\n" + bstream.write(metadataJson.getBytes(StandardCharsets.UTF_8)) val writer = new OutputStreamWriter(bstream, StandardCharsets.UTF_8) Utils.tryWithSafeFinally { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index ae55d1915fa4a..286924001e920 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.io.{File, FileOutputStream, InputStream, IOException} +import java.io.{File, InputStream} import java.util.Arrays import scala.collection.immutable.Map @@ -32,15 +32,15 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.{EventLogFileReader, SingleEventLogFileWriter} +import org.apache.spark.deploy.history.EventLogTestHelper._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ import org.apache.spark.io._ import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} - /** * Test whether EventLoggingListener logs events properly. * @@ -51,8 +51,6 @@ import org.apache.spark.util.{JsonProtocol, Utils} class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter with Logging { - import EventLoggingListenerSuite._ - private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) private var testDir: File = _ @@ -68,40 +66,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit Utils.deleteRecursively(testDir) } - test("Verify log file exist") { - // Verify logging directory exists - val conf = getLoggingConf(testDirPath) - val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf) - eventLogger.start() - - val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) - assert(fileSystem.exists(logPath)) - val logStatus = fileSystem.getFileStatus(logPath) - assert(!logStatus.isDirectory) - - // Verify log is renamed after stop() - eventLogger.stop() - assert(!fileSystem.getFileStatus(new Path(eventLogger.logPath)).isDirectory) - } - - test("Basic event logging") { - testEventLogging() - } - - test("spark.eventLog.compression.codec overrides spark.io.compression.codec") { - val conf = new SparkConf - conf.set(EVENT_LOG_COMPRESS, true) - - // The default value is `spark.io.compression.codec`. - val e = new EventLoggingListener("test", None, testDirPath.toUri(), conf) - assert(e.compressionCodecName.contains("lz4")) - - // `spark.eventLog.compression.codec` overrides `spark.io.compression.codec`. - conf.set(EVENT_LOG_COMPRESSION_CODEC, "zstd") - val e2 = new EventLoggingListener("test", None, testDirPath.toUri(), conf) - assert(e2.compressionCodecName.contains("zstd")) - } - test("Basic event logging with compression") { CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) @@ -131,35 +95,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(redactedProps(key) == "*********(redacted)") } - test("Log overwriting") { - val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", None) - val logPath = new Path(logUri).toUri.getPath - // Create file before writing the event log - new FileOutputStream(new File(logPath)).close() - // Expected IOException, since we haven't enabled log overwrite. - intercept[IOException] { testEventLogging() } - // Try again, but enable overwriting. - testEventLogging(extraConf = Map(EVENT_LOG_OVERWRITE.key -> "true")) - } - - test("Event log name") { - val baseDirUri = Utils.resolveURI("/base-dir") - // without compression - assert(s"${baseDirUri.toString}/app1" === EventLoggingListener.getLogPath( - baseDirUri, "app1", None)) - // with compression - assert(s"${baseDirUri.toString}/app1.lzf" === - EventLoggingListener.getLogPath(baseDirUri, "app1", None, Some("lzf"))) - // illegal characters in app ID - assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1" === - EventLoggingListener.getLogPath(baseDirUri, - "a fine:mind$dollar{bills}.1", None)) - // illegal characters in app ID with compression - assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1.lz4" === - EventLoggingListener.getLogPath(baseDirUri, - "a fine:mind$dollar{bills}.1", None, Some("lz4"))) - } - test("Executor metrics update") { testStageExecutorMetricsEventLogging() } @@ -168,8 +103,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit * Actual test logic * * ----------------- */ - import EventLoggingListenerSuite._ - /** * Test basic event logging functionality. * @@ -198,7 +131,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit eventLogger.stop() // Verify file contains exactly the two events logged - val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val logPath = eventLogger.logWriter.logPath + val logData = EventLogFileReader.openEventLog(new Path(logPath), fileSystem) try { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) @@ -226,9 +160,10 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit sc = new SparkContext("local-cluster[2,2,1024]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get - val eventLogPath = eventLogger.logPath + + val eventLogPath = eventLogger.logWriter.logPath val expectedLogDir = testDir.toURI() - assert(eventLogPath === EventLoggingListener.getLogPath( + assert(eventLogPath === SingleEventLogFileWriter.getLogPath( expectedLogDir, sc.applicationId, None, compressionCodec.map(CompressionCodec.getShortName))) // Begin listening for events that trigger asserts @@ -243,7 +178,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit eventExistenceListener.assertAllCallbacksInvoked() // Make sure expected events exist in the log file. - val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val logData = EventLogFileReader.openEventLog(new Path(eventLogger.logWriter.logPath), + fileSystem) val eventSet = mutable.Set( SparkListenerApplicationStart, SparkListenerBlockManagerAdded, @@ -466,7 +402,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // Verify the log file contains the expected events. // Posted events should be logged, except for ExecutorMetricsUpdate events -- these // are consolidated, and the peak values for each stage are logged at stage end. - val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val logData = EventLogFileReader.openEventLog(new Path(eventLogger.logWriter.logPath), + fileSystem) try { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) @@ -641,24 +578,3 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } - - -object EventLoggingListenerSuite { - - /** Get a SparkConf with event logging enabled. */ - def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { - val conf = new SparkConf - conf.set(EVENT_LOG_ENABLED, true) - conf.set(EVENT_LOG_BLOCK_UPDATES, true) - conf.set(EVENT_LOG_TESTING, true) - conf.set(EVENT_LOG_DIR, logDir.toString) - compressionCodec.foreach { codec => - conf.set(EVENT_LOG_COMPRESS, true) - conf.set(EVENT_LOG_COMPRESSION_CODEC, codec) - } - conf.set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) - conf - } - - def getUniqueApplicationId: String = "test-" + System.currentTimeMillis -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 55e7f5333c676..e6fbf9b09d43d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io._ -import java.net.URI import java.nio.charset.StandardCharsets import java.util.concurrent.atomic.AtomicInteger @@ -31,6 +30,8 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogFileReader +import org.apache.spark.deploy.history.EventLogTestHelper._ import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec} import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils} @@ -64,7 +65,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp // scalastyle:on println } - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val conf = getLoggingConf(logFilePath) val logData = fileSystem.open(logFilePath) val eventMonster = new EventBufferingListener try { @@ -110,14 +111,14 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp } // Read the compressed .inprogress file and verify only first event was parsed. - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val conf = getLoggingConf(logFilePath) val replayer = new ReplayListenerBus() val eventMonster = new EventBufferingListener replayer.addListener(eventMonster) // Verify the replay returns the events given the input maybe truncated. - val logData = EventLoggingListener.openEventLog(logFilePath, fileSystem) + val logData = EventLogFileReader.openEventLog(logFilePath, fileSystem) Utils.tryWithResource(new EarlyEOFInputStream(logData, buffered.size - 10)) { failingStream => replayer.replay(failingStream, logFilePath.toString, true) @@ -126,7 +127,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp } // Verify the replay throws the EOF exception since the input may not be truncated. - val logData2 = EventLoggingListener.openEventLog(logFilePath, fileSystem) + val logData2 = EventLogFileReader.openEventLog(logFilePath, fileSystem) Utils.tryWithResource(new EarlyEOFInputStream(logData2, buffered.size - 10)) { failingStream2 => intercept[EOFException] { replayer.replay(failingStream2, logFilePath.toString, false) @@ -149,7 +150,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp // scalastyle:on println } - val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val conf = getLoggingConf(logFilePath) val logData = fileSystem.open(logFilePath) val eventMonster = new EventBufferingListener try { @@ -195,7 +196,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp val logDirPath = new Path(logDir.toURI) fileSystem.mkdirs(logDirPath) - val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) + val conf = getLoggingConf(logDirPath, codecName) sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf) // Run a few jobs @@ -212,7 +213,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp assert(!eventLog.isDirectory) // Replay events - val logData = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) + val logData = EventLogFileReader.openEventLog(eventLog.getPath(), fileSystem) val eventMonster = new EventBufferingListener try { val replayer = new ReplayListenerBus() diff --git a/docs/configuration.md b/docs/configuration.md index 0bbfc2f9b806a..7fdcf4a958d9f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1008,6 +1008,21 @@ Apart from these, the following properties are also available, and may be useful Buffer size to use when writing to output streams, in KiB unless otherwise specified. + + spark.eventLog.rolling.enabled + false + + Whether rolling over event log files is enabled. If set to true, it cuts down each event + log file to the configured size. + + + + spark.eventLog.rolling.maxFileSize + 128m + + The max size of event log file before it's rolled over. + + spark.ui.dagGraph.retainedRootRDDs Int.MaxValue From 78bdcfade1898ecbde3db4542cac6b29f88eb26c Mon Sep 17 00:00:00 2001 From: "igor.calabria" Date: Thu, 17 Oct 2019 12:23:24 -0700 Subject: [PATCH 0352/1387] [SPARK-27812][K8S] Bump K8S client version to 4.6.1 ### What changes were proposed in this pull request? Updated kubernetes client. ### Why are the changes needed? https://issues.apache.org/jira/browse/SPARK-27812 https://issues.apache.org/jira/browse/SPARK-27927 We need this fix https://github.com/fabric8io/kubernetes-client/pull/1768 that was released on version 4.6 of the client. The root cause of the problem is better explained in https://github.com/apache/spark/pull/25785 ### Does this PR introduce any user-facing change? Nope, it should be transparent to users ### How was this patch tested? This patch was tested manually using a simple pyspark job ```python from pyspark.sql import SparkSession if __name__ == '__main__': spark = SparkSession.builder.getOrCreate() ``` The expected behaviour of this "job" is that both python's and jvm's process exit automatically after the main runs. This is the case for spark versions <= 2.4. On version 2.4.3, the jvm process hangs because there's a non daemon thread running ``` "OkHttp WebSocket https://10.96.0.1/..." #121 prio=5 os_prio=0 tid=0x00007fb27c005800 nid=0x24b waiting on condition [0x00007fb300847000] "OkHttp WebSocket https://10.96.0.1/..." #117 prio=5 os_prio=0 tid=0x00007fb28c004000 nid=0x247 waiting on condition [0x00007fb300e4b000] ``` This is caused by a bug on `kubernetes-client` library, which is fixed on the version that we are upgrading to. When the mentioned job is run with this patch applied, the behaviour from spark <= 2.4.3 is restored and both processes terminate successfully Closes #26093 from igorcalabria/k8s-client-update. Authored-by: igor.calabria Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 11 +++++------ dev/deps/spark-deps-hadoop-3.2 | 11 +++++------ resource-managers/kubernetes/core/pom.xml | 8 +------- .../kubernetes/integration-tests/pom.xml | 2 +- 4 files changed, 12 insertions(+), 20 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 81e32f13155f1..26dc6e7bd8bf9 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -107,7 +107,6 @@ jakarta.ws.rs-api-2.1.5.jar jakarta.xml.bind-api-2.3.2.jar janino-3.0.15.jar javassist-3.22.0-CR2.jar -javax.el-3.0.1-b11.jar javax.inject-1.jar javax.servlet-api-3.1.0.jar javolution-5.5.1.jar @@ -138,9 +137,9 @@ jsr305-3.0.0.jar jta-1.1.jar jul-to-slf4j-1.7.16.jar kryo-shaded-4.0.2.jar -kubernetes-client-4.4.2.jar -kubernetes-model-4.4.2.jar -kubernetes-model-common-4.4.2.jar +kubernetes-client-4.6.1.jar +kubernetes-model-4.6.1.jar +kubernetes-model-common-4.6.1.jar leveldbjni-all-1.8.jar libfb303-0.9.3.jar libthrift-0.12.0.jar @@ -158,8 +157,8 @@ minlog-1.3.0.jar netty-all-4.1.42.Final.jar objenesis-2.5.1.jar okapi-shade-0.4.2.jar -okhttp-3.8.1.jar -okio-1.13.0.jar +okhttp-3.12.0.jar +okio-1.15.0.jar opencsv-2.3.jar orc-core-1.5.6-nohive.jar orc-mapreduce-1.5.6-nohive.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index e28418d8bee1f..a92b7124cb4af 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -120,7 +120,6 @@ jakarta.ws.rs-api-2.1.5.jar jakarta.xml.bind-api-2.3.2.jar janino-3.0.15.jar javassist-3.22.0-CR2.jar -javax.el-3.0.1-b11.jar javax.inject-1.jar javax.jdo-3.2.0-m3.jar javax.servlet-api-3.1.0.jar @@ -168,9 +167,9 @@ kerby-pkix-1.0.1.jar kerby-util-1.0.1.jar kerby-xdr-1.0.1.jar kryo-shaded-4.0.2.jar -kubernetes-client-4.4.2.jar -kubernetes-model-4.4.2.jar -kubernetes-model-common-4.4.2.jar +kubernetes-client-4.6.1.jar +kubernetes-model-4.6.1.jar +kubernetes-model-common-4.6.1.jar leveldbjni-all-1.8.jar libfb303-0.9.3.jar libthrift-0.12.0.jar @@ -191,8 +190,8 @@ nimbus-jose-jwt-4.41.1.jar objenesis-2.5.1.jar okapi-shade-0.4.2.jar okhttp-2.7.5.jar -okhttp-3.8.1.jar -okio-1.13.0.jar +okhttp-3.12.0.jar +okio-1.15.0.jar opencsv-2.3.jar orc-core-1.5.6.jar orc-mapreduce-1.5.6.jar diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 952dbd31989dd..487876b12ce91 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -29,7 +29,7 @@ Spark Project Kubernetes kubernetes - 4.4.2 + 4.6.1 @@ -88,12 +88,6 @@ - - com.squareup.okhttp3 - okhttp - 3.8.1 - - org.mockito mockito-core diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 84d99115a93ac..c79caff164ce8 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -29,7 +29,7 @@ 1.3.0 1.4.0 - 4.4.2 + 4.6.1 3.2.2 1.0 kubernetes-integration-tests From 00347a3c782634e37a528cea7915dcfdcc9494f7 Mon Sep 17 00:00:00 2001 From: Ivan Gozali Date: Thu, 17 Oct 2019 14:36:01 -0700 Subject: [PATCH 0353/1387] [SPARK-28762][CORE] Read JAR main class if JAR is not located in local file system ### What changes were proposed in this pull request? JIRA: https://issues.apache.org/jira/browse/SPARK-28762 TL;DR: Automatically read the `Main-Class` from a JAR's manifest even if the JAR isn't in the local file system (i.e. in S3 or HDFS). ### Why are the changes needed? When deploying a fat JAR (e.g. using `sbt-assembly`) to S3/HDFS, users might choose to include the main class for the JAR in its manifest. This change allows the user to `spark-submit` the JAR without having to specify the main class again via the `--class` argument. ### Does this PR introduce any user-facing change? Yes. Previously, if the primary resource is a JAR and isn't located in the local file system, it will fail with the error: ``` $ spark-submit s3a://nonexistent.jar Exception in thread "main" org.apache.spark.SparkException: Cannot load main class from JAR s3a://nonexistent.jar with URI s3a. Please specify a class through --class. ... ``` With this PR, the main class will be read from the manifest, assuming the classpath contains the appropriate JAR to read the file system. ### How was this patch tested? Added some tests in `core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala`. Closes #25910 from igozali/SPARK-28762. Authored-by: Ivan Gozali Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/TestUtils.scala | 20 ++++- .../org/apache/spark/deploy/SparkSubmit.scala | 29 ++++++- .../spark/deploy/SparkSubmitArguments.scala | 28 ------- .../spark/deploy/SparkSubmitSuite.scala | 77 +++++++++++++++++++ 4 files changed, 122 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index b8c094dbea961..5d79394dde91b 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -26,7 +26,7 @@ import java.security.SecureRandom import java.security.cert.X509Certificate import java.util.{Arrays, EnumSet, Properties} import java.util.concurrent.{TimeoutException, TimeUnit} -import java.util.jar.{JarEntry, JarOutputStream} +import java.util.jar.{JarEntry, JarOutputStream, Manifest} import javax.net.ssl._ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} @@ -97,9 +97,23 @@ private[spark] object TestUtils { * Create a jar file that contains this set of files. All files will be located in the specified * directory or at the root of the jar. */ - def createJar(files: Seq[File], jarFile: File, directoryPrefix: Option[String] = None): URL = { + def createJar( + files: Seq[File], + jarFile: File, + directoryPrefix: Option[String] = None, + mainClass: Option[String] = None): URL = { + val manifest = mainClass match { + case Some(mc) => + val m = new Manifest() + m.getMainAttributes.putValue("Manifest-Version", "1.0") + m.getMainAttributes.putValue("Main-Class", mc) + m + case None => + new Manifest() + } + val jarFileStream = new FileOutputStream(jarFile) - val jarStream = new JarOutputStream(jarFileStream, new java.util.jar.Manifest()) + val jarStream = new JarOutputStream(jarFileStream, manifest) for (file <- files) { // The `name` for the argument in `JarEntry` should use / for its separator. This is 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 b776ec8f81e06..8a03af5e38c9b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -23,11 +23,12 @@ import java.net.{URI, URL} import java.security.PrivilegedExceptionAction import java.text.ParseException import java.util.{ServiceLoader, UUID} +import java.util.jar.JarInputStream import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.util.{Properties, Try} +import scala.util.{Failure, Properties, Success, Try} import org.apache.commons.io.FilenameUtils import org.apache.commons.lang3.StringUtils @@ -437,6 +438,32 @@ private[spark] class SparkSubmit extends Logging { }.orNull } + // At this point, we have attempted to download all remote resources. + // Now we try to resolve the main class if our primary resource is a JAR. + if (args.mainClass == null && !args.isPython && !args.isR) { + try { + val uri = new URI( + Option(localPrimaryResource).getOrElse(args.primaryResource) + ) + val fs = FileSystem.get(uri, hadoopConf) + + Utils.tryWithResource(new JarInputStream(fs.open(new Path(uri)))) { jar => + args.mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + } + } catch { + case e: Throwable => + error( + s"Failed to get main class in JAR with error '${e.getMessage}'. " + + " Please specify one with --class." + ) + } + + if (args.mainClass == null) { + // If we still can't figure out the main class at this point, blow up. + error("No main class set in JAR; please specify one with --class.") + } + } + // If we're running a python app, set the main class to our specific python runner if (args.isPython && deployMode == CLIENT) { if (args.primaryResource == PYSPARK_SHELL) { 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 ed1324baed0f1..f1ed4f2d62d70 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -19,10 +19,8 @@ package org.apache.spark.deploy import java.io.{ByteArrayOutputStream, File, PrintStream} import java.lang.reflect.InvocationTargetException -import java.net.URI import java.nio.charset.StandardCharsets import java.util.{List => JList} -import java.util.jar.JarFile import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -211,29 +209,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S dynamicAllocationEnabled = sparkProperties.get(DYN_ALLOCATION_ENABLED.key).exists("true".equalsIgnoreCase) - // 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) - val uriScheme = uri.getScheme() - - uriScheme match { - case "file" => - try { - Utils.tryWithResource(new JarFile(uri.getPath)) { jar => - // Note that this might still return null if no main-class is set; we catch that later - mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") - } - } catch { - case _: Exception => - error(s"Cannot load main class from JAR $primaryResource") - } - case _ => - error( - s"Cannot load main class from JAR $primaryResource with URI $uriScheme. " + - "Please specify a class through --class.") - } - } - // Global defaults. These should be keep to minimum to avoid confusing behavior. master = Option(master).getOrElse("local[*]") @@ -269,9 +244,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S if (primaryResource == null) { error("Must specify a primary resource (JAR or Python or R file)") } - if (mainClass == null && SparkSubmit.isUserJar(primaryResource)) { - error("No main class set in JAR; please specify one with --class") - } if (driverMemory != null && Try(JavaUtils.byteStringAsBytes(driverMemory)).getOrElse(-1L) <= 0) { error("Driver memory must be a positive number") 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 9244ab407cb68..9d4736825618e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -453,6 +453,83 @@ class SparkSubmitSuite conf.get("spark.kubernetes.driver.container.image") should be ("bar") } + /** + * Helper function for testing main class resolution on remote JAR files. + * + * @param tempDir path to temporary directory + * @param deployMode either "client" or "cluster" + * @return a pair of the JAR file and the 4-tuple returned by + * [[org.apache.spark.deploy.SparkSubmit#prepareSubmitEnvironment]] + */ + private def testResolveMainClassOnRemoteJar( + tempDir: File, + deployMode: String + ): (File, (Seq[String], Seq[String], SparkConf, String)) = { + val excFile = TestUtils.createCompiledClass("SomeMainClass", tempDir, "", null, Seq.empty) + val jarFile = new File(tempDir, "s3-mainClass-test-%s.jar".format(System.currentTimeMillis())) + val jarUrl = TestUtils.createJar( + Seq(excFile), + jarFile, + directoryPrefix = Some(tempDir.toString), + mainClass = Some("SomeMainClass")) + + val hadoopConf = new Configuration() + updateConfWithFakeS3Fs(hadoopConf) + + val clArgs = Seq( + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", deployMode, + "--conf", "spark.hadoop.fs.s3a.impl=org.apache.spark.deploy.TestFileSystem", + "--conf", "spark.hadoop.fs.s3a.impl.disable.cache=true", + s"s3a://${jarUrl.getPath}", + "arg1", "arg2") + + val appArgs = new SparkSubmitArguments(clArgs) + (jarFile, submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf))) + } + + test("automatically sets mainClass if primary resource is S3 JAR in client mode") { + withTempDir { tempDir => + val (jarFile, (childArgs, classpaths, _, mainClass_)) = testResolveMainClassOnRemoteJar( + tempDir, "client" + ) + + mainClass_ should be ("SomeMainClass") + classpaths should have length 1 + classpaths.head should endWith (jarFile.getName) + childArgs.mkString(" ") should be ("arg1 arg2") + } + } + + test("automatically sets mainClass if primary resource is S3 JAR in cluster mode") { + withTempDir { tempDir => + val (jarFile, (childArgs, classpaths, _, mainClass_)) = testResolveMainClassOnRemoteJar( + tempDir, "cluster" + ) + + mainClass_ should be (YARN_CLUSTER_SUBMIT_CLASS) + classpaths should have length 1 + classpaths.head should endWith (jarFile.getName) + childArgs.mkString(" ") should include ("--class SomeMainClass") + childArgs.mkString(" ") should endWith ("--arg arg1 --arg arg2") + } + } + + test("error informatively when mainClass isn't set and S3 JAR doesn't exist") { + val hadoopConf = new Configuration() + updateConfWithFakeS3Fs(hadoopConf) + + val clArgs = Seq( + "--name", "testApp", + "--master", "yarn", + "--conf", "spark.hadoop.fs.s3a.impl=org.apache.spark.deploy.TestFileSystem", + "--conf", "spark.hadoop.fs.s3a.impl.disable.cache=true", + s"s3a:///does-not-exist.jar") + + testPrematureExit(clArgs.toArray, "File /does-not-exist.jar does not exist") + } + test("handles confs with flag equivalents") { val clArgs = Seq( "--deploy-mode", "cluster", From 901ff929696a023f6e5d2a8d38f88e28d6d7c24a Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 17 Oct 2019 17:02:31 -0700 Subject: [PATCH 0354/1387] [SPARK-29464][PYTHON][ML] PySpark ML should expose Params.clear() to unset a user supplied Param ### What changes were proposed in this pull request? change PySpark ml ```Params._clear``` to ```Params.clear``` ### Why are the changes needed? PySpark ML currently has a private _clear() method that will unset a param. This should be made public to match the Scala API and give users a way to unset a user supplied param. ### Does this PR introduce any user-facing change? Yes. PySpark ml ```Params._clear``` ---> ```Params.clear``` ### How was this patch tested? Add test. Closes #26130 from huaxingao/spark-29464. Authored-by: Huaxin Gao Signed-off-by: Bryan Cutler --- python/pyspark/ml/classification.py | 4 ++-- python/pyspark/ml/param/__init__.py | 2 +- python/pyspark/ml/tests/test_param.py | 20 ++++++++++++++++++-- python/pyspark/ml/wrapper.py | 8 ++++++++ python/pyspark/testing/mlutils.py | 2 +- 5 files changed, 30 insertions(+), 6 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index c08705829c2f1..338b186f92b39 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -446,7 +446,7 @@ def setThreshold(self, value): Clears value of :py:attr:`thresholds` if it has been set. """ self._set(threshold=value) - self._clear(self.thresholds) + self.clear(self.thresholds) return self @since("1.4.0") @@ -477,7 +477,7 @@ def setThresholds(self, value): Clears value of :py:attr:`threshold` if it has been set. """ self._set(thresholds=value) - self._clear(self.threshold) + self.clear(self.threshold) return self @since("1.5.0") diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 5b6b70292f099..57ad1e6dfb3e6 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -452,7 +452,7 @@ def _set(self, **kwargs): self._paramMap[p] = value return self - def _clear(self, param): + def clear(self, param): """ Clears a param from the param map if it has been explicitly set. """ diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index ba6f483699706..4c7f01484dc21 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -27,8 +27,8 @@ from pyspark.ml.classification import LogisticRegression from pyspark.ml.clustering import KMeans from pyspark.ml.feature import Binarizer, Bucketizer, ElementwiseProduct, IndexToString, \ - VectorSlicer, Word2Vec -from pyspark.ml.linalg import DenseVector, SparseVector + MaxAbsScaler, VectorSlicer, Word2Vec +from pyspark.ml.linalg import DenseVector, SparseVector, Vectors from pyspark.ml.param import Param, Params, TypeConverters from pyspark.ml.param.shared import HasInputCol, HasMaxIter, HasSeed from pyspark.ml.wrapper import JavaParams @@ -224,6 +224,10 @@ def test_params(self): testParams.setMaxIter(100) self.assertTrue(testParams.isSet(maxIter)) self.assertEqual(testParams.getMaxIter(), 100) + testParams.clear(maxIter) + self.assertFalse(testParams.isSet(maxIter)) + self.assertEqual(testParams.getMaxIter(), 10) + testParams.setMaxIter(100) self.assertTrue(testParams.hasParam(inputCol.name)) self.assertFalse(testParams.hasDefault(inputCol)) @@ -248,6 +252,18 @@ def test_params(self): "maxIter: max number of iterations (>= 0). (default: 10, current: 100)", "seed: random seed. (default: 41, current: 43)"])) + def test_clear_param(self): + df = self.spark.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) + maScaler = MaxAbsScaler(inputCol="a", outputCol="scaled") + model = maScaler.fit(df) + self.assertTrue(model.isSet(model.outputCol)) + self.assertEqual(model.getOutputCol(), "scaled") + model.clear(model.outputCol) + self.assertFalse(model.isSet(model.outputCol)) + self.assertEqual(model.getOutputCol()[:12], 'MaxAbsScaler') + output = model.transform(df) + self.assertEqual(model.getOutputCol(), output.schema.names[1]) + def test_kmeans_param(self): algo = KMeans() self.assertEqual(algo.getInitMode(), "k-means||") diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 6534303b5df2b..47e4921541ea2 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -280,6 +280,14 @@ def copy(self, extra=None): that._transfer_params_to_java() return that + def clear(self, param): + """ + Clears a param from the param map if it has been explicitly set. + """ + super(JavaParams, self).clear(param) + java_param = self._java_obj.getParam(param.name) + self._java_obj.clear(java_param) + @inherit_doc class JavaEstimator(JavaParams, Estimator): diff --git a/python/pyspark/testing/mlutils.py b/python/pyspark/testing/mlutils.py index 12bf650a28ee1..a36d0709d8013 100644 --- a/python/pyspark/testing/mlutils.py +++ b/python/pyspark/testing/mlutils.py @@ -62,7 +62,7 @@ def check_params(test_self, py_stage, check_params_exist=True): continue # Random seeds between Spark and PySpark are different java_default = _java2py(test_self.sc, java_stage.clear(java_param).getOrDefault(java_param)) - py_stage._clear(p) + py_stage.clear(p) py_default = py_stage.getOrDefault(p) # equality test for NaN is always False if isinstance(java_default, float) and np.isnan(java_default): From 8616109061efc5b23b24bb9ec4a3c0f2745903c1 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Fri, 18 Oct 2019 15:39:35 +0800 Subject: [PATCH 0355/1387] [SPARK-9853][CORE][FOLLOW-UP] Regularize all the shuffle configurations related to adaptive execution ### What changes were proposed in this pull request? 1. Regularize all the shuffle configurations related to adaptive execution. 2. Add default value for `BlockStoreShuffleReader.shouldBatchFetch`. ### Why are the changes needed? It's a follow-up PR for #26040. Regularize the existing `spark.sql.adaptive.shuffle` namespace in SQLConf. ### Does this PR introduce any user-facing change? Rename one released user config `spark.sql.adaptive.minNumPostShufflePartitions` to `spark.sql.adaptive.shuffle.minNumPostShufflePartitions`, other changed configs is not released yet. ### How was this patch tested? Existing UT. Closes #26147 from xuanyuanking/SPARK-9853. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../shuffle/BlockStoreShuffleReader.scala | 2 +- .../BlockStoreShuffleReaderSuite.scala | 3 +- .../apache/spark/sql/internal/SQLConf.scala | 50 +++++++++---------- 3 files changed, 25 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 14080f8822f9e..d5a66db23301a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -36,7 +36,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( serializerManager: SerializerManager = SparkEnv.get.serializerManager, blockManager: BlockManager = SparkEnv.get.blockManager, mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker, - shouldBatchFetch: Boolean) + shouldBatchFetch: Boolean = false) extends ShuffleReader[K, C] with Logging { private val dep = handle.dependency diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 67adf5fa5e185..3f9536e224ded 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -138,8 +138,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext taskContext, metrics, serializerManager, - blockManager, - shouldBatchFetch = false) + blockManager) assert(shuffleReader.read().length === keyValuePairsPerMap * numMaps) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e2c1308cdc60f..eb2f13baf6fcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -349,15 +349,19 @@ object SQLConf { .checkValue(_ > 0, "The value of spark.sql.shuffle.partitions must be positive") .createWithDefault(200) + val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") + .doc("When true, enable adaptive query execution.") + .booleanConf + .createWithDefault(false) + val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE = buildConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize") .doc("The target post-shuffle input size in bytes of a task.") .bytesConf(ByteUnit.BYTE) .createWithDefault(64 * 1024 * 1024) - val FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED = - buildConf("spark.sql.adaptive.fetchShuffleBlocksInBatch.enabled") + buildConf("spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled") .doc("Whether to fetch the continuous shuffle blocks in batch. Instead of fetching blocks " + "one by one, fetching continuous shuffle blocks for the same map task in batch can " + "reduce IO and improve performance. Note, this feature also depends on a relocatable " + @@ -365,29 +369,15 @@ object SQLConf { .booleanConf .createWithDefault(true) - val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled") - .doc("When true, enable adaptive query execution.") - .booleanConf - .createWithDefault(false) - - val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = - buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") - .doc("The relation with a non-empty partition ratio lower than this config will not be " + - "considered as the build side of a broadcast-hash join in adaptive execution regardless " + - "of its size.") - .doubleConf - .checkValue(_ >= 0, "The non-empty partition ratio must be positive number.") - .createWithDefault(0.2) - val REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED = - buildConf("spark.sql.adaptive.reducePostShufflePartitions.enabled") + buildConf("spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled") .doc("When true and adaptive execution is enabled, this enables reducing the number of " + "post-shuffle partitions based on map output statistics.") .booleanConf .createWithDefault(true) val SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS = - buildConf("spark.sql.adaptive.minNumPostShufflePartitions") + buildConf("spark.sql.adaptive.shuffle.minNumPostShufflePartitions") .doc("The advisory minimum number of post-shuffle partitions used in adaptive execution.") .intConf .checkValue(_ > 0, "The minimum shuffle partition number " + @@ -395,7 +385,7 @@ object SQLConf { .createWithDefault(1) val SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS = - buildConf("spark.sql.adaptive.maxNumPostShufflePartitions") + buildConf("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions") .doc("The advisory maximum number of post-shuffle partitions used in adaptive execution. " + "This is used as the initial number of pre-shuffle partitions. By default it equals to " + "spark.sql.shuffle.partitions") @@ -405,13 +395,22 @@ object SQLConf { .createOptional val OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED = - buildConf("spark.sql.adaptive.optimizedLocalShuffleReader.enabled") + buildConf("spark.sql.adaptive.shuffle.optimizedLocalShuffleReader.enabled") .doc("When true and adaptive execution is enabled, this enables the optimization of" + " converting the shuffle reader to local shuffle reader for the shuffle exchange" + " of the broadcast hash join in probe side.") .booleanConf .createWithDefault(true) + val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = + buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") + .doc("The relation with a non-empty partition ratio lower than this config will not be " + + "considered as the build side of a broadcast-hash join in adaptive execution regardless " + + "of its size.") + .doubleConf + .checkValue(_ >= 0, "The non-empty partition ratio must be positive number.") + .createWithDefault(0.2) + val SUBEXPRESSION_ELIMINATION_ENABLED = buildConf("spark.sql.subexpressionElimination.enabled") .internal() @@ -2148,21 +2147,18 @@ class SQLConf extends Serializable with Logging { def numShufflePartitions: Int = getConf(SHUFFLE_PARTITIONS) - def targetPostShuffleInputSize: Long = - getConf(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) + def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) - def fetchShuffleBlocksInBatchEnabled: Boolean = - getConf(FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED) + def targetPostShuffleInputSize: Long = getConf(SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) - def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) + def fetchShuffleBlocksInBatchEnabled: Boolean = getConf(FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED) def nonEmptyPartitionRatioForBroadcastJoin: Double = getConf(NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN) def reducePostShufflePartitionsEnabled: Boolean = getConf(REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED) - def minNumPostShufflePartitions: Int = - getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) + def minNumPostShufflePartitions: Int = getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) def maxNumPostShufflePartitions: Int = getConf(SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS).getOrElse(numShufflePartitions) From ec5d698d99634e5bb8fc7b0fa1c270dd67c129c8 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Fri, 18 Oct 2019 15:53:13 +0800 Subject: [PATCH 0356/1387] [SPARK-29092][SQL] Report additional information about DataSourceScanExec in EXPLAIN FORMATTED # What changes were proposed in this pull request? Currently we report only output attributes of a scan while doing EXPLAIN FORMATTED. This PR implements the ```verboseStringWithOperatorId``` in DataSourceScanExec to report additional information about a scan such as pushed down filters, partition filters, location etc. **SQL** ``` EXPLAIN FORMATTED SELECT key, max(val) FROM explain_temp1 WHERE key > 0 GROUP BY key ORDER BY key ``` **Before** ``` == Physical Plan == * Sort (9) +- Exchange (8) +- * HashAggregate (7) +- Exchange (6) +- * HashAggregate (5) +- * Project (4) +- * Filter (3) +- * ColumnarToRow (2) +- Scan parquet default.explain_temp1 (1) (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] .... .... .... ``` **After** ``` == Physical Plan == * Sort (9) +- Exchange (8) +- * HashAggregate (7) +- Exchange (6) +- * HashAggregate (5) +- * Project (4) +- * Filter (3) +- * ColumnarToRow (2) +- Scan parquet default.explain_temp1 (1) (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] Batched: true DataFilters: [isnotnull(key#x), (key#x > 0)] Format: Parquet Location: InMemoryFileIndex[file:/tmp/apache/spark/spark-warehouse/explain_temp1] PushedFilters: [IsNotNull(key), GreaterThan(key,0)] ReadSchema: struct ... ... ... ``` ### Why are the changes needed? ### Does this PR introduce any user-facing change? ### How was this patch tested? Closes #26042 from dilipbiswal/verbose_string_datasrc_scanexec. Authored-by: Dilip Biswal Signed-off-by: Wenchen Fan --- .../sql/execution/DataSourceScanExec.scala | 43 ++++++++++- .../apache/spark/sql/execution/subquery.scala | 2 +- .../describe-part-after-analyze.sql.out | 28 ++++---- .../describe-table-after-alter-table.sql.out | 10 +-- .../sql-tests/results/describe.sql.out | 16 ++--- .../sql-tests/results/explain.sql.out | 72 +++++++++++++++++++ .../sql-tests/results/show-tables.sql.out | 6 +- .../org/apache/spark/sql/ExplainSuite.scala | 17 ++++- .../apache/spark/sql/SQLQueryTestSuite.scala | 4 +- 9 files changed, 162 insertions(+), 36 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 4a87049ac292b..b0fe4b741479f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -65,10 +65,26 @@ trait DataSourceScanExec extends LeafExecNode { s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]", maxFields)}$metadataStr") } + override def verboseStringWithOperatorId(): String = { + val metadataStr = metadata.toSeq.sorted.filterNot { + case (_, value) if (value.isEmpty || value.equals("[]")) => true + case (key, _) if (key.equals("DataFilters") || key.equals("Format")) => true + case (_, _) => false + }.map { + case (key, value) => s"$key: ${redact(value)}" + } + + s""" + |(${ExplainUtils.getOpId(this)}) $nodeName ${ExplainUtils.getCodegenId(this)} + |Output: ${producedAttributes.mkString("[", ", ", "]")} + |${metadataStr.mkString("\n")} + """.stripMargin + } + /** * Shorthand for calling redactString() without specifying redacting rules */ - private def redact(text: String): String = { + protected def redact(text: String): String = { Utils.redact(sqlContext.sessionState.conf.stringRedactionPattern, text) } @@ -342,6 +358,31 @@ case class FileSourceScanExec( withSelectedBucketsCount } + override def verboseStringWithOperatorId(): String = { + val metadataStr = metadata.toSeq.sorted.filterNot { + case (_, value) if (value.isEmpty || value.equals("[]")) => true + case (key, _) if (key.equals("DataFilters") || key.equals("Format")) => true + case (_, _) => false + }.map { + case (key, _) if (key.equals("Location")) => + val location = relation.location + val numPaths = location.rootPaths.length + val abbreviatedLoaction = if (numPaths <= 1) { + location.rootPaths.mkString("[", ", ", "]") + } else { + "[" + location.rootPaths.head + s", ... ${numPaths - 1} entries]" + } + s"$key: ${location.getClass.getSimpleName} ${redact(abbreviatedLoaction)}" + case (key, value) => s"$key: ${redact(value)}" + } + + s""" + |(${ExplainUtils.getOpId(this)}) $nodeName ${ExplainUtils.getCodegenId(this)} + |Output: ${producedAttributes.mkString("[", ", ", "]")} + |${metadataStr.mkString("\n")} + """.stripMargin + } + lazy val inputRDD: RDD[InternalRow] = { val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 22e3f8e035991..4215d62bfb780 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -193,7 +193,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { ) } val executedPlan = new QueryExecution(sparkSession, query).executedPlan - InSubqueryExec(expr, SubqueryExec(s"subquery${exprId.id}", executedPlan), exprId) + InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) } } } diff --git a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out index 17dd317f63b70..e12889f9bff83 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out @@ -56,12 +56,12 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 5 @@ -90,13 +90,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 7 @@ -125,13 +125,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 9 @@ -152,13 +152,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=11] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=11 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 4 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 10 @@ -187,13 +187,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=10] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 12 @@ -214,13 +214,13 @@ hr int Database default Table t Partition Values [ds=2017-08-01, hr=11] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=11 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 4 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 13 @@ -241,13 +241,13 @@ hr int Database default Table t Partition Values [ds=2017-09-01, hr=5] -Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-09-01/hr=5 +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-09-01/hr=5 Created Time [not included in comparison] Last Access [not included in comparison] Partition Statistics [not included in comparison] bytes, 2 rows # Storage Information -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t -- !query 14 diff --git a/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out index 7873085da5069..ffd6fb31620d3 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out @@ -29,7 +29,7 @@ Created By [not included in comparison] Type MANAGED Provider parquet Comment added -Location [not included in comparison]sql/core/spark-warehouse/table_with_comment +Location [not included in comparison]/{warehouse_dir}/table_with_comment -- !query 2 @@ -60,7 +60,7 @@ Type MANAGED Provider parquet Comment modified comment Table Properties [type=parquet] -Location [not included in comparison]sql/core/spark-warehouse/table_with_comment +Location [not included in comparison]/{warehouse_dir}/table_with_comment -- !query 4 @@ -95,7 +95,7 @@ Last Access [not included in comparison] Created By [not included in comparison] Type MANAGED Provider parquet -Location [not included in comparison]sql/core/spark-warehouse/table_comment +Location [not included in comparison]/{warehouse_dir}/table_comment -- !query 7 @@ -123,7 +123,7 @@ Created By [not included in comparison] Type MANAGED Provider parquet Comment added comment -Location [not included in comparison]sql/core/spark-warehouse/table_comment +Location [not included in comparison]/{warehouse_dir}/table_comment -- !query 9 @@ -150,7 +150,7 @@ Last Access [not included in comparison] Created By [not included in comparison] Type MANAGED Provider parquet -Location [not included in comparison]sql/core/spark-warehouse/table_comment +Location [not included in comparison]/{warehouse_dir}/table_comment -- !query 11 diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index f58bdb5446b64..0cb442a6d3496 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -131,7 +131,7 @@ Bucket Columns [`a`] Sort Columns [`b`] Comment table_comment Table Properties [t=test, e=3] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] Partition Provider Catalog @@ -163,7 +163,7 @@ Bucket Columns [`a`] Sort Columns [`b`] Comment table_comment Table Properties [t=test, e=3] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] Partition Provider Catalog @@ -203,7 +203,7 @@ Bucket Columns [`a`] Sort Columns [`b`] Comment table_comment Table Properties [t=test] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] Partition Provider Catalog @@ -242,7 +242,7 @@ Num Buckets 2 Bucket Columns [`a`] Sort Columns [`b`] Table Properties [t=test] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] Partition Provider Catalog @@ -280,7 +280,7 @@ d string Database default Table t Partition Values [c=Us, d=1] -Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1 +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 Storage Properties [a=1, b=2] Created Time [not included in comparison] Last Access [not included in comparison] @@ -289,7 +289,7 @@ Last Access [not included in comparison] Num Buckets 2 Bucket Columns [`a`] Sort Columns [`b`] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] @@ -311,7 +311,7 @@ d string Database default Table t Partition Values [c=Us, d=1] -Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1 +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 Storage Properties [a=1, b=2] Created Time [not included in comparison] Last Access [not included in comparison] @@ -320,7 +320,7 @@ Last Access [not included in comparison] Num Buckets 2 Bucket Columns [`a`] Sort Columns [`b`] -Location [not included in comparison]sql/core/spark-warehouse/t +Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index 4a08cfada292d..85c938773efec 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -58,6 +58,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -110,6 +114,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -165,6 +173,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -179,6 +191,10 @@ Input : [key#x, val#x] (5) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct (6) ColumnarToRow [codegen id : 2] Input: [key#x, val#x] @@ -227,6 +243,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] Input: [key#x, val#x] @@ -241,6 +261,10 @@ Input : [key#x, val#x] (5) Scan parquet default.explain_temp2 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct (6) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -284,12 +308,19 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] Input: [key#x, val#x] (3) Scan parquet default.explain_temp2 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct (4) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -334,6 +365,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), IsNotNull(val), GreaterThan(val,3)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -360,6 +395,10 @@ Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery (5) Scan parquet default.explain_temp2 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key), IsNotNull(val), EqualTo(val,2)] +ReadSchema: struct (6) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -393,6 +432,10 @@ Subquery:2 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery (12) Scan parquet default.explain_temp3 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp3] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct (13) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -437,6 +480,9 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -459,6 +505,10 @@ Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery (4) Scan parquet default.explain_temp2 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -492,6 +542,10 @@ Subquery:2 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery (11) Scan parquet default.explain_temp3 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp3] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -529,6 +583,9 @@ struct (1) Scan parquet default.explain_temp1 Output: [] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct<> (2) ColumnarToRow [codegen id : 1] Input: [] @@ -549,6 +606,9 @@ Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery (4) Scan parquet default.explain_temp1 Output: [key#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] Input: [key#x] @@ -591,6 +651,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] Input: [key#x, val#x] @@ -605,6 +669,10 @@ Input : [key#x, val#x] (5) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct (6) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] @@ -654,6 +722,10 @@ struct (1) Scan parquet default.explain_temp1 Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 1] Input: [key#x, val#x] diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index f22cb7e200e6c..ee05e2413799a 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -140,7 +140,7 @@ Last Access [not included in comparison] Created By [not included in comparison] Type: MANAGED Provider: parquet -Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1 +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t1 Partition Provider: Catalog Partition Columns: [`c`, `d`] Schema: root @@ -157,7 +157,7 @@ Last Access [not included in comparison] Created By [not included in comparison] Type: MANAGED Provider: parquet -Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t2 +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t2 Schema: root |-- b: string (nullable = true) |-- d: integer (nullable = true) @@ -183,7 +183,7 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1) struct -- !query 14 output showdb show_t1 false Partition Values: [c=Us, d=1] -Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1/c=Us/d=1 +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t1/c=Us/d=1 Created Time [not included in comparison] Last Access [not included in comparison] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index 630cf22fb20d4..e70aeae9e605e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -241,9 +241,22 @@ class ExplainSuite extends QueryTest with SharedSparkSession { |FROM df1 JOIN df2 ON df1.k = df2.k AND df2.id < 2 |""".stripMargin - val expected_pattern = "Subquery:1 Hosting operator id = 1 Hosting Expression = k#x" + val expected_pattern1 = + "Subquery:1 Hosting operator id = 1 Hosting Expression = k#xL IN subquery#x" + val expected_pattern2 = + "PartitionFilters: \\[isnotnull\\(k#xL\\), dynamicpruningexpression\\(k#xL " + + "IN subquery#x\\)\\]" + val expected_pattern3 = + "Location: PrunedInMemoryFileIndex \\[.*org.apache.spark.sql.ExplainSuite" + + "/df2/.*, ... 99 entries\\]" + val expected_pattern4 = + "Location: PrunedInMemoryFileIndex \\[.*org.apache.spark.sql.ExplainSuite" + + "/df1/.*, ... 999 entries\\]" withNormalizedExplain(sqlText) { normalizedOutput => - assert(expected_pattern.r.findAllMatchIn(normalizedOutput).length == 1) + assert(expected_pattern1.r.findAllMatchIn(normalizedOutput).length == 1) + assert(expected_pattern2.r.findAllMatchIn(normalizedOutput).length == 1) + assert(expected_pattern3.r.findAllMatchIn(normalizedOutput).length == 2) + assert(expected_pattern4.r.findAllMatchIn(normalizedOutput).length == 1) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 075f6920b1a85..58469fca8f3fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -456,8 +456,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { protected def replaceNotIncludedMsg(line: String): String = { line.replaceAll("#\\d+", "#x") .replaceAll( - s"Location.*/sql/core/spark-warehouse/$clsName/", - s"Location ${notIncludedMsg}sql/core/spark-warehouse/") + s"Location.*$clsName/", + s"Location ${notIncludedMsg}/{warehouse_dir}/") .replaceAll("Created By.*", s"Created By $notIncludedMsg") .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") From 78b0cbe265c4e8cc3d4d8bf5d734f2998c04d376 Mon Sep 17 00:00:00 2001 From: stczwd Date: Fri, 18 Oct 2019 16:06:54 +0800 Subject: [PATCH 0357/1387] [SPARK-29444] Add configuration to support JacksonGenrator to keep fields with null values ### Why are the changes needed? As mentioned in jira, sometimes we need to be able to support the retention of null columns when writing JSON. For example, sparkmagic(used widely in jupyter with livy) will generate sql query results based on DataSet.toJSON and parse JSON to pandas DataFrame to display. If there is a null column, it is easy to have some column missing or even the query result is empty. The loss of the null column in the first row, may cause parsing exceptions or loss of entire column data. ### Does this PR introduce any user-facing change? Example in spark-shell. scala> spark.sql("select null as a, 1 as b").toJSON.collect.foreach(println) {"b":1} scala> spark.sql("set spark.sql.jsonGenerator.struct.ignore.null=false") res2: org.apache.spark.sql.DataFrame = [key: string, value: string] scala> spark.sql("select null as a, 1 as b").toJSON.collect.foreach(println) {"a":null,"b":1} ### How was this patch tested? Add new test to JacksonGeneratorSuite Closes #26098 from stczwd/json. Lead-authored-by: stczwd Co-authored-by: Jackey Lee Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/json/JSONOptions.scala | 5 ++++ .../sql/catalyst/json/JacksonGenerator.scala | 3 +++ .../apache/spark/sql/internal/SQLConf.scala | 8 ++++++ .../catalyst/json/JacksonGeneratorSuite.scala | 27 +++++++++++++++++++ 4 files changed, 43 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index dc26a28c74f11..e7bfb77e46c26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.{JsonFactory, JsonParser} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf /** * Options for parsing JSON data into Spark SQL rows. @@ -76,6 +77,10 @@ private[sql] class JSONOptions( // Whether to ignore column of all null values or empty array/struct during schema inference val dropFieldIfAllNull = parameters.get("dropFieldIfAllNull").map(_.toBoolean).getOrElse(false) + // Whether to ignore null fields during json generating + val ignoreNullFields = parameters.getOrElse("ignoreNullFields", + SQLConf.get.jsonGeneratorIgnoreNullFields).toBoolean + // A language tag in IETF BCP 47 format val locale: Locale = parameters.get("locale").map(Locale.forLanguageTag).getOrElse(Locale.US) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index 3ee7e484690d5..aaf2ecf7923ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -184,6 +184,9 @@ private[sql] class JacksonGenerator( if (!row.isNullAt(i)) { gen.writeFieldName(field.name) fieldWriters(i).apply(row, i) + } else if (!options.ignoreNullFields) { + gen.writeFieldName(field.name) + gen.writeNull() } i += 1 } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index eb2f13baf6fcc..4944099fcc0d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1187,6 +1187,12 @@ object SQLConf { .booleanConf .createWithDefault(true) + val JSON_GENERATOR_IGNORE_NULL_FIELDS = + buildConf("spark.sql.jsonGenerator.ignoreNullFields") + .doc("If false, JacksonGenerator will generate null for null fields in Struct.") + .stringConf + .createWithDefault("true") + val FILE_SINK_LOG_DELETION = buildConf("spark.sql.streaming.fileSink.log.deletion") .internal() .doc("Whether to delete the expired log files in file stream sink.") @@ -2379,6 +2385,8 @@ class SQLConf extends Serializable with Logging { def sessionLocalTimeZone: String = getConf(SQLConf.SESSION_LOCAL_TIMEZONE) + def jsonGeneratorIgnoreNullFields: String = getConf(SQLConf.JSON_GENERATOR_IGNORE_NULL_FIELDS) + def parallelFileListingInStatsComputation: Boolean = getConf(SQLConf.PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala index 9b27490ed0e35..2bb948ec24fb3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JacksonGeneratorSuite.scala @@ -39,6 +39,33 @@ class JacksonGeneratorSuite extends SparkFunSuite { assert(writer.toString === """{"a":1}""") } + test("SPARK-29444: initial with StructType and write out an empty row " + + "with ignoreNullFields=false") { + val dataType = StructType(StructField("a", IntegerType) :: Nil) + val input = InternalRow(null) + val writer = new CharArrayWriter() + val allowNullOption = + new JSONOptions(Map("ignoreNullFields" -> "false"), gmtId) + val gen = new JacksonGenerator(dataType, writer, allowNullOption) + gen.write(input) + gen.flush() + assert(writer.toString === """{"a":null}""") + } + + test("SPARK-29444: initial with StructType field and write out a row " + + "with ignoreNullFields=false and struct inner null") { + val fieldType = StructType(StructField("b", IntegerType) :: Nil) + val dataType = StructType(StructField("a", fieldType) :: Nil) + val input = InternalRow(InternalRow(null)) + val writer = new CharArrayWriter() + val allowNullOption = + new JSONOptions(Map("ignoreNullFields" -> "false"), gmtId) + val gen = new JacksonGenerator(dataType, writer, allowNullOption) + gen.write(input) + gen.flush() + assert(writer.toString === """{"a":{"b":null}}""") + } + test("initial with StructType and write out rows") { val dataType = StructType(StructField("a", IntegerType) :: Nil) val input = new GenericArrayData(InternalRow(1) :: InternalRow(2) :: Nil) From ef4c298cc9a8712e22a6010f9c4310dcfd6afd32 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 18 Oct 2019 16:21:31 +0800 Subject: [PATCH 0358/1387] [SPARK-29405][SQL] Alter table / Insert statements should not change a table's ownership ### What changes were proposed in this pull request? In this change, we give preference to the original table's owner if it is not empty. ### Why are the changes needed? When executing 'insert into/overwrite ...' DML, or 'alter table set tblproperties ...' DDL, spark would change the ownership of the table the one who runs the spark application. ### Does this PR introduce any user-facing change? NO ### How was this patch tested? Compare with the behavior of Apache Hive Closes #26068 from yaooqinn/SPARK-29405. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../spark/sql/hive/client/HiveClientImpl.scala | 3 ++- .../spark/sql/hive/client/VersionsSuite.scala | 13 +++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) 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 296f9ff5dd0bb..783bc5b562ff4 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 @@ -573,8 +573,9 @@ private[hive] class HiveClientImpl( // If users explicitly alter these Hive-specific properties through ALTER TABLE DDL, we respect // these user-specified values. verifyColumnDataType(table.dataSchema) + val owner = Option(table.owner).filter(_.nonEmpty).getOrElse(userName) val hiveTable = toHiveTable( - table.copy(properties = table.ignoredProperties ++ table.properties), Some(userName)) + table.copy(properties = table.ignoredProperties ++ table.properties), Some(owner)) // Do not use `table.qualifiedName` here because this may be a rename val qualifiedTableName = s"$dbName.$tableName" shim.alterTable(client, qualifiedTableName, hiveTable) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index ac31557b94197..e3797041883ac 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -291,6 +291,19 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(client.getTable("default", "src").properties.contains("changed")) } + test(s"$version: alterTable - should respect the original catalog table's owner name") { + val ownerName = "SPARK-29405" + val originalTable = client.getTable("default", "src") + // mocking the owner is what we declared + val newTable = originalTable.copy(owner = ownerName) + client.alterTable(newTable) + assert(client.getTable("default", "src").owner === ownerName) + // mocking the owner is empty + val newTable2 = originalTable.copy(owner = "") + client.alterTable(newTable2) + assert(client.getTable("default", "src").owner === client.userName) + } + test(s"$version: alterTable(dbName: String, tableName: String, table: CatalogTable)") { val newTable = client.getTable("default", "src").copy(properties = Map("changedAgain" -> "")) client.alterTable("default", "src", newTable) From 5692680e3764383f0bbb1c6730a5f8a02e2786c6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 18 Oct 2019 16:35:44 +0800 Subject: [PATCH 0359/1387] [SPARK-29295][SQL] Insert overwrite to Hive external table partition should delete old data ### What changes were proposed in this pull request? This patch proposes to delete old Hive external partition directory even the partition does not exist in Hive, when insert overwrite Hive external table partition. ### Why are the changes needed? When insert overwrite to a Hive external table partition, if the partition does not exist, Hive will not check if the external partition directory exists or not before copying files. So if users drop the partition, and then do insert overwrite to the same partition, the partition will have both old and new data. For example: ```scala withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { // test is an external Hive table. sql("INSERT OVERWRITE TABLE test PARTITION(name='n1') SELECT 1") sql("ALTER TABLE test DROP PARTITION(name='n1')") sql("INSERT OVERWRITE TABLE test PARTITION(name='n1') SELECT 2") sql("SELECT id FROM test WHERE name = 'n1' ORDER BY id") // Got both 1 and 2. } ``` ### Does this PR introduce any user-facing change? Yes. This fix a correctness issue when users drop partition on a Hive external table partition and then insert overwrite it. ### How was this patch tested? Added test. Closes #25979 from viirya/SPARK-29295. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../hive/execution/InsertIntoHiveTable.scala | 68 +++++++++++++--- .../sql/hive/execution/SQLQuerySuite.scala | 80 +++++++++++++++++++ 2 files changed, 139 insertions(+), 9 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index ee1734b1f232c..801be64702519 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, Row, SparkSession} -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, ExternalCatalog} +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, ExternalCatalog, ExternalCatalogUtils} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan @@ -199,7 +199,7 @@ case class InsertIntoHiveTable( attr.withName(name.toLowerCase(Locale.ROOT)) } - saveAsHiveFile( + val writtenParts = saveAsHiveFile( sparkSession = sparkSession, plan = child, hadoopConf = hadoopConf, @@ -209,6 +209,42 @@ case class InsertIntoHiveTable( if (partition.nonEmpty) { if (numDynamicPartitions > 0) { + if (overwrite && table.tableType == CatalogTableType.EXTERNAL) { + // SPARK-29295: When insert overwrite to a Hive external table partition, if the + // partition does not exist, Hive will not check if the external partition directory + // exists or not before copying files. So if users drop the partition, and then do + // insert overwrite to the same partition, the partition will have both old and new + // data. We construct partition path. If the path exists, we delete it manually. + writtenParts.foreach { partPath => + val dpMap = partPath.split("/").map { part => + val splitPart = part.split("=") + assert(splitPart.size == 2, s"Invalid written partition path: $part") + ExternalCatalogUtils.unescapePathName(splitPart(0)) -> + ExternalCatalogUtils.unescapePathName(splitPart(1)) + }.toMap + + val updatedPartitionSpec = partition.map { + case (key, Some(value)) => key -> value + case (key, None) if dpMap.contains(key) => key -> dpMap(key) + case (key, _) => + throw new SparkException(s"Dynamic partition key $key is not among " + + "written partition paths.") + } + val partitionColumnNames = table.partitionColumnNames + val tablePath = new Path(table.location) + val partitionPath = ExternalCatalogUtils.generatePartitionPath(updatedPartitionSpec, + partitionColumnNames, tablePath) + + val fs = partitionPath.getFileSystem(hadoopConf) + if (fs.exists(partitionPath)) { + if (!fs.delete(partitionPath, true)) { + throw new RuntimeException( + "Cannot remove partition directory '" + partitionPath.toString) + } + } + } + } + externalCatalog.loadDynamicPartitions( db = table.database, table = table.identifier.table, @@ -230,18 +266,32 @@ case class InsertIntoHiveTable( var doHiveOverwrite = overwrite if (oldPart.isEmpty || !ifPartitionNotExists) { + // SPARK-29295: When insert overwrite to a Hive external table partition, if the + // partition does not exist, Hive will not check if the external partition directory + // exists or not before copying files. So if users drop the partition, and then do + // insert overwrite to the same partition, the partition will have both old and new + // data. We construct partition path. If the path exists, we delete it manually. + val partitionPath = if (oldPart.isEmpty && overwrite + && table.tableType == CatalogTableType.EXTERNAL) { + val partitionColumnNames = table.partitionColumnNames + val tablePath = new Path(table.location) + Some(ExternalCatalogUtils.generatePartitionPath(partitionSpec, + partitionColumnNames, tablePath)) + } else { + oldPart.flatMap(_.storage.locationUri.map(uri => new Path(uri))) + } + // SPARK-18107: Insert overwrite runs much slower than hive-client. // Newer Hive largely improves insert overwrite performance. As Spark uses older Hive // version and we may not want to catch up new Hive version every time. We delete the // Hive partition first and then load data file into the Hive partition. - if (oldPart.nonEmpty && overwrite) { - oldPart.get.storage.locationUri.foreach { uri => - val partitionPath = new Path(uri) - val fs = partitionPath.getFileSystem(hadoopConf) - if (fs.exists(partitionPath)) { - if (!fs.delete(partitionPath, true)) { + if (partitionPath.nonEmpty && overwrite) { + partitionPath.foreach { path => + val fs = path.getFileSystem(hadoopConf) + if (fs.exists(path)) { + if (!fs.delete(path, true)) { throw new RuntimeException( - "Cannot remove partition directory '" + partitionPath.toString) + "Cannot remove partition directory '" + path.toString) } // Don't let Hive do overwrite operation since it is slower. doHiveOverwrite = false diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 736a2dcfad297..79a490b8a2a36 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -2432,4 +2432,84 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } } + + test("SPARK-29295: insert overwrite external partition should not have old data") { + Seq("true", "false").foreach { convertParquet => + withTable("test") { + withTempDir { f => + sql("CREATE EXTERNAL TABLE test(id int) PARTITIONED BY (name string) STORED AS " + + s"PARQUET LOCATION '${f.getAbsolutePath}'") + + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertParquet) { + sql("INSERT OVERWRITE TABLE test PARTITION(name='n1') SELECT 1") + sql("ALTER TABLE test DROP PARTITION(name='n1')") + sql("INSERT OVERWRITE TABLE test PARTITION(name='n1') SELECT 2") + checkAnswer(sql("SELECT id FROM test WHERE name = 'n1' ORDER BY id"), + Array(Row(2))) + } + } + } + } + } + + test("SPARK-29295: dynamic insert overwrite external partition should not have old data") { + Seq("true", "false").foreach { convertParquet => + withTable("test") { + withTempDir { f => + sql("CREATE EXTERNAL TABLE test(id int) PARTITIONED BY (p1 string, p2 string) " + + s"STORED AS PARQUET LOCATION '${f.getAbsolutePath}'") + + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertParquet, + "hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql( + """ + |INSERT OVERWRITE TABLE test PARTITION(p1='n1', p2) + |SELECT * FROM VALUES (1, 'n2'), (2, 'n3') AS t(id, p2) + """.stripMargin) + checkAnswer(sql("SELECT id FROM test WHERE p1 = 'n1' and p2 = 'n2' ORDER BY id"), + Array(Row(1))) + checkAnswer(sql("SELECT id FROM test WHERE p1 = 'n1' and p2 = 'n3' ORDER BY id"), + Array(Row(2))) + + sql("INSERT OVERWRITE TABLE test PARTITION(p1='n1', p2) SELECT 4, 'n4'") + checkAnswer(sql("SELECT id FROM test WHERE p1 = 'n1' and p2 = 'n4' ORDER BY id"), + Array(Row(4))) + + sql("ALTER TABLE test DROP PARTITION(p1='n1',p2='n2')") + sql("ALTER TABLE test DROP PARTITION(p1='n1',p2='n3')") + + sql( + """ + |INSERT OVERWRITE TABLE test PARTITION(p1='n1', p2) + |SELECT * FROM VALUES (5, 'n2'), (6, 'n3') AS t(id, p2) + """.stripMargin) + checkAnswer(sql("SELECT id FROM test WHERE p1 = 'n1' and p2 = 'n2' ORDER BY id"), + Array(Row(5))) + checkAnswer(sql("SELECT id FROM test WHERE p1 = 'n1' and p2 = 'n3' ORDER BY id"), + Array(Row(6))) + // Partition not overwritten should not be deleted. + checkAnswer(sql("SELECT id FROM test WHERE p1 = 'n1' and p2 = 'n4' ORDER BY id"), + Array(Row(4))) + } + } + } + + withTable("test") { + withTempDir { f => + sql("CREATE EXTERNAL TABLE test(id int) PARTITIONED BY (p1 string, p2 string) " + + s"STORED AS PARQUET LOCATION '${f.getAbsolutePath}'") + + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertParquet, + "hive.exec.dynamic.partition.mode" -> "nonstrict") { + // We should unescape partition value. + sql("INSERT OVERWRITE TABLE test PARTITION(p1='n1', p2) SELECT 1, '/'") + sql("ALTER TABLE test DROP PARTITION(p1='n1',p2='/')") + sql("INSERT OVERWRITE TABLE test PARTITION(p1='n1', p2) SELECT 2, '/'") + checkAnswer(sql("SELECT id FROM test WHERE p1 = 'n1' and p2 = '/' ORDER BY id"), + Array(Row(2))) + } + } + } + } + } } From 6f8c001c8d5222e979532657005e236834ef5d91 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 18 Oct 2019 17:26:54 +0800 Subject: [PATCH 0360/1387] [SPARK-29381][FOLLOWUP][PYTHON][ML] Add 'private' _XXXParams classes for classification & regression ### What changes were proposed in this pull request? Add private _XXXParams classes for classification & regression ### Why are the changes needed? To keep parity between scala and python ### Does this PR introduce any user-facing change? Yes. Add gettters/setters for the following Model classes ``` LinearSVCModel: get/setRegParam get/setMaxIte get/setFitIntercept get/setTol get/setStandardization get/setWeightCol get/setAggregationDepth get/setThreshold LogisticRegressionModel: get/setRegParam get/setElasticNetParam get/setMaxIter get/setFitIntercept get/setTol get/setStandardization get/setWeightCol get/setAggregationDepth get/setThreshold NaiveBayesModel: get/setWeightCol LinearRegressionModel: get/setRegParam get/setElasticNetParam get/setMaxIter get/setTol get/setFitIntercept get/setStandardization get/setWeight get/setSolver get/setAggregationDepth get/setLoss GeneralizedLinearRegressionModel: get/setFitIntercept get/setMaxIter get/setTol get/setRegParam get/setWeightCol get/setSolver ``` ### How was this patch tested? Add a few doctest Closes #26142 from huaxingao/spark-29381. Authored-by: Huaxin Gao Signed-off-by: zhengruifeng --- python/pyspark/ml/classification.py | 501 +++++++++++++++------------- python/pyspark/ml/regression.py | 216 ++++++------ 2 files changed, 392 insertions(+), 325 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 338b186f92b39..d0c821329471f 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -149,10 +149,23 @@ def setThresholds(self, value): return self._set(thresholds=value) +class _LinearSVCParams(_JavaClassifierParams, HasRegParam, HasMaxIter, HasFitIntercept, HasTol, + HasStandardization, HasWeightCol, HasAggregationDepth, HasThreshold): + """ + Params for :py:class:`LinearSVC` and :py:class:`LinearSVCModel`. + + .. versionadded:: 3.0.0 + """ + + threshold = Param(Params._dummy(), "threshold", + "The threshold in binary classification applied to the linear model" + " prediction. This threshold can be any real number, where Inf will make" + " all predictions 0.0 and -Inf will make all predictions 1.0.", + typeConverter=TypeConverters.toFloat) + + @inherit_doc -class LinearSVC(JavaClassifier, HasMaxIter, HasRegParam, HasTol, - HasFitIntercept, HasStandardization, HasWeightCol, HasAggregationDepth, - HasThreshold, JavaMLWritable, JavaMLReadable): +class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable): """ `Linear SVM Classifier `_ @@ -170,6 +183,10 @@ class LinearSVC(JavaClassifier, HasMaxIter, HasRegParam, HasTol, LinearSVC... >>> model.getPredictionCol() 'newPrediction' + >>> model.setThreshold(0.5) + LinearSVC... + >>> model.getThreshold() + 0.5 >>> model.coefficients DenseVector([0.0, -0.2792, -0.1833]) >>> model.intercept @@ -202,12 +219,6 @@ class LinearSVC(JavaClassifier, HasMaxIter, HasRegParam, HasTol, .. versionadded:: 2.2.0 """ - threshold = Param(Params._dummy(), "threshold", - "The threshold in binary classification applied to the linear model" - " prediction. This threshold can be any real number, where Inf will make" - " all predictions 0.0 and -Inf will make all predictions 1.0.", - typeConverter=TypeConverters.toFloat) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", @@ -247,7 +258,7 @@ def _create_model(self, java_model): return LinearSVCModel(java_model) -class LinearSVCModel(JavaClassificationModel, JavaMLWritable, JavaMLReadable): +class LinearSVCModel(JavaClassificationModel, _LinearSVCParams, JavaMLWritable, JavaMLReadable): """ Model fitted by LinearSVC. @@ -271,10 +282,167 @@ def intercept(self): return self._call_java("intercept") +class _LogisticRegressionParams(_JavaProbabilisticClassifierParams, HasRegParam, + HasElasticNetParam, HasMaxIter, HasFitIntercept, HasTol, + HasStandardization, HasWeightCol, HasAggregationDepth, + HasThreshold): + """ + Params for :py:class:`LogisticRegression` and :py:class:`LogisticRegressionModel`. + + .. versionadded:: 3.0.0 + """ + + threshold = Param(Params._dummy(), "threshold", + "Threshold in binary classification prediction, in range [0, 1]." + + " If threshold and thresholds are both set, they must match." + + "e.g. if threshold is p, then thresholds must be equal to [1-p, p].", + typeConverter=TypeConverters.toFloat) + + family = Param(Params._dummy(), "family", + "The name of family which is a description of the label distribution to " + + "be used in the model. Supported options: auto, binomial, multinomial", + typeConverter=TypeConverters.toString) + + lowerBoundsOnCoefficients = Param(Params._dummy(), "lowerBoundsOnCoefficients", + "The lower bounds on coefficients if fitting under bound " + "constrained optimization. The bound matrix must be " + "compatible with the shape " + "(1, number of features) for binomial regression, or " + "(number of classes, number of features) " + "for multinomial regression.", + typeConverter=TypeConverters.toMatrix) + + upperBoundsOnCoefficients = Param(Params._dummy(), "upperBoundsOnCoefficients", + "The upper bounds on coefficients if fitting under bound " + "constrained optimization. The bound matrix must be " + "compatible with the shape " + "(1, number of features) for binomial regression, or " + "(number of classes, number of features) " + "for multinomial regression.", + typeConverter=TypeConverters.toMatrix) + + lowerBoundsOnIntercepts = Param(Params._dummy(), "lowerBoundsOnIntercepts", + "The lower bounds on intercepts if fitting under bound " + "constrained optimization. The bounds vector size must be" + "equal with 1 for binomial regression, or the number of" + "lasses for multinomial regression.", + typeConverter=TypeConverters.toVector) + + upperBoundsOnIntercepts = Param(Params._dummy(), "upperBoundsOnIntercepts", + "The upper bounds on intercepts if fitting under bound " + "constrained optimization. The bound vector size must be " + "equal with 1 for binomial regression, or the number of " + "classes for multinomial regression.", + typeConverter=TypeConverters.toVector) + + @since("1.4.0") + def setThreshold(self, value): + """ + Sets the value of :py:attr:`threshold`. + Clears value of :py:attr:`thresholds` if it has been set. + """ + self._set(threshold=value) + self.clear(self.thresholds) + return self + + @since("1.4.0") + def getThreshold(self): + """ + Get threshold for binary classification. + + If :py:attr:`thresholds` is set with length 2 (i.e., binary classification), + this returns the equivalent threshold: + :math:`\\frac{1}{1 + \\frac{thresholds(0)}{thresholds(1)}}`. + Otherwise, returns :py:attr:`threshold` if set or its default value if unset. + """ + self._checkThresholdConsistency() + if self.isSet(self.thresholds): + ts = self.getOrDefault(self.thresholds) + if len(ts) != 2: + raise ValueError("Logistic Regression getThreshold only applies to" + + " binary classification, but thresholds has length != 2." + + " thresholds: " + ",".join(ts)) + return 1.0/(1.0 + ts[0]/ts[1]) + else: + return self.getOrDefault(self.threshold) + + @since("1.5.0") + def setThresholds(self, value): + """ + Sets the value of :py:attr:`thresholds`. + Clears value of :py:attr:`threshold` if it has been set. + """ + self._set(thresholds=value) + self.clear(self.threshold) + return self + + @since("1.5.0") + def getThresholds(self): + """ + If :py:attr:`thresholds` is set, return its value. + Otherwise, if :py:attr:`threshold` is set, return the equivalent thresholds for binary + classification: (1-threshold, threshold). + If neither are set, throw an error. + """ + self._checkThresholdConsistency() + if not self.isSet(self.thresholds) and self.isSet(self.threshold): + t = self.getOrDefault(self.threshold) + return [1.0-t, t] + else: + return self.getOrDefault(self.thresholds) + + def _checkThresholdConsistency(self): + if self.isSet(self.threshold) and self.isSet(self.thresholds): + ts = self.getOrDefault(self.thresholds) + if len(ts) != 2: + raise ValueError("Logistic Regression getThreshold only applies to" + + " binary classification, but thresholds has length != 2." + + " thresholds: {0}".format(str(ts))) + t = 1.0/(1.0 + ts[0]/ts[1]) + t2 = self.getOrDefault(self.threshold) + if abs(t2 - t) >= 1E-5: + raise ValueError("Logistic Regression getThreshold found inconsistent values for" + + " threshold (%g) and thresholds (equivalent to %g)" % (t2, t)) + + @since("2.1.0") + def getFamily(self): + """ + Gets the value of :py:attr:`family` or its default value. + """ + return self.getOrDefault(self.family) + + @since("2.3.0") + def getLowerBoundsOnCoefficients(self): + """ + Gets the value of :py:attr:`lowerBoundsOnCoefficients` + """ + return self.getOrDefault(self.lowerBoundsOnCoefficients) + + @since("2.3.0") + def getUpperBoundsOnCoefficients(self): + """ + Gets the value of :py:attr:`upperBoundsOnCoefficients` + """ + return self.getOrDefault(self.upperBoundsOnCoefficients) + + @since("2.3.0") + def getLowerBoundsOnIntercepts(self): + """ + Gets the value of :py:attr:`lowerBoundsOnIntercepts` + """ + return self.getOrDefault(self.lowerBoundsOnIntercepts) + + @since("2.3.0") + def getUpperBoundsOnIntercepts(self): + """ + Gets the value of :py:attr:`upperBoundsOnIntercepts` + """ + return self.getOrDefault(self.upperBoundsOnIntercepts) + + @inherit_doc -class LogisticRegression(JavaProbabilisticClassifier, HasMaxIter, HasRegParam, HasTol, - HasElasticNetParam, HasFitIntercept, HasStandardization, HasThresholds, - HasWeightCol, HasAggregationDepth, JavaMLWritable, JavaMLReadable): +class LogisticRegression(JavaProbabilisticClassifier, _LogisticRegressionParams, JavaMLWritable, + JavaMLReadable): """ Logistic regression. This class supports multinomial logistic (softmax) and binomial logistic regression. @@ -294,6 +462,10 @@ class LogisticRegression(JavaProbabilisticClassifier, HasMaxIter, HasRegParam, H LogisticRegressionModel... >>> blorModel.getProbabilityCol() 'newProbability' + >>> blorModel.setThreshold(0.1) + LogisticRegressionModel... + >>> blorModel.getThreshold() + 0.1 >>> blorModel.coefficients DenseVector([-1.080..., -0.646...]) >>> blorModel.intercept @@ -341,49 +513,6 @@ class LogisticRegression(JavaProbabilisticClassifier, HasMaxIter, HasRegParam, H .. versionadded:: 1.3.0 """ - threshold = Param(Params._dummy(), "threshold", - "Threshold in binary classification prediction, in range [0, 1]." + - " If threshold and thresholds are both set, they must match." + - "e.g. if threshold is p, then thresholds must be equal to [1-p, p].", - typeConverter=TypeConverters.toFloat) - - family = Param(Params._dummy(), "family", - "The name of family which is a description of the label distribution to " + - "be used in the model. Supported options: auto, binomial, multinomial", - typeConverter=TypeConverters.toString) - - lowerBoundsOnCoefficients = Param(Params._dummy(), "lowerBoundsOnCoefficients", - "The lower bounds on coefficients if fitting under bound " - "constrained optimization. The bound matrix must be " - "compatible with the shape " - "(1, number of features) for binomial regression, or " - "(number of classes, number of features) " - "for multinomial regression.", - typeConverter=TypeConverters.toMatrix) - - upperBoundsOnCoefficients = Param(Params._dummy(), "upperBoundsOnCoefficients", - "The upper bounds on coefficients if fitting under bound " - "constrained optimization. The bound matrix must be " - "compatible with the shape " - "(1, number of features) for binomial regression, or " - "(number of classes, number of features) " - "for multinomial regression.", - typeConverter=TypeConverters.toMatrix) - - lowerBoundsOnIntercepts = Param(Params._dummy(), "lowerBoundsOnIntercepts", - "The lower bounds on intercepts if fitting under bound " - "constrained optimization. The bounds vector size must be" - "equal with 1 for binomial regression, or the number of" - "lasses for multinomial regression.", - typeConverter=TypeConverters.toVector) - - upperBoundsOnIntercepts = Param(Params._dummy(), "upperBoundsOnIntercepts", - "The upper bounds on intercepts if fitting under bound " - "constrained optimization. The bound vector size must be " - "equal with 1 for binomial regression, or the number of " - "classes for multinomial regression.", - typeConverter=TypeConverters.toVector) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, @@ -439,75 +568,6 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return LogisticRegressionModel(java_model) - @since("1.4.0") - def setThreshold(self, value): - """ - Sets the value of :py:attr:`threshold`. - Clears value of :py:attr:`thresholds` if it has been set. - """ - self._set(threshold=value) - self.clear(self.thresholds) - return self - - @since("1.4.0") - def getThreshold(self): - """ - Get threshold for binary classification. - - If :py:attr:`thresholds` is set with length 2 (i.e., binary classification), - this returns the equivalent threshold: - :math:`\\frac{1}{1 + \\frac{thresholds(0)}{thresholds(1)}}`. - Otherwise, returns :py:attr:`threshold` if set or its default value if unset. - """ - self._checkThresholdConsistency() - if self.isSet(self.thresholds): - ts = self.getOrDefault(self.thresholds) - if len(ts) != 2: - raise ValueError("Logistic Regression getThreshold only applies to" + - " binary classification, but thresholds has length != 2." + - " thresholds: " + ",".join(ts)) - return 1.0/(1.0 + ts[0]/ts[1]) - else: - return self.getOrDefault(self.threshold) - - @since("1.5.0") - def setThresholds(self, value): - """ - Sets the value of :py:attr:`thresholds`. - Clears value of :py:attr:`threshold` if it has been set. - """ - self._set(thresholds=value) - self.clear(self.threshold) - return self - - @since("1.5.0") - def getThresholds(self): - """ - If :py:attr:`thresholds` is set, return its value. - Otherwise, if :py:attr:`threshold` is set, return the equivalent thresholds for binary - classification: (1-threshold, threshold). - If neither are set, throw an error. - """ - self._checkThresholdConsistency() - if not self.isSet(self.thresholds) and self.isSet(self.threshold): - t = self.getOrDefault(self.threshold) - return [1.0-t, t] - else: - return self.getOrDefault(self.thresholds) - - def _checkThresholdConsistency(self): - if self.isSet(self.threshold) and self.isSet(self.thresholds): - ts = self.getOrDefault(self.thresholds) - if len(ts) != 2: - raise ValueError("Logistic Regression getThreshold only applies to" + - " binary classification, but thresholds has length != 2." + - " thresholds: {0}".format(str(ts))) - t = 1.0/(1.0 + ts[0]/ts[1]) - t2 = self.getOrDefault(self.threshold) - if abs(t2 - t) >= 1E-5: - raise ValueError("Logistic Regression getThreshold found inconsistent values for" + - " threshold (%g) and thresholds (equivalent to %g)" % (t2, t)) - @since("2.1.0") def setFamily(self, value): """ @@ -515,13 +575,6 @@ def setFamily(self, value): """ return self._set(family=value) - @since("2.1.0") - def getFamily(self): - """ - Gets the value of :py:attr:`family` or its default value. - """ - return self.getOrDefault(self.family) - @since("2.3.0") def setLowerBoundsOnCoefficients(self, value): """ @@ -529,13 +582,6 @@ def setLowerBoundsOnCoefficients(self, value): """ return self._set(lowerBoundsOnCoefficients=value) - @since("2.3.0") - def getLowerBoundsOnCoefficients(self): - """ - Gets the value of :py:attr:`lowerBoundsOnCoefficients` - """ - return self.getOrDefault(self.lowerBoundsOnCoefficients) - @since("2.3.0") def setUpperBoundsOnCoefficients(self, value): """ @@ -543,13 +589,6 @@ def setUpperBoundsOnCoefficients(self, value): """ return self._set(upperBoundsOnCoefficients=value) - @since("2.3.0") - def getUpperBoundsOnCoefficients(self): - """ - Gets the value of :py:attr:`upperBoundsOnCoefficients` - """ - return self.getOrDefault(self.upperBoundsOnCoefficients) - @since("2.3.0") def setLowerBoundsOnIntercepts(self, value): """ @@ -557,13 +596,6 @@ def setLowerBoundsOnIntercepts(self, value): """ return self._set(lowerBoundsOnIntercepts=value) - @since("2.3.0") - def getLowerBoundsOnIntercepts(self): - """ - Gets the value of :py:attr:`lowerBoundsOnIntercepts` - """ - return self.getOrDefault(self.lowerBoundsOnIntercepts) - @since("2.3.0") def setUpperBoundsOnIntercepts(self, value): """ @@ -571,16 +603,9 @@ def setUpperBoundsOnIntercepts(self, value): """ return self._set(upperBoundsOnIntercepts=value) - @since("2.3.0") - def getUpperBoundsOnIntercepts(self): - """ - Gets the value of :py:attr:`upperBoundsOnIntercepts` - """ - return self.getOrDefault(self.upperBoundsOnIntercepts) - -class LogisticRegressionModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable, - HasTrainingSummary): +class LogisticRegressionModel(JavaProbabilisticClassificationModel, _LogisticRegressionParams, + JavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by LogisticRegression. @@ -1647,8 +1672,36 @@ def evaluateEachIteration(self, dataset): return self._call_java("evaluateEachIteration", dataset) +class _NaiveBayesParams(_JavaPredictorParams, HasWeightCol): + """ + Params for :py:class:`NaiveBayes` and :py:class:`NaiveBayesModel`. + + .. versionadded:: 3.0.0 + """ + + smoothing = Param(Params._dummy(), "smoothing", "The smoothing parameter, should be >= 0, " + + "default is 1.0", typeConverter=TypeConverters.toFloat) + modelType = Param(Params._dummy(), "modelType", "The model type which is a string " + + "(case-sensitive). Supported options: multinomial (default) and bernoulli.", + typeConverter=TypeConverters.toString) + + @since("1.5.0") + def getSmoothing(self): + """ + Gets the value of smoothing or its default value. + """ + return self.getOrDefault(self.smoothing) + + @since("1.5.0") + def getModelType(self): + """ + Gets the value of modelType or its default value. + """ + return self.getOrDefault(self.modelType) + + @inherit_doc -class NaiveBayes(JavaProbabilisticClassifier, HasThresholds, HasWeightCol, +class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, HasWeightCol, JavaMLWritable, JavaMLReadable): """ Naive Bayes Classifiers. @@ -1674,6 +1727,8 @@ class NaiveBayes(JavaProbabilisticClassifier, HasThresholds, HasWeightCol, NaiveBayes_... >>> model.getLabelCol() 'newLabel' + >>> model.getSmoothing() + 1.0 >>> model.pi DenseVector([-0.81..., -0.58...]) >>> model.theta @@ -1712,12 +1767,6 @@ class NaiveBayes(JavaProbabilisticClassifier, HasThresholds, HasWeightCol, .. versionadded:: 1.5.0 """ - smoothing = Param(Params._dummy(), "smoothing", "The smoothing parameter, should be >= 0, " + - "default is 1.0", typeConverter=TypeConverters.toFloat) - modelType = Param(Params._dummy(), "modelType", "The model type which is a string " + - "(case-sensitive). Supported options: multinomial (default) and bernoulli.", - typeConverter=TypeConverters.toString) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", smoothing=1.0, @@ -1758,13 +1807,6 @@ def setSmoothing(self, value): """ return self._set(smoothing=value) - @since("1.5.0") - def getSmoothing(self): - """ - Gets the value of smoothing or its default value. - """ - return self.getOrDefault(self.smoothing) - @since("1.5.0") def setModelType(self, value): """ @@ -1772,15 +1814,9 @@ def setModelType(self, value): """ return self._set(modelType=value) - @since("1.5.0") - def getModelType(self): - """ - Gets the value of modelType or its default value. - """ - return self.getOrDefault(self.modelType) - -class NaiveBayesModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable): +class NaiveBayesModel(JavaProbabilisticClassificationModel, _NaiveBayesParams, JavaMLWritable, + JavaMLReadable): """ Model fitted by NaiveBayes. @@ -1804,9 +1840,60 @@ def theta(self): return self._call_java("theta") +class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, HasMaxIter, + HasTol, HasStepSize, HasSolver): + """ + Params for :py:class:`MultilayerPerceptronClassifier`. + + .. versionadded:: 3.0.0 + """ + + layers = Param(Params._dummy(), "layers", "Sizes of layers from input layer to output layer " + + "E.g., Array(780, 100, 10) means 780 inputs, one hidden layer with 100 " + + "neurons and output layer of 10 neurons.", + typeConverter=TypeConverters.toListInt) + blockSize = Param(Params._dummy(), "blockSize", "Block size for stacking input data in " + + "matrices. Data is stacked within partitions. If block size is more than " + + "remaining data in a partition then it is adjusted to the size of this " + + "data. Recommended size is between 10 and 1000, default is 128.", + typeConverter=TypeConverters.toInt) + solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + + "options: l-bfgs, gd.", typeConverter=TypeConverters.toString) + initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", + typeConverter=TypeConverters.toVector) + + @since("1.6.0") + def getLayers(self): + """ + Gets the value of layers or its default value. + """ + return self.getOrDefault(self.layers) + + @since("1.6.0") + def getBlockSize(self): + """ + Gets the value of blockSize or its default value. + """ + return self.getOrDefault(self.blockSize) + + @since("2.0.0") + def getStepSize(self): + """ + Gets the value of stepSize or its default value. + """ + return self.getOrDefault(self.stepSize) + + @since("2.0.0") + def getInitialWeights(self): + """ + Gets the value of initialWeights or its default value. + """ + return self.getOrDefault(self.initialWeights) + + @inherit_doc -class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, HasMaxIter, HasTol, HasSeed, - HasStepSize, HasSolver, JavaMLWritable, JavaMLReadable): +class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, _MultilayerPerceptronParams, + JavaMLWritable, JavaMLReadable): """ Classifier trainer based on the Multilayer Perceptron. Each layer has sigmoid activation function, output layer has softmax. @@ -1862,20 +1949,6 @@ class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, HasMaxIter, Ha .. versionadded:: 1.6.0 """ - layers = Param(Params._dummy(), "layers", "Sizes of layers from input layer to output layer " + - "E.g., Array(780, 100, 10) means 780 inputs, one hidden layer with 100 " + - "neurons and output layer of 10 neurons.", - typeConverter=TypeConverters.toListInt) - blockSize = Param(Params._dummy(), "blockSize", "Block size for stacking input data in " + - "matrices. Data is stacked within partitions. If block size is more than " + - "remaining data in a partition then it is adjusted to the size of this " + - "data. Recommended size is between 10 and 1000, default is 128.", - typeConverter=TypeConverters.toInt) - solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + - "options: l-bfgs, gd.", typeConverter=TypeConverters.toString) - initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", - typeConverter=TypeConverters.toVector) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, tol=1e-6, seed=None, layers=None, blockSize=128, stepSize=0.03, @@ -1920,13 +1993,6 @@ def setLayers(self, value): """ return self._set(layers=value) - @since("1.6.0") - def getLayers(self): - """ - Gets the value of layers or its default value. - """ - return self.getOrDefault(self.layers) - @since("1.6.0") def setBlockSize(self, value): """ @@ -1934,13 +2000,6 @@ def setBlockSize(self, value): """ return self._set(blockSize=value) - @since("1.6.0") - def getBlockSize(self): - """ - Gets the value of blockSize or its default value. - """ - return self.getOrDefault(self.blockSize) - @since("2.0.0") def setStepSize(self, value): """ @@ -1948,13 +2007,6 @@ def setStepSize(self, value): """ return self._set(stepSize=value) - @since("2.0.0") - def getStepSize(self): - """ - Gets the value of stepSize or its default value. - """ - return self.getOrDefault(self.stepSize) - @since("2.0.0") def setInitialWeights(self, value): """ @@ -1962,13 +2014,6 @@ def setInitialWeights(self, value): """ return self._set(initialWeights=value) - @since("2.0.0") - def getInitialWeights(self): - """ - Gets the value of initialWeights or its default value. - """ - return self.getOrDefault(self.initialWeights) - class MultilayerPerceptronClassificationModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable): diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 10450aca2322c..147ebed1d633a 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -24,7 +24,7 @@ _HasVarianceImpurity, _TreeRegressorParams from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams, \ - JavaPredictor, JavaPredictionModel, JavaWrapper + JavaPredictor, JavaPredictionModel, _JavaPredictorParams, JavaWrapper from pyspark.ml.common import inherit_doc from pyspark.sql import DataFrame @@ -40,10 +40,35 @@ 'RandomForestRegressor', 'RandomForestRegressionModel'] +class _LinearRegressionParams(_JavaPredictorParams, HasRegParam, HasElasticNetParam, HasMaxIter, + HasTol, HasFitIntercept, HasStandardization, HasWeightCol, HasSolver, + HasAggregationDepth, HasLoss): + """ + Params for :py:class:`LinearRegression` and :py:class:`LinearRegressionModel`. + + .. versionadded:: 3.0.0 + """ + + solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + + "options: auto, normal, l-bfgs.", typeConverter=TypeConverters.toString) + + loss = Param(Params._dummy(), "loss", "The loss function to be optimized. Supported " + + "options: squaredError, huber.", typeConverter=TypeConverters.toString) + + epsilon = Param(Params._dummy(), "epsilon", "The shape parameter to control the amount of " + + "robustness. Must be > 1.0. Only valid when loss is huber", + typeConverter=TypeConverters.toFloat) + + @since("2.3.0") + def getEpsilon(self): + """ + Gets the value of epsilon or its default value. + """ + return self.getOrDefault(self.epsilon) + + @inherit_doc -class LinearRegression(JavaPredictor, HasMaxIter, HasRegParam, HasTol, HasElasticNetParam, - HasFitIntercept, HasStandardization, HasSolver, HasWeightCol, - HasAggregationDepth, HasLoss, JavaMLWritable, JavaMLReadable): +class LinearRegression(JavaPredictor, _LinearRegressionParams, JavaMLWritable, JavaMLReadable): """ Linear regression. @@ -73,6 +98,8 @@ class LinearRegression(JavaPredictor, HasMaxIter, HasRegParam, HasTol, HasElasti LinearRegression... >>> model.setPredictionCol("newPrediction") LinearRegression... + >>> model.getMaxIter() + 5 >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> abs(model.predict(test0.head().features) - (-1.0)) < 0.001 True @@ -108,16 +135,6 @@ class LinearRegression(JavaPredictor, HasMaxIter, HasRegParam, HasTol, HasElasti .. versionadded:: 1.4.0 """ - solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + - "options: auto, normal, l-bfgs.", typeConverter=TypeConverters.toString) - - loss = Param(Params._dummy(), "loss", "The loss function to be optimized. Supported " + - "options: squaredError, huber.", typeConverter=TypeConverters.toString) - - epsilon = Param(Params._dummy(), "epsilon", "The shape parameter to control the amount of " + - "robustness. Must be > 1.0. Only valid when loss is huber", - typeConverter=TypeConverters.toFloat) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, @@ -162,16 +179,9 @@ def setEpsilon(self, value): """ return self._set(epsilon=value) - @since("2.3.0") - def getEpsilon(self): - """ - Gets the value of epsilon or its default value. - """ - return self.getOrDefault(self.epsilon) - -class LinearRegressionModel(JavaPredictionModel, GeneralJavaMLWritable, JavaMLReadable, - HasTrainingSummary): +class LinearRegressionModel(JavaPredictionModel, _LinearRegressionParams, GeneralJavaMLWritable, + JavaMLReadable, HasTrainingSummary): """ Model fitted by :class:`LinearRegression`. @@ -468,7 +478,7 @@ def totalIterations(self): return self._call_java("totalIterations") -class _IsotonicRegressionBase(HasFeaturesCol, HasLabelCol, HasPredictionCol, HasWeightCol): +class _IsotonicRegressionParams(HasFeaturesCol, HasLabelCol, HasPredictionCol, HasWeightCol): """ Params for :py:class:`IsotonicRegression` and :py:class:`IsotonicRegressionModel`. @@ -498,7 +508,7 @@ def getFeatureIndex(self): @inherit_doc -class IsotonicRegression(JavaEstimator, _IsotonicRegressionBase, HasWeightCol, +class IsotonicRegression(JavaEstimator, _IsotonicRegressionParams, HasWeightCol, JavaMLWritable, JavaMLReadable): """ Currently implemented using parallelized pool adjacent violators algorithm. @@ -577,8 +587,8 @@ def setFeatureIndex(self, value): return self._set(featureIndex=value) -class IsotonicRegressionModel(JavaModel, _IsotonicRegressionBase, - JavaMLWritable, JavaMLReadable): +class IsotonicRegressionModel(JavaModel, _IsotonicRegressionParams, JavaMLWritable, + JavaMLReadable): """ Model fitted by :class:`IsotonicRegression`. @@ -1460,9 +1470,85 @@ def predict(self, features): return self._call_java("predict", features) +class _GeneralizedLinearRegressionParams(_JavaPredictorParams, HasFitIntercept, HasMaxIter, + HasTol, HasRegParam, HasWeightCol, HasSolver): + """ + Params for :py:class:`GeneralizedLinearRegression` and + :py:class:`GeneralizedLinearRegressionModel`. + + .. versionadded:: 3.0.0 + """ + + family = Param(Params._dummy(), "family", "The name of family which is a description of " + + "the error distribution to be used in the model. Supported options: " + + "gaussian (default), binomial, poisson, gamma and tweedie.", + typeConverter=TypeConverters.toString) + link = Param(Params._dummy(), "link", "The name of link function which provides the " + + "relationship between the linear predictor and the mean of the distribution " + + "function. Supported options: identity, log, inverse, logit, probit, cloglog " + + "and sqrt.", typeConverter=TypeConverters.toString) + linkPredictionCol = Param(Params._dummy(), "linkPredictionCol", "link prediction (linear " + + "predictor) column name", typeConverter=TypeConverters.toString) + variancePower = Param(Params._dummy(), "variancePower", "The power in the variance function " + + "of the Tweedie distribution which characterizes the relationship " + + "between the variance and mean of the distribution. Only applicable " + + "for the Tweedie family. Supported values: 0 and [1, Inf).", + typeConverter=TypeConverters.toFloat) + linkPower = Param(Params._dummy(), "linkPower", "The index in the power link function. " + + "Only applicable to the Tweedie family.", + typeConverter=TypeConverters.toFloat) + solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + + "options: irls.", typeConverter=TypeConverters.toString) + offsetCol = Param(Params._dummy(), "offsetCol", "The offset column name. If this is not set " + + "or empty, we treat all instance offsets as 0.0", + typeConverter=TypeConverters.toString) + + @since("2.0.0") + def getFamily(self): + """ + Gets the value of family or its default value. + """ + return self.getOrDefault(self.family) + + @since("2.0.0") + def getLinkPredictionCol(self): + """ + Gets the value of linkPredictionCol or its default value. + """ + return self.getOrDefault(self.linkPredictionCol) + + @since("2.0.0") + def getLink(self): + """ + Gets the value of link or its default value. + """ + return self.getOrDefault(self.link) + + @since("2.2.0") + def getVariancePower(self): + """ + Gets the value of variancePower or its default value. + """ + return self.getOrDefault(self.variancePower) + + @since("2.2.0") + def getLinkPower(self): + """ + Gets the value of linkPower or its default value. + """ + return self.getOrDefault(self.linkPower) + + @since("2.3.0") + def getOffsetCol(self): + """ + Gets the value of offsetCol or its default value. + """ + return self.getOrDefault(self.offsetCol) + + @inherit_doc -class GeneralizedLinearRegression(JavaPredictor, HasFitIntercept, HasMaxIter, HasTol, HasRegParam, - HasWeightCol, HasSolver, JavaMLWritable, JavaMLReadable): +class GeneralizedLinearRegression(JavaPredictor, _GeneralizedLinearRegressionParams, + JavaMLWritable, JavaMLReadable): """ Generalized Linear Regression. @@ -1494,6 +1580,8 @@ class GeneralizedLinearRegression(JavaPredictor, HasFitIntercept, HasMaxIter, Ha >>> model = glr.fit(df) >>> model.setFeaturesCol("features") GeneralizedLinearRegression... + >>> model.getMaxIter() + 25 >>> transformed = model.transform(df) >>> abs(transformed.head().prediction - 1.5) < 0.001 True @@ -1521,30 +1609,6 @@ class GeneralizedLinearRegression(JavaPredictor, HasFitIntercept, HasMaxIter, Ha .. versionadded:: 2.0.0 """ - family = Param(Params._dummy(), "family", "The name of family which is a description of " + - "the error distribution to be used in the model. Supported options: " + - "gaussian (default), binomial, poisson, gamma and tweedie.", - typeConverter=TypeConverters.toString) - link = Param(Params._dummy(), "link", "The name of link function which provides the " + - "relationship between the linear predictor and the mean of the distribution " + - "function. Supported options: identity, log, inverse, logit, probit, cloglog " + - "and sqrt.", typeConverter=TypeConverters.toString) - linkPredictionCol = Param(Params._dummy(), "linkPredictionCol", "link prediction (linear " + - "predictor) column name", typeConverter=TypeConverters.toString) - variancePower = Param(Params._dummy(), "variancePower", "The power in the variance function " + - "of the Tweedie distribution which characterizes the relationship " + - "between the variance and mean of the distribution. Only applicable " + - "for the Tweedie family. Supported values: 0 and [1, Inf).", - typeConverter=TypeConverters.toFloat) - linkPower = Param(Params._dummy(), "linkPower", "The index in the power link function. " + - "Only applicable to the Tweedie family.", - typeConverter=TypeConverters.toFloat) - solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + - "options: irls.", typeConverter=TypeConverters.toString) - offsetCol = Param(Params._dummy(), "offsetCol", "The offset column name. If this is not set " + - "or empty, we treat all instance offsets as 0.0", - typeConverter=TypeConverters.toString) - @keyword_only def __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", family="gaussian", link=None, fitIntercept=True, maxIter=25, tol=1e-6, @@ -1591,13 +1655,6 @@ def setFamily(self, value): """ return self._set(family=value) - @since("2.0.0") - def getFamily(self): - """ - Gets the value of family or its default value. - """ - return self.getOrDefault(self.family) - @since("2.0.0") def setLinkPredictionCol(self, value): """ @@ -1605,13 +1662,6 @@ def setLinkPredictionCol(self, value): """ return self._set(linkPredictionCol=value) - @since("2.0.0") - def getLinkPredictionCol(self): - """ - Gets the value of linkPredictionCol or its default value. - """ - return self.getOrDefault(self.linkPredictionCol) - @since("2.0.0") def setLink(self, value): """ @@ -1619,13 +1669,6 @@ def setLink(self, value): """ return self._set(link=value) - @since("2.0.0") - def getLink(self): - """ - Gets the value of link or its default value. - """ - return self.getOrDefault(self.link) - @since("2.2.0") def setVariancePower(self, value): """ @@ -1633,13 +1676,6 @@ def setVariancePower(self, value): """ return self._set(variancePower=value) - @since("2.2.0") - def getVariancePower(self): - """ - Gets the value of variancePower or its default value. - """ - return self.getOrDefault(self.variancePower) - @since("2.2.0") def setLinkPower(self, value): """ @@ -1647,13 +1683,6 @@ def setLinkPower(self, value): """ return self._set(linkPower=value) - @since("2.2.0") - def getLinkPower(self): - """ - Gets the value of linkPower or its default value. - """ - return self.getOrDefault(self.linkPower) - @since("2.3.0") def setOffsetCol(self, value): """ @@ -1661,16 +1690,9 @@ def setOffsetCol(self, value): """ return self._set(offsetCol=value) - @since("2.3.0") - def getOffsetCol(self): - """ - Gets the value of offsetCol or its default value. - """ - return self.getOrDefault(self.offsetCol) - -class GeneralizedLinearRegressionModel(JavaPredictionModel, JavaMLWritable, - JavaMLReadable, HasTrainingSummary): +class GeneralizedLinearRegressionModel(JavaPredictionModel, _GeneralizedLinearRegressionParams, + JavaMLWritable, JavaMLReadable, HasTrainingSummary): """ Model fitted by :class:`GeneralizedLinearRegression`. From dba673f0e306ad1564e4e2e92a3c95b9f60f5ae0 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 18 Oct 2019 17:57:13 +0800 Subject: [PATCH 0361/1387] [SPARK-29489][ML][PYSPARK] ml.evaluation support log-loss ### What changes were proposed in this pull request? `ml.MulticlassClassificationEvaluator` & `mllib.MulticlassMetrics` support log-loss ### Why are the changes needed? log-loss is an important classification metric and is widely used in practice ### Does this PR introduce any user-facing change? Yes, add new option ("logloss") and a related param `eps` ### How was this patch tested? added testsuites & local tests refering to sklearn Closes #26135 from zhengruifeng/logloss. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../MulticlassClassificationEvaluator.scala | 77 +++++++++++++----- .../mllib/evaluation/MulticlassMetrics.scala | 62 +++++++++++---- ...lticlassClassificationEvaluatorSuite.scala | 20 +++++ .../evaluation/MulticlassMetricsSuite.scala | 78 +++++++++++++++++++ python/pyspark/ml/evaluation.py | 59 ++++++++++---- python/pyspark/mllib/evaluation.py | 22 +++++- 6 files changed, 270 insertions(+), 48 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 85a6138c98a46..bac3e23774391 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -18,21 +18,23 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol, HasWeightCol} -import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType /** - * Evaluator for multiclass classification, which expects two input columns: prediction and label. + * Evaluator for multiclass classification, which expects input columns: prediction, label, + * weight (optional) and probability (only for logLoss). */ @Since("1.5.0") class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") override val uid: String) - extends Evaluator with HasPredictionCol with HasLabelCol - with HasWeightCol with DefaultParamsWritable { + extends Evaluator with HasPredictionCol with HasLabelCol with HasWeightCol + with HasProbabilityCol with DefaultParamsWritable { import MulticlassClassificationEvaluator.supportedMetricNames @@ -40,8 +42,12 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid def this() = this(Identifiable.randomUID("mcEval")) /** - * param for metric name in evaluation (supports `"f1"` (default), `"weightedPrecision"`, - * `"weightedRecall"`, `"accuracy"`) + * param for metric name in evaluation (supports `"f1"` (default), `"accuracy"`, + * `"weightedPrecision"`, `"weightedRecall"`, `"weightedTruePositiveRate"`, + * `"weightedFalsePositiveRate"`, `"weightedFMeasure"`, `"truePositiveRateByLabel"`, + * `"falsePositiveRateByLabel"`, `"precisionByLabel"`, `"recallByLabel"`, + * `"fMeasureByLabel"`, `"logLoss"`) + * * @group param */ @Since("1.5.0") @@ -71,6 +77,10 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("3.0.0") def setWeightCol(value: String): this.type = set(weightCol, value) + /** @group setParam */ + @Since("3.0.0") + def setProbabilityCol(value: String): this.type = set(probabilityCol, value) + @Since("3.0.0") final val metricLabel: DoubleParam = new DoubleParam(this, "metricLabel", "The class whose metric will be computed in " + @@ -104,6 +114,21 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid setDefault(beta -> 1.0) + @Since("3.0.0") + final val eps: DoubleParam = new DoubleParam(this, "eps", + "log-loss is undefined for p=0 or p=1, so probabilities are clipped to " + + "max(eps, min(1 - eps, p)).", + ParamValidators.inRange(0, 0.5, false, false)) + + /** @group getParam */ + @Since("3.0.0") + def getEps: Double = $(eps) + + /** @group setParam */ + @Since("3.0.0") + def setEps(value: Double): this.type = set(eps, value) + + setDefault(eps -> 1e-15) @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { @@ -111,13 +136,29 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid SchemaUtils.checkColumnType(schema, $(predictionCol), DoubleType) SchemaUtils.checkNumericType(schema, $(labelCol)) - val predictionAndLabelsWithWeights = - dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType), - if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol))) + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + val rdd = if ($(metricName) == "logLoss") { + // probabilityCol is only needed to compute logloss + require(isDefined(probabilityCol) && $(probabilityCol).nonEmpty) + val p = DatasetUtils.columnToVector(dataset, $(probabilityCol)) + dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType), w, p) + .rdd.map { + case Row(prediction: Double, label: Double, weight: Double, probability: Vector) => + (prediction, label, weight, probability.toArray) + } + } else { + dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType), w) .rdd.map { case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight) } - val metrics = new MulticlassMetrics(predictionAndLabelsWithWeights) + } + + val metrics = new MulticlassMetrics(rdd) $(metricName) match { case "f1" => metrics.weightedFMeasure case "accuracy" => metrics.accuracy @@ -131,16 +172,14 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid case "precisionByLabel" => metrics.precision($(metricLabel)) case "recallByLabel" => metrics.recall($(metricLabel)) case "fMeasureByLabel" => metrics.fMeasure($(metricLabel), $(beta)) + case "logLoss" => metrics.logLoss($(eps)) } } @Since("1.5.0") - override def isLargerBetter: Boolean = { - $(metricName) match { - case "weightedFalsePositiveRate" => false - case "falsePositiveRateByLabel" => false - case _ => true - } + override def isLargerBetter: Boolean = $(metricName) match { + case "weightedFalsePositiveRate" | "falsePositiveRateByLabel" | "logLoss" => false + case _ => true } @Since("1.5.0") @@ -154,7 +193,7 @@ object MulticlassClassificationEvaluator private val supportedMetricNames = Array("f1", "accuracy", "weightedPrecision", "weightedRecall", "weightedTruePositiveRate", "weightedFalsePositiveRate", "weightedFMeasure", "truePositiveRateByLabel", "falsePositiveRateByLabel", "precisionByLabel", "recallByLabel", - "fMeasureByLabel") + "fMeasureByLabel", "logLoss") @Since("1.6.0") override def load(path: String): MulticlassClassificationEvaluator = super.load(path) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index c8245bf500a2f..9518f7e6828cf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -23,33 +23,35 @@ import scala.collection.mutable import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{Matrices, Matrix} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.DataFrame /** * Evaluator for multiclass classification. * - * @param predictionAndLabels an RDD of (prediction, label, weight) or - * (prediction, label) tuples. + * @param predictionAndLabels an RDD of (prediction, label, weight, probability) or + * (prediction, label, weight) or (prediction, label) tuples. */ @Since("1.1.0") class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[_ <: Product]) { /** * An auxiliary constructor taking a DataFrame. - * @param predictionAndLabels a DataFrame with two double columns: prediction and label + * @param predictionAndLabels a DataFrame with columns: prediction, label, weight (optional) + * and probability (only for logLoss) */ private[mllib] def this(predictionAndLabels: DataFrame) = - this(predictionAndLabels.rdd.map { - case Row(prediction: Double, label: Double, weight: Double) => - (prediction, label, weight) - case Row(prediction: Double, label: Double) => - (prediction, label, 1.0) - case other => - throw new IllegalArgumentException(s"Expected Row of tuples, got $other") + this(predictionAndLabels.rdd.map { r => + r.size match { + case 2 => (r.getDouble(0), r.getDouble(1), 1.0, null) + case 3 => (r.getDouble(0), r.getDouble(1), r.getDouble(2), null) + case 4 => (r.getDouble(0), r.getDouble(1), r.getDouble(2), r.getSeq[Double](3).toArray) + case _ => throw new IllegalArgumentException(s"Expected Row of tuples, got $r") + } }) - - private val confusions = predictionAndLabels.map { + private lazy val confusions = predictionAndLabels.map { + case (prediction: Double, label: Double, weight: Double, _) => + ((label, prediction), weight) case (prediction: Double, label: Double, weight: Double) => ((label, prediction), weight) case (prediction: Double, label: Double) => @@ -237,4 +239,38 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[_ <: Product]) */ @Since("1.1.0") lazy val labels: Array[Double] = tpByClass.keys.toArray.sorted + + /** + * Returns the log-loss, aka logistic loss or cross-entropy loss. + * @param eps log-loss is undefined for p=0 or p=1, so probabilities are + * clipped to max(eps, min(1 - eps, p)). + */ + @Since("3.0.0") + def logLoss(eps: Double = 1e-15): Double = { + require(eps > 0 && eps < 0.5, s"eps must be in range (0, 0.5), but got $eps") + val loss1 = - math.log(eps) + val loss2 = - math.log1p(-eps) + + val (lossSum, weightSum) = predictionAndLabels.map { + case (_, label: Double, weight: Double, probability: Array[Double]) => + require(label.toInt == label && label >= 0, s"Invalid label $label") + require(probability != null, "probability of each class can not be null") + val p = probability(label.toInt) + val loss = if (p < eps) { + loss1 + } else if (p > 1 - eps) { + loss2 + } else { + - math.log(p) + } + (loss * weight, weight) + + case other => + throw new IllegalArgumentException(s"Expected quadruples, got $other") + }.treeReduce { case ((l1, w1), (l2, w2)) => + (l1 + l2, w1 + w2) + } + + lossSum / weightSum + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala index b41df1b798af6..5b5212abdf7cc 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.ml.evaluation import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -60,4 +61,23 @@ class MulticlassClassificationEvaluatorSuite .setMetricLabel(1.0) assert(evaluator.evaluate(predictionAndLabels) ~== 3.0 / 4 absTol 1e-5) } + + test("MulticlassClassificationEvaluator support logloss") { + val labels = Seq(1.0, 2.0, 0.0, 1.0) + val probabilities = Seq( + Vectors.dense(0.1, 0.8, 0.1), + Vectors.dense(0.9, 0.05, 0.05), + Vectors.dense(0.8, 0.2, 0.0), + Vectors.dense(0.3, 0.65, 0.05)) + + val df = sc.parallelize(labels.zip(probabilities)).map { + case (label, probability) => + val prediction = probability.argmax.toDouble + (prediction, label, probability) + }.toDF("prediction", "label", "probability") + + val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("logLoss") + assert(evaluator.evaluate(df) ~== 0.9682005730687164 absTol 1e-5) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index 8779de590a256..e10295c905cdb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -176,4 +176,82 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { (weight0 * f2measure0 + weight1 * f2measure1 + weight2 * f2measure2) relTol delta) assert(metrics.labels === labels) } + + test("MulticlassMetrics supports binary class log-loss") { + /* + Using the following Python code to verify the correctness. + + from sklearn.metrics import log_loss + labels = [1, 0, 0, 1] + probabilities = [[.1, .9], [.9, .1], [.8, .2], [.35, .65]] + weights = [1.5, 2.0, 1.0, 0.5] + + >>> log_loss(y_true=labels, y_pred=probabilities, sample_weight=weights) + 0.16145936283256573 + >>> log_loss(y_true=labels, y_pred=probabilities) + 0.21616187468057912 + */ + + val labels = Seq(1.0, 0.0, 0.0, 1.0) + val probabilities = Seq( + Array(0.1, 0.9), + Array(0.9, 0.1), + Array(0.8, 0.2), + Array(0.35, 0.65)) + val weights = Seq(1.5, 2.0, 1.0, 0.5) + + val rdd = sc.parallelize(labels.zip(weights).zip(probabilities)).map { + case ((label, weight), probability) => + val prediction = probability.indexOf(probability.max).toDouble + (prediction, label, weight, probability) + } + val metrics = new MulticlassMetrics(rdd) + assert(metrics.logLoss() ~== 0.16145936283256573 relTol delta) + + val rdd2 = rdd.map { + case (prediction: Double, label: Double, weight: Double, probability: Array[Double]) => + (prediction, label, 1.0, probability) + } + val metrics2 = new MulticlassMetrics(rdd2) + assert(metrics2.logLoss() ~== 0.21616187468057912 relTol delta) + } + + test("MulticlassMetrics supports multi-class log-loss") { + /* + Using the following Python code to verify the correctness. + + from sklearn.metrics import log_loss + labels = [1, 2, 0, 1] + probabilities = [[.1, .8, .1], [.9, .05, .05], [.8, .2, .0], [.3, .65, .05]] + weights = [1.5, 2.0, 1.0, 0.5] + + >>> log_loss(y_true=labels, y_pred=probabilities, sample_weight=weights) + 1.3529429766879466 + >>> log_loss(y_true=labels, y_pred=probabilities) + 0.9682005730687164 + */ + + val labels = Seq(1.0, 2.0, 0.0, 1.0) + val probabilities = Seq( + Array(0.1, 0.8, 0.1), + Array(0.9, 0.05, 0.05), + Array(0.8, 0.2, 0.0), + Array(0.3, 0.65, 0.05)) + val weights = Seq(1.5, 2.0, 1.0, 0.5) + + val rdd = sc.parallelize(labels.zip(weights).zip(probabilities)).map { + case ((label, weight), probability) => + val prediction = probability.indexOf(probability.max).toDouble + (prediction, label, weight, probability) + } + val metrics = new MulticlassMetrics(rdd) + assert(metrics.logLoss() ~== 1.3529429766879466 relTol delta) + + val rdd2 = rdd.map { + case (prediction: Double, label: Double, weight: Double, probability: Array[Double]) => + (prediction, label, 1.0, probability) + } + val metrics2 = new MulticlassMetrics(rdd2) + assert(metrics2.logLoss() ~== 0.9682005730687164 relTol delta) + } } diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 6917cce615aa4..cdd9be7bf11b3 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -21,8 +21,8 @@ from pyspark import since, keyword_only from pyspark.ml.wrapper import JavaParams from pyspark.ml.param import Param, Params, TypeConverters -from pyspark.ml.param.shared import HasLabelCol, HasPredictionCol, HasRawPredictionCol, \ - HasFeaturesCol, HasWeightCol +from pyspark.ml.param.shared import HasLabelCol, HasPredictionCol, HasProbabilityCol, \ + HasRawPredictionCol, HasFeaturesCol, HasWeightCol from pyspark.ml.common import inherit_doc from pyspark.ml.util import JavaMLReadable, JavaMLWritable @@ -314,15 +314,14 @@ def setParams(self, predictionCol="prediction", labelCol="label", @inherit_doc class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol, HasWeightCol, - JavaMLReadable, JavaMLWritable): + HasProbabilityCol, JavaMLReadable, JavaMLWritable): """ - Evaluator for Multiclass Classification, which expects two input - columns: prediction and label. + Evaluator for Multiclass Classification, which expects input + columns: prediction, label, weight (optional) and probabilityCol (only for logLoss). >>> scoreAndLabels = [(0.0, 0.0), (0.0, 1.0), (0.0, 0.0), ... (1.0, 0.0), (1.0, 1.0), (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)] >>> dataset = spark.createDataFrame(scoreAndLabels, ["prediction", "label"]) - ... >>> evaluator = MulticlassClassificationEvaluator(predictionCol="prediction") >>> evaluator.evaluate(dataset) 0.66... @@ -340,13 +339,23 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio ... (1.0, 0.0, 1.0), (1.0, 1.0, 1.0), (1.0, 1.0, 1.0), (1.0, 1.0, 1.0), ... (2.0, 2.0, 1.0), (2.0, 0.0, 1.0)] >>> dataset = spark.createDataFrame(scoreAndLabelsAndWeight, ["prediction", "label", "weight"]) - ... >>> evaluator = MulticlassClassificationEvaluator(predictionCol="prediction", ... weightCol="weight") >>> evaluator.evaluate(dataset) 0.66... >>> evaluator.evaluate(dataset, {evaluator.metricName: "accuracy"}) 0.66... + >>> predictionAndLabelsWithProbabilities = [ + ... (1.0, 1.0, 1.0, [0.1, 0.8, 0.1]), (0.0, 2.0, 1.0, [0.9, 0.05, 0.05]), + ... (0.0, 0.0, 1.0, [0.8, 0.2, 0.0]), (1.0, 1.0, 1.0, [0.3, 0.65, 0.05])] + >>> dataset = spark.createDataFrame(predictionAndLabelsWithProbabilities, ["prediction", + ... "label", "weight", "probability"]) + >>> evaluator = MulticlassClassificationEvaluator(predictionCol="prediction", + ... probabilityCol="probability") + >>> evaluator.setMetricName("logLoss") + MulticlassClassificationEvaluator... + >>> evaluator.evaluate(dataset) + 0.9682... .. versionadded:: 1.5.0 """ @@ -354,7 +363,8 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio "metric name in evaluation " "(f1|accuracy|weightedPrecision|weightedRecall|weightedTruePositiveRate|" "weightedFalsePositiveRate|weightedFMeasure|truePositiveRateByLabel|" - "falsePositiveRateByLabel|precisionByLabel|recallByLabel|fMeasureByLabel)", + "falsePositiveRateByLabel|precisionByLabel|recallByLabel|fMeasureByLabel|" + "logLoss)", typeConverter=TypeConverters.toString) metricLabel = Param(Params._dummy(), "metricLabel", "The class whose metric will be computed in truePositiveRateByLabel|" @@ -365,18 +375,25 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio "The beta value used in weightedFMeasure|fMeasureByLabel." " Must be > 0. The default value is 1.", typeConverter=TypeConverters.toFloat) + eps = Param(Params._dummy(), "eps", + "log-loss is undefined for p=0 or p=1, so probabilities are clipped to " + "max(eps, min(1 - eps, p)). " + "Must be in range (0, 0.5). The default value is 1e-15.", + typeConverter=TypeConverters.toFloat) @keyword_only def __init__(self, predictionCol="prediction", labelCol="label", - metricName="f1", weightCol=None, metricLabel=0.0, beta=1.0): + metricName="f1", weightCol=None, metricLabel=0.0, beta=1.0, + probabilityCol="probability", eps=1e-15): """ __init__(self, predictionCol="prediction", labelCol="label", \ - metricName="f1", weightCol=None, metricLabel=0.0, beta=1.0) + metricName="f1", weightCol=None, metricLabel=0.0, beta=1.0, \ + probabilityCol="probability", eps=1e-15) """ super(MulticlassClassificationEvaluator, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator", self.uid) - self._setDefault(metricName="f1", metricLabel=0.0, beta=1.0) + self._setDefault(metricName="f1", metricLabel=0.0, beta=1.0, eps=1e-15) kwargs = self._input_kwargs self._set(**kwargs) @@ -422,13 +439,29 @@ def getBeta(self): """ return self.getOrDefault(self.beta) + @since("3.0.0") + def setEps(self, value): + """ + Sets the value of :py:attr:`eps`. + """ + return self._set(eps=value) + + @since("3.0.0") + def getEps(self): + """ + Gets the value of eps or its default value. + """ + return self.getOrDefault(self.eps) + @keyword_only @since("1.5.0") def setParams(self, predictionCol="prediction", labelCol="label", - metricName="f1", weightCol=None, metricLabel=0.0, beta=1.0): + metricName="f1", weightCol=None, metricLabel=0.0, beta=1.0, + probabilityCol="probability", eps=1e-15): """ setParams(self, predictionCol="prediction", labelCol="label", \ - metricName="f1", weightCol=None, metricLabel=0.0, beta=1.0) + metricName="f1", weightCol=None, metricLabel=0.0, beta=1.0, \ + probabilityCol="probability", eps=1e-15) Sets params for multiclass classification evaluator. """ kwargs = self._input_kwargs diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index 30df4a1846d0b..f3be827fb6e4f 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -20,7 +20,7 @@ from pyspark import since from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc from pyspark.sql import SQLContext -from pyspark.sql.types import StructField, StructType, DoubleType +from pyspark.sql.types import ArrayType, StructField, StructType, DoubleType __all__ = ['BinaryClassificationMetrics', 'RegressionMetrics', 'MulticlassMetrics', 'RankingMetrics'] @@ -182,7 +182,8 @@ class MulticlassMetrics(JavaModelWrapper): """ Evaluator for multiclass classification. - :param predictionAndLabels: an RDD of prediction, label and optional weight. + :param predictionAndLabels: an RDD of prediction, label, optional weight + and optional probability. >>> predictionAndLabels = sc.parallelize([(0.0, 0.0), (0.0, 1.0), (0.0, 0.0), ... (1.0, 0.0), (1.0, 1.0), (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)]) @@ -239,6 +240,12 @@ class MulticlassMetrics(JavaModelWrapper): 0.66... >>> metrics.weightedFMeasure(2.0) 0.65... + >>> predictionAndLabelsWithProbabilities = sc.parallelize([ + ... (1.0, 1.0, 1.0, [0.1, 0.8, 0.1]), (0.0, 2.0, 1.0, [0.9, 0.05, 0.05]), + ... (0.0, 0.0, 1.0, [0.8, 0.2, 0.0]), (1.0, 1.0, 1.0, [0.3, 0.65, 0.05])]) + >>> metrics = MulticlassMetrics(predictionAndLabelsWithProbabilities) + >>> metrics.logLoss() + 0.9682... .. versionadded:: 1.4.0 """ @@ -250,8 +257,10 @@ def __init__(self, predictionAndLabels): schema = StructType([ StructField("prediction", DoubleType(), nullable=False), StructField("label", DoubleType(), nullable=False)]) - if numCol == 3: + if numCol >= 3: schema.add("weight", DoubleType(), False) + if numCol == 4: + schema.add("probability", ArrayType(DoubleType(), False), False) df = sql_ctx.createDataFrame(predictionAndLabels, schema) java_class = sc._jvm.org.apache.spark.mllib.evaluation.MulticlassMetrics java_model = java_class(df._jdf) @@ -356,6 +365,13 @@ def weightedFMeasure(self, beta=None): else: return self.call("weightedFMeasure", beta) + @since('3.0.0') + def logLoss(self, eps=1e-15): + """ + Returns weighted logLoss. + """ + return self.call("logLoss", eps) + class RankingMetrics(JavaModelWrapper): """ From 74351468de3ad47e6a331c7f2d9353d8c00be8a9 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 18 Oct 2019 12:55:49 +0200 Subject: [PATCH 0362/1387] [SPARK-29482][SQL] ANALYZE TABLE should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add `AnalyzeTableStatement` and `AnalyzeColumnStatement`, and make ANALYZE TABLE go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog ANALYZE TABLE t // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? yes. When running ANALYZE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? new tests Closes #26129 from cloud-fan/analyze-table. Authored-by: Wenchen Fan Signed-off-by: Gengliang Wang --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 48 ++++++++++++ .../catalyst/plans/logical/statements.scala | 19 +++++ .../sql/catalyst/parser/DDLParserSuite.scala | 76 +++++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 20 ++++- .../spark/sql/execution/SparkSqlParser.scala | 49 ------------ .../sql/connector/DataSourceV2SQLSuite.scala | 17 +++++ .../sql/execution/SparkSqlParserSuite.scala | 62 --------------- 8 files changed, 180 insertions(+), 113 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index a7824d50ca84b..878c08a79813c 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -124,7 +124,7 @@ statement (COMMENT comment=STRING) | (TBLPROPERTIES tableProps=tablePropertyList))* (AS? query)? #replaceTable - | ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS + | ANALYZE TABLE multipartIdentifier partitionSpec? COMPUTE STATISTICS (identifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze | ALTER TABLE multipartIdentifier ADD (COLUMN | COLUMNS) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d138ff401944f..6a108901dd329 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2660,4 +2660,52 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging isExtended) } } + + /** + * Create an [[AnalyzeTableStatement]], or an [[AnalyzeColumnStatement]]. + * Example SQL for analyzing a table or a set of partitions : + * {{{ + * ANALYZE TABLE multi_part_name [PARTITION (partcol1[=val1], partcol2[=val2], ...)] + * COMPUTE STATISTICS [NOSCAN]; + * }}} + * + * Example SQL for analyzing columns : + * {{{ + * ANALYZE TABLE multi_part_name COMPUTE STATISTICS FOR COLUMNS column1, column2; + * }}} + * + * Example SQL for analyzing all columns of a table: + * {{{ + * ANALYZE TABLE multi_part_name COMPUTE STATISTICS FOR ALL COLUMNS; + * }}} + */ + override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) { + def checkPartitionSpec(): Unit = { + if (ctx.partitionSpec != null) { + logWarning("Partition specification is ignored when collecting column statistics: " + + ctx.partitionSpec.getText) + } + } + if (ctx.identifier != null && + ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { + throw new ParseException(s"Expected `NOSCAN` instead of `${ctx.identifier.getText}`", ctx) + } + + val tableName = visitMultipartIdentifier(ctx.multipartIdentifier()) + if (ctx.ALL() != null) { + checkPartitionSpec() + AnalyzeColumnStatement(tableName, None, allColumns = true) + } else if (ctx.identifierSeq() == null) { + val partitionSpec = if (ctx.partitionSpec != null) { + visitPartitionSpec(ctx.partitionSpec) + } else { + Map.empty[String, Option[String]] + } + AnalyzeTableStatement(tableName, partitionSpec, noScan = ctx.identifier != null) + } else { + checkPartitionSpec() + AnalyzeColumnStatement( + tableName, Option(visitIdentifierSeq(ctx.identifierSeq())), allColumns = false) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 757a8bfe219f3..208ecd0e99c88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -292,3 +292,22 @@ case class ShowNamespacesStatement(namespace: Option[Seq[String]], pattern: Opti * A USE statement, as parsed from SQL. */ case class UseStatement(isNamespaceSet: Boolean, nameParts: Seq[String]) extends ParsedStatement + +/** + * An ANALYZE TABLE statement, as parsed from SQL. + */ +case class AnalyzeTableStatement( + tableName: Seq[String], + partitionSpec: Map[String, Option[String]], + noScan: Boolean) extends ParsedStatement + +/** + * An ANALYZE TABLE FOR COLUMNS statement, as parsed from SQL. + */ +case class AnalyzeColumnStatement( + tableName: Seq[String], + columnNames: Option[Seq[String]], + allColumns: Boolean) extends ParsedStatement { + require(columnNames.isDefined ^ allColumns, "Parameter `columnNames` or `allColumns` are " + + "mutually exclusive. Only one of them should be specified.") +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index d9e50ef09fdd5..b42fec201352e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -879,6 +879,82 @@ class DDLParserSuite extends AnalysisTest { ShowNamespacesStatement(Some(Seq("testcat", "ns1")), Some("*pattern*"))) } + test("analyze table statistics") { + comparePlans(parsePlan("analyze table a.b.c compute statistics"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map.empty, noScan = false)) + comparePlans(parsePlan("analyze table a.b.c compute statistics noscan"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map.empty, noScan = true)) + comparePlans(parsePlan("analyze table a.b.c partition (a) compute statistics nOscAn"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map("a" -> None), noScan = true)) + + // Partitions specified + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")), noScan = false)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")), noScan = true)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09') COMPUTE STATISTICS noscan"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09")), noScan = true)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09"), "hr" -> None), noScan = false)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS noscan"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> Some("2008-04-09"), "hr" -> None), noScan = true)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr=11) COMPUTE STATISTICS noscan"), + AnalyzeTableStatement( + Seq("a", "b", "c"), Map("ds" -> None, "hr" -> Some("11")), noScan = true)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr) COMPUTE STATISTICS"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map("ds" -> None, "hr" -> None), noScan = false)) + comparePlans( + parsePlan("ANALYZE TABLE a.b.c PARTITION(ds, hr) COMPUTE STATISTICS noscan"), + AnalyzeTableStatement(Seq("a", "b", "c"), Map("ds" -> None, "hr" -> None), noScan = true)) + + intercept("analyze table a.b.c compute statistics xxxx", + "Expected `NOSCAN` instead of `xxxx`") + intercept("analyze table a.b.c partition (a) compute statistics xxxx", + "Expected `NOSCAN` instead of `xxxx`") + } + + test("analyze table column statistics") { + intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS", "") + + comparePlans( + parsePlan("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS key, value"), + AnalyzeColumnStatement(Seq("a", "b", "c"), Option(Seq("key", "value")), allColumns = false)) + + // Partition specified - should be ignored + comparePlans( + parsePlan( + s""" + |ANALYZE TABLE a.b.c PARTITION(ds='2017-06-10') + |COMPUTE STATISTICS FOR COLUMNS key, value + """.stripMargin), + AnalyzeColumnStatement(Seq("a", "b", "c"), Option(Seq("key", "value")), allColumns = false)) + + // Partition specified should be ignored in case of COMPUTE STATISTICS FOR ALL COLUMNS + comparePlans( + parsePlan( + s""" + |ANALYZE TABLE a.b.c PARTITION(ds='2017-06-10') + |COMPUTE STATISTICS FOR ALL COLUMNS + """.stripMargin), + AnalyzeColumnStatement(Seq("a", "b", "c"), None, allColumns = true)) + + intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value", + "mismatched input 'key' expecting ") + intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL", + "missing 'COLUMNS' at ''") + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index a96533dac97ea..0097af310a9d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -265,6 +265,24 @@ class ResolveSessionCatalog( // TODO (SPARK-29014): we should check if the current catalog is session catalog here. case ShowTablesStatement(None, pattern) if defaultCatalog.isEmpty => ShowTablesCommand(None, pattern) + + case AnalyzeTableStatement(tableName, partitionSpec, noScan) => + val CatalogAndIdentifierParts(catalog, parts) = tableName + if (!isSessionCatalog(catalog)) { + throw new AnalysisException("ANALYZE TABLE is only supported with v1 tables.") + } + if (partitionSpec.isEmpty) { + AnalyzeTableCommand(parts.asTableIdentifier, noScan) + } else { + AnalyzePartitionCommand(parts.asTableIdentifier, partitionSpec, noScan) + } + + case AnalyzeColumnStatement(tableName, columnNames, allColumns) => + val CatalogAndIdentifierParts(catalog, parts) = tableName + if (!isSessionCatalog(catalog)) { + throw new AnalysisException("ANALYZE TABLE is only supported with v1 tables.") + } + AnalyzeColumnCommand(parts.asTableIdentifier, columnNames, allColumns) } private def buildCatalogTable( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3dd392156d522..df63787fa508b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -89,55 +89,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ResetCommand } - /** - * Create an [[AnalyzeTableCommand]] command, or an [[AnalyzePartitionCommand]] - * or an [[AnalyzeColumnCommand]] command. - * Example SQL for analyzing a table or a set of partitions : - * {{{ - * ANALYZE TABLE [db_name.]tablename [PARTITION (partcol1[=val1], partcol2[=val2], ...)] - * COMPUTE STATISTICS [NOSCAN]; - * }}} - * - * Example SQL for analyzing columns : - * {{{ - * ANALYZE TABLE [db_name.]tablename COMPUTE STATISTICS FOR COLUMNS column1, column2; - * }}} - * - * Example SQL for analyzing all columns of a table: - * {{{ - * ANALYZE TABLE [db_name.]tablename COMPUTE STATISTICS FOR ALL COLUMNS; - * }}} - */ - override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) { - def checkPartitionSpec(): Unit = { - if (ctx.partitionSpec != null) { - logWarning("Partition specification is ignored when collecting column statistics: " + - ctx.partitionSpec.getText) - } - } - if (ctx.identifier != null && - ctx.identifier.getText.toLowerCase(Locale.ROOT) != "noscan") { - throw new ParseException(s"Expected `NOSCAN` instead of `${ctx.identifier.getText}`", ctx) - } - - val table = visitTableIdentifier(ctx.tableIdentifier) - if (ctx.ALL() != null) { - checkPartitionSpec() - AnalyzeColumnCommand(table, None, allColumns = true) - } else if (ctx.identifierSeq() == null) { - if (ctx.partitionSpec != null) { - AnalyzePartitionCommand(table, visitPartitionSpec(ctx.partitionSpec), - noscan = ctx.identifier != null) - } else { - AnalyzeTableCommand(table, noscan = ctx.identifier != null) - } - } else { - checkPartitionSpec() - AnalyzeColumnCommand(table, - Option(visitIdentifierSeq(ctx.identifierSeq())), allColumns = false) - } - } - /** * Create a [[ShowTablesCommand]] logical plan. * Example SQL : diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 3b42c2374f006..79ee8f2dd9740 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1164,6 +1164,23 @@ class DataSourceV2SQLSuite } } + test("ANALYZE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + val e = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $t COMPUTE STATISTICS") + } + assert(e.message.contains("ANALYZE TABLE is only supported with v1 tables")) + + val e2 = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $t COMPUTE STATISTICS FOR ALL COLUMNS") + } + assert(e2.message.contains("ANALYZE TABLE is only supported with v1 tables")) + } + } + private def assertAnalysisError(sqlStatement: String, expectedError: String): Unit = { val errMsg = intercept[AnalysisException] { sql(sqlStatement) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 9da160888618f..0c766429742df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -216,68 +216,6 @@ class SparkSqlParserSuite extends AnalysisTest { assertEqual("DESCRIBE " + query, DescribeQueryCommand(query, parser.parsePlan(query))) } - test("analyze table statistics") { - assertEqual("analyze table t compute statistics", - AnalyzeTableCommand(TableIdentifier("t"), noscan = false)) - assertEqual("analyze table t compute statistics noscan", - AnalyzeTableCommand(TableIdentifier("t"), noscan = true)) - assertEqual("analyze table t partition (a) compute statistics nOscAn", - AnalyzePartitionCommand(TableIdentifier("t"), Map("a" -> None), noscan = true)) - - // Partitions specified - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, - partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")))) - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")))) - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09') COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> Some("2008-04-09")))) - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, - partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> None))) - assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> None))) - assertEqual("ANALYZE TABLE t PARTITION(ds, hr=11) COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> None, "hr" -> Some("11")))) - assertEqual("ANALYZE TABLE t PARTITION(ds, hr) COMPUTE STATISTICS", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, - partitionSpec = Map("ds" -> None, "hr" -> None))) - assertEqual("ANALYZE TABLE t PARTITION(ds, hr) COMPUTE STATISTICS noscan", - AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, - partitionSpec = Map("ds" -> None, "hr" -> None))) - - intercept("analyze table t compute statistics xxxx", - "Expected `NOSCAN` instead of `xxxx`") - intercept("analyze table t partition (a) compute statistics xxxx", - "Expected `NOSCAN` instead of `xxxx`") - } - - test("analyze table column statistics") { - intercept("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS", "") - - assertEqual("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS key, value", - AnalyzeColumnCommand(TableIdentifier("t"), Option(Seq("key", "value")), allColumns = false)) - - // Partition specified - should be ignored - assertEqual("ANALYZE TABLE t PARTITION(ds='2017-06-10') " + - "COMPUTE STATISTICS FOR COLUMNS key, value", - AnalyzeColumnCommand(TableIdentifier("t"), Option(Seq("key", "value")), allColumns = false)) - - // Partition specified should be ignored in case of COMPUTE STATISTICS FOR ALL COLUMNS - assertEqual("ANALYZE TABLE t PARTITION(ds='2017-06-10') " + - "COMPUTE STATISTICS FOR ALL COLUMNS", - AnalyzeColumnCommand(TableIdentifier("t"), None, allColumns = true)) - - intercept("ANALYZE TABLE t COMPUTE STATISTICS FOR ALL COLUMNS key, value", - "mismatched input 'key' expecting ") - intercept("ANALYZE TABLE t COMPUTE STATISTICS FOR ALL", - "missing 'COLUMNS' at ''") - } - test("query organization") { // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows val baseSql = "select * from t" From 39af51dbc6140f2c006a709d62df881bd203c453 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 18 Oct 2019 22:45:42 +0800 Subject: [PATCH 0363/1387] [SPARK-29014][SQL] DataSourceV2: Fix current/default catalog usage ### What changes were proposed in this pull request? The handling of the catalog across plans should be as follows ([SPARK-29014](https://issues.apache.org/jira/browse/SPARK-29014)): * The *current* catalog should be used when no catalog is specified * The default catalog is the catalog *current* is initialized to * If the *default* catalog is not set, then *current* catalog is the built-in Spark session catalog. This PR addresses the issue where *current* catalog usage is not followed as describe above. ### Why are the changes needed? It is a bug as described in the previous section. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit tests added. Closes #26120 from imback82/cleanup_catalog. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 42 +++++++++-------- .../catalyst/analysis/ResolveCatalogs.scala | 5 +- .../connector/catalog/CatalogManager.scala | 15 ++++-- .../sql/connector/catalog/LookupCatalog.scala | 31 ++----------- .../catalog/LookupCatalogSuite.scala | 46 ++++++++++--------- .../apache/spark/sql/DataFrameWriter.scala | 18 ++++---- .../apache/spark/sql/DataFrameWriterV2.scala | 5 +- .../analysis/ResolveSessionCatalog.scala | 3 +- ...SourceV2DataFrameSessionCatalogSuite.scala | 2 +- .../DataSourceV2SQLSessionCatalogSuite.scala | 2 +- .../sql/connector/DataSourceV2SQLSuite.scala | 17 +++++++ .../command/PlanResolutionSuite.scala | 6 +-- 12 files changed, 98 insertions(+), 94 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 46eca4a1d4800..b913a9618d6eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -125,9 +125,9 @@ class Analyzer( maxIterations: Int) extends RuleExecutor[LogicalPlan] with CheckAnalysis with LookupCatalog { - private val catalog: SessionCatalog = catalogManager.v1SessionCatalog + private val v1SessionCatalog: SessionCatalog = catalogManager.v1SessionCatalog - override def isView(nameParts: Seq[String]): Boolean = catalog.isView(nameParts) + override def isView(nameParts: Seq[String]): Boolean = v1SessionCatalog.isView(nameParts) // Only for tests. def this(catalog: SessionCatalog, conf: SQLConf) = { @@ -225,7 +225,7 @@ class Analyzer( ResolveAggregateFunctions :: TimeWindowing :: ResolveInlineTables(conf) :: - ResolveHigherOrderFunctions(catalog) :: + ResolveHigherOrderFunctions(v1SessionCatalog) :: ResolveLambdaVariables(conf) :: ResolveTimeZone(conf) :: ResolveRandomSeed :: @@ -721,7 +721,7 @@ class Analyzer( // have empty defaultDatabase and all the relations in viewText have database part defined. def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match { case u @ UnresolvedRelation(AsTemporaryViewIdentifier(ident)) - if catalog.isTemporaryTable(ident) => + if v1SessionCatalog.isTemporaryTable(ident) => resolveRelation(lookupTableFromCatalog(ident, u, AnalysisContext.get.defaultDatabase)) case u @ UnresolvedRelation(AsTableIdentifier(ident)) if !isRunningDirectlyOnFiles(ident) => @@ -778,7 +778,7 @@ class Analyzer( val tableIdentWithDb = tableIdentifier.copy( database = tableIdentifier.database.orElse(defaultDatabase)) try { - catalog.lookupRelation(tableIdentWithDb) + v1SessionCatalog.lookupRelation(tableIdentWithDb) } catch { case _: NoSuchTableException | _: NoSuchDatabaseException => u @@ -792,8 +792,9 @@ class Analyzer( // Note that we are testing (!db_exists || !table_exists) because the catalog throws // an exception from tableExists if the database does not exist. private def isRunningDirectlyOnFiles(table: TableIdentifier): Boolean = { - table.database.isDefined && conf.runSQLonFile && !catalog.isTemporaryTable(table) && - (!catalog.databaseExists(table.database.get) || !catalog.tableExists(table)) + table.database.isDefined && conf.runSQLonFile && !v1SessionCatalog.isTemporaryTable(table) && + (!v1SessionCatalog.databaseExists(table.database.get) + || !v1SessionCatalog.tableExists(table)) } } @@ -1511,13 +1512,14 @@ class Analyzer( plan.resolveExpressions { case f: UnresolvedFunction if externalFunctionNameSet.contains(normalizeFuncName(f.name)) => f - case f: UnresolvedFunction if catalog.isRegisteredFunction(f.name) => f - case f: UnresolvedFunction if catalog.isPersistentFunction(f.name) => + case f: UnresolvedFunction if v1SessionCatalog.isRegisteredFunction(f.name) => f + case f: UnresolvedFunction if v1SessionCatalog.isPersistentFunction(f.name) => externalFunctionNameSet.add(normalizeFuncName(f.name)) f case f: UnresolvedFunction => withPosition(f) { - throw new NoSuchFunctionException(f.name.database.getOrElse(catalog.getCurrentDatabase), + throw new NoSuchFunctionException( + f.name.database.getOrElse(v1SessionCatalog.getCurrentDatabase), f.name.funcName) } } @@ -1532,7 +1534,7 @@ class Analyzer( val databaseName = name.database match { case Some(a) => formatDatabaseName(a) - case None => catalog.getCurrentDatabase + case None => v1SessionCatalog.getCurrentDatabase } FunctionIdentifier(funcName, Some(databaseName)) @@ -1557,7 +1559,7 @@ class Analyzer( } case u @ UnresolvedGenerator(name, children) => withPosition(u) { - catalog.lookupFunction(name, children) match { + v1SessionCatalog.lookupFunction(name, children) match { case generator: Generator => generator case other => failAnalysis(s"$name is expected to be a generator. However, " + @@ -1566,7 +1568,7 @@ class Analyzer( } case u @ UnresolvedFunction(funcId, children, isDistinct) => withPosition(u) { - catalog.lookupFunction(funcId, children) match { + v1SessionCatalog.lookupFunction(funcId, children) match { // AggregateWindowFunctions are AggregateFunctions that can only be evaluated within // the context of a Window clause. They do not need to be wrapped in an // AggregateExpression. @@ -2765,17 +2767,17 @@ class Analyzer( private def lookupV2RelationAndCatalog( identifier: Seq[String]): Option[(DataSourceV2Relation, CatalogPlugin, Identifier)] = identifier match { - case AsTemporaryViewIdentifier(ti) if catalog.isTemporaryTable(ti) => None - case CatalogObjectIdentifier(Some(v2Catalog), ident) => - CatalogV2Util.loadTable(v2Catalog, ident) match { - case Some(table) => Some((DataSourceV2Relation.create(table), v2Catalog, ident)) + case AsTemporaryViewIdentifier(ti) if v1SessionCatalog.isTemporaryTable(ti) => None + case CatalogObjectIdentifier(catalog, ident) if !CatalogV2Util.isSessionCatalog(catalog) => + CatalogV2Util.loadTable(catalog, ident) match { + case Some(table) => Some((DataSourceV2Relation.create(table), catalog, ident)) case None => None } - case CatalogObjectIdentifier(None, ident) => - CatalogV2Util.loadTable(catalogManager.v2SessionCatalog, ident) match { + case CatalogObjectIdentifier(catalog, ident) if CatalogV2Util.isSessionCatalog(catalog) => + CatalogV2Util.loadTable(catalog, ident) match { case Some(_: V1Table) => None case Some(table) => - Some((DataSourceV2Relation.create(table), catalogManager.v2SessionCatalog, ident)) + Some((DataSourceV2Relation.create(table), catalog, ident)) case None => None } case _ => None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 5689446785441..13a79a82a3858 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -177,9 +177,8 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case ShowTablesStatement(Some(NonSessionCatalog(catalog, nameParts)), pattern) => ShowTables(catalog.asTableCatalog, nameParts, pattern) - // TODO (SPARK-29014): we should check if the current catalog is not session catalog here. - case ShowTablesStatement(None, pattern) if defaultCatalog.isDefined => - ShowTables(defaultCatalog.get.asTableCatalog, catalogManager.currentNamespace, pattern) + case ShowTablesStatement(None, pattern) if !isSessionCatalog(currentCatalog) => + ShowTables(currentCatalog.asTableCatalog, catalogManager.currentNamespace, pattern) case UseStatement(isNamespaceSet, nameParts) => if (isNamespaceSet) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index 14ccfd5bfcc9c..c9d050768c15e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -53,7 +53,7 @@ class CatalogManager( } } - def defaultCatalog: Option[CatalogPlugin] = { + private def defaultCatalog: Option[CatalogPlugin] = { conf.defaultV2Catalog.flatMap { catalogName => try { Some(catalog(catalogName)) @@ -74,9 +74,16 @@ class CatalogManager( } } - // If the V2_SESSION_CATALOG_IMPLEMENTATION config is specified, we try to instantiate the - // user-specified v2 session catalog. Otherwise, return the default session catalog. - def v2SessionCatalog: CatalogPlugin = { + /** + * If the V2_SESSION_CATALOG config is specified, we try to instantiate the user-specified v2 + * session catalog. Otherwise, return the default session catalog. + * + * This catalog is a v2 catalog that delegates to the v1 session catalog. it is used when the + * session catalog is responsible for an identifier, but the source requires the v2 catalog API. + * This happens when the source implementation extends the v2 TableProvider API and is not listed + * in the fallback configuration, spark.sql.sources.write.useV1SourceList + */ + private def v2SessionCatalog: CatalogPlugin = { conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { customV2SessionCatalog => try { catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 02585fd5c463d..26ba93e57fc64 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -27,29 +27,11 @@ private[sql] trait LookupCatalog extends Logging { protected val catalogManager: CatalogManager - /** - * Returns the default catalog. When set, this catalog is used for all identifiers that do not - * set a specific catalog. When this is None, the session catalog is responsible for the - * identifier. - * - * If this is None and a table's provider (source) is a v2 provider, the v2 session catalog will - * be used. - */ - def defaultCatalog: Option[CatalogPlugin] = catalogManager.defaultCatalog - /** * Returns the current catalog set. */ def currentCatalog: CatalogPlugin = catalogManager.currentCatalog - /** - * This catalog is a v2 catalog that delegates to the v1 session catalog. it is used when the - * session catalog is responsible for an identifier, but the source requires the v2 catalog API. - * This happens when the source implementation extends the v2 TableProvider API and is not listed - * in the fallback configuration, spark.sql.sources.write.useV1SourceList - */ - def sessionCatalog: CatalogPlugin = catalogManager.v2SessionCatalog - /** * Extract catalog plugin and remaining identifier names. * @@ -69,16 +51,14 @@ private[sql] trait LookupCatalog extends Logging { } } - type CatalogObjectIdentifier = (Option[CatalogPlugin], Identifier) - /** - * Extract catalog and identifier from a multi-part identifier with the default catalog if needed. + * Extract catalog and identifier from a multi-part identifier with the current catalog if needed. */ object CatalogObjectIdentifier { - def unapply(parts: Seq[String]): Some[CatalogObjectIdentifier] = parts match { + def unapply(parts: Seq[String]): Some[(CatalogPlugin, Identifier)] = parts match { case CatalogAndIdentifier(maybeCatalog, nameParts) => Some(( - maybeCatalog.orElse(defaultCatalog), + maybeCatalog.getOrElse(currentCatalog), Identifier.of(nameParts.init.toArray, nameParts.last) )) } @@ -108,7 +88,7 @@ private[sql] trait LookupCatalog extends Logging { */ object AsTableIdentifier { def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { - case CatalogAndIdentifier(None, names) if defaultCatalog.isEmpty => + case CatalogAndIdentifier(None, names) if CatalogV2Util.isSessionCatalog(currentCatalog) => names match { case Seq(name) => Some(TableIdentifier(name)) @@ -146,8 +126,7 @@ private[sql] trait LookupCatalog extends Logging { Some((catalogManager.catalog(nameParts.head), nameParts.tail)) } catch { case _: CatalogNotFoundException => - // TODO (SPARK-29014): use current catalog here. - Some((defaultCatalog.getOrElse(sessionCatalog), nameParts)) + Some((currentCatalog, nameParts)) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index c4a28bb6420c3..513f7e0348d09 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.FakeV2SessionCatalog import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -36,6 +37,7 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ private val catalogs = Seq("prod", "test").map(x => x -> DummyCatalogPlugin(x)).toMap + private val sessionCatalog = FakeV2SessionCatalog override val catalogManager: CatalogManager = { val manager = mock(classOf[CatalogManager]) @@ -43,22 +45,22 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { val name = invocation.getArgument[String](0) catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) }) - when(manager.defaultCatalog).thenReturn(None) + when(manager.currentCatalog).thenReturn(sessionCatalog) manager } test("catalog object identifier") { Seq( - ("tbl", None, Seq.empty, "tbl"), - ("db.tbl", None, Seq("db"), "tbl"), - ("prod.func", catalogs.get("prod"), Seq.empty, "func"), - ("ns1.ns2.tbl", None, Seq("ns1", "ns2"), "tbl"), - ("prod.db.tbl", catalogs.get("prod"), Seq("db"), "tbl"), - ("test.db.tbl", catalogs.get("test"), Seq("db"), "tbl"), - ("test.ns1.ns2.ns3.tbl", catalogs.get("test"), Seq("ns1", "ns2", "ns3"), "tbl"), - ("`db.tbl`", None, Seq.empty, "db.tbl"), - ("parquet.`file:/tmp/db.tbl`", None, Seq("parquet"), "file:/tmp/db.tbl"), - ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", None, + ("tbl", sessionCatalog, Seq.empty, "tbl"), + ("db.tbl", sessionCatalog, Seq("db"), "tbl"), + ("prod.func", catalogs("prod"), Seq.empty, "func"), + ("ns1.ns2.tbl", sessionCatalog, Seq("ns1", "ns2"), "tbl"), + ("prod.db.tbl", catalogs("prod"), Seq("db"), "tbl"), + ("test.db.tbl", catalogs("test"), Seq("db"), "tbl"), + ("test.ns1.ns2.ns3.tbl", catalogs("test"), Seq("ns1", "ns2", "ns3"), "tbl"), + ("`db.tbl`", sessionCatalog, Seq.empty, "db.tbl"), + ("parquet.`file:/tmp/db.tbl`", sessionCatalog, Seq("parquet"), "file:/tmp/db.tbl"), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", sessionCatalog, Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { case (sql, expectedCatalog, namespace, name) => inside(parseMultipartIdentifier(sql)) { @@ -135,22 +137,22 @@ class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog wit val name = invocation.getArgument[String](0) catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) }) - when(manager.defaultCatalog).thenReturn(catalogs.get("prod")) + when(manager.currentCatalog).thenReturn(catalogs("prod")) manager } test("catalog object identifier") { Seq( - ("tbl", catalogs.get("prod"), Seq.empty, "tbl"), - ("db.tbl", catalogs.get("prod"), Seq("db"), "tbl"), - ("prod.func", catalogs.get("prod"), Seq.empty, "func"), - ("ns1.ns2.tbl", catalogs.get("prod"), Seq("ns1", "ns2"), "tbl"), - ("prod.db.tbl", catalogs.get("prod"), Seq("db"), "tbl"), - ("test.db.tbl", catalogs.get("test"), Seq("db"), "tbl"), - ("test.ns1.ns2.ns3.tbl", catalogs.get("test"), Seq("ns1", "ns2", "ns3"), "tbl"), - ("`db.tbl`", catalogs.get("prod"), Seq.empty, "db.tbl"), - ("parquet.`file:/tmp/db.tbl`", catalogs.get("prod"), Seq("parquet"), "file:/tmp/db.tbl"), - ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", catalogs.get("prod"), + ("tbl", catalogs("prod"), Seq.empty, "tbl"), + ("db.tbl", catalogs("prod"), Seq("db"), "tbl"), + ("prod.func", catalogs("prod"), Seq.empty, "func"), + ("ns1.ns2.tbl", catalogs("prod"), Seq("ns1", "ns2"), "tbl"), + ("prod.db.tbl", catalogs("prod"), Seq("db"), "tbl"), + ("test.db.tbl", catalogs("test"), Seq("db"), "tbl"), + ("test.ns1.ns2.ns3.tbl", catalogs("test"), Seq("ns1", "ns2", "ns3"), "tbl"), + ("`db.tbl`", catalogs("prod"), Seq.empty, "db.tbl"), + ("parquet.`file:/tmp/db.tbl`", catalogs("prod"), Seq("parquet"), "file:/tmp/db.tbl"), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", catalogs("prod"), Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { case (sql, expectedCatalog, namespace, name) => inside(parseMultipartIdentifier(sql)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 3d04a0616e96b..4f88cc6daa331 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -341,6 +341,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def insertInto(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ assertNotBucketed("insertInto") @@ -354,14 +355,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - val sessionCatalog = session.sessionState.analyzer.sessionCatalog session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { - case CatalogObjectIdentifier(Some(catalog), ident) => + case CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog) => insertInto(catalog, ident) - case CatalogObjectIdentifier(None, ident) if canUseV2 && ident.namespace().length <= 1 => - insertInto(sessionCatalog, ident) + case CatalogObjectIdentifier(catalog, ident) + if isSessionCatalog(catalog) && canUseV2 && ident.namespace().length <= 1 => + insertInto(catalog, ident) case AsTableIdentifier(tableIdentifier) => insertInto(tableIdentifier) @@ -480,17 +481,18 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def saveAsTable(tableName: String): Unit = { import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined - val sessionCatalog = session.sessionState.analyzer.sessionCatalog session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { - case CatalogObjectIdentifier(Some(catalog), ident) => + case CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog) => saveAsTable(catalog.asTableCatalog, ident) - case CatalogObjectIdentifier(None, ident) if canUseV2 && ident.namespace().length <= 1 => - saveAsTable(sessionCatalog.asTableCatalog, ident) + case CatalogObjectIdentifier(catalog, ident) + if isSessionCatalog(catalog) && canUseV2 && ident.namespace().length <= 1 => + saveAsTable(catalog.asTableCatalog, ident) case AsTableIdentifier(tableIdentifier) => saveAsTable(tableIdentifier) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index 17782e8ab1f0d..9d3ce6fde20a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -51,9 +51,8 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) private val (catalog, identifier) = { - val CatalogObjectIdentifier(maybeCatalog, identifier) = tableName - val catalog = maybeCatalog.getOrElse(catalogManager.currentCatalog).asTableCatalog - (catalog, identifier) + val CatalogObjectIdentifier(catalog, identifier) = tableName + (catalog.asTableCatalog, identifier) } private val logicalPlan = df.queryExecution.logical diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 0097af310a9d6..a781e2fb258ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -262,8 +262,7 @@ class ResolveSessionCatalog( } ShowTablesCommand(Some(nameParts.head), pattern) - // TODO (SPARK-29014): we should check if the current catalog is session catalog here. - case ShowTablesStatement(None, pattern) if defaultCatalog.isEmpty => + case ShowTablesStatement(None, pattern) if isSessionCatalog(currentCatalog) => ShowTablesCommand(None, pattern) case AnalyzeTableStatement(tableName, partitionSpec, noScan) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index e27575cecde25..08627e681f9e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -84,7 +84,7 @@ class DataSourceV2DataFrameSessionCatalogSuite val t1 = "prop_table" withTable(t1) { spark.range(20).write.format(v2Format).option("path", "abc").saveAsTable(t1) - val cat = spark.sessionState.catalogManager.v2SessionCatalog.asInstanceOf[TableCatalog] + val cat = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] val tableInfo = cat.loadTable(Identifier.of(Array.empty, t1)) assert(tableInfo.properties().get("location") === "abc") assert(tableInfo.properties().get("provider") === v2Format) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala index e1a5dbe3351ec..27725bcadbcd5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala @@ -44,7 +44,7 @@ class DataSourceV2SQLSessionCatalogSuite } override def getTableMetadata(tableName: String): Table = { - val v2Catalog = spark.sessionState.catalogManager.v2SessionCatalog + val v2Catalog = spark.sessionState.catalogManager.currentCatalog val nameParts = spark.sessionState.sqlParser.parseMultipartIdentifier(tableName) v2Catalog.asInstanceOf[TableCatalog] .loadTable(Identifier.of(Array.empty, nameParts.last)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 79ee8f2dd9740..1d2bc9d4c8eea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -727,6 +727,23 @@ class DataSourceV2SQLSuite expectV2Catalog = false) } + test("ShowTables: change current catalog and namespace with USE statements") { + sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo") + + // Initially, the v2 session catalog (current catalog) is used. + runShowTablesSql( + "SHOW TABLES", Seq(Row("", "source", true), Row("", "source2", true)), + expectV2Catalog = false) + + // Update the current catalog, and no table is matched since the current namespace is Array(). + sql("USE testcat") + runShowTablesSql("SHOW TABLES", Seq()) + + // Update the current namespace to match ns1.ns2.table. + sql("USE testcat.ns1.ns2") + runShowTablesSql("SHOW TABLES", Seq(Row("ns1.ns2", "table"))) + } + private def runShowTablesSql( sqlText: String, expected: Seq[Row], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 674efa9b8ba42..562e61390a53d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -96,8 +96,7 @@ class PlanResolutionSuite extends AnalysisTest { throw new CatalogNotFoundException(s"No such catalog: $name") } }) - when(manager.defaultCatalog).thenReturn(Some(testCat)) - when(manager.v2SessionCatalog).thenReturn(v2SessionCatalog) + when(manager.currentCatalog).thenReturn(testCat) when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) manager } @@ -112,8 +111,7 @@ class PlanResolutionSuite extends AnalysisTest { throw new CatalogNotFoundException(s"No such catalog: $name") } }) - when(manager.defaultCatalog).thenReturn(None) - when(manager.v2SessionCatalog).thenReturn(v2SessionCatalog) + when(manager.currentCatalog).thenReturn(v2SessionCatalog) when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) manager } From c3a0d02a4004bcd39d680dc8b7a597e37967a74d Mon Sep 17 00:00:00 2001 From: jiake Date: Fri, 18 Oct 2019 23:16:58 +0800 Subject: [PATCH 0364/1387] [SPARK-28560][SQL][FOLLOWUP] resolve the remaining comments for PR#25295 ### What changes were proposed in this pull request? A followup of [#25295](https://github.com/apache/spark/pull/25295). 1) change the logWarning to logDebug in `OptimizeLocalShuffleReader`. 2) update the test to check whether query stage reuse can work well with local shuffle reader. ### Why are the changes needed? make code robust ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing tests Closes #26157 from JkSelf/followup-25295. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../adaptive/OptimizeLocalShuffleReader.scala | 2 +- .../adaptive/AdaptiveQueryExecSuite.scala | 22 ++++++++++++++----- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 308e65e793d8f..89e2813695a65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -61,7 +61,7 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { val numExchangeAfter = numExchanges(EnsureRequirements(conf).apply(optimizedPlan)) if (numExchangeAfter > numExchangeBefore) { - logWarning("OptimizeLocalShuffleReader rule is not applied due" + + logDebug("OptimizeLocalShuffleReader rule is not applied due" + " to additional shuffles will be introduced.") plan } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index cd0bf726da9aa..43802968c4696 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} -import org.apache.spark.sql.execution.adaptive.rule.CoalescedShuffleReaderExec import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildRight, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf @@ -78,7 +77,7 @@ class AdaptiveQueryExecSuite } private def checkNumLocalShuffleReaders(plan: SparkPlan, expected: Int): Unit = { - val localReaders = plan.collect { + val localReaders = collect(plan) { case reader: LocalShuffleReaderExec => reader } assert(localReaders.length === expected) @@ -164,7 +163,7 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - + // additional shuffle exchange introduced, only one shuffle reader to local shuffle reader. checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -189,8 +188,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - - checkNumLocalShuffleReaders(adaptivePlan, 0) + // additional shuffle exchange introduced, only one shuffle reader to local shuffle reader. + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -214,7 +213,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - checkNumLocalShuffleReaders(adaptivePlan, 0) + // additional shuffle exchange introduced, only one shuffle reader to local shuffle reader. + checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -229,6 +229,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 2) + checkNumLocalShuffleReaders(adaptivePlan, 2) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.size == 1) } @@ -245,6 +247,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.size == 1) } @@ -263,6 +267,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) val sub = findReusedSubquery(adaptivePlan) @@ -282,6 +288,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) val sub = findReusedSubquery(adaptivePlan) @@ -304,6 +312,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) + checkNumLocalShuffleReaders(adaptivePlan, 1) + // Even with local shuffle reader, the query statge reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) assert(ex.head.plan.isInstanceOf[BroadcastQueryStageExec]) From 77fe8a8e7cdb0eae9f66b2d10c0d35f784663547 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 18 Oct 2019 23:54:59 +0800 Subject: [PATCH 0365/1387] [SPARK-28420][SQL] Support the `INTERVAL` type in `date_part()` ### What changes were proposed in this pull request? The `date_part()` function can accept the `source` parameter of the `INTERVAL` type (`CalendarIntervalType`). The following values of the `field` parameter are supported: - `"MILLENNIUM"` (`"MILLENNIA"`, `"MIL"`, `"MILS"`) - number of millenniums in the given interval. It is `YEAR / 1000`. - `"CENTURY"` (`"CENTURIES"`, `"C"`, `"CENT"`) - number of centuries in the interval calculated as `YEAR / 100`. - `"DECADE"` (`"DECADES"`, `"DEC"`, `"DECS"`) - decades in the `YEAR` part of the interval calculated as `YEAR / 10`. - `"YEAR"` (`"Y"`, `"YEARS"`, `"YR"`, `"YRS"`) - years in a values of `CalendarIntervalType`. It is `MONTHS / 12`. - `"QUARTER"` (`"QTR"`) - a quarter of year calculated as `MONTHS / 3 + 1` - `"MONTH"` (`"MON"`, `"MONS"`, `"MONTHS"`) - the months part of the interval calculated as `CalendarInterval.months % 12` - `"DAY"` (`"D"`, `"DAYS"`) - total number of days in `CalendarInterval.microseconds` - `"HOUR"` (`"H"`, `"HOURS"`, `"HR"`, `"HRS"`) - the hour part of the interval. - `"MINUTE"` (`"M"`, `"MIN"`, `"MINS"`, `"MINUTES"`) - the minute part of the interval. - `"SECOND"` (`"S"`, `"SEC"`, `"SECONDS"`, `"SECS"`) - the seconds part with fractional microsecond part. - `"MILLISECONDS"` (`"MSEC"`, `"MSECS"`, `"MILLISECON"`, `"MSECONDS"`, `"MS"`) - the millisecond part of the interval with fractional microsecond part. - `"MICROSECONDS"` (`"USEC"`, `"USECS"`, `"USECONDS"`, `"MICROSECON"`, `"US"`) - the total number of microseconds in the `second`, `millisecond` and `microsecond` parts of the given interval. - `"EPOCH"` - the total number of seconds in the interval including the fractional part with microsecond precision. Here we assume 365.25 days per year (leap year every four years). For example: ```sql > SELECT date_part('days', interval 1 year 10 months 5 days); 5 > SELECT date_part('seconds', interval 30 seconds 1 milliseconds 1 microseconds); 30.001001 ``` ### Why are the changes needed? To maintain feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT) ### Does this PR introduce any user-facing change? No ### How was this patch tested? - Added new test suite `IntervalExpressionsSuite` - Add new test cases to `date_part.sql` Closes #25981 from MaxGekk/extract-from-intervals. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../expressions/datetimeExpressions.scala | 40 +- .../expressions/intervalExpressions.scala | 111 +++++ .../sql/catalyst/util/IntervalUtils.scala | 91 ++++ .../IntervalExpressionsSuite.scala | 192 +++++++ .../resources/sql-tests/inputs/date_part.sql | 75 +++ .../sql-tests/results/date_part.sql.out | 470 +++++++++++++++++- 6 files changed, 970 insertions(+), 9 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index cddd8c9bd61b3..90eb3598b233b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import java.time.{DateTimeException, Instant, LocalDate, LocalDateTime, ZoneId} +import java.time.{DateTimeException, LocalDate, LocalDateTime, ZoneId} import java.time.temporal.IsoFields import java.util.{Locale, TimeZone} @@ -2032,10 +2032,11 @@ object DatePart { } @ExpressionDescription( - usage = "_FUNC_(field, source) - Extracts a part of the date/timestamp.", + usage = "_FUNC_(field, source) - Extracts a part of the date/timestamp or interval source.", arguments = """ Arguments: - * field - selects which part of the source should be extracted. Supported string values are: + * field - selects which part of the source should be extracted. + Supported string values of `field` for dates and timestamps are: ["MILLENNIUM", ("MILLENNIA", "MIL", "MILS"), "CENTURY", ("CENTURIES", "C", "CENT"), "DECADE", ("DECADES", "DEC", "DECS"), @@ -2055,7 +2056,21 @@ object DatePart { "MILLISECONDS", ("MSEC", "MSECS", "MILLISECON", "MSECONDS", "MS"), "MICROSECONDS", ("USEC", "USECS", "USECONDS", "MICROSECON", "US"), "EPOCH"] - * source - a date (or timestamp) column from where `field` should be extracted + Supported string values of `field` for intervals are: + ["MILLENNIUM", ("MILLENNIA", "MIL", "MILS"), + "CENTURY", ("CENTURIES", "C", "CENT"), + "DECADE", ("DECADES", "DEC", "DECS"), + "YEAR", ("Y", "YEARS", "YR", "YRS"), + "QUARTER", ("QTR"), + "MONTH", ("MON", "MONS", "MONTHS"), + "DAY", ("D", "DAYS"), + "HOUR", ("H", "HOURS", "HR", "HRS"), + "MINUTE", ("M", "MIN", "MINS", "MINUTES"), + "SECOND", ("S", "SEC", "SECONDS", "SECS"), + "MILLISECONDS", ("MSEC", "MSECS", "MILLISECON", "MSECONDS", "MS"), + "MICROSECONDS", ("USEC", "USECS", "USECONDS", "MICROSECON", "US"), + "EPOCH"] + * source - a date/timestamp or interval column from where `field` should be extracted """, examples = """ Examples: @@ -2067,6 +2082,10 @@ object DatePart { 224 > SELECT _FUNC_('SECONDS', timestamp'2019-10-01 00:00:01.000001'); 1.000001 + > SELECT _FUNC_('days', interval 1 year 10 months 5 days); + 5 + > SELECT _FUNC_('seconds', interval 5 hours 30 seconds 1 milliseconds 1 microseconds); + 30.001001 """, since = "3.0.0") case class DatePart(field: Expression, source: Expression, child: Expression) @@ -2082,9 +2101,16 @@ case class DatePart(field: Expression, source: Expression, child: Expression) Literal(null, DoubleType) } else { val fieldStr = fieldEval.asInstanceOf[UTF8String].toString - DatePart.parseExtractField(fieldStr, source, { - throw new AnalysisException(s"Literals of type '$fieldStr' are currently not supported.") - }) + val errMsg = s"Literals of type '$fieldStr' are currently not supported " + + s"for the ${source.dataType.catalogString} type." + if (source.dataType == CalendarIntervalType) { + ExtractIntervalPart.parseExtractField( + fieldStr, + source, + throw new AnalysisException(errMsg)) + } else { + DatePart.parseExtractField(fieldStr, source, throw new AnalysisException(errMsg)) + } } }) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala new file mode 100644 index 0000000000000..08360c75a474b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -0,0 +1,111 @@ +/* + * 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 java.util.Locale + +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.util.IntervalUtils +import org.apache.spark.sql.catalyst.util.IntervalUtils._ +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval + +abstract class ExtractIntervalPart( + child: Expression, + val dataType: DataType, + func: CalendarInterval => Any, + funcName: String) + extends UnaryExpression with ExpectsInputTypes with Serializable { + + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + + override protected def nullSafeEval(interval: Any): Any = { + func(interval.asInstanceOf[CalendarInterval]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.$funcName($c)") + } +} + +case class ExtractIntervalMillenniums(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getMillenniums, "getMillenniums") + +case class ExtractIntervalCenturies(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getCenturies, "getCenturies") + +case class ExtractIntervalDecades(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getDecades, "getDecades") + +case class ExtractIntervalYears(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getYears, "getYears") + +case class ExtractIntervalQuarters(child: Expression) + extends ExtractIntervalPart(child, ByteType, getQuarters, "getQuarters") + +case class ExtractIntervalMonths(child: Expression) + extends ExtractIntervalPart(child, ByteType, getMonths, "getMonths") + +case class ExtractIntervalDays(child: Expression) + extends ExtractIntervalPart(child, LongType, getDays, "getDays") + +case class ExtractIntervalHours(child: Expression) + extends ExtractIntervalPart(child, ByteType, getHours, "getHours") + +case class ExtractIntervalMinutes(child: Expression) + extends ExtractIntervalPart(child, ByteType, getMinutes, "getMinutes") + +case class ExtractIntervalSeconds(child: Expression) + extends ExtractIntervalPart(child, DecimalType(8, 6), getSeconds, "getSeconds") + +case class ExtractIntervalMilliseconds(child: Expression) + extends ExtractIntervalPart(child, DecimalType(8, 3), getMilliseconds, "getMilliseconds") + +case class ExtractIntervalMicroseconds(child: Expression) + extends ExtractIntervalPart(child, LongType, getMicroseconds, "getMicroseconds") + +// Number of seconds in 10000 years is 315576000001 (30 days per one month) +// which is 12 digits + 6 digits for the fractional part of seconds. +case class ExtractIntervalEpoch(child: Expression) + extends ExtractIntervalPart(child, DecimalType(18, 6), getEpoch, "getEpoch") + +object ExtractIntervalPart { + + def parseExtractField( + extractField: String, + source: Expression, + errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { + case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => ExtractIntervalMillenniums(source) + case "CENTURY" | "CENTURIES" | "C" | "CENT" => ExtractIntervalCenturies(source) + case "DECADE" | "DECADES" | "DEC" | "DECS" => ExtractIntervalDecades(source) + case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => ExtractIntervalYears(source) + case "QUARTER" | "QTR" => ExtractIntervalQuarters(source) + case "MONTH" | "MON" | "MONS" | "MONTHS" => ExtractIntervalMonths(source) + case "DAY" | "D" | "DAYS" => ExtractIntervalDays(source) + case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => ExtractIntervalHours(source) + case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => ExtractIntervalMinutes(source) + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => ExtractIntervalSeconds(source) + case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => + ExtractIntervalMilliseconds(source) + case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => + ExtractIntervalMicroseconds(source) + case "EPOCH" => ExtractIntervalEpoch(source) + case _ => errorHandleFunc + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala new file mode 100644 index 0000000000000..78d188f81f628 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -0,0 +1,91 @@ +/* + * 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.util + +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.CalendarInterval + +object IntervalUtils { + val MONTHS_PER_YEAR: Int = 12 + val MONTHS_PER_QUARTER: Byte = 3 + val YEARS_PER_MILLENNIUM: Int = 1000 + val YEARS_PER_CENTURY: Int = 100 + val YEARS_PER_DECADE: Int = 10 + val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS + val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS + val DAYS_PER_MONTH: Byte = 30 + val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY + /* 365.25 days per year assumes leap year every four years */ + val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 + + def getYears(interval: CalendarInterval): Int = { + interval.months / MONTHS_PER_YEAR + } + + def getMillenniums(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_MILLENNIUM + } + + def getCenturies(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_CENTURY + } + + def getDecades(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_DECADE + } + + def getMonths(interval: CalendarInterval): Byte = { + (interval.months % MONTHS_PER_YEAR).toByte + } + + def getQuarters(interval: CalendarInterval): Byte = { + (getMonths(interval) / MONTHS_PER_QUARTER + 1).toByte + } + + def getDays(interval: CalendarInterval): Long = { + interval.microseconds / DateTimeUtils.MICROS_PER_DAY + } + + def getHours(interval: CalendarInterval): Byte = { + ((interval.microseconds % DateTimeUtils.MICROS_PER_DAY) / MICROS_PER_HOUR).toByte + } + + def getMinutes(interval: CalendarInterval): Byte = { + ((interval.microseconds % MICROS_PER_HOUR) / MICROS_PER_MINUTE).toByte + } + + def getMicroseconds(interval: CalendarInterval): Long = { + interval.microseconds % MICROS_PER_MINUTE + } + + def getSeconds(interval: CalendarInterval): Decimal = { + Decimal(getMicroseconds(interval), 8, 6) + } + + def getMilliseconds(interval: CalendarInterval): Decimal = { + Decimal(getMicroseconds(interval), 8, 3) + } + + // Returns total number of seconds with microseconds fractional part in the given interval. + def getEpoch(interval: CalendarInterval): Decimal = { + var result = interval.microseconds + result += MICROS_PER_YEAR * (interval.months / MONTHS_PER_YEAR) + result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) + Decimal(result, 18, 6) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala new file mode 100644 index 0000000000000..078ec88800215 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -0,0 +1,192 @@ +/* + * 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.language.implicitConversions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.CalendarInterval + +class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + implicit def interval(s: String): Literal = { + Literal(CalendarInterval.fromString("interval " + s)) + } + + test("millenniums") { + checkEvaluation(ExtractIntervalMillenniums("0 years"), 0) + checkEvaluation(ExtractIntervalMillenniums("9999 years"), 9) + checkEvaluation(ExtractIntervalMillenniums("1000 years"), 1) + checkEvaluation(ExtractIntervalMillenniums("-2000 years"), -2) + // Microseconds part must not be taken into account + checkEvaluation(ExtractIntervalMillenniums("999 years 400 days"), 0) + // Millennium must be taken from years and months + checkEvaluation(ExtractIntervalMillenniums("999 years 12 months"), 1) + checkEvaluation(ExtractIntervalMillenniums("1000 years -1 months"), 0) + } + + test("centuries") { + checkEvaluation(ExtractIntervalCenturies("0 years"), 0) + checkEvaluation(ExtractIntervalCenturies("9999 years"), 99) + checkEvaluation(ExtractIntervalCenturies("1000 years"), 10) + checkEvaluation(ExtractIntervalCenturies("-2000 years"), -20) + // Microseconds part must not be taken into account + checkEvaluation(ExtractIntervalCenturies("99 years 400 days"), 0) + // Century must be taken from years and months + checkEvaluation(ExtractIntervalCenturies("99 years 12 months"), 1) + checkEvaluation(ExtractIntervalCenturies("100 years -1 months"), 0) + } + + test("decades") { + checkEvaluation(ExtractIntervalDecades("0 years"), 0) + checkEvaluation(ExtractIntervalDecades("9999 years"), 999) + checkEvaluation(ExtractIntervalDecades("1000 years"), 100) + checkEvaluation(ExtractIntervalDecades("-2000 years"), -200) + // Microseconds part must not be taken into account + checkEvaluation(ExtractIntervalDecades("9 years 400 days"), 0) + // Decade must be taken from years and months + checkEvaluation(ExtractIntervalDecades("9 years 12 months"), 1) + checkEvaluation(ExtractIntervalDecades("10 years -1 months"), 0) + } + + test("years") { + checkEvaluation(ExtractIntervalYears("0 years"), 0) + checkEvaluation(ExtractIntervalYears("9999 years"), 9999) + checkEvaluation(ExtractIntervalYears("1000 years"), 1000) + checkEvaluation(ExtractIntervalYears("-2000 years"), -2000) + // Microseconds part must not be taken into account + checkEvaluation(ExtractIntervalYears("9 years 400 days"), 9) + // Year must be taken from years and months + checkEvaluation(ExtractIntervalYears("9 years 12 months"), 10) + checkEvaluation(ExtractIntervalYears("10 years -1 months"), 9) + } + + test("quarters") { + checkEvaluation(ExtractIntervalQuarters("0 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("1 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("-1 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("2 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("-2 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("1 years -1 months"), 4.toByte) + checkEvaluation(ExtractIntervalQuarters("-1 years 1 months"), -2.toByte) + checkEvaluation(ExtractIntervalQuarters("2 years 3 months"), 2.toByte) + checkEvaluation(ExtractIntervalQuarters("-2 years -3 months"), 0.toByte) + checkEvaluation(ExtractIntervalQuarters("9999 years"), 1.toByte) + } + + test("months") { + checkEvaluation(ExtractIntervalMonths("0 year"), 0.toByte) + for (m <- -24 to 24) { + checkEvaluation(ExtractIntervalMonths(s"$m months"), (m % 12).toByte) + } + checkEvaluation(ExtractIntervalMonths("1 year 10 months"), 10.toByte) + checkEvaluation(ExtractIntervalMonths("-2 year -10 months"), -10.toByte) + checkEvaluation(ExtractIntervalMonths("9999 years"), 0.toByte) + } + + private val largeInterval: String = "9999 years 11 months " + + "31 days 11 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds" + + test("days") { + checkEvaluation(ExtractIntervalDays("0 days"), 0L) + checkEvaluation(ExtractIntervalDays("1 days 100 seconds"), 1L) + checkEvaluation(ExtractIntervalDays("-1 days -100 seconds"), -1L) + checkEvaluation(ExtractIntervalDays("-365 days"), -365L) + checkEvaluation(ExtractIntervalDays("365 days"), 365L) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalDays("100 year 10 months 5 days"), 5L) + checkEvaluation(ExtractIntervalDays(largeInterval), 31L) + } + + test("hours") { + checkEvaluation(ExtractIntervalHours("0 hours"), 0.toByte) + checkEvaluation(ExtractIntervalHours("1 hour"), 1.toByte) + checkEvaluation(ExtractIntervalHours("-1 hour"), -1.toByte) + checkEvaluation(ExtractIntervalHours("23 hours"), 23.toByte) + checkEvaluation(ExtractIntervalHours("-23 hours"), -23.toByte) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalHours("100 year 10 months 10 hours"), 10.toByte) + checkEvaluation(ExtractIntervalHours(largeInterval), 11.toByte) + } + + test("minutes") { + checkEvaluation(ExtractIntervalMinutes("0 minute"), 0.toByte) + checkEvaluation(ExtractIntervalMinutes("1 minute"), 1.toByte) + checkEvaluation(ExtractIntervalMinutes("-1 minute"), -1.toByte) + checkEvaluation(ExtractIntervalMinutes("59 minute"), 59.toByte) + checkEvaluation(ExtractIntervalMinutes("-59 minute"), -59.toByte) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalMinutes("100 year 10 months 10 minutes"), 10.toByte) + checkEvaluation(ExtractIntervalMinutes(largeInterval), 59.toByte) + } + + test("seconds") { + checkEvaluation(ExtractIntervalSeconds("0 second"), Decimal(0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("1 second"), Decimal(1.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("-1 second"), Decimal(-1.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("1 minute 59 second"), Decimal(59.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalSeconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds(largeInterval), Decimal(59.999999, 8, 6)) + checkEvaluation( + ExtractIntervalSeconds("10 seconds 1 milliseconds 1 microseconds"), + Decimal(10001001, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) + } + + test("milliseconds") { + checkEvaluation(ExtractIntervalMilliseconds("0 milliseconds"), Decimal(0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds("1 milliseconds"), Decimal(1.0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds("-1 milliseconds"), Decimal(-1.0, 8, 3)) + checkEvaluation( + ExtractIntervalMilliseconds("1 second 999 milliseconds"), + Decimal(1999.0, 8, 3)) + checkEvaluation( + ExtractIntervalMilliseconds("999 milliseconds 1 microsecond"), + Decimal(999.001, 8, 3)) + checkEvaluation( + ExtractIntervalMilliseconds("-1 second -999 milliseconds"), + Decimal(-1999.0, 8, 3)) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalMilliseconds("100 year 1 millisecond"), Decimal(1.0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds(largeInterval), Decimal(59999.999, 8, 3)) + } + + test("microseconds") { + checkEvaluation(ExtractIntervalMicroseconds("0 microseconds"), 0L) + checkEvaluation(ExtractIntervalMicroseconds("1 microseconds"), 1L) + checkEvaluation(ExtractIntervalMicroseconds("-1 microseconds"), -1L) + checkEvaluation(ExtractIntervalMicroseconds("1 second 999 microseconds"), 1000999L) + checkEvaluation(ExtractIntervalMicroseconds("999 milliseconds 1 microseconds"), 999001L) + checkEvaluation(ExtractIntervalMicroseconds("-1 second -999 microseconds"), -1000999L) + // Years and months must not be taken into account + checkEvaluation(ExtractIntervalMicroseconds("11 year 1 microseconds"), 1L) + checkEvaluation(ExtractIntervalMicroseconds(largeInterval), 59999999L) + } + + test("epoch") { + checkEvaluation(ExtractIntervalEpoch("0 months"), Decimal(0.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("10000 years"), Decimal(315576000000.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("1 year"), Decimal(31557600.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("-1 year"), Decimal(-31557600.0, 18, 6)) + checkEvaluation( + ExtractIntervalEpoch("1 second 1 millisecond 1 microsecond"), + Decimal(1.001001, 18, 6)) + } +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql index fd0fb50f71460..a63cdafb745a0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql @@ -68,3 +68,78 @@ select date_part('not_supported', c) from t; select date_part(c, c) from t; select date_part(null, c) from t; + +CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c; + +select date_part('millennium', c) from t2; +select date_part('millennia', c) from t2; +select date_part('mil', c) from t2; +select date_part('mils', c) from t2; + +select date_part('century', c) from t2; +select date_part('centuries', c) from t2; +select date_part('c', c) from t2; +select date_part('cent', c) from t2; + +select date_part('decade', c) from t2; +select date_part('decades', c) from t2; +select date_part('dec', c) from t2; +select date_part('decs', c) from t2; + +select date_part('year', c) from t2; +select date_part('y', c) from t2; +select date_part('years', c) from t2; +select date_part('yr', c) from t2; +select date_part('yrs', c) from t2; + +select date_part('quarter', c) from t2; +select date_part('qtr', c) from t2; + +select date_part('month', c) from t2; +select date_part('mon', c) from t2; +select date_part('mons', c) from t2; +select date_part('months', c) from t2; + +select date_part('day', c) from t2; +select date_part('d', c) from t2; +select date_part('days', c) from t2; + +select date_part('hour', c) from t2; +select date_part('h', c) from t2; +select date_part('hours', c) from t2; +select date_part('hr', c) from t2; +select date_part('hrs', c) from t2; + +select date_part('minute', c) from t2; +select date_part('m', c) from t2; +select date_part('min', c) from t2; +select date_part('mins', c) from t2; +select date_part('minutes', c) from t2; + +select date_part('second', c) from t2; +select date_part('s', c) from t2; +select date_part('sec', c) from t2; +select date_part('seconds', c) from t2; +select date_part('secs', c) from t2; + +select date_part('milliseconds', c) from t2; +select date_part('msec', c) from t2; +select date_part('msecs', c) from t2; +select date_part('millisecon', c) from t2; +select date_part('mseconds', c) from t2; +select date_part('ms', c) from t2; + +select date_part('microseconds', c) from t2; +select date_part('usec', c) from t2; +select date_part('usecs', c) from t2; +select date_part('useconds', c) from t2; +select date_part('microsecon', c) from t2; +select date_part('us', c) from t2; + +select date_part('epoch', c) from t2; + +select date_part('not_supported', c) from t2; + +select date_part(c, c) from t2; + +select date_part(null, c) from t2; diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index bbdfe6cc22f5a..8f4edf1960755 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 52 +-- Number of queries: 110 -- !query 0 @@ -400,7 +400,7 @@ select date_part('not_supported', c) from t struct<> -- !query 49 output org.apache.spark.sql.AnalysisException -Literals of type 'not_supported' are currently not supported.;; line 1 pos 7 +Literals of type 'not_supported' are currently not supported for the string type.;; line 1 pos 7 -- !query 50 @@ -418,3 +418,469 @@ select date_part(null, c) from t struct -- !query 51 output NULL + + +-- !query 52 +CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c +-- !query 52 schema +struct<> +-- !query 52 output + + + +-- !query 53 +select date_part('millennium', c) from t2 +-- !query 53 schema +struct +-- !query 53 output +1 + + +-- !query 54 +select date_part('millennia', c) from t2 +-- !query 54 schema +struct +-- !query 54 output +1 + + +-- !query 55 +select date_part('mil', c) from t2 +-- !query 55 schema +struct +-- !query 55 output +1 + + +-- !query 56 +select date_part('mils', c) from t2 +-- !query 56 schema +struct +-- !query 56 output +1 + + +-- !query 57 +select date_part('century', c) from t2 +-- !query 57 schema +struct +-- !query 57 output +10 + + +-- !query 58 +select date_part('centuries', c) from t2 +-- !query 58 schema +struct +-- !query 58 output +10 + + +-- !query 59 +select date_part('c', c) from t2 +-- !query 59 schema +struct +-- !query 59 output +10 + + +-- !query 60 +select date_part('cent', c) from t2 +-- !query 60 schema +struct +-- !query 60 output +10 + + +-- !query 61 +select date_part('decade', c) from t2 +-- !query 61 schema +struct +-- !query 61 output +101 + + +-- !query 62 +select date_part('decades', c) from t2 +-- !query 62 schema +struct +-- !query 62 output +101 + + +-- !query 63 +select date_part('dec', c) from t2 +-- !query 63 schema +struct +-- !query 63 output +101 + + +-- !query 64 +select date_part('decs', c) from t2 +-- !query 64 schema +struct +-- !query 64 output +101 + + +-- !query 65 +select date_part('year', c) from t2 +-- !query 65 schema +struct +-- !query 65 output +1010 + + +-- !query 66 +select date_part('y', c) from t2 +-- !query 66 schema +struct +-- !query 66 output +1010 + + +-- !query 67 +select date_part('years', c) from t2 +-- !query 67 schema +struct +-- !query 67 output +1010 + + +-- !query 68 +select date_part('yr', c) from t2 +-- !query 68 schema +struct +-- !query 68 output +1010 + + +-- !query 69 +select date_part('yrs', c) from t2 +-- !query 69 schema +struct +-- !query 69 output +1010 + + +-- !query 70 +select date_part('quarter', c) from t2 +-- !query 70 schema +struct +-- !query 70 output +4 + + +-- !query 71 +select date_part('qtr', c) from t2 +-- !query 71 schema +struct +-- !query 71 output +4 + + +-- !query 72 +select date_part('month', c) from t2 +-- !query 72 schema +struct +-- !query 72 output +9 + + +-- !query 73 +select date_part('mon', c) from t2 +-- !query 73 schema +struct +-- !query 73 output +9 + + +-- !query 74 +select date_part('mons', c) from t2 +-- !query 74 schema +struct +-- !query 74 output +9 + + +-- !query 75 +select date_part('months', c) from t2 +-- !query 75 schema +struct +-- !query 75 output +9 + + +-- !query 76 +select date_part('day', c) from t2 +-- !query 76 schema +struct +-- !query 76 output +8 + + +-- !query 77 +select date_part('d', c) from t2 +-- !query 77 schema +struct +-- !query 77 output +8 + + +-- !query 78 +select date_part('days', c) from t2 +-- !query 78 schema +struct +-- !query 78 output +8 + + +-- !query 79 +select date_part('hour', c) from t2 +-- !query 79 schema +struct +-- !query 79 output +7 + + +-- !query 80 +select date_part('h', c) from t2 +-- !query 80 schema +struct +-- !query 80 output +7 + + +-- !query 81 +select date_part('hours', c) from t2 +-- !query 81 schema +struct +-- !query 81 output +7 + + +-- !query 82 +select date_part('hr', c) from t2 +-- !query 82 schema +struct +-- !query 82 output +7 + + +-- !query 83 +select date_part('hrs', c) from t2 +-- !query 83 schema +struct +-- !query 83 output +7 + + +-- !query 84 +select date_part('minute', c) from t2 +-- !query 84 schema +struct +-- !query 84 output +6 + + +-- !query 85 +select date_part('m', c) from t2 +-- !query 85 schema +struct +-- !query 85 output +6 + + +-- !query 86 +select date_part('min', c) from t2 +-- !query 86 schema +struct +-- !query 86 output +6 + + +-- !query 87 +select date_part('mins', c) from t2 +-- !query 87 schema +struct +-- !query 87 output +6 + + +-- !query 88 +select date_part('minutes', c) from t2 +-- !query 88 schema +struct +-- !query 88 output +6 + + +-- !query 89 +select date_part('second', c) from t2 +-- !query 89 schema +struct +-- !query 89 output +5.004003 + + +-- !query 90 +select date_part('s', c) from t2 +-- !query 90 schema +struct +-- !query 90 output +5.004003 + + +-- !query 91 +select date_part('sec', c) from t2 +-- !query 91 schema +struct +-- !query 91 output +5.004003 + + +-- !query 92 +select date_part('seconds', c) from t2 +-- !query 92 schema +struct +-- !query 92 output +5.004003 + + +-- !query 93 +select date_part('secs', c) from t2 +-- !query 93 schema +struct +-- !query 93 output +5.004003 + + +-- !query 94 +select date_part('milliseconds', c) from t2 +-- !query 94 schema +struct +-- !query 94 output +5004.003 + + +-- !query 95 +select date_part('msec', c) from t2 +-- !query 95 schema +struct +-- !query 95 output +5004.003 + + +-- !query 96 +select date_part('msecs', c) from t2 +-- !query 96 schema +struct +-- !query 96 output +5004.003 + + +-- !query 97 +select date_part('millisecon', c) from t2 +-- !query 97 schema +struct +-- !query 97 output +5004.003 + + +-- !query 98 +select date_part('mseconds', c) from t2 +-- !query 98 schema +struct +-- !query 98 output +5004.003 + + +-- !query 99 +select date_part('ms', c) from t2 +-- !query 99 schema +struct +-- !query 99 output +5004.003 + + +-- !query 100 +select date_part('microseconds', c) from t2 +-- !query 100 schema +struct +-- !query 100 output +5004003 + + +-- !query 101 +select date_part('usec', c) from t2 +-- !query 101 schema +struct +-- !query 101 output +5004003 + + +-- !query 102 +select date_part('usecs', c) from t2 +-- !query 102 schema +struct +-- !query 102 output +5004003 + + +-- !query 103 +select date_part('useconds', c) from t2 +-- !query 103 schema +struct +-- !query 103 output +5004003 + + +-- !query 104 +select date_part('microsecon', c) from t2 +-- !query 104 schema +struct +-- !query 104 output +5004003 + + +-- !query 105 +select date_part('us', c) from t2 +-- !query 105 schema +struct +-- !query 105 output +5004003 + + +-- !query 106 +select date_part('epoch', c) from t2 +-- !query 106 schema +struct +-- !query 106 output +31873892788.332003 + + +-- !query 107 +select date_part('not_supported', c) from t2 +-- !query 107 schema +struct<> +-- !query 107 output +org.apache.spark.sql.AnalysisException +Literals of type 'not_supported' are currently not supported for the interval type.;; line 1 pos 7 + + +-- !query 108 +select date_part(c, c) from t2 +-- !query 108 schema +struct<> +-- !query 108 output +org.apache.spark.sql.AnalysisException +The field parameter needs to be a foldable string value.;; line 1 pos 7 + + +-- !query 109 +select date_part(null, c) from t2 +-- !query 109 schema +struct +-- !query 109 output +NULL From 9e42c52c7728250547aa044f8c28f93de9940460 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 18 Oct 2019 10:58:17 -0500 Subject: [PATCH 0366/1387] [MINOR][DOCS] Fix incorrect EqualNullSafe symbol in sql-migration-guide.md ### What changes were proposed in this pull request? This PR fixes the incorrect `EqualNullSafe` symbol in `sql-migration-guide.md`. ### Why are the changes needed? Fix documentation error. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #26163 from wangyum/EqualNullSafe-symbol. Authored-by: Yuming Wang Signed-off-by: Sean Owen --- docs/sql-migration-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index f250fec7d0689..d03ca663e8e3f 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -802,7 +802,7 @@ Spark SQL supports the vast majority of Hive features, such as: * `CLUSTER BY` * `SORT BY` * All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Relational operators (`=`, `<=>`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) * Logical operators (`AND`, `&&`, `OR`, `||`, etc) * Complex type constructors From 9a3dccae727994cfc9f4c07d9457f4fb2c27237c Mon Sep 17 00:00:00 2001 From: angerszhu Date: Sat, 19 Oct 2019 00:19:56 +0800 Subject: [PATCH 0367/1387] [SPARK-29379][SQL] SHOW FUNCTIONS show '!=', '<>' , 'between', 'case' ### What changes were proposed in this pull request? Current Spark SQL `SHOW FUNCTIONS` don't show `!=`, `<>`, `between`, `case` But these expressions is truly functions. We should show it in SQL `SHOW FUNCTIONS` ### Why are the changes needed? SHOW FUNCTIONS show '!=', '<>' , 'between', 'case' ### Does this PR introduce any user-facing change? SHOW FUNCTIONS show '!=', '<>' , 'between', 'case' ### How was this patch tested? UT Closes #26053 from AngersZhuuuu/SPARK-29379. Authored-by: angerszhu Signed-off-by: Wenchen Fan --- .../sql/execution/command/functions.scala | 18 +++++++++++++++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 4 +++- .../spark/sql/execution/command/DDLSuite.scala | 3 ++- .../sql/hive/execution/HiveUDFSuite.scala | 4 +++- .../sql/hive/execution/SQLQuerySuite.scala | 7 ++++++- 5 files changed, 31 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index d3b2491cd7056..6fdc7f4a58195 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchFunctionException} import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, FunctionResource} import org.apache.spark.sql.catalyst.expressions.{Attribute, ExpressionInfo} +import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.types.{StringType, StructField, StructType} @@ -222,6 +223,21 @@ case class ShowFunctionsCommand( case (f, "USER") if showUserFunctions => f.unquotedString case (f, "SYSTEM") if showSystemFunctions => f.unquotedString } - functionNames.sorted.map(Row(_)) + // Hard code "<>", "!=", "between", and "case" for now as there is no corresponding functions. + // "<>", "!=", "between", and "case" is SystemFunctions, only show when showSystemFunctions=true + if (showSystemFunctions) { + (functionNames ++ + StringUtils.filterPattern(FunctionsCommand.virtualOperators, pattern.getOrElse("*"))) + .sorted.map(Row(_)) + } else { + functionNames.sorted.map(Row(_)) + } + } } + +object FunctionsCommand { + // operators that do not have corresponding functions. + // They should be handled `DescribeFunctionCommand`, `ShowFunctionsCommand` + val virtualOperators = Seq("!=", "<>", "between", "case") +} 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 1afe3976b2a1a..630489ad9c603 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 @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.command.FunctionsCommand import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan @@ -59,7 +60,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { test("show functions") { def getFunctions(pattern: String): Seq[Row] = { StringUtils.filterPattern( - spark.sessionState.catalog.listFunctions("default").map(_._1.funcName), pattern) + spark.sessionState.catalog.listFunctions("default").map(_._1.funcName) + ++ FunctionsCommand.virtualOperators, pattern) .map(Row(_)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 16348096013ec..8dad242a8d618 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2064,7 +2064,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { test("show functions") { withUserDefinedFunction("add_one" -> true) { - val numFunctions = FunctionRegistry.functionSet.size.toLong + val numFunctions = FunctionRegistry.functionSet.size.toLong + + FunctionsCommand.virtualOperators.size.toLong assert(sql("show functions").count() === numFunctions) assert(sql("show system functions").count() === numFunctions) assert(sql("show all functions").count() === numFunctions) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 587eab4a24810..aa694ea274d75 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.io.{LongWritable, Writable} import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.execution.command.FunctionsCommand import org.apache.spark.sql.functions.max import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -563,7 +564,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { checkAnswer( sql("SELECT testUDFToListInt(s) FROM inputTable"), Seq(Row(Seq(1, 2, 3)))) - assert(sql("show functions").count() == numFunc + 1) + assert(sql("show functions").count() == + numFunc + FunctionsCommand.virtualOperators.size + 1) assert(spark.catalog.listFunctions().count() == numFunc + 1) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 79a490b8a2a36..116a62b2f6dc6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, Functio import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, CatalogUtils, HiveTableRelation} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} -import org.apache.spark.sql.execution.command.LoadDataCommand +import org.apache.spark.sql.execution.command.{FunctionsCommand, LoadDataCommand} import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} @@ -192,6 +192,11 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { allBuiltinFunctions.foreach { f => assert(allFunctions.contains(f)) } + + FunctionsCommand.virtualOperators.foreach { f => + assert(allFunctions.contains(f)) + } + withTempDatabase { db => def createFunction(names: Seq[String]): Unit = { names.foreach { name => From 23f45f18223e3ca34bc5e43e2b1d952c844f0d19 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Fri, 18 Oct 2019 21:30:36 +0000 Subject: [PATCH 0368/1387] [SPARK-29515][CORE] MapStatuses SerDeser Benchmark ### What changes were proposed in this pull request? Add benchmark code for MapStatuses serialization & deserialization performance. ### Why are the changes needed? For comparing the performance differences against optimization. ### Does this PR introduce any user-facing change? No ### How was this patch tested? No test is required. Closes #26169 from dbtsai/benchmark. Lead-authored-by: DB Tsai Co-authored-by: Dongjoon Hyun Co-authored-by: DB Tsai Signed-off-by: DB Tsai --- ...tatusesSerDeserBenchmark-jdk11-results.txt | 66 +++++++++ .../MapStatusesSerDeserBenchmark-results.txt | 66 +++++++++ .../org/apache/spark/MapOutputTracker.scala | 4 +- .../spark/MapStatusesSerDeserBenchmark.scala | 130 ++++++++++++++++++ 4 files changed, 264 insertions(+), 2 deletions(-) create mode 100644 core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt create mode 100644 core/benchmarks/MapStatusesSerDeserBenchmark-results.txt create mode 100644 core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..747aae09272f9 --- /dev/null +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt @@ -0,0 +1,66 @@ +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 609 631 22 0.3 3043.8 1.0X +Deserialization 840 897 67 0.2 4201.2 0.7X + +Compressed Serialized MapStatus sizes: 393 bytes +Compressed Serialized Broadcast MapStatus sizes: 3 MB + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 591 599 8 0.3 2955.3 1.0X +Deserialization 878 913 31 0.2 4392.2 0.7X + +Compressed Serialized MapStatus sizes: 3 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 1776 1778 2 0.1 8880.5 1.0X +Deserialization 1086 1086 0 0.2 5427.9 1.6X + +Compressed Serialized MapStatus sizes: 411 bytes +Compressed Serialized Broadcast MapStatus sizes: 15 MB + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 1725 1726 1 0.1 8624.9 1.0X +Deserialization 1093 1094 2 0.2 5463.6 1.6X + +Compressed Serialized MapStatus sizes: 15 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 12421 12522 142 0.0 62104.4 1.0X +Deserialization 3020 3043 32 0.1 15102.0 4.1X + +Compressed Serialized MapStatus sizes: 544 bytes +Compressed Serialized Broadcast MapStatus sizes: 131 MB + + +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 11719 11737 26 0.0 58595.3 1.0X +Deserialization 3018 3051 46 0.1 15091.7 3.9X + +Compressed Serialized MapStatus sizes: 131 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt new file mode 100644 index 0000000000000..1f479a49d5860 --- /dev/null +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt @@ -0,0 +1,66 @@ +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 625 639 9 0.3 3127.2 1.0X +Deserialization 875 931 49 0.2 4376.2 0.7X + +Compressed Serialized MapStatus sizes: 393 bytes +Compressed Serialized Broadcast MapStatus sizes: 3 MB + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 604 640 71 0.3 3018.4 1.0X +Deserialization 889 903 17 0.2 4443.8 0.7X + +Compressed Serialized MapStatus sizes: 3 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 1879 1880 2 0.1 9394.9 1.0X +Deserialization 1147 1150 5 0.2 5733.8 1.6X + +Compressed Serialized MapStatus sizes: 411 bytes +Compressed Serialized Broadcast MapStatus sizes: 15 MB + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 1825 1826 1 0.1 9123.3 1.0X +Deserialization 1147 1281 189 0.2 5735.7 1.6X + +Compressed Serialized MapStatus sizes: 15 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 12327 12518 270 0.0 61634.3 1.0X +Deserialization 3120 3133 18 0.1 15600.8 4.0X + +Compressed Serialized MapStatus sizes: 544 bytes +Compressed Serialized Broadcast MapStatus sizes: 131 MB + + +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Serialization 11928 11986 82 0.0 59642.2 1.0X +Deserialization 3137 3138 2 0.1 15683.3 3.8X + +Compressed Serialized MapStatus sizes: 131 MB +Compressed Serialized Broadcast MapStatus sizes: 0 bytes + + diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 24da855633db3..c181fac8b4d8e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -371,8 +371,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging */ private[spark] class MapOutputTrackerMaster( conf: SparkConf, - broadcastManager: BroadcastManager, - isLocal: Boolean) + private[spark] val broadcastManager: BroadcastManager, + private[spark] val isLocal: Boolean) extends MapOutputTracker(conf) { // The size at which we use Broadcast to send the map output statuses to the executors diff --git a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala new file mode 100644 index 0000000000000..53afe141981f4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala @@ -0,0 +1,130 @@ +/* + * 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 + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.benchmark.BenchmarkBase +import org.apache.spark.scheduler.CompressedMapStatus +import org.apache.spark.storage.BlockManagerId + +/** + * Benchmark for MapStatuses serialization & deserialization performance. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class --jars + * 2. build/sbt "core/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " + * Results will be written to "benchmarks/MapStatusesSerDeserBenchmark-results.txt". + * }}} + */ +object MapStatusesSerDeserBenchmark extends BenchmarkBase { + + var sc: SparkContext = null + var tracker: MapOutputTrackerMaster = null + + def serDeserBenchmark(numMaps: Int, blockSize: Int, enableBroadcast: Boolean): Unit = { + val minBroadcastSize = if (enableBroadcast) { + 0 + } else { + Int.MaxValue + } + + val benchmark = new Benchmark(s"$numMaps MapOutputs, $blockSize blocks " + { + if (enableBroadcast) "w/ " else "w/o " + } + "broadcast", numMaps, output = output) + + val shuffleId = 10 + + tracker.registerShuffle(shuffleId, numMaps) + val r = new scala.util.Random(912) + (0 until numMaps).foreach { i => + tracker.registerMapOutput(shuffleId, i, + new CompressedMapStatus(BlockManagerId(s"node$i", s"node$i.spark.apache.org", 1000), + Array.fill(blockSize) { + // Creating block size ranging from 0byte to 1GB + (r.nextDouble() * 1024 * 1024 * 1024).toLong + }, i)) + } + + val shuffleStatus = tracker.shuffleStatuses.get(shuffleId).head + + var serializedMapStatusSizes = 0 + var serializedBroadcastSizes = 0 + + val (serializedMapStatus, serializedBroadcast) = MapOutputTracker.serializeMapStatuses( + shuffleStatus.mapStatuses, tracker.broadcastManager, tracker.isLocal, minBroadcastSize) + serializedMapStatusSizes = serializedMapStatus.length + if (serializedBroadcast != null) { + serializedBroadcastSizes = serializedBroadcast.value.length + } + + benchmark.addCase("Serialization") { _ => + MapOutputTracker.serializeMapStatuses( + shuffleStatus.mapStatuses, tracker.broadcastManager, tracker.isLocal, minBroadcastSize) + } + + benchmark.addCase("Deserialization") { _ => + val result = MapOutputTracker.deserializeMapStatuses(serializedMapStatus) + assert(result.length == numMaps) + } + + benchmark.run() + // scalastyle:off + import org.apache.commons.io.FileUtils + benchmark.out.println("Compressed Serialized MapStatus sizes: " + + FileUtils.byteCountToDisplaySize(serializedMapStatusSizes)) + benchmark.out.println("Compressed Serialized Broadcast MapStatus sizes: " + + FileUtils.byteCountToDisplaySize(serializedBroadcastSizes) + "\n\n") + // scalastyle:on + + tracker.unregisterShuffle(shuffleId) + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + createSparkContext() + tracker = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val rpcEnv = sc.env.rpcEnv + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, sc.getConf) + rpcEnv.stop(tracker.trackerEndpoint) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) + + serDeserBenchmark(200000, 10, true) + serDeserBenchmark(200000, 10, false) + + serDeserBenchmark(200000, 100, true) + serDeserBenchmark(200000, 100, false) + + serDeserBenchmark(200000, 1000, true) + serDeserBenchmark(200000, 1000, false) + } + + def createSparkContext(): Unit = { + val conf = new SparkConf() + if (sc != null) { + sc.stop() + } + sc = new SparkContext("local", "MapStatusesSerializationBenchmark", conf) + } + + override def afterAll(): Unit = { + tracker.stop() + if (sc != null) { + sc.stop() + } + } +} From 4cfce3e5d03b0badb4e9685499be2ab0fca5747a Mon Sep 17 00:00:00 2001 From: Rahul Mahadev Date: Fri, 18 Oct 2019 16:45:25 -0500 Subject: [PATCH 0369/1387] [SPARK-29494][SQL] Fix for ArrayOutofBoundsException while converting string to timestamp ### What changes were proposed in this pull request? * Adding an additional check in `stringToTimestamp` to handle cases where the input has trailing ':' * Added a test to make sure this works. ### Why are the changes needed? In a couple of scenarios while converting from String to Timestamp `DateTimeUtils.stringToTimestamp` throws an array out of bounds exception if there is trailing ':'. The behavior of this method requires it to return `None` in case the format of the string is incorrect. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added a test in the `DateTimeTestUtils` suite to test if my fix works. Closes #26143 from rahulsmahadev/SPARK-29494. Lead-authored-by: Rahul Mahadev Co-authored-by: Rahul Shivu Mahadev <51690557+rahulsmahadev@users.noreply.github.com> Signed-off-by: Sean Owen --- .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 2 +- .../apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 088876921dccd..8dd541b1e4381 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -286,7 +286,7 @@ object DateTimeUtils { i += 1 } } else { - if (b == ':' || b == ' ') { + if (i < segments.length && (b == ':' || b == ' ')) { segments(i) = currentSegmentValue currentSegmentValue = 0 i += 1 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index a9e3c9006a334..1abbca6c8cd29 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -456,6 +456,12 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { } } + test("trailing characters while converting string to timestamp") { + val s = UTF8String.fromString("2019-10-31T10:59:23Z:::") + val time = DateTimeUtils.stringToTimestamp(s, defaultZoneId) + assert(time == None) + } + test("truncTimestamp") { def testTrunc( level: Int, From e4b4a35de253fa55245fe1adf54577b3217873d8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 18 Oct 2019 15:39:44 -0700 Subject: [PATCH 0370/1387] [SPARK-29466][WEBUI] Show `Duration` for running drivers in Standalone master web UI ### What changes were proposed in this pull request? This PR aims to add a new column `Duration` for running drivers in Apache Spark `Standalone` master web UI in order to improve UX. This help users like the other `Duration` columns in the `Running` and `Completed` application tables. ### Why are the changes needed? When we use `--supervise`, the drivers can survive longer. Technically, the `Duration` column is not the same. (Please see the image below.) ### Does this PR introduce any user-facing change? Yes. The red box is added newly. Screen Shot 2019-10-14 at 12 53 43 PM ### How was this patch tested? Manual since this is a UI column. After starting standalone cluster and jobs, kill the `DriverWrapper` and see the UI. ``` $ sbin/start-master.sh $ sbin/start-slave.sh spark://$(hostname):7077 $ bin/spark-submit --master spark://(hostname):7077 --deploy-mode cluster --supervise --class org.apache.spark.examples.JavaSparkPi examples/target/scala-2.12/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar 1000 $ jps 41521 DriverWrapper ... $ kill -9 41521 // kill the `DriverWrapper`. ``` Closes #26113 from dongjoon-hyun/SPARK-29466. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../spark/deploy/master/ui/MasterPage.scala | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 042ec54ee1240..f64b449851d86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -108,12 +108,17 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { val completedApps = state.completedApps.sortBy(_.endTime).reverse val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) - val driverHeaders = Seq("Submission ID", "Submitted Time", "Worker", "State", "Cores", - "Memory", "Resources", "Main Class") + val activeDriverHeaders = Seq("Submission ID", "Submitted Time", "Worker", "State", "Cores", + "Memory", "Resources", "Main Class", "Duration") val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse - val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers) + val activeDriversTable = + UIUtils.listingTable(activeDriverHeaders, activeDriverRow, activeDrivers) + + val completedDriverHeaders = Seq("Submission ID", "Submitted Time", "Worker", "State", "Cores", + "Memory", "Resources", "Main Class") val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse - val completedDriversTable = UIUtils.listingTable(driverHeaders, driverRow, completedDrivers) + val completedDriversTable = + UIUtils.listingTable(completedDriverHeaders, completedDriverRow, completedDrivers) // For now we only show driver information if the user has submitted drivers to the cluster. // This is until we integrate the notion of drivers and applications in the UI. @@ -310,7 +315,11 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } - private def driverRow(driver: DriverInfo): Seq[Node] = { + private def activeDriverRow(driver: DriverInfo) = driverRow(driver, showDuration = true) + + private def completedDriverRow(driver: DriverInfo) = driverRow(driver, showDuration = false) + + private def driverRow(driver: DriverInfo, showDuration: Boolean): Seq[Node] = { val killLink = if (parent.killEnabled && (driver.state == DriverState.RUNNING || driver.state == DriverState.SUBMITTED || @@ -345,6 +354,9 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {formatResourcesAddresses(driver.resources)} {driver.desc.command.arguments(2)} + {if (showDuration) { + {UIUtils.formatDuration(System.currentTimeMillis() - driver.startTime)} + }} } } From 24378782992cd2029633b069b6e08d7616aa34ff Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 18 Oct 2019 16:12:03 -0700 Subject: [PATCH 0371/1387] [SPARK-29502][SQL] typed interval expression should fail for invalid format ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/25241 . The typed interval expression should fail for invalid format. ### Why are the changes needed? Te be consistent with the typed timestamp/date expression ### Does this PR introduce any user-facing change? Yes. But this feature is not released yet. ### How was this patch tested? updated test Closes #26151 from cloud-fan/bug. Authored-by: Wenchen Fan Signed-off-by: Yuming Wang --- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 10 +++++++++- .../sql/catalyst/parser/ExpressionParserSuite.scala | 7 +++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 6a108901dd329..9c631c54ad2d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1769,7 +1769,15 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val zoneId = getZoneId(SQLConf.get.sessionLocalTimeZone) toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "INTERVAL" => - Literal(CalendarInterval.fromString(value), CalendarIntervalType) + val interval = try { + CalendarInterval.fromCaseInsensitiveString(value) + } catch { + case e: IllegalArgumentException => + val ex = new ParseException("Cannot parse the INTERVAL value: " + value, ctx) + ex.setStackTrace(e.getStackTrace) + throw ex + } + Literal(interval, CalendarIntervalType) case "X" => val padding = if (value.length % 2 != 0) "0" else "" Literal(DatatypeConverter.parseHexBinary(padding + value)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index c2e80c639f43b..e6eabcc1f3022 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -424,19 +424,18 @@ class ExpressionParserSuite extends AnalysisTest { test("type constructors") { // Dates. assertEqual("dAte '2016-03-11'", Literal(Date.valueOf("2016-03-11"))) - intercept("DAtE 'mar 11 2016'") + intercept("DAtE 'mar 11 2016'", "Cannot parse the DATE value") // Timestamps. assertEqual("tImEstAmp '2016-03-11 20:54:00.000'", Literal(Timestamp.valueOf("2016-03-11 20:54:00.000"))) - intercept("timestamP '2016-33-11 20:54:00.000'") + intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the TIMESTAMP value") // Interval. val intervalLiteral = Literal(CalendarInterval.fromString("interval 3 month 1 hour")) assertEqual("InterVal 'interval 3 month 1 hour'", intervalLiteral) assertEqual("INTERVAL '3 month 1 hour'", intervalLiteral) - assertEqual("Interval 'interval 3 monthsss 1 hoursss'", - Literal(null, CalendarIntervalType)) + intercept("Interval 'interval 3 monthsss 1 hoursss'", "Cannot parse the INTERVAL value") // Binary. assertEqual("X'A'", Literal(Array(0x0a).map(_.toByte))) From ab92e1715e7ac0df3d07635e4d276985fbf97e13 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 18 Oct 2019 22:43:58 -0700 Subject: [PATCH 0372/1387] [SPARK-29512][SQL] REPAIR TABLE should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add RepairTableStatement and make REPAIR TABLE go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog MSCK REPAIR TABLE t // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? yes. When running MSCK REPAIR TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? New unit tests Closes #26168 from imback82/repair_table. Authored-by: Terry Kim Signed-off-by: Liang-Chi Hsieh --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 12 +++++++ .../catalyst/plans/logical/statements.scala | 5 +++ .../sql/catalyst/parser/DDLParserSuite.scala | 6 ++++ .../analysis/ResolveSessionCatalog.scala | 32 ++++++++++++------- .../spark/sql/execution/SparkSqlParser.scala | 14 -------- .../sql/connector/DataSourceV2SQLSuite.scala | 12 +++++++ .../execution/command/DDLParserSuite.scala | 9 ------ 8 files changed, 56 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 878c08a79813c..1839203e3b235 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -212,7 +212,7 @@ statement | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData | TRUNCATE TABLE tableIdentifier partitionSpec? #truncateTable - | MSCK REPAIR TABLE tableIdentifier #repairTable + | MSCK REPAIR TABLE multipartIdentifier #repairTable | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 9c631c54ad2d4..8af7cf9ad8008 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2716,4 +2716,16 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging tableName, Option(visitIdentifierSeq(ctx.identifierSeq())), allColumns = false) } } + + /** + * Create a [[RepairTableStatement]]. + * + * For example: + * {{{ + * MSCK REPAIR TABLE multi_part_name + * }}} + */ + override def visitRepairTable(ctx: RepairTableContext): LogicalPlan = withOrigin(ctx) { + RepairTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 208ecd0e99c88..72d5cbb7d9045 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -311,3 +311,8 @@ case class AnalyzeColumnStatement( require(columnNames.isDefined ^ allColumns, "Parameter `columnNames` or `allColumns` are " + "mutually exclusive. Only one of them should be specified.") } + +/** + * A REPAIR TABLE statement, as parsed from SQL + */ +case class RepairTableStatement(tableName: Seq[String]) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index b42fec201352e..0eaf74f655065 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -955,6 +955,12 @@ class DDLParserSuite extends AnalysisTest { "missing 'COLUMNS' at ''") } + test("MSCK REPAIR table") { + comparePlans( + parsePlan("MSCK REPAIR TABLE a.b.c"), + RepairTableStatement(Seq("a", "b", "c"))) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index a781e2fb258ea..72f539f72008d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -266,22 +266,30 @@ class ResolveSessionCatalog( ShowTablesCommand(None, pattern) case AnalyzeTableStatement(tableName, partitionSpec, noScan) => - val CatalogAndIdentifierParts(catalog, parts) = tableName - if (!isSessionCatalog(catalog)) { - throw new AnalysisException("ANALYZE TABLE is only supported with v1 tables.") - } + val v1TableName = parseV1Table(tableName, "ANALYZE TABLE") if (partitionSpec.isEmpty) { - AnalyzeTableCommand(parts.asTableIdentifier, noScan) + AnalyzeTableCommand(v1TableName.asTableIdentifier, noScan) } else { - AnalyzePartitionCommand(parts.asTableIdentifier, partitionSpec, noScan) + AnalyzePartitionCommand(v1TableName.asTableIdentifier, partitionSpec, noScan) } case AnalyzeColumnStatement(tableName, columnNames, allColumns) => - val CatalogAndIdentifierParts(catalog, parts) = tableName - if (!isSessionCatalog(catalog)) { - throw new AnalysisException("ANALYZE TABLE is only supported with v1 tables.") - } - AnalyzeColumnCommand(parts.asTableIdentifier, columnNames, allColumns) + val v1TableName = parseV1Table(tableName, "ANALYZE TABLE") + AnalyzeColumnCommand(v1TableName.asTableIdentifier, columnNames, allColumns) + + case RepairTableStatement(tableName) => + val v1TableName = parseV1Table(tableName, "MSCK REPAIR TABLE") + AlterTableRecoverPartitionsCommand( + v1TableName.asTableIdentifier, + "MSCK REPAIR TABLE") + } + + private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { + val CatalogAndIdentifierParts(catalog, parts) = tableName + if (!isSessionCatalog(catalog)) { + throw new AnalysisException(s"$sql is only supported with v1 tables.") + } + parts } private def buildCatalogTable( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index df63787fa508b..3e7a54877cae8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -360,20 +360,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } - /** - * Create a [[AlterTableRecoverPartitionsCommand]] command. - * - * For example: - * {{{ - * MSCK REPAIR TABLE tablename - * }}} - */ - override def visitRepairTable(ctx: RepairTableContext): LogicalPlan = withOrigin(ctx) { - AlterTableRecoverPartitionsCommand( - visitTableIdentifier(ctx.tableIdentifier), - "MSCK REPAIR TABLE") - } - /** * Create a [[CreateDatabaseCommand]] command. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 1d2bc9d4c8eea..d253e6078ddc0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1198,6 +1198,18 @@ class DataSourceV2SQLSuite } } + test("MSCK REPAIR TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + val e = intercept[AnalysisException] { + sql(s"MSCK REPAIR TABLE $t") + } + assert(e.message.contains("MSCK REPAIR TABLE is only supported with v1 tables")) + } + } + private def assertAnalysisError(sqlStatement: String, expectedError: String): Unit = { val errMsg = intercept[AnalysisException] { sql(sqlStatement) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 303630d9d0cb9..5a5899cbccc55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -1444,15 +1444,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { intercept(sql2, "Found duplicate clauses: TBLPROPERTIES") } - test("MSCK REPAIR table") { - val sql = "MSCK REPAIR TABLE tab1" - val parsed = parser.parsePlan(sql) - val expected = AlterTableRecoverPartitionsCommand( - TableIdentifier("tab1", None), - "MSCK REPAIR TABLE") - comparePlans(parsed, expected) - } - test("create table like") { val v1 = "CREATE TABLE table1 LIKE table2" val (target, source, location, exists) = parser.parsePlan(v1).collect { From 4a6005c795926929a7963ae8cc75cf553f69a9ed Mon Sep 17 00:00:00 2001 From: shahid Date: Sat, 19 Oct 2019 15:23:57 -0500 Subject: [PATCH 0373/1387] [SPARK-29235][ML][PYSPARK] Support avgMetrics in read/write of CrossValidatorModel ### What changes were proposed in this pull request? Currently pyspark doesn't write/read `avgMetrics` in `CrossValidatorModel`, whereas scala supports it. ### Why are the changes needed? Test step to reproduce it: ``` dataset = spark.createDataFrame([(Vectors.dense([0.0]), 0.0), (Vectors.dense([0.4]), 1.0), (Vectors.dense([0.5]), 0.0), (Vectors.dense([0.6]), 1.0), (Vectors.dense([1.0]), 1.0)] * 10, ["features", "label"]) lr = LogisticRegression() grid = ParamGridBuilder().addGrid(lr.maxIter, [0, 1]).build() evaluator = BinaryClassificationEvaluator() cv = CrossValidator(estimator=lr, estimatorParamMaps=grid, evaluator=evaluator,parallelism=2) cvModel = cv.fit(dataset) cvModel.write().save("/tmp/model") cvModel2 = CrossValidatorModel.read().load("/tmp/model") print(cvModel.avgMetrics) # prints non empty result as expected print(cvModel2.avgMetrics) # Bug: prints an empty result. ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested Before patch: ``` >>> cvModel.write().save("/tmp/model_0") >>> cvModel2 = CrossValidatorModel.read().load("/tmp/model_0") >>> print(cvModel2.avgMetrics) [] ``` After patch: ``` >>> cvModel2 = CrossValidatorModel.read().load("/tmp/model_2") >>> print(cvModel2.avgMetrics[0]) 0.5 ``` Closes #26038 from shahidki31/avgMetrics. Authored-by: shahid Signed-off-by: Sean Owen --- python/pyspark/ml/tuning.py | 35 ++++++++++++++++++++++++++++------- 1 file changed, 28 insertions(+), 7 deletions(-) diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 8fa0183e4683d..8052163acd00a 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -22,7 +22,7 @@ from pyspark import since, keyword_only from pyspark.ml import Estimator, Model -from pyspark.ml.common import _py2java +from pyspark.ml.common import _py2java, _java2py from pyspark.ml.param import Params, Param, TypeConverters from pyspark.ml.param.shared import HasCollectSubModels, HasParallelism, HasSeed from pyspark.ml.util import * @@ -216,6 +216,8 @@ class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollec >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.ml.tuning import CrossValidatorModel + >>> import tempfile >>> dataset = spark.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), @@ -233,6 +235,12 @@ class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollec 3 >>> cvModel.avgMetrics[0] 0.5 + >>> path = tempfile.mkdtemp() + >>> model_path = path + "/model" + >>> cvModel.write().save(model_path) + >>> cvModelRead = CrossValidatorModel.read().load(model_path) + >>> cvModelRead.avgMetrics + [0.5, ... >>> evaluator.evaluate(cvModel.transform(dataset)) 0.8333... @@ -483,10 +491,12 @@ def _from_java(cls, java_stage): Given a Java CrossValidatorModel, create and return a Python wrapper of it. Used for ML persistence. """ + sc = SparkContext._active_spark_context bestModel = JavaParams._from_java(java_stage.bestModel()) + avgMetrics = _java2py(sc, java_stage.avgMetrics()) estimator, epms, evaluator = super(CrossValidatorModel, cls)._from_java_impl(java_stage) - py_stage = cls(bestModel=bestModel).setEstimator(estimator) + py_stage = cls(bestModel=bestModel, avgMetrics=avgMetrics).setEstimator(estimator) py_stage = py_stage.setEstimatorParamMaps(epms).setEvaluator(evaluator) if java_stage.hasSubModels(): @@ -505,11 +515,10 @@ def _to_java(self): """ sc = SparkContext._active_spark_context - # TODO: persist average metrics as well _java_obj = JavaParams._new_java_obj("org.apache.spark.ml.tuning.CrossValidatorModel", self.uid, self.bestModel._to_java(), - _py2java(sc, [])) + _py2java(sc, self.avgMetrics)) estimator, epms, evaluator = super(CrossValidatorModel, self)._to_java_impl() _java_obj.set("evaluator", evaluator) @@ -551,6 +560,8 @@ class TrainValidationSplit(Estimator, _TrainValidationSplitParams, HasParallelis >>> from pyspark.ml.classification import LogisticRegression >>> from pyspark.ml.evaluation import BinaryClassificationEvaluator >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.ml.tuning import TrainValidationSplitModel + >>> import tempfile >>> dataset = spark.createDataFrame( ... [(Vectors.dense([0.0]), 0.0), ... (Vectors.dense([0.4]), 1.0), @@ -566,6 +577,14 @@ class TrainValidationSplit(Estimator, _TrainValidationSplitParams, HasParallelis >>> tvsModel = tvs.fit(dataset) >>> tvsModel.getTrainRatio() 0.75 + >>> tvsModel.validationMetrics + [0.5, ... + >>> path = tempfile.mkdtemp() + >>> model_path = path + "/model" + >>> tvsModel.write().save(model_path) + >>> tvsModelRead = TrainValidationSplitModel.read().load(model_path) + >>> tvsModelRead.validationMetrics + [0.5, ... >>> evaluator.evaluate(tvsModel.transform(dataset)) 0.833... @@ -809,11 +828,14 @@ def _from_java(cls, java_stage): """ # Load information from java_stage to the instance. + sc = SparkContext._active_spark_context bestModel = JavaParams._from_java(java_stage.bestModel()) + validationMetrics = _java2py(sc, java_stage.validationMetrics()) estimator, epms, evaluator = super(TrainValidationSplitModel, cls)._from_java_impl(java_stage) # Create a new instance of this stage. - py_stage = cls(bestModel=bestModel).setEstimator(estimator) + py_stage = cls(bestModel=bestModel, + validationMetrics=validationMetrics).setEstimator(estimator) py_stage = py_stage.setEstimatorParamMaps(epms).setEvaluator(evaluator) if java_stage.hasSubModels(): @@ -830,12 +852,11 @@ def _to_java(self): """ sc = SparkContext._active_spark_context - # TODO: persst validation metrics as well _java_obj = JavaParams._new_java_obj( "org.apache.spark.ml.tuning.TrainValidationSplitModel", self.uid, self.bestModel._to_java(), - _py2java(sc, [])) + _py2java(sc, self.validationMetrics)) estimator, epms, evaluator = super(TrainValidationSplitModel, self)._to_java_impl() _java_obj.set("evaluator", evaluator) From 0f65b49f55a09eb3827a736d4473390d746e72a0 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 20 Oct 2019 13:31:59 -0700 Subject: [PATCH 0374/1387] [SPARK-29525][SQL][TEST] Fix the associated location already exists in SQLQueryTestSuite ### What changes were proposed in this pull request? This PR fix Fix the associated location already exists in `SQLQueryTestSuite`: ``` build/sbt "~sql/test-only *SQLQueryTestSuite -- -z postgreSQL/join.sql" ... [info] - postgreSQL/join.sql *** FAILED *** (35 seconds, 420 milliseconds) [info] postgreSQL/join.sql [info] Expected "[]", but got "[org.apache.spark.sql.AnalysisException [info] Can not create the managed table('`default`.`tt3`'). The associated location('file:/root/spark/sql/core/spark-warehouse/org.apache.spark.sql.SQLQueryTestSuite/tt3') already exists.;]" Result did not match for query #108 ``` ### Why are the changes needed? Fix bug. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #26181 from wangyum/TestError. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../sql-tests/inputs/postgreSQL/join.sql | 12 +- .../sql-tests/results/postgreSQL/join.sql.out | 472 ++++++++---------- 2 files changed, 228 insertions(+), 256 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql index 08f54fe0a40e5..1cfc463acc588 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql @@ -577,15 +577,15 @@ select count(*) from tenk1 a, tenk1 b -- regression test for 8.2 bug with improper re-ordering of left joins -- -DROP TABLE IF EXISTS tt3; -CREATE TABLE tt3(f1 int, f2 string) USING parquet; -INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x; +create or replace temporary view tt3 as select * from + (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) + as v(f1, f2); -- create index tt3i on tt3(f1); -- analyze tt3; -DROP TABLE IF EXISTS tt4; -CREATE TABLE tt4(f1 int) USING parquet; -INSERT INTO tt4 VALUES (0),(1),(9999); +create or replace temporary view tt4 as select * from + (values (0), (1), (9999)) + as v(f1); -- analyze tt4; SELECT a.f1 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out index f75fe0519645b..f823d4e1459d4 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 185 +-- Number of queries: 181 -- !query 0 @@ -2536,7 +2536,9 @@ struct -- !query 107 -DROP TABLE IF EXISTS tt3 +create or replace temporary view tt3 as select * from + (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) + as v(f1, f2) -- !query 107 schema struct<> -- !query 107 output @@ -2544,7 +2546,9 @@ struct<> -- !query 108 -CREATE TABLE tt3(f1 int, f2 string) USING parquet +create or replace temporary view tt4 as select * from + (values (0), (1), (9999)) + as v(f1) -- !query 108 schema struct<> -- !query 108 output @@ -2552,38 +2556,6 @@ struct<> -- !query 109 -INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x --- !query 109 schema -struct<> --- !query 109 output - - - --- !query 110 -DROP TABLE IF EXISTS tt4 --- !query 110 schema -struct<> --- !query 110 output - - - --- !query 111 -CREATE TABLE tt4(f1 int) USING parquet --- !query 111 schema -struct<> --- !query 111 output - - - --- !query 112 -INSERT INTO tt4 VALUES (0),(1),(9999) --- !query 112 schema -struct<> --- !query 112 output - - - --- !query 113 SELECT a.f1 FROM tt4 a LEFT JOIN ( @@ -2592,242 +2564,242 @@ LEFT JOIN ( WHERE c.f1 IS NULL ) AS d ON (a.f1 = d.f1) WHERE d.f1 IS NULL --- !query 113 schema +-- !query 109 schema struct --- !query 113 output +-- !query 109 output 0 1 9999 --- !query 114 +-- !query 110 create or replace temporary view tt5 as select * from (values (1, 10), (1, 11)) as v(f1, f2) --- !query 114 schema +-- !query 110 schema struct<> --- !query 114 output +-- !query 110 output --- !query 115 +-- !query 111 create or replace temporary view tt6 as select * from (values (1, 9), (1, 2), (2, 9)) as v(f1, f2) --- !query 115 schema +-- !query 111 schema struct<> --- !query 115 output +-- !query 111 output --- !query 116 +-- !query 112 select * from tt5,tt6 where tt5.f1 = tt6.f1 and tt5.f1 = tt5.f2 - tt6.f2 --- !query 116 schema +-- !query 112 schema struct --- !query 116 output +-- !query 112 output 1 10 1 9 --- !query 117 +-- !query 113 create or replace temporary view xx as select * from (values (1), (2), (3)) as v(pkxx) --- !query 117 schema +-- !query 113 schema struct<> --- !query 117 output +-- !query 113 output --- !query 118 +-- !query 114 create or replace temporary view yy as select * from (values (101, 1), (201, 2), (301, NULL)) as v(pkyy, pkxx) --- !query 118 schema +-- !query 114 schema struct<> --- !query 118 output +-- !query 114 output --- !query 119 +-- !query 115 select yy.pkyy as yy_pkyy, yy.pkxx as yy_pkxx, yya.pkyy as yya_pkyy, xxa.pkxx as xxa_pkxx, xxb.pkxx as xxb_pkxx from yy left join (SELECT * FROM yy where pkyy = 101) as yya ON yy.pkyy = yya.pkyy left join xx xxa on yya.pkxx = xxa.pkxx left join xx xxb on coalesce (xxa.pkxx, 1) = xxb.pkxx --- !query 119 schema +-- !query 115 schema struct --- !query 119 output +-- !query 115 output 101 1 101 1 1 201 2 NULL NULL 1 301 NULL NULL NULL 1 --- !query 120 +-- !query 116 create or replace temporary view zt1 as select * from (values (53)) as v(f1) --- !query 120 schema +-- !query 116 schema struct<> --- !query 120 output +-- !query 116 output --- !query 121 +-- !query 117 create or replace temporary view zt2 as select * from (values (53)) as v(f2) --- !query 121 schema +-- !query 117 schema struct<> --- !query 121 output +-- !query 117 output --- !query 122 +-- !query 118 create or replace temporary view zt3(f3 int) using parquet --- !query 122 schema +-- !query 118 schema struct<> --- !query 122 output +-- !query 118 output --- !query 123 +-- !query 119 select * from zt2 left join zt3 on (f2 = f3) left join zt1 on (f3 = f1) where f2 = 53 --- !query 123 schema +-- !query 119 schema struct --- !query 123 output +-- !query 119 output 53 NULL NULL --- !query 124 +-- !query 120 create temp view zv1 as select *,'dummy' AS junk from zt1 --- !query 124 schema +-- !query 120 schema struct<> --- !query 124 output +-- !query 120 output --- !query 125 +-- !query 121 select * from zt2 left join zt3 on (f2 = f3) left join zv1 on (f3 = f1) where f2 = 53 --- !query 125 schema +-- !query 121 schema struct --- !query 125 output +-- !query 121 output 53 NULL NULL NULL --- !query 126 +-- !query 122 select a.unique2, a.ten, b.tenthous, b.unique2, b.hundred from tenk1 a left join tenk1 b on a.unique2 = b.tenthous where a.unique1 = 42 and ((b.unique2 is null and a.ten = 2) or b.hundred = 3) --- !query 126 schema +-- !query 122 schema struct --- !query 126 output +-- !query 122 output --- !query 127 +-- !query 123 create or replace temporary view a (i integer) using parquet --- !query 127 schema +-- !query 123 schema struct<> --- !query 127 output +-- !query 123 output --- !query 128 +-- !query 124 create or replace temporary view b (x integer, y integer) using parquet --- !query 128 schema +-- !query 124 schema struct<> --- !query 128 output +-- !query 124 output --- !query 129 +-- !query 125 select * from a left join b on i = x and i = y and x = i --- !query 129 schema +-- !query 125 schema struct --- !query 129 output +-- !query 125 output --- !query 130 +-- !query 126 select t1.q2, count(t2.*) from int8_tbl t1 left join int8_tbl t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 130 schema +-- !query 126 schema struct --- !query 130 output +-- !query 126 output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 131 +-- !query 127 select t1.q2, count(t2.*) from int8_tbl t1 left join (select * from int8_tbl) t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 131 schema +-- !query 127 schema struct --- !query 131 output +-- !query 127 output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 132 +-- !query 128 select t1.q2, count(t2.*) from int8_tbl t1 left join (select q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 132 schema +-- !query 128 schema struct --- !query 132 output +-- !query 128 output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 133 +-- !query 129 create or replace temporary view a as select * from (values ('p'), ('q')) as v(code) --- !query 133 schema +-- !query 129 schema struct<> --- !query 133 output +-- !query 129 output --- !query 134 +-- !query 130 create or replace temporary view b as select * from (values ('p', 1), ('p', 2)) as v(a, num) --- !query 134 schema +-- !query 130 schema struct<> --- !query 134 output +-- !query 130 output --- !query 135 +-- !query 131 create or replace temporary view c as select * from (values ('A', 'p'), ('B', 'q'), ('C', null)) as v(name, a) --- !query 135 schema +-- !query 131 schema struct<> --- !query 135 output +-- !query 131 output --- !query 136 +-- !query 132 select c.name, ss.code, ss.b_cnt, ss.const from c left join (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const @@ -2837,15 +2809,15 @@ from c left join ) as ss on (c.a = ss.code) order by c.name --- !query 136 schema +-- !query 132 schema struct --- !query 136 output +-- !query 132 output A p 2 -1 B q 0 -1 C NULL NULL NULL --- !query 137 +-- !query 133 SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN @@ -2861,13 +2833,13 @@ LEFT JOIN ON sub4.key5 = sub3.key3 ) sub2 ON sub1.key1 = sub2.key3 --- !query 137 schema +-- !query 133 schema struct --- !query 137 output +-- !query 133 output 1 1 1 1 --- !query 138 +-- !query 134 SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN @@ -2883,13 +2855,13 @@ LEFT JOIN ON sub4.key5 = sub3.key3 ) sub2 ON sub1.key1 = sub2.key3 --- !query 138 schema +-- !query 134 schema struct --- !query 138 output +-- !query 134 output 1 1 1 1 --- !query 139 +-- !query 135 SELECT qq, unique1 FROM ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 @@ -2897,45 +2869,45 @@ SELECT qq, unique1 ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 USING (qq) INNER JOIN tenk1 c ON qq = unique2 --- !query 139 schema +-- !query 135 schema struct --- !query 139 output +-- !query 135 output 123 4596 123 4596 456 7318 --- !query 140 +-- !query 136 create or replace temporary view nt1 as select * from (values(1,true,true), (2,true,false), (3,false,false)) as v(id, a1, a2) --- !query 140 schema +-- !query 136 schema struct<> --- !query 140 output +-- !query 136 output --- !query 141 +-- !query 137 create or replace temporary view nt2 as select * from (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) as v(id, nt1_id, b1, b2) --- !query 141 schema +-- !query 137 schema struct<> --- !query 141 output +-- !query 137 output --- !query 142 +-- !query 138 create or replace temporary view nt3 as select * from (values(1,1,true), (2,2,false), (3,3,true)) as v(id, nt2_id, c1) --- !query 142 schema +-- !query 138 schema struct<> --- !query 142 output +-- !query 138 output --- !query 143 +-- !query 139 select nt3.id from nt3 as nt3 left join @@ -2947,17 +2919,17 @@ from nt3 as nt3 ) as ss2 on ss2.id = nt3.nt2_id where nt3.id = 1 and ss2.b3 --- !query 143 schema +-- !query 139 schema struct --- !query 143 output +-- !query 139 output 1 --- !query 144 +-- !query 140 select * from int4_tbl a full join int4_tbl b on true --- !query 144 schema +-- !query 140 schema struct --- !query 144 output +-- !query 140 output -123456 -123456 -123456 -2147483647 -123456 0 @@ -2985,11 +2957,11 @@ struct 2147483647 2147483647 --- !query 145 +-- !query 141 select * from int4_tbl a full join int4_tbl b on false --- !query 145 schema +-- !query 141 schema struct --- !query 145 output +-- !query 141 output -123456 NULL -2147483647 NULL 0 NULL @@ -3002,27 +2974,27 @@ NULL 123456 NULL 2147483647 --- !query 146 +-- !query 142 select count(*) from tenk1 a join tenk1 b on a.unique1 = b.unique2 left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand join int4_tbl on b.thousand = f1 --- !query 146 schema +-- !query 142 schema struct --- !query 146 output +-- !query 142 output 10 --- !query 147 +-- !query 143 select b.unique1 from tenk1 a join tenk1 b on a.unique1 = b.unique2 left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand join int4_tbl i1 on b.thousand = f1 right join int4_tbl i2 on i2.f1 = b.tenthous order by 1 --- !query 147 schema +-- !query 143 schema struct --- !query 147 output +-- !query 143 output NULL NULL NULL @@ -3030,7 +3002,7 @@ NULL 0 --- !query 148 +-- !query 144 select * from ( select unique1, q1, coalesce(unique1, -1) + q1 as fault @@ -3038,43 +3010,43 @@ select * from ) ss where fault = 122 order by fault --- !query 148 schema +-- !query 144 schema struct --- !query 148 output +-- !query 144 output NULL 123 122 --- !query 149 +-- !query 145 select q1, unique2, thousand, hundred from int8_tbl a left join tenk1 b on q1 = unique2 where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123) --- !query 149 schema +-- !query 145 schema struct --- !query 149 output +-- !query 145 output --- !query 150 +-- !query 146 select f1, unique2, case when unique2 is null then f1 else 0 end from int4_tbl a left join tenk1 b on f1 = unique2 where (case when unique2 is null then f1 else 0 end) = 0 --- !query 150 schema +-- !query 146 schema struct --- !query 150 output +-- !query 146 output 0 0 0 --- !query 151 +-- !query 147 select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44 --- !query 151 schema +-- !query 147 schema struct --- !query 151 output +-- !query 147 output --- !query 152 +-- !query 148 select * from text_tbl t1 inner join int8_tbl i8 @@ -3083,32 +3055,32 @@ select * from on t1.f1 = 'doh!' left join int4_tbl i4 on i8.q1 = i4.f1 --- !query 152 schema +-- !query 148 schema struct --- !query 152 output +-- !query 148 output doh! 123 456 doh! NULL doh! 123 456 hi de ho neighbor NULL --- !query 153 +-- !query 149 select * from (select 1 as id) as xx left join (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) on (xx.id = coalesce(yy.id)) --- !query 153 schema +-- !query 149 schema struct --- !query 153 output +-- !query 149 output 1 1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx 1 --- !query 154 +-- !query 150 select a.q2, b.q1 from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) where coalesce(b.q1, 1) > 0 --- !query 154 schema +-- !query 150 schema struct --- !query 154 output +-- !query 150 output -4567890123456789 NULL 123 123 123 123 @@ -3121,124 +3093,124 @@ struct 4567890123456789 4567890123456789 --- !query 155 +-- !query 151 create or replace temporary view parent as select * from (values (1, 10), (2, 20), (3, 30)) as v(k, pd) --- !query 155 schema +-- !query 151 schema struct<> --- !query 155 output +-- !query 151 output --- !query 156 +-- !query 152 create or replace temporary view child as select * from (values (1, 100), (4, 400)) as v(k, cd) --- !query 156 schema +-- !query 152 schema struct<> --- !query 156 output +-- !query 152 output --- !query 157 +-- !query 153 select p.* from parent p left join child c on (p.k = c.k) --- !query 157 schema +-- !query 153 schema struct --- !query 157 output +-- !query 153 output 1 10 2 20 3 30 --- !query 158 +-- !query 154 select p.*, linked from parent p left join (select c.*, true as linked from child c) as ss on (p.k = ss.k) --- !query 158 schema +-- !query 154 schema struct --- !query 158 output +-- !query 154 output 1 10 true 2 20 NULL 3 30 NULL --- !query 159 +-- !query 155 select p.* from parent p left join child c on (p.k = c.k) where p.k = 1 and p.k = 2 --- !query 159 schema +-- !query 155 schema struct --- !query 159 output +-- !query 155 output --- !query 160 +-- !query 156 select p.* from (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k where p.k = 1 and p.k = 2 --- !query 160 schema +-- !query 156 schema struct --- !query 160 output +-- !query 156 output --- !query 161 +-- !query 157 create or replace temporary view a as select * from (values (0), (1)) as v(id) --- !query 161 schema +-- !query 157 schema struct<> --- !query 161 output +-- !query 157 output --- !query 162 +-- !query 158 create or replace temporary view b as select * from (values (0, 0), (1, NULL)) as v(id, a_id) --- !query 162 schema +-- !query 158 schema struct<> --- !query 162 output +-- !query 158 output --- !query 163 +-- !query 159 SELECT * FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) --- !query 163 schema +-- !query 159 schema struct --- !query 163 output +-- !query 159 output 1 NULL NULL --- !query 164 +-- !query 160 SELECT b.* FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) --- !query 164 schema +-- !query 160 schema struct --- !query 164 output +-- !query 160 output 1 NULL --- !query 165 +-- !query 161 create or replace temporary view innertab as select * from (values (123L, 42L)) as v(id, dat1) --- !query 165 schema +-- !query 161 schema struct<> --- !query 165 output +-- !query 161 output --- !query 166 +-- !query 162 SELECT * FROM (SELECT 1 AS x) ss1 LEFT JOIN (SELECT q1, q2, COALESCE(dat1, q1) AS y FROM int8_tbl LEFT JOIN innertab ON q2 = id) ss2 ON true --- !query 166 schema +-- !query 162 schema struct --- !query 166 output +-- !query 162 output 1 123 456 123 1 123 4567890123456789 123 1 4567890123456789 -4567890123456789 4567890123456789 @@ -3246,163 +3218,163 @@ struct 1 4567890123456789 4567890123456789 4567890123456789 --- !query 167 +-- !query 163 select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = f1 --- !query 167 schema +-- !query 163 schema struct<> --- !query 167 output +-- !query 163 output org.apache.spark.sql.AnalysisException Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 63 --- !query 168 +-- !query 164 select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1 --- !query 168 schema +-- !query 164 schema struct<> --- !query 168 output +-- !query 164 output org.apache.spark.sql.AnalysisException cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63 --- !query 169 +-- !query 165 select * from int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1 --- !query 169 schema +-- !query 165 schema struct --- !query 169 output +-- !query 165 output --- !query 170 +-- !query 166 select t1.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 170 schema +-- !query 166 schema struct<> --- !query 170 output +-- !query 166 output org.apache.spark.sql.AnalysisException cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 171 +-- !query 167 select t2.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 171 schema +-- !query 167 schema struct<> --- !query 171 output +-- !query 167 output org.apache.spark.sql.AnalysisException cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 172 +-- !query 168 select uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 172 schema +-- !query 168 schema struct<> --- !query 172 output +-- !query 168 output org.apache.spark.sql.AnalysisException cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 173 +-- !query 169 select f1,g from int4_tbl a, (select f1 as g) ss --- !query 173 schema +-- !query 169 schema struct<> --- !query 173 output +-- !query 169 output org.apache.spark.sql.AnalysisException cannot resolve '`f1`' given input columns: []; line 1 pos 37 --- !query 174 +-- !query 170 select f1,g from int4_tbl a, (select a.f1 as g) ss --- !query 174 schema +-- !query 170 schema struct<> --- !query 174 output +-- !query 170 output org.apache.spark.sql.AnalysisException cannot resolve '`a.f1`' given input columns: []; line 1 pos 37 --- !query 175 +-- !query 171 select f1,g from int4_tbl a cross join (select f1 as g) ss --- !query 175 schema +-- !query 171 schema struct<> --- !query 175 output +-- !query 171 output org.apache.spark.sql.AnalysisException cannot resolve '`f1`' given input columns: []; line 1 pos 47 --- !query 176 +-- !query 172 select f1,g from int4_tbl a cross join (select a.f1 as g) ss --- !query 176 schema +-- !query 172 schema struct<> --- !query 176 output +-- !query 172 output org.apache.spark.sql.AnalysisException cannot resolve '`a.f1`' given input columns: []; line 1 pos 47 --- !query 177 +-- !query 173 CREATE TABLE j1 (id1 int, id2 int) USING parquet --- !query 177 schema +-- !query 173 schema struct<> --- !query 177 output +-- !query 173 output --- !query 178 +-- !query 174 CREATE TABLE j2 (id1 int, id2 int) USING parquet --- !query 178 schema +-- !query 174 schema struct<> --- !query 178 output +-- !query 174 output --- !query 179 +-- !query 175 INSERT INTO j1 values(1,1),(1,2) --- !query 179 schema +-- !query 175 schema struct<> --- !query 179 output +-- !query 175 output --- !query 180 +-- !query 176 INSERT INTO j2 values(1,1) --- !query 180 schema +-- !query 176 schema struct<> --- !query 180 output +-- !query 176 output --- !query 181 +-- !query 177 INSERT INTO j2 values(1,2) --- !query 181 schema +-- !query 177 schema struct<> --- !query 181 output +-- !query 177 output --- !query 182 +-- !query 178 select * from j1 inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1 --- !query 182 schema +-- !query 178 schema struct --- !query 182 output +-- !query 178 output 1 1 1 1 1 2 1 2 --- !query 183 +-- !query 179 drop table j1 --- !query 183 schema +-- !query 179 schema struct<> --- !query 183 output +-- !query 179 output --- !query 184 +-- !query 180 drop table j2 --- !query 184 schema +-- !query 180 schema struct<> --- !query 184 output +-- !query 180 output From f4d5aa42139ff8412c573c96a1631ef3ccf81844 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Sun, 20 Oct 2019 13:56:23 -0700 Subject: [PATCH 0375/1387] [SPARK-29434][CORE] Improve the MapStatuses Serialization Performance ### What changes were proposed in this pull request? Instead of using GZIP for compressing the serialized `MapStatuses`, ZStd provides better compression rate and faster compression time. The original approach is serializing and writing data directly into `GZIPOutputStream` as one step; however, the compression time is faster if a bigger chuck of the data is processed by the codec at once. As a result, in this PR, the serialized data is written into an uncompressed byte array first, and then the data is compressed. For smaller `MapStatues`, we find it's 2x faster. Here is the benchmark result. #### 20k map outputs, and each has 500 blocks 1. ZStd two steps in this PR: 0.402 ops/ms, 89,066 bytes 2. ZStd one step as the original approach: 0.370 ops/ms, 89,069 bytes 3. GZip: 0.092 ops/ms, 217,345 bytes #### 20k map outputs, and each has 5 blocks 1. ZStd two steps in this PR: 0.9 ops/ms, 75,449 bytes 2. ZStd one step as the original approach: 0.38 ops/ms, 75,452 bytes 3. GZip: 0.21 ops/ms, 160,094 bytes ### Why are the changes needed? Decrease the time for serializing the `MapStatuses` in large scale job. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26085 from dbtsai/mapStatus. Lead-authored-by: DB Tsai Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- ...tatusesSerDeserBenchmark-jdk11-results.txt | 42 +++++++------- .../MapStatusesSerDeserBenchmark-results.txt | 42 +++++++------- .../org/apache/spark/MapOutputTracker.scala | 58 ++++++++++++++----- 3 files changed, 87 insertions(+), 55 deletions(-) diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt index 747aae09272f9..7a6cfb7b23b94 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt @@ -2,21 +2,21 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 609 631 22 0.3 3043.8 1.0X -Deserialization 840 897 67 0.2 4201.2 0.7X +Serialization 205 213 13 1.0 1023.6 1.0X +Deserialization 908 939 27 0.2 4540.2 0.2X -Compressed Serialized MapStatus sizes: 393 bytes -Compressed Serialized Broadcast MapStatus sizes: 3 MB +Compressed Serialized MapStatus sizes: 400 bytes +Compressed Serialized Broadcast MapStatus sizes: 2 MB OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 591 599 8 0.3 2955.3 1.0X -Deserialization 878 913 31 0.2 4392.2 0.7X +Serialization 195 204 24 1.0 976.9 1.0X +Deserialization 913 940 33 0.2 4566.7 0.2X -Compressed Serialized MapStatus sizes: 3 MB +Compressed Serialized MapStatus sizes: 2 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes @@ -24,21 +24,21 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 1776 1778 2 0.1 8880.5 1.0X -Deserialization 1086 1086 0 0.2 5427.9 1.6X +Serialization 616 619 3 0.3 3079.1 1.0X +Deserialization 936 954 22 0.2 4680.5 0.7X -Compressed Serialized MapStatus sizes: 411 bytes -Compressed Serialized Broadcast MapStatus sizes: 15 MB +Compressed Serialized MapStatus sizes: 418 bytes +Compressed Serialized Broadcast MapStatus sizes: 14 MB OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 1725 1726 1 0.1 8624.9 1.0X -Deserialization 1093 1094 2 0.2 5463.6 1.6X +Serialization 586 588 3 0.3 2928.8 1.0X +Deserialization 929 933 4 0.2 4647.0 0.6X -Compressed Serialized MapStatus sizes: 15 MB +Compressed Serialized MapStatus sizes: 14 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes @@ -46,21 +46,21 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 12421 12522 142 0.0 62104.4 1.0X -Deserialization 3020 3043 32 0.1 15102.0 4.1X +Serialization 4740 4916 249 0.0 23698.5 1.0X +Deserialization 1578 1597 27 0.1 7890.6 3.0X -Compressed Serialized MapStatus sizes: 544 bytes -Compressed Serialized Broadcast MapStatus sizes: 131 MB +Compressed Serialized MapStatus sizes: 546 bytes +Compressed Serialized Broadcast MapStatus sizes: 123 MB OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 11719 11737 26 0.0 58595.3 1.0X -Deserialization 3018 3051 46 0.1 15091.7 3.9X +Serialization 4492 4573 115 0.0 22458.3 1.0X +Deserialization 1533 1547 20 0.1 7664.8 2.9X -Compressed Serialized MapStatus sizes: 131 MB +Compressed Serialized MapStatus sizes: 123 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt index 1f479a49d5860..0c649694f6b6e 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt @@ -2,21 +2,21 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 625 639 9 0.3 3127.2 1.0X -Deserialization 875 931 49 0.2 4376.2 0.7X +Serialization 236 245 18 0.8 1179.1 1.0X +Deserialization 842 885 37 0.2 4211.4 0.3X -Compressed Serialized MapStatus sizes: 393 bytes -Compressed Serialized Broadcast MapStatus sizes: 3 MB +Compressed Serialized MapStatus sizes: 400 bytes +Compressed Serialized Broadcast MapStatus sizes: 2 MB OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 604 640 71 0.3 3018.4 1.0X -Deserialization 889 903 17 0.2 4443.8 0.7X +Serialization 213 219 8 0.9 1065.1 1.0X +Deserialization 846 870 33 0.2 4228.6 0.3X -Compressed Serialized MapStatus sizes: 3 MB +Compressed Serialized MapStatus sizes: 2 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes @@ -24,21 +24,21 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 1879 1880 2 0.1 9394.9 1.0X -Deserialization 1147 1150 5 0.2 5733.8 1.6X +Serialization 624 709 167 0.3 3121.1 1.0X +Deserialization 885 908 22 0.2 4427.0 0.7X -Compressed Serialized MapStatus sizes: 411 bytes -Compressed Serialized Broadcast MapStatus sizes: 15 MB +Compressed Serialized MapStatus sizes: 418 bytes +Compressed Serialized Broadcast MapStatus sizes: 14 MB OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 1825 1826 1 0.1 9123.3 1.0X -Deserialization 1147 1281 189 0.2 5735.7 1.6X +Serialization 603 604 2 0.3 3014.9 1.0X +Deserialization 892 895 5 0.2 4458.7 0.7X -Compressed Serialized MapStatus sizes: 15 MB +Compressed Serialized MapStatus sizes: 14 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes @@ -46,21 +46,21 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 12327 12518 270 0.0 61634.3 1.0X -Deserialization 3120 3133 18 0.1 15600.8 4.0X +Serialization 4612 4945 471 0.0 23061.0 1.0X +Deserialization 1493 1495 2 0.1 7466.3 3.1X -Compressed Serialized MapStatus sizes: 544 bytes -Compressed Serialized Broadcast MapStatus sizes: 131 MB +Compressed Serialized MapStatus sizes: 546 bytes +Compressed Serialized Broadcast MapStatus sizes: 123 MB OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 11928 11986 82 0.0 59642.2 1.0X -Deserialization 3137 3138 2 0.1 15683.3 3.8X +Serialization 4452 4595 202 0.0 22261.4 1.0X +Deserialization 1464 1477 18 0.1 7321.4 3.0X -Compressed Serialized MapStatus sizes: 131 MB +Compressed Serialized MapStatus sizes: 123 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index c181fac8b4d8e..6f4a6239a09ed 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -17,10 +17,9 @@ package org.apache.spark -import java.io._ +import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} import java.util.concurrent.locks.ReentrantReadWriteLock -import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer, Map} @@ -29,6 +28,10 @@ import scala.concurrent.duration.Duration import scala.reflect.ClassTag import scala.util.control.NonFatal +import com.github.luben.zstd.ZstdInputStream +import com.github.luben.zstd.ZstdOutputStream +import org.apache.commons.io.output.{ByteArrayOutputStream => ApacheByteArrayOutputStream} + import org.apache.spark.broadcast.{Broadcast, BroadcastManager} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -885,13 +888,18 @@ private[spark] object MapOutputTracker extends Logging { private val BROADCAST = 1 // Serialize an array of map output locations into an efficient byte format so that we can send - // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will + // it to reduce tasks. We do this by compressing the serialized bytes using Zstd. They will // generally be pretty compressible because many map outputs will be on the same hostname. def serializeMapStatuses(statuses: Array[MapStatus], broadcastManager: BroadcastManager, isLocal: Boolean, minBroadcastSize: Int): (Array[Byte], Broadcast[Array[Byte]]) = { - val out = new ByteArrayOutputStream - out.write(DIRECT) - val objOut = new ObjectOutputStream(new GZIPOutputStream(out)) + // Using `org.apache.commons.io.output.ByteArrayOutputStream` instead of the standard one + // This implementation doesn't reallocate the whole memory block but allocates + // additional buffers. This way no buffers need to be garbage collected and + // the contents don't have to be copied to the new buffer. + val out = new ApacheByteArrayOutputStream() + val compressedOut = new ApacheByteArrayOutputStream() + + val objOut = new ObjectOutputStream(out) Utils.tryWithSafeFinally { // Since statuses can be modified in parallel, sync on it statuses.synchronized { @@ -900,18 +908,42 @@ private[spark] object MapOutputTracker extends Logging { } { objOut.close() } - val arr = out.toByteArray + + val arr: Array[Byte] = { + val zos = new ZstdOutputStream(compressedOut) + Utils.tryWithSafeFinally { + compressedOut.write(DIRECT) + // `out.writeTo(zos)` will write the uncompressed data from `out` to `zos` + // without copying to avoid unnecessary allocation and copy of byte[]. + out.writeTo(zos) + } { + zos.close() + } + compressedOut.toByteArray + } if (arr.length >= minBroadcastSize) { // Use broadcast instead. // Important arr(0) is the tag == DIRECT, ignore that while deserializing ! val bcast = broadcastManager.newBroadcast(arr, isLocal) // toByteArray creates copy, so we can reuse out out.reset() - out.write(BROADCAST) - val oos = new ObjectOutputStream(new GZIPOutputStream(out)) - oos.writeObject(bcast) - oos.close() - val outArr = out.toByteArray + val oos = new ObjectOutputStream(out) + Utils.tryWithSafeFinally { + oos.writeObject(bcast) + } { + oos.close() + } + val outArr = { + compressedOut.reset() + val zos = new ZstdOutputStream(compressedOut) + Utils.tryWithSafeFinally { + compressedOut.write(BROADCAST) + out.writeTo(zos) + } { + zos.close() + } + compressedOut.toByteArray + } logInfo("Broadcast mapstatuses size = " + outArr.length + ", actual size = " + arr.length) (outArr, bcast) } else { @@ -924,7 +956,7 @@ private[spark] object MapOutputTracker extends Logging { assert (bytes.length > 0) def deserializeObject(arr: Array[Byte], off: Int, len: Int): AnyRef = { - val objIn = new ObjectInputStream(new GZIPInputStream( + val objIn = new ObjectInputStream(new ZstdInputStream( new ByteArrayInputStream(arr, off, len))) Utils.tryWithSafeFinally { objIn.readObject() From 5b4d9170ed4226c53063a28ba5d124120549cb03 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 21 Oct 2019 14:32:31 +0800 Subject: [PATCH 0376/1387] [SPARK-27879][SQL] Add support for bit_and and bit_or aggregates ### What changes were proposed in this pull request? ``` bit_and(expression) -- The bitwise AND of all non-null input values, or null if none bit_or(expression) -- The bitwise OR of all non-null input values, or null if none ``` More details: https://www.postgresql.org/docs/9.3/functions-aggregate.html ### Why are the changes needed? Postgres, Mysql and many other popular db support them. ### Does this PR introduce any user-facing change? add two bit agg ### How was this patch tested? add ut Closes #26155 from yaooqinn/SPARK-27879. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../aggregate/bitwiseAggregates.scala | 99 +++++++++ .../inputs/postgreSQL/aggregates_part2.sql | 63 +++--- .../udf/postgreSQL/udf-aggregates_part2.sql | 54 ++--- .../postgreSQL/aggregates_part2.sql.out | 203 ++++++++++++------ .../postgreSQL/udf-aggregates_part2.sql.out | 164 ++++++++------ 6 files changed, 397 insertions(+), 188 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index addeca507c0f7..04e8963944fda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -525,6 +525,8 @@ object FunctionRegistry { expression[BitwiseOr]("|"), expression[BitwiseXor]("^"), expression[BitwiseCount]("bit_count"), + expression[BitAndAgg]("bit_and"), + expression[BitOrAgg]("bit_or"), // json expression[StructsToJson]("to_json"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala new file mode 100644 index 0000000000000..131fa2eb50555 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BitwiseAnd, BitwiseOr, ExpectsInputTypes, Expression, ExpressionDescription, If, IsNull, Literal} +import org.apache.spark.sql.types.{AbstractDataType, DataType, IntegralType} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the bitwise AND of all non-null input values, or null if none.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col); + 1 + """, + since = "3.0.0") +case class BitAndAgg(child: Expression) extends DeclarativeAggregate with ExpectsInputTypes { + + override def nodeName: String = "bit_and" + + override def children: Seq[Expression] = child :: Nil + + override def nullable: Boolean = true + + override def dataType: DataType = child.dataType + + override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) + + private lazy val bitAnd = AttributeReference("bit_and", child.dataType)() + + override lazy val aggBufferAttributes: Seq[AttributeReference] = bitAnd :: Nil + + override lazy val initialValues: Seq[Literal] = Literal.create(null, dataType) :: Nil + + override lazy val updateExpressions: Seq[Expression] = + If(IsNull(bitAnd), + child, + If(IsNull(child), bitAnd, BitwiseAnd(bitAnd, child))) :: Nil + + override lazy val mergeExpressions: Seq[Expression] = + If(IsNull(bitAnd.left), + bitAnd.right, + If(IsNull(bitAnd.right), bitAnd.left, BitwiseAnd(bitAnd.left, bitAnd.right))) :: Nil + + override lazy val evaluateExpression: AttributeReference = bitAnd +} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the bitwise OR of all non-null input values, or null if none.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col); + 7 + """, + since = "3.0.0") +case class BitOrAgg(child: Expression) extends DeclarativeAggregate with ExpectsInputTypes { + + override def nodeName: String = "bit_or" + + override def children: Seq[Expression] = child :: Nil + + override def nullable: Boolean = true + + override def dataType: DataType = child.dataType + + override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) + + private lazy val bitOr = AttributeReference("bit_or", child.dataType)() + + override lazy val aggBufferAttributes: Seq[AttributeReference] = bitOr :: Nil + + override lazy val initialValues: Seq[Literal] = Literal.create(null, dataType) :: Nil + + override lazy val updateExpressions: Seq[Expression] = + If(IsNull(bitOr), + child, + If(IsNull(child), bitOr, BitwiseOr(bitOr, child))) :: Nil + + override lazy val mergeExpressions: Seq[Expression] = + If(IsNull(bitOr.left), + bitOr.right, + If(IsNull(bitOr.right), bitOr.left, BitwiseOr(bitOr.left, bitOr.right))) :: Nil + + override lazy val evaluateExpression: AttributeReference = bitOr +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql index 144a3bc2a38fc..ba91366014e16 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql @@ -41,42 +41,37 @@ create temporary view int4_tbl as select * from values -- -- test for bitwise integer aggregates -- --- CREATE TEMPORARY TABLE bitwise_test( --- i2 INT2, --- i4 INT4, --- i8 INT8, --- i INTEGER, --- x INT2, --- y BIT(4) --- ); +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); -- empty case --- SELECT --- BIT_AND(i2) AS "?", --- BIT_OR(i4) AS "?" --- FROM bitwise_test; - --- COPY bitwise_test FROM STDIN NULL 'null'; --- 1 1 1 1 1 B0101 --- 3 3 3 null 2 B0100 --- 7 7 7 3 4 B1100 --- \. - --- SELECT --- BIT_AND(i2) AS "1", --- BIT_AND(i4) AS "1", --- BIT_AND(i8) AS "1", --- BIT_AND(i) AS "?", --- BIT_AND(x) AS "0", --- BIT_AND(y) AS "0100", --- --- BIT_OR(i2) AS "7", --- BIT_OR(i4) AS "7", --- BIT_OR(i8) AS "7", --- BIT_OR(i) AS "?", --- BIT_OR(x) AS "7", --- BIT_OR(y) AS "1101" --- FROM bitwise_test; +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null; + +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test; + +-- group by +SELECT b1 , bit_and(b2), bit_or(b4) FROM bitwise_test GROUP BY b1; + +--having +SELECT b1, bit_and(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7; + +-- window +SELECT b1, b2, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; +SELECT b1, b2, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; -- -- test boolean aggregates diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql index d82fcd90a22af..b4054850062b7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql @@ -43,42 +43,28 @@ create temporary view int4_tbl as select * from values -- -- test for bitwise integer aggregates -- --- CREATE TEMPORARY TABLE bitwise_test( --- i2 INT2, --- i4 INT4, --- i8 INT8, --- i INTEGER, --- x INT2, --- y BIT(4) --- ); +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); -- empty case --- SELECT --- BIT_AND(i2) AS "?", --- BIT_OR(i4) AS "?" --- FROM bitwise_test; - --- COPY bitwise_test FROM STDIN NULL 'null'; --- 1 1 1 1 1 B0101 --- 3 3 3 null 2 B0100 --- 7 7 7 3 4 B1100 --- \. - --- SELECT --- BIT_AND(i2) AS "1", --- BIT_AND(i4) AS "1", --- BIT_AND(i8) AS "1", --- BIT_AND(i) AS "?", --- BIT_AND(x) AS "0", --- BIT_AND(y) AS "0100", --- --- BIT_OR(i2) AS "7", --- BIT_OR(i4) AS "7", --- BIT_OR(i8) AS "7", --- BIT_OR(i) AS "?", --- BIT_OR(x) AS "7", --- BIT_OR(y) AS "1101" --- FROM bitwise_test; +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null; + + +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test; -- -- test boolean aggregates diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out index d5362809c8049..fdca51ff1325b 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 21 +-- Number of queries: 29 -- !query 0 @@ -17,6 +17,89 @@ struct<> -- !query 1 +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 +-- !query 2 schema +struct +-- !query 2 output +NULL NULL + + +-- !query 3 +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null +-- !query 3 schema +struct +-- !query 3 output +NULL NULL + + +-- !query 4 +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test +-- !query 4 schema +struct +-- !query 4 output +1 1 1 1 7 7 7 3 + + +-- !query 5 +SELECT b1 , bit_and(b2), bit_or(b4) FROM bitwise_test GROUP BY b1 +-- !query 5 schema +struct +-- !query 5 output +1 1 1 +3 3 NULL +7 7 3 + + +-- !query 6 +SELECT b1, bit_and(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 +-- !query 6 schema +struct +-- !query 6 output +1 1 +3 3 + + +-- !query 7 +SELECT b1, b2, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query 7 schema +struct +-- !query 7 output +1 1 1 +3 3 3 +7 7 7 + + +-- !query 8 +SELECT b1, b2, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query 8 schema +struct +-- !query 8 output +1 1 1 +3 3 3 +7 7 7 + + +-- !query 9 SELECT (NULL AND NULL) IS NULL AS `t`, (TRUE AND NULL) IS NULL AS `t`, @@ -27,13 +110,13 @@ SELECT NOT (TRUE AND FALSE) AS `t`, NOT (FALSE AND TRUE) AS `t`, NOT (FALSE AND FALSE) AS `t` --- !query 1 schema +-- !query 9 schema struct --- !query 1 output +-- !query 9 output true true false true false true true true true --- !query 2 +-- !query 10 SELECT (NULL OR NULL) IS NULL AS `t`, (TRUE OR NULL) IS NULL AS `t`, @@ -44,32 +127,32 @@ SELECT (TRUE OR FALSE) AS `t`, (FALSE OR TRUE) AS `t`, NOT (FALSE OR FALSE) AS `t` --- !query 2 schema +-- !query 10 schema struct --- !query 2 output +-- !query 10 output true false true false true true true true true --- !query 3 +-- !query 11 CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES (TRUE, null, FALSE, null), (FALSE, TRUE, null, null), (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) --- !query 3 schema +-- !query 11 schema struct<> --- !query 3 output +-- !query 11 output --- !query 4 +-- !query 12 SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 --- !query 4 schema +-- !query 12 schema struct --- !query 4 output +-- !query 12 output NULL NULL --- !query 5 +-- !query 13 SELECT BOOL_AND(b1) AS f1, BOOL_AND(b2) AS t2, @@ -78,13 +161,13 @@ SELECT BOOL_AND(NOT b2) AS f5, BOOL_AND(NOT b3) AS t6 FROM bool_test --- !query 5 schema +-- !query 13 schema struct --- !query 5 output +-- !query 13 output false true false NULL false true --- !query 6 +-- !query 14 SELECT EVERY(b1) AS f1, EVERY(b2) AS t2, @@ -93,13 +176,13 @@ SELECT EVERY(NOT b2) AS f5, EVERY(NOT b3) AS t6 FROM bool_test --- !query 6 schema +-- !query 14 schema struct --- !query 6 output +-- !query 14 output false true false NULL false true --- !query 7 +-- !query 15 SELECT BOOL_OR(b1) AS t1, BOOL_OR(b2) AS t2, @@ -108,113 +191,113 @@ SELECT BOOL_OR(NOT b2) AS f5, BOOL_OR(NOT b3) AS t6 FROM bool_test --- !query 7 schema +-- !query 15 schema struct --- !query 7 output +-- !query 15 output true true false NULL false true --- !query 8 +-- !query 16 select min(unique1) from tenk1 --- !query 8 schema +-- !query 16 schema struct --- !query 8 output +-- !query 16 output 0 --- !query 9 +-- !query 17 select max(unique1) from tenk1 --- !query 9 schema +-- !query 17 schema struct --- !query 9 output +-- !query 17 output 9999 --- !query 10 +-- !query 18 select max(unique1) from tenk1 where unique1 < 42 --- !query 10 schema +-- !query 18 schema struct --- !query 10 output +-- !query 18 output 41 --- !query 11 +-- !query 19 select max(unique1) from tenk1 where unique1 > 42 --- !query 11 schema +-- !query 19 schema struct --- !query 11 output +-- !query 19 output 9999 --- !query 12 +-- !query 20 select max(unique1) from tenk1 where unique1 > 42000 --- !query 12 schema +-- !query 20 schema struct --- !query 12 output +-- !query 20 output NULL --- !query 13 +-- !query 21 select max(tenthous) from tenk1 where thousand = 33 --- !query 13 schema +-- !query 21 schema struct --- !query 13 output +-- !query 21 output 9033 --- !query 14 +-- !query 22 select min(tenthous) from tenk1 where thousand = 33 --- !query 14 schema +-- !query 22 schema struct --- !query 14 output +-- !query 22 output 33 --- !query 15 +-- !query 23 select distinct max(unique2) from tenk1 --- !query 15 schema +-- !query 23 schema struct --- !query 15 output +-- !query 23 output 9999 --- !query 16 +-- !query 24 select max(unique2) from tenk1 order by 1 --- !query 16 schema +-- !query 24 schema struct --- !query 16 output +-- !query 24 output 9999 --- !query 17 +-- !query 25 select max(unique2) from tenk1 order by max(unique2) --- !query 17 schema +-- !query 25 schema struct --- !query 17 output +-- !query 25 output 9999 --- !query 18 +-- !query 26 select max(unique2) from tenk1 order by max(unique2)+1 --- !query 18 schema +-- !query 26 schema struct --- !query 18 output +-- !query 26 output 9999 --- !query 19 +-- !query 27 select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc --- !query 19 schema +-- !query 27 schema struct --- !query 19 output +-- !query 27 output 9999 3 9999 2 9999 1 --- !query 20 +-- !query 28 select max(100) from tenk1 --- !query 20 schema +-- !query 28 schema struct --- !query 20 output +-- !query 28 output 100 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out index 9efc7f65fdfc2..3b54637c449d8 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 21 +-- Number of queries: 25 -- !query 0 @@ -17,6 +17,50 @@ struct<> -- !query 1 +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 +-- !query 2 schema +struct +-- !query 2 output +NULL NULL + + +-- !query 3 +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null +-- !query 3 schema +struct +-- !query 3 output +NULL NULL + + +-- !query 4 +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test +-- !query 4 schema +struct +-- !query 4 output +1 1 1 1 7 7 7 3 + + +-- !query 5 SELECT (NULL AND NULL) IS NULL AS `t`, (TRUE AND NULL) IS NULL AS `t`, @@ -27,13 +71,13 @@ SELECT NOT (TRUE AND FALSE) AS `t`, NOT (FALSE AND TRUE) AS `t`, NOT (FALSE AND FALSE) AS `t` --- !query 1 schema +-- !query 5 schema struct --- !query 1 output +-- !query 5 output true true false true false true true true true --- !query 2 +-- !query 6 SELECT (NULL OR NULL) IS NULL AS `t`, (TRUE OR NULL) IS NULL AS `t`, @@ -44,32 +88,32 @@ SELECT (TRUE OR FALSE) AS `t`, (FALSE OR TRUE) AS `t`, NOT (FALSE OR FALSE) AS `t` --- !query 2 schema +-- !query 6 schema struct --- !query 2 output +-- !query 6 output true false true false true true true true true --- !query 3 +-- !query 7 CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES (TRUE, null, FALSE, null), (FALSE, TRUE, null, null), (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) --- !query 3 schema +-- !query 7 schema struct<> --- !query 3 output +-- !query 7 output --- !query 4 +-- !query 8 SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 --- !query 4 schema +-- !query 8 schema struct --- !query 4 output +-- !query 8 output NULL NULL --- !query 5 +-- !query 9 SELECT BOOL_AND(b1) AS f1, BOOL_AND(b2) AS t2, @@ -78,13 +122,13 @@ SELECT BOOL_AND(NOT b2) AS f5, BOOL_AND(NOT b3) AS t6 FROM bool_test --- !query 5 schema +-- !query 9 schema struct --- !query 5 output +-- !query 9 output false true false NULL false true --- !query 6 +-- !query 10 SELECT EVERY(b1) AS f1, EVERY(b2) AS t2, @@ -93,13 +137,13 @@ SELECT EVERY(NOT b2) AS f5, EVERY(NOT b3) AS t6 FROM bool_test --- !query 6 schema +-- !query 10 schema struct --- !query 6 output +-- !query 10 output false true false NULL false true --- !query 7 +-- !query 11 SELECT BOOL_OR(b1) AS t1, BOOL_OR(b2) AS t2, @@ -108,113 +152,113 @@ SELECT BOOL_OR(NOT b2) AS f5, BOOL_OR(NOT b3) AS t6 FROM bool_test --- !query 7 schema +-- !query 11 schema struct --- !query 7 output +-- !query 11 output true true false NULL false true --- !query 8 +-- !query 12 select min(udf(unique1)) from tenk1 --- !query 8 schema +-- !query 12 schema struct --- !query 8 output +-- !query 12 output 0 --- !query 9 +-- !query 13 select udf(max(unique1)) from tenk1 --- !query 9 schema +-- !query 13 schema struct --- !query 9 output +-- !query 13 output 9999 --- !query 10 +-- !query 14 select max(unique1) from tenk1 where udf(unique1) < 42 --- !query 10 schema +-- !query 14 schema struct --- !query 10 output +-- !query 14 output 41 --- !query 11 +-- !query 15 select max(unique1) from tenk1 where unique1 > udf(42) --- !query 11 schema +-- !query 15 schema struct --- !query 11 output +-- !query 15 output 9999 --- !query 12 +-- !query 16 select max(unique1) from tenk1 where udf(unique1) > 42000 --- !query 12 schema +-- !query 16 schema struct --- !query 12 output +-- !query 16 output NULL --- !query 13 +-- !query 17 select max(tenthous) from tenk1 where udf(thousand) = 33 --- !query 13 schema +-- !query 17 schema struct --- !query 13 output +-- !query 17 output 9033 --- !query 14 +-- !query 18 select min(tenthous) from tenk1 where udf(thousand) = 33 --- !query 14 schema +-- !query 18 schema struct --- !query 14 output +-- !query 18 output 33 --- !query 15 +-- !query 19 select distinct max(udf(unique2)) from tenk1 --- !query 15 schema +-- !query 19 schema struct --- !query 15 output +-- !query 19 output 9999 --- !query 16 +-- !query 20 select max(unique2) from tenk1 order by udf(1) --- !query 16 schema +-- !query 20 schema struct --- !query 16 output +-- !query 20 output 9999 --- !query 17 +-- !query 21 select max(unique2) from tenk1 order by max(udf(unique2)) --- !query 17 schema +-- !query 21 schema struct --- !query 17 output +-- !query 21 output 9999 --- !query 18 +-- !query 22 select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 --- !query 18 schema +-- !query 22 schema struct --- !query 18 output +-- !query 22 output 9999 --- !query 19 +-- !query 23 select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc --- !query 19 schema +-- !query 23 schema struct --- !query 19 output +-- !query 23 output 9999 3 9999 2 9999 1 --- !query 20 +-- !query 24 select udf(max(100)) from tenk1 --- !query 20 schema +-- !query 24 schema struct --- !query 20 output +-- !query 24 output 100 From e99a9f78ea2e18eabca722c5714280eb7a737499 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 21 Oct 2019 15:53:36 +0800 Subject: [PATCH 0377/1387] [SPARK-29498][SQL] CatalogTable to HiveTable should not change the table's ownership ### What changes were proposed in this pull request? `CatalogTable` to `HiveTable` will change the table's ownership. How to reproduce: ```scala import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.types.{LongType, StructType} val identifier = TableIdentifier("spark_29498", None) val owner = "SPARK-29498" val newTable = CatalogTable( identifier, tableType = CatalogTableType.EXTERNAL, storage = CatalogStorageFormat( locationUri = None, inputFormat = None, outputFormat = None, serde = None, compressed = false, properties = Map.empty), owner = owner, schema = new StructType().add("i", LongType, false), provider = Some("hive")) spark.sessionState.catalog.createTable(newTable, false) // The owner is not SPARK-29498 println(spark.sessionState.catalog.getTableMetadata(identifier).owner) ``` This PR makes it set the `HiveTable`'s owner to `CatalogTable`'s owner if it's owner is not empty when converting `CatalogTable` to `HiveTable`. ### Why are the changes needed? We should not change the ownership of the table when converting `CatalogTable` to `HiveTable`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? unit test Closes #26160 from wangyum/SPARK-29498. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan --- .../spark/sql/hive/client/HiveClientImpl.scala | 5 ++--- .../spark/sql/hive/HiveExternalCatalogSuite.scala | 15 +++++++++++++++ 2 files changed, 17 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 783bc5b562ff4..12c9a972c1aff 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 @@ -573,9 +573,8 @@ private[hive] class HiveClientImpl( // If users explicitly alter these Hive-specific properties through ALTER TABLE DDL, we respect // these user-specified values. verifyColumnDataType(table.dataSchema) - val owner = Option(table.owner).filter(_.nonEmpty).getOrElse(userName) val hiveTable = toHiveTable( - table.copy(properties = table.ignoredProperties ++ table.properties), Some(owner)) + table.copy(properties = table.ignoredProperties ++ table.properties), Some(userName)) // Do not use `table.qualifiedName` here because this may be a rename val qualifiedTableName = s"$dbName.$tableName" shim.alterTable(client, qualifiedTableName, hiveTable) @@ -1039,7 +1038,7 @@ private[hive] object HiveClientImpl { } hiveTable.setFields(schema.asJava) hiveTable.setPartCols(partCols.asJava) - userName.foreach(hiveTable.setOwner) + Option(table.owner).filter(_.nonEmpty).orElse(userName).foreach(hiveTable.setOwner) hiveTable.setCreateTime(MILLISECONDS.toSeconds(table.createTime).toInt) hiveTable.setLastAccessTime(MILLISECONDS.toSeconds(table.lastAccessTime).toInt) table.storage.locationUri.map(CatalogUtils.URIToString).foreach { loc => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 0a522b6a11c80..46623000405cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -113,4 +113,19 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { catalog.createDatabase(newDb("dbWithNullDesc").copy(description = null), ignoreIfExists = false) assert(catalog.getDatabase("dbWithNullDesc").description == "") } + + test("SPARK-29498 CatalogTable to HiveTable should not change the table's ownership") { + val catalog = newBasicCatalog() + val owner = "SPARK-29498" + val hiveTable = CatalogTable( + identifier = TableIdentifier("spark_29498", Some("db1")), + tableType = CatalogTableType.MANAGED, + storage = storageFormat, + owner = owner, + schema = new StructType().add("i", "int"), + provider = Some("hive")) + + catalog.createTable(hiveTable, ignoreIfExists = false) + assert(catalog.getTable("db1", "spark_29498").owner === owner) + } } From 5fc363b307e4a032df87182d477ff63473d6d02d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 21 Oct 2019 19:05:27 +0900 Subject: [PATCH 0378/1387] [SPARK-29528][BUILD][TEST-MAVEN] Upgrade scala-maven-plugin to 4.2.4 for Scala 2.13.1 ### What changes were proposed in this pull request? This PR upgrades `scala-maven-plugin` to `4.2.4` for Scala `2.13.1`. ### Why are the changes needed? Scala 2.13.1 seems to break the binary compatibility. We need to upgrade `scala-maven-plugin` to bring the the following fixes for the latest Scala 2.13.1. - https://github.com/davidB/scala-maven-plugin/issues/363 - https://github.com/sbt/zinc/issues/698 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? For now, we don't support Scala-2.13. This PR at least needs to pass the existing Jenkins with Maven to get prepared for Scala-2.13. Closes #26185 from dongjoon-hyun/SPARK-29528. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 20a63d41f5bf5..69b5b79b7b071 100644 --- a/pom.xml +++ b/pom.xml @@ -2241,7 +2241,7 @@ net.alchim31.maven scala-maven-plugin - 4.2.0 + 4.2.4 eclipse-add-source From 31a5dea48fd1269e70b9b3639fd746434400cfdf Mon Sep 17 00:00:00 2001 From: fuwhu Date: Mon, 21 Oct 2019 05:19:27 -0700 Subject: [PATCH 0379/1387] [SPARK-29531][SQL][TEST] refine ThriftServerQueryTestSuite.blackList to reuse black list in SQLQueryTestSuite ### What changes were proposed in this pull request? This pr refine the code in ThriftServerQueryTestSuite.blackList to reuse the black list of SQLQueryTestSuite instead of duplicating all test cases from SQLQueryTestSuite.blackList. ### Why are the changes needed? To reduce code duplication. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #26188 from fuwhu/SPARK-TBD. Authored-by: fuwhu Signed-off-by: Yuming Wang --- .../thriftserver/ThriftServerQueryTestSuite.scala | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 799f00a28fd49..abe91a280a14f 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -83,8 +83,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { override val isTestWithConfigSets = false /** List of test cases to ignore, in lower cases. */ - override def blackList: Set[String] = Set( - "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. + override def blackList: Set[String] = super.blackList ++ Set( // Missing UDF "postgreSQL/boolean.sql", "postgreSQL/case.sql", @@ -92,17 +91,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { "date.sql", // SPARK-28620 "postgreSQL/float4.sql", - // SPARK-28885 String value is not allowed to be stored as numeric type with - // ANSI store assignment policy. - "postgreSQL/numeric.sql", - "postgreSQL/int2.sql", - "postgreSQL/int4.sql", - "postgreSQL/int8.sql", - "postgreSQL/float8.sql", - // SPARK-28885 String value is not allowed to be stored as date/timestamp type with - // ANSI store assignment policy. - "postgreSQL/date.sql", - "postgreSQL/timestamp.sql", // SPARK-28636 "decimalArithmeticOperations.sql", "literals.sql", From 6ffec5e6a63555509fb66cfe4b6f9d19bcd1e27c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 22 Oct 2019 10:47:04 +0900 Subject: [PATCH 0380/1387] [SPARK-29533][SQL][TEST] Benchmark casting strings to intervals ### What changes were proposed in this pull request? Added new benchmark `IntervalBenchmark` to measure performance of interval related functions. In the PR, I added benchmarks for casting strings to interval. In particular, interval strings with `interval` prefix and without it because there is special code for this https://github.com/apache/spark/blob/da576a737c2db01e5ba5ce19ed0e8f900cb5efaf/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java#L100-L103 . And also I added benchmarks for different number of units in interval strings, for example 1 unit is `interval 10 years`, 2 units w/o interval is `10 years 5 months`, and etc. ### Why are the changes needed? - To find out current performance issues in casting to intervals - The benchmark can be used while refactoring/re-implementing `CalendarInterval.fromString()` or `CalendarInterval.fromCaseInsensitiveString()`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the benchmark via the command: ```shell SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.IntervalBenchmark" ``` Closes #26189 from MaxGekk/interval-from-string-benchmark. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../IntervalBenchmark-jdk11-results.txt | 25 +++++ .../benchmarks/IntervalBenchmark-results.txt | 25 +++++ .../benchmark/IntervalBenchmark.scala | 102 ++++++++++++++++++ 3 files changed, 152 insertions(+) create mode 100644 sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt create mode 100644 sql/core/benchmarks/IntervalBenchmark-results.txt create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..2a3903200a8ac --- /dev/null +++ b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt @@ -0,0 +1,25 @@ +OpenJDK 64-Bit Server VM 11.0.2+9 on Mac OS X 10.15 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +string w/ interval 471 513 57 2.1 470.7 1.0X +string w/o interval 437 444 8 2.3 436.9 1.1X +1 units w/ interval 726 758 45 1.4 726.3 0.6X +1 units w/o interval 712 717 5 1.4 711.7 0.7X +2 units w/ interval 926 935 12 1.1 925.9 0.5X +2 units w/o interval 943 947 3 1.1 943.4 0.5X +3 units w/ interval 1089 1116 31 0.9 1089.0 0.4X +3 units w/o interval 1105 1108 3 0.9 1105.1 0.4X +4 units w/ interval 1260 1261 1 0.8 1260.4 0.4X +4 units w/o interval 1276 1277 1 0.8 1275.9 0.4X +5 units w/ interval 1436 1445 11 0.7 1435.6 0.3X +5 units w/o interval 1455 1463 6 0.7 1455.5 0.3X +6 units w/ interval 1634 1639 4 0.6 1634.4 0.3X +6 units w/o interval 1642 1644 3 0.6 1641.7 0.3X +7 units w/ interval 1829 1838 8 0.5 1828.6 0.3X +7 units w/o interval 1850 1853 4 0.5 1849.5 0.3X +8 units w/ interval 2065 2070 5 0.5 2065.4 0.2X +8 units w/o interval 2070 2090 21 0.5 2070.0 0.2X +9 units w/ interval 2279 2290 10 0.4 2278.7 0.2X +9 units w/o interval 2276 2285 8 0.4 2275.7 0.2X + diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt new file mode 100644 index 0000000000000..9010b980c07b5 --- /dev/null +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -0,0 +1,25 @@ +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.15 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +string w/ interval 420 435 18 2.4 419.8 1.0X +string w/o interval 359 365 10 2.8 358.7 1.2X +1 units w/ interval 752 759 8 1.3 752.0 0.6X +1 units w/o interval 762 766 4 1.3 762.0 0.6X +2 units w/ interval 961 970 8 1.0 960.7 0.4X +2 units w/o interval 970 976 9 1.0 970.2 0.4X +3 units w/ interval 1130 1136 7 0.9 1130.4 0.4X +3 units w/o interval 1150 1158 9 0.9 1150.3 0.4X +4 units w/ interval 1333 1336 3 0.7 1333.5 0.3X +4 units w/o interval 1354 1359 4 0.7 1354.5 0.3X +5 units w/ interval 1523 1525 2 0.7 1523.3 0.3X +5 units w/o interval 1549 1551 3 0.6 1549.4 0.3X +6 units w/ interval 1661 1663 2 0.6 1660.8 0.3X +6 units w/o interval 1691 1704 13 0.6 1691.2 0.2X +7 units w/ interval 1811 1817 8 0.6 1810.6 0.2X +7 units w/o interval 1853 1854 1 0.5 1853.2 0.2X +8 units w/ interval 2029 2037 8 0.5 2028.7 0.2X +8 units w/o interval 2075 2075 1 0.5 2074.5 0.2X +9 units w/ interval 2170 2175 5 0.5 2170.0 0.2X +9 units w/o interval 2204 2212 8 0.5 2203.6 0.2X + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala new file mode 100644 index 0000000000000..4c1c75b815a02 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala @@ -0,0 +1,102 @@ +/* + * 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.benchmark + +import scala.collection.mutable.ListBuffer + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.Column +import org.apache.spark.sql.SaveMode.Overwrite +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +/** + * Synthetic benchmark for interval functions. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "sql/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/IntervalBenchmark-results.txt". + * }}} + */ +object IntervalBenchmark extends SqlBasedBenchmark { + import spark.implicits._ + + private def doBenchmark(cardinality: Long, exprs: Column*): Unit = { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + spark + .range(0, cardinality, 1, 1) + .select(exprs: _*) + .write + .format("noop") + .mode(Overwrite) + .save() + } + } + + private def addCase( + benchmark: Benchmark, + cardinality: Long, + name: String, + exprs: Column*): Unit = { + benchmark.addCase(name, numIters = 3) { _ => + doBenchmark(cardinality, exprs: _*) + } + } + + private def buildString(withPrefix: Boolean, units: Seq[String] = Seq.empty): Column = { + val init = lit(if (withPrefix) "interval" else "") :: + ($"id" % 10000).cast("string") :: + lit("years") :: Nil + + concat_ws(" ", (init ++ units.map(lit)): _*) + } + + private def addCase(benchmark: Benchmark, cardinality: Long, units: Seq[String]): Unit = { + Seq(true, false).foreach { withPrefix => + val expr = buildString(withPrefix, units).cast("interval") + val note = if (withPrefix) "w/ interval" else "w/o interval" + benchmark.addCase(s"${units.length + 1} units $note", numIters = 3) { _ => + doBenchmark(cardinality, expr) + } + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val N = 1000000 + val timeUnits = Seq( + "13 months", "100 weeks", "9 days", "12 hours", + "5 minutes", "45 seconds", "123 milliseconds", "567 microseconds") + val intervalToTest = ListBuffer[String]() + + val benchmark = new Benchmark("cast strings to intervals", N, output = output) + addCase(benchmark, N, "string w/ interval", buildString(true, timeUnits)) + addCase(benchmark, N, "string w/o interval", buildString(false, timeUnits)) + addCase(benchmark, N, intervalToTest) // Only years + + for (unit <- timeUnits) { + intervalToTest.append(unit) + addCase(benchmark, N, intervalToTest) + } + + benchmark.run() + } +} From eef11ba9ef789c582f83c1796dbff0d234254fd6 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 22 Oct 2019 10:47:54 +0900 Subject: [PATCH 0381/1387] [SPARK-29518][SQL][TEST] Benchmark `date_part` for `INTERVAL` ### What changes were proposed in this pull request? I extended `ExtractBenchmark` to support the `INTERVAL` type of the `source` parameter of the `date_part` function. ### Why are the changes needed? - To detect performance issues while changing implementation of the `date_part` function in the future. - To find out current performance bottlenecks in `date_part` for the `INTERVAL` type ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the benchmark and print out produced values per each `field` value. Closes #26175 from MaxGekk/extract-interval-benchmark. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../benchmarks/ExtractBenchmark-results.txt | 19 ++++++++++ .../benchmark/ExtractBenchmark.scala | 38 +++++++++++++------ 2 files changed, 45 insertions(+), 12 deletions(-) diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 07c794fe0c75d..89d3616d2c205 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -98,3 +98,22 @@ MILLISECONDS of date 1744 1749 MICROSECONDS of date 1592 1594 1 6.3 159.2 0.6X EPOCH of date 2368 2371 3 4.2 236.8 0.4X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.15 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to interval 1365 1395 31 7.3 136.5 1.0X +MILLENNIUM of interval 1620 1651 27 6.2 162.0 0.8X +CENTURY of interval 1469 1487 22 6.8 146.9 0.9X +DECADE of interval 1462 1473 17 6.8 146.2 0.9X +YEAR of interval 1438 1447 8 7.0 143.8 0.9X +QUARTER of interval 1456 1458 3 6.9 145.6 0.9X +MONTH of interval 1440 1452 16 6.9 144.0 0.9X +DAY of interval 1478 1485 6 6.8 147.8 0.9X +HOUR of interval 1579 1580 3 6.3 157.9 0.9X +MINUTE of interval 1598 1605 11 6.3 159.8 0.9X +SECOND of interval 1571 1579 10 6.4 157.1 0.9X +MILLISECONDS of interval 1570 1577 6 6.4 157.0 0.9X +MICROSECONDS of interval 1484 1488 5 6.7 148.4 0.9X +EPOCH of interval 1521 1522 1 6.6 152.1 0.9X + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index 2bd73c1dc14f8..941649df6b727 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -61,8 +61,10 @@ object ExtractBenchmark extends SqlBasedBenchmark { } private def castExpr(from: String): String = from match { - case "timestamp" => s"cast(id as timestamp)" - case "date" => s"cast(cast(id as timestamp) as date)" + case "timestamp" => "cast(id as timestamp)" + case "date" => "cast(cast(id as timestamp) as date)" + case "interval" => "(cast(cast(id as timestamp) as date) - date'0001-01-01') + " + + "(cast(id as timestamp) - timestamp'1000-01-01 01:02:03.123456')" case other => throw new IllegalArgumentException( s"Unsupported column type $other. Valid column types are 'timestamp' and 'date'") } @@ -74,8 +76,8 @@ object ExtractBenchmark extends SqlBasedBenchmark { field: String, from: String): Unit = { val expr = func match { - case "extract" => s"EXTRACT($field FROM ${castExpr(from)})" - case "date_part" => s"DATE_PART('$field', ${castExpr(from)})" + case "extract" => s"EXTRACT($field FROM ${castExpr(from)}) AS $field" + case "date_part" => s"DATE_PART('$field', ${castExpr(from)}) AS $field" case other => throw new IllegalArgumentException( s"Unsupported function '$other'. Valid functions are 'extract' and 'date_part'.") } @@ -84,24 +86,36 @@ object ExtractBenchmark extends SqlBasedBenchmark { } } + private case class Settings(fields: Seq[String], func: Seq[String], iterNum: Long) + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val N = 10000000L - val fields = Seq( + val datetimeFields = Seq( "MILLENNIUM", "CENTURY", "DECADE", "YEAR", "ISOYEAR", "QUARTER", "MONTH", "WEEK", "DAY", "DAYOFWEEK", "DOW", "ISODOW", "DOY", "HOUR", "MINUTE", "SECOND", "MILLISECONDS", "MICROSECONDS", "EPOCH") + val intervalFields = Seq( + "MILLENNIUM", "CENTURY", "DECADE", "YEAR", + "QUARTER", "MONTH", "DAY", + "HOUR", "MINUTE", "SECOND", + "MILLISECONDS", "MICROSECONDS", "EPOCH") + val settings = Map( + "timestamp" -> Settings(datetimeFields, Seq("extract", "date_part"), N), + "date" -> Settings(datetimeFields, Seq("extract", "date_part"), N), + "interval" -> Settings(intervalFields, Seq("date_part"), N)) + + for { + (dataType, Settings(fields, funcs, iterNum)) <- settings + func <- funcs} { - Seq("extract", "date_part").foreach { func => - Seq("timestamp", "date").foreach { dateType => - val benchmark = new Benchmark(s"Invoke $func for $dateType", N, output = output) + val benchmark = new Benchmark(s"Invoke $func for $dataType", N, output = output) - run(benchmark, N, s"cast to $dateType", castExpr(dateType)) - fields.foreach(run(benchmark, func, N, _, dateType)) + run(benchmark, iterNum, s"cast to $dataType", castExpr(dataType)) + fields.foreach(run(benchmark, func, iterNum, _, dataType)) - benchmark.run() - } + benchmark.run() } } } From bb4400c23ad3b12b9da6c54361f9cf893d955d14 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Tue, 22 Oct 2019 10:49:40 +0900 Subject: [PATCH 0382/1387] [SPARK-29108][SQL][TESTS] Port window.sql (Part 2) ### What changes were proposed in this pull request? This PR ports window.sql from PostgreSQL regression tests https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql from lines 320~562 The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/expected/window.out ## How was this patch tested? Pass the Jenkins. ### Why are the changes needed? To ensure compatibility with PGSQL ### Does this PR introduce any user-facing change? No ### How was this patch tested? Comparison with PgSQL results. Closes #26121 from DylanGuedes/spark-29108. Authored-by: DylanGuedes Signed-off-by: HyukjinKwon --- .../inputs/postgreSQL/window_part2.sql | 285 +++++++++++ .../results/postgreSQL/window_part2.sql.out | 479 ++++++++++++++++++ 2 files changed, 764 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql new file mode 100644 index 0000000000000..728e8cab0c3ba --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql @@ -0,0 +1,285 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L320-562 + +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- CREATE TEMP VIEW v_window AS +-- SELECT i, min(i) over (order by i range between '1 day' preceding and '10 days' following) as min_i +-- FROM range(now(), now()+'100 days', '1 hour') i; + +-- RANGE offset PRECEDING/FOLLOWING tests + +SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (order by four desc range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude no others), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 6 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 6 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following +-- exclude current row),unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date desc range between '1 year' preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date desc range between '1 year' following and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude current row), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude group), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- select first_value(salary) over(order by salary range between 1000 preceding and 1000 following), +-- lead(salary) over(order by salary range between 1000 preceding and 1000 following), +-- nth_value(salary, 1) over(order by salary range between 1000 preceding and 1000 following), +-- salary from empsalary; + +select last(salary) over(order by salary range between 1000 preceding and 1000 following), +lag(salary) over(order by salary range between 1000 preceding and 1000 following), +salary from empsalary; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- select first_value(salary) over(order by salary range between 1000 following and 3000 following +-- exclude current row), +-- lead(salary) over(order by salary range between 1000 following and 3000 following exclude ties), +-- nth_value(salary, 1) over(order by salary range between 1000 following and 3000 following +-- exclude ties), +-- salary from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select last(salary) over(order by salary range between 1000 following and 3000 following +-- exclude group), +-- lag(salary) over(order by salary range between 1000 following and 3000 following exclude group), +-- salary from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude group), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude group), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude current row), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude current row), +-- salary, enroll_date from empsalary; + +-- RANGE offset PRECEDING/FOLLOWING with null values +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls first range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls last range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls first range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls last range between 2 preceding and 2 following); + +-- Check overflow behavior for various integer sizes + +select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) +from range(32764, 32767) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 2147450885 following) +from range(-32766, -32765) x; + +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(2147483644, 2147483647) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-2147483646, -2147483645) x; + +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(9223372036854775804, 9223372036854775807) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-9223372036854775806, -9223372036854775805) x; + +-- Test in_range for other numeric datatypes + +create table numerics ( + id int, + f_float4 float, + f_float8 float, + f_numeric int +) using parquet; + +insert into numerics values +(1, -3, -3, -3), +(2, -1, -1, -1), +(3, 0, 0, 0), +(4, 1.1, 1.1, 1.1), +(5, 1.12, 1.12, 1.12), +(6, 2, 2, 2), +(7, 100, 100, 100); +-- (8, 'infinity', 'infinity', '1000'), +-- (9, 'NaN', 'NaN', 'NaN'), +-- (0, '-infinity', '-infinity', '-1000'); -- numeric type lacks infinities + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1 following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1.1 following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 'inf' preceding and 'inf' following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1 following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1.1 following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 'inf' preceding and 'inf' following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1 following); + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following); + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following); -- currently unsupported + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +drop table empsalary; +drop table numerics; diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out new file mode 100644 index 0000000000000..0d2c78847b97c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -0,0 +1,479 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 31 + + +-- !query 0 +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 1 schema +struct +-- !query 1 output +12 1 1 +12 5 1 +12 9 1 +23 3 3 +23 7 3 +27 2 2 +27 6 2 +NULL 0 0 +NULL 4 0 +NULL 8 0 + + +-- !query 2 +SELECT sum(unique1) over (order by four desc range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 2 schema +struct +-- !query 2 output +10 2 2 +10 6 2 +18 1 1 +18 5 1 +18 9 1 +23 0 0 +23 4 0 +23 8 0 +NULL 3 3 +NULL 7 3 + + +-- !query 3 +SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 3 schema +struct +-- !query 3 output +10 3 3 +10 7 3 +12 4 0 +12 8 0 +14 9 1 +15 5 1 +4 0 0 +6 1 1 +8 2 2 +8 6 2 + + +-- !query 4 +select last(salary) over(order by salary range between 1000 preceding and 1000 following), +lag(salary) over(order by salary range between 1000 preceding and 1000 following), +salary from empsalary +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +Window Frame specifiedwindowframe(RangeFrame, -1000, 1000) must match the required frame specifiedwindowframe(RowFrame, -1, -1); + + +-- !query 5 +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls first range between 2 preceding and 2 following) +-- !query 5 schema +struct +-- !query 5 output +1 1 1 3 +2 2 1 4 +3 3 1 5 +4 4 2 5 +5 5 3 5 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query 6 +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls last range between 2 preceding and 2 following) +-- !query 6 schema +struct +-- !query 6 output +1 1 1 3 +2 2 1 4 +3 3 1 5 +4 4 2 5 +5 5 3 5 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query 7 +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls first range between 2 preceding and 2 following) +-- !query 7 schema +struct +-- !query 7 output +1 1 3 1 +2 2 4 1 +3 3 5 1 +4 4 5 2 +5 5 5 3 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query 8 +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls last range between 2 preceding and 2 following) +-- !query 8 schema +struct +-- !query 8 output +1 1 3 1 +2 2 4 1 +3 3 5 1 +4 4 5 2 +5 5 5 3 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query 9 +select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) +from range(32764, 32767) x +-- !query 9 schema +struct +-- !query 9 output +32764 32766 +32765 32766 +32766 32766 + + +-- !query 10 +select x.id, last(x.id) over (order by x.id desc range between current row and 2147450885 following) +from range(-32766, -32765) x +-- !query 10 schema +struct +-- !query 10 output +-32766 -32766 + + +-- !query 11 +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(2147483644, 2147483647) x +-- !query 11 schema +struct +-- !query 11 output +2147483644 2147483646 +2147483645 2147483646 +2147483646 2147483646 + + +-- !query 12 +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-2147483646, -2147483645) x +-- !query 12 schema +struct +-- !query 12 output +-2147483646 -2147483646 + + +-- !query 13 +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(9223372036854775804, 9223372036854775807) x +-- !query 13 schema +struct<> +-- !query 13 output +java.lang.ArithmeticException +long overflow + + +-- !query 14 +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-9223372036854775806, -9223372036854775805) x +-- !query 14 schema +struct<> +-- !query 14 output +java.lang.ArithmeticException +long overflow + + +-- !query 15 +create table numerics ( + id int, + f_float4 float, + f_float8 float, + f_numeric int +) using parquet +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +insert into numerics values +(1, -3, -3, -3), +(2, -1, -1, -1), +(3, 0, 0, 0), +(4, 1.1, 1.1, 1.1), +(5, 1.12, 1.12, 1.12), +(6, 2, 2, 2), +(7, 100, 100, 100) +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1 following) +-- !query 17 schema +struct +-- !query 17 output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 3 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query 18 +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1.1 following) +-- !query 18 schema +struct +-- !query 18 output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 4 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query 19 +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 'inf' preceding and 'inf' following) +-- !query 19 schema +struct +-- !query 19 output +1 -3.0 1 7 +2 -1.0 1 7 +3 0.0 1 7 +4 1.1 1 7 +5 1.12 1 7 +6 2.0 1 7 +7 100.0 1 7 + + +-- !query 20 +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1.1 preceding and 'NaN' following) +-- !query 20 schema +struct +-- !query 20 output +1 -3.0 1 7 +2 -1.0 2 7 +3 0.0 2 7 +4 1.1 3 7 +5 1.12 4 7 +6 2.0 4 7 +7 100.0 7 7 + + +-- !query 21 +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1 following) +-- !query 21 schema +struct +-- !query 21 output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 3 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query 22 +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1.1 following) +-- !query 22 schema +struct +-- !query 22 output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 4 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query 23 +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 'inf' preceding and 'inf' following) +-- !query 23 schema +struct +-- !query 23 output +1 -3.0 1 7 +2 -1.0 1 7 +3 0.0 1 7 +4 1.1 1 7 +5 1.12 1 7 +6 2.0 1 7 +7 100.0 1 7 + + +-- !query 24 +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1.1 preceding and 'NaN' following) +-- !query 24 schema +struct +-- !query 24 output +1 -3.0 1 7 +2 -1.0 2 7 +3 0.0 2 7 +4 1.1 3 7 +5 1.12 4 7 +6 2.0 4 7 +7 100.0 7 7 + + +-- !query 25 +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1 following) +-- !query 25 schema +struct +-- !query 25 output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query 26 +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following) +-- !query 26 schema +struct +-- !query 26 output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query 27 +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following) +-- !query 27 schema +struct +-- !query 27 output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query 28 +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1.1 preceding and 'NaN' following) +-- !query 28 schema +struct +-- !query 28 output +1 -3 NULL NULL +2 -1 NULL NULL +3 0 NULL NULL +4 1 NULL NULL +5 1 NULL NULL +6 2 NULL NULL +7 100 NULL NULL + + +-- !query 29 +drop table empsalary +-- !query 29 schema +struct<> +-- !query 29 output + + + +-- !query 30 +drop table numerics +-- !query 30 schema +struct<> +-- !query 30 output + From 3d567a357c40836dc0e8da67308719a48cc51193 Mon Sep 17 00:00:00 2001 From: wuyi Date: Mon, 21 Oct 2019 21:10:21 -0500 Subject: [PATCH 0383/1387] [MINOR][SQL] Avoid unnecessary invocation on checkAndGlobPathIfNecessary ### What changes were proposed in this pull request? Only invoke `checkAndGlobPathIfNecessary()` when we have to use `InMemoryFileIndex`. ### Why are the changes needed? Avoid unnecessary function invocation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass Jenkins. Closes #26196 from Ngone51/dev-avoid-unnecessary-invocation-on-globpath. Authored-by: wuyi Signed-off-by: Sean Owen --- .../apache/spark/sql/execution/datasources/DataSource.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 0f5f1591623af..e9b8fae7cd735 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -378,8 +378,6 @@ case class DataSource( // This is a non-streaming file based datasource. case (format: FileFormat, _) => - val globbedPaths = - checkAndGlobPathIfNecessary(checkEmptyGlobPath = true, checkFilesExist = checkFilesExist) val useCatalogFileIndex = sparkSession.sqlContext.conf.manageFilesourcePartitions && catalogTable.isDefined && catalogTable.get.tracksPartitionsInCatalog && catalogTable.get.partitionColumnNames.nonEmpty @@ -391,6 +389,8 @@ case class DataSource( catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize)) (index, catalogTable.get.dataSchema, catalogTable.get.partitionSchema) } else { + val globbedPaths = checkAndGlobPathIfNecessary( + checkEmptyGlobPath = true, checkFilesExist = checkFilesExist) val index = createInMemoryFileIndex(globbedPaths) val (resultDataSchema, resultPartitionSchema) = getOrInferFileFormatSchema(format, () => index) From 484f93e25506f84d1548504783be9ce940149bb7 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 22 Oct 2019 10:38:06 +0800 Subject: [PATCH 0384/1387] [SPARK-29530][SQL] Make SQLConf in SQL parse process thread safe ### What changes were proposed in this pull request? As I have comment in [SPARK-29516](https://github.com/apache/spark/pull/26172#issuecomment-544364977) SparkSession.sql() method parse process not under current sparksession's conf, so some configuration about parser is not valid in multi-thread situation. In this pr, we add a SQLConf parameter to AbstractSqlParser and initial it with SessionState's conf. Then for each SparkSession's parser process. It will use's it's own SessionState's SQLConf and to be thread safe ### Why are the changes needed? Fix bug ### Does this PR introduce any user-facing change? NO ### How was this patch tested? NO Closes #26187 from AngersZhuuuu/SPARK-29530. Authored-by: angerszhu Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/ParseDriver.scala | 14 +++++++------- .../spark/sql/execution/SparkSqlParser.scala | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 85998e33140d0..a84d29b71ac42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.types.{DataType, StructType} /** * Base SQL parsing infrastructure. */ -abstract class AbstractSqlParser extends ParserInterface with Logging { +abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Logging { /** Creates/Resolves DataType for a given SQL string. */ override def parseDataType(sqlText: String): DataType = parse(sqlText) { parser => @@ -91,16 +91,16 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) - lexer.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced - lexer.ansi = SQLConf.get.ansiEnabled + lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced + lexer.ansi = conf.ansiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) parser.addParseListener(PostProcessor) parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) - parser.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced - parser.ansi = SQLConf.get.ansiEnabled + parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced + parser.ansi = conf.ansiEnabled try { try { @@ -134,12 +134,12 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { /** * Concrete SQL parser for Catalyst-only SQL statements. */ -class CatalystSqlParser(conf: SQLConf) extends AbstractSqlParser { +class CatalystSqlParser(conf: SQLConf) extends AbstractSqlParser(conf) { val astBuilder = new AstBuilder(conf) } /** For test-only. */ -object CatalystSqlParser extends AbstractSqlParser { +object CatalystSqlParser extends AbstractSqlParser(SQLConf.get) { val astBuilder = new AstBuilder(SQLConf.get) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3e7a54877cae8..cdee117813249 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.types.StructType /** * Concrete parser for Spark SQL statements. */ -class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser { +class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser(conf) { val astBuilder = new SparkSqlAstBuilder(conf) private val substitutor = new VariableSubstitution(conf) From 467c3f610f4f83f16405a3110870d57e59059435 Mon Sep 17 00:00:00 2001 From: denglingang Date: Tue, 22 Oct 2019 14:49:23 +0900 Subject: [PATCH 0385/1387] [SPARK-29529][DOCS] Remove unnecessary orc version and hive version in doc ### What changes were proposed in this pull request? This PR remove unnecessary orc version and hive version in doc. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A. Closes #26146 from denglingang/SPARK-24576. Lead-authored-by: denglingang Co-authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- docs/sql-data-sources-orc.md | 2 +- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sql-data-sources-orc.md b/docs/sql-data-sources-orc.md index 45bff17c6cf2b..bddffe02602e8 100644 --- a/docs/sql-data-sources-orc.md +++ b/docs/sql-data-sources-orc.md @@ -31,7 +31,7 @@ the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also spark.sql.orc.impl native - The name of ORC implementation. It can be one of native and hive. native means the native ORC support that is built on Apache ORC 1.4. `hive` means the ORC library in Hive 1.2.1. + The name of ORC implementation. It can be one of native and hive. native means the native ORC support. hive means the ORC library in Hive. spark.sql.orc.enableVectorizedReader diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4944099fcc0d8..75db52e334b86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -626,8 +626,8 @@ object SQLConf { .createWithDefault("snappy") val ORC_IMPLEMENTATION = buildConf("spark.sql.orc.impl") - .doc("When native, use the native version of ORC support instead of the ORC library in Hive " + - "1.2.1. It is 'hive' by default prior to Spark 2.4.") + .doc("When native, use the native version of ORC support instead of the ORC library in Hive." + + "It is 'hive' by default prior to Spark 2.4.") .internal() .stringConf .checkValues(Set("hive", "native")) From 811d563fbf60203377e8462e4fad271c1140b4fa Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 22 Oct 2019 16:18:34 +0900 Subject: [PATCH 0386/1387] [SPARK-29536][PYTHON] Upgrade cloudpickle to 1.1.1 to support Python 3.8 ### What changes were proposed in this pull request? Inline cloudpickle in PySpark to cloudpickle 1.1.1. See https://github.com/cloudpipe/cloudpickle/blob/v1.1.1/cloudpickle/cloudpickle.py https://github.com/cloudpipe/cloudpickle/pull/269 was added for Python 3.8 support (fixed from 1.1.0). Using 1.2.2 seems breaking PyPy 2 due to cloudpipe/cloudpickle#278 so this PR currently uses 1.1.1. Once we drop Python 2, we can switch to the highest version. ### Why are the changes needed? positional-only arguments was newly introduced from Python 3.8 (see https://docs.python.org/3/whatsnew/3.8.html#positional-only-parameters) Particularly the newly added argument to `types.CodeType` was the problem (https://docs.python.org/3/whatsnew/3.8.html#changes-in-the-python-api): > `types.CodeType` has a new parameter in the second position of the constructor (posonlyargcount) to support positional-only arguments defined in **PEP 570**. The first argument (argcount) now represents the total number of positional arguments (including positional-only arguments). The new `replace()` method of `types.CodeType` can be used to make the code future-proof. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested. Note that the optional dependency PyArrow looks not yet supporting Python 3.8; therefore, it was not tested. See "Details" below.

      ```bash cd python ./run-tests --python-executables=python3.8 ``` ``` Running PySpark tests. Output is in /Users/hyukjin.kwon/workspace/forked/spark/python/unit-tests.log Will test against the following Python executables: ['python3.8'] Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming'] Starting test(python3.8): pyspark.ml.tests.test_algorithms Starting test(python3.8): pyspark.ml.tests.test_feature Starting test(python3.8): pyspark.ml.tests.test_base Starting test(python3.8): pyspark.ml.tests.test_evaluation Finished test(python3.8): pyspark.ml.tests.test_base (12s) Starting test(python3.8): pyspark.ml.tests.test_image Finished test(python3.8): pyspark.ml.tests.test_evaluation (14s) Starting test(python3.8): pyspark.ml.tests.test_linalg Finished test(python3.8): pyspark.ml.tests.test_feature (23s) Starting test(python3.8): pyspark.ml.tests.test_param Finished test(python3.8): pyspark.ml.tests.test_image (22s) Starting test(python3.8): pyspark.ml.tests.test_persistence Finished test(python3.8): pyspark.ml.tests.test_param (25s) Starting test(python3.8): pyspark.ml.tests.test_pipeline Finished test(python3.8): pyspark.ml.tests.test_linalg (37s) Starting test(python3.8): pyspark.ml.tests.test_stat Finished test(python3.8): pyspark.ml.tests.test_pipeline (7s) Starting test(python3.8): pyspark.ml.tests.test_training_summary Finished test(python3.8): pyspark.ml.tests.test_stat (21s) Starting test(python3.8): pyspark.ml.tests.test_tuning Finished test(python3.8): pyspark.ml.tests.test_persistence (45s) Starting test(python3.8): pyspark.ml.tests.test_wrapper Finished test(python3.8): pyspark.ml.tests.test_algorithms (83s) Starting test(python3.8): pyspark.mllib.tests.test_algorithms Finished test(python3.8): pyspark.ml.tests.test_training_summary (32s) Starting test(python3.8): pyspark.mllib.tests.test_feature Finished test(python3.8): pyspark.ml.tests.test_wrapper (20s) Starting test(python3.8): pyspark.mllib.tests.test_linalg Finished test(python3.8): pyspark.mllib.tests.test_feature (32s) Starting test(python3.8): pyspark.mllib.tests.test_stat Finished test(python3.8): pyspark.mllib.tests.test_algorithms (70s) Starting test(python3.8): pyspark.mllib.tests.test_streaming_algorithms Finished test(python3.8): pyspark.mllib.tests.test_stat (37s) Starting test(python3.8): pyspark.mllib.tests.test_util Finished test(python3.8): pyspark.mllib.tests.test_linalg (70s) Starting test(python3.8): pyspark.sql.tests.test_arrow Finished test(python3.8): pyspark.sql.tests.test_arrow (1s) ... 53 tests were skipped Starting test(python3.8): pyspark.sql.tests.test_catalog Finished test(python3.8): pyspark.mllib.tests.test_util (15s) Starting test(python3.8): pyspark.sql.tests.test_column Finished test(python3.8): pyspark.sql.tests.test_catalog (24s) Starting test(python3.8): pyspark.sql.tests.test_conf Finished test(python3.8): pyspark.sql.tests.test_column (21s) Starting test(python3.8): pyspark.sql.tests.test_context Finished test(python3.8): pyspark.ml.tests.test_tuning (125s) Starting test(python3.8): pyspark.sql.tests.test_dataframe Finished test(python3.8): pyspark.sql.tests.test_conf (9s) Starting test(python3.8): pyspark.sql.tests.test_datasources Finished test(python3.8): pyspark.sql.tests.test_context (29s) Starting test(python3.8): pyspark.sql.tests.test_functions Finished test(python3.8): pyspark.sql.tests.test_datasources (32s) Starting test(python3.8): pyspark.sql.tests.test_group Finished test(python3.8): pyspark.sql.tests.test_dataframe (39s) ... 3 tests were skipped Starting test(python3.8): pyspark.sql.tests.test_pandas_udf Finished test(python3.8): pyspark.sql.tests.test_pandas_udf (1s) ... 6 tests were skipped Starting test(python3.8): pyspark.sql.tests.test_pandas_udf_cogrouped_map Finished test(python3.8): pyspark.sql.tests.test_pandas_udf_cogrouped_map (0s) ... 14 tests were skipped Starting test(python3.8): pyspark.sql.tests.test_pandas_udf_grouped_agg Finished test(python3.8): pyspark.sql.tests.test_pandas_udf_grouped_agg (1s) ... 15 tests were skipped Starting test(python3.8): pyspark.sql.tests.test_pandas_udf_grouped_map Finished test(python3.8): pyspark.sql.tests.test_pandas_udf_grouped_map (1s) ... 20 tests were skipped Starting test(python3.8): pyspark.sql.tests.test_pandas_udf_scalar Finished test(python3.8): pyspark.sql.tests.test_pandas_udf_scalar (1s) ... 49 tests were skipped Starting test(python3.8): pyspark.sql.tests.test_pandas_udf_window Finished test(python3.8): pyspark.sql.tests.test_pandas_udf_window (1s) ... 14 tests were skipped Starting test(python3.8): pyspark.sql.tests.test_readwriter Finished test(python3.8): pyspark.sql.tests.test_functions (29s) Starting test(python3.8): pyspark.sql.tests.test_serde Finished test(python3.8): pyspark.sql.tests.test_group (20s) Starting test(python3.8): pyspark.sql.tests.test_session Finished test(python3.8): pyspark.mllib.tests.test_streaming_algorithms (126s) Starting test(python3.8): pyspark.sql.tests.test_streaming Finished test(python3.8): pyspark.sql.tests.test_serde (25s) Starting test(python3.8): pyspark.sql.tests.test_types Finished test(python3.8): pyspark.sql.tests.test_readwriter (38s) Starting test(python3.8): pyspark.sql.tests.test_udf Finished test(python3.8): pyspark.sql.tests.test_session (32s) Starting test(python3.8): pyspark.sql.tests.test_utils Finished test(python3.8): pyspark.sql.tests.test_utils (17s) Starting test(python3.8): pyspark.streaming.tests.test_context Finished test(python3.8): pyspark.sql.tests.test_types (45s) Starting test(python3.8): pyspark.streaming.tests.test_dstream Finished test(python3.8): pyspark.sql.tests.test_udf (44s) Starting test(python3.8): pyspark.streaming.tests.test_kinesis Finished test(python3.8): pyspark.streaming.tests.test_kinesis (0s) ... 2 tests were skipped Starting test(python3.8): pyspark.streaming.tests.test_listener Finished test(python3.8): pyspark.streaming.tests.test_context (28s) Starting test(python3.8): pyspark.tests.test_appsubmit Finished test(python3.8): pyspark.sql.tests.test_streaming (60s) Starting test(python3.8): pyspark.tests.test_broadcast Finished test(python3.8): pyspark.streaming.tests.test_listener (11s) Starting test(python3.8): pyspark.tests.test_conf Finished test(python3.8): pyspark.tests.test_conf (17s) Starting test(python3.8): pyspark.tests.test_context Finished test(python3.8): pyspark.tests.test_broadcast (39s) Starting test(python3.8): pyspark.tests.test_daemon Finished test(python3.8): pyspark.tests.test_daemon (5s) Starting test(python3.8): pyspark.tests.test_join Finished test(python3.8): pyspark.tests.test_context (31s) Starting test(python3.8): pyspark.tests.test_profiler Finished test(python3.8): pyspark.tests.test_join (9s) Starting test(python3.8): pyspark.tests.test_rdd Finished test(python3.8): pyspark.tests.test_profiler (12s) Starting test(python3.8): pyspark.tests.test_readwrite Finished test(python3.8): pyspark.tests.test_readwrite (23s) ... 3 tests were skipped Starting test(python3.8): pyspark.tests.test_serializers Finished test(python3.8): pyspark.tests.test_appsubmit (94s) Starting test(python3.8): pyspark.tests.test_shuffle Finished test(python3.8): pyspark.streaming.tests.test_dstream (110s) Starting test(python3.8): pyspark.tests.test_taskcontext Finished test(python3.8): pyspark.tests.test_rdd (42s) Starting test(python3.8): pyspark.tests.test_util Finished test(python3.8): pyspark.tests.test_serializers (11s) Starting test(python3.8): pyspark.tests.test_worker Finished test(python3.8): pyspark.tests.test_shuffle (12s) Starting test(python3.8): pyspark.accumulators Finished test(python3.8): pyspark.tests.test_util (7s) Starting test(python3.8): pyspark.broadcast Finished test(python3.8): pyspark.accumulators (8s) Starting test(python3.8): pyspark.conf Finished test(python3.8): pyspark.broadcast (8s) Starting test(python3.8): pyspark.context Finished test(python3.8): pyspark.tests.test_worker (19s) Starting test(python3.8): pyspark.ml.classification Finished test(python3.8): pyspark.conf (4s) Starting test(python3.8): pyspark.ml.clustering Finished test(python3.8): pyspark.context (22s) Starting test(python3.8): pyspark.ml.evaluation Finished test(python3.8): pyspark.tests.test_taskcontext (49s) Starting test(python3.8): pyspark.ml.feature Finished test(python3.8): pyspark.ml.clustering (43s) Starting test(python3.8): pyspark.ml.fpm Finished test(python3.8): pyspark.ml.evaluation (27s) Starting test(python3.8): pyspark.ml.image Finished test(python3.8): pyspark.ml.image (8s) Starting test(python3.8): pyspark.ml.linalg.__init__ Finished test(python3.8): pyspark.ml.linalg.__init__ (0s) Starting test(python3.8): pyspark.ml.recommendation Finished test(python3.8): pyspark.ml.classification (63s) Starting test(python3.8): pyspark.ml.regression Finished test(python3.8): pyspark.ml.fpm (23s) Starting test(python3.8): pyspark.ml.stat Finished test(python3.8): pyspark.ml.stat (30s) Starting test(python3.8): pyspark.ml.tuning Finished test(python3.8): pyspark.ml.regression (51s) Starting test(python3.8): pyspark.mllib.classification Finished test(python3.8): pyspark.ml.feature (93s) Starting test(python3.8): pyspark.mllib.clustering Finished test(python3.8): pyspark.ml.tuning (39s) Starting test(python3.8): pyspark.mllib.evaluation Finished test(python3.8): pyspark.mllib.classification (38s) Starting test(python3.8): pyspark.mllib.feature Finished test(python3.8): pyspark.mllib.evaluation (25s) Starting test(python3.8): pyspark.mllib.fpm Finished test(python3.8): pyspark.mllib.clustering (64s) Starting test(python3.8): pyspark.mllib.linalg.__init__ Finished test(python3.8): pyspark.ml.recommendation (131s) Starting test(python3.8): pyspark.mllib.linalg.distributed Finished test(python3.8): pyspark.mllib.linalg.__init__ (0s) Starting test(python3.8): pyspark.mllib.random Finished test(python3.8): pyspark.mllib.feature (36s) Starting test(python3.8): pyspark.mllib.recommendation Finished test(python3.8): pyspark.mllib.fpm (31s) Starting test(python3.8): pyspark.mllib.regression Finished test(python3.8): pyspark.mllib.random (16s) Starting test(python3.8): pyspark.mllib.stat.KernelDensity Finished test(python3.8): pyspark.mllib.stat.KernelDensity (1s) Starting test(python3.8): pyspark.mllib.stat._statistics Finished test(python3.8): pyspark.mllib.stat._statistics (25s) Starting test(python3.8): pyspark.mllib.tree Finished test(python3.8): pyspark.mllib.regression (44s) Starting test(python3.8): pyspark.mllib.util Finished test(python3.8): pyspark.mllib.recommendation (49s) Starting test(python3.8): pyspark.profiler Finished test(python3.8): pyspark.mllib.linalg.distributed (53s) Starting test(python3.8): pyspark.rdd Finished test(python3.8): pyspark.profiler (14s) Starting test(python3.8): pyspark.serializers Finished test(python3.8): pyspark.mllib.tree (30s) Starting test(python3.8): pyspark.shuffle Finished test(python3.8): pyspark.shuffle (2s) Starting test(python3.8): pyspark.sql.avro.functions Finished test(python3.8): pyspark.mllib.util (30s) Starting test(python3.8): pyspark.sql.catalog Finished test(python3.8): pyspark.serializers (17s) Starting test(python3.8): pyspark.sql.column Finished test(python3.8): pyspark.rdd (31s) Starting test(python3.8): pyspark.sql.conf Finished test(python3.8): pyspark.sql.conf (7s) Starting test(python3.8): pyspark.sql.context Finished test(python3.8): pyspark.sql.avro.functions (19s) Starting test(python3.8): pyspark.sql.dataframe Finished test(python3.8): pyspark.sql.catalog (16s) Starting test(python3.8): pyspark.sql.functions Finished test(python3.8): pyspark.sql.column (27s) Starting test(python3.8): pyspark.sql.group Finished test(python3.8): pyspark.sql.context (26s) Starting test(python3.8): pyspark.sql.readwriter Finished test(python3.8): pyspark.sql.group (52s) Starting test(python3.8): pyspark.sql.session Finished test(python3.8): pyspark.sql.dataframe (73s) Starting test(python3.8): pyspark.sql.streaming Finished test(python3.8): pyspark.sql.functions (75s) Starting test(python3.8): pyspark.sql.types Finished test(python3.8): pyspark.sql.readwriter (57s) Starting test(python3.8): pyspark.sql.udf Finished test(python3.8): pyspark.sql.types (13s) Starting test(python3.8): pyspark.sql.window Finished test(python3.8): pyspark.sql.session (32s) Starting test(python3.8): pyspark.streaming.util Finished test(python3.8): pyspark.streaming.util (1s) Starting test(python3.8): pyspark.util Finished test(python3.8): pyspark.util (0s) Finished test(python3.8): pyspark.sql.streaming (30s) Finished test(python3.8): pyspark.sql.udf (27s) Finished test(python3.8): pyspark.sql.window (22s) Tests passed in 855 seconds ```

      Closes #26194 from HyukjinKwon/SPARK-29536. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/cloudpickle.py | 257 +++++++++++++++++++++++++++++----- python/setup.py | 1 + 2 files changed, 221 insertions(+), 37 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 7df5f6c748ad1..09d3a5e7cfb6f 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -44,7 +44,6 @@ import dis from functools import partial -import importlib import io import itertools import logging @@ -56,12 +55,26 @@ import traceback import types import weakref +import uuid +import threading + + +try: + from enum import Enum +except ImportError: + Enum = None # cloudpickle is meant for inter process communication: we expect all # communicating processes to run the same Python version hence we favor # communication speed over compatibility: DEFAULT_PROTOCOL = pickle.HIGHEST_PROTOCOL +# Track the provenance of reconstructed dynamic classes to make it possible to +# recontruct instances from the matching singleton class definition when +# appropriate and preserve the usual "isinstance" semantics of Python objects. +_DYNAMIC_CLASS_TRACKER_BY_CLASS = weakref.WeakKeyDictionary() +_DYNAMIC_CLASS_TRACKER_BY_ID = weakref.WeakValueDictionary() +_DYNAMIC_CLASS_TRACKER_LOCK = threading.Lock() if sys.version_info[0] < 3: # pragma: no branch from pickle import Pickler @@ -71,12 +84,37 @@ from StringIO import StringIO string_types = (basestring,) # noqa PY3 = False + PY2 = True + PY2_WRAPPER_DESCRIPTOR_TYPE = type(object.__init__) + PY2_METHOD_WRAPPER_TYPE = type(object.__eq__) + PY2_CLASS_DICT_BLACKLIST = (PY2_METHOD_WRAPPER_TYPE, + PY2_WRAPPER_DESCRIPTOR_TYPE) else: types.ClassType = type from pickle import _Pickler as Pickler from io import BytesIO as StringIO string_types = (str,) PY3 = True + PY2 = False + + +def _ensure_tracking(class_def): + with _DYNAMIC_CLASS_TRACKER_LOCK: + class_tracker_id = _DYNAMIC_CLASS_TRACKER_BY_CLASS.get(class_def) + if class_tracker_id is None: + class_tracker_id = uuid.uuid4().hex + _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id + _DYNAMIC_CLASS_TRACKER_BY_ID[class_tracker_id] = class_def + return class_tracker_id + + +def _lookup_class_or_track(class_tracker_id, class_def): + if class_tracker_id is not None: + with _DYNAMIC_CLASS_TRACKER_LOCK: + class_def = _DYNAMIC_CLASS_TRACKER_BY_ID.setdefault( + class_tracker_id, class_def) + _DYNAMIC_CLASS_TRACKER_BY_CLASS[class_def] = class_tracker_id + return class_def def _make_cell_set_template_code(): @@ -112,7 +150,7 @@ def inner(value): # NOTE: we are marking the cell variable as a free variable intentionally # so that we simulate an inner function instead of the outer function. This # is what gives us the ``nonlocal`` behavior in a Python 2 compatible way. - if not PY3: # pragma: no branch + if PY2: # pragma: no branch return types.CodeType( co.co_argcount, co.co_nlocals, @@ -130,24 +168,43 @@ def inner(value): (), ) else: - return types.CodeType( - co.co_argcount, - co.co_kwonlyargcount, - co.co_nlocals, - co.co_stacksize, - co.co_flags, - co.co_code, - co.co_consts, - co.co_names, - co.co_varnames, - co.co_filename, - co.co_name, - co.co_firstlineno, - co.co_lnotab, - co.co_cellvars, # this is the trickery - (), - ) - + if hasattr(types.CodeType, "co_posonlyargcount"): # pragma: no branch + return types.CodeType( + co.co_argcount, + co.co_posonlyargcount, # Python3.8 with PEP570 + co.co_kwonlyargcount, + co.co_nlocals, + co.co_stacksize, + co.co_flags, + co.co_code, + co.co_consts, + co.co_names, + co.co_varnames, + co.co_filename, + co.co_name, + co.co_firstlineno, + co.co_lnotab, + co.co_cellvars, # this is the trickery + (), + ) + else: + return types.CodeType( + co.co_argcount, + co.co_kwonlyargcount, + co.co_nlocals, + co.co_stacksize, + co.co_flags, + co.co_code, + co.co_consts, + co.co_names, + co.co_varnames, + co.co_filename, + co.co_name, + co.co_firstlineno, + co.co_lnotab, + co.co_cellvars, # this is the trickery + (), + ) _cell_set_template_code = _make_cell_set_template_code() @@ -220,7 +277,7 @@ def _walk_global_ops(code): global-referencing instructions in *code*. """ code = getattr(code, 'co_code', b'') - if not PY3: # pragma: no branch + if PY2: # pragma: no branch code = map(ord, code) n = len(code) @@ -250,6 +307,39 @@ def _walk_global_ops(code): yield op, instr.arg +def _extract_class_dict(cls): + """Retrieve a copy of the dict of a class without the inherited methods""" + clsdict = dict(cls.__dict__) # copy dict proxy to a dict + if len(cls.__bases__) == 1: + inherited_dict = cls.__bases__[0].__dict__ + else: + inherited_dict = {} + for base in reversed(cls.__bases__): + inherited_dict.update(base.__dict__) + to_remove = [] + for name, value in clsdict.items(): + try: + base_value = inherited_dict[name] + if value is base_value: + to_remove.append(name) + elif PY2: + # backward compat for Python 2 + if hasattr(value, "im_func"): + if value.im_func is getattr(base_value, "im_func", None): + to_remove.append(name) + elif isinstance(value, PY2_CLASS_DICT_BLACKLIST): + # On Python 2 we have no way to pickle those specific + # methods types nor to check that they are actually + # inherited. So we assume that they are always inherited + # from builtin types. + to_remove.append(name) + except KeyError: + pass + for name in to_remove: + clsdict.pop(name) + return clsdict + + class CloudPickler(Pickler): dispatch = Pickler.dispatch.copy() @@ -277,7 +367,7 @@ def save_memoryview(self, obj): dispatch[memoryview] = save_memoryview - if not PY3: # pragma: no branch + if PY2: # pragma: no branch def save_buffer(self, obj): self.save(str(obj)) @@ -300,12 +390,23 @@ def save_codeobject(self, obj): Save a code object """ if PY3: # pragma: no branch - args = ( - obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, - obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, obj.co_varnames, - obj.co_filename, obj.co_name, obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, - obj.co_cellvars - ) + if hasattr(obj, "co_posonlyargcount"): # pragma: no branch + args = ( + obj.co_argcount, obj.co_posonlyargcount, + obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, + obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, + obj.co_varnames, obj.co_filename, obj.co_name, + obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, + obj.co_cellvars + ) + else: + args = ( + obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, + obj.co_stacksize, obj.co_flags, obj.co_code, obj.co_consts, + obj.co_names, obj.co_varnames, obj.co_filename, + obj.co_name, obj.co_firstlineno, obj.co_lnotab, + obj.co_freevars, obj.co_cellvars + ) else: args = ( obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, @@ -460,15 +561,40 @@ def func(): # then discards the reference to it self.write(pickle.POP) - def save_dynamic_class(self, obj): + def _save_dynamic_enum(self, obj, clsdict): + """Special handling for dynamic Enum subclasses + + Use a dedicated Enum constructor (inspired by EnumMeta.__call__) as the + EnumMeta metaclass has complex initialization that makes the Enum + subclasses hold references to their own instances. """ - Save a class that can't be stored as module global. + members = dict((e.name, e.value) for e in obj) + + # Python 2.7 with enum34 can have no qualname: + qualname = getattr(obj, "__qualname__", None) + + self.save_reduce(_make_skeleton_enum, + (obj.__bases__, obj.__name__, qualname, members, + obj.__module__, _ensure_tracking(obj), None), + obj=obj) + + # Cleanup the clsdict that will be passed to _rehydrate_skeleton_class: + # Those attributes are already handled by the metaclass. + for attrname in ["_generate_next_value_", "_member_names_", + "_member_map_", "_member_type_", + "_value2member_map_"]: + clsdict.pop(attrname, None) + for member in members: + clsdict.pop(member) + + def save_dynamic_class(self, obj): + """Save a class that can't be stored as module global. This method is used to serialize classes that are defined inside functions, or that otherwise can't be serialized as attribute lookups from global modules. """ - clsdict = dict(obj.__dict__) # copy dict proxy to a dict + clsdict = _extract_class_dict(obj) clsdict.pop('__weakref__', None) # For ABCMeta in python3.7+, remove _abc_impl as it is not picklable. @@ -496,8 +622,8 @@ def save_dynamic_class(self, obj): for k in obj.__slots__: clsdict.pop(k, None) - # If type overrides __dict__ as a property, include it in the type kwargs. - # In Python 2, we can't set this attribute after construction. + # If type overrides __dict__ as a property, include it in the type + # kwargs. In Python 2, we can't set this attribute after construction. __dict__ = clsdict.pop('__dict__', None) if isinstance(__dict__, property): type_kwargs['__dict__'] = __dict__ @@ -524,8 +650,16 @@ def save_dynamic_class(self, obj): write(pickle.MARK) # Create and memoize an skeleton class with obj's name and bases. - tp = type(obj) - self.save_reduce(tp, (obj.__name__, obj.__bases__, type_kwargs), obj=obj) + if Enum is not None and issubclass(obj, Enum): + # Special handling of Enum subclasses + self._save_dynamic_enum(obj, clsdict) + else: + # "Regular" class definition: + tp = type(obj) + self.save_reduce(_make_skeleton_class, + (tp, obj.__name__, obj.__bases__, type_kwargs, + _ensure_tracking(obj), None), + obj=obj) # Now save the rest of obj's __dict__. Any references to obj # encountered while saving will point to the skeleton class. @@ -778,7 +912,7 @@ def save_inst(self, obj): save(stuff) write(pickle.BUILD) - if not PY3: # pragma: no branch + if PY2: # pragma: no branch dispatch[types.InstanceType] = save_inst def save_property(self, obj): @@ -1119,6 +1253,22 @@ def _make_skel_func(code, cell_count, base_globals=None): return types.FunctionType(code, base_globals, None, None, closure) +def _make_skeleton_class(type_constructor, name, bases, type_kwargs, + class_tracker_id, extra): + """Build dynamic class with an empty __dict__ to be filled once memoized + + If class_tracker_id is not None, try to lookup an existing class definition + matching that id. If none is found, track a newly reconstructed class + definition under that id so that other instances stemming from the same + class id will also reuse this class definition. + + The "extra" variable is meant to be a dict (or None) that can be used for + forward compatibility shall the need arise. + """ + skeleton_class = type_constructor(name, bases, type_kwargs) + return _lookup_class_or_track(class_tracker_id, skeleton_class) + + def _rehydrate_skeleton_class(skeleton_class, class_dict): """Put attributes from `class_dict` back on `skeleton_class`. @@ -1137,6 +1287,39 @@ def _rehydrate_skeleton_class(skeleton_class, class_dict): return skeleton_class +def _make_skeleton_enum(bases, name, qualname, members, module, + class_tracker_id, extra): + """Build dynamic enum with an empty __dict__ to be filled once memoized + + The creation of the enum class is inspired by the code of + EnumMeta._create_. + + If class_tracker_id is not None, try to lookup an existing enum definition + matching that id. If none is found, track a newly reconstructed enum + definition under that id so that other instances stemming from the same + class id will also reuse this enum definition. + + The "extra" variable is meant to be a dict (or None) that can be used for + forward compatibility shall the need arise. + """ + # enums always inherit from their base Enum class at the last position in + # the list of base classes: + enum_base = bases[-1] + metacls = enum_base.__class__ + classdict = metacls.__prepare__(name, bases) + + for member_name, member_value in members.items(): + classdict[member_name] = member_value + enum_class = metacls.__new__(metacls, name, bases, classdict) + enum_class.__module__ = module + + # Python 2.7 compat + if qualname is not None: + enum_class.__qualname__ = qualname + + return _lookup_class_or_track(class_tracker_id, enum_class) + + def _is_dynamic(module): """ Return True if the module is special module that cannot be imported by its @@ -1176,4 +1359,4 @@ def _reduce_method_descriptor(obj): import copy_reg as copyreg except ImportError: import copyreg - copyreg.pickle(method_descriptor, _reduce_method_descriptor) + copyreg.pickle(method_descriptor, _reduce_method_descriptor) \ No newline at end of file diff --git a/python/setup.py b/python/setup.py index ee5c32683efae..ea672309703b6 100644 --- a/python/setup.py +++ b/python/setup.py @@ -230,6 +230,7 @@ def _supports_symlinks(): 'Programming Language :: Python :: 3.5', 'Programming Language :: Python :: 3.6', 'Programming Language :: Python :: 3.7', + 'Programming Language :: Python :: 3.8', 'Programming Language :: Python :: Implementation :: CPython', 'Programming Language :: Python :: Implementation :: PyPy'] ) From 868d851dac6016a2fc5665fb2a3ea01ab184402a Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 22 Oct 2019 17:49:44 +0800 Subject: [PATCH 0387/1387] [SPARK-29232][ML] Update the parameter maps of the DecisionTreeRegression/Classification Models ### What changes were proposed in this pull request? The trees (Array[```DecisionTreeRegressionModel```]) in ```RandomForestRegressionModel``` only contains the default parameter value. Need to update the parameter maps for these trees. Same issues in ```RandomForestClassifier```, ```GBTClassifier``` and ```GBTRegressor``` ### Why are the changes needed? User wants to access each individual tree and build the trees back up for the random forest estimator. This doesn't work because trees don't have the correct parameter values ### Does this PR introduce any user-facing change? Yes. Now the trees in ```RandomForestRegressionModel```, ```RandomForestClassifier```, ```GBTClassifier``` and ```GBTRegressor``` have the correct parameter values. ### How was this patch tested? Add tests Closes #26154 from huaxingao/spark-29232. Authored-by: Huaxin Gao Signed-off-by: zhengruifeng --- .../ml/classification/GBTClassifier.scala | 1 + .../RandomForestClassifier.scala | 1 + .../spark/ml/regression/GBTRegressor.scala | 1 + .../ml/regression/RandomForestRegressor.scala | 1 + .../classification/GBTClassifierSuite.scala | 16 +++++++++++++++ .../RandomForestClassifierSuite.scala | 20 +++++++++++++++++++ .../ml/regression/GBTRegressorSuite.scala | 16 ++++++++++++++- .../RandomForestRegressorSuite.scala | 19 ++++++++++++++++++ 8 files changed, 74 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 09f81b0dcbdae..74624be360c6a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -203,6 +203,7 @@ class GBTClassifier @Since("1.4.0") ( } else { GradientBoostedTrees.run(trainDataset, boostingStrategy, $(seed), $(featureSubsetStrategy)) } + baseLearners.foreach(copyValues(_)) val numFeatures = baseLearners.head.numFeatures instr.logNumFeatures(numFeatures) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 731b43b67813f..245cda35d8ade 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -143,6 +143,7 @@ class RandomForestClassifier @Since("1.4.0") ( val trees = RandomForest .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) .map(_.asInstanceOf[DecisionTreeClassificationModel]) + trees.foreach(copyValues(_)) val numFeatures = trees.head.numFeatures instr.logNumClasses(numClasses) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 00c0bc9f5e282..0cc06d82bf3fd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -181,6 +181,7 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) GradientBoostedTrees.run(trainDataset, boostingStrategy, $(seed), $(featureSubsetStrategy)) } + baseLearners.foreach(copyValues(_)) val numFeatures = baseLearners.head.numFeatures instr.logNumFeatures(numFeatures) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 938aa5acac086..8f78fc1da18c8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -130,6 +130,7 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S val trees = RandomForest .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) .map(_.asInstanceOf[DecisionTreeRegressionModel]) + trees.foreach(copyValues(_)) val numFeatures = trees.head.numFeatures instr.logNamedValue(Instrumentation.loggerTags.numFeatures, numFeatures) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index af3dd201d3b51..530ca20d0eb0c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -456,6 +456,22 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { } } + test("tree params") { + val categoricalFeatures = Map.empty[Int, Int] + val df: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 2) + val gbt = new GBTClassifier() + .setMaxDepth(2) + .setCheckpointInterval(5) + .setSeed(123) + val model = gbt.fit(df) + + model.trees.foreach (i => { + assert(i.getMaxDepth === model.getMaxDepth) + assert(i.getCheckpointInterval === model.getCheckpointInterval) + assert(i.getSeed === model.getSeed) + }) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index f03ed0b76eb80..5958bfcf5ea6d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -230,6 +230,26 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { } } + test("tree params") { + val rdd = orderedLabeledPoints5_20 + val rf = new RandomForestClassifier() + .setImpurity("entropy") + .setMaxDepth(3) + .setNumTrees(3) + .setSeed(123) + val categoricalFeatures = Map.empty[Int, Int] + val numClasses = 2 + + val df: DataFrame = TreeTests.setMetadata(rdd, categoricalFeatures, numClasses) + val model = rf.fit(df) + + model.trees.foreach (i => { + assert(i.getMaxDepth === model.getMaxDepth) + assert(i.getSeed === model.getSeed) + assert(i.getImpurity === model.getImpurity) + }) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index 60007975c3b52..e2462af2ac1d1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -296,7 +296,21 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { } } - ///////////////////////////////////////////////////////////////////////////// + test("tree params") { + val gbt = new GBTRegressor() + .setMaxDepth(2) + .setCheckpointInterval(5) + .setSeed(123) + val model = gbt.fit(trainData.toDF) + + model.trees.foreach (i => { + assert(i.getMaxDepth === model.getMaxDepth) + assert(i.getCheckpointInterval === model.getCheckpointInterval) + assert(i.getSeed === model.getSeed) + }) + } + + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index 0243e8d2335ee..f3b0f0470e579 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -139,6 +139,25 @@ class RandomForestRegressorSuite extends MLTest with DefaultReadWriteTest{ } } + test("tree params") { + val rf = new RandomForestRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(10) + .setNumTrees(3) + .setSeed(123) + + val df = orderedLabeledPoints50_1000.toDF() + val model = rf.fit(df) + + model.trees.foreach (i => { + assert(i.getMaxDepth === model.getMaxDepth) + assert(i.getSeed === model.getSeed) + assert(i.getImpurity === model.getImpurity) + assert(i.getMaxBins === model.getMaxBins) + }) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// From 3163b6b43b99ca02642cf935d885ed2d0f98d633 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 22 Oct 2019 03:20:49 -0700 Subject: [PATCH 0388/1387] [SPARK-29516][SQL][TEST] Test ThriftServerQueryTestSuite asynchronously ### What changes were proposed in this pull request? This PR test `ThriftServerQueryTestSuite` in an asynchronous way. ### Why are the changes needed? The default value of `spark.sql.hive.thriftServer.async` is `true`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? ``` build/sbt "hive-thriftserver/test-only *.ThriftServerQueryTestSuite" -Phive-thriftserver build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite test -Phive-thriftserver ``` Closes #26172 from wangyum/SPARK-29516. Authored-by: Yuming Wang Signed-off-by: Yuming Wang --- .../thriftserver/ThriftServerQueryTestSuite.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index abe91a280a14f..04b1de00ccbff 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File -import java.sql.{DriverManager, Statement, Timestamp} +import java.sql.{DriverManager, SQLException, Statement, Timestamp} import java.util.{Locale, MissingFormatArgumentException} import scala.util.{Random, Try} @@ -75,11 +75,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } - override def sparkConf: SparkConf = super.sparkConf - // Hive Thrift server should not executes SQL queries in an asynchronous way - // because we may set session configuration. - .set(HiveUtils.HIVE_THRIFT_SERVER_ASYNC, false) - override val isTestWithConfigSets = false /** List of test cases to ignore, in lower cases. */ @@ -208,6 +203,12 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { s"Exception did not match for query #$i\n${expected.sql}, " + s"expected: ${expected.output}, but got: ${output.output}") + // SQLException should not exactly match. We only assert the result contains Exception. + case _ if output.output.startsWith(classOf[SQLException].getName) => + assert(expected.output.contains("Exception"), + s"Exception did not match for query #$i\n${expected.sql}, " + + s"expected: ${expected.output}, but got: ${output.output}") + case _ => assertResult(expected.output, s"Result did not match for query #$i\n${expected.sql}") { output.output From bb49c80c890452dc047a1975b16dcd876705ad23 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 22 Oct 2019 19:08:09 +0800 Subject: [PATCH 0389/1387] [SPARK-21492][SQL] Fix memory leak in SortMergeJoin ### What changes were proposed in this pull request? We shall have a new mechanism that the downstream operators may notify its parents that they may release the output data stream. In this PR, we implement the mechanism as below: - Add function named `cleanupResources` in SparkPlan, which default call children's `cleanupResources` function, the operator which need a resource cleanup should rewrite this with the self cleanup and also call `super.cleanupResources`, like SortExec in this PR. - Add logic support on the trigger side, in this PR is SortMergeJoinExec, which make sure and call the `cleanupResources` to do the cleanup job for all its upstream(children) operator. ### Why are the changes needed? Bugfix for SortMergeJoin memory leak, and implement a general framework for SparkPlan resource cleanup. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? UT: Add new test suite JoinWithResourceCleanSuite to check both standard and code generation scenario. Integrate Test: Test with driver/executor default memory set 1g, local mode 10 thread. The below test(thanks taosaildrone for providing this test [here](https://github.com/apache/spark/pull/23762#issuecomment-463303175)) will pass with this PR. ``` from pyspark.sql.functions import rand, col spark.conf.set("spark.sql.join.preferSortMergeJoin", "true") spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1) # spark.conf.set("spark.sql.sortMergeJoinExec.eagerCleanupResources", "true") r1 = spark.range(1, 1001).select(col("id").alias("timestamp1")) r1 = r1.withColumn('value', rand()) r2 = spark.range(1000, 1001).select(col("id").alias("timestamp2")) r2 = r2.withColumn('value2', rand()) joined = r1.join(r2, r1.timestamp1 == r2.timestamp2, "inner") joined = joined.coalesce(1) joined.explain() joined.show() ``` Closes #26164 from xuanyuanking/SPARK-21492. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../execution/UnsafeExternalRowSorter.java | 11 +++++- .../apache/spark/sql/execution/SortExec.scala | 27 +++++++++++-- .../spark/sql/execution/SparkPlan.scala | 9 +++++ .../execution/joins/SortMergeJoinExec.scala | 39 +++++++++++++------ .../org/apache/spark/sql/JoinSuite.scala | 33 +++++++++++++++- 5 files changed, 102 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 863d80b5cb9c5..3123f2187da83 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -52,6 +52,12 @@ public final class UnsafeExternalRowSorter { private final UnsafeExternalRowSorter.PrefixComputer prefixComputer; private final UnsafeExternalSorter sorter; + // This flag makes sure the cleanupResource() has been called. After the cleanup work, + // iterator.next should always return false. Downstream operator triggers the resource + // cleanup while they found there's no need to keep the iterator any more. + // See more details in SPARK-21492. + private boolean isReleased = false; + public abstract static class PrefixComputer { public static class Prefix { @@ -157,7 +163,8 @@ public long getSortTimeNanos() { return sorter.getSortTimeNanos(); } - private void cleanupResources() { + public void cleanupResources() { + isReleased = true; sorter.cleanupResources(); } @@ -176,7 +183,7 @@ public Iterator sort() throws IOException { @Override public boolean hasNext() { - return sortedIterator.hasNext(); + return !isReleased && sortedIterator.hasNext(); } @Override diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 0a955d6a75235..32d21d05e5f73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -62,6 +62,14 @@ case class SortExec( "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size")) + private[sql] var rowSorter: UnsafeExternalRowSorter = _ + + /** + * This method gets invoked only once for each SortExec instance to initialize an + * UnsafeExternalRowSorter, both `plan.execute` and code generation are using it. + * In the code generation code path, we need to call this function outside the class so we + * should make it public. + */ def createSorter(): UnsafeExternalRowSorter = { val ordering = newOrdering(sortOrder, output) @@ -87,13 +95,13 @@ case class SortExec( } val pageSize = SparkEnv.get.memoryManager.pageSizeBytes - val sorter = UnsafeExternalRowSorter.create( + rowSorter = UnsafeExternalRowSorter.create( schema, ordering, prefixComparator, prefixComputer, pageSize, canUseRadixSort) if (testSpillFrequency > 0) { - sorter.setTestSpillFrequency(testSpillFrequency) + rowSorter.setTestSpillFrequency(testSpillFrequency) } - sorter + rowSorter } protected override def doExecute(): RDD[InternalRow] = { @@ -181,4 +189,17 @@ case class SortExec( |$sorterVariable.insertRow((UnsafeRow)${row.value}); """.stripMargin } + + /** + * In SortExec, we overwrites cleanupResources to close UnsafeExternalRowSorter. + */ + override protected[sql] def cleanupResources(): Unit = { + if (rowSorter != null) { + // There's possible for rowSorter is null here, for example, in the scenario of empty + // iterator in the current task, the downstream physical node(like SortMergeJoinExec) will + // trigger cleanupResources before rowSorter initialized in createSorter. + rowSorter.cleanupResources() + } + super.cleanupResources() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index b4cdf9e16b7e5..125f76282e3df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -507,6 +507,15 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } newOrdering(order, Seq.empty) } + + /** + * Cleans up the resources used by the physical operator (if any). In general, all the resources + * should be cleaned up when the task finishes but operators like SortMergeJoinExec and LimitExec + * may want eager cleanup to free up tight resources (e.g., memory). + */ + protected[sql] def cleanupResources(): Unit = { + children.foreach(_.cleanupResources()) + } } trait LeafExecNode extends SparkPlan { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 189727a9bc88d..26fb0e5ffb1af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -191,7 +191,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(leftIter), RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) private[this] val joinRow = new JoinedRow @@ -235,7 +236,8 @@ case class SortMergeJoinExec( streamedIter = RowIterator.fromScala(leftIter), bufferedIter = RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) val rightNullRow = new GenericInternalRow(right.output.length) new LeftOuterIterator( @@ -249,7 +251,8 @@ case class SortMergeJoinExec( streamedIter = RowIterator.fromScala(rightIter), bufferedIter = RowIterator.fromScala(leftIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) val leftNullRow = new GenericInternalRow(left.output.length) new RightOuterIterator( @@ -283,7 +286,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(leftIter), RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) private[this] val joinRow = new JoinedRow @@ -318,7 +322,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(leftIter), RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) private[this] val joinRow = new JoinedRow @@ -360,7 +365,8 @@ case class SortMergeJoinExec( RowIterator.fromScala(leftIter), RowIterator.fromScala(rightIter), inMemoryThreshold, - spillThreshold + spillThreshold, + cleanupResources ) private[this] val joinRow = new JoinedRow @@ -640,6 +646,9 @@ case class SortMergeJoinExec( (evaluateVariables(leftVars), "") } + val thisPlan = ctx.addReferenceObj("plan", this) + val eagerCleanup = s"$thisPlan.cleanupResources();" + s""" |while (findNextInnerJoinRows($leftInput, $rightInput)) { | ${leftVarDecl.mkString("\n")} @@ -653,6 +662,7 @@ case class SortMergeJoinExec( | } | if (shouldStop()) return; |} + |$eagerCleanup """.stripMargin } } @@ -678,6 +688,7 @@ case class SortMergeJoinExec( * @param inMemoryThreshold Threshold for number of rows guaranteed to be held in memory by * internal buffer * @param spillThreshold Threshold for number of rows to be spilled by internal buffer + * @param eagerCleanupResources the eager cleanup function to be invoked when no join row found */ private[joins] class SortMergeJoinScanner( streamedKeyGenerator: Projection, @@ -686,7 +697,8 @@ private[joins] class SortMergeJoinScanner( streamedIter: RowIterator, bufferedIter: RowIterator, inMemoryThreshold: Int, - spillThreshold: Int) { + spillThreshold: Int, + eagerCleanupResources: () => Unit) { private[this] var streamedRow: InternalRow = _ private[this] var streamedRowKey: InternalRow = _ private[this] var bufferedRow: InternalRow = _ @@ -710,7 +722,8 @@ private[joins] class SortMergeJoinScanner( def getBufferedMatches: ExternalAppendOnlyUnsafeRowArray = bufferedMatches /** - * Advances both input iterators, stopping when we have found rows with matching join keys. + * Advances both input iterators, stopping when we have found rows with matching join keys. If no + * join rows found, try to do the eager resources cleanup. * @return true if matching rows have been found and false otherwise. If this returns true, then * [[getStreamedRow]] and [[getBufferedMatches]] can be called to construct the join * results. @@ -720,7 +733,7 @@ private[joins] class SortMergeJoinScanner( // Advance the streamed side of the join until we find the next row whose join key contains // no nulls or we hit the end of the streamed iterator. } - if (streamedRow == null) { + val found = if (streamedRow == null) { // We have consumed the entire streamed iterator, so there can be no more matches. matchJoinKey = null bufferedMatches.clear() @@ -760,17 +773,19 @@ private[joins] class SortMergeJoinScanner( true } } + if (!found) eagerCleanupResources() + found } /** * Advances the streamed input iterator and buffers all rows from the buffered input that - * have matching keys. + * have matching keys. If no join rows found, try to do the eager resources cleanup. * @return true if the streamed iterator returned a row, false otherwise. If this returns true, * then [[getStreamedRow]] and [[getBufferedMatches]] can be called to produce the outer * join results. */ final def findNextOuterJoinRows(): Boolean = { - if (!advancedStreamed()) { + val found = if (!advancedStreamed()) { // We have consumed the entire streamed iterator, so there can be no more matches. matchJoinKey = null bufferedMatches.clear() @@ -800,6 +815,8 @@ private[joins] class SortMergeJoinScanner( // If there is a streamed input then we always return true true } + if (!found) eagerCleanupResources() + found } // --- Private methods -------------------------------------------------------------------------- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 72742644ff34e..62f2d21e52704 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -22,12 +22,14 @@ import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer +import org.mockito.Mockito._ + import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.Filter -import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec} +import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf @@ -37,6 +39,23 @@ import org.apache.spark.sql.types.StructType class JoinSuite extends QueryTest with SharedSparkSession { import testImplicits._ + private def attachCleanupResourceChecker(plan: SparkPlan): Unit = { + // SPARK-21492: Check cleanupResources are finally triggered in SortExec node for every + // test case + plan.foreachUp { + case s: SortExec => + val sortExec = spy(s) + verify(sortExec, atLeastOnce).cleanupResources() + verify(sortExec.rowSorter, atLeastOnce).cleanupResources() + case _ => + } + } + + override protected def checkAnswer(df: => DataFrame, rows: Seq[Row]): Unit = { + attachCleanupResourceChecker(df.queryExecution.sparkPlan) + super.checkAnswer(df, rows) + } + setupTestData() def statisticSizeInByte(df: DataFrame): BigInt = { @@ -1039,4 +1058,16 @@ class JoinSuite extends QueryTest with SharedSparkSession { checkAnswer(df, Row(1, 2, 1, 2) :: Nil) } + + test("SPARK-21492: cleanupResource without code generation") { + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "1", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df1 = spark.range(0, 10, 1, 2) + val df2 = spark.range(10).select($"id".as("b1"), (- $"id").as("b2")) + val res = df1.join(df2, $"id" === $"b1" && $"id" === $"b2").select($"b1", $"b2", $"id") + checkAnswer(res, Row(0, 0, 0)) + } + } } From b4844eea1fc0cfb82cfe7e13f22655b9729c3ad4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 22 Oct 2019 19:17:28 +0800 Subject: [PATCH 0390/1387] [SPARK-29517][SQL] TRUNCATE TABLE should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add TruncateTableStatement and make TRUNCATE TABLE go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog TRUNCATE TABLE t // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? yes. When running TRUNCATE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26174 from viirya/SPARK-29517. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 14 ++++++++++++ .../catalyst/plans/logical/statements.scala | 7 ++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 10 +++++++++ .../analysis/ResolveSessionCatalog.scala | 8 ++++++- .../spark/sql/execution/SparkSqlParser.scala | 14 ------------ .../sql/connector/DataSourceV2SQLSuite.scala | 22 +++++++++++++++++++ 7 files changed, 61 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 1839203e3b235..4c93f1fe11978 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -211,7 +211,7 @@ statement | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData - | TRUNCATE TABLE tableIdentifier partitionSpec? #truncateTable + | TRUNCATE TABLE multipartIdentifier partitionSpec? #truncateTable | MSCK REPAIR TABLE multipartIdentifier #repairTable | op=(ADD | LIST) identifier .*? #manageResource | SET ROLE .*? #failNativeCommand diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 8af7cf9ad8008..862903246ed3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2728,4 +2728,18 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitRepairTable(ctx: RepairTableContext): LogicalPlan = withOrigin(ctx) { RepairTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) } + + /** + * Create a [[TruncateTableStatement]] command. + * + * For example: + * {{{ + * TRUNCATE TABLE multi_part_name [PARTITION (partcol1=val1, partcol2=val2 ...)] + * }}} + */ + override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { + TruncateTableStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 72d5cbb7d9045..1a69a6ab33809 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -316,3 +316,10 @@ case class AnalyzeColumnStatement( * A REPAIR TABLE statement, as parsed from SQL */ case class RepairTableStatement(tableName: Seq[String]) extends ParsedStatement + +/** + * A TRUNCATE TABLE statement, as parsed from SQL + */ +case class TruncateTableStatement( + tableName: Seq[String], + partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 0eaf74f655065..0d87d0ce9b0f7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -961,6 +961,16 @@ class DDLParserSuite extends AnalysisTest { RepairTableStatement(Seq("a", "b", "c"))) } + test("TRUNCATE table") { + comparePlans( + parsePlan("TRUNCATE TABLE a.b.c"), + TruncateTableStatement(Seq("a", "b", "c"), None)) + + comparePlans( + parsePlan("TRUNCATE TABLE a.b.c PARTITION(ds='2017-06-10')"), + TruncateTableStatement(Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10")))) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 72f539f72008d..978214778a4ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand, TruncateTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -282,6 +282,12 @@ class ResolveSessionCatalog( AlterTableRecoverPartitionsCommand( v1TableName.asTableIdentifier, "MSCK REPAIR TABLE") + + case TruncateTableStatement(tableName, partitionSpec) => + val v1TableName = parseV1Table(tableName, "TRUNCATE TABLE") + TruncateTableCommand( + v1TableName.asTableIdentifier, + partitionSpec) } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index cdee117813249..a51d29431dec7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -346,20 +346,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ) } - /** - * Create a [[TruncateTableCommand]] command. - * - * For example: - * {{{ - * TRUNCATE TABLE tablename [PARTITION (partcol1=val1, partcol2=val2 ...)] - * }}} - */ - override def visitTruncateTable(ctx: TruncateTableContext): LogicalPlan = withOrigin(ctx) { - TruncateTableCommand( - visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) - } - /** * Create a [[CreateDatabaseCommand]] command. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index d253e6078ddc0..01c051f156356 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1210,6 +1210,28 @@ class DataSourceV2SQLSuite } } + test("TRUNCATE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql( + s""" + |CREATE TABLE $t (id bigint, data string) + |USING foo + |PARTITIONED BY (id) + """.stripMargin) + + val e1 = intercept[AnalysisException] { + sql(s"TRUNCATE TABLE $t") + } + assert(e1.message.contains("TRUNCATE TABLE is only supported with v1 tables")) + + val e2 = intercept[AnalysisException] { + sql(s"TRUNCATE TABLE $t PARTITION(id='1')") + } + assert(e2.message.contains("TRUNCATE TABLE is only supported with v1 tables")) + } + } + private def assertAnalysisError(sqlStatement: String, expectedError: String): Unit = { val errMsg = intercept[AnalysisException] { sql(sqlStatement) From 877993847c0baa016003639e16708373e57ca64b Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 22 Oct 2019 08:55:37 -0500 Subject: [PATCH 0391/1387] [SPARK-28787][DOC][SQL] Document LOAD DATA statement in SQL Reference ### What changes were proposed in this pull request? Document LOAD DATA statement in SQL Reference ### Why are the changes needed? To complete the SQL Reference ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Tested using jykyll build --serve Here are the screen shots: ![image](https://user-images.githubusercontent.com/13592258/64073167-e7cd0800-cc4e-11e9-9fcc-92fe4cb5a942.png) ![image](https://user-images.githubusercontent.com/13592258/64073169-ee5b7f80-cc4e-11e9-9a36-cc023bcd32b1.png) ![image](https://user-images.githubusercontent.com/13592258/64073170-f4516080-cc4e-11e9-9101-2609a01fe6fe.png) Closes #25522 from huaxingao/spark-28787. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- docs/sql-ref-syntax-dml-load.md | 103 +++++++++++++++++++++++++++++++- 1 file changed, 100 insertions(+), 3 deletions(-) diff --git a/docs/sql-ref-syntax-dml-load.md b/docs/sql-ref-syntax-dml-load.md index fd25ba314e0b6..c2a6102db4aad 100644 --- a/docs/sql-ref-syntax-dml-load.md +++ b/docs/sql-ref-syntax-dml-load.md @@ -1,7 +1,7 @@ --- layout: global -title: LOAD -displayTitle: LOAD +title: LOAD DATA +displayTitle: LOAD DATA license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,4 +19,101 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`LOAD DATA` statement loads the data into a table from the user specified directory or file. If a directory is specified then all the files from the directory are loaded. If a file is specified then only the single file is loaded. Additionally the `LOAD DATA` statement takes an optional partition specification. When a partition is specified, the data files (when input source is a directory) or the single file (when input source is a file) are loaded into the partition of the target table. + +### Syntax +{% highlight sql %} +LOAD DATA [ LOCAL ] INPATH path [ OVERWRITE ] INTO TABLE table_name + [ PARTITION ( partition_col_name = partition_col_val [ , ... ] ) ] +{% endhighlight %} + +### Parameters +
      +
      path
      +
      Path of the file system. It can be either an absolute or a relative path.
      +
      + +
      +
      table_name
      +
      The name of an existing table.
      +
      + +
      +
      PARTITION ( partition_col_name = partition_col_val [ , ... ] )
      +
      Specifies one or more partition column and value pairs.
      +
      + +
      +
      LOCAL
      +
      If specified, it causes the INPATH to be resolved against the local file system, instead of the default file system, which is typically a distributed storage.
      +
      + +
      +
      OVERWRITE
      +
      By default, new data is appended to the table. If OVERWRITE is used, the table is instead overwritten with new data.
      +
      + +### Examples +{% highlight sql %} + -- Example without partition specification. + -- Assuming the students table has already been created and populated. + SELECT * FROM students; + + + -------------- + ------------------------------ + -------------- + + | name | address | student_id | + + -------------- + ------------------------------ + -------------- + + | Amy Smith | 123 Park Ave, San Jose | 111111 | + + -------------- + ------------------------------ + -------------- + + + CREATE TABLE test_load (name VARCHAR(64), address VARCHAR(64), student_id INT); + + -- Assuming the students table is in '/user/hive/warehouse/' + LOAD DATA LOCAL INPATH '/user/hive/warehouse/students' OVERWRITE INTO TABLE test_load; + + SELECT * FROM test_load; + + + -------------- + ------------------------------ + -------------- + + | name | address | student_id | + + -------------- + ------------------------------ + -------------- + + | Amy Smith | 123 Park Ave, San Jose | 111111 | + + -------------- + ------------------------------ + -------------- + + + -- Example with partition specification. + CREATE TABLE test_partition (c1 INT, c2 INT, c3 INT) USING HIVE PARTITIONED BY (c2, c3); + + INSERT INTO test_partition PARTITION (c2 = 2, c3 = 3) VALUES (1); + + INSERT INTO test_partition PARTITION (c2 = 5, c3 = 6) VALUES (4); + + INSERT INTO test_partition PARTITION (c2 = 8, c3 = 9) VALUES (7); + + SELECT * FROM test_partition; + + + ------- + ------- + ----- + + | c1 | c2 | c3 | + + ------- + --------------- + + | 1 | 2 | 3 | + + ------- + ------- + ----- + + | 4 | 5 | 6 | + + ------- + ------- + ----- + + | 7 | 8 | 9 | + + ------- + ------- + ----- + + + CREATE TABLE test_load_partition (c1 INT, c2 INT, c3 INT) USING HIVE PARTITIONED BY (c2, c3); + + -- Assuming the test_partition table is in '/user/hive/warehouse/' + LOAD DATA LOCAL INPATH '/user/hive/warehouse/test_partition/c2=2/c3=3' + OVERWRITE INTO TABLE test_load_partition PARTITION (c2=2, c3=3); + + SELECT * FROM test_load_partition; + + + ------- + ------- + ----- + + | c1 | c2 | c3 | + + ------- + --------------- + + | 1 | 2 | 3 | + + ------- + ------- + ----- + + + +{% endhighlight %} + From c1c64851ed4b8dac3ca4becaea9e6721eb25c589 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Tue, 22 Oct 2019 08:56:44 -0500 Subject: [PATCH 0392/1387] [SPARK-28793][DOC][SQL] Document CREATE FUNCTION in SQL Reference ### What changes were proposed in this pull request? Document CREATE FUNCTION statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2019-09-22 at 3 01 52 PM Screen Shot 2019-09-22 at 3 02 11 PM Screen Shot 2019-09-22 at 3 02 39 PM Screen Shot 2019-09-22 at 3 04 04 PM ### How was this patch tested? Tested using jykyll build --serve Closes #25894 from dilipbiswal/sql-ref-create-function. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/sql-getting-started.md | 3 + docs/sql-ref-syntax-ddl-create-function.md | 151 ++++++++++++++++++++- 2 files changed, 153 insertions(+), 1 deletion(-) diff --git a/docs/sql-getting-started.md b/docs/sql-getting-started.md index 5d18c48879f93..0ded2654719c1 100644 --- a/docs/sql-getting-started.md +++ b/docs/sql-getting-started.md @@ -346,6 +346,9 @@ For example:
    +## Scalar Functions +(to be filled soon) + ## Aggregations The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common diff --git a/docs/sql-ref-syntax-ddl-create-function.md b/docs/sql-ref-syntax-ddl-create-function.md index f95a9eba42c2f..4c09ebafb1f5d 100644 --- a/docs/sql-ref-syntax-ddl-create-function.md +++ b/docs/sql-ref-syntax-ddl-create-function.md @@ -19,4 +19,153 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +The `CREATE FUNCTION` statement is used to create a temporary or permanent function +in Spark. Temporary functions are scoped at a session level where as permanent +functions are created in the persistent catalog and are made available to +all sessions. The resources specified in the `USING` clause are made available +to all executors when they are executed for the first time. In addition to the +SQL interface, spark allows users to create custom user defined scalar and +aggregate functions using Scala, Python and Java APIs. Please refer to +[scalar_functions](sql-getting-started.html#scalar-functions) and +[aggregate functions](sql-getting-started#aggregations) for more information. + +### Syntax +{% highlight sql %} +CREATE [ OR REPLACE ] [ TEMPORARY ] FUNCTION [ IF NOT EXISTS ] + function_name AS class_name [ resource_locations ] +{% endhighlight %} + +### Parameters +
    +
    OR REPLACE
    +
    + If specified, the resources for the function are reloaded. This is mainly useful + to pick up any changes made to the implementation of the function. This + parameter is mutually exclusive to IF NOT EXISTS and can not + be specified together. +
    +
    TEMPORARY
    +
    + Indicates the scope of function being created. When TEMPORARY is specified, the + created function is valid and visible in the current session. No persistent + entry is made in the catalog for these kind of functions. +
    +
    IF NOT EXISTS
    +
    + If specified, creates the function only when it does not exist. The creation + of function succeeds (no error is thrown) if the specified function already + exists in the system. This parameter is mutually exclusive to OR REPLACE + and can not be specified together. +
    +
    function_name
    +
    + Specifies a name of funnction to be created. The function name may be + optionally qualified with a database name.

    + Syntax: + + [database_name.]function_name + +
    +
    class_name
    +
    + Specifies the name of the class that provides the implementation for function to be created. + The implementing class should extend one of the base classes as follows: +
      +
    • Should extend UDF or UDAF in org.apache.hadoop.hive.ql.exec package.
    • +
    • Should extend AbstractGenericUDAFResolver, GenericUDF, or + GenericUDTF in org.apache.hadoop.hive.ql.udf.generic package.
    • +
    • Should extend UserDefinedAggregateFunction in org.apache.spark.sql.expressions package.
    • +
    +
    +
    resource_locations
    +
    + Specifies the list of resources that contain the implementation of the function + along with its dependencies.

    + Syntax: + + USING { { (JAR | FILE ) resource_uri} , ...} + +
    +
    + +### Examples +{% highlight sql %} +-- 1. Create a simple UDF `SimpleUdf` that increments the supplied integral value by 10. +-- import org.apache.hadoop.hive.ql.exec.UDF; +-- public class SimpleUdf extends UDF { +-- public int evaluate(int value) { +-- return value + 10; +-- } +-- } +-- 2. Compile and place it in a JAR file called `SimpleUdf.jar` in /tmp. + +-- Create a table called `test` and insert two rows. +CREATE TABLE test(c1 INT); +INSERT INTO test VALUES (1), (2); + +-- Create a permanent function called `simple_udf`. +CREATE FUNCTION simple_udf AS 'SimpleUdf' + USING JAR '/tmp/SimpleUdf.jar'; + +-- Verify that the function is in the registry. +SHOW USER FUNCTIONS; + +------------------+ + | function| + +------------------+ + |default.simple_udf| + +------------------+ + +-- Invoke the function. Every selected value should be incremented by 10. +SELECT simple_udf(c1) AS function_return_value FROM t1; + +---------------------+ + |function_return_value| + +---------------------+ + | 11| + | 12| + +---------------------+ + +-- Created a temporary function. +CREATE TEMPORARY FUNCTION simple_temp_udf AS 'SimpleUdf' + USING JAR '/tmp/SimpleUdf.jar'; + +-- Verify that the newly created temporary function is in the registry. +-- Please note that the temporary function does not have a qualified +-- database associated with it. +SHOW USER FUNCTIONS; + +------------------+ + | function| + +------------------+ + |default.simple_udf| + | simple_temp_udf| + +------------------+ + +-- 1. Modify `SimpleUdf`'s implementation to add supplied integral value by 20. +-- import org.apache.hadoop.hive.ql.exec.UDF; + +-- public class SimpleUdfR extends UDF { +-- public int evaluate(int value) { +-- return value + 20; +-- } +-- } +-- 2. Compile and place it in a jar file called `SimpleUdfR.jar` in /tmp. + +-- Replace the implementation of `simple_udf` +CREATE OR REPLACE FUNCTION simple_udf AS 'SimpleUdfR' + USING JAR '/tmp/SimpleUdfR.jar'; + +-- Invoke the function. Every selected value should be incremented by 20. +SELECT simple_udf(c1) AS function_return_value FROM t1; ++---------------------+ +|function_return_value| ++---------------------+ +| 21| +| 22| ++---------------------+ + +{% endhighlight %} + +### Related statements +- [SHOW FUNCTIONS](sql-ref-syntax-aux-show-functions.html) +- [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) +- [DROP FUNCTION](sql-ref-syntax-ddl-drop-function.html) From 2036a8cca7a428672310ae11e71d0f1f51074cac Mon Sep 17 00:00:00 2001 From: chenjuanni Date: Tue, 22 Oct 2019 08:58:12 -0500 Subject: [PATCH 0393/1387] [SPARK-29488][WEBUI] In Web UI, stage page has js error when sort table MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? In Web UI, stage page has js error when sort table. https://issues.apache.org/jira/browse/SPARK-29488 ### Why are the changes needed? In Web UI, follow the steps below, get js error "Uncaught TypeError: Failed to execute 'removeChild' on 'Node': parameter 1 is not of type 'Node'.". 1) Click "Summary Metrics..." 's tablehead "Min" 2) Click "Aggregated Metrics by Executor" 's tablehead "Task Time" 3) Click "Summary Metrics..." 's tablehead "Min"(the same as step 1.) ### Does this PR introduce any user-facing change? No. ### How was this patch tested? In Web UI, follow the steps below, no error occur. 1) Click "Summary Metrics..." 's tablehead "Min" 2) Click "Aggregated Metrics by Executor" 's tablehead "Task Time" 3) Click "Summary Metrics..." 's tablehead "Min"(the same as step 1.) ![image](https://user-images.githubusercontent.com/7802338/66899878-464b1b80-f02e-11e9-9660-6cdaab283491.png) Closes #26136 from cjn082030/SPARK-1. Authored-by: chenjuanni Signed-off-by: Sean Owen --- .../org/apache/spark/ui/static/sorttable.js | 31 +++++++++++++------ 1 file changed, 22 insertions(+), 9 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js index 9960d5c34d1fc..ecd580e5c64aa 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js +++ b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js @@ -97,9 +97,14 @@ sorttable = { sorttable.reverse(this.sorttable_tbody); this.className = this.className.replace('sorttable_sorted', 'sorttable_sorted_reverse'); - this.removeChild(document.getElementById('sorttable_sortfwdind')); + rowlists = this.parentNode.getElementsByTagName("span"); + for (var j=0; j < rowlists.length; j++) { + if (rowlists[j].className.search(/\bsorttable_sortfwdind\b/)) { + rowlists[j].parentNode.removeChild(rowlists[j]); + } + } sortrevind = document.createElement('span'); - sortrevind.id = "sorttable_sortrevind"; + sortrevind.class = "sorttable_sortrevind"; sortrevind.innerHTML = stIsIE ? ' 5' : ' ▾'; this.appendChild(sortrevind); return; @@ -110,9 +115,14 @@ sorttable = { sorttable.reverse(this.sorttable_tbody); this.className = this.className.replace('sorttable_sorted_reverse', 'sorttable_sorted'); - this.removeChild(document.getElementById('sorttable_sortrevind')); + rowlists = this.parentNode.getElementsByTagName("span"); + for (var j=0; j < rowlists.length; j++) { + if (rowlists[j].className.search(/\sorttable_sortrevind\b/)) { + rowlists[j].parentNode.removeChild(rowlists[j]); + } + } sortfwdind = document.createElement('span'); - sortfwdind.id = "sorttable_sortfwdind"; + sortfwdind.class = "sorttable_sortfwdind"; sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▴'; this.appendChild(sortfwdind); return; @@ -126,14 +136,17 @@ sorttable = { cell.className = cell.className.replace('sorttable_sorted',''); } }); - sortfwdind = document.getElementById('sorttable_sortfwdind'); - if (sortfwdind) { sortfwdind.parentNode.removeChild(sortfwdind); } - sortrevind = document.getElementById('sorttable_sortrevind'); - if (sortrevind) { sortrevind.parentNode.removeChild(sortrevind); } + rowlists = this.parentNode.getElementsByTagName("span"); + for (var j=0; j < rowlists.length; j++) { + if (rowlists[j].className.search(/\bsorttable_sortfwdind\b/) + || rowlists[j].className.search(/\sorttable_sortrevind\b/) ) { + rowlists[j].parentNode.removeChild(rowlists[j]); + } + } this.className += ' sorttable_sorted'; sortfwdind = document.createElement('span'); - sortfwdind.id = "sorttable_sortfwdind"; + sortfwdind.class = "sorttable_sortfwdind"; sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▴'; this.appendChild(sortfwdind); From 80094688fdbf5cc0d10c295fadb92965b460de5d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 22 Oct 2019 14:14:59 -0700 Subject: [PATCH 0394/1387] [SPARK-29556][CORE] Avoid putting request path in error response in ErrorServlet ### What changes were proposed in this pull request? Don't include `$path` from user query in the error response. ### Why are the changes needed? The path could contain input that is then rendered as HTML in the error response. It's not clear whether it's exploitable, but better safe than sorry as the path info really isn't that important in this context. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #26211 from srowen/SPARK-29556. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/deploy/rest/RestSubmissionServer.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index e59bf3f0eaf44..f60d940b8c82a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -317,8 +317,7 @@ private class ErrorServlet extends RestServlet { versionMismatch = true s"Unknown protocol version '$unknownVersion'." case _ => - // never reached - s"Malformed path $path." + "Malformed path." } msg += s" Please submit requests through http://[host]:[port]/$serverVersion/submissions/..." val error = handleError(msg) From 3bf5355e24094153db5cac4d34bf5ccead31772a Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 22 Oct 2019 14:47:17 -0700 Subject: [PATCH 0395/1387] [SPARK-29539][SQL] SHOW PARTITIONS should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add ShowPartitionsStatement and make SHOW PARTITIONS go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. ### Does this PR introduce any user-facing change? Yes. When running SHOW PARTITIONS, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26198 from huaxingao/spark-29539. Authored-by: Huaxin Gao Signed-off-by: Liang-Chi Hsieh --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 17 ++++++++++++ .../catalyst/plans/logical/statements.scala | 6 +++++ .../sql/catalyst/parser/DDLParserSuite.scala | 26 +++++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 8 +++++- .../spark/sql/execution/SparkSqlParser.scala | 17 ------------ .../sql/connector/DataSourceV2SQLSuite.scala | 22 ++++++++++++++++ .../execution/command/DDLParserSuite.scala | 21 --------------- 8 files changed, 79 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 4c93f1fe11978..963077c35df92 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -194,7 +194,7 @@ statement ('(' key=tablePropertyKey ')')? #showTblProperties | SHOW COLUMNS (FROM | IN) tableIdentifier ((FROM | IN) db=errorCapturingIdentifier)? #showColumns - | SHOW PARTITIONS tableIdentifier partitionSpec? #showPartitions + | SHOW PARTITIONS multipartIdentifier partitionSpec? #showPartitions | SHOW identifier? FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))? #showFunctions | SHOW CREATE TABLE tableIdentifier #showCreateTable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 862903246ed3b..548042bc97674 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2742,4 +2742,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging visitMultipartIdentifier(ctx.multipartIdentifier), Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } + + /** + * A command for users to list the partition names of a table. If partition spec is specified, + * partitions that match the spec are returned. Otherwise an empty result set is returned. + * + * This function creates a [[ShowPartitionsStatement]] logical plan + * + * The syntax of using this command in SQL is: + * {{{ + * SHOW PARTITIONS multi_part_name [partition_spec]; + * }}} + */ + override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) { + val table = visitMultipartIdentifier(ctx.multipartIdentifier) + val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) + ShowPartitionsStatement(table, partitionKeys) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 1a69a6ab33809..a73a2975aa9ca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -323,3 +323,9 @@ case class RepairTableStatement(tableName: Seq[String]) extends ParsedStatement case class TruncateTableStatement( tableName: Seq[String], partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement + +/** + * A SHOW PARTITIONS statement, as parsed from SQL + */ +case class ShowPartitionsStatement(tableName: Seq[String], + partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 0d87d0ce9b0f7..1dacb2384ac13 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -971,6 +971,32 @@ class DDLParserSuite extends AnalysisTest { TruncateTableStatement(Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10")))) } + test("SHOW PARTITIONS") { + val sql1 = "SHOW PARTITIONS t1" + val sql2 = "SHOW PARTITIONS db1.t1" + val sql3 = "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" + val sql4 = "SHOW PARTITIONS a.b.c" + val sql5 = "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" + + val parsed1 = parsePlan(sql1) + val expected1 = ShowPartitionsStatement(Seq("t1"), None) + val parsed2 = parsePlan(sql2) + val expected2 = ShowPartitionsStatement(Seq("db1", "t1"), None) + val parsed3 = parsePlan(sql3) + val expected3 = ShowPartitionsStatement(Seq("t1"), + Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue"))) + val parsed4 = parsePlan(sql4) + val expected4 = ShowPartitionsStatement(Seq("a", "b", "c"), None) + val parsed5 = parsePlan(sql5) + val expected5 = ShowPartitionsStatement(Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10"))) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 978214778a4ad..4a2e6731d9d82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand, TruncateTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -288,6 +288,12 @@ class ResolveSessionCatalog( TruncateTableCommand( v1TableName.asTableIdentifier, partitionSpec) + + case ShowPartitionsStatement(tableName, partitionSpec) => + val v1TableName = parseV1Table(tableName, "SHOW PARTITIONS") + ShowPartitionsCommand( + v1TableName.asTableIdentifier, + partitionSpec) } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index a51d29431dec7..3f3f6b373eb0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -135,23 +135,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ShowColumnsCommand(Option(ctx.db).map(_.getText), visitTableIdentifier(ctx.tableIdentifier)) } - /** - * A command for users to list the partition names of a table. If partition spec is specified, - * partitions that match the spec are returned. Otherwise an empty result set is returned. - * - * This function creates a [[ShowPartitionsCommand]] logical plan - * - * The syntax of using this command in SQL is: - * {{{ - * SHOW PARTITIONS table_identifier [partition_spec]; - * }}} - */ - override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) { - val table = visitTableIdentifier(ctx.tableIdentifier) - val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) - ShowPartitionsCommand(table, partitionKeys) - } - /** * Creates a [[ShowCreateTableCommand]] */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 01c051f156356..39709ab426a09 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1232,6 +1232,28 @@ class DataSourceV2SQLSuite } } + test("SHOW PARTITIONS") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql( + s""" + |CREATE TABLE $t (id bigint, data string) + |USING foo + |PARTITIONED BY (id) + """.stripMargin) + + val e1 = intercept[AnalysisException] { + val partition = sql(s"SHOW PARTITIONS $t") + } + assert(e1.message.contains("SHOW PARTITIONS is only supported with v1 tables")) + + val e2 = intercept[AnalysisException] { + val partition2 = sql(s"SHOW PARTITIONS $t PARTITION(id='1')") + } + assert(e2.message.contains("SHOW PARTITIONS is only supported with v1 tables")) + } + } + private def assertAnalysisError(sqlStatement: String, expectedError: String): Unit = { val errMsg = intercept[AnalysisException] { sql(sqlStatement) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 5a5899cbccc55..0640d0540baa6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -870,27 +870,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { comparePlans(parsed4, expected4) } - - test("show partitions") { - val sql1 = "SHOW PARTITIONS t1" - val sql2 = "SHOW PARTITIONS db1.t1" - val sql3 = "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" - - val parsed1 = parser.parsePlan(sql1) - val expected1 = - ShowPartitionsCommand(TableIdentifier("t1", None), None) - val parsed2 = parser.parsePlan(sql2) - val expected2 = - ShowPartitionsCommand(TableIdentifier("t1", Some("db1")), None) - val expected3 = - ShowPartitionsCommand(TableIdentifier("t1", None), - Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue"))) - val parsed3 = parser.parsePlan(sql3) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - comparePlans(parsed3, expected3) - } - test("support for other types in DBPROPERTIES") { val sql = """ From f23c5d7f6705348ddeac0b714b29374cba3a4efe Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 22 Oct 2019 16:30:29 -0700 Subject: [PATCH 0396/1387] [SPARK-29560][BUILD] Add typesafe bintray repo for sbt-mima-plugin ### What changes were proposed in this pull request? This add `typesafe` bintray repo for `sbt-mima-plugin`. ### Why are the changes needed? Since Oct 21, the following plugin causes [Jenkins failures](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-2.4-test-sbt-hadoop-2.6/611/console ) due to the missing jar. - `branch-2.4`: `sbt-mima-plugin:0.1.17` is missing. - `master`: `sbt-mima-plugin:0.3.0` is missing. These versions of `sbt-mima-plugin` seems to be removed from the old repo. ``` $ rm -rf ~/.ivy2/ $ build/sbt scalastyle test:scalastyle ... [warn] :::::::::::::::::::::::::::::::::::::::::::::: [warn] :: UNRESOLVED DEPENDENCIES :: [warn] :::::::::::::::::::::::::::::::::::::::::::::: [warn] :: com.typesafe#sbt-mima-plugin;0.1.17: not found [warn] :::::::::::::::::::::::::::::::::::::::::::::: ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Check `GitHub Action` linter result. This PR should pass. Or, manual check. (Note that Jenkins PR builder didn't fail until now due to the local cache.) Closes #26217 from dongjoon-hyun/SPARK-29560. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- project/plugins.sbt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/project/plugins.sbt b/project/plugins.sbt index d1fe59a47217c..02525c27b6aac 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -32,6 +32,9 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.9.2") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "1.0.0") +// SPARK-29560 Only sbt-mima-plugin needs this repo +resolvers += Resolver.url("bintray", + new java.net.URL("https://dl.bintray.com/typesafe/sbt-plugins"))(Resolver.defaultIvyPatterns) addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.3.0") // sbt 1.0.0 support: https://github.com/AlpineNow/junit_xml_listener/issues/6 From e6749092f7a2cc1943899fde8d830ec2b8fa2186 Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Wed, 23 Oct 2019 10:24:38 +0900 Subject: [PATCH 0397/1387] [SPARK-29107][SQL][TESTS] Port window.sql (Part 1) ### What changes were proposed in this pull request? This PR ports window.sql from PostgreSQL regression tests https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql from lines 1~319 The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/expected/window.out ### Why are the changes needed? To ensure compatibility with PostgreSQL. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass the Jenkins. And, Comparison with PgSQL results. Closes #26119 from DylanGuedes/spark-29107. Authored-by: DylanGuedes Signed-off-by: HyukjinKwon --- .../inputs/postgreSQL/window_part1.sql | 352 +++++++++ .../results/postgreSQL/window_part1.sql.out | 725 ++++++++++++++++++ 2 files changed, 1077 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql new file mode 100644 index 0000000000000..ae2a015ada245 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql @@ -0,0 +1,352 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L1-L319 + +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- CREATE TABLE empsalary ( +-- depname string, +-- empno integer, +-- salary int, +-- enroll_date date +-- ) USING parquet; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- INSERT INTO empsalary VALUES ('develop', 10, 5200, '2007-08-01'); +-- INSERT INTO empsalary VALUES ('sales', 1, 5000, '2006-10-01'); +-- INSERT INTO empsalary VALUES ('personnel', 5, 3500, '2007-12-10'); +-- INSERT INTO empsalary VALUES ('sales', 4, 4800, '2007-08-08'); +-- INSERT INTO empsalary VALUES ('personnel', 2, 3900, '2006-12-23'); +-- INSERT INTO empsalary VALUES ('develop', 7, 4200, '2008-01-01'); +-- INSERT INTO empsalary VALUES ('develop', 9, 4500, '2008-01-01'); +-- INSERT INTO empsalary VALUES ('sales', 3, 4800, '2007-08-01'); +-- INSERT INTO empsalary VALUES ('develop', 8, 6000, '2006-10-01'); +-- INSERT INTO empsalary VALUES ('develop', 11, 5200, '2007-08-15'); + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname) FROM empsalary ORDER BY depname, salary; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, rank() OVER (PARTITION BY depname ORDER BY salary) FROM empsalary; + +-- with GROUP BY +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, sum(salary) OVER w FROM empsalary WINDOW w AS (PARTITION BY depname); + +-- [SPARK-28064] Order by does not accept a call to rank() +-- SELECT depname, empno, salary, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary) ORDER BY rank() OVER w; + +-- empty window specification +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10; + +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS (); + +-- no window operation +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten); + +-- cumulative aggregate +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10; + +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28065] ntile does not accept NULL as input +-- SELECT ntile(NULL) OVER (ORDER BY ten, four), ten, four FROM tenk1 LIMIT 2; + +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28068] `lag` second argument must be a literal in Spark +-- SELECT lag(ten, four) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28068] `lag` second argument must be a literal in Spark +-- SELECT lag(ten, four, 0) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- last returns the last row of the frame, which is CURRENT ROW in ORDER BY window. +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- SELECT nth_value(ten, four + 1) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s; + +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two; + +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10; + +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10; + +-- opexpr with different windows evaluation. +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum; + +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10; + +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten); + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- more than one window with GROUP BY +-- SELECT sum(salary), +-- row_number() OVER (ORDER BY depname), +-- sum(sum(salary)) OVER (ORDER BY depname DESC) +-- FROM empsalary GROUP BY depname; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- identical windows with different names +-- SELECT sum(salary) OVER w1, count(*) OVER w2 +-- FROM empsalary WINDOW w1 AS (ORDER BY salary), w2 AS (ORDER BY salary); + +-- subplan +-- [SPARK-28379] Correlated scalar subqueries must be aggregated +-- SELECT lead(ten, (SELECT two FROM tenk1 WHERE s.unique2 = unique2)) OVER (PARTITION BY four ORDER BY ten) +-- FROM tenk1 s WHERE unique2 < 10; + +-- empty table +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- mixture of agg/wfunc in the same window +-- SELECT sum(salary) OVER w, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary DESC); + +-- Cannot safely cast 'enroll_date': StringType to DateType; +-- SELECT empno, depname, salary, bonus, depadj, MIN(bonus) OVER (ORDER BY empno), MAX(depadj) OVER () FROM( +-- SELECT *, +-- CASE WHEN enroll_date < '2008-01-01' THEN 2008 - extract(year FROM enroll_date) END * 500 AS bonus, +-- CASE WHEN +-- AVG(salary) OVER (PARTITION BY depname) < salary +-- THEN 200 END AS depadj FROM empsalary +-- )s; + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- window function over ungrouped agg over empty row set (bug before 9.1) +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42; + +-- window function with ORDER BY an expression involving aggregates (9.1 bug) +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten; + +-- window and aggregate with GROUP BY expression (9.2 bug) +-- explain +-- select first(max(x)) over (), y +-- from (select unique1 as x, ten+four as y from tenk1) ss +-- group by y; + +-- test non-default frame specifications +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss; + +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss; + +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss; + +-- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift +-- SELECT four, ten/4 as two, +-- sum(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row), +-- last(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row) +-- FROM (select distinct ten, four from tenk1) ss; + +-- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift +-- SELECT four, ten/4 as two, +-- sum(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row), +-- last(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row) +-- FROM (select distinct ten, four from tenk1) ss; + +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude no others), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between current row and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- SELECT first_value(unique1) over w, +-- nth_value(unique1, 2) over w AS nth_2, +-- last_value(unique1) over w, unique1, four +-- FROM tenk1 WHERE unique1 < 10 +-- WINDOW w AS (order by four range between current row and unbounded following); + +-- [SPARK-28501] Frame bound value must be a literal. +-- SELECT sum(unique1) over +-- (order by unique1 +-- rows (SELECT unique1 FROM tenk1 ORDER BY unique1 LIMIT 1) + 1 PRECEDING), +-- unique1 +-- FROM tenk1 WHERE unique1 < 10; + +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i; + +SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude current row) as sum_rows FROM range(1, 10) i; + +-- SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude group) as sum_rows FROM range(1, 10) i; +-- SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude ties) as sum_rows FROM generate_series(1, 10) i; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude no others) as sum_rows FROM generate_series(1, 10) i; +-- SELECT * FROM v_window; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i.id, sum(i.id) over (order by i.id groups between 1 preceding and 1 following) as sum_rows FROM range(1, 11) i; +-- SELECT * FROM v_window; + +DROP VIEW v_window; +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- DROP TABLE empsalary; +DROP VIEW tenk2; +DROP VIEW int4_tbl; diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out new file mode 100644 index 0000000000000..45bc98ae97640 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out @@ -0,0 +1,725 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 43 + + +-- !query 0 +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten +-- !query 1 schema +struct +-- !query 1 output +0 0 0 0.0 +0 2 0 2.0 +0 4 0 4.0 +0 6 0 6.0 +0 8 0 8.0 +1 1 2500 1.0 +1 3 2500 3.0 +1 5 2500 5.0 +1 7 2500 7.0 +1 9 2500 9.0 +2 0 5000 0.0 +2 2 5000 2.0 +2 4 5000 4.0 +2 6 5000 6.0 +2 8 5000 8.0 +3 1 7500 1.0 +3 3 7500 3.0 +3 5 7500 5.0 +3 7 7500 7.0 +3 9 7500 9.0 + + +-- !query 2 +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10 +-- !query 2 schema +struct +-- !query 2 output +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 + + +-- !query 3 +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS () +-- !query 3 schema +struct +-- !query 3 output +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 + + +-- !query 4 +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten) +-- !query 4 schema +struct +-- !query 4 output + + + +-- !query 5 +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 5 schema +struct +-- !query 5 output +0 0 0 +0 0 0 +0 4 0 +1 7 1 +1 9 1 +2 0 2 +3 1 3 +3 3 3 +4 1 1 +5 1 1 + + +-- !query 6 +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10 +-- !query 6 schema +struct +-- !query 6 output +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 7 +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 7 schema +struct +-- !query 7 output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +1 1 1 +1 1 3 +2 3 3 +3 4 0 +3 7 1 +4 9 1 + + +-- !query 8 +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 8 schema +struct +-- !query 8 output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +1 1 1 +1 1 3 +2 3 3 +2 4 0 +2 7 1 +3 9 1 + + +-- !query 9 +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 9 schema +struct +-- !query 9 output +0.0 0 0 +0.0 0 0 +0.0 0 2 +0.0 1 1 +0.0 1 1 +0.0 1 3 +0.6666666666666666 7 1 +1.0 3 3 +1.0 4 0 +1.0 9 1 + + +-- !query 10 +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 10 schema +struct +-- !query 10 output +0.5 1 1 +0.5 1 1 +0.5 1 3 +0.6666666666666666 0 0 +0.6666666666666666 0 0 +0.75 7 1 +1.0 0 2 +1.0 3 3 +1.0 4 0 +1.0 9 1 + + +-- !query 11 +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 11 schema +struct +-- !query 11 output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +2 1 1 +2 1 3 +2 3 3 +3 4 0 +3 7 1 +3 9 1 + + +-- !query 12 +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 12 schema +struct +-- !query 12 output +0 0 0 +0 4 0 +1 1 1 +1 3 3 +1 7 1 +7 9 1 +NULL 0 0 +NULL 0 2 +NULL 1 1 +NULL 1 3 + + +-- !query 13 +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 13 schema +struct +-- !query 13 output +0 0 0 +1 1 1 +3 1 3 +4 0 0 +7 1 1 +9 7 1 +NULL 0 2 +NULL 3 3 +NULL 4 0 +NULL 9 1 + + +-- !query 14 +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 14 schema +struct +-- !query 14 output +0 0 0 +14 1 1 +18 7 1 +2 1 1 +6 1 3 +8 0 0 +NULL 0 2 +NULL 3 3 +NULL 4 0 +NULL 9 1 + + +-- !query 15 +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 15 schema +struct +-- !query 15 output +-1 0 2 +-1 3 3 +-1 4 0 +-1 9 1 +0 0 0 +14 1 1 +18 7 1 +2 1 1 +6 1 3 +8 0 0 + + +-- !query 16 +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 16 schema +struct +-- !query 16 output +0 0 0 +0 0 0 +0 0 2 +0 4 0 +1 1 1 +1 1 1 +1 1 3 +1 3 3 +1 7 1 +1 9 1 + + +-- !query 17 +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query 17 schema +struct +-- !query 17 output +0 4 0 +1 1 1 +1 1 1 +1 1 3 +1 7 1 +1 9 1 +2 0 0 +2 0 0 +2 0 2 +3 3 3 + + +-- !query 18 +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten +-- !query 18 schema +struct +-- !query 18 output +4 0 0 +4 0 0 +4 4 0 +9 1 1 +9 1 1 +9 7 1 +9 9 1 +0 0 2 +3 1 3 +3 3 3 + + +-- !query 19 +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two +-- !query 19 schema +struct +-- !query 19 output +0 0 45000 45000 +1 1 46000 46000 +2 0 47000 92000 +3 1 48000 94000 +4 0 49000 141000 +5 1 50000 144000 +6 0 51000 192000 +7 1 52000 196000 +8 0 53000 245000 +9 1 54000 250000 + + +-- !query 20 +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10 +-- !query 20 schema +struct +-- !query 20 output +2 3 +2 3 +4 1 +4 1 +4 1 +4 1 + + +-- !query 21 +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10 +-- !query 21 schema +struct +-- !query 21 output +136 +22 +22 +24 +24 +51 +82 +87 +92 +92 + + +-- !query 22 +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum +-- !query 22 schema +struct +-- !query 22 output + + + +-- !query 23 +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 +-- !query 23 schema +struct +-- !query 23 output +0.0 +0.0 +0.0 +1.0 +1.0 +1.0 +1.0 +2.0 +3.0 +3.0 + + +-- !query 24 +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten) +-- !query 24 schema +struct +-- !query 24 output +0 0 45000 45000 +1 1 46000 46000 +2 0 47000 92000 +3 1 48000 94000 +4 0 49000 141000 +5 1 50000 144000 +6 0 51000 192000 +7 1 52000 196000 +8 0 53000 245000 +9 1 54000 250000 + + +-- !query 25 +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s +-- !query 25 schema +struct +-- !query 25 output + + + +-- !query 26 +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42 +-- !query 27 schema +struct +-- !query 27 output +0 + + +-- !query 28 +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten +-- !query 28 schema +struct +-- !query 28 output +0 9976146 4 +1 10114187 9 +2 10059554 8 +3 9878541 1 +4 9881005 2 +5 9981670 5 +6 9947099 3 +7 10120309 10 +8 9991305 6 +9 10040184 7 + + +-- !query 29 +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss +-- !query 29 schema +struct +-- !query 29 output +0 0 0 0 +0 2 2 2 +0 4 6 4 +0 6 12 6 +0 8 20 8 +1 1 1 1 +1 3 4 3 +1 5 9 5 +1 7 16 7 +1 9 25 9 +2 0 0 0 +2 2 2 2 +2 4 6 4 +2 6 12 6 +2 8 20 8 +3 1 1 1 +3 3 4 3 +3 5 9 5 +3 7 16 7 +3 9 25 9 + + +-- !query 30 +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss +-- !query 30 schema +struct +-- !query 30 output +0 0 0 0 +0 2 2 2 +0 4 6 4 +0 6 12 6 +0 8 20 8 +1 1 1 1 +1 3 4 3 +1 5 9 5 +1 7 16 7 +1 9 25 9 +2 0 0 0 +2 2 2 2 +2 4 6 4 +2 6 12 6 +2 8 20 8 +3 1 1 1 +3 3 4 3 +3 5 9 5 +3 7 16 7 +3 9 25 9 + + +-- !query 31 +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss +-- !query 31 schema +struct +-- !query 31 output +0 0 20 8 +0 2 20 8 +0 4 20 8 +0 6 20 8 +0 8 20 8 +1 1 25 9 +1 3 25 9 +1 5 25 9 +1 7 25 9 +1 9 25 9 +2 0 20 8 +2 2 20 8 +2 4 20 8 +2 6 20 8 +2 8 20 8 +3 1 25 9 +3 3 25 9 +3 5 25 9 +3 7 25 9 +3 9 25 9 + + +-- !query 32 +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 32 schema +struct +-- !query 32 output +10 3 3 +10 7 3 +18 2 2 +18 6 2 +33 1 1 +33 5 1 +33 9 1 +45 0 0 +45 4 0 +45 8 0 + + +-- !query 33 +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 33 schema +struct +-- !query 33 output +0 0 0 +10 3 3 +15 5 1 +23 8 0 +32 9 1 +38 6 2 +39 1 1 +41 2 2 +45 4 0 +7 7 3 + + +-- !query 34 +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 34 schema +struct +-- !query 34 output +10 0 0 +13 2 2 +15 7 3 +22 1 1 +23 3 3 +26 6 2 +29 9 1 +31 8 0 +32 5 1 +7 4 0 + + +-- !query 35 +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 35 schema +struct +-- !query 35 output +10 0 0 +13 3 3 +15 8 0 +17 5 1 +3 6 2 +4 2 2 +6 1 1 +7 9 1 +8 7 3 +NULL 4 0 + + +-- !query 36 +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 36 schema +struct +-- !query 36 output +0 7 3 +10 5 1 +15 8 0 +16 2 2 +16 9 1 +22 6 2 +23 1 1 +7 3 3 +9 4 0 +NULL 0 0 + + +-- !query 37 +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query 37 schema +struct +-- !query 37 output +13 1 1 +22 6 2 +30 9 1 +35 8 0 +38 5 1 +45 0 0 +45 3 3 +45 7 3 +6 4 0 +7 2 2 + + +-- !query 38 +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i +-- !query 38 schema +struct<> +-- !query 38 output + + + +-- !query 39 +SELECT * FROM v_window +-- !query 39 schema +struct +-- !query 39 output +1 3 +10 19 +2 6 +3 9 +4 12 +5 15 +6 18 +7 21 +8 24 +9 27 + + +-- !query 40 +DROP VIEW v_window +-- !query 40 schema +struct<> +-- !query 40 output + + + +-- !query 41 +DROP VIEW tenk2 +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +DROP VIEW int4_tbl +-- !query 42 schema +struct<> +-- !query 42 output + From c128ac564d198effe6bb9754489ea32133dfeb89 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 23 Oct 2019 12:17:20 +0800 Subject: [PATCH 0398/1387] [SPARK-29511][SQL] DataSourceV2: Support CREATE NAMESPACE ### What changes were proposed in this pull request? This PR adds `CREATE NAMESPACE` support for V2 catalogs. ### Why are the changes needed? Currently, you cannot explicitly create namespaces for v2 catalogs. ### Does this PR introduce any user-facing change? The user can now perform the following: ```SQL CREATE NAMESPACE mycatalog.ns ``` to create a namespace `ns` inside `mycatalog` V2 catalog. ### How was this patch tested? Added unit tests. Closes #26166 from imback82/create_namespace. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 1 + .../spark/sql/catalyst/parser/SqlBase.g4 | 7 +- .../catalyst/analysis/ResolveCatalogs.scala | 7 ++ .../sql/catalyst/parser/AstBuilder.scala | 40 +++++++++ .../catalyst/plans/logical/statements.scala | 13 +++ .../catalyst/plans/logical/v2Commands.scala | 8 ++ .../sql/catalyst/parser/DDLParserSuite.scala | 84 +++++++++++++++++++ .../catalyst/parser/ParserUtilsSuite.scala | 2 +- .../analysis/ResolveSessionCatalog.scala | 15 +++- .../spark/sql/execution/SparkSqlParser.scala | 27 ------ .../datasources/v2/CreateNamespaceExec.scala | 55 ++++++++++++ .../datasources/v2/DataSourceV2Strategy.scala | 5 +- .../sql/connector/DataSourceV2SQLSuite.scala | 28 ++++++- .../execution/command/DDLParserSuite.scala | 58 ------------- 14 files changed, 259 insertions(+), 91 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 7a0e3efee8ffa..b4f8d8be11c4f 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -210,6 +210,7 @@ Below is a list of all the keywords in Spark SQL. PRECEDINGnon-reservednon-reservednon-reserved PRIMARYreservednon-reservedreserved PRINCIPALSnon-reservednon-reservednon-reserved + PROPERTIESnon-reservednon-reservednon-reserved PURGEnon-reservednon-reservednon-reserved QUERYnon-reservednon-reservednon-reserved RANGEnon-reservednon-reservedreserved diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 963077c35df92..7e5e16b8e32bd 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -83,10 +83,10 @@ statement : query #statementDefault | ctes? dmlStatementNoWith #dmlStatement | USE NAMESPACE? multipartIdentifier #use - | CREATE database (IF NOT EXISTS)? db=errorCapturingIdentifier + | CREATE (database | NAMESPACE) (IF NOT EXISTS)? multipartIdentifier ((COMMENT comment=STRING) | locationSpec | - (WITH DBPROPERTIES tablePropertyList))* #createDatabase + (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))* #createNamespace | ALTER database db=errorCapturingIdentifier SET DBPROPERTIES tablePropertyList #setDatabaseProperties | ALTER database db=errorCapturingIdentifier @@ -1039,6 +1039,7 @@ ansiNonReserved | POSITION | PRECEDING | PRINCIPALS + | PROPERTIES | PURGE | QUERY | RANGE @@ -1299,6 +1300,7 @@ nonReserved | PRECEDING | PRIMARY | PRINCIPALS + | PROPERTIES | PURGE | QUERY | RANGE @@ -1564,6 +1566,7 @@ POSITION: 'POSITION'; PRECEDING: 'PRECEDING'; PRIMARY: 'PRIMARY'; PRINCIPALS: 'PRINCIPALS'; +PROPERTIES: 'PROPERTIES'; PURGE: 'PURGE'; QUERY: 'QUERY'; RANGE: 'RANGE'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 13a79a82a3858..6553b3d57d7f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -168,6 +168,13 @@ class ResolveCatalogs(val catalogManager: CatalogManager) s"Can not specify catalog `${catalog.name}` for view ${viewName.quoted} " + s"because view support in catalog has not been implemented yet") + case c @ CreateNamespaceStatement(NonSessionCatalog(catalog, nameParts), _, _) => + CreateNamespace( + catalog.asNamespaceCatalog, + nameParts, + c.ifNotExists, + c.properties) + case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 548042bc97674..7c67952aba403 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2307,6 +2307,46 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Create a [[CreateNamespaceStatement]] command. + * + * For example: + * {{{ + * CREATE NAMESPACE [IF NOT EXISTS] ns1.ns2.ns3 + * create_namespace_clauses; + * + * create_namespace_clauses (order insensitive): + * [COMMENT namespace_comment] + * [LOCATION path] + * [WITH PROPERTIES (key1=val1, key2=val2, ...)] + * }}} + */ + override def visitCreateNamespace(ctx: CreateNamespaceContext): LogicalPlan = withOrigin(ctx) { + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + checkDuplicateClauses(ctx.PROPERTIES, "WITH PROPERTIES", ctx) + checkDuplicateClauses(ctx.DBPROPERTIES, "WITH DBPROPERTIES", ctx) + + if (!ctx.PROPERTIES.isEmpty && !ctx.DBPROPERTIES.isEmpty) { + throw new ParseException(s"Either PROPERTIES or DBPROPERTIES is allowed.", ctx) + } + + var properties = ctx.tablePropertyList.asScala.headOption + .map(visitPropertyKeyValues) + .getOrElse(Map.empty) + Option(ctx.comment).map(string).map { + properties += CreateNamespaceStatement.COMMENT_PROPERTY_KEY -> _ + } + ctx.locationSpec.asScala.headOption.map(visitLocationSpec).map { + properties += CreateNamespaceStatement.LOCATION_PROPERTY_KEY -> _ + } + + CreateNamespaceStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + ctx.EXISTS != null, + properties) + } + /** * Create a [[ShowNamespacesStatement]] command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index a73a2975aa9ca..3bd16187320f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -282,6 +282,19 @@ case class InsertIntoStatement( case class ShowTablesStatement(namespace: Option[Seq[String]], pattern: Option[String]) extends ParsedStatement +/** + * A CREATE NAMESPACE statement, as parsed from SQL. + */ +case class CreateNamespaceStatement( + namespace: Seq[String], + ifNotExists: Boolean, + properties: Map[String, String]) extends ParsedStatement + +object CreateNamespaceStatement { + val COMMENT_PROPERTY_KEY: String = "comment" + val LOCATION_PROPERTY_KEY: String = "location" +} + /** * A SHOW NAMESPACES statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index f89dfb1ec47d8..8f5731a4a7a7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -237,6 +237,14 @@ case class ReplaceTableAsSelect( } } +/** + * The logical plan of the CREATE NAMESPACE command that works for v2 catalogs. + */ +case class CreateNamespace( + catalog: SupportsNamespaces, + namespace: Seq[String], + ifNotExists: Boolean, + properties: Map[String, String]) extends Command /** * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 1dacb2384ac13..38ef357036a0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -845,6 +845,90 @@ class DDLParserSuite extends AnalysisTest { ShowTablesStatement(Some(Seq("tbl")), Some("*dog*"))) } + test("create namespace -- backward compatibility with DATABASE/DBPROPERTIES") { + val expected = CreateNamespaceStatement( + Seq("a", "b", "c"), + ifNotExists = true, + Map( + "a" -> "a", + "b" -> "b", + "c" -> "c", + "comment" -> "namespace_comment", + "location" -> "/home/user/db")) + + comparePlans( + parsePlan( + """ + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c') + |COMMENT 'namespace_comment' LOCATION '/home/user/db' + """.stripMargin), + expected) + + comparePlans( + parsePlan( + """ + |CREATE DATABASE IF NOT EXISTS a.b.c + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + |COMMENT 'namespace_comment' LOCATION '/home/user/db' + """.stripMargin), + expected) + } + + test("create namespace -- check duplicates") { + def createDatabase(duplicateClause: String): String = { + s""" + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |$duplicateClause + |$duplicateClause + """.stripMargin + } + val sql1 = createDatabase("COMMENT 'namespace_comment'") + val sql2 = createDatabase("LOCATION '/home/user/db'") + val sql3 = createDatabase("WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c')") + val sql4 = createDatabase("WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + + intercept(sql1, "Found duplicate clauses: COMMENT") + intercept(sql2, "Found duplicate clauses: LOCATION") + intercept(sql3, "Found duplicate clauses: WITH PROPERTIES") + intercept(sql4, "Found duplicate clauses: WITH DBPROPERTIES") + } + + test("create namespace - property values must be set") { + assertUnsupported( + sql = "CREATE NAMESPACE a.b.c WITH PROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("create namespace -- either PROPERTIES or DBPROPERTIES is allowed") { + val sql = + s""" + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c') + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + """.stripMargin + intercept(sql, "Either PROPERTIES or DBPROPERTIES is allowed") + } + + test("create namespace - support for other types in PROPERTIES") { + val sql = + """ + |CREATE NAMESPACE a.b.c + |LOCATION '/home/user/db' + |WITH PROPERTIES ('a'=1, 'b'=0.1, 'c'=TRUE) + """.stripMargin + comparePlans( + parsePlan(sql), + CreateNamespaceStatement( + Seq("a", "b", "c"), + ifNotExists = false, + Map( + "a" -> "1", + "b" -> "0.1", + "c" -> "true", + "location" -> "/home/user/db"))) + } + test("show databases: basic") { comparePlans( parsePlan("SHOW DATABASES"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala index 07f77ea889dba..c6434f2bdd3ec 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala @@ -50,7 +50,7 @@ class ParserUtilsSuite extends SparkFunSuite { |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') """.stripMargin ) { parser => - parser.statement().asInstanceOf[CreateDatabaseContext] + parser.statement().asInstanceOf[CreateNamespaceContext] } val emptyContext = buildContext("") { parser => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 4a2e6731d9d82..4cca9846e9965 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -255,6 +255,19 @@ class ResolveSessionCatalog( case DropViewStatement(SessionCatalog(catalog, viewName), ifExists) => DropTableCommand(viewName.asTableIdentifier, ifExists, isView = true, purge = false) + case c @ CreateNamespaceStatement(SessionCatalog(catalog, nameParts), _, _) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + + val comment = c.properties.get(CreateNamespaceStatement.COMMENT_PROPERTY_KEY) + val location = c.properties.get(CreateNamespaceStatement.LOCATION_PROPERTY_KEY) + val newProperties = c.properties - + CreateNamespaceStatement.COMMENT_PROPERTY_KEY - + CreateNamespaceStatement.LOCATION_PROPERTY_KEY + CreateDatabaseCommand(nameParts.head, c.ifNotExists, location, comment, newProperties) + case ShowTablesStatement(Some(SessionCatalog(catalog, nameParts)), pattern) => if (nameParts.length != 1) { throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 3f3f6b373eb0c..38f3c6e1b7509 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -329,33 +329,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ) } - /** - * Create a [[CreateDatabaseCommand]] command. - * - * For example: - * {{{ - * CREATE DATABASE [IF NOT EXISTS] database_name - * create_database_clauses; - * - * create_database_clauses (order insensitive): - * [COMMENT database_comment] - * [LOCATION path] - * [WITH DBPROPERTIES (key1=val1, key2=val2, ...)] - * }}} - */ - override def visitCreateDatabase(ctx: CreateDatabaseContext): LogicalPlan = withOrigin(ctx) { - checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) - checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) - checkDuplicateClauses(ctx.DBPROPERTIES, "WITH DBPROPERTIES", ctx) - - CreateDatabaseCommand( - ctx.db.getText, - ctx.EXISTS != null, - ctx.locationSpec.asScala.headOption.map(visitLocationSpec), - Option(ctx.comment).map(string), - ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues).getOrElse(Map.empty)) - } - /** * Create an [[AlterDatabasePropertiesCommand]] command. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala new file mode 100644 index 0000000000000..0f69f85dd8376 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala @@ -0,0 +1,55 @@ +/* + * 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.v2 + +import scala.collection.JavaConverters.mapAsJavaMapConverter + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.SupportsNamespaces + +/** + * Physical plan node for creating a namespace. + */ +case class CreateNamespaceExec( + catalog: SupportsNamespaces, + namespace: Seq[String], + ifNotExists: Boolean, + private var properties: Map[String, String]) + extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val ns = namespace.toArray + if (!catalog.namespaceExists(ns)) { + try { + catalog.createNamespace(ns, properties.asJava) + } catch { + case _: NamespaceAlreadyExistsException if ifNotExists => + logWarning(s"Namespace ${namespace.quoted} was created concurrently. Ignoring.") + } + } else if (!ifNotExists) { + throw new NamespaceAlreadyExistsException(ns) + } + + Seq.empty + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index c8d29520bcfce..49035c3cc3dad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} @@ -289,6 +289,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case AlterTable(catalog, ident, _, changes) => AlterTableExec(catalog, ident, changes) :: Nil + case CreateNamespace(catalog, namespace, ifNotExists, properties) => + CreateNamespaceExec(catalog, namespace, ifNotExists, properties) :: Nil + case r: ShowNamespaces => ShowNamespacesExec(r.output, r.catalog, r.namespace, r.pattern) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 39709ab426a09..2ea26787dbb19 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf @@ -764,6 +764,32 @@ class DataSourceV2SQLSuite assert(expected === df.collect()) } + test("CreateNameSpace: basic tests") { + // Session catalog is used. + sql("CREATE NAMESPACE ns") + testShowNamespaces("SHOW NAMESPACES", Seq("default", "ns")) + + // V2 non-session catalog is used. + sql("CREATE NAMESPACE testcat.ns1.ns2") + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1")) + testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns2")) + + // TODO: Add tests for validating namespace metadata when DESCRIBE NAMESPACE is available. + } + + test("CreateNameSpace: test handling of 'IF NOT EXIST'") { + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1") + + // The 'ns1' namespace already exists, so this should fail. + val exception = intercept[NamespaceAlreadyExistsException] { + sql("CREATE NAMESPACE testcat.ns1") + } + assert(exception.getMessage.contains("Namespace 'ns1' already exists")) + + // The following will be no-op since the namespace already exists. + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1") + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set("spark.sql.default.catalog", "testcat") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 0640d0540baa6..a9b94bea9517a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -74,46 +74,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { }.head } - test("create database") { - val sql = - """ - |CREATE DATABASE IF NOT EXISTS database_name - |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') - |COMMENT 'database_comment' LOCATION '/home/user/db' - """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = CreateDatabaseCommand( - "database_name", - ifNotExists = true, - Some("/home/user/db"), - Some("database_comment"), - Map("a" -> "a", "b" -> "b", "c" -> "c")) - comparePlans(parsed, expected) - } - - test("create database -- check duplicates") { - def createDatabase(duplicateClause: String): String = { - s""" - |CREATE DATABASE IF NOT EXISTS database_name - |$duplicateClause - |$duplicateClause - """.stripMargin - } - val sql1 = createDatabase("COMMENT 'database_comment'") - val sql2 = createDatabase("LOCATION '/home/user/db'") - val sql3 = createDatabase("WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") - - intercept(sql1, "Found duplicate clauses: COMMENT") - intercept(sql2, "Found duplicate clauses: LOCATION") - intercept(sql3, "Found duplicate clauses: WITH DBPROPERTIES") - } - - test("create database - property values must be set") { - assertUnsupported( - sql = "CREATE DATABASE my_db WITH DBPROPERTIES('key_without_value', 'key_with_value'='x')", - containsThesePhrases = Seq("key_without_value")) - } - test("drop database") { val sql1 = "DROP DATABASE IF EXISTS database_name RESTRICT" val sql2 = "DROP DATABASE IF EXISTS database_name CASCADE" @@ -870,24 +830,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { comparePlans(parsed4, expected4) } - test("support for other types in DBPROPERTIES") { - val sql = - """ - |CREATE DATABASE database_name - |LOCATION '/home/user/db' - |WITH DBPROPERTIES ('a'=1, 'b'=0.1, 'c'=TRUE) - """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = CreateDatabaseCommand( - "database_name", - ifNotExists = false, - Some("/home/user/db"), - None, - Map("a" -> "1", "b" -> "0.1", "c" -> "true")) - - comparePlans(parsed, expected) - } - test("Test CTAS #1") { val s1 = """ From 8c3469009cf84c95a81bb684244aac29b650d225 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 23 Oct 2019 00:14:43 -0700 Subject: [PATCH 0399/1387] [SPARK-29546][TESTS] Recover jersey-guava test dependency in docker-integration-tests ### What changes were proposed in this pull request? Currently, `docker-integration-tests` is broken in both JDK8/11. This PR aims to recover JDBC integration test for JDK8/11. ### Why are the changes needed? While SPARK-28737 upgraded `Jersey` to 2.29 for JDK11, `docker-integration-tests` is broken because `com.spotify.docker-client` still depends on `jersey-guava`. The latest `com.spotify.docker-client` also has this problem. - https://mvnrepository.com/artifact/com.spotify/docker-client/5.0.2 -> https://mvnrepository.com/artifact/org.glassfish.jersey.core/jersey-client/2.19 -> https://mvnrepository.com/artifact/org.glassfish.jersey.core/jersey-common/2.19 -> https://mvnrepository.com/artifact/org.glassfish.jersey.bundles.repackaged/jersey-guava/2.19 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manual because this is an integration test suite. ``` $ java -version openjdk version "1.8.0_222" OpenJDK Runtime Environment (AdoptOpenJDK)(build 1.8.0_222-b10) OpenJDK 64-Bit Server VM (AdoptOpenJDK)(build 25.222-b10, mixed mode) $ build/mvn install -DskipTests $ build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12 test ``` ``` $ java -version openjdk version "11.0.5" 2019-10-15 OpenJDK Runtime Environment AdoptOpenJDK (build 11.0.5+10) OpenJDK 64-Bit Server VM AdoptOpenJDK (build 11.0.5+10, mixed mode) $ build/mvn install -DskipTests $ build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12 test ``` **BEFORE** ``` *** RUN ABORTED *** com.spotify.docker.client.exceptions.DockerException: java.util.concurrent.ExecutionException: javax.ws.rs.ProcessingException: java.lang.NoClassDefFoundError: jersey/repackaged/com/google/common/util/concurrent/MoreExecutors at com.spotify.docker.client.DefaultDockerClient.propagate(DefaultDockerClient.java:1607) at com.spotify.docker.client.DefaultDockerClient.request(DefaultDockerClient.java:1538) at com.spotify.docker.client.DefaultDockerClient.ping(DefaultDockerClient.java:387) at org.apache.spark.sql.jdbc.DockerJDBCIntegrationSuite.beforeAll(DockerJDBCIntegrationSuite.scala:81) ``` **AFTER** ``` Run completed in 47 seconds, 999 milliseconds. Total number of tests run: 30 Suites: completed 6, aborted 0 Tests: succeeded 30, failed 0, canceled 0, ignored 6, pending 0 All tests passed. ``` Closes #26203 from dongjoon-hyun/SPARK-29546. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- external/docker-integration-tests/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index a4956ff5ee9cc..aff79b8b8e642 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -106,6 +106,14 @@ test-jar test
    + + + org.glassfish.jersey.bundles.repackaged + jersey-guava + 2.25.1 + test + mysql mysql-connector-java From cbe6eadc0c1d0384c1ee03f3a5b28cc583a60717 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 23 Oct 2019 10:56:19 +0200 Subject: [PATCH 0400/1387] [SPARK-29352][SQL][SS] Track active streaming queries in the SparkSession.sharedState ### What changes were proposed in this pull request? This moves the tracking of active queries from a per SparkSession state, to the shared SparkSession for better safety in isolated Spark Session environments. ### Why are the changes needed? We have checks to prevent the restarting of the same stream on the same spark session, but we can actually make that better in multi-tenant environments by actually putting that state in the SharedState instead of SessionState. This would allow a more comprehensive check for multi-tenant clusters. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added tests to StreamingQueryManagerSuite Closes #26018 from brkyvz/sharedStreamingQueryManager. Lead-authored-by: Burak Yavuz Co-authored-by: Burak Yavuz Signed-off-by: Burak Yavuz --- .../spark/sql/internal/SharedState.scala | 10 ++- .../sql/streaming/StreamingQueryManager.scala | 22 +++-- .../StreamingQueryManagerSuite.scala | 80 ++++++++++++++++++- 3 files changed, 102 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index f1a648176c3b3..d097f9f18f89b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.internal import java.net.URL -import java.util.Locale +import java.util.{Locale, UUID} +import java.util.concurrent.ConcurrentHashMap import scala.reflect.ClassTag import scala.util.control.NonFatal @@ -33,6 +34,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.streaming.StreamingQueryManager import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils @@ -110,6 +112,12 @@ private[sql] class SharedState( */ val cacheManager: CacheManager = new CacheManager + /** + * A map of active streaming queries to the session specific StreamingQueryManager that manages + * the lifecycle of that stream. + */ + private[sql] val activeStreamingQueries = new ConcurrentHashMap[UUID, StreamingQueryManager]() + /** * A status store to query SQL status/metrics of this Spark application, based on SQL-specific * [[org.apache.spark.scheduler.SparkListenerEvent]]s. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 9abe38dfda0be..9b43a83e7b94a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -352,8 +352,10 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo } } - // Make sure no other query with same id is active - if (activeQueries.values.exists(_.id == query.id)) { + // Make sure no other query with same id is active across all sessions + val activeOption = + Option(sparkSession.sharedState.activeStreamingQueries.putIfAbsent(query.id, this)) + if (activeOption.isDefined || activeQueries.values.exists(_.id == query.id)) { throw new IllegalStateException( s"Cannot start query with id ${query.id} as another query with same id is " + s"already active. Perhaps you are attempting to restart a query from checkpoint " + @@ -370,9 +372,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo query.streamingQuery.start() } catch { case e: Throwable => - activeQueriesLock.synchronized { - activeQueries -= query.id - } + unregisterTerminatedStream(query.id) throw e } query @@ -380,9 +380,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo /** Notify (by the StreamingQuery) that the query has been terminated */ private[sql] def notifyQueryTermination(terminatedQuery: StreamingQuery): Unit = { - activeQueriesLock.synchronized { - activeQueries -= terminatedQuery.id - } + unregisterTerminatedStream(terminatedQuery.id) awaitTerminationLock.synchronized { if (lastTerminatedQuery == null || terminatedQuery.exception.nonEmpty) { lastTerminatedQuery = terminatedQuery @@ -391,4 +389,12 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo } stateStoreCoordinator.deactivateInstances(terminatedQuery.runId) } + + private def unregisterTerminatedStream(terminatedQueryId: UUID): Unit = { + activeQueriesLock.synchronized { + // remove from shared state only if the streaming query manager also matches + sparkSession.sharedState.activeStreamingQueries.remove(terminatedQueryId, this) + activeQueries -= terminatedQueryId + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala index b26d2556b2e36..09580b94056b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.streaming +import java.io.File import java.util.concurrent.CountDownLatch import scala.concurrent.Future @@ -28,7 +29,7 @@ import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException -import org.apache.spark.sql.Dataset +import org.apache.spark.sql.{Dataset, Encoders} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.streaming.util.BlockingSource @@ -242,6 +243,83 @@ class StreamingQueryManagerSuite extends StreamTest { } } + testQuietly("can't start a streaming query with the same name in the same session") { + val ds1 = makeDataset._2 + val ds2 = makeDataset._2 + val queryName = "abc" + + val query1 = ds1.writeStream.format("noop").queryName(queryName).start() + try { + val e = intercept[IllegalArgumentException] { + ds2.writeStream.format("noop").queryName(queryName).start() + } + assert(e.getMessage.contains("query with that name is already active")) + } finally { + query1.stop() + } + } + + testQuietly("can start a streaming query with the same name in a different session") { + val session2 = spark.cloneSession() + + val ds1 = MemoryStream(Encoders.INT, spark.sqlContext).toDS() + val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() + val queryName = "abc" + + val query1 = ds1.writeStream.format("noop").queryName(queryName).start() + val query2 = ds2.writeStream.format("noop").queryName(queryName).start() + + query1.stop() + query2.stop() + } + + testQuietly("can't start multiple instances of the same streaming query in the same session") { + withTempDir { dir => + val (ms1, ds1) = makeDataset + val (ms2, ds2) = makeDataset + val chkLocation = new File(dir, "_checkpoint").getCanonicalPath + val dataLocation = new File(dir, "data").getCanonicalPath + + val query1 = ds1.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + ms1.addData(1, 2, 3) + try { + val e = intercept[IllegalStateException] { + ds2.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + } + assert(e.getMessage.contains("same id")) + } finally { + query1.stop() + } + } + } + + testQuietly( + "can't start multiple instances of the same streaming query in the different sessions") { + withTempDir { dir => + val session2 = spark.cloneSession() + + val ms1 = MemoryStream(Encoders.INT, spark.sqlContext) + val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() + val chkLocation = new File(dir, "_checkpoint").getCanonicalPath + val dataLocation = new File(dir, "data").getCanonicalPath + + val query1 = ms1.toDS().writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + ms1.addData(1, 2, 3) + try { + val e = intercept[IllegalStateException] { + ds2.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + } + assert(e.getMessage.contains("same id")) + } finally { + query1.stop() + } + } + } + /** Run a body of code by defining a query on each dataset */ private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[StreamingQuery] => Unit): Unit = { failAfter(streamingTimeout) { From 70dd9c0cabb52fac3ab20fbde7eeda41b19bad61 Mon Sep 17 00:00:00 2001 From: turbofei Date: Wed, 23 Oct 2019 20:31:06 +0900 Subject: [PATCH 0401/1387] [SPARK-29542][SQL][DOC] Make the descriptions of spark.sql.files.* be clearly ### What changes were proposed in this pull request? As described in [SPARK-29542](https://issues.apache.org/jira/browse/SPARK-29542) , the descriptions of `spark.sql.files.*` are confused. In this PR, I make their descriptions be clearly. ### Why are the changes needed? It makes the descriptions of `spark.sql.files.*` be clearly. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing UT. Closes #26200 from turboFei/SPARK-29542-partition-maxSize. Authored-by: turbofei Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/internal/SQLConf.scala | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 75db52e334b86..7f75bf84d65ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -980,7 +980,9 @@ object SQLConf { .createWithDefault(true) val FILES_MAX_PARTITION_BYTES = buildConf("spark.sql.files.maxPartitionBytes") - .doc("The maximum number of bytes to pack into a single partition when reading files.") + .doc("The maximum number of bytes to pack into a single partition when reading files. " + + "This configuration is effective only when using file-based sources such as Parquet, JSON " + + "and ORC.") .bytesConf(ByteUnit.BYTE) .createWithDefault(128 * 1024 * 1024) // parquet.block.size @@ -989,19 +991,24 @@ object SQLConf { .doc("The estimated cost to open a file, measured by the number of bytes could be scanned in" + " the same time. This is used when putting multiple files into a partition. It's better to" + " over estimated, then the partitions with small files will be faster than partitions with" + - " bigger files (which is scheduled first).") + " bigger files (which is scheduled first). This configuration is effective only when using" + + " file-based sources such as Parquet, JSON and ORC.") .longConf .createWithDefault(4 * 1024 * 1024) val IGNORE_CORRUPT_FILES = buildConf("spark.sql.files.ignoreCorruptFiles") .doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " + - "encountering corrupted files and the contents that have been read will still be returned.") + "encountering corrupted files and the contents that have been read will still be returned. " + + "This configuration is effective only when using file-based sources such as Parquet, JSON " + + "and ORC.") .booleanConf .createWithDefault(false) val IGNORE_MISSING_FILES = buildConf("spark.sql.files.ignoreMissingFiles") .doc("Whether to ignore missing files. If true, the Spark jobs will continue to run when " + - "encountering missing files and the contents that have been read will still be returned.") + "encountering missing files and the contents that have been read will still be returned. " + + "This configuration is effective only when using file-based sources such as Parquet, JSON " + + "and ORC.") .booleanConf .createWithDefault(false) From 0a7095156bdb565133f7dcc74546c51a5e5d2414 Mon Sep 17 00:00:00 2001 From: Xianyang Liu Date: Wed, 23 Oct 2019 13:46:09 +0200 Subject: [PATCH 0402/1387] [SPARK-29499][CORE][PYSPARK] Add mapPartitionsWithIndex for RDDBarrier ### What changes were proposed in this pull request? Add mapPartitionsWithIndex for RDDBarrier. ### Why are the changes needed? There is only one method in `RDDBarrier`. We often use the partition index as a label for the current partition. We need to get the index from `TaskContext` index in the method of `mapPartitions` which is not convenient. ### Does this PR introduce any user-facing change? No ### How was this patch tested? New UT. Closes #26148 from ConeyLiu/barrier-index. Authored-by: Xianyang Liu Signed-off-by: Xingbo Jiang --- .../org/apache/spark/rdd/RDDBarrier.scala | 22 ++++++++ .../apache/spark/rdd/RDDBarrierSuite.scala | 9 ++++ dev/sparktestsupport/modules.py | 1 + python/pyspark/rdd.py | 14 ++++++ python/pyspark/tests/test_rddbarrier.py | 50 +++++++++++++++++++ 5 files changed, 96 insertions(+) create mode 100644 python/pyspark/tests/test_rddbarrier.py diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDBarrier.scala b/core/src/main/scala/org/apache/spark/rdd/RDDBarrier.scala index 42802f7113a19..b70ea0073c9a0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDBarrier.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDBarrier.scala @@ -54,5 +54,27 @@ class RDDBarrier[T: ClassTag] private[spark] (rdd: RDD[T]) { ) } + /** + * :: Experimental :: + * Returns a new RDD by applying a function to each partition of the wrapped RDD, while tracking + * the index of the original partition. And all tasks are launched together in a barrier stage. + * The interface is the same as [[org.apache.spark.rdd.RDD#mapPartitionsWithIndex]]. + * Please see the API doc there. + * @see [[org.apache.spark.BarrierTaskContext]] + */ + @Experimental + @Since("3.0.0") + def mapPartitionsWithIndex[S: ClassTag]( + f: (Int, Iterator[T]) => Iterator[S], + preservesPartitioning: Boolean = false): RDD[S] = rdd.withScope { + val cleanedF = rdd.sparkContext.clean(f) + new MapPartitionsRDD( + rdd, + (_: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(index, iter), + preservesPartitioning, + isFromBarrier = true + ) + } + // TODO: [SPARK-25247] add extra conf to RDDBarrier, e.g., timeout. } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDBarrierSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDBarrierSuite.scala index 2f6c4d6a42ea3..f048f95430138 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDBarrierSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDBarrierSuite.scala @@ -29,6 +29,15 @@ class RDDBarrierSuite extends SparkFunSuite with SharedSparkContext { assert(rdd2.isBarrier()) } + test("RDDBarrier mapPartitionsWithIndex") { + val rdd = sc.parallelize(1 to 12, 4) + assert(rdd.isBarrier() === false) + + val rdd2 = rdd.barrier().mapPartitionsWithIndex((index, iter) => Iterator(index)) + assert(rdd2.isBarrier()) + assert(rdd2.collect().toList === List(0, 1, 2, 3)) + } + test("create an RDDBarrier in the middle of a chain of RDDs") { val rdd = sc.parallelize(1 to 10, 4).map(x => x * 2) val rdd2 = rdd.barrier().mapPartitions(iter => iter).map(x => (x, x + 1)) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index c7ea065b28ed8..1443584ccbcb8 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -329,6 +329,7 @@ def __hash__(self): "pyspark.tests.test_join", "pyspark.tests.test_profiler", "pyspark.tests.test_rdd", + "pyspark.tests.test_rddbarrier", "pyspark.tests.test_readwrite", "pyspark.tests.test_serializers", "pyspark.tests.test_shuffle", diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1edffaa4ca168..52ab86c0d88ee 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2535,6 +2535,20 @@ def func(s, iterator): return f(iterator) return PipelinedRDD(self.rdd, func, preservesPartitioning, isFromBarrier=True) + def mapPartitionsWithIndex(self, f, preservesPartitioning=False): + """ + .. note:: Experimental + + Returns a new RDD by applying a function to each partition of the wrapped RDD, while + tracking the index of the original partition. And all tasks are launched together + in a barrier stage. + The interface is the same as :func:`RDD.mapPartitionsWithIndex`. + Please see the API doc there. + + .. versionadded:: 3.0.0 + """ + return PipelinedRDD(self.rdd, f, preservesPartitioning, isFromBarrier=True) + class PipelinedRDD(RDD): diff --git a/python/pyspark/tests/test_rddbarrier.py b/python/pyspark/tests/test_rddbarrier.py new file mode 100644 index 0000000000000..8534fb4abb876 --- /dev/null +++ b/python/pyspark/tests/test_rddbarrier.py @@ -0,0 +1,50 @@ +# +# 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 pyspark.testing.utils import ReusedPySparkTestCase + + +class RDDBarrierTests(ReusedPySparkTestCase): + def test_map_partitions(self): + """Test RDDBarrier.mapPartitions""" + rdd = self.sc.parallelize(range(12), 4) + self.assertFalse(rdd._is_barrier()) + + rdd1 = rdd.barrier().mapPartitions(lambda it: it) + self.assertTrue(rdd1._is_barrier()) + + def test_map_partitions_with_index(self): + """Test RDDBarrier.mapPartitionsWithIndex""" + rdd = self.sc.parallelize(range(12), 4) + self.assertFalse(rdd._is_barrier()) + + def f(index, iterator): + yield index + rdd1 = rdd.barrier().mapPartitionsWithIndex(f) + self.assertTrue(rdd1._is_barrier()) + self.assertEqual(rdd1.collect(), [0, 1, 2, 3]) + + +if __name__ == "__main__": + import unittest + from pyspark.tests.test_rddbarrier import * + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) From df00b5c17d1770078f25f66504043bb3d6514ef7 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 23 Oct 2019 15:23:25 +0200 Subject: [PATCH 0403/1387] [SPARK-29569][BUILD][DOCS] Copy and paste minified jquery instead when post-processing badges in JavaDoc ### What changes were proposed in this pull request? This PR fixes our documentation build to copy minified jquery file instead. The original file `jquery.js` seems missing as of Scala 2.12 upgrade. Scala 2.12 seems started to use minified `jquery.min.js` instead. Since we dropped Scala 2.11, we won't have to take care about legacy `jquery.js` anymore. Note that, there seem multiple weird stuff in the current ScalaDoc (e.g., some pages are weird, it starts from `scala.collection.*` or some pages are missing, or some docs are truncated, some badges look missing). It needs a separate double check and investigation. This PR targets to make the documentation generation pass in order to unblock Spark 3.0 preview. ### Why are the changes needed? To fix and make our official documentation build able to run. ### Does this PR introduce any user-facing change? It will enable to build the documentation in our official way. **Before:** ``` Making directory api/scala cp -r ../target/scala-2.12/unidoc/. api/scala Making directory api/java cp -r ../target/javaunidoc/. api/java Updating JavaDoc files for badge post-processing Copying jquery.js from Scala API to Java API for page post-processing of badges jekyll 3.8.6 | Error: No such file or directory rb_sysopen - ./api/scala/lib/jquery.js ``` **After:** ``` Making directory api/scala cp -r ../target/scala-2.12/unidoc/. api/scala Making directory api/java cp -r ../target/javaunidoc/. api/java Updating JavaDoc files for badge post-processing Copying jquery.min.js from Scala API to Java API for page post-processing of badges Copying api_javadocs.js to Java API for page post-processing of badges Appending content of api-javadocs.css to JavaDoc stylesheet.css for badge styles ... ``` ### How was this patch tested? Manually tested via: ``` SKIP_PYTHONDOC=1 SKIP_RDOC=1 SKIP_SQLDOC=1 jekyll build ``` Closes #26228 from HyukjinKwon/SPARK-29569. Authored-by: HyukjinKwon Signed-off-by: Xingbo Jiang --- docs/_plugins/copy_api_dirs.rb | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 2d1a9547e3731..f95e4e2f97792 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -96,9 +96,9 @@ end # End updating JavaDoc files for badge post-processing - puts "Copying jquery.js from Scala API to Java API for page post-processing of badges" - jquery_src_file = "./api/scala/lib/jquery.js" - jquery_dest_file = "./api/java/lib/jquery.js" + puts "Copying jquery.min.js from Scala API to Java API for page post-processing of badges" + jquery_src_file = "./api/scala/lib/jquery.min.js" + jquery_dest_file = "./api/java/lib/jquery.min.js" mkdir_p("./api/java/lib") cp(jquery_src_file, jquery_dest_file) From 53a5f17803851dc232ec3b39242e85b881ade6ef Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 23 Oct 2019 08:26:47 -0700 Subject: [PATCH 0404/1387] [SPARK-29513][SQL] REFRESH TABLE should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add RefreshTableStatement and make REFRESH TABLE go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog REFRESH TABLE t // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? yes. When running REFRESH TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? New unit tests Closes #26183 from imback82/refresh_table. Lead-authored-by: Terry Kim Co-authored-by: Terry Kim Signed-off-by: Liang-Chi Hsieh --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../catalyst/analysis/ResolveCatalogs.scala | 3 + .../sql/catalyst/parser/AstBuilder.scala | 12 ++++ .../catalyst/plans/logical/statements.scala | 8 ++- .../catalyst/plans/logical/v2Commands.scala | 7 +++ .../sql/catalyst/parser/DDLParserSuite.scala | 6 ++ .../sql/connector/InMemoryTableCatalog.scala | 10 ++++ .../analysis/ResolveSessionCatalog.scala | 5 +- .../spark/sql/execution/SparkSqlParser.scala | 7 --- .../datasources/v2/DataSourceV2Strategy.scala | 5 +- .../datasources/v2/RefreshTableExec.scala | 33 +++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 59 +++++++++---------- 12 files changed, 115 insertions(+), 44 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RefreshTableExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 7e5e16b8e32bd..970d244071e0d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -201,9 +201,9 @@ statement | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction | (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? - multipartIdentifier partitionSpec? describeColName? #describeTable + multipartIdentifier partitionSpec? describeColName? #describeTable | (DESC | DESCRIBE) QUERY? query #describeQuery - | REFRESH TABLE tableIdentifier #refreshTable + | REFRESH TABLE multipartIdentifier #refreshTable | REFRESH (STRING | .*?) #refreshResource | CACHE LAZY? TABLE tableIdentifier (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 6553b3d57d7f9..9803fda0678ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -137,6 +137,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) writeOptions = c.options.filterKeys(_ != "path"), ignoreIfExists = c.ifNotExists) + case RefreshTableStatement(NonSessionCatalog(catalog, tableName)) => + RefreshTable(catalog.asTableCatalog, tableName.asIdentifier) + case c @ ReplaceTableStatement( NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => ReplaceTable( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 7c67952aba403..940dfd0fc333d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2799,4 +2799,16 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) ShowPartitionsStatement(table, partitionKeys) } + + /** + * Create a [[RefreshTableStatement]]. + * + * For example: + * {{{ + * REFRESH TABLE multi_part_name + * }}} + */ + override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) { + RefreshTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 3bd16187320f8..127d9026f802e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -340,5 +340,11 @@ case class TruncateTableStatement( /** * A SHOW PARTITIONS statement, as parsed from SQL */ -case class ShowPartitionsStatement(tableName: Seq[String], +case class ShowPartitionsStatement( + tableName: Seq[String], partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement + +/** + * A REFRESH TABLE statement, as parsed from SQL + */ +case class RefreshTableStatement(tableName: Seq[String]) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 8f5731a4a7a7d..d80c1c034a867 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -348,3 +348,10 @@ case class SetCatalogAndNamespace( catalogManager: CatalogManager, catalogName: Option[String], namespace: Option[Seq[String]]) extends Command + +/** + * The logical plan of the REFRESH TABLE command that works for v2 catalogs. + */ +case class RefreshTable( + catalog: TableCatalog, + ident: Identifier) extends Command diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 38ef357036a0b..8e605bd15f696 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1081,6 +1081,12 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsed5, expected5) } + test("REFRESH TABLE table") { + comparePlans( + parsePlan("REFRESH TABLE a.b.c"), + RefreshTableStatement(Seq("a", "b", "c"))) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala index 8724a38d08d1f..ece903a4c2838 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala @@ -34,6 +34,8 @@ class BasicInMemoryTableCatalog extends TableCatalog { protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() + private val invalidatedTables: util.Set[Identifier] = ConcurrentHashMap.newKeySet() + private var _name: Option[String] = None override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { @@ -55,6 +57,10 @@ class BasicInMemoryTableCatalog extends TableCatalog { } } + override def invalidateTable(ident: Identifier): Unit = { + invalidatedTables.add(ident) + } + override def createTable( ident: Identifier, schema: StructType, @@ -104,6 +110,10 @@ class BasicInMemoryTableCatalog extends TableCatalog { } } + def isTableInvalidated(ident: Identifier): Boolean = { + invalidatedTables.contains(ident) + } + def clearTables(): Unit = { tables.clear() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 4cca9846e9965..230b8f3906bd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand} -import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} +import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} @@ -216,6 +216,9 @@ class ResolveSessionCatalog( ignoreIfExists = c.ifNotExists) } + case RefreshTableStatement(SessionCatalog(_, tableName)) => + RefreshTable(tableName.asTableIdentifier) + // For REPLACE TABLE [AS SELECT], we should fail if the catalog is resolved to the // session catalog and the table provider is not v2. case c @ ReplaceTableStatement( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 38f3c6e1b7509..2439621f7725a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -143,13 +143,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ShowCreateTableCommand(table) } - /** - * Create a [[RefreshTable]] logical plan. - */ - override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) { - RefreshTable(visitTableIdentifier(ctx.tableIdentifier)) - } - /** * Create a [[RefreshResource]] logical plan. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 49035c3cc3dad..4a7cb7db45ded 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} @@ -193,6 +193,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { catalog, ident, parts, query, planLater(query), props, writeOptions, ifNotExists) :: Nil } + case RefreshTable(catalog, ident) => + RefreshTableExec(catalog, ident) :: Nil + case ReplaceTable(catalog, ident, schema, parts, props, orCreate) => catalog match { case staging: StagingTableCatalog => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RefreshTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RefreshTableExec.scala new file mode 100644 index 0000000000000..2a19ff304a9e0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RefreshTableExec.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} + +case class RefreshTableExec( + catalog: TableCatalog, + ident: Identifier) extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { + catalog.invalidateTable(ident) + Seq.empty + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 2ea26787dbb19..463147903c923 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1112,6 +1112,20 @@ class DataSourceV2SQLSuite } } + test("REFRESH TABLE: v2 table") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + val testCatalog = catalog("testcat").asTableCatalog.asInstanceOf[InMemoryTableCatalog] + val identifier = Identifier.of(Array("ns1", "ns2"), "tbl") + + assert(!testCatalog.isTableInvalidated(identifier)) + sql(s"REFRESH TABLE $t") + assert(testCatalog.isTableInvalidated(identifier)) + } + } + test("REPLACE TABLE: v1 table") { val e = intercept[AnalysisException] { sql(s"CREATE OR REPLACE TABLE tbl (a int) USING ${classOf[SimpleScanSource].getName}") @@ -1211,16 +1225,8 @@ class DataSourceV2SQLSuite val t = "testcat.ns1.ns2.tbl" withTable(t) { spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") - - val e = intercept[AnalysisException] { - sql(s"ANALYZE TABLE $t COMPUTE STATISTICS") - } - assert(e.message.contains("ANALYZE TABLE is only supported with v1 tables")) - - val e2 = intercept[AnalysisException] { - sql(s"ANALYZE TABLE $t COMPUTE STATISTICS FOR ALL COLUMNS") - } - assert(e2.message.contains("ANALYZE TABLE is only supported with v1 tables")) + testV1Command("ANALYZE TABLE", s"$t COMPUTE STATISTICS") + testV1Command("ANALYZE TABLE", s"$t COMPUTE STATISTICS FOR ALL COLUMNS") } } @@ -1228,11 +1234,7 @@ class DataSourceV2SQLSuite val t = "testcat.ns1.ns2.tbl" withTable(t) { spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") - - val e = intercept[AnalysisException] { - sql(s"MSCK REPAIR TABLE $t") - } - assert(e.message.contains("MSCK REPAIR TABLE is only supported with v1 tables")) + testV1Command("MSCK REPAIR TABLE", t) } } @@ -1246,15 +1248,8 @@ class DataSourceV2SQLSuite |PARTITIONED BY (id) """.stripMargin) - val e1 = intercept[AnalysisException] { - sql(s"TRUNCATE TABLE $t") - } - assert(e1.message.contains("TRUNCATE TABLE is only supported with v1 tables")) - - val e2 = intercept[AnalysisException] { - sql(s"TRUNCATE TABLE $t PARTITION(id='1')") - } - assert(e2.message.contains("TRUNCATE TABLE is only supported with v1 tables")) + testV1Command("TRUNCATE TABLE", t) + testV1Command("TRUNCATE TABLE", s"$t PARTITION(id='1')") } } @@ -1268,16 +1263,16 @@ class DataSourceV2SQLSuite |PARTITIONED BY (id) """.stripMargin) - val e1 = intercept[AnalysisException] { - val partition = sql(s"SHOW PARTITIONS $t") - } - assert(e1.message.contains("SHOW PARTITIONS is only supported with v1 tables")) + testV1Command("SHOW PARTITIONS", t) + testV1Command("SHOW PARTITIONS", s"$t PARTITION(id='1')") + } + } - val e2 = intercept[AnalysisException] { - val partition2 = sql(s"SHOW PARTITIONS $t PARTITION(id='1')") - } - assert(e2.message.contains("SHOW PARTITIONS is only supported with v1 tables")) + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { + val e = intercept[AnalysisException] { + sql(s"$sqlCommand $sqlParams") } + assert(e.message.contains(s"$sqlCommand is only supported with v1 tables")) } private def assertAnalysisError(sqlStatement: String, expectedError: String): Unit = { From bfbf2821f34afba1c3a8a720084b5421a9de77eb Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 24 Oct 2019 00:41:48 +0800 Subject: [PATCH 0405/1387] [SPARK-29503][SQL] Remove conversion CreateNamedStruct to CreateNamedStructUnsafe ### What changes were proposed in this pull request? There's a case where MapObjects has a lambda function which creates nested struct - unsafe data in safe data struct. In this case, MapObjects doesn't copy the row returned from lambda function (as outmost data type is safe data struct), which misses copying nested unsafe data. The culprit is that `UnsafeProjection.toUnsafeExprs` converts `CreateNamedStruct` to `CreateNamedStructUnsafe` (this is the only place where `CreateNamedStructUnsafe` is used) which incurs safe and unsafe being mixed up temporarily, which may not be needed at all at least logically, as it will finally assembly these evaluations to `UnsafeRow`. > Before the patch ``` /* 105 */ private ArrayData MapObjects_0(InternalRow i) { /* 106 */ boolean isNull_1 = i.isNullAt(0); /* 107 */ ArrayData value_1 = isNull_1 ? /* 108 */ null : (i.getArray(0)); /* 109 */ ArrayData value_0 = null; /* 110 */ /* 111 */ if (!isNull_1) { /* 112 */ /* 113 */ int dataLength_0 = value_1.numElements(); /* 114 */ /* 115 */ ArrayData[] convertedArray_0 = null; /* 116 */ convertedArray_0 = new ArrayData[dataLength_0]; /* 117 */ /* 118 */ /* 119 */ int loopIndex_0 = 0; /* 120 */ /* 121 */ while (loopIndex_0 < dataLength_0) { /* 122 */ value_MapObject_lambda_variable_1 = (int) (value_1.getInt(loopIndex_0)); /* 123 */ isNull_MapObject_lambda_variable_1 = value_1.isNullAt(loopIndex_0); /* 124 */ /* 125 */ ArrayData arrayData_0 = ArrayData.allocateArrayData( /* 126 */ -1, 1L, " createArray failed."); /* 127 */ /* 128 */ mutableStateArray_0[0].reset(); /* 129 */ /* 130 */ /* 131 */ mutableStateArray_0[0].zeroOutNullBytes(); /* 132 */ /* 133 */ /* 134 */ if (isNull_MapObject_lambda_variable_1) { /* 135 */ mutableStateArray_0[0].setNullAt(0); /* 136 */ } else { /* 137 */ mutableStateArray_0[0].write(0, value_MapObject_lambda_variable_1); /* 138 */ } /* 139 */ arrayData_0.update(0, (mutableStateArray_0[0].getRow())); /* 140 */ if (false) { /* 141 */ convertedArray_0[loopIndex_0] = null; /* 142 */ } else { /* 143 */ convertedArray_0[loopIndex_0] = arrayData_0 instanceof UnsafeArrayData? arrayData_0.copy() : arrayData_0; /* 144 */ } /* 145 */ /* 146 */ loopIndex_0 += 1; /* 147 */ } /* 148 */ /* 149 */ value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(convertedArray_0); /* 150 */ } /* 151 */ globalIsNull_0 = isNull_1; /* 152 */ return value_0; /* 153 */ } ``` > After the patch ``` /* 104 */ private ArrayData MapObjects_0(InternalRow i) { /* 105 */ boolean isNull_1 = i.isNullAt(0); /* 106 */ ArrayData value_1 = isNull_1 ? /* 107 */ null : (i.getArray(0)); /* 108 */ ArrayData value_0 = null; /* 109 */ /* 110 */ if (!isNull_1) { /* 111 */ /* 112 */ int dataLength_0 = value_1.numElements(); /* 113 */ /* 114 */ ArrayData[] convertedArray_0 = null; /* 115 */ convertedArray_0 = new ArrayData[dataLength_0]; /* 116 */ /* 117 */ /* 118 */ int loopIndex_0 = 0; /* 119 */ /* 120 */ while (loopIndex_0 < dataLength_0) { /* 121 */ value_MapObject_lambda_variable_1 = (int) (value_1.getInt(loopIndex_0)); /* 122 */ isNull_MapObject_lambda_variable_1 = value_1.isNullAt(loopIndex_0); /* 123 */ /* 124 */ ArrayData arrayData_0 = ArrayData.allocateArrayData( /* 125 */ -1, 1L, " createArray failed."); /* 126 */ /* 127 */ Object[] values_0 = new Object[1]; /* 128 */ /* 129 */ /* 130 */ if (isNull_MapObject_lambda_variable_1) { /* 131 */ values_0[0] = null; /* 132 */ } else { /* 133 */ values_0[0] = value_MapObject_lambda_variable_1; /* 134 */ } /* 135 */ /* 136 */ final InternalRow value_3 = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(values_0); /* 137 */ values_0 = null; /* 138 */ arrayData_0.update(0, value_3); /* 139 */ if (false) { /* 140 */ convertedArray_0[loopIndex_0] = null; /* 141 */ } else { /* 142 */ convertedArray_0[loopIndex_0] = arrayData_0 instanceof UnsafeArrayData? arrayData_0.copy() : arrayData_0; /* 143 */ } /* 144 */ /* 145 */ loopIndex_0 += 1; /* 146 */ } /* 147 */ /* 148 */ value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(convertedArray_0); /* 149 */ } /* 150 */ globalIsNull_0 = isNull_1; /* 151 */ return value_0; /* 152 */ } ``` ### Why are the changes needed? This patch fixes the bug described above. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? UT added which fails on master branch and passes on PR. Closes #26173 from HeartSaVioR/SPARK-29503. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/Projection.scala | 8 +--- .../expressions/complexTypeCreator.scala | 48 +++++-------------- .../sql/catalyst/optimizer/ComplexTypes.scala | 6 +-- .../optimizer/NormalizeFloatingNumbers.scala | 5 +- .../sql/catalyst/optimizer/expressions.scala | 4 +- .../expressions/ComplexTypeSuite.scala | 1 - .../scala/org/apache/spark/sql/Column.scala | 2 +- .../spark/sql/DataFrameComplexTypeSuite.scala | 22 +++++++++ 8 files changed, 43 insertions(+), 53 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index eaaf94baac216..300f075d32763 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -127,12 +127,6 @@ object UnsafeProjection InterpretedUnsafeProjection.createProjection(in) } - protected def toUnsafeExprs(exprs: Seq[Expression]): Seq[Expression] = { - exprs.map(_ transform { - case CreateNamedStruct(children) => CreateNamedStructUnsafe(children) - }) - } - /** * Returns an UnsafeProjection for given StructType. * @@ -153,7 +147,7 @@ object UnsafeProjection * Returns an UnsafeProjection for given sequence of bound Expressions. */ def create(exprs: Seq[Expression]): UnsafeProjection = { - createObject(toUnsafeExprs(exprs)) + createObject(exprs) } def create(expr: Expression): UnsafeProjection = create(Seq(expr)) 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 cae3c0528e136..3f722e8537c36 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 @@ -295,9 +295,20 @@ object CreateStruct extends FunctionBuilder { } /** - * Common base class for both [[CreateNamedStruct]] and [[CreateNamedStructUnsafe]]. + * Creates a struct with the given field names and values + * + * @param children Seq(name1, val1, name2, val2, ...) */ -trait CreateNamedStructLike extends Expression { +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.", + examples = """ + Examples: + > SELECT _FUNC_("a", 1, "b", 2, "c", 3); + {"a":1,"b":2,"c":3} + """) +// scalastyle:on line.size.limit +case class CreateNamedStruct(children: Seq[Expression]) extends Expression { lazy val (nameExprs, valExprs) = children.grouped(2).map { case Seq(name, value) => (name, value) }.toList.unzip @@ -348,23 +359,6 @@ trait CreateNamedStructLike extends Expression { override def eval(input: InternalRow): Any = { InternalRow(valExprs.map(_.eval(input)): _*) } -} - -/** - * Creates a struct with the given field names and values - * - * @param children Seq(name1, val1, name2, val2, ...) - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.", - examples = """ - Examples: - > SELECT _FUNC_("a", 1, "b", 2, "c", 3); - {"a":1,"b":2,"c":3} - """) -// scalastyle:on line.size.limit -case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStructLike { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val rowClass = classOf[GenericInternalRow].getName @@ -397,22 +391,6 @@ case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStruc override def prettyName: String = "named_struct" } -/** - * Creates a struct with the given field names and values. This is a variant that returns - * UnsafeRow directly. The unsafe projection operator replaces [[CreateStruct]] with - * this expression automatically at runtime. - * - * @param children Seq(name1, val1, name2, val2, ...) - */ -case class CreateNamedStructUnsafe(children: Seq[Expression]) extends CreateNamedStructLike { - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val eval = GenerateUnsafeProjection.createCode(ctx, valExprs) - ExprCode(code = eval.code, isNull = FalseLiteral, value = eval.value) - } - - override def prettyName: String = "named_struct_unsafe" -} - /** * Creates a map after splitting the input text into key/value pairs using delimiters */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala index db7d6d3254bd2..1743565ccb6c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule /** - * Simplify redundant [[CreateNamedStructLike]], [[CreateArray]] and [[CreateMap]] expressions. + * Simplify redundant [[CreateNamedStruct]], [[CreateArray]] and [[CreateMap]] expressions. */ object SimplifyExtractValueOps extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -37,8 +37,8 @@ object SimplifyExtractValueOps extends Rule[LogicalPlan] { case a: Aggregate => a case p => p.transformExpressionsUp { // Remove redundant field extraction. - case GetStructField(createNamedStructLike: CreateNamedStructLike, ordinal, _) => - createNamedStructLike.valExprs(ordinal) + case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) => + createNamedStruct.valExprs(ordinal) // Remove redundant array indexing. case GetArrayStructFields(CreateArray(elems), field, ordinal, _, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala index b036092cf1fcc..ea01d9e63eef7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, KnownFloatingPointNormalized, LambdaFunction, NamedLambdaVariable, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, KnownFloatingPointNormalized, LambdaFunction, NamedLambdaVariable, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, Window} @@ -114,9 +114,6 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { case CreateNamedStruct(children) => CreateNamedStruct(children.map(normalize)) - case CreateNamedStructUnsafe(children) => - CreateNamedStructUnsafe(children.map(normalize)) - case CreateArray(children) => CreateArray(children.map(normalize)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 0a6737ba42118..36ad796c08a38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -227,8 +227,8 @@ object OptimizeIn extends Rule[LogicalPlan] { if (newList.length == 1 // TODO: `EqualTo` for structural types are not working. Until SPARK-24443 is addressed, // TODO: we exclude them in this rule. - && !v.isInstanceOf[CreateNamedStructLike] - && !newList.head.isInstanceOf[CreateNamedStructLike]) { + && !v.isInstanceOf[CreateNamedStruct] + && !newList.head.isInstanceOf[CreateNamedStruct]) { EqualTo(v, newList.head) } else if (newList.length > SQLConf.get.optimizerInSetConversionThreshold) { val hSet = newList.map(e => e.eval(EmptyRow)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 0c4438987cd2a..9039cd6451590 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -369,7 +369,6 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val b = AttributeReference("b", IntegerType)() checkMetadata(CreateStruct(Seq(a, b))) checkMetadata(CreateNamedStruct(Seq("a", a, "b", b))) - checkMetadata(CreateNamedStructUnsafe(Seq("a", a, "b", b))) } test("StringToMap") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 7b903a3f7f148..ed10843b08596 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -200,7 +200,7 @@ class Column(val expr: Expression) extends Logging { UnresolvedAlias(a, Some(Column.generateAlias)) // Wait until the struct is resolved. This will generate a nicer looking alias. - case struct: CreateNamedStructLike => UnresolvedAlias(struct) + case struct: CreateNamedStruct => UnresolvedAlias(struct) case expr: Expression => Alias(expr, toPrettySQL(expr))() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index e9179a39d3b6d..4f25642906628 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -18,8 +18,12 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.DefinedByConstructorParams +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.objects.MapObjects import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.ArrayType /** * A test suite to test DataFrame/SQL functionalities with complex types (i.e. array, struct, map). @@ -64,6 +68,24 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession { val ds100_5 = Seq(S100_5()).toDS() ds100_5.rdd.count } + + test("SPARK-29503 nest unsafe struct inside safe array") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val df = spark.sparkContext.parallelize(Seq(Seq(1, 2, 3))).toDF("items") + + // items: Seq[Int] => items.map { item => Seq(Struct(item)) } + val result = df.select( + new Column(MapObjects( + (item: Expression) => array(struct(new Column(item))).expr, + $"items".expr, + df.schema("items").dataType.asInstanceOf[ArrayType].elementType + )) as "items" + ).collect() + + assert(result.size === 1) + assert(result === Row(Seq(Seq(Row(1)), Seq(Row(2)), Seq(Row(3)))) :: Nil) + } + } } class S100( From 7e8e4c0a146ef071808a611e256ab049b396212a Mon Sep 17 00:00:00 2001 From: jiake Date: Thu, 24 Oct 2019 01:18:07 +0800 Subject: [PATCH 0406/1387] [SPARK-29552][SQL] Execute the "OptimizeLocalShuffleReader" rule when creating new query stage and then can optimize the shuffle reader to local shuffle reader as much as possible ### What changes were proposed in this pull request? `OptimizeLocalShuffleReader` rule is very conservative and gives up optimization as long as there are extra shuffles introduced. It's very likely that most of the added local shuffle readers are fine and only one introduces extra shuffle. However, it's very hard to make `OptimizeLocalShuffleReader` optimal, a simple workaround is to run this rule again right before executing a query stage. ### Why are the changes needed? Optimize more shuffle reader to local shuffle reader. ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing ut Closes #26207 from JkSelf/resolve-multi-joins-issue. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../execution/adaptive/AdaptiveSparkPlanExec.scala | 9 +++++++++ .../execution/adaptive/AdaptiveQueryExecSuite.scala | 11 +++++++---- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index f45e3560b2cf1..f01947d8f5ed6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -92,6 +92,15 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReuseAdaptiveSubquery(conf, subqueryCache), + + // When adding local shuffle readers in 'OptimizeLocalShuffleReader`, we revert all the local + // readers if additional shuffles are introduced. This may be too conservative: maybe there is + // only one local reader that introduces shuffle, and we can still keep other local readers. + // Here we re-execute this rule with the sub-plan-tree of a query stage, to make sure necessary + // local readers are added before executing the query stage. + // This rule must be executed before `ReduceNumShufflePartitions`, as local shuffle readers + // can't change number of partitions. + OptimizeLocalShuffleReader(conf), ReduceNumShufflePartitions(conf), ApplyColumnarRulesAndInsertTransitions(session.sessionState.conf, session.sessionState.columnarRules), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 43802968c4696..649467a27d93c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -163,8 +163,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - // additional shuffle exchange introduced, only one shuffle reader to local shuffle reader. - checkNumLocalShuffleReaders(adaptivePlan, 1) + // The child of remaining one BroadcastHashJoin is not ShuffleQueryStage. + // So only two LocalShuffleReader. + checkNumLocalShuffleReaders(adaptivePlan, 2) } } @@ -188,7 +189,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - // additional shuffle exchange introduced, only one shuffle reader to local shuffle reader. + // The child of remaining two BroadcastHashJoin is not ShuffleQueryStage. + // So only two LocalShuffleReader. checkNumLocalShuffleReaders(adaptivePlan, 1) } } @@ -213,7 +215,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - // additional shuffle exchange introduced, only one shuffle reader to local shuffle reader. + // The child of remaining two BroadcastHashJoin is not ShuffleQueryStage. + // So only two LocalShuffleReader. checkNumLocalShuffleReaders(adaptivePlan, 1) } } From 5867707835a442827c9df17f79c28c86eb9f2c68 Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Wed, 23 Oct 2019 10:45:11 -0700 Subject: [PATCH 0407/1387] [SPARK-29557][BUILD] Update dropwizard/codahale metrics library to 3.2.6 ### What changes were proposed in this pull request? This proposes to update the dropwizard/codahale metrics library version used by Spark to `3.2.6` which is the last version supporting Ganglia. ### Why are the changes needed? Spark is currently using Dropwizard metrics version 3.1.5, a version that is no more actively developed nor maintained, according to the project's Github repo README. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests + manual tests on a YARN cluster. Closes #26212 from LucaCanali/updateDropwizardVersion. Authored-by: Luca Canali Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 8 ++++---- dev/deps/spark-deps-hadoop-3.2 | 8 ++++---- pom.xml | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 26dc6e7bd8bf9..f21e76bf4331a 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -149,10 +149,10 @@ lz4-java-1.6.0.jar machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.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 +metrics-core-3.2.6.jar +metrics-graphite-3.2.6.jar +metrics-json-3.2.6.jar +metrics-jvm-3.2.6.jar minlog-1.3.0.jar netty-all-4.1.42.Final.jar objenesis-2.5.1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index a92b7124cb4af..3ecc3c2b0d35a 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -179,10 +179,10 @@ lz4-java-1.6.0.jar machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.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 +metrics-core-3.2.6.jar +metrics-graphite-3.2.6.jar +metrics-json-3.2.6.jar +metrics-jvm-3.2.6.jar minlog-1.3.0.jar mssql-jdbc-6.2.1.jre7.jar netty-all-4.1.42.Final.jar diff --git a/pom.xml b/pom.xml index 69b5b79b7b071..c42ef5c6626d1 100644 --- a/pom.xml +++ b/pom.xml @@ -148,7 +148,7 @@ 0.9.3 2.4.0 2.0.8 - 3.1.5 + 3.2.6 1.8.2 hadoop2 1.8.10 From b91356e4c2a5c4a2e77c78a05a93a9d3979f1fce Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 23 Oct 2019 21:41:05 +0000 Subject: [PATCH 0408/1387] [SPARK-29533][SQL][TESTS][FOLLOWUP] Regenerate the result on EC2 ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/26189 to regenerate the result on EC2. ### Why are the changes needed? This will be used for the other PR reviews. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A. Closes #26233 from dongjoon-hyun/SPARK-29533. Authored-by: Dongjoon Hyun Signed-off-by: DB Tsai --- .../IntervalBenchmark-jdk11-results.txt | 44 +++++++++---------- .../benchmarks/IntervalBenchmark-results.txt | 44 +++++++++---------- 2 files changed, 44 insertions(+), 44 deletions(-) diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt index 2a3903200a8ac..6605bd2c13b47 100644 --- a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt @@ -1,25 +1,25 @@ -OpenJDK 64-Bit Server VM 11.0.2+9 on Mac OS X 10.15 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -string w/ interval 471 513 57 2.1 470.7 1.0X -string w/o interval 437 444 8 2.3 436.9 1.1X -1 units w/ interval 726 758 45 1.4 726.3 0.6X -1 units w/o interval 712 717 5 1.4 711.7 0.7X -2 units w/ interval 926 935 12 1.1 925.9 0.5X -2 units w/o interval 943 947 3 1.1 943.4 0.5X -3 units w/ interval 1089 1116 31 0.9 1089.0 0.4X -3 units w/o interval 1105 1108 3 0.9 1105.1 0.4X -4 units w/ interval 1260 1261 1 0.8 1260.4 0.4X -4 units w/o interval 1276 1277 1 0.8 1275.9 0.4X -5 units w/ interval 1436 1445 11 0.7 1435.6 0.3X -5 units w/o interval 1455 1463 6 0.7 1455.5 0.3X -6 units w/ interval 1634 1639 4 0.6 1634.4 0.3X -6 units w/o interval 1642 1644 3 0.6 1641.7 0.3X -7 units w/ interval 1829 1838 8 0.5 1828.6 0.3X -7 units w/o interval 1850 1853 4 0.5 1849.5 0.3X -8 units w/ interval 2065 2070 5 0.5 2065.4 0.2X -8 units w/o interval 2070 2090 21 0.5 2070.0 0.2X -9 units w/ interval 2279 2290 10 0.4 2278.7 0.2X -9 units w/o interval 2276 2285 8 0.4 2275.7 0.2X +string w/ interval 663 758 99 1.5 663.2 1.0X +string w/o interval 563 582 19 1.8 563.2 1.2X +1 units w/ interval 891 953 97 1.1 891.2 0.7X +1 units w/o interval 894 905 15 1.1 893.6 0.7X +2 units w/ interval 1142 1169 24 0.9 1141.8 0.6X +2 units w/o interval 1195 1200 7 0.8 1194.7 0.6X +3 units w/ interval 1449 1452 3 0.7 1449.5 0.5X +3 units w/o interval 1489 1491 1 0.7 1489.3 0.4X +4 units w/ interval 1688 1690 1 0.6 1688.4 0.4X +4 units w/o interval 1711 1722 11 0.6 1710.9 0.4X +5 units w/ interval 1961 1983 23 0.5 1961.3 0.3X +5 units w/o interval 1983 1991 10 0.5 1983.4 0.3X +6 units w/ interval 2217 2228 11 0.5 2216.9 0.3X +6 units w/o interval 2240 2244 5 0.4 2239.6 0.3X +7 units w/ interval 2454 2468 16 0.4 2454.1 0.3X +7 units w/o interval 2480 2491 15 0.4 2479.5 0.3X +8 units w/ interval 2762 2792 26 0.4 2761.9 0.2X +8 units w/o interval 2763 2778 14 0.4 2762.9 0.2X +9 units w/ interval 3036 3060 21 0.3 3036.4 0.2X +9 units w/o interval 3095 3111 15 0.3 3094.8 0.2X diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt index 9010b980c07b5..40169826cc627 100644 --- a/sql/core/benchmarks/IntervalBenchmark-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -1,25 +1,25 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.15 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -string w/ interval 420 435 18 2.4 419.8 1.0X -string w/o interval 359 365 10 2.8 358.7 1.2X -1 units w/ interval 752 759 8 1.3 752.0 0.6X -1 units w/o interval 762 766 4 1.3 762.0 0.6X -2 units w/ interval 961 970 8 1.0 960.7 0.4X -2 units w/o interval 970 976 9 1.0 970.2 0.4X -3 units w/ interval 1130 1136 7 0.9 1130.4 0.4X -3 units w/o interval 1150 1158 9 0.9 1150.3 0.4X -4 units w/ interval 1333 1336 3 0.7 1333.5 0.3X -4 units w/o interval 1354 1359 4 0.7 1354.5 0.3X -5 units w/ interval 1523 1525 2 0.7 1523.3 0.3X -5 units w/o interval 1549 1551 3 0.6 1549.4 0.3X -6 units w/ interval 1661 1663 2 0.6 1660.8 0.3X -6 units w/o interval 1691 1704 13 0.6 1691.2 0.2X -7 units w/ interval 1811 1817 8 0.6 1810.6 0.2X -7 units w/o interval 1853 1854 1 0.5 1853.2 0.2X -8 units w/ interval 2029 2037 8 0.5 2028.7 0.2X -8 units w/o interval 2075 2075 1 0.5 2074.5 0.2X -9 units w/ interval 2170 2175 5 0.5 2170.0 0.2X -9 units w/o interval 2204 2212 8 0.5 2203.6 0.2X +string w/ interval 600 641 37 1.7 600.3 1.0X +string w/o interval 536 544 12 1.9 536.4 1.1X +1 units w/ interval 1017 1027 9 1.0 1016.8 0.6X +1 units w/o interval 1055 1073 16 0.9 1054.7 0.6X +2 units w/ interval 1272 1292 29 0.8 1272.2 0.5X +2 units w/o interval 1309 1314 9 0.8 1309.0 0.5X +3 units w/ interval 1545 1566 20 0.6 1544.8 0.4X +3 units w/o interval 1606 1610 5 0.6 1605.8 0.4X +4 units w/ interval 1820 1826 6 0.5 1819.7 0.3X +4 units w/o interval 1882 1885 3 0.5 1881.5 0.3X +5 units w/ interval 2039 2043 7 0.5 2038.9 0.3X +5 units w/o interval 2131 2133 3 0.5 2130.6 0.3X +6 units w/ interval 2269 2272 4 0.4 2269.5 0.3X +6 units w/o interval 2327 2333 6 0.4 2327.2 0.3X +7 units w/ interval 2477 2485 10 0.4 2476.8 0.2X +7 units w/o interval 2536 2538 3 0.4 2536.0 0.2X +8 units w/ interval 2764 2781 27 0.4 2763.8 0.2X +8 units w/o interval 2843 2847 5 0.4 2842.9 0.2X +9 units w/ interval 2983 2997 12 0.3 2982.5 0.2X +9 units w/o interval 3071 3072 1 0.3 3071.1 0.2X From 7ecf968527a63bc5bd2397ed04f1149dd07821ca Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 23 Oct 2019 16:44:10 -0700 Subject: [PATCH 0409/1387] [SPARK-29567][TESTS] Update JDBC Integration Test Docker Images ### What changes were proposed in this pull request? This PR updates JDBC Integration Test DBMS Docker Images. | DBMS | Docker Image Tag | Release | | ------ | ------------------ | ------ | | MySQL | mysql:5.7.28 | Oct 13, 2019 | | PostgreSQL | postgres:12.0-alpine | Oct 3, 2019 | * For `MySQL`, `SET GLOBAL sql_mode = ''` is added to disable all strict modes because `test("Basic write test")` creates a table like the following. The latest MySQL rejects `0000-00-00 00:00:00` as TIMESTAMP and causes the test case failure. ``` mysql> desc datescopy; +-------+-----------+------+-----+---------------------+-----------------------------+ | Field | Type | Null | Key | Default | Extra | +-------+-----------+------+-----+---------------------+-----------------------------+ | d | date | YES | | NULL | | | t | timestamp | NO | | CURRENT_TIMESTAMP | on update CURRENT_TIMESTAMP | | dt | timestamp | NO | | 0000-00-00 00:00:00 | | | ts | timestamp | NO | | 0000-00-00 00:00:00 | | | yr | date | YES | | NULL | | +-------+-----------+------+-----+---------------------+-----------------------------+ ``` * For `PostgreSQL`, I chose the smallest image in `12` releases. It reduces the image size a lot, `312MB` -> `72.8MB`. This is good for CI/CI testing environment. ``` $ docker images | grep postgres postgres 12.0-alpine 5b681acb1cfc 2 days ago 72.8MB postgres 11.4 53912975086f 3 months ago 312MB ``` Note that - For `MsSqlServer`, we are using `2017-GA-ubuntu` and the next version `2019-CTP3.2-ubuntu` is still `Community Technology Preview` status. - For `DB2` and `Oracle`, the official images are not available. ### Why are the changes needed? This is to make it sure we are testing with the latest DBMS images during preparing `3.0.0`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Since this is the integration test, we need to run this manually. ``` build/mvn install -DskipTests build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12 test ``` Closes #26224 from dongjoon-hyun/SPARK-29567. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala | 4 +++- .../org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index 9cd5c4ec41a52..bba1b5275269b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "mysql:5.7.9" + override val imageName = "mysql:5.7.28" override val env = Map( "MYSQL_ROOT_PASSWORD" -> "rootpass" ) @@ -39,6 +39,8 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { } override def dataPreparation(conn: Connection): Unit = { + // Since MySQL 5.7.14+, we need to disable strict mode + conn.prepareStatement("SET GLOBAL sql_mode = ''").executeUpdate() conn.prepareStatement("CREATE DATABASE foo").executeUpdate() conn.prepareStatement("CREATE TABLE tbl (x INTEGER, y TEXT(8))").executeUpdate() conn.prepareStatement("INSERT INTO tbl VALUES (42,'fred')").executeUpdate() diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 89da9a1de6f74..599f00def0750 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "postgres:11.4" + override val imageName = "postgres:12.0-alpine" override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) From fd899d6331f4777a36a3f2a79a6b2fa123dccc1a Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Wed, 23 Oct 2019 18:17:37 -0700 Subject: [PATCH 0410/1387] [SPARK-29576][CORE] Use Spark's CompressionCodec for Ser/Deser of MapOutputStatus ### What changes were proposed in this pull request? Instead of using ZStd codec directly, we use Spark's CompressionCodec which wraps ZStd codec in a buffered stream to avoid overhead excessive of JNI call while trying to compress/decompress small amount of data. Also, by using Spark's CompressionCodec, we can easily to make it configurable in the future if it's needed. ### Why are the changes needed? Faster performance. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26235 from dbtsai/optimizeDeser. Lead-authored-by: DB Tsai Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- ...tatusesSerDeserBenchmark-jdk11-results.txt | 38 +++++----- .../MapStatusesSerDeserBenchmark-results.txt | 38 +++++----- .../org/apache/spark/MapOutputTracker.scala | 70 ++++++++----------- .../spark/MapStatusesSerDeserBenchmark.scala | 9 +-- 4 files changed, 72 insertions(+), 83 deletions(-) diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt index 7a6cfb7b23b94..db23cf5c12ea7 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-jdk11-results.txt @@ -2,10 +2,10 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 205 213 13 1.0 1023.6 1.0X -Deserialization 908 939 27 0.2 4540.2 0.2X +Serialization 170 178 9 1.2 849.7 1.0X +Deserialization 530 535 9 0.4 2651.1 0.3X -Compressed Serialized MapStatus sizes: 400 bytes +Compressed Serialized MapStatus sizes: 411 bytes Compressed Serialized Broadcast MapStatus sizes: 2 MB @@ -13,8 +13,8 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 195 204 24 1.0 976.9 1.0X -Deserialization 913 940 33 0.2 4566.7 0.2X +Serialization 157 165 7 1.3 785.4 1.0X +Deserialization 495 588 79 0.4 2476.7 0.3X Compressed Serialized MapStatus sizes: 2 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes @@ -24,21 +24,21 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 616 619 3 0.3 3079.1 1.0X -Deserialization 936 954 22 0.2 4680.5 0.7X +Serialization 344 351 4 0.6 1720.4 1.0X +Deserialization 527 579 99 0.4 2635.9 0.7X -Compressed Serialized MapStatus sizes: 418 bytes -Compressed Serialized Broadcast MapStatus sizes: 14 MB +Compressed Serialized MapStatus sizes: 427 bytes +Compressed Serialized Broadcast MapStatus sizes: 13 MB OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 586 588 3 0.3 2928.8 1.0X -Deserialization 929 933 4 0.2 4647.0 0.6X +Serialization 317 321 4 0.6 1583.8 1.0X +Deserialization 530 540 15 0.4 2648.3 0.6X -Compressed Serialized MapStatus sizes: 14 MB +Compressed Serialized MapStatus sizes: 13 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes @@ -46,21 +46,21 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 4740 4916 249 0.0 23698.5 1.0X -Deserialization 1578 1597 27 0.1 7890.6 3.0X +Serialization 1738 1849 156 0.1 8692.0 1.0X +Deserialization 946 977 33 0.2 4730.2 1.8X -Compressed Serialized MapStatus sizes: 546 bytes -Compressed Serialized Broadcast MapStatus sizes: 123 MB +Compressed Serialized MapStatus sizes: 556 bytes +Compressed Serialized Broadcast MapStatus sizes: 121 MB OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 4492 4573 115 0.0 22458.3 1.0X -Deserialization 1533 1547 20 0.1 7664.8 2.9X +Serialization 1379 1432 76 0.1 6892.6 1.0X +Deserialization 929 941 19 0.2 4645.5 1.5X -Compressed Serialized MapStatus sizes: 123 MB +Compressed Serialized MapStatus sizes: 121 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes diff --git a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt index 0c649694f6b6e..053f4bf771923 100644 --- a/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt +++ b/core/benchmarks/MapStatusesSerDeserBenchmark-results.txt @@ -2,10 +2,10 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 236 245 18 0.8 1179.1 1.0X -Deserialization 842 885 37 0.2 4211.4 0.3X +Serialization 178 187 15 1.1 887.5 1.0X +Deserialization 530 558 32 0.4 2647.5 0.3X -Compressed Serialized MapStatus sizes: 400 bytes +Compressed Serialized MapStatus sizes: 411 bytes Compressed Serialized Broadcast MapStatus sizes: 2 MB @@ -13,8 +13,8 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 10 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 213 219 8 0.9 1065.1 1.0X -Deserialization 846 870 33 0.2 4228.6 0.3X +Serialization 167 175 7 1.2 835.7 1.0X +Deserialization 523 537 22 0.4 2616.2 0.3X Compressed Serialized MapStatus sizes: 2 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes @@ -24,21 +24,21 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 624 709 167 0.3 3121.1 1.0X -Deserialization 885 908 22 0.2 4427.0 0.7X +Serialization 351 416 147 0.6 1754.4 1.0X +Deserialization 546 551 8 0.4 2727.6 0.6X -Compressed Serialized MapStatus sizes: 418 bytes -Compressed Serialized Broadcast MapStatus sizes: 14 MB +Compressed Serialized MapStatus sizes: 427 bytes +Compressed Serialized Broadcast MapStatus sizes: 13 MB OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 100 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 603 604 2 0.3 3014.9 1.0X -Deserialization 892 895 5 0.2 4458.7 0.7X +Serialization 320 321 1 0.6 1598.0 1.0X +Deserialization 542 549 7 0.4 2709.0 0.6X -Compressed Serialized MapStatus sizes: 14 MB +Compressed Serialized MapStatus sizes: 13 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes @@ -46,21 +46,21 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/ broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 4612 4945 471 0.0 23061.0 1.0X -Deserialization 1493 1495 2 0.1 7466.3 3.1X +Serialization 1671 1877 290 0.1 8357.3 1.0X +Deserialization 943 970 32 0.2 4715.8 1.8X -Compressed Serialized MapStatus sizes: 546 bytes -Compressed Serialized Broadcast MapStatus sizes: 123 MB +Compressed Serialized MapStatus sizes: 556 bytes +Compressed Serialized Broadcast MapStatus sizes: 121 MB OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200000 MapOutputs, 1000 blocks w/o broadcast: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Serialization 4452 4595 202 0.0 22261.4 1.0X -Deserialization 1464 1477 18 0.1 7321.4 3.0X +Serialization 1373 1436 89 0.1 6865.0 1.0X +Deserialization 940 970 37 0.2 4699.1 1.5X -Compressed Serialized MapStatus sizes: 123 MB +Compressed Serialized MapStatus sizes: 121 MB Compressed Serialized Broadcast MapStatus sizes: 0 bytes diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 6f4a6239a09ed..873efa76468ed 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -28,13 +28,12 @@ import scala.concurrent.duration.Duration import scala.reflect.ClassTag import scala.util.control.NonFatal -import com.github.luben.zstd.ZstdInputStream -import com.github.luben.zstd.ZstdOutputStream import org.apache.commons.io.output.{ByteArrayOutputStream => ApacheByteArrayOutputStream} import org.apache.spark.broadcast.{Broadcast, BroadcastManager} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.io.CompressionCodec import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{ExecutorCacheTaskLocation, MapStatus} import org.apache.spark.shuffle.MetadataFetchFailedException @@ -195,7 +194,8 @@ private class ShuffleStatus(numPartitions: Int) { def serializedMapStatus( broadcastManager: BroadcastManager, isLocal: Boolean, - minBroadcastSize: Int): Array[Byte] = { + minBroadcastSize: Int, + conf: SparkConf): Array[Byte] = { var result: Array[Byte] = null withReadLock { @@ -207,7 +207,7 @@ private class ShuffleStatus(numPartitions: Int) { if (result == null) withWriteLock { if (cachedSerializedMapStatus == null) { val serResult = MapOutputTracker.serializeMapStatuses( - mapStatuses, broadcastManager, isLocal, minBroadcastSize) + mapStatuses, broadcastManager, isLocal, minBroadcastSize, conf) cachedSerializedMapStatus = serResult._1 cachedSerializedBroadcast = serResult._2 } @@ -450,7 +450,8 @@ private[spark] class MapOutputTrackerMaster( " to " + hostPort) val shuffleStatus = shuffleStatuses.get(shuffleId).head context.reply( - shuffleStatus.serializedMapStatus(broadcastManager, isLocal, minSizeForBroadcast)) + shuffleStatus.serializedMapStatus(broadcastManager, isLocal, minSizeForBroadcast, + conf)) } catch { case NonFatal(e) => logError(e.getMessage, e) } @@ -799,7 +800,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr endPartition: Int) : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") - val statuses = getStatuses(shuffleId) + val statuses = getStatuses(shuffleId, conf) try { MapOutputTracker.convertMapStatuses( shuffleId, startPartition, endPartition, statuses) @@ -818,7 +819,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, mapIndex $mapIndex" + s"partitions $startPartition-$endPartition") - val statuses = getStatuses(shuffleId) + val statuses = getStatuses(shuffleId, conf) try { MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses, Some(mapIndex)) @@ -836,7 +837,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr * * (It would be nice to remove this restriction in the future.) */ - private def getStatuses(shuffleId: Int): Array[MapStatus] = { + private def getStatuses(shuffleId: Int, conf: SparkConf): Array[MapStatus] = { val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") @@ -846,7 +847,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr if (fetchedStatuses == null) { logInfo("Doing the fetch; tracker endpoint = " + trackerEndpoint) val fetchedBytes = askTracker[Array[Byte]](GetMapOutputStatuses(shuffleId)) - fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes) + fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes, conf) logInfo("Got the output locations") mapStatuses.put(shuffleId, fetchedStatuses) } @@ -890,16 +891,20 @@ private[spark] object MapOutputTracker extends Logging { // Serialize an array of map output locations into an efficient byte format so that we can send // it to reduce tasks. We do this by compressing the serialized bytes using Zstd. They will // generally be pretty compressible because many map outputs will be on the same hostname. - def serializeMapStatuses(statuses: Array[MapStatus], broadcastManager: BroadcastManager, - isLocal: Boolean, minBroadcastSize: Int): (Array[Byte], Broadcast[Array[Byte]]) = { + def serializeMapStatuses( + statuses: Array[MapStatus], + broadcastManager: BroadcastManager, + isLocal: Boolean, + minBroadcastSize: Int, + conf: SparkConf): (Array[Byte], Broadcast[Array[Byte]]) = { // Using `org.apache.commons.io.output.ByteArrayOutputStream` instead of the standard one // This implementation doesn't reallocate the whole memory block but allocates // additional buffers. This way no buffers need to be garbage collected and // the contents don't have to be copied to the new buffer. val out = new ApacheByteArrayOutputStream() - val compressedOut = new ApacheByteArrayOutputStream() - - val objOut = new ObjectOutputStream(out) + out.write(DIRECT) + val codec = CompressionCodec.createCodec(conf, "zstd") + val objOut = new ObjectOutputStream(codec.compressedOutputStream(out)) Utils.tryWithSafeFinally { // Since statuses can be modified in parallel, sync on it statuses.synchronized { @@ -908,42 +913,21 @@ private[spark] object MapOutputTracker extends Logging { } { objOut.close() } - - val arr: Array[Byte] = { - val zos = new ZstdOutputStream(compressedOut) - Utils.tryWithSafeFinally { - compressedOut.write(DIRECT) - // `out.writeTo(zos)` will write the uncompressed data from `out` to `zos` - // without copying to avoid unnecessary allocation and copy of byte[]. - out.writeTo(zos) - } { - zos.close() - } - compressedOut.toByteArray - } + val arr = out.toByteArray if (arr.length >= minBroadcastSize) { // Use broadcast instead. // Important arr(0) is the tag == DIRECT, ignore that while deserializing ! val bcast = broadcastManager.newBroadcast(arr, isLocal) // toByteArray creates copy, so we can reuse out out.reset() - val oos = new ObjectOutputStream(out) + out.write(BROADCAST) + val oos = new ObjectOutputStream(codec.compressedOutputStream(out)) Utils.tryWithSafeFinally { oos.writeObject(bcast) } { oos.close() } - val outArr = { - compressedOut.reset() - val zos = new ZstdOutputStream(compressedOut) - Utils.tryWithSafeFinally { - compressedOut.write(BROADCAST) - out.writeTo(zos) - } { - zos.close() - } - compressedOut.toByteArray - } + val outArr = out.toByteArray logInfo("Broadcast mapstatuses size = " + outArr.length + ", actual size = " + arr.length) (outArr, bcast) } else { @@ -952,11 +936,15 @@ private[spark] object MapOutputTracker extends Logging { } // Opposite of serializeMapStatuses. - def deserializeMapStatuses(bytes: Array[Byte]): Array[MapStatus] = { + def deserializeMapStatuses(bytes: Array[Byte], conf: SparkConf): Array[MapStatus] = { assert (bytes.length > 0) def deserializeObject(arr: Array[Byte], off: Int, len: Int): AnyRef = { - val objIn = new ObjectInputStream(new ZstdInputStream( + val codec = CompressionCodec.createCodec(conf, "zstd") + // The ZStd codec is wrapped in a `BufferedInputStream` which avoids overhead excessive + // of JNI call while trying to decompress small amount of data for each element + // of `MapStatuses` + val objIn = new ObjectInputStream(codec.compressedInputStream( new ByteArrayInputStream(arr, off, len))) Utils.tryWithSafeFinally { objIn.readObject() diff --git a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala index 53afe141981f4..5dbef88e73a9e 100644 --- a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala @@ -67,19 +67,20 @@ object MapStatusesSerDeserBenchmark extends BenchmarkBase { var serializedBroadcastSizes = 0 val (serializedMapStatus, serializedBroadcast) = MapOutputTracker.serializeMapStatuses( - shuffleStatus.mapStatuses, tracker.broadcastManager, tracker.isLocal, minBroadcastSize) + shuffleStatus.mapStatuses, tracker.broadcastManager, tracker.isLocal, minBroadcastSize, + sc.getConf) serializedMapStatusSizes = serializedMapStatus.length if (serializedBroadcast != null) { serializedBroadcastSizes = serializedBroadcast.value.length } benchmark.addCase("Serialization") { _ => - MapOutputTracker.serializeMapStatuses( - shuffleStatus.mapStatuses, tracker.broadcastManager, tracker.isLocal, minBroadcastSize) + MapOutputTracker.serializeMapStatuses(shuffleStatus.mapStatuses, tracker.broadcastManager, + tracker.isLocal, minBroadcastSize, sc.getConf) } benchmark.addCase("Deserialization") { _ => - val result = MapOutputTracker.deserializeMapStatuses(serializedMapStatus) + val result = MapOutputTracker.deserializeMapStatuses(serializedMapStatus, sc.getConf) assert(result.length == numMaps) } From 55ced9c148a0c47134f3218ebd2f7fb5bea944eb Mon Sep 17 00:00:00 2001 From: 07ARB Date: Thu, 24 Oct 2019 15:57:16 +0900 Subject: [PATCH 0411/1387] [SPARK-29571][SQL][TESTS][FOLLOWUP] Fix UT in AllExecutionsPageSuite ### What changes were proposed in this pull request? This is a follow-up of #24052 to correct assert condition. ### Why are the changes needed? To test IllegalArgumentException condition.. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manual Test (during fixing of SPARK-29453 find this issue) Closes #26234 from 07ARB/SPARK-29571. Authored-by: 07ARB Signed-off-by: HyukjinKwon --- .../apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala index 9e42056c19a0c..298afa880c930 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala @@ -73,7 +73,7 @@ class AllExecutionsPageSuite extends SharedSparkSession with BeforeAndAfter { map.put("failed.sort", Array("duration")) when(request.getParameterMap()).thenReturn(map) val html = renderSQLPage(request, tab, statusStore).toString().toLowerCase(Locale.ROOT) - assert(!html.contains("IllegalArgumentException")) + assert(!html.contains("illegalargumentexception")) assert(html.contains("duration")) } From 177bf672e47977cbb6ccfd88f3ec77687c1fdebe Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 24 Oct 2019 15:00:21 +0800 Subject: [PATCH 0412/1387] [SPARK-29522][SQL] CACHE TABLE should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add CacheTableStatement and make CACHE TABLE go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog CACHE TABLE t // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? yes. When running CACHE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26179 from viirya/SPARK-29522. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 24 +++++++++++++++++++ .../catalyst/plans/logical/statements.scala | 9 +++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 17 +++++++++++++ .../analysis/ResolveSessionCatalog.scala | 6 ++++- .../spark/sql/execution/SparkSqlParser.scala | 15 ------------ .../sql/connector/DataSourceV2SQLSuite.scala | 14 +++++++++++ .../spark/sql/hive/CachedTableSuite.scala | 2 +- 8 files changed, 71 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 970d244071e0d..01cd181010f97 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -205,7 +205,7 @@ statement | (DESC | DESCRIBE) QUERY? query #describeQuery | REFRESH TABLE multipartIdentifier #refreshTable | REFRESH (STRING | .*?) #refreshResource - | CACHE LAZY? TABLE tableIdentifier + | CACHE LAZY? TABLE multipartIdentifier (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable | UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable | CLEAR CACHE #clearCache diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 940dfd0fc333d..99e5c9feb8fa3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2769,6 +2769,30 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging RepairTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) } + /** + * Create a [[CacheTableStatement]]. + * + * For example: + * {{{ + * CACHE [LAZY] TABLE multi_part_name + * [OPTIONS tablePropertyList] [[AS] query] + * }}} + */ + override def visitCacheTable(ctx: CacheTableContext): LogicalPlan = withOrigin(ctx) { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val query = Option(ctx.query).map(plan) + val tableName = visitMultipartIdentifier(ctx.multipartIdentifier) + if (query.isDefined && tableName.length > 1) { + val catalogAndNamespace = tableName.init + throw new ParseException("It is not allowed to add catalog/namespace " + + s"prefix ${catalogAndNamespace.quoted} to " + + "the table name in CACHE TABLE AS SELECT", ctx) + } + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + CacheTableStatement(tableName, query, ctx.LAZY != null, options) + } + /** * Create a [[TruncateTableStatement]] command. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 127d9026f802e..4a91ee6d52d9e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -330,6 +330,15 @@ case class AnalyzeColumnStatement( */ case class RepairTableStatement(tableName: Seq[String]) extends ParsedStatement +/** + * A CACHE TABLE statement, as parsed from SQL + */ +case class CacheTableStatement( + tableName: Seq[String], + plan: Option[LogicalPlan], + isLazy: Boolean, + options: Map[String, String]) extends ParsedStatement + /** * A TRUNCATE TABLE statement, as parsed from SQL */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 8e605bd15f696..37349f7a33420 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1045,6 +1045,23 @@ class DDLParserSuite extends AnalysisTest { RepairTableStatement(Seq("a", "b", "c"))) } + test("CACHE table") { + comparePlans( + parsePlan("CACHE TABLE a.b.c"), + CacheTableStatement(Seq("a", "b", "c"), None, false, Map.empty)) + + comparePlans( + parsePlan("CACHE LAZY TABLE a.b.c"), + CacheTableStatement(Seq("a", "b", "c"), None, true, Map.empty)) + + comparePlans( + parsePlan("CACHE LAZY TABLE a.b.c OPTIONS('storageLevel' 'DISK_ONLY')"), + CacheTableStatement(Seq("a", "b", "c"), None, true, Map("storageLevel" -> "DISK_ONLY"))) + + intercept("CACHE TABLE a.b.c AS SELECT * FROM testData", + "It is not allowed to add catalog/namespace prefix a.b") + } + test("TRUNCATE table") { comparePlans( parsePlan("TRUNCATE TABLE a.b.c"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 230b8f3906bd2..65d95b600eaa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -299,6 +299,10 @@ class ResolveSessionCatalog( v1TableName.asTableIdentifier, "MSCK REPAIR TABLE") + case CacheTableStatement(tableName, plan, isLazy, options) => + val v1TableName = parseV1Table(tableName, "CACHE TABLE") + CacheTableCommand(v1TableName.asTableIdentifier, plan, isLazy, options) + case TruncateTableStatement(tableName, partitionSpec) => val v1TableName = parseV1Table(tableName, "TRUNCATE TABLE") TruncateTableCommand( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 2439621f7725a..fb13d01bd91dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -165,21 +165,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { unquotedPath } - /** - * Create a [[CacheTableCommand]] logical plan. - */ - override def visitCacheTable(ctx: CacheTableContext): LogicalPlan = withOrigin(ctx) { - val query = Option(ctx.query).map(plan) - val tableIdent = visitTableIdentifier(ctx.tableIdentifier) - if (query.isDefined && tableIdent.database.isDefined) { - val database = tableIdent.database.get - throw new ParseException(s"It is not allowed to add database prefix `$database` to " + - s"the table name in CACHE TABLE AS SELECT", ctx) - } - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - CacheTableCommand(tableIdent, query, ctx.LAZY != null, options) - } - /** * Create an [[UncacheTableCommand]] logical plan. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 463147903c923..4d1e70f68ba06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1268,6 +1268,20 @@ class DataSourceV2SQLSuite } } + test("CACHE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + testV1Command("CACHE TABLE", t) + + val e = intercept[AnalysisException] { + sql(s"CACHE LAZY TABLE $t") + } + assert(e.message.contains("CACHE TABLE is only supported with v1 tables")) + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 4cbc03d05c9e3..7b3fb68174234 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -300,7 +300,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val e = intercept[ParseException] { sql(s"CACHE TABLE $db.cachedTable AS SELECT 1") }.getMessage - assert(e.contains("It is not allowed to add database prefix ") && + assert(e.contains("It is not allowed to add catalog/namespace prefix ") && e.contains("to the table name in CACHE TABLE AS SELECT")) } } From 9e77d483158a6e9edc0b5b4c642e1231773be9ee Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 24 Oct 2019 15:43:13 +0800 Subject: [PATCH 0413/1387] [SPARK-21492][SQL][FOLLOW UP] Reimplement UnsafeExternalRowSorter in database style iterator ### What changes were proposed in this pull request? Reimplement the iterator in UnsafeExternalRowSorter in database style. This can be done by reusing the `RowIterator` in our code base. ### Why are the changes needed? During the job in #26164, after involving a var `isReleased` in `hasNext`, there's possible that `isReleased` is false when calling `hasNext`, but it becomes true before calling `next`. A safer way is using database-style iterator: `advanceNext` and `getRow`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing UT. Closes #26229 from xuanyuanking/SPARK-21492-follow-up. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../spark/sql/execution/RowIterator.scala | 0 .../execution/UnsafeExternalRowSorter.java | 48 ++++++++++--------- 2 files changed, 26 insertions(+), 22 deletions(-) rename sql/{core => catalyst}/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala (100%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/execution/RowIterator.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 3123f2187da83..90b55a8586de7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.function.Supplier; -import scala.collection.AbstractIterator; import scala.collection.Iterator; import scala.math.Ordering; @@ -168,7 +167,7 @@ public void cleanupResources() { sorter.cleanupResources(); } - public Iterator sort() throws IOException { + public Iterator sort() throws IOException { try { final UnsafeSorterIterator sortedIterator = sorter.getSortedIterator(); if (!sortedIterator.hasNext()) { @@ -176,31 +175,32 @@ public Iterator sort() throws IOException { // here in order to prevent memory leaks. cleanupResources(); } - return new AbstractIterator() { + return new RowIterator() { private final int numFields = schema.length(); private UnsafeRow row = new UnsafeRow(numFields); @Override - public boolean hasNext() { - return !isReleased && sortedIterator.hasNext(); - } - - @Override - public UnsafeRow next() { + public boolean advanceNext() { try { - sortedIterator.loadNext(); - row.pointTo( - sortedIterator.getBaseObject(), - sortedIterator.getBaseOffset(), - sortedIterator.getRecordLength()); - if (!hasNext()) { - UnsafeRow copy = row.copy(); // so that we don't have dangling pointers to freed page - row = null; // so that we don't keep references to the base object - cleanupResources(); - return copy; + if (!isReleased && sortedIterator.hasNext()) { + sortedIterator.loadNext(); + row.pointTo( + sortedIterator.getBaseObject(), + sortedIterator.getBaseOffset(), + sortedIterator.getRecordLength()); + // Here is the initial bug fix in SPARK-9364: the bug fix of use-after-free bug + // when returning the last row from an iterator. For example, in + // [[GroupedIterator]], we still use the last row after traversing the iterator + // in `fetchNextGroupIterator` + if (!sortedIterator.hasNext()) { + row = row.copy(); // so that we don't have dangling pointers to freed page + cleanupResources(); + } + return true; } else { - return row; + row = null; // so that we don't keep references to the base object + return false; } } catch (IOException e) { cleanupResources(); @@ -210,14 +210,18 @@ public UnsafeRow next() { } throw new RuntimeException("Exception should have been re-thrown in next()"); } - }; + + @Override + public UnsafeRow getRow() { return row; } + + }.toScala(); } catch (IOException e) { cleanupResources(); throw e; } } - public Iterator sort(Iterator inputIterator) throws IOException { + public Iterator sort(Iterator inputIterator) throws IOException { while (inputIterator.hasNext()) { insertRow(inputIterator.next()); } From 1296bbb8ac7f582f8689f3e2f36614cf541b80d4 Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Thu, 24 Oct 2019 11:14:31 +0200 Subject: [PATCH 0414/1387] [SPARK-29504][WEBUI] Toggle full job description on click ### What changes were proposed in this pull request? On clicking job description in jobs page, the description was not shown fully. Add the function for the click event on description. ### Why are the changes needed? when there is a long description of a job, it cannot be seen fully in the UI. The feature was added in https://github.com/apache/spark/pull/24145 But it is missed after https://github.com/apache/spark/pull/25374 Before change: ![Screenshot from 2019-10-23 11-23-00](https://user-images.githubusercontent.com/51401130/67361914-827b0080-f587-11e9-9181-d49a6a836046.png) After change: on Double click over decription ![Screenshot from 2019-10-23 11-20-02](https://user-images.githubusercontent.com/51401130/67361936-932b7680-f587-11e9-9e59-d290abed4b70.png) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually test Closes #26222 from PavithraRamachandran/jobs_description_tooltip. Authored-by: Pavithra Ramachandran Signed-off-by: Gengliang Wang --- .../main/resources/org/apache/spark/ui/static/webui.js | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.js b/core/src/main/resources/org/apache/spark/ui/static/webui.js index 89622106ff1f0..cf04db28804c1 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.js +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.js @@ -87,4 +87,11 @@ $(function() { collapseTablePageLoad('collapse-aggregated-runningExecutions','aggregated-runningExecutions'); collapseTablePageLoad('collapse-aggregated-completedExecutions','aggregated-completedExecutions'); collapseTablePageLoad('collapse-aggregated-failedExecutions','aggregated-failedExecutions'); -}); \ No newline at end of file +}); + +$(function() { + // Show/hide full job description on click event. + $(".description-input").click(function() { + $(this).toggleClass("description-input-full"); + }); +}); From 67cf0433ee4e4a7c33e0092e887bba53ad35627e Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 24 Oct 2019 21:55:03 +0900 Subject: [PATCH 0415/1387] [SPARK-29145][SQL] Support sub-queries in join conditions ### What changes were proposed in this pull request? Support SparkSQL use iN/EXISTS with subquery in JOIN condition. ### Why are the changes needed? Support SQL use iN/EXISTS with subquery in JOIN condition. ### Does this PR introduce any user-facing change? This PR is for enable user use subquery in `JOIN`'s ON condition. such as we have create three table ``` CREATE TABLE A(id String); CREATE TABLE B(id String); CREATE TABLE C(id String); ``` we can do query like : ``` SELECT A.id from A JOIN B ON A.id = B.id and A.id IN (select C.id from C) ``` ### How was this patch tested? ADDED UT Closes #25854 from AngersZhuuuu/SPARK-29145. Lead-authored-by: angerszhu Co-authored-by: AngersZhuuuu Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/analysis/Analyzer.scala | 2 + .../sql/catalyst/analysis/CheckAnalysis.scala | 4 +- .../analysis/ResolveSubquerySuite.scala | 14 +- .../org/apache/spark/sql/SubquerySuite.scala | 148 ++++++++++++++++++ 4 files changed, 165 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b913a9618d6eb..21bf926af50d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1697,6 +1697,8 @@ class Analyzer( // Only a few unary nodes (Project/Filter/Aggregate) can contain subqueries. case q: UnaryNode if q.childrenResolved => resolveSubQueries(q, q.children) + case j: Join if j.childrenResolved => + resolveSubQueries(j, Seq(j, j.left, j.right)) case s: SupportsSubquery if s.childrenResolved => resolveSubQueries(s, s.children) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 6a5d938f0fdc6..d9dc9ebbcaf3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -601,10 +601,10 @@ trait CheckAnalysis extends PredicateHelper { case inSubqueryOrExistsSubquery => plan match { - case _: Filter | _: SupportsSubquery => // Ok + case _: Filter | _: SupportsSubquery | _: Join => // Ok case _ => failAnalysis(s"IN/EXISTS predicate sub-queries can only be used in" + - s" Filter and a few commands: $plan") + s" Filter/Join and a few commands: $plan") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala index 74a8590b5eefe..5aa80e1a9bd7f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{InSubquery, ListQuery} -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, Project} +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical._ /** * Unit tests for [[ResolveSubquery]]. @@ -29,8 +30,10 @@ class ResolveSubquerySuite extends AnalysisTest { val a = 'a.int val b = 'b.int + val c = 'c.int val t1 = LocalRelation(a) val t2 = LocalRelation(b) + val t3 = LocalRelation(c) test("SPARK-17251 Improve `OuterReference` to be `NamedExpression`") { val expr = Filter( @@ -41,4 +44,13 @@ class ResolveSubquerySuite extends AnalysisTest { assert(m.contains( "Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses")) } + + test("SPARK-29145 Support subquery in join condition") { + val expr = Join(t1, + t2, + Inner, + Some(InSubquery(Seq(a), ListQuery(Project(Seq(UnresolvedAttribute("c")), t3)))), + JoinHint.NONE) + assertAnalysisSuccess(expr) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index a1d7792941ed9..266f8e23712db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -204,6 +204,154 @@ class SubquerySuite extends QueryTest with SharedSparkSession { } } + test("SPARK-29145: JOIN Condition use QueryList") { + withTempView("s1", "s2", "s3") { + Seq(1, 3, 5, 7, 9).toDF("id").createOrReplaceTempView("s1") + Seq(1, 3, 4, 6, 9).toDF("id").createOrReplaceTempView("s2") + Seq(3, 4, 6, 9).toDF("id").createOrReplaceTempView("s3") + + checkAnswer( + sql( + """ + | SELECT s1.id FROM s1 + | JOIN s2 ON s1.id = s2.id + | AND s1.id IN (SELECT 9) + """.stripMargin), + Row(9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id FROM s1 + | JOIN s2 ON s1.id = s2.id + | AND s1.id NOT IN (SELECT 9) + """.stripMargin), + Row(1) :: Row(3) :: Nil) + + // case `IN` + checkAnswer( + sql( + """ + | SELECT s1.id FROM s1 + | JOIN s2 ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(3) :: Row(9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id AS id2 FROM s1 + | LEFT SEMI JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(3) :: Row(9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id as id2 FROM s1 + | LEFT ANTI JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(1) :: Row(5) :: Row(7) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id as id2 FROM s1 + | LEFT OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(1, null) :: Row(3, 3) :: Row(5, null) :: Row(7, null) :: Row(9, 9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id as id2 FROM s1 + | RIGHT OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(null, 1) :: Row(3, 3) :: Row(null, 4) :: Row(null, 6) :: Row(9, 9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id AS id2 FROM s1 + | FULL OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id IN (SELECT id FROM s3) + """.stripMargin), + Row(1, null) :: Row(3, 3) :: Row(5, null) :: Row(7, null) :: Row(9, 9) :: + Row(null, 1) :: Row(null, 4) :: Row(null, 6) :: Nil) + + // case `NOT IN` + checkAnswer( + sql( + """ + | SELECT s1.id FROM s1 + | JOIN s2 ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id AS id2 FROM s1 + | LEFT SEMI JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id AS id2 FROM s1 + | LEFT ANTI JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(3) :: Row(5) :: Row(7) :: Row(9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id AS id2 FROM s1 + | LEFT OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1, 1) :: Row(3, null) :: Row(5, null) :: Row(7, null) :: Row(9, null) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id AS id2 FROM s1 + | RIGHT OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1, 1) :: Row(null, 3) :: Row(null, 4) :: Row(null, 6) :: Row(null, 9) :: Nil) + + checkAnswer( + sql( + """ + | SELECT s1.id, s2.id AS id2 FROM s1 + | FULL OUTER JOIN s2 + | ON s1.id = s2.id + | AND s1.id NOT IN (SELECT id FROM s3) + """.stripMargin), + Row(1, 1) :: Row(3, null) :: Row(5, null) :: Row(7, null) :: Row(9, null) :: + Row(null, 3) :: Row(null, 4) :: Row(null, 6) :: Row(null, 9) :: Nil) + } + } + test("SPARK-14791: scalar subquery inside broadcast join") { val df = sql("select a, sum(b) as s from l group by a having a > (select avg(a) from l)") val expected = Row(3, 2.0, 3, 3.0) :: Row(6, null, 6, null) :: Nil From 1ec1b2bd17ac5f6040336680c79e2aa9765b353d Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Thu, 24 Oct 2019 08:19:03 -0500 Subject: [PATCH 0416/1387] [SPARK-28791][DOC] Documentation for Alter table Command What changes were proposed in this pull request? Document ALTER TABLE statement in SQL Reference Guide. Why are the changes needed? Adding documentation for SQL reference. Does this PR introduce any user-facing change? yes Before: There was no documentation for this. After. ![1](https://user-images.githubusercontent.com/51401130/65674372-1087c800-e06a-11e9-9155-ac70b419b069.png) ![2](https://user-images.githubusercontent.com/51401130/65674384-14b3e580-e06a-11e9-9c57-bca566dfdbc2.png) ![3](https://user-images.githubusercontent.com/51401130/65674391-18e00300-e06a-11e9-950a-6cc948dedd7d.png) ![4](https://user-images.githubusercontent.com/51401130/65674397-1bdaf380-e06a-11e9-87b0-b1523a745f83.png) ![5](https://user-images.githubusercontent.com/51401130/65674406-209fa780-e06a-11e9-8440-7e8105a77117.png) ![6](https://user-images.githubusercontent.com/51401130/65674417-23020180-e06a-11e9-8fff-30511836bb08.png) How was this patch tested? Used jekyll build and serve to verify. Closes #25590 from PavithraRamachandran/alter_doc. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- docs/sql-ref-syntax-ddl-alter-table.md | 238 ++++++++++++++++++++++++- 1 file changed, 237 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index 7fcd397915825..e311691c6b801 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -19,4 +19,240 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`ALTER TABLE` statement changes the schema or properties of a table. + +### RENAME +`ALTER TABLE RENAME` statement changes the table name of an existing table in the database. + +#### Syntax +{% highlight sql %} +ALTER TABLE [db_name.]old_table_name RENAME TO [db_name.]new_table_name + +ALTER TABLE table_name PARTITION partition_spec RENAME TO PARTITION partition_spec; + +{% endhighlight %} + +#### Parameters +
    +
    old_table_name
    +
    Name of an existing table.
    +
    +
    +
    db_name
    +
    Name of the existing database.
    +
    + +
    +
    new_table_name
    +
    New name using which the table has to be renamed.
    +
    + +
    +
    partition_spec
    +
    Partition to be renamed.
    +
    + + +### ADD COLUMNS +`ALTER TABLE ADD COLUMNS` statement adds mentioned columns to an existing table. + +#### Syntax +{% highlight sql %} +ALTER TABLE table_name ADD COLUMNS (col_spec[, col_spec ...]) +{% endhighlight %} + +#### Parameters +
    +
    table_name
    +
    The name of an existing table.
    +
    + + +
    +
    COLUMNS (col_spec)
    +
    Specifies the columns to be added to be renamed.
    +
    + + +### SET AND UNSET + +#### SET TABLE PROPERTIES +`ALTER TABLE SET` command is used for setting the table properties. If a particular property was already set, +this overrides the old value with the new one. + +`ALTER TABLE UNSET` is used to drop the table property. + +##### Syntax +{% highlight sql %} + +--Set Table Properties +ALTER TABLE table_name SET TBLPROPERTIES (key1=val1, key2=val2, ...) + +--Unset Table Properties +ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] (key1, key2, ...) + +{% endhighlight %} + +#### SET SERDE +`ALTER TABLE SET` command is used for setting the SERDE or SERDE properties in Hive tables. If a particular property was already set, +this overrides the old value with the new one. + +##### Syntax +{% highlight sql %} + +--Set SERDE Propeties +ALTER TABLE table_name [PARTITION part_spec] + SET SERDEPROPERTIES (key1=val1, key2=val2, ...) + +ALTER TABLE table_name [PARTITION part_spec] SET SERDE serde_class_name + [WITH SERDEPROPERTIES (key1=val1, key2=val2, ...)] + +{% endhighlight %} + +#### SET LOCATION And SET FILE FORMAT +`ALTER TABLE SET` command can also be used for changing the file location and file format for +exsisting tables. + +##### Syntax +{% highlight sql %} + +--Changing File Format +ALTER TABLE table_name [PARTITION partition_spec] SET FILEFORMAT file_format; + +--Changing File Location +ALTER TABLE table_name [PARTITION partition_spec] SET LOCATION 'new_location'; + +{% endhighlight %} + +#### Parameters +
    +
    table_name
    +
    The name of an existing table.
    +
    + +
    +
    PARTITION (part_spec)
    +
    Specifies the partition on which the property has to be set.
    +
    + +
    +
    SERDEPROPERTIES (key1=val1, key2=val2, ...)
    +
    Specifies the SERDE properties to be set.
    +
    + + +### Examples +{% highlight sql %} + +--RENAME table +DESC student; ++--------------------------+------------+----------+--+ +| col_name | data_type | comment | ++--------------------------+------------+----------+--+ +| name | string | NULL | +| rollno | int | NULL | +| age | int | NULL | +| # Partition Information | | | +| # col_name | data_type | comment | +| age | int | NULL | ++--------------------------+------------+----------+--+ + +ALTER TABLE Student RENAME TO StudentInfo; + +--After Renaming the table + +DESC StudentInfo; ++--------------------------+------------+----------+--+ +| col_name | data_type | comment | ++--------------------------+------------+----------+--+ +| name | string | NULL | +| rollno | int | NULL | +| age | int | NULL | +| # Partition Information | | | +| # col_name | data_type | comment | +| age | int | NULL | ++--------------------------+------------+----------+--+ + +--RENAME partition + +SHOW PARTITIONS StudentInfo; ++------------+--+ +| partition | ++------------+--+ +| age=10 | +| age=11 | +| age=12 | ++------------+--+ + +ALTER TABLE default.StudentInfo PARTITION (age='10') RENAME TO PARTITION (age='15'); + +--After renaming Partition +SHOW PARTITIONS StudentInfo; ++------------+--+ +| partition | ++------------+--+ +| age=11 | +| age=12 | +| age=15 | ++------------+--+ + +-- Add new column to a table + +DESC StudentInfo; ++--------------------------+------------+----------+--+ +| col_name | data_type | comment | ++--------------------------+------------+----------+--+ +| name | string | NULL | +| rollno | int | NULL | +| age | int | NULL | +| # Partition Information | | | +| # col_name | data_type | comment | +| age | int | NULL | ++--------------------------+------------+----------+ + +ALTER TABLE StudentInfo ADD columns (LastName string, DOB timestamp); + +--After Adding New columns to the table +DESC StudentInfo; ++--------------------------+------------+----------+--+ +| col_name | data_type | comment | ++--------------------------+------------+----------+--+ +| name | string | NULL | +| rollno | int | NULL | +| LastName | string | NULL | +| DOB | timestamp | NULL | +| age | int | NULL | +| # Partition Information | | | +| # col_name | data_type | comment | +| age | int | NULL | ++--------------------------+------------+----------+--+ + + +--Change the fileformat +ALTER TABLE loc_orc SET fileformat orc; + +ALTER TABLE p1 partition (month=2, day=2) SET fileformat parquet; + +--Change the file Location +ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways' + +-- SET SERDE/ SERDE Properties +ALTER TABLE test_tab SET SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; + +ALTER TABLE dbx.tab1 SET SERDE 'org.apache.madoop' WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee') + +--SET TABLE PROPERTIES +ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('winner' = 'loser') + +--DROP TABLE PROPERTIES +ALTER TABLE dbx.tab1 UNSET TBLPROPERTIES ('winner') + +{% endhighlight %} + + +### Related Statements +- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) +- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) + + From 76d4bebb54d7b3960d48fcf473d2d3db0f5fbcde Mon Sep 17 00:00:00 2001 From: shahid Date: Thu, 24 Oct 2019 08:29:05 -0500 Subject: [PATCH 0417/1387] [SPARK-29559][WEBUI] Support pagination for JDBC/ODBC Server page ### What changes were proposed in this pull request? Supports pagination for SQL Statisitcs table in the JDBC/ODBC tab using existing Spark pagination framework. ### Why are the changes needed? It will easier for user to analyse the table and it may fix the potential issues like oom while loading the page, that may occur similar to the SQL page (refer https://github.com/apache/spark/pull/22645) ### Does this PR introduce any user-facing change? There will be no change in the `SQLStatistics` table in JDBC/ODBC server page execpt pagination support. ### How was this patch tested? Manually verified. Before PR: ![Screenshot 2019-10-22 at 11 37 29 PM](https://user-images.githubusercontent.com/23054875/67316080-73636680-f525-11e9-91bc-ff7e06e3736d.png) After PR: ![Screenshot 2019-10-22 at 10 33 00 PM](https://user-images.githubusercontent.com/23054875/67316092-778f8400-f525-11e9-93f8-1e2815abd66f.png) Closes #26215 from shahidki31/jdbcPagination. Authored-by: shahid Signed-off-by: Sean Owen --- .../thriftserver/ui/ThriftServerPage.scala | 364 +++++++++++++++--- 1 file changed, 302 insertions(+), 62 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 4056be4769d21..e472aaad5bdc6 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.hive.thriftserver.ui +import java.net.URLEncoder +import java.nio.charset.StandardCharsets.UTF_8 import java.util.Calendar import javax.servlet.http.HttpServletRequest -import scala.xml.Node +import scala.collection.JavaConverters._ +import scala.xml.{Node, Unparsed} import org.apache.commons.text.StringEscapeUtils @@ -29,7 +32,7 @@ import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, import org.apache.spark.sql.hive.thriftserver.ui.ToolTips._ import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ - +import org.apache.spark.util.Utils /** Page for Spark Web UI that shows statistics of the thrift server */ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("") with Logging { @@ -69,45 +72,56 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" /** Generate stats of batch statements of the thrift server program */ private def generateSQLStatsTable(request: HttpServletRequest): Seq[Node] = { + val numStatement = listener.getExecutionList.size + val table = if (numStatement > 0) { - val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", - "Execution Time", "Duration", "Statement", "State", "Detail") - val tooltips = Seq(None, None, None, None, Some(THRIFT_SERVER_FINISH_TIME), - Some(THRIFT_SERVER_CLOSE_TIME), Some(THRIFT_SERVER_EXECUTION), - Some(THRIFT_SERVER_DURATION), None, None, None) - assert(headerRow.length == tooltips.length) - val dataRows = listener.getExecutionList.sortBy(_.startTimestamp).reverse - - def generateDataRow(info: ExecutionInfo): Seq[Node] = { - val jobLink = info.jobId.map { id: String => - - [{id}] - + + val sqlTableTag = "sqlstat" + + val parameterOtherTable = request.getParameterMap().asScala + .filterNot(_._1.startsWith(sqlTableTag)) + .map { case (name, vals) => + name + "=" + vals(0) } - val detail = Option(info.detail).filter(!_.isEmpty).getOrElse(info.executePlan) - - {info.userName} - - {jobLink} - - {info.groupId} - {formatDate(info.startTimestamp)} - {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} - {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} - - {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} - - {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} - {info.statement} - {info.state} - {errorMessageCell(detail)} - - } - Some(UIUtils.listingTable(headerRow, generateDataRow, - dataRows, false, None, Seq(null), false, tooltipHeaders = tooltips)) + val parameterSqlTablePage = request.getParameter(s"$sqlTableTag.page") + val parameterSqlTableSortColumn = request.getParameter(s"$sqlTableTag.sort") + val parameterSqlTableSortDesc = request.getParameter(s"$sqlTableTag.desc") + val parameterSqlPageSize = request.getParameter(s"$sqlTableTag.pageSize") + + val sqlTablePage = Option(parameterSqlTablePage).map(_.toInt).getOrElse(1) + val sqlTableSortColumn = Option(parameterSqlTableSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("Start Time") + val sqlTableSortDesc = Option(parameterSqlTableSortDesc).map(_.toBoolean).getOrElse( + // New executions should be shown above old executions by default. + sqlTableSortColumn == "Start Time" + ) + val sqlTablePageSize = Option(parameterSqlPageSize).map(_.toInt).getOrElse(100) + + try { + Some(new SqlStatsPagedTable( + request, + parent, + listener.getExecutionList, + "sqlserver", + UIUtils.prependBaseUri(request, parent.basePath), + parameterOtherTable, + sqlTableTag, + pageSize = sqlTablePageSize, + sortColumn = sqlTableSortColumn, + desc = sqlTableSortDesc + ).table(sqlTablePage)) + } catch { + case e@(_: IllegalArgumentException | _: IndexOutOfBoundsException) => + Some(
    +

    Error while rendering job table:

    +
    +              {Utils.exceptionString(e)}
    +            
    +
    ) + } } else { None } @@ -123,30 +137,6 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" content } - private def errorMessageCell(errorMessage: String): Seq[Node] = { - val isMultiline = errorMessage.indexOf('\n') >= 0 - val errorSummary = StringEscapeUtils.escapeHtml4( - if (isMultiline) { - errorMessage.substring(0, errorMessage.indexOf('\n')) - } else { - errorMessage - }) - val details = if (isMultiline) { - // scalastyle:off - - + details - ++ - - // scalastyle:on - } else { - "" - } - {errorSummary}{details} - } - /** Generate stats of batch sessions of the thrift server program */ private def generateSessionStatsTable(request: HttpServletRequest): Seq[Node] = { val sessionList = listener.getSessionList @@ -185,7 +175,6 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" content } - /** * Returns a human-readable string representing a duration such as "5 second 35 ms" */ @@ -202,3 +191,254 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" } } +private[ui] class SqlStatsPagedTable( + request: HttpServletRequest, + parent: ThriftServerTab, + data: Seq[ExecutionInfo], + subPath: String, + basePath: String, + parameterOtherTable: Iterable[String], + sqlStatsTableTag: String, + pageSize: Int, + sortColumn: String, + desc: Boolean) extends PagedTable[SqlStatsTableRow] { + + override val dataSource = new SqlStatsTableDataSource(data, pageSize, sortColumn, desc) + + private val parameterPath = s"$basePath/$subPath/?${parameterOtherTable.mkString("&")}" + + override def tableId: String = sqlStatsTableTag + + override def tableCssClass: String = + "table table-bordered table-condensed table-striped " + + "table-head-clickable table-cell-width-limited" + + override def pageLink(page: Int): String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + parameterPath + + s"&$pageNumberFormField=$page" + + s"&$sqlStatsTableTag.sort=$encodedSortColumn" + + s"&$sqlStatsTableTag.desc=$desc" + + s"&$pageSizeFormField=$pageSize" + } + + override def pageSizeFormField: String = s"$sqlStatsTableTag.pageSize" + + override def pageNumberFormField: String = s"$sqlStatsTableTag.page" + + override def goButtonFormPath: String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + s"$parameterPath&$sqlStatsTableTag.sort=$encodedSortColumn&$sqlStatsTableTag.desc=$desc" + } + + override def headers: Seq[Node] = { + val sqlTableHeaders = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", + "Close Time", "Execution Time", "Duration", "Statement", "State", "Detail") + + val tooltips = Seq(None, None, None, None, Some(THRIFT_SERVER_FINISH_TIME), + Some(THRIFT_SERVER_CLOSE_TIME), Some(THRIFT_SERVER_EXECUTION), + Some(THRIFT_SERVER_DURATION), None, None, None) + + assert(sqlTableHeaders.length == tooltips.length) + + val headerRow: Seq[Node] = { + sqlTableHeaders.zip(tooltips).map { case (header, tooltip) => + if (header == sortColumn) { + val headerLink = Unparsed( + parameterPath + + s"&$sqlStatsTableTag.sort=${URLEncoder.encode(header, UTF_8.name())}" + + s"&$sqlStatsTableTag.desc=${!desc}" + + s"&$sqlStatsTableTag.pageSize=$pageSize" + + s"#$sqlStatsTableTag") + val arrow = if (desc) "▾" else "▴" // UP or DOWN + + if (tooltip.nonEmpty) { + + + + {header} {Unparsed(arrow)} + + + + } else { + + + {header} {Unparsed(arrow)} + + + } + } else { + val headerLink = Unparsed( + parameterPath + + s"&$sqlStatsTableTag.sort=${URLEncoder.encode(header, UTF_8.name())}" + + s"&$sqlStatsTableTag.pageSize=$pageSize" + + s"#$sqlStatsTableTag") + + if(tooltip.nonEmpty) { + + + + {header} + + + + } else { + + + {header} + + + } + } + } + } + + {headerRow} + + } + + override def row(sqlStatsTableRow: SqlStatsTableRow): Seq[Node] = { + val info = sqlStatsTableRow.executionInfo + val startTime = info.startTimestamp + val executionTime = sqlStatsTableRow.executionTime + val duration = sqlStatsTableRow.duration + + def jobLinks(jobData: Seq[String]): Seq[Node] = { + jobData.map { jobId => + [{jobId.toString}] + } + } + + + + {info.userName} + + + {jobLinks(sqlStatsTableRow.jobId)} + + + {info.groupId} + + + {UIUtils.formatDate(startTime)} + + + {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} + + + {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} + + + {UIUtils.formatDuration(executionTime)} + + + {UIUtils.formatDuration(duration)} + + + {info.statement} + + + {info.state} + + {errorMessageCell(sqlStatsTableRow.detail)} + + } + + + private def errorMessageCell(errorMessage: String): Seq[Node] = { + val isMultiline = errorMessage.indexOf('\n') >= 0 + val errorSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + errorMessage.substring(0, errorMessage.indexOf('\n')) + } else { + errorMessage + }) + val details = if (isMultiline) { + // scalastyle:off + + + details + ++ + + // scalastyle:on + } else { + "" + } + + {errorSummary}{details} + + } + + private def jobURL(request: HttpServletRequest, jobId: String): String = + "%s/jobs/job/?id=%s".format(UIUtils.prependBaseUri(request, parent.basePath), jobId) +} + + private[ui] class SqlStatsTableRow( + val jobId: Seq[String], + val duration: Long, + val executionTime: Long, + val executionInfo: ExecutionInfo, + val detail: String) + + private[ui] class SqlStatsTableDataSource( + info: Seq[ExecutionInfo], + pageSize: Int, + sortColumn: String, + desc: Boolean) extends PagedDataSource[SqlStatsTableRow](pageSize) { + + // Convert ExecutionInfo to SqlStatsTableRow which contains the final contents to show in + // the table so that we can avoid creating duplicate contents during sorting the data + private val data = info.map(sqlStatsTableRow).sorted(ordering(sortColumn, desc)) + + private var _slicedStartTime: Set[Long] = null + + override def dataSize: Int = data.size + + override def sliceData(from: Int, to: Int): Seq[SqlStatsTableRow] = { + val r = data.slice(from, to) + r.map(x => x) + _slicedStartTime = r.map(_.executionInfo.startTimestamp).toSet + r + } + + private def sqlStatsTableRow(executionInfo: ExecutionInfo): SqlStatsTableRow = { + val duration = executionInfo.totalTime(executionInfo.closeTimestamp) + val executionTime = executionInfo.totalTime(executionInfo.finishTimestamp) + val detail = Option(executionInfo.detail).filter(!_.isEmpty) + .getOrElse(executionInfo.executePlan) + val jobId = executionInfo.jobId.toSeq.sorted + + new SqlStatsTableRow(jobId, duration, executionTime, executionInfo, detail) + + } + + /** + * Return Ordering according to sortColumn and desc. + */ + private def ordering(sortColumn: String, desc: Boolean): Ordering[SqlStatsTableRow] = { + val ordering: Ordering[SqlStatsTableRow] = sortColumn match { + case "User" => Ordering.by(_.executionInfo.userName) + case "JobID" => Ordering by (_.jobId.headOption) + case "GroupID" => Ordering.by(_.executionInfo.groupId) + case "Start Time" => Ordering.by(_.executionInfo.startTimestamp) + case "Finish Time" => Ordering.by(_.executionInfo.finishTimestamp) + case "Close Time" => Ordering.by(_.executionInfo.closeTimestamp) + case "Execution Time" => Ordering.by(_.executionTime) + case "Duration" => Ordering.by(_.duration) + case "Statement" => Ordering.by(_.executionInfo.statement) + case "State" => Ordering.by(_.executionInfo.state) + case "Detail" => Ordering.by(_.detail) + case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") + } + if (desc) { + ordering.reverse + } else { + ordering + } + } + + } From a35fb4fd504b9ac14384eb63fc4c993fd53cd667 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 24 Oct 2019 08:30:27 -0500 Subject: [PATCH 0418/1387] [SPARK-29578][TESTS] Add "8634" as another skipped day for Kwajalein timzeone due to more recent timezone updates in later JDK 8 ### What changes were proposed in this pull request? Recent timezone definition changes in very new JDK 8 (and beyond) releases cause test failures. The below was observed on JDK 1.8.0_232. As before, the easy fix is to allow for these inconsequential variations in test results due to differing definition of timezones. ### Why are the changes needed? Keeps test passing on the latest JDK releases. ### Does this PR introduce any user-facing change? None ### How was this patch tested? Existing tests Closes #26236 from srowen/SPARK-29578. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 1abbca6c8cd29..10642b3ca8a4f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -555,12 +555,12 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { // There are some days are skipped entirely in some timezone, skip them here. val skipped_days = Map[String, Set[Int]]( - "Kwajalein" -> Set(8632, 8633), + "Kwajalein" -> Set(8632, 8633, 8634), "Pacific/Apia" -> Set(15338), "Pacific/Enderbury" -> Set(9130, 9131), "Pacific/Fakaofo" -> Set(15338), "Pacific/Kiritimati" -> Set(9130, 9131), - "Pacific/Kwajalein" -> Set(8632, 8633), + "Pacific/Kwajalein" -> Set(8632, 8633, 8634), "MIT" -> Set(15338)) for (tz <- ALL_TIMEZONES) { val skipped = skipped_days.getOrElse(tz.getID, Set.empty) From cdea520ff8954cf415fd98d034d9b674d6ca4f67 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 24 Oct 2019 09:15:59 -0700 Subject: [PATCH 0419/1387] [SPARK-29532][SQL] Simplify interval string parsing ### What changes were proposed in this pull request? Only use antlr4 to parse the interval string, and remove the duplicated parsing logic from `CalendarInterval`. ### Why are the changes needed? Simplify the code and fix inconsistent behaviors. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass the Jenkins with the updated test cases. Closes #26190 from cloud-fan/parser. Lead-authored-by: Wenchen Fan Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../spark/unsafe/types/CalendarInterval.java | 156 +++------------- .../unsafe/types/CalendarIntervalSuite.java | 174 ++---------------- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 + .../spark/sql/catalyst/expressions/Cast.scala | 9 +- .../sql/catalyst/expressions/TimeWindow.scala | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 40 +++- .../sql/catalyst/parser/ParseDriver.scala | 9 + .../sql/catalyst/util/IntervalUtils.scala | 51 ++++- .../CollectionExpressionsSuite.scala | 36 ++-- .../expressions/DateExpressionsSuite.scala | 18 +- .../expressions/HashExpressionsSuite.scala | 6 +- .../IntervalExpressionsSuite.scala | 4 +- .../expressions/MutableProjectionSuite.scala | 4 +- .../expressions/ObjectExpressionsSuite.scala | 5 +- .../expressions/UnsafeRowConverterSuite.scala | 2 +- .../parser/ExpressionParserSuite.scala | 6 +- .../catalyst/util/IntervalUtilsSuite.scala | 90 +++++++++ .../IntervalBenchmark-jdk11-results.txt | 40 ++-- .../benchmarks/IntervalBenchmark-results.txt | 41 +++-- .../scala/org/apache/spark/sql/Dataset.scala | 3 +- .../execution/streaming/GroupStateImpl.scala | 3 +- .../sql/execution/streaming/Triggers.scala | 4 +- .../resources/sql-tests/inputs/literals.sql | 13 ++ .../sql-tests/results/literals.sql.out | 164 ++++++++++++++--- .../benchmark/IntervalBenchmark.scala | 5 +- .../sql/hive/execution/SQLQuerySuite.scala | 46 ----- 26 files changed, 464 insertions(+), 473 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 28fb64f7cd0e0..184ddac9a71a6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -32,94 +32,11 @@ public final class CalendarInterval implements Serializable { public static final long MICROS_PER_DAY = MICROS_PER_HOUR * 24; public static final long MICROS_PER_WEEK = MICROS_PER_DAY * 7; - /** - * A function to generate regex which matches interval string's unit part like "3 years". - * - * First, we can leave out some units in interval string, and we only care about the value of - * unit, so here we use non-capturing group to wrap the actual regex. - * At the beginning of the actual regex, we should match spaces before the unit part. - * Next is the number part, starts with an optional "-" to represent negative value. We use - * capturing group to wrap this part as we need the value later. - * Finally is the unit name, ends with an optional "s". - */ - private static String unitRegex(String unit) { - return "(?:\\s+(-?\\d+)\\s+" + unit + "s?)?"; - } - - private static Pattern p = Pattern.compile("interval" + unitRegex("year") + unitRegex("month") + - unitRegex("week") + unitRegex("day") + unitRegex("hour") + unitRegex("minute") + - unitRegex("second") + unitRegex("millisecond") + unitRegex("microsecond"), - Pattern.CASE_INSENSITIVE); - - private static Pattern yearMonthPattern = - Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); + private static Pattern yearMonthPattern = Pattern.compile( + "^([+|-])?(\\d+)-(\\d+)$"); private static Pattern dayTimePattern = Pattern.compile( - "^(?:['|\"])?([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); - - private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$"); - - private static long toLong(String s) { - if (s == null) { - return 0; - } else { - return Long.parseLong(s); - } - } - - /** - * Convert a string to CalendarInterval. Return null if the input string is not a valid interval. - * This method is case-insensitive. - */ - public static CalendarInterval fromString(String s) { - try { - return fromCaseInsensitiveString(s); - } catch (IllegalArgumentException e) { - return null; - } - } - - /** - * Convert a string to CalendarInterval. This method can handle - * strings without the `interval` prefix and throws IllegalArgumentException - * when the input string is not a valid interval. - * - * @throws IllegalArgumentException if the string is not a valid internal. - */ - public static CalendarInterval fromCaseInsensitiveString(String s) { - if (s == null) { - throw new IllegalArgumentException("Interval cannot be null"); - } - String trimmed = s.trim(); - if (trimmed.isEmpty()) { - throw new IllegalArgumentException("Interval cannot be blank"); - } - String prefix = "interval"; - String intervalStr = trimmed; - // Checks the given interval string does not start with the `interval` prefix - if (!intervalStr.regionMatches(true, 0, prefix, 0, prefix.length())) { - // Prepend `interval` if it does not present because - // the regular expression strictly require it. - intervalStr = prefix + " " + trimmed; - } else if (intervalStr.length() == prefix.length()) { - throw new IllegalArgumentException("Interval string must have time units"); - } - - Matcher m = p.matcher(intervalStr); - if (!m.matches()) { - throw new IllegalArgumentException("Invalid interval: " + s); - } - - long months = toLong(m.group(1)) * 12 + toLong(m.group(2)); - long microseconds = toLong(m.group(3)) * MICROS_PER_WEEK; - microseconds += toLong(m.group(4)) * MICROS_PER_DAY; - microseconds += toLong(m.group(5)) * MICROS_PER_HOUR; - microseconds += toLong(m.group(6)) * MICROS_PER_MINUTE; - microseconds += toLong(m.group(7)) * MICROS_PER_SECOND; - microseconds += toLong(m.group(8)) * MICROS_PER_MILLI; - microseconds += toLong(m.group(9)); - return new CalendarInterval((int) months, microseconds); - } + "^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$"); public static long toLongWithRange(String fieldName, String s, long minValue, long maxValue) throws IllegalArgumentException { @@ -242,72 +159,59 @@ public static CalendarInterval fromDayTimeString(String s, String from, String t return result; } - public static CalendarInterval fromSingleUnitString(String unit, String s) + public static CalendarInterval fromUnitStrings(String[] units, String[] values) throws IllegalArgumentException { + assert units.length == values.length; + int months = 0; + long microseconds = 0; - CalendarInterval result = null; - if (s == null) { - throw new IllegalArgumentException(String.format("Interval %s string was null", unit)); - } - s = s.trim(); - Matcher m = quoteTrimPattern.matcher(s); - if (!m.matches()) { - throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s); - } else { + for (int i = 0; i < units.length; i++) { try { - switch (unit) { + switch (units[i]) { case "year": - int year = (int) toLongWithRange("year", m.group(1), - Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12); - result = new CalendarInterval(year * 12, 0L); + months = Math.addExact(months, Math.multiplyExact(Integer.parseInt(values[i]), 12)); break; case "month": - int month = (int) toLongWithRange("month", m.group(1), - Integer.MIN_VALUE, Integer.MAX_VALUE); - result = new CalendarInterval(month, 0L); + months = Math.addExact(months, Integer.parseInt(values[i])); break; case "week": - long week = toLongWithRange("week", m.group(1), - Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK); - result = new CalendarInterval(0, week * MICROS_PER_WEEK); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_WEEK)); break; case "day": - long day = toLongWithRange("day", m.group(1), - Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY); - result = new CalendarInterval(0, day * MICROS_PER_DAY); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_DAY)); break; case "hour": - long hour = toLongWithRange("hour", m.group(1), - Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR); - result = new CalendarInterval(0, hour * MICROS_PER_HOUR); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_HOUR)); break; case "minute": - long minute = toLongWithRange("minute", m.group(1), - Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE); - result = new CalendarInterval(0, minute * MICROS_PER_MINUTE); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MINUTE)); break; case "second": { - long micros = parseSecondNano(m.group(1)); - result = new CalendarInterval(0, micros); + microseconds = Math.addExact(microseconds, parseSecondNano(values[i])); break; } case "millisecond": - long millisecond = toLongWithRange("millisecond", m.group(1), - Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI); - result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI); + microseconds = Math.addExact( + microseconds, + Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MILLI)); break; - case "microsecond": { - long micros = Long.parseLong(m.group(1)); - result = new CalendarInterval(0, micros); + case "microsecond": + microseconds = Math.addExact(microseconds, Long.parseLong(values[i])); break; - } } } catch (Exception e) { throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e); } } - return result; + return new CalendarInterval(months, microseconds); } /** diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 587071332ce47..9f3262bf2aaa4 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -19,8 +19,6 @@ import org.junit.Test; -import java.util.Arrays; - import static org.junit.Assert.*; import static org.apache.spark.unsafe.types.CalendarInterval.*; @@ -62,72 +60,6 @@ public void toStringTest() { assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); } - @Test - public void fromStringTest() { - testSingleUnit("year", 3, 36, 0); - testSingleUnit("month", 3, 3, 0); - testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("microsecond", 3, 0, 3); - - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - Arrays.asList( - "interval -5 years 23 month", - " -5 years 23 month", - "interval -5 years 23 month ", - " -5 years 23 month ", - " interval -5 years 23 month ").forEach(input -> - assertEquals(fromString(input), result) - ); - - // Error cases - Arrays.asList( - "interval 3month 1 hour", - "3month 1 hour", - "interval 3 moth 1 hour", - "3 moth 1 hour", - "interval", - "int", - "", - null).forEach(input -> assertNull(fromString(input))); - } - - @Test - public void fromCaseInsensitiveStringTest() { - for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { - assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); - } - - for (String input : new String[]{null, "", " "}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - String msg = e.getMessage(); - if (input == null) assertTrue(msg.contains("cannot be null")); - else assertTrue(msg.contains("cannot be blank")); - } - } - - for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { - try { - fromCaseInsensitiveString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - String msg = e.getMessage(); - if (input.trim().equalsIgnoreCase("interval")) { - assertTrue(msg.contains("Interval string must have time units")); - } else { - assertTrue(msg.contains("Invalid interval:")); - } - } - } - } - @Test public void fromYearMonthStringTest() { String input; @@ -194,107 +126,25 @@ public void fromDayTimeStringTest() { } } - @Test - public void fromSingleUnitStringTest() { - String input; - CalendarInterval i; - - input = "12"; - i = new CalendarInterval(12 * 12, 0L); - assertEquals(fromSingleUnitString("year", input), i); - - input = "100"; - i = new CalendarInterval(0, 100 * MICROS_PER_DAY); - assertEquals(fromSingleUnitString("day", input), i); - - input = "1999.38888"; - i = new CalendarInterval(0, 1999 * MICROS_PER_SECOND + 38); - assertEquals(fromSingleUnitString("second", input), i); - - try { - input = String.valueOf(Integer.MAX_VALUE); - fromSingleUnitString("year", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - - try { - input = String.valueOf(Long.MAX_VALUE / MICROS_PER_HOUR + 1); - fromSingleUnitString("hour", input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("outside range")); - } - } - @Test public void addTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); + CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); + CalendarInterval input2 = new CalendarInterval(2, 100 * MICROS_PER_HOUR); + assertEquals(input1.add(input2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + input1 = new CalendarInterval(-10, -81 * MICROS_PER_HOUR); + input2 = new CalendarInterval(75, 200 * MICROS_PER_HOUR); + assertEquals(input1.add(input2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); } @Test public void subtractTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = fromString(input); - CalendarInterval interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = fromString(input); - interval2 = fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); - } - - private static void testSingleUnit(String unit, int number, int months, long microseconds) { - Arrays.asList("interval ", "").forEach(prefix -> { - String input1 = prefix + number + " " + unit; - String input2 = prefix + number + " " + unit + "s"; - CalendarInterval result = new CalendarInterval(months, microseconds); - assertEquals(fromString(input1), result); - assertEquals(fromString(input2), result); - }); - } - - @Test - public void fromStringCaseSensitivityTest() { - testSingleUnit("YEAR", 3, 36, 0); - testSingleUnit("Month", 3, 3, 0); - testSingleUnit("Week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("DAY", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("HouR", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("MiNuTe", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("Second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("MilliSecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("MicroSecond", 3, 0, 3); - - String input; - - input = "INTERVAL -5 YEARS 23 MONTHS"; - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - assertEquals(fromString(input), result); + CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); + CalendarInterval input2 = new CalendarInterval(2, 100 * MICROS_PER_HOUR); + assertEquals(input1.subtract(input2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - assertNull(fromString("INTERVAL")); - assertNull(fromString(" Interval ")); + input1 = new CalendarInterval(-10, -81 * MICROS_PER_HOUR); + input2 = new CalendarInterval(75, 200 * MICROS_PER_HOUR); + assertEquals(input1.subtract(input2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); } } diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 01cd181010f97..82401f91e31d6 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -79,6 +79,10 @@ singleTableSchema : colTypeList EOF ; +singleInterval + : INTERVAL? (intervalValue intervalUnit)+ EOF + ; + statement : query #statementDefault | ctes? dmlStatementNoWith #dmlStatement 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 d71f300dd26dd..862b2bb515a19 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 @@ -23,7 +23,7 @@ import java.util.Locale import java.util.concurrent.TimeUnit._ import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.{InternalRow, WalkedTypePath} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.UTF8StringBuilder -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.unsafe.types.UTF8String.{IntWrapper, LongWrapper} object Cast { @@ -466,7 +466,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // IntervalConverter private[this] def castToInterval(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => CalendarInterval.fromString(s.toString)) + buildCast[UTF8String](_, s => IntervalUtils.safeFromString(s.toString)) } // LongConverter @@ -1213,8 +1213,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def castToIntervalCode(from: DataType): CastFunction = from match { case StringType => + val util = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") (c, evPrim, evNull) => - code"""$evPrim = CalendarInterval.fromString($c.toString()); + code"""$evPrim = $util.safeFromString($c.toString()); if(${evPrim} == null) { ${evNull} = true; } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 9aae678deb4bc..b9ec933f31493 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( timeColumn: Expression, @@ -102,7 +102,7 @@ object TimeWindow { * precision. */ private def getIntervalInMicroSeconds(interval: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) + val cal = IntervalUtils.fromString(interval) if (cal.months > 0) { throw new IllegalArgumentException( s"Intervals greater than a month is not supported ($interval).") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 99e5c9feb8fa3..d8e1a0cdcb10f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -100,6 +101,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging withOrigin(ctx)(StructType(visitColTypeList(ctx.colTypeList))) } + override def visitSingleInterval(ctx: SingleIntervalContext): CalendarInterval = { + withOrigin(ctx) { + val units = ctx.intervalUnit().asScala.map { + u => normalizeInternalUnit(u.getText.toLowerCase(Locale.ROOT)) + }.toArray + val values = ctx.intervalValue().asScala.map(getIntervalValue).toArray + try { + CalendarInterval.fromUnitStrings(units, values) + } catch { + case i: IllegalArgumentException => + val e = new ParseException(i.getMessage, ctx) + e.setStackTrace(i.getStackTrace) + throw e + } + } + } + /* ******************************************************************************************** * Plan parsing * ******************************************************************************************** */ @@ -1770,7 +1788,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "INTERVAL" => val interval = try { - CalendarInterval.fromCaseInsensitiveString(value) + IntervalUtils.fromString(value) } catch { case e: IllegalArgumentException => val ex = new ParseException("Cannot parse the INTERVAL value: " + value, ctx) @@ -1930,15 +1948,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitIntervalField(ctx: IntervalFieldContext): CalendarInterval = withOrigin(ctx) { import ctx._ - val s = value.getText + val s = getIntervalValue(value) try { val unitText = unit.getText.toLowerCase(Locale.ROOT) val interval = (unitText, Option(to).map(_.getText.toLowerCase(Locale.ROOT))) match { - case (u, None) if u.endsWith("s") => - // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... - CalendarInterval.fromSingleUnitString(u.substring(0, u.length - 1), s) case (u, None) => - CalendarInterval.fromSingleUnitString(u, s) + CalendarInterval.fromUnitStrings(Array(normalizeInternalUnit(u)), Array(s)) case ("year", Some("month")) => CalendarInterval.fromYearMonthString(s) case ("day", Some("hour")) => @@ -1967,6 +1982,19 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + private def getIntervalValue(value: IntervalValueContext): String = { + if (value.STRING() != null) { + string(value.STRING()) + } else { + value.getText + } + } + + // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... + private def normalizeInternalUnit(s: String): String = { + if (s.endsWith("s")) s.substring(0, s.length - 1) else s + } + /* ******************************************************************************************** * DataType parsing * ******************************************************************************************** */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index a84d29b71ac42..b66cae7979416 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -29,12 +29,21 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.unsafe.types.CalendarInterval /** * Base SQL parsing infrastructure. */ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Logging { + /** + * Creates [[CalendarInterval]] for a given SQL String. Throws [[ParseException]] if the SQL + * string is not a valid interval format. + */ + def parseInterval(sqlText: String): CalendarInterval = parse(sqlText) { parser => + astBuilder.visitSingleInterval(parser.singleInterval()) + } + /** Creates/Resolves DataType for a given SQL string. */ override def parseDataType(sqlText: String): DataType = parse(sqlText) { parser => astBuilder.visitSingleDataType(parser.singleDataType()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 78d188f81f628..14fd153e15f58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -17,21 +17,24 @@ package org.apache.spark.sql.catalyst.util +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval object IntervalUtils { - val MONTHS_PER_YEAR: Int = 12 - val MONTHS_PER_QUARTER: Byte = 3 - val YEARS_PER_MILLENNIUM: Int = 1000 - val YEARS_PER_CENTURY: Int = 100 - val YEARS_PER_DECADE: Int = 10 - val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS - val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS - val DAYS_PER_MONTH: Byte = 30 - val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY + final val MONTHS_PER_YEAR: Int = 12 + final val MONTHS_PER_QUARTER: Byte = 3 + final val YEARS_PER_MILLENNIUM: Int = 1000 + final val YEARS_PER_CENTURY: Int = 100 + final val YEARS_PER_DECADE: Int = 10 + final val MICROS_PER_HOUR: Long = + DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS + final val MICROS_PER_MINUTE: Long = + DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS + final val DAYS_PER_MONTH: Byte = 30 + final val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY /* 365.25 days per year assumes leap year every four years */ - val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 + final val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 def getYears(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -88,4 +91,32 @@ object IntervalUtils { result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) Decimal(result, 18, 6) } + + /** + * Converts a string to [[CalendarInterval]] case-insensitively. + * + * @throws IllegalArgumentException if the input string is not in valid interval format. + */ + def fromString(str: String): CalendarInterval = { + if (str == null) throw new IllegalArgumentException("Interval string cannot be null") + try { + CatalystSqlParser.parseInterval(str) + } catch { + case e: ParseException => + val ex = new IllegalArgumentException(s"Invalid interval string: $str\n" + e.message) + ex.setStackTrace(e.getStackTrace) + throw ex + } + } + + /** + * A safe version of `fromString`. It returns null for invalid input string. + */ + def safeFromString(str: String): CalendarInterval = { + try { + fromString(str) + } catch { + case _: IllegalArgumentException => null + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 603073b40d7aa..e10aa60d52cf8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH @@ -720,7 +720,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:00")), - Literal(CalendarInterval.fromString("interval 12 hours"))), + Literal(IntervalUtils.fromString("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -729,7 +729,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:01")), - Literal(CalendarInterval.fromString("interval 12 hours"))), + Literal(IntervalUtils.fromString("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -738,7 +738,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 12 hours").negate())), + Literal(IntervalUtils.fromString("interval 12 hours").negate())), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -747,7 +747,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2017-12-31 23:59:59")), - Literal(CalendarInterval.fromString("interval 12 hours").negate())), + Literal(IntervalUtils.fromString("interval 12 hours").negate())), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -756,7 +756,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -765,7 +765,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month").negate())), + Literal(IntervalUtils.fromString("interval 1 month").negate())), Seq( Timestamp.valueOf("2018-03-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -774,7 +774,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-03 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month 1 day").negate())), + Literal(IntervalUtils.fromString("interval 1 month 1 day").negate())), Seq( Timestamp.valueOf("2018-03-03 00:00:00"), Timestamp.valueOf("2018-02-02 00:00:00"), @@ -783,7 +783,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-31 00:00:00")), Literal(Timestamp.valueOf("2018-04-30 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-31 00:00:00"), Timestamp.valueOf("2018-02-28 00:00:00"), @@ -793,7 +793,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(CalendarInterval.fromString("interval 1 month 1 second"))), + Literal(IntervalUtils.fromString("interval 1 month 1 second"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:01"))) @@ -801,7 +801,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:04:06")), - Literal(CalendarInterval.fromString("interval 1 month 2 minutes 3 seconds"))), + Literal(IntervalUtils.fromString("interval 1 month 2 minutes 3 seconds"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:02:03"), @@ -839,7 +839,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-25 01:30:00")), Literal(Timestamp.valueOf("2018-03-25 03:30:00")), - Literal(CalendarInterval.fromString("interval 30 minutes"))), + Literal(IntervalUtils.fromString("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-03-25 01:30:00"), Timestamp.valueOf("2018-03-25 03:00:00"), @@ -849,7 +849,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-10-28 01:30:00")), Literal(Timestamp.valueOf("2018-10-28 03:30:00")), - Literal(CalendarInterval.fromString("interval 30 minutes"))), + Literal(IntervalUtils.fromString("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-10-28 01:30:00"), noDST(Timestamp.valueOf("2018-10-28 02:00:00")), @@ -866,7 +866,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-01-05")), - Literal(CalendarInterval.fromString("interval 2 days"))), + Literal(IntervalUtils.fromString("interval 2 days"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-01-03"), @@ -875,7 +875,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-03-01")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-02-01"), @@ -884,7 +884,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-31")), Literal(Date.valueOf("2018-04-30")), - Literal(CalendarInterval.fromString("interval 1 month"))), + Literal(IntervalUtils.fromString("interval 1 month"))), Seq( Date.valueOf("2018-01-31"), Date.valueOf("2018-02-28"), @@ -905,14 +905,14 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence( Literal(Date.valueOf("1970-01-02")), Literal(Date.valueOf("1970-01-01")), - Literal(CalendarInterval.fromString("interval 1 day"))), + Literal(IntervalUtils.fromString("interval 1 day"))), EmptyRow, "sequence boundaries: 1 to 0 by 1") checkExceptionInExpression[IllegalArgumentException]( new Sequence( Literal(Date.valueOf("1970-01-01")), Literal(Date.valueOf("1970-02-01")), - Literal(CalendarInterval.fromString("interval 1 month").negate())), + Literal(IntervalUtils.fromString("interval 1 month").negate())), EmptyRow, s"sequence boundaries: 0 to 2678400000000 by -${28 * CalendarInterval.MICROS_PER_DAY}") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index e893e863b3675..6abadd77bd41a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT import org.apache.spark.sql.internal.SQLConf @@ -1075,16 +1075,16 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), new CalendarInterval(0, 0)) checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), - CalendarInterval.fromString("interval 18173 days " + + IntervalUtils.fromString("interval 18173 days " + "11 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds")) checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), - CalendarInterval.fromString("interval -18173 days " + + IntervalUtils.fromString("interval -18173 days " + "-11 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds")) checkEvaluation( SubtractTimestamps( Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), Literal(Instant.parse("0001-01-01T00:00:00Z"))), - CalendarInterval.fromString("interval 521722 weeks 4 days " + + IntervalUtils.fromString("interval 521722 weeks 4 days " + "23 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds")) } @@ -1093,18 +1093,18 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractDates(Literal(end), Literal(end)), new CalendarInterval(0, 0)) checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), - CalendarInterval.fromString("interval 1 days")) + IntervalUtils.fromString("interval 1 days")) checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), - CalendarInterval.fromString("interval -1 days")) + IntervalUtils.fromString("interval -1 days")) val epochDate = Literal(LocalDate.ofEpochDay(0)) checkEvaluation(SubtractDates(Literal(end), epochDate), - CalendarInterval.fromString("interval 49 years 9 months 4 days")) + IntervalUtils.fromString("interval 49 years 9 months 4 days")) checkEvaluation(SubtractDates(epochDate, Literal(end)), - CalendarInterval.fromString("interval -49 years -9 months -4 days")) + IntervalUtils.fromString("interval -49 years -9 months -4 days")) checkEvaluation( SubtractDates( Literal(LocalDate.of(10000, 1, 1)), Literal(LocalDate.of(1, 1, 1))), - CalendarInterval.fromString("interval 9999 years")) + IntervalUtils.fromString("interval 9999 years")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index f90c98be0b3fd..4b2da73abe562 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -30,9 +30,9 @@ import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.{ExamplePointUDT, RowEncoder} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData, IntervalUtils} import org.apache.spark.sql.types.{ArrayType, StructType, _} -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val random = new scala.util.Random @@ -252,7 +252,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("hive-hash for CalendarInterval type") { def checkHiveHashForIntervalType(interval: String, expected: Long): Unit = { - checkHiveHash(CalendarInterval.fromString(interval), CalendarIntervalType, expected) + checkHiveHash(IntervalUtils.fromString(interval), CalendarIntervalType, expected) } // ----- MICROSEC ----- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index 078ec88800215..818ee239dbbf8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.catalyst.expressions import scala.language.implicitConversions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.CalendarInterval class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { implicit def interval(s: String): Literal = { - Literal(CalendarInterval.fromString("interval " + s)) + Literal(IntervalUtils.fromString("interval " + s)) } test("millenniums") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala index 0d594eb10962e..23ba9c6ec7388 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -56,7 +56,7 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { testBothCodegenAndInterpreted("variable-length types") { val proj = createMutableProjection(variableLengthTypes) - val scalaValues = Seq("abc", BigDecimal(10), CalendarInterval.fromString("interval 1 day"), + val scalaValues = Seq("abc", BigDecimal(10), IntervalUtils.fromString("interval 1 day"), Array[Byte](1, 2), Array("123", "456"), Map(1 -> "a", 2 -> "b"), Row(1, "a"), new java.lang.Integer(5)) val inputRow = InternalRow.fromSeq(scalaValues.zip(variableLengthTypes).map { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index a171885471a36..4ccd4f7ce798d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -35,8 +35,7 @@ import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -486,7 +485,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { ("abcd".getBytes, BinaryType), ("abcd", StringType), (BigDecimal.valueOf(10), DecimalType.IntDecimal), - (CalendarInterval.fromString("interval 3 day"), CalendarIntervalType), + (IntervalUtils.fromString("interval 3 day"), CalendarIntervalType), (java.math.BigDecimal.valueOf(10), DecimalType.BigIntDecimal), (Array(3, 2, 1), ArrayType(IntegerType)) ).foreach { case (input, dt) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 323a3a901689f..20e77254ecdad 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -531,7 +531,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB // Simple tests val inputRow = InternalRow.fromSeq(Seq( false, 3.toByte, 15.toShort, -83, 129L, 1.0f, 8.0, UTF8String.fromString("test"), - Decimal(255), CalendarInterval.fromString("interval 1 day"), Array[Byte](1, 2) + Decimal(255), IntervalUtils.fromString("interval 1 day"), Array[Byte](1, 2) )) val fields1 = Array( BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index e6eabcc1f3022..86b3aa8190b45 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} -import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -432,7 +432,7 @@ class ExpressionParserSuite extends AnalysisTest { intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the TIMESTAMP value") // Interval. - val intervalLiteral = Literal(CalendarInterval.fromString("interval 3 month 1 hour")) + val intervalLiteral = Literal(IntervalUtils.fromString("interval 3 month 1 hour")) assertEqual("InterVal 'interval 3 month 1 hour'", intervalLiteral) assertEqual("INTERVAL '3 month 1 hour'", intervalLiteral) intercept("Interval 'interval 3 monthsss 1 hoursss'", "Cannot parse the INTERVAL value") @@ -597,7 +597,7 @@ class ExpressionParserSuite extends AnalysisTest { "microsecond") def intervalLiteral(u: String, s: String): Literal = { - Literal(CalendarInterval.fromSingleUnitString(u, s)) + Literal(CalendarInterval.fromUnitStrings(Array(u), Array(s))) } test("intervals") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala new file mode 100644 index 0000000000000..e48779af3c9aa --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -0,0 +1,90 @@ +/* + * 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.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.IntervalUtils.fromString +import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.CalendarInterval._ + +class IntervalUtilsSuite extends SparkFunSuite { + + test("fromString: basic") { + testSingleUnit("YEAR", 3, 36, 0) + testSingleUnit("Month", 3, 3, 0) + testSingleUnit("Week", 3, 0, 3 * MICROS_PER_WEEK) + testSingleUnit("DAY", 3, 0, 3 * MICROS_PER_DAY) + testSingleUnit("HouR", 3, 0, 3 * MICROS_PER_HOUR) + testSingleUnit("MiNuTe", 3, 0, 3 * MICROS_PER_MINUTE) + testSingleUnit("Second", 3, 0, 3 * MICROS_PER_SECOND) + testSingleUnit("MilliSecond", 3, 0, 3 * MICROS_PER_MILLI) + testSingleUnit("MicroSecond", 3, 0, 3) + + for (input <- Seq(null, "", " ")) { + try { + fromString(input) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + val msg = e.getMessage + if (input == null) { + assert(msg.contains("cannot be null")) + } + } + } + + for (input <- Seq("interval", "interval1 day", "foo", "foo 1 day")) { + try { + fromString(input) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + val msg = e.getMessage + assert(msg.contains("Invalid interval string")) + } + } + } + + test("fromString: random order field") { + val input = "1 day 1 year" + val result = new CalendarInterval(12, MICROS_PER_DAY) + assert(fromString(input) == result) + } + + test("fromString: duplicated fields") { + val input = "1 day 1 day" + val result = new CalendarInterval(0, 2 * MICROS_PER_DAY) + assert(fromString(input) == result) + } + + test("fromString: value with +/-") { + val input = "+1 year -1 day" + val result = new CalendarInterval(12, -MICROS_PER_DAY) + assert(fromString(input) == result) + } + + private def testSingleUnit(unit: String, number: Int, months: Int, microseconds: Long): Unit = { + for (prefix <- Seq("interval ", "")) { + val input1 = prefix + number + " " + unit + val input2 = prefix + number + " " + unit + "s" + val result = new CalendarInterval(months, microseconds) + assert(fromString(input1) == result) + assert(fromString(input2) == result) + } + } +} diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt index 6605bd2c13b47..221ac42022a15 100644 --- a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt @@ -2,24 +2,24 @@ OpenJDK 64-Bit Server VM 11.0.4+11-post-Ubuntu-1ubuntu218.04.3 on Linux 4.15.0-1 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -string w/ interval 663 758 99 1.5 663.2 1.0X -string w/o interval 563 582 19 1.8 563.2 1.2X -1 units w/ interval 891 953 97 1.1 891.2 0.7X -1 units w/o interval 894 905 15 1.1 893.6 0.7X -2 units w/ interval 1142 1169 24 0.9 1141.8 0.6X -2 units w/o interval 1195 1200 7 0.8 1194.7 0.6X -3 units w/ interval 1449 1452 3 0.7 1449.5 0.5X -3 units w/o interval 1489 1491 1 0.7 1489.3 0.4X -4 units w/ interval 1688 1690 1 0.6 1688.4 0.4X -4 units w/o interval 1711 1722 11 0.6 1710.9 0.4X -5 units w/ interval 1961 1983 23 0.5 1961.3 0.3X -5 units w/o interval 1983 1991 10 0.5 1983.4 0.3X -6 units w/ interval 2217 2228 11 0.5 2216.9 0.3X -6 units w/o interval 2240 2244 5 0.4 2239.6 0.3X -7 units w/ interval 2454 2468 16 0.4 2454.1 0.3X -7 units w/o interval 2480 2491 15 0.4 2479.5 0.3X -8 units w/ interval 2762 2792 26 0.4 2761.9 0.2X -8 units w/o interval 2763 2778 14 0.4 2762.9 0.2X -9 units w/ interval 3036 3060 21 0.3 3036.4 0.2X -9 units w/o interval 3095 3111 15 0.3 3094.8 0.2X +prepare string w/ interval 672 728 64 1.5 672.1 1.0X +prepare string w/o interval 580 602 19 1.7 580.4 1.2X +1 units w/ interval 9450 9575 138 0.1 9449.6 0.1X +1 units w/o interval 8948 8968 19 0.1 8948.3 0.1X +2 units w/ interval 10947 10966 19 0.1 10947.1 0.1X +2 units w/o interval 10470 10489 26 0.1 10469.5 0.1X +3 units w/ interval 12265 12333 72 0.1 12264.5 0.1X +3 units w/o interval 12001 12004 3 0.1 12000.6 0.1X +4 units w/ interval 13749 13828 69 0.1 13748.5 0.0X +4 units w/o interval 13467 13479 15 0.1 13467.3 0.0X +5 units w/ interval 15392 15446 51 0.1 15392.1 0.0X +5 units w/o interval 15090 15107 29 0.1 15089.7 0.0X +6 units w/ interval 16696 16714 20 0.1 16695.9 0.0X +6 units w/o interval 16361 16366 5 0.1 16361.4 0.0X +7 units w/ interval 18190 18270 71 0.1 18190.2 0.0X +7 units w/o interval 17757 17767 9 0.1 17756.7 0.0X +8 units w/ interval 19821 19870 43 0.1 19820.7 0.0X +8 units w/o interval 19479 19555 97 0.1 19479.5 0.0X +9 units w/ interval 21417 21481 56 0.0 21417.1 0.0X +9 units w/o interval 21058 21131 86 0.0 21058.2 0.0X diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt index 40169826cc627..60e8e5198353c 100644 --- a/sql/core/benchmarks/IntervalBenchmark-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -2,24 +2,25 @@ OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15. Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -string w/ interval 600 641 37 1.7 600.3 1.0X -string w/o interval 536 544 12 1.9 536.4 1.1X -1 units w/ interval 1017 1027 9 1.0 1016.8 0.6X -1 units w/o interval 1055 1073 16 0.9 1054.7 0.6X -2 units w/ interval 1272 1292 29 0.8 1272.2 0.5X -2 units w/o interval 1309 1314 9 0.8 1309.0 0.5X -3 units w/ interval 1545 1566 20 0.6 1544.8 0.4X -3 units w/o interval 1606 1610 5 0.6 1605.8 0.4X -4 units w/ interval 1820 1826 6 0.5 1819.7 0.3X -4 units w/o interval 1882 1885 3 0.5 1881.5 0.3X -5 units w/ interval 2039 2043 7 0.5 2038.9 0.3X -5 units w/o interval 2131 2133 3 0.5 2130.6 0.3X -6 units w/ interval 2269 2272 4 0.4 2269.5 0.3X -6 units w/o interval 2327 2333 6 0.4 2327.2 0.3X -7 units w/ interval 2477 2485 10 0.4 2476.8 0.2X -7 units w/o interval 2536 2538 3 0.4 2536.0 0.2X -8 units w/ interval 2764 2781 27 0.4 2763.8 0.2X -8 units w/o interval 2843 2847 5 0.4 2842.9 0.2X -9 units w/ interval 2983 2997 12 0.3 2982.5 0.2X -9 units w/o interval 3071 3072 1 0.3 3071.1 0.2X +prepare string w/ interval 596 647 61 1.7 596.0 1.0X +prepare string w/o interval 530 554 22 1.9 530.2 1.1X +1 units w/ interval 9168 9243 66 0.1 9167.8 0.1X +1 units w/o interval 8740 8744 5 0.1 8740.2 0.1X +2 units w/ interval 10815 10874 52 0.1 10815.0 0.1X +2 units w/o interval 10413 10419 11 0.1 10412.8 0.1X +3 units w/ interval 12490 12530 37 0.1 12490.3 0.0X +3 units w/o interval 12173 12180 9 0.1 12172.8 0.0X +4 units w/ interval 13788 13834 43 0.1 13788.0 0.0X +4 units w/o interval 13445 13456 10 0.1 13445.5 0.0X +5 units w/ interval 15313 15330 15 0.1 15312.7 0.0X +5 units w/o interval 14928 14942 16 0.1 14928.0 0.0X +6 units w/ interval 16959 17003 42 0.1 16959.1 0.0X +6 units w/o interval 16623 16627 5 0.1 16623.3 0.0X +7 units w/ interval 18955 18972 21 0.1 18955.4 0.0X +7 units w/o interval 18454 18462 7 0.1 18454.1 0.0X +8 units w/ interval 20835 20843 8 0.0 20835.4 0.0X +8 units w/o interval 20446 20463 19 0.0 20445.7 0.0X +9 units w/ interval 22981 23031 43 0.0 22981.4 0.0X +9 units w/o interval 22581 22603 25 0.0 22581.1 0.0X + 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 076270a9f1c6b..5f6e0a82be4ce 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 @@ -46,6 +46,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} import org.apache.spark.sql.execution.command._ @@ -724,7 +725,7 @@ class Dataset[T] private[sql]( def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { val parsedDelay = try { - CalendarInterval.fromCaseInsensitiveString(delayThreshold) + IntervalUtils.fromString(delayThreshold) } catch { case e: IllegalArgumentException => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala index dda9d41f630e6..d191a79187f28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala @@ -21,6 +21,7 @@ import java.sql.Date import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, ProcessingTimeTimeout} +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution.streaming.GroupStateImpl._ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout} import org.apache.spark.unsafe.types.CalendarInterval @@ -159,7 +160,7 @@ private[sql] class GroupStateImpl[S] private( def getTimeoutTimestamp: Long = timeoutTimestamp private def parseDuration(duration: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(duration) + val cal = IntervalUtils.fromString(duration) if (cal.milliseconds < 0 || cal.months < 0) { throw new IllegalArgumentException(s"Provided duration ($duration) is not positive") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 2bdb3402c14b1..daa70a12ba0e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -21,8 +21,8 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration.Duration +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.streaming.Trigger -import org.apache.spark.unsafe.types.CalendarInterval private object Triggers { def validate(intervalMs: Long): Unit = { @@ -30,7 +30,7 @@ private object Triggers { } def convert(interval: String): Long = { - val cal = CalendarInterval.fromCaseInsensitiveString(interval) + val cal = IntervalUtils.fromString(interval) if (cal.months > 0) { throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") } diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 816386c483209..0f95f85237828 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -85,6 +85,19 @@ select timestamp '2016-33-11 20:54:00.000'; -- interval select interval 13.123456789 seconds, interval -13.123456789 second; select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond, 9 microsecond; +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; +select interval '0 0:0:0.1' day to second; +select interval '10-9' year to month; +select interval '20 15:40:32.99899999' day to hour; +select interval '20 15:40:32.99899999' day to minute; +select interval '20 15:40:32.99899999' day to second; +select interval '15:40:32.99899999' hour to minute; +select interval '15:40.99899999' hour to second; +select interval '15:40' hour to second; +select interval '15:40:32.99899999' hour to second; +select interval '20 40:32.99899999' minute to second; +select interval '40:32.99899999' minute to second; +select interval '40:32' minute to second; -- ns is not supported select interval 10 nanoseconds; diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index aef23963da374..fd6e51b2385de 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 46 +-- Number of queries: 59 -- !query 0 @@ -337,10 +337,114 @@ interval 1 years 2 months 3 weeks 4 days 5 hours 6 minutes 7 seconds 8 milliseco -- !query 36 -select interval 10 nanoseconds +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 36 schema -struct<> +struct -- !query 36 output +interval 32 years 1 months -14 weeks -6 hours -35 minutes -110 milliseconds -13 microseconds + + +-- !query 37 +select interval '0 0:0:0.1' day to second +-- !query 37 schema +struct +-- !query 37 output +interval 100 milliseconds + + +-- !query 38 +select interval '10-9' year to month +-- !query 38 schema +struct +-- !query 38 output +interval 10 years 9 months + + +-- !query 39 +select interval '20 15:40:32.99899999' day to hour +-- !query 39 schema +struct +-- !query 39 output +interval 2 weeks 6 days 15 hours + + +-- !query 40 +select interval '20 15:40:32.99899999' day to minute +-- !query 40 schema +struct +-- !query 40 output +interval 2 weeks 6 days 15 hours 40 minutes + + +-- !query 41 +select interval '20 15:40:32.99899999' day to second +-- !query 41 schema +struct +-- !query 41 output +interval 2 weeks 6 days 15 hours 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 42 +select interval '15:40:32.99899999' hour to minute +-- !query 42 schema +struct +-- !query 42 output +interval 15 hours 40 minutes + + +-- !query 43 +select interval '15:40.99899999' hour to second +-- !query 43 schema +struct +-- !query 43 output +interval 15 minutes 40 seconds 998 milliseconds 999 microseconds + + +-- !query 44 +select interval '15:40' hour to second +-- !query 44 schema +struct +-- !query 44 output +interval 15 hours 40 minutes + + +-- !query 45 +select interval '15:40:32.99899999' hour to second +-- !query 45 schema +struct +-- !query 45 output +interval 15 hours 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 46 +select interval '20 40:32.99899999' minute to second +-- !query 46 schema +struct +-- !query 46 output +interval 2 weeks 6 days 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 47 +select interval '40:32.99899999' minute to second +-- !query 47 schema +struct +-- !query 47 output +interval 40 minutes 32 seconds 998 milliseconds 999 microseconds + + +-- !query 48 +select interval '40:32' minute to second +-- !query 48 schema +struct +-- !query 48 output +interval 40 minutes 32 seconds + + +-- !query 49 +select interval 10 nanoseconds +-- !query 49 schema +struct<> +-- !query 49 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'interval 10 nanoseconds'(line 1, pos 19) @@ -350,11 +454,11 @@ select interval 10 nanoseconds -------------------^^^ --- !query 37 +-- !query 50 select GEO '(10,-6)' --- !query 37 schema +-- !query 50 schema struct<> --- !query 37 output +-- !query 50 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'GEO' are currently not supported.(line 1, pos 7) @@ -364,19 +468,19 @@ select GEO '(10,-6)' -------^^^ --- !query 38 +-- !query 51 select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD --- !query 38 schema +-- !query 51 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> --- !query 38 output +-- !query 51 output 90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 --- !query 39 +-- !query 52 select 1.20E-38BD --- !query 39 schema +-- !query 52 schema struct<> --- !query 39 output +-- !query 52 output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38(line 1, pos 7) @@ -386,19 +490,19 @@ select 1.20E-38BD -------^^^ --- !query 40 +-- !query 53 select x'2379ACFe' --- !query 40 schema +-- !query 53 schema struct --- !query 40 output +-- !query 53 output #y�� --- !query 41 +-- !query 54 select X'XuZ' --- !query 41 schema +-- !query 54 schema struct<> --- !query 41 output +-- !query 54 output org.apache.spark.sql.catalyst.parser.ParseException contains illegal character for hexBinary: 0XuZ(line 1, pos 7) @@ -408,33 +512,33 @@ select X'XuZ' -------^^^ --- !query 42 +-- !query 55 SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 --- !query 42 schema +-- !query 55 schema struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10),-3.14E+8:decimal(3,-6),-3.14E-8:decimal(10,10),3.14E+8:decimal(3,-6),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10)> --- !query 42 output +-- !query 55 output 3.14 -3.14 314000000 0.0000000314 -314000000 -0.0000000314 314000000 314000000 0.0000000314 --- !query 43 +-- !query 56 select map(1, interval 1 day, 2, interval 3 week) --- !query 43 schema +-- !query 56 schema struct> --- !query 43 output +-- !query 56 output {1:interval 1 days,2:interval 3 weeks} --- !query 44 +-- !query 57 select interval 'interval 3 year 1 hour' --- !query 44 schema +-- !query 57 schema struct --- !query 44 output +-- !query 57 output interval 3 years 1 hours --- !query 45 +-- !query 58 select interval '3 year 1 hour' --- !query 45 schema +-- !query 58 schema struct --- !query 45 output +-- !query 58 output interval 3 years 1 hours diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala index 4c1c75b815a02..d75cb1040f31e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala @@ -88,8 +88,9 @@ object IntervalBenchmark extends SqlBasedBenchmark { val intervalToTest = ListBuffer[String]() val benchmark = new Benchmark("cast strings to intervals", N, output = output) - addCase(benchmark, N, "string w/ interval", buildString(true, timeUnits)) - addCase(benchmark, N, "string w/o interval", buildString(false, timeUnits)) + // The first 2 cases are used to show the overhead of preparing the interval string. + addCase(benchmark, N, "prepare string w/ interval", buildString(true, timeUnits)) + addCase(benchmark, N, "prepare string w/o interval", buildString(false, timeUnits)) addCase(benchmark, N, intervalToTest) // Only years for (unit <- timeUnits) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 116a62b2f6dc6..3e5834f33ea52 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -42,7 +42,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval case class Nested1(f1: Nested2) case class Nested2(f2: Nested3) @@ -1183,51 +1182,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } - test("Convert hive interval term into Literal of CalendarIntervalType") { - checkAnswer(sql("select interval '0 0:0:0.1' day to second"), - Row(CalendarInterval.fromString("interval 100 milliseconds"))) - checkAnswer(sql("select interval '10-9' year to month"), - Row(CalendarInterval.fromString("interval 10 years 9 months"))) - checkAnswer(sql("select interval '20 15:40:32.99899999' day to hour"), - Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours"))) - checkAnswer(sql("select interval '20 15:40:32.99899999' day to minute"), - Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours 40 minutes"))) - checkAnswer(sql("select interval '20 15:40:32.99899999' day to second"), - Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours 40 minutes " + - "32 seconds 998 milliseconds 999 microseconds"))) - checkAnswer(sql("select interval '15:40:32.99899999' hour to minute"), - Row(CalendarInterval.fromString("interval 15 hours 40 minutes"))) - checkAnswer(sql("select interval '15:40.99899999' hour to second"), - Row(CalendarInterval.fromString("interval 15 minutes 40 seconds 998 milliseconds " + - "999 microseconds"))) - checkAnswer(sql("select interval '15:40' hour to second"), - Row(CalendarInterval.fromString("interval 15 hours 40 minutes"))) - checkAnswer(sql("select interval '15:40:32.99899999' hour to second"), - Row(CalendarInterval.fromString("interval 15 hours 40 minutes 32 seconds 998 milliseconds " + - "999 microseconds"))) - checkAnswer(sql("select interval '20 40:32.99899999' minute to second"), - Row(CalendarInterval.fromString("interval 2 weeks 6 days 40 minutes 32 seconds " + - "998 milliseconds 999 microseconds"))) - checkAnswer(sql("select interval '40:32.99899999' minute to second"), - Row(CalendarInterval.fromString("interval 40 minutes 32 seconds 998 milliseconds " + - "999 microseconds"))) - checkAnswer(sql("select interval '40:32' minute to second"), - Row(CalendarInterval.fromString("interval 40 minutes 32 seconds"))) - checkAnswer(sql("select interval '30' year"), - Row(CalendarInterval.fromString("interval 30 years"))) - checkAnswer(sql("select interval '25' month"), - Row(CalendarInterval.fromString("interval 25 months"))) - checkAnswer(sql("select interval '-100' day"), - Row(CalendarInterval.fromString("interval -14 weeks -2 days"))) - checkAnswer(sql("select interval '40' hour"), - Row(CalendarInterval.fromString("interval 1 days 16 hours"))) - checkAnswer(sql("select interval '80' minute"), - Row(CalendarInterval.fromString("interval 1 hour 20 minutes"))) - checkAnswer(sql("select interval '299.889987299' second"), - Row(CalendarInterval.fromString( - "interval 4 minutes 59 seconds 889 milliseconds 987 microseconds"))) - } - test("specifying database name for a temporary view is not allowed") { withTempPath { dir => withTempView("db.t") { From dcf5eaf1a6c0330a9460e168c1c3fee21998ba65 Mon Sep 17 00:00:00 2001 From: stczwd Date: Thu, 24 Oct 2019 10:25:04 -0700 Subject: [PATCH 0420/1387] [SPARK-29444][FOLLOWUP] add doc and python parameter for ignoreNullFields in json generating # What changes were proposed in this pull request? Add description for ignoreNullFields, which is commited in #26098 , in DataFrameWriter and readwriter.py. Enable user to use ignoreNullFields in pyspark. ### Does this PR introduce any user-facing change? No ### How was this patch tested? run unit tests Closes #26227 from stczwd/json-generator-doc. Authored-by: stczwd Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/readwriter.py | 6 ++++-- .../apache/spark/sql/catalyst/json/JSONOptions.scala | 4 ++-- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 10 ++++++---- .../scala/org/apache/spark/sql/DataFrameWriter.scala | 2 ++ 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index f92face2d0573..18fd7de7ee547 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -788,7 +788,7 @@ def saveAsTable(self, name, format=None, mode=None, partitionBy=None, **options) @since(1.4) def json(self, path, mode=None, compression=None, dateFormat=None, timestampFormat=None, - lineSep=None, encoding=None): + lineSep=None, encoding=None, ignoreNullFields=None): """Saves the content of the :class:`DataFrame` in JSON format (`JSON Lines text format or newline-delimited JSON `_) at the specified path. @@ -817,13 +817,15 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm the default UTF-8 charset will be used. :param lineSep: defines the line separator that should be used for writing. If None is set, it uses the default value, ``\\n``. + :param ignoreNullFields: Whether to ignore null fields when generating JSON objects. + If None is set, it uses the default value, ``true``. >>> df.write.json(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode) self._set_opts( compression=compression, dateFormat=dateFormat, timestampFormat=timestampFormat, - lineSep=lineSep, encoding=encoding) + lineSep=lineSep, encoding=encoding, ignoreNullFields=ignoreNullFields) self._jwrite.json(path) @since(1.4) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index e7bfb77e46c26..4952540f1132d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -78,8 +78,8 @@ private[sql] class JSONOptions( val dropFieldIfAllNull = parameters.get("dropFieldIfAllNull").map(_.toBoolean).getOrElse(false) // Whether to ignore null fields during json generating - val ignoreNullFields = parameters.getOrElse("ignoreNullFields", - SQLConf.get.jsonGeneratorIgnoreNullFields).toBoolean + val ignoreNullFields = parameters.get("ignoreNullFields").map(_.toBoolean) + .getOrElse(SQLConf.get.jsonGeneratorIgnoreNullFields) // A language tag in IETF BCP 47 format val locale: Locale = parameters.get("locale").map(Locale.forLanguageTag).getOrElse(Locale.US) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7f75bf84d65ae..a228d9f064a1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1196,9 +1196,11 @@ object SQLConf { val JSON_GENERATOR_IGNORE_NULL_FIELDS = buildConf("spark.sql.jsonGenerator.ignoreNullFields") - .doc("If false, JacksonGenerator will generate null for null fields in Struct.") - .stringConf - .createWithDefault("true") + .doc("Whether to ignore null fields when generating JSON objects in JSON data source and " + + "JSON functions such as to_json. " + + "If false, it generates null for null fields in JSON objects.") + .booleanConf + .createWithDefault(true) val FILE_SINK_LOG_DELETION = buildConf("spark.sql.streaming.fileSink.log.deletion") .internal() @@ -2392,7 +2394,7 @@ class SQLConf extends Serializable with Logging { def sessionLocalTimeZone: String = getConf(SQLConf.SESSION_LOCAL_TIMEZONE) - def jsonGeneratorIgnoreNullFields: String = getConf(SQLConf.JSON_GENERATOR_IGNORE_NULL_FIELDS) + def jsonGeneratorIgnoreNullFields: Boolean = getConf(SQLConf.JSON_GENERATOR_IGNORE_NULL_FIELDS) def parallelFileListingInStatsComputation: Boolean = getConf(SQLConf.PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 4f88cc6daa331..68127c27a8cc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -687,6 +687,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
  • `encoding` (by default it is not set): specifies encoding (charset) of saved json * files. If it is not set, the UTF-8 charset will be used.
  • *
  • `lineSep` (default `\n`): defines the line separator that should be used for writing.
  • + *
  • `ignoreNullFields` (default `true`): Whether to ignore null fields + * when generating JSON objects.
  • * * * @since 1.4.0 From 92b25295ca0dc5b80aaddb1c8f8d5ef0a250d11e Mon Sep 17 00:00:00 2001 From: fuwhu Date: Thu, 24 Oct 2019 12:35:32 -0700 Subject: [PATCH 0421/1387] [SPARK-21287][SQL] Remove requirement of fetch_size>=0 from JDBCOptions ### What changes were proposed in this pull request? Remove the requirement of fetch_size>=0 from JDBCOptions to allow negative fetch size. ### Why are the changes needed? Namely, to allow data fetch in stream manner (row-by-row fetch) against MySQL database. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test (JDBCSuite) This closes #26230 . Closes #26244 from fuwhu/SPARK-21287-FIX. Authored-by: fuwhu Signed-off-by: Dongjoon Hyun --- .../sql/execution/datasources/jdbc/JDBCOptions.scala | 9 +-------- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 9 --------- 2 files changed, 1 insertion(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index d184f3cb71b1a..5d1feaed81a9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -147,14 +147,7 @@ class JDBCOptions( """.stripMargin ) - val fetchSize = { - val size = parameters.getOrElse(JDBC_BATCH_FETCH_SIZE, "0").toInt - require(size >= 0, - s"Invalid value `${size.toString}` for parameter " + - s"`$JDBC_BATCH_FETCH_SIZE`. The minimum value is 0. When the value is 0, " + - "the JDBC driver ignores the value and does the estimates.") - size - } + val fetchSize = parameters.getOrElse(JDBC_BATCH_FETCH_SIZE, "0").toInt // ------------------------------------------------------------ // Optional parameters only for writing diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 3c8ce0a3fc3e4..715534b0458dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -450,15 +450,6 @@ class JDBCSuite extends QueryTest urlWithUserAndPass, "TEST.PEOPLE", new Properties()).collect().length === 3) } - test("Basic API with illegal fetchsize") { - val properties = new Properties() - properties.setProperty(JDBCOptions.JDBC_BATCH_FETCH_SIZE, "-1") - val e = intercept[IllegalArgumentException] { - spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", properties).collect() - }.getMessage - assert(e.contains("Invalid value `-1` for parameter `fetchsize`")) - } - test("Missing partition columns") { withView("tempPeople") { val e = intercept[IllegalArgumentException] { From dec99d8ac5aeda045e611fe2f9e27facd4cecef4 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 24 Oct 2019 14:51:23 -0700 Subject: [PATCH 0422/1387] [SPARK-29526][SQL] UNCACHE TABLE should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add UncacheTableStatement and make UNCACHE TABLE go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog UNCACHE TABLE t // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? yes. When running UNCACHE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? New unit tests Closes #26237 from imback82/uncache_table. Authored-by: Terry Kim Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 7 +++++++ .../sql/catalyst/plans/logical/statements.scala | 7 +++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 16 +++++++++++++--- .../analysis/ResolveSessionCatalog.scala | 6 +++++- .../spark/sql/execution/SparkSqlParser.scala | 7 ------- .../sql/connector/DataSourceV2SQLSuite.scala | 10 ++++++++++ 7 files changed, 43 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 82401f91e31d6..1e89507411adb 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -211,7 +211,7 @@ statement | REFRESH (STRING | .*?) #refreshResource | CACHE LAZY? TABLE multipartIdentifier (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable - | UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable + | UNCACHE TABLE (IF EXISTS)? multipartIdentifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d8e1a0cdcb10f..b030227b48814 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2821,6 +2821,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging CacheTableStatement(tableName, query, ctx.LAZY != null, options) } + /** + * Create an [[UncacheTableStatement]] logical plan. + */ + override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { + UncacheTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier), ctx.EXISTS != null) + } + /** * Create a [[TruncateTableStatement]] command. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 4a91ee6d52d9e..ef8c922694347 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -339,6 +339,13 @@ case class CacheTableStatement( isLazy: Boolean, options: Map[String, String]) extends ParsedStatement +/** + * An UNCACHE TABLE statement, as parsed from SQL + */ +case class UncacheTableStatement( + tableName: Seq[String], + ifExists: Boolean) extends ParsedStatement + /** * A TRUNCATE TABLE statement, as parsed from SQL */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 37349f7a33420..f4375956f0af6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1039,13 +1039,13 @@ class DDLParserSuite extends AnalysisTest { "missing 'COLUMNS' at ''") } - test("MSCK REPAIR table") { + test("MSCK REPAIR TABLE") { comparePlans( parsePlan("MSCK REPAIR TABLE a.b.c"), RepairTableStatement(Seq("a", "b", "c"))) } - test("CACHE table") { + test("CACHE TABLE") { comparePlans( parsePlan("CACHE TABLE a.b.c"), CacheTableStatement(Seq("a", "b", "c"), None, false, Map.empty)) @@ -1062,6 +1062,16 @@ class DDLParserSuite extends AnalysisTest { "It is not allowed to add catalog/namespace prefix a.b") } + test("UNCACHE TABLE") { + comparePlans( + parsePlan("UNCACHE TABLE a.b.c"), + UncacheTableStatement(Seq("a", "b", "c"), ifExists = false)) + + comparePlans( + parsePlan("UNCACHE TABLE IF EXISTS a.b.c"), + UncacheTableStatement(Seq("a", "b", "c"), ifExists = true)) + } + test("TRUNCATE table") { comparePlans( parsePlan("TRUNCATE TABLE a.b.c"), @@ -1098,7 +1108,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsed5, expected5) } - test("REFRESH TABLE table") { + test("REFRESH TABLE") { comparePlans( parsePlan("REFRESH TABLE a.b.c"), RefreshTableStatement(Seq("a", "b", "c"))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 65d95b600eaa2..f91686cb544c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -303,6 +303,10 @@ class ResolveSessionCatalog( val v1TableName = parseV1Table(tableName, "CACHE TABLE") CacheTableCommand(v1TableName.asTableIdentifier, plan, isLazy, options) + case UncacheTableStatement(tableName, ifExists) => + val v1TableName = parseV1Table(tableName, "UNCACHE TABLE") + UncacheTableCommand(v1TableName.asTableIdentifier, ifExists) + case TruncateTableStatement(tableName, partitionSpec) => val v1TableName = parseV1Table(tableName, "TRUNCATE TABLE") TruncateTableCommand( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index fb13d01bd91dc..aef0a2d2e5951 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -165,13 +165,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { unquotedPath } - /** - * Create an [[UncacheTableCommand]] logical plan. - */ - override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { - UncacheTableCommand(visitTableIdentifier(ctx.tableIdentifier), ctx.EXISTS != null) - } - /** * Create a [[ClearCacheCommand]] logical plan. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 4d1e70f68ba06..4f2c1af8f7b56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1282,6 +1282,16 @@ class DataSourceV2SQLSuite } } + test("UNCACHE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + testV1Command("UNCACHE TABLE", t) + testV1Command("UNCACHE TABLE", s"IF EXISTS $t") + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") From 40df9d246e4049d7f858d4ff98801935fa9ee861 Mon Sep 17 00:00:00 2001 From: wenxuanguan Date: Fri, 25 Oct 2019 10:02:54 +0900 Subject: [PATCH 0423/1387] [SPARK-29227][SS] Track rule info in optimization phase ### What changes were proposed in this pull request? Track timing info for each rule in optimization phase using `QueryPlanningTracker` in Structured Streaming ### Why are the changes needed? In Structured Streaming we only track rule info in analysis phase, not in optimization phase. ### Does this PR introduce any user-facing change? No Closes #25914 from wenxuanguan/spark-29227. Authored-by: wenxuanguan Signed-off-by: HyukjinKwon --- .../streaming/IncrementalExecution.scala | 3 ++- .../QueryPlanningTrackerEndToEndSuite.scala | 24 +++++++++++++++++-- 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index af52af0d1d7e6..b8e18b89b54bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -77,7 +77,8 @@ class IncrementalExecution( */ override lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { - sparkSession.sessionState.optimizer.execute(withCachedData) transformAllExpressions { + sparkSession.sessionState.optimizer.executeAndTrack(withCachedData, + tracker) transformAllExpressions { case ts @ CurrentBatchTimestamp(timestamp, _, _) => logInfo(s"Current batch timestamp = $timestamp") ts.toLiteral diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala index 76006efda992f..987338cf6cbbf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution} +import org.apache.spark.sql.streaming.StreamTest -class QueryPlanningTrackerEndToEndSuite extends SharedSparkSession { +class QueryPlanningTrackerEndToEndSuite extends StreamTest { + import testImplicits._ test("programmatic API") { val df = spark.range(1000).selectExpr("count(*)") @@ -38,4 +40,22 @@ class QueryPlanningTrackerEndToEndSuite extends SharedSparkSession { assert(tracker.rules.nonEmpty) } + test("SPARK-29227: Track rule info in optimization phase in streaming") { + val inputData = MemoryStream[Int] + val df = inputData.toDF() + + def assertStatus(stream: StreamExecution): Unit = { + stream.processAllAvailable() + val tracker = stream.lastExecution.tracker + assert(tracker.phases.keys == Set("analysis", "optimization", "planning")) + assert(tracker.rules.nonEmpty) + } + + testStream(df)( + StartStream(), + AddData(inputData, 1, 2, 3), + Execute(assertStatus), + StopStream) + } + } From 7417c3e7d5a890b93420e6b4c507e6805e633cca Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 24 Oct 2019 20:51:31 -0700 Subject: [PATCH 0424/1387] [SPARK-29597][DOCS] Deprecate old Java 8 versions prior to 8u92 ### What changes were proposed in this pull request? This PR aims to deprecate old Java 8 versions prior to 8u92. ### Why are the changes needed? This is a preparation to use JVM Option `ExitOnOutOfMemoryError`. - https://www.oracle.com/technetwork/java/javase/8u92-relnotes-2949471.html ### Does this PR introduce any user-facing change? Yes. It's highly recommended for users to use the latest JDK versions of Java 8/11. ### How was this patch tested? NA (This is a doc change). Closes #26249 from dongjoon-hyun/SPARK-29597. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/index.md b/docs/index.md index edb1c421fb794..9e8af0d5f8e2b 100644 --- a/docs/index.md +++ b/docs/index.md @@ -47,6 +47,7 @@ locally on one machine --- all you need is to have `java` installed on your syst or the `JAVA_HOME` environment variable pointing to a Java installation. Spark runs on Java 8/11, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. +Java 8 prior to version 8u92 support is deprecated as of Spark 3.0.0. Python 2 support is deprecated as of Spark 3.0.0. R prior to version 3.4 support is deprecated as of Spark 3.0.0. For the Scala API, Spark {{site.SPARK_VERSION}} From 1474ed05fb2d3e9324f17e4bf4f5702037d0be62 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 24 Oct 2019 22:18:10 -0700 Subject: [PATCH 0425/1387] [SPARK-29562][SQL] Speed up and slim down metric aggregation in SQL listener First, a bit of background on the code being changed. The current code tracks metric updates for each task, recording which metrics the task is monitoring and the last update value. Once a SQL execution finishes, then the metrics for all the stages are aggregated, by building a list with all (metric ID, value) pairs collected for all tasks in the stages related to the execution, then grouping by metric ID, and then calculating the values shown in the UI. That is full of inefficiencies: - in normal operation, all tasks will be tracking and updating the same metrics. So recording the metric IDs per task is wasteful. - tracking by task means we might be double-counting values if you have speculative tasks (as a comment in the code mentions). - creating a list of (metric ID, value) is extremely inefficient, because now you have a huge map in memory storing boxed versions of the metric IDs and values. - same thing for the aggregation part, where now a Seq is built with the values for each metric ID. The end result is that for large queries, this code can become both really slow, thus affecting the processing of events, and memory hungry. The updated code changes the approach to the following: - stages track metrics by their ID; this means the stage tracking code naturally groups values, making aggregation later simpler. - each metric ID being tracked uses a long array matching the number of partitions of the stage; this means that it's cheap to update the value of the metric once a task ends. - when aggregating, custom code just concatenates the arrays corresponding to the matching metric IDs; this is cheaper than the previous, boxing-heavy approach. The end result is that the listener uses about half as much memory as before for tracking metrics, since it doesn't need to track metric IDs per task. I captured heap dumps with the old and the new code during metric aggregation in the listener, for an execution with 3 stages, 100k tasks per stage, 50 metrics updated per task. The dumps contained just reachable memory - so data kept by the listener plus the variables in the aggregateMetrics() method. With the old code, the thread doing aggregation references >1G of memory - and that does not include temporary data created by the "groupBy" transformation (for which the intermediate state is not referenced in the aggregation method). The same thread with the new code references ~250M of memory. The old code uses about ~250M to track all the metric values for that execution, while the new code uses about ~130M. (Note the per-thread numbers include the amount used to track the metrics - so, e.g., in the old case, aggregation was referencing about ~750M of temporary data.) I'm also including a small benchmark (based on the Benchmark class) so that we can measure how much changes to this code affect performance. The benchmark contains some extra code to measure things the normal Benchmark class does not, given that the code under test does not really map that well to the expectations of that class. Running with the old code (I removed results that don't make much sense for this benchmark): ``` [info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Linux 4.15.0-66-generic [info] Intel(R) Core(TM) i7-6820HQ CPU 2.70GHz [info] metrics aggregation (50 metrics, 100k tasks per stage): Best Time(ms) Avg Time(ms) [info] -------------------------------------------------------------------------------------- [info] 1 stage(s) 2113 2118 [info] 2 stage(s) 4172 4392 [info] 3 stage(s) 7755 8460 [info] [info] Stage Count Stage Proc. Time Aggreg. Time [info] 1 614 1187 [info] 2 620 2480 [info] 3 718 5069 ``` With the new code: ``` [info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Linux 4.15.0-66-generic [info] Intel(R) Core(TM) i7-6820HQ CPU 2.70GHz [info] metrics aggregation (50 metrics, 100k tasks per stage): Best Time(ms) Avg Time(ms) [info] -------------------------------------------------------------------------------------- [info] 1 stage(s) 727 886 [info] 2 stage(s) 1722 1983 [info] 3 stage(s) 2752 3013 [info] [info] Stage Count Stage Proc. Time Aggreg. Time [info] 1 408 177 [info] 2 389 423 [info] 3 372 660 ``` So the new code is faster than the old when processing task events, and about an order of maginute faster when aggregating metrics. Note this still leaves room for improvement; for example, using the above measurements, 600ms is still a huge amount of time to spend in an event handler. But I'll leave further enhancements for a separate change. Tested with benchmarking code + existing unit tests. Closes #26218 from vanzin/SPARK-29562. Authored-by: Marcelo Vanzin Signed-off-by: Dongjoon Hyun --- ...ricsAggregationBenchmark-jdk11-results.txt | 12 + .../MetricsAggregationBenchmark-results.txt | 12 + .../sql/execution/metric/SQLMetrics.scala | 14 +- .../execution/ui/SQLAppStatusListener.scala | 190 ++++++++++----- .../metric/SQLMetricsTestUtils.scala | 3 +- .../ui/MetricsAggregationBenchmark.scala | 219 ++++++++++++++++++ .../ui/SQLAppStatusListenerSuite.scala | 19 +- 7 files changed, 397 insertions(+), 72 deletions(-) create mode 100644 sql/core/benchmarks/MetricsAggregationBenchmark-jdk11-results.txt create mode 100644 sql/core/benchmarks/MetricsAggregationBenchmark-results.txt create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala diff --git a/sql/core/benchmarks/MetricsAggregationBenchmark-jdk11-results.txt b/sql/core/benchmarks/MetricsAggregationBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..e33ed30eaa559 --- /dev/null +++ b/sql/core/benchmarks/MetricsAggregationBenchmark-jdk11-results.txt @@ -0,0 +1,12 @@ +OpenJDK 64-Bit Server VM 11.0.4+11 on Linux 4.15.0-66-generic +Intel(R) Core(TM) i7-6820HQ CPU @ 2.70GHz +metrics aggregation (50 metrics, 100000 tasks per stage): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 stage(s) 672 841 179 0.0 671888474.0 1.0X +2 stage(s) 1700 1842 201 0.0 1699591662.0 0.4X +3 stage(s) 2601 2776 247 0.0 2601465786.0 0.3X + +Stage Count Stage Proc. Time Aggreg. Time + 1 436 164 + 2 537 354 + 3 480 602 diff --git a/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt b/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt new file mode 100644 index 0000000000000..4fae928258d32 --- /dev/null +++ b/sql/core/benchmarks/MetricsAggregationBenchmark-results.txt @@ -0,0 +1,12 @@ +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Linux 4.15.0-66-generic +Intel(R) Core(TM) i7-6820HQ CPU @ 2.70GHz +metrics aggregation (50 metrics, 100000 tasks per stage): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 stage(s) 740 883 147 0.0 740089816.0 1.0X +2 stage(s) 1661 1943 399 0.0 1660649192.0 0.4X +3 stage(s) 2711 2967 362 0.0 2711110178.0 0.3X + +Stage Count Stage Proc. Time Aggreg. Time + 1 405 179 + 2 375 414 + 3 364 644 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 19809b07508d9..b7f0ab2969e45 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.metric import java.text.NumberFormat -import java.util.Locale +import java.util.{Arrays, Locale} import scala.concurrent.duration._ @@ -150,7 +150,7 @@ object SQLMetrics { * A function that defines how we aggregate the final accumulator results among all tasks, * and represent it in string for a SQL physical operator. */ - def stringValue(metricsType: String, values: Seq[Long]): String = { + def stringValue(metricsType: String, values: Array[Long]): String = { if (metricsType == SUM_METRIC) { val numberFormat = NumberFormat.getIntegerInstance(Locale.US) numberFormat.format(values.sum) @@ -162,8 +162,9 @@ object SQLMetrics { val metric = if (validValues.isEmpty) { Seq.fill(3)(0L) } else { - val sorted = validValues.sorted - Seq(sorted(0), sorted(validValues.length / 2), sorted(validValues.length - 1)) + Arrays.sort(validValues) + Seq(validValues(0), validValues(validValues.length / 2), + validValues(validValues.length - 1)) } metric.map(v => numberFormat.format(v.toDouble / baseForAvgMetric)) } @@ -184,8 +185,9 @@ object SQLMetrics { val metric = if (validValues.isEmpty) { Seq.fill(4)(0L) } else { - val sorted = validValues.sorted - Seq(sorted.sum, sorted(0), sorted(validValues.length / 2), sorted(validValues.length - 1)) + Arrays.sort(validValues) + Seq(validValues.sum, validValues(0), validValues(validValues.length / 2), + validValues(validValues.length - 1)) } metric.map(strFormat) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index 2c4a7eacdf10b..da526612e7bcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -16,10 +16,11 @@ */ package org.apache.spark.sql.execution.ui -import java.util.{Date, NoSuchElementException} +import java.util.{Arrays, Date, NoSuchElementException} import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark.{JobExecutionStatus, SparkConf} import org.apache.spark.internal.Logging @@ -29,6 +30,7 @@ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.metric._ import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} +import org.apache.spark.util.collection.OpenHashMap class SQLAppStatusListener( conf: SparkConf, @@ -103,8 +105,10 @@ class SQLAppStatusListener( // Record the accumulator IDs for the stages of this job, so that the code that keeps // track of the metrics knows which accumulators to look at. val accumIds = exec.metrics.map(_.accumulatorId).toSet - event.stageIds.foreach { id => - stageMetrics.put(id, new LiveStageMetrics(id, 0, accumIds, new ConcurrentHashMap())) + if (accumIds.nonEmpty) { + event.stageInfos.foreach { stage => + stageMetrics.put(stage.stageId, new LiveStageMetrics(0, stage.numTasks, accumIds)) + } } exec.jobs = exec.jobs + (jobId -> JobExecutionStatus.RUNNING) @@ -118,9 +122,11 @@ class SQLAppStatusListener( } // Reset the metrics tracking object for the new attempt. - Option(stageMetrics.get(event.stageInfo.stageId)).foreach { metrics => - metrics.taskMetrics.clear() - metrics.attemptId = event.stageInfo.attemptNumber + Option(stageMetrics.get(event.stageInfo.stageId)).foreach { stage => + if (stage.attemptId != event.stageInfo.attemptNumber) { + stageMetrics.put(event.stageInfo.stageId, + new LiveStageMetrics(event.stageInfo.attemptNumber, stage.numTasks, stage.accumulatorIds)) + } } } @@ -140,7 +146,16 @@ class SQLAppStatusListener( override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { event.accumUpdates.foreach { case (taskId, stageId, attemptId, accumUpdates) => - updateStageMetrics(stageId, attemptId, taskId, accumUpdates, false) + updateStageMetrics(stageId, attemptId, taskId, SQLAppStatusListener.UNKNOWN_INDEX, + accumUpdates, false) + } + } + + override def onTaskStart(event: SparkListenerTaskStart): Unit = { + Option(stageMetrics.get(event.stageId)).foreach { stage => + if (stage.attemptId == event.stageAttemptId) { + stage.registerTask(event.taskInfo.taskId, event.taskInfo.index) + } } } @@ -165,7 +180,7 @@ class SQLAppStatusListener( } else { info.accumulables } - updateStageMetrics(event.stageId, event.stageAttemptId, info.taskId, accums, + updateStageMetrics(event.stageId, event.stageAttemptId, info.taskId, info.index, accums, info.successful) } @@ -181,17 +196,40 @@ class SQLAppStatusListener( private def aggregateMetrics(exec: LiveExecutionData): Map[Long, String] = { val metricTypes = exec.metrics.map { m => (m.accumulatorId, m.metricType) }.toMap - val metrics = exec.stages.toSeq + + val taskMetrics = exec.stages.toSeq .flatMap { stageId => Option(stageMetrics.get(stageId)) } - .flatMap(_.taskMetrics.values().asScala) - .flatMap { metrics => metrics.ids.zip(metrics.values) } - - val aggregatedMetrics = (metrics ++ exec.driverAccumUpdates.toSeq) - .filter { case (id, _) => metricTypes.contains(id) } - .groupBy(_._1) - .map { case (id, values) => - id -> SQLMetrics.stringValue(metricTypes(id), values.map(_._2)) + .flatMap(_.metricValues()) + + val allMetrics = new mutable.HashMap[Long, Array[Long]]() + + taskMetrics.foreach { case (id, values) => + val prev = allMetrics.getOrElse(id, null) + val updated = if (prev != null) { + prev ++ values + } else { + values } + allMetrics(id) = updated + } + + exec.driverAccumUpdates.foreach { case (id, value) => + if (metricTypes.contains(id)) { + val prev = allMetrics.getOrElse(id, null) + val updated = if (prev != null) { + val _copy = Arrays.copyOf(prev, prev.length + 1) + _copy(prev.length) = value + _copy + } else { + Array(value) + } + allMetrics(id) = updated + } + } + + val aggregatedMetrics = allMetrics.map { case (id, values) => + id -> SQLMetrics.stringValue(metricTypes(id), values) + }.toMap // Check the execution again for whether the aggregated metrics data has been calculated. // This can happen if the UI is requesting this data, and the onExecutionEnd handler is @@ -208,43 +246,13 @@ class SQLAppStatusListener( stageId: Int, attemptId: Int, taskId: Long, + taskIdx: Int, accumUpdates: Seq[AccumulableInfo], succeeded: Boolean): Unit = { Option(stageMetrics.get(stageId)).foreach { metrics => - if (metrics.attemptId != attemptId || metrics.accumulatorIds.isEmpty) { - return - } - - val oldTaskMetrics = metrics.taskMetrics.get(taskId) - if (oldTaskMetrics != null && oldTaskMetrics.succeeded) { - return + if (metrics.attemptId == attemptId) { + metrics.updateTaskMetrics(taskId, taskIdx, succeeded, accumUpdates) } - - val updates = accumUpdates - .filter { acc => acc.update.isDefined && metrics.accumulatorIds.contains(acc.id) } - .sortBy(_.id) - - if (updates.isEmpty) { - return - } - - val ids = new Array[Long](updates.size) - val values = new Array[Long](updates.size) - updates.zipWithIndex.foreach { case (acc, idx) => - ids(idx) = acc.id - // In a live application, accumulators have Long values, but when reading from event - // logs, they have String values. For now, assume all accumulators are Long and covert - // accordingly. - values(idx) = acc.update.get match { - case s: String => s.toLong - case l: Long => l - case o => throw new IllegalArgumentException(s"Unexpected: $o") - } - } - - // TODO: storing metrics by task ID can cause metrics for the same task index to be - // counted multiple times, for example due to speculation or re-attempts. - metrics.taskMetrics.put(taskId, new LiveTaskMetrics(ids, values, succeeded)) } } @@ -425,12 +433,76 @@ private class LiveExecutionData(val executionId: Long) extends LiveEntity { } private class LiveStageMetrics( - val stageId: Int, - var attemptId: Int, - val accumulatorIds: Set[Long], - val taskMetrics: ConcurrentHashMap[Long, LiveTaskMetrics]) - -private class LiveTaskMetrics( - val ids: Array[Long], - val values: Array[Long], - val succeeded: Boolean) + val attemptId: Int, + val numTasks: Int, + val accumulatorIds: Set[Long]) { + + /** + * Mapping of task IDs to their respective index. Note this may contain more elements than the + * stage's number of tasks, if speculative execution is on. + */ + private val taskIndices = new OpenHashMap[Long, Int]() + + /** Bit set tracking which indices have been successfully computed. */ + private val completedIndices = new mutable.BitSet() + + /** + * Task metrics values for the stage. Maps the metric ID to the metric values for each + * index. For each metric ID, there will be the same number of values as the number + * of indices. This relies on `SQLMetrics.stringValue` treating 0 as a neutral value, + * independent of the actual metric type. + */ + private val taskMetrics = new ConcurrentHashMap[Long, Array[Long]]() + + def registerTask(taskId: Long, taskIdx: Int): Unit = { + taskIndices.update(taskId, taskIdx) + } + + def updateTaskMetrics( + taskId: Long, + eventIdx: Int, + finished: Boolean, + accumUpdates: Seq[AccumulableInfo]): Unit = { + val taskIdx = if (eventIdx == SQLAppStatusListener.UNKNOWN_INDEX) { + if (!taskIndices.contains(taskId)) { + // We probably missed the start event for the task, just ignore it. + return + } + taskIndices(taskId) + } else { + // Here we can recover from a missing task start event. Just register the task again. + registerTask(taskId, eventIdx) + eventIdx + } + + if (completedIndices.contains(taskIdx)) { + return + } + + accumUpdates + .filter { acc => acc.update.isDefined && accumulatorIds.contains(acc.id) } + .foreach { acc => + // In a live application, accumulators have Long values, but when reading from event + // logs, they have String values. For now, assume all accumulators are Long and convert + // accordingly. + val value = acc.update.get match { + case s: String => s.toLong + case l: Long => l + case o => throw new IllegalArgumentException(s"Unexpected: $o") + } + + val metricValues = taskMetrics.computeIfAbsent(acc.id, _ => new Array(numTasks)) + metricValues(taskIdx) = value + } + + if (finished) { + completedIndices += taskIdx + } + } + + def metricValues(): Seq[(Long, Array[Long])] = taskMetrics.asScala.toSeq +} + +private object SQLAppStatusListener { + val UNKNOWN_INDEX = -1 +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index 5ab9b6f5fc2d4..57731e5f4920c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -232,7 +232,8 @@ trait SQLMetricsTestUtils extends SQLTestUtils { val (actualNodeName, actualMetricsMap) = actualMetrics(nodeId) assert(expectedNodeName === actualNodeName) for ((metricName, metricPredicate) <- expectedMetricsPredicatesMap) { - assert(metricPredicate(actualMetricsMap(metricName))) + assert(metricPredicate(actualMetricsMap(metricName)), + s"$nodeId / '$metricName' (= ${actualMetricsMap(metricName)}) did not match predicate.") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala new file mode 100644 index 0000000000000..a88abc8209a88 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.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.sql.execution.ui + +import java.util.Properties +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable +import scala.concurrent.duration._ + +import org.apache.spark.{SparkConf, TaskState} +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.internal.config.Status._ +import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.metric.SQLMetricInfo +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.{AccumulatorMetadata, LongAccumulator, Utils} +import org.apache.spark.util.kvstore.InMemoryStore + +/** + * Benchmark for metrics aggregation in the SQL listener. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class --jars + * 2. build/sbt "core/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " + * Results will be written to "benchmarks/MetricsAggregationBenchmark-results.txt". + * }}} + */ +object MetricsAggregationBenchmark extends BenchmarkBase { + + private def metricTrackingBenchmark( + timer: Benchmark.Timer, + numMetrics: Int, + numTasks: Int, + numStages: Int): Measurements = { + val conf = new SparkConf() + .set(LIVE_ENTITY_UPDATE_PERIOD, 0L) + .set(ASYNC_TRACKING_ENABLED, false) + val kvstore = new ElementTrackingStore(new InMemoryStore(), conf) + val listener = new SQLAppStatusListener(conf, kvstore, live = true) + val store = new SQLAppStatusStore(kvstore, Some(listener)) + + val metrics = (0 until numMetrics).map { i => + new SQLMetricInfo(s"metric$i", i.toLong, "average") + } + + val planInfo = new SparkPlanInfo( + getClass().getName(), + getClass().getName(), + Nil, + Map.empty, + metrics) + + val idgen = new AtomicInteger() + val executionId = idgen.incrementAndGet() + val executionStart = SparkListenerSQLExecutionStart( + executionId, + getClass().getName(), + getClass().getName(), + getClass().getName(), + planInfo, + System.currentTimeMillis()) + + val executionEnd = SparkListenerSQLExecutionEnd(executionId, System.currentTimeMillis()) + + val properties = new Properties() + properties.setProperty(SQLExecution.EXECUTION_ID_KEY, executionId.toString) + + timer.startTiming() + listener.onOtherEvent(executionStart) + + val taskEventsTime = (0 until numStages).map { _ => + val stageInfo = new StageInfo(idgen.incrementAndGet(), 0, getClass().getName(), + numTasks, Nil, Nil, getClass().getName()) + + val jobId = idgen.incrementAndGet() + val jobStart = SparkListenerJobStart( + jobId = jobId, + time = System.currentTimeMillis(), + stageInfos = Seq(stageInfo), + properties) + + val stageStart = SparkListenerStageSubmitted(stageInfo) + + val taskOffset = idgen.incrementAndGet().toLong + val taskEvents = (0 until numTasks).map { i => + val info = new TaskInfo( + taskId = taskOffset + i.toLong, + index = i, + attemptNumber = 0, + // The following fields are not used. + launchTime = 0, + executorId = "", + host = "", + taskLocality = null, + speculative = false) + info.markFinished(TaskState.FINISHED, 1L) + + val accumulables = (0 until numMetrics).map { mid => + val acc = new LongAccumulator + acc.metadata = AccumulatorMetadata(mid, None, false) + acc.toInfo(Some(i.toLong), None) + } + + info.setAccumulables(accumulables) + + val start = SparkListenerTaskStart(stageInfo.stageId, stageInfo.attemptNumber, info) + val end = SparkListenerTaskEnd(stageInfo.stageId, stageInfo.attemptNumber, + taskType = "", + reason = null, + info, + new ExecutorMetrics(), + null) + + (start, end) + } + + val jobEnd = SparkListenerJobEnd( + jobId = jobId, + time = System.currentTimeMillis(), + JobSucceeded) + + listener.onJobStart(jobStart) + listener.onStageSubmitted(stageStart) + + val (_, _taskEventsTime) = Utils.timeTakenMs { + taskEvents.foreach { case (start, end) => + listener.onTaskStart(start) + listener.onTaskEnd(end) + } + } + + listener.onJobEnd(jobEnd) + _taskEventsTime + } + + val (_, aggTime) = Utils.timeTakenMs { + listener.onOtherEvent(executionEnd) + val metrics = store.executionMetrics(executionId) + assert(metrics.size == numMetrics, s"${metrics.size} != $numMetrics") + } + + timer.stopTiming() + kvstore.close() + + Measurements(taskEventsTime, aggTime) + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val metricCount = 50 + val taskCount = 100000 + val stageCounts = Seq(1, 2, 3) + + val benchmark = new Benchmark( + s"metrics aggregation ($metricCount metrics, $taskCount tasks per stage)", 1, + warmupTime = 0.seconds, output = output) + + // Run this outside the measurement code so that classes are loaded and JIT is triggered, + // otherwise the first run tends to be much slower than others. Also because this benchmark is a + // bit weird and doesn't really map to what the Benchmark class expects, so it's a bit harder + // to use warmupTime and friends effectively. + stageCounts.foreach { count => + metricTrackingBenchmark(new Benchmark.Timer(-1), metricCount, taskCount, count) + } + + val measurements = mutable.HashMap[Int, Seq[Measurements]]() + + stageCounts.foreach { count => + benchmark.addTimerCase(s"$count stage(s)") { timer => + val m = metricTrackingBenchmark(timer, metricCount, taskCount, count) + val all = measurements.getOrElse(count, Nil) + measurements(count) = all ++ Seq(m) + } + } + + benchmark.run() + + benchmark.out.printf("Stage Count Stage Proc. Time Aggreg. Time\n") + stageCounts.foreach { count => + val data = measurements(count) + val eventsTimes = data.flatMap(_.taskEventsTimes) + val aggTimes = data.map(_.aggregationTime) + + val msg = " %d %d %d\n".format( + count, + eventsTimes.sum / eventsTimes.size, + aggTimes.sum / aggTimes.size) + benchmark.out.printf(msg) + } + } + + /** + * Finer-grained measurements of how long it takes to run some parts of the benchmark. This is + * collected by the benchmark method, so this collection slightly affects the overall benchmark + * results, but this data helps with seeing where the time is going, since this benchmark is + * triggering a whole lot of code in the listener class. + */ + case class Measurements( + taskEventsTimes: Seq[Long], + aggregationTime: Long) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 88864ccec7523..b8c0935b33a87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -79,9 +79,9 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils private def createStageInfo(stageId: Int, attemptId: Int): StageInfo = { new StageInfo(stageId = stageId, attemptId = attemptId, + numTasks = 8, // The following fields are not used in tests name = "", - numTasks = 0, rddInfos = Nil, parentIds = Nil, details = "") @@ -94,8 +94,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils val info = new TaskInfo( taskId = taskId, attemptNumber = attemptNumber, + index = taskId.toInt, // The following fields are not used in tests - index = 0, launchTime = 0, executorId = "", host = "", @@ -190,6 +190,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils ), createProperties(executionId))) listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 0))) + listener.onTaskStart(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0))) + listener.onTaskStart(SparkListenerTaskStart(0, 0, createTaskInfo(1, 0))) assert(statusStore.executionMetrics(executionId).isEmpty) @@ -217,6 +219,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils // Retrying a stage should reset the metrics listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 1))) + listener.onTaskStart(SparkListenerTaskStart(0, 1, createTaskInfo(0, 0))) + listener.onTaskStart(SparkListenerTaskStart(0, 1, createTaskInfo(1, 0))) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, accum updates) @@ -260,6 +264,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils // Summit a new stage listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(1, 0))) + listener.onTaskStart(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0))) + listener.onTaskStart(SparkListenerTaskStart(1, 0, createTaskInfo(1, 0))) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( // (task id, stage id, stage attempt, accum updates) @@ -490,8 +496,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils val statusStore = spark.sharedState.statusStore val oldCount = statusStore.executionsList().size - val expectedAccumValue = 12345 - val expectedAccumValue2 = 54321 + val expectedAccumValue = 12345L + val expectedAccumValue2 = 54321L val physicalPlan = MyPlan(sqlContext.sparkContext, expectedAccumValue, expectedAccumValue2) val dummyQueryExecution = new QueryExecution(spark, LocalRelation()) { override lazy val sparkPlan = physicalPlan @@ -517,8 +523,9 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils val metrics = statusStore.executionMetrics(execId) val driverMetric = physicalPlan.metrics("dummy") val driverMetric2 = physicalPlan.metrics("dummy2") - val expectedValue = SQLMetrics.stringValue(driverMetric.metricType, Seq(expectedAccumValue)) - val expectedValue2 = SQLMetrics.stringValue(driverMetric2.metricType, Seq(expectedAccumValue2)) + val expectedValue = SQLMetrics.stringValue(driverMetric.metricType, Array(expectedAccumValue)) + val expectedValue2 = SQLMetrics.stringValue(driverMetric2.metricType, + Array(expectedAccumValue2)) assert(metrics.contains(driverMetric.id)) assert(metrics(driverMetric.id) === expectedValue) From 091cbc3be0ab8678e9a6d21eb29a14dd554c9b39 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 25 Oct 2019 13:48:09 +0800 Subject: [PATCH 0426/1387] [SPARK-9612][ML] Add instance weight support for GBTs ### What changes were proposed in this pull request? add weight support for GBTs by sampling data before passing it to trees and then passing weights to trees in summary: 1, add setters of `minWeightFractionPerNode` & `weightCol` 2, update input types in private methods from `RDD[LabeledPoint]` to `RDD[Instance]`: `DecisionTreeRegressor.train`, `GradientBoostedTrees.run`, `GradientBoostedTrees.runWithValidation`, `GradientBoostedTrees.computeInitialPredictionAndError`, `GradientBoostedTrees.computeError`, `GradientBoostedTrees.evaluateEachIteration`, `GradientBoostedTrees.boost`, `GradientBoostedTrees.updatePredictionError` 3, add new private method `GradientBoostedTrees.computeError(data, predError)` to compute average error, since original `predError.values.mean()` do not take weights into account. 4, add new tests ### Why are the changes needed? GBTs should support sample weights like other algs ### Does this PR introduce any user-facing change? yes, new setters are added ### How was this patch tested? existing & added testsuites Closes #25926 from zhengruifeng/gbt_add_weight. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../spark/ml/classification/Classifier.scala | 2 +- .../ml/classification/GBTClassifier.scala | 68 ++++----- .../apache/spark/ml/feature/Instance.scala | 2 +- .../ml/regression/DecisionTreeRegressor.scala | 7 +- .../spark/ml/regression/GBTRegressor.scala | 48 +++--- .../ml/tree/impl/GradientBoostedTrees.scala | 144 ++++++++++-------- .../mllib/tree/GradientBoostedTrees.scala | 17 ++- .../classification/GBTClassifierSuite.scala | 61 ++++++-- .../LogisticRegressionSuite.scala | 2 - .../ml/regression/GBTRegressorSuite.scala | 51 ++++++- .../tree/impl/GradientBoostedTreesSuite.scala | 25 ++- 11 files changed, 261 insertions(+), 166 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 9ac673078d4ad..3bff236677e6b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -53,7 +53,7 @@ private[spark] trait ClassifierParams val validateInstance = (instance: Instance) => { val label = instance.label require(label.toLong == label && label >= 0 && label < numClasses, s"Classifier was given" + - s" dataset with invalid label $label. Labels must be integers in range" + + s" dataset with invalid label $label. Labels must be integers in range" + s" [0, $numClasses).") } extractInstances(dataset, validateInstance) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 74624be360c6a..5bc45f2b02a4b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -23,7 +23,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.regression.DecisionTreeRegressionModel @@ -34,7 +34,7 @@ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ /** @@ -79,6 +79,10 @@ class GBTClassifier @Since("1.4.0") ( @Since("1.4.0") def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + /** @group setParam */ + @Since("3.0.0") + def setMinWeightFractionPerNode(value: Double): this.type = set(minWeightFractionPerNode, value) + /** @group setParam */ @Since("1.4.0") def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) @@ -152,36 +156,34 @@ class GBTClassifier @Since("1.4.0") ( set(validationIndicatorCol, value) } + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * By default the weightCol is not set, so all instances have weight 1.0. + * + * @group setParam + */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + override protected def train( dataset: Dataset[_]): GBTClassificationModel = instrumented { instr => - val categoricalFeatures: Map[Int, Int] = - MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val withValidation = isDefined(validationIndicatorCol) && $(validationIndicatorCol).nonEmpty - // We copy and modify this from Classifier.extractLabeledPoints since GBT only supports - // 2 classes now. This lets us provide a more precise error message. - val convert2LabeledPoint = (dataset: Dataset[_]) => { - dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => - require(label == 0 || label == 1, s"GBTClassifier was given" + - s" dataset with invalid label $label. Labels must be in {0,1}; note that" + - s" GBTClassifier currently only supports binary classification.") - LabeledPoint(label, features) - } + val validateInstance = (instance: Instance) => { + val label = instance.label + require(label == 0 || label == 1, s"GBTClassifier was given" + + s" dataset with invalid label $label. Labels must be in {0,1}; note that" + + s" GBTClassifier currently only supports binary classification.") } val (trainDataset, validationDataset) = if (withValidation) { - ( - convert2LabeledPoint(dataset.filter(not(col($(validationIndicatorCol))))), - convert2LabeledPoint(dataset.filter(col($(validationIndicatorCol)))) - ) + (extractInstances(dataset.filter(not(col($(validationIndicatorCol)))), validateInstance), + extractInstances(dataset.filter(col($(validationIndicatorCol))), validateInstance)) } else { - (convert2LabeledPoint(dataset), null) + (extractInstances(dataset, validateInstance), null) } - val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) - val numClasses = 2 if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + @@ -191,12 +193,14 @@ class GBTClassifier @Since("1.4.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, predictionCol, leafCol, impurity, - lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, - seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, featureSubsetStrategy, - validationIndicatorCol, validationTol) + instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, leafCol, + impurity, lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, + minInstancesPerNode, minWeightFractionPerNode, seed, stepSize, subsamplingRate, cacheNodeIds, + checkpointInterval, featureSubsetStrategy, validationIndicatorCol, validationTol) instr.logNumClasses(numClasses) + val categoricalFeatures = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) val (baseLearners, learnerWeights) = if (withValidation) { GradientBoostedTrees.runWithValidation(trainDataset, validationDataset, boostingStrategy, $(seed), $(featureSubsetStrategy)) @@ -374,12 +378,9 @@ class GBTClassificationModel private[ml]( */ @Since("2.4.0") def evaluateEachIteration(dataset: Dataset[_]): Array[Double] = { - val data = dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => LabeledPoint(label, features) - } + val data = extractInstances(dataset) GradientBoostedTrees.evaluateEachIteration(data, trees, treeWeights, loss, - OldAlgo.Classification - ) + OldAlgo.Classification) } @Since("2.0.0") @@ -423,10 +424,9 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { val numFeatures = (metadata.metadata \ numFeaturesKey).extract[Int] val numTrees = (metadata.metadata \ numTreesKey).extract[Int] - val trees: Array[DecisionTreeRegressionModel] = treesData.map { + val trees = treesData.map { case (treeMetadata, root) => - val tree = - new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) + val tree = new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) treeMetadata.getAndSetParams(tree) tree } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index dd56fbbfa2b63..11d0c4689cbba 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -26,7 +26,7 @@ import org.apache.spark.ml.linalg.Vector * @param weight The weight of this instance. * @param features The vector of features for this data point. */ -private[ml] case class Instance(label: Double, weight: Double, features: Vector) +private[spark] case class Instance(label: Double, weight: Double, features: Vector) /** * Case class that represents an instance of data point with diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 602b5fac20d3b..05851d5116751 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -23,7 +23,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor} -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ @@ -132,15 +132,14 @@ class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S /** (private[ml]) Train a decision tree on an RDD */ private[ml] def train( - data: RDD[LabeledPoint], + data: RDD[Instance], oldStrategy: OldStrategy, featureSubsetStrategy: String): DecisionTreeRegressionModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(data) instr.logParams(this, params: _*) - val instances = data.map(_.toInstance) - val trees = RandomForest.run(instances, oldStrategy, numTrees = 1, + val trees = RandomForest.run(data, oldStrategy, numTrees = 1, featureSubsetStrategy, seed = $(seed), instr = Some(instr), parentUID = Some(uid)) trees.head.asInstanceOf[DecisionTreeRegressionModel] diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 0cc06d82bf3fd..9c38647642a61 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -24,7 +24,6 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} -import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ @@ -34,7 +33,7 @@ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} -import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset} import org.apache.spark.sql.functions._ /** @@ -78,6 +77,10 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("1.4.0") def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + /** @group setParam */ + @Since("3.0.0") + def setMinWeightFractionPerNode(value: Double): this.type = set(minWeightFractionPerNode, value) + /** @group setParam */ @Since("1.4.0") def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) @@ -151,29 +154,35 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) set(validationIndicatorCol, value) } - override protected def train(dataset: Dataset[_]): GBTRegressionModel = instrumented { instr => - val categoricalFeatures: Map[Int, Int] = - MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * By default the weightCol is not set, so all instances have weight 1.0. + * + * @group setParam + */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + override protected def train(dataset: Dataset[_]): GBTRegressionModel = instrumented { instr => val withValidation = isDefined(validationIndicatorCol) && $(validationIndicatorCol).nonEmpty val (trainDataset, validationDataset) = if (withValidation) { - ( - extractLabeledPoints(dataset.filter(not(col($(validationIndicatorCol))))), - extractLabeledPoints(dataset.filter(col($(validationIndicatorCol)))) - ) + (extractInstances(dataset.filter(not(col($(validationIndicatorCol))))), + extractInstances(dataset.filter(col($(validationIndicatorCol))))) } else { - (extractLabeledPoints(dataset), null) + (extractInstances(dataset), null) } - val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, predictionCol, leafCol, impurity, lossType, - maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, - seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, featureSubsetStrategy, - validationIndicatorCol, validationTol) + instr.logParams(this, labelCol, featuresCol, predictionCol, leafCol, weightCol, impurity, + lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, + minWeightFractionPerNode, seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, + featureSubsetStrategy, validationIndicatorCol, validationTol) + val categoricalFeatures = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) + val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) val (baseLearners, learnerWeights) = if (withValidation) { GradientBoostedTrees.runWithValidation(trainDataset, validationDataset, boostingStrategy, $(seed), $(featureSubsetStrategy)) @@ -323,9 +332,7 @@ class GBTRegressionModel private[ml]( */ @Since("2.4.0") def evaluateEachIteration(dataset: Dataset[_], loss: String): Array[Double] = { - val data = dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => LabeledPoint(label, features) - } + val data = extractInstances(dataset) GradientBoostedTrees.evaluateEachIteration(data, trees, treeWeights, convertToOldLossType(loss), OldAlgo.Regression) } @@ -368,10 +375,9 @@ object GBTRegressionModel extends MLReadable[GBTRegressionModel] { val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numTrees = (metadata.metadata \ "numTrees").extract[Int] - val trees: Array[DecisionTreeRegressionModel] = treesData.map { + val trees = treesData.map { case (treeMetadata, root) => - val tree = - new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) + val tree = new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) treeMetadata.getAndSetParams(tree) tree } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index c31334c92e1c9..744708258b0ac 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} @@ -34,13 +34,13 @@ private[spark] object GradientBoostedTrees extends Logging { /** * Method to train a gradient boosting model - * @param input Training dataset: RDD of `LabeledPoint`. + * @param input Training dataset: RDD of `Instance`. * @param seed Random seed. * @return tuple of ensemble models and weights: * (array of decision tree models, array of model weights) */ def run( - input: RDD[LabeledPoint], + input: RDD[Instance], boostingStrategy: OldBoostingStrategy, seed: Long, featureSubsetStrategy: String): (Array[DecisionTreeRegressionModel], Array[Double]) = { @@ -51,7 +51,7 @@ private[spark] object GradientBoostedTrees extends Logging { seed, featureSubsetStrategy) case OldAlgo.Classification => // Map labels to -1, +1 so binary classification can be treated as regression. - val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + val remappedInput = input.map(x => Instance((x.label * 2) - 1, x.weight, x.features)) GradientBoostedTrees.boost(remappedInput, remappedInput, boostingStrategy, validate = false, seed, featureSubsetStrategy) case _ => @@ -61,7 +61,7 @@ private[spark] object GradientBoostedTrees extends Logging { /** * Method to validate a gradient boosting model - * @param input Training dataset: RDD of `LabeledPoint`. + * @param input Training dataset: RDD of `Instance`. * @param validationInput Validation dataset. * This dataset should be different from the training dataset, * but it should follow the same distribution. @@ -72,8 +72,8 @@ private[spark] object GradientBoostedTrees extends Logging { * (array of decision tree models, array of model weights) */ def runWithValidation( - input: RDD[LabeledPoint], - validationInput: RDD[LabeledPoint], + input: RDD[Instance], + validationInput: RDD[Instance], boostingStrategy: OldBoostingStrategy, seed: Long, featureSubsetStrategy: String): (Array[DecisionTreeRegressionModel], Array[Double]) = { @@ -85,9 +85,9 @@ private[spark] object GradientBoostedTrees extends Logging { case OldAlgo.Classification => // Map labels to -1, +1 so binary classification can be treated as regression. val remappedInput = input.map( - x => new LabeledPoint((x.label * 2) - 1, x.features)) + x => Instance((x.label * 2) - 1, x.weight, x.features)) val remappedValidationInput = validationInput.map( - x => new LabeledPoint((x.label * 2) - 1, x.features)) + x => Instance((x.label * 2) - 1, x.weight, x.features)) GradientBoostedTrees.boost(remappedInput, remappedValidationInput, boostingStrategy, validate = true, seed, featureSubsetStrategy) case _ => @@ -106,13 +106,13 @@ private[spark] object GradientBoostedTrees extends Logging { * corresponding to every sample. */ def computeInitialPredictionAndError( - data: RDD[LabeledPoint], + data: RDD[Instance], initTreeWeight: Double, initTree: DecisionTreeRegressionModel, loss: OldLoss): RDD[(Double, Double)] = { - data.map { lp => - val pred = updatePrediction(lp.features, 0.0, initTree, initTreeWeight) - val error = loss.computeError(pred, lp.label) + data.map { case Instance(label, _, features) => + val pred = updatePrediction(features, 0.0, initTree, initTreeWeight) + val error = loss.computeError(pred, label) (pred, error) } } @@ -129,20 +129,17 @@ private[spark] object GradientBoostedTrees extends Logging { * corresponding to each sample. */ def updatePredictionError( - data: RDD[LabeledPoint], + data: RDD[Instance], predictionAndError: RDD[(Double, Double)], treeWeight: Double, tree: DecisionTreeRegressionModel, loss: OldLoss): RDD[(Double, Double)] = { - - val newPredError = data.zip(predictionAndError).mapPartitions { iter => - iter.map { case (lp, (pred, error)) => - val newPred = updatePrediction(lp.features, pred, tree, treeWeight) - val newError = loss.computeError(newPred, lp.label) + data.zip(predictionAndError).map { + case (Instance(label, _, features), (pred, _)) => + val newPred = updatePrediction(features, pred, tree, treeWeight) + val newError = loss.computeError(newPred, label) (newPred, newError) - } } - newPredError } /** @@ -166,29 +163,50 @@ private[spark] object GradientBoostedTrees extends Logging { * Method to calculate error of the base learner for the gradient boosting calculation. * Note: This method is not used by the gradient boosting algorithm but is useful for debugging * purposes. - * @param data Training dataset: RDD of `LabeledPoint`. + * @param data Training dataset: RDD of `Instance`. * @param trees Boosted Decision Tree models * @param treeWeights Learning rates at each boosting iteration. * @param loss evaluation metric. * @return Measure of model error on data */ - def computeError( - data: RDD[LabeledPoint], + def computeWeightedError( + data: RDD[Instance], trees: Array[DecisionTreeRegressionModel], treeWeights: Array[Double], loss: OldLoss): Double = { - data.map { lp => + val (errSum, weightSum) = data.map { case Instance(label, weight, features) => val predicted = trees.zip(treeWeights).foldLeft(0.0) { case (acc, (model, weight)) => - updatePrediction(lp.features, acc, model, weight) + updatePrediction(features, acc, model, weight) } - loss.computeError(predicted, lp.label) - }.mean() + (loss.computeError(predicted, label) * weight, weight) + }.treeReduce { case ((err1, weight1), (err2, weight2)) => + (err1 + err2, weight1 + weight2) + } + errSum / weightSum + } + + /** + * Method to calculate error of the base learner for the gradient boosting calculation. + * @param data Training dataset: RDD of `Instance`. + * @param predError Prediction and error. + * @return Measure of model error on data + */ + def computeWeightedError( + data: RDD[Instance], + predError: RDD[(Double, Double)]): Double = { + val (errSum, weightSum) = data.zip(predError).map { + case (Instance(_, weight, _), (_, err)) => + (err * weight, weight) + }.treeReduce { case ((err1, weight1), (err2, weight2)) => + (err1 + err2, weight1 + weight2) + } + errSum / weightSum } /** * Method to compute error or loss for every iteration of gradient boosting. * - * @param data RDD of `LabeledPoint` + * @param data RDD of `Instance` * @param trees Boosted Decision Tree models * @param treeWeights Learning rates at each boosting iteration. * @param loss evaluation metric. @@ -197,41 +215,34 @@ private[spark] object GradientBoostedTrees extends Logging { * containing the first i+1 trees */ def evaluateEachIteration( - data: RDD[LabeledPoint], + data: RDD[Instance], trees: Array[DecisionTreeRegressionModel], treeWeights: Array[Double], loss: OldLoss, algo: OldAlgo.Value): Array[Double] = { - - val sc = data.sparkContext val remappedData = algo match { - case OldAlgo.Classification => data.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + case OldAlgo.Classification => + data.map(x => Instance((x.label * 2) - 1, x.weight, x.features)) case _ => data } - val broadcastTrees = sc.broadcast(trees) - val localTreeWeights = treeWeights - val treesIndices = trees.indices - - val dataCount = remappedData.count() - val evaluation = remappedData.map { point => - treesIndices.map { idx => - val prediction = broadcastTrees.value(idx) - .rootNode - .predictImpl(point.features) - .prediction - prediction * localTreeWeights(idx) + val numTrees = trees.length + val (errSum, weightSum) = remappedData.mapPartitions { iter => + iter.map { case Instance(label, weight, features) => + val pred = Array.tabulate(numTrees) { i => + trees(i).rootNode.predictImpl(features) + .prediction * treeWeights(i) + } + val err = pred.scanLeft(0.0)(_ + _).drop(1) + .map(p => loss.computeError(p, label) * weight) + (err, weight) } - .scanLeft(0.0)(_ + _).drop(1) - .map(prediction => loss.computeError(prediction, point.label)) + }.treeReduce { case ((err1, weight1), (err2, weight2)) => + (0 until numTrees).foreach(i => err1(i) += err2(i)) + (err1, weight1 + weight2) } - .aggregate(treesIndices.map(_ => 0.0))( - (aggregated, row) => treesIndices.map(idx => aggregated(idx) + row(idx)), - (a, b) => treesIndices.map(idx => a(idx) + b(idx))) - .map(_ / dataCount) - broadcastTrees.destroy() - evaluation.toArray + errSum.map(_ / weightSum) } /** @@ -245,8 +256,8 @@ private[spark] object GradientBoostedTrees extends Logging { * (array of decision tree models, array of model weights) */ def boost( - input: RDD[LabeledPoint], - validationInput: RDD[LabeledPoint], + input: RDD[Instance], + validationInput: RDD[Instance], boostingStrategy: OldBoostingStrategy, validate: Boolean, seed: Long, @@ -280,8 +291,10 @@ private[spark] object GradientBoostedTrees extends Logging { } // Prepare periodic checkpointers + // Note: this is checkpointing the unweighted training error val predErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)]( treeStrategy.getCheckpointInterval, input.sparkContext) + // Note: this is checkpointing the unweighted validation error val validatePredErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)]( treeStrategy.getCheckpointInterval, input.sparkContext) @@ -299,26 +312,29 @@ private[spark] object GradientBoostedTrees extends Logging { baseLearners(0) = firstTreeModel baseLearnerWeights(0) = firstTreeWeight - var predError: RDD[(Double, Double)] = - computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) + var predError = computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) predErrorCheckpointer.update(predError) - logDebug("error of gbt = " + predError.values.mean()) + logDebug("error of gbt = " + computeWeightedError(input, predError)) // Note: A model of type regression is used since we require raw prediction timer.stop("building tree 0") - var validatePredError: RDD[(Double, Double)] = + var validatePredError = computeInitialPredictionAndError(validationInput, firstTreeWeight, firstTreeModel, loss) if (validate) validatePredErrorCheckpointer.update(validatePredError) - var bestValidateError = if (validate) validatePredError.values.mean() else 0.0 + var bestValidateError = if (validate) { + computeWeightedError(validationInput, validatePredError) + } else { + 0.0 + } var bestM = 1 var m = 1 var doneLearning = false while (m < numIterations && !doneLearning) { // Update data with pseudo-residuals - val data = predError.zip(input).map { case ((pred, _), point) => - LabeledPoint(-loss.gradient(pred, point.label), point.features) + val data = predError.zip(input).map { case ((pred, _), Instance(label, weight, features)) => + Instance(-loss.gradient(pred, label), weight, features) } timer.start(s"building tree $m") @@ -339,7 +355,7 @@ private[spark] object GradientBoostedTrees extends Logging { predError = updatePredictionError( input, predError, baseLearnerWeights(m), baseLearners(m), loss) predErrorCheckpointer.update(predError) - logDebug("error of gbt = " + predError.values.mean()) + logDebug("error of gbt = " + computeWeightedError(input, predError)) if (validate) { // Stop training early if @@ -350,7 +366,7 @@ private[spark] object GradientBoostedTrees extends Logging { validatePredError = updatePredictionError( validationInput, validatePredError, baseLearnerWeights(m), baseLearners(m), loss) validatePredErrorCheckpointer.update(validatePredError) - val currentValidateError = validatePredError.values.mean() + val currentValidateError = computeWeightedError(validationInput, validatePredError) if (bestValidateError - currentValidateError < validationTol * Math.max( currentValidateError, 0.01)) { doneLearning = true diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index d24d8da0dab48..d57f1b36a572c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.tree.impl.{GradientBoostedTrees => NewGBT} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.BoostingStrategy @@ -67,8 +67,9 @@ class GradientBoostedTrees private[spark] ( @Since("1.2.0") def run(input: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - val (trees, treeWeights) = NewGBT.run(input.map { point => - NewLabeledPoint(point.label, point.features.asML) + val (trees, treeWeights) = NewGBT.run(input.map { + case LabeledPoint(label, features) => + Instance(label, 1.0, features.asML) }, boostingStrategy, seed.toLong, "all") new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } @@ -97,10 +98,12 @@ class GradientBoostedTrees private[spark] ( input: RDD[LabeledPoint], validationInput: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - val (trees, treeWeights) = NewGBT.runWithValidation(input.map { point => - NewLabeledPoint(point.label, point.features.asML) - }, validationInput.map { point => - NewLabeledPoint(point.label, point.features.asML) + val (trees, treeWeights) = NewGBT.runWithValidation(input.map { + case LabeledPoint(label, features) => + Instance(label, 1.0, features.asML) + }, validationInput.map { + case LabeledPoint(label, features) => + Instance(label, 1.0, features.asML) }, boostingStrategy, seed.toLong, "all") new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index 530ca20d0eb0c..fdca71f8911c6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.ml.classification import com.github.fommil.netlib.BLAS import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.classification.LinearSVCSuite.generateSVMInput +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.regression.DecisionTreeRegressionModel @@ -52,8 +53,10 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { private var data: RDD[LabeledPoint] = _ private var trainData: RDD[LabeledPoint] = _ private var validationData: RDD[LabeledPoint] = _ + private var binaryDataset: DataFrame = _ private val eps: Double = 1e-5 private val absEps: Double = 1e-8 + private val seed = 42 override def beforeAll(): Unit = { super.beforeAll() @@ -65,6 +68,7 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) .map(_.asML) + binaryDataset = generateSVMInput(0.01, Array[Double](-1.5, 1.0), 1000, seed).toDF() } test("params") { @@ -362,7 +366,7 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { test("Tests of feature subset strategy") { val numClasses = 2 val gbt = new GBTClassifier() - .setSeed(42) + .setSeed(seed) .setMaxDepth(3) .setMaxIter(5) .setFeatureSubsetStrategy("all") @@ -397,13 +401,15 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { model3.trees.take(2), model3.treeWeights.take(2), model3.numFeatures, model3.numClasses) val evalArr = model3.evaluateEachIteration(validationData.toDF) - val remappedValidationData = validationData.map( - x => new LabeledPoint((x.label * 2) - 1, x.features)) - val lossErr1 = GradientBoostedTrees.computeError(remappedValidationData, + val remappedValidationData = validationData.map { + case LabeledPoint(label, features) => + Instance(label * 2 - 1, 1.0, features) + } + val lossErr1 = GradientBoostedTrees.computeWeightedError(remappedValidationData, model1.trees, model1.treeWeights, model1.getOldLossType) - val lossErr2 = GradientBoostedTrees.computeError(remappedValidationData, + val lossErr2 = GradientBoostedTrees.computeWeightedError(remappedValidationData, model2.trees, model2.treeWeights, model2.getOldLossType) - val lossErr3 = GradientBoostedTrees.computeError(remappedValidationData, + val lossErr3 = GradientBoostedTrees.computeWeightedError(remappedValidationData, model3.trees, model3.treeWeights, model3.getOldLossType) assert(evalArr(0) ~== lossErr1 relTol 1E-3) @@ -433,16 +439,19 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { assert(modelWithValidation.numTrees < numIter) val (errorWithoutValidation, errorWithValidation) = { - val remappedRdd = validationData.map(x => new LabeledPoint(2 * x.label - 1, x.features)) - (GradientBoostedTrees.computeError(remappedRdd, modelWithoutValidation.trees, + val remappedRdd = validationData.map { + case LabeledPoint(label, features) => + Instance(label * 2 - 1, 1.0, features) + } + (GradientBoostedTrees.computeWeightedError(remappedRdd, modelWithoutValidation.trees, modelWithoutValidation.treeWeights, modelWithoutValidation.getOldLossType), - GradientBoostedTrees.computeError(remappedRdd, modelWithValidation.trees, + GradientBoostedTrees.computeWeightedError(remappedRdd, modelWithValidation.trees, modelWithValidation.treeWeights, modelWithValidation.getOldLossType)) } assert(errorWithValidation < errorWithoutValidation) val evaluationArray = GradientBoostedTrees - .evaluateEachIteration(validationData, modelWithoutValidation.trees, + .evaluateEachIteration(validationData.map(_.toInstance), modelWithoutValidation.trees, modelWithoutValidation.treeWeights, modelWithoutValidation.getOldLossType, OldAlgo.Classification) assert(evaluationArray.length === numIter) @@ -472,6 +481,36 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { }) } + test("training with sample weights") { + val df = binaryDataset + val numClasses = 2 + val predEquals = (x: Double, y: Double) => x == y + // (maxIter, maxDepth) + val testParams = Seq( + (5, 5), + (5, 10) + ) + + for ((maxIter, maxDepth) <- testParams) { + val estimator = new GBTClassifier() + .setMaxIter(maxIter) + .setMaxDepth(maxDepth) + .setSeed(seed) + .setMinWeightFractionPerNode(0.049) + + MLTestingUtils.testArbitrarilyScaledWeights[GBTClassificationModel, + GBTClassifier](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, predEquals, 0.7)) + MLTestingUtils.testOutliersWithSmallWeights[GBTClassificationModel, + GBTClassifier](df.as[LabeledPoint], estimator, + numClasses, MLTestingUtils.modelPredictionEquals(df, predEquals, 0.8), + outlierRatio = 2) + MLTestingUtils.testOversamplingVsWeighting[GBTClassificationModel, + GBTClassifier](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, predEquals, 0.7), seed) + } + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 2b5a9a396effd..d2b8751360e9e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -1425,8 +1425,6 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { } test("multinomial logistic regression with zero variance (SPARK-21681)") { - val sqlContext = multinomialDatasetWithZeroVar.sqlContext - import sqlContext.implicits._ val mlr = new LogisticRegression().setFamily("multinomial").setFitIntercept(true) .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setWeightCol("weight") diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index e2462af2ac1d1..b772a3b7737d0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.util.LinearDataGenerator import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions.lit @@ -46,6 +47,8 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { private var data: RDD[LabeledPoint] = _ private var trainData: RDD[LabeledPoint] = _ private var validationData: RDD[LabeledPoint] = _ + private var linearRegressionData: DataFrame = _ + private val seed = 42 override def beforeAll(): Unit = { super.beforeAll() @@ -57,6 +60,9 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) .map(_.asML) + linearRegressionData = sc.parallelize(LinearDataGenerator.generateLinearInput( + intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), + xVariance = Array(0.7, 1.2), nPoints = 1000, seed, eps = 0.5), 2).map(_.asML).toDF() } test("Regression with continuous features") { @@ -202,7 +208,7 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { val gbt = new GBTRegressor() .setMaxDepth(3) .setMaxIter(5) - .setSeed(42) + .setSeed(seed) .setFeatureSubsetStrategy("all") // In this data, feature 1 is very important. @@ -237,11 +243,11 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { for (evalLossType <- GBTRegressor.supportedLossTypes) { val evalArr = model3.evaluateEachIteration(validationData.toDF, evalLossType) - val lossErr1 = GradientBoostedTrees.computeError(validationData, + val lossErr1 = GradientBoostedTrees.computeWeightedError(validationData.map(_.toInstance), model1.trees, model1.treeWeights, model1.convertToOldLossType(evalLossType)) - val lossErr2 = GradientBoostedTrees.computeError(validationData, + val lossErr2 = GradientBoostedTrees.computeWeightedError(validationData.map(_.toInstance), model2.trees, model2.treeWeights, model2.convertToOldLossType(evalLossType)) - val lossErr3 = GradientBoostedTrees.computeError(validationData, + val lossErr3 = GradientBoostedTrees.computeWeightedError(validationData.map(_.toInstance), model3.trees, model3.treeWeights, model3.convertToOldLossType(evalLossType)) assert(evalArr(0) ~== lossErr1 relTol 1E-3) @@ -272,17 +278,19 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { // early stop assert(modelWithValidation.numTrees < numIter) - val errorWithoutValidation = GradientBoostedTrees.computeError(validationData, + val errorWithoutValidation = GradientBoostedTrees.computeWeightedError( + validationData.map(_.toInstance), modelWithoutValidation.trees, modelWithoutValidation.treeWeights, modelWithoutValidation.getOldLossType) - val errorWithValidation = GradientBoostedTrees.computeError(validationData, + val errorWithValidation = GradientBoostedTrees.computeWeightedError( + validationData.map(_.toInstance), modelWithValidation.trees, modelWithValidation.treeWeights, modelWithValidation.getOldLossType) assert(errorWithValidation < errorWithoutValidation) val evaluationArray = GradientBoostedTrees - .evaluateEachIteration(validationData, modelWithoutValidation.trees, + .evaluateEachIteration(validationData.map(_.toInstance), modelWithoutValidation.trees, modelWithoutValidation.treeWeights, modelWithoutValidation.getOldLossType, OldAlgo.Regression) assert(evaluationArray.length === numIter) @@ -310,6 +318,35 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { }) } + test("training with sample weights") { + val df = linearRegressionData + val numClasses = 0 + // (maxIter, maxDepth) + val testParams = Seq( + (5, 5), + (5, 10) + ) + + for ((maxIter, maxDepth) <- testParams) { + val estimator = new GBTRegressor() + .setMaxIter(maxIter) + .setMaxDepth(maxDepth) + .setSeed(seed) + .setMinWeightFractionPerNode(0.1) + + MLTestingUtils.testArbitrarilyScaledWeights[GBTRegressionModel, + GBTRegressor](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.1, 0.95)) + MLTestingUtils.testOutliersWithSmallWeights[GBTRegressionModel, + GBTRegressor](df.as[LabeledPoint], estimator, numClasses, + MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.1, 0.95), + outlierRatio = 2) + MLTestingUtils.testOversamplingVsWeighting[GBTRegressionModel, + GBTRegressor](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.01, 0.95), seed) + } + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala index 366d5ec3a53fb..18fc1407557f1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.Instance import org.apache.spark.mllib.tree.{GradientBoostedTreesSuite => OldGBTSuite} import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ @@ -32,15 +32,12 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext */ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { - import testImplicits._ - test("runWithValidation stops early and performs better on a validation dataset") { // Set numIterations large enough so that it stops early. val numIterations = 20 - val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2).map(_.asML) - val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2).map(_.asML) - val trainDF = trainRdd.toDF() - val validateDF = validateRdd.toDF() + val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2).map(_.asML.toInstance) + val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2).map(_.asML.toInstance) + val seed = 42 val algos = Array(Regression, Regression, Classification) val losses = Array(SquaredError, AbsoluteError, LogLoss) @@ -50,21 +47,21 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext val boostingStrategy = new BoostingStrategy(treeStrategy, loss, numIterations, validationTol = 0.0) val (validateTrees, validateTreeWeights) = GradientBoostedTrees - .runWithValidation(trainRdd, validateRdd, boostingStrategy, 42L, "all") + .runWithValidation(trainRdd, validateRdd, boostingStrategy, seed, "all") val numTrees = validateTrees.length assert(numTrees !== numIterations) // Test that it performs better on the validation dataset. - val (trees, treeWeights) = GradientBoostedTrees.run(trainRdd, boostingStrategy, 42L, "all") + val (trees, treeWeights) = GradientBoostedTrees.run(trainRdd, boostingStrategy, seed, "all") val (errorWithoutValidation, errorWithValidation) = { if (algo == Classification) { - val remappedRdd = validateRdd.map(x => new LabeledPoint(2 * x.label - 1, x.features)) - (GradientBoostedTrees.computeError(remappedRdd, trees, treeWeights, loss), - GradientBoostedTrees.computeError(remappedRdd, validateTrees, + val remappedRdd = validateRdd.map(x => Instance(2 * x.label - 1, x.weight, x.features)) + (GradientBoostedTrees.computeWeightedError(remappedRdd, trees, treeWeights, loss), + GradientBoostedTrees.computeWeightedError(remappedRdd, validateTrees, validateTreeWeights, loss)) } else { - (GradientBoostedTrees.computeError(validateRdd, trees, treeWeights, loss), - GradientBoostedTrees.computeError(validateRdd, validateTrees, + (GradientBoostedTrees.computeWeightedError(validateRdd, trees, treeWeights, loss), + GradientBoostedTrees.computeWeightedError(validateRdd, validateTrees, validateTreeWeights, loss)) } } From cfbdd9d2932d8d80ab679f45f146641579546855 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 25 Oct 2019 16:32:06 +0900 Subject: [PATCH 0427/1387] [SPARK-29461][SQL] Measure the number of records being updated for JDBC writer ### What changes were proposed in this pull request? This patch adds the functionality to measure records being written for JDBC writer. In reality, the value is meant to be a number of records being updated from queries, as per JDBC spec it will return updated count. ### Why are the changes needed? Output metrics for JDBC writer are missing now. The value of "bytesWritten" is also missing, but we can't measure it from JDBC API. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test added. Closes #26109 from HeartSaVioR/SPARK-29461. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Takeshi Yamamuro --- .../datasources/jdbc/JdbcUtils.scala | 23 ++++++-- .../spark/sql/jdbc/JDBCWriteSuite.scala | 55 +++++++++++++++++++ 2 files changed, 73 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 86a27b5afc250..55ca4e3624bdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -605,6 +605,13 @@ object JdbcUtils extends Logging { * implementation changes elsewhere might easily render such a closure * non-Serializable. Instead, we explicitly close over all variables that * are used. + * + * Note that this method records task output metrics. It assumes the method is + * running in a task. For now, we only records the number of rows being written + * because there's no good way to measure the total bytes being written. Only + * effective outputs are taken into account: for example, metric will not be updated + * if it supports transaction and transaction is rolled back, but metric will be + * updated even with error if it doesn't support transaction, as there're dirty outputs. */ def savePartition( getConnection: () => Connection, @@ -615,7 +622,9 @@ object JdbcUtils extends Logging { batchSize: Int, dialect: JdbcDialect, isolationLevel: Int, - options: JDBCOptions): Iterator[Byte] = { + options: JDBCOptions): Unit = { + val outMetrics = TaskContext.get().taskMetrics().outputMetrics + val conn = getConnection() var committed = false @@ -643,7 +652,7 @@ object JdbcUtils extends Logging { } } val supportsTransactions = finalIsolationLevel != Connection.TRANSACTION_NONE - + var totalRowCount = 0 try { if (supportsTransactions) { conn.setAutoCommit(false) // Everything in the same db transaction. @@ -672,6 +681,7 @@ object JdbcUtils extends Logging { } stmt.addBatch() rowCount += 1 + totalRowCount += 1 if (rowCount % batchSize == 0) { stmt.executeBatch() rowCount = 0 @@ -687,7 +697,6 @@ object JdbcUtils extends Logging { conn.commit() } committed = true - Iterator.empty } catch { case e: SQLException => val cause = e.getNextException @@ -715,9 +724,13 @@ object JdbcUtils extends Logging { // tell the user about another problem. if (supportsTransactions) { conn.rollback() + } else { + outMetrics.setRecordsWritten(totalRowCount) } conn.close() } else { + outMetrics.setRecordsWritten(totalRowCount) + // The stage must succeed. We cannot propagate any exception close() might throw. try { conn.close() @@ -840,10 +853,10 @@ object JdbcUtils extends Logging { case Some(n) if n < df.rdd.getNumPartitions => df.coalesce(n) case _ => df } - repartitionedDF.rdd.foreachPartition(iterator => savePartition( + repartitionedDF.rdd.foreachPartition { iterator => savePartition( getConnection, table, iterator, rddSchema, insertStmt, batchSize, dialect, isolationLevel, options) - ) + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index b28c6531d42b2..8021ef1a17a18 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -21,10 +21,12 @@ import java.sql.DriverManager import java.util.Properties import scala.collection.JavaConverters.propertiesAsScalaMapConverter +import scala.collection.mutable.ArrayBuffer import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} @@ -543,4 +545,57 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { }.getMessage assert(errMsg.contains("Statement was canceled or the session timed out")) } + + test("metrics") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + + runAndVerifyRecordsWritten(2) { + df.write.mode(SaveMode.Append).jdbc(url, "TEST.BASICCREATETEST", new Properties()) + } + + runAndVerifyRecordsWritten(1) { + df2.write.mode(SaveMode.Overwrite).jdbc(url, "TEST.BASICCREATETEST", new Properties()) + } + + runAndVerifyRecordsWritten(1) { + df2.write.mode(SaveMode.Overwrite).option("truncate", true) + .jdbc(url, "TEST.BASICCREATETEST", new Properties()) + } + + runAndVerifyRecordsWritten(0) { + intercept[AnalysisException] { + df2.write.mode(SaveMode.ErrorIfExists).jdbc(url, "TEST.BASICCREATETEST", new Properties()) + } + } + + runAndVerifyRecordsWritten(0) { + df.write.mode(SaveMode.Ignore).jdbc(url, "TEST.BASICCREATETEST", new Properties()) + } + } + + private def runAndVerifyRecordsWritten(expected: Long)(job: => Unit): Unit = { + assert(expected === runAndReturnMetrics(job, _.taskMetrics.outputMetrics.recordsWritten)) + } + + private def runAndReturnMetrics(job: => Unit, collector: (SparkListenerTaskEnd) => Long): Long = { + val taskMetrics = new ArrayBuffer[Long]() + + // Avoid receiving earlier taskEnd events + sparkContext.listenerBus.waitUntilEmpty() + + val listener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + taskMetrics += collector(taskEnd) + } + } + sparkContext.addSparkListener(listener) + + job + + sparkContext.listenerBus.waitUntilEmpty() + + sparkContext.removeSparkListener(listener) + taskMetrics.sum + } } From 8bd8f492ea006ce03d215c3b272c31c1b8bc1858 Mon Sep 17 00:00:00 2001 From: redsk Date: Fri, 25 Oct 2019 08:06:36 -0500 Subject: [PATCH 0428/1387] [SPARK-29500][SQL][SS] Support partition column when writing to Kafka ### What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-29500 `KafkaRowWriter` now supports setting the Kafka partition by reading a "partition" column in the input dataframe. Code changes in commit nr. 1. Test changes in commit nr. 2. Doc changes in commit nr. 3. tcondie dongjinleekr srowen ### Why are the changes needed? While it is possible to configure a custom Kafka Partitioner with `.option("kafka.partitioner.class", "my.custom.Partitioner")`, this is not enough for certain use cases. See the Jira issue. ### Does this PR introduce any user-facing change? No, as this behaviour is optional. ### How was this patch tested? Two new UT were added and one was updated. Closes #26153 from redsk/feature/SPARK-29500. Authored-by: redsk Signed-off-by: Sean Owen --- .../structured-streaming-kafka-integration.md | 10 +++ .../spark/sql/kafka010/KafkaWriteTask.scala | 22 ++++- .../spark/sql/kafka010/KafkaWriter.scala | 11 ++- .../kafka010/KafkaContinuousSinkSuite.scala | 9 ++ .../spark/sql/kafka010/KafkaSinkSuite.scala | 88 ++++++++++++++++++- 5 files changed, 134 insertions(+), 6 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 89732d309aa27..badf0429545f3 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -614,6 +614,10 @@ The Dataframe being written to Kafka should have the following columns in schema topic (*optional) string + + partition (optional) + int + \* The topic column is required if the "topic" configuration option is not specified.
    @@ -622,6 +626,12 @@ a ```null``` valued key column will be automatically added (see Kafka semantics how ```null``` valued key values are handled). If a topic column exists then its value is used as the topic when writing the given row to Kafka, unless the "topic" configuration option is set i.e., the "topic" configuration option overrides the topic column. +If a "partition" column is not specified (or its value is ```null```) +then the partition is calculated by the Kafka producer. +A Kafka partitioner can be specified in Spark by setting the +```kafka.partitioner.class``` option. If not present, Kafka default partitioner +will be used. + The following options must be set for the Kafka sink for both batch and streaming queries. diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index b423ddc959c1b..5bdc1b5fe9f37 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.header.internals.RecordHeader import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} -import org.apache.spark.sql.types.{BinaryType, StringType} +import org.apache.spark.sql.types.{BinaryType, IntegerType, StringType} /** * Writes out data in a single Spark task, without any concerns about how @@ -92,8 +92,10 @@ private[kafka010] abstract class KafkaRowWriter( throw new NullPointerException(s"null topic present in the data. Use the " + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a default topic.") } + val partition: Integer = + if (projectedRow.isNullAt(4)) null else projectedRow.getInt(4) val record = if (projectedRow.isNullAt(3)) { - new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, null, key, value) + new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, partition, key, value) } else { val headerArray = projectedRow.getArray(3) val headers = (0 until headerArray.numElements()).map { i => @@ -101,7 +103,8 @@ private[kafka010] abstract class KafkaRowWriter( new RecordHeader(struct.getUTF8String(0).toString, struct.getBinary(1)) .asInstanceOf[Header] } - new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, null, key, value, headers.asJava) + new ProducerRecord[Array[Byte], Array[Byte]]( + topic.toString, partition, key, value, headers.asJava) } producer.send(record, callback) } @@ -156,12 +159,23 @@ private[kafka010] abstract class KafkaRowWriter( throw new IllegalStateException(s"${KafkaWriter.HEADERS_ATTRIBUTE_NAME} " + s"attribute unsupported type ${t.catalogString}") } + val partitionExpression = + inputSchema.find(_.name == KafkaWriter.PARTITION_ATTRIBUTE_NAME) + .getOrElse(Literal(null, IntegerType)) + partitionExpression.dataType match { + case IntegerType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.PARTITION_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t. ${KafkaWriter.PARTITION_ATTRIBUTE_NAME} " + + s"must be a ${IntegerType.catalogString}") + } UnsafeProjection.create( Seq( topicExpression, Cast(keyExpression, BinaryType), Cast(valueExpression, BinaryType), - headersExpression + headersExpression, + partitionExpression ), inputSchema ) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index bbb060356f730..9b0d11f137ce2 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.types.{BinaryType, MapType, StringType} +import org.apache.spark.sql.types.{BinaryType, IntegerType, MapType, StringType} import org.apache.spark.util.Utils /** @@ -41,6 +41,7 @@ private[kafka010] object KafkaWriter extends Logging { val KEY_ATTRIBUTE_NAME: String = "key" val VALUE_ATTRIBUTE_NAME: String = "value" val HEADERS_ATTRIBUTE_NAME: String = "headers" + val PARTITION_ATTRIBUTE_NAME: String = "partition" override def toString: String = "KafkaWriter" @@ -86,6 +87,14 @@ private[kafka010] object KafkaWriter extends Logging { throw new AnalysisException(s"$HEADERS_ATTRIBUTE_NAME attribute type " + s"must be a ${KafkaRecordToRowConverter.headersType.catalogString}") } + schema.find(_.name == PARTITION_ATTRIBUTE_NAME).getOrElse( + Literal(null, IntegerType) + ).dataType match { + case IntegerType => // good + case _ => + throw new AnalysisException(s"$PARTITION_ATTRIBUTE_NAME attribute type " + + s"must be an ${IntegerType.catalogString}") + } } def write( diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala index 65adbd6b9887c..cbf4952406c01 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala @@ -286,6 +286,15 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { } assert(ex3.getMessage.toLowerCase(Locale.ROOT).contains( "key attribute type must be a string or binary")) + + val ex4 = intercept[AnalysisException] { + /* partition field wrong type */ + createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "value as partition", "value" + ) + } + assert(ex4.getMessage.toLowerCase(Locale.ROOT).contains( + "partition attribute type must be an int")) } test("streaming - write to non-existing topic") { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index d77b9a3b6a9e1..aacb10f5197b0 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -22,6 +22,8 @@ import java.util.Locale import java.util.concurrent.atomic.AtomicInteger import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.producer.internals.DefaultPartitioner +import org.apache.kafka.common.Cluster import org.apache.kafka.common.serialization.ByteArraySerializer import org.scalatest.time.SpanSugar._ @@ -33,7 +35,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{BinaryType, DataType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, StringType, StructField, StructType} abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with KafkaTest { protected var testUtils: KafkaTestUtils = _ @@ -293,6 +295,21 @@ class KafkaSinkStreamingSuite extends KafkaSinkSuiteBase with StreamTest { } assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( "key attribute type must be a string or binary")) + + try { + ex = intercept[StreamingQueryException] { + /* partition field wrong type */ + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "value", "value as partition" + ) + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "partition attribute type must be an int")) } test("streaming - write to non-existing topic") { @@ -418,6 +435,65 @@ abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { ) } + def writeToKafka(df: DataFrame, topic: String, options: Map[String, String] = Map.empty): Unit = { + df + .write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .options(options) + .mode("append") + .save() + } + + def partitionsInTopic(topic: String): Set[Int] = { + createKafkaReader(topic) + .select("partition") + .map(_.getInt(0)) + .collect() + .toSet + } + + test("batch - partition column and partitioner priorities") { + val nrPartitions = 4 + val topic1 = newTopic() + val topic2 = newTopic() + val topic3 = newTopic() + val topic4 = newTopic() + testUtils.createTopic(topic1, nrPartitions) + testUtils.createTopic(topic2, nrPartitions) + testUtils.createTopic(topic3, nrPartitions) + testUtils.createTopic(topic4, nrPartitions) + val customKafkaPartitionerConf = Map( + "kafka.partitioner.class" -> "org.apache.spark.sql.kafka010.TestKafkaPartitioner" + ) + + val df = (0 until 5).map(n => (topic1, s"$n", s"$n")).toDF("topic", "key", "value") + + // default kafka partitioner + writeToKafka(df, topic1) + val partitionsInTopic1 = partitionsInTopic(topic1) + assert(partitionsInTopic1.size > 1) + + // custom partitioner (always returns 0) overrides default partitioner + writeToKafka(df, topic2, customKafkaPartitionerConf) + val partitionsInTopic2 = partitionsInTopic(topic2) + assert(partitionsInTopic2.size == 1) + assert(partitionsInTopic2.head == 0) + + // partition column overrides custom partitioner + val dfWithCustomPartition = df.withColumn("partition", lit(2)) + writeToKafka(dfWithCustomPartition, topic3, customKafkaPartitionerConf) + val partitionsInTopic3 = partitionsInTopic(topic3) + assert(partitionsInTopic3.size == 1) + assert(partitionsInTopic3.head == 2) + + // when the partition column value is null, it is ignored + val dfWithNullPartitions = df.withColumn("partition", lit(null).cast(IntegerType)) + writeToKafka(dfWithNullPartitions, topic4) + assert(partitionsInTopic(topic4) == partitionsInTopic1) + } + test("batch - null topic field value, and no topic option") { val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") val ex = intercept[SparkException] { @@ -515,3 +591,13 @@ class KafkaSinkBatchSuiteV2 extends KafkaSinkBatchSuiteBase { } } } + +class TestKafkaPartitioner extends DefaultPartitioner { + override def partition( + topic: String, + key: Any, + keyBytes: Array[Byte], + value: Any, + valueBytes: Array[Byte], + cluster: Cluster): Int = 0 +} From 0cf4f07c66b44770efa4b97db8d47d5fc394aeab Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 25 Oct 2019 22:19:19 +0900 Subject: [PATCH 0429/1387] [SPARK-29545][SQL] Add support for bit_xor aggregate function ### What changes were proposed in this pull request? bit_xor(expr) - Returns the bitwise XOR of all non-null input values, or null if none ### Why are the changes needed? As we support `bit_and`, `bit_or` now, we'd better support the related aggregate function **bit_xor** ahead of postgreSQL, because many other popular databases support it. http://infocenter.sybase.com/help/index.jsp?topic=/com.sybase.help.sqlanywhere.12.0.1/dbreference/bit-xor-function.html https://dev.mysql.com/doc/refman/5.7/en/group-by-functions.html#function_bit-or https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/Functions/Aggregate/BIT_XOR.htm?TocPath=SQL%20Reference%20Manual%7CSQL%20Functions%7CAggregate%20Functions%7C_____10 ### Does this PR introduce any user-facing change? add a new bit agg ### How was this patch tested? UTs added Closes #26205 from yaooqinn/SPARK-29545. Authored-by: Kent Yao Signed-off-by: Takeshi Yamamuro --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../aggregate/bitwiseAggregates.scala | 92 ++++++++++--------- .../resources/sql-tests/inputs/bitwise.sql | 31 +++++++ .../sql-tests/results/bitwise.sql.out | 71 +++++++++++++- 4 files changed, 151 insertions(+), 44 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 04e8963944fda..52e05b8203669 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -527,6 +527,7 @@ object FunctionRegistry { expression[BitwiseCount]("bit_count"), expression[BitAndAgg]("bit_and"), expression[BitOrAgg]("bit_or"), + expression[BitXorAgg]("bit_xor"), // json expression[StructsToJson]("to_json"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala index 131fa2eb50555..b77c3bd9cbde4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/bitwiseAggregates.scala @@ -17,20 +17,14 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BitwiseAnd, BitwiseOr, ExpectsInputTypes, Expression, ExpressionDescription, If, IsNull, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BinaryArithmetic, BitwiseAnd, BitwiseOr, BitwiseXor, ExpectsInputTypes, Expression, ExpressionDescription, If, IsNull, Literal} import org.apache.spark.sql.types.{AbstractDataType, DataType, IntegralType} -@ExpressionDescription( - usage = "_FUNC_(expr) - Returns the bitwise AND of all non-null input values, or null if none.", - examples = """ - Examples: - > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col); - 1 - """, - since = "3.0.0") -case class BitAndAgg(child: Expression) extends DeclarativeAggregate with ExpectsInputTypes { +abstract class BitAggregate extends DeclarativeAggregate with ExpectsInputTypes { - override def nodeName: String = "bit_and" + val child: Expression + + def bitOperator(left: Expression, right: Expression): BinaryArithmetic override def children: Seq[Expression] = child :: Nil @@ -40,23 +34,40 @@ case class BitAndAgg(child: Expression) extends DeclarativeAggregate with Expect override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) - private lazy val bitAnd = AttributeReference("bit_and", child.dataType)() - - override lazy val aggBufferAttributes: Seq[AttributeReference] = bitAnd :: Nil + private lazy val bitAgg = AttributeReference(nodeName, child.dataType)() override lazy val initialValues: Seq[Literal] = Literal.create(null, dataType) :: Nil + override lazy val aggBufferAttributes: Seq[AttributeReference] = bitAgg :: Nil + + override lazy val evaluateExpression: AttributeReference = bitAgg + override lazy val updateExpressions: Seq[Expression] = - If(IsNull(bitAnd), + If(IsNull(bitAgg), child, - If(IsNull(child), bitAnd, BitwiseAnd(bitAnd, child))) :: Nil + If(IsNull(child), bitAgg, bitOperator(bitAgg, child))) :: Nil override lazy val mergeExpressions: Seq[Expression] = - If(IsNull(bitAnd.left), - bitAnd.right, - If(IsNull(bitAnd.right), bitAnd.left, BitwiseAnd(bitAnd.left, bitAnd.right))) :: Nil + If(IsNull(bitAgg.left), + bitAgg.right, + If(IsNull(bitAgg.right), bitAgg.left, bitOperator(bitAgg.left, bitAgg.right))) :: Nil +} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the bitwise AND of all non-null input values, or null if none.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col); + 1 + """, + since = "3.0.0") +case class BitAndAgg(child: Expression) extends BitAggregate { - override lazy val evaluateExpression: AttributeReference = bitAnd + override def nodeName: String = "bit_and" + + override def bitOperator(left: Expression, right: Expression): BinaryArithmetic = { + BitwiseAnd(left, right) + } } @ExpressionDescription( @@ -67,33 +78,28 @@ case class BitAndAgg(child: Expression) extends DeclarativeAggregate with Expect 7 """, since = "3.0.0") -case class BitOrAgg(child: Expression) extends DeclarativeAggregate with ExpectsInputTypes { +case class BitOrAgg(child: Expression) extends BitAggregate { override def nodeName: String = "bit_or" - override def children: Seq[Expression] = child :: Nil - - override def nullable: Boolean = true - - override def dataType: DataType = child.dataType - - override def inputTypes: Seq[AbstractDataType] = Seq(IntegralType) - - private lazy val bitOr = AttributeReference("bit_or", child.dataType)() - - override lazy val aggBufferAttributes: Seq[AttributeReference] = bitOr :: Nil - - override lazy val initialValues: Seq[Literal] = Literal.create(null, dataType) :: Nil + override def bitOperator(left: Expression, right: Expression): BinaryArithmetic = { + BitwiseOr(left, right) + } +} - override lazy val updateExpressions: Seq[Expression] = - If(IsNull(bitOr), - child, - If(IsNull(child), bitOr, BitwiseOr(bitOr, child))) :: Nil +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the bitwise XOR of all non-null input values, or null if none.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (3), (5) AS tab(col); + 6 + """, + since = "3.0.0") +case class BitXorAgg(child: Expression) extends BitAggregate { - override lazy val mergeExpressions: Seq[Expression] = - If(IsNull(bitOr.left), - bitOr.right, - If(IsNull(bitOr.right), bitOr.left, BitwiseOr(bitOr.left, bitOr.right))) :: Nil + override def nodeName: String = "bit_xor" - override lazy val evaluateExpression: AttributeReference = bitOr + override def bitOperator(left: Expression, right: Expression): BinaryArithmetic = { + BitwiseXor(left, right) + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql b/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql index 993eecf0f89b6..5e665e4c0c384 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/bitwise.sql @@ -37,3 +37,34 @@ select bit_count(-9223372036854775808L); -- other illegal arguments select bit_count("bit count"); select bit_count('a'); + +-- test for bit_xor +-- +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (2, 3, 4, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); + +-- empty case +SELECT BIT_XOR(b3) AS n1 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_XOR(b4) AS n1 FROM bitwise_test where b4 is null; + +-- the suffix numbers show the expected answer +SELECT + BIT_XOR(cast(b1 as tinyint)) AS a4, + BIT_XOR(cast(b2 as smallint)) AS b5, + BIT_XOR(b3) AS c2, + BIT_XOR(b4) AS d2, + BIT_XOR(distinct b4) AS e2 +FROM bitwise_test; + +-- group by +SELECT bit_xor(b3) FROM bitwise_test GROUP BY b1 & 1; + +--having +SELECT b1, bit_xor(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7; + +-- window +SELECT b1, b2, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; diff --git a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out index 7cbd26e87bd2b..42c22a317eb46 100644 --- a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 20 +-- Number of queries: 27 -- !query 0 @@ -162,3 +162,72 @@ struct<> -- !query 19 output org.apache.spark.sql.AnalysisException cannot resolve 'bit_count('a')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''a'' is of string type.; line 1 pos 7 + + +-- !query 20 +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (2, 3, 4, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +SELECT BIT_XOR(b3) AS n1 FROM bitwise_test where 1 = 0 +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +SELECT BIT_XOR(b4) AS n1 FROM bitwise_test where b4 is null +-- !query 22 schema +struct +-- !query 22 output +NULL + + +-- !query 23 +SELECT + BIT_XOR(cast(b1 as tinyint)) AS a4, + BIT_XOR(cast(b2 as smallint)) AS b5, + BIT_XOR(b3) AS c2, + BIT_XOR(b4) AS d2, + BIT_XOR(distinct b4) AS e2 +FROM bitwise_test +-- !query 23 schema +struct +-- !query 23 output +4 5 2 2 2 + + +-- !query 24 +SELECT bit_xor(b3) FROM bitwise_test GROUP BY b1 & 1 +-- !query 24 schema +struct +-- !query 24 output +4 +6 + + +-- !query 25 +SELECT b1, bit_xor(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 +-- !query 25 schema +struct +-- !query 25 output +1 1 +2 3 + + +-- !query 26 +SELECT b1, b2, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query 26 schema +struct +-- !query 26 output +1 1 1 +2 3 3 +7 7 7 From 68dca9a0953e4a9472235acf78aecbb95c07acb6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 25 Oct 2019 23:09:08 +0800 Subject: [PATCH 0430/1387] [SPARK-29527][SQL] SHOW CREATE TABLE should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add ShowCreateTableStatement and make SHOW CREATE TABLE go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog SHOW CREATE TABLE t // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? yes. When running SHOW CREATE TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26184 from viirya/SPARK-29527. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 7 +++++++ .../spark/sql/catalyst/plans/logical/statements.scala | 5 +++++ .../apache/spark/sql/catalyst/parser/DDLParserSuite.scala | 6 ++++++ .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 6 +++++- .../org/apache/spark/sql/execution/SparkSqlParser.scala | 8 -------- .../apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 8 ++++++++ 7 files changed, 32 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 1e89507411adb..c97eb3c935be6 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -201,7 +201,7 @@ statement | SHOW PARTITIONS multipartIdentifier partitionSpec? #showPartitions | SHOW identifier? FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))? #showFunctions - | SHOW CREATE TABLE tableIdentifier #showCreateTable + | SHOW CREATE TABLE multipartIdentifier #showCreateTable | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction | (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index b030227b48814..4fa479f083e10 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2797,6 +2797,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging RepairTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) } + /** + * Creates a [[ShowCreateTableStatement]] + */ + override def visitShowCreateTable(ctx: ShowCreateTableContext): LogicalPlan = withOrigin(ctx) { + ShowCreateTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) + } + /** * Create a [[CacheTableStatement]]. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index ef8c922694347..655e87fce4e26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -330,6 +330,11 @@ case class AnalyzeColumnStatement( */ case class RepairTableStatement(tableName: Seq[String]) extends ParsedStatement +/** + * A SHOW CREATE TABLE statement, as parsed from SQL. + */ +case class ShowCreateTableStatement(tableName: Seq[String]) extends ParsedStatement + /** * A CACHE TABLE statement, as parsed from SQL */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index f4375956f0af6..da01c612b350e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1045,6 +1045,12 @@ class DDLParserSuite extends AnalysisTest { RepairTableStatement(Seq("a", "b", "c"))) } + test("SHOW CREATE table") { + comparePlans( + parsePlan("SHOW CREATE TABLE a.b.c"), + ShowCreateTableStatement(Seq("a", "b", "c"))) + } + test("CACHE TABLE") { comparePlans( parsePlan("CACHE TABLE a.b.c"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index f91686cb544c9..e7e34b1ef3127 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -299,6 +299,10 @@ class ResolveSessionCatalog( v1TableName.asTableIdentifier, "MSCK REPAIR TABLE") + case ShowCreateTableStatement(tableName) => + val v1TableName = parseV1Table(tableName, "SHOW CREATE TABLE") + ShowCreateTableCommand(v1TableName.asTableIdentifier) + case CacheTableStatement(tableName, plan, isLazy, options) => val v1TableName = parseV1Table(tableName, "CACHE TABLE") CacheTableCommand(v1TableName.asTableIdentifier, plan, isLazy, options) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index aef0a2d2e5951..20894b39ce5d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -135,14 +135,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ShowColumnsCommand(Option(ctx.db).map(_.getText), visitTableIdentifier(ctx.tableIdentifier)) } - /** - * Creates a [[ShowCreateTableCommand]] - */ - override def visitShowCreateTable(ctx: ShowCreateTableContext): LogicalPlan = withOrigin(ctx) { - val table = visitTableIdentifier(ctx.tableIdentifier()) - ShowCreateTableCommand(table) - } - /** * Create a [[RefreshResource]] logical plan. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 4f2c1af8f7b56..b8a8acbba57cd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1268,6 +1268,14 @@ class DataSourceV2SQLSuite } } + test("SHOW CREATE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + testV1Command("SHOW CREATE TABLE", t) + } + } + test("CACHE TABLE") { val t = "testcat.ns1.ns2.tbl" withTable(t) { From ae5b60da329ac63935d180d20a62f1bb181f5514 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 25 Oct 2019 11:13:06 -0700 Subject: [PATCH 0431/1387] [SPARK-29182][CORE][FOLLOWUP] Cache preferred locations of checkpointed RDD ### What changes were proposed in this pull request? This is a followup to #25856. This fixes the document about the config value of spark.rdd.checkpoint.cachePreferredLocsExpireTime. ### Why are the changes needed? The document is not correct. spark.rdd.checkpoint.cachePreferredLocsExpireTime can not be 0. ### Does this PR introduce any user-facing change? No ### How was this patch tested? This is document only change. Closes #26251 from viirya/SPARK-29182-followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 36211dc2ed4f8..444a1544777a1 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 @@ -804,7 +804,7 @@ package object config { .doc("Expire time in minutes for caching preferred locations of checkpointed RDD." + "Caching preferred locations can relieve query loading to DFS and save the query " + "time. The drawback is that the cached locations can be possibly outdated and " + - "lose data locality. If this config is not specified or is 0, it will not cache.") + "lose data locality. If this config is not specified, it will not cache.") .timeConf(TimeUnit.MINUTES) .checkValue(_ > 0, "The expire time for caching preferred locations cannot be non-positive.") .createOptional From 2baf7a1d8ff0d7018d6b70876c1e65b549ae30b0 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 25 Oct 2019 13:57:26 -0700 Subject: [PATCH 0432/1387] [SPARK-29608][BUILD] Add `hadoop-3.2` profile to release build ### What changes were proposed in this pull request? This PR aims to add `hadoop-3.2` profile to pre-built binary package releases. ### Why are the changes needed? Since Apache Spark 3.0.0, we provides Hadoop 3.2 pre-built binary. ### Does this PR introduce any user-facing change? No. (Although the artifacts are available, this change is for release managers). ### How was this patch tested? Manual. Please note that `DRY_RUN=0` disables these combination. ``` $ dev/create-release/release-build.sh package ... Packages to build: without-hadoop hadoop3.2 hadoop2.7 make_binary_release without-hadoop -Pscala-2.12 -Phadoop-provided 2.12 make_binary_release hadoop3.2 -Pscala-2.12 -Phadoop-3.2 -Phive -Phive-thriftserver 2.12 make_binary_release hadoop2.7 -Pscala-2.12 -Phadoop-2.7 -Phive -Phive-thriftserver withpip,withr 2.12 ``` Closes #26260 from dongjoon-hyun/SPARK-29608. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 61951e73f4bab..1f6fdb2a55ff4 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -280,6 +280,8 @@ if [[ "$1" == "package" ]]; then BINARY_PKGS_ARGS["without-hadoop"]="-Phadoop-provided" if [[ $SPARK_VERSION < "3.0." ]]; then BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" + else + BINARY_PKGS_ARGS["hadoop3.2"]="-Phadoop-3.2 $HIVE_PROFILES" fi fi From 25493919f82415c329c81c0c529eff576b491cd9 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Fri, 25 Oct 2019 14:11:35 -0700 Subject: [PATCH 0433/1387] [SPARK-29580][TESTS] Add kerberos debug messages for Kafka secure tests ### What changes were proposed in this pull request? `org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite` failed lately. After had a look at the logs it just shows the following fact without any details: ``` Caused by: sbt.ForkMain$ForkError: sun.security.krb5.KrbException: Server not found in Kerberos database (7) - Server not found in Kerberos database ``` Since the issue is intermittent and not able to reproduce it we should add more debug information and wait for reproduction with the extended logs. ### Why are the changes needed? Failing test doesn't give enough debug information. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? I've started the test manually and checked that such additional debug messages show up: ``` >>> KrbApReq: APOptions are 00000000 00000000 00000000 00000000 >>> EType: sun.security.krb5.internal.crypto.Aes128CtsHmacSha1EType Looking for keys for: kafka/localhostEXAMPLE.COM Added key: 17version: 0 Added key: 23version: 0 Added key: 16version: 0 Found unsupported keytype (3) for kafka/localhostEXAMPLE.COM >>> EType: sun.security.krb5.internal.crypto.Aes128CtsHmacSha1EType Using builtin default etypes for permitted_enctypes default etypes for permitted_enctypes: 17 16 23. >>> EType: sun.security.krb5.internal.crypto.Aes128CtsHmacSha1EType MemoryCache: add 1571936500/174770/16C565221B70AAB2BEFE31A83D13A2F4/client/localhostEXAMPLE.COM to client/localhostEXAMPLE.COM|kafka/localhostEXAMPLE.COM MemoryCache: Existing AuthList: #3: 1571936493/200803/8CD70D280B0862C5DA1FF901ECAD39FE/client/localhostEXAMPLE.COM #2: 1571936499/985009/BAD33290D079DD4E3579A8686EC326B7/client/localhostEXAMPLE.COM #1: 1571936499/995208/B76B9D78A9BE283AC78340157107FD40/client/localhostEXAMPLE.COM ``` Closes #26252 from gaborgsomogyi/SPARK-29580. Authored-by: Gabor Somogyi Signed-off-by: Dongjoon Hyun --- .../spark/sql/kafka010/KafkaTestUtils.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index bbb72bf9973e3..6c745987b4c23 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -67,6 +67,8 @@ class KafkaTestUtils( secure: Boolean = false) extends Logging { private val JAVA_AUTH_CONFIG = "java.security.auth.login.config" + private val IBM_KRB_DEBUG_CONFIG = "com.ibm.security.krb5.Krb5Debug" + private val SUN_KRB_DEBUG_CONFIG = "sun.security.krb5.debug" private val localCanonicalHostName = InetAddress.getLoopbackAddress().getCanonicalHostName() logInfo(s"Local host name is $localCanonicalHostName") @@ -133,6 +135,7 @@ class KafkaTestUtils( private def setUpMiniKdc(): Unit = { val kdcDir = Utils.createTempDir() val kdcConf = MiniKdc.createConf() + kdcConf.setProperty(MiniKdc.DEBUG, "true") kdc = new MiniKdc(kdcConf, kdcDir) kdc.start() kdcReady = true @@ -238,6 +241,7 @@ class KafkaTestUtils( } if (secure) { + setupKrbDebug() setUpMiniKdc() val jaasConfigFile = createKeytabsAndJaasConfigFile() System.setProperty(JAVA_AUTH_CONFIG, jaasConfigFile) @@ -252,6 +256,14 @@ class KafkaTestUtils( } } + private def setupKrbDebug(): Unit = { + if (System.getProperty("java.vendor").contains("IBM")) { + System.setProperty(IBM_KRB_DEBUG_CONFIG, "all") + } else { + System.setProperty(SUN_KRB_DEBUG_CONFIG, "true") + } + } + /** Teardown the whole servers, including Kafka broker and Zookeeper */ def teardown(): Unit = { if (leakDetector != null) { @@ -303,6 +315,15 @@ class KafkaTestUtils( kdc.stop() } UserGroupInformation.reset() + teardownKrbDebug() + } + + private def teardownKrbDebug(): Unit = { + if (System.getProperty("java.vendor").contains("IBM")) { + System.clearProperty(IBM_KRB_DEBUG_CONFIG) + } else { + System.clearProperty(SUN_KRB_DEBUG_CONFIG) + } } /** Create a Kafka topic and wait until it is propagated to the whole cluster */ From 5bdc58bf8a951df9b1be5a0298335b3668749358 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 25 Oct 2019 21:17:17 -0700 Subject: [PATCH 0434/1387] [SPARK-27653][SQL][FOLLOWUP] Fix `since` version of `min_by/max_by` ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/24557 to fix `since` version. ### Why are the changes needed? This is found during 3.0.0-preview preparation. The version will be exposed to our SQL document like the following. We had better fix this. - https://spark.apache.org/docs/latest/api/sql/#array_min ### Does this PR introduce any user-facing change? Yes. It's exposed at `DESC FUNCTION EXTENDED` SQL command and SQL doc, but this is new at 3.0.0. ### How was this patch tested? Manual. ``` spark-sql> DESC FUNCTION EXTENDED min_by; Function: min_by Class: org.apache.spark.sql.catalyst.expressions.aggregate.MinBy Usage: min_by(x, y) - Returns the value of `x` associated with the minimum value of `y`. Extended Usage: Examples: > SELECT min_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y); a Since: 3.0.0 ``` Closes #26264 from dongjoon-hyun/SPARK-27653. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala index c7fdb15130c4f..b69b341b0ee3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala @@ -98,7 +98,7 @@ abstract class MaxMinBy extends DeclarativeAggregate { > SELECT _FUNC_(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y); b """, - since = "3.0") + since = "3.0.0") case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy { override protected def funcName: String = "max_by" @@ -116,7 +116,7 @@ case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMin > SELECT _FUNC_(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y); a """, - since = "3.0") + since = "3.0.0") case class MinBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy { override protected def funcName: String = "min_by" From 9a4670279177353519a0d12d9d37f7207f72488e Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 25 Oct 2019 23:02:11 -0700 Subject: [PATCH 0435/1387] [SPARK-29554][SQL] Add `version` SQL function ### What changes were proposed in this pull request? ``` hive> select version(); OK 3.1.1 rf4e0529634b6231a0072295da48af466cf2f10b7 Time taken: 2.113 seconds, Fetched: 1 row(s) ``` ### Why are the changes needed? From hive behavior and I guess it is useful for debugging and developing etc. ### Does this PR introduce any user-facing change? add a misc func ### How was this patch tested? add ut Closes #26209 from yaooqinn/SPARK-29554. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../spark/sql/catalyst/expressions/misc.scala | 17 +++++++++++++++-- .../apache/spark/sql/MiscFunctionsSuite.scala | 7 +++++++ 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 52e05b8203669..019e1a08779e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -486,6 +486,7 @@ object FunctionRegistry { expression[CurrentDatabase]("current_database"), expression[CallMethodViaReflection]("reflect"), expression[CallMethodViaReflection]("java_method"), + expression[Version]("version"), // grouping sets expression[Cube]("cube"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 2af2b13ad77f5..b8c23a1f08912 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import java.util.UUID - +import org.apache.spark.{SPARK_REVISION, SPARK_VERSION_SHORT} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -164,3 +163,17 @@ case class Uuid(randomSeed: Option[Long] = None) extends LeafExpression with Sta override def freshCopy(): Uuid = Uuid(randomSeed) } + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """_FUNC_() - Returns the Spark version. The string contains 2 fields, the first being a release version and the second being a git revision.""", + since = "3.0.0") +// scalastyle:on line.size.limit +case class Version() extends LeafExpression with CodegenFallback { + override def nullable: Boolean = false + override def foldable: Boolean = true + override def dataType: DataType = StringType + override def eval(input: InternalRow): Any = { + UTF8String.fromString(SPARK_VERSION_SHORT + " " + SPARK_REVISION) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala index cad0821dbf5aa..5ab06b1ebebf6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.{SPARK_REVISION, SPARK_VERSION_SHORT} import org.apache.spark.sql.test.SharedSparkSession class MiscFunctionsSuite extends QueryTest with SharedSparkSession { @@ -31,6 +32,12 @@ class MiscFunctionsSuite extends QueryTest with SharedSparkSession { s"java_method('$className', 'method1', a, b)"), Row("m1one", "m1one")) } + + test("version") { + checkAnswer( + Seq("").toDF("a").selectExpr("version()"), + Row(SPARK_VERSION_SHORT + " " + SPARK_REVISION)) + } } object ReflectClass { From 2115bf61465b504bc21e37465cb34878039b5cb8 Mon Sep 17 00:00:00 2001 From: rongma1997 Date: Fri, 25 Oct 2019 23:11:01 -0700 Subject: [PATCH 0436/1387] [SPARK-29490][SQL] Reset 'WritableColumnVector' in 'RowToColumnarExec' ### What changes were proposed in this pull request? Reset the `WritableColumnVector` when getting "next" ColumnarBatch in `RowToColumnarExec` ### Why are the changes needed? When converting `Iterator[InternalRow]` to `Iterator[ColumnarBatch]`, the vectors used to create a new `ColumnarBatch` should be reset in the iterator's "next()" method. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #26137 from rongma1997/reset-WritableColumnVector. Authored-by: rongma1997 Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/execution/Columnar.scala | 1 + .../sql/SparkSessionExtensionSuite.scala | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index 9d1636ccf2718..b41a4ff766672 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -454,6 +454,7 @@ case class RowToColumnarExec(child: SparkPlan) extends UnaryExecNode { override def next(): ColumnarBatch = { cb.setNumRows(0) + vectors.foreach(_.reset()) var rowCount = 0 while (rowCount < numRows && rowIterator.hasNext) { val row = rowIterator.next() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index b8df6f2bebf53..2a4c15233fe39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.SQLConf.COLUMN_BATCH_SIZE import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS import org.apache.spark.sql.types.{DataType, Decimal, IntegerType, LongType, Metadata, StructType} import org.apache.spark.sql.vectorized.{ColumnarArray, ColumnarBatch, ColumnarMap, ColumnVector} @@ -171,6 +172,30 @@ class SparkSessionExtensionSuite extends SparkFunSuite { } } + test("reset column vectors") { + val session = SparkSession.builder() + .master("local[1]") + .config(COLUMN_BATCH_SIZE.key, 2) + .withExtensions { extensions => + extensions.injectColumnar(session => + MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) } + .getOrCreate() + + try { + assert(session.sessionState.columnarRules.contains( + MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) + import session.sqlContext.implicits._ + + val input = Seq((100L), (200L), (300L)) + val data = input.toDF("vals").repartition(1) + val df = data.selectExpr("vals + 1") + val result = df.collect() + assert(result sameElements input.map(x => Row(x + 2))) + } finally { + stop(session) + } + } + test("use custom class for extensions") { val session = SparkSession.builder() .master("local[1]") From 077fb99a26a9e92104503fade25c0a095fec5e5d Mon Sep 17 00:00:00 2001 From: shahid Date: Sat, 26 Oct 2019 15:46:24 -0500 Subject: [PATCH 0437/1387] [SPARK-29589][WEBUI] Support pagination for sqlstats session table in JDBC/ODBC Session page ### What changes were proposed in this pull request? In the PR https://github.com/apache/spark/pull/26215, we supported pagination for sqlstats table in JDBC/ODBC server page. In this PR, we are extending the support of pagination to sqlstats session table by making use of existing pagination classes in https://github.com/apache/spark/pull/26215. ### Why are the changes needed? Support pagination for sqlsessionstats table in JDBC/ODBC server page in the WEBUI. It will easier for user to analyse the table and it may fix the potential issues like oom while loading the page, that may occur similar to the SQL page (refer #22645) ### Does this PR introduce any user-facing change? There will be no change in the sqlsessionstats table in JDBC/ODBC server page execpt pagination support. ### How was this patch tested? Manually verified. Before: ![Screenshot 2019-10-24 at 11 32 27 PM](https://user-images.githubusercontent.com/23054875/67512507-96715000-f6b6-11e9-9f1f-ab1877eb24e6.png) After: ![Screenshot 2019-10-24 at 10 58 53 PM](https://user-images.githubusercontent.com/23054875/67512314-295dba80-f6b6-11e9-9e3e-dd50c6e62fe9.png) Closes #26246 from shahidki31/SPARK_29589. Authored-by: shahid Signed-off-by: Sean Owen --- .../thriftserver/ui/ThriftServerPage.scala | 16 +-- .../ui/ThriftServerSessionPage.scala | 127 +++++++----------- 2 files changed, 50 insertions(+), 93 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index e472aaad5bdc6..7258978e3bada 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -28,7 +28,7 @@ import scala.xml.{Node, Unparsed} import org.apache.commons.text.StringEscapeUtils import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState, SessionInfo} +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, SessionInfo} import org.apache.spark.sql.hive.thriftserver.ui.ToolTips._ import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ @@ -181,14 +181,6 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" private def formatDurationOption(msOption: Option[Long]): String = { msOption.map(formatDurationVerbose).getOrElse(emptyCell) } - - /** Generate HTML table from string data */ - private def listingTable(headers: Seq[String], data: Seq[Seq[String]]) = { - def generateDataRow(data: Seq[String]): Seq[Node] = { - {data.map(d => {d})} - } - UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) - } } private[ui] class SqlStatsPagedTable( @@ -328,11 +320,12 @@ private[ui] class SqlStatsPagedTable( {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} + - {UIUtils.formatDuration(executionTime)} + {formatDurationVerbose(executionTime)} - {UIUtils.formatDuration(duration)} + {formatDurationVerbose(duration)} {info.statement} @@ -400,7 +393,6 @@ private[ui] class SqlStatsPagedTable( override def sliceData(from: Int, to: Int): Seq[SqlStatsTableRow] = { val r = data.slice(from, to) - r.map(x => x) _slicedStartTime = r.map(_.executionInfo.startTimestamp).toSet r } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 0aa0a2b8335d8..8b275f8f7be0d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -20,15 +20,13 @@ package org.apache.spark.sql.hive.thriftserver.ui import java.util.Calendar import javax.servlet.http.HttpServletRequest +import scala.collection.JavaConverters._ import scala.xml.Node -import org.apache.commons.text.StringEscapeUtils - import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState} -import org.apache.spark.sql.hive.thriftserver.ui.ToolTips._ import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ +import org.apache.spark.util.Utils /** Page for Spark Web UI that shows statistics of jobs running in the thrift server */ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) @@ -36,7 +34,6 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) private val listener = parent.listener private val startTime = Calendar.getInstance().getTime() - private val emptyCell = "-" /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { @@ -80,45 +77,52 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) .filter(_.sessionId == sessionID) val numStatement = executionList.size val table = if (numStatement > 0) { - val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", - "Execution Time", "Duration", "Statement", "State", "Detail") - val tooltips = Seq(None, None, None, None, Some(THRIFT_SERVER_FINISH_TIME), - Some(THRIFT_SERVER_CLOSE_TIME), Some(THRIFT_SERVER_EXECUTION), - Some(THRIFT_SERVER_DURATION), None, None, None) - assert(headerRow.length == tooltips.length) - val dataRows = executionList.sortBy(_.startTimestamp).reverse - - def generateDataRow(info: ExecutionInfo): Seq[Node] = { - val jobLink = info.jobId.map { id: String => - - [{id}] - + + val sqlTableTag = "sqlsessionstat" + + val parameterOtherTable = request.getParameterMap().asScala + .filterNot(_._1.startsWith(sqlTableTag)) + .map { case (name, vals) => + name + "=" + vals(0) } - val detail = Option(info.detail).filter(!_.isEmpty).getOrElse(info.executePlan) - - {info.userName} - - {jobLink} - - {info.groupId} - {formatDate(info.startTimestamp)} - {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} - {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} - - {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} - - - {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} - - {info.statement} - {info.state} - {errorMessageCell(detail)} - - } - Some(UIUtils.listingTable(headerRow, generateDataRow, - dataRows, false, None, Seq(null), false, tooltipHeaders = tooltips)) + val parameterSqlTablePage = request.getParameter(s"$sqlTableTag.page") + val parameterSqlTableSortColumn = request.getParameter(s"$sqlTableTag.sort") + val parameterSqlTableSortDesc = request.getParameter(s"$sqlTableTag.desc") + val parameterSqlPageSize = request.getParameter(s"$sqlTableTag.pageSize") + + val sqlTablePage = Option(parameterSqlTablePage).map(_.toInt).getOrElse(1) + val sqlTableSortColumn = Option(parameterSqlTableSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("Start Time") + val sqlTableSortDesc = Option(parameterSqlTableSortDesc).map(_.toBoolean).getOrElse( + // New executions should be shown above old executions by default. + sqlTableSortColumn == "Start Time" + ) + val sqlTablePageSize = Option(parameterSqlPageSize).map(_.toInt).getOrElse(100) + + try { + Some(new SqlStatsPagedTable( + request, + parent, + executionList, + "sqlserver/session", + UIUtils.prependBaseUri(request, parent.basePath), + parameterOtherTable, + sqlTableTag, + pageSize = sqlTablePageSize, + sortColumn = sqlTableSortColumn, + desc = sqlTableSortDesc + ).table(sqlTablePage)) + } catch { + case e@(_: IllegalArgumentException | _: IndexOutOfBoundsException) => + Some(
    +

    Error while rendering job table:

    +
    +              {Utils.exceptionString(e)}
    +            
    +
    ) + } } else { None } @@ -133,43 +137,4 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) content } - - private def errorMessageCell(errorMessage: String): Seq[Node] = { - val isMultiline = errorMessage.indexOf('\n') >= 0 - val errorSummary = StringEscapeUtils.escapeHtml4( - if (isMultiline) { - errorMessage.substring(0, errorMessage.indexOf('\n')) - } else { - errorMessage - }) - val details = if (isMultiline) { - // scalastyle:off - - + details - ++ - - // scalastyle:on - } else { - "" - } - {errorSummary}{details} - } - - /** - * Returns a human-readable string representing a duration such as "5 second 35 ms" - */ - private def formatDurationOption(msOption: Option[Long]): String = { - msOption.map(formatDurationVerbose).getOrElse(emptyCell) - } - - /** Generate HTML table from string data */ - private def listingTable(headers: Seq[String], data: Seq[Seq[String]]) = { - def generateDataRow(data: Seq[String]): Seq[Node] = { - {data.map(d => {d})} - } - UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) - } } From 74514b46e53231e3567570d183be04dfa9d4af0a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 27 Oct 2019 13:48:43 -0700 Subject: [PATCH 0438/1387] [SPARK-29614][SQL][TEST] Fix failures of DateTimeUtilsSuite and TimestampFormatterSuite ### What changes were proposed in this pull request? The `DateTimeUtilsSuite` and `TimestampFormatterSuite` assume constant time difference between `timestamp'yesterday'`, `timestamp'today'` and `timestamp'tomorrow'` which is wrong on daylight switching day - day length can be 23 or 25 hours. In the PR, I propose to use Java 8 time API to calculate instances of `yesterday` and `tomorrow` timestamps. ### Why are the changes needed? The changes fix test failures and make the tests tolerant to daylight time switching. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites `DateTimeUtilsSuite` and `TimestampFormatterSuite`. Closes #26273 from MaxGekk/midnight-tolerant. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/util/DateTimeUtilsSuite.scala | 11 +++++++---- .../spark/sql/util/TimestampFormatterSuite.scala | 13 ++++++++----- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 10642b3ca8a4f..0eaf538231284 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -586,12 +586,15 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { val now = instantToMicros(LocalDateTime.now(zoneId).atZone(zoneId).toInstant) toTimestamp("NOW", zoneId).get should be (now +- tolerance) assert(toTimestamp("now UTC", zoneId) === None) - val today = instantToMicros(LocalDateTime.now(zoneId) + val localToday = LocalDateTime.now(zoneId) .`with`(LocalTime.MIDNIGHT) - .atZone(zoneId).toInstant) - toTimestamp(" Yesterday", zoneId).get should be (today - MICROS_PER_DAY +- tolerance) + .atZone(zoneId) + val yesterday = instantToMicros(localToday.minusDays(1).toInstant) + toTimestamp(" Yesterday", zoneId).get should be (yesterday +- tolerance) + val today = instantToMicros(localToday.toInstant) toTimestamp("Today ", zoneId).get should be (today +- tolerance) - toTimestamp(" tomorrow CET ", zoneId).get should be (today + MICROS_PER_DAY +- tolerance) + val tomorrow = instantToMicros(localToday.plusDays(1).toInstant) + toTimestamp(" tomorrow CET ", zoneId).get should be (tomorrow +- tolerance) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index 170daa6277c49..84581c0badd86 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter} -import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, instantToMicros, MICROS_PER_DAY} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, instantToMicros} import org.apache.spark.sql.internal.SQLConf class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers { @@ -146,12 +146,15 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers assert(formatter.parse("EPOCH") === 0) val now = instantToMicros(LocalDateTime.now(zoneId).atZone(zoneId).toInstant) formatter.parse("now") should be (now +- tolerance) - val today = instantToMicros(LocalDateTime.now(zoneId) + val localToday = LocalDateTime.now(zoneId) .`with`(LocalTime.MIDNIGHT) - .atZone(zoneId).toInstant) - formatter.parse("yesterday CET") should be (today - MICROS_PER_DAY +- tolerance) + .atZone(zoneId) + val yesterday = instantToMicros(localToday.minusDays(1).toInstant) + formatter.parse("yesterday CET") should be (yesterday +- tolerance) + val today = instantToMicros(localToday.toInstant) formatter.parse(" TODAY ") should be (today +- tolerance) - formatter.parse("Tomorrow ") should be (today + MICROS_PER_DAY +- tolerance) + val tomorrow = instantToMicros(localToday.plusDays(1).toInstant) + formatter.parse("Tomorrow ") should be (tomorrow +- tolerance) } } } From a43b966f00cf5622aa88c98c1636924d0e24d626 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 27 Oct 2019 16:15:54 -0700 Subject: [PATCH 0439/1387] [SPARK-29613][BUILD][SS] Upgrade to Kafka 2.3.1 ### What changes were proposed in this pull request? This PR aims to upgrade to Kafka 2.3.1 client library for client fixes like KAFKA-8950, KAFKA-8570, and KAFKA-8635. The following is the full release note. - https://archive.apache.org/dist/kafka/2.3.1/RELEASE_NOTES.html ### Why are the changes needed? - [KAFKA-8950 KafkaConsumer stops fetching](https://issues.apache.org/jira/browse/KAFKA-8950) - [KAFKA-8570 Downconversion could fail when log contains out of order message formats](https://issues.apache.org/jira/browse/KAFKA-8570) - [KAFKA-8635 Unnecessary wait when looking up coordinator before transactional request](https://issues.apache.org/jira/browse/KAFKA-8635) ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #26271 from dongjoon-hyun/SPARK-29613. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c42ef5c6626d1..f1a7cb3d106f1 100644 --- a/pom.xml +++ b/pom.xml @@ -136,7 +136,7 @@ 1.2.1 - 2.3.0 + 2.3.1 10.12.1.1 1.10.1 1.5.6 From b19fd487dfe307542d65391fd7b8410fa4992698 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 28 Oct 2019 11:36:10 +0800 Subject: [PATCH 0440/1387] [SPARK-29093][PYTHON][ML] Remove automatically generated param setters in _shared_params_code_gen.py ### What changes were proposed in this pull request? Remove automatically generated param setters in _shared_params_code_gen.py ### Why are the changes needed? To keep parity between scala and python ### Does this PR introduce any user-facing change? Yes Add some setters in Python ML XXXModels ### How was this patch tested? unit tests Closes #26232 from huaxingao/spark-29093. Authored-by: Huaxin Gao Signed-off-by: zhengruifeng --- .../ml/feature/QuantileDiscretizer.scala | 5 +- python/pyspark/ml/base.py | 12 + python/pyspark/ml/classification.py | 324 +++++- python/pyspark/ml/clustering.py | 254 ++++- python/pyspark/ml/evaluation.py | 152 ++- python/pyspark/ml/feature.py | 1016 ++++++++++++++++- python/pyspark/ml/fpm.py | 13 + .../ml/param/_shared_params_code_gen.py | 6 - python/pyspark/ml/param/shared.py | 186 --- python/pyspark/ml/recommendation.py | 48 +- python/pyspark/ml/regression.py | 366 +++++- python/pyspark/ml/tests/test_param.py | 12 +- python/pyspark/ml/tuning.py | 36 + 13 files changed, 2141 insertions(+), 289 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index aa4ab5903f711..eb78d8224fc3f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types.StructType * Params for [[QuantileDiscretizer]]. */ private[feature] trait QuantileDiscretizerBase extends Params - with HasHandleInvalid with HasInputCol with HasOutputCol { + with HasHandleInvalid with HasInputCol with HasOutputCol with HasInputCols with HasOutputCols { /** * Number of buckets (quantiles, or categories) into which data points are grouped. Must @@ -129,8 +129,7 @@ private[feature] trait QuantileDiscretizerBase extends Params */ @Since("1.6.0") final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val uid: String) - extends Estimator[Bucketizer] with QuantileDiscretizerBase with DefaultParamsWritable - with HasInputCols with HasOutputCols { + extends Estimator[Bucketizer] with QuantileDiscretizerBase with DefaultParamsWritable { @Since("1.6.0") def this() = this(Identifiable.randomUID("quantileDiscretizer")) diff --git a/python/pyspark/ml/base.py b/python/pyspark/ml/base.py index 82ff81c58d3c6..542cb25172ead 100644 --- a/python/pyspark/ml/base.py +++ b/python/pyspark/ml/base.py @@ -194,6 +194,18 @@ class UnaryTransformer(HasInputCol, HasOutputCol, Transformer): .. versionadded:: 2.3.0 """ + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @abstractmethod def createTransformFunc(self): """ diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d0c821329471f..c5cdf35729dd8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -177,7 +177,19 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable >>> df = sc.parallelize([ ... Row(label=1.0, features=Vectors.dense(1.0, 1.0, 1.0)), ... Row(label=0.0, features=Vectors.dense(1.0, 2.0, 3.0))]).toDF() - >>> svm = LinearSVC(maxIter=5, regParam=0.01) + >>> svm = LinearSVC() + >>> svm.getMaxIter() + 100 + >>> svm.setMaxIter(5) + LinearSVC... + >>> svm.getMaxIter() + 5 + >>> svm.getRegParam() + 0.0 + >>> svm.setRegParam(0.01) + LinearSVC... + >>> svm.getRegParam() + 0.01 >>> model = svm.fit(df) >>> model.setPredictionCol("newPrediction") LinearSVC... @@ -257,6 +269,62 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return LinearSVCModel(java_model) + @since("2.2.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("2.2.0") + def setRegParam(self, value): + """ + Sets the value of :py:attr:`regParam`. + """ + return self._set(regParam=value) + + @since("2.2.0") + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + return self._set(tol=value) + + @since("2.2.0") + def setFitIntercept(self, value): + """ + Sets the value of :py:attr:`fitIntercept`. + """ + return self._set(fitIntercept=value) + + @since("2.2.0") + def setStandardization(self, value): + """ + Sets the value of :py:attr:`standardization`. + """ + return self._set(standardization=value) + + @since("2.2.0") + def setThreshold(self, value): + """ + Sets the value of :py:attr:`threshold`. + """ + return self._set(threshold=value) + + @since("2.2.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + @since("2.2.0") + def setAggregationDepth(self, value): + """ + Sets the value of :py:attr:`aggregationDepth`. + """ + return self._set(aggregationDepth=value) + class LinearSVCModel(JavaClassificationModel, _LinearSVCParams, JavaMLWritable, JavaMLReadable): """ @@ -265,6 +333,13 @@ class LinearSVCModel(JavaClassificationModel, _LinearSVCParams, JavaMLWritable, .. versionadded:: 2.2.0 """ + @since("3.0.0") + def setThreshold(self, value): + """ + Sets the value of :py:attr:`threshold`. + """ + return self._set(threshold=value) + @property @since("2.2.0") def coefficients(self): @@ -454,7 +529,18 @@ class LogisticRegression(JavaProbabilisticClassifier, _LogisticRegressionParams, ... Row(label=0.0, weight=2.0, features=Vectors.dense(1.0, 2.0)), ... Row(label=1.0, weight=3.0, features=Vectors.dense(2.0, 1.0)), ... Row(label=0.0, weight=4.0, features=Vectors.dense(3.0, 3.0))]).toDF() - >>> blor = LogisticRegression(regParam=0.01, weightCol="weight") + >>> blor = LogisticRegression(weightCol="weight") + >>> blor.getRegParam() + 0.0 + >>> blor.setRegParam(0.01) + LogisticRegression... + >>> blor.getRegParam() + 0.01 + >>> blor.setMaxIter(10) + LogisticRegression... + >>> blor.getMaxIter() + 10 + >>> blor.clear(blor.maxIter) >>> blorModel = blor.fit(bdf) >>> blorModel.setFeaturesCol("features") LogisticRegressionModel... @@ -603,6 +689,54 @@ def setUpperBoundsOnIntercepts(self, value): """ return self._set(upperBoundsOnIntercepts=value) + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + def setRegParam(self, value): + """ + Sets the value of :py:attr:`regParam`. + """ + return self._set(regParam=value) + + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + return self._set(tol=value) + + def setElasticNetParam(self, value): + """ + Sets the value of :py:attr:`elasticNetParam`. + """ + return self._set(elasticNetParam=value) + + def setFitIntercept(self, value): + """ + Sets the value of :py:attr:`fitIntercept`. + """ + return self._set(fitIntercept=value) + + def setStandardization(self, value): + """ + Sets the value of :py:attr:`standardization`. + """ + return self._set(standardization=value) + + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + def setAggregationDepth(self, value): + """ + Sets the value of :py:attr:`aggregationDepth`. + """ + return self._set(aggregationDepth=value) + class LogisticRegressionModel(JavaProbabilisticClassificationModel, _LogisticRegressionParams, JavaMLWritable, JavaMLReadable, HasTrainingSummary): @@ -1148,6 +1282,27 @@ def setImpurity(self, value): """ return self._set(impurity=value) + @since("1.4.0") + def setCheckpointInterval(self, value): + """ + Sets the value of :py:attr:`checkpointInterval`. + """ + return self._set(checkpointInterval=value) + + @since("1.4.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + @inherit_doc class DecisionTreeClassificationModel(_DecisionTreeModel, JavaProbabilisticClassificationModel, @@ -1366,6 +1521,18 @@ def setFeatureSubsetStrategy(self, value): """ return self._set(featureSubsetStrategy=value) + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + def setCheckpointInterval(self, value): + """ + Sets the value of :py:attr:`checkpointInterval`. + """ + return self._set(checkpointInterval=value) + class RandomForestClassificationModel(_TreeEnsembleModel, JavaProbabilisticClassificationModel, _RandomForestClassifierParams, JavaMLWritable, @@ -1451,6 +1618,10 @@ class GBTClassifier(JavaProbabilisticClassifier, _GBTClassifierParams, >>> td = si_model.transform(df) >>> gbt = GBTClassifier(maxIter=5, maxDepth=2, labelCol="indexed", seed=42, ... leafCol="leafId") + >>> gbt.setMaxIter(5) + GBTClassifier... + >>> gbt.getMaxIter() + 5 >>> gbt.getFeatureSubsetStrategy() 'all' >>> model = gbt.fit(td) @@ -1630,6 +1801,34 @@ def setValidationIndicatorCol(self, value): """ return self._set(validationIndicatorCol=value) + @since("1.4.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("1.4.0") + def setCheckpointInterval(self, value): + """ + Sets the value of :py:attr:`checkpointInterval`. + """ + return self._set(checkpointInterval=value) + + @since("1.4.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + @since("1.4.0") + def setStepSize(self, value): + """ + Sets the value of :py:attr:`stepSize`. + """ + return self._set(stepSize=value) + class GBTClassificationModel(_TreeEnsembleModel, JavaProbabilisticClassificationModel, _GBTClassifierParams, JavaMLWritable, JavaMLReadable): @@ -1723,10 +1922,6 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, >>> model = nb.fit(df) >>> model.setFeaturesCol("features") NaiveBayes_... - >>> model.setLabelCol("newLabel") - NaiveBayes_... - >>> model.getLabelCol() - 'newLabel' >>> model.getSmoothing() 1.0 >>> model.pi @@ -1814,6 +2009,12 @@ def setModelType(self, value): """ return self._set(modelType=value) + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + class NaiveBayesModel(JavaProbabilisticClassificationModel, _NaiveBayesParams, JavaMLWritable, JavaMLReadable): @@ -1906,7 +2107,11 @@ class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, _MultilayerPer ... (1.0, Vectors.dense([0.0, 1.0])), ... (1.0, Vectors.dense([1.0, 0.0])), ... (0.0, Vectors.dense([1.0, 1.0]))], ["label", "features"]) - >>> mlp = MultilayerPerceptronClassifier(maxIter=100, layers=[2, 2, 2], blockSize=1, seed=123) + >>> mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], blockSize=1, seed=123) + >>> mlp.setMaxIter(100) + MultilayerPerceptronClassifier... + >>> mlp.getMaxIter() + 100 >>> model = mlp.fit(df) >>> model.setFeaturesCol("features") MultilayerPerceptronClassifier... @@ -2000,6 +2205,31 @@ def setBlockSize(self, value): """ return self._set(blockSize=value) + @since("2.0.0") + def setInitialWeights(self, value): + """ + Sets the value of :py:attr:`initialWeights`. + """ + return self._set(initialWeights=value) + + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + return self._set(tol=value) + @since("2.0.0") def setStepSize(self, value): """ @@ -2007,12 +2237,11 @@ def setStepSize(self, value): """ return self._set(stepSize=value) - @since("2.0.0") - def setInitialWeights(self, value): + def setSolver(self, value): """ - Sets the value of :py:attr:`initialWeights`. + Sets the value of :py:attr:`solver`. """ - return self._set(initialWeights=value) + return self._set(solver=value) class MultilayerPerceptronClassificationModel(JavaProbabilisticClassificationModel, JavaMLWritable, @@ -2134,6 +2363,42 @@ def setClassifier(self, value): """ return self._set(classifier=value) + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + def setRawPredictionCol(self, value): + """ + Sets the value of :py:attr:`rawPredictionCol`. + """ + return self._set(rawPredictionCol=value) + + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + def setParallelism(self, value): + """ + Sets the value of :py:attr:`parallelism`. + """ + return self._set(parallelism=value) + def _fit(self, dataset): labelCol = self.getLabelCol() featuresCol = self.getFeaturesCol() @@ -2287,6 +2552,43 @@ class OneVsRestModel(Model, _OneVsRestParams, JavaMLReadable, JavaMLWritable): .. versionadded:: 2.0.0 """ + @since("2.0.0") + def setClassifier(self, value): + """ + Sets the value of :py:attr:`classifier`. + """ + return self._set(classifier=value) + + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + def setRawPredictionCol(self, value): + """ + Sets the value of :py:attr:`rawPredictionCol`. + """ + return self._set(rawPredictionCol=value) + + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + def __init__(self, models): super(OneVsRestModel, self).__init__() self.models = models diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index cbbbd36955dc0..bb73dc78c4ab4 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -123,6 +123,27 @@ class GaussianMixtureModel(JavaModel, _GaussianMixtureParams, JavaMLWritable, Ja .. versionadded:: 2.0.0 """ + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("3.0.0") + def setProbabilityCol(self, value): + """ + Sets the value of :py:attr:`probabilityCol`. + """ + return self._set(probabilityCol=value) + @property @since("2.0.0") def weights(self): @@ -200,8 +221,13 @@ class GaussianMixture(JavaEstimator, _GaussianMixtureParams, JavaMLWritable, Jav ... (Vectors.dense([-0.83, -0.68]),), ... (Vectors.dense([-0.91, -0.76]),)] >>> df = spark.createDataFrame(data, ["features"]) - >>> gm = GaussianMixture(k=3, tol=0.0001, - ... maxIter=10, seed=10) + >>> gm = GaussianMixture(k=3, tol=0.0001, seed=10) + >>> gm.getMaxIter() + 100 + >>> gm.setMaxIter(10) + GaussianMixture... + >>> gm.getMaxIter() + 10 >>> model = gm.fit(df) >>> model.getFeaturesCol() 'features' @@ -290,6 +316,48 @@ def setK(self, value): """ return self._set(k=value) + @since("2.0.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("2.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("2.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("2.0.0") + def setProbabilityCol(self, value): + """ + Sets the value of :py:attr:`probabilityCol`. + """ + return self._set(probabilityCol=value) + + @since("2.0.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + @since("2.0.0") + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + return self._set(tol=value) + class GaussianMixtureSummary(ClusteringSummary): """ @@ -389,6 +457,20 @@ class KMeansModel(JavaModel, _KMeansParams, GeneralJavaMLWritable, JavaMLReadabl .. versionadded:: 1.5.0 """ + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + @since("1.5.0") def clusterCenters(self): """Get the cluster centers, represented as a list of NumPy arrays.""" @@ -425,7 +507,14 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] >>> df = spark.createDataFrame(data, ["features"]) - >>> kmeans = KMeans(k=2, seed=1) + >>> kmeans = KMeans(k=2) + >>> kmeans.setSeed(1) + KMeans... + >>> kmeans.setMaxIter(10) + KMeans... + >>> kmeans.getMaxIter() + 10 + >>> kmeans.clear(kmeans.maxIter) >>> model = kmeans.fit(df) >>> model.getDistanceMeasure() 'euclidean' @@ -531,6 +620,41 @@ def setDistanceMeasure(self, value): """ return self._set(distanceMeasure=value) + @since("1.5.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("1.5.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("1.5.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("1.5.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + @since("1.5.0") + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + return self._set(tol=value) + @inherit_doc class _BisectingKMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionCol, @@ -571,6 +695,20 @@ class BisectingKMeansModel(JavaModel, _BisectingKMeansParams, JavaMLWritable, Ja .. versionadded:: 2.0.0 """ + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + @since("2.0.0") def clusterCenters(self): """Get the cluster centers, represented as a list of NumPy arrays.""" @@ -629,6 +767,16 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] >>> df = spark.createDataFrame(data, ["features"]) >>> bkm = BisectingKMeans(k=2, minDivisibleClusterSize=1.0) + >>> bkm.setMaxIter(10) + BisectingKMeans... + >>> bkm.getMaxIter() + 10 + >>> bkm.clear(bkm.maxIter) + >>> bkm.setSeed(1) + BisectingKMeans... + >>> bkm.getSeed() + 1 + >>> bkm.clear(bkm.seed) >>> model = bkm.fit(df) >>> model.getMaxIter() 20 @@ -723,6 +871,34 @@ def setDistanceMeasure(self, value): """ return self._set(distanceMeasure=value) + @since("2.0.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("2.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("2.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("2.0.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + def _create_model(self, java_model): return BisectingKMeansModel(java_model) @@ -873,6 +1049,31 @@ class LDAModel(JavaModel, _LDAParams): .. versionadded:: 2.0.0 """ + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + @since("3.0.0") + def setTopicDistributionCol(self, value): + """ + Sets the value of :py:attr:`topicDistributionCol`. + + >>> algo = LDA().setTopicDistributionCol("topicDistributionCol") + >>> algo.getTopicDistributionCol() + 'topicDistributionCol' + """ + return self._set(topicDistributionCol=value) + @since("2.0.0") def isDistributed(self): """ @@ -1045,6 +1246,11 @@ class LDA(JavaEstimator, _LDAParams, JavaMLReadable, JavaMLWritable): >>> df = spark.createDataFrame([[1, Vectors.dense([0.0, 1.0])], ... [2, SparseVector(2, {0: 1.0})],], ["id", "features"]) >>> lda = LDA(k=2, seed=1, optimizer="em") + >>> lda.setMaxIter(10) + LDA... + >>> lda.getMaxIter() + 10 + >>> lda.clear(lda.maxIter) >>> model = lda.fit(df) >>> model.getTopicDistributionCol() 'topicDistribution' @@ -1125,6 +1331,20 @@ def setParams(self, featuresCol="features", maxIter=20, seed=None, checkpointInt kwargs = self._input_kwargs return self._set(**kwargs) + @since("2.0.0") + def setCheckpointInterval(self, value): + """ + Sets the value of :py:attr:`checkpointInterval`. + """ + return self._set(checkpointInterval=value) + + @since("2.0.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + @since("2.0.0") def setK(self, value): """ @@ -1236,6 +1456,20 @@ def setKeepLastCheckpoint(self, value): """ return self._set(keepLastCheckpoint=value) + @since("2.0.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("2.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + @inherit_doc class _PowerIterationClusteringParams(HasMaxIter, HasWeightCol): @@ -1392,6 +1626,20 @@ def setDstCol(self, value): """ return self._set(dstCol=value) + @since("2.4.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("2.4.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + @since("2.4.0") def assignClusters(self, dataset): """ diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index cdd9be7bf11b3..6539e2abaed12 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -119,7 +119,9 @@ class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPrediction ... [(0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)]) >>> dataset = spark.createDataFrame(scoreAndLabels, ["raw", "label"]) ... - >>> evaluator = BinaryClassificationEvaluator(rawPredictionCol="raw") + >>> evaluator = BinaryClassificationEvaluator() + >>> evaluator.setRawPredictionCol("raw") + BinaryClassificationEvaluator... >>> evaluator.evaluate(dataset) 0.70... >>> evaluator.evaluate(dataset, {evaluator.metricName: "areaUnderPR"}) @@ -196,6 +198,25 @@ def getNumBins(self): """ return self.getOrDefault(self.numBins) + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + def setRawPredictionCol(self, value): + """ + Sets the value of :py:attr:`rawPredictionCol`. + """ + return self._set(rawPredictionCol=value) + + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + @keyword_only @since("1.4.0") def setParams(self, rawPredictionCol="rawPrediction", labelCol="label", @@ -220,7 +241,9 @@ class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol, HasWeigh ... (-25.98418959, -22.0), (30.69731842, 33.0), (74.69283752, 71.0)] >>> dataset = spark.createDataFrame(scoreAndLabels, ["raw", "label"]) ... - >>> evaluator = RegressionEvaluator(predictionCol="raw") + >>> evaluator = RegressionEvaluator() + >>> evaluator.setPredictionCol("raw") + RegressionEvaluator... >>> evaluator.evaluate(dataset) 2.842... >>> evaluator.evaluate(dataset, {evaluator.metricName: "r2"}) @@ -299,6 +322,25 @@ def getThroughOrigin(self): """ return self.getOrDefault(self.throughOrigin) + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + @keyword_only @since("1.4.0") def setParams(self, predictionCol="prediction", labelCol="label", @@ -322,7 +364,9 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio >>> scoreAndLabels = [(0.0, 0.0), (0.0, 1.0), (0.0, 0.0), ... (1.0, 0.0), (1.0, 1.0), (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)] >>> dataset = spark.createDataFrame(scoreAndLabels, ["prediction", "label"]) - >>> evaluator = MulticlassClassificationEvaluator(predictionCol="prediction") + >>> evaluator = MulticlassClassificationEvaluator() + >>> evaluator.setPredictionCol("prediction") + MulticlassClassificationEvaluator... >>> evaluator.evaluate(dataset) 0.66... >>> evaluator.evaluate(dataset, {evaluator.metricName: "accuracy"}) @@ -453,6 +497,32 @@ def getEps(self): """ return self.getOrDefault(self.eps) + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("3.0.0") + def setProbabilityCol(self, value): + """ + Sets the value of :py:attr:`probabilityCol`. + """ + return self._set(probabilityCol=value) + + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + @keyword_only @since("1.5.0") def setParams(self, predictionCol="prediction", labelCol="label", @@ -482,7 +552,9 @@ class MultilabelClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio ... ([0.0, 1.0, 2.0], [0.0, 1.0]), ([1.0], [1.0, 2.0])] >>> dataset = spark.createDataFrame(scoreAndLabels, ["prediction", "label"]) ... - >>> evaluator = MultilabelClassificationEvaluator(predictionCol="prediction") + >>> evaluator = MultilabelClassificationEvaluator() + >>> evaluator.setPredictionCol("prediction") + MultilabelClassificationEvaluator... >>> evaluator.evaluate(dataset) 0.63... >>> evaluator.evaluate(dataset, {evaluator.metricName: "accuracy"}) @@ -549,6 +621,20 @@ def getMetricLabel(self): """ return self.getOrDefault(self.metricLabel) + @since("3.0.0") + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + @keyword_only @since("3.0.0") def setParams(self, predictionCol="prediction", labelCol="label", @@ -581,7 +667,9 @@ class ClusteringEvaluator(JavaEvaluator, HasPredictionCol, HasFeaturesCol, ... ([10.5, 11.5], 1.0), ([1.0, 1.0], 0.0), ([8.0, 6.0], 1.0)]) >>> dataset = spark.createDataFrame(featureAndPredictions, ["features", "prediction"]) ... - >>> evaluator = ClusteringEvaluator(predictionCol="prediction") + >>> evaluator = ClusteringEvaluator() + >>> evaluator.setPredictionCol("prediction") + ClusteringEvaluator... >>> evaluator.evaluate(dataset) 0.9079... >>> ce_path = temp_path + "/ce" @@ -613,6 +701,18 @@ def __init__(self, predictionCol="prediction", featuresCol="features", kwargs = self._input_kwargs self._set(**kwargs) + @keyword_only + @since("2.3.0") + def setParams(self, predictionCol="prediction", featuresCol="features", + metricName="silhouette", distanceMeasure="squaredEuclidean"): + """ + setParams(self, predictionCol="prediction", featuresCol="features", \ + metricName="silhouette", distanceMeasure="squaredEuclidean") + Sets params for clustering evaluator. + """ + kwargs = self._input_kwargs + return self._set(**kwargs) + @since("2.3.0") def setMetricName(self, value): """ @@ -627,18 +727,6 @@ def getMetricName(self): """ return self.getOrDefault(self.metricName) - @keyword_only - @since("2.3.0") - def setParams(self, predictionCol="prediction", featuresCol="features", - metricName="silhouette", distanceMeasure="squaredEuclidean"): - """ - setParams(self, predictionCol="prediction", featuresCol="features", \ - metricName="silhouette", distanceMeasure="squaredEuclidean") - Sets params for clustering evaluator. - """ - kwargs = self._input_kwargs - return self._set(**kwargs) - @since("2.4.0") def setDistanceMeasure(self, value): """ @@ -653,6 +741,18 @@ def getDistanceMeasure(self): """ return self.getOrDefault(self.distanceMeasure) + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + @inherit_doc class RankingEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol, @@ -669,7 +769,9 @@ class RankingEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol, ... ([1.0, 2.0, 3.0, 4.0, 5.0], [])] >>> dataset = spark.createDataFrame(scoreAndLabels, ["prediction", "label"]) ... - >>> evaluator = RankingEvaluator(predictionCol="prediction") + >>> evaluator = RankingEvaluator() + >>> evaluator.setPredictionCol("prediction") + RankingEvaluator... >>> evaluator.evaluate(dataset) 0.35... >>> evaluator.evaluate(dataset, {evaluator.metricName: "precisionAtK", evaluator.k: 2}) @@ -734,6 +836,20 @@ def getK(self): """ return self.getOrDefault(self.k) + @since("3.0.0") + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + @keyword_only @since("3.0.0") def setParams(self, predictionCol="prediction", labelCol="label", diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index a0883f1d54fed..11bb7941b5d9a 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -76,6 +76,12 @@ class Binarizer(JavaTransformer, HasThreshold, HasThresholds, HasInputCol, HasOu >>> df = spark.createDataFrame([(0.5,)], ["values"]) >>> binarizer = Binarizer(threshold=1.0, inputCol="values", outputCol="features") + >>> binarizer.setThreshold(1.0) + Binarizer... + >>> binarizer.setInputCol("values") + Binarizer... + >>> binarizer.setOutputCol("features") + Binarizer... >>> binarizer.transform(df).head().features 0.0 >>> binarizer.setParams(outputCol="freqs").transform(df).head().freqs @@ -154,6 +160,32 @@ def setThresholds(self, value): """ return self._set(thresholds=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + class _LSHParams(HasInputCol, HasOutputCol): """ @@ -183,12 +215,36 @@ def setNumHashTables(self, value): """ return self._set(numHashTables=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + class _LSHModel(JavaModel, _LSHParams): """ Mixin for Locality Sensitive Hashing (LSH) models. """ + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def approxNearestNeighbors(self, dataset, key, numNearestNeighbors, distCol="distCol"): """ Given a large dataset and an item, approximately find at most k items which have the @@ -269,8 +325,15 @@ class BucketedRandomProjectionLSH(_LSH, _BucketedRandomProjectionLSHParams, ... (2, Vectors.dense([1.0, -1.0 ]),), ... (3, Vectors.dense([1.0, 1.0]),)] >>> df = spark.createDataFrame(data, ["id", "features"]) - >>> brp = BucketedRandomProjectionLSH(inputCol="features", outputCol="hashes", - ... seed=12345, bucketLength=1.0) + >>> brp = BucketedRandomProjectionLSH() + >>> brp.setInputCol("features") + BucketedRandomProjectionLSH... + >>> brp.setOutputCol("hashes") + BucketedRandomProjectionLSH... + >>> brp.setSeed(12345) + BucketedRandomProjectionLSH... + >>> brp.setBucketLength(1.0) + BucketedRandomProjectionLSH... >>> model = brp.fit(df) >>> model.getBucketLength() 1.0 @@ -350,6 +413,24 @@ def setBucketLength(self, value): """ return self._set(bucketLength=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + def _create_model(self, java_model): return BucketedRandomProjectionLSHModel(java_model) @@ -366,6 +447,20 @@ class BucketedRandomProjectionLSHModel(_LSHModel, _BucketedRandomProjectionLSHPa .. versionadded:: 2.2.0 """ + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @inherit_doc class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, @@ -380,8 +475,13 @@ class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOu >>> values = [(0.1, 0.0), (0.4, 1.0), (1.2, 1.3), (1.5, float("nan")), ... (float("nan"), 1.0), (float("nan"), 0.0)] >>> df = spark.createDataFrame(values, ["values1", "values2"]) - >>> bucketizer = Bucketizer(splits=[-float("inf"), 0.5, 1.4, float("inf")], - ... inputCol="values1", outputCol="buckets") + >>> bucketizer = Bucketizer() + >>> bucketizer.setSplits([-float("inf"), 0.5, 1.4, float("inf")]) + Bucketizer... + >>> bucketizer.setInputCol("values1") + Bucketizer... + >>> bucketizer.setOutputCol("buckets") + Bucketizer... >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df).collect() >>> bucketed = bucketizer.setHandleInvalid("keep").transform(df.select("values1")) >>> bucketed.show(truncate=False) @@ -510,6 +610,38 @@ def getSplitsArray(self): """ return self.getOrDefault(self.splitsArray) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + class _CountVectorizerParams(JavaParams, HasInputCol, HasOutputCol): """ @@ -595,7 +727,11 @@ class CountVectorizer(JavaEstimator, _CountVectorizerParams, JavaMLReadable, Jav >>> df = spark.createDataFrame( ... [(0, ["a", "b", "c"]), (1, ["a", "b", "b", "c", "a"])], ... ["label", "raw"]) - >>> cv = CountVectorizer(inputCol="raw", outputCol="vectors") + >>> cv = CountVectorizer() + >>> cv.setInputCol("raw") + CountVectorizer... + >>> cv.setOutputCol("vectors") + CountVectorizer... >>> model = cv.fit(df) >>> model.transform(df).show(truncate=False) +-----+---------------+-------------------------+ @@ -695,6 +831,18 @@ def setBinary(self, value): """ return self._set(binary=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return CountVectorizerModel(java_model) @@ -707,6 +855,34 @@ class CountVectorizerModel(JavaModel, _CountVectorizerParams, JavaMLReadable, Ja .. versionadded:: 1.6.0 """ + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + @since("3.0.0") + def setMinTF(self, value): + """ + Sets the value of :py:attr:`minTF`. + """ + return self._set(minTF=value) + + @since("3.0.0") + def setBinary(self, value): + """ + Sets the value of :py:attr:`binary`. + """ + return self._set(binary=value) + @classmethod @since("2.4.0") def from_vocabulary(cls, vocabulary, inputCol, outputCol=None, minTF=None, binary=None): @@ -766,7 +942,13 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit >>> from pyspark.ml.linalg import Vectors >>> df1 = spark.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) - >>> dct = DCT(inverse=False, inputCol="vec", outputCol="resultVec") + >>> dct = DCT( ) + >>> dct.setInverse(False) + DCT... + >>> dct.setInputCol("vec") + DCT... + >>> dct.setOutputCol("resultVec") + DCT... >>> df2 = dct.transform(df1) >>> df2.head().resultVec DenseVector([10.969..., -0.707..., -2.041...]) @@ -820,6 +1002,18 @@ def getInverse(self): """ return self.getOrDefault(self.inverse) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @inherit_doc class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, @@ -831,8 +1025,13 @@ class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReada >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(Vectors.dense([2.0, 1.0, 3.0]),)], ["values"]) - >>> ep = ElementwiseProduct(scalingVec=Vectors.dense([1.0, 2.0, 3.0]), - ... inputCol="values", outputCol="eprod") + >>> ep = ElementwiseProduct() + >>> ep.setScalingVec(Vectors.dense([1.0, 2.0, 3.0])) + ElementwiseProduct... + >>> ep.setInputCol("values") + ElementwiseProduct... + >>> ep.setOutputCol("eprod") + ElementwiseProduct... >>> ep.transform(df).head().eprod DenseVector([2.0, 2.0, 9.0]) >>> ep.setParams(scalingVec=Vectors.dense([2.0, 3.0, 5.0])).transform(df).head().eprod @@ -884,6 +1083,18 @@ def getScalingVec(self): """ return self.getOrDefault(self.scalingVec) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @inherit_doc class FeatureHasher(JavaTransformer, HasInputCols, HasOutputCol, HasNumFeatures, JavaMLReadable, @@ -923,7 +1134,11 @@ class FeatureHasher(JavaTransformer, HasInputCols, HasOutputCol, HasNumFeatures, >>> data = [(2.0, True, "1", "foo"), (3.0, False, "2", "bar")] >>> cols = ["real", "bool", "stringNum", "string"] >>> df = spark.createDataFrame(data, cols) - >>> hasher = FeatureHasher(inputCols=cols, outputCol="features") + >>> hasher = FeatureHasher() + >>> hasher.setInputCols(cols) + FeatureHasher... + >>> hasher.setOutputCol("features") + FeatureHasher... >>> hasher.transform(df).head().features SparseVector(262144, {174475: 2.0, 247670: 1.0, 257907: 1.0, 262126: 1.0}) >>> hasher.setCategoricalCols(["real"]).transform(df).head().features @@ -978,6 +1193,24 @@ def getCategoricalCols(self): """ return self.getOrDefault(self.categoricalCols) + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + def setNumFeatures(self, value): + """ + Sets the value of :py:attr:`numFeatures`. + """ + return self._set(numFeatures=value) + @inherit_doc class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures, JavaMLReadable, @@ -991,7 +1224,9 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures, Java otherwise the features will not be mapped evenly to the columns. >>> df = spark.createDataFrame([(["a", "b", "c"],)], ["words"]) - >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") + >>> hashingTF = HashingTF(inputCol="words", outputCol="features") + >>> hashingTF.setNumFeatures(10) + HashingTF... >>> hashingTF.transform(df).head().features SparseVector(10, {5: 1.0, 7: 1.0, 8: 1.0}) >>> hashingTF.setParams(outputCol="freqs").transform(df).head().freqs @@ -1050,6 +1285,24 @@ def getBinary(self): """ return self.getOrDefault(self.binary) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + def setNumFeatures(self, value): + """ + Sets the value of :py:attr:`numFeatures`. + """ + return self._set(numFeatures=value) + @since("3.0.0") def indexOf(self, term): """ @@ -1086,7 +1339,11 @@ class IDF(JavaEstimator, _IDFParams, JavaMLReadable, JavaMLWritable): >>> from pyspark.ml.linalg import DenseVector >>> df = spark.createDataFrame([(DenseVector([1.0, 2.0]),), ... (DenseVector([0.0, 1.0]),), (DenseVector([3.0, 0.2]),)], ["tf"]) - >>> idf = IDF(minDocFreq=3, inputCol="tf", outputCol="idf") + >>> idf = IDF(minDocFreq=3) + >>> idf.setInputCol("tf") + IDF... + >>> idf.setOutputCol("idf") + IDF... >>> model = idf.fit(df) >>> model.getMinDocFreq() 3 @@ -1145,6 +1402,18 @@ def setMinDocFreq(self, value): """ return self._set(minDocFreq=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return IDFModel(java_model) @@ -1156,6 +1425,20 @@ class IDFModel(JavaModel, _IDFParams, JavaMLReadable, JavaMLWritable): .. versionadded:: 1.4.0 """ + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @property @since("2.0.0") def idf(self): @@ -1228,7 +1511,11 @@ class Imputer(JavaEstimator, _ImputerParams, JavaMLReadable, JavaMLWritable): >>> df = spark.createDataFrame([(1.0, float("nan")), (2.0, float("nan")), (float("nan"), 3.0), ... (4.0, 4.0), (5.0, 5.0)], ["a", "b"]) - >>> imputer = Imputer(inputCols=["a", "b"], outputCols=["out_a", "out_b"]) + >>> imputer = Imputer() + >>> imputer.setInputCols(["a", "b"]) + Imputer... + >>> imputer.setOutputCols(["out_a", "out_b"]) + Imputer... >>> model = imputer.fit(df) >>> model.getStrategy() 'mean' @@ -1308,6 +1595,20 @@ def setMissingValue(self, value): """ return self._set(missingValue=value) + @since("2.2.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + @since("2.2.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + def _create_model(self, java_model): return ImputerModel(java_model) @@ -1319,6 +1620,20 @@ class ImputerModel(JavaModel, _ImputerParams, JavaMLReadable, JavaMLWritable): .. versionadded:: 2.2.0 """ + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + @property @since("2.2.0") def surrogateDF(self): @@ -1342,7 +1657,11 @@ class Interaction(JavaTransformer, HasInputCols, HasOutputCol, JavaMLReadable, J with four categories, the output would then be `Vector(0, 0, 0, 0, 3, 4, 0, 0)`. >>> df = spark.createDataFrame([(0.0, 1.0), (2.0, 3.0)], ["a", "b"]) - >>> interaction = Interaction(inputCols=["a", "b"], outputCol="ab") + >>> interaction = Interaction() + >>> interaction.setInputCols(["a", "b"]) + Interaction... + >>> interaction.setOutputCol("ab") + Interaction... >>> interaction.transform(df).show() +---+---+-----+ | a| b| ab| @@ -1381,6 +1700,20 @@ def setParams(self, inputCols=None, outputCol=None): kwargs = self._input_kwargs return self._set(**kwargs) + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + class _MaxAbsScalerParams(HasInputCol, HasOutputCol): """ @@ -1400,7 +1733,9 @@ class MaxAbsScaler(JavaEstimator, _MaxAbsScalerParams, JavaMLReadable, JavaMLWri >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) - >>> maScaler = MaxAbsScaler(inputCol="a", outputCol="scaled") + >>> maScaler = MaxAbsScaler(outputCol="scaled") + >>> maScaler.setInputCol("a") + MaxAbsScaler... >>> model = maScaler.fit(df) >>> model.setOutputCol("scaledOutput") MaxAbsScaler... @@ -1449,6 +1784,18 @@ def setParams(self, inputCol=None, outputCol=None): kwargs = self._input_kwargs return self._set(**kwargs) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return MaxAbsScalerModel(java_model) @@ -1460,6 +1807,20 @@ class MaxAbsScalerModel(JavaModel, _MaxAbsScalerParams, JavaMLReadable, JavaMLWr .. versionadded:: 2.0.0 """ + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @property @since("2.0.0") def maxAbs(self): @@ -1487,7 +1848,13 @@ class MinHashLSH(_LSH, HasInputCol, HasOutputCol, HasSeed, JavaMLReadable, JavaM ... (1, Vectors.sparse(6, [2, 3, 4], [1.0, 1.0, 1.0]),), ... (2, Vectors.sparse(6, [0, 2, 4], [1.0, 1.0, 1.0]),)] >>> df = spark.createDataFrame(data, ["id", "features"]) - >>> mh = MinHashLSH(inputCol="features", outputCol="hashes", seed=12345) + >>> mh = MinHashLSH() + >>> mh.setInputCol("features") + MinHashLSH... + >>> mh.setOutputCol("hashes") + MinHashLSH... + >>> mh.setSeed(12345) + MinHashLSH... >>> model = mh.fit(df) >>> model.transform(df).head() Row(id=0, features=SparseVector(6, {0: 1.0, 1: 1.0, 2: 1.0}), hashes=[DenseVector([6179668... @@ -1544,6 +1911,12 @@ def setParams(self, inputCol=None, outputCol=None, seed=None, numHashTables=1): kwargs = self._input_kwargs return self._set(**kwargs) + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + def _create_model(self, java_model): return MinHashLSHModel(java_model) @@ -1606,7 +1979,9 @@ class MinMaxScaler(JavaEstimator, _MinMaxScalerParams, JavaMLReadable, JavaMLWri >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) - >>> mmScaler = MinMaxScaler(inputCol="a", outputCol="scaled") + >>> mmScaler = MinMaxScaler(outputCol="scaled") + >>> mmScaler.setInputCol("a") + MinMaxScaler... >>> model = mmScaler.fit(df) >>> model.setOutputCol("scaledOutput") MinMaxScaler... @@ -1675,6 +2050,18 @@ def setMax(self, value): """ return self._set(max=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return MinMaxScalerModel(java_model) @@ -1686,6 +2073,34 @@ class MinMaxScalerModel(JavaModel, _MinMaxScalerParams, JavaMLReadable, JavaMLWr .. versionadded:: 1.6.0 """ + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + @since("3.0.0") + def setMin(self, value): + """ + Sets the value of :py:attr:`min`. + """ + return self._set(min=value) + + @since("3.0.0") + def setMax(self, value): + """ + Sets the value of :py:attr:`max`. + """ + return self._set(max=value) + @property @since("2.0.0") def originalMin(self): @@ -1716,7 +2131,11 @@ class NGram(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWr returned. >>> df = spark.createDataFrame([Row(inputTokens=["a", "b", "c", "d", "e"])]) - >>> ngram = NGram(n=2, inputCol="inputTokens", outputCol="nGrams") + >>> ngram = NGram(n=2) + >>> ngram.setInputCol("inputTokens") + NGram... + >>> ngram.setOutputCol("nGrams") + NGram... >>> ngram.transform(df).head() Row(inputTokens=[u'a', u'b', u'c', u'd', u'e'], nGrams=[u'a b', u'b c', u'c d', u'd e']) >>> # Change n-gram length @@ -1779,6 +2198,18 @@ def getN(self): """ return self.getOrDefault(self.n) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @inherit_doc class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): @@ -1788,7 +2219,11 @@ class Normalizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Jav >>> from pyspark.ml.linalg import Vectors >>> svec = Vectors.sparse(4, {1: 4.0, 3: 3.0}) >>> df = spark.createDataFrame([(Vectors.dense([3.0, -4.0]), svec)], ["dense", "sparse"]) - >>> normalizer = Normalizer(p=2.0, inputCol="dense", outputCol="features") + >>> normalizer = Normalizer(p=2.0) + >>> normalizer.setInputCol("dense") + Normalizer... + >>> normalizer.setOutputCol("features") + Normalizer... >>> normalizer.transform(df).head().features DenseVector([0.6, -0.8]) >>> normalizer.setParams(inputCol="sparse", outputCol="freqs").transform(df).head().freqs @@ -1843,6 +2278,18 @@ def getP(self): """ return self.getOrDefault(self.p) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + class _OneHotEncoderParams(HasInputCols, HasOutputCols, HasHandleInvalid): """ @@ -1895,7 +2342,11 @@ class OneHotEncoder(JavaEstimator, _OneHotEncoderParams, JavaMLReadable, JavaMLW >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(0.0,), (1.0,), (2.0,)], ["input"]) - >>> ohe = OneHotEncoder(inputCols=["input"], outputCols=["output"]) + >>> ohe = OneHotEncoder() + >>> ohe.setInputCols(["input"]) + OneHotEncoder... + >>> ohe.setOutputCols(["output"]) + OneHotEncoder... >>> model = ohe.fit(df) >>> model.getHandleInvalid() 'error' @@ -1944,6 +2395,27 @@ def setDropLast(self, value): """ return self._set(dropLast=value) + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + + @since("3.0.0") + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + def _create_model(self, java_model): return OneHotEncoderModel(java_model) @@ -1955,6 +2427,34 @@ class OneHotEncoderModel(JavaModel, _OneHotEncoderParams, JavaMLReadable, JavaML .. versionadded:: 2.3.0 """ + @since("3.0.0") + def setDropLast(self, value): + """ + Sets the value of :py:attr:`dropLast`. + """ + return self._set(dropLast=value) + + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + + @since("3.0.0") + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + @property @since("2.3.0") def categorySizes(self): @@ -1977,7 +2477,11 @@ class PolynomialExpansion(JavaTransformer, HasInputCol, HasOutputCol, JavaMLRead >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(Vectors.dense([0.5, 2.0]),)], ["dense"]) - >>> px = PolynomialExpansion(degree=2, inputCol="dense", outputCol="expanded") + >>> px = PolynomialExpansion(degree=2) + >>> px.setInputCol("dense") + PolynomialExpansion... + >>> px.setOutputCol("expanded") + PolynomialExpansion... >>> px.transform(df).head().expanded DenseVector([0.5, 0.25, 2.0, 1.0, 4.0]) >>> px.setParams(outputCol="test").transform(df).head().test @@ -2030,6 +2534,18 @@ def getDegree(self): """ return self.getOrDefault(self.degree) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @inherit_doc class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, @@ -2060,8 +2576,13 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasInputCols >>> values = [(0.1,), (0.4,), (1.2,), (1.5,), (float("nan"),), (float("nan"),)] >>> df1 = spark.createDataFrame(values, ["values"]) - >>> qds1 = QuantileDiscretizer(numBuckets=2, - ... inputCol="values", outputCol="buckets", relativeError=0.01, handleInvalid="error") + >>> qds1 = QuantileDiscretizer(inputCol="values", outputCol="buckets") + >>> qds1.setNumBuckets(2) + QuantileDiscretizer... + >>> qds1.setRelativeError(0.01) + QuantileDiscretizer... + >>> qds1.setHandleInvalid("error") + QuantileDiscretizer... >>> qds1.getRelativeError() 0.01 >>> bucketizer = qds1.fit(df1) @@ -2213,6 +2734,38 @@ def getRelativeError(self): """ return self.getOrDefault(self.relativeError) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + def _create_model(self, java_model): """ Private method to convert the java_model to a Python model. @@ -2292,7 +2845,11 @@ class RobustScaler(JavaEstimator, _RobustScalerParams, JavaMLReadable, JavaMLWri ... (3, Vectors.dense([3.0, -3.0]),), ... (4, Vectors.dense([4.0, -4.0]),),] >>> df = spark.createDataFrame(data, ["id", "features"]) - >>> scaler = RobustScaler(inputCol="features", outputCol="scaled") + >>> scaler = RobustScaler() + >>> scaler.setInputCol("features") + RobustScaler... + >>> scaler.setOutputCol("scaled") + RobustScaler... >>> model = scaler.fit(df) >>> model.setOutputCol("output") RobustScaler... @@ -2373,6 +2930,20 @@ def setWithScaling(self, value): """ return self._set(withScaling=value) + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return RobustScalerModel(java_model) @@ -2384,6 +2955,20 @@ class RobustScalerModel(JavaModel, _RobustScalerParams, JavaMLReadable, JavaMLWr .. versionadded:: 3.0.0 """ + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @property @since("3.0.0") def median(self): @@ -2413,7 +2998,11 @@ class RegexTokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, It returns an array of strings that can be empty. >>> df = spark.createDataFrame([("A B c",)], ["text"]) - >>> reTokenizer = RegexTokenizer(inputCol="text", outputCol="words") + >>> reTokenizer = RegexTokenizer() + >>> reTokenizer.setInputCol("text") + RegexTokenizer... + >>> reTokenizer.setOutputCol("words") + RegexTokenizer... >>> reTokenizer.transform(df).head() Row(text=u'A B c', words=[u'a', u'b', u'c']) >>> # Change a parameter. @@ -2530,6 +3119,18 @@ def getToLowercase(self): """ return self.getOrDefault(self.toLowercase) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @inherit_doc class SQLTransformer(JavaTransformer, JavaMLReadable, JavaMLWritable): @@ -2629,7 +3230,11 @@ class StandardScaler(JavaEstimator, _StandardScalerParams, JavaMLReadable, JavaM >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"]) - >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled") + >>> standardScaler = StandardScaler() + >>> standardScaler.setInputCol("a") + StandardScaler... + >>> standardScaler.setOutputCol("scaled") + StandardScaler... >>> model = standardScaler.fit(df) >>> model.getInputCol() 'a' @@ -2694,6 +3299,18 @@ def setWithStd(self, value): """ return self._set(withStd=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return StandardScalerModel(java_model) @@ -2705,6 +3322,18 @@ class StandardScalerModel(JavaModel, _StandardScalerParams, JavaMLReadable, Java .. versionadded:: 1.4.0 """ + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @property @since("2.0.0") def std(self): @@ -2765,8 +3394,10 @@ class StringIndexer(JavaEstimator, _StringIndexerParams, JavaMLReadable, JavaMLW so the most frequent label gets index 0. The ordering behavior is controlled by setting :py:attr:`stringOrderType`. Its default value is 'frequencyDesc'. - >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed", handleInvalid="error", + >>> stringIndexer = StringIndexer(inputCol="label", outputCol="indexed", ... stringOrderType="frequencyDesc") + >>> stringIndexer.setHandleInvalid("error") + StringIndexer... >>> model = stringIndexer.fit(stringIndDf) >>> td = model.transform(stringIndDf) >>> sorted(set([(i[0], i[1]) for i in td.select(td.id, td.indexed).collect()]), @@ -2866,6 +3497,38 @@ def setStringOrderType(self, value): """ return self._set(stringOrderType=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + class StringIndexerModel(JavaModel, _StringIndexerParams, JavaMLReadable, JavaMLWritable): """ @@ -2874,6 +3537,39 @@ class StringIndexerModel(JavaModel, _StringIndexerParams, JavaMLReadable, JavaML .. versionadded:: 1.4.0 """ + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + + @since("2.4.0") + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + @classmethod @since("2.4.0") def from_labels(cls, labels, inputCol, outputCol=None, handleInvalid=None): @@ -2921,13 +3617,6 @@ def labels(self): """ return self._call_java("labels") - @since("2.4.0") - def setHandleInvalid(self, value): - """ - Sets the value of :py:attr:`handleInvalid`. - """ - return self._set(handleInvalid=value) - @inherit_doc class IndexToString(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): @@ -2981,6 +3670,18 @@ def getLabels(self): """ return self.getOrDefault(self.labels) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): """ @@ -2989,7 +3690,11 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadabl .. note:: null values from input array are preserved unless adding null to stopWords explicitly. >>> df = spark.createDataFrame([(["a", "b", "c"],)], ["text"]) - >>> remover = StopWordsRemover(inputCol="text", outputCol="words", stopWords=["b"]) + >>> remover = StopWordsRemover(stopWords=["b"]) + >>> remover.setInputCol("text") + StopWordsRemover... + >>> remover.setOutputCol("words") + StopWordsRemover... >>> remover.transform(df).head().words == ['a', 'c'] True >>> stopWordsRemoverPath = temp_path + "/stopwords-remover" @@ -3079,6 +3784,18 @@ def getLocale(self): """ return self.getOrDefault(self.locale) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @staticmethod @since("2.0.0") def loadDefaultStopWords(language): @@ -3099,7 +3816,9 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, Java splits it by white spaces. >>> df = spark.createDataFrame([("a b c",)], ["text"]) - >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") + >>> tokenizer = Tokenizer(outputCol="words") + >>> tokenizer.setInputCol("text") + Tokenizer... >>> tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) >>> # Change a parameter. @@ -3144,6 +3863,18 @@ def setParams(self, inputCol=None, outputCol=None): kwargs = self._input_kwargs return self._set(**kwargs) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @inherit_doc class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol, HasHandleInvalid, JavaMLReadable, @@ -3152,7 +3883,9 @@ class VectorAssembler(JavaTransformer, HasInputCols, HasOutputCol, HasHandleInva A feature transformer that merges multiple columns into a vector column. >>> df = spark.createDataFrame([(1, 0, 3)], ["a", "b", "c"]) - >>> vecAssembler = VectorAssembler(inputCols=["a", "b", "c"], outputCol="features") + >>> vecAssembler = VectorAssembler(outputCol="features") + >>> vecAssembler.setInputCols(["a", "b", "c"]) + VectorAssembler... >>> vecAssembler.transform(df).head().features DenseVector([1.0, 0.0, 3.0]) >>> vecAssembler.setParams(outputCol="freqs").transform(df).head().freqs @@ -3220,6 +3953,24 @@ def setParams(self, inputCols=None, outputCol=None, handleInvalid="error"): kwargs = self._input_kwargs return self._set(**kwargs) + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + class _VectorIndexerParams(HasInputCol, HasOutputCol, HasHandleInvalid): """ @@ -3288,7 +4039,9 @@ class VectorIndexer(JavaEstimator, _VectorIndexerParams, JavaMLReadable, JavaMLW >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(Vectors.dense([-1.0, 0.0]),), ... (Vectors.dense([0.0, 1.0]),), (Vectors.dense([0.0, 2.0]),)], ["a"]) - >>> indexer = VectorIndexer(maxCategories=2, inputCol="a", outputCol="indexed") + >>> indexer = VectorIndexer(maxCategories=2, inputCol="a") + >>> indexer.setOutputCol("indexed") + VectorIndexer... >>> model = indexer.fit(df) >>> indexer.getHandleInvalid() 'error' @@ -3359,6 +4112,24 @@ def setMaxCategories(self, value): """ return self._set(maxCategories=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + def _create_model(self, java_model): return VectorIndexerModel(java_model) @@ -3380,6 +4151,20 @@ class VectorIndexerModel(JavaModel, _VectorIndexerParams, JavaMLReadable, JavaML .. versionadded:: 1.4.0 """ + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @property @since("1.4.0") def numFeatures(self): @@ -3417,7 +4202,9 @@ class VectorSlicer(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, J ... (Vectors.dense([-2.0, 2.3, 0.0, 0.0, 1.0]),), ... (Vectors.dense([0.0, 0.0, 0.0, 0.0, 0.0]),), ... (Vectors.dense([0.6, -1.1, -3.0, 4.5, 3.3]),)], ["features"]) - >>> vs = VectorSlicer(inputCol="features", outputCol="sliced", indices=[1, 4]) + >>> vs = VectorSlicer(outputCol="sliced", indices=[1, 4]) + >>> vs.setInputCol("features") + VectorSlicer... >>> vs.transform(df).head().sliced DenseVector([2.3, 1.0]) >>> vectorSlicerPath = temp_path + "/vector-slicer" @@ -3488,6 +4275,18 @@ def getNames(self): """ return self.getOrDefault(self.names) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + class _Word2VecParams(HasStepSize, HasMaxIter, HasSeed, HasInputCol, HasOutputCol): """ @@ -3560,6 +4359,11 @@ class Word2Vec(JavaEstimator, _Word2VecParams, JavaMLReadable, JavaMLWritable): >>> sent = ("a b " * 100 + "a c " * 10).split(" ") >>> doc = spark.createDataFrame([(sent,), (sent,)], ["sentence"]) >>> word2Vec = Word2Vec(vectorSize=5, seed=42, inputCol="sentence", outputCol="model") + >>> word2Vec.setMaxIter(10) + Word2Vec... + >>> word2Vec.getMaxIter() + 10 + >>> word2Vec.clear(word2Vec.maxIter) >>> model = word2Vec.fit(doc) >>> model.getMinCount() 5 @@ -3666,12 +4470,36 @@ def setMaxSentenceLength(self, value): """ return self._set(maxSentenceLength=value) - @since("2.0.0") - def getMaxSentenceLength(self): + def setMaxIter(self, value): """ - Gets the value of maxSentenceLength or its default value. + Sets the value of :py:attr:`maxIter`. """ - return self.getOrDefault(self.maxSentenceLength) + return self._set(maxIter=value) + + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + @since("1.4.0") + def setStepSize(self, value): + """ + Sets the value of :py:attr:`stepSize`. + """ + return self._set(stepSize=value) def _create_model(self, java_model): return Word2VecModel(java_model) @@ -3692,6 +4520,18 @@ def getVectors(self): """ return self._call_java("getVectors") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @since("1.5.0") def findSynonyms(self, word, num): """ @@ -3747,7 +4587,9 @@ class PCA(JavaEstimator, _PCAParams, JavaMLReadable, JavaMLWritable): ... (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), ... (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] >>> df = spark.createDataFrame(data,["features"]) - >>> pca = PCA(k=2, inputCol="features", outputCol="pca_features") + >>> pca = PCA(k=2, inputCol="features") + >>> pca.setOutputCol("pca_features") + PCA... >>> model = pca.fit(df) >>> model.getK() 2 @@ -3800,6 +4642,18 @@ def setK(self, value): """ return self._set(k=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return PCAModel(java_model) @@ -3811,6 +4665,20 @@ class PCAModel(JavaModel, _PCAParams, JavaMLReadable, JavaMLWritable): .. versionadded:: 1.5.0 """ + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @property @since("2.0.0") def pc(self): @@ -4001,6 +4869,24 @@ def setStringIndexerOrderType(self, value): """ return self._set(stringIndexerOrderType=value) + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + def _create_model(self, java_model): return RFormulaModel(java_model) @@ -4228,6 +5114,24 @@ def setFwe(self, value): """ return self._set(fwe=value) + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + def _create_model(self, java_model): return ChiSqSelectorModel(java_model) @@ -4239,6 +5143,20 @@ class ChiSqSelectorModel(JavaModel, _ChiSqSelectorParams, JavaMLReadable, JavaML .. versionadded:: 2.0.0 """ + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @property @since("2.0.0") def selectedFeatures(self): @@ -4323,6 +5241,18 @@ def setSize(self, value): """ Sets size param, the size of vectors in `inputCol`.""" return self._set(size=value) + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + def setHandleInvalid(self, value): + """ + Sets the value of :py:attr:`handleInvalid`. + """ + return self._set(handleInvalid=value) + if __name__ == "__main__": import doctest diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index 652acbb34a901..5b34d555484d1 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -102,6 +102,13 @@ def setMinConfidence(self, value): """ return self._set(minConfidence=value) + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + @property @since("2.2.0") def freqItemsets(self): @@ -239,6 +246,12 @@ def setMinConfidence(self, value): """ return self._set(minConfidence=value) + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + def _create_model(self, java_model): return FPGrowthModel(java_model) diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index c99ec3f467ac6..8ea94e4760007 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -81,12 +81,6 @@ def _gen_param_code(name, doc, defaultValueStr): """ # TODO: How to correctly inherit instance attributes? template = ''' - def set$Name(self, value): - """ - Sets the value of :py:attr:`$name`. - """ - return self._set($name=value) - def get$Name(self): """ Gets the value of $name or its default value. diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 771b4bcd9ba02..26d74fab6975a 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -30,12 +30,6 @@ class HasMaxIter(Params): def __init__(self): super(HasMaxIter, self).__init__() - def setMaxIter(self, value): - """ - Sets the value of :py:attr:`maxIter`. - """ - return self._set(maxIter=value) - def getMaxIter(self): """ Gets the value of maxIter or its default value. @@ -53,12 +47,6 @@ class HasRegParam(Params): def __init__(self): super(HasRegParam, self).__init__() - def setRegParam(self, value): - """ - Sets the value of :py:attr:`regParam`. - """ - return self._set(regParam=value) - def getRegParam(self): """ Gets the value of regParam or its default value. @@ -77,12 +65,6 @@ def __init__(self): super(HasFeaturesCol, self).__init__() self._setDefault(featuresCol='features') - def setFeaturesCol(self, value): - """ - Sets the value of :py:attr:`featuresCol`. - """ - return self._set(featuresCol=value) - def getFeaturesCol(self): """ Gets the value of featuresCol or its default value. @@ -101,12 +83,6 @@ def __init__(self): super(HasLabelCol, self).__init__() self._setDefault(labelCol='label') - def setLabelCol(self, value): - """ - Sets the value of :py:attr:`labelCol`. - """ - return self._set(labelCol=value) - def getLabelCol(self): """ Gets the value of labelCol or its default value. @@ -125,12 +101,6 @@ def __init__(self): super(HasPredictionCol, self).__init__() self._setDefault(predictionCol='prediction') - def setPredictionCol(self, value): - """ - Sets the value of :py:attr:`predictionCol`. - """ - return self._set(predictionCol=value) - def getPredictionCol(self): """ Gets the value of predictionCol or its default value. @@ -149,12 +119,6 @@ def __init__(self): super(HasProbabilityCol, self).__init__() self._setDefault(probabilityCol='probability') - def setProbabilityCol(self, value): - """ - Sets the value of :py:attr:`probabilityCol`. - """ - return self._set(probabilityCol=value) - def getProbabilityCol(self): """ Gets the value of probabilityCol or its default value. @@ -173,12 +137,6 @@ def __init__(self): super(HasRawPredictionCol, self).__init__() self._setDefault(rawPredictionCol='rawPrediction') - def setRawPredictionCol(self, value): - """ - Sets the value of :py:attr:`rawPredictionCol`. - """ - return self._set(rawPredictionCol=value) - def getRawPredictionCol(self): """ Gets the value of rawPredictionCol or its default value. @@ -196,12 +154,6 @@ class HasInputCol(Params): def __init__(self): super(HasInputCol, self).__init__() - def setInputCol(self, value): - """ - Sets the value of :py:attr:`inputCol`. - """ - return self._set(inputCol=value) - def getInputCol(self): """ Gets the value of inputCol or its default value. @@ -219,12 +171,6 @@ class HasInputCols(Params): def __init__(self): super(HasInputCols, self).__init__() - def setInputCols(self, value): - """ - Sets the value of :py:attr:`inputCols`. - """ - return self._set(inputCols=value) - def getInputCols(self): """ Gets the value of inputCols or its default value. @@ -243,12 +189,6 @@ def __init__(self): super(HasOutputCol, self).__init__() self._setDefault(outputCol=self.uid + '__output') - def setOutputCol(self, value): - """ - Sets the value of :py:attr:`outputCol`. - """ - return self._set(outputCol=value) - def getOutputCol(self): """ Gets the value of outputCol or its default value. @@ -266,12 +206,6 @@ class HasOutputCols(Params): def __init__(self): super(HasOutputCols, self).__init__() - def setOutputCols(self, value): - """ - Sets the value of :py:attr:`outputCols`. - """ - return self._set(outputCols=value) - def getOutputCols(self): """ Gets the value of outputCols or its default value. @@ -290,12 +224,6 @@ def __init__(self): super(HasNumFeatures, self).__init__() self._setDefault(numFeatures=262144) - def setNumFeatures(self, value): - """ - Sets the value of :py:attr:`numFeatures`. - """ - return self._set(numFeatures=value) - def getNumFeatures(self): """ Gets the value of numFeatures or its default value. @@ -313,12 +241,6 @@ class HasCheckpointInterval(Params): def __init__(self): super(HasCheckpointInterval, self).__init__() - def setCheckpointInterval(self, value): - """ - Sets the value of :py:attr:`checkpointInterval`. - """ - return self._set(checkpointInterval=value) - def getCheckpointInterval(self): """ Gets the value of checkpointInterval or its default value. @@ -337,12 +259,6 @@ def __init__(self): super(HasSeed, self).__init__() self._setDefault(seed=hash(type(self).__name__)) - def setSeed(self, value): - """ - Sets the value of :py:attr:`seed`. - """ - return self._set(seed=value) - def getSeed(self): """ Gets the value of seed or its default value. @@ -360,12 +276,6 @@ class HasTol(Params): def __init__(self): super(HasTol, self).__init__() - def setTol(self, value): - """ - Sets the value of :py:attr:`tol`. - """ - return self._set(tol=value) - def getTol(self): """ Gets the value of tol or its default value. @@ -383,12 +293,6 @@ class HasStepSize(Params): def __init__(self): super(HasStepSize, self).__init__() - def setStepSize(self, value): - """ - Sets the value of :py:attr:`stepSize`. - """ - return self._set(stepSize=value) - def getStepSize(self): """ Gets the value of stepSize or its default value. @@ -406,12 +310,6 @@ class HasHandleInvalid(Params): def __init__(self): super(HasHandleInvalid, self).__init__() - def setHandleInvalid(self, value): - """ - Sets the value of :py:attr:`handleInvalid`. - """ - return self._set(handleInvalid=value) - def getHandleInvalid(self): """ Gets the value of handleInvalid or its default value. @@ -430,12 +328,6 @@ def __init__(self): super(HasElasticNetParam, self).__init__() self._setDefault(elasticNetParam=0.0) - def setElasticNetParam(self, value): - """ - Sets the value of :py:attr:`elasticNetParam`. - """ - return self._set(elasticNetParam=value) - def getElasticNetParam(self): """ Gets the value of elasticNetParam or its default value. @@ -454,12 +346,6 @@ def __init__(self): super(HasFitIntercept, self).__init__() self._setDefault(fitIntercept=True) - def setFitIntercept(self, value): - """ - Sets the value of :py:attr:`fitIntercept`. - """ - return self._set(fitIntercept=value) - def getFitIntercept(self): """ Gets the value of fitIntercept or its default value. @@ -478,12 +364,6 @@ def __init__(self): super(HasStandardization, self).__init__() self._setDefault(standardization=True) - def setStandardization(self, value): - """ - Sets the value of :py:attr:`standardization`. - """ - return self._set(standardization=value) - def getStandardization(self): """ Gets the value of standardization or its default value. @@ -501,12 +381,6 @@ class HasThresholds(Params): def __init__(self): super(HasThresholds, self).__init__() - def setThresholds(self, value): - """ - Sets the value of :py:attr:`thresholds`. - """ - return self._set(thresholds=value) - def getThresholds(self): """ Gets the value of thresholds or its default value. @@ -525,12 +399,6 @@ def __init__(self): super(HasThreshold, self).__init__() self._setDefault(threshold=0.5) - def setThreshold(self, value): - """ - Sets the value of :py:attr:`threshold`. - """ - return self._set(threshold=value) - def getThreshold(self): """ Gets the value of threshold or its default value. @@ -548,12 +416,6 @@ class HasWeightCol(Params): def __init__(self): super(HasWeightCol, self).__init__() - def setWeightCol(self, value): - """ - Sets the value of :py:attr:`weightCol`. - """ - return self._set(weightCol=value) - def getWeightCol(self): """ Gets the value of weightCol or its default value. @@ -572,12 +434,6 @@ def __init__(self): super(HasSolver, self).__init__() self._setDefault(solver='auto') - def setSolver(self, value): - """ - Sets the value of :py:attr:`solver`. - """ - return self._set(solver=value) - def getSolver(self): """ Gets the value of solver or its default value. @@ -595,12 +451,6 @@ class HasVarianceCol(Params): def __init__(self): super(HasVarianceCol, self).__init__() - def setVarianceCol(self, value): - """ - Sets the value of :py:attr:`varianceCol`. - """ - return self._set(varianceCol=value) - def getVarianceCol(self): """ Gets the value of varianceCol or its default value. @@ -619,12 +469,6 @@ def __init__(self): super(HasAggregationDepth, self).__init__() self._setDefault(aggregationDepth=2) - def setAggregationDepth(self, value): - """ - Sets the value of :py:attr:`aggregationDepth`. - """ - return self._set(aggregationDepth=value) - def getAggregationDepth(self): """ Gets the value of aggregationDepth or its default value. @@ -643,12 +487,6 @@ def __init__(self): super(HasParallelism, self).__init__() self._setDefault(parallelism=1) - def setParallelism(self, value): - """ - Sets the value of :py:attr:`parallelism`. - """ - return self._set(parallelism=value) - def getParallelism(self): """ Gets the value of parallelism or its default value. @@ -667,12 +505,6 @@ def __init__(self): super(HasCollectSubModels, self).__init__() self._setDefault(collectSubModels=False) - def setCollectSubModels(self, value): - """ - Sets the value of :py:attr:`collectSubModels`. - """ - return self._set(collectSubModels=value) - def getCollectSubModels(self): """ Gets the value of collectSubModels or its default value. @@ -690,12 +522,6 @@ class HasLoss(Params): def __init__(self): super(HasLoss, self).__init__() - def setLoss(self, value): - """ - Sets the value of :py:attr:`loss`. - """ - return self._set(loss=value) - def getLoss(self): """ Gets the value of loss or its default value. @@ -714,12 +540,6 @@ def __init__(self): super(HasDistanceMeasure, self).__init__() self._setDefault(distanceMeasure='euclidean') - def setDistanceMeasure(self, value): - """ - Sets the value of :py:attr:`distanceMeasure`. - """ - return self._set(distanceMeasure=value) - def getDistanceMeasure(self): """ Gets the value of distanceMeasure or its default value. @@ -737,12 +557,6 @@ class HasValidationIndicatorCol(Params): def __init__(self): super(HasValidationIndicatorCol, self).__init__() - def setValidationIndicatorCol(self, value): - """ - Sets the value of :py:attr:`validationIndicatorCol`. - """ - return self._set(validationIndicatorCol=value) - def getValidationIndicatorCol(self): """ Gets the value of validationIndicatorCol or its default value. diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index df9c765457ec1..3ebd0ac2765f3 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -212,7 +212,16 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): >>> df = spark.createDataFrame( ... [(0, 0, 4.0), (0, 1, 2.0), (1, 1, 3.0), (1, 2, 4.0), (2, 1, 1.0), (2, 2, 5.0)], ... ["user", "item", "rating"]) - >>> als = ALS(rank=10, maxIter=5, seed=0) + >>> als = ALS(rank=10, seed=0) + >>> als.setMaxIter(5) + ALS... + >>> als.getMaxIter() + 5 + >>> als.setRegParam(0.1) + ALS... + >>> als.getRegParam() + 0.1 + >>> als.clear(als.regParam) >>> model = als.fit(df) >>> model.getUserCol() 'user' @@ -402,6 +411,36 @@ def setColdStartStrategy(self, value): """ return self._set(coldStartStrategy=value) + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + def setRegParam(self, value): + """ + Sets the value of :py:attr:`regParam`. + """ + return self._set(regParam=value) + + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + def setCheckpointInterval(self, value): + """ + Sets the value of :py:attr:`checkpointInterval`. + """ + return self._set(checkpointInterval=value) + + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + class ALSModel(JavaModel, _ALSModelParams, JavaMLWritable, JavaMLReadable): """ @@ -431,6 +470,13 @@ def setColdStartStrategy(self, value): """ return self._set(coldStartStrategy=value) + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + @property @since("1.4.0") def rank(self): diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 147ebed1d633a..08e68d8bc3044 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -92,7 +92,17 @@ class LinearRegression(JavaPredictor, _LinearRegressionParams, JavaMLWritable, J >>> df = spark.createDataFrame([ ... (1.0, 2.0, Vectors.dense(1.0)), ... (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"]) - >>> lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight") + >>> lr = LinearRegression(regParam=0.0, solver="normal", weightCol="weight") + >>> lr.setMaxIter(5) + LinearRegression... + >>> lr.getMaxIter() + 5 + >>> lr.setRegParam(0.1) + LinearRegression... + >>> lr.getRegParam() + 0.1 + >>> lr.setRegParam(0.0) + LinearRegression... >>> model = lr.fit(df) >>> model.setFeaturesCol("features") LinearRegression... @@ -179,6 +189,66 @@ def setEpsilon(self, value): """ return self._set(epsilon=value) + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + def setRegParam(self, value): + """ + Sets the value of :py:attr:`regParam`. + """ + return self._set(regParam=value) + + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + return self._set(tol=value) + + def setElasticNetParam(self, value): + """ + Sets the value of :py:attr:`elasticNetParam`. + """ + return self._set(elasticNetParam=value) + + def setFitIntercept(self, value): + """ + Sets the value of :py:attr:`fitIntercept`. + """ + return self._set(fitIntercept=value) + + def setStandardization(self, value): + """ + Sets the value of :py:attr:`standardization`. + """ + return self._set(standardization=value) + + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + def setSolver(self, value): + """ + Sets the value of :py:attr:`solver`. + """ + return self._set(solver=value) + + def setAggregationDepth(self, value): + """ + Sets the value of :py:attr:`aggregationDepth`. + """ + return self._set(aggregationDepth=value) + + def setLoss(self, value): + """ + Sets the value of :py:attr:`loss`. + """ + return self._set(lossType=value) + class LinearRegressionModel(JavaPredictionModel, _LinearRegressionParams, GeneralJavaMLWritable, JavaMLReadable, HasTrainingSummary): @@ -522,10 +592,6 @@ class IsotonicRegression(JavaEstimator, _IsotonicRegressionParams, HasWeightCol, >>> model = ir.fit(df) >>> model.setFeaturesCol("features") IsotonicRegression... - >>> model.setLabelCol("newLabel") - IsotonicRegression... - >>> model.getLabelCol() - 'newLabel' >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 @@ -586,6 +652,34 @@ def setFeatureIndex(self, value): """ return self._set(featureIndex=value) + @since("1.6.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("1.6.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("1.6.0") + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + @since("1.6.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + class IsotonicRegressionModel(JavaModel, _IsotonicRegressionParams, JavaMLWritable, JavaMLReadable): @@ -595,6 +689,26 @@ class IsotonicRegressionModel(JavaModel, _IsotonicRegressionParams, JavaMLWritab .. versionadded:: 1.6.0 """ + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + def setFeatureIndex(self, value): + """ + Sets the value of :py:attr:`featureIndex`. + """ + return self._set(featureIndex=value) + @property @since("1.6.0") def boundaries(self): @@ -635,7 +749,9 @@ class DecisionTreeRegressor(JavaPredictor, _DecisionTreeRegressorParams, JavaMLW >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) - >>> dt = DecisionTreeRegressor(maxDepth=2, varianceCol="variance") + >>> dt = DecisionTreeRegressor(maxDepth=2) + >>> dt.setVarianceCol("variance") + DecisionTreeRegressor... >>> model = dt.fit(df) >>> model.getVarianceCol() 'variance' @@ -732,18 +848,21 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return DecisionTreeRegressionModel(java_model) + @since("1.4.0") def setMaxDepth(self, value): """ Sets the value of :py:attr:`maxDepth`. """ return self._set(maxDepth=value) + @since("1.4.0") def setMaxBins(self, value): """ Sets the value of :py:attr:`maxBins`. """ return self._set(maxBins=value) + @since("1.4.0") def setMinInstancesPerNode(self, value): """ Sets the value of :py:attr:`minInstancesPerNode`. @@ -757,18 +876,21 @@ def setMinWeightFractionPerNode(self, value): """ return self._set(minWeightFractionPerNode=value) + @since("1.4.0") def setMinInfoGain(self, value): """ Sets the value of :py:attr:`minInfoGain`. """ return self._set(minInfoGain=value) + @since("1.4.0") def setMaxMemoryInMB(self, value): """ Sets the value of :py:attr:`maxMemoryInMB`. """ return self._set(maxMemoryInMB=value) + @since("1.4.0") def setCacheNodeIds(self, value): """ Sets the value of :py:attr:`cacheNodeIds`. @@ -782,6 +904,34 @@ def setImpurity(self, value): """ return self._set(impurity=value) + @since("1.4.0") + def setCheckpointInterval(self, value): + """ + Sets the value of :py:attr:`checkpointInterval`. + """ + return self._set(checkpointInterval=value) + + @since("1.4.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + @since("2.0.0") + def setVarianceCol(self, value): + """ + Sets the value of :py:attr:`varianceCol`. + """ + return self._set(varianceCol=value) + @inherit_doc class DecisionTreeRegressionModel(_DecisionTreeModel, _DecisionTreeRegressorParams, @@ -792,6 +942,13 @@ class DecisionTreeRegressionModel(_DecisionTreeModel, _DecisionTreeRegressorPara .. versionadded:: 1.4.0 """ + @since("3.0.0") + def setVarianceCol(self, value): + """ + Sets the value of :py:attr:`varianceCol`. + """ + return self._set(varianceCol=value) + @property @since("2.0.0") def featureImportances(self): @@ -836,7 +993,9 @@ class RandomForestRegressor(JavaPredictor, _RandomForestRegressorParams, JavaMLW >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) - >>> rf = RandomForestRegressor(numTrees=2, maxDepth=2, seed=42) + >>> rf = RandomForestRegressor(numTrees=2, maxDepth=2) + >>> rf.setSeed(42) + RandomForestRegressor... >>> model = rf.fit(df) >>> model.getSeed() 42 @@ -987,6 +1146,18 @@ def setFeatureSubsetStrategy(self, value): """ return self._set(featureSubsetStrategy=value) + def setCheckpointInterval(self, value): + """ + Sets the value of :py:attr:`checkpointInterval`. + """ + return self._set(checkpointInterval=value) + + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + class RandomForestRegressionModel(_TreeEnsembleModel, _RandomForestRegressorParams, JavaMLWritable, JavaMLReadable): @@ -1052,7 +1223,11 @@ class GBTRegressor(JavaPredictor, _GBTRegressorParams, JavaMLWritable, JavaMLRea >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) - >>> gbt = GBTRegressor(maxIter=5, maxDepth=2, seed=42, leafCol="leafId") + >>> gbt = GBTRegressor(maxDepth=2, seed=42, leafCol="leafId") + >>> gbt.setMaxIter(5) + GBTRegressor... + >>> gbt.getMaxIter() + 5 >>> print(gbt.getImpurity()) variance >>> print(gbt.getFeatureSubsetStrategy()) @@ -1152,36 +1327,42 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return GBTRegressionModel(java_model) + @since("1.4.0") def setMaxDepth(self, value): """ Sets the value of :py:attr:`maxDepth`. """ return self._set(maxDepth=value) + @since("1.4.0") def setMaxBins(self, value): """ Sets the value of :py:attr:`maxBins`. """ return self._set(maxBins=value) + @since("1.4.0") def setMinInstancesPerNode(self, value): """ Sets the value of :py:attr:`minInstancesPerNode`. """ return self._set(minInstancesPerNode=value) + @since("1.4.0") def setMinInfoGain(self, value): """ Sets the value of :py:attr:`minInfoGain`. """ return self._set(minInfoGain=value) + @since("1.4.0") def setMaxMemoryInMB(self, value): """ Sets the value of :py:attr:`maxMemoryInMB`. """ return self._set(maxMemoryInMB=value) + @since("1.4.0") def setCacheNodeIds(self, value): """ Sets the value of :py:attr:`cacheNodeIds`. @@ -1223,6 +1404,34 @@ def setValidationIndicatorCol(self, value): """ return self._set(validationIndicatorCol=value) + @since("1.4.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("1.4.0") + def setCheckpointInterval(self, value): + """ + Sets the value of :py:attr:`checkpointInterval`. + """ + return self._set(checkpointInterval=value) + + @since("1.4.0") + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + @since("1.4.0") + def setStepSize(self, value): + """ + Sets the value of :py:attr:`stepSize`. + """ + return self._set(stepSize=value) + class GBTRegressionModel(_TreeEnsembleModel, _GBTRegressorParams, JavaMLWritable, JavaMLReadable): """ @@ -1330,6 +1539,11 @@ class AFTSurvivalRegression(JavaEstimator, _AFTSurvivalRegressionParams, ... (1.0, Vectors.dense(1.0), 1.0), ... (1e-40, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) >>> aftsr = AFTSurvivalRegression() + >>> aftsr.setMaxIter(10) + AFTSurvivalRegression... + >>> aftsr.getMaxIter() + 10 + >>> aftsr.clear(aftsr.maxIter) >>> model = aftsr.fit(df) >>> model.setFeaturesCol("features") AFTSurvivalRegression... @@ -1422,6 +1636,55 @@ def setQuantilesCol(self, value): """ return self._set(quantilesCol=value) + @since("1.6.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("1.6.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("1.6.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("1.6.0") + def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ + return self._set(labelCol=value) + + @since("1.6.0") + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + return self._set(tol=value) + + @since("1.6.0") + def setFitIntercept(self, value): + """ + Sets the value of :py:attr:`fitIntercept`. + """ + return self._set(fitIntercept=value) + + @since("2.1.0") + def setAggregationDepth(self, value): + """ + Sets the value of :py:attr:`aggregationDepth`. + """ + return self._set(aggregationDepth=value) + class AFTSurvivalRegressionModel(JavaModel, _AFTSurvivalRegressionParams, JavaMLWritable, JavaMLReadable): @@ -1431,6 +1694,34 @@ class AFTSurvivalRegressionModel(JavaModel, _AFTSurvivalRegressionParams, .. versionadded:: 1.6.0 """ + @since("3.0.0") + def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ + return self._set(featuresCol=value) + + @since("3.0.0") + def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ + return self._set(predictionCol=value) + + @since("3.0.0") + def setQuantileProbabilities(self, value): + """ + Sets the value of :py:attr:`quantileProbabilities`. + """ + return self._set(quantileProbabilities=value) + + @since("3.0.0") + def setQuantilesCol(self, value): + """ + Sets the value of :py:attr:`quantilesCol`. + """ + return self._set(quantilesCol=value) + @property @since("2.0.0") def coefficients(self): @@ -1577,6 +1868,16 @@ class GeneralizedLinearRegression(JavaPredictor, _GeneralizedLinearRegressionPar ... (2.0, Vectors.dense(0.0, 0.0)), ... (2.0, Vectors.dense(1.0, 1.0)),], ["label", "features"]) >>> glr = GeneralizedLinearRegression(family="gaussian", link="identity", linkPredictionCol="p") + >>> glr.setRegParam(0.1) + GeneralizedLinearRegression... + >>> glr.getRegParam() + 0.1 + >>> glr.clear(glr.regParam) + >>> glr.setMaxIter(10) + GeneralizedLinearRegression... + >>> glr.getMaxIter() + 10 + >>> glr.clear(glr.maxIter) >>> model = glr.fit(df) >>> model.setFeaturesCol("features") GeneralizedLinearRegression... @@ -1690,6 +1991,48 @@ def setOffsetCol(self, value): """ return self._set(offsetCol=value) + @since("2.0.0") + def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ + return self._set(maxIter=value) + + @since("2.0.0") + def setRegParam(self, value): + """ + Sets the value of :py:attr:`regParam`. + """ + return self._set(regParam=value) + + @since("2.0.0") + def setTol(self, value): + """ + Sets the value of :py:attr:`tol`. + """ + return self._set(tol=value) + + @since("2.2.0") + def setFitIntercept(self, value): + """ + Sets the value of :py:attr:`fitIntercept`. + """ + return self._set(fitIntercept=value) + + @since("2.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + @since("2.0.0") + def setSolver(self, value): + """ + Sets the value of :py:attr:`solver`. + """ + return self._set(solver=value) + class GeneralizedLinearRegressionModel(JavaPredictionModel, _GeneralizedLinearRegressionParams, JavaMLWritable, JavaMLReadable, HasTrainingSummary): @@ -1699,6 +2042,13 @@ class GeneralizedLinearRegressionModel(JavaPredictionModel, _GeneralizedLinearRe .. versionadded:: 2.0.0 """ + @since("3.0.0") + def setLinkPredictionCol(self, value): + """ + Sets the value of :py:attr:`linkPredictionCol`. + """ + return self._set(linkPredictionCol=value) + @property @since("2.0.0") def coefficients(self): diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index 4c7f01484dc21..75cd903b5d6d7 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -221,13 +221,6 @@ def test_params(self): self.assertFalse(testParams.isSet(maxIter)) self.assertTrue(testParams.isDefined(maxIter)) self.assertEqual(testParams.getMaxIter(), 10) - testParams.setMaxIter(100) - self.assertTrue(testParams.isSet(maxIter)) - self.assertEqual(testParams.getMaxIter(), 100) - testParams.clear(maxIter) - self.assertFalse(testParams.isSet(maxIter)) - self.assertEqual(testParams.getMaxIter(), 10) - testParams.setMaxIter(100) self.assertTrue(testParams.hasParam(inputCol.name)) self.assertFalse(testParams.hasDefault(inputCol)) @@ -244,13 +237,12 @@ def test_params(self): # Since the default is normally random, set it to a known number for debug str testParams._setDefault(seed=41) - testParams.setSeed(43) self.assertEqual( testParams.explainParams(), "\n".join(["inputCol: input column name. (undefined)", - "maxIter: max number of iterations (>= 0). (default: 10, current: 100)", - "seed: random seed. (default: 41, current: 43)"])) + "maxIter: max number of iterations (>= 0). (default: 10)", + "seed: random seed. (default: 41)"])) def test_clear_param(self): df = self.spark.createDataFrame([(Vectors.dense([1.0]),), (Vectors.dense([2.0]),)], ["a"]) diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 8052163acd00a..16c376296c20d 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -299,6 +299,24 @@ def setNumFolds(self, value): """ return self._set(numFolds=value) + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + def setParallelism(self, value): + """ + Sets the value of :py:attr:`parallelism`. + """ + return self._set(parallelism=value) + + def setCollectSubModels(self, value): + """ + Sets the value of :py:attr:`collectSubModels`. + """ + return self._set(collectSubModels=value) + def _fit(self, dataset): est = self.getOrDefault(self.estimator) epm = self.getOrDefault(self.estimatorParamMaps) @@ -643,6 +661,24 @@ def setTrainRatio(self, value): """ return self._set(trainRatio=value) + def setSeed(self, value): + """ + Sets the value of :py:attr:`seed`. + """ + return self._set(seed=value) + + def setParallelism(self, value): + """ + Sets the value of :py:attr:`parallelism`. + """ + return self._set(parallelism=value) + + def setCollectSubModels(self, value): + """ + Sets the value of :py:attr:`collectSubModels`. + """ + return self._set(collectSubModels=value) + def _fit(self, dataset): est = self.getOrDefault(self.estimator) epm = self.getOrDefault(self.estimatorParamMaps) From ba9d1610b63be647011e8d8f88593485b1671851 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 27 Oct 2019 21:11:17 -0700 Subject: [PATCH 0441/1387] [SPARK-29617][BUILD] Upgrade to ORC 1.5.7 ### What changes were proposed in this pull request? This PR aims to upgrade to Apache ORC 1.5.7. ### Why are the changes needed? This will bring the latest bug fixes. The following is the full release note. - https://issues.apache.org/jira/projects/ORC/versions/12345702 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #26276 from dongjoon-hyun/SPARK-29617. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 6 +++--- dev/deps/spark-deps-hadoop-3.2 | 6 +++--- pom.xml | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index f21e76bf4331a..edf4f026a32dc 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -160,9 +160,9 @@ okapi-shade-0.4.2.jar okhttp-3.12.0.jar okio-1.15.0.jar opencsv-2.3.jar -orc-core-1.5.6-nohive.jar -orc-mapreduce-1.5.6-nohive.jar -orc-shims-1.5.6.jar +orc-core-1.5.7-nohive.jar +orc-mapreduce-1.5.7-nohive.jar +orc-shims-1.5.7.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 3ecc3c2b0d35a..5f32e4686dcee 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -193,9 +193,9 @@ okhttp-2.7.5.jar okhttp-3.12.0.jar okio-1.15.0.jar opencsv-2.3.jar -orc-core-1.5.6.jar -orc-mapreduce-1.5.6.jar -orc-shims-1.5.6.jar +orc-core-1.5.7.jar +orc-mapreduce-1.5.7.jar +orc-shims-1.5.7.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index f1a7cb3d106f1..0f3721495ce71 100644 --- a/pom.xml +++ b/pom.xml @@ -139,7 +139,7 @@ 2.3.1 10.12.1.1 1.10.1 - 1.5.6 + 1.5.7 nohive com.twitter 1.6.0 From e6c56937e26abeb268bd6a8cf21f32bd1468ddbe Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 27 Oct 2019 21:45:56 -0700 Subject: [PATCH 0442/1387] [SPARK-29008][SQL][FOLLOW-UP] Checks correct param lengths of split methods ### What changes were proposed in this pull request? This pr is to fix wrong code to check parameter lengths of split methods in `subexpressionEliminationForWholeStageCodegen`. ### Why are the changes needed? Bug fix. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26267 from maropu/SPARK-29008-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 946fc7f421ad6..cbf90959e0534 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1050,7 +1050,10 @@ class CodegenContext extends Logging { } val codes = if (commonExprVals.map(_.code.length).sum > SQLConf.get.methodSplitThreshold) { - if (commonExprs.map(calculateParamLength).forall(isValidParamLength)) { + val inputVarsForAllFuncs = commonExprs.map { expr => + getLocalInputVariableValues(this, expr.head).toSeq + } + if (inputVarsForAllFuncs.map(calculateParamLengthFromExprValues).forall(isValidParamLength)) { commonExprs.zipWithIndex.map { case (exprs, i) => val expr = exprs.head val eval = commonExprVals(i) @@ -1068,7 +1071,7 @@ class CodegenContext extends Logging { // Generate the code for this expression tree and wrap it in a function. val fnName = freshName("subExpr") - val inputVars = getLocalInputVariableValues(this, expr).toSeq + val inputVars = inputVarsForAllFuncs(i) val argList = inputVars.map(v => s"${v.javaType.getName} ${v.variableName}") val returnType = javaType(expr.dataType) val fn = From 50cf48489ad5728ecf2040afc06b46a9323e1ff8 Mon Sep 17 00:00:00 2001 From: jiake Date: Mon, 28 Oct 2019 14:23:53 +0800 Subject: [PATCH 0443/1387] [SPARK-28560][SQL][FOLLOWUP] change the local shuffle reader from leaf node to unary node ### What changes were proposed in this pull request? ### Why are the changes needed? When make the `LocalShuffleReaderExec` to leaf node, there exists a potential issue: the leaf node will hide the running query stage and make the unfinished query stage as finished query stage when creating its parent query stage. This PR make the leaf node to unary node. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests Closes #26250 from JkSelf/updateLeafNodeofLocalReaderToUnaryExecNode. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/SparkPlanInfo.scala | 1 - .../execution/adaptive/AdaptiveSparkPlanHelper.scala | 1 - .../adaptive/OptimizeLocalShuffleReader.scala | 4 ++-- .../adaptive/ReduceNumShufflePartitions.scala | 12 ++++++++---- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 459311df22d23..9351b074c6590 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -56,7 +56,6 @@ private[execution] object SparkPlanInfo { case ReusedSubqueryExec(child) => child :: Nil case a: AdaptiveSparkPlanExec => a.executedPlan :: Nil case stage: QueryStageExec => stage.plan :: Nil - case localReader: LocalShuffleReaderExec => localReader.child :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala index 94e66b0c3a430..0ec8710e4db43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala @@ -125,7 +125,6 @@ trait AdaptiveSparkPlanHelper { private def allChildren(p: SparkPlan): Seq[SparkPlan] = p match { case a: AdaptiveSparkPlanExec => Seq(a.executedPlan) case s: QueryStageExec => Seq(s.plan) - case l: LocalShuffleReaderExec => Seq(l.child) case _ => p.children } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 89e2813695a65..d8dd7224fef38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight} import org.apache.spark.sql.internal.SQLConf @@ -70,7 +70,7 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { } } -case class LocalShuffleReaderExec(child: QueryStageExec) extends LeafExecNode { +case class LocalShuffleReaderExec(child: QueryStageExec) extends UnaryExecNode { override def output: Seq[Attribute] = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 1a85d5c02075b..5a505c213a262 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ReusedQueryStageExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.adaptive.{LocalShuffleReaderExec, QueryStageExec, ReusedQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils @@ -64,10 +64,14 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { return plan } - val shuffleStages = plan.collect { - case stage: ShuffleQueryStageExec => stage - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage + def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { + case _: LocalShuffleReaderExec => Nil + case stage: ShuffleQueryStageExec => Seq(stage) + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => Seq(stage) + case _ => plan.children.flatMap(collectShuffleStages) } + + val shuffleStages = collectShuffleStages(plan) // ShuffleExchanges introduced by repartition do not support changing the number of partitions. // We change the number of partitions in the stage only if all the ShuffleExchanges support it. if (!shuffleStages.forall(_.plan.canChangeNumPartitions)) { From a8d5134981ef176965101e96beba71a9979a554f Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 28 Oct 2019 20:46:28 +0900 Subject: [PATCH 0444/1387] Revert "[SPARK-29528][BUILD][TEST-MAVEN] Upgrade scala-maven-plugin to 4.2.4 for Scala 2.13.1" This reverts commit 5fc363b307e4a032df87182d477ff63473d6d02d. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0f3721495ce71..b5c34790f1d3f 100644 --- a/pom.xml +++ b/pom.xml @@ -2241,7 +2241,7 @@ net.alchim31.maven scala-maven-plugin - 4.2.4 + 4.2.0 eclipse-add-source From 0182817ea3ac11742f158a5d04cc8322fd992d14 Mon Sep 17 00:00:00 2001 From: uncleGen Date: Mon, 28 Oct 2019 20:50:34 +0900 Subject: [PATCH 0445/1387] [SPARK-28158][SQL] Hive UDFs supports UDT type ## What changes were proposed in this pull request? After this PR, we can create and register Hive UDFs to accept UDT type, like `VectorUDT` and `MatrixUDT`. These UDTs are widely used in Spark machine learning. ## How was this patch tested? add new ut Closes #24961 from uncleGen/SPARK-28158. Authored-by: uncleGen Signed-off-by: HyukjinKwon --- .../spark/sql/hive/HiveInspectors.scala | 5 ++ .../spark/sql/hive/HiveInspectorSuite.scala | 8 +- .../sql/hive/HiveUserDefinedTypeSuite.scala | 73 +++++++++++++++++++ 3 files changed, 85 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 33b5bcefd853f..5b627b8164130 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -787,6 +787,9 @@ private[hive] trait HiveInspectors { ObjectInspectorFactory.getStandardStructObjectInspector( java.util.Arrays.asList(fields.map(f => f.name) : _*), java.util.Arrays.asList(fields.map(f => toInspector(f.dataType)) : _*)) + case _: UserDefinedType[_] => + val sqlType = dataType.asInstanceOf[UserDefinedType[_]].sqlType + toInspector(sqlType) } /** @@ -849,6 +852,8 @@ private[hive] trait HiveInspectors { } case Literal(_, dt: StructType) => toInspector(dt) + case Literal(_, dt: UserDefinedType[_]) => + toInspector(dt.sqlType) // We will enumerate all of the possible constant expressions, throw exception if we missed case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].") // ideally, we don't test the foldable here(but in optimizer), however, some of the diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index c300660458fdd..5912992694e84 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn import org.apache.hadoop.io.LongWritable import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Row, TestUserClassUDT} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} @@ -214,6 +214,12 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors { }) } + test("wrap / unwrap UDT Type") { + val dt = new TestUserClassUDT + checkValue(1, unwrap(wrap(1, toInspector(dt), dt), toInspector(dt))) + checkValue(null, unwrap(wrap(null, toInspector(dt), dt), toInspector(dt))) + } + test("wrap / unwrap Struct Type") { val dt = StructType(dataTypes.zipWithIndex.map { case (t, idx) => StructField(s"c_$idx", t) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala new file mode 100644 index 0000000000000..bddb7688fe96a --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StandardListObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory + +import org.apache.spark.sql.{QueryTest, RandomDataGenerator, Row} +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT} +import org.apache.spark.sql.types.StructType + +class HiveUserDefinedTypeSuite extends QueryTest with TestHiveSingleton { + private val functionClass = classOf[org.apache.spark.sql.hive.TestUDF].getCanonicalName + + test("Support UDT in Hive UDF") { + val functionName = "get_point_x" + try { + val schema = new StructType().add("point", new ExamplePointUDT) + val inputGenerator = RandomDataGenerator.forType(schema, nullable = false).get + val input = inputGenerator.apply().asInstanceOf[Row] + val df = spark.createDataFrame(Array(input).toList.asJava, schema) + df.createOrReplaceTempView("src") + spark.sql(s"CREATE FUNCTION $functionName AS '$functionClass'") + + checkAnswer( + spark.sql(s"SELECT $functionName(point) FROM src"), + Row(input.getAs[ExamplePoint](0).x)) + } finally { + // If the test failed part way, we don't want to mask the failure by failing to remove + // temp tables that never got created. + spark.sql(s"DROP FUNCTION IF EXISTS $functionName") + assert( + !spark.sessionState.catalog.functionExists(FunctionIdentifier(functionName)), + s"Function $functionName should have been dropped. But, it still exists.") + } + } +} + +class TestUDF extends GenericUDF { + private var data: StandardListObjectInspector = _ + + override def getDisplayString(children: Array[String]): String = "get_point_x" + + override def initialize(arguments: Array[ObjectInspector]): ObjectInspector = { + data = arguments(0).asInstanceOf[StandardListObjectInspector] + PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + } + + override def evaluate(arguments: Array[GenericUDF.DeferredObject]): AnyRef = { + val point = data.getList(arguments(0).get()) + new java.lang.Double(point.get(0).asInstanceOf[Double]) + } +} From d6e33dc3771f30e44f5e9af612b59f011b76fda2 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Mon, 28 Oct 2019 08:45:21 -0500 Subject: [PATCH 0446/1387] [SPARK-29599][WEBUI] Support pagination for session table in JDBC/ODBC Tab ### What changes were proposed in this pull request? In this PR, extend the support of pagination to session table in `JDBC/PDBC` . ### Why are the changes needed? Some times we may connect a lot client and there a many session info shown in session tab. make it can be paged for better view. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manuel verify. After pr: Screen Shot 2019-10-25 at 4 19 27 PM Screen Shot 2019-10-25 at 4 19 38 PM Closes #26253 from AngersZhuuuu/SPARK-29599. Lead-authored-by: angerszhu Co-authored-by: AngersZhuuuu Signed-off-by: Sean Owen --- .../thriftserver/ui/ThriftServerPage.scala | 215 +++++++++++++++--- 1 file changed, 186 insertions(+), 29 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 7258978e3bada..d224157098604 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -39,7 +39,6 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" private val listener = parent.listener private val startTime = Calendar.getInstance().getTime() - private val emptyCell = "-" /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { @@ -139,33 +138,60 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" /** Generate stats of batch sessions of the thrift server program */ private def generateSessionStatsTable(request: HttpServletRequest): Seq[Node] = { - val sessionList = listener.getSessionList - val numBatches = sessionList.size - val table = if (numBatches > 0) { - val dataRows = sessionList.sortBy(_.startTimestamp).reverse - val headerRow = Seq("User", "IP", "Session ID", "Start Time", "Finish Time", "Duration", - "Total Execute") - def generateDataRow(session: SessionInfo): Seq[Node] = { - val sessionLink = "%s/%s/session/?id=%s".format( - UIUtils.prependBaseUri(request, parent.basePath), parent.prefix, session.sessionId) - - {session.userName} - {session.ip} - {session.sessionId} - {formatDate(session.startTimestamp)} - {if (session.finishTimestamp > 0) formatDate(session.finishTimestamp)} - - {formatDurationOption(Some(session.totalTime))} - {session.totalExecution.toString} - + val numSessions = listener.getSessionList.size + val table = if (numSessions > 0) { + + val sessionTableTag = "sessionstat" + + val parameterOtherTable = request.getParameterMap().asScala + .filterNot(_._1.startsWith(sessionTableTag)) + .map { case (name, vals) => + name + "=" + vals(0) + } + + val parameterSessionTablePage = request.getParameter(s"$sessionTableTag.page") + val parameterSessionTableSortColumn = request.getParameter(s"$sessionTableTag.sort") + val parameterSessionTableSortDesc = request.getParameter(s"$sessionTableTag.desc") + val parameterSessionPageSize = request.getParameter(s"$sessionTableTag.pageSize") + + val sessionTablePage = Option(parameterSessionTablePage).map(_.toInt).getOrElse(1) + val sessionTableSortColumn = Option(parameterSessionTableSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("Start Time") + val sessionTableSortDesc = Option(parameterSessionTableSortDesc).map(_.toBoolean).getOrElse( + // New session should be shown above old session by default. + (sessionTableSortColumn == "Start Time") + ) + val sessionTablePageSize = Option(parameterSessionPageSize).map(_.toInt).getOrElse(100) + + try { + Some(new SessionStatsPagedTable( + request, + parent, + listener.getSessionList, + "sqlserver", + UIUtils.prependBaseUri(request, parent.basePath), + parameterOtherTable, + sessionTableTag, + pageSize = sessionTablePageSize, + sortColumn = sessionTableSortColumn, + desc = sessionTableSortDesc + ).table(sessionTablePage)) + } catch { + case e@(_: IllegalArgumentException | _: IndexOutOfBoundsException) => + Some(
    +

    Error while rendering job table:

    +
    +              {Utils.exceptionString(e)}
    +            
    +
    ) } - Some(UIUtils.listingTable(headerRow, generateDataRow, dataRows, true, None, Seq(null), false)) } else { None } val content = -
    Session Statistics ({numBatches})
    ++ +
    Session Statistics ({numSessions})
    ++
      {table.getOrElse("No statistics have been generated yet.")} @@ -174,13 +200,6 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" content } - - /** - * Returns a human-readable string representing a duration such as "5 second 35 ms" - */ - private def formatDurationOption(msOption: Option[Long]): String = { - msOption.map(formatDurationVerbose).getOrElse(emptyCell) - } } private[ui] class SqlStatsPagedTable( @@ -370,6 +389,103 @@ private[ui] class SqlStatsPagedTable( "%s/jobs/job/?id=%s".format(UIUtils.prependBaseUri(request, parent.basePath), jobId) } +private[ui] class SessionStatsPagedTable( + request: HttpServletRequest, + parent: ThriftServerTab, + data: Seq[SessionInfo], + subPath: String, + basePath: String, + parameterOtherTable: Iterable[String], + sessionStatsTableTag: String, + pageSize: Int, + sortColumn: String, + desc: Boolean) extends PagedTable[SessionInfo] { + + override val dataSource = new SessionStatsTableDataSource(data, pageSize, sortColumn, desc) + + private val parameterPath = s"$basePath/$subPath/?${parameterOtherTable.mkString("&")}" + + override def tableId: String = sessionStatsTableTag + + override def tableCssClass: String = + "table table-bordered table-condensed table-striped " + + "table-head-clickable table-cell-width-limited" + + override def pageLink(page: Int): String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + parameterPath + + s"&$pageNumberFormField=$page" + + s"&$sessionStatsTableTag.sort=$encodedSortColumn" + + s"&$sessionStatsTableTag.desc=$desc" + + s"&$pageSizeFormField=$pageSize" + } + + override def pageSizeFormField: String = s"$sessionStatsTableTag.pageSize" + + override def pageNumberFormField: String = s"$sessionStatsTableTag.page" + + override def goButtonFormPath: String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, UTF_8.name()) + s"$parameterPath&$sessionStatsTableTag.sort=$encodedSortColumn&$sessionStatsTableTag.desc=$desc" + } + + override def headers: Seq[Node] = { + val sessionTableHeaders = + Seq("User", "IP", "Session ID", "Start Time", "Finish Time", "Duration", "Total Execute") + + val colWidthAttr = s"${100.toDouble / sessionTableHeaders.size}%" + + val headerRow: Seq[Node] = { + sessionTableHeaders.map { header => + if (header == sortColumn) { + val headerLink = Unparsed( + parameterPath + + s"&$sessionStatsTableTag.sort=${URLEncoder.encode(header, UTF_8.name())}" + + s"&$sessionStatsTableTag.desc=${!desc}" + + s"&$sessionStatsTableTag.pageSize=$pageSize" + + s"#$sessionStatsTableTag") + val arrow = if (desc) "▾" else "▴" // UP or DOWN + + + + {header} {Unparsed(arrow)} + + + } else { + val headerLink = Unparsed( + parameterPath + + s"&$sessionStatsTableTag.sort=${URLEncoder.encode(header, UTF_8.name())}" + + s"&$sessionStatsTableTag.pageSize=$pageSize" + + s"#$sessionStatsTableTag") + + + + {header} + + + } + } + } + + {headerRow} + + } + + override def row(session: SessionInfo): Seq[Node] = { + val sessionLink = "%s/%s/session/?id=%s".format( + UIUtils.prependBaseUri(request, parent.basePath), parent.prefix, session.sessionId) + + {session.userName} + {session.ip} + {session.sessionId} + {formatDate(session.startTimestamp)} + {if (session.finishTimestamp > 0) formatDate(session.finishTimestamp)} + {formatDurationVerbose(session.totalTime)} + {session.totalExecution.toString} + + } +} + private[ui] class SqlStatsTableRow( val jobId: Seq[String], val duration: Long, @@ -434,3 +550,44 @@ private[ui] class SqlStatsPagedTable( } } + + private[ui] class SessionStatsTableDataSource( + info: Seq[SessionInfo], + pageSize: Int, + sortColumn: String, + desc: Boolean) extends PagedDataSource[SessionInfo](pageSize) { + + // Sorting SessionInfo data + private val data = info.sorted(ordering(sortColumn, desc)) + + private var _slicedStartTime: Set[Long] = null + + override def dataSize: Int = data.size + + override def sliceData(from: Int, to: Int): Seq[SessionInfo] = { + val r = data.slice(from, to) + _slicedStartTime = r.map(_.startTimestamp).toSet + r + } + + /** + * Return Ordering according to sortColumn and desc. + */ + private def ordering(sortColumn: String, desc: Boolean): Ordering[SessionInfo] = { + val ordering: Ordering[SessionInfo] = sortColumn match { + case "User" => Ordering.by(_.userName) + case "IP" => Ordering.by(_.ip) + case "Session ID" => Ordering.by(_.sessionId) + case "Start Time" => Ordering by (_.startTimestamp) + case "Finish Time" => Ordering.by(_.finishTimestamp) + case "Duration" => Ordering.by(_.totalTime) + case "Total Execute" => Ordering.by(_.totalExecution) + case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") + } + if (desc) { + ordering.reverse + } else { + ordering + } + } + } From 2be1fe6abcbdd3ead89f10a22ddc3e5c5e07e41a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 29 Oct 2019 00:41:20 +0800 Subject: [PATCH 0447/1387] [SPARK-29521][SQL] LOAD DATA INTO TABLE should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add LoadDataStatement and make LOAD DATA INTO TABLE go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog LOAD DATA INPATH 'filepath' INTO TABLE t // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? yes. When running LOAD DATA INTO TABLE, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26178 from viirya/SPARK-29521. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 19 +++++++++++++ .../catalyst/plans/logical/statements.scala | 10 +++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 27 +++++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 11 +++++++- .../spark/sql/execution/SparkSqlParser.scala | 19 ------------- .../sql/connector/DataSourceV2SQLSuite.scala | 18 +++++++++++++ .../execution/command/DDLParserSuite.scala | 25 ----------------- 8 files changed, 85 insertions(+), 46 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index c97eb3c935be6..c991e866256c3 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -214,7 +214,7 @@ statement | UNCACHE TABLE (IF EXISTS)? multipartIdentifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE - tableIdentifier partitionSpec? #loadData + multipartIdentifier partitionSpec? #loadData | TRUNCATE TABLE multipartIdentifier partitionSpec? #truncateTable | MSCK REPAIR TABLE multipartIdentifier #repairTable | op=(ADD | LIST) identifier .*? #manageResource diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 4fa479f083e10..a00b8d7655087 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2797,6 +2797,25 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging RepairTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) } + /** + * Create a [[LoadDataStatement]]. + * + * For example: + * {{{ + * LOAD DATA [LOCAL] INPATH 'filepath' [OVERWRITE] INTO TABLE multi_part_name + * [PARTITION (partcol1=val1, partcol2=val2 ...)] + * }}} + */ + override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) { + LoadDataStatement( + tableName = visitMultipartIdentifier(ctx.multipartIdentifier), + path = string(ctx.path), + isLocal = ctx.LOCAL != null, + isOverwrite = ctx.OVERWRITE != null, + partition = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) + ) + } + /** * Creates a [[ShowCreateTableStatement]] */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 655e87fce4e26..8dfaaae11ddd2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -330,6 +330,16 @@ case class AnalyzeColumnStatement( */ case class RepairTableStatement(tableName: Seq[String]) extends ParsedStatement +/** + * A LOAD DATA INTO TABLE statement, as parsed from SQL + */ +case class LoadDataStatement( + tableName: Seq[String], + path: String, + isLocal: Boolean, + isOverwrite: Boolean, + partition: Option[TablePartitionSpec]) extends ParsedStatement + /** * A SHOW CREATE TABLE statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index da01c612b350e..e85b15f82c574 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1045,6 +1045,33 @@ class DDLParserSuite extends AnalysisTest { RepairTableStatement(Seq("a", "b", "c"))) } + test("LOAD DATA INTO table") { + comparePlans( + parsePlan("LOAD DATA INPATH 'filepath' INTO TABLE a.b.c"), + LoadDataStatement(Seq("a", "b", "c"), "filepath", false, false, None)) + + comparePlans( + parsePlan("LOAD DATA LOCAL INPATH 'filepath' INTO TABLE a.b.c"), + LoadDataStatement(Seq("a", "b", "c"), "filepath", true, false, None)) + + comparePlans( + parsePlan("LOAD DATA LOCAL INPATH 'filepath' OVERWRITE INTO TABLE a.b.c"), + LoadDataStatement(Seq("a", "b", "c"), "filepath", true, true, None)) + + comparePlans( + parsePlan( + s""" + |LOAD DATA LOCAL INPATH 'filepath' OVERWRITE INTO TABLE a.b.c + |PARTITION(ds='2017-06-10') + """.stripMargin), + LoadDataStatement( + Seq("a", "b", "c"), + "filepath", + true, + true, + Some(Map("ds" -> "2017-06-10")))) + } + test("SHOW CREATE table") { comparePlans( parsePlan("SHOW CREATE TABLE a.b.c"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index e7e34b1ef3127..c55105d62af33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, LoadDataCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -299,6 +299,15 @@ class ResolveSessionCatalog( v1TableName.asTableIdentifier, "MSCK REPAIR TABLE") + case LoadDataStatement(tableName, path, isLocal, isOverwrite, partition) => + val v1TableName = parseV1Table(tableName, "LOAD DATA") + LoadDataCommand( + v1TableName.asTableIdentifier, + path, + isLocal, + isOverwrite, + partition) + case ShowCreateTableStatement(tableName) => val v1TableName = parseV1Table(tableName, "SHOW CREATE TABLE") ShowCreateTableCommand(v1TableName.asTableIdentifier) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 20894b39ce5d7..1f1d38bfa4cd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -273,25 +273,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } - /** - * Create a [[LoadDataCommand]] command. - * - * For example: - * {{{ - * LOAD DATA [LOCAL] INPATH 'filepath' [OVERWRITE] INTO TABLE tablename - * [PARTITION (partcol1=val1, partcol2=val2 ...)] - * }}} - */ - override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) { - LoadDataCommand( - table = visitTableIdentifier(ctx.tableIdentifier), - path = string(ctx.path), - isLocal = ctx.LOCAL != null, - isOverwrite = ctx.OVERWRITE != null, - partition = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) - ) - } - /** * Create an [[AlterDatabasePropertiesCommand]] command. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index b8a8acbba57cd..b9a542183d1bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1268,6 +1268,24 @@ class DataSourceV2SQLSuite } } + test("LOAD DATA INTO TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql( + s""" + |CREATE TABLE $t (id bigint, data string) + |USING foo + |PARTITIONED BY (id) + """.stripMargin) + + testV1Command("LOAD DATA", s"INPATH 'filepath' INTO TABLE $t") + testV1Command("LOAD DATA", s"LOCAL INPATH 'filepath' INTO TABLE $t") + testV1Command("LOAD DATA", s"LOCAL INPATH 'filepath' OVERWRITE INTO TABLE $t") + testV1Command("LOAD DATA", + s"LOCAL INPATH 'filepath' OVERWRITE INTO TABLE $t PARTITION(id=1)") + } + } + test("SHOW CREATE TABLE") { val t = "testcat.ns1.ns2.tbl" withTable(t) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index a9b94bea9517a..25ddc9b00a0b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -1410,29 +1410,4 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { assert(source4.table == "table2") assert(location4 == Some("/spark/warehouse")) } - - test("load data") { - val v1 = "LOAD DATA INPATH 'path' INTO TABLE table1" - val (table, path, isLocal, isOverwrite, partition) = parser.parsePlan(v1).collect { - case LoadDataCommand(t, path, l, o, partition) => (t, path, l, o, partition) - }.head - assert(table.database.isEmpty) - assert(table.table == "table1") - assert(path == "path") - assert(!isLocal) - assert(!isOverwrite) - assert(partition.isEmpty) - - val v2 = "LOAD DATA LOCAL INPATH 'path' OVERWRITE INTO TABLE table1 PARTITION(c='1', d='2')" - val (table2, path2, isLocal2, isOverwrite2, partition2) = parser.parsePlan(v2).collect { - case LoadDataCommand(t, path, l, o, partition) => (t, path, l, o, partition) - }.head - assert(table2.database.isEmpty) - assert(table2.table == "table1") - assert(path2 == "path") - assert(isLocal2) - assert(isOverwrite2) - assert(partition2.nonEmpty) - assert(partition2.get.apply("c") == "1" && partition2.get.apply("d") == "2") - } } From 762db39c15c0751fe6f415dc190f7adec3d840e9 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 28 Oct 2019 11:14:18 -0700 Subject: [PATCH 0448/1387] [SPARK-29509][SQL][SS] Deduplicate codes from Kafka data source ### What changes were proposed in this pull request? This patch deduplicates code blocks in Kafka data source which are being repeated multiple times in a method. ### Why are the changes needed? This change would simplify the code and open possibility to simplify future code whenever fields are added to Kafka writer schema. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing UTs. Closes #26158 from HeartSaVioR/MINOR-deduplicate-kafka-source. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../spark/sql/kafka010/KafkaWriteTask.scala | 69 +------ .../spark/sql/kafka010/KafkaWriter.scala | 104 +++++----- .../kafka010/KafkaContinuousSinkSuite.scala | 166 +++++++--------- .../spark/sql/kafka010/KafkaSinkSuite.scala | 179 ++++++------------ 4 files changed, 188 insertions(+), 330 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index 5bdc1b5fe9f37..8b907065af1d0 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -25,9 +25,9 @@ import org.apache.kafka.clients.producer.{Callback, KafkaProducer, ProducerRecor import org.apache.kafka.common.header.Header import org.apache.kafka.common.header.internals.RecordHeader -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} -import org.apache.spark.sql.types.{BinaryType, IntegerType, StringType} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, UnsafeProjection} +import org.apache.spark.sql.types.BinaryType /** * Writes out data in a single Spark task, without any concerns about how @@ -116,66 +116,13 @@ private[kafka010] abstract class KafkaRowWriter( } private def createProjection = { - val topicExpression = topic.map(Literal(_)).orElse { - inputSchema.find(_.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME) - }.getOrElse { - throw new IllegalStateException(s"topic option required when no " + - s"'${KafkaWriter.TOPIC_ATTRIBUTE_NAME}' attribute is present") - } - topicExpression.dataType match { - case StringType => // good - case t => - throw new IllegalStateException(s"${KafkaWriter.TOPIC_ATTRIBUTE_NAME} " + - s"attribute unsupported type $t. ${KafkaWriter.TOPIC_ATTRIBUTE_NAME} " + - s"must be a ${StringType.catalogString}") - } - val keyExpression = inputSchema.find(_.name == KafkaWriter.KEY_ATTRIBUTE_NAME) - .getOrElse(Literal(null, BinaryType)) - keyExpression.dataType match { - case StringType | BinaryType => // good - case t => - throw new IllegalStateException(s"${KafkaWriter.KEY_ATTRIBUTE_NAME} " + - s"attribute unsupported type ${t.catalogString}") - } - val valueExpression = inputSchema - .find(_.name == KafkaWriter.VALUE_ATTRIBUTE_NAME).getOrElse( - throw new IllegalStateException("Required attribute " + - s"'${KafkaWriter.VALUE_ATTRIBUTE_NAME}' not found") - ) - valueExpression.dataType match { - case StringType | BinaryType => // good - case t => - throw new IllegalStateException(s"${KafkaWriter.VALUE_ATTRIBUTE_NAME} " + - s"attribute unsupported type ${t.catalogString}") - } - val headersExpression = inputSchema - .find(_.name == KafkaWriter.HEADERS_ATTRIBUTE_NAME).getOrElse( - Literal(CatalystTypeConverters.convertToCatalyst(null), - KafkaRecordToRowConverter.headersType) - ) - headersExpression.dataType match { - case KafkaRecordToRowConverter.headersType => // good - case t => - throw new IllegalStateException(s"${KafkaWriter.HEADERS_ATTRIBUTE_NAME} " + - s"attribute unsupported type ${t.catalogString}") - } - val partitionExpression = - inputSchema.find(_.name == KafkaWriter.PARTITION_ATTRIBUTE_NAME) - .getOrElse(Literal(null, IntegerType)) - partitionExpression.dataType match { - case IntegerType => // good - case t => - throw new IllegalStateException(s"${KafkaWriter.PARTITION_ATTRIBUTE_NAME} " + - s"attribute unsupported type $t. ${KafkaWriter.PARTITION_ATTRIBUTE_NAME} " + - s"must be a ${IntegerType.catalogString}") - } UnsafeProjection.create( Seq( - topicExpression, - Cast(keyExpression, BinaryType), - Cast(valueExpression, BinaryType), - headersExpression, - partitionExpression + KafkaWriter.topicExpression(inputSchema, topic), + Cast(KafkaWriter.keyExpression(inputSchema), BinaryType), + Cast(KafkaWriter.valueExpression(inputSchema), BinaryType), + KafkaWriter.headersExpression(inputSchema), + KafkaWriter.partitionExpression(inputSchema) ), inputSchema ) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index 9b0d11f137ce2..5ef4b3a1c19d4 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.types.{BinaryType, IntegerType, MapType, StringType} +import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType, StringType} import org.apache.spark.util.Utils /** @@ -49,51 +49,14 @@ private[kafka010] object KafkaWriter extends Logging { schema: Seq[Attribute], kafkaParameters: ju.Map[String, Object], topic: Option[String] = None): Unit = { - schema.find(_.name == TOPIC_ATTRIBUTE_NAME).getOrElse( - if (topic.isEmpty) { - throw new AnalysisException(s"topic option required when no " + - s"'$TOPIC_ATTRIBUTE_NAME' attribute is present. Use the " + - s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a topic.") - } else { - Literal.create(topic.get, StringType) - } - ).dataType match { - case StringType => // good - case _ => - throw new AnalysisException(s"Topic type must be a ${StringType.catalogString}") - } - schema.find(_.name == KEY_ATTRIBUTE_NAME).getOrElse( - Literal(null, StringType) - ).dataType match { - case StringType | BinaryType => // good - case _ => - throw new AnalysisException(s"$KEY_ATTRIBUTE_NAME attribute type " + - s"must be a ${StringType.catalogString} or ${BinaryType.catalogString}") - } - schema.find(_.name == VALUE_ATTRIBUTE_NAME).getOrElse( - throw new AnalysisException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") - ).dataType match { - case StringType | BinaryType => // good - case _ => - throw new AnalysisException(s"$VALUE_ATTRIBUTE_NAME attribute type " + - s"must be a ${StringType.catalogString} or ${BinaryType.catalogString}") - } - schema.find(_.name == HEADERS_ATTRIBUTE_NAME).getOrElse( - Literal(CatalystTypeConverters.convertToCatalyst(null), - KafkaRecordToRowConverter.headersType) - ).dataType match { - case KafkaRecordToRowConverter.headersType => // good - case _ => - throw new AnalysisException(s"$HEADERS_ATTRIBUTE_NAME attribute type " + - s"must be a ${KafkaRecordToRowConverter.headersType.catalogString}") - } - schema.find(_.name == PARTITION_ATTRIBUTE_NAME).getOrElse( - Literal(null, IntegerType) - ).dataType match { - case IntegerType => // good - case _ => - throw new AnalysisException(s"$PARTITION_ATTRIBUTE_NAME attribute type " + - s"must be an ${IntegerType.catalogString}") + try { + topicExpression(schema, topic) + keyExpression(schema) + valueExpression(schema) + headersExpression(schema) + partitionExpression(schema) + } catch { + case e: IllegalStateException => throw new AnalysisException(e.getMessage) } } @@ -110,4 +73,53 @@ private[kafka010] object KafkaWriter extends Logging { finallyBlock = writeTask.close()) } } + + def topicExpression(schema: Seq[Attribute], topic: Option[String] = None): Expression = { + topic.map(Literal(_)).getOrElse( + expression(schema, TOPIC_ATTRIBUTE_NAME, Seq(StringType)) { + throw new IllegalStateException(s"topic option required when no " + + s"'${TOPIC_ATTRIBUTE_NAME}' attribute is present. Use the " + + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a topic.") + } + ) + } + + def keyExpression(schema: Seq[Attribute]): Expression = { + expression(schema, KEY_ATTRIBUTE_NAME, Seq(StringType, BinaryType)) { + Literal(null, BinaryType) + } + } + + def valueExpression(schema: Seq[Attribute]): Expression = { + expression(schema, VALUE_ATTRIBUTE_NAME, Seq(StringType, BinaryType)) { + throw new IllegalStateException(s"Required attribute '${VALUE_ATTRIBUTE_NAME}' not found") + } + } + + def headersExpression(schema: Seq[Attribute]): Expression = { + expression(schema, HEADERS_ATTRIBUTE_NAME, Seq(KafkaRecordToRowConverter.headersType)) { + Literal(CatalystTypeConverters.convertToCatalyst(null), + KafkaRecordToRowConverter.headersType) + } + } + + def partitionExpression(schema: Seq[Attribute]): Expression = { + expression(schema, PARTITION_ATTRIBUTE_NAME, Seq(IntegerType)) { + Literal(null, IntegerType) + } + } + + private def expression( + schema: Seq[Attribute], + attrName: String, + desired: Seq[DataType])( + default: => Expression): Expression = { + val expr = schema.find(_.name == attrName).getOrElse(default) + if (!desired.exists(_.sameType(expr.dataType))) { + throw new IllegalStateException(s"$attrName attribute unsupported type " + + s"${expr.dataType.catalogString}. $attrName must be a(n) " + + s"${desired.map(_.catalogString).mkString(" or ")}") + } + expr + } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala index cbf4952406c01..031f609cb92be 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala @@ -19,12 +19,15 @@ package org.apache.spark.sql.kafka010 import java.util.Locale +import scala.reflect.ClassTag + import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.serialization.ByteArraySerializer import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.{AnalysisException, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} +import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.streaming._ import org.apache.spark.sql.types.{BinaryType, DataType} import org.apache.spark.util.Utils @@ -192,24 +195,9 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { val topic = newTopic() testUtils.createTopic(topic) - /* No topic field or topic option */ - var writer: StreamingQuery = null - var ex: Exception = null - try { - writer = createKafkaWriter(input.toDF())( - withSelectExpr = "CAST(null as STRING) as topic", "value" - ) - testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - ex = writer.exception.get - } - } finally { - writer.stop() + runAndVerifyException[StreamingQueryException](inputTopic, "null topic present in the data.") { + createKafkaWriter(input.toDF())(withSelectExpr = "CAST(null as STRING) as topic", "value") } - assert(ex.getCause.getCause.getMessage - .toLowerCase(Locale.ROOT) - .contains("null topic present in the data.")) } test("streaming - write data with bad schema") { @@ -226,24 +214,10 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { val topic = newTopic() testUtils.createTopic(topic) - val ex = intercept[AnalysisException] { - /* No topic field or topic option */ - createKafkaWriter(input.toDF())( - withSelectExpr = "value as key", "value" - ) - } - assert(ex.getMessage - .toLowerCase(Locale.ROOT) - .contains("topic option required when no 'topic' attribute is present")) - - val ex2 = intercept[AnalysisException] { - /* No value field */ - createKafkaWriter(input.toDF())( - withSelectExpr = s"'$topic' as topic", "value as key" - ) - } - assert(ex2.getMessage.toLowerCase(Locale.ROOT).contains( - "required attribute 'value' not found")) + assertWrongSchema(topic, input, Seq("value as key", "value"), + "topic option required when no 'topic' attribute is present") + assertWrongSchema(topic, input, Seq(s"'$topic' as topic", "value as key"), + "required attribute 'value' not found") } test("streaming - write data with valid schema but wrong types") { @@ -258,43 +232,18 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { .option("startingOffsets", "earliest") .load() .selectExpr("CAST(value as STRING) value") + .toDF() val topic = newTopic() testUtils.createTopic(topic) - val ex = intercept[AnalysisException] { - /* topic field wrong type */ - createKafkaWriter(input.toDF())( - withSelectExpr = s"CAST('1' as INT) as topic", "value" - ) - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("topic type must be a string")) - - val ex2 = intercept[AnalysisException] { - /* value field wrong type */ - createKafkaWriter(input.toDF())( - withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as value" - ) - } - assert(ex2.getMessage.toLowerCase(Locale.ROOT).contains( - "value attribute type must be a string or binary")) - - val ex3 = intercept[AnalysisException] { - /* key field wrong type */ - createKafkaWriter(input.toDF())( - withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as key", "value" - ) - } - assert(ex3.getMessage.toLowerCase(Locale.ROOT).contains( - "key attribute type must be a string or binary")) - - val ex4 = intercept[AnalysisException] { - /* partition field wrong type */ - createKafkaWriter(input.toDF())( - withSelectExpr = s"'$topic' as topic", "value as partition", "value" - ) - } - assert(ex4.getMessage.toLowerCase(Locale.ROOT).contains( - "partition attribute type must be an int")) + assertWrongSchema(topic, input, Seq("CAST('1' as INT) as topic", "value"), + "topic must be a(n) string") + assertWrongSchema(topic, input, Seq(s"'$topic' as topic", "CAST(value as INT) as value"), + "value must be a(n) string or binary") + assertWrongSchema(topic, input, Seq(s"'$topic' as topic", "CAST(value as INT) as key", "value"), + "key must be a(n) string or binary") + assertWrongSchema(topic, input, Seq(s"'$topic' as topic", "value as partition", "value"), + "partition must be a(n) int") } test("streaming - write to non-existing topic") { @@ -310,21 +259,9 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { .load() val topic = newTopic() - var writer: StreamingQuery = null - var ex: Exception = null - try { - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter(input.toDF(), withTopic = Some(topic))() - testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - } - throw writer.exception.get - } - } finally { - writer.stop() + runAndVerifyException[StreamingQueryException](inputTopic, "job aborted") { + createKafkaWriter(input.toDF(), withTopic = Some(topic))() } - assert(ex.getCause.getCause.getMessage.toLowerCase(Locale.ROOT).contains("job aborted")) } test("streaming - exception on config serializer") { @@ -339,21 +276,10 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { .option("subscribe", inputTopic) .load() - val ex = intercept[IllegalArgumentException] { - createKafkaWriter( - input.toDF(), - withOptions = Map("kafka.key.serializer" -> "foo"))() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "kafka option 'key.serializer' is not supported")) - - val ex2 = intercept[IllegalArgumentException] { - createKafkaWriter( - input.toDF(), - withOptions = Map("kafka.value.serializer" -> "foo"))() - } - assert(ex2.getMessage.toLowerCase(Locale.ROOT).contains( - "kafka option 'value.serializer' is not supported")) + assertWrongOption(inputTopic, input.toDF(), Map("kafka.key.serializer" -> "foo"), + "kafka option 'key.serializer' is not supported") + assertWrongOption(inputTopic, input.toDF(), Map("kafka.value.serializer" -> "foo"), + "kafka option 'value.serializer' is not supported") } test("generic - write big data with small producer buffer") { @@ -422,4 +348,48 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { withOptions.foreach(opt => stream.option(opt._1, opt._2)) stream.start() } + + private def runAndVerifyException[T <: Exception : ClassTag]( + inputTopic: String, + expectErrorMsg: String)( + writerFn: => StreamingQuery): Unit = { + var writer: StreamingQuery = null + val ex: Exception = try { + intercept[T] { + writer = writerFn + testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) + eventually(timeout(streamingTimeout)) { + assert(writer.exception.isDefined) + } + throw writer.exception.get + } + } finally { + if (writer != null) writer.stop() + } + val rootException = ex match { + case e: StreamingQueryException => e.getCause.getCause + case e => e + } + assert(rootException.getMessage.toLowerCase(Locale.ROOT).contains(expectErrorMsg)) + } + + private def assertWrongSchema( + inputTopic: String, + input: DataFrame, + selectExpr: Seq[String], + expectErrorMsg: String): Unit = { + runAndVerifyException[AnalysisException](inputTopic, expectErrorMsg) { + createKafkaWriter(input)(withSelectExpr = selectExpr: _*) + } + } + + private def assertWrongOption( + inputTopic: String, + input: DataFrame, + options: Map[String, String], + expectErrorMsg: String): Unit = { + runAndVerifyException[IllegalArgumentException](inputTopic, expectErrorMsg) { + createKafkaWriter(input, withOptions = options)() + } + } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index aacb10f5197b0..1705d76de758d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -211,38 +211,10 @@ class KafkaSinkStreamingSuite extends KafkaSinkSuiteBase with StreamTest { val topic = newTopic() testUtils.createTopic(topic) - /* No topic field or topic option */ - var writer: StreamingQuery = null - var ex: Exception = null - try { - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter(input.toDF())( - withSelectExpr = "value as key", "value" - ) - input.addData("1", "2", "3", "4", "5") - writer.processAllAvailable() - } - } finally { - writer.stop() - } - assert(ex.getMessage - .toLowerCase(Locale.ROOT) - .contains("topic option required when no 'topic' attribute is present")) - - try { - /* No value field */ - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter(input.toDF())( - withSelectExpr = s"'$topic' as topic", "value as key" - ) - input.addData("1", "2", "3", "4", "5") - writer.processAllAvailable() - } - } finally { - writer.stop() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "required attribute 'value' not found")) + assertWrongSchema(input, Seq("value as key", "value"), + "topic option required when no 'topic' attribute is present") + assertWrongSchema(input, Seq(s"'$topic' as topic", "value as key"), + "required attribute 'value' not found") } test("streaming - write data with valid schema but wrong types") { @@ -250,109 +222,31 @@ class KafkaSinkStreamingSuite extends KafkaSinkSuiteBase with StreamTest { val topic = newTopic() testUtils.createTopic(topic) - var writer: StreamingQuery = null - var ex: Exception = null - try { - /* topic field wrong type */ - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter(input.toDF())( - withSelectExpr = s"CAST('1' as INT) as topic", "value" - ) - input.addData("1", "2", "3", "4", "5") - writer.processAllAvailable() - } - } finally { - writer.stop() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("topic type must be a string")) - - try { - /* value field wrong type */ - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter(input.toDF())( - withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as value" - ) - input.addData("1", "2", "3", "4", "5") - writer.processAllAvailable() - } - } finally { - writer.stop() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "value attribute type must be a string or binary")) - - try { - ex = intercept[StreamingQueryException] { - /* key field wrong type */ - writer = createKafkaWriter(input.toDF())( - withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as key", "value" - ) - input.addData("1", "2", "3", "4", "5") - writer.processAllAvailable() - } - } finally { - writer.stop() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "key attribute type must be a string or binary")) - - try { - ex = intercept[StreamingQueryException] { - /* partition field wrong type */ - writer = createKafkaWriter(input.toDF())( - withSelectExpr = s"'$topic' as topic", "value", "value as partition" - ) - input.addData("1", "2", "3", "4", "5") - writer.processAllAvailable() - } - } finally { - writer.stop() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "partition attribute type must be an int")) + assertWrongSchema(input, Seq("CAST('1' as INT) as topic", "value"), + "topic must be a(n) string") + assertWrongSchema(input, Seq(s"'$topic' as topic", "CAST(value as INT) as value"), + "value must be a(n) string or binary") + assertWrongSchema(input, Seq(s"'$topic' as topic", "CAST(value as INT) as key", "value"), + "key must be a(n) string or binary") + assertWrongSchema(input, Seq(s"'$topic' as topic", "value", "value as partition"), + "partition must be a(n) int") } test("streaming - write to non-existing topic") { val input = MemoryStream[String] - val topic = newTopic() - var writer: StreamingQuery = null - var ex: Exception = null - try { - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter(input.toDF(), withTopic = Some(topic))() - input.addData("1", "2", "3", "4", "5") - writer.processAllAvailable() - } - } finally { - writer.stop() + runAndVerifyStreamingQueryException(input, "job aborted") { + createKafkaWriter(input.toDF(), withTopic = Some(newTopic()))() } - assert(ex.getCause.getCause.getMessage.toLowerCase(Locale.ROOT).contains("job aborted")) } test("streaming - exception on config serializer") { val input = MemoryStream[String] - var writer: StreamingQuery = null - var ex: Exception = null - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter( - input.toDF(), - withOptions = Map("kafka.key.serializer" -> "foo"))() - input.addData("1") - writer.processAllAvailable() - } - assert(ex.getCause.getMessage.toLowerCase(Locale.ROOT).contains( - "kafka option 'key.serializer' is not supported")) - - ex = intercept[StreamingQueryException] { - writer = createKafkaWriter( - input.toDF(), - withOptions = Map("kafka.value.serializer" -> "foo"))() - input.addData("1") - writer.processAllAvailable() - } - assert(ex.getCause.getMessage.toLowerCase(Locale.ROOT).contains( - "kafka option 'value.serializer' is not supported")) + + assertWrongOption(input, Map("kafka.key.serializer" -> "foo"), + "kafka option 'key.serializer' is not supported") + assertWrongOption(input, Map("kafka.value.serializer" -> "foo"), + "kafka option 'value.serializer' is not supported") } private def createKafkaWriter( @@ -379,6 +273,41 @@ class KafkaSinkStreamingSuite extends KafkaSinkSuiteBase with StreamTest { } stream.start() } + + private def runAndVerifyStreamingQueryException( + input: MemoryStream[String], + expectErrorMsg: String)( + writerFn: => StreamingQuery): Unit = { + var writer: StreamingQuery = null + val ex: Exception = try { + intercept[StreamingQueryException] { + writer = writerFn + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + if (writer != null) writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains(expectErrorMsg)) + } + + private def assertWrongSchema( + input: MemoryStream[String], + selectExpr: Seq[String], + expectErrorMsg: String): Unit = { + runAndVerifyStreamingQueryException(input, expectErrorMsg) { + createKafkaWriter(input.toDF())(withSelectExpr = selectExpr: _*) + } + } + + private def assertWrongOption( + input: MemoryStream[String], + options: Map[String, String], + expectErrorMsg: String): Unit = { + runAndVerifyStreamingQueryException(input, expectErrorMsg) { + createKafkaWriter(input.toDF(), withOptions = options)() + } + } } abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { From 59db1f617a973028fd66395b72eaaa064f14a8bc Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 28 Oct 2019 15:00:22 -0700 Subject: [PATCH 0449/1387] [SPARK-29609][SQL] DataSourceV2: Support DROP NAMESPACE ### What changes were proposed in this pull request? This PR adds `DROP NAMESPACE` support for V2 catalogs. ### Why are the changes needed? Currently, you cannot drop namespaces for v2 catalogs. ### Does this PR introduce any user-facing change? The user can now perform the following: ```SQL CREATE NAMESPACE mycatalog.ns DROP NAMESPACE mycatalog.ns SHOW NAMESPACES IN mycatalog # Will show no namespaces ``` to drop a namespace `ns` inside `mycatalog` V2 catalog. ### How was this patch tested? Added unit tests. Closes #26262 from imback82/drop_namespace. Authored-by: Terry Kim Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../catalyst/analysis/ResolveCatalogs.scala | 3 ++ .../sql/catalyst/parser/AstBuilder.scala | 15 ++++++ .../catalyst/plans/logical/statements.scala | 8 +++ .../catalyst/plans/logical/v2Commands.scala | 9 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 22 ++++++++ .../analysis/ResolveSessionCatalog.scala | 9 +++- .../spark/sql/execution/SparkSqlParser.scala | 12 ----- .../datasources/v2/DataSourceV2Strategy.scala | 5 +- .../datasources/v2/DropNamespaceExec.scala | 53 +++++++++++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 42 ++++++++++++++- .../execution/command/DDLParserSuite.scala | 45 ---------------- 12 files changed, 165 insertions(+), 62 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index c991e866256c3..be96f5a927bc2 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -95,8 +95,8 @@ statement SET DBPROPERTIES tablePropertyList #setDatabaseProperties | ALTER database db=errorCapturingIdentifier SET locationSpec #setDatabaseLocation - | DROP database (IF EXISTS)? db=errorCapturingIdentifier - (RESTRICT | CASCADE)? #dropDatabase + | DROP (database | NAMESPACE) (IF EXISTS)? multipartIdentifier + (RESTRICT | CASCADE)? #dropNamespace | SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)? (LIKE? pattern=STRING)? #showNamespaces | createTableHeader ('(' colTypeList ')')? tableProvider diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 9803fda0678ff..7bf0e25158805 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -178,6 +178,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) c.ifNotExists, c.properties) + case DropNamespaceStatement(NonSessionCatalog(catalog, nameParts), ifExists, cascade) => + DropNamespace(catalog.asNamespaceCatalog, nameParts, ifExists, cascade) + case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index a00b8d7655087..3666eaeaa3d2a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2375,6 +2375,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging properties) } + /** + * Create a [[DropNamespaceStatement]] command. + * + * For example: + * {{{ + * DROP (DATABASE|SCHEMA|NAMESPACE) [IF EXISTS] ns1.ns2 [RESTRICT|CASCADE]; + * }}} + */ + override def visitDropNamespace(ctx: DropNamespaceContext): LogicalPlan = withOrigin(ctx) { + DropNamespaceStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + ctx.EXISTS != null, + ctx.CASCADE != null) + } + /** * Create a [[ShowNamespacesStatement]] command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 8dfaaae11ddd2..465546f8ffe92 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -295,6 +295,14 @@ object CreateNamespaceStatement { val LOCATION_PROPERTY_KEY: String = "location" } +/** + * A DROP NAMESPACE statement, as parsed from SQL. + */ +case class DropNamespaceStatement( + namespace: Seq[String], + ifExists: Boolean, + cascade: Boolean) extends ParsedStatement + /** * A SHOW NAMESPACES statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index d80c1c034a867..f587ee2928fc2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -246,6 +246,15 @@ case class CreateNamespace( ifNotExists: Boolean, properties: Map[String, String]) extends Command +/** + * The logical plan of the DROP NAMESPACE command that works for v2 catalogs. + */ +case class DropNamespace( + catalog: SupportsNamespaces, + namespace: Seq[String], + ifExists: Boolean, + cascade: Boolean) extends Command + /** * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index e85b15f82c574..8bc4c68fed686 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -929,6 +929,28 @@ class DDLParserSuite extends AnalysisTest { "location" -> "/home/user/db"))) } + test("drop namespace") { + comparePlans( + parsePlan("DROP NAMESPACE a.b.c"), + DropNamespaceStatement(Seq("a", "b", "c"), ifExists = false, cascade = false)) + + comparePlans( + parsePlan("DROP NAMESPACE IF EXISTS a.b.c"), + DropNamespaceStatement(Seq("a", "b", "c"), ifExists = true, cascade = false)) + + comparePlans( + parsePlan("DROP NAMESPACE IF EXISTS a.b.c RESTRICT"), + DropNamespaceStatement(Seq("a", "b", "c"), ifExists = true, cascade = false)) + + comparePlans( + parsePlan("DROP NAMESPACE IF EXISTS a.b.c CASCADE"), + DropNamespaceStatement(Seq("a", "b", "c"), ifExists = true, cascade = true)) + + comparePlans( + parsePlan("DROP NAMESPACE a.b.c CASCADE"), + DropNamespaceStatement(Seq("a", "b", "c"), ifExists = false, cascade = true)) + } + test("show databases: basic") { comparePlans( parsePlan("SHOW DATABASES"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index c55105d62af33..4da434996d978 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, LoadDataCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropDatabaseCommand, DropTableCommand, LoadDataCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -271,6 +271,13 @@ class ResolveSessionCatalog( CreateNamespaceStatement.LOCATION_PROPERTY_KEY CreateDatabaseCommand(nameParts.head, c.ifNotExists, location, comment, newProperties) + case d @ DropNamespaceStatement(SessionCatalog(_, nameParts), _, _) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + DropDatabaseCommand(nameParts.head, d.ifExists, d.cascade) + case ShowTablesStatement(Some(SessionCatalog(catalog, nameParts)), pattern) => if (nameParts.length != 1) { throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 1f1d38bfa4cd5..13de3d3b2c510 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -304,18 +304,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ) } - /** - * Create a [[DropDatabaseCommand]] command. - * - * For example: - * {{{ - * DROP (DATABASE|SCHEMA) [IF EXISTS] database [RESTRICT|CASCADE]; - * }}} - */ - override def visitDropDatabase(ctx: DropDatabaseContext): LogicalPlan = withOrigin(ctx) { - DropDatabaseCommand(ctx.db.getText, ctx.EXISTS != null, ctx.CASCADE != null) - } - /** * Create a [[DescribeDatabaseCommand]] command. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 4a7cb7db45ded..3041e9e82d59f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} @@ -295,6 +295,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case CreateNamespace(catalog, namespace, ifNotExists, properties) => CreateNamespaceExec(catalog, namespace, ifNotExists, properties) :: Nil + case DropNamespace(catalog, namespace, ifExists, cascade) => + DropNamespaceExec(catalog, namespace, ifExists, cascade) :: Nil + case r: ShowNamespaces => ShowNamespacesExec(r.output, r.catalog, r.namespace, r.pattern) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala new file mode 100644 index 0000000000000..cd63ca6628abb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.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.v2 + +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.SupportsNamespaces + +/** + * Physical plan node for dropping a namespace. + */ +case class DropNamespaceExec( + catalog: SupportsNamespaces, + namespace: Seq[String], + ifExists: Boolean, + cascade: Boolean) + extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { + val ns = namespace.toArray + if (catalog.namespaceExists(ns)) { + try { + catalog.dropNamespace(ns) + } catch { + case e: IllegalStateException if cascade => + throw new SparkException( + "Cascade option for droping namespace is not supported in V2 catalog", e) + } + } else if (!ifExists) { + throw new NoSuchNamespaceException(ns) + } + + Seq.empty + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index b9a542183d1bc..b3b80753f83e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.connector import scala.collection.JavaConverters._ +import org.apache.spark.SparkException import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.internal.SQLConf @@ -790,6 +791,45 @@ class DataSourceV2SQLSuite sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1") } + test("DropNamespace: basic tests") { + // Session catalog is used. + sql("CREATE NAMESPACE ns") + testShowNamespaces("SHOW NAMESPACES", Seq("default", "ns")) + sql("DROP NAMESPACE ns") + testShowNamespaces("SHOW NAMESPACES", Seq("default")) + + // V2 non-session catalog is used. + sql("CREATE NAMESPACE testcat.ns1") + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1")) + sql("DROP NAMESPACE testcat.ns1") + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq()) + } + + test("DropNamespace: drop non-empty namespace") { + sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1")) + + val e1 = intercept[IllegalStateException] { + sql("DROP NAMESPACE testcat.ns1") + } + assert(e1.getMessage.contains("Cannot delete non-empty namespace: ns1")) + + val e2 = intercept[SparkException] { + sql("DROP NAMESPACE testcat.ns1 CASCADE") + } + assert(e2.getMessage.contains( + "Cascade option for droping namespace is not supported in V2 catalog")) + } + + test("DropNamespace: test handling of 'IF EXISTS'") { + sql("DROP NAMESPACE IF EXISTS testcat.unknown") + + val exception = intercept[NoSuchNamespaceException] { + sql("DROP NAMESPACE testcat.ns1") + } + assert(exception.getMessage.contains("Namespace 'ns1' not found")) + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set("spark.sql.default.catalog", "testcat") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 25ddc9b00a0b7..55482a95f8a65 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -74,51 +74,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { }.head } - test("drop database") { - val sql1 = "DROP DATABASE IF EXISTS database_name RESTRICT" - val sql2 = "DROP DATABASE IF EXISTS database_name CASCADE" - val sql3 = "DROP SCHEMA IF EXISTS database_name RESTRICT" - val sql4 = "DROP SCHEMA IF EXISTS database_name CASCADE" - // The default is restrict=true - val sql5 = "DROP DATABASE IF EXISTS database_name" - // The default is ifExists=false - val sql6 = "DROP DATABASE database_name" - val sql7 = "DROP DATABASE database_name CASCADE" - - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - val parsed3 = parser.parsePlan(sql3) - val parsed4 = parser.parsePlan(sql4) - val parsed5 = parser.parsePlan(sql5) - val parsed6 = parser.parsePlan(sql6) - val parsed7 = parser.parsePlan(sql7) - - val expected1 = DropDatabaseCommand( - "database_name", - ifExists = true, - cascade = false) - val expected2 = DropDatabaseCommand( - "database_name", - ifExists = true, - cascade = true) - val expected3 = DropDatabaseCommand( - "database_name", - ifExists = false, - cascade = false) - val expected4 = DropDatabaseCommand( - "database_name", - ifExists = false, - cascade = true) - - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - comparePlans(parsed3, expected1) - comparePlans(parsed4, expected2) - comparePlans(parsed5, expected1) - comparePlans(parsed6, expected3) - comparePlans(parsed7, expected4) - } - test("alter database set dbproperties") { // ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...) val sql1 = "ALTER DATABASE database_name SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')" From c137acbf6555ebf9024b02114de468ec4776e2bd Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 29 Oct 2019 11:11:41 +0800 Subject: [PATCH 0450/1387] [SPARK-29566][ML] Imputer should support single-column input/output ### What changes were proposed in this pull request? add single-column input/output support in Imputer ### Why are the changes needed? Currently, Imputer only has multi-column support. This PR adds single-column input/output support. ### Does this PR introduce any user-facing change? Yes. add single-column input/output support in Imputer ```Imputer.setInputCol``` ```Imputer.setOutputCol``` ### How was this patch tested? add unit tests Closes #26247 from huaxingao/spark-29566. Authored-by: Huaxin Gao Signed-off-by: zhengruifeng --- .../org/apache/spark/ml/feature/Imputer.scala | 66 +++- .../spark/ml/feature/ImputerSuite.scala | 292 ++++++++++++++++-- python/pyspark/ml/feature.py | 73 ++++- 3 files changed, 381 insertions(+), 50 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index 99c0a0df53672..5e593f731c624 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -23,7 +23,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCols} +import org.apache.spark.ml.param.shared.{HasInputCol, HasInputCols, HasOutputCol, HasOutputCols} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ @@ -32,7 +32,8 @@ import org.apache.spark.sql.types._ /** * Params for [[Imputer]] and [[ImputerModel]]. */ -private[feature] trait ImputerParams extends Params with HasInputCols with HasOutputCols { +private[feature] trait ImputerParams extends Params with HasInputCol with HasInputCols + with HasOutputCol with HasOutputCols { /** * The imputation strategy. Currently only "mean" and "median" are supported. @@ -63,15 +64,26 @@ private[feature] trait ImputerParams extends Params with HasInputCols with HasOu /** @group getParam */ def getMissingValue: Double = $(missingValue) + /** Returns the input and output column names corresponding in pair. */ + private[feature] def getInOutCols(): (Array[String], Array[String]) = { + if (isSet(inputCol)) { + (Array($(inputCol)), Array($(outputCol))) + } else { + ($(inputCols), $(outputCols)) + } + } + /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { - require($(inputCols).length == $(inputCols).distinct.length, s"inputCols contains" + - s" duplicates: (${$(inputCols).mkString(", ")})") - require($(outputCols).length == $(outputCols).distinct.length, s"outputCols contains" + - s" duplicates: (${$(outputCols).mkString(", ")})") - require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" + - s" and outputCols(${$(outputCols).length}) should have the same length") - val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) => + ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), Seq(outputCols)) + val (inputColNames, outputColNames) = getInOutCols() + require(inputColNames.length == inputColNames.distinct.length, s"inputCols contains" + + s" duplicates: (${inputColNames.mkString(", ")})") + require(outputColNames.length == outputColNames.distinct.length, s"outputCols contains" + + s" duplicates: (${outputColNames.mkString(", ")})") + require(inputColNames.length == outputColNames.length, s"inputCols(${inputColNames.length})" + + s" and outputCols(${outputColNames.length}) should have the same length") + val outputFields = inputColNames.zip(outputColNames).map { case (inputCol, outputCol) => val inputField = schema(inputCol) SchemaUtils.checkNumericType(schema, inputCol) StructField(outputCol, inputField.dataType, inputField.nullable) @@ -101,6 +113,14 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) @Since("2.2.0") def this() = this(Identifiable.randomUID("imputer")) + /** @group setParam */ + @Since("3.0.0") + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + @Since("3.0.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ @Since("2.2.0") def setInputCols(value: Array[String]): this.type = set(inputCols, value) @@ -126,7 +146,9 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) transformSchema(dataset.schema, logging = true) val spark = dataset.sparkSession - val cols = $(inputCols).map { inputCol => + val (inputColumns, _) = getInOutCols() + + val cols = inputColumns.map { inputCol => when(col(inputCol).equalTo($(missingValue)), null) .when(col(inputCol).isNaN, null) .otherwise(col(inputCol)) @@ -139,7 +161,7 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) // Function avg will ignore null automatically. // For a column only containing null, avg will return null. val row = dataset.select(cols.map(avg): _*).head() - Array.range(0, $(inputCols).length).map { i => + Array.range(0, inputColumns.length).map { i => if (row.isNullAt(i)) { Double.NaN } else { @@ -150,7 +172,7 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) case Imputer.median => // Function approxQuantile will ignore null automatically. // For a column only containing null, approxQuantile will return an empty array. - dataset.select(cols: _*).stat.approxQuantile($(inputCols), Array(0.5), 0.001) + dataset.select(cols: _*).stat.approxQuantile(inputColumns, Array(0.5), 0.001) .map { array => if (array.isEmpty) { Double.NaN @@ -160,7 +182,7 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) } } - val emptyCols = $(inputCols).zip(results).filter(_._2.isNaN).map(_._1) + val emptyCols = inputColumns.zip(results).filter(_._2.isNaN).map(_._1) if (emptyCols.nonEmpty) { throw new SparkException(s"surrogate cannot be computed. " + s"All the values in ${emptyCols.mkString(",")} are Null, Nan or " + @@ -168,7 +190,7 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) } val rows = spark.sparkContext.parallelize(Seq(Row.fromSeq(results))) - val schema = StructType($(inputCols).map(col => StructField(col, DoubleType, nullable = false))) + val schema = StructType(inputColumns.map(col => StructField(col, DoubleType, nullable = false))) val surrogateDF = spark.createDataFrame(rows, schema) copyValues(new ImputerModel(uid, surrogateDF).setParent(this)) } @@ -205,6 +227,14 @@ class ImputerModel private[ml] ( import ImputerModel._ + /** @group setParam */ + @Since("3.0.0") + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + @Since("3.0.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ def setInputCols(value: Array[String]): this.type = set(inputCols, value) @@ -213,9 +243,11 @@ class ImputerModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val surrogates = surrogateDF.select($(inputCols).map(col): _*).head().toSeq + val (inputColumns, outputColumns) = getInOutCols + val surrogates = surrogateDF.select(inputColumns.map(col): _*).head().toSeq + - val newCols = $(inputCols).zip($(outputCols)).zip(surrogates).map { + val newCols = inputColumns.zip(outputColumns).zip(surrogates).map { case ((inputCol, outputCol), surrogate) => val inputType = dataset.schema(inputCol).dataType val ic = col(inputCol).cast(DoubleType) @@ -224,7 +256,7 @@ class ImputerModel private[ml] ( .otherwise(ic) .cast(inputType) } - dataset.withColumns($(outputCols), newCols).toDF() + dataset.withColumns(outputColumns, newCols).toDF() } override def transformSchema(schema: StructType): StructType = { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala index 02ef261a6c067..dfee2b4029c8b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkException +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} @@ -36,7 +38,31 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { val imputer = new Imputer() .setInputCols(Array("value1", "value2")) .setOutputCols(Array("out1", "out2")) - ImputerSuite.iterateStrategyTest(imputer, df) + ImputerSuite.iterateStrategyTest(true, imputer, df) + } + + test("Single Column: Imputer for Double with default missing Value NaN") { + val df1 = spark.createDataFrame( Seq( + (0, 1.0, 1.0, 1.0), + (1, 11.0, 11.0, 11.0), + (2, 3.0, 3.0, 3.0), + (3, Double.NaN, 5.0, 3.0) + )).toDF("id", "value", "expected_mean_value", "expected_median_value") + val imputer1 = new Imputer() + .setInputCol("value") + .setOutputCol("out") + ImputerSuite.iterateStrategyTest(false, imputer1, df1) + + val df2 = spark.createDataFrame( Seq( + (0, 4.0, 4.0, 4.0), + (1, 12.0, 12.0, 12.0), + (2, Double.NaN, 10.0, 12.0), + (3, 14.0, 14.0, 14.0) + )).toDF("id", "value", "expected_mean_value", "expected_median_value") + val imputer2 = new Imputer() + .setInputCol("value") + .setOutputCol("out") + ImputerSuite.iterateStrategyTest(false, imputer2, df2) } test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") { @@ -48,7 +74,20 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { )).toDF("id", "value", "expected_mean_value", "expected_median_value") val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out")) .setMissingValue(-1.0) - ImputerSuite.iterateStrategyTest(imputer, df) + ImputerSuite.iterateStrategyTest(true, imputer, df) + } + + test("Single Column: Imputer should handle NaNs when computing surrogate value," + + " if missingValue is not NaN") { + val df = spark.createDataFrame( Seq( + (0, 1.0, 1.0, 1.0), + (1, 3.0, 3.0, 3.0), + (2, Double.NaN, Double.NaN, Double.NaN), + (3, -1.0, 2.0, 1.0) + )).toDF("id", "value", "expected_mean_value", "expected_median_value") + val imputer = new Imputer().setInputCol("value").setOutputCol("out") + .setMissingValue(-1.0) + ImputerSuite.iterateStrategyTest(false, imputer, df) } test("Imputer for Float with missing Value -1.0") { @@ -61,7 +100,20 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { )).toDF("id", "value", "expected_mean_value", "expected_median_value") val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out")) .setMissingValue(-1) - ImputerSuite.iterateStrategyTest(imputer, df) + ImputerSuite.iterateStrategyTest(true, imputer, df) + } + + test("Single Column: Imputer for Float with missing Value -1.0") { + val df = spark.createDataFrame( Seq( + (0, 1.0F, 1.0F, 1.0F), + (1, 3.0F, 3.0F, 3.0F), + (2, 10.0F, 10.0F, 10.0F), + (3, 10.0F, 10.0F, 10.0F), + (4, -1.0F, 6.0F, 3.0F) + )).toDF("id", "value", "expected_mean_value", "expected_median_value") + val imputer = new Imputer().setInputCol("value").setOutputCol("out") + .setMissingValue(-1) + ImputerSuite.iterateStrategyTest(false, imputer, df) } test("Imputer should impute null as well as 'missingValue'") { @@ -74,7 +126,20 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value") val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value") val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out")) - ImputerSuite.iterateStrategyTest(imputer, df) + ImputerSuite.iterateStrategyTest(true, imputer, df) + } + + test("Single Column: Imputer should impute null as well as 'missingValue'") { + val rawDf = spark.createDataFrame( Seq( + (0, 4.0, 4.0, 4.0), + (1, 10.0, 10.0, 10.0), + (2, 10.0, 10.0, 10.0), + (3, Double.NaN, 8.0, 10.0), + (4, -1.0, 8.0, 10.0) + )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value") + val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value") + val imputer = new Imputer().setInputCol("value").setOutputCol("out") + ImputerSuite.iterateStrategyTest(false, imputer, df) } test("Imputer should work with Structured Streaming") { @@ -99,6 +164,28 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { } } + test("Single Column: Imputer should work with Structured Streaming") { + val localSpark = spark + import localSpark.implicits._ + val df = Seq[(java.lang.Double, Double)]( + (4.0, 4.0), + (10.0, 10.0), + (10.0, 10.0), + (Double.NaN, 8.0), + (null, 8.0) + ).toDF("value", "expected_mean_value") + val imputer = new Imputer() + .setInputCol("value") + .setOutputCol("out") + .setStrategy("mean") + val model = imputer.fit(df) + testTransformer[(java.lang.Double, Double)](df, model, "expected_mean_value", "out") { + case Row(exp: java.lang.Double, out: Double) => + assert((exp.isNaN && out.isNaN) || (exp == out), + s"Imputed values differ. Expected: $exp, actual: $out") + } + } + test("Imputer throws exception when surrogate cannot be computed") { val df = spark.createDataFrame( Seq( (0, Double.NaN, 1.0, 1.0), @@ -117,6 +204,24 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { } } + test("Single Column: Imputer throws exception when surrogate cannot be computed") { + val df = spark.createDataFrame( Seq( + (0, Double.NaN, 1.0, 1.0), + (1, Double.NaN, 3.0, 3.0), + (2, Double.NaN, Double.NaN, Double.NaN) + )).toDF("id", "value", "expected_mean_value", "expected_median_value") + Seq("mean", "median").foreach { strategy => + val imputer = new Imputer().setInputCol("value").setOutputCol("out") + .setStrategy(strategy) + withClue("Imputer should fail all the values are invalid") { + val e: SparkException = intercept[SparkException] { + val model = imputer.fit(df) + } + assert(e.getMessage.contains("surrogate cannot be computed")) + } + } + } + test("Imputer input & output column validation") { val df = spark.createDataFrame( Seq( (0, 1.0, 1.0, 1.0), @@ -164,6 +269,14 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { testDefaultReadWrite(t) } + test("Single Column: Imputer read/write") { + val t = new Imputer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMissingValue(-1.0) + testDefaultReadWrite(t) + } + test("ImputerModel read/write") { val spark = this.spark import spark.implicits._ @@ -178,6 +291,20 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { assert(newInstance.surrogateDF.collect() === instance.surrogateDF.collect()) } + test("Single Column: ImputerModel read/write") { + val spark = this.spark + import spark.implicits._ + val surrogateDF = Seq(1.234).toDF("myInputCol") + + val instance = new ImputerModel( + "myImputer", surrogateDF) + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.surrogateDF.columns === instance.surrogateDF.columns) + assert(newInstance.surrogateDF.collect() === instance.surrogateDF.collect()) + } + test("Imputer for IntegerType with default missing value null") { val df = spark.createDataFrame(Seq[(Integer, Integer, Integer)]( @@ -195,7 +322,27 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { for (mType <- types) { // cast all columns to desired data type for testing val df2 = df.select(df.columns.map(c => col(c).cast(mType)): _*) - ImputerSuite.iterateStrategyTest(imputer, df2) + ImputerSuite.iterateStrategyTest(true, imputer, df2) + } + } + + test("Single Column Imputer for IntegerType with default missing value null") { + val df = spark.createDataFrame(Seq[(Integer, Integer, Integer)]( + (1, 1, 1), + (11, 11, 11), + (3, 3, 3), + (null, 5, 3) + )).toDF("value", "expected_mean_value", "expected_median_value") + + val imputer = new Imputer() + .setInputCol("value") + .setOutputCol("out") + + val types = Seq(IntegerType, LongType) + for (mType <- types) { + // cast all columns to desired data type for testing + val df2 = df.select(df.columns.map(c => col(c).cast(mType)): _*) + ImputerSuite.iterateStrategyTest(false, imputer, df2) } } @@ -217,7 +364,85 @@ class ImputerSuite extends MLTest with DefaultReadWriteTest { for (mType <- types) { // cast all columns to desired data type for testing val df2 = df.select(df.columns.map(c => col(c).cast(mType)): _*) - ImputerSuite.iterateStrategyTest(imputer, df2) + ImputerSuite.iterateStrategyTest(true, imputer, df2) + } + } + + test("Single Column: Imputer for IntegerType with missing value -1") { + val df = spark.createDataFrame(Seq[(Integer, Integer, Integer)]( + (1, 1, 1), + (11, 11, 11), + (3, 3, 3), + (-1, 5, 3) + )).toDF("value", "expected_mean_value", "expected_median_value") + + val imputer = new Imputer() + .setInputCol("value") + .setOutputCol("out") + .setMissingValue(-1.0) + + val types = Seq(IntegerType, LongType) + for (mType <- types) { + // cast all columns to desired data type for testing + val df2 = df.select(df.columns.map(c => col(c).cast(mType)): _*) + ImputerSuite.iterateStrategyTest(false, imputer, df2) + } + } + + test("assert exception is thrown if both multi-column and single-column params are set") { + import testImplicits._ + val df = Seq((0.5, 0.3), (0.5, -0.4)).toDF("feature1", "feature2") + ParamsSuite.testExclusiveParams(new Imputer, df, ("inputCol", "feature1"), + ("inputCols", Array("feature1", "feature2"))) + ParamsSuite.testExclusiveParams(new Imputer, df, ("inputCol", "feature1"), + ("outputCol", "result1"), ("outputCols", Array("result1", "result2"))) + + // this should fail because at least one of inputCol and inputCols must be set + ParamsSuite.testExclusiveParams(new Imputer, df, ("outputCol", "feature1")) + } + + test("Compare single/multiple column(s) Imputer in pipeline") { + val df = spark.createDataFrame( Seq( + (0, 1.0, 4.0), + (1, 11.0, 12.0), + (2, 3.0, Double.NaN), + (3, Double.NaN, 14.0) + )).toDF("id", "value1", "value2") + Seq("mean", "median").foreach { strategy => + val multiColsImputer = new Imputer() + .setInputCols(Array("value1", "value2")) + .setOutputCols(Array("result1", "result2")) + .setStrategy(strategy) + + val plForMultiCols = new Pipeline() + .setStages(Array(multiColsImputer)) + .fit(df) + + val imputerForCol1 = new Imputer() + .setInputCol("value1") + .setOutputCol("result1") + .setStrategy(strategy) + val imputerForCol2 = new Imputer() + .setInputCol("value2") + .setOutputCol("result2") + .setStrategy(strategy) + + val plForSingleCol = new Pipeline() + .setStages(Array(imputerForCol1, imputerForCol2)) + .fit(df) + + val resultForSingleCol = plForSingleCol.transform(df) + .select("result1", "result2") + .collect() + val resultForMultiCols = plForMultiCols.transform(df) + .select("result1", "result2") + .collect() + + resultForSingleCol.zip(resultForMultiCols).foreach { + case (rowForSingle, rowForMultiCols) => + assert(rowForSingle.getDouble(0) == rowForMultiCols.getDouble(0) && + rowForSingle.getDouble(1) == rowForMultiCols.getDouble(1)) + } } } } @@ -228,34 +453,45 @@ object ImputerSuite { * Imputation strategy. Available options are ["mean", "median"]. * @param df DataFrame with columns "id", "value", "expected_mean", "expected_median" */ - def iterateStrategyTest(imputer: Imputer, df: DataFrame): Unit = { + def iterateStrategyTest(isMultiCol: Boolean, imputer: Imputer, df: DataFrame): Unit = { Seq("mean", "median").foreach { strategy => imputer.setStrategy(strategy) val model = imputer.fit(df) val resultDF = model.transform(df) - imputer.getInputCols.zip(imputer.getOutputCols).foreach { case (inputCol, outputCol) => - - // check dataType is consistent between input and output - val inputType = resultDF.schema(inputCol).dataType - val outputType = resultDF.schema(outputCol).dataType - assert(inputType == outputType, "Output type is not the same as input type.") - - // check value - resultDF.select(s"expected_${strategy}_$inputCol", outputCol).collect().foreach { - case Row(exp: Float, out: Float) => - assert((exp.isNaN && out.isNaN) || (exp == out), - s"Imputed values differ. Expected: $exp, actual: $out") - case Row(exp: Double, out: Double) => - assert((exp.isNaN && out.isNaN) || (exp ~== out absTol 1e-5), - s"Imputed values differ. Expected: $exp, actual: $out") - case Row(exp: Integer, out: Integer) => - assert(exp == out, - s"Imputed values differ. Expected: $exp, actual: $out") - case Row(exp: Long, out: Long) => - assert(exp == out, - s"Imputed values differ. Expected: $exp, actual: $out") + if (isMultiCol) { + imputer.getInputCols.zip(imputer.getOutputCols).foreach { case (inputCol, outputCol) => + verifyTransformResult(strategy, inputCol, outputCol, resultDF) } + } else { + verifyTransformResult(strategy, imputer.getInputCol, imputer.getOutputCol, resultDF) } } } + + def verifyTransformResult( + strategy: String, + inputCol: String, + outputCol: String, + resultDF: DataFrame): Unit = { + // check dataType is consistent between input and output + val inputType = resultDF.schema(inputCol).dataType + val outputType = resultDF.schema(outputCol).dataType + assert(inputType == outputType, "Output type is not the same as input type.") + + // check value + resultDF.select(s"expected_${strategy}_$inputCol", outputCol).collect().foreach { + case Row(exp: Float, out: Float) => + assert((exp.isNaN && out.isNaN) || (exp == out), + s"Imputed values differ. Expected: $exp, actual: $out") + case Row(exp: Double, out: Double) => + assert((exp.isNaN && out.isNaN) || (exp ~== out absTol 1e-5), + s"Imputed values differ. Expected: $exp, actual: $out") + case Row(exp: Integer, out: Integer) => + assert(exp == out, + s"Imputed values differ. Expected: $exp, actual: $out") + case Row(exp: Long, out: Long) => + assert(exp == out, + s"Imputed values differ. Expected: $exp, actual: $out") + } + } } diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 11bb7941b5d9a..7645897ea5fc7 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1464,7 +1464,7 @@ def numDocs(self): return self._call_java("numDocs") -class _ImputerParams(HasInputCols, HasOutputCols): +class _ImputerParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCols): """ Params for :py:class:`Imputer` and :py:class:`ImputerModel`. @@ -1540,6 +1540,55 @@ class Imputer(JavaEstimator, _ImputerParams, JavaMLReadable, JavaMLWritable): +---+---+-----+-----+ |1.0|NaN| 4.0| NaN| ... + >>> df1 = spark.createDataFrame([(1.0,), (2.0,), (float("nan"),), (4.0,), (5.0,)], ["a"]) + >>> imputer1 = Imputer(inputCol="a", outputCol="out_a") + >>> model1 = imputer1.fit(df1) + >>> model1.surrogateDF.show() + +---+ + | a| + +---+ + |3.0| + +---+ + ... + >>> model1.transform(df1).show() + +---+-----+ + | a|out_a| + +---+-----+ + |1.0| 1.0| + |2.0| 2.0| + |NaN| 3.0| + ... + >>> imputer1.setStrategy("median").setMissingValue(1.0).fit(df1).transform(df1).show() + +---+-----+ + | a|out_a| + +---+-----+ + |1.0| 4.0| + ... + >>> df2 = spark.createDataFrame([(float("nan"),), (float("nan"),), (3.0,), (4.0,), (5.0,)], + ... ["b"]) + >>> imputer2 = Imputer(inputCol="b", outputCol="out_b") + >>> model2 = imputer2.fit(df2) + >>> model2.surrogateDF.show() + +---+ + | b| + +---+ + |4.0| + +---+ + ... + >>> model2.transform(df2).show() + +---+-----+ + | b|out_b| + +---+-----+ + |NaN| 4.0| + |NaN| 4.0| + |3.0| 3.0| + ... + >>> imputer2.setStrategy("median").setMissingValue(1.0).fit(df2).transform(df2).show() + +---+-----+ + | b|out_b| + +---+-----+ + |NaN| NaN| + ... >>> imputerPath = temp_path + "/imputer" >>> imputer.save(imputerPath) >>> loadedImputer = Imputer.load(imputerPath) @@ -1558,10 +1607,10 @@ class Imputer(JavaEstimator, _ImputerParams, JavaMLReadable, JavaMLWritable): @keyword_only def __init__(self, strategy="mean", missingValue=float("nan"), inputCols=None, - outputCols=None): + outputCols=None, inputCol=None, outputCol=None): """ __init__(self, strategy="mean", missingValue=float("nan"), inputCols=None, \ - outputCols=None): + outputCols=None, inputCol=None, outputCol=None): """ super(Imputer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Imputer", self.uid) @@ -1572,10 +1621,10 @@ def __init__(self, strategy="mean", missingValue=float("nan"), inputCols=None, @keyword_only @since("2.2.0") def setParams(self, strategy="mean", missingValue=float("nan"), inputCols=None, - outputCols=None): + outputCols=None, inputCol=None, outputCol=None): """ setParams(self, strategy="mean", missingValue=float("nan"), inputCols=None, \ - outputCols=None) + outputCols=None, inputCol=None, outputCol=None) Sets params for this Imputer. """ kwargs = self._input_kwargs @@ -1609,6 +1658,20 @@ def setOutputCols(self, value): """ return self._set(outputCols=value) + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return ImputerModel(java_model) From fb80dfee70011d0ce344473515ff15b17d4c4935 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 29 Oct 2019 11:57:25 +0800 Subject: [PATCH 0451/1387] [SPARK-28158][SQL][FOLLOWUP] HiveUserDefinedTypeSuite: don't use RandomDataGenerator to create row for UDT backed by ArrayType ### What changes were proposed in this pull request? There're some issues observed in `HiveUserDefinedTypeSuite."Support UDT in Hive UDF"`: 1) Neither function (TestUDF) nor test take "nullable" point column into account. 2) ExamplePointUDT. sqlType is ArrayType which doesn't provide information how many elements are expected. RandomDataGenerator may provide less elements than needed. This patch fixes `HiveUserDefinedTypeSuite."Support UDT in Hive UDF"` to change the type of "point" column to be non-nullable, as well as not use RandomDataGenerator to create row for UDT backed by ArrayType. ### Why are the changes needed? CI builds are failing in high occurrences. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested by running tests locally multiple times. Closes #26287 from HeartSaVioR/SPARK-28158-FOLLOWUP. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala index bddb7688fe96a..c160ff206a62a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.sql.hive import scala.collection.JavaConverters._ +import scala.util.Random import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, StandardListObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.spark.sql.{QueryTest, RandomDataGenerator, Row} +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT} @@ -35,9 +36,8 @@ class HiveUserDefinedTypeSuite extends QueryTest with TestHiveSingleton { test("Support UDT in Hive UDF") { val functionName = "get_point_x" try { - val schema = new StructType().add("point", new ExamplePointUDT) - val inputGenerator = RandomDataGenerator.forType(schema, nullable = false).get - val input = inputGenerator.apply().asInstanceOf[Row] + val schema = new StructType().add("point", new ExamplePointUDT, nullable = false) + val input = Row.fromSeq(Seq(new ExamplePoint(3.141592d, -3.141592d))) val df = spark.createDataFrame(Array(input).toList.asJava, schema) df.createOrReplaceTempView("src") spark.sql(s"CREATE FUNCTION $functionName AS '$functionClass'") From 5eddbb5f1d9789696927f435c55df887e50a1389 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Mon, 28 Oct 2019 22:31:29 -0700 Subject: [PATCH 0452/1387] Prepare Spark release v3.0.0-preview-rc1 ### What changes were proposed in this pull request? To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name. Made the following changes in this PR: * Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview` * Update the PySpark version from `3.0.0.dev0` to `3.0.0` **Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.** We shall revert the changes after 3.0.0-preview release passed. ### Why are the changes needed? To make the maven release repository to accept the built jars. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #26243 from jiangxb1987/3.0.0-preview-prepare. Lead-authored-by: Xingbo Jiang Co-authored-by: HyukjinKwon Signed-off-by: Xingbo Jiang --- R/pkg/R/sparkR.R | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10-token-provider/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graph/api/pom.xml | 2 +- graph/cypher/pom.xml | 2 +- graph/graph/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 41 files changed, 42 insertions(+), 42 deletions(-) diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index cdb59093781fb..dc1d1b2389239 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -337,7 +337,7 @@ sparkR.session <- function( # Check if version number of SparkSession matches version number of SparkR package jvmVersion <- callJMethod(sparkSession, "version") # Remove -SNAPSHOT from jvm versions - jvmVersionStrip <- gsub("-SNAPSHOT", "", jvmVersion) + jvmVersionStrip <- gsub("-preview", "", jvmVersion) rPackageVersion <- paste0(packageVersion("SparkR")) if (jvmVersionStrip != rPackageVersion) { diff --git a/assembly/pom.xml b/assembly/pom.xml index ef916fb99a04c..69efac5f43ca8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f042a12fda3d2..b19380d12312a 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2ee17800c10e4..4000f9296dc86 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index a6d99813a8501..f144b70b70c8d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 76a402bb2bd31..b355b29abc804 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 3c3c0d2d96a1c..5175e00739d11 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 883b73a69c9de..ca6de221c0378 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 93a4f67fd23f2..ad2ba6659e9cd 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 38eb8adac500e..19e67359a1c6f 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 57b8d716ee55c..b520819701651 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.0.0-SNAPSHOT -SPARK_VERSION_SHORT: 3.0.0 +SPARK_VERSION: 3.0.0-preview +SPARK_VERSION_SHORT: 3.0.0-preview SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index a099f1e042e99..f37f059a728c3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index ba6f20bfdbf58..60c5f80492d50 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index aff79b8b8e642..0f0176af7190d 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index f2dcf5d217a89..8442c46c3cb2b 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 693820da6af6b..cceace392d5ac 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index 01ca96b4f8f40..744423a88c1da 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index d11569d709b23..0e1d95cf822f4 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 0ce922349ea66..ae75d1b9f808e 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 7d69764b77de7..e284c52e84044 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index a23d255f9187c..190fffd909b15 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/graph/api/pom.xml b/graph/api/pom.xml index a18c5290195fe..953d0351b5b47 100644 --- a/graph/api/pom.xml +++ b/graph/api/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 12b3832d9777a..865a1bcf9d5a9 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml index 7438ee5e54fe4..8ed0b92a419d6 100644 --- a/graph/graph/pom.xml +++ b/graph/graph/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 444568a03d6c7..6f46cccc11871 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 02734e82ed26e..7c9357f492180 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e75e8345cd51d..86f9c1a0f2002 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 2eab868ac0dc8..497b70ddfed9c 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 11769ef548d7c..75ab818515a33 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/pom.xml b/pom.xml index b5c34790f1d3f..a10850fc7938e 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index c7de67e41ca94..284dc923313a4 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 487876b12ce91..d4a073376d112 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index c79caff164ce8..038613f810f0c 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 107ba365aa775..2038ddc2317e2 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 0e5df14e060da..15671c17a7523 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index bcebb225dfaca..ad7fe28e8d181 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3a8b7d22397ff..3565056b6206e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 9dd927084298a..64b78265e485b 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index f627227aa0380..1f3fbef6100df 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 87af6388e1118..7394391d014d2 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e380e869f55c7..e2608976938dd 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml From b33a58c0c6a40aa312cbd72f763d5e519182c3d1 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Mon, 28 Oct 2019 22:32:34 -0700 Subject: [PATCH 0453/1387] Revert "Prepare Spark release v3.0.0-preview-rc1" This reverts commit 5eddbb5f1d9789696927f435c55df887e50a1389. --- R/pkg/R/sparkR.R | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10-token-provider/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graph/api/pom.xml | 2 +- graph/cypher/pom.xml | 2 +- graph/graph/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 41 files changed, 42 insertions(+), 42 deletions(-) diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index dc1d1b2389239..cdb59093781fb 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -337,7 +337,7 @@ sparkR.session <- function( # Check if version number of SparkSession matches version number of SparkR package jvmVersion <- callJMethod(sparkSession, "version") # Remove -SNAPSHOT from jvm versions - jvmVersionStrip <- gsub("-preview", "", jvmVersion) + jvmVersionStrip <- gsub("-SNAPSHOT", "", jvmVersion) rPackageVersion <- paste0(packageVersion("SparkR")) if (jvmVersionStrip != rPackageVersion) { diff --git a/assembly/pom.xml b/assembly/pom.xml index 69efac5f43ca8..ef916fb99a04c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b19380d12312a..f042a12fda3d2 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 4000f9296dc86..2ee17800c10e4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f144b70b70c8d..a6d99813a8501 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index b355b29abc804..76a402bb2bd31 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 5175e00739d11..3c3c0d2d96a1c 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index ca6de221c0378..883b73a69c9de 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index ad2ba6659e9cd..93a4f67fd23f2 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 19e67359a1c6f..38eb8adac500e 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index b520819701651..57b8d716ee55c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.0.0-preview -SPARK_VERSION_SHORT: 3.0.0-preview +SPARK_VERSION: 3.0.0-SNAPSHOT +SPARK_VERSION_SHORT: 3.0.0 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index f37f059a728c3..a099f1e042e99 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 60c5f80492d50..ba6f20bfdbf58 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 0f0176af7190d..aff79b8b8e642 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 8442c46c3cb2b..f2dcf5d217a89 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index cceace392d5ac..693820da6af6b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index 744423a88c1da..01ca96b4f8f40 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 0e1d95cf822f4..d11569d709b23 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index ae75d1b9f808e..0ce922349ea66 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index e284c52e84044..7d69764b77de7 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 190fffd909b15..a23d255f9187c 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/api/pom.xml b/graph/api/pom.xml index 953d0351b5b47..a18c5290195fe 100644 --- a/graph/api/pom.xml +++ b/graph/api/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 865a1bcf9d5a9..12b3832d9777a 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml index 8ed0b92a419d6..7438ee5e54fe4 100644 --- a/graph/graph/pom.xml +++ b/graph/graph/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6f46cccc11871..444568a03d6c7 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 7c9357f492180..02734e82ed26e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 86f9c1a0f2002..e75e8345cd51d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 497b70ddfed9c..2eab868ac0dc8 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 75ab818515a33..11769ef548d7c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a10850fc7938e..b5c34790f1d3f 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 284dc923313a4..c7de67e41ca94 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index d4a073376d112..487876b12ce91 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 038613f810f0c..c79caff164ce8 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 2038ddc2317e2..107ba365aa775 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 15671c17a7523..0e5df14e060da 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ad7fe28e8d181..bcebb225dfaca 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3565056b6206e..3a8b7d22397ff 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 64b78265e485b..9dd927084298a 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1f3fbef6100df..f627227aa0380 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 7394391d014d2..87af6388e1118 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e2608976938dd..e380e869f55c7 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml From e92b75482fd4e5e3533a7f76f205faca94c71a7b Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 29 Oct 2019 13:54:07 +0800 Subject: [PATCH 0454/1387] [SPARK-29612][SQL] ALTER TABLE (RECOVER PARTITIONS) should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add AlterTableRecoverPartitionsStatement and make ALTER TABLE ... RECOVER PARTITIONS go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog ALTER TABLE t RECOVER PARTITIONS // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? Yes. When running ALTER TABLE ... RECOVER PARTITIONS Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26269 from huaxingao/spark-29612. Authored-by: Huaxin Gao Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/catalyst/parser/AstBuilder.scala | 13 +++++++++++++ .../sql/catalyst/plans/logical/statements.scala | 6 ++++++ .../spark/sql/catalyst/parser/DDLParserSuite.scala | 6 ++++++ .../catalyst/analysis/ResolveSessionCatalog.scala | 6 ++++++ .../apache/spark/sql/execution/SparkSqlParser.scala | 13 ------------- .../spark/sql/connector/DataSourceV2SQLSuite.scala | 11 +++++++++++ .../sql/execution/command/DDLParserSuite.scala | 8 -------- 8 files changed, 43 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index be96f5a927bc2..7ad008ae5d902 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -170,7 +170,7 @@ statement DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions | ALTER TABLE multipartIdentifier SET locationSpec #setTableLocation | ALTER TABLE tableIdentifier partitionSpec SET locationSpec #setPartitionLocation - | ALTER TABLE tableIdentifier RECOVER PARTITIONS #recoverPartitions + | ALTER TABLE multipartIdentifier RECOVER PARTITIONS #recoverPartitions | DROP TABLE (IF EXISTS)? multipartIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? multipartIdentifier #dropView | CREATE (OR REPLACE)? (GLOBAL? TEMPORARY)? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 3666eaeaa3d2a..893003d21828b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2911,4 +2911,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) { RefreshTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) } + + /** + * Create an [[AlterTableRecoverPartitionsStatement]] + * + * For example: + * {{{ + * ALTER TABLE multi_part_name RECOVER PARTITIONS; + * }}} + */ + override def visitRecoverPartitions( + ctx: RecoverPartitionsContext): LogicalPlan = withOrigin(ctx) { + AlterTableRecoverPartitionsStatement(visitMultipartIdentifier(ctx.multipartIdentifier)) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 465546f8ffe92..ac09dec13fff1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -181,6 +181,12 @@ case class AlterTableSetLocationStatement( tableName: Seq[String], location: String) extends ParsedStatement +/** + * ALTER TABLE ... RECOVER PARTITIONS command, as parsed from SQL. + */ +case class AlterTableRecoverPartitionsStatement( + tableName: Seq[String]) extends ParsedStatement + /** * ALTER VIEW ... SET TBLPROPERTIES command, as parsed from SQL. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 8bc4c68fed686..a1d74c5099c4a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1169,6 +1169,12 @@ class DDLParserSuite extends AnalysisTest { RefreshTableStatement(Seq("a", "b", "c"))) } + test("alter table: recover partitions") { + comparePlans( + parsePlan("ALTER TABLE a.b.c RECOVER PARTITIONS"), + AlterTableRecoverPartitionsStatement(Seq("a", "b", "c"))) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 4da434996d978..f60d7cdeae6e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -338,6 +338,12 @@ class ResolveSessionCatalog( ShowPartitionsCommand( v1TableName.asTableIdentifier, partitionSpec) + + case AlterTableRecoverPartitionsStatement(tableName) => + val v1TableName = parseV1Table(tableName, "ALTER TABLE RECOVER PARTITIONS") + AlterTableRecoverPartitionsCommand( + v1TableName.asTableIdentifier, + "ALTER TABLE RECOVER PARTITIONS") } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 13de3d3b2c510..99375a15f5230 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -515,19 +515,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { retainData = false) } - /** - * Create an [[AlterTableRecoverPartitionsCommand]] command - * - * For example: - * {{{ - * ALTER TABLE table RECOVER PARTITIONS; - * }}} - */ - override def visitRecoverPartitions( - ctx: RecoverPartitionsContext): LogicalPlan = withOrigin(ctx) { - AlterTableRecoverPartitionsCommand(visitTableIdentifier(ctx.tableIdentifier)) - } - /** * Create an [[AlterTableSetLocationCommand]] command for a partition. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index b3b80753f83e4..d4d53dc371044 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1358,6 +1358,17 @@ class DataSourceV2SQLSuite } } + test("ALTER TABLE RECOVER PARTITIONS") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + val e = intercept[AnalysisException] { + val partition = sql(s"ALTER TABLE $t RECOVER PARTITIONS") + } + assert(e.message.contains("ALTER TABLE RECOVER PARTITIONS is only supported with v1 tables")) + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 55482a95f8a65..beab219b17d8e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -549,14 +549,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { comparePlans(parsed2, expected2) } - test("alter table: recover partitions") { - val sql = "ALTER TABLE table_name RECOVER PARTITIONS" - val parsed = parser.parsePlan(sql) - val expected = AlterTableRecoverPartitionsCommand( - TableIdentifier("table_name", None)) - comparePlans(parsed, expected) - } - test("alter view: add partition (not supported)") { assertUnsupported( """ From 37690dea107623ebca1e47c64db59196ee388f2f Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 28 Oct 2019 23:20:21 -0700 Subject: [PATCH 0455/1387] [SPARK-29565][ML][PYTHON] OneHotEncoder should support single-column input/output ### What changes were proposed in this pull request? add single-column input/ouput support in OneHotEncoder ### Why are the changes needed? Currently, OneHotEncoder only has multi columns support. It makes sense to support single column as well. ### Does this PR introduce any user-facing change? Yes ```OneHotEncoder.setInputCol``` ```OneHotEncoder.setOutputCol``` ### How was this patch tested? Unit test Closes #26265 from huaxingao/spark-29565. Authored-by: Huaxin Gao Signed-off-by: Liang-Chi Hsieh --- .../spark/ml/feature/OneHotEncoder.scala | 52 +++-- .../spark/ml/feature/OneHotEncoderSuite.scala | 194 ++++++++++++++++++ python/pyspark/ml/feature.py | 33 ++- 3 files changed, 258 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index ec9792cbbda8f..459994c352da9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -25,7 +25,7 @@ import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCols, HasOutputCols} +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol, HasInputCols, HasOutputCol, HasOutputCols} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.expressions.UserDefinedFunction @@ -34,7 +34,7 @@ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} /** Private trait for params and common methods for OneHotEncoder and OneHotEncoderModel */ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid - with HasInputCols with HasOutputCols { + with HasInputCol with HasInputCols with HasOutputCol with HasOutputCols { /** * Param for how to handle invalid data during transform(). @@ -68,12 +68,21 @@ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid @Since("2.3.0") def getDropLast: Boolean = $(dropLast) + /** Returns the input and output column names corresponding in pair. */ + private[feature] def getInOutCols(): (Array[String], Array[String]) = { + if (isSet(inputCol)) { + (Array($(inputCol)), Array($(outputCol))) + } else { + ($(inputCols), $(outputCols)) + } + } + protected def validateAndTransformSchema( schema: StructType, dropLast: Boolean, keepInvalid: Boolean): StructType = { - val inputColNames = $(inputCols) - val outputColNames = $(outputCols) + ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), Seq(outputCols)) + val (inputColNames, outputColNames) = getInOutCols() require(inputColNames.length == outputColNames.length, s"The number of input columns ${inputColNames.length} must be the same as the number of " + @@ -83,7 +92,7 @@ private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid inputColNames.foreach(SchemaUtils.checkNumericType(schema, _)) // Prepares output columns with proper attributes by examining input columns. - val inputFields = $(inputCols).map(schema(_)) + val inputFields = inputColNames.map(schema(_)) val outputFields = inputFields.zip(outputColNames).map { case (inputField, outputColName) => OneHotEncoderCommon.transformOutputColumnSchema( @@ -123,6 +132,14 @@ class OneHotEncoder @Since("3.0.0") (@Since("3.0.0") override val uid: String) @Since("3.0.0") def this() = this(Identifiable.randomUID("oneHotEncoder")) + /** @group setParam */ + @Since("3.0.0") + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + @Since("3.0.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ @Since("3.0.0") def setInputCols(values: Array[String]): this.type = set(inputCols, values) @@ -150,13 +167,14 @@ class OneHotEncoder @Since("3.0.0") (@Since("3.0.0") override val uid: String) override def fit(dataset: Dataset[_]): OneHotEncoderModel = { transformSchema(dataset.schema) + val (inputColumns, outputColumns) = getInOutCols() // Compute the plain number of categories without `handleInvalid` and // `dropLast` taken into account. val transformedSchema = validateAndTransformSchema(dataset.schema, dropLast = false, keepInvalid = false) - val categorySizes = new Array[Int]($(outputCols).length) + val categorySizes = new Array[Int](outputColumns.length) - val columnToScanIndices = $(outputCols).zipWithIndex.flatMap { case (outputColName, idx) => + val columnToScanIndices = outputColumns.zipWithIndex.flatMap { case (outputColName, idx) => val numOfAttrs = AttributeGroup.fromStructField( transformedSchema(outputColName)).size if (numOfAttrs < 0) { @@ -170,8 +188,8 @@ class OneHotEncoder @Since("3.0.0") (@Since("3.0.0") override val uid: String) // Some input columns don't have attributes or their attributes don't have necessary info. // We need to scan the data to get the number of values for each column. if (columnToScanIndices.length > 0) { - val inputColNames = columnToScanIndices.map($(inputCols)(_)) - val outputColNames = columnToScanIndices.map($(outputCols)(_)) + val inputColNames = columnToScanIndices.map(inputColumns(_)) + val outputColNames = columnToScanIndices.map(outputColumns(_)) // When fitting data, we want the plain number of categories without `handleInvalid` and // `dropLast` taken into account. @@ -287,7 +305,7 @@ class OneHotEncoderModel private[ml] ( @Since("3.0.0") override def transformSchema(schema: StructType): StructType = { - val inputColNames = $(inputCols) + val (inputColNames, _) = getInOutCols() require(inputColNames.length == categorySizes.length, s"The number of input columns ${inputColNames.length} must be the same as the number of " + @@ -306,8 +324,9 @@ class OneHotEncoderModel private[ml] ( */ private def verifyNumOfValues(schema: StructType): StructType = { val configedSizes = getConfigedCategorySizes - $(outputCols).zipWithIndex.foreach { case (outputColName, idx) => - val inputColName = $(inputCols)(idx) + val (inputColNames, outputColNames) = getInOutCols() + outputColNames.zipWithIndex.foreach { case (outputColName, idx) => + val inputColName = inputColNames(idx) val attrGroup = AttributeGroup.fromStructField(schema(outputColName)) // If the input metadata specifies number of category for output column, @@ -327,10 +346,11 @@ class OneHotEncoderModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { val transformedSchema = transformSchema(dataset.schema, logging = true) val keepInvalid = $(handleInvalid) == OneHotEncoder.KEEP_INVALID + val (inputColNames, outputColNames) = getInOutCols() - val encodedColumns = $(inputCols).indices.map { idx => - val inputColName = $(inputCols)(idx) - val outputColName = $(outputCols)(idx) + val encodedColumns = inputColNames.indices.map { idx => + val inputColName = inputColNames(idx) + val outputColName = outputColNames(idx) val outputAttrGroupFromSchema = AttributeGroup.fromStructField(transformedSchema(outputColName)) @@ -345,7 +365,7 @@ class OneHotEncoderModel private[ml] ( encoder(col(inputColName).cast(DoubleType), lit(idx)) .as(outputColName, metadata) } - dataset.withColumns($(outputCols), encodedColumns) + dataset.withColumns(outputColNames, encodedColumns) } @Since("3.0.0") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala index 70f8c029a2575..897251d9815c8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.ml.feature +import org.apache.spark.ml.Pipeline import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamsSuite @@ -62,6 +63,34 @@ class OneHotEncoderSuite extends MLTest with DefaultReadWriteTest { } } + test("Single Column: OneHotEncoder dropLast = false") { + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val encoder = new OneHotEncoder() + .setInputCol("input") + .setOutputCol("output") + assert(encoder.getDropLast) + encoder.setDropLast(false) + assert(encoder.getDropLast === false) + val model = encoder.fit(df) + testTransformer[(Double, Vector)](df, model, "output", "expected") { + case Row(output: Vector, expected: Vector) => + assert(output === expected) + } + } + test("OneHotEncoder dropLast = true") { val data = Seq( Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), @@ -104,6 +133,22 @@ class OneHotEncoderSuite extends MLTest with DefaultReadWriteTest { } } + test("Single Column: input column with ML attribute") { + val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") + val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") + .select(col("size").as("size", attr.toMetadata())) + val encoder = new OneHotEncoder() + .setInputCol("size") + .setOutputCol("encoded") + val model = encoder.fit(df) + testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => + val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) + assert(group.size === 2) + assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("small").withIndex(0)) + assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("medium").withIndex(1)) + } + } + test("input column without ML attribute") { val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("index") val encoder = new OneHotEncoder() @@ -125,6 +170,13 @@ class OneHotEncoderSuite extends MLTest with DefaultReadWriteTest { testDefaultReadWrite(encoder) } + test("Single Column: read/write") { + val encoder = new OneHotEncoder() + .setInputCol("index") + .setOutputCol("encoded") + testDefaultReadWrite(encoder) + } + test("OneHotEncoderModel read/write") { val instance = new OneHotEncoderModel("myOneHotEncoderModel", Array(1, 2, 3)) val newInstance = testDefaultReadWrite(instance) @@ -173,6 +225,48 @@ class OneHotEncoderSuite extends MLTest with DefaultReadWriteTest { } } + test("Single Column: OneHotEncoder with varying types") { + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + class NumericTypeWithEncoder[A](val numericType: NumericType) + (implicit val encoder: Encoder[(A, Vector)]) + + val types = Seq( + new NumericTypeWithEncoder[Short](ShortType), + new NumericTypeWithEncoder[Long](LongType), + new NumericTypeWithEncoder[Int](IntegerType), + new NumericTypeWithEncoder[Float](FloatType), + new NumericTypeWithEncoder[Byte](ByteType), + new NumericTypeWithEncoder[Double](DoubleType), + new NumericTypeWithEncoder[Decimal](DecimalType(10, 0))(ExpressionEncoder())) + + for (t <- types) { + val dfWithTypes = df.select(col("input").cast(t.numericType), col("expected")) + val estimator = new OneHotEncoder() + .setInputCol("input") + .setOutputCol("output") + .setDropLast(false) + + val model = estimator.fit(dfWithTypes) + testTransformer(dfWithTypes, model, "output", "expected") { + case Row(output: Vector, expected: Vector) => + assert(output === expected) + }(t.encoder) + } + } + test("OneHotEncoder: encoding multiple columns and dropLast = false") { val data = Seq( Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0)))), @@ -211,6 +305,58 @@ class OneHotEncoderSuite extends MLTest with DefaultReadWriteTest { } } + test("Single Column: OneHotEncoder: encoding multiple columns and dropLast = false") { + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), 3.0, Vectors.sparse(4, Seq((3, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 1.0, Vectors.sparse(4, Seq((1, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input1", DoubleType), + StructField("expected1", new VectorUDT), + StructField("input2", DoubleType), + StructField("expected2", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val encoder1 = new OneHotEncoder() + .setInputCol("input1") + .setOutputCol("output1") + assert(encoder1.getDropLast) + encoder1.setDropLast(false) + assert(encoder1.getDropLast === false) + + val model1 = encoder1.fit(df) + testTransformer[(Double, Vector, Double, Vector)]( + df, + model1, + "output1", + "expected1") { + case Row(output1: Vector, expected1: Vector) => + assert(output1 === expected1) + } + + val encoder2 = new OneHotEncoder() + .setInputCol("input2") + .setOutputCol("output2") + assert(encoder2.getDropLast) + encoder2.setDropLast(false) + assert(encoder2.getDropLast === false) + + val model2 = encoder2.fit(df) + testTransformer[(Double, Vector, Double, Vector)]( + df, + model2, + "output2", + "expected2") { + case Row(output2: Vector, expected2: Vector) => + assert(output2 === expected2) + } + } + test("OneHotEncoder: encoding multiple columns and dropLast = true") { val data = Seq( Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 2.0, Vectors.sparse(3, Seq((2, 1.0)))), @@ -419,4 +565,52 @@ class OneHotEncoderSuite extends MLTest with DefaultReadWriteTest { expectedMessagePart = "OneHotEncoderModel expected 2 categorical values", firstResultCol = "encoded") } + + test("assert exception is thrown if both multi-column and single-column params are set") { + import testImplicits._ + val df = Seq((0.5, 0.3), (0.5, -0.4)).toDF("feature1", "feature2") + ParamsSuite.testExclusiveParams(new OneHotEncoder, df, ("inputCol", "feature1"), + ("inputCols", Array("feature1", "feature2"))) + ParamsSuite.testExclusiveParams(new OneHotEncoder, df, ("inputCol", "feature1"), + ("outputCol", "result1"), ("outputCols", Array("result1", "result2"))) + + // this should fail because at least one of inputCol and inputCols must be set + ParamsSuite.testExclusiveParams(new OneHotEncoder, df, ("outputCol", "feature1")) + } + + test("Compare single/multiple column(s) OneHotEncoder in pipeline") { + val df = Seq((0.0, 2.0), (1.0, 3.0), (2.0, 0.0), (0.0, 1.0), (0.0, 0.0), (2.0, 2.0)) + .toDF("input1", "input2") + + val multiColsEncoder = new OneHotEncoder() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("output1", "output2")) + + val plForMultiCols = new Pipeline() + .setStages(Array(multiColsEncoder)) + .fit(df) + + val encoderForCol1 = new OneHotEncoder() + .setInputCol("input1") + .setOutputCol("output1") + val encoderForCol2 = new OneHotEncoder() + .setInputCol("input2") + .setOutputCol("output2") + + val plForSingleCol = new Pipeline() + .setStages(Array(encoderForCol1, encoderForCol2)) + .fit(df) + + val resultForSingleCol = plForSingleCol.transform(df) + .select("output1", "output2") + .collect() + val resultForMultiCols = plForMultiCols.transform(df) + .select("output1", "output2") + .collect() + + resultForSingleCol.zip(resultForMultiCols).foreach { + case (rowForSingle, rowForMultiCols) => + assert(rowForSingle === rowForMultiCols) + } + } } diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 7645897ea5fc7..7ccdcf8560608 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2354,7 +2354,8 @@ def setOutputCol(self, value): return self._set(outputCol=value) -class _OneHotEncoderParams(HasInputCols, HasOutputCols, HasHandleInvalid): +class _OneHotEncoderParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCols, + HasHandleInvalid): """ Params for :py:class:`OneHotEncoder` and :py:class:`OneHotEncoderModel`. @@ -2415,6 +2416,10 @@ class OneHotEncoder(JavaEstimator, _OneHotEncoderParams, JavaMLReadable, JavaMLW 'error' >>> model.transform(df).head().output SparseVector(2, {0: 1.0}) + >>> single_col_ohe = OneHotEncoder(inputCol="input", outputCol="output") + >>> single_col_model = single_col_ohe.fit(df) + >>> single_col_model.transform(df).head().output + SparseVector(2, {0: 1.0}) >>> ohePath = temp_path + "/ohe" >>> ohe.save(ohePath) >>> loadedOHE = OneHotEncoder.load(ohePath) @@ -2430,9 +2435,11 @@ class OneHotEncoder(JavaEstimator, _OneHotEncoderParams, JavaMLReadable, JavaMLW """ @keyword_only - def __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True): + def __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True, + inputCol=None, outputCol=None): """ - __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True) + __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True, \ + inputCol=None, outputCol=None) """ super(OneHotEncoder, self).__init__() self._java_obj = self._new_java_obj( @@ -2443,9 +2450,11 @@ def __init__(self, inputCols=None, outputCols=None, handleInvalid="error", dropL @keyword_only @since("2.3.0") - def setParams(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True): + def setParams(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True, + inputCol=None, outputCol=None): """ - setParams(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True) + setParams(self, inputCols=None, outputCols=None, handleInvalid="error", dropLast=True, \ + inputCol=None, outputCol=None) Sets params for this OneHotEncoder. """ kwargs = self._input_kwargs @@ -2479,6 +2488,20 @@ def setHandleInvalid(self, value): """ return self._set(handleInvalid=value) + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + def _create_model(self, java_model): return OneHotEncoderModel(java_model) From 44c1c03924c9e829f84b0c1344feb699e062a5d7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 30 Oct 2019 01:15:18 +0800 Subject: [PATCH 0456/1387] [SPARK-29607][SQL] Move static methods from CalendarInterval to IntervalUtils ### What changes were proposed in this pull request? In the PR, I propose to move all static methods from the `CalendarInterval` class to the `IntervalUtils` object. All those methods are rewritten from Java to Scala. ### Why are the changes needed? - For consistency with other helper methods. Such methods were placed to the helper object `IntervalUtils`, see https://github.com/apache/spark/pull/26190 - Taking into account that `CalendarInterval` will be fully exposed to users in the future (see https://github.com/apache/spark/pull/25022), it would be nice to clean it up by moving service methods to an internal object. ### Does this PR introduce any user-facing change? No ### How was this patch tested? - By moved tests from `CalendarIntervalSuite` to `IntervalUtilsSuite` - By existing test suites Closes #26261 from MaxGekk/refactoring-calendar-interval. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../spark/unsafe/types/CalendarInterval.java | 205 ------------------ .../unsafe/types/CalendarIntervalSuite.java | 66 ------ .../sql/catalyst/parser/AstBuilder.scala | 18 +- .../sql/catalyst/util/IntervalUtils.scala | 198 +++++++++++++++++ .../CollectionExpressionsSuite.scala | 6 +- .../parser/ExpressionParserSuite.scala | 8 +- .../catalyst/util/IntervalUtilsSuite.scala | 63 +++++- 7 files changed, 276 insertions(+), 288 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 184ddac9a71a6..3c835519efd16 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -18,8 +18,6 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; -import java.util.regex.Matcher; -import java.util.regex.Pattern; /** * The internal representation of interval type. @@ -32,209 +30,6 @@ public final class CalendarInterval implements Serializable { public static final long MICROS_PER_DAY = MICROS_PER_HOUR * 24; public static final long MICROS_PER_WEEK = MICROS_PER_DAY * 7; - private static Pattern yearMonthPattern = Pattern.compile( - "^([+|-])?(\\d+)-(\\d+)$"); - - private static Pattern dayTimePattern = Pattern.compile( - "^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$"); - - public static long toLongWithRange(String fieldName, - String s, long minValue, long maxValue) throws IllegalArgumentException { - long result = 0; - if (s != null) { - result = Long.parseLong(s); - if (result < minValue || result > maxValue) { - throw new IllegalArgumentException(String.format("%s %d outside range [%d, %d]", - fieldName, result, minValue, maxValue)); - } - } - return result; - } - - /** - * Parse YearMonth string in form: [-]YYYY-MM - * - * adapted from HiveIntervalYearMonth.valueOf - */ - public static CalendarInterval fromYearMonthString(String s) throws IllegalArgumentException { - CalendarInterval result = null; - if (s == null) { - throw new IllegalArgumentException("Interval year-month string was null"); - } - s = s.trim(); - Matcher m = yearMonthPattern.matcher(s); - if (!m.matches()) { - throw new IllegalArgumentException( - "Interval string does not match year-month format of 'y-m': " + s); - } else { - try { - int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; - int years = (int) toLongWithRange("year", m.group(2), 0, Integer.MAX_VALUE); - int months = (int) toLongWithRange("month", m.group(3), 0, 11); - result = new CalendarInterval(sign * (years * 12 + months), 0); - } catch (Exception e) { - throw new IllegalArgumentException( - "Error parsing interval year-month string: " + e.getMessage(), e); - } - } - return result; - } - - /** - * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn - * - * adapted from HiveIntervalDayTime.valueOf - */ - public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumentException { - return fromDayTimeString(s, "day", "second"); - } - - /** - * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn - * - * adapted from HiveIntervalDayTime.valueOf. - * Below interval conversion patterns are supported: - * - DAY TO (HOUR|MINUTE|SECOND) - * - HOUR TO (MINUTE|SECOND) - * - MINUTE TO SECOND - */ - public static CalendarInterval fromDayTimeString(String s, String from, String to) - throws IllegalArgumentException { - CalendarInterval result = null; - if (s == null) { - throw new IllegalArgumentException("Interval day-time string was null"); - } - s = s.trim(); - Matcher m = dayTimePattern.matcher(s); - if (!m.matches()) { - throw new IllegalArgumentException( - "Interval string does not match day-time format of 'd h:m:s.n': " + s); - } else { - try { - int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; - long days = m.group(2) == null ? 0 : toLongWithRange("day", m.group(3), - 0, Integer.MAX_VALUE); - long hours = 0; - long minutes; - long seconds = 0; - if (m.group(5) != null || from.equals("minute")) { // 'HH:mm:ss' or 'mm:ss minute' - hours = toLongWithRange("hour", m.group(5), 0, 23); - minutes = toLongWithRange("minute", m.group(6), 0, 59); - seconds = toLongWithRange("second", m.group(7), 0, 59); - } else if (m.group(8) != null){ // 'mm:ss.nn' - minutes = toLongWithRange("minute", m.group(6), 0, 59); - seconds = toLongWithRange("second", m.group(7), 0, 59); - } else { // 'HH:mm' - hours = toLongWithRange("hour", m.group(6), 0, 23); - minutes = toLongWithRange("second", m.group(7), 0, 59); - } - // Hive allow nanosecond precision interval - String nanoStr = m.group(9) == null ? null : (m.group(9) + "000000000").substring(0, 9); - long nanos = toLongWithRange("nanosecond", nanoStr, 0L, 999999999L); - switch (to) { - case "hour": - minutes = 0; - seconds = 0; - nanos = 0; - break; - case "minute": - seconds = 0; - nanos = 0; - break; - case "second": - // No-op - break; - default: - throw new IllegalArgumentException( - String.format("Cannot support (interval '%s' %s to %s) expression", s, from, to)); - } - result = new CalendarInterval(0, sign * ( - days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + - seconds * MICROS_PER_SECOND + nanos / 1000L)); - } catch (Exception e) { - throw new IllegalArgumentException( - "Error parsing interval day-time string: " + e.getMessage(), e); - } - } - return result; - } - - public static CalendarInterval fromUnitStrings(String[] units, String[] values) - throws IllegalArgumentException { - assert units.length == values.length; - int months = 0; - long microseconds = 0; - - for (int i = 0; i < units.length; i++) { - try { - switch (units[i]) { - case "year": - months = Math.addExact(months, Math.multiplyExact(Integer.parseInt(values[i]), 12)); - break; - case "month": - months = Math.addExact(months, Integer.parseInt(values[i])); - break; - case "week": - microseconds = Math.addExact( - microseconds, - Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_WEEK)); - break; - case "day": - microseconds = Math.addExact( - microseconds, - Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_DAY)); - break; - case "hour": - microseconds = Math.addExact( - microseconds, - Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_HOUR)); - break; - case "minute": - microseconds = Math.addExact( - microseconds, - Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MINUTE)); - break; - case "second": { - microseconds = Math.addExact(microseconds, parseSecondNano(values[i])); - break; - } - case "millisecond": - microseconds = Math.addExact( - microseconds, - Math.multiplyExact(Long.parseLong(values[i]), MICROS_PER_MILLI)); - break; - case "microsecond": - microseconds = Math.addExact(microseconds, Long.parseLong(values[i])); - break; - } - } catch (Exception e) { - throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e); - } - } - return new CalendarInterval(months, microseconds); - } - - /** - * Parse second_nano string in ss.nnnnnnnnn format to microseconds - */ - public static long parseSecondNano(String secondNano) throws IllegalArgumentException { - String[] parts = secondNano.split("\\."); - if (parts.length == 1) { - return toLongWithRange("second", parts[0], Long.MIN_VALUE / MICROS_PER_SECOND, - Long.MAX_VALUE / MICROS_PER_SECOND) * MICROS_PER_SECOND; - - } else if (parts.length == 2) { - long seconds = parts[0].equals("") ? 0L : toLongWithRange("second", parts[0], - Long.MIN_VALUE / MICROS_PER_SECOND, Long.MAX_VALUE / MICROS_PER_SECOND); - long nanos = toLongWithRange("nanosecond", parts[1], 0L, 999999999L); - return seconds * MICROS_PER_SECOND + nanos / 1000L; - - } else { - throw new IllegalArgumentException( - "Interval string does not match second-nano format of ss.nnnnnnnnn"); - } - } - public final int months; public final long microseconds; diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 9f3262bf2aaa4..5e418c1c14a60 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -60,72 +60,6 @@ public void toStringTest() { assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); } - @Test - public void fromYearMonthStringTest() { - String input; - CalendarInterval i; - - input = "99-10"; - i = new CalendarInterval(99 * 12 + 10, 0L); - assertEquals(fromYearMonthString(input), i); - - input = "-8-10"; - i = new CalendarInterval(-8 * 12 - 10, 0L); - assertEquals(fromYearMonthString(input), i); - - try { - input = "99-15"; - fromYearMonthString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("month 15 outside range")); - } - } - - @Test - public void fromDayTimeStringTest() { - String input; - CalendarInterval i; - - input = "5 12:40:30.999999999"; - i = new CalendarInterval(0, 5 * MICROS_PER_DAY + 12 * MICROS_PER_HOUR + - 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L); - assertEquals(fromDayTimeString(input), i); - - input = "10 0:12:0.888"; - i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE + - 888 * MICROS_PER_MILLI); - assertEquals(fromDayTimeString(input), i); - - input = "-3 0:0:0"; - i = new CalendarInterval(0, -3 * MICROS_PER_DAY); - assertEquals(fromDayTimeString(input), i); - - try { - input = "5 30:12:20"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("hour 30 outside range")); - } - - try { - input = "5 30-12"; - fromDayTimeString(input); - fail("Expected to throw an exception for the invalid input"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("not match day-time format")); - } - - try { - input = "5 1:12:20"; - fromDayTimeString(input, "hour", "microsecond"); - fail("Expected to throw an exception for the invalid convention type"); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("Cannot support (interval")); - } - } - @Test public void addTest() { CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 893003d21828b..072b9a16e6d42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -108,7 +108,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging }.toArray val values = ctx.intervalValue().asScala.map(getIntervalValue).toArray try { - CalendarInterval.fromUnitStrings(units, values) + IntervalUtils.fromUnitStrings(units, values) } catch { case i: IllegalArgumentException => val e = new ParseException(i.getMessage, ctx) @@ -1953,21 +1953,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val unitText = unit.getText.toLowerCase(Locale.ROOT) val interval = (unitText, Option(to).map(_.getText.toLowerCase(Locale.ROOT))) match { case (u, None) => - CalendarInterval.fromUnitStrings(Array(normalizeInternalUnit(u)), Array(s)) + IntervalUtils.fromUnitStrings(Array(normalizeInternalUnit(u)), Array(s)) case ("year", Some("month")) => - CalendarInterval.fromYearMonthString(s) + IntervalUtils.fromYearMonthString(s) case ("day", Some("hour")) => - CalendarInterval.fromDayTimeString(s, "day", "hour") + IntervalUtils.fromDayTimeString(s, "day", "hour") case ("day", Some("minute")) => - CalendarInterval.fromDayTimeString(s, "day", "minute") + IntervalUtils.fromDayTimeString(s, "day", "minute") case ("day", Some("second")) => - CalendarInterval.fromDayTimeString(s, "day", "second") + IntervalUtils.fromDayTimeString(s, "day", "second") case ("hour", Some("minute")) => - CalendarInterval.fromDayTimeString(s, "hour", "minute") + IntervalUtils.fromDayTimeString(s, "hour", "minute") case ("hour", Some("second")) => - CalendarInterval.fromDayTimeString(s, "hour", "second") + IntervalUtils.fromDayTimeString(s, "hour", "second") case ("minute", Some("second")) => - CalendarInterval.fromDayTimeString(s, "minute", "second") + IntervalUtils.fromDayTimeString(s, "minute", "second") case (from, Some(t)) => throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 14fd153e15f58..f55b0545ee9cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -17,6 +17,10 @@ package org.apache.spark.sql.catalyst.util +import java.util.regex.Pattern + +import scala.util.control.NonFatal + import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval @@ -119,4 +123,198 @@ object IntervalUtils { case _: IllegalArgumentException => null } } + + private def toLongWithRange( + fieldName: String, + s: String, + minValue: Long, + maxValue: Long): Long = { + val result = if (s == null) 0L else s.toLong + require(minValue <= result && result <= maxValue, + s"$fieldName $result outside range [$minValue, $maxValue]") + + result + } + + private val yearMonthPattern = "^([+|-])?(\\d+)-(\\d+)$".r + + /** + * Parse YearMonth string in form: [+|-]YYYY-MM + * + * adapted from HiveIntervalYearMonth.valueOf + */ + def fromYearMonthString(input: String): CalendarInterval = { + require(input != null, "Interval year-month string must be not null") + def toInterval(yearStr: String, monthStr: String): CalendarInterval = { + try { + val years = toLongWithRange("year", yearStr, 0, Integer.MAX_VALUE).toInt + val months = toLongWithRange("month", monthStr, 0, 11).toInt + val totalMonths = Math.addExact(Math.multiplyExact(years, 12), months) + new CalendarInterval(totalMonths, 0) + } catch { + case NonFatal(e) => + throw new IllegalArgumentException( + s"Error parsing interval year-month string: ${e.getMessage}", e) + } + } + assert(input.length == input.trim.length) + input match { + case yearMonthPattern("-", yearStr, monthStr) => + toInterval(yearStr, monthStr).negate() + case yearMonthPattern(_, yearStr, monthStr) => + toInterval(yearStr, monthStr) + case _ => + throw new IllegalArgumentException( + s"Interval string does not match year-month format of 'y-m': $input") + } + } + + /** + * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn + * + * adapted from HiveIntervalDayTime.valueOf + */ + def fromDayTimeString(s: String): CalendarInterval = { + fromDayTimeString(s, "day", "second") + } + + private val dayTimePattern = + "^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$".r + + /** + * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn + * + * adapted from HiveIntervalDayTime.valueOf. + * Below interval conversion patterns are supported: + * - DAY TO (HOUR|MINUTE|SECOND) + * - HOUR TO (MINUTE|SECOND) + * - MINUTE TO SECOND + */ + def fromDayTimeString(input: String, from: String, to: String): CalendarInterval = { + require(input != null, "Interval day-time string must be not null") + assert(input.length == input.trim.length) + val m = dayTimePattern.pattern.matcher(input) + require(m.matches, s"Interval string must match day-time format of 'd h:m:s.n': $input") + + try { + val sign = if (m.group(1) != null && m.group(1) == "-") -1 else 1 + val days = if (m.group(2) == null) { + 0 + } else { + toLongWithRange("day", m.group(3), 0, Integer.MAX_VALUE) + } + var hours: Long = 0L + var minutes: Long = 0L + var seconds: Long = 0L + if (m.group(5) != null || from == "minute") { // 'HH:mm:ss' or 'mm:ss minute' + hours = toLongWithRange("hour", m.group(5), 0, 23) + minutes = toLongWithRange("minute", m.group(6), 0, 59) + seconds = toLongWithRange("second", m.group(7), 0, 59) + } else if (m.group(8) != null) { // 'mm:ss.nn' + minutes = toLongWithRange("minute", m.group(6), 0, 59) + seconds = toLongWithRange("second", m.group(7), 0, 59) + } else { // 'HH:mm' + hours = toLongWithRange("hour", m.group(6), 0, 23) + minutes = toLongWithRange("second", m.group(7), 0, 59) + } + // Hive allow nanosecond precision interval + val nanoStr = if (m.group(9) == null) { + null + } else { + (m.group(9) + "000000000").substring(0, 9) + } + var nanos = toLongWithRange("nanosecond", nanoStr, 0L, 999999999L) + to match { + case "hour" => + minutes = 0 + seconds = 0 + nanos = 0 + case "minute" => + seconds = 0 + nanos = 0 + case "second" => + // No-op + case _ => + throw new IllegalArgumentException( + s"Cannot support (interval '$input' $from to $to) expression") + } + var micros = nanos / DateTimeUtils.NANOS_PER_MICROS + micros = Math.addExact(micros, Math.multiplyExact(days, DateTimeUtils.MICROS_PER_DAY)) + micros = Math.addExact(micros, Math.multiplyExact(hours, MICROS_PER_HOUR)) + micros = Math.addExact(micros, Math.multiplyExact(minutes, MICROS_PER_MINUTE)) + micros = Math.addExact(micros, Math.multiplyExact(seconds, DateTimeUtils.MICROS_PER_SECOND)) + new CalendarInterval(0, sign * micros) + } catch { + case e: Exception => + throw new IllegalArgumentException( + s"Error parsing interval day-time string: ${e.getMessage}", e) + } + } + + def fromUnitStrings(units: Array[String], values: Array[String]): CalendarInterval = { + assert(units.length == values.length) + var months: Int = 0 + var microseconds: Long = 0 + var i = 0 + while (i < units.length) { + try { + units(i) match { + case "year" => + months = Math.addExact(months, Math.multiplyExact(values(i).toInt, 12)) + case "month" => + months = Math.addExact(months, values(i).toInt) + case "week" => + val weeksUs = Math.multiplyExact(values(i).toLong, 7 * DateTimeUtils.MICROS_PER_DAY) + microseconds = Math.addExact(microseconds, weeksUs) + case "day" => + val daysUs = Math.multiplyExact(values(i).toLong, DateTimeUtils.MICROS_PER_DAY) + microseconds = Math.addExact(microseconds, daysUs) + case "hour" => + val hoursUs = Math.multiplyExact(values(i).toLong, MICROS_PER_HOUR) + microseconds = Math.addExact(microseconds, hoursUs) + case "minute" => + val minutesUs = Math.multiplyExact(values(i).toLong, MICROS_PER_MINUTE) + microseconds = Math.addExact(microseconds, minutesUs) + case "second" => + microseconds = Math.addExact(microseconds, parseSecondNano(values(i))) + case "millisecond" => + val millisUs = Math.multiplyExact(values(i).toLong, DateTimeUtils.MICROS_PER_MILLIS) + microseconds = Math.addExact(microseconds, millisUs) + case "microsecond" => + microseconds = Math.addExact(microseconds, values(i).toLong) + } + } catch { + case e: Exception => + throw new IllegalArgumentException(s"Error parsing interval string: ${e.getMessage}", e) + } + i += 1 + } + new CalendarInterval(months, microseconds) + } + + /** + * Parse second_nano string in ss.nnnnnnnnn format to microseconds + */ + private def parseSecondNano(secondNano: String): Long = { + def parseSeconds(secondsStr: String): Long = { + toLongWithRange( + "second", + secondsStr, + Long.MinValue / DateTimeUtils.MICROS_PER_SECOND, + Long.MaxValue / DateTimeUtils.MICROS_PER_SECOND) * DateTimeUtils.MICROS_PER_SECOND + } + def parseNanos(nanosStr: String): Long = { + toLongWithRange("nanosecond", nanosStr, 0L, 999999999L) / DateTimeUtils.NANOS_PER_MICROS + } + + secondNano.split("\\.") match { + case Array(secondsStr) => parseSeconds(secondsStr) + case Array("", nanosStr) => parseNanos(nanosStr) + case Array(secondsStr, nanosStr) => + Math.addExact(parseSeconds(secondsStr), parseNanos(nanosStr)) + case _ => + throw new IllegalArgumentException( + "Interval string does not match second-nano format of ss.nnnnnnnnn") + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index e10aa60d52cf8..74d1606be2b52 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -810,7 +810,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2023-01-01 00:00:00")), - Literal(CalendarInterval.fromYearMonthString("1-5"))), + Literal(IntervalUtils.fromYearMonthString("1-5"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00.000"), Timestamp.valueOf("2019-06-01 00:00:00.000"), @@ -820,7 +820,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2022-04-01 00:00:00")), Literal(Timestamp.valueOf("2017-01-01 00:00:00")), - Literal(CalendarInterval.fromYearMonthString("1-5").negate())), + Literal(IntervalUtils.fromYearMonthString("1-5").negate())), Seq( Timestamp.valueOf("2022-04-01 00:00:00.000"), Timestamp.valueOf("2020-11-01 00:00:00.000"), @@ -894,7 +894,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2023-01-01")), - Literal(CalendarInterval.fromYearMonthString("1-5"))), + Literal(IntervalUtils.fromYearMonthString("1-5"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2019-06-01"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 86b3aa8190b45..5a7b3ffec53f7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -597,7 +597,7 @@ class ExpressionParserSuite extends AnalysisTest { "microsecond") def intervalLiteral(u: String, s: String): Literal = { - Literal(CalendarInterval.fromUnitStrings(Array(u), Array(s))) + Literal(IntervalUtils.fromUnitStrings(Array(u), Array(s))) } test("intervals") { @@ -637,7 +637,7 @@ class ExpressionParserSuite extends AnalysisTest { // Year-Month intervals. val yearMonthValues = Seq("123-10", "496-0", "-2-3", "-123-0") yearMonthValues.foreach { value => - val result = Literal(CalendarInterval.fromYearMonthString(value)) + val result = Literal(IntervalUtils.fromYearMonthString(value)) checkIntervals(s"'$value' year to month", result) } @@ -650,7 +650,7 @@ class ExpressionParserSuite extends AnalysisTest { "-1 0:0:0", "1 0:0:1") datTimeValues.foreach { value => - val result = Literal(CalendarInterval.fromDayTimeString(value)) + val result = Literal(IntervalUtils.fromDayTimeString(value)) checkIntervals(s"'$value' day to second", result) } @@ -662,7 +662,7 @@ class ExpressionParserSuite extends AnalysisTest { "0:0:0", "0:0:1") hourTimeValues.foreach { value => - val result = Literal(CalendarInterval.fromDayTimeString(value)) + val result = Literal(IntervalUtils.fromDayTimeString(value)) checkIntervals(s"'$value' hour to second", result) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index e48779af3c9aa..9addc396b8d33 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.IntervalUtils.fromString +import org.apache.spark.sql.catalyst.util.IntervalUtils.{fromDayTimeString, fromString, fromYearMonthString} import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.unsafe.types.CalendarInterval._ @@ -87,4 +87,65 @@ class IntervalUtilsSuite extends SparkFunSuite { assert(fromString(input2) == result) } } + + test("from year-month string") { + assert(fromYearMonthString("99-10") === new CalendarInterval(99 * 12 + 10, 0L)) + assert(fromYearMonthString("+99-10") === new CalendarInterval(99 * 12 + 10, 0L)) + assert(fromYearMonthString("-8-10") === new CalendarInterval(-8 * 12 - 10, 0L)) + + try { + fromYearMonthString("99-15") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("month 15 outside range")) + } + + try { + fromYearMonthString("9a9-15") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("Interval string does not match year-month format")) + } + } + + test("from day-time string") { + assert(fromDayTimeString("5 12:40:30.999999999") === + new CalendarInterval( + 0, + 5 * MICROS_PER_DAY + + 12 * MICROS_PER_HOUR + + 40 * MICROS_PER_MINUTE + + 30 * MICROS_PER_SECOND + 999999L)) + assert(fromDayTimeString("10 0:12:0.888") === + new CalendarInterval( + 0, + 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE + 888 * MICROS_PER_MILLI)) + assert(fromDayTimeString("-3 0:0:0") === new CalendarInterval(0, -3 * MICROS_PER_DAY)) + + try { + fromDayTimeString("5 30:12:20") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("hour 30 outside range")) + } + + try { + fromDayTimeString("5 30-12") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("must match day-time format")) + } + + try { + fromDayTimeString("5 1:12:20", "hour", "microsecond") + fail("Expected to throw an exception for the invalid convention type") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("Cannot support (interval")) + } + } } From 9c817a83fce4a2260f9da86b159d4c59632e974c Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 29 Oct 2019 12:00:52 -0700 Subject: [PATCH 0457/1387] [SPARK-29637][CORE] Add description to Job SHS web API ### Why are the changes needed? Starting from Spark 2.3, the SHS REST API endpoint `/applications//jobs/` is not including `description` in the JobData returned. This is not the case until Spark 2.2. In this PR I've added the mentioned field. ### Does this PR introduce any user-facing change? Yes. Old API response: ``` [ { "jobId" : 0, "name" : "foreach at :26", "submissionTime" : "2019-10-28T12:41:54.301GMT", "completionTime" : "2019-10-28T12:41:54.731GMT", "stageIds" : [ 0 ], "jobGroup" : "test", "status" : "SUCCEEDED", "numTasks" : 1, "numActiveTasks" : 0, "numCompletedTasks" : 1, "numSkippedTasks" : 0, "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 1, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, "numFailedStages" : 0, "killedTasksSummary" : { } } ] ``` New API response: ``` [ { "jobId" : 0, "name" : "foreach at :26", "description" : "job", <= This is the addition here "submissionTime" : "2019-10-28T13:37:24.107GMT", "completionTime" : "2019-10-28T13:37:24.613GMT", "stageIds" : [ 0 ], "jobGroup" : "test", "status" : "SUCCEEDED", "numTasks" : 1, "numActiveTasks" : 0, "numCompletedTasks" : 1, "numSkippedTasks" : 0, "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 1, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, "numFailedStages" : 0, "killedTasksSummary" : { } } ] ``` ### How was this patch tested? Extended + existing unit tests. Manually: * Open spark-shell ``` scala> sc.setJobGroup("test", "job", false); scala> val foo = sc.textFile("/user/foo.txt"); foo: org.apache.spark.rdd.RDD[String] = /user/foo.txt MapPartitionsRDD[1] at textFile at :24 scala> foo.foreach(println); ``` * Access REST API `http://SHS-host:port/api/v1/applications//jobs/` Closes #26295 from gaborgsomogyi/SPARK-29637. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../main/scala/org/apache/spark/status/AppStatusListener.scala | 3 +++ core/src/main/scala/org/apache/spark/status/LiveEntity.scala | 3 ++- .../scala/org/apache/spark/status/AppStatusListenerSuite.scala | 3 ++- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 7da0a9d2285b5..a5850fc2ac4b9 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -355,6 +355,8 @@ private[spark] class AppStatusListener( val lastStageInfo = event.stageInfos.sortBy(_.stageId).lastOption val jobName = lastStageInfo.map(_.name).getOrElse("") + val description = Option(event.properties) + .flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) } val jobGroup = Option(event.properties) .flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) } val sqlExecutionId = Option(event.properties) @@ -363,6 +365,7 @@ private[spark] class AppStatusListener( val job = new LiveJob( event.jobId, jobName, + description, if (event.time > 0) Some(new Date(event.time)) else None, event.stageIds, jobGroup, diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 00c991b499203..a0ef8da0a4b6b 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -62,6 +62,7 @@ private[spark] abstract class LiveEntity { private class LiveJob( val jobId: Int, name: String, + description: Option[String], val submissionTime: Option[Date], val stageIds: Seq[Int], jobGroup: Option[String], @@ -92,7 +93,7 @@ private class LiveJob( val info = new v1.JobData( jobId, name, - None, // description is always None? + description, submissionTime, completionTime, stageIds, diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 6bf163506e0ce..a289dddbdc9e6 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -155,6 +155,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2")) val jobProps = new Properties() + jobProps.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, "jobDescription") jobProps.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "jobGroup") jobProps.setProperty(SparkContext.SPARK_SCHEDULER_POOL, "schedPool") @@ -163,7 +164,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { check[JobDataWrapper](1) { job => assert(job.info.jobId === 1) assert(job.info.name === stages.last.name) - assert(job.info.description === None) + assert(job.info.description === Some("jobDescription")) assert(job.info.status === JobExecutionStatus.RUNNING) assert(job.info.submissionTime === Some(new Date(time))) assert(job.info.jobGroup === Some("jobGroup")) From 6958d7e62959313fb6a1288527aaa611aebee087 Mon Sep 17 00:00:00 2001 From: ulysses Date: Wed, 30 Oct 2019 08:25:34 +0900 Subject: [PATCH 0458/1387] [SPARK-28746][SQL] Add partitionby hint for sql queries ## What changes were proposed in this pull request? Now, `RepartitionByExpression` is allowed at Dataset method `Dataset.repartition()`. But in spark sql, we do not have an equivalent functionality. In hive, we can use `distribute by`, so it's worth to add a hint to support such function. Similar jira [SPARK-24940](https://issues.apache.org/jira/browse/SPARK-24940) ## Why are the changes needed? Make repartition hints consistent with repartition api . ## Does this PR introduce any user-facing change? This pr intends to support quries below; ``` // SQL cases - sql("SELECT /*+ REPARTITION(c) */ * FROM t") - sql("SELECT /*+ REPARTITION(1, c) */ * FROM t") - sql("SELECT /*+ REPARTITION_BY_RANGE(c) */ * FROM t") - sql("SELECT /*+ REPARTITION_BY_RANGE(1, c) */ * FROM t") ``` ## How was this patch tested? UT Closes #25464 from ulysses-you/SPARK-28746. Lead-authored-by: ulysses Co-authored-by: ulysses <646303253@qq.com> Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/analysis/ResolveHints.scala | 108 +++++++++++++++--- .../catalyst/analysis/ResolveHintsSuite.scala | 81 +++++++++++-- .../sql/catalyst/parser/PlanParserSuite.scala | 46 ++++++++ .../apache/spark/sql/DataFrameHintSuite.scala | 12 ++ 5 files changed, 220 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 21bf926af50d7..d92987887b70c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -186,7 +186,7 @@ class Analyzer( lazy val batches: Seq[Batch] = Seq( Batch("Hints", fixedPoint, new ResolveHints.ResolveJoinStrategyHints(conf), - ResolveHints.ResolveCoalesceHints), + new ResolveHints.ResolveCoalesceHints(conf)), Batch("Simple Sanity Check", Once, LookupFunctions), Batch("Substitution", fixedPoint, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index 3a9c4b7392e38..d904ba3aca5d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -22,7 +22,7 @@ import java.util.Locale import scala.collection.mutable import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.IntegerLiteral +import org.apache.spark.sql.catalyst.expressions.{Ascending, Expression, IntegerLiteral, SortOrder} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.CurrentOrigin @@ -137,31 +137,101 @@ object ResolveHints { } /** - * COALESCE Hint accepts name "COALESCE" and "REPARTITION". - * Its parameter includes a partition number. + * COALESCE Hint accepts names "COALESCE", "REPARTITION", and "REPARTITION_BY_RANGE". */ - object ResolveCoalesceHints extends Rule[LogicalPlan] { - private val COALESCE_HINT_NAMES = Set("COALESCE", "REPARTITION") + class ResolveCoalesceHints(conf: SQLConf) extends Rule[LogicalPlan] { + + /** + * This function handles hints for "COALESCE" and "REPARTITION". + * The "COALESCE" hint only has a partition number as a parameter. The "REPARTITION" hint + * has a partition number, columns, or both of them as parameters. + */ + private def createRepartition( + shuffle: Boolean, hint: UnresolvedHint): LogicalPlan = { + val hintName = hint.name.toUpperCase(Locale.ROOT) + + def createRepartitionByExpression( + numPartitions: Int, partitionExprs: Seq[Any]): RepartitionByExpression = { + val sortOrders = partitionExprs.filter(_.isInstanceOf[SortOrder]) + if (sortOrders.nonEmpty) throw new IllegalArgumentException( + s"""Invalid partitionExprs specified: $sortOrders + |For range partitioning use REPARTITION_BY_RANGE instead. + """.stripMargin) + val invalidParams = partitionExprs.filter(!_.isInstanceOf[UnresolvedAttribute]) + if (invalidParams.nonEmpty) { + throw new AnalysisException(s"$hintName Hint parameter should include columns, but " + + s"${invalidParams.mkString(", ")} found") + } + RepartitionByExpression( + partitionExprs.map(_.asInstanceOf[Expression]), hint.child, numPartitions) + } - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { - case h: UnresolvedHint if COALESCE_HINT_NAMES.contains(h.name.toUpperCase(Locale.ROOT)) => - val hintName = h.name.toUpperCase(Locale.ROOT) - val shuffle = hintName match { - case "REPARTITION" => true - case "COALESCE" => false + hint.parameters match { + case Seq(IntegerLiteral(numPartitions)) => + Repartition(numPartitions, shuffle, hint.child) + case Seq(numPartitions: Int) => + Repartition(numPartitions, shuffle, hint.child) + // The "COALESCE" hint (shuffle = false) must have a partition number only + case _ if !shuffle => + throw new AnalysisException(s"$hintName Hint expects a partition number as a parameter") + + case param @ Seq(IntegerLiteral(numPartitions), _*) if shuffle => + createRepartitionByExpression(numPartitions, param.tail) + case param @ Seq(numPartitions: Int, _*) if shuffle => + createRepartitionByExpression(numPartitions, param.tail) + case param @ Seq(_*) if shuffle => + createRepartitionByExpression(conf.numShufflePartitions, param) + } + } + + /** + * This function handles hints for "REPARTITION_BY_RANGE". + * The "REPARTITION_BY_RANGE" hint must have column names and a partition number is optional. + */ + private def createRepartitionByRange(hint: UnresolvedHint): RepartitionByExpression = { + val hintName = hint.name.toUpperCase(Locale.ROOT) + + def createRepartitionByExpression( + numPartitions: Int, partitionExprs: Seq[Any]): RepartitionByExpression = { + val invalidParams = partitionExprs.filter(!_.isInstanceOf[UnresolvedAttribute]) + if (invalidParams.nonEmpty) { + throw new AnalysisException(s"$hintName Hint parameter should include columns, but " + + s"${invalidParams.mkString(", ")} found") } - val numPartitions = h.parameters match { - case Seq(IntegerLiteral(numPartitions)) => - numPartitions - case Seq(numPartitions: Int) => - numPartitions - case _ => - throw new AnalysisException(s"$hintName Hint expects a partition number as parameter") + val sortOrder = partitionExprs.map { + case expr: SortOrder => expr + case expr: Expression => SortOrder(expr, Ascending) + } + RepartitionByExpression(sortOrder, hint.child, numPartitions) + } + + hint.parameters match { + case param @ Seq(IntegerLiteral(numPartitions), _*) => + createRepartitionByExpression(numPartitions, param.tail) + case param @ Seq(numPartitions: Int, _*) => + createRepartitionByExpression(numPartitions, param.tail) + case param @ Seq(_*) => + createRepartitionByExpression(conf.numShufflePartitions, param) + } + } + + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { + case hint @ UnresolvedHint(hintName, _, _) => hintName.toUpperCase(Locale.ROOT) match { + case "REPARTITION" => + createRepartition(shuffle = true, hint) + case "COALESCE" => + createRepartition(shuffle = false, hint) + case "REPARTITION_BY_RANGE" => + createRepartitionByRange(hint) + case _ => plan } - Repartition(numPartitions, shuffle, h.child) } } + object ResolveCoalesceHints { + val COALESCE_HINT_NAMES: Set[String] = Set("COALESCE", "REPARTITION", "REPARTITION_BY_RANGE") + } + /** * Removes all the hints, used to remove invalid hints provided by the user. * This must be executed after all the other hint rules are executed. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index 474e58a335e7c..cddcddd51e38d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -24,10 +24,11 @@ import org.apache.log4j.spi.LoggingEvent import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, Literal, SortOrder} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.types.IntegerType class ResolveHintsSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.analysis.TestRelations._ @@ -150,24 +151,86 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint("RePARTITion", Seq(Literal(200)), table("TaBlE")), Repartition(numPartitions = 200, shuffle = true, child = testRelation)) - val errMsgCoal = "COALESCE Hint expects a partition number as parameter" + val errMsg = "COALESCE Hint expects a partition number as a parameter" + assertAnalysisError( UnresolvedHint("COALESCE", Seq.empty, table("TaBlE")), - Seq(errMsgCoal)) + Seq(errMsg)) assertAnalysisError( UnresolvedHint("COALESCE", Seq(Literal(10), Literal(false)), table("TaBlE")), - Seq(errMsgCoal)) + Seq(errMsg)) assertAnalysisError( UnresolvedHint("COALESCE", Seq(Literal(1.0)), table("TaBlE")), - Seq(errMsgCoal)) + Seq(errMsg)) - val errMsgRepa = "REPARTITION Hint expects a partition number as parameter" - assertAnalysisError( + checkAnalysis( + UnresolvedHint("RePartition", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), + RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testRelation, 10)) + + checkAnalysis( + UnresolvedHint("REPARTITION", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), + RepartitionByExpression(Seq(AttributeReference("a", IntegerType)()), testRelation, 10)) + + checkAnalysis( UnresolvedHint("REPARTITION", Seq(UnresolvedAttribute("a")), table("TaBlE")), - Seq(errMsgRepa)) + RepartitionByExpression( + Seq(AttributeReference("a", IntegerType)()), testRelation, conf.numShufflePartitions)) + + val e = intercept[IllegalArgumentException] { + checkAnalysis( + UnresolvedHint("REPARTITION", + Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), + table("TaBlE")), + RepartitionByExpression( + Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testRelation, 10) + ) + } + e.getMessage.contains("For range partitioning use REPARTITION_BY_RANGE instead") + + checkAnalysis( + UnresolvedHint( + "REPARTITION_BY_RANGE", Seq(Literal(10), UnresolvedAttribute("a")), table("TaBlE")), + RepartitionByExpression( + Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), testRelation, 10)) + + checkAnalysis( + UnresolvedHint( + "REPARTITION_BY_RANGE", Seq(UnresolvedAttribute("a")), table("TaBlE")), + RepartitionByExpression( + Seq(SortOrder(AttributeReference("a", IntegerType)(), Ascending)), + testRelation, conf.numShufflePartitions)) + + val errMsg2 = "REPARTITION Hint parameter should include columns, but" + assertAnalysisError( UnresolvedHint("REPARTITION", Seq(Literal(true)), table("TaBlE")), - Seq(errMsgRepa)) + Seq(errMsg2)) + + assertAnalysisError( + UnresolvedHint("REPARTITION", + Seq(Literal(1.0), AttributeReference("a", IntegerType)()), + table("TaBlE")), + Seq(errMsg2)) + + val errMsg3 = "REPARTITION_BY_RANGE Hint parameter should include columns, but" + + assertAnalysisError( + UnresolvedHint("REPARTITION_BY_RANGE", + Seq(Literal(1.0), AttributeReference("a", IntegerType)()), + table("TaBlE")), + Seq(errMsg3)) + + assertAnalysisError( + UnresolvedHint("REPARTITION_BY_RANGE", + Seq(Literal(10), Literal(10)), + table("TaBlE")), + Seq(errMsg3)) + + assertAnalysisError( + UnresolvedHint("REPARTITION_BY_RANGE", + Seq(Literal(10), Literal(10), UnresolvedAttribute("a")), + table("TaBlE")), + Seq(errMsg3)) } test("log warnings for invalid hints") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 8b940a7aa2c3b..875096f615241 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -624,6 +624,52 @@ class PlanParserSuite extends AnalysisTest { table("t").select(star())))) intercept("SELECT /*+ COALESCE(30 + 50) */ * FROM t", "mismatched input") + + comparePlans( + parsePlan("SELECT /*+ REPARTITION(c) */ * FROM t"), + UnresolvedHint("REPARTITION", Seq(UnresolvedAttribute("c")), + table("t").select(star()))) + + comparePlans( + parsePlan("SELECT /*+ REPARTITION(100, c) */ * FROM t"), + UnresolvedHint("REPARTITION", Seq(Literal(100), UnresolvedAttribute("c")), + table("t").select(star()))) + + comparePlans( + parsePlan("SELECT /*+ REPARTITION(100, c), COALESCE(50) */ * FROM t"), + UnresolvedHint("REPARTITION", Seq(Literal(100), UnresolvedAttribute("c")), + UnresolvedHint("COALESCE", Seq(Literal(50)), + table("t").select(star())))) + + comparePlans( + parsePlan("SELECT /*+ REPARTITION(100, c), BROADCASTJOIN(u), COALESCE(50) */ * FROM t"), + UnresolvedHint("REPARTITION", Seq(Literal(100), UnresolvedAttribute("c")), + UnresolvedHint("BROADCASTJOIN", Seq($"u"), + UnresolvedHint("COALESCE", Seq(Literal(50)), + table("t").select(star()))))) + + comparePlans( + parsePlan( + """ + |SELECT + |/*+ REPARTITION(100, c), BROADCASTJOIN(u), COALESCE(50), REPARTITION(300, c) */ + |* FROM t + """.stripMargin), + UnresolvedHint("REPARTITION", Seq(Literal(100), UnresolvedAttribute("c")), + UnresolvedHint("BROADCASTJOIN", Seq($"u"), + UnresolvedHint("COALESCE", Seq(Literal(50)), + UnresolvedHint("REPARTITION", Seq(Literal(300), UnresolvedAttribute("c")), + table("t").select(star())))))) + + comparePlans( + parsePlan("SELECT /*+ REPARTITION_BY_RANGE(c) */ * FROM t"), + UnresolvedHint("REPARTITION_BY_RANGE", Seq(UnresolvedAttribute("c")), + table("t").select(star()))) + + comparePlans( + parsePlan("SELECT /*+ REPARTITION_BY_RANGE(100, c) */ * FROM t"), + UnresolvedHint("REPARTITION_BY_RANGE", Seq(Literal(100), UnresolvedAttribute("c")), + table("t").select(star()))) } test("SPARK-20854: select hint syntax with expressions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala index b33c26a0b75a2..37dc8f1bcc7f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala @@ -68,5 +68,17 @@ class DataFrameHintSuite extends AnalysisTest with SharedSparkSession { check( df.hint("REPARTITION", 100), UnresolvedHint("REPARTITION", Seq(100), df.logicalPlan)) + + check( + df.hint("REPARTITION", 10, $"id".expr), + UnresolvedHint("REPARTITION", Seq(10, $"id".expr), df.logicalPlan)) + + check( + df.hint("REPARTITION_BY_RANGE", $"id".expr), + UnresolvedHint("REPARTITION_BY_RANGE", Seq($"id".expr), df.logicalPlan)) + + check( + df.hint("REPARTITION_BY_RANGE", 10, $"id".expr), + UnresolvedHint("REPARTITION_BY_RANGE", Seq(10, $"id".expr), df.logicalPlan)) } } From 8e667db5d801bd1c74a75abbea62f2a0007c950b Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 30 Oct 2019 09:40:54 +0900 Subject: [PATCH 0459/1387] [SPARK-29629][SQL] Support typed integer literal expression ### What changes were proposed in this pull request? ``` postgres=# select date '2001-09-28' + integer '7'; ?column? ------------ 2001-10-05 (1 row)postgres=# select integer '7'; int4 ------ 7 (1 row) ``` Add support for typed integer literal expression from postgreSQL. ### Why are the changes needed? SPARK-27764 Feature Parity between PostgreSQL and Spark ### Does this PR introduce any user-facing change? support typed integer lit in SQL ### How was this patch tested? add uts Closes #26291 from yaooqinn/SPARK-29629. Authored-by: Kent Yao Signed-off-by: HyukjinKwon --- .../sql/catalyst/parser/AstBuilder.scala | 12 ++++++- .../resources/sql-tests/inputs/literals.sql | 5 +++ .../sql-tests/results/literals.sql.out | 32 ++++++++++++++++++- 3 files changed, 47 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 072b9a16e6d42..c9ecbdb658f4f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1769,7 +1769,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * {{{ * [TYPE] '[VALUE]' * }}} - * Currently Date, Timestamp, Interval and Binary typed literals are supported. + * Currently Date, Timestamp, Interval, Binary and INTEGER typed literals are supported. */ override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { val value = string(ctx.STRING) @@ -1799,6 +1799,16 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case "X" => val padding = if (value.length % 2 != 0) "0" else "" Literal(DatatypeConverter.parseHexBinary(padding + value)) + case "INTEGER" => + val i = try { + value.toInt + } catch { + case e: NumberFormatException => + val ex = new ParseException(s"Cannot parse the Int value: $value, $e", ctx) + ex.setStackTrace(e.getStackTrace) + throw ex + } + Literal(i, IntegerType) case other => throw new ParseException(s"Literals of type '$other' are currently not supported.", ctx) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 0f95f85237828..389b9cc53e02e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -125,3 +125,8 @@ select map(1, interval 1 day, 2, interval 3 week); -- typed interval expression select interval 'interval 3 year 1 hour'; select interval '3 year 1 hour'; + +-- typed integer expression +select integer '7'; +select integer'7'; +select integer '2147483648'; diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index fd6e51b2385de..115287821bf4f 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 59 +-- Number of queries: 62 -- !query 0 @@ -542,3 +542,33 @@ select interval '3 year 1 hour' struct -- !query 58 output interval 3 years 1 hours + + +-- !query 59 +select integer '7' +-- !query 59 schema +struct<7:int> +-- !query 59 output +7 + + +-- !query 60 +select integer'7' +-- !query 60 schema +struct<7:int> +-- !query 60 output +7 + + +-- !query 61 +select integer '2147483648' +-- !query 61 schema +struct<> +-- !query 61 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the Int value: 2147483648, java.lang.NumberFormatException: For input string: "2147483648"(line 1, pos 7) + +== SQL == +select integer '2147483648' +-------^^^ From 8682bb11ae72c89b84d4bb26b835736f5a788280 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 30 Oct 2019 09:45:19 +0900 Subject: [PATCH 0460/1387] [SPARK-29627][PYTHON][SQL] Allow array_contains to take column instances ### What changes were proposed in this pull request? This PR proposes to allow `array_contains` to take column instances. ### Why are the changes needed? For consistent support in Scala and Python APIs. Scala allows column instances at `array_contains` Scala: ```scala import org.apache.spark.sql.functions._ val df = Seq(Array("a", "b", "c"), Array.empty[String]).toDF("data") df.select(array_contains($"data", lit("a"))).show() ``` Python: ```python from pyspark.sql.functions import array_contains, lit df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) df.select(array_contains(df.data, lit("a"))).show() ``` However, PySpark sides does not allow. ### Does this PR introduce any user-facing change? Yes. ```python from pyspark.sql.functions import array_contains, lit df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) df.select(array_contains(df.data, lit("a"))).show() ``` **Before:** ``` Traceback (most recent call last): File "", line 1, in File "/.../spark/python/pyspark/sql/functions.py", line 1950, in array_contains return Column(sc._jvm.functions.array_contains(_to_java_column(col), value)) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1277, in __call__ File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1241, in _build_args File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1228, in _get_args File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_collections.py", line 500, in convert File "/.../spark/python/pyspark/sql/column.py", line 344, in __iter__ raise TypeError("Column is not iterable") TypeError: Column is not iterable ``` **After:** ``` +-----------------------+ |array_contains(data, a)| +-----------------------+ | true| | false| +-----------------------+ ``` ### How was this patch tested? Manually tested and added a doctest. Closes #26288 from HyukjinKwon/SPARK-29627. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/functions.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 95b78175d5561..9186d77f7876d 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1940,13 +1940,16 @@ def array_contains(col, value): given value, and false otherwise. :param col: name of column containing array - :param value: value to check for in array + :param value: value or column to check for in array >>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) >>> df.select(array_contains(df.data, "a")).collect() [Row(array_contains(data, a)=True), Row(array_contains(data, a)=False)] + >>> df.select(array_contains(df.data, lit("a"))).collect() + [Row(array_contains(data, a)=True), Row(array_contains(data, a)=False)] """ sc = SparkContext._active_spark_context + value = value._jc if isinstance(value, Column) else value return Column(sc._jvm.functions.array_contains(_to_java_column(col), value)) From 1bf65d97ac6f85d77572be62881e21372170d28f Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Wed, 30 Oct 2019 15:22:44 +0900 Subject: [PATCH 0461/1387] [SPARK-29110][SQL][TESTS] Port window.sql (Part 4) ### What changes were proposed in this pull request? This PR ports window.sql from PostgreSQL regression tests https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L913-L1278 The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/expected/window.out ### Why are the changes needed? To ensure compatibility with PostgreSQL. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass the Jenkins. And, Comparison with PgSQL results. Closes #26238 from DylanGuedes/spark-29110. Authored-by: DylanGuedes Signed-off-by: HyukjinKwon --- .../inputs/postgreSQL/window_part4.sql | 399 ++++++++++++++ .../results/postgreSQL/window_part4.sql.out | 507 ++++++++++++++++++ 2 files changed, 906 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql new file mode 100644 index 0000000000000..456b390fca6c3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql @@ -0,0 +1,399 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L913-L1278 + +-- Spark doesn't handle UDFs in SQL +-- test user-defined window function with named args and default args +-- CREATE FUNCTION nth_value_def(val anyelement, n integer = 1) RETURNS anyelement +-- LANGUAGE internal WINDOW IMMUTABLE STRICT AS 'window_nth_value'; + +-- Spark doesn't handle UDFs in SQL +-- SELECT nth_value_def(n := 2, val := ten) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten) s; + +-- Spark doesn't handle UDFs in SQL +-- SELECT nth_value_def(ten) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten) s; + +-- +-- Test the basic moving-aggregate machinery +-- + +-- create aggregates that record the series of transform calls (these are +-- intentionally not true inverses) + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_sfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT COALESCE($1, '') || '*' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_msfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT COALESCE($1, '') || '+' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_minvfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '-' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_nonstrict (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_nonstrict, +-- mstype = text, +-- msfunc = logging_msfunc_nonstrict, +-- minvfunc = logging_minvfunc_nonstrict +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_nonstrict_initcond (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_nonstrict, +-- mstype = text, +-- msfunc = logging_msfunc_nonstrict, +-- minvfunc = logging_minvfunc_nonstrict, +-- initcond = 'I', +-- minitcond = 'MI' +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_sfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '*' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_msfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '+' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_minvfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '-' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_strict (text) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_strict, +-- mstype = text, +-- msfunc = logging_msfunc_strict, +-- minvfunc = logging_minvfunc_strict +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_strict_initcond (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_strict, +-- mstype = text, +-- msfunc = logging_msfunc_strict, +-- minvfunc = logging_minvfunc_strict, +-- initcond = 'I', +-- minitcond = 'MI' +-- ); + +-- Spark doesn't handle UDFs in SQL +-- test strict and non-strict cases +-- SELECT +-- p::text || ',' || i::text || ':' || COALESCE(v::text, 'NULL') AS row, +-- logging_agg_nonstrict(v) over wnd as nstrict, +-- logging_agg_nonstrict_initcond(v) over wnd as nstrict_init, +-- logging_agg_strict(v::text) over wnd as strict, +-- logging_agg_strict_initcond(v) over wnd as strict_init +-- FROM (VALUES +-- (1, 1, NULL), +-- (1, 2, 'a'), +-- (1, 3, 'b'), +-- (1, 4, NULL), +-- (1, 5, NULL), +-- (1, 6, 'c'), +-- (2, 1, NULL), +-- (2, 2, 'x'), +-- (3, 1, 'z') +-- ) AS t(p, i, v) +-- WINDOW wnd AS (PARTITION BY P ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY p, i; + +-- Spark doesn't handle UDFs in SQL +-- and again, but with filter +-- SELECT +-- p::text || ',' || i::text || ':' || +-- CASE WHEN f THEN COALESCE(v::text, 'NULL') ELSE '-' END as row, +-- logging_agg_nonstrict(v) filter(where f) over wnd as nstrict_filt, +-- logging_agg_nonstrict_initcond(v) filter(where f) over wnd as nstrict_init_filt, +-- logging_agg_strict(v::text) filter(where f) over wnd as strict_filt, +-- logging_agg_strict_initcond(v) filter(where f) over wnd as strict_init_filt +-- FROM (VALUES +-- (1, 1, true, NULL), +-- (1, 2, false, 'a'), +-- (1, 3, true, 'b'), +-- (1, 4, false, NULL), +-- (1, 5, false, NULL), +-- (1, 6, false, 'c'), +-- (2, 1, false, NULL), +-- (2, 2, true, 'x'), +-- (3, 1, true, 'z') +-- ) AS t(p, i, f, v) +-- WINDOW wnd AS (PARTITION BY p ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY p, i; + +-- Spark doesn't handle UDFs in SQL +-- test that volatile arguments disable moving-aggregate mode +-- SELECT +-- i::text || ':' || COALESCE(v::text, 'NULL') as row, +-- logging_agg_strict(v::text) +-- over wnd as inverse, +-- logging_agg_strict(v::text || CASE WHEN random() < 0 then '?' ELSE '' END) +-- over wnd as noinverse +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- SELECT +-- i::text || ':' || COALESCE(v::text, 'NULL') as row, +-- logging_agg_strict(v::text) filter(where true) +-- over wnd as inverse, +-- logging_agg_strict(v::text) filter(where random() >= 0) +-- over wnd as noinverse +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- test that non-overlapping windows don't use inverse transitions +-- SELECT +-- logging_agg_strict(v::text) OVER wnd +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- test that returning NULL from the inverse transition functions +-- restarts the aggregation from scratch. The second aggregate is supposed +-- to test cases where only some aggregates restart, the third one checks +-- that one aggregate restarting doesn't cause others to restart. + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION sum_int_randrestart_minvfunc(int4, int4) RETURNS int4 AS +-- $$ SELECT CASE WHEN random() < 0.2 THEN NULL ELSE $1 - $2 END $$ +-- LANGUAGE SQL STRICT; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE sum_int_randomrestart (int4) +-- ( +-- stype = int4, +-- sfunc = int4pl, +-- mstype = int4, +-- msfunc = int4pl, +-- minvfunc = sum_int_randrestart_minvfunc +-- ); + +-- Spark doesn't handle UDFs in SQL +-- WITH +-- vs AS ( +-- SELECT i, (random() * 100)::int4 AS v +-- FROM generate_series(1, 100) AS i +-- ), +-- sum_following AS ( +-- SELECT i, SUM(v) OVER +-- (ORDER BY i DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS s +-- FROM vs +-- ) +-- SELECT DISTINCT +-- sum_following.s = sum_int_randomrestart(v) OVER fwd AS eq1, +-- -sum_following.s = sum_int_randomrestart(-v) OVER fwd AS eq2, +-- 100*3+(vs.i-1)*3 = length(logging_agg_nonstrict(''::text) OVER fwd) AS eq3 +-- FROM vs +-- JOIN sum_following ON sum_following.i = vs.i +-- WINDOW fwd AS ( +-- ORDER BY vs.i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING +-- ); + +-- +-- Test various built-in aggregates that have moving-aggregate support +-- + +-- test inverse transition functions handle NULLs properly +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.5),(2,2.5),(3,NULL),(4,NULL)) t(i,v); + +-- [SPARK-28602] Spark does not recognize 'interval' type as 'numeric' +-- SELECT i,AVG(v::interval) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1 sec'),(2,'2 sec'),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +-- The cast syntax is present in PgSQL for legacy reasons and Spark will not recognize a money field +-- SELECT i,SUM(v::money) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1.10'),(2,'2.20'),(3,NULL),(4,NULL)) t(i,v); + +-- [SPARK-28602] Spark does not recognize 'interval' type as 'numeric' +-- SELECT i,SUM(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1 sec'),(2,'2 sec'),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.1),(2,2.2),(3,NULL),(4,NULL)) t(i,v); + +SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.01),(2,2),(3,3)) v(i,n); + +SELECT i,COUNT(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,COUNT(*) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +-- For the following queries Spark result differs from PgSQL: +-- Spark handles division by zero as 'NaN' instead of 'NULL', which is the PgSQL behaviour +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +-- test that inverse transition functions work with various frame options +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,3),(4,4)) t(i,v); + +-- [SPARK-29638] Spark handles 'NaN' as 0 in sums +-- ensure aggregate over numeric properly recovers from NaN values +SELECT a, b, + SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b); + +-- It might be tempting for someone to add an inverse trans function for +-- float and double precision. This should not be done as it can give incorrect +-- results. This test should fail if anyone ever does this without thinking too +-- hard about it. +-- [SPARK-28516] adds `to_char` +-- SELECT to_char(SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING),'999999999999999999999D9') +-- FROM (VALUES(1,1e20),(2,1)) n(i,n); + +-- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) +-- SELECT i, b, bool_and(b) OVER w, bool_or(b) OVER w +-- FROM (VALUES (1,true), (2,true), (3,false), (4,false), (5,true)) v(i,b) +-- WINDOW w AS (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING); + +-- Tests for problems with failure to walk or mutate expressions +-- within window frame clauses. + +-- [SPARK-27974] Add built-in Aggregate Function: array_agg +-- test walker (fails with collation error if expressions are not walked) +-- SELECT array_agg(i) OVER w +-- FROM range(1,6) i +-- WINDOW w AS (ORDER BY i ROWS BETWEEN (('foo' < 'foobar')::integer) PRECEDING AND CURRENT ROW); + +-- Spark doesn't handle UDFs in SQL +-- test mutator (fails when inlined if expressions are not mutated) +-- CREATE FUNCTION pg_temp.f(group_size BIGINT) RETURNS SETOF integer[] +-- AS $$ +-- SELECT array_agg(s) OVER w +-- FROM generate_series(1,5) s +-- WINDOW w AS (ORDER BY s ROWS BETWEEN CURRENT ROW AND GROUP_SIZE FOLLOWING) +-- $$ LANGUAGE SQL STABLE; diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out new file mode 100644 index 0000000000000..f4ada1e18a90a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -0,0 +1,507 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 39 + + +-- !query 0 +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 0 schema +struct +-- !query 0 output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query 1 +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 1 schema +struct +-- !query 1 output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query 2 +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 2 schema +struct +-- !query 2 output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query 3 +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.5),(2,2.5),(3,NULL),(4,NULL)) t(i,v) +-- !query 3 schema +struct +-- !query 3 output +1 2 +2 2.5 +3 NULL +4 NULL + + +-- !query 4 +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 4 schema +struct +-- !query 4 output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query 5 +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 5 schema +struct +-- !query 5 output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query 6 +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 6 schema +struct +-- !query 6 output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query 7 +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.1),(2,2.2),(3,NULL),(4,NULL)) t(i,v) +-- !query 7 schema +struct +-- !query 7 output +1 3.3 +2 2.2 +3 NULL +4 NULL + + +-- !query 8 +SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.01),(2,2),(3,3)) v(i,n) +-- !query 8 schema +struct +-- !query 8 output +3 +5 +6.01 + + +-- !query 9 +SELECT i,COUNT(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 9 schema +struct +-- !query 9 output +1 2 +2 1 +3 0 +4 0 + + +-- !query 10 +SELECT i,COUNT(*) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 10 schema +struct +-- !query 10 output +1 4 +2 3 +3 2 +4 1 + + +-- !query 11 +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 11 schema +struct +-- !query 11 output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query 12 +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 12 schema +struct +-- !query 12 output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query 13 +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 13 schema +struct +-- !query 13 output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query 14 +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 14 schema +struct +-- !query 14 output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query 15 +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 15 schema +struct +-- !query 15 output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NaN + + +-- !query 16 +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 16 schema +struct +-- !query 16 output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NaN + + +-- !query 17 +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 17 schema +struct +-- !query 17 output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NaN + + +-- !query 18 +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 18 schema +struct +-- !query 18 output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NaN + + +-- !query 19 +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 19 schema +struct +-- !query 19 output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NaN + + +-- !query 20 +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 20 schema +struct +-- !query 20 output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NaN + + +-- !query 21 +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 21 schema +struct +-- !query 21 output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NaN + + +-- !query 22 +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 22 schema +struct +-- !query 22 output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NaN + + +-- !query 23 +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query 23 schema +struct +-- !query 23 output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query 24 +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query 24 schema +struct +-- !query 24 output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query 25 +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query 25 schema +struct +-- !query 25 output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query 26 +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query 26 schema +struct +-- !query 26 output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query 27 +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query 27 schema +struct +-- !query 27 output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NaN + + +-- !query 28 +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query 28 schema +struct +-- !query 28 output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NaN + + +-- !query 29 +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query 29 schema +struct +-- !query 29 output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NaN + + +-- !query 30 +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query 30 schema +struct +-- !query 30 output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NaN + + +-- !query 31 +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 31 schema +struct +-- !query 31 output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NaN + + +-- !query 32 +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 32 schema +struct +-- !query 32 output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NaN + + +-- !query 33 +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 33 schema +struct +-- !query 33 output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NaN + + +-- !query 34 +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query 34 schema +struct +-- !query 34 output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NaN + + +-- !query 35 +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 35 schema +struct +-- !query 35 output +1 1 +2 2 +3 NULL +4 NULL + + +-- !query 36 +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query 36 schema +struct +-- !query 36 output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query 37 +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,3),(4,4)) t(i,v) +-- !query 37 schema +struct +-- !query 37 output +1 3 +2 6 +3 9 +4 7 + + +-- !query 38 +SELECT a, b, + SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b) +-- !query 38 schema +struct +-- !query 38 output +1 1 1 +2 2 3 +3 NULL 2 +4 3 3 +5 4 7 From 5f1ef544f3eca756f5db3e3fffd5b088a4b65565 Mon Sep 17 00:00:00 2001 From: uncleGen Date: Wed, 30 Oct 2019 15:30:58 +0900 Subject: [PATCH 0462/1387] [MINOR][DOCS] Use proper html tag in markdown ### What changes were proposed in this pull request? This PR fix and use proper html tag in docs ### Why are the changes needed? Fix documentation format error. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #26302 from uncleGen/minor-doc. Authored-by: uncleGen Signed-off-by: HyukjinKwon --- docs/sql-data-sources-hive-tables.md | 10 +++++----- docs/sql-data-sources-jdbc.md | 10 +++++----- docs/sql-data-sources-parquet.md | 10 +++++----- docs/structured-streaming-kafka-integration.md | 6 +++--- docs/structured-streaming-programming-guide.md | 4 ++-- 5 files changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md index e4ce3e938b75e..f99b064949347 100644 --- a/docs/sql-data-sources-hive-tables.md +++ b/docs/sql-data-sources-hive-tables.md @@ -88,17 +88,17 @@ creating table, you can create a table using storage handler at Hive side, and u inputFormat, outputFormat - These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, - e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in a pair, and you can not - specify them if you already specified the `fileFormat` option. + These 2 options specify the name of a corresponding InputFormat and OutputFormat class as a string literal, + e.g. org.apache.hadoop.hive.ql.io.orc.OrcInputFormat. These 2 options must be appeared in a pair, and you can not + specify them if you already specified the fileFormat option. serde - This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option - if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" + This option specifies the name of a serde class. When the fileFormat option is specified, do not specify this option + if the given fileFormat already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" don't include the serde information and you can use this option with these 3 fileFormats. diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index c3502cbdea8e7..b0d37b11c7117 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -60,7 +60,7 @@ the following case-insensitive options: The JDBC table that should be read from or written into. Note that when using it in the read path anything that is valid in a FROM clause of a SQL query can be used. For example, instead of a full table you could also use a subquery in parentheses. It is not - allowed to specify `dbtable` and `query` options at the same time. + allowed to specify dbtable and query options at the same time. @@ -72,10 +72,10 @@ the following case-insensitive options: SELECT <columns> FROM (<user_specified_query>) spark_gen_alias

      Below are a couple of restrictions while using this option.
        -
      1. It is not allowed to specify `dbtable` and `query` options at the same time.
      2. -
      3. It is not allowed to specify `query` and `partitionColumn` options at the same time. When specifying - `partitionColumn` option is required, the subquery can be specified using `dbtable` option instead and - partition columns can be qualified using the subquery alias provided as part of `dbtable`.
        +
      4. It is not allowed to specify dbtable and query options at the same time.
      5. +
      6. It is not allowed to specify query and partitionColumn options at the same time. When specifying + partitionColumn option is required, the subquery can be specified using dbtable option instead and + partition columns can be qualified using the subquery alias provided as part of dbtable.
        Example:
        spark.read.format("jdbc")
        diff --git a/docs/sql-data-sources-parquet.md b/docs/sql-data-sources-parquet.md index b5309870f485b..53a1111cd8286 100644 --- a/docs/sql-data-sources-parquet.md +++ b/docs/sql-data-sources-parquet.md @@ -280,12 +280,12 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession spark.sql.parquet.compression.codec snappy - Sets the compression codec used when writing Parquet files. If either `compression` or - `parquet.compression` is specified in the table-specific options/properties, the precedence would be - `compression`, `parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: + Sets the compression codec used when writing Parquet files. If either compression or + parquet.compression is specified in the table-specific options/properties, the precedence would be + compression, parquet.compression, spark.sql.parquet.compression.codec. Acceptable values include: none, uncompressed, snappy, gzip, lzo, brotli, lz4, zstd. - Note that `zstd` requires `ZStandardCodec` to be installed before Hadoop 2.9.0, `brotli` requires - `BrotliCodec` to be installed. + Note that zstd requires ZStandardCodec to be installed before Hadoop 2.9.0, brotli requires + BrotliCodec to be installed. diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index badf0429545f3..8c17de92f3487 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -473,8 +473,8 @@ The following configurations are optional: Desired minimum number of partitions to read from Kafka. By default, Spark has a 1-1 mapping of topicPartitions to Spark partitions consuming from Kafka. If you set this option to a value greater than your topicPartitions, Spark will divvy up large - Kafka partitions to smaller pieces. Please note that this configuration is like a `hint`: the - number of Spark tasks will be **approximately** `minPartitions`. It can be less or more depending on + Kafka partitions to smaller pieces. Please note that this configuration is like a hint: the + number of Spark tasks will be approximately minPartitions. It can be less or more depending on rounding errors or Kafka partitions that didn't receive any new data. @@ -482,7 +482,7 @@ The following configurations are optional: string spark-kafka-source streaming and batch - Prefix of consumer group identifiers (`group.id`) that are generated by structured streaming + Prefix of consumer group identifiers (group.id) that are generated by structured streaming queries. If "kafka.group.id" is set, this option will be ignored. diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 2a405f36fd5fd..4abdf2afcba80 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1717,7 +1717,7 @@ Here is the compatibility matrix. Append, Update, Complete Append mode uses watermark to drop old aggregation state. But the output of a - windowed aggregation is delayed the late threshold specified in `withWatermark()` as by + windowed aggregation is delayed the late threshold specified in withWatermark() as by the modes semantics, rows can be added to the Result Table only once after they are finalized (i.e. after watermark is crossed). See the Late Data section for more details. @@ -2324,7 +2324,7 @@ Here are the different kinds of triggers that are supported. One-time micro-batch - The query will execute *only one* micro-batch to process all the available data and then + The query will execute only one micro-batch to process all the available data and then stop on its own. This is useful in scenarios you want to periodically spin up a cluster, process everything that is available since the last period, and then shutdown the cluster. In some case, this may lead to significant cost savings. From 44a27bdccdc39d5394ee95d935455eb7ff4b84c2 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 30 Oct 2019 01:06:31 -0700 Subject: [PATCH 0463/1387] [SPARK-29604][SQL] Force initialize SessionState before initializing HiveClient in SparkSQLEnv ### What changes were proposed in this pull request? This patch fixes the issue that external listeners are not initialized properly when `spark.sql.hive.metastore.jars` is set to either "maven" or custom list of jar. ("builtin" is not a case here - all jars in Spark classloader are also available in separate classloader) The culprit is lazy initialization (lazy val or passing builder function) & thread context classloader. HiveClient leverages IsolatedClientLoader to properly load Hive and relevant libraries without issue - to not mess up with Spark classpath it uses separate classloader with leveraging thread context classloader. But there's a messed-up case - SessionState is being initialized while HiveClient changed the thread context classloader from Spark classloader to Hive isolated one, and streaming query listeners are loaded from changed classloader while initializing SessionState. This patch forces initializing SessionState in SparkSQLEnv to avoid such case. ### Why are the changes needed? ClassNotFoundException could occur in spark-sql with specific configuration, as explained above. ### Does this PR introduce any user-facing change? No, as I don't think end users assume the classloader of external listeners is only containing jars for Hive client. ### How was this patch tested? New UT added which fails on master branch and passes with the patch. The error message with master branch when running UT: ``` java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':; org.apache.spark.sql.AnalysisException: java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':; at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:109) at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:221) at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:147) at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:137) at org.apache.spark.sql.hive.thriftserver.SparkSQLEnv$.init(SparkSQLEnv.scala:59) at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.$anonfun$new$2(SparkSQLEnvSuite.scala:44) at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.withSystemProperties(SparkSQLEnvSuite.scala:61) at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.$anonfun$new$1(SparkSQLEnvSuite.scala:43) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186) at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:149) at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184) at org.scalatest.FunSuiteLike.$anonfun$runTest$1(FunSuiteLike.scala:196) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:286) at org.scalatest.FunSuiteLike.runTest(FunSuiteLike.scala:196) at org.scalatest.FunSuiteLike.runTest$(FunSuiteLike.scala:178) at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:56) at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:221) at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:214) at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:56) at org.scalatest.FunSuiteLike.$anonfun$runTests$1(FunSuiteLike.scala:229) at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:393) at scala.collection.immutable.List.foreach(List.scala:392) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:381) at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:376) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:458) at org.scalatest.FunSuiteLike.runTests(FunSuiteLike.scala:229) at org.scalatest.FunSuiteLike.runTests$(FunSuiteLike.scala:228) at org.scalatest.FunSuite.runTests(FunSuite.scala:1560) at org.scalatest.Suite.run(Suite.scala:1124) at org.scalatest.Suite.run$(Suite.scala:1106) at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560) at org.scalatest.FunSuiteLike.$anonfun$run$1(FunSuiteLike.scala:233) at org.scalatest.SuperEngine.runImpl(Engine.scala:518) at org.scalatest.FunSuiteLike.run(FunSuiteLike.scala:233) at org.scalatest.FunSuiteLike.run$(FunSuiteLike.scala:232) at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:56) at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213) at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210) at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208) at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:56) at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45) at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1349) at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1343) at scala.collection.immutable.List.foreach(List.scala:392) at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1343) at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:1033) at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:1011) at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1509) at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1011) at org.scalatest.tools.Runner$.run(Runner.scala:850) at org.scalatest.tools.Runner.run(Runner.scala) at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2(ScalaTestRunner.java:133) at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:27) Caused by: java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder': at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1054) at org.apache.spark.sql.SparkSession.$anonfun$sessionState$2(SparkSession.scala:156) at scala.Option.getOrElse(Option.scala:189) at org.apache.spark.sql.SparkSession.sessionState$lzycompute(SparkSession.scala:154) at org.apache.spark.sql.SparkSession.sessionState(SparkSession.scala:151) at org.apache.spark.sql.SparkSession.$anonfun$new$3(SparkSession.scala:105) at scala.Option.map(Option.scala:230) at org.apache.spark.sql.SparkSession.$anonfun$new$1(SparkSession.scala:105) at org.apache.spark.sql.internal.SQLConf$.get(SQLConf.scala:164) at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:183) at org.apache.spark.sql.hive.client.HiveClientImpl.(HiveClientImpl.scala:127) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:300) at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:421) at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:314) at org.apache.spark.sql.hive.HiveExternalCatalog.client$lzycompute(HiveExternalCatalog.scala:68) at org.apache.spark.sql.hive.HiveExternalCatalog.client(HiveExternalCatalog.scala:67) at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$databaseExists$1(HiveExternalCatalog.scala:221) at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99) ... 58 more Caused by: java.lang.ClassNotFoundException: test.custom.listener.DummyQueryExecutionListener at java.net.URLClassLoader.findClass(URLClassLoader.java:382) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:348) at org.apache.spark.util.Utils$.classForName(Utils.scala:206) at org.apache.spark.util.Utils$.$anonfun$loadExtensions$1(Utils.scala:2746) at scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:245) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at scala.collection.TraversableLike.flatMap(TraversableLike.scala:245) at scala.collection.TraversableLike.flatMap$(TraversableLike.scala:242) at scala.collection.AbstractTraversable.flatMap(Traversable.scala:108) at org.apache.spark.util.Utils$.loadExtensions(Utils.scala:2744) at org.apache.spark.sql.util.ExecutionListenerManager.$anonfun$new$1(QueryExecutionListener.scala:83) at org.apache.spark.sql.util.ExecutionListenerManager.$anonfun$new$1$adapted(QueryExecutionListener.scala:82) at scala.Option.foreach(Option.scala:407) at org.apache.spark.sql.util.ExecutionListenerManager.(QueryExecutionListener.scala:82) at org.apache.spark.sql.internal.BaseSessionStateBuilder.$anonfun$listenerManager$2(BaseSessionStateBuilder.scala:293) at scala.Option.getOrElse(Option.scala:189) at org.apache.spark.sql.internal.BaseSessionStateBuilder.listenerManager(BaseSessionStateBuilder.scala:293) at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:320) at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1051) ... 80 more ``` Closes #26258 from HeartSaVioR/SPARK-29604. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../sql/util/QueryExecutionListener.scala | 5 ++ .../sql/hive/thriftserver/SparkSQLEnv.scala | 5 ++ .../hive/thriftserver/DummyListeners.scala | 39 ++++++++++ .../hive/thriftserver/SparkSQLEnvSuite.scala | 76 +++++++++++++++++++ 4 files changed, 125 insertions(+) create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/DummyListeners.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnvSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala index f1fe472afdc2a..01f81825f6bfd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala @@ -108,6 +108,11 @@ class ExecutionListenerManager private[sql](session: SparkSession, loadExtension listenerBus.removeAllListeners() } + /** Only exposed for testing. */ + private[sql] def listListeners(): Array[QueryExecutionListener] = { + listenerBus.listeners.asScala.toArray + } + /** * Get an identical copy of this listener manager. */ diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 2fda9d0a4f60f..8944b93d9b697 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -50,6 +50,11 @@ private[hive] object SparkSQLEnv extends Logging { sparkContext = sparkSession.sparkContext sqlContext = sparkSession.sqlContext + // SPARK-29604: force initialization of the session state with the Spark class loader, + // instead of having it happen during the initialization of the Hive client (which may use a + // different class loader). + sparkSession.sessionState + val metadataHive = sparkSession .sharedState.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog].client metadataHive.setOut(new PrintStream(System.out, true, UTF_8.name())) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/DummyListeners.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/DummyListeners.scala new file mode 100644 index 0000000000000..d056b3b2153cf --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/DummyListeners.scala @@ -0,0 +1,39 @@ +/* + * 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. + */ + +/** + * These classes in this package are intentionally placed to the outer package of spark, + * because IsolatedClientLoader leverages Spark classloader for shared classess including + * spark package, and the test should fail if Spark initializes these listeners with + * IsolatedClientLoader. + */ +package test.custom.listener + +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.streaming.StreamingQueryListener +import org.apache.spark.sql.util.QueryExecutionListener + +class DummyQueryExecutionListener extends QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = {} + override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} +} + +class DummyStreamingQueryListener extends StreamingQueryListener { + override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {} + override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {} + override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {} +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnvSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnvSuite.scala new file mode 100644 index 0000000000000..ffd1fc48f19fe --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnvSuite.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import test.custom.listener.{DummyQueryExecutionListener, DummyStreamingQueryListener} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.hive.HiveUtils.{HIVE_METASTORE_JARS, HIVE_METASTORE_VERSION} +import org.apache.spark.sql.hive.test.TestHiveContext +import org.apache.spark.sql.internal.StaticSQLConf.{QUERY_EXECUTION_LISTENERS, STREAMING_QUERY_LISTENERS, WAREHOUSE_PATH} + +class SparkSQLEnvSuite extends SparkFunSuite { + test("SPARK-29604 external listeners should be initialized with Spark classloader") { + withSystemProperties( + QUERY_EXECUTION_LISTENERS.key -> classOf[DummyQueryExecutionListener].getCanonicalName, + STREAMING_QUERY_LISTENERS.key -> classOf[DummyStreamingQueryListener].getCanonicalName, + WAREHOUSE_PATH.key -> TestHiveContext.makeWarehouseDir().toURI.getPath, + // The issue occured from "maven" and list of custom jars, but providing list of custom + // jars to initialize HiveClient isn't trivial, so just use "maven". + HIVE_METASTORE_JARS.key -> "maven", + HIVE_METASTORE_VERSION.key -> null, + SparkLauncher.SPARK_MASTER -> "local[2]", + "spark.app.name" -> "testApp") { + + try { + SparkSQLEnv.init() + + val session = SparkSession.getActiveSession + assert(session.isDefined) + assert(session.get.listenerManager.listListeners() + .exists(_.isInstanceOf[DummyQueryExecutionListener])) + assert(session.get.streams.listListeners() + .exists(_.isInstanceOf[DummyStreamingQueryListener])) + } finally { + SparkSQLEnv.stop() + } + } + } + + private def withSystemProperties(pairs: (String, String)*)(f: => Unit): Unit = { + def setProperties(properties: Seq[(String, String)]): Unit = { + properties.foreach { case (key, value) => + if (value != null) { + System.setProperty(key, value) + } else { + System.clearProperty(key) + } + } + } + + val oldValues = pairs.map { kv => kv._1 -> System.getProperty(kv._1) } + try { + setProperties(pairs) + f + } finally { + setProperties(oldValues) + } + } +} From dc987f0c8b864208cd1e157f84b997b6c43b68cd Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 30 Oct 2019 08:09:22 -0700 Subject: [PATCH 0464/1387] [SPARK-29653][SQL] Fix MICROS_PER_MONTH in IntervalUtils ### What changes were proposed in this pull request? MICROS_PER_MONTH = DAYS_PER_MONTH * MICROS_PER_DAY ### Why are the changes needed? fix bug ### Does this PR introduce any user-facing change? no ### How was this patch tested? add ut Closes #26321 from yaooqinn/SPARK-29653. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/catalyst/util/IntervalUtils.scala | 2 +- sql/core/src/test/resources/sql-tests/results/date_part.sql.out | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index f55b0545ee9cb..cc2d6ac7ab2ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -36,7 +36,7 @@ object IntervalUtils { final val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS final val DAYS_PER_MONTH: Byte = 30 - final val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY + final val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.MICROS_PER_DAY /* 365.25 days per year assumes leap year every four years */ final val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 8f4edf1960755..37de7159c2d5d 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -857,7 +857,7 @@ select date_part('epoch', c) from t2 -- !query 106 schema struct -- !query 106 output -31873892788.332003 +31897220765.004003 -- !query 107 From 472940b2f47450aba46c9b064c40effd06280b46 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 30 Oct 2019 09:07:38 -0700 Subject: [PATCH 0465/1387] [SPARK-29120][SQL][TESTS] Port create_view.sql ### What changes were proposed in this pull request? This PR ports create_view.sql from PostgreSQL regression tests https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/create_view.sql The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/expected/create_view.out ### Why are the changes needed? To check behaviour differences between Spark and PostgreSQL ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass the Jenkins. And, Comparison with PgSQL results Closes #26290 from maropu/SPARK-29120. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../inputs/postgreSQL/create_view.sql | 779 +++++++ .../results/postgreSQL/create_view.sql.out | 2029 +++++++++++++++++ 2 files changed, 2808 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/postgreSQL/create_view.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/create_view.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/create_view.sql new file mode 100644 index 0000000000000..f31097e65648b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/create_view.sql @@ -0,0 +1,779 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- CREATE VIEW +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/create_view.sql + +-- [SPARK-27764] Support geometric types +-- CREATE VIEW street AS +-- SELECT r.name, r.thepath, c.cname AS cname +-- FROM ONLY road r, real_city c +-- WHERE c.outline ## r.thepath; + +-- [SPARK-27764] Support geometric types +-- CREATE VIEW iexit AS +-- SELECT ih.name, ih.thepath, +-- interpt_pp(ih.thepath, r.thepath) AS exit +-- FROM ihighway ih, ramp r +-- WHERE ih.thepath ## r.thepath; + +CREATE TABLE emp ( + name string, + age int, + -- [SPARK-27764] Support geometric types + -- location point + salary int, + manager string +) USING parquet; + +CREATE VIEW toyemp AS + SELECT name, age, /* location ,*/ 12*salary AS annualsal + FROM emp; + +-- [SPARK-29659] Support COMMENT ON syntax +-- Test comments +-- COMMENT ON VIEW noview IS 'no view'; +-- COMMENT ON VIEW toyemp IS 'is a view'; +-- COMMENT ON VIEW toyemp IS NULL; + +DROP VIEW toyemp; +DROP TABLE emp; + +-- These views are left around mainly to exercise special cases in pg_dump. + +-- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark +CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)); +-- +CREATE VIEW key_dependent_view AS + SELECT * FROM view_base_table GROUP BY key; +-- +-- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark +-- ALTER TABLE view_base_table DROP CONSTRAINT view_base_table_pkey; -- fails + +CREATE VIEW key_dependent_view_no_cols AS + SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0; + +-- +-- CREATE OR REPLACE VIEW +-- + +CREATE TABLE viewtest_tbl (a int, b int) using parquet; +-- [SPARK-29386] Copy data between a file and a table +-- COPY viewtest_tbl FROM stdin; +-- 5 10 +-- 10 15 +-- 15 20 +-- 20 25 +-- \. +INSERT INTO viewtest_tbl VALUES (5, 10), (10, 15), (15, 20), (20, 25); + +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl; + +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl WHERE a > 10; + +SELECT * FROM viewtest; + +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b FROM viewtest_tbl WHERE a > 5 ORDER BY b DESC; + +SELECT * FROM viewtest; + +-- should fail +-- [SPARK-29660] Dropping columns and changing column names/types are prohibited in VIEW definition +CREATE OR REPLACE VIEW viewtest AS + SELECT a FROM viewtest_tbl WHERE a <> 20; + +-- should fail +-- [SPARK-29660] Dropping columns and changing column names/types are prohibited in VIEW definition +CREATE OR REPLACE VIEW viewtest AS + SELECT 1, * FROM viewtest_tbl; + +-- should fail +-- [SPARK-29660] Dropping columns and changing column names/types are prohibited in VIEW definition +CREATE OR REPLACE VIEW viewtest AS + SELECT a, decimal(b) FROM viewtest_tbl; + +-- should work +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b, 0 AS c FROM viewtest_tbl; + +DROP VIEW viewtest; +DROP TABLE viewtest_tbl; + +-- tests for temporary views + +-- [SPARK-29661] Support cascaded syntax in CREATE SCHEMA +-- CREATE SCHEMA temp_view_test +-- CREATE TABLE base_table (a int, id int) using parquet +-- CREATE TABLE base_table2 (a int, id int) using parquet; +CREATE SCHEMA temp_view_test; +CREATE TABLE temp_view_test.base_table (a int, id int) using parquet; +CREATE TABLE temp_view_test.base_table2 (a int, id int) using parquet; + +-- Replace SET with USE +-- SET search_path TO temp_view_test, public; +USE temp_view_test; + +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TEMPORARY VIEW instead +-- CREATE TEMPORARY TABLE temp_table (a int, id int); +CREATE TEMPORARY VIEW temp_table AS SELECT * FROM VALUES + (1, 1) as temp_table(a, id); + +-- should be created in temp_view_test schema +CREATE VIEW v1 AS SELECT * FROM base_table; +DESC TABLE EXTENDED v1; +-- should be created in temp object schema +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v1_temp AS SELECT * FROM temp_table; +-- should be created in temp object schema +CREATE TEMP VIEW v2_temp AS SELECT * FROM base_table; +DESC TABLE EXTENDED v2_temp; +-- should be created in temp_views schema +CREATE VIEW temp_view_test.v2 AS SELECT * FROM base_table; +DESC TABLE EXTENDED temp_view_test.v2; +-- should fail +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temp_view_test.v3_temp AS SELECT * FROM temp_table; +-- should fail +-- [SPARK-29661] Support cascaded syntax in CREATE SCHEMA +-- CREATE SCHEMA test_view_schema +-- CREATE TEMP VIEW testview AS SELECT 1; + +-- joins: if any of the join relations are temporary, the view +-- should also be temporary + +-- should be non-temp +CREATE VIEW v3 AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id; +DESC TABLE EXTENDED v3; +-- should be temp (one join rel is temp) +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v4_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, temp_table t2 + WHERE t1.id = t2.id; +-- should be temp +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v5_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a, t3.a AS t3_a + FROM base_table t1, base_table2 t2, temp_table t3 + WHERE t1.id = t2.id and t2.id = t3.id; + +-- subqueries +CREATE VIEW v4 AS SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2); +DESC TABLE EXTENDED v4; +CREATE VIEW v5 AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2; +DESC TABLE EXTENDED v5; +CREATE VIEW v6 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2); +DESC TABLE EXTENDED v6; +CREATE VIEW v7 AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2); +DESC TABLE EXTENDED v7; +CREATE VIEW v8 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1); +DESC TABLE EXTENDED v8; + +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v6_temp AS SELECT * FROM base_table WHERE id IN (SELECT id FROM temp_table); +CREATE VIEW v7_temp AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM temp_table) t2; +-- [SPARK-29630] Not allowed to create a permanent view by referencing a temporary view in EXISTS +CREATE VIEW v8_temp AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM temp_table); +CREATE VIEW v9_temp AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM temp_table); + +-- a view should also be temporary if it references a temporary view +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v10_temp AS SELECT * FROM v7_temp; +CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2; +CREATE VIEW v12_temp AS SELECT true FROM v11_temp; + +-- [SPARK-27764] Support ANSI SQL CREATE SEQUENCE +-- a view should also be temporary if it references a temporary sequence +-- CREATE SEQUENCE seq1; +-- CREATE TEMPORARY SEQUENCE seq1_temp; +-- CREATE VIEW v9 AS SELECT seq1.is_called FROM seq1; +-- CREATE VIEW v13_temp AS SELECT seq1_temp.is_called FROM seq1_temp; + +-- Skip the tests below because of PostgreSQL specific cases +-- SELECT relname FROM pg_class +-- WHERE relname LIKE 'v_' +-- AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'temp_view_test') +-- ORDER BY relname; +-- SELECT relname FROM pg_class +-- WHERE relname LIKE 'v%' +-- AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname LIKE 'pg_temp%') +-- ORDER BY relname; + +CREATE SCHEMA testviewschm2; +-- Replace SET with USE +-- SET search_path TO testviewschm2, public; +USE testviewschm2; + +CREATE TABLE t1 (num int, name string) using parquet; +CREATE TABLE t2 (num2 int, value string) using parquet; +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TEMPORARY VIEW instead +-- CREATE TEMP TABLE tt (num2 int, value string); +CREATE TEMP VIEW tt AS SELECT * FROM VALUES + (1, 'a') AS tt(num2, value); + +CREATE VIEW nontemp1 AS SELECT * FROM t1 CROSS JOIN t2; +DESC TABLE EXTENDED nontemp1; +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temporal1 AS SELECT * FROM t1 CROSS JOIN tt; +CREATE VIEW nontemp2 AS SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2; +DESC TABLE EXTENDED nontemp2; +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temporal2 AS SELECT * FROM t1 INNER JOIN tt ON t1.num = tt.num2; +CREATE VIEW nontemp3 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2; +DESC TABLE EXTENDED nontemp3; +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temporal3 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2; +CREATE VIEW nontemp4 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx'; +DESC TABLE EXTENDED nontemp4; +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temporal4 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 AND tt.value = 'xxx'; + +-- Skip the tests below because of PostgreSQL specific cases +-- SELECT relname FROM pg_class +-- WHERE relname LIKE 'nontemp%' +-- AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'testviewschm2') +-- ORDER BY relname; +-- SELECT relname FROM pg_class +-- WHERE relname LIKE 'temporal%' +-- AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname LIKE 'pg_temp%') +-- ORDER BY relname; + +CREATE TABLE tbl1 ( a int, b int) using parquet; +CREATE TABLE tbl2 (c int, d int) using parquet; +CREATE TABLE tbl3 (e int, f int) using parquet; +CREATE TABLE tbl4 (g int, h int) using parquet; +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TEMPORARY VIEW instead +-- CREATE TEMP TABLE tmptbl (i int, j int); +CREATE TEMP VIEW tmptbl AS SELECT * FROM VALUES + (1, 1) AS temptbl(i, j); + +--Should be in testviewschm2 +CREATE VIEW pubview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f); +DESC TABLE EXTENDED pubview; + +-- Skip the test below because of PostgreSQL specific cases +-- SELECT count(*) FROM pg_class where relname = 'pubview' +-- AND relnamespace IN (SELECT OID FROM pg_namespace WHERE nspname = 'testviewschm2'); + +--Should be in temp object schema +CREATE VIEW mytempview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j); +DESC TABLE EXTENDED mytempview; + +-- Skip the test below because of PostgreSQL specific cases +-- SELECT count(*) FROM pg_class where relname LIKE 'mytempview' +-- And relnamespace IN (SELECT OID FROM pg_namespace WHERE nspname LIKE 'pg_temp%'); + +-- +-- CREATE VIEW and WITH(...) clause +-- CREATE VIEW mysecview1 +-- AS SELECT * FROM tbl1 WHERE a = 0; +-- +-- Skip the tests below because Spark doesn't support `WITH options` +-- CREATE VIEW mysecview2 WITH (security_barrier=true) +-- AS SELECT * FROM tbl1 WHERE a > 0; +-- CREATE VIEW mysecview3 WITH (security_barrier=false) +-- AS SELECT * FROM tbl1 WHERE a < 0; +-- CREATE VIEW mysecview4 WITH (security_barrier) +-- AS SELECT * FROM tbl1 WHERE a <> 0; +-- Spark cannot support options in WITH clause +-- CREATE VIEW mysecview5 WITH (security_barrier=100) -- Error +-- AS SELECT * FROM tbl1 WHERE a > 100; +-- CREATE VIEW mysecview6 WITH (invalid_option) -- Error +-- AS SELECT * FROM tbl1 WHERE a < 100; +-- Skip the test below because of PostgreSQL specific cases +-- SELECT relname, relkind, reloptions FROM pg_class +-- WHERE oid in ('mysecview1'::regclass, 'mysecview2'::regclass, +-- 'mysecview3'::regclass, 'mysecview4'::regclass) +-- ORDER BY relname; + +-- CREATE OR REPLACE VIEW mysecview1 +-- AS SELECT * FROM tbl1 WHERE a = 256; +-- CREATE OR REPLACE VIEW mysecview2 +-- AS SELECT * FROM tbl1 WHERE a > 256; +-- CREATE OR REPLACE VIEW mysecview3 WITH (security_barrier=true) +-- AS SELECT * FROM tbl1 WHERE a < 256; +-- CREATE OR REPLACE VIEW mysecview4 WITH (security_barrier=false) +-- AS SELECT * FROM tbl1 WHERE a <> 256; +-- Skip the test below because of PostgreSQL specific cases +-- SELECT relname, relkind, reloptions FROM pg_class +-- WHERE oid in ('mysecview1'::regclass, 'mysecview2'::regclass, +-- 'mysecview3'::regclass, 'mysecview4'::regclass) +-- ORDER BY relname; + +-- Check that unknown literals are converted to "text" in CREATE VIEW, +-- so that we don't end up with unknown-type columns. + +-- Skip the tests below because of PostgreSQL specific cases +-- CREATE VIEW unspecified_types AS +-- SELECT 42 as i, 42.5 as num, 'foo' as u, 'foo'::unknown as u2, null as n; +-- \d+ unspecified_types +-- SELECT * FROM unspecified_types; + +-- This test checks that proper typmods are assigned in a multi-row VALUES + +CREATE VIEW tt1 AS + SELECT * FROM ( + VALUES + ('abc', '0123456789', 42, 'abcd'), + ('0123456789', 'abc', 42.12, 'abc') + ) vv(a,b,c,d); +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ tt1 +SELECT * FROM tt1; +SELECT string(a) FROM tt1; +DROP VIEW tt1; + +-- Test view decompilation in the face of relation renaming conflicts + +CREATE TABLE tt1 (f1 int, f2 int, f3 string) using parquet; +CREATE TABLE tx1 (x1 int, x2 int, x3 string) using parquet; +CREATE TABLE temp_view_test.tt1 (y1 int, f2 int, f3 string) using parquet; + +CREATE VIEW aliased_view_1 AS + select * from tt1 + where exists (select 1 from tx1 where tt1.f1 = tx1.x1); +CREATE VIEW aliased_view_2 AS + select * from tt1 a1 + where exists (select 1 from tx1 where a1.f1 = tx1.x1); +CREATE VIEW aliased_view_3 AS + select * from tt1 + where exists (select 1 from tx1 a2 where tt1.f1 = a2.x1); +CREATE VIEW aliased_view_4 AS + select * from temp_view_test.tt1 + where exists (select 1 from tt1 where temp_view_test.tt1.y1 = tt1.f1); + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +DESC TABLE aliased_view_1; +-- \d+ aliased_view_2 +DESC TABLE aliased_view_2; +-- \d+ aliased_view_3 +DESC TABLE aliased_view_3; +-- \d+ aliased_view_4 +DESC TABLE aliased_view_4; + +ALTER TABLE tx1 RENAME TO a1; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +DESC TABLE aliased_view_1; +-- \d+ aliased_view_2 +DESC TABLE aliased_view_2; +-- \d+ aliased_view_3 +DESC TABLE aliased_view_3; +-- \d+ aliased_view_4 +DESC TABLE aliased_view_4; + +ALTER TABLE tt1 RENAME TO a2; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +DESC TABLE aliased_view_1; +-- \d+ aliased_view_2 +DESC TABLE aliased_view_2; +-- \d+ aliased_view_3 +DESC TABLE aliased_view_3; +-- \d+ aliased_view_4 +DESC TABLE aliased_view_4; + +ALTER TABLE a1 RENAME TO tt1; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +DESC TABLE aliased_view_1; +-- \d+ aliased_view_2 +DESC TABLE aliased_view_2; +-- \d+ aliased_view_3 +DESC TABLE aliased_view_3; +-- \d+ aliased_view_4 +DESC TABLE aliased_view_4; + +ALTER TABLE a2 RENAME TO tx1; +-- [SPARK-29632] Support ALTER TABLE [relname] SET SCHEMA [dbname] +-- ALTER TABLE tx1 SET SCHEMA temp_view_test; + +-- \d+ aliased_view_1 +-- \d+ aliased_view_2 +-- \d+ aliased_view_3 +-- \d+ aliased_view_4 + +-- [SPARK-29632] Support ALTER TABLE [relname] SET SCHEMA [dbname] +-- ALTER TABLE temp_view_test.tt1 RENAME TO tmp1; +-- ALTER TABLE temp_view_test.tmp1 SET SCHEMA testviewschm2; +-- ALTER TABLE tmp1 RENAME TO tx1; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +-- \d+ aliased_view_2 +-- \d+ aliased_view_3 +-- \d+ aliased_view_4 + +-- Test aliasing of joins + +create view view_of_joins as +select * from + (select * from (tbl1 cross join tbl2) same) ss, + (tbl3 cross join tbl4) same; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ view_of_joins + +-- Test view decompilation in the face of column addition/deletion/renaming + +create table tt2 (a int, b int, c int) using parquet; +create table tt3 (ax bigint, b short, c decimal) using parquet; +create table tt4 (ay int, b int, q int) using parquet; + +create view v1 as select * from tt2 natural join tt3; +create view v1a as select * from (tt2 natural join tt3) j; +create view v2 as select * from tt2 join tt3 using (b,c) join tt4 using (b); +create view v2a as select * from (tt2 join tt3 using (b,c) join tt4 using (b)) j; +create view v3 as select * from tt2 join tt3 using (b,c) full join tt4 using (b); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('v1', true); +DESC TABLE v1; +-- select pg_get_viewdef('v1a', true); +DESC TABLE v1a; +-- select pg_get_viewdef('v2', true); +DESC TABLE v2; +-- select pg_get_viewdef('v2a', true); +DESC TABLE v2a; +-- select pg_get_viewdef('v3', true); +DESC TABLE v3; + +alter table tt2 add column d int; +alter table tt2 add column e int; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('v1', true); +DESC TABLE v1; +-- select pg_get_viewdef('v1a', true); +DESC TABLE v1a; +-- select pg_get_viewdef('v2', true); +DESC TABLE v2; +-- select pg_get_viewdef('v2a', true); +DESC TABLE v2a; +-- select pg_get_viewdef('v3', true); +DESC TABLE v3; + +-- [SPARK-27764] Make COLUMN optional in ALTER TABLE +-- [SPARK-27589] Spark file source V2 (For supporting RENAME COLUMN in ALTER TABLE) +-- alter table tt3 rename c to d; +drop table tt3; +create table tt3 (ax bigint, b short, d decimal) using parquet; + +-- select pg_get_viewdef('v1', true); +-- select pg_get_viewdef('v1a', true); +-- select pg_get_viewdef('v2', true); +-- select pg_get_viewdef('v2a', true); +-- select pg_get_viewdef('v3', true); + +alter table tt3 add column c int; +alter table tt3 add column e int; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('v1', true); +DESC TABLE v1; +-- select pg_get_viewdef('v1a', true); +DESC TABLE v1a; +-- select pg_get_viewdef('v2', true); +DESC TABLE v2; +-- select pg_get_viewdef('v2a', true); +DESC TABLE v2a; +-- select pg_get_viewdef('v3', true); +DESC TABLE v3; + +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +-- alter table tt2 drop column d; + +-- select pg_get_viewdef('v1', true); +-- select pg_get_viewdef('v1a', true); +-- select pg_get_viewdef('v2', true); +-- select pg_get_viewdef('v2a', true); +-- select pg_get_viewdef('v3', true); + +create table tt5 (a int, b int) using parquet; +create table tt6 (c int, d int) using parquet; +create view vv1 as select * from (tt5 cross join tt6) j(aa,bb,cc,dd); +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv1', true); +DESC TABLE vv1; +alter table tt5 add column c int; +-- select pg_get_viewdef('vv1', true); +DESC TABLE vv1; +alter table tt5 add column cc int; +-- select pg_get_viewdef('vv1', true); +DESC TABLE vv1; +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +-- alter table tt5 drop column c; +-- select pg_get_viewdef('vv1', true); + +-- Unnamed FULL JOIN USING is lots of fun too + +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +create table tt7 (x int, /* xx int, */ y int) using parquet; +-- alter table tt7 drop column xx; +create table tt8 (x int, z int) using parquet; + +create view vv2 as +select * from (values(1,2,3,4,5)) v(a,b,c,d,e) +union all +select * from tt7 full join tt8 using (x), tt8 tt8x; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv2', true); +DESC TABLE vv2; + +create view vv3 as +select * from (values(1,2,3,4,5,6)) v(a,b,c,x,e,f) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv3', true); +DESC TABLE vv3; + +create view vv4 as +select * from (values(1,2,3,4,5,6,7)) v(a,b,c,x,e,f,g) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) full join tt8 tt8y using (x); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv4', true); +DESC TABLE vv4; + +alter table tt7 add column zz int; +alter table tt7 add column z int; +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +-- alter table tt7 drop column zz; +alter table tt8 add column z2 int; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv2', true); +DESC TABLE vv2; +-- select pg_get_viewdef('vv3', true); +DESC TABLE vv3; +-- select pg_get_viewdef('vv4', true); +DESC TABLE vv4; + +-- Implicit coercions in a JOIN USING create issues similar to FULL JOIN + +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +create table tt7a (x date, /* xx int, */ y int) using parquet; +-- alter table tt7a drop column xx; +create table tt8a (x timestamp, z int) using parquet; + +-- To pass the query, added exact column names in the select stmt +create view vv2a as +select * from (values(now(),2,3,now(),5)) v(a,b,c,d,e) +union all +select * from tt7a left join tt8a using (x), tt8a tt8ax; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv4', true); +DESC TABLE vv4; +-- select pg_get_viewdef('vv2a', true); +DESC TABLE vv2a; + +-- +-- Also check dropping a column that existed when the view was made +-- + +create table tt9 (x int, xx int, y int) using parquet; +create table tt10 (x int, z int) using parquet; + +create view vv5 as select x,y,z from tt9 join tt10 using(x); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv5', true); +DESC TABLE vv5; + +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +-- alter table tt9 drop column xx; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv5', true); +DESC TABLE vv5; + +-- +-- Another corner case is that we might add a column to a table below a +-- JOIN USING, and thereby make the USING column name ambiguous +-- + +create table tt11 (x int, y int) using parquet; +create table tt12 (x int, z int) using parquet; +create table tt13 (z int, q int) using parquet; + +create view vv6 as select x,y,z,q from + (tt11 join tt12 using(x)) join tt13 using(z); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv6', true); +DESC TABLE vv6; + +alter table tt11 add column z int; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv6', true); +DESC TABLE vv6; + +-- +-- Check cases involving dropped/altered columns in a function's rowtype result +-- + +-- Skip the tests below because Spark does't support PostgreSQL-specific UDFs/transactions +-- create table tt14t (f1 text, f2 text, f3 text, f4 text); +-- insert into tt14t values('foo', 'bar', 'baz', '42'); +-- +-- alter table tt14t drop column f2; +-- +-- create function tt14f() returns setof tt14t as +-- $$ +-- declare +-- rec1 record; +-- begin +-- for rec1 in select * from tt14t +-- loop +-- return next rec1; +-- end loop; +-- end; +-- $$ +-- language plpgsql; +-- +-- create view tt14v as select t.* from tt14f() t; +-- +-- select pg_get_viewdef('tt14v', true); +-- select * from tt14v; +-- +-- begin; +-- +-- -- this perhaps should be rejected, but it isn't: +-- alter table tt14t drop column f3; +-- +-- -- f3 is still in the view ... +-- select pg_get_viewdef('tt14v', true); +-- -- but will fail at execution +-- select f1, f4 from tt14v; +-- select * from tt14v; +-- +-- rollback; +-- +-- begin; +-- +-- -- this perhaps should be rejected, but it isn't: +-- alter table tt14t alter column f4 type integer using f4::integer; +-- +-- -- f4 is still in the view ... +-- select pg_get_viewdef('tt14v', true); +-- -- but will fail at execution +-- select f1, f3 from tt14v; +-- select * from tt14v; +-- +-- rollback; + +-- check display of whole-row variables in some corner cases + +-- Skip the tests below because we do not support creating types +-- create type nestedcomposite as (x int8_tbl); +-- create view tt15v as select row(i)::nestedcomposite from int8_tbl i; +-- select * from tt15v; +-- select pg_get_viewdef('tt15v', true); +-- select row(i.*::int8_tbl)::nestedcomposite from int8_tbl i; +-- +-- create view tt16v as select * from int8_tbl i, lateral(values(i)) ss; +-- select * from tt16v; +-- select pg_get_viewdef('tt16v', true); +-- select * from int8_tbl i, lateral(values(i.*::int8_tbl)) ss; +-- +-- create view tt17v as select * from int8_tbl i where i in (values(i)); +-- select * from tt17v; +-- select pg_get_viewdef('tt17v', true); +-- select * from int8_tbl i where i.* in (values(i.*::int8_tbl)); + +-- check unique-ification of overlength names + +CREATE TABLE int8_tbl (q1 int, q2 int) USING parquet; + +create view tt18v as + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxy + union all + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxz; +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('tt18v', true); +DESC TABLE tt18v; +-- explain (costs off) select * from tt18v; + +-- check display of ScalarArrayOp with a sub-select + +-- Skip the tests below because of PostgreSQL specific cases +-- select 'foo'::text = any(array['abc','def','foo']::text[]); +-- select 'foo'::text = any((select array['abc','def','foo']::text[])); -- fail +-- select 'foo'::text = any((select array['abc','def','foo']::text[])::text[]); +-- +-- create view tt19v as +-- select 'foo'::text = any(array['abc','def','foo']::text[]) c1, +-- 'foo'::text = any((select array['abc','def','foo']::text[])::text[]) c2; +-- select pg_get_viewdef('tt19v', true); + +-- check display of assorted RTE_FUNCTION expressions + +-- [SPARK-28682] ANSI SQL: Collation Support +-- create view tt20v as +-- select * from +-- coalesce(1,2) as c, +-- collation for ('x'::text) col, +-- current_date as d, +-- localtimestamp(3) as t, +-- cast(1+2 as int4) as i4, +-- cast(1+2 as int8) as i8; +-- select pg_get_viewdef('tt20v', true); + +-- corner cases with empty join conditions + +create view tt21v as +select * from tt5 natural inner join tt6; +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('tt21v', true); +DESC TABLE tt21v; + +create view tt22v as +select * from tt5 natural left join tt6; +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('tt22v', true); +DESC TABLE tt22v; + +-- check handling of views with immediately-renamed columns + +create view tt23v (col_a, col_b) as +select q1 as other_name1, q2 as other_name2 from int8_tbl +union +select 42, 43; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('tt23v', true); +DESC TABLE tt23v; +-- Skip the test below because of PostgreSQL specific cases +-- select pg_get_ruledef(oid, true) from pg_rewrite +-- where ev_class = 'tt23v'::regclass and ev_type = '1'; + +-- clean up all the random objects we made above +DROP SCHEMA temp_view_test CASCADE; +DROP SCHEMA testviewschm2 CASCADE; + +DROP VIEW temp_table; +DROP VIEW tt; diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out new file mode 100644 index 0000000000000..9140dd6ca51bb --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -0,0 +1,2029 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 193 + + +-- !query 0 +CREATE TABLE emp ( + name string, + age int, + salary int, + manager string +) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE VIEW toyemp AS + SELECT name, age, /* location ,*/ 12*salary AS annualsal + FROM emp +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +DROP VIEW toyemp +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +DROP TABLE emp +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +Hive support is required to CREATE Hive TABLE (AS SELECT); + + +-- !query 5 +CREATE VIEW key_dependent_view AS + SELECT * FROM view_base_table GROUP BY key +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Table or view not found: view_base_table; line 2 pos 17 + + +-- !query 6 +CREATE VIEW key_dependent_view_no_cols AS + SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0 +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'FROM'(line 2, pos 10) + +== SQL == +CREATE VIEW key_dependent_view_no_cols AS + SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0 +----------^^^ + + +-- !query 7 +CREATE TABLE viewtest_tbl (a int, b int) using parquet +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO viewtest_tbl VALUES (5, 10), (10, 15), (15, 20), (20, 25) +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl WHERE a > 10 +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT * FROM viewtest +-- !query 11 schema +struct +-- !query 11 output +15 20 +20 25 + + +-- !query 12 +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b FROM viewtest_tbl WHERE a > 5 ORDER BY b DESC +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +SELECT * FROM viewtest +-- !query 13 schema +struct +-- !query 13 output +20 25 +15 20 +10 15 + + +-- !query 14 +CREATE OR REPLACE VIEW viewtest AS + SELECT a FROM viewtest_tbl WHERE a <> 20 +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +CREATE OR REPLACE VIEW viewtest AS + SELECT 1, * FROM viewtest_tbl +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +CREATE OR REPLACE VIEW viewtest AS + SELECT a, decimal(b) FROM viewtest_tbl +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b, 0 AS c FROM viewtest_tbl +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +DROP VIEW viewtest +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +DROP TABLE viewtest_tbl +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +CREATE SCHEMA temp_view_test +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +CREATE TABLE temp_view_test.base_table (a int, id int) using parquet +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +CREATE TABLE temp_view_test.base_table2 (a int, id int) using parquet +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +USE temp_view_test +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +CREATE TEMPORARY VIEW temp_table AS SELECT * FROM VALUES + (1, 1) as temp_table(a, id) +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +CREATE VIEW v1 AS SELECT * FROM base_table +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +DESC TABLE EXTENDED v1 +-- !query 26 schema +struct +-- !query 26 output +a int +id int + +# Detailed Table Information +Database temp_view_test +Table v1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table +View Original Text SELECT * FROM base_table +View Default Database temp_view_test +View Query Output Columns [a, id] +Table Properties [view.default.database=temp_view_test, view.query.out.col.1=id, view.query.out.col.0=a, view.query.out.numCols=2] + + +-- !query 27 +CREATE VIEW v1_temp AS SELECT * FROM temp_table +-- !query 27 schema +struct<> +-- !query 27 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `v1_temp` by referencing a temporary view `temp_table`; + + +-- !query 28 +CREATE TEMP VIEW v2_temp AS SELECT * FROM base_table +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +DESC TABLE EXTENDED v2_temp +-- !query 29 schema +struct +-- !query 29 output +a int +id int + + +-- !query 30 +CREATE VIEW temp_view_test.v2 AS SELECT * FROM base_table +-- !query 30 schema +struct<> +-- !query 30 output + + + +-- !query 31 +DESC TABLE EXTENDED temp_view_test.v2 +-- !query 31 schema +struct +-- !query 31 output +a int +id int + +# Detailed Table Information +Database temp_view_test +Table v2 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table +View Original Text SELECT * FROM base_table +View Default Database temp_view_test +View Query Output Columns [a, id] +Table Properties [view.default.database=temp_view_test, view.query.out.col.1=id, view.query.out.col.0=a, view.query.out.numCols=2] + + +-- !query 32 +CREATE VIEW temp_view_test.v3_temp AS SELECT * FROM temp_table +-- !query 32 schema +struct<> +-- !query 32 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `temp_view_test`.`v3_temp` by referencing a temporary view `temp_table`; + + +-- !query 33 +CREATE VIEW v3 AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +DESC TABLE EXTENDED v3 +-- !query 34 schema +struct +-- !query 34 output +t1_a int +t2_a int + +# Detailed Table Information +Database temp_view_test +Table v3 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id +View Original Text SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id +View Default Database temp_view_test +View Query Output Columns [t1_a, t2_a] +Table Properties [view.default.database=temp_view_test, view.query.out.col.1=t2_a, view.query.out.col.0=t1_a, view.query.out.numCols=2] + + +-- !query 35 +CREATE VIEW v4_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, temp_table t2 + WHERE t1.id = t2.id +-- !query 35 schema +struct<> +-- !query 35 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `v4_temp` by referencing a temporary view `temp_table`; + + +-- !query 36 +CREATE VIEW v5_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a, t3.a AS t3_a + FROM base_table t1, base_table2 t2, temp_table t3 + WHERE t1.id = t2.id and t2.id = t3.id +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `v5_temp` by referencing a temporary view `temp_table`; + + +-- !query 37 +CREATE VIEW v4 AS SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2) +-- !query 37 schema +struct<> +-- !query 37 output + + + +-- !query 38 +DESC TABLE EXTENDED v4 +-- !query 38 schema +struct +-- !query 38 output +a int +id int + +# Detailed Table Information +Database temp_view_test +Table v4 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2) +View Original Text SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2) +View Default Database temp_view_test +View Query Output Columns [a, id] +Table Properties [view.default.database=temp_view_test, view.query.out.col.1=id, view.query.out.col.0=a, view.query.out.numCols=2] + + +-- !query 39 +CREATE VIEW v5 AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2 +-- !query 39 schema +struct<> +-- !query 39 output + + + +-- !query 40 +DESC TABLE EXTENDED v5 +-- !query 40 schema +struct +-- !query 40 output +id int +a int + +# Detailed Table Information +Database temp_view_test +Table v5 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2 +View Original Text SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2 +View Default Database temp_view_test +View Query Output Columns [id, a] +Table Properties [view.default.database=temp_view_test, view.query.out.col.1=a, view.query.out.col.0=id, view.query.out.numCols=2] + + +-- !query 41 +CREATE VIEW v6 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2) +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +DESC TABLE EXTENDED v6 +-- !query 42 schema +struct +-- !query 42 output +a int +id int + +# Detailed Table Information +Database temp_view_test +Table v6 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2) +View Original Text SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2) +View Default Database temp_view_test +View Query Output Columns [a, id] +Table Properties [view.default.database=temp_view_test, view.query.out.col.1=id, view.query.out.col.0=a, view.query.out.numCols=2] + + +-- !query 43 +CREATE VIEW v7 AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2) +-- !query 43 schema +struct<> +-- !query 43 output + + + +-- !query 44 +DESC TABLE EXTENDED v7 +-- !query 44 schema +struct +-- !query 44 output +a int +id int + +# Detailed Table Information +Database temp_view_test +Table v7 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2) +View Original Text SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2) +View Default Database temp_view_test +View Query Output Columns [a, id] +Table Properties [view.default.database=temp_view_test, view.query.out.col.1=id, view.query.out.col.0=a, view.query.out.numCols=2] + + +-- !query 45 +CREATE VIEW v8 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1) +-- !query 45 schema +struct<> +-- !query 45 output + + + +-- !query 46 +DESC TABLE EXTENDED v8 +-- !query 46 schema +struct +-- !query 46 output +a int +id int + +# Detailed Table Information +Database temp_view_test +Table v8 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table WHERE EXISTS (SELECT 1) +View Original Text SELECT * FROM base_table WHERE EXISTS (SELECT 1) +View Default Database temp_view_test +View Query Output Columns [a, id] +Table Properties [view.default.database=temp_view_test, view.query.out.col.1=id, view.query.out.col.0=a, view.query.out.numCols=2] + + +-- !query 47 +CREATE VIEW v6_temp AS SELECT * FROM base_table WHERE id IN (SELECT id FROM temp_table) +-- !query 47 schema +struct<> +-- !query 47 output + + + +-- !query 48 +CREATE VIEW v7_temp AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM temp_table) t2 +-- !query 48 schema +struct<> +-- !query 48 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `v7_temp` by referencing a temporary view `temp_table`; + + +-- !query 49 +CREATE VIEW v8_temp AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM temp_table) +-- !query 49 schema +struct<> +-- !query 49 output + + + +-- !query 50 +CREATE VIEW v9_temp AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM temp_table) +-- !query 50 schema +struct<> +-- !query 50 output + + + +-- !query 51 +CREATE VIEW v10_temp AS SELECT * FROM v7_temp +-- !query 51 schema +struct<> +-- !query 51 output +org.apache.spark.sql.AnalysisException +Table or view not found: v7_temp; line 1 pos 38 + + +-- !query 52 +CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2 +-- !query 52 schema +struct<> +-- !query 52 output +org.apache.spark.sql.AnalysisException +Table or view not found: v10_temp; line 1 pos 63 + + +-- !query 53 +CREATE VIEW v12_temp AS SELECT true FROM v11_temp +-- !query 53 schema +struct<> +-- !query 53 output +org.apache.spark.sql.AnalysisException +Table or view not found: v11_temp; line 1 pos 41 + + +-- !query 54 +CREATE SCHEMA testviewschm2 +-- !query 54 schema +struct<> +-- !query 54 output + + + +-- !query 55 +USE testviewschm2 +-- !query 55 schema +struct<> +-- !query 55 output + + + +-- !query 56 +CREATE TABLE t1 (num int, name string) using parquet +-- !query 56 schema +struct<> +-- !query 56 output + + + +-- !query 57 +CREATE TABLE t2 (num2 int, value string) using parquet +-- !query 57 schema +struct<> +-- !query 57 output + + + +-- !query 58 +CREATE TEMP VIEW tt AS SELECT * FROM VALUES + (1, 'a') AS tt(num2, value) +-- !query 58 schema +struct<> +-- !query 58 output + + + +-- !query 59 +CREATE VIEW nontemp1 AS SELECT * FROM t1 CROSS JOIN t2 +-- !query 59 schema +struct<> +-- !query 59 output + + + +-- !query 60 +DESC TABLE EXTENDED nontemp1 +-- !query 60 schema +struct +-- !query 60 output +num int +name string +num2 int +value string + +# Detailed Table Information +Database testviewschm2 +Table nontemp1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t1 CROSS JOIN t2 +View Original Text SELECT * FROM t1 CROSS JOIN t2 +View Default Database testviewschm2 +View Query Output Columns [num, name, num2, value] +Table Properties [view.query.out.col.3=value, view.query.out.col.0=num, view.query.out.numCols=4, view.default.database=testviewschm2, view.query.out.col.1=name, view.query.out.col.2=num2] + + +-- !query 61 +CREATE VIEW temporal1 AS SELECT * FROM t1 CROSS JOIN tt +-- !query 61 schema +struct<> +-- !query 61 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `temporal1` by referencing a temporary view `tt`; + + +-- !query 62 +CREATE VIEW nontemp2 AS SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2 +-- !query 62 schema +struct<> +-- !query 62 output + + + +-- !query 63 +DESC TABLE EXTENDED nontemp2 +-- !query 63 schema +struct +-- !query 63 output +num int +name string +num2 int +value string + +# Detailed Table Information +Database testviewschm2 +Table nontemp2 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2 +View Original Text SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2 +View Default Database testviewschm2 +View Query Output Columns [num, name, num2, value] +Table Properties [view.query.out.col.3=value, view.query.out.col.0=num, view.query.out.numCols=4, view.default.database=testviewschm2, view.query.out.col.1=name, view.query.out.col.2=num2] + + +-- !query 64 +CREATE VIEW temporal2 AS SELECT * FROM t1 INNER JOIN tt ON t1.num = tt.num2 +-- !query 64 schema +struct<> +-- !query 64 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `temporal2` by referencing a temporary view `tt`; + + +-- !query 65 +CREATE VIEW nontemp3 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 +-- !query 65 schema +struct<> +-- !query 65 output + + + +-- !query 66 +DESC TABLE EXTENDED nontemp3 +-- !query 66 schema +struct +-- !query 66 output +num int +name string +num2 int +value string + +# Detailed Table Information +Database testviewschm2 +Table nontemp3 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 +View Original Text SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 +View Default Database testviewschm2 +View Query Output Columns [num, name, num2, value] +Table Properties [view.query.out.col.3=value, view.query.out.col.0=num, view.query.out.numCols=4, view.default.database=testviewschm2, view.query.out.col.1=name, view.query.out.col.2=num2] + + +-- !query 67 +CREATE VIEW temporal3 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 +-- !query 67 schema +struct<> +-- !query 67 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `temporal3` by referencing a temporary view `tt`; + + +-- !query 68 +CREATE VIEW nontemp4 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx' +-- !query 68 schema +struct<> +-- !query 68 output + + + +-- !query 69 +DESC TABLE EXTENDED nontemp4 +-- !query 69 schema +struct +-- !query 69 output +num int +name string +num2 int +value string + +# Detailed Table Information +Database testviewschm2 +Table nontemp4 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx' +View Original Text SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx' +View Default Database testviewschm2 +View Query Output Columns [num, name, num2, value] +Table Properties [view.query.out.col.3=value, view.query.out.col.0=num, view.query.out.numCols=4, view.default.database=testviewschm2, view.query.out.col.1=name, view.query.out.col.2=num2] + + +-- !query 70 +CREATE VIEW temporal4 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 AND tt.value = 'xxx' +-- !query 70 schema +struct<> +-- !query 70 output +org.apache.spark.sql.AnalysisException +Not allowed to create a permanent view `temporal4` by referencing a temporary view `tt`; + + +-- !query 71 +CREATE TABLE tbl1 ( a int, b int) using parquet +-- !query 71 schema +struct<> +-- !query 71 output + + + +-- !query 72 +CREATE TABLE tbl2 (c int, d int) using parquet +-- !query 72 schema +struct<> +-- !query 72 output + + + +-- !query 73 +CREATE TABLE tbl3 (e int, f int) using parquet +-- !query 73 schema +struct<> +-- !query 73 output + + + +-- !query 74 +CREATE TABLE tbl4 (g int, h int) using parquet +-- !query 74 schema +struct<> +-- !query 74 output + + + +-- !query 75 +CREATE TEMP VIEW tmptbl AS SELECT * FROM VALUES + (1, 1) AS temptbl(i, j) +-- !query 75 schema +struct<> +-- !query 75 output + + + +-- !query 76 +CREATE VIEW pubview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +-- !query 76 schema +struct<> +-- !query 76 output + + + +-- !query 77 +DESC TABLE EXTENDED pubview +-- !query 77 schema +struct +-- !query 77 output +a int +b int + +# Detailed Table Information +Database testviewschm2 +Table pubview +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +View Original Text SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +View Default Database testviewschm2 +View Query Output Columns [a, b] +Table Properties [view.default.database=testviewschm2, view.query.out.col.1=b, view.query.out.col.0=a, view.query.out.numCols=2] + + +-- !query 78 +CREATE VIEW mytempview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j) +-- !query 78 schema +struct<> +-- !query 78 output + + + +-- !query 79 +DESC TABLE EXTENDED mytempview +-- !query 79 schema +struct +-- !query 79 output +a int +b int + +# Detailed Table Information +Database testviewschm2 +Table mytempview +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j) +View Original Text SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j) +View Default Database testviewschm2 +View Query Output Columns [a, b] +Table Properties [view.default.database=testviewschm2, view.query.out.col.1=b, view.query.out.col.0=a, view.query.out.numCols=2] + + +-- !query 80 +CREATE VIEW tt1 AS + SELECT * FROM ( + VALUES + ('abc', '0123456789', 42, 'abcd'), + ('0123456789', 'abc', 42.12, 'abc') + ) vv(a,b,c,d) +-- !query 80 schema +struct<> +-- !query 80 output + + + +-- !query 81 +SELECT * FROM tt1 +-- !query 81 schema +struct +-- !query 81 output +0123456789 abc 42.12 abc +abc 0123456789 42 abcd + + +-- !query 82 +SELECT string(a) FROM tt1 +-- !query 82 schema +struct +-- !query 82 output +0123456789 +abc + + +-- !query 83 +DROP VIEW tt1 +-- !query 83 schema +struct<> +-- !query 83 output + + + +-- !query 84 +CREATE TABLE tt1 (f1 int, f2 int, f3 string) using parquet +-- !query 84 schema +struct<> +-- !query 84 output + + + +-- !query 85 +CREATE TABLE tx1 (x1 int, x2 int, x3 string) using parquet +-- !query 85 schema +struct<> +-- !query 85 output + + + +-- !query 86 +CREATE TABLE temp_view_test.tt1 (y1 int, f2 int, f3 string) using parquet +-- !query 86 schema +struct<> +-- !query 86 output + + + +-- !query 87 +CREATE VIEW aliased_view_1 AS + select * from tt1 + where exists (select 1 from tx1 where tt1.f1 = tx1.x1) +-- !query 87 schema +struct<> +-- !query 87 output + + + +-- !query 88 +CREATE VIEW aliased_view_2 AS + select * from tt1 a1 + where exists (select 1 from tx1 where a1.f1 = tx1.x1) +-- !query 88 schema +struct<> +-- !query 88 output + + + +-- !query 89 +CREATE VIEW aliased_view_3 AS + select * from tt1 + where exists (select 1 from tx1 a2 where tt1.f1 = a2.x1) +-- !query 89 schema +struct<> +-- !query 89 output + + + +-- !query 90 +CREATE VIEW aliased_view_4 AS + select * from temp_view_test.tt1 + where exists (select 1 from tt1 where temp_view_test.tt1.y1 = tt1.f1) +-- !query 90 schema +struct<> +-- !query 90 output + + + +-- !query 91 +DESC TABLE aliased_view_1 +-- !query 91 schema +struct +-- !query 91 output +f1 int +f2 int +f3 string + + +-- !query 92 +DESC TABLE aliased_view_2 +-- !query 92 schema +struct +-- !query 92 output +f1 int +f2 int +f3 string + + +-- !query 93 +DESC TABLE aliased_view_3 +-- !query 93 schema +struct +-- !query 93 output +f1 int +f2 int +f3 string + + +-- !query 94 +DESC TABLE aliased_view_4 +-- !query 94 schema +struct +-- !query 94 output +y1 int +f2 int +f3 string + + +-- !query 95 +ALTER TABLE tx1 RENAME TO a1 +-- !query 95 schema +struct<> +-- !query 95 output + + + +-- !query 96 +DESC TABLE aliased_view_1 +-- !query 96 schema +struct +-- !query 96 output +f1 int +f2 int +f3 string + + +-- !query 97 +DESC TABLE aliased_view_2 +-- !query 97 schema +struct +-- !query 97 output +f1 int +f2 int +f3 string + + +-- !query 98 +DESC TABLE aliased_view_3 +-- !query 98 schema +struct +-- !query 98 output +f1 int +f2 int +f3 string + + +-- !query 99 +DESC TABLE aliased_view_4 +-- !query 99 schema +struct +-- !query 99 output +y1 int +f2 int +f3 string + + +-- !query 100 +ALTER TABLE tt1 RENAME TO a2 +-- !query 100 schema +struct<> +-- !query 100 output + + + +-- !query 101 +DESC TABLE aliased_view_1 +-- !query 101 schema +struct +-- !query 101 output +f1 int +f2 int +f3 string + + +-- !query 102 +DESC TABLE aliased_view_2 +-- !query 102 schema +struct +-- !query 102 output +f1 int +f2 int +f3 string + + +-- !query 103 +DESC TABLE aliased_view_3 +-- !query 103 schema +struct +-- !query 103 output +f1 int +f2 int +f3 string + + +-- !query 104 +DESC TABLE aliased_view_4 +-- !query 104 schema +struct +-- !query 104 output +y1 int +f2 int +f3 string + + +-- !query 105 +ALTER TABLE a1 RENAME TO tt1 +-- !query 105 schema +struct<> +-- !query 105 output + + + +-- !query 106 +DESC TABLE aliased_view_1 +-- !query 106 schema +struct +-- !query 106 output +f1 int +f2 int +f3 string + + +-- !query 107 +DESC TABLE aliased_view_2 +-- !query 107 schema +struct +-- !query 107 output +f1 int +f2 int +f3 string + + +-- !query 108 +DESC TABLE aliased_view_3 +-- !query 108 schema +struct +-- !query 108 output +f1 int +f2 int +f3 string + + +-- !query 109 +DESC TABLE aliased_view_4 +-- !query 109 schema +struct +-- !query 109 output +y1 int +f2 int +f3 string + + +-- !query 110 +ALTER TABLE a2 RENAME TO tx1 +-- !query 110 schema +struct<> +-- !query 110 output + + + +-- !query 111 +create view view_of_joins as +select * from + (select * from (tbl1 cross join tbl2) same) ss, + (tbl3 cross join tbl4) same +-- !query 111 schema +struct<> +-- !query 111 output + + + +-- !query 112 +create table tt2 (a int, b int, c int) using parquet +-- !query 112 schema +struct<> +-- !query 112 output + + + +-- !query 113 +create table tt3 (ax bigint, b short, c decimal) using parquet +-- !query 113 schema +struct<> +-- !query 113 output + + + +-- !query 114 +create table tt4 (ay int, b int, q int) using parquet +-- !query 114 schema +struct<> +-- !query 114 output + + + +-- !query 115 +create view v1 as select * from tt2 natural join tt3 +-- !query 115 schema +struct<> +-- !query 115 output + + + +-- !query 116 +create view v1a as select * from (tt2 natural join tt3) j +-- !query 116 schema +struct<> +-- !query 116 output + + + +-- !query 117 +create view v2 as select * from tt2 join tt3 using (b,c) join tt4 using (b) +-- !query 117 schema +struct<> +-- !query 117 output + + + +-- !query 118 +create view v2a as select * from (tt2 join tt3 using (b,c) join tt4 using (b)) j +-- !query 118 schema +struct<> +-- !query 118 output + + + +-- !query 119 +create view v3 as select * from tt2 join tt3 using (b,c) full join tt4 using (b) +-- !query 119 schema +struct<> +-- !query 119 output + + + +-- !query 120 +DESC TABLE v1 +-- !query 120 schema +struct +-- !query 120 output +b int +c int +a int +ax bigint + + +-- !query 121 +DESC TABLE v1a +-- !query 121 schema +struct +-- !query 121 output +b int +c int +a int +ax bigint + + +-- !query 122 +DESC TABLE v2 +-- !query 122 schema +struct +-- !query 122 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 123 +DESC TABLE v2a +-- !query 123 schema +struct +-- !query 123 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 124 +DESC TABLE v3 +-- !query 124 schema +struct +-- !query 124 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 125 +alter table tt2 add column d int +-- !query 125 schema +struct<> +-- !query 125 output + + + +-- !query 126 +alter table tt2 add column e int +-- !query 126 schema +struct<> +-- !query 126 output + + + +-- !query 127 +DESC TABLE v1 +-- !query 127 schema +struct +-- !query 127 output +b int +c int +a int +ax bigint + + +-- !query 128 +DESC TABLE v1a +-- !query 128 schema +struct +-- !query 128 output +b int +c int +a int +ax bigint + + +-- !query 129 +DESC TABLE v2 +-- !query 129 schema +struct +-- !query 129 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 130 +DESC TABLE v2a +-- !query 130 schema +struct +-- !query 130 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 131 +DESC TABLE v3 +-- !query 131 schema +struct +-- !query 131 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 132 +drop table tt3 +-- !query 132 schema +struct<> +-- !query 132 output + + + +-- !query 133 +create table tt3 (ax bigint, b short, d decimal) using parquet +-- !query 133 schema +struct<> +-- !query 133 output + + + +-- !query 134 +alter table tt3 add column c int +-- !query 134 schema +struct<> +-- !query 134 output + + + +-- !query 135 +alter table tt3 add column e int +-- !query 135 schema +struct<> +-- !query 135 output + + + +-- !query 136 +DESC TABLE v1 +-- !query 136 schema +struct +-- !query 136 output +b int +c int +a int +ax bigint + + +-- !query 137 +DESC TABLE v1a +-- !query 137 schema +struct +-- !query 137 output +b int +c int +a int +ax bigint + + +-- !query 138 +DESC TABLE v2 +-- !query 138 schema +struct +-- !query 138 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 139 +DESC TABLE v2a +-- !query 139 schema +struct +-- !query 139 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 140 +DESC TABLE v3 +-- !query 140 schema +struct +-- !query 140 output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query 141 +create table tt5 (a int, b int) using parquet +-- !query 141 schema +struct<> +-- !query 141 output + + + +-- !query 142 +create table tt6 (c int, d int) using parquet +-- !query 142 schema +struct<> +-- !query 142 output + + + +-- !query 143 +create view vv1 as select * from (tt5 cross join tt6) j(aa,bb,cc,dd) +-- !query 143 schema +struct<> +-- !query 143 output + + + +-- !query 144 +DESC TABLE vv1 +-- !query 144 schema +struct +-- !query 144 output +aa int +bb int +cc int +dd int + + +-- !query 145 +alter table tt5 add column c int +-- !query 145 schema +struct<> +-- !query 145 output + + + +-- !query 146 +DESC TABLE vv1 +-- !query 146 schema +struct +-- !query 146 output +aa int +bb int +cc int +dd int + + +-- !query 147 +alter table tt5 add column cc int +-- !query 147 schema +struct<> +-- !query 147 output + + + +-- !query 148 +DESC TABLE vv1 +-- !query 148 schema +struct +-- !query 148 output +aa int +bb int +cc int +dd int + + +-- !query 149 +create table tt7 (x int, /* xx int, */ y int) using parquet +-- !query 149 schema +struct<> +-- !query 149 output + + + +-- !query 150 +create table tt8 (x int, z int) using parquet +-- !query 150 schema +struct<> +-- !query 150 output + + + +-- !query 151 +create view vv2 as +select * from (values(1,2,3,4,5)) v(a,b,c,d,e) +union all +select * from tt7 full join tt8 using (x), tt8 tt8x +-- !query 151 schema +struct<> +-- !query 151 output + + + +-- !query 152 +DESC TABLE vv2 +-- !query 152 schema +struct +-- !query 152 output +a int +b int +c int +d int +e int + + +-- !query 153 +create view vv3 as +select * from (values(1,2,3,4,5,6)) v(a,b,c,x,e,f) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) +-- !query 153 schema +struct<> +-- !query 153 output + + + +-- !query 154 +DESC TABLE vv3 +-- !query 154 schema +struct +-- !query 154 output +a int +b int +c int +x int +e int +f int + + +-- !query 155 +create view vv4 as +select * from (values(1,2,3,4,5,6,7)) v(a,b,c,x,e,f,g) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) full join tt8 tt8y using (x) +-- !query 155 schema +struct<> +-- !query 155 output + + + +-- !query 156 +DESC TABLE vv4 +-- !query 156 schema +struct +-- !query 156 output +a int +b int +c int +x int +e int +f int +g int + + +-- !query 157 +alter table tt7 add column zz int +-- !query 157 schema +struct<> +-- !query 157 output + + + +-- !query 158 +alter table tt7 add column z int +-- !query 158 schema +struct<> +-- !query 158 output + + + +-- !query 159 +alter table tt8 add column z2 int +-- !query 159 schema +struct<> +-- !query 159 output + + + +-- !query 160 +DESC TABLE vv2 +-- !query 160 schema +struct +-- !query 160 output +a int +b int +c int +d int +e int + + +-- !query 161 +DESC TABLE vv3 +-- !query 161 schema +struct +-- !query 161 output +a int +b int +c int +x int +e int +f int + + +-- !query 162 +DESC TABLE vv4 +-- !query 162 schema +struct +-- !query 162 output +a int +b int +c int +x int +e int +f int +g int + + +-- !query 163 +create table tt7a (x date, /* xx int, */ y int) using parquet +-- !query 163 schema +struct<> +-- !query 163 output + + + +-- !query 164 +create table tt8a (x timestamp, z int) using parquet +-- !query 164 schema +struct<> +-- !query 164 output + + + +-- !query 165 +create view vv2a as +select * from (values(now(),2,3,now(),5)) v(a,b,c,d,e) +union all +select * from tt7a left join tt8a using (x), tt8a tt8ax +-- !query 165 schema +struct<> +-- !query 165 output + + + +-- !query 166 +DESC TABLE vv4 +-- !query 166 schema +struct +-- !query 166 output +a int +b int +c int +x int +e int +f int +g int + + +-- !query 167 +DESC TABLE vv2a +-- !query 167 schema +struct +-- !query 167 output +a timestamp +b int +c int +d timestamp +e int + + +-- !query 168 +create table tt9 (x int, xx int, y int) using parquet +-- !query 168 schema +struct<> +-- !query 168 output + + + +-- !query 169 +create table tt10 (x int, z int) using parquet +-- !query 169 schema +struct<> +-- !query 169 output + + + +-- !query 170 +create view vv5 as select x,y,z from tt9 join tt10 using(x) +-- !query 170 schema +struct<> +-- !query 170 output + + + +-- !query 171 +DESC TABLE vv5 +-- !query 171 schema +struct +-- !query 171 output +x int +y int +z int + + +-- !query 172 +DESC TABLE vv5 +-- !query 172 schema +struct +-- !query 172 output +x int +y int +z int + + +-- !query 173 +create table tt11 (x int, y int) using parquet +-- !query 173 schema +struct<> +-- !query 173 output + + + +-- !query 174 +create table tt12 (x int, z int) using parquet +-- !query 174 schema +struct<> +-- !query 174 output + + + +-- !query 175 +create table tt13 (z int, q int) using parquet +-- !query 175 schema +struct<> +-- !query 175 output + + + +-- !query 176 +create view vv6 as select x,y,z,q from + (tt11 join tt12 using(x)) join tt13 using(z) +-- !query 176 schema +struct<> +-- !query 176 output + + + +-- !query 177 +DESC TABLE vv6 +-- !query 177 schema +struct +-- !query 177 output +x int +y int +z int +q int + + +-- !query 178 +alter table tt11 add column z int +-- !query 178 schema +struct<> +-- !query 178 output + + + +-- !query 179 +DESC TABLE vv6 +-- !query 179 schema +struct +-- !query 179 output +x int +y int +z int +q int + + +-- !query 180 +CREATE TABLE int8_tbl (q1 int, q2 int) USING parquet +-- !query 180 schema +struct<> +-- !query 180 output + + + +-- !query 181 +create view tt18v as + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxy + union all + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxz +-- !query 181 schema +struct<> +-- !query 181 output + + + +-- !query 182 +DESC TABLE tt18v +-- !query 182 schema +struct +-- !query 182 output +q1 int +q2 int + + +-- !query 183 +create view tt21v as +select * from tt5 natural inner join tt6 +-- !query 183 schema +struct<> +-- !query 183 output + + + +-- !query 184 +DESC TABLE tt21v +-- !query 184 schema +struct +-- !query 184 output +c int +a int +b int +cc int +d int + + +-- !query 185 +create view tt22v as +select * from tt5 natural left join tt6 +-- !query 185 schema +struct<> +-- !query 185 output + + + +-- !query 186 +DESC TABLE tt22v +-- !query 186 schema +struct +-- !query 186 output +c int +a int +b int +cc int +d int + + +-- !query 187 +create view tt23v (col_a, col_b) as +select q1 as other_name1, q2 as other_name2 from int8_tbl +union +select 42, 43 +-- !query 187 schema +struct<> +-- !query 187 output + + + +-- !query 188 +DESC TABLE tt23v +-- !query 188 schema +struct +-- !query 188 output +col_a int +col_b int + + +-- !query 189 +DROP SCHEMA temp_view_test CASCADE +-- !query 189 schema +struct<> +-- !query 189 output + + + +-- !query 190 +DROP SCHEMA testviewschm2 CASCADE +-- !query 190 schema +struct<> +-- !query 190 output + + + +-- !query 191 +DROP VIEW temp_table +-- !query 191 schema +struct<> +-- !query 191 output + + + +-- !query 192 +DROP VIEW tt +-- !query 192 schema +struct<> +-- !query 192 output + From d417113c251defb206ec607f703f5344c6120b7b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 30 Oct 2019 12:31:23 -0700 Subject: [PATCH 0466/1387] [SPARK-29668][DOCS] Deprecate Python 3 prior to version 3.6 ### What changes were proposed in this pull request? This PR aims to deprecate `Python 3.4 ~ 3.5`, which is prior to version 3.6 additionally. ### Why are the changes needed? Since `Python 3.8` is already out, we will focus on to support Python 3.6/3.7/3.8. ### Does this PR introduce any user-facing change? Yes. It's highly recommended to use Python 3.6/3.7. We will verify Python 3.8 before Apache Spark 3.0.0 release. ### How was this patch tested? NA (This is a doc-only change). Closes #26326 from dongjoon-hyun/SPARK-29668. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/index.md b/docs/index.md index 9e8af0d5f8e2b..5dd8d7816bddd 100644 --- a/docs/index.md +++ b/docs/index.md @@ -48,7 +48,7 @@ or the `JAVA_HOME` environment variable pointing to a Java installation. Spark runs on Java 8/11, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. Java 8 prior to version 8u92 support is deprecated as of Spark 3.0.0. -Python 2 support is deprecated as of Spark 3.0.0. +Python 2 and Python 3 prior to version 3.6 support is deprecated as of Spark 3.0.0. R prior to version 3.4 support is deprecated as of Spark 3.0.0. For the Scala API, Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version From fd6cfb1be307755827497cf6108d2ad34a30d3d2 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Wed, 30 Oct 2019 14:51:50 -0700 Subject: [PATCH 0467/1387] [SPARK-29646][BUILD] Allow pyspark version name format `${versionNumber}-preview` in release script ### What changes were proposed in this pull request? Update `release-build.sh`, to allow pyspark version name format `${versionNumber}-preview`, otherwise the release script won't generate pyspark release tarballs. ### How was this patch tested? Tested locally. Closes #26306 from jiangxb1987/buildPython. Authored-by: Xingbo Jiang Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 1f6fdb2a55ff4..83129e7f62044 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -219,7 +219,7 @@ if [[ "$1" == "package" ]]; then # Write out the VERSION to PySpark version info we rewrite the - into a . and SNAPSHOT # to dev0 to be closer to PEP440. - PYSPARK_VERSION=`echo "$SPARK_VERSION" | sed -r "s/-/./" | sed -r "s/SNAPSHOT/dev0/"` + PYSPARK_VERSION=`echo "$SPARK_VERSION" | sed -e "s/-/./" -e "s/SNAPSHOT/dev0/" -e "s/preview/dev0/"` echo "__version__='$PYSPARK_VERSION'" > python/pyspark/version.py # Get maven home set by MVN From 155a67d00cb2f12aad179f6df2d992feca8e003e Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Wed, 30 Oct 2019 14:57:51 -0700 Subject: [PATCH 0468/1387] [SPARK-29666][BUILD] Fix the publish release failure under dry-run mode ### What changes were proposed in this pull request? `release-build.sh` fail to publish release under dry run mode with the following error message: ``` /opt/spark-rm/release-build.sh: line 429: pushd: spark-repo-g4MBm/org/apache/spark: No such file or directory ``` We need to at least run the `mvn clean install` command once to create the `$tmp_repo` path, but now those steps are all skipped under dry-run mode. This PR fixes the issue. ### How was this patch tested? Tested locally. Closes #26329 from jiangxb1987/dryrun. Authored-by: Xingbo Jiang Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 83129e7f62044..f1069d4490b5a 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -414,13 +414,13 @@ if [[ "$1" == "publish-release" ]]; then # TODO: revisit for Scala 2.13 support - if ! is_dry_run && [[ $PUBLISH_SCALA_2_11 = 1 ]]; then + if [[ $PUBLISH_SCALA_2_11 = 1 ]]; then ./dev/change-scala-version.sh 2.11 $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests \ $SCALA_2_11_PROFILES $PUBLISH_PROFILES clean install fi - if ! is_dry_run && [[ $PUBLISH_SCALA_2_12 = 1 ]]; then + if [[ $PUBLISH_SCALA_2_12 = 1 ]]; then ./dev/change-scala-version.sh 2.12 $MVN -DzincPort=$((ZINC_PORT + 2)) -Dmaven.repo.local=$tmp_repo -DskipTests \ $SCALA_2_11_PROFILES $PUBLISH_PROFILES clean install From 007c873ae34f58651481ccba30e8e2ba38a692c4 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Mon, 28 Oct 2019 22:31:29 -0700 Subject: [PATCH 0469/1387] Prepare Spark release v3.0.0-preview-rc2 ### What changes were proposed in this pull request? To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name. Made the following changes in this PR: * Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview` * Update the sparkR version number check logic to allow jvm version like `3.0.0-preview` **Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.** We shall revert the changes after 3.0.0-preview release passed. ### Why are the changes needed? To make the maven release repository to accept the built jars. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A --- R/pkg/R/sparkR.R | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10-token-provider/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graph/api/pom.xml | 2 +- graph/cypher/pom.xml | 2 +- graph/graph/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 41 files changed, 42 insertions(+), 42 deletions(-) diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index cdb59093781fb..dc1d1b2389239 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -337,7 +337,7 @@ sparkR.session <- function( # Check if version number of SparkSession matches version number of SparkR package jvmVersion <- callJMethod(sparkSession, "version") # Remove -SNAPSHOT from jvm versions - jvmVersionStrip <- gsub("-SNAPSHOT", "", jvmVersion) + jvmVersionStrip <- gsub("-preview", "", jvmVersion) rPackageVersion <- paste0(packageVersion("SparkR")) if (jvmVersionStrip != rPackageVersion) { diff --git a/assembly/pom.xml b/assembly/pom.xml index ef916fb99a04c..69efac5f43ca8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f042a12fda3d2..b19380d12312a 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2ee17800c10e4..4000f9296dc86 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index a6d99813a8501..f144b70b70c8d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 76a402bb2bd31..b355b29abc804 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 3c3c0d2d96a1c..5175e00739d11 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 883b73a69c9de..ca6de221c0378 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 93a4f67fd23f2..ad2ba6659e9cd 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 38eb8adac500e..19e67359a1c6f 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 57b8d716ee55c..b520819701651 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.0.0-SNAPSHOT -SPARK_VERSION_SHORT: 3.0.0 +SPARK_VERSION: 3.0.0-preview +SPARK_VERSION_SHORT: 3.0.0-preview SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index a099f1e042e99..f37f059a728c3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index ba6f20bfdbf58..60c5f80492d50 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index aff79b8b8e642..0f0176af7190d 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index f2dcf5d217a89..8442c46c3cb2b 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 693820da6af6b..cceace392d5ac 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index 01ca96b4f8f40..744423a88c1da 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index d11569d709b23..0e1d95cf822f4 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 0ce922349ea66..ae75d1b9f808e 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 7d69764b77de7..e284c52e84044 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index a23d255f9187c..190fffd909b15 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/graph/api/pom.xml b/graph/api/pom.xml index a18c5290195fe..953d0351b5b47 100644 --- a/graph/api/pom.xml +++ b/graph/api/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 12b3832d9777a..865a1bcf9d5a9 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml index 7438ee5e54fe4..8ed0b92a419d6 100644 --- a/graph/graph/pom.xml +++ b/graph/graph/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 444568a03d6c7..6f46cccc11871 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 02734e82ed26e..7c9357f492180 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e75e8345cd51d..86f9c1a0f2002 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 2eab868ac0dc8..497b70ddfed9c 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 11769ef548d7c..75ab818515a33 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/pom.xml b/pom.xml index b5c34790f1d3f..a10850fc7938e 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index c7de67e41ca94..284dc923313a4 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 487876b12ce91..d4a073376d112 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index c79caff164ce8..038613f810f0c 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 107ba365aa775..2038ddc2317e2 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 0e5df14e060da..15671c17a7523 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index bcebb225dfaca..ad7fe28e8d181 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3a8b7d22397ff..3565056b6206e 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 9dd927084298a..64b78265e485b 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index f627227aa0380..1f3fbef6100df 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 87af6388e1118..7394391d014d2 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e380e869f55c7..e2608976938dd 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview ../pom.xml From 8207c835b44cbe40f3c1f1dad71c660c07bae9c6 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Wed, 30 Oct 2019 17:45:44 -0700 Subject: [PATCH 0470/1387] Revert "Prepare Spark release v3.0.0-preview-rc2" This reverts commit 007c873ae34f58651481ccba30e8e2ba38a692c4. --- R/pkg/R/sparkR.R | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10-token-provider/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graph/api/pom.xml | 2 +- graph/cypher/pom.xml | 2 +- graph/graph/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 41 files changed, 42 insertions(+), 42 deletions(-) diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index dc1d1b2389239..cdb59093781fb 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -337,7 +337,7 @@ sparkR.session <- function( # Check if version number of SparkSession matches version number of SparkR package jvmVersion <- callJMethod(sparkSession, "version") # Remove -SNAPSHOT from jvm versions - jvmVersionStrip <- gsub("-preview", "", jvmVersion) + jvmVersionStrip <- gsub("-SNAPSHOT", "", jvmVersion) rPackageVersion <- paste0(packageVersion("SparkR")) if (jvmVersionStrip != rPackageVersion) { diff --git a/assembly/pom.xml b/assembly/pom.xml index 69efac5f43ca8..ef916fb99a04c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b19380d12312a..f042a12fda3d2 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 4000f9296dc86..2ee17800c10e4 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f144b70b70c8d..a6d99813a8501 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index b355b29abc804..76a402bb2bd31 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 5175e00739d11..3c3c0d2d96a1c 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index ca6de221c0378..883b73a69c9de 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index ad2ba6659e9cd..93a4f67fd23f2 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 19e67359a1c6f..38eb8adac500e 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index b520819701651..57b8d716ee55c 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.0.0-preview -SPARK_VERSION_SHORT: 3.0.0-preview +SPARK_VERSION: 3.0.0-SNAPSHOT +SPARK_VERSION_SHORT: 3.0.0 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index f37f059a728c3..a099f1e042e99 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 60c5f80492d50..ba6f20bfdbf58 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 0f0176af7190d..aff79b8b8e642 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 8442c46c3cb2b..f2dcf5d217a89 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index cceace392d5ac..693820da6af6b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index 744423a88c1da..01ca96b4f8f40 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 0e1d95cf822f4..d11569d709b23 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index ae75d1b9f808e..0ce922349ea66 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index e284c52e84044..7d69764b77de7 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 190fffd909b15..a23d255f9187c 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/api/pom.xml b/graph/api/pom.xml index 953d0351b5b47..a18c5290195fe 100644 --- a/graph/api/pom.xml +++ b/graph/api/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 865a1bcf9d5a9..12b3832d9777a 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml index 8ed0b92a419d6..7438ee5e54fe4 100644 --- a/graph/graph/pom.xml +++ b/graph/graph/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6f46cccc11871..444568a03d6c7 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 7c9357f492180..02734e82ed26e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 86f9c1a0f2002..e75e8345cd51d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 497b70ddfed9c..2eab868ac0dc8 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 75ab818515a33..11769ef548d7c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a10850fc7938e..b5c34790f1d3f 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 284dc923313a4..c7de67e41ca94 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index d4a073376d112..487876b12ce91 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 038613f810f0c..c79caff164ce8 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 2038ddc2317e2..107ba365aa775 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 15671c17a7523..0e5df14e060da 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ad7fe28e8d181..bcebb225dfaca 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3565056b6206e..3a8b7d22397ff 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 64b78265e485b..9dd927084298a 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1f3fbef6100df..f627227aa0380 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 7394391d014d2..87af6388e1118 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e2608976938dd..e380e869f55c7 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview + 3.0.0-SNAPSHOT ../pom.xml From cfc80d0eb18e1ec2866204da3500acd5f4dde2ea Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 30 Oct 2019 18:07:34 -0700 Subject: [PATCH 0471/1387] [SPARK-29277][SQL] Add early DSv2 filter and projection pushdown ### What changes were proposed in this pull request? This adds a new rule, `V2ScanRelationPushDown`, to push filters and projections in to a new `DataSourceV2ScanRelation` in the optimizer. That scan is then used when converting to a physical scan node. The new relation correctly reports stats based on the scan. To run scan pushdown before rules where stats are used, this adds a new optimizer override, `earlyScanPushDownRules` and a batch for early pushdown in the optimizer, before cost-based join reordering. The other early pushdown rule, `PruneFileSourcePartitions`, is moved into the early pushdown rule set. This also moves pushdown helper methods from `DataSourceV2Strategy` into a util class. ### Why are the changes needed? This is needed for DSv2 sources to supply stats for cost-based rules in the optimizer. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This updates the implementation of stats from `DataSourceV2Relation` so tests will fail if stats are accessed before early pushdown for v2 relations. Closes #25955 from rdblue/move-v2-pushdown. Authored-by: Ryan Blue Signed-off-by: Ryan Blue --- .../sql/catalyst/analysis/Analyzer.scala | 14 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 14 ++ .../sql/catalyst/optimizer/Optimizer.scala | 14 +- .../catalyst/plans/logical/v2Commands.scala | 6 +- .../sql/connector/catalog/CatalogV2Util.scala | 7 +- .../datasources/v2/DataSourceV2Relation.scala | 52 +++++++- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../spark/sql/execution/SparkOptimizer.scala | 12 +- .../datasources/v2/DataSourceV2Strategy.scala | 123 ++---------------- .../datasources/v2/PushDownUtils.scala | 103 +++++++++++++++ .../v2/V2ScanRelationPushDown.scala | 66 ++++++++++ .../spark/sql/FileBasedDataSourceSuite.scala | 10 +- .../sql/connector/DataSourceV2Suite.scala | 8 +- .../execution/datasources/orc/OrcTest.scala | 11 +- .../orc/OrcV2SchemaPruningSuite.scala | 2 +- .../parquet/ParquetFilterSuite.scala | 11 +- .../datasources/orc/OrcFilterSuite.scala | 10 +- 17 files changed, 305 insertions(+), 162 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d92987887b70c..de8f3e252142e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -681,10 +681,18 @@ class Analyzer( .map(v2Relation => i.copy(table = v2Relation)) .getOrElse(i) + case desc @ DescribeTable(u: UnresolvedV2Relation, _) => + CatalogV2Util.loadRelation(u.catalog, u.tableName) + .map(rel => desc.copy(table = rel)) + .getOrElse(desc) + + case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => + CatalogV2Util.loadRelation(u.catalog, u.tableName) + .map(rel => alter.copy(table = rel)) + .getOrElse(alter) + case u: UnresolvedV2Relation => - CatalogV2Util.loadTable(u.catalog, u.tableName).map { table => - DataSourceV2Relation.create(table) - }.getOrElse(u) + CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index d9dc9ebbcaf3c..72612d1dc76c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -104,6 +104,20 @@ trait CheckAnalysis extends PredicateHelper { case u: UnresolvedV2Relation => u.failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + case AlterTable(_, _, u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => + u.failAnalysis( + s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") + + case AlterTable(_, _, u: UnresolvedV2Relation, _) => + failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + + case DescribeTable(u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => + u.failAnalysis( + s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") + + case DescribeTable(u: UnresolvedV2Relation, _) => + failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. // If the arguments of the higher-order functions are resolved but the type check fails, 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 eab4c3efe4f7f..85b65edbb64b4 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 @@ -119,7 +119,7 @@ abstract class Optimizer(catalogManager: CatalogManager) rulesWithoutInferFiltersFromConstraints: _*) :: Nil } - (Batch("Eliminate Distinct", Once, EliminateDistinct) :: + val batches = (Batch("Eliminate Distinct", Once, EliminateDistinct) :: // Technically some of the rules in Finish Analysis are not optimizer rules and belong more // in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime). // However, because we also use the analyzer to canonicalized queries (for view definition), @@ -170,6 +170,10 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveLiteralFromGroupExpressions, RemoveRepetitionFromGroupExpressions) :: Nil ++ operatorOptimizationBatch) :+ + // This batch pushes filters and projections into scan nodes. Before this batch, the logical + // plan may contain nodes that do not report stats. Anything that uses stats must run after + // this batch. + Batch("Early Filter and Projection Push-Down", Once, earlyScanPushDownRules: _*) :+ // Since join costs in AQP can change between multiple runs, there is no reason that we have an // idempotence enforcement on this batch. We thus make it FixedPoint(1) instead of Once. Batch("Join Reorder", FixedPoint(1), @@ -196,6 +200,9 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveNoopOperators) :+ // This batch must be executed after the `RewriteSubquery` batch, which creates joins. Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) + + // remove any batches with no rules. this may happen when subclasses do not add optional rules. + batches.filter(_.rules.nonEmpty) } /** @@ -253,6 +260,11 @@ abstract class Optimizer(catalogManager: CatalogManager) */ def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil + /** + * Override to provide additional rules for early projection and filter pushdown to scans. + */ + def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = Nil + /** * Returns (defaultBatches - (excludedRules - nonExcludableRules)), the rule batches that * eventually run in the Optimizer. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index f587ee2928fc2..3c625e9acb5a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -271,7 +271,7 @@ case class ShowNamespaces( */ case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { - override def children: Seq[LogicalPlan] = Seq(table) + override lazy val resolved: Boolean = table.resolved override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() } @@ -313,9 +313,7 @@ case class AlterTable( table: NamedRelation, changes: Seq[TableChange]) extends Command { - override def children: Seq[LogicalPlan] = Seq(table) - - override lazy val resolved: Boolean = childrenResolved && { + override lazy val resolved: Boolean = table.resolved && { changes.forall { case add: AddColumn => add.fieldNames match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 6d8c6f8456412..0f313e7b96d86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -24,9 +24,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.plans.logical.AlterTable import org.apache.spark.sql.connector.catalog.TableChange._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} private[sql] object CatalogV2Util { @@ -224,6 +225,10 @@ private[sql] object CatalogV2Util { case _: NoSuchNamespaceException => None } + def loadRelation(catalog: CatalogPlugin, ident: Identifier): Option[NamedRelation] = { + loadTable(catalog, ident).map(DataSourceV2Relation.create) + } + def isSessionCatalog(catalog: CatalogPlugin): Boolean = { catalog.name().equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 7da502fc298ea..87d3419e8115f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, Statistics => V2S import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils /** * A logical plan representing a data source v2 table. @@ -50,12 +51,53 @@ case class DataSourceV2Relation( s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" } - def newScanBuilder(): ScanBuilder = { - table.asReadable.newScanBuilder(options) + override def computeStats(): Statistics = { + if (Utils.isTesting) { + // when testing, throw an exception if this computeStats method is called because stats should + // not be accessed before pushing the projection and filters to create a scan. otherwise, the + // stats are not accurate because they are based on a full table scan of all columns. + throw new IllegalStateException( + s"BUG: computeStats called before pushdown on DSv2 relation: $name") + } else { + // when not testing, return stats because bad stats are better than failing a query + table.asReadable.newScanBuilder(options) match { + case r: SupportsReportStatistics => + val statistics = r.estimateStatistics() + DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes) + case _ => + Statistics(sizeInBytes = conf.defaultSizeInBytes) + } + } + } + + override def newInstance(): DataSourceV2Relation = { + copy(output = output.map(_.newInstance())) + } +} + +/** + * A logical plan for a DSv2 table with a scan already created. + * + * This is used in the optimizer to push filters and projection down before conversion to physical + * plan. This ensures that the stats that are used by the optimizer account for the filters and + * projection that will be pushed down. + * + * @param table a DSv2 [[Table]] + * @param scan a DSv2 [[Scan]] + * @param output the output attributes of this relation + */ +case class DataSourceV2ScanRelation( + table: Table, + scan: Scan, + output: Seq[AttributeReference]) extends LeafNode with NamedRelation { + + override def name: String = table.name() + + override def simpleString(maxFields: Int): String = { + s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" } override def computeStats(): Statistics = { - val scan = newScanBuilder().build() scan match { case r: SupportsReportStatistics => val statistics = r.estimateStatistics() @@ -64,10 +106,6 @@ case class DataSourceV2Relation( Statistics(sizeInBytes = conf.defaultSizeInBytes) } } - - override def newInstance(): DataSourceV2Relation = { - copy(output = output.map(_.newInstance())) - } } /** 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 5f6e0a82be4ce..607f495139ec1 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 @@ -51,7 +51,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, FileTable} import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.internal.SQLConf @@ -3218,7 +3218,7 @@ class Dataset[T] private[sql]( fr.inputFiles case r: HiveTableRelation => r.tableMeta.storage.locationUri.map(_.toString).toArray - case DataSourceV2Relation(table: FileTable, _, _) => + case DataSourceV2ScanRelation(table: FileTable, _, _) => table.fileIndex.inputFiles }.flatten files.toSet.toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 4a8b56faddf7e..e65faefad5b9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -20,10 +20,13 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.ExperimentalMethods import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.dynamicpruning.{CleanupDynamicPruningFilters, PartitionPruning} import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning +import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown import org.apache.spark.sql.execution.python.{ExtractGroupingPythonUDFFromAggregate, ExtractPythonUDFFromAggregate, ExtractPythonUDFs} class SparkOptimizer( @@ -32,10 +35,12 @@ class SparkOptimizer( experimentalMethods: ExperimentalMethods) extends Optimizer(catalogManager) { + override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = + // TODO: move SchemaPruning into catalyst + SchemaPruning :: PruneFileSourcePartitions :: V2ScanRelationPushDown :: Nil + override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ - Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+ - Batch("Schema Pruning", Once, SchemaPruning) :+ Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ @@ -64,7 +69,8 @@ class SparkOptimizer( override def nonExcludableRules: Seq[String] = super.nonExcludableRules :+ ExtractPythonUDFFromJoinCondition.ruleName :+ ExtractPythonUDFFromAggregate.ruleName :+ ExtractGroupingPythonUDFFromAggregate.ruleName :+ - ExtractPythonUDFs.ruleName + ExtractPythonUDFs.ruleName :+ + V2ScanRelationPushDown.ruleName /** * Optimization batches that are executed before the regular optimization batches (also before diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 3041e9e82d59f..bc66c154b57ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -18,127 +18,30 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ -import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} -import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} -import org.apache.spark.sql.sources import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Strategy extends Strategy with PredicateHelper { - /** - * Pushes down filters to the data source reader - * - * @return pushed filter and post-scan filters. - */ - private def pushFilters( - scanBuilder: ScanBuilder, - filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - scanBuilder match { - case r: SupportsPushDownFilters => - // A map from translated data source leaf node filters to original catalyst filter - // expressions. For a `And`/`Or` predicate, it is possible that the predicate is partially - // pushed down. This map can be used to construct a catalyst filter expression from the - // input filter, or a superset(partial push down filter) of the input filter. - val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] - val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] - // Catalyst filter expression that can't be translated to data source filters. - val untranslatableExprs = mutable.ArrayBuffer.empty[Expression] - - for (filterExpr <- filters) { - val translated = - DataSourceStrategy.translateFilterWithMapping(filterExpr, Some(translatedFilterToExpr)) - if (translated.isEmpty) { - untranslatableExprs += filterExpr - } else { - translatedFilters += translated.get - } - } - - // Data source filters that need to be evaluated again after scanning. which means - // the data source cannot guarantee the rows returned can pass these filters. - // As a result we must return it so Spark can plan an extra filter operator. - val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => - DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) - } - // The filters which are marked as pushed to this data source - val pushedFilters = r.pushedFilters().map { filter => - DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) - } - (pushedFilters, untranslatableExprs ++ postScanFilters) - - case _ => (Nil, filters) - } - } - - /** - * Applies column pruning to the data source, w.r.t. the references of the given expressions. - * - * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), - * and new output attributes after column pruning. - */ - // TODO: nested column pruning. - private def pruneColumns( - scanBuilder: ScanBuilder, - relation: DataSourceV2Relation, - exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { - scanBuilder match { - case r: SupportsPushDownRequiredColumns => - val requiredColumns = AttributeSet(exprs.flatMap(_.references)) - val neededOutput = relation.output.filter(requiredColumns.contains) - if (neededOutput != relation.output) { - r.pruneColumns(neededOutput.toStructType) - val scan = r.build() - val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap - scan -> scan.readSchema().toAttributes.map { - // We have to keep the attribute id during transformation. - a => a.withExprId(nameToAttr(a.name).exprId) - } - } else { - r.build() -> relation.output - } - - case _ => scanBuilder.build() -> relation.output - } - } - import DataSourceV2Implicits._ override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => - val scanBuilder = relation.newScanBuilder() - - val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) - val normalizedFilters = DataSourceStrategy.normalizeFilters( - withoutSubquery, relation.output) - - // `pushedFilters` will be pushed down and evaluated in the underlying data sources. - // `postScanFilters` need to be evaluated after the scan. - // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. - val (pushedFilters, postScanFiltersWithoutSubquery) = - pushFilters(scanBuilder, normalizedFilters) - val postScanFilters = postScanFiltersWithoutSubquery ++ withSubquery - val (scan, output) = pruneColumns(scanBuilder, relation, project ++ postScanFilters) - logInfo( - s""" - |Pushing operators to ${relation.name} - |Pushed Filters: ${pushedFilters.mkString(", ")} - |Post-Scan Filters: ${postScanFilters.mkString(",")} - |Output: ${output.mkString(", ")} - """.stripMargin) - - val batchExec = BatchScanExec(output, scan) - - val filterCondition = postScanFilters.reduceLeftOption(And) + case PhysicalOperation(project, filters, relation: DataSourceV2ScanRelation) => + // projection and filters were already pushed down in the optimizer. + // this uses PhysicalOperation to get the projection and ensure that if the batch scan does + // not support columnar, a projection is added to convert the rows to UnsafeRow. + val batchExec = BatchScanExec(relation.output, relation.scan) + + val filterCondition = filters.reduceLeftOption(And) val withFilter = filterCondition.map(FilterExec(_, batchExec)).getOrElse(batchExec) val withProjection = if (withFilter.output != project || !batchExec.supportsColumnar) { @@ -254,19 +157,19 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { OverwritePartitionsDynamicExec( r.table.asWritable, writeOptions.asOptions, planLater(query)) :: Nil - case DeleteFromTable(r: DataSourceV2Relation, condition) => + case DeleteFromTable(DataSourceV2ScanRelation(table, _, output), condition) => if (condition.exists(SubqueryExpression.hasSubquery)) { throw new AnalysisException( s"Delete by condition with subquery is not supported: $condition") } // fail if any filter cannot be converted. correctness depends on removing all matching data. - val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, r.output) + val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, output) .flatMap(splitConjunctivePredicates(_).map { f => DataSourceStrategy.translateFilter(f).getOrElse( throw new AnalysisException(s"Exec update failed:" + s" cannot translate expression to source filter: $f")) }).toArray - DeleteFromTableExec(r.table.asDeletable, filters) :: Nil + DeleteFromTableExec(table.asDeletable, filters) :: Nil case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil @@ -283,8 +186,8 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } - case desc @ DescribeTable(r: DataSourceV2Relation, isExtended) => - DescribeTableExec(desc.output, r.table, isExtended) :: Nil + case desc @ DescribeTable(DataSourceV2Relation(table, _, _), isExtended) => + DescribeTableExec(desc.output, table, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala new file mode 100644 index 0000000000000..634ecfdf7e1d5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.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.sql.execution.datasources.v2 + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.sources + +object PushDownUtils extends PredicateHelper { + /** + * Pushes down filters to the data source reader + * + * @return pushed filter and post-scan filters. + */ + def pushFilters( + scanBuilder: ScanBuilder, + filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + scanBuilder match { + case r: SupportsPushDownFilters => + // A map from translated data source leaf node filters to original catalyst filter + // expressions. For a `And`/`Or` predicate, it is possible that the predicate is partially + // pushed down. This map can be used to construct a catalyst filter expression from the + // input filter, or a superset(partial push down filter) of the input filter. + val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] + val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] + // Catalyst filter expression that can't be translated to data source filters. + val untranslatableExprs = mutable.ArrayBuffer.empty[Expression] + + for (filterExpr <- filters) { + val translated = + DataSourceStrategy.translateFilterWithMapping(filterExpr, Some(translatedFilterToExpr)) + if (translated.isEmpty) { + untranslatableExprs += filterExpr + } else { + translatedFilters += translated.get + } + } + + // Data source filters that need to be evaluated again after scanning. which means + // the data source cannot guarantee the rows returned can pass these filters. + // As a result we must return it so Spark can plan an extra filter operator. + val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => + DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) + } + // The filters which are marked as pushed to this data source + val pushedFilters = r.pushedFilters().map { filter => + DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) + } + (pushedFilters, untranslatableExprs ++ postScanFilters) + + case _ => (Nil, filters) + } + } + + /** + * Applies column pruning to the data source, w.r.t. the references of the given expressions. + * + * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), + * and new output attributes after column pruning. + */ + // TODO: nested column pruning. + def pruneColumns( + scanBuilder: ScanBuilder, + relation: DataSourceV2Relation, + exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { + scanBuilder match { + case r: SupportsPushDownRequiredColumns => + val requiredColumns = AttributeSet(exprs.flatMap(_.references)) + val neededOutput = relation.output.filter(requiredColumns.contains) + if (neededOutput != relation.output) { + r.pruneColumns(neededOutput.toStructType) + val scan = r.build() + val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap + scan -> scan.readSchema().toAttributes.map { + // We have to keep the attribute id during transformation. + a => a.withExprId(nameToAttr(a.name).exprId) + } + } else { + r.build() -> relation.output + } + + case _ => scanBuilder.build() -> relation.output + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala new file mode 100644 index 0000000000000..6aa8d989583d1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.expressions.{And, SubqueryExpression} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.DataSourceStrategy + +object V2ScanRelationPushDown extends Rule[LogicalPlan] { + import DataSourceV2Implicits._ + + override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => + val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options) + + val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) + val normalizedFilters = DataSourceStrategy.normalizeFilters( + withoutSubquery, relation.output) + + // `pushedFilters` will be pushed down and evaluated in the underlying data sources. + // `postScanFilters` need to be evaluated after the scan. + // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. + val (pushedFilters, postScanFiltersWithoutSubquery) = PushDownUtils.pushFilters( + scanBuilder, normalizedFilters) + val postScanFilters = postScanFiltersWithoutSubquery ++ withSubquery + val (scan, output) = PushDownUtils.pruneColumns( + scanBuilder, relation, project ++ postScanFilters) + logInfo( + s""" + |Pushing operators to ${relation.name} + |Pushed Filters: ${pushedFilters.mkString(", ")} + |Post-Scan Filters: ${postScanFilters.mkString(",")} + |Output: ${output.mkString(", ")} + """.stripMargin) + + val scanRelation = DataSourceV2ScanRelation(relation.table, scan, output) + + val filterCondition = postScanFilters.reduceLeftOption(And) + val withFilter = filterCondition.map(Filter(_, scanRelation)).getOrElse(scanRelation) + + val withProjection = if (withFilter.output != project) { + Project(project, withFilter) + } else { + withFilter + } + + withProjection + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index d08f4b9066d2b..a7f3e81904de0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.io.{File, FilenameFilter, FileNotFoundException} +import java.io.{File, FileNotFoundException} import java.nio.file.{Files, StandardOpenOption} import java.util.Locale @@ -27,9 +27,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} -import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT, NullData, NullUDT} +import org.apache.spark.sql.TestingUDT.{IntervalUDT, NullData, NullUDT} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ @@ -664,7 +664,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { dir.delete() spark.range(1000).write.orc(dir.toString) val df = spark.read.orc(dir.toString) - assert(df.queryExecution.logical.stats.sizeInBytes === BigInt(getLocalDirSize(dir))) + assert(df.queryExecution.optimizedPlan.stats.sizeInBytes === BigInt(getLocalDirSize(dir))) } } } @@ -720,7 +720,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { .option("path", paths.head.getCanonicalPath) .parquet(paths(1).getCanonicalPath, paths(2).getCanonicalPath) df.queryExecution.optimizedPlan match { - case PhysicalOperation(_, _, DataSourceV2Relation(table: ParquetTable, _, _)) => + case PhysicalOperation(_, _, DataSourceV2ScanRelation(table: ParquetTable, _, _)) => assert(table.paths.toSet == paths.map(_.getCanonicalPath).toSet) case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 138bbc3f04f64..55c71c7d02d2b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapabil import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Distribution, Partitioning} -import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, DataSourceV2ScanRelation} import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.functions._ @@ -195,7 +195,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { withClue(cls.getName) { val df = spark.read.format(cls.getName).load() val logical = df.queryExecution.optimizedPlan.collect { - case d: DataSourceV2Relation => d + case d: DataSourceV2ScanRelation => d }.head val statics = logical.computeStats() @@ -332,7 +332,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { test("SPARK-23315: get output from canonicalized data source v2 related plans") { def checkCanonicalizedOutput( df: DataFrame, logicalNumOutput: Int, physicalNumOutput: Int): Unit = { - val logical = df.queryExecution.optimizedPlan.collect { + val logical = df.queryExecution.logical.collect { case d: DataSourceV2Relation => d }.head assert(logical.canonicalized.output.length == logicalNumOutput) @@ -356,7 +356,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { .read .option(optionName, false) .format(classOf[DataSourceV2WithSessionConfig].getName).load() - val options = df.queryExecution.optimizedPlan.collectFirst { + val options = df.queryExecution.logical.collectFirst { case d: DataSourceV2Relation => d.options }.get assert(options.get(optionName) === "false") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index adbd93dcb4fe8..528c3474a17c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -27,9 +27,9 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileBasedDataSourceTest} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.execution.datasources.FileBasedDataSourceTest +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION @@ -119,11 +119,8 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2Relation(orcTable: OrcTable, _, options)) => + DataSourceV2ScanRelation(_, OrcScan(_, _, _, _, _, _, _, pushedFilters), _)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") - val scanBuilder = orcTable.newScanBuilder(options) - scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) - val pushedFilters = scanBuilder.pushedFilters() if (noneSupported) { assert(pushedFilters.isEmpty, "Unsupported filters should not show in pushed filters") } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala index b626edf5dc28e..80cfbd6a02676 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.orc import org.apache.spark.SparkConf -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.datasources.SchemaPruningSuite import org.apache.spark.sql.execution.datasources.v2.BatchScanExec diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 9671866fe1535..286bb1e920266 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -33,9 +33,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.orc.OrcFilters -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType @@ -1484,12 +1483,10 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { query.queryExecution.optimizedPlan.collectFirst { case PhysicalOperation(_, filters, - DataSourceV2Relation(parquetTable: ParquetTable, _, options)) => + DataSourceV2ScanRelation(_, scan: ParquetScan, _)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") - val scanBuilder = parquetTable.newScanBuilder(options) val sourceFilters = filters.flatMap(DataSourceStrategy.translateFilter).toArray - scanBuilder.pushFilters(sourceFilters) - val pushedFilters = scanBuilder.pushedFilters() + val pushedFilters = scan.pushedFilters assert(pushedFilters.nonEmpty, "No filter is pushed down") val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) val parquetFilters = createParquetFilters(schema) diff --git a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index b1a907f9cba27..80e330b3f2ae2 100644 --- a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -29,9 +29,8 @@ import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -55,11 +54,8 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2Relation(orcTable: OrcTable, _, options)) => + DataSourceV2ScanRelation(_, OrcScan(_, _, _, _, _, _, _, pushedFilters), _)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") - val scanBuilder = orcTable.newScanBuilder(options) - scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) - val pushedFilters = scanBuilder.pushedFilters() assert(pushedFilters.nonEmpty, "No filter is pushed down") val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pushedFilters") From 3206a9987001d78cf2f48509a93d73af86f51cfe Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 31 Oct 2019 09:20:46 +0800 Subject: [PATCH 0472/1387] [SPARK-29651][SQL] Fix parsing of interval seconds fraction ### What changes were proposed in this pull request? In the PR, I propose to extract parsing of the seconds interval units to the private method `parseNanos` in `IntervalUtils` and modify the code to correctly parse the fractional part of the seconds unit of intervals in the cases: - When the fractional part has less than 9 digits - The seconds unit is negative ### Why are the changes needed? The changes are needed to fix the issues: ```sql spark-sql> select interval '10.123456 seconds'; interval 10 seconds 123 microseconds ``` The correct result must be `interval 10 seconds 123 milliseconds 456 microseconds` ```sql spark-sql> select interval '-10.123456789 seconds'; interval -9 seconds -876 milliseconds -544 microseconds ``` but the whole interval should be negated, and the result must be `interval -10 seconds -123 milliseconds -456 microseconds`, taking into account the truncation to microseconds. ### Does this PR introduce any user-facing change? Yes. After changes: ```sql spark-sql> select interval '10.123456 seconds'; interval 10 seconds 123 milliseconds 456 microseconds spark-sql> select interval '-10.123456789 seconds'; interval -10 seconds -123 milliseconds -456 microseconds ``` ### How was this patch tested? By existing and new tests in `ExpressionParserSuite`. Closes #26313 from MaxGekk/fix-interval-nanos-parsing. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/IntervalUtils.scala | 36 +++++++++++-------- .../parser/ExpressionParserSuite.scala | 14 ++++++-- .../sql-tests/results/literals.sql.out | 4 +-- 3 files changed, 36 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index cc2d6ac7ab2ea..59865d3a02d42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -218,27 +218,22 @@ object IntervalUtils { minutes = toLongWithRange("second", m.group(7), 0, 59) } // Hive allow nanosecond precision interval - val nanoStr = if (m.group(9) == null) { - null - } else { - (m.group(9) + "000000000").substring(0, 9) - } - var nanos = toLongWithRange("nanosecond", nanoStr, 0L, 999999999L) + var secondsFraction = parseNanos(m.group(9), seconds < 0) to match { case "hour" => minutes = 0 seconds = 0 - nanos = 0 + secondsFraction = 0 case "minute" => seconds = 0 - nanos = 0 + secondsFraction = 0 case "second" => // No-op case _ => throw new IllegalArgumentException( s"Cannot support (interval '$input' $from to $to) expression") } - var micros = nanos / DateTimeUtils.NANOS_PER_MICROS + var micros = secondsFraction micros = Math.addExact(micros, Math.multiplyExact(days, DateTimeUtils.MICROS_PER_DAY)) micros = Math.addExact(micros, Math.multiplyExact(hours, MICROS_PER_HOUR)) micros = Math.addExact(micros, Math.multiplyExact(minutes, MICROS_PER_MINUTE)) @@ -292,6 +287,21 @@ object IntervalUtils { new CalendarInterval(months, microseconds) } + // Parses a string with nanoseconds, truncates the result and returns microseconds + private def parseNanos(nanosStr: String, isNegative: Boolean): Long = { + if (nanosStr != null) { + val maxNanosLen = 9 + val alignedStr = if (nanosStr.length < maxNanosLen) { + (nanosStr + "000000000").substring(0, maxNanosLen) + } else nanosStr + val nanos = toLongWithRange("nanosecond", alignedStr, 0L, 999999999L) + val micros = nanos / DateTimeUtils.NANOS_PER_MICROS + if (isNegative) -micros else micros + } else { + 0L + } + } + /** * Parse second_nano string in ss.nnnnnnnnn format to microseconds */ @@ -303,15 +313,13 @@ object IntervalUtils { Long.MinValue / DateTimeUtils.MICROS_PER_SECOND, Long.MaxValue / DateTimeUtils.MICROS_PER_SECOND) * DateTimeUtils.MICROS_PER_SECOND } - def parseNanos(nanosStr: String): Long = { - toLongWithRange("nanosecond", nanosStr, 0L, 999999999L) / DateTimeUtils.NANOS_PER_MICROS - } secondNano.split("\\.") match { case Array(secondsStr) => parseSeconds(secondsStr) - case Array("", nanosStr) => parseNanos(nanosStr) + case Array("", nanosStr) => parseNanos(nanosStr, false) case Array(secondsStr, nanosStr) => - Math.addExact(parseSeconds(secondsStr), parseNanos(nanosStr)) + val seconds = parseSeconds(secondsStr) + Math.addExact(seconds, parseNanos(nanosStr, seconds < 0)) case _ => throw new IllegalArgumentException( "Interval string does not match second-nano format of ss.nnnnnnnnn") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 5a7b3ffec53f7..5423f7516c2e5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} -import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, IntervalUtils} +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, IntervalUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -628,7 +628,17 @@ class ExpressionParserSuite extends AnalysisTest { // Hive nanosecond notation. checkIntervals("13.123456789 seconds", intervalLiteral("second", "13.123456789")) - checkIntervals("-13.123456789 second", intervalLiteral("second", "-13.123456789")) + checkIntervals( + "-13.123456789 second", + Literal(new CalendarInterval( + 0, + -13 * DateTimeUtils.MICROS_PER_SECOND - 123 * DateTimeUtils.MICROS_PER_MILLIS - 456))) + checkIntervals( + "13.123456 second", + Literal(new CalendarInterval( + 0, + 13 * DateTimeUtils.MICROS_PER_SECOND + 123 * DateTimeUtils.MICROS_PER_MILLIS + 456))) + checkIntervals("1.001 second", Literal(IntervalUtils.fromString("1 second 1 millisecond"))) // Non Existing unit intercept("interval 10 nanoseconds", diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 115287821bf4f..550b9bd936a05 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -323,9 +323,9 @@ select timestamp '2016-33-11 20:54:00.000' -- !query 34 select interval 13.123456789 seconds, interval -13.123456789 second -- !query 34 schema -struct +struct -- !query 34 output -interval 13 seconds 123 milliseconds 456 microseconds interval -12 seconds -876 milliseconds -544 microseconds +interval 13 seconds 123 milliseconds 456 microseconds interval -13 seconds -123 milliseconds -456 microseconds -- !query 35 From c29494377badd182e587246954d5bb661b58a61f Mon Sep 17 00:00:00 2001 From: Chris Martin Date: Thu, 31 Oct 2019 10:41:57 +0900 Subject: [PATCH 0473/1387] [SPARK-29126][PYSPARK][DOC] Pandas Cogroup udf usage guide This PR adds some extra documentation for the new Cogrouped map Pandas udfs. Specifically: - Updated the usage guide for the new `COGROUPED_MAP` Pandas udfs added in https://github.com/apache/spark/pull/24981 - Updated the docstring for pandas_udf to include the COGROUPED_MAP type as suggested by HyukjinKwon in https://github.com/apache/spark/pull/25939 Closes #26110 from d80tb7/SPARK-29126-cogroup-udf-usage-guide. Authored-by: Chris Martin Signed-off-by: HyukjinKwon --- docs/sql-pyspark-pandas-with-arrow.md | 35 ++++++++++++++++++ examples/src/main/python/sql/arrow.py | 32 +++++++++++++++++ python/pyspark/sql/functions.py | 52 +++++++++++++++++++++++++++ 3 files changed, 119 insertions(+) diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md index 7f01483d40583..d638278b42355 100644 --- a/docs/sql-pyspark-pandas-with-arrow.md +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -178,6 +178,41 @@ For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/p [`pyspark.sql.DataFrame.mapsInPandas`](api/python/pyspark.sql.html#pyspark.sql.DataFrame.mapInPandas). +### Cogrouped Map + +Cogrouped map Pandas UDFs allow two DataFrames to be cogrouped by a common key and then a python function applied to +each cogroup. They are used with `groupBy().cogroup().apply()` which consists of the following steps: + +* Shuffle the data such that the groups of each dataframe which share a key are cogrouped together. +* Apply a function to each cogroup. The input of the function is two `pandas.DataFrame` (with an optional Tuple +representing the key). The output of the function is a `pandas.DataFrame`. +* Combine the pandas.DataFrames from all groups into a new `DataFrame`. + +To use `groupBy().cogroup().apply()`, the user needs to define the following: +* A Python function that defines the computation for each cogroup. +* A `StructType` object or a string that defines the schema of the output `DataFrame`. + +The column labels of the returned `pandas.DataFrame` must either match the field names in the +defined output schema if specified as strings, or match the field data types by position if not +strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) +on how to label columns when constructing a `pandas.DataFrame`. + +Note that all data for a cogroup will be loaded into memory before the function is applied. This can lead to out of +memory exceptions, especially if the group sizes are skewed. The configuration for [maxRecordsPerBatch](#setting-arrow-batch-size) +is not applied and it is up to the user to ensure that the cogrouped data will fit into the available memory. + +The following example shows how to use `groupby().cogroup().apply()` to perform an asof join between two datasets. + +
        +
        +{% include_example cogrouped_map_pandas_udf python/sql/arrow.py %} +
        +
        + +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and +[`pyspark.sql.CoGroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.CoGroupedData.apply). + + ## Usage Notes ### Supported SQL Types diff --git a/examples/src/main/python/sql/arrow.py b/examples/src/main/python/sql/arrow.py index de8d4f755de6f..d5a3173ff9c0a 100644 --- a/examples/src/main/python/sql/arrow.py +++ b/examples/src/main/python/sql/arrow.py @@ -258,6 +258,36 @@ def filter_func(batch_iter): # $example off:map_iter_pandas_udf$ +def cogrouped_map_pandas_udf_example(spark): + # $example on:cogrouped_map_pandas_udf$ + import pandas as pd + + from pyspark.sql.functions import pandas_udf, PandasUDFType + + df1 = spark.createDataFrame( + [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], + ("time", "id", "v1")) + + df2 = spark.createDataFrame( + [(20000101, 1, "x"), (20000101, 2, "y")], + ("time", "id", "v2")) + + @pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP) + def asof_join(l, r): + return pd.merge_asof(l, r, on="time", by="id") + + df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() + # +--------+---+---+---+ + # | time| id| v1| v2| + # +--------+---+---+---+ + # |20000101| 1|1.0| x| + # |20000102| 1|3.0| x| + # |20000101| 2|2.0| y| + # |20000102| 2|4.0| y| + # +--------+---+---+---+ + # $example off:cogrouped_map_pandas_udf$ + + if __name__ == "__main__": spark = SparkSession \ .builder \ @@ -276,5 +306,7 @@ def filter_func(batch_iter): grouped_agg_pandas_udf_example(spark) print("Running pandas_udf map iterator example") map_iter_pandas_udf_example(spark) + print("Running pandas_udf cogrouped map example") + cogrouped_map_pandas_udf_example(spark) spark.stop() diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 9186d77f7876d..bd0766508dc06 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -3236,6 +3236,58 @@ def pandas_udf(f=None, returnType=None, functionType=None): | 1| 21| +---+---+ + 6. COGROUPED_MAP + + A cogrouped map UDF defines transformation: (`pandas.DataFrame`, `pandas.DataFrame`) -> + `pandas.DataFrame`. The `returnType` should be a :class:`StructType` describing the schema + of the returned `pandas.DataFrame`. The column labels of the returned `pandas.DataFrame` + must either match the field names in the defined `returnType` schema if specified as strings, + or match the field data types by position if not strings, e.g. integer indices. The length + of the returned `pandas.DataFrame` can be arbitrary. + + CoGrouped map UDFs are used with :meth:`pyspark.sql.CoGroupedData.apply`. + + >>> from pyspark.sql.functions import pandas_udf, PandasUDFType + >>> df1 = spark.createDataFrame( + ... [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], + ... ("time", "id", "v1")) + >>> df2 = spark.createDataFrame( + ... [(20000101, 1, "x"), (20000101, 2, "y")], + ... ("time", "id", "v2")) + >>> @pandas_udf("time int, id int, v1 double, v2 string", + ... PandasUDFType.COGROUPED_MAP) # doctest: +SKIP + ... def asof_join(l, r): + ... return pd.merge_asof(l, r, on="time", by="id") + >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() # doctest: +SKIP + +---------+---+---+---+ + | time| id| v1| v2| + +---------+---+---+---+ + | 20000101| 1|1.0| x| + | 20000102| 1|3.0| x| + | 20000101| 2|2.0| y| + | 20000102| 2|4.0| y| + +---------+---+---+---+ + + Alternatively, the user can define a function that takes three arguments. In this case, + the grouping key(s) will be passed as the first argument and the data will be passed as the + second and third arguments. The grouping key(s) will be passed as a tuple of numpy data + types, e.g., `numpy.int32` and `numpy.float64`. The data will still be passed in as two + `pandas.DataFrame` containing all columns from the original Spark DataFrames. + >>> @pandas_udf("time int, id int, v1 double, v2 string", + ... PandasUDFType.COGROUPED_MAP) # doctest: +SKIP + ... def asof_join(k, l, r): + ... if k == (1,): + ... return pd.merge_asof(l, r, on="time", by="id") + ... else: + ... return pd.DataFrame(columns=['time', 'id', 'v1', 'v2']) + >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() # doctest: +SKIP + +---------+---+---+---+ + | time| id| v1| v2| + +---------+---+---+---+ + | 20000101| 1|1.0| x| + | 20000102| 1|3.0| x| + +---------+---+---+---+ + .. note:: The user-defined functions are considered deterministic by default. Due to optimization, duplicate invocations may be eliminated or the function may even be invoked more times than it is present in the query. If your function is not deterministic, call From 401a5f77157b9a9f256ad3663825522e1b22c2a7 Mon Sep 17 00:00:00 2001 From: Unknown Date: Thu, 31 Oct 2019 10:13:12 +0800 Subject: [PATCH 0474/1387] [SPARK-29523][SQL] SHOW COLUMNS should do multi-catalog resolution ### What changes were proposed in this pull request? Add ShowColumnsStatement and make SHOW COLUMNS go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. USE my_catalog DESC t // success and describe the table t from my_catalog SHOW COLUMNS FROM t // report table not found as there is no table t in the session catalog ### Does this PR introduce any user-facing change? yes. When running SHOW COLUMNS Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26182 from planga82/feature/SPARK-29523_SHOW_COLUMNS_datasourceV2. Authored-by: Unknown Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 ++-- .../sql/catalyst/parser/AstBuilder.scala | 16 +++++++++++++ .../catalyst/plans/logical/statements.scala | 7 ++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 21 +++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 23 ++++++++++++++++++- .../spark/sql/execution/SparkSqlParser.scala | 13 ----------- .../spark/sql/execution/command/tables.scala | 4 ---- .../sql/connector/DataSourceV2SQLSuite.scala | 16 +++++++++++++ .../execution/command/DDLParserSuite.scala | 21 ----------------- .../sql/execution/command/DDLSuite.scala | 4 +++- 10 files changed, 87 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 7ad008ae5d902..e916396039f4c 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -196,8 +196,8 @@ statement LIKE pattern=STRING partitionSpec? #showTable | SHOW TBLPROPERTIES table=tableIdentifier ('(' key=tablePropertyKey ')')? #showTblProperties - | SHOW COLUMNS (FROM | IN) tableIdentifier - ((FROM | IN) db=errorCapturingIdentifier)? #showColumns + | SHOW COLUMNS (FROM | IN) table=multipartIdentifier + ((FROM | IN) namespace=multipartIdentifier)? #showColumns | SHOW PARTITIONS multipartIdentifier partitionSpec? #showPartitions | SHOW identifier? FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))? #showFunctions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c9ecbdb658f4f..f3137896cec89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2922,6 +2922,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging RefreshTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) } + /** + * A command for users to list the column names for a table. + * This function creates a [[ShowColumnsStatement]] logical plan. + * + * The syntax of using this command in SQL is: + * {{{ + * SHOW COLUMNS (FROM | IN) tableName=multipartIdentifier + * ((FROM | IN) namespace=multipartIdentifier)? + * }}} + */ + override def visitShowColumns(ctx: ShowColumnsContext): LogicalPlan = withOrigin(ctx) { + val table = visitMultipartIdentifier(ctx.table) + val namespace = Option(ctx.namespace).map(visitMultipartIdentifier) + ShowColumnsStatement(table, namespace) + } + /** * Create an [[AlterTableRecoverPartitionsStatement]] * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index ac09dec13fff1..a9e7f34421edf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -393,3 +393,10 @@ case class ShowPartitionsStatement( * A REFRESH TABLE statement, as parsed from SQL */ case class RefreshTableStatement(tableName: Seq[String]) extends ParsedStatement + +/** + * A SHOW COLUMNS statement, as parsed from SQL + */ +case class ShowColumnsStatement( + table: Seq[String], + namespace: Option[Seq[String]]) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index a1d74c5099c4a..a426abe34aeb8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1169,6 +1169,27 @@ class DDLParserSuite extends AnalysisTest { RefreshTableStatement(Seq("a", "b", "c"))) } + test("show columns") { + val sql1 = "SHOW COLUMNS FROM t1" + val sql2 = "SHOW COLUMNS IN db1.t1" + val sql3 = "SHOW COLUMNS FROM t1 IN db1" + val sql4 = "SHOW COLUMNS FROM db1.t1 IN db1" + + val parsed1 = parsePlan(sql1) + val expected1 = ShowColumnsStatement(Seq("t1"), None) + val parsed2 = parsePlan(sql2) + val expected2 = ShowColumnsStatement(Seq("db1", "t1"), None) + val parsed3 = parsePlan(sql3) + val expected3 = ShowColumnsStatement(Seq("t1"), Some(Seq("db1"))) + val parsed4 = parsePlan(sql4) + val expected4 = ShowColumnsStatement(Seq("db1", "t1"), Some(Seq("db1"))) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + } + test("alter table: recover partitions") { comparePlans( parsePlan("ALTER TABLE a.b.c RECOVER PARTITIONS"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index f60d7cdeae6e2..fa4926622ed8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropDatabaseCommand, DropTableCommand, LoadDataCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropDatabaseCommand, DropTableCommand, LoadDataCommand, ShowColumnsCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -339,6 +339,27 @@ class ResolveSessionCatalog( v1TableName.asTableIdentifier, partitionSpec) + case ShowColumnsStatement(table, namespace) => + val sql = "SHOW COLUMNS" + val v1TableName = parseV1Table(table, sql).asTableIdentifier + val resolver = conf.resolver + val db = namespace match { + case Some(db) if (v1TableName.database.exists(!resolver(_, db.head))) => + throw new AnalysisException( + s"SHOW COLUMNS with conflicting databases: " + + s"'${db.head}' != '${v1TableName.database.get}'") + case _ => namespace.map(_.head) + } + if (namespace.isDefined && namespace.get.length > 1) { + throw new AnalysisException( + s"Namespace name should have only one part if specified: ${namespace.get.quoted}") + } + if (table.length > 2) { + throw new AnalysisException( + s"Table name should have at most two parts: ${table.quoted}") + } + ShowColumnsCommand(db, v1TableName) + case AlterTableRecoverPartitionsStatement(tableName) => val v1TableName = parseV1Table(tableName, "ALTER TABLE RECOVER PARTITIONS") AlterTableRecoverPartitionsCommand( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 99375a15f5230..61ff4ad82f811 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -122,19 +122,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { Option(ctx.key).map(visitTablePropertyKey)) } - /** - * A command for users to list the column names for a table. - * This function creates a [[ShowColumnsCommand]] logical plan. - * - * The syntax of using this command in SQL is: - * {{{ - * SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database]; - * }}} - */ - override def visitShowColumns(ctx: ShowColumnsContext): LogicalPlan = withOrigin(ctx) { - ShowColumnsCommand(Option(ctx.db).map(_.getText), visitTableIdentifier(ctx.tableIdentifier)) - } - /** * Create a [[RefreshResource]] logical plan. */ 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 f9992a925261f..6925bce87d6f0 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 @@ -859,12 +859,8 @@ case class ShowColumnsCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - val resolver = sparkSession.sessionState.conf.resolver val lookupTable = databaseName match { case None => tableName - case Some(db) if tableName.database.exists(!resolver(_, db)) => - throw new AnalysisException( - s"SHOW COLUMNS with conflicting databases: '$db' != '${tableName.database.get}'") case Some(db) => TableIdentifier(tableName.identifier, Some(db)) } val table = catalog.getTempViewOrPermanentTableMetadata(lookupTable) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index d4d53dc371044..6e20b248ebc10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1358,6 +1358,22 @@ class DataSourceV2SQLSuite } } + test("SHOW COLUMNS") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + testV1Command("SHOW COLUMNS", s"FROM $t") + testV1Command("SHOW COLUMNS", s"IN $t") + + val e3 = intercept[AnalysisException] { + sql(s"SHOW COLUMNS FROM tbl IN testcat.ns1.ns2") + } + assert(e3.message.contains("Namespace name should have " + + "only one part if specified: testcat.ns1.ns2")) + } + } + test("ALTER TABLE RECOVER PARTITIONS") { val t = "testcat.ns1.ns2.tbl" withTable(t) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index beab219b17d8e..1e8ff3ef4be2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -756,27 +756,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { assert(e.contains("Found an empty partition key 'b'")) } - test("show columns") { - val sql1 = "SHOW COLUMNS FROM t1" - val sql2 = "SHOW COLUMNS IN db1.t1" - val sql3 = "SHOW COLUMNS FROM t1 IN db1" - val sql4 = "SHOW COLUMNS FROM db1.t1 IN db2" - - val parsed1 = parser.parsePlan(sql1) - val expected1 = ShowColumnsCommand(None, TableIdentifier("t1", None)) - val parsed2 = parser.parsePlan(sql2) - val expected2 = ShowColumnsCommand(None, TableIdentifier("t1", Some("db1"))) - val parsed3 = parser.parsePlan(sql3) - val expected3 = ShowColumnsCommand(Some("db1"), TableIdentifier("t1", None)) - val parsed4 = parser.parsePlan(sql4) - val expected4 = ShowColumnsCommand(Some("db2"), TableIdentifier("t1", Some("db1"))) - - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - comparePlans(parsed3, expected3) - comparePlans(parsed4, expected4) - } - test("Test CTAS #1") { val s1 = """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 8dad242a8d618..514062919dfc8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2089,7 +2089,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val message = intercept[AnalysisException] { sql(s"SHOW COLUMNS IN $db.showcolumn FROM ${db.toUpperCase(Locale.ROOT)}") }.getMessage - assert(message.contains("SHOW COLUMNS with conflicting databases")) + assert(message.contains( + s"SHOW COLUMNS with conflicting databases: " + + s"'${db.toUpperCase(Locale.ROOT)}' != '$db'")) } } } From 3a06c129f4b6819c1b42c02d2c7c271376c7d22c Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 31 Oct 2019 10:47:43 +0800 Subject: [PATCH 0475/1387] [SPARK-29592][SQL] ALTER TABLE (set partition location) should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Update `AlterTableSetLocationStatement` to store `partitionSpec` and make `ALTER TABLE a.b.c PARTITION(...) SET LOCATION 'loc'` fail if `partitionSpec` is set with unsupported message. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog ALTER TABLE t PARTITION(...) SET LOCATION 'loc' // report set location with partition spec is not supported. ``` ### Does this PR introduce any user-facing change? yes. When running ALTER TABLE (set partition location), Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? New unit tests Closes #26304 from imback82/alter_table_partition_loc. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 4 ++-- .../sql/catalyst/analysis/ResolveCatalogs.scala | 6 +++++- .../spark/sql/catalyst/parser/AstBuilder.scala | 1 + .../sql/catalyst/plans/logical/statements.scala | 1 + .../sql/catalyst/parser/DDLParserSuite.scala | 14 ++++++++++---- .../analysis/ResolveSessionCatalog.scala | 8 ++++++-- .../spark/sql/execution/SparkSqlParser.scala | 16 ---------------- .../spark/sql/connector/AlterTableTests.scala | 13 +++++++++++++ .../spark/sql/execution/SQLViewSuite.scala | 15 +++++++++++---- .../sql/execution/command/DDLParserSuite.scala | 12 ------------ 10 files changed, 49 insertions(+), 41 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index e916396039f4c..11b5d4ae5ebfe 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -168,8 +168,8 @@ statement DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions | ALTER VIEW tableIdentifier DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions - | ALTER TABLE multipartIdentifier SET locationSpec #setTableLocation - | ALTER TABLE tableIdentifier partitionSpec SET locationSpec #setPartitionLocation + | ALTER TABLE multipartIdentifier + (partitionSpec)? SET locationSpec #setTableLocation | ALTER TABLE multipartIdentifier RECOVER PARTITIONS #recoverPartitions | DROP TABLE (IF EXISTS)? multipartIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? multipartIdentifier #dropView diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 7bf0e25158805..ddd60fa5ec117 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -73,7 +73,11 @@ class ResolveCatalogs(val catalogManager: CatalogManager) createAlterTable(nameParts, catalog, tableName, changes) case AlterTableSetLocationStatement( - nameParts @ NonSessionCatalog(catalog, tableName), newLoc) => + nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, newLoc) => + if (partitionSpec.nonEmpty) { + throw new AnalysisException( + "ALTER TABLE SET LOCATION does not support partition for v2 tables.") + } val changes = Seq(TableChange.setProperty("location", newLoc)) createAlterTable(nameParts, catalog, tableName, changes) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f3137896cec89..911a5b3aa36ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2727,6 +2727,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) { AlterTableSetLocationStatement( visitMultipartIdentifier(ctx.multipartIdentifier), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), visitLocationSpec(ctx.locationSpec)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index a9e7f34421edf..41c7438eaa9ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -179,6 +179,7 @@ case class AlterTableUnsetPropertiesStatement( */ case class AlterTableSetLocationStatement( tableName: Seq[String], + partitionSpec: Option[TablePartitionSpec], location: String) extends ParsedStatement /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index a426abe34aeb8..53da49ead4e48 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -540,10 +540,16 @@ class DDLParserSuite extends AnalysisTest { } test("alter table: set location") { - val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'" - val parsed1 = parsePlan(sql1) - val expected1 = AlterTableSetLocationStatement(Seq("table_name"), "new location") - comparePlans(parsed1, expected1) + comparePlans( + parsePlan("ALTER TABLE a.b.c SET LOCATION 'new location'"), + AlterTableSetLocationStatement(Seq("a", "b", "c"), None, "new location")) + + comparePlans( + parsePlan("ALTER TABLE a.b.c PARTITION(ds='2017-06-10') SET LOCATION 'new location'"), + AlterTableSetLocationStatement( + Seq("a", "b", "c"), + Some(Map("ds" -> "2017-06-10")), + "new location")) } test("alter table: rename column") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index fa4926622ed8a..7b34f4f5ef696 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -118,11 +118,15 @@ class ResolveSessionCatalog( } case AlterTableSetLocationStatement( - nameParts @ SessionCatalog(catalog, tableName), newLoc) => + nameParts @ SessionCatalog(catalog, tableName), partitionSpec, newLoc) => loadTable(catalog, tableName.asIdentifier).collect { case v1Table: V1Table => - AlterTableSetLocationCommand(tableName.asTableIdentifier, None, newLoc) + AlterTableSetLocationCommand(tableName.asTableIdentifier, partitionSpec, newLoc) }.getOrElse { + if (partitionSpec.nonEmpty) { + throw new AnalysisException( + "ALTER TABLE SET LOCATION does not support partition for v2 tables.") + } val changes = Seq(TableChange.setProperty("location", newLoc)) createAlterTable(nameParts, catalog, tableName, changes) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 61ff4ad82f811..b53fc9d042dbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -502,22 +502,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { retainData = false) } - /** - * Create an [[AlterTableSetLocationCommand]] command for a partition. - * - * For example: - * {{{ - * ALTER TABLE table PARTITION spec SET LOCATION "loc"; - * }}} - */ - override def visitSetPartitionLocation( - ctx: SetPartitionLocationContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetLocationCommand( - visitTableIdentifier(ctx.tableIdentifier), - Some(visitNonOptionalPartitionSpec(ctx.partitionSpec)), - visitLocationSpec(ctx.locationSpec)) - } - /** * Create a [[AlterTableChangeColumnCommand]] command. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index eed07aeff0903..7392850f276cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -816,6 +816,19 @@ trait AlterTableTests extends SharedSparkSession { } } + test("AlterTable: set partition location") { + val t = s"${catalogAndNamespace}table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING $v2Format") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t PARTITION(ds='2017-06-10') SET LOCATION 's3://bucket/path'") + } + assert(exc.getMessage.contains( + "ALTER TABLE SET LOCATION does not support partition for v2 tables")) + } + } + test("AlterTable: set table property") { val t = s"${catalogAndNamespace}table_name" withTable(t) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 8ff293146127a..918e1960dbd55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -136,14 +136,21 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { assertNoSuchTable(s"ALTER TABLE $viewName SET SERDE 'whatever'") assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'") assertNoSuchTable(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')") - assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'") assertNoSuchTable(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") assertNoSuchTable(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')") assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')") assertNoSuchTable(s"ALTER TABLE $viewName RECOVER PARTITIONS") // For v2 ALTER TABLE statements, we have better error message saying view is not supported. - assertViewNotSupported(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'") + assertAnalysisError( + s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'", + s"'$viewName' is a view not a table") + + // For the following v2 ALERT TABLE statements, unsupported operations are checked first + // before resolving the relations. + assertAnalysisError( + s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'", + "ALTER TABLE SET LOCATION does not support partition for v2 tables") } } @@ -177,9 +184,9 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { } } - private def assertViewNotSupported(query: String): Unit = { + private def assertAnalysisError(query: String, message: String): Unit = { val e = intercept[AnalysisException](sql(query)) - assert(e.message.contains("'testView' is a view not a table")) + assert(e.message.contains(message)) } test("error handling: insert/load/truncate table commands against a view") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 1e8ff3ef4be2d..07935641efdf5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -635,18 +635,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { "SET FILEFORMAT PARQUET") } - test("alter table: set partition location") { - val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + - "SET LOCATION 'new location'" - val parsed2 = parser.parsePlan(sql2) - val tableIdent = TableIdentifier("table_name", None) - val expected2 = AlterTableSetLocationCommand( - tableIdent, - Some(Map("dt" -> "2008-08-08", "country" -> "us")), - "new location") - comparePlans(parsed2, expected2) - } - test("alter table: change column name/type/comment") { val sql1 = "ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT" val sql2 = "ALTER TABLE table_name CHANGE COLUMN col_name col_name INT COMMENT 'new_comment'" From aa3716896f36a086a926f5df35f710735d10c6f4 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 30 Oct 2019 20:36:45 -0700 Subject: [PATCH 0476/1387] [SPARK-29668][PYTHON] Add a deprecation warning for Python 3.4 and 3.5 ### What changes were proposed in this pull request? This PR proposes to show a warning for deprecated Python 3.4 and 3.5 in Pyspark. ### Why are the changes needed? It's officially deprecated. ### Does this PR introduce any user-facing change? Yes, it shows a warning message for Python 3.4 and 3.5: ``` ... Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties Setting default log level to "WARN". To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel). /.../spark/python/pyspark/context.py:220: DeprecationWarning: Support for Python 2 and Python 3 prior to version 3.6 is deprecated as of Spark 3.0. See also the plan for dropping Python 2 support at https://spark.apache.org/news/plan-for-dropping-python-2-support.html. DeprecationWarning) ... ``` ### How was this patch tested? Manually tested. Closes #26335 from HyukjinKwon/SPARK-29668. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- python/pyspark/context.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 4d140f91f032e..bf96fba90b2ca 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -210,12 +210,12 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') self.pythonVer = "%d.%d" % sys.version_info[:2] - if sys.version_info < (3, 0): + if sys.version_info < (3, 6): with warnings.catch_warnings(): warnings.simplefilter("once") warnings.warn( - "Support for Python 2 is deprecated as of Spark 3.0. " - "See the plan for dropping Python 2 support at " + "Support for Python 2 and Python 3 prior to version 3.6 is deprecated as " + "of Spark 3.0. See also the plan for dropping Python 2 support at " "https://spark.apache.org/news/plan-for-dropping-python-2-support.html.", DeprecationWarning) From 1e599e5005cdb99fb2e9750f238f0d8812eefba6 Mon Sep 17 00:00:00 2001 From: Xianyang Liu Date: Thu, 31 Oct 2019 13:10:44 +0900 Subject: [PATCH 0477/1387] [SPARK-29582][PYSPARK] Support `TaskContext.get()` in a barrier task from Python side ### What changes were proposed in this pull request? Add support of `TaskContext.get()` in a barrier task from Python side, this makes it easier to migrate legacy user code to barrier execution mode. ### Why are the changes needed? In Spark Core, there is a `TaskContext` object which is a singleton. We set a task context instance which can be TaskContext or BarrierTaskContext before the task function startup, and unset it to none after the function end. So we can both get TaskContext and BarrierTaskContext with the object. However we can only get the BarrierTaskContext with `BarrierTaskContext`, we will get `None` if we get it by `TaskContext.get` in a barrier stage. This is useful when people switch from normal code to barrier code, and only need a little update. ### Does this PR introduce any user-facing change? Yes. Previously: ```python def func(iterator): task_context = TaskContext.get() . # this could be None. barrier_task_context = BarrierTaskContext.get() # get the BarrierTaskContext instance ... rdd.barrier().mapPartitions(func) ``` Proposed: ```python def func(iterator): task_context = TaskContext.get() . # this could also get the BarrierTaskContext instance which is same as barrier_task_context barrier_task_context = BarrierTaskContext.get() # get the BarrierTaskContext instance ... rdd.barrier().mapPartitions(func) ``` ### How was this patch tested? New UT tests. Closes #26239 from ConeyLiu/barrier_task_context. Authored-by: Xianyang Liu Signed-off-by: HyukjinKwon --- python/pyspark/taskcontext.py | 7 ++ python/pyspark/tests/test_taskcontext.py | 85 ++++++++++++++++++++++++ python/pyspark/worker.py | 8 +++ 3 files changed, 100 insertions(+) diff --git a/python/pyspark/taskcontext.py b/python/pyspark/taskcontext.py index b0c32c15793ac..d648f63338514 100644 --- a/python/pyspark/taskcontext.py +++ b/python/pyspark/taskcontext.py @@ -53,6 +53,10 @@ def _getOrCreate(cls): cls._taskContext = TaskContext() return cls._taskContext + @classmethod + def _setTaskContext(cls, taskContext): + cls._taskContext = taskContext + @classmethod def get(cls): """ @@ -162,7 +166,10 @@ def get(cls): running tasks. .. note:: Must be called on the worker, not the driver. Returns None if not initialized. + An Exception will raise if it is not in a barrier stage. """ + if not isinstance(cls._taskContext, BarrierTaskContext): + raise Exception('It is not in a barrier stage') return cls._taskContext @classmethod diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index 66c5f9f3c2fd9..6095a384679af 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -25,6 +25,9 @@ from pyspark import SparkConf, SparkContext, TaskContext, BarrierTaskContext from pyspark.testing.utils import PySparkTestCase, SPARK_HOME +if sys.version_info[0] >= 3: + xrange = range + class TaskContextTests(PySparkTestCase): @@ -146,6 +149,49 @@ def f(iterator): self.assertTrue(len(taskInfos) == 4) self.assertTrue(len(taskInfos[0]) == 4) + def test_context_get(self): + """ + Verify that TaskContext.get() works both in or not in a barrier stage. + """ + rdd = self.sc.parallelize(range(10), 4) + + def f(iterator): + taskContext = TaskContext.get() + if isinstance(taskContext, BarrierTaskContext): + yield taskContext.partitionId() + 1 + elif isinstance(taskContext, TaskContext): + yield taskContext.partitionId() + 2 + else: + yield -1 + + # for normal stage + result1 = rdd.mapPartitions(f).collect() + self.assertTrue(result1 == [2, 3, 4, 5]) + # for barrier stage + result2 = rdd.barrier().mapPartitions(f).collect() + self.assertTrue(result2 == [1, 2, 3, 4]) + + def test_barrier_context_get(self): + """ + Verify that BarrierTaskContext.get() should only works in a barrier stage. + """ + rdd = self.sc.parallelize(range(10), 4) + + def f(iterator): + try: + taskContext = BarrierTaskContext.get() + except Exception: + yield -1 + else: + yield taskContext.partitionId() + + # for normal stage + result1 = rdd.mapPartitions(f).collect() + self.assertTrue(result1 == [-1, -1, -1, -1]) + # for barrier stage + result2 = rdd.barrier().mapPartitions(f).collect() + self.assertTrue(result2 == [0, 1, 2, 3]) + class TaskContextTestsWithWorkerReuse(unittest.TestCase): @@ -181,6 +227,45 @@ def context_barrier(x): for pid in pids: self.assertTrue(pid in worker_pids) + def test_task_context_correct_with_python_worker_reuse(self): + """Verify the task context correct when reused python worker""" + # start a normal job first to start all workers and get all worker pids + worker_pids = self.sc.parallelize(xrange(2), 2).map(lambda x: os.getpid()).collect() + # the worker will reuse in this barrier job + rdd = self.sc.parallelize(xrange(10), 2) + + def context(iterator): + tp = TaskContext.get().partitionId() + try: + bp = BarrierTaskContext.get().partitionId() + except Exception: + bp = -1 + + yield (tp, bp, os.getpid()) + + # normal stage after normal stage + normal_result = rdd.mapPartitions(context).collect() + tps, bps, pids = zip(*normal_result) + print(tps) + self.assertTrue(tps == (0, 1)) + self.assertTrue(bps == (-1, -1)) + for pid in pids: + self.assertTrue(pid in worker_pids) + # barrier stage after normal stage + barrier_result = rdd.barrier().mapPartitions(context).collect() + tps, bps, pids = zip(*barrier_result) + self.assertTrue(tps == (0, 1)) + self.assertTrue(bps == (0, 1)) + for pid in pids: + self.assertTrue(pid in worker_pids) + # normal stage after barrier stage + normal_result2 = rdd.mapPartitions(context).collect() + tps, bps, pids = zip(*normal_result2) + self.assertTrue(tps == (0, 1)) + self.assertTrue(bps == (-1, -1)) + for pid in pids: + self.assertTrue(pid in worker_pids) + def tearDown(self): self.sc.stop() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 698193d6bdd87..3a1200eef43f8 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -503,6 +503,9 @@ def main(infile, outfile): if isBarrier: taskContext = BarrierTaskContext._getOrCreate() BarrierTaskContext._initialize(boundPort, secret) + # Set the task context instance here, so we can get it by TaskContext.get for + # both TaskContext and BarrierTaskContext + TaskContext._setTaskContext(taskContext) else: taskContext = TaskContext._getOrCreate() # read inputs for TaskContext info @@ -596,6 +599,11 @@ def process(): profiler.profile(process) else: process() + + # Reset task context to None. This is a guard code to avoid residual context when worker + # reuse. + TaskContext._setTaskContext(None) + BarrierTaskContext._setTaskContext(None) except Exception: try: exc_info = traceback.format_exc() From bb478706b54cb247fe541ce0f860a7874ffbd42f Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 31 Oct 2019 13:52:28 +0800 Subject: [PATCH 0478/1387] [SPARK-29645][ML][PYSPARK] ML add param RelativeError ### What changes were proposed in this pull request? 1, add shared param `relativeError` 2, `Imputer`/`RobusterScaler`/`QuantileDiscretizer` extend `HasRelativeError` ### Why are the changes needed? It makes sense to expose RelativeError to end users, since it controls both the precision and memory overhead. `QuantileDiscretizer` had already added this param, while other algs not yet. ### Does this PR introduce any user-facing change? yes, new param is added in `Imputer`/`RobusterScaler` ### How was this patch tested? existing testsutes Closes #26305 from zhengruifeng/add_relative_err. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../org/apache/spark/ml/feature/Imputer.scala | 10 +++- .../ml/feature/QuantileDiscretizer.scala | 27 ++------- .../spark/ml/feature/RobustScaler.scala | 11 +++- .../ml/param/shared/SharedParamsCodeGen.scala | 3 + .../spark/ml/param/shared/sharedParams.scala | 19 +++++++ project/MimaExcludes.scala | 4 ++ python/pyspark/ml/feature.py | 55 ++++++++++--------- .../ml/param/_shared_params_code_gen.py | 2 + python/pyspark/ml/param/shared.py | 18 ++++++ 9 files changed, 97 insertions(+), 52 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index 5e593f731c624..fbccfb1041d1e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -23,7 +23,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasInputCols, HasOutputCol, HasOutputCols} +import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ @@ -33,7 +33,7 @@ import org.apache.spark.sql.types._ * Params for [[Imputer]] and [[ImputerModel]]. */ private[feature] trait ImputerParams extends Params with HasInputCol with HasInputCols - with HasOutputCol with HasOutputCols { + with HasOutputCol with HasOutputCols with HasRelativeError { /** * The imputation strategy. Currently only "mean" and "median" are supported. @@ -140,6 +140,10 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) @Since("2.2.0") def setMissingValue(value: Double): this.type = set(missingValue, value) + /** @group expertSetParam */ + @Since("3.0.0") + def setRelativeError(value: Double): this.type = set(relativeError, value) + setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN) override def fit(dataset: Dataset[_]): ImputerModel = { @@ -172,7 +176,7 @@ class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) case Imputer.median => // Function approxQuantile will ignore null automatically. // For a column only containing null, approxQuantile will return an empty array. - dataset.select(cols: _*).stat.approxQuantile(inputColumns, Array(0.5), 0.001) + dataset.select(cols: _*).stat.approxQuantile(inputColumns, Array(0.5), $(relativeError)) .map { array => if (array.isEmpty) { Double.NaN diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index eb78d8224fc3f..216d99d01f2f7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -22,7 +22,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol, HasInputCols, HasOutputCol, HasOutputCols} +import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.sql.Dataset import org.apache.spark.sql.types.StructType @@ -31,7 +31,8 @@ import org.apache.spark.sql.types.StructType * Params for [[QuantileDiscretizer]]. */ private[feature] trait QuantileDiscretizerBase extends Params - with HasHandleInvalid with HasInputCol with HasOutputCol with HasInputCols with HasOutputCols { + with HasHandleInvalid with HasInputCol with HasOutputCol with HasInputCols with HasOutputCols + with HasRelativeError { /** * Number of buckets (quantiles, or categories) into which data points are grouped. Must @@ -67,22 +68,6 @@ private[feature] trait QuantileDiscretizerBase extends Params /** @group getParam */ def getNumBucketsArray: Array[Int] = $(numBucketsArray) - /** - * Relative error (see documentation for - * `org.apache.spark.sql.DataFrameStatFunctions.approxQuantile` for description) - * Must be in the range [0, 1]. - * Note that in multiple columns case, relative error is applied to all columns. - * default: 0.001 - * @group param - */ - val relativeError = new DoubleParam(this, "relativeError", "The relative target precision " + - "for the approximate quantile algorithm used to generate buckets. " + - "Must be in the range [0, 1].", ParamValidators.inRange(0.0, 1.0)) - setDefault(relativeError -> 0.001) - - /** @group getParam */ - def getRelativeError: Double = getOrDefault(relativeError) - /** * Param for how to handle invalid entries. Options are 'skip' (filter out rows with * invalid values), 'error' (throw an error), or 'keep' (keep invalid values in a special @@ -98,7 +83,6 @@ private[feature] trait QuantileDiscretizerBase extends Params "error (throw an error), or keep (keep invalid values in a special additional bucket).", ParamValidators.inArray(Bucketizer.supportedHandleInvalids)) setDefault(handleInvalid, Bucketizer.ERROR_INVALID) - } /** @@ -110,7 +94,8 @@ private[feature] trait QuantileDiscretizerBase extends Params * parameter. If both of the `inputCol` and `inputCols` parameters are set, an Exception will be * thrown. To specify the number of buckets for each column, the `numBucketsArray` parameter can * be set, or if the number of buckets should be the same across columns, `numBuckets` can be - * set as a convenience. + * set as a convenience. Note that in multiple columns case, relative error is applied to all + * columns. * * NaN handling: * null and NaN values will be ignored from the column during `QuantileDiscretizer` fitting. This @@ -134,7 +119,7 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui @Since("1.6.0") def this() = this(Identifiable.randomUID("quantileDiscretizer")) - /** @group setParam */ + /** @group expertSetParam */ @Since("2.0.0") def setRelativeError(value: Double): this.type = set(relativeError, value) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala index 9dae39756d31e..1d609ef3190d7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol, HasRelativeError} import org.apache.spark.ml.util._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql._ @@ -34,7 +34,8 @@ import org.apache.spark.sql.types.{StructField, StructType} /** * Params for [[RobustScaler]] and [[RobustScalerModel]]. */ -private[feature] trait RobustScalerParams extends Params with HasInputCol with HasOutputCol { +private[feature] trait RobustScalerParams extends Params with HasInputCol with HasOutputCol + with HasRelativeError { /** * Lower quantile to calculate quantile range, shared by all features @@ -141,8 +142,12 @@ class RobustScaler (override val uid: String) /** @group setParam */ def setWithScaling(value: Boolean): this.type = set(withScaling, value) + /** @group expertSetParam */ + def setRelativeError(value: Double): this.type = set(relativeError, value) + override def fit(dataset: Dataset[_]): RobustScalerModel = { transformSchema(dataset.schema, logging = true) + val localRelativeError = $(relativeError) val summaries = dataset.select($(inputCol)).rdd.map { case Row(vec: Vector) => vec @@ -152,7 +157,7 @@ class RobustScaler (override val uid: String) val vec = iter.next() if (agg == null) { agg = Array.fill(vec.size)( - new QuantileSummaries(QuantileSummaries.defaultCompressThreshold, 0.001)) + new QuantileSummaries(QuantileSummaries.defaultCompressThreshold, localRelativeError)) } require(vec.size == agg.length, s"Number of dimensions must be ${agg.length} but got ${vec.size}") diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index 7ac9a288d285d..7ac680ec1183a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -83,6 +83,9 @@ private[shared] object SharedParamsCodeGen { isValid = "ParamValidators.inRange(0, 1)"), ParamDesc[Double]("tol", "the convergence tolerance for iterative algorithms (>= 0)", isValid = "ParamValidators.gtEq(0)"), + ParamDesc[Double]("relativeError", "the relative target precision for the approximate " + + "quantile algorithm. Must be in the range [0, 1]", + Some("0.001"), isValid = "ParamValidators.inRange(0, 1)", isExpertParam = true), ParamDesc[Double]("stepSize", "Step size to be used for each iteration of optimization (>" + " 0)", isValid = "ParamValidators.gt(0)", finalFields = false), ParamDesc[String]("weightCol", "weight column name. If this is not set or empty, we treat " + diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 6eeeb57e08fb0..44c993eeafddc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -418,6 +418,25 @@ trait HasTol extends Params { final def getTol: Double = $(tol) } +/** + * Trait for shared param relativeError (default: 0.001). This trait may be changed or + * removed between minor versions. + */ +@DeveloperApi +trait HasRelativeError extends Params { + + /** + * Param for the relative target precision for the approximate quantile algorithm. Must be in the range [0, 1]. + * @group expertParam + */ + final val relativeError: DoubleParam = new DoubleParam(this, "relativeError", "the relative target precision for the approximate quantile algorithm. Must be in the range [0, 1]", ParamValidators.inRange(0, 1)) + + setDefault(relativeError, 0.001) + + /** @group expertGetParam */ + final def getRelativeError: Double = $(relativeError) +} + /** * Trait for shared param stepSize. This trait may be changed or * removed between minor versions. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 53a5a5029f8c3..1da31ca9fad96 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -213,6 +213,10 @@ object MimaExcludes { // [SPARK-28780][ML] Delete the incorrect setWeightCol method in LinearSVCModel ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.LinearSVCModel.setWeightCol"), + // [SPARK-29645][ML][PYSPARK] ML add param RelativeError + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.QuantileDiscretizer.relativeError"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.ml.feature.QuantileDiscretizer.getRelativeError"), + // [SPARK-25959] GBTClassifier picks wrong impurity stats on loading ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.tree.HasVarianceImpurity.org$apache$spark$ml$tree$HasVarianceImpurity$_setter_$impurity_="), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.tree.HasVarianceImpurity.org$apache$spark$ml$tree$HasVarianceImpurity$_setter_$impurity_="), diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 7ccdcf8560608..9513b0caecb9a 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1464,7 +1464,7 @@ def numDocs(self): return self._call_java("numDocs") -class _ImputerParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCols): +class _ImputerParams(HasInputCol, HasInputCols, HasOutputCol, HasOutputCols, HasRelativeError): """ Params for :py:class:`Imputer` and :py:class:`ImputerModel`. @@ -1516,6 +1516,8 @@ class Imputer(JavaEstimator, _ImputerParams, JavaMLReadable, JavaMLWritable): Imputer... >>> imputer.setOutputCols(["out_a", "out_b"]) Imputer... + >>> imputer.getRelativeError() + 0.001 >>> model = imputer.fit(df) >>> model.getStrategy() 'mean' @@ -1607,24 +1609,24 @@ class Imputer(JavaEstimator, _ImputerParams, JavaMLReadable, JavaMLWritable): @keyword_only def __init__(self, strategy="mean", missingValue=float("nan"), inputCols=None, - outputCols=None, inputCol=None, outputCol=None): + outputCols=None, inputCol=None, outputCol=None, relativeError=0.001): """ __init__(self, strategy="mean", missingValue=float("nan"), inputCols=None, \ - outputCols=None, inputCol=None, outputCol=None): + outputCols=None, inputCol=None, outputCol=None, relativeError=0.001): """ super(Imputer, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Imputer", self.uid) - self._setDefault(strategy="mean", missingValue=float("nan")) + self._setDefault(strategy="mean", missingValue=float("nan"), relativeError=0.001) kwargs = self._input_kwargs self.setParams(**kwargs) @keyword_only @since("2.2.0") def setParams(self, strategy="mean", missingValue=float("nan"), inputCols=None, - outputCols=None, inputCol=None, outputCol=None): + outputCols=None, inputCol=None, outputCol=None, relativeError=0.001): """ setParams(self, strategy="mean", missingValue=float("nan"), inputCols=None, \ - outputCols=None, inputCol=None, outputCol=None) + outputCols=None, inputCol=None, outputCol=None, relativeError=0.001) Sets params for this Imputer. """ kwargs = self._input_kwargs @@ -1672,6 +1674,13 @@ def setOutputCol(self, value): """ return self._set(outputCol=value) + @since("3.0.0") + def setRelativeError(self, value): + """ + Sets the value of :py:attr:`relativeError`. + """ + return self._set(relativeError=value) + def _create_model(self, java_model): return ImputerModel(java_model) @@ -2635,7 +2644,7 @@ def setOutputCol(self, value): @inherit_doc class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, - HasHandleInvalid, JavaMLReadable, JavaMLWritable): + HasHandleInvalid, HasRelativeError, JavaMLReadable, JavaMLWritable): """ :py:class:`QuantileDiscretizer` takes a column with continuous features and outputs a column with binned categorical features. The number of bins can be set using the :py:attr:`numBuckets` @@ -2730,11 +2739,6 @@ class QuantileDiscretizer(JavaEstimator, HasInputCol, HasOutputCol, HasInputCols "categories) into which data points are grouped. Must be >= 2.", typeConverter=TypeConverters.toInt) - relativeError = Param(Params._dummy(), "relativeError", "The relative target precision for " + - "the approximate quantile algorithm used to generate buckets. " + - "Must be in the range [0, 1].", - typeConverter=TypeConverters.toFloat) - handleInvalid = Param(Params._dummy(), "handleInvalid", "how to handle invalid entries. " + "Options are skip (filter out rows with invalid values), " + "error (throw an error), or keep (keep invalid values in a special " + @@ -2813,13 +2817,6 @@ def setRelativeError(self, value): """ return self._set(relativeError=value) - @since("2.0.0") - def getRelativeError(self): - """ - Gets the value of relativeError or its default value. - """ - return self.getOrDefault(self.relativeError) - def setInputCol(self, value): """ Sets the value of :py:attr:`inputCol`. @@ -2869,7 +2866,7 @@ def _create_model(self, java_model): handleInvalid=self.getHandleInvalid()) -class _RobustScalerParams(HasInputCol, HasOutputCol): +class _RobustScalerParams(HasInputCol, HasOutputCol, HasRelativeError): """ Params for :py:class:`RobustScaler` and :py:class:`RobustScalerModel`. @@ -2965,24 +2962,25 @@ class RobustScaler(JavaEstimator, _RobustScalerParams, JavaMLReadable, JavaMLWri @keyword_only def __init__(self, lower=0.25, upper=0.75, withCentering=False, withScaling=True, - inputCol=None, outputCol=None): + inputCol=None, outputCol=None, relativeError=0.001): """ __init__(self, lower=0.25, upper=0.75, withCentering=False, withScaling=True, \ - inputCol=None, outputCol=None) + inputCol=None, outputCol=None, relativeError=0.001) """ super(RobustScaler, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.RobustScaler", self.uid) - self._setDefault(lower=0.25, upper=0.75, withCentering=False, withScaling=True) + self._setDefault(lower=0.25, upper=0.75, withCentering=False, withScaling=True, + relativeError=0.001) kwargs = self._input_kwargs self.setParams(**kwargs) @keyword_only @since("3.0.0") def setParams(self, lower=0.25, upper=0.75, withCentering=False, withScaling=True, - inputCol=None, outputCol=None): + inputCol=None, outputCol=None, relativeError=0.001): """ setParams(self, lower=0.25, upper=0.75, withCentering=False, withScaling=True, \ - inputCol=None, outputCol=None) + inputCol=None, outputCol=None, relativeError=0.001) Sets params for this RobustScaler. """ kwargs = self._input_kwargs @@ -3030,6 +3028,13 @@ def setOutputCol(self, value): """ return self._set(outputCol=value) + @since("3.0.0") + def setRelativeError(self, value): + """ + Sets the value of :py:attr:`relativeError`. + """ + return self._set(relativeError=value) + def _create_model(self, java_model): return RobustScalerModel(java_model) diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 8ea94e4760007..ded3ca84b30f2 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -123,6 +123,8 @@ def get$Name(self): ("seed", "random seed.", "hash(type(self).__name__)", "TypeConverters.toInt"), ("tol", "the convergence tolerance for iterative algorithms (>= 0).", None, "TypeConverters.toFloat"), + ("relativeError", "the relative target precision for the approximate quantile " + + "algorithm. Must be in the range [0, 1]", "0.001", "TypeConverters.toFloat"), ("stepSize", "Step size to be used for each iteration of optimization (>= 0).", None, "TypeConverters.toFloat"), ("handleInvalid", "how to handle invalid entries. Options are skip (which will filter " + diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 26d74fab6975a..8fc115691f1ab 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -283,6 +283,24 @@ def getTol(self): return self.getOrDefault(self.tol) +class HasRelativeError(Params): + """ + Mixin for param relativeError: the relative target precision for the approximate quantile algorithm. Must be in the range [0, 1] + """ + + relativeError = Param(Params._dummy(), "relativeError", "the relative target precision for the approximate quantile algorithm. Must be in the range [0, 1]", typeConverter=TypeConverters.toFloat) + + def __init__(self): + super(HasRelativeError, self).__init__() + self._setDefault(relativeError=0.001) + + def getRelativeError(self): + """ + Gets the value of relativeError or its default value. + """ + return self.getOrDefault(self.relativeError) + + class HasStepSize(Params): """ Mixin for param stepSize: Step size to be used for each iteration of optimization (>= 0). From 095f7b05fd7ae8ce0d8a82f0c4bc26aa92853762 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 30 Oct 2019 23:11:22 -0700 Subject: [PATCH 0479/1387] Revert "[SPARK-29277][SQL] Add early DSv2 filter and projection pushdown" This reverts commit cfc80d0eb18e1ec2866204da3500acd5f4dde2ea. --- .../sql/catalyst/analysis/Analyzer.scala | 14 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 14 -- .../sql/catalyst/optimizer/Optimizer.scala | 14 +- .../catalyst/plans/logical/v2Commands.scala | 6 +- .../sql/connector/catalog/CatalogV2Util.scala | 7 +- .../datasources/v2/DataSourceV2Relation.scala | 52 +------- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../spark/sql/execution/SparkOptimizer.scala | 12 +- .../datasources/v2/DataSourceV2Strategy.scala | 123 ++++++++++++++++-- .../datasources/v2/PushDownUtils.scala | 103 --------------- .../v2/V2ScanRelationPushDown.scala | 66 ---------- .../spark/sql/FileBasedDataSourceSuite.scala | 10 +- .../sql/connector/DataSourceV2Suite.scala | 8 +- .../execution/datasources/orc/OrcTest.scala | 11 +- .../orc/OrcV2SchemaPruningSuite.scala | 2 +- .../parquet/ParquetFilterSuite.scala | 11 +- .../datasources/orc/OrcFilterSuite.scala | 10 +- 17 files changed, 162 insertions(+), 305 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index de8f3e252142e..d92987887b70c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -681,18 +681,10 @@ class Analyzer( .map(v2Relation => i.copy(table = v2Relation)) .getOrElse(i) - case desc @ DescribeTable(u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.tableName) - .map(rel => desc.copy(table = rel)) - .getOrElse(desc) - - case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.tableName) - .map(rel => alter.copy(table = rel)) - .getOrElse(alter) - case u: UnresolvedV2Relation => - CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) + CatalogV2Util.loadTable(u.catalog, u.tableName).map { table => + DataSourceV2Relation.create(table) + }.getOrElse(u) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 72612d1dc76c9..d9dc9ebbcaf3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -104,20 +104,6 @@ trait CheckAnalysis extends PredicateHelper { case u: UnresolvedV2Relation => u.failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") - case AlterTable(_, _, u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => - u.failAnalysis( - s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") - - case AlterTable(_, _, u: UnresolvedV2Relation, _) => - failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") - - case DescribeTable(u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => - u.failAnalysis( - s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") - - case DescribeTable(u: UnresolvedV2Relation, _) => - failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") - case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. // If the arguments of the higher-order functions are resolved but the type check fails, 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 85b65edbb64b4..eab4c3efe4f7f 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 @@ -119,7 +119,7 @@ abstract class Optimizer(catalogManager: CatalogManager) rulesWithoutInferFiltersFromConstraints: _*) :: Nil } - val batches = (Batch("Eliminate Distinct", Once, EliminateDistinct) :: + (Batch("Eliminate Distinct", Once, EliminateDistinct) :: // Technically some of the rules in Finish Analysis are not optimizer rules and belong more // in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime). // However, because we also use the analyzer to canonicalized queries (for view definition), @@ -170,10 +170,6 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveLiteralFromGroupExpressions, RemoveRepetitionFromGroupExpressions) :: Nil ++ operatorOptimizationBatch) :+ - // This batch pushes filters and projections into scan nodes. Before this batch, the logical - // plan may contain nodes that do not report stats. Anything that uses stats must run after - // this batch. - Batch("Early Filter and Projection Push-Down", Once, earlyScanPushDownRules: _*) :+ // Since join costs in AQP can change between multiple runs, there is no reason that we have an // idempotence enforcement on this batch. We thus make it FixedPoint(1) instead of Once. Batch("Join Reorder", FixedPoint(1), @@ -200,9 +196,6 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveNoopOperators) :+ // This batch must be executed after the `RewriteSubquery` batch, which creates joins. Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) - - // remove any batches with no rules. this may happen when subclasses do not add optional rules. - batches.filter(_.rules.nonEmpty) } /** @@ -260,11 +253,6 @@ abstract class Optimizer(catalogManager: CatalogManager) */ def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil - /** - * Override to provide additional rules for early projection and filter pushdown to scans. - */ - def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = Nil - /** * Returns (defaultBatches - (excludedRules - nonExcludableRules)), the rule batches that * eventually run in the Optimizer. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 3c625e9acb5a7..f587ee2928fc2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -271,7 +271,7 @@ case class ShowNamespaces( */ case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { - override lazy val resolved: Boolean = table.resolved + override def children: Seq[LogicalPlan] = Seq(table) override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() } @@ -313,7 +313,9 @@ case class AlterTable( table: NamedRelation, changes: Seq[TableChange]) extends Command { - override lazy val resolved: Boolean = table.resolved && { + override def children: Seq[LogicalPlan] = Seq(table) + + override lazy val resolved: Boolean = childrenResolved && { changes.forall { case add: AddColumn => add.fieldNames match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 0f313e7b96d86..6d8c6f8456412 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -24,10 +24,9 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.plans.logical.AlterTable import org.apache.spark.sql.connector.catalog.TableChange._ -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} private[sql] object CatalogV2Util { @@ -225,10 +224,6 @@ private[sql] object CatalogV2Util { case _: NoSuchNamespaceException => None } - def loadRelation(catalog: CatalogPlugin, ident: Identifier): Option[NamedRelation] = { - loadTable(catalog, ident).map(DataSourceV2Relation.create) - } - def isSessionCatalog(catalog: CatalogPlugin): Boolean = { catalog.name().equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 87d3419e8115f..7da502fc298ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, Statistics => V2S import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.util.CaseInsensitiveStringMap -import org.apache.spark.util.Utils /** * A logical plan representing a data source v2 table. @@ -51,53 +50,12 @@ case class DataSourceV2Relation( s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" } - override def computeStats(): Statistics = { - if (Utils.isTesting) { - // when testing, throw an exception if this computeStats method is called because stats should - // not be accessed before pushing the projection and filters to create a scan. otherwise, the - // stats are not accurate because they are based on a full table scan of all columns. - throw new IllegalStateException( - s"BUG: computeStats called before pushdown on DSv2 relation: $name") - } else { - // when not testing, return stats because bad stats are better than failing a query - table.asReadable.newScanBuilder(options) match { - case r: SupportsReportStatistics => - val statistics = r.estimateStatistics() - DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes) - case _ => - Statistics(sizeInBytes = conf.defaultSizeInBytes) - } - } - } - - override def newInstance(): DataSourceV2Relation = { - copy(output = output.map(_.newInstance())) - } -} - -/** - * A logical plan for a DSv2 table with a scan already created. - * - * This is used in the optimizer to push filters and projection down before conversion to physical - * plan. This ensures that the stats that are used by the optimizer account for the filters and - * projection that will be pushed down. - * - * @param table a DSv2 [[Table]] - * @param scan a DSv2 [[Scan]] - * @param output the output attributes of this relation - */ -case class DataSourceV2ScanRelation( - table: Table, - scan: Scan, - output: Seq[AttributeReference]) extends LeafNode with NamedRelation { - - override def name: String = table.name() - - override def simpleString(maxFields: Int): String = { - s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" + def newScanBuilder(): ScanBuilder = { + table.asReadable.newScanBuilder(options) } override def computeStats(): Statistics = { + val scan = newScanBuilder().build() scan match { case r: SupportsReportStatistics => val statistics = r.estimateStatistics() @@ -106,6 +64,10 @@ case class DataSourceV2ScanRelation( Statistics(sizeInBytes = conf.defaultSizeInBytes) } } + + override def newInstance(): DataSourceV2Relation = { + copy(output = output.map(_.newInstance())) + } } /** 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 607f495139ec1..5f6e0a82be4ce 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 @@ -51,7 +51,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, FileTable} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.internal.SQLConf @@ -3218,7 +3218,7 @@ class Dataset[T] private[sql]( fr.inputFiles case r: HiveTableRelation => r.tableMeta.storage.locationUri.map(_.toString).toArray - case DataSourceV2ScanRelation(table: FileTable, _, _) => + case DataSourceV2Relation(table: FileTable, _, _) => table.fileIndex.inputFiles }.flatten files.toSet.toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index e65faefad5b9e..4a8b56faddf7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -20,13 +20,10 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.ExperimentalMethods import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.dynamicpruning.{CleanupDynamicPruningFilters, PartitionPruning} import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning -import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown import org.apache.spark.sql.execution.python.{ExtractGroupingPythonUDFFromAggregate, ExtractPythonUDFFromAggregate, ExtractPythonUDFs} class SparkOptimizer( @@ -35,12 +32,10 @@ class SparkOptimizer( experimentalMethods: ExperimentalMethods) extends Optimizer(catalogManager) { - override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = - // TODO: move SchemaPruning into catalyst - SchemaPruning :: PruneFileSourcePartitions :: V2ScanRelationPushDown :: Nil - override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ + Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+ + Batch("Schema Pruning", Once, SchemaPruning) :+ Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ @@ -69,8 +64,7 @@ class SparkOptimizer( override def nonExcludableRules: Seq[String] = super.nonExcludableRules :+ ExtractPythonUDFFromJoinCondition.ruleName :+ ExtractPythonUDFFromAggregate.ruleName :+ ExtractGroupingPythonUDFFromAggregate.ruleName :+ - ExtractPythonUDFs.ruleName :+ - V2ScanRelationPushDown.ruleName + ExtractPythonUDFs.ruleName /** * Optimization batches that are executed before the regular optimization batches (also before diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index bc66c154b57ab..3041e9e82d59f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -18,30 +18,127 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} -import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} +import org.apache.spark.sql.sources import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Strategy extends Strategy with PredicateHelper { + /** + * Pushes down filters to the data source reader + * + * @return pushed filter and post-scan filters. + */ + private def pushFilters( + scanBuilder: ScanBuilder, + filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + scanBuilder match { + case r: SupportsPushDownFilters => + // A map from translated data source leaf node filters to original catalyst filter + // expressions. For a `And`/`Or` predicate, it is possible that the predicate is partially + // pushed down. This map can be used to construct a catalyst filter expression from the + // input filter, or a superset(partial push down filter) of the input filter. + val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] + val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] + // Catalyst filter expression that can't be translated to data source filters. + val untranslatableExprs = mutable.ArrayBuffer.empty[Expression] + + for (filterExpr <- filters) { + val translated = + DataSourceStrategy.translateFilterWithMapping(filterExpr, Some(translatedFilterToExpr)) + if (translated.isEmpty) { + untranslatableExprs += filterExpr + } else { + translatedFilters += translated.get + } + } + + // Data source filters that need to be evaluated again after scanning. which means + // the data source cannot guarantee the rows returned can pass these filters. + // As a result we must return it so Spark can plan an extra filter operator. + val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => + DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) + } + // The filters which are marked as pushed to this data source + val pushedFilters = r.pushedFilters().map { filter => + DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) + } + (pushedFilters, untranslatableExprs ++ postScanFilters) + + case _ => (Nil, filters) + } + } + + /** + * Applies column pruning to the data source, w.r.t. the references of the given expressions. + * + * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), + * and new output attributes after column pruning. + */ + // TODO: nested column pruning. + private def pruneColumns( + scanBuilder: ScanBuilder, + relation: DataSourceV2Relation, + exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { + scanBuilder match { + case r: SupportsPushDownRequiredColumns => + val requiredColumns = AttributeSet(exprs.flatMap(_.references)) + val neededOutput = relation.output.filter(requiredColumns.contains) + if (neededOutput != relation.output) { + r.pruneColumns(neededOutput.toStructType) + val scan = r.build() + val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap + scan -> scan.readSchema().toAttributes.map { + // We have to keep the attribute id during transformation. + a => a.withExprId(nameToAttr(a.name).exprId) + } + } else { + r.build() -> relation.output + } + + case _ => scanBuilder.build() -> relation.output + } + } + import DataSourceV2Implicits._ override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case PhysicalOperation(project, filters, relation: DataSourceV2ScanRelation) => - // projection and filters were already pushed down in the optimizer. - // this uses PhysicalOperation to get the projection and ensure that if the batch scan does - // not support columnar, a projection is added to convert the rows to UnsafeRow. - val batchExec = BatchScanExec(relation.output, relation.scan) - - val filterCondition = filters.reduceLeftOption(And) + case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => + val scanBuilder = relation.newScanBuilder() + + val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) + val normalizedFilters = DataSourceStrategy.normalizeFilters( + withoutSubquery, relation.output) + + // `pushedFilters` will be pushed down and evaluated in the underlying data sources. + // `postScanFilters` need to be evaluated after the scan. + // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. + val (pushedFilters, postScanFiltersWithoutSubquery) = + pushFilters(scanBuilder, normalizedFilters) + val postScanFilters = postScanFiltersWithoutSubquery ++ withSubquery + val (scan, output) = pruneColumns(scanBuilder, relation, project ++ postScanFilters) + logInfo( + s""" + |Pushing operators to ${relation.name} + |Pushed Filters: ${pushedFilters.mkString(", ")} + |Post-Scan Filters: ${postScanFilters.mkString(",")} + |Output: ${output.mkString(", ")} + """.stripMargin) + + val batchExec = BatchScanExec(output, scan) + + val filterCondition = postScanFilters.reduceLeftOption(And) val withFilter = filterCondition.map(FilterExec(_, batchExec)).getOrElse(batchExec) val withProjection = if (withFilter.output != project || !batchExec.supportsColumnar) { @@ -157,19 +254,19 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { OverwritePartitionsDynamicExec( r.table.asWritable, writeOptions.asOptions, planLater(query)) :: Nil - case DeleteFromTable(DataSourceV2ScanRelation(table, _, output), condition) => + case DeleteFromTable(r: DataSourceV2Relation, condition) => if (condition.exists(SubqueryExpression.hasSubquery)) { throw new AnalysisException( s"Delete by condition with subquery is not supported: $condition") } // fail if any filter cannot be converted. correctness depends on removing all matching data. - val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, output) + val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, r.output) .flatMap(splitConjunctivePredicates(_).map { f => DataSourceStrategy.translateFilter(f).getOrElse( throw new AnalysisException(s"Exec update failed:" + s" cannot translate expression to source filter: $f")) }).toArray - DeleteFromTableExec(table.asDeletable, filters) :: Nil + DeleteFromTableExec(r.table.asDeletable, filters) :: Nil case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil @@ -186,8 +283,8 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } - case desc @ DescribeTable(DataSourceV2Relation(table, _, _), isExtended) => - DescribeTableExec(desc.output, table, isExtended) :: Nil + case desc @ DescribeTable(r: DataSourceV2Relation, isExtended) => + DescribeTableExec(desc.output, r.table, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala deleted file mode 100644 index 634ecfdf7e1d5..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala +++ /dev/null @@ -1,103 +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.sql.execution.datasources.v2 - -import scala.collection.mutable - -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper} -import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} -import org.apache.spark.sql.execution.datasources.DataSourceStrategy -import org.apache.spark.sql.sources - -object PushDownUtils extends PredicateHelper { - /** - * Pushes down filters to the data source reader - * - * @return pushed filter and post-scan filters. - */ - def pushFilters( - scanBuilder: ScanBuilder, - filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - scanBuilder match { - case r: SupportsPushDownFilters => - // A map from translated data source leaf node filters to original catalyst filter - // expressions. For a `And`/`Or` predicate, it is possible that the predicate is partially - // pushed down. This map can be used to construct a catalyst filter expression from the - // input filter, or a superset(partial push down filter) of the input filter. - val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] - val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] - // Catalyst filter expression that can't be translated to data source filters. - val untranslatableExprs = mutable.ArrayBuffer.empty[Expression] - - for (filterExpr <- filters) { - val translated = - DataSourceStrategy.translateFilterWithMapping(filterExpr, Some(translatedFilterToExpr)) - if (translated.isEmpty) { - untranslatableExprs += filterExpr - } else { - translatedFilters += translated.get - } - } - - // Data source filters that need to be evaluated again after scanning. which means - // the data source cannot guarantee the rows returned can pass these filters. - // As a result we must return it so Spark can plan an extra filter operator. - val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => - DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) - } - // The filters which are marked as pushed to this data source - val pushedFilters = r.pushedFilters().map { filter => - DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) - } - (pushedFilters, untranslatableExprs ++ postScanFilters) - - case _ => (Nil, filters) - } - } - - /** - * Applies column pruning to the data source, w.r.t. the references of the given expressions. - * - * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), - * and new output attributes after column pruning. - */ - // TODO: nested column pruning. - def pruneColumns( - scanBuilder: ScanBuilder, - relation: DataSourceV2Relation, - exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { - scanBuilder match { - case r: SupportsPushDownRequiredColumns => - val requiredColumns = AttributeSet(exprs.flatMap(_.references)) - val neededOutput = relation.output.filter(requiredColumns.contains) - if (neededOutput != relation.output) { - r.pruneColumns(neededOutput.toStructType) - val scan = r.build() - val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap - scan -> scan.readSchema().toAttributes.map { - // We have to keep the attribute id during transformation. - a => a.withExprId(nameToAttr(a.name).exprId) - } - } else { - r.build() -> relation.output - } - - case _ => scanBuilder.build() -> relation.output - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala deleted file mode 100644 index 6aa8d989583d1..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ /dev/null @@ -1,66 +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.sql.execution.datasources.v2 - -import org.apache.spark.sql.catalyst.expressions.{And, SubqueryExpression} -import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.DataSourceStrategy - -object V2ScanRelationPushDown extends Rule[LogicalPlan] { - import DataSourceV2Implicits._ - - override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { - case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => - val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options) - - val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) - val normalizedFilters = DataSourceStrategy.normalizeFilters( - withoutSubquery, relation.output) - - // `pushedFilters` will be pushed down and evaluated in the underlying data sources. - // `postScanFilters` need to be evaluated after the scan. - // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. - val (pushedFilters, postScanFiltersWithoutSubquery) = PushDownUtils.pushFilters( - scanBuilder, normalizedFilters) - val postScanFilters = postScanFiltersWithoutSubquery ++ withSubquery - val (scan, output) = PushDownUtils.pruneColumns( - scanBuilder, relation, project ++ postScanFilters) - logInfo( - s""" - |Pushing operators to ${relation.name} - |Pushed Filters: ${pushedFilters.mkString(", ")} - |Post-Scan Filters: ${postScanFilters.mkString(",")} - |Output: ${output.mkString(", ")} - """.stripMargin) - - val scanRelation = DataSourceV2ScanRelation(relation.table, scan, output) - - val filterCondition = postScanFilters.reduceLeftOption(And) - val withFilter = filterCondition.map(Filter(_, scanRelation)).getOrElse(scanRelation) - - val withProjection = if (withFilter.output != project) { - Project(project, withFilter) - } else { - withFilter - } - - withProjection - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index a7f3e81904de0..d08f4b9066d2b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.io.{File, FileNotFoundException} +import java.io.{File, FilenameFilter, FileNotFoundException} import java.nio.file.{Files, StandardOpenOption} import java.util.Locale @@ -27,9 +27,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} -import org.apache.spark.sql.TestingUDT.{IntervalUDT, NullData, NullUDT} +import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT, NullData, NullUDT} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ @@ -664,7 +664,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { dir.delete() spark.range(1000).write.orc(dir.toString) val df = spark.read.orc(dir.toString) - assert(df.queryExecution.optimizedPlan.stats.sizeInBytes === BigInt(getLocalDirSize(dir))) + assert(df.queryExecution.logical.stats.sizeInBytes === BigInt(getLocalDirSize(dir))) } } } @@ -720,7 +720,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { .option("path", paths.head.getCanonicalPath) .parquet(paths(1).getCanonicalPath, paths(2).getCanonicalPath) df.queryExecution.optimizedPlan match { - case PhysicalOperation(_, _, DataSourceV2ScanRelation(table: ParquetTable, _, _)) => + case PhysicalOperation(_, _, DataSourceV2Relation(table: ParquetTable, _, _)) => assert(table.paths.toSet == paths.map(_.getCanonicalPath).toSet) case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 55c71c7d02d2b..138bbc3f04f64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapabil import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Distribution, Partitioning} -import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.functions._ @@ -195,7 +195,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { withClue(cls.getName) { val df = spark.read.format(cls.getName).load() val logical = df.queryExecution.optimizedPlan.collect { - case d: DataSourceV2ScanRelation => d + case d: DataSourceV2Relation => d }.head val statics = logical.computeStats() @@ -332,7 +332,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { test("SPARK-23315: get output from canonicalized data source v2 related plans") { def checkCanonicalizedOutput( df: DataFrame, logicalNumOutput: Int, physicalNumOutput: Int): Unit = { - val logical = df.queryExecution.logical.collect { + val logical = df.queryExecution.optimizedPlan.collect { case d: DataSourceV2Relation => d }.head assert(logical.canonicalized.output.length == logicalNumOutput) @@ -356,7 +356,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { .read .option(optionName, false) .format(classOf[DataSourceV2WithSessionConfig].getName).load() - val options = df.queryExecution.logical.collectFirst { + val options = df.queryExecution.optimizedPlan.collectFirst { case d: DataSourceV2Relation => d.options }.get assert(options.get(optionName) === "false") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index 528c3474a17c5..adbd93dcb4fe8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -27,9 +27,9 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.FileBasedDataSourceTest -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileBasedDataSourceTest} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION @@ -119,8 +119,11 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2ScanRelation(_, OrcScan(_, _, _, _, _, _, _, pushedFilters), _)) => + DataSourceV2Relation(orcTable: OrcTable, _, options)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(options) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() if (noneSupported) { assert(pushedFilters.isEmpty, "Unsupported filters should not show in pushed filters") } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala index 80cfbd6a02676..b626edf5dc28e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.orc import org.apache.spark.SparkConf -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.datasources.SchemaPruningSuite import org.apache.spark.sql.execution.datasources.v2.BatchScanExec diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 286bb1e920266..9671866fe1535 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -33,8 +33,9 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.datasources.orc.OrcFilters +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType @@ -1483,10 +1484,12 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { query.queryExecution.optimizedPlan.collectFirst { case PhysicalOperation(_, filters, - DataSourceV2ScanRelation(_, scan: ParquetScan, _)) => + DataSourceV2Relation(parquetTable: ParquetTable, _, options)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = parquetTable.newScanBuilder(options) val sourceFilters = filters.flatMap(DataSourceStrategy.translateFilter).toArray - val pushedFilters = scan.pushedFilters + scanBuilder.pushFilters(sourceFilters) + val pushedFilters = scanBuilder.pushedFilters() assert(pushedFilters.nonEmpty, "No filter is pushed down") val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) val parquetFilters = createParquetFilters(schema) diff --git a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 80e330b3f2ae2..b1a907f9cba27 100644 --- a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -29,8 +29,9 @@ import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -54,8 +55,11 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2ScanRelation(_, OrcScan(_, _, _, _, _, _, _, pushedFilters), _)) => + DataSourceV2Relation(orcTable: OrcTable, _, options)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(options) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() assert(pushedFilters.nonEmpty, "No filter is pushed down") val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pushedFilters") From 5e9a155ebac8be46da876d0da9e83aab5f2d1376 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 31 Oct 2019 15:35:04 +0800 Subject: [PATCH 0480/1387] [SPARK-29520][SS] Fix checks of negative intervals ### What changes were proposed in this pull request? - Added `getDuration()` to calculate interval duration in specified time units assuming provided days per months - Added `isNegative()` which return `true` is the interval duration is less than 0 - Fix checking negative intervals by using `isNegative()` in structured streaming classes - Fix checking of `year-months` intervals ### Why are the changes needed? This fixes incorrect checking of negative intervals. An interval is negative when its duration is negative but not if interval's months **or** microseconds is negative. Also this fixes checking of `year-month` interval support because the `month` field could be negative. ### Does this PR introduce any user-facing change? Should not ### How was this patch tested? - Added tests for the `getDuration()` and `isNegative()` methods to `IntervalUtilsSuite` - By existing SS tests Closes #26177 from MaxGekk/interval-is-positive. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../analysis/StreamingJoinHelper.scala | 2 +- .../sql/catalyst/expressions/TimeWindow.scala | 2 +- .../plans/logical/EventTimeWatermark.scala | 5 +-- .../sql/catalyst/util/IntervalUtils.scala | 40 ++++++++++++++++++- .../catalyst/util/IntervalUtilsSuite.scala | 36 +++++++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../execution/streaming/GroupStateImpl.scala | 7 ++-- .../sql/execution/streaming/Triggers.scala | 2 +- .../FlatMapGroupsWithStateSuite.scala | 9 +++-- 9 files changed, 90 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala index c1d72f9b58a4b..c64aeff3c238c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala @@ -256,7 +256,7 @@ object StreamingJoinHelper extends PredicateHelper with Logging { val castedLit = lit.dataType match { case CalendarIntervalType => val calendarInterval = lit.value.asInstanceOf[CalendarInterval] - if (calendarInterval.months > 0) { + if (calendarInterval.months != 0) { invalid = true logWarning( s"Failed to extract state value watermark from condition $exprToCollectFrom " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index b9ec933f31493..d3aadad120520 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -103,7 +103,7 @@ object TimeWindow { */ private def getIntervalInMicroSeconds(interval: String): Long = { val cal = IntervalUtils.fromString(interval) - if (cal.months > 0) { + if (cal.months != 0) { throw new IllegalArgumentException( s"Intervals greater than a month is not supported ($interval).") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala index 8441c2c481ec5..b6bf7cd85d472 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types.MetadataBuilder import org.apache.spark.unsafe.types.CalendarInterval @@ -28,9 +29,7 @@ object EventTimeWatermark { val delayKey = "spark.watermarkDelayMs" def getDelayMs(delay: CalendarInterval): Long = { - // We define month as `31 days` to simplify calculation. - val millisPerMonth = TimeUnit.MICROSECONDS.toMillis(CalendarInterval.MICROS_PER_DAY) * 31 - delay.milliseconds + delay.months * millisPerMonth + IntervalUtils.getDuration(delay, TimeUnit.MILLISECONDS) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 59865d3a02d42..90e2402a5d7da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.util -import java.util.regex.Pattern +import java.util.concurrent.TimeUnit import scala.util.control.NonFatal @@ -325,4 +325,42 @@ object IntervalUtils { "Interval string does not match second-nano format of ss.nnnnnnnnn") } } + + /** + * Gets interval duration + * + * @param interval The interval to get duration + * @param targetUnit Time units of the result + * @param daysPerMonth The number of days per one month. The default value is 31 days + * per month. This value was taken as the default because it is used + * in Structured Streaming for watermark calculations. Having 31 days + * per month, we can guarantee that events are not dropped before + * the end of any month (February with 29 days or January with 31 days). + * @return Duration in the specified time units + */ + def getDuration( + interval: CalendarInterval, + targetUnit: TimeUnit, + daysPerMonth: Int = 31): Long = { + val monthsDuration = Math.multiplyExact( + daysPerMonth * DateTimeUtils.MICROS_PER_DAY, + interval.months) + val result = Math.addExact(interval.microseconds, monthsDuration) + targetUnit.convert(result, TimeUnit.MICROSECONDS) + } + + /** + * Checks the interval is negative + * + * @param interval The checked interval + * @param daysPerMonth The number of days per one month. The default value is 31 days + * per month. This value was taken as the default because it is used + * in Structured Streaming for watermark calculations. Having 31 days + * per month, we can guarantee that events are not dropped before + * the end of any month (February with 29 days or January with 31 days). + * @return true if duration of the given interval is less than 0 otherwise false + */ + def isNegative(interval: CalendarInterval, daysPerMonth: Int = 31): Boolean = { + getDuration(interval, TimeUnit.MICROSECONDS, daysPerMonth) < 0 + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 9addc396b8d33..22944035f31db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.util +import java.util.concurrent.TimeUnit + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.IntervalUtils.{fromDayTimeString, fromString, fromYearMonthString} import org.apache.spark.unsafe.types.CalendarInterval @@ -148,4 +150,38 @@ class IntervalUtilsSuite extends SparkFunSuite { assert(e.getMessage.contains("Cannot support (interval")) } } + + test("interval duration") { + def duration(s: String, unit: TimeUnit, daysPerMonth: Int): Long = { + IntervalUtils.getDuration(fromString(s), unit, daysPerMonth) + } + + assert(duration("0 seconds", TimeUnit.MILLISECONDS, 31) === 0) + assert(duration("1 month", TimeUnit.DAYS, 31) === 31) + assert(duration("1 microsecond", TimeUnit.MICROSECONDS, 30) === 1) + assert(duration("1 month -30 days", TimeUnit.DAYS, 31) === 1) + + try { + duration(Integer.MAX_VALUE + " month", TimeUnit.SECONDS, 31) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: ArithmeticException => + assert(e.getMessage.contains("overflow")) + } + } + + test("negative interval") { + def isNegative(s: String, daysPerMonth: Int): Boolean = { + IntervalUtils.isNegative(fromString(s), daysPerMonth) + } + + assert(isNegative("-1 months", 28)) + assert(isNegative("-1 microsecond", 30)) + assert(isNegative("-1 month 30 days", 31)) + assert(isNegative("2 months -61 days", 30)) + assert(isNegative("-1 year -2 seconds", 30)) + assert(!isNegative("0 months", 28)) + assert(!isNegative("1 year -360 days", 31)) + assert(!isNegative("-1 year 380 days", 31)) + } } 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 5f6e0a82be4ce..a88fd5111221c 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 @@ -732,7 +732,7 @@ class Dataset[T] private[sql]( s"Unable to parse time delay '$delayThreshold'", cause = Some(e)) } - require(parsedDelay.milliseconds >= 0 && parsedDelay.months >= 0, + require(!IntervalUtils.isNegative(parsedDelay), s"delay threshold ($delayThreshold) should not be negative.") EliminateEventTimeWatermark( EventTimeWatermark(UnresolvedAttribute(eventTime), parsedDelay, logicalPlan)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala index d191a79187f28..aac5da8104a8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala @@ -161,12 +161,11 @@ private[sql] class GroupStateImpl[S] private( private def parseDuration(duration: String): Long = { val cal = IntervalUtils.fromString(duration) - if (cal.milliseconds < 0 || cal.months < 0) { - throw new IllegalArgumentException(s"Provided duration ($duration) is not positive") + if (IntervalUtils.isNegative(cal)) { + throw new IllegalArgumentException(s"Provided duration ($duration) is negative") } - val millisPerMonth = TimeUnit.MICROSECONDS.toMillis(CalendarInterval.MICROS_PER_DAY) * 31 - cal.milliseconds + cal.months * millisPerMonth + IntervalUtils.getDuration(cal, TimeUnit.MILLISECONDS) } private def checkTimeoutTimestampAllowed(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index daa70a12ba0e7..48113d1c18b25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -31,7 +31,7 @@ private object Triggers { def convert(interval: String): Long = { val cal = IntervalUtils.fromString(interval) - if (cal.months > 0) { + if (cal.months != 0) { throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") } TimeUnit.MICROSECONDS.toMillis(cal.microseconds) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala index df7e9217f9140..d36c64f61a726 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala @@ -125,6 +125,8 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { var state: GroupStateImpl[Int] = GroupStateImpl.createForStreaming( None, 1000, 1000, ProcessingTimeTimeout, hasTimedOut = false, watermarkPresent = false) assert(state.getTimeoutTimestamp === NO_TIMESTAMP) + state.setTimeoutDuration("-1 month 31 days 1 second") + assert(state.getTimeoutTimestamp === 2000) state.setTimeoutDuration(500) assert(state.getTimeoutTimestamp === 1500) // can be set without initializing state testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) @@ -225,8 +227,9 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { testIllegalTimeout { state.setTimeoutDuration("-1 month") } + testIllegalTimeout { - state.setTimeoutDuration("1 month -1 day") + state.setTimeoutDuration("1 month -31 day") } state = GroupStateImpl.createForStreaming( @@ -241,7 +244,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { state.setTimeoutTimestamp(10000, "-1 month") } testIllegalTimeout { - state.setTimeoutTimestamp(10000, "1 month -1 day") + state.setTimeoutTimestamp(10000, "1 month -32 day") } testIllegalTimeout { state.setTimeoutTimestamp(new Date(-10000)) @@ -253,7 +256,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { state.setTimeoutTimestamp(new Date(-10000), "-1 month") } testIllegalTimeout { - state.setTimeoutTimestamp(new Date(-10000), "1 month -1 day") + state.setTimeoutTimestamp(new Date(-10000), "1 month -32 day") } } From 4d302cb7ed8a9bf3253c45db12642a709a5ece6b Mon Sep 17 00:00:00 2001 From: maryannxue Date: Thu, 31 Oct 2019 15:43:02 +0800 Subject: [PATCH 0481/1387] [SPARK-11150][SQL][FOLLOW-UP] Dynamic partition pruning ### What changes were proposed in this pull request? This is code cleanup PR for https://github.com/apache/spark/pull/25600, aiming to remove an unnecessary condition and to correct a code comment. ### Why are the changes needed? For code cleanup only. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Passed existing tests. Closes #26328 from maryannxue/dpp-followup. Authored-by: maryannxue Signed-off-by: Wenchen Fan --- .../sql/execution/datasources/PruneFileSourcePartitions.scala | 2 +- .../main/scala/org/apache/spark/sql/execution/subquery.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 927e77a53bf47..4725b19763930 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -48,7 +48,7 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { partitionSchema, sparkSession.sessionState.analyzer.resolver) val partitionSet = AttributeSet(partitionColumns) val partitionKeyFilters = ExpressionSet(normalizedFilters.filter { f => - f.references.subsetOf(partitionSet) && f.find(_.isInstanceOf[SubqueryExpression]).isEmpty + f.references.subsetOf(partitionSet) }) if (partitionKeyFilters.nonEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 4215d62bfb780..adb4956c6d3cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -172,7 +172,7 @@ case class InSubqueryExec( } /** - * Plans scalar subqueries from that are present in the given [[SparkPlan]]. + * Plans subqueries that are present in the given [[SparkPlan]]. */ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = { From cd39cd4bceab8c9d2082dc8de7f6675462b2aa20 Mon Sep 17 00:00:00 2001 From: jiake Date: Thu, 31 Oct 2019 21:28:15 +0800 Subject: [PATCH 0482/1387] [SPARK-28560][SQL][FOLLOWUP] support the build side to local shuffle reader as far as possible in BroadcastHashJoin ### What changes were proposed in this pull request? [PR#25295](https://github.com/apache/spark/pull/25295) already implement the rule of converting the shuffle reader to local reader for the `BroadcastHashJoin` in probe side. This PR support converting the shuffle reader to local reader in build side. ### Why are the changes needed? Improve performance ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing unit tests Closes #26289 from JkSelf/supportTwoSideLocalReader. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../adaptive/AdaptiveSparkPlanExec.scala | 2 - .../adaptive/OptimizeLocalShuffleReader.scala | 92 ++++++++-------- .../adaptive/ReduceNumShufflePartitions.scala | 3 +- .../ReduceNumShufflePartitionsSuite.scala | 2 +- .../adaptive/AdaptiveQueryExecSuite.scala | 101 +++++++++++++----- 5 files changed, 128 insertions(+), 72 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index f01947d8f5ed6..8d4731f34ddd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -31,13 +31,11 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._ -import org.apache.spark.sql.execution.adaptive.rule.ReduceNumShufflePartitions import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index d8dd7224fef38..4ddb2154116ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -24,31 +24,51 @@ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartit import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchangeExec} -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.internal.SQLConf -case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { - - def canUseLocalShuffleReaderLeft(join: BroadcastHashJoinExec): Boolean = { - join.buildSide == BuildRight && ShuffleQueryStageExec.isShuffleQueryStageExec(join.left) +object BroadcastJoinWithShuffleLeft { + def unapply(plan: SparkPlan): Option[(QueryStageExec, BuildSide)] = plan match { + case join: BroadcastHashJoinExec if ShuffleQueryStageExec.isShuffleQueryStageExec(join.left) => + Some((join.left.asInstanceOf[QueryStageExec], join.buildSide)) + case _ => None } +} - def canUseLocalShuffleReaderRight(join: BroadcastHashJoinExec): Boolean = { - join.buildSide == BuildLeft && ShuffleQueryStageExec.isShuffleQueryStageExec(join.right) +object BroadcastJoinWithShuffleRight { + def unapply(plan: SparkPlan): Option[(QueryStageExec, BuildSide)] = plan match { + case join: BroadcastHashJoinExec if ShuffleQueryStageExec.isShuffleQueryStageExec(join.right) => + Some((join.right.asInstanceOf[QueryStageExec], join.buildSide)) + case _ => None } +} + +/** + * A rule to optimize the shuffle reader to local reader as far as possible + * when converting the 'SortMergeJoinExec' to 'BroadcastHashJoinExec' in runtime. + * + * This rule can be divided into two steps: + * Step1: Add the local reader in probe side and then check whether additional + * shuffle introduced. If introduced, we will revert all the local + * reader in probe side. + * Step2: Add the local reader in build side and will not check whether + * additional shuffle introduced. Because the build side will not introduce + * additional shuffle. + */ +case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { return plan } - - val optimizedPlan = plan.transformDown { - case join: BroadcastHashJoinExec if canUseLocalShuffleReaderRight(join) => - val localReader = LocalShuffleReaderExec(join.right.asInstanceOf[QueryStageExec]) - join.copy(right = localReader) - case join: BroadcastHashJoinExec if canUseLocalShuffleReaderLeft(join) => - val localReader = LocalShuffleReaderExec(join.left.asInstanceOf[QueryStageExec]) - join.copy(left = localReader) + // Add local reader in probe side. + val withProbeSideLocalReader = plan.transformDown { + case join @ BroadcastJoinWithShuffleLeft(shuffleStage, BuildRight) => + val localReader = LocalShuffleReaderExec(shuffleStage) + join.asInstanceOf[BroadcastHashJoinExec].copy(left = localReader) + case join @ BroadcastJoinWithShuffleRight(shuffleStage, BuildLeft) => + val localReader = LocalShuffleReaderExec(shuffleStage) + join.asInstanceOf[BroadcastHashJoinExec].copy(right = localReader) } def numExchanges(plan: SparkPlan): Int = { @@ -56,16 +76,25 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { case e: ShuffleExchangeExec => e }.length } - + // Check whether additional shuffle introduced. If introduced, revert the local reader. val numExchangeBefore = numExchanges(EnsureRequirements(conf).apply(plan)) - val numExchangeAfter = numExchanges(EnsureRequirements(conf).apply(optimizedPlan)) - - if (numExchangeAfter > numExchangeBefore) { - logDebug("OptimizeLocalShuffleReader rule is not applied due" + + val numExchangeAfter = numExchanges(EnsureRequirements(conf).apply(withProbeSideLocalReader)) + val optimizedPlan = if (numExchangeAfter > numExchangeBefore) { + logDebug("OptimizeLocalShuffleReader rule is not applied in the probe side due" + " to additional shuffles will be introduced.") plan } else { - optimizedPlan + withProbeSideLocalReader + } + // Add the local reader in build side and and do not need to check whether + // additional shuffle introduced. + optimizedPlan.transformDown { + case join @ BroadcastJoinWithShuffleLeft(shuffleStage, BuildLeft) => + val localReader = LocalShuffleReaderExec(shuffleStage) + join.asInstanceOf[BroadcastHashJoinExec].copy(left = localReader) + case join @ BroadcastJoinWithShuffleRight(shuffleStage, BuildRight) => + val localReader = LocalShuffleReaderExec(shuffleStage) + join.asInstanceOf[BroadcastHashJoinExec].copy(right = localReader) } } } @@ -108,25 +137,4 @@ case class LocalShuffleReaderExec(child: QueryStageExec) extends UnaryExecNode { } cachedShuffleRDD } - - override def generateTreeString( - depth: Int, - lastChildren: Seq[Boolean], - append: String => Unit, - verbose: Boolean, - prefix: String = "", - addSuffix: Boolean = false, - maxFields: Int, - printNodeId: Boolean): Unit = { - super.generateTreeString(depth, - lastChildren, - append, - verbose, - prefix, - addSuffix, - maxFields, - printNodeId) - child.generateTreeString( - depth + 1, lastChildren :+ true, append, verbose, "", false, maxFields, printNodeId) - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 5a505c213a262..3b02ddadd2daf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.adaptive.rule +package org.apache.spark.sql.execution.adaptive import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration.Duration @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.{LocalShuffleReaderExec, QueryStageExec, ReusedQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index b5dbdd0b18b49..4d408cd8ebd70 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.{MapOutputStatistics, SparkConf, SparkFunSuite} import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.adaptive._ -import org.apache.spark.sql.execution.adaptive.rule.{CoalescedShuffleReaderExec, ReduceNumShufflePartitions} +import org.apache.spark.sql.execution.adaptive.{CoalescedShuffleReaderExec, ReduceNumShufflePartitions} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 649467a27d93c..b140b08950db4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -93,7 +93,7 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - checkNumLocalShuffleReaders(adaptivePlan, 1) + checkNumLocalShuffleReaders(adaptivePlan, 2) } } @@ -110,7 +110,7 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - checkNumLocalShuffleReaders(adaptivePlan, 1) + checkNumLocalShuffleReaders(adaptivePlan, 2) } } @@ -125,7 +125,7 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - checkNumLocalShuffleReaders(adaptivePlan, 1) + checkNumLocalShuffleReaders(adaptivePlan, 2) } } @@ -141,7 +141,7 @@ class AdaptiveQueryExecSuite val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - checkNumLocalShuffleReaders(adaptivePlan, 1) + checkNumLocalShuffleReaders(adaptivePlan, 2) } } @@ -163,9 +163,28 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - // The child of remaining one BroadcastHashJoin is not ShuffleQueryStage. - // So only two LocalShuffleReader. - checkNumLocalShuffleReaders(adaptivePlan, 2) + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // After applied the 'OptimizeLocalShuffleReader' rule, we can convert all the four + // shuffle reader to local shuffle reader in the bottom two 'BroadcastHashJoin'. + // For the top level 'BroadcastHashJoin', the probe side is not shuffle query stage + // and the build side shuffle query stage is also converted to local shuffle reader. + checkNumLocalShuffleReaders(adaptivePlan, 5) } } @@ -189,9 +208,24 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - // The child of remaining two BroadcastHashJoin is not ShuffleQueryStage. - // So only two LocalShuffleReader. - checkNumLocalShuffleReaders(adaptivePlan, 1) + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + checkNumLocalShuffleReaders(adaptivePlan, 4) } } @@ -215,9 +249,25 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 3) - // The child of remaining two BroadcastHashJoin is not ShuffleQueryStage. - // So only two LocalShuffleReader. - checkNumLocalShuffleReaders(adaptivePlan, 1) + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- Filter + // +- HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + checkNumLocalShuffleReaders(adaptivePlan, 4) } } @@ -232,8 +282,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 3) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 2) - checkNumLocalShuffleReaders(adaptivePlan, 2) - // Even with local shuffle reader, the query statge reuse can also work. + checkNumLocalShuffleReaders(adaptivePlan, 4) + // Even with local shuffle reader, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.size == 1) } @@ -250,8 +300,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - checkNumLocalShuffleReaders(adaptivePlan, 1) - // Even with local shuffle reader, the query statge reuse can also work. + checkNumLocalShuffleReaders(adaptivePlan, 2) + // Even with local shuffle reader, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.size == 1) } @@ -270,8 +320,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - checkNumLocalShuffleReaders(adaptivePlan, 1) - // Even with local shuffle reader, the query statge reuse can also work. + checkNumLocalShuffleReaders(adaptivePlan, 2) + // Even with local shuffle reader, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) val sub = findReusedSubquery(adaptivePlan) @@ -291,8 +341,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - checkNumLocalShuffleReaders(adaptivePlan, 1) - // Even with local shuffle reader, the query statge reuse can also work. + checkNumLocalShuffleReaders(adaptivePlan, 2) + // Even with local shuffle reader, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.isEmpty) val sub = findReusedSubquery(adaptivePlan) @@ -315,8 +365,8 @@ class AdaptiveQueryExecSuite assert(smj.size == 1) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - checkNumLocalShuffleReaders(adaptivePlan, 1) - // Even with local shuffle reader, the query statge reuse can also work. + checkNumLocalShuffleReaders(adaptivePlan, 2) + // Even with local shuffle reader, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) assert(ex.head.plan.isInstanceOf[BroadcastQueryStageExec]) @@ -393,8 +443,9 @@ class AdaptiveQueryExecSuite assert(smj.size == 2) val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) assert(bhj.size == 1) - // additional shuffle exchange introduced, so revert OptimizeLocalShuffleReader rule. - checkNumLocalShuffleReaders(adaptivePlan, 0) + // Even additional shuffle exchange introduced, we still + // can convert the shuffle reader to local reader in build side. + checkNumLocalShuffleReaders(adaptivePlan, 1) } } From faf220aad9051c224a630e678c54098861f6b482 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 31 Oct 2019 08:25:32 -0700 Subject: [PATCH 0483/1387] [SPARK-29277][SQL][test-hadoop3.2] Add early DSv2 filter and projection pushdown Bring back https://github.com/apache/spark/pull/25955 ### What changes were proposed in this pull request? This adds a new rule, `V2ScanRelationPushDown`, to push filters and projections in to a new `DataSourceV2ScanRelation` in the optimizer. That scan is then used when converting to a physical scan node. The new relation correctly reports stats based on the scan. To run scan pushdown before rules where stats are used, this adds a new optimizer override, `earlyScanPushDownRules` and a batch for early pushdown in the optimizer, before cost-based join reordering. The other early pushdown rule, `PruneFileSourcePartitions`, is moved into the early pushdown rule set. This also moves pushdown helper methods from `DataSourceV2Strategy` into a util class. ### Why are the changes needed? This is needed for DSv2 sources to supply stats for cost-based rules in the optimizer. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This updates the implementation of stats from `DataSourceV2Relation` so tests will fail if stats are accessed before early pushdown for v2 relations. Closes #26341 from cloud-fan/back. Lead-authored-by: Wenchen Fan Co-authored-by: Ryan Blue Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 14 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 14 ++ .../sql/catalyst/optimizer/Optimizer.scala | 14 +- .../catalyst/plans/logical/v2Commands.scala | 6 +- .../sql/connector/catalog/CatalogV2Util.scala | 7 +- .../datasources/v2/DataSourceV2Relation.scala | 52 +++++++- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../spark/sql/execution/SparkOptimizer.scala | 12 +- .../datasources/v2/DataSourceV2Strategy.scala | 123 ++---------------- .../datasources/v2/PushDownUtils.scala | 103 +++++++++++++++ .../v2/V2ScanRelationPushDown.scala | 66 ++++++++++ .../spark/sql/FileBasedDataSourceSuite.scala | 10 +- .../sql/connector/DataSourceV2Suite.scala | 8 +- .../execution/datasources/orc/OrcTest.scala | 11 +- .../orc/OrcV2SchemaPruningSuite.scala | 2 +- .../parquet/ParquetFilterSuite.scala | 11 +- .../datasources/orc/OrcFilterSuite.scala | 10 +- .../datasources/orc/OrcFilterSuite.scala | 9 +- 18 files changed, 308 insertions(+), 168 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d92987887b70c..de8f3e252142e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -681,10 +681,18 @@ class Analyzer( .map(v2Relation => i.copy(table = v2Relation)) .getOrElse(i) + case desc @ DescribeTable(u: UnresolvedV2Relation, _) => + CatalogV2Util.loadRelation(u.catalog, u.tableName) + .map(rel => desc.copy(table = rel)) + .getOrElse(desc) + + case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => + CatalogV2Util.loadRelation(u.catalog, u.tableName) + .map(rel => alter.copy(table = rel)) + .getOrElse(alter) + case u: UnresolvedV2Relation => - CatalogV2Util.loadTable(u.catalog, u.tableName).map { table => - DataSourceV2Relation.create(table) - }.getOrElse(u) + CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index d9dc9ebbcaf3c..72612d1dc76c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -104,6 +104,20 @@ trait CheckAnalysis extends PredicateHelper { case u: UnresolvedV2Relation => u.failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + case AlterTable(_, _, u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => + u.failAnalysis( + s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") + + case AlterTable(_, _, u: UnresolvedV2Relation, _) => + failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + + case DescribeTable(u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => + u.failAnalysis( + s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") + + case DescribeTable(u: UnresolvedV2Relation, _) => + failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. // If the arguments of the higher-order functions are resolved but the type check fails, 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 eab4c3efe4f7f..85b65edbb64b4 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 @@ -119,7 +119,7 @@ abstract class Optimizer(catalogManager: CatalogManager) rulesWithoutInferFiltersFromConstraints: _*) :: Nil } - (Batch("Eliminate Distinct", Once, EliminateDistinct) :: + val batches = (Batch("Eliminate Distinct", Once, EliminateDistinct) :: // Technically some of the rules in Finish Analysis are not optimizer rules and belong more // in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime). // However, because we also use the analyzer to canonicalized queries (for view definition), @@ -170,6 +170,10 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveLiteralFromGroupExpressions, RemoveRepetitionFromGroupExpressions) :: Nil ++ operatorOptimizationBatch) :+ + // This batch pushes filters and projections into scan nodes. Before this batch, the logical + // plan may contain nodes that do not report stats. Anything that uses stats must run after + // this batch. + Batch("Early Filter and Projection Push-Down", Once, earlyScanPushDownRules: _*) :+ // Since join costs in AQP can change between multiple runs, there is no reason that we have an // idempotence enforcement on this batch. We thus make it FixedPoint(1) instead of Once. Batch("Join Reorder", FixedPoint(1), @@ -196,6 +200,9 @@ abstract class Optimizer(catalogManager: CatalogManager) RemoveNoopOperators) :+ // This batch must be executed after the `RewriteSubquery` batch, which creates joins. Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) + + // remove any batches with no rules. this may happen when subclasses do not add optional rules. + batches.filter(_.rules.nonEmpty) } /** @@ -253,6 +260,11 @@ abstract class Optimizer(catalogManager: CatalogManager) */ def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = Nil + /** + * Override to provide additional rules for early projection and filter pushdown to scans. + */ + def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = Nil + /** * Returns (defaultBatches - (excludedRules - nonExcludableRules)), the rule batches that * eventually run in the Optimizer. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index f587ee2928fc2..3c625e9acb5a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -271,7 +271,7 @@ case class ShowNamespaces( */ case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { - override def children: Seq[LogicalPlan] = Seq(table) + override lazy val resolved: Boolean = table.resolved override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() } @@ -313,9 +313,7 @@ case class AlterTable( table: NamedRelation, changes: Seq[TableChange]) extends Command { - override def children: Seq[LogicalPlan] = Seq(table) - - override lazy val resolved: Boolean = childrenResolved && { + override lazy val resolved: Boolean = table.resolved && { changes.forall { case add: AddColumn => add.fieldNames match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 6d8c6f8456412..0f313e7b96d86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -24,9 +24,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.plans.logical.AlterTable import org.apache.spark.sql.connector.catalog.TableChange._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} private[sql] object CatalogV2Util { @@ -224,6 +225,10 @@ private[sql] object CatalogV2Util { case _: NoSuchNamespaceException => None } + def loadRelation(catalog: CatalogPlugin, ident: Identifier): Option[NamedRelation] = { + loadTable(catalog, ident).map(DataSourceV2Relation.create) + } + def isSessionCatalog(catalog: CatalogPlugin): Boolean = { catalog.name().equalsIgnoreCase(CatalogManager.SESSION_CATALOG_NAME) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 7da502fc298ea..87d3419e8115f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, Statistics => V2S import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.connector.write.WriteBuilder import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils /** * A logical plan representing a data source v2 table. @@ -50,12 +51,53 @@ case class DataSourceV2Relation( s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" } - def newScanBuilder(): ScanBuilder = { - table.asReadable.newScanBuilder(options) + override def computeStats(): Statistics = { + if (Utils.isTesting) { + // when testing, throw an exception if this computeStats method is called because stats should + // not be accessed before pushing the projection and filters to create a scan. otherwise, the + // stats are not accurate because they are based on a full table scan of all columns. + throw new IllegalStateException( + s"BUG: computeStats called before pushdown on DSv2 relation: $name") + } else { + // when not testing, return stats because bad stats are better than failing a query + table.asReadable.newScanBuilder(options) match { + case r: SupportsReportStatistics => + val statistics = r.estimateStatistics() + DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes) + case _ => + Statistics(sizeInBytes = conf.defaultSizeInBytes) + } + } + } + + override def newInstance(): DataSourceV2Relation = { + copy(output = output.map(_.newInstance())) + } +} + +/** + * A logical plan for a DSv2 table with a scan already created. + * + * This is used in the optimizer to push filters and projection down before conversion to physical + * plan. This ensures that the stats that are used by the optimizer account for the filters and + * projection that will be pushed down. + * + * @param table a DSv2 [[Table]] + * @param scan a DSv2 [[Scan]] + * @param output the output attributes of this relation + */ +case class DataSourceV2ScanRelation( + table: Table, + scan: Scan, + output: Seq[AttributeReference]) extends LeafNode with NamedRelation { + + override def name: String = table.name() + + override def simpleString(maxFields: Int): String = { + s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" } override def computeStats(): Statistics = { - val scan = newScanBuilder().build() scan match { case r: SupportsReportStatistics => val statistics = r.estimateStatistics() @@ -64,10 +106,6 @@ case class DataSourceV2Relation( Statistics(sizeInBytes = conf.defaultSizeInBytes) } } - - override def newInstance(): DataSourceV2Relation = { - copy(output = output.map(_.newInstance())) - } } /** 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 a88fd5111221c..aaa3f9dd71594 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 @@ -51,7 +51,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, FileTable} import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.internal.SQLConf @@ -3218,7 +3218,7 @@ class Dataset[T] private[sql]( fr.inputFiles case r: HiveTableRelation => r.tableMeta.storage.locationUri.map(_.toString).toArray - case DataSourceV2Relation(table: FileTable, _, _) => + case DataSourceV2ScanRelation(table: FileTable, _, _) => table.fileIndex.inputFiles }.flatten files.toSet.toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 4a8b56faddf7e..e65faefad5b9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -20,10 +20,13 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.ExperimentalMethods import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.dynamicpruning.{CleanupDynamicPruningFilters, PartitionPruning} import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions import org.apache.spark.sql.execution.datasources.SchemaPruning +import org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown import org.apache.spark.sql.execution.python.{ExtractGroupingPythonUDFFromAggregate, ExtractPythonUDFFromAggregate, ExtractPythonUDFs} class SparkOptimizer( @@ -32,10 +35,12 @@ class SparkOptimizer( experimentalMethods: ExperimentalMethods) extends Optimizer(catalogManager) { + override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] = + // TODO: move SchemaPruning into catalyst + SchemaPruning :: PruneFileSourcePartitions :: V2ScanRelationPushDown :: Nil + override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ - Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+ - Batch("Schema Pruning", Once, SchemaPruning) :+ Batch("PartitionPruning", Once, PartitionPruning, OptimizeSubqueries) :+ @@ -64,7 +69,8 @@ class SparkOptimizer( override def nonExcludableRules: Seq[String] = super.nonExcludableRules :+ ExtractPythonUDFFromJoinCondition.ruleName :+ ExtractPythonUDFFromAggregate.ruleName :+ ExtractGroupingPythonUDFFromAggregate.ruleName :+ - ExtractPythonUDFs.ruleName + ExtractPythonUDFs.ruleName :+ + V2ScanRelationPushDown.ruleName /** * Optimization batches that are executed before the regular optimization batches (also before diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 3041e9e82d59f..bc66c154b57ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -18,127 +18,30 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ -import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowNamespaces, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} -import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} -import org.apache.spark.sql.sources import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Strategy extends Strategy with PredicateHelper { - /** - * Pushes down filters to the data source reader - * - * @return pushed filter and post-scan filters. - */ - private def pushFilters( - scanBuilder: ScanBuilder, - filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - scanBuilder match { - case r: SupportsPushDownFilters => - // A map from translated data source leaf node filters to original catalyst filter - // expressions. For a `And`/`Or` predicate, it is possible that the predicate is partially - // pushed down. This map can be used to construct a catalyst filter expression from the - // input filter, or a superset(partial push down filter) of the input filter. - val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] - val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] - // Catalyst filter expression that can't be translated to data source filters. - val untranslatableExprs = mutable.ArrayBuffer.empty[Expression] - - for (filterExpr <- filters) { - val translated = - DataSourceStrategy.translateFilterWithMapping(filterExpr, Some(translatedFilterToExpr)) - if (translated.isEmpty) { - untranslatableExprs += filterExpr - } else { - translatedFilters += translated.get - } - } - - // Data source filters that need to be evaluated again after scanning. which means - // the data source cannot guarantee the rows returned can pass these filters. - // As a result we must return it so Spark can plan an extra filter operator. - val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => - DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) - } - // The filters which are marked as pushed to this data source - val pushedFilters = r.pushedFilters().map { filter => - DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) - } - (pushedFilters, untranslatableExprs ++ postScanFilters) - - case _ => (Nil, filters) - } - } - - /** - * Applies column pruning to the data source, w.r.t. the references of the given expressions. - * - * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), - * and new output attributes after column pruning. - */ - // TODO: nested column pruning. - private def pruneColumns( - scanBuilder: ScanBuilder, - relation: DataSourceV2Relation, - exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { - scanBuilder match { - case r: SupportsPushDownRequiredColumns => - val requiredColumns = AttributeSet(exprs.flatMap(_.references)) - val neededOutput = relation.output.filter(requiredColumns.contains) - if (neededOutput != relation.output) { - r.pruneColumns(neededOutput.toStructType) - val scan = r.build() - val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap - scan -> scan.readSchema().toAttributes.map { - // We have to keep the attribute id during transformation. - a => a.withExprId(nameToAttr(a.name).exprId) - } - } else { - r.build() -> relation.output - } - - case _ => scanBuilder.build() -> relation.output - } - } - import DataSourceV2Implicits._ override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => - val scanBuilder = relation.newScanBuilder() - - val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) - val normalizedFilters = DataSourceStrategy.normalizeFilters( - withoutSubquery, relation.output) - - // `pushedFilters` will be pushed down and evaluated in the underlying data sources. - // `postScanFilters` need to be evaluated after the scan. - // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. - val (pushedFilters, postScanFiltersWithoutSubquery) = - pushFilters(scanBuilder, normalizedFilters) - val postScanFilters = postScanFiltersWithoutSubquery ++ withSubquery - val (scan, output) = pruneColumns(scanBuilder, relation, project ++ postScanFilters) - logInfo( - s""" - |Pushing operators to ${relation.name} - |Pushed Filters: ${pushedFilters.mkString(", ")} - |Post-Scan Filters: ${postScanFilters.mkString(",")} - |Output: ${output.mkString(", ")} - """.stripMargin) - - val batchExec = BatchScanExec(output, scan) - - val filterCondition = postScanFilters.reduceLeftOption(And) + case PhysicalOperation(project, filters, relation: DataSourceV2ScanRelation) => + // projection and filters were already pushed down in the optimizer. + // this uses PhysicalOperation to get the projection and ensure that if the batch scan does + // not support columnar, a projection is added to convert the rows to UnsafeRow. + val batchExec = BatchScanExec(relation.output, relation.scan) + + val filterCondition = filters.reduceLeftOption(And) val withFilter = filterCondition.map(FilterExec(_, batchExec)).getOrElse(batchExec) val withProjection = if (withFilter.output != project || !batchExec.supportsColumnar) { @@ -254,19 +157,19 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { OverwritePartitionsDynamicExec( r.table.asWritable, writeOptions.asOptions, planLater(query)) :: Nil - case DeleteFromTable(r: DataSourceV2Relation, condition) => + case DeleteFromTable(DataSourceV2ScanRelation(table, _, output), condition) => if (condition.exists(SubqueryExpression.hasSubquery)) { throw new AnalysisException( s"Delete by condition with subquery is not supported: $condition") } // fail if any filter cannot be converted. correctness depends on removing all matching data. - val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, r.output) + val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, output) .flatMap(splitConjunctivePredicates(_).map { f => DataSourceStrategy.translateFilter(f).getOrElse( throw new AnalysisException(s"Exec update failed:" + s" cannot translate expression to source filter: $f")) }).toArray - DeleteFromTableExec(r.table.asDeletable, filters) :: Nil + DeleteFromTableExec(table.asDeletable, filters) :: Nil case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil @@ -283,8 +186,8 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } - case desc @ DescribeTable(r: DataSourceV2Relation, isExtended) => - DescribeTableExec(desc.output, r.table, isExtended) :: Nil + case desc @ DescribeTable(DataSourceV2Relation(table, _, _), isExtended) => + DescribeTableExec(desc.output, table, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala new file mode 100644 index 0000000000000..634ecfdf7e1d5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.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.sql.execution.datasources.v2 + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.sources + +object PushDownUtils extends PredicateHelper { + /** + * Pushes down filters to the data source reader + * + * @return pushed filter and post-scan filters. + */ + def pushFilters( + scanBuilder: ScanBuilder, + filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + scanBuilder match { + case r: SupportsPushDownFilters => + // A map from translated data source leaf node filters to original catalyst filter + // expressions. For a `And`/`Or` predicate, it is possible that the predicate is partially + // pushed down. This map can be used to construct a catalyst filter expression from the + // input filter, or a superset(partial push down filter) of the input filter. + val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] + val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] + // Catalyst filter expression that can't be translated to data source filters. + val untranslatableExprs = mutable.ArrayBuffer.empty[Expression] + + for (filterExpr <- filters) { + val translated = + DataSourceStrategy.translateFilterWithMapping(filterExpr, Some(translatedFilterToExpr)) + if (translated.isEmpty) { + untranslatableExprs += filterExpr + } else { + translatedFilters += translated.get + } + } + + // Data source filters that need to be evaluated again after scanning. which means + // the data source cannot guarantee the rows returned can pass these filters. + // As a result we must return it so Spark can plan an extra filter operator. + val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => + DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) + } + // The filters which are marked as pushed to this data source + val pushedFilters = r.pushedFilters().map { filter => + DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) + } + (pushedFilters, untranslatableExprs ++ postScanFilters) + + case _ => (Nil, filters) + } + } + + /** + * Applies column pruning to the data source, w.r.t. the references of the given expressions. + * + * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), + * and new output attributes after column pruning. + */ + // TODO: nested column pruning. + def pruneColumns( + scanBuilder: ScanBuilder, + relation: DataSourceV2Relation, + exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { + scanBuilder match { + case r: SupportsPushDownRequiredColumns => + val requiredColumns = AttributeSet(exprs.flatMap(_.references)) + val neededOutput = relation.output.filter(requiredColumns.contains) + if (neededOutput != relation.output) { + r.pruneColumns(neededOutput.toStructType) + val scan = r.build() + val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap + scan -> scan.readSchema().toAttributes.map { + // We have to keep the attribute id during transformation. + a => a.withExprId(nameToAttr(a.name).exprId) + } + } else { + r.build() -> relation.output + } + + case _ => scanBuilder.build() -> relation.output + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala new file mode 100644 index 0000000000000..6aa8d989583d1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.expressions.{And, SubqueryExpression} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.DataSourceStrategy + +object V2ScanRelationPushDown extends Rule[LogicalPlan] { + import DataSourceV2Implicits._ + + override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => + val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options) + + val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) + val normalizedFilters = DataSourceStrategy.normalizeFilters( + withoutSubquery, relation.output) + + // `pushedFilters` will be pushed down and evaluated in the underlying data sources. + // `postScanFilters` need to be evaluated after the scan. + // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. + val (pushedFilters, postScanFiltersWithoutSubquery) = PushDownUtils.pushFilters( + scanBuilder, normalizedFilters) + val postScanFilters = postScanFiltersWithoutSubquery ++ withSubquery + val (scan, output) = PushDownUtils.pruneColumns( + scanBuilder, relation, project ++ postScanFilters) + logInfo( + s""" + |Pushing operators to ${relation.name} + |Pushed Filters: ${pushedFilters.mkString(", ")} + |Post-Scan Filters: ${postScanFilters.mkString(",")} + |Output: ${output.mkString(", ")} + """.stripMargin) + + val scanRelation = DataSourceV2ScanRelation(relation.table, scan, output) + + val filterCondition = postScanFilters.reduceLeftOption(And) + val withFilter = filterCondition.map(Filter(_, scanRelation)).getOrElse(scanRelation) + + val withProjection = if (withFilter.output != project) { + Project(project, withFilter) + } else { + withFilter + } + + withProjection + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index d08f4b9066d2b..a7f3e81904de0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.io.{File, FilenameFilter, FileNotFoundException} +import java.io.{File, FileNotFoundException} import java.nio.file.{Files, StandardOpenOption} import java.util.Locale @@ -27,9 +27,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} -import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT, NullData, NullUDT} +import org.apache.spark.sql.TestingUDT.{IntervalUDT, NullData, NullUDT} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ @@ -664,7 +664,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { dir.delete() spark.range(1000).write.orc(dir.toString) val df = spark.read.orc(dir.toString) - assert(df.queryExecution.logical.stats.sizeInBytes === BigInt(getLocalDirSize(dir))) + assert(df.queryExecution.optimizedPlan.stats.sizeInBytes === BigInt(getLocalDirSize(dir))) } } } @@ -720,7 +720,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { .option("path", paths.head.getCanonicalPath) .parquet(paths(1).getCanonicalPath, paths(2).getCanonicalPath) df.queryExecution.optimizedPlan match { - case PhysicalOperation(_, _, DataSourceV2Relation(table: ParquetTable, _, _)) => + case PhysicalOperation(_, _, DataSourceV2ScanRelation(table: ParquetTable, _, _)) => assert(table.paths.toSet == paths.map(_.getCanonicalPath).toSet) case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 138bbc3f04f64..55c71c7d02d2b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapabil import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Distribution, Partitioning} -import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, DataSourceV2ScanRelation} import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.functions._ @@ -195,7 +195,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { withClue(cls.getName) { val df = spark.read.format(cls.getName).load() val logical = df.queryExecution.optimizedPlan.collect { - case d: DataSourceV2Relation => d + case d: DataSourceV2ScanRelation => d }.head val statics = logical.computeStats() @@ -332,7 +332,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { test("SPARK-23315: get output from canonicalized data source v2 related plans") { def checkCanonicalizedOutput( df: DataFrame, logicalNumOutput: Int, physicalNumOutput: Int): Unit = { - val logical = df.queryExecution.optimizedPlan.collect { + val logical = df.queryExecution.logical.collect { case d: DataSourceV2Relation => d }.head assert(logical.canonicalized.output.length == logicalNumOutput) @@ -356,7 +356,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { .read .option(optionName, false) .format(classOf[DataSourceV2WithSessionConfig].getName).load() - val options = df.queryExecution.optimizedPlan.collectFirst { + val options = df.queryExecution.logical.collectFirst { case d: DataSourceV2Relation => d.options }.get assert(options.get(optionName) === "false") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index adbd93dcb4fe8..528c3474a17c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -27,9 +27,9 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileBasedDataSourceTest} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.execution.datasources.FileBasedDataSourceTest +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION @@ -119,11 +119,8 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2Relation(orcTable: OrcTable, _, options)) => + DataSourceV2ScanRelation(_, OrcScan(_, _, _, _, _, _, _, pushedFilters), _)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") - val scanBuilder = orcTable.newScanBuilder(options) - scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) - val pushedFilters = scanBuilder.pushedFilters() if (noneSupported) { assert(pushedFilters.isEmpty, "Unsupported filters should not show in pushed filters") } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala index b626edf5dc28e..80cfbd6a02676 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.orc import org.apache.spark.SparkConf -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.datasources.SchemaPruningSuite import org.apache.spark.sql.execution.datasources.v2.BatchScanExec diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 9671866fe1535..286bb1e920266 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -33,9 +33,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.orc.OrcFilters -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType @@ -1484,12 +1483,10 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { query.queryExecution.optimizedPlan.collectFirst { case PhysicalOperation(_, filters, - DataSourceV2Relation(parquetTable: ParquetTable, _, options)) => + DataSourceV2ScanRelation(_, scan: ParquetScan, _)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") - val scanBuilder = parquetTable.newScanBuilder(options) val sourceFilters = filters.flatMap(DataSourceStrategy.translateFilter).toArray - scanBuilder.pushFilters(sourceFilters) - val pushedFilters = scanBuilder.pushedFilters() + val pushedFilters = scan.pushedFilters assert(pushedFilters.nonEmpty, "No filter is pushed down") val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) val parquetFilters = createParquetFilters(schema) diff --git a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index b1a907f9cba27..80e330b3f2ae2 100644 --- a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -29,9 +29,8 @@ import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -55,11 +54,8 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2Relation(orcTable: OrcTable, _, options)) => + DataSourceV2ScanRelation(_, OrcScan(_, _, _, _, _, _, _, pushedFilters), _)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") - val scanBuilder = orcTable.newScanBuilder(options) - scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) - val pushedFilters = scanBuilder.pushedFilters() assert(pushedFilters.nonEmpty, "No filter is pushed down") val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pushedFilters") diff --git a/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 65b0537a0a8c1..5e51b8c5aed6a 100644 --- a/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -30,8 +30,8 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.execution.datasources.v2.orc.{OrcScan, OrcTable} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -55,11 +55,8 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { query.queryExecution.optimizedPlan match { case PhysicalOperation(_, filters, - DataSourceV2Relation(orcTable: OrcTable, _, options)) => + DataSourceV2ScanRelation(_, OrcScan(_, _, _, _, _, _, _, pushedFilters), _)) => assert(filters.nonEmpty, "No filter is analyzed from the given query") - val scanBuilder = orcTable.newScanBuilder(options) - scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) - val pushedFilters = scanBuilder.pushedFilters() assert(pushedFilters.nonEmpty, "No filter is pushed down") val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pushedFilters") From 121510cb7b8619cccda63ac32f1b39619ae69160 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 31 Oct 2019 08:34:39 -0700 Subject: [PATCH 0484/1387] [SPARK-29604][SQL][FOLLOWUP][test-hadoop3.2] Let SparkSQLEnvSuite to be run in dedicated JVM ### What changes were proposed in this pull request? This patch addresses CI build issue on sbt Hadoop-3.2 Jenkins job: SparkSQLEnvSuite are failing. Looks like the reason of test failure is the test checks registered listeners from active SparkSession which could be interfered with other test suites running concurrently. If we isolate test suite the problem should be gone. ### Why are the changes needed? CI builds for "spark-master-test-sbt-hadoop-3.2" are failing. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? I've run the single test suite with below command and it passed 3 times sequentially: ``` build/sbt "hive-thriftserver/testOnly *.SparkSQLEnvSuite" -Phadoop-3.2 -Phive-thriftserver ``` so we expect the test suite will pass if we isolate the test suite. Closes #26342 from HeartSaVioR/SPARK-29604-FOLLOWUP. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 88f756b75789b..91d3a75849b0c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -475,7 +475,8 @@ object SparkParallelTestGrouping { "org.apache.spark.ml.classification.LogisticRegressionSuite", "org.apache.spark.ml.classification.LinearSVCSuite", "org.apache.spark.sql.SQLQueryTestSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite" + "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite", + "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite" ) private val DEFAULT_TEST_GROUP = "default_test_group" From 888cc4601a33f7b2479fa40d05dc23a3d05575ed Mon Sep 17 00:00:00 2001 From: ulysses Date: Thu, 31 Oct 2019 09:02:13 -0700 Subject: [PATCH 0485/1387] [SPARK-29675][SQL] Add exception when isolationLevel is Illegal ### What changes were proposed in this pull request? Now we use JDBC api and set an Illegal isolationLevel option, spark will throw a `scala.MatchError`, it's not friendly to user. So we should add an IllegalArgumentException. ### Why are the changes needed? Make exception friendly to user. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Add UT. Closes #26334 from ulysses-you/SPARK-29675. Authored-by: ulysses Signed-off-by: Dongjoon Hyun --- .../execution/datasources/jdbc/JDBCOptions.scala | 4 ++++ .../scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 13 +++++++++++++ 2 files changed, 17 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 5d1feaed81a9a..222ef1145b922 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -177,6 +177,10 @@ class JDBCOptions( case "READ_COMMITTED" => Connection.TRANSACTION_READ_COMMITTED case "REPEATABLE_READ" => Connection.TRANSACTION_REPEATABLE_READ case "SERIALIZABLE" => Connection.TRANSACTION_SERIALIZABLE + case other => throw new IllegalArgumentException( + s"Invalid value `$other` for parameter `$JDBC_TXN_ISOLATION_LEVEL`. This can be " + + "`NONE`, `READ_UNCOMMITTED`, `READ_COMMITTED`, `REPEATABLE_READ` or `SERIALIZABLE`." + ) } // An option to execute custom SQL before fetching data from the remote DB val sessionInitStatement = parameters.get(JDBC_SESSION_INIT_STATEMENT) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 715534b0458dc..472bdda01d137 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -1649,4 +1649,17 @@ class JDBCSuite extends QueryTest } } } + + test("Add exception when isolationLevel is Illegal") { + val e = intercept[IllegalArgumentException] { + spark.read.format("jdbc") + .option("Url", urlWithUserAndPass) + .option("dbTable", "test.people") + .option("isolationLevel", "test") + .load() + }.getMessage + assert(e.contains( + "Invalid value `test` for parameter `isolationLevel`. This can be " + + "`NONE`, `READ_UNCOMMITTED`, `READ_COMMITTED`, `REPEATABLE_READ` or `SERIALIZABLE`.")) + } } From 8a8ac002716d000431b4c8ff582fc1f56740aecc Mon Sep 17 00:00:00 2001 From: ulysses Date: Fri, 1 Nov 2019 08:35:00 +0900 Subject: [PATCH 0486/1387] [SPARK-29687][SQL] Fix JDBC metrics counter data type ### What changes were proposed in this pull request? Fix JDBC metrics counter data type. Related pull request [26109](https://github.com/apache/spark/pull/26109). ### Why are the changes needed? Avoid overflow. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Exists UT. Closes #26346 from ulysses-you/SPARK-29687. Authored-by: ulysses Signed-off-by: Takeshi Yamamuro --- .../apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 55ca4e3624bdd..319cb277cc475 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -652,7 +652,7 @@ object JdbcUtils extends Logging { } } val supportsTransactions = finalIsolationLevel != Connection.TRANSACTION_NONE - var totalRowCount = 0 + var totalRowCount = 0L try { if (supportsTransactions) { conn.setAutoCommit(false) // Everything in the same db transaction. From 3175f4bf1be9a847675b9123c6ee505bd446b419 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 1 Nov 2019 12:25:48 +0900 Subject: [PATCH 0487/1387] [SPARK-29664][PYTHON][SQL] Column.getItem behavior is not consistent with Scala ### What changes were proposed in this pull request? This PR changes the behavior of `Column.getItem` to call `Column.getItem` on Scala side instead of `Column.apply`. ### Why are the changes needed? The current behavior is not consistent with that of Scala. In PySpark: ```Python df = spark.range(2) map_col = create_map(lit(0), lit(100), lit(1), lit(200)) df.withColumn("mapped", map_col.getItem(col('id'))).show() # +---+------+ # | id|mapped| # +---+------+ # | 0| 100| # | 1| 200| # +---+------+ ``` In Scala: ```Scala val df = spark.range(2) val map_col = map(lit(0), lit(100), lit(1), lit(200)) // The following getItem results in the following exception, which is the right behavior: // java.lang.RuntimeException: Unsupported literal type class org.apache.spark.sql.Column id // at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:78) // at org.apache.spark.sql.Column.getItem(Column.scala:856) // ... 49 elided df.withColumn("mapped", map_col.getItem(col("id"))).show ``` ### Does this PR introduce any user-facing change? Yes. If the use wants to pass `Column` object to `getItem`, he/she now needs to use the indexing operator to achieve the previous behavior. ```Python df = spark.range(2) map_col = create_map(lit(0), lit(100), lit(1), lit(200)) df.withColumn("mapped", map_col[col('id'))].show() # +---+------+ # | id|mapped| # +---+------+ # | 0| 100| # | 1| 200| # +---+------+ ``` ### How was this patch tested? Existing tests. Closes #26351 from imback82/spark-29664. Authored-by: Terry Kim Signed-off-by: HyukjinKwon --- docs/pyspark-migration-guide.md | 3 +++ python/pyspark/sql/column.py | 12 +++++------- python/pyspark/sql/tests/test_column.py | 14 +++++++++++++- 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md index 889941c37bf43..1b8d1fc1c5776 100644 --- a/docs/pyspark-migration-guide.md +++ b/docs/pyspark-migration-guide.md @@ -84,6 +84,9 @@ Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide. - Since Spark 3.0, `createDataFrame(..., verifySchema=True)` validates `LongType` as well in PySpark. Previously, `LongType` was not verified and resulted in `None` in case the value overflows. To restore this behavior, `verifySchema` can be set to `False` to disable the validation. + - Since Spark 3.0, `Column.getItem` is fixed such that it does not call `Column.apply`. Consequently, if `Column` is used as an argument to `getItem`, the indexing operator should be used. + For example, `map_col.getItem(col('id'))` should be replaced with `map_col[col('id')]`. + ## Upgrading from PySpark 2.3 to 2.4 - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 7f12d2324e715..b472a4221cd0c 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -296,14 +296,12 @@ def getItem(self, key): +----+------+ | 1| value| +----+------+ - >>> df.select(df.l[0], df.d["key"]).show() - +----+------+ - |l[0]|d[key]| - +----+------+ - | 1| value| - +----+------+ + + .. versionchanged:: 3.0 + If `key` is a `Column` object, the indexing operator should be used instead. + For example, `map_col.getItem(col('id'))` should be replaced with `map_col[col('id')]`. """ - return self[key] + return _bin_op("getItem")(self, key) @since(1.3) def getField(self, name): diff --git a/python/pyspark/sql/tests/test_column.py b/python/pyspark/sql/tests/test_column.py index f7f2164dcd350..d9d933110dab5 100644 --- a/python/pyspark/sql/tests/test_column.py +++ b/python/pyspark/sql/tests/test_column.py @@ -18,6 +18,8 @@ import sys +from py4j.protocol import Py4JJavaError + from pyspark.sql import Column, Row from pyspark.sql.types import * from pyspark.sql.utils import AnalysisException @@ -85,7 +87,7 @@ def test_column_operators(self): "Cannot apply 'in' operator against a column", lambda: 1 in cs) - def test_column_getitem(self): + def test_column_apply(self): from pyspark.sql.functions import col self.assertIsInstance(col("foo")[1:3], Column) @@ -93,6 +95,16 @@ def test_column_getitem(self): self.assertIsInstance(col("foo")["bar"], Column) self.assertRaises(ValueError, lambda: col("foo")[0:10:2]) + def test_column_getitem(self): + from pyspark.sql.functions import col, create_map, lit + + map_col = create_map(lit(0), lit(100), lit(1), lit(200)) + self.assertRaisesRegexp( + Py4JJavaError, + "Unsupported literal type class org.apache.spark.sql.Column id", + lambda: map_col.getItem(col('id')) + ) + def test_column_select(self): df = self.df self.assertEqual(self.testData, df.select("*").collect()) From ae7450d1c977bcfde1d67745440079abbeb19c67 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 31 Oct 2019 20:28:31 -0700 Subject: [PATCH 0488/1387] [SPARK-29676][SQL] ALTER TABLE (RENAME PARTITION) should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add AlterTableRenamePartitionStatement and make ALTER TABLE ... RENAME TO PARTITION go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog ALTER TABLE t PARTITION (id=1) RENAME TO PARTITION (id=2) // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? Yes. When running ALTER TABLE ... RENAME TO PARTITION, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26350 from huaxingao/spark_29676. Authored-by: Huaxin Gao Signed-off-by: Liang-Chi Hsieh --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 16 ++++++++++++ .../catalyst/plans/logical/statements.scala | 8 ++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 26 +++++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 9 ++++++- .../spark/sql/execution/SparkSqlParser.scala | 16 ------------ .../sql/connector/DataSourceV2SQLSuite.scala | 17 +++++++++--- .../execution/command/DDLParserSuite.scala | 14 ---------- 8 files changed, 73 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 11b5d4ae5ebfe..70facc7282aff 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -162,7 +162,7 @@ statement partitionSpecLocation+ #addTablePartition | ALTER VIEW tableIdentifier ADD (IF NOT EXISTS)? partitionSpec+ #addTablePartition - | ALTER TABLE tableIdentifier + | ALTER TABLE multipartIdentifier from=partitionSpec RENAME TO to=partitionSpec #renameTablePartition | ALTER TABLE tableIdentifier DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 911a5b3aa36ad..93560a2cff70c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2951,4 +2951,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx: RecoverPartitionsContext): LogicalPlan = withOrigin(ctx) { AlterTableRecoverPartitionsStatement(visitMultipartIdentifier(ctx.multipartIdentifier)) } + + /** + * Create an [[AlterTableRenamePartitionStatement]] + * + * For example: + * {{{ + * ALTER TABLE multi_part_name PARTITION spec1 RENAME TO PARTITION spec2; + * }}} + */ + override def visitRenameTablePartition( + ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) { + AlterTableRenamePartitionStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + visitNonOptionalPartitionSpec(ctx.from), + visitNonOptionalPartitionSpec(ctx.to)) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 41c7438eaa9ed..c40b319366684 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -188,6 +188,14 @@ case class AlterTableSetLocationStatement( case class AlterTableRecoverPartitionsStatement( tableName: Seq[String]) extends ParsedStatement +/** + * ALTER TABLE ... RENAME PARTITION command, as parsed from SQL. + */ +case class AlterTableRenamePartitionStatement( + tableName: Seq[String], + from: TablePartitionSpec, + to: TablePartitionSpec) extends ParsedStatement + /** * ALTER VIEW ... SET TBLPROPERTIES command, as parsed from SQL. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 53da49ead4e48..2f5faf76e7b7b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1202,6 +1202,32 @@ class DDLParserSuite extends AnalysisTest { AlterTableRecoverPartitionsStatement(Seq("a", "b", "c"))) } + test("alter table: rename partition") { + val sql1 = + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |RENAME TO PARTITION (dt='2008-09-09', country='uk') + """.stripMargin + val parsed1 = parsePlan(sql1) + val expected1 = AlterTableRenamePartitionStatement( + Seq("table_name"), + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2008-09-09", "country" -> "uk")) + comparePlans(parsed1, expected1) + + val sql2 = + """ + |ALTER TABLE a.b.c PARTITION (ds='2017-06-10') + |RENAME TO PARTITION (ds='2018-06-10') + """.stripMargin + val parsed2 = parsePlan(sql2) + val expected2 = AlterTableRenamePartitionStatement( + Seq("a", "b", "c"), + Map("ds" -> "2017-06-10"), + Map("ds" -> "2018-06-10")) + comparePlans(parsed2, expected2) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 7b34f4f5ef696..e49e149a4a909 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropDatabaseCommand, DropTableCommand, LoadDataCommand, ShowColumnsCommand, ShowCreateTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} +import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -369,6 +369,13 @@ class ResolveSessionCatalog( AlterTableRecoverPartitionsCommand( v1TableName.asTableIdentifier, "ALTER TABLE RECOVER PARTITIONS") + + case AlterTableRenamePartitionStatement(tableName, from, to) => + val v1TableName = parseV1Table(tableName, "ALTER TABLE RENAME PARTITION") + AlterTableRenamePartitionCommand( + v1TableName.asTableIdentifier, + from, + to) } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index b53fc9d042dbc..ad75773adc5de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -461,22 +461,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ctx.EXISTS != null) } - /** - * Create an [[AlterTableRenamePartitionCommand]] command - * - * For example: - * {{{ - * ALTER TABLE table PARTITION spec1 RENAME TO PARTITION spec2; - * }}} - */ - override def visitRenameTablePartition( - ctx: RenameTablePartitionContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenamePartitionCommand( - visitTableIdentifier(ctx.tableIdentifier), - visitNonOptionalPartitionSpec(ctx.from), - visitNonOptionalPartitionSpec(ctx.to)) - } - /** * Create an [[AlterTableDropPartitionCommand]] command * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 6e20b248ebc10..b98120a0a1979 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1374,16 +1374,27 @@ class DataSourceV2SQLSuite } } - test("ALTER TABLE RECOVER PARTITIONS") { + test("ALTER TABLE RECOVER PARTITIONS") { val t = "testcat.ns1.ns2.tbl" withTable(t) { spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo") val e = intercept[AnalysisException] { - val partition = sql(s"ALTER TABLE $t RECOVER PARTITIONS") + sql(s"ALTER TABLE $t RECOVER PARTITIONS") } assert(e.message.contains("ALTER TABLE RECOVER PARTITIONS is only supported with v1 tables")) } - } + } + + test("ALTER TABLE RENAME PARTITION") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo PARTITIONED BY (id)") + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE $t PARTITION (id=1) RENAME TO PARTITION (id=2)") + } + assert(e.message.contains("ALTER TABLE RENAME PARTITION is only supported with v1 tables")) + } + } private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 07935641efdf5..71fce8fe8d167 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -558,20 +558,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { """.stripMargin) } - test("alter table: rename partition") { - val sql = - """ - |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') - |RENAME TO PARTITION (dt='2008-09-09', country='uk') - """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = AlterTableRenamePartitionCommand( - TableIdentifier("table_name", None), - Map("dt" -> "2008-08-08", "country" -> "us"), - Map("dt" -> "2008-09-09", "country" -> "uk")) - comparePlans(parsed, expected) - } - test("alter table: exchange partition (not supported)") { assertUnsupported( """ From 8a4378c6f0ca49f94fdeba08ca095721427e8635 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 1 Nov 2019 12:07:07 +0800 Subject: [PATCH 0489/1387] [SPARK-29686][ML] LinearSVC should persist instances if needed ### What changes were proposed in this pull request? persist the input if needed ### Why are the changes needed? training with non-cached dataset will hurt performance ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing tests Closes #26344 from zhengruifeng/linear_svc_cache. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../org/apache/spark/ml/classification/LinearSVC.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index e467228b4cc14..27cf2988aae89 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -37,6 +37,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.storage.StorageLevel /** Params for linear SVM Classifier. */ private[classification] trait LinearSVCParams extends ClassifierParams with HasRegParam @@ -159,7 +160,10 @@ class LinearSVC @Since("2.2.0") ( override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) override protected def train(dataset: Dataset[_]): LinearSVCModel = instrumented { instr => + val handlePersistence = dataset.storageLevel == StorageLevel.NONE + val instances = extractInstances(dataset) + if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) instr.logPipelineStage(this) instr.logDataset(dataset) @@ -268,6 +272,8 @@ class LinearSVC @Since("2.2.0") ( (Vectors.dense(coefficientArray), intercept, scaledObjectiveHistory.result()) } + if (handlePersistence) instances.unpersist() + copyValues(new LinearSVCModel(uid, coefficientVector, interceptVector)) } } From a4382f7fe1c36a51c64f460c6cb91e93470e0825 Mon Sep 17 00:00:00 2001 From: "Liu,Linhong" Date: Fri, 1 Nov 2019 18:12:33 +0800 Subject: [PATCH 0490/1387] [SPARK-29486][SQL] CalendarInterval should have 3 fields: months, days and microseconds ### What changes were proposed in this pull request? Current CalendarInterval has 2 fields: months and microseconds. This PR try to change it to 3 fields: months, days and microseconds. This is because one logical day interval may have different number of microseconds (daylight saving). ### Why are the changes needed? One logical day interval may have different number of microseconds (daylight saving). For example, in PST timezone, there will be 25 hours from 2019-11-2 12:00:00 to 2019-11-3 12:00:00 ### Does this PR introduce any user-facing change? no ### How was this patch tested? unit test and new added test cases Closes #26134 from LinhongLiu/calendarinterval. Authored-by: Liu,Linhong Signed-off-by: Wenchen Fan --- .../spark/unsafe/types/CalendarInterval.java | 39 ++++++----- .../unsafe/types/CalendarIntervalSuite.java | 65 +++++++++++-------- .../catalyst/expressions/UnsafeArrayData.java | 5 +- .../sql/catalyst/expressions/UnsafeRow.java | 5 +- .../expressions/codegen/UnsafeWriter.java | 5 +- .../spark/sql/vectorized/ColumnVector.java | 21 +++--- .../analysis/StreamingJoinHelper.scala | 3 +- .../sql/catalyst/expressions/TimeWindow.scala | 3 +- .../expressions/collectionOperations.scala | 37 +++++++---- .../expressions/datetimeExpressions.scala | 14 ++-- .../spark/sql/catalyst/expressions/hash.scala | 8 ++- .../expressions/intervalExpressions.scala | 4 +- .../sql/catalyst/expressions/literals.scala | 2 +- .../sql/catalyst/util/DateTimeUtils.scala | 6 +- .../sql/catalyst/util/IntervalUtils.scala | 30 +++++---- .../spark/sql/RandomDataGenerator.scala | 3 +- .../catalyst/analysis/TypeCoercionSuite.scala | 2 +- .../sql/catalyst/expressions/CastSuite.scala | 12 ++-- .../expressions/DateExpressionsSuite.scala | 42 ++++++++---- .../ExpressionSQLBuilderSuite.scala | 6 +- .../IntervalExpressionsSuite.scala | 32 ++++----- .../expressions/LiteralExpressionSuite.scala | 2 +- .../expressions/LiteralGenerator.scala | 4 +- .../optimizer/FilterPushdownSuite.scala | 8 +-- .../LeftSemiAntiJoinPushDownSuite.scala | 1 - .../parser/ExpressionParserSuite.scala | 9 +-- .../catalyst/util/DateTimeUtilsSuite.scala | 34 ++++++++-- .../catalyst/util/IntervalUtilsSuite.scala | 42 ++++++------ .../sql/catalyst/util/UnsafeArraySuite.scala | 3 +- .../vectorized/ColumnVectorUtils.java | 3 +- .../vectorized/WritableColumnVector.java | 7 +- .../apache/spark/sql/execution/Columnar.scala | 3 +- .../sql/execution/streaming/Triggers.scala | 3 +- .../sql-tests/results/ansi/interval.sql.out | 4 +- .../sql-tests/results/date_part.sql.out | 16 ++--- .../sql-tests/results/datetime.sql.out | 4 +- .../sql-tests/results/literals.sql.out | 4 +- .../results/postgreSQL/interval.sql.out | 4 +- .../results/postgreSQL/timestamp.sql.out | 12 ++-- .../apache/spark/sql/DateFunctionsSuite.scala | 16 ++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 14 ++-- .../vectorized/ColumnarBatchSuite.scala | 28 +++++--- 42 files changed, 337 insertions(+), 228 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 3c835519efd16..2a840c11304d4 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; +import java.util.Objects; /** * The internal representation of interval type. @@ -31,45 +32,50 @@ public final class CalendarInterval implements Serializable { public static final long MICROS_PER_WEEK = MICROS_PER_DAY * 7; public final int months; + public final int days; public final long microseconds; public long milliseconds() { return this.microseconds / MICROS_PER_MILLI; } - public CalendarInterval(int months, long microseconds) { + public CalendarInterval(int months, int days, long microseconds) { this.months = months; + this.days = days; this.microseconds = microseconds; } public CalendarInterval add(CalendarInterval that) { int months = this.months + that.months; + int days = this.days + that.days; long microseconds = this.microseconds + that.microseconds; - return new CalendarInterval(months, microseconds); + return new CalendarInterval(months, days, microseconds); } public CalendarInterval subtract(CalendarInterval that) { int months = this.months - that.months; + int days = this.days - that.days; long microseconds = this.microseconds - that.microseconds; - return new CalendarInterval(months, microseconds); + return new CalendarInterval(months, days, microseconds); } public CalendarInterval negate() { - return new CalendarInterval(-this.months, -this.microseconds); + return new CalendarInterval(-this.months, -this.days, -this.microseconds); } @Override - public boolean equals(Object other) { - if (this == other) return true; - if (other == null || !(other instanceof CalendarInterval)) return false; - - CalendarInterval o = (CalendarInterval) other; - return this.months == o.months && this.microseconds == o.microseconds; + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + CalendarInterval that = (CalendarInterval) o; + return months == that.months && + days == that.days && + microseconds == that.microseconds; } @Override public int hashCode() { - return 31 * months + (int) microseconds; + return Objects.hash(months, days, microseconds); } @Override @@ -81,12 +87,13 @@ public String toString() { appendUnit(sb, months % 12, "month"); } + if (days != 0) { + appendUnit(sb, days / 7, "week"); + appendUnit(sb, days % 7, "day"); + } + if (microseconds != 0) { long rest = microseconds; - appendUnit(sb, rest / MICROS_PER_WEEK, "week"); - rest %= MICROS_PER_WEEK; - appendUnit(sb, rest / MICROS_PER_DAY, "day"); - rest %= MICROS_PER_DAY; appendUnit(sb, rest / MICROS_PER_HOUR, "hour"); rest %= MICROS_PER_HOUR; appendUnit(sb, rest / MICROS_PER_MINUTE, "minute"); @@ -96,7 +103,7 @@ public String toString() { appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond"); rest %= MICROS_PER_MILLI; appendUnit(sb, rest, "microsecond"); - } else if (months == 0) { + } else if (months == 0 && days == 0) { sb.append(" 0 microseconds"); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 5e418c1c14a60..c3ed57c79d563 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -26,59 +26,72 @@ public class CalendarIntervalSuite { @Test public void equalsTest() { - CalendarInterval i1 = new CalendarInterval(3, 123); - CalendarInterval i2 = new CalendarInterval(3, 321); - CalendarInterval i3 = new CalendarInterval(1, 123); - CalendarInterval i4 = new CalendarInterval(3, 123); + CalendarInterval i1 = new CalendarInterval(3, 2, 123); + CalendarInterval i2 = new CalendarInterval(3, 2,321); + CalendarInterval i3 = new CalendarInterval(3, 4,123); + CalendarInterval i4 = new CalendarInterval(1, 2, 123); + CalendarInterval i5 = new CalendarInterval(1, 4, 321); + CalendarInterval i6 = new CalendarInterval(3, 2, 123); assertNotSame(i1, i2); assertNotSame(i1, i3); + assertNotSame(i1, i4); assertNotSame(i2, i3); - assertEquals(i1, i4); + assertNotSame(i2, i4); + assertNotSame(i3, i4); + assertNotSame(i1, i5); + assertEquals(i1, i6); } @Test public void toStringTest() { CalendarInterval i; - i = new CalendarInterval(0, 0); + i = new CalendarInterval(0, 0, 0); assertEquals("interval 0 microseconds", i.toString()); - i = new CalendarInterval(34, 0); + i = new CalendarInterval(34, 0, 0); assertEquals("interval 2 years 10 months", i.toString()); - i = new CalendarInterval(-34, 0); + i = new CalendarInterval(-34, 0, 0); assertEquals("interval -2 years -10 months", i.toString()); - i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 3 weeks 13 hours 123 microseconds", i.toString()); + i = new CalendarInterval(0, 31, 0); + assertEquals("interval 4 weeks 3 days", i.toString()); - i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); - assertEquals("interval -3 weeks -13 hours -123 microseconds", i.toString()); + i = new CalendarInterval(0, -31, 0); + assertEquals("interval -4 weeks -3 days", i.toString()); - i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals("interval 2 years 10 months 3 weeks 13 hours 123 microseconds", i.toString()); + i = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); + assertEquals("interval 3 hours 13 minutes 123 microseconds", i.toString()); + + i = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123); + assertEquals("interval -3 hours -13 minutes -123 microseconds", i.toString()); + + i = new CalendarInterval(34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); + assertEquals("interval 2 years 10 months 4 weeks 3 days 3 hours 13 minutes 123 microseconds", + i.toString()); } @Test public void addTest() { - CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); - CalendarInterval input2 = new CalendarInterval(2, 100 * MICROS_PER_HOUR); - assertEquals(input1.add(input2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); + CalendarInterval input1 = new CalendarInterval(3, 1, 1 * MICROS_PER_HOUR); + CalendarInterval input2 = new CalendarInterval(2, 4, 100 * MICROS_PER_HOUR); + assertEquals(input1.add(input2), new CalendarInterval(5, 5, 101 * MICROS_PER_HOUR)); - input1 = new CalendarInterval(-10, -81 * MICROS_PER_HOUR); - input2 = new CalendarInterval(75, 200 * MICROS_PER_HOUR); - assertEquals(input1.add(input2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + input1 = new CalendarInterval(-10, -30, -81 * MICROS_PER_HOUR); + input2 = new CalendarInterval(75, 150, 200 * MICROS_PER_HOUR); + assertEquals(input1.add(input2), new CalendarInterval(65, 120, 119 * MICROS_PER_HOUR)); } @Test public void subtractTest() { - CalendarInterval input1 = new CalendarInterval(3, 1 * MICROS_PER_HOUR); - CalendarInterval input2 = new CalendarInterval(2, 100 * MICROS_PER_HOUR); - assertEquals(input1.subtract(input2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); + CalendarInterval input1 = new CalendarInterval(3, 1, 1 * MICROS_PER_HOUR); + CalendarInterval input2 = new CalendarInterval(2, 4, 100 * MICROS_PER_HOUR); + assertEquals(input1.subtract(input2), new CalendarInterval(1, -3, -99 * MICROS_PER_HOUR)); - input1 = new CalendarInterval(-10, -81 * MICROS_PER_HOUR); - input2 = new CalendarInterval(75, 200 * MICROS_PER_HOUR); - assertEquals(input1.subtract(input2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); + input1 = new CalendarInterval(-10, -30, -81 * MICROS_PER_HOUR); + input2 = new CalendarInterval(75, 150, 200 * MICROS_PER_HOUR); + assertEquals(input1.subtract(input2), new CalendarInterval(-85, -180, -281 * MICROS_PER_HOUR)); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index db6401b18c0e4..9e686985b0607 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -230,9 +230,10 @@ public CalendarInterval getInterval(int ordinal) { if (isNullAt(ordinal)) return null; final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); - final int months = (int) Platform.getLong(baseObject, baseOffset + offset); + final int months = Platform.getInt(baseObject, baseOffset + offset); + final int days = Platform.getInt(baseObject, baseOffset + offset + 4); final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); - return new CalendarInterval(months, microseconds); + return new CalendarInterval(months, days, microseconds); } @Override diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index 8fd6029e976ee..dc6fa30e14f4d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -401,9 +401,10 @@ public CalendarInterval getInterval(int ordinal) { } else { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); - final int months = (int) Platform.getLong(baseObject, baseOffset + offset); + final int months = Platform.getInt(baseObject, baseOffset + offset); + final int days = Platform.getInt(baseObject, baseOffset + offset + 4); final long microseconds = Platform.getLong(baseObject, baseOffset + offset + 8); - return new CalendarInterval(months, microseconds); + return new CalendarInterval(months, days, microseconds); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java index 95263a0da95a8..4e4392734184a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java @@ -134,8 +134,9 @@ public final void write(int ordinal, CalendarInterval input) { // grow the global buffer before writing data. grow(16); - // Write the months and microseconds fields of Interval to the variable length portion. - Platform.putLong(getBuffer(), cursor(), input.months); + // Write the months, days and microseconds fields of Interval to the variable length portion. + Platform.putInt(getBuffer(), cursor(), input.months); + Platform.putInt(getBuffer(), cursor() + 4, input.days); Platform.putLong(getBuffer(), cursor() + 8, input.microseconds); setOffsetAndSize(ordinal, 16); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java index f18d00359c90c..2158ef8324845 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java @@ -267,21 +267,24 @@ public final ColumnarRow getStruct(int rowId) { * Returns the calendar interval type value for rowId. If the slot for rowId is null, it should * return null. * - * In Spark, calendar interval type value is basically an integer value representing the number of - * months in this interval, and a long value representing the number of microseconds in this - * interval. An interval type vector is the same as a struct type vector with 2 fields: `months` - * and `microseconds`. + * In Spark, calendar interval type value is basically two integer values representing the number + * of months and days in this interval, and a long value representing the number of microseconds + * in this interval. An interval type vector is the same as a struct type vector with 3 fields: + * `months`, `days` and `microseconds`. * - * To support interval type, implementations must implement {@link #getChild(int)} and define 2 + * To support interval type, implementations must implement {@link #getChild(int)} and define 3 * child vectors: the first child vector is an int type vector, containing all the month values of - * all the interval values in this vector. The second child vector is a long type vector, - * containing all the microsecond values of all the interval values in this vector. + * all the interval values in this vector. The second child vector is an int type vector, + * containing all the day values of all the interval values in this vector. The third child vector + * is a long type vector, containing all the microsecond values of all the interval values in this + * vector. */ public final CalendarInterval getInterval(int rowId) { if (isNullAt(rowId)) return null; final int months = getChild(0).getInt(rowId); - final long microseconds = getChild(1).getLong(rowId); - return new CalendarInterval(months, microseconds); + final int days = getChild(1).getInt(rowId); + final long microseconds = getChild(2).getLong(rowId); + return new CalendarInterval(months, days, microseconds); } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala index c64aeff3c238c..7e71fd8960249 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala @@ -264,7 +264,8 @@ object StreamingJoinHelper extends PredicateHelper with Logging { s"watermark calculation. Use interval in terms of day instead.") Literal(0.0) } else { - Literal(calendarInterval.microseconds.toDouble) + Literal(calendarInterval.days * CalendarInterval.MICROS_PER_DAY.toDouble + + calendarInterval.microseconds.toDouble) } case DoubleType => Multiply(lit, Literal(1000000.0)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index d3aadad120520..e84c9da5702ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGe import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval case class TimeWindow( timeColumn: Expression, @@ -107,7 +108,7 @@ object TimeWindow { throw new IllegalArgumentException( s"Intervals greater than a month is not supported ($interval).") } - cal.microseconds + cal.days * CalendarInterval.MICROS_PER_DAY + cal.microseconds } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 57b692ac16068..a348f7cd6b699 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -2610,25 +2610,33 @@ object Sequence { override val defaultStep: DefaultStep = new DefaultStep( (dt.ordering.lteq _).asInstanceOf[LessThanOrEqualFn], CalendarIntervalType, - new CalendarInterval(0, MICROS_PER_DAY)) + new CalendarInterval(0, 1, 0)) private val backedSequenceImpl = new IntegralSequenceImpl[T](dt) - private val microsPerMonth = 28 * CalendarInterval.MICROS_PER_DAY + private val microsPerDay = 24 * CalendarInterval.MICROS_PER_HOUR + // We choose a minimum days(28) in one month to calculate the `intervalStepInMicros` + // in order to make sure the estimated array length is long enough + private val microsPerMonth = 28 * microsPerDay override def eval(input1: Any, input2: Any, input3: Any): Array[T] = { val start = input1.asInstanceOf[T] val stop = input2.asInstanceOf[T] val step = input3.asInstanceOf[CalendarInterval] val stepMonths = step.months + val stepDays = step.days val stepMicros = step.microseconds - if (stepMonths == 0) { - backedSequenceImpl.eval(start, stop, fromLong(stepMicros / scale)) + if (stepMonths == 0 && stepMicros == 0 && scale == MICROS_PER_DAY) { + backedSequenceImpl.eval(start, stop, fromLong(stepDays)) + + } else if (stepMonths == 0 && stepDays == 0 && scale == 1) { + backedSequenceImpl.eval(start, stop, fromLong(stepMicros)) } else { // To estimate the resulted array length we need to make assumptions - // about a month length in microseconds - val intervalStepInMicros = stepMicros + stepMonths * microsPerMonth + // about a month length in days and a day length in microseconds + val intervalStepInMicros = + stepMicros + stepMonths * microsPerMonth + stepDays * microsPerDay val startMicros: Long = num.toLong(start) * scale val stopMicros: Long = num.toLong(stop) * scale val maxEstimatedArrayLength = @@ -2643,7 +2651,8 @@ object Sequence { while (t < exclusiveItem ^ stepSign < 0) { arr(i) = fromLong(t / scale) i += 1 - t = timestampAddInterval(startMicros, i * stepMonths, i * stepMicros, zoneId) + t = timestampAddInterval( + startMicros, i * stepMonths, i * stepDays, i * stepMicros, zoneId) } // truncate array to the correct length @@ -2659,6 +2668,7 @@ object Sequence { arr: String, elemType: String): String = { val stepMonths = ctx.freshName("stepMonths") + val stepDays = ctx.freshName("stepDays") val stepMicros = ctx.freshName("stepMicros") val stepScaled = ctx.freshName("stepScaled") val intervalInMicros = ctx.freshName("intervalInMicros") @@ -2673,18 +2683,21 @@ object Sequence { val sequenceLengthCode = s""" - |final long $intervalInMicros = $stepMicros + $stepMonths * ${microsPerMonth}L; + |final long $intervalInMicros = + | $stepMicros + $stepMonths * ${microsPerMonth}L + $stepDays * ${microsPerDay}L; |${genSequenceLengthCode(ctx, startMicros, stopMicros, intervalInMicros, arrLength)} """.stripMargin s""" |final int $stepMonths = $step.months; + |final int $stepDays = $step.days; |final long $stepMicros = $step.microseconds; | - |if ($stepMonths == 0) { - | final $elemType $stepScaled = ($elemType) ($stepMicros / ${scale}L); - | ${backedSequenceImpl.genCode(ctx, start, stop, stepScaled, arr, elemType)}; + |if ($stepMonths == 0 && $stepMicros == 0 && ${scale}L == ${MICROS_PER_DAY}L) { + | ${backedSequenceImpl.genCode(ctx, start, stop, stepDays, arr, elemType)}; | + |} else if ($stepMonths == 0 && $stepDays == 0 && ${scale}L == 1) { + | ${backedSequenceImpl.genCode(ctx, start, stop, stepMicros, arr, elemType)}; |} else { | final long $startMicros = $start * ${scale}L; | final long $stopMicros = $stop * ${scale}L; @@ -2702,7 +2715,7 @@ object Sequence { | $arr[$i] = ($elemType) ($t / ${scale}L); | $i += 1; | $t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval( - | $startMicros, $i * $stepMonths, $i * $stepMicros, $zid); + | $startMicros, $i * $stepMonths, $i * $stepDays, $i * $stepMicros, $zid); | } | | if ($arr.length > $i) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 90eb3598b233b..663d1e672905b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1090,14 +1090,14 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], itvl.months, itvl.microseconds, zoneId) + start.asInstanceOf[Long], itvl.months, itvl.days, itvl.microseconds, zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $zid)""" + s"""$dtu.timestampAddInterval($sd, $i.months, $i.days, $i.microseconds, $zid)""" }) } } @@ -1205,14 +1205,14 @@ case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[S override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, zoneId) + start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.days, 0 - itvl.microseconds, zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $zid)""" + s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.days, 0 - $i.microseconds, $zid)""" }) } } @@ -2121,7 +2121,7 @@ case class DatePart(field: Expression, source: Expression, child: Expression) } /** - * Returns the interval from startTimestamp to endTimestamp in which the `months` field + * Returns the interval from startTimestamp to endTimestamp in which the `months` and `day` field * is set to 0 and the `microseconds` field is initialized to the microsecond difference * between the given timestamps. */ @@ -2134,12 +2134,12 @@ case class SubtractTimestamps(endTimestamp: Expression, startTimestamp: Expressi override def dataType: DataType = CalendarIntervalType override def nullSafeEval(end: Any, start: Any): Any = { - new CalendarInterval(0, end.asInstanceOf[Long] - start.asInstanceOf[Long]) + new CalendarInterval(0, 0, end.asInstanceOf[Long] - start.asInstanceOf[Long]) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { defineCodeGen(ctx, ev, (end, start) => - s"new org.apache.spark.unsafe.types.CalendarInterval(0, $end - $start)") + s"new org.apache.spark.unsafe.types.CalendarInterval(0, 0, $end - $start)") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 2aa1e6cc518cb..84beb19782692 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -495,7 +495,7 @@ abstract class InterpretedHashFunction { val bytes = d.toJavaBigDecimal.unscaledValue().toByteArray hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET, bytes.length, seed) } - case c: CalendarInterval => hashInt(c.months, hashLong(c.microseconds, seed)) + case c: CalendarInterval => hashInt(c.months, hashInt(c.days, hashLong(c.microseconds, seed))) case a: Array[Byte] => hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) case s: UTF8String => @@ -902,11 +902,13 @@ object HiveHashFunction extends InterpretedHashFunction { * with nanosecond values will lead to wrong output hashes (ie. non adherent with Hive output) */ def hashCalendarInterval(calendarInterval: CalendarInterval): Long = { - val totalSeconds = calendarInterval.microseconds / CalendarInterval.MICROS_PER_SECOND.toInt + val totalMicroSeconds = + calendarInterval.days * CalendarInterval.MICROS_PER_DAY + calendarInterval.microseconds + val totalSeconds = totalMicroSeconds / CalendarInterval.MICROS_PER_SECOND.toInt val result: Int = (17 * 37) + (totalSeconds ^ totalSeconds >> 32).toInt val nanoSeconds = - (calendarInterval.microseconds - + (totalMicroSeconds - (totalSeconds * CalendarInterval.MICROS_PER_SECOND.toInt)).toInt * 1000 (result * 37) + nanoSeconds } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 08360c75a474b..c3a3b3cb58f4f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -63,10 +63,10 @@ case class ExtractIntervalMonths(child: Expression) extends ExtractIntervalPart(child, ByteType, getMonths, "getMonths") case class ExtractIntervalDays(child: Expression) - extends ExtractIntervalPart(child, LongType, getDays, "getDays") + extends ExtractIntervalPart(child, IntegerType, getDays, "getDays") case class ExtractIntervalHours(child: Expression) - extends ExtractIntervalPart(child, ByteType, getHours, "getHours") + extends ExtractIntervalPart(child, LongType, getHours, "getHours") case class ExtractIntervalMinutes(child: Expression) extends ExtractIntervalPart(child, ByteType, getMinutes, "getMinutes") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index f3c129934314c..5a5d7a17acd99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -162,7 +162,7 @@ object Literal { case TimestampType => create(0L, TimestampType) case StringType => Literal("") case BinaryType => Literal("".getBytes(StandardCharsets.UTF_8)) - case CalendarIntervalType => Literal(new CalendarInterval(0, 0)) + case CalendarIntervalType => Literal(new CalendarInterval(0, 0, 0)) case arr: ArrayType => create(Array(), arr) case map: MapType => create(Map(), map) case struct: StructType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 8dd541b1e4381..649fca38a682b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -575,11 +575,13 @@ object DateTimeUtils { def timestampAddInterval( start: SQLTimestamp, months: Int, + days: Int, microseconds: Long, zoneId: ZoneId): SQLTimestamp = { val resultTimestamp = microsToInstant(start) .atZone(zoneId) .plusMonths(months) + .plusDays(days) .plus(microseconds, ChronoUnit.MICROS) instantToMicros(resultTimestamp.toInstant) } @@ -963,7 +965,7 @@ object DateTimeUtils { LocalDate.ofEpochDay(startDate), LocalDate.ofEpochDay(endDate)) val months = period.getMonths + 12 * period.getYears - val microseconds = period.getDays * MICROS_PER_DAY - new CalendarInterval(months, microseconds) + val days = period.getDays + new CalendarInterval(months, days, 0) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 90e2402a5d7da..73e9f37c94528 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -64,12 +64,12 @@ object IntervalUtils { (getMonths(interval) / MONTHS_PER_QUARTER + 1).toByte } - def getDays(interval: CalendarInterval): Long = { - interval.microseconds / DateTimeUtils.MICROS_PER_DAY + def getDays(interval: CalendarInterval): Int = { + interval.days } - def getHours(interval: CalendarInterval): Byte = { - ((interval.microseconds % DateTimeUtils.MICROS_PER_DAY) / MICROS_PER_HOUR).toByte + def getHours(interval: CalendarInterval): Long = { + interval.microseconds / MICROS_PER_HOUR } def getMinutes(interval: CalendarInterval): Byte = { @@ -91,6 +91,7 @@ object IntervalUtils { // Returns total number of seconds with microseconds fractional part in the given interval. def getEpoch(interval: CalendarInterval): Decimal = { var result = interval.microseconds + result += DateTimeUtils.MICROS_PER_DAY * interval.days result += MICROS_PER_YEAR * (interval.months / MONTHS_PER_YEAR) result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) Decimal(result, 18, 6) @@ -150,7 +151,7 @@ object IntervalUtils { val years = toLongWithRange("year", yearStr, 0, Integer.MAX_VALUE).toInt val months = toLongWithRange("month", monthStr, 0, 11).toInt val totalMonths = Math.addExact(Math.multiplyExact(years, 12), months) - new CalendarInterval(totalMonths, 0) + new CalendarInterval(totalMonths, 0, 0) } catch { case NonFatal(e) => throw new IllegalArgumentException( @@ -201,7 +202,7 @@ object IntervalUtils { val days = if (m.group(2) == null) { 0 } else { - toLongWithRange("day", m.group(3), 0, Integer.MAX_VALUE) + toLongWithRange("day", m.group(3), 0, Integer.MAX_VALUE).toInt } var hours: Long = 0L var minutes: Long = 0L @@ -234,11 +235,10 @@ object IntervalUtils { s"Cannot support (interval '$input' $from to $to) expression") } var micros = secondsFraction - micros = Math.addExact(micros, Math.multiplyExact(days, DateTimeUtils.MICROS_PER_DAY)) micros = Math.addExact(micros, Math.multiplyExact(hours, MICROS_PER_HOUR)) micros = Math.addExact(micros, Math.multiplyExact(minutes, MICROS_PER_MINUTE)) micros = Math.addExact(micros, Math.multiplyExact(seconds, DateTimeUtils.MICROS_PER_SECOND)) - new CalendarInterval(0, sign * micros) + new CalendarInterval(0, sign * days, sign * micros) } catch { case e: Exception => throw new IllegalArgumentException( @@ -249,6 +249,7 @@ object IntervalUtils { def fromUnitStrings(units: Array[String], values: Array[String]): CalendarInterval = { assert(units.length == values.length) var months: Int = 0 + var days: Int = 0 var microseconds: Long = 0 var i = 0 while (i < units.length) { @@ -259,11 +260,9 @@ object IntervalUtils { case "month" => months = Math.addExact(months, values(i).toInt) case "week" => - val weeksUs = Math.multiplyExact(values(i).toLong, 7 * DateTimeUtils.MICROS_PER_DAY) - microseconds = Math.addExact(microseconds, weeksUs) + days = Math.addExact(days, Math.multiplyExact(values(i).toInt, 7)) case "day" => - val daysUs = Math.multiplyExact(values(i).toLong, DateTimeUtils.MICROS_PER_DAY) - microseconds = Math.addExact(microseconds, daysUs) + days = Math.addExact(days, values(i).toInt) case "hour" => val hoursUs = Math.multiplyExact(values(i).toLong, MICROS_PER_HOUR) microseconds = Math.addExact(microseconds, hoursUs) @@ -284,7 +283,7 @@ object IntervalUtils { } i += 1 } - new CalendarInterval(months, microseconds) + new CalendarInterval(months, days, microseconds) } // Parses a string with nanoseconds, truncates the result and returns microseconds @@ -345,7 +344,10 @@ object IntervalUtils { val monthsDuration = Math.multiplyExact( daysPerMonth * DateTimeUtils.MICROS_PER_DAY, interval.months) - val result = Math.addExact(interval.microseconds, monthsDuration) + val daysDuration = Math.multiplyExact( + DateTimeUtils.MICROS_PER_DAY, + interval.days) + val result = Math.addExact(interval.microseconds, Math.addExact(daysDuration, monthsDuration)) targetUnit.convert(result, TimeUnit.MICROSECONDS) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala index d361e6248e2f5..8971e1b501ae6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala @@ -193,8 +193,9 @@ object RandomDataGenerator { Some(generator) case CalendarIntervalType => Some(() => { val months = rand.nextInt(1000) + val days = rand.nextInt(10000) val ns = rand.nextLong() - new CalendarInterval(months, ns) + new CalendarInterval(months, days, ns) }) case DecimalType.Fixed(precision, scale) => Some( () => BigDecimal.apply( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 4f9e4ec0201dd..75bb460e25753 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1405,7 +1405,7 @@ class TypeCoercionSuite extends AnalysisTest { val dateTimeOperations = TypeCoercion.DateTimeOperations val date = Literal(new java.sql.Date(0L)) val timestamp = Literal(new Timestamp(0L)) - val interval = Literal(new CalendarInterval(0, 0)) + val interval = Literal(new CalendarInterval(0, 0, 0)) val str = Literal("2015-01-01") val intValue = Literal(0, IntegerType) 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 fc7a0d3af4e28..abeeab6dc7cf4 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 @@ -664,16 +664,16 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { import org.apache.spark.unsafe.types.CalendarInterval checkEvaluation(Cast(Literal(""), CalendarIntervalType), null) - checkEvaluation(Cast(Literal("interval -3 month 7 hours"), CalendarIntervalType), - new CalendarInterval(-3, 7 * CalendarInterval.MICROS_PER_HOUR)) + checkEvaluation(Cast(Literal("interval -3 month 1 day 7 hours"), CalendarIntervalType), + new CalendarInterval(-3, 1, 7 * CalendarInterval.MICROS_PER_HOUR)) checkEvaluation(Cast(Literal.create( - new CalendarInterval(15, -3 * CalendarInterval.MICROS_PER_DAY), CalendarIntervalType), + new CalendarInterval(15, 9, -3 * CalendarInterval.MICROS_PER_HOUR), CalendarIntervalType), StringType), - "interval 1 years 3 months -3 days") + "interval 1 years 3 months 1 weeks 2 days -3 hours") checkEvaluation(Cast(Literal("INTERVAL 1 Second 1 microsecond"), CalendarIntervalType), - new CalendarInterval(0, 1000001)) + new CalendarInterval(0, 0, 1000001)) checkEvaluation(Cast(Literal("1 MONTH 1 Microsecond"), CalendarIntervalType), - new CalendarInterval(1, 1)) + new CalendarInterval(1, 0, 1)) } test("cast string to boolean") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 6abadd77bd41a..b27459fb08d74 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -377,15 +377,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( TimeAdd( Literal(new Timestamp(sdf.parse("2016-01-29 10:00:00.000").getTime)), - Literal(new CalendarInterval(1, 123000L)), + Literal(new CalendarInterval(1, 2, 123000L)), timeZoneId), DateTimeUtils.fromJavaTimestamp( - new Timestamp(sdf.parse("2016-02-29 10:00:00.123").getTime))) + new Timestamp(sdf.parse("2016-03-02 10:00:00.123").getTime))) checkEvaluation( TimeAdd( Literal.create(null, TimestampType), - Literal(new CalendarInterval(1, 123000L)), + Literal(new CalendarInterval(1, 2, 123000L)), timeZoneId), null) checkEvaluation( @@ -415,22 +415,36 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( TimeSub( Literal(new Timestamp(sdf.parse("2016-03-31 10:00:00.000").getTime)), - Literal(new CalendarInterval(1, 0)), + Literal(new CalendarInterval(1, 0, 0)), timeZoneId), DateTimeUtils.fromJavaTimestamp( new Timestamp(sdf.parse("2016-02-29 10:00:00.000").getTime))) + checkEvaluation( + TimeSub( + Literal(new Timestamp(sdf.parse("2016-03-31 10:00:00.000").getTime)), + Literal(new CalendarInterval(1, 1, 0)), + timeZoneId), + DateTimeUtils.fromJavaTimestamp( + new Timestamp(sdf.parse("2016-02-28 10:00:00.000").getTime))) checkEvaluation( TimeSub( Literal(new Timestamp(sdf.parse("2016-03-30 00:00:01.000").getTime)), - Literal(new CalendarInterval(1, 2000000.toLong)), + Literal(new CalendarInterval(1, 0, 2000000.toLong)), timeZoneId), DateTimeUtils.fromJavaTimestamp( new Timestamp(sdf.parse("2016-02-28 23:59:59.000").getTime))) + checkEvaluation( + TimeSub( + Literal(new Timestamp(sdf.parse("2016-03-30 00:00:01.000").getTime)), + Literal(new CalendarInterval(1, 1, 2000000.toLong)), + timeZoneId), + DateTimeUtils.fromJavaTimestamp( + new Timestamp(sdf.parse("2016-02-27 23:59:59.000").getTime))) checkEvaluation( TimeSub( Literal.create(null, TimestampType), - Literal(new CalendarInterval(1, 123000L)), + Literal(new CalendarInterval(1, 2, 123000L)), timeZoneId), null) checkEvaluation( @@ -1073,25 +1087,25 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("timestamps difference") { val end = Instant.parse("2019-10-04T11:04:01.123456Z") checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), - new CalendarInterval(0, 0)) + new CalendarInterval(0, 0, 0)) checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), - IntervalUtils.fromString("interval 18173 days " + - "11 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds")) + IntervalUtils.fromString("interval " + + "436163 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds")) checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), - IntervalUtils.fromString("interval -18173 days " + - "-11 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds")) + IntervalUtils.fromString("interval " + + "-436163 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds")) checkEvaluation( SubtractTimestamps( Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), Literal(Instant.parse("0001-01-01T00:00:00Z"))), - IntervalUtils.fromString("interval 521722 weeks 4 days " + - "23 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds")) + IntervalUtils.fromString("interval " + + "87649415 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds")) } test("subtract dates") { val end = LocalDate.of(2019, 10, 5) checkEvaluation(SubtractDates(Literal(end), Literal(end)), - new CalendarInterval(0, 0)) + new CalendarInterval(0, 0, 0)) checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), IntervalUtils.fromString("interval 1 days")) checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala index 442a23218c010..28fd4f6310be4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala @@ -165,16 +165,16 @@ class ExpressionSQLBuilderSuite extends SparkFunSuite { } test("interval arithmetic") { - val interval = Literal(new CalendarInterval(0, CalendarInterval.MICROS_PER_DAY)) + val interval = Literal(new CalendarInterval(0, 0, CalendarInterval.MICROS_PER_HOUR)) checkSQL( TimeAdd('a, interval), - "`a` + interval 1 days" + "`a` + interval 1 hours" ) checkSQL( TimeSub('a, interval), - "`a` - interval 1 days" + "`a` - interval 1 hours" ) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index 818ee239dbbf8..0c292e11485aa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -103,25 +103,27 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { "31 days 11 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds" test("days") { - checkEvaluation(ExtractIntervalDays("0 days"), 0L) - checkEvaluation(ExtractIntervalDays("1 days 100 seconds"), 1L) - checkEvaluation(ExtractIntervalDays("-1 days -100 seconds"), -1L) - checkEvaluation(ExtractIntervalDays("-365 days"), -365L) - checkEvaluation(ExtractIntervalDays("365 days"), 365L) + checkEvaluation(ExtractIntervalDays("0 days"), 0) + checkEvaluation(ExtractIntervalDays("1 days 100 seconds"), 1) + checkEvaluation(ExtractIntervalDays("-1 days -100 seconds"), -1) + checkEvaluation(ExtractIntervalDays("-365 days"), -365) + checkEvaluation(ExtractIntervalDays("365 days"), 365) // Years and months must not be taken into account - checkEvaluation(ExtractIntervalDays("100 year 10 months 5 days"), 5L) - checkEvaluation(ExtractIntervalDays(largeInterval), 31L) + checkEvaluation(ExtractIntervalDays("100 year 10 months 5 days"), 5) + checkEvaluation(ExtractIntervalDays(largeInterval), 31) } test("hours") { - checkEvaluation(ExtractIntervalHours("0 hours"), 0.toByte) - checkEvaluation(ExtractIntervalHours("1 hour"), 1.toByte) - checkEvaluation(ExtractIntervalHours("-1 hour"), -1.toByte) - checkEvaluation(ExtractIntervalHours("23 hours"), 23.toByte) - checkEvaluation(ExtractIntervalHours("-23 hours"), -23.toByte) - // Years and months must not be taken into account - checkEvaluation(ExtractIntervalHours("100 year 10 months 10 hours"), 10.toByte) - checkEvaluation(ExtractIntervalHours(largeInterval), 11.toByte) + checkEvaluation(ExtractIntervalHours("0 hours"), 0L) + checkEvaluation(ExtractIntervalHours("1 hour"), 1L) + checkEvaluation(ExtractIntervalHours("-1 hour"), -1L) + checkEvaluation(ExtractIntervalHours("23 hours"), 23L) + checkEvaluation(ExtractIntervalHours("-23 hours"), -23L) + // Years, months and days must not be taken into account + checkEvaluation(ExtractIntervalHours("100 year 10 months 10 days 10 hours"), 10L) + // Minutes should be taken into account + checkEvaluation(ExtractIntervalHours("10 hours 100 minutes"), 11L) + checkEvaluation(ExtractIntervalHours(largeInterval), 11L) } test("minutes") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index 269f1a09ac533..18385f964930b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -75,7 +75,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Literal.default(DateType), LocalDate.ofEpochDay(0)) checkEvaluation(Literal.default(TimestampType), Instant.ofEpochSecond(0)) } - checkEvaluation(Literal.default(CalendarIntervalType), new CalendarInterval(0, 0L)) + checkEvaluation(Literal.default(CalendarIntervalType), new CalendarInterval(0, 0, 0L)) checkEvaluation(Literal.default(ArrayType(StringType)), Array()) checkEvaluation(Literal.default(MapType(IntegerType, StringType)), Map()) checkEvaluation(Literal.default(StructType(StructField("a", StringType) :: Nil)), Row("")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala index b111797c3588e..82bc38ebe037f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala @@ -135,10 +135,12 @@ object LiteralGenerator { Instant.parse("0001-01-01T00:00:00.000000Z"), Instant.parse("9999-12-31T23:59:59.999999Z")).getSeconds val maxMicros = TimeUnit.SECONDS.toMicros(maxDurationInSec) + val maxDays = TimeUnit.SECONDS.toDays(maxDurationInSec).toInt for { months <- Gen.choose(-1 * maxIntervalInMonths, maxIntervalInMonths) micros <- Gen.choose(-1 * maxMicros, maxMicros) - } yield Literal.create(new CalendarInterval(months, micros), CalendarIntervalType) + days <- Gen.choose(-1 * maxDays, maxDays) + } yield Literal.create(new CalendarInterval(months, days, micros), CalendarIntervalType) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 3ec8d18bc871d..70e29dca46e9e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -1154,7 +1154,7 @@ class FilterPushdownSuite extends PlanTest { } test("watermark pushdown: no pushdown on watermark attribute #1") { - val interval = new CalendarInterval(2, 2000L) + val interval = new CalendarInterval(2, 2, 2000L) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. @@ -1169,7 +1169,7 @@ class FilterPushdownSuite extends PlanTest { } test("watermark pushdown: no pushdown for nondeterministic filter") { - val interval = new CalendarInterval(2, 2000L) + val interval = new CalendarInterval(2, 2, 2000L) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. @@ -1184,7 +1184,7 @@ class FilterPushdownSuite extends PlanTest { } test("watermark pushdown: full pushdown") { - val interval = new CalendarInterval(2, 2000L) + val interval = new CalendarInterval(2, 2, 2000L) // Verify that all conditions except the watermark touching condition are pushed down // by the optimizer and others are not. @@ -1198,7 +1198,7 @@ class FilterPushdownSuite extends PlanTest { } test("watermark pushdown: no pushdown on watermark attribute #2") { - val interval = new CalendarInterval(2, 2000L) + val interval = new CalendarInterval(2, 2, 2000L) val originalQuery = EventTimeWatermark('a, interval, testRelation) .where('a === 5 && 'b === 10) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala index f6d1898dc64a8..a3da9f73ebd40 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.types.IntegerType -import org.apache.spark.unsafe.types.CalendarInterval class LeftSemiPushdownSuite extends PlanTest { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 5423f7516c2e5..d675c7c483a04 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -631,11 +631,13 @@ class ExpressionParserSuite extends AnalysisTest { checkIntervals( "-13.123456789 second", Literal(new CalendarInterval( + 0, 0, -13 * DateTimeUtils.MICROS_PER_SECOND - 123 * DateTimeUtils.MICROS_PER_MILLIS - 456))) checkIntervals( "13.123456 second", Literal(new CalendarInterval( + 0, 0, 13 * DateTimeUtils.MICROS_PER_SECOND + 123 * DateTimeUtils.MICROS_PER_MILLIS + 456))) checkIntervals("1.001 second", Literal(IntervalUtils.fromString("1 second 1 millisecond"))) @@ -682,12 +684,11 @@ class ExpressionParserSuite extends AnalysisTest { // Composed intervals. checkIntervals( - "3 months 22 seconds 1 millisecond", - Literal(new CalendarInterval(3, 22001000L))) + "3 months 4 days 22 seconds 1 millisecond", + Literal(new CalendarInterval(3, 4, 22001000L))) checkIntervals( "3 years '-1-10' year to month 3 weeks '1 0:0:2' day to second", - Literal(new CalendarInterval(14, - 22 * CalendarInterval.MICROS_PER_DAY + 2 * CalendarInterval.MICROS_PER_SECOND))) + Literal(new CalendarInterval(14, 22, 2 * CalendarInterval.MICROS_PER_SECOND))) } test("SPARK-23264 Interval Compatibility tests") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 0eaf538231284..f9deb4d05eab6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class DateTimeUtilsSuite extends SparkFunSuite with Matchers { @@ -373,13 +373,39 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers { test("timestamp add months") { val ts1 = date(1997, 2, 28, 10, 30, 0) val ts2 = date(2000, 2, 28, 10, 30, 0, 123000) - assert(timestampAddInterval(ts1, 36, 123000, defaultZoneId) === ts2) + assert(timestampAddInterval(ts1, 36, 0, 123000, defaultZoneId) === ts2) val ts3 = date(1997, 2, 27, 16, 0, 0, 0, TimeZonePST) val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, TimeZonePST) val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, TimeZoneGMT) - assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST.toZoneId) === ts4) - assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT.toZoneId) === ts5) + assert(timestampAddInterval(ts3, 36, 0, 123000, TimeZonePST.toZoneId) === ts4) + assert(timestampAddInterval(ts3, 36, 0, 123000, TimeZoneGMT.toZoneId) === ts5) + } + + test("timestamp add days") { + // 2019-3-9 is the end of Pacific Standard Time + val ts1 = date(2019, 3, 9, 12, 0, 0, 123000, TimeZonePST) + // 2019-3-10 is the start of Pacific Daylight Time + val ts2 = date(2019, 3, 10, 12, 0, 0, 123000, TimeZonePST) + val ts3 = date(2019, 5, 9, 12, 0, 0, 123000, TimeZonePST) + val ts4 = date(2019, 5, 10, 12, 0, 0, 123000, TimeZonePST) + // 2019-11-2 is the end of Pacific Daylight Time + val ts5 = date(2019, 11, 2, 12, 0, 0, 123000, TimeZonePST) + // 2019-11-3 is the start of Pacific Standard Time + val ts6 = date(2019, 11, 3, 12, 0, 0, 123000, TimeZonePST) + + // transit from Pacific Standard Time to Pacific Daylight Time + assert(timestampAddInterval( + ts1, 0, 0, 23 * CalendarInterval.MICROS_PER_HOUR, TimeZonePST.toZoneId) === ts2) + assert(timestampAddInterval(ts1, 0, 1, 0, TimeZonePST.toZoneId) === ts2) + // just a normal day + assert(timestampAddInterval( + ts3, 0, 0, 24 * CalendarInterval.MICROS_PER_HOUR, TimeZonePST.toZoneId) === ts4) + assert(timestampAddInterval(ts3, 0, 1, 0, TimeZonePST.toZoneId) === ts4) + // transit from Pacific Daylight Time to Pacific Standard Time + assert(timestampAddInterval( + ts5, 0, 0, 25 * CalendarInterval.MICROS_PER_HOUR, TimeZonePST.toZoneId) === ts6) + assert(timestampAddInterval(ts5, 0, 1, 0, TimeZonePST.toZoneId) === ts6) } test("monthsBetween") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 22944035f31db..9bdd5aac28a5f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -27,15 +27,15 @@ import org.apache.spark.unsafe.types.CalendarInterval._ class IntervalUtilsSuite extends SparkFunSuite { test("fromString: basic") { - testSingleUnit("YEAR", 3, 36, 0) - testSingleUnit("Month", 3, 3, 0) - testSingleUnit("Week", 3, 0, 3 * MICROS_PER_WEEK) - testSingleUnit("DAY", 3, 0, 3 * MICROS_PER_DAY) - testSingleUnit("HouR", 3, 0, 3 * MICROS_PER_HOUR) - testSingleUnit("MiNuTe", 3, 0, 3 * MICROS_PER_MINUTE) - testSingleUnit("Second", 3, 0, 3 * MICROS_PER_SECOND) - testSingleUnit("MilliSecond", 3, 0, 3 * MICROS_PER_MILLI) - testSingleUnit("MicroSecond", 3, 0, 3) + testSingleUnit("YEAR", 3, 36, 0, 0) + testSingleUnit("Month", 3, 3, 0, 0) + testSingleUnit("Week", 3, 0, 21, 0) + testSingleUnit("DAY", 3, 0, 3, 0) + testSingleUnit("HouR", 3, 0, 0, 3 * MICROS_PER_HOUR) + testSingleUnit("MiNuTe", 3, 0, 0, 3 * MICROS_PER_MINUTE) + testSingleUnit("Second", 3, 0, 0, 3 * MICROS_PER_SECOND) + testSingleUnit("MilliSecond", 3, 0, 0, 3 * MICROS_PER_MILLI) + testSingleUnit("MicroSecond", 3, 0, 0, 3) for (input <- Seq(null, "", " ")) { try { @@ -64,36 +64,37 @@ class IntervalUtilsSuite extends SparkFunSuite { test("fromString: random order field") { val input = "1 day 1 year" - val result = new CalendarInterval(12, MICROS_PER_DAY) + val result = new CalendarInterval(12, 1, 0) assert(fromString(input) == result) } test("fromString: duplicated fields") { val input = "1 day 1 day" - val result = new CalendarInterval(0, 2 * MICROS_PER_DAY) + val result = new CalendarInterval(0, 2, 0) assert(fromString(input) == result) } test("fromString: value with +/-") { val input = "+1 year -1 day" - val result = new CalendarInterval(12, -MICROS_PER_DAY) + val result = new CalendarInterval(12, -1, 0) assert(fromString(input) == result) } - private def testSingleUnit(unit: String, number: Int, months: Int, microseconds: Long): Unit = { + private def testSingleUnit( + unit: String, number: Int, months: Int, days: Int, microseconds: Long): Unit = { for (prefix <- Seq("interval ", "")) { val input1 = prefix + number + " " + unit val input2 = prefix + number + " " + unit + "s" - val result = new CalendarInterval(months, microseconds) + val result = new CalendarInterval(months, days, microseconds) assert(fromString(input1) == result) assert(fromString(input2) == result) } } test("from year-month string") { - assert(fromYearMonthString("99-10") === new CalendarInterval(99 * 12 + 10, 0L)) - assert(fromYearMonthString("+99-10") === new CalendarInterval(99 * 12 + 10, 0L)) - assert(fromYearMonthString("-8-10") === new CalendarInterval(-8 * 12 - 10, 0L)) + assert(fromYearMonthString("99-10") === new CalendarInterval(99 * 12 + 10, 0, 0L)) + assert(fromYearMonthString("+99-10") === new CalendarInterval(99 * 12 + 10, 0, 0L)) + assert(fromYearMonthString("-8-10") === new CalendarInterval(-8 * 12 - 10, 0, 0L)) try { fromYearMonthString("99-15") @@ -116,15 +117,16 @@ class IntervalUtilsSuite extends SparkFunSuite { assert(fromDayTimeString("5 12:40:30.999999999") === new CalendarInterval( 0, - 5 * MICROS_PER_DAY + + 5, 12 * MICROS_PER_HOUR + 40 * MICROS_PER_MINUTE + 30 * MICROS_PER_SECOND + 999999L)) assert(fromDayTimeString("10 0:12:0.888") === new CalendarInterval( 0, - 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE + 888 * MICROS_PER_MILLI)) - assert(fromDayTimeString("-3 0:0:0") === new CalendarInterval(0, -3 * MICROS_PER_DAY)) + 10, + 12 * MICROS_PER_MINUTE + 888 * MICROS_PER_MILLI)) + assert(fromDayTimeString("-3 0:0:0") === new CalendarInterval(0, -3, 0L)) try { fromDayTimeString("5 30:12:20") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 41adf845a6fab..e7b1c0810a033 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -53,7 +53,8 @@ class UnsafeArraySuite extends SparkFunSuite { BigDecimal("1.2345678901234567890123456").setScale(21, BigDecimal.RoundingMode.FLOOR), BigDecimal("2.3456789012345678901234567").setScale(21, BigDecimal.RoundingMode.FLOOR)) - val calenderintervalArray = Array(new CalendarInterval(3, 321), new CalendarInterval(1, 123)) + val calenderintervalArray = Array( + new CalendarInterval(3, 2, 321), new CalendarInterval(1, 2, 123)) val intMultiDimArray = Array(Array(1), Array(2, 20), Array(3, 30, 300)) val doubleMultiDimArray = Array( diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java index 829f3ce750fe6..bce6aa28c42a1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java @@ -165,7 +165,8 @@ private static void appendValue(WritableColumnVector dst, DataType t, Object o) CalendarInterval c = (CalendarInterval)o; dst.appendStruct(false); dst.getChild(0).appendInt(c.months); - dst.getChild(1).appendLong(c.microseconds); + dst.getChild(1).appendInt(c.days); + dst.getChild(2).appendLong(c.microseconds); } else if (t instanceof DateType) { dst.appendInt(DateTimeUtils.fromJavaDate((Date)o)); } else { diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java index 14fac72847af2..951ca39f1bed0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java @@ -736,10 +736,11 @@ protected WritableColumnVector(int capacity, DataType type) { this.childColumns[0] = reserveNewColumn(capacity, mapType.keyType()); this.childColumns[1] = reserveNewColumn(capacity, mapType.valueType()); } else if (type instanceof CalendarIntervalType) { - // Two columns. Months as int. Microseconds as Long. - this.childColumns = new WritableColumnVector[2]; + // Three columns. Months as int. Days as Int. Microseconds as Long. + this.childColumns = new WritableColumnVector[3]; this.childColumns[0] = reserveNewColumn(capacity, DataTypes.IntegerType); - this.childColumns[1] = reserveNewColumn(capacity, DataTypes.LongType); + this.childColumns[1] = reserveNewColumn(capacity, DataTypes.IntegerType); + this.childColumns[2] = reserveNewColumn(capacity, DataTypes.LongType); } else { this.childColumns = null; } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala index b41a4ff766672..e01cd8598db0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -323,7 +323,8 @@ private object RowToColumnConverter { val c = row.getInterval(column) cv.appendStruct(false) cv.getChild(0).appendInt(c.months) - cv.getChild(1).appendLong(c.microseconds) + cv.getChild(1).appendInt(c.days) + cv.getChild(2).appendLong(c.microseconds) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 48113d1c18b25..db404dec956fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -23,6 +23,7 @@ import scala.concurrent.duration.Duration import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.unsafe.types.CalendarInterval private object Triggers { def validate(intervalMs: Long): Unit = { @@ -34,7 +35,7 @@ private object Triggers { if (cal.months != 0) { throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") } - TimeUnit.MICROSECONDS.toMillis(cal.microseconds) + TimeUnit.MICROSECONDS.toMillis(cal.microseconds + cal.days * CalendarInterval.MICROS_PER_DAY) } def convert(interval: Duration): Long = interval.toMillis diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 43ad3c3f539f1..fcf6b6b10c8a1 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -238,7 +238,7 @@ from interval_arithmetic -- !query 17 schema struct -- !query 17 output -2012-01-01 00:00:00 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 2012-04-09 12:22:33.123456 2011-09-23 13:37:26.876544 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 +2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 -- !query 18 @@ -254,7 +254,7 @@ from interval_arithmetic -- !query 18 schema struct -- !query 18 output -2012-01-01 00:00:00 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 2012-04-09 12:22:33.123456 2011-09-23 13:37:26.876544 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 +2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 -- !query 19 diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 37de7159c2d5d..30bae9cd48a9e 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -615,7 +615,7 @@ struct -- !query 76 select date_part('day', c) from t2 -- !query 76 schema -struct +struct -- !query 76 output 8 @@ -623,7 +623,7 @@ struct -- !query 77 select date_part('d', c) from t2 -- !query 77 schema -struct +struct -- !query 77 output 8 @@ -631,7 +631,7 @@ struct -- !query 78 select date_part('days', c) from t2 -- !query 78 schema -struct +struct -- !query 78 output 8 @@ -639,7 +639,7 @@ struct -- !query 79 select date_part('hour', c) from t2 -- !query 79 schema -struct +struct -- !query 79 output 7 @@ -647,7 +647,7 @@ struct -- !query 80 select date_part('h', c) from t2 -- !query 80 schema -struct +struct -- !query 80 output 7 @@ -655,7 +655,7 @@ struct -- !query 81 select date_part('hours', c) from t2 -- !query 81 schema -struct +struct -- !query 81 output 7 @@ -663,7 +663,7 @@ struct -- !query 82 select date_part('hr', c) from t2 -- !query 82 schema -struct +struct -- !query 82 output 7 @@ -671,7 +671,7 @@ struct -- !query 83 select date_part('hrs', c) from t2 -- !query 83 schema -struct +struct -- !query 83 output 7 diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 0f4036cad6125..88a3d65a16e8b 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -136,7 +136,7 @@ select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' -- !query 15 schema struct -- !query 15 output -interval 12 weeks 2 days 14 hours 48 minutes 47 seconds 654 milliseconds 322 microseconds +interval 2078 hours 48 minutes 47 seconds 654 milliseconds 322 microseconds -- !query 16 @@ -144,4 +144,4 @@ select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' -- !query 16 schema struct -- !query 16 output -interval -12 weeks -2 days -14 hours -48 minutes -47 seconds -654 milliseconds -322 microseconds +interval -2078 hours -48 minutes -47 seconds -654 milliseconds -322 microseconds diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 550b9bd936a05..b7a0dcaa1ea0a 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -339,9 +339,9 @@ interval 1 years 2 months 3 weeks 4 days 5 hours 6 minutes 7 seconds 8 milliseco -- !query 36 select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 36 schema -struct +struct -- !query 36 output -interval 32 years 1 months -14 weeks -6 hours -35 minutes -110 milliseconds -13 microseconds +interval 32 years 1 months -14 weeks -2 days 41 hours 24 minutes 59 seconds 889 milliseconds 987 microseconds -- !query 37 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index bed5d7a56c1f8..31e0d65e9a512 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -21,9 +21,9 @@ interval 16 hours 39 minutes -- !query 2 SELECT interval '999' hour -- !query 2 schema -struct +struct -- !query 2 output -interval 5 weeks 6 days 15 hours +interval 999 hours -- !query 3 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out index 80322fb562895..f8bce0c97c8c5 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out @@ -219,11 +219,11 @@ SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff -- !query 24 schema struct<54:string,diff:interval> -- !query 24 output - interval -1409 weeks -8 hours + interval -236720 hours interval 0 microseconds - interval 246 weeks 2 days 17 hours 19 minutes 20 seconds interval 3 hours 4 minutes 5 seconds - interval 5 weeks 4 days 17 hours 32 minutes 1 seconds + interval 41393 hours 19 minutes 20 seconds + interval 953 hours 32 minutes 1 seconds -- !query 25 @@ -242,11 +242,11 @@ SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff -- !query 26 schema struct<54:string,diff:interval> -- !query 26 output - interval -1409 weeks -8 hours + interval -236720 hours interval 0 microseconds - interval 246 weeks 2 days 17 hours 19 minutes 20 seconds interval 3 hours 4 minutes 5 seconds - interval 5 weeks 4 days 17 hours 32 minutes 1 seconds + interval 41393 hours 19 minutes 20 seconds + interval 953 hours 32 minutes 1 seconds -- !query 27 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 99189a96b2995..2b7cc6f182c9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -290,15 +290,15 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val t2 = Timestamp.valueOf("2015-12-31 00:00:00") val d1 = Date.valueOf("2015-07-31") val d2 = Date.valueOf("2015-12-31") - val i = new CalendarInterval(2, 2000000L) + val i = new CalendarInterval(2, 2, 2000000L) val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( df.selectExpr(s"d + $i"), - Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2016-02-29")))) + Seq(Row(Date.valueOf("2015-10-02")), Row(Date.valueOf("2016-03-02")))) checkAnswer( df.selectExpr(s"t + $i"), - Seq(Row(Timestamp.valueOf("2015-10-01 00:00:01")), - Row(Timestamp.valueOf("2016-02-29 00:00:02")))) + Seq(Row(Timestamp.valueOf("2015-10-03 00:00:01")), + Row(Timestamp.valueOf("2016-03-02 00:00:02")))) } test("time_sub") { @@ -306,15 +306,15 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val t2 = Timestamp.valueOf("2016-02-29 00:00:02") val d1 = Date.valueOf("2015-09-30") val d2 = Date.valueOf("2016-02-29") - val i = new CalendarInterval(2, 2000000L) + val i = new CalendarInterval(2, 2, 2000000L) val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( df.selectExpr(s"d - $i"), - Seq(Row(Date.valueOf("2015-07-29")), Row(Date.valueOf("2015-12-28")))) + Seq(Row(Date.valueOf("2015-07-27")), Row(Date.valueOf("2015-12-26")))) checkAnswer( df.selectExpr(s"t - $i"), - Seq(Row(Timestamp.valueOf("2015-07-31 23:59:59")), - Row(Timestamp.valueOf("2015-12-29 00:00:00")))) + Seq(Row(Timestamp.valueOf("2015-07-29 23:59:59")), + Row(Timestamp.valueOf("2015-12-27 00:00:00")))) } test("function add_months") { 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 630489ad9c603..941304cb2553b 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 @@ -1556,7 +1556,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { import org.apache.spark.unsafe.types.CalendarInterval val df = sql("select interval 3 years -3 month 7 week 123 microseconds") - checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * 1000 * 1000 * 3600 * 24 * 7 + 123 ))) + checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7 * 7, 123 ))) withTempPath(f => { // Currently we don't yet support saving out values of interval data type. val e = intercept[AnalysisException] { @@ -1582,17 +1582,17 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { import org.apache.spark.unsafe.types.CalendarInterval.MICROS_PER_WEEK val df = sql("select interval 3 years -3 month 7 week 123 microseconds as i") - checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7L * MICROS_PER_WEEK + 123))) + checkAnswer(df, Row(new CalendarInterval(12 * 3 - 3, 7 * 7, 123))) - checkAnswer(df.select(df("i") + new CalendarInterval(2, 123)), - Row(new CalendarInterval(12 * 3 - 3 + 2, 7L * MICROS_PER_WEEK + 123 + 123))) + checkAnswer(df.select(df("i") + new CalendarInterval(2, 1, 123)), + Row(new CalendarInterval(12 * 3 - 3 + 2, 7 * 7 + 1, 123 + 123))) - checkAnswer(df.select(df("i") - new CalendarInterval(2, 123)), - Row(new CalendarInterval(12 * 3 - 3 - 2, 7L * MICROS_PER_WEEK + 123 - 123))) + checkAnswer(df.select(df("i") - new CalendarInterval(2, 1, 123)), + Row(new CalendarInterval(12 * 3 - 3 - 2, 7 * 7 - 1, 123 - 123))) // unary minus checkAnswer(df.select(-df("i")), - Row(new CalendarInterval(-(12 * 3 - 3), -(7L * MICROS_PER_WEEK + 123)))) + Row(new CalendarInterval(-(12 * 3 - 3), -7 * 7, -123))) } test("aggregation with codegen updates peak execution memory") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index c0f4bb4372bbf..7c7b5b70323b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -636,30 +636,40 @@ class ColumnarBatchSuite extends SparkFunSuite { assert(column.arrayData().elementsAppended == 0) } - testVector("CalendarInterval APIs", 4, CalendarIntervalType) { + testVector("CalendarInterval APIs", 5, CalendarIntervalType) { column => val reference = mutable.ArrayBuffer.empty[CalendarInterval] val months = column.getChild(0) - val microseconds = column.getChild(1) + val days = column.getChild(1) + val microseconds = column.getChild(2) assert(months.dataType() == IntegerType) + assert(days.dataType() == IntegerType) assert(microseconds.dataType() == LongType) months.putInt(0, 1) + days.putInt(0, 10) microseconds.putLong(0, 100) - reference += new CalendarInterval(1, 100) + reference += new CalendarInterval(1, 10, 100) months.putInt(1, 0) + days.putInt(1, 0) microseconds.putLong(1, 2000) - reference += new CalendarInterval(0, 2000) + reference += new CalendarInterval(0, 0, 2000) column.putNull(2) assert(column.getInterval(2) == null) reference += null months.putInt(3, 20) + days.putInt(3, 0) microseconds.putLong(3, 0) - reference += new CalendarInterval(20, 0) + reference += new CalendarInterval(20, 0, 0) + + months.putInt(4, 0) + days.putInt(4, 200) + microseconds.putLong(4, 0) + reference += new CalendarInterval(0, 200, 0) reference.zipWithIndex.foreach { case (v, i) => val errMsg = "VectorType=" + column.getClass.getSimpleName @@ -1311,7 +1321,7 @@ class ColumnarBatchSuite extends SparkFunSuite { Decimal("1234.23456"), DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("2015-01-01")), DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123")), - new CalendarInterval(1, 0), + new CalendarInterval(1, 0, 0), new GenericArrayData(Array(1, 2, 3, 4, null)), new GenericInternalRow(Array[Any](5.asInstanceOf[Any], 10)), mapBuilder.build() @@ -1332,7 +1342,7 @@ class ColumnarBatchSuite extends SparkFunSuite { Decimal("0.01000"), DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("1875-12-12")), DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("1880-01-05 12:45:21.321")), - new CalendarInterval(-10, -100), + new CalendarInterval(-10, -50, -100), new GenericArrayData(Array(5, 10, -100)), new GenericInternalRow(Array[Any](20.asInstanceOf[Any], null)), mapBuilder.build() @@ -1424,8 +1434,8 @@ class ColumnarBatchSuite extends SparkFunSuite { assert(columns(10).isNullAt(2)) assert(columns(11).dataType() == CalendarIntervalType) - assert(columns(11).getInterval(0) == new CalendarInterval(1, 0)) - assert(columns(11).getInterval(1) == new CalendarInterval(-10, -100)) + assert(columns(11).getInterval(0) == new CalendarInterval(1, 0, 0)) + assert(columns(11).getInterval(1) == new CalendarInterval(-10, -50, -100)) assert(columns(11).isNullAt(2)) assert(columns(12).dataType() == ArrayType(IntegerType)) From 14337f68e328c3faad81d3051a56be080a31509d Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 1 Nov 2019 18:29:04 +0800 Subject: [PATCH 0491/1387] [SPARK-29643][SQL] ALTER TABLE/VIEW (DROP PARTITION) should look up catalog/table like v2 commands ###What changes were proposed in this pull request? Add AlterTableDropPartitionStatement and make ALTER TABLE/VIEW ... DROP PARTITION go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC t // success and describe the table t from my_catalog ALTER TABLE t DROP PARTITION (id=1) // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? Yes. When running ALTER TABLE/VIEW ... DROP PARTITION, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26303 from huaxingao/spark-29643. Authored-by: Huaxin Gao Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 26 ++++++++++ .../catalyst/plans/logical/statements.scala | 10 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 50 +++++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 9 ++++ .../spark/sql/execution/SparkSqlParser.scala | 25 ---------- .../sql/connector/DataSourceV2SQLSuite.scala | 11 ++++ .../execution/command/DDLParserSuite.scala | 39 --------------- 8 files changed, 107 insertions(+), 67 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 70facc7282aff..96d1e42ffafe5 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -164,10 +164,8 @@ statement partitionSpec+ #addTablePartition | ALTER TABLE multipartIdentifier from=partitionSpec RENAME TO to=partitionSpec #renameTablePartition - | ALTER TABLE tableIdentifier + | ALTER (TABLE | VIEW) multipartIdentifier DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* PURGE? #dropTablePartitions - | ALTER VIEW tableIdentifier - DROP (IF EXISTS)? partitionSpec (',' partitionSpec)* #dropTablePartitions | ALTER TABLE multipartIdentifier (partitionSpec)? SET locationSpec #setTableLocation | ALTER TABLE multipartIdentifier RECOVER PARTITIONS #recoverPartitions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 93560a2cff70c..838fc4d84a5d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2967,4 +2967,30 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging visitNonOptionalPartitionSpec(ctx.from), visitNonOptionalPartitionSpec(ctx.to)) } + + /** + * Create an [[AlterTableDropPartitionStatement]] + * + * For example: + * {{{ + * ALTER TABLE multi_part_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] + * [PURGE]; + * ALTER VIEW view DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...]; + * }}} + * + * ALTER VIEW ... DROP PARTITION ... is not supported because the concept of partitioning + * is associated with physical tables + */ + override def visitDropTablePartitions( + ctx: DropTablePartitionsContext): LogicalPlan = withOrigin(ctx) { + if (ctx.VIEW != null) { + operationNotAllowed("ALTER VIEW ... DROP PARTITION", ctx) + } + AlterTableDropPartitionStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec), + ifExists = ctx.EXISTS != null, + purge = ctx.PURGE != null, + retainData = false) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index c40b319366684..dd3f8f8ce6c45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -196,6 +196,16 @@ case class AlterTableRenamePartitionStatement( from: TablePartitionSpec, to: TablePartitionSpec) extends ParsedStatement +/** + * ALTER TABLE ... DROP PARTITION command, as parsed from SQL + */ +case class AlterTableDropPartitionStatement( + tableName: Seq[String], + specs: Seq[TablePartitionSpec], + ifExists: Boolean, + purge: Boolean, + retainData: Boolean) extends ParsedStatement + /** * ALTER VIEW ... SET TBLPROPERTIES command, as parsed from SQL. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 2f5faf76e7b7b..10125627ba174 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1228,6 +1228,56 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsed2, expected2) } + // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] + // ALTER VIEW table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] + test("alter table: drop partition") { + val sql1_table = + """ + |ALTER TABLE table_name DROP IF EXISTS PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') + """.stripMargin + val sql2_table = + """ + |ALTER TABLE table_name DROP PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') + """.stripMargin + val sql1_view = sql1_table.replace("TABLE", "VIEW") + val sql2_view = sql2_table.replace("TABLE", "VIEW") + + val parsed1_table = parsePlan(sql1_table) + val parsed2_table = parsePlan(sql2_table) + val parsed1_purge = parsePlan(sql1_table + " PURGE") + + assertUnsupported(sql1_view) + assertUnsupported(sql2_view) + + val expected1_table = AlterTableDropPartitionStatement( + Seq("table_name"), + Seq( + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2009-09-09", "country" -> "uk")), + ifExists = true, + purge = false, + retainData = false) + val expected2_table = expected1_table.copy(ifExists = false) + val expected1_purge = expected1_table.copy(purge = true) + + comparePlans(parsed1_table, expected1_table) + comparePlans(parsed2_table, expected2_table) + comparePlans(parsed1_purge, expected1_purge) + + val sql3_table = "ALTER TABLE a.b.c DROP IF EXISTS PARTITION (ds='2017-06-10')" + val expected3_table = AlterTableDropPartitionStatement( + Seq("a", "b", "c"), + Seq(Map("ds" -> "2017-06-10")), + ifExists = true, + purge = false, + retainData = false) + + val parsed3_table = parsePlan(sql3_table) + comparePlans(parsed3_table, expected3_table) + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index e49e149a4a909..8076bacefb323 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -376,6 +376,15 @@ class ResolveSessionCatalog( v1TableName.asTableIdentifier, from, to) + + case AlterTableDropPartitionStatement(tableName, specs, ifExists, purge, retainData) => + val v1TableName = parseV1Table(tableName, "ALTER TABLE DROP PARTITION") + AlterTableDropPartitionCommand( + v1TableName.asTableIdentifier, + specs, + ifExists, + purge, + retainData) } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index ad75773adc5de..84641f2e93919 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -461,31 +461,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ctx.EXISTS != null) } - /** - * Create an [[AlterTableDropPartitionCommand]] command - * - * For example: - * {{{ - * ALTER TABLE table DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE]; - * ALTER VIEW view DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...]; - * }}} - * - * ALTER VIEW ... DROP PARTITION ... is not supported because the concept of partitioning - * is associated with physical tables - */ - override def visitDropTablePartitions( - ctx: DropTablePartitionsContext): LogicalPlan = withOrigin(ctx) { - if (ctx.VIEW != null) { - operationNotAllowed("ALTER VIEW ... DROP PARTITION", ctx) - } - AlterTableDropPartitionCommand( - visitTableIdentifier(ctx.tableIdentifier), - ctx.partitionSpec.asScala.map(visitNonOptionalPartitionSpec), - ifExists = ctx.EXISTS != null, - purge = ctx.PURGE != null, - retainData = false) - } - /** * Create a [[AlterTableChangeColumnCommand]] command. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index b98120a0a1979..096ae9690cbd8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1396,6 +1396,17 @@ class DataSourceV2SQLSuite } } + test("ALTER TABLE DROP PARTITIONS") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo PARTITIONED BY (id)") + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP PARTITION (id=1)") + } + assert(e.message.contains("ALTER TABLE DROP PARTITION is only supported with v1 tables")) + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 71fce8fe8d167..d82752f532151 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -566,45 +566,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { """.stripMargin) } - // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] - // ALTER VIEW table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] - test("alter table/view: drop partitions") { - val sql1_table = - """ - |ALTER TABLE table_name DROP IF EXISTS PARTITION - |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') - """.stripMargin - val sql2_table = - """ - |ALTER TABLE table_name DROP PARTITION - |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') - """.stripMargin - val sql1_view = sql1_table.replace("TABLE", "VIEW") - val sql2_view = sql2_table.replace("TABLE", "VIEW") - - val parsed1_table = parser.parsePlan(sql1_table) - val parsed2_table = parser.parsePlan(sql2_table) - val parsed1_purge = parser.parsePlan(sql1_table + " PURGE") - assertUnsupported(sql1_view) - assertUnsupported(sql2_view) - - val tableIdent = TableIdentifier("table_name", None) - val expected1_table = AlterTableDropPartitionCommand( - tableIdent, - Seq( - Map("dt" -> "2008-08-08", "country" -> "us"), - Map("dt" -> "2009-09-09", "country" -> "uk")), - ifExists = true, - purge = false, - retainData = false) - val expected2_table = expected1_table.copy(ifExists = false) - val expected1_purge = expected1_table.copy(purge = true) - - comparePlans(parsed1_table, expected1_table) - comparePlans(parsed2_table, expected2_table) - comparePlans(parsed1_purge, expected1_purge) - } - test("alter table: archive partition (not supported)") { assertUnsupported("ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')") } From f53be0a05e04c83a31420e5b3feaf4604f67481c Mon Sep 17 00:00:00 2001 From: DylanGuedes Date: Fri, 1 Nov 2019 22:05:40 +0900 Subject: [PATCH 0492/1387] [SPARK-29109][SQL][TESTS] Port window.sql (Part 3) ### What changes were proposed in this pull request? This PR ports window.sql from PostgreSQL regression tests https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L564-L911 The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/expected/window.out ### Why are the changes needed? To ensure compatibility with PostgreSQL. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass the Jenkins. And, Comparison with PgSQL results. Closes #26274 from DylanGuedes/spark-29109. Authored-by: DylanGuedes Signed-off-by: Takeshi Yamamuro --- .../inputs/postgreSQL/window_part3.sql | 451 ++++++++++++++++++ .../results/postgreSQL/window_part3.sql.out | 409 ++++++++++++++++ 2 files changed, 860 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql new file mode 100644 index 0000000000000..205c7d391a973 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql @@ -0,0 +1,451 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L564-L911 + +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet; + +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15'); + +-- Test in_range for other datetime datatypes + +-- Spark only supports timestamp +-- [SPARK-29636] Spark can't parse '11:00 BST' or '2000-10-19 10:23:54+01' signatures to timestamp +create table datetimes ( + id int, + f_time timestamp, + f_timetz timestamp, + f_interval timestamp, + f_timestamptz timestamp, + f_timestamp timestamp +) using parquet; + +-- Spark cannot safely cast StringType to TimestampType +-- [SPARK-29636] Spark can't parse '11:00 BST' or '2000-10-19 10:23:54+01' signatures to timestamp +insert into datetimes values +(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'), +(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'), +(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'), +(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'), +(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'), +(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'), +(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'), +(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54'); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_time, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_time range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_time, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_time desc range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timetz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timetz range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timetz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timetz desc range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_interval, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_interval range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_interval, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_interval desc range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamptz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamptz range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamptz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamptz desc range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamp, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamp range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamp, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamp desc range between +-- '1 year' preceding and '1 year' following); + +-- RANGE offset PRECEDING/FOLLOWING error cases +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date, salary range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by depname range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by enroll_date range between 1 preceding and 2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between -1 preceding and 2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between 1 preceding and -2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by enroll_date range between '1 year' preceding and '-2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- GROUPS tests + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between current row and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 1 preceding and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 1 following and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and 2 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 preceding), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 0 preceding and 0 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 following +-- exclude current row), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 following +-- exclude group), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 following +-- exclude ties), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following),unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude current row), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude group), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude ties), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select first_value(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- lead(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- nth_value(salary, 1) over(order by enroll_date groups between 1 preceding and 1 following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28508] Support for range frame+row frame in the same query +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select last(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- lag(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- salary, enroll_date from empsalary; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- select first_value(salary) over(order by enroll_date groups between 1 following and 3 following +-- exclude current row), +-- lead(salary) over(order by enroll_date groups between 1 following and 3 following exclude ties), +-- nth_value(salary, 1) over(order by enroll_date groups between 1 following and 3 following +-- exclude ties), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select last(salary) over(order by enroll_date groups between 1 following and 3 following +-- exclude group), +-- lag(salary) over(order by enroll_date groups between 1 following and 3 following exclude group), +-- salary, enroll_date from empsalary; + +-- Show differences in offset interpretation between ROWS, RANGE, and GROUPS +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following); + +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following); + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- WITH cte (x) AS ( +-- SELECT * FROM range(1, 36, 2) +-- ) +-- SELECT x, (sum(x) over w) +-- FROM cte +-- WINDOW w AS (ORDER BY x groups between 1 preceding and 1 following); + +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following); + +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following); + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- WITH cte (x) AS ( +-- select 1 union all select 1 union all select 1 union all +-- SELECT * FROM range(5, 50, 2) +-- ) +-- SELECT x, (sum(x) over w) +-- FROM cte +-- WINDOW w AS (ORDER BY x groups between 1 preceding and 1 following); + +-- with UNION +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 UNION ALL SELECT * FROM tenk2)s LIMIT 0; + +-- check some degenerate cases +create table t1 (f1 int, f2 int) using parquet; +insert into t1 values (1,1),(1,2),(2,2); + +select f1, sum(f1) over (partition by f1 + range between 1 preceding and 1 following) +from t1 where f1 = f2; -- error, must have order by + +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- explain +-- select f1, sum(f1) over (partition by f1 order by f2 +-- range between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1 order by f2 +range between 1 preceding and 1 following) +from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1, f1 order by f2 +range between 2 preceding and 1 preceding) +from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1, f2 order by f2 +range between 1 following and 2 following) +from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, +-- groups between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- explain +-- select f1, sum(f1) over (partition by f1 order by f2 +-- range between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1 order by f2 +-- groups between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, f1 order by f2 +-- groups between 2 preceding and 1 preceding) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, f2 order by f2 +-- groups between 1 following and 2 following) +-- from t1 where f1 = f2; + +-- ordering by a non-integer constant is allowed +SELECT rank() OVER (ORDER BY length('abc')); + +-- can't order by another window function +-- [SPARK-28566] window functions should not be allowed in window definitions +-- SELECT rank() OVER (ORDER BY rank() OVER (ORDER BY random())); + +-- some other errors +SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10; + +SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10; + +SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1; + +SELECT * FROM rank() OVER (ORDER BY random()); + +-- Original query: DELETE FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10; +SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10; + +-- Original query: DELETE FROM empsalary RETURNING rank() OVER (ORDER BY random()); +SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random()); + +-- [SPARK-28645] Throw an error on window redefinition +-- select count(*) OVER w FROM tenk1 WINDOW w AS (ORDER BY unique1), w AS (ORDER BY unique1); + +select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1; + +-- [SPARK-28646] Allow usage of `count` only for parameterless aggregate function +-- select count() OVER () FROM tenk1; + +-- The output is the expected one: `range` is not a window or aggregate function. +SELECT range(1, 100) OVER () FROM empsalary; + +SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1; + +-- filter + +-- [SPARK-28500] Adds support for `filter` clause +-- SELECT sum(salary), row_number() OVER (ORDER BY depname), sum( +-- sum(salary) FILTER (WHERE enroll_date > '2007-01-01') +-- ) +-- FROM empsalary GROUP BY depname; + +-- Test pushdown of quals into a subquery containing window functions + +-- pushdown is safe because all PARTITION BY clauses include depname: +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY depname) depsalary, +-- min(salary) OVER (PARTITION BY depname || 'A', depname) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- pushdown is unsafe because there's a PARTITION BY clause without depname: +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY enroll_date) enroll_salary, +-- min(salary) OVER (PARTITION BY depname) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- Test Sort node collapsing +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY depname order by empno) depsalary, +-- min(salary) OVER (PARTITION BY depname, empno order by enroll_date) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- Test Sort node reordering +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT +-- lead(1) OVER (PARTITION BY depname ORDER BY salary, enroll_date), +-- lag(1) OVER (PARTITION BY depname ORDER BY salary,enroll_date,empno) +-- FROM empsalary; + +-- cleanup +DROP TABLE empsalary; +DROP TABLE datetimes; +DROP TABLE t1; diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out new file mode 100644 index 0000000000000..80e4385ebd84b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out @@ -0,0 +1,409 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 29 + + +-- !query 0 +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +create table datetimes ( + id int, + f_time timestamp, + f_timetz timestamp, + f_interval timestamp, + f_timestamptz timestamp, + f_timestamp timestamp +) using parquet +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +insert into datetimes values +(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'), +(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'), +(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'), +(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'), +(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'), +(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'), +(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'), +(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) +-- !query 5 schema +struct +-- !query 5 output +1 4 +11 33 +13 39 +15 45 +17 51 +19 57 +21 63 +23 69 +25 75 +27 81 +29 87 +3 9 +31 93 +33 99 +35 68 +5 15 +7 21 +9 27 + + +-- !query 6 +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) +-- !query 6 schema +struct +-- !query 6 output +1 1 +11 11 +13 13 +15 15 +17 17 +19 19 +21 21 +23 23 +25 25 +27 27 +29 29 +3 3 +31 31 +33 33 +35 35 +5 5 +7 7 +9 9 + + +-- !query 7 +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) +-- !query 7 schema +struct +-- !query 7 output +1 2 +1 3 +1 7 +11 33 +13 39 +15 45 +17 51 +19 57 +21 63 +23 69 +25 75 +27 81 +29 87 +31 93 +33 99 +35 105 +37 111 +39 117 +41 123 +43 129 +45 135 +47 141 +49 96 +5 13 +7 21 +9 27 + + +-- !query 8 +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) +-- !query 8 schema +struct +-- !query 8 output +1 3 +1 3 +1 3 +11 11 +13 13 +15 15 +17 17 +19 19 +21 21 +23 23 +25 25 +27 27 +29 29 +31 31 +33 33 +35 35 +37 37 +39 39 +41 41 +43 43 +45 45 +47 47 +49 49 +5 5 +7 7 +9 9 + + +-- !query 9 +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 UNION ALL SELECT * FROM tenk2)s LIMIT 0 +-- !query 9 schema +struct +-- !query 9 output + + + +-- !query 10 +create table t1 (f1 int, f2 int) using parquet +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +insert into t1 values (1,1),(1,2),(2,2) +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +select f1, sum(f1) over (partition by f1 + range between 1 preceding and 1 following) +from t1 where f1 = f2 +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +cannot resolve '(PARTITION BY default.t1.`f1` RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 24 + + +-- !query 13 +select f1, sum(f1) over (partition by f1 order by f2 +range between 1 preceding and 1 following) +from t1 where f1 = f2 +-- !query 13 schema +struct +-- !query 13 output +1 1 +2 2 + + +-- !query 14 +select f1, sum(f1) over (partition by f1, f1 order by f2 +range between 2 preceding and 1 preceding) +from t1 where f1 = f2 +-- !query 14 schema +struct +-- !query 14 output +1 NULL +2 NULL + + +-- !query 15 +select f1, sum(f1) over (partition by f1, f2 order by f2 +range between 1 following and 2 following) +from t1 where f1 = f2 +-- !query 15 schema +struct +-- !query 15 output +1 NULL +2 NULL + + +-- !query 16 +SELECT rank() OVER (ORDER BY length('abc')) +-- !query 16 schema +struct +-- !query 16 output +1 + + +-- !query 17 +SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10 +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +It is not allowed to use window functions inside WHERE and HAVING clauses; + + +-- !query 18 +SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10 +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException + +The query operator `Join` contains one or more unsupported +expression types Aggregate, Window or Generate. +Invalid expressions: [row_number() OVER (ORDER BY default.empsalary.`salary` ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]; + + +-- !query 19 +SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1 +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException + +The query operator `Aggregate` contains one or more unsupported +expression types Aggregate, Window or Generate. +Invalid expressions: [RANK() OVER (ORDER BY 1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]; + + +-- !query 20 +SELECT * FROM rank() OVER (ORDER BY random()) +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'ORDER'(line 1, pos 27) + +== SQL == +SELECT * FROM rank() OVER (ORDER BY random()) +---------------------------^^^ + + +-- !query 21 +SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10 +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +It is not allowed to use window functions inside WHERE and HAVING clauses; + + +-- !query 22 +SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random()) +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +It is not allowed to use window functions inside WHERE and HAVING clauses; + + +-- !query 23 +select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1 +-- !query 23 schema +struct<> +-- !query 23 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'ORDER'(line 1, pos 39) + +== SQL == +select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1 +---------------------------------------^^^ + + +-- !query 24 +SELECT range(1, 100) OVER () FROM empsalary +-- !query 24 schema +struct<> +-- !query 24 output +org.apache.spark.sql.AnalysisException +Undefined function: 'range'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 + + +-- !query 25 +SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1 +-- !query 25 schema +struct<> +-- !query 25 output +org.apache.spark.sql.AnalysisException +cannot resolve 'ntile(0)' due to data type mismatch: Buckets expression must be positive, but got: 0; line 1 pos 7 + + +-- !query 26 +DROP TABLE empsalary +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +DROP TABLE datetimes +-- !query 27 schema +struct<> +-- !query 27 output + + + +-- !query 28 +DROP TABLE t1 +-- !query 28 schema +struct<> +-- !query 28 output + From 39fff9258af846cca2dd82816f0e2cb7d416a2c4 Mon Sep 17 00:00:00 2001 From: root1 Date: Fri, 1 Nov 2019 08:27:34 -0500 Subject: [PATCH 0493/1387] [SPARK-29452][WEBUI] Improve Storage tab tooltip ### What changes were proposed in this pull request? Added Tootips for each column in storage tab of Web UI. ### Why are the changes needed? Tooltips will help users in understanding columns of storage tabs. ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Manually Tested. Closes #26226 from iRakson/storage_tooltip. Authored-by: root1 Signed-off-by: Sean Owen --- .../apache/spark/ui/storage/StoragePage.scala | 14 ++++++- .../apache/spark/ui/storage/ToolTips.scala | 42 +++++++++++++++++++ .../spark/ui/storage/StoragePageSuite.scala | 17 +++++++- 3 files changed, 71 insertions(+), 2 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/storage/ToolTips.scala diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 2488197814ffd..fb43af357f7b8 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -25,6 +25,7 @@ import scala.xml.Node import org.apache.spark.status.{AppStatusStore, StreamBlockData} import org.apache.spark.status.api.v1 import org.apache.spark.ui._ +import org.apache.spark.ui.storage.ToolTips._ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ @@ -56,7 +57,8 @@ private[ui] class StoragePage(parent: SparkUITab, store: AppStatusStore) extends rddHeader, rddRow(request, _: v1.RDDStorageInfo), rdds, - id = Some("storage-by-rdd-table"))} + id = Some("storage-by-rdd-table"), + tooltipHeaders = tooltips)}
    } @@ -72,6 +74,16 @@ private[ui] class StoragePage(parent: SparkUITab, store: AppStatusStore) extends "Size in Memory", "Size on Disk") + /** Tooltips for header fields of the RDD table */ + val tooltips = Seq( + None, + Some(RDD_NAME), + Some(STORAGE_LEVEL), + Some(CACHED_PARTITIONS), + Some(FRACTION_CACHED), + Some(SIZE_IN_MEMORY), + Some(SIZE_ON_DISK)) + /** Render an HTML row representing an RDD */ private def rddRow(request: HttpServletRequest, rdd: v1.RDDStorageInfo): Seq[Node] = { // scalastyle:off diff --git a/core/src/main/scala/org/apache/spark/ui/storage/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/storage/ToolTips.scala new file mode 100644 index 0000000000000..4677eba63c830 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/storage/ToolTips.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.storage + +private[ui] object ToolTips { + + val RDD_NAME = + "Name of the persisted RDD" + + val STORAGE_LEVEL = + "StorageLevel displays where the persisted RDD is stored, " + + "format of the persisted RDD (serialized or de-serialized) and" + + "replication factor of the persisted RDD" + + val CACHED_PARTITIONS = + "Number of partitions cached" + + val FRACTION_CACHED = + "Fraction of total partitions cached" + + val SIZE_IN_MEMORY = + "Total size of partitions in memory" + + val SIZE_ON_DISK = + "Total size of partitions on the disk" +} + diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala index 06f01a60868f9..f93ecd3b006b2 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.ui.storage import javax.servlet.http.HttpServletRequest import org.mockito.Mockito._ +import scala.xml.{Node, Text} import org.apache.spark.SparkFunSuite import org.apache.spark.status.StreamBlockData @@ -74,7 +75,21 @@ class StoragePageSuite extends SparkFunSuite { "Fraction Cached", "Size in Memory", "Size on Disk") - assert((xmlNodes \\ "th").map(_.text) === headers) + + val headerRow: Seq[Node] = { + headers.view.zipWithIndex.map { x => + storagePage.tooltips(x._2) match { + case Some(tooltip) => + + + {Text(x._1)} + + + case None => {Text(x._1)} + } + }.toList + } + assert((xmlNodes \\ "th").map(_.text) === headerRow.map(_.text)) assert((xmlNodes \\ "tr").size === 3) assert(((xmlNodes \\ "tr")(0) \\ "td").map(_.text.trim) === From 1e1b7302f482a3b81e1fcd7060b4849a488376bf Mon Sep 17 00:00:00 2001 From: Matt Stillwell <18670089+mstill3@users.noreply.github.com> Date: Fri, 1 Nov 2019 11:55:29 -0700 Subject: [PATCH 0494/1387] [MINOR][PYSPARK][DOCS] Fix typo in example documentation ### What changes were proposed in this pull request? I propose that we change the example code documentation to call the proper function . For example, under the `foreachBatch` function, the example code was calling the `foreach()` function by mistake. ### Why are the changes needed? I suppose it could confuse some people, and it is a typo ### Does this PR introduce any user-facing change? No, there is no "meaningful" code being change, simply the documentation ### How was this patch tested? I made the change on a fork and it still worked Closes #26299 from mstill3/patch-1. Authored-by: Matt Stillwell <18670089+mstill3@users.noreply.github.com> Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/context.py | 2 +- python/pyspark/sql/dataframe.py | 112 +++++++++++++++++-------------- python/pyspark/sql/readwriter.py | 2 +- python/pyspark/sql/session.py | 20 +++--- python/pyspark/sql/streaming.py | 9 ++- 5 files changed, 78 insertions(+), 67 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index e6bd2eb31a844..68d5ef4ca7b15 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -318,7 +318,7 @@ def registerDataFrameAsTable(self, df, tableName): @since(1.6) def dropTempTable(self, tableName): - """ Remove the temp table from catalog. + """ Remove the temporary table from catalog. >>> sqlContext.registerDataFrameAsTable(df, "table1") >>> sqlContext.dropTempTable("table1") diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 03b37fa7d0d9b..4683c74277447 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -58,7 +58,7 @@ class DataFrame(object): Once created, it can be manipulated using the various domain-specific-language (DSL) functions defined in: :class:`DataFrame`, :class:`Column`. - To select a column from the data frame, use the apply method:: + To select a column from the :class:`DataFrame`, use the apply method:: ageCol = people.age @@ -124,7 +124,7 @@ def toJSON(self, use_unicode=True): @since(2.0) def createTempView(self, name): - """Creates a local temporary view with this DataFrame. + """Creates a local temporary view with this :class:`DataFrame`. The lifetime of this temporary table is tied to the :class:`SparkSession` that was used to create this :class:`DataFrame`. @@ -146,7 +146,7 @@ def createTempView(self, name): @since(2.0) def createOrReplaceTempView(self, name): - """Creates or replaces a local temporary view with this DataFrame. + """Creates or replaces a local temporary view with this :class:`DataFrame`. The lifetime of this temporary table is tied to the :class:`SparkSession` that was used to create this :class:`DataFrame`. @@ -164,7 +164,7 @@ def createOrReplaceTempView(self, name): @since(2.1) def createGlobalTempView(self, name): - """Creates a global temporary view with this DataFrame. + """Creates a global temporary view with this :class:`DataFrame`. The lifetime of this temporary view is tied to this Spark application. throws :class:`TempTableAlreadyExistsException`, if the view name already exists in the @@ -312,7 +312,7 @@ def isLocal(self): @property @since(2.0) def isStreaming(self): - """Returns true if this :class:`Dataset` contains one or more sources that continuously + """Returns ``True`` if this :class:`Dataset` contains one or more sources that continuously return data as it arrives. A :class:`Dataset` that reads data from a streaming source must be executed as a :class:`StreamingQuery` using the :func:`start` method in :class:`DataStreamWriter`. Methods that return a single answer, (e.g., :func:`count` or @@ -328,10 +328,10 @@ def show(self, n=20, truncate=True, vertical=False): """Prints the first ``n`` rows to the console. :param n: Number of rows to show. - :param truncate: If set to True, truncate strings longer than 20 chars by default. + :param truncate: If set to ``True``, truncate strings longer than 20 chars by default. If set to a number greater than one, truncates long strings to length ``truncate`` and align cells right. - :param vertical: If set to True, print output rows vertically (one line + :param vertical: If set to ``True``, print output rows vertically (one line per column value). >>> df @@ -373,7 +373,7 @@ def __repr__(self): return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes)) def _repr_html_(self): - """Returns a dataframe with html code when you enabled eager evaluation + """Returns a :class:`DataFrame` with html code when you enabled eager evaluation by 'spark.sql.repl.eagerEval.enabled', this only called by REPL you are using support eager evaluation with HTML. """ @@ -407,11 +407,11 @@ def _repr_html_(self): @since(2.1) def checkpoint(self, eager=True): """Returns a checkpointed version of this Dataset. Checkpointing can be used to truncate the - logical plan of this DataFrame, which is especially useful in iterative algorithms where the - plan may grow exponentially. It will be saved to files inside the checkpoint + logical plan of this :class:`DataFrame`, which is especially useful in iterative algorithms + where the plan may grow exponentially. It will be saved to files inside the checkpoint directory set with :meth:`SparkContext.setCheckpointDir`. - :param eager: Whether to checkpoint this DataFrame immediately + :param eager: Whether to checkpoint this :class:`DataFrame` immediately .. note:: Experimental """ @@ -421,11 +421,11 @@ def checkpoint(self, eager=True): @since(2.3) def localCheckpoint(self, eager=True): """Returns a locally checkpointed version of this Dataset. Checkpointing can be used to - truncate the logical plan of this DataFrame, which is especially useful in iterative - algorithms where the plan may grow exponentially. Local checkpoints are stored in the - executors using the caching subsystem and therefore they are not reliable. + truncate the logical plan of this :class:`DataFrame`, which is especially useful in + iterative algorithms where the plan may grow exponentially. Local checkpoints are + stored in the executors using the caching subsystem and therefore they are not reliable. - :param eager: Whether to checkpoint this DataFrame immediately + :param eager: Whether to checkpoint this :class:`DataFrame` immediately .. note:: Experimental """ @@ -468,7 +468,7 @@ def withWatermark(self, eventTime, delayThreshold): @since(2.2) def hint(self, name, *parameters): - """Specifies some hint on the current DataFrame. + """Specifies some hint on the current :class:`DataFrame`. :param name: A name of the hint. :param parameters: Optional parameters. @@ -523,8 +523,9 @@ def collect(self): def toLocalIterator(self, prefetchPartitions=False): """ Returns an iterator that contains all of the rows in this :class:`DataFrame`. - The iterator will consume as much memory as the largest partition in this DataFrame. - With prefetch it may consume up to the memory of the 2 largest partitions. + The iterator will consume as much memory as the largest partition in this + :class:`DataFrame`. With prefetch it may consume up to the memory of the 2 largest + partitions. :param prefetchPartitions: If Spark should pre-fetch the next partition before it is needed. @@ -633,7 +634,7 @@ def unpersist(self, blocking=False): """Marks the :class:`DataFrame` as non-persistent, and remove all blocks for it from memory and disk. - .. note:: `blocking` default has changed to False to match Scala in 2.0. + .. note:: `blocking` default has changed to ``False`` to match Scala in 2.0. """ self.is_cached = False self._jdf.unpersist(blocking) @@ -668,7 +669,7 @@ def coalesce(self, numPartitions): def repartition(self, numPartitions, *cols): """ Returns a new :class:`DataFrame` partitioned by the given partitioning expressions. The - resulting DataFrame is hash partitioned. + resulting :class:`DataFrame` is hash partitioned. :param numPartitions: can be an int to specify the target number of partitions or a Column. @@ -730,7 +731,7 @@ def repartition(self, numPartitions, *cols): def repartitionByRange(self, numPartitions, *cols): """ Returns a new :class:`DataFrame` partitioned by the given partitioning expressions. The - resulting DataFrame is range partitioned. + resulting :class:`DataFrame` is range partitioned. :param numPartitions: can be an int to specify the target number of partitions or a Column. @@ -790,7 +791,7 @@ def distinct(self): def sample(self, withReplacement=None, fraction=None, seed=None): """Returns a sampled subset of this :class:`DataFrame`. - :param withReplacement: Sample with replacement or not (default False). + :param withReplacement: Sample with replacement or not (default ``False``). :param fraction: Fraction of rows to generate, range [0.0, 1.0]. :param seed: Seed for sampling (default a random seed). @@ -862,7 +863,7 @@ def sampleBy(self, col, fractions, seed=None): sampling fraction for each stratum. If a stratum is not specified, we treat its fraction as zero. :param seed: random seed - :return: a new DataFrame that represents the stratified sample + :return: a new :class:`DataFrame` that represents the stratified sample >>> from pyspark.sql.functions import col >>> dataset = sqlContext.range(0, 100).select((col("id") % 3).alias("key")) @@ -898,8 +899,8 @@ def sampleBy(self, col, fractions, seed=None): def randomSplit(self, weights, seed=None): """Randomly splits this :class:`DataFrame` with the provided weights. - :param weights: list of doubles as weights with which to split the DataFrame. Weights will - be normalized if they don't sum up to 1.0. + :param weights: list of doubles as weights with which to split the :class:`DataFrame`. + Weights will be normalized if they don't sum up to 1.0. :param seed: The seed for sampling. >>> splits = df4.randomSplit([1.0, 2.0], 24) @@ -964,7 +965,7 @@ def colRegex(self, colName): def alias(self, alias): """Returns a new :class:`DataFrame` with an alias set. - :param alias: string, an alias name to be set for the DataFrame. + :param alias: string, an alias name to be set for the :class:`DataFrame`. >>> from pyspark.sql.functions import * >>> df_as1 = df.alias("df_as1") @@ -1056,7 +1057,7 @@ def sortWithinPartitions(self, *cols, **kwargs): """Returns a new :class:`DataFrame` with each partition sorted by the specified column(s). :param cols: list of :class:`Column` or column names to sort by. - :param ascending: boolean or list of boolean (default True). + :param ascending: boolean or list of boolean (default ``True``). Sort ascending vs. descending. Specify list for multiple sort orders. If a list is specified, length of the list must equal length of the `cols`. @@ -1077,7 +1078,7 @@ def sort(self, *cols, **kwargs): """Returns a new :class:`DataFrame` sorted by the specified column(s). :param cols: list of :class:`Column` or column names to sort by. - :param ascending: boolean or list of boolean (default True). + :param ascending: boolean or list of boolean (default ``True``). Sort ascending vs. descending. Specify list for multiple sort orders. If a list is specified, length of the list must equal length of the `cols`. @@ -1144,7 +1145,8 @@ def describe(self, *cols): given, this function computes statistics for all numerical or string columns. .. note:: This function is meant for exploratory data analysis, as we make no - guarantee about the backward compatibility of the schema of the resulting DataFrame. + guarantee about the backward compatibility of the schema of the resulting + :class:`DataFrame`. >>> df.describe(['age']).show() +-------+------------------+ @@ -1188,7 +1190,8 @@ def summary(self, *statistics): approximate quartiles (percentiles at 25%, 50%, and 75%), and max. .. note:: This function is meant for exploratory data analysis, as we make no - guarantee about the backward compatibility of the schema of the resulting DataFrame. + guarantee about the backward compatibility of the schema of the resulting + :class:`DataFrame`. >>> df.summary().show() +-------+------------------+-----+ @@ -1310,7 +1313,7 @@ def select(self, *cols): :param cols: list of column names (string) or expressions (:class:`Column`). If one of the column names is '*', that column is expanded to include all columns - in the current DataFrame. + in the current :class:`DataFrame`. >>> df.select('*').collect() [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] @@ -1414,7 +1417,7 @@ def rollup(self, *cols): def cube(self, *cols): """ Create a multi-dimensional cube for the current :class:`DataFrame` using - the specified columns, so we can run aggregation on them. + the specified columns, so we can run aggregations on them. >>> df.cube("name", df.age).count().orderBy("name", "age").show() +-----+----+-----+ @@ -1448,7 +1451,8 @@ def agg(self, *exprs): @since(2.0) def union(self, other): - """ Return a new :class:`DataFrame` containing union of rows in this and another frame. + """ Return a new :class:`DataFrame` containing union of rows in this and another + :class:`DataFrame`. This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union (that does deduplication of elements), use this function followed by :func:`distinct`. @@ -1459,7 +1463,8 @@ def union(self, other): @since(1.3) def unionAll(self, other): - """ Return a new :class:`DataFrame` containing union of rows in this and another frame. + """ Return a new :class:`DataFrame` containing union of rows in this and another + :class:`DataFrame`. This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union (that does deduplication of elements), use this function followed by :func:`distinct`. @@ -1470,7 +1475,8 @@ def unionAll(self, other): @since(2.3) def unionByName(self, other): - """ Returns a new :class:`DataFrame` containing union of rows in this and another frame. + """ Returns a new :class:`DataFrame` containing union of rows in this and another + :class:`DataFrame`. This is different from both `UNION ALL` and `UNION DISTINCT` in SQL. To do a SQL-style set union (that does deduplication of elements), use this function followed by :func:`distinct`. @@ -1493,7 +1499,7 @@ def unionByName(self, other): @since(1.3) def intersect(self, other): """ Return a new :class:`DataFrame` containing rows only in - both this frame and another frame. + both this :class:`DataFrame` and another :class:`DataFrame`. This is equivalent to `INTERSECT` in SQL. """ @@ -1501,8 +1507,8 @@ def intersect(self, other): @since(2.4) def intersectAll(self, other): - """ Return a new :class:`DataFrame` containing rows in both this dataframe and other - dataframe while preserving duplicates. + """ Return a new :class:`DataFrame` containing rows in both this :class:`DataFrame` + and another :class:`DataFrame` while preserving duplicates. This is equivalent to `INTERSECT ALL` in SQL. >>> df1 = spark.createDataFrame([("a", 1), ("a", 1), ("b", 3), ("c", 4)], ["C1", "C2"]) @@ -1523,8 +1529,8 @@ def intersectAll(self, other): @since(1.3) def subtract(self, other): - """ Return a new :class:`DataFrame` containing rows in this frame - but not in another frame. + """ Return a new :class:`DataFrame` containing rows in this :class:`DataFrame` + but not in another :class:`DataFrame`. This is equivalent to `EXCEPT DISTINCT` in SQL. @@ -1814,12 +1820,12 @@ def all_of_(xs): def approxQuantile(self, col, probabilities, relativeError): """ Calculates the approximate quantiles of numerical columns of a - DataFrame. + :class:`DataFrame`. The result of this algorithm has the following deterministic bound: - If the DataFrame has N elements and if we request the quantile at + If the :class:`DataFrame` has N elements and if we request the quantile at probability `p` up to error `err`, then the algorithm will return - a sample `x` from the DataFrame so that the *exact* rank of `x` is + a sample `x` from the :class:`DataFrame` so that the *exact* rank of `x` is close to (p * N). More precisely, floor((p - err) * N) <= rank(x) <= ceil((p + err) * N). @@ -1887,7 +1893,7 @@ def approxQuantile(self, col, probabilities, relativeError): @since(1.4) def corr(self, col1, col2, method=None): """ - Calculates the correlation of two columns of a DataFrame as a double value. + Calculates the correlation of two columns of a :class:`DataFrame` as a double value. Currently only supports the Pearson Correlation Coefficient. :func:`DataFrame.corr` and :func:`DataFrameStatFunctions.corr` are aliases of each other. @@ -1935,7 +1941,7 @@ def crosstab(self, col1, col2): :param col1: The name of the first column. Distinct items will make the first item of each row. :param col2: The name of the second column. Distinct items will make the column names - of the DataFrame. + of the :class:`DataFrame`. """ if not isinstance(col1, basestring): raise ValueError("col1 should be a string.") @@ -1952,7 +1958,8 @@ def freqItems(self, cols, support=None): :func:`DataFrame.freqItems` and :func:`DataFrameStatFunctions.freqItems` are aliases. .. note:: This function is meant for exploratory data analysis, as we make no - guarantee about the backward compatibility of the schema of the resulting DataFrame. + guarantee about the backward compatibility of the schema of the resulting + :class:`DataFrame`. :param cols: Names of the columns to calculate frequent items for as a list or tuple of strings. @@ -1974,8 +1981,8 @@ def withColumn(self, colName, col): Returns a new :class:`DataFrame` by adding a column or replacing the existing column that has the same name. - The column expression must be an expression over this DataFrame; attempting to add - a column from some other dataframe will raise an error. + The column expression must be an expression over this :class:`DataFrame`; attempting to add + a column from some other :class:`DataFrame` will raise an error. :param colName: string, name of the new column. :param col: a :class:`Column` expression for the new column. @@ -2090,8 +2097,8 @@ def toPandas(self): This is only available if Pandas is installed and available. - .. note:: This method should only be used if the resulting Pandas's DataFrame is expected - to be small, as all the data is loaded into the driver's memory. + .. note:: This method should only be used if the resulting Pandas's :class:`DataFrame` is + expected to be small, as all the data is loaded into the driver's memory. .. note:: Usage with spark.sql.execution.arrow.pyspark.enabled=True is experimental. @@ -2293,8 +2300,9 @@ def _to_scala_map(sc, jm): def _to_corrected_pandas_type(dt): """ - When converting Spark SQL records to Pandas DataFrame, the inferred data type may be wrong. - This method gets the corrected data type for Pandas if that type may be inferred uncorrectly. + When converting Spark SQL records to Pandas :class:`DataFrame`, the inferred data type may be + wrong. This method gets the corrected data type for Pandas if that type may be inferred + uncorrectly. """ import numpy as np if type(dt) == ByteType: diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 18fd7de7ee547..a392b29bb379f 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -733,7 +733,7 @@ def save(self, path=None, format=None, mode=None, partitionBy=None, **options): :param partitionBy: names of partitioning columns :param options: all other string options - >>> df.write.mode('append').parquet(os.path.join(tempfile.mkdtemp(), 'data')) + >>> df.write.mode("append").save(os.path.join(tempfile.mkdtemp(), 'data')) """ self.mode(mode).options(**options) if partitionBy is not None: diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index cdab840b2c40a..1c3c7778c7ee1 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -136,7 +136,7 @@ def appName(self, name): @since(2.0) def enableHiveSupport(self): """Enables Hive support, including connectivity to a persistent Hive metastore, support - for Hive serdes, and Hive user-defined functions. + for Hive SerDes, and Hive user-defined functions. """ return self.config("spark.sql.catalogImplementation", "hive") @@ -189,7 +189,7 @@ def getOrCreate(self): return session builder = Builder() - """A class attribute having a :class:`Builder` to construct :class:`SparkSession` instances""" + """A class attribute having a :class:`Builder` to construct :class:`SparkSession` instances.""" _instantiatedSession = None _activeSession = None @@ -310,7 +310,7 @@ def conf(self): @since(2.0) def catalog(self): """Interface through which the user may create, drop, alter or query underlying - databases, tables, functions etc. + databases, tables, functions, etc. :return: :class:`Catalog` """ @@ -624,20 +624,20 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr will be inferred from ``data``. When ``schema`` is ``None``, it will try to infer the schema (column names and types) - from ``data``, which should be an RDD of :class:`Row`, - or :class:`namedtuple`, or :class:`dict`. + from ``data``, which should be an RDD of either :class:`Row`, + :class:`namedtuple`, or :class:`dict`. When ``schema`` is :class:`pyspark.sql.types.DataType` or a datatype string, it must match the real data, or an exception will be thrown at runtime. If the given schema is not :class:`pyspark.sql.types.StructType`, it will be wrapped into a - :class:`pyspark.sql.types.StructType` as its only field, and the field name will be "value", - each record will also be wrapped into a tuple, which can be converted to row later. + :class:`pyspark.sql.types.StructType` as its only field, and the field name will be "value". + Each record will also be wrapped into a tuple, which can be converted to row later. If schema inference is needed, ``samplingRatio`` is used to determined the ratio of rows used for schema inference. The first row will be used if ``samplingRatio`` is ``None``. - :param data: an RDD of any kind of SQL data representation(e.g. row, tuple, int, boolean, - etc.), or :class:`list`, or :class:`pandas.DataFrame`. + :param data: an RDD of any kind of SQL data representation (e.g. row, tuple, int, boolean, + etc.), :class:`list`, or :class:`pandas.DataFrame`. :param schema: a :class:`pyspark.sql.types.DataType` or a datatype string or a list of column names, default is ``None``. The data type string format equals to :class:`pyspark.sql.types.DataType.simpleString`, except that top level struct type can @@ -846,7 +846,7 @@ def readStream(self): @since(2.0) def streams(self): """Returns a :class:`StreamingQueryManager` that allows managing all the - :class:`StreamingQuery` StreamingQueries active on `this` context. + :class:`StreamingQuery` instances active on `this` context. .. note:: Evolving. diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index c3ceaea211d80..f224526a4cc74 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -839,10 +839,13 @@ def trigger(self, processingTime=None, once=None, continuous=None): .. note:: Evolving. :param processingTime: a processing time interval as a string, e.g. '5 seconds', '1 minute'. - Set a trigger that runs a query periodically based on the processing - time. Only one trigger can be set. + Set a trigger that runs a microbatch query periodically based on the + processing time. Only one trigger can be set. :param once: if set to True, set a trigger that processes only one batch of data in a streaming query then terminates the query. Only one trigger can be set. + :param continuous: a time interval as a string, e.g. '5 seconds', '1 minute'. + Set a trigger that runs a continuous query with a given checkpoint + interval. Only one trigger can be set. >>> # trigger the query for execution every 5 seconds >>> writer = sdf.writeStream.trigger(processingTime='5 seconds') @@ -1062,7 +1065,7 @@ def foreachBatch(self, func): >>> def func(batch_df, batch_id): ... batch_df.collect() ... - >>> writer = sdf.writeStream.foreach(func) + >>> writer = sdf.writeStream.foreachBatch(func) """ from pyspark.java_gateway import ensure_callback_server_started From 28ccd31aee7e1d01ecf50cf5adbd88f0af79fdce Mon Sep 17 00:00:00 2001 From: dengziming Date: Fri, 1 Nov 2019 22:46:34 -0700 Subject: [PATCH 0495/1387] [SPARK-29611][WEBUI] Sort Kafka metadata by the number of messages ### What changes were proposed in this pull request? Sort metadata by the number of messages in each Kafka partition ### Why are the changes needed? help to find the data skewness problem. ### Does this PR introduce any user-facing change? Yes, add a column count to the metadata and sort by count ![image](https://user-images.githubusercontent.com/26023240/67617886-63e06800-f81a-11e9-8718-be3a0100952e.png) If you set `minPartitions` configurations with structure structured-streaming which doesn't have the Streaming page, my code changes in `DirectKafkaInputDStream` won't affect the WEB UI page just as it shows in the follow image ![image](https://user-images.githubusercontent.com/26023240/68020762-79520800-fcda-11e9-96cd-f0c64a36f505.png) ### How was this patch tested? Manual test Closes #26266 from dengziming/feature_ui_optimize. Lead-authored-by: dengziming Co-authored-by: dengziming Signed-off-by: Dongjoon Hyun --- .../spark/streaming/kafka010/DirectKafkaInputDStream.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index 88d6d0eea5367..a449a8bb7213e 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -237,9 +237,10 @@ private[spark] class DirectKafkaInputDStream[K, V]( val description = offsetRanges.filter { offsetRange => // Don't display empty ranges. offsetRange.fromOffset != offsetRange.untilOffset - }.map { offsetRange => + }.toSeq.sortBy(-_.count()).map { offsetRange => s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + - s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" + s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}\t" + + s"count: ${offsetRange.count()}" }.mkString("\n") // Copy offsetRanges to immutable.List to prevent from being modified by the user val metadata = Map( From 31ae446e9c0be4dff2b75e510a2e1b65773d757e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 2 Nov 2019 21:35:56 +0800 Subject: [PATCH 0496/1387] [SPARK-29623][SQL] do not allow multiple unit TO unit statements in interval literal syntax ### What changes were proposed in this pull request? re-arrange the parser rules to make it clear that multiple unit TO unit statement like `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH` is not allowed. ### Why are the changes needed? This is definitely an accident that we support such a weird syntax in the past. It's not supported by any other DBs and I can't think of any use case of it. Also no test covers this syntax in the current codebase. ### Does this PR introduce any user-facing change? Yes, and a migration guide item is added. ### How was this patch tested? new tests. Closes #26285 from cloud-fan/syntax. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 6 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 22 +- .../sql/catalyst/parser/AstBuilder.scala | 150 +++++----- .../parser/ExpressionParserSuite.scala | 7 +- .../resources/sql-tests/inputs/literals.sql | 23 ++ .../sql-tests/results/literals.sql.out | 278 +++++++++++++++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 11 - 7 files changed, 406 insertions(+), 91 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index d03ca663e8e3f..a97a4b04ded6d 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -9,9 +9,9 @@ license: | 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. @@ -218,6 +218,8 @@ license: | - Since Spark 3.0, the `size` function returns `NULL` for the `NULL` input. In Spark version 2.4 and earlier, this function gives `-1` for the same input. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.sizeOfNull` to `true`. + - Since Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 96d1e42ffafe5..a9e3ca6549863 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -80,7 +80,7 @@ singleTableSchema ; singleInterval - : INTERVAL? (intervalValue intervalUnit)+ EOF + : INTERVAL? multiUnitsInterval EOF ; statement @@ -759,12 +759,24 @@ booleanValue ; interval - : {ansi}? INTERVAL? intervalField+ - | {!ansi}? INTERVAL intervalField* + : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? + | {ansi}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; -intervalField - : value=intervalValue unit=intervalUnit (TO to=intervalUnit)? +errorCapturingMultiUnitsInterval + : multiUnitsInterval unitToUnitInterval? + ; + +multiUnitsInterval + : (intervalValue intervalUnit)+ + ; + +errorCapturingUnitToUnitInterval + : body=unitToUnitInterval (error1=multiUnitsInterval | error2=unitToUnitInterval)? + ; + +unitToUnitInterval + : value=intervalValue from=intervalUnit TO to=intervalUnit ; intervalValue diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 838fc4d84a5d2..3de13abf54561 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -102,20 +102,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } override def visitSingleInterval(ctx: SingleIntervalContext): CalendarInterval = { - withOrigin(ctx) { - val units = ctx.intervalUnit().asScala.map { - u => normalizeInternalUnit(u.getText.toLowerCase(Locale.ROOT)) - }.toArray - val values = ctx.intervalValue().asScala.map(getIntervalValue).toArray - try { - IntervalUtils.fromUnitStrings(units, values) - } catch { - case i: IllegalArgumentException => - val e = new ParseException(i.getMessage, ctx) - e.setStackTrace(i.getStackTrace) - throw e - } - } + withOrigin(ctx)(visitMultiUnitsInterval(ctx.multiUnitsInterval)) } /* ******************************************************************************************** @@ -1940,71 +1927,102 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Create a [[CalendarInterval]] literal expression. An interval expression can contain multiple - * unit value pairs, for instance: interval 2 months 2 days. + * Create a [[CalendarInterval]] literal expression. Two syntaxes are supported: + * - multiple unit value pairs, for instance: interval 2 months 2 days. + * - from-to unit, for instance: interval '1-2' year to month. */ override def visitInterval(ctx: IntervalContext): Literal = withOrigin(ctx) { - val intervals = ctx.intervalField.asScala.map(visitIntervalField) - validate(intervals.nonEmpty, "at least one time unit should be given for interval literal", ctx) - Literal(intervals.reduce(_.add(_))) + if (ctx.errorCapturingMultiUnitsInterval != null) { + val innerCtx = ctx.errorCapturingMultiUnitsInterval + if (innerCtx.unitToUnitInterval != null) { + throw new ParseException( + "Can only have a single from-to unit in the interval literal syntax", + innerCtx.unitToUnitInterval) + } + Literal(visitMultiUnitsInterval(innerCtx.multiUnitsInterval), CalendarIntervalType) + } else if (ctx.errorCapturingUnitToUnitInterval != null) { + val innerCtx = ctx.errorCapturingUnitToUnitInterval + if (innerCtx.error1 != null || innerCtx.error2 != null) { + val errorCtx = if (innerCtx.error1 != null) innerCtx.error1 else innerCtx.error2 + throw new ParseException( + "Can only have a single from-to unit in the interval literal syntax", + errorCtx) + } + Literal(visitUnitToUnitInterval(innerCtx.body), CalendarIntervalType) + } else { + throw new ParseException("at least one time unit should be given for interval literal", ctx) + } } /** - * Create a [[CalendarInterval]] for a unit value pair. Two unit configuration types are - * supported: - * - Single unit. - * - From-To unit ('YEAR TO MONTH', 'DAY TO HOUR', 'DAY TO MINUTE', 'DAY TO SECOND', - * 'HOUR TO MINUTE', 'HOUR TO SECOND' and 'MINUTE TO SECOND' are supported). + * Creates a [[CalendarInterval]] with multiple unit value pairs, e.g. 1 YEAR 2 DAYS. */ - override def visitIntervalField(ctx: IntervalFieldContext): CalendarInterval = withOrigin(ctx) { - import ctx._ - val s = getIntervalValue(value) - try { - val unitText = unit.getText.toLowerCase(Locale.ROOT) - val interval = (unitText, Option(to).map(_.getText.toLowerCase(Locale.ROOT))) match { - case (u, None) => - IntervalUtils.fromUnitStrings(Array(normalizeInternalUnit(u)), Array(s)) - case ("year", Some("month")) => - IntervalUtils.fromYearMonthString(s) - case ("day", Some("hour")) => - IntervalUtils.fromDayTimeString(s, "day", "hour") - case ("day", Some("minute")) => - IntervalUtils.fromDayTimeString(s, "day", "minute") - case ("day", Some("second")) => - IntervalUtils.fromDayTimeString(s, "day", "second") - case ("hour", Some("minute")) => - IntervalUtils.fromDayTimeString(s, "hour", "minute") - case ("hour", Some("second")) => - IntervalUtils.fromDayTimeString(s, "hour", "second") - case ("minute", Some("second")) => - IntervalUtils.fromDayTimeString(s, "minute", "second") - case (from, Some(t)) => - throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx) + override def visitMultiUnitsInterval(ctx: MultiUnitsIntervalContext): CalendarInterval = { + withOrigin(ctx) { + val units = ctx.intervalUnit().asScala.map { unit => + val u = unit.getText.toLowerCase(Locale.ROOT) + // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... + if (u.endsWith("s")) u.substring(0, u.length - 1) else u + }.toArray + + val values = ctx.intervalValue().asScala.map { value => + if (value.STRING() != null) { + string(value.STRING()) + } else { + value.getText + } + }.toArray + + try { + IntervalUtils.fromUnitStrings(units, values) + } catch { + case i: IllegalArgumentException => + val e = new ParseException(i.getMessage, ctx) + e.setStackTrace(i.getStackTrace) + throw e } - validate(interval != null, "No interval can be constructed", ctx) - interval - } catch { - // Handle Exceptions thrown by CalendarInterval - case e: IllegalArgumentException => - val pe = new ParseException(e.getMessage, ctx) - pe.setStackTrace(e.getStackTrace) - throw pe } } - private def getIntervalValue(value: IntervalValueContext): String = { - if (value.STRING() != null) { - string(value.STRING()) - } else { - value.getText + /** + * Creates a [[CalendarInterval]] with from-to unit, e.g. '2-1' YEAR TO MONTH. + */ + override def visitUnitToUnitInterval(ctx: UnitToUnitIntervalContext): CalendarInterval = { + withOrigin(ctx) { + val value = Option(ctx.intervalValue.STRING).map(string).getOrElse { + throw new ParseException("The value of from-to unit must be a string", ctx.intervalValue) + } + try { + val from = ctx.from.getText.toLowerCase(Locale.ROOT) + val to = ctx.to.getText.toLowerCase(Locale.ROOT) + (from, to) match { + case ("year", "month") => + IntervalUtils.fromYearMonthString(value) + case ("day", "hour") => + IntervalUtils.fromDayTimeString(value, "day", "hour") + case ("day", "minute") => + IntervalUtils.fromDayTimeString(value, "day", "minute") + case ("day", "second") => + IntervalUtils.fromDayTimeString(value, "day", "second") + case ("hour", "minute") => + IntervalUtils.fromDayTimeString(value, "hour", "minute") + case ("hour", "second") => + IntervalUtils.fromDayTimeString(value, "hour", "second") + case ("minute", "second") => + IntervalUtils.fromDayTimeString(value, "minute", "second") + case _ => + throw new ParseException(s"Intervals FROM $from TO $to are not supported.", ctx) + } + } catch { + // Handle Exceptions thrown by CalendarInterval + case e: IllegalArgumentException => + val pe = new ParseException(e.getMessage, ctx) + pe.setStackTrace(e.getStackTrace) + throw pe + } } } - // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... - private def normalizeInternalUnit(s: String): String = { - if (s.endsWith("s")) s.substring(0, s.length - 1) else s - } - /* ******************************************************************************************** * DataType parsing * ******************************************************************************************** */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index d675c7c483a04..859a20e4cb9b3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -644,7 +644,7 @@ class ExpressionParserSuite extends AnalysisTest { // Non Existing unit intercept("interval 10 nanoseconds", - "no viable alternative at input 'interval 10 nanoseconds'") + "no viable alternative at input '10 nanoseconds'") // Year-Month intervals. val yearMonthValues = Seq("123-10", "496-0", "-2-3", "-123-0") @@ -679,16 +679,13 @@ class ExpressionParserSuite extends AnalysisTest { } // Unknown FROM TO intervals - intercept("interval 10 month to second", + intercept("interval '10' month to second", "Intervals FROM month TO second are not supported.") // Composed intervals. checkIntervals( "3 months 4 days 22 seconds 1 millisecond", Literal(new CalendarInterval(3, 4, 22001000L))) - checkIntervals( - "3 years '-1-10' year to month 3 weeks '1 0:0:2' day to second", - Literal(new CalendarInterval(14, 22, 2 * CalendarInterval.MICROS_PER_SECOND))) } test("SPARK-23264 Interval Compatibility tests") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 389b9cc53e02e..3c3be2913abc1 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -130,3 +130,26 @@ select interval '3 year 1 hour'; select integer '7'; select integer'7'; select integer '2147483648'; + +-- malformed interval literal +select interval; +select interval 1 fake_unit; +select interval 1 year to month; +select interval '1' year to second; +select interval '10-9' year to month '2-1' year to month; +select interval '10-9' year to month '12:11:10' hour to second; +select interval '1 15:11' day to minute '12:11:10' hour to second; +select interval 1 year '2-1' year to month; +select interval 1 year '12:11:10' hour to second; +select interval '10-9' year to month '1' year; +select interval '12:11:10' hour to second '1' year; +-- malformed interval literal with ansi mode +SET spark.sql.ansi.enabled=true; +select interval; +select interval 1 fake_unit; +select interval 1 year to month; +select 1 year to month; +select interval '1' year to second; +select '1' year to second; +select interval 1 year '2-1' year to month; +select 1 year '2-1' year to month; diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index b7a0dcaa1ea0a..c5981414a2b7d 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 62 +-- Number of queries: 82 -- !query 0 @@ -447,7 +447,7 @@ struct<> -- !query 49 output org.apache.spark.sql.catalyst.parser.ParseException -no viable alternative at input 'interval 10 nanoseconds'(line 1, pos 19) +no viable alternative at input '10 nanoseconds'(line 1, pos 19) == SQL == select interval 10 nanoseconds @@ -572,3 +572,277 @@ Cannot parse the Int value: 2147483648, java.lang.NumberFormatException: For inp == SQL == select integer '2147483648' -------^^^ + + +-- !query 62 +select interval +-- !query 62 schema +struct<> +-- !query 62 output +org.apache.spark.sql.catalyst.parser.ParseException + +at least one time unit should be given for interval literal(line 1, pos 7) + +== SQL == +select interval +-------^^^ + + +-- !query 63 +select interval 1 fake_unit +-- !query 63 schema +struct<> +-- !query 63 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input '1 fake_unit'(line 1, pos 18) + +== SQL == +select interval 1 fake_unit +------------------^^^ + + +-- !query 64 +select interval 1 year to month +-- !query 64 schema +struct<> +-- !query 64 output +org.apache.spark.sql.catalyst.parser.ParseException + +The value of from-to unit must be a string(line 1, pos 16) + +== SQL == +select interval 1 year to month +----------------^^^ + + +-- !query 65 +select interval '1' year to second +-- !query 65 schema +struct<> +-- !query 65 output +org.apache.spark.sql.catalyst.parser.ParseException + +Intervals FROM year TO second are not supported.(line 1, pos 16) + +== SQL == +select interval '1' year to second +----------------^^^ + + +-- !query 66 +select interval '10-9' year to month '2-1' year to month +-- !query 66 schema +struct<> +-- !query 66 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '2-1' year to month +-------------------------------------^^^ + + +-- !query 67 +select interval '10-9' year to month '12:11:10' hour to second +-- !query 67 schema +struct<> +-- !query 67 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '12:11:10' hour to second +-------------------------------------^^^ + + +-- !query 68 +select interval '1 15:11' day to minute '12:11:10' hour to second +-- !query 68 schema +struct<> +-- !query 68 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) + +== SQL == +select interval '1 15:11' day to minute '12:11:10' hour to second +----------------------------------------^^^ + + +-- !query 69 +select interval 1 year '2-1' year to month +-- !query 69 schema +struct<> +-- !query 69 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) + +== SQL == +select interval 1 year '2-1' year to month +-----------------------^^^ + + +-- !query 70 +select interval 1 year '12:11:10' hour to second +-- !query 70 schema +struct<> +-- !query 70 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) + +== SQL == +select interval 1 year '12:11:10' hour to second +-----------------------^^^ + + +-- !query 71 +select interval '10-9' year to month '1' year +-- !query 71 schema +struct<> +-- !query 71 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '1' year +-------------------------------------^^^ + + +-- !query 72 +select interval '12:11:10' hour to second '1' year +-- !query 72 schema +struct<> +-- !query 72 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) + +== SQL == +select interval '12:11:10' hour to second '1' year +------------------------------------------^^^ + + +-- !query 73 +SET spark.sql.ansi.enabled=true +-- !query 73 schema +struct +-- !query 73 output +spark.sql.ansi.enabled true + + +-- !query 74 +select interval +-- !query 74 schema +struct<> +-- !query 74 output +org.apache.spark.sql.catalyst.parser.ParseException + +at least one time unit should be given for interval literal(line 1, pos 7) + +== SQL == +select interval +-------^^^ + + +-- !query 75 +select interval 1 fake_unit +-- !query 75 schema +struct<> +-- !query 75 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input '1 fake_unit'(line 1, pos 18) + +== SQL == +select interval 1 fake_unit +------------------^^^ + + +-- !query 76 +select interval 1 year to month +-- !query 76 schema +struct<> +-- !query 76 output +org.apache.spark.sql.catalyst.parser.ParseException + +The value of from-to unit must be a string(line 1, pos 16) + +== SQL == +select interval 1 year to month +----------------^^^ + + +-- !query 77 +select 1 year to month +-- !query 77 schema +struct<> +-- !query 77 output +org.apache.spark.sql.catalyst.parser.ParseException + +The value of from-to unit must be a string(line 1, pos 7) + +== SQL == +select 1 year to month +-------^^^ + + +-- !query 78 +select interval '1' year to second +-- !query 78 schema +struct<> +-- !query 78 output +org.apache.spark.sql.catalyst.parser.ParseException + +Intervals FROM year TO second are not supported.(line 1, pos 16) + +== SQL == +select interval '1' year to second +----------------^^^ + + +-- !query 79 +select '1' year to second +-- !query 79 schema +struct<> +-- !query 79 output +org.apache.spark.sql.catalyst.parser.ParseException + +Intervals FROM year TO second are not supported.(line 1, pos 7) + +== SQL == +select '1' year to second +-------^^^ + + +-- !query 80 +select interval 1 year '2-1' year to month +-- !query 80 schema +struct<> +-- !query 80 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) + +== SQL == +select interval 1 year '2-1' year to month +-----------------------^^^ + + +-- !query 81 +select 1 year '2-1' year to month +-- !query 81 schema +struct<> +-- !query 81 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 14) + +== SQL == +select 1 year '2-1' year to month +--------------^^^ 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 941304cb2553b..cbf4d1a2ad768 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 @@ -1564,17 +1564,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { } e.message.contains("Cannot save interval data type into external storage") }) - - val e1 = intercept[AnalysisException] { - sql("select interval") - } - assert(e1.message.contains("at least one time unit should be given for interval literal")) - - // Currently we don't yet support nanosecond - val e2 = intercept[AnalysisException] { - sql("select interval 23 nanosecond") - } - assert(e2.message.contains("no viable alternative at input 'interval 23 nanosecond'")) } test("SPARK-8945: add and subtract expressions for interval type") { From be022d9aeeb6b39f7b51427d848600e15a100c30 Mon Sep 17 00:00:00 2001 From: Eric Meisel Date: Sat, 2 Nov 2019 16:42:49 -0500 Subject: [PATCH 0497/1387] [SPARK-29677][DSTREAMS] amazon-kinesis-client 1.12.0 ### What changes were proposed in this pull request? Upgrading the amazon-kinesis-client dependency to 1.12.0. ### Why are the changes needed? The current amazon-kinesis-client version is 1.8.10. This version depends on the use of `describeStream`, which has a hard limit on an AWS account (10 reqs / second). Versions 1.9.0 and up leverage `listShards`, which has no such limit. For large customers, this can be a major problem. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests Closes #26333 from etspaceman/kclUpgrade. Authored-by: Eric Meisel Signed-off-by: Sean Owen --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b5c34790f1d3f..865a94f7d36a5 100644 --- a/pom.xml +++ b/pom.xml @@ -151,7 +151,7 @@ 3.2.6 1.8.2 hadoop2 - 1.8.10 + 1.12.0 1.11.271 From 91d990162f13acde546d01e1163ed3e898cbf9a7 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 2 Nov 2019 21:37:40 -0700 Subject: [PATCH 0498/1387] [SPARK-24152][R][TESTS] Disable check-cran from run-tests.sh ### What changes were proposed in this pull request? This PR aims to remove `check-cran` from `run-tests.sh`. We had better add an independent Jenkins job to run `check-cran`. ### Why are the changes needed? CRAN instability has been a blocker for our daily dev process. The following simple check causes consecutive failures in 4 of 9 Jenkins jobs + PR builder. ``` * checking CRAN incoming feasibility ...Error in .check_package_CRAN_incoming(pkgdir) : dims [product 24] do not match the length of object [0] ``` - spark-branch-2.4-test-sbt-hadoop-2.6 - spark-branch-2.4-test-sbt-hadoop-2.7 - spark-master-test-sbt-hadoop-2.7 - spark-master-test-sbt-hadoop-3.2 - PRBuilder ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Currently, PR builder is failing due to the above issue. This PR should pass the Jenkins. Closes #26375 from dongjoon-hyun/SPARK-24152. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- R/run-tests.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/R/run-tests.sh b/R/run-tests.sh index 86bd8aad5f113..27537bf3f7c2f 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -32,8 +32,10 @@ NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" CRAN_CHECK_LOG_FILE=$FWDIR/cran-check.out rm -f $CRAN_CHECK_LOG_FILE -NO_TESTS=1 NO_MANUAL=1 $FWDIR/check-cran.sh 2>&1 | tee -a $CRAN_CHECK_LOG_FILE -FAILED=$((PIPESTATUS[0]||$FAILED)) +# SPARK-24152 We will add this test back as a separate Jenkins job +# NO_TESTS=1 NO_MANUAL=1 $FWDIR/check-cran.sh 2>&1 | tee -a $CRAN_CHECK_LOG_FILE +# FAILED=$((PIPESTATUS[0]||$FAILED)) +touch $CRAN_CHECK_LOG_FILE NUM_CRAN_WARNING="$(grep -c WARNING$ $CRAN_CHECK_LOG_FILE)" NUM_CRAN_ERROR="$(grep -c ERROR$ $CRAN_CHECK_LOG_FILE)" From 1ac6bd9f7929e7b907a0c2262e42b4d8a24ebbaa Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 3 Nov 2019 10:42:38 -0800 Subject: [PATCH 0499/1387] [SPARK-29729][BUILD] Upgrade ASM to 7.2 ### What changes were proposed in this pull request? This PR aims to upgrade ASM to 7.2. - https://issues.apache.org/jira/browse/XBEAN-322 (Upgrade to ASM 7.2) - https://asm.ow2.io/versions.html ### Why are the changes needed? This will bring the following patches. - 317875: Infinite loop when parsing invalid method descriptor - 317873: Add support for RET instruction in AdviceAdapter - 317872: Throw an exception if visitFrame used incorrectly - add support for Java 14 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing UTs. Closes #26373 from dongjoon-hyun/SPARK-29729. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 2 +- dev/deps/spark-deps-hadoop-3.2 | 2 +- pom.xml | 6 +++--- project/plugins.sbt | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index edf4f026a32dc..41c23021ea74f 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -200,7 +200,7 @@ stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.8.3.jar validation-api-2.0.1.Final.jar -xbean-asm7-shaded-4.14.jar +xbean-asm7-shaded-4.15.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar xz-1.5.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 5f32e4686dcee..5ba4add05b906 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -235,7 +235,7 @@ univocity-parsers-2.8.3.jar validation-api-2.0.1.Final.jar velocity-1.5.jar woodstox-core-5.0.3.jar -xbean-asm7-shaded-4.14.jar +xbean-asm7-shaded-4.15.jar xz-1.5.jar zjsonpatch-0.3.0.jar zookeeper-3.4.14.jar diff --git a/pom.xml b/pom.xml index 865a94f7d36a5..73d5f82d84098 100644 --- a/pom.xml +++ b/pom.xml @@ -333,7 +333,7 @@ org.apache.xbean xbean-asm7-shaded - 4.14 + 4.15 - Shuffle Write - Logs - Thread Dump + Logs + Thread Dump diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js index 035d72f8956ff..25dec9d3788ba 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js +++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js @@ -513,7 +513,7 @@ function addTooltipsForRDDs(svgContainer) { if (tooltipText) { node.select("circle") .attr("data-toggle", "tooltip") - .attr("data-placement", "bottom") + .attr("data-placement", "top") .attr("data-html", "true") // to interpret line break, tooltipText is showing title .attr("title", tooltipText); } diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index f01d030c73a41..c05651681be42 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -297,26 +297,26 @@ $(document).ready(function () { ""); $('#scheduler_delay').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Scheduler delay includes time to ship the task from the scheduler to the executor, and time to send " + "the task result from the executor to the scheduler. If scheduler delay is large, consider decreasing the size of tasks or decreasing the size of task results."); $('#task_deserialization_time').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Time spent deserializing the task closure on the executor, including the time to read the broadcasted task."); $('#shuffle_read_blocked_time').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Time that the task spent blocked waiting for shuffle data to be read from remote machines."); $('#shuffle_remote_reads').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Total shuffle bytes read from remote executors. This is a subset of the shuffle read bytes; the remaining shuffle data is read locally. "); $('#result_serialization_time').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Time spent serializing the task result on the executor before sending it back to the driver."); $('#getting_result_time').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Time that the driver spends fetching task results from workers. If this is large, consider decreasing the amount of data returned from each task."); $('#peak_execution_memory').attr("data-toggle", "tooltip") - .attr("data-placement", "right") + .attr("data-placement", "top") .attr("title", "Execution memory refers to the memory used by internal data structures created during " + "shuffles, aggregations and joins when Tungsten is enabled. The value of this accumulator " + "should be approximately the sum of the peak sizes across all such data structures created " + diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index c7e73bcc13c5f..071b79135d641 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -73,7 +73,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
  • + data-placement="top"> Executor Limit: { if (app.executorLimit == Int.MaxValue) "Unlimited" else app.executorLimit diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 6dbe63b564e69..143303df0d10e 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -227,7 +227,7 @@ private[spark] object UIUtils extends Logging { {tab.name}
  • } - val helpButton: Seq[Node] = helpText.map(tooltip(_, "bottom")).getOrElse(Seq.empty) + val helpButton: Seq[Node] = helpText.map(tooltip(_, "top")).getOrElse(Seq.empty) @@ -428,7 +428,7 @@ private[spark] object UIUtils extends Logging { class="expand-dag-viz" onclick={s"toggleDagViz($forJob);"}> + data-placement="top"> DAG Visualization diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 11fcbf1c29c05..91e9caeae05db 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -123,7 +123,7 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We | 'group': 'executors', | 'start': new Date(${e.addTime.getTime()}), | 'content': '
    Executor ${e.id} added
    ' @@ -139,7 +139,7 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We | 'group': 'executors', | 'start': new Date(${removeTime.getTime()}), | 'content': '
    - + Event Timeline ++ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index f7aca507d6f93..12f1aa25e8d2a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -104,7 +104,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP | 'group': 'executors', | 'start': new Date(${e.addTime.getTime()}), | 'content': '
    Executor ${e.id} added
    ' @@ -120,7 +120,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP | 'group': 'executors', | 'start': new Date(${removeTime.getTime()}), | 'content': '
    - + Event Timeline ++ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index e24b2f2ec36db..09a215ba9f03d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -179,18 +179,18 @@ private[ui] class StagePagedTable( // stageHeadersAndCssClasses has three parts: header title, tooltip information, and sortable. // The tooltip information could be None, which indicates it does not have a tooltip. // Otherwise, it has two parts: tooltip text, and position (true for left, false for default). - val stageHeadersAndCssClasses: Seq[(String, Option[(String, Boolean)], Boolean)] = - Seq(("Stage Id", None, true)) ++ - {if (isFairScheduler) {Seq(("Pool Name", None, true))} else Seq.empty} ++ + val stageHeadersAndCssClasses: Seq[(String, String, Boolean)] = + Seq(("Stage Id", null, true)) ++ + {if (isFairScheduler) {Seq(("Pool Name", null, true))} else Seq.empty} ++ Seq( - ("Description", None, true), ("Submitted", None, true), ("Duration", None, true), - ("Tasks: Succeeded/Total", None, false), - ("Input", Some((ToolTips.INPUT, false)), true), - ("Output", Some((ToolTips.OUTPUT, false)), true), - ("Shuffle Read", Some((ToolTips.SHUFFLE_READ, false)), true), - ("Shuffle Write", Some((ToolTips.SHUFFLE_WRITE, true)), true) + ("Description", null, true), ("Submitted", null, true), ("Duration", null, true), + ("Tasks: Succeeded/Total", null, false), + ("Input", ToolTips.INPUT, true), + ("Output", ToolTips.OUTPUT, true), + ("Shuffle Read", ToolTips.SHUFFLE_READ, true), + ("Shuffle Write", ToolTips.SHUFFLE_WRITE, true) ) ++ - {if (isFailedStage) {Seq(("Failure Reason", None, false))} else Seq.empty} + {if (isFailedStage) {Seq(("Failure Reason", null, false))} else Seq.empty} if (!stageHeadersAndCssClasses.filter(_._3).map(_._1).contains(sortColumn)) { throw new IllegalArgumentException(s"Unknown column: $sortColumn") @@ -198,22 +198,13 @@ private[ui] class StagePagedTable( val headerRow: Seq[Node] = { stageHeadersAndCssClasses.map { case (header, tooltip, sortable) => - val headerSpan = tooltip.map { case (title, left) => - if (left) { - /* Place the shuffle write tooltip on the left (rather than the default position - of on top) because the shuffle write column is the last column on the right side and - the tooltip is wider than the column, so it doesn't fit on top. */ - + val headerSpan = if (null != tooltip && !tooltip.isEmpty) { + {header} - } else { - - {header} - - } - }.getOrElse( + } else { {header} - ) + } if (header == sortColumn) { val headerLink = Unparsed( diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js index 46d3fbc8c3cb4..2329fd262ddfb 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js @@ -61,7 +61,7 @@ function setupTooltipForSparkPlanNode(nodeId) { .on('mouseover', function(d) { var domNode = d3.select(this).node(); $(domNode).tooltip({ - title: nodeTooltip, trigger: "manual", container: "body", placement: "right" + title: nodeTooltip, trigger: "manual", container: "body", placement: "top" }); $(domNode).tooltip("show"); }) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index d16611f412034..f28fc21538066 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -321,7 +321,7 @@ private[ui] class StreamingPage(parent: StreamingTab) if (hasStream) { - + Input Rate @@ -351,7 +351,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Scheduling Delay {SparkUIUtils.tooltip("Time taken by Streaming scheduler to submit jobs of a batch", "right")}
    +
    Scheduling Delay {SparkUIUtils.tooltip("Time taken by Streaming scheduler to submit jobs of a batch", "top")}
    Avg: {schedulingDelay.formattedAvg}
    @@ -361,7 +361,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Processing Time {SparkUIUtils.tooltip("Time taken to process all jobs of a batch", "right")}
    +
    Processing Time {SparkUIUtils.tooltip("Time taken to process all jobs of a batch", "top")}
    Avg: {processingTime.formattedAvg}
    @@ -371,7 +371,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
    -
    Total Delay {SparkUIUtils.tooltip("Total time taken to handle a batch", "right")}
    +
    Total Delay {SparkUIUtils.tooltip("Total time taken to handle a batch", "top")}
    Avg: {totalDelay.formattedAvg}
    From 56a0b5421e41f46a65375c0e5ef9993e9502f93e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 13 Nov 2019 09:52:40 +0900 Subject: [PATCH 0591/1387] [SPARK-29399][CORE] Remove old ExecutorPlugin interface SPARK-29397 added new interfaces for creating driver and executor plugins. These were added in a new, more isolated package that does not pollute the main o.a.s package. The old interface is now redundant. Since it's a DeveloperApi and we're about to have a new major release, let's remove it instead of carrying more baggage forward. Closes #26390 from vanzin/SPARK-29399. Authored-by: Marcelo Vanzin Signed-off-by: HyukjinKwon --- .../java/org/apache/spark/ExecutorPlugin.java | 60 ------ .../apache/spark/ExecutorPluginContext.java | 50 ----- .../apache/spark/api/plugin/SparkPlugin.java | 2 +- .../scala/org/apache/spark/SparkConf.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 37 ---- .../spark/executor/ExecutorPluginSource.scala | 30 --- .../spark/internal/config/package.scala | 10 - .../org/apache/spark/ExecutorPluginSuite.java | 179 ------------------ .../plugin/PluginContainerSuite.scala | 2 +- docs/core-migration-guide.md | 17 +- project/MimaExcludes.scala | 8 +- 11 files changed, 20 insertions(+), 379 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/ExecutorPlugin.java delete mode 100644 core/src/main/java/org/apache/spark/ExecutorPluginContext.java delete mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala delete mode 100644 core/src/test/java/org/apache/spark/ExecutorPluginSuite.java diff --git a/core/src/main/java/org/apache/spark/ExecutorPlugin.java b/core/src/main/java/org/apache/spark/ExecutorPlugin.java deleted file mode 100644 index b25c46266247e..0000000000000 --- a/core/src/main/java/org/apache/spark/ExecutorPlugin.java +++ /dev/null @@ -1,60 +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; - -import org.apache.spark.annotation.DeveloperApi; - -/** - * A plugin which can be automatically instantiated within each Spark executor. Users can specify - * plugins which should be created with the "spark.executor.plugins" configuration. An instance - * of each plugin will be created for every executor, including those created by dynamic allocation, - * before the executor starts running any tasks. - * - * The specific api exposed to the end users still considered to be very unstable. We will - * hopefully be able to keep compatibility by providing default implementations for any methods - * added, but make no guarantees this will always be possible across all Spark releases. - * - * Spark does nothing to verify the plugin is doing legitimate things, or to manage the resources - * it uses. A plugin acquires the same privileges as the user running the task. A bad plugin - * could also interfere with task execution and make the executor fail in unexpected ways. - */ -@DeveloperApi -public interface ExecutorPlugin { - - /** - * Initialize the executor plugin. - * - *

    Each executor will, during its initialization, invoke this method on each - * plugin provided in the spark.executor.plugins configuration. The Spark executor - * will wait on the completion of the execution of the init method.

    - * - *

    Plugins should create threads in their implementation of this method for - * any polling, blocking, or intensive computation.

    - * - * @param pluginContext Context information for the executor where the plugin is running. - */ - default void init(ExecutorPluginContext pluginContext) {} - - /** - * Clean up and terminate this plugin. - * - *

    This function is called during the executor shutdown phase. The executor - * will wait for the plugin to terminate before continuing its own shutdown.

    - */ - default void shutdown() {} -} diff --git a/core/src/main/java/org/apache/spark/ExecutorPluginContext.java b/core/src/main/java/org/apache/spark/ExecutorPluginContext.java deleted file mode 100644 index 8f018732b8217..0000000000000 --- a/core/src/main/java/org/apache/spark/ExecutorPluginContext.java +++ /dev/null @@ -1,50 +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; - -import com.codahale.metrics.MetricRegistry; -import org.apache.spark.annotation.DeveloperApi; -import org.apache.spark.annotation.Private; - -/** - * Encapsulates information about the executor when initializing {@link ExecutorPlugin} instances. - */ -@DeveloperApi -public class ExecutorPluginContext { - - public final MetricRegistry metricRegistry; - public final SparkConf sparkConf; - public final String executorId; - public final String executorHostName; - public final boolean isLocal; - - @Private - public ExecutorPluginContext( - MetricRegistry registry, - SparkConf conf, - String id, - String hostName, - boolean local) { - metricRegistry = registry; - sparkConf = conf; - executorId = id; - executorHostName = hostName; - isLocal = local; - } - -} diff --git a/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java b/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java index a500f5d2188f0..21ddae37d8a0d 100644 --- a/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java +++ b/core/src/main/java/org/apache/spark/api/plugin/SparkPlugin.java @@ -24,7 +24,7 @@ * A plugin that can be dynamically loaded into a Spark application. *

    * Plugins can be loaded by adding the plugin's class name to the appropriate Spark configuration. - * Check the Spark configuration documentation for details. + * Check the Spark monitoring guide for details. *

    * Plugins have two optional components: a driver-side component, of which a single instance is * created per application, inside the Spark driver. And an executor-side component, of which one diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 3a2eaae092e8d..0e0291d2407d1 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -619,7 +619,9 @@ private[spark] object SparkConf extends Logging { "Not used anymore. Please use spark.shuffle.service.index.cache.size"), DeprecatedConfig("spark.yarn.credentials.file.retention.count", "2.4.0", "Not used anymore."), DeprecatedConfig("spark.yarn.credentials.file.retention.days", "2.4.0", "Not used anymore."), - DeprecatedConfig("spark.yarn.services", "3.0.0", "Feature no longer available.") + DeprecatedConfig("spark.yarn.services", "3.0.0", "Feature no longer available."), + DeprecatedConfig("spark.executor.plugins", "3.0.0", + "Feature replaced with new plugin API. See Monitoring documentation.") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 0f595d095a229..8cd98e47b8a4f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -137,35 +137,6 @@ private[spark] class Executor( // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. env.serializerManager.setDefaultClassLoader(replClassLoader) - private val executorPlugins: Seq[ExecutorPlugin] = { - val pluginNames = conf.get(EXECUTOR_PLUGINS) - if (pluginNames.nonEmpty) { - logInfo(s"Initializing the following plugins: ${pluginNames.mkString(", ")}") - - // Plugins need to load using a class loader that includes the executor's user classpath - val pluginList: Seq[ExecutorPlugin] = - Utils.withContextClassLoader(replClassLoader) { - val plugins = Utils.loadExtensions(classOf[ExecutorPlugin], pluginNames, conf) - plugins.foreach { plugin => - val pluginSource = new ExecutorPluginSource(plugin.getClass().getSimpleName()) - val pluginContext = new ExecutorPluginContext(pluginSource.metricRegistry, conf, - executorId, executorHostname, isLocal) - plugin.init(pluginContext) - logInfo("Successfully loaded plugin " + plugin.getClass().getCanonicalName()) - if (pluginSource.metricRegistry.getNames.size() > 0) { - env.metricsSystem.registerSource(pluginSource) - } - } - plugins - } - - logInfo("Finished initializing plugins") - pluginList - } else { - Nil - } - } - // Plugins need to load using a class loader that includes the executor's user classpath private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { PluginContainer(env) @@ -295,14 +266,6 @@ private[spark] class Executor( // Notify plugins that executor is shutting down so they can terminate cleanly Utils.withContextClassLoader(replClassLoader) { - executorPlugins.foreach { plugin => - try { - plugin.shutdown() - } catch { - case e: Exception => - logWarning("Plugin " + plugin.getClass().getCanonicalName() + " shutdown failed", e) - } - } plugins.foreach(_.shutdown()) } if (!isLocal) { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala deleted file mode 100644 index 5625e953c5e67..0000000000000 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorPluginSource.scala +++ /dev/null @@ -1,30 +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.executor - -import com.codahale.metrics.MetricRegistry - -import org.apache.spark.metrics.source.Source - -private[spark] -class ExecutorPluginSource(name: String) extends Source { - - override val metricRegistry = new MetricRegistry() - - override val sourceName = name -} 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 a5e2bbfca92b0..8e8e36dbda94e 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 @@ -1193,16 +1193,6 @@ package object config { .toSequence .createWithDefault(Nil) - private[spark] val EXECUTOR_PLUGINS = - ConfigBuilder("spark.executor.plugins") - .doc("Comma-separated list of class names for \"plugins\" implementing " + - "org.apache.spark.ExecutorPlugin. Plugins have the same privileges as any task " + - "in a Spark executor. They can also interfere with task execution and fail in " + - "unexpected ways. So be sure to only use this for trusted plugins.") - .stringConf - .toSequence - .createWithDefault(Nil) - private[spark] val CLEANER_PERIODIC_GC_INTERVAL = ConfigBuilder("spark.cleaner.periodicGC.interval") .timeConf(TimeUnit.SECONDS) diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java deleted file mode 100644 index ade13f02bde73..0000000000000 --- a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java +++ /dev/null @@ -1,179 +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; - -import com.codahale.metrics.Gauge; -import com.codahale.metrics.MetricRegistry; -import org.apache.spark.api.java.JavaSparkContext; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import static org.junit.Assert.*; - -public class ExecutorPluginSuite { - private static final String EXECUTOR_PLUGIN_CONF_NAME = "spark.executor.plugins"; - private static final String testBadPluginName = TestBadShutdownPlugin.class.getName(); - private static final String testPluginName = TestExecutorPlugin.class.getName(); - private static final String testSecondPluginName = TestSecondPlugin.class.getName(); - private static final String testMetricsPluginName = TestMetricsPlugin.class.getName(); - - // Static value modified by testing plugins to ensure plugins loaded correctly. - public static int numSuccessfulPlugins = 0; - - // Static value modified by testing plugins to verify plugins shut down properly. - public static int numSuccessfulTerminations = 0; - - // Static values modified by testing plugins to ensure metrics have been registered correctly. - public static MetricRegistry testMetricRegistry; - public static String gaugeName; - - private JavaSparkContext sc; - - @Before - public void setUp() { - sc = null; - numSuccessfulPlugins = 0; - numSuccessfulTerminations = 0; - } - - @After - public void tearDown() { - if (sc != null) { - sc.stop(); - sc = null; - } - } - - private SparkConf initializeSparkConf(String pluginNames) { - return new SparkConf() - .setMaster("local") - .setAppName("test") - .set(EXECUTOR_PLUGIN_CONF_NAME, pluginNames); - } - - @Test - public void testPluginClassDoesNotExist() { - SparkConf conf = initializeSparkConf("nonexistent.plugin"); - try { - sc = new JavaSparkContext(conf); - fail("No exception thrown for nonexistent plugin"); - } catch (Exception e) { - // We cannot catch ClassNotFoundException directly because Java doesn't think it'll be thrown - assertTrue(e.toString().startsWith("java.lang.ClassNotFoundException")); - } - } - - @Test - public void testAddPlugin() throws InterruptedException { - // Load the sample TestExecutorPlugin, which will change the value of numSuccessfulPlugins - SparkConf conf = initializeSparkConf(testPluginName); - sc = new JavaSparkContext(conf); - assertEquals(1, numSuccessfulPlugins); - sc.stop(); - sc = null; - assertEquals(1, numSuccessfulTerminations); - } - - @Test - public void testAddMultiplePlugins() throws InterruptedException { - // Load two plugins and verify they both execute. - SparkConf conf = initializeSparkConf(testPluginName + "," + testSecondPluginName); - sc = new JavaSparkContext(conf); - assertEquals(2, numSuccessfulPlugins); - sc.stop(); - sc = null; - assertEquals(2, numSuccessfulTerminations); - } - - @Test - public void testPluginShutdownWithException() { - // Verify an exception in one plugin shutdown does not affect the others - String pluginNames = testPluginName + "," + testBadPluginName + "," + testPluginName; - SparkConf conf = initializeSparkConf(pluginNames); - sc = new JavaSparkContext(conf); - assertEquals(3, numSuccessfulPlugins); - sc.stop(); - sc = null; - assertEquals(2, numSuccessfulTerminations); - } - - @Test - public void testPluginMetrics() { - // Verify that a custom metric is registered with the Spark metrics system - gaugeName = "test42"; - SparkConf conf = initializeSparkConf(testMetricsPluginName); - sc = new JavaSparkContext(conf); - assertEquals(1, numSuccessfulPlugins); - assertEquals(gaugeName, testMetricRegistry.getGauges().firstKey()); - sc.stop(); - sc = null; - assertEquals(1, numSuccessfulTerminations); - } - - public static class TestExecutorPlugin implements ExecutorPlugin { - public void init(ExecutorPluginContext pluginContext) { - ExecutorPluginSuite.numSuccessfulPlugins++; - } - - public void shutdown() { - ExecutorPluginSuite.numSuccessfulTerminations++; - } - } - - public static class TestSecondPlugin implements ExecutorPlugin { - public void init(ExecutorPluginContext pluginContext) { - ExecutorPluginSuite.numSuccessfulPlugins++; - } - - public void shutdown() { - ExecutorPluginSuite.numSuccessfulTerminations++; - } - } - - public static class TestBadShutdownPlugin implements ExecutorPlugin { - public void init(ExecutorPluginContext pluginContext) { - ExecutorPluginSuite.numSuccessfulPlugins++; - } - - public void shutdown() { - throw new RuntimeException("This plugin will fail to cleanly shut down"); - } - } - - public static class TestMetricsPlugin implements ExecutorPlugin { - public void init(ExecutorPluginContext myContext) { - MetricRegistry metricRegistry = myContext.metricRegistry; - // Registers a dummy metrics gauge for testing - String gaugeName = ExecutorPluginSuite.gaugeName; - metricRegistry.register(MetricRegistry.name(gaugeName), new Gauge() { - @Override - public Integer getValue() { - return 42; - } - }); - ExecutorPluginSuite.testMetricRegistry = metricRegistry; - ExecutorPluginSuite.numSuccessfulPlugins++; - } - - public void shutdown() { - ExecutorPluginSuite.numSuccessfulTerminations++; - } - } -} diff --git a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala index 24fa017363654..b432253ad80de 100644 --- a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala @@ -31,7 +31,7 @@ import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} -import org.apache.spark.{ExecutorPlugin => _, _} +import org.apache.spark._ import org.apache.spark.api.plugin._ import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 2d4d91dab075e..17d071d0779b2 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -9,9 +9,9 @@ license: | 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. @@ -24,10 +24,15 @@ license: | ## Upgrading from Core 2.4 to 3.0 -- In Spark 3.0, deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. +- The `org.apache.spark.ExecutorPlugin` interface and related configuration has been replaced with + `org.apache.spark.plugin.SparkPlugin`, which adds new functionality. Plugins using the old + interface need to be modified to extend the new interfaces. Check the + [Monitoring](monitoring.html) guide for more details. + +- Deprecated method `TaskContext.isRunningLocally` has been removed. Local execution was removed and it always has returned `false`. -- In Spark 3.0, deprecated method `shuffleBytesWritten`, `shuffleWriteTime` and `shuffleRecordsWritten` in `ShuffleWriteMetrics` have been removed. Instead, use `bytesWritten`, `writeTime ` and `recordsWritten` respectively. +- Deprecated method `shuffleBytesWritten`, `shuffleWriteTime` and `shuffleRecordsWritten` in `ShuffleWriteMetrics` have been removed. Instead, use `bytesWritten`, `writeTime ` and `recordsWritten` respectively. -- In Spark 3.0, deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. +- Deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. -- In Spark 3.0, event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. \ No newline at end of file +- Event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1da31ca9fad96..8830061c0d9ed 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,8 +36,8 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( - // [SPARK-28091[CORE] Extend Spark metrics system with user-defined metrics using executor plugins - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ExecutorPlugin.init"), + // [SPARK-29399][core] Remove old ExecutorPlugin interface. + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ExecutorPlugin"), // [SPARK-][SQL][CORE][MLLIB] Remove more old deprecated items in Spark 3 ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.createExternalTable"), @@ -108,13 +108,13 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.copy"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.this"), ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart$"), - + // [SPARK-27630][CORE] Properly handle task end events from completed stages ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerSpeculativeTaskSubmitted.apply"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerSpeculativeTaskSubmitted.copy"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerSpeculativeTaskSubmitted.this"), ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerSpeculativeTaskSubmitted$"), - + // [SPARK-26632][Core] Separate Thread Configurations of Driver and Executor ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.SparkTransportConf.fromSparkConf"), From eb79af8daec513071303fd9c383960abdba776f6 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 12 Nov 2019 17:34:03 -0800 Subject: [PATCH 0592/1387] [SPARK-29145][SQL][FOLLOW-UP] Move tests from `SubquerySuite` to `subquery/in-subquery/in-joins.sql` ### What changes were proposed in this pull request? Follow comment of https://github.com/apache/spark/pull/25854#discussion_r342383272 ### Why are the changes needed? NO ### Does this PR introduce any user-facing change? NO ### How was this patch tested? ADD TEST CASE Closes #26406 from AngersZhuuuu/SPARK-29145-FOLLOWUP. Authored-by: angerszhu Signed-off-by: Dongjoon Hyun --- .../inputs/subquery/in-subquery/in-joins.sql | 110 +++++++ .../subquery/in-subquery/in-joins.sql.out | 300 ++++++++++++++++-- .../org/apache/spark/sql/SubquerySuite.scala | 148 --------- 3 files changed, 377 insertions(+), 181 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql index f06be5fc6aa58..cd350a98e130b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -51,6 +51,18 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); +create temporary view s1 as select * from values + (1), (3), (5), (7), (9) + as s1(id); + +create temporary view s2 as select * from values + (1), (3), (4), (6), (9) + as s2(id); + +create temporary view s3 as select * from values + (3), (4), (6), (9) + as s3(id); + -- correlated IN subquery -- different JOIN in parent side -- TC 01.01 @@ -272,3 +284,101 @@ Group By t1a, t1b, t1c, t2a, t2b, t2c HAVING t2c IS NOT NULL ORDER By t2b DESC nulls last; + +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT 9); + + +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT 9); + + +-- IN with Subquery ON INNER JOIN +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON LEFT SEMI JOIN +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON LEFT ANTI JOIN +SELECT s1.id as id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON LEFT OUTER JOIN +SELECT s1.id, s2.id as id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON RIGHT OUTER JOIN +SELECT s1.id, s2.id as id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON FULL OUTER JOIN +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON INNER JOIN +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON LEFT SEMI JOIN +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON LEFT ANTI JOIN +SELECT s1.id AS id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON LEFT OUTER JOIN +SELECT s1.id, s2.id AS id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON RIGHT OUTER JOIN +SELECT s1.id, s2.id AS id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON FULL OUTER JOIN +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +DROP VIEW s1; + +DROP VIEW s2; + +DROP VIEW s3; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out index 7089fbd25e28c..fbb4dd3b5efd1 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 14 +-- Number of queries: 34 -- !query 0 @@ -64,9 +64,34 @@ create temporary view t3 as select * from values struct<> -- !query 2 output +-- !query 3 +create temporary view s1 as select * from values + (1), (3), (5), (7), (9) + as s1(id) +-- !query 3 schema +struct<> +-- !query 3 output --- !query 3 +-- !query 4 +create temporary view s2 as select * from values + (1), (3), (4), (6), (9) + as s2(id) +-- !query 4 schema +struct<> +-- !query 4 output + + +-- !query 5 +create temporary view s3 as select * from values + (3), (4), (6), (9) + as s3(id) +-- !query 5 schema +struct<> +-- !query 5 output + + +-- !query 6 SELECT t1a, t1b, t1c, t3a, t3b, t3c FROM t1 natural JOIN t3 WHERE t1a IN (SELECT t2a @@ -77,14 +102,14 @@ WHERE t1a IN (SELECT t2a ORDER BY t1a, t1b, t1c DESC nulls first --- !query 3 schema +-- !query 6 schema struct --- !query 3 output +-- !query 6 output val1b 8 16 val1b 8 16 val1b 8 16 val1b 8 16 --- !query 4 +-- !query 7 SELECT Count(DISTINCT(t1a)), t1b, t3a, @@ -103,9 +128,9 @@ GROUP BY t1a, t3b, t3c ORDER BY t1a DESC, t3b DESC, t3c ASC --- !query 4 schema +-- !query 7 schema struct --- !query 4 output +-- !query 7 output 1 10 val3b 8 NULL 1 10 val1b 8 16 1 10 val3a 6 12 @@ -113,7 +138,7 @@ struct 1 8 val3a 6 12 --- !query 5 +-- !query 8 SELECT Count(DISTINCT(t1a)) FROM t1 natural right JOIN t3 WHERE t1a IN @@ -129,13 +154,13 @@ AND t1d IN AND t1a = t3a GROUP BY t1a ORDER BY t1a --- !query 5 schema +-- !query 8 schema struct --- !query 5 output +-- !query 8 output 1 --- !query 6 +-- !query 9 SELECT t1a, t1b, t1c, @@ -151,9 +176,9 @@ where t1a IN AND t1b != t3b AND t1a = 'val1b' ORDER BY t1a --- !query 6 schema +-- !query 9 schema struct --- !query 6 output +-- !query 9 output val1b 8 16 val3a 6 12 val1b 8 16 val3a 6 12 val1b 8 16 val1b 10 12 @@ -162,7 +187,7 @@ val1b 8 16 val3c 17 16 val1b 8 16 val3c 17 16 --- !query 7 +-- !query 10 SELECT Count(DISTINCT(t1a)), t1b FROM t1 RIGHT JOIN t3 @@ -181,13 +206,13 @@ GROUP BY t1a, t1b HAVING t1b > 8 ORDER BY t1a --- !query 7 schema +-- !query 10 schema struct --- !query 7 output +-- !query 10 output 1 10 --- !query 8 +-- !query 11 SELECT Count(DISTINCT(t1a)) FROM t1 LEFT OUTER JOIN t3 @@ -199,15 +224,15 @@ WHERE t1a IN WHERE t1h < t2h ) GROUP BY t1a ORDER BY t1a --- !query 8 schema +-- !query 11 schema struct --- !query 8 output +-- !query 11 output 1 1 1 --- !query 9 +-- !query 12 SELECT Count(DISTINCT(t1a)), t1b FROM t1 INNER JOIN t2 @@ -224,14 +249,14 @@ OR t1a IN WHERE t2h < t1h) GROUP BY t1b HAVING t1b > 6 --- !query 9 schema +-- !query 12 schema struct --- !query 9 output +-- !query 12 output 1 10 1 8 --- !query 10 +-- !query 13 SELECT Count(DISTINCT(t1a)), t1b FROM t1 @@ -249,13 +274,13 @@ AND t1h IN where t2b = t3b) GROUP BY t1b HAVING t1b > 8 --- !query 10 schema +-- !query 13 schema struct --- !query 10 output +-- !query 13 output 1 10 --- !query 11 +-- !query 14 SELECT Count(DISTINCT(t1a)), t1b FROM t1 @@ -280,13 +305,13 @@ AND t1b IN GROUP BY t1b HAVING t1b > 8 --- !query 11 schema +-- !query 14 schema struct -- !query 11 output 1 10 --- !query 12 +-- !query 15 SELECT Count(DISTINCT(t1a)), t1b FROM t1 @@ -314,13 +339,13 @@ AND t1b IN AND t1a = t2a GROUP BY t1b ORDER BY t1b DESC --- !query 12 schema +-- !query 15 schema struct --- !query 12 output +-- !query 15 output 1 8 --- !query 13 +-- !query 16 SELECT t1a, t1b, t1c, @@ -345,9 +370,218 @@ and t1a = t2a Group By t1a, t1b, t1c, t2a, t2b, t2c HAVING t2c IS NOT NULL ORDER By t2b DESC nulls last --- !query 13 schema +-- !query 16 schema struct --- !query 13 output +-- !query 16 output val1b 8 16 1 10 12 val1b 8 16 1 8 16 val1b 8 16 1 NULL 16 + + + +-- !query 17 +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT 9) +-- !query 17 schema +struct +-- !query 17 output +9 + + +-- !query 18 +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT 9) +-- !query 18 schema +struct +-- !query 18 output +1 +3 + + +-- !query 19 +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query 19 schema +struct +-- !query 19 output +3 +9 + + +-- !query 20 +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query 20 schema +struct +-- !query 20 output +3 +9 + + +-- !query 21 +SELECT s1.id as id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query 21 schema +struct +-- !query 21 output +1 +5 +7 + + +-- !query 22 +SELECT s1.id, s2.id as id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query 22 schema +struct +-- !query 22 output +1 NULL +3 3 +5 NULL +7 NULL +9 9 + + +-- !query 23 +SELECT s1.id, s2.id as id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query 23 schema +struct +-- !query 23 output +3 3 +9 9 +NULL 1 +NULL 4 +NULL 6 + + +-- !query 24 +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query 24 schema +struct +-- !query 24 output +1 NULL +3 3 +5 NULL +7 NULL +9 9 +NULL 1 +NULL 4 +NULL 6 + + +-- !query 25 +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query 25 schema +struct +-- !query 25 output +1 + + +-- !query 26 +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query 26 schema +struct +-- !query 26 output +1 + + +-- !query 27 +SELECT s1.id AS id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query 27 schema +struct +-- !query 27 output +3 +5 +7 +9 + + +-- !query 28 +SELECT s1.id, s2.id AS id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query 28 schema +struct +-- !query 28 output +1 1 +3 NULL +5 NULL +7 NULL +9 NULL + + +-- !query 29 +SELECT s1.id, s2.id AS id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query 29 schema +struct +-- !query 29 output +1 1 +NULL 3 +NULL 4 +NULL 6 +NULL 9 + +-- !query 30 +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query 30 schema +struct +-- !query 30 output +1 1 +3 NULL +5 NULL +7 NULL +9 NULL +NULL 3 +NULL 4 +NULL 6 +NULL 9 + + +-- !query 31 +DROP VIEW s1 +-- !query 31 schema +struct<> +-- !query 31 output + +-- !query 32 +DROP VIEW s2 +-- !query 32 schema +struct<> +-- !query 32 output + +-- !query 33 +DROP VIEW s3 +-- !query 33 schema +struct<> +-- !query 33 output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index e05af08dfb748..c117ee7818c01 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -204,154 +204,6 @@ class SubquerySuite extends QueryTest with SharedSparkSession { } } - test("SPARK-29145: JOIN Condition use QueryList") { - withTempView("s1", "s2", "s3") { - Seq(1, 3, 5, 7, 9).toDF("id").createOrReplaceTempView("s1") - Seq(1, 3, 4, 6, 9).toDF("id").createOrReplaceTempView("s2") - Seq(3, 4, 6, 9).toDF("id").createOrReplaceTempView("s3") - - checkAnswer( - sql( - """ - | SELECT s1.id FROM s1 - | JOIN s2 ON s1.id = s2.id - | AND s1.id IN (SELECT 9) - """.stripMargin), - Row(9) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id FROM s1 - | JOIN s2 ON s1.id = s2.id - | AND s1.id NOT IN (SELECT 9) - """.stripMargin), - Row(1) :: Row(3) :: Nil) - - // case `IN` - checkAnswer( - sql( - """ - | SELECT s1.id FROM s1 - | JOIN s2 ON s1.id = s2.id - | AND s1.id IN (SELECT id FROM s3) - """.stripMargin), - Row(3) :: Row(9) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id AS id2 FROM s1 - | LEFT SEMI JOIN s2 - | ON s1.id = s2.id - | AND s1.id IN (SELECT id FROM s3) - """.stripMargin), - Row(3) :: Row(9) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id as id2 FROM s1 - | LEFT ANTI JOIN s2 - | ON s1.id = s2.id - | AND s1.id IN (SELECT id FROM s3) - """.stripMargin), - Row(1) :: Row(5) :: Row(7) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id, s2.id as id2 FROM s1 - | LEFT OUTER JOIN s2 - | ON s1.id = s2.id - | AND s1.id IN (SELECT id FROM s3) - """.stripMargin), - Row(1, null) :: Row(3, 3) :: Row(5, null) :: Row(7, null) :: Row(9, 9) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id, s2.id as id2 FROM s1 - | RIGHT OUTER JOIN s2 - | ON s1.id = s2.id - | AND s1.id IN (SELECT id FROM s3) - """.stripMargin), - Row(null, 1) :: Row(3, 3) :: Row(null, 4) :: Row(null, 6) :: Row(9, 9) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id, s2.id AS id2 FROM s1 - | FULL OUTER JOIN s2 - | ON s1.id = s2.id - | AND s1.id IN (SELECT id FROM s3) - """.stripMargin), - Row(1, null) :: Row(3, 3) :: Row(5, null) :: Row(7, null) :: Row(9, 9) :: - Row(null, 1) :: Row(null, 4) :: Row(null, 6) :: Nil) - - // case `NOT IN` - checkAnswer( - sql( - """ - | SELECT s1.id FROM s1 - | JOIN s2 ON s1.id = s2.id - | AND s1.id NOT IN (SELECT id FROM s3) - """.stripMargin), - Row(1) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id AS id2 FROM s1 - | LEFT SEMI JOIN s2 - | ON s1.id = s2.id - | AND s1.id NOT IN (SELECT id FROM s3) - """.stripMargin), - Row(1) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id AS id2 FROM s1 - | LEFT ANTI JOIN s2 - | ON s1.id = s2.id - | AND s1.id NOT IN (SELECT id FROM s3) - """.stripMargin), - Row(3) :: Row(5) :: Row(7) :: Row(9) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id, s2.id AS id2 FROM s1 - | LEFT OUTER JOIN s2 - | ON s1.id = s2.id - | AND s1.id NOT IN (SELECT id FROM s3) - """.stripMargin), - Row(1, 1) :: Row(3, null) :: Row(5, null) :: Row(7, null) :: Row(9, null) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id, s2.id AS id2 FROM s1 - | RIGHT OUTER JOIN s2 - | ON s1.id = s2.id - | AND s1.id NOT IN (SELECT id FROM s3) - """.stripMargin), - Row(1, 1) :: Row(null, 3) :: Row(null, 4) :: Row(null, 6) :: Row(null, 9) :: Nil) - - checkAnswer( - sql( - """ - | SELECT s1.id, s2.id AS id2 FROM s1 - | FULL OUTER JOIN s2 - | ON s1.id = s2.id - | AND s1.id NOT IN (SELECT id FROM s3) - """.stripMargin), - Row(1, 1) :: Row(3, null) :: Row(5, null) :: Row(7, null) :: Row(9, null) :: - Row(null, 3) :: Row(null, 4) :: Row(null, 6) :: Row(null, 9) :: Nil) - } - } - test("SPARK-14791: scalar subquery inside broadcast join") { val df = sql("select a, sum(b) as s from l group by a having a > (select avg(a) from l)") val expected = Row(3, 2.0, 3, 3.0) :: Row(6, null, 6, null) :: Nil From 80fbc382a60973db21367a922a0fb797e5ab382d Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 13 Nov 2019 13:12:20 +0900 Subject: [PATCH 0593/1387] Revert "[SPARK-29462] The data type of "array()" should be array" This reverts commit 0dcd739534eb2357daeaa576f6b1223aa4ca0a6e. --- docs/sql-migration-guide.md | 2 -- .../catalyst/expressions/complexTypeCreator.scala | 2 +- .../apache/spark/sql/DataFrameFunctionsSuite.scala | 12 ++++-------- 3 files changed, 5 insertions(+), 11 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 153e68b58e746..2d5afa919e668 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -217,8 +217,6 @@ license: | For example `SELECT timestamp 'tomorrow';`. - Since Spark 3.0, the `size` function returns `NULL` for the `NULL` input. In Spark version 2.4 and earlier, this function gives `-1` for the same input. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.sizeOfNull` to `true`. - - - Since Spark 3.0, when `array` function is called without parameters, it returns an empty array with `NullType` data type. In Spark version 2.4 and earlier, the data type of the result is `StringType`. - Since Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception. 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 8e00e32a719e6..3f722e8537c36 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 @@ -47,7 +47,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { override def dataType: ArrayType = { ArrayType( TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(children.map(_.dataType)) - .getOrElse(NullType), + .getOrElse(StringType), containsNull = children.exists(_.nullable)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 1dc7d34b52277..06484908f5e73 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -3400,9 +3400,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ).foreach(assertValuesDoNotChangeAfterCoalesceOrUnion(_)) } - test("SPARK-21281 use string types by default if map have no argument") { + test("SPARK-21281 use string types by default if array and map have no argument") { val ds = spark.range(1) var expectedSchema = new StructType() + .add("x", ArrayType(StringType, containsNull = false), nullable = false) + assert(ds.select(array().as("x")).schema == expectedSchema) + expectedSchema = new StructType() .add("x", MapType(StringType, StringType, valueContainsNull = false), nullable = false) assert(ds.select(map().as("x")).schema == expectedSchema) } @@ -3460,13 +3463,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df.select("x").filter("exists(i, x -> x % d == 0)"), Seq(Row(1))) } - - test("SPARK-29462: Use null type by default if array have no argument") { - val ds = spark.range(1) - var expectedSchema = new StructType() - .add("x", ArrayType(NullType, containsNull = false), nullable = false) - assert(ds.select(array().as("x")).schema == expectedSchema) - } } object DataFrameFunctionsSuite { From f926809a1f6b3dd6041518a98b115e42d9692836 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 13 Nov 2019 15:04:39 +0900 Subject: [PATCH 0594/1387] [SPARK-29390][SQL] Add the justify_days(), justify_hours() and justif_interval() functions ### What changes were proposed in this pull request? Add 3 interval functions justify_days, justify_hours, justif_interval to support justify interval values ### Why are the changes needed? For feature parity with postgres add three interval functions to justify interval values. justify_days(interval) | interval | Adjust interval so 30-day time periods are represented as months | justify_days(interval '35 days') | 1 mon 5 days -- | -- | -- | -- | -- justify_hours(interval) | interval | Adjust interval so 24-hour time periods are represented as days | justify_hours(interval '27 hours') | 1 day 03:00:00 justify_interval(interval) | interval | Adjust interval using justify_days and justify_hours, with additional sign adjustments | justify_interval(interval '1 mon -1 hour') | 29 days 23:00:00 ### Does this PR introduce any user-facing change? yes. new interval functions are added ### How was this patch tested? add ut Closes #26465 from yaooqinn/SPARK-29390. Authored-by: Kent Yao Signed-off-by: Takeshi Yamamuro --- .../catalyst/analysis/FunctionRegistry.scala | 3 + .../expressions/intervalExpressions.scala | 68 ++++++++ .../sql/catalyst/util/IntervalUtils.scala | 36 ++++ .../catalyst/util/IntervalUtilsSuite.scala | 31 ++++ .../resources/sql-tests/inputs/interval.sql | 14 ++ .../sql-tests/inputs/postgreSQL/interval.sql | 8 +- .../sql-tests/results/interval.sql.out | 98 ++++++++++- .../results/postgreSQL/interval.sql.out | 160 ++++++++++-------- 8 files changed, 344 insertions(+), 74 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 13a8382d17ec6..dc8757293b302 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -420,6 +420,9 @@ object FunctionRegistry { expression[MakeDate]("make_date"), expression[MakeTimestamp]("make_timestamp"), expression[MakeInterval]("make_interval"), + expression[JustifyDays]("justify_days"), + expression[JustifyHours]("justify_hours"), + expression[JustifyInterval]("justify_interval"), expression[DatePart]("date_part"), // collection functions diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 78314d6d85712..c8a40d0435a58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Locale +import scala.util.control.NonFatal + import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.catalyst.util.IntervalUtils._ @@ -257,3 +259,69 @@ case class MakeInterval( override def prettyName: String = "make_interval" } + +abstract class IntervalJustifyLike( + child: Expression, + justify: CalendarInterval => CalendarInterval, + justifyFuncName: String) extends UnaryExpression with ExpectsInputTypes { + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + + override def dataType: DataType = CalendarIntervalType + + override def nullSafeEval(input: Any): Any = { + try { + justify(input.asInstanceOf[CalendarInterval]) + } catch { + case NonFatal(_) => null + } + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + nullSafeCodeGen(ctx, ev, child => { + val iu = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") + s""" + |try { + | ${ev.value} = $iu.$justifyFuncName($child); + |} catch (java.lang.ArithmeticException e) { + | ${ev.isNull} = true; + |} + |""".stripMargin + }) + } + + override def prettyName: String = justifyFuncName +} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Adjust interval so 30-day time periods are represented as months", + examples = """ + Examples: + > SELECT _FUNC_(interval '1 month -59 day 25 hour'); + -29 days 25 hours + """, + since = "3.0.0") +case class JustifyDays(child: Expression) + extends IntervalJustifyLike(child, justifyDays, "justifyDays") + +@ExpressionDescription( + usage = "_FUNC_(expr) - Adjust interval so 24-hour time periods are represented as days", + examples = """ + Examples: + > SELECT _FUNC_(interval '1 month -59 day 25 hour'); + 1 months -57 days -23 hours + """, + since = "3.0.0") +case class JustifyHours(child: Expression) + extends IntervalJustifyLike(child, justifyHours, "justifyHours") + +@ExpressionDescription( + usage = "_FUNC_(expr) - Adjust interval using justifyHours and justifyDays, with additional" + + " sign adjustments", + examples = """ + Examples: + > SELECT _FUNC_(interval '1 month -59 day 25 hour'); + -27 days -23 hours + """, + since = "3.0.0") +case class JustifyInterval(child: Expression) + extends IntervalJustifyLike(child, justifyInterval, "justifyInterval") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 7e903274ef713..882c1d85267e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -637,4 +637,40 @@ object IntervalUtils { new CalendarInterval(totalMonths, totalDays, micros) } + + /** + * Adjust interval so 30-day time periods are represented as months. + */ + def justifyDays(interval: CalendarInterval): CalendarInterval = { + val monthToDays = interval.months * DAYS_PER_MONTH + val totalDays = monthToDays + interval.days + val months = Math.toIntExact(totalDays / DAYS_PER_MONTH) + val days = totalDays % DAYS_PER_MONTH + new CalendarInterval(months, days.toInt, interval.microseconds) + } + + /** + * Adjust interval so 24-hour time periods are represented as days. + */ + def justifyHours(interval: CalendarInterval): CalendarInterval = { + val dayToUs = MICROS_PER_DAY * interval.days + val totalUs = Math.addExact(interval.microseconds, dayToUs) + val days = totalUs / MICROS_PER_DAY + val microseconds = totalUs % MICROS_PER_DAY + new CalendarInterval(interval.months, days.toInt, microseconds) + } + + /** + * Adjust interval using justifyHours and justifyDays, with additional sign adjustments. + */ + def justifyInterval(interval: CalendarInterval): CalendarInterval = { + val monthToDays = DAYS_PER_MONTH * interval.months + val dayToUs = Math.multiplyExact(monthToDays + interval.days, MICROS_PER_DAY) + val totalUs = Math.addExact(interval.microseconds, dayToUs) + val microseconds = totalUs % MICROS_PER_DAY + val totalDays = totalUs / MICROS_PER_DAY + val days = totalDays % DAYS_PER_MONTH + val months = totalDays / DAYS_PER_MONTH + new CalendarInterval(months.toInt, days.toInt, microseconds) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 0a16dbfdebde7..8c84eb107cd30 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -266,4 +266,35 @@ class IntervalUtilsSuite extends SparkFunSuite { assert(e.getMessage.contains("divide by zero")) } } + + test("justify days") { + assert(justifyDays(fromString("1 month 35 day")) === new CalendarInterval(2, 5, 0)) + assert(justifyDays(fromString("-1 month 35 day")) === new CalendarInterval(0, 5, 0)) + assert(justifyDays(fromString("1 month -35 day")) === new CalendarInterval(0, -5, 0)) + assert(justifyDays(fromString("-1 month -35 day")) === new CalendarInterval(-2, -5, 0)) + assert(justifyDays(fromString("-1 month 2 day")) === new CalendarInterval(0, -28, 0)) + } + + test("justify hours") { + assert(justifyHours(fromString("29 day 25 hour")) === + new CalendarInterval(0, 30, 1 * MICROS_PER_HOUR)) + assert(justifyHours(fromString("29 day -25 hour")) === + new CalendarInterval(0, 27, 23 * MICROS_PER_HOUR)) + assert(justifyHours(fromString("-29 day 25 hour")) === + new CalendarInterval(0, -27, -23 * MICROS_PER_HOUR)) + assert(justifyHours(fromString("-29 day -25 hour")) === + new CalendarInterval(0, -30, -1 * MICROS_PER_HOUR)) + } + + test("justify interval") { + assert(justifyInterval(fromString("1 month 29 day 25 hour")) === + new CalendarInterval(2, 0, 1 * MICROS_PER_HOUR)) + assert(justifyInterval(fromString("-1 month 29 day -25 hour")) === + new CalendarInterval(0, -2, -1 * MICROS_PER_HOUR)) + assert(justifyInterval(fromString("1 month -29 day -25 hour")) === + new CalendarInterval(0, 0, -1 * MICROS_PER_HOUR)) + assert(justifyInterval(fromString("-1 month -29 day -25 hour")) === + new CalendarInterval(-2, 0, -1 * MICROS_PER_HOUR)) + intercept[ArithmeticException](justifyInterval(new CalendarInterval(2, 0, Long.MaxValue))) + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index 148d84942a682..226c13f961a85 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -74,3 +74,17 @@ select cast('+ 1 second' as interval); select cast('- 1 second' as interval); select cast('- -1 second' as interval); select cast('- +1 second' as interval); + +-- justify intervals +select justify_days(cast(null as interval)); +select justify_hours(cast(null as interval)); +select justify_interval(cast(null as interval)); +select justify_days(interval '1 month 59 day 25 hour'); +select justify_hours(interval '1 month 59 day 25 hour'); +select justify_interval(interval '1 month 59 day 25 hour'); +select justify_days(interval '1 month -59 day 25 hour'); +select justify_hours(interval '1 month -59 day 25 hour'); +select justify_interval(interval '1 month -59 day 25 hour'); +select justify_days(interval '1 month 59 day -25 hour'); +select justify_hours(interval '1 month 59 day -25 hour'); +select justify_interval(interval '1 month 59 day -25 hour'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql index eb8cc34419519..01df2a3fd1b21 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql @@ -157,13 +157,11 @@ -- select '100000000y 10mon -1000000000d -100000h -10min -10.000001s ago'::interval; -- test justify_hours() and justify_days() --- [SPARK-29390] Add the justify_days(), justify_hours() and justify_interval() functions --- SELECT justify_hours(interval '6 months 3 days 52 hours 3 minutes 2 seconds') as `6 mons 5 days 4 hours 3 mins 2 seconds`; --- SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as `7 mons 6 days 5 hours 4 mins 3 seconds`; +SELECT justify_hours(interval '6 months 3 days 52 hours 3 minutes 2 seconds') as `6 mons 5 days 4 hours 3 mins 2 seconds`; +SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as `7 mons 6 days 5 hours 4 mins 3 seconds`; -- test justify_interval() - --- SELECT justify_interval(interval '1 month -1 hour') as `1 month -1 hour`; +SELECT justify_interval(interval '1 month -1 hour') as `1 month -1 hour`; -- test fractional second input, and detection of duplicate units -- [SPARK-28259] Date/Time Output Styles and Date Order Conventions diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 1bbeeb2085e43..372f7d9e0b96d 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 45 +-- Number of queries: 57 -- !query 0 @@ -362,3 +362,99 @@ select cast('- +1 second' as interval) struct -- !query 44 output NULL + + +-- !query 45 +select justify_days(cast(null as interval)) +-- !query 45 schema +struct +-- !query 45 output +NULL + + +-- !query 46 +select justify_hours(cast(null as interval)) +-- !query 46 schema +struct +-- !query 46 output +NULL + + +-- !query 47 +select justify_interval(cast(null as interval)) +-- !query 47 schema +struct +-- !query 47 output +NULL + + +-- !query 48 +select justify_days(interval '1 month 59 day 25 hour') +-- !query 48 schema +struct +-- !query 48 output +2 months 29 days 25 hours + + +-- !query 49 +select justify_hours(interval '1 month 59 day 25 hour') +-- !query 49 schema +struct +-- !query 49 output +1 months 60 days 1 hours + + +-- !query 50 +select justify_interval(interval '1 month 59 day 25 hour') +-- !query 50 schema +struct +-- !query 50 output +3 months 1 hours + + +-- !query 51 +select justify_days(interval '1 month -59 day 25 hour') +-- !query 51 schema +struct +-- !query 51 output +-29 days 25 hours + + +-- !query 52 +select justify_hours(interval '1 month -59 day 25 hour') +-- !query 52 schema +struct +-- !query 52 output +1 months -57 days -23 hours + + +-- !query 53 +select justify_interval(interval '1 month -59 day 25 hour') +-- !query 53 schema +struct +-- !query 53 output +-27 days -23 hours + + +-- !query 54 +select justify_days(interval '1 month 59 day -25 hour') +-- !query 54 schema +struct +-- !query 54 output +2 months 29 days -25 hours + + +-- !query 55 +select justify_hours(interval '1 month 59 day -25 hour') +-- !query 55 schema +struct +-- !query 55 output +1 months 57 days 23 hours + + +-- !query 56 +select justify_interval(interval '1 month 59 day -25 hour') +-- !query 56 schema +struct +-- !query 56 output +2 months 27 days 23 hours diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index 019068c9b4f96..5ef1f8e5f0680 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -1,153 +1,153 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 24 +-- Number of queries: 27 -- !query 0 -SELECT interval '999' second +SELECT justify_hours(interval '6 months 3 days 52 hours 3 minutes 2 seconds') as `6 mons 5 days 4 hours 3 mins 2 seconds` -- !query 0 schema -struct<16 minutes 39 seconds:interval> +struct<6 mons 5 days 4 hours 3 mins 2 seconds:interval> -- !query 0 output -16 minutes 39 seconds +6 months 5 days 4 hours 3 minutes 2 seconds -- !query 1 -SELECT interval '999' minute +SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as `7 mons 6 days 5 hours 4 mins 3 seconds` -- !query 1 schema -struct<16 hours 39 minutes:interval> +struct<7 mons 6 days 5 hours 4 mins 3 seconds:interval> -- !query 1 output -16 hours 39 minutes +7 months 6 days 5 hours 4 minutes 3 seconds -- !query 2 -SELECT interval '999' hour +SELECT justify_interval(interval '1 month -1 hour') as `1 month -1 hour` -- !query 2 schema -struct<999 hours:interval> +struct<1 month -1 hour:interval> -- !query 2 output -999 hours +29 days 23 hours -- !query 3 -SELECT interval '999' day +SELECT interval '999' second -- !query 3 schema -struct<999 days:interval> +struct<16 minutes 39 seconds:interval> -- !query 3 output -999 days +16 minutes 39 seconds -- !query 4 -SELECT interval '999' month +SELECT interval '999' minute -- !query 4 schema -struct<83 years 3 months:interval> +struct<16 hours 39 minutes:interval> -- !query 4 output -83 years 3 months +16 hours 39 minutes -- !query 5 -SELECT interval '1' year +SELECT interval '999' hour -- !query 5 schema -struct<1 years:interval> +struct<999 hours:interval> -- !query 5 output -1 years +999 hours -- !query 6 -SELECT interval '2' month +SELECT interval '999' day -- !query 6 schema -struct<2 months:interval> +struct<999 days:interval> -- !query 6 output -2 months +999 days -- !query 7 -SELECT interval '3' day +SELECT interval '999' month -- !query 7 schema -struct<3 days:interval> +struct<83 years 3 months:interval> -- !query 7 output -3 days +83 years 3 months -- !query 8 -SELECT interval '4' hour +SELECT interval '1' year -- !query 8 schema -struct<4 hours:interval> +struct<1 years:interval> -- !query 8 output -4 hours +1 years -- !query 9 -SELECT interval '5' minute +SELECT interval '2' month -- !query 9 schema -struct<5 minutes:interval> +struct<2 months:interval> -- !query 9 output -5 minutes +2 months -- !query 10 -SELECT interval '6' second +SELECT interval '3' day -- !query 10 schema -struct<6 seconds:interval> +struct<3 days:interval> -- !query 10 output -6 seconds +3 days -- !query 11 -SELECT interval '1-2' year to month +SELECT interval '4' hour -- !query 11 schema -struct<1 years 2 months:interval> +struct<4 hours:interval> -- !query 11 output -1 years 2 months +4 hours -- !query 12 -SELECT interval '1 2:03' day to hour +SELECT interval '5' minute -- !query 12 schema -struct<1 days 2 hours:interval> +struct<5 minutes:interval> -- !query 12 output -1 days 2 hours +5 minutes -- !query 13 -SELECT interval '1 2:03:04' day to hour +SELECT interval '6' second -- !query 13 schema -struct<1 days 2 hours:interval> +struct<6 seconds:interval> -- !query 13 output -1 days 2 hours +6 seconds -- !query 14 -SELECT interval '1 2:03' day to minute +SELECT interval '1-2' year to month -- !query 14 schema -struct<1 days 2 hours 3 minutes:interval> +struct<1 years 2 months:interval> -- !query 14 output -1 days 2 hours 3 minutes +1 years 2 months -- !query 15 -SELECT interval '1 2:03:04' day to minute +SELECT interval '1 2:03' day to hour -- !query 15 schema -struct<1 days 2 hours 3 minutes:interval> +struct<1 days 2 hours:interval> -- !query 15 output -1 days 2 hours 3 minutes +1 days 2 hours -- !query 16 -SELECT interval '1 2:03' day to second +SELECT interval '1 2:03:04' day to hour -- !query 16 schema -struct<1 days 2 hours 3 minutes:interval> +struct<1 days 2 hours:interval> -- !query 16 output -1 days 2 hours 3 minutes +1 days 2 hours -- !query 17 -SELECT interval '1 2:03:04' day to second +SELECT interval '1 2:03' day to minute -- !query 17 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct<1 days 2 hours 3 minutes:interval> -- !query 17 output -1 days 2 hours 3 minutes 4 seconds +1 days 2 hours 3 minutes -- !query 18 -SELECT interval '1 2:03' hour to minute +SELECT interval '1 2:03:04' day to minute -- !query 18 schema struct<1 days 2 hours 3 minutes:interval> -- !query 18 output @@ -155,7 +155,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 19 -SELECT interval '1 2:03:04' hour to minute +SELECT interval '1 2:03' day to second -- !query 19 schema struct<1 days 2 hours 3 minutes:interval> -- !query 19 output @@ -163,32 +163,56 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 20 -SELECT interval '1 2:03' hour to second +SELECT interval '1 2:03:04' day to second -- !query 20 schema -struct<1 days 2 hours 3 minutes:interval> +struct<1 days 2 hours 3 minutes 4 seconds:interval> -- !query 20 output -1 days 2 hours 3 minutes +1 days 2 hours 3 minutes 4 seconds -- !query 21 -SELECT interval '1 2:03:04' hour to second +SELECT interval '1 2:03' hour to minute -- !query 21 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct<1 days 2 hours 3 minutes:interval> -- !query 21 output -1 days 2 hours 3 minutes 4 seconds +1 days 2 hours 3 minutes -- !query 22 -SELECT interval '1 2:03' minute to second +SELECT interval '1 2:03:04' hour to minute -- !query 22 schema -struct<1 days 2 minutes 3 seconds:interval> +struct<1 days 2 hours 3 minutes:interval> -- !query 22 output -1 days 2 minutes 3 seconds +1 days 2 hours 3 minutes -- !query 23 -SELECT interval '1 2:03:04' minute to second +SELECT interval '1 2:03' hour to second -- !query 23 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct<1 days 2 hours 3 minutes:interval> -- !query 23 output +1 days 2 hours 3 minutes + + +-- !query 24 +SELECT interval '1 2:03:04' hour to second +-- !query 24 schema +struct<1 days 2 hours 3 minutes 4 seconds:interval> +-- !query 24 output +1 days 2 hours 3 minutes 4 seconds + + +-- !query 25 +SELECT interval '1 2:03' minute to second +-- !query 25 schema +struct<1 days 2 minutes 3 seconds:interval> +-- !query 25 output +1 days 2 minutes 3 seconds + + +-- !query 26 +SELECT interval '1 2:03:04' minute to second +-- !query 26 schema +struct<1 days 2 hours 3 minutes 4 seconds:interval> +-- !query 26 output 1 days 2 hours 3 minutes 4 seconds From b5a2ed6a37fd8f6630a7b22791a07a45f5aeb556 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 13 Nov 2019 17:06:27 +0800 Subject: [PATCH 0595/1387] [SPARK-29851][SQL] V2 catalog: Change default behavior of dropping namespace to cascade ### What changes were proposed in this pull request? Currently, `SupportsNamespaces.dropNamespace` drops a namespace only if it is empty. Thus, to implement a cascading drop, one needs to iterate all objects (tables, view, etc.) within the namespace (including its sub-namespaces recursively) and drop them one by one. This can have a negative impact on the performance when there are large number of objects. Instead, this PR proposes to change the default behavior of dropping a namespace to cascading such that implementing cascading/non-cascading drop is simpler without performance penalties. ### Why are the changes needed? The new behavior makes implementing cascading/non-cascading drop simple without performance penalties. ### Does this PR introduce any user-facing change? Yes. The default behavior of `SupportsNamespaces.dropNamespace` is now cascading. ### How was this patch tested? Added new unit tests. Closes #26476 from imback82/drop_ns_cascade. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../connector/catalog/SupportsNamespaces.java | 8 ++-- .../catalyst/analysis/ResolveCatalogs.scala | 2 +- .../catalyst/plans/logical/v2Commands.scala | 4 +- .../sql/connector/InMemoryTableCatalog.scala | 14 +++---- .../connector/catalog/TableCatalogSuite.scala | 11 ++--- .../datasources/v2/DropNamespaceExec.scala | 26 ++++++++---- .../sql/connector/DataSourceV2SQLSuite.scala | 42 +++++++++++++++---- 7 files changed, 67 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java index 6845f9294a6de..40d9c2b122817 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java @@ -131,16 +131,14 @@ void alterNamespace( NamespaceChange... changes) throws NoSuchNamespaceException; /** - * Drop a namespace from the catalog. + * Drop a namespace from the catalog, recursively dropping all objects within the namespace. *

    - * This operation may be rejected by the catalog implementation if the namespace is not empty by - * throwing {@link IllegalStateException}. If the catalog implementation does not support this - * operation, it may throw {@link UnsupportedOperationException}. + * If the catalog implementation does not support this operation, it may throw + * {@link UnsupportedOperationException}. * * @param namespace a multi-part namespace * @return true if the namespace was dropped * @throws NoSuchNamespaceException If the namespace does not exist (optional) - * @throws IllegalStateException If the namespace is not empty * @throws UnsupportedOperationException If drop is not a supported operation */ boolean dropNamespace(String[] namespace) throws NoSuchNamespaceException; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 5b48f874a8701..b831e83ee21d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -183,7 +183,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) c.properties) case DropNamespaceStatement(NonSessionCatalog(catalog, nameParts), ifExists, cascade) => - DropNamespace(catalog.asNamespaceCatalog, nameParts, ifExists, cascade) + DropNamespace(catalog, nameParts, ifExists, cascade) case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index f27d10924ef97..6dc742610d04b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.analysis.{NamedRelation, Star, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Unevaluable} import org.apache.spark.sql.catalyst.plans.DescribeTableSchema -import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, SupportsNamespaces, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, SupportsNamespaces, TableCatalog, TableChange} import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.{DataType, StringType, StructType} @@ -250,7 +250,7 @@ case class CreateNamespace( * The logical plan of the DROP NAMESPACE command that works for v2 catalogs. */ case class DropNamespace( - catalog: SupportsNamespaces, + catalog: CatalogPlugin, namespace: Seq[String], ifExists: Boolean, cascade: Boolean) extends Command diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala index ece903a4c2838..fd1beb49b9e07 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala @@ -31,6 +31,9 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class BasicInMemoryTableCatalog extends TableCatalog { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + protected val namespaces: util.Map[List[String], Map[String, String]] = + new ConcurrentHashMap[List[String], Map[String, String]]() + protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() @@ -74,6 +77,7 @@ class BasicInMemoryTableCatalog extends TableCatalog { val table = new InMemoryTable(s"$name.${ident.quoted}", schema, partitions, properties) tables.put(ident, table) + namespaces.putIfAbsent(ident.namespace.toList, Map()) table } @@ -120,11 +124,6 @@ class BasicInMemoryTableCatalog extends TableCatalog { } class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamespaces { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - - protected val namespaces: util.Map[List[String], Map[String, String]] = - new ConcurrentHashMap[List[String], Map[String, String]]() - private def allNamespaces: Seq[Seq[String]] = { (tables.keySet.asScala.map(_.namespace.toSeq) ++ namespaces.keySet.asScala).toSeq.distinct } @@ -181,9 +180,8 @@ class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamesp } override def dropNamespace(namespace: Array[String]): Boolean = { - if (listTables(namespace).nonEmpty) { - throw new IllegalStateException(s"Cannot delete non-empty namespace: ${namespace.quoted}") - } + listNamespaces(namespace).map(dropNamespace) + listTables(namespace).map(dropTable) Option(namespaces.remove(namespace.toList)).isDefined } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala index 265a940d66a65..ec33a16bbebf1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/TableCatalogSuite.scala @@ -847,19 +847,16 @@ class TableCatalogSuite extends SparkFunSuite { assert(catalog.namespaceExists(testNs) === false) } - test("dropNamespace: fail if not empty") { + test("dropNamespace: drop even if it's not empty") { val catalog = newCatalog() catalog.createNamespace(testNs, Map("property" -> "value").asJava) catalog.createTable(testIdent, schema, Array.empty, emptyProps) - val exc = intercept[IllegalStateException] { - catalog.dropNamespace(testNs) - } + assert(catalog.dropNamespace(testNs)) - assert(exc.getMessage.contains(testNs.quoted)) - assert(catalog.namespaceExists(testNs) === true) - assert(catalog.loadNamespaceMetadata(testNs).asScala === Map("property" -> "value")) + assert(!catalog.namespaceExists(testNs)) + assert(catalog.listTables(testNs).isEmpty) } test("alterNamespace: basic behavior") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala index cd63ca6628abb..f7b4317ad65e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala @@ -21,26 +21,36 @@ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.SupportsNamespaces +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, SupportsNamespaces} /** * Physical plan node for dropping a namespace. */ case class DropNamespaceExec( - catalog: SupportsNamespaces, + catalog: CatalogPlugin, namespace: Seq[String], ifExists: Boolean, cascade: Boolean) extends V2CommandExec { override protected def run(): Seq[InternalRow] = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val nsCatalog = catalog.asNamespaceCatalog val ns = namespace.toArray - if (catalog.namespaceExists(ns)) { - try { - catalog.dropNamespace(ns) - } catch { - case e: IllegalStateException if cascade => + if (nsCatalog.namespaceExists(ns)) { + // The default behavior of `SupportsNamespace.dropNamespace()` is cascading, + // so make sure the namespace to drop is empty. + if (!cascade) { + if (catalog.asTableCatalog.listTables(ns).nonEmpty + || nsCatalog.listNamespaces(ns).nonEmpty) { throw new SparkException( - "Cascade option for droping namespace is not supported in V2 catalog", e) + s"Cannot drop a non-empty namespace: ${namespace.quoted}. " + + "Use CASCADE option to drop a non-empty namespace.") + } + } + + if (!nsCatalog.dropNamespace(ns)) { + throw new SparkException(s"Failed to drop a namespace: ${namespace.quoted}.") } } else if (!ifExists) { throw new NoSuchNamespaceException(ns) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 7defd669b5f40..3eb04ee129997 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -822,20 +822,44 @@ class DataSourceV2SQLSuite testShowNamespaces("SHOW NAMESPACES IN testcat", Seq()) } - test("DropNamespace: drop non-empty namespace") { + test("DropNamespace: drop non-empty namespace with a non-cascading mode") { sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") + sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo") testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1")) + testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns2")) - val e1 = intercept[IllegalStateException] { - sql("DROP NAMESPACE testcat.ns1") + def assertDropFails(): Unit = { + val e = intercept[SparkException] { + sql("DROP NAMESPACE testcat.ns1") + } + assert(e.getMessage.contains("Cannot drop a non-empty namespace: ns1")) } - assert(e1.getMessage.contains("Cannot delete non-empty namespace: ns1")) - val e2 = intercept[SparkException] { - sql("DROP NAMESPACE testcat.ns1 CASCADE") - } - assert(e2.getMessage.contains( - "Cascade option for droping namespace is not supported in V2 catalog")) + // testcat.ns1.table is present, thus testcat.ns1 cannot be dropped. + assertDropFails() + sql("DROP TABLE testcat.ns1.table") + + // testcat.ns1.ns2.table is present, thus testcat.ns1 cannot be dropped. + assertDropFails() + sql("DROP TABLE testcat.ns1.ns2.table") + + // testcat.ns1.ns2 namespace is present, thus testcat.ns1 cannot be dropped. + assertDropFails() + sql("DROP NAMESPACE testcat.ns1.ns2") + + // Now that testcat.ns1 is empty, it can be dropped. + sql("DROP NAMESPACE testcat.ns1") + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq()) + } + + test("DropNamespace: drop non-empty namespace with a cascade mode") { + sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") + sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo") + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1")) + testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns2")) + + sql("DROP NAMESPACE testcat.ns1 CASCADE") + testShowNamespaces("SHOW NAMESPACES IN testcat", Seq()) } test("DropNamespace: test handling of 'IF EXISTS'") { From d7bdc6aa1700110ddf9ac41736d2702fb937453c Mon Sep 17 00:00:00 2001 From: xy_xin Date: Wed, 13 Nov 2019 20:53:12 +0800 Subject: [PATCH 0596/1387] [SPARK-29835][SQL] Remove the unnecessary conversion from Statement to LogicalPlan for DELETE/UPDATE ### What changes were proposed in this pull request? The current parse and analyze flow for DELETE is: 1, the SQL string will be firstly parsed to `DeleteFromStatement`; 2, the `DeleteFromStatement` be converted to `DeleteFromTable`. However, the SQL string can be parsed to `DeleteFromTable` directly, where a `DeleteFromStatement` seems to be redundant. It is the same for UPDATE. This pr removes the unnecessary `DeleteFromStatement` and `UpdateTableStatement`. ### Why are the changes needed? This makes the codes for DELETE and UPDATE cleaner, and keep align with MERGE INTO. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existed tests and new tests. Closes #26464 from xianyinxin/SPARK-29835. Authored-by: xy_xin Signed-off-by: Wenchen Fan --- .../catalyst/analysis/ResolveCatalogs.scala | 13 ----- .../sql/catalyst/parser/AstBuilder.scala | 20 +++---- .../catalyst/plans/logical/statements.scala | 18 ------ .../catalyst/plans/logical/v2Commands.scala | 3 +- .../sql/catalyst/parser/DDLParserSuite.scala | 30 +++++----- .../analysis/ResolveSessionCatalog.scala | 11 ---- .../datasources/v2/DataSourceV2Strategy.scala | 30 ++++++---- .../sql/connector/DataSourceV2SQLSuite.scala | 17 +++++- .../command/PlanResolutionSuite.scala | 55 +++++++++++++++---- 9 files changed, 98 insertions(+), 99 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index b831e83ee21d5..f1a8e5bfda4a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -93,19 +93,6 @@ class ResolveCatalogs(val catalogManager: CatalogManager) s"Can not specify catalog `${catalog.name}` for view ${tableName.quoted} " + s"because view support in catalog has not been implemented yet") - case DeleteFromStatement( - nameParts @ NonSessionCatalog(catalog, tableName), tableAlias, condition) => - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) - val aliased = tableAlias.map(SubqueryAlias(_, r)).getOrElse(r) - DeleteFromTable(aliased, condition) - - case u @ UpdateTableStatement( - nameParts @ CatalogAndIdentifierParts(catalog, tableName), _, _, _, _) => - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) - val aliased = u.tableAlias.map(SubqueryAlias(_, r)).getOrElse(r) - val columns = u.columns.map(UnresolvedAttribute(_)) - UpdateTable(aliased, columns, u.values, u.condition) - case DescribeTableStatement( nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => if (partitionSpec.nonEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 15c17b59db0cc..c623b5c4d36a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -357,35 +357,29 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitDeleteFromTable( ctx: DeleteFromTableContext): LogicalPlan = withOrigin(ctx) { - val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) + val table = UnresolvedRelation(visitMultipartIdentifier(ctx.multipartIdentifier())) val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "DELETE") + val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) val predicate = if (ctx.whereClause() != null) { Some(expression(ctx.whereClause().booleanExpression())) } else { None } - - DeleteFromStatement(tableId, tableAlias, predicate) + DeleteFromTable(aliasedTable, predicate) } override def visitUpdateTable(ctx: UpdateTableContext): LogicalPlan = withOrigin(ctx) { - val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) + val table = UnresolvedRelation(visitMultipartIdentifier(ctx.multipartIdentifier())) val tableAlias = getTableAliasWithoutColumnAlias(ctx.tableAlias(), "UPDATE") - val (attrs, values) = ctx.setClause().assignmentList().assignment().asScala.map { - kv => visitMultipartIdentifier(kv.key) -> expression(kv.value) - }.unzip + val aliasedTable = tableAlias.map(SubqueryAlias(_, table)).getOrElse(table) + val assignments = withAssignments(ctx.setClause().assignmentList()) val predicate = if (ctx.whereClause() != null) { Some(expression(ctx.whereClause().booleanExpression())) } else { None } - UpdateTableStatement( - tableId, - tableAlias, - attrs, - values, - predicate) + UpdateTable(aliasedTable, assignments, predicate) } private def withAssignments(assignCtx: SqlBaseParser.AssignmentListContext): Seq[Assignment] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index a83fe43b84b4a..ec373d95fad88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -277,24 +277,6 @@ case class DescribeColumnStatement( colNameParts: Seq[String], isExtended: Boolean) extends ParsedStatement -/** - * A DELETE FROM statement, as parsed from SQL. - */ -case class DeleteFromStatement( - tableName: Seq[String], - tableAlias: Option[String], - condition: Option[Expression]) extends ParsedStatement - -/** - * A UPDATE tbl_name statement, as parsed from SQL. - */ -case class UpdateTableStatement( - tableName: Seq[String], - tableAlias: Option[String], - columns: Seq[Seq[String]], - values: Seq[Expression], - condition: Option[Expression]) extends ParsedStatement - /** * An INSERT INTO statement, as parsed from SQL. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 6dc742610d04b..7d8e9a0c18f65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -290,8 +290,7 @@ case class DeleteFromTable( */ case class UpdateTable( table: LogicalPlan, - columns: Seq[Expression], - values: Seq[Expression], + assignments: Seq[Assignment], condition: Option[Expression]) extends Command with SupportsSubquery { override def children: Seq[LogicalPlan] = table :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index a8938759ba45a..94171feba2ac7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -772,17 +772,15 @@ class DDLParserSuite extends AnalysisTest { test("delete from table: delete all") { parseCompare("DELETE FROM testcat.ns1.ns2.tbl", - DeleteFromStatement( - Seq("testcat", "ns1", "ns2", "tbl"), - None, + DeleteFromTable( + UnresolvedRelation(Seq("testcat", "ns1", "ns2", "tbl")), None)) } test("delete from table: with alias and where clause") { parseCompare("DELETE FROM testcat.ns1.ns2.tbl AS t WHERE t.a = 2", - DeleteFromStatement( - Seq("testcat", "ns1", "ns2", "tbl"), - Some("t"), + DeleteFromTable( + SubqueryAlias("t", UnresolvedRelation(Seq("testcat", "ns1", "ns2", "tbl"))), Some(EqualTo(UnresolvedAttribute("t.a"), Literal(2))))) } @@ -798,13 +796,12 @@ class DDLParserSuite extends AnalysisTest { parseCompare( """ |UPDATE testcat.ns1.ns2.tbl - |SET t.a='Robert', t.b=32 + |SET a='Robert', b=32 """.stripMargin, - UpdateTableStatement( - Seq("testcat", "ns1", "ns2", "tbl"), - None, - Seq(Seq("t", "a"), Seq("t", "b")), - Seq(Literal("Robert"), Literal(32)), + UpdateTable( + UnresolvedRelation(Seq("testcat", "ns1", "ns2", "tbl")), + Seq(Assignment(UnresolvedAttribute("a"), Literal("Robert")), + Assignment(UnresolvedAttribute("b"), Literal(32))), None)) } @@ -815,11 +812,10 @@ class DDLParserSuite extends AnalysisTest { |SET t.a='Robert', t.b=32 |WHERE t.c=2 """.stripMargin, - UpdateTableStatement( - Seq("testcat", "ns1", "ns2", "tbl"), - Some("t"), - Seq(Seq("t", "a"), Seq("t", "b")), - Seq(Literal("Robert"), Literal(32)), + UpdateTable( + SubqueryAlias("t", UnresolvedRelation(Seq("testcat", "ns1", "ns2", "tbl"))), + Seq(Assignment(UnresolvedAttribute("t.a"), Literal("Robert")), + Assignment(UnresolvedAttribute("t.b"), Literal(32))), Some(EqualTo(UnresolvedAttribute("t.c"), Literal(2))))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 280f4d9d7a889..340e09ae66adb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -158,17 +158,6 @@ class ResolveSessionCatalog( case AlterViewUnsetPropertiesStatement(SessionCatalog(catalog, tableName), keys, ifExists) => AlterTableUnsetPropertiesCommand(tableName.asTableIdentifier, keys, ifExists, isView = true) - case DeleteFromStatement( - nameParts @ SessionCatalog(catalog, tableName), tableAlias, condition) => - loadTable(catalog, tableName.asIdentifier).collect { - case v1Table: V1Table => - throw new AnalysisException("DELETE FROM is only supported with v2 tables.") - }.getOrElse { - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) - val aliased = tableAlias.map(SubqueryAlias(_, r)).getOrElse(r) - DeleteFromTable(aliased, condition) - } - case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 5115da4a39c70..01ff4a9303e98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -157,19 +157,25 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { OverwritePartitionsDynamicExec( r.table.asWritable, writeOptions.asOptions, planLater(query)) :: Nil - case DeleteFromTable(DataSourceV2ScanRelation(table, _, output), condition) => - if (condition.exists(SubqueryExpression.hasSubquery)) { - throw new AnalysisException( - s"Delete by condition with subquery is not supported: $condition") + case DeleteFromTable(relation, condition) => + relation match { + case DataSourceV2ScanRelation(table, _, output) => + if (condition.exists(SubqueryExpression.hasSubquery)) { + throw new AnalysisException( + s"Delete by condition with subquery is not supported: $condition") + } + // fail if any filter cannot be converted. + // correctness depends on removing all matching data. + val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, output) + .flatMap(splitConjunctivePredicates(_).map { + f => DataSourceStrategy.translateFilter(f).getOrElse( + throw new AnalysisException(s"Exec update failed:" + + s" cannot translate expression to source filter: $f")) + }).toArray + DeleteFromTableExec(table.asDeletable, filters) :: Nil + case _ => + throw new AnalysisException("DELETE is only supported with v2 tables.") } - // fail if any filter cannot be converted. correctness depends on removing all matching data. - val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, output) - .flatMap(splitConjunctivePredicates(_).map { - f => DataSourceStrategy.translateFilter(f).getOrElse( - throw new AnalysisException(s"Exec update failed:" + - s" cannot translate expression to source filter: $f")) - }).toArray - DeleteFromTableExec(table.asDeletable, filters) :: Nil case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 3eb04ee129997..3dd6b8f65da1a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1313,6 +1313,21 @@ class DataSourceV2SQLSuite } } + test("DeleteFrom: DELETE is only supported with v2 tables") { + // unset this config to use the default v2 session catalog. + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + val v1Table = "tbl" + withTable(v1Table) { + sql(s"CREATE TABLE $v1Table" + + s" USING ${classOf[SimpleScanSource].getName} OPTIONS (from=0,to=1)") + val exc = intercept[AnalysisException] { + sql(s"DELETE FROM $v1Table WHERE i = 2") + } + + assert(exc.getMessage.contains("DELETE is only supported with v2 tables")) + } + } + test("UPDATE TABLE") { val t = "testcat.ns1.ns2.tbl" withTable(t) { @@ -1326,7 +1341,7 @@ class DataSourceV2SQLSuite // UPDATE non-existing table assertAnalysisError( "UPDATE dummy SET name='abc'", - "Table not found") + "Table or view not found") // UPDATE non-existing column assertAnalysisError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index ffe80a0ac3dbb..50e35289b0a64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, EmptyFunc import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{EqualTo, IntegerLiteral, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DescribeTable, DropTable, InsertAction, LogicalPlan, MergeIntoTable, Project, SubqueryAlias, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeTable, DropTable, InsertAction, LogicalPlan, MergeIntoTable, Project, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.InMemoryTableProvider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.execution.datasources.CreateTable @@ -854,8 +854,41 @@ class PlanResolutionSuite extends AnalysisTest { assert(parsed4.isInstanceOf[DescribeTableCommand]) } + test("DELETE FROM") { + Seq("v2Table", "testcat.tab").foreach { tblName => + val sql1 = s"DELETE FROM $tblName" + val sql2 = s"DELETE FROM $tblName where name='Robert'" + val sql3 = s"DELETE FROM $tblName AS t where t.name='Robert'" + + val parsed1 = parseAndResolve(sql1) + val parsed2 = parseAndResolve(sql2) + val parsed3 = parseAndResolve(sql3) + + parsed1 match { + case DeleteFromTable(_: DataSourceV2Relation, None) => + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed1.treeString) + } + + parsed2 match { + case DeleteFromTable( + _: DataSourceV2Relation, + Some(EqualTo(name: UnresolvedAttribute, StringLiteral("Robert")))) => + assert(name.name == "name") + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed1.treeString) + } + + parsed3 match { + case DeleteFromTable( + SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + Some(EqualTo(name: UnresolvedAttribute, StringLiteral("Robert")))) => + assert(name.name == "t.name") + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed1.treeString) + } + } + } + test("UPDATE TABLE") { - Seq("v1Table", "v2Table", "testcat.tab").foreach { tblName => + Seq("v2Table", "testcat.tab").foreach { tblName => val sql1 = s"UPDATE $tblName SET name='Robert', age=32" val sql2 = s"UPDATE $tblName AS t SET name='Robert', age=32" val sql3 = s"UPDATE $tblName AS t SET name='Robert', age=32 WHERE p=1" @@ -865,10 +898,10 @@ class PlanResolutionSuite extends AnalysisTest { val parsed3 = parseAndResolve(sql3) parsed1 match { - case u @ UpdateTable( + case UpdateTable( _: DataSourceV2Relation, - Seq(name: UnresolvedAttribute, age: UnresolvedAttribute), - Seq(StringLiteral("Robert"), IntegerLiteral(32)), + Seq(Assignment(name: UnresolvedAttribute, StringLiteral("Robert")), + Assignment(age: UnresolvedAttribute, IntegerLiteral(32))), None) => assert(name.name == "name") assert(age.name == "age") @@ -879,8 +912,8 @@ class PlanResolutionSuite extends AnalysisTest { parsed2 match { case UpdateTable( SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), - Seq(name: UnresolvedAttribute, age: UnresolvedAttribute), - Seq(StringLiteral("Robert"), IntegerLiteral(32)), + Seq(Assignment(name: UnresolvedAttribute, StringLiteral("Robert")), + Assignment(age: UnresolvedAttribute, IntegerLiteral(32))), None) => assert(name.name == "name") assert(age.name == "age") @@ -891,8 +924,8 @@ class PlanResolutionSuite extends AnalysisTest { parsed3 match { case UpdateTable( SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), - Seq(name: UnresolvedAttribute, age: UnresolvedAttribute), - Seq(StringLiteral("Robert"), IntegerLiteral(32)), + Seq(Assignment(name: UnresolvedAttribute, StringLiteral("Robert")), + Assignment(age: UnresolvedAttribute, IntegerLiteral(32))), Some(EqualTo(p: UnresolvedAttribute, IntegerLiteral(1)))) => assert(name.name == "name") assert(age.name == "age") @@ -906,7 +939,7 @@ class PlanResolutionSuite extends AnalysisTest { val parsed = parseAndResolve(sql) parsed match { case u: UpdateTable => - assert(u.table.isInstanceOf[UnresolvedV2Relation]) + assert(u.table.isInstanceOf[UnresolvedRelation]) case _ => fail("Expect UpdateTable, but got:\n" + parsed.treeString) } } @@ -1193,8 +1226,6 @@ class PlanResolutionSuite extends AnalysisTest { assert(u.sourceTable.isInstanceOf[UnresolvedRelation]) case _ => fail("Expect MergeIntoTable, but got:\n" + parsed.treeString) } - - // TODO: v1 table is not supported. } // TODO: add tests for more commands. From 56be7318cc0a26758fee76e7e0590c6d3434839a Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Wed, 13 Nov 2019 07:06:08 -0600 Subject: [PATCH 0597/1387] [MINOR][BUILD] Fix an incorrect path in license-binary file ### What changes were proposed in this pull request? I want to say sorry! this PR follows the previous https://github.com/apache/spark/pull/26050. I didn't find them at the same time. The `LICENSE-binary` file exists a minor issue has an incorrect path. This PR will fix it. ### Why are the changes needed? This is a minor bug. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exists UT. Closes #26490 from beliefer/fix-minor-license-issue. Authored-by: gengjiaan Signed-off-by: Sean Owen --- LICENSE-binary | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/LICENSE-binary b/LICENSE-binary index 6858193515a87..0b45686f9ba39 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -502,7 +502,7 @@ com.github.scopt:scopt_2.12 core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js core/src/main/resources/org/apache/spark/ui/static/*dataTables* core/src/main/resources/org/apache/spark/ui/static/graphlib-dot.min.js -ore/src/main/resources/org/apache/spark/ui/static/jquery* +core/src/main/resources/org/apache/spark/ui/static/jquery* core/src/main/resources/org/apache/spark/ui/static/sorttable.js docs/js/vendor/anchor.min.js docs/js/vendor/jquery* From 2beca777b691cdcc7e46ee64628f67e5b7834ae5 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 13 Nov 2019 07:10:20 -0600 Subject: [PATCH 0598/1387] [SPARK-28795][FOLLOW-UP] Links should point to html instead of md files ### What changes were proposed in this pull request? Use html files for the links ### Why are the changes needed? links not working ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Used jekyll build and serve to verify. Closes #26494 from huaxingao/spark-28795. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- docs/sql-ref-syntax-ddl-create-view.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md index c7ca28ea5b62f..dcafdbb750d0c 100644 --- a/docs/sql-ref-syntax-ddl-create-view.md +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -78,5 +78,5 @@ CREATE GLOBAL TEMPORARY VIEW IF NOT EXISTS subscribed_movies {% endhighlight %} ### Related Statements -- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.md) -- [DROP VIEW](sql-ref-syntax-ddl-drop-view.md) +- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) +- [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) From 0c8d3d2a15a620eedf5b9c33b59fd94906d11dcd Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 13 Nov 2019 07:11:26 -0600 Subject: [PATCH 0599/1387] [SPARK-28798][FOLLOW-UP] Add alter view link to drop view ### What changes were proposed in this pull request? Add alter view link to drop view ### Why are the changes needed? create view has links to drop view and alter view alter view has links to create view and drop view drop view currently doesn't have a link to alter view. I think it's better to link to alter view as well. ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Tested using jykyll build --serve Closes #26495 from huaxingao/spark-28798. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- docs/sql-ref-syntax-ddl-drop-view.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sql-ref-syntax-ddl-drop-view.md b/docs/sql-ref-syntax-ddl-drop-view.md index f095a34567721..bf99150048d49 100644 --- a/docs/sql-ref-syntax-ddl-drop-view.md +++ b/docs/sql-ref-syntax-ddl-drop-view.md @@ -77,5 +77,6 @@ DROP VIEW IF EXISTS employeeView; ### Related Statements - [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) +- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) - [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) - [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) From 942753a44beeae5f0142ceefa307e90cbc1234c5 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 13 Nov 2019 21:27:36 +0800 Subject: [PATCH 0600/1387] [SPARK-29753][SQL] refine the default catalog config ### What changes were proposed in this pull request? rename the config to address the comment: https://github.com/apache/spark/pull/24594#discussion_r285431212 improve the config description, provide a default value to simplify the code. ### Why are the changes needed? make the config more understandable. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26395 from cloud-fan/config. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/connector/catalog/CatalogManager.scala | 16 +--------------- .../org/apache/spark/sql/internal/SQLConf.scala | 10 ++++------ .../connector/catalog/CatalogManagerSuite.scala | 4 ++-- .../sql/connector/DataSourceV2SQLSuite.scala | 10 +++++----- 4 files changed, 12 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index c9d050768c15e..d62148b2bbe45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -53,18 +53,6 @@ class CatalogManager( } } - private def defaultCatalog: Option[CatalogPlugin] = { - conf.defaultV2Catalog.flatMap { catalogName => - try { - Some(catalog(catalogName)) - } catch { - case NonFatal(e) => - logError(s"Cannot load default v2 catalog: $catalogName", e) - None - } - } - } - private def loadV2SessionCatalog(): CatalogPlugin = { Catalogs.load(SESSION_CATALOG_NAME, conf) match { case extension: CatalogExtension => @@ -127,9 +115,7 @@ class CatalogManager( private var _currentCatalogName: Option[String] = None def currentCatalog: CatalogPlugin = synchronized { - _currentCatalogName.map(catalogName => catalog(catalogName)) - .orElse(defaultCatalog) - .getOrElse(v2SessionCatalog) + catalog(_currentCatalogName.getOrElse(conf.getConf(SQLConf.DEFAULT_CATALOG))) } def setCurrentCatalog(catalogName: String): Unit = synchronized { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b704f4875f828..98acace79601b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -37,7 +37,6 @@ import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator import org.apache.spark.sql.catalyst.plans.logical.HintErrorHandler import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME -import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils @@ -2019,10 +2018,11 @@ object SQLConf { .booleanConf .createWithDefault(false) - val DEFAULT_V2_CATALOG = buildConf("spark.sql.default.catalog") - .doc("Name of the default v2 catalog, used when a catalog is not identified in queries") + val DEFAULT_CATALOG = buildConf("spark.sql.defaultCatalog") + .doc("Name of the default catalog. This will be the current catalog if users have not " + + "explicitly set the current catalog yet.") .stringConf - .createOptional + .createWithDefault(SESSION_CATALOG_NAME) val V2_SESSION_CATALOG_IMPLEMENTATION = buildConf(s"spark.sql.catalog.$SESSION_CATALOG_NAME") @@ -2556,8 +2556,6 @@ class SQLConf extends Serializable with Logging { def castDatetimeToString: Boolean = getConf(SQLConf.LEGACY_CAST_DATETIME_TO_STRING) - def defaultV2Catalog: Option[String] = getConf(DEFAULT_V2_CATALOG) - def ignoreDataLocality: Boolean = getConf(SQLConf.IGNORE_DATA_LOCALITY) /** ********************** SQLConf functionality methods ************ */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index fcbf139fd97c1..fc7b2e720b7b7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -43,7 +43,7 @@ class CatalogManagerSuite extends SparkFunSuite { assert(catalogManager.currentNamespace.sameElements(Array("default"))) conf.setConfString("spark.sql.catalog.dummy", classOf[DummyCatalog].getName) - conf.setConfString(SQLConf.DEFAULT_V2_CATALOG.key, "dummy") + conf.setConfString(SQLConf.DEFAULT_CATALOG.key, "dummy") // The current catalog should be changed if the default catalog is set. assert(catalogManager.currentCatalog.name() == "dummy") @@ -60,7 +60,7 @@ class CatalogManagerSuite extends SparkFunSuite { assert(catalogManager.currentNamespace.sameElements(Array("a", "b"))) conf.setConfString("spark.sql.catalog.dummy2", classOf[DummyCatalog].getName) - conf.setConfString(SQLConf.DEFAULT_V2_CATALOG.key, "dummy2") + conf.setConfString(SQLConf.DEFAULT_CATALOG.key, "dummy2") // The current catalog shouldn't be changed if it's set before. assert(catalogManager.currentCatalog.name() == "dummy") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 3dd6b8f65da1a..17f6e51f8454c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -222,7 +222,7 @@ class DataSourceV2SQLSuite } test("CreateTable: use default catalog for v2 sources when default catalog is set") { - spark.conf.set("spark.sql.default.catalog", "testcat") + spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING foo") val testCatalog = catalog("testcat").asTableCatalog @@ -489,7 +489,7 @@ class DataSourceV2SQLSuite } test("CreateTableAsSelect: use default catalog for v2 sources when default catalog is set") { - spark.conf.set("spark.sql.default.catalog", "testcat") + spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -710,7 +710,7 @@ class DataSourceV2SQLSuite } test("ShowTables: namespace is not specified and default v2 catalog is set") { - spark.conf.set("spark.sql.default.catalog", "testcat") + spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") spark.sql("CREATE TABLE testcat.table (id bigint, data string) USING foo") // v2 catalog is used where default namespace is empty for TestInMemoryTableCatalog. @@ -872,7 +872,7 @@ class DataSourceV2SQLSuite } test("ShowNamespaces: show root namespaces with default v2 catalog") { - spark.conf.set("spark.sql.default.catalog", "testcat") + spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") testShowNamespaces("SHOW NAMESPACES", Seq()) @@ -915,7 +915,7 @@ class DataSourceV2SQLSuite spark.conf.set( "spark.sql.catalog.testcat_no_namspace", classOf[BasicInMemoryTableCatalog].getName) - spark.conf.set("spark.sql.default.catalog", "testcat_no_namspace") + spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat_no_namspace") val exception = intercept[AnalysisException] { sql("SHOW NAMESPACES") From 4dcbdcd265eac2cea1ac7908b1e8cacb1a4a2db5 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 13 Nov 2019 21:42:42 +0800 Subject: [PATCH 0601/1387] [SPARK-29863][SQL] Rename EveryAgg/AnyAgg to BoolAnd/BoolOr ### What changes were proposed in this pull request? rename EveryAgg/AnyAgg to BoolAnd/BoolOr ### Why are the changes needed? Under ansi mode, `every`, `any` and `some` are reserved keywords and can't be used as function names. `EveryAgg`/`AnyAgg` has several aliases and I think it's better to not pick reserved keywords as the primary name. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26486 from cloud-fan/naming. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 10 +++--- .../aggregate/UnevaluableAggs.scala | 8 ++--- .../catalyst/optimizer/finishAnalysis.scala | 4 +-- .../ExpressionTypeCheckingSuite.scala | 4 +-- .../sql-tests/results/group-by.sql.out | 34 +++++++++---------- .../results/udf/udf-group-by.sql.out | 26 +++++++------- .../org/apache/spark/sql/ExplainSuite.scala | 4 +-- 7 files changed, 45 insertions(+), 45 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index dc8757293b302..cb18aa1a9479b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -313,11 +313,11 @@ object FunctionRegistry { expression[CollectList]("collect_list"), expression[CollectSet]("collect_set"), expression[CountMinSketchAgg]("count_min_sketch"), - expression[EveryAgg]("every"), - expression[EveryAgg]("bool_and"), - expression[AnyAgg]("any"), - expression[AnyAgg]("some"), - expression[AnyAgg]("bool_or"), + expression[BoolAnd]("every"), + expression[BoolAnd]("bool_and"), + expression[BoolOr]("any"), + expression[BoolOr]("some"), + expression[BoolOr]("bool_or"), // string functions expression[Ascii]("ascii"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala index a8220ec641d6a..c559fefe3a80c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala @@ -52,8 +52,8 @@ abstract class UnevaluableBooleanAggBase(arg: Expression) false """, since = "3.0.0") -case class EveryAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = "Every" +case class BoolAnd(arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = "bool_and" } @ExpressionDescription( @@ -68,6 +68,6 @@ case class EveryAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { false """, since = "3.0.0") -case class AnyAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = "Any" +case class BoolOr(arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = "bool_or" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 70277526cba8c..f64b6e00373f6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -47,8 +47,8 @@ object ReplaceExpressions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case e: RuntimeReplaceable => e.child case CountIf(predicate) => Count(new NullIf(predicate, Literal.FalseLiteral)) - case AnyAgg(arg) => Max(arg) - case EveryAgg(arg) => Min(arg) + case BoolOr(arg) => Max(arg) + case BoolAnd(arg) => Min(arg) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 93ea3221e7472..feb927264ba68 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -144,8 +144,8 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertSuccess(Sum('stringField)) assertSuccess(Average('stringField)) assertSuccess(Min('arrayField)) - assertSuccess(new EveryAgg('booleanField)) - assertSuccess(new AnyAgg('booleanField)) + assertSuccess(new BoolAnd('booleanField)) + assertSuccess(new BoolOr('booleanField)) assertError(Min('mapField), "min does not support ordering on type") assertError(Max('mapField), "max does not support ordering on type") diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 150ee8aab01e2..6a40702f6eb0a 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -293,7 +293,7 @@ struct<> -- !query 31 SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output NULL NULL NULL NULL NULL @@ -301,7 +301,7 @@ NULL NULL NULL NULL NULL -- !query 32 SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output NULL NULL NULL NULL NULL @@ -309,7 +309,7 @@ NULL NULL NULL NULL NULL -- !query 33 SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output false true true false true @@ -317,7 +317,7 @@ false true true false true -- !query 34 SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k -- !query 34 schema -struct +struct -- !query 34 output 1 false true true false true 2 true true true true true @@ -329,7 +329,7 @@ struct +struct -- !query 35 output 1 false 3 false @@ -339,7 +339,7 @@ struct -- !query 36 SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL -- !query 36 schema -struct +struct -- !query 36 output 4 NULL @@ -380,7 +380,7 @@ SELECT every(1) struct<> -- !query 39 output org.apache.spark.sql.AnalysisException -cannot resolve 'every(1)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 +cannot resolve 'bool_and(1)' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [int].; line 1 pos 7 -- !query 40 @@ -389,7 +389,7 @@ SELECT some(1S) struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve 'any(1S)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [smallint].; line 1 pos 7 +cannot resolve 'bool_or(1S)' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 @@ -398,7 +398,7 @@ SELECT any(1L) struct<> -- !query 41 output org.apache.spark.sql.AnalysisException -cannot resolve 'any(1L)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 +cannot resolve 'bool_or(1L)' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [bigint].; line 1 pos 7 -- !query 42 @@ -407,7 +407,7 @@ SELECT every("true") struct<> -- !query 42 output org.apache.spark.sql.AnalysisException -cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7 +cannot resolve 'bool_and('true')' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [string].; line 1 pos 7 -- !query 43 @@ -416,7 +416,7 @@ SELECT bool_and(1.0) struct<> -- !query 43 output org.apache.spark.sql.AnalysisException -cannot resolve 'every(1.0BD)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [decimal(2,1)].; line 1 pos 7 +cannot resolve 'bool_and(1.0BD)' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [decimal(2,1)].; line 1 pos 7 -- !query 44 @@ -425,13 +425,13 @@ SELECT bool_or(1.0D) struct<> -- !query 44 output org.apache.spark.sql.AnalysisException -cannot resolve 'any(1.0D)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [double].; line 1 pos 7 +cannot resolve 'bool_or(1.0D)' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [double].; line 1 pos 7 -- !query 45 SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg -- !query 45 schema -struct +struct -- !query 45 output 1 false false 1 true false @@ -448,7 +448,7 @@ struct +struct -- !query 46 output 1 false false 1 true true @@ -465,7 +465,7 @@ struct +struct -- !query 47 output 1 false false 1 true true @@ -482,7 +482,7 @@ struct +struct -- !query 48 output 1 false false 1 true false @@ -499,7 +499,7 @@ struct +struct -- !query 49 output 1 false false 1 true true diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index b762d18fb8390..ea2cab703eaae 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -293,7 +293,7 @@ struct<> -- !query 31 SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output NULL NULL NULL @@ -301,7 +301,7 @@ NULL NULL NULL -- !query 32 SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output NULL NULL NULL @@ -309,7 +309,7 @@ NULL NULL NULL -- !query 33 SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output false true true @@ -317,7 +317,7 @@ false true true -- !query 34 SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) -- !query 34 schema -struct +struct -- !query 34 output 1 false true true 2 true true true @@ -329,7 +329,7 @@ struct +struct -- !query 35 output 1 false 3 false @@ -339,7 +339,7 @@ struct -- !query 36 SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL -- !query 36 schema -struct +struct -- !query 36 output 4 NULL @@ -380,7 +380,7 @@ SELECT every(udf(1)) struct<> -- !query 39 output org.apache.spark.sql.AnalysisException -cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 +cannot resolve 'bool_and(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [int].; line 1 pos 7 -- !query 40 @@ -389,7 +389,7 @@ SELECT some(udf(1S)) struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve 'any(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [smallint].; line 1 pos 7 +cannot resolve 'bool_or(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 @@ -398,7 +398,7 @@ SELECT any(udf(1L)) struct<> -- !query 41 output org.apache.spark.sql.AnalysisException -cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 +cannot resolve 'bool_or(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [bigint].; line 1 pos 7 -- !query 42 @@ -407,13 +407,13 @@ SELECT udf(every("true")) struct<> -- !query 42 output org.apache.spark.sql.AnalysisException -cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11 +cannot resolve 'bool_and('true')' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [string].; line 1 pos 11 -- !query 43 SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg -- !query 43 schema -struct +struct -- !query 43 output 1 false false 1 true false @@ -430,7 +430,7 @@ struct +struct -- !query 44 output 1 false false 1 true true @@ -447,7 +447,7 @@ struct +struct -- !query 45 output 1 false false 1 true true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index e70aeae9e605e..d5c16a30ade5b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -95,8 +95,8 @@ class ExplainSuite extends QueryTest with SharedSparkSession { // plan should show the rewritten aggregate expression. val df = sql("SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k") checkKeywordsExistsInExplain(df, - "Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS any(v)#x, " + - "max(v#x) AS any(v)#x]") + "Aggregate [k#x], [k#x, min(v#x) AS bool_and(v)#x, max(v#x) AS bool_or(v)#x, " + + "max(v#x) AS bool_or(v)#x]") } } From 8c2bf64743e898efc6e5295058cd13c789a04dfa Mon Sep 17 00:00:00 2001 From: Aman Omer Date: Wed, 13 Nov 2019 08:16:06 -0600 Subject: [PATCH 0602/1387] [SPARK-29823][MLLIB] Improper persist strategy in mllib.clustering.KMeans.run() ### What changes were proposed in this pull request? Adjust RDD to persist. ### Why are the changes needed? To handle the improper persist strategy. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually Closes #26483 from amanomer/SPARK-29823. Authored-by: Aman Omer Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/mllib/clustering/KMeans.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 278d61d916735..ac85fbc235c93 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -223,12 +223,12 @@ class KMeans private ( // Compute squared norms and cache them. val norms = data.map(Vectors.norm(_, 2.0)) - norms.persist() val zippedData = data.zip(norms).map { case (v, norm) => new VectorWithNorm(v, norm) } + zippedData.persist() val model = runAlgorithm(zippedData, instr) - norms.unpersist() + zippedData.unpersist() // Warn at the end of the run as well, for increased visibility. if (data.getStorageLevel == StorageLevel.NONE) { From 1f4075d29eecae69f03666400222ba732ecfdbd1 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 13 Nov 2019 08:18:23 -0600 Subject: [PATCH 0603/1387] [SPARK-29808][ML][PYTHON] StopWordsRemover should support multi-cols ### What changes were proposed in this pull request? Add multi-cols support in StopWordsRemover ### Why are the changes needed? As a basic Transformer, StopWordsRemover should support multi-cols. Param stopWords can be applied across all columns. ### Does this PR introduce any user-facing change? ```StopWordsRemover.setInputCols``` ```StopWordsRemover.setOutputCols``` ### How was this patch tested? Unit tests Closes #26480 from huaxingao/spark-29808. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../spark/ml/feature/StopWordsRemover.scala | 62 ++++++-- .../ml/feature/StopWordsRemoverSuite.scala | 133 +++++++++++++++++- python/pyspark/ml/feature.py | 39 ++++- 3 files changed, 217 insertions(+), 17 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala index 57fe7c9c0aad0..b6ed4f2b000cc 100755 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -22,15 +22,19 @@ import java.util.Locale import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared.{HasInputCol, HasInputCols, HasOutputCol, HasOutputCols} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} -import org.apache.spark.sql.types.{ArrayType, StringType, StructType} +import org.apache.spark.sql.types.{ArrayType, StringType, StructField, StructType} /** * A feature transformer that filters out stop words from input. * + * Since 3.0.0, `StopWordsRemover` can filter out multiple columns at once by setting the + * `inputCols` parameter. Note that when both the `inputCol` and `inputCols` parameters are set, + * an Exception will be thrown. + * * @note null values from input array are preserved unless adding null to stopWords * explicitly. * @@ -38,7 +42,8 @@ import org.apache.spark.sql.types.{ArrayType, StringType, StructType} */ @Since("1.5.0") class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String) - extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + extends Transformer with HasInputCol with HasOutputCol with HasInputCols with HasOutputCols + with DefaultParamsWritable { @Since("1.5.0") def this() = this(Identifiable.randomUID("stopWords")) @@ -51,6 +56,14 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ + @Since("3.0.0") + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(value: Array[String]): this.type = set(outputCols, value) + /** * The words to be filtered out. * Default: English stop words @@ -121,6 +134,15 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String } } + /** Returns the input and output column names corresponding in pair. */ + private[feature] def getInOutCols(): (Array[String], Array[String]) = { + if (isSet(inputCol)) { + (Array($(inputCol)), Array($(outputCol))) + } else { + ($(inputCols), $(outputCols)) + } + } + setDefault(stopWords -> StopWordsRemover.loadDefaultStopWords("english"), caseSensitive -> false, locale -> getDefaultOrUS.toString) @@ -142,16 +164,38 @@ class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String terms.filter(s => !lowerStopWords.contains(toLower(s))) } } - val metadata = outputSchema($(outputCol)).metadata - dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) + + val (inputColNames, outputColNames) = getInOutCols() + val ouputCols = inputColNames.map { inputColName => + t(col(inputColName)) + } + val ouputMetadata = outputColNames.map(outputSchema(_).metadata) + dataset.withColumns(outputColNames, ouputCols, ouputMetadata) } @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.sameType(ArrayType(StringType)), "Input type must be " + - s"${ArrayType(StringType).catalogString} but got ${inputType.catalogString}.") - SchemaUtils.appendColumn(schema, $(outputCol), inputType, schema($(inputCol)).nullable) + ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), + Seq(outputCols)) + + if (isSet(inputCols)) { + require(getInputCols.length == getOutputCols.length, + s"StopWordsRemover $this has mismatched Params " + + s"for multi-column transform. Params ($inputCols, $outputCols) should have " + + "equal lengths, but they have different lengths: " + + s"(${getInputCols.length}, ${getOutputCols.length}).") + } + + val (inputColNames, outputColNames) = getInOutCols() + val newCols = inputColNames.zip(outputColNames).map { case (inputColName, outputColName) => + require(!schema.fieldNames.contains(outputColName), + s"Output Column $outputColName already exists.") + val inputType = schema(inputColName).dataType + require(inputType.sameType(ArrayType(StringType)), "Input type must be " + + s"${ArrayType(StringType).catalogString} but got ${inputType.catalogString}.") + StructField(outputColName, inputType, schema(inputColName).nullable) + } + StructType(schema.fields ++ newCols) } @Since("1.5.0") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala index 6d0b83e85733e..c142f83e05956 100755 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.feature import java.util.Locale +import org.apache.spark.ml.Pipeline import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.sql.{DataFrame, Row} @@ -181,12 +182,19 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { } test("read/write") { - val t = new StopWordsRemover() + val t1 = new StopWordsRemover() .setInputCol("myInputCol") .setOutputCol("myOutputCol") .setStopWords(Array("the", "a")) .setCaseSensitive(true) - testDefaultReadWrite(t) + testDefaultReadWrite(t1) + + val t2 = new StopWordsRemover() + .setInputCols(Array("input1", "input2", "input3")) + .setOutputCols(Array("result1", "result2", "result3")) + .setStopWords(Array("the", "a")) + .setCaseSensitive(true) + testDefaultReadWrite(t2) } test("StopWordsRemover output column already exists") { @@ -199,7 +207,7 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { testTransformerByInterceptingException[(Array[String], Array[String])]( dataSet, remover, - s"requirement failed: Column $outputCol already exists.", + s"requirement failed: Output Column $outputCol already exists.", "expected") } @@ -217,4 +225,123 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { Locale.setDefault(oldDefault) } } + + test("Multiple Columns: StopWordsRemover default") { + val remover = new StopWordsRemover() + .setInputCols(Array("raw1", "raw2")) + .setOutputCols(Array("filtered1", "filtered2")) + val df = Seq( + (Seq("test", "test"), Seq("test1", "test2"), Seq("test", "test"), Seq("test1", "test2")), + (Seq("a", "b", "c", "d"), Seq("a", "b"), Seq("b", "c", "d"), Seq("b")), + (Seq("a", "the", "an"), Seq("the", "an"), Seq(), Seq()), + (Seq("A", "The", "AN"), Seq("A", "The"), Seq(), Seq()), + (Seq(null), Seq(null), Seq(null), Seq(null)), + (Seq(), Seq(), Seq(), Seq()) + ).toDF("raw1", "raw2", "expected1", "expected2") + + remover.transform(df) + .select("filtered1", "expected1", "filtered2", "expected2") + .collect().foreach { + case Row(r1: Seq[String], e1: Seq[String], r2: Seq[String], e2: Seq[String]) => + assert(r1 === e1, + s"The result value is not correct after bucketing. Expected $e1 but found $r1") + assert(r2 === e2, + s"The result value is not correct after bucketing. Expected $e2 but found $r2") + } + } + + test("Multiple Columns: StopWordsRemover with particular stop words list") { + val stopWords = Array("test", "a", "an", "the") + val remover = new StopWordsRemover() + .setInputCols(Array("raw1", "raw2")) + .setOutputCols(Array("filtered1", "filtered2")) + .setStopWords(stopWords) + val df = Seq( + (Seq("test", "test"), Seq("test1", "test2"), Seq(), Seq("test1", "test2")), + (Seq("a", "b", "c", "d"), Seq("a", "b"), Seq("b", "c", "d"), Seq("b")), + (Seq("a", "the", "an"), Seq("a", "the", "test1"), Seq(), Seq("test1")), + (Seq("A", "The", "AN"), Seq("A", "The", "AN"), Seq(), Seq()), + (Seq(null), Seq(null), Seq(null), Seq(null)), + (Seq(), Seq(), Seq(), Seq()) + ).toDF("raw1", "raw2", "expected1", "expected2") + + remover.transform(df) + .select("filtered1", "expected1", "filtered2", "expected2") + .collect().foreach { + case Row(r1: Seq[String], e1: Seq[String], r2: Seq[String], e2: Seq[String]) => + assert(r1 === e1, + s"The result value is not correct after bucketing. Expected $e1 but found $r1") + assert(r2 === e2, + s"The result value is not correct after bucketing. Expected $e2 but found $r2") + } + } + + test("Compare single/multiple column(s) StopWordsRemover in pipeline") { + val df = Seq( + (Seq("test", "test"), Seq("test1", "test2")), + (Seq("a", "b", "c", "d"), Seq("a", "b")), + (Seq("a", "the", "an"), Seq("a", "the", "test1")), + (Seq("A", "The", "AN"), Seq("A", "The", "AN")), + (Seq(null), Seq(null)), + (Seq(), Seq()) + ).toDF("input1", "input2") + + val multiColsRemover = new StopWordsRemover() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("output1", "output2")) + + val plForMultiCols = new Pipeline() + .setStages(Array(multiColsRemover)) + .fit(df) + + val removerForCol1 = new StopWordsRemover() + .setInputCol("input1") + .setOutputCol("output1") + val removerForCol2 = new StopWordsRemover() + .setInputCol("input2") + .setOutputCol("output2") + + val plForSingleCol = new Pipeline() + .setStages(Array(removerForCol1, removerForCol2)) + .fit(df) + + val resultForSingleCol = plForSingleCol.transform(df) + .select("output1", "output2") + .collect() + val resultForMultiCols = plForMultiCols.transform(df) + .select("output1", "output2") + .collect() + + resultForSingleCol.zip(resultForMultiCols).foreach { + case (rowForSingle, rowForMultiCols) => + assert(rowForSingle === rowForMultiCols) + } + } + + test("Multiple Columns: Mismatched sizes of inputCols/outputCols") { + val remover = new StopWordsRemover() + .setInputCols(Array("input1")) + .setOutputCols(Array("result1", "result2")) + val df = Seq( + (Seq("A"), Seq("A")), + (Seq("The", "the"), Seq("The")) + ).toDF("input1", "input2") + intercept[IllegalArgumentException] { + remover.transform(df).count() + } + } + + test("Multiple Columns: Set both of inputCol/inputCols") { + val remover = new StopWordsRemover() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("result1", "result2")) + .setInputCol("input1") + val df = Seq( + (Seq("A"), Seq("A")), + (Seq("The", "the"), Seq("The")) + ).toDF("input1", "input2") + intercept[IllegalArgumentException] { + remover.transform(df).count() + } + } } diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 9513b0caecb9a..f6e531302317b 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -3774,9 +3774,13 @@ def setOutputCol(self, value): return self._set(outputCol=value) -class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritable): +class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, + JavaMLReadable, JavaMLWritable): """ A feature transformer that filters out stop words from input. + Since 3.0.0, :py:class:`StopWordsRemover` can filter out multiple columns at once by setting + the :py:attr:`inputCols` parameter. Note that when both the :py:attr:`inputCol` and + :py:attr:`inputCols` parameters are set, an Exception will be thrown. .. note:: null values from input array are preserved unless adding null to stopWords explicitly. @@ -3795,6 +3799,17 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadabl True >>> loadedRemover.getCaseSensitive() == remover.getCaseSensitive() True + >>> df2 = spark.createDataFrame([(["a", "b", "c"], ["a", "b"])], ["text1", "text2"]) + >>> remover2 = StopWordsRemover(stopWords=["b"]) + >>> remover2.setInputCols(["text1", "text2"]).setOutputCols(["words1", "words2"]) + StopWordsRemover... + >>> remover2.transform(df2).show() + +---------+------+------+------+ + | text1| text2|words1|words2| + +---------+------+------+------+ + |[a, b, c]|[a, b]|[a, c]| [a]| + +---------+------+------+------+ + ... .. versionadded:: 1.6.0 """ @@ -3808,10 +3823,10 @@ class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadabl @keyword_only def __init__(self, inputCol=None, outputCol=None, stopWords=None, caseSensitive=False, - locale=None): + locale=None, inputCols=None, outputCols=None): """ __init__(self, inputCol=None, outputCol=None, stopWords=None, caseSensitive=false, \ - locale=None) + locale=None, inputCols=None, outputCols=None) """ super(StopWordsRemover, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.StopWordsRemover", @@ -3824,10 +3839,10 @@ def __init__(self, inputCol=None, outputCol=None, stopWords=None, caseSensitive= @keyword_only @since("1.6.0") def setParams(self, inputCol=None, outputCol=None, stopWords=None, caseSensitive=False, - locale=None): + locale=None, inputCols=None, outputCols=None): """ setParams(self, inputCol=None, outputCol=None, stopWords=None, caseSensitive=false, \ - locale=None) + locale=None, inputCols=None, outputCols=None) Sets params for this StopWordRemover. """ kwargs = self._input_kwargs @@ -3887,6 +3902,20 @@ def setOutputCol(self, value): """ return self._set(outputCol=value) + @since("3.0.0") + def setInputCols(self, value): + """ + Sets the value of :py:attr:`inputCols`. + """ + return self._set(inputCols=value) + + @since("3.0.0") + def setOutputCols(self, value): + """ + Sets the value of :py:attr:`outputCols`. + """ + return self._set(outputCols=value) + @staticmethod @since("2.0.0") def loadDefaultStopWords(language): From 363af16c72abe19fc5cc5b5bdf9d8dc34975f2ba Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 13 Nov 2019 08:59:46 -0800 Subject: [PATCH 0604/1387] [SPARK-29568][SS] Stop existing running streams when a new stream is launched ### What changes were proposed in this pull request? This PR adds a SQL Conf: `spark.sql.streaming.stopActiveRunOnRestart`. When this conf is `true` (by default it is), an already running stream will be stopped, if a new copy gets launched on the same checkpoint location. ### Why are the changes needed? In multi-tenant environments where you have multiple SparkSessions, you can accidentally start multiple copies of the same stream (i.e. streams using the same checkpoint location). This will cause all new instantiations of the new stream to fail. However, sometimes you may want to turn off the old stream, as the old stream may have turned into a zombie (you no longer have access to the query handle or SparkSession). It would be nice to have a SQL flag that allows the stopping of the old stream for such zombie cases. ### Does this PR introduce any user-facing change? Yes. Now by default, if you launch a new copy of an already running stream on a multi-tenant cluster, the existing stream will be stopped. ### How was this patch tested? Unit tests in StreamingQueryManagerSuite Closes #26225 from brkyvz/stopStream. Lead-authored-by: Burak Yavuz Co-authored-by: Burak Yavuz Signed-off-by: Burak Yavuz --- .../apache/spark/sql/internal/SQLConf.scala | 9 ++ .../spark/sql/internal/SharedState.scala | 10 +- .../sql/streaming/StreamingQueryManager.scala | 82 ++++++++--- .../StreamingQueryManagerSuite.scala | 134 ++++++++++++++---- .../sql/streaming/StreamingQuerySuite.scala | 8 +- 5 files changed, 184 insertions(+), 59 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 98acace79601b..759586a2936fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1093,6 +1093,15 @@ object SQLConf { .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2") .createWithDefault(2) + val STREAMING_STOP_ACTIVE_RUN_ON_RESTART = + buildConf("spark.sql.streaming.stopActiveRunOnRestart") + .doc("Running multiple runs of the same streaming query concurrently is not supported. " + + "If we find a concurrent active run for a streaming query (in the same or different " + + "SparkSessions on the same cluster) and this flag is true, we will stop the old streaming " + + "query run to start the new one.") + .booleanConf + .createWithDefault(true) + val STREAMING_JOIN_STATE_FORMAT_VERSION = buildConf("spark.sql.streaming.join.stateFormatVersion") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index d097f9f18f89b..b810bedac471d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.internal import java.net.URL import java.util.{Locale, UUID} import java.util.concurrent.ConcurrentHashMap +import javax.annotation.concurrent.GuardedBy import scala.reflect.ClassTag import scala.util.control.NonFatal @@ -32,9 +33,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.CacheManager +import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.sql.streaming.StreamingQueryManager +import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils @@ -112,11 +114,15 @@ private[sql] class SharedState( */ val cacheManager: CacheManager = new CacheManager + /** A global lock for all streaming query lifecycle tracking and management. */ + private[sql] val activeQueriesLock = new Object + /** * A map of active streaming queries to the session specific StreamingQueryManager that manages * the lifecycle of that stream. */ - private[sql] val activeStreamingQueries = new ConcurrentHashMap[UUID, StreamingQueryManager]() + @GuardedBy("activeQueriesLock") + private[sql] val activeStreamingQueries = new ConcurrentHashMap[UUID, StreamExecution]() /** * A status store to query SQL status/metrics of this Spark application, based on SQL-specific diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 9b43a83e7b94a..e64f67cc755fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import java.util.UUID +import java.util.{ConcurrentModificationException, UUID} import java.util.concurrent.TimeUnit import javax.annotation.concurrent.GuardedBy @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} /** * A class to manage all the [[StreamingQuery]] active in a `SparkSession`. @@ -51,9 +51,10 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus) - @GuardedBy("activeQueriesLock") + @GuardedBy("activeQueriesSharedLock") private val activeQueries = new mutable.HashMap[UUID, StreamingQuery] - private val activeQueriesLock = new Object + // A global lock to keep track of active streaming queries across Spark sessions + private val activeQueriesSharedLock = sparkSession.sharedState.activeQueriesLock private val awaitTerminationLock = new Object @GuardedBy("awaitTerminationLock") @@ -77,7 +78,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo * * @since 2.0.0 */ - def active: Array[StreamingQuery] = activeQueriesLock.synchronized { + def active: Array[StreamingQuery] = activeQueriesSharedLock.synchronized { activeQueries.values.toArray } @@ -86,7 +87,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo * * @since 2.1.0 */ - def get(id: UUID): StreamingQuery = activeQueriesLock.synchronized { + def get(id: UUID): StreamingQuery = activeQueriesSharedLock.synchronized { activeQueries.get(id).orNull } @@ -343,27 +344,61 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo trigger, triggerClock) - activeQueriesLock.synchronized { + // The following code block checks if a stream with the same name or id is running. Then it + // returns an Option of an already active stream to stop outside of the lock + // to avoid a deadlock. + val activeRunOpt = activeQueriesSharedLock.synchronized { // Make sure no other query with same name is active userSpecifiedName.foreach { name => if (activeQueries.values.exists(_.name == name)) { - throw new IllegalArgumentException( - s"Cannot start query with name $name as a query with that name is already active") + throw new IllegalArgumentException(s"Cannot start query with name $name as a query " + + s"with that name is already active in this SparkSession") } } // Make sure no other query with same id is active across all sessions - val activeOption = - Option(sparkSession.sharedState.activeStreamingQueries.putIfAbsent(query.id, this)) - if (activeOption.isDefined || activeQueries.values.exists(_.id == query.id)) { - throw new IllegalStateException( - s"Cannot start query with id ${query.id} as another query with same id is " + - s"already active. Perhaps you are attempting to restart a query from checkpoint " + - s"that is already active.") + val activeOption = Option(sparkSession.sharedState.activeStreamingQueries.get(query.id)) + .orElse(activeQueries.get(query.id)) // shouldn't be needed but paranoia ... + + val shouldStopActiveRun = + sparkSession.sessionState.conf.getConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART) + if (activeOption.isDefined) { + if (shouldStopActiveRun) { + val oldQuery = activeOption.get + logWarning(s"Stopping existing streaming query [id=${query.id}, " + + s"runId=${oldQuery.runId}], as a new run is being started.") + Some(oldQuery) + } else { + throw new IllegalStateException( + s"Cannot start query with id ${query.id} as another query with same id is " + + s"already active. Perhaps you are attempting to restart a query from checkpoint " + + s"that is already active. You may stop the old query by setting the SQL " + + "configuration: " + + s"""spark.conf.set("${SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key}", true) """ + + "and retry.") + } + } else { + // nothing to stop so, no-op + None } + } + // stop() will clear the queryId from activeStreamingQueries as well as activeQueries + activeRunOpt.foreach(_.stop()) + + activeQueriesSharedLock.synchronized { + // We still can have a race condition when two concurrent instances try to start the same + // stream, while a third one was already active and stopped above. In this case, we throw a + // ConcurrentModificationException. + val oldActiveQuery = sparkSession.sharedState.activeStreamingQueries.put( + query.id, query.streamingQuery) // we need to put the StreamExecution, not the wrapper + if (oldActiveQuery != null) { + throw new ConcurrentModificationException( + "Another instance of this query was just started by a concurrent session.") + } activeQueries.put(query.id, query) } + try { // When starting a query, it will call `StreamingQueryListener.onQueryStarted` synchronously. // As it's provided by the user and can run arbitrary codes, we must not hold any lock here. @@ -372,7 +407,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo query.streamingQuery.start() } catch { case e: Throwable => - unregisterTerminatedStream(query.id) + unregisterTerminatedStream(query) throw e } query @@ -380,7 +415,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo /** Notify (by the StreamingQuery) that the query has been terminated */ private[sql] def notifyQueryTermination(terminatedQuery: StreamingQuery): Unit = { - unregisterTerminatedStream(terminatedQuery.id) + unregisterTerminatedStream(terminatedQuery) awaitTerminationLock.synchronized { if (lastTerminatedQuery == null || terminatedQuery.exception.nonEmpty) { lastTerminatedQuery = terminatedQuery @@ -390,11 +425,12 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo stateStoreCoordinator.deactivateInstances(terminatedQuery.runId) } - private def unregisterTerminatedStream(terminatedQueryId: UUID): Unit = { - activeQueriesLock.synchronized { - // remove from shared state only if the streaming query manager also matches - sparkSession.sharedState.activeStreamingQueries.remove(terminatedQueryId, this) - activeQueries -= terminatedQueryId + private def unregisterTerminatedStream(terminatedQuery: StreamingQuery): Unit = { + activeQueriesSharedLock.synchronized { + // remove from shared state only if the streaming execution also matches + sparkSession.sharedState.activeStreamingQueries.remove( + terminatedQuery.id, terminatedQuery) + activeQueries -= terminatedQuery.id } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala index 09580b94056b4..96f7efeef98e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.{Dataset, Encoders} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.BlockingSource import org.apache.spark.util.Utils @@ -274,48 +275,119 @@ class StreamingQueryManagerSuite extends StreamTest { } testQuietly("can't start multiple instances of the same streaming query in the same session") { - withTempDir { dir => - val (ms1, ds1) = makeDataset - val (ms2, ds2) = makeDataset - val chkLocation = new File(dir, "_checkpoint").getCanonicalPath - val dataLocation = new File(dir, "data").getCanonicalPath - - val query1 = ds1.writeStream.format("parquet") - .option("checkpointLocation", chkLocation).start(dataLocation) - ms1.addData(1, 2, 3) - try { - val e = intercept[IllegalStateException] { - ds2.writeStream.format("parquet") + withSQLConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key -> "false") { + withTempDir { dir => + val (ms1, ds1) = makeDataset + val (ms2, ds2) = makeDataset + val chkLocation = new File(dir, "_checkpoint").getCanonicalPath + val dataLocation = new File(dir, "data").getCanonicalPath + + val query1 = ds1.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + ms1.addData(1, 2, 3) + try { + val e = intercept[IllegalStateException] { + ds2.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + } + assert(e.getMessage.contains("same id")) + } finally { + spark.streams.active.foreach(_.stop()) + } + } + } + } + + testQuietly("new instance of the same streaming query stops old query in the same session") { + failAfter(90 seconds) { + withSQLConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key -> "true") { + withTempDir { dir => + val (ms1, ds1) = makeDataset + val (ms2, ds2) = makeDataset + val chkLocation = new File(dir, "_checkpoint").getCanonicalPath + val dataLocation = new File(dir, "data").getCanonicalPath + + val query1 = ds1.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + ms1.addData(1, 2, 3) + val query2 = ds2.writeStream.format("parquet") .option("checkpointLocation", chkLocation).start(dataLocation) + try { + ms2.addData(1, 2, 3) + query2.processAllAvailable() + assert(spark.sharedState.activeStreamingQueries.get(query2.id) === + query2.asInstanceOf[StreamingQueryWrapper].streamingQuery, + "The correct streaming query is not being tracked in global state") + + assert(!query1.isActive, + "First query should have stopped before starting the second query") + } finally { + spark.streams.active.foreach(_.stop()) + } } - assert(e.getMessage.contains("same id")) - } finally { - query1.stop() } } } testQuietly( "can't start multiple instances of the same streaming query in the different sessions") { - withTempDir { dir => - val session2 = spark.cloneSession() - - val ms1 = MemoryStream(Encoders.INT, spark.sqlContext) - val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() - val chkLocation = new File(dir, "_checkpoint").getCanonicalPath - val dataLocation = new File(dir, "data").getCanonicalPath + withSQLConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key -> "false") { + withTempDir { dir => + val session2 = spark.cloneSession() + + val ms1 = MemoryStream(Encoders.INT, spark.sqlContext) + val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() + val chkLocation = new File(dir, "_checkpoint").getCanonicalPath + val dataLocation = new File(dir, "data").getCanonicalPath + + val query1 = ms1.toDS().writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + ms1.addData(1, 2, 3) + try { + val e = intercept[IllegalStateException] { + ds2.writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + } + assert(e.getMessage.contains("same id")) + } finally { + spark.streams.active.foreach(_.stop()) + session2.streams.active.foreach(_.stop()) + } + } + } + } - val query1 = ms1.toDS().writeStream.format("parquet") - .option("checkpointLocation", chkLocation).start(dataLocation) - ms1.addData(1, 2, 3) - try { - val e = intercept[IllegalStateException] { - ds2.writeStream.format("parquet") + testQuietly( + "new instance of the same streaming query stops old query in a different session") { + failAfter(90 seconds) { + withSQLConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key -> "true") { + withTempDir { dir => + val session2 = spark.cloneSession() + + val ms1 = MemoryStream(Encoders.INT, spark.sqlContext) + val ds2 = MemoryStream(Encoders.INT, session2.sqlContext).toDS() + val chkLocation = new File(dir, "_checkpoint").getCanonicalPath + val dataLocation = new File(dir, "data").getCanonicalPath + + val query1 = ms1.toDS().writeStream.format("parquet") + .option("checkpointLocation", chkLocation).start(dataLocation) + ms1.addData(1, 2, 3) + val query2 = ds2.writeStream.format("parquet") .option("checkpointLocation", chkLocation).start(dataLocation) + try { + ms1.addData(1, 2, 3) + query2.processAllAvailable() + assert(spark.sharedState.activeStreamingQueries.get(query2.id) === + query2.asInstanceOf[StreamingQueryWrapper].streamingQuery, + "The correct streaming execution is not being tracked in global state") + + assert(!query1.isActive, + "First query should have stopped before starting the second query") + } finally { + spark.streams.active.foreach(_.stop()) + session2.streams.active.foreach(_.stop()) + } } - assert(e.getMessage.contains("same id")) - } finally { - query1.stop() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 760731d26f051..4121f499bd69c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -123,9 +123,11 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi assert(q3.runId !== q4.runId) // Only one query with same id can be active - val q5 = startQuery(restart = false) - val e = intercept[IllegalStateException] { - startQuery(restart = true) + withSQLConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key -> "false") { + val q5 = startQuery(restart = false) + val e = intercept[IllegalStateException] { + startQuery(restart = true) + } } } } From 833a9f12e2d46b5741e0522d5c86c9f6d88bb9d0 Mon Sep 17 00:00:00 2001 From: Nishchal Venkataramana Date: Wed, 13 Nov 2019 22:01:48 +0000 Subject: [PATCH 0605/1387] [SPARK-24203][CORE] Make executor's bindAddress configurable ### What changes were proposed in this pull request? With this change, executor's bindAddress is passed as an input parameter for RPCEnv.create. A previous PR https://github.com/apache/spark/pull/21261 which addressed the same, was using a Spark Conf property to get the bindAddress which wouldn't have worked for multiple executors. This PR is to enable anyone overriding CoarseGrainedExecutorBackend with their custom one to be able to invoke CoarseGrainedExecutorBackend.main() along with the option to configure bindAddress. ### Why are the changes needed? This is required when Kernel-based Virtual Machine (KVM)'s are used inside Linux container where the hostname is not the same as container hostname. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Tested by running jobs with executors on KVMs inside a linux container. Closes #26331 from nishchalv/SPARK-29670. Lead-authored-by: Nishchal Venkataramana Co-authored-by: nishchal Signed-off-by: DB Tsai --- .../scala/org/apache/spark/SparkEnv.scala | 14 ++++++- .../CoarseGrainedExecutorBackend.scala | 21 ++++++++-- .../CoarseGrainedExecutorBackendSuite.scala | 41 +++++++++++++++---- .../YarnCoarseGrainedExecutorBackend.scala | 4 +- 4 files changed, 66 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 15d1c081bc089..5fe1c663affa5 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -196,6 +196,7 @@ object SparkEnv extends Logging { private[spark] def createExecutorEnv( conf: SparkConf, executorId: String, + bindAddress: String, hostname: String, numCores: Int, ioEncryptionKey: Option[Array[Byte]], @@ -203,7 +204,7 @@ object SparkEnv extends Logging { val env = create( conf, executorId, - hostname, + bindAddress, hostname, None, isLocal, @@ -214,6 +215,17 @@ object SparkEnv extends Logging { env } + private[spark] def createExecutorEnv( + conf: SparkConf, + executorId: String, + hostname: String, + numCores: Int, + ioEncryptionKey: Option[Array[Byte]], + isLocal: Boolean): SparkEnv = { + createExecutorEnv(conf, executorId, hostname, + hostname, numCores, ioEncryptionKey, isLocal) + } + /** * Helper method to create a SparkEnv for a driver or an executor. */ 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 b4bca1e9401e2..beb9d98754509 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -46,6 +46,7 @@ private[spark] class CoarseGrainedExecutorBackend( override val rpcEnv: RpcEnv, driverUrl: String, executorId: String, + bindAddress: String, hostname: String, cores: Int, userClassPath: Seq[URL], @@ -227,6 +228,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case class Arguments( driverUrl: String, executorId: String, + bindAddress: String, hostname: String, cores: Int, appId: String, @@ -238,7 +240,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, Arguments, SparkEnv) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => new CoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.hostname, arguments.cores, arguments.userClassPath, env, + arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourcesFileOpt) } run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn) @@ -259,10 +261,12 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val executorConf = new SparkConf val fetcher = RpcEnv.create( "driverPropsFetcher", + arguments.bindAddress, arguments.hostname, -1, executorConf, new SecurityManager(executorConf), + numUsableCores = 0, clientMode = true) var driver: RpcEndpointRef = null @@ -297,8 +301,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } driverConf.set(EXECUTOR_ID, arguments.executorId) - val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.hostname, - arguments.cores, cfg.ioEncryptionKey, isLocal = false) + val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.bindAddress, + arguments.hostname, arguments.cores, cfg.ioEncryptionKey, isLocal = false) env.rpcEnv.setupEndpoint("Executor", backendCreateFn(env.rpcEnv, arguments, env)) arguments.workerUrl.foreach { url => @@ -311,6 +315,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { def parseArguments(args: Array[String], classNameForEntry: String): Arguments = { var driverUrl: String = null var executorId: String = null + var bindAddress: String = null var hostname: String = null var cores: Int = 0 var resourcesFileOpt: Option[String] = None @@ -327,6 +332,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case ("--executor-id") :: value :: tail => executorId = value argv = tail + case ("--bind-address") :: value :: tail => + bindAddress = value + argv = tail case ("--hostname") :: value :: tail => hostname = value argv = tail @@ -364,7 +372,11 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { printUsageAndExit(classNameForEntry) } - Arguments(driverUrl, executorId, hostname, cores, appId, workerUrl, + if (bindAddress == null) { + bindAddress = hostname + } + + Arguments(driverUrl, executorId, bindAddress, hostname, cores, appId, workerUrl, userClassPath, resourcesFileOpt) } @@ -377,6 +389,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | Options are: | --driver-url | --executor-id + | --bind-address | --hostname | --cores | --resourcesFile diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index e40cf0d66c19c..7e96039ca924f 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -54,7 +54,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => val testResourceArgs: JObject = ("" -> "") @@ -76,7 +76,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => val ra = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) @@ -101,7 +101,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) withTempDir { tmpDir => @@ -129,7 +129,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) // not enough gpu's on the executor @@ -168,7 +168,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) // executor resources < required @@ -200,7 +200,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) val parsedResources = backend.parseOrFindResources(None) @@ -226,7 +226,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", 4, Seq.empty[URL], env, None) val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) val ja = Extraction.decompose(Seq(gpuArgs)) @@ -254,7 +254,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val rpcEnv = RpcEnv.create("1", "localhost", 0, conf, securityMgr) val env = createMockEnv(conf, serializer, Some(rpcEnv)) backend = new CoarseGrainedExecutorBackend(env.rpcEnv, rpcEnv.address.hostPort, "1", - "host1", 4, Seq.empty[URL], env, None) + "host1", "host1", 4, Seq.empty[URL], env, None) assert(backend.taskResources.isEmpty) val taskId = 1000000 @@ -289,6 +289,31 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite } } + test("SPARK-24203 when bindAddress is not set, it defaults to hostname") { + val args1 = Array( + "--driver-url", "driverurl", + "--executor-id", "1", + "--hostname", "host1", + "--cores", "1", + "--app-id", "app1") + + val arg = CoarseGrainedExecutorBackend.parseArguments(args1, "") + assert(arg.bindAddress == "host1") + } + + test("SPARK-24203 when bindAddress is different, it does not default to hostname") { + val args1 = Array( + "--driver-url", "driverurl", + "--executor-id", "1", + "--hostname", "host1", + "--bind-address", "bindaddress1", + "--cores", "1", + "--app-id", "app1") + + val arg = CoarseGrainedExecutorBackend.parseArguments(args1, "") + assert(arg.bindAddress == "bindaddress1") + } + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer, rpcEnv: Option[RpcEnv] = None): SparkEnv = { val mockEnv = mock[SparkEnv] diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala index 2e5748b6144f9..d46424e5e99ad 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala @@ -34,6 +34,7 @@ private[spark] class YarnCoarseGrainedExecutorBackend( rpcEnv: RpcEnv, driverUrl: String, executorId: String, + bindAddress: String, hostname: String, cores: Int, userClassPath: Seq[URL], @@ -43,6 +44,7 @@ private[spark] class YarnCoarseGrainedExecutorBackend( rpcEnv, driverUrl, executorId, + bindAddress, hostname, cores, userClassPath, @@ -68,7 +70,7 @@ private[spark] object YarnCoarseGrainedExecutorBackend extends Logging { val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.hostname, arguments.cores, arguments.userClassPath, env, + arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, arguments.resourcesFileOpt) } val backendArgs = CoarseGrainedExecutorBackend.parseArguments(args, From 39b502af1728cbdd692388fed71ceac0f4de4b4f Mon Sep 17 00:00:00 2001 From: Wesley Hoffman Date: Wed, 13 Nov 2019 14:10:30 -0800 Subject: [PATCH 0606/1387] [SPARK-29778][SQL] pass writer options to saveAsTable in append mode ### What changes were proposed in this pull request? `saveAsTable` had an oversight where write options were not considered in the append save mode. ### Why are the changes needed? Address the bug so that write options can be considered during appends. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test added that looks in the logic plan of `AppendData` for the existing write options. Closes #26474 from SpaceRangerWes/master. Authored-by: Wesley Hoffman Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../DataSourceV2DataFrameSuite.scala | 38 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 68127c27a8cc2..3561c41f2b223 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -527,7 +527,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption)) case (SaveMode.Append, Some(table)) => - AppendData.byName(DataSourceV2Relation.create(table), df.logicalPlan) + AppendData.byName(DataSourceV2Relation.create(table), df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => ReplaceTableAsSelect( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index 8e921aaeb66f9..72d4629a1a320 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.connector import org.apache.spark.sql.{DataFrame, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.util.QueryExecutionListener class DataSourceV2DataFrameSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = false) { @@ -125,4 +128,39 @@ class DataSourceV2DataFrameSuite checkAnswer(spark.table(t1), Seq(Row("c", "d"))) } } + + testQuietly("SPARK-29778: saveAsTable: append mode takes write options") { + + var plan: LogicalPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.analyzed + } + override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} + } + + try { + spark.listenerManager.register(listener) + + val t1 = "testcat.ns1.ns2.tbl" + + sql(s"CREATE TABLE $t1 (id bigint, data string) USING foo") + + val df = Seq((1L, "a"), (2L, "b"), (3L, "c")).toDF("id", "data") + df.write.option("other", "20").mode("append").saveAsTable(t1) + + sparkContext.listenerBus.waitUntilEmpty() + plan match { + case p: AppendData => + assert(p.writeOptions == Map("other" -> "20")) + case other => + fail(s"Expected to parse ${classOf[AppendData].getName} from query," + + s"got ${other.getClass.getName}: $plan") + } + + checkAnswer(spark.table(t1), df) + } finally { + spark.listenerManager.unregister(listener) + } + } } From 15a72f3755435e6c5db81a8faae303d06ed9fe3f Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 13 Nov 2019 16:14:12 -0800 Subject: [PATCH 0607/1387] [SPARK-29287][CORE] Add LaunchedExecutor message to tell driver which executor is ready for making offers MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add `LaunchedExecuto`r message and send it to the driver when the executor if fully constructed, then the driver can assign the associated executor's totalCores to freeCores for making offers. ### Why are the changes needed? The executors send RegisterExecutor messages to the driver when onStart. The driver put the executor data in “the ready to serve map” if it could be, then send RegisteredExecutor back to the executor. The driver now can make an offer to this executor. But the executor is not fully constructed yet. When it received RegisteredExecutor, it start to construct itself, initializing block manager, maybe register to the local shuffle server in the way of retrying, then start the heart beating to driver ... The task allocated here may fail if the executor fails to start or cannot get heart beating to the driver in time. Sometimes, even worse, when dynamic allocation and blacklisting is enabled and when the runtime executor number down to min executor setting, and those executors receive tasks before fully constructed and if any error happens, the application may be blocked or tear down. ### Does this PR introduce any user-facing change? NO ### How was this patch tested? Closes #25964 from yaooqinn/SPARK-29287. Authored-by: Kent Yao Signed-off-by: Xingbo Jiang --- .../spark/executor/CoarseGrainedExecutorBackend.scala | 1 + .../scheduler/cluster/CoarseGrainedClusterMessage.scala | 2 ++ .../cluster/CoarseGrainedSchedulerBackend.scala | 9 +++++++-- .../spark/deploy/StandaloneDynamicAllocationSuite.scala | 3 ++- 4 files changed, 12 insertions(+), 3 deletions(-) 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 beb9d98754509..b1837c9c0c9ea 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -127,6 +127,7 @@ private[spark] class CoarseGrainedExecutorBackend( logInfo("Successfully registered with driver") try { executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + driver.get.send(LaunchedExecutor(executorId)) } catch { case NonFatal(e) => exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index a90fff02ac73d..9ce23683245eb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -69,6 +69,8 @@ private[spark] object CoarseGrainedClusterMessages { resources: Map[String, ResourceInformation]) extends CoarseGrainedClusterMessage + case class LaunchedExecutor(executorId: String) extends CoarseGrainedClusterMessage + case class StatusUpdate( executorId: String, taskId: Long, 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 ea045e6280e4c..7d9c1c6f96f6c 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 @@ -194,6 +194,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // automatically, so try to tell the executor to stop itself. See SPARK-13519. executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) + + case LaunchedExecutor(executorId) => + executorDataMap.get(executorId).foreach { data => + data.freeCores = data.totalCores + } + makeOffers(executorId) } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -230,7 +236,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp taskResourceNumParts.getOrElse(v.name, 1))) } val data = new ExecutorData(executorRef, executorAddress, hostname, - cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, + 0, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, resourcesInfo) // This must be synchronized because variables mutated // in this block are read when requesting executors @@ -249,7 +255,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(true) listenerBus.post( SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) - makeOffers() } case StopDriver => diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 17758783d2590..dd790b8dbb853 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.internal.config import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster._ -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RegisterExecutorFailed} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{LaunchedExecutor, RegisterExecutor, RegisterExecutorFailed} /** * End-to-end tests for dynamic allocation in standalone mode. @@ -634,6 +634,7 @@ class StandaloneDynamicAllocationSuite Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askSync[Boolean](message) + backend.driverEndpoint.send(LaunchedExecutor(id)) } } From 32d44b1d0e5e00f100df184ebe50a3ddf7e53865 Mon Sep 17 00:00:00 2001 From: shivsood Date: Wed, 13 Nov 2019 17:56:13 -0800 Subject: [PATCH 0608/1387] [SPARK-29644][SQL] Corrected ShortType and ByteType mapping to SmallInt and TinyInt in JDBCUtils ### What changes were proposed in this pull request? Corrected ShortType and ByteType mapping to SmallInt and TinyInt, corrected setter methods to set ShortType and ByteType as setShort() and setByte(). Changes in JDBCUtils.scala Fixed Unit test cases to where applicable and added new E2E test cases in to test table read/write using ShortType and ByteType. #### Problems - In master in JDBCUtils.scala line number 547 and 551 have a problem where ShortType and ByteType are set as Integers rather than set as Short and Byte respectively. ``` case ShortType => (stmt: PreparedStatement, row: Row, pos: Int) => stmt.setInt(pos + 1, row.getShort(pos)) The issue was pointed out by maropu case ByteType => (stmt: PreparedStatement, row: Row, pos: Int) => stmt.setInt(pos + 1, row.getByte(pos)) ``` - Also at line JDBCUtils.scala 247 TinyInt is interpreted wrongly as IntergetType in getCatalystType() ``` case java.sql.Types.TINYINT => IntegerType ``` - At line 172 ShortType was wrongly interpreted as IntegerType ``` case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) ``` - All thru out tests, ShortType and ByteType were being interpreted as IntegerTypes. ### Why are the changes needed? A given type should be set using the right type. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Corrected Unit test cases where applicable. Validated in CI/CD Added a test case in MsSqlServerIntegrationSuite.scala, PostgresIntegrationSuite.scala , MySQLIntegrationSuite.scala to write/read tables from dataframe with cols as shorttype and bytetype. Validated by manual as follows. ``` ./build/mvn install -DskipTests ./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12 ``` Closes #26301 from shivsood/shorttype_fix_maropu. Authored-by: shivsood Signed-off-by: Dongjoon Hyun --- .../jdbc/MsSqlServerIntegrationSuite.scala | 48 +++++++++++++++++-- .../sql/jdbc/MySQLIntegrationSuite.scala | 4 +- .../datasources/jdbc/JdbcUtils.scala | 12 ++--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 42 ++++++++++++++++ 5 files changed, 97 insertions(+), 13 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index efd7ca74c796b..f1cd3343b7925 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -59,7 +59,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { """ |INSERT INTO numbers VALUES ( |0, - |255, 32767, 2147483647, 9223372036854775807, + |127, 32767, 2147483647, 9223372036854775807, |123456789012345.123456789012345, 123456789012345.123456789012345, |123456789012345.123456789012345, |123, 12345.12, @@ -119,7 +119,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { val types = row.toSeq.map(x => x.getClass.toString) assert(types.length == 12) assert(types(0).equals("class java.lang.Boolean")) - assert(types(1).equals("class java.lang.Integer")) + assert(types(1).equals("class java.lang.Byte")) assert(types(2).equals("class java.lang.Short")) assert(types(3).equals("class java.lang.Integer")) assert(types(4).equals("class java.lang.Long")) @@ -131,7 +131,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types(10).equals("class java.math.BigDecimal")) assert(types(11).equals("class java.math.BigDecimal")) assert(row.getBoolean(0) == false) - assert(row.getInt(1) == 255) + assert(row.getByte(1) == 127) assert(row.getShort(2) == 32767) assert(row.getInt(3) == 2147483647) assert(row.getLong(4) == 9223372036854775807L) @@ -202,4 +202,46 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { df2.write.jdbc(jdbcUrl, "datescopy", new Properties) df3.write.jdbc(jdbcUrl, "stringscopy", new Properties) } + + test("SPARK-29644: Write tables with ShortType") { + import testImplicits._ + val df = Seq(-32768.toShort, 0.toShort, 1.toShort, 38.toShort, 32768.toShort).toDF("a") + val tablename = "shorttable" + df.write + .format("jdbc") + .mode("overwrite") + .option("url", jdbcUrl) + .option("dbtable", tablename) + .save() + val df2 = spark.read + .format("jdbc") + .option("url", jdbcUrl) + .option("dbtable", tablename) + .load() + assert(df.count == df2.count) + val rows = df2.collect() + val colType = rows(0).toSeq.map(x => x.getClass.toString) + assert(colType(0) == "class java.lang.Short") + } + + test("SPARK-29644: Write tables with ByteType") { + import testImplicits._ + val df = Seq(-127.toByte, 0.toByte, 1.toByte, 38.toByte, 128.toByte).toDF("a") + val tablename = "bytetable" + df.write + .format("jdbc") + .mode("overwrite") + .option("url", jdbcUrl) + .option("dbtable", tablename) + .save() + val df2 = spark.read + .format("jdbc") + .option("url", jdbcUrl) + .option("dbtable", tablename) + .load() + assert(df.count == df2.count) + val rows = df2.collect() + val colType = rows(0).toSeq.map(x => x.getClass.toString) + assert(colType(0) == "class java.lang.Byte") + } } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index bba1b5275269b..8401b0a8a752f 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -84,7 +84,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types.length == 9) assert(types(0).equals("class java.lang.Boolean")) assert(types(1).equals("class java.lang.Long")) - assert(types(2).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Short")) assert(types(3).equals("class java.lang.Integer")) assert(types(4).equals("class java.lang.Integer")) assert(types(5).equals("class java.lang.Long")) @@ -93,7 +93,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types(8).equals("class java.lang.Double")) assert(rows(0).getBoolean(0) == false) assert(rows(0).getLong(1) == 0x225) - assert(rows(0).getInt(2) == 17) + assert(rows(0).getShort(2) == 17) assert(rows(0).getInt(3) == 77777) assert(rows(0).getInt(4) == 123456789) assert(rows(0).getLong(5) == 123456789012345L) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 319cb277cc475..d4c7f005a16df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -170,8 +170,8 @@ object JdbcUtils extends Logging { case LongType => Option(JdbcType("BIGINT", java.sql.Types.BIGINT)) case DoubleType => Option(JdbcType("DOUBLE PRECISION", java.sql.Types.DOUBLE)) case FloatType => Option(JdbcType("REAL", java.sql.Types.FLOAT)) - case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) - case ByteType => Option(JdbcType("BYTE", java.sql.Types.TINYINT)) + case ShortType => Option(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) + case ByteType => Option(JdbcType("TINYINT", java.sql.Types.TINYINT)) case BooleanType => Option(JdbcType("BIT(1)", java.sql.Types.BIT)) case StringType => Option(JdbcType("TEXT", java.sql.Types.CLOB)) case BinaryType => Option(JdbcType("BLOB", java.sql.Types.BLOB)) @@ -235,7 +235,7 @@ object JdbcUtils extends Logging { case java.sql.Types.REF => StringType case java.sql.Types.REF_CURSOR => null case java.sql.Types.ROWID => LongType - case java.sql.Types.SMALLINT => IntegerType + case java.sql.Types.SMALLINT => ShortType case java.sql.Types.SQLXML => StringType case java.sql.Types.STRUCT => StringType case java.sql.Types.TIME => TimestampType @@ -244,7 +244,7 @@ object JdbcUtils extends Logging { case java.sql.Types.TIMESTAMP => TimestampType case java.sql.Types.TIMESTAMP_WITH_TIMEZONE => null - case java.sql.Types.TINYINT => IntegerType + case java.sql.Types.TINYINT => ByteType case java.sql.Types.VARBINARY => BinaryType case java.sql.Types.VARCHAR => StringType case _ => @@ -546,11 +546,11 @@ object JdbcUtils extends Logging { case ShortType => (stmt: PreparedStatement, row: Row, pos: Int) => - stmt.setInt(pos + 1, row.getShort(pos)) + stmt.setShort(pos + 1, row.getShort(pos)) case ByteType => (stmt: PreparedStatement, row: Row, pos: Int) => - stmt.setInt(pos + 1, row.getByte(pos)) + stmt.setByte(pos + 1, row.getByte(pos)) case BooleanType => (stmt: PreparedStatement, row: Row, pos: Int) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 43f6381c19790..432697015a482 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -578,8 +578,8 @@ class JDBCSuite extends QueryTest assert(rows.length === 1) assert(rows(0).getInt(0) === 1) assert(rows(0).getBoolean(1) === false) - assert(rows(0).getInt(2) === 3) - assert(rows(0).getInt(3) === 4) + assert(rows(0).getByte(2) === 3.toByte) + assert(rows(0).getShort(3) === 4.toShort) assert(rows(0).getLong(4) === 1234567890123L) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 8021ef1a17a18..f43776ad91987 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -574,6 +574,48 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { } } + test("SPARK-29644: Write tables with ShortType") { + import testImplicits._ + val df = Seq(-32768.toShort, 0.toShort, 1.toShort, 38.toShort, 32768.toShort).toDF("a") + val tablename = "shorttable" + df.write + .format("jdbc") + .mode("overwrite") + .option("url", url) + .option("dbtable", tablename) + .save() + val df2 = spark.read + .format("jdbc") + .option("url", url) + .option("dbtable", tablename) + .load() + assert(df.count == df2.count) + val rows = df2.collect() + val colType = rows(0).toSeq.map(x => x.getClass.toString) + assert(colType(0) == "class java.lang.Short") + } + + test("SPARK-29644: Write tables with ByteType") { + import testImplicits._ + val df = Seq(-127.toByte, 0.toByte, 1.toByte, 38.toByte, 128.toByte).toDF("a") + val tablename = "bytetable" + df.write + .format("jdbc") + .mode("overwrite") + .option("url", url) + .option("dbtable", tablename) + .save() + val df2 = spark.read + .format("jdbc") + .option("url", url) + .option("dbtable", tablename) + .load() + assert(df.count == df2.count) + val rows = df2.collect() + val colType = rows(0).toSeq.map(x => x.getClass.toString) + assert(colType(0) == "class java.lang.Byte") + } + private def runAndVerifyRecordsWritten(expected: Long)(job: => Unit): Unit = { assert(expected === runAndReturnMetrics(job, _.taskMetrics.outputMetrics.recordsWritten)) } From 39596b913b557de5d87fbfa9a613f4780c8f765e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 13 Nov 2019 18:01:38 -0800 Subject: [PATCH 0609/1387] [SPARK-29649][SQL] Stop task set if FileAlreadyExistsException was thrown when writing to output file ### What changes were proposed in this pull request? We already know task attempts that do not clean up output files in staging directory can cause job failure (SPARK-27194). There was proposals trying to fix it by changing output filename, or deleting existing output files. These proposals are not reliable completely. The difficulty is, as previous failed task attempt wrote the output file, at next task attempt the output file is still under same staging directory, even the output file name is different. If the job will go to fail eventually, there is no point to re-run the task until max attempts are reached. For the jobs running a lot of time, re-running the task can waste a lot of time. This patch proposes to let Spark detect such file already exist exception and stop the task set early. ### Why are the changes needed? For now, if FileAlreadyExistsException is thrown during data writing job in SQL, the job will continue re-running task attempts until max failure number is reached. It is no point for re-running tasks as task attempts will also fail because they can not write to the existing file too. We should stop the task set early. ### Does this PR introduce any user-facing change? Yes. If FileAlreadyExistsException is thrown during data writing job in SQL, no more task attempts are re-tried and the task set will be stoped early. ### How was this patch tested? Unit test. Closes #26312 from viirya/stop-taskset-if-outputfile-exists. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../TaskOutputFileAlreadyExistException.scala | 23 +++++++++++++ .../spark/scheduler/TaskSetManager.scala | 9 +++++ .../spark/scheduler/TaskSetManagerSuite.scala | 20 +++++++++++ .../datasources/FileFormatWriter.scala | 6 +++- .../spark/sql/sources/InsertSuite.scala | 33 +++++++++++++++++++ 5 files changed, 90 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala diff --git a/core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala b/core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala new file mode 100644 index 0000000000000..68054625bac21 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskOutputFileAlreadyExistException.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Exception thrown when a task cannot write to output file due to the file already exists. + */ +private[spark] class TaskOutputFileAlreadyExistException(error: Throwable) extends Exception(error) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 9defbefabb86a..5c0bc497dd1b3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -799,6 +799,15 @@ private[spark] class TaskSetManager( info.id, taskSet.id, tid, ef.description)) return } + if (ef.className == classOf[TaskOutputFileAlreadyExistException].getName) { + // If we can not write to output file in the task, there's no point in trying to + // re-execute it. + logError("Task %s in stage %s (TID %d) can not write to output file: %s; not retrying" + .format(info.id, taskSet.id, tid, ef.description)) + abort("Task %s in stage %s (TID %d) can not write to output file: %s".format( + info.id, taskSet.id, tid, ef.description)) + return + } val key = ef.description val now = clock.getTimeMillis() val (printFull, dupCount) = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 441ec6ab6e18b..89df5de97c444 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,6 +22,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.fs.FileAlreadyExistsException import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -1775,4 +1776,23 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(!manager.checkSpeculatableTasks(0)) assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) } + + test("TaskOutputFileAlreadyExistException lead to task set abortion") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = FakeTask.createTaskSet(1) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + assert(sched.taskSetsFailed.isEmpty) + + val offerResult = manager.resourceOffer("exec1", "host1", ANY) + assert(offerResult.isDefined, + "Expect resource offer on iteration 0 to return a task") + assert(offerResult.get.index === 0) + val reason = new ExceptionFailure( + new TaskOutputFileAlreadyExistException( + new FileAlreadyExistsException("file already exists")), + Seq.empty[AccumulableInfo]) + manager.handleFailedTask(offerResult.get.taskId, TaskState.FAILED, reason) + assert(sched.taskSetsFailed.contains(taskSet.id)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index f1fc5d762ad56..f02d9e92acb88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources import java.util.{Date, UUID} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileAlreadyExistsException, Path} import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl @@ -281,6 +281,10 @@ object FileFormatWriter extends Logging { } catch { case e: FetchFailedException => throw e + case f: FileAlreadyExistsException => + // If any output file to write already exists, it does not make sense to re-run this task. + // We throw the exception and let Executor throw ExceptionFailure to abort the job. + throw new TaskOutputFileAlreadyExistException(f) case t: Throwable => throw new SparkException("Task failed while writing rows.", t) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 9e33b8aaec5d9..fbde38322fca8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.sources import java.io.File import java.sql.Date +import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataOutputStream, Path, RawLocalFileSystem} + import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier @@ -735,4 +737,35 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { assert(msg.contains("Cannot write nullable values to non-null column 's'")) } } + + test("Stop task set if FileAlreadyExistsException was thrown") { + withSQLConf("fs.file.impl" -> classOf[FileExistingTestFileSystem].getName, + "fs.file.impl.disable.cache" -> "true") { + withTable("t") { + sql( + """ + |CREATE TABLE t(i INT, part1 INT) USING PARQUET + |PARTITIONED BY (part1) + """.stripMargin) + + val df = Seq((1, 1)).toDF("i", "part1") + val err = intercept[SparkException] { + df.write.mode("overwrite").format("parquet").insertInto("t") + } + assert(err.getCause.getMessage.contains("can not write to output file: " + + "org.apache.hadoop.fs.FileAlreadyExistsException")) + } + } + } +} + +class FileExistingTestFileSystem extends RawLocalFileSystem { + override def create( + f: Path, + overwrite: Boolean, + bufferSize: Int, + replication: Short, + blockSize: Long): FSDataOutputStream = { + throw new FileAlreadyExistsException(s"${f.toString} already exists") + } } From fe1f456b200075bef476b5d8eca2be505b486033 Mon Sep 17 00:00:00 2001 From: wuyi Date: Thu, 14 Nov 2019 11:55:01 +0800 Subject: [PATCH 0610/1387] [SPARK-29837][SQL] PostgreSQL dialect: cast to boolean ### What changes were proposed in this pull request? Make SparkSQL's `cast to boolean` behavior be consistent with PostgreSQL when spark.sql.dialect is configured as PostgreSQL. ### Why are the changes needed? SparkSQL and PostgreSQL have a lot different cast behavior between types by default. We should make SparkSQL's cast behavior be consistent with PostgreSQL when `spark.sql.dialect` is configured as PostgreSQL. ### Does this PR introduce any user-facing change? Yes. If user switches to PostgreSQL dialect now, they will * get an exception if they input a invalid string, e.g "erut", while they get `null` before; * get an exception if they input `TimestampType`, `DateType`, `LongType`, `ShortType`, `ByteType`, `DecimalType`, `DoubleType`, `FloatType` values, while they get `true` or `false` result before. And here're evidences for those unsupported types from PostgreSQL: timestamp: ``` postgres=# select cast(cast('2019-11-11' as timestamp) as boolean); ERROR: cannot cast type timestamp without time zone to boolean ``` date: ``` postgres=# select cast(cast('2019-11-11' as date) as boolean); ERROR: cannot cast type date to boolean ``` bigint: ``` postgres=# select cast(cast('20191111' as bigint) as boolean); ERROR: cannot cast type bigint to boolean ``` smallint: ``` postgres=# select cast(cast(2019 as smallint) as boolean); ERROR: cannot cast type smallint to boolean ``` bytea: ``` postgres=# select cast(cast('2019' as bytea) as boolean); ERROR: cannot cast type bytea to boolean ``` decimal: ``` postgres=# select cast(cast('2019' as decimal) as boolean); ERROR: cannot cast type numeric to boolean ``` float: ``` postgres=# select cast(cast('2019' as float) as boolean); ERROR: cannot cast type double precision to boolean ``` ### How was this patch tested? Added and tested manually. Closes #26463 from Ngone51/dev-postgre-cast2bool. Authored-by: wuyi Signed-off-by: Wenchen Fan --- .../catalyst/analysis/PostgreSQLDialect.scala | 12 +-- .../spark/sql/catalyst/expressions/Cast.scala | 10 +-- .../PostgreCastStringToBoolean.scala | 80 ------------------ .../postgreSQL/PostgreCastToBoolean.scala | 83 +++++++++++++++++++ .../expressions/postgreSQL/CastSuite.scala | 74 ++++++++++------- .../results/postgreSQL/boolean.sql.out | 60 ++++++++------ .../sql/PostgreSQLDialectQuerySuite.scala | 2 +- 7 files changed, 175 insertions(+), 146 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala index 934e53703e241..e7f0e571804d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Cast -import org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastStringToBoolean +import org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -27,19 +27,19 @@ import org.apache.spark.sql.types.{BooleanType, StringType} object PostgreSQLDialect { val postgreSQLDialectRules: List[Rule[LogicalPlan]] = - CastStringToBoolean :: + CastToBoolean :: Nil - object CastStringToBoolean extends Rule[LogicalPlan] with Logging { + object CastToBoolean extends Rule[LogicalPlan] with Logging { override def apply(plan: LogicalPlan): LogicalPlan = { // The SQL configuration `spark.sql.dialect` can be changed in runtime. // To make sure the configuration is effective, we have to check it during rule execution. val conf = SQLConf.get if (conf.usePostgreSQLDialect) { plan.transformExpressions { - case Cast(child, dataType, _) - if dataType == BooleanType && child.dataType == StringType => - PostgreCastStringToBoolean(child) + case Cast(child, dataType, timeZoneId) + if child.dataType != BooleanType && dataType == BooleanType => + PostgreCastToBoolean(child, timeZoneId) } } else { plan 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 8cd076e3c63ac..f3b58fa3137b1 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 @@ -274,7 +274,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def needsTimeZone: Boolean = Cast.needsTimeZone(child.dataType, dataType) // [[func]] assumes the input is no longer null because eval already does the null check. - @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) + @inline protected def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) @@ -377,7 +377,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } // UDFToBoolean - private[this] def castToBoolean(from: DataType): Any => Any = from match { + protected[this] def castToBoolean(from: DataType): Any => Any = from match { case StringType => buildCast[UTF8String](_, s => { if (StringUtils.isTrueString(s)) { @@ -782,7 +782,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } } - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = child.genCode(ctx) val nullSafeCast = nullSafeCastFunction(child.dataType, dataType, ctx) @@ -792,7 +792,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // The function arguments are: `input`, `result` and `resultIsNull`. We don't need `inputIsNull` // in parameter list, because the returned code will be put in null safe evaluation region. - private[this] type CastFunction = (ExprValue, ExprValue, ExprValue) => Block + protected type CastFunction = (ExprValue, ExprValue, ExprValue) => Block private[this] def nullSafeCastFunction( from: DataType, @@ -1234,7 +1234,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def timestampToDoubleCode(ts: ExprValue): Block = code"$ts / (double)$MICROS_PER_SECOND" - private[this] def castToBooleanCode(from: DataType): CastFunction = from match { + protected[this] def castToBooleanCode(from: DataType): CastFunction = from match { case StringType => val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" (c, evPrim, evNull) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala deleted file mode 100644 index 0e87707d01e47..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala +++ /dev/null @@ -1,80 +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.sql.catalyst.expressions.postgreSQL - -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant, UnaryExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, JavaCode} -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.util.postgreSQL.StringUtils -import org.apache.spark.sql.types.{BooleanType, DataType, StringType} -import org.apache.spark.unsafe.types.UTF8String - -case class PostgreCastStringToBoolean(child: Expression) - extends UnaryExpression with NullIntolerant { - - override def checkInputDataTypes(): TypeCheckResult = { - if (child.dataType == StringType) { - TypeCheckResult.TypeCheckSuccess - } else { - TypeCheckResult.TypeCheckFailure( - s"The expression ${getClass.getSimpleName} only accepts string input data type") - } - } - - override def nullSafeEval(input: Any): Any = { - val s = input.asInstanceOf[UTF8String].trim().toLowerCase() - if (StringUtils.isTrueString(s)) { - true - } else if (StringUtils.isFalseString(s)) { - false - } else { - null - } - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" - val eval = child.genCode(ctx) - val javaType = JavaCode.javaType(dataType) - val preprocessedString = ctx.freshName("preprocessedString") - val castCode = - code""" - boolean ${ev.isNull} = ${eval.isNull}; - $javaType ${ev.value} = false; - if (!${eval.isNull}) { - UTF8String $preprocessedString = ${eval.value}.trim().toLowerCase(); - if ($stringUtils.isTrueString($preprocessedString)) { - ${ev.value} = true; - } else if ($stringUtils.isFalseString($preprocessedString)) { - ${ev.value} = false; - } else { - ${ev.isNull} = true; - } - } - """ - ev.copy(code = eval.code + castCode) - } - - override def dataType: DataType = BooleanType - - override def nullable: Boolean = true - - override def toString: String = s"PostgreCastStringToBoolean($child as ${dataType.simpleString})" - - override def sql: String = s"CAST(${child.sql} AS ${dataType.sql})" -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala new file mode 100644 index 0000000000000..20559ba3cd79e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala @@ -0,0 +1,83 @@ +/* + * 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.postgreSQL + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.{CastBase, Expression, TimeZoneAwareExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.util.postgreSQL.StringUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +case class PostgreCastToBoolean(child: Expression, timeZoneId: Option[String]) + extends CastBase { + + override protected def ansiEnabled = + throw new UnsupportedOperationException("PostgreSQL dialect doesn't support ansi mode") + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override def checkInputDataTypes(): TypeCheckResult = child.dataType match { + case StringType | IntegerType | NullType => + TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"cannot cast type ${child.dataType} to boolean") + } + + override def castToBoolean(from: DataType): Any => Any = from match { + case StringType => + buildCast[UTF8String](_, str => { + val s = str.trim().toLowerCase() + if (StringUtils.isTrueString(s)) { + true + } else if (StringUtils.isFalseString(s)) { + false + } else { + throw new IllegalArgumentException(s"invalid input syntax for type boolean: $s") + } + }) + case IntegerType => + super.castToBoolean(from) + } + + override def castToBooleanCode(from: DataType): CastFunction = from match { + case StringType => + val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" + (c, evPrim, evNull) => + code""" + if ($stringUtils.isTrueString($c.trim().toLowerCase())) { + $evPrim = true; + } else if ($stringUtils.isFalseString($c.trim().toLowerCase())) { + $evPrim = false; + } else { + throw new IllegalArgumentException("invalid input syntax for type boolean: $c"); + } + """ + + case IntegerType => + super.castToBooleanCode(from) + } + + override def dataType: DataType = BooleanType + + override def nullable: Boolean = child.nullable + + override def toString: String = s"PostgreCastToBoolean($child as ${dataType.simpleString})" + + override def sql: String = s"CAST(${child.sql} AS ${dataType.sql})" +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala index 175904da21969..6c5218b379f31 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala @@ -16,44 +16,58 @@ */ package org.apache.spark.sql.catalyst.expressions.postgreSQL +import java.sql.{Date, Timestamp} + import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} class CastSuite extends SparkFunSuite with ExpressionEvalHelper { - private def checkPostgreCastStringToBoolean(v: Any, expected: Any): Unit = { - checkEvaluation(PostgreCastStringToBoolean(Literal(v)), expected) + private def checkPostgreCastToBoolean(v: Any, expected: Any): Unit = { + checkEvaluation(PostgreCastToBoolean(Literal(v), None), expected) } test("cast string to boolean") { - checkPostgreCastStringToBoolean("true", true) - checkPostgreCastStringToBoolean("tru", true) - checkPostgreCastStringToBoolean("tr", true) - checkPostgreCastStringToBoolean("t", true) - checkPostgreCastStringToBoolean("tRUe", true) - checkPostgreCastStringToBoolean(" tRue ", true) - checkPostgreCastStringToBoolean(" tRu ", true) - checkPostgreCastStringToBoolean("yes", true) - checkPostgreCastStringToBoolean("ye", true) - checkPostgreCastStringToBoolean("y", true) - checkPostgreCastStringToBoolean("1", true) - checkPostgreCastStringToBoolean("on", true) + checkPostgreCastToBoolean("true", true) + checkPostgreCastToBoolean("tru", true) + checkPostgreCastToBoolean("tr", true) + checkPostgreCastToBoolean("t", true) + checkPostgreCastToBoolean("tRUe", true) + checkPostgreCastToBoolean(" tRue ", true) + checkPostgreCastToBoolean(" tRu ", true) + checkPostgreCastToBoolean("yes", true) + checkPostgreCastToBoolean("ye", true) + checkPostgreCastToBoolean("y", true) + checkPostgreCastToBoolean("1", true) + checkPostgreCastToBoolean("on", true) + + checkPostgreCastToBoolean("false", false) + checkPostgreCastToBoolean("fals", false) + checkPostgreCastToBoolean("fal", false) + checkPostgreCastToBoolean("fa", false) + checkPostgreCastToBoolean("f", false) + checkPostgreCastToBoolean(" fAlse ", false) + checkPostgreCastToBoolean(" fAls ", false) + checkPostgreCastToBoolean(" FAlsE ", false) + checkPostgreCastToBoolean("no", false) + checkPostgreCastToBoolean("n", false) + checkPostgreCastToBoolean("0", false) + checkPostgreCastToBoolean("off", false) + checkPostgreCastToBoolean("of", false) - checkPostgreCastStringToBoolean("false", false) - checkPostgreCastStringToBoolean("fals", false) - checkPostgreCastStringToBoolean("fal", false) - checkPostgreCastStringToBoolean("fa", false) - checkPostgreCastStringToBoolean("f", false) - checkPostgreCastStringToBoolean(" fAlse ", false) - checkPostgreCastStringToBoolean(" fAls ", false) - checkPostgreCastStringToBoolean(" FAlsE ", false) - checkPostgreCastStringToBoolean("no", false) - checkPostgreCastStringToBoolean("n", false) - checkPostgreCastStringToBoolean("0", false) - checkPostgreCastStringToBoolean("off", false) - checkPostgreCastStringToBoolean("of", false) + intercept[IllegalArgumentException](PostgreCastToBoolean(Literal("o"), None).eval()) + intercept[IllegalArgumentException](PostgreCastToBoolean(Literal("abc"), None).eval()) + intercept[IllegalArgumentException](PostgreCastToBoolean(Literal(""), None).eval()) + } - checkPostgreCastStringToBoolean("o", null) - checkPostgreCastStringToBoolean("abc", null) - checkPostgreCastStringToBoolean("", null) + test("unsupported data types to cast to boolean") { + assert(PostgreCastToBoolean(Literal(new Timestamp(1)), None).checkInputDataTypes().isFailure) + assert(PostgreCastToBoolean(Literal(new Date(1)), None).checkInputDataTypes().isFailure) + assert(PostgreCastToBoolean(Literal(1.toLong), None).checkInputDataTypes().isFailure) + assert(PostgreCastToBoolean(Literal(1.toShort), None).checkInputDataTypes().isFailure) + assert(PostgreCastToBoolean(Literal(1.toByte), None).checkInputDataTypes().isFailure) + assert(PostgreCastToBoolean(Literal(BigDecimal(1.0)), None).checkInputDataTypes().isFailure) + assert(PostgreCastToBoolean(Literal(1.toDouble), None).checkInputDataTypes().isFailure) + assert(PostgreCastToBoolean(Literal(1.toFloat), None).checkInputDataTypes().isFailure) } } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out index 203806d43368a..e5f3425efc458 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out @@ -53,9 +53,10 @@ true -- !query 6 SELECT boolean('test') AS error -- !query 6 schema -struct +struct<> -- !query 6 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: test -- !query 7 @@ -69,9 +70,10 @@ false -- !query 8 SELECT boolean('foo') AS error -- !query 8 schema -struct +struct<> -- !query 8 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: foo -- !query 9 @@ -93,9 +95,10 @@ true -- !query 11 SELECT boolean('yeah') AS error -- !query 11 schema -struct +struct<> -- !query 11 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: yeah -- !query 12 @@ -117,9 +120,10 @@ false -- !query 14 SELECT boolean('nay') AS error -- !query 14 schema -struct +struct<> -- !query 14 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: nay -- !query 15 @@ -149,25 +153,28 @@ false -- !query 18 SELECT boolean('o') AS error -- !query 18 schema -struct +struct<> -- !query 18 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: o -- !query 19 SELECT boolean('on_') AS error -- !query 19 schema -struct +struct<> -- !query 19 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: on_ -- !query 20 SELECT boolean('off_') AS error -- !query 20 schema -struct +struct<> -- !query 20 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: off_ -- !query 21 @@ -181,9 +188,10 @@ true -- !query 22 SELECT boolean('11') AS error -- !query 22 schema -struct +struct<> -- !query 22 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: 11 -- !query 23 @@ -197,17 +205,19 @@ false -- !query 24 SELECT boolean('000') AS error -- !query 24 schema -struct +struct<> -- !query 24 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: 000 -- !query 25 SELECT boolean('') AS error -- !query 25 schema -struct +struct<> -- !query 25 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: -- !query 26 @@ -310,17 +320,19 @@ true false -- !query 38 SELECT boolean(string(' tru e ')) AS invalid -- !query 38 schema -struct +struct<> -- !query 38 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: tru e -- !query 39 SELECT boolean(string('')) AS invalid -- !query 39 schema -struct +struct<> -- !query 39 output -NULL +java.lang.IllegalArgumentException +invalid input syntax for type boolean: -- !query 40 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala index 1354dcfda45fe..7056f483609a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala @@ -36,7 +36,7 @@ class PostgreSQLDialectQuerySuite extends QueryTest with SharedSparkSession { } Seq("o", "abc", "").foreach { input => - checkAnswer(sql(s"select cast('$input' as boolean)"), Row(null)) + intercept[IllegalArgumentException](sql(s"select cast('$input' as boolean)").collect()) } } } From b5a02d37e63d512ea3521d8b314208a0c7ec31a0 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 14 Nov 2019 14:38:27 +0800 Subject: [PATCH 0611/1387] [SPARK-29873][SQL][TESTS] Support `--import` directive to load queries from another test case in SQLQueryTestSuite ### What changes were proposed in this pull request? This pr is to support `--import` directive to load queries from another test case in SQLQueryTestSuite. This fix comes from the cloud-fan suggestion in https://github.com/apache/spark/pull/26479#discussion_r345086978 ### Why are the changes needed? This functionality might reduce duplicate test code in `SQLQueryTestSuite`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Run `SQLQueryTestSuite`. Closes #26497 from maropu/ImportTests. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../ansi/decimalArithmeticOperations.sql | 32 ++ .../inputs/ansi/higher-order-functions.sql | 1 + .../sql-tests/inputs/ansi/interval.sql | 9 +- .../sql-tests/inputs/ansi/literals.sql | 2 + .../inputs/decimalArithmeticOperations.sql | 24 - .../inputs/higher-order-functions.sql | 5 - .../resources/sql-tests/inputs/interval.sql | 43 ++ .../resources/sql-tests/inputs/literals.sql | 53 +- .../ansi/decimalArithmeticOperations.sql.out | 138 +++++ .../ansi/higher-order-functions.sql.out | 284 ++++++++++ .../sql-tests/results/ansi/interval.sql.out | 246 ++++---- .../sql-tests/results/ansi/literals.sql.out | 474 ++++++++++++++++ .../decimalArithmeticOperations.sql.out | 129 +---- .../results/higher-order-functions.sql.out | 46 +- .../sql-tests/results/interval.sql.out | 322 ++++++++++- .../sql-tests/results/literals.sql.out | 532 ++---------------- .../apache/spark/sql/SQLQueryTestSuite.scala | 36 +- .../ThriftServerQueryTestSuite.scala | 11 +- 18 files changed, 1518 insertions(+), 869 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql new file mode 100644 index 0000000000000..0de4e2b91e7e9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql @@ -0,0 +1,32 @@ +-- SPARK-23179: SQL ANSI 2011 states that in case of overflow during arithmetic operations, +-- an exception should be thrown instead of returning NULL. +-- This is what most of the SQL DBs do (eg. SQLServer, DB2). + +-- tests for decimals handling in operations +create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet; + +insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), + (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789); + +-- test operations between decimals and constants +select id, a*10, b/10 from decimals_test order by id; + +-- test operations on constants +select 10.3 * 3.0; +select 10.3000 * 3.0; +select 10.30000 * 30.0; +select 10.300000000000000000 * 3.000000000000000000; +select 10.300000000000000000 * 3.0000000000000000000; + +-- arithmetic operations causing an overflow throw exception +select (5e36 + 0.1) + 5e36; +select (-4e36 - 0.1) - 7e36; +select 12345678901234567890.0 * 12345678901234567890.0; +select 1e35 / 0.1; + +-- arithmetic operations causing a precision loss throw exception +select 123456789123456789.1234567890 * 1.123456789123456789; +select 123456789123456789.1234567890 * 1.123456789123456789; +select 12345678912345.123456789123 / 0.000000012345678; + +drop table decimals_test; diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql new file mode 100644 index 0000000000000..4068a27fcb2a7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql @@ -0,0 +1 @@ +--import higher-order-functions.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql index 4a831855a091a..f662769f016c5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql @@ -1,6 +1,3 @@ --- Turns on ANSI mode -SET spark.sql.ansi.enabled=true; - select '1' second, 2 seconds, @@ -184,5 +181,7 @@ select date '2012-01-01' + interval (a + 1) day from t; select date '2012-01-01' + (a + 1) day from t; --- Turns off ANSI mode -SET spark.sql.ansi.enabled=false; +-- malformed interval literal with ansi mode +select 1 year to month; +select '1' year to second; +select 1 year '2-1' year to month; diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql new file mode 100644 index 0000000000000..170690ea699c0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql @@ -0,0 +1,2 @@ +--- malformed interval literal with ansi mode +--import literals.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql index a946123f709c7..28a0e20c0f495 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql @@ -83,28 +83,4 @@ select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.1 select 123456789123456789.1234567890 * 1.123456789123456789; select 12345678912345.123456789123 / 0.000000012345678; --- throw an exception instead of returning NULL, according to SQL ANSI 2011 -set spark.sql.ansi.enabled=true; - --- test operations between decimals and constants -select id, a*10, b/10 from decimals_test order by id; - --- test operations on constants -select 10.3 * 3.0; -select 10.3000 * 3.0; -select 10.30000 * 30.0; -select 10.300000000000000000 * 3.000000000000000000; -select 10.300000000000000000 * 3.0000000000000000000; - --- arithmetic operations causing an overflow throw exception -select (5e36 + 0.1) + 5e36; -select (-4e36 - 0.1) - 7e36; -select 12345678901234567890.0 * 12345678901234567890.0; -select 1e35 / 0.1; - --- arithmetic operations causing a precision loss throw exception -select 123456789123456789.1234567890 * 1.123456789123456789; -select 123456789123456789.1234567890 * 1.123456789123456789; -select 12345678912345.123456789123 / 0.000000012345678; - drop table decimals_test; diff --git a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql index 8d5d9fae7a733..7665346f86ba8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql @@ -87,8 +87,3 @@ select transform_values(ys, (k, v) -> k + v) as v from nested; -- use non reversed keywords: all is non reversed only if !ansi select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys); select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys); - -set spark.sql.ansi.enabled=true; -select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys); -select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys); -set spark.sql.ansi.enabled=false; diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index 226c13f961a85..b5c3416d0fc66 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -88,3 +88,46 @@ select justify_interval(interval '1 month -59 day 25 hour'); select justify_days(interval '1 month 59 day -25 hour'); select justify_hours(interval '1 month 59 day -25 hour'); select justify_interval(interval '1 month 59 day -25 hour'); + +-- interval +select interval 13.123456789 seconds, interval -13.123456789 second; +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond; +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; +select interval '0 0:0:0.1' day to second; +select interval '10-9' year to month; +select interval '20 15:40:32.99899999' day to hour; +select interval '20 15:40:32.99899999' day to minute; +select interval '20 15:40:32.99899999' day to second; +select interval '15:40:32.99899999' hour to minute; +select interval '15:40.99899999' hour to second; +select interval '15:40' hour to second; +select interval '15:40:32.99899999' hour to second; +select interval '20 40:32.99899999' minute to second; +select interval '40:32.99899999' minute to second; +select interval '40:32' minute to second; + +-- ns is not supported +select interval 10 nanoseconds; + +-- map + interval test +select map(1, interval 1 day, 2, interval 3 week); + +-- typed interval expression +select interval 'interval 3 year 1 hour'; +select interval '3 year 1 hour'; + +-- malformed interval literal +select interval; +select interval 1 fake_unit; +select interval 1 year to month; +select interval '1' year to second; +select interval '10-9' year to month '2-1' year to month; +select interval '10-9' year to month '12:11:10' hour to second; +select interval '1 15:11' day to minute '12:11:10' hour to second; +select interval 1 year '2-1' year to month; +select interval 1 year '12:11:10' hour to second; +select interval '10-9' year to month '1' year; +select interval '12:11:10' hour to second '1' year; + +-- awareness of the positive sign before interval type +select +interval '1 second'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 8c927d456c8be..d1dff7bc94686 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -82,25 +82,6 @@ select tImEstAmp '2016-03-11 20:54:00.000'; -- invalid timestamp select timestamp '2016-33-11 20:54:00.000'; --- interval -select interval 13.123456789 seconds, interval -13.123456789 second; -select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond; -select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; -select interval '0 0:0:0.1' day to second; -select interval '10-9' year to month; -select interval '20 15:40:32.99899999' day to hour; -select interval '20 15:40:32.99899999' day to minute; -select interval '20 15:40:32.99899999' day to second; -select interval '15:40:32.99899999' hour to minute; -select interval '15:40.99899999' hour to second; -select interval '15:40' hour to second; -select interval '15:40:32.99899999' hour to second; -select interval '20 40:32.99899999' minute to second; -select interval '40:32.99899999' minute to second; -select interval '40:32' minute to second; --- ns is not supported -select interval 10 nanoseconds; - -- unsupported data type select GEO '(10,-6)'; @@ -119,46 +100,14 @@ select X'XuZ'; -- Hive literal_double test. SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8; --- map + interval test -select map(1, interval 1 day, 2, interval 3 week); - --- typed interval expression -select interval 'interval 3 year 1 hour'; -select interval '3 year 1 hour'; - -- typed integer expression select integer '7'; select integer'7'; select integer '2147483648'; --- malformed interval literal -select interval; -select interval 1 fake_unit; -select interval 1 year to month; -select interval '1' year to second; -select interval '10-9' year to month '2-1' year to month; -select interval '10-9' year to month '12:11:10' hour to second; -select interval '1 15:11' day to minute '12:11:10' hour to second; -select interval 1 year '2-1' year to month; -select interval 1 year '12:11:10' hour to second; -select interval '10-9' year to month '1' year; -select interval '12:11:10' hour to second '1' year; --- malformed interval literal with ansi mode -SET spark.sql.ansi.enabled=true; -select interval; -select interval 1 fake_unit; -select interval 1 year to month; -select 1 year to month; -select interval '1' year to second; -select '1' year to second; -select interval 1 year '2-1' year to month; -select 1 year '2-1' year to month; -SET spark.sql.ansi.enabled=false; - --- awareness of the negative sign before type +-- awareness of the negative/positive sign before type select -integer '7'; select -date '1999-01-01'; select -timestamp '1999-01-01'; select -x'2379ACFe'; select +integer '7'; -select +interval '1 second'; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out new file mode 100644 index 0000000000000..e12f409f6d47e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -0,0 +1,138 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 16 + + +-- !query 0 +create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), + (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +select id, a*10, b/10 from decimals_test order by id +-- !query 2 schema +struct +-- !query 2 output +1 1000 99.9 +2 123451.23 1234.5123 +3 1.234567891011 123.41 +4 1234567891234567890 0.112345678912345679 + + +-- !query 3 +select 10.3 * 3.0 +-- !query 3 schema +struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> +-- !query 3 output +30.9 + + +-- !query 4 +select 10.3000 * 3.0 +-- !query 4 schema +struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> +-- !query 4 output +30.9 + + +-- !query 5 +select 10.30000 * 30.0 +-- !query 5 schema +struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> +-- !query 5 output +309 + + +-- !query 6 +select 10.300000000000000000 * 3.000000000000000000 +-- !query 6 schema +struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> +-- !query 6 output +30.9 + + +-- !query 7 +select 10.300000000000000000 * 3.0000000000000000000 +-- !query 7 schema +struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> +-- !query 7 output +30.9 + + +-- !query 8 +select (5e36 + 0.1) + 5e36 +-- !query 8 schema +struct<> +-- !query 8 output +java.lang.ArithmeticException +Decimal(expanded,10000000000000000000000000000000000000.1,39,1}) cannot be represented as Decimal(38, 1). + + +-- !query 9 +select (-4e36 - 0.1) - 7e36 +-- !query 9 schema +struct<> +-- !query 9 output +java.lang.ArithmeticException +Decimal(expanded,-11000000000000000000000000000000000000.1,39,1}) cannot be represented as Decimal(38, 1). + + +-- !query 10 +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query 10 schema +struct<> +-- !query 10 output +java.lang.ArithmeticException +Decimal(expanded,1.5241578753238836750190519987501905210E+38,38,-1}) cannot be represented as Decimal(38, 2). + + +-- !query 11 +select 1e35 / 0.1 +-- !query 11 schema +struct<> +-- !query 11 output +java.lang.ArithmeticException +Decimal(expanded,1000000000000000000000000000000000000,37,0}) cannot be represented as Decimal(38, 6). + + +-- !query 12 +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query 12 schema +struct<(CAST(123456789123456789.1234567890 AS DECIMAL(36,18)) * CAST(1.123456789123456789 AS DECIMAL(36,18))):decimal(38,18)> +-- !query 12 output +138698367904130467.654320988515622621 + + +-- !query 13 +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query 13 schema +struct<(CAST(123456789123456789.1234567890 AS DECIMAL(36,18)) * CAST(1.123456789123456789 AS DECIMAL(36,18))):decimal(38,18)> +-- !query 13 output +138698367904130467.654320988515622621 + + +-- !query 14 +select 12345678912345.123456789123 / 0.000000012345678 +-- !query 14 schema +struct<(CAST(12345678912345.123456789123 AS DECIMAL(29,15)) / CAST(1.2345678E-8 AS DECIMAL(29,15))):decimal(38,9)> +-- !query 14 output +1000000073899961059796.725866332 + + +-- !query 15 +drop table decimals_test +-- !query 15 schema +struct<> +-- !query 15 output + diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out new file mode 100644 index 0000000000000..e7dae6595a890 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out @@ -0,0 +1,284 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 29 + + +-- !query 0 +create or replace temporary view nested as values + (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), + (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), + (3, array(12), array(array(17))) + as t(x, ys, zs) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select upper(x -> x) as v +-- !query 1 schema +struct<> +-- !query 1 output +org.apache.spark.sql.AnalysisException +A lambda function should only be used in a higher order function. However, its class is org.apache.spark.sql.catalyst.expressions.Upper, which is not a higher order function.; line 1 pos 7 + + +-- !query 2 +select transform(zs, z -> z) as v from nested +-- !query 2 schema +struct>> +-- !query 2 output +[[12,99],[123,42],[1]] +[[17]] +[[6,96,65],[-1,-2]] + + +-- !query 3 +select transform(ys, y -> y * y) as v from nested +-- !query 3 schema +struct> +-- !query 3 output +[1024,9409] +[144] +[5929,5776] + + +-- !query 4 +select transform(ys, (y, i) -> y + i) as v from nested +-- !query 4 schema +struct> +-- !query 4 output +[12] +[32,98] +[77,-75] + + +-- !query 5 +select transform(zs, z -> concat(ys, z)) as v from nested +-- !query 5 schema +struct>> +-- !query 5 output +[[12,17]] +[[32,97,12,99],[32,97,123,42],[32,97,1]] +[[77,-76,6,96,65],[77,-76,-1,-2]] + + +-- !query 6 +select transform(ys, 0) as v from nested +-- !query 6 schema +struct> +-- !query 6 output +[0,0] +[0,0] +[0] + + +-- !query 7 +select transform(cast(null as array), x -> x + 1) as v +-- !query 7 schema +struct> +-- !query 7 output +NULL + + +-- !query 8 +select filter(ys, y -> y > 30) as v from nested +-- !query 8 schema +struct> +-- !query 8 output +[32,97] +[77] +[] + + +-- !query 9 +select filter(cast(null as array), y -> true) as v +-- !query 9 schema +struct> +-- !query 9 output +NULL + + +-- !query 10 +select transform(zs, z -> filter(z, zz -> zz > 50)) as v from nested +-- !query 10 schema +struct>> +-- !query 10 output +[[96,65],[]] +[[99],[123],[]] +[[]] + + +-- !query 11 +select aggregate(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query 11 schema +struct +-- !query 11 output +131 +15 +5 + + +-- !query 12 +select aggregate(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query 12 schema +struct +-- !query 12 output +0.5 +12.0 +64.5 + + +-- !query 13 +select transform(zs, z -> aggregate(z, 1, (acc, val) -> acc * val * size(z))) as v from nested +-- !query 13 schema +struct> +-- !query 13 output +[1010880,8] +[17] +[4752,20664,1] + + +-- !query 14 +select aggregate(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query 14 schema +struct +-- !query 14 output +NULL + + +-- !query 15 +select exists(ys, y -> y > 30) as v from nested +-- !query 15 schema +struct +-- !query 15 output +false +true +true + + +-- !query 16 +select exists(cast(null as array), y -> y > 30) as v +-- !query 16 schema +struct +-- !query 16 output +NULL + + +-- !query 17 +select zip_with(ys, zs, (a, b) -> a + size(b)) as v from nested +-- !query 17 schema +struct> +-- !query 17 output +[13] +[34,99,null] +[80,-74] + + +-- !query 18 +select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)) as v +-- !query 18 schema +struct> +-- !query 18 output +["ad","be","cf"] + + +-- !query 19 +select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v +-- !query 19 schema +struct> +-- !query 19 output +["a",null,"f"] + + +-- !query 20 +create or replace temporary view nested as values + (1, map(1, 1, 2, 2, 3, 3)), + (2, map(4, 4, 5, 5, 6, 6)) + as t(x, ys) +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +select transform_keys(ys, (k, v) -> k) as v from nested +-- !query 21 schema +struct> +-- !query 21 output +{1:1,2:2,3:3} +{4:4,5:5,6:6} + + +-- !query 22 +select transform_keys(ys, (k, v) -> k + 1) as v from nested +-- !query 22 schema +struct> +-- !query 22 output +{2:1,3:2,4:3} +{5:4,6:5,7:6} + + +-- !query 23 +select transform_keys(ys, (k, v) -> k + v) as v from nested +-- !query 23 schema +struct> +-- !query 23 output +{10:5,12:6,8:4} +{2:1,4:2,6:3} + + +-- !query 24 +select transform_values(ys, (k, v) -> v) as v from nested +-- !query 24 schema +struct> +-- !query 24 output +{1:1,2:2,3:3} +{4:4,5:5,6:6} + + +-- !query 25 +select transform_values(ys, (k, v) -> v + 1) as v from nested +-- !query 25 schema +struct> +-- !query 25 output +{1:2,2:3,3:4} +{4:5,5:6,6:7} + + +-- !query 26 +select transform_values(ys, (k, v) -> k + v) as v from nested +-- !query 26 schema +struct> +-- !query 26 output +{1:2,2:4,3:6} +{4:8,5:10,6:12} + + +-- !query 27 +select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) +-- !query 27 schema +struct<> +-- !query 27 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'all'(line 1, pos 21) + +== SQL == +select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) +---------------------^^^ + + +-- !query 28 +select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) +-- !query 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'all'(line 1, pos 22) + +== SQL == +select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) +----------------------^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 0085cacf0d0b3..a9c90ccc80938 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,16 +1,8 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 35 +-- Number of queries: 36 -- !query 0 -SET spark.sql.ansi.enabled=true --- !query 0 schema -struct --- !query 0 output -spark.sql.ansi.enabled true - - --- !query 1 select '1' second, 2 seconds, @@ -24,35 +16,35 @@ select 2 months, '1' year, 2 years --- !query 1 schema +-- !query 0 schema struct<1 seconds:interval,2 seconds:interval,1 minutes:interval,2 minutes:interval,1 hours:interval,2 hours:interval,1 days:interval,2 days:interval,1 months:interval,2 months:interval,1 years:interval,2 years:interval> --- !query 1 output +-- !query 0 output 1 seconds 2 seconds 1 minutes 2 minutes 1 hours 2 hours 1 days 2 days 1 months 2 months 1 years 2 years --- !query 2 +-- !query 1 select interval '10-11' year to month, interval '10' year, interval '11' month --- !query 2 schema +-- !query 1 schema struct<10 years 11 months:interval,10 years:interval,11 months:interval> --- !query 2 output +-- !query 1 output 10 years 11 months 10 years 11 months --- !query 3 +-- !query 2 select '10-11' year to month, '10' year, '11' month --- !query 3 schema +-- !query 2 schema struct<10 years 11 months:interval,10 years:interval,11 months:interval> --- !query 3 output +-- !query 2 output 10 years 11 months 10 years 11 months --- !query 4 +-- !query 3 select interval '10 9:8:7.987654321' day to second, interval '10' day, @@ -60,13 +52,13 @@ select interval '12' minute, interval '13' second, interval '13.123456789' second --- !query 4 schema +-- !query 3 schema struct<10 days 9 hours 8 minutes 7.987654 seconds:interval,10 days:interval,11 hours:interval,12 minutes:interval,13 seconds:interval,13.123456 seconds:interval> --- !query 4 output +-- !query 3 output 10 days 9 hours 8 minutes 7.987654 seconds 10 days 11 hours 12 minutes 13 seconds 13.123456 seconds --- !query 5 +-- !query 4 select '10 9:8:7.987654321' day to second, '10' day, @@ -74,40 +66,40 @@ select '12' minute, '13' second, '13.123456789' second --- !query 5 schema +-- !query 4 schema struct<10 days 9 hours 8 minutes 7.987654 seconds:interval,10 days:interval,11 hours:interval,12 minutes:interval,13 seconds:interval,13.123456 seconds:interval> --- !query 5 output +-- !query 4 output 10 days 9 hours 8 minutes 7.987654 seconds 10 days 11 hours 12 minutes 13 seconds 13.123456 seconds --- !query 6 +-- !query 5 select map(1, interval 1 day, 2, interval 3 week) --- !query 6 schema +-- !query 5 schema struct> --- !query 6 output +-- !query 5 output {1:1 days,2:21 days} --- !query 7 +-- !query 6 select map(1, 1 day, 2, 3 week) --- !query 7 schema +-- !query 6 schema struct> --- !query 7 output +-- !query 6 output {1:1 days,2:21 days} --- !query 8 +-- !query 7 create temporary view interval_arithmetic as select CAST(dateval AS date), CAST(tsval AS timestamp) from values ('2012-01-01', '2012-01-01') as interval_arithmetic(dateval, tsval) --- !query 8 schema +-- !query 7 schema struct<> --- !query 8 output +-- !query 7 output --- !query 9 +-- !query 8 select dateval, dateval - interval '2-2' year to month, @@ -117,13 +109,13 @@ select - interval '2-2' year to month + dateval, interval '2-2' year to month + dateval from interval_arithmetic --- !query 9 schema +-- !query 8 schema struct --- !query 9 output +-- !query 8 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 --- !query 10 +-- !query 9 select dateval, dateval - '2-2' year to month, @@ -133,13 +125,13 @@ select - '2-2' year to month + dateval, '2-2' year to month + dateval from interval_arithmetic --- !query 10 schema +-- !query 9 schema struct --- !query 10 output +-- !query 9 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 --- !query 11 +-- !query 10 select tsval, tsval - interval '2-2' year to month, @@ -149,13 +141,13 @@ select - interval '2-2' year to month + tsval, interval '2-2' year to month + tsval from interval_arithmetic --- !query 11 schema +-- !query 10 schema struct --- !query 11 output +-- !query 10 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 --- !query 12 +-- !query 11 select tsval, tsval - '2-2' year to month, @@ -165,35 +157,35 @@ select - '2-2' year to month + tsval, '2-2' year to month + tsval from interval_arithmetic --- !query 12 schema +-- !query 11 schema struct --- !query 12 output +-- !query 11 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 --- !query 13 +-- !query 12 select interval '2-2' year to month + interval '3-3' year to month, interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic --- !query 13 schema +-- !query 12 schema struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> --- !query 13 output +-- !query 12 output 5 years 5 months -1 years -1 months --- !query 14 +-- !query 13 select '2-2' year to month + '3-3' year to month, '2-2' year to month - '3-3' year to month from interval_arithmetic --- !query 14 schema +-- !query 13 schema struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> --- !query 14 output +-- !query 13 output 5 years 5 months -1 years -1 months --- !query 15 +-- !query 14 select dateval, dateval - interval '99 11:22:33.123456789' day to second, @@ -203,13 +195,13 @@ select -interval '99 11:22:33.123456789' day to second + dateval, interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic --- !query 15 schema +-- !query 14 schema struct --- !query 15 output +-- !query 14 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 --- !query 16 +-- !query 15 select dateval, dateval - '99 11:22:33.123456789' day to second, @@ -219,13 +211,13 @@ select - '99 11:22:33.123456789' day to second + dateval, '99 11:22:33.123456789' day to second + dateval from interval_arithmetic --- !query 16 schema +-- !query 15 schema struct --- !query 16 output +-- !query 15 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 --- !query 17 +-- !query 16 select tsval, tsval - interval '99 11:22:33.123456789' day to second, @@ -235,13 +227,13 @@ select -interval '99 11:22:33.123456789' day to second + tsval, interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic --- !query 17 schema +-- !query 16 schema struct --- !query 17 output +-- !query 16 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 --- !query 18 +-- !query 17 select tsval, tsval - '99 11:22:33.123456789' day to second, @@ -251,47 +243,47 @@ select - '99 11:22:33.123456789' day to second + tsval, '99 11:22:33.123456789' day to second + tsval from interval_arithmetic --- !query 18 schema +-- !query 17 schema struct --- !query 18 output +-- !query 17 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 --- !query 19 +-- !query 18 select interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic --- !query 19 schema +-- !query 18 schema struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> --- !query 19 output +-- !query 18 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds --- !query 20 +-- !query 19 select '99 11:22:33.123456789' day to second + '10 9:8:7.123456789' day to second, '99 11:22:33.123456789' day to second - '10 9:8:7.123456789' day to second from interval_arithmetic --- !query 20 schema +-- !query 19 schema struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> --- !query 20 output +-- !query 19 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds --- !query 21 +-- !query 20 select 30 day --- !query 21 schema +-- !query 20 schema struct<30 days:interval> --- !query 21 output +-- !query 20 output 30 days --- !query 22 +-- !query 21 select 30 day day --- !query 22 schema +-- !query 21 schema struct<> --- !query 22 output +-- !query 21 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 14) @@ -301,11 +293,11 @@ select 30 day day --------------^^^ --- !query 23 +-- !query 22 select 30 day day day --- !query 23 schema +-- !query 22 schema struct<> --- !query 23 output +-- !query 22 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 14) @@ -315,19 +307,19 @@ select 30 day day day --------------^^^ --- !query 24 +-- !query 23 select date '2012-01-01' - 30 day --- !query 24 schema +-- !query 23 schema struct --- !query 24 output +-- !query 23 output 2011-12-02 --- !query 25 +-- !query 24 select date '2012-01-01' - 30 day day --- !query 25 schema +-- !query 24 schema struct<> --- !query 25 output +-- !query 24 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 34) @@ -337,11 +329,11 @@ select date '2012-01-01' - 30 day day ----------------------------------^^^ --- !query 26 +-- !query 25 select date '2012-01-01' - 30 day day day --- !query 26 schema +-- !query 25 schema struct<> --- !query 26 output +-- !query 25 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 34) @@ -351,27 +343,27 @@ select date '2012-01-01' - 30 day day day ----------------------------------^^^ --- !query 27 +-- !query 26 select date '2012-01-01' + '-30' day --- !query 27 schema +-- !query 26 schema struct --- !query 27 output +-- !query 26 output 2011-12-02 --- !query 28 +-- !query 27 select date '2012-01-01' + interval '-30' day --- !query 28 schema +-- !query 27 schema struct --- !query 28 output +-- !query 27 output 2011-12-02 --- !query 29 +-- !query 28 select date '2012-01-01' + interval (-30) day --- !query 29 schema +-- !query 28 schema struct<> --- !query 29 output +-- !query 28 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 42) @@ -381,11 +373,11 @@ select date '2012-01-01' + interval (-30) day ------------------------------------------^^^ --- !query 30 +-- !query 29 select date '2012-01-01' + (-30) day --- !query 30 schema +-- !query 29 schema struct<> --- !query 30 output +-- !query 29 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 33) @@ -395,19 +387,19 @@ select date '2012-01-01' + (-30) day ---------------------------------^^^ --- !query 31 +-- !query 30 create temporary view t as select * from values (1), (2) as t(a) --- !query 31 schema +-- !query 30 schema struct<> --- !query 31 output +-- !query 30 output --- !query 32 +-- !query 31 select date '2012-01-01' + interval (a + 1) day from t --- !query 32 schema +-- !query 31 schema struct<> --- !query 32 output +-- !query 31 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 44) @@ -417,11 +409,11 @@ select date '2012-01-01' + interval (a + 1) day from t --------------------------------------------^^^ --- !query 33 +-- !query 32 select date '2012-01-01' + (a + 1) day from t --- !query 33 schema +-- !query 32 schema struct<> --- !query 33 output +-- !query 32 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 35) @@ -431,9 +423,43 @@ select date '2012-01-01' + (a + 1) day from t -----------------------------------^^^ +-- !query 33 +select 1 year to month +-- !query 33 schema +struct<> +-- !query 33 output +org.apache.spark.sql.catalyst.parser.ParseException + +The value of from-to unit must be a string(line 1, pos 7) + +== SQL == +select 1 year to month +-------^^^ + + -- !query 34 -SET spark.sql.ansi.enabled=false +select '1' year to second -- !query 34 schema -struct +struct<> -- !query 34 output -spark.sql.ansi.enabled false +org.apache.spark.sql.catalyst.parser.ParseException + +Intervals FROM year TO second are not supported.(line 1, pos 7) + +== SQL == +select '1' year to second +-------^^^ + + +-- !query 35 +select 1 year '2-1' year to month +-- !query 35 schema +struct<> +-- !query 35 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 14) + +== SQL == +select 1 year '2-1' year to month +--------------^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out new file mode 100644 index 0000000000000..e43e88c8c72a8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -0,0 +1,474 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 48 + + +-- !query 0 +select null, Null, nUll +-- !query 0 schema +struct +-- !query 0 output +NULL NULL NULL + + +-- !query 1 +select true, tRue, false, fALse +-- !query 1 schema +struct +-- !query 1 output +true true false false + + +-- !query 2 +select 1Y +-- !query 2 schema +struct<1:tinyint> +-- !query 2 output +1 + + +-- !query 3 +select 127Y, -128Y +-- !query 3 schema +struct<127:tinyint,-128:tinyint> +-- !query 3 output +127 -128 + + +-- !query 4 +select 128Y +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7) + +== SQL == +select 128Y +-------^^^ + + +-- !query 5 +select 1S +-- !query 5 schema +struct<1:smallint> +-- !query 5 output +1 + + +-- !query 6 +select 32767S, -32768S +-- !query 6 schema +struct<32767:smallint,-32768:smallint> +-- !query 6 output +32767 -32768 + + +-- !query 7 +select 32768S +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7) + +== SQL == +select 32768S +-------^^^ + + +-- !query 8 +select 1L, 2147483648L +-- !query 8 schema +struct<1:bigint,2147483648:bigint> +-- !query 8 output +1 2147483648 + + +-- !query 9 +select 9223372036854775807L, -9223372036854775808L +-- !query 9 schema +struct<9223372036854775807:bigint,-9223372036854775808:bigint> +-- !query 9 output +9223372036854775807 -9223372036854775808 + + +-- !query 10 +select 9223372036854775808L +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7) + +== SQL == +select 9223372036854775808L +-------^^^ + + +-- !query 11 +select 1, -1 +-- !query 11 schema +struct<1:int,-1:int> +-- !query 11 output +1 -1 + + +-- !query 12 +select 2147483647, -2147483648 +-- !query 12 schema +struct<2147483647:int,-2147483648:int> +-- !query 12 output +2147483647 -2147483648 + + +-- !query 13 +select 9223372036854775807, -9223372036854775808 +-- !query 13 schema +struct<9223372036854775807:bigint,-9223372036854775808:bigint> +-- !query 13 output +9223372036854775807 -9223372036854775808 + + +-- !query 14 +select 9223372036854775808, -9223372036854775809 +-- !query 14 schema +struct<9223372036854775808:decimal(19,0),-9223372036854775809:decimal(19,0)> +-- !query 14 output +9223372036854775808 -9223372036854775809 + + +-- !query 15 +select 1234567890123456789012345678901234567890 +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.catalyst.parser.ParseException + +decimal can only support precision up to 38 +== SQL == +select 1234567890123456789012345678901234567890 + + +-- !query 16 +select 1234567890123456789012345678901234567890.0 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.catalyst.parser.ParseException + +decimal can only support precision up to 38 +== SQL == +select 1234567890123456789012345678901234567890.0 + + +-- !query 17 +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 +-- !query 17 schema +struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> +-- !query 17 output +1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90 90 + + +-- !query 18 +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 +-- !query 18 schema +struct<-1.0:double,-1.2:double,-1E+10:decimal(1,-10),-1.5E+5:decimal(2,-4),-0.1:double,-0.1:double,-1E+4:decimal(1,-4)> +-- !query 18 output +-1.0 -1.2 -10000000000 -150000 -0.1 -0.1 -10000 + + +-- !query 19 +select .e3 +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'select .'(line 1, pos 7) + +== SQL == +select .e3 +-------^^^ + + +-- !query 20 +select 1E309, -1E309 +-- !query 20 schema +struct<1E+309:decimal(1,-309),-1E+309:decimal(1,-309)> +-- !query 20 output +1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 -1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 + + +-- !query 21 +select 0.3, -0.8, .5, -.18, 0.1111, .1111 +-- !query 21 schema +struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> +-- !query 21 output +0.3 -0.8 0.5 -0.18 0.1111 0.1111 + + +-- !query 22 +select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d +-- !query 22 schema +struct<1.2345678901234568E48:double,1.2345678901234568E48:double> +-- !query 22 output +1.2345678901234568E48 1.2345678901234568E48 + + +-- !query 23 +select "Hello Peter!", 'hello lee!' +-- !query 23 schema +struct +-- !query 23 output +Hello Peter! hello lee! + + +-- !query 24 +select 'hello' 'world', 'hello' " " 'lee' +-- !query 24 schema +struct +-- !query 24 output +helloworld hello lee + + +-- !query 25 +select "hello 'peter'" +-- !query 25 schema +struct +-- !query 25 output +hello 'peter' + + +-- !query 26 +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' +-- !query 26 schema +struct +-- !query 26 output +pattern% no-pattern\% pattern\% pattern\\% + + +-- !query 27 +select '\'', '"', '\n', '\r', '\t', 'Z' +-- !query 27 schema +struct<':string,":string, +:string, :string, :string,Z:string> +-- !query 27 output +' " + Z + + +-- !query 28 +select '\110\145\154\154\157\041' +-- !query 28 schema +struct +-- !query 28 output +Hello! + + +-- !query 29 +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' +-- !query 29 schema +struct +-- !query 29 output +World :) + + +-- !query 30 +select dAte '2016-03-12' +-- !query 30 schema +struct +-- !query 30 output +2016-03-12 + + +-- !query 31 +select date 'mar 11 2016' +-- !query 31 schema +struct<> +-- !query 31 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: mar 11 2016(line 1, pos 7) + +== SQL == +select date 'mar 11 2016' +-------^^^ + + +-- !query 32 +select tImEstAmp '2016-03-11 20:54:00.000' +-- !query 32 schema +struct +-- !query 32 output +2016-03-11 20:54:00 + + +-- !query 33 +select timestamp '2016-33-11 20:54:00.000' +-- !query 33 schema +struct<> +-- !query 33 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the TIMESTAMP value: 2016-33-11 20:54:00.000(line 1, pos 7) + +== SQL == +select timestamp '2016-33-11 20:54:00.000' +-------^^^ + + +-- !query 34 +select GEO '(10,-6)' +-- !query 34 schema +struct<> +-- !query 34 output +org.apache.spark.sql.catalyst.parser.ParseException + +Literals of type 'GEO' are currently not supported.(line 1, pos 7) + +== SQL == +select GEO '(10,-6)' +-------^^^ + + +-- !query 35 +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD +-- !query 35 schema +struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> +-- !query 35 output +90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 + + +-- !query 36 +select 1.20E-38BD +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.catalyst.parser.ParseException + +decimal can only support precision up to 38(line 1, pos 7) + +== SQL == +select 1.20E-38BD +-------^^^ + + +-- !query 37 +select x'2379ACFe' +-- !query 37 schema +struct +-- !query 37 output +#y�� + + +-- !query 38 +select X'XuZ' +-- !query 38 schema +struct<> +-- !query 38 output +org.apache.spark.sql.catalyst.parser.ParseException + +contains illegal character for hexBinary: 0XuZ(line 1, pos 7) + +== SQL == +select X'XuZ' +-------^^^ + + +-- !query 39 +SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 +-- !query 39 schema +struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10),-3.14E+8:decimal(3,-6),-3.14E-8:decimal(10,10),3.14E+8:decimal(3,-6),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10)> +-- !query 39 output +3.14 -3.14 314000000 0.0000000314 -314000000 -0.0000000314 314000000 314000000 0.0000000314 + + +-- !query 40 +select integer '7' +-- !query 40 schema +struct<7:int> +-- !query 40 output +7 + + +-- !query 41 +select integer'7' +-- !query 41 schema +struct<7:int> +-- !query 41 output +7 + + +-- !query 42 +select integer '2147483648' +-- !query 42 schema +struct<> +-- !query 42 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the Int value: 2147483648, java.lang.NumberFormatException: For input string: "2147483648"(line 1, pos 7) + +== SQL == +select integer '2147483648' +-------^^^ + + +-- !query 43 +select -integer '7' +-- !query 43 schema +struct<-7:int> +-- !query 43 output +-7 + + +-- !query 44 +select -date '1999-01-01' +-- !query 44 schema +struct<> +-- !query 44 output +org.apache.spark.sql.catalyst.parser.ParseException + +Literals of type '-DATE' are currently not supported.(line 1, pos 7) + +== SQL == +select -date '1999-01-01' +-------^^^ + + +-- !query 45 +select -timestamp '1999-01-01' +-- !query 45 schema +struct<> +-- !query 45 output +org.apache.spark.sql.catalyst.parser.ParseException + +Literals of type '-TIMESTAMP' are currently not supported.(line 1, pos 7) + +== SQL == +select -timestamp '1999-01-01' +-------^^^ + + +-- !query 46 +select -x'2379ACFe' +-- !query 46 schema +struct<> +-- !query 46 output +org.apache.spark.sql.catalyst.parser.ParseException + +Literals of type '-X' are currently not supported.(line 1, pos 7) + +== SQL == +select -x'2379ACFe' +-------^^^ + + +-- !query 47 +select +integer '7' +-- !query 47 schema +struct<7:int> +-- !query 47 output +7 diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index 00e139d90f488..cbf44548b3cce 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 54 +-- Number of queries: 40 -- !query 0 @@ -328,131 +328,8 @@ NULL -- !query 39 -set spark.sql.ansi.enabled=true --- !query 39 schema -struct --- !query 39 output -spark.sql.ansi.enabled true - - --- !query 40 -select id, a*10, b/10 from decimals_test order by id --- !query 40 schema -struct --- !query 40 output -1 1000 99.9 -2 123451.23 1234.5123 -3 1.234567891011 123.41 -4 1234567891234567890 0.1123456789123456789 - - --- !query 41 -select 10.3 * 3.0 --- !query 41 schema -struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> --- !query 41 output -30.9 - - --- !query 42 -select 10.3000 * 3.0 --- !query 42 schema -struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> --- !query 42 output -30.9 - - --- !query 43 -select 10.30000 * 30.0 --- !query 43 schema -struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> --- !query 43 output -309 - - --- !query 44 -select 10.300000000000000000 * 3.000000000000000000 --- !query 44 schema -struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,36)> --- !query 44 output -30.9 - - --- !query 45 -select 10.300000000000000000 * 3.0000000000000000000 --- !query 45 schema -struct<> --- !query 45 output -java.lang.ArithmeticException -Decimal(expanded,30.900000000000000000000000000000000000,38,36}) cannot be represented as Decimal(38, 37). - - --- !query 46 -select (5e36 + 0.1) + 5e36 --- !query 46 schema -struct<> --- !query 46 output -java.lang.ArithmeticException -Decimal(expanded,10000000000000000000000000000000000000.1,39,1}) cannot be represented as Decimal(38, 1). - - --- !query 47 -select (-4e36 - 0.1) - 7e36 --- !query 47 schema -struct<> --- !query 47 output -java.lang.ArithmeticException -Decimal(expanded,-11000000000000000000000000000000000000.1,39,1}) cannot be represented as Decimal(38, 1). - - --- !query 48 -select 12345678901234567890.0 * 12345678901234567890.0 --- !query 48 schema -struct<> --- !query 48 output -java.lang.ArithmeticException -Decimal(expanded,1.5241578753238836750190519987501905210E+38,38,-1}) cannot be represented as Decimal(38, 2). - - --- !query 49 -select 1e35 / 0.1 --- !query 49 schema -struct<> --- !query 49 output -java.lang.ArithmeticException -Decimal(expanded,1000000000000000000000000000000000000,37,0}) cannot be represented as Decimal(38, 3). - - --- !query 50 -select 123456789123456789.1234567890 * 1.123456789123456789 --- !query 50 schema -struct<> --- !query 50 output -java.lang.ArithmeticException -Decimal(expanded,138698367904130467.65432098851562262075,38,20}) cannot be represented as Decimal(38, 28). - - --- !query 51 -select 123456789123456789.1234567890 * 1.123456789123456789 --- !query 51 schema -struct<> --- !query 51 output -java.lang.ArithmeticException -Decimal(expanded,138698367904130467.65432098851562262075,38,20}) cannot be represented as Decimal(38, 28). - - --- !query 52 -select 12345678912345.123456789123 / 0.000000012345678 --- !query 52 schema -struct<> --- !query 52 output -java.lang.ArithmeticException -Decimal(expanded,1000000073899961059796.7258663315210392,38,16}) cannot be represented as Decimal(38, 18). - - --- !query 53 drop table decimals_test --- !query 53 schema +-- !query 39 schema struct<> --- !query 53 output +-- !query 39 output diff --git a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out index 0b78076588c17..ca1d747874c5d 100644 --- a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 33 +-- Number of queries: 29 -- !query 0 @@ -270,47 +270,3 @@ select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t( struct> -- !query 28 output [32,98] - - --- !query 29 -set spark.sql.ansi.enabled=true --- !query 29 schema -struct --- !query 29 output -spark.sql.ansi.enabled true - - --- !query 30 -select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) --- !query 30 schema -struct<> --- !query 30 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input 'all'(line 1, pos 21) - -== SQL == -select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) ----------------------^^^ - - --- !query 31 -select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) --- !query 31 schema -struct<> --- !query 31 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input 'all'(line 1, pos 22) - -== SQL == -select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) -----------------------^^^ - - --- !query 32 -set spark.sql.ansi.enabled=false --- !query 32 schema -struct --- !query 32 output -spark.sql.ansi.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 372f7d9e0b96d..91dd72dfa492b 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 57 +-- Number of queries: 88 -- !query 0 @@ -458,3 +458,323 @@ select justify_interval(interval '1 month 59 day -25 hour') struct -- !query 56 output 2 months 27 days 23 hours + + +-- !query 57 +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query 57 schema +struct<13.123456 seconds:interval,-13.123456 seconds:interval> +-- !query 57 output +13.123456 seconds -13.123456 seconds + + +-- !query 58 +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond +-- !query 58 schema +struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> +-- !query 58 output +1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds + + +-- !query 59 +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second +-- !query 59 schema +struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> +-- !query 59 output +32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds + + +-- !query 60 +select interval '0 0:0:0.1' day to second +-- !query 60 schema +struct<0.1 seconds:interval> +-- !query 60 output +0.1 seconds + + +-- !query 61 +select interval '10-9' year to month +-- !query 61 schema +struct<10 years 9 months:interval> +-- !query 61 output +10 years 9 months + + +-- !query 62 +select interval '20 15:40:32.99899999' day to hour +-- !query 62 schema +struct<20 days 15 hours:interval> +-- !query 62 output +20 days 15 hours + + +-- !query 63 +select interval '20 15:40:32.99899999' day to minute +-- !query 63 schema +struct<20 days 15 hours 40 minutes:interval> +-- !query 63 output +20 days 15 hours 40 minutes + + +-- !query 64 +select interval '20 15:40:32.99899999' day to second +-- !query 64 schema +struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +-- !query 64 output +20 days 15 hours 40 minutes 32.998999 seconds + + +-- !query 65 +select interval '15:40:32.99899999' hour to minute +-- !query 65 schema +struct<15 hours 40 minutes:interval> +-- !query 65 output +15 hours 40 minutes + + +-- !query 66 +select interval '15:40.99899999' hour to second +-- !query 66 schema +struct<15 minutes 40.998999 seconds:interval> +-- !query 66 output +15 minutes 40.998999 seconds + + +-- !query 67 +select interval '15:40' hour to second +-- !query 67 schema +struct<15 hours 40 minutes:interval> +-- !query 67 output +15 hours 40 minutes + + +-- !query 68 +select interval '15:40:32.99899999' hour to second +-- !query 68 schema +struct<15 hours 40 minutes 32.998999 seconds:interval> +-- !query 68 output +15 hours 40 minutes 32.998999 seconds + + +-- !query 69 +select interval '20 40:32.99899999' minute to second +-- !query 69 schema +struct<20 days 40 minutes 32.998999 seconds:interval> +-- !query 69 output +20 days 40 minutes 32.998999 seconds + + +-- !query 70 +select interval '40:32.99899999' minute to second +-- !query 70 schema +struct<40 minutes 32.998999 seconds:interval> +-- !query 70 output +40 minutes 32.998999 seconds + + +-- !query 71 +select interval '40:32' minute to second +-- !query 71 schema +struct<40 minutes 32 seconds:interval> +-- !query 71 output +40 minutes 32 seconds + + +-- !query 72 +select interval 10 nanoseconds +-- !query 72 schema +struct<> +-- !query 72 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input '10 nanoseconds'(line 1, pos 19) + +== SQL == +select interval 10 nanoseconds +-------------------^^^ + + +-- !query 73 +select map(1, interval 1 day, 2, interval 3 week) +-- !query 73 schema +struct> +-- !query 73 output +{1:1 days,2:21 days} + + +-- !query 74 +select interval 'interval 3 year 1 hour' +-- !query 74 schema +struct<3 years 1 hours:interval> +-- !query 74 output +3 years 1 hours + + +-- !query 75 +select interval '3 year 1 hour' +-- !query 75 schema +struct<3 years 1 hours:interval> +-- !query 75 output +3 years 1 hours + + +-- !query 76 +select interval +-- !query 76 schema +struct<> +-- !query 76 output +org.apache.spark.sql.catalyst.parser.ParseException + +at least one time unit should be given for interval literal(line 1, pos 7) + +== SQL == +select interval +-------^^^ + + +-- !query 77 +select interval 1 fake_unit +-- !query 77 schema +struct<> +-- !query 77 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input '1 fake_unit'(line 1, pos 18) + +== SQL == +select interval 1 fake_unit +------------------^^^ + + +-- !query 78 +select interval 1 year to month +-- !query 78 schema +struct<> +-- !query 78 output +org.apache.spark.sql.catalyst.parser.ParseException + +The value of from-to unit must be a string(line 1, pos 16) + +== SQL == +select interval 1 year to month +----------------^^^ + + +-- !query 79 +select interval '1' year to second +-- !query 79 schema +struct<> +-- !query 79 output +org.apache.spark.sql.catalyst.parser.ParseException + +Intervals FROM year TO second are not supported.(line 1, pos 16) + +== SQL == +select interval '1' year to second +----------------^^^ + + +-- !query 80 +select interval '10-9' year to month '2-1' year to month +-- !query 80 schema +struct<> +-- !query 80 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '2-1' year to month +-------------------------------------^^^ + + +-- !query 81 +select interval '10-9' year to month '12:11:10' hour to second +-- !query 81 schema +struct<> +-- !query 81 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '12:11:10' hour to second +-------------------------------------^^^ + + +-- !query 82 +select interval '1 15:11' day to minute '12:11:10' hour to second +-- !query 82 schema +struct<> +-- !query 82 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) + +== SQL == +select interval '1 15:11' day to minute '12:11:10' hour to second +----------------------------------------^^^ + + +-- !query 83 +select interval 1 year '2-1' year to month +-- !query 83 schema +struct<> +-- !query 83 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) + +== SQL == +select interval 1 year '2-1' year to month +-----------------------^^^ + + +-- !query 84 +select interval 1 year '12:11:10' hour to second +-- !query 84 schema +struct<> +-- !query 84 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) + +== SQL == +select interval 1 year '12:11:10' hour to second +-----------------------^^^ + + +-- !query 85 +select interval '10-9' year to month '1' year +-- !query 85 schema +struct<> +-- !query 85 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '1' year +-------------------------------------^^^ + + +-- !query 86 +select interval '12:11:10' hour to second '1' year +-- !query 86 schema +struct<> +-- !query 86 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) + +== SQL == +select interval '12:11:10' hour to second '1' year +------------------------------------------^^^ + + +-- !query 87 +select +interval '1 second' +-- !query 87 schema +struct<1 seconds:interval> +-- !query 87 output +1 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 8f641e51d8234..e43e88c8c72a8 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 89 +-- Number of queries: 48 -- !query 0 @@ -321,144 +321,10 @@ select timestamp '2016-33-11 20:54:00.000' -- !query 34 -select interval 13.123456789 seconds, interval -13.123456789 second --- !query 34 schema -struct<13.123456 seconds:interval,-13.123456 seconds:interval> --- !query 34 output -13.123456 seconds -13.123456 seconds - - --- !query 35 -select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond --- !query 35 schema -struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> --- !query 35 output -1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds - - --- !query 36 -select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second --- !query 36 schema -struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> --- !query 36 output -32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds - - --- !query 37 -select interval '0 0:0:0.1' day to second --- !query 37 schema -struct<0.1 seconds:interval> --- !query 37 output -0.1 seconds - - --- !query 38 -select interval '10-9' year to month --- !query 38 schema -struct<10 years 9 months:interval> --- !query 38 output -10 years 9 months - - --- !query 39 -select interval '20 15:40:32.99899999' day to hour --- !query 39 schema -struct<20 days 15 hours:interval> --- !query 39 output -20 days 15 hours - - --- !query 40 -select interval '20 15:40:32.99899999' day to minute --- !query 40 schema -struct<20 days 15 hours 40 minutes:interval> --- !query 40 output -20 days 15 hours 40 minutes - - --- !query 41 -select interval '20 15:40:32.99899999' day to second --- !query 41 schema -struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> --- !query 41 output -20 days 15 hours 40 minutes 32.998999 seconds - - --- !query 42 -select interval '15:40:32.99899999' hour to minute --- !query 42 schema -struct<15 hours 40 minutes:interval> --- !query 42 output -15 hours 40 minutes - - --- !query 43 -select interval '15:40.99899999' hour to second --- !query 43 schema -struct<15 minutes 40.998999 seconds:interval> --- !query 43 output -15 minutes 40.998999 seconds - - --- !query 44 -select interval '15:40' hour to second --- !query 44 schema -struct<15 hours 40 minutes:interval> --- !query 44 output -15 hours 40 minutes - - --- !query 45 -select interval '15:40:32.99899999' hour to second --- !query 45 schema -struct<15 hours 40 minutes 32.998999 seconds:interval> --- !query 45 output -15 hours 40 minutes 32.998999 seconds - - --- !query 46 -select interval '20 40:32.99899999' minute to second --- !query 46 schema -struct<20 days 40 minutes 32.998999 seconds:interval> --- !query 46 output -20 days 40 minutes 32.998999 seconds - - --- !query 47 -select interval '40:32.99899999' minute to second --- !query 47 schema -struct<40 minutes 32.998999 seconds:interval> --- !query 47 output -40 minutes 32.998999 seconds - - --- !query 48 -select interval '40:32' minute to second --- !query 48 schema -struct<40 minutes 32 seconds:interval> --- !query 48 output -40 minutes 32 seconds - - --- !query 49 -select interval 10 nanoseconds --- !query 49 schema -struct<> --- !query 49 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input '10 nanoseconds'(line 1, pos 19) - -== SQL == -select interval 10 nanoseconds --------------------^^^ - - --- !query 50 select GEO '(10,-6)' --- !query 50 schema +-- !query 34 schema struct<> --- !query 50 output +-- !query 34 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'GEO' are currently not supported.(line 1, pos 7) @@ -468,19 +334,19 @@ select GEO '(10,-6)' -------^^^ --- !query 51 +-- !query 35 select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD --- !query 51 schema +-- !query 35 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> --- !query 51 output +-- !query 35 output 90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 --- !query 52 +-- !query 36 select 1.20E-38BD --- !query 52 schema +-- !query 36 schema struct<> --- !query 52 output +-- !query 36 output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38(line 1, pos 7) @@ -490,19 +356,19 @@ select 1.20E-38BD -------^^^ --- !query 53 +-- !query 37 select x'2379ACFe' --- !query 53 schema +-- !query 37 schema struct --- !query 53 output +-- !query 37 output #y�� --- !query 54 +-- !query 38 select X'XuZ' --- !query 54 schema +-- !query 38 schema struct<> --- !query 54 output +-- !query 38 output org.apache.spark.sql.catalyst.parser.ParseException contains illegal character for hexBinary: 0XuZ(line 1, pos 7) @@ -512,59 +378,35 @@ select X'XuZ' -------^^^ --- !query 55 +-- !query 39 SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 --- !query 55 schema +-- !query 39 schema struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10),-3.14E+8:decimal(3,-6),-3.14E-8:decimal(10,10),3.14E+8:decimal(3,-6),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10)> --- !query 55 output +-- !query 39 output 3.14 -3.14 314000000 0.0000000314 -314000000 -0.0000000314 314000000 314000000 0.0000000314 --- !query 56 -select map(1, interval 1 day, 2, interval 3 week) --- !query 56 schema -struct> --- !query 56 output -{1:1 days,2:21 days} - - --- !query 57 -select interval 'interval 3 year 1 hour' --- !query 57 schema -struct<3 years 1 hours:interval> --- !query 57 output -3 years 1 hours - - --- !query 58 -select interval '3 year 1 hour' --- !query 58 schema -struct<3 years 1 hours:interval> --- !query 58 output -3 years 1 hours - - --- !query 59 +-- !query 40 select integer '7' --- !query 59 schema +-- !query 40 schema struct<7:int> --- !query 59 output +-- !query 40 output 7 --- !query 60 +-- !query 41 select integer'7' --- !query 60 schema +-- !query 41 schema struct<7:int> --- !query 60 output +-- !query 41 output 7 --- !query 61 +-- !query 42 select integer '2147483648' --- !query 61 schema +-- !query 42 schema struct<> --- !query 61 output +-- !query 42 output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the Int value: 2147483648, java.lang.NumberFormatException: For input string: "2147483648"(line 1, pos 7) @@ -574,301 +416,19 @@ select integer '2147483648' -------^^^ --- !query 62 -select interval --- !query 62 schema -struct<> --- !query 62 output -org.apache.spark.sql.catalyst.parser.ParseException - -at least one time unit should be given for interval literal(line 1, pos 7) - -== SQL == -select interval --------^^^ - - --- !query 63 -select interval 1 fake_unit --- !query 63 schema -struct<> --- !query 63 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input '1 fake_unit'(line 1, pos 18) - -== SQL == -select interval 1 fake_unit -------------------^^^ - - --- !query 64 -select interval 1 year to month --- !query 64 schema -struct<> --- !query 64 output -org.apache.spark.sql.catalyst.parser.ParseException - -The value of from-to unit must be a string(line 1, pos 16) - -== SQL == -select interval 1 year to month -----------------^^^ - - --- !query 65 -select interval '1' year to second --- !query 65 schema -struct<> --- !query 65 output -org.apache.spark.sql.catalyst.parser.ParseException - -Intervals FROM year TO second are not supported.(line 1, pos 16) - -== SQL == -select interval '1' year to second -----------------^^^ - - --- !query 66 -select interval '10-9' year to month '2-1' year to month --- !query 66 schema -struct<> --- !query 66 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '2-1' year to month --------------------------------------^^^ - - --- !query 67 -select interval '10-9' year to month '12:11:10' hour to second --- !query 67 schema -struct<> --- !query 67 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '12:11:10' hour to second --------------------------------------^^^ - - --- !query 68 -select interval '1 15:11' day to minute '12:11:10' hour to second --- !query 68 schema -struct<> --- !query 68 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) - -== SQL == -select interval '1 15:11' day to minute '12:11:10' hour to second -----------------------------------------^^^ - - --- !query 69 -select interval 1 year '2-1' year to month --- !query 69 schema -struct<> --- !query 69 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) - -== SQL == -select interval 1 year '2-1' year to month ------------------------^^^ - - --- !query 70 -select interval 1 year '12:11:10' hour to second --- !query 70 schema -struct<> --- !query 70 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) - -== SQL == -select interval 1 year '12:11:10' hour to second ------------------------^^^ - - --- !query 71 -select interval '10-9' year to month '1' year --- !query 71 schema -struct<> --- !query 71 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) - -== SQL == -select interval '10-9' year to month '1' year --------------------------------------^^^ - - --- !query 72 -select interval '12:11:10' hour to second '1' year --- !query 72 schema -struct<> --- !query 72 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) - -== SQL == -select interval '12:11:10' hour to second '1' year -------------------------------------------^^^ - - --- !query 73 -SET spark.sql.ansi.enabled=true --- !query 73 schema -struct --- !query 73 output -spark.sql.ansi.enabled true - - --- !query 74 -select interval --- !query 74 schema -struct<> --- !query 74 output -org.apache.spark.sql.catalyst.parser.ParseException - -at least one time unit should be given for interval literal(line 1, pos 7) - -== SQL == -select interval --------^^^ - - --- !query 75 -select interval 1 fake_unit --- !query 75 schema -struct<> --- !query 75 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input '1 fake_unit'(line 1, pos 18) - -== SQL == -select interval 1 fake_unit -------------------^^^ - - --- !query 76 -select interval 1 year to month --- !query 76 schema -struct<> --- !query 76 output -org.apache.spark.sql.catalyst.parser.ParseException - -The value of from-to unit must be a string(line 1, pos 16) - -== SQL == -select interval 1 year to month -----------------^^^ - - --- !query 77 -select 1 year to month --- !query 77 schema -struct<> --- !query 77 output -org.apache.spark.sql.catalyst.parser.ParseException - -The value of from-to unit must be a string(line 1, pos 7) - -== SQL == -select 1 year to month --------^^^ - - --- !query 78 -select interval '1' year to second --- !query 78 schema -struct<> --- !query 78 output -org.apache.spark.sql.catalyst.parser.ParseException - -Intervals FROM year TO second are not supported.(line 1, pos 16) - -== SQL == -select interval '1' year to second -----------------^^^ - - --- !query 79 -select '1' year to second --- !query 79 schema -struct<> --- !query 79 output -org.apache.spark.sql.catalyst.parser.ParseException - -Intervals FROM year TO second are not supported.(line 1, pos 7) - -== SQL == -select '1' year to second --------^^^ - - --- !query 80 -select interval 1 year '2-1' year to month --- !query 80 schema -struct<> --- !query 80 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) - -== SQL == -select interval 1 year '2-1' year to month ------------------------^^^ - - --- !query 81 -select 1 year '2-1' year to month --- !query 81 schema -struct<> --- !query 81 output -org.apache.spark.sql.catalyst.parser.ParseException - -Can only have a single from-to unit in the interval literal syntax(line 1, pos 14) - -== SQL == -select 1 year '2-1' year to month ---------------^^^ - - --- !query 82 -SET spark.sql.ansi.enabled=false --- !query 82 schema -struct --- !query 82 output -spark.sql.ansi.enabled false - - --- !query 83 +-- !query 43 select -integer '7' --- !query 83 schema +-- !query 43 schema struct<-7:int> --- !query 83 output +-- !query 43 output -7 --- !query 84 +-- !query 44 select -date '1999-01-01' --- !query 84 schema +-- !query 44 schema struct<> --- !query 84 output +-- !query 44 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type '-DATE' are currently not supported.(line 1, pos 7) @@ -878,11 +438,11 @@ select -date '1999-01-01' -------^^^ --- !query 85 +-- !query 45 select -timestamp '1999-01-01' --- !query 85 schema +-- !query 45 schema struct<> --- !query 85 output +-- !query 45 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type '-TIMESTAMP' are currently not supported.(line 1, pos 7) @@ -892,11 +452,11 @@ select -timestamp '1999-01-01' -------^^^ --- !query 86 +-- !query 46 select -x'2379ACFe' --- !query 86 schema +-- !query 46 schema struct<> --- !query 86 output +-- !query 46 output org.apache.spark.sql.catalyst.parser.ParseException Literals of type '-X' are currently not supported.(line 1, pos 7) @@ -906,17 +466,9 @@ select -x'2379ACFe' -------^^^ --- !query 87 +-- !query 47 select +integer '7' --- !query 87 schema +-- !query 47 schema struct<7:int> --- !query 87 output +-- !query 47 output 7 - - --- !query 88 -select +interval '1 second' --- !query 88 schema -struct<1 seconds:interval> --- !query 88 output -1 seconds diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 58469fca8f3fe..2ce50dd73a13d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -65,6 +65,7 @@ import org.apache.spark.tags.ExtendedSQLTest * 1. A list of SQL queries separated by semicolon. * 2. Lines starting with -- are treated as comments and ignored. * 3. Lines starting with --SET are used to run the file with the following set of configs. + * 4. Lines starting with --import are used to load queries from another test file. * * For example: * {{{ @@ -158,7 +159,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { ) // Create all the test cases. - listTestCases().foreach(createScalaTestCase) + listTestCases.foreach(createScalaTestCase) /** A single SQL query's output. */ protected case class QueryOutput(sql: String, schema: String, output: String) { @@ -186,6 +187,11 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { */ protected trait PgSQLTest + /** + * traits that indicate ANSI-related tests with the ANSI mode enabled. + */ + protected trait AnsiTest + protected trait UDFTest { val udf: TestUDF } @@ -212,6 +218,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { resultFile: String, udf: TestUDF) extends TestCase with UDFTest with PgSQLTest + /** An ANSI-related test case. */ + protected case class AnsiTestCase( + name: String, inputFile: String, resultFile: String) extends TestCase with AnsiTest + protected def createScalaTestCase(testCase: TestCase): Unit = { if (blackList.exists(t => testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { @@ -255,9 +265,21 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { val (comments, code) = input.split("\n").partition(_.trim.startsWith("--")) + // If `--import` found, load code from another test case file, then insert them + // into the head in this test. + val importedTestCaseName = comments.filter(_.startsWith("--import ")).map(_.substring(9)) + val importedCode = importedTestCaseName.flatMap { testCaseName => + listTestCases.find(_.name == testCaseName).map { testCase => + val input = fileToString(new File(testCase.inputFile)) + val (_, code) = input.split("\n").partition(_.trim.startsWith("--")) + code + } + }.flatten + // List of SQL queries to run // note: this is not a robust way to split queries using semicolon, but works for now. - val queries = code.mkString("\n").split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq + val queries = (importedCode ++ code).mkString("\n").split("(?<=[^\\\\]);") + .map(_.trim).filter(_ != "").toSeq // Fix misplacement when comment is at the end of the query. .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")).map(_.trim).filter(_ != "") @@ -321,10 +343,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - // PostgreSQL enabled cartesian product by default. - localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) + case _: AnsiTest => + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } @@ -457,7 +479,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { line.replaceAll("#\\d+", "#x") .replaceAll( s"Location.*$clsName/", - s"Location ${notIncludedMsg}/{warehouse_dir}/") + s"Location $notIncludedMsg/{warehouse_dir}/") .replaceAll("Created By.*", s"Created By $notIncludedMsg") .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") @@ -465,7 +487,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds } - protected def listTestCases(): Seq[TestCase] = { + protected lazy val listTestCases: Seq[TestCase] = { listFilesRecursively(new File(inputFilePath)).flatMap { file => val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" val absPath = file.getAbsolutePath @@ -484,6 +506,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { } } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) { PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}ansi")) { + AnsiTestCase(testCaseName, absPath, resultFile) :: Nil } else { RegularTestCase(testCaseName, absPath, resultFile) :: Nil } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 04b1de00ccbff..67193e8de16f4 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -27,12 +27,11 @@ import scala.util.control.NonFatal import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.SparkException import org.apache.spark.sql.SQLQueryTestSuite import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.util.fileToString import org.apache.spark.sql.execution.HiveResult -import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -108,10 +107,10 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { testCase match { case _: PgSQLTest => - // PostgreSQL enabled cartesian product by default. - statement.execute(s"SET ${SQLConf.CROSS_JOINS_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") + case _: AnsiTest => + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") case _ => } @@ -231,7 +230,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } - override def listTestCases(): Seq[TestCase] = { + override lazy val listTestCases: Seq[TestCase] = { listFilesRecursively(new File(inputFilePath)).flatMap { file => val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" val absPath = file.getAbsolutePath @@ -241,6 +240,8 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { Seq.empty } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) { PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}ansi")) { + AnsiTestCase(testCaseName, absPath, resultFile) :: Nil } else { RegularTestCase(testCaseName, absPath, resultFile) :: Nil } From e46e487b0831b39afa12ef9cff9b9133f111921b Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 14 Nov 2019 14:47:14 +0800 Subject: [PATCH 0612/1387] [SPARK-29682][SQL] Resolve conflicting attributes in Expand correctly ### What changes were proposed in this pull request? This PR addresses issues where conflicting attributes in `Expand` are not correctly handled. ### Why are the changes needed? ```Scala val numsDF = Seq(1, 2, 3, 4, 5, 6).toDF("nums") val cubeDF = numsDF.cube("nums").agg(max(lit(0)).as("agcol")) cubeDF.join(cubeDF, "nums").show ``` fails with the following exception: ``` org.apache.spark.sql.AnalysisException: Failure when resolving conflicting references in Join: 'Join Inner :- Aggregate [nums#38, spark_grouping_id#36], [nums#38, max(0) AS agcol#35] : +- Expand [List(nums#3, nums#37, 0), List(nums#3, null, 1)], [nums#3, nums#38, spark_grouping_id#36] : +- Project [nums#3, nums#3 AS nums#37] : +- Project [value#1 AS nums#3] : +- LocalRelation [value#1] +- Aggregate [nums#38, spark_grouping_id#36], [nums#38, max(0) AS agcol#58] +- Expand [List(nums#3, nums#37, 0), List(nums#3, null, 1)], [nums#3, nums#38, spark_grouping_id#36] ^^^^^^^ +- Project [nums#3, nums#3 AS nums#37] +- Project [value#1 AS nums#3] +- LocalRelation [value#1] Conflicting attributes: nums#38 ``` As you can see from the above plan, `num#38`, the output of `Expand` on the right side of `Join`, should have been handled to produce new attribute. Since the conflict is not resolved in `Expand`, the failure is happening upstream at `Aggregate`. This PR addresses handling conflicting attributes in `Expand`. ### Does this PR introduce any user-facing change? Yes, the previous example now shows the following output: ``` +----+-----+-----+ |nums|agcol|agcol| +----+-----+-----+ | 1| 0| 0| | 6| 0| 0| | 4| 0| 0| | 2| 0| 0| | 5| 0| 0| | 3| 0| 0| +----+-----+-----+ ``` ### How was this patch tested? Added new unit test. Closes #26441 from imback82/spark-29682. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/analysis/Analyzer.scala | 12 ++++++++++++ .../plans/logical/basicLogicalOperators.scala | 2 ++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 +++++++++ 3 files changed, 23 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index ead122ed9c56b..a7443e71c0ca3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -972,6 +972,18 @@ class Analyzer( val newOutput = oldVersion.generatorOutput.map(_.newInstance()) (oldVersion, oldVersion.copy(generatorOutput = newOutput)) + case oldVersion: Expand + if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => + val producedAttributes = oldVersion.producedAttributes + val newOutput = oldVersion.output.map { attr => + if (producedAttributes.contains(attr)) { + attr.newInstance() + } else { + attr + } + } + (oldVersion, oldVersion.copy(output = newOutput)) + case oldVersion @ Window(windowExpressions, _, _, child) if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) .nonEmpty => 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 d58c607cb87ee..75aafefa86636 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 @@ -672,6 +672,8 @@ case class Expand( override lazy val references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) + override def producedAttributes: AttributeSet = AttributeSet(output diff child.output) + // This operator can reuse attributes (for example making them null when doing a roll up) so // the constraints of the child may no longer be valid. override protected lazy val validConstraints: Set[Expression] = Set.empty[Expression] 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 0dd2e20e4643d..5b0de1baa553b 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 @@ -3304,6 +3304,15 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { """.stripMargin).collect() } } + + test("SPARK-29682: Conflicting attributes in Expand are resolved") { + val numsDF = Seq(1, 2, 3).toDF("nums") + val cubeDF = numsDF.cube("nums").agg(max(lit(0)).as("agcol")) + + checkAnswer( + cubeDF.join(cubeDF, "nums"), + Row(1, 0, 0) :: Row(2, 0, 0) :: Row(3, 0, 0) :: Nil) + } } case class Foo(bar: Option[String]) From 04e99c1e1b29b691a8fb51ecfcd7e99482ee0bb3 Mon Sep 17 00:00:00 2001 From: shane knapp Date: Thu, 14 Nov 2019 10:18:55 -0800 Subject: [PATCH 0613/1387] [SPARK-29672][PYSPARK] update spark testing framework to use python3 ### What changes were proposed in this pull request? remove python2.7 tests and test infra for 3.0+ ### Why are the changes needed? because python2.7 is finally going the way of the dodo. ### Does this PR introduce any user-facing change? newp. ### How was this patch tested? the build system will test this Closes #26330 from shaneknapp/remove-py27-tests. Lead-authored-by: shane knapp Co-authored-by: shane Signed-off-by: shane knapp --- dev/pip-sanity-check.py | 2 -- dev/run-pip-tests | 23 +++++++++-------------- dev/run-tests | 6 +++--- dev/run-tests-jenkins | 8 +++++--- dev/run-tests-jenkins.py | 3 +-- dev/run-tests.py | 5 ++--- dev/sparktestsupport/shellutils.py | 6 ++---- python/pyspark/context.py | 2 -- python/pyspark/version.py | 2 +- python/run-tests | 8 +++++++- python/run-tests.py | 17 ++++++++++------- python/setup.py | 7 +++---- 12 files changed, 43 insertions(+), 46 deletions(-) mode change 100644 => 100755 python/setup.py diff --git a/dev/pip-sanity-check.py b/dev/pip-sanity-check.py index 4171f28684d59..e9f10233b12b7 100644 --- a/dev/pip-sanity-check.py +++ b/dev/pip-sanity-check.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - from pyspark.sql import SparkSession from pyspark.mllib.linalg import * import sys diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 60cf4d8209416..1294a9096fb95 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -39,21 +39,16 @@ PYTHON_EXECS=() # Some systems don't have pip or virtualenv - in those cases our tests won't work. if hash virtualenv 2>/dev/null && [ ! -n "$USE_CONDA" ]; then echo "virtualenv installed - using. Note if this is a conda virtual env you may wish to set USE_CONDA" - # Figure out which Python execs we should test pip installation with - if hash python2 2>/dev/null; then - # We do this since we are testing with virtualenv and the default virtual env python - # is in /usr/bin/python - PYTHON_EXECS+=('python2') - elif hash python 2>/dev/null; then - # If python2 isn't installed fallback to python if available - PYTHON_EXECS+=('python') - fi + # test only against python3 if hash python3 2>/dev/null; then - PYTHON_EXECS+=('python3') + PYTHON_EXECS=('python3') + else + echo "Python3 not installed on system, skipping pip installability tests" + exit 0 fi elif hash conda 2>/dev/null; then echo "Using conda virtual environments" - PYTHON_EXECS=('3.5') + PYTHON_EXECS=('3.6') USE_CONDA=1 else echo "Missing virtualenv & conda, skipping pip installability tests" @@ -97,7 +92,7 @@ for python in "${PYTHON_EXECS[@]}"; do cd "$FWDIR"/python # Delete the egg info file if it exists, this can cache the setup file. rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" - python setup.py sdist + python3 setup.py sdist echo "Installing dist into virtual env" @@ -117,9 +112,9 @@ for python in "${PYTHON_EXECS[@]}"; do echo "Run basic sanity check on pip installed version with spark-submit" spark-submit "$FWDIR"/dev/pip-sanity-check.py echo "Run basic sanity check with import based" - python "$FWDIR"/dev/pip-sanity-check.py + python3 "$FWDIR"/dev/pip-sanity-check.py echo "Run the tests for context.py" - python "$FWDIR"/python/pyspark/context.py + python3 "$FWDIR"/python/pyspark/context.py cd "$FWDIR" diff --git a/dev/run-tests b/dev/run-tests index 9cf93d000d0ea..143d78ec63731 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -20,10 +20,10 @@ FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" -PYTHON_VERSION_CHECK=$(python -c 'import sys; print(sys.version_info < (2, 7, 0))') +PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 6, 0))') if [[ "$PYTHON_VERSION_CHECK" == "True" ]]; then - echo "Python versions prior to 2.7 are not supported." + echo "Python versions prior to 3.6 are not supported." exit -1 fi -exec python -u ./dev/run-tests.py "$@" +exec python3 -u ./dev/run-tests.py "$@" diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 5bc03e41d1f2d..c3adc696a5122 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -25,10 +25,12 @@ FWDIR="$( cd "$( dirname "$0" )/.." && pwd )" cd "$FWDIR" -PYTHON_VERSION_CHECK=$(python -c 'import sys; print(sys.version_info < (2, 7, 0))') +export PATH=/home/anaconda/envs/py36/bin:$PATH + +PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 6, 0))') if [[ "$PYTHON_VERSION_CHECK" == "True" ]]; then - echo "Python versions prior to 2.7 are not supported." + echo "Python versions prior to 3.6 are not supported." exit -1 fi -exec python -u ./dev/run-tests-jenkins.py "$@" +exec python3 -u ./dev/run-tests-jenkins.py "$@" diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index e9b0b327603be..5429aeba8ea13 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -17,7 +17,6 @@ # limitations under the License. # -from __future__ import print_function import os import sys import json diff --git a/dev/run-tests.py b/dev/run-tests.py index ea515708124db..82277720bb52f 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -17,7 +17,6 @@ # limitations under the License. # -from __future__ import print_function import itertools from argparse import ArgumentParser import os @@ -265,7 +264,7 @@ def exec_sbt(sbt_args=()): echo_proc.wait() for line in iter(sbt_proc.stdout.readline, b''): if not sbt_output_filter.match(line): - print(line, end='') + print(line.decode('utf-8'), end='') retcode = sbt_proc.wait() if retcode != 0: diff --git a/dev/sparktestsupport/shellutils.py b/dev/sparktestsupport/shellutils.py index ec6ea86269f5e..d9cb8aa45c8d2 100644 --- a/dev/sparktestsupport/shellutils.py +++ b/dev/sparktestsupport/shellutils.py @@ -15,14 +15,12 @@ # limitations under the License. # -from __future__ import print_function import os import shutil import subprocess import sys subprocess_check_output = subprocess.check_output -subprocess_check_call = subprocess.check_call def exit_from_command_with_retcode(cmd, retcode): @@ -55,9 +53,9 @@ def run_cmd(cmd, return_output=False): cmd = cmd.split() try: if return_output: - return subprocess_check_output(cmd).decode(sys.getdefaultencoding()) + return subprocess_check_output(cmd).decode('utf-8') else: - return subprocess_check_call(cmd) + return subprocess.run(cmd, universal_newlines=True, check=True) except subprocess.CalledProcessError as e: exit_from_command_with_retcode(e.cmd, e.returncode) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index e7e7bcd95a065..a6aa3a65568e9 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -15,8 +15,6 @@ # limitations under the License. # -from __future__ import print_function - import os import shutil import signal diff --git a/python/pyspark/version.py b/python/pyspark/version.py index ba2a40cec01e6..1abc41279ebe8 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/python/run-tests b/python/run-tests index 24949657ed7ab..b8c64d8a295a2 100755 --- a/python/run-tests +++ b/python/run-tests @@ -21,4 +21,10 @@ FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" -exec python -u ./python/run-tests.py "$@" +PYTHON_VERSION_CHECK=$(python3 -c 'import sys; print(sys.version_info < (3, 6, 0))') +if [[ "$PYTHON_VERSION_CHECK" == "True" ]]; then + echo "Python versions prior to 3.6 are not supported." + exit -1 +fi + +exec python3 -u ./python/run-tests.py "$@" diff --git a/python/run-tests.py b/python/run-tests.py index b1119b044d71f..5bcf8b0669129 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -17,7 +17,6 @@ # limitations under the License. # -from __future__ import print_function import logging from argparse import ArgumentParser import os @@ -160,11 +159,15 @@ def run_individual_python_test(target_dir, test_name, pyspark_python): def get_default_python_executables(): - python_execs = [x for x in ["python2.7", "python3.6", "pypy"] if which(x)] - if "python2.7" not in python_execs: - LOGGER.warning("Not testing against `python2.7` because it could not be found; falling" - " back to `python` instead") - python_execs.insert(0, "python") + python_execs = [x for x in ["python3.6", "python2.7", "pypy"] if which(x)] + + if "python3.6" not in python_execs: + p = which("python3") + if not p: + LOGGER.error("No python3 executable found. Exiting!") + os._exit(1) + else: + python_execs.insert(0, p) return python_execs diff --git a/python/setup.py b/python/setup.py old mode 100644 new mode 100755 index ea672309703b6..092bdd3f90117 --- a/python/setup.py +++ b/python/setup.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -16,15 +16,14 @@ # See the License for the specific language governing permissions and # limitations under the License. -from __future__ import print_function import glob import os import sys from setuptools import setup from shutil import copyfile, copytree, rmtree -if sys.version_info < (2, 7): - print("Python versions prior to 2.7 are not supported for pip installed PySpark.", +if sys.version_info < (3, 6): + print("Python versions prior to 3.6 are not supported for pip installed PySpark.", file=sys.stderr) sys.exit(-1) From fca0a6c394990b86304a8f9a64bf4c7ec58abbd6 Mon Sep 17 00:00:00 2001 From: Kevin Yu Date: Thu, 14 Nov 2019 14:58:32 -0600 Subject: [PATCH 0614/1387] [SPARK-28833][DOCS][SQL] Document ALTER VIEW command ### What changes were proposed in this pull request? Document ALTER VIEW statement in the SQL Reference Guide. ### Why are the changes needed? Currently Spark SQL doc lacks documentation on the supported SQL syntax. This pr is aimed to address this issue. ### Does this PR introduce any user-facing change? Yes #### Before: There was no documentation for this. #### After: ![Screen Shot 2019-11-13 at 10 51 33 PM](https://user-images.githubusercontent.com/7550280/68833575-ac947f80-0668-11ea-910f-c133407ef502.png) ![Screen Shot 2019-11-13 at 10 56 42 PM](https://user-images.githubusercontent.com/7550280/68833597-bae29b80-0668-11ea-9782-b7be94789c12.png) ![Screen Shot 2019-11-13 at 10 56 53 PM](https://user-images.githubusercontent.com/7550280/68833607-be762280-0668-11ea-8a30-5602e755bab8.png) ### How was this patch tested? Tested using jkyll build --serve Closes #25573 from kevinyu98/spark-28833-alterview. Authored-by: Kevin Yu Signed-off-by: Sean Owen --- docs/sql-ref-syntax-ddl-alter-view.md | 215 +++++++++++++++++++++++++- 1 file changed, 214 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-ddl-alter-view.md b/docs/sql-ref-syntax-ddl-alter-view.md index 7a7947a3e0ac4..d5cbb86d48821 100644 --- a/docs/sql-ref-syntax-ddl-alter-view.md +++ b/docs/sql-ref-syntax-ddl-alter-view.md @@ -19,4 +19,217 @@ license: | limitations under the License. --- -**This page is under construction** +### Description + +The `ALTER VIEW` statement can alter metadata associated with the view. It can change the definition of the view, change +the name of a view to a different name, set and unset the metadata of the view by setting `TBLPROPERTIES`. + +#### RENAME View +Renames the existing view. If the new view name already exists in the source database, a `TableAlreadyExistsException` is thrown. This operation +does not support moving the views across databases. + +#### Syntax +{% highlight sql %} +ALTER VIEW view_identifier RENAME TO view_identifier +{% endhighlight %} + +#### Parameters +

    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [database_name.]view_name + +
    +
    + +#### SET View Properties +Set one or more properties of an existing view. The properties are the key value pairs. If the properties' keys exist, +the values are replaced with the new values. If the properties' keys do not exist, the key value pairs are added into +the properties. + +#### Syntax +{% highlight sql %} +ALTER VIEW view_identifier SET TBLPROPERTIES (property_key=property_val [, ...]) +{% endhighlight %} + +#### Parameters +
    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [database_name.]view_name + +
    +
    property_key
    +
    + Specifies the property key. The key may consists of multiple parts separated by dot.

    + Syntax: + + [key_part1][.key_part2][...] + +
    +
    + +#### UNSET View Properties +Drop one or more properties of an existing view. If the specified keys do not exist, an exception is thrown. Use +`IF EXISTS` to avoid the exception. + +#### Syntax +{% highlight sql %} +ALTER VIEW view_identifier UNSET TBLPROPERTIES [IF EXISTS] (property_key [, ...]) +{% endhighlight %} + +#### Parameters +
    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [database_name.]view_name + +
    +
    property_key
    +
    + Specifies the property key. The key may consists of multiple parts separated by dot.

    + Syntax: + + [key_part1][.key_part2][...] + +
    +
    + +#### ALTER View AS SELECT +`ALTER VIEW view_identifier AS SELECT` statement changes the definition of a view, the `SELECT` statement must be valid, +and the `view_identifier` must exist. + +#### Syntax +{% highlight sql %} +ALTER VIEW view_identifier AS select_statement +{% endhighlight %} + +#### Parameters +
    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [database_name.]view_name + +
    +
    select_statement
    +
    + Specifies the definition of the view, detail check select_statement +
    +
    + +### Examples + +{% highlight sql %} +-- Rename only changes the view name. +-- The source and target databases of the view have to be the same. +-- Use qualified or unqualified name for the source and target view +ALTER VIEW tempdb1.v1 RENAME TO tempdb1.v2; + +-- Verify that the new view is created. +DESCRIBE TABLE EXTENDED tempdb1.v2; + ++----------------------------+----------+-------+ +|col_name |data_type |comment| ++----------------------------+----------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | ++----------------------------+----------+-------+ + +-- Before ALTER VIEW SET TBLPROPERTIES +DESC TABLE EXTENDED tempdb1.v2; + ++----------------------------+----------+-------+ +|col_name |data_type |comment| ++----------------------------+----------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | +|Table Properties |[....] | | ++----------------------------+----------+-------+ + +-- Set properties in TBLPROPERTIES +ALTER VIEW tempdb1.v2 SET TBLPROPERTIES ('created.by.user' = "John", 'created.date' = '01-01-2001' ); + +-- Use `DESCRIBE TABLE EXTENDED tempdb1.v2` to verify +DESC TABLE EXTENDED tempdb1.v2; + ++----------------------------+-----------------------------------------------------+-------+ +|col_name |data_type |comment| ++----------------------------+-----------------------------------------------------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | +|Table Properties |[created.by.user=John, created.date=01-01-2001, ....]| | ++----------------------------+-----------------------------------------------------+-------+ + +-- Remove the key `created.by.user` and `created.date` from `TBLPROPERTIES` +ALTER VIEW tempdb1.v2 UNSET TBLPROPERTIES ('created.by.user', 'created.date'); + +--Use `DESC TABLE EXTENDED tempdb1.v2` to verify the changes +DESC TABLE EXTENDED tempdb1.v2; + ++----------------------------+----------+-------+ +|col_name |data_type |comment| ++----------------------------+----------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | +|Table Properties |[....] | | ++----------------------------+----------+-------+ + +-- Change the view definition +ALTER VIEW tempdb1.v2 AS SELECT * FROM tempdb1.v1; + +-- Use `DESC TABLE EXTENDED` to verify +DESC TABLE EXTENDED tempdb1.v2; + ++----------------------------+---------------------------+-------+ +|col_name |data_type |comment| ++----------------------------+---------------------------+-------+ +|c1 |int |null | +|c2 |string |null | +| | | | +|# Detailed Table Information| | | +|Database |tempdb1 | | +|Table |v2 | | +|Type |VIEW | | +|View Text |select * from tempdb1.v1 | | +|View Original Text |select * from tempdb1.v1 | | ++----------------------------+---------------------------+-------+ +{% endhighlight %} + +### Related Statements + +- [describe-table](sql-ref-syntax-aux-describe-table.html) +- [create-view](sql-ref-syntax-ddl-create-view.html) +- [drop-view](sql-ref-syntax-ddl-drop-view.html) + +#### Note: + +`ALTER VIEW` statement does not support `SET SERDE` or `SET SERDEPROPERTIES` properties + From b095232f630221926a9eabb8233c20d03c9a6ebb Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Nov 2019 15:52:39 -0700 Subject: [PATCH 0615/1387] [SPARK-29865][K8S] Ensure client-mode executors have same name prefix This basically does what BasicDriverFeatureStep already does to achieve the same thing in cluster mode; but since that class (or any other feature) is not invoked in client mode, it needs to be done elsewhere. I also modified the client mode integration test to check the executor name prefix; while there I had to fix the minikube backend to parse the output from newer minikube versions (I have 1.5.2). Closes #26488 from vanzin/SPARK-29865. Authored-by: Marcelo Vanzin Signed-off-by: Erik Erlandson --- .../k8s/KubernetesClusterManager.scala | 13 ++++++++- .../ClientModeTestsSuite.scala | 27 +++++++++++++++++-- .../backend/minikube/Minikube.scala | 18 +++++++++---- 3 files changed, 50 insertions(+), 8 deletions(-) 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 6fff9dd4f9443..c591214d10796 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 @@ -23,7 +23,7 @@ import com.google.common.cache.CacheBuilder import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext -import org.apache.spark.deploy.k8s.{KubernetesUtils, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkKubernetesClientFactory} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging @@ -61,6 +61,17 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit None) } + // If KUBERNETES_EXECUTOR_POD_NAME_PREFIX is not set, initialize it so that all executors have + // the same prefix. This is needed for client mode, where the feature steps code that sets this + // configuration is not used. + // + // If/when feature steps are executed in client mode, they should instead take care of this, + // and this code should be removed. + if (!sc.conf.contains(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)) { + sc.conf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, + KubernetesConf.getResourceNamePrefix(sc.conf.get("spark.app.name"))) + } + val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( apiServerUri, Some(sc.conf.get(KUBERNETES_NAMESPACE)), diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala index 2720cdf74ca8f..4e205f69ce039 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala @@ -27,6 +27,7 @@ private[spark] trait ClientModeTestsSuite { k8sSuite: KubernetesSuite => val labels = Map("spark-app-selector" -> driverPodName) val driverPort = 7077 val blockManagerPort = 10000 + val executorLabel = "spark-client-it" val driverService = testBackend .getKubernetesClient .services() @@ -78,10 +79,11 @@ private[spark] trait ClientModeTestsSuite { k8sSuite: KubernetesSuite => "/var/run/secrets/kubernetes.io/serviceaccount/token") .addToArgs("--conf", "spark.kubernetes.authenticate.caCertFile=" + "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt") - .addToArgs("--conf", s"spark.kubernetes.driver.pod.name=$driverPodName") .addToArgs("--conf", "spark.executor.memory=500m") .addToArgs("--conf", "spark.executor.cores=1") - .addToArgs("--conf", "spark.executor.instances=1") + .addToArgs("--conf", "spark.executor.instances=2") + .addToArgs("--conf", "spark.kubernetes.executor.deleteOnTermination=false") + .addToArgs("--conf", s"spark.kubernetes.executor.label.$executorLabel=$executorLabel") .addToArgs("--conf", s"spark.driver.host=" + s"${driverService.getMetadata.getName}.${kubernetesTestComponents.namespace}.svc") @@ -99,6 +101,20 @@ private[spark] trait ClientModeTestsSuite { k8sSuite: KubernetesSuite => .getLog .contains("Pi is roughly 3"), "The application did not complete.") } + + val executors = kubernetesTestComponents + .kubernetesClient + .pods() + .inNamespace(kubernetesTestComponents.namespace) + .withLabel(executorLabel, executorLabel) + .list() + .getItems() + assert(executors.size === 2) + val prefixes = executors.asScala.map { pod => + val name = pod.getMetadata().getName() + name.substring(0, name.lastIndexOf("-")) + }.toSet + assert(prefixes.size === 1, s"Executor prefixes did not match: $prefixes") } finally { // Have to delete the service manually since it doesn't have an owner reference kubernetesTestComponents @@ -106,6 +122,13 @@ private[spark] trait ClientModeTestsSuite { k8sSuite: KubernetesSuite => .services() .inNamespace(kubernetesTestComponents.namespace) .delete(driverService) + // Delete all executors, since the test explicitly asks them not to be deleted by the app. + kubernetesTestComponents + .kubernetesClient + .pods() + .inNamespace(kubernetesTestComponents.namespace) + .withLabel(executorLabel, executorLabel) + .delete() } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala index ce2ce1c61c194..a7d35b67d1b92 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala @@ -30,6 +30,7 @@ private[spark] object Minikube extends Logging { private val KUBELET_PREFIX = "kubelet:" private val APISERVER_PREFIX = "apiserver:" private val KUBECTL_PREFIX = "kubectl:" + private val KUBECONFIG_PREFIX = "kubeconfig:" private val MINIKUBE_VM_PREFIX = "minikubeVM: " private val MINIKUBE_PREFIX = "minikube: " private val MINIKUBE_PATH = ".minikube" @@ -86,18 +87,23 @@ private[spark] object Minikube extends Logging { val kubeletString = statusString.find(_.contains(s"$KUBELET_PREFIX ")) val apiserverString = statusString.find(_.contains(s"$APISERVER_PREFIX ")) val kubectlString = statusString.find(_.contains(s"$KUBECTL_PREFIX ")) + val kubeconfigString = statusString.find(_.contains(s"$KUBECONFIG_PREFIX ")) + val hasConfigStatus = kubectlString.isDefined || kubeconfigString.isDefined - if (hostString.isEmpty || kubeletString.isEmpty - || apiserverString.isEmpty || kubectlString.isEmpty) { + if (hostString.isEmpty || kubeletString.isEmpty || apiserverString.isEmpty || + !hasConfigStatus) { MinikubeStatus.NONE } else { val status1 = hostString.get.replaceFirst(s"$HOST_PREFIX ", "") val status2 = kubeletString.get.replaceFirst(s"$KUBELET_PREFIX ", "") val status3 = apiserverString.get.replaceFirst(s"$APISERVER_PREFIX ", "") - val status4 = kubectlString.get.replaceFirst(s"$KUBECTL_PREFIX ", "") - if (!status4.contains("Correctly Configured:")) { - MinikubeStatus.NONE + val isConfigured = if (kubectlString.isDefined) { + val cfgStatus = kubectlString.get.replaceFirst(s"$KUBECTL_PREFIX ", "") + cfgStatus.contains("Correctly Configured:") } else { + kubeconfigString.get.replaceFirst(s"$KUBECONFIG_PREFIX ", "") == "Configured" + } + if (isConfigured) { val stats = List(status1, status2, status3) .map(MinikubeStatus.unapply) .map(_.getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))) @@ -106,6 +112,8 @@ private[spark] object Minikube extends Logging { } else { MinikubeStatus.RUNNING } + } else { + MinikubeStatus.NONE } } } From ab981f10a61ac15be14334b015fe2206e088314f Mon Sep 17 00:00:00 2001 From: turbofei Date: Thu, 14 Nov 2019 18:16:45 -0600 Subject: [PATCH 0616/1387] [SPARK-29857][WEB UI] Defer render the spark UI dataTables ### What changes were proposed in this pull request? This PR support defer render the spark UI page. ### Why are the changes needed? When there are many items, such as tasks and application lists, the renderer of dataTables is heavy, we can enable deferRender to optimize it. See details in https://datatables.net/examples/ajax/defer_render.html ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Not needed. Closes #26482 from turboFei/SPARK-29857-defer-render. Authored-by: turbofei Signed-off-by: Sean Owen --- .../main/resources/org/apache/spark/ui/static/executorspage.js | 3 ++- .../main/resources/org/apache/spark/ui/static/historypage.js | 1 + .../src/main/resources/org/apache/spark/ui/static/stagepage.js | 3 ++- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 11d7c77d0c667..090bc72dc9274 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -462,7 +462,8 @@ $(document).ready(function () { {"visible": false, "targets": 5}, {"visible": false, "targets": 6}, {"visible": false, "targets": 9} - ] + ], + "deferRender": true }; execDataTable = $(selector).DataTable(conf); diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index 5f291620e0e95..4df5f07f077d7 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -177,6 +177,7 @@ $(document).ready(function() { {name: 'eventLog'}, ], "autoWidth": false, + "deferRender": true }; if (hasMultipleAttempts) { diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index c05651681be42..4b2f007f866cb 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -880,7 +880,8 @@ $(document).ready(function () { { "visible": false, "targets": 16 }, { "visible": false, "targets": 17 }, { "visible": false, "targets": 18 } - ] + ], + "deferRender": true }; taskTableSelector = $(taskTable).DataTable(taskConf); $('#active-tasks-table_filter input').unbind(); From 17321782deac59516639aab648cb319a51c0bb9b Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 15 Nov 2019 11:13:36 +0900 Subject: [PATCH 0617/1387] [SPARK-26923][R][SQL][FOLLOW-UP] Show stderr in the exception whenever possible in RRunner ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23977 I made a mistake related to this line: https://github.com/apache/spark/commit/3725b1324f731d57dc776c256bc1a100ec9e6cd0#diff-71c2cad03f08cb5f6c70462aa4e28d3aL112 Previously, 1. the reader iterator for R worker read some initial data eagerly during RDD materialization. So it read the data before actual execution. For some reasons, in this case, it showed standard error from R worker. 2. After that, when error happens during actual execution, stderr wasn't shown: https://github.com/apache/spark/commit/3725b1324f731d57dc776c256bc1a100ec9e6cd0#diff-71c2cad03f08cb5f6c70462aa4e28d3aL260 After my change https://github.com/apache/spark/commit/3725b1324f731d57dc776c256bc1a100ec9e6cd0#diff-71c2cad03f08cb5f6c70462aa4e28d3aL112, it now ignores 1. case and only does 2. of previous code path, because 1. does not happen anymore as I avoided to such eager execution (which is consistent with PySpark code path). This PR proposes to do only 1. before/after execution always because It is pretty much possible R worker was failed during actual execution and it's best to show the stderr from R worker whenever possible. ### Why are the changes needed? It currently swallows standard error from R worker which makes debugging harder. ### Does this PR introduce any user-facing change? Yes, ```R df <- createDataFrame(list(list(n=1))) collect(dapply(df, function(x) { stop("asdkjasdjkbadskjbsdajbk") x }, structType("a double"))) ``` **Before:** ``` Error in handleErrors(returnStatus, conn) : org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 13.0 failed 1 times, most recent failure: Lost task 0.0 in stage 13.0 (TID 13, 192.168.35.193, executor driver): org.apache.spark.SparkException: R worker exited unexpectedly (cranshed) at org.apache.spark.api.r.RRunner$$anon$1.read(RRunner.scala:130) at org.apache.spark.api.r.BaseRRunner$ReaderIterator.hasNext(BaseRRunner.scala:118) at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:726) at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:337) at org.apache.spark. ``` **After:** ``` Error in handleErrors(returnStatus, conn) : org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, 192.168.35.193, executor driver): org.apache.spark.SparkException: R unexpectedly exited. R worker produced errors: Error in computeFunc(inputData) : asdkjasdjkbadskjbsdajbk at org.apache.spark.api.r.BaseRRunner$ReaderIterator$$anonfun$1.applyOrElse(BaseRRunner.scala:144) at org.apache.spark.api.r.BaseRRunner$ReaderIterator$$anonfun$1.applyOrElse(BaseRRunner.scala:137) at scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:38) at org.apache.spark.api.r.RRunner$$anon$1.read(RRunner.scala:128) at org.apache.spark.api.r.BaseRRunner$ReaderIterator.hasNext(BaseRRunner.scala:113) at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegen ``` ### How was this patch tested? Manually tested and unittest was added. Closes #26517 from HyukjinKwon/SPARK-26923-followup. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/pkg/tests/fulltests/test_sparkSQL.R | 7 +++++++ .../org/apache/spark/api/r/BaseRRunner.scala | 17 +++++++++++------ .../scala/org/apache/spark/api/r/RRunner.scala | 5 +---- .../spark/sql/execution/r/ArrowRRunner.scala | 6 +----- 4 files changed, 20 insertions(+), 15 deletions(-) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 035525a7a849b..c2b2458ec064b 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -3238,6 +3238,13 @@ test_that("Histogram", { expect_equal(histogram(df, "x")$counts, c(4, 0, 0, 0, 0, 0, 0, 0, 0, 1)) }) +test_that("dapply() should show error message from R worker", { + df <- createDataFrame(list(list(n = 1))) + expect_error({ + collect(dapply(df, function(x) stop("custom error message"), structType("a double"))) + }, "custom error message") +}) + test_that("dapply() and dapplyCollect() on a DataFrame", { df <- createDataFrame( list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), diff --git a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala index d8f9d1f1729b7..fdfe5f5b41d0a 100644 --- a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala @@ -82,12 +82,7 @@ private[spark] abstract class BaseRRunner[IN, OUT]( serverSocket.close() } - try { - newReaderIterator(dataStream, errThread) - } catch { - case e: Exception => - throw new SparkException("R computation failed with\n " + errThread.getLines(), e) - } + newReaderIterator(dataStream, errThread) } /** @@ -138,6 +133,16 @@ private[spark] abstract class BaseRRunner[IN, OUT]( * and then returns null. */ protected def read(): OUT + + protected val handleException: PartialFunction[Throwable, OUT] = { + case e: Exception => + var msg = "R unexpectedly exited." + val lines = errThread.getLines() + if (lines.trim().nonEmpty) { + msg += s"\nR worker produced errors: $lines\n" + } + throw new SparkException(msg, e) + } } /** diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 0327386b45ed5..20ab6fc2f348d 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -125,10 +125,7 @@ private[spark] class RRunner[IN, OUT]( eos = true null.asInstanceOf[OUT] } - } catch { - case eof: EOFException => - throw new SparkException("R worker exited unexpectedly (cranshed)", eof) - } + } catch handleException } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala index 0fe2b628fa38b..59f5a7078a151 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala @@ -191,11 +191,7 @@ class ArrowRRunner( null } } - } catch { - case eof: EOFException => - throw new SparkException( - "R worker exited unexpectedly (crashed)\n " + errThread.getLines(), eof) - } + } catch handleException } } } From d128ef13d8e4ccd46a6eac90834b7520f6684fa4 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 15 Nov 2019 11:29:28 +0900 Subject: [PATCH 0618/1387] [SPARK-29901][SQL][DOC] Fix broken links in SQL Reference ### What changes were proposed in this pull request? Fix broken links ### How was this patch tested? Tested using jykyll build --serve Closes #26528 from huaxingao/spark-29901. Authored-by: Huaxin Gao Signed-off-by: HyukjinKwon --- docs/sql-ref-syntax-aux-show-databases.md | 2 +- docs/sql-ref-syntax-aux-show-tblproperties.md | 3 ++- docs/sql-ref-syntax-ddl-create-view.md | 2 +- docs/sql-ref-syntax-ddl-truncate-table.md | 2 +- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/sql-ref-syntax-aux-show-databases.md b/docs/sql-ref-syntax-aux-show-databases.md index 39121f764de2e..efa32f2915143 100644 --- a/docs/sql-ref-syntax-aux-show-databases.md +++ b/docs/sql-ref-syntax-aux-show-databases.md @@ -74,6 +74,6 @@ SHOW SCHEMAS; +------------+ {% endhighlight %} ### Related Statements -- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-databases.html) +- [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) - [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) - [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) diff --git a/docs/sql-ref-syntax-aux-show-tblproperties.md b/docs/sql-ref-syntax-aux-show-tblproperties.md index e0738316e97b4..212db05017ad4 100644 --- a/docs/sql-ref-syntax-aux-show-tblproperties.md +++ b/docs/sql-ref-syntax-aux-show-tblproperties.md @@ -111,4 +111,5 @@ SHOW TBLPROPERTIES customer ('created.date'); ### Related Statements - [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) - [ALTER TABLE SET TBLPROPERTIES](sql-ref-syntax-ddl-alter-table.html) -- [SHOW TABLE](sql-ref-syntax-aux-show-table.html) +- [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) +- [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md index dcafdbb750d0c..16310caca0bd9 100644 --- a/docs/sql-ref-syntax-ddl-create-view.md +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -58,7 +58,7 @@ CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_nam
    query
    -
    A SELECT statement that constructs the view from base tables or other views.
    +
    A SELECT statement that constructs the view from base tables or other views.
    ### Examples diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md index 58d1b9618f0a0..7baaefe317ba1 100644 --- a/docs/sql-ref-syntax-ddl-truncate-table.md +++ b/docs/sql-ref-syntax-ddl-truncate-table.md @@ -82,5 +82,5 @@ No rows selected ### Related Statements - [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) -- [ALTER TABLE](sql-ref-syntax-ddl-alter-tabley.html) +- [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) From bb8b04d4a2b311fbaeb0f4cb9daa6a30e4478532 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 15 Nov 2019 10:38:51 +0800 Subject: [PATCH 0619/1387] [SPARK-29889][SQL][TEST] unify the interval tests ### What changes were proposed in this pull request? move interval tests to `interval.sql`, and import it to `ansi/interval.sql` ### Why are the changes needed? improve test coverage ### Does this PR introduce any user-facing change? no ### How was this patch tested? N/A Closes #26515 from cloud-fan/test. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql-tests/inputs/ansi/interval.sql | 190 +-- .../resources/sql-tests/inputs/group-by.sql | 64 - .../resources/sql-tests/inputs/interval.sql | 134 +- .../sql-tests/results/ansi/interval.sql.out | 1300 +++++++++++++---- .../sql-tests/results/group-by.sql.out | 176 +-- .../sql-tests/results/interval.sql.out | 651 ++++++--- 6 files changed, 1670 insertions(+), 845 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql index f662769f016c5..215ee7c074fa6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql @@ -1,187 +1,17 @@ -select - '1' second, - 2 seconds, - '1' minute, - 2 minutes, - '1' hour, - 2 hours, - '1' day, - 2 days, - '1' month, - 2 months, - '1' year, - 2 years; - -select - interval '10-11' year to month, - interval '10' year, - interval '11' month; - -select - '10-11' year to month, - '10' year, - '11' month; - -select - interval '10 9:8:7.987654321' day to second, - interval '10' day, - interval '11' hour, - interval '12' minute, - interval '13' second, - interval '13.123456789' second; - -select - '10 9:8:7.987654321' day to second, - '10' day, - '11' hour, - '12' minute, - '13' second, - '13.123456789' second; - -select map(1, interval 1 day, 2, interval 3 week); - -select map(1, 1 day, 2, 3 week); - --- Interval year-month arithmetic - -create temporary view interval_arithmetic as - select CAST(dateval AS date), CAST(tsval AS timestamp) from values - ('2012-01-01', '2012-01-01') - as interval_arithmetic(dateval, tsval); - -select - dateval, - dateval - interval '2-2' year to month, - dateval - interval '-2-2' year to month, - dateval + interval '2-2' year to month, - dateval + interval '-2-2' year to month, - - interval '2-2' year to month + dateval, - interval '2-2' year to month + dateval -from interval_arithmetic; - -select - dateval, - dateval - '2-2' year to month, - dateval - '-2-2' year to month, - dateval + '2-2' year to month, - dateval + '-2-2' year to month, - - '2-2' year to month + dateval, - '2-2' year to month + dateval -from interval_arithmetic; - -select - tsval, - tsval - interval '2-2' year to month, - tsval - interval '-2-2' year to month, - tsval + interval '2-2' year to month, - tsval + interval '-2-2' year to month, - - interval '2-2' year to month + tsval, - interval '2-2' year to month + tsval -from interval_arithmetic; - -select - tsval, - tsval - '2-2' year to month, - tsval - '-2-2' year to month, - tsval + '2-2' year to month, - tsval + '-2-2' year to month, - - '2-2' year to month + tsval, - '2-2' year to month + tsval -from interval_arithmetic; - -select - interval '2-2' year to month + interval '3-3' year to month, - interval '2-2' year to month - interval '3-3' year to month -from interval_arithmetic; - -select - '2-2' year to month + '3-3' year to month, - '2-2' year to month - '3-3' year to month -from interval_arithmetic; - --- Interval day-time arithmetic - -select - dateval, - dateval - interval '99 11:22:33.123456789' day to second, - dateval - interval '-99 11:22:33.123456789' day to second, - dateval + interval '99 11:22:33.123456789' day to second, - dateval + interval '-99 11:22:33.123456789' day to second, - -interval '99 11:22:33.123456789' day to second + dateval, - interval '99 11:22:33.123456789' day to second + dateval -from interval_arithmetic; - -select - dateval, - dateval - '99 11:22:33.123456789' day to second, - dateval - '-99 11:22:33.123456789' day to second, - dateval + '99 11:22:33.123456789' day to second, - dateval + '-99 11:22:33.123456789' day to second, - - '99 11:22:33.123456789' day to second + dateval, - '99 11:22:33.123456789' day to second + dateval -from interval_arithmetic; - -select - tsval, - tsval - interval '99 11:22:33.123456789' day to second, - tsval - interval '-99 11:22:33.123456789' day to second, - tsval + interval '99 11:22:33.123456789' day to second, - tsval + interval '-99 11:22:33.123456789' day to second, - -interval '99 11:22:33.123456789' day to second + tsval, - interval '99 11:22:33.123456789' day to second + tsval -from interval_arithmetic; - -select - tsval, - tsval - '99 11:22:33.123456789' day to second, - tsval - '-99 11:22:33.123456789' day to second, - tsval + '99 11:22:33.123456789' day to second, - tsval + '-99 11:22:33.123456789' day to second, - - '99 11:22:33.123456789' day to second + tsval, - '99 11:22:33.123456789' day to second + tsval -from interval_arithmetic; - -select - interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, - interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second -from interval_arithmetic; - -select - '99 11:22:33.123456789' day to second + '10 9:8:7.123456789' day to second, - '99 11:22:33.123456789' day to second - '10 9:8:7.123456789' day to second -from interval_arithmetic; - --- More tests for interval syntax alternatives - -select 30 day; +--import interval.sql +-- the `interval` keyword can be omitted with ansi mode +select 1 year 2 days; +select '10-9' year to month; +select '20 15:40:32.99899999' day to second; select 30 day day; - -select 30 day day day; - -select date '2012-01-01' - 30 day; - -select date '2012-01-01' - 30 day day; - -select date '2012-01-01' - 30 day day day; - -select date '2012-01-01' + '-30' day; - -select date '2012-01-01' + interval '-30' day; - --- Unsupported syntax for intervals - -select date '2012-01-01' + interval (-30) day; - -select date '2012-01-01' + (-30) day; - -create temporary view t as select * from values (1), (2) as t(a); - -select date '2012-01-01' + interval (a + 1) day from t; - -select date '2012-01-01' + (a + 1) day from t; +select date'2012-01-01' - '2-2' year to month; +select 1 month - 1 day; -- malformed interval literal with ansi mode select 1 year to month; select '1' year to second; select 1 year '2-1' year to month; +select (-30) day; +select (a + 1) day; +select 30 day day day; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index c405fb0aa9e83..d602f63e529d1 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -161,67 +161,3 @@ SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L; SELECT count(*) FROM test_agg WHERE count(*) > 1L; SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L; SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; - --- sum interval values --- null -select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where v is null; - --- empty set -select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0; - --- basic interval sum -select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v); -select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v); -select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v); -select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v); - --- group by -select - i, - sum(cast(v as interval)) -from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) -group by i; - --- having -select - sum(cast(v as interval)) as sv -from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) -having sv is not null; - --- window -SELECT - i, - Sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) -FROM VALUES(1,'1 seconds'),(1,'2 seconds'),(2,NULL),(2,NULL) t(i,v); - --- average with interval type --- null -select avg(cast(v as interval)) from VALUES (null) t(v); - --- empty set -select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0; - --- basic interval avg -select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v); -select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v); -select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v); -select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v); - --- group by -select - i, - avg(cast(v as interval)) -from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) -group by i; - --- having -select - avg(cast(v as interval)) as sv -from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) -having sv is not null; - --- window -SELECT - i, - avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) -FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v); diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index b5c3416d0fc66..94d5aae9bce31 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -53,9 +53,11 @@ select interval '2 seconds' / null; select interval '2 seconds' * null; select null * interval '2 seconds'; --- interval with a negative sign +-- interval with a positive/negative sign select -interval '-1 month 1 day -1 second'; select -interval -1 month 1 day -1 second; +select +interval '-1 month 1 day -1 second'; +select +interval -1 month 1 day -1 second; -- make intervals select make_interval(1); @@ -89,7 +91,7 @@ select justify_days(interval '1 month 59 day -25 hour'); select justify_hours(interval '1 month 59 day -25 hour'); select justify_interval(interval '1 month 59 day -25 hour'); --- interval +-- interval literal select interval 13.123456789 seconds, interval -13.123456789 second; select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond; select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; @@ -105,6 +107,7 @@ select interval '15:40:32.99899999' hour to second; select interval '20 40:32.99899999' minute to second; select interval '40:32.99899999' minute to second; select interval '40:32' minute to second; +select interval 30 day day; -- ns is not supported select interval 10 nanoseconds; @@ -128,6 +131,129 @@ select interval 1 year '2-1' year to month; select interval 1 year '12:11:10' hour to second; select interval '10-9' year to month '1' year; select interval '12:11:10' hour to second '1' year; +select interval (-30) day; +select interval (a + 1) day; +select interval 30 day day day; --- awareness of the positive sign before interval type -select +interval '1 second'; +-- sum interval values +-- null +select sum(cast(null as interval)); + +-- empty set +select sum(cast(v as interval)) from VALUES ('1 seconds') t(v) where 1=0; + +-- basic interval sum +select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v); +select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v); +select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v); +select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v); + +-- group by +select + i, + sum(cast(v as interval)) +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +group by i; + +-- having +select + sum(cast(v as interval)) as sv +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +having sv is not null; + +-- window +SELECT + i, + sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +FROM VALUES(1, '1 seconds'), (1, '2 seconds'), (2, NULL), (2, NULL) t(i,v); + +-- average with interval type +-- null +select avg(cast(v as interval)) from VALUES (null) t(v); + +-- empty set +select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0; + +-- basic interval avg +select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v); +select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v); +select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v); +select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v); + +-- group by +select + i, + avg(cast(v as interval)) +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +group by i; + +-- having +select + avg(cast(v as interval)) as sv +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +having sv is not null; + +-- window +SELECT + i, + avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v); + +-- Interval year-month arithmetic + +create temporary view interval_arithmetic as + select CAST(dateval AS date), CAST(tsval AS timestamp) from values + ('2012-01-01', '2012-01-01') + as interval_arithmetic(dateval, tsval); + +select + dateval, + dateval - interval '2-2' year to month, + dateval - interval '-2-2' year to month, + dateval + interval '2-2' year to month, + dateval + interval '-2-2' year to month, + - interval '2-2' year to month + dateval, + interval '2-2' year to month + dateval +from interval_arithmetic; + +select + tsval, + tsval - interval '2-2' year to month, + tsval - interval '-2-2' year to month, + tsval + interval '2-2' year to month, + tsval + interval '-2-2' year to month, + - interval '2-2' year to month + tsval, + interval '2-2' year to month + tsval +from interval_arithmetic; + +select + interval '2-2' year to month + interval '3-3' year to month, + interval '2-2' year to month - interval '3-3' year to month +from interval_arithmetic; + +-- Interval day-time arithmetic + +select + dateval, + dateval - interval '99 11:22:33.123456789' day to second, + dateval - interval '-99 11:22:33.123456789' day to second, + dateval + interval '99 11:22:33.123456789' day to second, + dateval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + dateval, + interval '99 11:22:33.123456789' day to second + dateval +from interval_arithmetic; + +select + tsval, + tsval - interval '99 11:22:33.123456789' day to second, + tsval - interval '-99 11:22:33.123456789' day to second, + tsval + interval '99 11:22:33.123456789' day to second, + tsval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + tsval, + interval '99 11:22:33.123456789' day to second + tsval +from interval_arithmetic; + +select + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second +from interval_arithmetic; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index a9c90ccc80938..73bf299c509cf 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,433 +1,1179 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 36 +-- Number of queries: 130 -- !query 0 -select - '1' second, - 2 seconds, - '1' minute, - 2 minutes, - '1' hour, - 2 hours, - '1' day, - 2 days, - '1' month, - 2 months, - '1' year, - 2 years +select interval '1 day' > interval '23 hour' -- !query 0 schema -struct<1 seconds:interval,2 seconds:interval,1 minutes:interval,2 minutes:interval,1 hours:interval,2 hours:interval,1 days:interval,2 days:interval,1 months:interval,2 months:interval,1 years:interval,2 years:interval> +struct<(1 days > 23 hours):boolean> -- !query 0 output -1 seconds 2 seconds 1 minutes 2 minutes 1 hours 2 hours 1 days 2 days 1 months 2 months 1 years 2 years +true -- !query 1 -select - interval '10-11' year to month, - interval '10' year, - interval '11' month +select interval '-1 day' >= interval '-23 hour' -- !query 1 schema -struct<10 years 11 months:interval,10 years:interval,11 months:interval> +struct<(-1 days >= -23 hours):boolean> -- !query 1 output -10 years 11 months 10 years 11 months +false -- !query 2 -select - '10-11' year to month, - '10' year, - '11' month +select interval '-1 day' > null -- !query 2 schema -struct<10 years 11 months:interval,10 years:interval,11 months:interval> +struct<(-1 days > CAST(NULL AS INTERVAL)):boolean> -- !query 2 output -10 years 11 months 10 years 11 months +NULL -- !query 3 -select - interval '10 9:8:7.987654321' day to second, - interval '10' day, - interval '11' hour, - interval '12' minute, - interval '13' second, - interval '13.123456789' second +select null > interval '-1 day' -- !query 3 schema -struct<10 days 9 hours 8 minutes 7.987654 seconds:interval,10 days:interval,11 hours:interval,12 minutes:interval,13 seconds:interval,13.123456 seconds:interval> +struct<(CAST(NULL AS INTERVAL) > -1 days):boolean> -- !query 3 output -10 days 9 hours 8 minutes 7.987654 seconds 10 days 11 hours 12 minutes 13 seconds 13.123456 seconds +NULL -- !query 4 -select - '10 9:8:7.987654321' day to second, - '10' day, - '11' hour, - '12' minute, - '13' second, - '13.123456789' second +select interval '1 minutes' < interval '1 hour' -- !query 4 schema -struct<10 days 9 hours 8 minutes 7.987654 seconds:interval,10 days:interval,11 hours:interval,12 minutes:interval,13 seconds:interval,13.123456 seconds:interval> +struct<(1 minutes < 1 hours):boolean> -- !query 4 output -10 days 9 hours 8 minutes 7.987654 seconds 10 days 11 hours 12 minutes 13 seconds 13.123456 seconds +true -- !query 5 -select map(1, interval 1 day, 2, interval 3 week) +select interval '-1 day' <= interval '-23 hour' -- !query 5 schema -struct> +struct<(-1 days <= -23 hours):boolean> -- !query 5 output -{1:1 days,2:21 days} +true -- !query 6 -select map(1, 1 day, 2, 3 week) +select interval '1 year' = interval '360 days' -- !query 6 schema -struct> +struct<(1 years = 360 days):boolean> -- !query 6 output -{1:1 days,2:21 days} +true -- !query 7 -create temporary view interval_arithmetic as - select CAST(dateval AS date), CAST(tsval AS timestamp) from values - ('2012-01-01', '2012-01-01') - as interval_arithmetic(dateval, tsval) +select interval '1 year 2 month' = interval '420 days' -- !query 7 schema -struct<> +struct<(1 years 2 months = 420 days):boolean> -- !query 7 output - +true -- !query 8 -select - dateval, - dateval - interval '2-2' year to month, - dateval - interval '-2-2' year to month, - dateval + interval '2-2' year to month, - dateval + interval '-2-2' year to month, - - interval '2-2' year to month + dateval, - interval '2-2' year to month + dateval -from interval_arithmetic +select interval '1 year' = interval '365 days' -- !query 8 schema -struct +struct<(1 years = 365 days):boolean> -- !query 8 output -2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 +false -- !query 9 -select - dateval, - dateval - '2-2' year to month, - dateval - '-2-2' year to month, - dateval + '2-2' year to month, - dateval + '-2-2' year to month, - - '2-2' year to month + dateval, - '2-2' year to month + dateval -from interval_arithmetic +select interval '1 month' = interval '30 days' -- !query 9 schema -struct +struct<(1 months = 30 days):boolean> -- !query 9 output -2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 +true -- !query 10 -select - tsval, - tsval - interval '2-2' year to month, - tsval - interval '-2-2' year to month, - tsval + interval '2-2' year to month, - tsval + interval '-2-2' year to month, - - interval '2-2' year to month + tsval, - interval '2-2' year to month + tsval -from interval_arithmetic +select interval '1 minutes' = interval '1 hour' -- !query 10 schema -struct +struct<(1 minutes = 1 hours):boolean> -- !query 10 output -2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 +false -- !query 11 -select - tsval, - tsval - '2-2' year to month, - tsval - '-2-2' year to month, - tsval + '2-2' year to month, - tsval + '-2-2' year to month, - - '2-2' year to month + tsval, - '2-2' year to month + tsval -from interval_arithmetic +select interval '1 minutes' = null -- !query 11 schema -struct +struct<(1 minutes = CAST(NULL AS INTERVAL)):boolean> -- !query 11 output -2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 +NULL -- !query 12 -select - interval '2-2' year to month + interval '3-3' year to month, - interval '2-2' year to month - interval '3-3' year to month -from interval_arithmetic +select null = interval '-1 day' -- !query 12 schema -struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> +struct<(CAST(NULL AS INTERVAL) = -1 days):boolean> -- !query 12 output -5 years 5 months -1 years -1 months +NULL -- !query 13 -select - '2-2' year to month + '3-3' year to month, - '2-2' year to month - '3-3' year to month -from interval_arithmetic +select interval '1 minutes' <=> null -- !query 13 schema -struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> +struct<(1 minutes <=> CAST(NULL AS INTERVAL)):boolean> -- !query 13 output -5 years 5 months -1 years -1 months +false -- !query 14 -select - dateval, - dateval - interval '99 11:22:33.123456789' day to second, - dateval - interval '-99 11:22:33.123456789' day to second, - dateval + interval '99 11:22:33.123456789' day to second, - dateval + interval '-99 11:22:33.123456789' day to second, - -interval '99 11:22:33.123456789' day to second + dateval, - interval '99 11:22:33.123456789' day to second + dateval -from interval_arithmetic +select null <=> interval '1 minutes' -- !query 14 schema -struct +struct<(CAST(NULL AS INTERVAL) <=> 1 minutes):boolean> -- !query 14 output -2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 +false -- !query 15 -select - dateval, - dateval - '99 11:22:33.123456789' day to second, - dateval - '-99 11:22:33.123456789' day to second, - dateval + '99 11:22:33.123456789' day to second, - dateval + '-99 11:22:33.123456789' day to second, - - '99 11:22:33.123456789' day to second + dateval, - '99 11:22:33.123456789' day to second + dateval -from interval_arithmetic +select INTERVAL '9 years 1 months -1 weeks -4 days -10 hours -46 minutes' > interval '1 minutes' -- !query 15 schema -struct +struct<(9 years 1 months -11 days -10 hours -46 minutes > 1 minutes):boolean> -- !query 15 output -2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 +true -- !query 16 -select - tsval, - tsval - interval '99 11:22:33.123456789' day to second, - tsval - interval '-99 11:22:33.123456789' day to second, - tsval + interval '99 11:22:33.123456789' day to second, - tsval + interval '-99 11:22:33.123456789' day to second, - -interval '99 11:22:33.123456789' day to second + tsval, - interval '99 11:22:33.123456789' day to second + tsval -from interval_arithmetic +select cast(v as interval) i from VALUES ('1 seconds'), ('4 seconds'), ('3 seconds') t(v) order by i -- !query 16 schema -struct +struct -- !query 16 output -2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 +1 seconds +3 seconds +4 seconds -- !query 17 -select - tsval, - tsval - '99 11:22:33.123456789' day to second, - tsval - '-99 11:22:33.123456789' day to second, - tsval + '99 11:22:33.123456789' day to second, - tsval + '-99 11:22:33.123456789' day to second, - - '99 11:22:33.123456789' day to second + tsval, - '99 11:22:33.123456789' day to second + tsval -from interval_arithmetic +select interval '1 month 120 days' > interval '2 month' -- !query 17 schema -struct +struct<(1 months 120 days > 2 months):boolean> -- !query 17 output -2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 +true -- !query 18 -select - interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, - interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second -from interval_arithmetic +select interval '1 month 30 days' = interval '2 month' -- !query 18 schema -struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> +struct<(1 months 30 days = 2 months):boolean> -- !query 18 output -109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds +true -- !query 19 -select - '99 11:22:33.123456789' day to second + '10 9:8:7.123456789' day to second, - '99 11:22:33.123456789' day to second - '10 9:8:7.123456789' day to second -from interval_arithmetic +select interval '1 month 29 days 40 hours' > interval '2 month' -- !query 19 schema -struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> +struct<(1 months 29 days 40 hours > 2 months):boolean> -- !query 19 output -109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds +true -- !query 20 -select 30 day +select max(cast(v as interval)) from VALUES ('1 seconds'), ('4 seconds'), ('3 seconds') t(v) -- !query 20 schema -struct<30 days:interval> +struct -- !query 20 output -30 days +4 seconds -- !query 21 -select 30 day day +select min(cast(v as interval)) from VALUES ('1 seconds'), ('4 seconds'), ('3 seconds') t(v) -- !query 21 schema -struct<> +struct -- !query 21 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input 'day'(line 1, pos 14) - -== SQL == -select 30 day day ---------------^^^ +1 seconds -- !query 22 -select 30 day day day +select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct<> +struct -- !query 22 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input 'day'(line 1, pos 14) - -== SQL == -select 30 day day day ---------------^^^ +30 hours 33 minutes 36.003006 seconds -- !query 23 -select date '2012-01-01' - 30 day +select interval 4 month 2 weeks 3 microseconds * 1.5 -- !query 23 schema -struct +struct -- !query 23 output -2011-12-02 +6 months 21 days 0.000005 seconds -- !query 24 -select date '2012-01-01' - 30 day day +select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5 -- !query 24 schema -struct<> +struct -- !query 24 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input 'day'(line 1, pos 34) - -== SQL == -select date '2012-01-01' - 30 day day -----------------------------------^^^ +16 hours -- !query 25 -select date '2012-01-01' - 30 day day day +select interval '2 seconds' / 0 -- !query 25 schema -struct<> +struct -- !query 25 output -org.apache.spark.sql.catalyst.parser.ParseException - -no viable alternative at input 'day'(line 1, pos 34) - -== SQL == -select date '2012-01-01' - 30 day day day -----------------------------------^^^ +NULL -- !query 26 -select date '2012-01-01' + '-30' day +select interval '2 seconds' / null -- !query 26 schema -struct +struct -- !query 26 output -2011-12-02 +NULL -- !query 27 -select date '2012-01-01' + interval '-30' day +select interval '2 seconds' * null -- !query 27 schema -struct +struct -- !query 27 output -2011-12-02 +NULL -- !query 28 -select date '2012-01-01' + interval (-30) day +select null * interval '2 seconds' -- !query 28 schema -struct<> +struct -- !query 28 output +NULL + + +-- !query 29 +select -interval '-1 month 1 day -1 second' +-- !query 29 schema +struct<1 months -1 days 1 seconds:interval> +-- !query 29 output +1 months -1 days 1 seconds + + +-- !query 30 +select -interval -1 month 1 day -1 second +-- !query 30 schema +struct<1 months -1 days 1 seconds:interval> +-- !query 30 output +1 months -1 days 1 seconds + + +-- !query 31 +select +interval '-1 month 1 day -1 second' +-- !query 31 schema +struct<-1 months 1 days -1 seconds:interval> +-- !query 31 output +-1 months 1 days -1 seconds + + +-- !query 32 +select +interval -1 month 1 day -1 second +-- !query 32 schema +struct<-1 months 1 days -1 seconds:interval> +-- !query 32 output +-1 months 1 days -1 seconds + + +-- !query 33 +select make_interval(1) +-- !query 33 schema +struct +-- !query 33 output +1 years + + +-- !query 34 +select make_interval(1, 2) +-- !query 34 schema +struct +-- !query 34 output +1 years 2 months + + +-- !query 35 +select make_interval(1, 2, 3) +-- !query 35 schema +struct +-- !query 35 output +1 years 2 months 21 days + + +-- !query 36 +select make_interval(1, 2, 3, 4) +-- !query 36 schema +struct +-- !query 36 output +1 years 2 months 25 days + + +-- !query 37 +select make_interval(1, 2, 3, 4, 5) +-- !query 37 schema +struct +-- !query 37 output +1 years 2 months 25 days 5 hours + + +-- !query 38 +select make_interval(1, 2, 3, 4, 5, 6) +-- !query 38 schema +struct +-- !query 38 output +1 years 2 months 25 days 5 hours 6 minutes + + +-- !query 39 +select make_interval(1, 2, 3, 4, 5, 6, 7.008009) +-- !query 39 schema +struct +-- !query 39 output +1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds + + +-- !query 40 +select cast('1 second' as interval) +-- !query 40 schema +struct +-- !query 40 output +1 seconds + + +-- !query 41 +select cast('+1 second' as interval) +-- !query 41 schema +struct +-- !query 41 output +1 seconds + + +-- !query 42 +select cast('-1 second' as interval) +-- !query 42 schema +struct +-- !query 42 output +-1 seconds + + +-- !query 43 +select cast('+ 1 second' as interval) +-- !query 43 schema +struct +-- !query 43 output +1 seconds + + +-- !query 44 +select cast('- 1 second' as interval) +-- !query 44 schema +struct +-- !query 44 output +-1 seconds + + +-- !query 45 +select cast('- -1 second' as interval) +-- !query 45 schema +struct +-- !query 45 output +NULL + + +-- !query 46 +select cast('- +1 second' as interval) +-- !query 46 schema +struct +-- !query 46 output +NULL + + +-- !query 47 +select justify_days(cast(null as interval)) +-- !query 47 schema +struct +-- !query 47 output +NULL + + +-- !query 48 +select justify_hours(cast(null as interval)) +-- !query 48 schema +struct +-- !query 48 output +NULL + + +-- !query 49 +select justify_interval(cast(null as interval)) +-- !query 49 schema +struct +-- !query 49 output +NULL + + +-- !query 50 +select justify_days(interval '1 month 59 day 25 hour') +-- !query 50 schema +struct +-- !query 50 output +2 months 29 days 25 hours + + +-- !query 51 +select justify_hours(interval '1 month 59 day 25 hour') +-- !query 51 schema +struct +-- !query 51 output +1 months 60 days 1 hours + + +-- !query 52 +select justify_interval(interval '1 month 59 day 25 hour') +-- !query 52 schema +struct +-- !query 52 output +3 months 1 hours + + +-- !query 53 +select justify_days(interval '1 month -59 day 25 hour') +-- !query 53 schema +struct +-- !query 53 output +-29 days 25 hours + + +-- !query 54 +select justify_hours(interval '1 month -59 day 25 hour') +-- !query 54 schema +struct +-- !query 54 output +1 months -57 days -23 hours + + +-- !query 55 +select justify_interval(interval '1 month -59 day 25 hour') +-- !query 55 schema +struct +-- !query 55 output +-27 days -23 hours + + +-- !query 56 +select justify_days(interval '1 month 59 day -25 hour') +-- !query 56 schema +struct +-- !query 56 output +2 months 29 days -25 hours + + +-- !query 57 +select justify_hours(interval '1 month 59 day -25 hour') +-- !query 57 schema +struct +-- !query 57 output +1 months 57 days 23 hours + + +-- !query 58 +select justify_interval(interval '1 month 59 day -25 hour') +-- !query 58 schema +struct +-- !query 58 output +2 months 27 days 23 hours + + +-- !query 59 +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query 59 schema +struct<13.123456 seconds:interval,-13.123456 seconds:interval> +-- !query 59 output +13.123456 seconds -13.123456 seconds + + +-- !query 60 +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond +-- !query 60 schema +struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> +-- !query 60 output +1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds + + +-- !query 61 +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second +-- !query 61 schema +struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> +-- !query 61 output +32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds + + +-- !query 62 +select interval '0 0:0:0.1' day to second +-- !query 62 schema +struct<0.1 seconds:interval> +-- !query 62 output +0.1 seconds + + +-- !query 63 +select interval '10-9' year to month +-- !query 63 schema +struct<10 years 9 months:interval> +-- !query 63 output +10 years 9 months + + +-- !query 64 +select interval '20 15:40:32.99899999' day to hour +-- !query 64 schema +struct<20 days 15 hours:interval> +-- !query 64 output +20 days 15 hours + + +-- !query 65 +select interval '20 15:40:32.99899999' day to minute +-- !query 65 schema +struct<20 days 15 hours 40 minutes:interval> +-- !query 65 output +20 days 15 hours 40 minutes + + +-- !query 66 +select interval '20 15:40:32.99899999' day to second +-- !query 66 schema +struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +-- !query 66 output +20 days 15 hours 40 minutes 32.998999 seconds + + +-- !query 67 +select interval '15:40:32.99899999' hour to minute +-- !query 67 schema +struct<15 hours 40 minutes:interval> +-- !query 67 output +15 hours 40 minutes + + +-- !query 68 +select interval '15:40.99899999' hour to second +-- !query 68 schema +struct<15 minutes 40.998999 seconds:interval> +-- !query 68 output +15 minutes 40.998999 seconds + + +-- !query 69 +select interval '15:40' hour to second +-- !query 69 schema +struct<15 hours 40 minutes:interval> +-- !query 69 output +15 hours 40 minutes + + +-- !query 70 +select interval '15:40:32.99899999' hour to second +-- !query 70 schema +struct<15 hours 40 minutes 32.998999 seconds:interval> +-- !query 70 output +15 hours 40 minutes 32.998999 seconds + + +-- !query 71 +select interval '20 40:32.99899999' minute to second +-- !query 71 schema +struct<20 days 40 minutes 32.998999 seconds:interval> +-- !query 71 output +20 days 40 minutes 32.998999 seconds + + +-- !query 72 +select interval '40:32.99899999' minute to second +-- !query 72 schema +struct<40 minutes 32.998999 seconds:interval> +-- !query 72 output +40 minutes 32.998999 seconds + + +-- !query 73 +select interval '40:32' minute to second +-- !query 73 schema +struct<40 minutes 32 seconds:interval> +-- !query 73 output +40 minutes 32 seconds + + +-- !query 74 +select interval 30 day day +-- !query 74 schema +struct<> +-- !query 74 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 23) + +== SQL == +select interval 30 day day +-----------------------^^^ + + +-- !query 75 +select interval 10 nanoseconds +-- !query 75 schema +struct<> +-- !query 75 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input '10 nanoseconds'(line 1, pos 19) + +== SQL == +select interval 10 nanoseconds +-------------------^^^ + + +-- !query 76 +select map(1, interval 1 day, 2, interval 3 week) +-- !query 76 schema +struct> +-- !query 76 output +{1:1 days,2:21 days} + + +-- !query 77 +select interval 'interval 3 year 1 hour' +-- !query 77 schema +struct<3 years 1 hours:interval> +-- !query 77 output +3 years 1 hours + + +-- !query 78 +select interval '3 year 1 hour' +-- !query 78 schema +struct<3 years 1 hours:interval> +-- !query 78 output +3 years 1 hours + + +-- !query 79 +select interval +-- !query 79 schema +struct<> +-- !query 79 output +org.apache.spark.sql.catalyst.parser.ParseException + +at least one time unit should be given for interval literal(line 1, pos 7) + +== SQL == +select interval +-------^^^ + + +-- !query 80 +select interval 1 fake_unit +-- !query 80 schema +struct<> +-- !query 80 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input '1 fake_unit'(line 1, pos 18) + +== SQL == +select interval 1 fake_unit +------------------^^^ + + +-- !query 81 +select interval 1 year to month +-- !query 81 schema +struct<> +-- !query 81 output +org.apache.spark.sql.catalyst.parser.ParseException + +The value of from-to unit must be a string(line 1, pos 16) + +== SQL == +select interval 1 year to month +----------------^^^ + + +-- !query 82 +select interval '1' year to second +-- !query 82 schema +struct<> +-- !query 82 output +org.apache.spark.sql.catalyst.parser.ParseException + +Intervals FROM year TO second are not supported.(line 1, pos 16) + +== SQL == +select interval '1' year to second +----------------^^^ + + +-- !query 83 +select interval '10-9' year to month '2-1' year to month +-- !query 83 schema +struct<> +-- !query 83 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '2-1' year to month +-------------------------------------^^^ + + +-- !query 84 +select interval '10-9' year to month '12:11:10' hour to second +-- !query 84 schema +struct<> +-- !query 84 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '12:11:10' hour to second +-------------------------------------^^^ + + +-- !query 85 +select interval '1 15:11' day to minute '12:11:10' hour to second +-- !query 85 schema +struct<> +-- !query 85 output org.apache.spark.sql.catalyst.parser.ParseException -no viable alternative at input 'day'(line 1, pos 42) +Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) == SQL == -select date '2012-01-01' + interval (-30) day +select interval '1 15:11' day to minute '12:11:10' hour to second +----------------------------------------^^^ + + +-- !query 86 +select interval 1 year '2-1' year to month +-- !query 86 schema +struct<> +-- !query 86 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) + +== SQL == +select interval 1 year '2-1' year to month +-----------------------^^^ + + +-- !query 87 +select interval 1 year '12:11:10' hour to second +-- !query 87 schema +struct<> +-- !query 87 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) + +== SQL == +select interval 1 year '12:11:10' hour to second +-----------------------^^^ + + +-- !query 88 +select interval '10-9' year to month '1' year +-- !query 88 schema +struct<> +-- !query 88 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) + +== SQL == +select interval '10-9' year to month '1' year +-------------------------------------^^^ + + +-- !query 89 +select interval '12:11:10' hour to second '1' year +-- !query 89 schema +struct<> +-- !query 89 output +org.apache.spark.sql.catalyst.parser.ParseException + +Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) + +== SQL == +select interval '12:11:10' hour to second '1' year ------------------------------------------^^^ --- !query 29 -select date '2012-01-01' + (-30) day --- !query 29 schema +-- !query 90 +select interval (-30) day +-- !query 90 schema struct<> --- !query 29 output +-- !query 90 output org.apache.spark.sql.catalyst.parser.ParseException -no viable alternative at input 'day'(line 1, pos 33) +no viable alternative at input 'day'(line 1, pos 22) == SQL == -select date '2012-01-01' + (-30) day ----------------------------------^^^ +select interval (-30) day +----------------------^^^ --- !query 30 -create temporary view t as select * from values (1), (2) as t(a) --- !query 30 schema +-- !query 91 +select interval (a + 1) day +-- !query 91 schema struct<> --- !query 30 output +-- !query 91 output +org.apache.spark.sql.catalyst.parser.ParseException +no viable alternative at input 'day'(line 1, pos 24) +== SQL == +select interval (a + 1) day +------------------------^^^ --- !query 31 -select date '2012-01-01' + interval (a + 1) day from t --- !query 31 schema + +-- !query 92 +select interval 30 day day day +-- !query 92 schema struct<> --- !query 31 output +-- !query 92 output org.apache.spark.sql.catalyst.parser.ParseException -no viable alternative at input 'day'(line 1, pos 44) +no viable alternative at input 'day'(line 1, pos 23) == SQL == -select date '2012-01-01' + interval (a + 1) day from t ---------------------------------------------^^^ +select interval 30 day day day +-----------------------^^^ --- !query 32 -select date '2012-01-01' + (a + 1) day from t --- !query 32 schema +-- !query 93 +select sum(cast(null as interval)) +-- !query 93 schema +struct +-- !query 93 output +NULL + + +-- !query 94 +select sum(cast(v as interval)) from VALUES ('1 seconds') t(v) where 1=0 +-- !query 94 schema +struct +-- !query 94 output +NULL + + +-- !query 95 +select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) +-- !query 95 schema +struct +-- !query 95 output +3 seconds + + +-- !query 96 +select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) +-- !query 96 schema +struct +-- !query 96 output +1 seconds + + +-- !query 97 +select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) +-- !query 97 schema +struct +-- !query 97 output +-3 seconds + + +-- !query 98 +select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) +-- !query 98 schema +struct +-- !query 98 output +-7 days 2 seconds + + +-- !query 99 +select + i, + sum(cast(v as interval)) +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +group by i +-- !query 99 schema +struct +-- !query 99 output +1 -2 days +2 2 seconds +3 NULL + + +-- !query 100 +select + sum(cast(v as interval)) as sv +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +having sv is not null +-- !query 100 schema +struct +-- !query 100 output +-2 days 2 seconds + + +-- !query 101 +SELECT + i, + sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +FROM VALUES(1, '1 seconds'), (1, '2 seconds'), (2, NULL), (2, NULL) t(i,v) +-- !query 101 schema +struct +-- !query 101 output +1 2 seconds +1 3 seconds +2 NULL +2 NULL + + +-- !query 102 +select avg(cast(v as interval)) from VALUES (null) t(v) +-- !query 102 schema +struct +-- !query 102 output +NULL + + +-- !query 103 +select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0 +-- !query 103 schema +struct +-- !query 103 output +NULL + + +-- !query 104 +select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) +-- !query 104 schema +struct +-- !query 104 output +1.5 seconds + + +-- !query 105 +select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) +-- !query 105 schema +struct +-- !query 105 output +0.5 seconds + + +-- !query 106 +select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) +-- !query 106 schema +struct +-- !query 106 output +-1.5 seconds + + +-- !query 107 +select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) +-- !query 107 schema +struct +-- !query 107 output +-3 days -11 hours -59 minutes -59 seconds + + +-- !query 108 +select + i, + avg(cast(v as interval)) +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +group by i +-- !query 108 schema +struct +-- !query 108 output +1 -1 days +2 2 seconds +3 NULL + + +-- !query 109 +select + avg(cast(v as interval)) as sv +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +having sv is not null +-- !query 109 schema +struct +-- !query 109 output +-15 hours -59 minutes -59.333333 seconds + + +-- !query 110 +SELECT + i, + avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v) +-- !query 110 schema +struct +-- !query 110 output +1 1.5 seconds +1 2 seconds +2 NULL +2 NULL + + +-- !query 111 +create temporary view interval_arithmetic as + select CAST(dateval AS date), CAST(tsval AS timestamp) from values + ('2012-01-01', '2012-01-01') + as interval_arithmetic(dateval, tsval) +-- !query 111 schema struct<> --- !query 32 output +-- !query 111 output + + + +-- !query 112 +select + dateval, + dateval - interval '2-2' year to month, + dateval - interval '-2-2' year to month, + dateval + interval '2-2' year to month, + dateval + interval '-2-2' year to month, + - interval '2-2' year to month + dateval, + interval '2-2' year to month + dateval +from interval_arithmetic +-- !query 112 schema +struct +-- !query 112 output +2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 + + +-- !query 113 +select + tsval, + tsval - interval '2-2' year to month, + tsval - interval '-2-2' year to month, + tsval + interval '2-2' year to month, + tsval + interval '-2-2' year to month, + - interval '2-2' year to month + tsval, + interval '2-2' year to month + tsval +from interval_arithmetic +-- !query 113 schema +struct +-- !query 113 output +2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 + + +-- !query 114 +select + interval '2-2' year to month + interval '3-3' year to month, + interval '2-2' year to month - interval '3-3' year to month +from interval_arithmetic +-- !query 114 schema +struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> +-- !query 114 output +5 years 5 months -1 years -1 months + + +-- !query 115 +select + dateval, + dateval - interval '99 11:22:33.123456789' day to second, + dateval - interval '-99 11:22:33.123456789' day to second, + dateval + interval '99 11:22:33.123456789' day to second, + dateval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + dateval, + interval '99 11:22:33.123456789' day to second + dateval +from interval_arithmetic +-- !query 115 schema +struct +-- !query 115 output +2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 + + +-- !query 116 +select + tsval, + tsval - interval '99 11:22:33.123456789' day to second, + tsval - interval '-99 11:22:33.123456789' day to second, + tsval + interval '99 11:22:33.123456789' day to second, + tsval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + tsval, + interval '99 11:22:33.123456789' day to second + tsval +from interval_arithmetic +-- !query 116 schema +struct +-- !query 116 output +2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 + + +-- !query 117 +select + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second +from interval_arithmetic +-- !query 117 schema +struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> +-- !query 117 output +109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds + + +-- !query 118 +select 1 year 2 days +-- !query 118 schema +struct<1 years 2 days:interval> +-- !query 118 output +1 years 2 days + + +-- !query 119 +select '10-9' year to month +-- !query 119 schema +struct<10 years 9 months:interval> +-- !query 119 output +10 years 9 months + + +-- !query 120 +select '20 15:40:32.99899999' day to second +-- !query 120 schema +struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +-- !query 120 output +20 days 15 hours 40 minutes 32.998999 seconds + + +-- !query 121 +select 30 day day +-- !query 121 schema +struct<> +-- !query 121 output org.apache.spark.sql.catalyst.parser.ParseException -no viable alternative at input 'day'(line 1, pos 35) +no viable alternative at input 'day'(line 1, pos 14) == SQL == -select date '2012-01-01' + (a + 1) day from t ------------------------------------^^^ +select 30 day day +--------------^^^ --- !query 33 +-- !query 122 +select date'2012-01-01' - '2-2' year to month +-- !query 122 schema +struct +-- !query 122 output +2009-11-01 + + +-- !query 123 +select 1 month - 1 day +-- !query 123 schema +struct<1 months -1 days:interval> +-- !query 123 output +1 months -1 days + + +-- !query 124 select 1 year to month --- !query 33 schema +-- !query 124 schema struct<> --- !query 33 output +-- !query 124 output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 7) @@ -437,11 +1183,11 @@ select 1 year to month -------^^^ --- !query 34 +-- !query 125 select '1' year to second --- !query 34 schema +-- !query 125 schema struct<> --- !query 34 output +-- !query 125 output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 7) @@ -451,11 +1197,11 @@ select '1' year to second -------^^^ --- !query 35 +-- !query 126 select 1 year '2-1' year to month --- !query 35 schema +-- !query 126 schema struct<> --- !query 35 output +-- !query 126 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 14) @@ -463,3 +1209,45 @@ Can only have a single from-to unit in the interval literal syntax(line 1, pos 1 == SQL == select 1 year '2-1' year to month --------------^^^ + + +-- !query 127 +select (-30) day +-- !query 127 schema +struct<> +-- !query 127 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 13) + +== SQL == +select (-30) day +-------------^^^ + + +-- !query 128 +select (a + 1) day +-- !query 128 schema +struct<> +-- !query 128 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 15) + +== SQL == +select (a + 1) day +---------------^^^ + + +-- !query 129 +select 30 day day day +-- !query 129 schema +struct<> +-- !query 129 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 14) + +== SQL == +select 30 day day day +--------------^^^ diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 6a40702f6eb0a..8b6e370a9867a 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 74 +-- Number of queries: 56 -- !query 0 @@ -573,177 +573,3 @@ org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. Expression in where clause: [(((test_agg.`k` = 1) OR (test_agg.`k` = 2)) OR (((count(1) + 1L) > 1L) OR (max(test_agg.`k`) > 1)))] Invalid expressions: [count(1), max(test_agg.`k`)]; - - --- !query 56 -select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where v is null --- !query 56 schema -struct --- !query 56 output -NULL - - --- !query 57 -select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0 --- !query 57 schema -struct --- !query 57 output -NULL - - --- !query 58 -select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 58 schema -struct --- !query 58 output -3 seconds - - --- !query 59 -select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 59 schema -struct --- !query 59 output -1 seconds - - --- !query 60 -select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 60 schema -struct --- !query 60 output --3 seconds - - --- !query 61 -select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 61 schema -struct --- !query 61 output --7 days 2 seconds - - --- !query 62 -select - i, - sum(cast(v as interval)) -from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) -group by i --- !query 62 schema -struct --- !query 62 output -1 -2 days -2 2 seconds -3 NULL - - --- !query 63 -select - sum(cast(v as interval)) as sv -from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) -having sv is not null --- !query 63 schema -struct --- !query 63 output --2 days 2 seconds - - --- !query 64 -SELECT - i, - Sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) -FROM VALUES(1,'1 seconds'),(1,'2 seconds'),(2,NULL),(2,NULL) t(i,v) --- !query 64 schema -struct --- !query 64 output -1 2 seconds -1 3 seconds -2 NULL -2 NULL - - --- !query 65 -select avg(cast(v as interval)) from VALUES (null) t(v) --- !query 65 schema -struct --- !query 65 output -NULL - - --- !query 66 -select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0 --- !query 66 schema -struct --- !query 66 output -NULL - - --- !query 67 -select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 67 schema -struct --- !query 67 output -1.5 seconds - - --- !query 68 -select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 68 schema -struct --- !query 68 output -0.5 seconds - - --- !query 69 -select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 69 schema -struct --- !query 69 output --1.5 seconds - - --- !query 70 -select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 70 schema -struct --- !query 70 output --3 days -11 hours -59 minutes -59 seconds - - --- !query 71 -select - i, - avg(cast(v as interval)) -from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) -group by i --- !query 71 schema -struct --- !query 71 output -1 -1 days -2 2 seconds -3 NULL - - --- !query 72 -select - avg(cast(v as interval)) as sv -from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) -having sv is not null --- !query 72 schema -struct --- !query 72 output --15 hours -59 minutes -59.333333 seconds - - --- !query 73 -SELECT - i, - avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) -FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v) --- !query 73 schema -struct --- !query 73 output -1 1.5 seconds -1 2 seconds -2 NULL -2 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 91dd72dfa492b..d0f2ff4b44d01 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 88 +-- Number of queries: 118 -- !query 0 @@ -253,295 +253,295 @@ struct<1 months -1 days 1 seconds:interval> -- !query 31 -select make_interval(1) +select +interval '-1 month 1 day -1 second' -- !query 31 schema -struct +struct<-1 months 1 days -1 seconds:interval> -- !query 31 output -1 years +-1 months 1 days -1 seconds -- !query 32 -select make_interval(1, 2) +select +interval -1 month 1 day -1 second -- !query 32 schema -struct +struct<-1 months 1 days -1 seconds:interval> -- !query 32 output -1 years 2 months +-1 months 1 days -1 seconds -- !query 33 -select make_interval(1, 2, 3) +select make_interval(1) -- !query 33 schema -struct +struct -- !query 33 output -1 years 2 months 21 days +1 years -- !query 34 -select make_interval(1, 2, 3, 4) +select make_interval(1, 2) -- !query 34 schema -struct +struct -- !query 34 output -1 years 2 months 25 days +1 years 2 months -- !query 35 -select make_interval(1, 2, 3, 4, 5) +select make_interval(1, 2, 3) -- !query 35 schema -struct +struct -- !query 35 output -1 years 2 months 25 days 5 hours +1 years 2 months 21 days -- !query 36 -select make_interval(1, 2, 3, 4, 5, 6) +select make_interval(1, 2, 3, 4) -- !query 36 schema -struct +struct -- !query 36 output -1 years 2 months 25 days 5 hours 6 minutes +1 years 2 months 25 days -- !query 37 -select make_interval(1, 2, 3, 4, 5, 6, 7.008009) +select make_interval(1, 2, 3, 4, 5) -- !query 37 schema -struct +struct -- !query 37 output -1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds +1 years 2 months 25 days 5 hours -- !query 38 -select cast('1 second' as interval) +select make_interval(1, 2, 3, 4, 5, 6) -- !query 38 schema -struct +struct -- !query 38 output -1 seconds +1 years 2 months 25 days 5 hours 6 minutes -- !query 39 -select cast('+1 second' as interval) +select make_interval(1, 2, 3, 4, 5, 6, 7.008009) -- !query 39 schema -struct +struct -- !query 39 output -1 seconds +1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds -- !query 40 -select cast('-1 second' as interval) +select cast('1 second' as interval) -- !query 40 schema -struct +struct -- !query 40 output --1 seconds +1 seconds -- !query 41 -select cast('+ 1 second' as interval) +select cast('+1 second' as interval) -- !query 41 schema -struct +struct -- !query 41 output 1 seconds -- !query 42 -select cast('- 1 second' as interval) +select cast('-1 second' as interval) -- !query 42 schema -struct +struct -- !query 42 output -1 seconds -- !query 43 -select cast('- -1 second' as interval) +select cast('+ 1 second' as interval) -- !query 43 schema -struct +struct -- !query 43 output -NULL +1 seconds -- !query 44 -select cast('- +1 second' as interval) +select cast('- 1 second' as interval) -- !query 44 schema -struct +struct -- !query 44 output -NULL +-1 seconds -- !query 45 -select justify_days(cast(null as interval)) +select cast('- -1 second' as interval) -- !query 45 schema -struct +struct -- !query 45 output NULL -- !query 46 -select justify_hours(cast(null as interval)) +select cast('- +1 second' as interval) -- !query 46 schema -struct +struct -- !query 46 output NULL -- !query 47 -select justify_interval(cast(null as interval)) +select justify_days(cast(null as interval)) -- !query 47 schema -struct +struct -- !query 47 output NULL -- !query 48 -select justify_days(interval '1 month 59 day 25 hour') +select justify_hours(cast(null as interval)) -- !query 48 schema -struct +struct -- !query 48 output -2 months 29 days 25 hours +NULL -- !query 49 -select justify_hours(interval '1 month 59 day 25 hour') +select justify_interval(cast(null as interval)) -- !query 49 schema -struct +struct -- !query 49 output -1 months 60 days 1 hours +NULL -- !query 50 -select justify_interval(interval '1 month 59 day 25 hour') +select justify_days(interval '1 month 59 day 25 hour') -- !query 50 schema -struct +struct -- !query 50 output -3 months 1 hours +2 months 29 days 25 hours -- !query 51 -select justify_days(interval '1 month -59 day 25 hour') +select justify_hours(interval '1 month 59 day 25 hour') -- !query 51 schema -struct +struct -- !query 51 output --29 days 25 hours +1 months 60 days 1 hours -- !query 52 -select justify_hours(interval '1 month -59 day 25 hour') +select justify_interval(interval '1 month 59 day 25 hour') -- !query 52 schema -struct +struct -- !query 52 output -1 months -57 days -23 hours +3 months 1 hours -- !query 53 -select justify_interval(interval '1 month -59 day 25 hour') +select justify_days(interval '1 month -59 day 25 hour') -- !query 53 schema -struct +struct -- !query 53 output --27 days -23 hours +-29 days 25 hours -- !query 54 -select justify_days(interval '1 month 59 day -25 hour') +select justify_hours(interval '1 month -59 day 25 hour') -- !query 54 schema -struct +struct -- !query 54 output -2 months 29 days -25 hours +1 months -57 days -23 hours -- !query 55 -select justify_hours(interval '1 month 59 day -25 hour') +select justify_interval(interval '1 month -59 day 25 hour') -- !query 55 schema -struct +struct -- !query 55 output -1 months 57 days 23 hours +-27 days -23 hours -- !query 56 -select justify_interval(interval '1 month 59 day -25 hour') +select justify_days(interval '1 month 59 day -25 hour') -- !query 56 schema -struct +struct -- !query 56 output -2 months 27 days 23 hours +2 months 29 days -25 hours -- !query 57 -select interval 13.123456789 seconds, interval -13.123456789 second +select justify_hours(interval '1 month 59 day -25 hour') -- !query 57 schema -struct<13.123456 seconds:interval,-13.123456 seconds:interval> +struct -- !query 57 output -13.123456 seconds -13.123456 seconds +1 months 57 days 23 hours -- !query 58 -select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond +select justify_interval(interval '1 month 59 day -25 hour') -- !query 58 schema -struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> +struct -- !query 58 output -1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds +2 months 27 days 23 hours -- !query 59 -select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second +select interval 13.123456789 seconds, interval -13.123456789 second -- !query 59 schema -struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> +struct<13.123456 seconds:interval,-13.123456 seconds:interval> -- !query 59 output -32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds +13.123456 seconds -13.123456 seconds -- !query 60 -select interval '0 0:0:0.1' day to second +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond -- !query 60 schema -struct<0.1 seconds:interval> +struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> -- !query 60 output -0.1 seconds +1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds -- !query 61 -select interval '10-9' year to month +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 61 schema -struct<10 years 9 months:interval> +struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> -- !query 61 output -10 years 9 months +32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds -- !query 62 -select interval '20 15:40:32.99899999' day to hour +select interval '0 0:0:0.1' day to second -- !query 62 schema -struct<20 days 15 hours:interval> +struct<0.1 seconds:interval> -- !query 62 output -20 days 15 hours +0.1 seconds -- !query 63 -select interval '20 15:40:32.99899999' day to minute +select interval '10-9' year to month -- !query 63 schema -struct<20 days 15 hours 40 minutes:interval> +struct<10 years 9 months:interval> -- !query 63 output -20 days 15 hours 40 minutes +10 years 9 months -- !query 64 -select interval '20 15:40:32.99899999' day to second +select interval '20 15:40:32.99899999' day to hour -- !query 64 schema -struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +struct<20 days 15 hours:interval> -- !query 64 output -20 days 15 hours 40 minutes 32.998999 seconds +20 days 15 hours -- !query 65 -select interval '15:40:32.99899999' hour to minute +select interval '20 15:40:32.99899999' day to minute -- !query 65 schema -struct<15 hours 40 minutes:interval> +struct<20 days 15 hours 40 minutes:interval> -- !query 65 output -15 hours 40 minutes +20 days 15 hours 40 minutes -- !query 66 -select interval '15:40.99899999' hour to second +select interval '20 15:40:32.99899999' day to second -- !query 66 schema -struct<15 minutes 40.998999 seconds:interval> +struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> -- !query 66 output -15 minutes 40.998999 seconds +20 days 15 hours 40 minutes 32.998999 seconds -- !query 67 -select interval '15:40' hour to second +select interval '15:40:32.99899999' hour to minute -- !query 67 schema struct<15 hours 40 minutes:interval> -- !query 67 output @@ -549,42 +549,66 @@ struct<15 hours 40 minutes:interval> -- !query 68 -select interval '15:40:32.99899999' hour to second +select interval '15:40.99899999' hour to second -- !query 68 schema -struct<15 hours 40 minutes 32.998999 seconds:interval> +struct<15 minutes 40.998999 seconds:interval> -- !query 68 output -15 hours 40 minutes 32.998999 seconds +15 minutes 40.998999 seconds -- !query 69 -select interval '20 40:32.99899999' minute to second +select interval '15:40' hour to second -- !query 69 schema -struct<20 days 40 minutes 32.998999 seconds:interval> +struct<15 hours 40 minutes:interval> -- !query 69 output -20 days 40 minutes 32.998999 seconds +15 hours 40 minutes -- !query 70 -select interval '40:32.99899999' minute to second +select interval '15:40:32.99899999' hour to second -- !query 70 schema -struct<40 minutes 32.998999 seconds:interval> +struct<15 hours 40 minutes 32.998999 seconds:interval> -- !query 70 output -40 minutes 32.998999 seconds +15 hours 40 minutes 32.998999 seconds -- !query 71 -select interval '40:32' minute to second +select interval '20 40:32.99899999' minute to second -- !query 71 schema -struct<40 minutes 32 seconds:interval> +struct<20 days 40 minutes 32.998999 seconds:interval> -- !query 71 output -40 minutes 32 seconds +20 days 40 minutes 32.998999 seconds -- !query 72 -select interval 10 nanoseconds +select interval '40:32.99899999' minute to second -- !query 72 schema -struct<> +struct<40 minutes 32.998999 seconds:interval> -- !query 72 output +40 minutes 32.998999 seconds + + +-- !query 73 +select interval '40:32' minute to second +-- !query 73 schema +struct<40 minutes 32 seconds:interval> +-- !query 73 output +40 minutes 32 seconds + + +-- !query 74 +select interval 30 day day +-- !query 74 schema +struct +-- !query 74 output +30 days + + +-- !query 75 +select interval 10 nanoseconds +-- !query 75 schema +struct<> +-- !query 75 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input '10 nanoseconds'(line 1, pos 19) @@ -594,35 +618,35 @@ select interval 10 nanoseconds -------------------^^^ --- !query 73 +-- !query 76 select map(1, interval 1 day, 2, interval 3 week) --- !query 73 schema +-- !query 76 schema struct> --- !query 73 output +-- !query 76 output {1:1 days,2:21 days} --- !query 74 +-- !query 77 select interval 'interval 3 year 1 hour' --- !query 74 schema +-- !query 77 schema struct<3 years 1 hours:interval> --- !query 74 output +-- !query 77 output 3 years 1 hours --- !query 75 +-- !query 78 select interval '3 year 1 hour' --- !query 75 schema +-- !query 78 schema struct<3 years 1 hours:interval> --- !query 75 output +-- !query 78 output 3 years 1 hours --- !query 76 +-- !query 79 select interval --- !query 76 schema +-- !query 79 schema struct<> --- !query 76 output +-- !query 79 output org.apache.spark.sql.catalyst.parser.ParseException at least one time unit should be given for interval literal(line 1, pos 7) @@ -632,11 +656,11 @@ select interval -------^^^ --- !query 77 +-- !query 80 select interval 1 fake_unit --- !query 77 schema +-- !query 80 schema struct<> --- !query 77 output +-- !query 80 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input '1 fake_unit'(line 1, pos 18) @@ -646,11 +670,11 @@ select interval 1 fake_unit ------------------^^^ --- !query 78 +-- !query 81 select interval 1 year to month --- !query 78 schema +-- !query 81 schema struct<> --- !query 78 output +-- !query 81 output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 16) @@ -660,11 +684,11 @@ select interval 1 year to month ----------------^^^ --- !query 79 +-- !query 82 select interval '1' year to second --- !query 79 schema +-- !query 82 schema struct<> --- !query 79 output +-- !query 82 output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 16) @@ -674,11 +698,11 @@ select interval '1' year to second ----------------^^^ --- !query 80 +-- !query 83 select interval '10-9' year to month '2-1' year to month --- !query 80 schema +-- !query 83 schema struct<> --- !query 80 output +-- !query 83 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -688,11 +712,11 @@ select interval '10-9' year to month '2-1' year to month -------------------------------------^^^ --- !query 81 +-- !query 84 select interval '10-9' year to month '12:11:10' hour to second --- !query 81 schema +-- !query 84 schema struct<> --- !query 81 output +-- !query 84 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -702,11 +726,11 @@ select interval '10-9' year to month '12:11:10' hour to second -------------------------------------^^^ --- !query 82 +-- !query 85 select interval '1 15:11' day to minute '12:11:10' hour to second --- !query 82 schema +-- !query 85 schema struct<> --- !query 82 output +-- !query 85 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) @@ -716,11 +740,11 @@ select interval '1 15:11' day to minute '12:11:10' hour to second ----------------------------------------^^^ --- !query 83 +-- !query 86 select interval 1 year '2-1' year to month --- !query 83 schema +-- !query 86 schema struct<> --- !query 83 output +-- !query 86 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -730,11 +754,11 @@ select interval 1 year '2-1' year to month -----------------------^^^ --- !query 84 +-- !query 87 select interval 1 year '12:11:10' hour to second --- !query 84 schema +-- !query 87 schema struct<> --- !query 84 output +-- !query 87 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -744,11 +768,11 @@ select interval 1 year '12:11:10' hour to second -----------------------^^^ --- !query 85 +-- !query 88 select interval '10-9' year to month '1' year --- !query 85 schema +-- !query 88 schema struct<> --- !query 85 output +-- !query 88 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -758,11 +782,11 @@ select interval '10-9' year to month '1' year -------------------------------------^^^ --- !query 86 +-- !query 89 select interval '12:11:10' hour to second '1' year --- !query 86 schema +-- !query 89 schema struct<> --- !query 86 output +-- !query 89 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) @@ -772,9 +796,304 @@ select interval '12:11:10' hour to second '1' year ------------------------------------------^^^ --- !query 87 -select +interval '1 second' --- !query 87 schema -struct<1 seconds:interval> --- !query 87 output +-- !query 90 +select interval (-30) day +-- !query 90 schema +struct<> +-- !query 90 output +org.apache.spark.sql.AnalysisException +Undefined function: 'interval'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 + + +-- !query 91 +select interval (a + 1) day +-- !query 91 schema +struct<> +-- !query 91 output +org.apache.spark.sql.AnalysisException +Undefined function: 'interval'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 + + +-- !query 92 +select interval 30 day day day +-- !query 92 schema +struct<> +-- !query 92 output +org.apache.spark.sql.catalyst.parser.ParseException + +extraneous input 'day' expecting (line 1, pos 27) + +== SQL == +select interval 30 day day day +---------------------------^^^ + + +-- !query 93 +select sum(cast(null as interval)) +-- !query 93 schema +struct +-- !query 93 output +NULL + + +-- !query 94 +select sum(cast(v as interval)) from VALUES ('1 seconds') t(v) where 1=0 +-- !query 94 schema +struct +-- !query 94 output +NULL + + +-- !query 95 +select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) +-- !query 95 schema +struct +-- !query 95 output +3 seconds + + +-- !query 96 +select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) +-- !query 96 schema +struct +-- !query 96 output 1 seconds + + +-- !query 97 +select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) +-- !query 97 schema +struct +-- !query 97 output +-3 seconds + + +-- !query 98 +select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) +-- !query 98 schema +struct +-- !query 98 output +-7 days 2 seconds + + +-- !query 99 +select + i, + sum(cast(v as interval)) +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +group by i +-- !query 99 schema +struct +-- !query 99 output +1 -2 days +2 2 seconds +3 NULL + + +-- !query 100 +select + sum(cast(v as interval)) as sv +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +having sv is not null +-- !query 100 schema +struct +-- !query 100 output +-2 days 2 seconds + + +-- !query 101 +SELECT + i, + sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +FROM VALUES(1, '1 seconds'), (1, '2 seconds'), (2, NULL), (2, NULL) t(i,v) +-- !query 101 schema +struct +-- !query 101 output +1 2 seconds +1 3 seconds +2 NULL +2 NULL + + +-- !query 102 +select avg(cast(v as interval)) from VALUES (null) t(v) +-- !query 102 schema +struct +-- !query 102 output +NULL + + +-- !query 103 +select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0 +-- !query 103 schema +struct +-- !query 103 output +NULL + + +-- !query 104 +select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) +-- !query 104 schema +struct +-- !query 104 output +1.5 seconds + + +-- !query 105 +select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) +-- !query 105 schema +struct +-- !query 105 output +0.5 seconds + + +-- !query 106 +select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) +-- !query 106 schema +struct +-- !query 106 output +-1.5 seconds + + +-- !query 107 +select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) +-- !query 107 schema +struct +-- !query 107 output +-3 days -11 hours -59 minutes -59 seconds + + +-- !query 108 +select + i, + avg(cast(v as interval)) +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +group by i +-- !query 108 schema +struct +-- !query 108 output +1 -1 days +2 2 seconds +3 NULL + + +-- !query 109 +select + avg(cast(v as interval)) as sv +from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) +having sv is not null +-- !query 109 schema +struct +-- !query 109 output +-15 hours -59 minutes -59.333333 seconds + + +-- !query 110 +SELECT + i, + avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v) +-- !query 110 schema +struct +-- !query 110 output +1 1.5 seconds +1 2 seconds +2 NULL +2 NULL + + +-- !query 111 +create temporary view interval_arithmetic as + select CAST(dateval AS date), CAST(tsval AS timestamp) from values + ('2012-01-01', '2012-01-01') + as interval_arithmetic(dateval, tsval) +-- !query 111 schema +struct<> +-- !query 111 output + + + +-- !query 112 +select + dateval, + dateval - interval '2-2' year to month, + dateval - interval '-2-2' year to month, + dateval + interval '2-2' year to month, + dateval + interval '-2-2' year to month, + - interval '2-2' year to month + dateval, + interval '2-2' year to month + dateval +from interval_arithmetic +-- !query 112 schema +struct +-- !query 112 output +2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 + + +-- !query 113 +select + tsval, + tsval - interval '2-2' year to month, + tsval - interval '-2-2' year to month, + tsval + interval '2-2' year to month, + tsval + interval '-2-2' year to month, + - interval '2-2' year to month + tsval, + interval '2-2' year to month + tsval +from interval_arithmetic +-- !query 113 schema +struct +-- !query 113 output +2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 + + +-- !query 114 +select + interval '2-2' year to month + interval '3-3' year to month, + interval '2-2' year to month - interval '3-3' year to month +from interval_arithmetic +-- !query 114 schema +struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> +-- !query 114 output +5 years 5 months -1 years -1 months + + +-- !query 115 +select + dateval, + dateval - interval '99 11:22:33.123456789' day to second, + dateval - interval '-99 11:22:33.123456789' day to second, + dateval + interval '99 11:22:33.123456789' day to second, + dateval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + dateval, + interval '99 11:22:33.123456789' day to second + dateval +from interval_arithmetic +-- !query 115 schema +struct +-- !query 115 output +2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 + + +-- !query 116 +select + tsval, + tsval - interval '99 11:22:33.123456789' day to second, + tsval - interval '-99 11:22:33.123456789' day to second, + tsval + interval '99 11:22:33.123456789' day to second, + tsval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + tsval, + interval '99 11:22:33.123456789' day to second + tsval +from interval_arithmetic +-- !query 116 schema +struct +-- !query 116 output +2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 + + +-- !query 117 +select + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second +from interval_arithmetic +-- !query 117 schema +struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> +-- !query 117 output +109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds From 65a189c7a1ddceb8ab482ccc60af5350b8da5ea5 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Fri, 15 Nov 2019 13:27:30 +0900 Subject: [PATCH 0620/1387] [SPARK-29376][SQL][PYTHON] Upgrade Apache Arrow to version 0.15.1 ### What changes were proposed in this pull request? Upgrade Apache Arrow to version 0.15.1. This includes Java artifacts and increases the minimum required version of PyArrow also. Version 0.12.0 to 0.15.1 includes the following selected fixes/improvements relevant to Spark users: * ARROW-6898 - [Java] Fix potential memory leak in ArrowWriter and several test classes * ARROW-6874 - [Python] Memory leak in Table.to_pandas() when conversion to object dtype * ARROW-5579 - [Java] shade flatbuffer dependency * ARROW-5843 - [Java] Improve the readability and performance of BitVectorHelper#getNullCount * ARROW-5881 - [Java] Provide functionalities to efficiently determine if a validity buffer has completely 1 bits/0 bits * ARROW-5893 - [C++] Remove arrow::Column class from C++ library * ARROW-5970 - [Java] Provide pointer to Arrow buffer * ARROW-6070 - [Java] Avoid creating new schema before IPC sending * ARROW-6279 - [Python] Add Table.slice method or allow slices in \_\_getitem\_\_ * ARROW-6313 - [Format] Tracking for ensuring flatbuffer serialized values are aligned in stream/files. * ARROW-6557 - [Python] Always return pandas.Series from Array/ChunkedArray.to_pandas, propagate field names to Series from RecordBatch, Table * ARROW-2015 - [Java] Use Java Time and Date APIs instead of JodaTime * ARROW-1261 - [Java] Add container type for Map logical type * ARROW-1207 - [C++] Implement Map logical type Changelog can be seen at https://arrow.apache.org/release/0.15.0.html ### Why are the changes needed? Upgrade to get bug fixes, improvements, and maintain compatibility with future versions of PyArrow. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests, manually tested with Python 3.7, 3.8 Closes #26133 from BryanCutler/arrow-upgrade-015-SPARK-29376. Authored-by: Bryan Cutler Signed-off-by: HyukjinKwon --- dev/deps/spark-deps-hadoop-2.7 | 7 +++---- dev/deps/spark-deps-hadoop-3.2 | 7 +++---- pom.xml | 4 ++-- python/pyspark/sql/utils.py | 6 +++++- python/setup.py | 2 +- .../spark/sql/execution/arrow/ArrowConverters.scala | 8 ++++---- 6 files changed, 18 insertions(+), 16 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index e6d29d04acbf3..54608d203133c 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -17,9 +17,9 @@ apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar -arrow-format-0.12.0.jar -arrow-memory-0.12.0.jar -arrow-vector-0.12.0.jar +arrow-format-0.15.1.jar +arrow-memory-0.15.1.jar +arrow-vector-0.15.1.jar audience-annotations-0.5.0.jar automaton-1.11-8.jar avro-1.8.2.jar @@ -83,7 +83,6 @@ hadoop-yarn-server-web-proxy-2.7.4.jar hk2-api-2.5.0.jar hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar -hppc-0.7.2.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.10.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 8f1e7fe125b9f..917fde61fad1a 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -12,9 +12,9 @@ antlr4-runtime-4.7.1.jar aopalliance-1.0.jar aopalliance-repackaged-2.5.0.jar arpack_combined_all-0.1.jar -arrow-format-0.12.0.jar -arrow-memory-0.12.0.jar -arrow-vector-0.12.0.jar +arrow-format-0.15.1.jar +arrow-memory-0.15.1.jar +arrow-vector-0.15.1.jar audience-annotations-0.5.0.jar automaton-1.11-8.jar avro-1.8.2.jar @@ -96,7 +96,6 @@ hive-vector-code-gen-2.3.6.jar hk2-api-2.5.0.jar hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar -hppc-0.7.2.jar htrace-core4-4.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.10.jar diff --git a/pom.xml b/pom.xml index 5110285547ab3..a6a82b3339d08 100644 --- a/pom.xml +++ b/pom.xml @@ -200,9 +200,9 @@ 1.0.0 - 0.12.0 + 0.15.1 ${java.home} diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 83afafdd8b138..4260c06f06060 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -160,9 +160,10 @@ def require_minimum_pyarrow_version(): """ Raise ImportError if minimum version of pyarrow is not installed """ # TODO(HyukjinKwon): Relocate and deduplicate the version specification. - minimum_pyarrow_version = "0.12.1" + minimum_pyarrow_version = "0.15.1" from distutils.version import LooseVersion + import os try: import pyarrow have_arrow = True @@ -174,6 +175,9 @@ def require_minimum_pyarrow_version(): if LooseVersion(pyarrow.__version__) < LooseVersion(minimum_pyarrow_version): raise ImportError("PyArrow >= %s must be installed; however, " "your version was %s." % (minimum_pyarrow_version, pyarrow.__version__)) + if os.environ.get("ARROW_PRE_0_15_IPC_FORMAT", "0") == "1": + raise RuntimeError("Arrow legacy IPC format is not supported in PySpark, " + "please unset ARROW_PRE_0_15_IPC_FORMAT") def require_test_compiled(): diff --git a/python/setup.py b/python/setup.py index 092bdd3f90117..138161ff13b41 100755 --- a/python/setup.py +++ b/python/setup.py @@ -105,7 +105,7 @@ def _supports_symlinks(): # For Arrow, you should also check ./pom.xml and ensure there are no breaking changes in the # binary format protocol with the Java version, see ARROW_HOME/format/* for specifications. _minimum_pandas_version = "0.23.2" -_minimum_pyarrow_version = "0.12.1" +_minimum_pyarrow_version = "0.15.1" try: # We copy the shell script to be under pyspark/python/pyspark so that the launcher scripts diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index 1a6f4acb63521..d1076d9d0156c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -26,7 +26,7 @@ import org.apache.arrow.flatbuf.MessageHeader import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector._ import org.apache.arrow.vector.ipc.{ArrowStreamWriter, ReadChannel, WriteChannel} -import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, MessageSerializer} +import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, IpcOption, MessageSerializer} import org.apache.spark.TaskContext import org.apache.spark.api.java.JavaRDD @@ -64,7 +64,7 @@ private[sql] class ArrowBatchStreamWriter( * End the Arrow stream, does not close output stream. */ def end(): Unit = { - ArrowStreamWriter.writeEndOfStream(writeChannel) + ArrowStreamWriter.writeEndOfStream(writeChannel, new IpcOption) } } @@ -251,8 +251,8 @@ private[sql] object ArrowConverters { // Only care about RecordBatch messages, skip Schema and unsupported Dictionary messages if (msgMetadata.getMessage.headerType() == MessageHeader.RecordBatch) { - // Buffer backed output large enough to hold the complete serialized message - val bbout = new ByteBufferOutputStream(4 + msgMetadata.getMessageLength + bodyLength) + // Buffer backed output large enough to hold 8-byte length + complete serialized message + val bbout = new ByteBufferOutputStream(8 + msgMetadata.getMessageLength + bodyLength) // Write message metadata to ByteBuffer output stream MessageSerializer.writeMessageBuffer( From d1ac25ba33174afaabc51893962f94597d3b1b22 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 15 Nov 2019 13:44:20 +0900 Subject: [PATCH 0621/1387] [SPARK-28752][BUILD][DOCS] Documentation build to support Python 3 ### What changes were proposed in this pull request? This PR proposes to switch `pygments.rb`, which only support Python 2 and seems inactive for the last few years (https://github.com/tmm1/pygments.rb), to Rouge which is pure Ruby code highlighter that is compatible with Pygments. I thought it would be pretty difficult to change but thankfully Rouge does a great job as the alternative. ### Why are the changes needed? We're moving to Python 3 and drop Python 2 completely. ### Does this PR introduce any user-facing change? Maybe a little bit of different syntax style but should not have a notable change. ### How was this patch tested? Manually tested the build and checked the documentation. Closes #26521 from HyukjinKwon/SPARK-28752. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/create-release/spark-rm/Dockerfile | 4 ++-- docs/README.md | 3 +-- docs/_config.yml | 2 +- docs/_plugins/include_example.rb | 6 ++++-- docs/css/pygments-default.css | 6 +++++- 5 files changed, 13 insertions(+), 8 deletions(-) diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 4bfecedbf0406..fde75495b4428 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -34,7 +34,7 @@ ENV DEBCONF_NONINTERACTIVE_SEEN true ARG APT_INSTALL="apt-get install --no-install-recommends -y" ARG BASE_PIP_PKGS="setuptools wheel virtualenv" -ARG PIP_PKGS="pyopenssl pypandoc numpy pygments sphinx" +ARG PIP_PKGS="pyopenssl pypandoc numpy sphinx" # Install extra needed repos and refresh. # - CRAN repo @@ -80,7 +80,7 @@ RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ $APT_INSTALL ruby2.3 ruby2.3-dev mkdocs && \ gem install jekyll --no-rdoc --no-ri -v 3.8.6 && \ gem install jekyll-redirect-from && \ - gem install pygments.rb + gem install rogue WORKDIR /opt/spark-rm/output diff --git a/docs/README.md b/docs/README.md index da531321aa5da..0bb1ada7167df 100644 --- a/docs/README.md +++ b/docs/README.md @@ -36,8 +36,7 @@ You need to have [Ruby](https://www.ruby-lang.org/en/documentation/installation/ installed. Also install the following libraries: ```sh -$ sudo gem install jekyll jekyll-redirect-from pygments.rb -$ sudo pip install Pygments +$ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' diff --git a/docs/_config.yml b/docs/_config.yml index 57b8d716ee55c..a888620139207 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,4 +1,4 @@ -highlighter: pygments +highlighter: rouge markdown: kramdown gems: - jekyll-redirect-from diff --git a/docs/_plugins/include_example.rb b/docs/_plugins/include_example.rb index 1e91f12518e0b..8a3af36e92305 100644 --- a/docs/_plugins/include_example.rb +++ b/docs/_plugins/include_example.rb @@ -16,7 +16,7 @@ # require 'liquid' -require 'pygments' +require 'rouge' module Jekyll class IncludeExampleTag < Liquid::Tag @@ -56,7 +56,9 @@ def render(context) end code = select_lines(code) - rendered_code = Pygments.highlight(code, :lexer => @lang) + formatter = Rouge::Formatters::HTML.new + lexer = Rouge::Lexer.find(@lang) + rendered_code = formatter.format(lexer.lex(code)) hint = "
    Find full example code at " \ "\"examples/src/main/#{snippet_file}\" in the Spark repo.
    " diff --git a/docs/css/pygments-default.css b/docs/css/pygments-default.css index a4d583b366603..0d62ad1a018aa 100644 --- a/docs/css/pygments-default.css +++ b/docs/css/pygments-default.css @@ -11,6 +11,10 @@ Also, I was thrown off for a while at first when I was using markdown code block inside my {% highlight scala %} ... {% endhighlight %} tags (I was using 4 spaces for this), when it turns out that pygments will insert the code (or pre?) tags for you. + +Note that due to Python 3 compatibility in the project, now we use +Rouge which claims Pygments compatibility, instead of pygments.rb which +does not support Python 3. See SPARK-28752. */ .hll { background-color: #ffffcc } @@ -73,4 +77,4 @@ insert the code (or pre?) tags for you. .vc { color: #bb60d5 } /* Name.Variable.Class */ .vg { color: #bb60d5 } /* Name.Variable.Global */ .vi { color: #bb60d5 } /* Name.Variable.Instance */ -.il { color: #40a070 } /* Literal.Number.Integer.Long */ \ No newline at end of file +.il { color: #40a070 } /* Literal.Number.Integer.Long */ From 0c68578fa9d03002a2d1708762c3010fd28430c5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 15 Nov 2019 13:33:30 +0800 Subject: [PATCH 0622/1387] [SPARK-29888][SQL] new interval string parser shall handle numeric with only fractional part ### What changes were proposed in this pull request? Current string to interval cast logic does not support i.e. cast('.111 second' as interval) which will fail in SIGN state and return null, actually, it is 00:00:00.111. ```scala -- !query 63 select interval '.111 seconds' -- !query 63 schema struct<0.111 seconds:interval> -- !query 63 output 0.111 seconds -- !query 64 select cast('.111 seconds' as interval) -- !query 64 schema struct -- !query 64 output NULL ```` ### Why are the changes needed? bug fix. ### Does this PR introduce any user-facing change? no ### How was this patch tested? add ut Closes #26514 from yaooqinn/SPARK-29888. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/IntervalUtils.scala | 22 ++++++++++++++----- .../catalyst/util/IntervalUtilsSuite.scala | 1 + 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 882c1d85267e4..991312bff30aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -496,6 +496,17 @@ object IntervalUtils { state = TRIM_BEFORE_SIGN case TRIM_BEFORE_SIGN => trimToNextState(b, SIGN) case SIGN => + currentValue = 0 + fraction = 0 + // We preset next state from SIGN to TRIM_BEFORE_VALUE. If we meet '.' in the SIGN state, + // it means that the interval value we deal with here is a numeric with only fractional + // part, such as '.11 second', which can be parsed to 0.11 seconds. In this case, we need + // to reset next state to `VALUE_FRACTIONAL_PART` to go parse the fraction part of the + // interval value. + state = TRIM_BEFORE_VALUE + // We preset the scale to an invalid value to track fraction presence in the UNIT_BEGIN + // state. If we meet '.', the scale become valid for the VALUE_FRACTIONAL_PART state. + fractionScale = -1 b match { case '-' => isNegative = true @@ -505,14 +516,13 @@ object IntervalUtils { i += 1 case _ if '0' <= b && b <= '9' => isNegative = false + case '.' => + isNegative = false + fractionScale = (NANOS_PER_SECOND / 10).toInt + i += 1 + state = VALUE_FRACTIONAL_PART case _ => return null } - currentValue = 0 - fraction = 0 - // Sets the scale to an invalid value to track fraction presence - // in the BEGIN_UNIT_NAME state - fractionScale = -1 - state = TRIM_BEFORE_VALUE case TRIM_BEFORE_VALUE => trimToNextState(b, VALUE) case VALUE => b match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 8c84eb107cd30..f919bd1644871 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -106,6 +106,7 @@ class IntervalUtilsSuite extends SparkFunSuite { checkFromString("-1.5 seconds", new CalendarInterval(0, 0, -1500000)) // truncate nanoseconds to microseconds checkFromString("0.999999999 seconds", new CalendarInterval(0, 0, 999999)) + checkFromString(".999999999 seconds", new CalendarInterval(0, 0, 999999)) checkFromInvalidString("0.123456789123 seconds", "Error parsing interval string") } From 4f10e54ba385daa37598efa49dbfb536a7726dbc Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 15 Nov 2019 15:49:24 +0800 Subject: [PATCH 0623/1387] [SPARK-29655][SQL] Read bucketed tables obeys spark.sql.shuffle.partitions ### What changes were proposed in this pull request? In order to avoid frequently changing the value of `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions`, we usually set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` much larger than `spark.sql.shuffle.partitions` after enabling adaptive execution, which causes some bucket map join lose efficacy and add more `ShuffleExchange`. How to reproduce: ```scala val bucketedTableName = "bucketed_table" spark.range(10000).write.bucketBy(500, "id").sortBy("id").mode(org.apache.spark.sql.SaveMode.Overwrite).saveAsTable(bucketedTableName) val bucketedTable = spark.table(bucketedTableName) val df = spark.range(8) spark.conf.set("spark.sql.autoBroadcastJoinThreshold", -1) // Spark 2.4. spark.sql.adaptive.enabled=false // We set spark.sql.shuffle.partitions <= 500 every time based on our data in this case. spark.conf.set("spark.sql.shuffle.partitions", 500) bucketedTable.join(df, "id").explain() // Since 3.0. We enabled adaptive execution and set spark.sql.adaptive.shuffle.maxNumPostShufflePartitions to a larger values to fit more cases. spark.conf.set("spark.sql.adaptive.enabled", true) spark.conf.set("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions", 1000) bucketedTable.join(df, "id").explain() ``` ``` scala> bucketedTable.join(df, "id").explain() == Physical Plan == *(4) Project [id#5L] +- *(4) SortMergeJoin [id#5L], [id#7L], Inner :- *(1) Sort [id#5L ASC NULLS FIRST], false, 0 : +- *(1) Project [id#5L] : +- *(1) Filter isnotnull(id#5L) : +- *(1) ColumnarToRow : +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct, SelectedBucketsCount: 500 out of 500 +- *(3) Sort [id#7L ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(id#7L, 500), true, [id=#49] +- *(2) Range (0, 8, step=1, splits=16) ``` vs ``` scala> bucketedTable.join(df, "id").explain() == Physical Plan == AdaptiveSparkPlan(isFinalPlan=false) +- Project [id#5L] +- SortMergeJoin [id#5L], [id#7L], Inner :- Sort [id#5L ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(id#5L, 1000), true, [id=#93] : +- Project [id#5L] : +- Filter isnotnull(id#5L) : +- FileScan parquet default.bucketed_table[id#5L] Batched: true, DataFilters: [isnotnull(id#5L)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/apache-spark/spark-3.0.0-SNAPSHOT-bin-3.2.0/spark-warehou..., PartitionFilters: [], PushedFilters: [IsNotNull(id)], ReadSchema: struct, SelectedBucketsCount: 500 out of 500 +- Sort [id#7L ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(id#7L, 1000), true, [id=#92] +- Range (0, 8, step=1, splits=16) ``` This PR makes read bucketed tables always obeys `spark.sql.shuffle.partitions` even enabling adaptive execution and set `spark.sql.adaptive.shuffle.maxNumPostShufflePartitions` to avoid add more `ShuffleExchange`. ### Why are the changes needed? Do not degrade performance after enabling adaptive execution. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test. Closes #26409 from wangyum/SPARK-29655. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan --- .../exchange/EnsureRequirements.scala | 19 +++++++++++- .../ReduceNumShufflePartitionsSuite.scala | 3 +- .../spark/sql/sources/BucketedReadSuite.scala | 31 +++++++++++++++++-- 3 files changed, 49 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index c56a5c015f32d..866b382a1d808 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -83,7 +83,24 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { numPartitionsSet.headOption } - val targetNumPartitions = requiredNumPartitions.getOrElse(childrenNumPartitions.max) + // If there are non-shuffle children that satisfy the required distribution, we have + // some tradeoffs when picking the expected number of shuffle partitions: + // 1. We should avoid shuffling these children. + // 2. We should have a reasonable parallelism. + val nonShuffleChildrenNumPartitions = + childrenIndexes.map(children).filterNot(_.isInstanceOf[ShuffleExchangeExec]) + .map(_.outputPartitioning.numPartitions) + val expectedChildrenNumPartitions = if (nonShuffleChildrenNumPartitions.nonEmpty) { + // Here we pick the max number of partitions among these non-shuffle children as the + // expected number of shuffle partitions. However, if it's smaller than + // `conf.numShufflePartitions`, we pick `conf.numShufflePartitions` as the + // expected number of shuffle partitions. + math.max(nonShuffleChildrenNumPartitions.max, conf.numShufflePartitions) + } else { + childrenNumPartitions.max + } + + val targetNumPartitions = requiredNumPartitions.getOrElse(expectedChildrenNumPartitions) children = children.zip(requiredChildDistributions).zipWithIndex.map { case ((child, distribution), index) if childrenIndexes.contains(index) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 4d408cd8ebd70..21ec1ac9bda08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -274,6 +274,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA .setMaster("local[*]") .setAppName("test") .set(UI_ENABLED, false) + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") @@ -507,7 +508,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA join, expectedAnswer.collect()) - // Then, let's make sure we do not reduce number of ppst shuffle partitions. + // Then, let's make sure we do not reduce number of post shuffle partitions. val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan val shuffleReaders = finalPlan.collect { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 7043b6d396977..a585f215ad681 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.execution.{DataSourceScanExec, SortExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.BucketingUtils import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec @@ -382,8 +383,16 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { joined.sort("bucketed_table1.k", "bucketed_table2.k"), df1.join(df2, joinCondition(df1, df2), joinType).sort("df1.k", "df2.k")) - assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoinExec]) - val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoinExec] + val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { + val executedPlan = + joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + assert(executedPlan.isInstanceOf[SortMergeJoinExec]) + executedPlan.asInstanceOf[SortMergeJoinExec] + } else { + val executedPlan = joined.queryExecution.executedPlan + assert(executedPlan.isInstanceOf[SortMergeJoinExec]) + executedPlan.asInstanceOf[SortMergeJoinExec] + } // check existence of shuffle assert( @@ -795,4 +804,22 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } + test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key -> "7") { + val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) + Seq(false, true).foreach { enableAdaptive => + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> s"$enableAdaptive") { + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec(None, expectedShuffle = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + } + } + } } From ee4784bf26421b830f02c6d65faae4e546c622a8 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 15 Nov 2019 08:12:41 -0600 Subject: [PATCH 0624/1387] [SPARK-26499][SQL][FOLLOW-UP] Replace `update` with `setByte` for ByteType in JdbcUtils.makeGetter ### What changes were proposed in this pull request? This is a follow-up pr to fix the code coming from #23400; it replaces `update` with `setByte` for ByteType in `JdbcUtils.makeGetter`. ### Why are the changes needed? For better code. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26532 from maropu/SPARK-26499-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Sean Owen --- .../apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index d4c7f005a16df..e3baa2a028da2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -445,7 +445,7 @@ object JdbcUtils extends Logging { case ByteType => (rs: ResultSet, row: InternalRow, pos: Int) => - row.update(pos, rs.getByte(pos + 1)) + row.setByte(pos, rs.getByte(pos + 1)) case StringType => (rs: ResultSet, row: InternalRow, pos: Int) => From 15218898cdc540420d6a6c957e5040f78e75cc61 Mon Sep 17 00:00:00 2001 From: shahid Date: Fri, 15 Nov 2019 08:20:10 -0600 Subject: [PATCH 0625/1387] [SPARK-29902][DOC][MINOR] Add listener event queue capacity configuration to documentation ### What changes were proposed in this pull request? Add listener event queue capacity configuration to documentation ### Why are the changes needed? We some time see many event drops happening in eventLog listener queue. So, instead of increasing all the queues size, using this config we just need to increase eventLog queue capacity. ``` scala> sc.parallelize(1 to 100000, 100000).count() [Stage 0:=================================================>(98299 + 4) / 100000]19/11/14 20:56:35 ERROR AsyncEventQueue: Dropping event from queue eventLog. This likely means one of the listeners is too slow and cannot keep up with the rate at which tasks are being started by the scheduler. 19/11/14 20:56:35 WARN AsyncEventQueue: Dropped 1 events from eventLog since the application started. ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests Closes #26529 from shahidki31/master1. Authored-by: shahid Signed-off-by: Sean Owen --- docs/configuration.md | 45 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 97ea1fb4ba041..0c7cc6022eb09 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1857,6 +1857,51 @@ Apart from these, the following properties are also available, and may be useful driver using more memory. + + spark.scheduler.listenerbus.eventqueue.shared.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for shared event queue in Spark listener bus, which hold events for external listener(s) + that register to the listener bus. Consider increasing value, if the listener events corresponding + to shared queue are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.appStatus.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for appStatus event queue, which hold events for internal application status listeners. + Consider increasing value, if the listener events corresponding to appStatus queue are dropped. + Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.executorManagement.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for executorManagement event queue in Spark listener bus, which hold events for internal + executor management listeners. Consider increasing value if the listener events corresponding to + executorManagement queue are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.eventLog.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for eventLog queue in Spark listener bus, which hold events for Event logging listeners + that write events to eventLogs. Consider increasing value if the listener events corresponding to eventLog queue + are dropped. Increasing this value may result in the driver using more memory. + + + + spark.scheduler.listenerbus.eventqueue.streams.capacity + spark.scheduler.listenerbus.eventqueue.capacity + + Capacity for streams queue in Spark listener bus, which hold events for internal streaming listener. + Consider increasing value if the listener events corresponding to streams queue are dropped. Increasing + this value may result in the driver using more memory. + + spark.scheduler.blacklist.unschedulableTaskSetTimeout 120s From 848bdfa218c5da55d7fbc0cf82866e6b1c4a09e6 Mon Sep 17 00:00:00 2001 From: Pablo Langa Date: Fri, 15 Nov 2019 14:25:33 -0800 Subject: [PATCH 0626/1387] [SPARK-29829][SQL] SHOW TABLE EXTENDED should do multi-catalog resolution MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add ShowTableStatement and make SHOW TABLE EXTENDED go through the same catalog/table resolution framework of v2 commands. We don’t have this methods in the catalog to implement an V2 command - catalog.getPartition - catalog.getTempViewOrPermanentTableMetadata ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing ```sql USE my_catalog DESC t // success and describe the table t from my_catalog SHOW TABLE EXTENDED FROM LIKE 't' // report table not found as there is no table t in the session catalog ``` ### Does this PR introduce any user-facing change? Yes. When running SHOW TABLE EXTENDED Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog. ### How was this patch tested? Unit tests. Closes #26540 from planga82/feature/SPARK-29481_ShowTableExtended. Authored-by: Pablo Langa Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 10 +++++ .../catalyst/plans/logical/statements.scala | 9 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 25 +++++++++++ .../analysis/ResolveSessionCatalog.scala | 9 ++++ .../spark/sql/execution/SparkSqlParser.scala | 17 ------- .../sql/connector/DataSourceV2SQLSuite.scala | 44 +++++++++++++++++++ 7 files changed, 98 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 49fba6b7f35df..7ca33a8ecc0c4 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -187,7 +187,7 @@ statement statement #explain | SHOW TABLES ((FROM | IN) multipartIdentifier)? (LIKE? pattern=STRING)? #showTables - | SHOW TABLE EXTENDED ((FROM | IN) db=errorCapturingIdentifier)? + | SHOW TABLE EXTENDED ((FROM | IN) namespace=multipartIdentifier)? LIKE pattern=STRING partitionSpec? #showTable | SHOW TBLPROPERTIES table=multipartIdentifier ('(' key=tablePropertyKey ')')? #showTblProperties diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c623b5c4d36a5..56025ccbf0cd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2719,6 +2719,16 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.pattern).map(string)) } + /** + * Create a [[ShowTableStatement]] command. + */ + override def visitShowTable(ctx: ShowTableContext): LogicalPlan = withOrigin(ctx) { + ShowTableStatement( + Option(ctx.namespace).map(visitMultipartIdentifier), + string(ctx.pattern), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) + } + /** * Parse new column info from ADD COLUMN into a QualifiedColType. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index ec373d95fad88..03d02a2b13cd7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -313,6 +313,15 @@ case class InsertIntoStatement( case class ShowTablesStatement(namespace: Option[Seq[String]], pattern: Option[String]) extends ParsedStatement +/** + * A SHOW TABLE EXTENDED statement, as parsed from SQL. + */ +case class ShowTableStatement( + namespace: Option[Seq[String]], + pattern: String, + partitionSpec: Option[TablePartitionSpec]) + extends ParsedStatement + /** * A CREATE NAMESPACE statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 94171feba2ac7..5fa25300793a4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1022,6 +1022,31 @@ class DDLParserSuite extends AnalysisTest { ShowTablesStatement(Some(Seq("tbl")), Some("*dog*"))) } + test("show table extended") { + comparePlans( + parsePlan("SHOW TABLE EXTENDED LIKE '*test*'"), + ShowTableStatement(None, "*test*", None)) + comparePlans( + parsePlan("SHOW TABLE EXTENDED FROM testcat.ns1.ns2 LIKE '*test*'"), + ShowTableStatement(Some(Seq("testcat", "ns1", "ns2")), "*test*", None)) + comparePlans( + parsePlan("SHOW TABLE EXTENDED IN testcat.ns1.ns2 LIKE '*test*'"), + ShowTableStatement(Some(Seq("testcat", "ns1", "ns2")), "*test*", None)) + comparePlans( + parsePlan("SHOW TABLE EXTENDED LIKE '*test*' PARTITION(ds='2008-04-09', hr=11)"), + ShowTableStatement(None, "*test*", Some(Map("ds" -> "2008-04-09", "hr" -> "11")))) + comparePlans( + parsePlan("SHOW TABLE EXTENDED FROM testcat.ns1.ns2 LIKE '*test*' " + + "PARTITION(ds='2008-04-09')"), + ShowTableStatement(Some(Seq("testcat", "ns1", "ns2")), "*test*", + Some(Map("ds" -> "2008-04-09")))) + comparePlans( + parsePlan("SHOW TABLE EXTENDED IN testcat.ns1.ns2 LIKE '*test*' " + + "PARTITION(ds='2008-04-09')"), + ShowTableStatement(Some(Seq("testcat", "ns1", "ns2")), "*test*", + Some(Map("ds" -> "2008-04-09")))) + } + test("create namespace -- backward compatibility with DATABASE/DBPROPERTIES") { val expected = CreateNamespaceStatement( Seq("a", "b", "c"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 340e09ae66adb..7adcd7836e952 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -301,6 +301,15 @@ class ResolveSessionCatalog( case ShowTablesStatement(None, pattern) if isSessionCatalog(currentCatalog) => ShowTablesCommand(None, pattern) + case ShowTableStatement(namespace, pattern, partitionsSpec) => + val db = namespace match { + case Some(namespace) if namespace.length != 1 => + throw new AnalysisException( + s"The database name is not valid: ${namespace.quoted}") + case _ => namespace.map(_.head) + } + ShowTablesCommand(db, Some(pattern), true, partitionsSpec) + case AnalyzeTableStatement(tableName, partitionSpec, noScan) => val v1TableName = parseV1Table(tableName, "ANALYZE TABLE") if (partitionSpec.isEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index b1271ad870565..ab5b1126dc536 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -89,23 +89,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ResetCommand } - /** - * Create a [[ShowTablesCommand]] logical plan. - * Example SQL : - * {{{ - * SHOW TABLE EXTENDED [(IN|FROM) database_name] LIKE 'identifier_with_wildcards' - * [PARTITION(partition_spec)]; - * }}} - */ - override def visitShowTable(ctx: ShowTableContext): LogicalPlan = withOrigin(ctx) { - val partitionSpec = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) - ShowTablesCommand( - Option(ctx.db).map(_.getText), - Option(ctx.pattern).map(string), - isExtended = true, - partitionSpec = partitionSpec) - } - /** * Create a [[RefreshResource]] logical plan. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 17f6e51f8454c..19b2ad601328c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -766,6 +766,50 @@ class DataSourceV2SQLSuite assert(expected === df.collect()) } + test("SHOW TABLE EXTENDED not valid v1 database") { + def testV1CommandNamespace(sqlCommand: String, namespace: String): Unit = { + val e = intercept[AnalysisException] { + sql(sqlCommand) + } + assert(e.message.contains(s"The database name is not valid: ${namespace}")) + } + + val namespace = "testcat.ns1.ns2" + val table = "tbl" + withTable(s"$namespace.$table") { + sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " + + s"USING foo PARTITIONED BY (id)") + + testV1CommandNamespace(s"SHOW TABLE EXTENDED FROM $namespace LIKE 'tb*'", + namespace) + testV1CommandNamespace(s"SHOW TABLE EXTENDED IN $namespace LIKE 'tb*'", + namespace) + testV1CommandNamespace("SHOW TABLE EXTENDED " + + s"FROM $namespace LIKE 'tb*' PARTITION(id=1)", + namespace) + testV1CommandNamespace("SHOW TABLE EXTENDED " + + s"IN $namespace LIKE 'tb*' PARTITION(id=1)", + namespace) + } + } + + test("SHOW TABLE EXTENDED valid v1") { + val expected = Seq(Row("", "source", true), Row("", "source2", true)) + val schema = new StructType() + .add("database", StringType, nullable = false) + .add("tableName", StringType, nullable = false) + .add("isTemporary", BooleanType, nullable = false) + .add("information", StringType, nullable = false) + + val df = sql("SHOW TABLE EXTENDED FROM default LIKE '*source*'") + val result = df.collect() + val resultWithoutInfo = result.map{ case Row(db, table, temp, _) => Row(db, table, temp)} + + assert(df.schema === schema) + assert(resultWithoutInfo === expected) + result.foreach{ case Row(_, _, _, info: String) => assert(info.nonEmpty)} + } + test("CreateNameSpace: basic tests") { // Session catalog is used. withNamespace("ns") { From c0507e0f7530032b26db6ac1611a6a53b9802d5a Mon Sep 17 00:00:00 2001 From: ulysses Date: Fri, 15 Nov 2019 16:17:24 -0800 Subject: [PATCH 0627/1387] [SPARK-29833][YARN] Add FileNotFoundException check for spark.yarn.jars ### What changes were proposed in this pull request? When set `spark.yarn.jars=/xxx/xxx` which is just a no schema path, spark will throw a NullPointerException. The reason is hdfs will return null if pathFs.globStatus(path) is not exist, and spark just use `pathFs.globStatus(path).filter(_.isFile())` without check it. ### Why are the changes needed? Avoid NullPointerException. ### Does this PR introduce any user-facing change? Yes. User will get a FileNotFoundException instead NullPointerException when `spark.yarn.jars` does not have schema and not exists. ### How was this patch tested? Add UT. Closes #26462 from ulysses-you/check-yarn-jars-path-exist. Authored-by: ulysses Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/deploy/yarn/Client.scala | 6 +++++- .../org/apache/spark/deploy/yarn/ClientSuite.scala | 14 +++++++++++++- 2 files changed, 18 insertions(+), 2 deletions(-) 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 a1b5d53b91416..696afaacb0e79 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 @@ -538,7 +538,11 @@ private[spark] class Client( if (!Utils.isLocalUri(jar)) { val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf) val pathFs = FileSystem.get(path.toUri(), hadoopConf) - pathFs.globStatus(path).filter(_.isFile()).foreach { entry => + val fss = pathFs.globStatus(path) + if (fss == null) { + throw new FileNotFoundException(s"Path ${path.toString} does not exist") + } + fss.filter(_.isFile()).foreach { entry => val uri = entry.getPath().toUri() statCache.update(uri, entry) distribute(uri.toString(), targetDir = Some(LOCALIZED_LIB_DIR)) 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 ba7c2dd8a1cdf..7cce908cd5fb7 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 @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import java.io.{File, FileInputStream, FileOutputStream} +import java.io.{File, FileInputStream, FileNotFoundException, FileOutputStream} import java.net.URI import java.util.Properties @@ -473,6 +473,18 @@ class ClientSuite extends SparkFunSuite with Matchers { assert(allResourceInfo.get(yarnMadeupResource).get === 5) } + test("test yarn jars path not exists") { + withTempDir { dir => + val conf = new SparkConf().set(SPARK_JARS, Seq(dir.getAbsolutePath + "/test")) + val client = new Client(new ClientArguments(Array()), conf, null) + withTempDir { distDir => + intercept[FileNotFoundException] { + client.prepareLocalResources(new Path(distDir.getAbsolutePath), Nil) + } + } + } + } + private val matching = Seq( ("files URI match test1", "file:///file1", "file:///file2"), ("files URI match test2", "file:///c:file1", "file://c:file2"), From 7720781695d47fe0375f6e1150f6981b886686bd Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 15 Nov 2019 18:37:33 -0800 Subject: [PATCH 0628/1387] [SPARK-29127][SQL][PYTHON] Add a clue for Python related version information in integrated UDF tests ### What changes were proposed in this pull request? This PR proposes to show Python, pandas and PyArrow versions in integrated UDF tests as a clue so when the test cases fail, it show the related version information. I think we don't really need this kind of version information in the test case name for now since I intend that integrated SQL test cases do not target to test different combinations of Python, Pandas and PyArrow. ### Why are the changes needed? To make debug easier. ### Does this PR introduce any user-facing change? It will change test name to include related Python, pandas and PyArrow versions. ### How was this patch tested? Manually tested: ``` [info] - udf/postgreSQL/udf-case.sql - Scala UDF *** FAILED *** (8 seconds, 229 milliseconds) [info] udf/postgreSQL/udf-case.sql - Scala UDF ... [info] - udf/postgreSQL/udf-case.sql - Regular Python UDF *** FAILED *** (6 seconds, 298 milliseconds) [info] udf/postgreSQL/udf-case.sql - Regular Python UDF [info] Python: 3.7 ... [info] - udf/postgreSQL/udf-case.sql - Scalar Pandas UDF *** FAILED *** (6 seconds, 376 milliseconds) [info] udf/postgreSQL/udf-case.sql - Scalar Pandas UDF [info] Python: 3.7 Pandas: 0.25.3 PyArrow: 0.14.0 ``` Closes #26538 from HyukjinKwon/investigate-flaky-test. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../spark/sql/IntegratedUDFTestUtils.scala | 20 ++++++++++++++++++- .../apache/spark/sql/SQLQueryTestSuite.scala | 16 ++++++++++++++- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index d62fe961117a9..3f6fe6eb62020 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -122,7 +122,7 @@ object IntegratedUDFTestUtils extends SQLHelper { true }.getOrElse(false) - private lazy val pythonVer = if (isPythonAvailable) { + lazy val pythonVer: String = if (isPythonAvailable) { Process( Seq(pythonExec, "-c", "import sys; print('%d.%d' % sys.version_info[:2])"), None, @@ -131,6 +131,24 @@ object IntegratedUDFTestUtils extends SQLHelper { throw new RuntimeException(s"Python executable [$pythonExec] is unavailable.") } + lazy val pandasVer: String = if (isPandasAvailable) { + Process( + Seq(pythonExec, "-c", "import pandas; print(pandas.__version__)"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!!.trim() + } else { + throw new RuntimeException("Pandas is unavailable.") + } + + lazy val pyarrowVer: String = if (isPyArrowAvailable) { + Process( + Seq(pythonExec, "-c", "import pyarrow; print(pyarrow.__version__)"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!!.trim() + } else { + throw new RuntimeException("PyArrow is unavailable.") + } + // Dynamically pickles and reads the Python instance into JVM side in order to mimic // Python native function within Python UDF. private lazy val pythonFunc: Array[Byte] = if (shouldTestPythonUDFs) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 2ce50dd73a13d..ffea03474ac4a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -384,7 +384,21 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // This is a temporary workaround for SPARK-28894. The test names are truncated after // the last dot due to a bug in SBT. This makes easier to debug via Jenkins test result // report. See SPARK-28894. - withClue(s"${testCase.name}${System.lineSeparator()}") { + // See also SPARK-29127. It is difficult to see the version information in the failed test + // cases so the version information related to Python was also added. + val clue = testCase match { + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && shouldTestPythonUDFs => + s"${testCase.name}${System.lineSeparator()}Python: $pythonVer${System.lineSeparator()}" + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && shouldTestScalarPandasUDFs => + s"${testCase.name}${System.lineSeparator()}" + + s"Python: $pythonVer Pandas: $pandasVer PyArrow: $pyarrowVer${System.lineSeparator()}" + case _ => + s"${testCase.name}${System.lineSeparator()}" + } + + withClue(clue) { // Read back the golden file. val expectedOutputs: Seq[QueryOutput] = { val goldenOutput = fileToString(new File(testCase.resultFile)) From 16e7195299d864b9e98ed17a9747d53c6a001024 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Fri, 15 Nov 2019 18:50:42 -0800 Subject: [PATCH 0629/1387] [SPARK-29834][SQL] DESC DATABASE should look up catalog like v2 commands ### What changes were proposed in this pull request? Add DescribeNamespaceStatement, DescribeNamespace and DescribeNamespaceExec to make "DESC DATABASE" look up catalog like v2 commands. ### Why are the changes needed? It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users. ### Does this PR introduce any user-facing change? Yes, add "DESC NAMESPACE" whose function is same as "DESC DATABASE" and "DESC SCHEMA". ### How was this patch tested? New unit test Closes #26513 from fuwhu/SPARK-29834. Authored-by: fuwhu Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 +- .../catalyst/analysis/ResolveCatalogs.scala | 3 + .../sql/catalyst/parser/AstBuilder.scala | 15 +++++ .../catalyst/plans/logical/statements.scala | 7 ++ .../catalyst/plans/logical/v2Commands.scala | 17 ++++- .../sql/catalyst/parser/DDLParserSuite.scala | 7 ++ .../analysis/ResolveSessionCatalog.scala | 7 ++ .../spark/sql/execution/SparkSqlParser.scala | 12 ---- .../datasources/v2/DataSourceV2Strategy.scala | 5 +- .../v2/DescribeNamespaceExec.scala | 66 +++++++++++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 20 +++++- .../execution/command/DDLParserSuite.scala | 19 ------ 12 files changed, 146 insertions(+), 35 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 7ca33a8ecc0c4..8a6bd5ed27360 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -199,7 +199,8 @@ statement | SHOW CREATE TABLE multipartIdentifier #showCreateTable | SHOW CURRENT NAMESPACE #showCurrentNamespace | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction - | (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase + | (DESC | DESCRIBE) (database | NAMESPACE) EXTENDED? + multipartIdentifier #describeNamespace | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? multipartIdentifier partitionSpec? describeColName? #describeTable | (DESC | DESCRIBE) QUERY? query #describeQuery diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index f1a8e5bfda4a9..22caf33b50ce6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -172,6 +172,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case DropNamespaceStatement(NonSessionCatalog(catalog, nameParts), ifExists, cascade) => DropNamespace(catalog, nameParts, ifExists, cascade) + case DescribeNamespaceStatement(NonSessionCatalog(catalog, nameParts), extended) => + DescribeNamespace(catalog, nameParts, extended) + case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 56025ccbf0cd8..73233f934bedc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2541,6 +2541,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.pattern).map(string)) } + /** + * Create a [[DescribeNamespaceStatement]]. + * + * For example: + * {{{ + * DESCRIBE (DATABASE|SCHEMA|NAMESPACE) [EXTENDED] database; + * }}} + */ + override def visitDescribeNamespace(ctx: DescribeNamespaceContext): LogicalPlan = + withOrigin(ctx) { + DescribeNamespaceStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), + ctx.EXTENDED != null) + } + /** * Create a table, returning a [[CreateTableStatement]] logical plan. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 03d02a2b13cd7..903f5cd5912c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -269,6 +269,13 @@ case class DescribeTableStatement( partitionSpec: TablePartitionSpec, isExtended: Boolean) extends ParsedStatement +/** + * A DESCRIBE NAMESPACE statement, as parsed from SQL. + */ +case class DescribeNamespaceStatement( + namespace: Seq[String], + extended: Boolean) extends ParsedStatement + /** * A DESCRIBE TABLE tbl_name col_name statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 7d8e9a0c18f65..3179bab8c2f7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.plans.DescribeTableSchema import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, SupportsNamespaces, TableCatalog, TableChange} import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.types.{DataType, StringType, StructType} +import org.apache.spark.sql.types.{DataType, MetadataBuilder, StringType, StructType} /** * Base trait for DataSourceV2 write commands @@ -255,6 +255,21 @@ case class DropNamespace( ifExists: Boolean, cascade: Boolean) extends Command +/** + * The logical plan of the DESCRIBE NAMESPACE command that works for v2 catalogs. + */ +case class DescribeNamespace( + catalog: CatalogPlugin, + namespace: Seq[String], + extended: Boolean) extends Command { + + override def output: Seq[Attribute] = Seq( + AttributeReference("name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column").build())(), + AttributeReference("value", StringType, nullable = true, + new MetadataBuilder().putString("comment", "value of the column").build())()) +} + /** * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 5fa25300793a4..b998127ebe02a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -653,6 +653,13 @@ class DDLParserSuite extends AnalysisTest { "DESC TABLE COLUMN for a specific partition is not supported")) } + test("describe database") { + val sql1 = "DESCRIBE DATABASE EXTENDED a.b" + val sql2 = "DESCRIBE DATABASE a.b" + comparePlans(parsePlan(sql1), DescribeNamespaceStatement(Seq("a", "b"), extended = true)) + comparePlans(parsePlan(sql2), DescribeNamespaceStatement(Seq("a", "b"), extended = false)) + } + test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 7adcd7836e952..3913f889eb128 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -158,6 +158,13 @@ class ResolveSessionCatalog( case AlterViewUnsetPropertiesStatement(SessionCatalog(catalog, tableName), keys, ifExists) => AlterTableUnsetPropertiesCommand(tableName.asTableIdentifier, keys, ifExists, isView = true) + case d @ DescribeNamespaceStatement(SessionCatalog(_, nameParts), _) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + DescribeDatabaseCommand(nameParts.head, d.extended) + case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index ab5b1126dc536..bfea908d106df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -258,18 +258,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ) } - /** - * Create a [[DescribeDatabaseCommand]] command. - * - * For example: - * {{{ - * DESCRIBE DATABASE [EXTENDED] database; - * }}} - */ - override def visitDescribeDatabase(ctx: DescribeDatabaseContext): LogicalPlan = withOrigin(ctx) { - DescribeDatabaseCommand(ctx.db.getText, ctx.EXTENDED != null) - } - /** * Create a plan for a DESCRIBE FUNCTION command. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 01ff4a9303e98..a04bceb18b9b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} @@ -192,6 +192,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { Nil } + case desc @ DescribeNamespace(catalog, namespace, extended) => + DescribeNamespaceExec(desc.output, catalog, namespace, extended) :: Nil + case desc @ DescribeTable(DataSourceV2Relation(table, _, _), isExtended) => DescribeTableExec(desc.output, table, isExtended) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala new file mode 100644 index 0000000000000..5c20e5ae08383 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.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.execution.datasources.v2 + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.connector.catalog.CatalogPlugin +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.COMMENT_TABLE_PROP +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.LOCATION_TABLE_PROP +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.RESERVED_PROPERTIES +import org.apache.spark.sql.types.StructType + +/** + * Physical plan node for describing a namespace. + */ +case class DescribeNamespaceExec( + output: Seq[Attribute], + catalog: CatalogPlugin, + namespace: Seq[String], + isExtended: Boolean) extends V2CommandExec { + + private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() + + override protected def run(): Seq[InternalRow] = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val rows = new ArrayBuffer[InternalRow]() + val nsCatalog = catalog.asNamespaceCatalog + val ns = namespace.toArray + val metadata = nsCatalog.loadNamespaceMetadata(ns) + + rows += toCatalystRow("Namespace Name", ns.last) + rows += toCatalystRow("Description", metadata.get(COMMENT_TABLE_PROP)) + rows += toCatalystRow("Location", metadata.get(LOCATION_TABLE_PROP)) + if (isExtended) { + val properties = metadata.asScala.toSeq.filter(p => !RESERVED_PROPERTIES.contains(p._1)) + if (properties.nonEmpty) { + rows += toCatalystRow("Properties", properties.mkString("(", ",", ")")) + } + } + rows + } + + private def toCatalystRow(strs: String*): InternalRow = { + encoder.toRow(new GenericRowWithSchema(strs.toArray, schema)).copy() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 19b2ad601328c..095bcff31ccd9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -834,7 +834,6 @@ class DataSourceV2SQLSuite assert(catalogPath.equals(catalogPath)) } } - // TODO: Add tests for validating namespace metadata when DESCRIBE NAMESPACE is available. } test("CreateNameSpace: test handling of 'IF NOT EXIST'") { @@ -915,6 +914,25 @@ class DataSourceV2SQLSuite assert(exception.getMessage.contains("Namespace 'ns1' not found")) } + test("DescribeNamespace using v2 catalog") { + withNamespace("testcat.ns1.ns2") { + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + + "'test namespace' LOCATION '/tmp/ns_test'") + val descriptionDf = sql("DESCRIBE NAMESPACE testcat.ns1.ns2") + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === + Seq( + ("name", StringType), + ("value", StringType) + )) + val description = descriptionDf.collect() + assert(description === Seq( + Row("Namespace Name", "ns2"), + Row("Description", "test namespace"), + Row("Location", "/tmp/ns_test") + )) + } + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index d98f2ca62972c..b2185f8559f36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -108,25 +108,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { comparePlans(parsed1, expected1) } - test("describe database") { - // DESCRIBE DATABASE [EXTENDED] db_name; - val sql1 = "DESCRIBE DATABASE EXTENDED db_name" - val sql2 = "DESCRIBE DATABASE db_name" - - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - - val expected1 = DescribeDatabaseCommand( - "db_name", - extended = true) - val expected2 = DescribeDatabaseCommand( - "db_name", - extended = false) - - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } - test("create function") { val sql1 = """ From 6d6b233791a0a18713a3234f10ba234d92083d68 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 15 Nov 2019 18:54:02 -0800 Subject: [PATCH 0630/1387] [SPARK-29343][SQL][FOLLOW-UP] Remove floating-point Sum/Average/CentralMomentAgg from order-insensitive aggregates ### What changes were proposed in this pull request? This pr is to remove floating-point `Sum/Average/CentralMomentAgg` from order-insensitive aggregates in `EliminateSorts`. This pr comes from the gatorsmile suggestion: https://github.com/apache/spark/pull/26011#discussion_r344583899 ### Why are the changes needed? Bug fix. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added tests in `SubquerySuite`. Closes #26534 from maropu/SPARK-29343-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/Optimizer.scala | 11 +++++------ .../org/apache/spark/sql/SubquerySuite.scala | 17 +++++++++++++++++ 2 files changed, 22 insertions(+), 6 deletions(-) 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 b78bdf082f333..473f846c9313b 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 @@ -1002,12 +1002,11 @@ object EliminateSorts extends Rule[LogicalPlan] { private def isOrderIrrelevantAggs(aggs: Seq[NamedExpression]): Boolean = { def isOrderIrrelevantAggFunction(func: AggregateFunction): Boolean = func match { - case _: Sum => true - case _: Min => true - case _: Max => true - case _: Count => true - case _: Average => true - case _: CentralMomentAgg => true + case _: Min | _: Max | _: Count => true + // Arithmetic operations for floating-point values are order-sensitive + // (they are not associative). + case _: Sum | _: Average | _: CentralMomentAgg => + !Seq(FloatType, DoubleType).exists(_.sameType(func.children.head.dataType)) case _ => false } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index c117ee7818c01..5020c1047f8dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1271,6 +1271,23 @@ class SubquerySuite extends QueryTest with SharedSparkSession { } } + test("Cannot remove sort for floating-point order-sensitive aggregates from subquery") { + Seq("float", "double").foreach { typeName => + Seq("SUM", "AVG", "KURTOSIS", "SKEWNESS", "STDDEV_POP", "STDDEV_SAMP", + "VAR_POP", "VAR_SAMP").foreach { aggName => + val query = + s""" + |SELECT k, $aggName(v) FROM ( + | SELECT k, v + | FROM VALUES (1, $typeName(2.0)), (2, $typeName(1.0)) t(k, v) + | ORDER BY v) + |GROUP BY k + """.stripMargin + assert(getNumSortsInQuery(query) == 1) + } + } + } + test("SPARK-25482: Forbid pushdown to datasources of filters containing subqueries") { withTempView("t1", "t2") { sql("create temporary view t1(a int) using parquet") From 1112fc6029f48918cdd05d0736e02f446262f398 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 15 Nov 2019 21:44:39 -0800 Subject: [PATCH 0631/1387] [SPARK-29867][ML][PYTHON] Add __repr__ in Python ML Models ### What changes were proposed in this pull request? Add ```__repr__``` in Python ML Models ### Why are the changes needed? In Python ML Models, some of them have ```__repr__```, others don't. In the doctest, when calling Model.setXXX, some of the Models print out the xxxModel... correctly, some of them can't because of lacking the ```__repr__``` method. For example: ``` >>> gm = GaussianMixture(k=3, tol=0.0001, seed=10) >>> model = gm.fit(df) >>> model.setPredictionCol("newPrediction") GaussianMixture... ``` After the change, the above code will become the following: ``` >>> gm = GaussianMixture(k=3, tol=0.0001, seed=10) >>> model = gm.fit(df) >>> model.setPredictionCol("newPrediction") GaussianMixtureModel... ``` ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? doctest Closes #26489 from huaxingao/spark-29876. Authored-by: Huaxin Gao Signed-off-by: Dongjoon Hyun --- python/pyspark/ml/classification.py | 11 ++++------- python/pyspark/ml/clustering.py | 8 +++++--- python/pyspark/ml/feature.py | 30 +++++++++++++++++++++++------ python/pyspark/ml/fpm.py | 2 +- python/pyspark/ml/recommendation.py | 2 ++ python/pyspark/ml/regression.py | 10 +++++----- python/pyspark/ml/tree.py | 6 ------ python/pyspark/ml/wrapper.py | 3 +++ 8 files changed, 44 insertions(+), 28 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index f9465bffc9a1a..66b8ffd7bc8d8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -192,11 +192,11 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable 0.01 >>> model = svm.fit(df) >>> model.setPredictionCol("newPrediction") - LinearSVC... + LinearSVCModel... >>> model.getPredictionCol() 'newPrediction' >>> model.setThreshold(0.5) - LinearSVC... + LinearSVCModel... >>> model.getThreshold() 0.5 >>> model.coefficients @@ -812,9 +812,6 @@ def evaluate(self, dataset): java_blr_summary = self._call_java("evaluate", dataset) return BinaryLogisticRegressionSummary(java_blr_summary) - def __repr__(self): - return self._call_java("toString") - class LogisticRegressionSummary(JavaWrapper): """ @@ -1921,7 +1918,7 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, >>> nb = NaiveBayes(smoothing=1.0, modelType="multinomial", weightCol="weight") >>> model = nb.fit(df) >>> model.setFeaturesCol("features") - NaiveBayes_... + NaiveBayesModel... >>> model.getSmoothing() 1.0 >>> model.pi @@ -2114,7 +2111,7 @@ class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, _MultilayerPer 100 >>> model = mlp.fit(df) >>> model.setFeaturesCol("features") - MultilayerPerceptronClassifier... + MultilayerPerceptronClassificationModel... >>> model.layers [2, 2, 2] >>> model.weights.size diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 39cc62670ae88..5aab7a3f5077b 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -234,7 +234,7 @@ class GaussianMixture(JavaEstimator, _GaussianMixtureParams, JavaMLWritable, Jav >>> model.getFeaturesCol() 'features' >>> model.setPredictionCol("newPrediction") - GaussianMixture... + GaussianMixtureModel... >>> model.predict(df.head().features) 2 >>> model.predictProbability(df.head().features) @@ -532,7 +532,7 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): >>> model.getDistanceMeasure() 'euclidean' >>> model.setPredictionCol("newPrediction") - KMeans... + KMeansModel... >>> model.predict(df.head().features) 0 >>> centers = model.clusterCenters() @@ -794,7 +794,7 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav >>> model.getMaxIter() 20 >>> model.setPredictionCol("newPrediction") - BisectingKMeans... + BisectingKMeansModel... >>> model.predict(df.head().features) 0 >>> centers = model.clusterCenters() @@ -1265,6 +1265,8 @@ class LDA(JavaEstimator, _LDAParams, JavaMLReadable, JavaMLWritable): 10 >>> lda.clear(lda.maxIter) >>> model = lda.fit(df) + >>> model.setSeed(1) + DistributedLDAModel... >>> model.getTopicDistributionCol() 'topicDistribution' >>> model.isDistributed() diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index f6e531302317b..e771221d5f06d 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -337,6 +337,8 @@ class BucketedRandomProjectionLSH(_LSH, _BucketedRandomProjectionLSHParams, >>> model = brp.fit(df) >>> model.getBucketLength() 1.0 + >>> model.setOutputCol("hashes") + BucketedRandomProjectionLSHModel... >>> model.transform(df).head() Row(id=0, features=DenseVector([-1.0, -1.0]), hashes=[DenseVector([-1.0])]) >>> data2 = [(4, Vectors.dense([2.0, 2.0 ]),), @@ -733,6 +735,8 @@ class CountVectorizer(JavaEstimator, _CountVectorizerParams, JavaMLReadable, Jav >>> cv.setOutputCol("vectors") CountVectorizer... >>> model = cv.fit(df) + >>> model.setInputCol("raw") + CountVectorizerModel... >>> model.transform(df).show(truncate=False) +-----+---------------+-------------------------+ |label|raw |vectors | @@ -1345,6 +1349,8 @@ class IDF(JavaEstimator, _IDFParams, JavaMLReadable, JavaMLWritable): >>> idf.setOutputCol("idf") IDF... >>> model = idf.fit(df) + >>> model.setOutputCol("idf") + IDFModel... >>> model.getMinDocFreq() 3 >>> model.idf @@ -1519,6 +1525,8 @@ class Imputer(JavaEstimator, _ImputerParams, JavaMLReadable, JavaMLWritable): >>> imputer.getRelativeError() 0.001 >>> model = imputer.fit(df) + >>> model.setInputCols(["a", "b"]) + ImputerModel... >>> model.getStrategy() 'mean' >>> model.surrogateDF.show() @@ -1810,7 +1818,7 @@ class MaxAbsScaler(JavaEstimator, _MaxAbsScalerParams, JavaMLReadable, JavaMLWri MaxAbsScaler... >>> model = maScaler.fit(df) >>> model.setOutputCol("scaledOutput") - MaxAbsScaler... + MaxAbsScalerModel... >>> model.transform(df).show() +-----+------------+ | a|scaledOutput| @@ -1928,6 +1936,8 @@ class MinHashLSH(_LSH, HasInputCol, HasOutputCol, HasSeed, JavaMLReadable, JavaM >>> mh.setSeed(12345) MinHashLSH... >>> model = mh.fit(df) + >>> model.setInputCol("features") + MinHashLSHModel... >>> model.transform(df).head() Row(id=0, features=SparseVector(6, {0: 1.0, 1: 1.0, 2: 1.0}), hashes=[DenseVector([6179668... >>> data2 = [(3, Vectors.sparse(6, [1, 3, 5], [1.0, 1.0, 1.0]),), @@ -2056,7 +2066,7 @@ class MinMaxScaler(JavaEstimator, _MinMaxScalerParams, JavaMLReadable, JavaMLWri MinMaxScaler... >>> model = mmScaler.fit(df) >>> model.setOutputCol("scaledOutput") - MinMaxScaler... + MinMaxScalerModel... >>> model.originalMin DenseVector([0.0]) >>> model.originalMax @@ -2421,6 +2431,8 @@ class OneHotEncoder(JavaEstimator, _OneHotEncoderParams, JavaMLReadable, JavaMLW >>> ohe.setOutputCols(["output"]) OneHotEncoder... >>> model = ohe.fit(df) + >>> model.setOutputCols(["output"]) + OneHotEncoderModel... >>> model.getHandleInvalid() 'error' >>> model.transform(df).head().output @@ -2935,7 +2947,7 @@ class RobustScaler(JavaEstimator, _RobustScalerParams, JavaMLReadable, JavaMLWri RobustScaler... >>> model = scaler.fit(df) >>> model.setOutputCol("output") - RobustScaler... + RobustScalerModel... >>> model.median DenseVector([2.0, -2.0]) >>> model.range @@ -3330,7 +3342,7 @@ class StandardScaler(JavaEstimator, _StandardScalerParams, JavaMLReadable, JavaM >>> model.getInputCol() 'a' >>> model.setOutputCol("output") - StandardScaler... + StandardScalerModel... >>> model.mean DenseVector([1.0]) >>> model.std @@ -3490,6 +3502,8 @@ class StringIndexer(JavaEstimator, _StringIndexerParams, JavaMLReadable, JavaMLW >>> stringIndexer.setHandleInvalid("error") StringIndexer... >>> model = stringIndexer.fit(stringIndDf) + >>> model.setHandleInvalid("error") + StringIndexerModel... >>> td = model.transform(stringIndDf) >>> sorted(set([(i[0], i[1]) for i in td.select(td.id, td.indexed).collect()]), ... key=lambda x: x[0]) @@ -4166,7 +4180,7 @@ class VectorIndexer(JavaEstimator, _VectorIndexerParams, JavaMLReadable, JavaMLW >>> indexer.getHandleInvalid() 'error' >>> model.setOutputCol("output") - VectorIndexer... + VectorIndexerModel... >>> model.transform(df).head().output DenseVector([1.0, 0.0]) >>> model.numFeatures @@ -4487,6 +4501,8 @@ class Word2Vec(JavaEstimator, _Word2VecParams, JavaMLReadable, JavaMLWritable): >>> model = word2Vec.fit(doc) >>> model.getMinCount() 5 + >>> model.setInputCol("sentence") + Word2VecModel... >>> model.getVectors().show() +----+--------------------+ |word| vector| @@ -4714,7 +4730,7 @@ class PCA(JavaEstimator, _PCAParams, JavaMLReadable, JavaMLWritable): >>> model.getK() 2 >>> model.setOutputCol("output") - PCA... + PCAModel... >>> model.transform(df).collect()[0].output DenseVector([1.648..., -4.013...]) >>> model.explainedVariance @@ -5139,6 +5155,8 @@ class ChiSqSelector(JavaEstimator, _ChiSqSelectorParams, JavaMLReadable, JavaMLW >>> model = selector.fit(df) >>> model.getFeaturesCol() 'features' + >>> model.setFeaturesCol("features") + ChiSqSelectorModel... >>> model.transform(df).head().selectedFeatures DenseVector([18.0]) >>> model.selectedFeatures diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index 5b34d555484d1..7d933daf9e032 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -166,7 +166,7 @@ class FPGrowth(JavaEstimator, _FPGrowthParams, JavaMLWritable, JavaMLReadable): >>> fp = FPGrowth(minSupport=0.2, minConfidence=0.7) >>> fpm = fp.fit(data) >>> fpm.setPredictionCol("newPrediction") - FPGrowth... + FPGrowthModel... >>> fpm.freqItemsets.show(5) +---------+----+ | items|freq| diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 3ebd0ac2765f3..ee276962c898b 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -225,6 +225,8 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): >>> model = als.fit(df) >>> model.getUserCol() 'user' + >>> model.setUserCol("user") + ALSModel... >>> model.getItemCol() 'item' >>> model.setPredictionCol("newPrediction") diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 84e39a035d80b..fdb04bb5115c5 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -105,9 +105,9 @@ class LinearRegression(JavaPredictor, _LinearRegressionParams, JavaMLWritable, J LinearRegression... >>> model = lr.fit(df) >>> model.setFeaturesCol("features") - LinearRegression... + LinearRegressionModel... >>> model.setPredictionCol("newPrediction") - LinearRegression... + LinearRegressionModel... >>> model.getMaxIter() 5 >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) @@ -591,7 +591,7 @@ class IsotonicRegression(JavaEstimator, _IsotonicRegressionParams, HasWeightCol, >>> ir = IsotonicRegression() >>> model = ir.fit(df) >>> model.setFeaturesCol("features") - IsotonicRegression... + IsotonicRegressionModel... >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction 0.0 @@ -1546,7 +1546,7 @@ class AFTSurvivalRegression(JavaEstimator, _AFTSurvivalRegressionParams, >>> aftsr.clear(aftsr.maxIter) >>> model = aftsr.fit(df) >>> model.setFeaturesCol("features") - AFTSurvivalRegression... + AFTSurvivalRegressionModel... >>> model.predict(Vectors.dense(6.3)) 1.0 >>> model.predictQuantiles(Vectors.dense(6.3)) @@ -1881,7 +1881,7 @@ class GeneralizedLinearRegression(JavaPredictor, _GeneralizedLinearRegressionPar >>> glr.clear(glr.maxIter) >>> model = glr.fit(df) >>> model.setFeaturesCol("features") - GeneralizedLinearRegression... + GeneralizedLinearRegressionModel... >>> model.getMaxIter() 25 >>> model.getAggregationDepth() diff --git a/python/pyspark/ml/tree.py b/python/pyspark/ml/tree.py index f38a7375c2c54..d97a950c9276e 100644 --- a/python/pyspark/ml/tree.py +++ b/python/pyspark/ml/tree.py @@ -56,9 +56,6 @@ def predictLeaf(self, value): """ return self._call_java("predictLeaf", value) - def __repr__(self): - return self._call_java("toString") - class _DecisionTreeParams(HasCheckpointInterval, HasSeed, HasWeightCol): """ @@ -208,9 +205,6 @@ def predictLeaf(self, value): """ return self._call_java("predictLeaf", value) - def __repr__(self): - return self._call_java("toString") - class _TreeEnsembleParams(_DecisionTreeParams): """ diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 47e4921541ea2..ae3a6ba24ffa5 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -372,6 +372,9 @@ def __init__(self, java_model=None): self._resetUid(java_model.uid()) + def __repr__(self): + return self._call_java("toString") + @inherit_doc class _JavaPredictorParams(HasLabelCol, HasFeaturesCol, HasPredictionCol): From f77c10de38d0563b2e42d1200a1fbbdb3018c2e9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 15 Nov 2019 23:58:15 -0800 Subject: [PATCH 0632/1387] [SPARK-29923][SQL][TESTS] Set io.netty.tryReflectionSetAccessible for Arrow on JDK9+ ### What changes were proposed in this pull request? This PR aims to add `io.netty.tryReflectionSetAccessible=true` to the testing configuration for JDK11 because this is an officially documented requirement of Apache Arrow. Apache Arrow community documented this requirement at `0.15.0` ([ARROW-6206](https://github.com/apache/arrow/pull/5078)). > #### For java 9 or later, should set "-Dio.netty.tryReflectionSetAccessible=true". > This fixes `java.lang.UnsupportedOperationException: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available`. thrown by netty. ### Why are the changes needed? After ARROW-3191, Arrow Java library requires the property `io.netty.tryReflectionSetAccessible` to be set to true for JDK >= 9. After https://github.com/apache/spark/pull/26133, JDK11 Jenkins job seem to fail. - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/676/ - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/677/ - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-3.2-jdk-11/678/ ```scala Previous exception in task: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available io.netty.util.internal.PlatformDependent.directBuffer(PlatformDependent.java:473) io.netty.buffer.NettyArrowBuf.getDirectBuffer(NettyArrowBuf.java:243) io.netty.buffer.NettyArrowBuf.nioBuffer(NettyArrowBuf.java:233) io.netty.buffer.ArrowBuf.nioBuffer(ArrowBuf.java:245) org.apache.arrow.vector.ipc.message.ArrowRecordBatch.computeBodyLength(ArrowRecordBatch.java:222) ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with JDK11. Closes #26552 from dongjoon-hyun/SPARK-ARROW-JDK11. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- R/run-tests.sh | 2 +- pom.xml | 4 ++-- project/SparkBuild.scala | 1 + python/run-tests.py | 5 +++-- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive/pom.xml | 2 +- 7 files changed, 10 insertions(+), 8 deletions(-) diff --git a/R/run-tests.sh b/R/run-tests.sh index 86bd8aad5f113..51ca7d600caf0 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -23,7 +23,7 @@ FAILED=0 LOGFILE=$FWDIR/unit-tests.out rm -f $LOGFILE -SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE +SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" --conf spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" --conf spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE FAILED=$((PIPESTATUS[0]||$FAILED)) NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" diff --git a/pom.xml b/pom.xml index a6a82b3339d08..44593b78c9a04 100644 --- a/pom.xml +++ b/pom.xml @@ -2326,7 +2326,7 @@ **/*Suite.java ${project.build.directory}/surefire-reports - -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} -Dio.netty.tryReflectionSetAccessible=true - -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} + -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} -Dio.netty.tryReflectionSetAccessible=true From 40ea4a11d7f1534023669f0b81faf5d398174e46 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sat, 16 Nov 2019 17:46:39 +0800 Subject: [PATCH 0633/1387] [SPARK-29807][SQL] Rename "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled" ### What changes were proposed in this pull request? Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled" ### Why are the changes needed? The relation between "spark.sql.ansi.enabled" and "spark.sql.dialect" is confusing, since the "PostgreSQL" dialect should contain the features of "spark.sql.ansi.enabled". To make things clearer, we can rename the "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled", thus the option "spark.sql.dialect.spark.ansi.enabled" is only for Spark dialect. For the casting and arithmetic operations, runtime exceptions should be thrown if "spark.sql.dialect" is "spark" and "spark.sql.dialect.spark.ansi.enabled" is true or "spark.sql.dialect" is PostgresSQL. ### Does this PR introduce any user-facing change? Yes, the config name changed. ### How was this patch tested? Existing UT. Closes #26444 from xuanyuanking/SPARK-29807. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 8 +++--- .../spark/sql/catalyst/parser/SqlBase.g4 | 6 ++-- .../spark/sql/catalyst/expressions/Cast.scala | 4 +-- .../sql/catalyst/expressions/arithmetic.scala | 2 +- .../sql/catalyst/parser/ParseDriver.scala | 4 +-- .../apache/spark/sql/internal/SQLConf.scala | 28 ++++++++++++------- .../encoders/ExpressionEncoderSuite.scala | 2 +- .../catalyst/encoders/RowEncoderSuite.scala | 4 +-- .../ArithmeticExpressionSuite.scala | 24 ++++++++-------- .../sql/catalyst/expressions/CastSuite.scala | 17 +++++++---- .../expressions/DecimalExpressionSuite.scala | 4 +-- .../catalyst/expressions/ScalaUDFSuite.scala | 4 +-- .../parser/ExpressionParserSuite.scala | 10 +++---- .../parser/TableIdentifierParserSuite.scala | 2 +- .../sql-tests/inputs/postgreSQL/text.sql | 4 +-- .../sql-tests/results/postgreSQL/text.sql.out | 8 +++--- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 4 +-- .../ThriftServerQueryTestSuite.scala | 4 +-- 19 files changed, 77 insertions(+), 64 deletions(-) diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 81d7ce37af178..79bc134596237 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,15 +19,15 @@ license: | limitations under the License. --- -When `spark.sql.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.ansi.enabled=true`. +When `spark.sql.dialect.spark.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. -By default `spark.sql.ansi.enabled` is false. +By default `spark.sql.dialect.spark.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 8a6bd5ed27360..9dda6cbabfe24 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -955,7 +955,7 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -// When `spark.sql.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.dialect.spark.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, // function, alias, etc. @@ -1155,9 +1155,9 @@ ansiNonReserved | YEARS ; -// When `spark.sql.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.dialect.spark.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. // - Non-reserved keywords: -// Same definition as the one when `spark.sql.ansi.enabled=true`. +// Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. // - Strict-non-reserved keywords: // A strict version of non-reserved keywords, which can not be used as table alias. // You can find the full keywords list by searching "Start of the keywords list" in this file. 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 f3b58fa3137b1..9e802edbc5edd 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 @@ -592,7 +592,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit * Change the precision / scale in a given decimal to those set in `decimalType` (if any), * modifying `value` in-place and returning it if successful. If an overflow occurs, it * either returns null or throws an exception according to the value set for - * `spark.sql.ansi.enabled`. + * `spark.sql.dialect.spark.ansi.enabled`. * * NOTE: this modifies `value` in-place, so don't call it on external data. */ @@ -611,7 +611,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit /** * Create new `Decimal` with precision and scale given in `decimalType` (if any). - * If overflow occurs, if `spark.sql.ansi.enabled` is false, null is returned; + * If overflow occurs, if `spark.sql.dialect.spark.ansi.enabled` is false, null is returned; * otherwise, an `ArithmeticException` is thrown. */ private[this] def toPrecision(value: Decimal, decimalType: DecimalType): Decimal = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 82a8e6d80a0bd..7650fb07a61cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -150,7 +150,7 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { sys.error("BinaryArithmetics must override either calendarIntervalMethod or genCode") // Name of the function for the exact version of this expression in [[Math]]. - // If the option "spark.sql.ansi.enabled" is enabled and there is corresponding + // If the option "spark.sql.dialect.spark.ansi.enabled" is enabled and there is corresponding // function in [[Math]], the exact function will be called instead of evaluation with [[symbol]]. def exactMathMethod: Option[String] = None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index b66cae7979416..a9d5e7c7544a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -101,7 +101,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - lexer.ansi = conf.ansiEnabled + lexer.ansi = conf.dialectSparkAnsiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -109,7 +109,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - parser.ansi = conf.ansiEnabled + parser.ansi = conf.dialectSparkAnsiEnabled try { try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 759586a2936fd..5f3a984cc8d5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1673,6 +1673,20 @@ object SQLConf { .checkValues(Dialect.values.map(_.toString)) .createWithDefault(Dialect.SPARK.toString) + val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") + .internal() + .doc("This configuration is deprecated and will be removed in the future releases." + + "It is replaced by spark.sql.dialect.spark.ansi.enabled.") + .booleanConf + .createWithDefault(false) + + val DIALECT_SPARK_ANSI_ENABLED = buildConf("spark.sql.dialect.spark.ansi.enabled") + .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + + "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + + "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + + "the SQL parser.") + .fallbackConf(ANSI_ENABLED) + val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") .internal() @@ -1784,14 +1798,6 @@ object SQLConf { .checkValues(StoreAssignmentPolicy.values.map(_.toString)) .createWithDefault(StoreAssignmentPolicy.ANSI.toString) - val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") - .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + - "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + - "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + - "the SQL parser.") - .booleanConf - .createWithDefault(false) - val SORT_BEFORE_REPARTITION = buildConf("spark.sql.execution.sortBeforeRepartition") .internal() @@ -2521,9 +2527,11 @@ class SQLConf extends Serializable with Logging { def storeAssignmentPolicy: StoreAssignmentPolicy.Value = StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY)) - def ansiEnabled: Boolean = getConf(ANSI_ENABLED) + def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString + + def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) - def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString() + def ansiEnabled: Boolean = usePostgreSQLDialect || dialectSparkAnsiEnabled def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index c1f1be3b30e4b..62e688e4d4bd6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -436,7 +436,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes testAndVerifyNotLeakingReflectionObjects( s"overflowing $testName, ansiEnabled=$ansiEnabled") { withSQLConf( - SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> ansiEnabled.toString ) { // Need to construct Encoder here rather than implicitly resolving it // so that SQLConf changes are respected. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 1a1cab823d4f3..fe068f7a5f6c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -169,7 +169,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } private def testDecimalOverflow(schema: StructType, row: Row): Unit = { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { val encoder = RowEncoder(schema).resolveAndBind() intercept[Exception] { encoder.toRow(row) @@ -182,7 +182,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { val encoder = RowEncoder(schema).resolveAndBind() assert(encoder.fromRow(encoder.toRow(row)).get(0) == null) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index ad8b1a1673679..6e3fc438e41ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -61,7 +61,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Add(positiveLongLit, negativeLongLit), -1L) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Add, tpe, tpe) } @@ -80,7 +80,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(UnaryMinus(Literal(Int.MinValue)), Int.MinValue) checkEvaluation(UnaryMinus(Literal(Short.MinValue)), Short.MinValue) checkEvaluation(UnaryMinus(Literal(Byte.MinValue)), Byte.MinValue) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { checkExceptionInExpression[ArithmeticException]( UnaryMinus(Literal(Long.MinValue)), "overflow") checkExceptionInExpression[ArithmeticException]( @@ -122,7 +122,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Subtract(positiveLongLit, negativeLongLit), positiveLong - negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Subtract, tpe, tpe) } @@ -144,7 +144,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Multiply(positiveLongLit, negativeLongLit), positiveLong * negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Multiply, tpe, tpe) } @@ -445,12 +445,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minLongLiteral, minLongLiteral) val e5 = Subtract(minLongLiteral, maxLongLiteral) val e6 = Multiply(minLongLiteral, minLongLiteral) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Long.MinValue) checkEvaluation(e2, Long.MinValue) checkEvaluation(e3, -2L) @@ -469,12 +469,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minIntLiteral, minIntLiteral) val e5 = Subtract(minIntLiteral, maxIntLiteral) val e6 = Multiply(minIntLiteral, minIntLiteral) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Int.MinValue) checkEvaluation(e2, Int.MinValue) checkEvaluation(e3, -2) @@ -493,12 +493,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minShortLiteral, minShortLiteral) val e5 = Subtract(minShortLiteral, maxShortLiteral) val e6 = Multiply(minShortLiteral, minShortLiteral) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Short.MinValue) checkEvaluation(e2, Short.MinValue) checkEvaluation(e3, (-2).toShort) @@ -517,12 +517,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minByteLiteral, minByteLiteral) val e5 = Subtract(minByteLiteral, maxByteLiteral) val e6 = Multiply(minByteLiteral, minByteLiteral) - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Byte.MinValue) checkEvaluation(e2, Byte.MinValue) checkEvaluation(e3, (-2).toByte) 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 fb99fc805c45b..12ca3e798b13d 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 @@ -891,7 +891,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to decimal type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { checkExceptionInExpression[ArithmeticException]( cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( @@ -957,7 +958,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to byte type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ByteType) Seq(Byte.MaxValue + 1, Byte.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ByteType), "overflow") @@ -982,7 +984,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to short type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ShortType) Seq(Short.MaxValue + 1, Short.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ShortType), "overflow") @@ -1007,7 +1010,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to int type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key ->requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(IntegerType) testLongMaxAndMin(IntegerType) @@ -1024,7 +1028,8 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to long type") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf( + SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testLongMaxAndMin(LongType) Seq(Long.MaxValue, 0, Long.MinValue).foreach { value => @@ -1201,7 +1206,7 @@ class CastSuite extends CastSuiteBase { } test("SPARK-28470: Cast should honor nullOnOverflow property") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(Cast(Literal("134.12"), DecimalType(3, 2)), null) checkEvaluation( Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala index 36bc3db580400..8609d888b7bc9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala @@ -32,7 +32,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("MakeDecimal") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) @@ -41,7 +41,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { evaluateWithoutCodegen(overflowExpr, null) checkEvaluationWithUnsafeProjection(overflowExpr, null) } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index c5ffc381b58e2..cf6ebfb0ecefb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -57,7 +57,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-28369: honor nullOnOverflow config for ScalaUDF") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, @@ -69,7 +69,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } assert(e2.getCause.isInstanceOf[ArithmeticException]) } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index a707b456c6bd1..17a2213d0ccb1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -615,7 +615,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual(s"${sign}interval $intervalValue", expectedLiteral) // SPARK-23264 Support interval values without INTERVAL clauses if ANSI SQL enabled - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } } @@ -701,12 +701,12 @@ class ExpressionParserSuite extends AnalysisTest { test("SPARK-23264 Interval Compatibility tests") { def checkIntervals(intervalValue: String, expected: Literal): Unit = { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } // Compatibility tests: If ANSI SQL disabled, `intervalValue` should be parsed as an alias - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { val aliases = defaultParser.parseExpression(intervalValue).collect { case a @ Alias(_: Literal, name) if intervalUnits.exists { unit => name.startsWith(unit.toString) } => a @@ -804,12 +804,12 @@ class ExpressionParserSuite extends AnalysisTest { } test("current date/timestamp braceless expressions") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { assertEqual("current_date", CurrentDate()) assertEqual("current_timestamp", CurrentTimestamp()) } - withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { assertEqual("current_date", UnresolvedAttribute.quoted("current_date")) assertEqual("current_timestamp", UnresolvedAttribute.quoted("current_timestamp")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index a9216174804d0..9560aec944d9a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -658,7 +658,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { } test("table identifier - reserved/non-reserved keywords if ANSI mode enabled") { - withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { reservedKeywordsInAnsiMode.foreach { keyword => val errMsg = intercept[ParseException] { parseTableIdentifier(keyword) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql index 7abf903bc6bee..a1fe95462ecae 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql @@ -46,9 +46,9 @@ select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); -- [SPARK-28036] Built-in udf left/right has inconsistent behavior -- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode -set spark.sql.ansi.enabled=false; +set spark.sql.dialect.spark.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; -set spark.sql.ansi.enabled=true; +set spark.sql.dialect.spark.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 2e1d639974ec6..cac1e7ee5ab12 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -151,11 +151,11 @@ edcba -- !query 18 -set spark.sql.ansi.enabled=false +set spark.sql.dialect.spark.ansi.enabled=false -- !query 18 schema struct -- !query 18 output -spark.sql.ansi.enabled false +spark.sql.dialect.spark.ansi.enabled false -- !query 19 @@ -177,11 +177,11 @@ struct -- !query 20 -set spark.sql.ansi.enabled=true +set spark.sql.dialect.spark.ansi.enabled=true -- !query 20 schema struct -- !query 20 output -spark.sql.ansi.enabled true +spark.sql.dialect.spark.ansi.enabled true -- !query 21 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 40f2e923d1d78..c79e2e9c76bb9 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 @@ -163,7 +163,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { DecimalData(BigDecimal("9"* 20 + ".123"), BigDecimal("9"* 20 + ".123")) :: Nil).toDF() Seq(true, false).foreach { ansiEnabled => - withSQLConf((SQLConf.ANSI_ENABLED.key, ansiEnabled.toString)) { + withSQLConf((SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, ansiEnabled.toString)) { val structDf = largeDecimals.select("a").agg(sum("a")) if (!ansiEnabled) { checkAnswer(structDf, Row(null)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index ffea03474ac4a..d693ff0da33cd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -343,10 +343,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) case _: AnsiTest => - localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) case _ => } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 67193e8de16f4..82da4c049fd41 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -107,10 +107,10 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { testCase match { case _: PgSQLTest => - statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") + statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") case _: AnsiTest => - statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") + statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") case _ => } From d0470d639412ecbe6e126f8d8abf5a5819b9e278 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 16 Nov 2019 09:26:01 -0800 Subject: [PATCH 0634/1387] [MINOR][TESTS] Ignore GitHub Action and AppVeyor file changes in testing ### What changes were proposed in this pull request? This PR aims to ignore `GitHub Action` and `AppVeyor` file changes. When we touch these files, Jenkins job should not trigger a full testing. ### Why are the changes needed? Currently, these files are categorized to `root` and trigger the full testing and ends up wasting the Jenkins resources. - https://github.com/apache/spark/pull/26555 ``` [info] Using build tool sbt with Hadoop profile hadoop2.7 under environment amplab_jenkins From https://github.com/apache/spark * [new branch] master -> master [info] Found the following changed modules: sparkr, root [info] Setup the following environment variables for tests: ``` ### Does this PR introduce any user-facing change? No. (Jenkins testing only). ### How was this patch tested? Manually. ``` $ dev/run-tests.py -h -v ... Trying: [x.name for x in determine_modules_for_files([".github/workflows/master.yml", "appveyor.xml"])] Expecting: [] ... ``` Closes #26556 from dongjoon-hyun/SPARK-IGNORE-APPVEYOR. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/run-tests.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 82277720bb52f..fc8b7251a85f8 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -43,15 +43,20 @@ def determine_modules_for_files(filenames): """ Given a list of filenames, return the set of modules that contain those files. If a file is not associated with a more specific submodule, then this method will consider that - file to belong to the 'root' module. + file to belong to the 'root' module. GitHub Action and Appveyor files are ignored. >>> sorted(x.name for x in determine_modules_for_files(["python/pyspark/a.py", "sql/core/foo"])) ['pyspark-core', 'sql'] >>> [x.name for x in determine_modules_for_files(["file_not_matched_by_any_subproject"])] ['root'] + >>> [x.name for x in determine_modules_for_files( \ + [".github/workflows/master.yml", "appveyor.yml"])] + [] """ changed_modules = set() for filename in filenames: + if filename in (".github/workflows/master.yml", "appveyor.yml"): + continue matched_at_least_one_module = False for module in modules.all_modules: if module.contains_file(filename): From 53364730049981e3d36b6e6a405b1df5314d321d Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Sat, 16 Nov 2019 13:20:05 -0600 Subject: [PATCH 0635/1387] [SPARK-29476][WEBUI] add tooltip for Thread ### What changes were proposed in this pull request? Adding tooltip for Thread Dump - Thread Locks Before: ![Screenshot from 2019-11-04 17-11-22](https://user-images.githubusercontent.com/51401130/68127349-b963f580-ff3b-11e9-8547-e01907382632.png) After: ![Screenshot from 2019-11-13 18-12-54](https://user-images.githubusercontent.com/51401130/68768698-08e7a700-0649-11ea-804b-2eb4d5f162b4.png) ### Why are the changes needed? Thread Dump tab do not have any tooltip for the columns, Some page provide tooltip , inorder to resolve the inconsistency and for better user experience. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual Closes #26386 from PavithraRamachandran/threadDump_tooltip. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- .../org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index a13037b5e24db..77564f48015f1 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -89,7 +89,12 @@ private[ui] class ExecutorThreadDumpPage( Thread ID Thread Name Thread State - Thread Locks + + + Thread Locks + + {dumpRows} From e88267cb5a26b687f69977c9792af18fc3833db1 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 16 Nov 2019 18:01:25 -0800 Subject: [PATCH 0636/1387] [SPARK-29928][SQL][TESTS] Check parsing timestamps up to microsecond precision by JSON/CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to add tests from the commit https://github.com/apache/spark/commit/9c7e8be1dca8285296f3052c41f35043699d7d10 for Spark 2.4 that check parsing of timestamp strings for various seconds fractions. ### Why are the changes needed? To make sure that current behavior is the same as in Spark 2.4 ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`. Closes #26558 from MaxGekk/parse-timestamp-micros-tests. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../sql/util/TimestampFormatterSuite.scala | 39 +++++++++++++++++++ .../apache/spark/sql/JsonFunctionsSuite.scala | 10 +++++ .../execution/datasources/csv/CSVSuite.scala | 12 ++++++ 3 files changed, 61 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index 6107a15f5c428..79e433e88437b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros +import org.apache.spark.unsafe.types.UTF8String class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers { @@ -154,4 +155,42 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers formatter.parse("Tomorrow ") should be(tomorrow +- tolerance) } } + + test("parsing timestamp strings with various seconds fractions") { + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + def check(pattern: String, input: String, reference: String): Unit = { + val formatter = TimestampFormatter(pattern, zoneId) + val expected = DateTimeUtils.stringToTimestamp( + UTF8String.fromString(reference), zoneId).get + val actual = formatter.parse(input) + assert(actual === expected) + } + + check("yyyy-MM-dd'T'HH:mm:ss.SSSSSSSXXX", + "2019-10-14T09:39:07.3220000Z", "2019-10-14T09:39:07.322Z") + check("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + "2019-10-14T09:39:07.322000", "2019-10-14T09:39:07.322") + check("yyyy-MM-dd'T'HH:mm:ss.SSSSSSX", + "2019-10-14T09:39:07.123456Z", "2019-10-14T09:39:07.123456Z") + check("yyyy-MM-dd'T'HH:mm:ss.SSSSSSX", + "2019-10-14T09:39:07.000010Z", "2019-10-14T09:39:07.00001Z") + check("yyyy HH:mm:ss.SSSSS", "1970 01:02:03.00004", "1970-01-01 01:02:03.00004") + check("yyyy HH:mm:ss.SSSS", "2019 00:00:07.0100", "2019-01-01 00:00:07.0100") + check("yyyy-MM-dd'T'HH:mm:ss.SSSX", + "2019-10-14T09:39:07.322Z", "2019-10-14T09:39:07.322Z") + check("yyyy-MM-dd'T'HH:mm:ss.SS", + "2019-10-14T09:39:07.10", "2019-10-14T09:39:07.1") + check("yyyy-MM-dd'T'HH:mm:ss.S", + "2019-10-14T09:39:07.1", "2019-10-14T09:39:07.1") + + try { + TimestampFormatter("yyyy/MM/dd HH_mm_ss.SSSSSS", zoneId) + .parse("2019/11/14 20#25#30.123456") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: java.time.format.DateTimeParseException => + assert(e.getMessage.contains("could not be parsed")) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index c41ca925721bf..af80fa3d9327a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -627,4 +627,14 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { assert(readback(0).getAs[Row](0).getAs[Date](0).getTime >= 0) } } + + test("from_json - timestamp in micros") { + val df = Seq("""{"time": "1970-01-01T00:00:00.123456"}""").toDS() + val schema = new StructType().add("time", TimestampType) + val options = Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SSSSSS") + + checkAnswer( + df.select(from_json($"value", schema, options)), + Row(Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.123456")))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 48a8c92aad759..dac305944dd0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2168,4 +2168,16 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } } } + + test("parse timestamp in microsecond precision") { + withTempPath { path => + val t = "2019-11-14 20:35:30.123456" + Seq(t).toDF("t").write.text(path.getAbsolutePath) + val readback = spark.read + .schema("t timestamp") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSSSSS") + .csv(path.getAbsolutePath) + checkAnswer(readback, Row(Timestamp.valueOf(t))) + } + } } From cc12cf6029b1b119599a381f4e4600c0a6525ff2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 16 Nov 2019 18:28:27 -0800 Subject: [PATCH 0637/1387] [SPARK-29378][R] Upgrade SparkR to use Arrow 0.15 API ### What changes were proposed in this pull request? [[SPARK-29376] Upgrade Apache Arrow to version 0.15.1](https://github.com/apache/spark/pull/26133) upgrades to Arrow 0.15 at Scala/Java/Python. This PR aims to upgrade `SparkR` to use Arrow 0.15 API. Currently, it's broken. ### Why are the changes needed? First of all, it turns out that our Jenkins jobs (including PR builder) ignores Arrow test. Arrow 0.15 has a breaking R API changes at [ARROW-5505](https://issues.apache.org/jira/browse/ARROW-5505) and we missed that. AppVeyor was the only one having SparkR Arrow tests but it's broken now. **Jenkins** ``` Skipped ------------------------------------------------------------------------ 1. createDataFrame/collect Arrow optimization (test_sparkSQL_arrow.R#25) - arrow not installed ``` Second, Arrow throws OOM on AppVeyor environment (Windows JDK8) like the following because it still has Arrow 0.14. ``` Warnings ----------------------------------------------------------------------- 1. createDataFrame/collect Arrow optimization (test_sparkSQL_arrow.R#39) - createDataFrame attempted Arrow optimization because 'spark.sql.execution.arrow.sparkr.enabled' is set to true; however, failed, attempting non-optimization. Reason: Error in handleErrors(returnStatus, conn): java.lang.OutOfMemoryError: Java heap space at java.nio.HeapByteBuffer.(HeapByteBuffer.java:57) at java.nio.ByteBuffer.allocate(ByteBuffer.java:335) at org.apache.arrow.vector.ipc.message.MessageSerializer.readMessage(MessageSerializer.java:669) at org.apache.spark.sql.execution.arrow.ArrowConverters$$anon$3.readNextBatch(ArrowConverters.scala:243) ``` It is due to the version mismatch. ```java int messageLength = MessageSerializer.bytesToInt(buffer.array()); if (messageLength == IPC_CONTINUATION_TOKEN) { buffer.clear(); // ARROW-6313, if the first 4 bytes are continuation message, read the next 4 for the length if (in.readFully(buffer) == 4) { messageLength = MessageSerializer.bytesToInt(buffer.array()); } } // Length of 0 indicates end of stream if (messageLength != 0) { // Read the message into the buffer. ByteBuffer messageBuffer = ByteBuffer.allocate(messageLength); ``` After upgrading this to 0.15, we are hitting ARROW-5505. This PR upgrades Arrow version in AppVeyor and fix the issue. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the AppVeyor. This PR passed here. - https://ci.appveyor.com/project/ApacheSoftwareFoundation/spark/builds/28909044 ``` SparkSQL Arrow optimization: Spark package found in SPARK_HOME: C:\projects\spark\bin\.. ................ ``` Closes #26555 from dongjoon-hyun/SPARK-R-TEST. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- R/pkg/R/SQLContext.R | 4 ++-- R/pkg/R/deserialize.R | 2 +- appveyor.yml | 4 +--- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index f27ef4ee28f16..cf96c658d34d2 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -166,9 +166,9 @@ writeToFileInArrow <- function(fileName, rdf, numPartitions) { for (rdf_slice in rdf_slices) { batch <- arrow::record_batch(rdf_slice) if (is.null(stream_writer)) { - stream <- arrow::FileOutputStream(fileName) + stream <- arrow::FileOutputStream$create(fileName) schema <- batch$schema - stream_writer <- arrow::RecordBatchStreamWriter(stream, schema) + stream_writer <- arrow::RecordBatchStreamWriter$create(stream, schema) } stream_writer$write_batch(batch) diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index a6febb1cbd132..ca4a6e342d772 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -242,7 +242,7 @@ readDeserializeInArrow <- function(inputCon) { # for now. dataLen <- readInt(inputCon) arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") - batches <- arrow::RecordBatchStreamReader(arrowData)$batches() + batches <- arrow::RecordBatchStreamReader$create(arrowData)$batches() if (useAsTibble) { as_tibble <- get("as_tibble", envir = asNamespace("arrow")) diff --git a/appveyor.yml b/appveyor.yml index b36175a787ae9..c3baa1f82a399 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -42,10 +42,8 @@ install: # Install maven and dependencies - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" - # Use Arrow R 0.14.1 for now. 0.15.0 seems not working for now. See SPARK-29378. + - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" - cmd: R -e "install.packages(c('assertthat', 'bit64', 'fs', 'purrr', 'R6', 'tidyselect'), repos='https://cloud.r-project.org/')" - - cmd: R -e "install.packages('https://cran.r-project.org/src/contrib/Archive/arrow/arrow_0.14.1.tar.gz', repos=NULL, type='source')" # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. # As of devtools 2.1.0, it requires testthat higher then 2.1.1 as a dependency. SparkR test requires testthat 1.0.2. # Therefore, we don't use devtools but installs it directly from the archive including its dependencies. From 388a737b985b048ed4330764f1788160c278d929 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Sat, 16 Nov 2019 19:50:02 -0800 Subject: [PATCH 0638/1387] [SPARK-29858][SQL] ALTER DATABASE (SET DBPROPERTIES) should look up catalog like v2 commands ### What changes were proposed in this pull request? Add AlterNamespaceSetPropertiesStatement, AlterNamespaceSetProperties and AlterNamespaceSetPropertiesExec to make ALTER DATABASE (SET DBPROPERTIES) command look up catalog like v2 commands. ### Why are the changes needed? It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users. ### Does this PR introduce any user-facing change? Yes, add "ALTER NAMESPACE ... SET (DBPROPERTIES | PROPERTIES) ..." whose function is same as "ALTER DATABASE ... SET DBPROPERTIES ..." and "ALTER SCHEMA ... SET DBPROPERTIES ...". ### How was this patch tested? New unit test Closes #26551 from fuwhu/SPARK-29858. Authored-by: fuwhu Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../catalyst/analysis/ResolveCatalogs.scala | 3 ++ .../sql/catalyst/parser/AstBuilder.scala | 17 ++++++++ .../catalyst/plans/logical/statements.scala | 7 +++ .../catalyst/plans/logical/v2Commands.scala | 9 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 32 ++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 7 +++ .../spark/sql/execution/SparkSqlParser.scala | 15 ------- .../v2/AlterNamespaceSetPropertiesExec.scala | 43 +++++++++++++++++++ .../datasources/v2/DataSourceV2Strategy.scala | 5 ++- .../sql/connector/DataSourceV2SQLSuite.scala | 15 +++++++ .../execution/command/DDLParserSuite.scala | 19 -------- 12 files changed, 139 insertions(+), 37 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 9dda6cbabfe24..d2d145606b627 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -91,8 +91,8 @@ statement ((COMMENT comment=STRING) | locationSpec | (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))* #createNamespace - | ALTER database db=errorCapturingIdentifier - SET DBPROPERTIES tablePropertyList #setDatabaseProperties + | ALTER (database | NAMESPACE) multipartIdentifier + SET (DBPROPERTIES | PROPERTIES) tablePropertyList #setNamespaceProperties | ALTER database db=errorCapturingIdentifier SET locationSpec #setDatabaseLocation | DROP (database | NAMESPACE) (IF EXISTS)? multipartIdentifier diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 22caf33b50ce6..0d8d58520edad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -93,6 +93,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) s"Can not specify catalog `${catalog.name}` for view ${tableName.quoted} " + s"because view support in catalog has not been implemented yet") + case AlterNamespaceSetPropertiesStatement(NonSessionCatalog(catalog, nameParts), properties) => + AlterNamespaceSetProperties(catalog, nameParts, properties) + case DescribeTableStatement( nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => if (partitionSpec.nonEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 73233f934bedc..6be163b4b3c2a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2528,6 +2528,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx.CASCADE != null) } + /** + * Create an [[AlterNamespaceSetPropertiesStatement]] logical plan. + * + * For example: + * {{{ + * ALTER (DATABASE|SCHEMA|NAMESPACE) database + * SET (DBPROPERTIES|PROPERTIES) (property_name=property_value, ...); + * }}} + */ + override def visitSetNamespaceProperties(ctx: SetNamespacePropertiesContext): LogicalPlan = { + withOrigin(ctx) { + AlterNamespaceSetPropertiesStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + visitPropertyKeyValues(ctx.tablePropertyList)) + } + } + /** * Create a [[ShowNamespacesStatement]] command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 903f5cd5912c6..875ff4e4a2821 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -350,6 +350,13 @@ case class DropNamespaceStatement( ifExists: Boolean, cascade: Boolean) extends ParsedStatement +/** + * ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET (DBPROPERTIES|PROPERTIES) command, as parsed from SQL. + */ +case class AlterNamespaceSetPropertiesStatement( + namespace: Seq[String], + properties: Map[String, String]) extends ParsedStatement + /** * A SHOW NAMESPACES statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 3179bab8c2f7f..22edb3619a929 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -270,6 +270,15 @@ case class DescribeNamespace( new MetadataBuilder().putString("comment", "value of the column").build())()) } +/** + * The logical plan of the ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET (DBPROPERTIES|PROPERTIES) + * command that works for v2 catalogs. + */ +case class AlterNamespaceSetProperties( + catalog: CatalogPlugin, + namespace: Seq[String], + properties: Map[String, String]) extends Command + /** * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index b998127ebe02a..e87f53910c2c7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1160,6 +1160,38 @@ class DDLParserSuite extends AnalysisTest { DropNamespaceStatement(Seq("a", "b", "c"), ifExists = false, cascade = true)) } + test("set namespace properties") { + comparePlans( + parsePlan("ALTER DATABASE a.b.c SET PROPERTIES ('a'='a', 'b'='b', 'c'='c')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("a" -> "a", "b" -> "b", "c" -> "c"))) + + comparePlans( + parsePlan("ALTER SCHEMA a.b.c SET PROPERTIES ('a'='a')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("a" -> "a"))) + + comparePlans( + parsePlan("ALTER NAMESPACE a.b.c SET PROPERTIES ('b'='b')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("b" -> "b"))) + + comparePlans( + parsePlan("ALTER DATABASE a.b.c SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("a" -> "a", "b" -> "b", "c" -> "c"))) + + comparePlans( + parsePlan("ALTER SCHEMA a.b.c SET DBPROPERTIES ('a'='a')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("a" -> "a"))) + + comparePlans( + parsePlan("ALTER NAMESPACE a.b.c SET DBPROPERTIES ('b'='b')"), + AlterNamespaceSetPropertiesStatement( + Seq("a", "b", "c"), Map("b" -> "b"))) + } + test("show databases: basic") { comparePlans( parsePlan("SHOW DATABASES"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 3913f889eb128..cac320edc47e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -165,6 +165,13 @@ class ResolveSessionCatalog( } DescribeDatabaseCommand(nameParts.head, d.extended) + case AlterNamespaceSetPropertiesStatement(SessionCatalog(_, nameParts), properties) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + AlterDatabasePropertiesCommand(nameParts.head, properties) + case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index bfea908d106df..7fe4c00f5dc8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -227,21 +227,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } - /** - * Create an [[AlterDatabasePropertiesCommand]] command. - * - * For example: - * {{{ - * ALTER (DATABASE|SCHEMA) database SET DBPROPERTIES (property_name=property_value, ...); - * }}} - */ - override def visitSetDatabaseProperties( - ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterDatabasePropertiesCommand( - ctx.db.getText, - visitPropertyKeyValues(ctx.tablePropertyList)) - } - /** * Create an [[AlterDatabaseSetLocationCommand]] command. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala new file mode 100644 index 0000000000000..fd6922cd1c3ab --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala @@ -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. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, NamespaceChange} + +/** + * Physical plan node for setting properties of namespace. + */ +case class AlterNamespaceSetPropertiesExec( + catalog: CatalogPlugin, + namespace: Seq[String], + props: Map[String, String]) + extends V2CommandExec { + override protected def run(): Seq[InternalRow] = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + val changes = props.map{ case (k, v) => + NamespaceChange.setProperty(k, v) + }.toSeq + catalog.asNamespaceCatalog.alterNamespace(namespace.toArray, changes: _*) + Seq.empty + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index a04bceb18b9b7..95caebe9e8938 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetProperties, AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} @@ -204,6 +204,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case AlterTable(catalog, ident, _, changes) => AlterTableExec(catalog, ident, changes) :: Nil + case AlterNamespaceSetProperties(catalog, namespace, properties) => + AlterNamespaceSetPropertiesExec(catalog, namespace, properties) :: Nil + case CreateNamespace(catalog, namespace, ifNotExists, properties) => CreateNamespaceExec(catalog, namespace, ifNotExists, properties) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 095bcff31ccd9..e5bf8e337c8d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -933,6 +933,21 @@ class DataSourceV2SQLSuite } } + test("AlterNamespaceSetProperties using v2 catalog") { + withNamespace("testcat.ns1.ns2") { + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + + "'test namespace' LOCATION '/tmp/ns_test' WITH PROPERTIES ('a'='a','b'='b','c'='c')") + sql("ALTER NAMESPACE testcat.ns1.ns2 SET PROPERTIES ('a'='b','b'='a')") + val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") + assert(descriptionDf.collect() === Seq( + Row("Namespace Name", "ns2"), + Row("Description", "test namespace"), + Row("Location", "/tmp/ns_test"), + Row("Properties", "((a,b),(b,a),(c,c))") + )) + } + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index b2185f8559f36..b11e02a9c52bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -74,25 +74,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { }.head } - test("alter database set dbproperties") { - // ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...) - val sql1 = "ALTER DATABASE database_name SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')" - val sql2 = "ALTER SCHEMA database_name SET DBPROPERTIES ('a'='a')" - - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - - val expected1 = AlterDatabasePropertiesCommand( - "database_name", - Map("a" -> "a", "b" -> "b", "c" -> "c")) - val expected2 = AlterDatabasePropertiesCommand( - "database_name", - Map("a" -> "a")) - - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } - test("alter database - property values must be set") { assertUnsupported( sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')", From a9959be2bcf55f753ef48e0a4daea6abe0d63c1c Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Sun, 17 Nov 2019 07:04:40 -0600 Subject: [PATCH 0639/1387] [SPARK-29456][WEBUI] Improve tooltip for Session Statistics Table column in JDBC/ODBC Server Tab What changes were proposed in this pull request? Some of the columns of JDBC/ODBC tab Session info in Web UI are hard to understand. Add tool tip for Start time, finish time , Duration and Total Execution ![Screenshot from 2019-10-16 12-33-17](https://user-images.githubusercontent.com/51401130/66901981-76d68980-f01d-11e9-9686-e20346a38c25.png) Why are the changes needed? To improve the understanding of the WebUI Does this PR introduce any user-facing change? No How was this patch tested? manual test Closes #26138 from PavithraRamachandran/JDBC_tooltip. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- .../thriftserver/ui/ThriftServerPage.scala | 43 ++++++++++++++----- .../sql/hive/thriftserver/ui/ToolTips.scala | 7 +++ 2 files changed, 39 insertions(+), 11 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 368db975ec886..d3351f3d6ca14 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -443,10 +443,13 @@ private[ui] class SessionStatsPagedTable( val sessionTableHeaders = Seq("User", "IP", "Session ID", "Start Time", "Finish Time", "Duration", "Total Execute") + val tooltips = Seq(None, None, None, None, None, Some(THRIFT_SESSION_DURATION), + Some(THRIFT_SESSION_TOTAL_EXECUTE)) + assert(sessionTableHeaders.length == tooltips.length) val colWidthAttr = s"${100.toDouble / sessionTableHeaders.size}%" val headerRow: Seq[Node] = { - sessionTableHeaders.map { header => + sessionTableHeaders.zip(tooltips).map { case (header, tooltip) => if (header == sortColumn) { val headerLink = Unparsed( parameterPath + @@ -455,12 +458,22 @@ private[ui] class SessionStatsPagedTable( s"&$sessionStatsTableTag.pageSize=$pageSize" + s"#$sessionStatsTableTag") val arrow = if (desc) "▾" else "▴" // UP or DOWN + + + { + if (tooltip.nonEmpty) { + + {header} {Unparsed(arrow)} + + } else { + + {header} {Unparsed(arrow)} + + } + } + + - - - {header} {Unparsed(arrow)} - - } else { val headerLink = Unparsed( parameterPath + @@ -468,11 +481,19 @@ private[ui] class SessionStatsPagedTable( s"&$sessionStatsTableTag.pageSize=$pageSize" + s"#$sessionStatsTableTag") - - - {header} - - + + + { + if (tooltip.nonEmpty) { + + {header} + + } else { + {header} + } + } + + } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala index 1990b8f2d3285..56ab766f4aabd 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ToolTips.scala @@ -29,4 +29,11 @@ private[ui] object ToolTips { val THRIFT_SERVER_DURATION = "Difference between start time and close time" + + val THRIFT_SESSION_TOTAL_EXECUTE = + "Number of operations submitted in this session" + + val THRIFT_SESSION_DURATION = + "Elapsed time since session start, or until closed if the session was closed" + } From e1fc38b3e409e8a2c65d0cc1fc2ec63da527bbc6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 17 Nov 2019 10:09:46 -0800 Subject: [PATCH 0640/1387] [SPARK-29932][R][TESTS] lint-r should do non-zero exit in case of errors ### What changes were proposed in this pull request? This PR aims to make `lint-r` exits with non-zero in case of errors. Please note that `lint-r` works correctly when everything are installed correctly. ### Why are the changes needed? There are two cases which hide errors from Jenkins/AppVeyor/GitHubAction. 1. `lint-r` exits with zero if there is no R installation. ```bash $ dev/lint-r dev/lint-r: line 25: type: Rscript: not found ERROR: You should install R $ echo $? 0 ``` 2. `lint-r` exits with zero if we didn't do `R/install-dev.sh`. ```bash $ dev/lint-r Error: You should install SparkR in a local directory with `R/install-dev.sh`. In addition: Warning message: In library(SparkR, lib.loc = LOCAL_LIB_LOC, logical.return = TRUE) : no library trees found in 'lib.loc' Execution halted lintr checks passed. // <=== Please note here $ echo $? 0 ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually check the above two cases. Closes #26561 from dongjoon-hyun/SPARK-29932. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/lint-r | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/lint-r b/dev/lint-r index bfda0bca15eb7..b08f5efecd5d3 100755 --- a/dev/lint-r +++ b/dev/lint-r @@ -17,6 +17,9 @@ # limitations under the License. # +set -o pipefail +set -e + SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" LINT_R_REPORT_FILE_NAME="$SPARK_ROOT_DIR/dev/lint-r-report.log" @@ -24,7 +27,7 @@ LINT_R_REPORT_FILE_NAME="$SPARK_ROOT_DIR/dev/lint-r-report.log" if ! type "Rscript" > /dev/null; then echo "ERROR: You should install R" - exit + exit 1 fi `which Rscript` --vanilla "$SPARK_ROOT_DIR/dev/lint-r.R" "$SPARK_ROOT_DIR" | tee "$LINT_R_REPORT_FILE_NAME" From 5eb8973f871fef557fb4ca3f494406ed676a431a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 17 Nov 2019 10:14:04 -0800 Subject: [PATCH 0641/1387] [SPARK-29930][SQL] Remove SQL configs declared to be removed in Spark 3.0 ### What changes were proposed in this pull request? In the PR, I propose to remove the following SQL configs: 1. `spark.sql.fromJsonForceNullableSchema` 2. `spark.sql.legacy.compareDateTimestampInTimestamp` 3. `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` that are declared to be removed in Spark 3.0 ### Why are the changes needed? To make code cleaner and improve maintainability. ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? By `TypeCoercionSuite`, `JsonExpressionsSuite` and `DDLSuite`. Closes #26559 from MaxGekk/remove-sql-configs. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/TypeCoercion.scala | 7 ++-- .../sql/catalyst/catalog/SessionCatalog.scala | 3 +- .../expressions/jsonExpressions.scala | 4 +-- .../apache/spark/sql/internal/SQLConf.scala | 31 ---------------- .../catalyst/analysis/TypeCoercionSuite.scala | 29 +++++---------- .../expressions/JsonExpressionsSuite.scala | 36 +++++++++---------- 6 files changed, 29 insertions(+), 81 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index b27d6ed0efed8..83c76c2d4e2bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -132,12 +132,9 @@ object TypeCoercion { case (NullType, StringType) => Some(StringType) // Cast to TimestampType when we compare DateType with TimestampType - // if conf.compareDateTimestampInTimestamp is true // i.e. TimeStamp('2017-03-01 00:00:00') eq Date('2017-03-01') = true - case (TimestampType, DateType) - => if (conf.compareDateTimestampInTimestamp) Some(TimestampType) else Some(StringType) - case (DateType, TimestampType) - => if (conf.compareDateTimestampInTimestamp) Some(TimestampType) else Some(StringType) + case (TimestampType, DateType) => Some(TimestampType) + case (DateType, TimestampType) => Some(TimestampType) // There is no proper decimal type we can pick, // using double type is the best we can do. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index be8526454f9f1..e72352714a697 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -327,8 +327,7 @@ class SessionCatalog( def validateTableLocation(table: CatalogTable): Unit = { // SPARK-19724: the default location of a managed table should be non-existent or empty. - if (table.tableType == CatalogTableType.MANAGED && - !conf.allowCreatingManagedTableUsingNonemptyLocation) { + if (table.tableType == CatalogTableType.MANAGED) { val tableLocation = new Path(table.storage.locationUri.getOrElse(defaultTablePath(table.identifier))) val fs = tableLocation.getFileSystem(hadoopConf) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index a13a6836c6be6..de7e1160185dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -515,12 +515,10 @@ case class JsonToStructs( timeZoneId: Option[String] = None) extends UnaryExpression with TimeZoneAwareExpression with CodegenFallback with ExpectsInputTypes { - val forceNullableSchema = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA) - // The JSON input data might be missing certain fields. We force the nullability // of the user-provided schema to avoid data corruptions. In particular, the parquet-mr encoder // can generate incorrect files if values are missing in columns declared as non-nullable. - val nullableSchema = if (forceNullableSchema) schema.asNullable else schema + val nullableSchema = schema.asNullable override def nullable: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5f3a984cc8d5d..8b714d6dcdda3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -720,14 +720,6 @@ object SQLConf { .stringConf .createWithDefault("_corrupt_record") - val FROM_JSON_FORCE_NULLABLE_SCHEMA = buildConf("spark.sql.fromJsonForceNullableSchema") - .internal() - .doc("When true, force the output schema of the from_json() function to be nullable " + - "(including all the fields). Otherwise, the schema might not be compatible with" + - "actual data, which leads to corruptions. This config will be removed in Spark 3.0.") - .booleanConf - .createWithDefault(true) - val BROADCAST_TIMEOUT = buildConf("spark.sql.broadcastTimeout") .doc("Timeout in seconds for the broadcast wait time in broadcast joins.") .timeConf(TimeUnit.SECONDS) @@ -1687,14 +1679,6 @@ object SQLConf { "the SQL parser.") .fallbackConf(ANSI_ENABLED) - val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = - buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") - .internal() - .doc("When this option is set to true, creating managed tables with nonempty location " + - "is allowed. Otherwise, an analysis exception is thrown. ") - .booleanConf - .createWithDefault(false) - val VALIDATE_PARTITION_COLUMNS = buildConf("spark.sql.sources.validatePartitionColumns") .internal() @@ -1913,16 +1897,6 @@ object SQLConf { .checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION) .createWithDefault(Deflater.DEFAULT_COMPRESSION) - val COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP = - buildConf("spark.sql.legacy.compareDateTimestampInTimestamp") - .internal() - .doc("When true (default), compare Date with Timestamp after converting both sides to " + - "Timestamp. This behavior is compatible with Hive 2.2 or later. See HIVE-15236. " + - "When false, restore the behavior prior to Spark 2.4. Compare Date with Timestamp after " + - "converting both sides to string. This config will be removed in Spark 3.0.") - .booleanConf - .createWithDefault(true) - val LEGACY_SIZE_OF_NULL = buildConf("spark.sql.legacy.sizeOfNull") .doc("If it is set to true, size of null returns -1. This behavior was inherited from Hive. " + "The size function returns null for null input if the flag is disabled.") @@ -2236,8 +2210,6 @@ class SQLConf extends Serializable with Logging { def caseSensitiveInferenceMode: HiveCaseSensitiveInferenceMode.Value = HiveCaseSensitiveInferenceMode.withName(getConf(HIVE_CASE_SENSITIVE_INFERENCE)) - def compareDateTimestampInTimestamp : Boolean = getConf(COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP) - def gatherFastStats: Boolean = getConf(GATHER_FASTSTAT) def optimizerMetadataOnly: Boolean = getConf(OPTIMIZER_METADATA_ONLY) @@ -2516,9 +2488,6 @@ class SQLConf extends Serializable with Logging { def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) - def allowCreatingManagedTableUsingNonemptyLocation: Boolean = - getConf(ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION) - def validatePartitionColumns: Boolean = getConf(VALIDATE_PARTITION_COLUMNS) def partitionOverwriteMode: PartitionOverwriteMode.Value = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index c7371a7911df5..567cf5ec8ebe6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1526,26 +1526,15 @@ class TypeCoercionSuite extends AnalysisTest { GreaterThan(Literal("1.5"), Literal(BigDecimal("0.5"))), GreaterThan(Cast(Literal("1.5"), DoubleType), Cast(Literal(BigDecimal("0.5")), DoubleType))) - Seq(true, false).foreach { convertToTS => - withSQLConf( - SQLConf.COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP.key -> convertToTS.toString) { - val date0301 = Literal(java.sql.Date.valueOf("2017-03-01")) - val timestamp0301000000 = Literal(Timestamp.valueOf("2017-03-01 00:00:00")) - val timestamp0301000001 = Literal(Timestamp.valueOf("2017-03-01 00:00:01")) - if (convertToTS) { - // `Date` should be treated as timestamp at 00:00:00 See SPARK-23549 - ruleTest(rule, EqualTo(date0301, timestamp0301000000), - EqualTo(Cast(date0301, TimestampType), timestamp0301000000)) - ruleTest(rule, LessThan(date0301, timestamp0301000001), - LessThan(Cast(date0301, TimestampType), timestamp0301000001)) - } else { - ruleTest(rule, LessThan(date0301, timestamp0301000000), - LessThan(Cast(date0301, StringType), Cast(timestamp0301000000, StringType))) - ruleTest(rule, LessThan(date0301, timestamp0301000001), - LessThan(Cast(date0301, StringType), Cast(timestamp0301000001, StringType))) - } - } - } + // Checks that dates/timestamps are not promoted to strings + val date0301 = Literal(java.sql.Date.valueOf("2017-03-01")) + val timestamp0301000000 = Literal(Timestamp.valueOf("2017-03-01 00:00:00")) + val timestamp0301000001 = Literal(Timestamp.valueOf("2017-03-01 00:00:01")) + // `Date` should be treated as timestamp at 00:00:00 See SPARK-23549 + ruleTest(rule, EqualTo(date0301, timestamp0301000000), + EqualTo(Cast(date0301, TimestampType), timestamp0301000000)) + ruleTest(rule, LessThan(date0301, timestamp0301000001), + LessThan(Cast(date0301, TimestampType), timestamp0301000001)) } test("cast WindowFrame boundaries to the type they operate upon") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index f8400a590606a..d5cc1d4f0fdde 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -702,26 +702,22 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with } test("from_json missing fields") { - for (forceJsonNullableSchema <- Seq(false, true)) { - withSQLConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA.key -> forceJsonNullableSchema.toString) { - val input = - """{ - | "a": 1, - | "c": "foo" - |} - |""".stripMargin - val jsonSchema = new StructType() - .add("a", LongType, nullable = false) - .add("b", StringType, nullable = !forceJsonNullableSchema) - .add("c", StringType, nullable = false) - val output = InternalRow(1L, null, UTF8String.fromString("foo")) - val expr = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), gmtId) - checkEvaluation(expr, output) - val schema = expr.dataType - val schemaToCompare = if (forceJsonNullableSchema) jsonSchema.asNullable else jsonSchema - assert(schemaToCompare == schema) - } - } + val input = + """{ + | "a": 1, + | "c": "foo" + |} + |""".stripMargin + val jsonSchema = new StructType() + .add("a", LongType, nullable = false) + .add("b", StringType, nullable = false) + .add("c", StringType, nullable = false) + val output = InternalRow(1L, null, UTF8String.fromString("foo")) + val expr = JsonToStructs(jsonSchema, Map.empty, Literal.create(input, StringType), gmtId) + checkEvaluation(expr, output) + val schema = expr.dataType + val schemaToCompare = jsonSchema.asNullable + assert(schemaToCompare == schema) } test("SPARK-24709: infer schema of json strings") { From c5f644c6ebb871d76f6b6b12b341ba761427492c Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Mon, 18 Nov 2019 10:05:42 +0800 Subject: [PATCH 0642/1387] [SPARK-16872][ML][PYSPARK] Impl Gaussian Naive Bayes Classifier ### What changes were proposed in this pull request? support `modelType` `gaussian` ### Why are the changes needed? current modelTypes do not support continuous data ### Does this PR introduce any user-facing change? yes, add a `modelType` option ### How was this patch tested? existing testsuites and added ones Closes #26413 from zhengruifeng/gnb. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- docs/ml-classification-regression.md | 9 +- .../spark/ml/classification/NaiveBayes.scala | 299 ++++++++++++++---- .../ml/classification/NaiveBayesSuite.scala | 193 ++++++++++- project/MimaExcludes.scala | 3 + python/pyspark/ml/classification.py | 24 +- 5 files changed, 451 insertions(+), 77 deletions(-) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index b83b4ba08a5fd..d8c7d8a729624 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -478,15 +478,16 @@ it computes the conditional probability distribution of each feature given each For prediction, it applies Bayes' theorem to compute the conditional probability distribution of each label given an observation. -MLlib supports both [multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) -and [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). +MLlib supports [Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), +[Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html) +and [Gaussian naive Bayes](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Gaussian_naive_Bayes). *Input data*: -These models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +These Multinomial and Bernoulli models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each feature represents a term. A feature's value is the frequency of the term (in multinomial Naive Bayes) or a zero or one indicating whether the term was found in the document (in Bernoulli Naive Bayes). -Feature values must be *non-negative*. The model type is selected with an optional parameter +Feature values for Multinomial and Bernoulli models must be *non-negative*. The model type is selected with an optional parameter "multinomial" or "bernoulli" with "multinomial" as the default. For document classification, the input feature vectors should usually be sparse vectors. Since the training data is only used once, it is not necessary to cache it. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index bcca40d159c9b..8062870794418 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -18,18 +18,22 @@ package org.apache.spark.ml.classification import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats +import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Since import org.apache.spark.ml.PredictorParams -import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.HasWeightCol +import org.apache.spark.ml.stat.Summarizer import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.VersionUtils /** * Params for Naive Bayes Classifiers. @@ -49,12 +53,13 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW /** * The model type which is a string (case-sensitive). - * Supported options: "multinomial" and "bernoulli". + * Supported options: "multinomial", "bernoulli", "gaussian". * (default = multinomial) * @group param */ final val modelType: Param[String] = new Param[String](this, "modelType", "The model type " + - "which is a string (case-sensitive). Supported options: multinomial (default) and bernoulli.", + "which is a string (case-sensitive). Supported options: multinomial (default), bernoulli" + + " and gaussian.", ParamValidators.inArray[String](NaiveBayes.supportedModelTypes.toArray)) /** @group getParam */ @@ -72,7 +77,11 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW * binary (0/1) data, it can also be used as Bernoulli NB * (see * here). - * The input feature values must be nonnegative. + * The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + * Since 3.0.0, it also supports Gaussian NB + * (see + * here) + * which can handle continuous data. */ // scalastyle:on line.size.limit @Since("1.5.0") @@ -103,7 +112,7 @@ class NaiveBayes @Since("1.5.0") ( */ @Since("1.5.0") def setModelType(value: String): this.type = set(modelType, value) - setDefault(modelType -> NaiveBayes.Multinomial) + setDefault(modelType -> Multinomial) /** * Sets the value of param [[weightCol]]. @@ -130,6 +139,9 @@ class NaiveBayes @Since("1.5.0") ( positiveLabel: Boolean): NaiveBayesModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) + instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, + probabilityCol, modelType, smoothing, thresholds) + if (positiveLabel && isDefined(thresholds)) { val numClasses = getNumClasses(dataset) instr.logNumClasses(numClasses) @@ -138,44 +150,55 @@ class NaiveBayes @Since("1.5.0") ( s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - val validateInstance = $(modelType) match { - case Multinomial => - (instance: Instance) => requireNonnegativeValues(instance.features) - case Bernoulli => - (instance: Instance) => requireZeroOneBernoulliValues(instance.features) + $(modelType) match { + case Bernoulli | Multinomial => + trainDiscreteImpl(dataset, instr) + case Gaussian => + trainGaussianImpl(dataset, instr) case _ => // This should never happen. throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } + } - instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, - probabilityCol, modelType, smoothing, thresholds) + private def trainDiscreteImpl( + dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ - val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size - instr.logNumFeatures(numFeatures) + val validateUDF = $(modelType) match { + case Multinomial => + udf { vector: Vector => requireNonnegativeValues(vector); vector } + case Bernoulli => + udf { vector: Vector => requireZeroOneBernoulliValues(vector); vector } + } + + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } // Aggregates term frequencies per label. - // TODO: Calling aggregateByKey and collect creates two stages, we can implement something - // TODO: similar to reduceByKeyLocally to save one stage. - val aggregated = extractInstances(dataset, validateInstance).map { instance => - (instance.label, (instance.weight, instance.features)) - }.aggregateByKey[(Double, DenseVector, Long)]((0.0, Vectors.zeros(numFeatures).toDense, 0L))( - seqOp = { - case ((weightSum, featureSum, count), (weight, features)) => - BLAS.axpy(weight, features, featureSum) - (weightSum + weight, featureSum, count + 1) - }, - combOp = { - case ((weightSum1, featureSum1, count1), (weightSum2, featureSum2, count2)) => - BLAS.axpy(1.0, featureSum2, featureSum1) - (weightSum1 + weightSum2, featureSum1, count1 + count2) - }).collect().sortBy(_._1) - - val numSamples = aggregated.map(_._2._3).sum + // TODO: Summarizer directly returns sum vector. + val aggregated = dataset.groupBy(col($(labelCol))) + .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "count") + .summary(validateUDF(col($(featuresCol))), w).as("summary")) + .select($(labelCol), "weightSum", "summary.mean", "summary.count") + .as[(Double, Double, Vector, Long)] + .map { case (label, weightSum, mean, count) => + BLAS.scal(weightSum, mean) + (label, weightSum, mean, count) + }.collect().sortBy(_._1) + + val numFeatures = aggregated.head._3.size + instr.logNumFeatures(numFeatures) + val numSamples = aggregated.map(_._4).sum instr.logNumExamples(numSamples) val numLabels = aggregated.length instr.logNumClasses(numLabels) - val numDocuments = aggregated.map(_._2._1).sum + val numDocuments = aggregated.map(_._2).sum val labelArray = new Array[Double](numLabels) val piArray = new Array[Double](numLabels) @@ -184,19 +207,17 @@ class NaiveBayes @Since("1.5.0") ( val lambda = $(smoothing) val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 - aggregated.foreach { case (label, (n, sumTermFreqs, _)) => + aggregated.foreach { case (label, n, sumTermFreqs, _) => labelArray(i) = label piArray(i) = math.log(n + lambda) - piLogDenom val thetaLogDenom = $(modelType) match { - case Multinomial => math.log(sumTermFreqs.values.sum + numFeatures * lambda) + case Multinomial => math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) case Bernoulli => math.log(n + 2.0 * lambda) - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } var j = 0 + val offset = i * numFeatures while (j < numFeatures) { - thetaArray(i * numFeatures + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom + thetaArray(offset + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom j += 1 } i += 1 @@ -204,7 +225,86 @@ class NaiveBayes @Since("1.5.0") ( val pi = Vectors.dense(piArray) val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi, theta).setOldLabels(labelArray) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + .setOldLabels(labelArray) + } + + private def trainGaussianImpl( + dataset: Dataset[_], + instr: Instrumentation): NaiveBayesModel = { + val spark = dataset.sparkSession + import spark.implicits._ + + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + // Aggregates mean vector and square-sum vector per label. + // TODO: Summarizer directly returns square-sum vector. + val aggregated = dataset.groupBy(col($(labelCol))) + .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "normL2") + .summary(col($(featuresCol)), w).as("summary")) + .select($(labelCol), "weightSum", "summary.mean", "summary.normL2") + .as[(Double, Double, Vector, Vector)] + .map { case (label, weightSum, mean, normL2) => + (label, weightSum, mean, Vectors.dense(normL2.toArray.map(v => v * v))) + }.collect().sortBy(_._1) + + val numFeatures = aggregated.head._3.size + instr.logNumFeatures(numFeatures) + + val numLabels = aggregated.length + instr.logNumClasses(numLabels) + + val numInstances = aggregated.map(_._2).sum + + // If the ratio of data variance between dimensions is too small, it + // will cause numerical errors. To address this, we artificially + // boost the variance by epsilon, a small fraction of the standard + // deviation of the largest dimension. + // Refer to scikit-learn's implementation + // [https://github.com/scikit-learn/scikit-learn/blob/0.21.X/sklearn/naive_bayes.py#L348] + // and discussion [https://github.com/scikit-learn/scikit-learn/pull/5349] for detail. + val epsilon = Iterator.range(0, numFeatures).map { j => + var globalSum = 0.0 + var globalSqrSum = 0.0 + aggregated.foreach { case (_, weightSum, mean, squareSum) => + globalSum += mean(j) * weightSum + globalSqrSum += squareSum(j) + } + globalSqrSum / numInstances - + globalSum * globalSum / numInstances / numInstances + }.max * 1e-9 + + val piArray = new Array[Double](numLabels) + + // thetaArray in Gaussian NB store the means of features per label + val thetaArray = new Array[Double](numLabels * numFeatures) + + // thetaArray in Gaussian NB store the variances of features per label + val sigmaArray = new Array[Double](numLabels * numFeatures) + + var i = 0 + val logNumInstances = math.log(numInstances) + aggregated.foreach { case (_, weightSum, mean, squareSum) => + piArray(i) = math.log(weightSum) - logNumInstances + var j = 0 + val offset = i * numFeatures + while (j < numFeatures) { + val m = mean(j) + thetaArray(offset + j) = m + sigmaArray(offset + j) = epsilon + squareSum(j) / weightSum - m * m + j += 1 + } + i += 1 + } + + val pi = Vectors.dense(piArray) + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) + val sigma = new DenseMatrix(numLabels, numFeatures, sigmaArray, true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, sigma.compressed) } @Since("1.5.0") @@ -219,8 +319,11 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** String name for Bernoulli model type. */ private[classification] val Bernoulli: String = "bernoulli" + /** String name for Gaussian model type. */ + private[classification] val Gaussian: String = "gaussian" + /* Set of modelTypes that NaiveBayes supports */ - private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli) + private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli, Gaussian) private[NaiveBayes] def requireNonnegativeValues(v: Vector): Unit = { val values = v match { @@ -248,19 +351,24 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** * Model produced by [[NaiveBayes]] - * @param pi log of class priors, whose dimension is C (number of classes) + * + * @param pi log of class priors, whose dimension is C (number of classes) * @param theta log of class conditional probabilities, whose dimension is C (number of classes) * by D (number of features) + * @param sigma variance of each feature, whose dimension is C (number of classes) + * by D (number of features). This matrix is only available when modelType + * is set Gaussian. */ @Since("1.5.0") class NaiveBayesModel private[ml] ( @Since("1.5.0") override val uid: String, @Since("2.0.0") val pi: Vector, - @Since("2.0.0") val theta: Matrix) + @Since("2.0.0") val theta: Matrix, + @Since("3.0.0") val sigma: Matrix) extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] with NaiveBayesParams with MLWritable { - import NaiveBayes.{Bernoulli, Multinomial} + import NaiveBayes.{Bernoulli, Multinomial, Gaussian} /** * mllib NaiveBayes is a wrapper of ml implementation currently. @@ -280,18 +388,36 @@ class NaiveBayesModel private[ml] ( * This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra * application of this condition (in predict function). */ - private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { - case Multinomial => (None, None) + @transient private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { case Bernoulli => val negTheta = theta.map(value => math.log1p(-math.exp(value))) val ones = new DenseVector(Array.fill(theta.numCols) {1.0}) val thetaMinusNegTheta = theta.map { value => value - math.log1p(-math.exp(value)) } - (Option(thetaMinusNegTheta), Option(negTheta.multiply(ones))) + (thetaMinusNegTheta, negTheta.multiply(ones)) + case _ => + // This should never happen. + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + + "Variables thetaMinusNegTheta and negThetaSum should only be precomputed in Bernoulli NB.") + } + + /** + * Gaussian scoring requires sum of log(Variance). + * This precomputes sum of log(Variance) which are used for the linear algebra + * application of this condition (in predict function). + */ + @transient private lazy val logVarSum = $(modelType) match { + case Gaussian => + Array.tabulate(numClasses) { i => + Iterator.range(0, numFeatures).map { j => + math.log(sigma(i, j)) + }.sum + } case _ => // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + + "Variables logVarSum should only be precomputed in Gaussian NB.") } @Since("1.6.0") @@ -311,24 +437,42 @@ class NaiveBayesModel private[ml] ( require(value == 0.0 || value == 1.0, s"Bernoulli naive Bayes requires 0 or 1 feature values but found $features.") ) - val prob = thetaMinusNegTheta.get.multiply(features) + val prob = thetaMinusNegTheta.multiply(features) BLAS.axpy(1.0, pi, prob) - BLAS.axpy(1.0, negThetaSum.get, prob) + BLAS.axpy(1.0, negThetaSum, prob) prob } - override protected def predictRaw(features: Vector): Vector = { + private def gaussianCalculation(features: Vector) = { + val prob = Array.ofDim[Double](numClasses) + var i = 0 + while (i < numClasses) { + var s = 0.0 + var j = 0 + while (j < numFeatures) { + val d = features(j) - theta(i, j) + s += d * d / sigma(i, j) + j += 1 + } + prob(i) = pi(i) - (s + logVarSum(i)) / 2 + i += 1 + } + Vectors.dense(prob) + } + + @transient private lazy val predictRawFunc = { $(modelType) match { case Multinomial => - multinomialCalculation(features) + features: Vector => multinomialCalculation(features) case Bernoulli => - bernoulliCalculation(features) - case _ => - // This should never happen. - throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") + features: Vector => bernoulliCalculation(features) + case Gaussian => + features: Vector => gaussianCalculation(features) } } + override protected def predictRaw(features: Vector): Vector = predictRawFunc(features) + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { rawPrediction match { case dv: DenseVector => @@ -354,7 +498,7 @@ class NaiveBayesModel private[ml] ( @Since("1.5.0") override def copy(extra: ParamMap): NaiveBayesModel = { - copyValues(new NaiveBayesModel(uid, pi, theta).setParent(this.parent), extra) + copyValues(new NaiveBayesModel(uid, pi, theta, sigma).setParent(this.parent), extra) } @Since("1.5.0") @@ -378,34 +522,61 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { /** [[MLWriter]] instance for [[NaiveBayesModel]] */ private[NaiveBayesModel] class NaiveBayesModelWriter(instance: NaiveBayesModel) extends MLWriter { + import NaiveBayes._ private case class Data(pi: Vector, theta: Matrix) + private case class GaussianData(pi: Vector, theta: Matrix, sigma: Matrix) override protected def saveImpl(path: String): Unit = { // Save metadata and Params DefaultParamsWriter.saveMetadata(instance, path, sc) - // Save model data: pi, theta - val data = Data(instance.pi, instance.theta) val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + + instance.getModelType match { + case Multinomial | Bernoulli => + // Save model data: pi, theta + require(instance.sigma == null) + val data = Data(instance.pi, instance.theta) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + + case Gaussian => + require(instance.sigma != null) + val data = GaussianData(instance.pi, instance.theta, instance.sigma) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } } } private class NaiveBayesModelReader extends MLReader[NaiveBayesModel] { + import NaiveBayes._ /** Checked against metadata when loading model */ private val className = classOf[NaiveBayesModel].getName override def load(path: String): NaiveBayesModel = { + implicit val format = DefaultFormats val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion) + val modelTypeJson = metadata.getParamValue("modelType") + val modelType = Param.jsonDecode[String](compact(render(modelTypeJson))) val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) val vecConverted = MLUtils.convertVectorColumnsToML(data, "pi") - val Row(pi: Vector, theta: Matrix) = MLUtils.convertMatrixColumnsToML(vecConverted, "theta") - .select("pi", "theta") - .head() - val model = new NaiveBayesModel(metadata.uid, pi, theta) + + val model = if (major.toInt < 3 || modelType != Gaussian) { + val Row(pi: Vector, theta: Matrix) = + MLUtils.convertMatrixColumnsToML(vecConverted, "theta") + .select("pi", "theta") + .head() + new NaiveBayesModel(metadata.uid, pi, theta, null) + } else { + val Row(pi: Vector, theta: Matrix, sigma: Matrix) = + MLUtils.convertMatrixColumnsToML(vecConverted, "theta", "sigma") + .select("pi", "theta", "sigma") + .head() + new NaiveBayesModel(metadata.uid, pi, theta, sigma) + } metadata.getAndSetParams(model) model diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 9100ef1db6e12..9e4844ff89079 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -22,15 +22,15 @@ import scala.util.Random import breeze.linalg.{DenseVector => BDV, Vector => BV} import breeze.stats.distributions.{Multinomial => BrzMultinomial, RandBasis => BrzRandBasis} -import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.ml.classification.NaiveBayes.{Bernoulli, Multinomial} +import org.apache.spark.SparkException +import org.apache.spark.ml.classification.NaiveBayes._ import org.apache.spark.ml.classification.NaiveBayesSuite._ import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Dataset, Row} class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @@ -38,6 +38,8 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @transient var dataset: Dataset[_] = _ @transient var bernoulliDataset: Dataset[_] = _ + @transient var gaussianDataset: Dataset[_] = _ + @transient var gaussianDataset2: Dataset[_] = _ private val seed = 42 @@ -53,6 +55,23 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { dataset = generateNaiveBayesInput(pi, theta, 100, seed).toDF() bernoulliDataset = generateNaiveBayesInput(pi, theta, 100, seed, "bernoulli").toDF() + + // theta for gaussian nb + val theta2 = Array( + Array(0.70, 0.10, 0.10, 0.10), // label 0: mean + Array(0.10, 0.70, 0.10, 0.10), // label 1: mean + Array(0.10, 0.10, 0.70, 0.10) // label 2: mean + ) + + // sigma for gaussian nb + val sigma = Array( + Array(0.10, 0.10, 0.50, 0.10), // label 0: variance + Array(0.50, 0.10, 0.10, 0.10), // label 1: variance + Array(0.10, 0.10, 0.10, 0.50) // label 2: variance + ) + gaussianDataset = generateGaussianNaiveBayesInput(pi, theta2, sigma, 1000, seed).toDF() + gaussianDataset2 = spark.read.format("libsvm") + .load("../data/mllib/sample_multiclass_classification_data.txt") } def validatePrediction(predictionAndLabels: Seq[Row]): Unit = { @@ -67,10 +86,17 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { def validateModelFit( piData: Vector, thetaData: Matrix, + sigmaData: Matrix, model: NaiveBayesModel): Unit = { assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~== Vectors.dense(piData.toArray.map(math.exp)) absTol 0.05, "pi mismatch") assert(model.theta.map(math.exp) ~== thetaData.map(math.exp) absTol 0.05, "theta mismatch") + if (sigmaData == null) { + assert(model.sigma == null, "sigma mismatch") + } else { + assert(model.sigma.map(math.exp) ~== sigmaData.map(math.exp) absTol 0.05, + "sigma mismatch") + } } def expectedMultinomialProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { @@ -90,6 +116,19 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { Vectors.dense(classProbs.map(_ / classProbsSum)) } + def expectedGaussianProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { + val pi = model.pi.toArray.map(math.exp) + val classProbs = pi.indices.map { i => + feature.toArray.zipWithIndex.map { case (v, j) => + val mean = model.theta(i, j) + val variance = model.sigma(i, j) + math.exp(- (v - mean) * (v - mean) / variance / 2) / math.sqrt(variance * math.Pi * 2) + }.product * pi(i) + }.toArray + val classProbsSum = classProbs.sum + Vectors.dense(classProbs.map(_ / classProbsSum)) + } + def validateProbabilities( featureAndProbabilities: Seq[Row], model: NaiveBayesModel, @@ -102,6 +141,8 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { expectedMultinomialProbabilities(model, features) case Bernoulli => expectedBernoulliProbabilities(model, features) + case Gaussian => + expectedGaussianProbabilities(model, features) case _ => throw new IllegalArgumentException(s"Invalid modelType: $modelType.") } @@ -112,12 +153,14 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { test("model types") { assert(Multinomial === "multinomial") assert(Bernoulli === "bernoulli") + assert(Gaussian === "gaussian") } test("params") { ParamsSuite.checkParams(new NaiveBayes) val model = new NaiveBayesModel("nb", pi = Vectors.dense(Array(0.2, 0.8)), - theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4))) + theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4)), + sigma = null) ParamsSuite.checkParams(model) } @@ -146,7 +189,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("multinomial") val model = nb.fit(testDataset) - validateModelFit(pi, theta, model) + validateModelFit(pi, theta, null, model) assert(model.hasParent) MLTestingUtils.checkCopyAndUids(nb, model) @@ -192,12 +235,17 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { test("Naive Bayes with weighted samples") { val numClasses = 3 def modelEquals(m1: NaiveBayesModel, m2: NaiveBayesModel): Unit = { + assert(m1.getModelType === m2.getModelType) assert(m1.pi ~== m2.pi relTol 0.01) assert(m1.theta ~== m2.theta relTol 0.01) + if (m1.getModelType == Gaussian) { + assert(m1.sigma ~== m2.sigma relTol 0.01) + } } val testParams = Seq[(String, Dataset[_])]( ("bernoulli", bernoulliDataset), - ("multinomial", dataset) + ("multinomial", dataset), + ("gaussian", gaussianDataset) ) testParams.foreach { case (family, dataset) => // NaiveBayes is sensitive to constant scaling of the weights unless smoothing is set to 0 @@ -228,7 +276,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("bernoulli") val model = nb.fit(testDataset) - validateModelFit(pi, theta, model) + validateModelFit(pi, theta, null, model) assert(model.hasParent) val validationDataset = @@ -308,14 +356,112 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { } } + test("Naive Bayes Gaussian") { + val piArray = Array(0.5, 0.1, 0.4).map(math.log) + + val thetaArray = Array( + Array(0.70, 0.10, 0.10, 0.10), // label 0: mean + Array(0.10, 0.70, 0.10, 0.10), // label 1: mean + Array(0.10, 0.10, 0.70, 0.10) // label 2: mean + ) + + val sigmaArray = Array( + Array(0.10, 0.10, 0.50, 0.10), // label 0: variance + Array(0.50, 0.10, 0.10, 0.10), // label 1: variance + Array(0.10, 0.10, 0.10, 0.50) // label 2: variance + ) + + val pi = Vectors.dense(piArray) + val theta = new DenseMatrix(3, 4, thetaArray.flatten, true) + val sigma = new DenseMatrix(3, 4, sigmaArray.flatten, true) + + val nPoints = 10000 + val testDataset = + generateGaussianNaiveBayesInput(piArray, thetaArray, sigmaArray, nPoints, 42).toDF() + val gnb = new NaiveBayes().setModelType("gaussian") + val model = gnb.fit(testDataset) + + validateModelFit(pi, theta, sigma, model) + assert(model.hasParent) + + val validationDataset = + generateGaussianNaiveBayesInput(piArray, thetaArray, sigmaArray, nPoints, 17).toDF() + + val predictionAndLabels = model.transform(validationDataset).select("prediction", "label") + validatePrediction(predictionAndLabels.collect()) + + val featureAndProbabilities = model.transform(validationDataset) + .select("features", "probability") + validateProbabilities(featureAndProbabilities.collect(), model, "gaussian") + } + + test("Naive Bayes Gaussian - Model Coefficients") { + /* + Using the following Python code to verify the correctness. + + import numpy as np + from sklearn.naive_bayes import GaussianNB + from sklearn.datasets import load_svmlight_file + + path = "./data/mllib/sample_multiclass_classification_data.txt" + X, y = load_svmlight_file(path) + X = X.toarray() + clf = GaussianNB() + clf.fit(X, y) + + >>> clf.class_prior_ + array([0.33333333, 0.33333333, 0.33333333]) + >>> clf.theta_ + array([[ 0.27111101, -0.18833335, 0.54305072, 0.60500005], + [-0.60777778, 0.18166667, -0.84271174, -0.88000014], + [-0.09111114, -0.35833336, 0.10508474, 0.0216667 ]]) + >>> clf.sigma_ + array([[0.12230125, 0.07078052, 0.03430001, 0.05133607], + [0.03758145, 0.0988028 , 0.0033903 , 0.00782224], + [0.08058764, 0.06701387, 0.02486641, 0.02661392]]) + */ + + val gnb = new NaiveBayes().setModelType(Gaussian) + val model = gnb.fit(gaussianDataset2) + assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~= + Vectors.dense(0.33333333, 0.33333333, 0.33333333) relTol 1E-5) + + val thetaRows = model.theta.rowIter.toArray + assert(thetaRows(0) ~= + Vectors.dense(0.27111101, -0.18833335, 0.54305072, 0.60500005)relTol 1E-5) + assert(thetaRows(1) ~= + Vectors.dense(-0.60777778, 0.18166667, -0.84271174, -0.88000014)relTol 1E-5) + assert(thetaRows(2) ~= + Vectors.dense(-0.09111114, -0.35833336, 0.10508474, 0.0216667)relTol 1E-5) + + val sigmaRows = model.sigma.rowIter.toArray + assert(sigmaRows(0) ~= + Vectors.dense(0.12230125, 0.07078052, 0.03430001, 0.05133607)relTol 1E-5) + assert(sigmaRows(1) ~= + Vectors.dense(0.03758145, 0.0988028, 0.0033903, 0.00782224)relTol 1E-5) + assert(sigmaRows(2) ~= + Vectors.dense(0.08058764, 0.06701387, 0.02486641, 0.02661392)relTol 1E-5) + } + test("read/write") { def checkModelData(model: NaiveBayesModel, model2: NaiveBayesModel): Unit = { + assert(model.getModelType === model2.getModelType) assert(model.pi === model2.pi) assert(model.theta === model2.theta) + if (model.getModelType == "gaussian") { + assert(model.sigma === model2.sigma) + } else { + assert(model.sigma === null && model2.sigma === null) + } } val nb = new NaiveBayes() testEstimatorAndModelReadWrite(nb, dataset, NaiveBayesSuite.allParamSettings, NaiveBayesSuite.allParamSettings, checkModelData) + + val gnb = new NaiveBayes().setModelType("gaussian") + testEstimatorAndModelReadWrite(gnb, gaussianDataset, + NaiveBayesSuite.allParamSettingsForGaussian, + NaiveBayesSuite.allParamSettingsForGaussian, checkModelData) } test("should support all NumericType labels and weights, and not support other types") { @@ -324,6 +470,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { nb, spark) { (expected, actual) => assert(expected.pi === actual.pi) assert(expected.theta === actual.theta) + assert(expected.sigma === null && actual.sigma === null) } } } @@ -340,6 +487,16 @@ object NaiveBayesSuite { "smoothing" -> 0.1 ) + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettingsForGaussian: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "modelType" -> "gaussian" + ) + private def calcLabel(p: Double, pi: Array[Double]): Int = { var sum = 0.0 for (j <- 0 until pi.length) { @@ -384,4 +541,26 @@ object NaiveBayesSuite { LabeledPoint(y, Vectors.dense(xi)) } } + + // Generate input + def generateGaussianNaiveBayesInput( + pi: Array[Double], // 1XC + theta: Array[Array[Double]], // CXD + sigma: Array[Array[Double]], // CXD + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val D = theta(0).length + val rnd = new Random(seed) + val _pi = pi.map(math.exp) + + for (i <- 0 until nPoints) yield { + val y = calcLabel(rnd.nextDouble(), _pi) + val xi = Array.tabulate[Double] (D) { j => + val mean = theta(y)(j) + val variance = sigma(y)(j) + mean + rnd.nextGaussian() * math.sqrt(variance) + } + LabeledPoint(y, Vectors.dense(xi)) + } + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 8830061c0d9ed..617eb173f4f49 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -118,6 +118,9 @@ object MimaExcludes { // [SPARK-26632][Core] Separate Thread Configurations of Driver and Executor ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.network.netty.SparkTransportConf.fromSparkConf"), + // [SPARK-16872][ML][PYSPARK] Impl Gaussian Naive Bayes Classifier + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.NaiveBayesModel.this"), + // [SPARK-25765][ML] Add training cost to BisectingKMeans summary ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel.this"), diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 66b8ffd7bc8d8..03087c34b9d0b 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1878,7 +1878,8 @@ class _NaiveBayesParams(_JavaPredictorParams, HasWeightCol): smoothing = Param(Params._dummy(), "smoothing", "The smoothing parameter, should be >= 0, " + "default is 1.0", typeConverter=TypeConverters.toFloat) modelType = Param(Params._dummy(), "modelType", "The model type which is a string " + - "(case-sensitive). Supported options: multinomial (default) and bernoulli.", + "(case-sensitive). Supported options: multinomial (default), bernoulli " + + "and gaussian.", typeConverter=TypeConverters.toString) @since("1.5.0") @@ -1907,7 +1908,10 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, TF-IDF vectors, it can be used for document classification. By making every vector a binary (0/1) data, it can also be used as `Bernoulli NB `_. - The input feature values must be nonnegative. + The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + Since 3.0.0, it also supports Gaussian NB + `_. + which can handle continuous data. >>> from pyspark.sql import Row >>> from pyspark.ml.linalg import Vectors @@ -1925,6 +1929,8 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, DenseVector([-0.81..., -0.58...]) >>> model.theta DenseMatrix(2, 2, [-0.91..., -0.51..., -0.40..., -1.09...], 1) + >>> model.sigma == None + True >>> test0 = sc.parallelize([Row(features=Vectors.dense([1.0, 0.0]))]).toDF() >>> model.predict(test0.head().features) 1.0 @@ -1955,6 +1961,12 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, >>> result = model3.transform(test0).head() >>> result.prediction 0.0 + >>> nb3 = NaiveBayes().setModelType("gaussian") + >>> model4 = nb3.fit(df) + >>> model4.getModelType() + 'gaussian' + >>> model4.sigma + DenseMatrix(2, 2, [0.0, 0.25, 0.0, 0.0], 1) .. versionadded:: 1.5.0 """ @@ -2037,6 +2049,14 @@ def theta(self): """ return self._call_java("theta") + @property + @since("3.0.0") + def sigma(self): + """ + variance of each feature. + """ + return self._call_java("sigma") + class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, HasMaxIter, HasTol, HasStepSize, HasSolver): From d83cacfcf5510e0b466bbe17459811bb42d72250 Mon Sep 17 00:00:00 2001 From: xy_xin Date: Mon, 18 Nov 2019 11:48:56 +0800 Subject: [PATCH 0643/1387] [SPARK-29907][SQL] Move DELETE/UPDATE/MERGE relative rules to dmlStatementNoWith to support cte ### What changes were proposed in this pull request? SPARK-27444 introduced `dmlStatementNoWith` so that any dml that needs cte support can leverage it. It be better if we move DELETE/UPDATE/MERGE rules to `dmlStatementNoWith`. ### Why are the changes needed? Wit this change, we can support syntax like "With t AS (SELECT) DELETE FROM xxx", and so as UPDATE/MERGE. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? New cases added. Closes #26536 from xianyinxin/SPARK-29907. Authored-by: xy_xin Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 16 +-- .../command/PlanResolutionSuite.scala | 119 ++++++++++++++++-- 2 files changed, 116 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index d2d145606b627..c062377ee02a9 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -218,14 +218,6 @@ statement | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration - | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable - | UPDATE multipartIdentifier tableAlias setClause whereClause? #updateTable - | MERGE INTO target=multipartIdentifier targetAlias=tableAlias - USING (source=multipartIdentifier | - '(' sourceQuery=query')') sourceAlias=tableAlias - ON mergeCondition=booleanExpression - matchedClause* - notMatchedClause* #mergeIntoTable | unsupportedHiveNativeCommands .*? #failNativeCommand ; @@ -402,6 +394,14 @@ resource dmlStatementNoWith : insertInto queryTerm queryOrganization #singleInsertQuery | fromClause multiInsertQueryBody+ #multiInsertQuery + | DELETE FROM multipartIdentifier tableAlias whereClause? #deleteFromTable + | UPDATE multipartIdentifier tableAlias setClause whereClause? #updateTable + | MERGE INTO target=multipartIdentifier targetAlias=tableAlias + USING (source=multipartIdentifier | + '(' sourceQuery=query')') sourceAlias=tableAlias + ON mergeCondition=booleanExpression + matchedClause* + notMatchedClause* #mergeIntoTable ; queryOrganization diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 50e35289b0a64..d5c65a7c1e1b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -26,11 +26,11 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions.{EqualTo, IntegerLiteral, StringLiteral} +import org.apache.spark.sql.catalyst.expressions.{EqualTo, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeTable, DropTable, InsertAction, LogicalPlan, MergeIntoTable, Project, SubqueryAlias, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeTable, DropTable, InsertAction, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.InMemoryTableProvider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.execution.datasources.CreateTable @@ -138,6 +138,7 @@ class PlanResolutionSuite extends AnalysisTest { } val analyzer = new Analyzer(catalogManager, conf) val rules = Seq( + CTESubstitution, new ResolveCatalogs(catalogManager), new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), analyzer.ResolveTables) @@ -859,10 +860,16 @@ class PlanResolutionSuite extends AnalysisTest { val sql1 = s"DELETE FROM $tblName" val sql2 = s"DELETE FROM $tblName where name='Robert'" val sql3 = s"DELETE FROM $tblName AS t where t.name='Robert'" + val sql4 = + s""" + |WITH s(name) AS (SELECT 'Robert') + |DELETE FROM $tblName AS t WHERE t.name IN (SELECT s.name FROM s) + """.stripMargin val parsed1 = parseAndResolve(sql1) val parsed2 = parseAndResolve(sql2) val parsed3 = parseAndResolve(sql3) + val parsed4 = parseAndResolve(sql4) parsed1 match { case DeleteFromTable(_: DataSourceV2Relation, None) => @@ -874,7 +881,7 @@ class PlanResolutionSuite extends AnalysisTest { _: DataSourceV2Relation, Some(EqualTo(name: UnresolvedAttribute, StringLiteral("Robert")))) => assert(name.name == "name") - case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed1.treeString) + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed2.treeString) } parsed3 match { @@ -882,7 +889,24 @@ class PlanResolutionSuite extends AnalysisTest { SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), Some(EqualTo(name: UnresolvedAttribute, StringLiteral("Robert")))) => assert(name.name == "t.name") - case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed1.treeString) + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed3.treeString) + } + + parsed4 match { + case DeleteFromTable(SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + Some(InSubquery(values, query))) => + assert(values.size == 1 && values.head.isInstanceOf[UnresolvedAttribute]) + assert(values.head.asInstanceOf[UnresolvedAttribute].name == "t.name") + query match { + case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", None), + UnresolvedSubqueryColumnAliases(outputColumnNames, Project(_, _: OneRowRelation)))), + _, _, _) => + assert(projects.size == 1 && projects.head.name == "s.name") + assert(outputColumnNames.size == 1 && outputColumnNames.head == "name") + case o => fail("Unexpected subquery: \n" + o.treeString) + } + + case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed4.treeString) } } } @@ -892,10 +916,18 @@ class PlanResolutionSuite extends AnalysisTest { val sql1 = s"UPDATE $tblName SET name='Robert', age=32" val sql2 = s"UPDATE $tblName AS t SET name='Robert', age=32" val sql3 = s"UPDATE $tblName AS t SET name='Robert', age=32 WHERE p=1" + val sql4 = + s""" + |WITH s(name) AS (SELECT 'Robert') + |UPDATE $tblName AS t + |SET t.age=32 + |WHERE t.name IN (SELECT s.name FROM s) + """.stripMargin val parsed1 = parseAndResolve(sql1) val parsed2 = parseAndResolve(sql2) val parsed3 = parseAndResolve(sql3) + val parsed4 = parseAndResolve(sql4) parsed1 match { case UpdateTable( @@ -933,6 +965,25 @@ class PlanResolutionSuite extends AnalysisTest { case _ => fail("Expect UpdateTable, but got:\n" + parsed3.treeString) } + + parsed4 match { + case UpdateTable(SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + Seq(Assignment(key: UnresolvedAttribute, IntegerLiteral(32))), + Some(InSubquery(values, query))) => + assert(key.name == "t.age") + assert(values.size == 1 && values.head.isInstanceOf[UnresolvedAttribute]) + assert(values.head.asInstanceOf[UnresolvedAttribute].name == "t.name") + query match { + case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", None), + UnresolvedSubqueryColumnAliases(outputColumnNames, Project(_, _: OneRowRelation)))), + _, _, _) => + assert(projects.size == 1 && projects.head.name == "s.name") + assert(outputColumnNames.size == 1 && outputColumnNames.head == "name") + case o => fail("Unexpected subquery: \n" + o.treeString) + } + + case _ => fail("Expect UpdateTable, but got:\n" + parsed4.treeString) + } } val sql = "UPDATE non_existing SET id=1" @@ -1015,8 +1066,7 @@ class PlanResolutionSuite extends AnalysisTest { test("MERGE INTO TABLE") { Seq(("v2Table", "v2Table1"), ("testcat.tab", "testcat.tab1")).foreach { - case(target, source) => - + case(target, source) => // basic val sql1 = s""" @@ -1059,11 +1109,25 @@ class PlanResolutionSuite extends AnalysisTest { |WHEN NOT MATCHED AND (target.s='insert') | THEN INSERT (target.i, target.s) values (source.i, source.s) """.stripMargin + // cte + val sql5 = + s""" + |WITH source(i, s) AS + | (SELECT * FROM $source) + |MERGE INTO $target AS target + |USING source + |ON target.i = source.i + |WHEN MATCHED AND (target.s='delete') THEN DELETE + |WHEN MATCHED AND (target.s='update') THEN UPDATE SET target.s = source.s + |WHEN NOT MATCHED AND (target.s='insert') + |THEN INSERT (target.i, target.s) values (source.i, source.s) + """.stripMargin val parsed1 = parseAndResolve(sql1) val parsed2 = parseAndResolve(sql2) val parsed3 = parseAndResolve(sql3) val parsed4 = parseAndResolve(sql4) + val parsed5 = parseAndResolve(sql5) parsed1 match { case MergeIntoTable( @@ -1090,7 +1154,7 @@ class PlanResolutionSuite extends AnalysisTest { assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed2.treeString) + case _ => fail("Expect MergeIntoTable, but got:\n" + parsed1.treeString) } parsed2 match { @@ -1130,7 +1194,7 @@ class PlanResolutionSuite extends AnalysisTest { assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed2.treeString) + case _ => fail("Expect MergeIntoTable, but got:\n" + parsed3.treeString) } parsed4 match { @@ -1157,8 +1221,41 @@ class PlanResolutionSuite extends AnalysisTest { assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed2.treeString) - } + case _ => fail("Expect MergeIntoTable, but got:\n" + parsed4.treeString) + } + + parsed5 match { + case MergeIntoTable( + SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("source", None), + UnresolvedSubqueryColumnAliases(outputColumnNames, + Project(projects, _: DataSourceV2Relation))), + EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), + Seq(DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), + UpdateAction(Some(EqualTo(ul: UnresolvedAttribute, StringLiteral("update"))), + updateAssigns)), + Seq(InsertAction(Some(EqualTo(il: UnresolvedAttribute, StringLiteral("insert"))), + insertAssigns))) => + assert(outputColumnNames.size == 2 && + outputColumnNames.head == "i" && + outputColumnNames.last == "s") + assert(projects.size == 1 && projects.head.isInstanceOf[UnresolvedStar]) + assert(l.name == "target.i" && r.name == "source.i") + assert(dl.name == "target.s") + assert(ul.name == "target.s") + assert(il.name == "target.s") + assert(updateAssigns.size == 1) + assert(updateAssigns.head.key.isInstanceOf[UnresolvedAttribute] && + updateAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.s") + assert(updateAssigns.head.value.isInstanceOf[UnresolvedAttribute] && + updateAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.s") + assert(insertAssigns.head.key.isInstanceOf[UnresolvedAttribute] && + insertAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.i") + assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && + insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") + + case _ => fail("Expect MergeIntoTable, but got:\n" + parsed5.treeString) + } } // no aliases From f280c6aa54d80251da66ab370d32a7d93b01f225 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 18 Nov 2019 12:54:21 +0900 Subject: [PATCH 0644/1387] [SPARK-29378][R][FOLLOW-UP] Remove manual installation of Arrow dependencies in AppVeyor build ### What changes were proposed in this pull request? This PR remove manual installation of Arrow dependencies in AppVeyor build ### Why are the changes needed? It's unnecessary. See https://github.com/apache/spark/pull/26555#discussion_r347178368 ### Does this PR introduce any user-facing change? No ### How was this patch tested? AppVeyor will test. Closes #26566 from HyukjinKwon/SPARK-29378. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- appveyor.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/appveyor.yml b/appveyor.yml index c3baa1f82a399..00c688ba18eb6 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -43,7 +43,6 @@ install: - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" - - cmd: R -e "install.packages(c('assertthat', 'bit64', 'fs', 'purrr', 'R6', 'tidyselect'), repos='https://cloud.r-project.org/')" # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. # As of devtools 2.1.0, it requires testthat higher then 2.1.1 as a dependency. SparkR test requires testthat 1.0.2. # Therefore, we don't use devtools but installs it directly from the archive including its dependencies. From 42f8f79ff0311f7de629177b8609b2cbc73ad1c4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 17 Nov 2019 21:01:01 -0800 Subject: [PATCH 0645/1387] [SPARK-29936][R] Fix SparkR lint errors and add lint-r GitHub Action ### What changes were proposed in this pull request? This PR fixes SparkR lint errors and adds `lint-r` GitHub Action to protect the branch. ### Why are the changes needed? It turns out that we currently don't run it. It's recovered yesterday. However, after that, our Jenkins linter jobs (`master`/`branch-2.4`) has been broken on `lint-r` tasks. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the GitHub Action on this PR in addition to Jenkins R and AppVeyor R. Closes #26564 from dongjoon-hyun/SPARK-29936. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 25 ++++++++++++++++++++++++- R/pkg/.lintr | 2 +- R/pkg/R/DataFrame.R | 8 ++++---- R/pkg/R/SQLContext.R | 4 ++-- R/pkg/R/context.R | 2 +- R/pkg/R/group.R | 2 +- R/pkg/R/utils.R | 6 +++--- R/pkg/inst/worker/worker.R | 2 +- R/pkg/tests/fulltests/test_sparkSQL.R | 4 ++-- dev/lint-r.R | 2 +- 10 files changed, 40 insertions(+), 17 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 5eb00c4aba0f9..d2b7dca3684f0 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -50,7 +50,7 @@ jobs: lint: runs-on: ubuntu-latest - name: Linters + name: Linters (Java/Scala/Python), licenses, dependencies steps: - uses: actions/checkout@master - uses: actions/setup-java@v1 @@ -72,3 +72,26 @@ jobs: run: ./dev/check-license - name: Dependencies run: ./dev/test-dependencies.sh + + lintr: + runs-on: ubuntu-latest + name: Linter (R) + steps: + - uses: actions/checkout@master + - uses: actions/setup-java@v1 + with: + java-version: '11' + - name: install R + run: | + echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' | sudo tee -a /etc/apt/sources.list + sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9 + sudo apt-get update + sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev + - name: install R packages + run: | + sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" + sudo Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')" + - name: package and install SparkR + run: ./R/install-dev.sh + - name: lint-r + run: ./dev/lint-r diff --git a/R/pkg/.lintr b/R/pkg/.lintr index c83ad2adfe0ef..67dc1218ea551 100644 --- a/R/pkg/.lintr +++ b/R/pkg/.lintr @@ -1,2 +1,2 @@ -linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, object_name_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE)) +linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, object_name_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE), object_usage_linter = NULL, cyclocomp_linter = NULL) exclusions: list("inst/profile/general.R" = 1, "inst/profile/shell.R") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 6f3c7c120ba3c..593d3ca16220d 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2252,7 +2252,7 @@ setMethod("mutate", # The last column of the same name in the specific columns takes effect deDupCols <- list() - for (i in 1:length(cols)) { + for (i in seq_len(length(cols))) { deDupCols[[ns[[i]]]] <- alias(cols[[i]], ns[[i]]) } @@ -2416,7 +2416,7 @@ setMethod("arrange", # builds a list of columns of type Column # example: [[1]] Column Species ASC # [[2]] Column Petal_Length DESC - jcols <- lapply(seq_len(length(decreasing)), function(i){ + jcols <- lapply(seq_len(length(decreasing)), function(i) { if (decreasing[[i]]) { desc(getColumn(x, by[[i]])) } else { @@ -2749,7 +2749,7 @@ genAliasesForIntersectedCols <- function(x, intersectedColNames, suffix) { col <- getColumn(x, colName) if (colName %in% intersectedColNames) { newJoin <- paste(colName, suffix, sep = "") - if (newJoin %in% allColNames){ + if (newJoin %in% allColNames) { stop("The following column name: ", newJoin, " occurs more than once in the 'DataFrame'.", "Please use different suffixes for the intersected columns.") } @@ -3475,7 +3475,7 @@ setMethod("str", cat(paste0("'", class(object), "': ", length(names), " variables:\n")) if (nrow(localDF) > 0) { - for (i in 1 : ncol(localDF)) { + for (i in seq_len(ncol(localDF))) { # Get the first elements for each column firstElements <- if (types[i] == "character") { diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index cf96c658d34d2..f48a334ed6766 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -197,7 +197,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) { as.list(schema) } if (is.null(names)) { - names <- lapply(1:length(firstRow), function(x) { + names <- lapply(seq_len(length(firstRow)), function(x) { paste0("_", as.character(x)) }) } @@ -213,7 +213,7 @@ getSchema <- function(schema, firstRow = NULL, rdd = NULL) { }) types <- lapply(firstRow, infer_type) - fields <- lapply(1:length(firstRow), function(i) { + fields <- lapply(seq_len(length(firstRow)), function(i) { structField(names[[i]], types[[i]], TRUE) }) schema <- do.call(structType, fields) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 93ba1307043a3..d96a287f818a2 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -416,7 +416,7 @@ spark.getSparkFiles <- function(fileName) { #' @examples #'\dontrun{ #' sparkR.session() -#' doubled <- spark.lapply(1:10, function(x){2 * x}) +#' doubled <- spark.lapply(1:10, function(x) {2 * x}) #'} #' @note spark.lapply since 2.0.0 spark.lapply <- function(list, func) { diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 6e8f4dc3a7907..2b7995e1e37f6 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -162,7 +162,7 @@ methods <- c("avg", "max", "mean", "min", "sum") #' @note pivot since 2.0.0 setMethod("pivot", signature(x = "GroupedData", colname = "character"), - function(x, colname, values = list()){ + function(x, colname, values = list()) { stopifnot(length(colname) == 1) if (length(values) == 0) { result <- callJMethod(x@sgd, "pivot", colname) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index c3501977e64bc..15e2410d3a31c 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -131,7 +131,7 @@ hashCode <- function(key) { } else { asciiVals <- sapply(charToRaw(key), function(x) { strtoi(x, 16L) }) hashC <- 0 - for (k in 1:length(asciiVals)) { + for (k in seq_len(length(asciiVals))) { hashC <- mult31AndAdd(hashC, asciiVals[k]) } as.integer(hashC) @@ -724,7 +724,7 @@ assignNewEnv <- function(data) { stopifnot(length(cols) > 0) env <- new.env() - for (i in 1:length(cols)) { + for (i in seq_len(length(cols))) { assign(x = cols[i], value = data[, cols[i], drop = F], envir = env) } env @@ -750,7 +750,7 @@ launchScript <- function(script, combinedArgs, wait = FALSE, stdout = "", stderr if (.Platform$OS.type == "windows") { scriptWithArgs <- paste(script, combinedArgs, sep = " ") # on Windows, intern = F seems to mean output to the console. (documentation on this is missing) - shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) # nolint + shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) } else { # http://stat.ethz.ch/R-manual/R-devel/library/base/html/system2.html # stdout = F means discard output diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index dfe69b7f4f1fb..1ef05ea621e83 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -194,7 +194,7 @@ if (isEmpty != 0) { } else { # gapply mode outputs <- list() - for (i in 1:length(data)) { + for (i in seq_len(length(data))) { # Timing reading input data for execution inputElap <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index c2b2458ec064b..cb47353d600db 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -172,7 +172,7 @@ test_that("structField type strings", { typeList <- c(primitiveTypes, complexTypes) typeStrings <- names(typeList) - for (i in seq_along(typeStrings)){ + for (i in seq_along(typeStrings)) { typeString <- typeStrings[i] expected <- typeList[[i]] testField <- structField("_col", typeString) @@ -203,7 +203,7 @@ test_that("structField type strings", { errorList <- c(primitiveErrors, complexErrors) typeStrings <- names(errorList) - for (i in seq_along(typeStrings)){ + for (i in seq_along(typeStrings)) { typeString <- typeStrings[i] expected <- paste0("Unsupported type for SparkDataframe: ", errorList[[i]]) expect_error(structField("_col", typeString), expected) diff --git a/dev/lint-r.R b/dev/lint-r.R index a4261d266bbc0..7e165319e316a 100644 --- a/dev/lint-r.R +++ b/dev/lint-r.R @@ -27,7 +27,7 @@ if (! library(SparkR, lib.loc = LOCAL_LIB_LOC, logical.return = TRUE)) { # Installs lintr from Github in a local directory. # NOTE: The CRAN's version is too old to adapt to our rules. if ("lintr" %in% row.names(installed.packages()) == FALSE) { - devtools::install_github("jimhester/lintr@5431140") + devtools::install_github("jimhester/lintr@v2.0.0") } library(lintr) From ee3bd6d76887ccc4961fd520c5d03f7edd3742ac Mon Sep 17 00:00:00 2001 From: Zhou Jiang Date: Mon, 18 Nov 2019 05:44:00 +0000 Subject: [PATCH 0646/1387] [SPARK-25694][SQL] Add a config for `URL.setURLStreamHandlerFactory` ### What changes were proposed in this pull request? Add a property `spark.fsUrlStreamHandlerFactory.enabled` to allow users turn off the default registration of `org.apache.hadoop.fs.FsUrlStreamHandlerFactory` ### Why are the changes needed? This [SPARK-25694](https://issues.apache.org/jira/browse/SPARK-25694) is a long-standing issue. Originally, [[SPARK-12868][SQL] Allow adding jars from hdfs](https://github.com/apache/spark/pull/17342 ) added this for better Hive support. However, this have a side-effect when the users use Apache Spark without `-Phive`. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users. ### Does this PR introduce any user-facing change? Yes. This provides a new user-configurable property. By default, the behavior is unchanged. ### How was this patch tested? Manual testing. **BEFORE** ``` $ build/sbt package $ bin/spark-shell scala> sql("show tables").show +--------+---------+-----------+ |database|tableName|isTemporary| +--------+---------+-----------+ +--------+---------+-----------+ scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory()) java.lang.Error: factory already defined at java.net.URL.setURLStreamHandlerFactory(URL.java:1134) ... 47 elided ``` **AFTER** ``` $ build/sbt package $ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false scala> sql("show tables").show +--------+---------+-----------+ |database|tableName|isTemporary| +--------+---------+-----------+ +--------+---------+-----------+ scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory()) ``` Closes #26530 from jiangzho/master. Lead-authored-by: Zhou Jiang Co-authored-by: Dongjoon Hyun Co-authored-by: zhou-jiang Signed-off-by: DB Tsai --- .../spark/sql/internal/SharedState.scala | 25 ++++++++++++---- .../spark/sql/internal/config/package.scala | 29 +++++++++++++++++++ 2 files changed, 49 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index b810bedac471d..81a9c76511d8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.internal.config.DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils @@ -52,6 +53,8 @@ private[sql] class SharedState( initialConfigs: scala.collection.Map[String, String]) extends Logging { + SharedState.setFsUrlStreamHandlerFactory(sparkContext.conf) + // Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on // the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf. val warehousePath: String = { @@ -191,11 +194,23 @@ private[sql] class SharedState( } object SharedState extends Logging { - try { - URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) - } catch { - case e: Error => - logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory") + @volatile private var fsUrlStreamHandlerFactoryInitialized = false + + private def setFsUrlStreamHandlerFactory(conf: SparkConf): Unit = { + if (!fsUrlStreamHandlerFactoryInitialized && + conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) { + synchronized { + if (!fsUrlStreamHandlerFactoryInitialized) { + try { + URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) + fsUrlStreamHandlerFactoryInitialized = true + } catch { + case NonFatal(_) => + logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory") + } + } + } + } } private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala new file mode 100644 index 0000000000000..e26c4aadaf135 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.internal + +import org.apache.spark.internal.config.ConfigBuilder + +package object config { + + private[spark] val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED = + ConfigBuilder("spark.sql.defaultUrlStreamHandlerFactory.enabled") + .doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations") + .booleanConf + .createWithDefault(true) +} From 73912379d06e49e29aa8192ac4fb746d73c97fc5 Mon Sep 17 00:00:00 2001 From: gschiavon Date: Mon, 18 Nov 2019 16:07:05 +0900 Subject: [PATCH 0647/1387] [SPARK-29020][SQL] Improving array_sort behaviour ### What changes were proposed in this pull request? I've noticed that there are two functions to sort arrays sort_array and array_sort. sort_array is from 1.5.0 and it has the possibility of ordering both ascending and descending array_sort is from 2.4.0 and it only has the possibility of ordering in ascending. Basically I just added the possibility of ordering either ascending or descending using array_sort. I think it would be good to have unified behaviours and not having to user sort_array when you want to order in descending order. Imagine that you are new to spark, I'd like to be able to sort array using the newest spark functions. ### Why are the changes needed? Basically to be able to sort the array in descending order using *array_sort* instead of using *sort_array* from 1.5.0 ### Does this PR introduce any user-facing change? Yes, now you are able to sort the array in descending order. Note that it has the same behaviour with nulls than sort_array ### How was this patch tested? Test's added This is the link to the [jira](https://issues.apache.org/jira/browse/SPARK-29020) Closes #25728 from Gschiavon/improving-array-sort. Lead-authored-by: gschiavon Co-authored-by: Takuya UESHIN Co-authored-by: gschiavon Signed-off-by: HyukjinKwon --- .../expressions/collectionOperations.scala | 48 -------- .../expressions/higherOrderFunctions.scala | 108 ++++++++++++++++++ .../CollectionExpressionsSuite.scala | 10 -- .../HigherOrderFunctionsSuite.scala | 50 ++++++++ .../org/apache/spark/sql/functions.scala | 2 +- .../spark/sql/DataFrameFunctionsSuite.scala | 82 ++++++++++++- 6 files changed, 240 insertions(+), 60 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 5d964b602e634..d5d42510842ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -900,54 +900,6 @@ case class SortArray(base: Expression, ascendingOrder: Expression) override def prettyName: String = "sort_array" } - -/** - * Sorts the input array in ascending order according to the natural ordering of - * the array elements and returns it. - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = """ - _FUNC_(array) - Sorts the input array in ascending order. The elements of the input array must - be orderable. Null elements will be placed at the end of the returned array. - """, - examples = """ - Examples: - > SELECT _FUNC_(array('b', 'd', null, 'c', 'a')); - ["a","b","c","d",null] - """, - since = "2.4.0") -// scalastyle:on line.size.limit -case class ArraySort(child: Expression) extends UnaryExpression with ArraySortLike { - - override def dataType: DataType = child.dataType - override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType) - - override def arrayExpression: Expression = child - override def nullOrder: NullOrder = NullOrder.Greatest - - override def checkInputDataTypes(): TypeCheckResult = child.dataType match { - case ArrayType(dt, _) if RowOrdering.isOrderable(dt) => - TypeCheckResult.TypeCheckSuccess - case ArrayType(dt, _) => - val dtSimple = dt.catalogString - TypeCheckResult.TypeCheckFailure( - s"$prettyName does not support sorting array of type $dtSimple which is not orderable") - case _ => - TypeCheckResult.TypeCheckFailure(s"$prettyName only supports array input.") - } - - override def nullSafeEval(array: Any): Any = { - sortEval(array, true) - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, c => sortCodegen(ctx, ev, c, "true")) - } - - override def prettyName: String = "array_sort" -} - /** * Returns a random permutation of the given array. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index 317ebb62c07ec..adeda0981fe8e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.Comparator import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable @@ -285,6 +286,113 @@ case class ArrayTransform( override def prettyName: String = "transform" } +/** + * Sorts elements in an array using a comparator function. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """_FUNC_(expr, func) - Sorts the input array in ascending order. The elements of the + input array must be orderable. Null elements will be placed at the end of the returned + array. Since 3.0.0 this function also sorts and returns the array based on the given + comparator function. The comparator will take two arguments + representing two elements of the array. + It returns -1, 0, or 1 as the first element is less than, equal to, or greater + than the second element. If the comparator function returns other + values (including null), the function will fail and raise an error. + """, + examples = """ + Examples: + > SELECT _FUNC_(array(5, 6, 1), (left, right) -> case when left < right then -1 when left > right then 1 else 0 end); + [1,5,6] + > SELECT _FUNC_(array('bc', 'ab', 'dc'), (left, right) -> case when left is null and right is null then 0 when left is null then -1 when right is null then 1 when left < right then 1 when left > right then -1 else 0 end); + ["dc","bc","ab"] + > SELECT _FUNC_(array('b', 'd', null, 'c', 'a')); + ["a","b","c","d",null] + """, + since = "2.4.0") +// scalastyle:on line.size.limit +case class ArraySort( + argument: Expression, + function: Expression) + extends ArrayBasedSimpleHigherOrderFunction with CodegenFallback { + + def this(argument: Expression) = this(argument, ArraySort.defaultComparator) + + @transient lazy val elementType: DataType = + argument.dataType.asInstanceOf[ArrayType].elementType + + override def dataType: ArrayType = argument.dataType.asInstanceOf[ArrayType] + override def checkInputDataTypes(): TypeCheckResult = { + checkArgumentDataTypes() match { + case TypeCheckResult.TypeCheckSuccess => + argument.dataType match { + case ArrayType(dt, _) if RowOrdering.isOrderable(dt) => + if (function.dataType == IntegerType) { + TypeCheckResult.TypeCheckSuccess + } else { + TypeCheckResult.TypeCheckFailure("Return type of the given function has to be " + + "IntegerType") + } + case ArrayType(dt, _) => + val dtSimple = dt.catalogString + TypeCheckResult.TypeCheckFailure( + s"$prettyName does not support sorting array of type $dtSimple which is not " + + "orderable") + case _ => + TypeCheckResult.TypeCheckFailure(s"$prettyName only supports array input.") + } + case failure => failure + } + } + + override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): ArraySort = { + val ArrayType(elementType, containsNull) = argument.dataType + copy(function = + f(function, (elementType, containsNull) :: (elementType, containsNull) :: Nil)) + } + + @transient lazy val LambdaFunction(_, + Seq(firstElemVar: NamedLambdaVariable, secondElemVar: NamedLambdaVariable), _) = function + + def comparator(inputRow: InternalRow): Comparator[Any] = { + val f = functionForEval + (o1: Any, o2: Any) => { + firstElemVar.value.set(o1) + secondElemVar.value.set(o2) + f.eval(inputRow).asInstanceOf[Int] + } + } + + override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { + val arr = argumentValue.asInstanceOf[ArrayData].toArray[AnyRef](elementType) + if (elementType != NullType) { + java.util.Arrays.sort(arr, comparator(inputRow)) + } + new GenericArrayData(arr.asInstanceOf[Array[Any]]) + } + + override def prettyName: String = "array_sort" +} + +object ArraySort { + + def comparator(left: Expression, right: Expression): Expression = { + val lit0 = Literal(0) + val lit1 = Literal(1) + val litm1 = Literal(-1) + + If(And(IsNull(left), IsNull(right)), lit0, + If(IsNull(left), lit1, If(IsNull(right), litm1, + If(LessThan(left, right), litm1, If(GreaterThan(left, right), lit1, lit0))))) + } + + val defaultComparator: LambdaFunction = { + val left = UnresolvedNamedLambdaVariable(Seq("left")) + val right = UnresolvedNamedLambdaVariable(Seq("right")) + LambdaFunction(comparator(left, right), Seq(left, right)) + } +} + /** * Filters entries in a map using the provided function. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index 3287c83b1dd87..dd6d437d8063e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -364,16 +364,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper val arrayArrayStruct = Literal.create(Seq(aas2, aas1), typeAAS) checkEvaluation(new SortArray(arrayArrayStruct), Seq(aas1, aas2)) - - checkEvaluation(ArraySort(a0), Seq(1, 2, 3)) - checkEvaluation(ArraySort(a1), Seq[Integer]()) - checkEvaluation(ArraySort(a2), Seq("a", "b")) - checkEvaluation(ArraySort(a3), Seq("a", "b", null)) - checkEvaluation(ArraySort(a4), Seq(d1, d2)) - checkEvaluation(ArraySort(a5), Seq(null, null)) - checkEvaluation(ArraySort(arrayStruct), Seq(create_row(1), create_row(2))) - checkEvaluation(ArraySort(arrayArray), Seq(aa1, aa2)) - checkEvaluation(ArraySort(arrayArrayStruct), Seq(aas1, aas2)) } test("Array contains") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala index 4cdee447fa45a..e7b713840b884 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala @@ -84,6 +84,15 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper ArrayTransform(expr, createLambda(et, cn, IntegerType, false, f)).bind(validateBinding) } + def arraySort(expr: Expression): Expression = { + arraySort(expr, ArraySort.comparator) + } + + def arraySort(expr: Expression, f: (Expression, Expression) => Expression): Expression = { + val ArrayType(et, cn) = expr.dataType + ArraySort(expr, createLambda(et, cn, et, cn, f)).bind(validateBinding) + } + def filter(expr: Expression, f: Expression => Expression): Expression = { val ArrayType(et, cn) = expr.dataType ArrayFilter(expr, createLambda(et, cn, f)).bind(validateBinding) @@ -167,6 +176,47 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper Seq("[1, 3, 5]", null, "[4, 6]")) } + test("ArraySort") { + val a0 = Literal.create(Seq(2, 1, 3), ArrayType(IntegerType)) + val a1 = Literal.create(Seq[Integer](), ArrayType(IntegerType)) + val a2 = Literal.create(Seq("b", "a"), ArrayType(StringType)) + val a3 = Literal.create(Seq("b", null, "a"), ArrayType(StringType)) + val d1 = new Decimal().set(10) + val d2 = new Decimal().set(100) + val a4 = Literal.create(Seq(d2, d1), ArrayType(DecimalType(10, 0))) + val a5 = Literal.create(Seq(null, null), ArrayType(NullType)) + + val typeAS = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) + val arrayStruct = Literal.create(Seq(create_row(2), create_row(1)), typeAS) + + val typeAA = ArrayType(ArrayType(IntegerType)) + val aa1 = Array[java.lang.Integer](1, 2) + val aa2 = Array[java.lang.Integer](3, null, 4) + val arrayArray = Literal.create(Seq(aa2, aa1), typeAA) + + val typeAAS = ArrayType(ArrayType(StructType(StructField("a", IntegerType) :: Nil))) + val aas1 = Array(create_row(1)) + val aas2 = Array(create_row(2)) + val arrayArrayStruct = Literal.create(Seq(aas2, aas1), typeAAS) + + checkEvaluation(arraySort(a0), Seq(1, 2, 3)) + checkEvaluation(arraySort(a1), Seq[Integer]()) + checkEvaluation(arraySort(a2), Seq("a", "b")) + checkEvaluation(arraySort(a3), Seq("a", "b", null)) + checkEvaluation(arraySort(a4), Seq(d1, d2)) + checkEvaluation(arraySort(a5), Seq(null, null)) + checkEvaluation(arraySort(arrayStruct), Seq(create_row(1), create_row(2))) + checkEvaluation(arraySort(arrayArray), Seq(aa1, aa2)) + checkEvaluation(arraySort(arrayArrayStruct), Seq(aas1, aas2)) + + checkEvaluation(arraySort(a0, (left, right) => UnaryMinus(ArraySort.comparator(left, right))), + Seq(3, 2, 1)) + checkEvaluation(arraySort(a3, (left, right) => UnaryMinus(ArraySort.comparator(left, right))), + Seq(null, "b", "a")) + checkEvaluation(arraySort(a4, (left, right) => UnaryMinus(ArraySort.comparator(left, right))), + Seq(d2, d1)) + } + test("MapFilter") { def mapFilter(expr: Expression, f: (Expression, Expression) => Expression): Expression = { val MapType(kt, vt, vcn) = expr.dataType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 2ba34647dbca8..72e9e337c4258 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3335,7 +3335,7 @@ object functions { * @group collection_funcs * @since 2.4.0 */ - def array_sort(e: Column): Column = withExpr { ArraySort(e.expr) } + def array_sort(e: Column): Column = withExpr { new ArraySort(e.expr) } /** * Remove all elements that equal to element from the given array. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 06484908f5e73..6c6d29a539197 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -312,6 +312,86 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ) } + test("array_sort with lambda functions") { + + spark.udf.register("fAsc", (x: Int, y: Int) => { + if (x < y) -1 + else if (x == y) 0 + else 1 + }) + + spark.udf.register("fDesc", (x: Int, y: Int) => { + if (x < y) 1 + else if (x == y) 0 + else -1 + }) + + spark.udf.register("fString", (x: String, y: String) => { + if (x == null && y == null) 0 + else if (x == null) 1 + else if (y == null) -1 + else if (x < y) 1 + else if (x == y) 0 + else -1 + }) + + spark.udf.register("fStringLength", (x: String, y: String) => { + if (x == null && y == null) 0 + else if (x == null) 1 + else if (y == null) -1 + else if (x.length < y.length) -1 + else if (x.length == y.length) 0 + else 1 + }) + + val df1 = Seq(Array[Int](3, 2, 5, 1, 2)).toDF("a") + checkAnswer( + df1.selectExpr("array_sort(a, (x, y) -> fAsc(x, y))"), + Seq( + Row(Seq(1, 2, 2, 3, 5))) + ) + + checkAnswer( + df1.selectExpr("array_sort(a, (x, y) -> fDesc(x, y))"), + Seq( + Row(Seq(5, 3, 2, 2, 1))) + ) + + val df2 = Seq(Array[String]("bc", "ab", "dc")).toDF("a") + checkAnswer( + df2.selectExpr("array_sort(a, (x, y) -> fString(x, y))"), + Seq( + Row(Seq("dc", "bc", "ab"))) + ) + + val df3 = Seq(Array[String]("a", "abcd", "abc")).toDF("a") + checkAnswer( + df3.selectExpr("array_sort(a, (x, y) -> fStringLength(x, y))"), + Seq( + Row(Seq("a", "abc", "abcd"))) + ) + + val df4 = Seq((Array[Array[Int]](Array(2, 3, 1), Array(4, 2, 1, 4), + Array(1, 2)), "x")).toDF("a", "b") + checkAnswer( + df4.selectExpr("array_sort(a, (x, y) -> fAsc(cardinality(x), cardinality(y)))"), + Seq( + Row(Seq[Seq[Int]](Seq(1, 2), Seq(2, 3, 1), Seq(4, 2, 1, 4)))) + ) + + val df5 = Seq(Array[String]("bc", null, "ab", "dc")).toDF("a") + checkAnswer( + df5.selectExpr("array_sort(a, (x, y) -> fString(x, y))"), + Seq( + Row(Seq("dc", "bc", "ab", null))) + ) + + spark.sql("drop temporary function fAsc") + spark.sql("drop temporary function fDesc") + spark.sql("drop temporary function fString") + spark.sql("drop temporary function fStringLength") + } + test("sort_array/array_sort functions") { val df = Seq( (Array[Int](2, 1, 3), Array("b", "c", "a")), @@ -383,7 +463,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { assert(intercept[AnalysisException] { df3.selectExpr("array_sort(a)").collect() - }.getMessage().contains("only supports array input")) + }.getMessage().contains("argument 1 requires array type, however, '`a`' is of string type")) } def testSizeOfArray(sizeOfNull: Any): Unit = { From 5cebe587c7132fa6ea502084d45e0d8b203481b8 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 18 Nov 2019 15:42:22 +0800 Subject: [PATCH 0648/1387] [SPARK-29783][SQL] Support SQL Standard/ISO_8601 output style for interval type MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add 3 interval output types which are named as `SQL_STANDARD`, `ISO_8601`, `MULTI_UNITS`. And we add a new conf `spark.sql.dialect.intervalOutputStyle` for this. The `MULTI_UNITS` style displays the interval values in the former behavior and it is the default. The newly added `SQL_STANDARD`, `ISO_8601` styles can be found in the following table. Style | conf | Year-Month Interval | Day-Time Interval | Mixed Interval -- | -- | -- | -- | -- Format With Time Unit Designators | MULTI_UNITS | 1 year 2 mons | 1 days 2 hours 3 minutes 4.123456 seconds | interval 1 days 2 hours 3 minutes 4.123456 seconds SQL STANDARD | SQL_STANDARD | 1-2 | 3 4:05:06 | -1-2 3 -4:05:06 ISO8601 Basic Format| ISO_8601| P1Y2M| P3DT4H5M6S|P-1Y-2M3D-4H-5M-6S ### Why are the changes needed? for ANSI SQL support ### Does this PR introduce any user-facing change? yes,interval out now has 3 output styles ### How was this patch tested? add new unit tests cc cloud-fan maropu MaxGekk HyukjinKwon thanks. Closes #26418 from yaooqinn/SPARK-29783. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../spark/unsafe/types/CalendarInterval.java | 36 +----- .../unsafe/types/CalendarIntervalSuite.java | 30 ----- .../spark/sql/catalyst/expressions/Cast.scala | 18 +++ .../sql/catalyst/expressions/literals.scala | 1 + .../sql/catalyst/json/JacksonGenerator.scala | 11 +- .../sql/catalyst/util/IntervalUtils.scala | 106 ++++++++++++++++++ .../apache/spark/sql/internal/SQLConf.scala | 19 ++++ .../catalyst/util/IntervalUtilsSuite.scala | 64 +++++++++++ .../spark/sql/execution/HiveResult.scala | 16 ++- .../inputs/interval-display-iso_8601.sql | 3 + .../inputs/interval-display-sql_standard.sql | 3 + .../sql-tests/inputs/interval-display.sql | 14 +++ .../sql-tests/inputs/postgreSQL/interval.sql | 10 +- .../results/interval-display-iso_8601.sql.out | 21 ++++ .../interval-display-sql_standard.sql.out | 21 ++++ .../results/interval-display.sql.out | 21 ++++ .../results/postgreSQL/interval.sql.out | 29 ++++- .../apache/spark/sql/DateFunctionsSuite.scala | 10 +- .../ThriftServerQueryTestSuite.scala | 5 +- 19 files changed, 361 insertions(+), 77 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display.sql.out diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index d2abb36b0727c..0464e886e1441 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -18,7 +18,6 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; -import java.math.BigDecimal; import java.time.Duration; import java.time.Period; import java.time.temporal.ChronoUnit; @@ -80,39 +79,8 @@ public int compareTo(CalendarInterval that) { @Override public String toString() { - if (months == 0 && days == 0 && microseconds == 0) { - return "0 seconds"; - } - - StringBuilder sb = new StringBuilder(); - - if (months != 0) { - appendUnit(sb, months / 12, "years"); - appendUnit(sb, months % 12, "months"); - } - - appendUnit(sb, days, "days"); - - if (microseconds != 0) { - long rest = microseconds; - appendUnit(sb, rest / MICROS_PER_HOUR, "hours"); - rest %= MICROS_PER_HOUR; - appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes"); - rest %= MICROS_PER_MINUTE; - if (rest != 0) { - String s = BigDecimal.valueOf(rest, 6).stripTrailingZeros().toPlainString(); - sb.append(s).append(" seconds "); - } - } - - sb.setLength(sb.length() - 1); - return sb.toString(); - } - - private void appendUnit(StringBuilder sb, long value, String unit) { - if (value != 0) { - sb.append(value).append(' ').append(unit).append(' '); - } + return "CalendarInterval(months= " + months + ", days = " + days + ", microsecond = " + + microseconds + ")"; } /** diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 6397f26c02f3a..01bf7eb2438ad 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -46,36 +46,6 @@ public void equalsTest() { assertEquals(i1, i6); } - @Test - public void toStringTest() { - CalendarInterval i; - - i = new CalendarInterval(0, 0, 0); - assertEquals("0 seconds", i.toString()); - - i = new CalendarInterval(34, 0, 0); - assertEquals("2 years 10 months", i.toString()); - - i = new CalendarInterval(-34, 0, 0); - assertEquals("-2 years -10 months", i.toString()); - - i = new CalendarInterval(0, 31, 0); - assertEquals("31 days", i.toString()); - - i = new CalendarInterval(0, -31, 0); - assertEquals("-31 days", i.toString()); - - i = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); - assertEquals("3 hours 13 minutes 0.000123 seconds", i.toString()); - - i = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123); - assertEquals("-3 hours -13 minutes -0.000123 seconds", i.toString()); - - i = new CalendarInterval(34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); - assertEquals("2 years 10 months 31 days 3 hours 13 minutes 0.000123 seconds", - i.toString()); - } - @Test public void periodAndDurationTest() { CalendarInterval interval = new CalendarInterval(120, -40, 123456); 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 9e802edbc5edd..ef99c8efef834 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 @@ -30,7 +30,9 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.IntervalStyle._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -281,6 +283,14 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { + case CalendarIntervalType => SQLConf.get.intervalOutputStyle match { + case SQL_STANDARD => + buildCast[CalendarInterval](_, i => UTF8String.fromString(toSqlStandardString(i))) + case ISO_8601 => + buildCast[CalendarInterval](_, i => UTF8String.fromString(toIso8601String(i))) + case MULTI_UNITS => + buildCast[CalendarInterval](_, i => UTF8String.fromString(toMultiUnitsString(i))) + } case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes) case DateType => buildCast[Int](_, d => UTF8String.fromString(dateFormatter.format(d))) case TimestampType => buildCast[Long](_, @@ -985,6 +995,14 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit timestampFormatter.getClass) (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString( org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($tf, $c));""" + case CalendarIntervalType => + val iu = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") + val funcName = SQLConf.get.intervalOutputStyle match { + case SQL_STANDARD => "toSqlStandardString" + case ISO_8601 => "toIso8601String" + case MULTI_UNITS => "toMultiUnitsString" + } + (c, evPrim, _) => code"""$evPrim = UTF8String.fromString($iu.$funcName($c));""" case ArrayType(et, _) => (c, evPrim, evNull) => { val buffer = ctx.freshVariable("buffer", classOf[UTF8StringBuilder]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 5a5d7a17acd99..f426863de41a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -409,6 +409,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) s"TIMESTAMP('${formatter.format(v)}')" case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'" + case (v: CalendarInterval, CalendarIntervalType) => IntervalUtils.toMultiUnitsString(v) case _ => value.toString } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index aaf2ecf7923ce..544f6e9108325 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -119,6 +119,10 @@ private[sql] class JacksonGenerator( (row: SpecializedGetters, ordinal: Int) => gen.writeNumber(row.getDouble(ordinal)) + case CalendarIntervalType => + (row: SpecializedGetters, ordinal: Int) => + gen.writeString(IntervalUtils.toMultiUnitsString(row.getInterval(ordinal))) + case StringType => (row: SpecializedGetters, ordinal: Int) => gen.writeString(row.getUTF8String(ordinal).toString) @@ -214,10 +218,15 @@ private[sql] class JacksonGenerator( private def writeMapData( map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = { val keyArray = map.keyArray() + val keyString = mapType.keyType match { + case CalendarIntervalType => + (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i)) + case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString + } val valueArray = map.valueArray() var i = 0 while (i < map.numElements()) { - gen.writeFieldName(keyArray.get(i, mapType.keyType).toString) + gen.writeFieldName(keyString(i)) if (!valueArray.isNullAt(i)) { fieldWriter.apply(valueArray, i) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 991312bff30aa..104c65b6cd037 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.util +import java.math.BigDecimal import java.util.concurrent.TimeUnit import scala.util.control.NonFatal @@ -424,6 +425,111 @@ object IntervalUtils { fromDoubles(interval.months / num, interval.days / num, interval.microseconds / num) } + def toMultiUnitsString(interval: CalendarInterval): String = { + if (interval.months == 0 && interval.days == 0 && interval.microseconds == 0) { + return "0 seconds" + } + val sb = new StringBuilder + if (interval.months != 0) { + appendUnit(sb, interval.months / 12, "years") + appendUnit(sb, interval.months % 12, "months") + } + appendUnit(sb, interval.days, "days") + if (interval.microseconds != 0) { + var rest = interval.microseconds + appendUnit(sb, rest / MICROS_PER_HOUR, "hours") + rest %= MICROS_PER_HOUR + appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes") + rest %= MICROS_PER_MINUTE + if (rest != 0) { + val s = BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString + sb.append(s).append(" seconds ") + } + } + sb.setLength(sb.length - 1) + sb.toString + } + + private def appendUnit(sb: StringBuilder, value: Long, unit: String): Unit = { + if (value != 0) sb.append(value).append(' ').append(unit).append(' ') + } + + def toSqlStandardString(interval: CalendarInterval): String = { + val yearMonthPart = if (interval.months < 0) { + val ma = math.abs(interval.months) + "-" + ma / 12 + "-" + ma % 12 + } else if (interval.months > 0) { + "+" + interval.months / 12 + "-" + interval.months % 12 + } else { + "" + } + + val dayPart = if (interval.days < 0) { + interval.days.toString + } else if (interval.days > 0) { + "+" + interval.days + } else { + "" + } + + val timePart = if (interval.microseconds != 0) { + val sign = if (interval.microseconds > 0) "+" else "-" + val sb = new StringBuilder(sign) + var rest = math.abs(interval.microseconds) + sb.append(rest / MICROS_PER_HOUR) + sb.append(':') + rest %= MICROS_PER_HOUR + val minutes = rest / MICROS_PER_MINUTE; + if (minutes < 10) { + sb.append(0) + } + sb.append(minutes) + sb.append(':') + rest %= MICROS_PER_MINUTE + val bd = BigDecimal.valueOf(rest, 6) + if (bd.compareTo(new BigDecimal(10)) < 0) { + sb.append(0) + } + val s = bd.stripTrailingZeros().toPlainString + sb.append(s) + sb.toString() + } else { + "" + } + + val intervalList = Seq(yearMonthPart, dayPart, timePart).filter(_.nonEmpty) + if (intervalList.nonEmpty) intervalList.mkString(" ") else "0" + } + + def toIso8601String(interval: CalendarInterval): String = { + val sb = new StringBuilder("P") + + val year = interval.months / 12 + if (year != 0) sb.append(year + "Y") + val month = interval.months % 12 + if (month != 0) sb.append(month + "M") + + if (interval.days != 0) sb.append(interval.days + "D") + + if (interval.microseconds != 0) { + sb.append('T') + var rest = interval.microseconds + val hour = rest / MICROS_PER_HOUR + if (hour != 0) sb.append(hour + "H") + rest %= MICROS_PER_HOUR + val minute = rest / MICROS_PER_MINUTE + if (minute != 0) sb.append(minute + "M") + rest %= MICROS_PER_MINUTE + if (rest != 0) { + val bd = BigDecimal.valueOf(rest, 6) + sb.append(bd.stripTrailingZeros().toPlainString + "S") + } + } else if (interval.days == 0 && interval.months == 0) { + sb.append("T0S") + } + sb.toString() + } + private object ParseState extends Enumeration { type ParseState = Value diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8b714d6dcdda3..d4fcefe99ee75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1782,6 +1782,23 @@ object SQLConf { .checkValues(StoreAssignmentPolicy.values.map(_.toString)) .createWithDefault(StoreAssignmentPolicy.ANSI.toString) + object IntervalStyle extends Enumeration { + type IntervalStyle = Value + val SQL_STANDARD, ISO_8601, MULTI_UNITS = Value + } + + val INTERVAL_STYLE = buildConf("spark.sql.intervalOutputStyle") + .doc("When converting interval values to strings (i.e. for display), this config decides the" + + " interval string format. The value SQL_STANDARD will produce output matching SQL standard" + + " interval literals (i.e. '+3-2 +10 -00:00:01'). The value ISO_8601 will produce output" + + " matching the ISO 8601 standard (i.e. 'P3Y2M10DT-1S'). The value MULTI_UNITS (which is the" + + " default) will produce output in form of value unit pairs, (i.e. '3 year 2 months 10 days" + + " -1 seconds'") + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValues(IntervalStyle.values.map(_.toString)) + .createWithDefault(IntervalStyle.MULTI_UNITS.toString) + val SORT_BEFORE_REPARTITION = buildConf("spark.sql.execution.sortBeforeRepartition") .internal() @@ -2496,6 +2513,8 @@ class SQLConf extends Serializable with Logging { def storeAssignmentPolicy: StoreAssignmentPolicy.Value = StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY)) + def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE)) + def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index f919bd1644871..65cfe896b2024 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -298,4 +298,68 @@ class IntervalUtilsSuite extends SparkFunSuite { new CalendarInterval(-2, 0, -1 * MICROS_PER_HOUR)) intercept[ArithmeticException](justifyInterval(new CalendarInterval(2, 0, Long.MaxValue))) } + + test("to ansi sql standard string") { + val i1 = new CalendarInterval(0, 0, 0) + assert(IntervalUtils.toSqlStandardString(i1) === "0") + val i2 = new CalendarInterval(34, 0, 0) + assert(IntervalUtils.toSqlStandardString(i2) === "+2-10") + val i3 = new CalendarInterval(-34, 0, 0) + assert(IntervalUtils.toSqlStandardString(i3) === "-2-10") + val i4 = new CalendarInterval(0, 31, 0) + assert(IntervalUtils.toSqlStandardString(i4) === "+31") + val i5 = new CalendarInterval(0, -31, 0) + assert(IntervalUtils.toSqlStandardString(i5) === "-31") + val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toSqlStandardString(i6) === "+3:13:00.000123") + val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) + assert(IntervalUtils.toSqlStandardString(i7) === "-3:13:00.000123") + val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toSqlStandardString(i8) === "-2-10 +31 +3:13:00.000123") + val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) + assert(IntervalUtils.toSqlStandardString(i9) === "-3000:00:00") + } + + test("to iso 8601 string") { + val i1 = new CalendarInterval(0, 0, 0) + assert(IntervalUtils.toIso8601String(i1) === "PT0S") + val i2 = new CalendarInterval(34, 0, 0) + assert(IntervalUtils.toIso8601String(i2) === "P2Y10M") + val i3 = new CalendarInterval(-34, 0, 0) + assert(IntervalUtils.toIso8601String(i3) === "P-2Y-10M") + val i4 = new CalendarInterval(0, 31, 0) + assert(IntervalUtils.toIso8601String(i4) === "P31D") + val i5 = new CalendarInterval(0, -31, 0) + assert(IntervalUtils.toIso8601String(i5) === "P-31D") + val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toIso8601String(i6) === "PT3H13M0.000123S") + val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) + assert(IntervalUtils.toIso8601String(i7) === "PT-3H-13M-0.000123S") + val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toIso8601String(i8) === "P-2Y-10M31DT3H13M0.000123S") + val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) + assert(IntervalUtils.toIso8601String(i9) === "PT-3000H") + } + + test("to multi units string") { + val i1 = new CalendarInterval(0, 0, 0) + assert(IntervalUtils.toMultiUnitsString(i1) === "0 seconds") + val i2 = new CalendarInterval(34, 0, 0) + assert(IntervalUtils.toMultiUnitsString(i2) === "2 years 10 months") + val i3 = new CalendarInterval(-34, 0, 0) + assert(IntervalUtils.toMultiUnitsString(i3) === "-2 years -10 months") + val i4 = new CalendarInterval(0, 31, 0) + assert(IntervalUtils.toMultiUnitsString(i4) === "31 days") + val i5 = new CalendarInterval(0, -31, 0) + assert(IntervalUtils.toMultiUnitsString(i5) === "-31 days") + val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toMultiUnitsString(i6) === "3 hours 13 minutes 0.000123 seconds") + val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) + assert(IntervalUtils.toMultiUnitsString(i7) === "-3 hours -13 minutes -0.000123 seconds") + val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) + assert(IntervalUtils.toMultiUnitsString(i8) === + "-2 years -10 months 31 days 3 hours 13 minutes 0.000123 seconds") + val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) + assert(IntervalUtils.toMultiUnitsString(i9) === "-3000 hours") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index 75abac4cfd1da..d4e10b3ffc733 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -22,9 +22,12 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.execution.command.{DescribeCommandBase, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.IntervalStyle._ import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval /** * Runs a query returning the result in Hive compatible form. @@ -97,7 +100,12 @@ object HiveResult { case (null, _) => "null" case (s: String, StringType) => "\"" + s + "\"" case (decimal, DecimalType()) => decimal.toString - case (interval, CalendarIntervalType) => interval.toString + case (interval: CalendarInterval, CalendarIntervalType) => + SQLConf.get.intervalOutputStyle match { + case SQL_STANDARD => toSqlStandardString(interval) + case ISO_8601 => toIso8601String(interval) + case MULTI_UNITS => toMultiUnitsString(interval) + } case (other, tpe) if primitiveTypes contains tpe => other.toString } @@ -120,6 +128,12 @@ object HiveResult { DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) + case (interval: CalendarInterval, CalendarIntervalType) => + SQLConf.get.intervalOutputStyle match { + case SQL_STANDARD => toSqlStandardString(interval) + case ISO_8601 => toIso8601String(interval) + case MULTI_UNITS => toMultiUnitsString(interval) + } case (interval, CalendarIntervalType) => interval.toString case (other, _ : UserDefinedType[_]) => other.toString case (other, tpe) if primitiveTypes.contains(tpe) => other.toString diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql new file mode 100644 index 0000000000000..62f3f43bd2c48 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql @@ -0,0 +1,3 @@ +-- tests for interval output style with iso_8601 format +--SET spark.sql.intervalOutputStyle = ISO_8601 +--import interval-display.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql new file mode 100644 index 0000000000000..375b4899e760e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql @@ -0,0 +1,3 @@ +-- tests for interval output style with sql standard format +--SET spark.sql.intervalOutputStyle = SQL_STANDARD +--import interval-display.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql new file mode 100644 index 0000000000000..ae19f1b6374ba --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql @@ -0,0 +1,14 @@ +-- tests for interval output style + +SELECT + cast(null as interval), -- null + interval '0 day', -- 0 + interval '1 year', -- year only + interval '1 month', -- month only + interval '1 year 2 month', -- year month only + interval '1 day -1 hours', + interval '-1 day -1 hours', + interval '-1 day 1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql index 01df2a3fd1b21..3b25ef7334c0a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql @@ -270,10 +270,12 @@ SELECT interval '1 2:03:04' minute to second; -- test output of couple non-standard interval values in the sql style -- [SPARK-29406] Interval output styles -- SET IntervalStyle TO sql_standard; --- SELECT interval '1 day -1 hours', --- interval '-1 days +1 hours', --- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', --- - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; +set spark.sql.intervalOutputStyle=SQL_STANDARD; +SELECT interval '1 day -1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; +set spark.sql.intervalOutputStyle=MULTI_UNITS; -- test outputting iso8601 intervals -- [SPARK-29406] Interval output styles diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out new file mode 100644 index 0000000000000..57fe8a3f4fcc6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out @@ -0,0 +1,21 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 1 + + +-- !query 0 +SELECT + cast(null as interval), -- null + interval '0 day', -- 0 + interval '1 year', -- year only + interval '1 month', -- month only + interval '1 year 2 month', -- year month only + interval '1 day -1 hours', + interval '-1 day -1 hours', + interval '-1 day 1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' +-- !query 0 schema +struct +-- !query 0 output +NULL PT0S P1Y P1M P1Y2M P1DT-1H P-1DT-1H P-1DT1H P-1DT1H P1Y2M-3DT4H5M6.789S P-1Y-2M3DT-4H-5M-6.789S diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out new file mode 100644 index 0000000000000..9e40f52151475 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out @@ -0,0 +1,21 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 1 + + +-- !query 0 +SELECT + cast(null as interval), -- null + interval '0 day', -- 0 + interval '1 year', -- year only + interval '1 month', -- month only + interval '1 year 2 month', -- year month only + interval '1 day -1 hours', + interval '-1 day -1 hours', + interval '-1 day 1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' +-- !query 0 schema +struct +-- !query 0 output +NULL 0 +1-0 +0-1 +1-2 +1 -1:00:00 -1 -1:00:00 -1 +1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out new file mode 100644 index 0000000000000..340496e404326 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out @@ -0,0 +1,21 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 1 + + +-- !query 0 +SELECT + cast(null as interval), -- null + interval '0 day', -- 0 + interval '1 year', -- year only + interval '1 month', -- month only + interval '1 year 2 month', -- year month only + interval '1 day -1 hours', + interval '-1 day -1 hours', + interval '-1 day 1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' +-- !query 0 schema +struct +-- !query 0 output +NULL 0 seconds 1 years 1 months 1 years 2 months 1 days -1 hours -1 days -1 hours -1 days 1 hours -1 days 1 hours 1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds -1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index 5ef1f8e5f0680..d981ed15e37f4 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 27 +-- Number of queries: 30 -- !query 0 @@ -216,3 +216,30 @@ SELECT interval '1 2:03:04' minute to second struct<1 days 2 hours 3 minutes 4 seconds:interval> -- !query 26 output 1 days 2 hours 3 minutes 4 seconds + + +-- !query 27 +set spark.sql.intervalOutputStyle=SQL_STANDARD +-- !query 27 schema +struct +-- !query 27 output +spark.sql.intervalOutputStyle SQL_STANDARD + + +-- !query 28 +SELECT interval '1 day -1 hours', + interval '-1 days +1 hours', + interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', + - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' +-- !query 28 schema +struct<1 days -1 hours:interval,-1 days 1 hours:interval,1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds:interval,-1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds:interval> +-- !query 28 output ++1 -1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 + + +-- !query 29 +set spark.sql.intervalOutputStyle=MULTI_UNITS +-- !query 29 schema +struct +-- !query 29 output +spark.sql.intervalOutputStyle MULTI_UNITS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 6ea37baeaf24c..c80e675b149d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -23,7 +23,7 @@ import java.time.Instant import java.util.Locale import java.util.concurrent.TimeUnit -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -293,10 +293,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val i = new CalendarInterval(2, 2, 2000000L) val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( - df.selectExpr(s"d + INTERVAL'$i'"), + df.selectExpr(s"d + INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), Seq(Row(Date.valueOf("2015-10-02")), Row(Date.valueOf("2016-03-02")))) checkAnswer( - df.selectExpr(s"t + INTERVAL'$i'"), + df.selectExpr(s"t + INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), Seq(Row(Timestamp.valueOf("2015-10-03 00:00:01")), Row(Timestamp.valueOf("2016-03-02 00:00:02")))) } @@ -309,10 +309,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val i = new CalendarInterval(2, 2, 2000000L) val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( - df.selectExpr(s"d - INTERVAL'$i'"), + df.selectExpr(s"d - INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), Seq(Row(Date.valueOf("2015-07-27")), Row(Date.valueOf("2015-12-26")))) checkAnswer( - df.selectExpr(s"t - INTERVAL'$i'"), + df.selectExpr(s"t - INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), Seq(Row(Timestamp.valueOf("2015-07-29 23:59:59")), Row(Timestamp.valueOf("2015-12-27 00:00:00")))) } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 82da4c049fd41..b50a6045f5378 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -93,7 +93,10 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { "subquery/in-subquery/in-group-by.sql", "subquery/in-subquery/simple-in.sql", "subquery/in-subquery/in-order-by.sql", - "subquery/in-subquery/in-set-operations.sql" + "subquery/in-subquery/in-set-operations.sql", + // SPARK-29783: need to set conf + "interval-display-iso_8601.sql", + "interval-display-sql_standard.sql" ) override def runQueries( From 50f6d930dafc508d1e5162dd0dd580d1f91bfbd1 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 18 Nov 2019 15:50:06 +0800 Subject: [PATCH 0649/1387] [SPARK-29870][SQL] Unify the logic of multi-units interval string to CalendarInterval ### What changes were proposed in this pull request? We now have two different implementation for multi-units interval strings to CalendarInterval type values. One is used to covert interval string literals to CalendarInterval. This approach will re-delegate the interval string to spark parser which handles the string as a `singleInterval` -> `multiUnitsInterval` -> eventually call `IntervalUtils.fromUnitStrings` The other is used in `Cast`, which eventually calls `IntervalUtils.stringToInterval`. This approach is ~10 times faster than the other. We should unify these two for better performance and simple logic. this pr uses the 2nd approach. ### Why are the changes needed? We should unify these two for better performance and simple logic. ### Does this PR introduce any user-facing change? no ### How was this patch tested? we shall not fail on existing uts Closes #26491 from yaooqinn/SPARK-29870. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 - .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/TimeWindow.scala | 3 +- .../sql/catalyst/parser/AstBuilder.scala | 34 ++-- .../sql/catalyst/parser/ParseDriver.scala | 9 -- .../sql/catalyst/util/IntervalUtils.scala | 147 +++++------------- .../CollectionExpressionsSuite.scala | 37 +++-- .../expressions/DateExpressionsSuite.scala | 22 +-- .../expressions/HashExpressionsSuite.scala | 4 +- .../IntervalExpressionsSuite.scala | 16 +- .../expressions/MutableProjectionSuite.scala | 4 +- .../expressions/ObjectExpressionsSuite.scala | 3 +- .../expressions/UnsafeRowConverterSuite.scala | 3 +- .../parser/ExpressionParserSuite.scala | 13 +- .../catalyst/util/IntervalUtilsSuite.scala | 75 +++++---- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../execution/streaming/GroupStateImpl.scala | 4 +- .../sql/execution/streaming/Triggers.scala | 3 +- .../execution/command/DDLParserSuite.scala | 2 +- 19 files changed, 169 insertions(+), 222 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index c062377ee02a9..de0353494f003 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -79,10 +79,6 @@ singleTableSchema : colTypeList EOF ; -singleInterval - : INTERVAL? multiUnitsInterval EOF - ; - statement : query #statementDefault | ctes? dmlStatementNoWith #dmlStatement 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 ef99c8efef834..8d11f4663a3ef 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 @@ -477,7 +477,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // IntervalConverter private[this] def castToInterval(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, s => IntervalUtils.stringToInterval(s)) + buildCast[UTF8String](_, s => IntervalUtils.safeStringToInterval(s)) } // LongConverter @@ -1234,7 +1234,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case StringType => val util = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") (c, evPrim, evNull) => - code"""$evPrim = $util.stringToInterval($c); + code"""$evPrim = $util.safeStringToInterval($c); if(${evPrim} == null) { ${evNull} = true; } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 69badb9562dc3..caacb71814f17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String case class TimeWindow( timeColumn: Expression, @@ -103,7 +104,7 @@ object TimeWindow { * precision. */ private def getIntervalInMicroSeconds(interval: String): Long = { - val cal = IntervalUtils.fromString(interval) + val cal = IntervalUtils.stringToInterval(UTF8String.fromString(interval)) if (cal.months != 0) { throw new IllegalArgumentException( s"Intervals greater than a month is not supported ($interval).") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 6be163b4b3c2a..68b1862a5d236 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -102,10 +102,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging withOrigin(ctx)(StructType(visitColTypeList(ctx.colTypeList))) } - override def visitSingleInterval(ctx: SingleIntervalContext): CalendarInterval = { - withOrigin(ctx)(visitMultiUnitsInterval(ctx.multiUnitsInterval)) - } - /* ******************************************************************************************** * Plan parsing * ******************************************************************************************** */ @@ -1870,7 +1866,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "INTERVAL" => val interval = try { - IntervalUtils.fromString(value) + IntervalUtils.stringToInterval(UTF8String.fromString(value)) } catch { case e: IllegalArgumentException => val ex = new ParseException("Cannot parse the INTERVAL value: " + value, ctx) @@ -2069,22 +2065,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitMultiUnitsInterval(ctx: MultiUnitsIntervalContext): CalendarInterval = { withOrigin(ctx) { - val units = ctx.intervalUnit().asScala.map { unit => - val u = unit.getText.toLowerCase(Locale.ROOT) - // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... - if (u.endsWith("s")) u.substring(0, u.length - 1) else u - }.map(IntervalUtils.IntervalUnit.withName).toArray - - val values = ctx.intervalValue().asScala.map { value => - if (value.STRING() != null) { - string(value.STRING()) - } else { - value.getText - } - }.toArray - + val units = ctx.intervalUnit().asScala + val values = ctx.intervalValue().asScala try { - IntervalUtils.fromUnitStrings(units, values) + assert(units.length == values.length) + val kvs = units.indices.map { i => + val u = units(i).getText + val v = if (values(i).STRING() != null) { + string(values(i).STRING()) + } else { + values(i).getText + } + UTF8String.fromString(" " + v + " " + u) + } + IntervalUtils.stringToInterval(UTF8String.concat(kvs: _*)) } catch { case i: IllegalArgumentException => val e = new ParseException(i.getMessage, ctx) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index a9d5e7c7544a6..e291d5f9cd84c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -29,21 +29,12 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.unsafe.types.CalendarInterval /** * Base SQL parsing infrastructure. */ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Logging { - /** - * Creates [[CalendarInterval]] for a given SQL String. Throws [[ParseException]] if the SQL - * string is not a valid interval format. - */ - def parseInterval(sqlText: String): CalendarInterval = parse(sqlText) { parser => - astBuilder.visitSingleInterval(parser.singleInterval()) - } - /** Creates/Resolves DataType for a given SQL string. */ override def parseDataType(sqlText: String): DataType = parse(sqlText) { parser => astBuilder.visitSingleDataType(parser.singleDataType()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 104c65b6cd037..725ff81ef410e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit import scala.util.control.NonFatal -import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -102,34 +101,6 @@ object IntervalUtils { Decimal(result, 18, 6) } - /** - * Converts a string to [[CalendarInterval]] case-insensitively. - * - * @throws IllegalArgumentException if the input string is not in valid interval format. - */ - def fromString(str: String): CalendarInterval = { - if (str == null) throw new IllegalArgumentException("Interval string cannot be null") - try { - CatalystSqlParser.parseInterval(str) - } catch { - case e: ParseException => - val ex = new IllegalArgumentException(s"Invalid interval string: $str\n" + e.message) - ex.setStackTrace(e.getStackTrace) - throw ex - } - } - - /** - * A safe version of `fromString`. It returns null for invalid input string. - */ - def safeFromString(str: String): CalendarInterval = { - try { - fromString(str) - } catch { - case _: IllegalArgumentException => null - } - } - private def toLongWithRange( fieldName: IntervalUnit, s: String, @@ -251,46 +222,6 @@ object IntervalUtils { } } - def fromUnitStrings(units: Array[IntervalUnit], values: Array[String]): CalendarInterval = { - assert(units.length == values.length) - var months: Int = 0 - var days: Int = 0 - var microseconds: Long = 0 - var i = 0 - while (i < units.length) { - try { - units(i) match { - case YEAR => - months = Math.addExact(months, Math.multiplyExact(values(i).toInt, 12)) - case MONTH => - months = Math.addExact(months, values(i).toInt) - case WEEK => - days = Math.addExact(days, Math.multiplyExact(values(i).toInt, 7)) - case DAY => - days = Math.addExact(days, values(i).toInt) - case HOUR => - val hoursUs = Math.multiplyExact(values(i).toLong, MICROS_PER_HOUR) - microseconds = Math.addExact(microseconds, hoursUs) - case MINUTE => - val minutesUs = Math.multiplyExact(values(i).toLong, MICROS_PER_MINUTE) - microseconds = Math.addExact(microseconds, minutesUs) - case SECOND => - microseconds = Math.addExact(microseconds, parseSecondNano(values(i))) - case MILLISECOND => - val millisUs = Math.multiplyExact(values(i).toLong, MICROS_PER_MILLIS) - microseconds = Math.addExact(microseconds, millisUs) - case MICROSECOND => - microseconds = Math.addExact(microseconds, values(i).toLong) - } - } catch { - case e: Exception => - throw new IllegalArgumentException(s"Error parsing interval string: ${e.getMessage}", e) - } - i += 1 - } - new CalendarInterval(months, days, microseconds) - } - // Parses a string with nanoseconds, truncates the result and returns microseconds private def parseNanos(nanosStr: String, isNegative: Boolean): Long = { if (nanosStr != null) { @@ -306,30 +237,6 @@ object IntervalUtils { } } - /** - * Parse second_nano string in ss.nnnnnnnnn format to microseconds - */ - private def parseSecondNano(secondNano: String): Long = { - def parseSeconds(secondsStr: String): Long = { - toLongWithRange( - SECOND, - secondsStr, - Long.MinValue / MICROS_PER_SECOND, - Long.MaxValue / MICROS_PER_SECOND) * MICROS_PER_SECOND - } - - secondNano.split("\\.") match { - case Array(secondsStr) => parseSeconds(secondsStr) - case Array("", nanosStr) => parseNanos(nanosStr, false) - case Array(secondsStr, nanosStr) => - val seconds = parseSeconds(secondsStr) - Math.addExact(seconds, parseNanos(nanosStr, seconds < 0)) - case _ => - throw new IllegalArgumentException( - "Interval string does not match second-nano format of ss.nnnnnnnnn") - } - } - /** * Gets interval duration * @@ -558,18 +465,37 @@ object IntervalUtils { private final val millisStr = unitToUtf8(MILLISECOND) private final val microsStr = unitToUtf8(MICROSECOND) + /** + * A safe version of `stringToInterval`. It returns null for invalid input string. + */ + def safeStringToInterval(input: UTF8String): CalendarInterval = { + try { + stringToInterval(input) + } catch { + case _: IllegalArgumentException => null + } + } + + /** + * Converts a string to [[CalendarInterval]] case-insensitively. + * + * @throws IllegalArgumentException if the input string is not in valid interval format. + */ def stringToInterval(input: UTF8String): CalendarInterval = { import ParseState._ + def throwIAE(msg: String, e: Exception = null) = { + throw new IllegalArgumentException(s"Error parsing '$input' to interval, $msg", e) + } if (input == null) { - return null + throwIAE("interval string cannot be null") } // scalastyle:off caselocale .toLowerCase val s = input.trim.toLowerCase // scalastyle:on val bytes = s.getBytes if (bytes.isEmpty) { - return null + throwIAE("interval string cannot be empty") } var state = PREFIX var i = 0 @@ -588,13 +514,19 @@ object IntervalUtils { } } + def currentWord: UTF8String = { + val strings = s.split(UTF8String.blankString(1), -1) + val lenRight = s.substring(i, s.numBytes()).split(UTF8String.blankString(1), -1).length + strings(strings.length - lenRight) + } + while (i < bytes.length) { val b = bytes(i) state match { case PREFIX => if (s.startsWith(intervalStr)) { if (s.numBytes() == intervalStr.numBytes()) { - return null + throwIAE("interval string cannot be empty") } else { i += intervalStr.numBytes() } @@ -627,7 +559,7 @@ object IntervalUtils { fractionScale = (NANOS_PER_SECOND / 10).toInt i += 1 state = VALUE_FRACTIONAL_PART - case _ => return null + case _ => throwIAE( s"unrecognized number '$currentWord'") } case TRIM_BEFORE_VALUE => trimToNextState(b, VALUE) case VALUE => @@ -636,13 +568,13 @@ object IntervalUtils { try { currentValue = Math.addExact(Math.multiplyExact(10, currentValue), (b - '0')) } catch { - case _: ArithmeticException => return null + case e: ArithmeticException => throwIAE(e.getMessage, e) } case ' ' => state = TRIM_BEFORE_UNIT case '.' => fractionScale = (NANOS_PER_SECOND / 10).toInt state = VALUE_FRACTIONAL_PART - case _ => return null + case _ => throwIAE(s"invalid value '$currentWord'") } i += 1 case VALUE_FRACTIONAL_PART => @@ -653,14 +585,17 @@ object IntervalUtils { case ' ' => fraction /= NANOS_PER_MICROS.toInt state = TRIM_BEFORE_UNIT - case _ => return null + case _ if '0' <= b && b <= '9' => + throwIAE(s"interval can only support nanosecond precision, '$currentWord' is out" + + s" of range") + case _ => throwIAE(s"invalid value '$currentWord'") } i += 1 case TRIM_BEFORE_UNIT => trimToNextState(b, UNIT_BEGIN) case UNIT_BEGIN => // Checks that only seconds can have the fractional part if (b != 's' && fractionScale >= 0) { - return null + throwIAE(s"'$currentWord' cannot have fractional part") } if (isNegative) { currentValue = -currentValue @@ -704,18 +639,18 @@ object IntervalUtils { } else if (s.matchAt(microsStr, i)) { microseconds = Math.addExact(microseconds, currentValue) i += microsStr.numBytes() - } else return null - case _ => return null + } else throwIAE(s"invalid unit '$currentWord'") + case _ => throwIAE(s"invalid unit '$currentWord'") } } catch { - case _: ArithmeticException => return null + case e: ArithmeticException => throwIAE(e.getMessage, e) } state = UNIT_SUFFIX case UNIT_SUFFIX => b match { case 's' => state = UNIT_END case ' ' => state = TRIM_BEFORE_SIGN - case _ => return null + case _ => throwIAE(s"invalid unit '$currentWord'") } i += 1 case UNIT_END => @@ -723,7 +658,7 @@ object IntervalUtils { case ' ' => i += 1 state = TRIM_BEFORE_SIGN - case _ => return null + case _ => throwIAE(s"invalid unit '$currentWord'") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index dd6d437d8063e..b4343b648110f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -32,9 +32,12 @@ import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH +import org.apache.spark.unsafe.types.UTF8String class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) + def testSize(sizeOfNull: Any): Unit = { val a0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType)) val a1 = Literal.create(Seq[Integer](), ArrayType(IntegerType)) @@ -711,7 +714,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:00")), - Literal(fromString("interval 12 hours"))), + Literal(stringToInterval("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -720,7 +723,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-02 00:00:01")), - Literal(fromString("interval 12 hours"))), + Literal(stringToInterval("interval 12 hours"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -729,7 +732,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(negate(fromString("interval 12 hours")))), + Literal(negate(stringToInterval("interval 12 hours")))), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -738,7 +741,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-02 00:00:00")), Literal(Timestamp.valueOf("2017-12-31 23:59:59")), - Literal(negate(fromString("interval 12 hours")))), + Literal(negate(stringToInterval("interval 12 hours")))), Seq( Timestamp.valueOf("2018-01-02 00:00:00"), Timestamp.valueOf("2018-01-01 12:00:00"), @@ -747,7 +750,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(fromString("interval 1 month"))), + Literal(stringToInterval("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -756,7 +759,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-01 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(negate(fromString("interval 1 month")))), + Literal(negate(stringToInterval("interval 1 month")))), Seq( Timestamp.valueOf("2018-03-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:00"), @@ -765,7 +768,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-03 00:00:00")), Literal(Timestamp.valueOf("2018-01-01 00:00:00")), - Literal(negate(fromString("interval 1 month 1 day")))), + Literal(negate(stringToInterval("interval 1 month 1 day")))), Seq( Timestamp.valueOf("2018-03-03 00:00:00"), Timestamp.valueOf("2018-02-02 00:00:00"), @@ -774,7 +777,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-31 00:00:00")), Literal(Timestamp.valueOf("2018-04-30 00:00:00")), - Literal(fromString("interval 1 month"))), + Literal(stringToInterval("interval 1 month"))), Seq( Timestamp.valueOf("2018-01-31 00:00:00"), Timestamp.valueOf("2018-02-28 00:00:00"), @@ -784,7 +787,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:00:00")), - Literal(fromString("interval 1 month 1 second"))), + Literal(stringToInterval("interval 1 month 1 second"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:00:01"))) @@ -792,7 +795,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-01-01 00:00:00")), Literal(Timestamp.valueOf("2018-03-01 00:04:06")), - Literal(fromString("interval 1 month 2 minutes 3 seconds"))), + Literal(stringToInterval("interval 1 month 2 minutes 3 seconds"))), Seq( Timestamp.valueOf("2018-01-01 00:00:00"), Timestamp.valueOf("2018-02-01 00:02:03"), @@ -830,7 +833,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-03-25 01:30:00")), Literal(Timestamp.valueOf("2018-03-25 03:30:00")), - Literal(fromString("interval 30 minutes"))), + Literal(stringToInterval("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-03-25 01:30:00"), Timestamp.valueOf("2018-03-25 03:00:00"), @@ -840,7 +843,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Timestamp.valueOf("2018-10-28 01:30:00")), Literal(Timestamp.valueOf("2018-10-28 03:30:00")), - Literal(fromString("interval 30 minutes"))), + Literal(stringToInterval("interval 30 minutes"))), Seq( Timestamp.valueOf("2018-10-28 01:30:00"), noDST(Timestamp.valueOf("2018-10-28 02:00:00")), @@ -857,7 +860,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-01-05")), - Literal(fromString("interval 2 days"))), + Literal(stringToInterval("interval 2 days"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-01-03"), @@ -866,7 +869,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-01")), Literal(Date.valueOf("2018-03-01")), - Literal(fromString("interval 1 month"))), + Literal(stringToInterval("interval 1 month"))), Seq( Date.valueOf("2018-01-01"), Date.valueOf("2018-02-01"), @@ -875,7 +878,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new Sequence( Literal(Date.valueOf("2018-01-31")), Literal(Date.valueOf("2018-04-30")), - Literal(fromString("interval 1 month"))), + Literal(stringToInterval("interval 1 month"))), Seq( Date.valueOf("2018-01-31"), Date.valueOf("2018-02-28"), @@ -896,14 +899,14 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper new Sequence( Literal(Date.valueOf("1970-01-02")), Literal(Date.valueOf("1970-01-01")), - Literal(fromString("interval 1 day"))), + Literal(stringToInterval("interval 1 day"))), EmptyRow, "sequence boundaries: 1 to 0 by 1") checkExceptionInExpression[IllegalArgumentException]( new Sequence( Literal(Date.valueOf("1970-01-01")), Literal(Date.valueOf("1970-02-01")), - Literal(negate(fromString("interval 1 month")))), + Literal(negate(stringToInterval("interval 1 month")))), EmptyRow, s"sequence boundaries: 0 to 2678400000000 by -${28 * MICROS_PER_DAY}") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 5f043ce972bed..5cd4d11e32f7a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -1090,17 +1090,17 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), new CalendarInterval(0, 0, 0)) checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), - IntervalUtils.fromString("interval " + - "436163 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval " + + "436163 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds"))) checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), - IntervalUtils.fromString("interval " + - "-436163 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval " + + "-436163 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds"))) checkEvaluation( SubtractTimestamps( Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), Literal(Instant.parse("0001-01-01T00:00:00Z"))), - IntervalUtils.fromString("interval " + - "87649415 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval " + + "87649415 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds"))) } test("subtract dates") { @@ -1108,18 +1108,18 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractDates(Literal(end), Literal(end)), new CalendarInterval(0, 0, 0)) checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), - IntervalUtils.fromString("interval 1 days")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval 1 days"))) checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), - IntervalUtils.fromString("interval -1 days")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval -1 days"))) val epochDate = Literal(LocalDate.ofEpochDay(0)) checkEvaluation(SubtractDates(Literal(end), epochDate), - IntervalUtils.fromString("interval 49 years 9 months 4 days")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval 49 years 9 months 4 days"))) checkEvaluation(SubtractDates(epochDate, Literal(end)), - IntervalUtils.fromString("interval -49 years -9 months -4 days")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval -49 years -9 months -4 days"))) checkEvaluation( SubtractDates( Literal(LocalDate.of(10000, 1, 1)), Literal(LocalDate.of(1, 1, 1))), - IntervalUtils.fromString("interval 9999 years")) + IntervalUtils.stringToInterval(UTF8String.fromString("interval 9999 years"))) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index 4b2da73abe562..3a68847ecb1f4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -36,6 +36,7 @@ import org.apache.spark.unsafe.types.UTF8String class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val random = new scala.util.Random + implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) test("md5") { checkEvaluation(Md5(Literal("ABC".getBytes(StandardCharsets.UTF_8))), @@ -252,7 +253,8 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("hive-hash for CalendarInterval type") { def checkHiveHashForIntervalType(interval: String, expected: Long): Unit = { - checkHiveHash(IntervalUtils.fromString(interval), CalendarIntervalType, expected) + checkHiveHash(IntervalUtils.stringToInterval(UTF8String.fromString(interval)), + CalendarIntervalType, expected) } // ----- MICROSEC ----- diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index e483f028ffff3..ddcb6a66832af 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -21,13 +21,15 @@ import scala.language.implicitConversions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeConstants._ -import org.apache.spark.sql.catalyst.util.IntervalUtils.fromString +import org.apache.spark.sql.catalyst.util.IntervalUtils.stringToInterval import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) + implicit def interval(s: String): Literal = { - Literal(fromString("interval " + s)) + Literal(stringToInterval( "interval " + s)) } test("millenniums") { @@ -197,8 +199,8 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("multiply") { def check(interval: String, num: Double, expected: String): Unit = { checkEvaluation( - MultiplyInterval(Literal(fromString(interval)), Literal(num)), - if (expected == null) null else fromString(expected)) + MultiplyInterval(Literal(stringToInterval(interval)), Literal(num)), + if (expected == null) null else stringToInterval(expected)) } check("0 seconds", 10, "0 seconds") @@ -215,8 +217,8 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("divide") { def check(interval: String, num: Double, expected: String): Unit = { checkEvaluation( - DivideInterval(Literal(fromString(interval)), Literal(num)), - if (expected == null) null else fromString(expected)) + DivideInterval(Literal(stringToInterval(interval)), Literal(num)), + if (expected == null) null else stringToInterval(expected)) } check("0 seconds", 10, "0 seconds") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala index 23ba9c6ec7388..63700a1e94a3e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -56,7 +57,8 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { testBothCodegenAndInterpreted("variable-length types") { val proj = createMutableProjection(variableLengthTypes) - val scalaValues = Seq("abc", BigDecimal(10), IntervalUtils.fromString("interval 1 day"), + val scalaValues = Seq("abc", BigDecimal(10), + IntervalUtils.stringToInterval(UTF8String.fromString("interval 1 day")), Array[Byte](1, 2), Array("123", "456"), Map(1 -> "a", 2 -> "b"), Row(1, "a"), new java.lang.Integer(5)) val inputRow = InternalRow.fromSeq(scalaValues.zip(variableLengthTypes).map { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index 4ccd4f7ce798d..ef7764dba1e9e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -485,7 +485,8 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { ("abcd".getBytes, BinaryType), ("abcd", StringType), (BigDecimal.valueOf(10), DecimalType.IntDecimal), - (IntervalUtils.fromString("interval 3 day"), CalendarIntervalType), + (IntervalUtils.stringToInterval(UTF8String.fromString("interval 3 day")), + CalendarIntervalType), (java.math.BigDecimal.valueOf(10), DecimalType.BigIntDecimal), (Array(3, 2, 1), ArrayType(IntegerType)) ).foreach { case (input, dt) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 20e77254ecdad..b80b30a4e07ae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -531,7 +531,8 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB // Simple tests val inputRow = InternalRow.fromSeq(Seq( false, 3.toByte, 15.toShort, -83, 129L, 1.0f, 8.0, UTF8String.fromString("test"), - Decimal(255), IntervalUtils.fromString("interval 1 day"), Array[Byte](1, 2) + Decimal(255), IntervalUtils.stringToInterval(UTF8String.fromString( "interval 1 day")), + Array[Byte](1, 2) )) val fields1 = Array( BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 17a2213d0ccb1..a7dbd4e4e3459 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** * Test basic expression parsing. @@ -43,6 +43,8 @@ class ExpressionParserSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ + implicit def stringToUTF8Str(str: String): UTF8String = UTF8String.fromString(str) + val defaultParser = CatalystSqlParser def assertEqual( @@ -434,13 +436,13 @@ class ExpressionParserSuite extends AnalysisTest { intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the TIMESTAMP value") // Interval. - val intervalLiteral = Literal(IntervalUtils.fromString("interval 3 month 1 hour")) + val intervalLiteral = Literal(IntervalUtils.stringToInterval("interval 3 month 1 hour")) assertEqual("InterVal 'interval 3 month 1 hour'", intervalLiteral) assertEqual("INTERVAL '3 month 1 hour'", intervalLiteral) intercept("Interval 'interval 3 monthsss 1 hoursss'", "Cannot parse the INTERVAL value") assertEqual( "-interval '3 month 1 hour'", - Literal(IntervalUtils.fromString("interval -3 month -1 hour"))) + Literal(IntervalUtils.stringToInterval("interval -3 month -1 hour"))) // Binary. assertEqual("X'A'", Literal(Array(0x0a).map(_.toByte))) @@ -602,7 +604,7 @@ class ExpressionParserSuite extends AnalysisTest { MICROSECOND) def intervalLiteral(u: IntervalUnit, s: String): Literal = { - Literal(IntervalUtils.fromUnitStrings(Array(u), Array(s))) + Literal(IntervalUtils.stringToInterval(s + " " + u.toString)) } test("intervals") { @@ -651,7 +653,8 @@ class ExpressionParserSuite extends AnalysisTest { 0, 0, 13 * MICROS_PER_SECOND + 123 * MICROS_PER_MILLIS + 456))) - checkIntervals("1.001 second", Literal(IntervalUtils.fromString("1 second 1 millisecond"))) + checkIntervals("1.001 second", + Literal(IntervalUtils.stringToInterval("1 second 1 millisecond"))) // Non Existing unit intercept("interval 10 nanoseconds", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 65cfe896b2024..73a2adbaec1db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -28,20 +28,31 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class IntervalUtilsSuite extends SparkFunSuite { private def checkFromString(input: String, expected: CalendarInterval): Unit = { - assert(fromString(input) === expected) assert(stringToInterval(UTF8String.fromString(input)) === expected) + assert(safeStringToInterval(UTF8String.fromString(input)) === expected) + } + + private def checkFromStringWithFunc( + input: String, + months: Int, + days: Int, + us: Long, + func: CalendarInterval => CalendarInterval): Unit = { + val expected = new CalendarInterval(months, days, us) + assert(func(stringToInterval(UTF8String.fromString(input))) === expected) + assert(func(safeStringToInterval(UTF8String.fromString(input))) === expected) } private def checkFromInvalidString(input: String, errorMsg: String): Unit = { try { - fromString(input) + stringToInterval(UTF8String.fromString(input)) fail("Expected to throw an exception for the invalid input") } catch { case e: IllegalArgumentException => val msg = e.getMessage assert(msg.contains(errorMsg)) } - assert(stringToInterval(UTF8String.fromString(input)) === null) + assert(safeStringToInterval(UTF8String.fromString(input)) === null) } private def testSingleUnit( @@ -69,7 +80,7 @@ class IntervalUtilsSuite extends SparkFunSuite { checkFromInvalidString(null, "cannot be null") for (input <- Seq("", " ", "interval", "interval1 day", "foo", "foo 1 day")) { - checkFromInvalidString(input, "Invalid interval string") + checkFromInvalidString(input, "Error parsing") } } @@ -93,8 +104,18 @@ class IntervalUtilsSuite extends SparkFunSuite { // Allow duplicated units and summarize their values checkFromString("1 day 10 day", new CalendarInterval(0, 11, 0)) // Only the seconds units can have the fractional part - checkFromInvalidString("1.5 days", "Error parsing interval string") - checkFromInvalidString("1. hour", "Error parsing interval string") + checkFromInvalidString("1.5 days", "'days' cannot have fractional part") + checkFromInvalidString("1. hour", "'hour' cannot have fractional part") + checkFromInvalidString("1 hourX", "invalid unit 'hourx'") + checkFromInvalidString("~1 hour", "unrecognized number '~1'") + checkFromInvalidString("1 Mour", "invalid unit 'mour'") + checkFromInvalidString("1 aour", "invalid unit 'aour'") + checkFromInvalidString("1a1 hour", "invalid value '1a1'") + checkFromInvalidString("1.1a1 seconds", "invalid value '1.1a1'") + checkFromInvalidString("2234567890 days", "integer overflow") + checkFromInvalidString("\n", "Error parsing '\n' to interval") + checkFromInvalidString("\t", "Error parsing '\t' to interval") + } test("string to interval: seconds with fractional part") { @@ -107,7 +128,7 @@ class IntervalUtilsSuite extends SparkFunSuite { // truncate nanoseconds to microseconds checkFromString("0.999999999 seconds", new CalendarInterval(0, 0, 999999)) checkFromString(".999999999 seconds", new CalendarInterval(0, 0, 999999)) - checkFromInvalidString("0.123456789123 seconds", "Error parsing interval string") + checkFromInvalidString("0.123456789123 seconds", "'0.123456789123' is out of range") } test("from year-month string") { @@ -174,7 +195,7 @@ class IntervalUtilsSuite extends SparkFunSuite { test("interval duration") { def duration(s: String, unit: TimeUnit, daysPerMonth: Int): Long = { - IntervalUtils.getDuration(fromString(s), unit, daysPerMonth) + IntervalUtils.getDuration(stringToInterval(UTF8String.fromString(s)), unit, daysPerMonth) } assert(duration("0 seconds", TimeUnit.MILLISECONDS, 31) === 0) @@ -193,7 +214,7 @@ class IntervalUtilsSuite extends SparkFunSuite { test("negative interval") { def isNegative(s: String, daysPerMonth: Int): Boolean = { - IntervalUtils.isNegative(fromString(s), daysPerMonth) + IntervalUtils.isNegative(stringToInterval(UTF8String.fromString(s)), daysPerMonth) } assert(isNegative("-1 months", 28)) @@ -269,33 +290,27 @@ class IntervalUtilsSuite extends SparkFunSuite { } test("justify days") { - assert(justifyDays(fromString("1 month 35 day")) === new CalendarInterval(2, 5, 0)) - assert(justifyDays(fromString("-1 month 35 day")) === new CalendarInterval(0, 5, 0)) - assert(justifyDays(fromString("1 month -35 day")) === new CalendarInterval(0, -5, 0)) - assert(justifyDays(fromString("-1 month -35 day")) === new CalendarInterval(-2, -5, 0)) - assert(justifyDays(fromString("-1 month 2 day")) === new CalendarInterval(0, -28, 0)) + checkFromStringWithFunc("1 month 35 day", 2, 5, 0, justifyDays) + checkFromStringWithFunc("-1 month 35 day", 0, 5, 0, justifyDays) + checkFromStringWithFunc("1 month -35 day", 0, -5, 0, justifyDays) + checkFromStringWithFunc("-1 month -35 day", -2, -5, 0, justifyDays) + checkFromStringWithFunc("-1 month 2 day", 0, -28, 0, justifyDays) } test("justify hours") { - assert(justifyHours(fromString("29 day 25 hour")) === - new CalendarInterval(0, 30, 1 * MICROS_PER_HOUR)) - assert(justifyHours(fromString("29 day -25 hour")) === - new CalendarInterval(0, 27, 23 * MICROS_PER_HOUR)) - assert(justifyHours(fromString("-29 day 25 hour")) === - new CalendarInterval(0, -27, -23 * MICROS_PER_HOUR)) - assert(justifyHours(fromString("-29 day -25 hour")) === - new CalendarInterval(0, -30, -1 * MICROS_PER_HOUR)) + checkFromStringWithFunc("29 day 25 hour", 0, 30, 1 * MICROS_PER_HOUR, justifyHours) + checkFromStringWithFunc("29 day -25 hour", 0, 27, 23 * MICROS_PER_HOUR, justifyHours) + checkFromStringWithFunc("-29 day 25 hour", 0, -27, -23 * MICROS_PER_HOUR, justifyHours) + checkFromStringWithFunc("-29 day -25 hour", 0, -30, -1 * MICROS_PER_HOUR, justifyHours) } test("justify interval") { - assert(justifyInterval(fromString("1 month 29 day 25 hour")) === - new CalendarInterval(2, 0, 1 * MICROS_PER_HOUR)) - assert(justifyInterval(fromString("-1 month 29 day -25 hour")) === - new CalendarInterval(0, -2, -1 * MICROS_PER_HOUR)) - assert(justifyInterval(fromString("1 month -29 day -25 hour")) === - new CalendarInterval(0, 0, -1 * MICROS_PER_HOUR)) - assert(justifyInterval(fromString("-1 month -29 day -25 hour")) === - new CalendarInterval(-2, 0, -1 * MICROS_PER_HOUR)) + checkFromStringWithFunc("1 month 29 day 25 hour", 2, 0, 1 * MICROS_PER_HOUR, justifyInterval) + checkFromStringWithFunc("-1 month 29 day -25 hour", 0, -2, -1 * MICROS_PER_HOUR, + justifyInterval) + checkFromStringWithFunc("1 month -29 day -25 hour", 0, 0, -1 * MICROS_PER_HOUR, justifyInterval) + checkFromStringWithFunc("-1 month -29 day -25 hour", -2, 0, -1 * MICROS_PER_HOUR, + justifyInterval) intercept[ArithmeticException](justifyInterval(new CalendarInterval(2, 0, Long.MaxValue))) } 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 aaa3f9dd71594..0fcd82276392c 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 @@ -60,7 +60,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.Utils private[sql] object Dataset { @@ -725,7 +725,7 @@ class Dataset[T] private[sql]( def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { val parsedDelay = try { - IntervalUtils.fromString(delayThreshold) + IntervalUtils.stringToInterval(UTF8String.fromString(delayThreshold)) } catch { case e: IllegalArgumentException => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala index aac5da8104a8b..59ce7c3707b27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, Processing import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.execution.streaming.GroupStateImpl._ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout} -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.UTF8String /** @@ -160,7 +160,7 @@ private[sql] class GroupStateImpl[S] private( def getTimeoutTimestamp: Long = timeoutTimestamp private def parseDuration(duration: String): Long = { - val cal = IntervalUtils.fromString(duration) + val cal = IntervalUtils.stringToInterval(UTF8String.fromString(duration)) if (IntervalUtils.isNegative(cal)) { throw new IllegalArgumentException(s"Provided duration ($duration) is negative") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 2dd287cb734bf..1a27fe61d9602 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -24,6 +24,7 @@ import scala.concurrent.duration.Duration import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.unsafe.types.UTF8String private object Triggers { def validate(intervalMs: Long): Unit = { @@ -31,7 +32,7 @@ private object Triggers { } def convert(interval: String): Long = { - val cal = IntervalUtils.fromString(interval) + val cal = IntervalUtils.stringToInterval(UTF8String.fromString(interval)) if (cal.months != 0) { throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index b11e02a9c52bd..9c28529ed710f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -751,7 +751,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { assertError("select interval '23:61:15' hour to second", "minute 61 outside range [0, 59]") assertError("select interval '.1111111111' second", - "nanosecond 1111111111 outside range") + "'.1111111111' is out of range") } test("use native json_tuple instead of hive's UDTF in LATERAL VIEW") { From c32e2286898fe8660c7deed9303f1c5c15b60757 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Mon, 18 Nov 2019 20:40:23 +0800 Subject: [PATCH 0650/1387] [SPARK-29859][SQL] ALTER DATABASE (SET LOCATION) should look up catalog like v2 commands ### What changes were proposed in this pull request? Add AlterNamespaceSetLocationStatement, AlterNamespaceSetLocation, AlterNamespaceSetLocationExec to make ALTER DATABASE (SET LOCATION) look up catalog like v2 commands. And also refine the code of AlterNamespaceSetProperties, AlterNamespaceSetPropertiesExec, DescribeNamespace, DescribeNamespaceExec to use SupportsNamespaces instead of CatalogPlugin for catalog parameter. ### Why are the changes needed? It's important to make all the commands have the same catalog/namespace resolution behavior, to avoid confusing end-users. ### Does this PR introduce any user-facing change? Yes, add "ALTER NAMESPACE ... SET LOCATION" whose function is same as "ALTER DATABASE ... SET LOCATION" and "ALTER SCHEMA ... SET LOCATION". ### How was this patch tested? New unit tests Closes #26562 from fuwhu/SPARK-29859. Authored-by: fuwhu Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 4 ++-- .../sql/catalyst/analysis/ResolveCatalogs.scala | 8 ++++++-- .../spark/sql/catalyst/parser/AstBuilder.scala | 16 ++++++++++++++++ .../sql/catalyst/plans/logical/statements.scala | 7 +++++++ .../sql/catalyst/plans/logical/v2Commands.scala | 4 ++-- .../sql/catalyst/parser/DDLParserSuite.scala | 14 ++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 7 +++++++ .../spark/sql/execution/SparkSqlParser.scala | 16 ---------------- .../v2/AlterNamespaceSetPropertiesExec.scala | 11 ++++------- .../datasources/v2/DescribeNamespaceExec.scala | 10 +++------- .../sql/connector/DataSourceV2SQLSuite.scala | 14 ++++++++++++++ .../sql/execution/command/DDLParserSuite.scala | 9 --------- 12 files changed, 75 insertions(+), 45 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index de0353494f003..13f1f64ba7b8d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -89,8 +89,8 @@ statement (WITH (DBPROPERTIES | PROPERTIES) tablePropertyList))* #createNamespace | ALTER (database | NAMESPACE) multipartIdentifier SET (DBPROPERTIES | PROPERTIES) tablePropertyList #setNamespaceProperties - | ALTER database db=errorCapturingIdentifier - SET locationSpec #setDatabaseLocation + | ALTER (database | NAMESPACE) multipartIdentifier + SET locationSpec #setNamespaceLocation | DROP (database | NAMESPACE) (IF EXISTS)? multipartIdentifier (RESTRICT | CASCADE)? #dropNamespace | SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 0d8d58520edad..5a4fb2e865903 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -94,7 +94,11 @@ class ResolveCatalogs(val catalogManager: CatalogManager) s"because view support in catalog has not been implemented yet") case AlterNamespaceSetPropertiesStatement(NonSessionCatalog(catalog, nameParts), properties) => - AlterNamespaceSetProperties(catalog, nameParts, properties) + AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts, properties) + + case AlterNamespaceSetLocationStatement(NonSessionCatalog(catalog, nameParts), location) => + AlterNamespaceSetProperties( + catalog.asNamespaceCatalog, nameParts, Map("location" -> location)) case DescribeTableStatement( nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => @@ -176,7 +180,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) DropNamespace(catalog, nameParts, ifExists, cascade) case DescribeNamespaceStatement(NonSessionCatalog(catalog, nameParts), extended) => - DescribeNamespace(catalog, nameParts, extended) + DescribeNamespace(catalog.asNamespaceCatalog, nameParts, extended) case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 68b1862a5d236..59c79f4021fb0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2539,6 +2539,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Create an [[AlterNamespaceSetLocationStatement]] logical plan. + * + * For example: + * {{{ + * ALTER (DATABASE|SCHEMA|NAMESPACE) namespace SET LOCATION path; + * }}} + */ + override def visitSetNamespaceLocation(ctx: SetNamespaceLocationContext): LogicalPlan = { + withOrigin(ctx) { + AlterNamespaceSetLocationStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + visitLocationSpec(ctx.locationSpec)) + } + } + /** * Create a [[ShowNamespacesStatement]] command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 875ff4e4a2821..7a98cccc3d7a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -357,6 +357,13 @@ case class AlterNamespaceSetPropertiesStatement( namespace: Seq[String], properties: Map[String, String]) extends ParsedStatement +/** + * ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET LOCATION command, as parsed from SQL. + */ +case class AlterNamespaceSetLocationStatement( + namespace: Seq[String], + location: String) extends ParsedStatement + /** * A SHOW NAMESPACES statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 22edb3619a929..f7f8b2778d234 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -259,7 +259,7 @@ case class DropNamespace( * The logical plan of the DESCRIBE NAMESPACE command that works for v2 catalogs. */ case class DescribeNamespace( - catalog: CatalogPlugin, + catalog: SupportsNamespaces, namespace: Seq[String], extended: Boolean) extends Command { @@ -275,7 +275,7 @@ case class DescribeNamespace( * command that works for v2 catalogs. */ case class AlterNamespaceSetProperties( - catalog: CatalogPlugin, + catalog: SupportsNamespaces, namespace: Seq[String], properties: Map[String, String]) extends Command diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index e87f53910c2c7..d3ca6f7a8eee2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1192,6 +1192,20 @@ class DDLParserSuite extends AnalysisTest { Seq("a", "b", "c"), Map("b" -> "b"))) } + test("set namespace location") { + comparePlans( + parsePlan("ALTER DATABASE a.b.c SET LOCATION '/home/user/db'"), + AlterNamespaceSetLocationStatement(Seq("a", "b", "c"), "/home/user/db")) + + comparePlans( + parsePlan("ALTER SCHEMA a.b.c SET LOCATION '/home/user/db'"), + AlterNamespaceSetLocationStatement(Seq("a", "b", "c"), "/home/user/db")) + + comparePlans( + parsePlan("ALTER NAMESPACE a.b.c SET LOCATION '/home/user/db'"), + AlterNamespaceSetLocationStatement(Seq("a", "b", "c"), "/home/user/db")) + } + test("show databases: basic") { comparePlans( parsePlan("SHOW DATABASES"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index cac320edc47e2..708203119f4bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -172,6 +172,13 @@ class ResolveSessionCatalog( } AlterDatabasePropertiesCommand(nameParts.head, properties) + case AlterNamespaceSetLocationStatement(SessionCatalog(_, nameParts), location) => + if (nameParts.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${nameParts.quoted}") + } + AlterDatabaseSetLocationCommand(nameParts.head, location) + case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 7fe4c00f5dc8d..8241f850e3aa7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -227,22 +227,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } - /** - * Create an [[AlterDatabaseSetLocationCommand]] command. - * - * For example: - * {{{ - * ALTER (DATABASE|SCHEMA) database SET LOCATION path; - * }}} - */ - override def visitSetDatabaseLocation( - ctx: SetDatabaseLocationContext): LogicalPlan = withOrigin(ctx) { - AlterDatabaseSetLocationCommand( - ctx.db.getText, - visitLocationSpec(ctx.locationSpec) - ) - } - /** * Create a plan for a DESCRIBE FUNCTION command. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala index fd6922cd1c3ab..1eebe4cdb6a86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceSetPropertiesExec.scala @@ -19,23 +19,20 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.{CatalogPlugin, NamespaceChange} +import org.apache.spark.sql.connector.catalog.{NamespaceChange, SupportsNamespaces} /** * Physical plan node for setting properties of namespace. */ case class AlterNamespaceSetPropertiesExec( - catalog: CatalogPlugin, + catalog: SupportsNamespaces, namespace: Seq[String], - props: Map[String, String]) - extends V2CommandExec { + props: Map[String, String]) extends V2CommandExec { override protected def run(): Seq[InternalRow] = { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - val changes = props.map{ case (k, v) => NamespaceChange.setProperty(k, v) }.toSeq - catalog.asNamespaceCatalog.alterNamespace(namespace.toArray, changes: _*) + catalog.alterNamespace(namespace.toArray, changes: _*) Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala index 5c20e5ae08383..7c5cfcbbc7e3c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} -import org.apache.spark.sql.connector.catalog.CatalogPlugin +import org.apache.spark.sql.connector.catalog.SupportsNamespaces import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.COMMENT_TABLE_PROP import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.LOCATION_TABLE_PROP import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.RESERVED_PROPERTIES @@ -34,19 +34,15 @@ import org.apache.spark.sql.types.StructType */ case class DescribeNamespaceExec( output: Seq[Attribute], - catalog: CatalogPlugin, + catalog: SupportsNamespaces, namespace: Seq[String], isExtended: Boolean) extends V2CommandExec { - private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() override protected def run(): Seq[InternalRow] = { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - val rows = new ArrayBuffer[InternalRow]() - val nsCatalog = catalog.asNamespaceCatalog val ns = namespace.toArray - val metadata = nsCatalog.loadNamespaceMetadata(ns) + val metadata = catalog.loadNamespaceMetadata(ns) rows += toCatalystRow("Namespace Name", ns.last) rows += toCatalystRow("Description", metadata.get(COMMENT_TABLE_PROP)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index e5bf8e337c8d3..36d2deaa309ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -948,6 +948,20 @@ class DataSourceV2SQLSuite } } + test("AlterNamespaceSetLocation using v2 catalog") { + withNamespace("testcat.ns1.ns2") { + sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + + "'test namespace' LOCATION '/tmp/ns_test_1'") + sql("ALTER NAMESPACE testcat.ns1.ns2 SET LOCATION '/tmp/ns_test_2'") + val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") + assert(descriptionDf.collect() === Seq( + Row("Namespace Name", "ns2"), + Row("Description", "test namespace"), + Row("Location", "/tmp/ns_test_2") + )) + } + } + test("ShowNamespaces: show root namespaces with default v2 catalog") { spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 9c28529ed710f..11131310fa4f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -80,15 +80,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { containsThesePhrases = Seq("key_without_value")) } - test("alter database set location") { - // ALTER (DATABASE|SCHEMA) database_name SET LOCATION - val sql1 = "ALTER DATABASE database_name SET LOCATION '/home/user/db'" - val parsed1 = parser.parsePlan(sql1) - - val expected1 = AlterDatabaseSetLocationCommand("database_name", "/home/user/db") - comparePlans(parsed1, expected1) - } - test("create function") { val sql1 = """ From ae6b711b262de21b701434b8e4d031ca265cc376 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 18 Nov 2019 23:30:31 +0800 Subject: [PATCH 0651/1387] [SPARK-29941][SQL] Add ansi type aliases for char and decimal ### What changes were proposed in this pull request? Checked with SQL Standard and PostgreSQL > CHAR is equivalent to CHARACTER. DEC is equivalent to DECIMAL. INT is equivalent to INTEGER. VARCHAR is equivalent to CHARACTER VARYING. ... ```sql postgres=# select dec '1.0'; numeric --------- 1.0 (1 row) postgres=# select CHARACTER '. second'; bpchar ---------- . second (1 row) postgres=# select CHAR '. second'; bpchar ---------- . second (1 row) ``` ### Why are the changes needed? For better ansi support ### Does this PR introduce any user-facing change? yes, we add character as char and dec as decimal ### How was this patch tested? add ut Closes #26574 from yaooqinn/SPARK-29941. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 8 ++++---- .../spark/sql/catalyst/parser/DataTypeParserSuite.scala | 3 +++ 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 59c79f4021fb0..bb8eab0de6089 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2153,12 +2153,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case ("date", Nil) => DateType case ("timestamp", Nil) => TimestampType case ("string", Nil) => StringType - case ("char", length :: Nil) => CharType(length.getText.toInt) + case ("character" | "char", length :: Nil) => CharType(length.getText.toInt) case ("varchar", length :: Nil) => VarcharType(length.getText.toInt) case ("binary", Nil) => BinaryType - case ("decimal", Nil) => DecimalType.USER_DEFAULT - case ("decimal", precision :: Nil) => DecimalType(precision.getText.toInt, 0) - case ("decimal", precision :: scale :: Nil) => + case ("decimal" | "dec", Nil) => DecimalType.USER_DEFAULT + case ("decimal" | "dec", precision :: Nil) => DecimalType(precision.getText.toInt, 0) + case ("decimal" | "dec", precision :: scale :: Nil) => DecimalType(precision.getText.toInt, scale.getText.toInt) case ("interval", Nil) => CalendarIntervalType case (dt, params) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala index 1a6286067a618..d519fdf378786 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala @@ -51,10 +51,13 @@ class DataTypeParserSuite extends SparkFunSuite { checkDataType("dOUBle", DoubleType) checkDataType("decimal(10, 5)", DecimalType(10, 5)) checkDataType("decimal", DecimalType.USER_DEFAULT) + checkDataType("Dec(10, 5)", DecimalType(10, 5)) + checkDataType("deC", DecimalType.USER_DEFAULT) checkDataType("DATE", DateType) checkDataType("timestamp", TimestampType) checkDataType("string", StringType) checkDataType("ChaR(5)", StringType) + checkDataType("ChaRacter(5)", StringType) checkDataType("varchAr(20)", StringType) checkDataType("cHaR(27)", StringType) checkDataType("BINARY", BinaryType) From ea010a2bc226197b3161a5c4db9cfe5a4444c7a4 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 19 Nov 2019 01:32:13 +0800 Subject: [PATCH 0652/1387] [SPARK-29873][SQL][TEST][FOLLOWUP] set operations should not escape when regen golden file with --SET --import both specified ### What changes were proposed in this pull request? When regenerating golden files, the set operations via `--SET` will not be done, but those with --import should be exceptions because we need the set command. ### Why are the changes needed? fix test tool. ### Does this PR introduce any user-facing change? ### How was this patch tested? add ut, but I'm not sure we need these tests for tests itself. cc maropu cloud-fan Closes #26557 from yaooqinn/SPARK-29873. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../inputs/ansi/higher-order-functions.sql | 2 +- .../sql-tests/inputs/ansi/interval.sql | 2 +- .../sql-tests/inputs/ansi/literals.sql | 2 +- .../inputs/interval-display-iso_8601.sql | 2 +- .../inputs/interval-display-sql_standard.sql | 2 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 18 +++++++++++------- 6 files changed, 16 insertions(+), 12 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql index 4068a27fcb2a7..1e2424fe47cad 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql @@ -1 +1 @@ ---import higher-order-functions.sql +--IMPORT higher-order-functions.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql index 215ee7c074fa6..087914eebb077 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql @@ -1,4 +1,4 @@ ---import interval.sql +--IMPORT interval.sql -- the `interval` keyword can be omitted with ansi mode select 1 year 2 days; diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql index 170690ea699c0..698e8fa886307 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql @@ -1,2 +1,2 @@ --- malformed interval literal with ansi mode ---import literals.sql +--IMPORT literals.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql index 62f3f43bd2c48..3b63c715a6aa1 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql @@ -1,3 +1,3 @@ -- tests for interval output style with iso_8601 format --SET spark.sql.intervalOutputStyle = ISO_8601 ---import interval-display.sql +--IMPORT interval-display.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql index 375b4899e760e..d96865b160bb6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql @@ -1,3 +1,3 @@ -- tests for interval output style with sql standard format --SET spark.sql.intervalOutputStyle = SQL_STANDARD ---import interval-display.sql +--IMPORT interval-display.sql diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index d693ff0da33cd..ae0ccf556fbc3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -65,7 +65,7 @@ import org.apache.spark.tags.ExtendedSQLTest * 1. A list of SQL queries separated by semicolon. * 2. Lines starting with -- are treated as comments and ignored. * 3. Lines starting with --SET are used to run the file with the following set of configs. - * 4. Lines starting with --import are used to load queries from another test file. + * 4. Lines starting with --IMPORT are used to load queries from another test file. * * For example: * {{{ @@ -265,9 +265,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { val (comments, code) = input.split("\n").partition(_.trim.startsWith("--")) - // If `--import` found, load code from another test case file, then insert them + // If `--IMPORT` found, load code from another test case file, then insert them // into the head in this test. - val importedTestCaseName = comments.filter(_.startsWith("--import ")).map(_.substring(9)) + val importedTestCaseName = comments.filter(_.startsWith("--IMPORT ")).map(_.substring(9)) val importedCode = importedTestCaseName.flatMap { testCaseName => listTestCases.find(_.name == testCaseName).map { testCase => val input = fileToString(new File(testCase.inputFile)) @@ -283,13 +283,17 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // Fix misplacement when comment is at the end of the query. .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")).map(_.trim).filter(_ != "") - // When we are regenerating the golden files, we don't need to set any config as they - // all need to return the same result - if (regenerateGoldenFiles || !isTestWithConfigSets) { + // When we are regenerating the golden files for test cases without '--IMPORT' specified, or + // running test cases against [[ThriftServerQueryTestSuite], we don't need to set any config as + // they all need to return the same result. + // When we use '--SET' and '--IMPORT' together for those import queries, we want to run the + // same queries from the original file but with different settings and save the answers. So the + // `--SET` will be respected in this case. + if ((regenerateGoldenFiles && importedTestCaseName.isEmpty) || !isTestWithConfigSets) { runQueries(queries, testCase, None) } else { val configSets = { - val configLines = comments.filter(_.startsWith("--SET")).map(_.substring(5)) + val configLines = comments.filter(_.startsWith("--SET ")).map(_.substring(6)) val configs = configLines.map(_.split(",").map { confAndValue => val (conf, value) = confAndValue.span(_ != '=') conf.trim -> value.substring(1).trim From 9514b822a70d77a6298ece48e6c053200360302c Mon Sep 17 00:00:00 2001 From: Hossein Date: Tue, 19 Nov 2019 09:04:59 +0900 Subject: [PATCH 0653/1387] [SPARK-29777][SPARKR] SparkR::cleanClosure aggressively removes a function required by user function ### What changes were proposed in this pull request? The implementation for walking through the user function AST and picking referenced variables and functions, had an optimization to skip a branch if it had already seen it. This runs into an interesting problem in the following example ``` df <- createDataFrame(data.frame(x=1)) f1 <- function(x) x + 1 f2 <- function(x) f1(x) + 2 dapplyCollect(df, function(x) { f1(x); f2(x) }) ``` Results in error: ``` org.apache.spark.SparkException: R computation failed with Error in f1(x) : could not find function "f1" Calls: compute -> computeFunc -> f2 ``` ### Why are the changes needed? Bug fix ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit tests in `test_utils.R` Closes #26429 from falaki/SPARK-29777. Authored-by: Hossein Signed-off-by: HyukjinKwon --- R/pkg/R/utils.R | 8 ++++++-- R/pkg/tests/fulltests/test_utils.R | 9 +++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 15e2410d3a31c..a8c1ddb3dd20b 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -543,10 +543,14 @@ processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { funcList <- mget(nodeChar, envir = checkedFuncs, inherits = F, ifnotfound = list(list(NULL)))[[1]] found <- sapply(funcList, function(func) { - ifelse(identical(func, obj), TRUE, FALSE) + ifelse( + identical(func, obj) && + # Also check if the parent environment is identical to current parent + identical(parent.env(environment(func)), func.env), + TRUE, FALSE) }) if (sum(found) > 0) { - # If function has been examined, ignore. + # If function has been examined ignore break } # Function has not been examined, record it and recursively clean its closure. diff --git a/R/pkg/tests/fulltests/test_utils.R b/R/pkg/tests/fulltests/test_utils.R index b2b6f34aaa085..c4fcbecee18e9 100644 --- a/R/pkg/tests/fulltests/test_utils.R +++ b/R/pkg/tests/fulltests/test_utils.R @@ -110,6 +110,15 @@ test_that("cleanClosure on R functions", { actual <- get("y", envir = env, inherits = FALSE) expect_equal(actual, y) + # Test for combination for nested and sequenctial functions in a closure + f1 <- function(x) x + 1 + f2 <- function(x) f1(x) + 2 + userFunc <- function(x) { f1(x); f2(x) } + cUserFuncEnv <- environment(cleanClosure(userFunc)) + expect_equal(length(cUserFuncEnv), 2) + innerCUserFuncEnv <- environment(cUserFuncEnv$f2) + expect_equal(length(innerCUserFuncEnv), 1) + # Test for function (and variable) definitions. f <- function(x) { g <- function(y) { y * 2 } From 8469614c0513fbed87977d4e741649db3fdd8add Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 19 Nov 2019 09:08:20 +0900 Subject: [PATCH 0654/1387] [SPARK-25694][SQL][FOLLOW-UP] Move 'spark.sql.defaultUrlStreamHandlerFactory.enabled' into StaticSQLConf.scala ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/26530 and proposes to move the configuration `spark.sql.defaultUrlStreamHandlerFactory.enabled` to `StaticSQLConf.scala` for consistency. ### Why are the changes needed? To put the similar configurations together and for readability. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested as described in https://github.com/apache/spark/pull/26530. Closes #26570 from HyukjinKwon/SPARK-25694. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../spark/sql/internal/StaticSQLConf.scala | 7 +++++ .../spark/sql/internal/SharedState.scala | 2 -- .../spark/sql/internal/config/package.scala | 29 ------------------- 3 files changed, 7 insertions(+), 31 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index d665d16ae4195..d2f27da239016 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -161,4 +161,11 @@ object StaticSQLConf { "defaults, dropping any overrides in its parent SparkSession.") .booleanConf .createWithDefault(false) + + val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED = + buildStaticConf("spark.sql.defaultUrlStreamHandlerFactory.enabled") + .doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations") + .internal() + .booleanConf + .createWithDefault(true) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 81a9c76511d8b..de3805e105802 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -36,8 +36,6 @@ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.sql.internal.config.DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED -import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala deleted file mode 100644 index e26c4aadaf135..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala +++ /dev/null @@ -1,29 +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.sql.internal - -import org.apache.spark.internal.config.ConfigBuilder - -package object config { - - private[spark] val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED = - ConfigBuilder("spark.sql.defaultUrlStreamHandlerFactory.enabled") - .doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations") - .booleanConf - .createWithDefault(true) -} From 882f54b0a323fb5cd827d600b3c3332e1fcdf65a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 19 Nov 2019 09:11:41 +0900 Subject: [PATCH 0655/1387] [SPARK-29870][SQL][FOLLOW-UP] Keep CalendarInterval's toString ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/26418. This PR removed `CalendarInterval`'s `toString` with an unfinished changes. ### Why are the changes needed? 1. Ideally we should make each PR isolated and separate targeting one issue without touching unrelated codes. 2. There are some other places where the string formats were exposed to users. For example: ```scala scala> sql("select interval 1 days as a").selectExpr("to_csv(struct(a))").show() ``` ``` +--------------------------+ |to_csv(named_struct(a, a))| +--------------------------+ | "CalendarInterval...| +--------------------------+ ``` 3. Such fixes: ```diff private def writeMapData( map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = { val keyArray = map.keyArray() + val keyString = mapType.keyType match { + case CalendarIntervalType => + (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i)) + case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString + } ``` can cause performance regression due to type dispatch for each map. ### Does this PR introduce any user-facing change? Yes, see 2. case above. ### How was this patch tested? Manually tested. Closes #26572 from HyukjinKwon/SPARK-29783. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../spark/unsafe/types/CalendarInterval.java | 36 +++++++++++++++++-- .../sql/catalyst/expressions/literals.scala | 1 - .../sql/catalyst/json/JacksonGenerator.scala | 11 +----- .../sql/catalyst/util/IntervalUtils.scala | 30 ++-------------- 4 files changed, 37 insertions(+), 41 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 0464e886e1441..d2abb36b0727c 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; +import java.math.BigDecimal; import java.time.Duration; import java.time.Period; import java.time.temporal.ChronoUnit; @@ -79,8 +80,39 @@ public int compareTo(CalendarInterval that) { @Override public String toString() { - return "CalendarInterval(months= " + months + ", days = " + days + ", microsecond = " + - microseconds + ")"; + if (months == 0 && days == 0 && microseconds == 0) { + return "0 seconds"; + } + + StringBuilder sb = new StringBuilder(); + + if (months != 0) { + appendUnit(sb, months / 12, "years"); + appendUnit(sb, months % 12, "months"); + } + + appendUnit(sb, days, "days"); + + if (microseconds != 0) { + long rest = microseconds; + appendUnit(sb, rest / MICROS_PER_HOUR, "hours"); + rest %= MICROS_PER_HOUR; + appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes"); + rest %= MICROS_PER_MINUTE; + if (rest != 0) { + String s = BigDecimal.valueOf(rest, 6).stripTrailingZeros().toPlainString(); + sb.append(s).append(" seconds "); + } + } + + sb.setLength(sb.length() - 1); + return sb.toString(); + } + + private void appendUnit(StringBuilder sb, long value, String unit) { + if (value != 0) { + sb.append(value).append(' ').append(unit).append(' '); + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index f426863de41a1..5a5d7a17acd99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -409,7 +409,6 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) s"TIMESTAMP('${formatter.format(v)}')" case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'" - case (v: CalendarInterval, CalendarIntervalType) => IntervalUtils.toMultiUnitsString(v) case _ => value.toString } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index 544f6e9108325..aaf2ecf7923ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -119,10 +119,6 @@ private[sql] class JacksonGenerator( (row: SpecializedGetters, ordinal: Int) => gen.writeNumber(row.getDouble(ordinal)) - case CalendarIntervalType => - (row: SpecializedGetters, ordinal: Int) => - gen.writeString(IntervalUtils.toMultiUnitsString(row.getInterval(ordinal))) - case StringType => (row: SpecializedGetters, ordinal: Int) => gen.writeString(row.getUTF8String(ordinal).toString) @@ -218,15 +214,10 @@ private[sql] class JacksonGenerator( private def writeMapData( map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = { val keyArray = map.keyArray() - val keyString = mapType.keyType match { - case CalendarIntervalType => - (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i)) - case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString - } val valueArray = map.valueArray() var i = 0 while (i < map.numElements()) { - gen.writeFieldName(keyString(i)) + gen.writeFieldName(keyArray.get(i, mapType.keyType).toString) if (!valueArray.isNullAt(i)) { fieldWriter.apply(valueArray, i) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 725ff81ef410e..61bd9242f3e24 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -332,34 +332,8 @@ object IntervalUtils { fromDoubles(interval.months / num, interval.days / num, interval.microseconds / num) } - def toMultiUnitsString(interval: CalendarInterval): String = { - if (interval.months == 0 && interval.days == 0 && interval.microseconds == 0) { - return "0 seconds" - } - val sb = new StringBuilder - if (interval.months != 0) { - appendUnit(sb, interval.months / 12, "years") - appendUnit(sb, interval.months % 12, "months") - } - appendUnit(sb, interval.days, "days") - if (interval.microseconds != 0) { - var rest = interval.microseconds - appendUnit(sb, rest / MICROS_PER_HOUR, "hours") - rest %= MICROS_PER_HOUR - appendUnit(sb, rest / MICROS_PER_MINUTE, "minutes") - rest %= MICROS_PER_MINUTE - if (rest != 0) { - val s = BigDecimal.valueOf(rest, 6).stripTrailingZeros.toPlainString - sb.append(s).append(" seconds ") - } - } - sb.setLength(sb.length - 1) - sb.toString - } - - private def appendUnit(sb: StringBuilder, value: Long, unit: String): Unit = { - if (value != 0) sb.append(value).append(' ').append(unit).append(' ') - } + // `toString` implementation in CalendarInterval is the multi-units format currently. + def toMultiUnitsString(interval: CalendarInterval): String = interval.toString def toSqlStandardString(interval: CalendarInterval): String = { val yearMonthPart = if (interval.months < 0) { From 28a502c6e92ce44601e798d203364422063c1e07 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 18 Nov 2019 18:13:11 -0800 Subject: [PATCH 0656/1387] [SPARK-28527][FOLLOW-UP][SQL][TEST] Add guides for ThriftServerQueryTestSuite ### What changes were proposed in this pull request? This PR add guides for `ThriftServerQueryTestSuite`. ### Why are the changes needed? Add guides ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #26587 from wangyum/SPARK-28527-FOLLOW-UP. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- .../thriftserver/ThriftServerQueryTestSuite.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index b50a6045f5378..0c56cf9ef5d50 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -37,7 +37,16 @@ import org.apache.spark.sql.types._ /** * Re-run all the tests in SQLQueryTestSuite via Thrift Server. - * Note that this TestSuite does not support maven. + * + * To run the entire test suite: + * {{{ + * build/sbt "hive-thriftserver/test-only *ThriftServerQueryTestSuite" -Phive-thriftserver + * }}} + * + * This test suite won't generate golden files. To re-generate golden files for entire suite, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite" + * }}} * * TODO: * 1. Support UDF testing. @@ -74,6 +83,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } + // We only test this test suite with the default configuration to reduce test time. override val isTestWithConfigSets = false /** List of test cases to ignore, in lower cases. */ From a834dba120e3569e44c5e4b9f8db9c6eef58161b Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 18 Nov 2019 18:44:16 -0800 Subject: [PATCH 0657/1387] Revert "[SPARK-29644][SQL] Corrected ShortType and ByteType mapping to SmallInt and TinyInt in JDBCUtils This reverts commit f7e53865 i.e PR #26301 from master Closes #26583 from shivsood/revert_29644_master. Authored-by: shivsood Signed-off-by: Dongjoon Hyun --- .../jdbc/MsSqlServerIntegrationSuite.scala | 48 ++----------------- .../sql/jdbc/MySQLIntegrationSuite.scala | 4 +- .../datasources/jdbc/JdbcUtils.scala | 12 ++--- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 42 ---------------- 5 files changed, 13 insertions(+), 97 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index f1cd3343b7925..efd7ca74c796b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -59,7 +59,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { """ |INSERT INTO numbers VALUES ( |0, - |127, 32767, 2147483647, 9223372036854775807, + |255, 32767, 2147483647, 9223372036854775807, |123456789012345.123456789012345, 123456789012345.123456789012345, |123456789012345.123456789012345, |123, 12345.12, @@ -119,7 +119,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { val types = row.toSeq.map(x => x.getClass.toString) assert(types.length == 12) assert(types(0).equals("class java.lang.Boolean")) - assert(types(1).equals("class java.lang.Byte")) + assert(types(1).equals("class java.lang.Integer")) assert(types(2).equals("class java.lang.Short")) assert(types(3).equals("class java.lang.Integer")) assert(types(4).equals("class java.lang.Long")) @@ -131,7 +131,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types(10).equals("class java.math.BigDecimal")) assert(types(11).equals("class java.math.BigDecimal")) assert(row.getBoolean(0) == false) - assert(row.getByte(1) == 127) + assert(row.getInt(1) == 255) assert(row.getShort(2) == 32767) assert(row.getInt(3) == 2147483647) assert(row.getLong(4) == 9223372036854775807L) @@ -202,46 +202,4 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { df2.write.jdbc(jdbcUrl, "datescopy", new Properties) df3.write.jdbc(jdbcUrl, "stringscopy", new Properties) } - - test("SPARK-29644: Write tables with ShortType") { - import testImplicits._ - val df = Seq(-32768.toShort, 0.toShort, 1.toShort, 38.toShort, 32768.toShort).toDF("a") - val tablename = "shorttable" - df.write - .format("jdbc") - .mode("overwrite") - .option("url", jdbcUrl) - .option("dbtable", tablename) - .save() - val df2 = spark.read - .format("jdbc") - .option("url", jdbcUrl) - .option("dbtable", tablename) - .load() - assert(df.count == df2.count) - val rows = df2.collect() - val colType = rows(0).toSeq.map(x => x.getClass.toString) - assert(colType(0) == "class java.lang.Short") - } - - test("SPARK-29644: Write tables with ByteType") { - import testImplicits._ - val df = Seq(-127.toByte, 0.toByte, 1.toByte, 38.toByte, 128.toByte).toDF("a") - val tablename = "bytetable" - df.write - .format("jdbc") - .mode("overwrite") - .option("url", jdbcUrl) - .option("dbtable", tablename) - .save() - val df2 = spark.read - .format("jdbc") - .option("url", jdbcUrl) - .option("dbtable", tablename) - .load() - assert(df.count == df2.count) - val rows = df2.collect() - val colType = rows(0).toSeq.map(x => x.getClass.toString) - assert(colType(0) == "class java.lang.Byte") - } } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index 8401b0a8a752f..bba1b5275269b 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -84,7 +84,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types.length == 9) assert(types(0).equals("class java.lang.Boolean")) assert(types(1).equals("class java.lang.Long")) - assert(types(2).equals("class java.lang.Short")) + assert(types(2).equals("class java.lang.Integer")) assert(types(3).equals("class java.lang.Integer")) assert(types(4).equals("class java.lang.Integer")) assert(types(5).equals("class java.lang.Long")) @@ -93,7 +93,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { assert(types(8).equals("class java.lang.Double")) assert(rows(0).getBoolean(0) == false) assert(rows(0).getLong(1) == 0x225) - assert(rows(0).getShort(2) == 17) + assert(rows(0).getInt(2) == 17) assert(rows(0).getInt(3) == 77777) assert(rows(0).getInt(4) == 123456789) assert(rows(0).getLong(5) == 123456789012345L) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index e3baa2a028da2..c1e1aed83bae5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -170,8 +170,8 @@ object JdbcUtils extends Logging { case LongType => Option(JdbcType("BIGINT", java.sql.Types.BIGINT)) case DoubleType => Option(JdbcType("DOUBLE PRECISION", java.sql.Types.DOUBLE)) case FloatType => Option(JdbcType("REAL", java.sql.Types.FLOAT)) - case ShortType => Option(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) - case ByteType => Option(JdbcType("TINYINT", java.sql.Types.TINYINT)) + case ShortType => Option(JdbcType("INTEGER", java.sql.Types.SMALLINT)) + case ByteType => Option(JdbcType("BYTE", java.sql.Types.TINYINT)) case BooleanType => Option(JdbcType("BIT(1)", java.sql.Types.BIT)) case StringType => Option(JdbcType("TEXT", java.sql.Types.CLOB)) case BinaryType => Option(JdbcType("BLOB", java.sql.Types.BLOB)) @@ -235,7 +235,7 @@ object JdbcUtils extends Logging { case java.sql.Types.REF => StringType case java.sql.Types.REF_CURSOR => null case java.sql.Types.ROWID => LongType - case java.sql.Types.SMALLINT => ShortType + case java.sql.Types.SMALLINT => IntegerType case java.sql.Types.SQLXML => StringType case java.sql.Types.STRUCT => StringType case java.sql.Types.TIME => TimestampType @@ -244,7 +244,7 @@ object JdbcUtils extends Logging { case java.sql.Types.TIMESTAMP => TimestampType case java.sql.Types.TIMESTAMP_WITH_TIMEZONE => null - case java.sql.Types.TINYINT => ByteType + case java.sql.Types.TINYINT => IntegerType case java.sql.Types.VARBINARY => BinaryType case java.sql.Types.VARCHAR => StringType case _ => @@ -546,11 +546,11 @@ object JdbcUtils extends Logging { case ShortType => (stmt: PreparedStatement, row: Row, pos: Int) => - stmt.setShort(pos + 1, row.getShort(pos)) + stmt.setInt(pos + 1, row.getShort(pos)) case ByteType => (stmt: PreparedStatement, row: Row, pos: Int) => - stmt.setByte(pos + 1, row.getByte(pos)) + stmt.setInt(pos + 1, row.getByte(pos)) case BooleanType => (stmt: PreparedStatement, row: Row, pos: Int) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 432697015a482..43f6381c19790 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -578,8 +578,8 @@ class JDBCSuite extends QueryTest assert(rows.length === 1) assert(rows(0).getInt(0) === 1) assert(rows(0).getBoolean(1) === false) - assert(rows(0).getByte(2) === 3.toByte) - assert(rows(0).getShort(3) === 4.toShort) + assert(rows(0).getInt(2) === 3) + assert(rows(0).getInt(3) === 4) assert(rows(0).getLong(4) === 1234567890123L) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index f43776ad91987..8021ef1a17a18 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -574,48 +574,6 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter { } } - test("SPARK-29644: Write tables with ShortType") { - import testImplicits._ - val df = Seq(-32768.toShort, 0.toShort, 1.toShort, 38.toShort, 32768.toShort).toDF("a") - val tablename = "shorttable" - df.write - .format("jdbc") - .mode("overwrite") - .option("url", url) - .option("dbtable", tablename) - .save() - val df2 = spark.read - .format("jdbc") - .option("url", url) - .option("dbtable", tablename) - .load() - assert(df.count == df2.count) - val rows = df2.collect() - val colType = rows(0).toSeq.map(x => x.getClass.toString) - assert(colType(0) == "class java.lang.Short") - } - - test("SPARK-29644: Write tables with ByteType") { - import testImplicits._ - val df = Seq(-127.toByte, 0.toByte, 1.toByte, 38.toByte, 128.toByte).toDF("a") - val tablename = "bytetable" - df.write - .format("jdbc") - .mode("overwrite") - .option("url", url) - .option("dbtable", tablename) - .save() - val df2 = spark.read - .format("jdbc") - .option("url", url) - .option("dbtable", tablename) - .load() - assert(df.count == df2.count) - val rows = df2.collect() - val colType = rows(0).toSeq.map(x => x.getClass.toString) - assert(colType(0) == "class java.lang.Byte") - } - private def runAndVerifyRecordsWritten(expected: Long)(job: => Unit): Unit = { assert(expected === runAndReturnMetrics(job, _.taskMetrics.outputMetrics.recordsWritten)) } From 3d45779b687e3f8008044aa251d57024f2a31807 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 19 Nov 2019 12:03:29 +0800 Subject: [PATCH 0658/1387] [SPARK-29728][SQL] Datasource V2: Support ALTER TABLE RENAME TO ### What changes were proposed in this pull request? This PR adds `ALTER TABLE a.b.c RENAME TO x.y.x` support for V2 catalogs. ### Why are the changes needed? The current implementation doesn't support this command V2 catalogs. ### Does this PR introduce any user-facing change? Yes, now the renaming table works for v2 catalogs: ``` scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show +---------+---------+ |namespace|tableName| +---------+---------+ | ns1.ns2| old| +---------+---------+ scala> spark.sql("ALTER TABLE testcat.ns1.ns2.old RENAME TO testcat.ns1.ns2.new").show scala> spark.sql("SHOW TABLES IN testcat.ns1.ns2").show +---------+---------+ |namespace|tableName| +---------+---------+ | ns1.ns2| new| +---------+---------+ ``` ### How was this patch tested? Added unit tests. Closes #26539 from imback82/rename_table. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../catalyst/analysis/ResolveCatalogs.scala | 6 +++ .../sql/catalyst/parser/AstBuilder.scala | 16 ++++++++ .../catalyst/plans/logical/statements.scala | 8 ++++ .../catalyst/plans/logical/v2Commands.scala | 8 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 9 +++++ .../analysis/ResolveSessionCatalog.scala | 3 ++ .../spark/sql/execution/SparkSqlParser.scala | 16 -------- .../datasources/v2/DataSourceV2Strategy.scala | 5 ++- .../datasources/v2/RenameTableExec.scala | 40 +++++++++++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 18 +++++++++ .../execution/command/DDLParserSuite.scala | 26 ------------ 12 files changed, 114 insertions(+), 45 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 13f1f64ba7b8d..41f8e3552977d 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -140,8 +140,8 @@ statement '(' columns=multipartIdentifierList ')' #dropTableColumns | ALTER TABLE multipartIdentifier DROP (COLUMN | COLUMNS) columns=multipartIdentifierList #dropTableColumns - | ALTER (TABLE | VIEW) from=tableIdentifier - RENAME TO to=tableIdentifier #renameTable + | ALTER (TABLE | VIEW) from=multipartIdentifier + RENAME TO to=multipartIdentifier #renameTable | ALTER (TABLE | VIEW) multipartIdentifier SET TBLPROPERTIES tablePropertyList #setTableProperties | ALTER (TABLE | VIEW) multipartIdentifier diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 5a4fb2e865903..2f2e4e619eb4a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -100,6 +100,12 @@ class ResolveCatalogs(val catalogManager: CatalogManager) AlterNamespaceSetProperties( catalog.asNamespaceCatalog, nameParts, Map("location" -> location)) + case RenameTableStatement(NonSessionCatalog(catalog, oldName), newNameParts, isView) => + if (isView) { + throw new AnalysisException("Renaming view is not supported in v2 catalogs.") + } + RenameTable(catalog.asTableCatalog, oldName.asIdentifier, newNameParts.asIdentifier) + case DescribeTableStatement( nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => if (partitionSpec.nonEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index bb8eab0de6089..5b9c204fcd4ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3245,6 +3245,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging query = plan(ctx.query)) } + /** + * Create a [[RenameTableStatement]] command. + * + * For example: + * {{{ + * ALTER TABLE multi_part_name1 RENAME TO multi_part_name2; + * ALTER VIEW multi_part_name1 RENAME TO multi_part_name2; + * }}} + */ + override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { + RenameTableStatement( + visitMultipartIdentifier(ctx.from), + visitMultipartIdentifier(ctx.to), + ctx.VIEW != null) + } + /** * A command for users to list the properties for a table. If propertyKey is specified, the value * for the propertyKey is returned. If propertyKey is not specified, all the keys and their diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 7a98cccc3d7a4..7d7d6bdbfdd2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -246,6 +246,14 @@ case class AlterViewAsStatement( originalText: String, query: LogicalPlan) extends ParsedStatement +/** + * ALTER TABLE ... RENAME TO command, as parsed from SQL. + */ +case class RenameTableStatement( + oldName: Seq[String], + newName: Seq[String], + isView: Boolean) extends ParsedStatement + /** * A DROP TABLE statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index f7f8b2778d234..d87758a7df7b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -400,6 +400,14 @@ case class AlterTable( } } +/** + * The logical plan of the ALTER TABLE RENAME command that works for v2 tables. + */ +case class RenameTable( + catalog: TableCatalog, + oldIdent: Identifier, + newIdent: Identifier) extends Command + /** * The logical plan of the SHOW TABLE command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index d3ca6f7a8eee2..d2575dabf847c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -623,6 +623,15 @@ class DDLParserSuite extends AnalysisTest { } } + test("alter table/view: rename table/view") { + comparePlans( + parsePlan("ALTER TABLE a.b.c RENAME TO x.y.z"), + RenameTableStatement(Seq("a", "b", "c"), Seq("x", "y", "z"), isView = false)) + comparePlans( + parsePlan("ALTER VIEW a.b.c RENAME TO x.y.z"), + RenameTableStatement(Seq("a", "b", "c"), Seq("x", "y", "z"), isView = true)) + } + test("describe table column") { comparePlans(parsePlan("DESCRIBE t col"), DescribeColumnStatement( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 708203119f4bb..eb53e3accc3d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -179,6 +179,9 @@ class ResolveSessionCatalog( } AlterDatabaseSetLocationCommand(nameParts.head, location) + case RenameTableStatement(SessionCatalog(_, oldName), newNameParts, isView) => + AlterTableRenameCommand(oldName.asTableIdentifier, newNameParts.asTableIdentifier, isView) + case DescribeTableStatement( nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => loadTable(catalog, tableName.asIdentifier).collect { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8241f850e3aa7..44e60767e6b1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -316,22 +316,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ctx.TEMPORARY != null) } - /** - * Create a [[AlterTableRenameCommand]] command. - * - * For example: - * {{{ - * ALTER TABLE table1 RENAME TO table2; - * ALTER VIEW view1 RENAME TO view2; - * }}} - */ - override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenameCommand( - visitTableIdentifier(ctx.from), - visitTableIdentifier(ctx.to), - ctx.VIEW != null) - } - /** * Convert a nested constants list into a sequence of string sequences. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 95caebe9e8938..a0d10f1d09e63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetProperties, AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetProperties, AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, RenameTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} @@ -204,6 +204,9 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case AlterTable(catalog, ident, _, changes) => AlterTableExec(catalog, ident, changes) :: Nil + case RenameTable(catalog, oldIdent, newIdent) => + RenameTableExec(catalog, oldIdent, newIdent) :: Nil + case AlterNamespaceSetProperties(catalog, namespace, properties) => AlterNamespaceSetPropertiesExec(catalog, namespace, properties) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala new file mode 100644 index 0000000000000..a650607d5f129 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameTableExec.scala @@ -0,0 +1,40 @@ +/* + * 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.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} + +/** + * Physical plan node for renaming a table. + */ +case class RenameTableExec( + catalog: TableCatalog, + oldIdent: Identifier, + newIdent: Identifier) extends V2CommandExec { + + override def output: Seq[Attribute] = Seq.empty + + override protected def run(): Seq[InternalRow] = { + catalog.invalidateTable(oldIdent) + catalog.renameTable(oldIdent, newIdent) + + Seq.empty + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 36d2deaa309ee..2a44251e102a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1535,6 +1535,24 @@ class DataSourceV2SQLSuite } } + test("AlterTable: rename table basic test") { + withTable("testcat.ns1.new") { + sql(s"CREATE TABLE testcat.ns1.ns2.old USING foo AS SELECT id, data FROM source") + checkAnswer(sql("SHOW TABLES FROM testcat.ns1.ns2"), Seq(Row("ns1.ns2", "old"))) + + sql(s"ALTER TABLE testcat.ns1.ns2.old RENAME TO ns1.new") + checkAnswer(sql("SHOW TABLES FROM testcat.ns1.ns2"), Seq.empty) + checkAnswer(sql("SHOW TABLES FROM testcat.ns1"), Seq(Row("ns1", "new"))) + } + } + + test("AlterTable: renaming views are not supported") { + val e = intercept[AnalysisException] { + sql(s"ALTER VIEW testcat.ns.tbl RENAME TO ns.view") + } + assert(e.getMessage.contains("Renaming view is not supported in v2 catalogs")) + } + test("ANALYZE TABLE") { val t = "testcat.ns1.ns2.tbl" withTable(t) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 11131310fa4f2..41a2168b4a18a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -373,32 +373,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { "Directory path and 'path' in OPTIONS should be specified one, but not both")) } - // ALTER TABLE table_name RENAME TO new_table_name; - // ALTER VIEW view_name RENAME TO new_view_name; - test("alter table/view: rename table/view") { - val sql_table = "ALTER TABLE table_name RENAME TO new_table_name" - val sql_view = sql_table.replace("TABLE", "VIEW") - val parsed_table = parser.parsePlan(sql_table) - val parsed_view = parser.parsePlan(sql_view) - val expected_table = AlterTableRenameCommand( - TableIdentifier("table_name"), - TableIdentifier("new_table_name"), - isView = false) - val expected_view = AlterTableRenameCommand( - TableIdentifier("table_name"), - TableIdentifier("new_table_name"), - isView = true) - comparePlans(parsed_table, expected_table) - comparePlans(parsed_view, expected_view) - } - - test("alter table: rename table with database") { - val query = "ALTER TABLE db1.tbl RENAME TO db1.tbl2" - val plan = parseAs[AlterTableRenameCommand](query) - assert(plan.oldName == TableIdentifier("tbl", Some("db1"))) - assert(plan.newName == TableIdentifier("tbl2", Some("db1"))) - } - test("alter table - property values must be set") { assertUnsupported( sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", From 2e71a6e7ba20743e22a234f5209c1b120f2a7948 Mon Sep 17 00:00:00 2001 From: yudovin Date: Mon, 18 Nov 2019 22:05:34 -0800 Subject: [PATCH 0659/1387] [SPARK-27558][CORE] Gracefully cleanup task when it fails with OOM exception ### What changes were proposed in this pull request? When a task fails with OOM exception, the `UnsafeInMemorySorter.array` could be `null`. In the meanwhile, the `cleanupResources()` on task completion would call `UnsafeInMemorySorter.getMemoryUsage` in turn, and that lead to another NPE thrown. ### Why are the changes needed? Check if `array` is null in `UnsafeInMemorySorter.getMemoryUsage` and it should help to avoid NPE. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? It was tested manually. Closes #26349 from ayudovin/fix-npe-in-listener. Authored-by: yudovin Signed-off-by: Xingbo Jiang --- .../util/collection/unsafe/sort/UnsafeInMemorySorter.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 1a9453a8b3e80..e14964d68119b 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -205,6 +205,10 @@ public long getSortTimeNanos() { } public long getMemoryUsage() { + if (array == null) { + return 0L; + } + return array.size() * 8; } From 5ac37a82656f440e8f58564974668ed9e0ca6b72 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Tue, 19 Nov 2019 15:22:08 +0800 Subject: [PATCH 0660/1387] [SPARK-29869][SQL] improve error message in HiveMetastoreCatalog#convertToLogicalRelation ### What changes were proposed in this pull request? In our production, HiveMetastoreCatalog#convertToLogicalRelation throws AssertError occasionally: ```sql scala> spark.table("hive_table").show java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:208) at org.apache.spark.sql.hive.HiveMetastoreCatalog.convertToLogicalRelation(HiveMetastoreCatalog.scala:261) at org.apache.spark.sql.hive.HiveMetastoreCatalog.convert(HiveMetastoreCatalog.scala:137) at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:220) at org.apache.spark.sql.hive.RelationConversions$$anonfun$apply$4.applyOrElse(HiveStrategies.scala:207) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$2(AnalysisHelper.scala:108) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$1(AnalysisHelper.scala:108) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown(AnalysisHelper.scala:106) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown$(AnalysisHelper.scala:104) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$4(AnalysisHelper.scala:113) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374) at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDown$1(AnalysisHelper.scala:113) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:194) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown(AnalysisHelper.scala:106) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDown$(AnalysisHelper.scala:104) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDown(LogicalPlan.scala:29) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators(AnalysisHelper.scala:73) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators$(AnalysisHelper.scala:72) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:29) at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:207) at org.apache.spark.sql.hive.RelationConversions.apply(HiveStrategies.scala:191) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:130) at scala.collection.IndexedSeqOptimized.foldLeft(IndexedSeqOptimized.scala:60) at scala.collection.IndexedSeqOptimized.foldLeft$(IndexedSeqOptimized.scala:68) at scala.collection.mutable.ArrayBuffer.foldLeft(ArrayBuffer.scala:49) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:127) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:119) at scala.collection.immutable.List.foreach(List.scala:392) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:119) at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:168) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:162) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:122) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:98) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:98) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:146) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:201) at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:145) at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:66) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:63) at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:63) at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:55) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:86) at org.apache.spark.sql.SparkSession.table(SparkSession.scala:585) at org.apache.spark.sql.SparkSession.table(SparkSession.scala:581) ... 47 elided ```` Most of cases occurred in reading a table which created by an old Spark version. After recreated the table, the issue will be gone. After deep dive, the root cause is this external table is a non-partitioned table but the `LOCATION` set to a partitioned path {{/tablename/dt=yyyymmdd}}. The partitionSpec is inferred. ### Why are the changes needed? Above error message is very confused. We need more details about assert failure information. This issue caused by `PartitioningAwareFileIndex#inferPartitioning()`. For non-HiveMetastore Spark, it's useful. But for Hive table, it shouldn't infer partition if Hive tell us it's a non partitioned table. (new added) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add UT. Closes #26499 from LantaoJin/SPARK-29869. Authored-by: LantaoJin Signed-off-by: Wenchen Fan --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 18 ++++++++++++--- .../sql/hive/HiveMetastoreCatalogSuite.scala | 22 ++++++++++++++++++- 2 files changed, 36 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 5ad2caba07fc0..2981e391c0439 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -257,8 +257,20 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } // The inferred schema may have different field names as the table schema, we should respect // it, but also respect the exprId in table relation output. - assert(result.output.length == relation.output.length && - result.output.zip(relation.output).forall { case (a1, a2) => a1.dataType == a2.dataType }) + if (result.output.length != relation.output.length) { + throw new AnalysisException( + s"Converted table has ${result.output.length} columns, " + + s"but source Hive table has ${relation.output.length} columns. " + + s"Set ${HiveUtils.CONVERT_METASTORE_PARQUET.key} to false, " + + s"or recreate table ${relation.tableMeta.identifier} to workaround.") + } + if (!result.output.zip(relation.output).forall { + case (a1, a2) => a1.dataType == a2.dataType }) { + throw new AnalysisException( + s"Column in converted table has different data type with source Hive table's. " + + s"Set ${HiveUtils.CONVERT_METASTORE_PARQUET.key} to false, " + + s"or recreate table ${relation.tableMeta.identifier} to workaround.") + } val newOutput = result.output.zip(relation.output).map { case (a1, a2) => a1.withExprId(a2.exprId) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 0e45e18c4b175..9f2906df19bff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.{QueryTest, Row, SaveMode} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -358,4 +358,24 @@ class DataSourceWithHiveMetastoreCatalogSuite Seq(table("src").count().toString)) } } + + test("SPARK-29869: Fix convertToLogicalRelation throws unclear AssertionError") { + withTempPath(dir => { + val baseDir = s"${dir.getCanonicalFile.toURI.toString}/non_partition_table" + val partitionLikeDir = s"$baseDir/dt=20191113" + spark.range(3).selectExpr("id").write.parquet(partitionLikeDir) + withTable("non_partition_table") { + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { + spark.sql( + s""" + |CREATE TABLE non_partition_table (id bigint) + |STORED AS PARQUET LOCATION '$baseDir' + |""".stripMargin) + val e = intercept[AnalysisException]( + spark.table("non_partition_table")).getMessage + assert(e.contains("Converted table has 2 columns, but source Hive table has 1 columns.")) + } + } + }) + } } From 16134d6d0f349a6880c90770ab7e393a266b978d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Nov 2019 15:37:35 +0800 Subject: [PATCH 0661/1387] [SPARK-29948][SQL] make the default alias consistent between date, timestamp and interval ### What changes were proposed in this pull request? Update `Literal.sql` to make date, timestamp and interval consistent. They should all use the `TYPE 'value'` format. ### Why are the changes needed? Make the default alias consistent. For example, without this patch we will see ``` scala> sql("select interval '1 day', date '2000-10-10'").show +------+-----------------+ |1 days|DATE '2000-10-10'| +------+-----------------+ |1 days| 2000-10-10| +------+-----------------+ ``` ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26579 from cloud-fan/sql. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/literals.scala | 4 +- .../ExpressionSQLBuilderSuite.scala | 6 +- .../expressions/LiteralExpressionSuite.scala | 2 +- .../sql-tests/results/ansi/interval.sql.out | 136 +++++++++--------- .../sql-tests/results/ansi/literals.sql.out | 2 +- .../resources/sql-tests/results/array.sql.out | 2 +- .../resources/sql-tests/results/cast.sql.out | 2 +- .../sql-tests/results/datetime.sql.out | 4 +- .../results/interval-display-iso_8601.sql.out | 2 +- .../interval-display-sql_standard.sql.out | 2 +- .../results/interval-display.sql.out | 2 +- .../sql-tests/results/interval.sql.out | 126 ++++++++-------- .../sql-tests/results/literals.sql.out | 2 +- .../results/postgreSQL/interval.sql.out | 50 +++---- .../subquery/in-subquery/in-joins.sql.out | 13 +- .../typeCoercion/native/arrayJoin.sql.out | 2 +- .../native/dateTimeOperations.sql.out | 78 +++++----- .../sql-tests/results/udf/udf-window.sql.out | 2 +- .../sql-tests/results/window.sql.out | 2 +- 19 files changed, 225 insertions(+), 214 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 5a5d7a17acd99..48b8c9c0fbf8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -407,7 +407,9 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { case (v: Long, TimestampType) => val formatter = TimestampFormatter.getFractionFormatter( DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) - s"TIMESTAMP('${formatter.format(v)}')" + s"TIMESTAMP '${formatter.format(v)}'" + case (i: CalendarInterval, CalendarIntervalType) => + s"INTERVAL '${IntervalUtils.toMultiUnitsString(i)}'" case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'" case _ => value.toString } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala index ada3f7abd7e3a..492d97ba9d524 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSQLBuilderSuite.scala @@ -89,7 +89,7 @@ class ExpressionSQLBuilderSuite extends SparkFunSuite { val timestamp = LocalDateTime.of(2016, 1, 1, 0, 0, 0, 987654321) .atZone(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) .toInstant - checkSQL(Literal(timestamp), "TIMESTAMP('2016-01-01 00:00:00.987654')") + checkSQL(Literal(timestamp), "TIMESTAMP '2016-01-01 00:00:00.987654'") // TODO tests for decimals } @@ -169,12 +169,12 @@ class ExpressionSQLBuilderSuite extends SparkFunSuite { checkSQL( TimeAdd('a, interval), - "`a` + 1 hours" + "`a` + INTERVAL '1 hours'" ) checkSQL( TimeSub('a, interval), - "`a` - 1 hours" + "`a` - INTERVAL '1 hours'" ) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index 03c9cf9c8a94d..4714635a3370b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -302,7 +302,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { val timestamp = LocalDateTime.of(2019, 3, 21, 0, 2, 3, 456000000) .atZone(ZoneOffset.UTC) .toInstant - val expected = "TIMESTAMP('2019-03-21 01:02:03.456')" + val expected = "TIMESTAMP '2019-03-21 01:02:03.456'" val literalStr = Literal.create(timestamp).sql assert(literalStr === expected) } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 73bf299c509cf..11c70d4987eec 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -5,7 +5,7 @@ -- !query 0 select interval '1 day' > interval '23 hour' -- !query 0 schema -struct<(1 days > 23 hours):boolean> +struct<(INTERVAL '1 days' > INTERVAL '23 hours'):boolean> -- !query 0 output true @@ -13,7 +13,7 @@ true -- !query 1 select interval '-1 day' >= interval '-23 hour' -- !query 1 schema -struct<(-1 days >= -23 hours):boolean> +struct<(INTERVAL '-1 days' >= INTERVAL '-23 hours'):boolean> -- !query 1 output false @@ -21,7 +21,7 @@ false -- !query 2 select interval '-1 day' > null -- !query 2 schema -struct<(-1 days > CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '-1 days' > CAST(NULL AS INTERVAL)):boolean> -- !query 2 output NULL @@ -29,7 +29,7 @@ NULL -- !query 3 select null > interval '-1 day' -- !query 3 schema -struct<(CAST(NULL AS INTERVAL) > -1 days):boolean> +struct<(CAST(NULL AS INTERVAL) > INTERVAL '-1 days'):boolean> -- !query 3 output NULL @@ -37,7 +37,7 @@ NULL -- !query 4 select interval '1 minutes' < interval '1 hour' -- !query 4 schema -struct<(1 minutes < 1 hours):boolean> +struct<(INTERVAL '1 minutes' < INTERVAL '1 hours'):boolean> -- !query 4 output true @@ -45,7 +45,7 @@ true -- !query 5 select interval '-1 day' <= interval '-23 hour' -- !query 5 schema -struct<(-1 days <= -23 hours):boolean> +struct<(INTERVAL '-1 days' <= INTERVAL '-23 hours'):boolean> -- !query 5 output true @@ -53,7 +53,7 @@ true -- !query 6 select interval '1 year' = interval '360 days' -- !query 6 schema -struct<(1 years = 360 days):boolean> +struct<(INTERVAL '1 years' = INTERVAL '360 days'):boolean> -- !query 6 output true @@ -61,7 +61,7 @@ true -- !query 7 select interval '1 year 2 month' = interval '420 days' -- !query 7 schema -struct<(1 years 2 months = 420 days):boolean> +struct<(INTERVAL '1 years 2 months' = INTERVAL '420 days'):boolean> -- !query 7 output true @@ -69,7 +69,7 @@ true -- !query 8 select interval '1 year' = interval '365 days' -- !query 8 schema -struct<(1 years = 365 days):boolean> +struct<(INTERVAL '1 years' = INTERVAL '365 days'):boolean> -- !query 8 output false @@ -77,7 +77,7 @@ false -- !query 9 select interval '1 month' = interval '30 days' -- !query 9 schema -struct<(1 months = 30 days):boolean> +struct<(INTERVAL '1 months' = INTERVAL '30 days'):boolean> -- !query 9 output true @@ -85,7 +85,7 @@ true -- !query 10 select interval '1 minutes' = interval '1 hour' -- !query 10 schema -struct<(1 minutes = 1 hours):boolean> +struct<(INTERVAL '1 minutes' = INTERVAL '1 hours'):boolean> -- !query 10 output false @@ -93,7 +93,7 @@ false -- !query 11 select interval '1 minutes' = null -- !query 11 schema -struct<(1 minutes = CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '1 minutes' = CAST(NULL AS INTERVAL)):boolean> -- !query 11 output NULL @@ -101,7 +101,7 @@ NULL -- !query 12 select null = interval '-1 day' -- !query 12 schema -struct<(CAST(NULL AS INTERVAL) = -1 days):boolean> +struct<(CAST(NULL AS INTERVAL) = INTERVAL '-1 days'):boolean> -- !query 12 output NULL @@ -109,7 +109,7 @@ NULL -- !query 13 select interval '1 minutes' <=> null -- !query 13 schema -struct<(1 minutes <=> CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '1 minutes' <=> CAST(NULL AS INTERVAL)):boolean> -- !query 13 output false @@ -117,7 +117,7 @@ false -- !query 14 select null <=> interval '1 minutes' -- !query 14 schema -struct<(CAST(NULL AS INTERVAL) <=> 1 minutes):boolean> +struct<(CAST(NULL AS INTERVAL) <=> INTERVAL '1 minutes'):boolean> -- !query 14 output false @@ -125,7 +125,7 @@ false -- !query 15 select INTERVAL '9 years 1 months -1 weeks -4 days -10 hours -46 minutes' > interval '1 minutes' -- !query 15 schema -struct<(9 years 1 months -11 days -10 hours -46 minutes > 1 minutes):boolean> +struct<(INTERVAL '9 years 1 months -11 days -10 hours -46 minutes' > INTERVAL '1 minutes'):boolean> -- !query 15 output true @@ -143,7 +143,7 @@ struct -- !query 17 select interval '1 month 120 days' > interval '2 month' -- !query 17 schema -struct<(1 months 120 days > 2 months):boolean> +struct<(INTERVAL '1 months 120 days' > INTERVAL '2 months'):boolean> -- !query 17 output true @@ -151,7 +151,7 @@ true -- !query 18 select interval '1 month 30 days' = interval '2 month' -- !query 18 schema -struct<(1 months 30 days = 2 months):boolean> +struct<(INTERVAL '1 months 30 days' = INTERVAL '2 months'):boolean> -- !query 18 output true @@ -159,7 +159,7 @@ true -- !query 19 select interval '1 month 29 days 40 hours' > interval '2 month' -- !query 19 schema -struct<(1 months 29 days 40 hours > 2 months):boolean> +struct<(INTERVAL '1 months 29 days 40 hours' > INTERVAL '2 months'):boolean> -- !query 19 output true @@ -183,7 +183,7 @@ struct -- !query 22 select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct +struct -- !query 22 output 30 hours 33 minutes 36.003006 seconds @@ -191,7 +191,7 @@ struct +struct -- !query 23 output 6 months 21 days 0.000005 seconds @@ -199,7 +199,7 @@ struct +struct -- !query 24 output 16 hours @@ -207,7 +207,7 @@ struct +struct -- !query 25 output NULL @@ -215,7 +215,7 @@ NULL -- !query 26 select interval '2 seconds' / null -- !query 26 schema -struct +struct -- !query 26 output NULL @@ -223,7 +223,7 @@ NULL -- !query 27 select interval '2 seconds' * null -- !query 27 schema -struct +struct -- !query 27 output NULL @@ -231,7 +231,7 @@ NULL -- !query 28 select null * interval '2 seconds' -- !query 28 schema -struct +struct -- !query 28 output NULL @@ -239,7 +239,7 @@ NULL -- !query 29 select -interval '-1 month 1 day -1 second' -- !query 29 schema -struct<1 months -1 days 1 seconds:interval> +struct -- !query 29 output 1 months -1 days 1 seconds @@ -247,7 +247,7 @@ struct<1 months -1 days 1 seconds:interval> -- !query 30 select -interval -1 month 1 day -1 second -- !query 30 schema -struct<1 months -1 days 1 seconds:interval> +struct -- !query 30 output 1 months -1 days 1 seconds @@ -255,7 +255,7 @@ struct<1 months -1 days 1 seconds:interval> -- !query 31 select +interval '-1 month 1 day -1 second' -- !query 31 schema -struct<-1 months 1 days -1 seconds:interval> +struct -- !query 31 output -1 months 1 days -1 seconds @@ -263,7 +263,7 @@ struct<-1 months 1 days -1 seconds:interval> -- !query 32 select +interval -1 month 1 day -1 second -- !query 32 schema -struct<-1 months 1 days -1 seconds:interval> +struct -- !query 32 output -1 months 1 days -1 seconds @@ -407,7 +407,7 @@ NULL -- !query 50 select justify_days(interval '1 month 59 day 25 hour') -- !query 50 schema -struct +struct -- !query 50 output 2 months 29 days 25 hours @@ -415,7 +415,7 @@ struct -- !query 51 select justify_hours(interval '1 month 59 day 25 hour') -- !query 51 schema -struct +struct -- !query 51 output 1 months 60 days 1 hours @@ -423,7 +423,7 @@ struct -- !query 52 select justify_interval(interval '1 month 59 day 25 hour') -- !query 52 schema -struct +struct -- !query 52 output 3 months 1 hours @@ -431,7 +431,7 @@ struct -- !query 53 select justify_days(interval '1 month -59 day 25 hour') -- !query 53 schema -struct +struct -- !query 53 output -29 days 25 hours @@ -439,7 +439,7 @@ struct -- !query 54 select justify_hours(interval '1 month -59 day 25 hour') -- !query 54 schema -struct +struct -- !query 54 output 1 months -57 days -23 hours @@ -447,7 +447,7 @@ struct -- !query 55 select justify_interval(interval '1 month -59 day 25 hour') -- !query 55 schema -struct +struct -- !query 55 output -27 days -23 hours @@ -455,7 +455,7 @@ struct -- !query 56 select justify_days(interval '1 month 59 day -25 hour') -- !query 56 schema -struct +struct -- !query 56 output 2 months 29 days -25 hours @@ -463,7 +463,7 @@ struct -- !query 57 select justify_hours(interval '1 month 59 day -25 hour') -- !query 57 schema -struct +struct -- !query 57 output 1 months 57 days 23 hours @@ -471,7 +471,7 @@ struct -- !query 58 select justify_interval(interval '1 month 59 day -25 hour') -- !query 58 schema -struct +struct -- !query 58 output 2 months 27 days 23 hours @@ -479,7 +479,7 @@ struct -- !query 59 select interval 13.123456789 seconds, interval -13.123456789 second -- !query 59 schema -struct<13.123456 seconds:interval,-13.123456 seconds:interval> +struct -- !query 59 output 13.123456 seconds -13.123456 seconds @@ -487,7 +487,7 @@ struct<13.123456 seconds:interval,-13.123456 seconds:interval> -- !query 60 select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond -- !query 60 schema -struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> +struct -- !query 60 output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds @@ -495,7 +495,7 @@ struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> -- !query 61 select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 61 schema -struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> +struct -- !query 61 output 32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds @@ -503,7 +503,7 @@ struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interva -- !query 62 select interval '0 0:0:0.1' day to second -- !query 62 schema -struct<0.1 seconds:interval> +struct -- !query 62 output 0.1 seconds @@ -511,7 +511,7 @@ struct<0.1 seconds:interval> -- !query 63 select interval '10-9' year to month -- !query 63 schema -struct<10 years 9 months:interval> +struct -- !query 63 output 10 years 9 months @@ -519,7 +519,7 @@ struct<10 years 9 months:interval> -- !query 64 select interval '20 15:40:32.99899999' day to hour -- !query 64 schema -struct<20 days 15 hours:interval> +struct -- !query 64 output 20 days 15 hours @@ -527,7 +527,7 @@ struct<20 days 15 hours:interval> -- !query 65 select interval '20 15:40:32.99899999' day to minute -- !query 65 schema -struct<20 days 15 hours 40 minutes:interval> +struct -- !query 65 output 20 days 15 hours 40 minutes @@ -535,7 +535,7 @@ struct<20 days 15 hours 40 minutes:interval> -- !query 66 select interval '20 15:40:32.99899999' day to second -- !query 66 schema -struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 66 output 20 days 15 hours 40 minutes 32.998999 seconds @@ -543,7 +543,7 @@ struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> -- !query 67 select interval '15:40:32.99899999' hour to minute -- !query 67 schema -struct<15 hours 40 minutes:interval> +struct -- !query 67 output 15 hours 40 minutes @@ -551,7 +551,7 @@ struct<15 hours 40 minutes:interval> -- !query 68 select interval '15:40.99899999' hour to second -- !query 68 schema -struct<15 minutes 40.998999 seconds:interval> +struct -- !query 68 output 15 minutes 40.998999 seconds @@ -559,7 +559,7 @@ struct<15 minutes 40.998999 seconds:interval> -- !query 69 select interval '15:40' hour to second -- !query 69 schema -struct<15 hours 40 minutes:interval> +struct -- !query 69 output 15 hours 40 minutes @@ -567,7 +567,7 @@ struct<15 hours 40 minutes:interval> -- !query 70 select interval '15:40:32.99899999' hour to second -- !query 70 schema -struct<15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 70 output 15 hours 40 minutes 32.998999 seconds @@ -575,7 +575,7 @@ struct<15 hours 40 minutes 32.998999 seconds:interval> -- !query 71 select interval '20 40:32.99899999' minute to second -- !query 71 schema -struct<20 days 40 minutes 32.998999 seconds:interval> +struct -- !query 71 output 20 days 40 minutes 32.998999 seconds @@ -583,7 +583,7 @@ struct<20 days 40 minutes 32.998999 seconds:interval> -- !query 72 select interval '40:32.99899999' minute to second -- !query 72 schema -struct<40 minutes 32.998999 seconds:interval> +struct -- !query 72 output 40 minutes 32.998999 seconds @@ -591,7 +591,7 @@ struct<40 minutes 32.998999 seconds:interval> -- !query 73 select interval '40:32' minute to second -- !query 73 schema -struct<40 minutes 32 seconds:interval> +struct -- !query 73 output 40 minutes 32 seconds @@ -627,7 +627,7 @@ select interval 10 nanoseconds -- !query 76 select map(1, interval 1 day, 2, interval 3 week) -- !query 76 schema -struct> +struct> -- !query 76 output {1:1 days,2:21 days} @@ -635,7 +635,7 @@ struct> -- !query 77 select interval 'interval 3 year 1 hour' -- !query 77 schema -struct<3 years 1 hours:interval> +struct -- !query 77 output 3 years 1 hours @@ -643,7 +643,7 @@ struct<3 years 1 hours:interval> -- !query 78 select interval '3 year 1 hour' -- !query 78 schema -struct<3 years 1 hours:interval> +struct -- !query 78 output 3 years 1 hours @@ -1040,7 +1040,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query 112 schema -struct +struct -- !query 112 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -1056,7 +1056,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query 113 schema -struct +struct -- !query 113 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -1067,7 +1067,7 @@ select interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic -- !query 114 schema -struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> +struct<(INTERVAL '2 years 2 months' + INTERVAL '3 years 3 months'):interval,(INTERVAL '2 years 2 months' - INTERVAL '3 years 3 months'):interval> -- !query 114 output 5 years 5 months -1 years -1 months @@ -1083,7 +1083,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query 115 schema -struct +struct -- !query 115 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -1099,7 +1099,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query 116 schema -struct +struct -- !query 116 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 @@ -1110,7 +1110,7 @@ select interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic -- !query 117 schema -struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> +struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> -- !query 117 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds @@ -1118,7 +1118,7 @@ struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minute -- !query 118 select 1 year 2 days -- !query 118 schema -struct<1 years 2 days:interval> +struct -- !query 118 output 1 years 2 days @@ -1126,7 +1126,7 @@ struct<1 years 2 days:interval> -- !query 119 select '10-9' year to month -- !query 119 schema -struct<10 years 9 months:interval> +struct -- !query 119 output 10 years 9 months @@ -1134,7 +1134,7 @@ struct<10 years 9 months:interval> -- !query 120 select '20 15:40:32.99899999' day to second -- !query 120 schema -struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 120 output 20 days 15 hours 40 minutes 32.998999 seconds @@ -1156,7 +1156,7 @@ select 30 day day -- !query 122 select date'2012-01-01' - '2-2' year to month -- !query 122 schema -struct +struct -- !query 122 output 2009-11-01 @@ -1164,7 +1164,7 @@ struct +struct -- !query 123 output 1 months -1 days diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index e43e88c8c72a8..6ee7148755c34 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -301,7 +301,7 @@ select date 'mar 11 2016' -- !query 32 select tImEstAmp '2016-03-11 20:54:00.000' -- !query 32 schema -struct +struct -- !query 32 output 2016-03-11 20:54:00 diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 5f5d988771847..4195205e275ae 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -92,7 +92,7 @@ select array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') from primitive_arrays -- !query 6 schema -struct +struct -- !query 6 output true false true false true false true false true false true false true false true false true false true false diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 4884f9d491f2d..609d283da555c 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -284,6 +284,6 @@ struct -- !query 34 SELECT CAST(interval 3 month 1 hour AS string) -- !query 34 schema -struct +struct -- !query 34 output 3 months 1 hours diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index a40f5acb6d538..b2c6b878f4715 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -134,7 +134,7 @@ struct -- !query 15 select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' -- !query 15 schema -struct +struct -- !query 15 output 2078 hours 48 minutes 47.654322 seconds @@ -142,6 +142,6 @@ struct +struct -- !query 16 output -2078 hours -48 minutes -47.654322 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out index 57fe8a3f4fcc6..0e053940df38f 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL PT0S P1Y P1M P1Y2M P1DT-1H P-1DT-1H P-1DT1H P-1DT1H P1Y2M-3DT4H5M6.789S P-1Y-2M3DT-4H-5M-6.789S diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out index 9e40f52151475..2633522bb1d08 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL 0 +1-0 +0-1 +1-2 +1 -1:00:00 -1 -1:00:00 -1 +1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out index 340496e404326..faf27bda1549f 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL 0 seconds 1 years 1 months 1 years 2 months 1 days -1 hours -1 days -1 hours -1 days 1 hours -1 days 1 hours 1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds -1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index d0f2ff4b44d01..6d09f62a6dcec 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -5,7 +5,7 @@ -- !query 0 select interval '1 day' > interval '23 hour' -- !query 0 schema -struct<(1 days > 23 hours):boolean> +struct<(INTERVAL '1 days' > INTERVAL '23 hours'):boolean> -- !query 0 output true @@ -13,7 +13,7 @@ true -- !query 1 select interval '-1 day' >= interval '-23 hour' -- !query 1 schema -struct<(-1 days >= -23 hours):boolean> +struct<(INTERVAL '-1 days' >= INTERVAL '-23 hours'):boolean> -- !query 1 output false @@ -21,7 +21,7 @@ false -- !query 2 select interval '-1 day' > null -- !query 2 schema -struct<(-1 days > CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '-1 days' > CAST(NULL AS INTERVAL)):boolean> -- !query 2 output NULL @@ -29,7 +29,7 @@ NULL -- !query 3 select null > interval '-1 day' -- !query 3 schema -struct<(CAST(NULL AS INTERVAL) > -1 days):boolean> +struct<(CAST(NULL AS INTERVAL) > INTERVAL '-1 days'):boolean> -- !query 3 output NULL @@ -37,7 +37,7 @@ NULL -- !query 4 select interval '1 minutes' < interval '1 hour' -- !query 4 schema -struct<(1 minutes < 1 hours):boolean> +struct<(INTERVAL '1 minutes' < INTERVAL '1 hours'):boolean> -- !query 4 output true @@ -45,7 +45,7 @@ true -- !query 5 select interval '-1 day' <= interval '-23 hour' -- !query 5 schema -struct<(-1 days <= -23 hours):boolean> +struct<(INTERVAL '-1 days' <= INTERVAL '-23 hours'):boolean> -- !query 5 output true @@ -53,7 +53,7 @@ true -- !query 6 select interval '1 year' = interval '360 days' -- !query 6 schema -struct<(1 years = 360 days):boolean> +struct<(INTERVAL '1 years' = INTERVAL '360 days'):boolean> -- !query 6 output true @@ -61,7 +61,7 @@ true -- !query 7 select interval '1 year 2 month' = interval '420 days' -- !query 7 schema -struct<(1 years 2 months = 420 days):boolean> +struct<(INTERVAL '1 years 2 months' = INTERVAL '420 days'):boolean> -- !query 7 output true @@ -69,7 +69,7 @@ true -- !query 8 select interval '1 year' = interval '365 days' -- !query 8 schema -struct<(1 years = 365 days):boolean> +struct<(INTERVAL '1 years' = INTERVAL '365 days'):boolean> -- !query 8 output false @@ -77,7 +77,7 @@ false -- !query 9 select interval '1 month' = interval '30 days' -- !query 9 schema -struct<(1 months = 30 days):boolean> +struct<(INTERVAL '1 months' = INTERVAL '30 days'):boolean> -- !query 9 output true @@ -85,7 +85,7 @@ true -- !query 10 select interval '1 minutes' = interval '1 hour' -- !query 10 schema -struct<(1 minutes = 1 hours):boolean> +struct<(INTERVAL '1 minutes' = INTERVAL '1 hours'):boolean> -- !query 10 output false @@ -93,7 +93,7 @@ false -- !query 11 select interval '1 minutes' = null -- !query 11 schema -struct<(1 minutes = CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '1 minutes' = CAST(NULL AS INTERVAL)):boolean> -- !query 11 output NULL @@ -101,7 +101,7 @@ NULL -- !query 12 select null = interval '-1 day' -- !query 12 schema -struct<(CAST(NULL AS INTERVAL) = -1 days):boolean> +struct<(CAST(NULL AS INTERVAL) = INTERVAL '-1 days'):boolean> -- !query 12 output NULL @@ -109,7 +109,7 @@ NULL -- !query 13 select interval '1 minutes' <=> null -- !query 13 schema -struct<(1 minutes <=> CAST(NULL AS INTERVAL)):boolean> +struct<(INTERVAL '1 minutes' <=> CAST(NULL AS INTERVAL)):boolean> -- !query 13 output false @@ -117,7 +117,7 @@ false -- !query 14 select null <=> interval '1 minutes' -- !query 14 schema -struct<(CAST(NULL AS INTERVAL) <=> 1 minutes):boolean> +struct<(CAST(NULL AS INTERVAL) <=> INTERVAL '1 minutes'):boolean> -- !query 14 output false @@ -125,7 +125,7 @@ false -- !query 15 select INTERVAL '9 years 1 months -1 weeks -4 days -10 hours -46 minutes' > interval '1 minutes' -- !query 15 schema -struct<(9 years 1 months -11 days -10 hours -46 minutes > 1 minutes):boolean> +struct<(INTERVAL '9 years 1 months -11 days -10 hours -46 minutes' > INTERVAL '1 minutes'):boolean> -- !query 15 output true @@ -143,7 +143,7 @@ struct -- !query 17 select interval '1 month 120 days' > interval '2 month' -- !query 17 schema -struct<(1 months 120 days > 2 months):boolean> +struct<(INTERVAL '1 months 120 days' > INTERVAL '2 months'):boolean> -- !query 17 output true @@ -151,7 +151,7 @@ true -- !query 18 select interval '1 month 30 days' = interval '2 month' -- !query 18 schema -struct<(1 months 30 days = 2 months):boolean> +struct<(INTERVAL '1 months 30 days' = INTERVAL '2 months'):boolean> -- !query 18 output true @@ -159,7 +159,7 @@ true -- !query 19 select interval '1 month 29 days 40 hours' > interval '2 month' -- !query 19 schema -struct<(1 months 29 days 40 hours > 2 months):boolean> +struct<(INTERVAL '1 months 29 days 40 hours' > INTERVAL '2 months'):boolean> -- !query 19 output true @@ -183,7 +183,7 @@ struct -- !query 22 select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct +struct -- !query 22 output 30 hours 33 minutes 36.003006 seconds @@ -191,7 +191,7 @@ struct +struct -- !query 23 output 6 months 21 days 0.000005 seconds @@ -199,7 +199,7 @@ struct +struct -- !query 24 output 16 hours @@ -207,7 +207,7 @@ struct +struct -- !query 25 output NULL @@ -215,7 +215,7 @@ NULL -- !query 26 select interval '2 seconds' / null -- !query 26 schema -struct +struct -- !query 26 output NULL @@ -223,7 +223,7 @@ NULL -- !query 27 select interval '2 seconds' * null -- !query 27 schema -struct +struct -- !query 27 output NULL @@ -231,7 +231,7 @@ NULL -- !query 28 select null * interval '2 seconds' -- !query 28 schema -struct +struct -- !query 28 output NULL @@ -239,7 +239,7 @@ NULL -- !query 29 select -interval '-1 month 1 day -1 second' -- !query 29 schema -struct<1 months -1 days 1 seconds:interval> +struct -- !query 29 output 1 months -1 days 1 seconds @@ -247,7 +247,7 @@ struct<1 months -1 days 1 seconds:interval> -- !query 30 select -interval -1 month 1 day -1 second -- !query 30 schema -struct<1 months -1 days 1 seconds:interval> +struct -- !query 30 output 1 months -1 days 1 seconds @@ -255,7 +255,7 @@ struct<1 months -1 days 1 seconds:interval> -- !query 31 select +interval '-1 month 1 day -1 second' -- !query 31 schema -struct<-1 months 1 days -1 seconds:interval> +struct -- !query 31 output -1 months 1 days -1 seconds @@ -263,7 +263,7 @@ struct<-1 months 1 days -1 seconds:interval> -- !query 32 select +interval -1 month 1 day -1 second -- !query 32 schema -struct<-1 months 1 days -1 seconds:interval> +struct -- !query 32 output -1 months 1 days -1 seconds @@ -407,7 +407,7 @@ NULL -- !query 50 select justify_days(interval '1 month 59 day 25 hour') -- !query 50 schema -struct +struct -- !query 50 output 2 months 29 days 25 hours @@ -415,7 +415,7 @@ struct -- !query 51 select justify_hours(interval '1 month 59 day 25 hour') -- !query 51 schema -struct +struct -- !query 51 output 1 months 60 days 1 hours @@ -423,7 +423,7 @@ struct -- !query 52 select justify_interval(interval '1 month 59 day 25 hour') -- !query 52 schema -struct +struct -- !query 52 output 3 months 1 hours @@ -431,7 +431,7 @@ struct -- !query 53 select justify_days(interval '1 month -59 day 25 hour') -- !query 53 schema -struct +struct -- !query 53 output -29 days 25 hours @@ -439,7 +439,7 @@ struct -- !query 54 select justify_hours(interval '1 month -59 day 25 hour') -- !query 54 schema -struct +struct -- !query 54 output 1 months -57 days -23 hours @@ -447,7 +447,7 @@ struct -- !query 55 select justify_interval(interval '1 month -59 day 25 hour') -- !query 55 schema -struct +struct -- !query 55 output -27 days -23 hours @@ -455,7 +455,7 @@ struct -- !query 56 select justify_days(interval '1 month 59 day -25 hour') -- !query 56 schema -struct +struct -- !query 56 output 2 months 29 days -25 hours @@ -463,7 +463,7 @@ struct -- !query 57 select justify_hours(interval '1 month 59 day -25 hour') -- !query 57 schema -struct +struct -- !query 57 output 1 months 57 days 23 hours @@ -471,7 +471,7 @@ struct -- !query 58 select justify_interval(interval '1 month 59 day -25 hour') -- !query 58 schema -struct +struct -- !query 58 output 2 months 27 days 23 hours @@ -479,7 +479,7 @@ struct -- !query 59 select interval 13.123456789 seconds, interval -13.123456789 second -- !query 59 schema -struct<13.123456 seconds:interval,-13.123456 seconds:interval> +struct -- !query 59 output 13.123456 seconds -13.123456 seconds @@ -487,7 +487,7 @@ struct<13.123456 seconds:interval,-13.123456 seconds:interval> -- !query 60 select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond -- !query 60 schema -struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> +struct -- !query 60 output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds @@ -495,7 +495,7 @@ struct<1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds:interval> -- !query 61 select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second -- !query 61 schema -struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interval> +struct -- !query 61 output 32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds @@ -503,7 +503,7 @@ struct<32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds:interva -- !query 62 select interval '0 0:0:0.1' day to second -- !query 62 schema -struct<0.1 seconds:interval> +struct -- !query 62 output 0.1 seconds @@ -511,7 +511,7 @@ struct<0.1 seconds:interval> -- !query 63 select interval '10-9' year to month -- !query 63 schema -struct<10 years 9 months:interval> +struct -- !query 63 output 10 years 9 months @@ -519,7 +519,7 @@ struct<10 years 9 months:interval> -- !query 64 select interval '20 15:40:32.99899999' day to hour -- !query 64 schema -struct<20 days 15 hours:interval> +struct -- !query 64 output 20 days 15 hours @@ -527,7 +527,7 @@ struct<20 days 15 hours:interval> -- !query 65 select interval '20 15:40:32.99899999' day to minute -- !query 65 schema -struct<20 days 15 hours 40 minutes:interval> +struct -- !query 65 output 20 days 15 hours 40 minutes @@ -535,7 +535,7 @@ struct<20 days 15 hours 40 minutes:interval> -- !query 66 select interval '20 15:40:32.99899999' day to second -- !query 66 schema -struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 66 output 20 days 15 hours 40 minutes 32.998999 seconds @@ -543,7 +543,7 @@ struct<20 days 15 hours 40 minutes 32.998999 seconds:interval> -- !query 67 select interval '15:40:32.99899999' hour to minute -- !query 67 schema -struct<15 hours 40 minutes:interval> +struct -- !query 67 output 15 hours 40 minutes @@ -551,7 +551,7 @@ struct<15 hours 40 minutes:interval> -- !query 68 select interval '15:40.99899999' hour to second -- !query 68 schema -struct<15 minutes 40.998999 seconds:interval> +struct -- !query 68 output 15 minutes 40.998999 seconds @@ -559,7 +559,7 @@ struct<15 minutes 40.998999 seconds:interval> -- !query 69 select interval '15:40' hour to second -- !query 69 schema -struct<15 hours 40 minutes:interval> +struct -- !query 69 output 15 hours 40 minutes @@ -567,7 +567,7 @@ struct<15 hours 40 minutes:interval> -- !query 70 select interval '15:40:32.99899999' hour to second -- !query 70 schema -struct<15 hours 40 minutes 32.998999 seconds:interval> +struct -- !query 70 output 15 hours 40 minutes 32.998999 seconds @@ -575,7 +575,7 @@ struct<15 hours 40 minutes 32.998999 seconds:interval> -- !query 71 select interval '20 40:32.99899999' minute to second -- !query 71 schema -struct<20 days 40 minutes 32.998999 seconds:interval> +struct -- !query 71 output 20 days 40 minutes 32.998999 seconds @@ -583,7 +583,7 @@ struct<20 days 40 minutes 32.998999 seconds:interval> -- !query 72 select interval '40:32.99899999' minute to second -- !query 72 schema -struct<40 minutes 32.998999 seconds:interval> +struct -- !query 72 output 40 minutes 32.998999 seconds @@ -591,7 +591,7 @@ struct<40 minutes 32.998999 seconds:interval> -- !query 73 select interval '40:32' minute to second -- !query 73 schema -struct<40 minutes 32 seconds:interval> +struct -- !query 73 output 40 minutes 32 seconds @@ -621,7 +621,7 @@ select interval 10 nanoseconds -- !query 76 select map(1, interval 1 day, 2, interval 3 week) -- !query 76 schema -struct> +struct> -- !query 76 output {1:1 days,2:21 days} @@ -629,7 +629,7 @@ struct> -- !query 77 select interval 'interval 3 year 1 hour' -- !query 77 schema -struct<3 years 1 hours:interval> +struct -- !query 77 output 3 years 1 hours @@ -637,7 +637,7 @@ struct<3 years 1 hours:interval> -- !query 78 select interval '3 year 1 hour' -- !query 78 schema -struct<3 years 1 hours:interval> +struct -- !query 78 output 3 years 1 hours @@ -1024,7 +1024,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query 112 schema -struct +struct -- !query 112 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -1040,7 +1040,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query 113 schema -struct +struct -- !query 113 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -1051,7 +1051,7 @@ select interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic -- !query 114 schema -struct<(2 years 2 months + 3 years 3 months):interval,(2 years 2 months - 3 years 3 months):interval> +struct<(INTERVAL '2 years 2 months' + INTERVAL '3 years 3 months'):interval,(INTERVAL '2 years 2 months' - INTERVAL '3 years 3 months'):interval> -- !query 114 output 5 years 5 months -1 years -1 months @@ -1067,7 +1067,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query 115 schema -struct +struct -- !query 115 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -1083,7 +1083,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query 116 schema -struct +struct -- !query 116 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 @@ -1094,6 +1094,6 @@ select interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic -- !query 117 schema -struct<(99 days 11 hours 22 minutes 33.123456 seconds + 10 days 9 hours 8 minutes 7.123456 seconds):interval,(99 days 11 hours 22 minutes 33.123456 seconds - 10 days 9 hours 8 minutes 7.123456 seconds):interval> +struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> -- !query 117 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index e43e88c8c72a8..6ee7148755c34 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -301,7 +301,7 @@ select date 'mar 11 2016' -- !query 32 select tImEstAmp '2016-03-11 20:54:00.000' -- !query 32 schema -struct +struct -- !query 32 output 2016-03-11 20:54:00 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index d981ed15e37f4..4eaeee6030d66 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -29,7 +29,7 @@ struct<1 month -1 hour:interval> -- !query 3 SELECT interval '999' second -- !query 3 schema -struct<16 minutes 39 seconds:interval> +struct -- !query 3 output 16 minutes 39 seconds @@ -37,7 +37,7 @@ struct<16 minutes 39 seconds:interval> -- !query 4 SELECT interval '999' minute -- !query 4 schema -struct<16 hours 39 minutes:interval> +struct -- !query 4 output 16 hours 39 minutes @@ -45,7 +45,7 @@ struct<16 hours 39 minutes:interval> -- !query 5 SELECT interval '999' hour -- !query 5 schema -struct<999 hours:interval> +struct -- !query 5 output 999 hours @@ -53,7 +53,7 @@ struct<999 hours:interval> -- !query 6 SELECT interval '999' day -- !query 6 schema -struct<999 days:interval> +struct -- !query 6 output 999 days @@ -61,7 +61,7 @@ struct<999 days:interval> -- !query 7 SELECT interval '999' month -- !query 7 schema -struct<83 years 3 months:interval> +struct -- !query 7 output 83 years 3 months @@ -69,7 +69,7 @@ struct<83 years 3 months:interval> -- !query 8 SELECT interval '1' year -- !query 8 schema -struct<1 years:interval> +struct -- !query 8 output 1 years @@ -77,7 +77,7 @@ struct<1 years:interval> -- !query 9 SELECT interval '2' month -- !query 9 schema -struct<2 months:interval> +struct -- !query 9 output 2 months @@ -85,7 +85,7 @@ struct<2 months:interval> -- !query 10 SELECT interval '3' day -- !query 10 schema -struct<3 days:interval> +struct -- !query 10 output 3 days @@ -93,7 +93,7 @@ struct<3 days:interval> -- !query 11 SELECT interval '4' hour -- !query 11 schema -struct<4 hours:interval> +struct -- !query 11 output 4 hours @@ -101,7 +101,7 @@ struct<4 hours:interval> -- !query 12 SELECT interval '5' minute -- !query 12 schema -struct<5 minutes:interval> +struct -- !query 12 output 5 minutes @@ -109,7 +109,7 @@ struct<5 minutes:interval> -- !query 13 SELECT interval '6' second -- !query 13 schema -struct<6 seconds:interval> +struct -- !query 13 output 6 seconds @@ -117,7 +117,7 @@ struct<6 seconds:interval> -- !query 14 SELECT interval '1-2' year to month -- !query 14 schema -struct<1 years 2 months:interval> +struct -- !query 14 output 1 years 2 months @@ -125,7 +125,7 @@ struct<1 years 2 months:interval> -- !query 15 SELECT interval '1 2:03' day to hour -- !query 15 schema -struct<1 days 2 hours:interval> +struct -- !query 15 output 1 days 2 hours @@ -133,7 +133,7 @@ struct<1 days 2 hours:interval> -- !query 16 SELECT interval '1 2:03:04' day to hour -- !query 16 schema -struct<1 days 2 hours:interval> +struct -- !query 16 output 1 days 2 hours @@ -141,7 +141,7 @@ struct<1 days 2 hours:interval> -- !query 17 SELECT interval '1 2:03' day to minute -- !query 17 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 17 output 1 days 2 hours 3 minutes @@ -149,7 +149,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 18 SELECT interval '1 2:03:04' day to minute -- !query 18 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 18 output 1 days 2 hours 3 minutes @@ -157,7 +157,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 19 SELECT interval '1 2:03' day to second -- !query 19 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 19 output 1 days 2 hours 3 minutes @@ -165,7 +165,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 20 SELECT interval '1 2:03:04' day to second -- !query 20 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct -- !query 20 output 1 days 2 hours 3 minutes 4 seconds @@ -173,7 +173,7 @@ struct<1 days 2 hours 3 minutes 4 seconds:interval> -- !query 21 SELECT interval '1 2:03' hour to minute -- !query 21 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 21 output 1 days 2 hours 3 minutes @@ -181,7 +181,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 22 SELECT interval '1 2:03:04' hour to minute -- !query 22 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 22 output 1 days 2 hours 3 minutes @@ -189,7 +189,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 23 SELECT interval '1 2:03' hour to second -- !query 23 schema -struct<1 days 2 hours 3 minutes:interval> +struct -- !query 23 output 1 days 2 hours 3 minutes @@ -197,7 +197,7 @@ struct<1 days 2 hours 3 minutes:interval> -- !query 24 SELECT interval '1 2:03:04' hour to second -- !query 24 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct -- !query 24 output 1 days 2 hours 3 minutes 4 seconds @@ -205,7 +205,7 @@ struct<1 days 2 hours 3 minutes 4 seconds:interval> -- !query 25 SELECT interval '1 2:03' minute to second -- !query 25 schema -struct<1 days 2 minutes 3 seconds:interval> +struct -- !query 25 output 1 days 2 minutes 3 seconds @@ -213,7 +213,7 @@ struct<1 days 2 minutes 3 seconds:interval> -- !query 26 SELECT interval '1 2:03:04' minute to second -- !query 26 schema -struct<1 days 2 hours 3 minutes 4 seconds:interval> +struct -- !query 26 output 1 days 2 hours 3 minutes 4 seconds @@ -232,7 +232,7 @@ SELECT interval '1 day -1 hours', interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 28 schema -struct<1 days -1 hours:interval,-1 days 1 hours:interval,1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds:interval,-1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds:interval> +struct -- !query 28 output +1 -1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out index fbb4dd3b5efd1..9a11ef13cdf1f 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out @@ -64,6 +64,8 @@ create temporary view t3 as select * from values struct<> -- !query 2 output + + -- !query 3 create temporary view s1 as select * from values (1), (3), (5), (7), (9) @@ -73,6 +75,7 @@ struct<> -- !query 3 output + -- !query 4 create temporary view s2 as select * from values (1), (3), (4), (6), (9) @@ -82,6 +85,7 @@ struct<> -- !query 4 output + -- !query 5 create temporary view s3 as select * from values (3), (4), (6), (9) @@ -91,6 +95,7 @@ struct<> -- !query 5 output + -- !query 6 SELECT t1a, t1b, t1c, t3a, t3b, t3c FROM t1 natural JOIN t3 @@ -307,7 +312,7 @@ GROUP BY t1b HAVING t1b > 8 -- !query 14 schema struct --- !query 11 output +-- !query 14 output 1 10 @@ -378,7 +383,6 @@ val1b 8 16 1 8 16 val1b 8 16 1 NULL 16 - -- !query 17 SELECT s1.id FROM s1 JOIN s2 ON s1.id = s2.id @@ -548,6 +552,7 @@ NULL 4 NULL 6 NULL 9 + -- !query 30 SELECT s1.id, s2.id AS id2 FROM s1 FULL OUTER JOIN s2 @@ -573,12 +578,16 @@ DROP VIEW s1 struct<> -- !query 31 output + + -- !query 32 DROP VIEW s2 -- !query 32 schema struct<> -- !query 32 output + + -- !query 33 DROP VIEW s3 -- !query 33 schema diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out index c3d5fad0870bc..4e4f592a137fb 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out @@ -77,7 +77,7 @@ struct -- !query 9 SELECT array_join(array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), ', ') -- !query 9 schema -struct +struct -- !query 9 output 2016-11-15 20:54:00, 2016-11-12 20:54:00 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out index 462ad63aaaf2b..8608fa494d6cd 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out @@ -16,7 +16,7 @@ select cast(1 as tinyint) + interval 2 day struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) + 2 days)' (tinyint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS TINYINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) + INTERVAL '2 days')' (tinyint and interval).; line 1 pos 7 -- !query 2 @@ -25,7 +25,7 @@ select cast(1 as smallint) + interval 2 day struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) + 2 days)' (smallint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS SMALLINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) + INTERVAL '2 days')' (smallint and interval).; line 1 pos 7 -- !query 3 @@ -34,7 +34,7 @@ select cast(1 as int) + interval 2 day struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS INT) + 2 days)' (int and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS INT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS INT) + INTERVAL '2 days')' (int and interval).; line 1 pos 7 -- !query 4 @@ -43,7 +43,7 @@ select cast(1 as bigint) + interval 2 day struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) + 2 days)' (bigint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BIGINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) + INTERVAL '2 days')' (bigint and interval).; line 1 pos 7 -- !query 5 @@ -52,7 +52,7 @@ select cast(1 as float) + interval 2 day struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) + 2 days)' (float and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS FLOAT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) + INTERVAL '2 days')' (float and interval).; line 1 pos 7 -- !query 6 @@ -61,7 +61,7 @@ select cast(1 as double) + interval 2 day struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) + 2 days)' (double and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DOUBLE) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) + INTERVAL '2 days')' (double and interval).; line 1 pos 7 -- !query 7 @@ -70,13 +70,13 @@ select cast(1 as decimal(10, 0)) + interval 2 day struct<> -- !query 7 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + 2 days)' (decimal(10,0) and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days')' (decimal(10,0) and interval).; line 1 pos 7 -- !query 8 select cast('2017-12-11' as string) + interval 2 day -- !query 8 schema -struct +struct -- !query 8 output 2017-12-13 00:00:00 @@ -84,7 +84,7 @@ struct +struct -- !query 9 output 2017-12-13 09:30:00 @@ -95,7 +95,7 @@ select cast('1' as binary) + interval 2 day struct<> -- !query 10 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) + 2 days)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + 2 days)' (binary and interval).; line 1 pos 7 +cannot resolve '(CAST('1' AS BINARY) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + INTERVAL '2 days')' (binary and interval).; line 1 pos 7 -- !query 11 @@ -104,13 +104,13 @@ select cast(1 as boolean) + interval 2 day struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) + 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + 2 days)' (boolean and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BOOLEAN) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + INTERVAL '2 days')' (boolean and interval).; line 1 pos 7 -- !query 12 select cast('2017-12-11 09:30:00.0' as timestamp) + interval 2 day -- !query 12 schema -struct +struct -- !query 12 output 2017-12-13 09:30:00 @@ -118,7 +118,7 @@ struct +struct -- !query 13 output 2017-12-13 @@ -129,7 +129,7 @@ select interval 2 day + cast(1 as tinyint) struct<> -- !query 14 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS TINYINT))' (interval and tinyint).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS TINYINT))' (interval and tinyint).; line 1 pos 7 -- !query 15 @@ -138,7 +138,7 @@ select interval 2 day + cast(1 as smallint) struct<> -- !query 15 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS SMALLINT))' (interval and smallint).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS SMALLINT))' (interval and smallint).; line 1 pos 7 -- !query 16 @@ -147,7 +147,7 @@ select interval 2 day + cast(1 as int) struct<> -- !query 16 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS INT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS INT))' (interval and int).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS INT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS INT))' (interval and int).; line 1 pos 7 -- !query 17 @@ -156,7 +156,7 @@ select interval 2 day + cast(1 as bigint) struct<> -- !query 17 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS BIGINT))' (interval and bigint).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS BIGINT))' (interval and bigint).; line 1 pos 7 -- !query 18 @@ -165,7 +165,7 @@ select interval 2 day + cast(1 as float) struct<> -- !query 18 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(2 days + CAST(1 AS FLOAT))' (interval and float).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS FLOAT))' (interval and float).; line 1 pos 7 -- !query 19 @@ -174,7 +174,7 @@ select interval 2 day + cast(1 as double) struct<> -- !query 19 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(2 days + CAST(1 AS DOUBLE))' (interval and double).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS DOUBLE))' (interval and double).; line 1 pos 7 -- !query 20 @@ -183,13 +183,13 @@ select interval 2 day + cast(1 as decimal(10, 0)) struct<> -- !query 20 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(2 days + CAST(1 AS DECIMAL(10,0)))' (interval and decimal(10,0)).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS DECIMAL(10,0)))' (interval and decimal(10,0)).; line 1 pos 7 -- !query 21 select interval 2 day + cast('2017-12-11' as string) -- !query 21 schema -struct +struct -- !query 21 output 2017-12-13 00:00:00 @@ -197,7 +197,7 @@ struct +struct -- !query 22 output 2017-12-13 09:30:00 @@ -208,7 +208,7 @@ select interval 2 day + cast('1' as binary) struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(2 days + CAST('1' AS BINARY))' (interval and binary).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST('1' AS BINARY))' (interval and binary).; line 1 pos 7 -- !query 24 @@ -217,13 +217,13 @@ select interval 2 day + cast(1 as boolean) struct<> -- !query 24 output org.apache.spark.sql.AnalysisException -cannot resolve '(2 days + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(2 days + CAST(1 AS BOOLEAN))' (interval and boolean).; line 1 pos 7 +cannot resolve '(INTERVAL '2 days' + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS BOOLEAN))' (interval and boolean).; line 1 pos 7 -- !query 25 select interval 2 day + cast('2017-12-11 09:30:00.0' as timestamp) -- !query 25 schema -struct +struct -- !query 25 output 2017-12-13 09:30:00 @@ -231,7 +231,7 @@ struct +struct -- !query 26 output 2017-12-13 @@ -242,7 +242,7 @@ select cast(1 as tinyint) - interval 2 day struct<> -- !query 27 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) - 2 days)' (tinyint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS TINYINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) - INTERVAL '2 days')' (tinyint and interval).; line 1 pos 7 -- !query 28 @@ -251,7 +251,7 @@ select cast(1 as smallint) - interval 2 day struct<> -- !query 28 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) - 2 days)' (smallint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS SMALLINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) - INTERVAL '2 days')' (smallint and interval).; line 1 pos 7 -- !query 29 @@ -260,7 +260,7 @@ select cast(1 as int) - interval 2 day struct<> -- !query 29 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS INT) - 2 days)' (int and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS INT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS INT) - INTERVAL '2 days')' (int and interval).; line 1 pos 7 -- !query 30 @@ -269,7 +269,7 @@ select cast(1 as bigint) - interval 2 day struct<> -- !query 30 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) - 2 days)' (bigint and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BIGINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) - INTERVAL '2 days')' (bigint and interval).; line 1 pos 7 -- !query 31 @@ -278,7 +278,7 @@ select cast(1 as float) - interval 2 day struct<> -- !query 31 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) - 2 days)' (float and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS FLOAT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) - INTERVAL '2 days')' (float and interval).; line 1 pos 7 -- !query 32 @@ -287,7 +287,7 @@ select cast(1 as double) - interval 2 day struct<> -- !query 32 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) - 2 days)' (double and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DOUBLE) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) - INTERVAL '2 days')' (double and interval).; line 1 pos 7 -- !query 33 @@ -296,13 +296,13 @@ select cast(1 as decimal(10, 0)) - interval 2 day struct<> -- !query 33 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - 2 days)' (decimal(10,0) and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days')' (decimal(10,0) and interval).; line 1 pos 7 -- !query 34 select cast('2017-12-11' as string) - interval 2 day -- !query 34 schema -struct +struct -- !query 34 output 2017-12-09 00:00:00 @@ -310,7 +310,7 @@ struct +struct -- !query 35 output 2017-12-09 09:30:00 @@ -321,7 +321,7 @@ select cast('1' as binary) - interval 2 day struct<> -- !query 36 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) - 2 days)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - 2 days)' (binary and interval).; line 1 pos 7 +cannot resolve '(CAST('1' AS BINARY) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - INTERVAL '2 days')' (binary and interval).; line 1 pos 7 -- !query 37 @@ -330,13 +330,13 @@ select cast(1 as boolean) - interval 2 day struct<> -- !query 37 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) - 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - 2 days)' (boolean and interval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BOOLEAN) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - INTERVAL '2 days')' (boolean and interval).; line 1 pos 7 -- !query 38 select cast('2017-12-11 09:30:00.0' as timestamp) - interval 2 day -- !query 38 schema -struct +struct -- !query 38 output 2017-12-09 09:30:00 @@ -344,6 +344,6 @@ struct +struct -- !query 39 output 2017-12-09 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index b5ce121d2bc6a..7fed9e147608b 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -154,7 +154,7 @@ SELECT val_timestamp, udf(cate), avg(val_timestamp) OVER(PARTITION BY udf(cate) RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData ORDER BY udf(cate), val_timestamp -- !query 9 schema -struct +struct -- !query 9 output NULL NULL NULL 2017-07-31 17:00:00 NULL 1.5015456E9 diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 1698ac081be95..f3b9c252b8330 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -154,7 +154,7 @@ SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY v RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData ORDER BY cate, val_timestamp -- !query 9 schema -struct +struct -- !query 9 output NULL NULL NULL 2017-07-31 17:00:00 NULL 1.5015456E9 From ffc97530371433bc0221e06d8c1d11af8d92bd94 Mon Sep 17 00:00:00 2001 From: "wangguangxin.cn" Date: Tue, 19 Nov 2019 16:10:22 +0800 Subject: [PATCH 0662/1387] [SPARK-29918][SQL] RecordBinaryComparator should check endianness when compared by long MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR try to make sure the comparison results of `compared by 8 bytes at a time` and `compared by bytes wise` in RecordBinaryComparator is *consistent*, by reverse long bytes if it is little-endian and using Long.compareUnsigned. ### Why are the changes needed? If the architecture supports unaligned or the offset is 8 bytes aligned, `RecordBinaryComparator` compare 8 bytes at a time by reading 8 bytes as a long. Related code is ``` if (Platform.unaligned() || (((leftOff + i) % 8 == 0) && ((rightOff + i) % 8 == 0))) { while (i <= leftLen - 8) { final long v1 = Platform.getLong(leftObj, leftOff + i); final long v2 = Platform.getLong(rightObj, rightOff + i); if (v1 != v2) { return v1 > v2 ? 1 : -1; } i += 8; } } ``` Otherwise, it will compare bytes by bytes.  Related code is ``` while (i < leftLen) { final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff; final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff; if (v1 != v2) { return v1 > v2 ? 1 : -1; } i += 1; } ``` However, on little-endian machine,  the result of *compared by a long value* and *compared bytes by bytes* maybe different. For two same records, its offsets may vary in the first run and second run, which will lead to compare them using long comparison or byte-by-byte comparison, the result maybe different. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add new test cases in RecordBinaryComparatorSuite Closes #26548 from WangGuangxin/binary_comparator. Authored-by: wangguangxin.cn Signed-off-by: Wenchen Fan --- .../sql/execution/RecordBinaryComparator.java | 30 ++++++++---- .../sort/RecordBinaryComparatorSuite.java | 47 ++++++++++++++++++- 2 files changed, 67 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java b/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java index 40c2cc806e87a..1f243406c77e0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java @@ -20,8 +20,13 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.util.collection.unsafe.sort.RecordComparator; +import java.nio.ByteOrder; + public final class RecordBinaryComparator extends RecordComparator { + private static final boolean LITTLE_ENDIAN = + ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN); + @Override public int compare( Object leftObj, long leftOff, int leftLen, Object rightObj, long rightOff, int rightLen) { @@ -38,10 +43,10 @@ public int compare( // check if stars align and we can get both offsets to be aligned if ((leftOff % 8) == (rightOff % 8)) { while ((leftOff + i) % 8 != 0 && i < leftLen) { - final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff; - final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff; + final int v1 = Platform.getByte(leftObj, leftOff + i); + final int v2 = Platform.getByte(rightObj, rightOff + i); if (v1 != v2) { - return v1 > v2 ? 1 : -1; + return (v1 & 0xff) > (v2 & 0xff) ? 1 : -1; } i += 1; } @@ -49,10 +54,17 @@ public int compare( // for architectures that support unaligned accesses, chew it up 8 bytes at a time if (Platform.unaligned() || (((leftOff + i) % 8 == 0) && ((rightOff + i) % 8 == 0))) { while (i <= leftLen - 8) { - final long v1 = Platform.getLong(leftObj, leftOff + i); - final long v2 = Platform.getLong(rightObj, rightOff + i); + long v1 = Platform.getLong(leftObj, leftOff + i); + long v2 = Platform.getLong(rightObj, rightOff + i); if (v1 != v2) { - return v1 > v2 ? 1 : -1; + if (LITTLE_ENDIAN) { + // if read as little-endian, we have to reverse bytes so that the long comparison result + // is equivalent to byte-by-byte comparison result. + // See discussion in https://github.com/apache/spark/pull/26548#issuecomment-554645859 + v1 = Long.reverseBytes(v1); + v2 = Long.reverseBytes(v2); + } + return Long.compareUnsigned(v1, v2); } i += 8; } @@ -60,10 +72,10 @@ public int compare( // this will finish off the unaligned comparisons, or do the entire aligned comparison // whichever is needed. while (i < leftLen) { - final int v1 = Platform.getByte(leftObj, leftOff + i) & 0xff; - final int v2 = Platform.getByte(rightObj, rightOff + i) & 0xff; + final int v1 = Platform.getByte(leftObj, leftOff + i); + final int v2 = Platform.getByte(rightObj, rightOff + i); if (v1 != v2) { - return v1 > v2 ? 1 : -1; + return (v1 & 0xff) > (v2 & 0xff) ? 1 : -1; } i += 1; } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java index 92dabc79d2bff..68f984ae0c1e3 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java @@ -33,6 +33,7 @@ import org.apache.spark.util.collection.unsafe.sort.*; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -273,7 +274,7 @@ public void testBinaryComparatorWhenSubtractionIsDivisibleByMaxIntValue() throws insertRow(row1); insertRow(row2); - assert(compare(0, 1) < 0); + assert(compare(0, 1) > 0); } @Test @@ -321,4 +322,48 @@ public void testBinaryComparatorWhenOnlyTheLastColumnDiffers() throws Exception assert(compare(0, 1) < 0); } + + @Test + public void testCompareLongsAsLittleEndian() { + long arrayOffset = 12; + + long[] arr1 = new long[2]; + Platform.putLong(arr1, arrayOffset, 0x0100000000000000L); + long[] arr2 = new long[2]; + Platform.putLong(arr2, arrayOffset + 4, 0x0000000000000001L); + // leftBaseOffset is not aligned while rightBaseOffset is aligned, + // it will start by comparing long + int result1 = binaryComparator.compare(arr1, arrayOffset, 8, arr2, arrayOffset + 4, 8); + + long[] arr3 = new long[2]; + Platform.putLong(arr3, arrayOffset, 0x0100000000000000L); + long[] arr4 = new long[2]; + Platform.putLong(arr4, arrayOffset, 0x0000000000000001L); + // both left and right offset is not aligned, it will start with byte-by-byte comparison + int result2 = binaryComparator.compare(arr3, arrayOffset, 8, arr4, arrayOffset, 8); + + Assert.assertEquals(result1, result2); + } + + @Test + public void testCompareLongsAsUnsigned() { + long arrayOffset = 12; + + long[] arr1 = new long[2]; + Platform.putLong(arr1, arrayOffset + 4, 0xa000000000000000L); + long[] arr2 = new long[2]; + Platform.putLong(arr2, arrayOffset + 4, 0x0000000000000000L); + // both leftBaseOffset and rightBaseOffset are aligned, so it will start by comparing long + int result1 = binaryComparator.compare(arr1, arrayOffset + 4, 8, arr2, arrayOffset + 4, 8); + + long[] arr3 = new long[2]; + Platform.putLong(arr3, arrayOffset, 0xa000000000000000L); + long[] arr4 = new long[2]; + Platform.putLong(arr4, arrayOffset, 0x0000000000000000L); + // both leftBaseOffset and rightBaseOffset are not aligned, + // so it will start with byte-by-byte comparison + int result2 = binaryComparator.compare(arr3, arrayOffset, 8, arr4, arrayOffset, 8); + + Assert.assertEquals(result1, result2); + } } From a8d98833b88e7a366734c18acec36236357a41e4 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 19 Nov 2019 19:18:08 +0800 Subject: [PATCH 0663/1387] [SPARK-29893] improve the local shuffle reader performance by changing the reading task number from 1 to multi ### What changes were proposed in this pull request? This PR update the local reader task number from 1 to multi `partitionStartIndices.length`. ### Why are the changes needed? Improve the performance of local shuffle reader. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing UTs Closes #26516 from JkSelf/improveLocalShuffleReader. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../org/apache/spark/MapOutputTracker.scala | 3 +- .../adaptive/AdaptiveSparkPlanExec.scala | 13 +- .../adaptive/LocalShuffledRowRDD.scala | 52 ++++++-- .../adaptive/OptimizeLocalShuffleReader.scala | 114 ++++++++++-------- .../exchange/ShuffleExchangeExec.scala | 5 +- .../adaptive/AdaptiveQueryExecSuite.scala | 61 ++++++++++ 6 files changed, 170 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 873efa76468ed..3c6c181f9428c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -701,8 +701,7 @@ private[spark] class MapOutputTrackerMaster( if (shuffleStatus != null) { shuffleStatus.withMapStatuses { statuses => if (mapId >= 0 && mapId < statuses.length) { - Seq( ExecutorCacheTaskLocation(statuses(mapId).location.host, - statuses(mapId).location.executorId).toString) + Seq(statuses(mapId).location.host) } else { Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 8d4731f34ddd6..02d8f185a7825 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -82,7 +82,6 @@ case class AdaptiveSparkPlanExec( // plan should reach a final status of query stages (i.e., no more addition or removal of // Exchange nodes) after running these rules. private def queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( - OptimizeLocalShuffleReader(conf), ensureRequirements ) @@ -90,16 +89,10 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReuseAdaptiveSubquery(conf, subqueryCache), - - // When adding local shuffle readers in 'OptimizeLocalShuffleReader`, we revert all the local - // readers if additional shuffles are introduced. This may be too conservative: maybe there is - // only one local reader that introduces shuffle, and we can still keep other local readers. - // Here we re-execute this rule with the sub-plan-tree of a query stage, to make sure necessary - // local readers are added before executing the query stage. - // This rule must be executed before `ReduceNumShufflePartitions`, as local shuffle readers - // can't change number of partitions. - OptimizeLocalShuffleReader(conf), ReduceNumShufflePartitions(conf), + // The rule of 'OptimizeLocalShuffleReader' need to make use of the 'partitionStartIndices' + // in 'ReduceNumShufflePartitions' rule. So it must be after 'ReduceNumShufflePartitions' rule. + OptimizeLocalShuffleReader(conf), ApplyColumnarRulesAndInsertTransitions(session.sessionState.conf, session.sessionState.columnarRules), CollapseCodegenStages(conf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala index 5fccb5ce65783..44254593f8895 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -17,20 +17,24 @@ package org.apache.spark.sql.execution.adaptive +import scala.collection.mutable.ArrayBuffer + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} - /** - * The [[Partition]] used by [[LocalShuffledRowRDD]]. A pre-shuffle partition - * (identified by `preShufflePartitionIndex`) contains a range of post-shuffle partitions - * (`startPostShufflePartitionIndex` to `endPostShufflePartitionIndex - 1`, inclusive). + * The [[Partition]] used by [[LocalShuffledRowRDD]]. + * @param mapIndex the index of mapper. + * @param startPartition the start partition ID in mapIndex mapper. + * @param endPartition the end partition ID in mapIndex mapper. */ private final class LocalShuffledRowRDDPartition( - val preShufflePartitionIndex: Int) extends Partition { - override val index: Int = preShufflePartitionIndex + override val index: Int, + val mapIndex: Int, + val startPartition: Int, + val endPartition: Int) extends Partition { } /** @@ -52,7 +56,8 @@ private final class LocalShuffledRowRDDPartition( */ class LocalShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], - metrics: Map[String, SQLMetric]) + metrics: Map[String, SQLMetric], + advisoryParallelism : Option[Int] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { private[this] val numReducers = dependency.partitioner.numPartitions @@ -60,11 +65,33 @@ class LocalShuffledRowRDD( override def getDependencies: Seq[Dependency[_]] = List(dependency) + /** + * To equally divide n elements into m buckets, basically each bucket should have n/m elements, + * for the remaining n%m elements, add one more element to the first n%m buckets each. Returns + * a sequence with length numBuckets and each value represents the start index of each bucket. + */ + private def equallyDivide(numElements: Int, numBuckets: Int): Seq[Int] = { + val elementsPerBucket = numElements / numBuckets + val remaining = numElements % numBuckets + val splitPoint = (elementsPerBucket + 1) * remaining + (0 until remaining).map(_ * (elementsPerBucket + 1)) ++ + (remaining until numBuckets).map(i => splitPoint + (i - remaining) * elementsPerBucket) + } + override def getPartitions: Array[Partition] = { + val partitionStartIndices: Array[Int] = { + val expectedParallelism = advisoryParallelism.getOrElse(numReducers) + // TODO split by data size in the future. + equallyDivide(numReducers, math.max(1, expectedParallelism / numMappers)).toArray + } - Array.tabulate[Partition](numMappers) { i => - new LocalShuffledRowRDDPartition(i) + val partitions = ArrayBuffer[LocalShuffledRowRDDPartition]() + for (mapIndex <- 0 until numMappers) { + (partitionStartIndices :+ numReducers).sliding(2, 1).foreach { case Array(start, end) => + partitions += new LocalShuffledRowRDDPartition(partitions.length, mapIndex, start, end) + } } + partitions.toArray } override def getPreferredLocations(partition: Partition): Seq[String] = { @@ -74,17 +101,16 @@ class LocalShuffledRowRDD( override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val localRowPartition = split.asInstanceOf[LocalShuffledRowRDDPartition] - val mapIndex = localRowPartition.index + val mapIndex = localRowPartition.mapIndex val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, // as well as the `tempMetrics` for basic shuffle metrics. val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) - val reader = SparkEnv.get.shuffleManager.getReaderForOneMapper( dependency.shuffleHandle, mapIndex, - 0, - numReducers, + localRowPartition.startPartition, + localRowPartition.endPartition, context, sqlMetricsReporter) reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 87d745bf976ab..a57cfae6d0991 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -27,49 +27,47 @@ import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExcha import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.internal.SQLConf -object BroadcastJoinWithShuffleLeft { - def unapply(plan: SparkPlan): Option[(QueryStageExec, BuildSide)] = plan match { - case join: BroadcastHashJoinExec if ShuffleQueryStageExec.isShuffleQueryStageExec(join.left) => - Some((join.left.asInstanceOf[QueryStageExec], join.buildSide)) - case _ => None - } -} - -object BroadcastJoinWithShuffleRight { - def unapply(plan: SparkPlan): Option[(QueryStageExec, BuildSide)] = plan match { - case join: BroadcastHashJoinExec if ShuffleQueryStageExec.isShuffleQueryStageExec(join.right) => - Some((join.right.asInstanceOf[QueryStageExec], join.buildSide)) - case _ => None - } -} - /** - * A rule to optimize the shuffle reader to local reader as far as possible - * when converting the 'SortMergeJoinExec' to 'BroadcastHashJoinExec' in runtime. - * - * This rule can be divided into two steps: - * Step1: Add the local reader in probe side and then check whether additional - * shuffle introduced. If introduced, we will revert all the local - * reader in probe side. - * Step2: Add the local reader in build side and will not check whether - * additional shuffle introduced. Because the build side will not introduce - * additional shuffle. + * A rule to optimize the shuffle reader to local reader iff no additional shuffles + * will be introduced: + * 1. if the input plan is a shuffle, add local reader directly as we can never introduce + * extra shuffles in this case. + * 2. otherwise, add local reader to the probe side of broadcast hash join and + * then run `EnsureRequirements` to check whether additional shuffle introduced. + * If introduced, we will revert all the local readers. */ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { + import OptimizeLocalShuffleReader._ - override def apply(plan: SparkPlan): SparkPlan = { - if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { - return plan - } - // Add local reader in probe side. - val withProbeSideLocalReader = plan.transformDown { + def withProbeSideLocalReader(plan: SparkPlan): SparkPlan = { + plan.transformDown { case join @ BroadcastJoinWithShuffleLeft(shuffleStage, BuildRight) => - val localReader = LocalShuffleReaderExec(shuffleStage) + val localReader = createLocalReader(shuffleStage) join.asInstanceOf[BroadcastHashJoinExec].copy(left = localReader) case join @ BroadcastJoinWithShuffleRight(shuffleStage, BuildLeft) => - val localReader = LocalShuffleReaderExec(shuffleStage) + val localReader = createLocalReader(shuffleStage) join.asInstanceOf[BroadcastHashJoinExec].copy(right = localReader) } + } + + def createLocalReader(plan: SparkPlan): LocalShuffleReaderExec = { + plan match { + case c: CoalescedShuffleReaderExec => + LocalShuffleReaderExec(c.child, Some(c.partitionStartIndices.length)) + case q: QueryStageExec => LocalShuffleReaderExec(q) + } + } + + override def apply(plan: SparkPlan): SparkPlan = { + if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { + return plan + } + + val optimizedPlan = plan match { + case s: SparkPlan if canUseLocalShuffleReader(s) => + createLocalReader(s) + case s: SparkPlan => withProbeSideLocalReader(s) + } def numExchanges(plan: SparkPlan): Int = { plan.collect { @@ -77,26 +75,38 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { }.length } // Check whether additional shuffle introduced. If introduced, revert the local reader. - val numExchangeBefore = numExchanges(EnsureRequirements(conf).apply(plan)) - val numExchangeAfter = numExchanges(EnsureRequirements(conf).apply(withProbeSideLocalReader)) - val optimizedPlan = if (numExchangeAfter > numExchangeBefore) { - logDebug("OptimizeLocalShuffleReader rule is not applied in the probe side due" + + if (numExchanges(EnsureRequirements(conf).apply(optimizedPlan)) > 0) { + logDebug("OptimizeLocalShuffleReader rule is not applied due" + " to additional shuffles will be introduced.") plan } else { - withProbeSideLocalReader + optimizedPlan } - // Add the local reader in build side and and do not need to check whether - // additional shuffle introduced. - optimizedPlan.transformDown { - case join @ BroadcastJoinWithShuffleLeft(shuffleStage, BuildLeft) => - val localReader = LocalShuffleReaderExec(shuffleStage) - join.asInstanceOf[BroadcastHashJoinExec].copy(left = localReader) - case join @ BroadcastJoinWithShuffleRight(shuffleStage, BuildRight) => - val localReader = LocalShuffleReaderExec(shuffleStage) - join.asInstanceOf[BroadcastHashJoinExec].copy(right = localReader) + } +} + +object OptimizeLocalShuffleReader { + + object BroadcastJoinWithShuffleLeft { + def unapply(plan: SparkPlan): Option[(SparkPlan, BuildSide)] = plan match { + case join: BroadcastHashJoinExec if canUseLocalShuffleReader(join.left) => + Some((join.left, join.buildSide)) + case _ => None + } + } + + object BroadcastJoinWithShuffleRight { + def unapply(plan: SparkPlan): Option[(SparkPlan, BuildSide)] = plan match { + case join: BroadcastHashJoinExec if canUseLocalShuffleReader(join.right) => + Some((join.right, join.buildSide)) + case _ => None } } + + def canUseLocalShuffleReader(plan: SparkPlan): Boolean = { + ShuffleQueryStageExec.isShuffleQueryStageExec(plan) || + plan.isInstanceOf[CoalescedShuffleReaderExec] + } } /** @@ -107,7 +117,9 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the * shuffle exchange node during canonicalization. */ -case class LocalShuffleReaderExec(child: SparkPlan) extends UnaryExecNode { +case class LocalShuffleReaderExec( + child: SparkPlan, + advisoryParallelism: Option[Int] = None) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -124,9 +136,9 @@ case class LocalShuffleReaderExec(child: SparkPlan) extends UnaryExecNode { if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createLocalShuffleRDD() + stage.plan.createLocalShuffleRDD(advisoryParallelism) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createLocalShuffleRDD() + stage.plan.createLocalShuffleRDD(advisoryParallelism) } } cachedShuffleRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 2f94c522712b1..baca974a6450b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -83,8 +83,9 @@ case class ShuffleExchangeExec( new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) } - def createLocalShuffleRDD(): LocalShuffledRowRDD = { - new LocalShuffledRowRDD(shuffleDependency, readMetrics) + def createLocalShuffleRDD( + advisoryParallelism: Option[Int] = None): LocalShuffledRowRDD = { + new LocalShuffledRowRDD(shuffleDependency, readMetrics, advisoryParallelism) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index f6b6b23b29140..84bd99a2a8635 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -103,6 +103,67 @@ class AdaptiveQueryExecSuite } } + test("Reuse the parallelism of CoalescedShuffleReaderExec in LocalShuffleReaderExec") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "10") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + val localReaders = collect(adaptivePlan) { + case reader: LocalShuffleReaderExec => reader + } + assert(localReaders.length == 2) + // The pre-shuffle partition size is [0, 0, 0, 72, 0] + // And the partitionStartIndices is [0, 3, 4] + assert(localReaders(0).advisoryParallelism.get == 3) + // The pre-shuffle partition size is [0, 72, 0, 72, 126] + // And the partitionStartIndices is [0, 1, 2, 3, 4] + assert(localReaders(1).advisoryParallelism.get == 5) + + val localShuffleRDD0 = localReaders(0).execute().asInstanceOf[LocalShuffledRowRDD] + val localShuffleRDD1 = localReaders(1).execute().asInstanceOf[LocalShuffledRowRDD] + // the final parallelism is + // math.max(1, advisoryParallelism / numMappers): math.max(1, 3/2) = 1 + // and the partitions length is 1 * numMappers = 2 + assert(localShuffleRDD0.getPartitions.length == 2) + // the final parallelism is + // math.max(1, advisoryParallelism / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + + test("Reuse the default parallelism in LocalShuffleReaderExec") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", + SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + val localReaders = collect(adaptivePlan) { + case reader: LocalShuffleReaderExec => reader + } + assert(localReaders.length == 2) + val localShuffleRDD0 = localReaders(0).execute().asInstanceOf[LocalShuffledRowRDD] + val localShuffleRDD1 = localReaders(1).execute().asInstanceOf[LocalShuffledRowRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD0.getPartitions.length == 4) + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + test("Scalar subquery") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", From 79ed4ae2dbdbc73c4225fbabeac5a1ddb10c65d1 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 19 Nov 2019 21:01:26 +0800 Subject: [PATCH 0664/1387] [SPARK-29926][SQL] Fix weird interval string whose value is only a dangling decimal point ### What changes were proposed in this pull request? Currently, we support to parse '1. second' to 1s or even '. second' to 0s. ```sql -- !query 118 select interval '1. seconds' -- !query 118 schema struct<1 seconds:interval> -- !query 118 output 1 seconds -- !query 119 select interval '. seconds' -- !query 119 schema struct<0 seconds:interval> -- !query 119 output 0 seconds ``` ```sql postgres=# select interval '1. second'; ERROR: invalid input syntax for type interval: "1. second" LINE 1: select interval '1. second'; postgres=# select interval '. second'; ERROR: invalid input syntax for type interval: ". second" LINE 1: select interval '. second'; ``` We fix this by fixing the new interval parser's VALUE_FRACTIONAL_PART state With further digging, we found that 1. is valid in python, r, scala, and presto and so on... so this PR ONLY forbid the invalid interval value in the form of '. seconds'. ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? yes, now we treat '. second' .... as invalid intervals ### How was this patch tested? add ut Closes #26573 from yaooqinn/SPARK-29926. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/util/IntervalUtils.scala | 10 +++++++--- .../spark/sql/catalyst/util/IntervalUtilsSuite.scala | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 61bd9242f3e24..9418d8eec3376 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -479,7 +479,9 @@ object IntervalUtils { var days: Int = 0 var microseconds: Long = 0 var fractionScale: Int = 0 + val initialFractionScale = (NANOS_PER_SECOND / 10).toInt var fraction: Int = 0 + var pointPrefixed: Boolean = false def trimToNextState(b: Byte, next: ParseState): Unit = { b match { @@ -519,6 +521,7 @@ object IntervalUtils { // We preset the scale to an invalid value to track fraction presence in the UNIT_BEGIN // state. If we meet '.', the scale become valid for the VALUE_FRACTIONAL_PART state. fractionScale = -1 + pointPrefixed = false b match { case '-' => isNegative = true @@ -530,7 +533,8 @@ object IntervalUtils { isNegative = false case '.' => isNegative = false - fractionScale = (NANOS_PER_SECOND / 10).toInt + fractionScale = initialFractionScale + pointPrefixed = true i += 1 state = VALUE_FRACTIONAL_PART case _ => throwIAE( s"unrecognized number '$currentWord'") @@ -546,7 +550,7 @@ object IntervalUtils { } case ' ' => state = TRIM_BEFORE_UNIT case '.' => - fractionScale = (NANOS_PER_SECOND / 10).toInt + fractionScale = initialFractionScale state = VALUE_FRACTIONAL_PART case _ => throwIAE(s"invalid value '$currentWord'") } @@ -556,7 +560,7 @@ object IntervalUtils { case _ if '0' <= b && b <= '9' && fractionScale > 0 => fraction += (b - '0') * fractionScale fractionScale /= 10 - case ' ' => + case ' ' if !pointPrefixed || fractionScale < initialFractionScale => fraction /= NANOS_PER_MICROS.toInt state = TRIM_BEFORE_UNIT case _ if '0' <= b && b <= '9' => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index 73a2adbaec1db..ee3db0391ed00 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -115,7 +115,7 @@ class IntervalUtilsSuite extends SparkFunSuite { checkFromInvalidString("2234567890 days", "integer overflow") checkFromInvalidString("\n", "Error parsing '\n' to interval") checkFromInvalidString("\t", "Error parsing '\t' to interval") - + checkFromInvalidString(". seconds", "invalid value '.'") } test("string to interval: seconds with fractional part") { From 6fb8b8606544f26dc2d9719a2d009eb5aea65ba2 Mon Sep 17 00:00:00 2001 From: Jobit Mathew Date: Tue, 19 Nov 2019 21:30:38 +0800 Subject: [PATCH 0665/1387] [SPARK-29913][SQL] Improve Exception in postgreCastToBoolean ### What changes were proposed in this pull request? Exception improvement. ### Why are the changes needed? After selecting pgSQL dialect, queries which are failing because of wrong syntax will give long exception stack trace. For example, `explain select cast ("abc" as boolean);` Current output: > ERROR SparkSQLDriver: Failed in [explain select cast ("abc" as boolean)] > java.lang.IllegalArgumentException: invalid input syntax for type boolean: abc > at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$2(PostgreCastToBoolean.scala:51) > at org.apache.spark.sql.catalyst.expressions.CastBase.buildCast(Cast.scala:277) > at org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean.$anonfun$castToBoolean$1(PostgreCastToBoolean.scala:44) > at org.apache.spark.sql.catalyst.expressions.CastBase.nullSafeEval(Cast.scala:773) > at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:460) > at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:52) > at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:45) > at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$1(TreeNode.scala:286) > at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:72) > at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:286) > at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDown$3(TreeNode.scala:291) > at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$mapChildren$1(TreeNode.scala:376) > at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:214) > at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:374) > at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:327) > at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291) > at org.apache.spark.sql.catalyst.plans.QueryPlan. > . > . > . ### Does this PR introduce any user-facing change? Yes. After this PR, output for above query will be: > == Physical Plan == > org.apache.spark.sql.AnalysisException: invalid input syntax for type boolean: abc; > > Time taken: 0.044 seconds, Fetched 1 row(s) > 19/11/15 15:38:57 INFO SparkSQLCLIDriver: Time taken: 0.044 seconds, Fetched 1 row(s) ### How was this patch tested? Updated existing test cases. Closes #26546 from jobitmathew/pgsqlexception. Authored-by: Jobit Mathew Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/execution/QueryExecution.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index f9394473d06e0..258f9cea05b82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -141,6 +141,7 @@ class QueryExecution( ExplainUtils.processPlan(executedPlan, concat.append) } catch { case e: AnalysisException => concat.append(e.toString) + case e: IllegalArgumentException => concat.append(e.toString) } } else { QueryPlan.append(executedPlan, concat.append, verbose = false, addSuffix = false) From 3d2a6f464f98c04f011509733a15972fd2b6701d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Nov 2019 10:39:38 -0800 Subject: [PATCH 0666/1387] [SPARK-29906][SQL] AQE should not introduce extra shuffle for outermost limit ### What changes were proposed in this pull request? `AdaptiveSparkPlanExec` should forward `executeCollect` and `executeTake` to the underlying physical plan. ### Why are the changes needed? some physical plan has optimization in `executeCollect` and `executeTake`. For example, `CollectLimitExec` won't do shuffle for outermost limit. ### Does this PR introduce any user-facing change? no ### How was this patch tested? a new test This closes #26560 Closes #26576 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: Xiao Li --- .../adaptive/AdaptiveSparkPlanExec.scala | 23 ++++++++++++------- .../adaptive/AdaptiveQueryExecSuite.scala | 21 +++++++++++++++++ 2 files changed, 36 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 02d8f185a7825..b4eea620b93a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -126,10 +126,8 @@ case class AdaptiveSparkPlanExec( override def doCanonicalize(): SparkPlan = initialPlan.canonicalized - override def doExecute(): RDD[InternalRow] = lock.synchronized { - if (isFinalPlan) { - currentPhysicalPlan.execute() - } else { + private def getFinalPhysicalPlan(): SparkPlan = lock.synchronized { + if (!isFinalPlan) { // Make sure we only update Spark UI if this plan's `QueryExecution` object matches the one // retrieved by the `sparkContext`'s current execution ID. Note that sub-queries do not have // their own execution IDs and therefore rely on the main query to update UI. @@ -210,12 +208,21 @@ case class AdaptiveSparkPlanExec( // Run the final plan when there's no more unfinished stages. currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) isFinalPlan = true - - val ret = currentPhysicalPlan.execute() logDebug(s"Final plan: $currentPhysicalPlan") - executionId.foreach(onUpdatePlan) - ret } + currentPhysicalPlan + } + + override def executeCollect(): Array[InternalRow] = { + getFinalPhysicalPlan().executeCollect() + } + + override def executeTake(n: Int): Array[InternalRow] = { + getFinalPhysicalPlan().executeTake(n) + } + + override def doExecute(): RDD[InternalRow] = { + getFinalPhysicalPlan().execute() } override def verboseString(maxFields: Int): String = simpleString(maxFields) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 84bd99a2a8635..d5be45b0cdda0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.adaptive +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} import org.apache.spark.sql.execution.exchange.Exchange @@ -534,4 +535,24 @@ class AdaptiveQueryExecSuite } } } + + test("SPARK-29906: AQE should not introduce extra shuffle for outermost limit") { + var numStages = 0 + val listener = new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + numStages = jobStart.stageInfos.length + } + } + try { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + spark.sparkContext.addSparkListener(listener) + spark.range(0, 100, 1, numPartitions = 10).take(1) + spark.sparkContext.listenerBus.waitUntilEmpty() + // Should be only one stage since there is no shuffle. + assert(numStages == 1) + } + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } } From e804ed5e330c7dc6cd46812b520dd6b610a584b6 Mon Sep 17 00:00:00 2001 From: John Bauer Date: Tue, 19 Nov 2019 14:15:00 -0800 Subject: [PATCH 0667/1387] [SPARK-29691][ML][PYTHON] ensure Param objects are valid in fit, transform modify Param._copyValues to check valid Param objects supplied as extra ### What changes were proposed in this pull request? Estimator.fit() and Model.transform() accept a dictionary of extra parameters whose values are used to overwrite those supplied at initialization or by default. Additionally, the ParamGridBuilder.addGrid accepts a parameter and list of values. The keys are presumed to be valid Param objects. This change adds a check that only Param objects are supplied as keys. ### Why are the changes needed? Param objects are created by and bound to an instance of Params (Estimator, Model, or Transformer). They may be obtained from their parent as attributes, or by name through getParam. The documentation does not state that keys must be valid Param objects, nor describe how one may be obtained. The current behavior is to silently ignore keys which are not valid Param objects. ### Does this PR introduce any user-facing change? If the user does not pass in a Param object as required for keys in `extra` for Estimator.fit() and Model.transform(), and `param` for ParamGridBuilder.addGrid, an error will be raised indicating it is an invalid object. ### How was this patch tested? Added method test_copy_param_extras_check to test_param.py. Tested with Python 3.7 Closes #26527 from JohnHBauer/paramExtra. Authored-by: John Bauer Signed-off-by: Bryan Cutler --- python/pyspark/ml/param/__init__.py | 12 ++++++++++-- python/pyspark/ml/tests/test_param.py | 4 ++++ python/pyspark/ml/tests/test_tuning.py | 9 +++++++++ python/pyspark/ml/tuning.py | 8 +++++++- 4 files changed, 30 insertions(+), 3 deletions(-) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 57ad1e6dfb3e6..fe61f9f0fffd6 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -484,8 +484,16 @@ def _copyValues(self, to, extra=None): :return: the target instance with param values copied """ paramMap = self._paramMap.copy() - if extra is not None: - paramMap.update(extra) + if isinstance(extra, dict): + for param, value in extra.items(): + if isinstance(param, Param): + paramMap[param] = value + else: + raise TypeError("Expecting a valid instance of Param, but received: {}" + .format(param)) + elif extra is not None: + raise TypeError("Expecting a dict, but received an object of type {}." + .format(type(extra))) for param in self.params: # copy default params if param in self._defaultParamMap and to.hasParam(param.name): diff --git a/python/pyspark/ml/tests/test_param.py b/python/pyspark/ml/tests/test_param.py index 75cd903b5d6d7..777b4930ce8c9 100644 --- a/python/pyspark/ml/tests/test_param.py +++ b/python/pyspark/ml/tests/test_param.py @@ -307,6 +307,10 @@ def test_copy_param_extras(self): copied_no_extra[k] = v self.assertEqual(tp._paramMap, copied_no_extra) self.assertEqual(tp._defaultParamMap, tp_copy._defaultParamMap) + with self.assertRaises(TypeError): + tp.copy(extra={"unknown_parameter": None}) + with self.assertRaises(TypeError): + tp.copy(extra=["must be a dict"]) def test_logistic_regression_check_thresholds(self): self.assertIsInstance( diff --git a/python/pyspark/ml/tests/test_tuning.py b/python/pyspark/ml/tests/test_tuning.py index 176e99d052d30..9d8ba37c60da4 100644 --- a/python/pyspark/ml/tests/test_tuning.py +++ b/python/pyspark/ml/tests/test_tuning.py @@ -63,6 +63,15 @@ def _fit(self, dataset): return model +class ParamGridBuilderTests(SparkSessionTestCase): + + def test_addGrid(self): + with self.assertRaises(TypeError): + grid = (ParamGridBuilder() + .addGrid("must be an instance of Param", ["not", "string"]) + .build()) + + class CrossValidatorTests(SparkSessionTestCase): def test_copy(self): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 16c376296c20d..5eb8ae44d3d66 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -88,8 +88,14 @@ def __init__(self): def addGrid(self, param, values): """ Sets the given parameters in this grid to fixed values. + + param must be an instance of Param associated with an instance of Params + (such as Estimator or Transformer). """ - self._param_grid[param] = values + if isinstance(param, Param): + self._param_grid[param] = values + else: + raise TypeError("param must be an instance of Param") return self From e753aa30e659706c3fa3414bf38566a79e0af8d6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 19 Nov 2019 15:56:50 -0800 Subject: [PATCH 0668/1387] [SPARK-29964][BUILD] lintr github workflows failed due to buggy GnuPG ### What changes were proposed in this pull request? Linter (R) github workflows failed sometimes like: https://github.com/apache/spark/pull/26509/checks?check_run_id=310718016 Failed message: ``` Executing: /tmp/apt-key-gpghome.8r74rQNEjj/gpg.1.sh --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9 gpg: connecting dirmngr at '/tmp/apt-key-gpghome.8r74rQNEjj/S.dirmngr' failed: IPC connect call failed gpg: keyserver receive failed: No dirmngr ##[error]Process completed with exit code 2. ``` It is due to a buggy GnuPG. Context: https://github.com/sbt/website/pull/825 https://github.com/sbt/sbt/issues/4261 https://github.com/microsoft/WSL/issues/3286 ### Why are the changes needed? Make lint-r github workflows work. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass github workflows. Closes #26602 from viirya/SPARK-29964. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d2b7dca3684f0..2cd48153b190c 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -84,7 +84,7 @@ jobs: - name: install R run: | echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' | sudo tee -a /etc/apt/sources.list - sudo apt-key adv --keyserver keyserver.ubuntu.com --recv-keys E298A3A825C0D65DFD57CBB651716619E084DAB9 + curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add sudo apt-get update sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev - name: install R packages From 40b8a08b8b7f71685c5703605fd8a8b101de32f7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 20 Nov 2019 10:34:25 +0900 Subject: [PATCH 0669/1387] [SPARK-29963][SQL][TESTS] Check formatting timestamps up to microsecond precision by JSON/CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to add tests from the commit https://github.com/apache/spark/commit/47cb1f359af62383e24198dbbaa0b4503348cd04 for Spark 2.4 that check formatting of timestamp strings for various seconds fractions. ### Why are the changes needed? To make sure that current behavior is the same as in Spark 2.4 ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `CSVSuite`, `JsonFunctionsSuite` and `TimestampFormatterSuite`. Closes #26601 from MaxGekk/format-timestamp-micros-tests. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../sql/util/TimestampFormatterSuite.scala | 40 +++++++++++++++++++ .../apache/spark/sql/JsonFunctionsSuite.scala | 7 ++++ .../execution/datasources/csv/CSVSuite.scala | 15 +++++++ 3 files changed, 62 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala index 79e433e88437b..082849c88669a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -193,4 +193,44 @@ class TimestampFormatterSuite extends SparkFunSuite with SQLHelper with Matchers } } } + + test("formatting timestamp strings up to microsecond precision") { + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + def check(pattern: String, input: String, expected: String): Unit = { + val formatter = TimestampFormatter(pattern, zoneId) + val timestamp = DateTimeUtils.stringToTimestamp( + UTF8String.fromString(input), zoneId).get + val actual = formatter.format(timestamp) + assert(actual === expected) + } + + check( + "yyyy-MM-dd HH:mm:ss.SSSSSSS", "2019-10-14T09:39:07.123456", + "2019-10-14 09:39:07.1234560") + check( + "yyyy-MM-dd HH:mm:ss.SSSSSS", "1960-01-01T09:39:07.123456", + "1960-01-01 09:39:07.123456") + check( + "yyyy-MM-dd HH:mm:ss.SSSSS", "0001-10-14T09:39:07.1", + "0001-10-14 09:39:07.10000") + check( + "yyyy-MM-dd HH:mm:ss.SSSS", "9999-12-31T23:59:59.999", + "9999-12-31 23:59:59.9990") + check( + "yyyy-MM-dd HH:mm:ss.SSS", "1970-01-01T00:00:00.0101", + "1970-01-01 00:00:00.010") + check( + "yyyy-MM-dd HH:mm:ss.SS", "2019-10-14T09:39:07.09", + "2019-10-14 09:39:07.09") + check( + "yyyy-MM-dd HH:mm:ss.S", "2019-10-14T09:39:07.2", + "2019-10-14 09:39:07.2") + check( + "yyyy-MM-dd HH:mm:ss.S", "2019-10-14T09:39:07", + "2019-10-14 09:39:07.0") + check( + "yyyy-MM-dd HH:mm:ss", "2019-10-14T09:39:07.123456", + "2019-10-14 09:39:07") + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index af80fa3d9327a..f3483151d26da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -637,4 +637,11 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { df.select(from_json($"value", schema, options)), Row(Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.123456")))) } + + test("to_json - timestamp in micros") { + val s = "2019-11-18 11:56:00.123456" + val df = Seq(java.sql.Timestamp.valueOf(s)).toDF("t").select( + to_json(struct($"t"), Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss.SSSSSS"))) + checkAnswer(df, Row(s"""{"t":"$s"}""")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index dac305944dd0d..a6c3a51858aed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2180,4 +2180,19 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { checkAnswer(readback, Row(Timestamp.valueOf(t))) } } + + test("Roundtrip in reading and writing timestamps in microsecond precision") { + withTempPath { path => + val timestamp = Timestamp.valueOf("2019-11-18 11:56:00.123456") + Seq(timestamp).toDF("t") + .write + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSSSSS") + .csv(path.getAbsolutePath) + val readback = spark.read + .schema("t timestamp") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSSSSS") + .csv(path.getAbsolutePath) + checkAnswer(readback, Row(timestamp)) + } + } } From 9e58b10c8e1e066e3daae06f57e48f369a24e9ec Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 20 Nov 2019 11:08:04 +0900 Subject: [PATCH 0670/1387] [SPARK-29945][SQL] do not handle negative sign specially in the parser ### What changes were proposed in this pull request? Remove the special handling of the negative sign in the parser (interval literal and type constructor) ### Why are the changes needed? The negative sign is an operator (UnaryMinus). We don't need to handle it specially, which is kind of doing constant folding at parser side. ### Does this PR introduce any user-facing change? The error message becomes a little different. Now it reports type mismatch for the `-` operator. ### How was this patch tested? existing tests Closes #26578 from cloud-fan/interval. Authored-by: Wenchen Fan Signed-off-by: Takeshi Yamamuro --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 31 +++------ .../parser/ExpressionParserSuite.scala | 5 +- .../resources/sql-tests/inputs/literals.sql | 5 +- .../sql-tests/results/ansi/interval.sql.out | 12 ++-- .../sql-tests/results/ansi/literals.sql.out | 65 ++++++++++--------- .../results/interval-display-iso_8601.sql.out | 2 +- .../interval-display-sql_standard.sql.out | 2 +- .../results/interval-display.sql.out | 2 +- .../sql-tests/results/interval.sql.out | 12 ++-- .../sql-tests/results/literals.sql.out | 65 ++++++++++--------- .../results/postgreSQL/interval.sql.out | 2 +- 12 files changed, 98 insertions(+), 109 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 41f8e3552977d..4f81dc47394a5 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -764,7 +764,7 @@ primaryExpression constant : NULL #nullLiteral | interval #intervalLiteral - | negativeSign=MINUS? identifier STRING #typeConstructor + | identifier STRING #typeConstructor | number #numericLiteral | booleanValue #booleanLiteral | STRING+ #stringLiteral @@ -787,7 +787,7 @@ booleanValue ; interval - : negativeSign=MINUS? INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? + : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? | {ansi}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 5b9c204fcd4ae..e0476879f0e80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1850,7 +1850,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { val value = string(ctx.STRING) val valueType = ctx.identifier.getText.toUpperCase(Locale.ROOT) - val isNegative = ctx.negativeSign != null def toLiteral[T](f: UTF8String => Option[T], t: DataType): Literal = { f(UTF8String.fromString(value)).map(Literal(_, t)).getOrElse { @@ -1859,9 +1858,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } try { valueType match { - case "DATE" if !isNegative => + case "DATE" => toLiteral(stringToDate(_, getZoneId(SQLConf.get.sessionLocalTimeZone)), DateType) - case "TIMESTAMP" if !isNegative => + case "TIMESTAMP" => val zoneId = getZoneId(SQLConf.get.sessionLocalTimeZone) toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "INTERVAL" => @@ -1873,9 +1872,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ex.setStackTrace(e.getStackTrace) throw ex } - val signedInterval = if (isNegative) IntervalUtils.negate(interval) else interval - Literal(signedInterval, CalendarIntervalType) - case "X" if !isNegative => + Literal(interval, CalendarIntervalType) + case "X" => val padding = if (value.length % 2 != 0) "0" else "" Literal(DatatypeConverter.parseHexBinary(padding + value)) case "INTEGER" => @@ -1887,10 +1885,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ex.setStackTrace(e.getStackTrace) throw ex } - Literal(if (isNegative) -i else i, IntegerType) + Literal(i, IntegerType) case other => - val negativeSign: String = if (isNegative) "-" else "" - throw new ParseException(s"Literals of type '$negativeSign$other' are currently not" + + throw new ParseException(s"Literals of type '$other' are currently not" + " supported.", ctx) } } catch { @@ -2020,14 +2017,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } - private def applyNegativeSign(sign: Token, interval: CalendarInterval): CalendarInterval = { - if (sign != null) { - IntervalUtils.negate(interval) - } else { - interval - } - } - /** * Create a [[CalendarInterval]] literal expression. Two syntaxes are supported: * - multiple unit value pairs, for instance: interval 2 months 2 days. @@ -2041,10 +2030,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Can only have a single from-to unit in the interval literal syntax", innerCtx.unitToUnitInterval) } - val interval = applyNegativeSign( - ctx.negativeSign, - visitMultiUnitsInterval(innerCtx.multiUnitsInterval)) - Literal(interval, CalendarIntervalType) + Literal(visitMultiUnitsInterval(innerCtx.multiUnitsInterval), CalendarIntervalType) } else if (ctx.errorCapturingUnitToUnitInterval != null) { val innerCtx = ctx.errorCapturingUnitToUnitInterval if (innerCtx.error1 != null || innerCtx.error2 != null) { @@ -2053,8 +2039,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "Can only have a single from-to unit in the interval literal syntax", errorCtx) } - val interval = applyNegativeSign(ctx.negativeSign, visitUnitToUnitInterval(innerCtx.body)) - Literal(interval, CalendarIntervalType) + Literal(visitUnitToUnitInterval(innerCtx.body), CalendarIntervalType) } else { throw new ParseException("at least one time unit should be given for interval literal", ctx) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index a7dbd4e4e3459..371b702722a69 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -442,7 +442,7 @@ class ExpressionParserSuite extends AnalysisTest { intercept("Interval 'interval 3 monthsss 1 hoursss'", "Cannot parse the INTERVAL value") assertEqual( "-interval '3 month 1 hour'", - Literal(IntervalUtils.stringToInterval("interval -3 month -1 hour"))) + UnaryMinus(Literal(IntervalUtils.stringToInterval("interval 3 month 1 hour")))) // Binary. assertEqual("X'A'", Literal(Array(0x0a).map(_.toByte))) @@ -611,8 +611,7 @@ class ExpressionParserSuite extends AnalysisTest { def checkIntervals(intervalValue: String, expected: Literal): Unit = { Seq( "" -> expected, - "-" -> expected.copy( - value = IntervalUtils.negate(expected.value.asInstanceOf[CalendarInterval])) + "-" -> UnaryMinus(expected) ).foreach { case (sign, expectedLiteral) => assertEqual(s"${sign}interval $intervalValue", expectedLiteral) diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index d1dff7bc94686..61b02d86bb51b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -107,7 +107,10 @@ select integer '2147483648'; -- awareness of the negative/positive sign before type select -integer '7'; +select +integer '7'; +select +date '1999-01-01'; +select +timestamp '1999-01-01'; +-- can't negate date/timestamp/binary select -date '1999-01-01'; select -timestamp '1999-01-01'; select -x'2379ACFe'; -select +integer '7'; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 11c70d4987eec..bceb6bd1d2ea9 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -239,7 +239,7 @@ NULL -- !query 29 select -interval '-1 month 1 day -1 second' -- !query 29 schema -struct +struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 29 output 1 months -1 days 1 seconds @@ -247,7 +247,7 @@ struct -- !query 30 select -interval -1 month 1 day -1 second -- !query 30 schema -struct +struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 30 output 1 months -1 days 1 seconds @@ -1040,7 +1040,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query 112 schema -struct +struct -- !query 112 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -1056,7 +1056,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query 113 schema -struct +struct -- !query 113 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -1083,7 +1083,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query 115 schema -struct +struct -- !query 115 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -1099,7 +1099,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query 116 schema -struct +struct -- !query 116 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 6ee7148755c34..c1488eaf2aa36 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 48 +-- Number of queries: 50 -- !query 0 @@ -419,56 +419,57 @@ select integer '2147483648' -- !query 43 select -integer '7' -- !query 43 schema -struct<-7:int> +struct<(- 7):int> -- !query 43 output -7 -- !query 44 -select -date '1999-01-01' +select +integer '7' -- !query 44 schema -struct<> +struct<7:int> -- !query 44 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-DATE' are currently not supported.(line 1, pos 7) - -== SQL == -select -date '1999-01-01' --------^^^ +7 -- !query 45 -select -timestamp '1999-01-01' +select +date '1999-01-01' -- !query 45 schema -struct<> +struct -- !query 45 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-TIMESTAMP' are currently not supported.(line 1, pos 7) - -== SQL == -select -timestamp '1999-01-01' --------^^^ +1999-01-01 -- !query 46 -select -x'2379ACFe' +select +timestamp '1999-01-01' -- !query 46 schema -struct<> +struct -- !query 46 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-X' are currently not supported.(line 1, pos 7) - -== SQL == -select -x'2379ACFe' --------^^^ +1999-01-01 00:00:00 -- !query 47 -select +integer '7' +select -date '1999-01-01' -- !query 47 schema -struct<7:int> +struct<> -- !query 47 output -7 +org.apache.spark.sql.AnalysisException +cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 + + +-- !query 48 +select -timestamp '1999-01-01' +-- !query 48 schema +struct<> +-- !query 48 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 + + +-- !query 49 +select -x'2379ACFe' +-- !query 49 schema +struct<> +-- !query 49 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out index 0e053940df38f..2da098f17e168 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL PT0S P1Y P1M P1Y2M P1DT-1H P-1DT-1H P-1DT1H P-1DT1H P1Y2M-3DT4H5M6.789S P-1Y-2M3DT-4H-5M-6.789S diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out index 2633522bb1d08..7ef2f9c51218e 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL 0 +1-0 +0-1 +1-2 +1 -1:00:00 -1 -1:00:00 -1 +1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out index faf27bda1549f..a292fe3c83028 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out @@ -16,6 +16,6 @@ SELECT interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 0 schema -struct +struct -- !query 0 output NULL 0 seconds 1 years 1 months 1 years 2 months 1 days -1 hours -1 days -1 hours -1 days 1 hours -1 days 1 hours 1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds -1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 6d09f62a6dcec..bda5fc6d8b8e9 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -239,7 +239,7 @@ NULL -- !query 29 select -interval '-1 month 1 day -1 second' -- !query 29 schema -struct +struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 29 output 1 months -1 days 1 seconds @@ -247,7 +247,7 @@ struct -- !query 30 select -interval -1 month 1 day -1 second -- !query 30 schema -struct +struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 30 output 1 months -1 days 1 seconds @@ -1024,7 +1024,7 @@ select interval '2-2' year to month + dateval from interval_arithmetic -- !query 112 schema -struct +struct -- !query 112 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 @@ -1040,7 +1040,7 @@ select interval '2-2' year to month + tsval from interval_arithmetic -- !query 113 schema -struct +struct -- !query 113 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 @@ -1067,7 +1067,7 @@ select interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic -- !query 115 schema -struct +struct -- !query 115 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 @@ -1083,7 +1083,7 @@ select interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic -- !query 116 schema -struct +struct -- !query 116 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 6ee7148755c34..c1488eaf2aa36 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 48 +-- Number of queries: 50 -- !query 0 @@ -419,56 +419,57 @@ select integer '2147483648' -- !query 43 select -integer '7' -- !query 43 schema -struct<-7:int> +struct<(- 7):int> -- !query 43 output -7 -- !query 44 -select -date '1999-01-01' +select +integer '7' -- !query 44 schema -struct<> +struct<7:int> -- !query 44 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-DATE' are currently not supported.(line 1, pos 7) - -== SQL == -select -date '1999-01-01' --------^^^ +7 -- !query 45 -select -timestamp '1999-01-01' +select +date '1999-01-01' -- !query 45 schema -struct<> +struct -- !query 45 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-TIMESTAMP' are currently not supported.(line 1, pos 7) - -== SQL == -select -timestamp '1999-01-01' --------^^^ +1999-01-01 -- !query 46 -select -x'2379ACFe' +select +timestamp '1999-01-01' -- !query 46 schema -struct<> +struct -- !query 46 output -org.apache.spark.sql.catalyst.parser.ParseException - -Literals of type '-X' are currently not supported.(line 1, pos 7) - -== SQL == -select -x'2379ACFe' --------^^^ +1999-01-01 00:00:00 -- !query 47 -select +integer '7' +select -date '1999-01-01' -- !query 47 schema -struct<7:int> +struct<> -- !query 47 output -7 +org.apache.spark.sql.AnalysisException +cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 + + +-- !query 48 +select -timestamp '1999-01-01' +-- !query 48 schema +struct<> +-- !query 48 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 + + +-- !query 49 +select -x'2379ACFe' +-- !query 49 schema +struct<> +-- !query 49 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index 4eaeee6030d66..f23415a108ff0 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -232,7 +232,7 @@ SELECT interval '1 day -1 hours', interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' -- !query 28 schema -struct +struct -- !query 28 output +1 -1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 From 5a70af7a6c271e12a1f8b508dcede804e30a1b75 Mon Sep 17 00:00:00 2001 From: Nikita Konda Date: Tue, 19 Nov 2019 20:01:42 -0800 Subject: [PATCH 0671/1387] [SPARK-29029][SQL] Use AttributeMap in PhysicalOperation.collectProjectsAndFilters ### What changes were proposed in this pull request? This PR fixes the issue of substituting aliases while collecting filters in `PhysicalOperation.collectProjectsAndFilters`. When the `AttributeReference` in alias map differs from the `AttributeReference` in filter condition only in qualifier, it does not substitute alias and throws exception saying `key videoid#47L not found` in the following scenario. ``` [1] Project [userid#0] +- [2] Filter (isnotnull(videoid#47L) && NOT (videoid#47L = 30)) +- [3] Project [factorial(videoid#1) AS videoid#47L, userid#0] +- [4] Filter (isnotnull(avebitrate#2) && (avebitrate#2 < 10)) +- [5] Relation[userid#0,videoid#1,avebitrate#2] ``` ### Why are the changes needed? We need to use `AttributeMap` where the key is `AttributeReference`'s `ExprId` instead of `Map[Attribute, Expression]` while collecting and substituting aliases in `PhysicalOperation.collectProjectsAndFilters`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? New unit tests were added in `TestPhysicalOperation` which reproduces the bug Closes #25761 from nikitagkonda/SPARK-29029-use-attributemap-for-aliasmap-in-physicaloperation. Authored-by: Nikita Konda Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/planning/patterns.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 51d2a73ea97b7..c2a12eda19137 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -56,7 +56,7 @@ object PhysicalOperation extends PredicateHelper { * }}} */ private def collectProjectsAndFilters(plan: LogicalPlan): - (Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan, Map[Attribute, Expression]) = + (Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan, AttributeMap[Expression]) = plan match { case Project(fields, child) if fields.forall(_.deterministic) => val (_, filters, other, aliases) = collectProjectsAndFilters(child) @@ -72,14 +72,15 @@ object PhysicalOperation extends PredicateHelper { collectProjectsAndFilters(h.child) case other => - (None, Nil, other, Map.empty) + (None, Nil, other, AttributeMap(Seq())) } - private def collectAliases(fields: Seq[Expression]): Map[Attribute, Expression] = fields.collect { - case a @ Alias(child, _) => a.toAttribute -> child - }.toMap + private def collectAliases(fields: Seq[Expression]): AttributeMap[Expression] = + AttributeMap(fields.collect { + case a: Alias => (a.toAttribute, a.child) + }) - private def substitute(aliases: Map[Attribute, Expression])(expr: Expression): Expression = { + private def substitute(aliases: AttributeMap[Expression])(expr: Expression): Expression = { expr.transform { case a @ Alias(ref: AttributeReference, name) => aliases.get(ref) From 0032d85153e34b9ac69598b7dff530094ed0f640 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 20 Nov 2019 21:13:51 +0800 Subject: [PATCH 0672/1387] [SPARK-29968][SQL] Remove the Predicate code from SparkPlan ### What changes were proposed in this pull request? This is to refactor Predicate code; it mainly removed `newPredicate` from `SparkPlan`. Modifications are listed below; - Move `Predicate` from `o.a.s.sqlcatalyst.expressions.codegen.GeneratePredicate.scala` to `o.a.s.sqlcatalyst.expressions.predicates.scala` - To resolve the name conflict, rename `o.a.s.sqlcatalyst.expressions.codegen.Predicate` to `o.a.s.sqlcatalyst.expressions.BasePredicate` - Extend `CodeGeneratorWithInterpretedFallback ` for `BasePredicate` This comes from the cloud-fan suggestion: https://github.com/apache/spark/pull/26420#discussion_r348005497 ### Why are the changes needed? For better code/test coverage. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26604 from maropu/RefactorPredicate. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../catalog/ExternalCatalogUtils.scala | 5 +- .../sql/catalyst/expressions/Projection.scala | 4 +- .../codegen/GeneratePredicate.scala | 22 ++------- .../sql/catalyst/expressions/predicates.scala | 47 +++++++++++++++++-- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../catalyst/expressions/PredicateSuite.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 2 +- .../spark/sql/execution/SparkPlan.scala | 25 +--------- .../execution/basicPhysicalOperators.scala | 3 +- .../columnar/InMemoryTableScanExec.scala | 2 +- .../PartitioningAwareFileIndex.scala | 2 +- .../joins/BroadcastNestedLoopJoinExec.scala | 4 +- .../joins/CartesianProductExec.scala | 5 +- .../spark/sql/execution/joins/HashJoin.scala | 2 +- .../execution/joins/SortMergeJoinExec.scala | 2 +- .../StreamingSymmetricHashJoinExec.scala | 17 +++---- .../streaming/statefulOperators.scala | 10 ++-- .../sql/sources/SimpleTextRelation.scala | 4 +- 18 files changed, 78 insertions(+), 82 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala index 4cff162c116a4..ae3b75dc3334b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI -import java.util.Locale import org.apache.hadoop.fs.Path import org.apache.hadoop.util.Shell @@ -26,7 +25,7 @@ import org.apache.hadoop.util.Shell import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, Predicate} object ExternalCatalogUtils { // This duplicates default value of Hive `ConfVars.DEFAULTPARTITIONNAME`, since catalyst doesn't @@ -148,7 +147,7 @@ object ExternalCatalogUtils { } val boundPredicate = - InterpretedPredicate.create(predicates.reduce(And).transform { + Predicate.createInterpreted(predicates.reduce(And).transform { case att: AttributeReference => val index = partitionSchema.indexWhere(_.name == att.name) BoundReference(index, partitionSchema(index).dataType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 300f075d32763..b4a85e3e50bec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -89,14 +89,14 @@ object MutableProjection } /** - * Returns an MutableProjection for given sequence of bound Expressions. + * Returns a MutableProjection for given sequence of bound Expressions. */ def create(exprs: Seq[Expression]): MutableProjection = { createObject(exprs) } /** - * Returns an MutableProjection for given sequence of Expressions, which will be bound to + * Returns a MutableProjection for given sequence of Expressions, which will be bound to * `inputSchema`. */ def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): MutableProjection = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index e0fabad6d089a..6ba646d360d2e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -20,31 +20,17 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -/** - * Interface for generated predicate - */ -abstract class Predicate { - def eval(r: InternalRow): Boolean - - /** - * Initializes internal states given the current partition index. - * This is used by nondeterministic expressions to set initial states. - * The default implementation does nothing. - */ - def initialize(partitionIndex: Int): Unit = {} -} - /** * Generates bytecode that evaluates a boolean [[Expression]] on a given input [[InternalRow]]. */ -object GeneratePredicate extends CodeGenerator[Expression, Predicate] { +object GeneratePredicate extends CodeGenerator[Expression, BasePredicate] { protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer.execute(in) protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression = BindReferences.bindReference(in, inputSchema) - protected def create(predicate: Expression): Predicate = { + protected def create(predicate: Expression): BasePredicate = { val ctx = newCodeGenContext() val eval = predicate.genCode(ctx) @@ -53,7 +39,7 @@ object GeneratePredicate extends CodeGenerator[Expression, Predicate] { return new SpecificPredicate(references); } - class SpecificPredicate extends ${classOf[Predicate].getName} { + class SpecificPredicate extends ${classOf[BasePredicate].getName} { private final Object[] references; ${ctx.declareMutableStates()} @@ -79,6 +65,6 @@ object GeneratePredicate extends CodeGenerator[Expression, Predicate] { logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}") val (clazz, _) = CodeGenerator.compile(code) - clazz.generate(ctx.references.toArray).asInstanceOf[Predicate] + clazz.generate(ctx.references.toArray).asInstanceOf[BasePredicate] } } 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 4c0998412f729..bcd442ad3cc35 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 @@ -21,8 +21,9 @@ import scala.collection.immutable.TreeSet import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral, GenerateSafeProjection, GenerateUnsafeProjection, Predicate => BasePredicate} +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.util.TypeUtils @@ -30,11 +31,18 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -object InterpretedPredicate { - def create(expression: Expression, inputSchema: Seq[Attribute]): InterpretedPredicate = - create(BindReferences.bindReference(expression, inputSchema)) +/** + * A base class for generated/interpreted predicate + */ +abstract class BasePredicate { + def eval(r: InternalRow): Boolean - def create(expression: Expression): InterpretedPredicate = new InterpretedPredicate(expression) + /** + * Initializes internal states given the current partition index. + * This is used by nondeterministic expressions to set initial states. + * The default implementation does nothing. + */ + def initialize(partitionIndex: Int): Unit = {} } case class InterpretedPredicate(expression: Expression) extends BasePredicate { @@ -56,6 +64,35 @@ trait Predicate extends Expression { override def dataType: DataType = BooleanType } +/** + * The factory object for `BasePredicate`. + */ +object Predicate extends CodeGeneratorWithInterpretedFallback[Expression, BasePredicate] { + + override protected def createCodeGeneratedObject(in: Expression): BasePredicate = { + GeneratePredicate.generate(in) + } + + override protected def createInterpretedObject(in: Expression): BasePredicate = { + InterpretedPredicate(in) + } + + def createInterpreted(e: Expression): InterpretedPredicate = InterpretedPredicate(e) + + /** + * Returns a BasePredicate for an Expression, which will be bound to `inputSchema`. + */ + def create(e: Expression, inputSchema: Seq[Attribute]): BasePredicate = { + createObject(bindReference(e, inputSchema)) + } + + /** + * Returns a BasePredicate for a given bound Expression. + */ + def create(e: Expression): BasePredicate = { + createObject(e) + } +} trait PredicateHelper { protected def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { 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 473f846c9313b..9d0bd358aa24c 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 @@ -1506,7 +1506,7 @@ object ConvertToLocalRelation extends Rule[LogicalPlan] { case Filter(condition, LocalRelation(output, data, isStreaming)) if !hasUnevaluableExpr(condition) => - val predicate = InterpretedPredicate.create(condition, output) + val predicate = Predicate.create(condition, output) predicate.initialize(0) LocalRelation(output, data.filter(row => predicate.eval(row)), isStreaming) } 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 52cdd988caa2e..67a41e7cc2767 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 @@ -510,7 +510,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Interpreted Predicate should initialize nondeterministic expressions") { - val interpreted = InterpretedPredicate.create(LessThan(Rand(7), Literal(1.0))) + val interpreted = Predicate.create(LessThan(Rand(7), Literal(1.0))) interpreted.initialize(0) assert(interpreted.eval(new UnsafeRow())) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index b0fe4b741479f..88f5673aa9a1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -230,7 +230,7 @@ case class FileSourceScanExec( // call the file index for the files matching all filters except dynamic partition filters val predicate = dynamicPartitionFilters.reduce(And) val partitionColumns = relation.partitionSchema - val boundPredicate = newPredicate(predicate.transform { + val boundPredicate = Predicate.create(predicate.transform { case a: AttributeReference => val index = partitionColumns.indexWhere(a.name == _.name) BoundReference(index, partitionColumns(index).dataType, nullable = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 125f76282e3df..738af995376e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -21,7 +21,6 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, Da import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext import org.codehaus.commons.compiler.CompileException import org.codehaus.janino.InternalCompilerException @@ -33,7 +32,7 @@ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{Predicate => GenPredicate, _} +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ @@ -471,28 +470,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ MutableProjection.create(expressions, inputSchema) } - private def genInterpretedPredicate( - expression: Expression, inputSchema: Seq[Attribute]): InterpretedPredicate = { - val str = expression.toString - val logMessage = if (str.length > 256) { - str.substring(0, 256 - 3) + "..." - } else { - str - } - logWarning(s"Codegen disabled for this expression:\n $logMessage") - InterpretedPredicate.create(expression, inputSchema) - } - - protected def newPredicate( - expression: Expression, inputSchema: Seq[Attribute]): GenPredicate = { - try { - GeneratePredicate.generate(expression, inputSchema) - } catch { - case _ @ (_: InternalCompilerException | _: CompileException) if codeGenFallBack => - genInterpretedPredicate(expression, inputSchema) - } - } - protected def newOrdering( order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[InternalRow] = { GenerateOrdering.generate(order, inputSchema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 3ed42f359c0a4..e128d59dca6ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.{LongType, StructType} @@ -227,7 +226,7 @@ case class FilterExec(condition: Expression, child: SparkPlan) protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") child.execute().mapPartitionsWithIndexInternal { (index, iter) => - val predicate = newPredicate(condition, child.output) + val predicate = Predicate.create(condition, child.output) predicate.initialize(0) iter.filter { row => val r = predicate.eval(row) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 8d13cfb93d270..f03c2586048bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -310,7 +310,7 @@ case class InMemoryTableScanExec( val buffers = relation.cacheBuilder.cachedColumnBuffers buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = newPredicate( + val partitionFilter = Predicate.create( partitionFilters.reduceOption(And).getOrElse(Literal(true)), schema) partitionFilter.initialize(index) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 3adec2f790730..21ddeb6491155 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -171,7 +171,7 @@ abstract class PartitioningAwareFileIndex( if (partitionPruningPredicates.nonEmpty) { val predicate = partitionPruningPredicates.reduce(expressions.And) - val boundPredicate = InterpretedPredicate.create(predicate.transform { + val boundPredicate = Predicate.createInterpreted(predicate.transform { case a: AttributeReference => val index = partitionColumns.indexWhere(a.name == _.name) BoundReference(index, partitionColumns(index).dataType, nullable = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index f526a19876670..5517c0dcdb188 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -19,14 +19,12 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.collection.{BitSet, CompactBuffer} case class BroadcastNestedLoopJoinExec( @@ -84,7 +82,7 @@ case class BroadcastNestedLoopJoinExec( @transient private lazy val boundCondition = { if (condition.isDefined) { - newPredicate(condition.get, streamed.output ++ broadcast.output).eval _ + Predicate.create(condition.get, streamed.output ++ broadcast.output).eval _ } else { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 88d98530991c9..29645a736548c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -20,9 +20,8 @@ package org.apache.spark.sql.execution.joins import org.apache.spark._ import org.apache.spark.rdd.{CartesianPartition, CartesianRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, JoinedRow, Predicate, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner -import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.execution.{BinaryExecNode, ExplainUtils, ExternalAppendOnlyUnsafeRowArray, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.CompletionIterator @@ -93,7 +92,7 @@ case class CartesianProductExec( pair.mapPartitionsWithIndexInternal { (index, iter) => val joiner = GenerateUnsafeRowJoiner.create(left.schema, right.schema) val filtered = if (condition.isDefined) { - val boundCondition = newPredicate(condition.get, left.output ++ right.output) + val boundCondition = Predicate.create(condition.get, left.output ++ right.output) boundCondition.initialize(index) val joined = new JoinedRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index e8938cb22e890..137f0b87a2f3d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -99,7 +99,7 @@ trait HashJoin { UnsafeProjection.create(streamedKeys) @transient private[this] lazy val boundCondition = if (condition.isDefined) { - newPredicate(condition.get, streamedPlan.output ++ buildPlan.output).eval _ + Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ } else { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 26fb0e5ffb1af..cd3c596435a21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -168,7 +168,7 @@ case class SortMergeJoinExec( left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => val boundCondition: (InternalRow) => Boolean = { condition.map { cond => - newPredicate(cond, left.output ++ right.output).eval _ + Predicate.create(cond, left.output ++ right.output).eval _ }.getOrElse { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index 6bb4dc1672900..f1bfe97610fed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit.NANOSECONDS import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow, JoinedRow, Literal, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow, JoinedRow, Literal, Predicate, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -233,8 +233,9 @@ case class StreamingSymmetricHashJoinExec( val joinedRow = new JoinedRow + val inputSchema = left.output ++ right.output val postJoinFilter = - newPredicate(condition.bothSides.getOrElse(Literal(true)), left.output ++ right.output).eval _ + Predicate.create(condition.bothSides.getOrElse(Literal(true)), inputSchema).eval _ val leftSideJoiner = new OneSideHashJoiner( LeftSide, left.output, leftKeys, leftInputIter, condition.leftSideOnly, postJoinFilter, stateWatermarkPredicates.left) @@ -417,7 +418,7 @@ case class StreamingSymmetricHashJoinExec( // Filter the joined rows based on the given condition. val preJoinFilter = - newPredicate(preJoinFilterExpr.getOrElse(Literal(true)), inputAttributes).eval _ + Predicate.create(preJoinFilterExpr.getOrElse(Literal(true)), inputAttributes).eval _ private val joinStateManager = new SymmetricHashJoinStateManager( joinSide, inputAttributes, joinKeys, stateInfo, storeConf, hadoopConfBcast.value.value, @@ -428,16 +429,16 @@ case class StreamingSymmetricHashJoinExec( case Some(JoinStateKeyWatermarkPredicate(expr)) => // inputSchema can be empty as expr should only have BoundReferences and does not require // the schema to generated predicate. See [[StreamingSymmetricHashJoinHelper]]. - newPredicate(expr, Seq.empty).eval _ + Predicate.create(expr, Seq.empty).eval _ case _ => - newPredicate(Literal(false), Seq.empty).eval _ // false = do not remove if no predicate + Predicate.create(Literal(false), Seq.empty).eval _ // false = do not remove if no predicate } private[this] val stateValueWatermarkPredicateFunc = stateWatermarkPredicate match { case Some(JoinStateValueWatermarkPredicate(expr)) => - newPredicate(expr, inputAttributes).eval _ + Predicate.create(expr, inputAttributes).eval _ case _ => - newPredicate(Literal(false), Seq.empty).eval _ // false = do not remove if no predicate + Predicate.create(Literal(false), Seq.empty).eval _ // false = do not remove if no predicate } private[this] var updatedStateRowsCount = 0 @@ -457,7 +458,7 @@ case class StreamingSymmetricHashJoinExec( val nonLateRows = WatermarkSupport.watermarkExpression(watermarkAttribute, eventTimeWatermark) match { case Some(watermarkExpr) => - val predicate = newPredicate(watermarkExpr, inputAttributes) + val predicate = Predicate.create(watermarkExpr, inputAttributes) inputIter.filter { row => !predicate.eval(row) } case None => inputIter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index d689a6f3c9819..01b309c3cf345 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -26,7 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeProjection, Predicate} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ @@ -156,17 +156,17 @@ trait WatermarkSupport extends UnaryExecNode { } /** Predicate based on keys that matches data older than the watermark */ - lazy val watermarkPredicateForKeys: Option[Predicate] = watermarkExpression.flatMap { e => + lazy val watermarkPredicateForKeys: Option[BasePredicate] = watermarkExpression.flatMap { e => if (keyExpressions.exists(_.metadata.contains(EventTimeWatermark.delayKey))) { - Some(newPredicate(e, keyExpressions)) + Some(Predicate.create(e, keyExpressions)) } else { None } } /** Predicate based on the child output that matches data older than the watermark. */ - lazy val watermarkPredicateForData: Option[Predicate] = - watermarkExpression.map(newPredicate(_, child.output)) + lazy val watermarkPredicateForData: Option[BasePredicate] = + watermarkExpression.map(Predicate.create(_, child.output)) protected def removeKeysOlderThanWatermark(store: StateStore): Unit = { if (watermarkPredicateForKeys.nonEmpty) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 60a4638f610b3..d1b97b2852fbc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.sql.{sources, SparkSession} import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedPredicate, InterpretedProjection, JoinedRow, Literal} +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedProjection, JoinedRow, Literal, Predicate} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.{DataType, StructType} @@ -88,7 +88,7 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { val attribute = inputAttributes.find(_.name == column).get expressions.GreaterThan(attribute, literal) }.reduceOption(expressions.And).getOrElse(Literal(true)) - InterpretedPredicate.create(filterCondition, inputAttributes) + Predicate.create(filterCondition, inputAttributes) } // Uses a simple projection to simulate column pruning From 56a65b971d4f1b43c37a5664a3a4e2e4fcad1c14 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 20 Nov 2019 08:20:16 -0600 Subject: [PATCH 0673/1387] [SPARK-18409][ML] LSH approxNearestNeighbors should use approxQuantile instead of sort ### What changes were proposed in this pull request? ```LSHModel.approxNearestNeighbors``` sorts the full dataset on the hashDistance in order to find a threshold. This PR uses approxQuantile instead. ### Why are the changes needed? To improve performance. ### Does this PR introduce any user-facing change? Yes. Changed ```LSH``` to make it extend ```HasRelativeError``` ```LSH``` and ```LSHModel``` have new APIs ```setRelativeError/getRelativeError``` ### How was this patch tested? Existing tests. Also added a couple doc test in python to test newly added ```getRelativeError``` Closes #26415 from huaxingao/spark-18409. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../org/apache/spark/ml/feature/LSH.scala | 37 ++++++++++++------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala index b20852383a6ff..4885d03220e95 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala @@ -112,7 +112,9 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] numNearestNeighbors: Int, singleProbe: Boolean, distCol: String): Dataset[_] = { - require(numNearestNeighbors > 0, "The number of nearest neighbors cannot be less than 1") + val count = dataset.count() + require(numNearestNeighbors > 0 && numNearestNeighbors <= count, "The number of" + + " nearest neighbors cannot be less than 1 or greater than the number of elements in dataset") // Get Hash Value of the key val keyHash = hashFunction(key) val modelDataset: DataFrame = if (!dataset.columns.contains($(outputCol))) { @@ -137,14 +139,21 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] val hashDistUDF = udf((x: Seq[Vector]) => hashDistance(x, keyHash), DataTypes.DoubleType) val hashDistCol = hashDistUDF(col($(outputCol))) - // Compute threshold to get exact k elements. - // TODO: SPARK-18409: Use approxQuantile to get the threshold - val modelDatasetSortedByHash = modelDataset.sort(hashDistCol).limit(numNearestNeighbors) - val thresholdDataset = modelDatasetSortedByHash.select(max(hashDistCol)) - val hashThreshold = thresholdDataset.take(1).head.getDouble(0) - - // Filter the dataset where the hash value is less than the threshold. - modelDataset.filter(hashDistCol <= hashThreshold) + // Compute threshold to get around k elements. + // To guarantee to have enough neighbors in one pass, we need (p - err) * N >= M + // so we pick quantile p = M / N + err + // M: the number of nearest neighbors; N: the number of elements in dataset + val relativeError = 0.05 + val approxQuantile = numNearestNeighbors.toDouble / count + relativeError + val modelDatasetWithDist = modelDataset.withColumn(distCol, hashDistCol) + if (approxQuantile >= 1) { + modelDatasetWithDist + } else { + val hashThreshold = modelDatasetWithDist.stat + .approxQuantile(distCol, Array(approxQuantile), relativeError) + // Filter the dataset where the hash value is less than the threshold. + modelDatasetWithDist.filter(hashDistCol <= hashThreshold(0)) + } } // Get the top k nearest neighbor by their distance to the key @@ -169,11 +178,11 @@ private[ml] abstract class LSHModel[T <: LSHModel[T]] * to show the distance between each row and the key. */ def approxNearestNeighbors( - dataset: Dataset[_], - key: Vector, - numNearestNeighbors: Int, - distCol: String): Dataset[_] = { - approxNearestNeighbors(dataset, key, numNearestNeighbors, true, distCol) + dataset: Dataset[_], + key: Vector, + numNearestNeighbors: Int, + distCol: String): Dataset[_] = { + approxNearestNeighbors(dataset, key, numNearestNeighbors, true, distCol) } /** From b5df40bd87656ee71210db1008ce8b89b5a777d1 Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Wed, 20 Nov 2019 23:20:33 +0800 Subject: [PATCH 0674/1387] [SPARK-29894][SQL][WEBUI] Add Codegen Stage Id to Spark plan graphs in Web UI SQL Tab ### What changes were proposed in this pull request? The Web UI SQL Tab provides information on the executed SQL using plan graphs and by reporting SQL execution plans. Both sources provide useful information. Physical execution plans report Codegen Stage Ids. This PR adds Codegen Stage Ids to the plan graphs. ### Why are the changes needed? It is useful to have Codegen Stage Id information also reported in plan graphs, this allows to more easily match physical plans and graphs with metrics when troubleshooting SQL execution. Example snippet to show the proposed change: ![](https://issues.apache.org/jira/secure/attachment/12985837/snippet__plan_graph_with_Codegen_Stage_Id_Annotated.png) Example of the current state: ![](https://issues.apache.org/jira/secure/attachment/12985838/snippet_plan_graph_before_patch.png) Physical plan: ![](https://issues.apache.org/jira/secure/attachment/12985932/Physical_plan_Annotated.png) ### Does this PR introduce any user-facing change? This PR adds Codegen Stage Id information to SQL plan graphs in the Web UI/SQL Tab. ### How was this patch tested? Added a test + manually tested Closes #26519 from LucaCanali/addCodegenStageIdtoWEBUIGraphs. Authored-by: Luca Canali Signed-off-by: Wenchen Fan --- docs/img/webui-sql-dag.png | Bin 343391 -> 74288 bytes docs/web-ui.md | 14 +++++++++----- .../spark/sql/execution/SparkPlanInfo.scala | 11 +++++++++-- .../sql/execution/ui/SparkPlanGraph.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 2 +- .../ui/SQLAppStatusListenerSuite.scala | 7 +++++++ 6 files changed, 27 insertions(+), 9 deletions(-) diff --git a/docs/img/webui-sql-dag.png b/docs/img/webui-sql-dag.png index 4ca21092e8b397fec9657f64c74b39bb3862b53a..1c83c176da325fae542248fe3fdb343b0ca1fa5a 100644 GIT binary patch literal 74288 zcmd?RS5%Wx*De~GAV?Ddr78-jG?6O3_m1=`O%MU;HH0G41VltoI!N!mC!jRxJxG8^ zuOXB`AR%PO|D20+#@XlU`}e-t7mSrRV`aUo&1XKd5pSTUPEE-|2><}7H8oU>005!| z0Du%uaf9$p!N%4HLPO+hq^<<08E4xiJdnOt)Kvrk>Qkt$UXc-=Z@$y8@C5*9e*OC+ z8uTi20s!#EnktGfgKQ6Qpv*f^|Lv5#)VFLSsYO9?AFb-EdRqXz_Uj+t78qVVYqTZd zi45X$gSsBPes#yxu~CCl`B`56JSR&kdHql-YKSA&)gE$^d0@tcxW#(QsarOTJ*j89 zDwTZjeszZ7Mp+W$fHxODuD(AE*)CtHI4PCi#wI`KP5ZT_;dQq+FYZ5Xm9 zbYoY0aM10f^lB`9_B@w$=Y#ZNi3jKKZ-180X1qIpQXh(jz5S94x(@z+k%|c!pQGC8 zMzA;eoITxp<5k4h*k$WNS*tsJu_`n**=rbbF3o^?3JNq~6);~^Y+b&-X^90}+9Ux+ zZTB2{!I-sE)YT9}6L!Yk^^%vPa7wfLzB3P`KIIUDWAwu&Klpype(h6o`ctk0=NLGp zY$vuh+@-|;e={jSYdkgG>-6Ge$9>?d8SOXAEqx8?NWsVh`TT2k ziV9|o1QjTl@koXykABLB{Sp-y-|DhPzmxs;W18w*pfS1ysM>nWI=GvR-l=8%kSKb6*}0Qj%Po6aYjkY; zs#u;p_~`ViH+HA&tZwm~lcPjgzta8ht}^Y62XvP#Dt?fo5OSS+O}Mc+Z^^z6atP3W zuB}a(EEqhQJq5oA;qLEuq@7BGPjM>DU0r2hl71vHfo&k&VR|i&%X3FR>CWU28(V8T zBIVB7d3P6}%kEpdwQ`!Y@L&>~ax+7n$w{0tN@gjWi2+qzYxNa)XzYi7E%mq*KfARd zwR0#BPFjL=d9%poOkLapfUlY5#Cp6%K;MYv9-*|0p$1NG!&4 z_s98pz;Q>sX%$S3mYt>acLOUw`Q3Q`!=n$DNh_^N_;W#)a8*!w@L9hhOGj&pC+?Y$ zS#6eFi1gVqmP`txROjEXsOMr6E{n>xrOPnOD=o6SGuY%ww(R6lddk!cU)}z^d@;be z3a`z-EKzp=wK2WM)fO-zF)c09sd$voQ7 zH$%XM-odGARCY%ljkWs)k+rubKhQQC?lWSfd$TKHW$VdAQmB58<01RwHAhjIt>x5G z52EeGcDYgQCi%KjyU=tVu$&6|sIAsEqc(mB9PFHV*w8Y@blz96+@xq{mecyS2pb6c zX~D51H-^rAH<1(MA$j)~cla?^_i6{LYi}siWYm7&`p`*#QxDoej4=9gK3rRd)(H?H zTAr79al%&Jy-w5MXeku#@i7;Qq-KURrifR8(+@*79?N12={pleFO^b19KsYJ`1tUr zdXADcs5y_OEhZYu5bihmAkO9y-JNd8I(S1q%O~qp%46F8ck&_fNow z97-67>1AQ)*^aUw7N|ej2ClIM9`kQ{UH~l%DR^ONl)Alq!|T zfZ6VDiAPqy)ZCJ@U0Bp%d+YQ9QuP(1?57p;bh@vx;>eBf{Ly;iCbi3az<+s~%IdeSY}0uNPA-aU zt)PeTaqrATDc{}Nbk~dAGgM63vCV^?Uod2$PTV~W{DL;0G@iY>TNCu) zE^g77h{-QLSL$srn_O2yN6kF4^!KS9-o4&G!+pzC#uyOswA}a8jF?{%ujCFfuF@9@ zN>9iGMW`jw+0;jb$OXpbUJu+&0To7GtohO~&ZL4Mn;X8!KyIcu@-W|8c?&Jzt3V_$d_da^p0047_uOrV+Wzs)p zou_kAbNV8)*{wJWI^}<)8ClE4`fx6)b;>s?2;BK7AKoJ+SFH#UFzww(;h)Tecw{m5ztDk6c@;7SE-BNp%V@0&=}q_R z6glLT8V*h~^B)$#qWCkHp6*@J&ZF%@6LOOb7HR^}ilqd^wn9}gJ_=CYrrtC?lg=<# zRN}_im2$DoO88Gwo9&{-&HN{a88O{G`#iZnO2J#Um%ibUv=wO^`m2>^%V(Te8!tNF zQWX74wf=LrN0EWWcX$&jUonN6pA&3cuNY>=hIpby5yHlf)< zj60`7>ehZpO5adBO^MnTnMe95(~Zg7TTh-!ammMeV$l-jv`!{HYayNb*0nqNR}b?F zxn~$X0Sb@&nHH!-ciX+;3eYZ-?$9&xcsB6oc%yeJE_Z6fw;uynSE!rTlo5V2v1XIi zN=pHpbn{N8_woh3ieMan)B{1!MiHf?=z@tQExwX_4{Qe9=e82NGCQ1QUmsd~kzUTc zUMJ)0e!fag*U6<`-oJZQ_T_D6WN`f*nXyu0-$m(N@z?zA8vS06nIi{fvpb_rgE|K({Uh~(-`2-3iNnj?%jJWA81G+os-fM0aDh-TXczXBwJk> zQXshOet)WA$lNx^IIybdErlTejvQzvuBWg za!u1i6A=eJPonBO#0^-0`fYjLLCH3^Q!Pt-BzBieHtXxpmX@hd53(2j;*TiLxpHvH z0**FKlL`EKF)y^smRVvq%9ez#ek;$LBOg)AW5`%QhGT)YQ^C?_^3(dT>q?1NYS7rL zd$~{F9h(5hNnFQ*`!FBaV~J};CMx%yYWnVc0C_o^V7}K``~Vh8Af||9iDt06O!q!o z=7trUcG+{YuE?#uRz(>+b!b9Ab(4JoS{%1WJB=C}hc z@3+}JJVk>#yx>qhGHRUPXIlX)|1Te|<%@=?pKt)Q;1|Tp%`@$D#fb~}K8X)*u2Aug zv!cK$@b;498c-h`^d*yAj^G&8sW)`P{5KRatYD>9{P zn)(29e3V@F0gal(<)K~#fl&Ow$iY1MM>XlR)djG?1FHJJ+mpkg*x{0I*1py3*(Qa9 z9HCL8ceczRr=+;!*B_Ls0!osumjw<Wt2s@y>(99QXgrS!WKaLYHYX%!Uj8;O zZk^5Y-8Qouo?YA8tR&(T=NV)T%BL9y%{@Yv5szo123rjv0+imaHrkz#FG3UV`d);| zx4^VS%!xq4SI}GnH(U+CC{;bNTU&L+XY3j$`Xk>7uBBMg^5!jV^ITA@OJiNfcp-lo zaDp9p_f9wG$vb%(TBtsZSQBi#$|PRWgB0*m6&RgOY8HNEEPAv}KniO}B=|xjB~a&i z#0zL*RyAm`H=DI=CU;~v|FgyDrM=KHWZ<=|MN{=3T{L``64sB@`SDk<>vKmy*tZs- zWU}nzK18z6G$=`+=GfxsHx9LlNUt)Qnn&0PGC*fnvuE*VcM-7ccl`c7nJwPnjX^cp z7e96pty8ZhgeW{tv(dl=xn?yG#o7s3)9M6gN$x%$+ZiL3E21XW#=H6}+HaEcLjO1+ zeKytHnhbo)01$yOVvg)S=^*p_r7mG`UGFG*5-dBwq_nRaIFmsO`gRC${kyV6*cNEAC&y;i#naTRX&BDm^|`v(^g#f2XGA)*>hTI+IU<^G z_i%!Q;O&KK^t+>pc}7xWR# z&%foi0#?pc_GmpQNv)ODq{S9yZeGiOD+SuI7c93fXbL~yIN@{q%}eg;w4uq;7D!Tk z^_Li(eRIot462+{#D{Q{FQo*GjFtKI!h*_#)}nn@G@R|@-HVgpB8+@kDkJFe{7 zsquXa`_gNZ10sN^NS!@VnXuV7_4k4?dRX8Lr-C(6rZB-9R3>J&<@$S0&-@!r6_yjV zS@p1m2p~o#=s<7<-6tUoNNE4p`icHi&y*8a-FixsacC#DV}J68?dJ%y54(cbxUE+e zYtSjDc7y_tTaXC4-$wK*;d*$&mo^wvb;TWy&9Q8izEL7H##iP6b`0Hlfdc}<5TrIj zX|V*$mwrBF9c6%3UbEwDNFsu3Ur4NO$G{vH_dS%~Nq0lT7vwxfn+xn_4&=!vytrzl z70vT9*4_6X5~-W%S`lRy{1B|W8thp0#wm(~z!UaVS_T#*jLE^TNS8GtZr7xo8^atO zMa4^Ou`6@>d~Z|#`XAmcv! zKVa%M&ae{6_W}KV+8_vdnRKfp&vf6*OHW=JxM!lm3`;$hFYYA)#-hF@`U4<~$Zs3@K-71h(203ga$bui30~VFy9s8^OLQ zr&hEXXuHe6L;pq#)Xdha(8LO47kn2;{#Ih=%Hvw-3Q{YikNakQF75Db*_O5|c}ZP` zHGAD`&>ks&Ax&jxr|E#odu!GQC&TlU(MB&aEo~DG1(2Z%=92UGAJpLhqqm>4M6(yK zX`VU3I<{V&iDHK=-=xFKN?jn>p*DCF|8jyGQ$z+dN5i^R{cs~Owz@GCt_ zX3Xy^w`NxjbMh-(U~zvTB6mnPF~9t=;MKa$GHqJXadK z={Ik?TVY>+HGuNQ^Vm}E_D;XQ&~__osXTkVwU<@CUt;+E@p0_m>mqAbe;L^aA!-jM z8lBGs|FA8YcCt?a1wU=W8{}$jwP{n95oa{dOr-$;TO%fOf8|#T0&HQ9`a?6gre2Di zM?(m472LU$5nXppZGrUK$5<1!cFJxVbtY%DWOvlq$CcLJ(-*+c9H0pIm#D_UW-IPA z-o&r2q=`j+9m3iLE7k$~^VZ5966uL|67#mVu7{0Z-ZAoSd}yk@b66kiRxMylqxB3) z@bumtdL(-$R#?OOOC>!cQ4VC571Bd-TYGFLvG?|)l+_Y(@AsQ!>mJ1|fn-;#N7oG47k+td*CA!l+siw0%o>W+`|@hm%7M2G2fOp$skELA2tK&Np}0?5@2i9u zSVzlpCM?hbL^5Xu7Pzxg3~iU_sHeHA=$s61hF)nd$Qj+rmUl?9c%8=-bezONH3O;l z)Uds9*>AD%R5y1<-j>LBLeTAdU8kjt`^3?_Rws zrG>LWs?#4cycO~}co{bjew?aB}SzkYU+lvA>Cli`&05g%OTrDHL}8IQ0vz! zQ>2}bsU9=_VEweZE-XRMuHabp#Lb36?$5p)4uWLJjZH>r23!e2%K9~}I$D}p_r|N< z(+upkq0RRsUbeI(FKc_|wI;H*A`4%~WWT^$x7rDkJim1u@-cs~ka2 z8ShdJ(J%T?O@qHlOW?jdW^MiMJd3&z@IFh%k#z=gKd6~{mR%oq*^yOq+-A1y3U9iL z@2Hy~w$7?)!z`7`2(qkk-;`$dVSC|pYv#1>F9kQGFT6ke=h9R!h2c`q*LU?Du}jZ3 zqKo2N4Q$D3(GEA4$7q$@kjxQ04{Rdm1fS^JkIt>B<-;-FgI_Ohl*2K{ZPCkTM$M*u z)Nz9yS3|7%NxuAFvCxj)5!S@6DM}7bKb!9pz$fd%3r=mOHTzuRwL=QI_nX=p5<9M1 zdmZEWZO0d!Z(S+M9F%MvPtdI2rum*LA`nGp70=hXLNdU!+59Rh?bcohb+>7>iUo~u zTd&ddQ&dvY$TgL$?dDHipk?+IOpiC2DB_P&2Qd~bD7G?4B(iu`Ht>d2qJ74R}xU-wnW{>IUihlPoPG@WNoW3U3W`WIt!quU@6{8`DPy5&Avm zJ73y9w8zXfzamddlIN;{mbstpHP2!-y>phlFy?XV1SEVzE%b$q#^eCZVT;-qGH$Z{#=P8}w#_@@uQ#-jwPUT*-e@vc3Ymm(t4C=F#OPuA1Ho3|9akAXi4vEU#eLzwo<7vb2lACluO6|;tOw|T@ zC}s4C&${yY<4*G5k-^Wf zIdvEfgv4{pdlC0^sPsR~Aqnv`z^B`P$X@*W(0KPh4W2!rJdl*#*tmbzU3hx{O82eN zq-jvV*2b<)w%kmX@QO*g*@39^IsZ3ydkJJ!hJCXsPk|-!gjZ*JJKe9do*t+GNKyF4U?DMv&E3a0aRk}C5-TreERaF#EU)gH|u!aoVoWOSD?r{PF|fY zCd`m|JISC##X+TnLdZhr(=WD%mP{~wSq13L5mQDoIdoLVh|~9q{HMoGo{9#rb*o5cV6n#T#K~EvAp7<5%?C0oQQNX9@tY4+Yju0yS@hX> zjodNjTr?AOV7Hd}4xM3Y-wtYfCN8AieD~^+ZkQQlZpd*W|E~-o9_m+K-(&ilxAXDj zR_Nh_P4Vtu?$HCk;>^f5k=|sL{<^g+BHiX42-&2OP2NIx&<1C zUal7KZy8RL2sk z8J!@ewLFm1UdqDz6jKcYqRw=;A?@Xfo;9)gAAZ(&nYkwN3RC_YPv-`)G=U_=nT3Cg zTjAKVab&B98e|334?niE6>7SQw2Y0M^oW0eM9KR=$ zAbC$JYJp}YWb?^WIgj>A!nG59)a*?^;`SuaYM#%^(Ivd9kRXS8_C}&>t?11A4@;s? z?B*18=i7Ea(^^>3Cfa_D)LaQDL(GNNR4Jm|0Dz;J=iHP3 zx9p73s}lAPAbRgV4OKsCvIyg~TAFQ#h6~+} z!tB7d{s@P#-Iwq_cYFY2a+EL4YdSaF1QX;rahB|BzyGgw zOtcFM%qU{uq{wF8V+uOWz!;PA*C@*u#&BK&a+WG=LC5>@g>$KEgKHSj?uuPrDd<$M z1DQ85bM7$eu6u9_8F0&5m_Xydxn5%7yG^=a&u&v8azKQS5z~VAQ!Q^3Mz=PDi~3NL z<*cb%S(7O(fXGc}i<;{ax)O=~TCU8i@s!<*tlC0&E9DKL>*Y$De9l-;;8b)x&9fD6fryY99pR=>$+Vw>LV zQlnaeF^LIXDtLq!NNEHvUu*5>F@Vp#-r;;Z(n?%=rPQ>Z;fJIfup?}q=5mxxg9WoZ zv~K7n<1N%uMiU)sQm!R}J+QGpImRAgJpIQTDeTOhwYHnWT}n8S$$wGPQlG*J zcs~@bNePz(Y9kLMoMe1$uCZc#cy7{!` z(8bdCuI6fqtMlJw~Vs-zqz?| zp>$t(X1DdvJGf54`av9AMld~-He8)oBVVxgbA($nJ zuwSO&!`dHH!{zt;&eYuE!D-jV{ES}3DQ{=j(Da4!i>JC6*e7q;j6NlO_+VEi%tQly zHjkcBXVbP!I-8cardXyWWUGkY;A;yJkR5f{rlp60<N?vh*P95+6y`Im=I{vqq_zEMLn@#Pgw80g8sAdl(74` zZ6}z}>s-D4l=Swm*J-k2YF#uCyuXTgJ*t^ybTmSMGhLLQ%OD^(wsV7xVwy#JLY*j6z_e`IM2bPBvx~Gk%`H9*N56m zoe6f#AX=O0jfuFqA1jGqkOXpcu!n+5^n>{&RwVhUW;v(2c7luMWfm?DVHrD+rMS>uWLJfY|uK1TBk%iF(NDqB>m);aBYgL_24aj8t!J!(~im(+mUmHF$ zub})n6}O3PD&F{M`tq_x=I05xbA%*D2D^5?)5A+nn1a>E^sT>fT*fQO$)V`fup$}KAUc2_KVS8kGmxV32h_&|t655Ju9UR49t51BLUnBRxQuuQy7 z9WRoW6@*``v9Kn`T@SHVJ$8uW1LQw{V)ZX83Si$r<@3+a327Cn2;t8;D) z)J|t=tyRH<&|CZ+Ce|3W1#O-D3eUAD`bzCDstLjQB$a$6AoM+@|KPV-pCw9QElX^o%2d zV_z?Rq#wFRILQCbthY==HpaN)tYgG&qTm!mRmjQSp&A~z?+?-v+ClDML7Jf#o$7i) za5H5w2o&LGJ9j9}I6h;X*RuG0`J6O6`h#K%f+=JVi)q}7QNL)b-hb%S{pS`bQ^cR~ zE^$o$1(H=ruWUj|1`Isl!{8)#8GV1r$ScC>@W7)E@w4`?b@xI1`lxTEJHMIjiXcH7 zjw*Sbe;#TG$rx2;VQ)UQ4y2aVQARNr3M}RF>ff3oR}J;Vq3otpP?jPj$M=sjchGYn z4G2ygmo9TX;x@RAQvWrrhmi!vaD+GH(#j=~{nu1rHAR)9>Jr&ER9s?!1}+{(HTrv`@`y!z>P!2HGOWDx;`O+S$(d z`fgthY`G6Fl^u+mu05?Pb*{;p^Y^XH*bZ#zyqp1Pk?kH*zl+^BTm5ovYwiW#0#UxMP!QNJ@N&&KOcRn*m~K91oyc4M zcYsm~F=T%gUu?PXi~9RAt(|0cluv*{yN=DAFG*LEcfa62@9A!-u^)NEvM0tQWsIU5=Qw9 zLAV=3Tb1N=f5Kbl$?BfE(-JoC{7{6rHg|~s3qwiruYaqcPx7wpFc#M{NFAgmX?>+K(w4HUz~;#PJ%V$P z`+${-n#oT<369s6a_Wp@ch6aFjxxq>1eGZ^FOvoy2bKGXWC86U-}+_sr4Nps818UG zveY5nQOm2cFGUe^SOZVD?j>*7UimYcDV*U7m|WJeE*YxW*<=DY5sCQ?=5a8ZD7U7j z__t<5371H9((0_YF+Ci#Y_}Y@J~T)k&C<{e-@N(h6||OeU}_&z(p$DE*DBzpZ<+oz z{}jpjqCnN4R%kv4e~(*gc+X00*^WLhiRv7=DC1an-Qrt!oBWo4*73`0+0OFc{;N`m zDUYolK1R0U3=qGNfPBpDS!lidxh z5kZ=evvb!^?#F*ye`7tnXpC=XC@A%YGPVck_=#QAJ{g+K!Bz5O|hC(7V{SpINyV({zjhMTus}c9mE^b>v z;YX>|Vt$SB=gW9Azjaw#s+v(P2lF`po@6Pe6P20a`^+sR+3V%w-!;?TB(KM)A2?m> zPhE{cU-Ige7YxP&Pu(vI^pb)r))zDbh)DgHf7e^shC9TKzku4@cy~0H04W?~S$p+E z_}@$@pYH3vL;YY`_x-wC@lf&ft}~^qwbLMm3};lwVl|G*$>SH1v5i_|@UGgHm5duVh^Z8w*2R6`HZ+JQ%kI1J+$2vk?RC*s zLiknkr+S6pB1Wd62Tf4P$ zH4y=Um9Z3hZ|S3FlcQ_>A7Ld3_@zN|@3&iV>FH9?OO9nRp^D)6R=IgK5dE_xMV?!A z)fEbTa<&Jvp1NYtW8sjO;akBO@MrVUZLbc4R>nx1d=>TM_Vp~>SIDEC{eT**PE0={ zNGbPL6)pD0mk0rVj?U2;%}T^orx(7)Q)q&I&W*0v1k%%%V^`9d3wco@X%fmt{6Oc< z$zLPA>~(vIFxoK*LATwSq!;7Y;@)hL4z*j4rMIn*ex_y5PknFg_$w8{eN@05Cz`Cx zJ@riUnTs@mZBHw>)YVVV1P@rGSDsAzhuk{nC>$J)0yT9`<%75%=o@20rg9X=DZd5Cy#6C{Me9| zZA-EPK294lPm%`8{$rhHVtgq%%z^qZBjov``JhK>PD?_9MyRFhYzr4^`f#%g9z5Lnia zs9T~9rIkZbi;Mj{MITQ6hL+xgYd=rGB30jnXouW;cZEomfaOOpTZ9XQ*H|$Ab$D-? zGzeT5ks`JrAC}}AU+>Aw0-FeucvSW9?ABy^&KH-hfBKVnwe8+9Sn%PdwNMmgSXRx# z@6!XmzC+UA$NpyG<7{`VGu~$oPVrmRKmR8^lo}FS2@szp7f{o;DkHa%GOLtz%?uXa zJbXeRn{Ze78~=Bql6Akm*S$Zm!)46K%sg-AKEE>rlbpJeILpk-<$F^SsnII>VmJbb1ZCq;v}(S4i=1eb)e?XgI1w;cp8jv0@O86do^o3{*G zz5K@Q(H(2Ms`~WbfW!RSGwkRjIT{A-zbRUI)%Q)f0FCS|GWZ&i&X=LzxcDp7<2te$ z5aJ>mk_#erZ=>e#|ACM)@e?}Hga@yv?26wjRf}_pY8F*aj^17|(rx@CCQi7`F0!XP zTcOvV^h($zUs8R&jvt&`X&QaM{#W>~Bgh=a??g>w-AAxb_WrL15Jw>2gvbB3lAHh3 z{A|*>iNbdHZqL+CHMF)&X4#fP03shi7>UBXtIuWLHR6*Wo0>j#$zKiB?g;gKj6~A% zhCfra>)Q3f>7J5Al6^awPKm#sfap!J_QOYOAq6pl3=$fEip|e$R=CrN(w*0}^xfDo z0Y60T9$EfwWRJ@&(g2s^D~Qe%ZMd3WqhzUvnUaNKh)$Q3s(yUdu@zjS=`Zx1#~7(g zVUNQtCMRO5efjJ6JLuZ(TWgd?v9(XYEf*~7?$i-#q{LLuS2@FFR1v;bI~6Z5Q~;w7 zwF2OQ3rmuG`pBNB*~@SNzh*4oNq;s}abUkB6#tlDrR$PNC0PdAQhvpa?;{ zC;-ua#2^Ixs(wI@4BdiXZ10-aHse(wWB^evI`e1l7F}nzWql*A`zTR-XmeABJ|u2l!vSuUAmrOERxn^$^%CNJSKlcaZQQgy(~s3Ty2kuT#(B z^O5KYHpcw~Ah6|~oVPT2Yhp?#N8g1YK#Wjw#*$LcMIDF7VCD@guYOV(mc6%AT2bD=Yh<`dQhqq!Rjo;4 zI>Eo@QF>TJZtBx%J;nJ#qJZh{Xf&}^*~hwJ+IaGW=@(36p0aZL58b2*NmL|+H73uy zXY2)T)|GwVXDQ}&@mJhr^s=BhL!g-vj8Q>yGY zN9*j@ja?FxjyN%`KK;o81*m)E3Hhwt$Wj|qIJ#bz+03EJfX`s0#d2h6A_Vy2MrFJw zFCd?TaQL{tJ&%}}_KQ`bOUpkR>i=fy^< zuXmRFeI1%j_ORW4!tR8~HerL#^8dLgp*hld;YnY+n`2f`QaCbKEdqHiECoGN4dm}R zWgW^L%!Q~(tb(UdN5J*>ZC+u=CEtucPz5RwB=dh)RbysBcBx}`NknMUJyIeKOrZL1&5yBA1mgl%0=Lsyk2FO-Z8U zL=GFz%7<&B*14q`G9E*mcys0xiCk|GK-;P3r~CG9VE$8|{T-rxqT;gw&%GbUFQXW# zzEe=}m&q=&z#!X9P_nk|_qOJTi?ZE=HZ?1ppZ3e(4JJiZ&(V##Q#SLV5jkPv#B1f<8EIa+YRANxaI!B;P0XH-vROKR@2H4 zU(%UoFPgoRcm3f$hdD5ZEP-iQ=d#|Cd;xW*z@<7kj z#b9Y>wepk`QK$IRp#kE}A6bS~+Okc<6q#+iwTo?EZfqh2r~Y!e-xJB^-CK87^t9y; zqDrZNcN~rcH#vNQsSK-j`G2#8$MjdAN9ovw5sZU2!P!Cgs<-6|J@+_U7aJ>MwHlHN zaWGZ+3y%B4tLNUF@yZMycJLbTnaH~m-R2Y+a+kN0Hp38mPRyuZ(b5R`Ea{0o9Xc1Z z90fYJhvdEJ_;M0Fqvvg%zO;i;bV-*?0Rs2SOhD8MB7}ljf2Fv7DFKCcam%umen%`5 z&^TAUUG?gl*M<)3W}og#bn{^XIQ)T@s+&oU-1z#BZcLy`BD8*}(m5gg4K4kB7QTd7 zlE7ehG}Ds6BKU!2fBH7Ej-reWMQh^`v860{X>e(fQ(D$T7TR(en-q zFwEk6smW>NP=NV86U*dd{W1Y@lLglN#RpMHeImkH8Vq>FXJWCZX9vrTYNIGX{1L3$ zUpKj8ijX2`sm9pp_jHD87T~ATW}SkYe73V*`_0<%A7bKmCXNl6&i`aQZGZ3Ra_|@c!_nwTBtj5tBDk6xssFaO`PO2S&tZ*xE1_DcAUI3JT&Hz~RIE8Iuh%LGX0}vM3Zs% z6Ye_QNJ#-=^GbKp^9XbjYFTzht>s5l#sn_m0qLPY*Ukww;SZ<0@F7$+<@bK-eB}Dj z6E%8S(Dnd)()a{ZenAp7Bbg1t#3G0BpmlcSXMD7expplbI<^wNpyWkQ`__I75IIIj z<;fhox@70uz~e9L9Zx_27lur7LV5)t!u&sp)40;D_~>SZ5dmCo{GSkRBCL_+JttV^ zfgK>?o8rh91xT%Qpd5}xAk>|zd)J>p)Lou|>||6RSN#)byoG7{ay)p*@x1f~ew5Mc z+CK!{K)bIG7`6UK3mE<%KD8cV172|5>O++F^=TR?(ioKp!hXBDK7*X79eX8);X6jG zc8sGzd~lLyY2V_BsR+6D{%d1q+dBuX`ZI8Lp#&&Jj!du-_(B$hp6X&HXd$Wm&w$%-FjB)Q`7A!CxBUI#4rHz{}8+a2E1Qu;YyHeRNaA^(mspUU$IA0!AlL7XK#_PA~v zq=p)!GWJ`a6KHBOafTBCB0{|1_$26opQ|GCU`nTwPt4OIbo6?DP1rqT!LQ!&-@IP~ zh!}`Ks)p&Wg)+d46mStedeIr-I;e93K1A6?w-Eza$ickF-SJ4n{BQ;P(%zUn{i)8C zDnj0y0!c5L2do~6A&ShoA&=!&Kj;l1*jOB4UTdLu$~gajB+O-KS2kmcd}~E%Pzys< zLf~rBW?_)+_t%9X#I2k?POD7ywq5=NkM9o>(trWq+yNS96qdji2$kzu=?qN}G(dMx z9{`vvQ?P$6Rm{a0+mDA~%k{4$eA*WJ)*kiwdy-S{{Od5JV9yGG##Gwo3ACBI%sy8Z z<^+eGI7O=U;ejK82H1dOmo^1Q4vFJnsrmiuaC~kK7@;C0fe?YSLR)?8_A@5dP;6-z z%i(aT-oa7CYSz)ihR~OTTclD|$^q!H^)NKZ8D#xS_3i+Z#Oh zaL(_2?gXt7YLZ!Q*wN^`vypXZpuaI*67TW_V@{WJwCh-iDDuHQJk<{tn4Y}OROh?V z-x8G5tn9b(Q_(Wj(R(cZ{A}hvGBhksD$t))?mgjzstI;}GCd2SJb=LTuZ zc(&ZQ;QwImt%B-`x^=;gTL=~)xVt2{TX1)GcL**)0|bZQF2UX1-8HzoyKG!~Z(2u{j?vp&1G|r@!3>waCD=l8yhsTtiP6S6{nGJuO7fzJKXY5Smv{Z9&*6I z>qf@NlNu%aZKpHix^T@{TjQ;A94%Pay5vpqzyQcvprIQ`rhY1O3ko+#t7tn}$vEJY zFP!g9JlZ-skPmntV{^A7?}=rIpOtD}UbC_f2yn{}!k#kCON|ChF&Yj^?GNdboO6d1 z11&`JyZ_EHiDAv9fuPEF3E$7%92QAc4g!T-gP%d50wiF_^y?+`%9f^#M}LA=REc9h zeI|_NU9KE3(B$M4l+v(9aV}+AppN*_Xx?Nl2$bnXV7gepM zMF_3E7O*1w9b5eEZf0|ndoF5~d$~b9q`~#%9BaOW4;F8UNbIuy+2j2)9DjnybD*mI zQ!o*A18x#02s9g=NaOFL?{@C>#HMvOkL5XYa*K^DSIsKg>2_v%i`SXcnT$|1*T(!M z*L`8Mi;^fIkxFdR)|BH*Ae!IoL!mF(Q(6+^@b<}gm%Iu0$D>Z0XWGTlBv8UX>_@YeMond1L+Ob&5Xq}UP2+x48mIRYtS<-#m2cy&LUXdsnU!(laQ!HRGEGl6|JcFqJ$z zD-INV5^r<+m0E!{Ij~P*T}j-Tepd9)mBG{by^bmY9_YK^0lj%12?y87q6VVhjHdOv z{DmXJn?9}x=jMB{Z-SN9`6b5Rebdr5WT6JZ9((^dRS-EGWE##kY?G3P-cCPf2+Kd?=kr&KWoc0Awr>AAJKu?2lmqaU zpFyjDpvzwjt6s76t!xNs+FRv9IpinmRrQ`(>$h3BZ-IsjG$ThovCq+G=$I#&C$XJ7 zPHZ(y;;rSc(Kh(Aqt~&V6E~0`Ie%51T6WToj8kbHPJ-@N2Vg8+RDw5pRoqPgw1+5 z^UWjm!@+SCcxSKHO#<^Dr*T=-=qF3hQ5yP!*EWhxk8SDG8zysCN#9oUgQ_DP?_5cr zQwFtyL;9e4;QH7Ee-I&m4&xNfFCn}Ifqa0`1MP6FOZU0TPQ^`=UXs#8bMsM}nX1Zi zbfn{2CZ<|E-d9#33PZ_i%)=KS(@YBuugV2@RIl9qu)9`l*!Q)95Wi ztv!76sOud2$&Nm(9q7nB*9o$s*-75{5k2@T;wo8zJkF5d;Gic4kinCn0ZUN7eDiUh zwpvIwo~pGr;uS-SLC)Qdqi1x}))(A-I9?7lQ;2Fh^|y5o)sIoJ^V%Os%;bm;|Dd@t zjM@_AWzzTMPXJtVkLhadKE<=PR)=NgjREZ)OY3hpm+%G)_3B7>c+OHlXIOv&%}$p` zVHjB|b&|bOV?dvDxtLw$On+~PPJ1rC6;YcgIyaGfH@BAQAdN$gLDiMtKDvXRNzZ%e zfRp54_RUtg&Lbos->9o%HO2|!NjaeRm4se{3SC`0?%I%QK~&e8LA}Enr?jycV|kKS zFAEz*8SV(;h<1HfkK<#*KRMn<)Wx#RBX;JY{-%>fHtT2)@Jx>rSG~-}?Z{xSk%EK3 z7y@@)^`?`4SQ3^ZPFvlO?tE)$7Oe1q1NN5-b;Z8x#vVs3tQ*ZUJ-t1IR86rn{O{z6 z4qda`EQ(> z+g~krI~ePeUJ(SW#AVt`gbr&qPCx&CV$L3mI3o@F>}UBO!T8BUhJ2^k==$L*t+WNHobH0GTrDK`RrffB^Z0$(;IEpClExh8L(W9g@DQuQ zbscs$uFY9aD#tU58`(#X|Gk7Ls#l2m&Mdy{CT&-4!YSYOr!g*UunJTGh(oX9c+a3w&QjRSvP>=0*ZRE z4FcpZWupA2;8pdS>hUW6O%WU)h>?a)n^Rl&TFL~pM*TBzS<+gkm)8=X%Hrk2_Xb9H zkQo{3B7KZ9>67K}LYW~OSu~CV@)N|i87q}bnYsttoNX(Nmp-iUVOkT@dxUTB{v!`$ zcPxZh9#&gv1>0tBVM0*gr12v@TFww(o+du|cEy{Box9S!mGQ3rFH4-&ouU@ z2~MHOr|)B@f{`=&>{J#uNvIZl{It2wvfM%f2v2ONTnXY-1;6QY*?!13TcSdMWMQD$ zSyV_Nd{YJP_bua0)`;6k6<`Z5S&aJPO)=2}$s;yZf_6t?(oOVA#IwmPv)^iBhOu96 z!h{Jwqu*K+zd46N%IU>Ot~2O4XrEbvnn&T z70hrD2j6V_@jE)28%hKB@YxI5L&Od%9qFgb;^n>yb&lZwop>?5y50?$AKJx)!3{6} z3-S|#fLzIQAN(Xe>%xhR-jrL$ryjvPkVSh*t!PgxiC-Z8@>W98!L*3PyN~-xLQN3T z2b78mZ!^$jCr#n3NUpgMj(d??RozW&vyks>`^&gBFn^R2!O@HLt;yE-E@=And>drJ;NG!(Z#TY9v6c4U+Wgy5eO&nd|jFclo(^9Ll#ZEu2$9P?A&#`Hsj zP*fzJi^p8t%YAgS^FiId{{-h3FW#2pd*!{*$zu2zSAX4QBRw#0MS9HFy%7< zsaCxq6DH3Lwke?YGq&8h2(JT~pO+E3__LWHX zgxEJJhTpg<(5QV-&$Rg{>)qL>$IYZr)3>y>+I=qe8|?R zoq{{AGlCB^@TlLter_70KPSxiaCTO&5X$=w*4ugalg!pTceRK$`OZ1CWZ^u!gMR?B zKkEhVEJgB{XL!EBh=V8k<1ciEQ9FJ=BHDB(xoYoo=iTm%qm(NzGJV@Koz|QZ85LcMB|Y?P$B*j@w~t-ySy&TQ97iuiY^NZ%rdmz!Mp@aB6w**WHp*3W z&AFd-J8BQ7(f-^j<)g6^3n#=>vOU5;Qn66N_l`kXfGDr zWaoPfE~oj5Nw)#NrB(P$ZRoSd{9<~vdpElxrNU2%}ojy)QvqU zzmB70BDoHhH)DOtS~!It!8os971;jx3Lui5X8afp7?|yaN-hTA9sEbQA)5QV=V9-B zYzfl7aHoA-x4j_oV(BkT9B);SCpEU!NMWUEX$}j`Up`8Ntw#^D*{u4Z-1axqs4ni; zDRA7C#LU;|*72R+#S+Agq}k{s!4*>_hHzk%2&`H!w8b~(zC7J(CKaQC?{haF$Dpi( z<_Nw*qb=Y;rD6b`9T}2c>ki)lZFKzm$X(nZR9J!K<7UH?_KK~Ul*WUvHXvR98 zZNoU)`*_jr+Y8;KG0!odg%Gu{WH#9e7R;1R9V|=z{GGA2&W(-|%lzmO4I+c3lBzwm zL;D6zR-?CDaQ=Fv|?<{5vw1s`QLem*FW>H#hgC+8s62SSyK z?QG5vQCsAsU1$J1V;7zsugc}EVM3#I-z#${XH@KFI-(%{y5pXc(e}Ed%N)|zY*=@J zn#ENVbl(|jQLnKeR3{7jQq*u(a6ZJui1W}dMD3`ZUX*3e>nTHPCEz(0H(0$}!~b!g z>F4O2UHnNiy_UMo&Ct5CXBwBjpQO@LkZFx%Lz?c2xA=H+RmGOoYW(1eQAA-F|G_53 z%z-qe1qoaq%r;L)kuh@*!&4|x*RFdWjvgxaD%moQd7FHg8f9W9Osi@k*zM2NZ7|^h zKF3Vm?8e6wC$ic4N!+(h!{8%V-Wv;{n)hSN zOp|w-9*)SVV@6&VvJzTZNYZ<#nxIMJ`F@hXt~0dfBJ(gQhse45OXwfDeZ8vkVy_V< z?_fAA`jEb7m2%zKfqaU!x$}6EJ<^XfVtW2VuYS170o$ngqzU^}#R+NrU`Adhj~YP} zQNe%|IC=o2;Gx%OFe~#c>A)@Ijk2<;87Zb$)vla=hN%CgTa*^^qqSKHi3AfF@HZG^k*Rv(7@&+-mplnLC|v)IIIoGTQtI6u)p3@?+d zW$ZsUA?&4CHx5DLo+O=28V;Qe%Dz=B+jYn@pZ&>bM-FJmWLC~!62+QIq12*A>Y==)l%t_ zw|=gHh9JC(?M$N36TnzNZjHdA3Za%Qv|lV$)c*nrBIY5kViSrSbX;yJ;&O9{3wN?J zidzb`?t+EeDZ2e(D!GDz+RCRxI&MN?)rc`xC^@zbKUM)(pFe~^09kUHCz$H3ykZCi zAf`aI_c8j+WlB&BtUl3_r+IMCnW+dS*|7mW0*i`4#Xaq27|2O5d?R^$*9?|IPSXiv$8zg^-2wM(d6mJ z+(15SQ9Inaa|9c4jeRl(YvO<~PzjS*3Q-mwTfV4-(tp>avW+&Ysfru0TV?Vlk%yTk z`VlhEA$3Qd*%R;+mYD4KaPeE$&y49=V#^=o2bQNgX;L=R@DZ4yqy>_5xS3*e3vdoF z1_hA*8C&0xM=E;3QlXsDYO}TYDOq>iiIg|JHz_op%UGc7X8$gBytl5DuuB~U6%m9U zB2-n_dzWn{r?{iyp9_9u-6z{6`-TCmh>?YYs;@|bH-JzeQa&~~tPh3Q5dO~?b`c%9 z--kR9+*&zfpcyGt_7Xq>q4q5j?_t{(_gt$l<>KaiFu<5vZV$q;!;fH##{0)1rQwiB zZh@7=#VQ~Zc%Y>cW5=SVifd2e zOk*ssvEy}&Dj4SMbJEVh3vWX3i0!)%KZ_UGyBSY}GR+K4{)O>{hHpd0Wq_vr^i--c zt-j=JC_ZZa>_fVxau8y}8)1Frd4d1Ve)1$Bg!m@N6lhwPc`vkP|kj+J;XrVI3HrnA+&yP2TCM8|A3K&Sa*#nm(b`j z0Z}MVNqnQM`!?0*?`uyW+;`5SR5w{6$erk#`ToL!=mj)d>`Gow1G$ z$(OCIvA~66SOeI9?6f>CuY>Y&yuC_(8c?S!>rLQ!!|+K53cW4PN=G};9B*r;~4{9R?EU+#-VtM~!GggIGY6<26S;%;vW zRf5@VuVDRR#RYnkG5Fswe3=)`9 z9CVV7&FojhnB}Y3DoTE;oTf#aj$dCuZjz((zHy#eeq4@q9z=chr)G*-0fZjcLL}S^ z6^LDj(z?G~*>gFgyCuo6L2#f<9?w{vG4RLzOd*xls2}F%6jeyoJ}rh1{qqd$PkbOTPJT9 z^QF)`LyMxiHl|Ju0kJIvGO~g0K@Wb;IDbZ<`95Yb8X@A@Y<)=?CA8WZ2HCyo(IU@XU`y83lO) zB&9Kx->03x917!7ASB&TjlCr!C1hd<=onKk#8ejoW(NEJ%uDOQ#8DxMs6n#=nFWAc zjflBtL7%K%5g-SjT8op#P&Gd{yVP||s6B6&yVT(afm@!xbJzw1!;L8bFz=rnHoh6E z-BCo7YZ;O&QFTw|9hISS>qxA$-A5m!vEkEsrPledw- zcfLOlwd%l}9Bzn3sV>_xL@T){o;(iXkNfQ2U#Az@R=KeC3}dF--jwz z$#xfm(CRwN-eqo5hNEslAEEDFQQhoN-4QYYX30znHAk`_hNJHS!5<|fq}=5& zbcEApZ~m+%7{3^WIIlR`wiMcyd@AVJUPV3kwd@r-O1Q#hWwP?n$uRe6e%LC1GSd4) zz7x#GE$%`pZ@W00{qTkArbg9}%)!fvQ{OPrX-DH#I?-CEBI)^N`>)YT0v}L)!QJI3 z&P79o^qWr`Z*6kG1JAN;1qRMo0{NzW>Vg2(RCbia69G02B--?d@EdWFCzbHIO_F4 zDE7$#T0v98=FuZCe9Z^VZfeLW2HvzA1~DSN6qkieE#W}ma_gHxYe>LlL2)|ls-65V z8tl2j?$MV^;Z)y`wtJQuG>2X0;OUK>C3rLUz=T3-HACrs@@6i!`7}GHaUpL|Qb5Rho z%#QZhKmE)*5IEf;g=Y!Y=*$}x=;=Wgg@5`@A?N7cHPhG;jO##^41nk`@nz-gW8>a8f6IX;xJ&S`iVYuo6KjVl(*N59g42@^Qy-0@-BOj`y- zufxNijYNh#2eYOAA*NPaBF$b2{RYi^`CS?-3&W_Zc*PbQeif_eUwya;VAX1@6-0;9fQ2}hrjwB66u8`N(s&*&t}|JMM!^8QMjwo;~xeL zqRSZJ6=ggbXoVVsMe_D6!M00_6`ULejtUB@1tr6WPtnG}xZ!|t$J2ea$TZi4`E+Kb z#K=6losg1oy=6LYla$@&Al_@&4F@Z(`dBTJlRxm#L%R^zyQa|YPx`G+~Nh|{D zX+XH=W-{l|PeyylC{2c5m5s2`$J#Wo>3GcGZm|OR>{D;Y8-?kDuY1@K>ZI~d4UI_$66F7QWJuTaGWrXF$4uAEmPO#Pox zbW;@gYtdp&T=3FiGG$Y8TviPxn8r632cK{iLUQ?}bU&Jhm+6Pf?0&8c zil9QBciB6YfBBBaJ~2Le>R@`6a?qE0GXR4)Qt`g&U1EgBy`!cg028wfCZahumio)Zn!xR>%%Du-6Z;o;EQJdr2=K3VR=cl+AYefw)-}_9;&f=M zo%h6;t=AB2-)e%$!20Az4A7%;`C~jw-Do*hHX{PAgUVmO&xD4rL<@N%;C_xGOD-3N zBE~}1P{5?*!K6YgN=Bw(m$La=<#Tes83RpN+kgm}m#P2>Qm_X|GEdUKq(3>ovzCMX zNDEg{GrSSh=s_$H@HS>eN6=YowKFB5M-$Jt+8UW2G59Ie^LzsvM}WrNc;Vnyiwnb1 z|IJ&%WLdO!EH#X{b3^hO%pOoqS^508EH_sC)9t5-O&P&!_c8mkPwr2P=^I0?-&^C^ zIEXv8i_skQNnVK^da=w>QJ3TCC!I@k^POE#AW(_v` z6d#$J=8Hv3(xL5pq?4^Qj^)tEfyQizt8jMq@jgmNfBuygu9hF=#Ih8hA6$Fg;OW>R z1wlP&dAc7HS(W%QuksVWZ9UeE*M-XC%?O*LUjLf63vF@9-Gp1Z(f7=&wdTU*`oqLX zeU&`@OqS`m7M*qM%~WsbH&UyS#9R>#{)lu0Np}d41vhaO3&aol*^zN=J#>2U$E{>z zFJ?bQP$B&qh6$#N8yck1ITsH#@Z4)XWv9N9!&{R7 zBRZm5u@if^5L;ES;&jT6wdVC#UuRw8&x_Jyq?lLbG3KyO z_%5(ydreVy;KcUzk~m{76B28;)~f^mzDnRh5`K4i^Z1M2>|!a;{hWF{0cdI#pqc`4 z4wQ$_IYQIwQS+^q82h*@b>f(p+T*ZQ6Lz_+(Gy2a4)c>Gi5E%n@`IQ|+aAil40{8j zxI+NJd|LEE`0LctP;572X9(kH5cjiOw0;aT!$-@Vu?Tt8B2rH2OA>*7u1{%S2Kr-E za)p+?fCt8MxD!Pw9)b{?oju7ptIT}wuga{)WbBj=0#YIYL{?tjWhVs%9F?Cz3i7{C zeQ|;LVBnH9BS9n}6cX6|>_hD90+Gi5ZErAk7Sp8&CE{?~0phQ2U#XPkj|uMp#4@0) zk?ZX2IUwH(E~`;P$cbI^);!F;reqG2AF#G*LmvV1e`2z~?TXK23B3qj8pDtz@ze{6 z6gL*4z8si0k;He|xc|hpX;$rQbYuKI$~rDo*n44)b{^*9Rn?#x5I+n(Aj1I*j0V^8ICCh9f!`52!scComf z<$QHRf1hl05T3reS703Q#_P);e_lP&pXg0T=;4MWHI6}Yc7%>2$!7l(=NTE zzE3~LafS$_GC3%pYER7jJp_!FK9*d&d2bhRm(3#<2tGRLTCH7?9Dm|ayAda7@okUZ z+`W}~U>^;Y8n|GfB({GumoX~_CXN;GDs(BM)+UvQKWJcuKLLC3N8EmzA<)|OGaWS# z{OOEw@^=w}HrI2c=$Qyl{i3Z0|0n(W8(hNE1iyGf1rl}&abB6uAD6Wdp98Lu*7Bd2 zWO#7G6DcdO;8xk{3H6uPE03G`H9l)yjO&ZQ1FM@FB<((zWbz~Fu^jww5o;%bNudoy z2GhGvrGITk#`L$l4c(@AZKxbZnYDaL_Pv}H+<=4c;oxaZ9feN7|D&R+CH4_}j)GGAXCvGO#zBawcdd>u|i5EOto2-dOngsl_#X zngx2EL<#Iz2LwK95f$Grzf;od{HoYC^VUkahkZbQp53xi3zO0_L(?Mk5xh*Ex@N<$ zRFsLcm7W(%uCxu0IVkB8yvoPKciAdLQV&F9$}awH_zDxh7s$-Bf{i#@Mv`=W)^mwq zOZs*^H^|HMh?H*o-?t4d(r!S7#Sg!cZ~=fuKZ#le{c(F3V;f;v zs+B6MZ^m7#yw0~Hh1PA(Id*1HexUODW}sxZH&3;WxJH&eVVTOr{P$X=tE@g4&vj+_ zOp44v($<_C7)sm4-?je?<;lg{=!Bx3XYW9lKRz#F3~j>hNApF%?2hlDG4^$Z-SmXt zTf-ZcpwmB$r?4bKMx}Bcms9_IXyD_>mzppeILak;`%)73_qWZ=rs1Q2%f|ZEy`szJ zeo?#;^p!%jx~>Q|BqMD2%SL&){0GPkEPGYsL92}x;#P-Z#9jN6hNr*0Us6T9v3=i9 z-()I{lm>=6X--n728#z4z#feQ)`hC>WQe-r$&=lva5-EdcZa+yC=&m{V5IaI6qJF! zn}i-hbEQ_eIVj3)I2&$KR#MG@saqEIiid@e%=bXnh$yS*Rh*(wUP^=c&CvujLAvAf zLBb|WYx<{;VYxk`g|f3h$o^RbkCVC|rr=d&)@3DMQwJl^qGs$k*3M0f@msqOf zEeE|NU;4WezM3qj6N?qSD{}*woMAs^u0x#A~~ z8(zO7d=q;X_9(CD;~Ydvn+EP?qPX>*w>j@W6?=dC*gbzi|XncdDnoAI&L{Z#VIFvUz~_eGU% zS+V(-iovfZ+f^n$(L?^Fe!S#zYDx(YjuXwcdWc{~vk$zk5y2;But0;5PEpXLC6Dh` zQwP6=j}ngcVgHs86TbWBNd~Qwf06fm7m(`c2gC-kE4$N9vh z&7y_M9oJdVp2an~%%_?GQEbEn#8sL-H}n~oo`y1+bndm72u9-+@fhQ;{&H8biYlvA zlgMl|2>%57umH&@M>LtVXI(Gw!!bdvFV!ORtBckxzpPj>XCf=$)&UG~EW-JRX=+%R zxPMtSJ%D6dNk9+&S$in&nVmgRn$ytkuhg@QndHA1F(~MUU(g7e9>bL&NZp<*$GCwd zbLKxlnk_2azbcj4(lz0uDyJq@T@pfPFZ#~2;!Yl=byUgN7OcD|np;+NAR(Z5|5uEW zmhQQey2_GXj6x#PiBny%tLkuS2Fg&|Q~XH^CGXkq7dwyJq7ifm`u7E$e}_Bg=yjud zMlZXbnx0O|R5EqLtX1iIFqugn-gt5^6nLTBr5rlOs|gs8ih<|z6V|x08o>zlKT%r4 zaiClXxrfv`07s#N!6XYbz=;7eWpQ=ZzuksQKMBMC^Xo2banhh2Oov%X8`Z08PFUMe z0lC&LK$uMAhmEucLYQN5q~mV2NvAmF+cbNtKN~6GrJq4l8ZQ?}%D2%1DLMD`IfORt zelR@9>#m|m|1n@CVU_{gixyG*Uz{6_9WjRt_Qc$GHK}e{Rx~- zAI)Vsk=%5&kKFBwz2%>Dccb)xK`fA9WHWWy{9pOP!p#OL0x7tEJ*aA znZ3yi4U+fw$!LPOcc`yvg1<3T5Tmj{+c;l!d?CRT27PgDjANaO3f6Dzfluh2+v!ALLW$Vy2~tOb3L*v9sH)QonDs zx>ddqfu>f+sw0RmqC|GgQ>|2JbGB_-u966h<-|C=NI7vo3>QV>3Gz2Nr* zyLLqCF7ohFAbHlEk-xi?<7u{*OTecx`be}Mii=rGxGUb|#Oe|%pCed8~*wX}HO zSzh>)MyIQd2XFs*)*yjBUyl1+7Idi?W?9T9rrbJi2p*rFJKujC^iOtNcpdP7m;c+^ zmZ4E^rJCp0@z3{$jK>Z*o&L-6mhrGS$S;P3MGxfXmv8*tXd25B$f{`G$F;JzU-;*2 z1KbKAm)#wrch2^mWw%3l#sz&}KJVLHqPZ*(XdG~V`f)q%L~{IHakS5M>wI2&!rQ}r z8{c@5eF_kGfO4(wC7XC+X2l}VOBWGp)P6w(c2Bm(aZg0_$Xi)9o7|A*L6wI zgA{~j5b3;v&FSx4|90lI#x?uz#w(CGDinEso1}P30un8Ge1QAaHmdCnFkoRk?7x1& z4t?7iRKB&YwEu&A(GlUOUSyj6CTO6%9o(bU{;plAxRSsC7~+1bf&^{39!EaO0{JHQ zZTF6z;Z;^buic+*q<9~On z0Gv$!?rNF|s7ZJ2qfgHgJHd#C{$aCpNho&ML~eYC@VAyriFU_HE9<0*n6r=r3ykY` zT~b+1P@pb)7)Wht$N__e#uHP3ql)aeZwkI{?K7{g&}0V$He}oH9ebW_4c@^oXHaW5 zv*u>D9Q6kevUq@o^!*o=#BuuKIaJ$ueSp%Gx@{fw-#>tZO=V?d9NQN4r*^CWzmV}N zmifx41#+X}j=lOxRD<|5?rerN0$$A;*k}tB?a72Wer|T6QY+N||w@ z5D- zbeMv~L#^)q8x}Cn#2_Ud`tNm?EI>$JhLxl9E5mW%JwTKOOq?@>I#EyTJfKbw=nOnV zgJu!oZIoxu7`!@)BDKKUsL}n}K~=W81p;%aKwgFdFRII=GLe(+Y<=S4sdSYQL^6V) zf%Q*E7(5PxX6YJ>W(3`ZiVg3h7P+OC<7^081)pKl+jpCHJ}rte&E>hR$97@BoHoDo z_Mar9`|juG)65Jj-9gb47sID=x+SsNRsM=9?~x0)zdo-#^?a?qc7KEqc2f~2U3=)I z>~ejH_MUkOs_bV5R~y?~`&w3mf4qrlB|>eJVW)CDJL_I=o3>UBx`th>o?VZ+fJ=;# z>>8t$V#0dYW?gczbU=d2z%&GqWU}%4tR<^{*iy5;1-(wa1WTMkXJ$*wyx>HHsw~sG zw;pP6sM{&&Tdl6Iu=t%Th9vgYN3AlozwlR@e}$3q=``D-7%_A3`T2QBMGAr!@N}rk zwj8ndml7(zODb~buKwL`im!6N_b~bLc$ZRGdo#xlObKr!wHXZGcKdpNiku4A*Xy1O z6rGJ0dH?Vaqe>`}&HU7fotXXVVe%vC9Q&dJ8HjYS{NCzbtD9a}-bqCdfT??)-$c}H zb)*7nB9kn>E25NNr&GDLPU4?oA2`RIQ<>&3Khuf}7=-ZSEp-H`E$ShF1ZMzF5VMu= zL;IHN`Pm%DyGOh>`9|KLhGNus1=F+jMFJ^5mYE{gH}c1tnxMOK#is6x^mWc*TiXDAC0QdpQq+?7T=*vQBcEHU+i$XI7l{< z0%2C?s2WHFO3&^YIP5l$>HB5|Z{|Hnwc6|e2QMZ=b>nk!B==lva#05h3e6tjN01=P zKi<-YfQD|O3AJ~v&XVQvFHMf2p)JnB{wn@nuyNE)0;Porym95t%4i;*D#8+ULZ2&^ zo;IVJW0v&_btZ{gN5+3*k?489IS*O+N;o?JQ=`_U+|*)4e!R7PZqPN0{RZaqF{gY! z%cth2`Ouu`29w0 zs|H-5<=_7+c|>+a%|n%X@>`)p0#9W?ajR1tUpfecivw?id6!%+OZOP4*@})y0e>g8cO-(oxe> zQG4?pgD!`X*JfK)BYYD!Iy8C(2f-ItpR^ZBj;td0hxk3hip>5|51qNXnCMWb%$~uT zoj(J5Ykt&h;1QQ3C{XIZk!ZB3MweUO=X^*EWn$}#(51ybqs*Cm^^`oDv`<)O3g{hw$OLSEOAA`qEVIu|Vi z^Wc#+PirG~?jfSf(Gl<<2G@} z?_wR$EH1pw%->`HQ^A7t_3RzH5~mjbZM4E^t?B^I9h#5F1Qv8#7ZP_`Y!DkPnG=b-s`TF^1T7*JAUr#c?MYPuu;KF!ii zOSB$eO9H2?kT#l9jk&D_v_++T)o#8?f zWxXv!*|XR&`}(fzCLOPGK8ISaIU-8Z9R+cB!j)O~o*bieU^)9H=X~e@lTM$X7|%Og zSFa3iA5|7P9UQ%PCrgUP5;!ewL~0h?pS9$HJtsC0CzBcu3{R}F{c!(Hh_W%K`jmOp zQ^{@0jx}&OawN3p<~&*U>4A@Jc4OWrj};j`q!&ae6H&W&F(dnRwDNV40yh`wBDhkV zyw-E>G8R|}K5kPDEtACQgnRr4`MQ-zspk)~~_xh8% zSh;#)&`5D65X&_K>ymD$;yd2-n(N5ldZ23dkwqkf$5)fDS*ATwbjSnsXwj46-(6P40$gWq1ee|p%-_0)#h9r_2mfi!L=c%@7XK(~;;^88jVaPy0 z{=_0KN!CDqX{kfp>u4D3cJX^%Q?r!Y*Vf4X1LK~SQpzm*SntAbCJsNT;{{c)jQWqD zbQ;I#Kt}$kM!PKD!5?!Q1+A%%ydOKHd1a9QBkqTr zZxNY;+WfZ!jK`g3VXR`C{Pd!zqDS1m$$2<8^zC_Bd}il3qnJT6z-;^U#luI1`8}P( zVqk$?(`KznKIr59cR$UDD}K!g;(`Pnwo^5*h^o5uZ1aqe5wlfBlI+yCk9IS881>cz z-C%wI<0YnF^7V?KyP0)Ufw`@B6|jQIXIjtZ6+sK|oc?L@J_9%WCd9j#vmdg=sp?)L zsot&6{J2U?trVkO-r)32Icf zqPWd8c-}*RAOUL_0v=hM5c!_u=TmwfFpl{4F0z>xT4%IH^v6KOQELuNn$MVggK3ZA!?S@(fWqtQ{J@BUE4hSbKQ}q&@CqjXSf9EE$HHm(^Ff4_);CWcG=*dv3qy#}=}X7wlrNj-U3P zYJ<|@PJq-}A>au>kbB71X>`EY{;`^?sLaiot9+=Vp#Q^UX)h^U0$^C^8|5~lbgX;@R z=(AEZPVMtr(z|OPtvMwNi>T&Gb?gk%w5~_kQ$JU%?cVM~b*+nNQnk|!wz`z0M)pfH zWBa?0Lj>2dEx6igETIBIehUsHSP$5VKGT>$**=Lq6iww_8xW*vAY9?l38S=n2VgJ- zk{4o2s8@F|RWi><*JV4Z94n7zYU+LZhDY&>DQ*HO9q|i(SFD9yt|O+j;jRb~x4Q)8 z+wX71gs=nqPW9>VMpV0vHFzF~kRrV_?-Eq9$%JdOofBZ{?=Zy1^`-m`MefK=>+m2! z7ho7zXFtzaI4l#E1{&H*e%ikVtyh!P~+s!@}OnEN8Z=D=U%0S~M!s*)U{0qc*v|um3;TddKj(x~Tnk$7<}xZmc$T zW81cEHfn6!Mq}Hy-Pml)=a2^h)+#du8r5=NR`res`u;Yi6m*sD9j{ zjY)F{%DN;n<85!y=5Nj8%z*n_LDCfp{CyR1>HRM|w1_aEQlJ*<&v!cDf~w&hYY+l| z#u>@Q9*0*}Q3@Ck+1$AJ_7U^isFKCb%Xp#Yz*Tv4)o3XSFlC~*k`~cbjDZ^kbV;v! zMAq-LOqdX*?xA4q+tyCpVxoFuyyl=s(E(A|tvtN+*>XYs3Lbsh83H2Db? z;l32c`s=94luWVoo_r=O?bl*qu3D%jqezxiHi#HFxA>ujqk)L%X91Yc``< zcBpvrgm}E+j$DXJ1sEyvd$CBhI)6(1HF0!z!p|mum=_@Yt5|W!INkrnMe?^`-q-+M z-yHr%^>9QLSsu1CC5Fo=qVBCnwDdDxA$v7+1_haNm;j8j+gEw3%-Faax zo}2WYe*s%f2`~>!P5y-VnM?d(o8!HN{_HyEKR+DozYMuMw)6m~2$Z$3oGnvP*U%tk zKyAWET|GEm!E6sJ^=rG-o%v96KXw30tvR$7O97-o^Y zC6&F5**ky1`|C_~Ak*ap7=n-iY0dr|eE|c)009UurNCscOMorUzyDwcHeGPn$%o6r zR~CZ~Zk?|OzZMzwmWa3|+S$G0mQ_wQ9X!{@2(!+0?54Md-}Y{4qcpea5GJ|EkKDqW z&Ms14MH)BjC-`|q+R7%c!k?umE!U!AIR3r%6tEc%+GmR&o~Z1l(dF9qxqRtfz*WN_ zZLr)^Y}Tco)<`5hbV?fYDA5JO<0x>#zWbwGOVE625$0$njQ3f)bFk{7{DFwU)`N>% z?;>INh^BEfLEN`JDJO*pX`g zJQ~q8$lyu~*){zbZrLu-mN;1_W6bS8?(|l^5HdQ8j39gK3j&$*5ZAZ}!bLbFwJ}bm z7M>+(Qo|Dz3WzSy-=ow31rdWQq4vpGluGyK@{2yp!?p5EDK%fW86^Sj!;X6lHZ?;X z#1o2RAg<0D5YB=Ds$$!^>3 zMP+YW>Exsso}}|Sg&F077sKGtqQM94k#%Z$hFFF@j?&uYn*aTjLHs}4K8UgyrTk}y zkrlMZy(Uk&!6{4{(1rdWLhc{C?y%YOb>XTdv@`7Gp3e{qE!48T9JL|^j`wPnHfSBT zT6SyNN&Ub#xb1?;kLuezrPGwfN~f$Q@s)eI4B*iP|_Oz{~c_ug<$-UUnuc_ zj?(7iXFnrNAu;>S`5GoHsdqk!2e)N+=J=@M;T0cV?=BSv@gqJg9c-?@ZQ;Dg)OZy8 z@N(GFQci^!tjW8UO8FC7JnMnrv$(?QFsPIt0FcdnFOj1!r%WmsQw}~pcDEmR*05^B z0Yw4OC_?u-g5OmYHO6mi^tjLMpTaNMqJk;c_1lvsKjXf2f1Q#$u$D3X`*RRx;VRR_ zz?QqS`#b`ACA4e5V^A5G@_Bspf5BJ>c*p_+N|Bj+cLZ9}?EKU0ms+;PJY_cCP4+!r zNat2P&HN{%OUaLKysF*9&S5l<_h1zl8@4ZPw}fI(t(XI+X8aP-8v1_IjAxnGW{-9P z1c7@((~YxzKf$9$@5&$GBCy`axzY=4pAlsGyg20+H?E&3e^z;UE|K41vu9kBKhH+O z{v3(AyOLaxYqS3*tr=Is?DRF>BbrmLCf#X_Gk{YoVM2~>kgD_vw4I4pW&SNH-ECcB zbAg{w8HiLoV(66PG;7H)uTn2Q`jt$Oam+|1A^HiRMX0%HaE-@Bp)c4zjFM(ktqtY0 z`t082_3+V#?ycoNOrM1{4PnLV>>A{4+qhkArP~OEfc@@Ks0Od9TPYW6SJ}-LbH{-WfmtWtiyU6U= z3?zQh{#4FJWhbEu`7t%(tc`Fm3Ld2n1#P`TdbY8ChDEevjOI$XQ%wCa_u<6!o2IB$ zP$}2TFA^mQu2FWwVi!7@nbmLJv$lWBloMnDR!+h;3<=?frIS~&YV2<=Q=BHnTEBWT z{&L=K$kuaw?J#=KVeN9xcC2*a<|jT5q68b22cHiIp8)|^CvA+hS!(=7$ezv~=1U=j z{P6L2C_%D&zb3fHq|`CCf<99BaQIV>|_ zE3>S4lTVi`yVvBZYa2dv`2JBn9*QJwbq{$6@bu57@E_luDiiqm(>CdYxI(DNieKAT zf`2@2USUfg2o*POMpvwSC;9+pu1s9hC`#4AZj=K!-gVeB`O~_QPVI*_`{Iw)^o{p& zyO^nGYpez&*>A>>K8BV`_lrPm*w|-qej195fUD?-@3J`02eZtnmTGaF1r5J`LZB^e zjuU@0+XxrCr#jtG-JX`9=6Izbs$9?t!-pv$Sz~jTZw;@qnnY1wL;VH~0@rd<-kH?6 z_^8tuH!$Ds;B(`i$X><^!*lbb?2Ptm&h^+do4TPZyHy#jzNbDXL-+O={sAWD(_6JXDb(X727d~j>)t`lW6^h!67h)b*lv2t`19`Ita`# zAvM$qZ*v+yHbrQ`)|TM@*edFgxhXTSM)~gz^Vq}j?}3PBO+gDZO!?xWdnHR`5H(Po zP|s|PzO737#yWD1VHJX6HsJ+bAqwjRGMjmY&saCbpavW+O9<{GHV%9V&7JbQ{J}@x z@L<$y>Iqz5sxFFML#;f4S+t6+s{RbP-3|qPdCdzBA8mZg&H)wTBRp<}<9`ijjY#}E zZI3aBUjK&wCUkIUS1Kge&dfP2vFzw$POEVHJTHo7+knzW0a$uLW;8-8%2=FVig*-R zE}!B%W3F9Pb})X-(?WzSJQ-)+@NDtfsX_I%ys$8OS<}&3$>Tma2-_4}sz`TOl#ZeL?Q-C;`f}i^>f|kW&2!?}HVhUD z(AJ=9h*vejI#x42(-Au&1KzLCkY4lYO-Qo#aKK`w``e zXMo>TWcO}|YHv@WTkfQ7w-8l)2H6x9_&N|njlvJe$F*N7BUzw2J`Y^bDPzr9qOky0D=Wx_$DP?^?eY#GVoGCzHR>Dup`QRaLf9|i$g{)i0pZ)6x?Ik+-F-$$c zY$LP3z_Qq^)~n{{Zcsqhveu$4;*iJn_;Uxg2Opz{Gp0|1v1BAqoR~S>?i#w3TFaD1 zGm{g+ZmerAe^@TwT#oE#IVwqKHhxQ7mG*=GWrs6ma4`m&9O%#x#^ZYtV@?I;RK`$6 zoP{63al<`>rLF=ejc=LKb|RNeO^gUgEMxQqgJAhs-!s7Fk8+}V67px#?eKyol<-q| zgeWXO#q2i-&nLLCfw+5vlZKdacy+DUCRP8=KFvp9Mff6x#F&)r9X`Y!^N@463dY~< zl&}paJh&7c!nKbogHl_8r4@-EbFg6V61{OBos?91X3y{$6g)o*WjcOC5jah_jdLEX zK$`HuDE)ec8fw37GM)U01TM;#$6<2^^qCd-8|@kwkA5l(JZrmC$tepR)l4+*kjF=+ z3Pi5#F=`blTK^2bPGNl{5iR2UC37VaX=Vm@e3F+Q4c`lQF)M^ck@x*>|IfXSlHVP3 zVXsG7)?d~+FK1TClj{CWzb?>h4$Mjur`QY2?pMY#*e4__bQmwtmxxEEvtbCw8%@V9 zL{k-Fv$Vm(_ zH5G?|W~=hwrN+1Ahzenk<7JtZ{77#WVwp~j43d_R_uLfv={NvQs1Z9*dMXzA zOAg;;);T{?AKJ+#x%9OtBP{V+Yl--U))}5aG9l}QJ0#tX_)HG<|5OspJNijQ(P#Ar z)R<+T{X8#szZ(`4#DL!K3OJTE{De1xYHwQLP_mv-fQvkVb*HKB5OPm|4L+b#Gid=m z;F%|&;Dl)jwF>%h8Rt)gSw21xT;<=;q=HHqSVfu_IN^b($XVJ8_d}rIr;!x?H)Rw4 zr4K5oB_#1TclTLBeTbk_IG}NRoEX3|E)!hFB||K!uuDcnZi)LlJ-2(#Dg$NA+rLd- zjGbHF17-ryLcjFqKz60Q!%G=k=M8q_0XCv>yn4aZwfBsORyjvC&5J{Fz!>-Yls4dr z+vGAAnEv_sQgGRXPB&VGVkzm1frrjAO#)Vd#v%jB!T zhl4h@)=gPc;%-dtA-FH_!kHqWF{4{49(wxAP3yYtL_8Od4TF|HLLgrc>@xl+5~S4I zN~g3(2D#7-q;_%cVBxz(Z$6)EsNt7f#=Lq}o)Hhw?udDyAgS9NxzavXzpPR`Y0i)2 zL3diqDfnrGkq^^dXG-1FO~LuORw=uDm~!b!tIZtaw#=z97i0z8Vwu2HHG*6OAdw|6 zMRcq>tqEsYn8mf6jAulvy8*0L6GbnR-x7jO;f*P!eg1b3mW%-SJokh;i+d=%$Kq`FyBq-ASc_z)I)Nn`Tv>C{da7)t@WAR>hXvcf}M8J zxR{)^-SmCSN?;a8D49<#kP7|=9i~_GRtN73(-1Y@dVuXP%C4*St13Z-20rOi?^P7e zKVdJ2Zr39pz}l=+@e#ped=GHH2F`knFWml8&M6!*ueR-8M_8M$PM$DM_<748$(}VR zW8}x9C-^DW&~w~}@f~!+oD8uY*auH_5VpHqlkQ_+U1y|$ryEF1Aj``ut9IqOZ-Z<5 zM*{rJmEpPDcpTKh{o+a)lGIUX+jG#&$h76l*vGrg9yF@0m_QUEr3Rg*<;mlD&kM!h z-9Z(g_(3(1sU*oAeix-7*3agGK{zZp29Vph}ADk-ZU(kgF(O~OMq9hQcZTX%9Y zu$~-#pqZ$$eV&)8W>6v+%b?NH6T)8`M3nbEyl7`vkFj#uQ7+Xti>jRK$BMH+DA_=N zm2aJK>wBx)EN~(!tWRc|{E+|S%j$IYF1)UTY=q=;()%2@{72FEcLK!x!%qi!`*CUQ<)fJ^OpnVC+-~uax}RH4rK>`Ua2{!Gv|J+HzFB0Ko(Q zrnhCOE!lV5c8wA*X79peC`j2ur>i>>6um&DB=q7*96qr$4#~Yn&1pvqyn-)}C{@$W zM3^YUtvRj>jE*TUxJhnkfy!b38rkuE3X^7IB}_$Q>Tx4`T|anBVj{j)0boMlIrsTJ zUJZ>kM)jAQ60_bVsa(_pmu7VOpcZZ6{REmp=fj-&<_qW>(4?utG|;fMglpe8Cx$^3Nlf<$@L^f+DQmFtKwqx+Jg*Yr2$Al2l&l^cY&Ys zqoT-$hiJT#fHw1nd`5D4)Fj`R)Am z(3R36CR@^=D~5Ygm=axm;n1NOS}sLlJtoUow@Js3NRz2YEbI9@D;6UDj{iL(~POIC8H`EKEZi_r-}LtmW`8KWj51+h#u0 zB%MoT`%(JddUB@;6#a6)iJz#K`jeO;7>WQr;M%{`Ww+s+3_$R~0mQRI!cztLlWmuN z_fslf%ZD~N+QTWV2C5@##ogwROn9Cy@+)dI{#suEG25UrBFLIHQ6gP$mrnH+1Bk7D zzq*fI865uu?Q!LeNh z4nJJK%`}W%{?@#3$aXGz-pfb|A%V<+&AUTNW(sKFqSpq}N2r+Umq9noCA zL$DXG)96rEzd5ph59qyW4R*y zzsrw3=Z8usS>;FAu=V^bDzjLLlCOHTe~*)hILTt@`F`s)^oR#sgL8#O z+L@X!h*$+!ZH63j3&R@TiA5xK6K3C!LnqD)4#hof3EKw{yYv{fGD@~uMc8|JrwbR) zJevy+?Zz$|u3|@zEUMHgovv>Y6ORHMWi)H6Gr!`w7@-Pzw8+?~RgOEk7nL@+DQw&O z4^3PRg?}cNnk@5Z%~}BR*OH>+AiNB*X!Ln}uzm1NB<`~yiNGBSlI}F`>o#y}H;HNP zH)nl6&b;iOOJ^?RwD?v!UnCP{BnXp*_XK>{#m_u*eBdOV0+VY5V18Y3Go{x&Sy@+2 zF64qfo9@9FG}_$`FPyP0!*oHjN)!IeD&tso2htkF(TqP*O1cV?gRfe@bFO1o`o$sw zt+$_){%T8-{?E_S`^O!(A8Xf}kNKYgbI?GJ3>tV+twdhao@ub*V@eAg)M9!PyFEK| zXQiNK-evhbUn|3jqb1FJ-0#hkJp+w08X}`gAtd$`y^t=tTJpL3mCvELrYVHP6>ra;4 z{wJufBY3=8eV2TzjkwxwNjJrq4R=tCy0*5(M`g~GbX5im%yqDE6(#9HP)X7Kdu-mZ zQ<}(RMXX);!w!RDm#n@Yzs8(7?i1wM{C95fQYy4Qzc?-kKCQL%w z*M-(BQGO}|>)6}=OP6bboaub*BZEg#l3zR>ifqu;geag9z^|f{3V{3Db+F3>Gvn)Tl3s&=?d9h>QuYe~2;t#acObj^4wu{|#QV}@X-HlOKs!+3G-NuQsq z+H%m12t5}9lOjIC9Az#ce*myH9w<7QeaIkbWLWF0*k0xcZS1lS@MZbLEpFVF9i#hr z$tso{@*<+jb%Pv|2qwM$w=UzTsw;o;HPtA07(Iq+_hpoK!m9-kP^ngq@HC&8DL*O8 zi!L}E-|%IX{)HHBHwML%wXnF4R*iCz)BR*xHZk7*9blT68|qWnlX_$6V{J4(h{!kP zt|uV){g;C6qGPRu?EXF8>hvFhM4^RhLKwA|6>B zax16oLGM2QDFdNg}KojW|@quc0 z$T9ALbXb6?(L|YADw@rgf9UsJPC}xt`ddAsh7yIcy@aqTITk;6UPqRT7<6FJ-{hP( z&fL()J<~C`VeY7}B2KDk6e48zDZwaQ+0P^;JCRPV6`>y_B$iB^yVFz7^~lQqvqNot zrUVJ}1Dh;VwzL4v+!v;GG=Q-TGX7;XkSXIQ?(Kip2rE3$SC57;Hz(qitaB zw&y1H;oP)&9Z%ZHE@CLDEkX@v)~g%6JYf?OIAwL^yO~1s+{=f&HAxB%)d7J2G8vv8_$boUzoUdbRyQa@m z_Uo|R_;>Aa_s6_$@ERh~^Ivg-0A@L@y+V&Au8S&En#wKqcPsKf> z@aML^NTNubM!LeW4r@a?iihGd5sd$j+)P%1z%?2i!-`(JxeYks-#wX4>xyk($i(Za zb4DZ?ds`R`4p@+YG2{qWAj1rB=ZR5Yw%7zdq+A@{@0%eJkGA=`Pb7__tug#OhUWs1 z(Gb8M9I%mWkZc_9QX*3$#lb{npDWd}v6%^vxTXiB-2HuO1a5-EO^PwV&SL{kcsmmE ze!42&aFuy9;&rwt3Sq|@k=|ee_*JJEwM&?i0j#gbkbm0%(z9XCr-TG9EV~_9*r+)@ zKI4H}2}3i!VL!vL?K6MOkjfF$=b=Y{Juh5QG6?v-HO4FC5Zx z!^-2vMn@LinH}syXR`jFQ1TXkmbs3mIW)ZmoigZjw|dLWFW%}<0qcI;rhES8_A}n? z!2_Pd76dNvZpH@B0G~`e&#`+>qe_ekMv3GVvHW4Q)}bFHe@UXbZEit)@D(zg_S@Kh zogp(^#Dtn$<-)l?kkuQS%&O$!tZL0e*WNmHXaK?aBN6O*CD}{RZS6=^!F`naY++|t z(FQo`(%EJIW?L6EU={_c4sb%d59#x%Sb6J54(;EB?}UWiBNZsqa*}7hTX!pn&s{m{ zBpm1U!rMoL#KH6pl>_{61fV%ZxUKCWNpW~Yjw}HWm=*2Esf=|0(% zcsXdhWEN?pC?`5Md5Jk=q}cdz^GqiJSo=Y+z-nDVp`tVTe6M>an-ChPMRDWom#cDGEw~Y@}BCJu*j zlKo(yVnA3J8VUc@<=um%h!PNJrF^Nk=g2G08|R`$23yHS2I*%bp68;2LR_xyezYZa z;{QgHAC^0Q2NmmNrx79p-YxilT_wplo&0VwDm|HW!hq1ua!g39G zpM^sKB;-T7f_u-%n9a-zHnjVgj9Mwe3jCsAJf}6Na`0^TbH|%Dl2iQ4Je5D4Gf~-{ zLN$h1+m{>bqDGEhoFo@toym!(8MHE3+$FBu1k}T1yLSJh+MOniw@gY|FcIJQ$ zZ0K;vy_Z|$F8I&v%y%!I>wRI@FIlf54}sPtXUT+U$PCpH9Hil?!@4mHE}N^gcYOiX zC@8=b2BtvWUkvkNFj{Qn$7f;xB#^yMjx-Qjn9K?G4vhH8-u5JnKTM~%co zfl_^dT8YVjwu@L~pkK5rMBH>sQKOEd2e1zV@qD1g-O-MNaSq_;?tu*htQ{Ynyg|)L z99ezzc2&m$8|=D`5u%ywVU*J)tfYZMwX!n`rwk-lzW@3s+z~E>00zB8{M&OUALaT1 zjjn;s{Wjq63>cQoQW0~)Yqz<>j>P;k>-?B-?A+DYHr7=%*T+>_x}IzN z{!PN=5^3N8u<%BEa7Q2d;H*@Vv?gy-rwAx(5lZL8Rg%M0WPy`)FejDwu`E@jU=N%X zT#w8keM;a`5oX~o_Ry0Gw?i-x_n;efndh;Sr!aqUXOzCG(V)%9yuQNl{*J(;t1I>? zkUbp?7be{HNDFM8Z;4?9T8O7=0Tvi=uxPtAk`lnQ5e!f_42YW(MI4jA5*m|$wHIc> z!@>g^8GKrbvf^vkfiH8mZ``+~OR=gG`f`~|^&e>0>R6tUdrnmvvWGvmSX8j#uFsVs z+~v0+K&BKZD_p2aGkkCg63<$8=SyKXobsXcU+qZ1{5Y7b<#oaZ4iJ2x53|&{8{ssjC zc*%J9xH=m+y`^KBpRlyQ-MOD!R%7T3d(BrrUyNLwtmIva{lC;Ze_( z(I?-WJI61sA1-XX9)L=)d%Js>MjXZJ?@&F{WWa}yUTMuU0+5Fs@Ru%wN6MbCmD5NM z_h$4;FHLJS%k@((KzcS3wz;&g>g$IHtZ(kX_Z7UI^-lJ%7|4>F{iaK5r`ZMhNa>TC zY1F1uS+}`=hN!nJr{Va*3RcZ$F?nJWma>*I>QA@UF|f8gTa>7W#>A$ssw}{dGe9qZ za_7!7($U{V!xjK|MQjBjv2J)~+3Iy}e#_@FVGOjc7+&!EOEr`Sd^pwS0GOBb9Ww$H z@QM7(MQgr^M`%ap?7TGmPWESp+z3^l7DiiGpl{=kW8**nbv~k92QiY$LZc<>uLjTuFa7ehA5>ACzRP7am zkHOIK#7>Ou!6<~s^;KH_9LVkL@OftJZap~8|JR!}=5~&=QkG~Bu|d&T-1GY6mh>UA zZMDs@G}tE3!D}oWs+YpEPJKjC8MkQ~rq>7!zE|?_W)TDfmxwo7u1QL1Y0&EoWjL32 z$@3q`oO`9KTm|zYvc%7P;FJ7U^3HG~F>=HHJ6)kGqj@D^=x%5OPBC2UT58j3olpRx zDlmje-~w?e&%O_Twg!ab1z1FC$+rO$ef-&{eL`0bKnzk_?=#w!7{#AQ3{3fCr}xho zJsenRXaq^dZ7gg%QM&u7j|EZ4b}DDE-d4~+pO z7atlpQFn81fS-PUzL-mLpn2u7B2thIg$tW}v9}rp2Emp^FYeLQz}hP}Va@-=faOXU zxKTnu9~6qdM30ootOAmNbyY#j4ZcpeRtz?1=RI%4_&ffWDb?hO&HBo@RJ@q+VC!K~ z7>d?)6-GJN9O5T8%Ck5-PqAZn^5*T|GE9yLED*fR|#3eJZt99=9>w2rCXdt~GM#Ad~IVH%&9I_r(ho zwn=XF-emK-aMU9U_q#-C=2{qPMJP!X|?X;@^1LiTJHuG}6a-rsXb#WBR0^6q6w zpD_|G%mxO!Tl}o0LkN*fXOKs9cofYridGigFI<$cUN+HQuvJIqApLW^i1Gk?wuiAx zuS5rG47BZCTG;tCCx{h@O^T|y1Lv=Aq~{^&!sjl4t>|@xlTwftQZ~sL8)4QfVKZm3 zRx{b%XyQUwh`$jIMxn2kO)$<}Gvu;S#Y`x7_`#-{(WwG3U*3@~fjza#Uj>3OzvfDD ztej3jriJqTehbh}Y8K4T>F|U`meXiw-K`u1@h!G~jN=7&Q{+jL|8y$3ak80?kfKzY_|=`yyI6>^o~TU5p;UqY;51 zM1*UYdkVVV=>q5YfZpC?R9fwFUKFW}!*M}LkrEHn5@gYCP{v}GOv2ocN}cmKl!dtK6I)3@dyUL$IlvU`Kf75^aA4R} zOQQr{()_T9Qien+B@_(>T7h} zh%T3048cH)fH7RxUAiU5+%4auLr(i{Z@uP0;hbE?^NT#|k6);Juec(x-jbRq-A2HEZhzHm?C* z=-UT#AdYQe7RL~|7z&U^(tA<+RSo}|4yd7!^0Dr$Siw@caB|P=AiL+10<~=qL(W!K zLw8Ea*A}dhv6~9LWGV4hSY+{(bj>MS>4QE;{&q-PvGatxkK@#g(FcZhWZ${Z$=l@@ z+oG-H2qRuY?25xf|8mAuXMR-vjgxeLz<^*QrN1pg5aKA3Zld7R__yV4Zrb@}imz!) zup&QtS?2U4gXv7su41DRu&6-9P_^asU1J3?4LkK*;B9psJ$1e$4Cm2by?Sk|H9F|z z-#(%Jv01ry7=KRvL0|vBrjHr`)bAquRPRMlW@JCbIxW>)SNyx=%>NLu3(NE^pp#7wx_Njj_DxK$@rv;L|d-X|GZ zcpF_W5%9Lpbuy=r*=~faFSuTOEV|HHFe{P5r=9y&>0BSV6midG^0p^A{An%zs1M># zr-jr2;?}&|>@c&&it%ENijUcIxSgfrWsPn!H(3*&a8VccnjA!)Pf2vtM4OK=M89Vo zV)Nh$UeZx(P;T*H^!yg|yt?eCs7>gChTQ$-h{i71~dMDOB$mk4|9qF<$s|8Sy04dtLZ z*P@@Lgp zeizN~HcG%|c=8!N&PqL5@T`RJ4koign{mM+M|*i+?BVTay$E`yOX)2$5)L*FjH<{~ zW5aWGpM$%dFgnXoeXmxuOd}Q68=9j+YaGO~irsk4KnMbjyYR1xKuv9#fGA1w7A#1G zR}4vU!BANFYXLeupGL}A*$-Nr;GBs4wCWUbF-9&4& z4raAZvn-ZE#WXs`c^R~Rr-+hJv_*Mm3AE+fn#;Nn_=c67xI?wsEVzP4KtIL#ZJfre zp4YpnEfCc#E7sm^I2zcx>>YND8k{`kN5|Rm^gR>b?As;(-gGli=0aM(V1E?kvf_WptXb?F`NuJH&OL_DBQVS}kV(6^q~$&e<^V-6E*p zH_Rsb9!$>4@qCEs>e^n*GxH88u~pX~0>>utrd`r^G(*oz(gL`)FPYO*ZedG5PCh_X zc?w7*#gD;UBu1V;A`}zKs-JN`rVpr;2=s#i7uXith)MA&$9YfF0m+iP8jp!4%@5nf zV`$Y|5Yt3`DWO9Fw$?d7%X!v6sgD#^29n4n(sH}n{*yzfiO(Ph(sH^=@CBl^M*=Pc zx_F8M{yralioNm;SZDRR+c}G>osIEKxJ%aoVs~Z06pN-^)VxT-j^<3zz0#-MpS^l0 z`EhaC$x(r_+oOE!g7SZ2k8yvPq&Q%;{O&YyK)P2Y9qzRVZTdHi`Hi?(Q^g*973k)p z^FUJuGBmu1NB0KFoX7OE6WHYjRaWfZ*<=`Q!+FTc!p6I+6kZ1Q8;L=TbDRZ#B;IDW$d83sfJ)-ilX#i1cD zxuxJ^6JND0eZVax0rf^M2RGrIAC9FDGi=9KYgB>IJvSD z<6XqUUa#ZNi;WzCHM(BZm$7i~7*C*pK31eor9Nza(=5pBz=7AF zsy!xr=fikSf;k}bG?))4|7W-vv)voDEhJMN2Cq_&wsbD|&a2^I*o#`~K&k?TxS+@8 zxS>%Tj6vj$Tlj}ugB=#P$YyeWU-~OT6*UzARCzzo2e5zzo7hO!*@Ck1yl^vRV=`y_ z-n5f9#Zgi#1scvO8Qak*($P;akHtS0cXy10{q6GTHbDLWh(>)GOL0G2`wWoz0ciwC zm!dr<2WMHAgOpfG0$lU8+_>HtY+KED@2a2U4Y5P;9)4U`~ z^|UN;ZnAqToR+yz>ki>2WS;nTOF|YcQ~NExJQ3}oo%{^CcXrFQQ!!*_7aq8m1yHS_ zC32P81DSLJO9dF$9J%P~WWssuVnuxIOin9px*T!6!2{7;@yr|P0XjUotjk-+Pz-1lV*#%1FwGN^^r4kip zsu1&f>oB_hZMRQjec^$^X9s}+xtx0-e;I#M$%kq7*Uew%!kAf_B#=gOacj-rVXZe& zgfy;%?*vCeU@_D{Nx--yl0E7)%STEB!7}lp-*DQkjrm|`lQ6w^E~Ant;2?9LfmM&h zuq%T(k8dh#OHlQr9g{!+f;^q^>;DX3i=H!r1c33Cw!S^fq=cc6-b>x2)Ne;pw>Y_y zir1^}03L>*SmRxm6YtJoH?A7IM%2RwS?s?KkmSopPDw4q95G&EI@m6vy3>e?3%TWQ zP(48B(SlPjgbmA?C4?YxuT%-?y4606Q2njKGiz-owXc$pV?u_dBEv1-NUkcUMF3(f z&17u1sz4YunDHM!vh3ynI{QM2V`Z*=Cn`4EBeTQ^qNLPM2Xk-Wyj37pS^7U~&M!t# zIdBF+Po!NL{i0<$h(YjJebINP#Z!%GTHK5Ase7O0*{|tub~+O2-6}MXi&RR%sxJlh zY0#o-J^iBZx(Vev>DtiWacasAyOOHv_H;c$wG3yabdz2i=5@BG7k_@KW-hx5fSa6h zfw|6QqFu3}`$Z=7j^v7Ht?W7E4n<3~a--TpYV~cv)u@5>vKFTL>?9)fD^#YOCXYJj zAY3GO>yB+iGug{bx6Zmq<@QzcGx3&uFzyn^4k6WJnNO4l4;1Qhz|qz2e@tY!%?G0; z=3Cc`J6zw%*5R_H<~QAa&&$@s2@P8lO1|iwGtf6Cpr@q&pB7cUigurKa=if(Ws{X} z*Jx^T%u>NE9k^E+t zEv-52LZ(yPYd$g^(RSou+clnjyz}&M=u{_+ePD2~hWeOranRkGc1Zwc0i2^SDJx<+ zetRWAIdjc3(x}vz1=Qx_XXqF`U~Al|=vUENb)Dixiv2QH$&QVb^VEn5bEmHeJ~a)D zXBf{dL}8&XnDUz7jKQ`+U7yK0xLG+k=wp!+$sY98Qc znXY~OPQZ>xdi0q=u65<2Ez;yvycu!!cf`dVMyw+X_+z?F!G`9|s_}IEku16geW}xq z>84?d^@MM7%eu<&u`bNk9p)yGxV}ck%MV0+6Z(z~HzL4EvViXZ_B4nUPtH0u9w$a- z+w!&6^nJuZrF?pTC?A{tBhrRom0c<;ULKy6Y5`fw<5R)~LPgJN&X{5$+P2lW*4E6g zTLCMbB9nc*+RVSykQD4W3uX>HVTvTsEP@oXFxifteV!7EVr(cL=MHzQ>zA|R=8p!i z_tK=dn({|IQ(jpz%z=`x)u^i$N){K$)Teulqovf&hMs)5>fdUy%{EC-MQxopGKy!5h``0?C;)` zd!jw7#?+;`5Tzudzy8Al8Z>GYqY zyH&o&T+t3-cUtdBbWLZ*3Nko)&qMCxfW>1ujx;?T(yg&Li)saRT;dumu7UBX6cAx$ zI&k?!^Yn765XWpf!l}`b>kp`_YiPrgKzqxqm|TBM2bO9MgqwHE*cmsL!7Cl#kyq$@ zJM}u%XQy9xpb<#`>ir81B=!j#Z75f;F(BHDB^g-;&|t6s4~!^i@Kd1FLKlBcklm-E zYtF>=;L@YTHUI^d(9J7KW0$(?H;q$T$7bSOi4?jEDgcDbXZOXGM zr@}L|^mjtBDo~r8_p$$Uw}1`1BG+eVwEF-4&On4zSfLH&KAKw~QhnBOu5N_pao&}S zJKg5#t$H8K(d^Tg9%Jx2wl~so+~!#EoiAvRa#s_jw1?5j=cu${q^*lH->nUqN7|hK zj``3yHfb!D;!Un?jB$F6=<_B`bpe`Xj_(UDPbONcIDFitK3IWU@rqLwcJ7D)QxS%&=WcRX|&| z90ZQAOG5Qiti@UI;=NIUfsdbz716xJxu;a!HApObbwbP)Ua6)Jv^lJT#y%S9k{EHa zz?SmB1--6IvM;)ymFuI|XWs!_J(BrzV05ZRQdHjzzE01!4%&+dHHqnJn*4|I{2~w@hnYu1CRi1!V8Lw9)qkOiQS|*N zGhX6Uo&-TD-ep`tq4}qhK#S+}Tr4xSLDg8O#-8Hd4K2xzXLaQS?141X zNXev`xXHXQ);=a=GRkyr3GU)m)4#uCWXKxR>RIUPiv7SqW8Zq%zyU(ZQr|nFq~C)k zE3aHUc694=nbZO2364ns{NF#lD1vZfzKP^O_6Wx=X=ST_N7AKS zY$3tCb^Qd@dB&-7+K?6N^r*|75(MZEAi4@@fQLMo#LdRraNnn10NbNfs zu5;1OhDe!pvB>J^f<7~4K$jG*S($y|rFu5YmH=K;9J0E8xPueQCdth?Ly?&ZT4vpSOgSTH0P_1Vf4-dRsFy-I?8 zx2CsI4R#BY>xKYT@NkkY!k;pl7_|E#Bxl9Xwiy-nE)Yyoz!E^gT2{f6fE8wm7um-ig|7yGcIJ90FUZmOL4g4v<_T^y5*um&&E(0q*Pk;Rp@4 z;AiuUU`+gzGyro6>{O_d+9`AT`RVb!wp#*HEVFGEc~a58b%y=fb+?~#NJdmF+F9mB zV^)(JDI~aOOoLHQMqsEwrkucWnyE=it9q2Gdc2&4lBA6=1i@*TQll9kQyASkisps^ zITZ~{LH}DE4ZB$#;wbYG(_=9eNe9xtgmU!%A?+=L;tJk%-@$@=a7%D^*O1^E+%0%; zcTbSu?he7-A-KB)celZ9aPG=~-(Ba{`EvKE^J!I4!}QGbTHUWa{d*K5p$1=3b!>q{ zchbvjYup-)(3|?x z&Y^vj%G!kh_cSv86&{qu05=DFZFAC24~UcC-zri^u_R({Mz8$jZ1vkL%Hn`D@0P6; zhYJm;pWZ#`7YYrcHrFE}q$#Y8}$0x}qq?f(maPyNktK%ZVVy)ZVy zKU79S!E#gCwx6+kZW;VOOvlgyeInWPLjQ-Hg(ZY(4prHG>);m;V$sbT)efy}tNmG- zmX8Ohr6isY$*04&*PWzShrOygBi8B8g#LkT&AJvQ6{np5FKJ|v1p^R95HjAY^s9*5 zncdJERA}!i4@Ubr6Vpwu2gW?0okRW9&CM1jG(3v%uwlapwNV0MMM=vLo*|b1{}TlV84Jo+5_0F$NIK2fw#~&O`tu z9`B)tJk13#(wo|q{McYDr2<~gIu)klSrn#QG>QQDIffbc2yviR*M^95(eqo=*=yc2 znSUw@Ug+D_WZ1sJ(&r}qh0o`n*JEwdJQ-cSPL97wg`C3HU#tKsH^awD!@G;!x6bxD zsM`G2vuA>U8aA{6QUdfzd`hTV6AIIxG@3{gm1eFySX7coFy#x&CDudu{VGaZO!&F!sa3uNY39 z2pz1i__P_gb^4OItq(r?mCpY;^R0Ti^8vorWXnO;Q<1iz-5Uo2qb7d!(Cf=&=<8kf zzQxmsxbqitFRpv}Iryg$BC)4z0mZV7710GfBtUKG?;5<~ZVPT1KWNP6%E`2#4zQ;57lIv3(%gnVIQTq++>7ETbadbyz@ONRm5dy}!tTyJ1T)UT8l_c%a_zMzF zROQB%Lmie&ndjD1SMgmRf57Fpdg@49hD^mT%hO1rs!ZahO_GeGDmypXCdk7qI7``2 zxY49UKU+~V#*Pll=S=5j2-$90gRJ@Ip)6T6!pt@Ln8uE8#z2y{e}`QDjhY%R^CZQf zCIlRShlB}0YroB!{Zgd%DI*2p~?-u z&qJ()@I5CUAYa7g*I>*weTK+l4`^rRCaY1DdPL}R6W%@ZsC|)bnp)mCoFUqePiZ!2 ze8}Ov5a0&a3chv0)q`cPW&|chE*v~lf-N8h-oKbRRt|qDPNU&!yXgpA%)n-Th4c(^ z-we4dd7kU!yBXYd6V1LwPlugf$k#?yzGXaOuSx=sWLMxIXT-dACg2K|SDpZ$p!WM! z_*uSDYGP`1vP8rgMz4}F)&JmZp;BLq*p1zIt;>~e=d)=n%6j@YL}8Z&c$wAWlZpB5 z(()pg+;lEyCot6&-@@9nq(K3jgBNvk?>IZRX(I#pw#~azUiQeb{bS<&L$5JwWtp0m zzX)eKN$oL(;|aX#!_IkF^IyHutO~oAbvwSn=p?4y#B;$?QtKJA(wi?DLwmBeN^f{a zZFGY=7XOx)-$t|Z#|$Fg673y7ncpHwwcDiIz0ct0?1_NDmoWq|T$>j!C7jn&7t_j_ zzx}H+Q1~>2ypgA9+WfTv&J)?^uG!;!%eVC}&MueVr|i92T1^wZX~VMbO@bc|C&Sgs zVD~55NRf3@|Gbs7HKtF}#u~X@Q}1lQ)O&^^FuIDDpPWSb-s3uTB2>T9czi%IbWmG>)WG{vKDqftkqLf7 z7YQMwYF#7NsStiPyn`98<1Q? z3~I}2!i8V)v}w7PwdL#nSd3?fHs8E{)$zoH)YMD`SvmBjPn&9NLv&hCl3xZYj<>p7 zMXwDXs}ubpgc_wngDR3VsxOPeXCGVj@5-#6ZZJAoO%^1jAGkM|$SuR#u9Dwy$pp;Q z-VLgw%Xy!M%)z{`lBm`Xhm8nD&TdN=wuNgh5b-mD0ak|;5Du>YQ{m*ewneLh`u0_h z?QF6S>yjG2q>ixk51&@5>O?c$ByEgL5JT@(+v4$~(uYvXq-2-ab&mzqdYy9KI=x(` zo2!K$+@j{MOKECga>5$;Fq2i`6MS+Lx^;hEy4zgv9d4^&-Tfn!*ofVaYFl+GoqKiN z7M4fmCk(jlfAHS>xgAHKE(=3?4hI2?Zk; z-Xeegw-x?$fJcr(&^LNCO6{u9Bn7LZhFB99Ih^sx2?paZfrZgv9Lf_dV0@vI?aNs> zwWcBw=)%Dc3*y9@;FC%{bX#bBb?=+ z_%3iF+i$ZOQ}o*RcHK1lZ(e>yhs!J^MZ1Z4(T+Ltrh8y?qFaWT)gl>oUx)$7^YJc}# zMG*==ySKWkAp=%}ix`4<-aM+on%bXMq=v_9WZ%h4rG*T49oWE!2gkaDRqZ8}yMs1p{His*THLYP+Xu1rHdl?d{g)|3fUUDpmDRn1$-l>8dEm$ z>wXwZ7Wp3kjS5c~WZ#!EBWoogL(`i$>5z9i zSl%SoVjnPd)LQ#QK%hf_{hUCF%D#Uc5JG~-)7Gw++#DazW9L!Y-)c>UKG~a8nO-bX z?+s+pr-?0@bGK`s>z|OD(r#%5pU{OGqvz9Z#R`}^ibn{OBHP}d+#9zs_v!8$Ex`0b zNa9{}fQPl7ir?*gAMZwUXoXBy#y}pQ|6>@;YDNnZ!RTSZQFy2_wBj1qmjxX+I={2Y zFyV2qU=pWPV;A{_+@Y)Q$-Bp^Y#KYYi z3kJn!uaar7skk%x$cMbudW>)O@I1%(ZWd?ZEwjt7d6$YJ8ff%?pdanND4!h&lVVpo z-wR}*tv<0edrtxqPv>9GJ#<9T(=TUsM~|^=ZE;D;ZFw+wEss$}b`Iz0nPR=v-I|BM z!)+m3KX-SJ^%wpLE)QO|oHLX6Lspl8;~Kx^qpKftQUk5TqFY(T{-!&(ot-}xQTkFC zj5!uexO3n`8Ws-h@PT|>L>J-w9`dkC7NaUjI>P#tDe+)fX(D{kOsY9_pp@Q=eS)tX zJXH=lX*-wmxuhYRcf!2y$8#4{I9m_B#;G05$poBT|NR1DX=8m3CoUG?d@`t>Vt|VH z#<(=tp2Pedk}@7Ue+r%OLXlW8|F^nl*G92;v+feuv$k0%(fJ0=SfW`M#B^ux1|?&U zU8vu_!z9l8G>)|tdrn#+Z=to+dXNre$f)m7>d|lJ>ifZ3&3@7j8B$_g)_(yNJ$a8u zQ0g(}&8`?ry6TUX)f{Q&EnT0!z2aIy30fVgS|=E zw?r>Mmr!BkNjG6!=nONsjm%|dxb_qwHmzcSMkYrnw%BN8qgd*YAtAV$5*&r9>lkR~ zelJrrI^*PjCR25%j8K;y zaf52>#dzAceV0$C0ondk>6)+6n^)4VB8d<(=kP&k6WSDcD#|IcCmu~Kq`jEkr&~Xq zScvHG^v`U{^S1{*{_0^;#r+yC7iOhFE}hKV&l?(l=fCft z?|iW(t3O*%M1R!Q{3oE6hX$UHMValPjdv4V{I}wS`69!37P};BWat>TbJHZJsCOvk z>AB-lcPs$M63DH=#gG8=&k$D__t*3URPl09Wl*F6}XA8E&56&CW}Zf7X%pEZc8y|3Uq{+*miKN#{2pBD z;Qa3jN#Jk+Cm+H-oKcPGm5o=|w5%)zUH6TrO0)_fYq$F61SINx-(>rnLAdz`E))F= z!Bt~VoYi4)x`<^X2&DbbqX%_+xj|^wWjy5Y5nRQi^SQF)@w>Kne`3AKIDZ`+XzKgF zyX1qF-Gfi-^GF$mI%8*t*#pw!EoxI56Ubj37RcCPJl-0}awjU$r??h$jTEXkmoV1FNtQ_@et`CP^MkTl_8x#{P-p7L}(r_y!dCwFYtD3m} z(qS8oR7$daSDfhI<@p;2?#}Dj%vW!1kw~~YbUh4373BZ!+KEb4($6_-y^{1sWacw@ zxSYa=9z|bi=Y2deB0vP^PaUvAZX4fhv(1gfi3aF0?X$tR$B38T04i!u*d}+_3bk$i zryZaS)x}_k;j@`(28^DDV4L0?2^|Ua1Yzv5Xj%^g=aDEuAV!q`ew%;Qi~p6~mP;y} z>WT3%cSJ@_q@j+TKs`QxYw$UDsLS=liq7t8sactQx@!MdgYoh2Y7{cJxVlL35R%xT znMhfv7;*5l*aT&ASOI|1@=t~u=3;jIP)mkuswG%+k zhyB5<+wSelSkGQ}r!^3M^fZ}?pO1L!X8*!`yk4SDOqf3$ntqQ$LV_UjHnqN1vtx4J zv{$!M0oG}&8po>RrC2BshI~B{8eSONW8L;F&bCEcU7Nm|>%Bz-fo?(2;7m69N>^Az zmrexzu3g(!j}?4WVTY_NVXrKS1Ox@^mCAnOJJ4Fu-R(&>(;6PDL_M8Nrwcl|k{i!- z>Ft+emsxplTWbNF#Ghr8BC zaZOLq?7c`P$%`}1%sw9Gp;8=cK4&rIJMW%|E`K$2U@E>sokqxh+NHfcX`#FP2nz!3 z6Csqll~1Mo6z@90Y;_yH8)x}mW@8w=bjDg!@c~!g2%aqHXa5PtW3ih}@*jFeunLg- zmg6jT39ZVOxwp6R>{CY}vBqN6h~gT_w!C}{0xzU$3|huj8Of5l$nq^%j(SY;W-9xA z4%hDNtUI@5N=qWX#l-0B_{?%^>re%lyAcAd>1o>V$e+9WHdcK|#^Fpk)rpU%t%ed&HDkWg3?o*Ik zuKqd}8U8Ps2~lmi1zZR%fYSu;oR0C^beBff3M;Dd%lf|W^*2kFNWru`67(K!ZAA_U z*}&1-W9xLwV;2BNGcb56nY=7Kk~NpuM&=~R2iAz0alC=jj388i4N}DB7-^2u0@e3& zzO|y}s592Hy)08%Q#a~)A3{8SCzMl()4VfzMtAYFj9v81-27f!GOLysY5Zh+Ln^1; zDl9kbzS)*W#R(B_IT6YXqBfC_^t^?_<(oPYvWuu-5a>G`^opVS=~PmQW@_1! zWxoPRC?$07OROTwk+3E%8d{9<{BpZb>m(O4nDlIERyNZ!e&AOeX4gcA4`;!ALtrQ4lux=bPy5Y91c*d zEA<{JZMyA6e=krjOZ;zMsCBCXE|>To`_#e@ea=~lmZ2KccyOtHLC_Gbe<|7e^{3QS zob3N(h5pLp&b(OF>7M0@T*3Acky;ntCWNGlFv7~mLFR?*-N(cg1Ga%vk_fNN%-p}n zip)(sAW%IyaakgIZS$h%~2x=dc}Q{ z{N4XxulhfIi4C_I`qTm5hJ>U?paY~q?z%BuVHohM8aF2tjdHv~5k!|yOUxKj_40~N zZ!p{Ww-6LTSu^V5I%PwFj^DxJs*3qeZ(E+xwDuZA^<~nx2(9yey^@y_Fg@f9ZsnYEU|sEo-3mCki0 zbr;hEo{d3G&B2rTy;l1L7%Xt_<{1%wu3(Y5^AdWftQJ{vr^@L^goQ70V6C$Rgs7ocEke(`-+dk36CFXEwJPn44+S+Sb1fQ> zzrSd19HLJ&G#b)~PMDWjuP9MV-B3z>wwKjVUEQOuk9mz%1!EX@nGX~qru#+;XD#Qs z#Hm`J#`qFxX)&{P?V`)Xe0at2@8p!aXX7(Onq*bm-C8w-2RG*W8ll$?GW@=+gkqLh z_zbyyyzQn9V7S9R&7ONw-9sUZBY9Zlkl^eaRNI(LB1dH;EwUA!(5gHjk6b=yXvlgH zP|7a!ZH?Y2>wE36IgX+Ef-ZTmEPTAgQ8_xg9vxdz;|UK^g@&DHhzd>M)XeQtg7|;u z9JF{AYk*?WItVis?`qARoZ8)DBP6cRRzkhZ!HVtO+3PoV)YU5aG7CWR1)k2527K9v z$MNYF2-1Ai^%G28JtI7mn7FKPsjz+Yu1c>irBpYxKPzSHI1H2jo5hw*&mGK^4fZMcQt9eYdT_NTNGtgyW^OZlrjptwI&$ z#jJ)F!Dfk6{pyAn50T<|U%t0_ND?0CXOJ)F5xUo9N%y(5>S^VihRhD;`nCf6b!Z&5 zHQ;C?(*QwQ@rqlPXp855hx*;;V)y{Bu`e?Ip6;4P`p}^7z`cZnD{@+j-Bga%&00F& z@cv52dDtYAW_MP9ZnMfe?ktkCd>Q(iMy0o~<_NrG*lp66&{P%pN6SFHQOA>@+4C>D zgb}kLSj^z8hVa=Y&5qmWiFi7l$&=6Pe~U)G>Gm@pf84D1Km-+&Ae6K8#o0fq_46M( zG2#zW3G(GJk_`H2BAQO-BR*=$=6!D~_EIq@*ia5Mxq@wWhDhl491NUAH%G|2yGoF$ zBzhzS6*_iRu7({E)XneO`_LkDhxT#J7YLrb_bhP>-x!d(_>tmaRz3G;CN!nIFwdKSJI@zHvb?(+(D*f8#Q zsD@)IA?j-roDnT@h*8kJG?&ezAJKjTsD&Ac{oc#X$#zk&hl!|l50E=SW!ztMA|x^wovkSIcEI(@AqALAI6$C@?6f(bAavzQCi0TcW1cj ziCmD#i%lZS=O?r8ymT#Gbs;3m=xM20<7{zFUT`wWhqY&!CP5`?8okxcSrWy*Mb*yA zevUf?vt$*me-2riS^abO`w{2aJ*wj>9WRBc0-aHFYS&)1u6D`^<)|c!gOjhFlh|qn zfqF1(o<~L(hkbo93;-a{0(B5hk+n99A8{~E$3a+^BwAn6_n}-OBkPDgBF^w{(YTrc zt{2CL>MV!4V_4{JV;t<&-mr6~fTx)r2pk|26L5v%e~F+5Neb)yt^yu0B@~Fyma^7{ znKp@iUPLqKp@Ibuf2)) ztmT7MyxU%Q7`g_a(TK(WP!IXOAEyTr=cCkg%39~}VK_!GWaDy1&IIv6fRpBCbx zVIi3u`L|-Rqh;qn2m0a#F%#NoX}OQ(`QuMTJ)EH+|0HN&Aja?_xO|mU)kkvAf8Z4*hiKC^)lL)~h)N zt2w`-{cD6Ewj$0w?DQwSINn)_*T*!)Z$8!LEt6byG1e%E_fQt1z|7A& zvmaUEEh_a^D|3HCY7`g?H`#$o>doTScV)kvWspjj*}x9&RvQV@a@VihQ?uj<0x`kB z&JzvWj+)5U#zE!-C>Kme%6Wl02$q|Zp2f_gj5A`&KJt>2~7oC60p3 z*KVnBDTpI-O`3;AC*%Ea-^Vc>@N5Q#TI}ENf%3B@r3nL8^O-yqt z4f5BUlk7-&H@xkRNvc0dGsc6Kaz2+{XGJ~3^(M$?Z0;vEDU@QB9*<|$wklw*;3r`` zQaiA(BH{w-C(@Yk=Kn-z$0l#rXvlrMEu%> zfRI_x9+L$jjmD_E#8@?e!L7prV1kETxUc21iXwn=3jMp7G5DYEl+L%r_5mZtHVJH7}#g0A6({~D9Dkml!%#pq;x(I9(6=~U?@B`cFe9L zSV((#5jb~}O54n_O=zcPRw;y7UAb_gud1;dVACc9oWsjmY}|b^6pCYV&k^QS%0K#v zR9O{;?l3+4@zWZeb)gq`WMV1`bUi>xeE9s5b0!2{2s8<&!3t=_kWNLaW~*M<^hSP1 z+fN-bBwwa)XMA|lmae{-YnHu%Ly!Kgs@I3@PQk;$x+FFQeRq>~+B z?32@~HDgG76wDUal>@~_su$_Y>!#ox|M*ky`NkiX=5sz-4PN8~U3fu#8#vf`>`Whc zOOql`OtJ<1uLTavG<+7(C+G3+oE_UIq(`%fNTawj7_wL&zGrL3`}xiZsi?BH*h=zh zeP)}#$6qAT)Gav>Nc4~kGC@R|`viAgS?U91wVrJ`BaJn*#$WgNljQpc(s7S&*t3Y# zAXFn?2J5&rYWOGXCNj!RCpmETPknl*Qw_t9qzP5erHJ9nqUhpL7q68>)`%^;dajO) z&ENuXN``jy#=Sy#_TQI@seU0EBPxq3Vm9Pu(^2WUE!6+Z-?{_jzf!&cOMQzU^>}3! zhT2F1H!S|k2>}Hm`++KC&>!V#+l4vXsk1NTD|+fnWBGH-DsFQcAbNPn4p3OzPViLOTAsHh(Ng=ApASwTui?6zERr*65i$R^Qf%Nk5rarqReGt3Jfy~*G2u1=I zUArm&gD=??D@I08-t}!#yGR{kbXgVKP&JNhcIxd@Y{&e|v4auUBx7OaH``RCqaP2R(fy;R_2V?SGCIuW=jQq* z-5d4TE=nQ9{I2)YT*0d^r~0DTUGlrgqY(J1%75cOKsk;GIf(p}eSwxvaCex{1y>5T zMSG5uu3P|6DrVDBq1b2Zp$#Vr%UKZj zM7{WlbZfn2_!5!i&?c-_;U2FA=H(S7Prrfk(bkyTr|Ap#{Vk@EV}--OA#fyA1;FQC zibZVdeI>O@Y!)TG{a{fqK6WD;9Df|bEVY&bYriyvI@2tYA3b`*1LrfkdQnW{i_IA4 zLE4R^Jkw!<6;060&Cq|E>`Kjw;g-9@jN4>FV4@5v5>eG1cr@js@1O?zszSz#hgUKp zaVuIqr38C!G36d}rl1t5CAwF$w;)M7Z6$H_8__pGPS)K+qNnrW|_w{2xI9 zx1qKucJz_ff30nnAHLz@p$XeY`5g7D zX}mv~UgJ|SOI@@qbSE!@cvOGbW6KtskFwSQn@`qiG}Y5PzLY^E=1JFly-BL+*5ee( zJH=FE+;33SiR=Z8@u$kRt~ThxMo&M+i7V&jJ*+LGURBi&i2s;?y>C$}HvOB$zAYrX z$g4N230=aZ+(<5;!4>ubC)_+9pTz3_XuhJIs8<}h8HLV<932Q+hFBhLpe>zClz?;o zzI3mlHHZ|OJ^S7h?&(kOZCQVx{v?2%T&?ft7fNm`!XI8VY#-{tIouO4-Ep(EG$=Qmey0?2|}kbd8s+V*)_p3Iv=n(l;f`&6E|g zva54jF&|IK6;dPqprjr^U5^RMp|J@l)Q!@oGBmw%dTxg~(F}xUAXrTKb_G@h{50yW z<=Po5B23>&cv3peC}&S7{0`=N5;W{E!aH`&%jI{c_(b6j?H1WlBw_}}O;sFyzOoD8$<(Z6!8+CrVfalqlO`q9W?GlJ?1L#)G?Q zB8#(W*$XUo-lLvHnD((R1@=UI)hY2apODj;>p9RoLRkZ%SSlcj9r|#k8^Awk>4*-u z_2^LGFqvqr*R8Hj>PCf3f2-7__7->Y`BwrZ7EU6~ws6JUy`GP^OQ!LbOI=~v;Fn-9 zE?wJ_{xoas?THPqRC;piEjwitMdCTN9oEwcGpquhh0`L?7{Nv;mozIH?74LKL&aZurqR_EL=}4ucL5?T=tCL!&`z?$uEi}B-$|iYqJiSbC`INZX(^Sw7=S+nvFN5;q zDbx@5cd1AEPU9KwIiDbWVcS7suqRk<@f!>btW`wbYHlMfg;^OU?|zpb&AWC`-ccd$ zX1CK_+0V2zvW&GLgDhk|RIjrhod>GAv>I^mr_16~QX=BlGM26m&u5~DuG!W?Xc=F* z_Kqq(Jb1;lhrtNYlU^4+@$&5%Ji3e1U;<_16sb-2~TUe4OrCvei#4542*|Z z*)OM?T-_XD^%LABsA-~kc~rfkr3EQbc~wIYvFA-KaxV!VO^aqgSNcwHL{&=p(GMx( z_Bb^4r+6~g{#o+-$p4hh1vU!Ch2v-8wQ(lw@v@qf!)%x_r5zCuvgl}#uX8?{DsNXY z4td*ZF;y1v@i&ofe|PBn+GbH^uVtQxJ(4U3QHNtkO{23K_U^#S1#&Y0qrN^RDw5mm z@Mgi5qM{6sqy>(n@C=ViZ;-L>if(Z38#$7FZ{sQ{@rXzjUGdkgbOd|MPmAl4f9h=K zYP%Ooc-gUHqg<3)C&5@0ocim#Pw1u<3RrhCL;1Ie5d8Gh7e9kSD5qQZ4=|wne<{vs zWnmWy0ZxRPfb-_BbqyW z8*Y=Y;6-_Ws#+(g9t}ft>RI*bt5Opn&-k_eU(qm^y)q)`86by#2ph(vudH7xr5BBU zsyMjFbNvkz^0kdrw4kTVJ{h(f>HUQ7qe|u+>ywuBo|s~1_@D|RBFp^}y2>(q3g zX!U4vAz8lu$=vrydXU?`-+Je(S0zv>Qxc7pT)|ITB(wWIrtHfoXUkdKdf6n=AD)^@ z89tYC{9&K+wP6ox-s)S^0qkolm+`G1@O;=hs+??};2<;W4s|ueIKJm+uKFhsiwTk> zwFNDAk?X}n`@7s-FOdH{EHpSZ%*I$XX7{N^x36?7wkDEW1wDEag%EcRK|Joxsd8ne z27sHQXyyS>t zt>lmZld4kleP$3L0_;4Gg0eRc9-EGo&=WkgQG#xtXx29cWz2WAL-lTK1K|?7qnr|3 ze9pLq9|YplN7bSo;74X7z)`$8^oTvp3gy63xf0U|sG3ONs{{-7x;CHn zlxs|TMq=AkyAbx)N)h(_V3fv@v){y;&wS9ey^=?X@nA`P_$bXV8+r&)DRX9A35 zj`%9)rVh1r6|3c$>6>^=)Zt@pnCnIC^t zNZ++nqMOG$$1I`4%x$+;+7V~b4_h(iRlTAKD;?X=rCuWVb3*}B9V!KJfEU3RT~`qjvR;K`7& zxJm_nF%T`**M7oA$nh(QB?UkmB!4e0CaT)@EB-M@mKvbn-vW8)Z*!XI;^v8Oh%aq! z)3Glq0Xld4HU*5J9bpciWd49>*E~_zDk`q~);EW5D|du?05bEAvI1c>Vs63&TObN{ z;fK#~SSW2-=Sb_3f>$=CZ6$h`MC7bqxjw~t!%H2tX~x z7;Js`16lBo`7O$=xmZuo@xL<#Fvz%^_(GJ_qmio^i{Tg6nSnzy7J-QG2!*zw726~g zU^zjjut13rE$l)*yRNauc{(^(n6OF`LyJ(JwfoZ-8r`q+V4skuW7cz4&^RJc>bdll zZ>M~RCXJhM^4!92Pql;pPCz%W-eBB}g#92aA}VQ;0J|!MuTbn3yP2O+ zvRG!V)na?ciLZ21xt3P!&C~X+;biYh;h%j%ii4)nV;M)x!DCie)WSix(aNXKryXOP z><{Z>UUkQGds#CZn05;X^#{Lv-CT!DH}Gv++=rXp0$Vq?#6)WW=afw9gx+<vVbeOjzC;`k>PJZECFKxCJW<+jbx*sLeZ8JK8s{dD{a>CM+im*8j%FSubRJRs@wgDzRHpL zXk*h)d8wg!i*lc?I_g7bIWw*)@AzyHM<->U31_z6tqdFDOfDCuWRixX79uhYs(8ug zBv~(@n+^v^KI(tuixQTk!rZ+gama&jI@2THh%VXC71fkMNYs3nno)a3@_^P;t|GnE zk#|$_#3xs15ol-dRnbpF$#*PRDWZ${^K1HI1Zm1mh z7L4QMxu~;_>Oa! zHSZ+!6o+94Jwi9zhqL)FfgO>EEirb5@1yv2+tx;`XN+u97SzAigJj?LQR0oFtY;pq z$OM$J=$`HOA`1Io<4spnniXFJ-v<`EwNu#BG55t3rzCdaI#iCy%JBtAui$+VgucA2 z^??1VqiOxn(zDkZ8XqdSI15YTCPi?-ko7ZTsftob`lQ_hmUV0EbGgx=c!n4!mFW&` z`9aOhM9_N?i{hsouT(8_T|5V@OU^7L1s<*KBdDAxpf*O{V=-uWr98vpCXzWpAbO2}3?O41`@NckP=v-=S<<~e=*zJNk_?E$F?h_gBp(CDAXIXw| z?BS5~VAqX(!iCY{6B~kA@=Ccu`bl&h$jG-Q53jxMt|+T{**>^G;&b_bCDW-qW9vgi zkH;C~$e6Jl&ea-1M?2GuWMrl!>qsYv?|C-&Wfa8970m}oq6F1l(tQLZ!Rb$48hryM zl8#0&fN-Ror+yCi&Z_@$ui0SE7`Bj;bU>UgNi#~0Z(4X!>sUxvs^+4WQQ4NWrE1c_ z>sJ@=VDwE0TPYhccU?W@jlq1+g~Zv$Zt%_T1Yh|OO|p}}iWsOS+7HtBq8fup9Xd6F znZn)fo%vXIN;7r)3G?zy)kDSbRJS?t31aHi_n+Fksp8Xt80nkHQlDP zvW$qcTP3%-o*(zU-xDIO5nilQ#43URGZT_LKfDO1jShTlXEKe&!J?fTBj_4DT1k^^XO%ro}tJ!U#2 zEo%mBS)pqMOMfMP8C`kY>RLqp#X_s!(+muKNE0QDFXP+C<2+#H1ebcPHJDgour7f4 zFNKE|*a!_*gb2K2R`K2DVXmMYr5dKv_2nkWJhOZ5l1fBX-m`GnLzpgRU`F*prK@YB zgbShK4qbXlR&NFvC}IOTyTzCqSzg>lI$Gt`?|JODA?Y{kV;}fU`)%evH+`2ZbedNG zg{u)(FhN!6*|8(IO>fVd;A!an+@(38mFyUE0EOyL2*ejbM8q1bbFH4=7G50rkJ;@F zGIw=*mkSeTHxE0=0h4b;6sVHOpmR>3Mf#z6n11U~uP zI9)wsU`>P0bKr6ii8_sr+7F{mS&-cnXcaHmygHetVd5yKZipDXOc{zS;@*3^9q*iI-s$d%`aycVN-qakYkV*L zk$!8y<^U4*bwkX#7h(P>a&s8UYF>e}u^x3HE)sYfsV?YQ4oYYK5x}|46f}x^ zqY1NS9HP@#z#I{Cm?sU3n@kV+b$Ht!f9a$Vcyzx1Qe&8L$k!ls-1qYMR8ov?a7J+! z%|sjPvt;cY3dbitI!vB+Vl<7}dTe)c9ZQL^-_QqB+&S%4A!nE@8jQM!}KWcNaVP2# zae*EFUw!2(6)`9K-q^=BPzo_k>4?x0DW~T*|5oxZ3lf%2>Ax5(&^}b>e0T4GtoJrd zh_8HhZVJCuCqd1` yZH)u*{S^i^7tZXNYO6;P$V&WCM7snYf=D?yoIy{mSUF$J( zHgB}ed|>z|NkA_BOfDz?E!aCi$HQ=10kNPYW80Y(K^A=fQqf*Q;ZC=v9cYxnzyYT6Ax&ux+Z$Rtj%?GiiOf2V?|b7zE!zhe z^606?)H;#Nc@fzvGzrxnN%3_s!?%l*{JqvE4RG19j`l5MnlcSxA@)Np z%EzJz!shcN;N(f6u3j_c;2nFRcN^hMI^Tr)Lw@%$S?lwhM|!XOC_dwZ{C@%x0uJob z+;cpLL1C)%FHQx1kyA|R>!wiZ1uKnhgrXT_EDqYcn^Ak1_Q}j%j{QLIZ}N)^ec-cSTdgI~ z`nCPnOD!guhkra3D}~;XFyzP4EBz#DMnJmZr{59C;DzyS1{XUTd3T3j>VV$EOwa7S zBRdQnx=@@~%S@;a(^X3xFswE-g0;Zr-B11pPgCJ#Dc6rtkCBDEa$m59HR2@{ChJv% zH5~O<*;OB7;?PTq>N7g1;VZK^Mb)ZGH=vJPvTjy>d+l1weJfdMbzR#wHPUdJpnL!t zpis~&w!DaA{f~gb!gedCY-O>5G^4?|s=hWX7_WFIsap<8)4U){nTC0R&)!_EeS+Sq zwXMt6SZ$nU+v()HMs=le=JkDt?sS*7t`pTTJg6QpPI~vzcD>7{v?!sBPyo}PoPD}Q z)8OT%8IWav%Fu=4iPKzn7*vnB*LNJ?U%*Y}{PaVB-^2AT!HUilLw0R#&~}|ZHe*~N z^_%qBLmMG!N45fyA{6qK6oH(Mz!?N26`+!%Eh<4?5Hs5_1gH@1JW^H)@ofwJb$qcs&rj`MjQ#Z%E( zLq9cY^>t<%`&v=?Cp7DsLnHb>EX2tCVhq3IM-2BN^?L%13Behb=k-Gy34hgyDvB)P zyl(X_ttCPda?fuTuDqXszo@7ZE!ONBnDu zxZNT0IU(H&)-n|{wwNdAwR58 z12of97Ns?_6o+ih*8;O4AqMh4wHoPj_rZFf>}OwO%!^4Z=sr~IPgl0tBLq8yQ#Bh= zsD}_#m4;ZK{0|+$x zBoOMdIbz!^wvr&96X}q;>DgB;{An}|{lgM2GeL8BzI+z0pZp}y5=olPojw}t_|^sq z^2!?W!SY5ggv56QU%VzeEONg!=!STq2Lbw4aIBN@im711#94m)OY~K=Fp#0WSWpL1 zo*%<&%I+_ZmMS;2(hpqss7^dLn2GvNb|k3IgI-I8ev( za2*XF{f*X=QD^t%xsIPu$DtnN2_v^AVO3Mq#wbcdgx=l@Z__KXm<_dRmiIHWpF%2r z9LxEBxMo!vxJ->2s?`4j-D5&TpyIfca>MziibMmpah_}3Tuujmp$~Q$}?;G z%+5yuWn}e1s!hWs?gdtPl1!LOBRON8%8uQa=_2(A2Fz_K%7qP^z(A(M<-j24H(*v6 zXKKtwQRTpQ{LpSeU)J`R-ANa-kuaY28_zW#?7dXD;RW>162)}SR}$xlzK9vTsG)y> zz@pm+*P^%NXaTi@orvAd(;C~7x_{-5=|jq^9h#^z8Vj1!EWxma9m|H*hZeTEv(k~c zzk|c+5e5eb<6QCRDpk_R?4!HNmEJP05{oa#6Gvqty|y|{y0-|LGlM02Uel`jZnr{y zDjzyaK5~fE)P=eg-FPL3#PL(`^(@)290*IjT!sj{emp-?*^?`_J7@xFBLiDXpq446 zQoFC^mUj#Gif!KZNZXg>uixT)UTO*BCdtsxhSM{I!`!uG^`g>JbaWH9E^R8U6qK5z zk*y8t$RV&$mshDd37R>7OG#SKHM3XbhHUjpksjpc)D46>@xF+=B)>hMyq0f7NN2@v zBseLifM;L8`(AF3R#s{56v!OWtbXubuVmJEf!c$&nB57417L?1ynmM;?VZ$+7RO&Ms}CN;*?CADC+27N=`b|dMeHuPc&Ta{vx|B4fUddS+Gkm2vrGjgfojXi$K|;$F9W@Ub%*2iSRSlD>#A>0lK-i;G#-Tt-!y~7jr_+ZrjRks{taEfG`v0mb1+? z)P0u>YNr4`pJb&WKN%3(@k+FRiLA{MtpKXYz06rN!q{lR(qP&6%QA&}<327*QF@Q3 zo4kaKc)dSiOTQcg6Woxq0HH`VIL~IY!&p$ai5)|8(mpBWQhqvMhtWB1qEcB?7Z8#D z(5m>+=QI%n2iS@r;~~yW1y9 z1+XKaTcDtbO=?c?W5KGGkFI+Nq;_)Ka%?M&YybaLcb-vAE@>Qp5djeafeR>AAgu40TmY&kVtPTASEolhoVReBvL}hhLjL$=p@<6-Fx?(d-lUU zd+zRs{WRyidFN!x^US<6^ZfqvF{IL(E_Lqv-*XkjL;wxbKo_;yRb=Rk`z zUH`KGZAN&qaslR-m#Q`3ZVC+tP^;XVq~YWkCn1;A5*0uvoYt__^cw-e?~p&;Ar|{Z z8QOm`fU^nJYZl96c6?VCCbPh!d!QH`sW9=t8OKe%CWj+N!_1-ygWt2NPI`e~fsWFM zxC){_Ywcl)K}N2a8qn&}?UTXzUfTLOwP~u!P<$cdsF1#WK@2?vT~`E?+8tbm&yU=` zug9(#N|hSUC+ps@?`b?(y`I;m`5g|#4MM;$bQNq_E~;k8MfnIg+%ZuWOR2b5+{2_? zpUzk~9t?O_$R3D^Mg>}GkSc9nrW@8i<_f56thc}94QVL1x^vYU=XldXw>Jc}22p9; zApZXJ@1%zttEIml6PZ}-E4NYUV* zBasTA$k^e|QL8tp;LXp#r}HtRy~*QZ_U;y%JbzlH zCQ{yQxfS} z9!=j%<6|g&A4tJRB%#r3jZ|aGq3g`Ng>w})q5Qb7JjB2lu(LYMz!Pb)(`)4U938Rq z*?5;ylKQWLo@k{9xZM9RNqe{*Zd)V?MQ;65`y=QtCOHd-13dbhK#|LBoj=x~ ziw2gsF$lfWDHHtNUHnBDBm*ONt^3npYYG zr9oAylRtbX2Kcasa>K)x-ZHoqo7zWNN{Tw__c3k&F~Qb&AqL#lYRM&ZM6F*tk*}_T zI@cE*@(q?}Aa%otjyNk1oqAEU5Ab9BB$<1cMMK%w`s_fl=W#1PIvm=j;gQF@8#=b- zEW^k;+4RS}aJV6}px}SC8UJE0{I|1fBuUH4in2hoQyAJkNIuJ&p!Arz7Q>ZaH^SLwQ>P*39hWX@Z?-sjymO=x&i`%N&#a5E`SXy zb$e50S^vq6xqfTh^+S8u%>J~~zo@?NmDyxTtKiv}cmHm&BOkxALY`hp{hh_5zr(Vb zFHHUghN$Ut6Ino#r)J1jW+I)e z8=4iqZjcvfX{(^Xd?0#%!xWu}=WQXn4~d}C;%r*9NiBiL$%OXPd$lo+=LBg-4m?$= z_oE|rL+2J|@pJQdAT?tUQJTMSxMb`LJY#oEfQ*jF$nTw**32e%ogNZ_93opE)XbqM zTPhezhI7Zu=MO`?NWwn~R}J&j>)1pDAp0nrBhIdGUk+~$8U2{I3DC48FyO`TXYnxT zl;9CbXIILKdOxlj8KC=|t&rUkX<1-d3q|^Yy_eEE_|?!E4-<{4&+yp>2Iw_VnQ+Gh zJJ`b>1lBEkW%_^sj!!h|^bf|BbFn&*W2$b{oRRJNy4*Z70Hg*soVJ9J!@hR4-7-RH z$-{7;451!LL_OJjXdU31oX5?|Ta_;BF!A78vqbomLbI|`X7%_~hKv-Yrmq$BHS#gU z`;%kf;s>8?h4>)oB`60=8d%H}1rX1Xz(wm;+g-)-IGD5V zB+TxXWog1hv%z%lF}2(+qM11AT!C(-si53PlVW{o9VCX4zy&Tja9}nbS?V@k{DQo( z+-b26RY?G2?U7M3>{uN5oa5gdfQxMt*{sz~a@f-W$Rd}4msc}SSATVweqE>Y!C^!K z2(5XxSNY~ygZ7tUZyhsd!dd6APWVb*yMFU@V@IEtX6;X zMF;DbK6~1PyO=AJO-m2t%kThfaG2`Hx=h#mxNwKP4etI%VT5+TxWyxFHPrlgs{E#0 zdTx29X{iiK-`txU5k1SHh^j+k+Rx9l->_&?&dzMZo`A-W1BoT9-xiT&Ulmg$ozv|` zSbhVD%%D+51sWlzy&3qZq0!;8#JE_ATBxMU(Rb_)tMc$e!rx}e3=)Zr>e1~c4M_UfpZNk`XfbXtWxr8=DvcU% zn{$l9&yCO-{irh;9cx~?0%Mvzf|TZ~vp%tg^oi3nhawMb+x@X4Dy!rT63`~Gq6*m` zpu}g$3j*XN*1Sfwr%ry*r&bjBih@>wa72~7=gq*K^f}DYLjK=;XO)Z29W%VLn3H|* zsU)(Y&A-Q=1=0n(kmG630s^oz&2Xj{rJ{duwP5N8Ekv1x)=bcXO7mXq*pYX!gy)1r zLNb8|dd`;jsO`aLSJ^q2uYDO>T$QdHnY?L&r+C#PlIEGHD&n{fRVzW#O3K$Ufk+=< zjSs%;qxx`)$z5vOM!uvp&STHB4Tk8u`_qDLB=JU5TprmB*kRXWPETlKmOu))eePL0waZw_IK2kU8myI~?88&OY^Z`gRj zivgf0l(V}w!h}$NL3}-99meyw-EJb~iV6}ICuh8r%3b$0;N#?H$B_?{@UN7h8 z(ZLVz4`=X5A{x=2YQ8(MEcD)SSkMf*9qFwj*DRGTF1&ZGqKq5L@@#QRSiX}S^+k1C zs9C>88;>kHg{Nz;i&p8^xER#kfQNb>iP>7|PlY5veNR2%jRK<9X`f%4O65|qu3T3} z3%|;eY>d*b9VPBn8@FZpeF)Xp-XV=$XWR6~=%38{=-7{&IhaaMn5X%Qv4vt1mkuYj z`dkhYYf4YJ#j&I~A$Buq&?wZOS8!Ry)v_}nBIJy?dEma=@Z5`8phKv^=g&)96F_~J uf1<7K%8&osya}HE*(ch6^ooDnJe6Lg`+c+ln;#tHg!HrxwaPBpKKKXpA?|Yk literal 343391 zcmb4}19N3v*RI13I_}uEZQHhOr#rT7+eXK>?R0G0&bRZt=MS8kmD*K1YtNcmbBu9c zcbL4a7%VgFDYOEXmWhled0yeCdhE;V0e! z^Y7wMA&k7ZA$)-V>5iq0K?C8x@^?O*gOnl31+4J@_!X#LKA>T#T|R(esa^C}BJzm8 z0|JOKYB<3JIClf&p;FKrb$_-CxUNDNKN%=jD40E#ZWjtaI?z>GcaxeV_;(M%-p}mH zZU@sy`&(BHB6Xbz$&oz6R^}A8qc7{>ujCHxL3OzNd?{3g29?qS*v=U(@!NjtYB`5W=o@}S@tb<-)F9HI!2fXZX*r*WPA?%do6~x=|?UM#US9uMsQ#2R?4hU15X!ah`g;Lh3)Pz=i47jYk{)w9qt#m=7t=44AhJ!buxL-bNZ?ZrfaLN zD`-)aA0wBfAKbB$vmvvYMnpA{>pqX=T#k(EauhY~ik^PSp9gV`d0j=)3cnbZH5#s~ zUm>W(sI4tf|M?r3t8##-;yBL|1y(FV?H~cv4KSJE`2pqMikGX~t7oqQJZkvVid}fX zX35Uc(wQWb`gJ=QdLz4%&ql+w>pB~r!1CT=3pHga(u+^P2H9_w<9t7S?KqT8y_Rk& zf!UuSfrieeInU6pZ`8qicHp`MRZEw4#(n~OXE|^GGJ%=!W3l&rXaS|$hW8ghPlYrT zAQT5@(oYew=Na#@riZ8gVUi0{334XDvI5fNr?V|q3(p)Nw9Wnun+bulP1ypgCI{~d z{n~3qABqnNl}7|KfL_OMjc7K2b0Yu}M^B6yCZ-tYO#CwrAKDoA7FN-(QlLDpN}T7G z$q|7gI#Yx*E@Q;25Y5f6LyRFY7&F(j6o(V;LExV`a(bw&SpAG=13Ha}TCR#YP8)=c zU`)Z$Y=t@6D>6i#pK~^w-h2I@(f5XMY$w zsM!v!J-FDe1UugJFxvL`jio2rW?;?$-QLrU_xmqDS$Y!oU+<9GeaM2S2CR9J@)9co z?ELKDImNhrR)ldQ#H+tLcji+iCrGDAXvu5|FG+BTej~Mqff6H)rtV8a;}6EW7J?{Z zSP)sjpOBv*m4)0VSBf;@w@}vrwhzAz=xl>P_>? z{FRvuGlVs&IJYYIq7bKGqp-7Z{^a+)zB^{8ZHY!S-sv-zHAPy$g*ro!T7vRu2zAE{8QVdFRswE44o>SoL%dt zZD+l^+`Yu>?^lCYkPpQVn^&^?b`13X^j?E&j4k>mCRqk|n9v`OKePj^1E2%&U{_&Z zVIN?Ju}HB{F_zGASff9D>m0%KLw_QTr8iQ~XqnNPr1GQ!#y{#N>Sm3khFD{~Sf@<2 zS6deMWMkg;Pfya%+BP1VGP>4;n~|DTo6#<1FFl_ypQv!1W9(ysaA|O-hq+?VqPMs| zgUR}x;*XG~S*K%kH+30nL~C^Iu&$i1G_THhfABi+j(bf#BY6sWqP^nYdL11dsBJe- zU(xxXePMrXe9Doo$RWtC@aA##Q2L_PvgA_5qNL(Jd+Dpinjp}X(jE)cy42#kid`Eu z5piX7hI~*Dun40%;_t%_$1vqMQ6vA1f~0}BfuDz$g06sTN3%uAM%hHz#M{AOVB1wE zCSas=blG>Q#z>j2I(n45&t~`89VORP7F2dw2rw@(*Kj@~cAguMup-+CtBZ;oU>^2Y zw>@87X?J*ZcyZ{KC6$fN&}Mp3=+Juac%Fxf3`~P+|D{V7Ok6G9k+mONC~hgWB$1u0 zo%2pg!fYZVxt!>rbSTG>V;N5=Ya>BaL{f4gj+KhZ>hiIWZIL}cR;-g@%hzfC`VWJY z$lc(1>UzpjC=eblvsDgR;m z1epe~hE=`0h(NxiLQX1CH7eh9^u%WT2Pqk=nc0-Qt>on_UlTh?X$+#9Y%y#gHf1kij)vmXJzJbin+P~5IC$tk3iE!Ep zhxB^uslDCnUA)6xrx&NSLyajd?yFDF$M>qAvEUd#S5YF z3^{l|IO=kGt3GYdLrJ4rRdcUe?+9;^{Cm23n!DJ(bX-NDJKyF6d+@|*{L=N8RpVafxl^cf=W3%PzXC1P$&a<;W8JD{M zd|Amc-Jhh_=5LISV`!piqqLqFpH$z$ycOT?ho?{Y^w!WCYizowKZo?Jead)x*yT3^ znI|ny?NC|%G!`~F<7T?nRxNfv^wb#nY`Bfw%1;{f_fy5l$0XunaB6xwZ>Qd?Z{1Yx zZP+la+2*7=?v9O)lC8;!WO3eUJ=>AV*304X{IWls(de#+n10E$;=OX|I`KaW0`2zr z^v>O==sd}P8cc@Uie8$6$`Ydm*WvW$xWBbiYf+ohi)vkUqd8fPI-9Tlvw^Vg>d5qT zc&8QA{KJl}O{hi5TkXmAfM?#r<;(GXd6TxLy8lV&O3yX+diyt9_uGlkOFukMQOt1+ zC7u>{wa@P7zPz2Q-7ucg*RdO=7w3xFW65St<1}{m$F1769@=Zk{S$Wz51xD7d(q8! zR?SeiB~~a_%M<6-!*<4P^N`)2&O5h1uU=zs!vXbtvAoJ&&hIO)6sPl9c^!EK6mmmK zHsJK^GZWH4y60fN+0ajLI6(3x<!t(qTY_&FSeu*M&(F*ZU(o-{0Ky(u z%b`mW3^WV4QhCeVs!H*#IoSG%;q~@MB_uYQ1UtpW#ez%oA?ipW?pKrbCt&^bN4Yp;CV}|>? z)dsoc0zaWX2Cz6~K%@g2raj#Shkl0V8bx;lt?gU5-Y8D&h) zO&e7|Q4O4D|z6 zsd==9+Lo7}DBFJ{)-Huu4Sy5VX`lxdhmA{EP6zMi0>i_hVKO=XvA)RnU;pZ{-y<5) z`S~jweumv;cEi`hVAFc;dlF8#=6UfPa@K9#y=VH)s$1p~KjBJBN`wOIPttADzvuF) z!A<`;JvTeKA{PpiOJY;4*?3yx^KqtOmeshd>v7RAFKb{cs->y)-Cc?xyf*PRoS;h0 z{!I8^p{mBW%~x}jv=)7uB5{kIPm&w{fkmw~Z^^JmxdCai}$J?`XXY(dDLKW5o}6pXYw1 zoYw`t&)XEl`S#_`OBc4*QJHc#ujU0*#6@^7%zD-hyK2UQyqu@O>*>q6$2Gf}jK&kq zPUwED<>7X1Lb^o3tlsCh3*YXJVS9;?+6urN5@;h%!q2&Y}=!vqEtfW znOInYgcQ*ZpA1U}f3I-chduP{`f(ZY4`+XzoB5nb=`qUM!vserTr{m+-llZFH&(k; z1;Fz@r!RMyzUK5UP2ZfkRyebJ)ln*ikx#5d7_)2EFhYv_D9=}a=z8kB(z$X+o09W+ zoev9x>X8IW*FyWH-s*V3ag-^<4ov|+JKayLP~tErPdvMGe^#t|>%Ada@o>cV^%6=H&3-fR3s{4jpTOgB#VCer>tzbw>sTm- zPES^-TT;)ae8R4C?AN8%iW-Whtb`tK=7MD095ZqqGGc<|;0SxK_*64lpIN5E5S8vt z`1ihh%Vgd=pH($`_kiMFhl+9c#Z(*0b4?=l#3^^_x8}CRs^42)PZin+l&N6EBKt~3a|GmN*dTd(*v_Z%Wg@c}SbZr*-n+kY z*x#@FRh&yzQ3?hYeW_w%yzdL_Jom8FD%IoKp3870LgK?yAtzN}<6q?sNNV zisu5D3Y(HeTPV-j(9+(qXAxvst~Rc`pL{-+eQsC}reYT>)W$W;Nxk3Jwm!?`yiP&! zo_+Q7Hp=l3J?3o3Ga#Dq;i=JEvd)Gg3lm1290~Ew4~L>+MYVARGfHmw zTTddg%%#GUxcer`&i=0=Pbc8h^TJAREeM(UPm}y@3E#)bq@DNO9^RKLrwXxA)?guANyo{X$k^AXgiKY=p3`6Sxb5E^WeiM_-^B3h^fnh z^dOKp)m77s^TT0w|8&3ML@N?0=|&t2kkPqwO)^N}ei$JA!gyW=Y?rSTb8Os<^Y2D8 zw6IjKx^ln1Yp8*Ru59L#5!g}w3t$NBZTd3RNOj7k)5V3-tcnIyIVfd=A{A@iue+SD zcfdx8WG&`XelInZY1y&iF_fHZ-?aB56DImgq#5^Kea=T^tUqC&B6S!r9Mc)TeR)K= z5JLDVLC}e5WxN_!0ezKl7ppRhGLHh5U#aAo3KU_w5_o4n9GCV}+p<^hPhI>_ygI<` z=>4Z{K5Ip&zit@!?dt3E>bbE>^4^8D%}qbzy2rm6lyd_Xw15OuluT$O3Sm#IkC=%4 zI+(3vXM`w*IkjpPvg4u3?qgKWThKRtq_B;UF;7KU7|I(mlQ={XCUtKzUm|sunoN{m zmm}_v`kJ6rqoHU^Cb9Q`B^Bi32o+z*=xB-RKF$!3InuUMsbOllFfS+7MJzwEs9dB# zLYTT^COf6!t$A_;()*FmC)q;N3h`ZhN(Y$!Llm=DxZcM04>ozpZu*o(a~7C-*1%Gf zw}!*Ml&6;@3=cSL43B zdxX}y|Jou-i_=#cv1aui!3+%xytM21B!Q-I*i`AxVrWE*bIW!{kmAQ@b^A>zN}3+Z=e1e zGkt$;^Rda!>r9J~&wA5kS;ysw_D}22N#gm4RWxigjITSsFQ?m4vgOMh=Q#;@w-KTB z^3IJ5+ds|i8#I!R)J2ee{xGU=$9jo#g#4>eb| zC8Z6Tnhg(wJUgKr(vIRuv9#xKW?vsqfB14O`=#X`ZcYFJ#Oumq%l`Rm2M`{c)@-G2 zyT9JLH_}#|rg>OW5THrxl;n6+)U}r%H~#1;kk%F7lVG-dq#8xn@r6K;AC+>g@4&%( ztn_OKBF2*65S=!@6)m@KK3sY~wfmeK;k~UN@eVJXX&-S`=bP6WDT-f+vb7<)!5LGk zd=D{(_w^X?W*kweR?1t-WXMgSa68`}t+kKQ8R~B#zh!A`Oh$llYZ(Cos*8LPJk%XXgP29EUD(w7YIxJ{S(}2Ca zbYE^q`n91N!Cx+4ot!BM1no)CWe!Bb5L^fhgWm(@3}oEYI6#SFp}O3MM+x47qpBoO zu8KV-fKD&j?kGKLVCi#TfOjH`jFga?MJdVmz60+Bm-Xz}Qc2UN>Nyi`KpNol*6iaH zN+B{4N!SSp&Os%%KlCXTbq4mR^j@HY5SZQ!<}y2fSrCRSdWY2VS~8r;1VQc%%NnY6 z?4khfunbV8h5;tP8wmS#6XSEth%sLXMc0EPwLZNs7f+4&)e1O3*}7CLCSBJ-bPFt1 z!l?z)v4;U{ohLFZ>I0@51VV1u5y|IM3)SKIY-vFG!Zj>nVL6c!t1NXz;#CNv0n6tS zNpm?I5Z^&YwBmBg7#3n@Mm|u}5_g4>hrXR(H};0(G@mC=xxCYNPs z*~H@0I>yOsRm}v&GF;V3g8d#y`C>wepVCk5%$RPJHQ~@ z@UJV>p~;m4qxBqp>`hy#%?Qql)yozj5R`d{&c%bsDr#b3B$TrIRV@`|vnL2~>!&Qt zZu3xWHI!o#mQ*fQ*{6uCEWBSY(`v{PSo<~hK~#m|Wy;nYa=%`bV^Z5r{MXCZLGTC! z%(_h*it6Rp@&-Z@6Y>nmZ(x8q4=4m7*q9MzTWVM+yJ=zqt<@qicLFXsKPSjYS z?dhL@c$NSx*QF|EG-jjIoLtyji6Jp%{YAG%1LIj`~)j<~$FOg%|B3 z4S%}pj-3FP1@+B^{{-QYL2>d-bY4P9tzO!ddUix12_((bALv6_(Q_M-C$>}|W-jX}_UYeLMp!r+g{r4fDX<=lV!|DlQXGt_FyD^VFjPANY-VIy zwZy_?MUk?gFj8N-v^&63c5jQv!Dn%+C}zVB@@3+8$o7r@)Qg{5y`3Snx-mhRwfVNGE&Gm(fPcLGmYgcjf{?UdEEdJVgl7BP`6GRfRC!uBfBDi@?Pm z6j+!aHF zo!fM3Aw(?_nw$>NBn(f@Veb?!03r%Y)4%dOI^;W0lLP{yHcTIZjsY^TD`_-$!5g}b zkmHMO<7&1|!iG;X$-M?kVJ=T)SXRU=kMuFi%_k?~77oX;kMJge@ooLdmEQqM7U?Hz9+|)r_ks zWvK?FWCgUcw@2JZ9E4Sm76{R}y-{G$kWvDueJ+yx@FXY_mXLb3Uo6k;&)Wh0%Cl8X z%YeA5dvgQGiq+@i3El3UBmcqhDTx405n!17UJzI$mwqN$bP~$aSpHkW-;OuT6SG`LmZtev=pb2`j98Uo8FYyJ04}o|>JV|e7Xo{D z;Wf^Xbzf?_J^4-n!)z-aR#saLhgpzJU^sof*I;~F2VtMv2CNO-fw7^%{kT!#Ixg|r zoRj!YoEWYKTSB2E^9V&*Bjzl3t%;ldsM>mP2pN(0$C+L~b-^HsThqEGw7?xZ63^L`M9gX#d*8!EDMx2*Es{;fic6q#~7au&VVlL}s zukD>5)DY|(nC^{0c^1-Auu6<3v)a#owx>dlcjk*3ZU1Rmo<{7Z10t7s0eI%WsNoWQ z$9YCE>v@T)@f8(>jnC!6N$Xykake6>5A5waN&6M_#X^nx)fJhRh0Cb=4;6YUS%v)w$1AF*(^ZK{swAn1(H+kcs>7bN;=8Eg=XHFF zPFc|!FAI{gTH*VstIXR@f1}A{DYQGsg4)B!&A&MKMIF%@0YtCSq#H1QJ7@5 z!(cCGf=MO9gthoHI|`N9kNsi&hGZ1vXnf6G_HXG_UL_&KIrQd?LB%TWG+ z;ipKMVqaMHai)0Ulv;rB^(jP<$`7|DHLi%ei(x_f+7c7#7fUEig!V})kvxE7$B$+U zUMQ0L+J@;=09iSpGg<>(;zlVfsQJ^0=;C%y@+l93IUVPPh_2+BSZ`z|gm)N(6g7`2 zV<7IA&pgCx#;dX%BsL1K%ERKX^2hDqHJJnxnZ5=!^y z8D1uFSUqkI8Ja?R%oBkTnqhvZtfl3AXilXc`HiSodfWt*adX?XaLQ<%l==q;!#@B3{1ykmWB7LA)N0U5qpEa5U4`b`0p`eg5 zVm$`p{`;lw}HSVS@^^0_k8lhLIDrl0t(6AlEYw95dXlv1w|@~i3~Mxc|9qjbE@ z;|M7f9{Oz|;G?F7*~q5XLKH7~LQFwGkC^JcG*JNOLhm-aNHU*lAEl9o9-KyshQ}a> zYpB}x7O+Php3R)K+^RgvY;zQE22NO9isYpdWsAF_$X&ZFFX6+DjszEF)_D#?>bD(A=uhTmrpf@3;^)Sq&$c8432Yri@ds3m6` zp|P$p(H9L#WfHF@zyZc@P7&oFIl?X+SjHS&WFb9S7w%{ZW`MXbMiT-arxf@8UPx=2 zSRweAjymz&un#1elIovkA&Tj7E`p>zO*)i(DYI~ro?iR&epnc$hNKfOnHBjjb!#at zAkA%Ewu11oXLBs*m-*rpbiAO-gK5o9!Wu<8jE0Z?q8L@|auj%?Kz7~tT{N@9roqn4WT@6$N<4$oSoAbC&+KI-8PbWCG$&~ zP`HlPW}1D(zuxm-GV>@eYTdSx&in>}p5CX>K~phwFl29STTsa1ar`(}6iFstgi6B^ zM{2Le1EVJpwhTnxy%Bj7+0XXGidl8St{0&d1c*bhEAdeON{k;opy`^6*6ziN0j^!>yfdjJtksY zoo8;6T}=x}_RZxX;AKt-kqtSeeINimw}&6F`q|W$WEqkpyEWvqN?@KPpSogt@QeQ za%J>&MS7@REIQB)I6umg7cK`!_V1)B)QP_Nl&gL({5ErqjRtOMsgC#g<`PbY-7`9v zoGyGyL%?dh`tD;{Ftg@eLxv(VVJ4u6Wrl}w%dKh&!FD|1uC!Y5PshP0-LGm!8;>?+ zP316@0AauJI3Qx2yqBWO>XtcEx$Eisp79RvnxPdKYQ>8eGkl~iZX&jz+2i&|h$i0O zrt}R!n)i(gzo?J+2YV%t-9#qn-$bNNQ@n-Hmf!0NajeHOAd(jV#4u%{Q^xIo9J2U{ zzn!D?#A}tA^*@kDUG@#>$~-avslj)XxdQZ zfm(izXDEl>-7dS3LrD3)eU^7b@QZujiCP-nd^7PQ{OEmB zpltS^3>XzWPGXMk2i}q~F*CTgs(Xu~yTMoKB1CcBBsRbmT*gzYzx_LU=?@A+b#{x; zk*%my>i#l^t8$Iq1bY6_pLWY!rfJc=g^)sqH=1JiVE0NHAtBL6t*}+{L@^`*o2mnO zMX6{C5jT@Z6kTV|Dq^{U(FZCTRnrTPXQAM@;j+hA0=Itqkcb|Zh$I%|*o$Xs*7k<*ev~%rNced%HL@7fM=+=zF>CD`~Q*Rx|X|h>rZF1KZ;;K)Ca~+j(n^iHThUFb729to`;k{Njj&ktJFBC%^CrTdl$`A=#f-Qa%QOQjc3X z;c}{nkWg+J6MpPWDQPJbYKfJ$5P#+>9UkAp=4@;p;na6t5k(ItK_TK8k60~CLwHzH z7DAwyVjo&6V#PUOZp3FADWPyd0qo2vE}GBb)=;FH9>H3aYF>6KQH87m8#)q8rm<+K zD#3V%RX$NlNS&tz&@ruX(eM6UfwLT9PnMm(tJZN@h)wZ*y>I;<&AZY!LC6mu<_{dBBO}~36q-y zXXKb*wd^`wByUKV$?nj*`rnx;gG2k=k4t8?CzO!W$jh9|11aP-Ai3_7usfrHyo?SeHSMst|KNLOz6xZ$TqPYfoUb_lC4ItvH;oa%5fu!yL}djEt$H{+mkKSN zhjKNM0c$-0PXHHSVPH@=3ckjnbRVSB?PYbzxeg=~UL+l5={?-v-g^M>;0oNmX&;Ve?5J9f@arV@pl2H zy$LuE5|m#501yd4C}j$|PNhDh|8x|bTKM;T3wi*$@M+z3B4bxh6Gl8a?K;Ht4P4`h zg;F(@S{D0ZZS6tFsQ8M?Jpo+*^;E699IY5)BhN2#;OGiaB~NhPz=vRaoYytF3}BdL z-t?h$#cC-NIDNy+r?}=dTP~MxmDT%9Yb#&->B{qn3xGD|O3023?!U2&eYhoa$-fIb zACJakW^hgmiE=5j!*PK8`^}IRDKi188fyRqD$F{D0`+*Q$Lf6cBY-4M1U+)p=C?m$*20$ppe-n#2y^N7b;kAI%DJ96zj6jU(#xxVTH`A_qYJ> z<2Yk|pOo*p7rp@!nD4PJh5}q}>v@PRcRCz}DjT3wC%gb;)``AvYC%d+nyx3AUF~(7 zkmCuMw>#kPl>h*`j(DG1p8)=b;?i%N1vY1*v@JoFh0N@89e_6x1)#@PG;QC+D1+b= z01rdq={$FSR?BKLr3vIbbsW7ANMqF{_b?1&@2y5OrU#68hx)vv_y7cST_fn$ z)y3+=BPl0RHSCM@v^h?1Z^%~BhX~E0MFNL{`f-p<9?itcEF}h)ByRu~18s-dL)~D$ zN*Bp~nJ_~t_9H3Y0U+Xy>SS8Lp*Sl(4R@SNU(t5C=zQ9Qvue-Fd&If4hsYWf({dX) zFO9MRD8K5C@a$ix>5e}t9smq@{i*fa6T$cPZ@OCgU6ROGWD)NZ;054nEc^zDLL!ZZ z2X@530g*)jJrP$0w1RrK5}Hj>i1Kq6ty^Bm#CU z_)BUdNXMggD!4!=8~@C5CL^&njz#fVM7D!(^Z=C)j^}cS*5)vzR?+^H zkQaczCCCT67$@dRV%)1I87B5{r`8-G_VV;|)SoG7R{i zXN2HYW*@lh;}+bHxV9`hMJINMaBo5bO~A-|djWjMdc#coO|(S-Zq*iGCE|T3lWW8_ zg|bmCGw#7yi{u}d2E7_$>tZN2p(wrwI2UpZW@}tf{+X%|O^wJWDZ|ezg&5W(>uWkI z$eDUQ2C$w0-cQ%_lCS*)xvsU1OVlotb`@MwKzHl;%P0P%>Vz8T#V+sKw-LDXn|YUo ztE(!2D_aajW6&6?sG) zCG6e0KOU77;*5jV$|%xQ@8e*)xIOOy@3tLz&B$n4th=SgXFER8Qq3*ka@)*%!+7@1 ze#&e<{`T$c(~LUF?#@uBG-Mp^Gh30rf5RMEfV01)!mC=2Aew1KKe%d1o5C>}6rYWN zz+%c#Z7Z4H@u-a;iUQ3kR{&=9x;ET_q#_?BU%!qd#=OzNDk?y3j-;D%sG}%ZwZ}PG z8THTiizG2;U065F3^X>HO8k4xhAsC8wf>V+&YQMWZRg|aI0@^Tuv*(&S-00pKkks` z@*cf^lln+}6n4T=2T0eFEoTex+eVvlJUPT$Seh%f$Y8YJ&p@44$0G9U5Z-%&7JO2o zlb>{Pky1B8wc+j3Y6v-H`E2X?N|I0kSokAM0FT1-ttL)_ksQYXa--s)$uBe2K#(QR zPhSXbUt%6Dn?{tZ;|4(#Jr2()r|dm5$1o#G&d=vPaAtHN18Y<-Q{3Er)J!Vv9caho zaXJ75HJsaSw$D3k=GyiJU68SOEk#EB`Ibqr`Y-^nU$HL(6B3-@e`uSHIiLqA6Fh4S ze4#eCh%nb@PUe$L0TX!qf|)EYE#)MEJAf%A6G|;NdMy$j%}Q>q5>)?HR|NnE8N;9v zV3&~)q~R`_s$310^()A(;Km%|S-Tiyx9Rlt9RTx`wU8|Kl3*mziXwwh^aL624bR<-O%!5&}6W%>qKlGsoCL@(SrW1kbU6Ii~XHSc)Ka zvA=cTn!B_NaTd=2v1gcf11gv9OVPq$WE>8eo!`9JjP1TSS z2*5vlLZpc%@1eY}y9Hr%NJI~=IqGphzga_N~Gj0+ZEgbaW zrK3kxj`6gmmKfe6Wan5SG8DPEa%ty$EO#8TEE4JiAJ6%v1_L!zfLZbD_y$Bq_K9;6 zCp1B_Bm1&Kv`|2BqG%8sI^tjFgd0Qa7*v2!dKw??I9k1}AtSNPPn29a!6K z%@&3|g{+(WN=O;A>B)xN-XzE;LFjE9a)D!ZN2sN^aRfEYP%=r^n8UXjw!22@j_1m5 z{aBm&kA^&XeJm!UDBaDvPOg>?U}H!ZQ&cLBUueZoh76jl}QdQ9&C){YDlju6Zh@n3r;abK26 zXF_aIps3@5k|sxSCT*t?^jxCcR)AkYK-a;T;L11d*za^vR>jaVQSUfLAk{csKIQ%O zoGifsH^DjfSEap8L+myrj|I;U`9aE7p^Z8$VT7g4c5zB1^FFkB`~q4&A&C@5O$i;w zb&=vPrxkv;J}9!#aCPUO+K_q($=mpH(Ekcl_E96A)|2&U#{%yYv#!i)TRdocg01o?IaQ+u>yVX$!@0Bn!kH^SZGw9CvmZandd zb%#46VWU|n95>o`7RX}I0%2UGv<^KL2%h@|Vl=(jcR5`?+n{)m^>qADq;TCL1}ym9 zxjg=fp(i&s7dlB6!goFK?)trJz)bl60_h7$$ZSznMg8Nh8h*dPv_?vkbt06X*}g3* zLv<`UkdWewOTr`f1WtiP0%KEb6mtuW5ZJ^~J({unIrr3=Dx$JeR;_IL2+M!%5z_Fs4~W-mM*IDvenUI|x4;0-GZj!}X2;UTGN+$eA>x2$SmTdC1v#2Rqdk zY(syABKvOLNICMrxleVG~1u?#_J8r4@q z7gNMD(8UdB1{LhzD+XYJUhkLGXxjOd*?9wt!LXr$Hl)1qM8Q2#m3~kF_B?W)|2s&pRUyBN^z_V<2S>BK&1Z?NrYFqfr!y z(m=qWy|B#APDuP%nY43J>Tw#7Mb$R;wOS$>29>|SErB~#%QYuw@uZj;66YG`Ala1> zJ9*!TVlxYW+RWJR$BFeVsR0t&#Eebr=Yaai1?I@@{(EJLL^gr)c$zh4eGprkRar88 zeShJ0o(!8HFm6pl%m_DnHO3HN4Jn<^FcNRNDzayf`&qFJ^ukpo&vDNo#{?fPUgnW%rLS-(G8yj977QkmN2$^XJME`|fy7_(Qox1;X7wSL$eL+)9@ z4X^8G2bZ4&DTdnp4vW-{mJ8s=zbHlp_l$ ziW`>_BUh9ziD1c6V&;f4vWtSnpqfRV{=*W}8lwLYC6ccNf)S$z|J!4gi67L2oTj{f z&D^fcd$@c4&vma=@Go!x31O9ERJpBxoKS3(nuua)`VXn))BcBlIlmv4Ep@Pa@_v6H zuqJs-$;)b`j-D7GO};Bd2tJ>*F3aUK`BpxZWeJ{$S4|P0X|M?rbcJV7zOd*RIpAQ; z5l&({BJC>k!LXoJw2UKV^Eh~<7Bes7Q6Act8p7$|a89l($>=Q|?Yk`5OaMgGgZ~&v z0?P1693|r0D=1Ku1gVIEBDml}f6(t5)yk$IbWZHQmZhpZl>>s4>aiWFgwte=gOb$Z zDuGrTt7k5h674Splr|ctqD54`!f6HJxQ2=<>S2%zrxwS6+5|=FWXtfPz^uG-zEFg| zA_&FIXzhn)*-%Mqm+>+6#4OmodzLCi6#qqf{{vKwl8_NAyteaq;@#NRmB4c3Jhk`9 z9_A^jLN=39xe=<)v+ioxzZ$$Cs;)cii3yFTGrS`|7}abx8*ky@a4}WGlEY-I4hCl6 zSjMb}TXGefcgi7M_3HGLROr4aZ?aTomN&tyoKYdXRo*p)uH>`+fC77R5|L+h@}mJ( z<=X?7u?|Y2+?H#69#=CGF@<5>81K{u7b&^mgj2#ug%eblmrT1*XkJln7_Z$Isp(u9 zWv|1qsJZhLBt$ooDgvq2kDw{t1<6y^9vUuS)2)9&hqar0>`X4glu0_DXlzDf?_@P_^YiiVBjj=_qFtB!)5h)`Kz-F85-LYYMk(vpz4q#N;Sj> zmm3IVx~HlV{%YIOMp3GmuDM0C9LfiOom9vd$s{-&B5w{{N3WwS2)pOBIiUZL(bd_Z z&)g^8TPy8zR(G11As|c$zOKmW?hZ68lW?Z0pz(hA08~0?{?Q zv3*g@Mh=ETGIX+|@>Nj{e7MI)_m%S=Dg^!$oL*QwUZY*=2k>d`v+zdtjg+XOHd{cM zmO>DcXDgFZH+Cj2b9za*~&AUWct3~K?W139|KPKZ9SNWimjRxb}A zJ?KZ&vW!l27`%fk-UIh|0#$-AI`SgySR*-BgaQqu3qWBh#<-IN!b-?x{RYO&i#ad> zI5_zEdJKS`RL6Gb%jYXUwn%DmG|%U|AC3YdU#jeXzc}}0!&tcEr~XdJ90yP3!qdq{ z2`X;QaHB( z3XQfXcFh95bV9hRlK#N{@&6#nnyZ8p|cSNI>)i z**0Q<&F%x-Qb#L4q7?%GYWA{;+M$5Q@u$~2yUg$mUQYa-Z7}S9YhgzDJ zfU6g!hHVp3N5^)oxz)IUs^y1E&76Rh?+kv7iZRqN`)wGFjpTv3Ar)1U905|qKvncF z_uG;`QSyIEDKipn^cORY$PLP3qL@9RdjyghNZnJ)a_WZps{p(Yeb_(7&as_eovNUb z&b_`Bi`9Q4R!gqgrDC#qRy40p)e5L9j{o1GNmr0ZvI`W$C zJ+#F+gum}d_|1IJR0vck4v?9HMN#_DNu>X(mi0c0_wnfbyt+5DsKVO@q2V#h?j0M` zRg45T>+h(imlX~FriPJ^cs@j)3&SKuVv`4;5K7WJwb{fnI_(gKE3M)f9=(U$Z+F|o z(rR8bBCVP5V8t}Gr)5!CKChwQ;1@SbO|BRKi*fz61g&LH;Z+h2 zFSPW`$SSH=Uy5=gYji}KdQB5y0vr6AEKN55Dv1iSzr>>5UPbbdHlz#6*lp&%0Bw7_ zr8+%SU@T|VT>T3JaVHG>I_nAiL}4f%SoF@BXtkAjz*FSwC2NSGVJu{ULEFQ}?pTl4!a!>zCC8{X#u+4(l$ZJsWQ8?KusYkUgaV7FF(DK+H+J_LII+V5(iFKLbq5*S`3caYS5!O563wW#&tEU<;zYR}ci9Ud6iW_qGE zp+c^mA=JzNF$Tbj_Vf!p36Q!{AmpO12)sllGNYK3Q>YN9`yC(vrKu>Si+CvD6hOnf zlEYdq_+30kgq*p)FKDHth(OHs?vZ8(SpuzOF-LR&L<;$j0^M=y^wIx#10XI`*7S<0 zP*pnCQgD&0ViPMFj|2wwakWT=tP(C+jy;UOpjLm6?CC{neowu8SQdN1G{4f7{R7O? z+??irM&_lxFNEhYJC9YPmuy`~Dgk&YYW-eAY__~dTs3&$@6(2QZ z_8xHwxlS;l-onhdFyr?l96j)J;cz~p6n);v3*;i9dhKH)MF}WPJfYn@t9YCraK9&V z8e-~s2akqy%upkH1(K92M2}Lo=cEnFKn#Lo(gmkgqpQ&facJz|xccb+a#DNjBL(}| zs!2q|Bz^;6$%1UYBiR_Y5ZqO@Iuyyz4Liw2j{mZs98d5^iTLS4XO@VQpQQDqd9TWI z1k?yqpy_M=+nkLc5B%gk@%&W(kZ%FBvl!a)9B6UO^wP&VBxx(FFSC2SF6C&_ya1U| zXgu#hZVruA=aI&X6NU(gGF($M*lJUahgVj@buH&x&U8cy6LI&I5h#UyAeD|UaU9toEb1*Bz1%!I&+~|0qj0#21>rGo`E_I8bvD&J z^2T8EZKNn>=`KZDI>taGaf=+|fecl~K`|RwFy42((eQ&uWO@SX+R+Y0{`HfnWU^cL-^S;e;fZulZUc{lFd^ z(iW-JKS)PhA$Qbu{n!kX#^4o_EvI-NI#Yg?<|X>FMy;*+QDtqQd($=P19&~WuNs3o zh*7}{f54uY79p}sZ3qrIgR5+pF0L@G&v8YJ-F>`&Li9fjsj!PLb$K3cK{(%5Rb7}M zZ&KdD^e;9A#q}n>+2nlVab{$HB=QGuVDR0o5G3|UQ)OamcQd#<>)vYX-l{BxI!t3~ zx}k?mWLP{VBB)NvAS5EV-umS5_f51*=U+|f-arHA+Gt-jR1Jw|)PV3*RgI0Jo#hQK z6@fB#s_m~>NhS<;CO>I!77~Dvxh0MJf!lH%(6Ha{@K@W9g2g~4i*}rA7}zslM$!QK zJ%Dw*yD$g)?HIht?a&VZ;?Tg|N&XbZs9E{ASe|3lt;@F<#SLip`ch3xaHuL(j2Hv% z)=M1ey2Ng<3mpd`GortYIF$Lz@gX+-y!<{U1H{)hRyq5_tjkFT$@E4?!ZEx~|;=cB?Y8vr0`sQ=Vof*YU14cXy4O z9b=H=E7!&`FVczxd{%nH4VtUxGn+Kvpot-)*=+poXbF+){K;pP$f5J}9MVam#+GS| zn*0~5+sDwa%?y_zZ^HJ2w)jeqJsvBk_;m@BJa;2UGmkZmn;JI4+x*Q+Ib#o_@df!( zC&Fa6Q<@AxzO{gw87!c@_k!K8$wX%kWV4z?zwRT-S77Qs!+4?|TS2hSZMy{R^}RqB znIiR-G8sdRZ&*1#y7UFHSky`_b?~Wtv4{mZdA7jTKW7zt2%Z>Y77~F@=WK`?Db4y_ z@mb7mybtC6C3Y<->+Pg@iR0Khvx{r_;l+@y5{! z^xED5+sDDAwCm%d_uVGZ#|`nK`@i@bT(s()Xw%t5DViUo3r`Fli;JJkEw4*8_LCVn zvqyWPs{N<5xHg%*E$1iK$E_O|N~;wwgHG~?nh(F!L$(>$?t4~FeJlJi?^Xl3J&EkNFLID*A^cK9ch#sFy%f}(%_sX1{^b@I1A zgf+ZiiWE3c`-~7(P|J#9YOqI-r2(6h#v?uGGx0F?pl30hCh&94(YmN?E!_wePO+&M zHN9Ie4DT-~hU&0#SSAc?^zhOvLxrrOo+tW*uKn$e%N_7RY1}QdtsPJ0m!D4*iL1Lt zW3=qnFUdIETNTi7lIj)OA8^XAYng?gFQ?k?AK~24N9TeB{dL`stRO_Y!XyKD(`YrL{aTyj=mQDne-H5ndKv&bC{aEo z3(Y$i8+4Fy$EwwOfz~nSwUYqq|FD?@rc75LI;550w7+&XWGnb80QC$cQQ~5}?M`^+LncUAB)O{aXqO9}Q zmg4$G*3a+!=(dk|wylDj?a=rN+hE!*iXLe7i~t?v<{sDh%U5DJxDW6$9aVCqf4)Hi(-iNui??}|!OL%ddD8nBBKrL_ zgF@HMBghDtKoRJ(v4Egk?0Frgeb!#KwN$F%X?X)whd__AW)CTmxXu{qLZx065Xd06 zS_Gef*5}S+en&Ry{R{jr=c`E7;9<}>W!5IVQ9FW01Y)Lf;<*9X!V~QNsptXHCTni6 zB`tG(*YzRKrE3RFHXKew3k%#2DLQI9KkL(~5s^8q+myvL+Y{3ccH8;)hfViPnlgZQ zewhFDbckic2Ic}kz$NvaP-zy>E58xM!h?ZMyUGf)wC*c_EGmtZFez!ZE<`%_KEc+> zQEfDTU+Fqu<~NUC0+L0ZcKNo4bon&*$#aS@_zt8hTKlD}L_s>y;bR?;&w}<&H=N%} zZh;eW*Qyw@(>bivp9IGzJOBoz!2=hb653&W2{uFP?jC>$ggJB>=B1e|5Pr(HBRbH>xL0{`JyY||7-Hh1 z`Wb=x=cJ}fupc$+0L_HKEt}yZP0A4)MeHAY9K1yf(%X4Fy z!EO69gNaAw96Gco8YMeQxqZKvY4zfz%bkD{+e7=MJ!O#^o&}YFtab;s8V@LE)h0%qZ`ZhhM2sb6L zH~KA9P{_*09}lKH04d9P&32GEuYzFKZ;pCF(=CUHwmX|iJH1d4KM0il1{{%{FA`Sk zamBoHDK~Uxt3d8$Sy;?u8&N0;jNKteaZ;iS_V0I;NHOBB7D3t`=2l9SZCk4E8}GYB zA4f}3o^MT21nzgOksT`^%KHmDF^FkgWexx6C)q;Nt5A;vL;GK=W;4OE3FxCF?HlVK zD79~12wTsuS|~PDO@zj8Jl>81)X=5A2{+(S$wsC-W8>ST(ww!;R$P8g1xG&j zg+z6pMvM}k_+uEj5t`X%*-({@eYRNWUsJ=+3$b zkU>;zumbig1wQbA_bq5%o7Tx0)VF0ej&CH#8Qx1?;kH%hF|$Z(w&N4Ikc zhnhDaKiC$rY9Lrdw>`^%Ecw01SD)q)AIv-!Go>qpBEw~MbtAz?hbOX&Cq>#R#Yiyt ziC9Qnew6vPp^@mhHuLSSDwun2&Ss7)(&L#%D;v&yv?F3?_m_HEHC5{x2!1*)-XrDi zT#Y4NsVhDN4alHT2#de$D_`XM@p9#RmvO_UeLoYSch&HS+cFXv5!x>{6gNn*M$Nty zPTn>x40bvsv6R@BW*q~YF0VxQ9`H-2izXzR$uzcKFZbH%AHW&;iV3ZCd9m2(Q7vo? zX(>F)6w9n0L{fJ@s{LuSd^DuhdijSL{xGgJnZV3h1ad%P-yjf3ECctH>96rFB=Z4_ z^9G)c+!wY{0x$=^4V+iE*n6Py)++)GL540OzsNNmc!VLCuzyY6FcW=j+{t=ElB>N$ zA2y9_+a2upz+fdl?PEs))QS-Qu!JHSA9_CzV8}|VCFPI}5TpY&Y2;fk%9}O#zF={G z9%lHEqcUK&t|=w62GNwAtrQ>?$-V;LoGO3z%+HR0CW(`HDk+!)PnTqevnSZwoC&%DoRJdf2oxt}ME zwXO8_y4pv>f5)<6J4K8X(PGIESWRB9Z2osfZ}|#=HDb~g(fNY@NJHI)l1Ty}e5<0( zgtYhMn@M?T)I`NWCi6M&i|YWIEbm!#LCsCzo%%VqYNEtK?Ymmt4~WLCa%F0Z(!fF+&_TcjkIrt-mKl*@#r;|=|kjx$3B^UhDW5UomyQwwsqlBMPoI`&tKQbRx^bUkAJ`@`zw;8yNe7LtRpTsUD_tOBIu z=6!{8egYr+0d{-WcHI}iTsEIrLt|@jl`P{a&b=K3O*lj`={8|M?n(|-epxd+(Sx6` z@$is-ce{!2c((XiZ4o?N&fxyyY90x=7cBuMLQYduy`x?nOKw=91zv?up|I@czAUDvL}57*E=}73a+c4Ed5YGCOF(L=FCE0fe)Fjn|EW)K>q!hi(|4KK(1(f{$ zCSH=^w4cH%mV{lGK`ux7f%vn0?e@($zHlM0Cg!%L4m^xHM$*Z>UE>j z@)18z)*rv}hT~*sSG2Qh{uBGEfXQi4O*EnN2{t~_3L)tCD4-Ey1q&&Y#X^6xTd1Kb zUxgG4P^ls^>gE|rzDw{G=O9NX?uv%yjg$$7+AS5nf;}Y@2emP41Mr*YG8KZTmG)CZ zVQfxY=qqw#;Vp!dT4_78y+iU61&!k#nOU|jIC7(bUBPAWFi@+Ao|Jq3)EV@9P|BMd z?6pcfqx~`&dul}3N4nw>eZG07K42%4E)n|=f|3we!Ohja9O?e{sC@Me4T`VZ-=Z|} zJBe|zlvzOwl|o7FkDRI6PJS$Cl-Z&9Kbt*}RFx}W3;T$Le@5J-qaaPKdcN9ar(YOd zYm`-YP52BPPklb88%=bM+uyEZ@SPWk0|_Ov5`|c~_5(#D0DnQnW~s+t>a2Qj1GY~lp zybeVzeITwZR1fGFn(JQo(rtH|KNvE#ag(Kz7L}0a=v(?F!8KhB7S2js(bQHIzDk=r zo+|7uLO;@z6BU9VqQ3*+;OgtFy90lf@{!4{XDdXbqz6&|oD;{;D&*64`P;x+*vzUR zaUA{P5iD-*O%k$v_>Dlx(o;|0;N>PYR71V5t<}UNBLK1nm=#E>_zumLE11#{uJ|= z9>EbJYR|(cz?|V=R6&|d&XDO)ZE@%2tn-ilfpVuyt9>>U8}MpExBk`1bc`JSPb}Z{ zR4^x=EBWi1on)7NBLJr8YH^yVWd1XFC4I8E$Mc8Y>71BFM-1`Al&i0F{cn+ zeY%u4Fxs>Es{!!2_hGh9^w6~ePtA00qswIlyppQGA=MQUC-js`8ULS(fa<36z&}W4 zkzM>@e8s13{>k(;=oSY28VW*+C{(;2iBd#!{?$h<-_&`*BW*1^+{i13dn^S^;INH@31 zHQ_@VlrYfe7w#IV*kWN}T|V=E9smsL8_pxO4gRaVfH&_yNKa)m_*%UYs-(wB1Pc!L z#?uGC9WA55BtG``#=0I&IU1gvoVp1_=0j{TtKd5l(JzCo{^y~5;hrjQJ%lr!<_ex7 z^ZC_?ni<&r~l~)bp*?%P+ooKGqgr-T%SJ&5OVzD_#E*My{eD8SZAfo|W0~$uY0$!6YeG!X# zgew9QIrd-z!qLwzu$<=Ug_dI~n+l<3bz0b0_9Y93=zq;~)Pp>oSu-9Q$dgJ#{7>?m zdhF6TYYFbQw~YZtIe5ge=qguNpt>tIc-Rg5y4x`xfABSTfNp^vyGi&CGJwon-{;Rx ztA7;FxJ_}}T)XS!S%97K3ksb-7dUpuE<0|&&CsRQ?!7h{x3jHtRB;`>W$4cuvr$pg zngDo`u8V68)te+zpT9OBi349dF#sIZn<_M=e{?gA`w*BseR->R0)7)+NnzoLa+fDp zVlD+mp~xVVK1mO`JOFCezC+bvrBXKOziSLLQrRdt35kBkY@GSyaB=c=HDdedO{GTV zWC4fZW%<})C_SBbBDaDbie|!6t@-)Hw$gGgWcquuBMC|e{Y3+6roNV%#@fpl=m9*; z7%-}lPZkzP2*;rCzLTi%FtuhHFbAiWOiN^CW-&{*JPpHoF zyCze$djB0uLIAeTjm4OXf!6I^P+7lfL|Q(H1lKBqx;|+rcHcO#eaKZVo8u3yks30j z_&ABAU+>qaMId5--0z2U=5@*w`?J|yr4qH(^yLz3ZVMa>UAOeELAv}VMvbBLF*hz0 zq-7!^;GCRK+jX3vNopI$ZRs)P6YHkOLDp(xGlt^0qQ1ND>s$qUfBSGbXf7>0$4JXa zPQheP#=%L&$025v{Y~@#MywZnRWkgdVb)1`Nw>;_u_{8hhoCmbN4VPxTGfH8!+`-5 zIAU@Y%0?rG;UXCm1m7m`ziqg`4Sy!7u{O*%{Mgj@FSgBEUnk1IZ93PH-SBBm3;FN6 z#~!(*K%@$X@sx+N5?Tn0S;CtYr(4U}KV)$IJ*p4xGZ!s*l)5xH$ z(7sQGu(>YInf`f|;7#jT>&RWT4(4T*Clsl#7om0+<8+=BE@-f6WWD_YMG|_zW7iX) zp57j{Kb`=K9WRPbwC>;{0|gYVo(P&}?sMl7#h{Nucb|K6I&VPXg2=yHz4N2c(y8@q zVpOCdXU2+;xMZULHyJmfvv%Kr+Wmnqq-&POw}jH3nrSK|tSwP{_`esS)q}&-d15EPvlIq@AQ3W9*~o99D~44hjZoHOtfn{^s|q;^2|3nVu^Bti>IPF)C0O z3MHFZNjQL^b(k}eplfg@sk|dTl`qSgy$;uGhWsXmvCW}eMOECebUX}IEr;rrRGTAl zHAu&VhhL}X+#ACCJl=a*cj@Kd?ZlegWOkXBJKn5qcOiRD#<;7e%<+0lmrfup(b;4q z&E?L>!csrZO7aG#S&_Z%I=JQQSIU|a$OO|MC`A6?unnG=2gBO`TxJdq zuMM748n^$kzIl@-isL!mV3pk5Vox{mo-WVq!9BS7)H&N;;Z)yZOm%QlXj2qFfme%) zF}`T|nU*2-gyjYjS8TPpdC_|L5i8HR&NH_WV5u3SNXKDg?C0E)={4Y*o144)d&Mwy zIHu8E%*n0x9Jmsydk}#65G~#$_7La|f&|I9>BkBEhKg!cizs?@FcUd#sHK;3YZ#CR z2U%$oHGUJkTiDe(P5v*OSX~P>W6ADa2AGnqB7~TriSxpog(93u^iqk8`cjVyRVDGu_N+sm!hgP`H8CRA~Cs734(*J{Z~4+7R?7<5h^lponB?7?vO z2SUyJUJ5x4Xe}KnKHYSbm#2!(@>=&f>Je|O7fOUaR7+$1a;kVQHU1d8Iq+|&%~iZf z3luigB=|5+V%{A+Fb-Kle~Uw6VR;llsAk{^mgpd;T9uW1GvvxzC*go~*_ko(b}c=w_ohL8AePX)LW@T;?tl;%nRvQC z_2n zxJKiW+))p{&r8#9>;%@Vh0r6W%k`VxvLXqybpkETQ}*?hvB^AeQR^Z1@YwR3gJVu7 zFi=>-2`9=e%>}V)_`B+l!jaAtYB^=Z9JCAA`_ToN!8Qen(Lkocfz5i84XR-kpCwpk@)G!$iJZk&!)YubTYUsdSg>q`&D$61If!t12a!D!gItV9gYyyRe@ZqS zg*IOg_skj@Yw22633V)R8*{s*U3>b9QQz^I#|6<_A9-6$ zi-V7em%azM%@vv=PF658k;Di;E1A=X1j}#x??OoxGliZQ5X|nm0;)c4B$G=`a7?-S zv$^CTp9M;jlIU~b(S$sBbf^2(KQwf#jmp$75{wb+wVY%pCEfGfJ@oYMNnuX3*7eJg z@I%b&-7RBqAO#a3UD~Tv>nts(eIW^K;~BMRe@8`_<_IFY zW;)&O(Ypv{196bl;|Sx~zvZ?levZxU)!5mWIN+?sW)1}f%?wU_!>pj%j5VAt%CLT5 zZai5%L(ncmup-}D2+vftV~v}gI9_>fO)GC_c37e@H2^xKT)y{iz%KhLnt~0wQkJcO zR^HZw8&Xn3a9hc$cEHMQK(rgx(3V3J(xhgjG<4C|XVsi+6bet&;PS8}*E_*dIh-Z$ z&&KBFQ23rL1LC*0=!rbPSJ74oF$u$W$e0Rst-`gce@sGAmNjo5sr+>7E@sLgbfg&l z?Z#L=CdqsyOe{w8GfPsItSPrmFakSjW^hJ|k*9vZJiTlNV>>)N(XnL$ODOfHV$)V8 ze1Vm){-_blp&vQz^|a9vqKbRf#y$UqQewU+kV8sc1UI~L4h!*9b8VJuq_AdeUn!-@ zu8TiDItsYI`DLx0;y0iUzI;u^Ku)O1R9~ax4Eq{cNn5%Vax*~ASMcLs2K_eVbs#E1 zL!P6vz1_GS&BAbe&*5Q20@jo>yS0V`Jf0k62Q-=kjJ-@E#tZANeyOowLs(azU}FNq zsKH+ehrF_DtRs4PX)KydwGI!9Htv7c+dRe7)po=`I|f=@=iuBJb(K_hwG4xgXXSHu z$>6{`L<$#^q(|13jpABC53*z%Y2nL|0xn0n8#!jVhbkflOYg5{ge|1E2p`fx{~V5O z*Ej)t4yxSWEz+Ig!SQMKNvD`w%=VLw(>OaU^bXe_s}8@JqzI2y(pd5Q1k}3i zKO3ht8|^QzTG!qK<@l8NSSBRz5qYO2eTLIH#LsTEZpG3EmWF^QR@R7Fzi;}TuftIQ zyFC=H^P1Prk3swL{8lLfJ<2}?+`TY?jp-bXwA#uQFUwJ0E(giR7xww37Y&mD{oT>R zK?gBU^Yo8X#uW^I$hW<|yOZSj!;cRvip-Zn-t)HA4CR$i z+}zPd)n+3L4-eJCwg8b4@l4D0c9EyHHRDd@%=bWGd`4cZbjyKbg?(%5nuJN<{)=Wo z>^RJ7?P(kRN2RA#((Ao=XD}lC=}bH;gUzd9tHQ^_y2tU2>FGG;9;igHRr1k?I_%DV zokQ%^Akg3~fy6bpT?&*uM>S11dEYJ?Ww`dV%W{x_iOGO1tw$~Ujpg9=lLY_dcp1Rk zm37sRr`7mMm-u#| zW%_1kzDA^ab!b{~?WwFRg)3Gm-T5MQ^C`s||GuI7uB67NCmn}thi34yX>jAP6;Jgd z?^jVO7N<$W0VYTFf)*?1OFLSp>x^@!IsCZgr2|`RXj5C{)LMTc>qbJqM1^zrVC2O? z_l}}9F-sz0Q*(_38R3t(#lVLXy72F=%|ttmlS3g6qm9@|al=h#sx;oC@gEvPrCjtR zE%e?RGBgOnI9i7K4plYkQK+oW0Ik&FCX^zad@=;S51eczaH*n;6@K^SUpS|czI#xD+>b$IV&y{R?-byGhpLX*C zKAyTqm_!ODUze2u4YTXko;{zQhK!z(?05GS?&aNgo9ol7y7$eiZNIc~$?YJm7pb{A{@Fh2X|vq98Qff=&h@@|R9JX| za8+r_U?#FHdfG5H)}MuDkb?VA$?G2r6G`b{0v1SyM(r4wr`dkr(h%=JgDQZj{{F;1 z^L}se{pyJU!D(fGQ`@Fb5wU3lmcq+b?|ET%_3~R_V|vDe)A4b-1>l;@9|dpHcp_yTHHA76A)LrP@!w5?z088`y(cKg2!O8J-$misSD~PD9foB`Vq=nuxq?4Eg zln_6%EFTIa1_}n~HDgAo>RlhmL7LE=^q60$4=MjJ2XqKE$6*~U-V*ooYkNuZa{t?U zv5&*gJ11pj4bR0c_nmZrjD->INLbn3WadxDH|5-4NCceurnPuh z$BMa5?cf|DXV(p$r8BgOtzCL&6Ewg=D)(U&Sp=tf;w?t6k(`d4@G(Afm$v{b26fgT4 zif3i&r8HVKenCTz``k|nziU>+U!P9;@oZ zc~89rCVO~SL3t{igg8TUn|iPWgEJ{9B*&6)Bf)rHC&Vpoz7Oo0tay_5Ck@;XSiEF$ zfAK0MI&~(@R*#nHzdCFu-RA8CouJX*16zO<-tOQ1rV>AT)WkU96!jlDLq&CUpZ^xv z5*|Z%rvGC!lOxn!&AFbV$p{Fe>9sqgq!@?0~KYMTuMk3&L+fu(8qN+bB!{srUpM%{!n6PwqZH$Lc zv`S(-jPLNg@OT=AJD!L404)^46t6eHt$sR7o&~tO`=Hz{$D$f}JzSE_c6{jV)ZL%5 z0^E1!WcRd$kZG*^%J(fj(3{QvT*@8no@O(Lx1jTfnfL( zrO*7X9|)V+cM6nr4u~r8SHG+gri5g{@a^_HYhX8%vtM+9THm!e&k2SZq7@S52gY=z zIGFHh1J>V~RFtH*drFe&U4!Qeq0y&0ad_kLwRTs3Hi|;UR91^n^zW?U8P00}qcdd) zVS_fS{+UV) z#+|jGN|W9b#Y1gJ$4qY|X6iiJ6z1oG2P{eKpUl(y9fuXI`99fOJdc~+>j;a^&37@= zXdM}ho+I!w04wx;jPs?qT4%N~ckyvE<3>v+)0+Tqo@>%GM*Mb~v)8RnaBd8=$}_L| zC6wNYlck^e{Ov;a#+Mt-1YmO|%0$5Ao8JaEx`ig)ETf6%e3a-gZCnOJ%DV&gCoAje zCZ9X+jSb~m4c6D(L7`Ed*e~}1*OAnU-xA?fw$0kYrl!Z{+`=e%o06CDWHQTxY)|`VeLk{JAEc}u zg7Of7$Gbi7N@zpQnJ|$age=^c;-N4DVCNZ~d8Bx)FB3^Wi(@?f)RyJz%^VO!{+DSp z2p0B3XMiaD_ob92neRHL%^ZOovOFly5?xc7cc}eAE-6mkHu8E-GI7Y1^RU>9l|izg z2lGnFH^-k%P)P&+7t}1rw=WdB!Pu|9kk+>^=bVf$W%c$; zYRP3v+~&<$owjuHwiCmvg4Pgr?rfI(Ji;O8u^ip6bbBeE_xcCPy?Ag91#y1UDlr45gqG_K*TEY%?S#UZh+69tVWI`P z*F4=1$CEN*GdFME!o99mXt{Y+?EbbpPa-bs#HPtz-eKl1JQ*wX*xbIX<1ONpNe?KOIG(72 ziu5d7t`;hd!#wr2HeUV%3+UzHa_*>K<^2{GORiRDaGWu}iAA{N{K8QVk3T_fxq{=w z7P#5!9wpfxK9wrN<0cz+u&w7kWc}>5|A?ObTX_87oJ(zI#w>GXEiqlW${d=ms1tcX zusNzu`&)1SZk|&?(O#a_2XD;{oY=8GA20_wU9on6k2LA8jjzsI4v4rh7Oe1@tSy?0 z5ZF;uOo7^3BuMApZ{+mk(CntCBYI@9Kb_$a;c;T<Mf_2 zwn?;BZ_a>c`_J5Zxj9|AxVZQT3(THYvp}(uJ`gF8(DOL|C1Ud1vPQ$xQ$QY^7he1~ zPM!*_#>bK6m-iU!M5(l2&n3WzsvD3}`|y+exw)~IAy{e7^-LVKZr)G8f+UMdW z2i@rtMI^?7H|9!(hiIy6=d&qRmY$(qm z=t1aWv$zmou)=_zIT8PxB5pGG0l0Dh)1wl46xzZ)oAq(^2I?VB{0FzN+M3kHmaGla z!=;^=<@bzK*QYgjhMOD^WBHjwePFiO#DXR3d={d8b`cQnnMD>QeEQJy0$1SIV=Bsp411hzU-ay*bRyDWr`w=Cjzb zd79tLs;s5Cva^`Naza}!BY9#%{UQfr=^aO_`SCL|cxjFaG-R%06rWlgmS<1d{t-D~ zf@Bj!T~ilD6Y*DG$}O`68S|oVLYt4j;=qtmuJjNbaoyFr3QE0cK7UWLef^0jrG!y zOS)ZLY`nXz?<1(FYpgvtv|irhJXEl49PsSm-FH(cJDuB3`bP!7FE9XJEN+f$gCgcM z`jrmzl^rFdO4S@F50{}|&ep4BO$@3b%N`Jw9f~)yO!=j^-`lOc_lGG4BvZ6NFx_H4 zZikKOIJGc`7~D>)DKa|`Gn%y{RjY4;1H_L#H;D{Ll~Y)}?10rg4Ipm0PIJ)wvSl+v z;W@=irgnwnJPwDwcD8%mo9!(+uE#$stC!wG zrwC>0b$iPIr0-mf?iv;BQHwSj-izRYn3FI2yldX0=b4|c?Lsu2kGQOz>+%+k?w0hK z7OeeG*|E69G_w&X@4xqo-v%*FOJeVlxW$$e-ziYKjj;oRs&z)A>p4E=%;M9osLVdl zMx`62>PGw9##7OP!0NL2DA;M`)n$QJ9BR@9)*dg5WH@plOItb~)$z0Hh$+npqS3?S z4BA+$oP<&kGg5|ng6ropX;cp<(oB(YhpImo*^fH z{%~H7`3mHb9#$+BP8=G_5*T{}A`(M^njawBNP1l20KO!bvrk(?Yp!IjDPF|ytj2I# zw1&4W^D%ogd$5uscc2oH_z4DrKDr_@$#$rrH#fe)>8!1ddCU~$ zZCo8iOH9QP<#Nq-OR)j zp<#RXG_jCjQOzd$->ZKM5N{7-vDgG6_>=KX~^anmG3ra(uhs_P9cbbmwYMRDKGUnbK)Bgi4R! zfKpp_6ZHQN$v`&0?AiTbyNne17Ro#I0a3xfuQO%bf8VzK%WD>qQb##gfiRXUMchf> z4}!3k5O^!gfmVlNf5Rb%5_w|-gtaJeH^9CRgq<>TwhFk}Y`&ntG;mFaCkP2TF=>Pl zz+%V59jMQ-1tf@}BpC$g-@=uNLV^v{7G9ssqyvlXBs~&Hk>rQMoSPJoC|K-v*&ZTj z4nSD~e)0t}*bY<5JDVE~2eNV4!hm?O%>%q(z|X;Ogyyrw>I3GfvJ?-iu5REPwK}Z1 zmQbbm_+(EV%2a|3NHCG|i(ObC55WUOp&AKk)s;wqv56T=$Y*sq;@oL9lc|oE^jrvE zAdb~%wOJFA=~PuFRK{uL05Xr=bsyiRoZsC z%pR9qg!q{F6;e|wrTPOtzYHq029gU$$mYO^6*}N(null+HO~;LhAc^`>C_J^v!#rl zG@&JAL+vEwO0JTQgsB8liCI{Pgb_nVF-OI!H7qta^AAp8u!QYHhK}B~b@M6p8)fd? zvS|JiygFQVMZoToO-43be5LA%R*O$9xKa>W7&6lC>=d%0FmBkfF+i))BD9jQ0*t15 z?L<-fFaNHO#|%y2J|0`;1S%7q3P%T)i@F!#fxxivO5hdzD}5O=9l`wEgS&TEs8lhf za^+I1i+<|_AOk*Y;Su(X*w~{{Bg5`+lT>`bzjN#6xQYoCQ&WT1B%9reG#S<#oCBbJ zavXseJrdXhfZ@{o*kMw+O93Qvv4AQ7WY`=QpI^Z=!-sI?A#X7(vo*eu&M?O0a|`~3-)aQ4AHg++Pw>ekM2CNmzuo-5p8 ze4!X1H5&_s4h$1iS!^xY#k`*yU>I`bC;#)O)&MJ%nP|LK88*jh!;>){IY{Hf$Uwux)7<}-qY zrW3}T2(XbT*sZW+@PywlU;4&31QJjC?wb%51><#1@0R$`bc_34|I$T<0S9huOYd|t60ARPW zpB$7yRa)T`5({}N96x_VZi7sr@TTi;xc;BFwK}WCUEMnyrG-XOQH^uYf8m{vZ0@+c zg9je|S8w0iwGz5c&F5ZztwEDB{iWXLAMUel&UBdSDyKJoe(>n{#3YzZ+1f+O3bkRq zZ2+PeWfiFpMHNjqm~0zW7f2E|lxc-*MTak!jp7lyA(#ka0ycR6?YHGEnNM=I+Vx)@ z@ShqDo4`ju-S6&g(uUxwHt=sQ3;$0B)o|j)dr`ye7ql>(Q zUI&uzkY6}z!;Z)p`&JM_CLvZvJGPOF^(x-dPUKF%XV1BugCiT5L~w(TGDzYdKIntV z@4RhzWZKc|nYTN3x+G-t4;%97gx6j-Qd5nrLud88w$HOKQMKG%+wZ@lmywri6qlZJ z^|k%_Jyz&RU>uFJ`I&SuSW#mtg4Rp1;^r%sz zKKXnE7PSy(Q9#g}f@wv2%*fJ=nsBD5M80g(nH2b6C_wLuuKU4pM)?55x9C@Z7aZqW zQt-xu|2psLuEUou={Mlj$?px^xpA{I;NP@#$)`_0`p|nHj##+-!^J<;%%~q{81KCH zO2bndk6*O-jW0(o{BqO}D_0w8`HZQ1zg;+g&SY!2 zIKLpL+ciCgf464%idD@{KV{h9_fTFNKX&xC9b1RapF3>9!b5q5%oczNu#!WWBN00|8ENs*Z}z>HfR*WPs7 zXG>QPS+=2SqJ7L4ABV$Uj@L;_uQ_PZ%8?6~-O#7s(AQtH73ar$3*UO+z8;T1HR0PI zZhGX=_xe5R%P(?-2<;Eqojw~sx6c;GA}SUKMf`*8ILvqvL?{G(fIfl3FfwIWaaeAV zGufB~P*V6`R2Pxj*ldl@YW<(-b4M*+(XQvsgWh<~;VZ;|Y;j5W%}H~{%v{iaxd=VXF{g;Md49a3&Be{`Zm0u%_h0oJ-S{M?{T$1|NMb<>MUEhuu1&} z!#;Yi*@fq|X??a2@dzrt;b39T{>`&zJTpYpB@O@pKmbWZK~!|O&uXuKO5>_6TP>V7 z`-~Q6TJ0`o)@a`C*|K%s^r;t|cizN_6XW6&dfs%?wr$&H&YWDmTFpzZ=-{NbVoGw0b6UaYRY^^@)vS@9pO18+ zZo@`(>YswHh$CpUZgutt1K+`>(&UNXbn0?NVoGIJ%r5KNW6re6ci(5Ztqcf&(bOgLXKmQDF|AJR8*c6e zKP)Dp5a`HqsxbFJXxsDJ*h$a`HK>!ZD{~*dsKq?g?{R_lVDOx?TMaEP%+5TxZOaC> z>Ri#aTXD#^vTL^?ue`kc+r<}l=zt8w&){J(#*7)$pibRQn>K9UvE_mbFKl+^Ib%nU z*|&H1Wu32Ra9T6F(-}CDyI|qMrOUsqkXotRbvJVLaasba3jw6ovOd5-(zJOq7juNt zqQ-UW&dWRyK&s+&^}6G3lnVoX->HpGF_J6p+P1Y;)wC`1=e_#rXXwn-ZFp++W=+4H zJF~}m9etSSb~vyjxcu9N1=-ngad9+9qf;Aoy}J9*f^%AX!oia5TbTirpyv>> zSFL?Y$^Kn%rLc%3A~KnPf5v}a>w8;IqvXiC6^owkeQV#_t_%B)e7Ah=!#8$MDm`K^ z$bO*LjjO(0*r;Beb!%25)kmzYWV=erO*z!EXK_x}#$~fdeen96X|r#*sTV?C5tobp zN|?J<%a$2wRjX95m2>z=LY;cfI1e1DBi`ewSFhkmHUfSb4AAlj_@<8?_0qk!AK19o zpOrb_j@usYb>rUkD=m2kpS$szgL}4wgT9ZRe{|FbZ#S-=aqz&N-0Z_pTiQrwBrwCs z>)A!gk(4FtmM)kv;`5lp36|V1seCIqcgut~YXXS~PD)GeXsj+Vx0; zYSpb@?_gFYauJRjMiPQ8s!dDh4eERUxZxvGJ)SS0?)&idSAR2PP?b36`+a)P`FfrP~cRaL~J)ivvW8@SP>WAlfORKw$fpjs2hM=i9UW^b0!IsF5M7aY$K2 z4s6}D=7&$8fBv4g`Xge=&dRBnR6%fBl;O^*9-*r)Zx1Ksb}tf^R1mFfy+C0v87c{t&-*igKh3x9)9Zbd!Om`=xYtm zZL?tVIFDQO0}G3b(P6NJ3;ZRS6;f-2ENp{DDgkQbh7-2399ZJai4;-jgs{vckMI+B<>v7uy zLl&%lf9Aq3Cr%qS=>5ci*Bv(EL%vm0M}GO@<9$CJ<7(6j^CRGuzo^9S_qr{mfx=9q zay37;qXJ=ALTbRdU4}KGX7hG=wu-3@&wwLsd`{audzGYG>`SyBKD^hTR{OkOcYW~W zV~^Z=!}hhy9CE-7hNp!#%vIc?`+^R*y+Nz<2haZQ?L|90ZsXwx@1vehe>mO<9oW3_ zvEFxdz5o7p7hUPMC%P&o6~M(CK^H5p0h=%17nfR%6F=o>6+;+Lngc4_5 zdebu-o!2qXk(%SEkkRz)OxW14zsvo7r;nd_eYY#;jvMU?m2#9ZtETKy6eZ0t!e~n= z+;ei;XOrg*UbVf&jeTCZ?#e*sE`qs&frPw$FW!Gkr8*66d;EEhUyDmfV0$yFLO-j$W@vstgY zx;xW_9-Z2M_WsL;Hxt1W!u0fvnwbZ8i@7eO@*;dAIZ0905AxWex;OtE8vrZQK$lDYAupB?WmU zdv?^US(6IEoKjKI-$2JkbKs)gVs&B+i$AmqxQsSS*ywoC1z*mYb=jqtzVqbc3+Bxi zO+?lwLjbpslu9ri;1mUC*jvHvyrz2>(q6S{l`ZJ^7a#f8y?1*&uKusT?i3|RpTX!+V@Bui*}QGlqUAHE zw`$kkRWZFr+Y5%i|E{Ajch&sa+25^d->DO48u$g~1Pc&!GbS(>R|*?k5vDtx%<=I@ z%h~GqZqb4RyLWc#-1+qL+wIx4lkb~7KS7M@Su6-)jlFx={O7g?mM@yMV$Kvt(c#^j zH%^;0;j->mu@`UUyh(50->diIkJYVJD{t@iBbj?k!=X-BcN;t8ll%kQb9Qf^JnG9! zueu8LRwZ75ACxVWkPnPvQ;k4X*zRC+Q6S`y;-Y%zD9t;#X!g{~X;p6RbvrnpwQJXw zmCHtc_7OEbPNFSc|g9j2P^yRPu>gVXMVS6=Hb<=FX`Nc zLDxh`lW^Q@7spuV?x_eDh_O(MwzRy!N82t_isk9(eSrzW3bLrS-Xn+w;(?1FC1# z!i0q=da(V3?k+&dNnL3F0tN7z4lkk%xM38!Eq3Pb%UAp`^^=d-x1QX*dEb7|2@`5Z zUxPAOQs59PCVl_?_h+=bps1+mqi3HUkeh3yrFQ6iMb8^c+w|Rv>pMyz+h2SArB_}>zH9=bNF*zB0jtvoyvW8!#3qhXQGTKBR9=*+ z3yO-~eEH=!v$Bn(qziAp?er#1`@j6EE1{y(ZeweuEGmM>UjNV`Lxw*0lo97?d2xq3 zdfmxbwP4XA<8bbr@e}7vngu6HiQb+vJuR)~L;c@+^N|O}aF9Zk%5VSYKWJ*RF~=nv zlGwNp|LiCYh!PFDm=;$Y%H(JT`oaMy2=`A3KN&jg^}df9afuDjY2CG3_eD!rEdO>% zpMO2*vs%%U5q>)um_L91*Z+Q#iewmLY1awzIQlkpSteP6vm` zD}_bd(q$|D{o(r*Hl^9Ak3RV{W6QeLtBmw&Y>7f*15)F-#su9aYzsw{6A@SR{?Gbr zZov7oD0@7mD|oGYnC$Qnu>qJV9FlCW4zp01pO?$IFphY(lfd2qa&%=`-(z{rX>+>b z6F_FMSb@kQ96ie~jcWcG_MmU>gs;9BGVH@IM`2Rbg_({+SxJ?X#6TTqv4^cLc5w$~ z`!`9#!?XP|Y%Z>!u;2Igfc{k~r(S>4tzKca?Qp`P#}X)E+Yu@=C|ojefX*aH+sXt-*_=V? z1%mm!e(bSOp6skF#3sV-he9O|G;u_09^;G-@dz=>G9IAnE^)cDSkQT=Y%-P*x#)=3 z=ZjBHaM`TBg8UnL-gxU>z1w%X!Ygwsk!{CNTf{-|>Pkkyy=kiidYaCp!2d)6LG4%{ zgS{0#RJb?U-3=mG3~wmlPD%o30wSLlhL)YyoY5lF^Mq9JyI2+sB@v8#`mNUZv>Gho z(b1)ED|{DaVgvw~5@B6XfJCFbm^6_t&R_tvg9c2l7M5 z4;?*r($pE9uE4^zB`!7HZwDPkw5qm%6UTx0LdF~B`>bNZ$s??{*gPqfu_*0DM@0dL z64;QU*sEgE)Ie>S@>{K?AdASZ$xhDKWk`_wkR_o~Wgl90&?Fdc+N#ZI&6^`82kHHO zp#_-g4ck4*Rj7xTL=krgg%I*sv0NfPkW;Lt*rBJC&EZK(k1tW0P7%ZeMltK+sZ@pS@1<75?PAfea@+PDP~j%H?ERF{*Mj9jM}@eM zKj>o&p!5z0#DNZEEe%bT5q|{3_BgRyKvk2{)8z)sD%fSEgo4Wynt_b1o2pU8+Ba&A zGR~2R7%-6GPX;Ype1a?9;$?$vC^UKEgbJz27k2Cfs|P9g$%mwz{=np8FrA`+$#I(> z!XH2Z-LE3tk?LOXZn|fZI@pD@634v`sn+)d)P#Xn;sTHgRXO&^5Jq;Q&~F1wx~2C< z0+0?z)UxT$#lh%;WkiRk1ptIILu!E?D+H@Pl`s4=2fBzTRd9_E5+1G`bveHmUXMKC z7FAxH{31-fI0edb3l|KCQDBj>ip>-10&huzveK%sNsx(bmAsZvlo;$L1@UC=K%H#@ zlyJksnNvcw06>;DFS`V7*y4b(hcgyz71Je>KXGMX5X_U(lA3_2l7pkwsH}8R22Spk zm*mD3f(3kG2LcJA2%f3|MAA-YWSY#{!GdBRZAI%bL z(4j_A;frNH!2Clf$bYOs`K z-5Mc@@&p-i*!>V_0Cot5+(hA&N)yZynKQZaUg9hHEue&UI7thO4Hif&c8Ulhu#_mq zp;Sf4l*ENkrxp@MiXq8T0jUmEqqyn@E>b{B!bwI^M|7!?1X9`>QGoL?)Ho12BbR5( zydb-51_FbXBV|S@qRI{ag7yMc;2oF?P*DScNBjg6oe!8i*{) zQCLzuiJiGPc@j7NA5nGH#RMgQ3hB4yo$A1Nh>TNE42OJ!2aI{B2ZHmXZvvZw2DYHG zMbrR|@;F|i7{@EpVI#OI38B=SowCDC%SfT}kmwSWszo_O#a|N~Hd2WNL?#Pd z3IG9pBoBkko#e{_oKjR-(iD85LaC=BU9Fd>PeBvJ4o z3!OA6QX_eZmrz61Rx$Ar;e(1yj#3w{F`}qsMUg|*7;ha{v%RqV!kWXItAa_1NLl@) z_)=*exmAghHbaQWvDEOQ@**|;QbMWn2mx_YDfaAy?d$}QX4dtHkVO_r6p3^@st70h z#U#ba{ssI|^%tZ#+Qq;KZ{E_Y1!QEmPNc&}I=Fg@M&*MPFwt>!SY8Q2X(xQ6dMlm# zCt0eD2&i7Fw?_w2EL)iucTzCr0aX6HQ*U(`0`Me1m9=^knY{B#Qc(T`j)lilr>wxI zEC(r}1lMUq>Kl7a9b-LHQi#1!bWoWPMXX%mcq;+4uXw03M!;I0f9fP`S$HWePf%O? zNx^iSXc(2Ectt(ro`OkR=mMxXTvTBqu?Z8SgGBf_7o(57RPC)Rt8-MPie_4Fc>B4O zPA)%_<373C`wy=z9ZX!bt7?$aez8ugH34*(h{#f+(JSG=)YeOhAc(AhD1u^aN+bl# zp!8vtRtz;HrUE{RMnbx;t1?8(qcV_WB}8OyA3+Avm$y2EaM2<;)fEtlf%Z`n8PP@I zwSLDsCHYW(ojWhIlcK2*QY0B+Bw?J2DhJ!EdyP7=mjHGw=}349ryS3!<}7b&4=GiQ zu;dRHud+dMqXQgu)t-uA;?HCGdrd>_qlZHXj3<~$NtKT>o2y2PNX?_k>$uUUqyMU+ zs_>Fb`Ak(f3MwIVg(Ae(+2JMGtESiOSe7ucm@_bEksvs(KPpmL*puwe1n!fp!T-$a z{s>3P_g;lkl6MqDfG~QvuvpS<0b<_iZXGS54#5XnmaGaL>BBmhba`281kI{mA z6-HNGm$`h(Dov^tO(>d|PC#X*qN=1dp`t;e?XH6AfvP-dl}IKQR@1Wl``D|K?#2Y} zldj4C=n9v0&lq;Zve3O$IyBu)1yn%9o#2&{x59MGUpi&EelI~mH4z2K8le&;3+SVB zq2NV{7*rI2ceq1h>kds!ZiKG~CL?eYDUJe5m4YN6aRS36q%lFo)!esUGmH9@iwqG| z2Y5$Vt&$=IRVzv*399wI2y?HNQzd0gH&FwbOo-Fcb0UclT2LD&kx3LWcDYi?iwNve z6OpnAy~IhJD4~QhF;}hzlCO5<1K@+WSxM9-1<6*WCbZ{HMbV-g6<=~Bj1(5BJMkon z&{a_@l7XHXsH53=fta0kg*IwnBZw+qtSs^D{iU(e5CwNS&H z(AjjV8<=D;De$LJK;w}>gX*3dRMaA)>fv~n;fF2+&k>kXaivegi)nT&ieD0vh_R5| z6IkQ5dZ&^lB?TMgq(5{~+lwNcPC^He`-me#`Aj0dYRyfUSp^~z;Kx0uEFqcNGZtwN zKur|^%p#J)C$8#V1`N`RyjFgZ+#-J>*_A7>4oe-hOd~2Hm!Tn=I6+jxGQgDkMF2OF zvTl@Ub)-~6qUZpliWDtYv|Osm%F0(pBh{j@(2)>7em*K+l{+Csq)r*Q6Y095n5IUg z#%dCyW9qlM*ky${iSJB~`y_7sKd$Pro)$||LzI3j-`_-u0$+Nmyc6`3w`gBPBJiRN z6ZCGECFeS} zvWTR^TS-E6J;aAlk!%zns*n^WHX@G}0D`L)5lTdFLggtb0^~a3C|N{|tJ8`kDM?cX zDII^MY{Ha_9~UyhCKRfUfHy=_yALDOP>ilFBr%DrQ&v=#e036%u1=O>a4VR|KUF48 zA5|}iCc7gwYwB_M4bUp=4P7oo7}ObYyE6p#)XLtU$kD&u*5<$i6>#X3Lh=Js?=A-CL}4x zh=#%qZ^S7g=vxwD3qJP*j-?z8r^Cus$`kc1f3I`W3CE%nGFk?S8>2v3?Nm}+VSTTk)h7|*>&yI4;yV+# zPq@a{s1WHM^cfA=x+@-CnOLVd+S`2kODQ12A>nvv(#;^ zf*4Mw-(6V^ZK}C=i|0{md)V>o}%>ny%9Saca)3i>mwlQ9WNxft9#d!3J!f z>GM}o0Mw?LqAD4pa1wLKvf#nX zoYsl~BKM4-c8qOj@lVtk5Mr9wV?qJlI%++Z7NmKkEp(g7y$mAqU#-b$K|D{UGb!-v z6fimNU#HJ;_*R3q6pZd4MF4q>tr7j`UW2)LJysNumQo9e(pb@Z`Kz9J73IS9dlksM zm=yR`3YZ-Cuaf7_e_5W7K03E(cP__A6F`|@li^m>!J?g?JDy@(Hjm0f{+LXhP~n(6 z#bxWZ2B}40A}Gz)P?|D=*Sft;@IK~kbwU)t1nv`}naTY&GziE- zm`uY>cYIKQkyWolint`=q#gB$8}o|3GXI(s_(cksqLN=E&9QwUd&jWVj1n1oyGjH~ z_vXiOUmU{v6Yr$%w6vDh93^vEH`_+y} zIY)x#0mErv;G3VRsBC^PzmozcaQ{w1{nf$bj4qDL7P~zW2V7&^h`kqB#RSkf+yri# z_9G;4AW~Enu)J)fjIoO*V*dS$DWHx3lZg+Nv^k+E6M0;5ei zwZNPp{l)EbqGo4`N{(^M{Sr^iv0QX%Ibaa%RyrXRl#{ZpHqP;bISc!IoZ4se1_Dl( zom2kA#<9ZgSc2NJMC8PiV_eIVH+zmllS47pu`V`sVm${6(^yWFf_!49LyyD^v1*W( z*_G)`3j8JoOyK@a5*?cWFxOg6LiXD5JU*P|Dhpb0+;n-ToSMkqZjQtjGountEp!Pp zCqfnV*PML|HIAVZNf(k3hafViJIb_(X%nqT%PG|62uDF}i`5?r=_7iNVQI~*PZSE6 zzPvr^P8OMi{E~uVKyfESjsF?rJv*eS*%g)v)0rMNzRtuo6v*lnG#R z+=`mV?83%LPT~#ur9u6SxM9QPFkD#mqp3JgQ;ATJb14BALo!XP2jOE@U^C+rf&wOR zAE(U(yN!$S+WHX)C@D3vvU0O>b5hdMvXA5m1CD@OSsjY#nMEt4H+S$PD`4jJ7*T*D z&=76g><*tVV0Aj2PG@0pL6X~3Ewy6BcsbXMmL&+y#EJ%1RMifa#HpZ&VLd;_&UxHEn@LG7SngRSN)6Yj&tj?CQp%}-181rq}yw-I&` zkb_VZvrQDicVYn&qAQI2gqB5S8s^MH)-Z8)>>r~7ZUyE80SoFWz=)h9PH?nW!r`DT z9I^$20rFC*iHQ?ZlBj|=L6xL67Aaj1A6(%z!4J4)Q8q&ly!%U%HHt#jjA5XEiR%7 zY*17R6`Lyk86`2({5w&=1n%RvH9;%U?S)4R$rRh?^EsWO4or!GDy~*>!cp4+O0Z19 zJy*i3d*F{fIEF)kg#cgx(TT)GdjXJx#RKmE3)Bj6#J5;QZyZljBPv*Oq;Q zOG$|oQY-8S0;=ZXVLFonzeWKQxPOg0$MBu%)sjCj1vc>mxssBMP=Ff+jPS8A%EHLv zDX0f#Y27j_0x&CB9{|1_KmqzE@YymEx7%383|fMISaCr#;*x~l?+>swM9%udhl2(| zc8#P!vqFERRBpFg*%KtN3(t)@1WTf@=l&p=&J91ZwfKWhxa?rS5e_?T99Crmb}7DF zq`1OzqD>(X3N7kc{Jz>N^-?O=nWz zcTm8TasLh?9it%hWC1KtGRVr6QCwWi4bE=0@Q70f^sygD_)(xcB*2Phf>G*MTsBO! z$S{z%ScxcrB^)joVFe(-O~3*tstmv${zyzn9L80K0h`-y^99in7ssdo3l8Y>$7Z#9 z5tBF=JS09GHzR~i@ysBx9R=lZ7*CWa><))WIVdMAc))PEtll6%9N;Y~j0^xU?je7G z!Gv*uk%H75R;2gHR#cXNG~)(!4Lr1a-zArJJoAF{-@A8rl`54Qp5D|EpX`-AqQX#%$b<61Uvs3SA2Tf$;9kTvc1kXA%veQg zQcH#b?x?atZEc!ds`$%jD1*gM0R1ofASfGS9*rpN#dxKFTMBGClLEg*0TZ}?i$cff zJsnt3C@QelrGU*uL;y&^rh$Q=*!Zzuz481Lp7?~BD}LZvxXb`s;T7JCOcj&=Z3yZM23<(PgWhQq#veI!=#+-_ znxEeWQH@;=@`lTn4-W)qMfBC@o*6UfV+Fk;*J#`2s#o9n2zZ7UMKnU;C>te!k9Ht? z5gs*&cpT+9oIswMDX4MB;ztA{UZgE$(;%n;QF$Jtw*0wlDLl=GlQ9@FD)k|mLqJZ` znH2af3YftCTNFA*?=^7i%(>Sqdpyb-U)W+t))BHGnFgm2T04vYvSYtm2n3d~StO{Z zRt>>qWFHo%3;4r09UQm|==b^5eh&rNfq+A_O2R%Xauyp}AV8m;G_96lqsHXq5#NsV>Ib9B$15TZgqO1}| zgziO zp#!5KyB2>*%ZjuoGT0o$Qkhww5EL+h`}l1p;0DJPBo*Rl4A>&pvYnwxHpfJv$ zvdAo8w+%OJadG}vqepx*e*EE_+(r%R-}KL0&p5jk%&RrzpEz#ZHzP;n=4RJF?ev>& zxizg?)rsGXpEGT$&2G8q(oS8vclRW?{jxzjE|@d;HD6S5EPmb^noT3S^{h02T@ z*;)H1eKT&>j9Ca@F749ssw=y?6BA*FQ4L(TdgVWFn3-SXZF@nxd+xl8#Wz4TBx;Cg z=>;+m9{%#fw-zj1=t+pb`s$un_vm@(z-}~A94^OdkRE7_4DD1LXRxFzL z#pj>x-LfmCO66;A{^vPuS}&e6XWW>tii=8{w>b0K8*i#wE$!Qd3zjWf0(NiMys@CP zu=5p{UD4y8>{LS6L}W6;r_17}cHp?o4x*2&d>%tFHv%9^#d0fG-O#i$*ljwK0>6g> zCUE~ADjkC`x(~yV(XaJ98Xaby5LE-*fJMWuuz-!IZnjAW~Z{vowUkv$l(Zcy2tF^Q!fBkpM z4s2fi=!-AQ`XcLkaPp!;f%9*}fe?_g8ij{5ujxupFEFF)b;7hc!(vKL=_vq6()EG}a9bjQ>?SF=$$yxhjupUn!dqI9kGERU|Xt6Nm!kCypmju}|KjqWsMMX(R&< zI5u|8s0O6}BQp^Vxj<;uw@a4Io?-ZX!^eKTXyMXs-MhynC4TUqH`lHFcHX#=aMFXn z8Z&PoToNvGJnCMw?F;>naQm2V|^vXdT3$$ z&ZsXwv-nCXS52QWW$N=!KAm2v^5n^rYuBweV#>5SO`2_6v*y~%J3ae&UvJTou(#;7 zM;->X?|Z1v)LAn|Gw{6lVwcXHznwpS^|Ei@c= zykWGZZMLo3c350-u;q=Lol_KYv_7}p*imDP3)XF2_XCTI4tHGR=B^Ww$S2v zvz~wa(Ic5ziscLB6&1481}VC%nK6sU9;qyzDdB=>;00t0q}5osVE%JYJYID0kZhz0 zmy{Nj!uN+Qj>M#j9WLpFf$-B?p3Ou7%-^_iqilxp2hkbazII(WH#aLkyGf%)Mu-p3 zwr|hA!a%TIv$ID|oi%sb?7sKjGjZ7GhQH|MJMJ*7r9f`Si!YT0O2zN*-LdQ0M;_d` za=A1ZY1_g%*;y95B{`!;{d$dDNr|VNdU|P2VQKDG1_=VV98UDtaiV)J+^p0>%SI@* z=}Zco&=gP%+X-ERldbS9REh?IAS>b$6!l~mcsS&;hTK4Iz?+nm#EzPf*LOHK2S7h? zV4p26&Yha#j*AO>OMHd-#RY{%BM7{ zCshiE-+cKQ)*FX>_Oab%xuVB4&ChKs40_lY@Z6K#dj2z~sL=28BL5(0TBWr2-hGEn zjUhCpmF*G07aa#+8r4I_36M}FDGQ@HIXSYx7Ye0RsPNH;9|P(r9SVPLx8xq&_vMhm zXP$k=XPQvmDX<9Qm0X)gR?T% ztzFf!`I!xC)t~(JSN6EX)N1MJ8TC^epO(FK>stf*-;jUI6M6_$qQEZqiUn6U=EbDI ziADhvxKFSalWmi-i_zv|FQU!H#u$I0&mRunb^VQk%NFOqUVQPLpVlRRFZqn?E+i#!n$-ut%KS*wIRjc=V`f0B}IR5it z<3ArNrmd{jCT&_j{_L|Bhv%L5KIzlzmUZhl-rMs!K)1`yw>;4I(N~{%Y`}j8GE%5a zSvz(6C?paiM_j;4E=)mCIJEjvZbZWdrAfO3-9d&Q7F8nxpA)RL&ElFA_<0J5TEfr2 z`Qv=qH(h8L3$`3to8wQdR>{e>3sI{ro4eBeab?T>e_YllR!WM8mLiMNh@uIiV@)V@ zv{$U(*7kyWb&65omfb3-mWVik-H8$gwi2MS!7#I-9PMdv_VD3D-r|xQP87>*lxv&u64<^PZIGhNzLq40^fxb0X|LwLCu{3CPB9IY#(z1cv zi&a9FD1AP5Y!($2#m6T);%H37&<=r9Hr@vVhjX$hLvl(LIn^8%UBuC#zK;~JITi2* zat~+6yWFX%sZvUm_QIje0|&8jm7bn%cRGA*CX^*mh6yL!z0X^en3U{_j|09#B_+j$ zMOLRPA*lk>0OZ%~%*0q0IwYu>JFM1F0A+M1@)Pvzm@>!!#GW14@#uE|>D0%jY+yy` z9D5lJ^q2gliHmKv*LnIy+d#RwBQvGJX zqSam@y}H`ZBc~t+EjDL`6j{8($_McnEjVPy$I2cX+&Pe3Boc^FiJR{ z*k-rKCnb0i$hVZGK#`}QjZHFalX0Y^X8@@Itoi}Uh;w*Fnb~<`WltxaDZXk(LdY+6 zOwm7Q>VPagEhB@R#R4K#vpYb4P`e~x#HFTI2vwvPdYK8JhZt{*bH}6SM=(s_viFGb z3?4uhObNp<<8aEZAC_t{8_UilR9!>^M_C9^{E21}POv67i)&KgCloM&`zIthzCGYA zfmgBir}zn3L1ddFWfomv+5RnNUSY)bghoUhO2R9%9L0fPZHo97yVj5c$VfoUtHy9m zxK$LR3C+5ou>CS52-|NHq?f>K%#<8Cf1ax~Ao-!FO$uUA%4YEa$Q|Mg%y-~pL%>2ArBcX(c;ir6i)j;;k$5Sl3i!sUWUxN2 zyw%DoK7!XI%AJZL;mAS7V(OqD^|VIc|F2g?z(^vxK#>QPW#m4R@6YBHp+@<1em0!x zX;MJ4z{svK&gGSYzwz>Qqtk7e(qb~BHAx=k(8qu$&2!tmGdv}T^9d;xht1=G#M>8(zWZD`PZaCi~=Ud9V3N#^J^6N6+i(!fAY5MyZYbxT#eB`TR2|*^lf<@)JXc%xcL7x z;!lJ9c|K8!{ld8P=VffB{I{Th$#I|X%@~D%U#LhqjxQQgP&I<5Kj8tRVI;K}=4y=N z)z5I0k!(@f=IzlutY``T*u^wDe*EI7XDOjBNeo_p34&F;vT;J4h$((LqEtZ%*|iVh z%KW4EDuqAhQl5H8rC5&66iLbeagGMM#;TMf_14@D>*-1F>~s zRAr^!QV~g$T{e`JtMIDqwvk+@0~J(3$xi((S4UOTC_#i2LXb%2u^ZCYUOFS~ukR&y z{zt1I#ZZA`aV3DtUWmq9E}m6K<&pe?5^;JC8I9Zv-=0>p$Myh1UK>t**05F#P@0!Tp) z`-(b(1?1pLc1(S9aHqZZc5U0Xx3=weyR~iGwr$(?*52CP+P3}s?DL&>=Kb$Z=APgr zIXTIdbDeRZKxCwq02eColf)u!`e@v!1!~I3T@@)Z>KsN~4`V|3_K7Eay4MEiT=;au z@?Qh-u*Xy9;WV|veAFzsH1vkt(xFHVQv*jXl|bUCh=g~C=>6rTSggsNPUe9Ju>wV@ zAP#J%P_2=K06HRGY<nehbk9g1V3!xHh?@nY zXoMo%z`a**-^z$GTWXgL3I${{Vb^4oqwR0Ve^`Yd(z!bnjIuveegudrd_sOC+Ehe< z4fr9>T0bfMx zl@U1ZOe@k@idyJei0t+cjTT8ZMa9JEiq>q;j&$+x;bx3{>J0JpfN+6o8gO7aKMcoA z6?iC}CU*J3Qu#iGk=EF7NMX6&yp;(m!f}NAOn&h~Q}ag_Zydf%l08ozJ5MErHAz+spcqZ>$Xs#5fa zXd;9Y;A$j)t4U}ro4ERhVj`mAe2kzi*~5OTX=6+;vNJLlkSZYcFCYjL1=v&p`<={A z7TYqmOU}xfn zJBh#{MT0d}#-G5*_5SO72fAetVe&ARNP$hM%oPcwRAmEE_;eaP!%9N3DB>fa!w#s4 z-73Q&bQ?%ogABV|;u86)w||UFK&!k%f*@LM`;{bx&j!x%A1gnsyN)>tZ1R9;HH7w< z&Q7V$D$jpS4|cOr-|$c=)|!3C&YH#>$pdrqsY-(h?tBXVxS>T0OhL5<@dFVILXia} zPqzo@t~TTZgC!0Q@Hw2|Ln#~NtUZ1E0~2uz^R2=n%()Oe9Glg)@$TD$K3&aIY|2xg(d zmoE7uGAmNs-Y`GVf491b;=kOY zI)E|IcQY|2ijVI7Q%;zs5vnhdPN3anOcJXC*0d6Bz!9_~9z$yREEhM2vn6z~8(msz zVkm_Wy+Bten0Ikbh-I7$ZA~N0)plPyHy_?e&vn&lx7C7AjGZ{=ACTuq-REZBi5dYC zxq`Yp2Mt2?l`sz(?`>r|!=%mdqDsdGeGm1Na%=#N*ouNe`7BA9K0q6aql&)eI>4c4 zw^|<%-dH8Bh^(a`n3~NN^jqL+Jh+{#5CxMz8J9e4m!_LZ**bu%Pe(q;m=}txp&AFq zQT&j_=Pi5XEc@lfojzOM?{VH2D~Uy5mgCb6}Tn9>Cb+d z*)#z&Wl!3q1K?$@xh0VzCoP7)Qy0R_EEu^!jRe$r3jS(3q+Fq!u z6?dhMTo1a^_~C;s2SB(L`|n1|&6&rvL68aznJE+`S+elccNm{^4_xTr_#R@YkR7T- z9?_oze4|gN9|alqPq0o{aK_=NRWy7k2uW~AoQwaqknCNAbmUKjWN0q@CnEMna9dIo zdoDh{OsL@a;2$*;f_s?ZE9nZY2)N6=pdj1*ec2f-jGYhjT6W#Hi=psTSlTp7B*RF5 z_N33MYL$tQSs{aei*XXb`M(SGx*b2&_|$bR>nTSgB2A_w+ZaT_0>=w0xNB6_C#@EP)3zHV3redwn(w1n9vCh`1ZlV5V``Dodc$M%(rKkux7|RerTL*h2$BU0=&BhU zEcP>YZPeJ*dqBDG>7r@oYb~TK&hXsBE%sSaii+WfysVFOs&iZU&$h$dqA$vIJs}1M zZL!IM>tw>lgino*7RxfSCgXC%z7Q6+=;0_#zvoT2jEsyt?>p#GgvA=Im0f%9=S@Tw zvuW$)YHa9(!z1s9ISIK;)?MG7J~z$;$CsPk?wVBHmg3@r{WyUy*V{eK7G`A{Rj@9J zID*Z5p`y($SNN}s1$r&^yuUB3%A`EX6bTJRqen+axvqaLl5NKm$!XPUH@e)e^heLeK#|-<@yDz_vP^jsbTHckMa*qcQm3ORJ{kT{nmm>OKXrw0OXZT_&Ug}q$(2C5QWeI}&=#v@+0DCR^Z&;Mkj36%vA?iI z$axbc7Ip8_69kEnJy#eaONkXUFao%hl8D1Pjdc>~#9_0vuOHm#a0)}f=k|GXu3Rfu zB8^Rdxfw(xriL_fOg(zP?u{!OMU|)lFcRFW)@a(Ox7i+yC7@Es?s)%!fX5YF^?83< zsZxt$p-Kk@-n%)9v0Sn*{At)!quz0*sA{@WyV>tfZr{iRki&ip*QOMMNkD35*449; zKVx1(Q~EyIU0idmPXGPBI&%m&jDauSPOzUUlIOt69$2lvX;*Bme=1wJp6G?*S4&7o z_ZV^b2@GU!9!Jx+$g|Jw*?r5NwjJN&3r`&%Atv40^I+qx2!s~A9#X}4fltwv`*2>K z4gD1E=Ml^EmD@j~$U&Mq25$i{dS{WREb^@M& z^UnGVzq6ZiJ*T_-_R{6dcDuL1VC6Vr_rt+TUe7rKdQB$BFZ@%rQ@R60p4eIFBI$Mrofjy4{% z{rTa?1sVjxm%Ax77eTb?cK(Z+ueGcdE)epQ2)Zn8*R^V?)c6%sqBd#=@JQs5>$L`> zu_%S7j@GIgRsp;3E67fljW*wob*c=u!z>d($TE6%c7`+L!IgEg*jq#Z6wJ$}L5^DEWuMF2*c0e*$lGVKWf-imBu1(aO zF9$7$+p16gitb8Ki|;VLtrogQNay>i>ay#SLDOIg1ylh)IWR;O0S_7xo;JbJ>|v+8 zlSZANZ0+~j@_OW6hx zj~6d*(n+|$%X9mAM)&tho!)xW!Ydy^_iik|!qsZa`Psc@KgEjeY`TW8r}#*CgON;N zzbU;IzvCadlFz8#4@33tbvEN0BK5Je6k0zv+Y5m&4_GgoNJO)lR|C0vNBBr%k>uzL z60D2uUcdV%{7`T9boAe}UC$@WwSe`~)a82FXg+ROydK9p&$w={M1urFC}p_(PjR+COBYU^Vh|;`!tUw)OY#)>wN=#;dm-j zEZI!gbN2$4n}p?R>ork>#gv`1CS!^MHgIgsz$Jm5#~RPZ^m^gAK*$chyV9x+KpQyO z!5f$z4?@i%*1VeMd;1#2*u!}&c@>ufoT&&@W>_FKJPUnoq5+jhGtEt%7$$2I6@cWAVvUC$@R_<%gAl7a3} zA+S|j)ry_RUPM_`>!tPDZsK2TwvZ(WN3rEuoC{S{T4epxYgzBuHpXeXnOjK!eE|PR zjqvr-$I9IHve|>b!7qcOtEPUCZUJ>;CzXoR-a+jdSeSM*DtB5L4M>YNTI2kDz&z*;-3ji zId+_uivr{fHFwiBS?XYD4lM>I=>{E&LxU;$MaO(6W1^OwOY`jaBSF4?J ztlVI(8oVvU2KNE=7}NYBy}{@bSyUl|jzPAnT1`&VPsC7d8tctmc5993zRRDS5%3tx z=Mt%xOc%b4cQ6zvKrKnxT;aOnJfSR9hAuFkq1JNit<-SuEQlAo!yT8548m|%v36UH z=mAgHYiqj$Pxoara2VhiGuhuCuywMzTnlQ+y|%l7_YB?b9_RN8RtrxPu#6Yav!EIo z@5tHj33*N^xRg$g59BmkLJ+b=Cg)F0NAy`=n>cRX1DC&TTSKsUNIW^4U%ngM2<%nS zkGFe2x9I8UV7t(O!G$`(iz8T-6dMdgVua-A8ir6%<)TTbma&S&`*r(4*wMs*fvE$* zPk%o*CSQ8Jzkg13JU5!7eDp?MG@9|d%>Y2Y36>eIV~!c#`$-nE)n||~7)kLn*Lp5{ zL4?&lA3+TKNHB3E7W6D+7jSAg8FmdmE9`;;i`H$Iw@*O`FIvy;L=Nm=NRWV>^;S^b zi}CO`E!>;}6R1%2sJ+Ls1`Z9ZW}sHyul6YVp?Kh_XlxK$UQLxqhCuB0le>~Xs{qH0 zA2rZZ>#aFlc3YF+u%%O30xnx1OlcrZPUrO&#l!e{aZ`cBOQ+-24K{yN#{~V9wY8Ny zeIzbBJLE6H>U0KAl$&9h&}+JoGI!#Ybv&yu!`E zXO`u(3c^aFh`7L!m@t2o&fYdEDn{O!Tkj5p;B!0s)8O=2y{iscyUP8uqP&bEdC8tR z(_=Hjjv!}`#~y-+fSzBNSqx+}P2q#LAG6q*A#veD05-t@O=f1xx$DW|k`fBu<6RZK zR@>=DNfr;i(_&TK0js6A$^u3PTcgBpuA`8)5jqXd>&z;MqLEgZ65(DuHSR+b#H7Xm4}H$vNK5nOl9COdb{uOZ)F9h z?&Uux`~IRwzQDr-<}_;A2Vyoh93`cx!^m=F=X5$F93LyZIt!0af28I5A@DwO>#QmP zJ(;H=L+51kV9mY@X%SG`-nmvMxiKcfNY8;wP>;^Ygt%yW8-{ z?LH^q!Q`C7yVv~(KBN|?1gQ|5RDEgR7YR*Z$QpCECqM!dpxcE~@AD87%v&)ds@JS4Yp+tKW<M0(foi%WA*h9zv7lb$H}H z9tP#C)NIu|oHx?jwEv!9eR?)O%}LZ~H&AKop4>BqNX>-&=9p*T1LjweM6SzOVQRB^ zPs_vZFr5p`!By>f`xq0abgtMe&!DC6TF)>Oc-`?a%U;HXc;q>+rXeUlCI4F@( z^?sjnqjWv7Kh2k6r`JZJq0e}JT#>onRQiIeEn?Dn~@waMD@IFEVa z^ID!9PY7s(WNTAOk_2EcJg3XW7h5X^gcHv*wOw~keajANon~a<_QQST{@KwF9=rz7 z>+^XeOO0~eSfLRR;W0QMVBwuQYaZ|n-B*Zy9Y>m?8%_+p{vTuA25>*K2~4M`;JG$; zpHi?x=y2;n^^nULi@~8!VE3%J)~j4e+2+)cHevs(`ZN-xVd(&Ros z6_BxZ%}IQ@zvsDLjeYO$&=QShrQO#l*)3IMNkl;);5c7zU-P#Zjb4<>t>-QVh0Zr` zbid`yaGyusGoCp0mmFuiVAQAO^_H1+?VtKX==!+)F^UsQ*ltqWrvB}jGgnnc%5t9k zQ{mHg*PDWM?f5H^w$9yiY}QQA$7q>jdmswwOP4n`|Kvup_8DR|`>VoCxBYR%3Nj}) zl+Se$8lT{ErFVaeb*DvQd%-k@aHT~pFMVda;e9->`)y*}N4}qBVk8Fly2p}fo!N|S z4F7Ftp{KcT5K+K;QE7_6=Rt{&c7$0)MKzXw;-+`wbWHZ#>xqj)&x^hYubD5ur8I|@ z-y}yXi{+bXxu=!u#t4B-mGI+bgb^#Kv~hJey{A!s?#f<1h?}2Zba%Grc{!0Kaqvp7 z{)k3VWw_O;wx2DJ^~BV9yVGiV@W*JIjH=u9LbXSOwgFI}I*6f4GJAetRcr)?kpHqMET5-Vp=^0g?$3i7N3q!2I`{Q|ZmOYLF&+;4EP7DvlJ z?TDVmpbp7l!r)>dE$0XIWZ7Wh$BDF?$#gByEfyFmEWUK&ZmFpS6x0=3We7_NDD-R= z>eBL!NM|Tb^nA{SgwrF}C5I*BxKhQ4%Scan2e78m zEwC{Q`4pB!pUEuN^eeRQM~VV?9s7y+Xsa;xp>RYl&&))B(~7e2+7jYWPT7+vk7n;G zN+}7|WNS9_9NnR(C4+nWx(_g`qA4;MaD0SgprMtPmCday!8Hxhgh}JaDz_Q^jResX zpMuap5I|f)5Xb=SUL{T=6*!~%+rHbH8)Nm&R5Uy_sF9;DvmxMplyaQemV`Mg{G`HB z7UBlclF-g2W<@_tFSz3X&ITU-hARd{LO%g#KRTss>n}}fZ=SAUZqrdA9Fd=R@Qfgt z*aPMOPXgI*^r`kdzvL1Si;Ntv&n)fjcuJrzRu| z&GJ>sLC}n>NEP9HyhVH=U@ni8uPc=Tv}I=)bZr(RWHu8J)^Z;?y-J3(gF|rv7ussL zH&cT72+Y_*q)NkQNL}duw_;jKk_ZUvUb#xXhRljZMf1U4Dd`@Rmrwbaka6bOE#&-W zF47>vUu-v1W0M^)*d?aQ|l-6(tph`EYi|0gC3YkbXItNSBF41(X^~pn=7v@Mfk>&;j2@S=IIZI8l6{1kr^* z_3DlCHVO2k$j)S%2-Fv%P6a{SDCmEn(`7T9ZQQK_{HisR-;s7HhGrGYQ&drtbq~r- z(S<+j#m-=nR5kK>ub^l$#TD~AMPdZP*=OR-$nYpghi2O{Bsu9gbmyrnoKtE-UW$Vp zEs1r3lk(Ih-KK6R)*Rn^rvknQ0v0i!^{&?E!jeI~$Cqcq0z#sR!Tf+?$&5*fQNjp- z?!#e>1F<{Dg_cdT&*rty{b){N!8`Pm=7BWPYz)zhm>)r7f$(-FHWKdoU?{?fY`^34 zMLm*H#AJkBtfbO9;n6Y+YosIck@-Y?G+^PO`^;cgCt3p zVCaKYOKr)U=4E!8cY-ug=JP$l?`VW8kXGb|E2EUbl~hZl4GdT*DSn?R6QtPS|Kfk0 z6l)X=@(tA3iSq(8HfYAuRz`Or`mIdOpm6XL>6tD)IA3d2uJsQt^bfByaSM6jQs6P0 zHzpB}Mi{2$sUs5IAq_e9<|<>h5T$6S1C(4UxI!2>J%y_|qYUnObd0~g|U~Rdb`L_e`(xb@%A~HT4BEdZY z9B_k7VDx?@&PpfJUdlTIGBXp*-1IIC?xaosoSQ{KjKVs>^_}{i6#XUgBIq-s-=e-0 zIONimPhhPXk-^KAe}}TF%{i!m4ORGmP}mprNKY67DO(-VkE&CNsz(RaVyo54E?JKJ z6ea#U&<6HI{?~$yL?}zCgOX_dj8X*!8>O&WGikO77f$@WFf>0S5Q63hw3KnnB1r-r zSV5!ZvcQ6&4vMyI@1$>Wh=dghNsDJV7E4N(ncG)_1V>>2Qqd!~^Bnb<8V%1X-pkBC z%T(b0P%R4mMMJOg_3P)O^lfFuj{@e^eq&A7DNV5u1PM8`HSbMpI*NEfSCMVn%Sti< zpKmZ&kC339TzjOF_5is??t}Gke?7j>w)t~HwNCxtg(nHTwh;r;vi=aZxI`tUG)Wkm z&~x!YwxzBUL;>FBK?O*NSlGboRnXw!mSqy}FoNjts3eu4WC`--4V4(-( zDIwzK!i%!Wa1fi4r^deMhd;XTc1{%zki|emphKv{#ZIA$P|)U2g>c!jAW-YBgry#4 zvK8AJijebV6pFJ#f1#X|tYOZjLgZ$3S_P))Wu0R&Xe7+~!vvEe&Z&TXX6A@?;X86q z7-4~Q+*TeQ1E|6SMJde>TjJOuzaSy4cfJ5-?=L%GsuLb?Z-`DX?&yiF*}u8}){qi3zV9AU9wk#T=E0=(3o?(GWn~H1Hm;bw!3?y3IMv z@E}H`yNSf`a;Al-6-$idK@+JBxwZz>106HxP>^eM0|qWK6^2|5=38o*2N_frY0wo< znY0Ldi4kh_rRed6Sg75F3T8w0mHK?l=e+1rIH;sSG8LJN6(>?2n~PD8#L0dvZaTKQ ze7U7)8*u6A{mqpuC4T|ye75^Z%pB++Ip?g z!{n1_(1baU@;g(28!|_QD zr&vSqP)EN8F{S{uSI8ec)I(gVWOOJTT^R_4E%Ff>iRZ)sZBb4A$}?FpjM${$JB$34 zG%7qk5m|~>|%V;uTd_y5a_PL`mCWIq3c~QxzOHIRCGF-Sb4v{P=g(H!!hKAj4;e zP^_O&&=1>W^^q-&Ds;R7ze8O%*m4(^VE&p!OW_ThU=8!6GHh-%&n2*dVkHfM=o)*E z@AQWUrIh^thv-Z&4BrA=G2kgq+;sgvV0klhdoM>?ZJ`)-Q!}zNR5Ub+o|^27=^avm zU=hLo;^X>H9$>~~$dm_GL58Lk9E!E>Liq=_1u-E%`GcN0-Aj;?(e()rsUl{j_n3gI zxW8!Ar1GVK1@kA!{1GCXxFKXZIo9NK`+1qE$f+Vi+>+}K1&&ZI9hh^$@-wuPCBuL` zo%U{pQ%gBC;)CQNX>tsl_s4eJ!JLEtfZEz+NP~up)rPMqh-!tBV|-Qy5rQ|x25J03 z`wS&z@PtSRhc0+vr*h1uptu@usXMgTjej4tic zw_-gcK7Bw>5CKQ;hM!j?4_do-n^`gR>uAW*xDnFdnpzHS%u#~p`C{D9@+5D~e&;0V zJ%z-dBIBnCPP;(C@`B8;M&*-e5s(k-Cf<}SFKcoH@IO68{ZCJo*;hE0y#7f2dz?}eQ~A>rL7)5gYjo~_x#|(i=vx$&6$-y0C#@;# zxfE@<^g-QPd>&Ftc2mVR5zBupFA0P58o7QHDQ6&XkGO= z$%~N_D*3mWrQ9{B*cB>!CRUoW|NKpuv-zeM@o())N4HkJM&PY(U%gVAHYTm%mFoiJ z*yYK0T{*9b+fy}>)OgBE_D{6&^_#WM0tjP|5%|`C3SY+mWl}ax$h+Hf>K8rr?^fxY z?M?#Ywj&;wS+z|a`A-E5CIYG;OpR-)WS>yAQCp%kTlhS81V~sfZQIpGAOQ5Ka6GS% zj7O^F2++i(w_CN=5v^&di;9VfCv9s_8eG5(sIl`HlZ3#Ke=RwdcQ1159I}f2J`I#9G@wsRPDj z4N#(+i5CewA3Nl~UpRsOO?#jq06*-1>2m9T@@(Wo&LYSH{g;L3|4%6XLQgjTORKO0 zO4M1)4VR1i*GG0pgg)RI;Hz~&1s{cY`f<2syjQWKRBU_B=gNnpMe+Z& zdMYbQX+dxfW_zWF^;xiKQr4H67Wg&kqR3AIS)gIm{~97oXkB8P%F-g15wFg4!Rl|N zze+7ji8QzgqXnJiJm4Av7M$gDE+|{M3N!aq0T{CQf2|8eY*FSI@2d)#g1(U>YN%f& zm*O3P767w%lT~J)T4Vfe{3ME+qS7C}1HH@V@aN0SX@}|G5Qg0j)>O-(y0UbT7-$Bv z3zU&s)~_ViB?UH7=qn~7DNB9^Ic@&4GS5Z<15>rlB2Y}v{$gFvebtyu9B&jc^&_Bz zmPS@BoLp(MvzZwuCbGk-mOi{q|2Ijymc~U9pnWNz5pf6hHo2R%6XU=|2g@p?g4sic z1#Q54LpmDUI!o_O`-S9)bAiNP$P zy0{mv!!Z=(83`yf#%pm!0#qKoUQZ!_I->r`KzF7|x<#D1bpgbX2xl_3Bj8jWObK_^ zl}EU?eH{w{W=hBfiNb!}Ff1Cmw^J-+sq{;rVvUUE}hFfQ--H6!(bnw+7-Mh~nv1 z8cju~rRe@jEHZB+#c<`!i87OysFCB7n||B0^S#GqrP2;2x4L&iR=%+-dYW;^ab2#Q z0QSKJ?148{jz9@q$S5?ZI(cmRg6FL^LN;wIXW?Psv$yuTIo2dnah_l{(1Gl*`ze_9 zX~nfa0h+oz^xulW;eQZWI|!GtvYE979p(OkAf;%{VeRQWXe!~w!cuq}E>g8+bRw+T zV&`|{am@8ww=Bv+#_CJi(?+-wUh-<05#1;cu-YyFD)HIiJDgSxl-w59Wq)iYlv!uT zpmO7s>vAKYMu9W83xi>0tZL;h7`5qqnI2Yxl?B_)b1ohUskf-L-6q|Pu#)zlnyFRz zLTbVNgb+}mJBk_J_)*iJFhY8T&;=?%xL$CDEa$^!InuD}%B@MTUGdz38H!xXr)wqD z&ANGX{Nv3*tIH=mw?OBz;GdYozOYqjVUX&9wecjS?jS;HLM}7iokQ`roESUHf33SR z6|36nM!AlfE{{hg_U-qzRZVttjb;uFyjc#jbv3X63Le~zcG+LTc9Zr8=xMFT_iE3&yG}L7@fqkhAFZw zpYbP_z%*nhj%+A*YqFi#Zd7j_{Z%~q0pVcoRUso^E>|D9wZhkoZ;T>?rGs!atWgX6KBa>kct7MTyM8-;R z(NENtk_SJT7;+OYC0l6c*s-O8t{{)gRV!!%$BfIs>rPZ3*NOx>4yt``7vB2rKG`3kM)%tZ~nStrCcnVL+-HOAzWwaBGcQ~WaLH|C#R&OXi@Imag!j2&74Mx7a_Qj zlz2=(?dm)1)B|6UL1BvngK@e(wFkEoZ&Ac`wNrJQy{D4mu%4_=(h44@Vmrf1#9+x7 z!$>RNiAA1!>Y<)kZs`a$8KT@$)JcogVuDZ#3*)h?Ny(y3qLP_v-m292{;VC|RYPxIhl2E81q~gY9j* zpM%IGd!eysH3e*Re;L4pTYP4~w`tQou#f|VL#zCtmKrp{vdBFcF{}8M8Q|_X7FB^R zE4wEH;0^b_!ft^J7cUt?jn{3rs!oOX*4nYMrEcJIz^RPBP#B&tGZ*J8UN2;d*vwWf z&-@9!M7BF)S<3vuO+5a?vZ;~+L<^ZCu8%&`@D|I89G?JZGzA|48jMxoLJ44g=K4py3OVqjeXB@xxzjVv65xtqPLs6_6bloVa*Q(g^A0Ao_t=s^(Hfg zGSt|c;Kvc-(0hfMM0g ze}j_@3SJCd;Y$cuwR!_LZymvwdK~_yxi{Nv(zHZ3nC9wBa%Vld?JV5I(y6KMzl;=Z z8Ts_%-OdvW&3P?z5M;F6OOOJWn57JoM0PUn#NRF7PlxEy=VTG+ZFi`VWpsR2pk0=K zWZ@zTxTuHjHV`HkLJ_f{K6Z}@+%2Ek2afY5=!3Rjq7QT?Jqk;PEWcHrFbl=bIG1Bv z!6U!JPFS7*8zI}?S3%j$PthLMEUc57dbdHrU2Imw^`+zoO&Z`XHmpWPFT;g6{}o!% zH*}KpgQT)zxH8$8q6DSBY4STYM=bXho4Y`MGM~LFcqjzj#Xc6!uKIHvPC)h?ZlT*& zjSl76-5pHxFTv0{+Ho|SGA|eB!|LI;9Rk^wYi^M4?}Mz7zHkmMyg4+J)r6D`4GcIX z`PE`2ZXP^i1Qm~+S|}}yI`hPxIe+)XODZb5+O(f)6xd4C;EK6epCWKGPPgGhMraiS zcLy_>6D;}_cj;k2gsEuMKb=?9MaZUQbui$erPH&ANGp`wn6hR2sp?pdkA_RFbo|y6{F8D7TS$!M>%;^fgz?l^jMJ(@357%QeS}X2}$d3vIN{XW{(R5S!$` zlf^mVAhKZl+`)e?0%oSeuxJ+6!)1dd^Kgel(Ql^Wc28a@(VQEHfo(g0D(l&dlaI2MMASMNCW1qPhY|ZnczgF=qCaS zQynAcC`cTU8pI=Km!(pY5})<5@YN3i5A>{nFDE0K({`BFQBI5~q9iyh;}?!V0avG2 zL%%KOVOSqX#+vkC9!n0)`=wU6KFbHHKVe8Nn`}|Ms8U?9C2b9d1(%`fxd2M8@T%bO zmcdT*cQL(PS)h11ezTHIqr%1ICKdyiOuOujc+#F%o`MVr!q|pDPdLda)A5F?L53d) z(p`L6E7M(R>VDRX%!q!Y#7Jtd53imMZBd`?F_S!L?HdsVva&nYOQ7dwi|j_8F;&&( z<}b9u6mG~4kaBzWocSbX946)kOls<4ZPxkSi<0B$z+so$(f%M+nQgA~KxRXd0p|od zDOhHNa3NC}=r5A;o*X^dkB2J8Uh~b$%hxnSaaT%AQm<$&WKs8&{e+#ZOI#lUJ5%}E zKUom)qBvt=1P3y#=U0-Pp|}W=!76d1>eR|5xHpCJu_NpdXhu?;%&cjRNlIE|iNacF zzk{ml+QDAp73ld|RCtDnCbcN%_gTQJD8}-2zB09uixh-Ys#zfZc|vj?Xi>~w7Qt1||8-re)-WaQ3(@2kQGIl3ztcfp0_OwX zeaTBgLCR)oycC!K&+Cna5`SSo%cb9w9i;oGP9=aG4XXMv^^~pY0Q|R?xE;_7upob% zXy=0G6K|Em^$b5w*cWcRPM7y77o{s_m(l3nZ?8U?H3NzI%u+1zVEH31YI=eg{biV9 zkIj55=l%^lGld+^%lo*!T}RsmLa3eWSVgKm2XQ^0-HfI#=Mz8gbqzh)g1Vh0Q{Cj; zLF|ZEsvkHm;)S~9kw(3ApI*&)w|Z&QRBON{E={?YDetmHgHUD(Ouq#szkQ1=~& zE%w+gNTz?sUc8MQRN|4yEO6H|)ssQUi9-ha_XaXh1>Sv+WO2yorO76!f^|%(W@WaK zVrRk1qw)m;14ssU+E!m-!3iMxxi!Ty!%T8u2#A)PTf>&BGR$?C-ba5j#n)i2Q>nUl z$kIlu3}NSC?SVK*LX0D8Em@hS6tz#d#U<`MA!5v*>|-LF24PUcigxwvGnWSKkUDqe29?AaVV#*mBQLa|yNC;8+4w;i z9`-a7LXz{$bM)AOw>#fZ*GtCa>ap$TrE>jKp(MF{>k}#hzd(|rFYXn$ev*d~$fc}+ z8cO0f!9#cg6w}$QQtAVf0~K(a(IJ(OwrwyD6>wbz;yqXO@t#uijgo+l9OoDsp}zw2 zfgJ(;WlEX~B?Sl*S4R4VnT9Fdd5vg|p zAze$~>e=1?f-7dGZy6Qqv|9l##lwIu!}ABg6S;87ojGntl0Mcx{XRS=|Cd5RO%8nd68Me<9xPI zb{#(Y)nRsw;EYM$ulRjGed_Bt!=dZ@m83uc7G(1KJ=^v0apirU1*vv4?tN!d$u1e~ z!e+g-WfaqVwfWVquG?~@RwvK<;dXIGz$oM7)a*&%y8FH-r|o9bP5gST6WQ~siWp~`Fuf?yUj*$Q z0h3wA(56Q=(%xU0o6^~u^+mursIa#22vRJ8_Sg&>H<#9FxG^C{?|)NbIisAq3Dgeu z6$E?^=0*z-3o>2Kdid`2k_6nn?$VmR97it~D145d-_Hr=+y^^dyxh=<9&NSTY@gA z+}al!oz8t8CY&F(J|pY+ik`>aw%n&CsTYuDtd{*aJjoJb+f^|~rC%k!{YW%`@Ceo?L<(K{WMymm5X zRfqL)Ac82H^C1@V&|DbEr!ezvdfzDa`z=SROl*IX-aGxw#Bl6HXaMu}kmWz4*27 z?Fe~(-2N{leXSs3*lzepR3jgs$@3Sini7I4AIBRmdrl`FYBaI{JTl=^;5(*k%IOjO z!$Z5@cM~})dVZtu93S_)VY1w%I)%pwoIY>KQro`Ik3rAyF@S)H+D(QFZ!R*J41%N% zme;xaBR%(fO{7|F*2HX)t}QXLIk!u4ZQHpSS6S>|BGn0N%nk$TZEm((?)@Q{x{jwM zHzPE>q+1;HP7QUrU(y)=HzWf0%bJsCB%qG&Vy0xVA zL^kfpLO>I@H|T15pIh&Qre-CK>C9_Ns&8}e*FAh^@Dc|77YZX`kuU>uo>ZB-nZG3c?7e z82;XGHjl?DzP3MadpRwhoLuq!9Eqc)r$4MB%g+{)39PmWG{AZ(xw(%kO@|3vP`F7ezdc^iwviF6oMT!}jFS^fBzQR7feZOr*#}L z-$N1MqTNCPRqU0DGQ^N=G~;};ICWJ5GhW8|+955=Wr1>|s7jms{V>TAPw77qQ?>GS z8Ox(KnPp`x3!T62deL+E#B zzxBI~QvfSGOyJG9n_WJ7n814{K%eVAU|377MuYkJ9T3i}pyAPE$@Xt4pO=%%`GVW- ziXNAD0QG_y6U@!)f@UiphDMuL-PZQh=ZGY`IhD%$t&Rc$mrj*hU22NdhKprgE`_dJ znS4e!3^)F5EM<-giktK4+Wq}_Qa!s6EWgv+L+{sP>^%N@^ePTaN|>=%7zuzeW)wAA zOtnTMW*WaIdo2j$hlAM>TdV!<9)k||eFn(pgFSIJZvhhXF}lX*x1TOBkTF#gBUaTA zj9Ws_)a4-H26^Re!qaq|Q?aZ+9vf+{Ky#!*RJ{t!-7P&YSuCl+v$9%_tNMhy+JD`X zS)Wx%`Vxl9-@NY}9O>i|Boymg%pZRF7mpGiXM^uOvqFS05Y*$=(`eRxo1MV*yo~R2 z*Eal#3(Ct_xBm7|g{-g74NUQm_a$V~tIksjKhxGq2aoQa=5pM);s>aC3zpZ{=@X@` z2SNF%Oc%@Jp~Bcg9k28Quj`y2W!a7}f5Q}oe!y7yd5#2<%eY-_`KYj_=H^a6=uv5F z6Tm?*Gh;C7B=$PBQ3xbC%I0`Jk~m029nse?9{-LLcr4IYG2 z240j@&Y{inxT}aft+ndSds%f$HJ-_F;94*$B8guaOecVG3tT)YMlFR6Neax*cPZ9Dz2Nrct=4AZRZXe#IE8}W8 zU&eo3?0s|zxUO|~m9N?4X-mCa(`ol{>%zF~&dAuk+j+U&e}Ce+uhP(4tV&zIPh`gs zpf4V|gJ^jX61nu5cfPBrxLagtp=yW=iwBK3HUzVi>A?`hr3LC?G^!hBX7G_*Re7az zx)PDJZK?3wPWvgB8GV_(a2SLb#eLXvC+gRvH&4>y{p>+$So z*KR^dN≷w_G;6%ZbRhv-^TB_U(WjgznA50%=;chMt@*XjcY1WF;#k76PRhz;OUU zbj>i#=xriUtZR@9{tSwa?P!|=N*s^kgO}@%+9h(zoL?_>tV<#Xs(wrk5Y}97sllVR*h3r`7aw;TX4U|z1l{7P7D~+;*!_j z1Q^IEO^3Cvp2p%$NMdfTuC43%Vl9nMqXf&bM+ZmwcXK1)fcx+20 zECXUEDpv;c^pE&fACm;8KRCC%d(sM{$Jk7mqB0<>9oYr;yzwdMW0Qs%2<#!w>?56P z!(4Z)#|D&v9&jsU0BT-_WGWcdNPbNRTax7-|3w2qZ zE`ba0;{j?GhwaIH$)DzkycEn-CgQdC#V_Ky{?P#(s7~BaaLo^bdk?D>fj~Zp6X@pm zj=|fba*i~5-NnjZSuy9!zFZzKCbT$<@08sc__Q)3=T zB^{6#j-V%EqA8aUurQ>+d12VXi__}BFv&ues#treWGL`NKoUQUhhc(JVoGtLEIy4r zB#+zKgaiduO|u(3J~K4{eo(kP5DfxlV(0JIgsCgV)XmoFUBEK^*hX7}2PLjmt|DQ4 zv+Zj?l#H?%x_DXWK%856QK3fPf>O@P4K|Q4=;>*)A5LE)>Us-IV!2$t+}AHbr+eG! zhN7uqcS>4E$A%6i)a*QQJn``={dbggMN%@Ub-1H~3LWZ3UCXuCdKCZRecu6~OY*-! z`~&J_>xtxw&f^tOJlG;Vn#pu(KY*m@@!~9DMXwrPhG18lU_v~&fpq>fp?ZUf6w%#@ zHK8C_JaT#>=Wk(Ir`z@qN0ZV{^-J(Z#iw)e>ybsT(gAB)aS#)$Udc(!B^)hEJE?+q z@LmzQbbeNQ7C141O?=a=4RT^|+y_5U-;KeLEl3TmQkUY9N6eK_O`krdNaPUIL3%=; zWo%Hb77;u$IKqpPfOnFbhhwptO)L+3dQ}NDirEelzUw>m;c%&DXE*;MPubZlcSYg2 zCPt!*lWSLw6#Fe3iu?U34zhF}5r#6PqPR}Q={(Jig(&~gCtp9U|AR^@9+u_}2xhQ} zkOosHF^8Po}=QN^h+Yy|ff=j<;c zyjW7hKjug*K-$_BT8UwTS9`JWn#^2mDs<5hVjyk6xrq zl$5#(jJ|QM3LW@QLHHa{5f}%Vhl6s7hhmk5be(OV@+Xmo#GKpyREj${3YZtB2#f-j zUC5&8WH-(IyXAr@0Kf*qD|hXqk3ytIGhPLuleq-ebP#pLkfnSe%5No82)6D^Bh0U; zu`i~C;EPZ(R3SfLpjZ{ zD^3dZal^0^y|H4MOj#HJ?E&!c46gXTiV4|0M%k9yR**@y~mR3JELK+$?sDGAiVP(?~; zwQl~eBZUi6k&ObD7YzF>g;4Z zkx(U_I(NvW8wuQ}q*Tc%Tu!x}T*{OR$y!no_KD2_n*)`Z11S)z%tZdew1HXH&)}p% z*#A;P1+YdW7uE^N4IJ+SK67ayZ^dFN;V1K~Ke6l`=tLDw>Q|OH(oSiOm6<167&>)lTA0>KGDF4Mby8#{U>VK+qFi zFkp?KOByV!6M(xvc1JCXZWh_FU%O222o?}q@UC+!R(C1}Y(KCZG>L(AS>y?IWiTjf z69tvvW_wHi$z;&#HH6NlK+$7?vQf|*)K)Gfvs$VYCMIz(?J#)_RTXbpmF*v6U)db6 zIZz2Xu-`{r3F$kiBnb_ZL4vAKFnRrh|A^b@A_U(RGg*l&{dg*;by7iRZBhZR7Q*f; z*d$cnAalU|hQVvu*;AV4oEgUqWD~l98VeFmp1|k$gN<|a}%2?3CE$}T8iA;F? zW#`VV8IiJApZ@o^vu1mvu{Zzy_|h3uWYaC(h1Tofw+YpAoO0cmSmQ3%x%T_{bHAAC zLsF(>$y)i1ll1`k}QZH4?Gos5M3+~yz zkxk@{mAQ*0qO8Ja%!NI=X*CVepn;|F(?9=c%Zf#AHk(ff{#tg#YX1mc%ORGvMdWp| zkq{mMw3?qq{i%+`I93s7uyCjOsN}iDkU@h0nh+0@OPq!kZ2M{4u5g5g@$bI%*|@g@ z@p5l8#DY(rh#pfs1Wi%$h+XQQ;G()P&c#fX36ZREf($VjerS#X8d+T=J9;7sUecG9 zg?8@_S!J=qC|MTww*wl)_|;g=)=U=Rum z3-7<XHD;b`Qe4?kSHdJSmKwgbzTFXtp`Rx_raImn1Fd9Z+fVBE(ajdsJ&hO`aH*M-9v!c_i0>V=`s1}b+nKb#b zK^L=r2cKCKrgiePIUXM8zb ze3x%sy_rhybkcol(<1mFy_F#lAZCR=1>qS+tM=Jn%@qno*nACEs8tZ-cKaZjj*)H> ztdGYwZd~8DTbKMDThu7cM)Roti<-=E+CJDEsI(lYcEF9}X{`kp%(18t1PcQVe97Pq zew;uz4V1iT){}de4VXsG@q?DZhQ(?%oLOKx`gX)QgRz6ns*>4)f0#7`};xF7F|lIOL%g&qq1P~1hg z2NT#6juwK~$nlJAs0f|~@C~LeST(UpGJOC~kjIJVuW+agM0dvMqBLlsrl!prpL4;$ zC;s)A`W5-A;aQ6?JJ=kkavb1`s9pi^K_tZK)3Sw2rcRo+Wy_B4J$m%(*YB&TllSi3 zo10tjlTSYO1$-A@@|#9Yj^@Mv_f!A)?WLF1%c+Be=>c&8dSFrVV+?oRs8(VT<|E^) zmoK04*~A^2H+MOucmLD+P5W{R(t7_h&KdjGyPGzy^WzpJ>O}JFJK&7w#~i(Q!S|nx z`>3S2?5wlSI{u_?5g=L8Bwj=!Pe1)666||p-d?<6!{Do~rek=2?3hK1=X*T9ey0!U zcw*N`DERC%&yeP$_ugH;ZvEijU%P7g@)$24!zp~Hz^P5-j6U{6+7*2GDZTD5I=_IVew$0mr1jsrD=R`Cd39idRs^CO<= z-}|&LCQU9Z*?Yl77p7;{c>C?QVb?{2E@{x@Xu2Kb-_JfBKYi-#jEsy6FTSW*vu0p- zX>rLrZ@#r^`Lg5MwP08KnA~Bpubg^B`3W$^^KaeqrC&^()T2kQF=O7&sa@;x%P(KE zX3e;9r^o5%Rj&`gZtY>gfodmlBLd?bb<@ga zzdQAm;GR8=>*da0@SWQcnm=>;BY(f^p4)G$pIvL_q)#pz&=0YW9em$<;<4R3wg7^{ zXw=Fcl)da>fag_oECk7wTURZ+qTi`S#k=wv*Z+RO_o#}$p8n-$<30?)-FO6lf020k zhabK@_RUut)~&Vh+u6TA>-6%%Jy~hq8!x(G`SO(>w?yDbD2BX=kN zWZ~CUiV&3b_)ZB&(%jA^i{^Fd)>S$hN93J1U%T$NmoAz6O{8$opDww0@P+3spFcNT zwEGX|4cNGL4U+iSQE#kR@I5`rH?wEmd)v)VKlVsYX2w5<4Snpv`_W9$Z!y}SAQUv{ zu<0Beq2Qa3J#y9MmxYT87tWk{?WKb_kCc_n|7yks15S6ugZy6n_ha|mbX|7sj4hj2 zUv_q1Z>TUmT5|L629Ny5Be}I}e*VcvpS|;zMnCElE^n%GT(UisT&6m`tvQ<*Un5E{rrfBhup<;LV@##-~Vpe{O=pr%e&+H>!wbc*f1}5 z@poT8I($gL5q|cm$EHu2+USU*vg_v6Z-3m%B@2Dd$h^687cX9XSbW8HBdQ$-s-48G zo+=hw$L5ML=jE4PbVQS8K}mR;UJK>tPntX>GrQ(hzrEz-_9x7n`(5)^?a*YSRft8U zQ30zsB5RwmIa&;0MkD$SYu7p=!OMShN#iC*m6J%@=twn8O&1Tk&=D(Rhnmrk{qx25 zKB!Z>=8e}s-2eRZ9)4gL*t>JAQ|htBAEhyD51XP@|I&$9;IeCsW*JvRLGv(M;y+5mhDPMQ39X24%mShi@v z!Y?LFIHT_=YLIj|C<#P`7e7pY7!7ym*7?zYJj~#u-Q>xBulLD+ zJT=ed>X4R^O4I4Z+{Qid?d-T+cFN;Sb z9b32j?(117krvLFHu}?vt=qP7#Y3Gsb(Gc_R!ifKj+*Kzm&b*=2Khez@WT%u+pbN$ znt=!J`unqEKkU@0Q?rKkhp_9!6RuD=GWGM%&;k_~7cQ7LcgnbOjR%=(ULg(}N<4;DU__2e{fvU%WYA12yf)z~Y*sV|VlX{=o7l1oL;Mr)00*svu?ds?Y{YTjvK`)8WQ}iN-r(#fVN<8`p~s# z6wkaM3RbVXL-%eix^+44cWn@?nKx}Fxn|8;M~xaS+a1UD3MkKe z^y;~K=_1*EprFN$LQ|~qV7UC{*IqpCxVGK8cAPzR@~-^tnoC5_j&?UBy#w=nh^@@p zZCt;$$q|j|bg-#$ySyiK>Ku0a!VYh%<4)|-qqoKdF>l-U9d~ctO8=qBT0thAmHXH} zr}#26qb}but=bl@UgMEBT1oQ32%DWbd{{05uQ12uZg)b*h|kyPn3ni2@6xp=s`Z8q znxOH{-?4MkI(o|Zoj2dw;fVSjn>1!GwXeVVe&veQj1wrF!~#OZC;vaQ4EZQr45#OZHFT_vUEr9o(T?S*H%ozT8}rw-pt zpSE}BE;`xV295r3*FBHiHRP(BZ)<)`D?Fex$y`6T!Pd=N9qhm<<(0qceX=&iHU~6g zRLz3yCt^-Cu%kx3Idb&q2~!)#v?-HE zKJ?&gFTb#C@#2B!pD$|@8OS@n_~MH|ZtfS8r?M%sNQ_4EcWi4={|KpI(019u=0Mft zK(zyI01Ciavt;S~Z{~FC+WDf3F3MlLED9XPoEw%b`S`uD<$HHM^Y|lqA>^DY>;YvB@hnIW3_yMQzP@z}Q} zjCtq2A@@WJc9-wk$tLP&^gnCDgb5qgZ}R(U%=u>i<{i7EPNZGzteswC?2+57`Qk;( zks`Zw?t+&0-;e*JaNDLu-+lewdmo&0!5~cAuzId@RMRC37etFoini_aXV*TeL4#FG z7QFk;n*ci7*|6C|EFO{)8f^n))4RA`N4IRXZp~UOJS4FLtiU%-kolupB4xE4U1`RV zD1e(*gPl_MsE+_RX}U0#eqrG^vq!)9tZ)#76-uioArsfaeR?1o#n<^85>rZPxtxSKhd5#4|!VG!IS>W-7yOy@kyR&OLoV*JF?Ce(dp+ zr%fMl-uXu~YktYVi#oMx^W2NClxsUOiQ+;*rbFYgORl)`>3=-dy-(lly7itM{nm%$ zKRNZp&V$cBx7~?duD|sTRqFL!Hu$QM|9G-jpT7P3pV^?{k)4n4bjjscbnnwA=JAx% z$$0%yuOG`j@Hnh}bkKNvTD5C8W7;%{+nxaX$D7$pG!aa?A}(VZiE_pKKxnrw=x|~6 z$F}!GhGTB#Wa+pfpfG8o@lQVZVAs}dU^Fr{N6agQ-|)xnfjMlyN&g{}Kxe!RSs#59 zc9(A7b$`5N{>nA|PwLgD&uLSq&4iPlw2YgF4}a;2f1P~nv9G@IMyGyflAXc8Q%^kZ zh=lyw>k|fRD<3w|L?we+AgqG1)S-Sw-6{Hf_A`q2KVv}N5lv1!?u1JQU)jA^ZwgN3-hGANt@!BOZKU_3BmN`_^TPo3?1q<9_(z2VZ&RC75Fen*&vj1Hy}g zO*HdA#SX{xl?97-hjSX#@nPvD8A`&Q6|^4CSbin0|9}2jt=Ly4Wq{E^mt)(u!kRv3 z%O*7fT0IC9Xx~LyF|)pA@9x^QYa^nvPVcD)hp$|@^68ge-@0pWt*rF4w2UC)92#P0 znTzFz8d(*sDE~fT*i!Idd=Ey zw-=>Q!LA+sk8d|?#VUtCll~6{2by#iATdNRI~xnaHHWE52iS#Hhb?|Em1j;kSX{KH zW@Zi85(#UFgoU}<1qb0O>LW*V=-=<8TfOVPM;UU2LP3oO($Y;!?ZbcJK;pyFPzQO0 zBkve|Gnc%tv2wR-UC6Ou@%H1I<@wA|Q}+&XU^EwEm=$#tZQO8M&&IWXSe+S|X%u+a z2EKyo6~I0=f#gHxyRVf~8=&Gn1X#5|+wBiKJ+AnO>Mu}*&;(CfCJR~EKawpYGjei6M*WTr zjAmaQ+Ey6)mD%VSXr9Lxxc8nRci%l^eD8j=#}n{ssUhZ#;2_hC9@g1NLMJ2rNZbcR z^A*Z;9n6>dUD^`ho5n^@?sjsUHf@3|q?DdpT`;6dAXWz!f;3XQz2GtyEhzTsStuB4 z=mi{gbL()IOpxjE`oj1KaC-By+zmGJIv!xi-;|xLs=rzK=$gC@`U|A z*ORpP+m+$sIX=K#KeL_$C$+tPydOBRF+3P4a_dZyDKU|eBZ zSG>H`7YPR<5ftc!TekREUXNxx9uLXO2N3B@1DJ}7)-9bED%#2FLBvvoRREWIJMjNWDyrgj3mVzCdgT;INtlURWj?fJ*Bh53*imjiF zQRrHr4FNWK#T8c+?A_x;Wb7E*)VsoEYZonv6cjl^<_D9bX1ELmtB2B5Pc4-A6*#6AYIe1BqR)g zo(_PgK}vOmEIc(m2Y?R2Ff>M=>T;Kc^t_0GKVed!=#R#mwQN0Q!P2bU2BO-e*GA2D zusKi#IZ*9@8%Y*vjg{gvzL{_1fi2d9p%^SOmchHOSDr|9v5`3lep)DfJOc?Iiw6RyW zE`tURJni@s?)lqaS>kEIErK(yPG--X*7KMagU-L8Z^sTFj2(ly2}mofcRIt7(C6dF z^*jF9(>r$T*}B!`mz=w5$$V)xoStwHQJKMjT#jIl!nEx6EgO1v?|$w%XZP*c@$SF< z*&QxLVg)&+wQxkfp7G_$&6-|z-g!ORz|yhkM=$}gy2*Gn^zr*+&p!UxQ#&1ha;xT7 zUVhoiWs9P*@V!HZJn`5+WCk9K2QV#^I^Id|6w&0afqa%YvCfqa1bqoQ6VB<=m}_8& z0QsE3MsPX8U`0%kppA*97j>4Tb$TEk2tlq6ST*JK5e@iLv^Z!7n*&vo1J&-`1&Xi} zj+Ykw<+^Kce&CUBm#+Tv@CS$8d`nrdoH09C>37CgEu8z>BM-hX{?nO@SNH7Eb?7~J zfpH**UyIyv0fGWLcFLeZUoKrda?Cr^KYoAAm^Xkwt;azaL@oytN=qNQd&s?a-^03^ z`|lglrftiw7cYKo!e^ho_VR*nXUD_x@dR2$%gb)Q?&{lz-uvCM)z=Su@cuvE6fP}B z-C;cy*2Q=+D6Cg^`ihmWPMBC)RQklDk6|VVrb}RyNuGpC&?^=RK6LNh9gl4@bMAMq zPn`Jq>#xk4Gn4XV0TXuw%Svy(>dIU1ANtLTmDk@t{DIqVEic+D!5S-)xYOrBbdF@! zuQz?gvKK%9ELyVr@kbs&d`7I5nuErGz2GKojyi`Axu?T%ZD%i8^1`Q|em?ru1#`Xx zEG^TNP^kQt-(U6TVfTN#Z23*Y?t9>t+k(3bQJx?&(+!}N@KQ^?x|5c!cEetc9*qsEA!vfh;gI}Hk(lO3 zaPeqtA0_+a9EJxWS6mvh$e<9ViZOfv&EbZYo-+PGX(ifB*H<+^U7PCa%zSD z@rai2A;cl5q9+Zcr=8fPbN#k$ckS8%e2^FMlR`kN@$bFYFt_e0r<{Ue$?VTQx#ss* zF;~~Jb*t80J5Qc4!KvmLIU`=M_u`8#LdLx4;>$qtZ@!vgL}3Zi#;g#xN^Vae z?U*)gyL9fhW9v4_wus4L$^MoRGc}eJ7k)kA4f}kn*nL9!J$>+i6&Q( zQVR=9=6p8csw)Pw2&ql`cE?c5iw&-%(yTsY|Bix~k7qLx_v+u3sfN|s(q zzaI=~ol!KDmQ|~HtCpQRb=J zrm^x9QPL(^IwHC;L}#O}p*i$2>T=*DIUtN@LH9#uX@mfMvucT=7}QuLSX{D?HV3K} z2ddrVB$Bho>)NnkU2f}^?zCDA$pe{@?4}KN?bymk4vvRh?sc1Yv~1N5X%z5kc=S;q zL*cgVIaxJXKpv8-ISjhPJ2$NTcFL!-r_I{FV&(adKhB&Z00_9dKIo9O&f3BwjRe&P8XvKFb%_tNNLI5#ZxA+X#Tyo zKK$#@VU#Q-2l{Nr2F(942jFmQ+Oi59ZQ7y@8Nt3|TD01-bvq6NNShXSZC<~jUc2KQ z>1oUzGV__!tiiVY_40krtYgfX<{>^_T(W2RlnL|aFMR*)F@L@1KI}vQOnYT`sDILF3zXr3u4S^L$qSttQJQXpH0o)I zAReKfEKF=t3NpN=01_pVg^;wGFm;%67-tZaOhNHJtxI);ElFMd1T-XdGNwANAen<^ z4-}C4O(`_ZMBqe*V3oj-LMb%Bp>~}^twsdY&V#WE5MdUnkRodzc`*>>R_&&J%8E6o z{%KKw%2Z~&&y$XLy_=@|L@i3F?v_YeZM6a#=)=t&F-Q6IB-|!m(ONYOB~}uP9rTP! zA*~awOc|V*LhQpqP9mFOz9l^3g+pQ@D%TuyzdvhNBUmH+|>sJ+*4qL?jG?jgDAaMrL7gk@d70 z7Vu?q3$4pv{&v^y_3Jb0=lZ?R=p!IWYp|vB+|Tj^Ia=fVo&MvqZu+Fh5ZPSAxzLE!9o^`Gn`ui zqySS$q!SYNF5Q3Bdg$d{fG;8QFMmm%$ek!7XDb?fAzn+47A^--%JC>RJp)#RO3P4u^WFr@%Ijp;!LZH;I>e$ivuZ@j%CKg^j2-QJQAt^y zoLt5XaE)3ZggMfhb?bgI``hWO*I#|z4S&7%4-6DgQrD}O3*O}K*cHXOikX=uJ#Y{F z(|7T(Qtzi;R)cyJsZu2&O3JJFQ=3vUVIn6YEI%bC7f%6bw(w5ajnc*Z_SGUGR9B=| zNy&3!h>yU1gKQQJrZfsv|Ig2&L{oL# z{)hjQrTsA8zCM^7*!P?cW-XPdYzcrKU%mPbH!fKcD=Wbm1OVEzW?jR^jS!IaEk%P( z8nbeGA)D6Vr+?MTHOREN4eINSN6mtCif&nQJ2GgzR;^mSd-vYCeEH6uJ82}~?gQ1F z&c;VI*}82T0(--T4T)d5aycdYaL2K5;gL;`qN~7F6LuYS^Kv(0C{kYL!Ief?>E3ng z>fwM1I<&?I*-qBtM%Ao^nrEJV2H3Fp`|sHS2R#a67k8ls46mD4kI!!Ps#Qq*@mOf# zlEp_hYYHWf2xi@}h7B9-Sh6%yT88O(SyAEMwdvF8cGg6-UbaCY+AT5SXxH6 z7Az^-y=Fs$#z$#MsxSo_aUUY*RzQi*7wF%=KM*u;-aKS;{D330*UM!*fh~;m*VqmM z^ag+|R1m6I!C+155`-7%{3w{W+TfQGnA@p5TvX|Xd?pUO#rU6F-pUb$aDdTgsM(;=`T^WN%!9m zl#^<>&Qlas$U@`~<0Yyq{2)F-Muof;ir+8RzBt$%;2o%D0V{=Qylcgw+b2z2P*FDs``#N#a&;v#Uy+qZ7RI!wHe)To(#Y>z%KJUs&E)YGRXj_=$V*CGhcP9NVa%1$|G-Mo6` z;%{c(H*^T!N&D8VzW?UyS6>|Ih?IkIxRJtIu6FJ0cD+u1{@JIa#d~H>owWM9ug^Z` zoRYoypM5X}f2ZBMo#aSMd-LVzU1deDz4&Zq<0IO3JRUs8mOSQ+c+p-*OMKDt?W>lr z_pd+7z-BO>w91>o|Db)m=hIC;dgBjdsHnNz-4{`J@A3_RZz zE&ueRaoe`nlx`jYh1XR6yw1xpX3 zOH(bkOajtaIZ`tJZC1&z=Jgpz#>?z=O+I zu0H9QR<~St!`&mEtJmnLc?%W|zvUJr&9t<@6C<8}a>UceHET9{^yrZoP zkJQRQrZmfT?mfF_?<)ph5vWn?jgLQRSiivs@4vficRqR&(PPx~VB*l>4^T_DmMw3) z@x~!fkH~9u#KJ|3hyU@GjT<*+)W~{b#B=}p*S|ZoX#U3N*PngqMO5L4;#9+`_h2{_ z^LmT(cb?g&_mx*%Q6szd%O8G}SGV>@<38BFeLE#0*tk<|1OJIiHbmP=7XX3B>pU*!3^~Yw!rj|wxdE3GwHi3yx;~J_IqP35F|cz zl1N@LTZIxhaOD=eAw!Hso|cR$=s_kWQtVBNgd(16F%vzq?vPX$ib(*Rg{q*x6=P)q zs}gdtxnVsyE2Z)!#CFa!-)Y%qSn*cX=(ypysgX3AL57pcWyoQ)1R2Qw#2E7vQ+0&{ z8I3TWh_Xnt;20XnWl>_q^FR|#Gwm`7h)F0?$YmuaS2CEp^nt3yDnosfl}u?!Y52oU zZdkn9m)qP-G%Z;pydqd_(W{mURyO0SWMjEw{?hI3o8`$D(6168()LyIt8lu~RKgHg z044ZD;PmA+Xow9pFEkNgOJ=QFpdrLE!|2P%ZlQqX$Xkkfn0V>7 zaWcxME9Hgkn(=cqWcctQLx$j~4mU(?7aniRY%wFfBeJ)A-KC+ zaCdiiD;2CoN?mAiN-5=O3lu1Y;O_1rxI@qY2|+{LbNv5Za}rA1=idMOx%X-N-WPUH z!#?}W9$7QZjGie}vW1ZauL89Mh&v8~o1O`3vRa;7ww-SY)B$ zc5fFeY;;lrgp~DP7(ZE%tx_uV$&#aCDDuqB0TkRzg=muSGg;p7MnUHycP=H~(vJPhj-<|g2-=ATSsOttaY51*t#s9pbjsLuwCx3t~!4K$+xHN5N<%Tq~|3(k$x+Wlw{S8?Q+!527L~krJ!| zSNHGVe>d{Mmy@SLD4L0_F-D`r%|cJAtS5Ox#8GCl`Utd(4=05o`|F%U2$(Z%GRkq0 zN0l`*?V%tVgPT7^3-gf1g5A0=KL0p(-dwP!$U9eOPx%rXcr0cJnB_9KuyZHX-p!kK zZQU$DB5%QI9HIoLB2*VocPx4EWye)XbtqMkD#;(*2 zlP7%^`}C0v0NDi=gmnW^dP*uE$OwKV_Uo4~ z5T6}iQ8M9G!BfQHWG=fdacV1K@jmNwpO=9`TRyOyE z@$tB#B0*pvBtub(kW0E*=8c=vk%i>*`ue9RkkzsORPld*xeQ?@zftRY`Z)65of~+V z;OMb)*S5ur7Rn@Wk5Z+SQ52Xs8bL%c8TT5jO7M8JxwR!gjncH(!Ai+YO*)aelrYF-mGZ$FW$7&X zWkS)SW>Lu~3b>J!-~jfEZo@K_m`N@=2?f?llq8Ml%jE^^L&y#8A@iDyB!-D#Fri4C znwW==?%upEF8@yF_uqfNcmF~1p~iD^n#~}u#Q*(A=ZPrYDJMZ1P;j&plL@I1DnIIk8r!)va0rjB!vqul_Tn`fi7>TJX zH38h@2p`e;KwJCXUzaAw$134fdR<->s<`^~?R8-P9{P>?GfqH%KvUXn$BR`B9wXL1 zuCTDM*K2mM)Wb7KW)mCdEd`bDwA#VUOC34!@P{$QfQ8Ii`B0Wynh z3~t068IH_9q-ml{%W#!=)OeIJ6(lAq3w)*|NfBT4)8`^qeuz^g|V!_SrT}10jJzDNj~H$8+`)62#R5_m9d?Ye@ywA3+VC6d3!SJwhza+Qdjf`F3Ml6v za9E9i7SU%#*N&JcbDnj801!1$xWfq*8cn0-El=ORYp_}6p%~Gd(%CCTLZ@kR)|f=D z)hQ28kzoavU7lL!^e;XwR-)8LpMFXX*iAw$QKrLLomLhJG|6SN%$Pi>M2X^+E0m`u z{6;n8^fH5%V;#UO8TSwioP*!Oh^3FInQq;>wMLV+bC>o8Fc02KqoBN)a!S;4{D_Pa z01{4XXxhBl@oQHxbtet967CHQiUSs-L#PAw(WhmJzjEU;OHf9u8H0ao2I9C;Li@r; zg@p7ArC&WXD}HQ`-kDkDpL*B2@}TkY~8wV?_PO}6?L!s zpb*1UNWlO@t)>iKe+S*5f7V5^-@ar&eEsi#o&v8NaHBwF-frKx;m)-y&mKJZ$z<&N z#+zZ+Zzjekb?n&j9nTAaoH)+vQqcZ~#1licI zZdF87WUV@Ny7%m@W{OyVAcPX(ddUxS2M!$Q>+io{!GaF$S{*uch&6SuzWvISudJ8= z@)c4GmeID$m(HIKJ#Npib?Df+Y_%#Vd~q|pbpG6doqLU1efth=kAxiW_trZcJ77sk zT)%qN&0BY>R0}GQI{;CxX}h*3LQkc}KKJ(WIu;T-dGchtJu`UOs+)Ii<}X}m;E*@` zef@#Lrw^a3Tm5TFQgV|fjqgT8)M?tPWSO#wvCmH)J9_!T#h_}~F?PUfbp6^@k#}yz zJh?Y-j;8nUG5hu&C{wOHM{W{fB;wv(j!dHoWy+N5)4!jmub(5+vT)wqE*(4V+`k`> z&3=7)lqy$|2z|5;aowcH%3L7$VZ`0dTX!bK#I$YSp=RSoVEUe|TXJ}LRd3M(^OU>S zZ=61NcHr>gcO&la-@awy=n*1Qi((v8#bA1<75;{b16_jP(WZ54 z@87xo{K?~avrU7BznLQ-@b>N7TeoeFjeb_TRLNe0hUW0~*|2VH_^n%yq9P0?Q@>%u zjV902p&{qboyn6cpmXQ0f%ytz0t9jk1R3ZxE74dr7E-Z%gCW!OWnAl5^>C6k8{b9!XA!_w+D_gs43G^xn05--*+wnl^3PtXVT$ zNzshmyLn^7hK&}hvt6goHR{%f2jNb-d*_a0$B!1upFf|ECjtQgjTBApU^d7O^8s^N zz-?H$ylm-Gw{Bj)dGkj5HtlNFskdz9%BV+Ax^(GMvsSGPt2I78{>1Uamo8qcQMGd0 zcJ0mnfm^rjaAcCv4<$l)VL9z1?FWXO=-gZj_^>4&Ma=bSuy_QLtgqsEL9Jsu1L zuU#Xj?}>E(Z=i^00`IS{zk>oY%U-QZX)Op7ih+h0xQR$8NAB+4vBgX4yng)9v=PIC z7cUI(^_nyB{kQw|Ui!;ofZ&VaLw9amhgDm0YHX*r%`TljQMPE|xie?1S+)!*5@}SV zOxBk6%+=q{NRNJ^v!t({^UbhDx2f!<0s>W4Yg*Z zn=;dewQZU3^pTfAbN|xC_queBxOdB>x4+q^%c?okY$=HdX&;UmwP5zF0)c^hckcN3 zjR7YP9{_PqpE@yp(&v*u{?M73feL2C;DH;K|5B<@{v*40eEq?@-W=SMncS^egWbEf z=FaImcj_05CVv@m0f!C;2jbuYX)~r-kB+)CwqK8Xw^3_Yj_lim-UB%k7``2LY3g3QTM`wXU#A<9Wjq1mQ9~Be)K4Bz5d9ytz!lcGC6G+$8!t>u_VQViAhH68J+XO z@ne%my}jbsB|av@w;zmqt52^LOBYcqM%SJ#>ovBFOXp7wY}3Y`neO9Z7~7|3_^s>6 z!asiZ?WB?Kc$m#tc78Ky62h_c6oe2_ol!>|L}oAy&%R~#uOkNbxOM$vdScAj0o~r{ z+v)bT%b95jBfE8c5Psdqq?i8z>{&<`c|Y>_@#C_lQ35tPLaT&vXAK}i$m{DLq`)gDaWnaur3l5R zLqZA_Dl~fRI8pr)*Mk+MH3KED&ck!rvSo-r@lp4WY~Ob5-UDwBkEEEVr;Z)z(Xa1{ zUw-lTGzb5(h!uH2j+|R}?ioCID95#AiUL%$C`?hE-9qSBmg_@@4t;OTyVlh7pfVNC zo<7sMb5~R=XtdFi7b=kd^yN#)WR}NKtJegF9y{8jSI+~xcGPR#>Zhf@AiGv9QTU7Z zKS24InUr*H@80!$_LQhp1&w9XF~iZ2Frj@ai1E{AysSQmI(Q2WdjwOrcPoMH--@d&= zLXQomZcWvLGLCBG#jT$y!^*(Iqpdn)> zq}l9^if=l9`b6UnJvf5hg4sDr2~?{t#}{9G(Y$$cP@q$nZU+w>C|9a97{p2&k0G~S zUbbAt6i1Xrq(>74|3I?}WZXP?a^0TorK?oVaM(K4s^NwSuvn}kU7EfF28JFBDPFAP zyJOyC7h&W(GlN>RSh{p6#|(At)w5guI%kd_>)ogO(IbaS6)!RF-S-?u&(0k?4(i>_ zo|$^BX7x9E_Kv)Nk0a~g1F~r>JFkR};bdq1!-&WnMT(j|yify+9@*P_`LY$n){TAm z;K;6B$FJY?^7BiNevBhkmp*;bLQr!YLs=8B*qAl4`SBqqM^Vm2i&sELn}>ORCagF zUJjiJFoh7fL~1!nOQQ?I&(Q3FS7Pb%6%u1(4Bp-VN6C^U5AHu;u)88|+)j#&s$QzJ zC^2Q}ZUU@P{<3!Fq!sj*O1Y_Jt<8@$>orQVyS%)8iw6ZoK6(s3p%7QfL>rEmlc#@u zVCODR51S?R@t~T1fq63Qx?4Bz0ut>SHx^B+?35X1&i0<}M%d*~KKp_*zB25Ze1(d7 z#$ug1+=Iyl8a z!IQG_){04_meVYWw{1YWza4h%{m(zf5QV5+d5RYI%NH0G5w6iR``ip)v>qq2bB8fWF8YAD>xSFAw2o<&Fda?TD%Xu<{4aQ9oVxI0)&U%3O^CDd+lmv>Y;sFKej{L z*1&PY0GmK$zxws4BVb*pe!ZiiCj>b;P=XL(h}r1tIqr_?VDzam)U4OQrZuyKC{w9g z?fMO^Fb-ffSZzRtu{bmgo7Kb$zBah+N>PlO)5dYzUa z!xSu-pPR>zALlDv9MZ6{%z6rD&P0HooJc3U`StaWQs9;2oGhfuY5h}ehp zjd}o$quT5da{_`*p)MefHm94zBFvs1q9Q>RskDIT6Mz2chkN07_a4}H?&A5t5+#`b zSQfCi`SeLN#bFl6A>~XK=A<{L*`}tZSuE5F+{KsyeJrX2q|boB+*s)`V@sDS5BeNE zb^6G~i$^YBK6L3ywQ4oJy}je&;%G&t-IsO#K?8 zFwk7=4n;1M50FGw;#a9P`1pE1h>8%`BsQ`vw#;YI2&#f|#Gmr=71BW=(aef!Qx+S5 znb@JA(9or%gZ6rC%)_qVs8OQ^Dk+%R^Vn#FHxfU3@{lb-D||`4Bh*F2WeKj;unf|q zq@-qM(t6m*m8)T_)af9EN`=B~n6V2f(Y4e^WAO2gdianMp>3+wn*8K(E`NWi57
    L#Sy%$FEk%gvO{bw6ak2scACZVfBSfZ(yYnwE$U=tALUqh0qk^1K)RD97 zF3Zj;jM!DCeEETIy&ZbxO2{Sp@#3OI3-fVAr&dF+k(~VR+_^)Eq-Qr2wnrP?7z%7N zdPzfuGUTN9%tn(J-HNi^Xy#A~4M8G5ee12!BS&1m9X@ONGW(x(Hkc51}YmusG>z8v}l1*CLAN#f;V7;wk>a8xpe0IIZq$2n|JSAxEaP#V+?Nv zTBHN+-E)Yi7?7|oMSPQUfn81u)*CX<#Z*@t7yUe6F2B4veD`kJ9vczKoOBpmZCbV7 zv1#r7bD>FhuPmQ8QoQ=|rac#q?sdgISoXslgwh6e>xoQFi0QV}K^|s>8AHUo&j+Q6XE`-VHt4qJ+6>--c?#aUb;}7WcqqY9I0pzF7q)Tau6GqJT{QJ^q&+>|Nc?d`X?EizPNB|eAci(; zD5KtCuwmDZPE4>Je@(WYEE zD>Z7dXz}v6hY{CLg>GN7!Q1Zgwx-ykBj!#1T$h%pb!DPrf|-aDlw?o;7{yQqBO46i z=^HqP!_F>3dO|Xm+$D>b!nHMcX)w$L8}Zb{xUu6#f66g*AAbDNN1r4l#<1BLdG}u4 zGNp)N{NUl8BS#KNY+&+Lyo~`YqJj(o^`Wm{zJa)}ufK%?uO4vYgdpen>f3efaOvFT zx)mx%J$)j=G&asG=CRzBGq#wUs5y-m#yuB8I!Bf3%S%+oQ*g7*nTg#g9Jz zJRvbfKorPfb4zTyfgyv_N|MorLf!8%U4>uW@F)s zRS}dBB)j(No>4O+2w zf>2@W`etMsjvNm67Av@wQX~8dVGW=-sj1hreC9N|4p5 zt=q6ktwzn;wQSwFd(Wx_F~`tOZ@|l+MUI`LnZ;0zg}c$y%V+H5=?mul*r-~K6>HYi zZqqJP$lxKz1Os^zflJrEOSki9FVw79`SG*3k?)SJ->}Ju0fXyTtF_m_h)tXV ztRZojVs~4aXD~Jp-Vw_IRW%-j8IY`f$Iit{m+RN1d+mx97cXCKw+W9PJ$C%LbLZRF zYdG@VajiOb&OnIPXwO}|bUgHw$?PSA6ZWp6!H}o$7zoAQ3BwO_L-NN~RR^85g^Cq# z*{}cLp1qsYty8c>Y4xI(5I1=X7U|J+gP$qR^DHWzz+8R=K-if<|jjqX-U56Sij8K996g^6%$gaFjdpt~GMkBv9} zFVc{`d-twgy*fBJ*s7H5QfWq!Xwm3gw+*?w>CQ5t#AEh9WR+GJ0rCCZ!8SW@4uc z9LDzn+!8s616eGb>Z_1~Fj2C&C4mvR(dazAaxkwzK;lU;kua^5xx=(|S+p**2^STS zL4g_uyV52IasiQe12VIKXZ|8#dFIF|@jYY}N-bPRLqZZ_Vrti_zWLa;#miQ<(L|N}GCS=O*6L!g~8UnUp3TQHbj~aGaaZy4nHKND> zn!Wvu9^Mk0g-I)12}#aF4%-!lDq;UrfuiQ6v7*rszjqM>T^uh7dRj##Ey5}KZpiAL zD-eN{tOPm{pJ~4Mn+P8lvojz}i&9==G<%zko&;}UD@+_e&?G=^S+Pq>!8n7C{84)F zJ1sJJdgt))M$1DoszGHSGCa@I18kBSa&j9FEdY>c6ohZ{9uhv{Dli87qJu!iEr z=Z8G*1{!(-F&80Ec5QeOIVg6GoZx=4;!&)3^^lX3ld^*`uh)acfpuW1khS zTJ`%4n~Rh#Yq6m|muZimoSaSIutz_6V9T%-EmaZ_6W31og2{CF`0>TFXCkE4s9pQ# zmEW;pr5YtdsuanGOQE5x6=buEIf@6ls6Oyak^J?9tHm!-oM19i;vYnYmn>OYsZhZv zb{EBalFYz7FbzBvpLnoJxB&}lPCmmalDq^=MKpK9y|8O0FnB6JY4Y`hA5bF5Ol8>h z60uz&ue@T=Wk5;k4XGr}gh$XYf)oh%Ath-G-Sn^wTjD1xq7j2v|$^AJs$&rYOppkpu}cT!M*A= zvq^0g!dl==!ZXm7Xj(LH#oCHAiiYaMWEMm&13DMhSO= zfj|muD`^UpWU#4Lo6P1+C}0@^AuoA$ejC2um4u65~KzX1TCAbKMoFO&xl~D*A=Z_vb?ZbYGCx0>ZS@%&7p4`qJX?Vc=Zkw#h5a)b>+ zo59F8YC!Q!#DwQh-W)bKChP{HT|ns)v*ygL98?Q~IdUaOA4?jQNAJw+w18iNW|+>Y zy)3{^w6|hA0~r7!N)x?}6=!M*bRj6|uxU+%=+LZKwBTn>a>$ia6C2xZ;Lxw8ePcnL zqCgeJ1gmixV8gYJI>8sv4oQqARyIg`D1v03yATyo!U8=`&DOQ6*Q{ByZp${TBBXl2 zgDe{G5l6C?sK2C#V8a!*b{oK^%_v z#83w)r>VfJM4J)J%A~IiV)sJx(A;2MhoDM@agsu&rV$JVx?iMe8l@0P%B6CuXvTvR zryBSTSjAH~HVQGP6Ym~5Oap^LBy{9-RbjsJE~X%;h^Zbd4Y?H-qP_^nlWHr(lg(lW zN4UY8I3`XW%F;o#S5_NFYRHxA-0<#uU*M zf(r{|nPri1C<5oA#uM&KkBdMeJX;M>wN@Zk8|o#fqZWrWljg{5kP169^*T|N+<020 zRDL!D{vP0_s;_uhS2gn#9=#%eh^!<=Xh5MsKjQN*L+?f$yK`^(x(y6W(HL`vx-6kZ zu3(!Uk3^&?<{C4B^|gse7veL*auE%PI85R=r?cTxqqfzU0IX8MKZ|wlobP+|8(6$Z zaqPG9c2GAJz3hk0$BLZ!L^-4&T<}2tCX_VIq z9|*;`s0k&JE!ugW3+%cJLS+Ly^P9vPRITPGqJm)zDgrcYQsxH~MM$O{lqQwJY)C|G zsTCC=cfN8jCy%ORa3jE(k1PP-yyU}Rfi1Iw5nvVsCtF#vkg!_9U!&CindyMmeOr-3EpKaP(K)?)K5P06tfoU&QG*g$;{MEn5c9xE!G zI6^MzeclrCiuDseQ>Ls3WIZMLEM!sbR}`R2;Xn$trJ&c>KSF_5PE{fT3joS^bvgb0 zeGNE718R(3rlH!jWWXZGoH%wYBO%VqY&v@MX!#12dJi1DbLaL`CqtSyY1*t+dqj5j zCl4GxcslfSVBUPa2M_ZL3}Czi4NSDi2N6d$Y&fV6RP5EO58%G+mqoX3hsz`o4{X5; z%0fTD5d;hWHM?EAcKi12BeGzR-n|0+b8r+6zxM3izd+H#m=IqJyK?-(`elUXp+wVK=XG z*ki4_4OXsL@%TYx_Z~fnNCHMWGp&aXNYISH+_`%Vek)I4E+#y65G!a9ne~TOKYGkK zA(Gxux@>umJb7M)wUV#A)r~Opu(c?!Bu^mU~jEWU1fAGmCpM3OTr!JkTryBxgm$#^H!Aa2DJ|%Pc#7l0TJmi`DR8__!#&aqYt7oKAwH zfOjNuU*rQ*=PTS=1Ys$e3Q&wRgkKeVlC->NG>RIBpM(gqeBmw7T%u43LuE@GRzN;V zOr|DOUg|~=E0G4|{Af~1@43}1vUea6zrcFNj40CiRqZE(>*`)Mejp#ZGF4jvd})Hy zg;G)0LnIbfCu~*mLREW}^!1;AgaX2+U#SbBQ;BC{1igLhhF=%``1pP};wWo)vgRquept_r>z2&V<>&SJ+rzhPUe5@h{OLzae)^HnOE{RZ z<1e~hW+PL@X(P%{k?K`BN0>)4`Y{C7n>84;$-G8Qku{Y?+_73#&H3hy{(bG48JEwU z89AWuoBjIOE!Hz9Pxa~C&exF{b^X#?ty`fH&X+siwTo9x1df*3C7=Q5Al3m)T+BUb zmCJtU&;h(9`B}tCL_ZfHiZmNnE+08$VEEMwwuHEe{re8@+c*5m6-!#mo1Hs6jJW5a zaZLSW!jgG&E0-!A{qS*20=|@@G*vhv79t{(!f6C>Q1Unk@Ra#Hrgr@XvO~vOfw^q6 zKYJLl^qXlPy+1y{$9Lt@MQ;rqIAP4YIemPWE&OTb-G(6CJY~1G;c2C(}>7O zY&oiOy5_T|kPeU$gaRtDBNV`u_NWFbSyuru(k$S_DkPhcDBJ^xL0u7GVZ<7?xmnHwO%PQTcP+nwfhzV#aSOnNapG$>AlI17anG-8a zex)~6nF76Jp7mF;5Ve)cHl`#)=^(>^(q#)l_@HV9O_owr4W%LSRp>6TEbLIxMDP=? zrjkkilt|6pW${hYfWt%&pqXSM_M>WhRw^%L(I^3WSh{Fnq#;U1k(`7wDodIM52HDJ zWYMAWXK6?x+N;PTm8Mp_B41UV3;Z~N|h=hp|DSgeU>LiCdAQ{Vr4QgwV1j*m%^A3F6ZbG?+h9~ zqEmqdf=>`xJoJ=W`%)$4be%zHW@oLswPwwlfsUx^;llti5x1RAYmH+^&z(6_wnXvk zSFhBoU1xCrzC>M#H>3DfhK33|kOUF;1lLdp=`|^d@uNq)+2M_0xeFE&OB>nrV{2Bl zscdr1nmN0C`SQ7Q`po|Fs~^{GtY5!=v4VM~Oqw)f_BY`-Z{{mlpl`2UI{yHw#SsY} zUY@$Vxg+l1t(qrawxEZIuOv``3qc%;Rl0;f&JO`21!o|&x)2*7Drh9%{$)9iBZ}I$3iK%|pCHykG2|S`o z46pyq^}qks{h~WVVIEne`*VK6rMPIq%ccK0-Rmd+AO(IKCI4V0{?4K?!eFx$qrJXE zhK?Hh{?g!86Q@jBwq%h=RX_?>J2ag`+FHi`pXVFCJb=V^#%782)7|Jd` zJDFl>CsGbbuGaNP4;uVJ?)^*;Y_!m}rj zSuMaGu&)q1K(YNsiJ-+YkAIFH*z^5oAO5gzW9~vlMHQqJsSYvLqZC8+I`zdlQD-b4 zRI^s&Mpl}rH&v=w<>`}Wu$R#jCTv<6+@^K&om)1@p)GC`KT)v){z!{L7P3rDNfI=FPLGXW+|+R9n1o!RW^iVP7Z|+#k9CybuAAy-m3h6jo381?dsM5&aGD#dFcV zyML4C66{+N3xYe*Mv;kZcuH2keMq1(RKKeGY=Gu1`H7G4UWL5m{ijQ*VR)YX;a_=^ z7pW>pr+!h@$xf4%tE8d7q|34{cWLhD*$}BR;gP#pq+tCgD(UR!?w{OcDcL(KPY_mp z{+PczrMqR>_gTe5DR(K^-{oIXoV;}XI|>Mi<@5UbM=9{Te)>mi@;4PORwS~_5X%8p zf2=l%wrr1!!!blOk7}ifM%2!M03K#pn;VETia{_D>QNpdTynO6gn4V-y5-R;SGsrW z{Poz;`}XV+jRAXnC}N#XysIBRdcch4d;@^b8)tBPrp}R}wP(0xLQD%}(_c2wNF{MI zu)Vz^(BB(_@Q3Zf@5{Lt-kVj<84x7?8 zkP9c&6fa(U)0S;=EXffzMMwV(uDs-E3$LS`uC47{^94BivGhd zufP7sDe!Lx*nhl~|3(>_^|HCosaLzV??_8Xc>Ew@!B6uWwC|WBFt-Xm#}F5(1M>zk zUsKyWs0cA46O0TDnQPE)PmYre}k!@ESpr}H;IgW zrgp8m{re7ha^#5Zo-@WydHwJt8%sEJ4u$>02&-gcFKg zx#Pr%6AvCd?AonIqlQhw@7!_NoN0-v_PB&%#Y#{$HsIXKSW!nK!V^wQRIY9*GV5

    zxi%B`#Yul^+ym7B^u>)s7>41(WFpYActZh{mgJx5dJ1~Fer z^e-xfN=!(S1@7+LvR+mI>VDV#+}dXoR2FLIEI$;Z7Mao@lAnzcp|(yGe*IfsA$M3trLmR0+IpX~M1{~QHG_6Ukg$|2&RYZ;$V|!a2YZFMaLnVQS9;|KjPGekunMp zEcWIv`Xy@AyBcqwn)Ms^A3TH|HUx4Epuyj=Wy?pNzmS-ifMevQO`8^e{;`OZE?4s! zH44;fuyNZiiU1HX{TAt%auJ02DptN8*luHc2j~e5={@d6MSebVBpYg81qw|0X#tCX zpZ@jz=U;p!fTQf^cz^STjku9a|Kw9aNvGbcScyZ&LJ72j${yz+Y-Y$Dp&Q{CBLVcw zBvX4|$f*>HipMbj=S81fxTNv&sN107kp2UY9X)#Z;Gqvc`Yc2C;;4r!GBPsk)JagG zWAm08gD3O)v!52W=+s44Axd9wC$v9y+ib_tHe7dj5t&`$b5TJI(lsF`PHvpP0BRSl zUS-CNnI?no__;HtJb82F&zCMM9jGiwEpU@P_qpZCiwneYgPL@cs2#F@$yPqNUw7kG z){npY;(ks2RX@9PaHo8^pzM5q`}ak;e?UXv?+-7kO7hUXXDm7VApuYRkcm8zgzlA! zy379a(z}Y=XFnHmWiL{G_ilE(sD}G;=in~!MQ+@vC+=6XGgbe-NTaHwxRl~RS$hV* zOZ$f}|0cITqg^Q;CTMve@m3@iBM_5~>sjbq9XWXsJ<) zh+&;lVt^qCICw#bFwM9WryQOTM+jLMGw|f31P@|Lm_2P+DIhoUHXHXiMQb!)eDJ=X zpWla*zQk-xtQ7c&A+LC5a0|pC8KG>z?97a;3#uA%#1S_Zeh~*1J>t6-qCyI#E_VlU>gGrgRO-4b5BSFT?AM#8i^Rh!3)K0Lq$m z{7(db9^$Sd!2}7Dq%5OdHrh*Mb4rAnK~Ex{qfX&mUDEsedY~u5VX#e`Ht&u9pl#>Q z^41GMQAxR3xTB=YYzz|phJux>tur^6y2sm#^nA@4%}TTsB1?a}Qwde&XZcEZA$#(p zdXV+d{e^l9YjE4^xO0}W)Z*wxdU;b_WKQGxD%ObdRu)I$MK@g1FK%RZXXSqAewV-5 zb(7nyhi*ClrMA0UDz9g=F1c5YQmuHAncTP&bCZqiSv7SpWq7V^XGHFn%l(&|EE2t_ zyqnrWF8BK~E~zXFu~s}&o^f-HX>2z;^PaR$c)BNmti3RkJ!UxC?ommi6}zk5i!itXfjD<;NV@Yhi^`k zj~}?sJB(wt^hAILz_q&f-~V9C>b2o_?-Egn6JOb;Bakd6%rX&}18N3>$*=<}A@(s( zCSP!zvXra_e*k>53esr(0s}dwm*nxuDPunW)YqFhqGHFwB*g0)^IbRxo|gDzVrokz z!2qfRn5!gm!h`HE(k`(I)*%V2wY5a~pjfocoDYEaIRRLRhy*6*$(^rt+YXXO)GDGe zAZmwAXTr}e(?wiUR6?|qm=)sxrxNI0Xt&XCNWtW2gn%>wQc$i1(y(OFXuN!VJ-mIa zVnwXoxPE=UJbBx6>Hywnqc(2}@R1w}x$#1cATkvk;o`5{WTTAw^?&u}1rGi$f#jSG ze~M1>>kF{^T~hgh|FiS?{cqW@_1n`wXY$gUa;p;15icHd^YVRGR_?aHoLg3+Yz%cH z<4di0xehPqsYs@ZaHBSL=0naTlKd4N+=*06B_Am(>(NU+pnmy#5Lt?Pef<|G@Q3dC zFTC;pcoK#s_{=C2hcO@wtmNa;nsIUiwiWR4!02>%I9y)X+ez#)DebG{-dyg2OsyA7YXo*E3(@zVJL8_OJ zIK)v=y-l$I7zmmRfC6ckPQwSCoJ2rWM!&qw7sM9+O5`NcvfQ9;I>XQrBlDCfg2O2M z%ZP&=)kFC~8gw@<+lhT!H?Le`cBLasQ;}7>W|#KBp8eJ8)?U17CE6Y`Vetd$2y$w|>IL13SV~KMmA=}J;h?CzB`1=SwUTU= zV2i>>)FcvO5M+nyQltlq7`gzK22(^4EV@7kC!#NzH9hj;S=wl2y+H+G7h+bg!6X|+ z5)~6pr-;F+F!=sM7RCtixeDUF*LXrqZf$=1?-`D5U)A23A0qpTvkdi=tZ0G+ z$r8{f4X>kkVN3S(iDNj-kQ1p9?x`oQN?1`*xPyM|!ICBfZ*_7K`$po)u^_OJCki4V z#pTOa@FrF21g)|gq%nB0JLb%q`OU0ZXzj73fmo1|3S;!b=%V&Q8lfW~>6-V$5Bv5W zBCafnsVCYVVh%!QX1#!&)KH&pB?xwc%sQB|kH~N$4UF+9HJ&05`JhCQ2J5BSYNOBhDA&KZZ{!%X0fNk+R|4|cQfqIQ%;OUf4da; z@m|)e*Htpa5451JvSFXVzXl2C%FNmAg0LeXsNN)!!gEb!`=g@L=*BL zbQNLKhEpZ)A&~;u$oj}WW;+JgazZY~h6s1q;o+W$tq8V?fD5kGbPr60s#3b}00~N^ z^@2qp2BM31a|n^fh;4+98E6v?JC#scb=rl%sx}w_A)EEyt+4m|^|YraVwmiZoV_l1XNbIhX=2~*wzLEZ4M%HPm)I;GsLd&h1p(qDRH0+}v6V!*f~z!mjj&JfRx>-IfH9IRd`oZGvSSygALx*an^H6TH2mn+pt8UujvaS!>D}?L0)Op?=Q0CLz|`kFI^%& z2tSd}i=W^|LT<&DB&}Oz^!pcI%H#Fz|9lGQzxwK{zq*yoK@`DJ@s{TqPA^|C=BgUU zSrUbmu5MrY{(oGf-{s+6ho+{2JKDe;W?+tTL1dtWV+rQ}@UQ%N1GcPRyK&v>s=Y#wAd3u1&k*1qX&`uH?Mzp z;&`xY{n`y@PMr>_9t4yjjsbuVBkylsv&z$GSh{%8<%<_9lqnN_Gi=eqAD%pVRJ3?0 zKW`5w>vfxT^|Ii#!7J|Hzf-Avd4t|~Dm3)!<#UfC?=4&M%k!sCDp#qvch}C9%YLmfJH@_w3$v`b6m26UU1cE8^|r z2QJ;Y9=2t}`f4@n2~l*K$cX#vgO}a8bSXolNl#BNU8=0pX4$fS)wEt}v3`Sax`VmFbd;$q^ub?=E`I1)HS z20g#|^0Ow*nl@?Ko>&<_&6}Oe*LPL$ibDqu6e^T2JvH&y#f#6JK3$+-!5senp#1uE z>k(k{=gG5S%}SRwBjjkvx;3kbpj5PI5vmPuQ1KJ#CRv`qSsm8Q1@pcO@b_K4V&$O& z`wQmF?XYGnUA*vA=rJLJpFbUHvsl)w2&OIf!f%%^Q_9QJ8z%Ss;e%fm{=9R?7A$Fw z9o*NbNfTUzUYLAHKo9D#(?mtkUn{+h`d=0 zi|5X-U%lc~=<%F>zIk&8Ub%FBC#_)_Su`mhuW-} z-_7|pFd%pCz}#|3ge(Ju{qu))oO{5-=DAS&1>TVeFB7&aAlCkF78H4-vxPSYaw^sY@gs~e|EHk+**H4EI zZq+O!In~G0YkZ&H_rtDw=!~;x%-p(eZMouw7k~flkKfD`d`4r<*s$AHF8X=y#P{n4 z1+iFm+ge}=_9!x9$UwS7U%Rb%>Cs3!M(dJ|M|xNZ_`&J-q^NbjR{-o z*r)B9G+48GS&1SA&K}#}vtg}gk0Z@G`?eKJ$GtiDz>cjP39)O#hVARtfhGu*D}P;b z_xe>NYajy8M;b0#Qe<~zRt!a~>F>Qc^xMf_2l)8JJd7IGr}w_?n~=T{ZuzhPFmV8z z%{uDsx2|5jnm2du4@SLneE%+Fb3|h_5LUuS(`NJmxQ1G77cN|=9aIxBmY^$Zzn$~e zfWCxbxpLv`8@;=}JG8$eGxhwbsE*M>?n{s&(tYXQl7=pcJhR0&;)vq z)eZTh{(gG@_ODaF`tpPEIlatlm;Lh2fWGgKd?$~;->-`oO_}sLEcV&MhvSD0qeYyj z^Je=tu}`APlqk7s<;ry{R_JY(PbZAErKWg$`3F_4WqsF*}B}|K`EzjKly!_XV*=|H~-~SJvudkc;?AWXn%={)n-kZ;%heA z(o@Z@j2{>N{K@ngsK}9FckkTw{iLrKZ`fR=N)?HqCc&oB)QW36)AHNdvr1I1=ASdi z_mjR@x@u*OdUf&@E;9YADSF53%jeEtKYD0q$ce&*i?(Z3ziW#&8R@C$VKn$jF8dXG z!>GqmCwK2YdEA|%*hiWeS7sFuD2~M@#uw%X02x8xDj>g)cJMk&tH7@IqI1r z1qy7~yqRUUm&Ubb-TG1E#;2vF9$dS2!O9gJD??m-qE446QDV@5LA!#Nj~O$@)5rVr zxzm>p?F-(uyK2>{Xre#;^wUK@&EK_a_kaQY)~{Oq&Zu`ieRHr?xqr_NZ!dFtQqrwU z=RywcZPl)==p0aNu+fPXAW9qB?9e1cMwKd4TB)gl8T-f)BL=Al-$LP8q0YRlmhoGS!!>rF&4 zMOVU+^Q_riV@8Y^^7fc^?ZO7MZ#%G0zflw4zkB~)k0wo`?%jz^NW69I=+Err9jjKH3k~hmr|%@zsuSNg zz(A4`#*Q72^v+>1#$35@2d$E~-mcwyHF}RsIio=47mzHM8*ttCF9A8XFMe_3xjYf% zR+sxXzGfxL$}IcmS5EfIZul$x@2dyg%3|AM!@$qgtW{fV_3$sOT(j2dpMF9O9Cq#E zg_Fm&t=ph>Fm18VYYiVeKJ@tEnO{#yd>Co=Gh36B9Vj}ORZjcIqeg!)bw*I_+BW!r zc;DDLfQop>*j1yFDP5@wXW*79TM2`YDzzKhjK*SRDri#DkRNY`T|;r&v1vmULW%7= z{k3aX5m&3%tfMny$)YV?uFCVc6qA=P=Mt7ESMK1!!!VjVx39Ns*}`u00OAdLUE}60 zA|t{Pf$_Cz(zK;jYewYn+N0+WGp2@zUAh+)Wz3(iM*aF$6J9n*e!vcOrY!evnm#-{ zT;uIqzDiI!3v#2TPNPQq))MuQl`jU19AYbe?~csOl}i^d`DKwKJq>INs!lHI=QHELU3#*!7P zLay2k8(X!eA|=bB>3{O%;e!W{kk7j|ZX)}wXmE78D_5@EzH_Hui)Jn(CX%K?B}#eZ z%TG4M?`F>wrXW#RVHct;5#ndBRoF+BSo}QGJZk;`Q z&faaCj9wm^)MP^JveD7LQ-|$q7SEgV>HM`@4cX{kz7G8KZV4nWp+~1N+UJ_d~;Gt@4*F#dahn9D*{kqo&7l*~XFNY~qQ;go<2E`91byhdS-(9Yywg(ykf8cXS2;<#Qd5{ z`}yU#dg&tB5e3_3ON)H)kTcg2eetyP&Ebz2%qxC*^VV(KZtspQH*en_GHj?oq-<2G zr8mH>#6RR~mxsgG*H@F4mY5jtACL40fG0n&KA88dpk=AN8d2;2UIgg({4JcGZ4sM}N7<$W*6bQ~_X@VNO2+N=# zxq%{Vz2Ye<$7`uQS2!W8MV+`#2C^fFrH3?{lynq;Mz|Pnj*<{H7th3OTn$}t*FcPc z(xl%T8o~mF%Civg$QgLx*s-hU&%qPeOHAaLx|nG4mEA=weX#9B-gob7is?GVu|;a& zD0+$e#eS~dOxvUm>=0^QOBOA>f9K}Dg9i#1D$uM^BlaiRCXJ1Ke(}P2uY7rr91E${ zuo*1^QzIfC6B* zU-|9(>$_Kq06E3t+(It|w^S-2o3DkA*C+0AQ$+qQ3q9d(;d-FEER|L{Six2MPH z)0ZASjZcb8LbXz+bjkSmxK+W+1$|_IATH^@o-!k-a@C>z2gE#yBJ`B9lNF>wD=W-F z9jOC`vl zq%$^a-a7Ke-Q8<~?eQ^(_w2fP;$+*loyAHUIMi4{6O11$dXMe{2d!BC%h~<=x^(U4 z&|_>uRmq3{^ge86YK;#F6_T1(t7aV<@xz=MNzWeKICo~p`i(t$_O@$H0Rj0bI^y~j zZAMD;(S%ntHg8x@kd~OFn7xPhqYG=*vdz8Ir$YAcPLFxK@VglrMzQ6!QHwRD^ap*?_QYBwe8-kP{}g=x^}Nww!+fjl{SaISm{zN`}QB$ zwMUcMbqkhYy)V!tax^*?y?O3W^L=yX95`@La#9j=5cv+BvdEi21mFdBGqn)6MWu?M z%=V$qDmI3uuV&9#zHE7eDph;*?sMz@J!YxiY{d2kzZwfTE6y%BXEI5V3r&dXI2<-< z-G0PpUrw9&e*KE&KN>muozEsUY0<{w#DU&`?gm3eCN*Y^^%^!IfNGO=?FtqwEUq@F z&YAP-%3ZK<-Hx4y_UQHHPtibne0>L~U#- zQHfY^TtWj5L}~RFNTx3SM+u_5D1I(1M5NkasL3OP%~O}=`#IkdgOOZlkJOtZ ziNQNOdMJ^11;4OAHSF7Hw|A2cBA42vj)v7k?(rv<9 zBO6x>%Cy+L3lwHC6B`@1dhXmnIYwRb7UYLnVy!5hg1eG@;W29gm(D|iu4qkoy$%@i z#^tb^wJTI!vT}8aszLlpq{B-G4t+XlTItF`9fu7cH({d9l6m&*X=8!>{<#7&Gc%{W zJNkG?2u0AD5`+dfF1wTCX`PxuB}N!g0BOeZ6WFkTvmN-DdD8Riil`o)i{u%T*xIBn~MQs)P+ywun9o_5Lqv z;%>!nd6E^FV$o@yJxlgCY0DM$^^jm7BJ8l6hhbgAGG&8z?J8NmVwxq>%g>j20=(ed z!7NHmNnr)<<>Lo!#9kS@hKx*nKv2w5RVUhHm(DB)0E$>B-W<4!FwGPOiBKqLBepBV z{bP#p9`7J30%VwYi;e~{+G=6y#m2_wDNvA~q!$2R&hSM`j;RI`D&Qru8S=UWj1rNJ zd@(b}MK3PURH-#jPM+APE3f!?hS&_h~(7t9DaTv4s{_LS4d1<4#AjuhZx|EbONZ{e&0qD^UhYs#twtU6Xm1~55t7=F94w?aaifO;d zur3VS4RX|j`pRmF{sTjuiOC75q>$R_7wo%vNqU_?D|R_e&1p+*-=W=@(PKMx>Bb;Q zNJ{a|k%Rh5GPP&`b}=iYN<5*4j4G)lzPeJ7q+so$z?z=N0AG4mdV}!dKl}!YgEH#~7qy>LIj3x5Qz=kyg#s>(SLp<|S&dEGz%|i=to0 zD*5Zl{NJerk*GAT%(OHnR7y%pa$*8X=fs4#jEoE`BAH%JrD8_hyBG27`EyZIGU0LZ z^Dy6!e1zDm(c@4pfRl;Yny9ly)Fq#r*KfG&R)7!~rvTZglw$+2=*>(@PG&SS+YvS6 zO6$@3=PS~IP%C5 z$sFDb4(N0kKOpMG#75t~bsO}|6`0pRloB8qH%-(oPCM+x%g0{@`QspPOS;8jb?NsU zIvgGmiMlyGJuN9AHvZ{jduAFQg-{FVEL^N)uQ%VBHRn4--BiA#bD%t@3##6{dDE7W z4t!HOssT${D)5>Z6PJ>h6dRv_$E3I*0kV2sW=7hD^Jn|NJIdSJn=M{Y2~x_ofWwLP zAr!V|B&#h&Oq<0^6{WbvP9PRXW_r2=ebf?sCVz5LT4H>X&1&WJ%b0{DpB(4qFvSB?*Vx$0o$a#mC3f z`)P>o5>WsI6E+_c{q)MEOBO3(*EOrxt<9M$uzlw)+|%!w>B$jyZsA1=#*6=?gB=rb zM`9GD#K%%&G6(%p-NZ#lTQe-Q?!@sBNMY7E5}!q<#73va$Kkf5vEo3aNl#9-XQs<` ztpaNm?TlODrLV77lL7(_uhfMwgSXrA<-xRibibbMf-1JHRH|d;GB{`$fZfdWS)YH_ zs$#jfhxG4Ovl?>-@qFjTb@hX)4D8ararv^FRxTIm3z1vIFo(%$UGl?sttyor*1KDS zie)3NU1aqRC~#aXlCoK=o%G2^vuAu|LI>^0T)ucgr)t%r!tdR>adY;huaZ;JBBG+& zRjb&gMzuDT%6D(wa{so?;%$d#Ii?>>QJe+=@yzt0y?Y(oyE8HRVb=~VyELrbt8wj) zE$ZL8d|EaLZI%v|%XO+=o~VHxgDMSe-R$w*yR&A^x^d%1y`akd+qWFvw!_*rYeel1 zGz)?_be6OU?+oi6RAFfIdi^@KcBUi^R78^OB|IO=RXzEdGzxVz}lV;4|yiSB@ax-c5%YR5 zne4W(v*+5EFLz?!zVw8I?JK@RjqjQ%H?KFn~Dko;FWlg zL>rf-3Y3B4zBlTfwpA(&?bW?Om5STeF2_9FXie`}qiUO~<=R)P*g2@u(w~21kmwwi z^w{Tvx_9ncr~1hLy(YXn66v_lz+t}xuaQW(mWeWk=zMk~$*I(PM znS$CPj=64KyYlPonag+WI(zwYgU0n|eEq4&KsLx&E8gwzhII%(qgjFd!l8lWhOQ3QH?e0FTzgb^Ml z=NNPk8Z_w4z5Azb->z1%+|q?V0mI5%&c12KmUFjmoVj!B+}*put5)U;%-O$BueXK| zxe|8y#NE3+yY-|}RE%mP`EFRb;^x&0Ay+PhUb&c_5VvaSFR&Wc5Tflxwc;66IN%6D4 znF*^w6M`7);Y}Pwk!n#00ECFTTSAXR$+f$8?q9R=R@gNFhv;3++P6D@^VZor;m2>@ z^fZai>1^onsbj}1SiSP>*|SH(!pf8>C#wskdeUU1jvqbpy|2H%bmPXvNuN!dI6f&M zUMPdutC$U{Cyus2P<{R=>g_@O`;QsZt5>f)xpSSm8(ynMO_UgTu}_;l^XP?hpM3gJ z@VvQKFJGZzl*r&o>4bZ=GV$PBU z^I0aN&t}sSdu_H2o3&`ANl$UwE$D?Ah-{1@QjyclnKR%267@M4$^5deP^=30#k4@{= zT)%QjDumEyG#kDB%apHBqgI``m}umFl-Z(dKz#vpAq*1M(B*h%{Mf)^#q6G@vK1@o za^#A<8$p;obi)RWAU!?w-adNhuP2&|^XG;0>ve2byHNvd>b?DPat4rSij*SgROkUs zN=(-Q{jl!t)w?fz?#$U!m}fvE(NjwrkQXzV#G%6mwrtyc)W;ulgnULu`uv}MNQ_Hh zds$*VnY?r6E7Z7UTQ+->;^R?oQ%%8r=z^iQY)N7xsQ>WGug!ir9Y$k=X3aDyDbdl< zM4l1Fv)N3{TZ7rpVDJI-FJHcV{p87U<0cq9J#{9Ne_(*@3`0Fk3|*PqwruP>aERGI zx5+K2L@)C876oB zey4gs-F@HZd-vJJRi-aPhuinotvXfpKc`NrH7o~Nuh1}tL|l>LO`0^-WM=2s?D$s> z`e4wP!@f{ze{?K#f+$YB8uNSzTfQdeKj22Gii$z6%|Uk5>8>Rt?%%w%W2eqc-fLH{ zYtl39xwd}&`VAjB;>NAp=(znX=GWfu*RSu}6UUE#_np2U4jP1WH**<`WBy#fezjb= z3dD4v03~b>TuZ#0jHrmRm|sC&=$o8)=WjiF^myy-m)f_(Vx#o@>b^G>h1J(kK?(F$XtJbc6@BI(T*R5Np zc5R3zpiLpN-~pM%0EoU=i&;Xevk|bqXRjePws2e_A+Jt1ZqnpMOP3|5rQu&rw;nli zL=zN5xKN*W`p%v+FCFuPTC#902HBhEPM0oQj-&x@y(PY6nWURH#ZoPzGE|fr(+fMd z?-Sb1sc=Zhr_)&Nxj)RCTdGdOA|)!K=)99~S)H+D=8m*;4gC@-B2_r}P`CF=1slY5pibGhY971^5&ldy%N=)Lap><~77g(H1^Oqtp8`?;L z&HR-_e*n;pt5-t`7YgvVf|so7Ng^2JOj|a4%I=@m5+#I&0ui-!pAUi~3oo8Jp+&i( zCw6YvyB&$QE*{yjIWsM@eAz04`*d48b(9$;y9k-2T(xPkbFBV>fSp0__7AXX((eD2KA%kV(i4Ui@P!Pgx`4qPD|v@#~*$$@vG1M3>tr=V~lg? z=JmiA`U|naiXH>=fL3dEIc+H^eR}<^Rgaz#g$vn?+LBcoJm03%ovWAL=-B4f_N~nB z-1K{?+`MxB{P^gIAfv0-^Nmg)*kg4&$dHbSGc_eU&=O=qLE$0gw%HmIb05!3B{HLU z5noBSF6Y**16ww=ZQns_u?UNji&-SNNW{usJ523mixz%4u>bLmYuT1F?S4AGf2^5U zAwX2r=m_n=tTI9S<2RoLx!xe z3@-cXCG#SS7pqaHE^;$|`@9M(fpF6EQlSxFeD=+-Aqi(rhlB+?oH;O~z#xA^c=+v0 zSI9L+RHuW+qvGaT-;pmVfU{HnI*={VNb<*u)#b1)m@)11(c=&*G0%ygwO0St*WZ?? zT)X|tFV`zkZ2R`@HJ)zLy<^9}z1ip0o;}8n8s5HH^VLUAgha-IXL#jW&Bom9ESA4| zgW-OTjYV#7a3~wBg9|{J)5f-8nrB~nG3jpND_y%RTfNplG}3|p1L;t4C1a&89(A6@ zOO`|wDvku#vvrF?MM^x?qK&9vIb6u*HwzVsXEMVGBr?3rWN&er9`g0@Ap-`Mtyb%$u05zIC@;VQf0B%KT1#~F)~(x7 z;?JKy|D#^7RXcPfBr*o)8(;)46EPbK#M45LJu~C)Z}cuwqU49{;s!41H4UF`RKG!8 zcBnja`edh;&!0PUwoq&V?t5>*N8P%0qwveuZmnIrwo>&c1cEhMG9YH zCl*FLK)^T!bW|yzKy0@zork{Dw^-22N zI0p^;Qq()*ji+oC6h*yBsH@Xoxp1*d_a3i*IPkgWo5RLP7yJJE?^*DV@BiN0Z@f-= z!b%OEmcoMqCrp~8H~E#VP_Ad27HlaO6cZ2W{MhK(<<8B?X30R6bF#7xQPHSkWCuHj z3;BE|t%ORlY(Wu+6;v`AT{`*X-J$v>62QuTe?qczUFPuJw zwThHU=gwU~GaVfjmj?$PCuyvar`TP6{KRp3js30HUv=6tfeSEAw4J1~AoN2;#}T1Z z?TOmKh2sw#IKUYsFC|fF3KlGQShd=fiH03$5Si`TCukH%P$kl=M{MaSb&f9Q|c;xg%m?8#HLJYx@=yUC)m%{A5oF$`AL|pM`Nm3vGUo|=YZ7$g$fZoiiapteP|fL zsG7TX6G?qenJj1`OP08F?kshH&0al!p-AacxB*gOR7PU5)_QDCPfWoga~3YywR-KS z5hF$SBur89DIQ6~#oDXanXHwoRS$}nAp!jvlUGP`P>|6j3n1JdX_V?y=~v&8-zbpp zfSWu^zyU%Nb9Va78C`mIH(1QFZKK;&u2SW&Qe{SsnUHorb?LGt_IqiSYt*h#xeA$~ zHg4JIaM`zP-OeN~Sh&#T%a^xq`(92F+><9xW9jPB zy&H{tzIE#jYuD}FyZ25)!mLG$Yd3F+?0Rtjp|ht>0|6wWnlon};q2G1UYj|0Uh5ZM zf=>X};0Fdq0V?#o8bgP!-P`tfrEm9cXHTC7liB})7)7K-gh$l$7cN~qa`eQfL%-mL z3Y9CRBqa|2VpvX=4S5~#Ay6OS2EM?5BX#;n%V0&5|Hi*iQKYzGGH+S_9%eQac$~6X@P9Pci;xAwB+@Zs>Pd6o#;FarFZrp(E_8V8P?AW>u zKk?&7kD|`*(z6H1+oajEn)Iycvu3BHrp}%{TN@Qyv{(r`0=8;0nb=Wb_RJY~Zr!+Z z`}Uk!Gum|Qg#2$)!A(#Ge?j7AF))-YU-6TP6BkXGv}DN=C@zKpDk{6A$=JA^c<0#h z6QjNxn||X4sn6-YjI2z}z2qWAiohHdC#5Ha<@k>LNr8N)D#0}yNez9)`SX{rA3xQ8 z+Dt(KBwXYdmuuwM$$xw8mCcnaF(=+1F}irA8Z5M58}#v)gFpJ@WTiEBtY)>E zfb6mbvzi{*U!`{aR|kCf=1c9lzEr)sAK@&h7tN#7?lfT*!=u?~$o1%QwT9SI6-LgO z{a&X|V~Z58QMo*V6{avjNsYc|+jbhGsdD3{L#IzKS)nr92craF^=>tw)eP4~L>}F< zw{HD^&08Hjdg`5y z9RbmpI#tGe{RRHsqrd$!CN6Hw=uur>c@^7N%bL|7M9ZG9v~1VGCZ2T?xF_x<0Efhv zYf+>Q`0SH2*Dv+#+WE(|>zyW3uF=HKzli_Ne4$|j$ntFWS6}SZiMX0!y?YJ#c0}9uFMm66@&~5eOrs-lJ#dGW5!5i^tBI)4D~|-o1NIo;+ESap7Q&W?XL^{l({_zxXDuYK>m+ z_ix*=lf`77Flx-8&pwBBO8h3vCy(=&om(7t@2*?4Xx-9VLg8#s2{*KfTuddO$y0ti;TcfNo_Ikl&15Tcqsnc5>{?N-60x@%zJ5} zp^-Y1*=YiKy*co-j*M()u1y~k&Wf5EXWq{=`1`Tcpc<$o(o&P~J2sdCM7+mv$tzj| zG(qkx>L7WdfJbtqia07MR^SY*_hE9V=S}R|g@PR5)R{7jvg6mNp`V0=hxL4`Kgt2q=?H3x>5=in2$SZ~-+EHz(wLS_o__1Cx0<(oi4;`) zf)smW^K#Eg6Cg zeHoUbDF#r;4;K^_%dD3*KR}&URX6zBtor&15psYI2=Ui_vP(nhn;V zTvQtZTj)+j@4(_$#5r~xqg;V%oV&$9$SY7Io6ZV^YI98fV58Dkle17(1K^ZMXfPP9 z7W3n?FT5~+-t1+|mc96LS2kXx464e+N69L3Abw^R|Dd=a9tE5bDXujc(IXRi$!@(LYjz}B%%N_bS;(?3k zPaLA)7B-`^C}X8KHGp+?n=7n9p=X*j11`ns4YRC59h{fN?j|~7sH!D42R_IikiikL zV5@_0O@d<}vphlPG7)$b2$SP}$Ol_62L$H;PH1Q_19*&K1>$ny0f02+KnE8MCFEU| zqGNYSQN&X{o!&w(;OYsd5hq1xY4Ed2usA%~C~&ONCf&c^d+1QBpFg5Hd$vj55m=_t zjHQ7h@S+xpR(KgG>d~0}!s6m#6gKji7%Vat(6G}~tu8D&4hk@qgohZcra(XGL2)S5 zT0_DaK9c-}2l8l~nP?-P*9L=friT&A=n}SJwr~d276?WTklr`YM(EGN#UakCY>&4j zG&1J-*6j$rl(s`wcpxDKj~caQaQKctBL(suaD!9=)(FA?5VKB?G=^Zq++r$ycYRN5MEuN*CZq)RH;(s?Adb;Dj~9D-T?;j zNqjAc^kK>i)FP^pR2oSP=L89Lk;;p-k0Fyhg(5ryOrSRK0fO@sam*kcxL}Lu1?onj28j}RMb!hI=Gg646I&94%O;#RF^$rf&^*3asjkIJVUne`4pEFOk{*+ug9u30;+?Jf z8nKjHy-nZK(|6^kuBP`YxiQ?wm`V3!Fjqu zb81PQsfq8%hZGR;EngjM74BVz5)_GKtb}7h1tu8tj{OcLfCFKfnaq49(KRV1AQDi3 zLj|){+`O2jK&C)Fvl?wh^6l%(CytRYHj!0P4!DrnKw~vU#Uop#1XL4Trnc#cS>Qx5LgA$$#d z0L}!11T$d?5C&38^>q69|02OudRrxaxp!~{?=Yb2;*{5zK(klksNk3;&K^yPaZ+wQRG7OFi5OCiiCh7uA9lo?X-If1ZIj}D*R5Fg?Ps5=P$MavU#X`s z1^@>gk;Y|m+ReZ$ZZgyd=RB8_s2PG@AdAZCC)#$P3)EE_WC(@R306SAM+ zE~5<5zC&i0M#qaq`MBBEIy&zvwJA{cdI6gJBp>sH^rb|tP@@x42@pFX(v_O(m1 zW=u~^xKX7-*>$T|%$qd>>8e11ILwzfE?-=?>W9|t+I&BG{ML1={LR(^DggxuzGdy| z1>a9U{LAk6LUAEMflix!%D6FMLH>*8&dtrqEn2J?bwYLmCmr^jRg31YSUC6K?j8On zL*YV&5)*GspD_CDu_GRXnUz7|5+#+isUhhgCbNS(cg&wYb;qU+c)yh_Re};u9y_*u z)26Bw%7cdKDfebhnNYQSxmC-SZ(P4NH7OzKc0$E+6^`!Te{BCjTUy$j$rBIl`6WKC zaAZhGTGH+Br%Yn!0h0yIf^+utDFJ?#%coAPoImr@=`&8dComv5JUkKsSA=1wbH&2h zq*2_q=BHB!4pgjCBPZ?tyzi$hojd#D*;6ES2@MI_xPJY~6GtkPDPz!g=X(XYCNYO~X+OV+jsOVT! zD^|U0-t5`Ww0$WmHV(TSV@be`eumY0aaLeVfK(yB77ekShhYdfionakt3aLH>AlJe zn}U$4NnAn~a4Fx^1$o#GI`f@-;gJt;SgEw+EcYPwi*`*e`rTjN2v;Z752Al6uA*5Ti_Uy(1bn!i2&_AS^UZ@bvyd>>NR}WXMz6K8&@vB(f*}% zKdmBk-mB9<%Pf)?@}{7VsA6VKo8G6>%eY~ntseHy+g}bDijaH$^r}Yu9YqS>Fu(T!eaO&gY+e^yQEtC?p8e{BB@>9PmE=aNv|Nqbrv#NhX%$ zq`UZ}@7S>;BjY~Or?e7AhOWl#Ieh%&kE_=W7&M?_gZh;!RYKTD;>NN{A{asK2VD(O z)HD@yU!E4iiz|RBBJ;1G6@05pAKWVV=!Hn{E9b#hG(`$wA3_iuLPYQ19RKju`w%$^ zzImAhpM_PZdxSqcIDY$Cu6$4dewWX`xy5^>oKbbAX}_=ggA#r3`H}*nkjz&H(~$r+ zqR~-Ubbj@(=01wTk6W9{8nXh4Q4D=+LPv3I8y8*?~5m=J6ccx97yBjkDIREmyugxtrhb_ujbg z#%|iUp?B}Uub4BxXYaQ|qhk>;f_A%P-#^GPKhxJ;D7-)vFG9>+QGSeur&! zP$HuWx3T#o`d0wXt?k&U!?qn;i3RV|=RKg8nmE($E#9&@Fe0`?&)1)+T>iwd13{q@ z{)a9C>73tfQ#t0COs-9^h|||6q>(s70$>rMi0}t za(>!$fMEJJ-hH=Lty=TuFMOqY56_)DmroxDcJQ>bW!bJ>x{`K3NnGu5Dm?1|Fzew`m75EiP{4DU}Ywn5xxIojCsB>eU)g=4WFjVCrN0OoH^W3W&(d zpAt*G=%%#Ef|wlMSNR05lUMP_ej_`i=>_L)vMjOeub2qY7!ya$&7HZ zg_T$LtWe78oXm3!&t*Q6qlQvfJdP9vTYVO;& z?{;DWs4=i_-^xWwR4Z9Vla+n&@F4&Rsj^nRdKk?B76A?M=WyM)cu^BzDJzKvO-`5l z$tN4$C8HkuxQOEfaXKdKX#gB}fxj;)2|ie-;S8jWPg3yTQHco`BF5mU0*!GrtdVyugxBjPR&h2l2CJ_Cjv93jQG z0ktv<6-lFkJPcO@KA&vZkbTw(yS{w!RQ-na@UFqT1&-LbQKR$c&l9O%vU=5nd-v?x zxxG#6=NmM7YWL2adw1_@_G~je>iWF(?y3a~8$I1*-B0Vh;g38HaT+faa8n2(VCSAN zVZz=E7p5;-H29r&cJJI}&vxM6hp#1-#L^)Bmxq9Ywhs@b@8%m|%_VY`0$DG7i+9lP z57BgZ(ck;iLns6&A70E!?`IzU;P93#-}1zK^^u}fX;KWgH|=Bk>y1$_9yos%}kKXe7=Yi1IA*0VDLERzI^QgAkdFpx{tA0s0ng zSTL~6=7*pUb)YEN3Af-S78Dc$fG6EaC{nfp+Z`k)-Ue<1!$K^6es>dZVK&4EfmX13 zkfatsHz{G|USvoey7Kh&OuizU6DXf}?p&oRA6v;0=w`#ld!fN$Y)t03m*~ODNwwfB zc&CBp@C^J>B(t>_Vl;svq2iaJ(*y_lA2@nScmyd1HSSwiFGUrM$2+I-Gfg&b-NIf9 zecpQO?%liSRU9^F=Z-Jq#rW(CE$Y-eK6lov&;H&!)Zf2Wqo=h-@&w?}fmX(2lFdBD zcbmuQEq=A?)G1uK^7gHpC{TRi=_Y*maJW_~CUBUJgEaI5iw!36uSyhPk?$a(D8}g~ znc+y&e3aAjo4_FN+$QH0)bd&V^eY^D@t5b)6I?|QiomWaO};CwE>0+3hT)a+A=QI| z#XE(|^B%>YQ?TQ_>R-wg{d; zhF2sPxtvHswqQr5)#}EN9haVVf5W=Aj9QvZ7 zq6$?gH*x&9OBc@Fx|uL}(u5L~D#sNrRP%`^9QW>R+q{X5#|X9*p9}FW(?&){vhgFq zoet1gtG{vb#E&ah-M^nY_Pfz+{qsbvT2V34#@N`UOBb_`@XYV0(<ScEXvnixw{+vUKzMbtiW2YVq82T->Np!^7LzPv&Ip z+O=!du5obd&&PJ{AjcC;SoPCd_K57&`)`KOkSpiT+q1KlEn7-@LeV6XNY4Q?)h=JU zZ2!SSSvK3|EnBajIMt?gOZM-?fd@Sj4@C*&g)9J#KhbcAVmyzj@S7J{y_hRsIY2-q z;d;OQ!>bn=ISDcXonG|JgY18}R6XV4_1+&Jew7g+jgZsIbouMezeVH+XQ{H61E}jC z+^DXkn7rd*;qqN^6*(jKd7qYdneQ!MQ6S#|H|}`s_i7XU1KYkhTx>~hH09{6*;@Z> zt+i#3*Y2jLwy0KagFsE>_$8~L<#WQH^Flig^|Ks$@% zuF>8}y56`_$$@>|ZvD5vm#S3Vq4$5~?SZE*UT;&k>htxg?f&^^t;V(l8Q_WLl$9OBSnFeg%vm~RxY04C0M+-CWD z%A91ot5cH(ZR%FPlb$XHl?{9sYRjn2m69vNL7_By)`Pr83P8>Q= zuUzrO+t-NrDPFGZv@xTesaR^sgt3FZ9h+;8u<1kJ8#s8y;-!ttl{s_eswOZ(t6_(G zzv^|L35tn(rd)|_tCtdu1J6*~IG7z;f49zGR0KLr_2!*k8T9$6{{5aVU9|rjZ@m8L z(7Mf@&C=-#R;;XvDOCH}7aZo`z(VB$%GWAVznMOwFka4crcY{Bt!jhn)yma=qGh)( zhmW5c*|*=#>o@Q?G_gGq@a(c*yKw5AwoMxqiyA&|(%ZwnDp$Rh3#V*gokvPMI+83R zIq3+L%Q;`^BlIW+ys$A|u*wh>HPSFNgH!Xn=uQyM*}MYKhg+k|R#s4%?W8RbKT1(* zbDxo7JQ#RMZa~N7#tcllNP%0#{*?e zr4F235S;mmU+6*p@a`9k=WYr1VSU32#(OY0NDt1vJVI7Gyi$%lN>&E4)ewbnQn(d= zFk|5~5+X#1!UMn;^%up9@(B-mDI^c*;c^6yK`-HxVz?$K2q#RX$>%3(=5?jXV_@RZ zNzv3Bp#hbgs)mwwxK3Eqg9qkS-uL#$P=Jol8(NRxN024?f@iGJ-vPi`RTtLiAHxuHs(;=CtCCP^@ox6mlkeX35A-wn z1vqsUfKnVYv|0(0Ht2064iyaoE*>I<>)rOO)Fi(6hlDw_D8WS@*Eno!sgA^osZb*Q z2*e@d3JMmtBP;!Wa9HSwFTc8S=G4sZXWULr@e7O8!kJ`x3kq^g_Wg{2&>%EIHnt*w z8nW4fCiPxYXn3R`COi)g#v@TQu zlq{$0>?IPTKtWDs1~t@M2zDXn+?bM*78o2PWL80*dG&GHvs2S7q*db4>^6s@#-Pom zpIEe@t)eDyAyoq&CTN73p0A>=mr!zOB`8EQu+hpGmmZW05!=aIz?IZP5S)dU5LryK z#B*X&V1*;37aft77;`QIF6m10z|aKmDvgUF9E_K~kT@JjAh^s>K^X<^QXLa6=PXp{ zXIY`hB^+Esr(o*9e;@i#2%YAgfmBV^0oEd$D@bX|tKHjhltf~@#r583cCj_){rZSdWFK?6@a%vtSWMJy6bk$@l$uI6ptoGNc2dL{-ADzLL|F188@!?Ygjn! zAD^gtMEKidd*nMEwDM6j0`?LNZbv3#)AMx<>A8Q7tleQz391DYcktKd7@ASg!bsI;buE2l0Xxb zL_CNz)*e@2WQ;^|$|NS&A$tNSA&}}Yw}A;nQ9&PbL~K0D4x9!N?xX~2Mfj3r&O&69 zoK0x#*}@T}IS(`hC_4d9*N_9mMG7Y+03trar39Dh33zHiM1Wmv0l=Ks^y2O+osnchH(CaDkAM>M9wWJdwM1#L)^BG4~Cau=OVMPRSQE8#sv&M0VR zf!Z(wEV2eNt5CQEA}Qb__8nCd)CERq0f|<0CI|(LiCeu8mOkYRFwDF?z$-LnY#`MO0P1vhhuds|%W#8q6RkF3a&X9&m5fO^ z1Hq`cYAOMr1_{o}I6+{b09mOCFY*3C5B@MD@DQmX#f#)1FrJGemFhK|r#LIOLLtR$ zs3_g6YABKjMG0$qa7abDmOsw>jz5+Hk26#HA1TSW1VpgcF**g&h{KkFE<=xLOg@lo zhg-|$?96yPB?M(;bP5VHjj?3P^b_HRxxw4btJl69_8G&U5+wv%M=Dox*CDYm%{-KY zlnrVF9S#LysK}Wmz!!qhjH#0k?%O4>N=?+NGsmT2%N$wfGm!*4nf4@RA~&f>OlmI4 z!zqJ_D~b8x5C#*XzWRylQ4%26y0>rNxnRK}DnQmGnkKWHP5yx{iZGC) zRazrusp-0T^TtK<<^sJyCl%&$2kUZ^fdvy-$*P12N=ib-9+X~!d9Fwdi7=T?pFVy6 zzJo$uZWhsxMj@m_Bqa!|Wr93Q*%46?IfRf#Qge9azG;)jAKbeeR5^G0)Xwd@xEyI3 zsfy55>O(OyQc1uB-U?&^r~*l>mvB#vgQu5G`KFmo^;Xxt-OCL&hGk{uEN}fki6vinvVDnndV`wqsPio8a8EwQ3 zOU>aK3<{~aEF(PbjI>nR3N0i&SbX{UC6BxhtL^*n-=csTw*Re%|2Lk?5(9CLMKpVB zF$@hBquoW;Kko#Q0c~Q3b5f&{lL`D{(lTU4zv-Q6f&~JjG8sTs=iP*a6_X~pxmsjE znFbzLF3JR%O~5gz!8tHYivyzM4*^U$!w+1xVfE6BCk`^N)HO;bGzwxO>xifhbn=-y zCH<_%tXw!)dMnJ4d4=!~j(`HLT(?5M+48EJ&o@q zZaMpR!wV%VA5W*^)DLzGMFm#{t>9x|tAHonN*XCAm6zSz+Do-4&mQhLh+k=-OqM3{DK&zw3r^3%cW2FGtm$LuHuFJj9HSSVN0OzNPj zD^CHEF~cHk3o&GDpnQ`y@`oXsSMEhl$9IJ;A)6qW%y8v#j5u`OysDjac#3_)_uqQ} zZq{Vb1AZ#YMRIP@Ww2Tqw6G(ou3C-I!y-E~a$)Q&DVPFwj|IAW$dbu!KeAR}?1{G( z;evEPx9bt`T9NIsY)1i2xMM$ z1LGE71(xv45!sDbl)}U|y4X*Q%pnY7#y#mr80erIk`5qevpaMaGgA_AiQTzWFu#j^ zs+gl9!;|P$sTxEdBi1p|Q=pzkDTyDUq_`H@8Z`4*MH+ON(O65`C0R-%E}`!Q$Bjr- zth~9Dk?DpdxHzMPGXZ5B3h^rASC;oVqQ`Xmu}HT&CG$1w18QtX>Fh$WZ4~5S-pWQX z7L(P+M)W{8Kqj*qwjvuL8d;CiOhhf6f$X$;lCGlcHt2I@J1|OQZW{p@kXp1CE-+pE z)xaWgibPDaW94OOfGCJ=l;sGn)*_XYG8_*Un&ps$Fv{Z*?KpMQdVX5EV8OBcA<0nY!~4jkceeEMD0tk0h@Y zoWg`NCtHb9lzY|UB>{SU`?jrFw3r~3%V9EDu=0Ub)KlYNqcZLilz--_28~Y~&y^*Y z2ER+OXQHIZrCVj8fV2;HkzS`e6~U#KfmG0+XeN}(4jUPyoOZXrNhEr7O|oDEx5YH7 z7>%H-Xiiu?Fj}nafXs+=h_oRZ4F-cizqCw7jLTy&aUZZy zDy904KaK+VE?0@jJP3BEO_UAs1G z#*8N$Kk0Vnu3os{hxxM({<59?)UgGNqGZ{(ZS(!)!~zAPr%oDwXwUvaafSVfjnwHZ z9{2XOs~63jzHj%=g7F1|gM*-itPMnTbspHU_5QtN_NbgbapJ+dp)EAQS(xRa7vtW;^g0Ds^H z??AULd-C|v<%<_TeCY7Pftod{?byC;>6{t6w(STF$G1Kb5-G+cdXipUI)Cx`u!2S7FJHXy!-9FC7W4NL zC!aWY5VmptoTo+CGR>USdw1_v&iJOAg68)IT4 zLc@dcNfD_TeF?ZHJexECwtYLdojQ8x=B4xVzn`9R`$m1y`LbnEr!J$iTpmLWM>C$V}V4b@Spmv#*>#Umz+v z#Lw@irHjvTDjbI&wc^V6!Rh=?Jd4jw;r@VV0`5ah(Jqz;*s zTFu{k|IN>8UjNgNuXO2ve9AI?)VJTPUbeJkeB8wEMvwaTYn|P;a`A$9UVEirWVl7A zyLA2nrZ)u1F{4Ir-MYokVt&7Sm!H?K6VZvuKGMvpN<{Z{j`F>8bt7;#}4)H|6X`h__j^! zUwXcUlQ?yad&ZFZ?^An?edlI$cSCLevy?p$mF!ACnxpj z+Ud%L^W{pF$heo%>e;3z59}!(7d>kDSMz7h7Lm~7u#oQyBdrMiA_>wLyrNRz6}fcJ z?qAleS&c;h>A(*rj~+!ii8pShrKPdV#|SB|8n`#e4vmxr5^BiXH4p#y!~_bW1gn`e zg{T5t#}4lA_FNOT(Xr|c?BaIz)G@Qxy=>mBzxV37a^*@y-5(Y&e7{%E*ytFR7FVuZ zp-j3IBLTuC5kvN*r0UhHvJ|_0>*n`geKqKV0sa>2f_d}))~(xnZ@nED6fkS{%+H2= z!b0cjl?&5{e=9<+E!@3xcO~x=?ww@fOsvVowryKGHYc3ePIl~xZQIra6Wg|J^V#3u zSLD8)mzNiy*_l=-&&kz1%cOe zqT!%NgXuRYWEF%Wso2ZN+`c|ov0j$k2&KkqbRT6w{9XEjhq}!w7jyR5O*tII!nR;r zW?=zUpPD`9+6U>=u_{!&Hrf}|Xp^=&ykJB$63l;I@YiCK_oY_}Hq^*8aM-M}mr|D_ z-xKcc14m$((--Ow?->79DHRMhDDp2HykLcgyCDooAika78AfvRg76I`3R$lg+r%k~ zIZ-i@XX|-g>{NU@9Q5cmE=i{g67p%`@NcV0v|er(-sM`fJ+SgcoT8j6pTW)eFy7aw zzagzG<_90FJe|F8j-0$?R#*4lAVfrB2-bZ(=cZT}t*H#^SW3CK13%FRqkpe;G9#Q+ zCMF)Nxk3d=cC8kTC8(9N+KnYq^S_Mc{gd#WYxbec*wAWHPAY&wR;ykEGi|frQ^K>* zHObI!e=~Hq2Y}4O9_7kK1(L|2A;Cndd2`!>Bu{_5#gn8r;; zY8U68-_G0J&#zR#aa)Gl{k_AgcWPnK({?_-f56@X+PGa7UoLl+DVOag8sr-01q)W2 zUkF}~^#;RB)O8oF2m%UP>Jz?2pp=99TW)|PP;$P{_8?2+PT|{Y#t!+Y8hivblP%z@ zEHXv4dW)!1>OD12RM((j-SGRc|DWLA=_aoyTf4_#za?tr?deH9#I;VkRcj?Ni|BLz zaA;{J-4Zz1Pv1k`gaP-9$BVAB#R~zCK|!W%SLM-7y$qfv%Vz7#dz6ODw-fXmn(*%| zm0I2PIxz(3Lx0L{&k1EgdXx z5R;T7*$lzjuA1lsSM$A3+K)udz1s=)0qatU#SJOBDSx0;NM=+wTRUy)@Lh?n1Gjy-pOc@iop!q5aW3o$QRHKGSaW6M zr!&}%#5}Mh*_eCd5iP4fFx(lCO%blF(rTcfMv+OTt|BfVe#Q2v##5jiX=Q5yA%9<( zWFpimvQzC7Er+ssMHF7d(qSM8nGYa@_53ks}8P zC0Bag%2lc(w>RRT&3k|NlAjEj%k5ceuXw6Y{A5wP=}aaj%)DHaE|a4q94h#HFtD9q zNYbr}=Ru6HW&xX4BTqjt#E{{W%9KV`PNs9>$6wDE<&;UrB|5#+3%9qvjLRajUulNn zuBXd!^s?AUpZnoQ+T*X+*}SHRnonl< zZI7otUaU2M(E02>)xi|x9&e!Pm7ng`XVGMCcz3to>4lge1aT%qX90`tCYfBwfB?Eu zrHBM`FE|#Ys^J-@)_9av!lH{HwFBTv-)HmOXZLlXR}|*giHXzw@otfZ^*YOQLtaC% z0uK8-0^K`N@xWjr2HOIIP9LM z5HwD-^BzuG*|C_zLo2ATNWmA7#O1cGs36*YH)qkn%8EXfS*}}UY@HXtXSAP)lYx}A z?tVXBj85tGcuHSAVPb%q#UG!+ySK@IjSUsV&$~ByqEd_Q*A1pQ-c`ukk7#CA%sV%0 zQLsV=7i+gA-;YA;0{t5iI%95T_u!9KTbc@`0*Y15qDsDZTAy0Eji9gRSUBrh44$6T zducCnG;J-^^j;9+oe0~wN(4NuiUQoS&dR^2t~FeNCF8vlC&m`f&pXbo^)43#oBh1e zZd6gKV;X&1&DC5?y$Cnfct$j;5+i7ilxDG}^b60YO~>8-Vp2&?0zr1$)$b6DaUcIg z0*U!a>~u$11g+OOjuR9Q<g{;&r-V!MVPgSjdCxTf2+ z5R;_%F58+bsnY3uLZU`vC*b*e?uHc?IcLprH@_DTLnHrwLkI3W&}hb!A(NlJSOK17 zAKecfG}#NTZkMI{JH`02S&=BQMCg2{`i*@r%$!9WV!2+nu+J8?Ug7ZMUPo7Y~bd%2m!tyB7OPZLxdur zLg{rV-}%%v%Fl5~ML9V(32{tN&0+q;Nj8}QTX+~qvC^&7@%RmtB5;qUj7VV^T5Xn| zJDR-sFx^;X$0;k@ipCTFNb5oi`o@-@(Q_aw#s-Hed@E{9V#97V+}MawO{?Dd9IXS_ za5w>zyDZ{xl-+g2Xkfe1Y||y7n8oXt9S%efBr{OU=d4g%Co1%wMgR;h#hz3wOlAsv z94^of#Slt5xWG-k3Uu?+H6x zoWCT=rkEBL|H0=~K*vyS@C4;20B5tLLJ{a>9Y zR3oNWY+60v({qo8!RRbD7x?U3L|o3lzURyS|3a4sGT-+o6|;oJQGQ}Me9W5axc(Un zS%gY)?Uk5O0wQ?#ttp?4h|4XUaaylNP|DTlcgdj|9ZhCK@AQ$9DrO5rNB)VN7CvyL zKLf871;*~PzSf2fpQSLEnwcc1fhcE(f>nBM7lSSh3-#t(fBOqfsJ^$}(ut{R4qa|| zfsmGqH${J%aM_3o^4t%1$^%QUTrugfa*|?W6QZZ0vWeu3P|?Z?4B-?7+JT=~aH=pf zn$~brggbS3?__im0F(}{Ut!4mjhIc}VHKQ) zeI8BriZv87H~g-1bNsf0$24NH_z?Xw_@3~nbrtZqoiA<=M$m`$UPfxAn# z2}!N=ANNkf1ftAE9Ydk1hGAhC3hyREQUpgPSywgE*IKP+@=kGaNTz2I+(Tk%dg4nV|F0Kmy79P)8~Etl+79g1And6Iykyw z$sYnD4|Y39pl)+1L3+K28J9UJ-y?^^au-3ZLd|P8T!#I(^T{Hn{zAUm+ON<*9CTVC z^V+~B*t6l0{86U(j-G1PMcZj6I+Awb0U4{MYMZ%wwGm@+FM+My5l7R*@l3OZ1ET$W zOOk!mzhQHmpIK|X?ADrPQZqVkm)=Ziu| z8Nd*U1dY}yJl=neh{&r2wi_g$R0$|>sp&@cd}#6@+dUzd z?|M*35^e1RO#EM2s~2BHg{yw6AIku$J`u!;Y7|%Z&f4$E z_lz?-J6c~_gyB+SB1(#4n2bi#-EZ_%(MdpwTU2PKc0;j-*jyr2LZP}fLt#1A253n; zS}Hmb^gci>EjuZ1D7ELq=)>eh_J{AnX$1RkROT3$xLU#F<8#|=Qb09%2DC^4qMKRE zT&cspiAgGbt5xI>D>`N#Jtbx2b~9LGnhER<)=H{o!oecT$#mD~Ug{i`qu9uqt63o% zosQX}qLpkCjZMP=bZD??oS{rb3k`FQ{SPgJ$Y1Oys*cL>m(al>7H}(O{&f2Tc_^y( z5rb1_Mo*9ZKJ4Wo`OlsbkUNty<`c4MUGW@CelTbblQ zgUu6@<+_+C8T=p3kJ)%-lNdBK$w zc+L^zlHq4qdz@6a3-K{(lL@O?1i1vs<8-+$3l|w@0!C4l;T_u3xWd>|9H31{6&9|@ zjIsb*1FR;=*yEKdh!s=n0xPw~G2*aLXVhb{IlWIh;{Yzr7LdqCu1Eigq>BVVm1%j; zZnNc62lPSoTS&#Oi%SxtX6O5?cDg=f1|t+tWrWsF$&5yCU^+qQRXs!Q7lRxo6T*qM z!xOFVr0}iwvMy%khleK6p&{3NAiQhti9Vvs%5c{#ZE*x`Cf_&1UN6sX_)Z5|5~ZfJxI|FgM3q3dbj_o{Ujwc3smZqp(%2Ot%P16e|5X`FDGlr2!`b?vz<) z!=g=3EaB`W#vX8!dN@L2A}MG=wEJ!PTRmF}qXA#Lhh-MegYAY)HnWp0;3mFkG#Mrg z)h@M_g#Bcksx!^D-c0gff+9TsG8X@DG&qp#pN?cU2~nhZ`nP476n9=YS>blK-*BtW zXp+A)z_IJ~i*3o2;-ve%i+}8LO=w3FUK()IZ0oa8+Y?eyS~>YkV_@gTlCwdRq8a8& zj`YCA6~tXQuL6&MWRj`4IVK?P)wGKK9QK1vpQQg(k97PW5WhFDnSj zk&Xy*Nk(&J!v>DKbcp8Xm&RKJ0>A-EwvsjUC+%VWl=9VM7IkLmY8ZH86^_JB$I0v~ zhIPTa@3~-Xp!amJ5Q(F}XtjmPLX6N%`lKg^x&}NnHGU zTPb0_AZJ8Jb7=#4s;qnI$ZEqthz`xIi5N1Jk?5>Y;*i*p=Al>&`>3SpB# zVO6S6u*RI|(SI=+=7x4T z)X;HU6DGywd8($zKlMI6N-Bdqc`d1cSXm*(!HTi0WM4r#7G1z#ve)N&6GZSX1O!AX z*zkIQ4fFa`&gO{qi*4kI;S?IJ~78Wk@tnxp+Xo<=^oC~szri)BFLarJ2U`w-N86X%pfCx zNZ7;`E^88<#x&oavz0zHY&-y>UhEJ?%!mRL*q2$vw%LP%1v4*=!9s_Zy1W}g#3Sv| z3l+@g$CnMiEjJjm8Kqz*UD_89Ayi`h1A{D(7^uV+yVk{gM#LSAZh7LoZ6m_NkL<P zyX~55o*q5-b|OaQ>wNMtEwNY#sG@Jbv7BWB)b-QDUS-qh56muisf}wXK1i2_m7}+% zrD|pbHVUby(1s+TG5O)_Br22iV-Lf8lrM&%$&Y1diuSdfZ!r2^MN?}vu(l!fr8gZ> zcsiT)U}w%AID`D|sM-Q#VAU3K(E=N2LV08jSb6HAu4x)WzLMzHtxEmqPiF= ziHWy5_J^TbfAngHin@P?!BY^B!tDwzciJ@HS|gP9AA)QZ|1_Fs#T8pMVi|ExWWCj! zmKRRQbjZas`#5%evtr*9{GwvL_^gfl)qJtO(sDNTvq3r`;qAq3FWd5o0)BxFhUKq^|x5(`B&HV2RIK zCYJO&0sAvH)49fJ%$(D9jQ}N@X^c#%6^`|GQ%K z%qH#$yG?;#d4o6|&rZWLE+Ap}s9PPr|7b)CsNY_O3vQl&4XT{jaRH>l9RZ5V8Q)Rt zr3L=o2-zUD=@CGtj=JPmA4b;Vw`MB++RG4jMM8rvMrzaKk4&aUD`UinDg%Og`5eoP zGk<236Bq=;ZmzmB-fcqSn^cYpK1byL;S&|`deb;1gJJ1VU3*jODxX@#8!tP}jU)$W z77WLZe9H7xTQF0W;`BK1QS(_3-3PnuqCaG90+5gWYnx>hTa?a>!#()j%9t6`*OhXU zl+o+Mjd7-&sYheZgCN0r+RC?Sh_e zDSvO6^|0doSdju|)7`?!n(pvl(CQxYhh6IUJS+Eu9QZt7U7*T^DBCwpE%v<+$}{;Y z3Q~TnuMl~D&%f0TkeC&KGUto-j72-4j6NF7;E;Vww7PJOqllz2{qyw|Mm}vqyqS8R z67~uWEB?p7K5tb9wa2RqW9|WCw^;++r@v53d_rR&{#(y=&Hud1`*9p=7cj`@71{+@ z{*f(ql>LeH6V2@Pi2e(Vl_qeVx?Rq~%rvvi>hvwV#NgO$is5MF)+9a|qsjC&B%^?u ze7ZQfl1eJAigvx`4D87yDvb~!gp^**PSvsw&YC&`3M}SwkrxE>DI#am+Xh?lJi_h!><*rIDTPC_jzI1*-v>Rs+akRZQMdIVTE7>U5vF#h-~ z0);Sk^Dpm@X;e$yciy=PCa}zwcd0V1{%;8W+bTF0St~}|j~3}oPL@k8?9*~XGVyS2 z(RbmB=MMJGf`fvH%_14J^SFW4oG0??EFxZx4Nmr|V2&fUsOX@Ay+>z_m_zh)RFM)6 zt_yVIe*XJoYr`z-_c-R|IjUMAx$`QWmK}&)S)$W;gA2W{-z_iIPf9(K{_k@B-F)_5 z!gpVr!9QNYDZ|6Vcal3mFm4?bbiM7tk-6q^>pnwAmf2YbT{=~=>}z!8^*wc>uYUK- z@Q6RqpuXX9m&4kMt1@)TNDOKb6dbsUnZf$dJi2{<(`<#Vj8FTFsfC3puSmZ;!(}q>c>@v55{`q7r}p{9(}f8lhGx z=5lg$vs?2$+1^>SSw&UiG#YmUQIhf`TcaZJfBx#g=Jgfyc?FR?!k5Sgd^?cvVN=XQlWU5KEGyFi(VbRN-YTg;Q zD#=Q~!SIV^d-m7Ji%g1diQ82#I;9wPM28orrE-zX9gj;+g-VHOg*v@X z%Rdh?>I41V5p?p=!~x8-TYK9)P1J+`ROBZNUn(8P*&ya}j4u4Stl%vNVgH$CVaRTA zfv;ffGOi*z1|SWJ+9$UAJ9l+(lykD&CM#u<=ZT1W3CdmhYRlM*k?nH%uXur4FIMV{ zo*1CXXW4rF9fBjOzdWo~yY_3t?6KQc*xG*$DXM$Fo&a-&c1Pv`6s%O4J$AGEW#>@n zKlW=_wTAhHA0&J@+qIr9gLn7p{EROPO-ja$*y3M4bIvY#_X>J)vLNJ02W#IGG()$> zB>W!R1(6=SzLVS9wS-}F914@$B7j^^gt`snT96}ifRuhr5ZK`8%pe*M@y~9tJknlI zm?kqDns?>Q;NT#K-DiMUMACq(^1brF#DeWL8SL?jsnTf$gY~UiI-@FR z6UQKzIUunVeGZW2=;a97M{zEgVY7Y3<3Yw<$pEtmZ^o(-(TB=>fS$E1_kK(VmHli5 zJ>?Fvgxs;=f(L-_e!1f>d%T-ZVuVniKJdfqC}&8d+{pc3A1prP?*>%Yt(Rt9{!e{rJ8cEpa8bHLOM zRSw|dVratR@B91vETkzZMc8IpvORgE$IYDbhmv);?9^6nQ4*O|j3e^ca9TBg98@eO z(~{eh*X(b{1r3(gir>K0kMZ-tQVJEpVQ>ifpK8$@&D5wp`rImQGfIHhNcD3_jMsN- zE`04S_;>@~2zf-HT46iW<%%;i@dWI_DrGL{;Tp;RK!;K6rOQ;pL^Vi;$Jdebmb^!i zF}%bjCFFdgkJtx<=m%X(8FT20PNz$q0!misw3ftDAc|%Ctfh0z4lzcWx&mNnu>Bo^ zd`z4@3XKWlH~ZZ{yVX(=FpQhvk$*uGgu=9?{fuDuv-Zc0Oa4~ds2SzTs}&5@|FYK^ zHc}W(lGON}7pi%`@el(0C(?H2y5cfpn~tt{qKN2q5^_LW{GQ6Tj&D-5(m9DRFL_Pf z3>8D*JLK=kg}5M!LJsu>z*9Ip)Rs-$1g5?hEh+g65=avP7}p9gG}&2Mf$sO%*1ZO; zqkbOSgE4{KFrH`*XT!XS&f{j)z3R8tZ}7(mEx=mkEa3AkgQ7^G#6J<5-`}(2wRFsr<*NA**0G(65(jH?1=+zrJPs)B ziejgFovwct!D-xHEv*WaKK%NsHs}3W10j79yB)OecBt5_100Wc{mBsKFJ$^1K(RiQ zP2qP4eahb2fVM}=+qAmkAzwv(f;HGYQ7F%9F|}D|T0}k&EDP`a_HX6M+;;YtQp?a{ zMbIW{n5vgQHWiBNmHuMhKRA%fs_~WfhnnFi09{yTTT36ubl!!T1&$E@+>iO#pB=#Uy7=vd5|`r#Gw1&L@h?ph z(debY+cY*w-g>8;#C(i6O^44~c-%2e;a~jusKjpbiM+MLi0AqeO4x%0 ze!W|m#GW~GL!gQjunC}>X_6nhQSZ2mc6!Q7zz;>Bvkl#pM{rFC&UnsjIt>Y#3h@mg z_)CHtGFOlfa`>x+)UWTXz^ZAUB2()PJm2s-wU$<=x8h z%3t8)%zmHrt^*&Yjsj5kyNA}D&-jU0J!Y1t{@6R%v9J~x4=LrMsXz3(NkIJHg_Fsf zMQm!G2yoj*OK&YiFoeFa>bvGY4D5AGM6UavUUTxnsEA0|km8W3E;l6OZn>Y?m*TME zlkn0L@D3eUu2*knzPe?=-;Mnc)}7It@jS>cp-GWxwI3Z^TJO!&Wm^9kGu}>8Zk-hr z_54Cuk42OvV%FPkx+MeIWCIOxAtCE#tQe16fxPfI!jSxPD%DJht@Kaz6(LpZI$AA1AfbhS7wTz;i_#TXUeCF6ej-qBEClNWv=*Tl3U z-)h{zw?ah`3ISPMNB%rP*0;kNy72NV;Oc)boL$)F`x0WBGEjY;Ys3Nq+xocE=;RQ= z9TozOy@+NtDHgtg<4w`F^}D^J{wJcf3L4Zrp*A}&2Q3BT3e}(_|7S4X&Oqf!ryId8 z7W=jyP*n4P0RF3MSIEyBZ}Te!*Ilvde(R_TUi=}y6J97hf!D7VDlmpQsB-MWoM&X$ z$R+wCEyTFC-VO?7LH=XStvAw=|Dvfs9OC;$1&so{V^mgCrP3FVxO&(s?ytMR?`YfS}<@_GI31_c-hc_S6^=alFyCj~>b?Uv@_b&M0Y{30R z1J(H<@-+Rc`>fcn&q9mvCH9ST?>Wxb+nW6HEBd!S*YlR|$I%^fQJ>%Eh}+>-AyDIY zECKkSnwGa2rUC=R=X1~27176*clOnzUls&UIH5X>&mC@Ua`gx=M*dx&Z$&-Jb-EJa z`>~fF8;1KdZA|gKpm~{$>$MEZSILr~IW0kqHuFLeZ>Qev*<0Y?M+6e`>aLiulun1+kWda%j$eWe5hBI_b+I*lCIygcY#!1)H zCL9E~2<%DQ!&}yh7b9mtjSrbn{g`P3ZwK$e`1n^Qx6@(Nwu{9xdR!xM8>y*^%jgSI zWC>WpZWtx1Ds|_WF{Q)YyoK)Z$>%?(9W`+W8R^6CVyQ^*R?faN=_CWxA8@(A=-izS z=BSf_a6W7$tw5x>K<@Zgc*5gpEc>z_!IzPeGFa}T&T~fseBsLDV4*QPNkky zf5QA z7^6CKmk^Y#B$!|oExC|#I_jtRz9R?6qHs}6#LyiH=o_d?^_4j4)p%G~@oTD1;~7j+ zzY>n&KrEOiv#fIc8RL%Y55b`QQVPvWl#8(b{I z7k|E>4b%l={d5c_YC~lrL8vHHF#BF)SfygAkP|LIG33Sdn@~(W4J-^sQ6vP6tpx23 z+jm@W;@VzJOxGir>wGbt2098TId}{VD3Mm;j5}Dd1O*EP-bAwW#9SE$3}xrf5aFin z$im8wKBu{tIRqq6k|BW?1EwoE_*Bp}i5|t@6CZiF8mfraV1WX}LJ0u&CLJ+LEE@A( zOvSW0YMKZHVhNs?VqTME<**4-^d^LAwDDAVcnp5{AMWurx-x#o<4S0@%M1rzd8O+Z zT=w6Fj>Pt1i<*u91RUo#vLGiw{I|S!9H>NjxG)lktrZLZ;Bd|KAVI7H=@~xwPD5yX zOloObvD4al8vtex);NblhXn^qgDM3S{2zz_dSYFrR(xd}Y`yv7eki1M=qJ`gScrDD zTCF0F=6W5?(%S)Ddl&0LYf!v<$mRWkQ)2?S7`@9|@WZeX%R%90L2WWx_-GjIfqo1p zXgR!hpx+=ngS&|9ZB`wp719uG=D1&-UIb*sVpcfHJ6vX_=lw67UhvD-t`12(61e&+ zN?1v7WN^kU@tp_yMp0oqr^QmUl|BBrv)rmA;1j>(VE@r@_l=^N*#DzJ-%Q&=YLG%( zr%RLuPk{L1h4LoQ(w<7!zH!`JGWn(4Yt8{+Rb7K%ct6r`F0#qXhEQswDh-6}u~lD` zc)3L5ELvE?If zlu<(xK*H!&JcK2(6^SBcykmnDcY+X}DyAmh#hK*$Klk~X665-HKL2ON<-^OjHg4v2 zMQ}GzPoE(yNvr3b;%?h~AOap0B2svN){m;p*x<_5u2gU+Sl@is&(u|BsT7=M}s zm8#PNMyu5-0H0E`MpEZ3qXcXgB|Y}Jd|cHt%ht8m)P}_t|GRgh;LcU*9mxe&Bh)4M zZ8*`i+Nkg@`1gOf#9T;t>~*0fg0RZQK>xHf7)FARUSt(|A^&u~y7j7Or_ou{?3xKL zmB>0r#ShJM?A_{r4<9&Ura)F)M*QKVi-i2lm)y!-wg6pI2h#Gm~F$~z7lr!`A<@#ZV>_<_z(Wa`)-7ftTB z)z$iCrvGl)kSdVhx95Or$;zujvD?pCI~{z6)&^`bePFhxjL0_3v&mKEG?ash^46IJ zmL77d@ID8?!Pp=585@lj9F2goQip?S`d7~{ z2WYm?>R$TjxTyRFz~Xka5vB|(S+iCl$}f)7RWREV``_s3Cn4#{J7a(DvgvkY>9LJZ zDOTj*gQ$f_!5gMy!!k>X$&%vLq6og1#&Iq*;;aF1iWt^jfHGsV1Ee z!Pfx_!ZO4yct-sUee)bLzq6;5Hk_r9%qo4dxXaP@=47I!aodsrNZ^E>o}rVvz!3Gy&R zZ~3}g?ZDV#>--tC++@70e6T}dShsrjjujJ}S37WZXYWOwJUi|Lt|Wxq>Z0xi$vgnh z=QU$Pn~F6rFjoyv^$|(F8USOPmWmO;azT}lSiX%~icf42di`OX7o8RZk>8WHDSkmK zQT3yxRKj`VY-pQyl=O@SV3_UE1iUqt@CsJ~J4XRZ95(-)`vpN^jucGU8=Ft}4GY4a zR=M1zl+27t9GWX&r?gZ|7PGKs#^Y|GFYT47CITL6xffqaTRR~4HjQ7OI_H?q5=Xsz z$qrq3t#4lH(^gA0%rJS+uLz9RZXSs%fUJ=WtLNX&z`6fmq3Y%ib0%Bq7kg6VSBHq9 zjBimW+UV%BL@(>1A2E$A^zlFF-v;mZEKbi~g}{4GHPA>m+Un|j)h>nQ)&aAHN-U6Q zy*y98q=$`P+o+lYsYpsEdZ`@^&46&`9_r3UMLp!Urp5GPiC^XpLgLF(_1LD`va&45 z>`R1$R)Kb*WdW1QPcQT#O5LShGd014>DUxd0nO&dHC0G2R*fhF&q=prK^)~TnbF+6 zjEpye%EJ%>x8$fKt#K5wS%gh+9Ggs2;z=O-o^grkyT}DN88#EAZIauTVb}_R9Q=L( zFyNP|k})K$ixoiHxRKh@2-MdRS`MY8E#{dBc&1RQ{uZ@aI=KsE?x$Dl8^kxK>F2{% zg9n?4>yB-8EEgrfe1-oT%V0jsnvCyJ_I$A*Skj3*lm6|ikZ+(xupLE@;j zT^?^Z#ipvzy9ui=$@HD5{CKt{j4k0#=ev=NpR48%sJ%1!=wM4SRQ|iEzfAukv7u1v z0>g|N3s|@FqM}5jWxFU6!-44}Vx<9$Nx}z2RnmVvq?mJJ-i58iZH@Aj5%b_+@7j-* zh=v>PIDv*0fC(XBh6D-u5g?uI|13?Y$^rZPPKkOWLSbpnjNx};KrGocOdQ{HXV^Qn z{WcP_P@V>CE@G;HXz#E-p0}B|S#~4=)TF9cEq{=_qdem7LRRkHW>AURb$5fC%}D;+zEd%{{4kjO-~Jywz#rmiNN>96x_pCn zMO>g0++i51`<%AMbwO#I`dp2_Pu7mBGMCkI@GA#E)Wgtw?n{~hRgvR9l7h0QN}wT+a4mjeWb zhp`qqLRZ3v*`{E3b+?P;KX@IwJ|3z{X+&y#(25r42K7fGYaEOG90 zp~ByO?H^1N%pN`j=K~GdG8BHdFI}LXKfXd1tdy-CHE37v=!Q?@gKer|W*h~S02QNs zgVw=O8;c?m3(45?k^HUq0&?vwsHWd_gfLq7{dxUVP#x%YnPi?qd%}(ei)QY}?*@dy zD;7@g++FlIF-UWp{pXc6XhMY#Mh4>6?5m?XF?$x)FJGhMH!ypcAXDC7$QZ$9@hkZ z&qbgds!oU4y*3NSKK6?K?vNz6r&OsxA(IQ7?WpwEU=+?;iyfzJj^`_ViXuA!D|K8D0GcM~qgY5@UNabheWw}%{g-FzG=Aiw3fo}1)n=FI%~qbkb9QRhIL z$K7Gg-&Yz|R+UqQr4)J{3Lf;Fs}Nk<+O-P_mfU9Wts{Ch)_gxja1`!&x4!(Q8@1Y) zVB)u}A=AGOf2mN)qp6?mbl7MZ07y^s+R_0iF zO-(}OYTdTQDxGhO5?tm4;@&3uSV&NlP!mTI7bHQH!J@064NEqA+INCcPm7)OpKTCI z5pFMqaP~D}0@`>NccfMtm24a9W}FuiI?{I@-JZK74SlTwiRjeSAGcsaPWCgP=zM;> z;kM9 z+}uCn$?H;*tk(akShqRil>ZJWrC-1--xnE4N((bcNHgMzbfyvax{tmm27b#PWsMxT zT72i|1KBjkR0Q%T6nnW9U^fQl6Oe0#TSn6Yb~@A@?hsKtku~FSW=L8x<;^I z&=b-nb2uF=E$kcCa8N7=lbMq0$yE2&$o`HuPf(<~8QF2oY zl&vX+ZHEd4*+gE7*Dq!v@vmCaqyDP!6JJ<0&FbMKBET5K*{5-O+eNB?GWe7wY&}_$ z!Gx$vC*=b^^o=U!#k^ERsK!jMzNQVh}WmnA%3^wZ&AZS;N7o5JAhF6w;1N> zj1^*w$)xmWMbNiQyl171@ya6muB=shq_+RY>AR5Q9hE&i2HbY>#B^aHBdHj!uf2(V zXpIIx@FRYqYR$D6+E=rsLq9`1eC20kQBpUzu+4S`)(fLqXTN!p2L>P1b{0p*Jc5fb zRIt_l`6_V`>AOg;d6i}fSXL2yz{i*!oMviRp~?Z%hTY8L1(zy0N4v_#tOLvEj-X;} z_e4PtHAn%Rnc1X*?X$q+6&dsVw+6N`CHXg_nY>~b8c8FdTW#^-ZBDSSBfUW<+1G_f!kPFJBn!5U#2j>J_Ot)YaB(*04G zkT)_B#_PrhqoH8G5+nzWjL3f18LNs1xz3X4Lc`Pj#Q3{ghk~58XvzRl@k69h)>oTE z)A4SqNC_W}g*W#ej+;SO-Uv}>C{I$y^OC}{{+F1N-!b({0pa{?^%M~M@ zJSrd>Tm^QUs5AD^nxroF1Jh{OS*RI*%8BK6PD_rc;?8=R!+~7Z85Vo~@kM;29Epz* z&iV{3dMT1VkxiLdF;2>vb{GMoB1IaEJ4@E1H(z;Q#XV4F%{Jx5?U>7BLmHb}FdBFHMc!3X`}_W;H=!xw5>ID^|B z#Le(ruSKJ#70m(vPN9U+;l;_^SAs zcI1*ztiuT#@Quh$`dh-{C2;~{^8zF=-O^Qydqr=o#}2mbOx zne+68WV^g4a0s>9J3rFpm#k&Pg&csE<1-%8V0!8WDE@8Y8J-nI4q@t>+$|$7W`5Co2z&LNr81RnyK!+h5Towi`(G(trKi@Xf2`g$3O(m?^ zD?gEwk?qREPR_}QOU$!GOjvtTc(ag#^!-u3k4UuVxlUMqR|oD{CsVn z7Ih?PTj5>5WP(DZnTY@g{yKo!;HbBJCe|hjBHcUz0!j`%uKq-_mw;U4yA!+7%o+-P z;^fq@WE>_WLhAUJ%e^>CYaV#CyvaTaMnGylsX7meiFRx(N=J3~=WOI1MKpG533$X~ zzre4(!R2_E}xr0%w@4Obf{@+lFT`uY8?OecZ<}DiM^Cf&BGd$6+j{ z!U7mNI?h381k?PyMC?0LkVfqDsD(5jIxH_p!iR~J`bp3m5E`^2zT1b2ox$#c`_Oe8 zRV)5XWtqMHRRzfgkq&Ow<5IDu^4GJ;Q$8Kh_thhpnS)^y2RHgNzy=xrILk7+G)>7o zH~P^dQ$0D<{lm6NnYLUGjdGk`yG+jIn|9LWsvi*T*z#^C%|Pt+ z7Ko)%)n#SWNq^}G6$&waebv?D)+Bx0ROnrs-*44?^%p`-q?rSOo_Z{pXuc#ezd<=! zDk}KWLo=eLACfak->o*YK5;lw2oRE{<}1Q@JSU>AMs* z+9#J&N7>7_)>>**8pb~lQo{EJDJUs9EmwDYePdLseS@Vo2TLDfpxCeE;dOt&gUvv@ z1};e;$;~q~rV9nAcsER=(Uk^GO09=vWfL{VLtHC2?x!}}eBBf`p^20GN99`Cb|2op zY~np1uMXVNE|e~wUA{S+t2X^?Qc*17_qO81z%_`%`#kpXiDTIO`Q%hoUvzoljTrdv z_*TtY$pMo_4HwkukkP)t=B+oFNTu_4J^aH9mCwv>IXyYc2}DcPuI`YiqJ6Yo>7#U7 zs>I`e&9D>vf{>RFuQ#1jQ6AQa2_^9Te8C=y`f<}sTXSo)a=M^!zA4|FiAhtX+2lA6 zLTU5uf7@i#s9LT2@M6;#zoqB(<_I0IEeOoLWS~Mzze&g7iiD6_F%~63KTemYx4&ko zQlK>Eia=9b`cq_bm?_g%q}Jpv9|^C>n1vaGWA951h#F)MQ3WfwHO3=hX*uWiI=8r8 zEt1XgJdf-3^Vzri2E$>s_`djd6~mZy{hlkFM5A%K+ND!sRsDI~)lz`I`?JLYseIC) z(_}s93g9*tl_&0Z9`4_8G%0+)=zXa+1_BS*uZ*hEa$GIHUxd|IGlU*G(xr0_{po|k zkDlr>+37RacVEQBW%YZMlkDNOxpgyTl*PLr%(y$8y4o!0nDB>QJDVE&e4gNJ%#3OY z?c}sNtWgiuyW6kUYknairuWV^|Fr>qy<5A!1KB-D4rAqtdc&6Z_WKQ4rG7Y-Ps>NnzJR_^@V85`q zIB%9grA+(IiubpI#>DD_qhglAO2^gm%)bs!8$O47`?=-|M`zhFq^0Fw#;JqrEp|29 zi)DgOEkn3Vt$is;6>SeLR-BgLZ0GjyD(eisKKB91TqcSEX&CL+T9qM#mvr+?M|$ub z@ezf8GF(l=mR?Fdikn_c;0Ma<7@#LO?t+%htvEWO*)PYrZC6^AI&p|MBnB#VyG|LG z^a*hZb1<**xnlcOw4yGj?#{*?Bh)yir!vV@IAL%5UmmV&)ENzwTNEaY{)If8%pI9X zgv^RDXIZv%82^6&6G800beo-IWHhPk4u?ybviMJ>rr2s$-@0{kZmv5zI#9K0HA!6y zy#3*a4H`AEJEHtnm#btsW`u~vt#Y2PROXFIfs23wAorR6brIyJ$Y-IbP#bX|pCaMh z+xtO-h$3xpdj$-w*B2 zNj(XglN@$|ZgzQMbvvjH%LwQtOP6VS)zz_yiC+(SZ^6_bzZ*Fs;N}&JD=MzTEjO1) zEIVb&q^zv0sWYa{nm3!Z`D7^<7g~;Um#fG(Q?q~cr(wV;udbl62mAbOwRg))S^sNOjG!$E?TvlJ&qjbBg&wYKDEckWF zf+b59Ez_gpZBa2EtIdOuZn66;jvTC6T+t{!#5I-G{5EbFjb%vq2NP z|5g2(&rAvw2MQDws}e&fIG3u;uqxTAtV-+%%IZxqMb-Uw_Mmk8FxNn{BL3oK$&QTq zFx?i%PgYsNj*nQx2CaHlYNfh8+b7>OS3p!xNib<~douj!CvP;K@i& z$F4geA%Q4(YR78|b}G76mfHkmVxC1nEStx!n>bTbl36je6`2_lDLXj2r77oGXsGGAsi(TO zu1c@KqwBf4;6+lBB9)EeB5{iiv^o&1#p#D#&-LKv$%c(xE{hoju5B!mR4dkH0Emhs z&nIgju?Rsn$F-n6$`;Rmd`7b@k_C@GaNp*0`|QL(d$zGk25x`>p*1Sr2`j*3tQ&xK zI6SgACxFp7>AFNk&Y~XLcVNV*(R+99s8hGbkpue{&6&$m;%>Kf=X1A=LM;i26%6gU zVM~I^IZNkR&_mwL!=%6kpg^z({c*te$NLMAv*MU!Q338hfioIg@CT$SUDt<9``y#D0z#6m4ZFa;`(E`*|S!H^@|sh$3H(_*Pj2 zE5BNy7gW!O7fK|O3iMPlKq?idi3}l1U68*<-KiQ8Sz8Wi=SGclk5w#=8l-(n*P+VH z`ckwg;8Rfn?gAszp8=MUU>N3j4O|uUIRiaFbUq_jtX5Eh;7!o&_StOKe*GSO?63XM zkrS1qVzJ>Q7A*>D^iU58mr8XI9;nsE;m`|sVdR7}MK;!GoR7?y-Tr(~(Tt-QP@wn$ zx8S6btp9!Oo{{*Bm1D*iV$jN=06HjLAde`K1Yy|lm>tpvitreQejH2Tx(fGIc!pI@ zWpUq)DqUxHT(K+%uW@4{7qYg~SVgQBtf^#`up=OkBK1L~2|D zb7u#h^;Y2o7&!^&Cm(A6FF80DGZLbijq`t@ruj{Apg^&cxRsymV9lPTkvK<<^F9a# zw1jcWc*#KR{MaqKSsJJvj>z~{h03Ou%wRId=wYh{1XTgzfW>Cd!#AK!bldnWWLpIh zMS(07=%M4!0PGK03cViEHHYud$Hm!cg{0#%xeJ9D=^1c*=4m9pN5+<)hU(i8t243t zd{mtoM-imJnSNCS3wNH16h;>C#K7fn2w-IVjHJTh>NfbU@d#O)3+0|$g@f9-hec1wik=tkiH=WiQK85( zOWALS5K(vYP%LNJ5>w8i8YUf;MFS;`w8BY~lW^+c()~a1REicpB4aib`aEKI5Y)n} zDa!E6=e$rO42X4x#*~Lcl}M<>5t-Y_k{VCVvq=Gy0tN+w9mXWbpHTq7!r+In7>S6i z&6R8!EW6-vD9Dxzk|R|v*c6IsOp$DHhJ`huwsiOHFYCSSAsvWJ&|lEmY}ks zlrUJV$eVH^8=;JGL452D6nrY`n&3f*@geudQ%OsekqZhd+emP$l8|A9*r&^x43kgw z2`?$Wr2=6`*vY6kRYc@L6;raMC?nbOiH$luK-uJknS#7x9wr4$3Pe&s^}NV{VsH(D z02&4!1+T#l5K`08L1SkP4HRw)&O#u5pc`T`SRxRLb~=c0I17|XMmb>EP*NB#_Eah7EIt!+bYbtzjBZ#wT-au0uOXrnz%`IWP z$-5MVGf^357H9G^Vk0dWgXmtwua>+9cOaEux@rnkz04;9wd%}mdd$Y^Q~kQ zA{gAo%9|WgYRMS67EqQ;BH?}VtX41?nMJMdQxO_y9?ToDk$ZzUVN53w{wA6t1>cG! zlCF7}6fi092MQ?g_(NEcz6Ej6S_qcRuKs}h~64IY-Djm#ew83iVp5E&IX~xFogI8m_bhwB-u+19OORWWM6lQoQI+% zRzH!yG4w_PM)wUI3j7H^vK1x!Sc2AwgIZC9!iwpRy^1d>^IElaPtWBIyh zDL4cMm4wc7;0EbZ4nQg3$337W;C0$Sb|1)!4qNwm*`fgxOJLpDnjHiNfZzwDMGT0a z>_JBIBA{|lOuTJjBVd)1++%4X+Y=kODFGtgYqDdoXjBL=z)vm|FM)R;o6Gy01b;>3 z54h}Xyd(J*XqJuXC0byZxw5icisWi=74k z9xp5hRE2baxbP$q{B?Gy;43c3jHrf`3~t0j@!1(U%xs8XUZN}k&1qv)Aqn~1eqk40 zUlg7axd`~hr;B-*6fh}p76po(9XE2T2;55Litq~(iL@wVbKh@semi;cWUpSmSb7f5 zVP36f6u}s=!sTeWJ{yLayuE0>S^ZepvJZ4yhCIGJ|( z)}FnQZdn(J#UC00j7C6xfRd93)DtuWzGYi?w}7#zno)}&jf#bd5^_b})MUGNUIUIr zi4HWwIRrt$G{sEd5H_(?LHHH4Btk@E0eZx9AE-KI$`q^J(zWZYUUo{>*uBA)=k|&l zL^6WXTrCKq>|i>iAOa+HBz-&nu-xL%3h*{$d?`P29&2!`U z1$-8+Ag{~zD1yjDLje^hfZ8ZlWQ3qbHt2Y0ga#iI^v94!lvKhLArB=v@_Ulg=`2;I z3~muA46b0z!=!*ofj>|{&H{A=f^gFcPAkPX z8zBHId?f-P%NwnTtDcOLC;Q!YYt3rat5>U>o^s-`+wZudTGi@RD{fx1dco}3MCUY9Rkq*5R--2Z$u@wBi0q)C?}G z14wiSJPw<_cG+@!eEbsw`xD!5(`VRACj9O3N0C<7%$#WeyZ*Q{t&8Y7kkSR;wq^gP&M(;1yM}%`NV{MSmcIzDVZ63Xo^j zq^QhP;_N#(DT>sLirE)%+QC>r99=b<2lN>fR@k!_#k{D7tVBq8nd($6fU^4y{1%ZJ z9g$Tww-hZ5Y4WpJ+-|qi?qJmL!ZT0H%YNUkojV5i>9t_%=9uU>c+2arynK1hD;n0V z&nN)iU~m+^B|juA?8onSKxc)~C}tSPppD`~VKtHhZ$f8(Y;4RYpA2o@q;XnW+Knw+ z{QArE+k4z$D!ElPnJ1G1`6yt0_0?DN1r)g(x27d!X(!UM%9oCjAvt&<$Rt>H;`p&2 zr~L5U#Bm4r?QPby$+D$OPN${C#>R~pIr7+vlL*sxhcoTOiD6%US-(+3m&;}K28Mq* zET&Y6y}NdNKl0nd2M=a@bCQ#fPXB3I(*AvUK3~bwCHHLJd@B8Pr!HMU*^QgGj2by& z_OCNzqGKvosY0qsHWnN`uy6mku@eBk;ln+2zs!*|FsZu2Y zYCv$>)St(W8MAcR($b|%mnmDu>+}9R`G;|1$1PpH+~IIks$ALQ@vK;}V*UE{zb7S) z8U1a$_E)=e^S&81V)DfA*^1%l;UhR4zwWwgGc&X1{x;{EZ%1$5yy^1GYeYv!F%W}Pb{WwTYTSixzxM>!nhe)_pl z(LTG*`L4vX1TbSp)Wr_?A_Pny;=U0gWv!2 z4QpPG-Q#gua=-mz=#g#PV!Ukjk!O`HgW2^)G_gpuC_(bPcA2u&{SG~_b@{T#uD{li zcA}KkJ?O43d$+B1Xl}dL?<500?fCxv*i(R?oKSo^6}f$9l4!y7#X4UwWZ@tn2jt%}?ImdEWFN6+c=->`D&LpQXyXCEu?a1Xxy z<~=(%T6|g4#trM=tIOI&vtlg1mwISC|tNBztFAvR^C*h;dl)2 zk#OSS9S)U{Up%d=6bWaZSsuu(ydxQqnB^BdRATDGP(U!B=YA*rA)gSrDnjr>LPAl5 zGhn2v@*rVne#V8$|(wL`W2p@JShhbUf=)m+j{nV@L?-< zcu1XEp6B~XKUy8p@d>4nTVHwU1?f<#qoSBV7c3?i{y>Y?tx^tl*}eb3fF}oww%YI6 zvp?x?0|x4jsJTC)G#53+haU{Rxo5A}Uw<8?#F6AD9;TVCNo6fzI0#@KvZ>xx^JUR;^fmMdiz;{W876si&T1%ZPOwH`wf!DpjiVzWbh^#*hEo<4-u_N<@2n3l}Y_ zR<-hg0gwB1>qkRA=yc~DZ@u{@WIA&6*zB1z>({NzOl*TjjXwJHi-2xBnzU!>(xvS? zcYp8wfBGzZfi6nTb1G>4W%itrUwq;{e)8cb{~j9?1B{A3NTfS~!k|N42_QTubErRO z58+*XIz=D<6vZg*ga z!j3t1HcJRtw{P4ueZqGQs#d96vBJ`svlcH|7GSG0)Ya^Ci87r|Y}M+wXbw{R{`>D} zd7th7Xyd9?n%BHS%g9*0di9R2TeO&%#x2_9>MqUV#MW-#-o5tNn7WM`xUH5POH|9N zI;QT~?e#LhX{p(?WmL&BdAeoKo;_NLxLOUGWNNlRbX=(*mcDZHA{NHwO*<7wd#RngGeA%4YTh_1D5=zx+&=@X4xm#Ry z6_cl;h8J?UEm*6taFZEFzvi?!#4H3c6TQABb}TzKuAe>ey9QONG^&32qUqCDEL)CP z%xa!?9j|v=tl0r;lZK5C95~=*LKMg2Vvzz9!5p4B1d_7!dFaul8@5bdxa7MjKYsD? zr%|$mC@5-@9AFp3J3qLd&Kwz{jwfkwoIN+CT}?a8O#YWI|Qn+lnLbJiJ>3!oU-;(tW)YW zXwbM}o&JwLBCGJRERTvM23bR-JA9fWU`I`*ENYk{Mrp+}zJh}&JvIXoF_;PXs#UAj zty|Zrzsz|0>1Qf#+u27E;g~h0A<7J{Y0MWP&$A$UHHyWs;Q0_%!m9{E?Y=CI!x>K=I?) zYB>y+3~8rRf15e0a^=d8^nVOk*mvL{0=3QRKpO&?L^&Z_OIi{uRdSXuKYh|fU)rhB zUw%m+#Dp8cleOhsVWw?o{Lh+3QK85_@gn?LWW(PZx^?+w`t(DG4%wq!Yu2y({m2pQ zNdP>|@i4v+qYlA$o5$ymE0J(*x7)^jHz^}4FV_>8IcHv;W^MMD4)pfnL*Gk3y!&|4 zuHD-}+k4!FSvD}rU_op%bbr%UZCRu^^0SXTsmW8n z8+UZ;W=g>uGp5ZTxkvjwWG@qsMq7pjz&T~nEX^A?(Ok|EAAZ>Jrkl#us9CXcg;5`V z*r9zpNFl&Yewk6VIvFWI6+#$VlrCMC>s_0-1ah(&Xp~MUvv>PWPgWLrx&D?eGiUvF zVE@4=N6f|zTaO$*F7iFgouDY+`UUqXy2a;Nw_@3rO`D0$Vz;xaGDDHY3l=PyJx$9x zy?gWesXtD-vsVxFGXNTG^pDMyPy^|L!fo&!UIShE5vw6G8xph#vmmEf!7I&j@GHO* z$PVJERJyu0I!QP@$Rt8sfm=S~&PYhkLFpJq7Ys8}G0u_z`Dd)4#Ok0#Z=|7y4Sdd_ z-r}!l?Tr*eul(s>Bi%oJY(6N`6!>G0DAI+M!T}Nd7@z0lx}SaSg=Tf?-+5=xUXMS~ zph+{37>x&-5W2mHgMJ4xgl?`|AJ*dT%5a~(69SFJfI0e{9Yz`!7 zJ_4555}rpl-P)~kjhel>-`=WTqtCwj+JjsthT5VKpcjjQvW~SM%NMa#(eCdzV9Va5 z>)N*K*zx)~3zvYaB`>S+&Nrif_<8CLty*{Q)M@wLJ!L9XdVkE=pQcT{wng*%Zoj>0 zvlfpJc#=0!*W?9o4}ztJ9u-&O@mF6P^XcbTHE1wq-1t`4cS6e7bo;R3Bd%-Frb*N0 zOeln>wre`ws9EgITK@%I3j6XatwR>Wf%lq8fyG8v*AAa^F zqB^Y2&3>|?Zi07UDe3efef#wxLq7cGn{Pz_BEuUn&!3r@{_Hc)x2{$9p+_FYEa&dN z_j_dW@Jy3bhz8+;2e;veXF?S|hF+1s>MOz7KR?6;f^kT4#_w!?VX%r&ERuS}FB~l2 z8_}sV?+CROG)o>r38_obGKCMZ=#O|B4mU4M3Y<*=Wlei_a8U`NBV=gWl4IJo{l^>E zO2iDCzJUcdn8o-STkh23$4iz@K#~ND)w*slO4h>25)!tERN^;=&v`j{IT;ycN|r*! z0oDTkh;%+i`xYAu2tDF4O09sgSe*fcad6y<-oRniPM^3J_ z9g+VrmlmNG13w}KC-4X2%gxS?i(!2X;{qpTfxBd6raPTh<~-5H$UgPLoMg!&?*!br zcBd@|jA00nmt(U#bETAs+v32)Bk?mnkW5Oh$S{=h`!mxsVq;?|DM_*nKPxLMCN6=7 zgAV4@sZ&mSRBTKv)*@J=`}}$EDtHXlVWaTeSgaYDS*~a&$qRCW=~kFe0G@d|^;BF! z3Cg7uGpK`+^yN@*N%cv-#YedjtiaGvWk@(rMHVpxTlKOMkR^+P#9{-)Q~4H3N^J5b zaS}itAV3H@^NYqoNF{RPonY$5%Mm#Vf38xKG(C)NV1L-k8`^CnCCDzaKeZ<1;t>M~ zDHl~Pb#l%_a&DCFT&bV)lk*pNR!PYVWb|0HV<$2*(@u41S-CXEML{y3zjV!*{)7U> z4!EUh2|$U9CGp-Q-B!8;EfbIu#Ve~81&c)|!a)a?5in1s+wfQXdSF-rJb`!$bfn)I zhjih97o{)iZ_p8FN2kj>bVpJo9$BGC`U=4X@DR3jaGtn$84a(>kHT@hpAfr)0I|51 zlvpvzNpi15S5{Waoq`+H*9F;R%_E+wcu8gs>ELR`qLhOd#o?*|rHY2a`rOJB7P@B} z8Vn$fmdx9t}@eXm2R5T#*-%Km~AtJW(L1L*P$EEnuqx#VjQB zR1i>N2m!YNqmeJ59SS~Gs4ayx;w3=^;(Wpd5k^42Btcyh&JSZw^@W^_Z&iha!$KFK zHsOUu1}sdQ81G9~r7X(sP_ZAQMkQSs3JYbG$x$el$iEVW3EYtsDbCl@%8S)ulbJOz zM|pPw%v5g_M6vj1&YJ$?gz+|}0+m-lglIaF(r!qPg;L-qSSHSrbYw%{`sGVscyhqc z`ULv&#;dCw@_?kQRv`f9ifX3*}WsV_Bv+hY}uxv4;fuS!b!%!z-MX zEP=(KmzSTIScAOiu!W}(mz0% zUCJnB;Vt+E#8rv$j$Bd`QYTUoybNcO%o|ZD5SNYtXMzGE9S71+rM~v!i`i+XHs)E!; zL5#PJPo!SWqu5iR*a5eoLC86u^a^^Qf*5+~#&v5K%$bFLK{}m?w<73?ml4)y8{L~A z!2%Jvv(i%g_qe@Ur3#l*vj$gT$?v;eE*eKa&mGc7hWk5hB4ry=5-5? z;9K#LB3RA0WYy>K!Xyxu!548dD-jD;gW9OnH z>iXr&z#U;l9;RVIKNjDM#}XNW`15A}I&H_cO+p=U0wZ}wU1PTdhP?guyKlcGZUDqq zP$4pIwwMEOGa`pFL?=1K(bY%KJ-2JuF4@3SHhE*vPCiAO!A~GM2=b7IAym+Co+N~% z;)YjFS0Aa+-0!70$!LPqe>)o@L;I;Zm z8mheH=`8prCG#lO6p+SNtPYH?0UM+x5txuZMYsc~MR_YCtvIsI1!?F%f(E>V>>~i> zcY|Se2K9(@V7Gte!nxM?=z&i@)vQ&^rHkj-DJ6uf*VTJi)SZ+GrP5Y`wlC&?=2A>=k$5*zy9jo z+cqGyiX&NE4`QbRbh^Dvi~{(=gNPWRIN(39Yul53dT4nWymtL{*Dc<<8Ph|Q$T6UH zjyu|Fwfj6#sIEbofE80e(Pt}wuzEEYEQS2b3QrL^1q}u6{WhO>`s4{i-h9oQ!>UmP zb!fqu!jC%N>dA3wKCI3Cx$fyxzJKq{mszptXRWE-%C{5^c;ZbL!ZSmHfY<7<<3CUc zz(@xcq6PNv*fDLw_@|zF3U}f5K){6?8F!A034>gpL#%Nu81fJeHJ4DT^wI+dnzU*u zg@$k*B7n}kBRmY2D+ns&7p%m%5E?=k7D>VRTr6qz9hC5I0me*DCV>NT#w zXz%o~V_yybvPtu1$aeHNe3jtLWdELBQ^tOiv~OR!+n;hW<<}o35<`wZP^N6TU0c>M zHFrz5+W_MoTQ+|+?6Y~l&5ClmDpsy6#N*s8&5LIyyZiW#sVTHEUfS z8y$1_;DPa@N4II!V&a5xp4_ZiGiR(=xhgTS+;?NgG;P>m*ry+?Tejr1H*oaup;paX z02jZ_o-y+4;Tu-3u3V{NOhN*^8}C<$^hpQzj2$_A?}oL^Ql8FZ4k<8c(xjHnng!hM z&p!INbjecRjvB$(qH?**+&S)%BfeU`cz)H&l@dyo0ZDDS*%QW$fhPO*?x|b5wi7dI zz*0>HEL${x>cnxowr^vdQG9%85dU}vApKZ{(kc08nx;)Y}`!jaFEj>OF(es8cbIX>IUw(P`P}1eqsz*5;aMq7M9a^(iEr>a6 z=#UEK%1)RtVb+Wp70Q-QEL94>CjxlF5rtoh+une7aQ~jkW4|pK7xTsEpKRTcc4ggquvMqSzG?m1ufH6=WWl1i zn5fxPru2R2AxnHI6{V8AMI;xU`idosH>_KI?9jncUk^WZ;&_94b*BIP)7a4?n18NR zr8>}@la;k(!Tbr|j@q?r$7PpQC=naGZr!R$lfG}pu#(XX(^5ZC!#Cqn(3>g@aP|r zQnBOVjH$(AszuMs&ggwxx77>hHmF^5&FbZdoQ(I^EM73=-M1>1EBoOqFAsh19cRFc z+I8YbALV3aBlqdvbz z^Q7-4{O#d~X8btW?)UEBv*Vk0-a)t)DISAr_86e&`rRXjefsji0hP*^N!q{bjxIO& z1Gp6wD?o4GRJ)IPTxK~ZC>A|m8<&py6fTl?{%l2 z+OlHlUANqX5}ac6zrW{%kzdrRUUB+QQ(k=T@1%-kh>$G)xe*#!9w~}*O6$ocixNY^ST9qn(`p&yQj2RPS zw=A7M|F1XS^82pc(Kg%jgP(#AFcc8|IeIGck8(zo;7q7`)+}H0R{#FvzWzGF<@o2{ z2ieQ??;hWLlj~UI- z=O52Lmz$YwcRHH1YT-{#K6T<)`sq`1X3s>sMJ>WNl@d#Oty2HXU%BBZOw=|~G|v|Y z1w>z3tPVV1dohT!TeDbTm~`;Qfdg*6{f_$|xYv)ylu4!RobSJ%Y;nanW8%h)9{u`D zFM?1geHnuzUZ{CF4@3Soty`uY?6z-T(x8EZQAy%EY5&240|pHWSY2}`;b>bFw?l{g z^ZMIvegB;|Q5l~&mOOX%ueIyevE$+uHLKr$OP6lT7cO|9=bcOD%}K0UdBN+#7 z+|lDUPi|hb#!XhQSl+U2JGwGnRjXI4aaYf~fBk0kqmT9XB_u9ew9o*ZT)!JUd-TzM z58nSkmm9CWt4o(V@44@Z0Rxiv@4T_YH3#?ZIg*;TZo!QzYS>Hy4C2BBVT#-rQ-*8 zr=B`JaPU*i$myP(6&u$&Y|i6Hj{Y=u-0H=PZn>*FkSB12P6K_qECSLz9(UI7hpJYs z$`D8VdIz+(-+sGJqb3Q7<%T{xc*La1En2n3N6H&-{KFb}5Bf7`lCx}_#zNkg_KEw)L!@9`}wDzw7&6%uRb4!7u_o^um1Yrfw=ZX zB|@^YJPVZ@N)C?)o=Tl+v9Xw?qXtPna(LmiX{&ecbhw-+Po7$}YK!=#~lav?JQrd#%piC`|=Zy+dR1`$;pET4Z@G3D>lYjrqu8I526TKxNsrU zvNCr;-HWQ^4~Nk34?*T_@u5Jmt9v!uCNgV6$ud12dGzHc2iFuBMw}kJ$d&PAsd@a0UXriBeQF${@I;tJ@A8Oak*>8~8-SiWQsHuBl~a zZCbaM9Eyx7N^<;-M%=v8i^roD_5 z-oE3t`}XaFydKs;YAR7(&p4fq7w@L^Yj(M@qn4SPa_kT^S3|@P{uQoaj1#RE&2>=-76dA`nB2(8nGA9mCah^X^yIO>TBs~zTB+s z+cup%w7*I9Dh;bveCMTSx2{->$J*UnHeA)Az1yng1$-^rwE?Hu`bZ#Dn1&1&1XZPK zWYvGeR;_HVXrIMiy;l92ZQEO-OL(l#s&%hCdhnpt=l^~Gu3mR`Z*fJ9XP+930ZHn~ zV=(v5ojW&c*5s3ay!yfWLtN1@xKChRShXsZj~{~?q##K#G=wS0+@@$qiVrTCCH*%P zD0UJz{SA!9N}Gwj=U#mA?z`^z;Dh(??b>b5<_(Bx%;0%M(Tl55=B}6)(_^FXGpwCT zIgyZ1B0DQXJ9P?_770#9%&Pm+znIO_t!}gfjDeM)2PD@H%V;t~A!Il_7Py`Yer+`+56VW)y zl65n}dSv}4u2;w)z+CI{>OjTZC5s)g31SO_Kl11}xDhFfjf1jY1bCH3$inKItsB;Y zSu9EdIrkk%s#38Mk=SewD8d)S`Q?{iptzqga|Vw(weKLh7WfC69Jr8dSd5V~+0+ut zlp=#^sVOn#Dk|XSQ)WrAL?L6C5DIO{3;|@5wfF+Y%$>4sUubXR5w&KEilV4QD={ne z3-hf)Gl<3{u7J2$?Fg@xYn$eP-@t*0%6nzqx`&gJMvoth2LejU{%Wj9Dp#XMa!RtS zc-1Z2cJAO!rVAh=Dk=QXLS?3}i3>AFG1I5A09(>xiqq~4Ko)?VHAn0$pXwvp-wv1y&_|UH5oz zU-4C9i;7*gX8ob$6ZC0t0^?rfai`0A^-b4(^4_bhyWWyv)w^`*^649Ice?2gx8}lh z+D|fQ)UneOt3Bzl*d^krs>Bubk8D`0rJs(DNoaNBE#FO^lAe)4r=B)z7QXkGmm~jR zlsn)po0y>a^A2p^WKBQmVwXvgK*1`T=E085=GC2kOi_FeN3PxFG0Z0djQJb1YVC-M z{eI$PHatF_b!r-RjZQTGe8?;uuHKPTJ|SyRn62-49d}?1mLU zXn0Xf2}k9syY}q|(J<$RCjp<1fbJ0CS}n|h&4_|UklL%|X!cycjY&I->tORsN{5Lb z`})8xoPPI=DU%k?n(2f?V{Qock#8AiiIt!&fQg1M4_3Bf|ACW25Yc0?UDfG^U0b&= zU9~PMHh$;sq^(a)gqa{lq`pk|#r zeLQ*MxUW9nw0?!dm+ekXcAq?15%Efi+q6#Mh>W`UnMX0E zK(VWSH5f+*wWVcbKJ(Jct?D)GerL}<{Rh-<(wwGj-b)CAy zM~*@y1ZG%6kD5UY0d;GK>+uTWyzaW|F$rtcu7hTGUe)eeM7=yjIGaP>6R#;sPSmkH z*2&ww_dodQKi}`rsWW2o<4-=hb=Tg`?K)i3?wW;*mP&sXv5fWQh<~=yWiR_nms@+^ z+3S@z-cp+b16eNd#6l*OB#vMxV}1Y;xCh(tl!eghaK8H9`(KV8+rClr9zA;PPdbQ% z&o^Km&XgqV)oOah<_t5h%yz-8yWpx|6btmp$NSZMPJ0sKY5Mbq`1rSJ=u2>G7 zE5C_Q0QXy9E`tI`E@K6B9e9HwC%WB>m>dxC7OaGfy4~)9G&uXh#jaS~1wQ%sW7sPG z){(}AjUuzdjB#mTwdKj?_6`L3T;?-$rg@R*33pVhRriiZAAjnvkG80L<&)1npOTSD zh2C=e9ntZn`gFVP=4-BN)S{J;NVg>)KQ?ygN2SUnGFG4(8S1FoR@KDR20TIPR{VEy zEh8TCAJ7Lr|HAtpf6}IYBUnC)CVb<5|Krp(^A^1D{44eAHM*tm1B3tmELxxq>o(ad zR4Q98anr`lZw`Fo$dP24tMX4^ird8)q|a4!ncP2Dco7MsY=7`oBBC3efq|&3G7n^> zr6N;5~jN|GU>^-XcRB(7Vk?xl@=ElZ*@A|4C6%;AZiUmE2{hQGSgF|;u2-k zN^&M+d?a3(StFxJoCJe1gj;Z`A{N7fG;D>#CX=aYX=TfnmBO)h5pd)=iIZpit5~#C zsp)YE@kA-S%9qEAm{O%n14%gKA~uIegL-;;2Cil8RvSAxWTj`sCzKE)3}6k|m1UwB zf5W7d&O@U0Cu19PW9Mm$7nUkp9^DQ2$~!n7;6?CS#8|AW#Yqgf2#$l#!k@T=)NZx< zhQSFXNp*Zd#e& zFB`K#1X9V($w9xQv>uRJiRCm`1R;07Z?l@qRB;}KnDh0 z5J^F8x)wm2CrVo+JKgDwqU!@2!H&lx@B;O)jo~tAz^4F3AeZN$6_5asp!q~60>H@5 z3NpAsOrwV@T)@T-Z{JRO0dX>6snCN|4Syg4Rw!FVN=LPV_{p%I^*gN8a70@icGMPp zf(+<%Md>#2oh9lA;s-OB_fyxf3P*G-7%5PP1cab$%Cu#^D zfDvTQVvmj!eBniy3QJMEc_OJpC_bln4ri2X5-D;ela`jK*bgM_e<4-wev zv@w@J26$ifb8y!tE)_*zE)k0UMYIfFxK){igsfcmw3#zI-*mGF4H42bcm=u8gDZJo zAQu(BtZ0=IN$SL8w|G3lK^gE+1w?ux`Dn>1RhqSI#kDFRD_mfdi087d2J(XTxgHP9 zR2HVgtC(_RFahg@Y7` z&h7C=#l#?7Fb$5iJ2WFIITUoNlW||T=U~D{UPQv63QRlh!mWc~Y9d)&kc0~-QOfoQ ziCb1?>tJqxwJP*%Jdb#C+4(E26uTK%oybLY9o1^y-rYN@Rjq-DdfE}iGK#1`AnSA* zwj9-}Rf{eWFRmv-K^-7g0WCmhb3dkH6 zPMzF7cAytgl*6Y)M>Z;;r-N_+s2aCH$<$r z`SMP(MF_hjoUGIYjqC zdLl{6+IME~PymxHaWw^hQaT*X%h&^*48Tnc!VSo{TXTE%?%d&U#gwmdg~f$-THd$e zaWLaF@@k0^@jxLN2Chkk8DkD;3v_Z1Qx#4ulM)Ph7|qC_2VIt;HYPq==IJ!B+f^b? zK$b7C#8&b(P%ZC-e@9XjQbV{ zVP!Xl3p0NHxkZD9J-c>q-LS!wabp1@Xf$&87uPjv(D&vWJJhc`f6fdB$7CImO@Np= zE2X0Vr|5UOh=s&20wXZWnS-uK-{dDEvFIpJg$h7$hnd5PqepIR`xh3TSOT6sJ2y9N z+~CeGH$QOmP2e8xBpS$j4A9^~h1J|!o3Q;(KDVB{=Xt#-VSs?l)YP7L+;MY@W)Jr6Ie6d` z#LTYkUAo@VrEQzrJG5u-%B`E$@*2YiK^zj0Rp5XTHtu3IffGWQ6G%ihh+K%{#WOi3 zmPoo3TZjOc5};OldkR_!45MY&r%m~xb&bpKy{B)FYp&a|dk+Z^4ns}axBI&Kbzb=U z-?5bj^;!`5D@DTrS-9Y%-+98rTXMDTfk`*r5My?gocm?V^~ zb_f^y8kHZ#Ov#66rGQRIN(h{tg933DH42a51YQ`2=roYcS$HL=4Go2zNTF2L_?BV; z=)y8#oHFE61t&sgV!0t6ilHcABmqsxjsPKpA+jxAP9QGdikEiAC&BJ%uuv1=84Rmr=CEw`uF9wpah9N+U7PlJ4TS7CbJ^=4)7 z*|KT)8?U`F^rIE~_uc;3qt8D**pq`Ck2EQRUipwe%1;w2X_*dFp7lh~@**Be4;D#~ zE)AeF*j6Y2vf%2IozWdW_lNJjecRo4v-~70E#=-G-TOX#|ElfVmh4IDa_gPQW5-fX ze(>=ptGDf(yKzTSQp)sc(?CXCfdk5-CKiwrxUu=YQ796`N*YrQRtfo>SgpAk8-Q;p zS^P?sz?a|?Fo*&|<{SlF7-aZ;f9uyTp+v&+9eWpV-u>D;LkK>ZdaU<7eV6Y4eev!C z^_w&u{^e(o4Zl>#oeb?|iV-^$UtTo&8u7AZ!G!>;bdZ)^s3t@!Yh{>$2X5qh$)mCt zkp%hFX3GKaM04(2w_@p=PYfD5cEXBv8|Q7`Rl9ag;+G*zAor<({TyXtP=mX0{2{s? z(iH0*&{S0aC>@}u$gbi@Ph?(@6m}>I82p%fQKfs_Zr;4Ldsmi5{x;(mP@RbZEKJA) z85ZgxTr2Puh*bI|It8=HR>lkBQigd%3!@d}cNFC^nBXzS>;mG-u?cl7KupDQD|#lt zR+*Vw+_W&S2=~ISNl#cQpTZf0dn>*oi0$Pgc!3%)P;01&W;=w^RWdRQjy0{r%}@odMt46-()n^kTY)3)dW(8JCqW1NM}NkLEq( z)GeDgTjNW1>vD6Bu0QbL{Zqd9X#3_(by{34=4S$Zffb7Y{mH?@)Wnncl%v;R<2eRS$OyM9Si5oOG-+*y5qIicb45I zB?7<*kRtn9V9SQ}yO%7PFngvG<3Fn%Z!M_Q8�QPXGWw07*naR5WN%ukMw=4tCS+ zuWt9=yZG&6oFGyqrRJ+iZo5yuJfwA!EKBcktlB?zi93uxS%U0w<0i zn>Taj*3BEM)U4U>k;l*y$YdU6WAGvC_Te0!FF*PC>MmW{wP_1n#>7NnLV?~J-t+bF z&tqK9KD~PH+;>D^8=z%u;`Tdr%aX+lu=tLOidnE=Ufp`NdiU-<mo8^qLixNF z{1ix?&Yd#{qra}*Z)m#+zxwnugP(cnc`W3rRITbuO+9_`6srbv)6EJBQ|f`Prwx`gCZ6S~ZRy zN=i#R4PJ=iK*+Ce-?F_%ix$`|0OcE3uWr@2>1Q8&FzC_8=FOUptvLg7EG67PjK!Om zeR9>JU)x;OhINOCr3m>5&-kL6@cw%b^}5TKle1yX%7=UP`fKmrd70U(mM^=jdlxvy zvF)4tcI=RI`b6c*iJLa8L(CH$xQxpI6~s;v2T50pb!}d^a=@bxO&mEqH|-P>r%18N zz{cVCf`T2dxfV>pDv@H7JW}=#ugxJ%7-&kF)2|qp2(l#O$ABZy=ngJ%gV&R`Ra==J^R$=b*pKP zVBCRi+lGJg$v@wHAJsH0mgI0vPHE9AxqaUC%ND-g|F2_5kBp0s8v5Lmeeb$;)QI6F zO5Lc8eY5IkN(ym*NWv5%rDqAJM7E!JEWS1CuyGepMDx+#o)asF<;c zCrFez=&&F>7Bd72bC8K`Vw36+mdJ-h2Qz@L3TmVK67l>@N-8;30(E7ciXR1voy1Ky zkZB)*AB0JC9rP%fATp{R{R4Ks=TuUY9utjEK}Sq{)`5+4XU$u7#9WA8oSt17lO?$b{iNr2D; z0YWd*d+#L(NG}SaG!al#1QC$lRZx2GU3!rsHT2#B1VSp1kV@+5`TlF45J6wP_x*m~ zeeeA)*&I&x*?acvnb~`uwdR?%)H*nz%w>s z5EK(SGC=os*c;ZZGh^mVhs|8)#37E&|7ijAa=l}RubvNV5zxsJ9v1M~sQ0>d=|8Z) z4fut^Ky?&+1FKYao5_@v#6JE$!-jwQ>_yDL_ugHx_t(l5D~$eP)R?cnBsd8u%XLgG z(qe+&Dz%$Y@35Fv?rv+=tO4%gqaL3+u=mvcC)h;3d>(b~^yyYTKMs8yQoy%xuby39 zJiKfaSfk11>8Z}`9q}Zja{gj0$7~etg&akz1`nN$pWXLStA%*x)T?jjk*wp{>Hepk z{~;xr$OBYTD^K}6MIg8STnw zRA!to0fWZG@*yQ{K=)n)hmN}VFrs_6UOig3iI0r|)(`%AtpC8F2`R~gh787WBY%M+ zFb(L;tXvNIp%_)KQ%~pSj%r-_ovIb<);D^3<3v-cbor>LXW+uf&qp5Iu)ckp=KFX5 z!c?F#xd-&2<7uCTNBR}<6{BoIRO^h51GBm!x-VY6w)=-4&ze2=_LOfrYup?Gh<;hQQYE7Gd3kwtA2w{#XCwRd z>2ve)H3Qx*5;_s4g~q4V;QE=mMDve?UAHT&Y@l|5d z%SXUH`y*M249Na=rlEs5KiJ?fui*Ru&CY|OJ-oteo*kF-fJB%mbH6-hUBWRSy}4%rBy46lmrdFuEv^zgxhJ~0~I$BZ6J z=e~UL5~`k8?mTeQQzuUr2MXdDl`W4Qu^8Ub!`K+n8r@Kh8T{gSDXRoXs12O{#36;d zwQb+-^sO83c4$B8i_cG;{0&xww4etW6e^tW`SYiAaRl5BHe*NBA)`2O`*U+q7A(#z9hNS7`z<6;@4 zh!G5`x>Oz-K<$+~AO4U!gA1ib_b*g1|I^4QMkCGI2kbv`^x%;rkDokw`aGJTH);@T6NkNI$zuEW@1HPv(wY^^f1LNdEXu6PWC&8Q zU||xF1*i>hN^Z(*At3nLMWBF&13L7mR=44(G2dt@tFl@co;GNq7z2vv6}4HMio~?qMpzH4!kgRA zp9_qN4DZ~zbK}O1gCE{^(P`6?U)tgmeM^?aq=n|k9&1#dU&q62Lwt15(HGM#@zS2)My4HOMOqo6%#S#Y`0WW&9MoRz?fDo0FFQp_h z1_0YF8pD&Yu<@S`!#cseSc!=X7b7271ZVu@@6dS>DIhK3QwkRG8~eljSzmts!-NSQ z`ExH^vQ(!x9@uw~2;|eoO%TOatudA;ar|^3_W+(I5!Q}5Ixw*MP1E8_vjG{D`?z7QjJ|8yZ-KI^LGERtoXxi;48#c9{ItO<3T9B{PpiY!QmfN<3IfTA5r2D zX@Gl)HRoVX3uLu`GUfB%+ZyzcKX^_uM5UJq>|*#u zXM)O5{Nk{k1^WOaCectWQxX&N|tdcKY&D4j|l@M*IW zHU$55ndelTJ}^ZVdtTfc8H>w;xa!RG^U3#m5GJMUfA99ZEC|#ob)!Tb?K^4Z{I! zAVep{7sg^C&~i9aP78<=xo8bA1{w#BqXvV{-948LPePrJ_}82v46aZ4&Y_+)=F^Go}QD-JB-XH-$CL)&5!BCD=IxnAm#Ek~3=xkyHp;hNE zSU_S*C`~3z3vSZ_ix{Xu6YFI34Z0CZRU8R{J26=Segv1`JM3y#kKEEp!8=^Q8K*Pa z$4doMu}tIFW4>+IxWW5<`W7l(+ChL1SEJt5o%nfjD?n0E3x^X`x)p;_Y zLL6){N|Zh+&NGM*bXAbrflrDUWRngio|crRLy};&!5>*D z$l+iK8(IcmLyN^+uYQ9n)vH;=gkB>D3E;=2WUAJ<<<3Ks5wVY`ER-M9aSmg{4zU3; zQCq1hFbz3`kqK-GRZ}>8SxAlg>I_6_M4}*y5IAJ3b#XJgyAv^nBJJ3^BVS&h4xQhV zr^!f%MV1u-zrn{UP0&IUFl5Y1l&KTrIBKRNrFq*cgpWv5tWErN@#X(*f&nwn*ba22@ zIb)WH9fB8N42qi*diZIxiI0W=4X7f3AQo(QZ{9xp+tE>@zGV21a>prQrBO@~6=;9# z0(MNe;gAi0iPeQNYbD1~tTEaXK7(YW5eH3X{RC~KpnSL+ot_X;WVG=6nXOv1EL**X z$Rtrn#TQ@PHDxwZ7EOleA=ap;F0Vj`lWzcu zQ)hI}-8uJ_lwzvP=F}FsC0wxJIQpid&f?JsTyu~k&bumrEtmY=Mn$9%VPYMcfYJoLlpxowt^_EXq@G*7}Vc z{`QBGIW6*fB4-=O9bYd`R!v_|_@A@yY`3ghWR*Wh(JJv{+_o(=^a)og#Ehf|wM7TlYF@;})~H=Opnba*HEZ|n z+ZX?DrPZB}Ud9|$mQOYiJI&4neE)Lel%R07ftoX+9DQ|(?7TufCwRR&^RNA@fVDKB z^HCuD*c58zFYo&%s=krTiQ3tfdgENqkDM0x_qD*=_)aolmXW>U*@3R=4(R~j0fB3? z!AdLhePZbhK!RpOg= z7BT-&OML1~2x$^)SlWz8ab|I`vsqjuEVOVQWI#5I-%;yy7+5o9D5{@>*r}OjD<*!R zH`KYxW|hEr4k?T>3J`IEt_!211Shd!&IkoV!C_&H@b@xW;KOP=u5%Xrbm8`$BbTq= zKY#J?frB8aa!P0m6m3QhR8@`$?>cdY-O#&s2@~zZYd4s9A^Yi-q zoPCfBWuuUM{+UflnU&o6C+o*QzLCmnXy1LqK!De~(?C-mD>*ikyZ%nvlnR4~a z?OnfN1A;E)&+Fq>sIWPM$vsikfhUnMfHK&v0G2e1^N8~OI%Z`j&VF5-(8eXQary61 z={)rMx03to2mZzTXxZ6!cn!m|kNxEi)yb}@vvHg)!v(Lm#$R5ab0Vh&{!d!qt$Q26 zP+(g$-Au>ct(!jkc))=@yB((Vw7BP;S~b3X?NZ62Mdr_(vTf}O7qw%{`c=cac8!UQ z^2wd&%$YM_{izGm1O7Nh7Eq6md#`n~BYSp~ z^er-F;)Jy4&t%C+^3TE05VE|+9|vi5~C=0kcw@b|N?j(~%|Lt*x{ zZ|=w~6adXm={)jR->={0e@yeb0{({tusweue6kZbv;FfJB0ItVeuq}i{``LRtMWT7 zyMI5PvzOBXf7JqSpT-SB!FLEv%}M|cKi`5|cmINFz3its?t0z2RV(!dmjd1%`}XYZ z)uZQ+-;QTBz)wbuaOjP&A{nVkE2mBUbi#zeg9d>G?^LTg`SZ_54Iipzg&4JC(&ULv zo42&3r>yyY_7~G<^zGXl;=ydS6SxkJH9RB))%%0qJ&O1h$8^N1U=pCfC|)QI2`AER zvD#gIwwixC zcGRYG>CmA==Z3Wb1vfYMUw=I+i{;3aCIpVcwOefm4jfpsXpt&DcF0#>d$_qUQRmc@ z<<76Y8m|=aENsovrWKwfJCz`~6Vnx3&OZL;j(|2tNDL||oACT>&RaMSarSSv*HivY zvOk>SxxB7!JJ0_4$RDnH{il@qwcnKg@5}jG+ML~-7WhAEfwvC0VWuVKv5*6J8`_(< zXaP?GsNBC7loSzJvs76j5s2%oQbmS_VaM0JU0XZWeu`0ZbW{XPbJG?rGT^|qx@vV< zK-Ts&GDM6x9FAJ`>J#$rY2-74nbagssossS*$o;ty?E{-Ml5w2H}bDruXE!@t$K7H zKIYr}h5WFU22fy|A#aFNSfg%ZtIm}v(|gx%+&g>bAh;|+urgo7B@G$*)$wD$&YC_c zIqb2!m#Zlu)u2*82)f^>bsL9~b&ZXMN|tmlTokW&6V|m@>`@;u5raN(k||cLcF@p~ z!-jqu`SAXL{vRZot)ssF26Gx1lB~B3qFnaGcOiATTlU?P%$|~o0P77qW!0){-`_O;)4tgl&%*@PA0_Vx9Rc^-`&IE&1_e3_UQ{T#@mB-D&4QXFGMDiDf z4~V4seS3woq9Fgi?>z5B6Tbhj^Xlq1(r1H|b0-U-BwP7p|9o}r8-L%(=#4Y~yC2m0 ze|N#F)D(|a;lqxzz^~5a{LN{B|F{--qX+-TmF{0qAjMh)V+QU%U7=x&i_$J==Ixo3i=i)RH@whWj{T+eH{k;^7*s*3;PzTSZUSrg<ENYts1jBxHe{NQ?D-}6MXZ65g_wsBI9q)s@cfxW zhcZ)>AKbk|)RbPm-^U?H*(kCCm`Yr3MimYbE#jz+6}Qg6;yWEb9{ly-p|#6b9R2OY_>^>;R`=uLCCRBN zJ=(TuQ@zHKBgcr@G=I^ORFk=D>sB2aH;s);`0=Mj8W%SrW)*M1%i8q|7WW6ZiQG}8R^29@-}`*XkVaLi zWtgpQg$v_Ho?$Uh9Wmm>i4y`npd|1DQU3Dfm~Y0_E?1%d$CT&YKK=R=Vgz_aF-O4% ztysMUjE8Nv?s&=Txca^PDWSutni*2;Qdq?Al#6Pk;C=O$L_S!k|S z)_TXF9t3wLmfFb$uQnwIsiRU_XFgf`dOZPtG38Vlsj84^c2a(F>axIrQ4|0SyA!T;qoeB8npmQT<=K{xBzjOZNw7}oC0PpVa zFZbW3{Qu=i>h?#^Nr?3R2C=1w%V3%%3vxH z7a4;V6KmyQQhWk9mM2dhAO?J~Ijnl43z47ct0>PzL;|4_5)$&}^&*)7RVJ)Cwk)>f zmx+1v<`q_*xKvUSiCbc`*kNnH8Dckq6bd%j(_f|l?gke(@~3D-9!X0}!)?c%xYjDQ zmGvE2GFWX-Pf12#5WGq2u%e?v1Er>=yLxy)jW7o*6uMQ`NFyfHix=^p-mKj&{@w^J zFp0H|S-BCg!|_M>a4H9WayOPC))9J}9f6+g7sLrzU=4{41`JpP8VRc^0HlC0$vMJr zfZmxC*8{XHqR6JuOA^hG^P*fUISUB`c!7qZ@XD(wIT*xshy}~|37EqBqwa&bBw%Sr zFqU>!PG*&jyVGujs+Eji6_A7V$w?+9K`j(fdOo;~g&&F(Zz3^3#Ft!e z%$+ZvnSO;HZqP|E4Xvxr6*CNaHn!NjzvygQcW)nhtjP+V$f~%6E=sqsoS;@&MpvcN zV+@_??c)O>0p2h@NT0;2nk>EYW>!=f>L$5I^P@^`w3u-%6IoSakGS$R6 z@ywT(Vi16Z^(2u;#H)g-cUZA8mwhQGw}uk|f>oFF5CUA(losmLm&K_n|Ss-PTk`n3LjN^+(c z;y=Y}KxL&d?X*9Ey10rLfOF2~<&Po;oJZtn&X&^xf87GoH{PlZKmlE(Q)fEXu2_0G z@GQVcZy}_d*mnr!DBYFyapV9Lmt`puX`46D@|lZ4`*&sn&6YfVwEz;9pO!so=*uy_XW9Rxm)U?upd%2pehr96^I2j zWa(QyAVy?CsW4Nf}iraEXSNX~z zK|V^}61$O8WMwnUfI@kW3<*Sj(0KcI@7}*}FPGEs6wb-D2qG&&h%zvcqFE$!O0Brg z@+@Vb!edf)oP~UKO43kT8de^bR+W~LVoFbE^rHx?teId%3QCj%2_>lrBpy@FmeT@% z-vVzPa65G+(aY4TLwoi-dGtU+KY@03T*gJqLUROOAwB8jMg;-ELGZ{)cS>tYa8;Q+ z15i;Y5AQ$lKobrakP}Ji` zj`rx@jg|xDDGMo)G@@GzVItquRFGVyKDclH>Q$>09EPwW74r*1orBL#HJ0Z904ke2 z6C@@%h}I7gM7(fzxPa;_DLBsfMxHEKuc+%B<_P=j@F)pafS|m1;T-<1R;!s~v1e(Hdwu;VOw^V$|wb293-BN`gaysd_EwX;RxP zgzGlwh#f1OCS;4=U;qTjofsziOxu7~O`{&(F*qz_rXk!FXijV?hg}baV&^s>K2Qxt z8H7O4pa?^igVew-ST7N>I6~AD5Smef-7c%ZpnL*DpbD5nhSW=j5JXJ131uv!0=4MW zuqI>=D1*@GyvP$jJj=Anlg3Y;01Hmm0_ap3+_F$>V)A8DeQqQN zHR`X%j9#;09fKS}H&kpEYJkI5z!42sbY9V|<*6Zr_(o=Mmca_Zj7m#MIK-uVX9*Q& ztVE=v!t#Ts#JNQtA;3swAcC0PDl?IGn~PE8wMb0m7Sga(2yr4vj7+C`GInA`2P6Y^ zr(hfrxy6`41UJEHwKgm38%r{Bq@fi_pekA1uS3u73x50o5hy1Rr4&X4fg})_%`J1b zoEG>eTHvkYxMjFY&`#DN1Rey`G7UzZ#cmBeb1v}YX^YiVt$NMYojbEQ00~o5(st}v z$6{lJ3K#Cx_x;>?3tql*_4D;a7tvk0*ojEsr_;BC`UggB`V=ofXYoDSp@7}WM!Q)UCBZh81^6Re` z&Yp5}^XSpLU(pi989E0)ethESaZI)w)o&0G5MU-iHkiUJs>!l^$;y_^TO2!bxN^0s z4VpF&diY@f-ks?wsa=S0QnMbi;`xggmroqMdHZgKYSr7cX_qTkF7Pep#nY{uHnJ{6 zK%3T-UhLW7c!41ogVkjD`Ntnn>G$m1`8fDdzhOhcmvez<&YV1BHk+$guimliyO#8{ zML+#a8ltHX`|$lwMny+I4?J`F=AE0Bt5$2>x^?bc-T)$vOal_31ys_p2&cuiXV-34 zm2A?qDGk4A!=|!j%RUGS!p^--n>ICT*4(sdQ$$2W*REZwRH=f>eD2)2_{2Dt9}5f& zjE#-WlQ-Y7W5+61s*H~)_Vr{c%CJ+Pbt>9g?dDCJu$Uyz`}gnDP?UD_=FO3jk)6A| zN1jgQ`^b?aXV0E>b#v>}r%#CzC8!-7`i6DuA3eTby<%lqvYGh42u2yDpf!uij%0Br zTe@T+wZph5B_*Y2&z_#1o*OrA1k5RO@#4ivPhBYXlP6DzX~U{%&z?Qov13QaE?xY5 zi)N&!tz5dSOXp4_MvQ1vtxDLCA$}!_Z`iasw~r5b!gwlc^E<*{St^m~`jxT`Z<%*k^nj388uOfyL+6bj?`AXWR2eF* zg;9w~+%Gfd&YiPw=k^K(3T~S_d+qdTMxkr~R;$8U8#6TqZh<_S-0bVkEfT6Z$99+L(Oy5=ygKpSfMt|Mw-Oqcz8+Pl0 zDlOsD4$b4A+;?|mtY0{P%4b80=Q5;)+qoPhc2=VgrVxcld>khO1!u1<^qwYn_vjZE~pE-5< zz3x3x$0S@7_93j0$nv}5s3?4P$FA><7&jU7kldd7!-s$E*>_;27ESi=-PfR2t?SpX zCxnG{?bgH1Gxx?h6T5cpUbAUylPUe@Y18NY{B!FL-GGehRVrP%b~P#aC0X_D(W{t$ zNsFjsik&f>DR>3xu2OaD(PQK{lc>tay?VUcv-`9eQ%zP&((|z2emgn1a`lYV&sS~M}4(={zBB@M$K9PZPO-wm)pbDo|bCJ zG%Z=SV)T@0v`N-mCDU_47sClWc<`WppT1BKcC%0fwQJX%J$ts@Vy%7RFloM@zd)-u zRy}p>;>C;al_-PQK(M0k3Q=S-uiLoA)7_(hr^nijn}$zD1(pd+vNq~vnKZ48d5EE_ zu2ZMptT}TuR!gm;zmj6^k3V6_bN1x%OXmXfr* z*s9FT^5x5hgg$PeGt8Vj7m>qUs*HPNZUYL9mno~|a{lDBz~8jMTlbm>Ck=}#sE0wa z9b=N`5l|J{GIQ+|Q zYr1Lw{)26L^>y>eB_=G&sGMPPWJDOx>FZ&`s}?U>y+kopO45aESL!usUanrFE=?Ma z|Kf|J*cTA7!i4hBSg4mlwOU=jVMD)IvEBm)^tWvZ( znl`Lk+JDXRmFcmuf(L@cKMz;g zO^q8hx2auOG^cjGdW;Vu!b63;rPd-yP#4XY&z)}3s9w!7#g{JoIX&UI9+C+R!n97> zf(3bmct}jQ zh0E3$j8z*o!Tu7lSgL%b*syR}BZwuM8F+UY%90!MrcjZBVNv0%t0_?wczq=FF5+H^ zswk7JIa^K({NomQ>wue{Lx&U(W*iOCxLHWS-Gg?D3=e0YE$SJ(v)a=;H*PH<;gL*X zVF8AB3rgMEl`Fy?KR$Bw$jxh4ef>+Jj!EBUO3evUgd2nrP;_&oLY#yvk=ePBM~_R@ zs)Hdyi&kyupMj^(?Ap2Wy&m0gi4^NUibEH7IEohaWq2>Zr)z8 za&_{{m*0FmX!zhE_ktfyoia&z7cxr}^NV;I$yz%od8sdxlA|KfYyl@SW4O*YcnRi; zO3y?kK82a8u&^+M9T*z*G!kLqo!18tBLRv!s%vUO!q|ZWh7bGX&XY$oW=^Fx%sSGH z+>UHTWfj|TKu}Y%q(6+a6A5W~T#)2O*gYNdG}WL?p=6e#6Hn#IezN2!KdJflc!;^WrZWV zBRV>|Xi-0B`@rTS(q7d9-Ci+a;atv^(*l3n0&g8~W1falvoIRCi7phvLpkL|6&Duo-JwCK~ttKV$?xk}H(Gkxg!hC#k`Q*vHclR$* zkAvyzC}V64foWth^^nCC4Vb7P8P+db0W99RaDMyd4ei^u6>!q&d-v}3&6qFq6fRh; zP8|y~V`^2A5~bh}GE62H5BGZQ+U(rFCpA6IY_;q^bP(e;W(cocydzMx*P+tJfl*N5>~506t~QmQGBHU%Pg-nB!&I zn>B3^cH{iXU-y{epD+G?mZDOk0n0F*7U6?=iB}2VNB3^3%ozoW6f0V_)|!=zBOct1 zi+r+j@xsDYsun0#4B9BSfAP@!cT_3KNeS^}Qp(>y?nV6CHR~vijQXXJ+|6Ocng!Pz znW$8kD_8DGXgK9ZHZa8suv#!m!L9{^a&h<;pqDa`1D6Bn8YOTFk4cCTX<^|<#0+11 z>&mrtE0@ds0CK`97?`<`8B2YJVuJ&SYc)plmrF!N*J@qd>%ZHPIA5u$$t+=U?6=d% zzq*YZqpUCcah^Rf;lQ>nVRvp&Xnar)?b?0v_;JOs8v7qDZlcmQ+zs`cxl1ddVx+Vf zqKFNnz&Jr1V+{H7_!TQ0^6h7~Kt^+uz+efOci ze`!Sg)$`{b-M@#Bv0KCrQl5a*_K=;NEvE(kwguig;6_oxgBmv)I-XUdHRHcxaQkWD zlC*S7_m(X?G^(GFl~gHTu7zT3sOMCrbry49&zWzq807^4;QFVVbr&))C# zXjrR(FX3vzPmQ*6wdzeeb^3D9;6_!eWLT^oMf@x{`-mML*~(_gRDUpF;IcXM8a8eY z)iCd;MX4F~F0ER(tKTpwDS5%-r3@?Z^6dNJhl^&;ZPB7t@sj>6d-m$rrE9ZBjSH11 zBPKRFBUF^o#A-|=fjEJ3N?V~y<I^pIjcBhdV-KN(K_H}o!RpXjiNrz$ zRV*hbKy2p&l#79i$T{c^MkpAxB)o`$X+UE|ZlQJ~04P|xsNkfu^Y}~xu0-xtr}K1h zE7j=-423G=E$kjc-HHUhS5`}tzEjfex*v7wbX6bj$;m+ZQ=YMX0dGqG2T)7g7Lz#J|2p(Zla<-fn_$OL` zVWtcM|64ce_J>#{7MsnWD9Dh-7=MV3v#5h&KR|F)#;Qz3;_@Q8Cv1R5lVM9wN=z

    Os4s8|p4GB-ga9&|8?W|O3heE-|nSroJE*u8&cQ>GT z-oFyq|Gkb!ZrtIb72U!#?H7(Q4XfPUxMuOQhmLu@W zEcTF!)(%JH^ts0V83#``&h!cUTEN}IgSdlY^CA!g0VL+lh%`1#-BgzJcCA{B`(|>hRxMb^Q@9Zgd^Cy0 zM4DW1a4`ygV{R@AGj{RtPF$;)abx!a5@YmYg`a8Lw(a!UbE?*?rP%a9LExHX!+KiU z2XN;CkwG#g$`_AVVXNgPGzm;a-rPP0qYKd#fq0|J%=BhPx|K!dYz_<0DCK2J%c?bh zd;Cc6>dO7e)ndW*l;R?592v}IFtdTlI|C*Piw>?3;tUx0@iIO(cPi(xgfMrd$6X-4<+(NK?for4`KUE`wT}RhSV3=0E~fW=2|CNN}*bn=A30__(^c z(hYI4PKb|#uy7$Hw_fiC2`+>E%(xe^tZ0)fj}Oc#kcmEqK?B;P3&`~{@yV0NdA+=t z;sb*Ko=iN;!8N_Xl#!N_oE#S$|Ki#6iQ~t1?AfbHtF|x$PPAf{%>Z?(*Tuhx(dmpy ziHVfYWHG0vW|%Te=yWbnysoYmOM395V8Gp|!_8iUA12U@WC6`sOwyWRdi3a#&S3QN z^u~Of+u~YdV-gF6pUBVyMnC!x)`g;wQPqWIKt-?VGn#%wX;hy>t*wEr}+JgUlUL(sEwh$f{6)nFCam5 zdKbzfVYDcy-4+@W(y~P}&Wm)RAuuyL= zPmt8z(=$`6XGB74a3gGrh~CAduTF0uKni(2jdzIuy*X1RFJ7}!L;`h} zppx7{rpjebL4cm7`VZbHt!!B+&XZ*%#8A~@GL^A=sober z#g0`f{V;O|%H5^F6AerGec0vQ78S}K+_MuWN0dJB6sZ9}di~U~#{NYHbO~rvvDBWO z8x3gBiY^E|LBGr$H!`3~nT|EzY3S!WcGzHRa=d7Wd|;u4v|zTAxF_t|4({1Lq+hRA zWlHScu!hj8U_9DBsAqJTe%-OYUDa}XwyY(@9q~z)|1`f#ht>f#E8n|y5o$#uiz@>T zpba8w#e%t%d<)k3dbqHg3PvzOR7%FxNN_UE~+%a{DHbK6GmlzIH{4lK6llnP)e4XA*& zf@1>G0_eOGcr*dl5}*bt&Qrq72z{NBm#kPW@ZMbyue^)}P}pTonrxjj!@`mol{OJy7{9!cUI4uvw7dXyR0SFtl9L5<3S=F-54E>RRwGogWB{7lqkw3e&oZ3d^~2_q-#O<4T{|{yma_niLX?AhOzy#=P^ky^f*Loh?_a4b zbOaGds3WiiddEBqpEO~N*24u+#qFs0ojZ3v{GcE9)vj8Fp9OHyKSgD9+`Dyi_3Rm2 z4(z#m=T_rpO(uUg4tqg(6;Mg}(&6^mwnz8CySGCbHwIoz1)&OMWM&Q-^;LeqQW+XU z=_=J!d429bd;ozkf6>yeeclIFX)*#*Fp5`c^*=6I*11^W4WNvjI))Janrx(yie_~M0>q1m)4IGybxebU$$DSKspGYgDTqc;>7sJ^j7zz04Y8haMjg z==AEvK#)$a7w;4t^*P(IS8vV+bnStfclq2U114>Z@D+511ku37O)~)AX<(=pkF|;#Hgo_iN@vt*6P#A;gTs-D1TpZB54`7Gq5&A!K5z=N>^SH5N zKORbXDqv3u+k;i$f>owTWig^L+4 zlSJXB;-cXbJH{2_Nbluct$zKY6)VKVJV#k?+p1+Qqv6=W1IR6*;CY%NY$-kpHMJ3n zK(cW1AkxlCh!rQqOIM@9S<16uF+xP}pNz9a1}T7WF$Pkek$lTl@;Gv~oEG?JTHvh% zZXpN+WH1L{kWCnFm|FwUNK}alwd>SXV(-xlL2a?ECTNl9ScHrM(Mr; zR067=Mn;w?T>&q8htB9*660=DT=X-cBb3C>XAt!?;?#+gtkfPA64Ikb4?#_`m7WiX z+V0%EQKvxz_!kSNC!h_ArCPIU-I@Mx5CKMOBdzP!N|3u1%b-6LPg;w)w+6OR_; zhx%D$RQ}2pb#87{#LdGikH3F>%nJZrR6B0L?bT``qWpI1g}W4KM~@7hzlk zbf_%og9TqycvOYTl}&23Nuw=<>9k6f@FJF$6bT_HD0=6!r%z9uJ~L^`Dfo!B_=ZP^4&a@=ADtmp}8X%8h7hiUU1w#+P6unP*yPgn9S*`}d^UJM`+ofk74ajC9{m(!46s7c zOhr?K=TuNwf|xQXiQ*@EG$bZr{Irh5#CZRbWo01mu)DdkzOdC|!e#?g2D&0HCAc7F z-n@A&Wa5~_h^R^pYT3-#r-5@a6Ge(lTgKx@54zXvD~4#oz7pzBefH$Zq=@h~9onN) zCMCvsl49O}fA+dC6e-vSCZrr@QDU*~(tH#NY28G-K0&iZs z_9T3Ka(HCL+O;uYXWCAA8i)!FiZ?4c6QxUH63CRDhy@`S99Bc7Iw2}z&bQxg+qYk> z7i$bFQB$xRVV;4Qi2P-gforJ(tO#5Qq^r$LOo}N}%3qJ&WTuL+P^4v+mCCV@1&EA6 z5%Z3BuU(lQ_B1Ix;+=+dGR!6h7HIAz{7diNvrm>r5(!6JEAN0JpLlxYpftFG)bbel ziAPd4q>C7HZZcV|$Tr%ReL4Q>lpI&~MStgw5?F`2abB16oznt;*8;!4YJZpNzv!Hx zrGO(a0@s*nN`5)~lR>!Ejh`?9!vdIN=|jpU>Mx+doErG6#3mI80S$y~1$BjrWjOBY z;hvh2ijv3B+mw+`XLQTufuR65;)C%VZh!&rb~qatc<#=v!UYSB95F&h-yn<3lG1TC zAt50Y0Cz!DO;H9N=B-;c1q>MI>X{epC8S?^x``eRNO4U@hRMS-mxvOk&9K08%VkP8 z%TQgM{WLhlc;)s+nM5*4^d&KT(6h4pi4!Nz1qJyQDKO^CFK~%OgG9X*S8-Wl2Gvyz zrKR4Ax=YRi>sIT;ao?7~YvxJl!9$0!qI-D%9_t&RJqyCpi0J8DBv%RhOyLnc-kx6R zX{idz*)e6IcHZ9Jk|9ve&4f|O4Cv6|#HrJPK@SQQF7V}-U&;tTz!B;Qi$*RL0s{qC z%&2J|E|b=l?b~tVN|h@|5Wd5Q564753%+;%&h2{?L@el?$&`jw=qV^5h7ndTFVD2J z6vehgmHLvvFDlR6&hqf^1><=gItB!sJ$>f0Iuzw=qe@hbRWvqAUC?J(RCE=5g-q*W${Iqdgkf(88q*F^;sH$R=$1>@@sS@0}h zlV;80ABUtTB||qrXpp;B=UTkDKa2;E<%Fka;UU+4J=mjff3wO+z`i0yN<6!J-(<2t zO_(z>VjhK*Dutg5ECT$1ra-}>(YJ$asb+)2k&&Df7aCm5zbJ7q)Gl}@!k`0u=)w9- ztpfHLa5G(@=#Y;%UmDOa_YVs8-lHb?w*l#BZshX4+ zfA8+?hYy0lLxC-33NXV1f}|lBY=Gv_9lAnAiiJLWq_i0V36)wuL-?KzZ^NDu(qU zPncN%s>}1?(VAE!$jhTy>y`;2A+hmsFhb!`;U)cxAw!s}ASY+Rq!yxn3-a=j8Yw?T z2Uk8qjF6Q7iueb|88r}BU8#vE)N)qYn4o;U?~#BbIaVdFnkVP)KWKrsjw-=R5OZCY z7&ND)eLA4OIW2YCj9JmKi8fu5cbNIE{sP zr>eyGglEsA-1GRDY}VsPk97@b*S=kcLCZQ_xpbjo)r!lOENt7OkE^?93aEh2O>e|+ zBU6>0k`^BI?9s#El`B@)Y0)}wZm$fyH=gXVt+Ra-xuV3%?;Q-1; z8DRDyC|Jl?f_z&nDM>FA;$vN0bIqPJ2Pg!Z;j;U5dC;L_rygCquslpm^b0(U0H=hc z#F&^EpWJx}_i^CZiB26lwQmerFqwh)Nl8he zVG$1>-e0>t~?5PjE}x3@4}cP zWMzDl-I;h;pm6r~`-xYFc(g*20Q$^s@*app5p{rLa-j$S`Oa3r`Q-$6<;1I8a{lJD zz+be$TL;`ivdTbQ9T^t>@Z1GB<{nKO;+DhEdfZQo-|N(=Nv&!soA^sl88Hy0a^B|M zO&d1oQiXtU%x^iyeg26`Yy5uW7K}Q69679B^|O_#SN(9zn4!IT0Kyd-Hyk=;W->N! z4F6F>1s9l?)1EtceBgx4?n~m1F8fcKT+7UnUyM++p%Sn`t#3O(>*=I zymj;TZ>G%@#a>p0v>9^cnzL-x*Fy#_n=!|$P{BD{Hkl1>w=Z9t_t~hbRjZaPQG&op zUkx6zXyz(;MdKRiPY?%Jwz zGuSmI8=5&7mPt?%%{QM9yZhS_m05fJ)P>y}H+CC2{PV9z32JNHu+uYX^QNhCnYBbu zwRWlLuP_*fF~h%6>Br35ut~3WoilR8J5?)HsaL)Kn9+lJ_Tqgj*SP7Z@1~^a+?=ct z3^!M?ZqV67d(R%(uPx}?wd)7{J{UmC{kwJ%s)ca^LAe;KpdA}+j&9W}3;5|Y-+nfd zr!#Nk)&|X*o5j!7VRqP)U2-L9jaDjSvvjLj5wJ%w{_3+YRl4!>)~t0gxXvBbYW2MxsKCzC*1cC|rYhS$!nwM20i+oA2H!(Aji1#q$=;MsW z@whX%x$6nd0qI~inXM+zTpsCqHwXtYv=e$<2FFa>fsDeh2yY=>8U>f>EwPB!s9kJU zOx)TuYdUGllzL5Dfdk4SXQYuh8}xdp2`1XeFgfN$KA*hFdZj!zTY7rBi<<|97v6># z-0=x{^A{p4twhkXF!c&|f%wQYW~M7dF89^2^np}RrE1=?*|fQ{Yt^l17J6ANmW$%4 z2%BqjWgRiKCLNZ=NOV0`3Ze$Ak_Itj#lYEwN~~qYHM4k?NLW%$2LAlw&}2-+In$H5 zS=`p`R#S%AVsp>sX=8#B4<8gv(J!}aXyZJDe%!MMbRF}TD$p(PbQThFkC8emY!jHOs9G9Iuer>47m zc)_h_+S2Vdh6HBUTwcV66{l83iI(!<>u66Gv_(qlv}Vi=WlF<%^w6P2-_Jd8=#b6L z)y&w5g3Dw!;~^xm*!1x#Ai0?yV1xv4V~D~m2-61G8HqB^$VsdinX?7TQMhSCL=Zwo zf`XA=XcmEaX%kM$JHeNN*dk8;12<=>l!qwquWZhX~3Kq-&G-O(nb4E80SY5$6JTUp` zS+EcxsmYYdG-gem@|er5T36Czxrmz$wtqUU5#N>ujapQ#R)=K~VQ*!YQRrbt>X-@Y zQ*qw|yz~ZEQlzTc^OnG=tmL4Wlo0h#EaVh1BtQwA+3g;v%$V_II*7QWaq}d0fXFRS z-DWmOrKmn~iZzFf)YqDc9f-&k9tZWU)1ZF!n$?Kwf}g$EN>XlS2%)X159|svrIHUQ zEfd9<1&L?8(?Wwrq*0Nb;GRa4&eH^_#al=3>g7o`W+C6*9BNlq`UGgC`Q_PBGnsr~ z;+Q*OW@MJ9%!9-ZP@A5bF?rgI1@phR>J2my)U&`5l9Qq;LnR&vSZpB_xbsC~K^{s5 zku8YMOoU3$kip_yY!kVSISt}P>|F?g$kRjeQ!Q>xdDsN0IV3xZR96nliJUE`1^&7P z-b(Q(YZriQC`OKJH?Ds*=IeEvw}>hP%*lkI_>|E52mmce#RgPrR3>E*jD<7_z%`V+ zB}mW602v=nS;3)K$U-Us zMf_F>sG`9d52X|VK;bEjZo=fL#Q9)i6hz?-;4q+$PY)=E?uwrct$`(?Gm@fA!oJ#Q zG-5fC1E>!E*=0gfh! zg5|X4lxZ{YO}%iD>Oe(M7vXPcLf8ZB(@kJ10>l86immS|RlilxOQ z<{~Nai{g=C3Mr;FG$6u{>T!_rOQp$(!rSb(Z{7H5{sIvgsOfs6B+GQTxVfG@eYRHJ z`WRp03IszYjmCqZ+>!&x&!gs{IZ0_=4TXH8D%^-w6$~nig)oAO(9OhbJchvl`aa@{ z+EIRF2kkxB>z>UPMJbxqH_D6sc6Z_Uv7;d=;UK@HQ0f9@}amSpZBlY=9{1EnppF z#Xp4`8K z8Ks3dwxSvW?TQ3qSp&ins%!^VEsuKU>XiGEGIgn&=3NhQZ?+znI5I$ zqRk3-f%dMnwMJ)lE>SEup^1Thp_Kt$jPV3{p;XYVH5%fXJPQl`Wy=QQ$`J1$A;D%zhhK99b^M=*ZHeZNR0I zr(BIL9CslgEQO@z;vprok@7|<)sZSWq1?Hb^PSTIf7=3YA5{V`36o&UBwo^~GpE$v zxrntb>&n>7My(!OV)~%~C0JlJ&z(M{xL@H)<;oIYo4${-Mc8{n_qiJlD!Unr5Nr#z za06OJQ3WiS$`ia%R5SQx)&tP0%_b|9I+%e;8&iQi4pdaQSF4GTAZjReG+R(AZ@%0Y zLmq-AP!@*>#lLmS-h+oMz^GnK!bt@)LLd}>KflTqE5Jrim@w9yZV{MvI3%PY8#a7W zY%Jjn7%+mk+>6kxHj7mcMGWkjGj!OhWAzASEIPGtAz0YVoc^7kuiqCHCm<-$_PLh! zaWm@7rVJEONuyTHoHE|OM5z&F${C@p?KWTv_jlw4wlS9UbV5K;0?cpFDFH?FSy6LU zs(Ewf^yu5iub3}C0dA^hwOENVE7AK%W5%Lf4BBB;#*ZBXDw@ToNxY(@5veq!Gq#M< zb_f~{BN4=?UfR&k6BObDZVIx4!~DY}29#$;5tL={gheADDV&5Dc8DJ=8PfU?JvNJp z8v*{}dr4lX{d#UhD@lkK3PdU7uN^-ijhV$pFX7Dqvy_VW1)+c&I!8X97@s`KUbDUiqOPBDMOLEV*;rwn}CFA3qG*x@j#Qg$0Wg1s1^%>F~D9 zP3zYIltl~VbJeJqFIq%A)&f4>I-7Ozg83y%l(d^|SI-A-UAsQ~!Ht3ieB8aTV9Qvt zWWIk%-&@zN?b*I9_})#zWqRh$o!iGpFu-ovyJ_R;Uk}B+hPl2<6@pZjeb_KVg(PQ?n&_d&1;wK+rB9}GNOE$(nc4<_D$1&b8$a&w`ks}?Ql)VF`Wf(6Bx4+2E33JDJS z@$0XatX@OB-18UC-MoD1d1Tm@)ytzJ!mCuQc5?&@4bSJDiebhFbW^7x2oEy)Iz@oD|n9x(k=Axr1gK-f! zGT4_1F>He!*RNdIzk4U-NNiN(ia9gC9z7Z(^tG#2e$>C;s1YN4ymGVB*o?{J0i@KV zq>imx-MW0Sc;Ui7&YHD-^Clt6(GW9PB^~0J;Rf0$-rW+Y|VF)KIkyvibojvQ9omBo6fMtsUcjf{+(KXob+ zOh!Novmr{hvWf1j&fKi2tm2pAY)Zc>WvM7t}!1Xeb*sV|Bb7#&K zEa5xyv(e1Jp@JGP&9fq1q&m~){!p=E#R3KL&HMJ-B|8q(s#T+~Z{F#XCVuBIZ``=b z-Ceh4+creS7u3e)4!RzY?E}9BtL>7R-IWR`V7=uiFTD z(6LMRE{z(WK6$!NulLXWc8rx0hmRVKX~d)%GYOt&Pf6F>)bm#_nfl!%lduyKESr)* zH$b&BvHRrFeEJNG4EYMdAv6z!45WwX-E39r)nln08oBzZ>hNdtep=wr*pHn0?ec|F{>6Gt zn=%#B%wfr}X4;mnUOQ>(cLR_a2lg*t`2C)}`^U_f$_yKR9sdt|=K&s7weJ1NB-7FZq|*}!B{b>1cR>)5UKJ@86jVe- z5kbY?5fG#o1(n`=@1YY)2qYmP36P%Y`F`)3BqHaYd(ZPd&+(pXvK@!Xp0(FrYwcP8 z_g(+>78J7N!~|5%PMy0jd$EgP#}M5URP^ZM|SBN|3jer3``9#zKubY=a34FhXt zJ$f}XsQ4zbO(?OJL`y?pNUn1UtcqPYbTO$FVT3(wDtB#-=sM(ysa!RJZMy~y9|Pzk zA|f;-#Jo~870_wbZTXz=zi$Hf%fg%G^ts0&b+yrv~<=Al7`V0w~K*cba9k) z(eORI(vp*0#4)l)bbd@W)WH*M)U-L};jwGftXZvDGor-cb6K@|R7z43jP=E{ zXV$G)9$mRgohp&L*KIg*q(C7|bxP5=?*wVZv{B zG;7&P=x>8p^;*#_TepSM2417<)=f!DgcLilcVFAKZ6B*%^@jy>9cfsPu$QZwF=NJ( z@4gKS4eHmgA9b^55)>MG>((tBySm~8AU{00Bkd?XN=M-TcLYkD#*LoF?OVTf`#sxtAKS6_tFM3P*6s0KTX&w>dwBMzU!hq~ ze(B}r&6>`+bkUiW+P-N^8_RWeC1G$C&mav$y>xdMO;(r@JbFX{gpI&AMNGw&*&!{L zv*3&vpUHm)g8>==fIy)m8=Ep6*H(|Jm3-^w@}*0O+KL4X*gUUHvqG2 zn@W4w@@3ahA|F0yI_qj&x4bu!Fh<-#MRzZiZn>ii~X2uA7#vb!UvWrvbY z5i$!KFt}v2xsZ_{_`w@vo}K>rr`?|HK`fVP<6eSxC(r2d!-qV}gT%n>tj0@=_f(DxtDsXAsLKC}`ZuL&br1H|EbOiqU5h!uL60ikSkwff!naWrQ zIEc5`yl3x~KmU@KZ^q?sCa#{kAer3xf+`O5YnTeSMiZPOd*^MPbU{J`lt5-=fi>0vW zJFv=^A!>ARz9SPEE(oU2mT|fB5Kelb7e!>(?(_x=3NJ12W83 zG*nYSxsc0OuG-CJr^VjAU;h=We?EF}U(Wrz-+uiK8h`VatpS|ikg$uFu3)bX3r`m# zszwrwY2lK^d`94iMINa#ZDt@96B;lGB(4@2b@BWKRw~d+#2#UYI0+0>CrcT+EOy+e z`C=*!ipv-x1LVym_y-ZT9a!2vK0sXuI}#>g=iG&8k;Rxu5av zoOxMUnZtW@?cS*VrcE2rjO+#O#1etaKzSr~pPNC45j6qc@G#-PYovYW&VW?A9zF8i z-5a%Pjb7OK@h3>=jmA6irB{CV{Ofknb@H)w<03bFBQeO2vDv(H1pp-$_m*eg;Zmfrrsd4Im4x0lQH`NyPv&jxg{rX|zUe zS}VHFlg~UiW&GHd)gz|8`Pzf~>AA-H*RHYSNUZ?K`|O zWKh@obyokf=F6o&2LuKKUfp{4IeqF>o0?GuqptIS!6W+(?AEjyumx7~0+Hdk5ms~@ zQ^3<%E! z3riIQRcmlsHRn#8-oJAvmLZ56rF}yvc~}eRsm>yJv5)d@OL1(~SfRV%R4dFheNf zflk5Pzz*S(%8T1!e1d(Vc;#q3Do0jtKWNaX{sX!P7#Ub6)*(W0E5k>@Zf-31EQ+VwJEZ1;gv1*W(!t%m^hd69_kID|AF!TAIIq8F#&j=p|xeO$zqh ztbB`wh^**eMvsE@v{a+Fx87i4b_3C9c~}8sL*ml%7$m#~xk@+GHK#Ndi8t=%X0hV&kp~LdZANW@1y^pH#n(RoIf0^z?XioQ zK<_oe_+zu3pY7-8mqY3Y>d4N@&B^j97tCHFo;WC(P|E~t$94=$)^YNn`WmDoj>{P^V;?55|f?SD3~YW ziY3Gin++@x6c;3N5yw&BoG(NOaYv^)H^-A76OR{nB%xdoAtG=1J$P!R%iA~q{`ps5 zZP>a4){S20=I49z?9{T1T9ho~q5j9~Px7;+wd%cag-{x+#5EfKJMSfqnhh)|9yq ziGfPYvK_MscwRTWOPG*MctDkU!DL*rQHX&JfJGo!>n3POh$vhrg*%o=7a)oi*K2%s z^w2M&eE9~IgLB2`P^cKRQ?4cNx|azLcOgWC6~-Jw@B*GiPz%T)G=@wszC{n#6XysP zfptA?)_Mha5mbcppdQFA_$PrvfLPQ?%nywwJu#$|o20wWw~Vh(fH+t(8d?Y$je%C# zwcdWjXqBpUUSzjnl_5Zhw4$}}P7H-;j_Q|;tMbhs0>wwtcnQ zb+|q4!K9DTB+Qr?D$4=CF5C-Ssq%-45)Q$_?U1hqFJC^P-BVtt*Fj-P0VJqimrL|8 zhi&=?vp)UeOAS#3cwmH0@$uo8BT`?9TzLj^6pootOoia6PM5xwj=(<|0j7;hnEsRR z|Bs)hZ;xo)qUH!v0D-cmhD?o(yF%WUu)t8JxA;aYPFbc9dK04~8+{yv(G$}K(dm>6 zAX+nM%cN%fMC}%|6pjQRKJI{-F&`|s`RtJy{5`2AE$JRAZMAw(3O(Qu zfOdFbVcykbX6Y|`fmXJ~#N+}zf&{?RnpL}KXlxfU$;EJ7i5{>YzQ%erF0OoqN?tzR z^aZiPjc5t&WK95+!3)D@D60%U({=~yF;`}0hh6m0dB;GhRd<1 zMrNfXC+FwqSFI8$mjLWyY9eYk?;sMrgSSAlror@vz==fowxgKS0s@C9tz62`U#r8M zms7^y&xRkSf<-`C$Q03=K}ivGAgHVia1EC`D1Kako={XCL?)pg5Kr%c<38k&5@5&% z^?Ze@aY|HM1SX@$pAzLzgqA!@d32H)Lm{;2(`@&mG4q`eI#NKym3#)oWh7<9kpW70 z;)cl?wiVdEEf9kl9zEqL#7Yj;TVe#V0)tMG<$#-8UIyzkdI8Q$ya6KM(Ch*?@voP@ zikKCyNItW$`cDOJ)l_M#9ED;Aimo&|pld|_`giq1{_Ai1zjsHOCo<>7S0Xk3YL6E6 zp6%7~sRlJ>zwwIRmWR&F<`YxV29xPtVE0hzRb(IeV69M!#bLF#R{0y7#XpeGSXh?hqk z4koX`4O#;XV_q<~<$i>Rm2_or+tj8?;Mm?> zyH%^S^OqGSv_N4+JN6@xw}OWv9*8U@T|jjPw-^n) z#z90D`6epJzW?gzf&ClT9NwzI$R2I&_tK3p#EPjQ6Pnm|Ks>k?=p~PugiK*#n+dIC zARadntGFH^BM}n;W&TXLdA(=j+WE(jpFB8YDPQwGR9Wc_-w#1X^-9szp?JqW^$TKKp{^i7u=V(1m15 zya+b&`lpBMA*%O(TINq*6k_F{&i#*n{A2w*5oxTfGhBxYKwT{@KB>w)TDwxKPGVYSoQy z-gNJdEz!+dpkfN+tZ|$?wEup3im$eU%EaLC5r*Bw7{v`xDX9Xoe%2v^~D zWZ$m+d-s92kN4?Yqi$VASxQJIq6yCaaMtJFd`-%j4I4J}@891LP!CZObydUOv?bvbqmaRGYd6Xo(?ur%5Sj#^(a8OXCNU~ODBq#6QzUAE6)77d) z3>rJxsMlHuu!82z`arfG?MIFt(B$Ow=|8}#(>*b0&}Y-%#6=UgLr7cHTch56|BlV{ z*J$wEyMG^k*nyLbKa(KmxqE2$d zBBXX97t$5f|1W=_ARTR^0C46)J;1Q=s-g?n1t|K+zw*#se~J=bd=z6S9~6sST%kx2 zE^^V(@A1cB}`t#65s@jK9yLbeOuaYyamK2>TEU(%l zEi64s-UvMMtNQy7fDjPikkgrTCjnhCGBT1KK|B?sqN465Cy8f`Iw*7U;QpB}j7P8Z z@$vqA%Im`hKD}h&!ZHE=Q^$;6^UKNtyKT+tRg=a(7akZ0-o(enVaC83A0-j(6~xy( zNUQT$Lw2dI#m3@{&Ke&>;L24alM@pqzBFrT_OQ5{&A#~4Pe%}{=goOKPsTM1SH?&9=AJE(k3B60XeLEtm2KJBmvs9>3^@*&j@O^%WnJ*Yd@S#*7#?`Q;bb;#@d)-Urj)B}q*}+_mAIJD4*bgogy4 zI(Zb7_9+t(SS$MCrSq`pj06b9fVf`Vkv|I`hU>FJs6XrA45@uvnAo|kj3Hat}DAJ51dju-cGy&f)H`nhxj z9vOiWr*X3hB_xNvfUQDwMQ?9^Y-*WXo?brj@$qoBY@?wJLkCM35HHibm8%RIXG+qo z0~8XiF4)5>#^gzL->24NMoW{U+Xfz+q{w%6?-GJcG z&)#}t-s)c(G;1DGzT!vkP4oEp!<*M)8TcVX28C9tKQ19I`8yT1CNGxAO=L7L85RNm8Jp`U~GtzhNV`^CF_Evn|-LNfQvOa^ z{r!Chj2rvjq*q6d9C`ZmX`CZzE~dxIua6=aePG?@CFF_11R9okKQpjw5D2gQD4E#= z&DK5Eu(7*`kt|OTv28Xwjl)o7S(rF@+_93xC4G!V(j2BU%Ka=?!(LFI)&-gi(bP@nMBp2CxX@ z3cr5nA6LoZ@~Y$?Jj_D{|MbySo?HKS2j9A0st`37BfIM=+Rx!?TX6&Upk4}U)eVZy zxGwrv--|2!S1%|n_ct8@*U#*4+PVL#TNr795g=o5a46t%<7zD1aVRe_=VPi?uO`OG zlJ-y{0g=#3Z}1i%!Nx*@!Rm3N#&#G$6w3ykDZFCUwOh8o`Qb;4zW;vy7oS1&3Un$X zv(UiMgq!gwx;XKai;TP;dsT}opUraP+BI;Wc8S7>^#~ZOu-?OHAea#^kV`NAfnvcR z${BQu)BW`+)7I@j9N_8sY_IOwSr0G-=8(c_qZLmg(jX+CJ&$X?j<6c$-1|3gViZV> zBy3UudCDRjB&G-Mh{_}xr*zK1-eAiiIgvH)U3&N36mzLl-vKiwys&@k4oohD1M$G1 zf-A)c1kJr%WK`UhtN8V>YkDR2LiOlcR`x|PyztQD7lVe(NHI)|p>&EaVihNvvup++ z{Io{Xl2xm}T>10GOR>+59O3DvF%=Y~-$@P)3Z=Vr9Iz8p4{0j^C$%s_e3F*}K0))M z7oUs&EPl@FB&OQ(qNEg1m#e`p`m?YG3b*NGhRZXA%%XC+h1krvUWI3*Yy86GCG{6IgKvuu*L~%Ts-Hhy2S4OTWJd8N zr62zGBT({yThv2*`kcW*!L?hpo-=R0-JE}D-`=>>XL|H`62m_rRg_?2Yd{x>H--Qo zRwhd85n{YGkY|QCJKm7#gqw0dEEDl<=Y@_k^Uc7h_F~k?no9kVva=pe)Sv}BH99|@r z6ZgP1Um3>1NB|aMc@6EZjM=DX(}vC5?78R08OjC4#l_vbd-s!Bv!K`E1}I3gusQ70 z@98B!Eh32`vGG=|T7|(3R(e?aF+Aw(vf5xJKw2F9!s#9eC437RYzWoFs>Or7%2TJ# zv~1pDK>w#x&R=wpn9*vz91~NcMhzG5;esJRP~jj@?z*#pql~HoEY}Cu=R&wtbJX=a zc3tawckN8NO1i)fSS&go`lwF1it+K``ilw|{_FaRA~JFUntxPMe7UQr`nLG$!YNh^ zxW!)-o+*NR&Qp)8Qoj0gd8y=)yA;<``k@4lK#AkHh47WI*Z5r6-=F=-sA0oeRgcv8 zc)$70_rVpaSj7twQ(Pf$*;iD7I004<*^$#j!b%b>K%6NA^c_C1>$mUb0BhwRYw-Su zAKBQdgJ2eoTpJh?t_cbxxJS!Yt%eSN=5)+?f@4s!@0c;Y`t%_sfoRO&Gf|>o)dgu3 zh7vjz=RFnK0|z{$2QHM^+1YQs`MM@OUE|}`|HYRZ)vNpN^cfyTFJKba8XO6seo)ds z|6$IgapUHF^r3rj@S;VFB_4=43W;C1hy*c(mc}6t2-9a4S8vg4F*`vG#Heuj{J9^D zAEEK}Y|yk#|Dl8T?LBbh;GQYdCgTuDc1CQaVep5HdiFxh*-lLwW5D>tn6bV346tj+ zOn_^W$QkiY!f=@sUMyJvad9Q3ew0nHNa8dh58fK{9O=n?Dpj5K&G#C20O!UUz zfBWgj9|)*lpffVD^Km7J5DCp zh>LwVnQ99h>Gx87%LEX`3HG^T$Ic&p`dOWN^>8W|wT}o@0vQ(e46zdrghR)F4?k>{ z2wIj5BtZtSh`$8*!bRgcX$A=$1hgWv4^UvU>Tv#XI&!n`lSp0T<;OigeDA$-L4l)R zc%FK2Y{aS^e4$%9hczqn0j)6k_}R6%0t&DKq_QxVwC<$PyukM6D*Y7>g7o$Yp$iX~qJEO8lQNv8QH9p2@qb z|5QTDrIXx3gpKP*B<=e59xjkj1HjGIy-H)1{CkJ|zpq_>PbKOg*R5m`!1oPJM@PVG zjaOip&BFwvfL97ZJjj3}aKQ978qGxeL$j5X-uS^81-slli04O4n7ulW! zeBDh1$VRh5HAPh<;5Ge|LjUm}Qhoo%n@w z3qs`u1ebTJc!4t4QRiT>fnHH(b>eHSAPf~7iXI?f@{D_ms5c% z!)6B0OgXfxLE)jG^dcJ&=K{9kr7w0hT*4#fz&Qzl!bg!4@=16q@Ky>*^Tl%=$fmil zUNQnWYZdx2fz?u>06G`)L}dKYxaH@?r6MO-5uk%hxN_;QUJW~XhEQs>Tl{vB4733c z)Iw96zqQgOy)Hfe*$9+4jT>zUJ%wq6DrU7x#&N5hYbRpSEK&Nv9S#@sKz)Myq9BUe zXyXAOLl_{71IrjXHFx0;gyn18q?v4p3hV+~(if-Il9`d=&mJ<^C!~u6GC_KPlwc>l zqQM}e(^h~%eYdVX{LA>!5lA5pKwSn+G++6X`ywM13lFqQJNcF_b6~?YG~yYu~xFM~gu#AhVa=~!Yf_0QjPns81Vn;9)Hba|DRg%7kAN#2$d=nD(9Gw8mqX2OBg1hfcZ-F zOd&1f68)P$X4p4-7?!@AJD-i`ja~u8xRbjwvY%yw?e{Fjtj-Y z4@K6^K)(cEP;JRUlb)2gddU(ab(!s~ErbmK+8y@e`*v!r`3i6xW}>5jYU<#&sKDSO zoW_Ym2b{JXpqLi3#{g=}f&<$eJ+_4&n}7X1{^CW6FeQ4Z8NYdMX}5r;EVG#&%5vFhWEah0 zqX=_RW<5xO`bPYq0t@smsxaZa^*|qL5ys#$jg{sNE|PbVwK>BnK{+(Drcc|v?pM4k zQCIm2_yofsl|gXW1pvoDjz(p~P4IpVQDYqz60W#AtsVt7BnRyQGod*3cmol}T7x(# zxOwIB%Yz4zX^h$+2Sf&-hyj&A02(6i)DOS`io^*KCPsis`spB>BKTqVkf10ul4dy2 z+qr~ISmXr&0(?oAVJTRgTWva>yOl%|AeDotDNY(eiZdd&P(7T30!&egziy*^`D0u;=AUH?b@v-fh0c(;ZVt=m&vX8AegEX*mcb zt);06H{;f=S~dBVNfI%JVHZKk?ynFZ0+V(N@4^xn*nzpvCQ6Otx+e z{b}yE>(>7K;9gqYS~Y>rpI5GO%gNlkZPUhIR+5V#EU4_Vg+Fds`*To$Ur0!h0z3PS ztCyDiIA{H@KPTUci>@AFFnX_8vhcw6P3b9jQc~~Nu36jDv4yL9oVc|RoHj*F~R(bvleSd-%6=Vi;5E?N+G z^=i!;kz{kHRoFo;o%7Y!b?ZUiy}S1`Yt=lmMifj3x4`%^-)hmL5jdPD4(z{p=2X(H zn~Q&%mzt6sU8CCWjq6t~USzZ8MO3MXm_&xf96S8msuhP0?j!I(NKoLZ6Gzwoys~k< zI&7-M#>TE#zVflUb*ojW@ZLMqpBXcb1<_|8e^4&4Twq9u(zwMA6G0(M2x|Mwj>3D% zDgmtkq02%fH_vJ{o1?=0$jT~<7FpVr9{=_cC~+FMD4{I8m4f&4@+A|-jNZKAHzGru zb2Gbl>2&hMkt!9!zWC&WrHdDWn9Tp@Mhv+bdm$$+W#Z65&kh-U>FjA+ZuY1rdM4e9 zM|C}Q{K(Kw9nPOQSvfRt{+w?KDgq5HewXCFL5C8PGv}!j$4B(&>c}^fkLJaJ1MtBC zB@rGZbAVkHmDuTgZq#slP8I~q_<{Ye#a^;m%^$z~a$asOTuQ>NggNic@I%KI-T|f9 zW<(Wt*vCHe?9v4bA}dzhx^exqsZ-JE-hKP6@4or6Mzv}wcM|U3OH&GX7s3~3x23kwepq?>!wbbf)bh(e|zk(A@=+n{C7t6?0)x7GV6wK zzWm~yabr#H?#Z{~KYnMLD8s@f$W9unH+d3-P(oKlOD)jk<>Yi~-E#e^RaL`74({IB zr)Be$+wp*%4&p%6e^K-ACMS;_F~Xjg<7UYl-TU!7canr12K5yDD04m78`&^8b^Jur z=H$ABO*Sw3 zzmI9b+#f!iHG>REJd$Tej$HKPyr_tX9a}cN{rVJ?=#2FAzMVR3-LNh^sNAd>@8l-k z7GoP#JMXX`WNL8-KP{wOMhChdf8&OQm4H7T7GVhm&w?mUVU@Q9V~@xcI&>q zd+*%2!`6$8zwFEMt0~pE|LyqrAA18-hmv)${RaL36lmP&VWE{OEZVRMU17-=vjYvf z#S4~VL=ooWyJ7vh=SGZRe$;8%dip1yJIvXS?K|Ss_VoiZ{=bpnyjvTIBDeU{t zWfW_g?~WZUpU%8`;bf~uO@|K{P+)qN z=+$uTcDNS@M6%9+C(PXkn=Zx?f$G$;Jr{Ou*?iy_smFVCXxFoAckAb$gj6hl?A%$C zR+Du%am}h#M-Oc4)xFo^Pd=RT-pnV*jDy?9I`QCvebCGx6?W)AVOZrV1BVaax@_JH zFHH9I_WSzV?|{X%`%XEHenTct>)NRP@v!R@=ZUy1kj^t_XreB$7)eglUFRfsrz_NSad`9D1tX z(=$Jq_2%SBP$k)^Nt0fE1$SP*0AFoD!0j9H^_n*Qb;lk?iP3?Y0kK7`EsShls^OL_ z1atMa!~wUMe25jW>|?Po@6@q_Fv8k`ix)4Z-M-bZT6ML@FPR!iHHp65q**g|19f`O zS`8j++^iK$3(P~U=-Nq1cVLMx#+-kB+BD1@s3bf*+&44`UrLl*=v%Q5^0cho^9L7QxvsV7jxnAgqJ42J$}NTgU2Sm`g-?X zz1h~(y6gE~ZCbiVHfreS?@Pa~q4K+FdbDpRMh|SYxoKl!Vye}MdVki8T^l!=;BoWK z!F%nxS1zjNDG_EQUT@`#OPv*c41PefG+rwl#rOifN<5gr~I z<}!@qrcIkR#a=$DdQGsaMT-`ksZ_BF8-c`f5#W|>R{&FNMcTA$%Xtu#Nb6@#pF`Oe zyBDfsd7#Fe!m=H!)z+>P4dYWy#fPa=uK|se&$(nYvt7J$nPKV55V60>PNl{z=G=Ks zcI?>6$O(;GrScUFp}{O8u-Q|l z9Xq$t)9lR5FFtyI=Z0UsOa@JE?!B}$c>NyTy8p6r#b@u&n6r9~M21mEL`1}$l;rvf zocV#URAljj2;$(k#fj1lO4mS@wD%GR+%gLbjqLc{>wUcg*d}5di^!`<35g3w~e zFUmYRt|9?6^%^v)Q>*6a7bY^Hg8KKYzYq%hphs zeFqL6)~)mLBl|Lv6MH`12S*J&cK{vI9iUfMt5yxku;JKIy}=VL55Vy9^V1njjaqhG zxp?t=6UO^{dN%IZMRv`Y^O!wowB>?A?0EP%-KtfOfJol3f1kHc8NqZCAqIGV@WHIO zYuC4I-BBqlxNXNyKyz?NAoemTcW;M8*R!}9;Y3A|3ca&Q`JknGG1OL7u&9f>V=BJ`6dL~>DZm2G3TFZ4Rb7N1G6?ZMQC1(0}h z!kkmY@#DO?*RI6w*tsh-uv~}Eoe*h^A;R*^@e_u?;63~H*KgUD1|#xr-MSSR5{#Iq zCk(v=1QuHv`X@g*q@wDX^riGDQ6s=_xnv#u)P?for!NU%!C=X~&3ta(xl8={OGl3$ zHJMD;<8Pn6bj8g6p(st*wc))&7Ao;YWWkL_i76Jq!K+uFek)h3KD>W_PR4`pf0%>O zUyBwk(cOYV!cfpil+K*&*tz@0ZM$zJCYU@u4<9?3bT?g8at^E-@D683$HU`?@8{%X zXa4^C@3f#@hxXhts#fjrI`v+d@O1nPrp~TOkhw@$Zs1qr^Vf*g+Bg%^;@-`x8TR*q}ws4Pp$fSHOmM{CX(Gy zrx+-@YXZua0Y5IsUdgqZTeokQdMS4Gij}t9yq#OOUpjQCbI+aty}!R7vgUH^HA}vk zojQ`lbNse$LwbB1@VIz_}`+*V`8W?8hk2MCQ6sY{{l?;UPA9eM9ci zg2YloPVy4dZi&YNuBp%}z#vk_!(wA6)&2LizwY0&7ZhZ~F`Z;RAfOME&i=G42e6qG z3wqkX7$PL-S-pl0H9r2!R{lb}7#Wb%!)R*VzQd(cF?)AzPftt!{PWK#DOWok-+ujN z+>PtXXGa7Y&9H*f(nxl3#s7&)ln9qGqg6CGIr(n6;2^(%GV3>QNxgAP)+KJ5x8Ir8 zwqxfX7cKsI{(_j8b3Ck^oE+=jv?|pi7^02qH>BQ4;RnM2xXOz_QsSp%P2kersu3u0 zz|E9qQo*QVlxNda)W}UvzJ@?kLW75kFSi3tha z`t%F08ZqkeUahM~{x zbI{OXwv4=|`VSGG6s(pk&g~psW9X2luU)&$F5XY` zzwg;7x=o|{wHr0+H+UG+TXZtCG!O#}m&hk5Y#vUX&)c7Vwc@9r+DAtX8rbjZjmvPg z!$yxid;U_x$ZCt0Ee(sVtMJr){Nz_pojB2_S@Y*dKij=`U#qD5=#0vtQB@rh9MI;8 zCr0&q>aBO*>C(O1m^Y_>`0A9_QMKM2Gw%6!XLabxlX+y{RzB&ldtMFYu56qC#OC$y31n?EjZJMhO&-G zyL)%pXP*a_4M0|)Rzug5sNN#fjN}eR^=BOcHz&KgMB{QNXuJIUMm#h6?1c+WYSvt| zVpT}(x>}uX_1ZPZwr+=Zt68sJx1mE{e)(mq)q4EIDUZ;QvVrBW+xYN>7mgl2EYe2~ zQdx)yW1qPc$4cf1uuT0sOKtl0NURc+P)JCo52_2nBUskr*2jt#)rnY4POCXPGb=DC z6tgsZ@IV3dGsZ~5N-^VwcVM~Aauo|OVTfTO*lNqq%plNqnX*BsRD>E4%@MSBS~GJp z{R7J4Q$pqD%yd@D?w&r%&yk3F=uHw;!-%c1(*CouGV~q>qn8gM2~pGxZnimJeY1Jf zw)LC0!PgM|3MWIS!(zFg<>~DMeu+7jJ@5X5RR6NU9v(V@3^#4&gZo~d-Z+?}kkVx7 zlZT{~#4yli#UL3kd9j-ojnJN(asOUGU=R^HBrFOB+ICxBR*uQvgsvz`o$v}|>vg-A znidokB!9YTaoI(M7d^O_-lIXC1J|zUj6OUte(Zp$)@kND zf(P&byc|}@5DTjjjso^{wN{o`$W_YPtoQHV4-5)H25|6DC}GdZp$(qiWI!e$f%}ea zJC`k4vTW%Jt0=-mfDtX2;X{THu>gMEV9(=OnBhX$h$z57UPmBYemU6e=VfL3`T5Z* z`lc)vd0Ko!b9kBvNL#ON9bcL_u}kkK`H5I_b3Hvh@gHRKJ|@RpMD))F@g%v98vo4Xwex-h5$mM zf@_Id;pOK~J0;YhyBJzjEyL3jQ zFuW3jNf=)dX2jt#dgJJh=8wv+_w-@olnfO_uR;v4+X+j--GOf&34a1#Wy+Mx&dJ@i zb617=7-PD26lLQ98322(SRg(FtjH&*Z%;1~R7g6?$nu3CwK{j5k=${FlmXD= zZ{2R#s8OXVRl#T;1QjBR_(xJ$hy>xFgIa>g$UY`$PWSMhR3`j-qmkbX$g7?p3k1Eo zk?=qKu6|y%I{x;}H{N;+Z02emSBdaIR7o0X`IT{ntR4SCSGZ1sa!W6j zK#NQBv*%lKva|UKdwQ8XjRpu!cBd$V?;Z0~w!JvCw)FJW0RIsDxv`dryAg{uUQl2N z7z%EJHIye{9k6V%X5YARqe7(!Pm?zYGJo#e#b3=H|I)-!&%I!EXdc{8L%TG1`ItPt zgpZdQi}jrBgIOImKvO6XHJc!aZ^z&A^70KN6DEe=5?~2;PH7K1a$bI3{LSlCDwHPy z8|T3b+9gZk?LkD28h`l~ z4@Ll%Ty?^$2%iEWgC#XGSbWA21@P1=n3vp@-qLefX35hNXp}C=lM*x)vBCf$Tftw6 zPeN_dAWj0}3Z)q>ZcW9Ew!}Z+r@)Sgf?6IBSuQX(C9W9-utrr4mKz~JNz||{aJ-ly6+iF!u_wCiWdZle^S2G*OjGJ)q(v6|x zUO>@W|J$lw4QutNSFK~U3gZU!iM@DQk2QzbqRZ|m`ZAihB5MoM6K;2J({gB!E`1wD z&wBecjZF|%`a>WNH_eg#`#VNfdZu^R4mGN5`eilqA0-rQ66kYSx2#|Lc>P-4>ecKP zQGV>ezOfh1&_b%E2oHrJ;Sa$Vr!_U{R;Si2hCbfy@yF`CKV_oRE^){O??e@}-%X6~ z)S}4{hO}Yb*;8L7G%k8MBn<=&@@d`gYkJhH+Ot8m_SMUe9s2au^Ct}j_7C5A^Se(z z79V}le_byS9(x-tkpTcq7i!ZCP?RwP4H-L8InmPv8yPTxV8KW1On^#tmM^dcLPm04 zlyurJB!hsaNGxL5u@uG<8KL79{!yGaQ!dj8(bQ?fsHKR0O zD(3=)uq>@ybxP%YR6DR_c;-AJNJ&{GW?W;-&A6|~BxxkQpyk5f>GV>+>W+-R$Wa%G zMT=x8qBK-s z4AT}jpRiEw^6S>EmYmG5KK&R|YkWJ=bcxH#)DTP|Qc{72PSvYZ`}DbUE7q@Hw`|$z zlgE^mjle&UIODx{>(#GwIwt1l^}qkV=%=%%j&paKDr~Hq1VSkr794Zw(%Q`%8MH4y z{utf(^~sZ0FIjXuE)GE?I;{AV!lg@46WK;khscGZ&xNP{yMN?YEpV#aFzezhf{^FR?R%>r$1cJ}e@TSh+f ztcTIFTJ7kl7A<#d--W#}svfJ}!v_y)EY>Fn49xQ|^z8RE>)#WHj-UbwehIfhB_ywL zczF9p)v4d4apMPflTq6oHuB8K{*fjh9dGo)HhmB__nw0v^HFuWD4S$C@=u zPDx`WZDGGoO=Tizpg;n7x%bkKZ`m~B+0n$ju2#ElRFmdAcI;5VPJT(ZWBYe$^0Imj z7?kH>>OEjEs^QTihai;^L>P*=vxDix@Z9O`1l zs!m$>Y&UnKuWz}Ci1fRuvNx%G0Ksu|)%=|N1BVV?xNzaOHEXB7{T7o}86(K#WRElE z%&Atb+W@h}np!8U&p{V~6);^j4Ke|v4l`#>Z_2{pqvBWYNhqc5kXrGW5}Q8h4zfkEdkDJd!3 z0USc%y_=9wp%&fo#%`UxLEoy8_tR3bG1nVBY-Y)S23(qRa}ORoc;fu!Rllrx`<=Jg zgoC9;od@0(r7nB3X?IfSOOCDPCA3S{g+=Vm0t@&-rJMSXrSX(q}(elH0-;uy3YIm333xf}C zsm%iLk;8?Od=Q{4IyecCxe64Wd-Pw0orNchul;YXa@8iki9ZhNkP}?)LRb0hf@?wA zVhDD9;N5j~@p+27(B1V_;dwFBR3-l?S6uraw=Vpou)FZ{pZfdHFZqW$N>BWKMxew2 zH!OzCcBdvU8}QM2dy{pHY33xRPkv6eWDdg>#6yQSwzoGa-sLMeaW7w=tlS(>Pz8q) zlQNV@_AaNtJvHOzO~0UEFHawIV2Xpq7>&RM$6mQGwCQl*Ej{A_EGV0Ga@J`x`FP_F zhZ-j;vzz8YMuy4DQ;ZTtx7GOiduL^5a8oOE2t*1yXlVCY*)!gqem5@OGbqRl;b13$ z4DgKdr6bb{4T00RcP|ZG(R+FV&1|+Aygl>o0W~L`z*Qm5u&S~ATLgNg8 zj`0Ix0F;E@(PU@6{nnc)@d;j`(#+Q2@rEk+v%og#%-%{dpuxnS|X#_M84mM6zG(tTCF z2y^A>7lECaWHuqYcC+}KsS>rSklfNEEHk@pW>tJ>qb+V%0__rj_;ReUe5f8McI(3D`hG-;BEvT0R_g)#O8j5RbgPJTz0V4>M=dIa-JVjzyn_G{v4)b3WQN z;sJ&{CpW81*|O|^0;FIrNpFEMpRfw>9;dEdeg4H4-W>nj{gh;($eFV04fF^8ynMXT z3Nx~DU@f@RnxE(E>yIxWnj%$udV5*0gun@#w%M&&8TZSUEh~s5`-*Ir3D@Bj9=2=G z&QsUo#!eVN<=HX!(o+iv#;t?Y!75UeO$?>I5M~;pYve*|P_f;TS0*r!%{0~@bjR1n zhi@^2XaB%S4AZ>a&`=2^LSwZgU=!PFcT-TgJ^OZ_xElN7#FwUy9g})DneRfv!)RqX zivWql!rr3Y)aT!X6~3b?K< zK2Hs@5&!=i7covv$(D=)gN zunYW2fq(2saSf#({@x={{7d?Kck_?lfpYSPvmiJ)7{I%B{i+owb~op_b1~JcL_qtZ z>S4Pa5*C_t?mV0cWy!PtAntnA%9ZecF$^JdH}hv0ke& z+U&7s&Q_0%f~Lko21C8D@Cxzg&xyq}8Enkv%($CXD^`MFAqoeuis=c$OI-QcT{U=h z>Dq&0m(N`w-X4g~Rx8o8WnJ&)3<}3+(iwj%4gr8$(dC$!@)6ZA(tsV35{Y#uC5ZvV6pTTqFP9jT9&$XSlgjPEII$rwgm3v?B<~-G`cuKKs*Re} zEtEmK;G@_DQnNBmDQpr2kX@WCf?bj3#h(Rvm4+_E_7K9l8YAstO(DSlrx)kBKlfr0 zLla!xEbiYQ%~hYI+DA^6{#jy1pu_<;+Mg?|L`ZOO-BzvU&Yg>9z4yTWYbQ?j?b9be zC->JiYt2?mt2S*k-ku8=%tsYnxNxCYg|Nm=8Ui|OKf+%#r_h!`z#H+mjvYTWYu5Yh zy49;w7lUXqX(K+hRudT+iD@=Udb!}>TCLlD_uY4Pi~aDveK(FDd!kows@=M2^WBuY zZCbU&6nxQwg-*L;(U0@JD}*|%L~l^9Zd%IS z*&lv{^#wCsSeAkUH0U~Q+kgAj*XEquBm4JUKXIaW?>^XjZ{D;q_-jC9q zdGqG^RjAnLv4+fiRv5xz!#)={Nd9{>;r5YZ$3OVsBYywUwQ8iNrF}T-{ha(fNz|?o z25gOP*Y4YIzqXn45A4}{?ZolNdp&`qBW@IQc5rBz8qy!~a@f9%xhSs%_u$kne`2ma*x&A75vD`U-#>!nOR$*oB11g2ew3&PZ&>K6;3 zORdG~t&}P*?7D>aKOv?%`|z3KVvl?f)Gj*Z!dcf}oOj)#=w0>+U7w|ZV6J*E{_yZg z^`*jP^{GhLrZcV?D7}5?DC&3Nsp1P=?fK)T4_#9Fu|$o4>t|Dfx{N^^ZZ>3Q%q!5m z`N`~yH?DSwjC|wSac|C9=u@c%!TCOYd169Bg3jdg$=oI1eZRC_rSPAZF8+4m%3PhV z1wH_WK>V}a-1Bh}e~>kxZJY5!2J8HT=55$frhIue29+m<-s2{Z*5_cENZBHF73@Y! zEks`Fu_(wKyooD{OZ}9MS-FsdZ_$WFDOT#U>)myxNK@s6 zq!@BX6)b#Lcag>viHAqNjArgr~u9ckak_VkYQx>Kidzsh;7HIWs;0c*45alEdF_~GO z=-TP^sc*FI)SZ1s52xk+y>wr{GVTVv(z)1~lXcI}$BR@7T;{ap=4NIF1_t5bC~>b8 z{mra|DJGc<#^K_2ExVF-vw?`&&X3-CJ1{tS9Tu}*Yr^}(N_ZxX^TT(h zhJ}QV9Qz`2jz7f#Q9^AM7#z0rw7Y&pEA-G=B$%nwl9Po4DYa1qgBdJv{K^D~Ket{7 zwJtdu#WqDUxO)@y6$=v~-h~W-L`uAq)UQjoU$$=#uTUPv91UH9iimZO{4oE{zs~Ko8SY*q^e_X5a4i2(lO5$?T!z)t#Xwh>;C&FXf!(?&uu)6C> z2SAJvs?>*@Au`~C2`WO+ zaHlp`G7=1#*EI8&N3V=?@rz=wpDqCBivy!n( z>m@w6Y?{I^YlFh8cJ9%KEm8_0!?*`x&u(LsbY6i$cD+F;A^Jv44=*BH5m5wrKx{lW zzrYZUjXavKAinY|L4YphKCV=fVvo4C@IwhdRmIx z|K9muTKoUe5h!^$>uT$xz`i!~ZtI6wk6i5|^% zqXaDihEN_sU8#iPTr=l~IVj^HAq4s48oqI}Su7vSm|nX^WUboK{hoeW@8!b^7fo{K zjvXhCpQu&4Hpq_Vjup0Y*Cyo^Xh!%8?oK2k?46a;g0LuX{`lSZpjWvH;bLVX@kmfn zX_xk~MoqkZiE}Mmf(oBS7Xs`6L|RJQ&}va#=_u{yA}NJ>D?OrsI0^AJW#lVP@fR4c zb!ue@Ry$PBGltk(-&Fa4SB*-y5Sz?5~AJ;P%89^+21J zZipCQb5^Jh&dY_gj1L6gkl~d>Jbm1CM7ENxT=eSp9ojc-*4!OGA2Cdnz#|w4a&Sk+ zSjhg;<3D5s3LizG`i=1Ahh3U1_LPh=D0o5}nj~>{&9k*|23oHi=WL@F6k70@jN#DhY8!ki4DpV{D;GFOsdp2*n za_%(zGj`BUY&Af0fQ&#ysKt^QtpMLf!lDw730M~M5+o2G#8!k*N}@{3wxTH1pgvtF zz#b8BR6>VDY4GP7y&x<9$tGn+cE*N9^K&!OrE7GAx(Lky>Py=SzWwUU&UI^!=+mQR zjmkTJU&HEyyHG!05qFS5WD&#i86y)sZp6NV%2DDOaS)tVquq-uzl1QQPCh{9P)=_! zdKe&@3fLW$u|h&{YZcqC6t(|XK%w-e|M?@pv?^H#svzp7!>TbE4Q@`G2Tpycs-pVZ z^jK08ARn+2)Z-)^L4n3%A$};auq6^FxQf3y8ab3QWCN*qWpfLJl~~v=UwM>UEKg8Q zA+>Q_;Vax@By=psWF*d~d&fRG?(gN|+y!R3yf^8*(^p$Kwy#j!C!|BBN#mxFI)$a<(wo) zBMu0aW1|fLLIY_Mm}>AS&~qDXs0lHPU57GOffoUeIe_ApEn9CsdgRR6QzJ$WpZ)&( zR478m?$GNcp%0j=cqD;cv}gW7#K`_C;LcareY-eo5mjmjj_hu9fikp_R$vLPlx_(R z3pb^(yHwkjp)Wm3N8leupu~Ajf+L~>3WN!G$r4_|Lb)H?zwf}F-T4+rgC;Gy_JuwM z${j>HUAtoOm8;i6%2yaVY`CXyz`2;S#}Dnto2Ezao^|Wibx7Q0Jk0fk@Mc*rE*W?j zQYT7pPUe#DzrK9^=3~v9Kl#*CLRA3!V4K}(@-W;_P5pVr(wlMDNHP3Gzy9Hss|yw5 zaGW`IZ148Xd3olht=jeI-W>rzWuSEO?f8vr*LLaDnT(O!wrwNDT0ogHLq?AbC|gb; zo)~@{+P60+?T(L^=bl4{rca+iB$O4)7M;5gQ@L`LVI!XL^7S)Yt>@03Ik0ypJ~&+- zf4o+m`b3~cn?HBz=#EXBDBrwUGk9#;cqcx7{!dFLy!wi#uRp>IO_}}G22Gl2Z29a& zMn%=kxieqss)HS`iu_aE3AA7l^1}oB(}gJ1Gng z-%tVM2~P(ZN*RUYa8oc01 z=6v_kkimF?vwQabi!aQ4XBx3ct=Sn*bnE=X5A({GFCP~dN9d1~zD= zdvs4rNb7tv^m7jd}*|^tV7iAyD!{uD6V48gbyGr*U>ktM*4tZB`bg4n$ znvGgX<{$tq@_={J9k@oubqVi$i_(o2&+rz1Rr;ZH1WHGscmzt^#6YPd6*vhLP>T6L zb}pwmXWq;i6W*OMdh|2yT0`vyjb}}s@cP6t>wozr>*lqc*YA)rit&OUA=1Z@3+Et( zAEczLS+(rw-hF+Zej11Ym>do(W+Z?(8*~EVT1}59`@Qt$3}^u_f8Q-zHVqg$T;V%N z2d5=-I_U_uX zZ&z^TYMnZDc(QY6bN)x+VZlewoi%#6rzI!;ylTmjT|4^pAN0dFUp4OD4f$kuGlW*J z`{_$#aJH;bulAO`doUuTCxY7YdvWAuJ(x1-<(wPW%2ln#f6(7WfB@;BHs8t`MBYWDNf#(y2>0@(QoThWl^&%dP&xwtFajk` z<3{fjjhLmoPNH1{JqfpNuvTx`y0unkG}|1_S~Q23O}rI%@$8vSJ-V5^$yEqMK~}qE zWj>fX<+X;<(On@i^78Mer-D;Z+hDQ8^bijfQBF}Y0iKRsx{~pW#N>?|H@?E*<@DT3beLDMYQsNKa ze&cSGoHY|CzOsJVvd$ekuK(?K53;)0V8BH`r6iSCsQ;L6Kmg}rZ^Qw$9ow{uu2SX6 zE?q&wa~IBMXJx$o=3DinYIg70OOunEk(vtX$HtuP(4iBq{_q`5TehU-Ah386(F`$c z5$8Y=IT{UalY92;IdbL7kRe0I^m`hUa0HcnL8le8tIvgR(DN9L!2Od{2%{BVpl~S= zKRiUGpG!yJ|9b>V9B{i#;#_9VsESUf)z{YxfPx6IvWuwK-A+sZyvmg;XE2!JZzbUF z$37wSE&Fvp&iO7eF7CH&TTjHql&w?|h(iqrvkdSRVuXx+G<%%jl&oViA7qFMX}4d$ zbg6p%`ed^edlkt%?1ryO!p%4XuJ@RKxVgpOhzkl0CUpR6XDFYMZ~72g&_1zUeEOe|BT45oN&VpgsmiQUPr zLq|8BKezS5`7P%!*KOGJ$2s$FT)DPk-=Slt&IVPEL>zHdg=$eLiK$2&+IRi(c`Vq; zH-=;*SO}s+2xz4a!+fwg>PDVKAD=-_4aA)C)QQs!pxS3F{P}_6#V_`DW#1Faci~!O zml6`39w7?;;Z?{p{eS<6OD`-Pfs!@?B@Vbj5n24Bj!K*;+_4RTAz@+B|Bt=*0I#Cz z+P_aPX{68+dheakYv>4w6zL)&HUvbmE25$TDgx4bC-hzfq=epUNJ2?S0_i~d>F0dE zH3xltp7(jK@4c?KsDCoXaL#0A&)$1x?zQ&4TD=AH=HH8peUNl->C(kz8a6H!R-t|8 zu6JYNwr|`}@Z`zWi{~;k(x`VH6ckdjWa;fYc05W<1eIunOx-S=$Ob2>mAno0|)lqI(N2f_ikm%RS2zFXTidy zcVgmlvNF$|K80Lh5m2*E!$m9Bu9>@V@#2LzpX}OqfM|H~5$c0ONczFzz{>@T!4I`t zsXvJaOoPUaRQ~?!wrn;T^qG$y9XfhUV=$+sXMpTMLB)2-AWwq;06+jqL_t*U-j$K~ zP`m+k#?D>3Z`-y#=4NzKLc)@Hb7j4t(-!3C?%uxLX0tP0VlWilTeW!MqS&}w+1c4k zSFTd|2guUIyD3pTUftWPp7Wa%SqJcU?qobW}s*W{<|~q%)Lnf5M#!x z4^Gyh(%IE!o8EW!{KdAy!eQMb`n7HTC?oBsxr?3pfQaY%jeh@=A3u1vW3B3MzxhT^ zcHZ!juU@)#qjlA48`iH6t5?qs8j*#X;^pMmlY;8PTHpu)JQGV8Tr#v*^YGXD_9v1ZjtZ@u>VXCHMWRi0Y+^THL6bMi;@?AD`E z-GhhrJ9Qo_p_qC@-IlGUEM30z>o2!$Shr}&;;zlYn>K3LsA-cI2M-ZP47@h@ANipo z5EPWfEl_1J7^cjazI?@s&NZsN)URLSy}J-P!$*yby&c=8dZqR2R+q2az^OIawWeMJ zUL=R@h@Rd1wQ19;NfTg}#$vBu{bBsu=_wDqbHxgRy~mFn7~Zj6$9i=y-@G|<@j~AK zGqLdu`Hvfy_HKgK_`8U=r7Q@TDzO0WKoP$Ni~IH;f4s>+WKYpnGy_F5@L!(+dF}t} zNB`}PWNiPGs|0f!%&Z=>B))67Z8)ULBoJd#J9Bd$LqHb`DsCZ)h2Rs(4XvX5OmPr9Ebz6n^1U{S}!z*D?jg%j#3V0U{S;>?1Fe4dOS2p5g9sG zkduuYxY2BOXw29KMIXkm5ms7piv}Sn&PxQQ#$$sC%1_TEbqN_oq_}~@N)!17c@$F6 zc&K_#Fg;X_$LMzCKYk2+vl?(n0HY*>&B-tDEf$C{VoahwBkNo?M?pR-481SOPw0}k zDmqB-DoK%KKK=;5TnKI)4qXTmt%g=xpB%a=2@8YT` zSIM7w*901#sSV#VmhiY=3Gf_j)CQ-__kEn0B(!Z|^5Xa@1S&~kv2 zk~eC#u~?AS$KRnb5hV|uj`a`lO^CfUwUb|PLIZ@ zG5Hw-OIS3FXgF~NrN{>;*+lp&IICb4q%cktl67A&6vfw*o|$>$RxGZ8Rx)y^!CVgJQcxKv z(DD`CTuI5Kd6@iQ>w+b?d!ze;xu~gbBOx%ZeS7!DMcQ2+d*$)IvJ&L$VJ6 zbdqV~)~Nh7H6?l5qQy`#fHA38Fa}B*ES-+rtz@wk>Y0^*Mq_hIiFi6jdJ$3nuaZE92>{zCtbb!zX}u|pabQTRut{NDn$x6Pv8MKe$|1OILYo;^QquZpLU6n)Ur zMC0fIOss$;8XD?aaqetHs8cmj5`}2+8=)GF6*3qWf#7&?$Rp{r z+Adzz5EMl8xMxrPxP1BYjcb04ueg{%DIe^ygLzKK5+RIT>yrdP=fC$=7Ni2e^df zQ|VFVBWk%;^3MvnAwmF%v3}#WLx&S0+vVc*y86t5&a_@%9^iBHC*L zg2WLMcN0!gz@4uC=k0gWTs%Q(ixFSMO)Df{OL3B%mMfBGtT0j3JKM!R*H^`vpZa` zjvoCeD>HJ#x}@afCe2$695@hs+q7Xr%_oA0#EKX3rk+%4ljo#>U2O*&6AzySjJl z)vQ%3kRC<^EOy}_!lqWoC*0k%b!T4Aqb}W^Ytgm?pbSXCy#uLEhwbdCGe?gcE_6C^ z3GdmbH-#wW(jM2&ty`{LzJeWczzh9QjZv!c4sY~8p2+Kro) ztJNIy<{RJ^VTxckP`hFEsyel5UA}q)q4mluuRhMqTC;XpdPYja#!Uwd9PDtaEmrHf zGf}6bj+F~5->X-j5@pJ9376Hfdd;f1xVT33>p5&zACe`i)o0F}xpd|7*s)`!s-yt3 z`^2de6nuMe&=9;7`@cAF&L^LoJ#)HM_Z~P(da;v1dvWmh+wuDcY2u&3(TmzLQ-3^+ z&)y@&44~lqKc2`EMX=iozJHJrg6aIvpZ;ARe!oi5hoTwyYtFzQp69=22mj}L#iCw# z0~F6)8`ix0>dU*fY;{^Kg^wQh?%L(jg^QKSSD5qNcN>;2HhP>}HmrGb#IQS8uR5P( zzdLN$%P;niy>Z!Q$$zVF@8pDg2Dc;n{E1;5+FUt*p=_zr%YU9dvUd+;000LcLs4@q zzm%xFdG6GxF6|u_i;qb^aZtbHm>Wi{eR!V$+FERik{5i`%Oh0Ig4BDr-W~k>?U)-Z z0&w9b=EtnktXZ~f;_yLKUGUMXzZ?J7^eLa4+%_La;U{muIc>r?(cL|c>o=}oAqC8h zZp(({3x1q5v3Yn?Rbl?nUe6snb*yHc>K87arPcxjiV#E$-s#vGx%TZ>M(o?M-C0;@ z$xQ3pt<%*jm&3}IpZo39Evr_Tb(;PA_ENX3Wbxqhr;d&o+>bOXT9@PPmtR`+!;G>e zOCH+4|CddX8mh4oPn48&@?;dF#+in&QiDg8@!&ydo!TaUUqV2Ae0@VImd{E{1zW|b zjT@NY4HS0~2^O?fC>y$d#}2Ax zEuJ;Y-)LO9Y?(o?FCE~wd-v|qqepq%_GT?wO#Ajbe5D&6K4df)7R;Gj=yZo4je_B= zQE}+(AHHkRx#PU$Yq^0#2Rz@Sbt_^EhYcHX`S`K&Wy?;OI1z;TdFJ#U17Dar^-GS) zd60DEm%T65s-f1yKgfC>FawMxPo5kR5n)m}`t*7J*slj_SFb`SCIREtyxfITr;hvL z^HHyk26#zpHFd(ThTH)x zPX-RaF3QBZ9XPbObF%SK39#ZQZp7cl}XZR|!>jiu7Ugfvx{N0 z);&%8hzLrU11NE^x3ZHG!z))}uf+MO;b|>+1vYBk)~z>Cx~q2m1}$5)vOClUgSlq4 zS{Z4X92I>%`s2?(lki8arewtm=CIO9_me?+l6FwI3o0Dd`uer2*Uz2Yzj?C&h})go zvfbzpKFx=L=9|LVvcFBcPA-kU(4lJ7to8Zx=f{nG3qM7w6Qm?1qK~&|+tE(;J*_6Z zc^eeShsh~%v9T(nPm}Og6wM(tkswT*^~eGH(Oa*-_vO^)&06935fBpEZ}ix2$G=U| zl}VF6ty#a3#$g zjI_nq+`L&{g>#eFx1=uIrQn_$~ph=%L}`|0k?9PwD}u$%MKcc z6#D&I5GlV2jDLye|ImEVm!cW?E6>0)2i%g%Rfl&QNkk-zKNVj14v@i6r#7s=Y|R>A zR}6DnP|gHyX!K?)RTHJ;mR4&t;XX*HqVgv(lld1bM&dy6BOy7r#cD}U3k(d@=?u7q zP^&@7N~)?|uU^BtwcmK_ZQL~!JExh1dfs?Ka0CsQ=OaHiFSvLKzTLPU-K14>*(Dw? zYCp27s#Pf|DOIXh1L#we9|GKgfx#xDkIF*Q*=)a{;^Zj-n-sSV_Y1>D{5)@7>u!C@ zRH%xx%qNpSA33zgPcvt~+`HG|8@G%`BSTima4^;#5FCPkh+eC&Qlolu!}=@MtOH~b z4;%mwK4IM3ZNeKZoqNM(wX_OvPLHTArYgv_*(^Gv2{CaiCbm)STJYY$J@q+(Vl0oM z6l>IB<;p+2dDDi|x7o*GD|`~45LUhtV+Z6FQeWUzhWMM#`v*vs{r`63tlx-fIam40 z!M}mEoTs#`AimPL7op`o{&ub38vnNv;kN^d{wZk~;(yOcu%Sj%kAq|FJ3z9A7GBTbNkBm8>r1VeZx~5C^0}~0!bVq6=lT3M5;V`>~>qP zUcGi~-E!%~347L~6_gossoJ-1Z?{`RLrdMfc^&he(`oP4>-qhMk32|8F_?_!&R@*P z%Cy@oL^t9i0%*IPxD+}U&7G5wn@vD1$f`tnuR$Z-YPCjn8Yvug;){H0nHf~S6auXLBy~>%I6#wfl`&@4P zbg0Ir!W+F+yY?!7pAD;*8eE0T7tZyoP`PPyiH{fR;vWGNJmp7Jc>%1`L^@*cjZ&Iz+N0X;K$}5x#ytqrN zHCC5+c}sjF4$g$_h-Rf80VeG_-Z98;&bLh~v4I3|Cy>3zD=8zI4bS86`UcIke zy4=*zKas9?AO=koXQ`6GhH>m&V#CHdb zezkVpdZZP!lCp|8Ae~{>{CSWzFLdnOyF*0QlgBgX&efTGLqfx5EM2jC@BXgg;e(&= zbLY+-=wy}0;4%46pFKAyxcKnl!&6gJKl$Y2X0_`L95{I3*tZ)t3J0b^UeFWe7jZ!) zcnTTAwpXfE`}r|%zW4gr@LF|0`1Fg%c?C|5cKoFGex5tGX`On70-3me>kU5s?|%K= zvN>}*HfXqZ?fRD8dppRl<)&_aKaMH-R5SyB$r(WL`+xcQWo-Wx@-FLnTrsdu;&0%RKt&WqKm}tRYM9ez z&CSg%UaB-@xnRJ#Wex&PBtxEfd;>ka{lJ)L>SPv_>XEYFZgb}3+2UF^DTAHX+->#4IYOG5w$KaFHfh}ko#LyPY40QH#Z~> zVeYtEXJ_Y9gT}|tSA6-DxESsn&5~O+Ffc&L=_-aoD@`GnJP!K6(tt{w5!V%FK@*uKgaM~St*-U0h@&U*cJ4xvxAxmudQ9@Wb zr4T)%;g51hplFaRc*)QoqE{s;74xrgQl$vmhjA%+S(qImbC^ROU5J0KjtCSGhce<5 zDmkJ_Qi;j1u$Xc#WEkR^jXyDM{Dt3gBRNF*EmB7|+0VQYTdT~j98k2$49G*m^eMkZ zWU>{7ynAQk&-wk#aokGXTNGV+3f($=q0{9);+_)2VEb z5CQ{|^;(Dx6)Bu32LmVE^uv}R+7*&m+`kIsmHAHg(&3VNfziNG2{R0zw_`1lng zOkDn=1b-tiz;h8HN~JR*Ys|$0XxT1+ArK;2VyQlff~+<0%Re}T41^p;ls-i=L3+>} zc=8V}2^r3QxNp94CB|hZIS72YK}RjL;Nn5-mLIVqdgD6w0kLZS$Eiv2)`SrzOi z>__9H_0yRo*wS0CokNit3a~=)OExYFHHiK%u%LwRs2p~wZp^@44u=*NOD3HK?(!Q`}qn&L2lD+S6j*dtmqZ#^$RRoj6T@JLbLK!l@54&IBq1s)hs zCxHr?Z^Z$Fe?WiigXoD*ZH6KYBs_++gR&stPXLvu*n(DE0^|Zi!5V-SfYr!AuqZ0% zC5j}A1HPhQ(l2Nbe8g8Rn0NK;X`u^5tHtk8I1vu#gEC;|Rb&c5} zCQlH}uG$s3DKGO8N;G?cQ9zP&83PA%WrF>gBHEV?n=2g_8!UfS^3B5EunUl<4!waa zva}%mWQTw`R}%7ECIH_a;&=2!j0yqbo;Q2ewX0WTvSiLFoXpV(SFIlDDUwgF$V^F@ zN?8waEyl*ZlUv69MTKV{m!Z0a+;C=;v|@4%X3931K}imF*hsY z;^~ush=S*Ws>BR}-3mYeQc8&z!6yL@(cWB6@I)5)YW0sl{&@KCVWA#q3cXgU34=DA z19c#KB(a~%Y2XvRLSGhCR9d##uADvPv=n+_7x)I{L|`Z=Nlz)BS(ux5{>&LbN6|!G zTvF6jgbt7=g2wa9u3dL;#V}e01Hmj|^!Ykt+Vttuzf(e3Kx#tTRg{{f#Id(yjGdv; z>Z9x@KYZ}P?b~q#ddj$HSmXo<4LGBy%3wiK(bE<9{Wu>H6A0;N^}4bXSZzLur}#_0t4o1%$O`)Uao=JP1o}C}Uv<`5IM5&XnAc+8sN#jTkbRvv~{{7-w;_(6~h@ zSJ)D{UeQ)G1Ao;Sc;%5jBn?R*)C=8dpK#vq?;fuI+Xm z+`Z%bsbAtI=vCE$B{&3P6`d|I^1(u&$^jl|c0x!1VZ8n>JMPus!tPyr$GrZgAht`= z<`K0hMmho~#rIi24OJT73<{{zH}DnQ5!iLPlI}!L7|_e?%qLKY=LcOLEffBlTqtzX zzRtoqja8G|sI?estaXZ(o`QZKc z+%6luv8Rd$hfIPaL5 zXQ_I|m~aHABZBn?S^{%f4KXaY0~0gzM7|dTg�BsIV)zg?=J#iQJ61W6}+#40m{C z=8Ytwb=XvPNpn-&UA=aPLmZJirYj5shL`pabap46SG@nt{La3}~lJnetce z-+#Yva)6O7!-HOXo+wIi5hoi)YVn z+q~(-(ZfOh=F+7?&zv~6W7FE$n>PxaZnLj%@!+7#7te3myy58KL)4QgRl1buocsv9 z2+E&~I<|Jrs?(=V7AqE9vP8+P8`p#1cAI16s+H%@o>YF$+_@dsv~g4L_rShAYgR44 zbpBMu^5uPt1woj5*ZU9e-&?<8`JL+*tqxm`#a6X?&Fm*xYgR1ZwsGCHE0-!%s8~E8 z;KHe>Es^VQM_)y2HTd}j2N%C`{@nJ>8&4lSLdNhCp`mnX*P2ze>(sB&sEKgg&^q+d zYMcJe)HV?j5uG~28O)tCJIFt9v%a#cY4zb#8OBT$j zUa69wuixg#O%9vo)bS%5*R3+>v=u5<5|&1IV}WBKO!(z?K1__?xM8i2!L(w@;^-@v z>(r=vCpLD`{CTO#NmVPBQ=70UsNIEm8`rI0y=Kk5+jkn)tA{QOVrHeJEL*&A|L)yB zTGcNH_O$cyLN0}y=tY$ zW~)}IB5_REoUdItvvpJCv17+Wn3Ry>*Dqh%xiPY7!v-e3E-vo&`t=(cHmFy}UGCcL}ZMDk*2x^HWaytoABtz`@k)v$+ z1x{<>E6?`?2U$0`C}XcCX{+QGMAwBCHfdE?E}U-?PMD9@RahAL{g0!E4DuA_-8dWd z)`%BJ4jbyR7hXAibV%PGYPZd0Em%Ke+LM%ISpUe?E8iMA=;DbZzAD!TFAqQb%U<{o z6m?i*avL&!IA>A@rru9jHTAnsK77~Dpxv}=!P~=!PI~LDkO05ci{?z9@*xO~O8({> zudQCXs8*HA`?v4-aKbxgl{z;yy-!4^13Pz>2?_do@@KY;bf<%ayJV$R*blH<3Q&-F zSIzLh8vFX%RV&RV!%v@nII@4Abt{(_5Agr|wb6SvMZ%aQCnt>^H3}S8+3lmdcYl(a zUaL~o$mPqou3Ce3|M8@Wh55O~0(=_Qt8L9leVmnrA@szF6AvCd0CK==SqZVWDcXu= z;4eG_EHR(0jaL$VN?bdvwx!d*{qXCr`i~q&)tAp*wx|=QSvQvr4Lfn=lCMrvke0Y( z-HKz!jtm(z;6cKz$Jvi3PMCmax7BJtd-FO=>_=JYJJznCIAo80m4R9@wy8aLW3Gpl z3med{zi&`5OLjP1_y_2MGG#(1P8biFJ?_od8bx#%{mM($Ld(+czWHR<4+CH5t9Du& zHfVhQ%;_erJJ2~4*~*nG4;eCK@46LljUA^8EIx0}&mL>u(bLCNM$@DT6Pq<_28@34 z`Q-cYcOO5_c;}sWED-e`?U`%WP-q{eKiCqv?$pVs7X}R0!Ri9pYKUM+Z0HLj84L3g z?^mu|6(14t>QJfPe&hAmUmYEHt5d)3okzX*%Ea+wA0+nf*`Z^?y}PAKl}3#gibKc^ zO~XbF=Fgf51C*4Mc;?ip&fR-JWQ(m*)*DzaVMRQ8VQD-wrhnJ4R;|jx{$G9i`LeCs znzm?GwOr{blRx>;<+^&Azv|QzG5ncLs>SW6+*b*5DLjU^90kd8=sBh11 z9U?HX0@E}moCydnw~es~<)ZfSS+#m4H!Cmk?vecm&cw!#O?`b+OD0@u8_~Uxzj$H z7+$SHkFMRE7CSL7=*kA-WIR4LkIClI6Os7kNAESQS*LHuuB!a}?9>!61sxs-IAV5Q z7%|4K(syXtXwi?~cx>1eQ337nHm+#)QqpC0`H!a1-i7gL9o)MNC4%&w6hDwu>`GDl%B}*c2YBW@-{TsAxU#K;ftXxy2@kvX4q{W+t{M&fBXl*W4L5r?ka`nDB+Thwv zB9iYVn5l(k_YiwZH62)DwaO&eild3zRjF>%0z*Jp?FK5np=QhW7M)+EDs?@1F0Cam z@%AlH{<*fT8&?eN->Dr;_SNXCH)5~1>e|8K(-l0hM;ekg{VJ9E*3X5r1j5eKdozP?1+kiQAN z53myd4aL!zI+JEU-?(crnB$yI8yaXr;{CE^%c2}Ao_TIerd)l|q6INAF$WL-8WtMb zp<`#75uZNc^U;N+O>tatI0-Lee|maqnKI>AYS0J01|J!rkr{+I{7Iw>fR63dDXuUI zT1j7oNWd})dUB}Z`vRx0#il53?@TQAQiDP@FvW5hLQANBzKLG1d<1v3RjXEIrhYkj z!rw1|Dip+I`uY1uMIAeP{u~@O1GnVnI#M3u%p<;fN|zu_yxghG4Bb#XaHLte0{(}J zhq+#l6DPNF{(`yp?%g|l@V7iX5g5W%<4H?+DQG>$A zk6ZwD=h5XwQI)BTU$1) zcje_A+_O9S+=b3PdlP(LqGajh`|)|XS(bwACy&#L2L%QP1?}CjD=Q%h|7Vm|#gh)y zRfGXBsnVFtzQ%IpZztS=LB<)6zX()wMZs_!LRcP%w<2Ol{6>&j;`;*C&JRx1lsm?W z9B&)}a>T*TP3zAbKPoPKcDqKhxCk^WR|E6;EE}>wp=(5hINh4?rp;8oe%m*1VrGaq zJ$y8Z+HajBI^4q~dSIXJamL*5zvZH=6m%NhoF9I;8ykZLOx{6pY^MoY*BiXqOGHW5 z>D><29x&Aoh<@n)^z^jiC4!3;3)s7NPkQ`aI*&#AlaD@Z9}%%;!@6|~7GAx45r?0` z+#GjiR;4PHxoZ?x%gV}n8s@{e{uOF7n5W}W237Q{Xa@eo8F=P^8;%!kk#`kfGp+l$Rr_V~{pH?Vi(IH`QN`$uR-FIZa zL6fI^+cBc+8=rnQ^}~Zi9V?ei<}k#Pzs4 zO>5O#xo%^bnsw1uZ8$m_4diZx6au_~Fpv-8rb+XbS1w#cnU(SjYNFrKpwWB@#>96{ z%8yZAQWg?W*Mi|>Xwf;LizL#L_)01T3y4enkIpc6_H1~3L3v^(rF^Am(45OC6y=@M zb#`)3>9h`vn9>s*;Ei8?JAL7@RV^Ac?K^N#TtcEsYpmI@aYX+?@4ot4+lCEsW$_6u zh0|ktO6L06vx)}?@`gAZ_-i5{xHQ!=aY&T0V3{QxpEN40mDq2?JgnA?diAyIH)6tT z)mgq~UDl(WQ%K;J-8js6xt0_`h`U-|ryC_RqOW6!HNSAO;UJm&<9*ewES{Ffq|%;I5^`xp8ljDBqT^D*#+sTv;#YVwipl`#F=RB1v$AvegS$YU@pgn-Byj( zXeTHV9L5hERo3l(keE26Tf~7QM}tDkg3ma9|%YS%sC3HBa#TXmyy$ZFjG9W zWv7?xr2~&LANOrq@A$14LqMR7h<@Dq+!AF8lMPgMFAvqPTU-*qBK-J}14qb$fr}#5OPV z=VpQ3WQid%o`x6~v-;;LZvm$fOlZ?5@p;4u0V&{3%yIHvqlD2XC#ZFzc z{jbl!v**Vx-XG|-@ER!IqQrr}lGsRZbif-Z>?sU^pD)Qb1Zr45vztUyR)^cy&je85 z<_pPdBpZ~3F%ppuYkl2J`U6Hk$Bole)t~$ikUoutOyj3!y~03&u+%p`G{? zt=2#?RTM^aQhte|3vejV!OKGYx!Kpm2O(W?=Fn*U{R2qP1afJ029O${=Pwj45TJz9 z(5THO{1$Z(=ONOCK%7}q_(nEBI64npX7 zM#j%a!G|BY z^b*tK9hn|yM$Z3NJjwt1i}<&=z*FFUisB+%)v6*t?my+$DQe%Ja^?S%i}Hy5lg5p0 zi+2~a6LXr&X3ft_Nlp%e)h-n#NB|zWJ=qy4x%qiv6)VDBSY29!L1Om2s)QKktT_qs zx5Fw_Hv9M!&?i_dd686<>9yt;SPLGLTO*)YF_b&%!3h2c9Sa=DfI%2SCLd*G1qTH9 z7}SLY`DyV9RPuEieHH2*_@$s)A;k(FWfv4LT}H3h+i@y1Xi!E`XYmAA73AE#mr$W< z6#$=1;feyQB>Yo(tOfb@!U8l#AA{b<*WaNxD|tBVTCJXxSoDBG?hKsr5NZ;I7vv@; zCRC|i2WCSjrDYO(MJxHiry$0MAnmItq9nq;Q z{3Yv2e#x-15}nDp{7phAS#~&7cI&u!&{?I!a;J)vZkdJO zH)fjcDR7G%EYi4@N%+%#J#*_C1`LMs8uR*|KXtN0v!+A4bnOw|Y~HLNHRO(RItCAZ zu~T^X3+>x=?GzCo6D^V3prgFUE^Le!PMv67t?Kap{UT~r-?wuIFS_Wm0FUB2^7WUW zKNlX}tx8kIM!TS;?@3X47p{sO(K^CzR)*QheQ?+YF4*50*sliF#MY$PBo z;Xf+rcK3$$yESUkwr0(jhYh+DeM9=opkX5`E5YE<92af2wlXztKF@G~^wvY$VFs%7;m!v_xRTCdjbt(#dD2wkIi z-O$NByLWeO*q}?}hOKH;9X@zqT=Y#+arN!p!UY zhb}uWUO9IwX22^iPyKqb&TS!5G4|HYHZ9wftJfHx3iNB-D!kgd(CU_giLbpf?we_s zVsF3m?UZlc8C#g24bG5yi2<>3A$^&!lQFjc!)odHGyX++bWz5=iM>PIIYJC-2-_d+@M5CwJzz-;f0bGbGc-5{KrP z0R#Y2rg4W(J1<>38-4f1k)vl!{aR(q(`#HYx30Ht-8!sZLyQVa8WWOtBak3hTpC;M zgx6nw`|GbR#oT)DtEu0PAD^F_jSzyC?OLKz#i(P4@sI1$turMhNorB6TIH;?RKaB7eqkA0 zXQGZ#baBwo5e}7pz@Wjvz^PLwl~-NVUd3PAV>asjOvcvX;pJ-Ae)2d=*5~|5On+Qf zy=TYva-pT$w{3&=*QilrOmj3|yLK&SS_)Q5K7aJ-CojJ;rc~)LyzV#=n;u63F|>Z^ z{25hVcE7>HF$4DR-=94fFPsy_Sji%dn>n4t_u0p6tX-pKv*t~+9%bMnM=LPMct@iG zqsyXHB8r+c4JR{)QKzd}wOamzWOx#dUOVZ7_g@)3resNqSaObxSr}x&a1T1KDmQP? zpdogTy5Ea~lszZq(Sfv3_-R2;jmK=#)v8&&Ws7EyGcvLLiX)}sZ^QZq4CgB?H*V6T zM5$6xK-H^MwLDCr0h)P#(uZTlycQbDsD&a2X<5byfx;M1MyYah1`Zic=^3dU6VatJB`o&u-bKm*#bsN;W=;UT`KU~&^6i;>{*_S| z{rcO_fKYqS)CM`rc(r;%Q1KE#GCXhPsk(mM+KtQ6`IEkS>E#>~{<3~;&J@z2} zZofHmwAj0#W8t_JxVV1(X7g4p1z^E@hwb{+Yxi!PU9@!BsP{iq8H|*OL4$;Rhkupg zmIAaKDfb86H97u%*&4MC20y~}eayuID^z)ung+PCN+2H|rcutZX6Ic$b?(ab%iA_? z{`l)}p-8}M^iGNr2*69mZLm)5uv>3ky%>A-^73U%Uj5)>Y@38dih0#*UnEbsosdkl zbF8wM6CDqe;zMiKHu(kQxiuzVQ%J?~nQ7o7O9)wS$dB4pn3r|__%Ana-P*c!>&KJ7 zmQ;h3Q*hY9XtA|Yir#yIvoQMFb-DD?Wv{&d0U5ZRMAW0?O6fkL5U`BaX>U!)-K;)!E7Z{64txo+aOKaqWr63@+B zwz+cUO5Ya-y2RP$DPckvq{~Pony2H^K+#q-1Ao66c;sqc{8L~Jj2m&msag8#6(HP7p~d>{r1$df-Y!+OO7rH;v3vsC zAiFpQ;*sKV8T5Lqc!WULh)EzX4?jph39&Ky!7by5Q)sn_^EZ7{GIr6XJvaA@FQ;VO zy@zu&U@iV5)IiYVq$dLxPZPydSy)gwapJ`M_}e8aRc6$v|8f8rjp(B_!D94jIY(hP zR5>|0l<1@U3iOjhfbeAcfAGZV)A#O=e(UYYZ@rO|_7F@2=tY$l5Ej=!E^hV@+;{xc@w@j23>-3M;ETDB zpTNK~3I#)PpVPbSHc7P#j4CAzOg`oUOQGn_D64LFZlTRTs5nTBP$7*q&pVwphn3rN zb3r~a?F`oJ<6CI6(v(z0P-}AYtwF)1JRZ@?MQ0{B$m#YAD}VgdxwE$u zMvtE~_4QYuq^2OM%7=yBj*X^ff@DZkUP)T>{;-fh6teFrW%5T*|2w_BMSXpCXW*Fw zZd6Jo4Tz%R>9k$Dcbodvm#BVu57QboZtC@7LpcsAUVOm7L3eIOr>AECO&HBUpc0{F zA4bPwm($s$sBq4mJC!O`p=_h~@d10Vba_Af`1yD1-ZLXER*L*eOb2Kwgm2|46*JS) z*(0>F*uxya@zCMJ0P9N^E&{QlmO88y2Y_8hKNjy7bXq$OCz{YQIs?@R|UAoi4^{ZE?Er6gRxRL+q1a5C0l3*zdEe;zo zd??EBg$oyi%t0gwQsZ3c!R-RyJ}N&F1VmTMlq;Wd^QN_+Kp=(?g4T`aw$9L{Q)h%o^v!Fy@kyHsnS#QP*btf!iGc76(D(jH-0`@+r~L_WWD|+by|yx$*I3%9mrjObDH2<1=!DrIOxg?$*5r zm$`ZQDl&>Rt(B@)fu(2K>8kg>h-{D_*#xzVwxSvMd(OZ!2iyWsddfDUIJ)jeM_aR= zSe`tY_5Dv(nzm{Z-YP9AY3te*`B|i3dayonL!)*bOO^=E&B@-oeFxreZQ6Gx#YyCf zrACKk!-{2rtt@>!JCx=hg4r!#B7jj)LF}SjPIj zZjBo8w_{XpyFsU3{PR!b)OKb+TDfqMDlnvZi?)aq%j1mf)cel-$JT-;SsAH0xsPm? zygfTMxLpM;TeVRIglt|s$6(7_y6|V83gsI#ZLTxwtxmTCof05(*$gUM`n{NIr=q_9 z`b#wPn)Msrx^!;!;`yu)6#Novo>65d5qa9(Tee5(YgQ~(1^BjV(-O6}fH=s9i7tG| zbDlg&PqpRcSo5;UC<43ErcFDQzwd_C%M7l<r}sW1_!P~W{5U#(7UGO2^wID^ZAL>O&!Zm(+p_k3TA zdR;UF$_zYnz>R)JDREw3aAeWk`5o%iYu}*3u8T%muTGuQCr^o9?s6Xc_3*&)KhKACAz}9W z?@a90y2Yiaqg$6R?b@{Q@+C{JUA;Vg;`pTd2_y*oX~BXevwvz-wbI7O4NI0RAz!sc z08z|<_>Jfdx`Nb)-P*MtGje1xU%yp5c9aS&vuE2@#E>Y`ur&x4bX%3{*rELc+qG?5 zug z`k_&!3KaBNxNxC(T8pZS7BA!pYY1CbX7AQ5#*BK&Y&5OezCEmT>FwLMW~M&GNPq{q zSH0zosDlUkv}@C*Ufmm4uFhVtm;yCChabHAZkO<;jC$jeC0)atE|@nrCi>dc3FFce z<4rz3Kh2r9c;TD|70Yg3w|eoS`P>mMCt?7QCT2?QDacIk+_vqgmxc%W`L5o!qh#@t zT=2w+Q+4XrSDZAZVCB;epxytbaf@A&pQ5d32L6UK@W)l;-?01te_duZ{Lhh-y!O1s zq_|^s?^YKSSV&G3;0rk+&L5<}U=5y?{TTb3Qv8uaab8g8DPHSo5rf zM8TVV{Dg(&7{Xrg06{w!6D72Xj-*bI8#%c-0RjFJ$Ia$)W@crFln(V0x4;+DhHkk@ zWrdVrP{MViWqL?L4ZQVh*WtTqGwL^QMzkQ>F@J@qhCj}?6oRLI{=Rfo{40b(66qqm zgXCJ$I)sH$<>y%Z%{WIAg+xvk&!Z;=!Nmh5Q6MD0B*Ri-Vd%RM7pSw+K?**SLd4*( zbnn{Zt7$(qXcCUIjCg>c?Tg^hTCFzDCF=_pgTG-h0OjHxjn{S%{<{~NGsm7v?4p1MwjE*uI)2_n!WYV5v|Dtn?vFP7p^6^?_)A$ znA{A;iOf-4BpEd~k=dmw!FMI_$0Jr%(nZRMw0hi%FgduT9T*?E3cZk7<}-Jnv&DN- zaorTX9@$8|D^CD*G^PJ6wv+{j^j?C$yhrk&s405H7aHR#^h!LCq)o4IQ4GAwSJLsa zbWj4vB%Def6lsuav6Gb?;R0MjZQwHxo3tkPLdNK%T(Z~ch~9G7Xot=;K$$a2 zd@=(PDJHH$++2?licfit>7w#P@#e@A|8y)uLXkCIl}WCWoUGo>Di*_bBG)Y1{x4?W znbWv=vY%oEkEfOjh~|I*18*aiGoERk9!sK%2zU}+@X!j7DO<4sU%4A7u0V!fUA$B< z1s5t7eN$d-Jw9gk`em&*OFC8EOb$d-FKND}8L|5`M z%DV@IK>`3~6w#GD*Rs>kFF?>$O8QCe6sppI@7y^~5tx&IgtP{=d53A%`{1D96D8Q9 zb(=a32y78obWZW&9fzW26Q2H54^?X%c7R6OBu%(BuIxF z0f;Hymi*=vc_b*xqdXPznLSYU-dime1VK{9i#dcj@2AmgO08^Z;rS!X` zOCn*DV2CK02DRbSFQ;nFW=HNk;j|5nDt?`R#--|n-y!sM-L(_mI z9bkCWW#7AP>+Y=^d^GOc*DjtpdYJPBUJ!1KfGv##S%??g;5kjZX)vWJAlV?dDW15ijxzY5JW}ZFRVC;XX?b*8n z%7e&zN9t)=6!ILZc|hf%lvk8TQ-Kd&I9_;p2iPmG91BU@zXgL7UE9SwhOe@n7fFNn zngA(bio78}BDhH6s)XCIyo0=5ygWj|$bP{t{>Kl*>jcf^G9D&vnLn3TQeJQVD4YU7 zAt(jX3l^i=I|UF`cH7w#Q6Q9rL*gP$%UnsKw?&o1?+qEjpMV%)52*MBA^0vwm`N5{9b4luW_0M)>D(>w@|KLW=SxKcU!W~+MMU2 zj=8Ni1RF8~1Y*44g-j=jSdkV=@T3A9f|g7nxWjq${NlM&7tfz_*lqjv?OnWNA=8L> zkff9;s{Bt?(1(JXp!5V_EbPw+rIv@!P z3ET2G=(W=>CG)r!+!KEA%b|nZ5gK7XrJnJFWNK*w=OH8Bmu_du{d-&1uN6_JJhwE> z#Yt|U+!xZ<0NgGkoS$@wHj!!sh$)l~0)h@WoYav(lpvesuF37;Fi9Rjhvm^ytEYZ3 zdG5@g@NeY+o;aQX`cJboS+xBXW`H;G+1g<4KF=wjPgJG*G)eM_zJHk#`XaNIHs2=(gHTqA$_7>}HI-y+Ap? z0CTVw>~cB_A&5oiV<93e0bPU|wmBVEoySJnLZ{0L`~VO_raPU<@weX}(o5^ero*Ix z<#1v$(XrKTmL;;lXL%!Bxmz%Z>GgpLa5_v(0rSF!c4#oY^U((@BO{TDvZ8m`Oez;Y z8b_fdf))BoTp`iWu|fcxJgmY?Go>EajISr}-?4=RFW@O%f(?;@vrs_V@f~>bxPJJ0 z%AuV*bx=JbsWD6o{E5i6VR{gmCyFzVg5+Dr+(6e=WOUrMWt(2B>HT~!FE4@E(8;Y5 zo*Q>Y=FKU7JTi|F5fM?>uakRS21zy;CYu!QqUFN+i>%^#5aR$*0C=(!*~H3(Kf4)} z$VjkNo(Ji$Xt7*`Swc>LvfOm-fG$na^->GNn2Me zd3Yz5$q}K*0*kqkt~38AF@p3(+n+cCV9T?$3Bf0&3}i9BHF9mWN)@i%x{0Fp^2k>+ zGcz}?TT9@3czE-E0|t<;b?c^$RVvoH6BiQ~d!t*oZs9Fku3o)5IWfLpzX5d{wNMml zb^Ptv9XocVrKQxaS$*ix!5-gY8#hKK#N2$8oTxUMM~ogF6y$&X%GI6Qw>zx%=X&&M z)~YpiX9y@)piK>{4sa*lyR&8Mj_fCodOr7D)7EYG?A%VEP@mp?i5bMQw06y^(jorm zPaZ}MojY$TLRg+XdHnqOiz7zA<{LmY928G&+heOaA(Knt(oZ=)oa!mIBb}|pU;*JYwpHIXFPZS**$pFYh+13eJbkQ=_u00 z^zPj|q-2;p`U<5hOp5_vXTDgn*uS6t&CEGJ!>&XfJxrMkUtj-Ue?3&UL7hQ^2kzN* z=-jz;9U|Jd>llF}PpH;R6SvxVa%gWs8}GmQ$2pMHki>6WL|>@y~s^D_UqBq z^vn@M1`iuCZ0UlZKbkrB0#kqf{Md2hxS&^G5v2ESzy164YtcXdS2G|w$1}CD>V|0G zML?I?v2x{yZ@hMJ=N>09;E%JO@7nR|<#Salm;3R%shd_UH@V&0)~*{ja_GGqS1gaR z-W@jRr5F3h-@4(>&v~s+&xi4GX0`j;>0`q?wU4=SzGCST%jeA+{z7kbL-=N@)fh#| z1T@vx&Yl?4y_2gTC(ze8esI6|xSOa>%(;*%@B?UaYR{Na!(G-~yx+#Z*gqlqhSp~N zX53p&OD^6CX{kv+PyUenO_Fv*v4;t%;i9RGeLVT2kErZxay!SpI%?gr#nmg7JGg86XCJ<& z@z_%m<0p<9>BxDkvgC~F(K#nA*{pIbp7q05W8Xkq!xeJomtP_T5H@IzqUq6Vh;`I2 zltoxrl^gHxo*mol*|?!nnKCC29~#=GX=Z$kEHaQw%!!yv3&)Kay70%DN3@?c*@!PL4O(U&d^ZQuU(waa1ULKiKZKXl;pDr+8shpDDDL@dn6f&F`@ zP5nBwM98UQ2S*GUi2kUA>oX8F7%d|b3PhKYN%pbiI!zm)fL}auWa_xr*DPHW#-Bg_;N||$uUok!B+&2k*GKK$yw>1$-MDn_ z)t;RT^B?(|)Z>Q^KyLt#m(5@B#p`ckMt_`^HskXzP_Gq{BRVtBg&mcf9dH~P_qKH_ zUw`RE0-*Bq9)CDu&})PG#ay}Q%+GzZcaQYgn=H0w{xD;6Se{8ONhX~mL-MuR!n*LUyk-J@R~$%4LF^JdehO{2I~ z!{f&Z3!E`+I)p=`Bgav@Dp#yD`-kbRBHGSdwu(1?_=_)eZ`JbnuZM>Y9(>{OzA9C# zy*Xh59Lmo#zwa~n#i`#+<=sk8PCk5a|Cl;;?eaSj<+D?dUi)@$uh6tfr%s(JlrFw?^@_3M-VQLCuU))&xU2Si8@oU zLWS;~+xPF*-IkYEAvBaMD`uVcNm}Zrjhjy%Ki;cX@8v&y|KT^^4IDWNz4UW?;i)rc z!8NO07y#g?T!jiFM!xjxngwr-d)weweD0i|T~^DX)2GQBHE#Snt(w)_uzJP&UrbZL zT{P~i=TApR9a+8qX#F~MRSs+G)@=#&l>P~|jf`{0-?<%DHtfZL{mlVE=yaC69A1y4 zySG34z4m)|`M8$Gt-+k{Ll=AW4f8R8sQm#;YS-S;TG*}F%#A^m!JY}PhS>WvyOF!@1ZU`T0h0U`iG1_0z% z!2jOAA8#&S79ULN{|VV)z59acl)~_qAHwJ@%$3j;!n3J;eO9lOXC@~p{@DJ5QL%T-zCO9B ziD#mYJU{T|+24QPb5LgOSVlBIxCd|1d2{CgxvdWEx_>UYhWHtzTGOEbRxrgVjJodyi-(4`anU6pFpQtsSgNi=xyU^2+lGK7iv zp9tMwq-p=j(EZG5+~A0qnP3so5;fX(ox3`y!=g6CMBjRnc)wYdY7NSlo$<-1(Pz$C zC}pP6hPP@>N^6hKRHI&#mhC#bwPw3pTdRJPl+<)gIng(7wvXrpfyo+zXAPr?b?CXzA19!&|xpZIxB`DP9ogK5O*UQ80giyb)B;1KyAD=u3nFR zd;EkwtJXZ%wadO8JJ2+RNk`j*aK?zLa@T0y+$W$I4m5XT??8?AYS*rA`SSg{bfXKT z>$BwKP5$V!hBfPT?$%9JV9iK>^e`m@ZMs>r7Gf%OtJ=2dfTE4*5o|zxM(qN3gr*jX z6oaojF8*$-4xQ8{U*?m#z!4q0+`kvk+s-kV{#e)FyL(sV=U=6MLknZq8{2p4EdF?y z%fv}pyflUkALayKTDF`wdzQ^wfOu7@?VpW%J-k);o8#ZHVhz<`Xpm(Gw-5&w=G;Z4 zNvvyb=dJ5kI&|)8rzW!27+S8pPpPnkd-tJytsJ8nywO`Ellv8dFwYS(Vq zs-?*{0FUnKHEN|kNM(LwZd^aOKC)5yiuEg2Ik|JsnUkj;CMD5;qyy9IY+7yewh?%Q zFk?K(xbo>m5s}AQOp!cV8g+Q9Rx&WHwoZfkZ925a_|Eb58#Z{5@(@G$`4dr{JG5<9 zy~?6R3+<1yaIRqvC%pgOm0LIWZ`$_h=U?E_&QGb*<#;mT59u@an)cYdqQW~0@1COH zf65FzgBrJF%n;WRXh)TrQdpSGnR2wkUT4IH zgtj3oQtXF1oiV^K@j)^+M;vX@P4d%Hiv3F$h9pQ^8huP%gf6P2o6D4f|zXFG#p}hz~DhcyLUfz;%IJqdiS2acrmeO!NrRu z&E>9LyEgC8;j^ctUL|%c%A^Msqr)AqUR}Lv#TVno8vFv!IMQM6(iblp!%E*%!e)G^yY40BvMw45rhkjz}fH*)gT ztC#E5Z%BhOQHTPSs%oXG`}Q5!xpm9eQ@$#+wO~zc+qDyOu4M&Ye|g z$?4Rlo!f;Mkc11axOQx;PoQ(|to zy-GhoXs{XcrdkSY`kH*`pbSj;Gj|<@n;GC%dz9MDxD~<5a0yXh`BzCi%b&Rr1E^8E zcC-4m-*|ltZl}a!sZD0+Af_-o=W&?Lsx=s6V`HV~;;AY9kcWy#36m>*;JL<4#9-#x z<|c}CrM?gTM~DqE4{P$XA59uP^7HR!_Itk1@xy!Hd2=joIaa6h#*G_@V0E!#7cN|E z+p{NNo|c+IYvgU0MwN|;(i((Q@|x9u+grAWJ$u}}aqaxY3s{Zr+=+{hj^>5nmE-lZIqX>D@P6|+Z5Zf8S=ZtZfF0h^`-Og+ zwrxFiA}T-U(SrGN_*3hSZ5=9i>9S>FVxwJ_LXXYTt5=_0dk#EEO2(jn>HMY4j7Rd) zi)0Xn=Cy5`);RMrj{UOd=BZO% zd-XvDR7=7t&{o~NO$U|6uy+3J!i<#Nn>VCHUjt7Z9tXKfaC+0LbQtTB67Ewip>5|* zA_IUSkLLLo2EFy^ryq^|FL%-@Ksdl-@o_fwsbln5CWkDLWj_lCMeRB-m!oR2!dco zkuKQQiiq^4paP0C6_6IX^j?yXgpvZJm)qa(b0&!(x~uX#3B%l(GpEk^JTKl;jz}6=iv`1wviJ>8Mhva`wfz+}vC^d^SW7 z&dVf$T#!g~nnlzr4jR#0LFi8h!|;+qrDBz%bkOCqkwNz8XTJY-%(+v_YKITbhU^NZ zY&$Qa-9&JEP?fCE{(G=$Pe<@3T1> zNmq{V+4;>EOU!zd2;DcXrMxv`I5*~UZJ!R37Wx%=H?#RAzW9+EW<0niQ`}}jSzWj3bwIr*@jvi(C@GyMp#>B*I z+O&yIhZ67+!F2QY7fs)yV)J|8t^;o78`!Iu5ici38Ssp-JbGQF>NUCz9sc~V5skxZ zy)bS%lJFe?D-id5s#EuO_wc{7_hph=E;uG>@qJ+2SQci-!+mj*7fSxEE1oxRi9Y zO}z%MJu|xJQ^VlUQ9FS%A2eu?J>`18egmK`;AP363QfB8eyV%-$y49ZS*(-Z`*88X zr7a_ybQ>`6OzcJ2zj^Z)w2N%gyjhF74IB08+aG>VfsM!mU@3K0Z}bJ>#!w2G`1ZSV z=Phhpr(UlCkDN!OU@$)Z5+=bT!4vLp7)v|Nh?j;wHDcnFR&DRM z!R^vr;E9(BHgMGRw-UHA831Q&{(8ZpxYuE0F4;*Y>GveyC)Tf>q)$+c!!v+p$ z+o0a!rAuZm|1zLdS%b;?V3*DZ_U>t1Bm7EI(wG-t`1s?G8`iFu3($)5S}PsP1>t|F zi49hn55sK22c)i4(6K`YL3V~1^L%(~1C}%BC(;UG$_><+-e8@BpaOwA(jmJjkqe<` z`2klAki!FCNlaS)!G|SFl%U2UnU{#mC}x{<7BhRMbf(j2wTt4PEIMeZ-C)L!s!P}I zU{zWlp3jKKv837Aw zr!6n-`iIb=Pbk&$6GSml znGK8Q@%Hv*f`PaeckbNTAFtuq_@m9_!-86@s}1_}>#6?!0ZdDb+;D%22(nyI1)(Xj zICo{GXIU&JBHF+eq9qO1fZ75XvOsXjsu^i0DaAZTmzbDHj5O_I<0O-(IXvb2x!+Gs`I-CKjDUy$uZt}FLB}$YS^~|$ufl@X( z%I7KHs1m(n4TNI_EPEBTW9$)(!ni;`kVGNi!s9&e2pyEI2#3XtN{cWPRO3=YY_|v6 zulwPLz_98rzGPdTHy#L#M$9bb+vDp&0#@X!L_XiY=D6#CoAo10V7^x6?nCS#R2MMI zU@~AvjSjmD!KScTG7V(4$xL7#0wRJM;Glf|z?_;NYSg%5;xQkE$;IOnKCGEP2xQ=c zxT`Ql#M&}dv8M(dgedSpM7vBHUKK}FQVLuW0_n8m>q!^S4jIsI=Dhh$TC{{Egoc1! z=r)KF93+RcARSnYY+d4(f%T#SA?z&+Z~+*2Fo8}HY6x4;v=LSE6T2{A+5f<=sSYlZD3oU{-HBpy;XY9h+TB6R?X#9$BG zcVVS5=V1X)k<_6&8f?P`oh8f!HZfTfP8#8;cz&38Ndjln(P| zQ9KZVp>BH~`#j8DKR}|%ya4I&z-mEgMSPiHFC(X5ipfG={J~ve*QidizUb6 z4?zol+O8NNvS$uqeEKn#t zEfba*C+UC<$-oW}mK6zgrW+X$LRAoZM+{hm76R1*pww9j2Gnpt@1!Fr8$=W2b;?>w zLcvq0)LCFxNTUW(KqTEKq!T*lAth0A1WcjGB%uP(Rgc?7dbbgKSnMqY5P?OmX$w?Q zuuRcJ(t!NbR_ZK7K%g0Fgh7v5c+qMWF1#yvPbnNi1w=JQstgkIOOXT`23W%#^Vz5f zO$bdaP&L6>09c3~C5k;H8!ke+9C#y9D;34`ORbb~rqrJvfOzwHU@cE6Lan4YiWYZ= zsJ{r!gbTsGUsXjf0+No`= zn%{o2iiyWefX(5HhYeyi-KkN72OHJDw_4T59~p%BRWOBVFLOpRf&WDpqIb_;TqIt; zbbqVX{T}YhD^n)CE(>iweeea~(CK#V+SRI7je)(ow~7c~_2mjEuU6Z=mbA3E%5 zq|a;CtbMRy{q~LP-&4Kn6ORmJ*OckXB$mo>fPE5ZhO%{6uO{8sqD9|sT{|{xH2L-M z%s}c46bj_HJ^Q0}HwmlKzh}?;YDIjzdKKvrs))vgqB85&e)nMg`VTZ{(4uaGwoUJc zJ9TQ@X#A_=MDszRB~zWYe)rv)_Kg~}Z``m&c=bmI4m@-ABmgjN%A^n9|A2~uZo+;7 zD~bcuBqV$#V#W`4KrgAQEMEwoz`8(&f&0L>V5`DhDx*O^n*kOX9+30=gg_)o3^B7 z+h@&~YIWwxev=z#8&C-Ffw+o#>N8{VOA!rg?mcmE@y0b9mn=N7|A0lj0|{7o{EKaKF}yWpc=83$S4Fm89kQ6*uhm3-PWBjg2S{}7)-A$CN=3m_&B7YJJQP8VWKh-#HNOMRit&|Rsb{8H0jULGV$p0Hf{m+!a zr-oefMq0+cA2$zvbTDG#nzbToHfy?L#}2p;s)6-6s9;QAcJ{C#TGsMf4W(@R&b;<28o4=V(?gL7;vmJ1GU)w*?7R_30K>z^7r zTyL^sxKaJy7TbQ>%sN)q-EQ6P-8*!d>3#e4v%B5hdi7>j5dl)rODZCVV3E`1YcbcZ zUbRKj$fT5H_%~1sHjKfOi7ITFCo?1M=kGTRdGaab*|lob3BUK=pEhrTkrFuYc=qhx zt;^2r)3-Nk?cRO+lkMJJyI~w*-EsL3;|_ZcQ|9MwtyQfWvBB8$ZJ!BLy zK?%1f&}yYE%^F1}5m4A*tPm2~bKszlKl}iX9Kv`?djT1KpzHyV%Rwu&LmK{D2lq+J zpWuj==66!^v@o~AoK!Cr{>s0QveX|J{Lu~k-EytUy!EwTO4ahz(vwE;`ldsEsBB5V zkzeP61hDj~lD|w(m764$`>Hnz?-aaL^#5Jd10n*uyAF|fxLFCJfd^v~UcHv+gz4PX z!ow5dFJo}Q20d&3q)S)I)v9GOTTy4TSgpmXRK0p71+f7PuFa*hvI(iz=VV+z5ViT} z(W5I@u9`Syx-4(SCq3{4f8scG{BV<&_uzgoF)1GLb4XZqrBnPvh-igFkWLOQITs2u>0BXv+OK@X=NB+4vhzX)vmg^msSZRYq>&VqrS3>}LSMB* zT#f+k6$2)6=ste z1;L!m4ER}G<}<<5(^HLRD~xBJ-RWa6`uh20Wo4)>I_4$n+p@ppNy~ig)rpA-7yZkY zvEm|D=B!u(3D2$AOprVyJ(bciq!f@Ph)+P^rTB|1one_hc28<*x{rSV5COs>VDa}0 zggX{S8`{987OQV|owBlC9sk;uxHz9OWih*u-8Gi)NK8>k1Hqdl~>1IiMi-sK17sZ zsDn-)TBYi>^Jg6PTn}EuQ3at<&^dA@6!tN0Q2oRDJsSuLPUR$eaNhY6=* z%gIGURvcy<+;+Xkw?rT+$xeq2+h2V0XQ!u?2=tR6RP6c4mK}4><6pk~rXBnCM8}OB zIr7yJ!&6hQBPM}|m07RTMMp=6R;+>o6&fi>8MCvqd5Ksid0E+I%b@Yi9v8LHS$zU? z^AM1tz94!P*_k<7{9M`ABRA1caQg;V+Oqq=t_znQeRkaVp~J3UO`;uyM-ee4+@Fh3 zHF|#^e>lvXtZXw_s5ekt@sed&JfQu{HkY?wv8*fyO;zLJ|RIXZ0R0No~E=Rcv6%x;!$%WI`IVtP<`RGbv zl>uuiEN!6-#2uP^-)w2ut{t6p>iBW9DC?poBb+zo(x~8yp>X?g(a~lUg5CBrCys=M zRi-# z3BX&a(q%~i*%1oIfz^oyZ2~vyLuIX0fM_BT5me=oLZt$}TEP0YgJklTf4}gXFSw9Q z&|TB@7#Asv6*k&3Bn?W_TG{(X8#Rpw^2{Z&?lDYg5Y3lh^-O`K20a&W)kk#?it ztaV7iucH6&+8)UNQ0|`gi{Kbm)u<0B8Bn)P^VzdLaAs%hkJ=f1@Mj{PWTYf-SiKS> zz*enV>3l59=FRowq%N8_+b^_IL}WwJuwX)|Jw3Quk6nUD9o@HQ%H)Z3VEwucc#hzb z*^OOGjheOME?xjMfwMZTTQ8dVfmd$ou5CY@JFurimrh{i=C$8sBwW0wRcl><|B?lt zn4H;*=FTcsF{E*1Q(hNaIFLk?^6YMBPGZ7^6Z>|)J8_)OV2P+x`^M#qA5NdBw`ZfX z0psj6dIgpZifGej{_L6V+_c?0x5OX*xkLAd^9=eQR<6!SN^a4jh0eztC*#K4ocVJ; z^A8P?Fuo#s7M#OJIasf==Ut0WIJ|%NJ2R$Hc7r;#)2>|k$FwQVoNPc>n0vj>H!vXL zzI*3<`kpH%dEfSp(ff9E>-I2eHXFWOm7I9Bc&So#+q7LUdzLNZ#_sK#FCN(2vuAho zf6Yq1fN%#*3|x&T#a=kNH|otdCbL+mSFavnCoe^xFJ7&d&dbkBhe9~rC-j5@aWf5= zf_ED7LKV5@U>LYl2+<_g?z1OAaxH;|ebBvp4T=I*UCT<#fwh9r#sZYQ1z5=lQYB*` znTryL;?q?fsD}K>Q-OQoYNcAg92%dl3Mr_imKA3XEl({urAy`IzM72EnBV@Rs?u~t zz2uT=t2ekTY=WdvR7&e4eij}D_X}?nUHpxEpkUMY)Z`Aibo zZbNQt$i0!-rDf|Uh77Uz`!8I#u}s+@o{CblOeC+`wd?HOv5oj=sGv-mI{nP~bCKcU z!$*yrG;?Ofu&Nn3xf4e`gMWInkKenW%v>;gcEhmBpMUYytU2>wNMUiA#WJN?0C?&1 zu3hiYy3OFn9>uc!v+vfHDOc{qu_HUTZS%H@(K*QgI-6TRW%~4UXD>7huldYVBVM2R zN%_!9nRz*rMvvLLeW%&m=e^l;KmFk2M%Ai+v2x`nbLJUwJfiePR1QF;!G1lhU5l1O zhCFVtTIa7_SE5YN;h*u{z0=3rTX=pJB4CvlrOco$O(SZJczVS2Pi6-Nmq+<`>ZoV8 z?bu15OrG)PnRDlxM${fT;+ZKkXW@{Vr2{fzh3%@aezS!%*xA^vI*%zKh?A%d#hwa-L*q-Crn)^f-%N)d*_Yc5o^ z@^5JMl(s9BSFa1HP(Smj7ir|q?=V_`pi?bati%_1<6v`<3~8EDrU1KS#}F_3_z!vF>1QAm+7yMK zWrGnxISM3Y%9IB3@WIV)o+C4dFfk&%krq1%#8tYC5Ex7}72IVof#&v zOyMkkmq0@JR<#r$ErZ01teGSihFe*Lq7#X*NCe%`QAsTM(+b{_;?TNLg;5q2kb#oA zF~BTjBOqRsK>k388Uv=q(bfS1T8vl=;xDwGqEz z>^e{rOrJ0vle%bpJ7iCCmqS|$2qja6a@O9JXMgHYtu7!ETJEw}jDdbqN>)}{>O)OJ zO6rKZrkeSuaxO~zf9e4l(Yxvp^=QFg5wv2JfK!fQNM^n8rY2B;Y}snHDq*kA24I?f zeA!+z0wSQ3a9gN&GoNrq0TPH&94aVEty{oMB9;+lSp1*hX$~>&gvnDTO_|CTCsK2_ zmpQOhu-@Ve_OMgO2ksZ(4@$$wqQQ+1juOG;b6Im+*s&DREz=vxH2{iDB2S4AKviAh z#|}$wuVQ7&weQkZgwtTIQXLmf3@=ljOIJE17#AUeAh7+o^7IeD+>;$uQSGppiRdCZ_Z|$n8%rHv=l`{ks){$FBR0O^Fw4yla+cRMId<>d2Gp`Ah$_e z;?a?C$bNx=w1}CE)IHCwD-m4IZSsZzR}yRWePJwux(K_oXqT8wxgLWJ-AXSrA%>^E z@zx{5hF7RmS^7}?V4#U^#Q6z$O+O%{C#*fh5}&%-hbn6tF+Ks^7%mXhjc$zSfGdtz zal6WGFGHZ{6{Cjk6fz}R%QYy>o6uPNjR3X+mx7)`1!w|8qh$R6G73T5SGY{A)V5(E zY716tN{QknQQpe`sgRMr<$aUHSLmJ}nDPa(MESajT>J#g1?+)Ob(Z|17#1?EpD;g-?J z`+mc^Yw0Ni20hA#93JwR6XJzKS)eeN*eDZEalix^4#c?JM-LoWxqMl}h7Ef6>xVY? ziqF4j8rj&VM5*hkX?+I_0@a<4Yz$kB-T@#Z2dDvYAP*++g>$FBTeV{RE3epbm(1Ry zaPL${ocJLThHJ+)3K19V%8{WUj^YGC1c~Uv)D(Wsfzlp|A2>l0nxfKhA0MDr*l(zd zEC*D1OlD#GRGXRcN|lgz&{q+kPhtXYz{WlET1tz1@691@1f?nVUW6M0@MJE;2E@ep z8Lx6Q2U>$`X>7>>uy8j zNCxK*oiKY7n;wlz>0xS3_AKMB~GQm4um4 zF0iJp9$kL)Bj{6UF-G3&Ld{1}Pp%&ued`V%MnqvSB zY>&rv;@HuhTesqN3c&$#5!jmySC6Ao+Ya@Gf4J}{3M%8M#XByW1 zq0o@v(9nvf4sO!3AO&gFb?}B5=`NNl1u3a}uY6qAkw+cR78`X6Zf#I_}^eP#@_X+`ikIhK~y?QJxS?Gar;|5hF^+vTBC8kuWhR zEF<5A&jv=A5h!~bP{k8QNfhHqpx8av&pWmmi}{Wj`&^^Q#(Q>dGnEV&`RwQ>E$%hC z?8su_{gg!_5XpolF1av(@KcnC!zLsc%t&z^X`UoVp*5050wVrDY4lBYPe5FblmIF3p%*)(9PBQ6KE`Hm=D0EkIXOPKIr z89qe~Pzu!e6vND$3U^pS*z@eKjem8)vSkk1Zp$3pzIDv$!)AlSq_eMIz2coI<1D&7 z(LMw=U8o_7gDY4?&o*ldiZXq1tEsbE>ad- zEATeCB^X+j*NIS?5XO8rov)(iA z)mK(8S$sJzM&5$Z(R_vPptlf;Ulo0GHHxnP<~`swY0{*>c@zFc^@&#W*Ihr4{XXvxO)>-_zEDwi++ z!`ihcjvlR9r#9dV0a&_d9_;GIwQDXMJ;XK&qW}Ov07*naRFvm-+{n(ZQmwkdma}5n z(q#)5$3{ojsZ|$v!ZGWf9ov>ITCj7+PQ1C6EL8@?WBHz$mOA&-nIV-b7x(weN=aHc zZ%)d`AwTN{AA__0Y2Vee(^adcJboH<0p<+sZwRZC+{BIy(=@%mXw&3 zot3rns}(V)Ph{ui`uY`T&-47r;|u0|_S2?K-d6AOAr*j<)k_z;Y_=Ucwtl&M1PFODw{Fd%g$o$#$`vaBeKD~y%a?!Nv~lCO z*bB6`Wbwd-^X48qe4tw8O1}PqWC#kv)e1k(=OJhJo~@fs96o$GAz|_S`N>z}>(!1} zw`R@qFP0GrBCJv*kOyycJ#ph z0mGhAAE-z<1b+oh86VH8&zCrJb2k0>!|Ih^n+=`{<;pHvuwd=lHKhXl%LkXk{52^l zar>4Zmn~k9dE-WC#qtC(!28X~<45a8)G&EmJAT@{BWhQR7A>>WQnzp4hEn|Q-Fw$> z{Ju%!CeR_-EznTAfKEsq)b-6b{>99>{c--w`B_8ZXis*Ioj?`g(}G#sUs3iz{pmls4u|R!(I&^@|e?UPl%6wws()16UTJf>39cpuj{=7?H_%a4n64RYdOkAHHAz?(}Iv#rzLOZGEC&cUc&K ztnm6oi6nGAyNiq-xOO=~mv)`V<@EgisI8Mn4PUfyo{z8f!&jbvqHp&V3+9o->w_QJ zu<|RD&H+!l^qn`U?X~!ei>FU{;hAAR2G`a#tB3XL37;y$CB4oLc(V**#h{8ivTy(7 zkxwt3zaY@Z`u-~~5ANIN%O#5gtlqCa@%Xw`U!fs$WZ%BQ9ojjvGyTk_F$4OZJ$Vct zW3y+zH|3dO@ZHH*F1<5pg282DG+2&sz{QIuyULuOUoTra?6F6p&z#M=k^20gfrIv*Zf54tu3au)xWE$Xqxb&t)#ppYLqg`yeE);#(@?(&sSy0Z3(ucDc9h7IFFiW2 zN`+v=GEJLA9zArJYRAXL{!P`I{yuO0Q9-L4%h@) zQoG$a*2s#FedxI{_jm99*pR_|)GoFHbF#i(zQW?~8&E88>Eb1m#*YJgKpjozj0pi$ zBg<&sv}t-=Y|Mq|vCofXs#x{53m2ltj2S~DgCAs9h+QHN!pF}2`^=a!ksnvCT-yBO zkJW0_mX)(?LXrn70nh8NjjPw88%yMeJ9gZ$ZSzx)K00y2ge{wX?EFxtwQE*YZQQ6$ zo7TZWeWfIhv=#Y*IA zK~k8hBCl|D>(Vjmr)`h+>b`yR4`o9`qPA`6_HftzyLY#5dmp5QS`6sb^QCdGyr-&0Cwr(xu~7n+q7>l@&*yDcm@vYyKl#)u&}VPV@BHy`VZfGt696Y z^XJcJDLft>(RRx zLym-+wT_nvr=8QMOm5V${zGls^zG7_2*Slnl{T0~@OOi0s?cAlXmae-mC9`_x|xthkkwEnK2zSPD)JNvSm|PIJ_#6C4Q+vv=jJ#YSmH{NX0sA1>FAA4i!XCSa_!tXv^o3r z?$dYRAY^{*eBez%ZO7bQQ~R!J?On%lBV&~SPFU%cD&2F`v#*XF`}Ckk&+Y%&NHhY2 zwbp%YJpTUKI`8TcjWgq}dLd&(E{ibSjU72IOhk{)gIb;kA53Pa-r^!Qq?b1_vv9hX zW6%g|P_MFr$+qX}c@%Gm5H@b2& z4*#^FUE5YPP3Q5h-}-?pqc6L9`~*zUxh7^NqUb!cIa4A`;;YX>w}9G6ZciRP^8NCa zwJV0zu2?>5!@9ju`@I}yx0g@zHtlna-g$a+y}FI#&L`;fKJ3!5_5k;tI&Ztd&!P9r zfn@2O=a21c@<4}lqcy`jz^`JMcliqO(Xj}|-FjjVSw%4Jc3q0O@W_CH^{ZBW>DlMO z+su>%Gug=NqIv;l%hE@$PmGJJUbzaJ33wtz)v|um77nvD+hDFzvq`OH?e!(fWqMh| zYSg`$kcd^N+T8=SMQBc^g*Uu6+bgh2%?3K1_tk_1D;p6`hecL3qCz9{U!?b*M)x*z zn*8lvzG1a%)oj^97g!?8=u^FV^(&Vy8|=1ohYo+g;zm?_9ff_s*!~ONmhS zd+uq49~YL>OtQj4FQ1Cj!;o=4DA|j2N498@t~c9#idC&!r}=$tbM%&6 zop0S*)o&zSvJeP(>reOHcVA@ns*7jMv?bx4Mr;LMeBt?XCysAewd&;w6YO}h)SJo$ zmxU}PC*y|Kgd_&OfUiXuZny*f=)qt9q-90NozVk#AIB||SA-=-kKvVMu?}tb4d{0xCFOyR9YsS;G`p12n88rBYIQ2NW$zw~w;8Z=n*6NZ{^)3| zTeo5T>Mz-@xR&R6`Sr$@Y*d}v@%<)OBO8-tW-6$av0vM zXdmuD&_iIi7NpYnL5GW_fDWlExK3lB8f<()2WT)Zwj0&0g-0#;WrBGbEdd~*sIQ6D z3_T*s(Lf$Mgm}$`?L%QvG?_8EMCSp$X_wm^SO%bE)6nCvArZ>Gnpg^RQ~b(GBR0v(si{ZvhrQG#tGo+xc_n$VAW+HFJ0c zR8t83N#3+U1SaUmOLe6xbf5&?k*raOq&}>2ULN|?Ywx_(yIbd|?>9X2^a#mH=RR@l zI6ITNfPh1X4z+k7h+HmTy~=o%E?b7LOLT^4yn<@ZMaLc71B$=6gWGpo)0m@75WelS z)WmI@zONV*G-}i+W>-x7d4i-mTmTWO87wwQ!?M#^x>7}Bxu7*)eUWqh>XLa21ViB0 zv11n%VKLA~q)3!^v#5apG0aO?KxEm5`yYI8?S}Q2FJC6Y@9w>OFI~NYIxs5H2*t^q zr8p9pPd|Du=lYdR8@@ZfE9(9Z9f8}3h#Dag;m?g4*|m3ng9SgYW`s6~b2g;PhJjT-msj! zbpDJjH#67a?C{V-8#iu2YQ@r@cWwij7JWPu656EsJ-Fm9Tej@EQzsl*nczK&lI(Co zt7d)n3F{5E=|sB4sNr=>kD9hP#6}u``>L!<2{}1Jeo%v>AR_LfI0#Bh{9)Q6y66rw z8XGD^C%w(MIHcQ^J`Lb0xq{X*%Ab8SGxq!i4Mtgd(0>BvkXuHYm4kQ*rxL6TL<)c_(mQC@O<9#gVL;Lq7TuvxfvP8X>E#}Xib1gpp7K%$V|O~?q{MUzM{_A^1zyHoXaMul|@GZ=6 zw_2+XWs#RM~tlBC=w~HSgaQMdl-=xEl< zh!Nq+Kwp%2k(-E1YYSF_YSuGtjArZ`iIVVe_g5G)hs4s3_f z6V+V9<)q~M>ehRGr5Z<;-Q2Az^9va-M&%`NH zVag`G`PSTpi<;K0*K@#t^VG+n-xjs2=l$&)R<9ACm^|i%m)K5(F$CRUco)A^aJWtHZBaa^ zLerYHzgfSbIEh#9+42Oo$(}3KG10xF>&1MqOTp&fR)cs8Y3Wx1LQR>dcxmm+)Yiqz)hR!oGb69&CQ^u%RQn z^yr0cBZ{F1_U+rgZ7ciCtS3avS{QEmghcQo#!zqpLNKW?kFYppl~=A(r3VHKe7sxF z`|oWTRH+K?a8X>GKI1Lc89n;+@7ZTi>9XagPM?8?f*n!2>$9%X8@~JQyYVl)AUuIA z7yizeR8;pr&;zV973c8>lKvf&(6(RsDv^aMh=u4HmQd$Pxq1ag36D!4juTlmD&9Jb z-DR=Q+8GYiB>q!9EbMKl<@$KT>T;W2L)I*NWgbzd20KJ+8>Y^!W1~{m)QCu#{CZEO zL`Corr9tih3@p2uoDLMGmB3p^^a6q^n2>m&YvR}qmOR{hh`tJO z2t65Tsd$(&8rf4s5aLQsO(}!R5tiT-Z_wxF=D_m?1V{t}J46GGLzeeUV2^?QEnz>E z3b05m6cTBg%W_kB@St9>o|@h|D+x->%_e%g$zt^tk-o>7o}M01vNVmx{4pmx3!w&G z0S^psh%}b2vr!LtNo;c@_=DgPf*|MywS(am-NekS0RLiqY=qOX=h-sTGE0^zjjlP8 z8SF-+tDU;+>|8&V7j$DD{JqWUgKY}V1`y-0m4|LO)QH87+MtlQlb)4dFJHD`!Rl4t z8hm_37e>SmbgvS)pdByFwhT7(QV69mqNu^sT||Zysk3nD9WtO_7b3>kgJzNLi^ zPE?lIwxI9PbBCJI9Q?YASDP!B6Z^Jr_4W5Z5dIBnmXVc(n;iq8QI$a}ZHI1156DaT zOAJLFVt@*BeD7Y_%eghXrPNflx>&BK4qb z-$1?5Y9p*O@aRNZ!nyZaZP{o zt^UJ?hgJ#$*=d!Ze;}j*)E14&qT`P4fxC|5X4%XXAnZ3gSQeKjH#03JF1ljn%2q$z zBni^7ZRq83Vt}J3I8CtK`~s94e4uLP@X7*TvR$Xo|EU7qaJ)T@^h zs#XcJ`S^Gl%*Zo^%b^rRq8aHqnd$IVe*OW(mPA-#cQ7$UAd7*!Y@m^g&XAgz^zG-1 z=gfR>=+ndc4jKUZAaXHc?FSD98=RM$aN&IAiXkrI+TeXkP(}isGmouay1cCPOEEFQ z!4)jUOMu9E@Z30j6?RkS%DPT89I9EYSTV3qr1AzVG-Wl-;sddY>`4MlZdZD03fFMV z$bG53#oON-9vObwnRDU7h4LX)@IZ@=rXaOgG8M}E1QZubNl;s! zxseN?s+f_LPX84zUK}S@=%@fU7>gp<1-TIEAmR`pcG$AhQ$_6(h8>7Snql=NKo2Tz zkk9xF=R!g&n*IFfQ2+^yX0Za0i~e3t#-%f-Duq>X`v$s9RvN4LQPGvQrKj3)5}?ze z6v%3Zo&btd@Xi<%1Pvetn*+5#s0-^WY!2l`L3YlR$)E@=QcE*%ofQRQdtsp1u%zCKyZUiUtVSquZi&K8f)6A_NX0H@zUt zv5-F=bD+Fv7WyXyOU^=UVBlq=RW0vG;y}rUZBN8|iZCO5Be5_5P-ZDY&ILyi$6e49 zcgp{F9mmc50HkCtxLxxW%xw`7@z`UJw~44Zaojiz(r_F)d)6mCyL9f{xXGELM`7c{ zGF|W46Sb{nc#T25d$x&)So_UN#KUk)B92f_^Jabg-hFjy4Cv9VNrU?M<#0JM$x)L~ zxK7=qSI2d19ND2!gBI1RyfAu1PU>}UR{vs(E#g}3Lqd1Ef1jQW8`an4+2?%tal59? z+Bc5u7}?C+()`=YkDs9L3euO7`J!oOL*ob|HkZ%_vj0_b*b-P|I)TECt>Th*yW z-B_%PKm~~`;ONm!9{)B`Q88>4hVZSkKa3?@)`VDG+;jT?7} zY}CGfWXFd5HGFIO6pYdL@7md{N|ix9y4_c+`o`~8({;l3(rmpRlaQ#bo151R@7L|& zwzVSGNzqCa0eZ0d_qZlbdc8}NMx7!XG_M>oX5`a(2!)g~4QcpMvi?8bd5bN@pLg$WS*7wo+Ep{0%#mwB4}e0b4A$W&Xt%6ht^a_2%_Ab# zfAdKa7UdKCgTFrh!A1=pY}AOJ8T-uBx#>5cd=GW*vU~eB)Ek`+Cv6uy6(zT3 zfEChGeW@6oqp**m6;F+#g2Kv4(R8v0#u+%R6F4I?k-Tsp& zo3?55#-#B;jBF7h8-=Z8KI%OW^zOa+#t85U$14W?XhuNW8$LDUEI9?5OcamkKXoN*RxLzeSY$!pHH70Ipd9YM?d4qOv5=8 zD{jEi?aUoJZ18i_CLTC_bkvMV)5ea>%e(<1DM^Mx0k`|XUVVN#dGga$tD?SL{q@pC zG)SJ}WdmG6OAq)m=o;SJWXrkeP0_Kt&&F@qf0&x}?$*gGFJsJ*N1mNF?cmw7V<(TF zHs(QEym7MLNoKHMEyV~-P@>9|!i9eN-mLKP$<4)^= zyKYd`Y!Xo>%lx4upD7<&3DIYG&06MCB~PC?X|Y&l&ibTJpFYeHcIRMlP_o97O_!a~ zr(a)W2)%muXJQ>Xuon)DN$znv)_$|PW{cLf>NP|R+W+w<&PPRMrKRyeG^3f!#OlQl zw#h5d-=|q*!%DU4TuZzvZj!)tnRnt!Y5kgS%LSD|ivx8955i@i056Nrx^?S!x160m)(RgDglwfo_D$@X0Lujd*6UdRwgK_3PJ`u2};e z@ZCGM>THrZY7c$-^+p76aNk}s)_A&I+qda5(|h(CXmjei_v(qL;O7JTgr#KELMxrVli4R=%WYMs1ezq{{FqYX;<%F zJ@60SyKf(gKCmYWP(o7$=67$@PF+TN-@bhj)A#7z51=}*YZti4ph%yhnqf2`Ql{vp zjUy}9jYv#P1f1_{*S1t(vGw15D|iTaOJcg>x2J$$UI8d`@#{lls`Ky4P|~iO53sf1 z5R{ca4wd1rQc_!R|35#0B^q z4sisCWeY|TF&EC+uU(J4_Z|^Vd+BqrM>ptfNIwl;7K7``rMNP+B5>E?GMK$B#xhmI zu3nB4ejFL4$DNX#T)k#39$}4Lu}V0s?6u^SiX}=5udb#e++R{c+}_RWc17)pi;aHo z{h5#hJ?^KOnfRRYxQ-n;+yb|PCJZuR&xH{K*%mBWI(+C;@T8Y6#Ru1`tG8HP{ImL$ zs~U#1zj7(4c7)Z>Kf`VC^$jdhsZvVPRhlH?0;2(EhdNho;-$nrTQ=<6 zu`4zv=A92dVbj=UGDJ3Oc_Mx8NjQ zs~*cq19VMiLx#EltTkxXFc@B5W6~vk0Wed9h`I|7$zNS7Ar=qviy(j0#&0fB(EQd? zG<+|(SCHdx{{M#({pOoRH~;5);I0F1+1nDw6-*AQAu{Y zeJy60#|R3+QJp?5Elqr4usRlPHUp8sF#1!^Wlf%(oD@>2s_6cDq0jBh2xBS38N3E# zcuV-f0|(Auj82P>uiLbRQjZgvrp7%Hl>OZKGfkQf0>{N_MZC`HqPA_%h>z>|@WV{$ zwDcQRbnoB~Maxj)=A>n$i+mguaC%ZwQUU`55&I&4z}nFtd;RQeczn@yL1Lm7`&ijs zycnAj7hkJc3j!}tH!5#bQF5S7ny43TZX?PudQa5$O>Dw;>DG<-Yw78k!~0wnvE4?ytViuAG*>z)?k7Fs())s_!8-0BOurMU_J5K;HR3(ZBloIr3 zFV0Kv@9&#u%Rw=YJ2rHjb&ldC{Ndus06`?a08#hu(yiy_-AA^bjw=&f{@F1j$RCve z)LP(4F?)`QzECc>B2gE>UieOuhXSvWEL-3 z+UCf!yAT%}K5)NNzbf$e8vf-8i)IQCg| zW)P7pIkm8MB3f0uV=(b-eSNj!%ZDF*3@#Ra-OtD0mW%5?q_SSXF3?$`Y;d;di5v0M z=j~^)JF;|?kDUuDANKsn(WnIC;|~o= z5`k8@e`V_pU1tQqJ_h}=g$o{fVyMx#n8EDh?Gu=tlZoyoBL)4-$jmKPytHU<3mNeW z^eL8;od*d4cH!2uvUB~5l|m*Y`=&HVw4%K_Jlu8Do`YLXohuzw`nfUBun`NWhXz-Q zkGaIaC~w`+BmVL~eBOZ(<}V*Q6`E@xt5S#oC4W?#zgv!ipDH)WP|E?LAap_f2}pBM zATp8H)qPT`2FeY}V&TD}OMd+7C@A9gqJN`4+Pl0`_^0kb@%Z~y#zjB>rakamU&-IJ zDSxcmDp(g#AeiOO${g|dquxgSoKHWNJ$fcRvI-*CCTZn zPrZ?HBQ7qqa%JWn))WqxJ0v9Z%!%X34hS?BfBrPG7NYTjCZMV)RTDnZh2zro>#T7+M)b!`B^0P*8C4@e_`mY+sWBmBjSexJqHwF_f45sRQ_+Ru(`wrWx(p zwM#sAIwd*LjD@?>Y}3G%5J9qtBGX6{w z638cnH2g^;6#W2HX_0&n>Q7o%cxc#H@Kb}me5@BXSsoOEyN02hHLhF(S?y`zBhqVS z6=W&@Ybh1POX2zX%Y~vV@|nnSQ?~i*0PRKXY0>$v>VaFn(z~JMXbZETh84ACWjy`( zz?{^i*IpZkPoKoNXlG`c!EVpGcI|rNRg`sek`ps8U&2hTWy_X2|9~Y6=6gAuOBc@b z3#rhcaih%ily6skNyw4TogO~Db?fncJKR|*3unF4y5FN_j7PLBeqdO{2yS~$=G6Bk>6Zuda35~Yv?i+v-zcwWZp;Wgr7W56=fVu-Tri=~wt*2AL{0(W4q`8_Rk zKEBJBF4EgGm(Baow^H?b4IVz?b*ZxwHL4D%EcgiMJHO zZ)^M=|7T~UUX4AsbMuBzXMNPXLx<9(OCSDu-@n0UsP|{sCq;l6eQ5Gzqe&`ov(HAym|iaoaJ*r^$V^L8QC~HEqUEHU*Z^` zIllV*d{-=)Yj$VM|Lnbh(9p(Bn!(MBpF#LsqUspD2tA$@8@+Yoy4kbmG;ZIiM9H$) z;>VmnSGQJ#_MM9yOKA-Oi0JpRzJl9U3njG*5&l%Lvha_$xOTF$fFPs*OlgHC^j68p zkTinTtYi4YVFGu6SI)?}ctkr({d@1osQlt7{5V zVjL|iEjJQqoW6EeZxo&Gs2;fcWZdj+!HhXw@iEcI_C_IF?b`l+nGRl_$sf+@@=)Ig z8b#_{Ik1c4M~#GEeP{XlcCA}}JoBSb!=IWpV}^Hdg}L+Rfh=cEpL%QTnELf18#HJ% z{DtQS_vnsCq=v2T9ryYqEZ1ZL!X1jVg)Kn+mNlz4fA?)bxpF;w^cwKklcGzfM9c_1 z5iZxSUw`=*UyL3zmRcZ=_;T5@$!|<$I+FvrV1V=SXP=FFa>$2oO!g^TcJ{(=v4RG9 zXFNN$cEe`XDu>Pb^s{G&J@vsGQv-u5&Yr*6X!6d5m!+8k8Q_s?>)P+Ou3zt4HmG-x z-u)kY3^DyT-+V)mSr}t%62-_6%ML)pVOzOy-nbd>qCDd!G#a~t-ahYtI&18(A)if~ z92iu7-r}YB(mHVBz>G1YBkR_wRJHoMpU!=8=p&!K^@e|$67!dQ0U1K1gqRzC8>tec zX!j2r9^CLvvGSo0b??`6@RMlKty%Sz$;V%|7eE}bVkPhd!P9J?FJ3Zr>U3Eoh}^|( zwVFSfJ$uy9Cq9}!BcNP`ISW6>;OOLuBX2%8wpQK9>S0x8e)93C;ZJ_}#zf!p73a-g zE)5nkNw>*D1dYLM>o;uMxZbB+`5xW64}AP7m~4(+yLS$L>~SGWqH+gp^OBm+Y6lRW zy>N2Nl@e<6KWSv!Sp5_M82a&wRhK3Rn=adrB$ljH2DDsms)JGBO6Zi5O>5b52F0a= zRJQ7^{PfyO1+S};)Gd{kmd3mZ*{xo$jI@kdOUu-dy>J)&3A{H2kFq7PU#h zw@&M_^!lhPx>NCq4XOqaFtA`s{Lw}WZ>Tnv5O=U+R;pAf?r43+HXwU<${d3jh7Oy} z#@^r`&9@+(8PyQGou~+}z4jWh3j|Yu2nrDI8e-}K{|L5{CL=uy6GVitzztQRbU!~o zQt}FE*g@qfVhprpcuT9cmXZV*j;lQ7jFbeK$=h$g-Jn4OY(RJy1r^Q`&|V$9%b~Rz zEN0EYo+J(DCa;rcR#vvRx3^{q!izU<+<=SMN~A0;&2N5IJnHnoeKTgvXxON+G)>5w z=p(9}HE5EeHDKZq#|`{`lqyA6Ajv`iLaQm7cS%Iij7aIyJe6AAT%v+b@KC;OK{r@n zy&gAAAAR)E`t|Eg-rkH?fzgUcb&8JP>j6<8XFL>BFIM>&@v=TscQ_+cOuLblmXg}_ zo{(b1*ZA}E>-Wkny8q96;AT4i&r|-7KP_m860a6wfZ!m9;5q0dOxK@n2B5%O-~yJ= zAR~x@DxQ`J>Lu7Eb4>z^#f;|3pU)mv4&k?X^3eQ(t=GRh) zPzbc{c&4C|f(7a*f-_jjK3<6uCAikIRow-95`z^kl^;CdGqFt z8aC!WX?R!s!*Hc|*P9aOk0!0duPlAeh3HV~b((sQmJ*W5-mF$DtwgDYYA7Nm={*5$ z;f{GTGduf@=`$uxnS!o5JpgKpW+85CijIHU16mKN#bf?gr4d4zQOLB6EZX=u<(~TV z{%KxC5C45VpqRUVUv__0PKv3@=dH9l$aH1a>UHTl-QMfEqvum1!%G|S@lS++KXYif zKvw(#lo~Q>7XXStqTrVlB&ea)<4@(!q8XSLzT4>q9t!&j`XT8AZ8-QPnjxatp-R^v zO)E=L5h;wHJVkMvL#vfm6YdwLB!4zw1^$G8l5#||L}a33xdRdS1Jr zo>~L7pITX*xruN;U(Q=l0TEu)KH?MQ+E4+cy+F2@2r;IJp>Has=<>IEfRRz!jeKV6 z^G3waHv}^SQs;Vf>FIgt$;l5js|alue=$YJf29X*pCkX3V*h=G@GWU*Df3h|b^sdz z0eiu-u_{;CPf`1mL#~mKaGzSqZ?4EuU28dTM%hE*DVd2yxE-B2 z^hlIcNpZ^LqP?4+gUTa66AjAM9r$r+CTFb_&g@2VRvn_9f43iak-EyOv_<%6SwRSi zRYgm|>d+brdEwAXp*ZqaMF@4#>Ok{VHRQhJtVXJEcTNbNmYveYrhDjNJ^>B2*;*ib z-YPo&RS(E#B>(8%f{_Bj31%UpNg=3gkjMsYQD6S6p8R({|I^VgfLy1a+KV)A@R-;T7L2&*k2%KY+O9>X(cH|my zwk-Aqz%^!Bq{gI{`}vs9wyPJ>R|EhhS02MGsZ2%0EW9W7HPRxqv03)YZk-Sf$y4nY zAQBgfBpa4I6z*K6u-xKW&h#q9i4>VFP1I>b5P@si{0+Ka_Xfj3Uy&ORXfXmPR4O%b zISna1q-OBul34gWlwL&CB|WSxsH9}bFkCbKP{rF9_esUJsKym)`DiL4T&ot)arBvV zpWDIeUSW~k;>|zwCEPxCcgG#omvqvtz|0mmv96BOm$vVa#*H32M{M-KIBbNM&<%U zTU#_*9 z@Q_EcImuHFt!~5-6F{T+%6|xHM1Zail~S!xw9P$X^QA?SsXA-zExf~1Rmc3NTK;Ss zlBXv6=YlkpUYJASyV@N2-vXDCO@% zgz3U=SDjXL`p@>j-=5>vW}b8sS2urOYF=_1J`H_{%Ef)sQnCX|6Vm0jZ}7Gr)=)#c z&HNU)RJwgAy72>e7JP$iRf})6bauj;FR z{XJ-z+?{7_SZF(BQY4Y;JC{Mg*$p9&47#+;+-l7$0^Axiqn0msXIK4KTP^edzgpnG zy)adMfPh2eu=1~k5f*?0iEz(>Df)`d$3nwv<822a7NWlo(L2we7mXU zq|}C2A)V9|?4vBs8vIF)u&7?x8>yK|qf|?O)vm%eX|zT1pQZZe8Kw5t&*j7?k*%Qa(QL;z3dgDQN+3NV%v*O7~ahJ^JzxKEtI5g0a&;Mp9 zDX8Vmlc(U`o$$Xl5E`U{uo}qT0_cCHmD)F;^T;M@?TGx8lbX zhB6oPZk;YFQJ0oxLolb-_@bM}^k${4tT?n*Xgyi*y7ZnhkSw@S z^#32|0X8AEsexEY7O^@lci)PMf_`2)4S^OM{m zz^x6GnzQ-1#`hy1lUlr?IRIPE#N|)CnyQ1uD!u&p3qQ7-i!Q9sKdI8qX;eG@^+~!y zweb(0|8!q9f>gImdvc37~|Dm9} z>J_c`3VTfZNms*YnZ7fJ^!jXn+l}I$%)tD$)UQYCcQVeueQSUIv!2vm_)oX@pV#((|0(;pYO<(N zD41;8NU6%>_m~|@xa~hTO#k~0{~xd6-yXR0VLHE?wIwo&fbuN~7K@UYJY&VjHn?a? zh&u59WA8iwtthVlzwNeHc=X;B6nn>t*s!5u!yYwOj3w3>OJa@Ps4@0fK}1Cn>Agrt zL`5t}mEL*%mc6_8|2Z@FUceZCn!k_0dplhA?at0lIXnBEGiS~@vpMK#Afl42RCV*6 zZFIKPfRVk}dL;UBMYek4$2b4&`qxWwe_j6nR^a~;@iX<|?SKeac!CtXCMECw)$e7i z+HWB%*%HuobWS9-)f4|m?d#xeJ|)qTS8DW1!w1Ln*b``eiHWRbMWny#_X;fo0N*B^ zWQPattQ0^IOF&#cLoF8(`w%ft!Hgu$uHMdOO#;o2D?}H_*3W8y*B8O@e?b8TQChv zv0^~#D7=tx;OyXpRg)fHp(vl0sNQZhpUr0yocz`ls3pMt6JKh!d+|qXN|?g&C;KI9 zM2D0jn%luRZlyl{Tla$PAV1An1K;@*^?u~=U)SzGJ-Hw&aQ8P9!`IR`4W6@CfDS7d_8z4c$ zF^5Q{iTO_-o2359pdZ-kF~MRYvfpG*sLeKA60*mw9#A*7-@}en*Z-S}0s`vgZ^Z@U zD-cUf?4?+}6kug~Bp^=0?8y3mXS_v{%^jDUt2h3 zWTBKN`6!W(DJMgD;e=pz7G-~J-a+~ukpe%VEH3u-d}tZdO)KH4c!1_B+!p@^sNt(2w|tI@uyP@TBJ34kGR85) z;{RSx%YtL=q<}0L%S=mp1?~o75oM$wsj~>wsF5HG;k6I9x5Wk^>^Y+GS$L;v!zeY3uhtq%S5@K6KJ6<< z(K!!@zQA-r$rrYR08eMAHvBo)W zwQeAKd1=e6GR(EU@#fGkLJeAl3mf3~KI5=d$%I)f??tyrYZu8PgEpcGj|92EfdH%9x>%L!7 zxo^8>hqo(W5z?VMso9%w1j6vCzqti)by*7!H4fY&w-g|@@Xxo{Fa}Os(y`M%cF1Dg zS8Z#cCNqqT1SQkVtyl(A<624q^};t#u7op^tqMh)jWz&7d`XdQB-7ltqgG{xsiD5LEyLfRdy3;<*jC$Qr=--FG@fe}#7I(Dt3L(s$jI||T8&?ZS+M)USNeKKcZgQGfh3gdu?ZiR$I z`VoiP4cYRwY=bs>hU3bbq_MorNO|dKOq|yEKs2x|q7lYQ(d=fr!;9p{b#Sc36kt0Z zR7cXL?XhRENKLYDTK#FU2q>^M1!U(CVwC5rNaIFd%r665)KX-Q zB|t?@Nx4CA&3Y@Kt8IsEF*rd!h|3m7>|nrW3Y&!zJ)bOPXJt~Lco6Ca1@f<2kI+cW zB!-L4p%#B`n;RLN{dW{F*^}DGXPl7+oeS!%uddGa9hNAwtCZjhDdeCf#6s>DF9Gse+sbWyUV|K z#&h_k3|zMu+6saPJt3gLmJ|@_nu2N>?;}=eB8`|;Aqs#YKS5jpars;0=q82yZ$gbX zP02rG?89#m8kaEI%yjr=sVd|bdaG$Ob7ZVCgb=~8)>1(2)~tGkf@Tx3kiawTRdZqi zb;Kr~jOIa42q^Gl3NTq`EsRbBhOltW=GUZP8o!ylR*6i*81uS{rJdR0mgcav}DST zq#KEd7LF*tHpFpBF9KVH53kjs#(~l_#48^^W#cSEr7%`}l8O(PFOxIHf-!T34X@SI z4jL!l6)zp@8-5y^wP*y=&;htjcX0f)EqVjB|MLqv#IL1*GXD?_QhtDi*zM_{&{}$z zbR-E1yK61eUE{#5smjr|Z9gqvBP^9H0B6TU5l- z1uO_n`7abuq)@HNCeXLID>WCLvk z;YyJI=x0GU8z030b;h>vpT&$5ttvGvAq$FVPc0-BlvZ1U2{6gm$Z9?=G}%&uf-_($ zer;u7@LVZW;k6VkSj&Jy5R8?YY!S$Y%BlLPSUFSpPD7=R6hsu3;vqZ=0Kq0(vm$a3 z?En%o@CaygPz7mZ%(hhVxl$AEX4H;=GSWz8#8aQ`GDvMMRyBtQRSfZROa0UUnXNRH z&(~ilkh|n0AGXXhA6dswYVC+r$KPwv{9j6eAEAOUq+o_Hwdg@+L*fRjQnmZNP{xkeb2R5(Rxtoq2)S#}%1 zm*Rz;^%)XOpV>$z6S72`7T%X+_*BxUiV+I;Ro3!H0+>pPQno;<)LeqJNU!|!h!!$< zX(JTI!GH`Kwuw^;wQme|6-)~#IfI?EXFFogtTYCEys4%c$$W#FkfUiiNtc!*0ab&I z8d~g}ZyEKFw2H_1gMOzx!zoJoav}lZ(HP;ARaN<;5nNsfiL!968==QJ{iw=GOsZfk<}N1@*u<`QB3Lm|m$@g$j6S1XnnVt7AMJlMiKwnD8Tlp-9RSSQp3j zWP*Y%`Tr;gubPFs9JJ;Bl9CZazz<&8HeBQ=mJtdWR4g2JxaGU)kYzC*fTQHBei_;T zs6bc0NLd*NSp+i$0!E8c1iZN_2q;(RLAi`2GQ$v57TDb?=xRy4{8UvGC<|(*z%Q+;#ND2hG+LP}BC!=#ZPCTOQV68hH`Crj1Bg zM62M5ZHGWQ;c4N1m9}}rbR945dud|jaqtx=*8*`cFm2e5Ft+&o_dVZ9r%Nn9oJfRV zI5iv#B8z-N|40qD9YQ}Dp*dLzkBj8OOe(a6y9d@}Nk|0Dg7*l~)q~O+%9S5TYkGoz zrUPCf$04q354@B%SuIxZ(DO&Dw};IwQ|#yVJEBLblu8|1>eovuwhgCj=MD5xdVde0=Qws7cjpF&H-VK z+m;0=O3wOW$%deCPfmz6sXGr!Y&MPQ;i$cmm$KUxzZd)BQVr+y)nawe)Hwbz4zUx+vzkL)<7U^v?ta8}7U zgfo{Sr2^b%#WD-U{3Q+y7h#bWG)YZ4S0(+ZS%T(@QaVlDJ_f*Dx8HxAZ@tD4!4!d& zY;ZE}*R!TP)awd(&gTXWYSCe@l|L-Hq5F}BA3yhjmo7U0obnB;uQ|FaaCGy-56tTS zDguSF3ofbOq?M_J!sLR`0B}<911AgKSi%@*(i>JUpV_|;VufpN{ZkSFHGI8-DUKR^ zLuG64zUz)fpG*)|n+)vex%WMI_F3mG`)cln4IA3*xpz^c#w-gW*$^(4B*+)YUIvL= z?WIXd3=X_2gw%MarO8YbSVgr_*{N9n#y=h()2Fv=BZG(pGpfK0f4k$&YJe^%!y&Q;`r{TnNG~KxG6HFFw+F}4B1huZ1E~D5)kyM+|w{k z6NpDXo<(q}`If_so-yParn1Og)bYOoaM$q;5Hx=?Uk8YQDS{*!<6OnzG)BctRut9=}l5egu^`fH;JV_vI)5wrJWXcEh6AUwd=q`b{g=m$%t% zuZZv7c1{m!_NV7wIIP=ADL~RgCl1Hu88%&}5YH4Dd)UhYpvWLVVs_w0Ok*GnaY@X; zK=~;vB;xyY;?6(cI)CygtEAwr2Or<5(;l;D&ak4fNEn3|#)A|@0Nbf|5-)Hb@@^5; zhkVx&>4q2a;{|@oPN(pk9#PT~Kar~E*qIbMk6~fc(QK6Bpe^GCqZl<5$XAfM$(5*p(kzA4vZoCjW~uWmi!DGO;Pw_DYXIC^5F?!A*H)F~tPDydPEU|dpC^PxMS2JeB8a63u+aX-s%C(EZ#Cr+hML!my93Hmu$I(4%{teSS&noo&;cH*3(jvF?3$OrEysw&&JZx<`6->B`*7hZfBE1Ofs zj~>{scj<=pyX|wpMHgSzpyS@3O_?z7(}}j9nm2D=JmnpA!l}pi=xG)<5oTN{r3U~R zQDt6I_qym$wm1OppCn5VSyw2aC+5P<2`*@HgV?>KjL;EBcl&n42h+xkOqOl@e)1=V zR|)fK_^J7GKc6viEbQSCM;&8^V$AM2A2)JXapR^xd_Hr=(1CAE`4qLfD^5Sj*tpIt zDFIT)_3J(E-G0vw9T(MRe=PB3{NBA^AMn&;M#gR4s>P&%?@t;z;=>7(Pv~*>oT+zz zF@EB`_uhBq4YxFD*+!{~GqAv0;}}`QB9q3Cd*#W;jZ}G)2K9!${?deDL;4LL|K*oo z-t+qlP=6_G({|R_QHIN2fp(AcFPuAn-h&sP%haLOE*DtAgjG?>+&7030HFW0Kaku~dsV?1)zxtXhSARLp zNT-Thw5V9If;CHf|3kt?`j(q-STt=+IA%wk(7-;geYT?XsmCAvu+OV~9(xpWHr%Pr z^ueDB-CDJ~?1BsW_I~x9Xa4~Wo3v~(WxzXAfcEM0=FFHp>W$|OHy&-;Dp6Lwh_WKd){HPZOsGTQBAnebRA~>mw6v%PhKsGW5t`uJj4Kxo7bN_e&n)6^Wqy;AAH!MX~fJhg|^-E4}W}r#AvMl3_m${#-t(dyk%@! zjnQG>F|$9OI6K;;C5kD7`oEV+vY3Y~!S~8b2R#0W;a1-Hmpk77puemGx~X|ze0Ip` z7hLt_3O>(+bt=w5$&TvqaoWHgg{^SNhZ4I7-)CTwEMKr7WJOxE?%&j^YGGLd)l~{8rRk zh&5_9aqOsO&6^(9^|*tMIw|ZWCXF3oBFHT+*k$)UL{|>L&zSoi*2Q%q9rxR}Y1`K8 zR;*mQaDGw4dPg03Sf-%FwWD2*IHI_?Xvz2Ag~fV>!N6L(a+Rzt`k6=Wxwq>gS|pa|rVArssx3bE3SxWiwGZ;5eGfa*EnT;2-q%YPelLpO z1qB^;-Q6>tE=L~KxJ3)>ffg^AC#^8evrj$!cjsJW*afs`#^k9>zy7LX`Lcr#I~?JE z=fe*rPjtdpEn6n7@H3D8<=$fs>Eb2W%FvrVYbI(p4O+F?=kO!#l13f&*gIXiv10K; z7CX0uX_*qqP{c0?LV+JoMgpn?6sS`aP=eHKi$zgGY29&++O(Uoj_lTb%z$_1fAdYK zsQv**9M^j1-QIcfo^sP_(zad40}jvFEGHt7;33lqM_?M!_lUDVX*xxPCCRG9nq@0Q z#Nj8GFI$}P-Db@iq4q3U8&1nst%dbAty};0aK{4wcJ5zr$`B zI}$O?bX7%RLDWbk4cAL-+!SuyY~_m8c0p03uwJtk%?(rnGhV!+ys)rj-gn=u->?=i zMT1baBJ|I9`#S{3!A8`rIE-@fg_dEcO%gPM4=HthuEZR^_mAK2@l zgGqq-Phm;(zOTPxg=35dDHIf8{6TDGEyP|(Xx#{TRZ!TiIPhFdUH%(z+;#cw2W`(& z0A)9kb8C?|vEshNwmNq?s$%te1iNl|`F@9WZMo}S(fW-HE3$6Y`UAThgU=BkpfQ>2 zJ88U2nT~76yl`PW;~aYAF>Y1zyKfeJ{C=OIAAj)O=W{C8Z|d6ZIu zo6Y(1t3*Xb>H77Vs_GWaTlMYR7yC-gW71CasBR}DU4OuUffGgyd-cWV^r{}h>ux8X zQdyZCGjZz1^2)hi&l^2y+Tu0qx}AP@(H?suz3Fwy`D2F;nmuXU(+}VG`YSK(vfEyW zEMSYDdFGkP6DACPx1SfScU+IN_Bo`B0JP`7_S$Rx-+OoM*WVeH@w6T3xX*q@QHc?b zjT|$+tU5V*;>7XOKds-Qb;pCdgc>z}^VL_Uj2k<+Uq2%r2d|xig3c$N9BI~K=%^88 zm6gLMOd2_PCN)g@R?>8=Sdq&bD15yGJ=sf|DT9Rk8&Q&y0Iw8LAWQad@F$=^-K0Pk zAF_>ytrNvGO#Qqx(9s@r1F4;8{eAW?XwuYm!h0Tfnh|bbSPk|$?6`O;16~{9Lc=Kp z!F@BFunOXKaXMUlaJQ4syY(-IQ}q6Wcm3<0yA30J?w{^EuWS$yr+K!ycGcQ12M>DWii_{P>;k0Tdv`nO%4=>;8?jR_yu_$V z4t?og=bv%*)i>V8gYUoi(!}@PGn{&cU7Ro@C!c*@VbfOeb(@~J?N96Geyt=JINLLi zIpx$=d+(oKzv-UeUGmy(w~ih%0yV;f8U4p=Z|`+tccY@>l{;?zd#~Tmc;~gxKbuk1 zq}83T_BHA?o;YaGBR5=YtX{T9*X|cye|x&P@dbB3Kr4p4{^q|PdDN(1-)P>x!ic0} z4X(WJAqKD^ufKZ7MLj=88oqpSrDIVj*-^J{ zH9fS(<^3sk3O0;M^ULOLbQ`s7jH5`n8Ck{81wCEe`U}e43;xz-3h2HkJwbN}K2&$p zx_{hr)0t=W-2JHTaXVVmtrtP6+R`E4CBbgw)=tL5g$Jt1rW-G?apqyfLk&}gT|d1^ zxkQD(jwF#^n3-6*l8-0>A1#n0N^(>i@Q-?NG$l&cEzOwW5^QrK#R)pPnT{Dr6h&OK z9tJH%MtsAnrRY4DG-~NZ3e#4k*o`MD%Stz_M5(%7v*xae453I=IIK9LjCf^LNxcSk z*hmp7D}ckIXhG5rRXQ=2NgEi+^3^N(j>Zii{PJV>VP)Tc?h?<56=dA_#!cyXyk1dp zvR-{OUSO6{$Br7+b~0YRcC~Gp^&2+Ggo~=ILbSJ|nMB#z<%XMRSigR}q&3>r=%0u1 zJ{zw{mX(IX;lkn)IASt^k#=k|R9#xWZlx8*%CynSXd@Wrf^^b~S0^`Z3a68g{q62A z=6>C3p97wL`L(n!*j%x08O(c8<7Rfapvoy^(+pM)&>2q%a+@(9x3r;6;V?pW%>Oa7 zL_w36vbnE7XpBfLW4&ioEL(cZQEeOkxR+>Acx~oN0~z;^-+6Goqfh{r16>T5Ft{|1 z>I5SXnPcjOD0Q@L04y>ZNL?~UJQB);>N$m@iHoR0BWW5NsTGo%lso&A@n20H)v`eWEq%LBU!%DBVcpM6IVEm5 zR88j1>LbA~rC}D0z_5~`z$wer_ha>qom+r|tHfQn_%nfaNh4yk0-BArP*R35rSk$S zT(7_@Y?8E{a#dTIZa5r_ih`!rPR-p+dKI#MAHjcmK;O5P&zsw6*EVaHEuB7Lv|$z< z*ZrI-qa=x-9y`tU^`Q^W$I01*D0V1eIpSRfdr!xxG751514kLgA_u~X=R#gi4!Pw; z1tD0R4En&1Ndb-2V7G>EeG&>`ui^ozOP4HZ-L`EQp)<>Nws^|kw08Bzb?ZCrvKxzk zw74=E$D#ya2DL#mY`~pxG*Yh+NJQ~Wv;Xja!sg{1A!RUDG;IvK!9Hb77)mBG;gI;u zBNbjs06mQ{zH3B7M)}(H#RcJrU6}Dwrk7f>ba|VdcN5lGd1577O2U-fcB~XA&a)?bBX4G;qqKNmV~A5R5A*KKhJv zd)@RW?KOq;(jssfS15RMm1c>PSQ$GMSL!l`sWZrkpFi$!A{6GXI$;x(u$?I{-LQJ) zidL=K6%^KktO%6QMz5+YU$tsY8>k%(%Ue+6Oj)%%VdB&YAFY(U8n?W>*Nw+_KRv-H zicUKVzj(INIa3pHSx#9DX=Y`4X$bS4a7?_O(@Mc>p_BYkVe?rorq2EiIPN<8>T6ZQ z1qAVcnl?WD`1NO>cuJgK_|_%2+}87)bJJ<}_CMdcVAL?9r~r+Q-{1b{6MLTjX75)g zJoitagqceLqY%QIt;j2i%K)zL1p!o^jE{ zaFO=N3M6NTKgJQFMLuAgY(oP#ulRVBEyR~9Rl??-0U%?rg-vMV+SQlzJk6*sH>x&u z=zQoyFT83Nz%M%)JB-&6@!<_4%Om*80w39>2E=COA&T1wX>c!s!zmO3@h8iSfH?$F zCL~Fb4jh*u2{iZLd;5)nk3M8Hs%NZUdq~goZ@%YojMZOz;UA-4c-9Dq5SjG4|Doeg zJ|)gz>6;f^bKUt@_rjnb#+jkeXFN3{pt0}zkqGh-nGy+s8I5pg4kek;xaq0GhJN(X zM=$p2=g~@Dp!87MsFKQXBqORubGcKu0|X30K!F{P0)V9A)THM5lXyJ-?BD*{>*`BK zO&oL8Er0C&apQL;PyTqqxGQhy_2!e0BvaK_Ty^EU)29uXIb+E5Nq1rh z6N?<#<%kT>DM4^8?5D%N8}c}MVJ{uc_?#f?M&X%X0eA+3ob)2N9Kmac!S2SwHcD+W zcmhej`0Ue1pM6%F_U#rfT-d5vv#!S+P0Fm!isBnVM*&0;2;lm|MHU?Psv~YS=P|bD z@)L#1^|lNT0Cg%JMZ}zTHQb9K3JkC8f+oNii^cx(_+x{|kLy2r?847JA3kw%DjZFR zqt&<&%ou2#J8nGWCL#zPJl9MnW2!C4tntelLT17kK-1E0Sc*u7aJs6TzQP>K1`8mj z3@4eYs`x(#4<0;n^z;4t%=qvETdgF7Gk%7~F8rcLf~UeA{weK=iR4Udl;oApK{ z)ESm2>$|*UvodRJXEn!pIR+M&1Fm z{1K0>U|fh?=R?48*ZD56V-i3(;w=J~GPUP_WxPp~7Vxe0>Nh|T=!NaH6TbfTzk+oL zt`F;S%&U(*xqidOCT(_t2ZSN5ms&Gm(8oufd#)WTHZ9;<_*wA!s~Ho^*R6^~3a3t; zyvOdlo^{q4!$*vsGiT2JhaTGH$YU}_)Q`tMn>2CO^eH7x8=cNvU?AYRmQQYjZH%jgf^-mSwb*#0zxZ2kWj|>eA0x9jcdb& z;mK2{bn3L**=L?Le$3dJpUpb((1W{lJ=Qjh*w$_C)R9xCPVneBsVNefYt<^$KHe-2aywufC#hpEu!c|9RuJqlXM&o$>Dv z-S^fDPw&2SYs9`~Wo7WcaHlxYOS{RX3%+Zx*PhX$V#-b67cdk1nL7FRq zZ07y0olcB=?%CU}y2?u?zxZ_eP1pV5k2hSCj#qs#eexeKx-eoH%jVC&^X$`;r5jo{ zY4r6sb3tU0db61s*&%rlfjj0h{;D+_#!dX>(Z~NDjzo_;=2!#|Hj;7QkA`J~wIKuE z|MOLsFPuNWw6yfWD=)sO*X8rS`zBFY@!%yFty{bhCx1^oc>n7!{&Tku?N%;dx@prU zXiDWoR--b3^+ipFVTrBEQm(&f$x>teh64`RFO9m1Va)t&_Pg)D2f-ZMFp|vZ@p#A&_T%khYlX}x10XZwrK;? zPb^!xBJDVP?YF7=^_TKleh&vNx<)Hhbjgw`YB66_!-5TQ+;zq%%%GbHR6Cciwl;vwEI; za-<|>qjzip`qT$Tb>aL4yLZ}^?Sol$BpYDRLqkPC5K*v65!6{Wp<%~LHQ=0m+0_?c zeDR4VpL|o#vrjnxyz8&JX8n>ME>WmsZc=+3IAbmdb4}XY-qAYsK1jDr0a4lv6-FC-=4!!lh2djPal1~O3ru)~& z9>u8g+{u%^oi??7>sCv?`?m7|`=8PCoRgzPE-EF6pppK->!Q|zexoq($i3Z2CX=kL zy78tPx?Xfa^Hwb}39YWEeC&aHjfHb#ZQC5!`5?w8=tkXWBM(_vjYgwSz4!(q^tH>D z%^fqg@26jdBe6~EHqM$lz5AKxe!JkihAo<(diohgy`~s0CDKMwy+WgY!ICB4?}~Sq ztg=s*~HA9fIQyMgg!-_&p_o)klCV9#8zC*Hsr@ef3AvXPk8AnKz$v z?z)vLBk+EHdezc}w_ktV88_T`ELJB-h`|@O6^mC5?DuZZ^Dh+RLsrbt$`(Zrz?Nxx zrsMwm+tC6eT(Hy5yS6*%;BaC6q#5qmt{rYh;Crv^b?JDDxfx2 ztyS$4afJA%af9&{+HiOpKH zrZ#KWtm=I5fh`a0eAyXi+;`7k%QmcGgERC+gEk_-vt$$pquUU~VDapPVdIB?LQ zA#c3)dI&>AB|?Wh?6t;Z0Eby=1VQ0gr#<#O^zb8)JNqGLmt8t;TDuZ~#FbZH^T~$; zPCx6cF(Zc|USe1gI~Zy%V~p6vWPGIIW~!?EPdDDswta`|Z~hYl6*mft3qBk)u>bT= z&phY+doH`QvUCHDbMX~G0})r*R;PmwG9CO_MRwn1&o=uX9EugOYP!qL9agVg37OC8 zdA4VTd!F3=U$4GgT~)?Y6xVF0c5BwGLT_;g>yg{Gx&d{zExm)%ep3nv#;TxCrc9Xv zzjwj!e{U8RUUl7dc)tGr%UKpyDNDb<;j#-)JO8ZTU42blbfOUjxLC1F9yJ16k0Xvg z4x>I60fB8GN|ZHF{WCmRZvYzn6$wZvBq)iPA{-PKKkz22E5?s>K5@zU~I(0It;`sfDyQ#6C zOyEmz+_c%6Rm*O>$7FY|X0G_&j3r?z^de@4VgZ;wv+eBFx+2F2QKw zE&)(jVe}#x1Z3)Od3Gqt5z&Tiq(Gcd7@l&}xN(SmFTV5=qjAF}ix!KEJw;4v)p6Gj z-+cGA=i9Nz#jcv6Z zWhR4C%9pH&9(VwiFP-nLyrwK4Zvmu_m`7Zi+@?D-%tE@1`|s%viKjF1qMqqon@Q#q+=U zV%7)!-omxFsM(16I1X9MZ@S^yL{-JDH{ZN|{aU6J)z$3mWAuLG)gKlw+*DfD_uco5 z!omX&IaG!>v5w?1L{AaaD>3Orft%HXl!*0FdSttvFoiNU&YU@G`HB@MopjRPhaK_5 zg87c;RaKN3r4_B)cldX;KQ$Qa$o=&Cf#e402!95S`df*}N_UmJ8D*bHG;Hw7?5{x&Ur$>)#+M@ORg^S|fe0$O} z&my4~$%qn0^9M*}YVV?`W}pWMKWtaBOc#wJLxIUnhYHfDNSo=tZ@u>VEuR{N1?>+w zq{n%eAV`Ogmhm8yX}9xErQd#CUS3*Quko+}{Rceph*(mZ)>X$GRj^;@7yG=2C|KDj zV&si51?4zwaCi>Mawua)(4)leJ`zFZ!~Sz^~6)pyzt_rJ?H!v|M|~QLA|hP;zbr!K1Q0Q%NK3@Y#M~Q z{>(E)24h7odF1Kt-Mdd8GiKzIk5JK8M;vv_Kb|)lGy}H9rbbO9M7zbtA7PYL%#>rt zsS?wfjGe+iEqkq*_NOyuKKshwnDjK<_YS`7l7LWu>$lmQ(ZNmpD;8x3i+BhEF?G)@b z_#&*U=oG^>BJwbBnhIQsy<^6Ymv3Z;FyD&d78TIsg+<4$EH5uF zXoTs!S-oNH#m5}`=I3*9+eF)y6vW4*Cgdkd*R5{Uuo)5zY7Sdyp3~up4IExKXg0s6ogkO)vG~Sy*%ELdeNE_=RELLlI{1w zpMU~85CwoOf<|qPykd0j8`V}Axr0-`1q`NwCLAOchyfqUdSRplA!Xzazne({{g&(` zCWD60egrJG33DfFb7Fih8?yjYSfFsp0KkkNp+Ve;ihLN^amEXmG)5@F-3)@~Od1I? zbpV=ilyvn?fBs{S9!qZyQgiZ^v4r(YX$cA^ELAeBAfhqi zctK$Zk{xtqh0cuDb7CbZgrMKDvLb%k?RSb1ztZ3qw9i;D^9)giEjq--0g0VR;lvP$ zyo_oX{G;ZG;3$GUI{dRi=9Y4x!qx z(u@INn|&{-ugyv*v~ckI%Ibk=#J~YibobRUUH?WAH}<8Z>HlRJW5sZCRWY{1>!lWRnteqmf+H z`4Dj2b-oMyFA1Q&NHzK8j1TjdbKKNgvz!{ibt44F`QE`Vz(TA?01fT~Mr6&USmYrJ z*Uu6MA_xde*+4YTrw}y*LZFRs#OX9%s1N~*%t7j=QUV{$;29wZV2!L|${Hq82`QTy zJ$V9O!Yih#K)86MfakcP9x%UPA0G0ko+O|k_*kmJWPvRoLk5;LQXyhIHze91a;_KT zD9d+obRs*)XJCLu2L@TaltnUKJ;{sOBCC)gL0T#~P&HDz3@+kq3p}JS5nXeF*ggY( zB9EYz&|b>QP6d$MI2QOzhSJ`o9|~FQ(nAhR2)HSyBvm=!U7Q`+B8+s(xe7A{N|bV$ zF0gtjCZjl86bi@-GGzHzmV9jnQ6ZbWA?ibm65=D!iiQy!AtI+X3};dU739&}MZj@G z`yd1q*zqVJD5Ji4s?L`W^eIpf+<{w#fd*~_wK10jD&SAC#1R>^0#Ghhivb4^OTi=Y ztPF~vn)N}fWk6(13I%QyERvfTbMrXXk)k;$Pl20)#vqi^(H39?{{S3`0meffU?yz8 zV2@y{2v{s!65^u@%tDVPrR>dFAcVb+r+_(0#^bCyQi>p&Y8pq_Utvn|a>ekI_83LL zV&QFbbv%au%-adsZ~n$hi7DyN)m+r43P-g zStm{7qe_*-K!Pz6`4mPmCuLc|4vA`yPRd_?1Tvrs>NGe=KMz7cfgO(mU|Alx)d>pA z0+(Qpu$f?kU>+Eu;0)kX-@QEL8A)*qO_PQ1lLRCL5k!bBCLf?D*P9<#SfTP1UKR8d z)ek`*1w*PDTr&3;E|N?|m;hq3F%B{lJVB9Ak2uuZpkw=~*&YyS^6%j#C z&g2%WdXk}>lR>_v7`A}F#E>Zi07C+SK?{V{Uoz?S2qx9TO9|o#xL1`A)uVoZDS^8X zN<9lw%cI2Te5B^K4U!@jCmsOjFbW zfL;DWCy6Yw32unU1E^Ao1^*Oed!r^=jMgpt)T@|3vN))9(*=2WthB~wb0B}A2&1E~})Q<|zRQIrlP zMVUHRko?_@Z)74oF!}IT5Et=`YKE}RLS+7AYefw$nVm@OR4V{jZV8hO*0S)b&Pl@q zDho|aOWu~G&k+;o71@f!Ng=t(4W~RMWwAVvucKrnQXLc*vxRaiTP2|qHPmHsE-5wX z)I}XX=x$ZVi4Zh@+fhI>iTU!j{47ktW+C&D!AU@Y z+Dd_dziMq3ofLnyK+{nUv_K6=gSRww8 z;)2V70=1a}0mogN-|qI=Ho@5}0&ixwsjB9eIp$WsNcbywXp;Vl)Y~U%f?5R>_+=Cb zIPPCYry$jDM1h>%Mi#men22u+B|kyK1C0pTR#9{qq-SD*frj3XO8$5LTH|$282+Q= z!DT>!T1bI_@)|;Vs)j%6~^~66mLL*>3qn za2`;gc2XeVxNGNo-F{o8gd`&77HbQ+$igl$iuJ~}4vP^~fyKB3e;1-_BC5WqFmuEh zoz2n308cM&=GwXSRW3I88&IH@QeZ2Nrk1ubXx8>e0a=jzaRA1N)SJj)I*2nQz*=k& zm8mqhxWw9Aq5Y3p@4RBkk24Lf0}9ke3IxDi8{g`7+$N>}hqDY>NEBI#mY0Y{qL{^r zb+#BnV|<6_)eKIKLc(La!jzgOZ#DQGbf@F*A-u*n$;FYTq^=9=3R-C`mBb-gUp@ied#= zP|6+!S>JTx#v12GTLwUU>xeCH2Im0)OB}D4(OP>5Rd~Z)}4eAn5pr$BLOMqMUF-5~nDJo!_$4}P4P(MI_K?^{cb!Rdm zpFK{11o74!%DN1e4W%3pH_>e8l$~wMA9YF$S@DEC!%uN^L`ohijgFUc#krl7%TdzH zJ|AqvhmU++q9oS|NTV|oNdcaY)guy>t0$*YF24B%y(Dl;2c0>`T+rp`u5!GmPL?Z0 zyxDIZws<~UsN|D-YU{MW^_5hHtp;Sv7bwqm>GZ&iYg?7!hKKjKwA0>4$i^^K$F^JS z)A_(27sRcCq!o49jhEM{HBK~htr~%hoF8wFA8&zQ5Pn+<=-!^|LfKA2SCP+(SHO|a zjU9H>7pFe1-?sc9x1Xj!t&wqW9)WZxLmUcXvHO&`qQ~*Q%(R8ulUT|ih3x+ z=C6G73dc2(1QUP&B0*sV2jmGT@Du^0;2)>jtWR)Q-GYl0P|9M5l(pG0KeyN!fEZn( zB$ONovNMZ1*OxrbKM~;O;`>GRSSwcBz-5GyeA*G;Psw@BO}+JG*@K4dKdrc<3Dy$@QaHI68;Jb?C`)X9Y&$j)`%usOtA%$ z?4|_Iz;1Y+5rdOsGgG~<2fYmJmWRbD2AfEUnFm2(-j)~#SkoB+HPQ;pr?8h8fgAmZ zItn0D4HYks3)3w}okk;meu7V0lIUCPVx=!7u^t;#J`$^f)KUI#&R<=q2pzAkw`{(y z{}wm1IjEwt=lS|m^UdweXI*nAiY(Mk8mmgtrQ%7_D&HH9lX~RADI^yKB zD?)|Y_fm8933C2f3P>*!8&~NhydpWOj%7PWB9#d{>^Lu5P09`S&yn4W0^t5xg8oJ~ zcX;3yFp|#1l$ow0oeV&vZ=q$z5|>CvG2j3uETdY@=E}?B(#M$d$sW>ZYoI z%1YXyPV`Y1D33}u3WHTuxhs2g=W0UzRkmsvP>;*u-z$J@K?hZH*|`AGg^JPVa-nPR z+a6_<6qX+$iIkSNpp^NMmo6%9xOdm%stu=I`<*&3^At=KUafp@=2%?G5 zo9*w6c*^jjM$$4?ZP>7S?V4uM`s^Z2=i|uq988cs@Z^tlO#=BSKm^JqJ}bP_6Z|aJ zs1j8V75rrkQ1J?0G(^f?irMURtmMn5Ex@gSlG7~I>Ks%HB$6OwkOU3E`D`RcqGg2A zC8`}8h&9|3!WGXCg zx6Vg)*lT~w39l=QV{ntUs;Obk)hEdL=PAI;Qk_=19NyAZhmp==MZ=3$tcVoEc4^Z( z2BIg8kR8sj_hr6^Nxut1O;KQn2W|*Mm(p6joNxse@kcZed?U%SNz_PWjFqd_7S${4 z)^(3qLS}rDKs~4*;Ft$8>81+KRBpn#=Mhb(p=SamVmG3K22D-s*b6roJAfKMQ zI;Q51nh4}iN!fEsKa>Z@yZx>rU+>Y{0B!U+~{S*aj5X;elIE`DOC0esnNoY(yf$_8lbJg59fD zuWMFZZfd@6=WPV;aq0G*4^mZfAV`f**TP5?j?Nj}+} zb5CiA-25(yS5PFpcAi1WC6*`^&tLViNJPG>a-piD2LryS99(Bjz)6{VIxF7MpIkmR zi_zqw(ouxupiAE(M#tC0adQP!6}Zg4I3^Nmy(>%1v{A?lP01V7uOm4APbdJh!aL$C z;K&q|PXGlh39gK<#g@@{|MnxNeZR-vyT(u!Qg?#GHlV-`PwE^jW|5B>ot!E02f`1g zcVDs4V1|4I)-2$OKcLdk3%B~(5203cn z@~rO4pq2hfK42^H=)34q0hx}}T4o9&pXMmH$W`jbr661WE%_NSPN6u-nOUi*i8DZm zLp=fqS|zuXsNV<7rJd?yl9c$~BVROYq|Q=>G-ydp<&mthqGVH}!PPZXB_cCr_6;&K zZe?f)P3EtvFf>V{2%jG_YqU;5=08gT`HW0D6Wt{v>-g$RqVGw!l{aP-hr_ArIOn<} z|7>Q#&EJLsc}Vzevio1D_D$9AxZG2!GkESS{_iY)5uzy3m0sO#<(CR52bNH3bT)=|1dgI zzD*<#k~6s}wB@L*%J8kJeSV*oqmN>#W^aWa&sluEZz1P5GtSoVn2 zhg-0!WF??SVBR1)jRfUStWB$srlqF7k>C; zQB_vDX2o(pkpw7M6UXX(-KrHE*R7ESbGjp}um~dr^8i{P(h4~lk){|vst#CV8s?Z0 z!~b^Ay&t^ujw%I6x6 zAOxvW-Hz#c`tiq(9{f=x<2$L;{5hZZJoLcR58S7!bUgwPQi!MMEE^7E4k*i`$U|K5 z=THCjznU{=;@ENHM~@;u7oJ(56ebZlcslH*dcXM3GY{VX^n(waG;%~p^>0|(XxM~D z4+%{MO@;g0N;>A57!U?06+jqL_t)a;^%mAT>FD#JQDKKY|awPcu_caq}!e! zHvEX?htlx^cpSm1%(HQZQNy6v4L&(li-vYpy0i&cBEP0Jbm-TQEhuEt%)(8KaE%jI#V;F16(s zsNQ05s38jM_%d$wZS%F$w}joL4{3oZ(qYBHA%xB#O5m;l0UKX=vQp_KGhPy*21|3A zvR*35^GFjSe%h}rA2Vjmpn(IbD#{N!_`r)Uy|Scvn`AsOe$aqH1Nv_$D?Pl+k(XU{ zHJLVrvQQ% zf>XwW6&I@HJ39(=hwgClp zc;KeH2(PWW3tzs-ZngXX69M{{^f!S6VZceFSMAW^Ff&MdQ;Fn^88hNG66wtPrSsU7 z7UlwZ#pt1`FOgho1e*HeL{cG^xp<|}Zy5sJ@%oy=;VaRFLvf1*5-7S8+U3opL)#_P}Da}P$xZQHjW)_(@5 z7;E10%HLo5)Z>p%oH(gzG`+E`@~N-yU%Fu4@m&vN4goQokO2!HHoQMvdFkwF(~F{Z z$ceoD+P{`8TKw=!Zz__>L5Wy?<71X5Q^PGKaLFs<&FBdI9yOyasxTV0Yo;bO(Yo4~ zXKpB!@ii9!Ygs+=d=J5$ut&`7)R?Qjwp^;gp&qB5#yB`jo^2K#` z|1DGynK<&JsF%)EZ5-3@l{G(n+x@Jw&-vZ&i=4~{Z@m;U5|x`)%^vbG8z#Onbo`*{ zUllfQ>%nz1FY4QMP{a zNADz(31}tUusR|HM;}s<^|$&7P|FyqfBfO4uu!Ytqs_vwXtBd^TX>QYBfWgm;N*Wn z0X-j*X_96jUR)22<=-*oexy{`D^-F_n` zj=$@#_jm7cmhYvSx8J#K`<XvOwv60yUvJv!}XJJvBrcKLeN40!*^lP9yP7>X9|vD zK13{U9C>6{D)!yH`LfA?;dMRgD49yg!lf~8;*_~x&P6x9%VCG}E{{0;(DB2E`fe&@ z`cuY_z4zYxlPi7@X32LG@f2-msTQ?G)+IUrX8t^J_LT8YKI&-kS&pHRk^cPi&ksB7 zFxEoR7^DKO>!YnGieEtpC=gI!yP-hc18(6P?64J9`Wy}{gKLElo%nV{U>b#H&-=?Q zw=Q44yvGF>U2)ZwXB@e|lC+q|pWOZ6&%b=@)t7re_v~MqB5#r(c1olJ(E=)AdKmTy?? z`nbm|M7Hi$Rwm;Wg@wi6e)Z*s^{def6fRV-+o@l_J_z1v*F8B$#?gPoNGB3CoJhE^ zI6^X$qRonR8?BvF3%>h?;lwf{rR&z*an)s5{BZgFZJgn$A81-2Us)VU8*Pzmw3 zhXlZOSt)ZOX)Ba8orK6%!rXH`f8FYJD9N>I-QtZm-Vhm|sBc?Ch7B7vZsIu?T@-87 z$cV)j{jdbvNDNp2#I%nUiwJ6vkT3mV$%5HmxaHMPKmYPu{XTf_qmOU=^Br#5A2ej> zo_p;hoU&;@{p8>Ky#D(9Z@<~Fu@vAx;NU|Hh7HsD$J5XBe(SaQbHA!CEyYqsxd;!# z3wx+qnC9;L?QayYge5oq;sy=h=-pS!r2sqBpkb2@n>Ni`xOm;Vb$jo)xK%E0_fo;pcZ?G4M7$KazvSA)Nf_5eZ zY9oILWugVw-*!8k@{5l@KCt(jMySAWVxaA-|9btgJMKKK%hB` zKpYC8M+XNVGpuO84?i&Kl{l^1?RLN+t{qL7;iFGGQ{;aYRa0lpy8Xe20niU#dF82l z?{x|bo3?IgJF$Y|#<$%6uwjMYf8}3K{pFr;VPV6Tt?Yt=1oa6=5s!%LaF*-~>)rk6 z;~32JfA!@{PU-&oKb~5&XaRPR*WPwpBJItYI`vQ2^b)1)g2G5)ajTAk4vV_J@Td85 z^~&`Y%hvfjzx+hEVsoB62~M$p!s^xKC1G>7cEwQ-F@q`=5X3?~UX5Z2tgcc6M3V{C zY}|#21&ryZJl9QDRlWWY%O?|4W zEAU8t;t_`>egEn^@4w`#UQwkX>)D~Ciw(WQ%kokc*X^Ipp0m@Q2mI^px6P1KT2{f6;b_E(gs`l4m9>PENtTwDQXM2PAhYYbYu9Zg z^Ja}3Aw)B>x+BW=gd79Cv&0J}nlW`*jK=P3Sw_-6Li*Mur z2v0EnW|a^N8+rh`C~(;!FNVMut3{ukJ)Ah^Dt0JRToNm3z|Qe$^iuQ&IZiamAOuTn zTPZi>*b!{nF}1gyuwq&nlI)Bf7FQ%@#&Z+TKKc0i)yq2{eBk);rFao$uS{ zt%YBIu}7z!*DPB$Zukhp44-;WPi6yYKNBr(U=%TYSa{k~Kn(nv2`dt5*wjG+g9k3` zeIf-7o5w82smAjl1*ox4Bej@oD#)gFH9x-p_~Q$%0}2Eb_)iq5bHFXEFHnZQpyweX z5z}zQ4Z>iGi4KARjCFu@xC*?UiImu|F>%k>EMN*gizwShk|1`4;x0;@LE)Q3RKS5k z+<)@`JENiEO6D?~fwdM{j2Nur5bB4OYbK5uKJ3E*By84k_Xi(%s9lGh(QjF_Y{j%u zBSwDk9(8Emq0{{jKi+<)4hcpVGH2=_ejpfN;4=)DWy4S??V;vVTWLR>>aXuyR zu7fzkLRWiq9vMZ24tuWkU?lE8~ZxSG>L|SsoyXY|tP_ zwmy&;9i%>bgiw_bfaiolfq1fE;gJi;Q{t7KQC1Ks+xujP7C~(l1e$eK)w9`1ipV9< zF~KpQKtO@*lmc}QxO3Q~kx;->L#p^;P?*b$(w5em^VI069Oj8P7@$zCN zN>GoxX)IaDNFpSVOnG=f+JW+*=&h)r?knCl%Sdm*1yed;O9|Makz8jhPYE&u-{R#% z2h|ph?;=VelIejYlibu|B2|(+vICSZoH++U)@dn7MQtu(%SK76AOsW$D6l!!1N9!t<&iSjmZKKIh;$|Co#fEG6auHyk^3r2X=w{1w~^t1T(>e6Ap=Bp`+4DOU(rR16OSvMBlht4gax^2o!kt|20A<-|lDAU)X% zaObpGfMfF4S}XXGl0?)<=Ptq;87Q>WTeXqfI#)@P_qHrg1tFk7K!NR;0vbcM;|l)P zRRf`NIJr6NJhbJbwDcf<3E*wT<>ok6#jR`X%cdnOfm+}?x8A7nliS?dDycv&oWF`w z7z>Q@SFl%^yQo^}7=_@QYlB|5dR87+x0Hw@o5$v4z;PC}RXwxA13Bs{ZCx38OqIC# z`QR*|KtO@*lLB=PxYOe6S@i7yDVc8IN7zw*ghiQK*irr?QpS6ph=)<@BaPZpPduRV z6bv=MFK`L&=^q>}zC>8kRI8dihrx@m!LqX$UZv5?#zI{%QEnM&HiQJ2m1F~xm=tz2 zNTSJ5#uMZqSUD0Sb)gj0vpu}@+3fi1`Y4~nony*~-zg9o-dlNR#TpWKS1BvQk2KVZ z1muh>K(=;dT0tu?24aFiIpP$H83lYjc1JY9H$&(WrV{L(hQ>ELo_XwHhT9d~12Buw z;{jHqq!V9(8FU%hPe*7*l#eARxnUHVP3g*}_wo#tplFgwGlg7Qmt3X@;mtvnYZ9? zfX3=^i*nfz4#QTmfEE%@eS;8CUWsX)sb( zVkEHx7l6yYTimbW;6yDbH&x=9_#1S5PZByLp188|tZY0M=t??kvGm6U1;@QcMJwG-2Nn#1O7vRyk$ zhraA@Ni|eT5$w<$Tpmg~igIzHBEy4%RV_y`G3Li8U+%J@aLB=X2}2w1rKp3Vm4o|3 z?vgpg5%{yLL?WpvLhDG$JxUOVD9RbEq{+|*u@91V;SqpMQk0$x&q8E~Ks*(r#4z3f zKxTnu*>c|X*o0m-@5Bi>$)pa9pi(G>Q#~0J@x=&96|Kxv$|I6$Xgg2F1DX%TtPpfm zRAD?-H5XR|;$Vg?MTOLiv*O$_2mu9lKnm13;07=F;aVvyp11jUh3_^&P(SUuDS$%| z4lGdc3=RW^Y>p11r}s507L*# zVdr6pc}Dieb}U@_W!PUm?aLOu=dg0}6n8+9Q-eFfbmX7(XTU=pw-OOxGt@ z4DqwOz6)zhwOv?w;L^dsU4TrWSVjq{B$#hHfELcAh?UxN&8P#n3STe7i*T_v%s!}U zv6+MyiZZ$&ATp>XM)nLe(18ZYY{4|b;jrK`B#@Cseze+4xR~UN!!7Cw!>f39;F9#f zYcyoZupuJ~I4g>aG8%An*(+H*$9gzz5kCY>6HJDI82hX7ES$KEqf9~=KXKy4n-D7E zrhr>i-VWJ9GbT)8RLVxo?1dzQftwO%0}y~wj|?FrEu<|}pPYr?=TZm}gn$A&9tG+g za0|OD%&a&P;oFB3RIDGsA)p4|2)eON4R11;lzDXjcl%Uq+JM`?S6+Bw=Co;!X}@zb80dmjVuy$>-IzbM+c(US08ZKysaJZa{@WCEAa`^Z2zNOAGI>4ce zqZM2k3STUe3^z?+DBx1I@655ZR6qtWGROZS+;*D1m?d*cwj4VNxhR;Mu*|OJm#F|J zjBU&uydi@IjT}6PYO`$+?N37-GLxAK!;(O(OqPTbXUvhB3q@(Zs=tt$y$@9dbgQ8Z z2_%}J#<*l68LzCs4F~( zAHVjmR{&DP4#LVJPzD>}qhY)?5aVVfjSE7=>#(5m7JwTw3@`4_|7Py`Rm*&)`@S3E zF(ur_87y*K&%swADC~bf=d+C~mKHJqBn>u<=UEtB_*F`zjIU&nEA5V!O9)2BnU5rj}#1ux1&u$EMSlo#JKla`NzN({Z`#yf+0TL{DaCdi? z0zryvTdV~t)C(=eTda450&Stiy%a0%?h*n7NFW4ANJ1c<TOKQSG39A3VPWwE|&uieB18&e$R6@83yTeO34cU+uG_;@w!-0TwE<1yI zI8;=>puB-6ZbUY1)DWl>MG@FyuQSHuOff2&7!)&`+0fAe!Z3FTKmdr~@O$r!TfXc^ zw>{g-Wa6&OiLtFPVIV5N5Vy?;D7)PjHRh)kLe=msTBO_ka{D$>SHqdvv=BJk;RP?D zb9xzJt|cCJCeyjHhIVWhb>*Ugy(M)n_+TREL3DW>)dF!K8>|G|^5CJ6NRb@Cxx<$A z$;aa-efkMZG+L+5VfRBrc4V{Rv4VK6WO>1UBgfqC&6_rk8aW&&G#MER;7QyeqXQQV z6162tEXuwp+|;BDDWEbCV`SdJx6xtqF?cN8blMq-K;jE!0nvfqTNo8!78RO|I$W?T zB6I8xtB0C*X!{Q#ySOV;kxJG{Sa#%b*{xKZgL-qqxLYYT@i^^NICUbzjGVY#7Qt}j zh&V>)BXUS;%&A_VyL5?SVJ~+3V%Fy(UYVq3f`QbZAJ1#xKd=VChG**#D%s-!2d`ec zxPRB42MI}ao3?1zq2tLz2U1fW1?SJZYx}QWUS2Qu?q8HG8C{e6-n>Xq@a_rNZQ zS)jCi*B8r{DH9cOZqvGT_}cE!v2)cH%|t&WDs)PlUzw07my8_}aikG*{H+?kUnZ$wA6Y}2Me^Ok6@ zY-|asZ7_I2IS8-{sJS<6+z?VUq*bey4yWyxHR~&sDH|Da`Np*?ZQ8Z2UBBLnRjXrS zZuRKay-fY)SeE>}c1@)U6)V@MbL8-$l!UnadGqYqd!Ry(o$0U_3q>2@3(UGDkzffFZM26ihV4pF#EpkJC7XL?-!V- zXWxD$OO;{L(w33BcG*wYuY}jCR^8~d>n(ol<%asra6n9Zpg25Wy(jM0t!*1Owrk&M z?dmo8@&pbZJn+Wt=uKOX^{d1BA?s@eKrF)SyUhp;>^KTQ{x`YTG(9CAnmw0^t`gQKu(Q9NW5O3oS%gV5LgXXmJ02S1z5;PEUKOdss|#G}39!$`w~G zT_mbg$Bs_<=>4ze&I%0hojGaZS94~;JP5l((wbDjR0E14J9F9GSwjX6vS-`Qo;>!_ zkO4yn4|LjXXO10yG3*6~7kZ~{$-?;`y*Z{pzQB7i(S16#^*GY?_N);@hkQMEc1YpE zutchYuYxZtNR@e*Ql#`1bQj zldLc)ZqJC9Ub%MZa?#*?AHMPCo?m|r3C@4<^qJYyr!gFv|Mgb~_a6u;T#Ow!3s$Uf z`{oT)_zM@#UAcDke?>2TUWjL|2A(;Mn;A!MgCX{I46nLp?*Tx`OpZbKiux#20UEhb!nbclqkRwT)2~#? z;;ELrLIf*|H`Md4y?U8lb~MVR|C+mas(&h8!CxuFzrQz%yBCYe7KJ9Pb(%MpWLdRehV z@YIGHjT+3DGuvU$s8fBf$ICqHtFI{Fo&yKZo;XpiSW&i$fM-xFheMZ@zGUT6Z~r`g z`SPqd`|>x9_Z2r3!>n)22=B7^CC(7E+pM zv3Oy|0J4XQ7ZGHbGHZ7As@3x6%QN%i@$=XHT(?2}QbmePo;YR7EP@z!?b+vVwPd9x zT)uee*xp0kI(7Yg`i#zv8m#{5hYjD(Te0V$!OO>~cUP`n^IGJUmhD`yV8<~K9 zemt*%|Fjx-<}_}IV;Ihcjhj|%)T~>JmhZmzL2^=pp3TVI_F9b^2NfuU^;q7#L4_(+ zI(hOWMhdVYz!Dh6=v{2;0C|*Xpvw&7BKj;p8Z~NMrD^jn65B@);^Kr$fuJB638=T* zvPKRcG31?hYt*a*^k2GgVbiKLITi9t?292snei-{bNUtBqhXS7{hcWOgs8J zfX#d$FpW+e6BARoaA7h|0XXV6Xv8Ksdb3yA>NOfQZZ1=^CUceQHSXWNCyK6!CO9Ry zV||TgE&crSVmVj7Vx`CE1#+x-m%hiU1N=rX^@>KNo8#-^F%GGMvnV>XVo40I1KHU)0R62}0 z=8x2_)FGS6F$~a^uUv(>WQkv|t69H3^-!iv8C`a!4Fe#jlX|FKyH@@B4UZo?nfM@q zToo-+c+~rE&-vuzm&U$UzI=Ix3B23_Aw^>E#Y((J6MMqR$@AlR4g9Coz%vKj=)zb7 z`{c>{@#TUA7)3kKT}cLVL(Q84waAP-0eN((8JUk$;TbG?cTChx zAEGrg=49rT{r0vq#^JbVVkpwIzqE^T_lfB5bY)k~1`ZQThNDS=cMYgFGr`VP= zHB7Y~II_&fOKCc0(foBf5L0+@jeCg7q@bK`4^=kL;*E{LzwQRc$z3%qaE1Q4Wkeu}3 z>#x7o`TC;93PNCk#zY_1Z=f;kh{5Cr9QBws$chhO3gn8odhNo+h&pxa_v+U_AtD?F z5WvJLoDu%kUpDSu_w$4apFm|idi2Qda(C?7Wy_B3QFmf|{ryjzIPoAU(dMv=Rh-gW zkp=n7l)ZB0N_Ivj8Q=xYo%^rHNeSmpo%m(*rfx6xvRceer#(;U&?{H2WoKtUOiDBc z2Nx|~{K~a!o7Sxr6J;?>G@?F>`3%`7ljPK3C{w2F?dT|%!;U=&s~a>^X5;SH5Xe!7zbFTeiUu};mAJ;Ey2s`cu)_v~tM%D|U~FPJsAMaxzlJ9dOVZeFe0kU@hQ zw{9)84hslPq>j;@*e1$)6+vo4jaoI29X)JenKzJ%B5pD;BEY=~3&v4IF>iDljM;G2 zATd~I@x_F5jXGlY*4pz(e6IXsYyH&Lct5>b8)xJGKf(o7g z)ohENtveywML5W8g2@WCVU%G3ln!w+W?&a6l&zK!pu;`@E=KL^YR?N-ll*FW@OP4EGtj4_Um!HpA{o@b+iJkU&o}br% zOtSsiwP4rj(sjDSH&QMqJAy)j*;iaRTJ)yeL$HG1|GzS#J^WjoLhwWU-ZYwQb()^Vzd2)osX9Mcx`8 zDoK=L1I`87p86qi4Bmlb$R=b4^^uWu-zOlz;O#98yWR+Ehi)Pf-kk77YOZzS_zxLU-%22Z_$jM+3jS*I=tVTbf8tfiSzYP|XS3tp{V5<=-OzI*A0CL!#hE1E-s9lE{MetCTx`?49M7B|8V?|8KlqoZlO0`${ zK0o8(3EmGux$)h`f&K3T~M#URGZ}@D_B* zrr(>5k5kejBCciwSGX~9fuSIqfES8tM!GF2>EXk~OGR5Td6`&mk0vW@2GgV- zeSBg{;={)oP%1peA|0Tbf(cdv;sR@Sp%-vNOD1(UqN3vCAHa+l;c*m^t-eK8*&VKY z`GePQ*ns-XKgqQ?I05wxPP;uZA&CRh9IC@1zM8;Con7N9gsNanLTc!_!i@}id1%~& zL^YToBm5Oo9UT_xgtAw zGqPAykNYg))A43yDps__&|xDemr&d}(unfvQU8UoCr+Lf`*aqAGJ2xIMc$AMYBzKi zOOIsDyAjtYA|bP}P9Ygsd!wOrnR3T2Ud$Jae5Ekd9z4Fttmnt`8u*W^foBf5p(?yt zIqA%}bYyp{l7&Zh?bM-c$lkTfN5A>b^m&VHqJ6ppJocaF&gfjZ#48vfS1xt;{7Kei zLt>!G0UB1fbH;?X!m5ULt5vpD$>0gY2Rc(83Za6c4YU!`;I_IlPwd$?pl2r^U3O~J zmF|sd4ei>vXWi1X-WluXNONEg!cuY!Je-~zdp5K!k@wZEZQGSAv3vbWi&NHiqaia_ zXtlfculu2AjbdTd%C#+1Vr>7ecP|_UXhATBtAG`pkI+64Fz)Qs=qo*%R3F@_WzX7` zr;dNqV9SI`FhZNN0^Z<>*tfk^iJ+l9I(Dj6Y43)gyqsCAp0P7r5E9wB>*t@k)-DxR zwS3#q;&1fq9DnJgx6NS{SKSB)l4GJzpwp~#rrZhd+O%rFjxECKR-HBeZNKa^r^n(k z2)k|4J1-sB+O}lSm~L&`l`gVx<#I2(9VQ1U2*tr_p?&L@_N-Q-OSSUt%NKd8Z|ArR zC%l~5GbT=4ICH9xJImtAW{AREf}K$U+C2-|8MELWT##A@;LL!+OX1+fLdGztp{lX= zMrUS3gOC$xCWqLQNaav4WQq=IoZEzy?nDo&QcV)NU3BskqmX@5UKm*11%%0 z({CSv`P_@@5uVaKeWUL4kw~=gDo*W+1mL6Y=A@#f`1E(~-JA$>9#OQOJgQ>gb*o@f zKJ~1)9p|Hp`@6IVDS>%1H(lr``KY>4hCn%4B}R_i9Ll*$MXpFmwXzdHu8Sos6~^bu zC`IZ|9faiISu*$h_-kt5nX73OM$rvjt}!Ery*=%-3s z&z!Y%+pp&$t~Y7gbkd|r;4!)>P{3OeCDr3;*SpV&3l~W%k$z5sTD22d` zOxR8Q_>+mBe9F%$Q>Im|TJ`khYb$r``DN+S)2B}pSmMEplFdG1 z^2XTAv{ckeV$vvDlwz*}1y0?FTC-zUYD&u7*|SK8r?svkOCm*_NTQRcPN`h2`l*ZI zD|YYRzI55Sv!{d#R(WOp=jhR6-kUltJmT88snaLEIW8judRL7egfftp=3lVziEELo zcJ55eOq(@(HjJ^T{Paad;Lt@0hOP9pX)~%+t$yM1)fK;P-?D7k`7@_^nvn*bnXi!U zEx8?ilW_paQLz+UDwEL$g9|Nl)Rlq;t<>Z;2Qn0J$hrFDfQ|whDUv$4`-f+9B2mxg zhWfXE1xnS25X7$~p@mlGf@L|$|1l0N0advhd3fRlCSHQ`7QIu%`;}UnZm;k*}u$Dv9*u#`BYZMh!gOu>Ki2`jc}( zi$sk)a^#>cJ+)h(zBYrod;h`ACZ0HPN(R;VFE#3SZrxb5QNv2rYiAj}hYlVRdGaJc zL%ta7qxeA$fGj32t6$#0nsw@ws8a3W!(=dDu5(9e!8ijcR-jPxR;?eWXP(@?X5X-W1Mqut_QWCaiUb(2_R(b%NzPju_^$_?9kTt#ZQ#hj#4{Lvo$Y zs&}6}c1V|+`eMI)q=SSwVkS*cN@-Me4Tj_e@FwK*`>?sca;__&12z>P3V@=+<_3|APd+ju zq6Rkt-&!gfNQ#ga5g;F>js9LUkxT2MXzoe+0YuL@CpPKhmang=*i<6V3 zoD?Mj0aQtk7(^iPQ8CMd5?IR+wIbK$qYj>;QZ*H#tcYqY2-;d#6CK7pmn#@)MbwZJa}zit-9 zSFKzHKdK&!DKs<`4KN`g!7pEsS|2Dq9IRv^;{F4NjvqW2b1Qo0yoE|fq4Iytb7h8}nS0V#);E(=M_Y+!|Q z<>T+inT;mdrN-&N_OW1vau%z%&1m)V_X-LPy&rdvu21iF+OPnT4xW*cdi3bA^Ovrz z`+4056Q;Dg1jPfblfdi>bsix*a{TJtf(5zPDC?)r7=PM$UuEE3PU zAUk9g{6AiG`4<6Kzl#&zYlOrbz z!L^)gNCI_6li{yA&sCFudyY5tBnow+@|go=Poff3{XY&Zw|}?{h65C%mqXpi$%6_h*A%SF-Tx{}a#@vH#(lpJKs--Zm&EY=_?v3r*#mA- zFkPvS9tZgP$pRB`@2TTFJ2Rc#XRs~7(l6z4Mw!xnY@h|&c>DO#VKcJQ8N}0+X}EM9 zOy45TUzqUG_>_D1sx)kxo}Nxe0)x-7s>2NzSK7sa0OL^?)>qDGEu;>haem<$G&^{b!&@LYf8zb1=S()QMdN2J}jCa0# z*e^2mia}tB(9#cY-9pPYS&=#dkK|0`mPt@pKuGnC-{3|h?-K+O68Q;~D>#$j>V)e8 zwc14$CI|IIdlu|g@v7+m8$8mY(t--62@xiGD^Y7P$g?V{Cn3cQlxU@bMPb)c)t=0a z=eKfcS0tnA$j!-<3-UD2YS*=rY6)pqv_c5ZtK@QGK>(^`wM>fKRpEHtsFZyfTK@jf z9@m0tcjTPUZvi=bQh~pJSp6b~-{;}K2=n~X|5Oe9cW>fa57@@;dI#p3m4kIfA85_(IZ|R_14Id2@m4Pu?0^lWX~i9dFZMB{um?c zGg8xlf1b<8&I&GAfLBVwoN;BP zV?JTec0IgxKcsjl_$8*`U?z|%>u0clK%N-dv}=p?TKL5?U?JXL1s^fzmQ?lm3l;$6 z?nGT@J2ajP55E{1T9VCo$T{m=3KuPMHzLBGl?}!-uy}myR*BG1#t#-FRly2JJ=;KJ zvk3yFwU|4$??g`xKYxMEdsu73)(32;W(2!+`XWV&0R7RoZsInU)X!bGRHAelXb=Qa zrrlP!Sh4%zmpqwSknb!YdKi7XWSR1sUeCavZG@>VS(4Lc-UqL5CQXwlVk(OCbgym;0$)n;>Nl1=zgHb4TBj@W$ z>G-V*D&9j^M6nc{OtX!Tlk1t%oK(q{p0th{6I-nnr8?59(w0}M4PSGj!Q?2qie z1}2K%4;f1^R;bimQn6*EQDxOO~!y ztF|DEsN_g0$|EacVq@-{J9lo%jL(pg)v8rvo1W-`@Gn)T$xOr;r)q|WTsA(TOdq9v)*`O%dcV0TaTacY0H+a z@CXfoO>a@w0Sxcm)oau~eB^|amFard2X_ZJI)_~x7W^-Gmlw{qEn@4sisBJMs!0_Y7OyT{^fdzcc|tm&wsL(M*c-*4Gf zG=KiB8#m#$RD!4_II^73XqY&C#=?Cm>e z-h%mG&udt^2zpR%qkhV?nb&W|G^tSTomYpC|7KpnQl;69V#eq(2M!#7_L;kI;WyuqO6aN; zE5BW|SUkGG_aNnfTQ<94iD`0LkH+;z3?69l3s|^yN4`SETsq6KQ>SaysSBqADzgHU zIR*em+9eB9S&<~zC&EwzL+;Cr0C#1+R9cukr>VdS;+#~!@Cy=&Oa{oxAPLDwCQ+3| zGvff|x)8xCdZkFeTs9o3!ClhggeMtUXg~QSU?&O7!X?Ho@cyhQhUeo&@CG~YjwpPg2{a)VkAEjp?W~ofL3s+gq*Zglg!O9 zcU5ZY_Vcg5qXrn5%KQFPkAIY}66sT{q@V-H4matKA0-zkP*88MIPgp&ZZlGue8VFh0W2JK%9JcoDJ5HZDOY~U+ipMDQIO3G;a2_Vz7ji~#6~)!#fb}*72CKrl zaO?3fIiXNUh|NV`7YEtc!s~=H11#U2rq29#mh1L8tIHMJieh_!RN9$&rx9`YbtR((r$r>}6CfUVZ(H}sN z+s?XBC$@oR1N+2VEI4`*Ql6S1PKV5IiqinW905H+Q}lF!E)Y;4O8}SPQj4-7hBSC) z7>GeYP;bcMz7SDjgXm?Jt>rb6f%+#1Fvr0c*9k-0VV4K>enf5vs{^S3^ds8{rN$P5 zfFQmG{USkR1*t4^6?q_8#Pz#G$^uZ4^rC}nl8AWZii(X8B*)TFWS@-m0Dra0lLkoR zB5Oz-3eGFUM3o_+L?}{89w$2DK~zYDh)yZE0Jj2U$+3hHm=@PEG+-<>9X2kDD_y8F zB33d&GAxm*Jd;aV&nrz*Lh=pNOO$$Dwow`fMH8 zB?D<<#DVhR?Hf=q1YaAMA}$@=yO$1W@%6VGz(*F^!S8xm^919xM6a2sDODXzfOrO@ z9V}770iMx<_6i-Lc@m@pY*@A|BjvH)YW;Nj^y!~Yb!FLbt>>~>1A{|6CM((?iwHCH zdheiMmdMjjaI568dixhBVmAqD%W88vHlzXBLj^OLvhmVq^m4F)Bq|bGlfhc2QPaxR zYY1qY#bc^CMUkc6D5&PVg-W9vsXa=B3VuY5{Ub#Xo53F z9Uf717Hz1OZsHEG(SLbV!nR^U%Sg!xFSgsHe0DB!~@Ix zeFO8#nkJz+%;eVN z;w55k+?p|I3Por109=Ye3nw1^){Pss|MCmqqFfIE{y3|`pOlDR$q?Yh!cf6Pg)=(IK|`O5n~GphVV}3LtwH2$n%Z_ zszt9CbybA3)VTo<0UI}LUcnoBd8qIRwezt>it(Cx0duX{Qb-Uw=_JlZ}TMFkG*{PGOm$eHh$T(Y0;uZv}93f z#kUd2hM}D}N0kGZcYqh7wkX|4hK;J9tKK4hqCLXl>U5ap&3^Cgs}Ywm!LHY+@t!?< zysTC_Im=jZ|If;AdJEn&3>LEu_+Zan>_&8W;SkgV6o6%%h`3!gum)^mAr8F`MM!$P z3})qJ(d(zpm_`g_+l^}#vMMYp|8R`OdKn-{a)-Q;o-9eQrV%w-R<}#3aEpb4N=Aqf z+JO=Q$4=+Hd-twfjifdx;q19z?B2U)(v+$A7$IF^kPYl51A=8T)<>-sO-^8d)C3^} zHN=G$hTqtStX=S}gb+nfbYU|q%%y(Fqk%O=%&1Z>YMV>_`VBaD{CG-wTH@V1o98c7 zF-nxOGDy6m=r5c)as0?3zz%&7Y=h!qU=e@kPUMwvG-nF~0j4~~b*yOVQd4RfCpeZE z7kib7Tn}EtWT_E3k$vc8=0KU^xLm*fv^f3Y1A$ZkzCc00h21jYDi!VO+o#vAo!dnr z7x5{=RJJJMCofT&2E|9ub<=oefAX>2p3MM7YY$X`?61UPdNRa^ns+f?$T?RcR#4kQ z4tN20E94zBm~*XSU8OV#KuzWv1{@xnU23db^9{$qQp*d)%bpFpKPbv~xqDsZG7NhmU z4?hM<$cNR0qLz*0ZS3*bt5^DwbYMInK^}mG9t(a`MLTnYJ!X^HrpQq=U*L>SaDhCh zV{bsbIF#>W(TCuO?0B~g}gA*$+E+PyFh?r^K zw*ArYD_)e+$zB#to)SYDlw3ReCm=9!GD7OOJW)=jh|lG+8a$s(`LuYc(8`r7LYwe5 zWh~@ypxaZEiYtd-a(^~uQt1*!t5vQn`YN{&Y}s~;iD_7;(ZYQJEi=`RDl4_YKtf2E zt!VdpLyxd-BC8Mz3aA;eFnYjIl}l7n>W|vjH*eYe$fffL<}62+`0ZlJJ}=fLBn+uh zdm<7={>$q|zvd9xLzN<@9r%f1EQ9MCE(6O51|jEorEpd*3MCW=Lq$9B4R|CDO!dqBC9k9(IFIqOOc)*e^U)SbHGj8rG>6u zz3S?f%lGcbeq*)_d1d(d3zyPUGhXQ2{>0$}>{Da$&U0}8p^D{84;nCVD8@vL|sTC&aB^y(<3B-Rd=KckkCaLx;ZX<(22V?|x|CsqNm~ zyONV0_UhfQOqJ?*@8h7?3*+f@o;!Q`_~Aq8_Uz7`yH=`N1)Or)GB2IK@axVUsNS91 zwLfy~=-?5r`uh5^hx!i-mtMaWRkd1;!a+fRP>c3$j~qFc{_v5v)$8EVBNHcmYPUOA ztzHumevw^Z2EFt$+v@=P=+PTiuV7H$rcIk$x9`+$*ramh$`9i19Y1yi(~j!ZYPRp# zDbU|<{fd>>A}-yFz5T7#tN+Wx*KXcivwGEvRVrJI=9^L1H*fkmHa50OrAmGJ_4oG6 zhc@@cjA`Avh3(k6=i#H2-o1O5Enfl2fwdnXjwVmx=-_v6N3B}7KIL&zr!HMeX#qw>)@|P^wg^euH1~@wTpA zv+DYlhzBuIUzjXIMvP%<;njRI&Q{9yoBQQiZbp`}g0pW$T$UXPP!?*}82TL`2N3=>6NbjTk;W?%v%k z+qSlE*Kx!8_4)h*yZ7oHP^ge#pg^I3oyaI;smF78|L)Ax6qD8K$iV|uDwXfkyWh4Q zyN(Y^q0Lj z@!N%HSJ+KMk`tk-4Xc(eu3Vw~rAwEiZ$`Ci-Hf%n-+i|{_Wr%@-MiPWQwMQ>KQ8Xn zk;9iRoU2)*YP$~Yjb4FUwrz7|r}yjgA}?+K{sXt~J{U4+;EMzL{V;#t)X!%fKXvBh z@snd-A1hKO7ontF6zKW!x7GkUz%zCz_E_Lk&BD-E*u3%d@nc&zuSfB`aQNWlS4XV; z={tWf!&ehNeC5Tkr9Ujl<7@qR#IRr1tTcM;Dar9&+qF4=>O|2Zg}#_GbLG+{=u!d# zGESF8DJ%c=MZ&#%Mz?+W?3u#`3{FjWm=tq!P}>&C@o@-AAq*9{eE(kTn8AY_S((=C zwBeoFJ$~?j-WVBvX;inc$SaXP7VD^?{g%&}k&&Fl{sV7~8TI{_vkCb3P-h?1X~}H6wQ%tfIiJTeb4sxeGWi zym;);faZ% zF$;46JP0Ka_PkJ$*+xEy7B*4r2<5z}mS+klc+50ik zi>6Jt>UH;HZY`Vf`M6g{j=Dsy3}xG4_v{(1YE0@`o zV^r7+cWzvlS3h^orgiH|6)E!lmvg`T{4;cglEsV9efOQ~m%@|p$GtarNb#aYczKQL z)H-+K7<&8dTTyJ8El8~mzo;}8sWCfGX@tR%59<1pZ%&>ce@zWMa~e0mf!5onZ{MQ_ z4u+N}^5)prap*zMrTNjH%?$ayeU~i$-ej^ozI}7w)@=u)Zv=SzW+o(^IC8jqpI)n0 zEcXxe|8da|KuK`^fpG zFoxUoI2eUuMT(xfd>J#0%!IpZmoGnh_z3phyMEo)uv4dR7A|6_Td7FUryqVm;KvV> z&hFl^cH7o+RjWd+wy#&u%qZUF0J$7ViTf^I^3I=^t>Q24-oEYNq0kZ~+qZ4mqeDAe z+RRfYPm`DZ`}f03^z72EbMuy<6CG{()G1yji_>O@z@Gp8yvd(T7|^fRkwg1Rlq&t! zTW`pk9!Out%zDGub3bp?y5(0tE}<;FyLax=xY@ovyPGv?OvfEIc<`XnV{IN?gOZ`A zj~s2@ts79txEsAs(dW9!pH67mqM6=h3+vf?@6KOwp8{|QFDr)(xfS~azkzNFLoJpe z!cXgTr;newdhF2pUwD&m&_Xio`e)zQP1borq-!Z>xEV=dcF{b!Tg|dHW&pLAA1Px8qhA8wxNvSqR_0oVFHEJ+p+B96lG}^b>YH)q|#T<5b zYq)pUsbfcblrA%Q^5lGoKzp{ul{s(WcjKqds#(2y=OII;PX4rb(PC|T_io<2IWbo# zQ#v{7CIn8OK7D%k>ZhtmxDy!%7fgySe z>W7$=7%jjK#shGjSZR1;yatFF95q_Ber)mv&`Z{9e)oPXw5bDwG&p6CD=j5)%CxDw zwr%lcJ}@i$VM44~XWzbQ)2LU+!WXc4d7Wl0NP>=e{P<~hE-qcQT9(_&MlwxWw8C=H zX7sWdtflMM^UoJ-_n0ED++r(;7aBLy;fgJg_ggWI>Q8H90PSK7Xp0or@3`$SEe~$VSY)m{O%m$s{CA#9*x5sFBlR zK_f3ysb)fQmOdaj-Ds{{uGp_T_vzhex;`k;l1HVum#*C`)od(Op)$hY&i#7<1qwkB zL&7N28j1^HVrHA^T8)}vKEzyfsp_>VrKB7E0-?9dlr6t&`>)93sPku{4nEwp3P~yX zif{a=Q>PA;qCx#;(B4dS)@#vX@9~q&7E|GNwulo16=q*7*0C%ym})j`YBS*>-Bh}I z{W^^sV~>Pvt5m5HQ}Nzz`~LlmJf=R3y6K-M!1*YJ@gJj&citM;qG7$O$B*p~kC4qM zJf4Duiqf3olTv~U`-^@r6D_h3OKL@lB4O=wF^e}26{LL&C-nUIOKLzgj%VrsqbV;s z3;h!f1$qF?053ognSqk6W7vF_5TewW09{mZH-=?q+>C%W6)IFHQ@#4}^A}HqUpjF9 z+>Q$u=r&A0p^LEE95A9qV;U&B5TMHxlPE_XGn)gDuAqccz1&V5nZ`ul+Oc!jxr^t6 zLraNqhF*_zk4MP~Y>c3DW#5dvP8Lyoy}iA48JVne0p$?0S0ch?j#H^Wc>>wJOlAbj zlq*Zb`%fG@d@+3AnX`K?hS#iF%g-<1!GlE1?ualsDIqgHp8P+4oczwnkz+@XK6mrx zv`Lf6BvGMq^39~2On4EIpsHxOKK}l&+tiZLG2~GDjsNA>{%h7`n^k>w7om4uq0y1)q^=i;u^# z40*)+6)DKz-sOIC+#4lJmOOPO{Fe)XDS~&BI5&cP67DAFhvYSNRI&oa0rHx0icds3gtLKuO}WJ;X%?6`kDa> zak42A^2@C&TdC@xQKOEWKY#SX`6HJuv8K0R{`_Q&&6ZeKgGJ4a$m`gLK+0ipWI+@t zpj7KiB|6fiMARwg7N*fTCvxVL)qcv!$&Wu6@#<@%UJZ|m#;-cjQUvT#uSH(rVOG7K zJ$sgy!o>#=cY1e$;2?fTR%OCi%FZEXh0oEMO8L8wXAZc52*83SSU7*s?W_3KNTDCY zZh%;rmC-q6K(Cr8)=#WjVP7vRJr|oTvjqfc+oA2vi{YnFowfLQ-?()%JUpCdu|Gli zqBn_=Ae5slQ3J_9L)tl;sS4gGd<5SmExJs6d|bZ3fP#7RuHUpFDJEKW>SC+%j_rS5 zw=(j=$p^QtFZlXPr912D)Tu+q|9;-QtVc;(*RG1XeBSBE7LyVZps%9i8w|DU*3||0 zFJHA1TZQDrkG^XSa=n>J=+c-GD^WFgN$ zBW5!|8zykjIF!-M3M96_bQu=S{nDP4ynFq|$m1v5wr+*cYTUT#?%jLtL|%%$d1c+| zB_s^&3Y8!nh!ZmvGKl#}y@vHLFZpiXm-f_$Th=a%x)yp6edgWz*VqXAd69iceTJ?^{aOxJg5By=bvw5#g7e*-+e0+|L#(S&Zy0_;$XC zL*6Ww;vAwnighy(icMr_nKCK2Vz9iY0ieUmOW{P8)4;GJD}>MG0hDnuJQO8UqzE)Q zH^DgilEP~;>qN>f`0A@$k&)u&g?52l<3XTXq!5ZLiDG=j2t$m!B_i~3kG@N{ZaaS6 z5qsyZ#moEHiQ`Ob=Lzty(x}0Y-~W(!_ujb^C)cfAO_8yONl!|cJ$14!GedGisxq@G zVqT@Wc1tegB>J z-g!GQ;hvXScm2k-pwcDH7JY1N)ZYF3$Tn5QH_bwmI)686Jx?FS@)mNf$sPWcYZ-^hL{-H>_5xcmF}xB5$ApR;f{|WzXIpj(DY6jVjq$89Hxo zQZHVrRJ%U?`o7q!dHu#kN|r^H#K#~qr&H(7XHT83SGmgF_yikj7}n)F{ixSpZ`88& z=pjR!RI9Og+43(}uMR3$h?oI7mL1XYsU`=NN!ek0l_E)9nXPaL#s>+>O{&&-f9zYm zhYc%VxeD7N*J;wMZj%yU4rZ(6?P)W9{CZx~ zs?}GlSXHk>N4&Lgi$p<5vn%ewWKFIoQe z$~6Ux6vtM+O~(%BFND`EU+HdK!q{=|e)F%d>sG18>M9^o77B?`vdLv&8Ig!Tr`^SV z(0TJ0&=n|z;z^&7{n5ZcG2vwAPgGgfxrp?|mXIO~RbXOys!0}AVL=WotKpR7Q+8_h zptLTTJsVFoGMWLpWn_U7pFwXC0tXRKi{Yh9%rr2GR;sjJ=gvh-m+jZBN8O557A#(p ziCbZV@r}3NIeGjn#@!L5fv>R+PZJQf!%txY}h!YOgV@?mUm6~bW%vk(4PGU^y)vPK#?LdXU=4u z@X531aNWbZ*tBWm$M3(d-m}02S2(0ZGO~Jp{GBx*TEL&3{y%ETI{*#;Y(-$vz1$9* zl=|h#iz%CI9pGTcU9r+d-2#PBoat+9v(CA!9CQ%IOFAGcP4HKpkrtf4Fqk4EVX>TL z@uZl-F}u&88rl=ClLZghu`$9$vlLC6ng+tr+Jqv-7fCk8VZQl->AM2_7>Rq_m`Tv* z^ZNU4SpMVZpH4b?J0AF8mdS3*OwG<@-mGQia>K`r8#Lx!wt%3w(vRd>3*rC&R+jdl}TEp99!BaMSDq!l1vP{m!9$e>P3*C&& z>;OOULu>aqs1Tfj!~17u*x1)CIDa4*XUDKkmea8CTfmfbT{=vgMd%3!Bou>=HT)=e zc8@DPHO(JsVKLhoAHc5Lupl<3K1vSo^}&TV^~^$RV2cdU0vXaHpFi zooxrMO(@_}VnCYF2A2M~PqfQwDY8ZmDK$PG*^UG0wP(-1<;$0&*fM@WZey_{e3Dqg z$RvaaImNUvbI68WMyTMy068HOYBum%lC%$qU^i| zl~~;U`@LGVS-)#f;bKJ@dEh(K2ZK~4v{2-ER)!sFeo}CQCGQ4ZtnJB?chO$t`k#D| zlV^O%`?EJWNy`F}n9&3G3`V~Ie|igGWOq7DRtv*ox|x+$%;s5`7Sjk>uS~0l61B2% z0T-m3kM<)~DKqD829-?$APo#8Zo&w#>02V7gFmo1vCo(J-&5V(;>t^Q_+}>&%r96WL>H5~cxQFUk`gfjAVoDKbKj@sG#i%s`_wnq(UzKT|$$E76Mc0SxMK z&&9w={FcjDiqc^{tz)^V&@xD6hC%Axk$J{eq#~*>r4h1X_3{<#Hf-$Kudlor1z1!m z%V>oPlW9!GQc^|a%|^yd7S@YO=49Z->tbXEGs40b)8Ln1My3kPi`XYPum@=&tO;wm zkmd|@RD+@5l!tiKvKoWV8i!N=Q>OKO zZA?h}2sD%2Y~FhdNi4h1@-W~pjmR}@VT zD_1BoYd{IW`=u;^4005(1lHt1h1~+MpdO%t5`-jIs=E}FJdiL$Hk@HKZ{0@fSi@EANpx;rv*es8 zMC7UwhQyV%OK z;+z*Fs48ujpS%G|hj_+smvHqIm}bmNAQ5@XQV;=T>RRHEiv8x>c|60IO#+EVfZ)^w zwiFNbf*PZp%N^~={i}Zc?nh35=YRf$8hGY_n}$x~qMeQz``YPEzvz6u>2vRXI;mT~ zL0sRq@#kq1KbB#&$@u1kk9zg%hqZ`wE!sAnn9eD_>*P0$S{gdtN<3=NN97|e-Ni4t(Oe#o zPR}34`z~D8ef{;?BFhuAVu#bLE z(>Zzk=<6>H(q*MHH#_*@E0ibj|>k4VW~XrMXRiW+Crho2tWoO zlnBE~%0NhYKr#c1BwkeH7U$%Fc7ueFhC!-Y4hbjul2BxkKe^xP^!eAHUjtgD|M}_s zV^W|=(o29B9Fy+3e)GiD>u-JV{;bIpv8Kq%OrP}8yKlZR_Tsh3cRm{bugR13YRj>l z?u3Sd-K)nX9#F+YDehM}r)LShB28V{|I#x+8M+thIK6@XNxz~KSUjFDX3pq2Xh3MG z66=>Ozj^)Iu851vWQvK4TQ7#svbj9^BmEW)6yFte zNijZy#I&=@k^D%BfC;*e37auYAwUjSLP8uq?pV9bWU08KlJ!AgIuOYZ>1t3dGL+B} z3ETr-Q=GW1!nv&|ztSi~X+!HY;YJmx0G3FFZmgt{e1nj-thZhp(S7KEleccpT)bfE z+?kP)5mE|)5-|=3<4uxfn?>c58+3m7R7wC@fRaxNWLBot)Eq#Uu{LZA`{IpL!Y8Qsfdqe3{Y@Kbbj z@m8vpPqG4%a!42kNR&e26mZgFQ@NGeqe6K<>LcMLGlJa;EO}fqC^e=~S6+|cI`5M# zQwsT+lQASaxs&SR6*4j)hmt(0@(WuU8jwgMK0K~op4dL2wYx>huBUT-GHEjG0xDRmR;|96@Nrxmi!y?AsVT)n3c74r!R(i)$8D1r z&Xx#Hf2L<1JG2+B!(_A`I&`RFl?tpY+xhFRW5w9hv%kKJVH`(lsHTQ(1WbDRv~J$gpj z%ze(E_htJIZT9Tgnf^F+;J|@KZ?83L)}VLw>(e(+iPGqEk>M8)?%flY5Z|~-^VaRU zNPh!_G&9M@B|=?$DuL zy?Q?_S#kI7{jM)|V`Ca7WaHuC+)%9I&CazswQ1+W+_qaNx!cWy_yk{(bA zh>G@Y+s!Ch;=rE0 zd*b)p+icpue=pSq%60A0vqsfQAAImWM~CjcEHa*89)fIZ(FycOtnwu#VDKSmri73n zouJxpsG>s27Y}QX_%F_X5G{U>cP36Cd=gCuAgsQG`ujQt$WrnLljpbf^x_CLy=05lQ z@84bnIlba<&&OYu0aRtWJRC98O`tZK8vdkw3Kc0?tfZfh|HyaWefy2E^S_++;X7}< z^YJHkqpXgjEyEx(!asN9(5EAZEnB+C-_QExdvD{fW9g!W`TTq*jCg6&>ScZg{lxd* z{pO2V70ZX-zI8qMQ6eBoUlpIOdi~X_my6e_>CJBICUexyo8`(?FvBgo45iD4-iy7Z zP(oB2{P5}R8LQ{a9M-qLJ|iRI*iL*w{O18wiNeS`HHy|l-VL}5XPLylWuN6$I zeads%Eo?oXCs@O&2A5iRKuST90Gz;IE+pmr6r6hUtYEdGdgT$Mz?1M=gcO-8B!oet zRH*!7R3rltB*6!q<)fXekm}-7$o?Cq^Q`)aC=xMKid0hCtJ&6Nd+uzs>uO{eqq z`RS*hSeEwi&aDHRw;YVT;p^j{^6=itV@F@?Gk`6I8`Q2bWI*pBFAa7XEaHHho`5Fp z)J%XfcDgUkV;unay)MFdJZ25R(!fS6nqjGu>Ve z7~gC>H>+t$RC|wI9Dg@#Sa%wJEj*t3ySB0Nkc~0%Ki*=`))Pk$weR#o+U;BI$Bf0V z;mbo`QWJDS9ng~j6w%t51NHW8)qmidqhB95c;JO|r|b_ODhIX#>JS>pL>7Het8-?~ zDqFo;aDn`DKm2IvnzfZ1H!V`A@QkTbjh}xOeeGJ|!iDGg{6%y6LJbmZskU23tbz82G`cqdp<&h^MLGJu6^S=eneM6&p8c-@N$;AANZDZY&mu^kK}U*^ZLO7V1G!@Gj z=I6w?&0wrmx1IqLMWeB1t$K|cw_@54qq&Mzs>UZKdj}QhIcDs%cg77F*yr+vGsFh? zz@IXw2+e{x!-~1t8Qa|+Hx*yf(A zTRHK`Bqmgo$`9x86;C4JI3RKB#_ z%5`;zn{r=;(F!YJ)JINK#<)(jd^k}LNMm zYfqg@TEEZvQ#YTV{twr{|I5qxA5QHb5FMz9M$24W%+1I?UAw$8Xu#;#MuE=EFz!9D zp9cKen9<(;0ps3z2d(zpspG78r0KJ>2}{Gx7B7?8$`l(7o(0QBlNUAzxH(1?*c^6@ z!CSX&clg}-?%ldge&fykyZ18oM~#%8rq`D$S^VDZJKzqcAVFoy+>E@&)S}7bxE>W< z7El9L6w1TiFzL@TaLE`Jh}j$Ca|8)Q6)uUxe{=_!1t z%to77kCs+or{F^#(g5YDi#b=Q4@(h;Wh&KNXvV<&s?aLuPUx(*9~$`a{gxt*7ho_YXb)^;g%xf8AvN>RkMV>9KkZ zQ-`=)H-`4^-L6&3UOmDRV{gaZi3U&MMs(?q4(!?K%uL_Ab31UHziD4WMXc(8aK zm0LAl&Xux)UWs}sJfpA#9`~MYzp_+v*sx*Nyg@Nh(HNY`@)J6<+f%k=Xw;2JAj@ho z_v+bW`Hw$5jJf^zPRx(XSNC9kPjolct6-@;tbxHo1crs4Mi;~qgP4p~+0F|(Ss!`% z+QqZy>esE)zjv?L%jfmZ?9CfiA3u1&X}5#zWDp+OZO=+cxR)A#7dzyX2ML^OBLHLx zeNx;HslC(q0e0)HMu*I4id_n@jU8}mQu4lCyF)@k#=bEI0Tv%~`|9Zvs}}!2Zm0?@ zV3?3Z@LaigDU;nC9>#w@b6V-zwQ4q~i!BK8*gRiG|v- zSn$x{eRpD`!+LaY(yT?q)oTn#GE-7?i3y>_OK1?p8bcWzX+V>UN;Fyd$VKsCqJ1cJ zF4k}(=aeu|&@y!Zjwwi$Z)t2QH|h%0adIK&P;nDN0%_@J=eaS;RWZxS(VYpUGN(mB z9<&efOMcXclbl2NaS5OuoNEEJZ+>c5h?x%u*U7h(QVz*K4NOflU}~c7N>;SHl2)JK)G7Q!Yr_U5EUhIwW6F#2sA-f-xtXX^9q)9bv*Lp}V{eRdy5Adq0 zZQt*{cX}nEca)yc1*D2}R1icE#X=Q96i`70Q9!Do^xg@f_uhN&5FqqGAS9%>-QRD_ z9Xy_M-@E6$_rBxtyJUTvu=d(>%`(=S|1suh6K7eG(}hffb`EI#!cHhk~hu3W^;no@umJILHPd5Z0IP~(31!3S5*o^DmIHfo2rdiQGg z##_qY3b7v#T#2<4yknLj-g+Q}IZqBJNfoOjfd0YlyF+__jMcP2(SqZr{$|$eW=@?t z&YJH8tvOo5=!P}cy5}cRmtA{ZV&VlEe1Y!%O zh^H{_a%jMFv$BSL^Nl7cUgPWY{%3t7Dpwlu<4*+LA*~}DvfO|*fT?q4O*_3I8Uj_T zUw6*@IUd}=kSj4hFd&7P{Dr*%$_xcnzLNFDa=;*u9zV8{tWR!j>FRYyj+qFF*_X!@ z79L(8Op>>#kW}@8g3ba>0$%d86aeMPp3(;1IE+&~>eLfaMN~-vFn)OsZ|W=$rGCGz zI<3g;xsShpyhXI$SV*Y}u=}GSRF(cid+Ix7fOUn=fUIiwABw22)P}MJ8b}A}wE|+# zjei>iKk-E!o?@JOI%B>N$U2ho@BDUs_bZJ}1WWA#)>#*tVs0 z8Hc*?clYLh`72N$&q?6l+Lgalt`gjb5hHlT#cZcuUzdUKXAnsQ9VQ;KU~Y11U_cN- zgalKNi9)$!OCAZ4XBAq5M8u+jik=C=-1hv65id70mFQ}d+1E2CP?ezL0IvYXVP6dj z@%JA3?f0b8CNo8PS|&zLZ;Od)5kEbiSWG9}0WlS-mXKb>>lVjqI!Cv`CdnKP>v1@^ zJv2BFNAi?}2Q6#Y+IRD&-r_4kHZ&|jAWF8Pb)|a&_9a$Bwz-mlU|tlsmgy@xaYiBo zM_2|KJ@-l@&S%Mquo;y5)Y+;@qY0BHmakTmyc?)qBxqC|i?z=(GK>VA`+BkE7@>XQ z{*A^)krCmZ$WIRiAR9rYqGTffwBQsJ6zH+$I?*!u`+AX$RBO#{*ZQ@71NygY{|>1? zR6sn-9@UWedMiLgeJ_;&<)W&GBf#hKFpT~XlPy@If~16Th*P_0TUhs?-51UzunIHa z8YQU{daBzgr8u$h1}LaZl*DcFXwajHE45TBHIQCJ&Sei})-^&&va5<9Pt`grb@3l{ z%3sJT#$wJDN_qfY&J{jNvmSU=U*+9X&q7H57dkx1t=Oa}4WIX!6S~cFB>Uw(o95Eh zgvc`CB=Thxf5Ji2FLW;!!K7q*c%qgO9v6i@78dk2Kfd0 z@YK}sMVEjLmi}{?SiXqEr?4@x$-=i7jOskW^jw!HM9%>qp_9-LXHg>ZMPDR8;DE$4 zmQsT6JS(BCEZRIgo}fyliJ^ni0rXukS1Bju=>S2kQk1^SH5!0gDqh0?Uair`~ z=yLAcwsqC=rRY;c-V%k2h@Zfj)t2=v@hKrQ;u}K6a-gtR=aO^E1&;)fO)mTXUEAXB z#1bqDa(Zx(4zMh1-=*8UsAz}LV#kdKk}1$jh~UiJ;wKLB5HYC` zhjFwJCvL2dz`Y31x+jn0H_n`Hkn~F;jVZxU?!nQ7djUR)EkxIhibFI93RkEVVbJ1D zej`q}MSzi+O~WoS1sd&E*3E*^B(zfVTw^CYxT=jtU@Y|0fy#wGz*z(aT%>A4rUk_e zM$AY+IWBnIk3R|5IbnE=NMSXMh;O!>lA2bP+iv`w8LuN*xVa0oShFjN^y) z(-fvokA_`_BXq$@T(IZr#S4e_>=6wMNGJ3|<4Q}m$rHzp8}~C+Q3k4sI#}sa=n-%a zNHRtbubTGjZ#%Z`klf_-(UvQ|h7yp8Tx`&pETAH-QMrd2Fix8d`cSLvD25V==Od$o zZp7$=ydn!rSGeSGkcU7Ru&_3YQK#TD0=j$k@@0?i-XYZ&%$ROKQqWf30hNLZKEPdM zu;M4)4VD-*1lyz`O0u*7x>@D8@%@+NHE&Py^@(m z$cW1*R9p1!9}Y!4ZkNMz{tpH6zvffmzo&q_^DoqaPa7%@rWDYG0FC4Xaw*6{J_ebQ zj9FqMh1r4=C=-gvG@uiOOiE7W;=2!D8OU|5Ls)Jh{It+ zbTql`q>E(>dB96hKq4CjQ-LB1g9rjNZg4@g0Elc#76c8Q-jX|2_F7?z(~OA!T5|W{ z7|1R(R-|81PN0XC6*~KqiR&RdS}E8QyB;t}MmPYG(O~v8>YNX6UmeutEuABmrclwM zHzC^)iM&lEz=|yBM0uqvT#0!aB9KYG60-dsP{idD0`wk$U6u$*C$V;Mg)E|=R zNx`VBdt|pwp)pTN_$kD8nl)}TG6s7tUNrDDqtnK{lKe=xjYDNzfj*E^mLoi6s4B`8 zXD-{=^B^qZ!R}y1!?37UD5;hUByE9+us`VFa@aI3tC1bZ7#lV7hA=nC_lqA6#NjEE ze@Vk8mUu=A(DNGLVd-NwqNi`){!P{#dvP%#0 zb-`;tfA-9nix*xlS0?uQ)!EZ$#>K^zE?ZW%0t7ppjztUSu3WL~ zcFeWPl`5LerXvRqUO9g@J}z#-f;ms(A6BbUY4e8l%a$x=JDSp^OM`|mnDmrq%N8wI z6%}>u%9SF83;X#8A{ee(y4aqRxog)BGN>Fse6&pIlD_`_;Mcv_8>?0jI6BErAn~_*2XofqLwZ_arBs%#e(ZL5VHZ?u{jX#R?a6s5x*BLTCj3O)a@JBs#L6qPd!6UczAE&{JEPptRpGPzMVT7 zHfmT3mw7#D*43&C4k~t9`~ADO*F`N4@ba2BclOmw7unjM46sew`2erKL7rZ-YU}m{pJ5b-(ENt&!k1*wdZ6- z{W9UbcRI#iyAoisOd0acnk9?OF6)Kkhdyc9+>yi9Eyh9by>s)*8D!hZ6UMGyv9dy` zQVXX}nlgGMpetrI_P5cxRxX_V>)_98R@mZ?wYb^o+}fl2d(lf5mMc-@z~)UO2M1@d{d#;f z?)%|k!68rX-5$`f-Tv+CeYDvq6DZBP038J)YLD*SE?+!TBrNcoem#%u*^WvY2twzA zasZ6x?LkJR@TKz?szpSi!p7ZY`J@Sb-hBtT`1FY*J-T%2*{!p~k$L>&f%b1UGiWl% zHn=Kk$>mFDA?)g93ww9#eD?TZwoDk*rNhCU8`=24(*XPxxm_GO(7RZzQIjWr+~sYg z_;bfk_I|H(udeU#1Wz41+NoV@qr-}j+w+6>qZchISF-57t?RxW{5dcd`P!i2I>na`+U6-%encdDu_ioJ}{nK}Y2KpEcOBc-R^YI5?eLf)A z&u8wenG=5c!K85{#>Wll{s98D!7SF8YV`7s ztXwsj zy%#N*ueX>UKY4U)>$a0IF-D8|(c}1&CysV~r{nU)3+&k$2Tq+rlm5l0pVh5fclnYf z-+%ixDjj6u=ymJSb(oyar7M>A=|3PNBW=&hl{1$v0UboF)M`tWD%G`X*A0u83>f&i zmyiFs<3}#;*}Zt%jw&oQbk47b44OJ=!uIXk-|g0QWmI&ZzMqmmMC-Ke-@Vh@;>8wg z*Ds#jyK`I97O$%X139ei29q}nDHa*loSck%_sRLEte(70&d>Vv?%cai>$Y#a_j;4} z1`X={#g}*4JFVd>akp<2FI!QJwgO&k{;P?oR&5H2Vl=u(aj_>3A8yp*P1H|H#2UAg}~z(1wH3kTeY%d(&ovy|?&W-V|8CLX0)tvbtR&A<+M>D>8?hmLGpxmF0LbtTuW z+joHOz`p(C$B#?9b;CQ@FZ)?Cawtk$ToJzM^XZ_UeyUyfl^k(!!$QcOvuyZh!DPs{ z3YDu{wYpLjDkJ7a)~tocZ#fc=WMvTCax>-{nt-;iHc%oM4QVc3xysTnqGlbf*$cIF z=?Ya6k~56H{v=vrr`@gFcYwV&Z{BR&xFI`$GXem^vUi}7XqEr{5o7I2{Sw!zG@r(CjZ?lQS%T=n_i6?Sv>ojP( zBYHimZ>tE2;+xFc9v#KvxpNmTM4^$PbJ?rctU(MU+phqvN=g(rJTIsFVSGH)3NKhN zo3#uOsC8GZQ=1%HCCa_T*sIsAYtkmuqH?yVg<&Wpf+ifE=AO;Wz*|QRjQ2Uyk^}7xb2lJSsJbU<444G-okqP zMuSGOhDW5<>MmZq5Od?kt4*3}NX)A>7AaZQr%-Wmrl(OF0x46n)Z_TXm&FYn#hdU3%qFaK#H`CBfLrCM8RQK_nZp(3a+-Ht z%KXzQn_nQG0=%y;AQ6~P2y+ZPg8*x{18;$tm}~3`#?F3Kszz3=T&2&a{lt3?RM6{F zpCy0WtJkneV>`5OcVy4bfj#>Q=rg}`y0_l#Htn}*jheL#E?$vNB50N|soC2)a2@p*>PGx$s zL5WL=M9uj4`J#aZfW-krNdMQcc0ILQo?y>Kr%s| za*i)2o73CGBZRS(D_0HXJ`-GBM&j_@k12!;`=H4 zc!^Fz*tdYIbeQ%SIBFkSkr@FU#3@v^mt=>H`E%!yL1p`n?L~?dY0HK*Y z_U>xazCHUN1D?1Wv->U~he09Xbl}$Yt6AA;)n9o9w3zYB__T+2Zk#&0b@j?OJG{-t z4xzz84DIfX7*+<)5}sHJgcK`M^hV6hRcqEMj~Gb~A_{)o<6tVe2&oAAX5S*kNPp;w zS7OM>$>ijc;X`azBe=i|kb&2@0OFkv0NZe7;fmU>g%J?(C>v(7o&C;D>ksVR1G|FI zfE_kUasERFmkcK$IaX4UN^>1{y?FnSlhF}TvzErsf92{`cyXj8rSIOg-=;A%YuP6D za`udCduQ#8wWLfl#sj=tJ=^!bcmf;3Uj{jweJN!b!Gp!mTkT}3{Ad`yq#2+Wuma)h@+9+*dSgA5iI=|Ds zUHg^|8W%2625}G71fm~m2b<3H>x}t+K?S6r{Z_3C6)p;c0sQC= zuFwoTtWc4WjYt=e&;slV2Z2;*N z!5Vx<+o8IoN9a~>qMq#)tS;bEgj`~vq~}WBgq@)-XTd^L&To^)dE821bKo`5#!eK3 z0@)F!`56z=gxY$>|&b%2$XfQJ!q*4n6fYd6WhswYq)q@%hVF zYE`JTXvNCnRjQ&ziCVep(t!g*zWcsHWVJWCfAYl_gSh+jsT1bHMf^fS$XGjiz~_7S z?uFgLo!~XBb)W%idzg-iQL42F?LjU20Cv+XZ%4j@`=29LehEc^=g}trPI53OGc}sS zv1wP5?IDFjO~mgpBmW>>2#oSwB(VSI!*jE$=y@GX19C?^d6FJr(v~mbXK~_Q&2Km_ zyPTP+4XadKv~_FA(q&Lh`1l71o+3pO<)P7}XJnxV_V)J&ddy5J7w%_SBvXNoj>EM^Lo$%i z@PeQnLN@gibcZi9iTXt1W7*CvC|JM}38R~; z2ql14fSb0kXA4<`#%j=ioj-XKOK~^#L_GOb=pamq8IYf=>``G?5Gz!xcy;$c_L>-dCDKzW|Db2ZvGyb*i5H7;pdsLGzRm zZ8-H)2{H7rA5W;g^ckIt(JQ=AQRICI@0L5TZi%>!SPGuS-Muq;!uXe7uF$YSV^AHq zP?(1zK&BfQ$EK7aAwg8d7QjgHXovg@6w>+lU`Hk8A--`AESDO6Kwu!%qL{Wjoc;kp zO3;RLAS$w*u!J?QA3Sh-u`B@AG&)+%6<;0u;1(cp_EoHzLZhWNo#y?u@}+T zpqDHK;HgW)4#lu=9uEWz2@T7|nMQBH?uvny=6I^2r#BcF%ZioJVc|uZw{0hHg_Z;? zzCwMcOVW_T5F`{61ryb&Jn0ewsv{1QA}E7kAv_LxBnY2>dW}!$o+v!OYAdA`S`bR!mG7;*!cu;RHXCr#r_E-xcqKe}vTgI)$cV}m$Vg7L zr6y`}vrJ|$WKpn5dWASjeIiH8WX?>_#DT<%Kp>a>(Y@Ol&youG`;*-RNyn^lB|naL ze5fq{CnNpvqtT>@%&YaLJH?j}Bo4+;(t zF%a1pcWHPA;g>X*^2~w{w}F5}Ff}tHDZxu`U{lIlH)2YaC`K$yN>YL&JKb%y`Vij2 z4si67u5*iG5wFxoNz=rLUIZyL*BXEO7V8u*FE;Jen9SPj^ki#Bx-}zBW5-i5&BreQ zu%%A9J2(4I%(a}XoWQ`K;E=G0>b2oipgc%t&Ca+Vd)3d|%fRNr_-3&7LEu+OX0*24 z?2HT+3bbY+%qk@%J1fItwya;dyjamcI%!>Q~od3w2I{+Z?R@+w{*5-R z-+r}0$A%4CG;MO}!bNeD%gt?Bt47nv%FS!lXkM-Af;n>-6hqs4{){6x=YtL%y1)0{ zz<~oj6(Y^CbimuX_a`5=u3NKn(?%WMXqT4qOjaIbQv!g*;OA`hTGO4|wz0qd>JBjb1L>t;1W5D!jaD!y5EkpdC$$x?ZQ*v|&S=7A*yl zUD~rJPc*MmnPAtHXNlv74352d9U2ZF{(Y)b_U!D~8 z{4jEO+gddb?%ylMR3m9kk+Pll?%in_QFXw8PpSFp#fy_hjL6N(L~GXKwWiIhN4AKJ zs8_qz#PMT!j&b+zwy5_Ce(3U+!9@R(p#2}3srk+RcNBQxfE$S#&9g`fdVSx~llI+w zyzkc21uHj@nIbnU@uwjJhxG4r{&LKqA4ZKEIozHN$dW%$Yu4Cfubx{pZuF8ZTTWiS zRJ%#zQ6s)ZvrRREwM<$zlXPLW%w02YhQ(>a8Bk-_`nug47R=Mwa(qY+qH#5B()8%n zSRcP&Sv%`AUre63<=VCHzW6k1%A{-OFCxB*Hdr1O!n|(#+SNW9Ysa=NQ&OM3-R+}8 z58{v9x>YeEa@owuW=FOOW1ic!WY7Ksv2pwE#P{kym?b?5`T>3V1cU_dy>owm?7fkb zrn34+cn6~hA6gT&m`}U8CB|BfjqE%FM(*h*lkq4Q_&39N)k~ zjpkue()c;^j>g{GcJ0QKxY%857I~AtL*uU9w*8UVd%N$Z?z;KJU^ZEt_M3-ykMGy> zmlaX_FI?Mk^G@ON73@a6UFSq*cW37g{^a9thJSw|=63&4W5y2X>&!?5w($bNDaFK! z#F=YxXB(W^iH~CYc7Lbq=Y2cB+chY((4hy9BkR>i=4U7SCS5uoymocY>ZlVNqt~xk zOa>A(7HBUZwdjMuU-7>u*^4Ii{>@*9f2}pSh@KT@05fwyPeXE2N~ibVKk(r3p*!~@ zs=Tyx_9QE?ZW2Wd%+BI=dfRd}*7UXWXBb>IFI-3LZi`m8e!(n_Bg4mT^K-iEz24;T z^=k&dU?+Qxx^3T1{CUgyi=Pi35e94?`HWDc=`ex=0+Kr6#qJ;|L+YnwLNEFHjENntT)TNu&Z!F)i zVWo&jJdQtj|HIhhC(=_>xJ@*w2wIr+bjBKW>!01X@80d(3e8=qDamW+&x@#28)&51 zgZp=9O`A%vj)jJtPCv6HJSeD5>lWDk5)%_yfr{uWYH)DDId06)!-fsR2d;dDN}apD zhmjCpkwy()J$>#xBft*N1FxeMt{ynX#Ubn3Wu`!-R=0+GNJyQ7KLLUib>SFetmGkfv!6$OhFcG?}s z4>Gkp@PQc=j~m%V1d_OdyNz@Svk4fBRdtNzEDO~F<0s|ZQHgZsuQZgkk$hoi;qd~V-b4-1eq~?YNJ-HJsQH*f_%L(@zI7No#x2#BbuztPMy23 zDCpFwGb^97C(m*Vlt#ycAyFi4;5j}0qwXIxYT35`z%Td>X?O0_Uz46Zf+iHep5gCp zsavaNu}T$@YQ=gh^$VFP$y!Q!j-0D;e?9Emk9+nkU$Fwblf+S;;X<}bRVuUY;r+la z-Kb&1^Jg!hv7sSu#5l=co?5k6>z~-a|G~Zc)?7Q3N9tcWe?iq(>SE`lmH0ci=gpeQ z;}?~T+wE;K77Ph^vvn&vlJGQ%EByxy*uHW_T-*az0x#Zdl(#DX_&+V_*fXMPkAeXHr5$Lk8lKJ+u6cXKu z_Z&7ijVmrTwoHZc{SAVKz*>c87QuKmZBeQJex|s^RBkWga_n zq+Z>IG-J@4ub;m-ZT#3PCy$z)_L__T0n=%}$1U3zc&V9m{5I&Mr> z!ef4dK_)%CcYNQju_L~VC*ZzAdu}A`MeLBG*_RaIjGfM`iC+FVA?f43qzr?7k)~0c zCp;5_A~>ZmNEQJH_Z@t#aWf$p?+UyKXH79+y30pKRC%@D%4JKuv@X3pcm0}`B_pdv z)~>@(_(Ue&j$J%@6ck1o!rG18IQuuQ{ic84Ds>yZTE7WeChQGGDpiQNepNW2^eF$x zr@%j_zzYZ5%uOZ~vSIU9Eh8hUA3C_dW8((XCr@G-&E#s_xG4o@Pnlx&_Nr8ge1SqA z@B)KjV!~5#6+o4&H+XsZa5XbC6X6{X29k?Pd>}?Cr~Bg`pRAuhH#aQ>u$w<`PMgeyk|NhXPeWgm4k*aV%knlNtShRQV-}5O}%*)qD5Kv-R z#0zin;>Aa{Ztc;xKah%}^xOUV1%-qzSg-&c{khX8*||OONrF2&d*{xbwQJX2vT)IW zPy6C-MzzR&4!aFor=M>CoB$UM|A0VEZY~H!?F>-{g+rmuzA0*DNSTs#Ua1AUqBo+p zrmqrwD7}Id6DLj_G3@*JTd~2x!FES3)bU&w?E;zyG1rQEn%)vwKXl08ppXJhTeOA+ zRH;_I^&4;8fB2}^`(4Sqf>tOcjz(c{q5rA zQD6NuJZi~Y9in5dwQJ|jy+0dp_}c9^-s$o|o7T@BJwONoCV&&L!=9U|b6A;$+1Z)Q z=-|)-Tt`bva5VNm5^JDsY0#{hcd??|*R9^ab&EaM{(AE^)F88yS*hGRaF7r~ltwpf z)aXMO&h9&T_0W;Cb7#(y9MG8EnB;EV{kTV^%9a8_(3OiWRqHyiW836!zni{lUBTid zN%)Tw$-u9_A2i~pO*?nhXz|*TMRO6W36K4Uyeo^#0=8ks0aFzWHV_GA1t%3k9pv9e`*^smSh2 z)~q{qJ$A{K9WzFcKDuw4$(2n9p%31HxWT|8#OoP$;B-v<`KL1{kIY%T%w_R*8VxmH zt>1IdkWtfSMMp>7JbvtU>{awKVDjLh-+k1t|L`#rs@AQyZPP}PI(a~Dou9W~R(b}* zmmqt+Aw4}cAixiAc*JBb3b)Ye{lW_N8$5W#mtV$Qz9d4vNc|9*Trpt3ff2NRv*Z4w zXAfV!RiJ3;&-)M5yBsE?F4r#JgTf6}gy*+Ihm?D%^ut?scW&DB^kLl1YgaE{xx@+! zDxr^JvE#DY_wCp*Y3Q&it5=7WE$6Wx(f}pso3K8ev0&*0D2>6KYY~a1E(%-J9Yll>0@)IPeJ+z*9#Rb^5D@!P@YEfk9-RJ zD-?L)fLrM~1&dK}*lcWx<*HEeW!uw)lxNQl?Arsb_5QT4m%r~90|zl@&z(Ju4iuML zhs$2HMB(dKE|@h=qsD$W_9mbc92Sn;CAf^10SpvL5Zu!D>htNmIkVS9ul%@2AHCUI zWLg&ZASPlQksUSyt-*1mHu2F84KLE5QR4&q_H#2@8H6-uIyo_J-@Xf6+HJWAh=Kly*SX&E_v8G$m}%9a*)a$B&{&(c(rQ_HoC@*&P}h3I--5B)FZV zW|HxSmoI<+`gIG;!0Egbd#l{b6$Iu0FrWzZ0rgy(f&~jLTDIcw#?6x_Od$GPOst|4 zFwj*(QxG*7gvn%xs9YtWSn;!`P9r$eVb&!e6jcE`WWF}K9g{}?xP9ZQ^{ZEghlk?? zjmr(rb_@&!N12i((A(U(bH_h0040sl#D}CJuE7xLTZ{DH+qC2)m zTtpA=+40pU-KQ;?U;mXlREQNqSc}k%fs3((=cYHBd-Uqb>SyVa#l1fH*vLLUP+Y24 z=#7JlIKyBLmxTcO6YK|=0S+BLzb7R@-?N=b6;`BpZgLW;Y8?AowQY6f z;NFX;jyN+?=FObi41*yq+r)xELdoOr3EpNOFw>iz-br~7clN-cw>ozvF&^NG2QhGg z`!UWVi+->Bhu|8&gYXDmWlNXNc@USKnS(WN|IY10dv+i3 z)9~^o3nxCflNcY5F8J}?TSxb7H(0Z-o;k^G2_Jm$0UCtn&70d3pRQOq&yk+GZ2lY~ zZHkvI7k~He{yn>Kwr$(K-O7b?Qy$z(e)NERuy1$nf)W<}y!cmuQX<5%ymytWP<_;_ z`KxD4TfclM`d@-fP@!P{B-awDP?F>B+&q7J`jlVN?mu|zt#=a=GbfB0WhbmuU=G#= zTe40XGjh$cc~gHG52YVJe2|%(NDA_UJGLg?yOsSkVfwVGUg6Hh|y!HDZFI^t> zGJ^R3>_1}EbZRGe- z<%$(({vI!nuql8Wi!U*jS}Mb3M&F@Rm&CjG&L2P4;hoM1;(DFr%#!Dc;|e-` zlCZ^V>W7m!bOVp88PC!A4l0g9Z}$x~;3{_@L; zH6kOc)$H-rSKT{x#DAoy$Qy&PM`KqOG^(7D@dTB z+w4uwRS|DVlO$$NlSCgHOjCaxGj-&c!jYBx3?AIMTURiN+A!JiJPH&D(HAIi_Uzd@ z^%|{Nzm7Ube)p}&c(h%*!l6TlQd3jE1gr(fT(7ok_hzR~BwGqER_y!9Q-1pLi?Jg| z8N&+Cm^aVp9LbXz8s^ zp9}Dsy@Ah}a~8B|-Lhw|-ZSUUlLZU0Hs~l^+9TU{wcWncT(H1fIHr8~2}+4g8`t|- zd|2!tkJ}}wm4v^dcELFY>jKFXrB1#1@}+ZQKOb1LY89vl*++c+jX<@=;9Io#&XX7D z1}-V$PD5NM`YF8;HMh~nxBaL6vNO^H!ix|r%1cIb^cSXwip*p(5t!=MnJ14PH+jr2 z#i~YrHsqUkx^#nsrz9uFUAlr6o@!<3mVe|^;9sDC(hdF#<^F$){MB-mFc#LlnC#LL zlYG4`?53vE;_u8VmNa%&#GbU&jNrf!@wP_KtJkKbWn#@k|6tZRvU4)6cBhYTAo>x2 z6d(b(nZzXULley{b}dnbXg|rnZD0*8q8Y|6aks#YjLm;h&;roXK6y59YjbFEC*cJ3 zxKcb&7*No-wy#+^Z}yz%tvi(i2?D!dR^IxX^|e-?OIK1;0)zYsCW7n02@nQxfS?aa z&r+}ol8gfd2%9KoXWM*z0%U=&ab={X8qF4q1s@Dp32F&DO9uJ{bfRR(eCN>dKQbia zCg+MMfS>|xgInZV!5I!6JnqWWhQG(ipMDN2;D>UFN^A}isz?$%wq&HMz!v<5(bHFH zOQT7AnvCuxAT)sSVuD51$Mp`0G&A)Xo{l(>2nC#Yhi7{E5SU765Wec}l*F{4uuuX? z+2WWmbFqJ-$DleTee)k}DcX?LRuoV;Vxw|`D*(7285RYKJaJq;+^#lFnv5SizGAJq zc;}21^fVZs<#y3&sDsBN1y!RgDOBC^gSykB8>cY} zOBdz3=RPrpN^kmHelk(F@1onIs^z!cT2*!9wDNz$K;<5i`tWQ4l-*LJDv$T23S~+`R(>@d>~p zW@Ea02Uuj5;{Ty0CbAmA9CpbG>XF^31L0FoHfl5^pdzzV@ZP>K=jkn*HmHtwjdhK!4@^G3K^8os zN#jIZig z1h_HibBQk@`r6G_J|b;H8M1b;+?SOAJ4NuG0*d7EGZHoKC{QrCb*omu9o)-gGTNa7 z%MLO6i6NF~B9NP%=~25AyB`u34ts#H$oB3I(bQ83{xUv+p^|IYWXQ7G%sRc7AGy6m zRt28vTVQw?=#I)*P+RqcRgEV(2_sUEf$kHOBg}!Ju(*L;K`I%q8nsLK(^tpZ`!c)$Z-U70-Wg2@0>s?$UpKa@PDAd z3ny^{$5^hILM)b|WaLK?JWxsII* zOa_A-4`O4YSFalQ{SPQj_zkhq1lTaKrKEfnEZ{^K1alL~8nmDykr6QoO4<+(jB-p{ zRfa?s^n~~qu;xkpqtz=G<>cfH`)MroyGaGb77b)-rH2UEAROHVz`$rw2GkI|7EVAP zsh816unFJA85}8G@W(DVh_@f7l#$Lk?I0(Zqb!oL1OUb)C81bQ)jp6OA8T~CRE~Qb zIiX0FD7_Wt1d6$>Y{V`(?i5}NB?aqYCm?vP-6qL7=qO4Ckc}M}kn16aSZGC|B4V#Z zlM1I1YQtHeo-O_1y}Pid(C{##=_Sn|2@{2z2({!9Kv5wX3kUUG8jw6+uvPVp@^XGL zc%CE8GCJWTZsC!F`3hfUDNJ+hzZQh6w4t~S62AUCZ z0k{}A-J`Q?l8G0GC{O5*(`g||sc;dc91#~1SdS0^*_Lt8rQbOdH%Pb=gy$!SBQ%zt z!z74x1nOvxjptu|rQW-5wr^LvX7~5HVrHaO49f!l!j`zh!$AcZIn;v!F8Tc7m8K$f z2IrZQ?Cbi`IT0sVIHw6$q#acFL@2B z{we+;T~HT!7oe#J@u`8{7)(Yk>dgiy%L)$?<*{&SKmW+5z<)=9KfLAtU9tZ`zlq0Rfl+BAbAr?WHU2}VwfiB@x6!V&U`uOo0e@m*oZb1&8>h1?Qs}T-Z*oCRC$Uh#}PEev(7HPRnjw+ z;?VZ>`GvzHL74q=xr$7J!|i_?S8)A z;YACH)2_>TXxFZVb7!A9b*5<1BEg{{Wc8adZK}VI_sSJfcAKqOk>d0P+=rKdzw1}6 zSRJ+ew7s=fFJCNOx)J;xn&Q%}w`VV%Hi?YYGGR5EV|#ZT-L=K&wx2k-_osfn zRxO$DYcWjxW>EL8??f$H80zcyU5}n?R!2)9kI3EH-k*HT2DgBCzjr#`yc&b+--_8& zhxO}OFf=s&!TonzwPee1w=FyBmkGVQce@pH$!5)F!LB3IoeLS+{^`f>Px$WJAU~hS z4{m+lsl(naTfIpWC?XgM_wjWRxDv%QBC6ZHaQf7*-s_y5mW<#(YCzAiBffIka$>K? ztX{ep$r>!mOv_j?aYA-VlHQerO_2oJ$eY}K>gb`dBZrUpW(cW{4(!<4xotDOEyJ1l zZ29=nw%jz*5}I80h0}k#arKIstXmqbx0lKYh2RXT$mTG>jBDM}gPs z{Wgv3ty>jcs#vj82X?+)Co<{o9cyaRxF5bXlM~Tl{c-5vF+U75X>xV;^a(>gGud-T zelvLPuahdhR5~H6loS1!-j!Ma(^HL*6jz1SJu7^;+33S%}Eu0j-QctSp*T z0cjQzJ(qb|j!~g3lb|~*5LQo8Sh~lDRye2TJSke5L7rFfn7=QrxKEi#DJvkS0A2kr z=P*|>@GA8jYaLmCh=fmL`A0ql{y7DHSM~k#qW?2Wh=`=L)fm4%`k-sSe*GM_jA~_G zI(h7H+qQ3@;U--L7DI`I#U@Yt5EC$3&6Z^_fgPfwma z{_eZ)P?(uHeAv*YO`AfcR_)$6uz!EqQpKl@{Nd**zqV=LnfX~Gvhu~Vrz%&iLc5(h zcKG_6VeACM3LHNRblYc-pSZMV*Ya&!Dpe?lQ25o?Ls<~)-nsLWk3L$nXnvnQy@LuC zvH?TL+d5rVM*8fZM-Be|`>s7c0)ZMhE`X=l$;8h_pwKvTxr$+}}~$ z17GawpfjO6K^jFh<@|ckmjyyXTeN5)_zFW+QZhc*vJJbiG8rgHimZxWif!g zJkqWFF2xYw*$ETgqm0jzJPO?Bx_Amof!}YJb7AnXNC9Z-RpPYA%&Vr1dW$F~CDmVG zUi~i@s8Ea=o*!sIPV!t$0D37wb7B~fYWYYjcos>naQQX=$fv+RrGUJzFVrD|7>iV< zna&j1;I#~$x6>3-x=z!$2dSuSAqD;#Z2#?YBjgf8VaPGMvkclQbsM_PfeyFj<(DHW z)N5$=^UE~oOIE4y@cu0eIEinB+fd{6Rw*W5G|lzv*FST7fBfAWOwgg9_pe%_NR86P z*rVn2nUmsPr`6SM+$7sz%+;H%SV^5`qstg~9wL+0Xwod#;G3=Ss!_KQqFwymM{0jb9N6qQJtJZ=6n3Jt zyGRP{%Rpo-I)l|@eyLFt?~sByI$!UgB8AIGoIP>IfF}crdM7|HGCpd5E2?sn!KyWq zMVcUf!AqeGodIo}R=uuX^A=WvC0B2{em%Bv-g4ckk>af5cYz=``{7xVNK{0SI zh0qcJ<8-srDJ4DVD$+M%u&AeyveD_w$f%9WYMAn$SR5oZqj=Bi{{S4=||MA$>b;|QWE)TT1)=lk^OQ?oc}IpXf$4=P_?q;F-S~5s{&!JBg-|+0UdUV+AK# zcTQ@mcTf=X<>JMQwd>a{UZM=M=hwOOs@1AP15#m;>V^C(~RRC+v3ngnJrp`eUv>3aSzIreTnRSF%ROHXnsL~ zUOwLO@$p~;H6+A8khMEd&lbQgHYaeIQ54_=;&x=9U=f!RH=B{i$HYXA9HBK@5Tw(d zJ@fVTVJ9ue(*y*0QLPB_iJP+n)X+pqylfofAq%#7at0T(V`VN*AkikEqC z_imwLB~XSFe}C}sq3rDJ{-1sEio5EmeVSf~jAMLdVM@UTQi#syT~ z0wFJrsWfCnT?w_QM%6=4rB!hbyTbn=f^^3VUR6!1LVe`^zet;$700K4TSmc-b&7vtC<0eM8;8QhIuYs&6r znNJf=?%jLy)Cr_JBic=bDdl{PWG0zJBr__0Y2<4{?lTbof))V>m07<`bR<99wsq_E zQ)e1AX=e5bc(wJL(b22ewu*4)EyO^o{1u=RaLn|EQy>Gf^Dat(n)T`-G0psCTyk>RB&Ry9(9I1#o=5*vBJcFO&72(zvEnK8T#qtZL|9b!W<%hAc^JdL16;X{$ zNhJPurl#)Oy(={#e#yMK%yGB^8Ab{gDRle#WqW#(J2%6qu|2+cZu|PEoRmj1ejTsz z52#e5c42IrLBVTQE_G+8E}Am~)CBOr#n6zDySHv^icbw1J;6ZlE#Y^G!P5I3hq!9m=5nk1STOkj1%G4BU(!TJ@5rQiE}ux z!@%T~peprl2uvfGOO^uAS*x)kdKxs=JP5Gc6)XS%KmbWZK~w~?3ss7fjX$m z$;fS7z2?XtM^%2cp^`jUag)Yzq}fIuX)3!O`9~^d-w>k?QHeSyHD&4jUwqYPZ6L1h~Th-lb1zp z*tF?&+?G4M8BwEV|AB)DamiuUA^jQ*BL8B*5|EII_uKRtIk~n@EnBy1)Hpdk>z7}r z5g8v=xJZYOdwtQf*XyIt`YBB})~kT(w-? zhCV^z=oaQKUev5k?OJu~RjFR<-7fE4yB;(4i-Gs=-(!FbQ@reT`ltm*^j)2WkCx_)VKf5pWoZ!H(?~I7wTLFOLHt7WL+U@ zFnGNT5bBwXBr8_6Jtv*|B7gqaNpAOa_<3)CFO`4tZ=?XP-9LNm|KF|urE--Z;Ue%D z#S<8vf^2pI49MhD6IGU(PFHGLG8+Ye3moiH$SPPgry7mTEy;=4OP3gK-gaJVuopa$ za76S7r1sC0UE57&1az)A*{6ymu1}sm!x7M6B0dF~4;a>y-AHB&cY)ucjZkPTJ_R;5 zz9R9r+2rMLo1_v}>S(O(c0%iHVlq`;4|cm3Ql-r@PglEJiAYi_PJJJ82Z zL~8W=vYl@g-{GltJ?AgxP)^tqMYo^^mJc;f0M-vmkkNxr&35H z!vqbeN}z$I4)#;HR$5vb#!O!?w&x^MfFUzG%gfIXP9_mX03u5qC?#Pj=mkg#&XWLr zMbhUvC|_{erKBiWI2gE{ZQIF|0>n6o3GyWUkmn~sF8~WBTfr;AJE#rRaLy@z`8^MD zc*3G*kYehf0hlXVB~emIM+7kQFi+kMSU2b{C{;6k?T;LH(_6&t0AiOrIQ`=7CLf>1skm*>D1i z{+xp$*mRmCrzRur!IzDS2mfQMf9vf({_;QN`wOSzMm5b-B?TNXKnw;-JXwCqCWH<) zDr14HH|c!?!9VoKci$2$Y*06|)ue0&pmTc#hR_9KK^X{&cAL$K8eMN;O9bj>5sW^A&<<@bIaZ8jlg@-L zg2k`REm;7)g9>nqT@(%i030R@SZV_H7zjX*0l*U)ZXtPy0s}Rt1$~@gs1i9veuR^; zS)#=PW`p=hy1+CR0~YI$0!1|TMzL**XbRWDdxZ|WeS(7ENs`?KpE!-q+uMhHE-*IY zk{}V9ZXkn{go64wTY%CK0=lVL$KwEzsZkyU7)keK3k#jyL4q$M8EA>X>oqpsQV!6Tia_H3 z-GM{_q1552BkwONm*0zfc!uXm;_S42V`C=I)Ac4BV`FY?+qP}nwr$(q*ya`6$;M7L zw(a}+{h!b9yqY&NJ>AvaQ#I9f&Ve>g_$neb7P0vDFf2LQneeuIc!=5cu>06|Cbqq0`bkwdEX*0Cp6LPSaWhKb7mx=H!BU`+GF;SkOeE8U7OmlTK%bnH z%2=s-)$RUyR-Er=Q1^5Ev>zhgjtOc)?x1)AB@iBLuN$%;z|)Ud2=`A0iC?7)Q6Ga$4AQ^u%02h$%c5BT>(2)>={W-q+h0|jF)(Tf8csfjtoZHFFbWqHS{6>oXZvJx5oALJASP0yiR32`D< z#$q5r8f2P|ckmpbL;ct%AYpfM%6)`>f4e4i%nxVinP`U>f;p-8@SK*LUc}E!<;_Fw zN6v)=&{aS-B~A#P#Z+<*7EweDv8BbRf@n2ogH6YTx!LU_yM2)Mh%xt?`+Q~C3uGPS zOR3l>FpPNcPwQ`B`!!)EjRfnJWU4{?WrPxN>Lf8=phmHXrZL6MCN~wh9rzS zIX=eWen!F~O5|=Y=KCw?v2l+uT#)JV`21UB zi9|jPA#{nq<;eIanI2U2Qs(If`c#W~v9vhFA2CwhVhU*y%UoqT*zX%0mQm5l)TM+n zQB#;h)?{}yQ$qTVDOk_lKie7j;xd^m{Y5|ST&9#$kLeGWKEOty=DlVXaVPnfp3Ccl6+iN+L^TD)nXPUbj+67B6)wh%M_6h1L)kbt?m z-zNUNRW+nW8I=*UW4t(Mi~=_oxfjAiX1y&>_92*pDs~k})me!+)&YAFtu$(Y#rfw= zt~Jbq-7jh)8!(eK^j6Lk@weYQ-2@+Wo2g%=Q%9t2h7+HBO*NwKp`4yV?C>8to+|@1 zeGf9ltOy!q_J*kxGxO2`dX9M)QK+OuG?a_k16mA{9?`SJw~4xIfrNqB5j|_f`rsAf z1|3DskZ?!33KheTH;K-AE;$Pk0+2v#U(kODA=YlSnvAETyLt(WhN&ZeT4=lwOD%II znuB}4%*LY*L7&{-;+x}OiIsRXb!>q7)16MU)WGRghLcJUf~z5H$GfjwE?29$x1$zo z@)%4z5c6C$B%H{SV(FM5Ic45%46mS9AhK=)+T5rfv}pjhpQaovxbXNld-X9=oBfK~ zDx1kDNaR-}dW5@(9{`h@M6S4G%h8@u6Zo`U=$J+{7xT2la@aGe!di$FrQ^+l5br!H}Z#S`QRg ze9_pMtP)%bk^;JXIGzc)?A0INUn($N9vxB{t_HDk-k<9&HQ6in8l738JiF`Zbb1p}AJT_;Jb+$@tIZlCBHRJMp zrOo1bWJen}H2fDH9Lz6D z9R{DvM$y#C^lof#^Wi(R`q;hUsBtkc83O2#xq8_5sU0#5I0Vf$FV%+$G^~mWRVvle zIc9MLJq~v=zN`Ku(P(UTyFIs6_L|5zEx^lSr13Vo^zOiL=Jj#hm3P_$NteT%CbKy( zD_1QtR~RTrSdQ8E3}i{82eLE2MD8ERLBqg2ShRtlNB4U?^yNJq6c>PoqR(WzpRFP_ z#=Ss7l9KS~arn!bh@b_}O%3I7dwbRA+gms=K>U}lrqORU*@&~H5S&P-YoX!-Y);av zM#sc>JJ`j(XFQLAV;J)#_;Us#$ld@;$Owh6#hH zi_vOuet#O+H3Wx6y%DL0x}h40sgD^sT}>QJpULxkJJ}uZY|1AkQ>oDKn;e0kOsDy0 ziB7L6)hG?nmB;7TWbuC*akWlLPNGt+hAI<8b9=ow4Rtod=KUJ;j~}efSQ*q}OlQ{;~#=;CQFg zYGQitLufi;Zk5Jb)9ZI>G$@fWyxnZ)lE>lJd|L${Ovy0KH`{Ko@qO*4Q1&iY>PmM^ zz#0e1MWINFC}b($lTbDpB;;f7tAvt4D>RVa>lU3~soi!oX><8GZ~qn~yhEOpm`=j-`KBb>IJ} zFKqvQx$1FCbrZi91*m|ap^Avsi=cuR8QGLGVoO&`enrx%UC^>2cZxM2{}vGy5J{<> z_r8YI@wDCfzR&i&b|fhOw*@>jW9E5sGdC~uF+b++{hYQpE8N{nav1+>9RGT-)rP$^ z+xeN+%XizBtxo%;=Qb$1X`I<5F_!kq(J)EZOT^B6UgW;r@p9oTQrNKhe%<|9v!ZXe z()hZMz|Ziu&HX9&(7mGnu~2@G%~~Cf*=sb+l<8*?iVt|MmDKmiyLA7$>r7N0^(Sei z6p8tWuVm{w4~4wOYnh^MJx{(rgXv@7x}{p66Hm;H*ZIjppZ@Yww2>(;j9NvC(Re>jHQ$uk~ZnFfetx z1+40&435v{Hy5iykG<}^casS5jhgd$Z_Kz3W+E1z>>#p_hKIhGb{Ds~JD}(L`lrjS z zr~Nno-e$YUR|dCz3lXW0Q#aS!6|6WM_4>$1nP~5CmI8xM7!mUt{{sJ@mT?~a|FA?o z=ap4?{%@461DL*_Exo__E?O98mnWy+&P6eNUY0e)3Z>a%k79ps`wz0h z9#3ud_+7?gq(2&JAQT^WlGwJCgsh^nKaI_rLiv)a8=Pf55`e5QfNSu)Mqtnv z9?ou3DBj|j8-8Q#a*co1Lfx5gV^OB?y1pOd z#i? zyYunm*_`o#rn&dWYMTe3)&Bd@d|$iG=Ifnb=k)EvKm$3cUvF+R8jk>|fc77f;bOgY zcfIO+Oa_Zh0gq`fX5;%y{opVB`LbLlM}u{E92`?K@RM-JzEhCj{vN0Oo7KwW>V}!w z**eU&0R#CIe@Tc1V0Ac=Q!2Z*3$Kb43`5a&)qCB;mdmwNoBVz9{rQ@?^6jj#(<7VB z+LJzZnwlynOv5BCrNE{0(CJJMxQj9%@$slqV=nbwdR|0VVVOV|tNT1To%4vK3VZbb zNy9Kcw9;a&Ci`Zvx#$Vpi{`+}M8^q?{9Im}la0FGl>Ih+&s}h;5<$6<&i!rt*G|}c zkAvOqQQYrMX89s%`?aG4?%Gc5LY3Tn8VH(v*bK$_>;_7)#Y<0*YwxQMNFtv$=zUG9 z&v9}5Xb5-NHICd#xeHF!L<-?e>O!!ysrCf4=XNy)$TLGT-}0`M!MfhbB$UR=MZo zg;=cR7M_OiDWfH`gvrSpm&-N!t(R7tHNt7FPD@xA*Xe!a&p1B7EMZNa|5L^4h^vG! zFpQgb{SXK0=`9XE3xCG(DKwfRs`iGh)g@GFNzzRo`h}4 zu-bx%U3dF%du#InA=tVj40u8ljaAYV2ja2h?r}Xn95qgav+>XmjMNNCUase!fkY8! zf5`J%KQ#kx2D>^!tJGbs*BZU6mY+8r?)NGdgfJT;yU0L)gD~J9+~YIzS%v&corxLF z)8GK#sy=_I?K%Ru>v{T~Gz9Z`?AwNcFt`(P!jT&Pn=K}XyI>aiP>yo95Q)I=l)>Uh zElD^I&urrNIc%w=&B9-AUQ2P@?|Cswnpfix$7Nu$KG-`c&xWp)2fiY=0T{3^HY$^(kk71HOkW>~tN%(uwz6h&p4WhO2Ggx?>}mv34co!0K|#q-=X zCec^)9wE)z?YA01*uEB8R=yd|H*SsfJ716Zwew_boUj+G#4qhN+T_Go7gbbMeSI!( z-0|Z^Om&6{6h0%gFQ_-^$>n{XX1MkJ6d%gW^E|ok+y3OEK;)m;X?(walli<|y_>`F zbDxMYGc~c>>CB$ttjYn)zq%9XEpjr`#-p?K@RRA`tboT0O73pY1viuW3SjXL+9u_{Mq|n24a|3w)4L!N~$B;0PHrR1Wlw zgf|32BEoItR=cFr1i@OJUkI-UPX|J+#W0}*JrDGP_&4s7PbmCjvSNZ_=e-fVv~;~UP0qTp-S0$Pa@tsU^^EO z*d+5OXy@>77hMsFFeiFmd<3Q9XSgr-X^`*bhO5O!_h|826Zy=C+u6mMve~pz`s`7S zw(Vv+$RZ}p>(zUQ3q%+95EcK_I2IB@GcnI}*py4_a}sN?gZi$@O=d`-@|7zHGQh+S^cw>BqLz>^u zZQ5GTTwRjcZfXR_<$}WKKD?H9FW=}j`p;9s2y=HSj_hx6CR8@Oi$c~hiT${Z@|`|n z^R@0*n~|j@zLV(XvU%TaH{i0S>qGJ@^!iLLCx5?2aXKA#g|H*ajTBp}M(DZ!7;u~j z^Hb7db0HFQ#vh-S<*n9xo#t33kM=z6CG6xVg{0U2u&@q1o>{u`N==Vx1=cyhh=EoNu`K2e_WIDBk-DI7J}FZ%Wq zp%za{V;6vy!pxw5_qz(e?on)a9o$E_S1It0$Aiaa9kWvS&Slr5TW@?CKE@MA`(e9i zv&i%Pmu{B#aVW`i2i<;+#djV9H8F7@?UF0T^R+}B-|t^Z**A0bQ*9cpivq^Yofmz9KlxyL`;Vet33=7iSd~r-tgf(|LWw@@ferT z*E?z~fd$Fj?dfPT6OWRr|0HAS5Z|Rmp)`$`n%G^OnbZb&Xc{U?3L4Hqm*w2;lH=ED zv}~T=%VvK)o;{ww#DKVwVshI_zD&}S)o8akomiGV;&V9k2Z0B1QyDQd)SypmUjqNX z>+2b|?~1iC5|h2aK>RQhA^^e{r>13G6~dm-#YYJs26k^QiHHnI(P4PLCp{(7=!FZ0 z8G=|Wiq}4Eo`2LevKcV|;MG}wa~%g8e^xL0J~RYl%w$QoGBf|=r^(?r-o{B&!Dl#m zc^>U)FxkFbVdWlfnP~pgey*bdjVBp04jT{^;NF3EuPmF!3S$ni;y4{9xS76)4>NmTIa|i*x_gU^E6DoXu`G-|PMCJj-X^ zHzyt#05`$iV>lRgsnz*(;VdS!-=DG3VLpo@hvV`{it&8yMX~Nc&MkgGip1mLKIc#M zlV{t@a;Q@`x^x;C8tvyW%i;g6QlCtt70rqjEtAV*c>n@awA%E#-KMA6cd9Lfh;|tb z!AwoU2*jj=btDAW1z6S#1iO>p@EpG^$?@ka4^IbYP_1bkU69LJiyqI|eg3_se{o;s z8hbwisZ^7_UCzCxmsX4q!XmaCosUQHJpQfXZH@9g{o@A)Zo`heUy8log9F3T@%(pV zK%ym|TW}ldUrckEf|XWl6@$hITmCX_GxU77C;V4EgXu^F{_Hz0`>X7F_HWh~o?85(&O_ZFxU+eUFplbzCm9f8AsMzJkMdolC{g z-)ypd@BKKW2k&$vT7SIUVDHuI{>tT!`Tj?&v{Fbeg;GtKI`O+h8E!if9fVpR7xL$w?m!nAWw4LEg!)R|^_-`tr?|`2$L$Mn zd2=;eHkp~{ds*en<>q-FWg^}iN1#=!>VwbuKOQa`{*2>nR;5l;>DmqZ9b$|mPiQe& ztKFNIo6c^2L{q7@7p{u+dQf*}&-v#X)t@n4g;oMGnOGlWkx-xVti}PTw71gxbQ5gT zK*HM+7U?bpI9{?6i#-g(!2bDqbjj^*ILdDG5~0QF zVyg)fU=w0_0X&b%&e}D`t{$5ff%i&D+Ps|;_Bf(3i4|?PdI5YdGS;_{Qii*u|2kQ zvt0|KXlitB$TXObObBv8-)# zIf?(W*#fkP#bWc}dueC!-b#18#0ZQFG;=}w4kQsbQo!&?w-42r_;4wL`Sf%~Dzte9 zA<<%0DTixbD>N!_44e@hP*_b$Li9<=%&fqdDOG(FNynxl*XMS- zATf!>=B$j437^*_{Q)LrBXjjkc^EUIUpdma?R8$mXgpGXfgwtqEg~9w4k=ys*mRul ze|LM|j-gM0XV#^JgUZT(i#jX=s;Z--GzG)lVs(0hnvW#+R8&w)net%A2LbN1}t zuEUVRs#;H$`tlr(;QRB|pa-nAtC_~BPaWZ(#P5_dg$3Au&lxRVhj$B*2%fLW^4V;* zF=jx9I^8v_wHSydD+2rolIi{CWaRiJof_5o8dqO_&}n(JEAk0ngEk33R-I}g)_$Kt z)%+h%aZwvI-4fL*F?cP?i}7sfDx)-ZlhUK)ds{eBuQVFHrweBJ8qE%?v&D+GZ7=If zws4m#Dl(%8=}y6j-ikulLr5%?01qp(H)m#MqDGy_02%$zT*=t1`c?)E;$jxGmZ-3y zVtseqpZE_i5`)TcL1|dxO~um`@x1FK)w0|s$WdmtNogk$h?jvYf3&6844nS7&j$Gu zn&MuJiU*NuYV%8fIYxQ5$Z6sQO}mbLHnhfNGH7aIl}oW;C4w-Cg+2vl%%b(9zjWQR0}H!6(xjiV@m(J68w){!0;=LwMFMM0?+GIB1&GCz-jntje0$7`sn#)Z zV8lZ8jD2NTAd~ev3WR)txO81qbl>)+r+1;y0YaLeAf~e@XCm877V|4vd?cfg7`fTQ zq=l-MtL?S{@Lc_Sg@qE5NHLKoWNc7IO*ANXLQFE^+1tONOvuA7tFb#8uZ!v#Xig0) z5G)g@7;0456_>^o<0fGQIkM8|2t12i9fuWP$Drl=e`8@;%sQqR-_cuGQ@O|k~#_{|w<_ybnJoXHEk7jdYvl1|{E5IuNrjyIF zpYN~sN51^Pi#*@s;zfruUB^Mvm+eN^>qu419R;-Z#E|s#Bt5@pjV{@juK%T~_|u8k zev7(c{1oKtdW*Fj$6dtq`AS0sZcWr*2c%L|t&usRJamBQgR=ewtJP8-+x2_vGLBli~4Ckm9(^k{7!43}uH8c0y>D9+%* z6qhwA!Kd`igCa}Zmq5bDTQMZ@r*h(XJzaF|4G6=<7*$lIJ%)PWk!Sc@z)X?*m!6+o zD-~rN<~D&bcieZRph49TjEgS0rNW7BcCTok1VSzXNuVUBvd87m6IREQ%z-%33gb<= ziS#x;I<=W8=qUg+GxI@tx1ONl`fSuy=xq~6IW)klQ#sXxYF^aFh@980NVmfT1ZBSV zbwQPelhOSv$5}kCuK>*%b~$SP6?spf=Lu|J`$u#oQ~UEL!ac(ELX~B@+|+rnE+dHY z9;6nc=o%u(h(;P@Q6dC9HXJq+Cbj_`{%woh+F6QW8UNkCpA<%j4T@n!#m3A!avmg< zj>ZS%7xYB%%DW5@3YXBHXdQ~D)tYsx+|wd}-CNTFotz6Ofyu>C2{4qffoSwGMMlN0 znA;PnPr}xDMnpYF&>D3-na!lp4wbySoA`YdJ}&YktS@Jo5rG^n(W=4_j6EdeANaU> zkVoZ>v=tU*pD60y--mRJh#h7h-S$=1jfKR7#V~b~!ztt=z}=ys&+5`JMJygIhC$ao zY*)K!p7Z`_q7lTKqGCrv6~5$tn3}*svgXTibv7dzNY+*TC%LTHDOAjpm_yyv^q8B_ zp!6qou#+Z_(!eFTA08pZWAX{9ohto0ei8QGku>COwrG=tq?k3D6c=Y)<@|H9tK@2X zlt95c5_`c!CpMMmS%dg&$Hg7s@%Nt&kG*m7Sxi2hfrt9N6sW&M7xhN?vRI(BiKG`M zC8zvb9t)xv`t$XIysYh1B(UFi3kOxN>F9h$LlNh8?OzG4m#F*m7l*Xp`}p54<5hJv z8m%d2LQNvjlXtZ~JlCq@^B9Pv!p@ld9wq%n?Ztvtr3;su_pU(~7Txo|REavZT?<*I zm{DHUSYJdiEvxu7oPexUn zp_!D*Pi!1W-U0u(is@eXP*mqeCZ#KqahK!|l3!AjUtH7e52 zN`%M3MwqB2{)hyDz%7wWF>(r0PRDT%F7j0wDTX963nC0-t^`rg>MbsMyG`^^=^na1 zaSjc5`0L`~)0HT1;%m!P_mc#+w&`KtB32%pSm824Th=|^l;aCax%YIf+kSP}M(tq+ zi4DWh3~T_Yc-)J)(T~MCP3TB5A73n;^e6+<_7&8JJL)844INdiQ|Y`4I=Tvf(vbNm z0tA-(rm(*c;VmrUd~v#NqX*aeZLHD>OmxqzN~jP~8s-iuZB)B#X~T=8|1CPMPkgIlA18x$hJrjDz^Oa{WTdnUb= z}{qwY4GF?+rN89EfL&3_4LkI){oQ(YDTVwzgkKBQK z;b4+$P81vsFFY0x$|cxRYU}*O-?K2nz=hk!@?!%U9ph4`qf#_mI3+EUt=7??6H8@9 zrgH?3{)?7=B#?+GS`E9@^qXPZaz<(ik-hkIq5O4x(E;z@r#VsBIe2LWh*HpfDuJb8JRY!Q)jPEJLN7CQrX?Yo=D7*9ZRWg zgBN*Y1>Di9L!+!}VP4#50f~%PS{%qL}rHNUnH_^3Oh z3eb_PkcLF5Q;PF&F$HDJry`kdIxCk4YbaJ0O}MySkQ<#ceknJaKx$nAV2J9Em-){@ zBPD_09fx4-n(xo4mNE-~7T4y^5W1%(7jyP2n`aOy>(N>(oAM{h$wfqoXJ`+S zlP%UAxt+a;s!Dvo{hd(H_fqO@d!vLl(TPGVcT*!(%)H#IU26_&5jkfWOSeR`U^Qfq z$b16lpO`ZHV_-r|Hr$U4P`v*o+(E&YS}!ZDC|^&P4R-`JCrmE1+{{9Q^g@nYJ5?x+*QKsCK56p9x9~F=OM)z%ABk+x;`=~X?uV5A8K#tY zHe3!|fxJ|>WFAVqTtsp)Ir+G`w6_yo;E7`@y`(e)Q_)!#3JxC=Eeac$4TT;7|BteT z*fKeHI^6o93h3uiBvM0l!>3w$T^3(y{wm`U{U0jB1)Kax-+1yBjR8M01^3ceD{YQ` z!V)-z%T)f|EZIRt7;n95b!9oqU;=yc=ul3JhLOS4C9X5xg1GpJwbAJLbBab~Qb%Ol z>nYRp6b>`P7EFUZ!olbFWLSOFPmABxPX$jf4n{KDznBLzme@Xw8p-XM8*a>q35~f5 zmnD;2-%Qa2G(jcGeFp)LK&?D6!?>s5wLy)oae(u?BO(~T?8!T{Q5QTYM4(laKnt;5 zjMLf|&%fDmi%p~HB++dLXg59Wl{)&>=6)yPeq8Og8qL!bR6E_Ky)SONShQ;;E=dui zRxI~mZnir8hSlLFhtVh3s$lR9ri(G+qHE1o2foA`-f=q4Oh)eJZiCVFCDM&nfN1Cj zyWMJwXEwfZl7dJ7W{2&1vmo0jQ8rqs3FU$>61yrVQ*7#5cNJHy zI5cmC+jKF3Uhg~E%okM!)(n$hh=FI~n|BT2FXyLA$2WDQGKO^APPYkqzHw(u$!kgM zx_++-e7-%Ie7;XHnfh{>2kOopXvIt>{+BH)d7E|DD=6eWcNfmV?+;!VbKz~aYj+dE z2<531Hmwi|Snz5OODp=EJ@VJ)czy=M1FxfxSFBcx6f8iVtG1fYuW(>M)xgmbag0n! z?!7ZZtzKf`y6$6tjXamZ>A;iI^|xP( zS?&sap;%nKcPD+H!6I(kmfzj@U{{X~1h#1);s#}uNS@CZI0U|8umNTmz1?Q}^Ey6r z0B?H`(`9I?Y%X%$akV1G=T#D{X)BzSxY)6&>dGomsMBk{|Fw;;J2GDq$KRoEy1mVqeI6xZ zU>{jWv>+#`Rj4*1*Vn4mZ0UUTim}Q3=9k9*tJ5y>FqUCu*oQ6|H86dA>VHD+Oi(;txAgDmKXoE|EiHb{#dUe&?@& z;r(sTV+4PIp8Y<|J~LPRVt4PHrV-zv^T>O@fX!2gdTJ9n-XoF=ECPrh zB>*{E_&=hFG~>xIVxcDEV>r6bV_t}mEh{;4qSi|9 z^ODlr6eI;Kwao)U<%C~c=Jv<}T)Z|6}ocgUqx?~UVF`FlPX*7An z!tcZ%f8Dq<^LcCm~*j1|b?XbzJ=M@V4T@rI!tn?GTKh9P%*h@^kJn@NrI_lJ_u zEss)gC9LFo`5%Jbd!8dyleN>$RM#9`7t8K zJOvw$n=%gH%6*uNkMb=~jCgE&ChX=mDA{t$D1sr4yEi;~(K@Z|)RfWJ7sP*C$>Klp zJFrqiodzL7K1bFhp2wVFdp8v!1NUci%>lliD>R?InTq0Gfpc=kVuW zoJm}ens)^s+9hys1xg{XJ9v^T|Cr+n$mk_qOpN+92n6;nf5!8f(aD%Iwbpqhp&juGwL|kc`!z^^83Rht+hC` zoAp33c>dfb68c`fyo(j`Bb@O%t}!8&#j`(mA?rgmhzk0m0kW~DQBl^Ix5co_?NJcijafKh*)60_ll%!?bMEWu|O zzY-b@V{V#WXsD2Bv?vS52sovnp^BC^`URzLeVL}Fq!d4r$_}cU*Iv>=ttsFu7d5Y95s(hcWrjrI z7GT2LTb>FcD6-nVY<{?`rh-fEE_;guozJtf*U6a%30esdVI}pY{S3zT`WI0g<_M~l zv`4Fr?{TwJmhb2F3}QQ%EU;0krxYDf%Tz(^8W!{$M!a)cywazva4sp)Kl83Huo_z( z1WLto?+AB(ua&{IVkPWZ9Ig?UJ{vSzFC?AD#?)(AMmq=3f)d=}16ID*?MTB8iUL~B zJxK_Rp`CmL`&f-*uYQY**OmZ%JPMq73z=Kv0M=x5lx~uiA8^kwnlDJOJuVeISBQ*8 z?piqYy~-@kXJFh+=-Z^y$y9b2ERafW=jGuUj!Bz19t)aew@>tIOo-4UVmcEFicc{c zTb&ZIV8G#QFe0jO22)mzZG&i#lkPo;scEUmyhSYvPJKmYh+9k5XB`P%LR}XbLD6O0 ziccwAY*TJPX`4Jj55T!i_&OX;<$8J$gh6K~on%K^D|I-RFCM9t*sJm5wrHfB^>Wot zzzydGNu1;=yB94kz)CKmy|$?RqbMuG&e|k`pi`EcnkRUPwx5dLQFv4fF(FHMB<<8~ zes8IVnW-LJw}?*^Pas$-;!LeSwWdo+zi?6MRS7vqD_Jy)S8mxZuyNQ*BSPD|vfwBI z8?3Oh^2G#gkamQ?LVUio*eb)L;#|?JcaNCWOb47$;V|-dj)ym@I;@F=-J9uvjfgXQ zUQRlZ%`_JBYP$-7LM7GS_^5ArJN{15lTxB&vKdaGNN#ynU=0e4MdC+?RDyT&YnmO?xG4 z&u*!c%+A7pi!68Pj%pIkc*6L9ysifg``GdO0k6!| zkEK}GZt12dqUGeC__}f*1nIhexJ$ZAtIIBu(QcbeO)WNkA;Lbj$ZX$Lg^IZnZ-0q+ zNxY=u&!m#zlqC#e0aeOhS|OxOTzl(CHxb<3WCEm`ODRFJD#Ii+5=Y{fM#=rJncjer_UL?f?IZL+HE-uAS`35E16%$k~sh@#x`?o(Px&5p2`ry zfkJ=;3t$r|2qHhz3tp{^ulJIGql^$3ATmG|{{#j0NnL^IRDrk@4tQUH;6HQ3auMX` z*@MMRHYiLmIM80qK$j&u>mO8dsB!V)pGKRq0>In)fX}azgQAIVdXpl=lUICgqW=$}s+0veLS984!Y)gK9PsG)>B zH9cLE#};_ls}_m-pq0m!)}{+@9Yn5%)7fiuM?q%44i+Q`3@RMCO7HVT{#6;v7*CS8$gk)lIY#^WU%5Wb}W?ly>l4FFr!Mvc6H`vsaYT~=wJ6kdMh3@p9XLkkF44NjK-K# zD^j=}iG@1m%J**xX#yFk%GT4E>k|)IDob8;01}rhH{+8bZLisCP&WgSJR~yR;YClk zyKPnFN084pXB)L&x8f}DTIfHCMzFw=plL|_(2^lDI%!SffrV~r#}i~0FqUkRLLOA zYP((QizcGITPap*H;VDBDjVw>GZj=?f=eT)>q;s6&+A*dbs z11F))f9UTit=qMo0Dt_NU}yxVUpuC3Ddr@xL#NLY!_h$yA8tvE7FB>{<3u>iLwX!% zDJ3Y=QQZ)cW<2lA`XXSFk6to`o@KP*6Hh)sz~@Kyb3cIsHb{zfD^yT@I8-o)A$qY8 zqg{eM+k^cZx%@aiuWkoQo55rsvBUz@t7WNL7}?91Jcu>1W@Mq1YN*ergJqm$M1UXc zgW}Q$9^^WSFXxVKHR*U~|0K~&7c@j57-$!#T~OSdnJ^}1A{mEPOZw0-9=+PSZc1;Z z2zDNU>~BZe1awDWlY-?wfm@gtP?H_9o2|?YFI@`cgaVQ zi2{W!mD~X|__HIb7+?pu6sl@lxz-4z8WrlI+faX8_u)YKBm3;mBzjaR9Re`poG^_8@P z86bs3SoA?-kUz2n_oE&YApJW`NEh!#yH!N2#ac*%N~VvF*qspX)^wtt)(J&q&lnb@ zaZL)~`$aL2K8-PJGR7jjj#|b7>PTtXc2n$11bDrrDsTJq@bhSsfB*`Lm!zCnazTMEJHS&VDLpzT@w4@Tokf#fB6@O zXmsnVa~65Fcyb5KxK!FxhV%bL1M9>F5!{zmtEhj0-baER49>AkJ2va?KX-2#A3%83 zTxX>GpO+`tLSz`qwPUrKF#a!E-~|n9U+9UF1^Lfyy9oCox&xNVDdT}2G2sacb5HJ_ ziVd6v==SE}L4-F?)rv}a{&UFxKVEjBx5;)HZ3W}e2bH21{xvH7yQAoh`6Tdp~ zk!*->iAf(UfJIzyZcVd4ChKAu{==unA7GMWy+^%Z?9R7f-^xV6=w?dl5rK?sDoHsi8b8kdMe6=jOR>wCc7NLf(b$c6;jpO38gec zLvx*=&N)j=^Z%v}41^n<`I4S=Og&R1`Fb?Zsr?{dd%Jt+KA3hn(>#yU{OLl1}zGD8NOmHdBM%PBi$xWicdSY_sv!UEw5!Bgw}LBpChDZ3Ai#SmN(-TI;2;cX0a?e-BfDl<~=y#KhK(_ zj+bLKlPG*csai^H?y<6bz^5b+MSv^!usU5Tn_3U^ai+jfx=~ltgpHp5I)XUPV!l|? zvPLu-oi+^qx7Y(-Jb9+XTGU{x@VZegGWs^ z`Q`0MiJ}^xOQRYONzwFN&CcQvgjOxcZO&!Cj`P`(hb7<=GhVhdrefO@EwGAJsoF}y zJt)Fm#7wJ&>t3vwDNFf5#YS_#Tzv{iH>KYexxm_)Yh?%kyh0yZq>@68yA`^}0`E6Ss1r?zE8Ji%GB#HKBH#XHIw& zR%Bg*1)xX_L%M+m>Y>GZ&TFP!ijCAl6ximj1DxcCz*CtV>Dreab>^Ozn?W;R*;@Ne zqtP2wD*WA#!6cB{2Tx5?VqwuC25ZVIHOUba0@j?;=l1khEgm2QcL^F?LI@fl5S#>e3l?00yGwxFg*@ML-}l^m?*H58%uIK6 zO?7oowN=-2POx~6MIF*l=>0R(ETI4`3 z>2EVGsnluP0D@UNb@Fe+O|14|kd{AEeU}5%kWIvA{*^I5UxZSm5LwmC<5+|ShLhEm zh407wq@0T|?+jC!W#FDzZ0gw{WnvUj|wHW0L zimxM!#V0sCyMCRnFSd&IQ;hQoZFK4W1-aHDC9RXNqKLV3zB$l64pHhQy9Yxn@di6E zW38Fi(vTW^il$=e*qPr7jBT+Nrzo7LX>m8p8$&uF`blnoBPE$8Bo!MY(%MjO)GR1% zE3Z&Z<26C4ATgpw1iYQq$~R^D5$mR@R&y~yurlB5qwHqJ2C_tUKYpR1yMrizabqDd za{h^CFe(x$Kj|m8WFTb@!3s)^+%Lkoo!Sy|F!-0}({DjWBQ$8}BHod5@=ZFQuuDKl|7HYlQ)lQixNF&j5l4_OQi5%~EYX{w zX3fcPEH`;2?wx?&lUHmTvz-d&&ZWrVPzlswA@A|YMklacas`Yy+AuVv$wlZL&X-IU z+|(vT?Ge;_A-L_E<_p2=o83x~&<9V7fqq=xF-r1sa}5$)Dbl}8His}(yirUQDGYsW zr*H3JVEA^W&e85BZ!KKMv*Ru0aPWCjh2dEGiA1iQ2!5<(y~Te!#yj|&aWJ4T>58hP6^nw<9K8bLq31F~-ZGGJe!Wh$$P z4IA5Ti_KcT!qWt$;UX(&)ct`T-d&_tcX+*bC97T0m4ALBe81`Wp zdRxEU{8l`*GLbSTGjb2YPr-O5Qk2{EFav}jd*Kb40gUsg_V*d^?H0~`WyW>l3{7A^ z6t_ag^Y)xOU6<9SWn5l56b7Pz+G{c1;ATLOM9jO((=u7sL5X65%b7VqtuKr5r3C__ zedS!9Cc?zz{q$>j#jq!doZFQ7TYD{x>yG+JAr)|Q!4}tr8~G??OW<*F_$>^8$bh=d zi679j#v1^a6G^J;l#h&l?;Pl)o_tCtf2l1T6o=j@cSFLA=0%~H%M?snc zW}`!*+Fv3<^`9u7Idua24?d&Y+lR`u?J>NHR^S1CD=>R`MaN8SszsB(5c{&WIpMZ9 zq;7YaJ;8pz4n`uabcXxfKD~GCh{+g9{RohIl8vY(vS!Xr7n#WYY|g)tyjKYKn7T~~ zKP`ULCA=^@neGlV&UPvVSlpXrfZ`M}Di9^q>EAuV2GC}fZAz`kTCg;0Nszs17KvLWZ$xh09SNY#M-e}5Tn#B zAvR6a?o%mgecCI=DXRAC{5u>Xnrq^ILPV3HF(JIG-dsASyiEy%R77dQ4GlL>BUw2O zm87uVFNlY-<|fa)o44%bLvizIduD=v<5T0ZYLyMS<=;}PG|Nm)sr01Oa|K{N`y>gk zh9&Wp&D-s|l1V8eF~K-qfDbD=E4=G*fY2qEeKniO6748L*5TrWA5|8cq_sO#fixhw zWe1xiYv-9GnSznKQ&h=p|D91OVfWV4*nlL&sEUSzZUg1_)Y2W%z!Db!iJ~8hNU_qG z&sid^tXdcw{G{2%5_&Jn=TP(+#7BGprM-HWhz9iB)C$F<;rJ z_~2XhMgafeT;)f8rynJzZT==_$7u-to`^5d%n77m@%7Bb=KBlSrQCfn1D9|9s^G2|pg@1YL^z<4`An zWyZ>FD0R*|=l{fzeTA&;K+E5II4#|D5dDF4y-hXp^1tg=ZU=0=$k#&1nnA_F1-Ax5qo>FzK8oRhb>Db%DeY|aWn89Z8Zdam#28TRVvjvsoH?J zFf3vH-5%@PR#!`mr}NGpXK?R<{2@=y57EFFoG!`=fBcI6!TlUBf@CE>c;&0}%`i0DXu zCeaujlE+x)AfV_CHVi1T%N3QdUiexJDt&lAXG+9o!e`U_$r#IamqstzMH7ErzT0L- z#;R?*QGCJmlsAvjVCOTUZj{JJ4H1)_OT* zNCZj?IE!Bs_S~Y~!Q(lTpT{uPDY@ZD=S0~tejmYi1(v-#Id zJufsr+chL7&*j<MZPjk~|e#}3b^rBFn6E>m-0C!eHlu*2co6*#Pzn|SKucDaV-xNj7W z^nmm^+76fVrWc&uM%P#*C#4jsECE+Ma!N1(QI1lTA-F<{3I6qsnM(3bEG6IN499Ju zfAKJMP|5{0L0gck+RfBgHq1D-28Jb5V=B@EyCzw5BZcPOPb=bl8Cy5rSJLet!$=aFBge_FJ@RI6Cb=*KaJ|fBwOb%nQrhsl<0TW>&W78v(V=Q3Lq=a;ss zWu<=0`(q9AW)PN)kd`tBvTRqpI{A(BdXipw(VWD}4dwfjg}z<24I3PJz_X1ekjyvT zDNB;`pe7kc_s?!th-yW?ECt(ojJUQ7d8H6*iP}GYhmK)_tW|mvvIqr&g8Ia}>P7CQ zDTWE7jkTp8URM}@^q>qFlkPPDxuz4Y z@eAn+Db-;;>6A>Xp_1;n=ei!R?O6GI)cP9r+v>?Uvm3t1XRT;K6Kv#V6Y(#si?P#0 zc(EZtSrSy5qFo5kP!NI%y@<-5s^tm8DOB2cc&ixyrv;L+6_NtEd3An)nz_g((7y-C zrGaahDqa#Xx-(1ih{b&zrSKPa(GN=`n*0Ii&yxLZqFA5Zh8}3cg=#6^;oj74Z)Mwi za@3Sefvx(KETKxHx}(6~a;ED8_nYE5C=B4u(~$X?+#(ifMnkr$nK&*S`; z6I}k>9bHjNO*uGg1V9yDen0)=U!kz8lIN@d|Dj=b+Fro^;O+J4 zCJ_NaSA+0|N(NtxRe^FH|7eLf4Hqglp0{+_P)L13E+mB(YZpJAjtQBwcc}N?NRf{` z17flJYaNMZ1qq!HA0L0Fp`lJS1$^u?X;*2`PtV+l3U=U7OIf&rs|AV?;k{A?VzOF# zb#9s>tzntOt$L0)bLq-=T{Rj>Cc6XsKORcuw~ZE*K1U5r2l#*q)L*r>etT6tiQq0`@j7Mc*13?oNq+4Vi_|DkP zd|^#^ZNGQ25iFt~QgImkL$>vFgGTk5P+VA$r&m(7IUubZM zKKbMTVq|BpNO+f#l@Wv-6yG}$zY*%bn-I0jTkJHjqgAGaJ7@BF@^GNHZ>U`(!+V>O zer0ZJ%I{T+Wq4w^ex~Bti&u;I#v*GeGhF@SOnUylYE8X9n)#aFbGpg}Hucka`xNI5 zr+)?xN|ZNC_B62%mdq>t5%{dFte!H^uKm7#n>jH?`ouBwCDkMf@l)U=uiEu^SYhzT zv{u}=N(tGJm4srGQG$>nbp9FwG|S%snw7Wj!4Gs>S)^U z=0z%{B#Hd|Mzx8(^8)kG_cc4etouvoJ>q+0-o$#}w#rWoIgLlV(Q}^24;~BpGq6;O z6|K^#+oUE)TYY;6BzcMVHh2igh{N+9I*1v8q%E9IXisE*cJ-PL5J}DKl&)b5W~6D4 zd=&>9jqZ7Nt&GY#&T}fc;!&m&T-((*L9q>6Eh`^VmUcJ)o1aw`cTBDRvyf&gT}17l z(Xu*Cgb@U9@hGbIkNdG=@Px+q6vcs818ab@IKQ-ed(2$pIROQt(?toIn`YgYVX zJLC)YFX35GTw^`q2eR#YhC#4I6?~KV$N`iI!~_jgw9F1s=L)&5z?bi+Uz0|8`-yG5 zYTt+*L6hlhor_3Rurj{ku=<{AslOwA^8(NT*byAJ?k%$EqyjdO?pRW@YDrdiVY8di za0xK+D1Tds&FByK8Yjq&QuyTQDPhr*v;IulOcu$a=;7;@G<(Jk^UrZv05u-7fF8CP zmd@EJyJi4i=kj}8CRhBB2(d7d6nYD%tiN_I}$2n|BU7wq2qm^ zDcVLEx0G1|V$bmOV~N?d@#GA#J18cm4pN6HA>_T%U@d&@4_{Vde){ahgwz^mCgu$g zioC_O{OaPjc5`SvCgf$)5wnmbar$W)E69pe1O;SYLO_y{i&Irh9K3sk9hMNVqDJk) z*(zI^ZzUob6^u`S7VAf1rlHRfk0tS%Cqv1LE-s3yk~qg~e$^-&_Y=uv_!*+z97R{f zYDl7VxBAL!Mq{orm0K~@;D&6wb0&w@<0OKHzA)Am=HF{(0eVe3%Ih?A5yBPl4W}`l zC8Go`99qLqc<)n01E@tuan)OeYU_RB4rfJ2h`i=z#vytZ#uhSK04=8NFG4J94=all zz`~Z=#mNuGQN@)N?{D{TW1$5ZU?KuwnPXB?@-@ErCkMar9GMRZ4#K?;gpIz zzpk#PVt|o$QjsMmuvCA<1@B=6PrmQrk zmS*um;#+fUay}`uD*ZBV99LWMJKN3ak_Oj4!t^S}#^>P%otD=Cc}DW5q+o4WCk9%( zE2EAjM9W5qQpLDE5a^%;kTD>V)R{#kkTvZ$&Q8|SdWLdq{v@QJTez;#HLw;<4o&rY zKL;VR18-FIOtsn3enpoAT}QN6wL4DmCIMv7gt262E;}R)omOi zocFGR21;8IMz^=78(q_v`Y5xEr`(pj@BN!X!Qet$x?`NH2v5yl(B3 znaFRDw@qb9Y#1MVjS{Rf1qu8Pvn|r%jZx*BXWvL)nY!bUS1m5v1+3K0v*ZQj{YbwG z2}HFTDC&S|Bt9RsI2z-NBSx{)En$e~a4|jYp^8^T#C5V}J~bP!kVY{(lY^$=cLcj) zG_Gs>vfqH8m!K^xkHN#mE&Hyn$Z5!WN1Z83I>Aifj(@wO@4)OsN0JzbK#J>gry=`r ziZVf=S46h-6u&ytHsuE+T;2pyMEHu8Y|mZkGB-9Ed~FXoFj9^xudDPbNjdT#Lmg)| z_!v-(ab{5J5kTesps1j@pocu#`22qwOFkis)~75xp?C@%gy{QuD5?7nmMkDTUNBiI z0k#Qd5F*)R>e&X4OObYd3d!t42@;emjV9f;vSiM7&Z~+czLvTK8MW<@F~2z=UuFlz zG|kG|amhmcXa6+e6v>ZezA=rf8S|Q09ooLE(*G|_nwMh5$PiwKu_9pU$@SxkVQdwz1kA_&q z)hOhhSOP%hnl|P$#~LuJJGG%G$P7EXl&SBAh?G+8PP6-$mq@cn)j)k(O$FGU<~ z@OpN>tGA>$ow`fW3!Mzy>K}P~VT{!t7!gLtZmj0sQg$Ql81BYpvHB_N43uw|sbF$g z;Y-iYBl3Wp&mW~jO%O*7-UGizM<|f0#whVb166PVFu8JTWW{vB?tk% z2Pc{F0Yjb!Q>RV=jpjcpj8k}5nj8ZH&dM=`lV^i75~a@ZNuunHX#6aWM2>7FZ z+uFonufYD+;wE`QPO1FlN9h1SrK3$|x$QPWp>?=Vu9y9SN2vwd0Et#M`4wusp-*?syGtIUW#Q@W?c_GK4*nPT1WWf|w*edMVN!n4J%VVX=-Jx;**6C=t zC8wq*(Z*39!+6(!r7snGs54MqE&WC2l2w|iEJg*y{fVIo&Hv(%gnd6yRk$^ zvZ+dg#+`Lf@$9Sg!vt~cN>CZvDSFKg^qYluuA>d9`psp*(qe=CwEfp;m_wZE%`^3C zMy6TGmRKLe9B9@q7KQ)R*uwiHJspKy*}#1{?{@L=56=^@pKd=l^Vv3x=6 zpoc1wO-rML=A5^Fo!;_ER-ybFvl*n3N{Ng}Za3s1Pe`2ajYW_9^mD2t z9XtuDYFxal1kr;Qyf$YJ4=JYremt2`2NpGSb~Zc{9IU_E7OHjh1!e9BV-#JxU#xgP zpD#yp9k2@xb`pF#Ae5*2j(k8;8#f}j*@RMNVfE$G5Vm*DOuZ5ZE__!0V4|pHi-?t0 zzoHzcXUCW)WQWu;rDJ-~M*ljQV%OWh4Y>jM%t8jJ@ma$@Hz{`a0|g82=Yj@xmL&@a z3(0LadTh4Obnu8Z`#f(M&9$D9!@}j(u00=7?oa6a=V8q0%w#A1)VuXhn1NQ{Ik%r| zAa|$u>&t;q<*Sn>?6%CN*FR#@rsv?thzpPGtz!x()#(+y@cK1~hfL`Lv#y*j#Wc;` zbDGk(=x+V$vrj6lHP7DGZOm_M^ud0>a`BP&bhFAxDlIqt`|CNm>Y>fR7EP)~pooiJ_=7F%n3sy8cK`SqvPFYh+zfJ-YB z!gaPkyf<`4+~e-!teZTxetC*KojaJybNq@$f%8S2_)q#Yp0NLD)iv9!ivH)5+L6|u z-D)509Z?|;i!WD!hukodTA3>UA8A!f4v~~Uoy?Zl7q}cuxlDyT;rg8id&Lsqv*)ZX zO4QU)h%DQMFUka^YSn%WWWr}4tx}E-*RGH0Y^fDUG;Hg#WvDB) zsQpop%M99hkPJAx9M7K1%yaEA2;ofh;?xkkTOh4+F?C1CHLgvOX;9mDPdVGp_{?AM%r-UKn#mC zMVcUEv%w1)b0DvUkcOFe6>++#Q^jNSLY7lt46Pu-^oL`bQx^?Sib(_R3eU(zGtu(IhPKxi^@miAXZn?`As z%!^?EUX>6oGa4(?5v%zNpp2r3Q6_ibYiq!x$Mac^Xis!@qLS0%^hV0}*Pj;0Rx6KR z>)rhmv}QdXGBm}?Y3yV}B{B)VCiR?|9!9`tM7(2hhT!S`M2B2XN${X%yXe%jty$JVfj@W}WRL45hCcoZ8w1;6Efm@7^CyL0Pbn=B6v%Pe)aX zUxs3Baf1(7a@sg6cJth^_ZKa?n;Q&uKrI#%B=P>Gw|m}7I{sJZ{mAaS6ZR9ME>fCe z6o5mcAFAERYMOgC1YZeHYQJ=rug zgeMI$rYhI0G=zOeJ|}Ljs^XB*2vIU~!=W0{lYJJItv{>2YdtFZe=Z8s1xYUY{Cwjv zq+hH2Ik*J9|J^_1Nj6>X``g{xPn*GQr9I>{wXAozd`w_@h>dmiI5Rw%WFY~-@-llE zooE`3aJ{oVc3r>Ld-JU-eW=(~na(#759zN^8`b~m7svw9in@0M3RQatBGA zNS7@ZDk!!AT7_sV<9CYZee-w(H;e0f#QU5X$aJV*RON{Yxv{la1iALRQT4I<@yf1S znW|5!%Y0J0v<8Kkk93qaRM}@ByaRw2^RBOrO7mX*NQxJU^FUJRZpD%FS{>26-(|H& zx*_ke+t4Hafb%4U|3ABeQkbps2HXG*UZ|v0xuHknx#UGCfBV_l2h*Hcty-qy2a`1R zhoNlhd!F^jrULW9j!IcO<1HzCAiVuBA%cBU(8Pk*)7<}0UOW1;JZl3+m|eel^5kg= zOhyl;0rsSFc6G9{b%0R8yqzIbFV)mIpP*Sopb%GEi~os2qk3?h0(@vZy!--uf_!`& zaK~fGW68n61%B|uodEpe!XG>jez<#t!5Mnw^e^b){($o6{)zwJ z;RiyG6yQ((zc?H`+#K9o+}zw8f}D>8|AoUtTn{|_llOn&aC}ZQ`0XL$A%%b0w1T?_ zqHy?smk%f8A)62I|H>8zj{w&{W%Llu!GU&|Hl6z9$+IL|Ej^f?@+%<=Cr5U$rIR^? z-OJJW$%7LS_i_eX*h64c<`8RJs3^l>QyT-7t)(b~F0V3&va=M###Y|P6{6{*qGjP@ zZy{*OATEX`;w1zp;0S?%sk|H=pl(85q7098h2Z`NFo=Qb5d~&1%Anu~wuT4^$#~gX zX*oGLP;qjwb8s-giMU!?328{n{zC#D5@oP~!JLIaAWu(Ec2910Cs%6_r=Xx9h=U8n z#l;54U~}_^!oXf^P&XPl1vIKh8qyFq3s+lbn5`3(>VYQM+{qm#%D@1Zz`xwVY_0w` zZK&HnMF1}h&_fJ}lbr(ur$$93@(>h~w1v5;L0mP#j?NB{2k_riG$AgJfxjUKTQLC< z&_il)-y_ar;2$KBN5cOnrN0S07RW0I)EZ_Z#>vUc^AGlag8yaa5vXfx2}k7-fT#G# z;eP=CMbwuvmvYr~H-BW?7Ha+fV4Ul}82>jv{|)|&^WXXTzohXO?LWQ%{JTDu1pLE3 jc>fg?c;H`AgFxZ( spark.sql("select name,sum(count) from global_temp.df group by name").sho

    Now the above three dataframe/SQL operators are shown in the list. If we click the -'show at \: 24' link of the last query, we will see the DAG of the job. +'show at \: 24' link of the last query, we will see the DAG and details of the query execution.

    -We can see that details information of each stage. The first block 'WholeStageCodegen' -compile multiple operator ('LocalTableScan' and 'HashAggregate') together into a single Java -function to improve performance, and metrics like number of rows and spill size are listed in -the block. The second block 'Exchange' shows the metrics on the shuffle exchange, including +The query details page displays information about the query execution time, its duration, +the list of associated jobs, and the query execution DAG. +The first block 'WholeStageCodegen (1)' compiles multiple operators ('LocalTableScan' and 'HashAggregate') together into a single Java +function to improve performance, and metrics like number of rows and spill size are listed in the block. +The annotation '(1)' in the block name is the code generation id. +The second block 'Exchange' shows the metrics on the shuffle exchange, including number of written shuffle records, total data size, etc. @@ -362,6 +364,8 @@ number of written shuffle records, total data size, etc.

    Clicking the 'Details' link on the bottom displays the logical plans and the physical plan, which illustrate how Spark parses, analyzes, optimizes and performs the query. +Steps in the physical plan subject to whole stage code generation optimization, are prefixed by a star followed by +the code generation id, for example: '*(1) LocalTableScan' ### SQL metrics diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 9351b074c6590..ac66a71fe7ec0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -62,15 +62,22 @@ private[execution] object SparkPlanInfo { new SQLMetricInfo(metric.name.getOrElse(key), metric.id, metric.metricType) } + val nodeName = plan match { + case physicalOperator: WholeStageCodegenExec => + s"${plan.nodeName} (${physicalOperator.codegenStageId})" + case _ => plan.nodeName + } + // dump the file scan metadata (e.g file path) to event log val metadata = plan match { case fileScan: FileSourceScanExec => fileScan.metadata case _ => Map[String, String]() } new SparkPlanInfo( - plan.nodeName, + nodeName, plan.simpleString(SQLConf.get.maxToStringFields), children.map(fromSparkPlan), - metadata, metrics) + metadata, + metrics) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index f898236c537a8..bd14be702a407 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -78,7 +78,7 @@ object SparkPlanGraph { subgraph: SparkPlanGraphCluster, exchanges: mutable.HashMap[SparkPlanInfo, SparkPlanGraphNode]): Unit = { planInfo.nodeName match { - case "WholeStageCodegen" => + case name if name.startsWith("WholeStageCodegen") => val metrics = planInfo.metrics.map { metric => SQLPlanMetric(metric.name, metric.accumulatorId, metric.metricType) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index ccefb53fc4b39..fbf97e9a3240c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -83,7 +83,7 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { // TODO: update metrics in generated operators val ds = spark.range(10).filter('id < 5) testSparkPlanMetricsWithPredicates(ds.toDF(), 1, Map( - 0L -> (("WholeStageCodegen", Map( + 0L -> (("WholeStageCodegen (1)", Map( "duration total (min, med, max)" -> {_.toString.matches(timingMetricPattern)}))) ), true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index b8c0935b33a87..a8b77b8202123 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.functions.count import org.apache.spark.sql.internal.StaticSQLConf.UI_RETAINED_EXECUTIONS import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.status.ElementTrackingStore @@ -616,6 +617,12 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils assert(statusStore.executionsCount === 2) assert(statusStore.execution(2) === None) } + + test("SPARK-29894 test Codegen Stage Id in SparkPlanInfo") { + val df = createTestDataFrame.select(count("*")) + val sparkPlanInfo = SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan) + assert(sparkPlanInfo.nodeName === "WholeStageCodegen (2)") + } } From 6eeb131941e3a4afb6f8c55de06e9c26f942e155 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 20 Nov 2019 08:32:13 -0800 Subject: [PATCH 0675/1387] [SPARK-28885][SQL][FOLLOW-UP] Re-enable the ported PgSQL regression tests of SQLQueryTestSuite ### What changes were proposed in this pull request? SPARK-28885(#26107) has supported the ANSI store assignment rules and stopped running some ported PgSQL regression tests that violate the rules. To re-activate these tests, this pr is to modify them for passing tests with the rules. ### Why are the changes needed? To make the test coverage better. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26492 from maropu/SPARK-28885-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../sql-tests/inputs/postgreSQL/date.sql | 34 +- .../sql-tests/inputs/postgreSQL/float4.sql | 12 +- .../sql-tests/inputs/postgreSQL/float8.sql | 24 +- .../sql-tests/inputs/postgreSQL/int2.sql | 14 +- .../sql-tests/inputs/postgreSQL/int4.sql | 14 +- .../sql-tests/inputs/postgreSQL/int8.sql | 12 +- .../sql-tests/inputs/postgreSQL/numeric.sql | 990 +++++++++--------- .../sql-tests/inputs/postgreSQL/timestamp.sql | 32 +- .../sql-tests/results/postgreSQL/date.sql.out | 38 +- .../results/postgreSQL/float4.sql.out | 10 +- .../results/postgreSQL/float8.sql.out | 20 +- .../sql-tests/results/postgreSQL/int2.sql.out | 10 +- .../sql-tests/results/postgreSQL/int4.sql.out | 10 +- .../sql-tests/results/postgreSQL/int8.sql.out | 10 +- .../results/postgreSQL/numeric.sql.out | 941 ++++++++--------- .../results/postgreSQL/timestamp.sql.out | 44 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 14 +- 17 files changed, 1150 insertions(+), 1079 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql index d3cd46e4e6b89..0bab2f884d976 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/date.sql @@ -7,23 +7,25 @@ CREATE TABLE DATE_TBL (f1 date) USING parquet; -INSERT INTO DATE_TBL VALUES ('1957-04-09'); -INSERT INTO DATE_TBL VALUES ('1957-06-13'); -INSERT INTO DATE_TBL VALUES ('1996-02-28'); -INSERT INTO DATE_TBL VALUES ('1996-02-29'); -INSERT INTO DATE_TBL VALUES ('1996-03-01'); -INSERT INTO DATE_TBL VALUES ('1996-03-02'); -INSERT INTO DATE_TBL VALUES ('1997-02-28'); +-- PostgreSQL implicitly casts string literals to data with date types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO DATE_TBL VALUES (date('1957-04-09')); +INSERT INTO DATE_TBL VALUES (date('1957-06-13')); +INSERT INTO DATE_TBL VALUES (date('1996-02-28')); +INSERT INTO DATE_TBL VALUES (date('1996-02-29')); +INSERT INTO DATE_TBL VALUES (date('1996-03-01')); +INSERT INTO DATE_TBL VALUES (date('1996-03-02')); +INSERT INTO DATE_TBL VALUES (date('1997-02-28')); -- [SPARK-27923] Skip invalid date: 1997-02-29 --- INSERT INTO DATE_TBL VALUES ('1997-02-29'); -INSERT INTO DATE_TBL VALUES ('1997-03-01'); -INSERT INTO DATE_TBL VALUES ('1997-03-02'); -INSERT INTO DATE_TBL VALUES ('2000-04-01'); -INSERT INTO DATE_TBL VALUES ('2000-04-02'); -INSERT INTO DATE_TBL VALUES ('2000-04-03'); -INSERT INTO DATE_TBL VALUES ('2038-04-08'); -INSERT INTO DATE_TBL VALUES ('2039-04-09'); -INSERT INTO DATE_TBL VALUES ('2040-04-10'); +-- INSERT INTO DATE_TBL VALUES ('1997-02-29')); +INSERT INTO DATE_TBL VALUES (date('1997-03-01')); +INSERT INTO DATE_TBL VALUES (date('1997-03-02')); +INSERT INTO DATE_TBL VALUES (date('2000-04-01')); +INSERT INTO DATE_TBL VALUES (date('2000-04-02')); +INSERT INTO DATE_TBL VALUES (date('2000-04-03')); +INSERT INTO DATE_TBL VALUES (date('2038-04-08')); +INSERT INTO DATE_TBL VALUES (date('2039-04-09')); +INSERT INTO DATE_TBL VALUES (date('2040-04-10')); SELECT f1 AS `Fifteen` FROM DATE_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql index 058467695a608..2989569e219ff 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float4.sql @@ -7,11 +7,13 @@ CREATE TABLE FLOAT4_TBL (f1 float) USING parquet; -INSERT INTO FLOAT4_TBL VALUES (' 0.0'); -INSERT INTO FLOAT4_TBL VALUES ('1004.30 '); -INSERT INTO FLOAT4_TBL VALUES (' -34.84 '); -INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e+20'); -INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e-20'); +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')); +INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')); +INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')); +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')); +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')); -- [SPARK-28024] Incorrect numeric values when out of range -- test for over and under flow diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql index 957dabdebab4e..932cdb95fcf3a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/float8.sql @@ -7,11 +7,13 @@ CREATE TABLE FLOAT8_TBL(f1 double) USING parquet; -INSERT INTO FLOAT8_TBL VALUES (' 0.0 '); -INSERT INTO FLOAT8_TBL VALUES ('1004.30 '); -INSERT INTO FLOAT8_TBL VALUES (' -34.84'); -INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200'); -INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200'); +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 ')); +INSERT INTO FLOAT8_TBL VALUES (double('1004.30 ')); +INSERT INTO FLOAT8_TBL VALUES (double(' -34.84')); +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200')); +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200')); -- [SPARK-28024] Incorrect numeric values when out of range -- test for underflow and overflow handling @@ -227,15 +229,17 @@ SELECT atanh(double('NaN')); TRUNCATE TABLE FLOAT8_TBL; -INSERT INTO FLOAT8_TBL VALUES ('0.0'); +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT8_TBL VALUES (double('0.0')); -INSERT INTO FLOAT8_TBL VALUES ('-34.84'); +INSERT INTO FLOAT8_TBL VALUES (double('-34.84')); -INSERT INTO FLOAT8_TBL VALUES ('-1004.30'); +INSERT INTO FLOAT8_TBL VALUES (double('-1004.30')); -INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200'); +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200')); -INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200'); +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200')); SELECT '' AS five, * FROM FLOAT8_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql index f64ec5d75afcf..07f5976ca6d2f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql @@ -8,19 +8,23 @@ CREATE TABLE INT2_TBL(f1 smallint) USING parquet; -- [SPARK-28023] Trim the string when cast string type to other types -INSERT INTO INT2_TBL VALUES (trim('0 ')); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT2_TBL VALUES (smallint(trim('0 '))); -INSERT INTO INT2_TBL VALUES (trim(' 1234 ')); +INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 '))); -INSERT INTO INT2_TBL VALUES (trim(' -1234')); +INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234'))); -- [SPARK-27923] Invalid input syntax for type short throws exception at PostgreSQL -- INSERT INTO INT2_TBL VALUES ('34.5'); -- largest and smallest values -INSERT INTO INT2_TBL VALUES ('32767'); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT2_TBL VALUES (smallint('32767')); -INSERT INTO INT2_TBL VALUES ('-32767'); +INSERT INTO INT2_TBL VALUES (smallint('-32767')); -- bad input values -- should give errors -- INSERT INTO INT2_TBL VALUES ('100000'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql index 1c2320ff7fad6..3a409eea34837 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql @@ -9,19 +9,23 @@ CREATE TABLE INT4_TBL(f1 int) USING parquet; -- [SPARK-28023] Trim the string when cast string type to other types -INSERT INTO INT4_TBL VALUES (trim(' 0 ')); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT4_TBL VALUES (int(trim(' 0 '))); -INSERT INTO INT4_TBL VALUES (trim('123456 ')); +INSERT INTO INT4_TBL VALUES (int(trim('123456 '))); -INSERT INTO INT4_TBL VALUES (trim(' -123456')); +INSERT INTO INT4_TBL VALUES (int(trim(' -123456'))); -- [SPARK-27923] Invalid input syntax for integer: "34.5" at PostgreSQL -- INSERT INTO INT4_TBL(f1) VALUES ('34.5'); -- largest and smallest values -INSERT INTO INT4_TBL VALUES ('2147483647'); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT4_TBL VALUES (int('2147483647')); -INSERT INTO INT4_TBL VALUES ('-2147483647'); +INSERT INTO INT4_TBL VALUES (int('-2147483647')); -- [SPARK-27923] Spark SQL insert these bad inputs to NULL -- bad input values diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql index d29bf3bfad4ca..5fea758e73084 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql @@ -8,11 +8,13 @@ -- CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet; -INSERT INTO INT8_TBL VALUES(trim(' 123 '),trim(' 456')); -INSERT INTO INT8_TBL VALUES(trim('123 '),'4567890123456789'); -INSERT INTO INT8_TBL VALUES('4567890123456789','123'); -INSERT INTO INT8_TBL VALUES(+4567890123456789,'4567890123456789'); -INSERT INTO INT8_TBL VALUES('+4567890123456789','-4567890123456789'); +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))); +INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')); +INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')); +INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')); +INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')); -- [SPARK-27923] Spark SQL insert there bad inputs to NULL -- bad inputs diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql index c447a0dc2c7f2..dbdb2cace0e0c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql @@ -26,466 +26,508 @@ CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet; -- ****************************** -- BEGIN TRANSACTION; -INSERT INTO num_exp_add VALUES (0,0,'0'); -INSERT INTO num_exp_sub VALUES (0,0,'0'); -INSERT INTO num_exp_mul VALUES (0,0,'0'); -INSERT INTO num_exp_div VALUES (0,0,'NaN'); -INSERT INTO num_exp_add VALUES (0,1,'0'); -INSERT INTO num_exp_sub VALUES (0,1,'0'); -INSERT INTO num_exp_mul VALUES (0,1,'0'); -INSERT INTO num_exp_div VALUES (0,1,'NaN'); -INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047'); -INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047'); -INSERT INTO num_exp_mul VALUES (0,2,'0'); -INSERT INTO num_exp_div VALUES (0,2,'0'); -INSERT INTO num_exp_add VALUES (0,3,'4.31'); -INSERT INTO num_exp_sub VALUES (0,3,'-4.31'); -INSERT INTO num_exp_mul VALUES (0,3,'0'); -INSERT INTO num_exp_div VALUES (0,3,'0'); -INSERT INTO num_exp_add VALUES (0,4,'7799461.4119'); -INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119'); -INSERT INTO num_exp_mul VALUES (0,4,'0'); -INSERT INTO num_exp_div VALUES (0,4,'0'); -INSERT INTO num_exp_add VALUES (0,5,'16397.038491'); -INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491'); -INSERT INTO num_exp_mul VALUES (0,5,'0'); -INSERT INTO num_exp_div VALUES (0,5,'0'); -INSERT INTO num_exp_add VALUES (0,6,'93901.57763026'); -INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026'); -INSERT INTO num_exp_mul VALUES (0,6,'0'); -INSERT INTO num_exp_div VALUES (0,6,'0'); -INSERT INTO num_exp_add VALUES (0,7,'-83028485'); -INSERT INTO num_exp_sub VALUES (0,7,'83028485'); -INSERT INTO num_exp_mul VALUES (0,7,'0'); -INSERT INTO num_exp_div VALUES (0,7,'0'); -INSERT INTO num_exp_add VALUES (0,8,'74881'); -INSERT INTO num_exp_sub VALUES (0,8,'-74881'); -INSERT INTO num_exp_mul VALUES (0,8,'0'); -INSERT INTO num_exp_div VALUES (0,8,'0'); -INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420'); -INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420'); -INSERT INTO num_exp_mul VALUES (0,9,'0'); -INSERT INTO num_exp_div VALUES (0,9,'0'); -INSERT INTO num_exp_add VALUES (1,0,'0'); -INSERT INTO num_exp_sub VALUES (1,0,'0'); -INSERT INTO num_exp_mul VALUES (1,0,'0'); -INSERT INTO num_exp_div VALUES (1,0,'NaN'); -INSERT INTO num_exp_add VALUES (1,1,'0'); -INSERT INTO num_exp_sub VALUES (1,1,'0'); -INSERT INTO num_exp_mul VALUES (1,1,'0'); -INSERT INTO num_exp_div VALUES (1,1,'NaN'); -INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047'); -INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047'); -INSERT INTO num_exp_mul VALUES (1,2,'0'); -INSERT INTO num_exp_div VALUES (1,2,'0'); -INSERT INTO num_exp_add VALUES (1,3,'4.31'); -INSERT INTO num_exp_sub VALUES (1,3,'-4.31'); -INSERT INTO num_exp_mul VALUES (1,3,'0'); -INSERT INTO num_exp_div VALUES (1,3,'0'); -INSERT INTO num_exp_add VALUES (1,4,'7799461.4119'); -INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119'); -INSERT INTO num_exp_mul VALUES (1,4,'0'); -INSERT INTO num_exp_div VALUES (1,4,'0'); -INSERT INTO num_exp_add VALUES (1,5,'16397.038491'); -INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491'); -INSERT INTO num_exp_mul VALUES (1,5,'0'); -INSERT INTO num_exp_div VALUES (1,5,'0'); -INSERT INTO num_exp_add VALUES (1,6,'93901.57763026'); -INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026'); -INSERT INTO num_exp_mul VALUES (1,6,'0'); -INSERT INTO num_exp_div VALUES (1,6,'0'); -INSERT INTO num_exp_add VALUES (1,7,'-83028485'); -INSERT INTO num_exp_sub VALUES (1,7,'83028485'); -INSERT INTO num_exp_mul VALUES (1,7,'0'); -INSERT INTO num_exp_div VALUES (1,7,'0'); -INSERT INTO num_exp_add VALUES (1,8,'74881'); -INSERT INTO num_exp_sub VALUES (1,8,'-74881'); -INSERT INTO num_exp_mul VALUES (1,8,'0'); -INSERT INTO num_exp_div VALUES (1,8,'0'); -INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420'); -INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420'); -INSERT INTO num_exp_mul VALUES (1,9,'0'); -INSERT INTO num_exp_div VALUES (1,9,'0'); -INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047'); -INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047'); -INSERT INTO num_exp_mul VALUES (2,0,'0'); -INSERT INTO num_exp_div VALUES (2,0,'NaN'); -INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047'); -INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047'); -INSERT INTO num_exp_mul VALUES (2,1,'0'); -INSERT INTO num_exp_div VALUES (2,1,'NaN'); -INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094'); -INSERT INTO num_exp_sub VALUES (2,2,'0'); -INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209'); -INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047'); -INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047'); -INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257'); -INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266'); -INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047'); -INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047'); -INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593'); -INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685'); -INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047'); -INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047'); -INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077'); -INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429'); -INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787'); -INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307'); -INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222'); -INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940'); -INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047'); -INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953'); -INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795'); -INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518'); -INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047'); -INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047'); -INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407'); -INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476'); -INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467'); -INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627'); -INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740'); -INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811'); -INSERT INTO num_exp_add VALUES (3,0,'4.31'); -INSERT INTO num_exp_sub VALUES (3,0,'4.31'); -INSERT INTO num_exp_mul VALUES (3,0,'0'); -INSERT INTO num_exp_div VALUES (3,0,'NaN'); -INSERT INTO num_exp_add VALUES (3,1,'4.31'); -INSERT INTO num_exp_sub VALUES (3,1,'4.31'); -INSERT INTO num_exp_mul VALUES (3,1,'0'); -INSERT INTO num_exp_div VALUES (3,1,'NaN'); -INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047'); -INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047'); -INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257'); -INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352'); -INSERT INTO num_exp_add VALUES (3,3,'8.62'); -INSERT INTO num_exp_sub VALUES (3,3,'0'); -INSERT INTO num_exp_mul VALUES (3,3,'18.5761'); -INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (3,4,'7799465.7219'); -INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019'); -INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289'); -INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552'); -INSERT INTO num_exp_add VALUES (3,5,'16401.348491'); -INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491'); -INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621'); -INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504'); -INSERT INTO num_exp_add VALUES (3,6,'93905.88763026'); -INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026'); -INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206'); -INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595'); -INSERT INTO num_exp_add VALUES (3,7,'-83028480.69'); -INSERT INTO num_exp_sub VALUES (3,7,'83028489.31'); -INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35'); -INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240'); -INSERT INTO num_exp_add VALUES (3,8,'74885.31'); -INSERT INTO num_exp_sub VALUES (3,8,'-74876.69'); -INSERT INTO num_exp_mul VALUES (3,8,'322737.11'); -INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553'); -INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420'); -INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420'); -INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020'); -INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854'); -INSERT INTO num_exp_add VALUES (4,0,'7799461.4119'); -INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119'); -INSERT INTO num_exp_mul VALUES (4,0,'0'); -INSERT INTO num_exp_div VALUES (4,0,'NaN'); -INSERT INTO num_exp_add VALUES (4,1,'7799461.4119'); -INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119'); -INSERT INTO num_exp_mul VALUES (4,1,'0'); -INSERT INTO num_exp_div VALUES (4,1,'NaN'); -INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047'); -INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047'); -INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593'); -INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385'); -INSERT INTO num_exp_add VALUES (4,3,'7799465.7219'); -INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019'); -INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289'); -INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883'); -INSERT INTO num_exp_add VALUES (4,4,'15598922.8238'); -INSERT INTO num_exp_sub VALUES (4,4,'0'); -INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161'); -INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (4,5,'7815858.450391'); -INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409'); -INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429'); -INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061'); -INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026'); -INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974'); -INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094'); -INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606'); -INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881'); -INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119'); -INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715'); -INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637'); -INSERT INTO num_exp_add VALUES (4,8,'7874342.4119'); -INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119'); -INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839'); -INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143'); -INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420'); -INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420'); -INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980'); -INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409'); -INSERT INTO num_exp_add VALUES (5,0,'16397.038491'); -INSERT INTO num_exp_sub VALUES (5,0,'16397.038491'); -INSERT INTO num_exp_mul VALUES (5,0,'0'); -INSERT INTO num_exp_div VALUES (5,0,'NaN'); -INSERT INTO num_exp_add VALUES (5,1,'16397.038491'); -INSERT INTO num_exp_sub VALUES (5,1,'16397.038491'); -INSERT INTO num_exp_mul VALUES (5,1,'0'); -INSERT INTO num_exp_div VALUES (5,1,'NaN'); -INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047'); -INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047'); -INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077'); -INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446'); -INSERT INTO num_exp_add VALUES (5,3,'16401.348491'); -INSERT INTO num_exp_sub VALUES (5,3,'16392.728491'); -INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621'); -INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584'); -INSERT INTO num_exp_add VALUES (5,4,'7815858.450391'); -INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409'); -INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429'); -INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192'); -INSERT INTO num_exp_add VALUES (5,5,'32794.076982'); -INSERT INTO num_exp_sub VALUES (5,5,'0'); -INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081'); -INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (5,6,'110298.61612126'); -INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926'); -INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766'); -INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689'); -INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509'); -INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491'); -INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135'); -INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710'); -INSERT INTO num_exp_add VALUES (5,8,'91278.038491'); -INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509'); -INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571'); -INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228'); -INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420'); -INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420'); -INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220'); -INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427'); -INSERT INTO num_exp_add VALUES (6,0,'93901.57763026'); -INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026'); -INSERT INTO num_exp_mul VALUES (6,0,'0'); -INSERT INTO num_exp_div VALUES (6,0,'NaN'); -INSERT INTO num_exp_add VALUES (6,1,'93901.57763026'); -INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026'); -INSERT INTO num_exp_mul VALUES (6,1,'0'); -INSERT INTO num_exp_div VALUES (6,1,'NaN'); -INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787'); -INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307'); -INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222'); -INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823'); -INSERT INTO num_exp_add VALUES (6,3,'93905.88763026'); -INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026'); -INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206'); -INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907'); -INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026'); -INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974'); -INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094'); -INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469'); -INSERT INTO num_exp_add VALUES (6,5,'110298.61612126'); -INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926'); -INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766'); -INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679'); -INSERT INTO num_exp_add VALUES (6,6,'187803.15526052'); -INSERT INTO num_exp_sub VALUES (6,6,'0'); -INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676'); -INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974'); -INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026'); -INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610'); -INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980'); -INSERT INTO num_exp_add VALUES (6,8,'168782.57763026'); -INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026'); -INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906'); -INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184'); -INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160'); -INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680'); -INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920'); -INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789'); -INSERT INTO num_exp_add VALUES (7,0,'-83028485'); -INSERT INTO num_exp_sub VALUES (7,0,'-83028485'); -INSERT INTO num_exp_mul VALUES (7,0,'0'); -INSERT INTO num_exp_div VALUES (7,0,'NaN'); -INSERT INTO num_exp_add VALUES (7,1,'-83028485'); -INSERT INTO num_exp_sub VALUES (7,1,'-83028485'); -INSERT INTO num_exp_mul VALUES (7,1,'0'); -INSERT INTO num_exp_div VALUES (7,1,'NaN'); -INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047'); -INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953'); -INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795'); -INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700'); -INSERT INTO num_exp_add VALUES (7,3,'-83028480.69'); -INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31'); -INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35'); -INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974'); -INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881'); -INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119'); -INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715'); -INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686'); -INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509'); -INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491'); -INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135'); -INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574'); -INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974'); -INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026'); -INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610'); -INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294'); -INSERT INTO num_exp_add VALUES (7,7,'-166056970'); -INSERT INTO num_exp_sub VALUES (7,7,'0'); -INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225'); -INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (7,8,'-82953604'); -INSERT INTO num_exp_sub VALUES (7,8,'-83103366'); -INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285'); -INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118'); -INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420'); -INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580'); -INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700'); -INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382'); -INSERT INTO num_exp_add VALUES (8,0,'74881'); -INSERT INTO num_exp_sub VALUES (8,0,'74881'); -INSERT INTO num_exp_mul VALUES (8,0,'0'); -INSERT INTO num_exp_div VALUES (8,0,'NaN'); -INSERT INTO num_exp_add VALUES (8,1,'74881'); -INSERT INTO num_exp_sub VALUES (8,1,'74881'); -INSERT INTO num_exp_mul VALUES (8,1,'0'); -INSERT INTO num_exp_div VALUES (8,1,'NaN'); -INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047'); -INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047'); -INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407'); -INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615'); -INSERT INTO num_exp_add VALUES (8,3,'74885.31'); -INSERT INTO num_exp_sub VALUES (8,3,'74876.69'); -INSERT INTO num_exp_mul VALUES (8,3,'322737.11'); -INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410'); -INSERT INTO num_exp_add VALUES (8,4,'7874342.4119'); -INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119'); -INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839'); -INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956'); -INSERT INTO num_exp_add VALUES (8,5,'91278.038491'); -INSERT INTO num_exp_sub VALUES (8,5,'58483.961509'); -INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571'); -INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456'); -INSERT INTO num_exp_add VALUES (8,6,'168782.57763026'); -INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026'); -INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906'); -INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424'); -INSERT INTO num_exp_add VALUES (8,7,'-82953604'); -INSERT INTO num_exp_sub VALUES (8,7,'83103366'); -INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285'); -INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172'); -INSERT INTO num_exp_add VALUES (8,8,'149762'); -INSERT INTO num_exp_sub VALUES (8,8,'0'); -INSERT INTO num_exp_mul VALUES (8,8,'5607164161'); -INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000'); -INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420'); -INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420'); -INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020'); -INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735'); -INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420'); -INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420'); -INSERT INTO num_exp_mul VALUES (9,0,'0'); -INSERT INTO num_exp_div VALUES (9,0,'NaN'); -INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420'); -INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420'); -INSERT INTO num_exp_mul VALUES (9,1,'0'); -INSERT INTO num_exp_div VALUES (9,1,'NaN'); -INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467'); -INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627'); -INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740'); -INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526'); -INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420'); -INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420'); -INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020'); -INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677'); -INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420'); -INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420'); -INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980'); -INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484'); -INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420'); -INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420'); -INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220'); -INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807'); -INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160'); -INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680'); -INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920'); -INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280'); -INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420'); -INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580'); -INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700'); -INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689'); -INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420'); -INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420'); -INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020'); -INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748'); -INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840'); -INSERT INTO num_exp_sub VALUES (9,9,'0'); -INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400'); -INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_exp_add VALUES (0,0,0); +INSERT INTO num_exp_sub VALUES (0,0,0); +INSERT INTO num_exp_mul VALUES (0,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (0,0,double('NaN')); +INSERT INTO num_exp_add VALUES (0,1,0); +INSERT INTO num_exp_sub VALUES (0,1,0); +INSERT INTO num_exp_mul VALUES (0,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (0,1,double('NaN')); +INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047); +INSERT INTO num_exp_mul VALUES (0,2,0); +INSERT INTO num_exp_div VALUES (0,2,0); +INSERT INTO num_exp_add VALUES (0,3,4.31); +INSERT INTO num_exp_sub VALUES (0,3,-4.31); +INSERT INTO num_exp_mul VALUES (0,3,0); +INSERT INTO num_exp_div VALUES (0,3,0); +INSERT INTO num_exp_add VALUES (0,4,7799461.4119); +INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119); +INSERT INTO num_exp_mul VALUES (0,4,0); +INSERT INTO num_exp_div VALUES (0,4,0); +INSERT INTO num_exp_add VALUES (0,5,16397.038491); +INSERT INTO num_exp_sub VALUES (0,5,-16397.038491); +INSERT INTO num_exp_mul VALUES (0,5,0); +INSERT INTO num_exp_div VALUES (0,5,0); +INSERT INTO num_exp_add VALUES (0,6,93901.57763026); +INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026); +INSERT INTO num_exp_mul VALUES (0,6,0); +INSERT INTO num_exp_div VALUES (0,6,0); +INSERT INTO num_exp_add VALUES (0,7,-83028485); +INSERT INTO num_exp_sub VALUES (0,7,83028485); +INSERT INTO num_exp_mul VALUES (0,7,0); +INSERT INTO num_exp_div VALUES (0,7,0); +INSERT INTO num_exp_add VALUES (0,8,74881); +INSERT INTO num_exp_sub VALUES (0,8,-74881); +INSERT INTO num_exp_mul VALUES (0,8,0); +INSERT INTO num_exp_div VALUES (0,8,0); +INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420); +INSERT INTO num_exp_mul VALUES (0,9,0); +INSERT INTO num_exp_div VALUES (0,9,0); +INSERT INTO num_exp_add VALUES (1,0,0); +INSERT INTO num_exp_sub VALUES (1,0,0); +INSERT INTO num_exp_mul VALUES (1,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (1,0,double('NaN')); +INSERT INTO num_exp_add VALUES (1,1,0); +INSERT INTO num_exp_sub VALUES (1,1,0); +INSERT INTO num_exp_mul VALUES (1,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (1,1,double('NaN')); +INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047); +INSERT INTO num_exp_mul VALUES (1,2,0); +INSERT INTO num_exp_div VALUES (1,2,0); +INSERT INTO num_exp_add VALUES (1,3,4.31); +INSERT INTO num_exp_sub VALUES (1,3,-4.31); +INSERT INTO num_exp_mul VALUES (1,3,0); +INSERT INTO num_exp_div VALUES (1,3,0); +INSERT INTO num_exp_add VALUES (1,4,7799461.4119); +INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119); +INSERT INTO num_exp_mul VALUES (1,4,0); +INSERT INTO num_exp_div VALUES (1,4,0); +INSERT INTO num_exp_add VALUES (1,5,16397.038491); +INSERT INTO num_exp_sub VALUES (1,5,-16397.038491); +INSERT INTO num_exp_mul VALUES (1,5,0); +INSERT INTO num_exp_div VALUES (1,5,0); +INSERT INTO num_exp_add VALUES (1,6,93901.57763026); +INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026); +INSERT INTO num_exp_mul VALUES (1,6,0); +INSERT INTO num_exp_div VALUES (1,6,0); +INSERT INTO num_exp_add VALUES (1,7,-83028485); +INSERT INTO num_exp_sub VALUES (1,7,83028485); +INSERT INTO num_exp_mul VALUES (1,7,0); +INSERT INTO num_exp_div VALUES (1,7,0); +INSERT INTO num_exp_add VALUES (1,8,74881); +INSERT INTO num_exp_sub VALUES (1,8,-74881); +INSERT INTO num_exp_mul VALUES (1,8,0); +INSERT INTO num_exp_div VALUES (1,8,0); +INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420); +INSERT INTO num_exp_mul VALUES (1,9,0); +INSERT INTO num_exp_div VALUES (1,9,0); +INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047); +INSERT INTO num_exp_mul VALUES (2,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (2,0,double('NaN')); +INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047); +INSERT INTO num_exp_mul VALUES (2,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (2,1,double('NaN')); +INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094); +INSERT INTO num_exp_sub VALUES (2,2,0); +INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209); +INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047); +INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047); +INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257); +INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266); +INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047); +INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047); +INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593); +INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685); +INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047); +INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047); +INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077); +INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429); +INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787); +INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307); +INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222); +INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940); +INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047); +INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953); +INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795); +INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518); +INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047); +INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047); +INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407); +INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476); +INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467); +INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627); +INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740); +INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811); +INSERT INTO num_exp_add VALUES (3,0,4.31); +INSERT INTO num_exp_sub VALUES (3,0,4.31); +INSERT INTO num_exp_mul VALUES (3,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (3,0,double('NaN')); +INSERT INTO num_exp_add VALUES (3,1,4.31); +INSERT INTO num_exp_sub VALUES (3,1,4.31); +INSERT INTO num_exp_mul VALUES (3,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (3,1,double('NaN')); +INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047); +INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047); +INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257); +INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352); +INSERT INTO num_exp_add VALUES (3,3,8.62); +INSERT INTO num_exp_sub VALUES (3,3,0); +INSERT INTO num_exp_mul VALUES (3,3,18.5761); +INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (3,4,7799465.7219); +INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019); +INSERT INTO num_exp_mul VALUES (3,4,33615678.685289); +INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552); +INSERT INTO num_exp_add VALUES (3,5,16401.348491); +INSERT INTO num_exp_sub VALUES (3,5,-16392.728491); +INSERT INTO num_exp_mul VALUES (3,5,70671.23589621); +INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504); +INSERT INTO num_exp_add VALUES (3,6,93905.88763026); +INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026); +INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206); +INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595); +INSERT INTO num_exp_add VALUES (3,7,-83028480.69); +INSERT INTO num_exp_sub VALUES (3,7,83028489.31); +INSERT INTO num_exp_mul VALUES (3,7,-357852770.35); +INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240); +INSERT INTO num_exp_add VALUES (3,8,74885.31); +INSERT INTO num_exp_sub VALUES (3,8,-74876.69); +INSERT INTO num_exp_mul VALUES (3,8,322737.11); +INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553); +INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420); +INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420); +INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020); +INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854); +INSERT INTO num_exp_add VALUES (4,0,7799461.4119); +INSERT INTO num_exp_sub VALUES (4,0,7799461.4119); +INSERT INTO num_exp_mul VALUES (4,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (4,0,double('NaN')); +INSERT INTO num_exp_add VALUES (4,1,7799461.4119); +INSERT INTO num_exp_sub VALUES (4,1,7799461.4119); +INSERT INTO num_exp_mul VALUES (4,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (4,1,double('NaN')); +INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047); +INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047); +INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593); +INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385); +INSERT INTO num_exp_add VALUES (4,3,7799465.7219); +INSERT INTO num_exp_sub VALUES (4,3,7799457.1019); +INSERT INTO num_exp_mul VALUES (4,3,33615678.685289); +INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883); +INSERT INTO num_exp_add VALUES (4,4,15598922.8238); +INSERT INTO num_exp_sub VALUES (4,4,0); +INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161); +INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (4,5,7815858.450391); +INSERT INTO num_exp_sub VALUES (4,5,7783064.373409); +INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429); +INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061); +INSERT INTO num_exp_add VALUES (4,6,7893362.98953026); +INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974); +INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094); +INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606); +INSERT INTO num_exp_add VALUES (4,7,-75229023.5881); +INSERT INTO num_exp_sub VALUES (4,7,90827946.4119); +INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715); +INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637); +INSERT INTO num_exp_add VALUES (4,8,7874342.4119); +INSERT INTO num_exp_sub VALUES (4,8,7724580.4119); +INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839); +INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143); +INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420); +INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420); +INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980); +INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409); +INSERT INTO num_exp_add VALUES (5,0,16397.038491); +INSERT INTO num_exp_sub VALUES (5,0,16397.038491); +INSERT INTO num_exp_mul VALUES (5,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (5,0,double('NaN')); +INSERT INTO num_exp_add VALUES (5,1,16397.038491); +INSERT INTO num_exp_sub VALUES (5,1,16397.038491); +INSERT INTO num_exp_mul VALUES (5,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (5,1,double('NaN')); +INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047); +INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047); +INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077); +INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446); +INSERT INTO num_exp_add VALUES (5,3,16401.348491); +INSERT INTO num_exp_sub VALUES (5,3,16392.728491); +INSERT INTO num_exp_mul VALUES (5,3,70671.23589621); +INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584); +INSERT INTO num_exp_add VALUES (5,4,7815858.450391); +INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409); +INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429); +INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192); +INSERT INTO num_exp_add VALUES (5,5,32794.076982); +INSERT INTO num_exp_sub VALUES (5,5,0); +INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081); +INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (5,6,110298.61612126); +INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926); +INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766); +INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689); +INSERT INTO num_exp_add VALUES (5,7,-83012087.961509); +INSERT INTO num_exp_sub VALUES (5,7,83044882.038491); +INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135); +INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710); +INSERT INTO num_exp_add VALUES (5,8,91278.038491); +INSERT INTO num_exp_sub VALUES (5,8,-58483.961509); +INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571); +INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228); +INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420); +INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420); +INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220); +INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427); +INSERT INTO num_exp_add VALUES (6,0,93901.57763026); +INSERT INTO num_exp_sub VALUES (6,0,93901.57763026); +INSERT INTO num_exp_mul VALUES (6,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (6,0,double('NaN')); +INSERT INTO num_exp_add VALUES (6,1,93901.57763026); +INSERT INTO num_exp_sub VALUES (6,1,93901.57763026); +INSERT INTO num_exp_mul VALUES (6,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (6,1,double('NaN')); +INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787); +INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307); +INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222); +INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823); +INSERT INTO num_exp_add VALUES (6,3,93905.88763026); +INSERT INTO num_exp_sub VALUES (6,3,93897.26763026); +INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206); +INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907); +INSERT INTO num_exp_add VALUES (6,4,7893362.98953026); +INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974); +INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094); +INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469); +INSERT INTO num_exp_add VALUES (6,5,110298.61612126); +INSERT INTO num_exp_sub VALUES (6,5,77504.53913926); +INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766); +INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679); +INSERT INTO num_exp_add VALUES (6,6,187803.15526052); +INSERT INTO num_exp_sub VALUES (6,6,0); +INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676); +INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974); +INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026); +INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610); +INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980); +INSERT INTO num_exp_add VALUES (6,8,168782.57763026); +INSERT INTO num_exp_sub VALUES (6,8,19020.57763026); +INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906); +INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184); +INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160); +INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680); +INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920); +INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789); +INSERT INTO num_exp_add VALUES (7,0,-83028485); +INSERT INTO num_exp_sub VALUES (7,0,-83028485); +INSERT INTO num_exp_mul VALUES (7,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (7,0,double('NaN')); +INSERT INTO num_exp_add VALUES (7,1,-83028485); +INSERT INTO num_exp_sub VALUES (7,1,-83028485); +INSERT INTO num_exp_mul VALUES (7,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (7,1,double('NaN')); +INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047); +INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953); +INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795); +INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700); +INSERT INTO num_exp_add VALUES (7,3,-83028480.69); +INSERT INTO num_exp_sub VALUES (7,3,-83028489.31); +INSERT INTO num_exp_mul VALUES (7,3,-357852770.35); +INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974); +INSERT INTO num_exp_add VALUES (7,4,-75229023.5881); +INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119); +INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715); +INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686); +INSERT INTO num_exp_add VALUES (7,5,-83012087.961509); +INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491); +INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135); +INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574); +INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974); +INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026); +INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610); +INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294); +INSERT INTO num_exp_add VALUES (7,7,-166056970); +INSERT INTO num_exp_sub VALUES (7,7,0); +INSERT INTO num_exp_mul VALUES (7,7,6893729321395225); +INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (7,8,-82953604); +INSERT INTO num_exp_sub VALUES (7,8,-83103366); +INSERT INTO num_exp_mul VALUES (7,8,-6217255985285); +INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118); +INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420); +INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580); +INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700); +INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382); +INSERT INTO num_exp_add VALUES (8,0,74881); +INSERT INTO num_exp_sub VALUES (8,0,74881); +INSERT INTO num_exp_mul VALUES (8,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (8,0,double('NaN')); +INSERT INTO num_exp_add VALUES (8,1,74881); +INSERT INTO num_exp_sub VALUES (8,1,74881); +INSERT INTO num_exp_mul VALUES (8,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (8,1,double('NaN')); +INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047); +INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047); +INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407); +INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615); +INSERT INTO num_exp_add VALUES (8,3,74885.31); +INSERT INTO num_exp_sub VALUES (8,3,74876.69); +INSERT INTO num_exp_mul VALUES (8,3,322737.11); +INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410); +INSERT INTO num_exp_add VALUES (8,4,7874342.4119); +INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119); +INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839); +INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956); +INSERT INTO num_exp_add VALUES (8,5,91278.038491); +INSERT INTO num_exp_sub VALUES (8,5,58483.961509); +INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571); +INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456); +INSERT INTO num_exp_add VALUES (8,6,168782.57763026); +INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026); +INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906); +INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424); +INSERT INTO num_exp_add VALUES (8,7,-82953604); +INSERT INTO num_exp_sub VALUES (8,7,83103366); +INSERT INTO num_exp_mul VALUES (8,7,-6217255985285); +INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172); +INSERT INTO num_exp_add VALUES (8,8,149762); +INSERT INTO num_exp_sub VALUES (8,8,0); +INSERT INTO num_exp_mul VALUES (8,8,5607164161); +INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420); +INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420); +INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020); +INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735); +INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420); +INSERT INTO num_exp_mul VALUES (9,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (9,0,double('NaN')); +INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420); +INSERT INTO num_exp_mul VALUES (9,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (9,1,double('NaN')); +INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467); +INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627); +INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740); +INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526); +INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420); +INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420); +INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020); +INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677); +INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420); +INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420); +INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980); +INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484); +INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420); +INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420); +INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220); +INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807); +INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160); +INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680); +INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920); +INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280); +INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420); +INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580); +INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700); +INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689); +INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420); +INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420); +INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020); +INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748); +INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840); +INSERT INTO num_exp_sub VALUES (9,9,0); +INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400); +INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_exp_sqrt VALUES (0,'0'); -INSERT INTO num_exp_sqrt VALUES (1,'0'); -INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505'); -INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396'); -INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923'); -INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473'); -INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406'); -INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230'); -INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542'); -INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_exp_sqrt VALUES (0,0); +INSERT INTO num_exp_sqrt VALUES (1,0); +INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505); +INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396); +INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923); +INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473); +INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406); +INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230); +INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542); +INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_exp_ln VALUES (0,'NaN'); -INSERT INTO num_exp_ln VALUES (1,'NaN'); -INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514'); -INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971'); -INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464'); -INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038'); -INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127'); -INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991'); -INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668'); -INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_ln VALUES (0,double('NaN')); +INSERT INTO num_exp_ln VALUES (1,double('NaN')); +INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514); +INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971); +INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464); +INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038); +INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127); +INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991); +INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668); +INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_exp_log10 VALUES (0,'NaN'); -INSERT INTO num_exp_log10 VALUES (1,'NaN'); -INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459'); -INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075'); -INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345'); -INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626'); -INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671'); -INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914'); -INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138'); -INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_log10 VALUES (0,double('NaN')); +INSERT INTO num_exp_log10 VALUES (1,double('NaN')); +INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459); +INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075); +INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345); +INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626); +INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671); +INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914); +INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138); +INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_exp_power_10_ln VALUES (0,'NaN'); -INSERT INTO num_exp_power_10_ln VALUES (1,'NaN'); -INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184'); -INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393'); -INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636'); -INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098'); -INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067'); -INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627'); -INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952'); -INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')); +INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')); +INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184); +INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393); +INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636); +INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098); +INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067); +-- In Spark, decimal can only support precision up to 38 +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627); +INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952); +INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457); -- COMMIT TRANSACTION; -- BEGIN TRANSACTION; -INSERT INTO num_data VALUES (0, '0'); -INSERT INTO num_data VALUES (1, '0'); -INSERT INTO num_data VALUES (2, '-34338492.215397047'); -INSERT INTO num_data VALUES (3, '4.31'); -INSERT INTO num_data VALUES (4, '7799461.4119'); -INSERT INTO num_data VALUES (5, '16397.038491'); -INSERT INTO num_data VALUES (6, '93901.57763026'); -INSERT INTO num_data VALUES (7, '-83028485'); -INSERT INTO num_data VALUES (8, '74881'); -INSERT INTO num_data VALUES (9, '-24926804.045047420'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_data VALUES (0, 0); +INSERT INTO num_data VALUES (1, 0); +INSERT INTO num_data VALUES (2, -34338492.215397047); +INSERT INTO num_data VALUES (3, 4.31); +INSERT INTO num_data VALUES (4, 7799461.4119); +INSERT INTO num_data VALUES (5, 16397.038491); +INSERT INTO num_data VALUES (6, 93901.57763026); +INSERT INTO num_data VALUES (7, -83028485); +INSERT INTO num_data VALUES (8, 74881); +INSERT INTO num_data VALUES (9, -24926804.045047420); -- COMMIT TRANSACTION; SELECT * FROM num_data; @@ -657,16 +699,22 @@ SELECT AVG(val) FROM num_data; -- Check for appropriate rounding and overflow CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet; -INSERT INTO fract_only VALUES (1, '0.0'); -INSERT INTO fract_only VALUES (2, '0.1'); +INSERT INTO fract_only VALUES (1, 0.0); +INSERT INTO fract_only VALUES (2, 0.1); -- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL -- INSERT INTO fract_only VALUES (3, '1.0'); -- should fail -INSERT INTO fract_only VALUES (4, '-0.9999'); -INSERT INTO fract_only VALUES (5, '0.99994'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO fract_only VALUES (4, -0.9999); +INSERT INTO fract_only VALUES (5, 0.99994); -- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL -- INSERT INTO fract_only VALUES (6, '0.99995'); -- should fail -INSERT INTO fract_only VALUES (7, '0.00001'); -INSERT INTO fract_only VALUES (8, '0.00017'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO fract_only VALUES (7, 0.00001); +INSERT INTO fract_only VALUES (8, 0.00017); SELECT * FROM fract_only; DROP TABLE fract_only; @@ -682,13 +730,16 @@ SELECT decimal(float('-Infinity')); -- Simple check that ceil(), floor(), and round() work correctly CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet; -INSERT INTO ceil_floor_round VALUES ('-5.5'); -INSERT INTO ceil_floor_round VALUES ('-5.499999'); -INSERT INTO ceil_floor_round VALUES ('9.5'); -INSERT INTO ceil_floor_round VALUES ('9.4999999'); -INSERT INTO ceil_floor_round VALUES ('0.0'); -INSERT INTO ceil_floor_round VALUES ('0.0000001'); -INSERT INTO ceil_floor_round VALUES ('-0.000001'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO ceil_floor_round VALUES (-5.5); +INSERT INTO ceil_floor_round VALUES (-5.499999); +INSERT INTO ceil_floor_round VALUES (9.5); +INSERT INTO ceil_floor_round VALUES (9.4999999); +INSERT INTO ceil_floor_round VALUES (0.0); +INSERT INTO ceil_floor_round VALUES (0.0000001); +INSERT INTO ceil_floor_round VALUES (-0.000001); SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round; DROP TABLE ceil_floor_round; @@ -853,11 +904,14 @@ DROP TABLE ceil_floor_round; CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet; -- good inputs -INSERT INTO num_input_test VALUES (trim(' 123')); -INSERT INTO num_input_test VALUES (trim(' 3245874 ')); -INSERT INTO num_input_test VALUES (trim(' -93853')); -INSERT INTO num_input_test VALUES ('555.50'); -INSERT INTO num_input_test VALUES ('-555.50'); +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_input_test VALUES (double(trim(' 123'))); +INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))); +INSERT INTO num_input_test VALUES (double(trim(' -93853'))); +INSERT INTO num_input_test VALUES (555.50); +INSERT INTO num_input_test VALUES (-555.50); -- [SPARK-28315] Decimal can not accept NaN as input -- INSERT INTO num_input_test VALUES (trim('NaN ')); -- INSERT INTO num_input_test VALUES (trim(' nan')); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql index 260e8ea93d22d..bf69da295a960 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/timestamp.sql @@ -16,19 +16,23 @@ CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; -- block is entered exactly at local midnight; then 'now' and 'today' have -- the same values and the counts will come out different. -INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')); -- SELECT pg_sleep(0.1); -- BEGIN; -INSERT INTO TIMESTAMP_TBL VALUES ('now'); -INSERT INTO TIMESTAMP_TBL VALUES ('today'); -INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('today')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('yesterday')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow')); -- time zone should be ignored by this data type -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow EST')); -- [SPARK-29024] Ignore case while resolving time zones -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow Zulu'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow Zulu')); SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; @@ -54,7 +58,9 @@ TRUNCATE TABLE TIMESTAMP_TBL; -- Special values -- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); -- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); -INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('epoch')); -- [SPARK-27923] Spark SQL insert there obsolete special values to NULL -- Obsolete special values -- INSERT INTO TIMESTAMP_TBL VALUES ('invalid'); @@ -73,14 +79,16 @@ INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); -- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); -- ISO 8601 format -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05'); -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08')); -- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); -- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); -- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); -- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); -INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20')); -- POSIX format (note that the timezone abbrev is just decoration here) -- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index 29fcf61bd5b78..68ff2a9f68899 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO DATE_TBL VALUES ('1957-04-09') +INSERT INTO DATE_TBL VALUES (date('1957-04-09')) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO DATE_TBL VALUES ('1957-06-13') +INSERT INTO DATE_TBL VALUES (date('1957-06-13')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO DATE_TBL VALUES ('1996-02-28') +INSERT INTO DATE_TBL VALUES (date('1996-02-28')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO DATE_TBL VALUES ('1996-02-29') +INSERT INTO DATE_TBL VALUES (date('1996-02-29')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO DATE_TBL VALUES ('1996-03-01') +INSERT INTO DATE_TBL VALUES (date('1996-03-01')) -- !query 5 schema struct<> -- !query 5 output @@ -51,7 +51,7 @@ struct<> -- !query 6 -INSERT INTO DATE_TBL VALUES ('1996-03-02') +INSERT INTO DATE_TBL VALUES (date('1996-03-02')) -- !query 6 schema struct<> -- !query 6 output @@ -59,7 +59,7 @@ struct<> -- !query 7 -INSERT INTO DATE_TBL VALUES ('1997-02-28') +INSERT INTO DATE_TBL VALUES (date('1997-02-28')) -- !query 7 schema struct<> -- !query 7 output @@ -67,7 +67,7 @@ struct<> -- !query 8 -INSERT INTO DATE_TBL VALUES ('1997-03-01') +INSERT INTO DATE_TBL VALUES (date('1997-03-01')) -- !query 8 schema struct<> -- !query 8 output @@ -75,7 +75,7 @@ struct<> -- !query 9 -INSERT INTO DATE_TBL VALUES ('1997-03-02') +INSERT INTO DATE_TBL VALUES (date('1997-03-02')) -- !query 9 schema struct<> -- !query 9 output @@ -83,7 +83,7 @@ struct<> -- !query 10 -INSERT INTO DATE_TBL VALUES ('2000-04-01') +INSERT INTO DATE_TBL VALUES (date('2000-04-01')) -- !query 10 schema struct<> -- !query 10 output @@ -91,7 +91,7 @@ struct<> -- !query 11 -INSERT INTO DATE_TBL VALUES ('2000-04-02') +INSERT INTO DATE_TBL VALUES (date('2000-04-02')) -- !query 11 schema struct<> -- !query 11 output @@ -99,7 +99,7 @@ struct<> -- !query 12 -INSERT INTO DATE_TBL VALUES ('2000-04-03') +INSERT INTO DATE_TBL VALUES (date('2000-04-03')) -- !query 12 schema struct<> -- !query 12 output @@ -107,7 +107,7 @@ struct<> -- !query 13 -INSERT INTO DATE_TBL VALUES ('2038-04-08') +INSERT INTO DATE_TBL VALUES (date('2038-04-08')) -- !query 13 schema struct<> -- !query 13 output @@ -115,7 +115,7 @@ struct<> -- !query 14 -INSERT INTO DATE_TBL VALUES ('2039-04-09') +INSERT INTO DATE_TBL VALUES (date('2039-04-09')) -- !query 14 schema struct<> -- !query 14 output @@ -123,7 +123,7 @@ struct<> -- !query 15 -INSERT INTO DATE_TBL VALUES ('2040-04-10') +INSERT INTO DATE_TBL VALUES (date('2040-04-10')) -- !query 15 schema struct<> -- !query 15 output @@ -582,7 +582,7 @@ struct -- !query 55 SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 55 schema -struct +struct -- !query 55 output 0 @@ -790,7 +790,7 @@ true -- !query 81 SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 81 schema -struct +struct -- !query 81 output 20 @@ -798,7 +798,7 @@ struct -- !query 82 SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 82 schema -struct +struct -- !query 82 output 1001-01-01 00:07:02 @@ -814,7 +814,7 @@ struct -- !query 84 SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query 84 schema -struct +struct -- !query 84 output 1901-01-01 00:00:00 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out index c205f70a65f2d..64608a349b610 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO FLOAT4_TBL VALUES (' 0.0') +INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO FLOAT4_TBL VALUES ('1004.30 ') +INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO FLOAT4_TBL VALUES (' -34.84 ') +INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e+20') +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e-20') +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out index 9a785670fa950..d38e36e956985 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO FLOAT8_TBL VALUES (' 0.0 ') +INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 ')) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO FLOAT8_TBL VALUES ('1004.30 ') +INSERT INTO FLOAT8_TBL VALUES (double('1004.30 ')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO FLOAT8_TBL VALUES (' -34.84') +INSERT INTO FLOAT8_TBL VALUES (double(' -34.84')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200') +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200') +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200')) -- !query 5 schema struct<> -- !query 5 output @@ -684,7 +684,7 @@ struct<> -- !query 76 -INSERT INTO FLOAT8_TBL VALUES ('0.0') +INSERT INTO FLOAT8_TBL VALUES (double('0.0')) -- !query 76 schema struct<> -- !query 76 output @@ -692,7 +692,7 @@ struct<> -- !query 77 -INSERT INTO FLOAT8_TBL VALUES ('-34.84') +INSERT INTO FLOAT8_TBL VALUES (double('-34.84')) -- !query 77 schema struct<> -- !query 77 output @@ -700,7 +700,7 @@ struct<> -- !query 78 -INSERT INTO FLOAT8_TBL VALUES ('-1004.30') +INSERT INTO FLOAT8_TBL VALUES (double('-1004.30')) -- !query 78 schema struct<> -- !query 78 output @@ -708,7 +708,7 @@ struct<> -- !query 79 -INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200') +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200')) -- !query 79 schema struct<> -- !query 79 output @@ -716,7 +716,7 @@ struct<> -- !query 80 -INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200') +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200')) -- !query 80 schema struct<> -- !query 80 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out index 569d137891dd3..8f41accfc8c49 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO INT2_TBL VALUES (trim('0 ')) +INSERT INTO INT2_TBL VALUES (smallint(trim('0 '))) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO INT2_TBL VALUES (trim(' 1234 ')) +INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 '))) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO INT2_TBL VALUES (trim(' -1234')) +INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234'))) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO INT2_TBL VALUES ('32767') +INSERT INTO INT2_TBL VALUES (smallint('32767')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO INT2_TBL VALUES ('-32767') +INSERT INTO INT2_TBL VALUES (smallint('-32767')) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index 52221d58392f0..6b1a8eb4a5883 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO INT4_TBL VALUES (trim(' 0 ')) +INSERT INTO INT4_TBL VALUES (int(trim(' 0 '))) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO INT4_TBL VALUES (trim('123456 ')) +INSERT INTO INT4_TBL VALUES (int(trim('123456 '))) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO INT4_TBL VALUES (trim(' -123456')) +INSERT INTO INT4_TBL VALUES (int(trim(' -123456'))) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO INT4_TBL VALUES ('2147483647') +INSERT INTO INT4_TBL VALUES (int('2147483647')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO INT4_TBL VALUES ('-2147483647') +INSERT INTO INT4_TBL VALUES (int('-2147483647')) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index 20bb5dbda645f..c4d8affe75267 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO INT8_TBL VALUES(trim(' 123 '),trim(' 456')) +INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO INT8_TBL VALUES(trim('123 '),'4567890123456789') +INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO INT8_TBL VALUES('4567890123456789','123') +INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO INT8_TBL VALUES(+4567890123456789,'4567890123456789') +INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO INT8_TBL VALUES('+4567890123456789','-4567890123456789') +INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index ed649feaaebb2..5ffa29a93d5de 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -83,7 +83,7 @@ struct<> -- !query 10 -INSERT INTO num_exp_add VALUES (0,0,'0') +INSERT INTO num_exp_add VALUES (0,0,0) -- !query 10 schema struct<> -- !query 10 output @@ -91,7 +91,7 @@ struct<> -- !query 11 -INSERT INTO num_exp_sub VALUES (0,0,'0') +INSERT INTO num_exp_sub VALUES (0,0,0) -- !query 11 schema struct<> -- !query 11 output @@ -99,7 +99,7 @@ struct<> -- !query 12 -INSERT INTO num_exp_mul VALUES (0,0,'0') +INSERT INTO num_exp_mul VALUES (0,0,0) -- !query 12 schema struct<> -- !query 12 output @@ -107,7 +107,7 @@ struct<> -- !query 13 -INSERT INTO num_exp_div VALUES (0,0,'NaN') +INSERT INTO num_exp_div VALUES (0,0,double('NaN')) -- !query 13 schema struct<> -- !query 13 output @@ -115,7 +115,7 @@ struct<> -- !query 14 -INSERT INTO num_exp_add VALUES (0,1,'0') +INSERT INTO num_exp_add VALUES (0,1,0) -- !query 14 schema struct<> -- !query 14 output @@ -123,7 +123,7 @@ struct<> -- !query 15 -INSERT INTO num_exp_sub VALUES (0,1,'0') +INSERT INTO num_exp_sub VALUES (0,1,0) -- !query 15 schema struct<> -- !query 15 output @@ -131,7 +131,7 @@ struct<> -- !query 16 -INSERT INTO num_exp_mul VALUES (0,1,'0') +INSERT INTO num_exp_mul VALUES (0,1,0) -- !query 16 schema struct<> -- !query 16 output @@ -139,7 +139,7 @@ struct<> -- !query 17 -INSERT INTO num_exp_div VALUES (0,1,'NaN') +INSERT INTO num_exp_div VALUES (0,1,double('NaN')) -- !query 17 schema struct<> -- !query 17 output @@ -147,7 +147,7 @@ struct<> -- !query 18 -INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047') +INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047) -- !query 18 schema struct<> -- !query 18 output @@ -155,7 +155,7 @@ struct<> -- !query 19 -INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047') +INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047) -- !query 19 schema struct<> -- !query 19 output @@ -163,7 +163,7 @@ struct<> -- !query 20 -INSERT INTO num_exp_mul VALUES (0,2,'0') +INSERT INTO num_exp_mul VALUES (0,2,0) -- !query 20 schema struct<> -- !query 20 output @@ -171,7 +171,7 @@ struct<> -- !query 21 -INSERT INTO num_exp_div VALUES (0,2,'0') +INSERT INTO num_exp_div VALUES (0,2,0) -- !query 21 schema struct<> -- !query 21 output @@ -179,7 +179,7 @@ struct<> -- !query 22 -INSERT INTO num_exp_add VALUES (0,3,'4.31') +INSERT INTO num_exp_add VALUES (0,3,4.31) -- !query 22 schema struct<> -- !query 22 output @@ -187,7 +187,7 @@ struct<> -- !query 23 -INSERT INTO num_exp_sub VALUES (0,3,'-4.31') +INSERT INTO num_exp_sub VALUES (0,3,-4.31) -- !query 23 schema struct<> -- !query 23 output @@ -195,7 +195,7 @@ struct<> -- !query 24 -INSERT INTO num_exp_mul VALUES (0,3,'0') +INSERT INTO num_exp_mul VALUES (0,3,0) -- !query 24 schema struct<> -- !query 24 output @@ -203,7 +203,7 @@ struct<> -- !query 25 -INSERT INTO num_exp_div VALUES (0,3,'0') +INSERT INTO num_exp_div VALUES (0,3,0) -- !query 25 schema struct<> -- !query 25 output @@ -211,7 +211,7 @@ struct<> -- !query 26 -INSERT INTO num_exp_add VALUES (0,4,'7799461.4119') +INSERT INTO num_exp_add VALUES (0,4,7799461.4119) -- !query 26 schema struct<> -- !query 26 output @@ -219,7 +219,7 @@ struct<> -- !query 27 -INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119') +INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119) -- !query 27 schema struct<> -- !query 27 output @@ -227,7 +227,7 @@ struct<> -- !query 28 -INSERT INTO num_exp_mul VALUES (0,4,'0') +INSERT INTO num_exp_mul VALUES (0,4,0) -- !query 28 schema struct<> -- !query 28 output @@ -235,7 +235,7 @@ struct<> -- !query 29 -INSERT INTO num_exp_div VALUES (0,4,'0') +INSERT INTO num_exp_div VALUES (0,4,0) -- !query 29 schema struct<> -- !query 29 output @@ -243,7 +243,7 @@ struct<> -- !query 30 -INSERT INTO num_exp_add VALUES (0,5,'16397.038491') +INSERT INTO num_exp_add VALUES (0,5,16397.038491) -- !query 30 schema struct<> -- !query 30 output @@ -251,7 +251,7 @@ struct<> -- !query 31 -INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491') +INSERT INTO num_exp_sub VALUES (0,5,-16397.038491) -- !query 31 schema struct<> -- !query 31 output @@ -259,7 +259,7 @@ struct<> -- !query 32 -INSERT INTO num_exp_mul VALUES (0,5,'0') +INSERT INTO num_exp_mul VALUES (0,5,0) -- !query 32 schema struct<> -- !query 32 output @@ -267,7 +267,7 @@ struct<> -- !query 33 -INSERT INTO num_exp_div VALUES (0,5,'0') +INSERT INTO num_exp_div VALUES (0,5,0) -- !query 33 schema struct<> -- !query 33 output @@ -275,7 +275,7 @@ struct<> -- !query 34 -INSERT INTO num_exp_add VALUES (0,6,'93901.57763026') +INSERT INTO num_exp_add VALUES (0,6,93901.57763026) -- !query 34 schema struct<> -- !query 34 output @@ -283,7 +283,7 @@ struct<> -- !query 35 -INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026') +INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026) -- !query 35 schema struct<> -- !query 35 output @@ -291,7 +291,7 @@ struct<> -- !query 36 -INSERT INTO num_exp_mul VALUES (0,6,'0') +INSERT INTO num_exp_mul VALUES (0,6,0) -- !query 36 schema struct<> -- !query 36 output @@ -299,7 +299,7 @@ struct<> -- !query 37 -INSERT INTO num_exp_div VALUES (0,6,'0') +INSERT INTO num_exp_div VALUES (0,6,0) -- !query 37 schema struct<> -- !query 37 output @@ -307,7 +307,7 @@ struct<> -- !query 38 -INSERT INTO num_exp_add VALUES (0,7,'-83028485') +INSERT INTO num_exp_add VALUES (0,7,-83028485) -- !query 38 schema struct<> -- !query 38 output @@ -315,7 +315,7 @@ struct<> -- !query 39 -INSERT INTO num_exp_sub VALUES (0,7,'83028485') +INSERT INTO num_exp_sub VALUES (0,7,83028485) -- !query 39 schema struct<> -- !query 39 output @@ -323,7 +323,7 @@ struct<> -- !query 40 -INSERT INTO num_exp_mul VALUES (0,7,'0') +INSERT INTO num_exp_mul VALUES (0,7,0) -- !query 40 schema struct<> -- !query 40 output @@ -331,7 +331,7 @@ struct<> -- !query 41 -INSERT INTO num_exp_div VALUES (0,7,'0') +INSERT INTO num_exp_div VALUES (0,7,0) -- !query 41 schema struct<> -- !query 41 output @@ -339,7 +339,7 @@ struct<> -- !query 42 -INSERT INTO num_exp_add VALUES (0,8,'74881') +INSERT INTO num_exp_add VALUES (0,8,74881) -- !query 42 schema struct<> -- !query 42 output @@ -347,7 +347,7 @@ struct<> -- !query 43 -INSERT INTO num_exp_sub VALUES (0,8,'-74881') +INSERT INTO num_exp_sub VALUES (0,8,-74881) -- !query 43 schema struct<> -- !query 43 output @@ -355,7 +355,7 @@ struct<> -- !query 44 -INSERT INTO num_exp_mul VALUES (0,8,'0') +INSERT INTO num_exp_mul VALUES (0,8,0) -- !query 44 schema struct<> -- !query 44 output @@ -363,7 +363,7 @@ struct<> -- !query 45 -INSERT INTO num_exp_div VALUES (0,8,'0') +INSERT INTO num_exp_div VALUES (0,8,0) -- !query 45 schema struct<> -- !query 45 output @@ -371,7 +371,7 @@ struct<> -- !query 46 -INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420') +INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420) -- !query 46 schema struct<> -- !query 46 output @@ -379,7 +379,7 @@ struct<> -- !query 47 -INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420') +INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420) -- !query 47 schema struct<> -- !query 47 output @@ -387,7 +387,7 @@ struct<> -- !query 48 -INSERT INTO num_exp_mul VALUES (0,9,'0') +INSERT INTO num_exp_mul VALUES (0,9,0) -- !query 48 schema struct<> -- !query 48 output @@ -395,7 +395,7 @@ struct<> -- !query 49 -INSERT INTO num_exp_div VALUES (0,9,'0') +INSERT INTO num_exp_div VALUES (0,9,0) -- !query 49 schema struct<> -- !query 49 output @@ -403,7 +403,7 @@ struct<> -- !query 50 -INSERT INTO num_exp_add VALUES (1,0,'0') +INSERT INTO num_exp_add VALUES (1,0,0) -- !query 50 schema struct<> -- !query 50 output @@ -411,7 +411,7 @@ struct<> -- !query 51 -INSERT INTO num_exp_sub VALUES (1,0,'0') +INSERT INTO num_exp_sub VALUES (1,0,0) -- !query 51 schema struct<> -- !query 51 output @@ -419,7 +419,7 @@ struct<> -- !query 52 -INSERT INTO num_exp_mul VALUES (1,0,'0') +INSERT INTO num_exp_mul VALUES (1,0,0) -- !query 52 schema struct<> -- !query 52 output @@ -427,7 +427,7 @@ struct<> -- !query 53 -INSERT INTO num_exp_div VALUES (1,0,'NaN') +INSERT INTO num_exp_div VALUES (1,0,double('NaN')) -- !query 53 schema struct<> -- !query 53 output @@ -435,7 +435,7 @@ struct<> -- !query 54 -INSERT INTO num_exp_add VALUES (1,1,'0') +INSERT INTO num_exp_add VALUES (1,1,0) -- !query 54 schema struct<> -- !query 54 output @@ -443,7 +443,7 @@ struct<> -- !query 55 -INSERT INTO num_exp_sub VALUES (1,1,'0') +INSERT INTO num_exp_sub VALUES (1,1,0) -- !query 55 schema struct<> -- !query 55 output @@ -451,7 +451,7 @@ struct<> -- !query 56 -INSERT INTO num_exp_mul VALUES (1,1,'0') +INSERT INTO num_exp_mul VALUES (1,1,0) -- !query 56 schema struct<> -- !query 56 output @@ -459,7 +459,7 @@ struct<> -- !query 57 -INSERT INTO num_exp_div VALUES (1,1,'NaN') +INSERT INTO num_exp_div VALUES (1,1,double('NaN')) -- !query 57 schema struct<> -- !query 57 output @@ -467,7 +467,7 @@ struct<> -- !query 58 -INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047') +INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047) -- !query 58 schema struct<> -- !query 58 output @@ -475,7 +475,7 @@ struct<> -- !query 59 -INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047') +INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047) -- !query 59 schema struct<> -- !query 59 output @@ -483,7 +483,7 @@ struct<> -- !query 60 -INSERT INTO num_exp_mul VALUES (1,2,'0') +INSERT INTO num_exp_mul VALUES (1,2,0) -- !query 60 schema struct<> -- !query 60 output @@ -491,7 +491,7 @@ struct<> -- !query 61 -INSERT INTO num_exp_div VALUES (1,2,'0') +INSERT INTO num_exp_div VALUES (1,2,0) -- !query 61 schema struct<> -- !query 61 output @@ -499,7 +499,7 @@ struct<> -- !query 62 -INSERT INTO num_exp_add VALUES (1,3,'4.31') +INSERT INTO num_exp_add VALUES (1,3,4.31) -- !query 62 schema struct<> -- !query 62 output @@ -507,7 +507,7 @@ struct<> -- !query 63 -INSERT INTO num_exp_sub VALUES (1,3,'-4.31') +INSERT INTO num_exp_sub VALUES (1,3,-4.31) -- !query 63 schema struct<> -- !query 63 output @@ -515,7 +515,7 @@ struct<> -- !query 64 -INSERT INTO num_exp_mul VALUES (1,3,'0') +INSERT INTO num_exp_mul VALUES (1,3,0) -- !query 64 schema struct<> -- !query 64 output @@ -523,7 +523,7 @@ struct<> -- !query 65 -INSERT INTO num_exp_div VALUES (1,3,'0') +INSERT INTO num_exp_div VALUES (1,3,0) -- !query 65 schema struct<> -- !query 65 output @@ -531,7 +531,7 @@ struct<> -- !query 66 -INSERT INTO num_exp_add VALUES (1,4,'7799461.4119') +INSERT INTO num_exp_add VALUES (1,4,7799461.4119) -- !query 66 schema struct<> -- !query 66 output @@ -539,7 +539,7 @@ struct<> -- !query 67 -INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119') +INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119) -- !query 67 schema struct<> -- !query 67 output @@ -547,7 +547,7 @@ struct<> -- !query 68 -INSERT INTO num_exp_mul VALUES (1,4,'0') +INSERT INTO num_exp_mul VALUES (1,4,0) -- !query 68 schema struct<> -- !query 68 output @@ -555,7 +555,7 @@ struct<> -- !query 69 -INSERT INTO num_exp_div VALUES (1,4,'0') +INSERT INTO num_exp_div VALUES (1,4,0) -- !query 69 schema struct<> -- !query 69 output @@ -563,7 +563,7 @@ struct<> -- !query 70 -INSERT INTO num_exp_add VALUES (1,5,'16397.038491') +INSERT INTO num_exp_add VALUES (1,5,16397.038491) -- !query 70 schema struct<> -- !query 70 output @@ -571,7 +571,7 @@ struct<> -- !query 71 -INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491') +INSERT INTO num_exp_sub VALUES (1,5,-16397.038491) -- !query 71 schema struct<> -- !query 71 output @@ -579,7 +579,7 @@ struct<> -- !query 72 -INSERT INTO num_exp_mul VALUES (1,5,'0') +INSERT INTO num_exp_mul VALUES (1,5,0) -- !query 72 schema struct<> -- !query 72 output @@ -587,7 +587,7 @@ struct<> -- !query 73 -INSERT INTO num_exp_div VALUES (1,5,'0') +INSERT INTO num_exp_div VALUES (1,5,0) -- !query 73 schema struct<> -- !query 73 output @@ -595,7 +595,7 @@ struct<> -- !query 74 -INSERT INTO num_exp_add VALUES (1,6,'93901.57763026') +INSERT INTO num_exp_add VALUES (1,6,93901.57763026) -- !query 74 schema struct<> -- !query 74 output @@ -603,7 +603,7 @@ struct<> -- !query 75 -INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026') +INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026) -- !query 75 schema struct<> -- !query 75 output @@ -611,7 +611,7 @@ struct<> -- !query 76 -INSERT INTO num_exp_mul VALUES (1,6,'0') +INSERT INTO num_exp_mul VALUES (1,6,0) -- !query 76 schema struct<> -- !query 76 output @@ -619,7 +619,7 @@ struct<> -- !query 77 -INSERT INTO num_exp_div VALUES (1,6,'0') +INSERT INTO num_exp_div VALUES (1,6,0) -- !query 77 schema struct<> -- !query 77 output @@ -627,7 +627,7 @@ struct<> -- !query 78 -INSERT INTO num_exp_add VALUES (1,7,'-83028485') +INSERT INTO num_exp_add VALUES (1,7,-83028485) -- !query 78 schema struct<> -- !query 78 output @@ -635,7 +635,7 @@ struct<> -- !query 79 -INSERT INTO num_exp_sub VALUES (1,7,'83028485') +INSERT INTO num_exp_sub VALUES (1,7,83028485) -- !query 79 schema struct<> -- !query 79 output @@ -643,7 +643,7 @@ struct<> -- !query 80 -INSERT INTO num_exp_mul VALUES (1,7,'0') +INSERT INTO num_exp_mul VALUES (1,7,0) -- !query 80 schema struct<> -- !query 80 output @@ -651,7 +651,7 @@ struct<> -- !query 81 -INSERT INTO num_exp_div VALUES (1,7,'0') +INSERT INTO num_exp_div VALUES (1,7,0) -- !query 81 schema struct<> -- !query 81 output @@ -659,7 +659,7 @@ struct<> -- !query 82 -INSERT INTO num_exp_add VALUES (1,8,'74881') +INSERT INTO num_exp_add VALUES (1,8,74881) -- !query 82 schema struct<> -- !query 82 output @@ -667,7 +667,7 @@ struct<> -- !query 83 -INSERT INTO num_exp_sub VALUES (1,8,'-74881') +INSERT INTO num_exp_sub VALUES (1,8,-74881) -- !query 83 schema struct<> -- !query 83 output @@ -675,7 +675,7 @@ struct<> -- !query 84 -INSERT INTO num_exp_mul VALUES (1,8,'0') +INSERT INTO num_exp_mul VALUES (1,8,0) -- !query 84 schema struct<> -- !query 84 output @@ -683,7 +683,7 @@ struct<> -- !query 85 -INSERT INTO num_exp_div VALUES (1,8,'0') +INSERT INTO num_exp_div VALUES (1,8,0) -- !query 85 schema struct<> -- !query 85 output @@ -691,7 +691,7 @@ struct<> -- !query 86 -INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420') +INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420) -- !query 86 schema struct<> -- !query 86 output @@ -699,7 +699,7 @@ struct<> -- !query 87 -INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420') +INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420) -- !query 87 schema struct<> -- !query 87 output @@ -707,7 +707,7 @@ struct<> -- !query 88 -INSERT INTO num_exp_mul VALUES (1,9,'0') +INSERT INTO num_exp_mul VALUES (1,9,0) -- !query 88 schema struct<> -- !query 88 output @@ -715,7 +715,7 @@ struct<> -- !query 89 -INSERT INTO num_exp_div VALUES (1,9,'0') +INSERT INTO num_exp_div VALUES (1,9,0) -- !query 89 schema struct<> -- !query 89 output @@ -723,7 +723,7 @@ struct<> -- !query 90 -INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047') +INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047) -- !query 90 schema struct<> -- !query 90 output @@ -731,7 +731,7 @@ struct<> -- !query 91 -INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047') +INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047) -- !query 91 schema struct<> -- !query 91 output @@ -739,7 +739,7 @@ struct<> -- !query 92 -INSERT INTO num_exp_mul VALUES (2,0,'0') +INSERT INTO num_exp_mul VALUES (2,0,0) -- !query 92 schema struct<> -- !query 92 output @@ -747,7 +747,7 @@ struct<> -- !query 93 -INSERT INTO num_exp_div VALUES (2,0,'NaN') +INSERT INTO num_exp_div VALUES (2,0,double('NaN')) -- !query 93 schema struct<> -- !query 93 output @@ -755,7 +755,7 @@ struct<> -- !query 94 -INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047') +INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047) -- !query 94 schema struct<> -- !query 94 output @@ -763,7 +763,7 @@ struct<> -- !query 95 -INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047') +INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047) -- !query 95 schema struct<> -- !query 95 output @@ -771,7 +771,7 @@ struct<> -- !query 96 -INSERT INTO num_exp_mul VALUES (2,1,'0') +INSERT INTO num_exp_mul VALUES (2,1,0) -- !query 96 schema struct<> -- !query 96 output @@ -779,7 +779,7 @@ struct<> -- !query 97 -INSERT INTO num_exp_div VALUES (2,1,'NaN') +INSERT INTO num_exp_div VALUES (2,1,double('NaN')) -- !query 97 schema struct<> -- !query 97 output @@ -787,7 +787,7 @@ struct<> -- !query 98 -INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094') +INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094) -- !query 98 schema struct<> -- !query 98 output @@ -795,7 +795,7 @@ struct<> -- !query 99 -INSERT INTO num_exp_sub VALUES (2,2,'0') +INSERT INTO num_exp_sub VALUES (2,2,0) -- !query 99 schema struct<> -- !query 99 output @@ -803,7 +803,7 @@ struct<> -- !query 100 -INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209') +INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209) -- !query 100 schema struct<> -- !query 100 output @@ -811,7 +811,7 @@ struct<> -- !query 101 -INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000) -- !query 101 schema struct<> -- !query 101 output @@ -819,7 +819,7 @@ struct<> -- !query 102 -INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047') +INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047) -- !query 102 schema struct<> -- !query 102 output @@ -827,7 +827,7 @@ struct<> -- !query 103 -INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047') +INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047) -- !query 103 schema struct<> -- !query 103 output @@ -835,7 +835,7 @@ struct<> -- !query 104 -INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257') +INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257) -- !query 104 schema struct<> -- !query 104 output @@ -843,7 +843,7 @@ struct<> -- !query 105 -INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266') +INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266) -- !query 105 schema struct<> -- !query 105 output @@ -851,7 +851,7 @@ struct<> -- !query 106 -INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047') +INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047) -- !query 106 schema struct<> -- !query 106 output @@ -859,7 +859,7 @@ struct<> -- !query 107 -INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047') +INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047) -- !query 107 schema struct<> -- !query 107 output @@ -867,7 +867,7 @@ struct<> -- !query 108 -INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593') +INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593) -- !query 108 schema struct<> -- !query 108 output @@ -875,7 +875,7 @@ struct<> -- !query 109 -INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685') +INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685) -- !query 109 schema struct<> -- !query 109 output @@ -883,7 +883,7 @@ struct<> -- !query 110 -INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047') +INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047) -- !query 110 schema struct<> -- !query 110 output @@ -891,7 +891,7 @@ struct<> -- !query 111 -INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047') +INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047) -- !query 111 schema struct<> -- !query 111 output @@ -899,7 +899,7 @@ struct<> -- !query 112 -INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077') +INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077) -- !query 112 schema struct<> -- !query 112 output @@ -907,7 +907,7 @@ struct<> -- !query 113 -INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429') +INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429) -- !query 113 schema struct<> -- !query 113 output @@ -915,7 +915,7 @@ struct<> -- !query 114 -INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787') +INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787) -- !query 114 schema struct<> -- !query 114 output @@ -923,7 +923,7 @@ struct<> -- !query 115 -INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307') +INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307) -- !query 115 schema struct<> -- !query 115 output @@ -931,7 +931,7 @@ struct<> -- !query 116 -INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222') +INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222) -- !query 116 schema struct<> -- !query 116 output @@ -939,7 +939,7 @@ struct<> -- !query 117 -INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940') +INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940) -- !query 117 schema struct<> -- !query 117 output @@ -947,7 +947,7 @@ struct<> -- !query 118 -INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047') +INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047) -- !query 118 schema struct<> -- !query 118 output @@ -955,7 +955,7 @@ struct<> -- !query 119 -INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953') +INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953) -- !query 119 schema struct<> -- !query 119 output @@ -963,7 +963,7 @@ struct<> -- !query 120 -INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795') +INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795) -- !query 120 schema struct<> -- !query 120 output @@ -971,7 +971,7 @@ struct<> -- !query 121 -INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518') +INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518) -- !query 121 schema struct<> -- !query 121 output @@ -979,7 +979,7 @@ struct<> -- !query 122 -INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047') +INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047) -- !query 122 schema struct<> -- !query 122 output @@ -987,7 +987,7 @@ struct<> -- !query 123 -INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047') +INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047) -- !query 123 schema struct<> -- !query 123 output @@ -995,7 +995,7 @@ struct<> -- !query 124 -INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407') +INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407) -- !query 124 schema struct<> -- !query 124 output @@ -1003,7 +1003,7 @@ struct<> -- !query 125 -INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476') +INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476) -- !query 125 schema struct<> -- !query 125 output @@ -1011,7 +1011,7 @@ struct<> -- !query 126 -INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467') +INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467) -- !query 126 schema struct<> -- !query 126 output @@ -1019,7 +1019,7 @@ struct<> -- !query 127 -INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627') +INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627) -- !query 127 schema struct<> -- !query 127 output @@ -1027,7 +1027,7 @@ struct<> -- !query 128 -INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740') +INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740) -- !query 128 schema struct<> -- !query 128 output @@ -1035,7 +1035,7 @@ struct<> -- !query 129 -INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811') +INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811) -- !query 129 schema struct<> -- !query 129 output @@ -1043,7 +1043,7 @@ struct<> -- !query 130 -INSERT INTO num_exp_add VALUES (3,0,'4.31') +INSERT INTO num_exp_add VALUES (3,0,4.31) -- !query 130 schema struct<> -- !query 130 output @@ -1051,7 +1051,7 @@ struct<> -- !query 131 -INSERT INTO num_exp_sub VALUES (3,0,'4.31') +INSERT INTO num_exp_sub VALUES (3,0,4.31) -- !query 131 schema struct<> -- !query 131 output @@ -1059,7 +1059,7 @@ struct<> -- !query 132 -INSERT INTO num_exp_mul VALUES (3,0,'0') +INSERT INTO num_exp_mul VALUES (3,0,0) -- !query 132 schema struct<> -- !query 132 output @@ -1067,7 +1067,7 @@ struct<> -- !query 133 -INSERT INTO num_exp_div VALUES (3,0,'NaN') +INSERT INTO num_exp_div VALUES (3,0,double('NaN')) -- !query 133 schema struct<> -- !query 133 output @@ -1075,7 +1075,7 @@ struct<> -- !query 134 -INSERT INTO num_exp_add VALUES (3,1,'4.31') +INSERT INTO num_exp_add VALUES (3,1,4.31) -- !query 134 schema struct<> -- !query 134 output @@ -1083,7 +1083,7 @@ struct<> -- !query 135 -INSERT INTO num_exp_sub VALUES (3,1,'4.31') +INSERT INTO num_exp_sub VALUES (3,1,4.31) -- !query 135 schema struct<> -- !query 135 output @@ -1091,7 +1091,7 @@ struct<> -- !query 136 -INSERT INTO num_exp_mul VALUES (3,1,'0') +INSERT INTO num_exp_mul VALUES (3,1,0) -- !query 136 schema struct<> -- !query 136 output @@ -1099,7 +1099,7 @@ struct<> -- !query 137 -INSERT INTO num_exp_div VALUES (3,1,'NaN') +INSERT INTO num_exp_div VALUES (3,1,double('NaN')) -- !query 137 schema struct<> -- !query 137 output @@ -1107,7 +1107,7 @@ struct<> -- !query 138 -INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047') +INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047) -- !query 138 schema struct<> -- !query 138 output @@ -1115,7 +1115,7 @@ struct<> -- !query 139 -INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047') +INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047) -- !query 139 schema struct<> -- !query 139 output @@ -1123,7 +1123,7 @@ struct<> -- !query 140 -INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257') +INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257) -- !query 140 schema struct<> -- !query 140 output @@ -1131,7 +1131,7 @@ struct<> -- !query 141 -INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352') +INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352) -- !query 141 schema struct<> -- !query 141 output @@ -1139,7 +1139,7 @@ struct<> -- !query 142 -INSERT INTO num_exp_add VALUES (3,3,'8.62') +INSERT INTO num_exp_add VALUES (3,3,8.62) -- !query 142 schema struct<> -- !query 142 output @@ -1147,7 +1147,7 @@ struct<> -- !query 143 -INSERT INTO num_exp_sub VALUES (3,3,'0') +INSERT INTO num_exp_sub VALUES (3,3,0) -- !query 143 schema struct<> -- !query 143 output @@ -1155,7 +1155,7 @@ struct<> -- !query 144 -INSERT INTO num_exp_mul VALUES (3,3,'18.5761') +INSERT INTO num_exp_mul VALUES (3,3,18.5761) -- !query 144 schema struct<> -- !query 144 output @@ -1163,7 +1163,7 @@ struct<> -- !query 145 -INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000) -- !query 145 schema struct<> -- !query 145 output @@ -1171,7 +1171,7 @@ struct<> -- !query 146 -INSERT INTO num_exp_add VALUES (3,4,'7799465.7219') +INSERT INTO num_exp_add VALUES (3,4,7799465.7219) -- !query 146 schema struct<> -- !query 146 output @@ -1179,7 +1179,7 @@ struct<> -- !query 147 -INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019') +INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019) -- !query 147 schema struct<> -- !query 147 output @@ -1187,7 +1187,7 @@ struct<> -- !query 148 -INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289') +INSERT INTO num_exp_mul VALUES (3,4,33615678.685289) -- !query 148 schema struct<> -- !query 148 output @@ -1195,7 +1195,7 @@ struct<> -- !query 149 -INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552') +INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552) -- !query 149 schema struct<> -- !query 149 output @@ -1203,7 +1203,7 @@ struct<> -- !query 150 -INSERT INTO num_exp_add VALUES (3,5,'16401.348491') +INSERT INTO num_exp_add VALUES (3,5,16401.348491) -- !query 150 schema struct<> -- !query 150 output @@ -1211,7 +1211,7 @@ struct<> -- !query 151 -INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491') +INSERT INTO num_exp_sub VALUES (3,5,-16392.728491) -- !query 151 schema struct<> -- !query 151 output @@ -1219,7 +1219,7 @@ struct<> -- !query 152 -INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621') +INSERT INTO num_exp_mul VALUES (3,5,70671.23589621) -- !query 152 schema struct<> -- !query 152 output @@ -1227,7 +1227,7 @@ struct<> -- !query 153 -INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504') +INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504) -- !query 153 schema struct<> -- !query 153 output @@ -1235,7 +1235,7 @@ struct<> -- !query 154 -INSERT INTO num_exp_add VALUES (3,6,'93905.88763026') +INSERT INTO num_exp_add VALUES (3,6,93905.88763026) -- !query 154 schema struct<> -- !query 154 output @@ -1243,7 +1243,7 @@ struct<> -- !query 155 -INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026') +INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026) -- !query 155 schema struct<> -- !query 155 output @@ -1251,7 +1251,7 @@ struct<> -- !query 156 -INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206') +INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206) -- !query 156 schema struct<> -- !query 156 output @@ -1259,7 +1259,7 @@ struct<> -- !query 157 -INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595') +INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595) -- !query 157 schema struct<> -- !query 157 output @@ -1267,7 +1267,7 @@ struct<> -- !query 158 -INSERT INTO num_exp_add VALUES (3,7,'-83028480.69') +INSERT INTO num_exp_add VALUES (3,7,-83028480.69) -- !query 158 schema struct<> -- !query 158 output @@ -1275,7 +1275,7 @@ struct<> -- !query 159 -INSERT INTO num_exp_sub VALUES (3,7,'83028489.31') +INSERT INTO num_exp_sub VALUES (3,7,83028489.31) -- !query 159 schema struct<> -- !query 159 output @@ -1283,7 +1283,7 @@ struct<> -- !query 160 -INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35') +INSERT INTO num_exp_mul VALUES (3,7,-357852770.35) -- !query 160 schema struct<> -- !query 160 output @@ -1291,7 +1291,7 @@ struct<> -- !query 161 -INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240') +INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240) -- !query 161 schema struct<> -- !query 161 output @@ -1299,7 +1299,7 @@ struct<> -- !query 162 -INSERT INTO num_exp_add VALUES (3,8,'74885.31') +INSERT INTO num_exp_add VALUES (3,8,74885.31) -- !query 162 schema struct<> -- !query 162 output @@ -1307,7 +1307,7 @@ struct<> -- !query 163 -INSERT INTO num_exp_sub VALUES (3,8,'-74876.69') +INSERT INTO num_exp_sub VALUES (3,8,-74876.69) -- !query 163 schema struct<> -- !query 163 output @@ -1315,7 +1315,7 @@ struct<> -- !query 164 -INSERT INTO num_exp_mul VALUES (3,8,'322737.11') +INSERT INTO num_exp_mul VALUES (3,8,322737.11) -- !query 164 schema struct<> -- !query 164 output @@ -1323,7 +1323,7 @@ struct<> -- !query 165 -INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553') +INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553) -- !query 165 schema struct<> -- !query 165 output @@ -1331,7 +1331,7 @@ struct<> -- !query 166 -INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420') +INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420) -- !query 166 schema struct<> -- !query 166 output @@ -1339,7 +1339,7 @@ struct<> -- !query 167 -INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420') +INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420) -- !query 167 schema struct<> -- !query 167 output @@ -1347,7 +1347,7 @@ struct<> -- !query 168 -INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020') +INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020) -- !query 168 schema struct<> -- !query 168 output @@ -1355,7 +1355,7 @@ struct<> -- !query 169 -INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854') +INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854) -- !query 169 schema struct<> -- !query 169 output @@ -1363,7 +1363,7 @@ struct<> -- !query 170 -INSERT INTO num_exp_add VALUES (4,0,'7799461.4119') +INSERT INTO num_exp_add VALUES (4,0,7799461.4119) -- !query 170 schema struct<> -- !query 170 output @@ -1371,7 +1371,7 @@ struct<> -- !query 171 -INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119') +INSERT INTO num_exp_sub VALUES (4,0,7799461.4119) -- !query 171 schema struct<> -- !query 171 output @@ -1379,7 +1379,7 @@ struct<> -- !query 172 -INSERT INTO num_exp_mul VALUES (4,0,'0') +INSERT INTO num_exp_mul VALUES (4,0,0) -- !query 172 schema struct<> -- !query 172 output @@ -1387,7 +1387,7 @@ struct<> -- !query 173 -INSERT INTO num_exp_div VALUES (4,0,'NaN') +INSERT INTO num_exp_div VALUES (4,0,double('NaN')) -- !query 173 schema struct<> -- !query 173 output @@ -1395,7 +1395,7 @@ struct<> -- !query 174 -INSERT INTO num_exp_add VALUES (4,1,'7799461.4119') +INSERT INTO num_exp_add VALUES (4,1,7799461.4119) -- !query 174 schema struct<> -- !query 174 output @@ -1403,7 +1403,7 @@ struct<> -- !query 175 -INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119') +INSERT INTO num_exp_sub VALUES (4,1,7799461.4119) -- !query 175 schema struct<> -- !query 175 output @@ -1411,7 +1411,7 @@ struct<> -- !query 176 -INSERT INTO num_exp_mul VALUES (4,1,'0') +INSERT INTO num_exp_mul VALUES (4,1,0) -- !query 176 schema struct<> -- !query 176 output @@ -1419,7 +1419,7 @@ struct<> -- !query 177 -INSERT INTO num_exp_div VALUES (4,1,'NaN') +INSERT INTO num_exp_div VALUES (4,1,double('NaN')) -- !query 177 schema struct<> -- !query 177 output @@ -1427,7 +1427,7 @@ struct<> -- !query 178 -INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047') +INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047) -- !query 178 schema struct<> -- !query 178 output @@ -1435,7 +1435,7 @@ struct<> -- !query 179 -INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047') +INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047) -- !query 179 schema struct<> -- !query 179 output @@ -1443,7 +1443,7 @@ struct<> -- !query 180 -INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593') +INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593) -- !query 180 schema struct<> -- !query 180 output @@ -1451,7 +1451,7 @@ struct<> -- !query 181 -INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385') +INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385) -- !query 181 schema struct<> -- !query 181 output @@ -1459,7 +1459,7 @@ struct<> -- !query 182 -INSERT INTO num_exp_add VALUES (4,3,'7799465.7219') +INSERT INTO num_exp_add VALUES (4,3,7799465.7219) -- !query 182 schema struct<> -- !query 182 output @@ -1467,7 +1467,7 @@ struct<> -- !query 183 -INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019') +INSERT INTO num_exp_sub VALUES (4,3,7799457.1019) -- !query 183 schema struct<> -- !query 183 output @@ -1475,7 +1475,7 @@ struct<> -- !query 184 -INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289') +INSERT INTO num_exp_mul VALUES (4,3,33615678.685289) -- !query 184 schema struct<> -- !query 184 output @@ -1483,7 +1483,7 @@ struct<> -- !query 185 -INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883') +INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883) -- !query 185 schema struct<> -- !query 185 output @@ -1491,7 +1491,7 @@ struct<> -- !query 186 -INSERT INTO num_exp_add VALUES (4,4,'15598922.8238') +INSERT INTO num_exp_add VALUES (4,4,15598922.8238) -- !query 186 schema struct<> -- !query 186 output @@ -1499,7 +1499,7 @@ struct<> -- !query 187 -INSERT INTO num_exp_sub VALUES (4,4,'0') +INSERT INTO num_exp_sub VALUES (4,4,0) -- !query 187 schema struct<> -- !query 187 output @@ -1507,7 +1507,7 @@ struct<> -- !query 188 -INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161') +INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161) -- !query 188 schema struct<> -- !query 188 output @@ -1515,7 +1515,7 @@ struct<> -- !query 189 -INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000) -- !query 189 schema struct<> -- !query 189 output @@ -1523,7 +1523,7 @@ struct<> -- !query 190 -INSERT INTO num_exp_add VALUES (4,5,'7815858.450391') +INSERT INTO num_exp_add VALUES (4,5,7815858.450391) -- !query 190 schema struct<> -- !query 190 output @@ -1531,7 +1531,7 @@ struct<> -- !query 191 -INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409') +INSERT INTO num_exp_sub VALUES (4,5,7783064.373409) -- !query 191 schema struct<> -- !query 191 output @@ -1539,7 +1539,7 @@ struct<> -- !query 192 -INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429') +INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429) -- !query 192 schema struct<> -- !query 192 output @@ -1547,7 +1547,7 @@ struct<> -- !query 193 -INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061') +INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061) -- !query 193 schema struct<> -- !query 193 output @@ -1555,7 +1555,7 @@ struct<> -- !query 194 -INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026') +INSERT INTO num_exp_add VALUES (4,6,7893362.98953026) -- !query 194 schema struct<> -- !query 194 output @@ -1563,7 +1563,7 @@ struct<> -- !query 195 -INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974') +INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974) -- !query 195 schema struct<> -- !query 195 output @@ -1571,7 +1571,7 @@ struct<> -- !query 196 -INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094') +INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094) -- !query 196 schema struct<> -- !query 196 output @@ -1579,7 +1579,7 @@ struct<> -- !query 197 -INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606') +INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606) -- !query 197 schema struct<> -- !query 197 output @@ -1587,7 +1587,7 @@ struct<> -- !query 198 -INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881') +INSERT INTO num_exp_add VALUES (4,7,-75229023.5881) -- !query 198 schema struct<> -- !query 198 output @@ -1595,7 +1595,7 @@ struct<> -- !query 199 -INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119') +INSERT INTO num_exp_sub VALUES (4,7,90827946.4119) -- !query 199 schema struct<> -- !query 199 output @@ -1603,7 +1603,7 @@ struct<> -- !query 200 -INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715') +INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715) -- !query 200 schema struct<> -- !query 200 output @@ -1611,7 +1611,7 @@ struct<> -- !query 201 -INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637') +INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637) -- !query 201 schema struct<> -- !query 201 output @@ -1619,7 +1619,7 @@ struct<> -- !query 202 -INSERT INTO num_exp_add VALUES (4,8,'7874342.4119') +INSERT INTO num_exp_add VALUES (4,8,7874342.4119) -- !query 202 schema struct<> -- !query 202 output @@ -1627,7 +1627,7 @@ struct<> -- !query 203 -INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119') +INSERT INTO num_exp_sub VALUES (4,8,7724580.4119) -- !query 203 schema struct<> -- !query 203 output @@ -1635,7 +1635,7 @@ struct<> -- !query 204 -INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839') +INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839) -- !query 204 schema struct<> -- !query 204 output @@ -1643,7 +1643,7 @@ struct<> -- !query 205 -INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143') +INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143) -- !query 205 schema struct<> -- !query 205 output @@ -1651,7 +1651,7 @@ struct<> -- !query 206 -INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420') +INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420) -- !query 206 schema struct<> -- !query 206 output @@ -1659,7 +1659,7 @@ struct<> -- !query 207 -INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420') +INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420) -- !query 207 schema struct<> -- !query 207 output @@ -1667,7 +1667,7 @@ struct<> -- !query 208 -INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980') +INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980) -- !query 208 schema struct<> -- !query 208 output @@ -1675,7 +1675,7 @@ struct<> -- !query 209 -INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409') +INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409) -- !query 209 schema struct<> -- !query 209 output @@ -1683,7 +1683,7 @@ struct<> -- !query 210 -INSERT INTO num_exp_add VALUES (5,0,'16397.038491') +INSERT INTO num_exp_add VALUES (5,0,16397.038491) -- !query 210 schema struct<> -- !query 210 output @@ -1691,7 +1691,7 @@ struct<> -- !query 211 -INSERT INTO num_exp_sub VALUES (5,0,'16397.038491') +INSERT INTO num_exp_sub VALUES (5,0,16397.038491) -- !query 211 schema struct<> -- !query 211 output @@ -1699,7 +1699,7 @@ struct<> -- !query 212 -INSERT INTO num_exp_mul VALUES (5,0,'0') +INSERT INTO num_exp_mul VALUES (5,0,0) -- !query 212 schema struct<> -- !query 212 output @@ -1707,7 +1707,7 @@ struct<> -- !query 213 -INSERT INTO num_exp_div VALUES (5,0,'NaN') +INSERT INTO num_exp_div VALUES (5,0,double('NaN')) -- !query 213 schema struct<> -- !query 213 output @@ -1715,7 +1715,7 @@ struct<> -- !query 214 -INSERT INTO num_exp_add VALUES (5,1,'16397.038491') +INSERT INTO num_exp_add VALUES (5,1,16397.038491) -- !query 214 schema struct<> -- !query 214 output @@ -1723,7 +1723,7 @@ struct<> -- !query 215 -INSERT INTO num_exp_sub VALUES (5,1,'16397.038491') +INSERT INTO num_exp_sub VALUES (5,1,16397.038491) -- !query 215 schema struct<> -- !query 215 output @@ -1731,7 +1731,7 @@ struct<> -- !query 216 -INSERT INTO num_exp_mul VALUES (5,1,'0') +INSERT INTO num_exp_mul VALUES (5,1,0) -- !query 216 schema struct<> -- !query 216 output @@ -1739,7 +1739,7 @@ struct<> -- !query 217 -INSERT INTO num_exp_div VALUES (5,1,'NaN') +INSERT INTO num_exp_div VALUES (5,1,double('NaN')) -- !query 217 schema struct<> -- !query 217 output @@ -1747,7 +1747,7 @@ struct<> -- !query 218 -INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047') +INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047) -- !query 218 schema struct<> -- !query 218 output @@ -1755,7 +1755,7 @@ struct<> -- !query 219 -INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047') +INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047) -- !query 219 schema struct<> -- !query 219 output @@ -1763,7 +1763,7 @@ struct<> -- !query 220 -INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077') +INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077) -- !query 220 schema struct<> -- !query 220 output @@ -1771,7 +1771,7 @@ struct<> -- !query 221 -INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446') +INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446) -- !query 221 schema struct<> -- !query 221 output @@ -1779,7 +1779,7 @@ struct<> -- !query 222 -INSERT INTO num_exp_add VALUES (5,3,'16401.348491') +INSERT INTO num_exp_add VALUES (5,3,16401.348491) -- !query 222 schema struct<> -- !query 222 output @@ -1787,7 +1787,7 @@ struct<> -- !query 223 -INSERT INTO num_exp_sub VALUES (5,3,'16392.728491') +INSERT INTO num_exp_sub VALUES (5,3,16392.728491) -- !query 223 schema struct<> -- !query 223 output @@ -1795,7 +1795,7 @@ struct<> -- !query 224 -INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621') +INSERT INTO num_exp_mul VALUES (5,3,70671.23589621) -- !query 224 schema struct<> -- !query 224 output @@ -1803,7 +1803,7 @@ struct<> -- !query 225 -INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584') +INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584) -- !query 225 schema struct<> -- !query 225 output @@ -1811,7 +1811,7 @@ struct<> -- !query 226 -INSERT INTO num_exp_add VALUES (5,4,'7815858.450391') +INSERT INTO num_exp_add VALUES (5,4,7815858.450391) -- !query 226 schema struct<> -- !query 226 output @@ -1819,7 +1819,7 @@ struct<> -- !query 227 -INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409') +INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409) -- !query 227 schema struct<> -- !query 227 output @@ -1827,7 +1827,7 @@ struct<> -- !query 228 -INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429') +INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429) -- !query 228 schema struct<> -- !query 228 output @@ -1835,7 +1835,7 @@ struct<> -- !query 229 -INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192') +INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192) -- !query 229 schema struct<> -- !query 229 output @@ -1843,7 +1843,7 @@ struct<> -- !query 230 -INSERT INTO num_exp_add VALUES (5,5,'32794.076982') +INSERT INTO num_exp_add VALUES (5,5,32794.076982) -- !query 230 schema struct<> -- !query 230 output @@ -1851,7 +1851,7 @@ struct<> -- !query 231 -INSERT INTO num_exp_sub VALUES (5,5,'0') +INSERT INTO num_exp_sub VALUES (5,5,0) -- !query 231 schema struct<> -- !query 231 output @@ -1859,7 +1859,7 @@ struct<> -- !query 232 -INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081') +INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081) -- !query 232 schema struct<> -- !query 232 output @@ -1867,7 +1867,7 @@ struct<> -- !query 233 -INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000) -- !query 233 schema struct<> -- !query 233 output @@ -1875,7 +1875,7 @@ struct<> -- !query 234 -INSERT INTO num_exp_add VALUES (5,6,'110298.61612126') +INSERT INTO num_exp_add VALUES (5,6,110298.61612126) -- !query 234 schema struct<> -- !query 234 output @@ -1883,7 +1883,7 @@ struct<> -- !query 235 -INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926') +INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926) -- !query 235 schema struct<> -- !query 235 output @@ -1891,7 +1891,7 @@ struct<> -- !query 236 -INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766') +INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766) -- !query 236 schema struct<> -- !query 236 output @@ -1899,7 +1899,7 @@ struct<> -- !query 237 -INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689') +INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689) -- !query 237 schema struct<> -- !query 237 output @@ -1907,7 +1907,7 @@ struct<> -- !query 238 -INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509') +INSERT INTO num_exp_add VALUES (5,7,-83012087.961509) -- !query 238 schema struct<> -- !query 238 output @@ -1915,7 +1915,7 @@ struct<> -- !query 239 -INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491') +INSERT INTO num_exp_sub VALUES (5,7,83044882.038491) -- !query 239 schema struct<> -- !query 239 output @@ -1923,7 +1923,7 @@ struct<> -- !query 240 -INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135') +INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135) -- !query 240 schema struct<> -- !query 240 output @@ -1931,7 +1931,7 @@ struct<> -- !query 241 -INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710') +INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710) -- !query 241 schema struct<> -- !query 241 output @@ -1939,7 +1939,7 @@ struct<> -- !query 242 -INSERT INTO num_exp_add VALUES (5,8,'91278.038491') +INSERT INTO num_exp_add VALUES (5,8,91278.038491) -- !query 242 schema struct<> -- !query 242 output @@ -1947,7 +1947,7 @@ struct<> -- !query 243 -INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509') +INSERT INTO num_exp_sub VALUES (5,8,-58483.961509) -- !query 243 schema struct<> -- !query 243 output @@ -1955,7 +1955,7 @@ struct<> -- !query 244 -INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571') +INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571) -- !query 244 schema struct<> -- !query 244 output @@ -1963,7 +1963,7 @@ struct<> -- !query 245 -INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228') +INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228) -- !query 245 schema struct<> -- !query 245 output @@ -1971,7 +1971,7 @@ struct<> -- !query 246 -INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420') +INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420) -- !query 246 schema struct<> -- !query 246 output @@ -1979,7 +1979,7 @@ struct<> -- !query 247 -INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420') +INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420) -- !query 247 schema struct<> -- !query 247 output @@ -1987,7 +1987,7 @@ struct<> -- !query 248 -INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220') +INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220) -- !query 248 schema struct<> -- !query 248 output @@ -1995,7 +1995,7 @@ struct<> -- !query 249 -INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427') +INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427) -- !query 249 schema struct<> -- !query 249 output @@ -2003,7 +2003,7 @@ struct<> -- !query 250 -INSERT INTO num_exp_add VALUES (6,0,'93901.57763026') +INSERT INTO num_exp_add VALUES (6,0,93901.57763026) -- !query 250 schema struct<> -- !query 250 output @@ -2011,7 +2011,7 @@ struct<> -- !query 251 -INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026') +INSERT INTO num_exp_sub VALUES (6,0,93901.57763026) -- !query 251 schema struct<> -- !query 251 output @@ -2019,7 +2019,7 @@ struct<> -- !query 252 -INSERT INTO num_exp_mul VALUES (6,0,'0') +INSERT INTO num_exp_mul VALUES (6,0,0) -- !query 252 schema struct<> -- !query 252 output @@ -2027,7 +2027,7 @@ struct<> -- !query 253 -INSERT INTO num_exp_div VALUES (6,0,'NaN') +INSERT INTO num_exp_div VALUES (6,0,double('NaN')) -- !query 253 schema struct<> -- !query 253 output @@ -2035,7 +2035,7 @@ struct<> -- !query 254 -INSERT INTO num_exp_add VALUES (6,1,'93901.57763026') +INSERT INTO num_exp_add VALUES (6,1,93901.57763026) -- !query 254 schema struct<> -- !query 254 output @@ -2043,7 +2043,7 @@ struct<> -- !query 255 -INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026') +INSERT INTO num_exp_sub VALUES (6,1,93901.57763026) -- !query 255 schema struct<> -- !query 255 output @@ -2051,7 +2051,7 @@ struct<> -- !query 256 -INSERT INTO num_exp_mul VALUES (6,1,'0') +INSERT INTO num_exp_mul VALUES (6,1,0) -- !query 256 schema struct<> -- !query 256 output @@ -2059,7 +2059,7 @@ struct<> -- !query 257 -INSERT INTO num_exp_div VALUES (6,1,'NaN') +INSERT INTO num_exp_div VALUES (6,1,double('NaN')) -- !query 257 schema struct<> -- !query 257 output @@ -2067,7 +2067,7 @@ struct<> -- !query 258 -INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787') +INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787) -- !query 258 schema struct<> -- !query 258 output @@ -2075,7 +2075,7 @@ struct<> -- !query 259 -INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307') +INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307) -- !query 259 schema struct<> -- !query 259 output @@ -2083,7 +2083,7 @@ struct<> -- !query 260 -INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222') +INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222) -- !query 260 schema struct<> -- !query 260 output @@ -2091,7 +2091,7 @@ struct<> -- !query 261 -INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823') +INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823) -- !query 261 schema struct<> -- !query 261 output @@ -2099,7 +2099,7 @@ struct<> -- !query 262 -INSERT INTO num_exp_add VALUES (6,3,'93905.88763026') +INSERT INTO num_exp_add VALUES (6,3,93905.88763026) -- !query 262 schema struct<> -- !query 262 output @@ -2107,7 +2107,7 @@ struct<> -- !query 263 -INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026') +INSERT INTO num_exp_sub VALUES (6,3,93897.26763026) -- !query 263 schema struct<> -- !query 263 output @@ -2115,7 +2115,7 @@ struct<> -- !query 264 -INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206') +INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206) -- !query 264 schema struct<> -- !query 264 output @@ -2123,7 +2123,7 @@ struct<> -- !query 265 -INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907') +INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907) -- !query 265 schema struct<> -- !query 265 output @@ -2131,7 +2131,7 @@ struct<> -- !query 266 -INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026') +INSERT INTO num_exp_add VALUES (6,4,7893362.98953026) -- !query 266 schema struct<> -- !query 266 output @@ -2139,7 +2139,7 @@ struct<> -- !query 267 -INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974') +INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974) -- !query 267 schema struct<> -- !query 267 output @@ -2147,7 +2147,7 @@ struct<> -- !query 268 -INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094') +INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094) -- !query 268 schema struct<> -- !query 268 output @@ -2155,7 +2155,7 @@ struct<> -- !query 269 -INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469') +INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469) -- !query 269 schema struct<> -- !query 269 output @@ -2163,7 +2163,7 @@ struct<> -- !query 270 -INSERT INTO num_exp_add VALUES (6,5,'110298.61612126') +INSERT INTO num_exp_add VALUES (6,5,110298.61612126) -- !query 270 schema struct<> -- !query 270 output @@ -2171,7 +2171,7 @@ struct<> -- !query 271 -INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926') +INSERT INTO num_exp_sub VALUES (6,5,77504.53913926) -- !query 271 schema struct<> -- !query 271 output @@ -2179,7 +2179,7 @@ struct<> -- !query 272 -INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766') +INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766) -- !query 272 schema struct<> -- !query 272 output @@ -2187,7 +2187,7 @@ struct<> -- !query 273 -INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679') +INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679) -- !query 273 schema struct<> -- !query 273 output @@ -2195,7 +2195,7 @@ struct<> -- !query 274 -INSERT INTO num_exp_add VALUES (6,6,'187803.15526052') +INSERT INTO num_exp_add VALUES (6,6,187803.15526052) -- !query 274 schema struct<> -- !query 274 output @@ -2203,7 +2203,7 @@ struct<> -- !query 275 -INSERT INTO num_exp_sub VALUES (6,6,'0') +INSERT INTO num_exp_sub VALUES (6,6,0) -- !query 275 schema struct<> -- !query 275 output @@ -2211,7 +2211,7 @@ struct<> -- !query 276 -INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676') +INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676) -- !query 276 schema struct<> -- !query 276 output @@ -2219,7 +2219,7 @@ struct<> -- !query 277 -INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000) -- !query 277 schema struct<> -- !query 277 output @@ -2227,7 +2227,7 @@ struct<> -- !query 278 -INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974') +INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974) -- !query 278 schema struct<> -- !query 278 output @@ -2235,7 +2235,7 @@ struct<> -- !query 279 -INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026') +INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026) -- !query 279 schema struct<> -- !query 279 output @@ -2243,7 +2243,7 @@ struct<> -- !query 280 -INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610') +INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610) -- !query 280 schema struct<> -- !query 280 output @@ -2251,7 +2251,7 @@ struct<> -- !query 281 -INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980') +INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980) -- !query 281 schema struct<> -- !query 281 output @@ -2259,7 +2259,7 @@ struct<> -- !query 282 -INSERT INTO num_exp_add VALUES (6,8,'168782.57763026') +INSERT INTO num_exp_add VALUES (6,8,168782.57763026) -- !query 282 schema struct<> -- !query 282 output @@ -2267,7 +2267,7 @@ struct<> -- !query 283 -INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026') +INSERT INTO num_exp_sub VALUES (6,8,19020.57763026) -- !query 283 schema struct<> -- !query 283 output @@ -2275,7 +2275,7 @@ struct<> -- !query 284 -INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906') +INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906) -- !query 284 schema struct<> -- !query 284 output @@ -2283,7 +2283,7 @@ struct<> -- !query 285 -INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184') +INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184) -- !query 285 schema struct<> -- !query 285 output @@ -2291,7 +2291,7 @@ struct<> -- !query 286 -INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160') +INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160) -- !query 286 schema struct<> -- !query 286 output @@ -2299,7 +2299,7 @@ struct<> -- !query 287 -INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680') +INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680) -- !query 287 schema struct<> -- !query 287 output @@ -2307,7 +2307,7 @@ struct<> -- !query 288 -INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920') +INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920) -- !query 288 schema struct<> -- !query 288 output @@ -2315,7 +2315,7 @@ struct<> -- !query 289 -INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789') +INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789) -- !query 289 schema struct<> -- !query 289 output @@ -2323,7 +2323,7 @@ struct<> -- !query 290 -INSERT INTO num_exp_add VALUES (7,0,'-83028485') +INSERT INTO num_exp_add VALUES (7,0,-83028485) -- !query 290 schema struct<> -- !query 290 output @@ -2331,7 +2331,7 @@ struct<> -- !query 291 -INSERT INTO num_exp_sub VALUES (7,0,'-83028485') +INSERT INTO num_exp_sub VALUES (7,0,-83028485) -- !query 291 schema struct<> -- !query 291 output @@ -2339,7 +2339,7 @@ struct<> -- !query 292 -INSERT INTO num_exp_mul VALUES (7,0,'0') +INSERT INTO num_exp_mul VALUES (7,0,0) -- !query 292 schema struct<> -- !query 292 output @@ -2347,7 +2347,7 @@ struct<> -- !query 293 -INSERT INTO num_exp_div VALUES (7,0,'NaN') +INSERT INTO num_exp_div VALUES (7,0,double('NaN')) -- !query 293 schema struct<> -- !query 293 output @@ -2355,7 +2355,7 @@ struct<> -- !query 294 -INSERT INTO num_exp_add VALUES (7,1,'-83028485') +INSERT INTO num_exp_add VALUES (7,1,-83028485) -- !query 294 schema struct<> -- !query 294 output @@ -2363,7 +2363,7 @@ struct<> -- !query 295 -INSERT INTO num_exp_sub VALUES (7,1,'-83028485') +INSERT INTO num_exp_sub VALUES (7,1,-83028485) -- !query 295 schema struct<> -- !query 295 output @@ -2371,7 +2371,7 @@ struct<> -- !query 296 -INSERT INTO num_exp_mul VALUES (7,1,'0') +INSERT INTO num_exp_mul VALUES (7,1,0) -- !query 296 schema struct<> -- !query 296 output @@ -2379,7 +2379,7 @@ struct<> -- !query 297 -INSERT INTO num_exp_div VALUES (7,1,'NaN') +INSERT INTO num_exp_div VALUES (7,1,double('NaN')) -- !query 297 schema struct<> -- !query 297 output @@ -2387,7 +2387,7 @@ struct<> -- !query 298 -INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047') +INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047) -- !query 298 schema struct<> -- !query 298 output @@ -2395,7 +2395,7 @@ struct<> -- !query 299 -INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953') +INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953) -- !query 299 schema struct<> -- !query 299 output @@ -2403,7 +2403,7 @@ struct<> -- !query 300 -INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795') +INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795) -- !query 300 schema struct<> -- !query 300 output @@ -2411,7 +2411,7 @@ struct<> -- !query 301 -INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700') +INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700) -- !query 301 schema struct<> -- !query 301 output @@ -2419,7 +2419,7 @@ struct<> -- !query 302 -INSERT INTO num_exp_add VALUES (7,3,'-83028480.69') +INSERT INTO num_exp_add VALUES (7,3,-83028480.69) -- !query 302 schema struct<> -- !query 302 output @@ -2427,7 +2427,7 @@ struct<> -- !query 303 -INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31') +INSERT INTO num_exp_sub VALUES (7,3,-83028489.31) -- !query 303 schema struct<> -- !query 303 output @@ -2435,7 +2435,7 @@ struct<> -- !query 304 -INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35') +INSERT INTO num_exp_mul VALUES (7,3,-357852770.35) -- !query 304 schema struct<> -- !query 304 output @@ -2443,7 +2443,7 @@ struct<> -- !query 305 -INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974') +INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974) -- !query 305 schema struct<> -- !query 305 output @@ -2451,7 +2451,7 @@ struct<> -- !query 306 -INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881') +INSERT INTO num_exp_add VALUES (7,4,-75229023.5881) -- !query 306 schema struct<> -- !query 306 output @@ -2459,7 +2459,7 @@ struct<> -- !query 307 -INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119') +INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119) -- !query 307 schema struct<> -- !query 307 output @@ -2467,7 +2467,7 @@ struct<> -- !query 308 -INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715') +INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715) -- !query 308 schema struct<> -- !query 308 output @@ -2475,7 +2475,7 @@ struct<> -- !query 309 -INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686') +INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686) -- !query 309 schema struct<> -- !query 309 output @@ -2483,7 +2483,7 @@ struct<> -- !query 310 -INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509') +INSERT INTO num_exp_add VALUES (7,5,-83012087.961509) -- !query 310 schema struct<> -- !query 310 output @@ -2491,7 +2491,7 @@ struct<> -- !query 311 -INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491') +INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491) -- !query 311 schema struct<> -- !query 311 output @@ -2499,7 +2499,7 @@ struct<> -- !query 312 -INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135') +INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135) -- !query 312 schema struct<> -- !query 312 output @@ -2507,7 +2507,7 @@ struct<> -- !query 313 -INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574') +INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574) -- !query 313 schema struct<> -- !query 313 output @@ -2515,7 +2515,7 @@ struct<> -- !query 314 -INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974') +INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974) -- !query 314 schema struct<> -- !query 314 output @@ -2523,7 +2523,7 @@ struct<> -- !query 315 -INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026') +INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026) -- !query 315 schema struct<> -- !query 315 output @@ -2531,7 +2531,7 @@ struct<> -- !query 316 -INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610') +INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610) -- !query 316 schema struct<> -- !query 316 output @@ -2539,7 +2539,7 @@ struct<> -- !query 317 -INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294') +INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294) -- !query 317 schema struct<> -- !query 317 output @@ -2547,7 +2547,7 @@ struct<> -- !query 318 -INSERT INTO num_exp_add VALUES (7,7,'-166056970') +INSERT INTO num_exp_add VALUES (7,7,-166056970) -- !query 318 schema struct<> -- !query 318 output @@ -2555,7 +2555,7 @@ struct<> -- !query 319 -INSERT INTO num_exp_sub VALUES (7,7,'0') +INSERT INTO num_exp_sub VALUES (7,7,0) -- !query 319 schema struct<> -- !query 319 output @@ -2563,7 +2563,7 @@ struct<> -- !query 320 -INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225') +INSERT INTO num_exp_mul VALUES (7,7,6893729321395225) -- !query 320 schema struct<> -- !query 320 output @@ -2571,7 +2571,7 @@ struct<> -- !query 321 -INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000) -- !query 321 schema struct<> -- !query 321 output @@ -2579,7 +2579,7 @@ struct<> -- !query 322 -INSERT INTO num_exp_add VALUES (7,8,'-82953604') +INSERT INTO num_exp_add VALUES (7,8,-82953604) -- !query 322 schema struct<> -- !query 322 output @@ -2587,7 +2587,7 @@ struct<> -- !query 323 -INSERT INTO num_exp_sub VALUES (7,8,'-83103366') +INSERT INTO num_exp_sub VALUES (7,8,-83103366) -- !query 323 schema struct<> -- !query 323 output @@ -2595,7 +2595,7 @@ struct<> -- !query 324 -INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285') +INSERT INTO num_exp_mul VALUES (7,8,-6217255985285) -- !query 324 schema struct<> -- !query 324 output @@ -2603,7 +2603,7 @@ struct<> -- !query 325 -INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118') +INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118) -- !query 325 schema struct<> -- !query 325 output @@ -2611,7 +2611,7 @@ struct<> -- !query 326 -INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420') +INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420) -- !query 326 schema struct<> -- !query 326 output @@ -2619,7 +2619,7 @@ struct<> -- !query 327 -INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580') +INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580) -- !query 327 schema struct<> -- !query 327 output @@ -2627,7 +2627,7 @@ struct<> -- !query 328 -INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700') +INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700) -- !query 328 schema struct<> -- !query 328 output @@ -2635,7 +2635,7 @@ struct<> -- !query 329 -INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382') +INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382) -- !query 329 schema struct<> -- !query 329 output @@ -2643,7 +2643,7 @@ struct<> -- !query 330 -INSERT INTO num_exp_add VALUES (8,0,'74881') +INSERT INTO num_exp_add VALUES (8,0,74881) -- !query 330 schema struct<> -- !query 330 output @@ -2651,7 +2651,7 @@ struct<> -- !query 331 -INSERT INTO num_exp_sub VALUES (8,0,'74881') +INSERT INTO num_exp_sub VALUES (8,0,74881) -- !query 331 schema struct<> -- !query 331 output @@ -2659,7 +2659,7 @@ struct<> -- !query 332 -INSERT INTO num_exp_mul VALUES (8,0,'0') +INSERT INTO num_exp_mul VALUES (8,0,0) -- !query 332 schema struct<> -- !query 332 output @@ -2667,7 +2667,7 @@ struct<> -- !query 333 -INSERT INTO num_exp_div VALUES (8,0,'NaN') +INSERT INTO num_exp_div VALUES (8,0,double('NaN')) -- !query 333 schema struct<> -- !query 333 output @@ -2675,7 +2675,7 @@ struct<> -- !query 334 -INSERT INTO num_exp_add VALUES (8,1,'74881') +INSERT INTO num_exp_add VALUES (8,1,74881) -- !query 334 schema struct<> -- !query 334 output @@ -2683,7 +2683,7 @@ struct<> -- !query 335 -INSERT INTO num_exp_sub VALUES (8,1,'74881') +INSERT INTO num_exp_sub VALUES (8,1,74881) -- !query 335 schema struct<> -- !query 335 output @@ -2691,7 +2691,7 @@ struct<> -- !query 336 -INSERT INTO num_exp_mul VALUES (8,1,'0') +INSERT INTO num_exp_mul VALUES (8,1,0) -- !query 336 schema struct<> -- !query 336 output @@ -2699,7 +2699,7 @@ struct<> -- !query 337 -INSERT INTO num_exp_div VALUES (8,1,'NaN') +INSERT INTO num_exp_div VALUES (8,1,double('NaN')) -- !query 337 schema struct<> -- !query 337 output @@ -2707,7 +2707,7 @@ struct<> -- !query 338 -INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047') +INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047) -- !query 338 schema struct<> -- !query 338 output @@ -2715,7 +2715,7 @@ struct<> -- !query 339 -INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047') +INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047) -- !query 339 schema struct<> -- !query 339 output @@ -2723,7 +2723,7 @@ struct<> -- !query 340 -INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407') +INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407) -- !query 340 schema struct<> -- !query 340 output @@ -2731,7 +2731,7 @@ struct<> -- !query 341 -INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615') +INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615) -- !query 341 schema struct<> -- !query 341 output @@ -2739,7 +2739,7 @@ struct<> -- !query 342 -INSERT INTO num_exp_add VALUES (8,3,'74885.31') +INSERT INTO num_exp_add VALUES (8,3,74885.31) -- !query 342 schema struct<> -- !query 342 output @@ -2747,7 +2747,7 @@ struct<> -- !query 343 -INSERT INTO num_exp_sub VALUES (8,3,'74876.69') +INSERT INTO num_exp_sub VALUES (8,3,74876.69) -- !query 343 schema struct<> -- !query 343 output @@ -2755,7 +2755,7 @@ struct<> -- !query 344 -INSERT INTO num_exp_mul VALUES (8,3,'322737.11') +INSERT INTO num_exp_mul VALUES (8,3,322737.11) -- !query 344 schema struct<> -- !query 344 output @@ -2763,7 +2763,7 @@ struct<> -- !query 345 -INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410') +INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410) -- !query 345 schema struct<> -- !query 345 output @@ -2771,7 +2771,7 @@ struct<> -- !query 346 -INSERT INTO num_exp_add VALUES (8,4,'7874342.4119') +INSERT INTO num_exp_add VALUES (8,4,7874342.4119) -- !query 346 schema struct<> -- !query 346 output @@ -2779,7 +2779,7 @@ struct<> -- !query 347 -INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119') +INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119) -- !query 347 schema struct<> -- !query 347 output @@ -2787,7 +2787,7 @@ struct<> -- !query 348 -INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839') +INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839) -- !query 348 schema struct<> -- !query 348 output @@ -2795,7 +2795,7 @@ struct<> -- !query 349 -INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956') +INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956) -- !query 349 schema struct<> -- !query 349 output @@ -2803,7 +2803,7 @@ struct<> -- !query 350 -INSERT INTO num_exp_add VALUES (8,5,'91278.038491') +INSERT INTO num_exp_add VALUES (8,5,91278.038491) -- !query 350 schema struct<> -- !query 350 output @@ -2811,7 +2811,7 @@ struct<> -- !query 351 -INSERT INTO num_exp_sub VALUES (8,5,'58483.961509') +INSERT INTO num_exp_sub VALUES (8,5,58483.961509) -- !query 351 schema struct<> -- !query 351 output @@ -2819,7 +2819,7 @@ struct<> -- !query 352 -INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571') +INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571) -- !query 352 schema struct<> -- !query 352 output @@ -2827,7 +2827,7 @@ struct<> -- !query 353 -INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456') +INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456) -- !query 353 schema struct<> -- !query 353 output @@ -2835,7 +2835,7 @@ struct<> -- !query 354 -INSERT INTO num_exp_add VALUES (8,6,'168782.57763026') +INSERT INTO num_exp_add VALUES (8,6,168782.57763026) -- !query 354 schema struct<> -- !query 354 output @@ -2843,7 +2843,7 @@ struct<> -- !query 355 -INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026') +INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026) -- !query 355 schema struct<> -- !query 355 output @@ -2851,7 +2851,7 @@ struct<> -- !query 356 -INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906') +INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906) -- !query 356 schema struct<> -- !query 356 output @@ -2859,7 +2859,7 @@ struct<> -- !query 357 -INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424') +INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424) -- !query 357 schema struct<> -- !query 357 output @@ -2867,7 +2867,7 @@ struct<> -- !query 358 -INSERT INTO num_exp_add VALUES (8,7,'-82953604') +INSERT INTO num_exp_add VALUES (8,7,-82953604) -- !query 358 schema struct<> -- !query 358 output @@ -2875,7 +2875,7 @@ struct<> -- !query 359 -INSERT INTO num_exp_sub VALUES (8,7,'83103366') +INSERT INTO num_exp_sub VALUES (8,7,83103366) -- !query 359 schema struct<> -- !query 359 output @@ -2883,7 +2883,7 @@ struct<> -- !query 360 -INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285') +INSERT INTO num_exp_mul VALUES (8,7,-6217255985285) -- !query 360 schema struct<> -- !query 360 output @@ -2891,7 +2891,7 @@ struct<> -- !query 361 -INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172') +INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172) -- !query 361 schema struct<> -- !query 361 output @@ -2899,7 +2899,7 @@ struct<> -- !query 362 -INSERT INTO num_exp_add VALUES (8,8,'149762') +INSERT INTO num_exp_add VALUES (8,8,149762) -- !query 362 schema struct<> -- !query 362 output @@ -2907,7 +2907,7 @@ struct<> -- !query 363 -INSERT INTO num_exp_sub VALUES (8,8,'0') +INSERT INTO num_exp_sub VALUES (8,8,0) -- !query 363 schema struct<> -- !query 363 output @@ -2915,7 +2915,7 @@ struct<> -- !query 364 -INSERT INTO num_exp_mul VALUES (8,8,'5607164161') +INSERT INTO num_exp_mul VALUES (8,8,5607164161) -- !query 364 schema struct<> -- !query 364 output @@ -2923,7 +2923,7 @@ struct<> -- !query 365 -INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000) -- !query 365 schema struct<> -- !query 365 output @@ -2931,7 +2931,7 @@ struct<> -- !query 366 -INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420') +INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420) -- !query 366 schema struct<> -- !query 366 output @@ -2939,7 +2939,7 @@ struct<> -- !query 367 -INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420') +INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420) -- !query 367 schema struct<> -- !query 367 output @@ -2947,7 +2947,7 @@ struct<> -- !query 368 -INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020') +INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020) -- !query 368 schema struct<> -- !query 368 output @@ -2955,7 +2955,7 @@ struct<> -- !query 369 -INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735') +INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735) -- !query 369 schema struct<> -- !query 369 output @@ -2963,7 +2963,7 @@ struct<> -- !query 370 -INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420') +INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420) -- !query 370 schema struct<> -- !query 370 output @@ -2971,7 +2971,7 @@ struct<> -- !query 371 -INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420') +INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420) -- !query 371 schema struct<> -- !query 371 output @@ -2979,7 +2979,7 @@ struct<> -- !query 372 -INSERT INTO num_exp_mul VALUES (9,0,'0') +INSERT INTO num_exp_mul VALUES (9,0,0) -- !query 372 schema struct<> -- !query 372 output @@ -2987,7 +2987,7 @@ struct<> -- !query 373 -INSERT INTO num_exp_div VALUES (9,0,'NaN') +INSERT INTO num_exp_div VALUES (9,0,double('NaN')) -- !query 373 schema struct<> -- !query 373 output @@ -2995,7 +2995,7 @@ struct<> -- !query 374 -INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420') +INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420) -- !query 374 schema struct<> -- !query 374 output @@ -3003,7 +3003,7 @@ struct<> -- !query 375 -INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420') +INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420) -- !query 375 schema struct<> -- !query 375 output @@ -3011,7 +3011,7 @@ struct<> -- !query 376 -INSERT INTO num_exp_mul VALUES (9,1,'0') +INSERT INTO num_exp_mul VALUES (9,1,0) -- !query 376 schema struct<> -- !query 376 output @@ -3019,7 +3019,7 @@ struct<> -- !query 377 -INSERT INTO num_exp_div VALUES (9,1,'NaN') +INSERT INTO num_exp_div VALUES (9,1,double('NaN')) -- !query 377 schema struct<> -- !query 377 output @@ -3027,7 +3027,7 @@ struct<> -- !query 378 -INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467') +INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467) -- !query 378 schema struct<> -- !query 378 output @@ -3035,7 +3035,7 @@ struct<> -- !query 379 -INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627') +INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627) -- !query 379 schema struct<> -- !query 379 output @@ -3043,7 +3043,7 @@ struct<> -- !query 380 -INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740') +INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740) -- !query 380 schema struct<> -- !query 380 output @@ -3051,7 +3051,7 @@ struct<> -- !query 381 -INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526') +INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526) -- !query 381 schema struct<> -- !query 381 output @@ -3059,7 +3059,7 @@ struct<> -- !query 382 -INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420') +INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420) -- !query 382 schema struct<> -- !query 382 output @@ -3067,7 +3067,7 @@ struct<> -- !query 383 -INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420') +INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420) -- !query 383 schema struct<> -- !query 383 output @@ -3075,7 +3075,7 @@ struct<> -- !query 384 -INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020') +INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020) -- !query 384 schema struct<> -- !query 384 output @@ -3083,7 +3083,7 @@ struct<> -- !query 385 -INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677') +INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677) -- !query 385 schema struct<> -- !query 385 output @@ -3091,7 +3091,7 @@ struct<> -- !query 386 -INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420') +INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420) -- !query 386 schema struct<> -- !query 386 output @@ -3099,7 +3099,7 @@ struct<> -- !query 387 -INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420') +INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420) -- !query 387 schema struct<> -- !query 387 output @@ -3107,7 +3107,7 @@ struct<> -- !query 388 -INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980') +INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980) -- !query 388 schema struct<> -- !query 388 output @@ -3115,7 +3115,7 @@ struct<> -- !query 389 -INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484') +INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484) -- !query 389 schema struct<> -- !query 389 output @@ -3123,7 +3123,7 @@ struct<> -- !query 390 -INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420') +INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420) -- !query 390 schema struct<> -- !query 390 output @@ -3131,7 +3131,7 @@ struct<> -- !query 391 -INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420') +INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420) -- !query 391 schema struct<> -- !query 391 output @@ -3139,7 +3139,7 @@ struct<> -- !query 392 -INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220') +INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220) -- !query 392 schema struct<> -- !query 392 output @@ -3147,7 +3147,7 @@ struct<> -- !query 393 -INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807') +INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807) -- !query 393 schema struct<> -- !query 393 output @@ -3155,7 +3155,7 @@ struct<> -- !query 394 -INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160') +INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160) -- !query 394 schema struct<> -- !query 394 output @@ -3163,7 +3163,7 @@ struct<> -- !query 395 -INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680') +INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680) -- !query 395 schema struct<> -- !query 395 output @@ -3171,7 +3171,7 @@ struct<> -- !query 396 -INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920') +INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920) -- !query 396 schema struct<> -- !query 396 output @@ -3179,7 +3179,7 @@ struct<> -- !query 397 -INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280') +INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280) -- !query 397 schema struct<> -- !query 397 output @@ -3187,7 +3187,7 @@ struct<> -- !query 398 -INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420') +INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420) -- !query 398 schema struct<> -- !query 398 output @@ -3195,7 +3195,7 @@ struct<> -- !query 399 -INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580') +INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580) -- !query 399 schema struct<> -- !query 399 output @@ -3203,7 +3203,7 @@ struct<> -- !query 400 -INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700') +INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700) -- !query 400 schema struct<> -- !query 400 output @@ -3211,7 +3211,7 @@ struct<> -- !query 401 -INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689') +INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689) -- !query 401 schema struct<> -- !query 401 output @@ -3219,7 +3219,7 @@ struct<> -- !query 402 -INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420') +INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420) -- !query 402 schema struct<> -- !query 402 output @@ -3227,7 +3227,7 @@ struct<> -- !query 403 -INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420') +INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420) -- !query 403 schema struct<> -- !query 403 output @@ -3235,7 +3235,7 @@ struct<> -- !query 404 -INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020') +INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020) -- !query 404 schema struct<> -- !query 404 output @@ -3243,7 +3243,7 @@ struct<> -- !query 405 -INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748') +INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748) -- !query 405 schema struct<> -- !query 405 output @@ -3251,7 +3251,7 @@ struct<> -- !query 406 -INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840') +INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840) -- !query 406 schema struct<> -- !query 406 output @@ -3259,7 +3259,7 @@ struct<> -- !query 407 -INSERT INTO num_exp_sub VALUES (9,9,'0') +INSERT INTO num_exp_sub VALUES (9,9,0) -- !query 407 schema struct<> -- !query 407 output @@ -3267,7 +3267,7 @@ struct<> -- !query 408 -INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400') +INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400) -- !query 408 schema struct<> -- !query 408 output @@ -3275,7 +3275,7 @@ struct<> -- !query 409 -INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000') +INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000) -- !query 409 schema struct<> -- !query 409 output @@ -3283,7 +3283,7 @@ struct<> -- !query 410 -INSERT INTO num_exp_sqrt VALUES (0,'0') +INSERT INTO num_exp_sqrt VALUES (0,0) -- !query 410 schema struct<> -- !query 410 output @@ -3291,7 +3291,7 @@ struct<> -- !query 411 -INSERT INTO num_exp_sqrt VALUES (1,'0') +INSERT INTO num_exp_sqrt VALUES (1,0) -- !query 411 schema struct<> -- !query 411 output @@ -3299,7 +3299,7 @@ struct<> -- !query 412 -INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505') +INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505) -- !query 412 schema struct<> -- !query 412 output @@ -3307,7 +3307,7 @@ struct<> -- !query 413 -INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396') +INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396) -- !query 413 schema struct<> -- !query 413 output @@ -3315,7 +3315,7 @@ struct<> -- !query 414 -INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923') +INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923) -- !query 414 schema struct<> -- !query 414 output @@ -3323,7 +3323,7 @@ struct<> -- !query 415 -INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473') +INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473) -- !query 415 schema struct<> -- !query 415 output @@ -3331,7 +3331,7 @@ struct<> -- !query 416 -INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406') +INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406) -- !query 416 schema struct<> -- !query 416 output @@ -3339,7 +3339,7 @@ struct<> -- !query 417 -INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230') +INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230) -- !query 417 schema struct<> -- !query 417 output @@ -3347,7 +3347,7 @@ struct<> -- !query 418 -INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542') +INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542) -- !query 418 schema struct<> -- !query 418 output @@ -3355,7 +3355,7 @@ struct<> -- !query 419 -INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766') +INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766) -- !query 419 schema struct<> -- !query 419 output @@ -3363,7 +3363,7 @@ struct<> -- !query 420 -INSERT INTO num_exp_ln VALUES (0,'NaN') +INSERT INTO num_exp_ln VALUES (0,double('NaN')) -- !query 420 schema struct<> -- !query 420 output @@ -3371,7 +3371,7 @@ struct<> -- !query 421 -INSERT INTO num_exp_ln VALUES (1,'NaN') +INSERT INTO num_exp_ln VALUES (1,double('NaN')) -- !query 421 schema struct<> -- !query 421 output @@ -3379,7 +3379,7 @@ struct<> -- !query 422 -INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514') +INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514) -- !query 422 schema struct<> -- !query 422 output @@ -3387,7 +3387,7 @@ struct<> -- !query 423 -INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971') +INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971) -- !query 423 schema struct<> -- !query 423 output @@ -3395,7 +3395,7 @@ struct<> -- !query 424 -INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464') +INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464) -- !query 424 schema struct<> -- !query 424 output @@ -3403,7 +3403,7 @@ struct<> -- !query 425 -INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038') +INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038) -- !query 425 schema struct<> -- !query 425 output @@ -3411,7 +3411,7 @@ struct<> -- !query 426 -INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127') +INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127) -- !query 426 schema struct<> -- !query 426 output @@ -3419,7 +3419,7 @@ struct<> -- !query 427 -INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991') +INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991) -- !query 427 schema struct<> -- !query 427 output @@ -3427,7 +3427,7 @@ struct<> -- !query 428 -INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668') +INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668) -- !query 428 schema struct<> -- !query 428 output @@ -3435,7 +3435,7 @@ struct<> -- !query 429 -INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962') +INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962) -- !query 429 schema struct<> -- !query 429 output @@ -3443,7 +3443,7 @@ struct<> -- !query 430 -INSERT INTO num_exp_log10 VALUES (0,'NaN') +INSERT INTO num_exp_log10 VALUES (0,double('NaN')) -- !query 430 schema struct<> -- !query 430 output @@ -3451,7 +3451,7 @@ struct<> -- !query 431 -INSERT INTO num_exp_log10 VALUES (1,'NaN') +INSERT INTO num_exp_log10 VALUES (1,double('NaN')) -- !query 431 schema struct<> -- !query 431 output @@ -3459,7 +3459,7 @@ struct<> -- !query 432 -INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459') +INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459) -- !query 432 schema struct<> -- !query 432 output @@ -3467,7 +3467,7 @@ struct<> -- !query 433 -INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075') +INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075) -- !query 433 schema struct<> -- !query 433 output @@ -3475,7 +3475,7 @@ struct<> -- !query 434 -INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345') +INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345) -- !query 434 schema struct<> -- !query 434 output @@ -3483,7 +3483,7 @@ struct<> -- !query 435 -INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626') +INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626) -- !query 435 schema struct<> -- !query 435 output @@ -3491,7 +3491,7 @@ struct<> -- !query 436 -INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671') +INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671) -- !query 436 schema struct<> -- !query 436 output @@ -3499,7 +3499,7 @@ struct<> -- !query 437 -INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914') +INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914) -- !query 437 schema struct<> -- !query 437 output @@ -3507,7 +3507,7 @@ struct<> -- !query 438 -INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138') +INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138) -- !query 438 schema struct<> -- !query 438 output @@ -3515,7 +3515,7 @@ struct<> -- !query 439 -INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059') +INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059) -- !query 439 schema struct<> -- !query 439 output @@ -3523,7 +3523,7 @@ struct<> -- !query 440 -INSERT INTO num_exp_power_10_ln VALUES (0,'NaN') +INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')) -- !query 440 schema struct<> -- !query 440 output @@ -3531,7 +3531,7 @@ struct<> -- !query 441 -INSERT INTO num_exp_power_10_ln VALUES (1,'NaN') +INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')) -- !query 441 schema struct<> -- !query 441 output @@ -3539,7 +3539,7 @@ struct<> -- !query 442 -INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184') +INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184) -- !query 442 schema struct<> -- !query 442 output @@ -3547,7 +3547,7 @@ struct<> -- !query 443 -INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393') +INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393) -- !query 443 schema struct<> -- !query 443 output @@ -3555,7 +3555,7 @@ struct<> -- !query 444 -INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636') +INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636) -- !query 444 schema struct<> -- !query 444 output @@ -3563,7 +3563,7 @@ struct<> -- !query 445 -INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098') +INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098) -- !query 445 schema struct<> -- !query 445 output @@ -3571,7 +3571,7 @@ struct<> -- !query 446 -INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067') +INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067) -- !query 446 schema struct<> -- !query 446 output @@ -3579,15 +3579,19 @@ struct<> -- !query 447 -INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627') +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) -- !query 447 schema struct<> -- !query 447 output +org.apache.spark.sql.catalyst.parser.ParseException +decimal can only support precision up to 38 +== SQL == +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) -- !query 448 -INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952') +INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952) -- !query 448 schema struct<> -- !query 448 output @@ -3595,7 +3599,7 @@ struct<> -- !query 449 -INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457') +INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457) -- !query 449 schema struct<> -- !query 449 output @@ -3603,7 +3607,7 @@ struct<> -- !query 450 -INSERT INTO num_data VALUES (0, '0') +INSERT INTO num_data VALUES (0, 0) -- !query 450 schema struct<> -- !query 450 output @@ -3611,7 +3615,7 @@ struct<> -- !query 451 -INSERT INTO num_data VALUES (1, '0') +INSERT INTO num_data VALUES (1, 0) -- !query 451 schema struct<> -- !query 451 output @@ -3619,7 +3623,7 @@ struct<> -- !query 452 -INSERT INTO num_data VALUES (2, '-34338492.215397047') +INSERT INTO num_data VALUES (2, -34338492.215397047) -- !query 452 schema struct<> -- !query 452 output @@ -3627,7 +3631,7 @@ struct<> -- !query 453 -INSERT INTO num_data VALUES (3, '4.31') +INSERT INTO num_data VALUES (3, 4.31) -- !query 453 schema struct<> -- !query 453 output @@ -3635,7 +3639,7 @@ struct<> -- !query 454 -INSERT INTO num_data VALUES (4, '7799461.4119') +INSERT INTO num_data VALUES (4, 7799461.4119) -- !query 454 schema struct<> -- !query 454 output @@ -3643,7 +3647,7 @@ struct<> -- !query 455 -INSERT INTO num_data VALUES (5, '16397.038491') +INSERT INTO num_data VALUES (5, 16397.038491) -- !query 455 schema struct<> -- !query 455 output @@ -3651,7 +3655,7 @@ struct<> -- !query 456 -INSERT INTO num_data VALUES (6, '93901.57763026') +INSERT INTO num_data VALUES (6, 93901.57763026) -- !query 456 schema struct<> -- !query 456 output @@ -3659,7 +3663,7 @@ struct<> -- !query 457 -INSERT INTO num_data VALUES (7, '-83028485') +INSERT INTO num_data VALUES (7, -83028485) -- !query 457 schema struct<> -- !query 457 output @@ -3667,7 +3671,7 @@ struct<> -- !query 458 -INSERT INTO num_data VALUES (8, '74881') +INSERT INTO num_data VALUES (8, 74881) -- !query 458 schema struct<> -- !query 458 output @@ -3675,7 +3679,7 @@ struct<> -- !query 459 -INSERT INTO num_data VALUES (9, '-24926804.045047420') +INSERT INTO num_data VALUES (9, -24926804.045047420) -- !query 459 schema struct<> -- !query 459 output @@ -4196,7 +4200,6 @@ struct 4 7405685069595001 7405685069594999.0773399947 5 5068226527.321263 5068226527.3212726541 6 281839893606.99365 281839893606.9937234336 -7 1716699575118595580 1716699575118597095.4233081991 8 167361463828.0749 167361463828.0749132007 9 107511333880051872 107511333880052007.0414112467 @@ -4218,7 +4221,7 @@ struct<> -- !query 499 -INSERT INTO fract_only VALUES (1, '0.0') +INSERT INTO fract_only VALUES (1, 0.0) -- !query 499 schema struct<> -- !query 499 output @@ -4226,7 +4229,7 @@ struct<> -- !query 500 -INSERT INTO fract_only VALUES (2, '0.1') +INSERT INTO fract_only VALUES (2, 0.1) -- !query 500 schema struct<> -- !query 500 output @@ -4234,7 +4237,7 @@ struct<> -- !query 501 -INSERT INTO fract_only VALUES (4, '-0.9999') +INSERT INTO fract_only VALUES (4, -0.9999) -- !query 501 schema struct<> -- !query 501 output @@ -4242,7 +4245,7 @@ struct<> -- !query 502 -INSERT INTO fract_only VALUES (5, '0.99994') +INSERT INTO fract_only VALUES (5, 0.99994) -- !query 502 schema struct<> -- !query 502 output @@ -4250,7 +4253,7 @@ struct<> -- !query 503 -INSERT INTO fract_only VALUES (7, '0.00001') +INSERT INTO fract_only VALUES (7, 0.00001) -- !query 503 schema struct<> -- !query 503 output @@ -4258,7 +4261,7 @@ struct<> -- !query 504 -INSERT INTO fract_only VALUES (8, '0.00017') +INSERT INTO fract_only VALUES (8, 0.00017) -- !query 504 schema struct<> -- !query 504 output @@ -4343,7 +4346,7 @@ struct<> -- !query 514 -INSERT INTO ceil_floor_round VALUES ('-5.5') +INSERT INTO ceil_floor_round VALUES (-5.5) -- !query 514 schema struct<> -- !query 514 output @@ -4351,7 +4354,7 @@ struct<> -- !query 515 -INSERT INTO ceil_floor_round VALUES ('-5.499999') +INSERT INTO ceil_floor_round VALUES (-5.499999) -- !query 515 schema struct<> -- !query 515 output @@ -4359,7 +4362,7 @@ struct<> -- !query 516 -INSERT INTO ceil_floor_round VALUES ('9.5') +INSERT INTO ceil_floor_round VALUES (9.5) -- !query 516 schema struct<> -- !query 516 output @@ -4367,7 +4370,7 @@ struct<> -- !query 517 -INSERT INTO ceil_floor_round VALUES ('9.4999999') +INSERT INTO ceil_floor_round VALUES (9.4999999) -- !query 517 schema struct<> -- !query 517 output @@ -4375,7 +4378,7 @@ struct<> -- !query 518 -INSERT INTO ceil_floor_round VALUES ('0.0') +INSERT INTO ceil_floor_round VALUES (0.0) -- !query 518 schema struct<> -- !query 518 output @@ -4383,7 +4386,7 @@ struct<> -- !query 519 -INSERT INTO ceil_floor_round VALUES ('0.0000001') +INSERT INTO ceil_floor_round VALUES (0.0000001) -- !query 519 schema struct<> -- !query 519 output @@ -4391,7 +4394,7 @@ struct<> -- !query 520 -INSERT INTO ceil_floor_round VALUES ('-0.000001') +INSERT INTO ceil_floor_round VALUES (-0.000001) -- !query 520 schema struct<> -- !query 520 output @@ -4429,7 +4432,7 @@ struct<> -- !query 524 -INSERT INTO num_input_test VALUES (trim(' 123')) +INSERT INTO num_input_test VALUES (double(trim(' 123'))) -- !query 524 schema struct<> -- !query 524 output @@ -4437,7 +4440,7 @@ struct<> -- !query 525 -INSERT INTO num_input_test VALUES (trim(' 3245874 ')) +INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))) -- !query 525 schema struct<> -- !query 525 output @@ -4445,7 +4448,7 @@ struct<> -- !query 526 -INSERT INTO num_input_test VALUES (trim(' -93853')) +INSERT INTO num_input_test VALUES (double(trim(' -93853'))) -- !query 526 schema struct<> -- !query 526 output @@ -4453,7 +4456,7 @@ struct<> -- !query 527 -INSERT INTO num_input_test VALUES ('555.50') +INSERT INTO num_input_test VALUES (555.50) -- !query 527 schema struct<> -- !query 527 output @@ -4461,7 +4464,7 @@ struct<> -- !query 528 -INSERT INTO num_input_test VALUES ('-555.50') +INSERT INTO num_input_test VALUES (-555.50) -- !query 528 schema struct<> -- !query 528 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out index f8bce0c97c8c5..6cb3690a464e1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out @@ -11,7 +11,7 @@ struct<> -- !query 1 -INSERT INTO TIMESTAMP_TBL VALUES ('now') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')) -- !query 1 schema struct<> -- !query 1 output @@ -19,7 +19,7 @@ struct<> -- !query 2 -INSERT INTO TIMESTAMP_TBL VALUES ('now') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')) -- !query 2 schema struct<> -- !query 2 output @@ -27,7 +27,7 @@ struct<> -- !query 3 -INSERT INTO TIMESTAMP_TBL VALUES ('today') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('today')) -- !query 3 schema struct<> -- !query 3 output @@ -35,7 +35,7 @@ struct<> -- !query 4 -INSERT INTO TIMESTAMP_TBL VALUES ('yesterday') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('yesterday')) -- !query 4 schema struct<> -- !query 4 output @@ -43,7 +43,7 @@ struct<> -- !query 5 -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow')) -- !query 5 schema struct<> -- !query 5 output @@ -51,7 +51,7 @@ struct<> -- !query 6 -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow EST')) -- !query 6 schema struct<> -- !query 6 output @@ -59,7 +59,7 @@ struct<> -- !query 7 -INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow Zulu') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow Zulu')) -- !query 7 schema struct<> -- !query 7 output @@ -99,7 +99,7 @@ struct<> -- !query 12 -INSERT INTO TIMESTAMP_TBL VALUES ('epoch') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('epoch')) -- !query 12 schema struct<> -- !query 12 output @@ -107,7 +107,7 @@ struct<> -- !query 13 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02')) -- !query 13 schema struct<> -- !query 13 output @@ -115,7 +115,7 @@ struct<> -- !query 14 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05')) -- !query 14 schema struct<> -- !query 14 output @@ -123,7 +123,7 @@ struct<> -- !query 15 -INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08')) -- !query 15 schema struct<> -- !query 15 output @@ -131,7 +131,7 @@ struct<> -- !query 16 -INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20')) -- !query 16 schema struct<> -- !query 16 output @@ -219,11 +219,11 @@ SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff -- !query 24 schema struct<54:string,diff:interval> -- !query 24 output - interval -236720 hours - interval 0 microseconds - interval 3 hours 4 minutes 5 seconds - interval 41393 hours 19 minutes 20 seconds - interval 953 hours 32 minutes 1 seconds + -236720 hours + 0 seconds + 3 hours 4 minutes 5 seconds + 41393 hours 19 minutes 20 seconds + 953 hours 32 minutes 1 seconds -- !query 25 @@ -242,11 +242,11 @@ SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff -- !query 26 schema struct<54:string,diff:interval> -- !query 26 output - interval -236720 hours - interval 0 microseconds - interval 3 hours 4 minutes 5 seconds - interval 41393 hours 19 minutes 20 seconds - interval 953 hours 32 minutes 1 seconds + -236720 hours + 0 seconds + 3 hours 4 minutes 5 seconds + 41393 hours 19 minutes 20 seconds + 953 hours 32 minutes 1 seconds -- !query 27 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index ae0ccf556fbc3..960e644732990 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -143,19 +143,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { /** List of test cases to ignore, in lower cases. */ protected def blackList: Set[String] = Set( - "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. - // SPARK-28885 String value is not allowed to be stored as numeric type with - // ANSI store assignment policy. - "postgreSQL/numeric.sql", - "postgreSQL/int2.sql", - "postgreSQL/int4.sql", - "postgreSQL/int8.sql", - "postgreSQL/float4.sql", - "postgreSQL/float8.sql", - // SPARK-28885 String value is not allowed to be stored as date/timestamp type with - // ANSI store assignment policy. - "postgreSQL/date.sql", - "postgreSQL/timestamp.sql" + "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. ) // Create all the test cases. From 23b3c4fafdf37a482b3f823a5701d99d96236519 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 21 Nov 2019 00:56:48 +0800 Subject: [PATCH 0676/1387] [SPARK-29951][SQL] Make the behavior of Postgre dialect independent of ansi mode config ### What changes were proposed in this pull request? Fix the inconsistent behavior of build-in function SQL LEFT/RIGHT. ### Why are the changes needed? As the comment in https://github.com/apache/spark/pull/26497#discussion_r345708065, Postgre dialect should not be affected by the ANSI mode config. During reran the existing tests, only the LEFT/RIGHT build-in SQL function broke the assumption. We fix this by following https://www.postgresql.org/docs/12/sql-keywords-appendix.html: `LEFT/RIGHT reserved (can be function or type)` ### Does this PR introduce any user-facing change? Yes, the Postgre dialect will not be affected by the ANSI mode config. ### How was this patch tested? Existing UT. Closes #26584 from xuanyuanking/SPARK-29951. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 7 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 26 +-- .../sql/catalyst/parser/AstBuilder.scala | 16 +- .../sql/catalyst/parser/ParseDriver.scala | 12 +- .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../sql-tests/inputs/postgreSQL/text.sql | 4 - .../sql-tests/results/postgreSQL/text.sql.out | 160 ++++++++---------- .../apache/spark/sql/SQLQueryTestSuite.scala | 1 - 8 files changed, 119 insertions(+), 111 deletions(-) mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 79bc134596237..3117ee40a8c9b 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,12 +19,13 @@ license: | limitations under the License. --- -When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.dialect=PostgreSQL` or keep default `spark.sql.dialect=Spark` with setting `spark.sql.dialect.spark.ansi.enabled` to true, Spark SQL will use the ANSI mode parser. +In this mode, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.dialect.spark.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. +When the ANSI mode is disabled, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when the ANSI mode enabled. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. By default `spark.sql.dialect.spark.ansi.enabled` is false. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 4f81dc47394a5..cc273fd36011e 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -46,9 +46,9 @@ grammar SqlBase; } /** - * When true, ANSI SQL parsing mode is enabled. + * When true, the behavior of keywords follows ANSI SQL standard. */ - public boolean ansi = false; + public boolean SQL_standard_keyword_behavior = false; } singleStatement @@ -744,7 +744,7 @@ primaryExpression | qualifiedName '.' ASTERISK #star | '(' namedExpression (',' namedExpression)+ ')' #rowConstructor | '(' query ')' #subqueryExpression - | qualifiedName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' + | functionName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' (OVER windowSpec)? #functionCall | identifier '->' expression #lambda | '(' identifier (',' identifier)+ ')' '->' expression #lambda @@ -788,7 +788,7 @@ booleanValue interval : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? - | {ansi}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) + | {SQL_standard_keyword_behavior}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; errorCapturingMultiUnitsInterval @@ -908,6 +908,12 @@ qualifiedNameList : qualifiedName (',' qualifiedName)* ; +functionName + : qualifiedName + | LEFT + | RIGHT + ; + qualifiedName : identifier ('.' identifier)* ; @@ -927,14 +933,14 @@ errorCapturingIdentifierExtra identifier : strictIdentifier - | {!ansi}? strictNonReserved + | {!SQL_standard_keyword_behavior}? strictNonReserved ; strictIdentifier : IDENTIFIER #unquotedIdentifier | quotedIdentifier #quotedIdentifierAlternative - | {ansi}? ansiNonReserved #unquotedIdentifier - | {!ansi}? nonReserved #unquotedIdentifier + | {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier + | {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier ; quotedIdentifier @@ -951,7 +957,7 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -// When `spark.sql.dialect.spark.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. +// When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, // function, alias, etc. @@ -1151,9 +1157,9 @@ ansiNonReserved | YEARS ; -// When `spark.sql.dialect.spark.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. +// When `SQL_standard_keyword_behavior=false`, there are 2 kinds of keywords in Spark SQL. // - Non-reserved keywords: -// Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. +// Same definition as the one when `SQL_standard_keyword_behavior=true`. // - Strict-non-reserved keywords: // A strict version of non-reserved keywords, which can not be used as table alias. // You can find the full keywords list by searching "Start of the keywords list" in this file. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index e0476879f0e80..7bec46678f58d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1581,7 +1581,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) { // Create the function call. - val name = ctx.qualifiedName.getText + val name = ctx.functionName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) val arguments = ctx.argument.asScala.map(expression) match { case Seq(UnresolvedStar(None)) @@ -1591,7 +1591,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case expressions => expressions } - val function = UnresolvedFunction(visitFunctionName(ctx.qualifiedName), arguments, isDistinct) + val function = UnresolvedFunction( + getFunctionIdentifier(ctx.functionName), arguments, isDistinct) // Check if the function is evaluated in a windowed context. ctx.windowSpec match { @@ -1631,6 +1632,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Get a function identifier consist by database (optional) and name. + */ + protected def getFunctionIdentifier(ctx: FunctionNameContext): FunctionIdentifier = { + if (ctx.qualifiedName != null) { + visitFunctionName(ctx.qualifiedName) + } else { + FunctionIdentifier(ctx.getText, None) + } + } + /** * Create an [[LambdaFunction]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index e291d5f9cd84c..30c36598d81d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.Dialect import org.apache.spark.sql.types.{DataType, StructType} /** @@ -88,11 +89,18 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = { logDebug(s"Parsing command: $command") + // When we use PostgreSQL dialect or use Spark dialect with setting + // `spark.sql.dialect.spark.ansi.enabled=true`, the parser will use ANSI SQL standard keywords. + val SQLStandardKeywordBehavior = conf.dialect match { + case Dialect.POSTGRESQL => true + case Dialect.SPARK => conf.dialectSparkAnsiEnabled + } + val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - lexer.ansi = conf.dialectSparkAnsiEnabled + lexer.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -100,7 +108,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced - parser.ansi = conf.dialectSparkAnsiEnabled + parser.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior try { try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d4fcefe99ee75..74046cd91c962 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2515,7 +2515,9 @@ class SQLConf extends Serializable with Logging { def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE)) - def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString + def dialect: Dialect.Value = Dialect.withName(getConf(DIALECT)) + + def usePostgreSQLDialect: Boolean = dialect == Dialect.POSTGRESQL def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql index a1fe95462ecae..05953123da86f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql @@ -44,11 +44,7 @@ select concat_ws(',',10,20,null,30); select concat_ws('',10,20,null,30); select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); --- [SPARK-28036] Built-in udf left/right has inconsistent behavior --- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode -set spark.sql.dialect.spark.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; -set spark.sql.dialect.spark.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out old mode 100644 new mode 100755 index cac1e7ee5ab12..311b0eb5a5844 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 44 +-- Number of queries: 42 -- !query 0 @@ -151,18 +151,10 @@ edcba -- !query 18 -set spark.sql.dialect.spark.ansi.enabled=false --- !query 18 schema -struct --- !query 18 output -spark.sql.dialect.spark.ansi.enabled false - - --- !query 19 select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i --- !query 19 schema +-- !query 18 schema struct --- !query 19 output +-- !query 18 output -5 -4 -3 @@ -176,200 +168,192 @@ struct 5 ahoj ahoj --- !query 20 -set spark.sql.dialect.spark.ansi.enabled=true --- !query 20 schema -struct --- !query 20 output -spark.sql.dialect.spark.ansi.enabled true - - --- !query 21 +-- !query 19 /* * format */ select format_string(NULL) --- !query 21 schema +-- !query 19 schema struct --- !query 21 output +-- !query 19 output NULL --- !query 22 +-- !query 20 select format_string('Hello') --- !query 22 schema +-- !query 20 schema struct --- !query 22 output +-- !query 20 output Hello --- !query 23 +-- !query 21 select format_string('Hello %s', 'World') --- !query 23 schema +-- !query 21 schema struct --- !query 23 output +-- !query 21 output Hello World --- !query 24 +-- !query 22 select format_string('Hello %%') --- !query 24 schema +-- !query 22 schema struct --- !query 24 output +-- !query 22 output Hello % --- !query 25 +-- !query 23 select format_string('Hello %%%%') --- !query 25 schema +-- !query 23 schema struct --- !query 25 output +-- !query 23 output Hello %% --- !query 26 +-- !query 24 select format_string('Hello %s %s', 'World') --- !query 26 schema +-- !query 24 schema struct<> --- !query 26 output +-- !query 24 output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 27 +-- !query 25 select format_string('Hello %s') --- !query 27 schema +-- !query 25 schema struct<> --- !query 27 output +-- !query 25 output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 28 +-- !query 26 select format_string('Hello %x', 20) --- !query 28 schema +-- !query 26 schema struct --- !query 28 output +-- !query 26 output Hello 14 --- !query 29 +-- !query 27 select format_string('%1$s %3$s', 1, 2, 3) --- !query 29 schema +-- !query 27 schema struct --- !query 29 output +-- !query 27 output 1 3 --- !query 30 +-- !query 28 select format_string('%1$s %12$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 30 schema +-- !query 28 schema struct --- !query 30 output +-- !query 28 output 1 12 --- !query 31 +-- !query 29 select format_string('%1$s %4$s', 1, 2, 3) --- !query 31 schema +-- !query 29 schema struct<> --- !query 31 output +-- !query 29 output java.util.MissingFormatArgumentException Format specifier '%4$s' --- !query 32 +-- !query 30 select format_string('%1$s %13$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 32 schema +-- !query 30 schema struct<> --- !query 32 output +-- !query 30 output java.util.MissingFormatArgumentException Format specifier '%13$s' --- !query 33 +-- !query 31 select format_string('%0$s', 'Hello') --- !query 33 schema +-- !query 31 schema struct --- !query 33 output +-- !query 31 output Hello --- !query 34 +-- !query 32 select format_string('Hello %s %1$s %s', 'World', 'Hello again') --- !query 34 schema +-- !query 32 schema struct --- !query 34 output +-- !query 32 output Hello World World Hello again --- !query 35 +-- !query 33 select format_string('Hello %s %s, %2$s %2$s', 'World', 'Hello again') --- !query 35 schema +-- !query 33 schema struct --- !query 35 output +-- !query 33 output Hello World Hello again, Hello again Hello again --- !query 36 +-- !query 34 select format_string('>>%10s<<', 'Hello') --- !query 36 schema +-- !query 34 schema struct>%10s<<, Hello):string> --- !query 36 output +-- !query 34 output >> Hello<< --- !query 37 +-- !query 35 select format_string('>>%10s<<', NULL) --- !query 37 schema +-- !query 35 schema struct>%10s<<, NULL):string> --- !query 37 output +-- !query 35 output >> null<< --- !query 38 +-- !query 36 select format_string('>>%10s<<', '') --- !query 38 schema +-- !query 36 schema struct>%10s<<, ):string> --- !query 38 output +-- !query 36 output >> << --- !query 39 +-- !query 37 select format_string('>>%-10s<<', '') --- !query 39 schema +-- !query 37 schema struct>%-10s<<, ):string> --- !query 39 output +-- !query 37 output >> << --- !query 40 +-- !query 38 select format_string('>>%-10s<<', 'Hello') --- !query 40 schema +-- !query 38 schema struct>%-10s<<, Hello):string> --- !query 40 output +-- !query 38 output >>Hello << --- !query 41 +-- !query 39 select format_string('>>%-10s<<', NULL) --- !query 41 schema +-- !query 39 schema struct>%-10s<<, NULL):string> --- !query 41 output +-- !query 39 output >>null << --- !query 42 +-- !query 40 select format_string('>>%1$10s<<', 'Hello') --- !query 42 schema +-- !query 40 schema struct>%1$10s<<, Hello):string> --- !query 42 output +-- !query 40 output >> Hello<< --- !query 43 +-- !query 41 DROP TABLE TEXT_TBL --- !query 43 schema +-- !query 41 schema struct<> --- !query 43 output +-- !query 41 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 960e644732990..6e86d37a310d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -335,7 +335,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) case _: AnsiTest => localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) From 1febd373ea806326d269a60048ee52543a76c918 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 20 Nov 2019 14:04:15 -0600 Subject: [PATCH 0677/1387] [MINOR][TESTS] Replace JVM assert with JUnit Assert in tests ### What changes were proposed in this pull request? Use JUnit assertions in tests uniformly, not JVM assert() statements. ### Why are the changes needed? assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything. ### Does this PR introduce any user-facing change? No. The assertion logic should be identical. ### How was this patch tested? Existing tests. Closes #26581 from srowen/assertToJUnit. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../shuffle/TestShuffleDataContext.java | 3 +- .../org/apache/spark/DistributedSuite.scala | 1 + .../spark/MapStatusesSerDeserBenchmark.scala | 2 ++ .../apache/spark/benchmark/Benchmark.scala | 1 + .../apache/spark/executor/ExecutorSuite.scala | 1 + .../apache/spark/rpc/TestRpcEndpoint.scala | 1 + .../scheduler/SchedulerIntegrationSuite.scala | 3 +- .../scheduler/TaskResultGetterSuite.scala | 1 + .../spark/scheduler/TaskSetManagerSuite.scala | 3 +- .../ShuffleDriverComponentsSuite.scala | 1 + .../util/PeriodicRDDCheckpointerSuite.scala | 1 + .../spark/sql/kafka010/KafkaTestUtils.scala | 6 ++-- .../JavaKinesisInputDStreamBuilderSuite.java | 34 +++++++++++-------- .../util/PeriodicGraphCheckpointerSuite.scala | 3 +- .../stat/JavaKolmogorovSmirnovTestSuite.java | 5 +-- .../ml/classification/LinearSVCSuite.scala | 1 + .../LogisticRegressionSuite.scala | 2 ++ .../ml/classification/OneVsRestSuite.scala | 2 ++ .../ProbabilisticClassifierSuite.scala | 2 ++ .../org/apache/spark/ml/feature/LSHTest.scala | 2 ++ .../LogisticRegressionSuite.scala | 1 + .../spark/mllib/clustering/LDASuite.scala | 1 + .../spark/mllib/tree/EnsembleTestHelper.scala | 2 ++ .../spark/scheduler/cluster/mesos/Utils.scala | 7 ++-- .../sql/streaming/JavaOutputModeSuite.java | 5 +-- .../analysis/AnalysisErrorSuite.scala | 2 ++ .../expressions/LiteralGenerator.scala | 1 + .../spark/sql/connector/InMemoryTable.scala | 2 ++ .../sort/RecordBinaryComparatorSuite.java | 30 ++++++++-------- .../org/apache/spark/sql/DatasetSuite.scala | 1 + .../spark/sql/IntegratedUDFTestUtils.scala | 2 ++ .../HashedRelationMetricsBenchmark.scala | 4 ++- .../execution/benchmark/JoinBenchmark.scala | 2 ++ .../benchmark/WideSchemaBenchmark.scala | 2 ++ .../CompressionSchemeBenchmark.scala | 1 + .../internal/ExecutorSideSQLConfSuite.scala | 2 ++ .../spark/sql/hive/HiveSparkSubmitSuite.scala | 5 +-- .../execution/ScriptTransformationSuite.scala | 1 + ...iveHadoopDelegationTokenManagerSuite.scala | 1 + .../spark/streaming/MasterFailureTest.scala | 1 + .../streaming/util/WriteAheadLogSuite.scala | 1 + 41 files changed, 102 insertions(+), 47 deletions(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 457805feeac45..fb67d7220a0b4 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -28,6 +28,7 @@ import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.JavaUtils; +import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -122,7 +123,7 @@ private void insertFile(String filename) throws IOException { private void insertFile(String filename, byte[] block) throws IOException { OutputStream dataStream = null; File file = ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, filename); - assert(!file.exists()) : "this test file has been already generated"; + Assert.assertFalse("this test file has been already generated", file.exists()); try { dataStream = new FileOutputStream( ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, filename)); diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 8173a8e545ebb..3f309819065be 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark +import org.scalatest.Assertions._ import org.scalatest.Matchers import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.time.{Millis, Span} diff --git a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala index 5dbef88e73a9e..78f1246295bf8 100644 --- a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala @@ -17,6 +17,8 @@ package org.apache.spark +import org.scalatest.Assertions._ + import org.apache.spark.benchmark.Benchmark import org.apache.spark.benchmark.BenchmarkBase import org.apache.spark.scheduler.CompressedMapStatus diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index 022fcbb25b0af..9629f5ab1a3dd 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -26,6 +26,7 @@ import scala.util.Try import org.apache.commons.io.output.TeeOutputStream import org.apache.commons.lang3.SystemUtils +import org.scalatest.Assertions._ import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 2115ee8b1b723..7272a98c9770b 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -33,6 +33,7 @@ import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{inOrder, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer +import org.scalatest.Assertions._ import org.scalatest.PrivateMethodTester import org.scalatest.concurrent.Eventually import org.scalatestplus.mockito.MockitoSugar diff --git a/core/src/test/scala/org/apache/spark/rpc/TestRpcEndpoint.scala b/core/src/test/scala/org/apache/spark/rpc/TestRpcEndpoint.scala index 5e8da3e205ab0..7c65f3b126e3d 100644 --- a/core/src/test/scala/org/apache/spark/rpc/TestRpcEndpoint.scala +++ b/core/src/test/scala/org/apache/spark/rpc/TestRpcEndpoint.scala @@ -20,6 +20,7 @@ package org.apache.spark.rpc import scala.collection.mutable.ArrayBuffer import org.scalactic.TripleEquals +import org.scalatest.Assertions._ class TestRpcEndpoint extends ThreadSafeRpcEndpoint with TripleEquals { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 4f737c9499ad6..dff8975a4fe49 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -26,6 +26,7 @@ import scala.concurrent.duration.{Duration, SECONDS} import scala.reflect.ClassTag import org.scalactic.TripleEquals +import org.scalatest.Assertions import org.scalatest.Assertions.AssertionsHelper import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ @@ -463,7 +464,7 @@ class MockRDD( override def toString: String = "MockRDD " + id } -object MockRDD extends AssertionsHelper with TripleEquals { +object MockRDD extends AssertionsHelper with TripleEquals with Assertions { /** * make sure all the shuffle dependencies have a consistent number of output partitions * (mostly to make sure the test setup makes sense, not that Spark itself would get this wrong) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 8439be955c738..406bd9244870e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.MoreExecutors import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{spy, times, verify} +import org.scalatest.Assertions._ import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 89df5de97c444..34bcae8abd512 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileAlreadyExistsException import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock +import org.scalatest.Assertions._ import org.apache.spark._ import org.apache.spark.internal.Logging @@ -128,7 +129,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex def removeExecutor(execId: String): Unit = { executors -= execId val host = executorIdToHost.get(execId) - assert(host != None) + assert(host.isDefined) val hostId = host.get val executorsOnHost = hostToExecutors(hostId) executorsOnHost -= execId diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala index d8657ecdff676..3d70ff1fed29f 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleDriverComponentsSuite.scala @@ -21,6 +21,7 @@ import java.util.{Map => JMap} import java.util.concurrent.atomic.AtomicBoolean import com.google.common.collect.ImmutableMap +import org.scalatest.Assertions._ import org.scalatest.BeforeAndAfterEach import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} diff --git a/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala b/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala index 06c2ceb68bd79..f14ec175232be 100644 --- a/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/PeriodicRDDCheckpointerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import org.apache.hadoop.fs.Path +import org.scalatest.Assertions._ import org.apache.spark.{SharedSparkContext, SparkContext, SparkFunSuite} import org.apache.spark.rdd.RDD diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 6c745987b4c23..6e97c37af7df0 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -18,10 +18,9 @@ package org.apache.spark.sql.kafka010 import java.io.{File, IOException} -import java.lang.{Integer => JInt} import java.net.{InetAddress, InetSocketAddress} import java.nio.charset.StandardCharsets -import java.util.{Collections, Map => JMap, Properties, UUID} +import java.util.{Collections, Properties, UUID} import java.util.concurrent.TimeUnit import javax.security.auth.login.Configuration @@ -41,13 +40,12 @@ import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.SaslConfigs -import org.apache.kafka.common.header.Header -import org.apache.kafka.common.header.internals.RecordHeader import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol.{PLAINTEXT, SASL_PLAINTEXT} import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} import org.apache.zookeeper.server.auth.SASLAuthenticationProvider +import org.scalatest.Assertions._ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ diff --git a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java index 03becd73d1a06..7af0abe0e8d90 100644 --- a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java +++ b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java @@ -18,12 +18,14 @@ package org.apache.spark.streaming.kinesis; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.junit.Assert; +import org.junit.Test; + import org.apache.spark.streaming.kinesis.KinesisInitialPositions.TrimHorizon; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.LocalJavaStreamingContext; import org.apache.spark.streaming.Seconds; -import org.junit.Test; public class JavaKinesisInputDStreamBuilderSuite extends LocalJavaStreamingContext { /** @@ -49,13 +51,14 @@ public void testJavaKinesisDStreamBuilder() { .checkpointInterval(checkpointInterval) .storageLevel(storageLevel) .build(); - assert(kinesisDStream.streamName() == streamName); - assert(kinesisDStream.endpointUrl() == endpointUrl); - assert(kinesisDStream.regionName() == region); - assert(kinesisDStream.initialPosition().getPosition() == initialPosition.getPosition()); - assert(kinesisDStream.checkpointAppName() == appName); - assert(kinesisDStream.checkpointInterval() == checkpointInterval); - assert(kinesisDStream._storageLevel() == storageLevel); + Assert.assertEquals(streamName, kinesisDStream.streamName()); + Assert.assertEquals(endpointUrl, kinesisDStream.endpointUrl()); + Assert.assertEquals(region, kinesisDStream.regionName()); + Assert.assertEquals(initialPosition.getPosition(), + kinesisDStream.initialPosition().getPosition()); + Assert.assertEquals(appName, kinesisDStream.checkpointAppName()); + Assert.assertEquals(checkpointInterval, kinesisDStream.checkpointInterval()); + Assert.assertEquals(storageLevel, kinesisDStream._storageLevel()); ssc.stop(); } @@ -83,13 +86,14 @@ public void testJavaKinesisDStreamBuilderOldApi() { .checkpointInterval(checkpointInterval) .storageLevel(storageLevel) .build(); - assert(kinesisDStream.streamName() == streamName); - assert(kinesisDStream.endpointUrl() == endpointUrl); - assert(kinesisDStream.regionName() == region); - assert(kinesisDStream.initialPosition().getPosition() == InitialPositionInStream.LATEST); - assert(kinesisDStream.checkpointAppName() == appName); - assert(kinesisDStream.checkpointInterval() == checkpointInterval); - assert(kinesisDStream._storageLevel() == storageLevel); + Assert.assertEquals(streamName, kinesisDStream.streamName()); + Assert.assertEquals(endpointUrl, kinesisDStream.endpointUrl()); + Assert.assertEquals(region, kinesisDStream.regionName()); + Assert.assertEquals(InitialPositionInStream.LATEST, + kinesisDStream.initialPosition().getPosition()); + Assert.assertEquals(appName, kinesisDStream.checkpointAppName()); + Assert.assertEquals(checkpointInterval, kinesisDStream.checkpointInterval()); + Assert.assertEquals(storageLevel, kinesisDStream._storageLevel()); ssc.stop(); } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala index e0c65e6940f66..e3471759b3a70 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.graphx.util import org.apache.hadoop.fs.Path +import org.scalatest.Assertions import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.graphx.{Edge, Graph, LocalSparkContext} @@ -88,7 +89,7 @@ class PeriodicGraphCheckpointerSuite extends SparkFunSuite with LocalSparkContex } } -private object PeriodicGraphCheckpointerSuite { +private object PeriodicGraphCheckpointerSuite extends Assertions { private val defaultStorageLevel = StorageLevel.MEMORY_ONLY_SER case class GraphToCheck(graph: Graph[Double, Double], gIndex: Int) diff --git a/mllib/src/test/java/org/apache/spark/ml/stat/JavaKolmogorovSmirnovTestSuite.java b/mllib/src/test/java/org/apache/spark/ml/stat/JavaKolmogorovSmirnovTestSuite.java index 830f668fe07b8..9037f6b854724 100644 --- a/mllib/src/test/java/org/apache/spark/ml/stat/JavaKolmogorovSmirnovTestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/stat/JavaKolmogorovSmirnovTestSuite.java @@ -23,6 +23,7 @@ import org.apache.commons.math3.distribution.NormalDistribution; import org.apache.spark.sql.Encoders; +import org.junit.Assert; import org.junit.Test; import org.apache.spark.SharedSparkSession; @@ -60,7 +61,7 @@ public void testKSTestCDF() { .test(dataset, "sample", stdNormalCDF).head(); double pValue1 = results.getDouble(0); // Cannot reject null hypothesis - assert(pValue1 > pThreshold); + Assert.assertTrue(pValue1 > pThreshold); } @Test @@ -72,6 +73,6 @@ public void testKSTestNamedDistribution() { .test(dataset, "sample", "norm", 0.0, 1.0).head(); double pValue1 = results.getDouble(0); // Cannot reject null hypothesis - assert(pValue1 > pThreshold); + Assert.assertTrue(pValue1 > pThreshold); } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala index cb9b8f9b6b472..dc38f17d296f2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.classification import scala.util.Random import breeze.linalg.{DenseVector => BDV} +import org.scalatest.Assertions._ import org.apache.spark.ml.classification.LinearSVCSuite._ import org.apache.spark.ml.feature.{Instance, LabeledPoint} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 07116606dfb52..60c9cce6a4879 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -21,6 +21,8 @@ import scala.collection.JavaConverters._ import scala.util.Random import scala.util.control.Breaks._ +import org.scalatest.Assertions._ + import org.apache.spark.SparkException import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.classification.LogisticRegressionSuite._ diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index b6e8c927403ad..adffd83ab1bd1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.classification +import org.scalatest.Assertions._ + import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.classification.LogisticRegressionSuite._ import org.apache.spark.ml.feature.LabeledPoint diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala index 1c8c9829f18d1..87a8b345a65a3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.classification +import org.scalatest.Assertions._ + import org.apache.spark.SparkFunSuite import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala index db4f56ed60d32..76a4acd798e34 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.feature +import org.scalatest.Assertions._ + import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.util.{MLTestingUtils, SchemaUtils} import org.apache.spark.sql.Dataset diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index d4e9da3c6263e..d96a4da46a630 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import scala.util.Random import scala.util.control.Breaks._ +import org.scalatest.Assertions._ import org.scalatest.Matchers import org.apache.spark.SparkFunSuite diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 8906e52faebe5..321df05e272db 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.clustering import java.util.{ArrayList => JArrayList} import breeze.linalg.{argmax, argtopk, max, DenseMatrix => BDM} +import org.scalatest.Assertions import org.apache.spark.SparkFunSuite import org.apache.spark.graphx.Edge diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala index e04d7b7c327a8..5458a43b4f2c6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.tree import scala.collection.mutable +import org.scalatest.Assertions._ + import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala index 65e595e3cf2bf..5a4bf1dd2d409 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala @@ -28,6 +28,7 @@ import org.apache.mesos.protobuf.ByteString import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{times, verify} +import org.scalatest.Assertions._ import org.apache.spark.deploy.mesos.config.MesosSecretConfig @@ -161,12 +162,14 @@ object Utils { val variableOne = envVars.filter(_.getName == "USER").head assert(variableOne.getSecret.isInitialized) assert(variableOne.getSecret.getType == Secret.Type.VALUE) - assert(variableOne.getSecret.getValue.getData == ByteString.copyFrom("user".getBytes)) + assert(variableOne.getSecret.getValue.getData == + ByteString.copyFrom("user".getBytes)) assert(variableOne.getType == Environment.Variable.Type.SECRET) val variableTwo = envVars.filter(_.getName == "PASSWORD").head assert(variableTwo.getSecret.isInitialized) assert(variableTwo.getSecret.getType == Secret.Type.VALUE) - assert(variableTwo.getSecret.getValue.getData == ByteString.copyFrom("password".getBytes)) + assert(variableTwo.getSecret.getValue.getData == + ByteString.copyFrom("password".getBytes)) assert(variableTwo.getType == Environment.Variable.Type.SECRET) } diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java index d8845e0c838ff..ca2b18b8eed49 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java @@ -19,6 +19,7 @@ import java.util.Locale; +import org.junit.Assert; import org.junit.Test; public class JavaOutputModeSuite { @@ -26,8 +27,8 @@ public class JavaOutputModeSuite { @Test public void testOutputModes() { OutputMode o1 = OutputMode.Append(); - assert(o1.toString().toLowerCase(Locale.ROOT).contains("append")); + Assert.assertTrue(o1.toString().toLowerCase(Locale.ROOT).contains("append")); OutputMode o2 = OutputMode.Complete(); - assert (o2.toString().toLowerCase(Locale.ROOT).contains("complete")); + Assert.assertTrue(o2.toString().toLowerCase(Locale.ROOT).contains("complete")); } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 3dabbca9deeee..e0fa1f2ecb88e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import org.scalatest.Assertions._ + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala index a89937068a87d..d92eb01b69bf0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala @@ -22,6 +22,7 @@ import java.time.{Duration, Instant, LocalDate} import java.util.concurrent.TimeUnit import org.scalacheck.{Arbitrary, Gen} +import org.scalatest.Assertions._ import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 414f9d5834868..e7b2dd2ebc9da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -22,6 +22,8 @@ import java.util import scala.collection.JavaConverters._ import scala.collection.mutable +import org.scalatest.Assertions._ + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.{IdentityTransform, Transform} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java index 68f984ae0c1e3..4b23615275871 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java @@ -82,14 +82,14 @@ private void insertRow(UnsafeRow row) { int recordLength = row.getSizeInBytes(); Object baseObject = dataPage.getBaseObject(); - assert(pageCursor + recordLength <= dataPage.getBaseOffset() + dataPage.size()); + Assert.assertTrue(pageCursor + recordLength <= dataPage.getBaseOffset() + dataPage.size()); long recordAddress = memoryManager.encodePageNumberAndOffset(dataPage, pageCursor); UnsafeAlignedOffset.putSize(baseObject, pageCursor, recordLength); pageCursor += uaoSize; Platform.copyMemory(recordBase, recordOffset, baseObject, pageCursor, recordLength); pageCursor += recordLength; - assert(pos < 2); + Assert.assertTrue(pos < 2); array.set(pos, recordAddress); pos++; } @@ -142,8 +142,8 @@ public void testBinaryComparatorForSingleColumnRow() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) < 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) < 0); } @Test @@ -167,8 +167,8 @@ public void testBinaryComparatorForMultipleColumnRow() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) < 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) < 0); } @Test @@ -194,8 +194,8 @@ public void testBinaryComparatorForArrayColumn() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) > 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) > 0); } @Test @@ -227,8 +227,8 @@ public void testBinaryComparatorForMixedColumns() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) > 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) > 0); } @Test @@ -253,8 +253,8 @@ public void testBinaryComparatorForNullColumns() throws Exception { insertRow(row1); insertRow(row2); - assert(compare(0, 0) == 0); - assert(compare(0, 1) > 0); + Assert.assertEquals(0, compare(0, 0)); + Assert.assertTrue(compare(0, 1) > 0); } @Test @@ -274,7 +274,7 @@ public void testBinaryComparatorWhenSubtractionIsDivisibleByMaxIntValue() throws insertRow(row1); insertRow(row2); - assert(compare(0, 1) > 0); + Assert.assertTrue(compare(0, 1) > 0); } @Test @@ -294,7 +294,7 @@ public void testBinaryComparatorWhenSubtractionCanOverflowLongValue() throws Exc insertRow(row1); insertRow(row2); - assert(compare(0, 1) < 0); + Assert.assertTrue(compare(0, 1) < 0); } @Test @@ -320,7 +320,7 @@ public void testBinaryComparatorWhenOnlyTheLastColumnDiffers() throws Exception insertRow(row1); insertRow(row2); - assert(compare(0, 1) < 0); + Assert.assertTrue(compare(0, 1) < 0); } @Test diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 9c50e374f74de..fc6f087efaefd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.io.{Externalizable, ObjectInput, ObjectOutput} import java.sql.{Date, Timestamp} +import org.scalatest.Assertions._ import org.scalatest.exceptions.TestFailedException import org.scalatest.prop.TableDrivenPropertyChecks._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala index 3f6fe6eb62020..d39019bcda9a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -22,6 +22,8 @@ import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ import scala.util.Try +import org.scalatest.Assertions._ + import org.apache.spark.TestUtils import org.apache.spark.api.python.{PythonBroadcast, PythonEvalType, PythonFunction, PythonUtils} import org.apache.spark.broadcast.Broadcast diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala index ebe278bff7d86..f3647b3bb2631 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.benchmark +import org.scalatest.Assertions._ + import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED @@ -71,7 +73,7 @@ object HashedRelationMetricsBenchmark extends SqlBasedBenchmark { thread.start() thread } - threads.map(_.join()) + threads.foreach(_.join()) map.free() } benchmark.run() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala index ad81711a13947..f4786368bd9e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.benchmark +import org.scalatest.Assertions._ + import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala index f4642e7d353e6..683d398faeeab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.benchmark import java.io.File +import org.scalatest.Assertions._ + import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.DataFrame import org.apache.spark.util.Utils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala index 8ea20f28a37b2..fcb18392235c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala @@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets import org.apache.commons.lang3.RandomStringUtils import org.apache.commons.math3.distribution.LogNormalDistribution +import org.scalatest.Assertions._ import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.sql.catalyst.expressions.GenericInternalRow diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index c0238069afcc4..42213b9a81882 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.internal +import org.scalatest.Assertions._ + import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 3399a2d925925..8056deefa07c9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -24,6 +24,7 @@ import scala.util.Properties import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.fs.Path import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.Assertions._ import org.apache.spark._ import org.apache.spark.internal.Logging @@ -806,14 +807,14 @@ object SPARK_18360 { // Hive will use the value of `hive.metastore.warehouse.dir` to generate default table // location for tables in default database. assert(rawTable.storage.locationUri.map( - CatalogUtils.URIToString(_)).get.contains(newWarehousePath)) + CatalogUtils.URIToString).get.contains(newWarehousePath)) hiveClient.dropTable("default", "test_tbl", ignoreIfNotExists = false, purge = false) spark.sharedState.externalCatalog.createTable(tableMeta, ignoreIfExists = false) val readBack = spark.sharedState.externalCatalog.getTable("default", "test_tbl") // Spark SQL will use the location of default database to generate default table // location for tables in default database. - assert(readBack.storage.locationUri.map(CatalogUtils.URIToString(_)) + assert(readBack.storage.locationUri.map(CatalogUtils.URIToString) .get.contains(defaultDbLocation)) } finally { hiveClient.dropTable("default", "test_tbl", ignoreIfNotExists = true, purge = false) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index ed3b376f6eda1..80a50c18bcb93 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.scalatest.Assertions._ import org.scalatest.BeforeAndAfterEach import org.scalatest.exceptions.TestFailedException diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala index ce40cf51746b2..97eab4f3f4f77 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.security import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration +import org.scalatest.Assertions._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.security.HadoopDelegationTokenManager diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 2615e7c3bc509..d0a5ababc7cac 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -30,6 +30,7 @@ import scala.util.Random import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.scalatest.Assertions._ import org.apache.spark.internal.Logging import org.apache.spark.streaming.dstream.DStream diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 8e9481b5a99d3..bb60d6fa7bf78 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -34,6 +34,7 @@ import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} import org.mockito.Mockito.{times, verify, when} import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach, PrivateMethodTester} +import org.scalatest.Assertions._ import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ import org.scalatestplus.mockito.MockitoSugar From 06e203b85682b63ee250b96520558fc79aae0a17 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Wed, 20 Nov 2019 18:19:30 -0600 Subject: [PATCH 0678/1387] [SPARK-29911][SQL] Uncache cached tables when session closed ### What changes were proposed in this pull request? The local temporary view is session-scoped. Its lifetime is the lifetime of the session that created it. But now cache data is cross-session. Its lifetime is the lifetime of the Spark application. That's will cause the memory leak if cache a local temporary view in memory when the session closed. In this PR, we uncache the cached data of local temporary view when session closed. This PR doesn't impact the cached data of global temp view and persisted view. How to reproduce: 1. create a local temporary view v1 2. cache it in memory 3. close session without drop table v1. The application will hold the memory forever. In a long running thrift server scenario. It's worse. ```shell 0: jdbc:hive2://localhost:10000> CACHE TABLE testCacheTable AS SELECT 1; CACHE TABLE testCacheTable AS SELECT 1; +---------+--+ | Result | +---------+--+ +---------+--+ No rows selected (1.498 seconds) 0: jdbc:hive2://localhost:10000> !close !close Closing: 0: jdbc:hive2://localhost:10000 0: jdbc:hive2://localhost:10000 (closed)> !connect 'jdbc:hive2://localhost:10000' !connect 'jdbc:hive2://localhost:10000' Connecting to jdbc:hive2://localhost:10000 Enter username for jdbc:hive2://localhost:10000: lajin Enter password for jdbc:hive2://localhost:10000: *** Connected to: Spark SQL (version 3.0.0-SNAPSHOT) Driver: Hive JDBC (version 1.2.1.spark2) Transaction isolation: TRANSACTION_REPEATABLE_READ 1: jdbc:hive2://localhost:10000> select * from testCacheTable; select * from testCacheTable; Error: Error running query: org.apache.spark.sql.AnalysisException: Table or view not found: testCacheTable; line 1 pos 14; 'Project [*] +- 'UnresolvedRelation [testCacheTable] (state=,code=0) ``` Screen Shot 2019-11-15 at 2 03 49 PM ### Why are the changes needed? Resolve memory leak for thrift server ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual test in UI storage tab And add an UT Closes #26543 from LantaoJin/SPARK-29911. Authored-by: LantaoJin Signed-off-by: Sean Owen --- .../sql/catalyst/catalog/SessionCatalog.scala | 4 ++++ .../thriftserver/SparkSQLSessionManager.scala | 2 ++ .../ThriftServerQueryTestSuite.scala | 23 +++++++++++++++++++ 3 files changed, 29 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e72352714a697..e77f3c70f0bdf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -575,6 +575,10 @@ class SessionCatalog( tempViews.get(formatTableName(name)) } + def getTempViewNames(): Seq[String] = synchronized { + tempViews.keySet.toSeq + } + /** * Return a global temporary view exactly as it was stored. */ diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index c4248bfde38cc..41b324d70c315 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -75,6 +75,8 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: override def closeSession(sessionHandle: SessionHandle): Unit = { HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) + val ctx = sparkSqlOperationManager.sessionToContexts.getOrDefault(sessionHandle, sqlContext) + ctx.sparkSession.sessionState.catalog.getTempViewNames().foreach(ctx.uncacheTable) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToActivePool.remove(sessionHandle) sparkSqlOperationManager.sessionToContexts.remove(sessionHandle) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 0c56cf9ef5d50..dc1609d3326cd 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -269,6 +269,29 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } + test("SPARK-29911: Uncache cached tables when session closed") { + val cacheManager = spark.sharedState.cacheManager + val globalTempDB = spark.sharedState.globalTempViewManager.database + withJdbcStatement { statement => + statement.execute("CACHE TABLE tempTbl AS SELECT 1") + } + // the cached data of local temporary view should be uncached + assert(cacheManager.isEmpty) + try { + withJdbcStatement { statement => + statement.execute("CREATE GLOBAL TEMP VIEW globalTempTbl AS SELECT 1, 2") + statement.execute(s"CACHE TABLE $globalTempDB.globalTempTbl") + } + // the cached data of global temporary view shouldn't be uncached + assert(!cacheManager.isEmpty) + } finally { + withJdbcStatement { statement => + statement.execute(s"UNCACHE TABLE IF EXISTS $globalTempDB.globalTempTbl") + } + assert(cacheManager.isEmpty) + } + } + /** ThriftServer wraps the root exception, so it needs to be extracted. */ override def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { super.handleExceptions { From e6b157cf704544a6217a9f9d191f542518581040 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 21 Nov 2019 09:59:31 +0900 Subject: [PATCH 0679/1387] [SPARK-29978][SQL][TESTS] Check `json_tuple` does not truncate results ### What changes were proposed in this pull request? I propose to add a test from the commit https://github.com/apache/spark/commit/a9365221133caadffbbbbce1aae1ace799a588a3 for 2.4. I extended the test by a few more lengths of requested field to cover more code branches in Jackson Core. In particular, [the optimization](https://github.com/apache/spark/blob/5eb8973f871fef557fb4ca3f494406ed676a431a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala#L473-L476) calls Jackson's method https://github.com/FasterXML/jackson-core/blob/42b8b566845e8f8d77537f51187a439029ed9bff/src/main/java/com/fasterxml/jackson/core/json/UTF8JsonGenerator.java#L742-L746 where the internal buffer size is **8000**. In this way: - 2000 to check 2000+2000+2000 < 8000 - 2800 from the 2.4 commit. It covers the specific case: https://github.com/FasterXML/jackson-core/blob/42b8b566845e8f8d77537f51187a439029ed9bff/src/main/java/com/fasterxml/jackson/core/json/UTF8JsonGenerator.java#L746 - 8000-1, 8000, 8000+1 are sizes around the size of the internal buffer - 65535 to test an outstanding large field. ### Why are the changes needed? To be sure that the current implementation and future versions of Spark don't have the bug fixed in 2.4. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `JsonFunctionsSuite`. Closes #26613 from MaxGekk/json_tuple-test. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/JsonFunctionsSuite.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index f3483151d26da..4059079d3f288 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -644,4 +644,15 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { to_json(struct($"t"), Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss.SSSSSS"))) checkAnswer(df, Row(s"""{"t":"$s"}""")) } + + test("json_tuple - do not truncate results") { + Seq(2000, 2800, 8000 - 1, 8000, 8000 + 1, 65535).foreach { len => + val str = Array.tabulate(len)(_ => "a").mkString + val json_tuple_result = Seq(s"""{"test":"$str"}""").toDF("json") + .withColumn("result", json_tuple('json, "test")) + .select('result) + .as[String].head.length + assert(json_tuple_result === len) + } + } } From 7a706703455d218a874cafe114ac8d720700a802 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 21 Nov 2019 10:28:32 +0900 Subject: [PATCH 0680/1387] [SPARK-29961][SQL] Implement builtin function - typeof ### What changes were proposed in this pull request? Add typeof function for Spark to get the underlying type of value. ```sql -- !query 0 select typeof(1) -- !query 0 schema struct -- !query 0 output int -- !query 1 select typeof(1.2) -- !query 1 schema struct -- !query 1 output decimal(2,1) -- !query 2 select typeof(array(1, 2)) -- !query 2 schema struct -- !query 2 output array -- !query 3 select typeof(a) from (values (1), (2), (3.1)) t(a) -- !query 3 schema struct -- !query 3 output decimal(11,1) decimal(11,1) decimal(11,1) ``` ##### presto ```sql presto> select typeof(array[1]); _col0 ---------------- array(integer) (1 row) ``` ##### PostgreSQL ```sql postgres=# select pg_typeof(a) from (values (1), (2), (3.0)) t(a); pg_typeof ----------- numeric numeric numeric (3 rows) ``` ##### impala https://issues.apache.org/jira/browse/IMPALA-1597 ### Why are the changes needed? a function which is better we have to help us debug, test, develop ... ### Does this PR introduce any user-facing change? add a new function ### How was this patch tested? add ut and example Closes #26599 from yaooqinn/SPARK-29961. Authored-by: Kent Yao Signed-off-by: HyukjinKwon --- .../spark/ml/linalg/VectorUDTSuite.scala | 14 ++++- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../spark/sql/catalyst/expressions/misc.scala | 21 +++++++ .../sql-tests/inputs/misc-functions.sql | 10 ++++ .../sql-tests/results/misc-functions.sql.out | 58 +++++++++++++++++++ .../spark/sql/UserDefinedTypeSuite.scala | 10 ++++ 6 files changed, 112 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 67c64f762b25e..3bb47755b2703 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.ml.linalg -import org.apache.spark.SparkFunSuite +import java.util.Arrays + import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.JavaTypeInference +import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class VectorUDTSuite extends SparkFunSuite { +class VectorUDTSuite extends QueryTest with SharedSparkSession { test("preloaded VectorUDT") { val dv1 = Vectors.dense(Array.empty[Double]) @@ -44,4 +47,11 @@ class VectorUDTSuite extends SparkFunSuite { assert(dataType.asInstanceOf[StructType].fields.map(_.dataType) === Seq(new VectorUDT, DoubleType)) } + + test("typeof Vector") { + val data = Arrays.asList(Row(Vectors.dense(1.0, 2.0))) + val schema = new StructType().add("v", new VectorUDT) + checkAnswer(spark.createDataFrame(data, schema).selectExpr("typeof(v)"), + Seq(Row("struct,values:array>"))) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index cb18aa1a9479b..c0e0330ff1e14 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -491,6 +491,7 @@ object FunctionRegistry { expression[CallMethodViaReflection]("reflect"), expression[CallMethodViaReflection]("java_method"), expression[Version]("version"), + expression[TypeOf]("typeof"), // grouping sets expression[Cube]("cube"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index b8c23a1f08912..def81f8dfb72b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -177,3 +177,24 @@ case class Version() extends LeafExpression with CodegenFallback { UTF8String.fromString(SPARK_VERSION_SHORT + " " + SPARK_REVISION) } } + +@ExpressionDescription( + usage = """_FUNC_(expr) - Return DDL-formatted type string for the data type of the input.""", + examples = """ + Examples: + > SELECT _FUNC_(1); + int + > SELECT _FUNC_(array(1)); + array + """, + since = "3.0.0") +case class TypeOf(child: Expression) extends UnaryExpression { + override def nullable: Boolean = false + override def foldable: Boolean = true + override def dataType: DataType = StringType + override def eval(input: InternalRow): Any = UTF8String.fromString(child.dataType.catalogString) + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, _ => s"""UTF8String.fromString(${child.dataType.catalogString})""") + } +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql new file mode 100644 index 0000000000000..95f71925e9294 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/misc-functions.sql @@ -0,0 +1,10 @@ +-- test for misc functions + +-- typeof +select typeof(null); +select typeof(true); +select typeof(1Y), typeof(1S), typeof(1), typeof(1L); +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2); +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days'); +select typeof(x'ABCD'), typeof('SPARK'); +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')); diff --git a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out new file mode 100644 index 0000000000000..cd0818a5189b5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out @@ -0,0 +1,58 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 7 + + +-- !query 0 +select typeof(null) +-- !query 0 schema +struct +-- !query 0 output +null + + +-- !query 1 +select typeof(true) +-- !query 1 schema +struct +-- !query 1 output +boolean + + +-- !query 2 +select typeof(1Y), typeof(1S), typeof(1), typeof(1L) +-- !query 2 schema +struct +-- !query 2 output +tinyint smallint int bigint + + +-- !query 3 +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2) +-- !query 3 schema +struct +-- !query 3 output +float double decimal(2,1) + + +-- !query 4 +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days') +-- !query 4 schema +struct +-- !query 4 output +date timestamp interval + + +-- !query 5 +select typeof(x'ABCD'), typeof('SPARK') +-- !query 5 schema +struct +-- !query 5 output +binary string + + +-- !query 6 +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')) +-- !query 6 schema +struct +-- !query 6 output +array map struct diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 2b2fedd3ca218..ffc2018d2132d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.util.Arrays + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, GenericInternalRow, Literal} @@ -277,4 +279,12 @@ class UserDefinedTypeSuite extends QueryTest with SharedSparkSession with Parque val udt = new TestUDT.MyDenseVectorUDT() assert(!Cast.canUpCast(udt, StringType)) } + + test("typeof user defined type") { + val schema = new StructType().add("a", new TestUDT.MyDenseVectorUDT()) + val data = Arrays.asList( + RowFactory.create(new TestUDT.MyDenseVector(Array(1.0, 3.0, 5.0, 7.0, 9.0)))) + checkAnswer(spark.createDataFrame(data, schema).selectExpr("typeof(a)"), + Seq(Row("array"))) + } } From 74cb1ffd686d67188a4540c74c6111affd6cce90 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 21 Nov 2019 10:54:01 +0900 Subject: [PATCH 0681/1387] [SPARK-22340][PYTHON][FOLLOW-UP] Add a better message and improve documentation for pinned thread mode ### What changes were proposed in this pull request? This PR proposes to show different warning message when the pinned thread mode is enabled: When enabled: > PYSPARK_PIN_THREAD feature is enabled. However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. > To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. When disabled: > Currently, 'setLocalProperty' (set to local properties) with multiple threads does not properly work. > Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties for each thread on PVM. > To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. > To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. ### Why are the changes needed? Currently, it shows the same warning message regardless of PYSPARK_PIN_THREAD being set. In the warning message it says "you can set PYSPARK_PIN_THREAD to true ..." which is confusing. ### Does this PR introduce any user-facing change? Documentation and warning message as shown above. ### How was this patch tested? Manually tested. ```bash $ PYSPARK_PIN_THREAD=true ./bin/pyspark ``` ```python sc.setJobGroup("a", "b") ``` ``` .../pyspark/util.py:141: UserWarning: PYSPARK_PIN_THREAD feature is enabled. However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. warnings.warn(msg, UserWarning) ``` ```bash $ ./bin/pyspark ``` ```python sc.setJobGroup("a", "b") ``` ``` .../pyspark/util.py:141: UserWarning: Currently, 'setJobGroup' (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties for each thread on PVM. To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. warnings.warn(msg, UserWarning) ``` Closes #26588 from HyukjinKwon/SPARK-22340. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/context.py | 88 +++++++++++++-------------------------- python/pyspark/util.py | 29 +++++++++++++ 2 files changed, 59 insertions(+), 58 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a6aa3a65568e9..6cc343e3e495c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -40,6 +40,7 @@ from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler +from pyspark.util import _warn_pin_thread if sys.version > '3': xrange = range @@ -1008,30 +1009,20 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): ensure that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, where HDFS may respond to Thread.interrupt() by marking nodes as dead. - .. note:: Currently, setting a group ID (set to local properties) with a thread does - not properly work. Internally threads on PVM and JVM are not synced, and JVM thread - can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. To work around this, you can set `PYSPARK_PIN_THREAD` to + .. note:: Currently, setting a group ID (set to local properties) with multiple threads + does not properly work. Internally threads on PVM and JVM are not synced, and JVM + thread can be reused for multiple threads on PVM, which fails to isolate local + properties for each thread on PVM. + + To work around this, you can set `PYSPARK_PIN_THREAD` to `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. To work around this, you should manually copy and set the local + local properties. + + To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. """ - warnings.warn( - "Currently, setting a group ID (set to local properties) with a thread does " - "not properly work. " - "\n" - "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " - "for multiple threads on PVM, which fails to isolate local properties for each " - "thread on PVM. " - "\n" - "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread.", - UserWarning) + _warn_pin_thread("setJobGroup") self._jsc.setJobGroup(groupId, description, interruptOnCancel) def setLocalProperty(self, key, value): @@ -1039,29 +1030,20 @@ def setLocalProperty(self, key, value): Set a local property that affects jobs submitted from this thread, such as the Spark fair scheduler pool. - .. note:: Currently, setting a local property with a thread does - not properly work. Internally threads on PVM and JVM are not synced, and JVM thread + .. note:: Currently, setting a local property with multiple threads does not properly work. + Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. To work around this, you can set `PYSPARK_PIN_THREAD` to + for each thread on PVM. + + To work around this, you can set `PYSPARK_PIN_THREAD` to `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. To work around this, you should manually copy and set the local + local properties. + + To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. """ - warnings.warn( - "Currently, setting a local property with a thread does not properly work. " - "\n" - "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " - "for multiple threads on PVM, which fails to isolate local properties for each " - "thread on PVM. " - "\n" - "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread.", - UserWarning) + _warn_pin_thread("setLocalProperty") self._jsc.setLocalProperty(key, value) def getLocalProperty(self, key): @@ -1075,30 +1057,20 @@ def setJobDescription(self, value): """ Set a human readable description of the current job. - .. note:: Currently, setting a job description (set to local properties) with a thread does - not properly work. Internally threads on PVM and JVM are not synced, and JVM thread - can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. To work around this, you can set `PYSPARK_PIN_THREAD` to + .. note:: Currently, setting a job description (set to local properties) with multiple + threads does not properly work. Internally threads on PVM and JVM are not synced, + and JVM thread can be reused for multiple threads on PVM, which fails to isolate + local properties for each thread on PVM. + + To work around this, you can set `PYSPARK_PIN_THREAD` to `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. To work around this, you should manually copy and set the local + local properties. + + To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. """ - warnings.warn( - "Currently, setting a job description (set to local properties) with a thread does " - "not properly work. " - "\n" - "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " - "for multiple threads on PVM, which fails to isolate local properties for each " - "thread on PVM. " - "\n" - "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread.", - UserWarning) + _warn_pin_thread("setJobDescription") self._jsc.setJobDescription(value) def sparkUser(self): diff --git a/python/pyspark/util.py b/python/pyspark/util.py index ad4cf7c9b1c8b..93137560de25e 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -19,6 +19,8 @@ import re import sys import traceback +import os +import warnings import inspect from py4j.protocol import Py4JJavaError @@ -112,6 +114,33 @@ def wrapper(*args, **kwargs): return wrapper +def _warn_pin_thread(name): + if os.environ.get("PYSPARK_PIN_THREAD", "false").lower() == "true": + msg = ( + "PYSPARK_PIN_THREAD feature is enabled. " + "However, note that it cannot inherit the local properties from the parent thread " + "although it isolates each thread on PVM and JVM with its own local properties. " + "\n" + "To work around this, you should manually copy and set the local properties from " + "the parent thread to the child thread when you create another thread.") + else: + msg = ( + "Currently, '%s' (set to local properties) with multiple threads does " + "not properly work. " + "\n" + "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " + "for multiple threads on PVM, which fails to isolate local properties for each " + "thread on PVM. " + "\n" + "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " + "However, note that it cannot inherit the local properties from the parent thread " + "although it isolates each thread on PVM and JVM with its own local properties. " + "\n" + "To work around this, you should manually copy and set the local properties from " + "the parent thread to the child thread when you create another thread." % name) + warnings.warn(msg, UserWarning) + + def _print_missing_jar(lib_name, pkg_name, jar_name, spark_version): print(""" ________________________________________________________________________________________________ From d555f8fcc964e6e81f2d849de62d06877dca70c5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 20 Nov 2019 21:02:22 -0800 Subject: [PATCH 0682/1387] [SPARK-29961][SQL][FOLLOWUP] Remove useless test for VectorUDT ### What changes were proposed in this pull request? A follow-up to rm useless test in VectorUDTSuite ### Why are the changes needed? rm useless test, which is already covered. ### Does this PR introduce any user-facing change? no ### How was this patch tested? no Closes #26620 from yaooqinn/SPARK-29961-f. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../apache/spark/ml/linalg/VectorUDTSuite.scala | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 3bb47755b2703..67c64f762b25e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -17,15 +17,12 @@ package org.apache.spark.ml.linalg -import java.util.Arrays - +import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.LabeledPoint -import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.JavaTypeInference -import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class VectorUDTSuite extends QueryTest with SharedSparkSession { +class VectorUDTSuite extends SparkFunSuite { test("preloaded VectorUDT") { val dv1 = Vectors.dense(Array.empty[Double]) @@ -47,11 +44,4 @@ class VectorUDTSuite extends QueryTest with SharedSparkSession { assert(dataType.asInstanceOf[StructType].fields.map(_.dataType) === Seq(new VectorUDT, DoubleType)) } - - test("typeof Vector") { - val data = Arrays.asList(Row(Vectors.dense(1.0, 2.0))) - val schema = new StructType().add("v", new VectorUDT) - checkAnswer(spark.createDataFrame(data, schema).selectExpr("typeof(v)"), - Seq(Row("struct,values:array>"))) - } } From affaefe1f3e623e6ae2042648ab78ee7b89d1ed6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 21 Nov 2019 15:43:57 +0900 Subject: [PATCH 0683/1387] [MINOR][INFRA] Add `io` and `net` to GitHub Action Cache ### What changes were proposed in this pull request? This PR aims to cache `~/.m2/repository/net` and `~/.m2/repository/io` to reduce the flakiness. ### Why are the changes needed? This will stabilize GitHub Action more before adding `hive-1.2` and `hive-2.3` combination. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? After the GitHub Action on this PR passes, check the log. Closes #26621 from dongjoon-hyun/SPARK-GHA-CACHE. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- .github/workflows/master.yml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 2cd48153b190c..5298492d219f8 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -36,6 +36,18 @@ jobs: key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-org-${{ hashFiles('**/pom.xml') }} restore-keys: | ${{ matrix.java }}-${{ matrix.hadoop }}-maven-org- + - uses: actions/cache@v1 + with: + path: ~/.m2/repository/net + key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-net-${{ hashFiles('**/pom.xml') }} + restore-keys: | + ${{ matrix.java }}-${{ matrix.hadoop }}-maven-net- + - uses: actions/cache@v1 + with: + path: ~/.m2/repository/io + key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-io-${{ hashFiles('**/pom.xml') }} + restore-keys: | + ${{ matrix.java }}-${{ matrix.hadoop }}-maven-io- - name: Set up JDK ${{ matrix.java }} uses: actions/setup-java@v1 with: From 85c004d5b0303435dc207e139cdc51f0f2d3e160 Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Thu, 21 Nov 2019 16:13:42 +0900 Subject: [PATCH 0684/1387] [SPARK-29885][PYTHON][CORE] Improve the exception message when reading the daemon port ### What changes were proposed in this pull request? In production environment, my PySpark application occurs an exception and it's message as below: ``` 19/10/28 16:15:03 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) org.apache.spark.SparkException: No port number in pyspark.daemon's stdout at org.apache.spark.api.python.PythonWorkerFactory.startDaemon(PythonWorkerFactory.scala:204) at org.apache.spark.api.python.PythonWorkerFactory.createThroughDaemon(PythonWorkerFactory.scala:122) at org.apache.spark.api.python.PythonWorkerFactory.create(PythonWorkerFactory.scala:95) at org.apache.spark.SparkEnv.createPythonWorker(SparkEnv.scala:117) at org.apache.spark.api.python.BasePythonRunner.compute(PythonRunner.scala:108) at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337) at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:121) at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` At first, I think a physical node has many ports are occupied by a large number of processes. But I found the total number of ports in use is only 671. ``` [yarnr1115 ~]$ netstat -a | wc -l 671 671 ``` I checked the code of PythonWorkerFactory in line 204 and found: ``` daemon = pb.start() val in = new DataInputStream(daemon.getInputStream) try { daemonPort = in.readInt() } catch { case _: EOFException => throw new SparkException(s"No port number in $daemonModule's stdout") } ``` I added some code here: ``` logError("Meet EOFException, daemon is alive: ${daemon.isAlive()}") logError("Exit value: ${daemon.exitValue()}") ``` Then I recurrent the exception and it's message as below: ``` 19/10/28 16:15:03 ERROR PythonWorkerFactory: Meet EOFException, daemon is alive: false 19/10/28 16:15:03 ERROR PythonWorkerFactory: Exit value: 139 19/10/28 16:15:03 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) org.apache.spark.SparkException: No port number in pyspark.daemon's stdout at org.apache.spark.api.python.PythonWorkerFactory.startDaemon(PythonWorkerFactory.scala:206) at org.apache.spark.api.python.PythonWorkerFactory.createThroughDaemon(PythonWorkerFactory.scala:122) at org.apache.spark.api.python.PythonWorkerFactory.create(PythonWorkerFactory.scala:95) at org.apache.spark.SparkEnv.createPythonWorker(SparkEnv.scala:117) at org.apache.spark.api.python.BasePythonRunner.compute(PythonRunner.scala:108) at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:337) at org.apache.spark.rdd.RDD$$anonfun$7.apply(RDD.scala:335) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1182) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:1091) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:1156) at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:882) at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:335) at org.apache.spark.rdd.RDD.iterator(RDD.scala:286) at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324) at org.apache.spark.rdd.RDD.iterator(RDD.scala:288) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:121) at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` I think the exception message has caused me a lot of confusion. This PR will add meaningful log for exception information. ### Why are the changes needed? In order to clarify the exception and try three times default. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Exists UT. Closes #26510 from beliefer/improve-except-message. Authored-by: gengjiaan Signed-off-by: HyukjinKwon --- .../org/apache/spark/api/python/PythonWorkerFactory.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 1926a5268227c..df236ba8926c1 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -212,8 +212,13 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { daemonPort = in.readInt() } catch { + case _: EOFException if daemon.isAlive => + throw new SparkException("EOFException occurred while reading the port number " + + s"from $daemonModule's stdout") case _: EOFException => - throw new SparkException(s"No port number in $daemonModule's stdout") + throw new SparkException( + s"EOFException occurred while reading the port number from $daemonModule's" + + s" stdout and terminated with code: ${daemon.exitValue}.") } // test that the returned port number is within a valid range. From 297cbab98e68fed08cd790c24020b3df416e1e03 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 21 Nov 2019 18:22:05 +0800 Subject: [PATCH 0685/1387] [SPARK-29942][ML] Impl Complement Naive Bayes Classifier ### What changes were proposed in this pull request? Impl Complement Naive Bayes Classifier as a `modelType` option in `NaiveBayes` ### Why are the changes needed? 1, it is a better choice for text classification: it is said in [scikit-learn](https://scikit-learn.org/stable/modules/naive_bayes.html#complement-naive-bayes) that 'CNB regularly outperforms MNB (often by a considerable margin) on text classification tasks.' 2, CNB is highly similar to existing MNB, only a small part of existing MNB need to be changed, so it is a easy win to support CNB. ### Does this PR introduce any user-facing change? yes, a new `modelType` is supported ### How was this patch tested? added testsuites Closes #26575 from zhengruifeng/cnb. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- docs/ml-classification-regression.md | 7 +- .../spark/ml/classification/NaiveBayes.scala | 96 +++++++++++++++---- .../ml/classification/NaiveBayesSuite.scala | 77 +++++++++++++-- python/pyspark/ml/classification.py | 13 +++ 4 files changed, 162 insertions(+), 31 deletions(-) diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index d8c7d8a729624..05c688960f04c 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -479,16 +479,17 @@ For prediction, it applies Bayes' theorem to compute the conditional probability of each label given an observation. MLlib supports [Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), +[Complement naive Bayes](https://people.csail.mit.edu/jrennie/papers/icml03-nb.pdf), [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html) and [Gaussian naive Bayes](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Gaussian_naive_Bayes). *Input data*: -These Multinomial and Bernoulli models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +These Multinomial, Complement and Bernoulli models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each feature represents a term. -A feature's value is the frequency of the term (in multinomial Naive Bayes) or +A feature's value is the frequency of the term (in Multinomial or Complement Naive Bayes) or a zero or one indicating whether the term was found in the document (in Bernoulli Naive Bayes). Feature values for Multinomial and Bernoulli models must be *non-negative*. The model type is selected with an optional parameter -"multinomial" or "bernoulli" with "multinomial" as the default. +"multinomial", "complement", "bernoulli" or "gaussian", with "multinomial" as the default. For document classification, the input feature vectors should usually be sparse vectors. Since the training data is only used once, it is not necessary to cache it. diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 8062870794418..52a0f4d9b9828 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -53,13 +53,13 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW /** * The model type which is a string (case-sensitive). - * Supported options: "multinomial", "bernoulli", "gaussian". + * Supported options: "multinomial", "complement", "bernoulli", "gaussian". * (default = multinomial) * @group param */ final val modelType: Param[String] = new Param[String](this, "modelType", "The model type " + - "which is a string (case-sensitive). Supported options: multinomial (default), bernoulli" + - " and gaussian.", + "which is a string (case-sensitive). Supported options: multinomial (default), complement, " + + "bernoulli and gaussian.", ParamValidators.inArray[String](NaiveBayes.supportedModelTypes.toArray)) /** @group getParam */ @@ -78,6 +78,11 @@ private[classification] trait NaiveBayesParams extends PredictorParams with HasW * (see * here). * The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + * Since 3.0.0, it supports Complement NB which is an adaptation of the Multinomial NB. Specifically, + * Complement NB uses statistics from the complement of each class to compute the model's coefficients + * The inventors of Complement NB show empirically that the parameter estimates for CNB are more stable + * than those for Multinomial NB. Like Multinomial NB, the input feature values for Complement NB must + * be nonnegative. * Since 3.0.0, it also supports Gaussian NB * (see * here) @@ -106,7 +111,7 @@ class NaiveBayes @Since("1.5.0") ( /** * Set the model type using a string (case-sensitive). - * Supported options: "multinomial" and "bernoulli". + * Supported options: "multinomial", "complement", "bernoulli", and "gaussian". * Default is "multinomial" * @group setParam */ @@ -151,7 +156,7 @@ class NaiveBayes @Since("1.5.0") ( } $(modelType) match { - case Bernoulli | Multinomial => + case Bernoulli | Multinomial | Complement => trainDiscreteImpl(dataset, instr) case Gaussian => trainGaussianImpl(dataset, instr) @@ -168,7 +173,7 @@ class NaiveBayes @Since("1.5.0") ( import spark.implicits._ val validateUDF = $(modelType) match { - case Multinomial => + case Multinomial | Complement => udf { vector: Vector => requireNonnegativeValues(vector); vector } case Bernoulli => udf { vector: Vector => requireZeroOneBernoulliValues(vector); vector } @@ -204,14 +209,29 @@ class NaiveBayes @Since("1.5.0") ( val piArray = new Array[Double](numLabels) val thetaArray = new Array[Double](numLabels * numFeatures) + val aggIter = $(modelType) match { + case Multinomial | Bernoulli => aggregated.iterator + case Complement => + val featureSum = Vectors.zeros(numFeatures) + aggregated.foreach { case (_, _, sumTermFreqs, _) => + BLAS.axpy(1.0, sumTermFreqs, featureSum) + } + aggregated.iterator.map { case (label, n, sumTermFreqs, count) => + val comp = featureSum.copy + BLAS.axpy(-1.0, sumTermFreqs, comp) + (label, n, comp, count) + } + } + val lambda = $(smoothing) val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 - aggregated.foreach { case (label, n, sumTermFreqs, _) => + aggIter.foreach { case (label, n, sumTermFreqs, _) => labelArray(i) = label piArray(i) = math.log(n + lambda) - piLogDenom val thetaLogDenom = $(modelType) match { - case Multinomial => math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) + case Multinomial | Complement => + math.log(sumTermFreqs.toArray.sum + numFeatures * lambda) case Bernoulli => math.log(n + 2.0 * lambda) } var j = 0 @@ -224,9 +244,16 @@ class NaiveBayes @Since("1.5.0") ( } val pi = Vectors.dense(piArray) - val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) - .setOldLabels(labelArray) + $(modelType) match { + case Multinomial | Bernoulli => + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + .setOldLabels(labelArray) + case Complement => + // Since the CNB compute the coefficient in a complement way. + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray.map(v => -v), true) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + } } private def trainGaussianImpl( @@ -322,10 +349,14 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { /** String name for Gaussian model type. */ private[classification] val Gaussian: String = "gaussian" + /** String name for Complement model type. */ + private[classification] val Complement: String = "complement" + /* Set of modelTypes that NaiveBayes supports */ - private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli, Gaussian) + private[classification] val supportedModelTypes = + Set(Multinomial, Bernoulli, Gaussian, Complement) - private[NaiveBayes] def requireNonnegativeValues(v: Vector): Unit = { + private[ml] def requireNonnegativeValues(v: Vector): Unit = { val values = v match { case sv: SparseVector => sv.values case dv: DenseVector => dv.values @@ -335,7 +366,7 @@ object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { s"Naive Bayes requires nonnegative feature values but found $v.") } - private[NaiveBayes] def requireZeroOneBernoulliValues(v: Vector): Unit = { + private[ml] def requireZeroOneBernoulliValues(v: Vector): Unit = { val values = v match { case sv: SparseVector => sv.values case dv: DenseVector => dv.values @@ -368,7 +399,7 @@ class NaiveBayesModel private[ml] ( extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] with NaiveBayesParams with MLWritable { - import NaiveBayes.{Bernoulli, Multinomial, Gaussian} + import NaiveBayes._ /** * mllib NaiveBayes is a wrapper of ml implementation currently. @@ -427,16 +458,39 @@ class NaiveBayesModel private[ml] ( override val numClasses: Int = pi.size private def multinomialCalculation(features: Vector) = { + requireNonnegativeValues(features) val prob = theta.multiply(features) BLAS.axpy(1.0, pi, prob) prob } + private def complementCalculation(features: Vector) = { + requireNonnegativeValues(features) + val probArray = theta.multiply(features).toArray + // the following lines equal to: + // val logSumExp = math.log(probArray.map(math.exp).sum) + // However, it easily returns Infinity/NaN values. + // Here follows 'scipy.special.logsumexp' (which is used in Scikit-Learn's ComplementNB) + // to compute the log of the sum of exponentials of elements in a numeric-stable way. + val max = probArray.max + var sumExp = 0.0 + var j = 0 + while (j < probArray.length) { + sumExp += math.exp(probArray(j) - max) + j += 1 + } + val logSumExp = math.log(sumExp) + max + + j = 0 + while (j < probArray.length) { + probArray(j) = probArray(j) - logSumExp + j += 1 + } + Vectors.dense(probArray) + } + private def bernoulliCalculation(features: Vector) = { - features.foreachActive((_, value) => - require(value == 0.0 || value == 1.0, - s"Bernoulli naive Bayes requires 0 or 1 feature values but found $features.") - ) + requireZeroOneBernoulliValues(features) val prob = thetaMinusNegTheta.multiply(features) BLAS.axpy(1.0, pi, prob) BLAS.axpy(1.0, negThetaSum, prob) @@ -464,6 +518,8 @@ class NaiveBayesModel private[ml] ( $(modelType) match { case Multinomial => features: Vector => multinomialCalculation(features) + case Complement => + features: Vector => complementCalculation(features) case Bernoulli => features: Vector => bernoulliCalculation(features) case Gaussian => @@ -533,7 +589,7 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { val dataPath = new Path(path, "data").toString instance.getModelType match { - case Multinomial | Bernoulli => + case Multinomial | Bernoulli | Complement => // Save model data: pi, theta require(instance.sigma == null) val data = Data(instance.pi, instance.theta) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 9e4844ff89079..4a555ad3ed071 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -40,6 +40,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { @transient var bernoulliDataset: Dataset[_] = _ @transient var gaussianDataset: Dataset[_] = _ @transient var gaussianDataset2: Dataset[_] = _ + @transient var complementDataset: Dataset[_] = _ private val seed = 42 @@ -70,8 +71,12 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { Array(0.10, 0.10, 0.10, 0.50) // label 2: variance ) gaussianDataset = generateGaussianNaiveBayesInput(pi, theta2, sigma, 1000, seed).toDF() + gaussianDataset2 = spark.read.format("libsvm") .load("../data/mllib/sample_multiclass_classification_data.txt") + + complementDataset = spark.read.format("libsvm") + .load("../data/mllib/sample_libsvm_data.txt") } def validatePrediction(predictionAndLabels: Seq[Row]): Unit = { @@ -154,6 +159,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { assert(Multinomial === "multinomial") assert(Bernoulli === "bernoulli") assert(Gaussian === "gaussian") + assert(Complement === "complement") } test("params") { @@ -218,8 +224,6 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { Array(0.10, 0.70, 0.10, 0.10), // label 1 Array(0.10, 0.10, 0.70, 0.10) // label 2 ).map(_.map(math.log)) - val pi = Vectors.dense(piArray) - val theta = new DenseMatrix(3, 4, thetaArray.flatten, true) val trainDataset = generateNaiveBayesInput(piArray, thetaArray, nPoints, seed, "multinomial").toDF() @@ -245,6 +249,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val testParams = Seq[(String, Dataset[_])]( ("bernoulli", bernoulliDataset), ("multinomial", dataset), + ("complement", dataset), ("gaussian", gaussianDataset) ) testParams.foreach { case (family, dataset) => @@ -428,19 +433,75 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val thetaRows = model.theta.rowIter.toArray assert(thetaRows(0) ~= - Vectors.dense(0.27111101, -0.18833335, 0.54305072, 0.60500005)relTol 1E-5) + Vectors.dense(0.27111101, -0.18833335, 0.54305072, 0.60500005) relTol 1E-5) assert(thetaRows(1) ~= - Vectors.dense(-0.60777778, 0.18166667, -0.84271174, -0.88000014)relTol 1E-5) + Vectors.dense(-0.60777778, 0.18166667, -0.84271174, -0.88000014) relTol 1E-5) assert(thetaRows(2) ~= - Vectors.dense(-0.09111114, -0.35833336, 0.10508474, 0.0216667)relTol 1E-5) + Vectors.dense(-0.09111114, -0.35833336, 0.10508474, 0.0216667) relTol 1E-5) val sigmaRows = model.sigma.rowIter.toArray assert(sigmaRows(0) ~= - Vectors.dense(0.12230125, 0.07078052, 0.03430001, 0.05133607)relTol 1E-5) + Vectors.dense(0.12230125, 0.07078052, 0.03430001, 0.05133607) relTol 1E-5) assert(sigmaRows(1) ~= - Vectors.dense(0.03758145, 0.0988028, 0.0033903, 0.00782224)relTol 1E-5) + Vectors.dense(0.03758145, 0.0988028, 0.0033903, 0.00782224) relTol 1E-5) assert(sigmaRows(2) ~= - Vectors.dense(0.08058764, 0.06701387, 0.02486641, 0.02661392)relTol 1E-5) + Vectors.dense(0.08058764, 0.06701387, 0.02486641, 0.02661392) relTol 1E-5) + } + + test("Naive Bayes Complement") { + /* + Using the following Python code to verify the correctness. + + import numpy as np + from sklearn.naive_bayes import ComplementNB + from sklearn.datasets import load_svmlight_file + + path = "./data/mllib/sample_libsvm_data.txt" + X, y = load_svmlight_file(path) + X = X.toarray() + clf = ComplementNB() + clf.fit(X, y) + + >>> clf.feature_log_prob_[:, -5:] + array([[ 7.2937608 , 10.26577655, 13.73151245, 13.73151245, 13.73151245], + [ 6.99678043, 7.51387415, 7.74399483, 8.32904552, 9.53119848]]) + >>> clf.predict_log_proba(X[:5]) + array([[ 0. , -74732.70765355], + [-36018.30169185, 0. ], + [-37126.4015229 , 0. ], + [-27649.81038619, 0. ], + [-28767.84075587, 0. ]]) + >>> clf.predict_proba(X[:5]) + array([[1., 0.], + [0., 1.], + [0., 1.], + [0., 1.], + [0., 1.]]) + */ + + val cnb = new NaiveBayes().setModelType(Complement) + val model = cnb.fit(complementDataset) + + val thetaRows = model.theta.rowIter.map(vec => Vectors.dense(vec.toArray.takeRight(5))).toArray + assert(thetaRows(0) ~= + Vectors.dense(7.2937608, 10.26577655, 13.73151245, 13.73151245, 13.73151245) relTol 1E-5) + assert(thetaRows(1) ~= + Vectors.dense(6.99678043, 7.51387415, 7.74399483, 8.32904552, 9.53119848) relTol 1E-5) + + val preds = model.transform(complementDataset) + .select("rawPrediction", "probability") + .as[(Vector, Vector)] + .take(5) + assert(preds(0)._1 ~= Vectors.dense(0.0, -74732.70765355) relTol 1E-5) + assert(preds(0)._2 ~= Vectors.dense(1.0, 0.0) relTol 1E-5) + assert(preds(1)._1 ~= Vectors.dense(-36018.30169185, 0.0) relTol 1E-5) + assert(preds(1)._2 ~= Vectors.dense(0.0, 1.0) relTol 1E-5) + assert(preds(2)._1 ~= Vectors.dense(-37126.4015229, 0.0) relTol 1E-5) + assert(preds(2)._2 ~= Vectors.dense(0.0, 1.0) relTol 1E-5) + assert(preds(3)._1 ~= Vectors.dense(-27649.81038619, 0.0) relTol 1E-5) + assert(preds(3)._2 ~= Vectors.dense(0.0, 1.0) relTol 1E-5) + assert(preds(4)._1 ~= Vectors.dense(-28767.84075587, 0.0) relTol 1E-5) + assert(preds(4)._2 ~= Vectors.dense(0.0, 1.0) relTol 1E-5) } test("read/write") { diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 03087c34b9d0b..d6fe26dc69da8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1909,6 +1909,11 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, binary (0/1) data, it can also be used as `Bernoulli NB `_. The input feature values for Multinomial NB and Bernoulli NB must be nonnegative. + Since 3.0.0, it supports Complement NB which is an adaptation of the Multinomial NB. + Specifically, Complement NB uses statistics from the complement of each class to compute + the model's coefficients. The inventors of Complement NB show empirically that the parameter + estimates for CNB are more stable than those for Multinomial NB. Like Multinomial NB, the + input feature values for Complement NB must be nonnegative. Since 3.0.0, it also supports Gaussian NB `_. which can handle continuous data. @@ -1967,6 +1972,14 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, 'gaussian' >>> model4.sigma DenseMatrix(2, 2, [0.0, 0.25, 0.0, 0.0], 1) + >>> nb5 = NaiveBayes(smoothing=1.0, modelType="complement", weightCol="weight") + >>> model5 = nb5.fit(df) + >>> model5.getModelType() + 'complement' + >>> model5.theta + DenseMatrix(2, 2, [...], 1) + >>> model5.sigma == None + True .. versionadded:: 1.5.0 """ From 0f40d2a6ee0bba3f95785b8c36171e104c12615a Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 21 Nov 2019 18:32:28 +0800 Subject: [PATCH 0686/1387] [SPARK-29960][ML][PYSPARK] MulticlassClassificationEvaluator support hammingLoss ### What changes were proposed in this pull request? MulticlassClassificationEvaluator support hammingLoss ### Why are the changes needed? 1, it is an easy to compute hammingLoss based on confusion matrix 2, scikit-learn supports it ### Does this PR introduce any user-facing change? yes ### How was this patch tested? added testsuites Closes #26597 from zhengruifeng/multi_class_hamming_loss. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../MulticlassClassificationEvaluator.scala | 8 +++-- .../mllib/evaluation/MulticlassMetrics.scala | 17 ++++++++++ .../evaluation/MulticlassMetricsSuite.scala | 31 +++++++++++++++++++ python/pyspark/ml/evaluation.py | 6 +++- 4 files changed, 58 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index ab14227f06be1..435708186242f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -46,7 +46,7 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid * `"weightedPrecision"`, `"weightedRecall"`, `"weightedTruePositiveRate"`, * `"weightedFalsePositiveRate"`, `"weightedFMeasure"`, `"truePositiveRateByLabel"`, * `"falsePositiveRateByLabel"`, `"precisionByLabel"`, `"recallByLabel"`, - * `"fMeasureByLabel"`, `"logLoss"`) + * `"fMeasureByLabel"`, `"logLoss"`, `"hammingLoss"`) * * @group param */ @@ -172,13 +172,15 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid case "precisionByLabel" => metrics.precision($(metricLabel)) case "recallByLabel" => metrics.recall($(metricLabel)) case "fMeasureByLabel" => metrics.fMeasure($(metricLabel), $(beta)) + case "hammingLoss" => metrics.hammingLoss case "logLoss" => metrics.logLoss($(eps)) } } @Since("1.5.0") override def isLargerBetter: Boolean = $(metricName) match { - case "weightedFalsePositiveRate" | "falsePositiveRateByLabel" | "logLoss" => false + case "weightedFalsePositiveRate" | "falsePositiveRateByLabel" | "logLoss" | "hammingLoss" => + false case _ => true } @@ -199,7 +201,7 @@ object MulticlassClassificationEvaluator private val supportedMetricNames = Array("f1", "accuracy", "weightedPrecision", "weightedRecall", "weightedTruePositiveRate", "weightedFalsePositiveRate", "weightedFMeasure", "truePositiveRateByLabel", "falsePositiveRateByLabel", "precisionByLabel", "recallByLabel", - "fMeasureByLabel", "logLoss") + "fMeasureByLabel", "logLoss", "hammingLoss") @Since("1.6.0") override def load(path: String): MulticlassClassificationEvaluator = super.load(path) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index 9518f7e6828cf..050ebb0fa4fbd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -240,6 +240,23 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[_ <: Product]) @Since("1.1.0") lazy val labels: Array[Double] = tpByClass.keys.toArray.sorted + /** + * Returns Hamming-loss + */ + @Since("3.0.0") + lazy val hammingLoss: Double = { + var numerator = 0.0 + var denominator = 0.0 + confusions.iterator.foreach { + case ((label, prediction), weight) => + if (label != prediction) { + numerator += weight + } + denominator += weight + } + numerator / denominator + } + /** * Returns the log-loss, aka logistic loss or cross-entropy loss. * @param eps log-loss is undefined for p=0 or p=1, so probabilities are diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index e10295c905cdb..a8c6339ba6824 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -254,4 +254,35 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val metrics2 = new MulticlassMetrics(rdd2) assert(metrics2.logLoss() ~== 0.9682005730687164 relTol delta) } + + test("MulticlassMetrics supports hammingLoss") { + /* + Using the following Python code to verify the correctness. + + from sklearn.metrics import hamming_loss + y_true = [2, 2, 3, 4] + y_pred = [1, 2, 3, 4] + weights = [1.5, 2.0, 1.0, 0.5] + + >>> hamming_loss(y_true, y_pred) + 0.25 + >>> hamming_loss(y_true, y_pred, sample_weight=weights) + 0.3 + */ + + val preds = Seq(1.0, 2.0, 3.0, 4.0) + val labels = Seq(2.0, 2.0, 3.0, 4.0) + val weights = Seq(1.5, 2.0, 1.0, 0.5) + + val rdd = sc.parallelize(preds.zip(labels)) + val metrics = new MulticlassMetrics(rdd) + assert(metrics.hammingLoss ~== 0.25 relTol delta) + + val rdd2 = sc.parallelize(preds.zip(labels).zip(weights)) + .map { case ((pred, label), weight) => + (pred, label, weight) + } + val metrics2 = new MulticlassMetrics(rdd2) + assert(metrics2.hammingLoss ~== 0.3 relTol delta) + } } diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 6539e2abaed12..556a2f85c708d 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -374,6 +374,10 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio >>> evaluator.evaluate(dataset, {evaluator.metricName: "truePositiveRateByLabel", ... evaluator.metricLabel: 1.0}) 0.75... + >>> evaluator.setMetricName("hammingLoss") + MulticlassClassificationEvaluator... + >>> evaluator.evaluate(dataset) + 0.33... >>> mce_path = temp_path + "/mce" >>> evaluator.save(mce_path) >>> evaluator2 = MulticlassClassificationEvaluator.load(mce_path) @@ -408,7 +412,7 @@ class MulticlassClassificationEvaluator(JavaEvaluator, HasLabelCol, HasPredictio "(f1|accuracy|weightedPrecision|weightedRecall|weightedTruePositiveRate|" "weightedFalsePositiveRate|weightedFMeasure|truePositiveRateByLabel|" "falsePositiveRateByLabel|precisionByLabel|recallByLabel|fMeasureByLabel|" - "logLoss)", + "logLoss|hammingLoss)", typeConverter=TypeConverters.toString) metricLabel = Param(Params._dummy(), "metricLabel", "The class whose metric will be computed in truePositiveRateByLabel|" From 6146dc4562739c1c947eb944897c2fe85d1016e0 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 21 Nov 2019 18:43:21 +0800 Subject: [PATCH 0687/1387] [SPARK-29874][SQL] Optimize Dataset.isEmpty() ### What changes were proposed in this pull request? In origin way to judge if a DataSet is empty by ``` def isEmpty: Boolean = withAction("isEmpty", limit(1).groupBy().count().queryExecution) { plan => plan.executeCollect().head.getLong(0) == 0 } ``` will add two shuffles by `limit()`, `groupby() and count()`, then collect all data to driver. In this way we can avoid `oom` when collect data to driver. But it will trigger all partitions calculated and add more shuffle process. We change it to ``` def isEmpty: Boolean = withAction("isEmpty", select().queryExecution) { plan => plan.executeTake(1).isEmpty } ``` After these pr, we will add a column pruning to origin LogicalPlan and use `executeTake()` API. then we won't add more shuffle process and just compute only one partition's data in last stage. In this way we can reduce cost when we call `DataSet.isEmpty()` and won't bring memory issue to driver side. ### Why are the changes needed? Optimize Dataset.isEmpty() ### Does this PR introduce any user-facing change? No ### How was this patch tested? Origin UT Closes #26500 from AngersZhuuuu/SPARK-29874. Authored-by: angerszhu Signed-off-by: Wenchen Fan --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 0fcd82276392c..e1bca44dfccf5 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 @@ -586,8 +586,8 @@ class Dataset[T] private[sql]( * @group basic * @since 2.4.0 */ - def isEmpty: Boolean = withAction("isEmpty", limit(1).groupBy().count().queryExecution) { plan => - plan.executeCollect().head.getLong(0) == 0 + def isEmpty: Boolean = withAction("isEmpty", select().queryExecution) { plan => + plan.executeTake(1).isEmpty } /** From cdcd43cbf2479b258f4c5cfa0f6306f475d25cf2 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 21 Nov 2019 23:51:12 +0800 Subject: [PATCH 0688/1387] [SPARK-29977][SQL] Remove newMutableProjection/newOrdering/newNaturalAscendingOrdering from SparkPlan ### What changes were proposed in this pull request? This is to refactor `SparkPlan` code; it mainly removed `newMutableProjection`/`newOrdering`/`newNaturalAscendingOrdering` from `SparkPlan`. The other modifications are listed below; - Move `BaseOrdering` from `o.a.s.sqlcatalyst.expressions.codegen.GenerateOrdering.scala` to `o.a.s.sqlcatalyst.expressions.ordering.scala` - `RowOrdering` extends `CodeGeneratorWithInterpretedFallback ` for `BaseOrdering` - Remove the unused variables (`subexpressionEliminationEnabled` and `codeGenFallBack`) from `SparkPlan` ### Why are the changes needed? For better code/test coverage. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing. Closes #26615 from maropu/RefactorOrdering. Authored-by: Takeshi Yamamuro Signed-off-by: Wenchen Fan --- .../codegen/GenerateOrdering.scala | 10 +---- .../sql/catalyst/expressions/ordering.scala | 40 +++++++++++++++++-- .../sql/execution/UnsafeKVExternalSorter.java | 2 +- .../apache/spark/sql/execution/SortExec.scala | 2 +- .../spark/sql/execution/SparkPlan.scala | 38 ------------------ .../aggregate/HashAggregateExec.scala | 7 ++-- .../aggregate/ObjectAggregationIterator.scala | 2 +- .../aggregate/ObjectHashAggregateExec.scala | 2 +- .../aggregate/SortAggregateExec.scala | 2 +- .../execution/joins/SortMergeJoinExec.scala | 2 +- .../sql/execution/python/EvalPythonExec.scala | 2 +- .../sql/execution/window/WindowExecBase.scala | 12 +++--- .../spark/sql/execution/ReferenceSort.scala | 4 +- 13 files changed, 55 insertions(+), 70 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index b66b80ad31dc2..63bd59e7628b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -29,19 +29,11 @@ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -/** - * Inherits some default implementation for Java from `Ordering[Row]` - */ -class BaseOrdering extends Ordering[InternalRow] { - def compare(a: InternalRow, b: InternalRow): Int = { - throw new UnsupportedOperationException - } -} /** * Generates bytecode for an [[Ordering]] of rows for a given set of expressions. */ -object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalRow]] with Logging { +object GenerateOrdering extends CodeGenerator[Seq[SortOrder], BaseOrdering] with Logging { protected def canonicalize(in: Seq[SortOrder]): Seq[SortOrder] = in.map(ExpressionCanonicalizer.execute(_).asInstanceOf[SortOrder]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index c9706c09f6949..8867a03a4633b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -19,18 +19,28 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering import org.apache.spark.sql.types._ +/** + * A base class for generated/interpreted row ordering. + */ +class BaseOrdering extends Ordering[InternalRow] { + def compare(a: InternalRow, b: InternalRow): Int = { + throw new UnsupportedOperationException + } +} + /** * An interpreted row ordering comparator. */ -class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow] { +class InterpretedOrdering(ordering: Seq[SortOrder]) extends BaseOrdering { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = this(bindReferences(ordering, inputSchema)) - def compare(a: InternalRow, b: InternalRow): Int = { + override def compare(a: InternalRow, b: InternalRow): Int = { var i = 0 val size = ordering.size while (i < size) { @@ -67,7 +77,7 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow } i += 1 } - return 0 + 0 } } @@ -83,7 +93,7 @@ object InterpretedOrdering { } } -object RowOrdering { +object RowOrdering extends CodeGeneratorWithInterpretedFallback[Seq[SortOrder], BaseOrdering] { /** * Returns true iff the data type can be ordered (i.e. can be sorted). @@ -102,4 +112,26 @@ object RowOrdering { * Returns true iff outputs from the expressions can be ordered. */ def isOrderable(exprs: Seq[Expression]): Boolean = exprs.forall(e => isOrderable(e.dataType)) + + override protected def createCodeGeneratedObject(in: Seq[SortOrder]): BaseOrdering = { + GenerateOrdering.generate(in) + } + + override protected def createInterpretedObject(in: Seq[SortOrder]): BaseOrdering = { + new InterpretedOrdering(in) + } + + def create(order: Seq[SortOrder], inputSchema: Seq[Attribute]): BaseOrdering = { + createObject(bindReferences(order, inputSchema)) + } + + /** + * Creates a row ordering for the given schema, in natural ascending order. + */ + def createNaturalAscendingOrdering(dataTypes: Seq[DataType]): BaseOrdering = { + val order: Seq[SortOrder] = dataTypes.zipWithIndex.map { + case (dt, index) => SortOrder(BoundReference(index, dt, nullable = true), Ascending) + } + create(order, Seq.empty) + } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 09426117a24b9..acd54fe25d62d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -29,7 +29,7 @@ import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; -import org.apache.spark.sql.catalyst.expressions.codegen.BaseOrdering; +import org.apache.spark.sql.catalyst.expressions.BaseOrdering; import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering; import org.apache.spark.sql.types.StructType; import org.apache.spark.storage.BlockManager; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index 24f664ca595c7..6b6ca531c6d3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -71,7 +71,7 @@ case class SortExec( * should make it public. */ def createSorter(): UnsafeExternalRowSorter = { - val ordering = newOrdering(sortOrder, output) + val ordering = RowOrdering.create(sortOrder, output) // The comparator for comparing prefix val boundSortExpression = BindReferences.bindReference(sortOrder.head, output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 738af995376e6..ef9f38b8f9927 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -22,9 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer -import org.codehaus.commons.compiler.CompileException -import org.codehaus.janino.InternalCompilerException - import org.apache.spark.{broadcast, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec @@ -32,13 +29,11 @@ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch object SparkPlan { @@ -72,16 +67,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ val id: Int = SparkPlan.newPlanId() - // sqlContext will be null when SparkPlan nodes are created without the active sessions. - val subexpressionEliminationEnabled: Boolean = if (sqlContext != null) { - sqlContext.conf.subexpressionEliminationEnabled - } else { - false - } - - // whether we should fallback when hitting compilation errors caused by codegen - private val codeGenFallBack = (sqlContext == null) || sqlContext.conf.codegenFallback - /** * Return true if this stage of the plan supports columnar execution. */ @@ -462,29 +447,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ buf.toArray } - protected def newMutableProjection( - expressions: Seq[Expression], - inputSchema: Seq[Attribute], - useSubexprElimination: Boolean = false): MutableProjection = { - log.debug(s"Creating MutableProj: $expressions, inputSchema: $inputSchema") - MutableProjection.create(expressions, inputSchema) - } - - protected def newOrdering( - order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[InternalRow] = { - GenerateOrdering.generate(order, inputSchema) - } - - /** - * Creates a row ordering for the given schema, in natural ascending order. - */ - protected def newNaturalAscendingOrdering(dataTypes: Seq[DataType]): Ordering[InternalRow] = { - val order: Seq[SortOrder] = dataTypes.zipWithIndex.map { - case (dt, index) => SortOrder(BoundReference(index, dt, nullable = true), Ascending) - } - newOrdering(order, Seq.empty) - } - /** * Cleans up the resources used by the physical operator (if any). In general, all the resources * should be cleaned up when the task finishes but operators like SortMergeJoinExec and LimitExec diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 95bef308e453d..ad8976c77b16a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -126,7 +126,7 @@ case class HashAggregateExec( initialInputBufferOffset, resultExpressions, (expressions, inputSchema) => - newMutableProjection(expressions, inputSchema, subexpressionEliminationEnabled), + MutableProjection.create(expressions, inputSchema), child.output, iter, testFallbackStartsAt, @@ -486,10 +486,9 @@ case class HashAggregateExec( // Create a MutableProjection to merge the rows of same key together val mergeExpr = declFunctions.flatMap(_.mergeExpressions) - val mergeProjection = newMutableProjection( + val mergeProjection = MutableProjection.create( mergeExpr, - aggregateBufferAttributes ++ declFunctions.flatMap(_.inputAggBufferAttributes), - subexpressionEliminationEnabled) + aggregateBufferAttributes ++ declFunctions.flatMap(_.inputAggBufferAttributes)) val joinedRow = new JoinedRow() var currentKey: UnsafeRow = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala index b88ddba8e48d3..1f325c11c9e44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala @@ -22,7 +22,7 @@ import org.apache.spark.internal.{config, Logging} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.expressions.codegen.{BaseOrdering, GenerateOrdering} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index 151da241144be..953622afebf89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -122,7 +122,7 @@ case class ObjectHashAggregateExec( initialInputBufferOffset, resultExpressions, (expressions, inputSchema) => - newMutableProjection(expressions, inputSchema, subexpressionEliminationEnabled), + MutableProjection.create(expressions, inputSchema), child.output, iter, fallbackCountThreshold, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 7ab6ecc08a7bc..0ddf95771d5b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -93,7 +93,7 @@ case class SortAggregateExec( initialInputBufferOffset, resultExpressions, (expressions, inputSchema) => - newMutableProjection(expressions, inputSchema, subexpressionEliminationEnabled), + MutableProjection.create(expressions, inputSchema), numOutputRows) if (!hasInput && groupingExpressions.isEmpty) { // There is no input and there is no grouping expressions. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index cd3c596435a21..f327e84563da9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -175,7 +175,7 @@ case class SortMergeJoinExec( } // An ordering that can be used to compare keys from both sides. - val keyOrdering = newNaturalAscendingOrdering(leftKeys.map(_.dataType)) + val keyOrdering = RowOrdering.createNaturalAscendingOrdering(leftKeys.map(_.dataType)) val resultProj: InternalRow => InternalRow = UnsafeProjection.create(output, output) joinType match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala index 3554bdb5c9e0c..a0f23e925d237 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala @@ -113,7 +113,7 @@ abstract class EvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], } }.toArray }.toArray - val projection = newMutableProjection(allInputs, child.output) + val projection = MutableProjection.create(allInputs, child.output) val schema = StructType(dataTypes.zipWithIndex.map { case (dt, i) => StructField(s"_$i", dt) }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala index dcb86f48bdf32..e8248b7028757 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala @@ -73,7 +73,7 @@ abstract class WindowExecBase( RowBoundOrdering(offset) case (RangeFrame, CurrentRow) => - val ordering = newOrdering(orderSpec, child.output) + val ordering = RowOrdering.create(orderSpec, child.output) RangeBoundOrdering(ordering, IdentityProjection, IdentityProjection) case (RangeFrame, offset: Expression) if orderSpec.size == 1 => @@ -82,7 +82,7 @@ abstract class WindowExecBase( val expr = sortExpr.child // Create the projection which returns the current 'value'. - val current = newMutableProjection(expr :: Nil, child.output) + val current = MutableProjection.create(expr :: Nil, child.output) // Flip the sign of the offset when processing the order is descending val boundOffset = sortExpr.direction match { @@ -97,13 +97,13 @@ abstract class WindowExecBase( TimeAdd(expr, boundOffset, Some(timeZone)) case (a, b) if a == b => Add(expr, boundOffset) } - val bound = newMutableProjection(boundExpr :: Nil, child.output) + val bound = MutableProjection.create(boundExpr :: Nil, child.output) // Construct the ordering. This is used to compare the result of current value projection // to the result of bound value projection. This is done manually because we want to use // Code Generation (if it is enabled). val boundSortExprs = sortExpr.copy(BoundReference(0, expr.dataType, expr.nullable)) :: Nil - val ordering = newOrdering(boundSortExprs, Nil) + val ordering = RowOrdering.create(boundSortExprs, Nil) RangeBoundOrdering(ordering, current, bound) case (RangeFrame, _) => @@ -167,7 +167,7 @@ abstract class WindowExecBase( ordinal, child.output, (expressions, schema) => - newMutableProjection(expressions, schema, subexpressionEliminationEnabled)) + MutableProjection.create(expressions, schema)) } // Create the factory @@ -182,7 +182,7 @@ abstract class WindowExecBase( functions.map(_.asInstanceOf[OffsetWindowFunction]), child.output, (expressions, schema) => - newMutableProjection(expressions, schema, subexpressionEliminationEnabled), + MutableProjection.create(expressions, schema), offset) // Entire Partition Frame. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala index 6abcb1f067968..25b4464823e5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala @@ -21,7 +21,7 @@ import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Attribute, RowOrdering, SortOrder} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter @@ -41,7 +41,7 @@ case class ReferenceSort( protected override def doExecute(): RDD[InternalRow] = attachTree(this, "sort") { child.execute().mapPartitions( { iterator => - val ordering = newOrdering(sortOrder, child.output) + val ordering = RowOrdering.create(sortOrder, child.output) val sorter = new ExternalSorter[InternalRow, Null, InternalRow]( TaskContext.get(), ordering = Some(ordering)) sorter.insertAll(iterator.map(r => (r.copy(), null))) From 54c5087a3ae306ad766df81a5a6279f219b2ea47 Mon Sep 17 00:00:00 2001 From: Ximo Guanter Date: Fri, 22 Nov 2019 00:19:25 +0800 Subject: [PATCH 0689/1387] [SPARK-29248][SQL] provider number of partitions when creating v2 data writer factory ### What changes were proposed in this pull request? When implementing a ScanBuilder, we require the implementor to provide the schema of the data and the number of partitions. However, when someone is implementing WriteBuilder we only pass them the schema, but not the number of partitions. This is an asymetrical developer experience. This PR adds a PhysicalWriteInfo interface that is passed to createBatchWriterFactory and createStreamingWriterFactory that adds the number of partitions of the data that is going to be written. ### Why are the changes needed? Passing in the number of partitions on the WriteBuilder would enable data sources to provision their write targets before starting to write. For example: it could be used to provision a Kafka topic with a specific number of partitions it could be used to scale a microservice prior to sending the data to it it could be used to create a DsV2 that sends the data to another spark cluster (currently not possible since the reader wouldn't be able to know the number of partitions) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Tests passed Closes #26591 from edrevo/temp. Authored-by: Ximo Guanter Signed-off-by: Wenchen Fan --- .../spark/sql/kafka010/KafkaBatchWrite.scala | 4 +-- .../sql/kafka010/KafkaStreamingWrite.scala | 5 +-- .../spark/sql/connector/write/BatchWrite.java | 8 +++-- .../connector/write/DataWriterFactory.java | 5 +-- .../connector/write/PhysicalWriteInfo.java | 33 +++++++++++++++++++ .../streaming/StreamingDataWriterFactory.java | 5 +-- .../write/streaming/StreamingWrite.java | 9 +++-- .../write/PhysicalWriteInfoImpl.scala | 20 +++++++++++ .../spark/sql/connector/InMemoryTable.scala | 2 +- .../datasources/noop/NoopDataSource.scala | 9 ++--- .../datasources/v2/FileBatchWrite.scala | 4 +-- .../v2/WriteToDataSourceV2Exec.scala | 29 ++++++++-------- .../WriteToContinuousDataSourceExec.scala | 7 ++-- .../streaming/sources/ConsoleWrite.scala | 5 +-- .../sources/ForeachWriterTable.scala | 5 +-- .../streaming/sources/MicroBatchWrite.scala | 6 ++-- .../execution/streaming/sources/memory.scala | 4 +-- .../connector/SimpleWritableDataSource.scala | 2 +- .../sources/StreamingDataSourceV2Suite.scala | 5 +-- 19 files changed, 119 insertions(+), 48 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/PhysicalWriteInfo.java create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/PhysicalWriteInfoImpl.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala index 8e29e38b2a644..56c0fdd7c35b7 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery import org.apache.spark.sql.types.StructType @@ -40,7 +40,7 @@ private[kafka010] class KafkaBatchWrite( validateQuery(schema.toAttributes, producerParams, topic) - override def createBatchWriterFactory(): KafkaBatchWriterFactory = + override def createBatchWriterFactory(info: PhysicalWriteInfo): KafkaBatchWriterFactory = KafkaBatchWriterFactory(topic, producerParams, schema) override def commit(messages: Array[WriterCommitMessage]): Unit = {} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala index 2b50b771e694e..bcf9e3416f843 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{DataWriter, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery import org.apache.spark.sql.types.StructType @@ -41,7 +41,8 @@ private[kafka010] class KafkaStreamingWrite( validateQuery(schema.toAttributes, producerParams, topic) - override def createStreamingWriterFactory(): KafkaStreamWriterFactory = + override def createStreamingWriterFactory( + info: PhysicalWriteInfo): KafkaStreamWriterFactory = KafkaStreamWriterFactory(topic, producerParams, schema) override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java index 37c5539d2518f..3e8b14172d6b2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/BatchWrite.java @@ -23,8 +23,8 @@ * An interface that defines how to write the data to data source for batch processing. * * The writing procedure is: - * 1. Create a writer factory by {@link #createBatchWriterFactory()}, serialize and send it to all - * the partitions of the input data(RDD). + * 1. Create a writer factory by {@link #createBatchWriterFactory(PhysicalWriteInfo)}, serialize + * and send it to all the partitions of the input data(RDD). * 2. For each partition, create the data writer, and write the data of the partition with this * writer. If all the data are written successfully, call {@link DataWriter#commit()}. If * exception happens during the writing, call {@link DataWriter#abort()}. @@ -45,8 +45,10 @@ public interface BatchWrite { * * If this method fails (by throwing an exception), the action will fail and no Spark job will be * submitted. + * + * @param info Physical information about the input data that will be written to this table. */ - DataWriterFactory createBatchWriterFactory(); + DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info); /** * Returns whether Spark should use the commit coordinator to ensure that at most one task for diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java index bcf8d8a59e5e5..310575df05d97 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriterFactory.java @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.InternalRow; /** - * A factory of {@link DataWriter} returned by {@link BatchWrite#createBatchWriterFactory()}, - * which is responsible for creating and initializing the actual data writer at executor side. + * A factory of {@link DataWriter} returned by + * {@link BatchWrite#createBatchWriterFactory(PhysicalWriteInfo)}, which is responsible for + * creating and initializing the actual data writer at executor side. * * Note that, the writer factory will be serialized and sent to executors, then the data writer * will be created on executors and do the actual writing. So this interface must be diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/PhysicalWriteInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/PhysicalWriteInfo.java new file mode 100644 index 0000000000000..55a092e39970e --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/PhysicalWriteInfo.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.write; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory; + +/** + * This interface contains physical write information that data sources can use when + * generating a {@link DataWriterFactory} or a {@link StreamingDataWriterFactory}. + */ +@Evolving +public interface PhysicalWriteInfo { + /** + * The number of partitions of the input data that is going to be written. + */ + int numPartitions(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java index daaa18d5bc4e7..9946867e8ea65 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java @@ -23,11 +23,12 @@ import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; /** * A factory of {@link DataWriter} returned by - * {@link StreamingWrite#createStreamingWriterFactory()}, which is responsible for creating - * and initializing the actual data writer at executor side. + * {@link StreamingWrite#createStreamingWriterFactory(PhysicalWriteInfo)}, which is responsible for + * creating and initializing the actual data writer at executor side. * * Note that, the writer factory will be serialized and sent to executors, then the data writer * will be created on executors and do the actual writing. So this interface must be diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java index 0821b34891654..4f930e1c158e5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java @@ -19,14 +19,15 @@ import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; import org.apache.spark.sql.connector.write.WriterCommitMessage; /** * An interface that defines how to write the data to data source in streaming queries. * * The writing procedure is: - * 1. Create a writer factory by {@link #createStreamingWriterFactory()}, serialize and send it to - * all the partitions of the input data(RDD). + * 1. Create a writer factory by {@link #createStreamingWriterFactory(PhysicalWriteInfo)}, + * serialize and send it to all the partitions of the input data(RDD). * 2. For each epoch in each partition, create the data writer, and write the data of the epoch in * the partition with this writer. If all the data are written successfully, call * {@link DataWriter#commit()}. If exception happens during the writing, call @@ -48,8 +49,10 @@ public interface StreamingWrite { * * If this method fails (by throwing an exception), the action will fail and no Spark job will be * submitted. + * + * @param info Information about the RDD that will be written to this data writer */ - StreamingDataWriterFactory createStreamingWriterFactory(); + StreamingDataWriterFactory createStreamingWriterFactory(PhysicalWriteInfo info); /** * Commits this writing job for the specified epoch with a list of commit messages. The commit diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/PhysicalWriteInfoImpl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/PhysicalWriteInfoImpl.scala new file mode 100644 index 0000000000000..a663822f3eb45 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/PhysicalWriteInfoImpl.scala @@ -0,0 +1,20 @@ +/* + * 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.connector.write + +private[sql] case class PhysicalWriteInfoImpl(numPartitions: Int) extends PhysicalWriteInfo diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index e7b2dd2ebc9da..201860e5135ba 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -124,7 +124,7 @@ class InMemoryTable( } private abstract class TestBatchWrite extends BatchWrite { - override def createBatchWriterFactory(): DataWriterFactory = { + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = { BufferedRowsWriterFactory } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala index 3f4f29c3e135a..03e5f43a2a0af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.StructType @@ -58,7 +58,8 @@ private[noop] object NoopWriteBuilder extends WriteBuilder with SupportsTruncate } private[noop] object NoopBatchWrite extends BatchWrite { - override def createBatchWriterFactory(): DataWriterFactory = NoopWriterFactory + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = + NoopWriterFactory override def commit(messages: Array[WriterCommitMessage]): Unit = {} override def abort(messages: Array[WriterCommitMessage]): Unit = {} } @@ -74,8 +75,8 @@ private[noop] object NoopWriter extends DataWriter[InternalRow] { } private[noop] object NoopStreamingWrite extends StreamingWrite { - override def createStreamingWriterFactory(): StreamingDataWriterFactory = - NoopStreamingDataWriterFactory + override def createStreamingWriterFactory( + info: PhysicalWriteInfo): StreamingDataWriterFactory = NoopStreamingDataWriterFactory override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala index e7d9a247533c4..266c834909363 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala @@ -20,7 +20,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult} import org.apache.spark.sql.execution.datasources.FileFormatWriter.processStats @@ -44,7 +44,7 @@ class FileBatchWrite( committer.abortJob(job) } - override def createBatchWriterFactory(): DataWriterFactory = { + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = { FileWriterFactory(description, committer) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 9f4392da6ab4d..7d8a115c126eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, SupportsWrite, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfoImpl, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.sources.{AlwaysTrue, Filter} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -353,17 +353,20 @@ trait V2TableWriteExec extends UnaryExecNode { override def output: Seq[Attribute] = Nil protected def writeWithV2(batchWrite: BatchWrite): RDD[InternalRow] = { - val writerFactory = batchWrite.createBatchWriterFactory() - val useCommitCoordinator = batchWrite.useCommitCoordinator - val rdd = query.execute() - // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single - // partition rdd to make sure we at least set up one write task to write the metadata. - val rddWithNonEmptyPartitions = if (rdd.partitions.length == 0) { - sparkContext.parallelize(Array.empty[InternalRow], 1) - } else { - rdd + val rdd: RDD[InternalRow] = { + val tempRdd = query.execute() + // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single + // partition rdd to make sure we at least set up one write task to write the metadata. + if (tempRdd.partitions.length == 0) { + sparkContext.parallelize(Array.empty[InternalRow], 1) + } else { + tempRdd + } } - val messages = new Array[WriterCommitMessage](rddWithNonEmptyPartitions.partitions.length) + val writerFactory = batchWrite.createBatchWriterFactory( + PhysicalWriteInfoImpl(rdd.getNumPartitions)) + val useCommitCoordinator = batchWrite.useCommitCoordinator + val messages = new Array[WriterCommitMessage](rdd.partitions.length) val totalNumRowsAccumulator = new LongAccumulator() logInfo(s"Start processing data source write support: $batchWrite. " + @@ -371,10 +374,10 @@ trait V2TableWriteExec extends UnaryExecNode { try { sparkContext.runJob( - rddWithNonEmptyPartitions, + rdd, (context: TaskContext, iter: Iterator[InternalRow]) => DataWritingSparkTask.run(writerFactory, context, iter, useCommitCoordinator), - rddWithNonEmptyPartitions.partitions.indices, + rdd.partitions.indices, (index, result: DataWritingSparkTaskResult) => { val commitMessage = result.writerCommitMessage messages(index) = commitMessage diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala index d4e522562e914..f1898ad3f27ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala @@ -24,6 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.streaming.StreamExecution @@ -38,8 +39,10 @@ case class WriteToContinuousDataSourceExec(write: StreamingWrite, query: SparkPl override def output: Seq[Attribute] = Nil override protected def doExecute(): RDD[InternalRow] = { - val writerFactory = write.createStreamingWriterFactory() - val rdd = new ContinuousWriteRDD(query.execute(), writerFactory) + val queryRdd = query.execute() + val writerFactory = write.createStreamingWriterFactory( + PhysicalWriteInfoImpl(queryRdd.getNumPartitions)) + val rdd = new ContinuousWriteRDD(queryRdd, writerFactory) logInfo(s"Start processing data source write support: $write. " + s"The input RDD has ${rdd.partitions.length} partitions.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala index 6afb811a4d998..ad5c7cf24caf7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation -import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.connector.write.{PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -38,7 +38,8 @@ class ConsoleWrite(schema: StructType, options: CaseInsensitiveStringMap) assert(SparkSession.getActiveSession.isDefined) protected val spark = SparkSession.getActiveSession.get - def createStreamingWriterFactory(): StreamingDataWriterFactory = PackedRowWriterFactory + def createStreamingWriterFactory(info: PhysicalWriteInfo): StreamingDataWriterFactory = + PackedRowWriterFactory override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { // We have to print a "Batch" label for the epoch for compatibility with the pre-data source V2 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala index bae7fa7d07356..53d4bca1a5f7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.connector.write.{DataWriter, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{DataWriter, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.python.PythonForeachWriter import org.apache.spark.sql.types.StructType @@ -72,7 +72,8 @@ case class ForeachWriterTable[T]( override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} - override def createStreamingWriterFactory(): StreamingDataWriterFactory = { + override def createStreamingWriterFactory( + info: PhysicalWriteInfo): StreamingDataWriterFactory = { val rowConverter: InternalRow => T = converter match { case Left(enc) => val boundEnc = enc.resolveAndBind( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala index 5f12832cd2550..c2adc1dd6742a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} /** @@ -36,8 +36,8 @@ class MicroBatchWrite(eppchId: Long, val writeSupport: StreamingWrite) extends B writeSupport.abort(eppchId, messages) } - override def createBatchWriterFactory(): DataWriterFactory = { - new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory()) + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = { + new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory(info)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala index 51ab5ce3578af..a976876b4d8e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.streaming.Sink import org.apache.spark.sql.types.StructType @@ -140,7 +140,7 @@ class MemoryStreamingWrite( val sink: MemorySink, schema: StructType, needTruncate: Boolean) extends StreamingWrite { - override def createStreamingWriterFactory: MemoryWriterFactory = { + override def createStreamingWriterFactory(info: PhysicalWriteInfo): MemoryWriterFactory = { MemoryWriterFactory(schema) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala index 22d3750022c57..306da996e2caa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala @@ -99,7 +99,7 @@ class SimpleWritableDataSource extends TableProvider with SessionConfigSupport { } class MyBatchWrite(queryId: String, path: String, conf: Configuration) extends BatchWrite { - override def createBatchWriterFactory(): DataWriterFactory = { + override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = { SimpleCounter.resetCounter new CSVDataWriterFactory(path, queryId, new SerializableConfiguration(conf)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index e9d148c38e6cb..c4f0751a955d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, SupportsRea import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, MicroBatchStream, Offset, PartitionOffset} -import org.apache.spark.sql.connector.write.{WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.connector.write.{PhysicalWriteInfo, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RateStreamOffset, Sink, StreamingQueryWrapper} @@ -68,7 +68,8 @@ class FakeScanBuilder extends ScanBuilder with Scan { class FakeWriteBuilder extends WriteBuilder with StreamingWrite { override def buildForStreaming(): StreamingWrite = this - override def createStreamingWriterFactory(): StreamingDataWriterFactory = { + override def createStreamingWriterFactory( + info: PhysicalWriteInfo): StreamingDataWriterFactory = { throw new IllegalStateException("fake sink - cannot actually write") } override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { From 6b4b6a87cde8e29da5cbc2ee00242ec74d5477b2 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 21 Nov 2019 09:47:42 -0800 Subject: [PATCH 0690/1387] [SPARK-29558][SQL] ResolveTables and ResolveRelations should be order-insensitive ### What changes were proposed in this pull request? Make `ResolveRelations` call `ResolveTables` at the beginning, and make `ResolveTables` call `ResolveTempViews`(newly added) at the beginning, to ensure the relation resolution priority. ### Why are the changes needed? To resolve an `UnresolvedRelation`, the general process is: 1. try to resolve to (global) temp view first. If it's not a temp view, move on 2. if the table name specifies a catalog, lookup the table from the specified catalog. Otherwise, lookup table from the current catalog. 3. when looking up table from session catalog, return a v1 relation if the table provider is v1. Currently, this process is done by 2 rules: `ResolveTables` and `ResolveRelations`. To avoid rule conflicts, we add a lot of checks: 1. `ResolveTables` only resolves `UnresolvedRelation` if it's not a temp view and the resolved table is not v1. 2. `ResolveRelations` only resolves `UnresolvedRelation` if the table name has less than 2 parts. This requires to run `ResolveTables` before `ResolveRelations`, otherwise we may resolve a v2 table to a v1 relation. To clearly guarantee the resolution priority, and avoid massive changes, this PR proposes to call one rule in another rule to ensure the rule execution order. Now the process is simple: 1. first run `ResolveTempViews`, see if we can resolve relation to temp view 2. then run `ResolveTables`, see if we can resolve relation to v2 tables. 3. finally run `ResolveRelations`, see if we can resolve relation to v1 tables. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26214 from cloud-fan/resolve. Authored-by: Wenchen Fan Signed-off-by: Ryan Blue --- .../sql/catalyst/analysis/Analyzer.scala | 26 ++++++++++++------- .../sql/catalyst/catalog/SessionCatalog.scala | 19 ++++++++++++++ .../connector/catalog/CatalogManager.scala | 2 +- .../sql/connector/catalog/LookupCatalog.scala | 17 ++++++++++-- .../sql/connector/DataSourceV2SQLSuite.scala | 16 +++++++++++- 5 files changed, 67 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a7443e71c0ca3..625ef2153c711 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -198,7 +198,6 @@ class Analyzer( ResolveTableValuedFunctions :: new ResolveCatalogs(catalogManager) :: ResolveInsertInto :: - ResolveTables :: ResolveRelations :: ResolveReferences :: ResolveCreateNamedStruct :: @@ -666,12 +665,26 @@ class Analyzer( } /** - * Resolve table relations with concrete relations from v2 catalog. + * Resolve relations to temp views. This is not an actual rule, and is only called by + * [[ResolveTables]]. + */ + object ResolveTempViews extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case u @ UnresolvedRelation(Seq(part1)) => + v1SessionCatalog.lookupTempView(part1).getOrElse(u) + case u @ UnresolvedRelation(Seq(part1, part2)) => + v1SessionCatalog.lookupGlobalTempView(part1, part2).getOrElse(u) + } + } + + /** + * Resolve table relations with concrete relations from v2 catalog. This is not an actual rule, + * and is only called by [[ResolveRelations]]. * * [[ResolveRelations]] still resolves v1 tables. */ object ResolveTables extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + def apply(plan: LogicalPlan): LogicalPlan = ResolveTempViews(plan).resolveOperatorsUp { case u: UnresolvedRelation => lookupV2Relation(u.multipartIdentifier) .getOrElse(u) @@ -733,10 +746,6 @@ class Analyzer( // Note this is compatible with the views defined by older versions of Spark(before 2.2), which // have empty defaultDatabase and all the relations in viewText have database part defined. def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match { - case u @ UnresolvedRelation(AsTemporaryViewIdentifier(ident)) - if v1SessionCatalog.isTemporaryTable(ident) => - resolveRelation(lookupTableFromCatalog(ident, u, AnalysisContext.get.defaultDatabase)) - case u @ UnresolvedRelation(AsTableIdentifier(ident)) if !isRunningDirectlyOnFiles(ident) => val defaultDatabase = AnalysisContext.get.defaultDatabase val foundRelation = lookupTableFromCatalog(ident, u, defaultDatabase) @@ -767,7 +776,7 @@ class Analyzer( case _ => plan } - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + def apply(plan: LogicalPlan): LogicalPlan = ResolveTables(plan).resolveOperatorsUp { case i @ InsertIntoStatement(u @ UnresolvedRelation(AsTableIdentifier(ident)), _, child, _, _) if child.resolved => EliminateSubqueryAliases(lookupTableFromCatalog(ident, u)) match { @@ -2839,7 +2848,6 @@ class Analyzer( private def lookupV2RelationAndCatalog( identifier: Seq[String]): Option[(DataSourceV2Relation, CatalogPlugin, Identifier)] = identifier match { - case AsTemporaryViewIdentifier(ti) if v1SessionCatalog.isTemporaryTable(ti) => None case CatalogObjectIdentifier(catalog, ident) if !CatalogV2Util.isSessionCatalog(catalog) => CatalogV2Util.loadTable(catalog, ident) match { case Some(table) => Some((DataSourceV2Relation.create(table), catalog, ident)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e77f3c70f0bdf..96ca1ac73e043 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -767,6 +767,25 @@ class SessionCatalog( } } + def lookupTempView(table: String): Option[SubqueryAlias] = { + val formattedTable = formatTableName(table) + getTempView(formattedTable).map { view => + SubqueryAlias(formattedTable, view) + } + } + + def lookupGlobalTempView(db: String, table: String): Option[SubqueryAlias] = { + val formattedDB = formatDatabaseName(db) + if (formattedDB == globalTempViewManager.database) { + val formattedTable = formatTableName(table) + getGlobalTempView(formattedTable).map { view => + SubqueryAlias(formattedTable, formattedDB, view) + } + } else { + None + } + } + /** * Return whether a table with the specified name is a temporary view. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index d62148b2bbe45..135c180ef4000 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -71,7 +71,7 @@ class CatalogManager( * This happens when the source implementation extends the v2 TableProvider API and is not listed * in the fallback configuration, spark.sql.sources.write.useV1SourceList */ - private def v2SessionCatalog: CatalogPlugin = { + private[sql] def v2SessionCatalog: CatalogPlugin = { conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { customV2SessionCatalog => try { catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 26ba93e57fc64..613c0d1797cc6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.connector.catalog import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} /** * A trait to encapsulate catalog lookup function and helpful extractors. @@ -120,10 +121,22 @@ private[sql] trait LookupCatalog extends Logging { * Extract catalog and the rest name parts from a multi-part identifier. */ object CatalogAndIdentifierParts { - def unapply(nameParts: Seq[String]): Some[(CatalogPlugin, Seq[String])] = { + private val globalTempDB = SQLConf.get.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = { assert(nameParts.nonEmpty) try { - Some((catalogManager.catalog(nameParts.head), nameParts.tail)) + // Conceptually global temp views are in a special reserved catalog. However, the v2 catalog + // API does not support view yet, and we have to use v1 commands to deal with global temp + // views. To simplify the implementation, we put global temp views in a special namespace + // in the session catalog. The special namespace has higher priority during name resolution. + // For example, if the name of a custom catalog is the same with `GLOBAL_TEMP_DATABASE`, + // this custom catalog can't be accessed. + if (nameParts.head.equalsIgnoreCase(globalTempDB)) { + Some((catalogManager.v2SessionCatalog, nameParts)) + } else { + Some((catalogManager.catalog(nameParts.head), nameParts.tail)) + } } catch { case _: CatalogNotFoundException => Some((currentCatalog, nameParts)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 2a44251e102a0..2958b57bf06ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableExceptio import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.sources.SimpleScanSource import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructType} @@ -1786,6 +1786,20 @@ class DataSourceV2SQLSuite } } + test("global temp view should not be masked by v2 catalog") { + val globalTempDB = spark.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + spark.conf.set(s"spark.sql.catalog.$globalTempDB", classOf[InMemoryTableCatalog].getName) + + try { + sql("create global temp view v as select 1") + sql(s"alter view $globalTempDB.v rename to v2") + checkAnswer(spark.table(s"$globalTempDB.v2"), Row(1)) + sql(s"drop view $globalTempDB.v2") + } finally { + spark.sharedState.globalTempViewManager.clear() + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") From e2f056f4a89b1bd9864be8c111d39af6558c839b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 22 Nov 2019 10:56:28 +0900 Subject: [PATCH 0691/1387] [SPARK-29975][SQL] introduce --CONFIG_DIM directive ### What changes were proposed in this pull request? allow the sql test files to specify different dimensions of config sets during testing. For example, ``` --CONFIG_DIM1 a=1 --CONFIG_DIM1 b=2,c=3 --CONFIG_DIM2 x=1 --CONFIG_DIM2 y=1,z=2 ``` This example defines 2 config dimensions, and each dimension defines 2 config sets. We will run the queries 4 times: 1. a=1, x=1 2. a=1, y=1, z=2 3. b=2, c=3, x=1 4. b=2, c=3, y=1, z=2 ### Why are the changes needed? Currently `SQLQueryTestSuite` takes a long time. This is because we run each test at least 3 times, to check with different codegen modes. This is not necessary for most of the tests, e.g. DESC TABLE. We should only check these codegen modes for certain tests. With the --CONFIG_DIM directive, we can do things like: test different join operator(broadcast or shuffle join) X different codegen modes. After reducing testing time, we should be able to run thrifter server SQL tests with config settings. ### Does this PR introduce any user-facing change? no ### How was this patch tested? test only Closes #26612 from cloud-fan/test. Authored-by: Wenchen Fan Signed-off-by: HyukjinKwon --- .../resources/sql-tests/inputs/group-by.sql | 5 ++ .../inputs/higher-order-functions.sql | 5 ++ .../resources/sql-tests/inputs/inner-join.sql | 15 +++-- .../sql-tests/inputs/join-empty-relation.sql | 5 -- .../sql-tests/inputs/natural-join.sql | 5 -- .../inputs/order-by-nulls-ordering.sql | 5 ++ .../resources/sql-tests/inputs/outer-join.sql | 21 +++---- .../inputs/postgreSQL/aggregates_part1.sql | 5 ++ .../inputs/postgreSQL/aggregates_part2.sql | 5 ++ .../inputs/postgreSQL/aggregates_part3.sql | 5 ++ .../inputs/postgreSQL/aggregates_part4.sql | 5 ++ .../sql-tests/inputs/postgreSQL/join.sql | 15 +++-- .../inputs/postgreSQL/window_part1.sql | 5 ++ .../inputs/postgreSQL/window_part2.sql | 5 ++ .../inputs/postgreSQL/window_part3.sql | 5 ++ .../inputs/postgreSQL/window_part4.sql | 5 ++ .../exists-subquery/exists-aggregate.sql | 5 ++ .../exists-joins-and-set-ops.sql | 16 +++-- .../exists-subquery/exists-orderby-limit.sql | 5 ++ .../subquery/in-subquery/in-group-by.sql | 5 ++ .../inputs/subquery/in-subquery/in-joins.sql | 16 +++-- .../subquery/in-subquery/in-order-by.sql | 5 ++ .../subquery/in-subquery/not-in-group-by.sql | 5 ++ .../subquery/in-subquery/not-in-joins.sql | 4 -- .../test/resources/sql-tests/inputs/udaf.sql | 5 ++ .../inputs/udf/udf-join-empty-relation.sql | 5 -- .../sql-tests/inputs/udf/udf-natural-join.sql | 5 -- .../sql-tests/inputs/udf/udf-outer-join.sql | 4 -- .../resources/sql-tests/inputs/window.sql | 5 ++ .../sql-tests/results/outer-join.sql.out | 22 +------ .../apache/spark/sql/SQLQueryTestSuite.scala | 58 ++++++++----------- .../ThriftServerQueryTestSuite.scala | 18 +++--- 32 files changed, 183 insertions(+), 116 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index d602f63e529d1..fedf03d774e42 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -1,3 +1,8 @@ +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) diff --git a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql index 7665346f86ba8..cfa06aea82b04 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql @@ -1,3 +1,8 @@ +-- Test higher order functions with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create or replace temporary view nested as values (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), diff --git a/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql b/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql index e87c660cb1fe6..5623161839331 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql @@ -1,7 +1,14 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); diff --git a/sql/core/src/test/resources/sql-tests/inputs/join-empty-relation.sql b/sql/core/src/test/resources/sql-tests/inputs/join-empty-relation.sql index 2e6a5f362a8fa..8afa3270f4de4 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/join-empty-relation.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/join-empty-relation.sql @@ -1,8 +1,3 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false - CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); diff --git a/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql index e0abeda3eb44f..71a50157b766c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql @@ -1,8 +1,3 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false - create temporary view nt1 as select * from values ("one", 1), ("two", 2), diff --git a/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql b/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql index f7637b444b9fe..ad3977465c835 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql @@ -1,3 +1,8 @@ +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- Q1. testing window functions with order by create table spark_10747(col1 int, col2 int, col3 int) using parquet; diff --git a/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql b/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql index ce09c21568f13..ceb438ec34b2d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql @@ -1,7 +1,14 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN -- SPARK-17099: Incorrect result when HAVING clause is added to group by query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES @@ -29,9 +36,6 @@ CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1); --- Set the cross join enabled flag for the LEFT JOIN test since there's no join condition. --- Ultimately the join should be optimized away. -set spark.sql.crossJoin.enabled = true; SELECT * FROM ( SELECT @@ -39,6 +43,3 @@ SELECT FROM t1 LEFT JOIN t2 ON false ) t where (t.int_col) is not null; -set spark.sql.crossJoin.enabled = false; - - diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql index 5d54be9341148..63f80bd2efa73 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql @@ -8,6 +8,11 @@ -- avoid bit-exact output here because operations may not be bit-exact. -- SET extra_float_digits = 0; +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + SELECT avg(four) AS avg_1 FROM onek; SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql index ba91366014e16..a8af1db77563c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part2.sql @@ -5,6 +5,11 @@ -- AGGREGATES [Part 2] -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create temporary view int4_tbl as select * from values (0), (123456), diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql index 78fdbf6ae6cd2..6f5e549644bbf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql @@ -5,6 +5,11 @@ -- AGGREGATES [Part 3] -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605 +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- [SPARK-28865] Table inheritance -- try it on an inheritance tree -- create table minmaxtest(f1 int); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql index 6fa2306cf1475..0d255bed24e9c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part4.sql @@ -5,6 +5,11 @@ -- AGGREGATES [Part 4] -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L607-L997 +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- [SPARK-27980] Ordered-Set Aggregate Functions -- ordered-set aggregates diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql index 1ada723d6ae22..cc07b00cc3670 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/join.sql @@ -7,10 +7,17 @@ -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/join.sql -- --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql index ae2a015ada245..087d7a5befd19 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part1.sql @@ -3,6 +3,11 @@ -- Window Functions Testing -- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L1-L319 +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; -- [SPARK-29540] Thrift in some cases can't parse string to date diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql index 728e8cab0c3ba..395149e48d5c8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql @@ -3,6 +3,11 @@ -- Window Functions Testing -- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L320-562 +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TABLE empsalary ( depname string, empno integer, diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql index 205c7d391a973..8187f8a2773ff 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part3.sql @@ -3,6 +3,11 @@ -- Window Functions Testing -- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L564-L911 +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; CREATE TABLE empsalary ( diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql index 456b390fca6c3..64ba8e3b7a5ad 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part4.sql @@ -3,6 +3,11 @@ -- Window Functions Testing -- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L913-L1278 +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- Spark doesn't handle UDFs in SQL -- test user-defined window function with named args and default args -- CREATE FUNCTION nth_value_def(val anyelement, n integer = 1) RETURNS anyelement diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql index b5f458f2cb184..ae6a9641aae66 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql @@ -1,5 +1,10 @@ -- Tests aggregate expressions in outer query and EXISTS subquery. +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql index cefc3fe6272ab..667573b30d265 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql @@ -1,9 +1,17 @@ -- Tests EXISTS subquery support. Tests Exists subquery -- used in Joins (Both when joins occurs in outer and suquery blocks) --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql index 19fc18833760c..580fc1d4162eb 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql @@ -1,5 +1,10 @@ -- Tests EXISTS subquery support with ORDER BY and LIMIT clauses. +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql index b1d96b32c2478..b06e1cccca5ab 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql @@ -1,6 +1,11 @@ -- A test suite for GROUP BY in parent side, subquery, and both predicate subquery -- It includes correlated cases. +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create temporary view t1 as select * from values ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql index cd350a98e130b..200a71ebbb622 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -1,9 +1,17 @@ -- A test suite for IN JOINS in parent side, subquery, and both predicate subquery -- It includes correlated cases. --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql index 892e39ff47c1f..042966b0a4e26 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql @@ -1,6 +1,11 @@ -- A test suite for ORDER BY in parent side, subquery, and both predicate subquery -- It includes correlated cases. +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql index 58cf109e136c5..54b74534c1162 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql @@ -1,6 +1,11 @@ -- A test suite for NOT IN GROUP BY in parent side, subquery, and both predicate subquery -- It includes correlated cases. +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql index bebc18a61894b..fcdb667ad4523 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql @@ -1,9 +1,5 @@ -- A test suite for not-in-joins in parent side, subquery, and both predicate subquery -- It includes correlated cases. --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), diff --git a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql index 58613a1325dfa..0374d98feb6e6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql @@ -1,3 +1,8 @@ +-- Test aggregate operator and UDAF with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1), (2), (3), (4) as t1(int_col1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql index 47fb70d02394b..b46206d4530ed 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql @@ -1,8 +1,3 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false - -- This test file was converted from join-empty-relation.sql. CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql index e5eb812d69a1c..7cf080ea1b4eb 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql @@ -1,8 +1,3 @@ --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false - -- This test file was converted from natural-join.sql. create temporary view nt1 as select * from values diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql index 4eb0805c9cc67..4b09bcb988d25 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql @@ -1,8 +1,4 @@ -- This test file was converted from outer-join.sql. --- List of configuration the test suite is run against: ---SET spark.sql.autoBroadcastJoinThreshold=10485760 ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true ---SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false -- SPARK-17099: Incorrect result when HAVING clause is added to group by query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index faab4c61c8640..e25a252418301 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -1,3 +1,8 @@ +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), diff --git a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out index 5db3bae5d0379..b39fdb0e58720 100644 --- a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 8 +-- Number of queries: 6 -- !query 0 @@ -59,14 +59,6 @@ struct<> -- !query 5 -set spark.sql.crossJoin.enabled = true --- !query 5 schema -struct --- !query 5 output -spark.sql.crossJoin.enabled true - - --- !query 6 SELECT * FROM ( SELECT @@ -74,15 +66,7 @@ SELECT FROM t1 LEFT JOIN t2 ON false ) t where (t.int_col) is not null --- !query 6 schema +-- !query 5 schema struct --- !query 6 output +-- !query 5 output 97 - - --- !query 7 -set spark.sql.crossJoin.enabled = false --- !query 7 schema -struct --- !query 7 output -spark.sql.crossJoin.enabled false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 6e86d37a310d6..ef30573017212 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -111,7 +111,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { import IntegratedUDFTestUtils._ private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" - protected val isTestWithConfigSets: Boolean = true protected val baseResourcePath = { // We use a path based on Spark home for 2 reasons: @@ -236,17 +235,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { } } - // For better test coverage, runs the tests on mixed config sets: WHOLESTAGE_CODEGEN_ENABLED - // and CODEGEN_FACTORY_MODE. - private lazy val codegenConfigSets = Array( - ("true", "CODEGEN_ONLY"), - ("false", "CODEGEN_ONLY"), - ("false", "NO_CODEGEN") - ).map { case (wholeStageCodegenEnabled, codegenFactoryMode) => - Array(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> wholeStageCodegenEnabled, - SQLConf.CODEGEN_FACTORY_MODE.key -> codegenFactoryMode) - } - /** Run a test case. */ protected def runTest(testCase: TestCase): Unit = { val input = fileToString(new File(testCase.inputFile)) @@ -271,36 +259,36 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // Fix misplacement when comment is at the end of the query. .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")).map(_.trim).filter(_ != "") - // When we are regenerating the golden files for test cases without '--IMPORT' specified, or - // running test cases against [[ThriftServerQueryTestSuite], we don't need to set any config as - // they all need to return the same result. - // When we use '--SET' and '--IMPORT' together for those import queries, we want to run the - // same queries from the original file but with different settings and save the answers. So the - // `--SET` will be respected in this case. - if ((regenerateGoldenFiles && importedTestCaseName.isEmpty) || !isTestWithConfigSets) { - runQueries(queries, testCase, None) + val settingLines = comments.filter(_.startsWith("--SET ")).map(_.substring(6)) + val settings = settingLines.flatMap(_.split(",").map { kv => + val (conf, value) = kv.span(_ != '=') + conf.trim -> value.substring(1).trim + }) + + if (regenerateGoldenFiles) { + runQueries(queries, testCase, Some(settings)) } else { - val configSets = { - val configLines = comments.filter(_.startsWith("--SET ")).map(_.substring(6)) - val configs = configLines.map(_.split(",").map { confAndValue => - val (conf, value) = confAndValue.span(_ != '=') + // A config dimension has multiple config sets, and a config set has multiple configs. + // - config dim: Seq[Seq[(String, String)]] + // - config set: Seq[(String, String)] + // - config: (String, String)) + // We need to do cartesian product for all the config dimensions, to get a list of + // config sets, and run the query once for each config set. + val configDimLines = comments.filter(_.startsWith("--CONFIG_DIM")).map(_.substring(12)) + val configDims = configDimLines.groupBy(_.takeWhile(_ != ' ')).mapValues { lines => + lines.map(_.dropWhile(_ != ' ').substring(1)).map(_.split(",").map { kv => + val (conf, value) = kv.span(_ != '=') conf.trim -> value.substring(1).trim - }) + }.toSeq).toSeq + } - if (configs.nonEmpty) { - codegenConfigSets.flatMap { codegenConfig => - configs.map { config => - config ++ codegenConfig - } - } - } else { - codegenConfigSets - } + val configSets = configDims.values.foldLeft(Seq(Seq[(String, String)]())) { (res, dim) => + dim.flatMap { configSet => res.map(_ ++ configSet) } } configSets.foreach { configSet => try { - runQueries(queries, testCase, Some(configSet)) + runQueries(queries, testCase, Some(settings ++ configSet)) } catch { case e: Throwable => val configs = configSet.map { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index dc1609d3326cd..30cca95dbbce7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -83,9 +83,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } - // We only test this test suite with the default configuration to reduce test time. - override val isTestWithConfigSets = false - /** List of test cases to ignore, in lower cases. */ override def blackList: Set[String] = super.blackList ++ Set( // Missing UDF @@ -103,10 +100,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { "subquery/in-subquery/in-group-by.sql", "subquery/in-subquery/simple-in.sql", "subquery/in-subquery/in-order-by.sql", - "subquery/in-subquery/in-set-operations.sql", - // SPARK-29783: need to set conf - "interval-display-iso_8601.sql", - "interval-display-sql_standard.sql" + "subquery/in-subquery/in-set-operations.sql" ) override def runQueries( @@ -118,13 +112,21 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { loadTestData(statement) + configSet.foreach { configs => + for ((k, v) <- configs) { + statement.execute(s"SET $k = $v") + } + } + testCase match { case _: PgSQLTest => - statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") case _: AnsiTest => + statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") case _ => + statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") + statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = false") } // Run the SQL queries preparing them for comparison. From 9ec2a4e58caa4128e9c690d72239cebd6b732084 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Fri, 22 Nov 2019 18:36:50 +0900 Subject: [PATCH 0692/1387] [SPARK-29911][SQL][FOLLOWUP] Move related unit test to ThriftServerWithSparkContextSuite ### What changes were proposed in this pull request? This is follow up of #26543 See https://github.com/apache/spark/pull/26543#discussion_r348934276 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Exist UT. Closes #26628 from LantaoJin/SPARK-29911_FOLLOWUP. Authored-by: LantaoJin Signed-off-by: HyukjinKwon --- .../ThriftServerQueryTestSuite.scala | 23 ---- .../ThriftServerWithSparkContextSuite.scala | 104 ++++++++++++++++++ 2 files changed, 104 insertions(+), 23 deletions(-) create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 30cca95dbbce7..7c48579fd2ca8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -271,29 +271,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } } - test("SPARK-29911: Uncache cached tables when session closed") { - val cacheManager = spark.sharedState.cacheManager - val globalTempDB = spark.sharedState.globalTempViewManager.database - withJdbcStatement { statement => - statement.execute("CACHE TABLE tempTbl AS SELECT 1") - } - // the cached data of local temporary view should be uncached - assert(cacheManager.isEmpty) - try { - withJdbcStatement { statement => - statement.execute("CREATE GLOBAL TEMP VIEW globalTempTbl AS SELECT 1, 2") - statement.execute(s"CACHE TABLE $globalTempDB.globalTempTbl") - } - // the cached data of global temporary view shouldn't be uncached - assert(!cacheManager.isEmpty) - } finally { - withJdbcStatement { statement => - statement.execute(s"UNCACHE TABLE IF EXISTS $globalTempDB.globalTempTbl") - } - assert(cacheManager.isEmpty) - } - } - /** ThriftServer wraps the root exception, so it needs to be extracted. */ override def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { super.handleExceptions { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala new file mode 100644 index 0000000000000..b6ed5e3156a46 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.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.sql.hive.thriftserver + +import java.sql.{DriverManager, Statement} + +import scala.util.{Random, Try} + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSparkSession + +class ThriftServerWithSparkContextSuite extends QueryTest with SharedSparkSession { + + private var hiveServer2: HiveThriftServer2 = _ + + override def beforeAll(): Unit = { + super.beforeAll() + // Chooses a random port between 10000 and 19999 + var listeningPort = 10000 + Random.nextInt(10000) + + // Retries up to 3 times with different port numbers if the server fails to start + (1 to 3).foldLeft(Try(startThriftServer(listeningPort, 0))) { case (started, attempt) => + started.orElse { + listeningPort += 1 + Try(startThriftServer(listeningPort, attempt)) + } + }.recover { + case cause: Throwable => + throw cause + }.get + logInfo("HiveThriftServer2 started successfully") + } + + override def afterAll(): Unit = { + try { + hiveServer2.stop() + } finally { + super.afterAll() + } + } + + test("SPARK-29911: Uncache cached tables when session closed") { + val cacheManager = spark.sharedState.cacheManager + val globalTempDB = spark.sharedState.globalTempViewManager.database + withJdbcStatement { statement => + statement.execute("CACHE TABLE tempTbl AS SELECT 1") + } + // the cached data of local temporary view should be uncached + assert(cacheManager.isEmpty) + try { + withJdbcStatement { statement => + statement.execute("CREATE GLOBAL TEMP VIEW globalTempTbl AS SELECT 1, 2") + statement.execute(s"CACHE TABLE $globalTempDB.globalTempTbl") + } + // the cached data of global temporary view shouldn't be uncached + assert(!cacheManager.isEmpty) + } finally { + withJdbcStatement { statement => + statement.execute(s"UNCACHE TABLE IF EXISTS $globalTempDB.globalTempTbl") + } + assert(cacheManager.isEmpty) + } + } + + private def startThriftServer(port: Int, attempt: Int): Unit = { + logInfo(s"Trying to start HiveThriftServer2: port=$port, attempt=$attempt") + val sqlContext = spark.newSession().sqlContext + sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, port.toString) + hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) + } + + private def withJdbcStatement(fs: (Statement => Unit)*): Unit = { + val user = System.getProperty("user.name") + + val serverPort = hiveServer2.getHiveConf.get(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname) + val connections = + fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } + val statements = connections.map(_.createStatement()) + + try { + statements.zip(fs).foreach { case (s, f) => f(s) } + } finally { + statements.foreach(_.close()) + connections.foreach(_.close()) + } + } +} From 2dd6807e421c96d0aaafc57ceb48f50f66f9d2e7 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 22 Nov 2019 19:32:27 +0800 Subject: [PATCH 0693/1387] [SPARK-28023][SQL] Add trim logic in UTF8String's toInt/toLong to make it consistent with other string-numeric casting ### What changes were proposed in this pull request? Modify `UTF8String.toInt/toLong` to support trim spaces for both sides before converting it to byte/short/int/long. With this kind of "cheap" trim can help improve performance for casting string to integrals. The idea is from https://github.com/apache/spark/pull/24872#issuecomment-556917834 ### Why are the changes needed? make the behavior consistent. ### Does this PR introduce any user-facing change? yes, cast string to an integral type, and binary comparison between string and integrals will trim spaces first. their behavior will be consistent with float and double. ### How was this patch tested? 1. add ut. 2. benchmark tests the benchmark is modified based on https://github.com/apache/spark/pull/24872#issuecomment-503827016 ```scala /* * 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.benchmark import org.apache.spark.benchmark.Benchmark /** * Benchmark trim the string when casting string type to Boolean/Numeric types. * To run this benchmark: * {{{ * 1. without sbt: * bin/spark-submit --class --jars * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/CastBenchmark-results.txt". * }}} */ object CastBenchmark extends SqlBasedBenchmark { This conversation was marked as resolved by yaooqinn override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val title = "Cast String to Integral" runBenchmark(title) { withTempPath { dir => val N = 500L << 14 val df = spark.range(N) val types = Seq("int", "long") (1 to 5).by(2).foreach { i => df.selectExpr(s"concat(id, '${" " * i}') as str") .write.mode("overwrite").parquet(dir + i.toString) } val benchmark = new Benchmark(title, N, minNumIters = 5, output = output) Seq(true, false).foreach { trim => types.foreach { t => val str = if (trim) "trim(str)" else "str" val expr = s"cast($str as $t) as c_$t" (1 to 5).by(2).foreach { i => benchmark.addCase(expr + s" - with $i spaces") { _ => spark.read.parquet(dir + i.toString).selectExpr(expr).collect() } } } } benchmark.run() } } } } ``` #### benchmark result. normal trim v.s. trim in toInt/toLong ```java ================================================================================================ Cast String to Integral ================================================================================================ Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1 Intel(R) Core(TM) i5-5287U CPU 2.90GHz Cast String to Integral: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ cast(trim(str) as int) as c_int - with 1 spaces 10220 12994 1337 0.8 1247.5 1.0X cast(trim(str) as int) as c_int - with 3 spaces 4763 8356 357 1.7 581.4 2.1X cast(trim(str) as int) as c_int - with 5 spaces 4791 8042 NaN 1.7 584.9 2.1X cast(trim(str) as long) as c_long - with 1 spaces 4014 6755 NaN 2.0 490.0 2.5X cast(trim(str) as long) as c_long - with 3 spaces 4737 6938 NaN 1.7 578.2 2.2X cast(trim(str) as long) as c_long - with 5 spaces 4478 6919 1404 1.8 546.6 2.3X cast(str as int) as c_int - with 1 spaces 4443 6222 NaN 1.8 542.3 2.3X cast(str as int) as c_int - with 3 spaces 3659 3842 170 2.2 446.7 2.8X cast(str as int) as c_int - with 5 spaces 4372 7996 NaN 1.9 533.7 2.3X cast(str as long) as c_long - with 1 spaces 3866 5838 NaN 2.1 471.9 2.6X cast(str as long) as c_long - with 3 spaces 3793 5449 NaN 2.2 463.0 2.7X cast(str as long) as c_long - with 5 spaces 4947 5961 1198 1.7 603.9 2.1X ``` Closes #26622 from yaooqinn/cheapstringtrim. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../apache/spark/unsafe/types/UTF8String.java | 44 +++++++------ docs/sql-migration-guide.md | 2 + .../test/resources/sql-tests/inputs/cast.sql | 10 +++ .../resources/sql-tests/inputs/comparator.sql | 10 +++ .../resources/sql-tests/results/cast.sql.out | 66 ++++++++++++++++++- .../sql-tests/results/comparator.sql.out | 66 ++++++++++++++++++- 6 files changed, 176 insertions(+), 22 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index d7a498d1c1c2f..deecd4f015824 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1063,7 +1063,7 @@ public static class IntWrapper implements Serializable { } /** - * Parses this UTF8String to long. + * Parses this UTF8String(trimmed if needed) to long. * * Note that, in this method we accumulate the result in negative format, and convert it to * positive format at the end, if this string is not started with '-'. This is because min value @@ -1077,18 +1077,20 @@ public static class IntWrapper implements Serializable { * @return true if the parsing was successful else false */ public boolean toLong(LongWrapper toLongResult) { - if (numBytes == 0) { - return false; - } + int offset = 0; + while (offset < this.numBytes && getByte(offset) <= ' ') offset++; + if (offset == this.numBytes) return false; - byte b = getByte(0); + int end = this.numBytes - 1; + while (end > offset && getByte(end) <= ' ') end--; + + byte b = getByte(offset); final boolean negative = b == '-'; - int offset = 0; if (negative || b == '+') { - offset++; - if (numBytes == 1) { + if (end - offset == 0) { return false; } + offset++; } final byte separator = '.'; @@ -1096,7 +1098,7 @@ public boolean toLong(LongWrapper toLongResult) { final long stopValue = Long.MIN_VALUE / radix; long result = 0; - while (offset < numBytes) { + while (offset <= end) { b = getByte(offset); offset++; if (b == separator) { @@ -1131,7 +1133,7 @@ public boolean toLong(LongWrapper toLongResult) { // This is the case when we've encountered a decimal separator. The fractional // part will not change the number, but we will verify that the fractional part // is well formed. - while (offset < numBytes) { + while (offset <= end) { byte currentByte = getByte(offset); if (currentByte < '0' || currentByte > '9') { return false; @@ -1151,7 +1153,7 @@ public boolean toLong(LongWrapper toLongResult) { } /** - * Parses this UTF8String to int. + * Parses this UTF8String(trimmed if needed) to int. * * Note that, in this method we accumulate the result in negative format, and convert it to * positive format at the end, if this string is not started with '-'. This is because min value @@ -1168,18 +1170,20 @@ public boolean toLong(LongWrapper toLongResult) { * @return true if the parsing was successful else false */ public boolean toInt(IntWrapper intWrapper) { - if (numBytes == 0) { - return false; - } + int offset = 0; + while (offset < this.numBytes && getByte(offset) <= ' ') offset++; + if (offset == this.numBytes) return false; - byte b = getByte(0); + int end = this.numBytes - 1; + while (end > offset && getByte(end) <= ' ') end--; + + byte b = getByte(offset); final boolean negative = b == '-'; - int offset = 0; if (negative || b == '+') { - offset++; - if (numBytes == 1) { + if (end - offset == 0) { return false; } + offset++; } final byte separator = '.'; @@ -1187,7 +1191,7 @@ public boolean toInt(IntWrapper intWrapper) { final int stopValue = Integer.MIN_VALUE / radix; int result = 0; - while (offset < numBytes) { + while (offset <= end) { b = getByte(offset); offset++; if (b == separator) { @@ -1222,7 +1226,7 @@ public boolean toInt(IntWrapper intWrapper) { // This is the case when we've encountered a decimal separator. The fractional // part will not change the number, but we will verify that the fractional part // is well formed. - while (offset < numBytes) { + while (offset <= end) { byte currentByte = getByte(offset); if (currentByte < '0' || currentByte > '9') { return false; diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 2d5afa919e668..6fc78893e6881 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -222,6 +222,8 @@ license: | - Since Spark 3.0, when casting interval values to string type, there is no "interval" prefix, e.g. `1 days 2 hours`. In Spark version 2.4 and earlier, the string contains the "interval" prefix like `interval 1 days 2 hours`. + - Since Spark 3.0, when casting string value to integral types, including tinyint, smallint, int and bigint type, the leading and trailing white spaces(<= ACSII 32) will be trimmed before convert to integral values, e.g. `cast(' 1 ' as int)` results `1`. In Spark version 2.4 and earlier, the result will be `null`. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/core/src/test/resources/sql-tests/inputs/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/cast.sql index 8a035f594be54..3c1702e6f837e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cast.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cast.sql @@ -60,3 +60,13 @@ DESC FUNCTION EXTENDED boolean; -- cast string to interval and interval to string SELECT CAST('interval 3 month 1 hour' AS interval); SELECT CAST(interval 3 month 1 hour AS string); + +-- trim string before cast to numeric +select cast(' 1' as tinyint); +select cast(' 1\t' as tinyint); +select cast(' 1' as smallint); +select cast(' 1' as INT); +select cast(' 1' as bigint); +select cast(' 1' as float); +select cast(' 1 ' as DOUBLE); +select cast('1.0 ' as DEC); \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/comparator.sql b/sql/core/src/test/resources/sql-tests/inputs/comparator.sql index 3e2447723e576..70af4f75ac431 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/comparator.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/comparator.sql @@ -1,3 +1,13 @@ -- binary type select x'00' < x'0f'; select x'00' < x'ff'; + +-- trim string to numeric +select '1 ' = 1Y; +select '\t1 ' = 1Y; +select '1 ' = 1S; +select '1 ' = 1; +select ' 1' = 1L; +select ' 1' = cast(1.0 as float); +select ' 1.0 ' = 1.0D; +select ' 1.0 ' = 1.0BD; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 609d283da555c..bf1e873a48d17 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 35 +-- Number of queries: 43 -- !query 0 @@ -287,3 +287,67 @@ SELECT CAST(interval 3 month 1 hour AS string) struct -- !query 34 output 3 months 1 hours + + +-- !query 35 +select cast(' 1' as tinyint) +-- !query 35 schema +struct +-- !query 35 output +1 + + +-- !query 36 +select cast(' 1\t' as tinyint) +-- !query 36 schema +struct +-- !query 36 output +1 + + +-- !query 37 +select cast(' 1' as smallint) +-- !query 37 schema +struct +-- !query 37 output +1 + + +-- !query 38 +select cast(' 1' as INT) +-- !query 38 schema +struct +-- !query 38 output +1 + + +-- !query 39 +select cast(' 1' as bigint) +-- !query 39 schema +struct +-- !query 39 output +1 + + +-- !query 40 +select cast(' 1' as float) +-- !query 40 schema +struct +-- !query 40 output +1.0 + + +-- !query 41 +select cast(' 1 ' as DOUBLE) +-- !query 41 schema +struct +-- !query 41 output +1.0 + + +-- !query 42 +select cast('1.0 ' as DEC) +-- !query 42 schema +struct +-- !query 42 output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/comparator.sql.out b/sql/core/src/test/resources/sql-tests/results/comparator.sql.out index afc7b5448b7b6..a182da2a79637 100644 --- a/sql/core/src/test/resources/sql-tests/results/comparator.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/comparator.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 2 +-- Number of queries: 10 -- !query 0 @@ -16,3 +16,67 @@ select x'00' < x'ff' struct<(X'00' < X'FF'):boolean> -- !query 1 output true + + +-- !query 2 +select '1 ' = 1Y +-- !query 2 schema +struct<(CAST(1 AS TINYINT) = 1):boolean> +-- !query 2 output +true + + +-- !query 3 +select '\t1 ' = 1Y +-- !query 3 schema +struct<(CAST( 1 AS TINYINT) = 1):boolean> +-- !query 3 output +true + + +-- !query 4 +select '1 ' = 1S +-- !query 4 schema +struct<(CAST(1 AS SMALLINT) = 1):boolean> +-- !query 4 output +true + + +-- !query 5 +select '1 ' = 1 +-- !query 5 schema +struct<(CAST(1 AS INT) = 1):boolean> +-- !query 5 output +true + + +-- !query 6 +select ' 1' = 1L +-- !query 6 schema +struct<(CAST( 1 AS BIGINT) = 1):boolean> +-- !query 6 output +true + + +-- !query 7 +select ' 1' = cast(1.0 as float) +-- !query 7 schema +struct<(CAST( 1 AS FLOAT) = CAST(1.0 AS FLOAT)):boolean> +-- !query 7 output +true + + +-- !query 8 +select ' 1.0 ' = 1.0D +-- !query 8 schema +struct<(CAST( 1.0 AS DOUBLE) = 1.0):boolean> +-- !query 8 output +true + + +-- !query 9 +select ' 1.0 ' = 1.0BD +-- !query 9 schema +struct<(CAST( 1.0 AS DOUBLE) = CAST(1.0 AS DOUBLE)):boolean> +-- !query 9 output +true From 6e581cf164c3a2930966b270ac1406dc1195c942 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 22 Nov 2019 10:26:54 -0800 Subject: [PATCH 0694/1387] [SPARK-29893][SQL][FOLLOWUP] code cleanup for local shuffle reader ### What changes were proposed in this pull request? A few cleanups for https://github.com/apache/spark/pull/26516: 1. move the calculating of partition start indices from the RDD to the rule. We can reuse code from "shrink number of reducers" in the future if we split partitions by size. 2. only check extra shuffles when adding local readers to the probe side. 3. add comments. 4. simplify the config name: `optimizedLocalShuffleReader` -> `localShuffleReader` ### Why are the changes needed? make code more maintainable. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26625 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: Xiao Li --- .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../adaptive/LocalShuffledRowRDD.scala | 32 ++---- .../adaptive/OptimizeLocalShuffleReader.scala | 98 +++++++++++++------ .../exchange/ShuffleExchangeExec.scala | 4 +- .../adaptive/AdaptiveQueryExecSuite.scala | 13 +-- 5 files changed, 87 insertions(+), 64 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 74046cd91c962..33f91d045f7d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -393,8 +393,8 @@ object SQLConf { "must be a positive integer.") .createOptional - val OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED = - buildConf("spark.sql.adaptive.shuffle.optimizedLocalShuffleReader.enabled") + val LOCAL_SHUFFLE_READER_ENABLED = + buildConf("spark.sql.adaptive.shuffle.localShuffleReader.enabled") .doc("When true and adaptive execution is enabled, this enables the optimization of" + " converting the shuffle reader to local shuffle reader for the shuffle exchange" + " of the broadcast hash join in probe side.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala index 44254593f8895..6385ea67c49fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -44,7 +44,7 @@ private final class LocalShuffledRowRDDPartition( * data of another input table of the join that reads from shuffle. Each partition of the RDD reads * the whole data from just one mapper output locally. So actually there is no data transferred * from the network. - + * * This RDD takes a [[ShuffleDependency]] (`dependency`). * * The `dependency` has the parent RDD of this RDD, which represents the dataset before shuffle @@ -53,11 +53,15 @@ private final class LocalShuffledRowRDDPartition( * `dependency.partitioner.numPartitions` is the number of pre-shuffle partitions. (i.e. the number * of partitions of the map output). The post-shuffle partition number is the same to the parent * RDD's partition number. + * + * `partitionStartIndicesPerMapper` specifies how to split the shuffle blocks of each mapper into + * one or more partitions. For a mapper `i`, the `j`th partition includes shuffle blocks from + * `partitionStartIndicesPerMapper[i][j]` to `partitionStartIndicesPerMapper[i][j+1]` (exclusive). */ class LocalShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], metrics: Map[String, SQLMetric], - advisoryParallelism : Option[Int] = None) + partitionStartIndicesPerMapper: Array[Array[Int]]) extends RDD[InternalRow](dependency.rdd.context, Nil) { private[this] val numReducers = dependency.partitioner.numPartitions @@ -65,30 +69,12 @@ class LocalShuffledRowRDD( override def getDependencies: Seq[Dependency[_]] = List(dependency) - /** - * To equally divide n elements into m buckets, basically each bucket should have n/m elements, - * for the remaining n%m elements, add one more element to the first n%m buckets each. Returns - * a sequence with length numBuckets and each value represents the start index of each bucket. - */ - private def equallyDivide(numElements: Int, numBuckets: Int): Seq[Int] = { - val elementsPerBucket = numElements / numBuckets - val remaining = numElements % numBuckets - val splitPoint = (elementsPerBucket + 1) * remaining - (0 until remaining).map(_ * (elementsPerBucket + 1)) ++ - (remaining until numBuckets).map(i => splitPoint + (i - remaining) * elementsPerBucket) - } - override def getPartitions: Array[Partition] = { - val partitionStartIndices: Array[Int] = { - val expectedParallelism = advisoryParallelism.getOrElse(numReducers) - // TODO split by data size in the future. - equallyDivide(numReducers, math.max(1, expectedParallelism / numMappers)).toArray - } - val partitions = ArrayBuffer[LocalShuffledRowRDDPartition]() for (mapIndex <- 0 until numMappers) { - (partitionStartIndices :+ numReducers).sliding(2, 1).foreach { case Array(start, end) => - partitions += new LocalShuffledRowRDDPartition(partitions.length, mapIndex, start, end) + (partitionStartIndicesPerMapper(mapIndex) :+ numReducers).sliding(2, 1).foreach { + case Array(start, end) => + partitions += new LocalShuffledRowRDDPartition(partitions.length, mapIndex, start, end) } } partitions.toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index a57cfae6d0991..176e5ec8312e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -39,8 +39,12 @@ import org.apache.spark.sql.internal.SQLConf case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { import OptimizeLocalShuffleReader._ - def withProbeSideLocalReader(plan: SparkPlan): SparkPlan = { - plan.transformDown { + private val ensureRequirements = EnsureRequirements(conf) + + // The build side is a broadcast query stage which should have been optimized using local reader + // already. So we only need to deal with probe side here. + private def createProbeSideLocalReader(plan: SparkPlan): SparkPlan = { + val optimizedPlan = plan.transformDown { case join @ BroadcastJoinWithShuffleLeft(shuffleStage, BuildRight) => val localReader = createLocalReader(shuffleStage) join.asInstanceOf[BroadcastHashJoinExec].copy(left = localReader) @@ -48,39 +52,71 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { val localReader = createLocalReader(shuffleStage) join.asInstanceOf[BroadcastHashJoinExec].copy(right = localReader) } + + val numShuffles = ensureRequirements.apply(optimizedPlan).collect { + case e: ShuffleExchangeExec => e + }.length + + // Check whether additional shuffle introduced. If introduced, revert the local reader. + if (numShuffles > 0) { + logDebug("OptimizeLocalShuffleReader rule is not applied due" + + " to additional shuffles will be introduced.") + plan + } else { + optimizedPlan + } } - def createLocalReader(plan: SparkPlan): LocalShuffleReaderExec = { + private def createLocalReader(plan: SparkPlan): LocalShuffleReaderExec = { plan match { - case c: CoalescedShuffleReaderExec => - LocalShuffleReaderExec(c.child, Some(c.partitionStartIndices.length)) - case q: QueryStageExec => LocalShuffleReaderExec(q) + case c @ CoalescedShuffleReaderExec(q: QueryStageExec, _) => + LocalShuffleReaderExec( + q, getPartitionStartIndices(q, Some(c.partitionStartIndices.length))) + case q: QueryStageExec => + LocalShuffleReaderExec(q, getPartitionStartIndices(q, None)) + } + } + + // TODO: this method assumes all shuffle blocks are the same data size. We should calculate the + // partition start indices based on block size to avoid data skew. + private def getPartitionStartIndices( + shuffle: QueryStageExec, + advisoryParallelism: Option[Int]): Array[Array[Int]] = { + val shuffleDep = shuffle match { + case s: ShuffleQueryStageExec => s.plan.shuffleDependency + case ReusedQueryStageExec(_, s: ShuffleQueryStageExec, _) => s.plan.shuffleDependency } + val numReducers = shuffleDep.partitioner.numPartitions + val expectedParallelism = advisoryParallelism.getOrElse(numReducers) + val numMappers = shuffleDep.rdd.getNumPartitions + Array.fill(numMappers) { + equallyDivide(numReducers, math.max(1, expectedParallelism / numMappers)).toArray + } + } + + /** + * To equally divide n elements into m buckets, basically each bucket should have n/m elements, + * for the remaining n%m elements, add one more element to the first n%m buckets each. Returns + * a sequence with length numBuckets and each value represents the start index of each bucket. + */ + private def equallyDivide(numElements: Int, numBuckets: Int): Seq[Int] = { + val elementsPerBucket = numElements / numBuckets + val remaining = numElements % numBuckets + val splitPoint = (elementsPerBucket + 1) * remaining + (0 until remaining).map(_ * (elementsPerBucket + 1)) ++ + (remaining until numBuckets).map(i => splitPoint + (i - remaining) * elementsPerBucket) } override def apply(plan: SparkPlan): SparkPlan = { - if (!conf.getConf(SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED)) { + if (!conf.getConf(SQLConf.LOCAL_SHUFFLE_READER_ENABLED)) { return plan } - val optimizedPlan = plan match { + plan match { case s: SparkPlan if canUseLocalShuffleReader(s) => createLocalReader(s) - case s: SparkPlan => withProbeSideLocalReader(s) - } - - def numExchanges(plan: SparkPlan): Int = { - plan.collect { - case e: ShuffleExchangeExec => e - }.length - } - // Check whether additional shuffle introduced. If introduced, revert the local reader. - if (numExchanges(EnsureRequirements(conf).apply(optimizedPlan)) > 0) { - logDebug("OptimizeLocalShuffleReader rule is not applied due" + - " to additional shuffles will be introduced.") - plan - } else { - optimizedPlan + case s: SparkPlan => + createProbeSideLocalReader(s) } } } @@ -110,16 +146,20 @@ object OptimizeLocalShuffleReader { } /** - * A wrapper of shuffle query stage, which submits one reduce task per mapper to read the shuffle - * files written by one mapper. By doing this, it's very likely to read the shuffle files locally, - * as the shuffle files that a reduce task needs to read are in one node. + * A wrapper of shuffle query stage, which submits one or more reduce tasks per mapper to read the + * shuffle files written by one mapper. By doing this, it's very likely to read the shuffle files + * locally, as the shuffle files that a reduce task needs to read are in one node. * * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the * shuffle exchange node during canonicalization. + * @param partitionStartIndicesPerMapper A mapper usually writes many shuffle blocks, and it's + * better to launch multiple tasks to read shuffle blocks of + * one mapper. This array contains the partition start + * indices for each mapper. */ case class LocalShuffleReaderExec( child: SparkPlan, - advisoryParallelism: Option[Int] = None) extends UnaryExecNode { + partitionStartIndicesPerMapper: Array[Array[Int]]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -136,9 +176,9 @@ case class LocalShuffleReaderExec( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createLocalShuffleRDD(advisoryParallelism) + stage.plan.createLocalShuffleRDD(partitionStartIndicesPerMapper) case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createLocalShuffleRDD(advisoryParallelism) + stage.plan.createLocalShuffleRDD(partitionStartIndicesPerMapper) } } cachedShuffleRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index baca974a6450b..b876183c78ec2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -84,8 +84,8 @@ case class ShuffleExchangeExec( } def createLocalShuffleRDD( - advisoryParallelism: Option[Int] = None): LocalShuffledRowRDD = { - new LocalShuffledRowRDD(shuffleDependency, readMetrics, advisoryParallelism) + partitionStartIndicesPerMapper: Array[Array[Int]]): LocalShuffledRowRDD = { + new LocalShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndicesPerMapper) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index d5be45b0cdda0..4fa4dadf7f654 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -119,19 +119,16 @@ class AdaptiveQueryExecSuite case reader: LocalShuffleReaderExec => reader } assert(localReaders.length == 2) - // The pre-shuffle partition size is [0, 0, 0, 72, 0] - // And the partitionStartIndices is [0, 3, 4] - assert(localReaders(0).advisoryParallelism.get == 3) - // The pre-shuffle partition size is [0, 72, 0, 72, 126] - // And the partitionStartIndices is [0, 1, 2, 3, 4] - assert(localReaders(1).advisoryParallelism.get == 5) - val localShuffleRDD0 = localReaders(0).execute().asInstanceOf[LocalShuffledRowRDD] val localShuffleRDD1 = localReaders(1).execute().asInstanceOf[LocalShuffledRowRDD] + // The pre-shuffle partition size is [0, 0, 0, 72, 0] + // And the partitionStartIndices is [0, 3, 4], so advisoryParallelism = 3. // the final parallelism is // math.max(1, advisoryParallelism / numMappers): math.max(1, 3/2) = 1 // and the partitions length is 1 * numMappers = 2 assert(localShuffleRDD0.getPartitions.length == 2) + // The pre-shuffle partition size is [0, 72, 0, 72, 126] + // And the partitionStartIndices is [0, 1, 2, 3, 4], so advisoryParallelism = 5. // the final parallelism is // math.max(1, advisoryParallelism / numMappers): math.max(1, 5/2) = 2 // and the partitions length is 2 * numMappers = 4 @@ -492,7 +489,7 @@ class AdaptiveQueryExecSuite test("Change merge join to broadcast join without local shuffle reader") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.OPTIMIZE_LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.LOCAL_SHUFFLE_READER_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( """ From 6b0e391aa49acd5029d00fefc0c90fcdfdf88cb6 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 22 Nov 2019 10:34:26 -0800 Subject: [PATCH 0695/1387] [SPARK-29427][SQL] Add API to convert RelationalGroupedDataset to KeyValueGroupedDataset ### What changes were proposed in this pull request? This PR proposes to add `as` API to RelationalGroupedDataset. It creates KeyValueGroupedDataset instance using given grouping expressions, instead of a typed function in groupByKey API. Because it can leverage existing columns, it can use existing data partition, if any, when doing operations like cogroup. ### Why are the changes needed? Currently if users want to do cogroup on DataFrames, there is no good way to do except for KeyValueGroupedDataset. 1. KeyValueGroupedDataset ignores existing data partition if any. That is a problem. 2. groupByKey calls typed function to create additional keys. You can not reuse existing columns, if you just need grouping by them. ```scala // df1 and df2 are certainly partitioned and sorted. val df1 = Seq((1, 2, 3), (2, 3, 4)).toDF("a", "b", "c") .repartition($"a").sortWithinPartitions("a") val df2 = Seq((1, 2, 4), (2, 3, 5)).toDF("a", "b", "c") .repartition($"a").sortWithinPartitions("a") ``` ```scala // This groupBy.as.cogroup won't unnecessarily repartition the data val df3 = df1.groupBy("a").as[Int] .cogroup(df2.groupBy("a").as[Int]) { case (key, data1, data2) => data1.zip(data2).map { p => p._1.getInt(2) + p._2.getInt(2) } } ``` ``` == Physical Plan == *(5) SerializeFromObject [input[0, int, false] AS value#11247] +- CoGroup org.apache.spark.sql.DataFrameSuite$$Lambda$4922/12067092816eec1b6f, a#11209: int, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [a#11209], [a#11225], [a#11209, b#11210, c#11211], [a#11225, b#11226, c#11227], obj#11246: int :- *(2) Sort [a#11209 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(a#11209, 5), false, [id=#10218] : +- *(1) Project [_1#11202 AS a#11209, _2#11203 AS b#11210, _3#11204 AS c#11211] : +- *(1) LocalTableScan [_1#11202, _2#11203, _3#11204] +- *(4) Sort [a#11225 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(a#11225, 5), false, [id=#10223] +- *(3) Project [_1#11218 AS a#11225, _2#11219 AS b#11226, _3#11220 AS c#11227] +- *(3) LocalTableScan [_1#11218, _2#11219, _3#11220] ``` ```scala // Current approach creates additional AppendColumns and repartition data again val df4 = df1.groupByKey(r => r.getInt(0)).cogroup(df2.groupByKey(r => r.getInt(0))) { case (key, data1, data2) => data1.zip(data2).map { p => p._1.getInt(2) + p._2.getInt(2) } } ``` ``` == Physical Plan == *(7) SerializeFromObject [input[0, int, false] AS value#11257] +- CoGroup org.apache.spark.sql.DataFrameSuite$$Lambda$4933/138102700737171997, value#11252: int, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [value#11252], [value#11254], [a#11209, b#11210, c#11211], [a#11225, b#11226, c#11227], obj#11256: int :- *(3) Sort [value#11252 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(value#11252, 5), true, [id=#10302] : +- AppendColumns org.apache.spark.sql.DataFrameSuite$$Lambda$4930/19529195347ce07f47, createexternalrow(a#11209, b#11210, c#11211, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [input[0, int, false] AS value#11252] : +- *(2) Sort [a#11209 ASC NULLS FIRST], false, 0 : +- Exchange hashpartitioning(a#11209, 5), false, [id=#10297] : +- *(1) Project [_1#11202 AS a#11209, _2#11203 AS b#11210, _3#11204 AS c#11211] : +- *(1) LocalTableScan [_1#11202, _2#11203, _3#11204] +- *(6) Sort [value#11254 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(value#11254, 5), true, [id=#10312] +- AppendColumns org.apache.spark.sql.DataFrameSuite$$Lambda$4932/15265288491f0e0c1f, createexternalrow(a#11225, b#11226, c#11227, StructField(a,IntegerType,false), StructField(b,IntegerType,false), StructField(c,IntegerType,false)), [input[0, int, false] AS value#11254] +- *(5) Sort [a#11225 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(a#11225, 5), false, [id=#10307] +- *(4) Project [_1#11218 AS a#11225, _2#11219 AS b#11226, _3#11220 AS c#11227] +- *(4) LocalTableScan [_1#11218, _2#11219, _3#11220] ``` ### Does this PR introduce any user-facing change? Yes, this adds a new `as` API to RelationalGroupedDataset. Users can use it to create KeyValueGroupedDataset and do cogroup. ### How was this patch tested? Unit tests. Closes #26509 from viirya/SPARK-29427-2. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../spark/sql/RelationalGroupedDataset.scala | 32 ++++++++++ .../org/apache/spark/sql/DataFrameSuite.scala | 59 +++++++++++++++++++ .../org/apache/spark/sql/DatasetSuite.scala | 21 +++++++ 3 files changed, 112 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 4d4731870700c..b1ba7d4538732 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -26,6 +26,7 @@ import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} +import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -129,6 +130,37 @@ class RelationalGroupedDataset protected[sql]( (inputExpr: Expression) => exprToFunc(inputExpr) } + /** + * Returns a `KeyValueGroupedDataset` where the data is grouped by the grouping expressions + * of current `RelationalGroupedDataset`. + * + * @since 3.0.0 + */ + def as[K: Encoder, T: Encoder]: KeyValueGroupedDataset[K, T] = { + val keyEncoder = encoderFor[K] + val valueEncoder = encoderFor[T] + + // Resolves grouping expressions. + val dummyPlan = Project(groupingExprs.map(alias), LocalRelation(df.logicalPlan.output)) + val analyzedPlan = df.sparkSession.sessionState.analyzer.execute(dummyPlan) + .asInstanceOf[Project] + df.sparkSession.sessionState.analyzer.checkAnalysis(analyzedPlan) + val aliasedGroupings = analyzedPlan.projectList + + // Adds the grouping expressions that are not in base DataFrame into outputs. + val addedCols = aliasedGroupings.filter(g => !df.logicalPlan.outputSet.contains(g.toAttribute)) + val qe = Dataset.ofRows( + df.sparkSession, + Project(df.logicalPlan.output ++ addedCols, df.logicalPlan)).queryExecution + + new KeyValueGroupedDataset( + keyEncoder, + valueEncoder, + qe, + df.logicalPlan.output, + aliasedGroupings.map(_.toAttribute)) + } + /** * (Scala-specific) Compute aggregates by specifying the column names and * aggregate methods. The resulting `DataFrame` will also contain the grouping columns. 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 c79e2e9c76bb9..5983fe63c79ec 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 @@ -30,6 +30,7 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.Uuid import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} @@ -2221,4 +2222,62 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { val idTuples = sampled.collect().map(row => row.getLong(0) -> row.getLong(1)) assert(idTuples.length == idTuples.toSet.size) } + + test("groupBy.as") { + val df1 = Seq((1, 2, 3), (2, 3, 4)).toDF("a", "b", "c") + .repartition($"a", $"b").sortWithinPartitions("a", "b") + val df2 = Seq((1, 2, 4), (2, 3, 5)).toDF("a", "b", "c") + .repartition($"a", $"b").sortWithinPartitions("a", "b") + + implicit val valueEncoder = RowEncoder(df1.schema) + + val df3 = df1.groupBy("a", "b").as[GroupByKey, Row] + .cogroup(df2.groupBy("a", "b").as[GroupByKey, Row]) { case (_, data1, data2) => + data1.zip(data2).map { p => + p._1.getInt(2) + p._2.getInt(2) + } + }.toDF + + checkAnswer(df3.sort("value"), Row(7) :: Row(9) :: Nil) + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = df3.queryExecution.executedPlan.collect { + case h: ShuffleExchangeExec => h + } + assert(exchanges.size == 2) + } + + test("groupBy.as: custom grouping expressions") { + val df1 = Seq((1, 2, 3), (2, 3, 4)).toDF("a1", "b", "c") + .repartition($"a1", $"b").sortWithinPartitions("a1", "b") + val df2 = Seq((1, 2, 4), (2, 3, 5)).toDF("a1", "b", "c") + .repartition($"a1", $"b").sortWithinPartitions("a1", "b") + + implicit val valueEncoder = RowEncoder(df1.schema) + + val groupedDataset1 = df1.groupBy(($"a1" + 1).as("a"), $"b").as[GroupByKey, Row] + val groupedDataset2 = df2.groupBy(($"a1" + 1).as("a"), $"b").as[GroupByKey, Row] + + val df3 = groupedDataset1 + .cogroup(groupedDataset2) { case (_, data1, data2) => + data1.zip(data2).map { p => + p._1.getInt(2) + p._2.getInt(2) + } + }.toDF + + checkAnswer(df3.sort("value"), Row(7) :: Row(9) :: Nil) + } + + test("groupBy.as: throw AnalysisException for unresolved grouping expr") { + val df = Seq((1, 2, 3), (2, 3, 4)).toDF("a", "b", "c") + + implicit val valueEncoder = RowEncoder(df.schema) + + val err = intercept[AnalysisException] { + df.groupBy($"d", $"b").as[GroupByKey, Row] + } + assert(err.getMessage.contains("cannot resolve '`d`'")) + } } + +case class GroupByKey(a: Int, b: Int) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index fc6f087efaefd..9a367d48d2144 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1861,6 +1861,27 @@ class DatasetSuite extends QueryTest with SharedSparkSession { } } } + + test("groupBy.as") { + val df1 = Seq(DoubleData(1, "one"), DoubleData(2, "two"), DoubleData( 3, "three")).toDS() + .repartition($"id").sortWithinPartitions("id") + val df2 = Seq(DoubleData(5, "one"), DoubleData(1, "two"), DoubleData( 3, "three")).toDS() + .repartition($"id").sortWithinPartitions("id") + + val df3 = df1.groupBy("id").as[Int, DoubleData] + .cogroup(df2.groupBy("id").as[Int, DoubleData]) { case (key, data1, data2) => + if (key == 1) { + Iterator(DoubleData(key, (data1 ++ data2).foldLeft("")((cur, next) => cur + next.val1))) + } else Iterator.empty + } + checkDataset(df3, DoubleData(1, "onetwo")) + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = df3.queryExecution.executedPlan.collect { + case h: ShuffleExchangeExec => h + } + assert(exchanges.size == 2) + } } object AssertExecutionId { From f28eab2de72f7da883b970fc19edd4f569340bd7 Mon Sep 17 00:00:00 2001 From: Norman Maurer Date: Fri, 22 Nov 2019 15:20:54 -0800 Subject: [PATCH 0696/1387] [SPARK-29971][CORE] Fix buffer leaks in `TransportFrameDecoder/TransportCipher` ### What changes were proposed in this pull request? - Correctly release `ByteBuf` in `TransportCipher` in all cases - Move closing / releasing logic to `handlerRemoved(...)` so we are guaranteed that is always called. - Correctly release `frameBuf` it is not null when the handler is removed (and so also when the channel becomes inactive) ### Why are the changes needed? We need to carefully manage the ownership / lifecycle of `ByteBuf` instances so we don't leak any of these. We did not correctly do this in all cases: - when end up in invalid cipher state. - when partial data was received and the channel is closed before the full frame is decoded Fixes https://github.com/netty/netty/issues/9784. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the newly added UTs. Closes #26609 from normanmaurer/fix_leaks. Authored-by: Norman Maurer Signed-off-by: Marcelo Vanzin --- .../spark/network/crypto/TransportCipher.java | 50 ++++++---- .../util/ByteArrayReadableChannel.java | 24 ++--- .../network/util/TransportFrameDecoder.java | 28 +++++- .../network/crypto/TransportCipherSuite.java | 91 +++++++++++++++++++ 4 files changed, 157 insertions(+), 36 deletions(-) create mode 100644 common/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java index 8995bbc940f63..36ca73f6ac0f0 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -90,7 +90,8 @@ CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException return new CryptoOutputStream(cipher, conf, ch, key, new IvParameterSpec(outIv)); } - private CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { + @VisibleForTesting + CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { return new CryptoInputStream(cipher, conf, ch, key, new IvParameterSpec(inIv)); } @@ -166,34 +167,45 @@ private static class DecryptionHandler extends ChannelInboundHandlerAdapter { @Override public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { - if (!isCipherValid) { - throw new IOException("Cipher is in invalid state."); - } - byteChannel.feedData((ByteBuf) data); - - byte[] decryptedData = new byte[byteChannel.readableBytes()]; - int offset = 0; - while (offset < decryptedData.length) { - // SPARK-25535: workaround for CRYPTO-141. - try { - offset += cis.read(decryptedData, offset, decryptedData.length - offset); - } catch (InternalError ie) { - isCipherValid = false; - throw ie; + ByteBuf buffer = (ByteBuf) data; + + try { + if (!isCipherValid) { + throw new IOException("Cipher is in invalid state."); + } + byte[] decryptedData = new byte[buffer.readableBytes()]; + byteChannel.feedData(buffer); + + int offset = 0; + while (offset < decryptedData.length) { + // SPARK-25535: workaround for CRYPTO-141. + try { + offset += cis.read(decryptedData, offset, decryptedData.length - offset); + } catch (InternalError ie) { + isCipherValid = false; + throw ie; + } } - } - ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length)); + ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length)); + } finally { + buffer.release(); + } } @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + // We do the closing of the stream / channel in handlerRemoved(...) as + // this method will be called in all cases: + // + // - when the Channel becomes inactive + // - when the handler is removed from the ChannelPipeline try { if (isCipherValid) { cis.close(); } } finally { - super.channelInactive(ctx); + super.handlerRemoved(ctx); } } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java index 25d103d0e316f..fe461d0b39862 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java @@ -19,23 +19,27 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; import java.nio.channels.ReadableByteChannel; import io.netty.buffer.ByteBuf; public class ByteArrayReadableChannel implements ReadableByteChannel { private ByteBuf data; + private boolean closed; - public int readableBytes() { - return data.readableBytes(); - } - - public void feedData(ByteBuf buf) { + public void feedData(ByteBuf buf) throws ClosedChannelException { + if (closed) { + throw new ClosedChannelException(); + } data = buf; } @Override public int read(ByteBuffer dst) throws IOException { + if (closed) { + throw new ClosedChannelException(); + } int totalRead = 0; while (data.readableBytes() > 0 && dst.remaining() > 0) { int bytesToRead = Math.min(data.readableBytes(), dst.remaining()); @@ -43,20 +47,16 @@ public int read(ByteBuffer dst) throws IOException { totalRead += bytesToRead; } - if (data.readableBytes() == 0) { - data.release(); - } - return totalRead; } @Override - public void close() throws IOException { + public void close() { + closed = true; } @Override public boolean isOpen() { - return true; + return !closed; } - } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java index 1980361a15523..cef0e415aa40a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java @@ -184,8 +184,12 @@ private ByteBuf decodeNext() { return null; } - // Reset buf and size for next frame. + return consumeCurrentFrameBuf(); + } + + private ByteBuf consumeCurrentFrameBuf() { ByteBuf frame = frameBuf; + // Reset buf and size for next frame. frameBuf = null; consolidatedFrameBufSize = 0; consolidatedNumComponents = 0; @@ -215,13 +219,9 @@ private ByteBuf nextBufferForFrame(int bytesToRead) { @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { - for (ByteBuf b : buffers) { - b.release(); - } if (interceptor != null) { interceptor.channelInactive(); } - frameLenBuf.release(); super.channelInactive(ctx); } @@ -233,6 +233,24 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E super.exceptionCaught(ctx, cause); } + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + // Release all buffers that are still in our ownership. + // Doing this in handlerRemoved(...) guarantees that this will happen in all cases: + // - When the Channel becomes inactive + // - When the decoder is removed from the ChannelPipeline + for (ByteBuf b : buffers) { + b.release(); + } + buffers.clear(); + frameLenBuf.release(); + ByteBuf frame = consumeCurrentFrameBuf(); + if (frame != null) { + frame.release(); + } + super.handlerRemoved(ctx); + } + public void setInterceptor(Interceptor interceptor) { Preconditions.checkState(this.interceptor == null, "Already have an interceptor."); this.interceptor = interceptor; diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java new file mode 100644 index 0000000000000..6b2186f73cd0c --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java @@ -0,0 +1,91 @@ +/* + * 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.network.crypto; + +import javax.crypto.spec.SecretKeySpec; +import java.io.IOException; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; +import org.apache.commons.crypto.stream.CryptoInputStream; +import org.apache.commons.crypto.stream.CryptoOutputStream; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; +import org.hamcrest.CoreMatchers; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TransportCipherSuite { + + @Test + public void testBufferNotLeaksOnInternalError() throws IOException { + String algorithm = "TestAlgorithm"; + TransportConf conf = new TransportConf("Test", MapConfigProvider.EMPTY); + TransportCipher cipher = new TransportCipher(conf.cryptoConf(), conf.cipherTransformation(), + new SecretKeySpec(new byte[256], algorithm), new byte[0], new byte[0]) { + + @Override + CryptoOutputStream createOutputStream(WritableByteChannel ch) { + return null; + } + + @Override + CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { + CryptoInputStream mockInputStream = mock(CryptoInputStream.class); + when(mockInputStream.read(any(byte[].class), anyInt(), anyInt())) + .thenThrow(new InternalError()); + return mockInputStream; + } + }; + + EmbeddedChannel channel = new EmbeddedChannel(); + cipher.addToChannel(channel); + + ByteBuf buffer = Unpooled.wrappedBuffer(new byte[] { 1, 2 }); + ByteBuf buffer2 = Unpooled.wrappedBuffer(new byte[] { 1, 2 }); + + try { + channel.writeInbound(buffer); + fail("Should have raised InternalError"); + } catch (InternalError expected) { + // expected + assertEquals(0, buffer.refCnt()); + } + + try { + channel.writeInbound(buffer2); + fail("Should have raised an exception"); + } catch (Throwable expected) { + assertThat(expected, CoreMatchers.instanceOf(IOException.class)); + assertEquals(0, buffer2.refCnt()); + } + + // Simulate closing the connection + assertFalse(channel.finish()); + } +} From fc7a37b147d76edcd0de97ca1645dba736317a96 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 23 Nov 2019 17:24:56 +0900 Subject: [PATCH 0697/1387] [SPARK-30003][SQL] Do not throw stack overflow exception in non-root unknown hint resolution ### What changes were proposed in this pull request? This is rather a followup of https://github.com/apache/spark/pull/25464 (see https://github.com/apache/spark/pull/25464/files#r349543286) It will cause an infinite recursion via mapping children - we should return the hint rather than its parent plan in unknown hint resolution. ### Why are the changes needed? Prevent Stack over flow during hint resolution. ### Does this PR introduce any user-facing change? Yes, it avoids stack overflow exception It was caused by https://github.com/apache/spark/pull/25464 and this is only in the master. No behaviour changes to end users as it happened only in the master. ### How was this patch tested? Unittest was added. Closes #26642 from HyukjinKwon/SPARK-30003. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../apache/spark/sql/catalyst/analysis/ResolveHints.scala | 2 +- .../spark/sql/catalyst/analysis/ResolveHintsSuite.scala | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index d904ba3aca5d5..5b77d67bd1340 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -223,7 +223,7 @@ object ResolveHints { createRepartition(shuffle = false, hint) case "REPARTITION_BY_RANGE" => createRepartitionByRange(hint) - case _ => plan + case _ => hint } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index cddcddd51e38d..49ab34d2ea3a0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -245,4 +245,11 @@ class ResolveHintsSuite extends AnalysisTest { e => e.getLevel == Level.WARN && e.getRenderedMessage.contains("Unrecognized hint: unknown_hint"))) } + + test("SPARK-30003: Do not throw stack overflow exception in non-root unknown hint resolution") { + checkAnalysis( + Project(testRelation.output, UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE"))), + Project(testRelation.output, testRelation), + caseSensitive = false) + } } From c98e5eb3396a6db92f2420e743afa9ddff319ca2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2019 10:02:22 -0800 Subject: [PATCH 0698/1387] [SPARK-29981][BUILD] Add hive-1.2/2.3 profiles ### What changes were proposed in this pull request? This PR aims the followings. - Add two profiles, `hive-1.2` and `hive-2.3` (default) - Validate if we keep the existing combination at least. (Hadoop-2.7 + Hive 1.2 / Hadoop-3.2 + Hive 2.3). For now, we assumes that `hive-1.2` is explicitly used with `hadoop-2.7` and `hive-2.3` with `hadoop-3.2`. The followings are beyond the scope of this PR. - SPARK-29988 Adjust Jenkins jobs for `hive-1.2/2.3` combination - SPARK-29989 Update release-script for `hive-1.2/2.3` combination - SPARK-29991 Support `hive-1.2/2.3` in PR Builder ### Why are the changes needed? This will help to switch our dependencies to update the exposed dependencies. ### Does this PR introduce any user-facing change? This is a dev-only change that the build profile combinations are changed. - `-Phadoop-2.7` => `-Phadoop-2.7 -Phive-1.2` - `-Phadoop-3.2` => `-Phadoop-3.2 -Phive-2.3` ### How was this patch tested? Pass the Jenkins with the dependency check and tests to make it sure we don't change anything for now. - [Jenkins (-Phadoop-2.7 -Phive-1.2)](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/114192/consoleFull) - [Jenkins (-Phadoop-3.2 -Phive-2.3)](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/114192/consoleFull) Also, from now, GitHub Action validates the following combinations. ![gha](https://user-images.githubusercontent.com/9700541/69355365-822d5e00-0c36-11ea-93f7-e00e5459e1d0.png) Closes #26619 from dongjoon-hyun/SPARK-29981. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 9 ++++++-- appveyor.yml | 2 +- dev/run-tests.py | 4 ++-- dev/test-dependencies.sh | 11 ++++++--- pom.xml | 43 ++++++++++++++++++++++-------------- sql/hive/pom.xml | 2 +- 6 files changed, 46 insertions(+), 25 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 5298492d219f8..30f3272c8b933 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -16,10 +16,15 @@ jobs: matrix: java: [ '1.8', '11' ] hadoop: [ 'hadoop-2.7', 'hadoop-3.2' ] + hive: [ 'hive-1.2', 'hive-2.3' ] exclude: - java: '11' hadoop: 'hadoop-2.7' - name: Build Spark with JDK ${{ matrix.java }} and ${{ matrix.hadoop }} + - java: '11' + hive: 'hive-1.2' + - hadoop: 'hadoop-3.2' + hive: 'hive-1.2' + name: Build Spark - JDK${{ matrix.java }}/${{ matrix.hadoop }}/${{ matrix.hive }} steps: - uses: actions/checkout@master @@ -56,7 +61,7 @@ jobs: run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" - ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install + ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -P${{ matrix.hive }} -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install rm -rf ~/.m2/repository/org/apache/spark diff --git a/appveyor.yml b/appveyor.yml index 00c688ba18eb6..325fd67abc674 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -53,7 +53,7 @@ install: build_script: # '-Djna.nosys=true' is required to avoid kernel32.dll load failure. # See SPARK-28759. - - cmd: mvn -DskipTests -Psparkr -Phive -Djna.nosys=true package + - cmd: mvn -DskipTests -Psparkr -Phive -Phive-1.2 -Djna.nosys=true package environment: NOT_CRAN: true diff --git a/dev/run-tests.py b/dev/run-tests.py index fc8b7251a85f8..2d52ead06a041 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -283,8 +283,8 @@ def get_hadoop_profiles(hadoop_version): """ sbt_maven_hadoop_profiles = { - "hadoop2.7": ["-Phadoop-2.7"], - "hadoop3.2": ["-Phadoop-3.2"], + "hadoop2.7": ["-Phadoop-2.7", "-Phive-1.2"], + "hadoop3.2": ["-Phadoop-3.2", "-Phive-2.3"], } if hadoop_version in sbt_maven_hadoop_profiles: diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index cc0292e9c2ea5..7d5725aaf137e 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -67,15 +67,20 @@ $MVN -q versions:set -DnewVersion=$TEMP_VERSION -DgenerateBackupPoms=false > /de # Generate manifests for each Hadoop profile: for HADOOP_PROFILE in "${HADOOP_PROFILES[@]}"; do + if [[ $HADOOP_PROFILE == **hadoop-3** ]]; then + HIVE_PROFILE=hive-2.3 + else + HIVE_PROFILE=hive-1.2 + fi echo "Performing Maven install for $HADOOP_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE jar:jar jar:test-jar install:install clean -q + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q echo "Performing Maven validate for $HADOOP_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE validate -q + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE validate -q echo "Generating dependency manifest for $HADOOP_PROFILE" mkdir -p dev/pr-deps - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE dependency:build-classpath -pl assembly -am \ + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE dependency:build-classpath -pl assembly -am \ | grep "Dependencies classpath:" -A 1 \ | tail -n 1 | tr ":" "\n" | rev | cut -d "/" -f 1 | rev | sort \ | grep -v spark > dev/pr-deps/spark-deps-$HADOOP_PROFILE diff --git a/pom.xml b/pom.xml index 44593b78c9a04..a9b544bba158d 100644 --- a/pom.xml +++ b/pom.xml @@ -128,19 +128,19 @@ 3.4.14 2.7.1 0.4.2 - org.spark-project.hive - + org.apache.hive + core - 1.2.1.spark2 + 2.3.6 2.3.6 - 1.2.1 + 2.3.5 2.3.1 10.12.1.1 1.10.1 1.5.7 - nohive + com.twitter 1.6.0 9.4.18.v20190429 @@ -181,7 +181,7 @@ 3.8.1 2.6.2 - 3.2.10 + 4.1.17 3.0.15 2.29 2.10.5 @@ -228,7 +228,7 @@ --> compile compile - ${hive.deps.scope} + provided compile compile test @@ -2921,16 +2921,27 @@ 3.2.0 2.13.0 - org.apache.hive - core - ${hive23.version} - 2.3.5 - - provided - - 4.1.17 + + + + hive-1.2 + + org.spark-project.hive + + + 1.2.1.spark2 + + 1.2.1 + ${hive.deps.scope} + nohive + 3.2.10 + + + + + hive-2.3 + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 882cabd347073..69961acd510d8 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -209,7 +209,7 @@ - hadoop-3.2 + hive-2.3 ${hive.group} From 6625b69027661d34352aa22e75dee6f31d069b41 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2019 12:50:50 -0800 Subject: [PATCH 0699/1387] [SPARK-29981][BUILD][FOLLOWUP] Change hive.version.short ### What changes were proposed in this pull request? This is a follow-up according to liancheng 's advice. - https://github.com/apache/spark/pull/26619#discussion_r349326090 ### Why are the changes needed? Previously, we chose the full version to be carefully. As of today, it seems that `Apache Hive 2.3` branch seems to become stable. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the compile combination on GitHub Action. 1. hadoop-2.7/hive-1.2/JDK8 2. hadoop-2.7/hive-2.3/JDK8 3. hadoop-3.2/hive-2.3/JDK8 4. hadoop-3.2/hive-2.3/JDK11 Also, pass the Jenkins with `hadoop-2.7` and `hadoop-3.2` for (1) and (4). (2) and (3) is not ready in Jenkins. Closes #26645 from dongjoon-hyun/SPARK-RENAME-HIVE-DIRECTORY. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 4 ++-- .../spark/sql/execution/datasources/orc/OrcColumnVector.java | 0 .../spark/sql/execution/datasources/orc/OrcFilters.scala | 0 .../spark/sql/execution/datasources/orc/OrcShimUtils.scala | 0 .../spark/sql/execution/datasources/orc/OrcFilterSuite.scala | 0 .../spark/sql/execution/datasources/orc/OrcColumnVector.java | 0 .../spark/sql/execution/datasources/orc/OrcFilters.scala | 0 .../spark/sql/execution/datasources/orc/OrcShimUtils.scala | 0 .../spark/sql/execution/datasources/orc/OrcFilterSuite.scala | 0 sql/hive-thriftserver/{v1.2.1 => v1.2}/if/TCLIService.thrift | 0 .../org/apache/hive/service/cli/thrift/TArrayTypeEntry.java | 0 .../org/apache/hive/service/cli/thrift/TBinaryColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TBoolColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TBoolValue.java | 0 .../java/org/apache/hive/service/cli/thrift/TByteColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TByteValue.java | 0 .../java/org/apache/hive/service/cli/thrift/TCLIService.java | 0 .../apache/hive/service/cli/thrift/TCLIServiceConstants.java | 0 .../hive/service/cli/thrift/TCancelDelegationTokenReq.java | 0 .../hive/service/cli/thrift/TCancelDelegationTokenResp.java | 0 .../apache/hive/service/cli/thrift/TCancelOperationReq.java | 0 .../apache/hive/service/cli/thrift/TCancelOperationResp.java | 0 .../apache/hive/service/cli/thrift/TCloseOperationReq.java | 0 .../apache/hive/service/cli/thrift/TCloseOperationResp.java | 0 .../org/apache/hive/service/cli/thrift/TCloseSessionReq.java | 0 .../org/apache/hive/service/cli/thrift/TCloseSessionResp.java | 0 .../gen/java/org/apache/hive/service/cli/thrift/TColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TColumnDesc.java | 0 .../java/org/apache/hive/service/cli/thrift/TColumnValue.java | 0 .../org/apache/hive/service/cli/thrift/TDoubleColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TDoubleValue.java | 0 .../apache/hive/service/cli/thrift/TExecuteStatementReq.java | 0 .../apache/hive/service/cli/thrift/TExecuteStatementResp.java | 0 .../org/apache/hive/service/cli/thrift/TFetchOrientation.java | 0 .../org/apache/hive/service/cli/thrift/TFetchResultsReq.java | 0 .../org/apache/hive/service/cli/thrift/TFetchResultsResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetCatalogsReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetCatalogsResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetColumnsReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetColumnsResp.java | 0 .../hive/service/cli/thrift/TGetDelegationTokenReq.java | 0 .../hive/service/cli/thrift/TGetDelegationTokenResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetFunctionsReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetFunctionsResp.java | 0 .../java/org/apache/hive/service/cli/thrift/TGetInfoReq.java | 0 .../java/org/apache/hive/service/cli/thrift/TGetInfoResp.java | 0 .../java/org/apache/hive/service/cli/thrift/TGetInfoType.java | 0 .../org/apache/hive/service/cli/thrift/TGetInfoValue.java | 0 .../hive/service/cli/thrift/TGetOperationStatusReq.java | 0 .../hive/service/cli/thrift/TGetOperationStatusResp.java | 0 .../hive/service/cli/thrift/TGetResultSetMetadataReq.java | 0 .../hive/service/cli/thrift/TGetResultSetMetadataResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetSchemasReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetSchemasResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetTableTypesReq.java | 0 .../apache/hive/service/cli/thrift/TGetTableTypesResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetTablesReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetTablesResp.java | 0 .../org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java | 0 .../org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java | 0 .../org/apache/hive/service/cli/thrift/THandleIdentifier.java | 0 .../java/org/apache/hive/service/cli/thrift/TI16Column.java | 0 .../java/org/apache/hive/service/cli/thrift/TI16Value.java | 0 .../java/org/apache/hive/service/cli/thrift/TI32Column.java | 0 .../java/org/apache/hive/service/cli/thrift/TI32Value.java | 0 .../java/org/apache/hive/service/cli/thrift/TI64Column.java | 0 .../java/org/apache/hive/service/cli/thrift/TI64Value.java | 0 .../org/apache/hive/service/cli/thrift/TMapTypeEntry.java | 0 .../org/apache/hive/service/cli/thrift/TOpenSessionReq.java | 0 .../org/apache/hive/service/cli/thrift/TOpenSessionResp.java | 0 .../org/apache/hive/service/cli/thrift/TOperationHandle.java | 0 .../org/apache/hive/service/cli/thrift/TOperationState.java | 0 .../org/apache/hive/service/cli/thrift/TOperationType.java | 0 .../apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java | 0 .../org/apache/hive/service/cli/thrift/TProtocolVersion.java | 0 .../hive/service/cli/thrift/TRenewDelegationTokenReq.java | 0 .../hive/service/cli/thrift/TRenewDelegationTokenResp.java | 0 .../src/gen/java/org/apache/hive/service/cli/thrift/TRow.java | 0 .../gen/java/org/apache/hive/service/cli/thrift/TRowSet.java | 0 .../org/apache/hive/service/cli/thrift/TSessionHandle.java | 0 .../gen/java/org/apache/hive/service/cli/thrift/TStatus.java | 0 .../java/org/apache/hive/service/cli/thrift/TStatusCode.java | 0 .../org/apache/hive/service/cli/thrift/TStringColumn.java | 0 .../java/org/apache/hive/service/cli/thrift/TStringValue.java | 0 .../org/apache/hive/service/cli/thrift/TStructTypeEntry.java | 0 .../java/org/apache/hive/service/cli/thrift/TTableSchema.java | 0 .../java/org/apache/hive/service/cli/thrift/TTypeDesc.java | 0 .../java/org/apache/hive/service/cli/thrift/TTypeEntry.java | 0 .../gen/java/org/apache/hive/service/cli/thrift/TTypeId.java | 0 .../apache/hive/service/cli/thrift/TTypeQualifierValue.java | 0 .../org/apache/hive/service/cli/thrift/TTypeQualifiers.java | 0 .../org/apache/hive/service/cli/thrift/TUnionTypeEntry.java | 0 .../apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java | 0 .../main/java/org/apache/hive/service/AbstractService.java | 0 .../main/java/org/apache/hive/service/CompositeService.java | 0 .../src/main/java/org/apache/hive/service/CookieSigner.java | 0 .../main/java/org/apache/hive/service/ServiceOperations.java | 0 .../src/main/java/org/apache/hive/service/ServiceUtils.java | 0 .../java/org/apache/hive/service/auth/HiveAuthFactory.java | 0 .../main/java/org/apache/hive/service/auth/HttpAuthUtils.java | 0 .../java/org/apache/hive/service/auth/KerberosSaslHelper.java | 0 .../java/org/apache/hive/service/auth/PlainSaslHelper.java | 0 .../org/apache/hive/service/auth/TSetIpAddressProcessor.java | 0 .../src/main/java/org/apache/hive/service/cli/CLIService.java | 0 .../src/main/java/org/apache/hive/service/cli/Column.java | 0 .../main/java/org/apache/hive/service/cli/ColumnBasedSet.java | 0 .../java/org/apache/hive/service/cli/ColumnDescriptor.java | 0 .../main/java/org/apache/hive/service/cli/ColumnValue.java | 0 .../org/apache/hive/service/cli/EmbeddedCLIServiceClient.java | 0 .../java/org/apache/hive/service/cli/FetchOrientation.java | 0 .../main/java/org/apache/hive/service/cli/GetInfoType.java | 0 .../main/java/org/apache/hive/service/cli/GetInfoValue.java | 0 .../src/main/java/org/apache/hive/service/cli/Handle.java | 0 .../java/org/apache/hive/service/cli/HandleIdentifier.java | 0 .../java/org/apache/hive/service/cli/HiveSQLException.java | 0 .../main/java/org/apache/hive/service/cli/ICLIService.java | 0 .../java/org/apache/hive/service/cli/OperationHandle.java | 0 .../main/java/org/apache/hive/service/cli/OperationState.java | 0 .../main/java/org/apache/hive/service/cli/OperationType.java | 0 .../java/org/apache/hive/service/cli/PatternOrIdentifier.java | 0 .../main/java/org/apache/hive/service/cli/RowBasedSet.java | 0 .../src/main/java/org/apache/hive/service/cli/RowSet.java | 0 .../main/java/org/apache/hive/service/cli/RowSetFactory.java | 0 .../main/java/org/apache/hive/service/cli/SessionHandle.java | 0 .../main/java/org/apache/hive/service/cli/TableSchema.java | 0 .../src/main/java/org/apache/hive/service/cli/Type.java | 0 .../main/java/org/apache/hive/service/cli/TypeDescriptor.java | 0 .../main/java/org/apache/hive/service/cli/TypeQualifiers.java | 0 .../hive/service/cli/operation/ClassicTableTypeMapping.java | 0 .../hive/service/cli/operation/ExecuteStatementOperation.java | 0 .../hive/service/cli/operation/GetCatalogsOperation.java | 0 .../hive/service/cli/operation/GetColumnsOperation.java | 0 .../hive/service/cli/operation/GetFunctionsOperation.java | 0 .../hive/service/cli/operation/GetSchemasOperation.java | 0 .../hive/service/cli/operation/GetTableTypesOperation.java | 0 .../apache/hive/service/cli/operation/GetTablesOperation.java | 0 .../hive/service/cli/operation/GetTypeInfoOperation.java | 0 .../hive/service/cli/operation/HiveCommandOperation.java | 0 .../hive/service/cli/operation/HiveTableTypeMapping.java | 0 .../apache/hive/service/cli/operation/MetadataOperation.java | 0 .../java/org/apache/hive/service/cli/operation/Operation.java | 0 .../apache/hive/service/cli/operation/OperationManager.java | 0 .../org/apache/hive/service/cli/operation/SQLOperation.java | 0 .../apache/hive/service/cli/operation/TableTypeMapping.java | 0 .../java/org/apache/hive/service/cli/session/HiveSession.java | 0 .../org/apache/hive/service/cli/session/HiveSessionBase.java | 0 .../org/apache/hive/service/cli/session/HiveSessionImpl.java | 0 .../hive/service/cli/session/HiveSessionImplwithUGI.java | 0 .../org/apache/hive/service/cli/session/SessionManager.java | 0 .../hive/service/cli/thrift/ThriftBinaryCLIService.java | 0 .../org/apache/hive/service/cli/thrift/ThriftCLIService.java | 0 .../hive/service/cli/thrift/ThriftCLIServiceClient.java | 0 .../apache/hive/service/cli/thrift/ThriftHttpCLIService.java | 0 .../org/apache/hive/service/cli/thrift/ThriftHttpServlet.java | 0 .../main/java/org/apache/hive/service/server/HiveServer2.java | 0 .../apache/hive/service/server/ThreadWithGarbageCleanup.java | 0 .../spark/sql/hive/thriftserver/ThriftserverShimUtils.scala | 0 sql/hive-thriftserver/{v2.3.5 => v2.3}/if/TCLIService.thrift | 0 .../org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java | 0 .../org/apache/hive/service/rpc/thrift/TBinaryColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TBoolColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TBoolValue.java | 0 .../java/org/apache/hive/service/rpc/thrift/TByteColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TByteValue.java | 0 .../java/org/apache/hive/service/rpc/thrift/TCLIService.java | 0 .../apache/hive/service/rpc/thrift/TCLIServiceConstants.java | 0 .../hive/service/rpc/thrift/TCancelDelegationTokenReq.java | 0 .../hive/service/rpc/thrift/TCancelDelegationTokenResp.java | 0 .../apache/hive/service/rpc/thrift/TCancelOperationReq.java | 0 .../apache/hive/service/rpc/thrift/TCancelOperationResp.java | 0 .../apache/hive/service/rpc/thrift/TCloseOperationReq.java | 0 .../apache/hive/service/rpc/thrift/TCloseOperationResp.java | 0 .../org/apache/hive/service/rpc/thrift/TCloseSessionReq.java | 0 .../org/apache/hive/service/rpc/thrift/TCloseSessionResp.java | 0 .../gen/java/org/apache/hive/service/rpc/thrift/TColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TColumnDesc.java | 0 .../java/org/apache/hive/service/rpc/thrift/TColumnValue.java | 0 .../org/apache/hive/service/rpc/thrift/TDoubleColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TDoubleValue.java | 0 .../apache/hive/service/rpc/thrift/TExecuteStatementReq.java | 0 .../apache/hive/service/rpc/thrift/TExecuteStatementResp.java | 0 .../org/apache/hive/service/rpc/thrift/TFetchOrientation.java | 0 .../org/apache/hive/service/rpc/thrift/TFetchResultsReq.java | 0 .../org/apache/hive/service/rpc/thrift/TFetchResultsResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetColumnsReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetColumnsResp.java | 0 .../apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java | 0 .../hive/service/rpc/thrift/TGetCrossReferenceResp.java | 0 .../hive/service/rpc/thrift/TGetDelegationTokenReq.java | 0 .../hive/service/rpc/thrift/TGetDelegationTokenResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java | 0 .../java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java | 0 .../java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java | 0 .../java/org/apache/hive/service/rpc/thrift/TGetInfoType.java | 0 .../org/apache/hive/service/rpc/thrift/TGetInfoValue.java | 0 .../hive/service/rpc/thrift/TGetOperationStatusReq.java | 0 .../hive/service/rpc/thrift/TGetOperationStatusResp.java | 0 .../apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java | 0 .../apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java | 0 .../hive/service/rpc/thrift/TGetResultSetMetadataReq.java | 0 .../hive/service/rpc/thrift/TGetResultSetMetadataResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetSchemasReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetSchemasResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java | 0 .../apache/hive/service/rpc/thrift/TGetTableTypesResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTablesReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTablesResp.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java | 0 .../org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java | 0 .../org/apache/hive/service/rpc/thrift/THandleIdentifier.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI16Column.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI16Value.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI32Column.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI32Value.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI64Column.java | 0 .../java/org/apache/hive/service/rpc/thrift/TI64Value.java | 0 .../apache/hive/service/rpc/thrift/TJobExecutionStatus.java | 0 .../org/apache/hive/service/rpc/thrift/TMapTypeEntry.java | 0 .../org/apache/hive/service/rpc/thrift/TOpenSessionReq.java | 0 .../org/apache/hive/service/rpc/thrift/TOpenSessionResp.java | 0 .../org/apache/hive/service/rpc/thrift/TOperationHandle.java | 0 .../org/apache/hive/service/rpc/thrift/TOperationState.java | 0 .../org/apache/hive/service/rpc/thrift/TOperationType.java | 0 .../apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java | 0 .../apache/hive/service/rpc/thrift/TProgressUpdateResp.java | 0 .../org/apache/hive/service/rpc/thrift/TProtocolVersion.java | 0 .../hive/service/rpc/thrift/TRenewDelegationTokenReq.java | 0 .../hive/service/rpc/thrift/TRenewDelegationTokenResp.java | 0 .../src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java | 0 .../gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java | 0 .../org/apache/hive/service/rpc/thrift/TSessionHandle.java | 0 .../gen/java/org/apache/hive/service/rpc/thrift/TStatus.java | 0 .../java/org/apache/hive/service/rpc/thrift/TStatusCode.java | 0 .../org/apache/hive/service/rpc/thrift/TStringColumn.java | 0 .../java/org/apache/hive/service/rpc/thrift/TStringValue.java | 0 .../org/apache/hive/service/rpc/thrift/TStructTypeEntry.java | 0 .../java/org/apache/hive/service/rpc/thrift/TTableSchema.java | 0 .../java/org/apache/hive/service/rpc/thrift/TTypeDesc.java | 0 .../java/org/apache/hive/service/rpc/thrift/TTypeEntry.java | 0 .../gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java | 0 .../apache/hive/service/rpc/thrift/TTypeQualifierValue.java | 0 .../org/apache/hive/service/rpc/thrift/TTypeQualifiers.java | 0 .../org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java | 0 .../apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java | 0 .../main/java/org/apache/hive/service/AbstractService.java | 0 .../main/java/org/apache/hive/service/CompositeService.java | 0 .../src/main/java/org/apache/hive/service/CookieSigner.java | 0 .../main/java/org/apache/hive/service/ServiceOperations.java | 0 .../src/main/java/org/apache/hive/service/ServiceUtils.java | 0 .../java/org/apache/hive/service/auth/HiveAuthFactory.java | 0 .../main/java/org/apache/hive/service/auth/HttpAuthUtils.java | 0 .../java/org/apache/hive/service/auth/KerberosSaslHelper.java | 0 .../java/org/apache/hive/service/auth/PlainSaslHelper.java | 0 .../org/apache/hive/service/auth/TSetIpAddressProcessor.java | 0 .../src/main/java/org/apache/hive/service/cli/CLIService.java | 0 .../main/java/org/apache/hive/service/cli/ColumnBasedSet.java | 0 .../java/org/apache/hive/service/cli/ColumnDescriptor.java | 0 .../main/java/org/apache/hive/service/cli/ColumnValue.java | 0 .../java/org/apache/hive/service/cli/FetchOrientation.java | 0 .../main/java/org/apache/hive/service/cli/GetInfoType.java | 0 .../main/java/org/apache/hive/service/cli/GetInfoValue.java | 0 .../src/main/java/org/apache/hive/service/cli/Handle.java | 0 .../java/org/apache/hive/service/cli/HandleIdentifier.java | 0 .../java/org/apache/hive/service/cli/HiveSQLException.java | 0 .../main/java/org/apache/hive/service/cli/ICLIService.java | 0 .../java/org/apache/hive/service/cli/OperationHandle.java | 0 .../main/java/org/apache/hive/service/cli/OperationState.java | 0 .../main/java/org/apache/hive/service/cli/OperationType.java | 0 .../main/java/org/apache/hive/service/cli/RowBasedSet.java | 0 .../src/main/java/org/apache/hive/service/cli/RowSet.java | 0 .../main/java/org/apache/hive/service/cli/RowSetFactory.java | 0 .../main/java/org/apache/hive/service/cli/SessionHandle.java | 0 .../main/java/org/apache/hive/service/cli/TableSchema.java | 0 .../main/java/org/apache/hive/service/cli/TypeDescriptor.java | 0 .../main/java/org/apache/hive/service/cli/TypeQualifiers.java | 0 .../hive/service/cli/operation/ClassicTableTypeMapping.java | 0 .../hive/service/cli/operation/ExecuteStatementOperation.java | 0 .../hive/service/cli/operation/GetCatalogsOperation.java | 0 .../hive/service/cli/operation/GetColumnsOperation.java | 0 .../service/cli/operation/GetCrossReferenceOperation.java | 0 .../hive/service/cli/operation/GetFunctionsOperation.java | 0 .../hive/service/cli/operation/GetPrimaryKeysOperation.java | 0 .../hive/service/cli/operation/GetSchemasOperation.java | 0 .../hive/service/cli/operation/GetTableTypesOperation.java | 0 .../apache/hive/service/cli/operation/GetTablesOperation.java | 0 .../hive/service/cli/operation/GetTypeInfoOperation.java | 0 .../hive/service/cli/operation/HiveCommandOperation.java | 0 .../hive/service/cli/operation/HiveTableTypeMapping.java | 0 .../apache/hive/service/cli/operation/MetadataOperation.java | 0 .../java/org/apache/hive/service/cli/operation/Operation.java | 0 .../apache/hive/service/cli/operation/OperationManager.java | 0 .../org/apache/hive/service/cli/operation/SQLOperation.java | 0 .../apache/hive/service/cli/operation/TableTypeMapping.java | 0 .../java/org/apache/hive/service/cli/session/HiveSession.java | 0 .../org/apache/hive/service/cli/session/HiveSessionBase.java | 0 .../hive/service/cli/session/HiveSessionHookContext.java | 0 .../hive/service/cli/session/HiveSessionHookContextImpl.java | 0 .../org/apache/hive/service/cli/session/HiveSessionImpl.java | 0 .../hive/service/cli/session/HiveSessionImplwithUGI.java | 0 .../org/apache/hive/service/cli/session/SessionManager.java | 0 .../hive/service/cli/thrift/ThriftBinaryCLIService.java | 0 .../org/apache/hive/service/cli/thrift/ThriftCLIService.java | 0 .../hive/service/cli/thrift/ThriftCLIServiceClient.java | 0 .../apache/hive/service/cli/thrift/ThriftHttpCLIService.java | 0 .../org/apache/hive/service/cli/thrift/ThriftHttpServlet.java | 0 .../main/java/org/apache/hive/service/server/HiveServer2.java | 0 .../apache/hive/service/server/ThreadWithGarbageCleanup.java | 0 .../spark/sql/hive/thriftserver/ThriftserverShimUtils.scala | 0 311 files changed, 2 insertions(+), 2 deletions(-) rename sql/core/{v1.2.1 => v1.2}/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java (100%) rename sql/core/{v1.2.1 => v1.2}/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala (100%) rename sql/core/{v1.2.1 => v1.2}/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala (100%) rename sql/core/{v1.2.1 => v1.2}/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala (100%) rename sql/core/{v2.3.5 => v2.3}/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java (100%) rename sql/core/{v2.3.5 => v2.3}/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala (100%) rename sql/core/{v2.3.5 => v2.3}/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala (100%) rename sql/core/{v2.3.5 => v2.3}/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/if/TCLIService.thrift (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/AbstractService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/CompositeService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/CookieSigner.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/ServiceOperations.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/ServiceUtils.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/CLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/Column.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/ColumnValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/FetchOrientation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/GetInfoType.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/GetInfoValue.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/Handle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/HiveSQLException.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/ICLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/OperationHandle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/OperationState.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/OperationType.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/RowBasedSet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/RowSet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/RowSetFactory.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/SessionHandle.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/TableSchema.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/Type.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/Operation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/HiveSession.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/session/SessionManager.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/server/HiveServer2.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java (100%) rename sql/hive-thriftserver/{v1.2.1 => v1.2}/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/if/TCLIService.thrift (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/AbstractService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/CompositeService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/CookieSigner.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/ServiceOperations.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/ServiceUtils.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/CLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/ColumnValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/FetchOrientation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/GetInfoType.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/GetInfoValue.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/Handle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/HiveSQLException.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/ICLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/OperationHandle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/OperationState.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/OperationType.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/RowBasedSet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/RowSet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/RowSetFactory.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/SessionHandle.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/TableSchema.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/Operation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSession.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/session/SessionManager.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/server/HiveServer2.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java (100%) rename sql/hive-thriftserver/{v2.3.5 => v2.3}/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala (100%) diff --git a/pom.xml b/pom.xml index a9b544bba158d..825a227e8669b 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,7 @@ 2.3.6 2.3.6 - 2.3.5 + 2.3 2.3.1 10.12.1.1 @@ -2932,7 +2932,7 @@ 1.2.1.spark2 - 1.2.1 + 1.2 ${hive.deps.scope} nohive 3.2.10 diff --git a/sql/core/v1.2.1/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java similarity index 100% rename from sql/core/v1.2.1/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java rename to sql/core/v1.2/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala similarity index 100% rename from sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala rename to sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala similarity index 100% rename from sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala rename to sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala diff --git a/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala similarity index 100% rename from sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala rename to sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala diff --git a/sql/core/v2.3.5/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java similarity index 100% rename from sql/core/v2.3.5/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java rename to sql/core/v2.3/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala similarity index 100% rename from sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala rename to sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala similarity index 100% rename from sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala rename to sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala diff --git a/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala similarity index 100% rename from sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala rename to sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala diff --git a/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift b/sql/hive-thriftserver/v1.2/if/TCLIService.thrift similarity index 100% rename from sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift rename to sql/hive-thriftserver/v1.2/if/TCLIService.thrift diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java rename to sql/hive-thriftserver/v1.2/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/AbstractService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/AbstractService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CompositeService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CompositeService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CookieSigner.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/CookieSigner.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceOperations.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceOperations.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceUtils.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/ServiceUtils.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/CLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/CLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Column.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Column.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Column.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ColumnValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/FetchOrientation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/FetchOrientation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoType.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoType.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoValue.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/GetInfoValue.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Handle.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Handle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HiveSQLException.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/HiveSQLException.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ICLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/ICLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationHandle.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationHandle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationState.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationState.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationType.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/OperationType.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowBasedSet.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowBasedSet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSet.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/RowSetFactory.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/SessionHandle.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/SessionHandle.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TableSchema.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TableSchema.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Type.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Type.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/Type.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/Operation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSession.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSession.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/SessionManager.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/session/SessionManager.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/HiveServer2.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/HiveServer2.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java rename to sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala similarity index 100% rename from sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala rename to sql/hive-thriftserver/v1.2/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala diff --git a/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift b/sql/hive-thriftserver/v2.3/if/TCLIService.thrift similarity index 100% rename from sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift rename to sql/hive-thriftserver/v2.3/if/TCLIService.thrift diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java rename to sql/hive-thriftserver/v2.3/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/AbstractService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/AbstractService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/AbstractService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CompositeService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CompositeService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CompositeService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CookieSigner.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CookieSigner.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/CookieSigner.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceOperations.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceOperations.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceOperations.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceUtils.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceUtils.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/ServiceUtils.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/CLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/CLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/CLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ColumnValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/FetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/FetchOrientation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/FetchOrientation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoType.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoType.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoType.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoValue.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/GetInfoValue.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/Handle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/Handle.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/Handle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HiveSQLException.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HiveSQLException.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/HiveSQLException.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ICLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ICLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/ICLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationHandle.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationHandle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationState.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationState.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationState.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationType.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationType.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/OperationType.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowBasedSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowBasedSet.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowBasedSet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSet.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSetFactory.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSetFactory.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/RowSetFactory.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/SessionHandle.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/SessionHandle.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/SessionHandle.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TableSchema.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TableSchema.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TableSchema.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/Operation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/Operation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSession.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSession.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSession.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/SessionManager.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/session/SessionManager.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/HiveServer2.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/HiveServer2.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/HiveServer2.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java rename to sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala similarity index 100% rename from sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala rename to sql/hive-thriftserver/v2.3/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala From 6cd6d5f57ed53aed234b169ad5be3e133dab608f Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 23 Nov 2019 16:16:24 -0800 Subject: [PATCH 0700/1387] [SPARK-29970][WEBUI] Preserver open/close state of Timelineview ### What changes were proposed in this pull request? Fix a bug related to Timelineview that does not preserve open/close state properly. ### Why are the changes needed? To preserve open/close state is originally intended but it doesn't work. ### Does this PR introduce any user-facing change? Yes. open/close state for Timeineview is to be preserved so if you open Timelineview in Stage page and go to another page, and then go back to Stage page, Timelineview should keep open. ### How was this patch tested? Manual test. Closes #26607 from sarutak/fix-timeline-view-state. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/ui/static/timeline-view.js | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js index 705a08f0293d3..b2cd616791734 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.js @@ -83,8 +83,9 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { }); } -$(function (){ - if (window.localStorage.getItem("expand-application-timeline") == "true") { +$(function () { + if ($("span.expand-application-timeline").length && + window.localStorage.getItem("expand-application-timeline") == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-application-timeline", "false"); $("span.expand-application-timeline").trigger('click'); @@ -159,8 +160,9 @@ function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { }); } -$(function (){ - if (window.localStorage.getItem("expand-job-timeline") == "true") { +$(function () { + if ($("span.expand-job-timeline").length && + window.localStorage.getItem("expand-job-timeline") == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-job-timeline", "false"); $("span.expand-job-timeline").trigger('click'); @@ -226,8 +228,9 @@ function drawTaskAssignmentTimeline(groupArray, eventObjArray, minLaunchTime, ma }); } -$(function (){ - if (window.localStorage.getItem("expand-task-assignment-timeline") == "true") { +$(function () { + if ($("span.expand-task-assignment-timeline").length && + window.localStorage.getItem("expand-task-assignment-timeline") == "true") { // Set it to false so that the click function can revert it window.localStorage.setItem("expand-task-assignment-timeline", "false"); $("span.expand-task-assignment-timeline").trigger('click'); From 6898be9f02828fabe3c417244f63e0fc79ba58d3 Mon Sep 17 00:00:00 2001 From: Prakhar Jain Date: Sat, 23 Nov 2019 18:09:02 -0800 Subject: [PATCH 0701/1387] [SPARK-29681][WEBUI] Support column sorting in Environment tab ### What changes were proposed in this pull request? Add extra classnames to table headers in EnvironmentPage tables in Spark UI. ### Why are the changes needed? SparkUI uses sorttable.js to provide the sort functionality in different tables. This library tries to guess the datatype of each column during initialization phase - numeric/alphanumeric etc and uses it to sort the columns whenever user clicks on a column. That way it guesses incorrect data type for environment tab. sorttable.js has way to hint the datatype of table columns explicitly. This is done by passing custom HTML class attribute. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested sorting in tables in Environment tab in Spark UI. ![Annotation 2019-11-22 154058](https://user-images.githubusercontent.com/2551496/69417432-a8d6bc00-0d3e-11ea-865b-f8017976c6f4.png) ![Annotation 2019-11-22 153600](https://user-images.githubusercontent.com/2551496/69417433-a8d6bc00-0d3e-11ea-9a75-8e1f4d66107e.png) ![Annotation 2019-11-22 153841](https://user-images.githubusercontent.com/2551496/69417435-a96f5280-0d3e-11ea-85f6-9f61b015e161.png) Closes #26638 from prakharjain09/SPARK-29681-SPARK-UI-SORT. Authored-by: Prakhar Jain Signed-off-by: Dongjoon Hyun --- .../apache/spark/ui/env/EnvironmentPage.scala | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index 76537afd81ce0..c6eb461ad601c 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -39,15 +39,20 @@ private[ui] class EnvironmentPage( "Scala Version" -> appEnv.runtime.scalaVersion) val runtimeInformationTable = UIUtils.listingTable( - propertyHeader, jvmRow, jvmInformation.toSeq.sorted, fixedWidth = true) + propertyHeader, jvmRow, jvmInformation.toSeq.sorted, fixedWidth = true, + headerClasses = headerClasses) val sparkPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, - Utils.redact(conf, appEnv.sparkProperties.sorted), fixedWidth = true) + Utils.redact(conf, appEnv.sparkProperties.sorted), fixedWidth = true, + headerClasses = headerClasses) val hadoopPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, - Utils.redact(conf, appEnv.hadoopProperties.sorted), fixedWidth = true) + Utils.redact(conf, appEnv.hadoopProperties.sorted), fixedWidth = true, + headerClasses = headerClasses) val systemPropertiesTable = UIUtils.listingTable(propertyHeader, propertyRow, - Utils.redact(conf, appEnv.systemProperties.sorted), fixedWidth = true) + Utils.redact(conf, appEnv.systemProperties.sorted), fixedWidth = true, + headerClasses = headerClasses) val classpathEntriesTable = UIUtils.listingTable( - classPathHeaders, classPathRow, appEnv.classpathEntries.sorted, fixedWidth = true) + classPathHeader, classPathRow, appEnv.classpathEntries.sorted, fixedWidth = true, + headerClasses = headerClasses) val content = {kv._1}{kv._2} private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} From 564826d960f523036e505545dd008f78c12df7ee Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Sat, 23 Nov 2019 19:34:19 -0800 Subject: [PATCH 0702/1387] [SPARK-28812][SQL][DOC] Document SHOW PARTITIONS in SQL Reference ### What changes were proposed in this pull request? Document SHOW PARTITIONS statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before** **After** ![image](https://user-images.githubusercontent.com/14225158/69405056-89468180-0cb3-11ea-8eb7-93046eaf551c.png) ![image](https://user-images.githubusercontent.com/14225158/69405067-93688000-0cb3-11ea-810a-11cab9e4a041.png) ![image](https://user-images.githubusercontent.com/14225158/69405120-c01c9780-0cb3-11ea-91c0-91eeaa9238a0.png) Closes #26635 from dilipbiswal/show_partitions. Authored-by: Dilip Biswal Signed-off-by: Dongjoon Hyun --- docs/sql-ref-syntax-aux-show-partitions.md | 83 +++++++++++++++++++++- 1 file changed, 82 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-aux-show-partitions.md b/docs/sql-ref-syntax-aux-show-partitions.md index c6499de9cbb9e..216f3f0d679ec 100644 --- a/docs/sql-ref-syntax-aux-show-partitions.md +++ b/docs/sql-ref-syntax-aux-show-partitions.md @@ -18,5 +18,86 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- +### Description -**This page is under construction** +The `SHOW PARTITIONS` statement is used to list partitions of a table. An optional +partition spec may be specified to return the partitions matching the supplied +partition spec. + +### Syntax +{% highlight sql %} +SHOW PARTITIONS table_name + [ PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) ] +{% endhighlight %} + +### Parameters +
    +
    table_name
    +
    The name of an existing table.
    +
    +
    +
    PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] )
    +
    An optional parameter that specifies a comma separated list of key and value pairs for + partitions. When specified, the partitions that match the partition spec are returned.
    +
    + +### Examples +{% highlight sql %} +-- create a partitioned table and insert a few rows. +USE salesdb; +CREATE TABLE customer(id INT, name STRING) PARTITIONED BY (state STRING, city STRING); +INSERT INTO customer PARTITION (state = 'CA', city = 'Fremont') VALUES (100, 'John'); +INSERT INTO customer PARTITION (state = 'CA', city = 'San Jose') VALUES (200, 'Marry'); +INSERT INTO customer PARTITION (state = 'AZ', city = 'Peoria') VALUES (300, 'Daniel'); + +-- Lists all partitions for table `customer` +SHOW PARTITIONS customer; + +----------------------+ + |partition | + +----------------------+ + |state=AZ/city=Peoria | + |state=CA/city=Fremont | + |state=CA/city=San Jose| + +----------------------+ + +-- Lists all partitions for the qualified table `customer` +SHOW PARTITIONS salesdb.customer; + +----------------------+ + |partition | + +----------------------+ + |state=AZ/city=Peoria | + |state=CA/city=Fremont | + |state=CA/city=San Jose| + +----------------------+ + +-- Specify a full partition spec to list specific partition +SHOW PARTITIONS customer PARTITION (state = 'CA', city = 'Fremont'); + +---------------------+ + |partition | + +---------------------+ + |state=CA/city=Fremont| + +---------------------+ + +-- Specify a partial partition spec to list the specific partitions +SHOW PARTITIONS customer PARTITION (state = 'CA'); + +----------------------+ + |partition | + +----------------------+ + |state=CA/city=Fremont | + |state=CA/city=San Jose| + +----------------------+ + +-- Specify a partial spec to list specific partition +SHOW PARTITIONS customer PARTITION (city = 'San Jose'); + +----------------------+ + |partition | + +----------------------+ + |state=CA/city=San Jose| + +----------------------+ +{% endhighlight %} + +### Related statements +- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) +- [INSERT STATEMENT](sql-ref-syntax-dml-insert.html) +- [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) +- [SHOW TABLE](sql-ref-syntax-aux-show-table.html) From 13338eaa9509fa526438067aeb4cd41f4048931a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2019 19:53:52 -0800 Subject: [PATCH 0703/1387] [SPARK-29554][SQL][FOLLOWUP] Rename Version to SparkVersion ### What changes were proposed in this pull request? This is a follow-up of https://github.com/apache/spark/pull/26209 . This renames class `Version` to class `SparkVersion`. ### Why are the changes needed? According to the review comment, this uses more specific class name. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #26647 from dongjoon-hyun/SPARK-29554. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/analysis/FunctionRegistry.scala | 2 +- .../scala/org/apache/spark/sql/catalyst/expressions/misc.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c0e0330ff1e14..7cc64d43858c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -490,7 +490,7 @@ object FunctionRegistry { expression[CurrentDatabase]("current_database"), expression[CallMethodViaReflection]("reflect"), expression[CallMethodViaReflection]("java_method"), - expression[Version]("version"), + expression[SparkVersion]("version"), expression[TypeOf]("typeof"), // grouping sets diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index def81f8dfb72b..f576873829f27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -169,7 +169,7 @@ case class Uuid(randomSeed: Option[Long] = None) extends LeafExpression with Sta usage = """_FUNC_() - Returns the Spark version. The string contains 2 fields, the first being a release version and the second being a git revision.""", since = "3.0.0") // scalastyle:on line.size.limit -case class Version() extends LeafExpression with CodegenFallback { +case class SparkVersion() extends LeafExpression with CodegenFallback { override def nullable: Boolean = false override def foldable: Boolean = true override def dataType: DataType = StringType From a60da23d648c186dd17f43611226892db2508b09 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 23 Nov 2019 22:34:21 -0800 Subject: [PATCH 0704/1387] [SPARK-30007][INFRA] Publish snapshot/release artifacts with `-Phive-2.3` only ### What changes were proposed in this pull request? This PR aims to add `-Phive-2.3` to publish profiles. Since Apache Spark 3.0.0, Maven artifacts will be publish with Apache Hive 2.3 profile only. This PR also will recover `SNAPSHOT` publishing Jenkins job. - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20Packaging/job/spark-master-maven-snapshots/ We will provide the pre-built distributions (with Hive 1.2.1 also) like Apache Spark 2.4. SPARK-29989 will update the release script to generate all combinations. ### Why are the changes needed? This will reduce the explicit dependency on the illegitimate Hive fork in Maven repository. ### Does this PR introduce any user-facing change? Yes, but this is dev only changes. ### How was this patch tested? Manual. Closes #26648 from dongjoon-hyun/SPARK-30007. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/release-build.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index f1069d4490b5a..99c4b20102929 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -138,7 +138,8 @@ fi # Hive-specific profiles for some builds HIVE_PROFILES="-Phive -Phive-thriftserver" # Profiles for publishing snapshots and release to Maven Central -PUBLISH_PROFILES="$BASE_PROFILES $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl" +# We use Apache Hive 2.3 for publishing +PUBLISH_PROFILES="$BASE_PROFILES $HIVE_PROFILES -Phive-2.3 -Pspark-ganglia-lgpl -Pkinesis-asl" # Profiles for building binary releases BASE_RELEASE_PROFILES="$BASE_PROFILES -Psparkr" From 3d740901d686b33a04bff9459a6b3866e617d2ad Mon Sep 17 00:00:00 2001 From: uncleGen Date: Sun, 24 Nov 2019 08:08:15 -0600 Subject: [PATCH 0705/1387] [SPARK-29973][SS] Make `processedRowsPerSecond` calculated more accurately and meaningfully ### What changes were proposed in this pull request? Give `processingTimeSec` 0.001 when a micro-batch completed under 1ms. ### Why are the changes needed? The `processingTimeSec` of batch may be less than 1 ms. As `processingTimeSec` is calculated in ms, so `processingTimeSec` equals 0L. If there is no data in this batch, the `processedRowsPerSecond` equals `0/0.0d`, i.e. `Double.NaN`. If there are some data in this batch, the `processedRowsPerSecond` equals `N/0.0d`, i.e. `Double.Infinity`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add new UT Closes #26610 from uncleGen/SPARK-29973. Authored-by: uncleGen Signed-off-by: Sean Owen --- .../streaming/ProgressReporter.scala | 4 +- ...StreamingQueryStatusAndProgressSuite.scala | 42 +++++++++++++++++++ 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 8b4b1aee0dd12..4f42992126c49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -148,8 +148,8 @@ trait ProgressReporter extends Logging { currentTriggerEndTimestamp = triggerClock.getTimeMillis() val executionStats = extractExecutionStats(hasNewData) - val processingTimeSec = - (currentTriggerEndTimestamp - currentTriggerStartTimestamp).toDouble / MILLIS_PER_SECOND + val processingTimeSec = Math.max(1L, + currentTriggerEndTimestamp - currentTriggerStartTimestamp).toDouble / MILLIS_PER_SECOND val inputTimeSec = if (lastTriggerStartTimestamp >= 0) { (currentTriggerStartTimestamp - lastTriggerStartTimestamp).toDouble / MILLIS_PER_SECOND diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index e784d318b4ffa..da291f490b76c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -24,12 +24,15 @@ import scala.collection.JavaConverters._ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.scalatest.concurrent.Eventually +import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQueryStatusAndProgressSuite._ +import org.apache.spark.sql.streaming.StreamingQuerySuite.clock +import org.apache.spark.sql.streaming.util.StreamManualClock class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { test("StreamingQueryProgress - prettyJson") { @@ -215,6 +218,45 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { } } + test("SPARK-29973: Make `processedRowsPerSecond` calculated more accurately and meaningfully") { + import testImplicits._ + + clock = new StreamManualClock + val inputData = MemoryStream[Int] + val query = inputData.toDS() + + testStream(query)( + StartStream(Trigger.ProcessingTime(1000), triggerClock = clock), + AdvanceManualClock(1000), + waitUntilBatchProcessed, + AssertOnQuery(query => { + assert(query.lastProgress.numInputRows == 0) + assert(query.lastProgress.processedRowsPerSecond == 0.0d) + true + }), + AddData(inputData, 1, 2), + AdvanceManualClock(1000), + waitUntilBatchProcessed, + AssertOnQuery(query => { + assert(query.lastProgress.numInputRows == 2) + assert(query.lastProgress.processedRowsPerSecond == 2000d) + true + }), + StopStream + ) + } + + def waitUntilBatchProcessed: AssertOnQuery = Execute { q => + eventually(Timeout(streamingTimeout)) { + if (q.exception.isEmpty) { + assert(clock.isStreamWaitingAt(clock.getTimeMillis())) + } + } + if (q.exception.isDefined) { + throw q.exception.get + } + } + def assertJson(source: String, expected: String): Unit = { assert( source.replaceAll("\r\n|\r|\n", System.lineSeparator) === From 3f3a18fff116a02ff7996d45a1061f48a2de3102 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 24 Nov 2019 08:30:24 -0800 Subject: [PATCH 0706/1387] [SPARK-24690][SQL] Add a config to control plan stats computation in LogicalRelation ### What changes were proposed in this pull request? This pr proposes a new independent config so that `LogicalRelation` could use `rowCount` to compute data statistics in logical plans even if CBO disabled. In the master, we currently cannot enable `StarSchemaDetection.reorderStarJoins` because we need to turn off CBO to enable it but `StarSchemaDetection` internally references the `rowCount` that is used in LogicalRelation if CBO disabled. ### Why are the changes needed? Plan stats are pretty useful other than CBO, e.g., star-schema detector and dynamic partition pruning. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added tests in `DataFrameJoinSuite`. Closes #21668 from maropu/PlanStatsConf. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/catalog/interface.scala | 10 +-- .../apache/spark/sql/internal/SQLConf.scala | 8 +++ .../datasources/LogicalRelation.scala | 2 +- .../apache/spark/sql/DataFrameJoinSuite.scala | 67 ++++++++++++++++++- 4 files changed, 79 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index f653bf41c1624..7ee191ca39d97 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -388,16 +388,16 @@ case class CatalogStatistics( * Convert [[CatalogStatistics]] to [[Statistics]], and match column stats to attributes based * on column names. */ - def toPlanStats(planOutput: Seq[Attribute], cboEnabled: Boolean): Statistics = { - if (cboEnabled && rowCount.isDefined) { + def toPlanStats(planOutput: Seq[Attribute], planStatsEnabled: Boolean): Statistics = { + if (planStatsEnabled && rowCount.isDefined) { val attrStats = AttributeMap(planOutput .flatMap(a => colStats.get(a.name).map(a -> _.toPlanStat(a.name, a.dataType)))) // Estimate size as number of rows * row size. val size = EstimationUtils.getOutputSize(planOutput, rowCount.get, attrStats) Statistics(sizeInBytes = size, rowCount = rowCount, attributeStats = attrStats) } else { - // When CBO is disabled or the table doesn't have other statistics, we apply the size-only - // estimation strategy and only propagate sizeInBytes in statistics. + // When plan statistics are disabled or the table doesn't have other statistics, + // we apply the size-only estimation strategy and only propagate sizeInBytes in statistics. Statistics(sizeInBytes = sizeInBytes) } } @@ -634,7 +634,7 @@ case class HiveTableRelation( ) override def computeStats(): Statistics = { - tableMeta.stats.map(_.toPlanStats(output, conf.cboEnabled)) + tableMeta.stats.map(_.toPlanStats(output, conf.cboEnabled || conf.planStatsEnabled)) .orElse(tableStats) .getOrElse { throw new IllegalStateException("table stats must be specified.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 33f91d045f7d9..e9f8a60978220 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1396,6 +1396,12 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PLAN_STATS_ENABLED = + buildConf("spark.sql.cbo.planStats.enabled") + .doc("When true, the logical plan will fetch row counts and column statistics from catalog.") + .booleanConf + .createWithDefault(false) + val JOIN_REORDER_ENABLED = buildConf("spark.sql.cbo.joinReorder.enabled") .doc("Enables join reorder in CBO.") @@ -2427,6 +2433,8 @@ class SQLConf extends Serializable with Logging { def cboEnabled: Boolean = getConf(SQLConf.CBO_ENABLED) + def planStatsEnabled: Boolean = getConf(SQLConf.PLAN_STATS_ENABLED) + def autoSizeUpdateEnabled: Boolean = getConf(SQLConf.AUTO_SIZE_UPDATE_ENABLED) def joinReorderEnabled: Boolean = getConf(SQLConf.JOIN_REORDER_ENABLED) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 35bda5682fda1..33a3486bf6f67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -41,7 +41,7 @@ case class LogicalRelation( override def computeStats(): Statistics = { catalogTable - .flatMap(_.stats.map(_.toPlanStats(output, conf.cboEnabled))) + .flatMap(_.stats.map(_.toPlanStats(output, conf.cboEnabled || conf.planStatsEnabled))) .getOrElse(Statistics(sizeInBytes = relation.sizeInBytes)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 3a217e6e28060..8e0cb839fbab5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter} -import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -256,4 +257,66 @@ class DataFrameJoinSuite extends QueryTest with SharedSparkSession { df.join(df, df("id") <=> df("id")).queryExecution.optimizedPlan } } + + def extractLeftDeepInnerJoins(plan: LogicalPlan): Seq[LogicalPlan] = plan match { + case j @ Join(left, right, _: InnerLike, _, _) => right +: extractLeftDeepInnerJoins(left) + case Filter(_, child) => extractLeftDeepInnerJoins(child) + case Project(_, child) => extractLeftDeepInnerJoins(child) + case _ => Seq(plan) + } + + test("SPARK-24690 enables star schema detection even if CBO disabled") { + withTable("r0", "r1", "r2", "r3") { + withTempDir { dir => + + withSQLConf( + SQLConf.STARSCHEMA_DETECTION.key -> "true", + SQLConf.CBO_ENABLED.key -> "false", + SQLConf.PLAN_STATS_ENABLED.key -> "true") { + + val path = dir.getAbsolutePath + + // Collects column statistics first + spark.range(300).selectExpr("id AS a", "id AS b", "id AS c") + .write.mode("overwrite").parquet(s"$path/r0") + spark.read.parquet(s"$path/r0").write.saveAsTable("r0") + spark.sql("ANALYZE TABLE r0 COMPUTE STATISTICS FOR COLUMNS a, b, c") + + spark.range(10).selectExpr("id AS a", "id AS d") + .write.mode("overwrite").parquet(s"$path/r1") + spark.read.parquet(s"$path/r1").write.saveAsTable("r1") + spark.sql("ANALYZE TABLE r1 COMPUTE STATISTICS FOR COLUMNS a") + + spark.range(50).selectExpr("id AS b", "id AS e") + .write.mode("overwrite").parquet(s"$path/r2") + spark.read.parquet(s"$path/r2").write.saveAsTable("r2") + spark.sql("ANALYZE TABLE r2 COMPUTE STATISTICS FOR COLUMNS b") + + spark.range(1).selectExpr("id AS c", "id AS f") + .write.mode("overwrite").parquet(s"$path/r3") + spark.read.parquet(s"$path/r3").write.saveAsTable("r3") + spark.sql("ANALYZE TABLE r3 COMPUTE STATISTICS FOR COLUMNS c") + + val resultDf = sql( + s"""SELECT * FROM r0, r1, r2, r3 + | WHERE + | r0.a = r1.a AND + | r1.d >= 3 AND + | r0.b = r2.b AND + | r2.e >= 5 AND + | r0.c = r3.c AND + | r3.f <= 100 + """.stripMargin) + + val optimized = resultDf.queryExecution.optimizedPlan + val optJoins = extractLeftDeepInnerJoins(optimized) + val joinOrder = optJoins + .flatMap(_.collect { case p: LogicalRelation => p.catalogTable }.head) + .map(_.identifier.identifier) + + assert(joinOrder === Seq("r2", "r1", "r3", "r0")) + } + } + } + } } From a1706e2fa745a8d4aebeebd1663366e13ba80c8c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 24 Nov 2019 10:14:02 -0800 Subject: [PATCH 0707/1387] [SPARK-30005][INFRA] Update `test-dependencies.sh` to check `hive-1.2/2.3` profile ### What changes were proposed in this pull request? This PR aims to update `test-dependencies.sh` to validate all available `Hadoop/Hive` combination. ### Why are the changes needed? Previously, we have been checking only `Hadoop2.7/Hive1.2` and `Hadoop3.2/Hive2.3`. We need to validate `Hadoop2.7/Hive2.3` additionally for Apache Spark 3.0. ### Does this PR introduce any user-facing change? No. (This is a dev-only change). ### How was this patch tested? Pass the GitHub Action (Linter) with the newly updated manifest because this is only dependency check. Closes #26646 from dongjoon-hyun/SPARK-30005. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- ...oop-2.7 => spark-deps-hadoop-2.7-hive-1.2} | 0 dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 224 ++++++++++++++++++ ...oop-3.2 => spark-deps-hadoop-3.2-hive-2.3} | 0 dev/test-dependencies.sh | 30 ++- 4 files changed, 242 insertions(+), 12 deletions(-) rename dev/deps/{spark-deps-hadoop-2.7 => spark-deps-hadoop-2.7-hive-1.2} (100%) create mode 100644 dev/deps/spark-deps-hadoop-2.7-hive-2.3 rename dev/deps/{spark-deps-hadoop-3.2 => spark-deps-hadoop-3.2-hive-2.3} (100%) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 similarity index 100% rename from dev/deps/spark-deps-hadoop-2.7 rename to dev/deps/spark-deps-hadoop-2.7-hive-1.2 diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 new file mode 100644 index 0000000000000..1fb9bbbd9b054 --- /dev/null +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -0,0 +1,224 @@ +HikariCP-2.5.1.jar +JLargeArrays-1.5.jar +JTransforms-3.1.jar +RoaringBitmap-0.7.45.jar +ST4-4.0.4.jar +activation-1.1.1.jar +aircompressor-0.10.jar +algebra_2.12-2.0.0-M2.jar +antlr-runtime-3.5.2.jar +antlr4-runtime-4.7.1.jar +aopalliance-1.0.jar +aopalliance-repackaged-2.5.0.jar +apacheds-i18n-2.0.0-M15.jar +apacheds-kerberos-codec-2.0.0-M15.jar +api-asn1-api-1.0.0-M20.jar +api-util-1.0.0-M20.jar +arpack_combined_all-0.1.jar +arrow-format-0.15.1.jar +arrow-memory-0.15.1.jar +arrow-vector-0.15.1.jar +audience-annotations-0.5.0.jar +automaton-1.11-8.jar +avro-1.8.2.jar +avro-ipc-1.8.2.jar +avro-mapred-1.8.2-hadoop2.jar +bonecp-0.8.0.RELEASE.jar +breeze-macros_2.12-1.0.jar +breeze_2.12-1.0.jar +cats-kernel_2.12-2.0.0-M4.jar +chill-java-0.9.3.jar +chill_2.12-0.9.3.jar +commons-beanutils-1.9.4.jar +commons-cli-1.2.jar +commons-codec-1.10.jar +commons-collections-3.2.2.jar +commons-compiler-3.0.15.jar +commons-compress-1.8.1.jar +commons-configuration-1.6.jar +commons-crypto-1.0.0.jar +commons-dbcp-1.4.jar +commons-digester-1.8.jar +commons-httpclient-3.1.jar +commons-io-2.4.jar +commons-lang-2.6.jar +commons-lang3-3.8.1.jar +commons-logging-1.1.3.jar +commons-math3-3.4.1.jar +commons-net-3.1.jar +commons-pool-1.5.4.jar +commons-text-1.6.jar +compress-lzf-1.0.3.jar +core-1.1.2.jar +curator-client-2.7.1.jar +curator-framework-2.7.1.jar +curator-recipes-2.7.1.jar +datanucleus-api-jdo-4.2.4.jar +datanucleus-core-4.1.17.jar +datanucleus-rdbms-4.1.19.jar +derby-10.12.1.1.jar +dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar +flatbuffers-java-1.9.0.jar +generex-1.0.2.jar +gson-2.2.4.jar +guava-14.0.1.jar +guice-3.0.jar +guice-servlet-3.0.jar +hadoop-annotations-2.7.4.jar +hadoop-auth-2.7.4.jar +hadoop-client-2.7.4.jar +hadoop-common-2.7.4.jar +hadoop-hdfs-2.7.4.jar +hadoop-mapreduce-client-app-2.7.4.jar +hadoop-mapreduce-client-common-2.7.4.jar +hadoop-mapreduce-client-core-2.7.4.jar +hadoop-mapreduce-client-jobclient-2.7.4.jar +hadoop-mapreduce-client-shuffle-2.7.4.jar +hadoop-yarn-api-2.7.4.jar +hadoop-yarn-client-2.7.4.jar +hadoop-yarn-common-2.7.4.jar +hadoop-yarn-server-common-2.7.4.jar +hadoop-yarn-server-web-proxy-2.7.4.jar +hive-beeline-2.3.6.jar +hive-cli-2.3.6.jar +hive-common-2.3.6.jar +hive-exec-2.3.6-core.jar +hive-jdbc-2.3.6.jar +hive-llap-common-2.3.6.jar +hive-metastore-2.3.6.jar +hive-serde-2.3.6.jar +hive-service-rpc-2.3.6.jar +hive-shims-0.23-2.3.6.jar +hive-shims-2.3.6.jar +hive-shims-common-2.3.6.jar +hive-shims-scheduler-2.3.6.jar +hive-storage-api-2.6.0.jar +hive-vector-code-gen-2.3.6.jar +hk2-api-2.5.0.jar +hk2-locator-2.5.0.jar +hk2-utils-2.5.0.jar +htrace-core-3.1.0-incubating.jar +httpclient-4.5.6.jar +httpcore-4.4.10.jar +istack-commons-runtime-3.0.8.jar +ivy-2.4.0.jar +jackson-annotations-2.10.0.jar +jackson-core-2.10.0.jar +jackson-core-asl-1.9.13.jar +jackson-databind-2.10.0.jar +jackson-dataformat-yaml-2.10.0.jar +jackson-jaxrs-1.9.13.jar +jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations-2.10.0.jar +jackson-module-paranamer-2.10.0.jar +jackson-module-scala_2.12-2.10.0.jar +jackson-xc-1.9.13.jar +jakarta.activation-api-1.2.1.jar +jakarta.annotation-api-1.3.4.jar +jakarta.inject-2.5.0.jar +jakarta.ws.rs-api-2.1.5.jar +jakarta.xml.bind-api-2.3.2.jar +janino-3.0.15.jar +javassist-3.22.0-CR2.jar +javax.inject-1.jar +javax.jdo-3.2.0-m3.jar +javax.servlet-api-3.1.0.jar +javolution-5.5.1.jar +jaxb-api-2.2.2.jar +jaxb-runtime-2.3.2.jar +jcl-over-slf4j-1.7.16.jar +jdo-api-3.0.1.jar +jersey-client-2.29.jar +jersey-common-2.29.jar +jersey-container-servlet-2.29.jar +jersey-container-servlet-core-2.29.jar +jersey-hk2-2.29.jar +jersey-media-jaxb-2.29.jar +jersey-server-2.29.jar +jetty-6.1.26.jar +jetty-sslengine-6.1.26.jar +jetty-util-6.1.26.jar +jline-2.14.6.jar +joda-time-2.10.5.jar +jodd-core-3.5.2.jar +jpam-1.1.jar +json-1.8.jar +json4s-ast_2.12-3.6.6.jar +json4s-core_2.12-3.6.6.jar +json4s-jackson_2.12-3.6.6.jar +json4s-scalap_2.12-3.6.6.jar +jsp-api-2.1.jar +jsr305-3.0.0.jar +jta-1.1.jar +jul-to-slf4j-1.7.16.jar +kryo-shaded-4.0.2.jar +kubernetes-client-4.6.1.jar +kubernetes-model-4.6.1.jar +kubernetes-model-common-4.6.1.jar +leveldbjni-all-1.8.jar +libfb303-0.9.3.jar +libthrift-0.12.0.jar +log4j-1.2.17.jar +logging-interceptor-3.12.0.jar +lz4-java-1.6.0.jar +machinist_2.12-0.6.8.jar +macro-compat_2.12-1.1.1.jar +mesos-1.4.0-shaded-protobuf.jar +metrics-core-4.1.1.jar +metrics-graphite-4.1.1.jar +metrics-jmx-4.1.1.jar +metrics-json-4.1.1.jar +metrics-jvm-4.1.1.jar +minlog-1.3.0.jar +netty-all-4.1.42.Final.jar +objenesis-2.5.1.jar +okapi-shade-0.4.2.jar +okhttp-3.12.0.jar +okio-1.15.0.jar +opencsv-2.3.jar +orc-core-1.5.7.jar +orc-mapreduce-1.5.7.jar +orc-shims-1.5.7.jar +oro-2.0.8.jar +osgi-resource-locator-1.0.3.jar +paranamer-2.8.jar +parquet-column-1.10.1.jar +parquet-common-1.10.1.jar +parquet-encoding-1.10.1.jar +parquet-format-2.4.0.jar +parquet-hadoop-1.10.1.jar +parquet-jackson-1.10.1.jar +protobuf-java-2.5.0.jar +py4j-0.10.8.1.jar +pyrolite-4.30.jar +scala-collection-compat_2.12-2.1.1.jar +scala-compiler-2.12.10.jar +scala-library-2.12.10.jar +scala-parser-combinators_2.12-1.1.2.jar +scala-reflect-2.12.10.jar +scala-xml_2.12-1.2.0.jar +shapeless_2.12-2.3.3.jar +shims-0.7.45.jar +slf4j-api-1.7.16.jar +slf4j-log4j12-1.7.16.jar +snakeyaml-1.24.jar +snappy-java-1.1.7.3.jar +spire-macros_2.12-0.17.0-M1.jar +spire-platform_2.12-0.17.0-M1.jar +spire-util_2.12-0.17.0-M1.jar +spire_2.12-0.17.0-M1.jar +stax-api-1.0-2.jar +stax-api-1.0.1.jar +stream-2.9.6.jar +super-csv-2.2.0.jar +transaction-api-1.1.jar +univocity-parsers-2.8.3.jar +validation-api-2.0.1.Final.jar +velocity-1.5.jar +xbean-asm7-shaded-4.15.jar +xercesImpl-2.9.1.jar +xmlenc-0.52.jar +xz-1.5.jar +zjsonpatch-0.3.0.jar +zookeeper-3.4.14.jar +zstd-jni-1.4.3-1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 similarity index 100% rename from dev/deps/spark-deps-hadoop-3.2 rename to dev/deps/spark-deps-hadoop-3.2-hive-2.3 diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 7d5725aaf137e..ad5cc75d72d10 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -31,9 +31,10 @@ export LC_ALL=C # NOTE: These should match those in the release publishing script HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive" MVN="build/mvn" -HADOOP_PROFILES=( - hadoop-2.7 - hadoop-3.2 +HADOOP_HIVE_PROFILES=( + hadoop-2.7-hive-1.2 + hadoop-2.7-hive-2.3 + hadoop-3.2-hive-2.3 ) # We'll switch the version to a temp. one, publish POMs using that new version, then switch back to @@ -66,24 +67,29 @@ trap reset_version EXIT $MVN -q versions:set -DnewVersion=$TEMP_VERSION -DgenerateBackupPoms=false > /dev/null # Generate manifests for each Hadoop profile: -for HADOOP_PROFILE in "${HADOOP_PROFILES[@]}"; do - if [[ $HADOOP_PROFILE == **hadoop-3** ]]; then +for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do + if [[ $HADOOP_HIVE_PROFILE == **hadoop-3.2-hive-2.3** ]]; then + HADOOP_PROFILE=hadoop-3.2 + HIVE_PROFILE=hive-2.3 + elif [[ $HADOOP_HIVE_PROFILE == **hadoop-2.7-hive-2.3** ]]; then + HADOOP_PROFILE=hadoop-2.7 HIVE_PROFILE=hive-2.3 else + HADOOP_PROFILE=hadoop-2.7 HIVE_PROFILE=hive-1.2 fi - echo "Performing Maven install for $HADOOP_PROFILE" + echo "Performing Maven install for $HADOOP_HIVE_PROFILE" $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE jar:jar jar:test-jar install:install clean -q - echo "Performing Maven validate for $HADOOP_PROFILE" + echo "Performing Maven validate for $HADOOP_HIVE_PROFILE" $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE validate -q - echo "Generating dependency manifest for $HADOOP_PROFILE" + echo "Generating dependency manifest for $HADOOP_HIVE_PROFILE" mkdir -p dev/pr-deps $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE dependency:build-classpath -pl assembly -am \ | grep "Dependencies classpath:" -A 1 \ | tail -n 1 | tr ":" "\n" | rev | cut -d "/" -f 1 | rev | sort \ - | grep -v spark > dev/pr-deps/spark-deps-$HADOOP_PROFILE + | grep -v spark > dev/pr-deps/spark-deps-$HADOOP_HIVE_PROFILE done if [[ $@ == **replace-manifest** ]]; then @@ -93,13 +99,13 @@ if [[ $@ == **replace-manifest** ]]; then exit 0 fi -for HADOOP_PROFILE in "${HADOOP_PROFILES[@]}"; do +for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do set +e dep_diff="$( git diff \ --no-index \ - dev/deps/spark-deps-$HADOOP_PROFILE \ - dev/pr-deps/spark-deps-$HADOOP_PROFILE \ + dev/deps/spark-deps-$HADOOP_HIVE_PROFILE \ + dev/pr-deps/spark-deps-$HADOOP_HIVE_PROFILE \ )" set -e if [ "$dep_diff" != "" ]; then From cb68e58f88e8481e76b358f46fd4356d656e8277 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 24 Nov 2019 12:35:57 -0800 Subject: [PATCH 0708/1387] [MINOR][INFRA] Use GitHub Action Cache for `build` ### What changes were proposed in this pull request? This PR adds `GitHub Action Cache` task on `build` directory. ### Why are the changes needed? This will replace the Maven downloading with the cache. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually check the GitHub Action log of this PR. Closes #26652 from dongjoon-hyun/SPARK-MAVEN-CACHE. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 30f3272c8b933..9338dc7bcbd5e 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -29,6 +29,12 @@ jobs: steps: - uses: actions/checkout@master # We split caches because GitHub Action Cache has a 400MB-size limit. + - uses: actions/cache@v1 + with: + path: build + key: build-${{ hashFiles('**/pom.xml') }} + restore-keys: | + build- - uses: actions/cache@v1 with: path: ~/.m2/repository/com From 0d3d46db2110f6d84c44d10049b11c5c98af1f51 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sun, 24 Nov 2019 15:31:06 -0800 Subject: [PATCH 0709/1387] [SPARK-29999][SS] Handle FileStreamSink metadata correctly for empty partition ### What changes were proposed in this pull request? This patch checks the existence of output file for each task while committing the task, so that it doesn't throw FileNotFoundException while creating SinkFileStatus. The check is newly required for DSv2 implementation of FileStreamSink, as it is changed to create the output file lazily (as an improvement). JSON writer for example: https://github.com/apache/spark/blob/9ec2a4e58caa4128e9c690d72239cebd6b732084/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala#L49-L60 ### Why are the changes needed? Without this patch, FileStreamSink throws FileNotFoundException when writing empty partition. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added UT. Closes #26639 from HeartSaVioR/SPARK-29999. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../ManifestFileCommitProtocol.scala | 23 ++++++--- .../sql/streaming/FileStreamSinkSuite.scala | 50 +++++++++++++++++++ 2 files changed, 67 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index f6cc8116c6c4c..be7973b9d9300 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -22,7 +22,7 @@ import java.util.UUID import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.spark.internal.Logging @@ -89,9 +89,7 @@ class ManifestFileCommitProtocol(jobId: String, path: String) try { val fs = path.getFileSystem(jobContext.getConfiguration) // this is to make sure the file can be seen from driver as well - if (fs.exists(path)) { - fs.delete(path, false) - } + deleteIfExists(fs, path) } catch { case e: IOException => logWarning(s"Fail to remove temporary file $path, continue removing next.", e) @@ -139,7 +137,14 @@ class ManifestFileCommitProtocol(jobId: String, path: String) if (addedFiles.nonEmpty) { val fs = new Path(addedFiles.head).getFileSystem(taskContext.getConfiguration) val statuses: Seq[SinkFileStatus] = - addedFiles.map(f => SinkFileStatus(fs.getFileStatus(new Path(f)))) + addedFiles.flatMap { f => + val path = new Path(f) + if (fs.exists(path)) { + Some(SinkFileStatus(fs.getFileStatus(path))) + } else { + None + } + } new TaskCommitMessage(statuses) } else { new TaskCommitMessage(Seq.empty[SinkFileStatus]) @@ -150,7 +155,13 @@ class ManifestFileCommitProtocol(jobId: String, path: String) // best effort cleanup of incomplete files if (addedFiles.nonEmpty) { val fs = new Path(addedFiles.head).getFileSystem(taskContext.getConfiguration) - addedFiles.foreach { file => fs.delete(new Path(file), false) } + addedFiles.foreach { file => deleteIfExists(fs, new Path(file)) } + } + } + + private def deleteIfExists(fs: FileSystem, path: Path, recursive: Boolean = false): Unit = { + if (fs.exists(path)) { + fs.delete(path, recursive) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 0b885c8429c65..f04da8bfc4485 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -600,11 +600,61 @@ class FileStreamSinkV1Suite extends FileStreamSinkSuite { } class FileStreamSinkV2Suite extends FileStreamSinkSuite { + import testImplicits._ + override protected def sparkConf: SparkConf = super .sparkConf .set(SQLConf.USE_V1_SOURCE_LIST, "") + test("SPARK-29999 Handle FileStreamSink metadata correctly for empty partition") { + Seq("parquet", "orc", "text", "json").foreach { format => + val inputData = MemoryStream[String] + val df = inputData.toDF() + + withTempDir { outputDir => + withTempDir { checkpointDir => + var query: StreamingQuery = null + try { + // repartition to more than the input to leave empty partitions + query = + df.repartition(10) + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format(format) + .start(outputDir.getCanonicalPath) + + inputData.addData("1", "2", "3") + inputData.addData("4", "5") + + failAfter(streamingTimeout) { + query.processAllAvailable() + } + } finally { + if (query != null) { + query.stop() + } + } + + val fs = new Path(outputDir.getCanonicalPath).getFileSystem( + spark.sessionState.newHadoopConf()) + val sinkLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, spark, + outputDir.getCanonicalPath) + + val allFiles = sinkLog.allFiles() + // only files from non-empty partition should be logged + assert(allFiles.length < 10) + assert(allFiles.forall(file => fs.exists(new Path(file.path)))) + + // the query should be able to read all rows correctly with metadata log + val outputDf = spark.read.format(format).load(outputDir.getCanonicalPath) + .selectExpr("CAST(value AS INT)").as[Int] + checkDatasetUnorderly(outputDf, 1, 2, 3, 4, 5) + } + } + } + } + override def checkQueryExecution(df: DataFrame): Unit = { // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has // been inferred From a8d907ce947a523f61c24eda2a7efb0e809deadd Mon Sep 17 00:00:00 2001 From: ulysses Date: Sun, 24 Nov 2019 16:32:09 -0800 Subject: [PATCH 0710/1387] [SPARK-29937][SQL] Make FileSourceScanExec class fields lazy ### What changes were proposed in this pull request? Since JIRA SPARK-28346,PR [25111](https://github.com/apache/spark/pull/25111), QueryExecution will copy all node stage-by-stage. This make all node instance twice almost. So we should make all class fields lazy to avoid create more unexpected object. ### Why are the changes needed? Avoid create more unexpected object. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Exists UT. Closes #26565 from ulysses-you/make-val-lazy. Authored-by: ulysses Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/execution/DataSourceScanExec.scala | 5 ++--- .../spark/sql/execution/datasources/FileSourceStrategy.scala | 2 ++ 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 88f5673aa9a1e..0d759085a7e2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -187,7 +187,7 @@ case class FileSourceScanExec( partitionSchema = relation.partitionSchema, relation.sparkSession.sessionState.conf) - val driverMetrics: HashMap[String, Long] = HashMap.empty + private lazy val driverMetrics: HashMap[String, Long] = HashMap.empty /** * Send the driver-side metrics. Before calling this function, selectedPartitions has @@ -325,8 +325,7 @@ case class FileSourceScanExec( } @transient - private val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter) - logInfo(s"Pushed Filters: ${pushedDownFilters.mkString(",")}") + private lazy val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter) override lazy val metadata: Map[String, String] = { def seqToString(seq: Seq[Any]) = seq.mkString("[", ", ", "]") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index c8a42f043f15f..afc9bfeda84a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -177,6 +177,8 @@ object FileSourceStrategy extends Strategy with Logging { // Partition keys are not available in the statistics of the files. val dataFilters = normalizedFiltersWithoutSubqueries.filter(_.references.intersect(partitionSet).isEmpty) + logInfo(s"Pushed Filters: " + + s"${dataFilters.flatMap(DataSourceStrategy.translateFilter).mkString(",")}") // Predicates with both partition keys and attributes need to be evaluated after the scan. val afterScanFilters = filterSet -- partitionKeyFilters.filter(_.references.nonEmpty) From 13896e4eae4cbd3f908eb2b19d5b298a4b0c7f6e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 24 Nov 2019 18:23:34 -0800 Subject: [PATCH 0711/1387] [SPARK-30013][SQL] For scala 2.13, omit parens in various BigDecimal value() methods ### What changes were proposed in this pull request? Omit parens on calls like BigDecimal.longValue() ### Why are the changes needed? For some reason, this won't compile in Scala 2.13. The calls are otherwise equivalent in 2.12. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests Closes #26653 from srowen/SPARK-30013. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- pom.xml | 4 ++++ .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 2 +- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 4 ++-- .../plans/logical/statsEstimation/EstimationUtils.scala | 2 +- .../main/scala/org/apache/spark/sql/types/Decimal.scala | 8 ++++---- 5 files changed, 12 insertions(+), 8 deletions(-) diff --git a/pom.xml b/pom.xml index 825a227e8669b..1c12b9c5c7e4d 100644 --- a/pom.xml +++ b/pom.xml @@ -3013,6 +3013,10 @@ scala-2.13 + + 2.13.1 + 2.13 + 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 8d11f4663a3ef..017ace4106cd0 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 @@ -447,7 +447,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } private[this] def decimalToTimestamp(d: Decimal): Long = { - (d.toBigDecimal * MICROS_PER_SECOND).longValue() + (d.toBigDecimal * MICROS_PER_SECOND).longValue } private[this] def doubleToTimestamp(d: Double): Any = { if (d.isNaN || d.isInfinite) null else (d * MICROS_PER_SECOND).toLong diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 7bec46678f58d..37ff4ae981c33 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1934,9 +1934,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitIntegerLiteral(ctx: IntegerLiteralContext): Literal = withOrigin(ctx) { BigDecimal(ctx.getText) match { case v if v.isValidInt => - Literal(v.intValue()) + Literal(v.intValue) case v if v.isValidLong => - Literal(v.longValue()) + Literal(v.longValue) case v => Literal(v.underlying()) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala index 11d2f024c13a0..e1dbef9ebeede 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala @@ -63,7 +63,7 @@ object EstimationUtils { } } - def ceil(bigDecimal: BigDecimal): BigInt = bigDecimal.setScale(0, RoundingMode.CEILING).toBigInt() + def ceil(bigDecimal: BigDecimal): BigInt = bigDecimal.setScale(0, RoundingMode.CEILING).toBigInt /** Get column stats for output attributes. */ def getOutputMap(inputMap: AttributeMap[ColumnStat], output: Seq[Attribute]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 3136669f64803..24edbb2c82a32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -188,7 +188,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { def toScalaBigInt: BigInt = { if (decimalVal.ne(null)) { - decimalVal.toBigInt() + decimalVal.toBigInt } else { BigInt(toLong) } @@ -220,15 +220,15 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } - def toDouble: Double = toBigDecimal.doubleValue() + def toDouble: Double = toBigDecimal.doubleValue - def toFloat: Float = toBigDecimal.floatValue() + def toFloat: Float = toBigDecimal.floatValue def toLong: Long = { if (decimalVal.eq(null)) { longVal / POW_10(_scale) } else { - decimalVal.longValue() + decimalVal.longValue } } From 5cf475d2889d3a3b15d038b3f6faba77ac54006c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 25 Nov 2019 12:47:07 +0800 Subject: [PATCH 0712/1387] [SPARK-30000][SQL] Trim the string when cast string type to decimals ### What changes were proposed in this pull request? https://bugs.openjdk.java.net/browse/JDK-8170259 https://bugs.openjdk.java.net/browse/JDK-8170563 When we cast string type to decimal type, we rely on java.math. BigDecimal. It can't accept leading and training spaces, as you can see in the above links. This behavior is not consistent with other numeric types now. we need to fix it and keep consistency. ### Why are the changes needed? make string to numeric types be consistent ### Does this PR introduce any user-facing change? yes, string removed trailing or leading white spaces will be able to convert to decimal if the trimmed is valid ### How was this patch tested? 1. modify ut #### Benchmark ```scala /* * 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.benchmark import org.apache.spark.benchmark.Benchmark /** * Benchmark trim the string when casting string type to Boolean/Numeric types. * To run this benchmark: * {{{ * 1. without sbt: * bin/spark-submit --class --jars * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/CastBenchmark-results.txt". * }}} */ object CastBenchmark extends SqlBasedBenchmark { override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val title = "Cast String to Integral" runBenchmark(title) { withTempPath { dir => val N = 500L << 14 val df = spark.range(N) val types = Seq("decimal") (1 to 5).by(2).foreach { i => df.selectExpr(s"concat(id, '${" " * i}') as str") .write.mode("overwrite").parquet(dir + i.toString) } val benchmark = new Benchmark(title, N, minNumIters = 5, output = output) Seq(true, false).foreach { trim => types.foreach { t => val str = if (trim) "trim(str)" else "str" val expr = s"cast($str as $t) as c_$t" (1 to 5).by(2).foreach { i => benchmark.addCase(expr + s" - with $i spaces") { _ => spark.read.parquet(dir + i.toString).selectExpr(expr).collect() } } } } benchmark.run() } } } } ``` #### string trim vs not trim ```java [info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1 [info] Intel(R) Core(TM) i9-9980HK CPU 2.40GHz [info] Cast String to Integral: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative [info] ------------------------------------------------------------------------------------------------------------------------ [info] cast(trim(str) as decimal) as c_decimal - with 1 spaces 3362 5486 NaN 2.4 410.4 1.0X [info] cast(trim(str) as decimal) as c_decimal - with 3 spaces 3251 5655 NaN 2.5 396.8 1.0X [info] cast(trim(str) as decimal) as c_decimal - with 5 spaces 3208 5725 NaN 2.6 391.7 1.0X [info] cast(str as decimal) as c_decimal - with 1 spaces 13962 16233 1354 0.6 1704.3 0.2X [info] cast(str as decimal) as c_decimal - with 3 spaces 14273 14444 179 0.6 1742.4 0.2X [info] cast(str as decimal) as c_decimal - with 5 spaces 14318 14535 125 0.6 1747.8 0.2X ``` #### string trim vs this fix ```java [info] Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1 [info] Intel(R) Core(TM) i9-9980HK CPU 2.40GHz [info] Cast String to Integral: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative [info] ------------------------------------------------------------------------------------------------------------------------ [info] cast(trim(str) as decimal) as c_decimal - with 1 spaces 3265 6299 NaN 2.5 398.6 1.0X [info] cast(trim(str) as decimal) as c_decimal - with 3 spaces 3183 6241 693 2.6 388.5 1.0X [info] cast(trim(str) as decimal) as c_decimal - with 5 spaces 3167 5923 1151 2.6 386.7 1.0X [info] cast(str as decimal) as c_decimal - with 1 spaces 3161 5838 1126 2.6 385.9 1.0X [info] cast(str as decimal) as c_decimal - with 3 spaces 3046 3457 837 2.7 371.8 1.1X [info] cast(str as decimal) as c_decimal - with 5 spaces 3053 4445 NaN 2.7 372.7 1.1X [info] ``` Closes #26640 from yaooqinn/SPARK-30000. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 6 ++++-- sql/core/src/test/resources/sql-tests/results/cast.sql.out | 2 +- 2 files changed, 5 insertions(+), 3 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 017ace4106cd0..a871a746d64ff 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 @@ -632,7 +632,9 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { case StringType => buildCast[UTF8String](_, s => try { - changePrecision(Decimal(new JavaBigDecimal(s.toString)), target) + // According the benchmark test, `s.toString.trim` is much faster than `s.trim.toString`. + // Please refer to https://github.com/apache/spark/pull/26640 + changePrecision(Decimal(new JavaBigDecimal(s.toString.trim)), target) } catch { case _: NumberFormatException => null }) @@ -1128,7 +1130,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (c, evPrim, evNull) => code""" try { - Decimal $tmp = Decimal.apply(new java.math.BigDecimal($c.toString())); + Decimal $tmp = Decimal.apply(new java.math.BigDecimal($c.toString().trim())); ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast)} } catch (java.lang.NumberFormatException e) { $evNull = true; diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index bf1e873a48d17..112826a3d3158 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -350,4 +350,4 @@ select cast('1.0 ' as DEC) -- !query 42 schema struct -- !query 42 output -NULL +1 From 456cfe6e4693efd26d64f089d53c4e01bf8150a2 Mon Sep 17 00:00:00 2001 From: wuyi Date: Sun, 24 Nov 2019 21:21:19 -0800 Subject: [PATCH 0713/1387] [SPARK-29939][CORE] Add spark.shuffle.mapStatus.compression.codec conf ### What changes were proposed in this pull request? Add a new conf named `spark.shuffle.mapStatus.compression.codec` for user to decide which codec should be used(default by `zstd`) for `MapStatus` compression. ### Why are the changes needed? We already have this functionality for `broadcast`/`rdd`/`shuffle`/`shuflleSpill`, so it might be better to have the same functionality for `MapStatus` as well. ### Does this PR introduce any user-facing change? Yes, user now could use `spark.shuffle.mapStatus.compression.codec` to decide which codec should be used during `MapStatus` compression. ### How was this patch tested? N/A Closes #26611 from Ngone51/SPARK-29939. Authored-by: wuyi Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 4 ++-- .../scala/org/apache/spark/internal/config/package.scala | 9 +++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 3c6c181f9428c..d2169246fe7e8 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -902,7 +902,7 @@ private[spark] object MapOutputTracker extends Logging { // the contents don't have to be copied to the new buffer. val out = new ApacheByteArrayOutputStream() out.write(DIRECT) - val codec = CompressionCodec.createCodec(conf, "zstd") + val codec = CompressionCodec.createCodec(conf, conf.get(MAP_STATUS_COMPRESSION_CODEC)) val objOut = new ObjectOutputStream(codec.compressedOutputStream(out)) Utils.tryWithSafeFinally { // Since statuses can be modified in parallel, sync on it @@ -939,7 +939,7 @@ private[spark] object MapOutputTracker extends Logging { assert (bytes.length > 0) def deserializeObject(arr: Array[Byte], off: Int, len: Int): AnyRef = { - val codec = CompressionCodec.createCodec(conf, "zstd") + val codec = CompressionCodec.createCodec(conf, conf.get(MAP_STATUS_COMPRESSION_CODEC)) // The ZStd codec is wrapped in a `BufferedInputStream` which avoids overhead excessive // of JNI call while trying to decompress small amount of data for each element // of `MapStatuses` 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 8e8e36dbda94e..6a2eaf17d9f6c 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 @@ -1016,6 +1016,15 @@ package object config { .booleanConf .createWithDefault(true) + private[spark] val MAP_STATUS_COMPRESSION_CODEC = + ConfigBuilder("spark.shuffle.mapStatus.compression.codec") + .internal() + .doc("The codec used to compress MapStatus, which is generated by ShuffleMapTask. " + + "By default, Spark provides four codecs: lz4, lzf, snappy, and zstd. You can also " + + "use fully qualified class names to specify the codec.") + .stringConf + .createWithDefault("zstd") + private[spark] val SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD = ConfigBuilder("spark.shuffle.spill.initialMemoryThreshold") .internal() From de21f28f8a0a41dd7eb8ed1ff8b35a6d7538958b Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 25 Nov 2019 14:37:04 +0800 Subject: [PATCH 0714/1387] [SPARK-29986][SQL] casting string to date/timestamp/interval should trim all whitespaces ### What changes were proposed in this pull request? A java like string trim method trims all whitespaces that less or equal than 0x20. currently, our UTF8String handle the space =0x20 ONLY. This is not suitable for many cases in Spark, like trim for interval strings, date, timestamps, PostgreSQL like cast string to boolean. ### Why are the changes needed? improve the white spaces handling in UTF8String, also with some bugs fixed ### Does this PR introduce any user-facing change? yes, string with `control character` at either end can be convert to date/timestamp and interval now ### How was this patch tested? add ut Closes #26626 from yaooqinn/SPARK-29986. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../apache/spark/unsafe/types/UTF8String.java | 32 +++++++- .../spark/unsafe/types/UTF8StringSuite.java | 1 + docs/sql-migration-guide.md | 2 + .../postgreSQL/PostgreCastToBoolean.scala | 2 +- .../sql/catalyst/util/DateTimeUtils.scala | 4 +- .../sql/catalyst/util/IntervalUtils.scala | 2 +- .../resources/sql-tests/inputs/datetime.sql | 3 + .../resources/sql-tests/inputs/interval.sql | 5 +- .../sql-tests/results/ansi/interval.sql.out | 80 ++++++++++--------- .../sql-tests/results/datetime.sql.out | 18 ++++- .../sql-tests/results/interval.sql.out | 10 ++- 11 files changed, 114 insertions(+), 45 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index deecd4f015824..3754a1a0374a8 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -538,14 +538,42 @@ private UTF8String copyUTF8String(int start, int end) { public UTF8String trim() { int s = 0; // skip all of the space (0x20) in the left side - while (s < this.numBytes && getByte(s) == 0x20) s++; + while (s < this.numBytes && getByte(s) == ' ') s++; if (s == this.numBytes) { // Everything trimmed return EMPTY_UTF8; } // skip all of the space (0x20) in the right side int e = this.numBytes - 1; - while (e > s && getByte(e) == 0x20) e--; + while (e > s && getByte(e) == ' ') e--; + if (s == 0 && e == numBytes - 1) { + // Nothing trimmed + return this; + } + return copyUTF8String(s, e); + } + + /** + * Trims whitespaces (<= ASCII 32) from both ends of this string. + * + * Note that, this method is the same as java's {@link String#trim}, and different from + * {@link UTF8String#trim()} which remove only spaces(= ASCII 32) from both ends. + * + * @return A UTF8String whose value is this UTF8String, with any leading and trailing white + * space removed, or this UTF8String if it has no leading or trailing whitespace. + * + */ + public UTF8String trimAll() { + int s = 0; + // skip all of the whitespaces (<=0x20) in the left side + while (s < this.numBytes && getByte(s) <= ' ') s++; + if (s == this.numBytes) { + // Everything trimmed + return EMPTY_UTF8; + } + // skip all of the whitespaces (<=0x20) in the right side + int e = this.numBytes - 1; + while (e > s && getByte(e) <= ' ') e--; if (s == 0 && e == numBytes - 1) { // Nothing trimmed return this; diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index dbede9bc7f127..8f933877f82e6 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -227,6 +227,7 @@ public void substring() { @Test public void trims() { assertEquals(fromString("1"), fromString("1").trim()); + assertEquals(fromString("1"), fromString("1\t").trimAll()); assertEquals(fromString("hello"), fromString(" hello ").trim()); assertEquals(fromString("hello "), fromString(" hello ").trimLeft()); diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 6fc78893e6881..74ba694800d2d 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -224,6 +224,8 @@ license: | - Since Spark 3.0, when casting string value to integral types, including tinyint, smallint, int and bigint type, the leading and trailing white spaces(<= ACSII 32) will be trimmed before convert to integral values, e.g. `cast(' 1 ' as int)` results `1`. In Spark version 2.4 and earlier, the result will be `null`. + - Since Spark 3.0, when casting string value to date, timestamp and interval values, the leading and trailing white spaces(<= ACSII 32) will be trimmed before casing, e.g. `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, only the trailing space will be removed, thus, the result is `null`. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala index 20559ba3cd79e..02bc6f0d0d8bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala @@ -42,7 +42,7 @@ case class PostgreCastToBoolean(child: Expression, timeZoneId: Option[String]) override def castToBoolean(from: DataType): Any => Any = from match { case StringType => buildCast[UTF8String](_, str => { - val s = str.trim().toLowerCase() + val s = str.trimAll().toLowerCase() if (StringUtils.isTrueString(s)) { true } else if (StringUtils.isFalseString(s)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index f9f5e83d77b62..6800abb2ae109 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -205,7 +205,7 @@ object DateTimeUtils { val segments: Array[Int] = Array[Int](1, 1, 1, 0, 0, 0, 0, 0, 0) var i = 0 var currentSegmentValue = 0 - val bytes = s.trim.getBytes + val bytes = s.trimAll().getBytes val specialTimestamp = convertSpecialTimestamp(bytes, timeZoneId) if (specialTimestamp.isDefined) return specialTimestamp var j = 0 @@ -372,7 +372,7 @@ object DateTimeUtils { val segments: Array[Int] = Array[Int](1, 1, 1) var i = 0 var currentSegmentValue = 0 - val bytes = s.trim.getBytes + val bytes = s.trimAll().getBytes val specialDate = convertSpecialDate(bytes, zoneId) if (specialDate.isDefined) return specialDate var j = 0 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 9418d8eec3376..b8c7e4a9bac0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -465,7 +465,7 @@ object IntervalUtils { throwIAE("interval string cannot be null") } // scalastyle:off caselocale .toLowerCase - val s = input.trim.toLowerCase + val s = input.trimAll().toLowerCase // scalastyle:on val bytes = s.getBytes if (bytes.isEmpty) { diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 0e22af1fbdf29..de2040c651230 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -36,3 +36,6 @@ select date '2001-10-01' - 7; select date '2001-10-01' - date '2001-09-28'; select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'; select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'; + +select date '2019-01-01\t'; +select timestamp '2019-01-01\t'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index 94d5aae9bce31..886e242460547 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -256,4 +256,7 @@ from interval_arithmetic; select interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second -from interval_arithmetic; \ No newline at end of file +from interval_arithmetic; + +-- control characters as white spaces +select interval '\t interval 1 day'; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index bceb6bd1d2ea9..56628f615c3dc 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 130 +-- Number of queries: 131 -- !query 0 @@ -1116,34 +1116,42 @@ struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 -- !query 118 -select 1 year 2 days +select interval '\t interval 1 day' -- !query 118 schema -struct +struct -- !query 118 output -1 years 2 days +1 days -- !query 119 -select '10-9' year to month +select 1 year 2 days -- !query 119 schema -struct +struct -- !query 119 output -10 years 9 months +1 years 2 days -- !query 120 -select '20 15:40:32.99899999' day to second +select '10-9' year to month -- !query 120 schema -struct +struct -- !query 120 output -20 days 15 hours 40 minutes 32.998999 seconds +10 years 9 months -- !query 121 -select 30 day day +select '20 15:40:32.99899999' day to second -- !query 121 schema -struct<> +struct -- !query 121 output +20 days 15 hours 40 minutes 32.998999 seconds + + +-- !query 122 +select 30 day day +-- !query 122 schema +struct<> +-- !query 122 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 14) @@ -1153,27 +1161,27 @@ select 30 day day --------------^^^ --- !query 122 +-- !query 123 select date'2012-01-01' - '2-2' year to month --- !query 122 schema +-- !query 123 schema struct --- !query 122 output +-- !query 123 output 2009-11-01 --- !query 123 +-- !query 124 select 1 month - 1 day --- !query 123 schema +-- !query 124 schema struct --- !query 123 output +-- !query 124 output 1 months -1 days --- !query 124 +-- !query 125 select 1 year to month --- !query 124 schema +-- !query 125 schema struct<> --- !query 124 output +-- !query 125 output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 7) @@ -1183,11 +1191,11 @@ select 1 year to month -------^^^ --- !query 125 +-- !query 126 select '1' year to second --- !query 125 schema +-- !query 126 schema struct<> --- !query 125 output +-- !query 126 output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 7) @@ -1197,11 +1205,11 @@ select '1' year to second -------^^^ --- !query 126 +-- !query 127 select 1 year '2-1' year to month --- !query 126 schema +-- !query 127 schema struct<> --- !query 126 output +-- !query 127 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 14) @@ -1211,11 +1219,11 @@ select 1 year '2-1' year to month --------------^^^ --- !query 127 +-- !query 128 select (-30) day --- !query 127 schema +-- !query 128 schema struct<> --- !query 127 output +-- !query 128 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 13) @@ -1225,11 +1233,11 @@ select (-30) day -------------^^^ --- !query 128 +-- !query 129 select (a + 1) day --- !query 128 schema +-- !query 129 schema struct<> --- !query 128 output +-- !query 129 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 15) @@ -1239,11 +1247,11 @@ select (a + 1) day ---------------^^^ --- !query 129 +-- !query 130 select 30 day day day --- !query 129 schema +-- !query 130 schema struct<> --- !query 129 output +-- !query 130 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 14) diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index b2c6b878f4715..a269420a9a8b3 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 17 +-- Number of queries: 19 -- !query 0 @@ -145,3 +145,19 @@ select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' struct -- !query 16 output -2078 hours -48 minutes -47.654322 seconds + + +-- !query 17 +select date '2019-01-01\t' +-- !query 17 schema +struct +-- !query 17 output +2019-01-01 + + +-- !query 18 +select timestamp '2019-01-01\t' +-- !query 18 schema +struct +-- !query 18 output +2019-01-01 00:00:00 diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index bda5fc6d8b8e9..bf7b2184b68fc 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 118 +-- Number of queries: 119 -- !query 0 @@ -1097,3 +1097,11 @@ from interval_arithmetic struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> -- !query 117 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds + + +-- !query 118 +select interval '\t interval 1 day' +-- !query 118 schema +struct +-- !query 118 output +1 days From bd9ce830631e047b7a5650b5168a003cd0e47c4e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 25 Nov 2019 20:45:31 +0900 Subject: [PATCH 0715/1387] [SPARK-29975][SQL][FOLLOWUP] document --CONFIG_DIM ### What changes were proposed in this pull request? add document to address https://github.com/apache/spark/pull/26612#discussion_r349844327 ### Why are the changes needed? help people understand how to use --CONFIG_DIM ### Does this PR introduce any user-facing change? no ### How was this patch tested? N/A Closes #26661 from cloud-fan/test. Authored-by: Wenchen Fan Signed-off-by: Takeshi Yamamuro --- .../apache/spark/sql/SQLQueryTestSuite.scala | 22 ++++++++++++++----- .../ThriftServerQueryTestSuite.scala | 8 +++---- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index ef30573017212..e6dcf0b86308a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -64,8 +64,17 @@ import org.apache.spark.tags.ExtendedSQLTest * The format for input files is simple: * 1. A list of SQL queries separated by semicolon. * 2. Lines starting with -- are treated as comments and ignored. - * 3. Lines starting with --SET are used to run the file with the following set of configs. + * 3. Lines starting with --SET are used to specify the configs when running this testing file. You + * can set multiple configs in one --SET, using comma to separate them. Or you can use multiple + * --SET statements. * 4. Lines starting with --IMPORT are used to load queries from another test file. + * 5. Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing file. + * The dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 + * belongs to dimension 1. One dimension can have multiple lines, each line representing one + * config set (one or more configs, separated by comma). Spark will run this testing file many + * times, each time picks one config set from each dimension, until all the combinations are + * tried. For example, if dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file + * will be run 6 times (cartesian product). * * For example: * {{{ @@ -266,7 +275,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { }) if (regenerateGoldenFiles) { - runQueries(queries, testCase, Some(settings)) + runQueries(queries, testCase, settings) } else { // A config dimension has multiple config sets, and a config set has multiple configs. // - config dim: Seq[Seq[(String, String)]] @@ -288,7 +297,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { configSets.foreach { configSet => try { - runQueries(queries, testCase, Some(settings ++ configSet)) + runQueries(queries, testCase, settings ++ configSet) } catch { case e: Throwable => val configs = configSet.map { @@ -304,7 +313,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { protected def runQueries( queries: Seq[String], testCase: TestCase, - configSet: Option[Seq[(String, String)]]): Unit = { + configSet: Seq[(String, String)]): Unit = { // Create a local SparkSession to have stronger isolation between different test cases. // This does not isolate catalog changes. val localSparkSession = spark.newSession() @@ -329,12 +338,13 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { case _ => } - if (configSet.isDefined) { + if (configSet.nonEmpty) { // Execute the list of set operation in order to add the desired configs - val setOperations = configSet.get.map { case (key, value) => s"set $key=$value" } + val setOperations = configSet.map { case (key, value) => s"set $key=$value" } logInfo(s"Setting configs: ${setOperations.mkString(", ")}") setOperations.foreach(localSparkSession.sql) } + // Run the SQL queries preparing them for comparison. val outputs: Seq[QueryOutput] = queries.map { sql => val (schema, output) = handleExceptions(getNormalizedResult(localSparkSession, sql)) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 7c48579fd2ca8..b564bb2d24005 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -106,16 +106,14 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { override def runQueries( queries: Seq[String], testCase: TestCase, - configSet: Option[Seq[(String, String)]]): Unit = { + configSet: Seq[(String, String)]): Unit = { // We do not test with configSet. withJdbcStatement { statement => loadTestData(statement) - configSet.foreach { configs => - for ((k, v) <- configs) { - statement.execute(s"SET $k = $v") - } + configSet.foreach { case (k, v) => + statement.execute(s"SET $k = $v") } testCase match { From 2d5de25a999e0e5580cf4024937b61e6c9265672 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 25 Nov 2019 09:36:39 -0600 Subject: [PATCH 0716/1387] [SPARK-29415][CORE] Stage Level Sched: Add base ResourceProfile and Request classes ### What changes were proposed in this pull request? This PR is adding the base classes needed for Stage level scheduling. Its adding a ResourceProfile and the executor and task resource request classes. These are made private for now until we get all the parts implemented, at which point this will become public interfaces. I am adding them first as all the other subtasks for this feature require these classes. If people have better ideas on breaking this feature up please let me know. See https://issues.apache.org/jira/browse/SPARK-29415 for more detailed design. ### Why are the changes needed? New API for stage level scheduling. Its easier to add these first because the other jira for this features will all use them. ### Does this PR introduce any user-facing change? Yes adds API to create a ResourceProfile with executor/task resources, see the spip jira https://issues.apache.org/jira/browse/SPARK-27495 Example of the api: val rp = new ResourceProfile() rp.require(new ExecutorResourceRequest("cores", 2)) rp.require(new ExecutorResourceRequest("gpu", 1, Some("/opt/gpuScripts/getGpus"))) rp.require(new TaskResourceRequest("gpu", 1)) ### How was this patch tested? Tested using Unit tests added with this PR. Closes #26284 from tgravescs/SPARK-29415. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../resource/ExecutorResourceRequest.scala | 77 ++++++++ .../resource/ExecutorResourceRequests.scala | 122 +++++++++++++ .../spark/resource/ResourceProfile.scala | 147 ++++++++++++++++ .../apache/spark/resource/ResourceUtils.scala | 7 +- .../spark/resource/TaskResourceRequest.scala | 43 +++++ .../spark/resource/TaskResourceRequests.scala | 75 ++++++++ .../resource/JavaResourceProfileSuite.java | 66 +++++++ .../spark/resource/ResourceProfileSuite.scala | 166 ++++++++++++++++++ 8 files changed, 701 insertions(+), 2 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala create mode 100644 core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala create mode 100644 core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala create mode 100644 core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala create mode 100644 core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala create mode 100644 core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java create mode 100644 core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala new file mode 100644 index 0000000000000..88ceaad69b672 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import scala.collection.mutable + +import org.apache.spark.resource.ResourceUtils.RESOURCE_DOT + +/** + * An Executor resource request. This is used in conjunction with the ResourceProfile to + * programmatically specify the resources needed for an RDD that will be applied at the + * stage level. + * + * This is used to specify what the resource requirements are for an Executor and how + * Spark can find out specific details about those resources. Not all the parameters are + * required for every resource type. The resources names supported + * correspond to the regular Spark configs with the prefix removed. For instance overhead + * memory in this api is memoryOverhead, which is spark.executor.memoryOverhead with + * spark.executor removed. Resources like GPUs are resource.gpu + * (spark configs spark.executor.resource.gpu.*). The amount, discoveryScript, and vendor + * parameters for resources are all the same parameters a user would specify through the + * configs: spark.executor.resource.{resourceName}.{amount, discoveryScript, vendor}. + * + * For instance, a user wants to allocate an Executor with GPU resources on YARN. The user has + * to specify the resource name (resource.gpu), the amount or number of GPUs per Executor, + * the discovery script would be specified so that when the Executor starts up it can + * discovery what GPU addresses are available for it to use because YARN doesn't tell + * Spark that, then vendor would not be used because its specific for Kubernetes. + * + * See the configuration and cluster specific docs for more details. + * + * Use ExecutorResourceRequests class as a convenience API. + * + * @param resourceName Name of the resource + * @param amount Amount requesting + * @param discoveryScript Optional script used to discover the resources. This is required on some + * cluster managers that don't tell Spark the addresses of the resources + * allocated. The script runs on Executors startup to discover the addresses + * of the resources available. + * @param vendor Optional vendor, required for some cluster managers + * + * This api is currently private until the rest of the pieces are in place and then it + * will become public. + */ +private[spark] class ExecutorResourceRequest( + val resourceName: String, + val amount: Long, + val discoveryScript: String = "", + val vendor: String = "") extends Serializable { + + // A list of allowed Spark internal resources. Custom resources (spark.executor.resource.*) + // like GPUs/FPGAs are also allowed, see the check below. + private val allowedExecutorResources = mutable.HashSet[String]( + ResourceProfile.MEMORY, + ResourceProfile.OVERHEAD_MEM, + ResourceProfile.PYSPARK_MEM, + ResourceProfile.CORES) + + if (!allowedExecutorResources.contains(resourceName) && !resourceName.startsWith(RESOURCE_DOT)) { + throw new IllegalArgumentException(s"Executor resource not allowed: $resourceName") + } +} diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala new file mode 100644 index 0000000000000..6ffcc0c296468 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import scala.collection.mutable + +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.resource.ResourceProfile._ + +/** + * A set of Executor resource requests. This is used in conjunction with the ResourceProfile to + * programmatically specify the resources needed for an RDD that will be applied at the + * stage level. + * + * This api is currently private until the rest of the pieces are in place and then it + * will become public. + */ +private[spark] class ExecutorResourceRequests() extends Serializable { + + private val _executorResources = new mutable.HashMap[String, ExecutorResourceRequest]() + + def requests: Map[String, ExecutorResourceRequest] = _executorResources.toMap + + /** + * Specify heap memory. The value specified will be converted to MiB. + * + * @param amount Amount of memory. In the same format as JVM memory strings (e.g. 512m, 2g). + * Default unit is MiB if not specified. + */ + def memory(amount: String): this.type = { + val amountMiB = JavaUtils.byteStringAsMb(amount) + val rr = new ExecutorResourceRequest(MEMORY, amountMiB) + _executorResources(MEMORY) = rr + this + } + + /** + * Specify overhead memory. The value specified will be converted to MiB. + * + * @param amount Amount of memory. In the same format as JVM memory strings (e.g. 512m, 2g). + * Default unit is MiB if not specified. + */ + def memoryOverhead(amount: String): this.type = { + val amountMiB = JavaUtils.byteStringAsMb(amount) + val rr = new ExecutorResourceRequest(OVERHEAD_MEM, amountMiB) + _executorResources(OVERHEAD_MEM) = rr + this + } + + /** + * Specify pyspark memory. The value specified will be converted to MiB. + * + * @param amount Amount of memory. In the same format as JVM memory strings (e.g. 512m, 2g). + * Default unit is MiB if not specified. + */ + def pysparkMemory(amount: String): this.type = { + val amountMiB = JavaUtils.byteStringAsMb(amount) + val rr = new ExecutorResourceRequest(PYSPARK_MEM, amountMiB) + _executorResources(PYSPARK_MEM) = rr + this + } + + /** + * Specify number of cores per Executor. + * + * @param amount Number of cores to allocate per Executor. + */ + def cores(amount: Int): this.type = { + val t = new ExecutorResourceRequest(CORES, amount) + _executorResources(CORES) = t + this + } + + /** + * Amount of a particular custom resource(GPU, FPGA, etc) to use. The resource names supported + * correspond to the regular Spark configs with the prefix removed. For instance, resources + * like GPUs are resource.gpu (spark configs spark.executor.resource.gpu.*) + * + * @param resourceName Name of the resource. + * @param amount amount of that resource per executor to use. + * @param discoveryScript Optional script used to discover the resources. This is required on + * some cluster managers that don't tell Spark the addresses of + * the resources allocated. The script runs on Executors startup to + * of the resources available. + * @param vendor Optional vendor, required for some cluster managers + */ + def resource( + resourceName: String, + amount: Long, + discoveryScript: String = "", + vendor: String = ""): this.type = { + // a bit weird but for Java api use empty string as meaning None because empty + // string is otherwise invalid for those paramters anyway + val eReq = new ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor) + _executorResources(resourceName) = eReq + this + } + + def addRequest(ereq: ExecutorResourceRequest): this.type = { + _executorResources(ereq.resourceName) = ereq + this + } + + override def toString: String = { + s"Executor resource requests: ${_executorResources}" + } +} diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala new file mode 100644 index 0000000000000..876a655b13ad6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import java.util.{Map => JMap} +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.SparkConf +import org.apache.spark.annotation.Evolving +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceUtils.RESOURCE_PREFIX + +/** + * Resource profile to associate with an RDD. A ResourceProfile allows the user to + * specify executor and task requirements for an RDD that will get applied during a + * stage. This allows the user to change the resource requirements between stages. + * + * This class is private now for initial development, once we have the feature in place + * this will become public. + */ +@Evolving +private[spark] class ResourceProfile() extends Serializable { + + private val _id = ResourceProfile.getNextProfileId + private val _taskResources = new mutable.HashMap[String, TaskResourceRequest]() + private val _executorResources = new mutable.HashMap[String, ExecutorResourceRequest]() + + def id: Int = _id + def taskResources: Map[String, TaskResourceRequest] = _taskResources.toMap + def executorResources: Map[String, ExecutorResourceRequest] = _executorResources.toMap + + /** + * (Java-specific) gets a Java Map of resources to TaskResourceRequest + */ + def taskResourcesJMap: JMap[String, TaskResourceRequest] = _taskResources.asJava + + /** + * (Java-specific) gets a Java Map of resources to ExecutorResourceRequest + */ + def executorResourcesJMap: JMap[String, ExecutorResourceRequest] = _executorResources.asJava + + def reset(): Unit = { + _taskResources.clear() + _executorResources.clear() + } + + def require(requests: ExecutorResourceRequests): this.type = { + _executorResources ++= requests.requests + this + } + + def require(requests: TaskResourceRequests): this.type = { + _taskResources ++= requests.requests + this + } + + override def toString(): String = { + s"Profile: id = ${_id}, executor resources: ${_executorResources}, " + + s"task resources: ${_taskResources}" + } +} + +private[spark] object ResourceProfile extends Logging { + val UNKNOWN_RESOURCE_PROFILE_ID = -1 + val DEFAULT_RESOURCE_PROFILE_ID = 0 + + val CPUS = "cpus" + val CORES = "cores" + val MEMORY = "memory" + val OVERHEAD_MEM = "memoryOverhead" + val PYSPARK_MEM = "pyspark.memory" + + private lazy val nextProfileId = new AtomicInteger(0) + + // The default resource profile uses the application level configs. + // Create the default profile immediately to get ID 0, its initialized later when fetched. + private val defaultProfileRef: AtomicReference[ResourceProfile] = + new AtomicReference[ResourceProfile](new ResourceProfile()) + + assert(defaultProfileRef.get().id == DEFAULT_RESOURCE_PROFILE_ID, + s"Default Profile must have the default profile id: $DEFAULT_RESOURCE_PROFILE_ID") + + def getNextProfileId: Int = nextProfileId.getAndIncrement() + + def getOrCreateDefaultProfile(conf: SparkConf): ResourceProfile = { + val defaultProf = defaultProfileRef.get() + // check to see if the default profile was initialized yet + if (defaultProf.executorResources == Map.empty) { + synchronized { + val prof = defaultProfileRef.get() + if (prof.executorResources == Map.empty) { + addDefaultTaskResources(prof, conf) + addDefaultExecutorResources(prof, conf) + } + prof + } + } else { + defaultProf + } + } + + private def addDefaultTaskResources(rprof: ResourceProfile, conf: SparkConf): Unit = { + val cpusPerTask = conf.get(CPUS_PER_TASK) + val treqs = new TaskResourceRequests().cpus(cpusPerTask) + val taskReq = ResourceUtils.parseResourceRequirements(conf, SPARK_TASK_PREFIX) + taskReq.foreach { req => + val name = s"${RESOURCE_PREFIX}.${req.resourceName}" + treqs.resource(name, req.amount) + } + rprof.require(treqs) + } + + private def addDefaultExecutorResources(rprof: ResourceProfile, conf: SparkConf): Unit = { + val ereqs = new ExecutorResourceRequests() + ereqs.cores(conf.get(EXECUTOR_CORES)) + ereqs.memory(conf.get(EXECUTOR_MEMORY).toString) + val execReq = ResourceUtils.parseAllResourceRequests(conf, SPARK_EXECUTOR_PREFIX) + execReq.foreach { req => + val name = s"${RESOURCE_PREFIX}.${req.id.resourceName}" + ereqs.resource(name, req.amount, req.discoveryScript.getOrElse(""), + req.vendor.getOrElse("")) + } + rprof.require(ereqs) + } + + // for testing purposes + def resetDefaultProfile(conf: SparkConf): Unit = getOrCreateDefaultProfile(conf).reset() +} diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index e5ae7a90f568c..ce4fd05cbef16 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -36,7 +36,7 @@ import org.apache.spark.util.Utils.executeAndGetOutput * @param resourceName gpu, fpga, etc */ private[spark] case class ResourceID(componentName: String, resourceName: String) { - def confPrefix: String = s"$componentName.resource.$resourceName." // with ending dot + def confPrefix: String = s"$componentName.${ResourceUtils.RESOURCE_PREFIX}.$resourceName." def amountConf: String = s"$confPrefix${ResourceUtils.AMOUNT}" def discoveryScriptConf: String = s"$confPrefix${ResourceUtils.DISCOVERY_SCRIPT}" def vendorConf: String = s"$confPrefix${ResourceUtils.VENDOR}" @@ -111,7 +111,7 @@ private[spark] object ResourceUtils extends Logging { } def listResourceIds(sparkConf: SparkConf, componentName: String): Seq[ResourceID] = { - sparkConf.getAllWithPrefix(s"$componentName.resource.").map { case (key, _) => + sparkConf.getAllWithPrefix(s"$componentName.$RESOURCE_DOT").map { case (key, _) => key.substring(0, key.indexOf('.')) }.toSet.toSeq.map(name => ResourceID(componentName, name)) } @@ -258,4 +258,7 @@ private[spark] object ResourceUtils extends Logging { // known types of resources final val GPU: String = "gpu" final val FPGA: String = "fpga" + + final val RESOURCE_PREFIX: String = "resource" + final val RESOURCE_DOT: String = s"$RESOURCE_PREFIX." } diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala new file mode 100644 index 0000000000000..22eda52c423a1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala @@ -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. + */ + +package org.apache.spark.resource + +import scala.collection.mutable + +import org.apache.spark.resource.ResourceUtils.RESOURCE_DOT + +/** + * A task resource request. This is used in conjuntion with the ResourceProfile to + * programmatically specify the resources needed for an RDD that will be applied at the + * stage level. + * + * Use TaskResourceRequests class as a convenience API. + * + * This api is currently private until the rest of the pieces are in place and then it + * will become public. + */ +private[spark] class TaskResourceRequest(val resourceName: String, val amount: Double) + extends Serializable { + + assert(amount <= 0.5 || amount % 1 == 0, + s"The resource amount ${amount} must be either <= 0.5, or a whole number.") + + if (!resourceName.equals(ResourceProfile.CPUS) && !resourceName.startsWith(RESOURCE_DOT)) { + throw new IllegalArgumentException(s"Task resource not allowed: $resourceName") + } +} diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala new file mode 100644 index 0000000000000..21cbc5d805ac0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import scala.collection.mutable + +import org.apache.spark.resource.ResourceProfile._ +import org.apache.spark.resource.ResourceUtils._ + +/** + * A set of task resource requests. This is used in conjuntion with the ResourceProfile to + * programmatically specify the resources needed for an RDD that will be applied at the + * stage level. + * + * This api is currently private until the rest of the pieces are in place and then it + * will become public. + */ +private[spark] class TaskResourceRequests() extends Serializable { + + private val _taskResources = new mutable.HashMap[String, TaskResourceRequest]() + + def requests: Map[String, TaskResourceRequest] = _taskResources.toMap + + /** + * Specify number of cpus per Task. + * + * @param amount Number of cpus to allocate per Task. + */ + def cpus(amount: Int): this.type = { + val t = new TaskResourceRequest(CPUS, amount) + _taskResources(CPUS) = t + this + } + + /** + * Amount of a particular custom resource(GPU, FPGA, etc) to use. The resource names supported + * correspond to the regular Spark configs with the prefix removed. For instance, resources + * like GPUs are resource.gpu (spark configs spark.task.resource.gpu.*) + * + * @param resourceName Name of the resource. + * @param amount Amount requesting as a Double to support fractional resource requests. + * Valid values are less than or equal to 0.5 or whole numbers. This essentially + * lets you configure X number of tasks to run on a single resource, + * ie amount equals 0.5 translates into 2 tasks per resource address. + */ + def resource(rName: String, amount: Double): this.type = { + val t = new TaskResourceRequest(rName, amount) + _taskResources(rName) = t + this + } + + def addRequest(treq: TaskResourceRequest): this.type = { + _taskResources(treq.resourceName) = treq + this + } + + override def toString: String = { + s"Task resource requests: ${_taskResources}" + } +} diff --git a/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java b/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java new file mode 100644 index 0000000000000..077120724c291 --- /dev/null +++ b/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java @@ -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.resource; + +import java.util.Map; + +import static org.junit.Assert.*; +import org.junit.Test; + +// Test the ResourceProfile and Request api's from Java +public class JavaResourceProfileSuite { + + String GpuResource = "resource.gpu"; + String FPGAResource = "resource.fpga"; + + @Test + public void testResourceProfileAccessFromJava() throws Exception { + ExecutorResourceRequests execReqGpu = + new ExecutorResourceRequests().resource(GpuResource, 2,"myscript", ""); + ExecutorResourceRequests execReqFpga = + new ExecutorResourceRequests().resource(FPGAResource, 3, "myfpgascript", "nvidia"); + + ResourceProfile rprof = new ResourceProfile(); + rprof.require(execReqGpu); + rprof.require(execReqFpga); + TaskResourceRequests taskReq1 = new TaskResourceRequests().resource(GpuResource, 1); + rprof.require(taskReq1); + + assertEquals(rprof.executorResources().size(), 2); + Map eresources = rprof.executorResourcesJMap(); + assert(eresources.containsKey(GpuResource)); + ExecutorResourceRequest gpuReq = eresources.get(GpuResource); + assertEquals(gpuReq.amount(), 2); + assertEquals(gpuReq.discoveryScript(), "myscript"); + assertEquals(gpuReq.vendor(), ""); + + assert(eresources.containsKey(FPGAResource)); + ExecutorResourceRequest fpgaReq = eresources.get(FPGAResource); + assertEquals(fpgaReq.amount(), 3); + assertEquals(fpgaReq.discoveryScript(), "myfpgascript"); + assertEquals(fpgaReq.vendor(), "nvidia"); + + assertEquals(rprof.taskResources().size(), 1); + Map tresources = rprof.taskResourcesJMap(); + assert(tresources.containsKey(GpuResource)); + TaskResourceRequest taskReq = tresources.get(GpuResource); + assertEquals(taskReq.amount(), 1.0, 0); + assertEquals(taskReq.resourceName(), GpuResource); + } +} + diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala new file mode 100644 index 0000000000000..a087f18b3fc6d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ + +class ResourceProfileSuite extends SparkFunSuite { + + override def afterEach() { + try { + ResourceProfile.resetDefaultProfile(new SparkConf) + } finally { + super.afterEach() + } + } + + test("Default ResourceProfile") { + val rprof = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) + assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + assert(rprof.executorResources.size === 2, + "Executor resources should contain cores and memory by default") + assert(rprof.executorResources(ResourceProfile.CORES).amount === 1, + s"Executor resources should have 1 core") + assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 1024, + s"Executor resources should have 1024 memory") + assert(rprof.taskResources.size === 1, + "Task resources should just contain cpus by default") + assert(rprof.taskResources(ResourceProfile.CPUS).amount === 1, + s"Task resources should have 1 cpu") + } + + test("Default ResourceProfile with app level resources specified") { + val conf = new SparkConf + conf.set("spark.task.resource.gpu.amount", "1") + conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.amount", "1") + conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.discoveryScript", "nameOfScript") + val rprof = ResourceProfile.getOrCreateDefaultProfile(conf) + assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val execResources = rprof.executorResources + assert(execResources.size === 3, + "Executor resources should contain cores, memory, and gpu " + execResources) + assert(rprof.taskResources.size === 2, + "Task resources should just contain cpus and gpu") + assert(execResources.contains("resource.gpu"), "Executor resources should have gpu") + assert(rprof.taskResources.contains("resource.gpu"), "Task resources should have gpu") + } + + test("Create ResourceProfile") { + val rprof = new ResourceProfile() + assert(rprof.id > ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + assert(rprof.executorResources === Map.empty) + assert(rprof.taskResources === Map.empty) + + val taskReq = new TaskResourceRequests().resource("resource.gpu", 1) + val eReq = new ExecutorResourceRequests().resource("resource.gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(eReq) + + assert(rprof.executorResources.size === 1) + assert(rprof.executorResources.contains("resource.gpu"), + "Executor resources should have gpu") + assert(rprof.executorResources.get("resource.gpu").get.vendor === "nvidia", + "gpu vendor should be nvidia") + assert(rprof.executorResources.get("resource.gpu").get.discoveryScript === "myscript", + "discoveryScript should be myscript") + assert(rprof.executorResources.get("resource.gpu").get.amount === 2, + "gpu amount should be 2") + + assert(rprof.taskResources.size === 1, "Should have 1 task resource") + assert(rprof.taskResources.contains("resource.gpu"), "Task resources should have gpu") + assert(rprof.taskResources.get("resource.gpu").get.amount === 1, + "Task resources should have 1 gpu") + + val ereqs = new ExecutorResourceRequests() + ereqs.cores(2).memory("4096") + ereqs.memoryOverhead("2048").pysparkMemory("1024") + val treqs = new TaskResourceRequests() + treqs.cpus(1) + + rprof.require(treqs) + rprof.require(ereqs) + + assert(rprof.executorResources.size === 5) + assert(rprof.executorResources(ResourceProfile.CORES).amount === 2, + s"Executor resources should have 2 cores") + assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, + s"Executor resources should have 4096 memory") + assert(rprof.executorResources(ResourceProfile.OVERHEAD_MEM).amount === 2048, + s"Executor resources should have 2048 overhead memory") + assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount === 1024, + s"Executor resources should have 1024 pyspark memory") + + assert(rprof.taskResources.size === 2) + assert(rprof.taskResources("cpus").amount === 1, "Task resources should have cpu") + + val error = intercept[IllegalArgumentException] { + rprof.require(new ExecutorResourceRequests().resource("bogusResource", 1)) + }.getMessage() + assert(error.contains("Executor resource not allowed")) + + val taskError = intercept[IllegalArgumentException] { + rprof.require(new TaskResourceRequests().resource("bogusTaskResource", 1)) + }.getMessage() + assert(taskError.contains("Task resource not allowed")) + } + + test("Test ExecutorResourceRequests memory helpers") { + val rprof = new ResourceProfile() + val ereqs = new ExecutorResourceRequests() + ereqs.memory("4g") + ereqs.memoryOverhead("2000m").pysparkMemory("512000k") + rprof.require(ereqs) + + assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, + s"Executor resources should have 4096 memory") + assert(rprof.executorResources(ResourceProfile.OVERHEAD_MEM).amount === 2000, + s"Executor resources should have 2000 overhead memory") + assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount === 500, + s"Executor resources should have 512 pyspark memory") + } + + test("Test TaskResourceRequest fractional") { + val rprof = new ResourceProfile() + val treqs = new TaskResourceRequests().resource("resource.gpu", 0.33) + rprof.require(treqs) + + assert(rprof.taskResources.size === 1, "Should have 1 task resource") + assert(rprof.taskResources.contains("resource.gpu"), "Task resources should have gpu") + assert(rprof.taskResources.get("resource.gpu").get.amount === 0.33, + "Task resources should have 0.33 gpu") + + val fpgaReqs = new TaskResourceRequests().resource("resource.fpga", 4.0) + rprof.require(fpgaReqs) + + assert(rprof.taskResources.size === 2, "Should have 2 task resource") + assert(rprof.taskResources.contains("resource.fpga"), "Task resources should have gpu") + assert(rprof.taskResources.get("resource.fpga").get.amount === 4.0, + "Task resources should have 4.0 gpu") + + var taskError = intercept[AssertionError] { + rprof.require(new TaskResourceRequests().resource("resource.gpu", 1.5)) + }.getMessage() + assert(taskError.contains("The resource amount 1.5 must be either <= 0.5, or a whole number.")) + + taskError = intercept[AssertionError] { + rprof.require(new TaskResourceRequests().resource("resource.gpu", 0.7)) + }.getMessage() + assert(taskError.contains("The resource amount 0.7 must be either <= 0.5, or a whole number.")) + } +} + From f09c1a36c4b0ca1fb450e274b22294dca590d8f8 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 26 Nov 2019 00:06:19 +0800 Subject: [PATCH 0717/1387] [SPARK-29890][SQL] DataFrameNaFunctions.fill should handle duplicate columns ### What changes were proposed in this pull request? `DataFrameNaFunctions.fill` doesn't handle duplicate columns even when column names are not specified. ```Scala val left = Seq(("1", null), ("3", "4")).toDF("col1", "col2") val right = Seq(("1", "2"), ("3", null)).toDF("col1", "col2") val df = left.join(right, Seq("col1")) df.printSchema df.na.fill("hello").show ``` produces ``` root |-- col1: string (nullable = true) |-- col2: string (nullable = true) |-- col2: string (nullable = true) org.apache.spark.sql.AnalysisException: Reference 'col2' is ambiguous, could be: col2, col2.; at org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:259) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveQuoted(LogicalPlan.scala:121) at org.apache.spark.sql.Dataset.resolve(Dataset.scala:221) at org.apache.spark.sql.Dataset.col(Dataset.scala:1268) ``` The reason for the above failure is that columns are looked up with `DataSet.col()` which tries to resolve a column by name and if there are multiple columns with the same name, it will fail due to ambiguity. This PR updates `DataFrameNaFunctions.fill` such that if the columns to fill are not specified, it will resolve ambiguity gracefully by applying `fill` to all the eligible columns. (Note that if the user specifies the columns, it will still continue to fail due to ambiguity). ### Why are the changes needed? If column names are not specified, `fill` should not fail due to ambiguity since it should still be able to apply `fill` to the eligible columns. ### Does this PR introduce any user-facing change? Yes, now the above example displays the following: ``` +----+-----+-----+ |col1| col2| col2| +----+-----+-----+ | 1|hello| 2| | 3| 4|hello| +----+-----+-----+ ``` ### How was this patch tested? Added new unit tests. Closes #26593 from imback82/na_fill. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../spark/sql/DataFrameNaFunctions.scala | 64 +++++++++++++------ .../spark/sql/DataFrameNaFunctionsSuite.scala | 45 +++++++++++++ 2 files changed, 88 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 6dd21f114c902..07b0a54ba0772 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -130,20 +130,20 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 2.2.0 */ - def fill(value: Long): DataFrame = fill(value, df.columns) + def fill(value: Long): DataFrame = fillValue(value, outputAttributes) /** * Returns a new `DataFrame` that replaces null or NaN values in numeric columns with `value`. * @since 1.3.1 */ - def fill(value: Double): DataFrame = fill(value, df.columns) + def fill(value: Double): DataFrame = fillValue(value, outputAttributes) /** * Returns a new `DataFrame` that replaces null values in string columns with `value`. * * @since 1.3.1 */ - def fill(value: String): DataFrame = fill(value, df.columns) + def fill(value: String): DataFrame = fillValue(value, outputAttributes) /** * Returns a new `DataFrame` that replaces null or NaN values in specified numeric columns. @@ -167,7 +167,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 2.2.0 */ - def fill(value: Long, cols: Seq[String]): DataFrame = fillValue(value, cols) + def fill(value: Long, cols: Seq[String]): DataFrame = fillValue(value, toAttributes(cols)) /** * (Scala-specific) Returns a new `DataFrame` that replaces null or NaN values in specified @@ -175,7 +175,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def fill(value: Double, cols: Seq[String]): DataFrame = fillValue(value, cols) + def fill(value: Double, cols: Seq[String]): DataFrame = fillValue(value, toAttributes(cols)) /** @@ -192,14 +192,14 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def fill(value: String, cols: Seq[String]): DataFrame = fillValue(value, cols) + def fill(value: String, cols: Seq[String]): DataFrame = fillValue(value, toAttributes(cols)) /** * Returns a new `DataFrame` that replaces null values in boolean columns with `value`. * * @since 2.3.0 */ - def fill(value: Boolean): DataFrame = fill(value, df.columns) + def fill(value: Boolean): DataFrame = fillValue(value, outputAttributes) /** * (Scala-specific) Returns a new `DataFrame` that replaces null values in specified @@ -207,7 +207,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 2.3.0 */ - def fill(value: Boolean, cols: Seq[String]): DataFrame = fillValue(value, cols) + def fill(value: Boolean, cols: Seq[String]): DataFrame = fillValue(value, toAttributes(cols)) /** * Returns a new `DataFrame` that replaces null values in specified boolean columns. @@ -433,15 +433,24 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { /** * Returns a [[Column]] expression that replaces null value in `col` with `replacement`. + * It selects a column based on its name. */ private def fillCol[T](col: StructField, replacement: T): Column = { val quotedColName = "`" + col.name + "`" - val colValue = col.dataType match { + fillCol(col.dataType, col.name, df.col(quotedColName), replacement) + } + + /** + * Returns a [[Column]] expression that replaces null value in `expr` with `replacement`. + * It uses the given `expr` as a column. + */ + private def fillCol[T](dataType: DataType, name: String, expr: Column, replacement: T): Column = { + val colValue = dataType match { case DoubleType | FloatType => - nanvl(df.col(quotedColName), lit(null)) // nanvl only supports these types - case _ => df.col(quotedColName) + nanvl(expr, lit(null)) // nanvl only supports these types + case _ => expr } - coalesce(colValue, lit(replacement).cast(col.dataType)).as(col.name) + coalesce(colValue, lit(replacement).cast(dataType)).as(name) } /** @@ -468,12 +477,22 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { s"Unsupported value type ${v.getClass.getName} ($v).") } + private def toAttributes(cols: Seq[String]): Seq[Attribute] = { + cols.map(name => df.col(name).expr).collect { + case a: Attribute => a + } + } + + private def outputAttributes: Seq[Attribute] = { + df.queryExecution.analyzed.output + } + /** - * Returns a new `DataFrame` that replaces null or NaN values in specified - * numeric, string columns. If a specified column is not a numeric, string - * or boolean column it is ignored. + * Returns a new `DataFrame` that replaces null or NaN values in the specified + * columns. If a specified column is not a numeric, string or boolean column, + * it is ignored. */ - private def fillValue[T](value: T, cols: Seq[String]): DataFrame = { + private def fillValue[T](value: T, cols: Seq[Attribute]): DataFrame = { // the fill[T] which T is Long/Double, // should apply on all the NumericType Column, for example: // val input = Seq[(java.lang.Integer, java.lang.Double)]((null, 164.3)).toDF("a","b") @@ -487,9 +506,8 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { s"Unsupported value type ${value.getClass.getName} ($value).") } - val columnEquals = df.sparkSession.sessionState.analyzer.resolver - val filledColumns = df.schema.fields.filter { f => - val typeMatches = (targetType, f.dataType) match { + val projections = outputAttributes.map { col => + val typeMatches = (targetType, col.dataType) match { case (NumericType, dt) => dt.isInstanceOf[NumericType] case (StringType, dt) => dt == StringType case (BooleanType, dt) => dt == BooleanType @@ -497,8 +515,12 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { throw new IllegalArgumentException(s"$targetType is not matched at fillValue") } // Only fill if the column is part of the cols list. - typeMatches && cols.exists(col => columnEquals(f.name, col)) + if (typeMatches && cols.exists(_.semanticEquals(col))) { + fillCol(col.dataType, col.name, Column(col), value) + } else { + Column(col) + } } - df.withColumns(filledColumns.map(_.name), filledColumns.map(fillCol[T](_, value))) + df.select(projections : _*) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index 75642a0bd9325..1afe733b855b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{StringType, StructType} class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { import testImplicits._ @@ -266,6 +267,33 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { assert(message.contains("Reference 'f2' is ambiguous")) } + test("fill with col(*)") { + val df = createDF() + // If columns are specified with "*", they are ignored. + checkAnswer(df.na.fill("new name", Seq("*")), df.collect()) + } + + test("fill with nested columns") { + val schema = new StructType() + .add("c1", new StructType() + .add("c1-1", StringType) + .add("c1-2", StringType)) + + val data = Seq( + Row(Row(null, "a2")), + Row(Row("b1", "b2")), + Row(null)) + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(data), schema) + + checkAnswer(df.select("c1.c1-1"), + Row(null) :: Row("b1") :: Row(null) :: Nil) + + // Nested columns are ignored for fill(). + checkAnswer(df.na.fill("a1", Seq("c1.c1-1")), data) + } + test("replace") { val input = createDF() @@ -340,4 +368,21 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { )).na.drop("name" :: Nil).select("name"), Row("Alice") :: Row("David") :: Nil) } + + test("SPARK-29890: duplicate names are allowed for fill() if column names are not specified.") { + val left = Seq(("1", null), ("3", "4")).toDF("col1", "col2") + val right = Seq(("1", "2"), ("3", null)).toDF("col1", "col2") + val df = left.join(right, Seq("col1")) + + // If column names are specified, the following fails due to ambiguity. + val exception = intercept[AnalysisException] { + df.na.fill("hello", Seq("col2")) + } + assert(exception.getMessage.contains("Reference 'col2' is ambiguous")) + + // If column names are not specified, fill() is applied to all the eligible columns. + checkAnswer( + df.na.fill("hello"), + Row("1", "hello", "2") :: Row("3", "4", "hello") :: Nil) + } } From 29ebd9336c23a0f7228c5123a8906fab3346b8c0 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Tue, 26 Nov 2019 01:24:43 +0800 Subject: [PATCH 0718/1387] [SPARK-29979][SQL] Add basic/reserved property key constants in TableCatalog and SupportsNamespaces ### What changes were proposed in this pull request? Add "comment" and "location" property key constants in TableCatalog and SupportNamespaces. And update code of implementation classes to use these constants instead of hard code. ### Why are the changes needed? Currently, some basic/reserved keys (eg. "location", "comment") of table and namespace properties are hard coded or defined in specific logical plan implementation class. These keys can be centralized in TableCatalog and SupportsNamespaces interface and shared across different implementation classes. ### Does this PR introduce any user-facing change? no ### How was this patch tested? Existing unit test Closes #26617 from fuwhu/SPARK-29979. Authored-by: fuwhu Signed-off-by: Wenchen Fan --- .../connector/catalog/SupportsNamespaces.java | 19 ++++++++++++++++++ .../sql/connector/catalog/TableCatalog.java | 12 +++++++++++ .../catalyst/analysis/ResolveCatalogs.scala | 8 ++++---- .../sql/catalyst/parser/AstBuilder.scala | 5 +++-- .../catalyst/plans/logical/statements.scala | 5 ----- .../sql/connector/catalog/CatalogV2Util.scala | 13 ++++++------ .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../analysis/ResolveSessionCatalog.scala | 14 ++++++------- .../v2/DescribeNamespaceExec.scala | 11 +++++----- .../datasources/v2/V2SessionCatalog.scala | 20 +++++++++---------- .../sql/connector/DataSourceV2SQLSuite.scala | 3 +-- .../v2/V2SessionCatalogSuite.scala | 6 +++--- 12 files changed, 71 insertions(+), 47 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java index 40d9c2b122817..171777777dbac 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java @@ -21,6 +21,8 @@ import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import java.util.Arrays; +import java.util.List; import java.util.Map; /** @@ -39,6 +41,23 @@ @Experimental public interface SupportsNamespaces extends CatalogPlugin { + /** + * A property to specify the location of the namespace. If the namespace + * needs to store files, it should be under this location. + */ + String PROP_LOCATION = "location"; + + /** + * A property to specify the description of the namespace. The description + * will be returned in the result of "DESCRIBE NAMESPACE" command. + */ + String PROP_COMMENT = "comment"; + + /** + * The list of reserved namespace properties. + */ + List RESERVED_PROPERTIES = Arrays.asList(PROP_COMMENT, PROP_LOCATION); + /** * Return a default namespace for the catalog. *

    diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index b72f7845a6ea5..a349480359e0e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -37,6 +37,18 @@ */ @Experimental public interface TableCatalog extends CatalogPlugin { + + /** + * A property to specify the location of the table. The files of the table + * should be under this location. + */ + String PROP_LOCATION = "location"; + + /** + * A property to specify the description of the table. + */ + String PROP_COMMENT = "comment"; + /** * List the tables in a namespace from the catalog. *

    diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 2f2e4e619eb4a..896b2830d524e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange} /** * Resolves catalogs from the multi-part identifiers in SQL statements, and convert the statements @@ -78,7 +78,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) throw new AnalysisException( "ALTER TABLE SET LOCATION does not support partition for v2 tables.") } - val changes = Seq(TableChange.setProperty("location", newLoc)) + val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, newLoc)) createAlterTable(nameParts, catalog, tableName, changes) case AlterViewSetPropertiesStatement( @@ -97,8 +97,8 @@ class ResolveCatalogs(val catalogManager: CatalogManager) AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts, properties) case AlterNamespaceSetLocationStatement(NonSessionCatalog(catalog, nameParts), location) => - AlterNamespaceSetProperties( - catalog.asNamespaceCatalog, nameParts, Map("location" -> location)) + AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts, + Map(SupportsNamespaces.PROP_LOCATION -> location)) case RenameTableStatement(NonSessionCatalog(catalog, oldName), newNameParts, isView) => if (isView) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 37ff4ae981c33..c450e47e41b5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit +import org.apache.spark.sql.connector.catalog.SupportsNamespaces import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -2492,10 +2493,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging .map(visitPropertyKeyValues) .getOrElse(Map.empty) Option(ctx.comment).map(string).map { - properties += CreateNamespaceStatement.COMMENT_PROPERTY_KEY -> _ + properties += SupportsNamespaces.PROP_COMMENT -> _ } ctx.locationSpec.asScala.headOption.map(visitLocationSpec).map { - properties += CreateNamespaceStatement.LOCATION_PROPERTY_KEY -> _ + properties += SupportsNamespaces.PROP_LOCATION -> _ } CreateNamespaceStatement( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 7d7d6bdbfdd2d..903f218e9f0af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -345,11 +345,6 @@ case class CreateNamespaceStatement( ifNotExists: Boolean, properties: Map[String, String]) extends ParsedStatement -object CreateNamespaceStatement { - val COMMENT_PROPERTY_KEY: String = "comment" - val LOCATION_PROPERTY_KEY: String = "location" -} - /** * A DROP NAMESPACE statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 0f313e7b96d86..fa45d72062423 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -245,11 +245,11 @@ private[sql] object CatalogV2Util { "you can only specify one of them.") } - if ((options.contains("comment") || properties.contains("comment")) - && comment.isDefined) { + if ((options.contains(TableCatalog.PROP_COMMENT) + || properties.contains(TableCatalog.PROP_COMMENT)) && comment.isDefined) { throw new AnalysisException( - "COMMENT and option/property 'comment' are both used to set the table comment, you can " + - "only specify one of them.") + s"COMMENT and option/property '${TableCatalog.PROP_COMMENT}' " + + s"are both used to set the table comment, you can only specify one of them.") } if (options.contains("provider") || properties.contains("provider")) { @@ -267,8 +267,9 @@ private[sql] object CatalogV2Util { // convert USING, LOCATION, and COMMENT clauses to table properties tableProperties += ("provider" -> provider) - comment.map(text => tableProperties += ("comment" -> text)) - location.orElse(options.get("path")).map(loc => tableProperties += ("location" -> loc)) + comment.map(text => tableProperties += (TableCatalog.PROP_COMMENT -> text)) + location.orElse(options.get("path")).map( + loc => tableProperties += (TableCatalog.PROP_LOCATION -> loc)) tableProperties.toMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 3561c41f2b223..42f145d5673d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -519,7 +519,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { def getLocationIfExists: Option[(String, String)] = { val opts = CaseInsensitiveMap(extraOptions.toMap) - opts.get("path").map("location" -> _) + opts.get("path").map(TableCatalog.PROP_LOCATION -> _) } val command = (mode, tableOpt) match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index eb53e3accc3d5..d95ddf968b30b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.catalyst.analysis +import scala.collection.JavaConverters._ + import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, SupportsNamespaces, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} @@ -147,7 +149,7 @@ class ResolveSessionCatalog( throw new AnalysisException( "ALTER TABLE SET LOCATION does not support partition for v2 tables.") } - val changes = Seq(TableChange.setProperty("location", newLoc)) + val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, newLoc)) createAlterTable(nameParts, catalog, tableName, changes) } @@ -301,11 +303,9 @@ class ResolveSessionCatalog( s"The database name is not valid: ${nameParts.quoted}") } - val comment = c.properties.get(CreateNamespaceStatement.COMMENT_PROPERTY_KEY) - val location = c.properties.get(CreateNamespaceStatement.LOCATION_PROPERTY_KEY) - val newProperties = c.properties - - CreateNamespaceStatement.COMMENT_PROPERTY_KEY - - CreateNamespaceStatement.LOCATION_PROPERTY_KEY + val comment = c.properties.get(SupportsNamespaces.PROP_COMMENT) + val location = c.properties.get(SupportsNamespaces.PROP_LOCATION) + val newProperties = c.properties -- SupportsNamespaces.RESERVED_PROPERTIES.asScala CreateDatabaseCommand(nameParts.head, c.ifNotExists, location, comment, newProperties) case d @ DropNamespaceStatement(SessionCatalog(_, nameParts), _, _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala index 7c5cfcbbc7e3c..7ad872f10e847 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala @@ -24,9 +24,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} import org.apache.spark.sql.connector.catalog.SupportsNamespaces -import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.COMMENT_TABLE_PROP -import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.LOCATION_TABLE_PROP -import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog.RESERVED_PROPERTIES import org.apache.spark.sql.types.StructType /** @@ -45,10 +42,12 @@ case class DescribeNamespaceExec( val metadata = catalog.loadNamespaceMetadata(ns) rows += toCatalystRow("Namespace Name", ns.last) - rows += toCatalystRow("Description", metadata.get(COMMENT_TABLE_PROP)) - rows += toCatalystRow("Location", metadata.get(LOCATION_TABLE_PROP)) + rows += toCatalystRow("Description", metadata.get(SupportsNamespaces.PROP_COMMENT)) + rows += toCatalystRow("Location", metadata.get(SupportsNamespaces.PROP_LOCATION)) if (isExtended) { - val properties = metadata.asScala.toSeq.filter(p => !RESERVED_PROPERTIES.contains(p._1)) + val properties = + metadata.asScala.toSeq.filter(p => + !SupportsNamespaces.RESERVED_PROPERTIES.contains(p._1)) if (properties.nonEmpty) { rows += toCatalystRow("Properties", properties.mkString("(", ",", ")")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index dffb9cb67b5c2..f452201a8d3ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -85,7 +85,7 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) val (partitionColumns, maybeBucketSpec) = V2SessionCatalog.convertTransforms(partitions) val provider = properties.getOrDefault("provider", conf.defaultDataSourceName) val tableProperties = properties.asScala - val location = Option(properties.get(LOCATION_TABLE_PROP)) + val location = Option(properties.get(TableCatalog.PROP_LOCATION)) val storage = DataSource.buildStorageFormatFromOptions(tableProperties.toMap) .copy(locationUri = location.map(CatalogUtils.stringToURI)) val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED @@ -100,7 +100,7 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) bucketSpec = maybeBucketSpec, properties = tableProperties.toMap, tracksPartitionsInCatalog = conf.manageFilesourcePartitions, - comment = Option(properties.get(COMMENT_TABLE_PROP))) + comment = Option(properties.get(TableCatalog.PROP_COMMENT))) try { catalog.createTable(tableDesc, ignoreIfExists = false) @@ -227,7 +227,8 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) case Array(db) => // validate that this catalog's reserved properties are not removed changes.foreach { - case remove: RemoveProperty if RESERVED_PROPERTIES.contains(remove.property) => + case remove: RemoveProperty + if SupportsNamespaces.RESERVED_PROPERTIES.contains(remove.property) => throw new UnsupportedOperationException( s"Cannot remove reserved property: ${remove.property}") case _ => @@ -262,9 +263,6 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) } private[sql] object V2SessionCatalog { - val COMMENT_TABLE_PROP: String = "comment" - val LOCATION_TABLE_PROP: String = "location" - val RESERVED_PROPERTIES: Set[String] = Set(COMMENT_TABLE_PROP, LOCATION_TABLE_PROP) /** * Convert v2 Transforms to v1 partition columns and an optional bucket spec. @@ -294,12 +292,12 @@ private[sql] object V2SessionCatalog { defaultLocation: Option[URI] = None): CatalogDatabase = { CatalogDatabase( name = db, - description = metadata.getOrDefault(COMMENT_TABLE_PROP, ""), - locationUri = Option(metadata.get(LOCATION_TABLE_PROP)) + description = metadata.getOrDefault(SupportsNamespaces.PROP_COMMENT, ""), + locationUri = Option(metadata.get(SupportsNamespaces.PROP_LOCATION)) .map(CatalogUtils.stringToURI) .orElse(defaultLocation) .getOrElse(throw new IllegalArgumentException("Missing database location")), - properties = metadata.asScala.toMap -- Seq("comment", "location")) + properties = metadata.asScala.toMap -- SupportsNamespaces.RESERVED_PROPERTIES.asScala) } private implicit class CatalogDatabaseHelper(catalogDatabase: CatalogDatabase) { @@ -309,8 +307,8 @@ private[sql] object V2SessionCatalog { catalogDatabase.properties.foreach { case (key, value) => metadata.put(key, value) } - metadata.put(LOCATION_TABLE_PROP, catalogDatabase.locationUri.toString) - metadata.put(COMMENT_TABLE_PROP, catalogDatabase.description) + metadata.put(SupportsNamespaces.PROP_LOCATION, catalogDatabase.locationUri.toString) + metadata.put(SupportsNamespaces.PROP_COMMENT, catalogDatabase.description) metadata.asJava } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 2958b57bf06ea..018c0c22c607e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME -import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.sources.SimpleScanSource @@ -830,7 +829,7 @@ class DataSourceV2SQLSuite sql(s"CREATE NAMESPACE testcat.test LOCATION '$path'") val metadata = catalog("testcat").asNamespaceCatalog.loadNamespaceMetadata(Array("test")).asScala - val catalogPath = metadata(V2SessionCatalog.LOCATION_TABLE_PROP) + val catalogPath = metadata(SupportsNamespaces.PROP_LOCATION) assert(catalogPath.equals(catalogPath)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index 6bd6e0d89a677..a02998856f789 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -27,8 +27,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.connector.catalog.{Catalogs, Identifier, NamespaceChange, TableChange} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.connector.catalog.{Identifier, NamespaceChange, SupportsNamespaces, TableChange} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -757,7 +756,8 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { expected: scala.collection.Map[String, String], actual: scala.collection.Map[String, String]): Unit = { // remove location and comment that are automatically added by HMS unless they are expected - val toRemove = V2SessionCatalog.RESERVED_PROPERTIES.filter(expected.contains) + val toRemove = + SupportsNamespaces.RESERVED_PROPERTIES.asScala.filter(expected.contains) assert(expected -- toRemove === actual) } From bec2068ae8cc8ca78785a22d127573c6bfd2452f Mon Sep 17 00:00:00 2001 From: shahid Date: Mon, 25 Nov 2019 10:04:25 -0800 Subject: [PATCH 0719/1387] [SPARK-26260][CORE] For disk store tasks summary table should show only successful tasks summary MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …sks metrics for disk store ### What changes were proposed in this pull request? After https://github.com/apache/spark/pull/23088 task Summary table in the stage page shows successful tasks metrics for lnMemory store. In this PR, it added for disk store also. ### Why are the changes needed? Now both InMemory and disk store will be consistent in showing the task summary table in the UI, if there are non successful tasks ### Does this PR introduce any user-facing change? no ### How was this patch tested? Added UT. Manually verified Test steps: 1. add the config in spark-defaults.conf -> **spark.history.store.path /tmp/store** 2. sbin/start-hitoryserver 3. bin/spark-shell 4. `sc.parallelize(1 to 1000, 2).map(x => throw new Exception("fail")).count` ![Screenshot 2019-11-14 at 3 51 39 AM](https://user-images.githubusercontent.com/23054875/68809546-268d2e80-0692-11ea-8b2c-bee767478135.png) Closes #26508 from shahidki31/task. Authored-by: shahid Signed-off-by: Marcelo Vanzin --- .../apache/spark/status/AppStatusStore.scala | 82 ++++--------- .../org/apache/spark/status/LiveEntity.scala | 102 ++++++++++++---- .../org/apache/spark/status/storeTypes.scala | 76 +++++++----- .../spark/status/AppStatusStoreSuite.scala | 112 ++++++++++++++---- 4 files changed, 234 insertions(+), 138 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 964ab27a524c4..6b89812cc2bf0 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -136,12 +136,6 @@ private[spark] class AppStatusStore( store.read(classOf[StageDataWrapper], Array(stageId, stageAttemptId)).locality } - // SPARK-26119: we only want to consider successful tasks when calculating the metrics summary, - // but currently this is very expensive when using a disk store. So we only trigger the slower - // code path when we know we have all data in memory. The following method checks whether all - // the data will be in memory. - private def isInMemoryStore: Boolean = store.isInstanceOf[InMemoryStore] || listener.isDefined - /** * Calculates a summary of the task metrics for the given stage attempt, returning the * requested quantiles for the recorded metrics. @@ -162,21 +156,11 @@ private[spark] class AppStatusStore( // cheaper for disk stores (avoids deserialization). val count = { Utils.tryWithResource( - if (isInMemoryStore) { - // For Live UI, we should count the tasks with status "SUCCESS" only. - store.view(classOf[TaskDataWrapper]) - .parent(stageKey) - .index(TaskIndexNames.STATUS) - .first("SUCCESS") - .last("SUCCESS") - .closeableIterator() - } else { - store.view(classOf[TaskDataWrapper]) - .parent(stageKey) - .index(TaskIndexNames.EXEC_RUN_TIME) - .first(0L) - .closeableIterator() - } + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(TaskIndexNames.EXEC_RUN_TIME) + .first(0L) + .closeableIterator() ) { it => var _count = 0L while (it.hasNext()) { @@ -245,50 +229,30 @@ private[spark] class AppStatusStore( // stabilize once the stage finishes. It's also slow, especially with disk stores. val indices = quantiles.map { q => math.min((q * count).toLong, count - 1) } - // TODO: Summary metrics needs to display all the successful tasks' metrics (SPARK-26119). - // For InMemory case, it is efficient to find using the following code. But for diskStore case - // we need an efficient solution to avoid deserialization time overhead. For that, we need to - // rework on the way indexing works, so that we can index by specific metrics for successful - // and failed tasks differently (would be tricky). Also would require changing the disk store - // version (to invalidate old stores). def scanTasks(index: String)(fn: TaskDataWrapper => Long): IndexedSeq[Double] = { - if (isInMemoryStore) { - val quantileTasks = store.view(classOf[TaskDataWrapper]) + Utils.tryWithResource( + store.view(classOf[TaskDataWrapper]) .parent(stageKey) .index(index) .first(0L) - .asScala - .filter { _.status == "SUCCESS"} // Filter "SUCCESS" tasks - .toIndexedSeq - - indices.map { index => - fn(quantileTasks(index.toInt)).toDouble - }.toIndexedSeq - } else { - Utils.tryWithResource( - store.view(classOf[TaskDataWrapper]) - .parent(stageKey) - .index(index) - .first(0L) - .closeableIterator() - ) { it => - var last = Double.NaN - var currentIdx = -1L - indices.map { idx => - if (idx == currentIdx) { + .closeableIterator() + ) { it => + var last = Double.NaN + var currentIdx = -1L + indices.map { idx => + if (idx == currentIdx) { + last + } else { + val diff = idx - currentIdx + currentIdx = idx + if (it.skip(diff - 1)) { + last = fn(it.next()).toDouble last } else { - val diff = idx - currentIdx - currentIdx = idx - if (it.skip(diff - 1)) { - last = fn(it.next()).toDouble - last - } else { - Double.NaN - } + Double.NaN } - }.toIndexedSeq - } + } + }.toIndexedSeq } } @@ -582,7 +546,7 @@ private[spark] class AppStatusStore( private[spark] object AppStatusStore { - val CURRENT_VERSION = 1L + val CURRENT_VERSION = 2L /** * Create an in-memory store for a live application. diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index a0ef8da0a4b6b..5ac7a56b216f2 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -184,6 +184,19 @@ private class LiveTask( info.timeRunning(lastUpdateTime.getOrElse(System.currentTimeMillis())) } + val hasMetrics = metrics.executorDeserializeTime >= 0 + + /** + * SPARK-26260: For non successful tasks, store the metrics as negative to avoid + * the calculation in the task summary. `toApi` method in the `TaskDataWrapper` will make + * it actual value. + */ + val taskMetrics: v1.TaskMetrics = if (hasMetrics && !info.successful) { + makeNegative(metrics) + } else { + metrics + } + new TaskDataWrapper( info.taskId, info.index, @@ -199,30 +212,31 @@ private class LiveTask( newAccumulatorInfos(info.accumulables), errorMessage, - metrics.executorDeserializeTime, - metrics.executorDeserializeCpuTime, - metrics.executorRunTime, - metrics.executorCpuTime, - metrics.resultSize, - metrics.jvmGcTime, - metrics.resultSerializationTime, - metrics.memoryBytesSpilled, - metrics.diskBytesSpilled, - metrics.peakExecutionMemory, - metrics.inputMetrics.bytesRead, - metrics.inputMetrics.recordsRead, - metrics.outputMetrics.bytesWritten, - metrics.outputMetrics.recordsWritten, - metrics.shuffleReadMetrics.remoteBlocksFetched, - metrics.shuffleReadMetrics.localBlocksFetched, - metrics.shuffleReadMetrics.fetchWaitTime, - metrics.shuffleReadMetrics.remoteBytesRead, - metrics.shuffleReadMetrics.remoteBytesReadToDisk, - metrics.shuffleReadMetrics.localBytesRead, - metrics.shuffleReadMetrics.recordsRead, - metrics.shuffleWriteMetrics.bytesWritten, - metrics.shuffleWriteMetrics.writeTime, - metrics.shuffleWriteMetrics.recordsWritten, + hasMetrics, + taskMetrics.executorDeserializeTime, + taskMetrics.executorDeserializeCpuTime, + taskMetrics.executorRunTime, + taskMetrics.executorCpuTime, + taskMetrics.resultSize, + taskMetrics.jvmGcTime, + taskMetrics.resultSerializationTime, + taskMetrics.memoryBytesSpilled, + taskMetrics.diskBytesSpilled, + taskMetrics.peakExecutionMemory, + taskMetrics.inputMetrics.bytesRead, + taskMetrics.inputMetrics.recordsRead, + taskMetrics.outputMetrics.bytesWritten, + taskMetrics.outputMetrics.recordsWritten, + taskMetrics.shuffleReadMetrics.remoteBlocksFetched, + taskMetrics.shuffleReadMetrics.localBlocksFetched, + taskMetrics.shuffleReadMetrics.fetchWaitTime, + taskMetrics.shuffleReadMetrics.remoteBytesRead, + taskMetrics.shuffleReadMetrics.remoteBytesReadToDisk, + taskMetrics.shuffleReadMetrics.localBytesRead, + taskMetrics.shuffleReadMetrics.recordsRead, + taskMetrics.shuffleWriteMetrics.bytesWritten, + taskMetrics.shuffleWriteMetrics.writeTime, + taskMetrics.shuffleWriteMetrics.recordsWritten, stageId, stageAttemptId) @@ -710,6 +724,46 @@ private object LiveEntityHelpers { addMetrics(m1, m2, -1) } + /** + * Convert all the metric values to negative as well as handle zero values. + * This method assumes that all the metric values are greater than or equal to zero + */ + def makeNegative(m: v1.TaskMetrics): v1.TaskMetrics = { + // To handle 0 metric value, add 1 and make the metric negative. + // To recover actual value do `math.abs(metric + 1)` + // Eg: if the metric values are (5, 3, 0, 1) => Updated metric values will be (-6, -4, -1, -2) + // To get actual metric value, do math.abs(metric + 1) => (5, 3, 0, 1) + def updateMetricValue(metric: Long): Long = { + metric * -1L - 1L + } + + createMetrics( + updateMetricValue(m.executorDeserializeTime), + updateMetricValue(m.executorDeserializeCpuTime), + updateMetricValue(m.executorRunTime), + updateMetricValue(m.executorCpuTime), + updateMetricValue(m.resultSize), + updateMetricValue(m.jvmGcTime), + updateMetricValue(m.resultSerializationTime), + updateMetricValue(m.memoryBytesSpilled), + updateMetricValue(m.diskBytesSpilled), + updateMetricValue(m.peakExecutionMemory), + updateMetricValue(m.inputMetrics.bytesRead), + updateMetricValue(m.inputMetrics.recordsRead), + updateMetricValue(m.outputMetrics.bytesWritten), + updateMetricValue(m.outputMetrics.recordsWritten), + updateMetricValue(m.shuffleReadMetrics.remoteBlocksFetched), + updateMetricValue(m.shuffleReadMetrics.localBlocksFetched), + updateMetricValue(m.shuffleReadMetrics.fetchWaitTime), + updateMetricValue(m.shuffleReadMetrics.remoteBytesRead), + updateMetricValue(m.shuffleReadMetrics.remoteBytesReadToDisk), + updateMetricValue(m.shuffleReadMetrics.localBytesRead), + updateMetricValue(m.shuffleReadMetrics.recordsRead), + updateMetricValue(m.shuffleWriteMetrics.bytesWritten), + updateMetricValue(m.shuffleWriteMetrics.writeTime), + updateMetricValue(m.shuffleWriteMetrics.recordsWritten)) + } + private def addMetrics(m1: v1.TaskMetrics, m2: v1.TaskMetrics, mult: Int): v1.TaskMetrics = { createMetrics( m1.executorDeserializeTime + m2.executorDeserializeTime * mult, diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index 9da5bea8bf5c4..f0a94d84d8a04 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -177,10 +177,13 @@ private[spark] class TaskDataWrapper( val accumulatorUpdates: Seq[AccumulableInfo], val errorMessage: Option[String], + val hasMetrics: Boolean, // The following is an exploded view of a TaskMetrics API object. This saves 5 objects - // (= 80 bytes of Java object overhead) per instance of this wrapper. If the first value - // (executorDeserializeTime) is -1L, it means the metrics for this task have not been - // recorded. + // (= 80 bytes of Java object overhead) per instance of this wrapper. Non successful + // tasks' metrics will have negative values in `TaskDataWrapper`. `TaskData` will have + // actual metric values. To recover the actual metric value from `TaskDataWrapper`, + // need use `getMetricValue` method. If `hasMetrics` is false, it means the metrics + // for this task have not been recorded. @KVIndexParam(value = TaskIndexNames.DESER_TIME, parent = TaskIndexNames.STAGE) val executorDeserializeTime: Long, @KVIndexParam(value = TaskIndexNames.DESER_CPU_TIME, parent = TaskIndexNames.STAGE) @@ -233,39 +236,46 @@ private[spark] class TaskDataWrapper( val stageId: Int, val stageAttemptId: Int) { - def hasMetrics: Boolean = executorDeserializeTime >= 0 + // SPARK-26260: To handle non successful tasks metrics (Running, Failed, Killed). + private def getMetricValue(metric: Long): Long = { + if (status != "SUCCESS") { + math.abs(metric + 1) + } else { + metric + } + } def toApi: TaskData = { val metrics = if (hasMetrics) { Some(new TaskMetrics( - executorDeserializeTime, - executorDeserializeCpuTime, - executorRunTime, - executorCpuTime, - resultSize, - jvmGcTime, - resultSerializationTime, - memoryBytesSpilled, - diskBytesSpilled, - peakExecutionMemory, + getMetricValue(executorDeserializeTime), + getMetricValue(executorDeserializeCpuTime), + getMetricValue(executorRunTime), + getMetricValue(executorCpuTime), + getMetricValue(resultSize), + getMetricValue(jvmGcTime), + getMetricValue(resultSerializationTime), + getMetricValue(memoryBytesSpilled), + getMetricValue(diskBytesSpilled), + getMetricValue(peakExecutionMemory), new InputMetrics( - inputBytesRead, - inputRecordsRead), + getMetricValue(inputBytesRead), + getMetricValue(inputRecordsRead)), new OutputMetrics( - outputBytesWritten, - outputRecordsWritten), + getMetricValue(outputBytesWritten), + getMetricValue(outputRecordsWritten)), new ShuffleReadMetrics( - shuffleRemoteBlocksFetched, - shuffleLocalBlocksFetched, - shuffleFetchWaitTime, - shuffleRemoteBytesRead, - shuffleRemoteBytesReadToDisk, - shuffleLocalBytesRead, - shuffleRecordsRead), + getMetricValue(shuffleRemoteBlocksFetched), + getMetricValue(shuffleLocalBlocksFetched), + getMetricValue(shuffleFetchWaitTime), + getMetricValue(shuffleRemoteBytesRead), + getMetricValue(shuffleRemoteBytesReadToDisk), + getMetricValue(shuffleLocalBytesRead), + getMetricValue(shuffleRecordsRead)), new ShuffleWriteMetrics( - shuffleBytesWritten, - shuffleWriteTime, - shuffleRecordsWritten))) + getMetricValue(shuffleBytesWritten), + getMetricValue(shuffleWriteTime), + getMetricValue(shuffleRecordsWritten)))) } else { None } @@ -296,8 +306,10 @@ private[spark] class TaskDataWrapper( @JsonIgnore @KVIndex(value = TaskIndexNames.SCHEDULER_DELAY, parent = TaskIndexNames.STAGE) def schedulerDelay: Long = { if (hasMetrics) { - AppStatusUtils.schedulerDelay(launchTime, resultFetchStart, duration, executorDeserializeTime, - resultSerializationTime, executorRunTime) + AppStatusUtils.schedulerDelay(launchTime, resultFetchStart, duration, + getMetricValue(executorDeserializeTime), + getMetricValue(resultSerializationTime), + getMetricValue(executorRunTime)) } else { -1L } @@ -330,7 +342,7 @@ private[spark] class TaskDataWrapper( @JsonIgnore @KVIndex(value = TaskIndexNames.SHUFFLE_TOTAL_READS, parent = TaskIndexNames.STAGE) private def shuffleTotalReads: Long = { if (hasMetrics) { - shuffleLocalBytesRead + shuffleRemoteBytesRead + getMetricValue(shuffleLocalBytesRead) + getMetricValue(shuffleRemoteBytesRead) } else { -1L } @@ -339,7 +351,7 @@ private[spark] class TaskDataWrapper( @JsonIgnore @KVIndex(value = TaskIndexNames.SHUFFLE_TOTAL_BLOCKS, parent = TaskIndexNames.STAGE) private def shuffleTotalBlocks: Long = { if (hasMetrics) { - shuffleLocalBlocksFetched + shuffleRemoteBlocksFetched + getMetricValue(shuffleLocalBlocksFetched) + getMetricValue(shuffleRemoteBlocksFetched) } else { -1L } diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala index 165fdb71cc78b..735e51942626f 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala @@ -18,7 +18,9 @@ package org.apache.spark.status import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.util.Distribution +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.{TaskInfo, TaskLocality} +import org.apache.spark.util.{Distribution, Utils} import org.apache.spark.util.kvstore._ class AppStatusStoreSuite extends SparkFunSuite { @@ -76,42 +78,61 @@ class AppStatusStoreSuite extends SparkFunSuite { assert(store.count(classOf[CachedQuantile]) === 2) } - private def createLiveStore(inMemoryStore: InMemoryStore): AppStatusStore = { + private def createAppStore(disk: Boolean, live: Boolean): AppStatusStore = { val conf = new SparkConf() - val store = new ElementTrackingStore(inMemoryStore, conf) - val listener = new AppStatusListener(store, conf, true, None) - new AppStatusStore(store, listener = Some(listener)) - } + if (live) { + return AppStatusStore.createLiveStore(conf) + } - test("SPARK-28638: only successful tasks have taskSummary when with in memory kvstore") { - val store = new InMemoryStore() - (0 until 5).foreach { i => store.write(newTaskData(i, status = "FAILED")) } - Seq(new AppStatusStore(store), createLiveStore(store)).foreach { appStore => - val summary = appStore.taskSummary(stageId, attemptId, uiQuantiles) - assert(summary.size === 0) + val store: KVStore = if (disk) { + val testDir = Utils.createTempDir() + val diskStore = KVUtils.open(testDir, getClass.getName) + new ElementTrackingStore(diskStore, conf) + } else { + new ElementTrackingStore(new InMemoryStore, conf) } + new AppStatusStore(store) } - test("SPARK-28638: summary should contain successful tasks only when with in memory kvstore") { - val store = new InMemoryStore() + Seq( + "disk" -> createAppStore(disk = true, live = false), + "in memory" -> createAppStore(disk = false, live = false), + "in memory live" -> createAppStore(disk = false, live = true) + ).foreach { case (hint, appStore) => + test(s"SPARK-26260: summary should contain only successful tasks' metrics (store = $hint)") { + val store = appStore.store + + // Success and failed tasks metrics + for (i <- 0 to 5) { + if (i % 2 == 0) { + writeTaskDataToStore(i, store, "FAILED") + } else { + writeTaskDataToStore(i, store, "SUCCESS") + } + } - for (i <- 0 to 5) { - if (i % 2 == 1) { - store.write(newTaskData(i, status = "FAILED")) - } else { - store.write(newTaskData(i)) + // Running tasks metrics (-1 = no metrics reported, positive = metrics have been reported) + Seq(-1, 6).foreach { metric => + writeTaskDataToStore(metric, store, "RUNNING") } - } - Seq(new AppStatusStore(store), createLiveStore(store)).foreach { appStore => + /** + * Following are the tasks metrics, + * 1, 3, 5 => Success + * 0, 2, 4 => Failed + * -1, 6 => Running + * + * Task summary will consider (1, 3, 5) only + */ val summary = appStore.taskSummary(stageId, attemptId, uiQuantiles).get - val values = Array(0.0, 2.0, 4.0) + val values = Array(1.0, 3.0, 5.0) val dist = new Distribution(values, 0, values.length).getQuantiles(uiQuantiles.sorted) dist.zip(summary.executorRunTime).foreach { case (expected, actual) => assert(expected === actual) } + appStore.close() } } @@ -133,9 +154,54 @@ class AppStatusStoreSuite extends SparkFunSuite { private def newTaskData(i: Int, status: String = "SUCCESS"): TaskDataWrapper = { new TaskDataWrapper( - i, i, i, i, i, i, i.toString, i.toString, status, i.toString, false, Nil, None, + i.toLong, i, i, i, i, i, i.toString, i.toString, status, i.toString, false, Nil, None, true, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, stageId, attemptId) } + + private def writeTaskDataToStore(i: Int, store: KVStore, status: String): Unit = { + val liveTask = new LiveTask(new TaskInfo( i.toLong, i, i, i.toLong, i.toString, + i.toString, TaskLocality.ANY, false), stageId, attemptId, None) + + if (status == "SUCCESS") { + liveTask.info.finishTime = 1L + } else if (status == "FAILED") { + liveTask.info.failed = true + liveTask.info.finishTime = 1L + } + + val taskMetrics = getTaskMetrics(i) + liveTask.updateMetrics(taskMetrics) + liveTask.write(store.asInstanceOf[ElementTrackingStore], 1L) + } + + private def getTaskMetrics(i: Int): TaskMetrics = { + val taskMetrics = new TaskMetrics() + taskMetrics.setExecutorDeserializeTime(i) + taskMetrics.setExecutorDeserializeCpuTime(i) + taskMetrics.setExecutorRunTime(i) + taskMetrics.setExecutorCpuTime(i) + taskMetrics.setResultSize(i) + taskMetrics.setJvmGCTime(i) + taskMetrics.setResultSerializationTime(i) + taskMetrics.incMemoryBytesSpilled(i) + taskMetrics.incDiskBytesSpilled(i) + taskMetrics.incPeakExecutionMemory(i) + taskMetrics.inputMetrics.incBytesRead(i) + taskMetrics.inputMetrics.incRecordsRead(i) + taskMetrics.outputMetrics.setBytesWritten(i) + taskMetrics.outputMetrics.setRecordsWritten(i) + taskMetrics.shuffleReadMetrics.incRemoteBlocksFetched(i) + taskMetrics.shuffleReadMetrics.incLocalBlocksFetched(i) + taskMetrics.shuffleReadMetrics.incFetchWaitTime(i) + taskMetrics.shuffleReadMetrics.incRemoteBytesRead(i) + taskMetrics.shuffleReadMetrics.incRemoteBytesReadToDisk(i) + taskMetrics.shuffleReadMetrics.incLocalBytesRead(i) + taskMetrics.shuffleReadMetrics.incRecordsRead(i) + taskMetrics.shuffleWriteMetrics.incBytesWritten(i) + taskMetrics.shuffleWriteMetrics.incWriteTime(i) + taskMetrics.shuffleWriteMetrics.incRecordsWritten(i) + taskMetrics + } } From 1466863cee7dfab75459e9f2c1698d97d4a54fee Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 25 Nov 2019 10:54:14 -0800 Subject: [PATCH 0720/1387] [SPARK-30015][BUILD] Move hive-storage-api dependency from `hive-2.3` to `sql/core` # What changes were proposed in this pull request? This PR aims to relocate the following internal dependencies to compile `sql/core` without `-Phive-2.3` profile. 1. Move the `hive-storage-api` to `sql/core` which is using `hive-storage-api` really. **BEFORE (sql/core compilation)** ``` $ ./build/mvn -DskipTests --pl sql/core --am compile ... [ERROR] [Error] /Users/dongjoon/APACHE/spark/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala:21: object hive is not a member of package org.apache.hadoop ... [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ ``` **AFTER (sql/core compilation)** ``` $ ./build/mvn -DskipTests --pl sql/core --am compile ... [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 02:04 min [INFO] Finished at: 2019-11-25T00:20:11-08:00 [INFO] ------------------------------------------------------------------------ ``` 2. For (1), add `commons-lang:commons-lang` test dependency to `spark-core` module to manage the dependency explicitly. Without this, `core` module fails to build the test classes. ``` $ ./build/mvn -DskipTests --pl core --am package -Phadoop-3.2 ... [INFO] --- scala-maven-plugin:4.3.0:testCompile (scala-test-compile-first) spark-core_2.12 --- [INFO] Using incremental compilation using Mixed compile order [INFO] Compiler bridge file: /Users/dongjoon/.sbt/1.0/zinc/org.scala-sbt/org.scala-sbt-compiler-bridge_2.12-1.3.1-bin_2.12.10__52.0-1.3.1_20191012T045515.jar [INFO] Compiling 271 Scala sources and 26 Java sources to /spark/core/target/scala-2.12/test-classes ... [ERROR] [Error] /spark/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala:23: object lang is not a member of package org.apache.commons [ERROR] [Error] /spark/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala:49: not found: value SerializationUtils [ERROR] two errors found ``` **BEFORE (commons-lang:commons-lang)** The following is the previous `core` module's `commons-lang:commons-lang` dependency. 1. **branch-2.4** ``` $ mvn dependency:tree -Dincludes=commons-lang:commons-lang [INFO] --- maven-dependency-plugin:3.0.2:tree (default-cli) spark-core_2.11 --- [INFO] org.apache.spark:spark-core_2.11:jar:2.4.5-SNAPSHOT [INFO] \- org.spark-project.hive:hive-exec:jar:1.2.1.spark2:provided [INFO] \- commons-lang:commons-lang:jar:2.6:compile ``` 2. **v3.0.0-preview (-Phadoop-3.2)** ``` $ mvn dependency:tree -Dincludes=commons-lang:commons-lang -Phadoop-3.2 [INFO] --- maven-dependency-plugin:3.1.1:tree (default-cli) spark-core_2.12 --- [INFO] org.apache.spark:spark-core_2.12:jar:3.0.0-preview [INFO] \- org.apache.hive:hive-storage-api:jar:2.6.0:compile [INFO] \- commons-lang:commons-lang:jar:2.6:compile ``` 3. **v3.0.0-preview(default)** ``` $ mvn dependency:tree -Dincludes=commons-lang:commons-lang [INFO] --- maven-dependency-plugin:3.1.1:tree (default-cli) spark-core_2.12 --- [INFO] org.apache.spark:spark-core_2.12:jar:3.0.0-preview [INFO] \- org.apache.hadoop:hadoop-client:jar:2.7.4:compile [INFO] \- org.apache.hadoop:hadoop-common:jar:2.7.4:compile [INFO] \- commons-lang:commons-lang:jar:2.6:compile ``` **AFTER (commons-lang:commons-lang)** ``` $ mvn dependency:tree -Dincludes=commons-lang:commons-lang [INFO] --- maven-dependency-plugin:3.1.1:tree (default-cli) spark-core_2.12 --- [INFO] org.apache.spark:spark-core_2.12:jar:3.0.0-SNAPSHOT [INFO] \- commons-lang:commons-lang:jar:2.6:test ``` Since we wanted to verify that this PR doesn't change `hive-1.2` profile, we merged [SPARK-30005 Update `test-dependencies.sh` to check `hive-1.2/2.3` profile](a1706e2fa7) before this PR. ### Why are the changes needed? - Apache Spark 2.4's `sql/core` is using `Apache ORC (nohive)` jars including shaded `hive-storage-api` to access ORC data sources. - Apache Spark 3.0's `sql/core` is using `Apache Hive` jars directly. Previously, `-Phadoop-3.2` hid this `hive-storage-api` dependency. Now, we are using `-Phive-2.3` instead. As I mentioned [previously](https://github.com/apache/spark/pull/26619#issuecomment-556926064), this PR is required to compile `sql/core` module without `-Phive-2.3`. - For `sql/hive` and `sql/hive-thriftserver`, it's natural that we need `-Phive-1.2` or `-Phive-2.3`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This will pass the Jenkins (with the dependency check and unit tests). We need to check manually with `./build/mvn -DskipTests --pl sql/core --am compile`. This closes #26657 . Closes #26658 from dongjoon-hyun/SPARK-30015. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/pom.xml | 6 +++++- pom.xml | 22 ++++++++++++++-------- sql/core/pom.xml | 4 ++++ 3 files changed, 23 insertions(+), 9 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 3eedc69c9593b..04a3fe521ef87 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -163,7 +163,11 @@ javax.servlet-api ${javaxservlet.version} - + + commons-lang + commons-lang + test + org.apache.commons commons-lang3 diff --git a/pom.xml b/pom.xml index 1c12b9c5c7e4d..4910b0c5fa0f1 100644 --- a/pom.xml +++ b/pom.xml @@ -229,6 +229,7 @@ compile compile provided + compile compile compile test @@ -2184,6 +2185,18 @@ univocity-parsers 2.8.3 + + org.apache.hive + hive-storage-api + 2.6.0 + ${hive.storage.scope} + + + commons-lang + commons-lang + + + @@ -2934,6 +2947,7 @@ 1.2 ${hive.deps.scope} + provided nohive 3.2.10 @@ -2942,14 +2956,6 @@ hive-2.3 - - - - org.apache.hive - hive-storage-api - 2.6.0 - - diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 5cd7c656ea725..87874e2a0eb07 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -95,6 +95,10 @@ orc-mapreduce ${orc.classifier} + + org.apache.hive + hive-storage-api + org.apache.parquet parquet-column From 38240a74dc047796e9f239e44d9bc0bbc66e1f7f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 25 Nov 2019 12:03:15 -0800 Subject: [PATCH 0721/1387] [SPARK-30030][INFRA] Use RegexChecker instead of TokenChecker to check `org.apache.commons.lang.` ### What changes were proposed in this pull request? This PR replace `TokenChecker` with `RegexChecker` in `scalastyle` and fixes the missed instances. ### Why are the changes needed? This will remove the old `comons-lang2` dependency from `core` module **BEFORE** ``` $ dev/scalastyle Scalastyle checks failed at following occurrences: [error] /Users/dongjoon/PRS/SPARK-SerializationUtils/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala:23:7: Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead [error] of Commons Lang 2 (package org.apache.commons.lang.*) [error] Total time: 23 s, completed Nov 25, 2019 11:47:44 AM ``` **AFTER** ``` $ dev/scalastyle Scalastyle checks passed. ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the GitHub Action linter. Closes #26666 from dongjoon-hyun/SPARK-29081-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/util/PropertiesCloneBenchmark.scala | 2 +- scalastyle-config.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala b/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala index 0726886c70fe6..baacc7527a806 100644 --- a/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala @@ -20,7 +20,7 @@ import java.util.Properties import scala.util.Random -import org.apache.commons.lang.SerializationUtils +import org.apache.commons.lang3.SerializationUtils import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 4892819ae9973..73ac14fdba1cf 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -258,7 +258,7 @@ This file is divided into 3 sections: scala.collection.JavaConverters._ and use .asScala / .asJava methods - + org\.apache\.commons\.lang\. Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead of Commons Lang 2 (package org.apache.commons.lang.*) From 2a28c73d81dd12fa39eb52d3eb343e651123a51d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 25 Nov 2019 15:17:27 -0800 Subject: [PATCH 0722/1387] [SPARK-30031][BUILD][SQL] Remove `hive-2.3` profile from `sql/hive` module ### What changes were proposed in this pull request? This PR aims to remove `hive-2.3` profile from `sql/hive` module. ### Why are the changes needed? Currently, we need `-Phive-1.2` or `-Phive-2.3` additionally to build `hive` or `hive-thriftserver` module. Without specifying it, the build fails like the following. This PR will recover it. ``` $ build/mvn -DskipTests compile --pl sql/hive ... [ERROR] [Error] /Users/dongjoon/APACHE/spark-merge/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala:32: object serde is not a member of package org.apache.hadoop.hive ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? 1. Pass GitHub Action dependency check with no manifest change. 2. Pass GitHub Action build for all combinations. 3. Pass the Jenkins UT. Closes #26668 from dongjoon-hyun/SPARK-30031. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 8 +++++++ sql/hive/pom.xml | 58 ++++++++++++++++++------------------------------ 2 files changed, 29 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index 4910b0c5fa0f1..046c3d4899d14 100644 --- a/pom.xml +++ b/pom.xml @@ -230,6 +230,10 @@ compile provided compile + compile + compile + compile + compile compile compile test @@ -2948,6 +2952,10 @@ 1.2 ${hive.deps.scope} provided + provided + provided + provided + provided nohive 3.2.10 diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 69961acd510d8..c37582386347b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -88,12 +88,11 @@ ${protobuf.version} --> - ${hive.group} hive-exec @@ -103,16 +102,26 @@ ${hive.group} hive-metastore - + + ${hive.group} + hive-serde + ${hive.serde.scope} + + + ${hive.group} + hive-shims + ${hive.shims.scope} + + + org.apache.hive + hive-llap-common + ${hive.llap.scope} + + + org.apache.hive + hive-llap-client + ${hive.llap.scope} + org.apache.avro @@ -208,31 +217,6 @@ - - hive-2.3 - - - ${hive.group} - hive-common - - - ${hive.group} - hive-serde - - - ${hive.group} - hive-shims - - - org.apache.hive - hive-llap-common - - - org.apache.hive - hive-llap-client - - - From 53e19f3678f75f8bc65d352ba0b6c4d2b2cbc0cc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 25 Nov 2019 20:08:11 -0800 Subject: [PATCH 0723/1387] [SPARK-30032][BUILD] Upgrade to ORC 1.5.8 ### What changes were proposed in this pull request? This PR aims to upgrade to Apache ORC 1.5.8. ### Why are the changes needed? This will bring the latest bug fixes. The following is the full release note. - https://issues.apache.org/jira/projects/ORC/versions/12346462 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #26669 from dongjoon-hyun/SPARK-ORC-1.5.8. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 6 +++--- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 6 +++--- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 6 +++--- pom.xml | 2 +- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 54608d203133c..0821a27de8ee9 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -160,9 +160,9 @@ okapi-shade-0.4.2.jar okhttp-3.12.0.jar okio-1.15.0.jar opencsv-2.3.jar -orc-core-1.5.7-nohive.jar -orc-mapreduce-1.5.7-nohive.jar -orc-shims-1.5.7.jar +orc-core-1.5.8-nohive.jar +orc-mapreduce-1.5.8-nohive.jar +orc-shims-1.5.8.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 1fb9bbbd9b054..b953d65412167 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -176,9 +176,9 @@ okapi-shade-0.4.2.jar okhttp-3.12.0.jar okio-1.15.0.jar opencsv-2.3.jar -orc-core-1.5.7.jar -orc-mapreduce-1.5.7.jar -orc-shims-1.5.7.jar +orc-core-1.5.8.jar +orc-mapreduce-1.5.8.jar +orc-shims-1.5.8.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 917fde61fad1a..c9461f6c7f085 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -193,9 +193,9 @@ okhttp-2.7.5.jar okhttp-3.12.0.jar okio-1.15.0.jar opencsv-2.3.jar -orc-core-1.5.7.jar -orc-mapreduce-1.5.7.jar -orc-shims-1.5.7.jar +orc-core-1.5.8.jar +orc-mapreduce-1.5.8.jar +orc-shims-1.5.8.jar oro-2.0.8.jar osgi-resource-locator-1.0.3.jar paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index 046c3d4899d14..d49f645da947e 100644 --- a/pom.xml +++ b/pom.xml @@ -139,7 +139,7 @@ 2.3.1 10.12.1.1 1.10.1 - 1.5.7 + 1.5.8 com.twitter 1.6.0 From 8b0121bea8bc65d37f5ef4dfc80d9faea905e7eb Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 26 Nov 2019 12:49:56 +0800 Subject: [PATCH 0724/1387] [MINOR][DOC] Fix the CalendarIntervalType description ### What changes were proposed in this pull request? fix the overdue and incorrect description about CalendarIntervalType ### Why are the changes needed? api doc correctness ### Does this PR introduce any user-facing change? no ### How was this patch tested? no Closes #26659 from yaooqinn/intervaldoc. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../apache/spark/sql/types/CalendarIntervalType.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala index 8a9a2e246a30b..641fb354dd322 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala @@ -21,12 +21,15 @@ import org.apache.spark.annotation.Stable import org.apache.spark.unsafe.types.CalendarInterval /** - * The data type representing calendar time intervals. The calendar time interval is stored - * internally in two components: number of months the number of microseconds. + * The data type representing calendar intervals. The calendar interval is stored internally in + * three components: + * an integer value representing the number of `months` in this interval, + * an integer value representing the number of `days` in this interval, + * a long value representing the number of `microseconds` in this interval. * - * Please use the singleton `DataTypes.CalendarIntervalType`. + * Please use the singleton `DataTypes.CalendarIntervalType` to refer the type. * - * @note Calendar intervals are not comparable. + * @note Calendar intervals support comparison and ordering since 3.0.0. * * @since 1.5.0 */ From 780555bf60f03db761f3e89693a927edc7b51fd9 Mon Sep 17 00:00:00 2001 From: wuyi Date: Tue, 26 Nov 2019 12:54:34 +0800 Subject: [PATCH 0725/1387] [MINOR][CORE] Make EventLogger codec be consistent between EventLogFileWriter and SparkContext ### What changes were proposed in this pull request? Use the same function (`codecName(conf: SparkConf)`) between `EventLogFileWriter` and `SparkContext` to get the consistent codec name for EventLogger. ### Why are the changes needed? #24921 added a new conf for EventLogger's compression codec. We should reflect this change into `SparkContext` as well. Though I didn't find any places that `SparkContext.eventLogCodec` really takes an effect, I think it'd be better to have it as a right value. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass Jenkins. Closes #26665 from Ngone51/consistent-eventLogCodec. Authored-by: wuyi Signed-off-by: Wenchen Fan --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index db3f2266cf33c..fc9d1f04cd7b5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -444,7 +444,7 @@ class SparkContext(config: SparkConf) extends Logging { _eventLogCodec = { val compress = _conf.get(EVENT_LOG_COMPRESS) if (compress && isEventLogEnabled) { - Some(CompressionCodec.getCodecName(_conf)).map(CompressionCodec.getShortName) + Some(_conf.get(EVENT_LOG_COMPRESSION_CODEC)).map(CompressionCodec.getShortName) } else { None } From 373c2c3f44f693f86ef60224697cf80b639e655e Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 26 Nov 2019 14:10:46 +0800 Subject: [PATCH 0726/1387] [SPARK-29862][SQL] CREATE (OR REPLACE) ... VIEW should look up catalog/table like v2 commands ### What changes were proposed in this pull request? Add CreateViewStatement and make CREARE VIEW go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users. e.g. ``` USE my_catalog DESC v // success and describe the view v from my_catalog CREATE VIEW v AS SELECT 1 // report view not found as there is no view v in the session catalog ``` ### Does this PR introduce any user-facing change? Yes. When running CREATE VIEW ... Spark fails the command if the current catalog is set to a v2 catalog, or the view name specified a v2 catalog. ### How was this patch tested? unit tests Closes #26649 from huaxingao/spark-29862. Authored-by: Huaxin Gao Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/catalyst/analysis/view.scala | 35 +++++- .../sql/catalyst/parser/AstBuilder.scala | 51 +++++++++ .../catalyst/plans/logical/statements.scala | 17 ++- .../sql/catalyst/parser/DDLParserSuite.scala | 101 +++++++++++++++++- .../analysis/ResolveSessionCatalog.scala | 16 +++ .../spark/sql/execution/SparkSqlParser.scala | 52 --------- .../spark/sql/execution/command/views.scala | 37 +------ .../sql/connector/DataSourceV2SQLSuite.scala | 8 ++ .../execution/command/DDLParserSuite.scala | 51 --------- 10 files changed, 227 insertions(+), 143 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index cc273fd36011e..8cff591938753 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -165,7 +165,7 @@ statement | DROP TABLE (IF EXISTS)? multipartIdentifier PURGE? #dropTable | DROP VIEW (IF EXISTS)? multipartIdentifier #dropView | CREATE (OR REPLACE)? (GLOBAL? TEMPORARY)? - VIEW (IF NOT EXISTS)? tableIdentifier + VIEW (IF NOT EXISTS)? multipartIdentifier identifierCommentList? ((COMMENT STRING) | (PARTITIONED ON identifierList) | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala index 76bf3740ed0fc..65601640fa044 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf /** - * This file defines analysis rules related to views. + * This file defines view types and analysis rules related to views. */ /** @@ -90,3 +90,36 @@ object EliminateView extends Rule[LogicalPlan] with CastSupport { child } } + +/** + * ViewType is used to specify the expected view type when we want to create or replace a view in + * [[CreateViewStatement]]. + */ +sealed trait ViewType { + override def toString: String = getClass.getSimpleName.stripSuffix("$") +} + +/** + * LocalTempView means session-scoped local temporary views. Its lifetime is the lifetime of the + * session that created it, i.e. it will be automatically dropped when the session terminates. It's + * not tied to any databases, i.e. we can't use `db1.view1` to reference a local temporary view. + */ +object LocalTempView extends ViewType + +/** + * GlobalTempView means cross-session global temporary views. Its lifetime is the lifetime of the + * Spark application, i.e. it will be automatically dropped when the application terminates. It's + * tied to a system preserved database `global_temp`, and we must use the qualified name to refer a + * global temp view, e.g. SELECT * FROM global_temp.view1. + */ +object GlobalTempView extends ViewType + +/** + * PersistedView means cross-session persisted views. Persisted views stay until they are + * explicitly dropped by user command. It's always tied to a database, default to the current + * database if not specified. + * + * Note that, Existing persisted view with the same name are not visible to the current session + * while the local temporary view exists, unless the view name is qualified by database. + */ +object PersistedView extends ViewType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c450e47e41b5b..3974efea9e352 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3228,6 +3228,57 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } + /** + * Create or replace a view. This creates a [[CreateViewStatement]] + * + * For example: + * {{{ + * CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] multi_part_name + * [(column_name [COMMENT column_comment], ...) ] + * create_view_clauses + * + * AS SELECT ...; + * + * create_view_clauses (order insensitive): + * [COMMENT view_comment] + * [TBLPROPERTIES (property_name = property_value, ...)] + * }}} + */ + override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) { + if (!ctx.identifierList.isEmpty) { + operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx) + } + + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED ON", ctx) + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) + + val userSpecifiedColumns = Option(ctx.identifierCommentList).toSeq.flatMap { icl => + icl.identifierComment.asScala.map { ic => + ic.identifier.getText -> Option(ic.STRING).map(string) + } + } + + val viewType = if (ctx.TEMPORARY == null) { + PersistedView + } else if (ctx.GLOBAL != null) { + GlobalTempView + } else { + LocalTempView + } + CreateViewStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + userSpecifiedColumns, + ctx.STRING.asScala.headOption.map(string), + ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues) + .getOrElse(Map.empty), + Option(source(ctx.query)), + plan(ctx.query), + ctx.EXISTS != null, + ctx.REPLACE != null, + viewType) + } + /** * Alter the query of a view. This creates a [[AlterViewAsStatement]] * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 903f218e9f0af..5db099e1de631 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.analysis.ViewType import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.{DataType, StructType} @@ -86,6 +87,20 @@ case class CreateTableAsSelectStatement( override def children: Seq[LogicalPlan] = Seq(asSelect) } +/** + * A CREATE VIEW statement, as parsed from SQL. + */ +case class CreateViewStatement( + viewName: Seq[String], + userSpecifiedColumns: Seq[(String, Option[String])], + comment: Option[String], + properties: Map[String, String], + originalText: Option[String], + child: LogicalPlan, + allowExisting: Boolean, + replace: Boolean, + viewType: ViewType) extends ParsedStatement + /** * A REPLACE TABLE command, as parsed from SQL. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index d2575dabf847c..aa2cde2e201b4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ @@ -1669,6 +1669,105 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsed, expected) } + test("create view -- basic") { + val v1 = "CREATE VIEW view1 AS SELECT * FROM tab1" + val parsed1 = parsePlan(v1) + + val expected1 = CreateViewStatement( + Seq("view1"), + Seq.empty[(String, Option[String])], + None, + Map.empty[String, String], + Some("SELECT * FROM tab1"), + parsePlan("SELECT * FROM tab1"), + false, + false, + PersistedView) + comparePlans(parsed1, expected1) + + val v2 = "CREATE TEMPORARY VIEW a.b.c AS SELECT * FROM tab1" + val parsed2 = parsePlan(v2) + + val expected2 = CreateViewStatement( + Seq("a", "b", "c"), + Seq.empty[(String, Option[String])], + None, + Map.empty[String, String], + Some("SELECT * FROM tab1"), + parsePlan("SELECT * FROM tab1"), + false, + false, + LocalTempView) + comparePlans(parsed2, expected2) + } + + test("create view - full") { + val v1 = + """ + |CREATE OR REPLACE VIEW view1 + |(col1, col3 COMMENT 'hello') + |TBLPROPERTIES('prop1Key'="prop1Val") + |COMMENT 'BLABLA' + |AS SELECT * FROM tab1 + """.stripMargin + val parsed1 = parsePlan(v1) + val expected1 = CreateViewStatement( + Seq("view1"), + Seq("col1" -> None, "col3" -> Some("hello")), + Some("BLABLA"), + Map("prop1Key" -> "prop1Val"), + Some("SELECT * FROM tab1"), + parsePlan("SELECT * FROM tab1"), + false, + true, + PersistedView) + comparePlans(parsed1, expected1) + + val v2 = + """ + |CREATE OR REPLACE GLOBAL TEMPORARY VIEW a.b.c + |(col1, col3 COMMENT 'hello') + |TBLPROPERTIES('prop1Key'="prop1Val") + |COMMENT 'BLABLA' + |AS SELECT * FROM tab1 + """.stripMargin + val parsed2 = parsePlan(v2) + val expected2 = CreateViewStatement( + Seq("a", "b", "c"), + Seq("col1" -> None, "col3" -> Some("hello")), + Some("BLABLA"), + Map("prop1Key" -> "prop1Val"), + Some("SELECT * FROM tab1"), + parsePlan("SELECT * FROM tab1"), + false, + true, + GlobalTempView) + comparePlans(parsed2, expected2) + } + + test("create view -- partitioned view") { + val v1 = "CREATE VIEW view1 partitioned on (ds, hr) as select * from srcpart" + intercept[ParseException] { + parsePlan(v1) + } + } + + test("create view - duplicate clauses") { + def createViewStatement(duplicateClause: String): String = { + s""" + |CREATE OR REPLACE VIEW view1 + |(col1, col3 COMMENT 'hello') + |$duplicateClause + |$duplicateClause + |AS SELECT * FROM tab1 + """.stripMargin + } + val sql1 = createViewStatement("COMMENT 'BLABLA'") + val sql2 = createViewStatement("TBLPROPERTIES('prop1Key'=\"prop1Val\")") + intercept(sql1, "Found duplicate clauses: COMMENT") + intercept(sql2, "Found duplicate clauses: TBLPROPERTIES") + } + test("SHOW TBLPROPERTIES table") { comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index d95ddf968b30b..4cc701274a1f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -451,6 +451,22 @@ class ResolveSessionCatalog( originalText, query) + case CreateViewStatement( + tableName, userSpecifiedColumns, comment, properties, + originalText, child, allowExisting, replace, viewType) => + + val v1TableName = parseV1Table(tableName, "CREATE VIEW") + CreateViewCommand( + v1TableName.asTableIdentifier, + userSpecifiedColumns, + comment, + properties, + originalText, + child, + allowExisting, + replace, + viewType) + case ShowTablePropertiesStatement(SessionCatalog(_, tableName), propertyKey) => ShowTablePropertiesCommand( tableName.asTableIdentifier, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 44e60767e6b1a..128ba82dd471b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -715,58 +715,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } } - /** - * Create or replace a view. This creates a [[CreateViewCommand]] command. - * - * For example: - * {{{ - * CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name - * [(column_name [COMMENT column_comment], ...) ] - * create_view_clauses - * - * AS SELECT ...; - * - * create_view_clauses (order insensitive): - * [COMMENT view_comment] - * [TBLPROPERTIES (property_name = property_value, ...)] - * }}} - */ - override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) { - if (!ctx.identifierList.isEmpty) { - operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx) - } - - checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) - checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED ON", ctx) - checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) - - val userSpecifiedColumns = Option(ctx.identifierCommentList).toSeq.flatMap { icl => - icl.identifierComment.asScala.map { ic => - ic.identifier.getText -> Option(ic.STRING).map(string) - } - } - - val viewType = if (ctx.TEMPORARY == null) { - PersistedView - } else if (ctx.GLOBAL != null) { - GlobalTempView - } else { - LocalTempView - } - - CreateViewCommand( - name = visitTableIdentifier(ctx.tableIdentifier), - userSpecifiedColumns = userSpecifiedColumns, - comment = ctx.STRING.asScala.headOption.map(string), - properties = ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues) - .getOrElse(Map.empty), - originalText = Option(source(ctx.query)), - child = plan(ctx.query), - allowExisting = ctx.EXISTS != null, - replace = ctx.REPLACE != null, - viewType = viewType) - } - /** * Create a [[ScriptInputOutputSchema]]. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 2440c9bf07cd5..a8dbd57e42700 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, UnresolvedFunction, UnresolvedRelation, ViewType} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Alias, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan @@ -29,41 +29,6 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} import org.apache.spark.sql.types.{MetadataBuilder, StructType} import org.apache.spark.sql.util.SchemaUtils - -/** - * ViewType is used to specify the expected view type when we want to create or replace a view in - * [[CreateViewCommand]]. - */ -sealed trait ViewType { - override def toString: String = getClass.getSimpleName.stripSuffix("$") -} - -/** - * LocalTempView means session-scoped local temporary views. Its lifetime is the lifetime of the - * session that created it, i.e. it will be automatically dropped when the session terminates. It's - * not tied to any databases, i.e. we can't use `db1.view1` to reference a local temporary view. - */ -object LocalTempView extends ViewType - -/** - * GlobalTempView means cross-session global temporary views. Its lifetime is the lifetime of the - * Spark application, i.e. it will be automatically dropped when the application terminates. It's - * tied to a system preserved database `global_temp`, and we must use the qualified name to refer a - * global temp view, e.g. SELECT * FROM global_temp.view1. - */ -object GlobalTempView extends ViewType - -/** - * PersistedView means cross-session persisted views. Persisted views stay until they are - * explicitly dropped by user command. It's always tied to a database, default to the current - * database if not specified. - * - * Note that, Existing persisted view with the same name are not visible to the current session - * while the local temporary view exists, unless the view name is qualified by database. - */ -object PersistedView extends ViewType - - /** * Create or replace a view with given query plan. This command will generate some view-specific * properties(e.g. view default database, view query output column names) and store them as diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 018c0c22c607e..9279d2e882997 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1728,6 +1728,14 @@ class DataSourceV2SQLSuite assert(e.message.contains("ALTER VIEW QUERY is only supported with v1 tables")) } + test("CREATE VIEW") { + val v = "testcat.ns1.ns2.v" + val e = intercept[AnalysisException] { + sql(s"CREATE VIEW $v AS SELECT * FROM tab1") + } + assert(e.message.contains("CREATE VIEW is only supported with v1 tables")) + } + test("SHOW TBLPROPERTIES: v2 table") { val t = "testcat.ns1.ns2.tbl" withTable(t) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 41a2168b4a18a..813cae29d2683 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -972,57 +972,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { assert(desc.comment == Some("no comment")) } - test("create view -- basic") { - val v1 = "CREATE VIEW view1 AS SELECT * FROM tab1" - val command = parser.parsePlan(v1).asInstanceOf[CreateViewCommand] - assert(!command.allowExisting) - assert(command.name.database.isEmpty) - assert(command.name.table == "view1") - assert(command.originalText == Some("SELECT * FROM tab1")) - assert(command.userSpecifiedColumns.isEmpty) - } - - test("create view - full") { - val v1 = - """ - |CREATE OR REPLACE VIEW view1 - |(col1, col3 COMMENT 'hello') - |TBLPROPERTIES('prop1Key'="prop1Val") - |COMMENT 'BLABLA' - |AS SELECT * FROM tab1 - """.stripMargin - val command = parser.parsePlan(v1).asInstanceOf[CreateViewCommand] - assert(command.name.database.isEmpty) - assert(command.name.table == "view1") - assert(command.userSpecifiedColumns == Seq("col1" -> None, "col3" -> Some("hello"))) - assert(command.originalText == Some("SELECT * FROM tab1")) - assert(command.properties == Map("prop1Key" -> "prop1Val")) - assert(command.comment == Some("BLABLA")) - } - - test("create view -- partitioned view") { - val v1 = "CREATE VIEW view1 partitioned on (ds, hr) as select * from srcpart" - intercept[ParseException] { - parser.parsePlan(v1) - } - } - - test("create view - duplicate clauses") { - def createViewStatement(duplicateClause: String): String = { - s""" - |CREATE OR REPLACE VIEW view1 - |(col1, col3 COMMENT 'hello') - |$duplicateClause - |$duplicateClause - |AS SELECT * FROM tab1 - """.stripMargin - } - val sql1 = createViewStatement("COMMENT 'BLABLA'") - val sql2 = createViewStatement("TBLPROPERTIES('prop1Key'=\"prop1Val\")") - intercept(sql1, "Found duplicate clauses: COMMENT") - intercept(sql2, "Found duplicate clauses: TBLPROPERTIES") - } - test("create table like") { val v1 = "CREATE TABLE table1 LIKE table2" val (target, source, provider, location, exists) = parser.parsePlan(v1).collect { From 9b9d130f158010402f37b3262fc3b1e5067c95a9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 26 Nov 2019 20:55:02 +0900 Subject: [PATCH 0727/1387] [SPARK-30030][BUILD][FOLLOWUP] Remove unused org.apache.commons.lang ### What changes were proposed in this pull request? This PR aims to remove the unused test dependency `commons-lang:commons-lang` from `core` module. ### Why are the changes needed? SPARK-30030 already removed all usage of `Apache Commons Lang2` in `core`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #26673 from dongjoon-hyun/SPARK-30030-2. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- core/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 04a3fe521ef87..f5a2c2c1c01a9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -163,11 +163,6 @@ javax.servlet-api ${javaxservlet.version} - - commons-lang - commons-lang - test - org.apache.commons commons-lang3 From c2d513f8e9f5d96dae7990325617013bcc92fe5d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 26 Nov 2019 21:31:02 +0900 Subject: [PATCH 0728/1387] [SPARK-30035][BUILD] Upgrade to Apache Commons Lang 3.9 ### What changes were proposed in this pull request? This PR aims to upgrade to `Apache Commons Lang 3.9`. ### Why are the changes needed? `Apache Commons Lang 3.9` is the first official release to support JDK9+. The following is the full release note. - https://commons.apache.org/proper/commons-lang/release-notes/RELEASE-NOTES-3.9.txt ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Closes #26672 from dongjoon-hyun/SPARK-30035. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 2 +- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 0821a27de8ee9..371270702c56e 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -44,7 +44,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.8.1.jar +commons-lang3-3.9.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-3.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index b953d65412167..a2b603b33c0e3 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -42,7 +42,7 @@ commons-digester-1.8.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.8.1.jar +commons-lang3-3.9.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-3.1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index c9461f6c7f085..b2c1777ee2ad0 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -39,7 +39,7 @@ commons-dbcp-1.4.jar commons-httpclient-3.1.jar commons-io-2.4.jar commons-lang-2.6.jar -commons-lang3-3.8.1.jar +commons-lang3-3.9.jar commons-logging-1.1.3.jar commons-math3-3.4.1.jar commons-net-3.1.jar diff --git a/pom.xml b/pom.xml index d49f645da947e..59c0c11703b1f 100644 --- a/pom.xml +++ b/pom.xml @@ -178,7 +178,7 @@ 2.6 - 3.8.1 + 3.9 2.6.2 4.1.17 From 7b1b60c7583faca70aeab2659f06d4e491efa5c0 Mon Sep 17 00:00:00 2001 From: wuyi Date: Tue, 26 Nov 2019 08:20:26 -0800 Subject: [PATCH 0729/1387] [SPARK-28574][CORE][FOLLOW-UP] Several minor improvements for event queue capacity config ### What changes were proposed in this pull request? * Replace hard-coded conf `spark.scheduler.listenerbus.eventqueue` with a constant variable(`LISTENER_BUS_EVENT_QUEUE_PREFIX `) defined in `config/package.scala`. * Update documentation for `spark.scheduler.listenerbus.eventqueue.capacity` in both `config/package.scala` and `docs/configuration.md`. ### Why are the changes needed? * Better code maintainability * Better user guidance of the conf ### Does this PR introduce any user-facing change? No behavior changes but user will see the updated document. ### How was this patch tested? Pass Jenkins. Closes #26676 from Ngone51/SPARK-28574-followup. Authored-by: wuyi Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/internal/config/package.scala | 5 +++++ .../org/apache/spark/scheduler/AsyncEventQueue.scala | 10 +++++----- docs/configuration.md | 8 +++++--- 3 files changed, 15 insertions(+), 8 deletions(-) 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 6a2eaf17d9f6c..543b8f67731e5 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 @@ -35,6 +35,7 @@ package object config { private[spark] val SPARK_DRIVER_PREFIX = "spark.driver" private[spark] val SPARK_EXECUTOR_PREFIX = "spark.executor" private[spark] val SPARK_TASK_PREFIX = "spark.task" + private[spark] val LISTENER_BUS_EVENT_QUEUE_PREFIX = "spark.scheduler.listenerbus.eventqueue" private[spark] val SPARK_RESOURCES_COORDINATE = ConfigBuilder("spark.resources.coordinate.enable") @@ -596,6 +597,10 @@ package object config { private[spark] val LISTENER_BUS_EVENT_QUEUE_CAPACITY = ConfigBuilder("spark.scheduler.listenerbus.eventqueue.capacity") + .doc("The default capacity for event queues. Spark will try to initialize " + + "an event queue using capacity specified by `spark.scheduler.listenerbus" + + ".eventqueue.queueName.capacity` first. If it's not configured, Spark will " + + "use the default capacity specified by this config.") .intConf .checkValue(_ > 0, "The capacity of listener bus event queue must be positive") .createWithDefault(10000) diff --git a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala index 11e2c475d9b45..1bcddaceb3576 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala @@ -50,11 +50,11 @@ private class AsyncEventQueue( // if no such conf is specified, use the value specified in // LISTENER_BUS_EVENT_QUEUE_CAPACITY private[scheduler] def capacity: Int = { - val queuesize = conf.getInt(s"spark.scheduler.listenerbus.eventqueue.${name}.capacity", - conf.get(LISTENER_BUS_EVENT_QUEUE_CAPACITY)) - assert(queuesize > 0, s"capacity for event queue $name must be greater than 0, " + - s"but $queuesize is configured.") - queuesize + val queueSize = conf.getInt(s"$LISTENER_BUS_EVENT_QUEUE_PREFIX.$name.capacity", + conf.get(LISTENER_BUS_EVENT_QUEUE_CAPACITY)) + assert(queueSize > 0, s"capacity for event queue $name must be greater than 0, " + + s"but $queueSize is configured.") + queueSize } private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](capacity) diff --git a/docs/configuration.md b/docs/configuration.md index 0c7cc6022eb09..8cd285cf83ca7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1852,9 +1852,11 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.listenerbus.eventqueue.capacity 10000 - Capacity for event queue in Spark listener bus, must be greater than 0. Consider increasing - value (e.g. 20000) if listener events are dropped. Increasing this value may result in the - driver using more memory. + The default capacity for event queues. Spark will try to initialize an event queue + using capacity specified by `spark.scheduler.listenerbus.eventqueue.queueName.capacity` + first. If it's not configured, Spark will use the default capacity specified by this + config. Note that capacity must be greater than 0. Consider increasing value (e.g. 20000) + if listener events are dropped. Increasing this value may result in the driver using more memory. From 29018025bacbf6c4cc74141211beb3e62b416633 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 26 Nov 2019 08:25:53 -0800 Subject: [PATCH 0730/1387] [SPARK-30009][CORE][SQL] Support different floating-point Ordering for Scala 2.12 / 2.13 ### What changes were proposed in this pull request? Make separate source trees for Scala 2.12/2.13 in order to accommodate mutually-incompatible support for Ordering of double, float. Note: This isn't the last change that will need a split source tree for 2.13. But this particular change could go several ways: - (Split source tree) - Inline the Scala 2.12 implementation - Reflection For this change alone any are possible, and splitting the source tree is a bit overkill. But if it will be necessary for other JIRAs (see umbrella SPARK-25075), then it might be the easiest way to implement this. ### Why are the changes needed? Scala 2.13 split Ordering.Double into Ordering.Double.TotalOrdering and Ordering.Double.IeeeOrdering. Neither can be used in a single build that supports 2.12 and 2.13. TotalOrdering works like java.lang.Double.compare. IeeeOrdering works like Scala 2.12 Ordering.Double. They differ in how NaN is handled - compares always above other values? or always compares as 'false'? In theory they have different uses: TotalOrdering is important if floating-point values are sorted. IeeeOrdering behaves like 2.12 and JVM comparison operators. I chose TotalOrdering as I think we care more about stable sorting, and because elsewhere we rely on java.lang comparisons. It is also possible to support with two methods. ### Does this PR introduce any user-facing change? Pending tests, will see if it obviously affects any sort order. We need to see if it changes NaN sort order. ### How was this patch tested? Existing tests so far. Closes #26654 from srowen/SPARK-30009. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- core/pom.xml | 27 +++++++++++++-- .../org/apache/spark/util/OrderingUtil.scala | 33 ++++++++++++++++++ .../org/apache/spark/util/OrderingUtil.scala | 34 +++++++++++++++++++ .../spark/util/collection/SorterSuite.scala | 3 +- .../org/apache/spark/sql/types/numerics.scala | 10 ++++-- 5 files changed, 100 insertions(+), 7 deletions(-) create mode 100644 core/src/main/scala-2.12/org/apache/spark/util/OrderingUtil.scala create mode 100644 core/src/main/scala-2.13/org/apache/spark/util/OrderingUtil.scala diff --git a/core/pom.xml b/core/pom.xml index f5a2c2c1c01a9..24d61bf7a1912 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -26,12 +26,15 @@ spark-core_2.12 - - core - jar Spark Project Core http://spark.apache.org/ + + + core + src/main/scala-${scala.binary.version} + + com.thoughtworks.paranamer @@ -515,6 +518,24 @@ + + org.codehaus.mojo + build-helper-maven-plugin + + + add-sources + generate-sources + + add-source + + + + ${extra.source.dir} + + + + + diff --git a/core/src/main/scala-2.12/org/apache/spark/util/OrderingUtil.scala b/core/src/main/scala-2.12/org/apache/spark/util/OrderingUtil.scala new file mode 100644 index 0000000000000..193ae36f3c089 --- /dev/null +++ b/core/src/main/scala-2.12/org/apache/spark/util/OrderingUtil.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * This class only exists to bridge the difference between Scala 2.12 and Scala 2.13's + * support for floating-point ordering. It is implemented separately for both as there + * is no method that exists in both for comparison. + * + * It functions like Ordering.Double in Scala 2.12. + */ +private[spark] object OrderingUtil { + + def compareDouble(x: Double, y: Double): Int = Ordering.Double.compare(x, y) + + def compareFloat(x: Float, y: Float): Int = Ordering.Float.compare(x, y) + +} diff --git a/core/src/main/scala-2.13/org/apache/spark/util/OrderingUtil.scala b/core/src/main/scala-2.13/org/apache/spark/util/OrderingUtil.scala new file mode 100644 index 0000000000000..e861506672706 --- /dev/null +++ b/core/src/main/scala-2.13/org/apache/spark/util/OrderingUtil.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * This class only exists to bridge the difference between Scala 2.12 and Scala 2.13's + * support for floating-point ordering. It is implemented separately for both as there + * is no method that exists in both for comparison. + * + * It functions like Ordering.Double.TotalOrdering in Scala 2.13, which matches java.lang.Double + * rather than Scala 2.12's Ordering.Double in handling of NaN. + */ +private[spark] object OrderingUtil { + + def compareDouble(x: Double, y: Double): Int = Ordering.Double.TotalOrdering.compare(x, y) + + def compareFloat(x: Float, y: Float): Int = Ordering.Float.TotalOrdering.compare(x, y) + +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index d1603b85a8e94..bd1921d7b770e 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging +import org.apache.spark.util.OrderingUtil import org.apache.spark.util.Utils.timeIt import org.apache.spark.util.random.XORShiftRandom @@ -59,7 +60,7 @@ class SorterSuite extends SparkFunSuite with Logging { Arrays.sort(keys) new Sorter(new KVArraySortDataFormat[Double, Number]) - .sort(keyValueArray, 0, keys.length, Ordering.Double) + .sort(keyValueArray, 0, keys.length, OrderingUtil.compareDouble) keys.zipWithIndex.foreach { case (k, i) => assert(k === keyValueArray(2 * i)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala index 518255ecc42cc..71fed3e69b651 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala @@ -21,7 +21,7 @@ import scala.math.Numeric._ import scala.math.Ordering import org.apache.spark.sql.types.Decimal.DecimalIsConflicted - +import org.apache.spark.util.OrderingUtil object ByteExactNumeric extends ByteIsIntegral with Ordering.ByteOrdering { private def checkOverflow(res: Int, x: Byte, y: Byte, op: String): Unit = { @@ -118,7 +118,7 @@ object LongExactNumeric extends LongIsIntegral with Ordering.LongOrdering { } } -object FloatExactNumeric extends FloatIsFractional with Ordering.FloatOrdering { +object FloatExactNumeric extends FloatIsFractional { private def overflowException(x: Float, dataType: String) = throw new ArithmeticException(s"Casting $x to $dataType causes overflow") @@ -148,9 +148,11 @@ object FloatExactNumeric extends FloatIsFractional with Ordering.FloatOrdering { overflowException(x, "int") } } + + override def compare(x: Float, y: Float): Int = OrderingUtil.compareFloat(x, y) } -object DoubleExactNumeric extends DoubleIsFractional with Ordering.DoubleOrdering { +object DoubleExactNumeric extends DoubleIsFractional { private def overflowException(x: Double, dataType: String) = throw new ArithmeticException(s"Casting $x to $dataType causes overflow") @@ -174,6 +176,8 @@ object DoubleExactNumeric extends DoubleIsFractional with Ordering.DoubleOrderin overflowException(x, "long") } } + + override def compare(x: Double, y: Double): Int = OrderingUtil.compareDouble(x, y) } object DecimalExactNumeric extends DecimalIsConflicted { From ed0c33fdd428a02831880532ab9c2e2de808bb52 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Nov 2019 01:20:38 +0800 Subject: [PATCH 0731/1387] [SPARK-30026][SQL] Whitespaces can be identified as delimiters in interval string ### What changes were proposed in this pull request? We are now able to handle whitespaces for integral and fractional types, and the leading or trailing whitespaces for interval, date, and timestamps. But the current interval parser is not able to identify whitespaces as separates as PostgreSQL can do. This PR makes the whitespaces handling be consistent for nterval values. Typed interval literal, multi-unit representation, and casting from strings are all supported. ```sql postgres=# select interval E'1 \t day'; interval ---------- 1 day (1 row) postgres=# select interval E'1\t' day; interval ---------- 1 day (1 row) ``` ### Why are the changes needed? Whitespace handling should be consistent for interval value, and across different types in Spark. PostgreSQL feature parity. ### Does this PR introduce any user-facing change? Yes, the interval string of multi-units values which separated by whitespaces can be valid now. ### How was this patch tested? add ut. Closes #26662 from yaooqinn/SPARK-30026. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/IntervalUtils.scala | 56 +++++----- .../catalyst/util/IntervalUtilsSuite.scala | 11 +- .../benchmarks/IntervalBenchmark-results.txt | 52 ++++----- .../resources/sql-tests/inputs/interval.sql | 4 + .../sql-tests/results/ansi/interval.sql.out | 104 ++++++++++++------ .../sql-tests/results/interval.sql.out | 34 +++++- 6 files changed, 170 insertions(+), 91 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index b8c7e4a9bac0c..6e33c5b24f478 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -425,7 +425,7 @@ object IntervalUtils { UNIT_SUFFIX, UNIT_END = Value } - private final val intervalStr = UTF8String.fromString("interval ") + private final val intervalStr = UTF8String.fromString("interval") private def unitToUtf8(unit: IntervalUnit): UTF8String = { UTF8String.fromString(unit.toString) } @@ -484,15 +484,17 @@ object IntervalUtils { var pointPrefixed: Boolean = false def trimToNextState(b: Byte, next: ParseState): Unit = { - b match { - case ' ' => i += 1 - case _ => state = next + if (b <= ' ') { + i += 1 + } else { + state = next } } - def currentWord: UTF8String = { - val strings = s.split(UTF8String.blankString(1), -1) - val lenRight = s.substring(i, s.numBytes()).split(UTF8String.blankString(1), -1).length + def currentWord: String = { + val sep = "\\s+" + val strings = s.toString.split(sep) + val lenRight = s.substring(i, s.numBytes()).toString.split(sep).length strings(strings.length - lenRight) } @@ -503,8 +505,10 @@ object IntervalUtils { if (s.startsWith(intervalStr)) { if (s.numBytes() == intervalStr.numBytes()) { throwIAE("interval string cannot be empty") + } else if (bytes(i + intervalStr.numBytes()) > ' ') { + throwIAE(s"invalid interval prefix $currentWord") } else { - i += intervalStr.numBytes() + i += intervalStr.numBytes() + 1 } } state = TRIM_BEFORE_SIGN @@ -548,7 +552,7 @@ object IntervalUtils { } catch { case e: ArithmeticException => throwIAE(e.getMessage, e) } - case ' ' => state = TRIM_BEFORE_UNIT + case _ if b <= ' ' => state = TRIM_BEFORE_UNIT case '.' => fractionScale = initialFractionScale state = VALUE_FRACTIONAL_PART @@ -556,17 +560,17 @@ object IntervalUtils { } i += 1 case VALUE_FRACTIONAL_PART => - b match { - case _ if '0' <= b && b <= '9' && fractionScale > 0 => - fraction += (b - '0') * fractionScale - fractionScale /= 10 - case ' ' if !pointPrefixed || fractionScale < initialFractionScale => - fraction /= NANOS_PER_MICROS.toInt - state = TRIM_BEFORE_UNIT - case _ if '0' <= b && b <= '9' => - throwIAE(s"interval can only support nanosecond precision, '$currentWord' is out" + - s" of range") - case _ => throwIAE(s"invalid value '$currentWord'") + if ('0' <= b && b <= '9' && fractionScale > 0) { + fraction += (b - '0') * fractionScale + fractionScale /= 10 + } else if (b <= ' ' && (!pointPrefixed || fractionScale < initialFractionScale)) { + fraction /= NANOS_PER_MICROS.toInt + state = TRIM_BEFORE_UNIT + } else if ('0' <= b && b <= '9') { + throwIAE(s"interval can only support nanosecond precision, '$currentWord' is out" + + s" of range") + } else { + throwIAE(s"invalid value '$currentWord'") } i += 1 case TRIM_BEFORE_UNIT => trimToNextState(b, UNIT_BEGIN) @@ -627,16 +631,16 @@ object IntervalUtils { case UNIT_SUFFIX => b match { case 's' => state = UNIT_END - case ' ' => state = TRIM_BEFORE_SIGN + case _ if b <= ' ' => state = TRIM_BEFORE_SIGN case _ => throwIAE(s"invalid unit '$currentWord'") } i += 1 case UNIT_END => - b match { - case ' ' => - i += 1 - state = TRIM_BEFORE_SIGN - case _ => throwIAE(s"invalid unit '$currentWord'") + if (b <= ' ') { + i += 1 + state = TRIM_BEFORE_SIGN + } else { + throwIAE(s"invalid unit '$currentWord'") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index ee3db0391ed00..c98bb61b6ee7b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -79,7 +79,7 @@ class IntervalUtilsSuite extends SparkFunSuite { checkFromInvalidString(null, "cannot be null") - for (input <- Seq("", " ", "interval", "interval1 day", "foo", "foo 1 day")) { + for (input <- Seq("", "interval", "foo", "foo 1 day")) { checkFromInvalidString(input, "Error parsing") } } @@ -113,9 +113,16 @@ class IntervalUtilsSuite extends SparkFunSuite { checkFromInvalidString("1a1 hour", "invalid value '1a1'") checkFromInvalidString("1.1a1 seconds", "invalid value '1.1a1'") checkFromInvalidString("2234567890 days", "integer overflow") + checkFromInvalidString(". seconds", "invalid value '.'") + } + + test("string to interval: whitespaces") { + checkFromInvalidString(" ", "Error parsing ' ' to interval") checkFromInvalidString("\n", "Error parsing '\n' to interval") checkFromInvalidString("\t", "Error parsing '\t' to interval") - checkFromInvalidString(". seconds", "invalid value '.'") + checkFromString("1 \t day \n 2 \r hour", new CalendarInterval(0, 1, 2 * MICROS_PER_HOUR)) + checkFromInvalidString("interval1 \t day \n 2 \r hour", "invalid interval prefix interval1") + checkFromString("interval\r1\tday", new CalendarInterval(0, 1, 0)) } test("string to interval: seconds with fractional part") { diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt index 6ae5a4bd09f6d..8976fb590d8dd 100644 --- a/sql/core/benchmarks/IntervalBenchmark-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -1,29 +1,29 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.14.6 -Intel(R) Core(TM) i5-5287U CPU @ 2.90GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1 +Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare string w/ interval 531 566 34 1.9 530.5 1.0X -prepare string w/o interval 466 479 21 2.1 466.5 1.1X -1 units w/ interval 475 521 63 2.1 475.0 1.1X -1 units w/o interval 440 457 25 2.3 440.1 1.2X -2 units w/ interval 614 621 11 1.6 613.7 0.9X -2 units w/o interval 596 605 8 1.7 596.5 0.9X -3 units w/ interval 1115 1120 4 0.9 1115.0 0.5X -3 units w/o interval 1100 1107 6 0.9 1100.2 0.5X -4 units w/ interval 1255 1263 9 0.8 1255.1 0.4X -4 units w/o interval 1254 1393 130 0.8 1253.8 0.4X -5 units w/ interval 1367 1373 5 0.7 1367.2 0.4X -5 units w/o interval 1366 1376 9 0.7 1366.2 0.4X -6 units w/ interval 1526 1530 6 0.7 1526.0 0.3X -6 units w/o interval 1504 1510 7 0.7 1504.0 0.4X -7 units w/ interval 1748 1778 27 0.6 1748.0 0.3X -7 units w/o interval 1740 1744 5 0.6 1740.0 0.3X -8 units w/ interval 2092 2107 14 0.5 2092.5 0.3X -8 units w/o interval 2094 2098 5 0.5 2094.4 0.3X -9 units w/ interval 1874 1880 5 0.5 1873.9 0.3X -9 units w/o interval 1867 1872 4 0.5 1867.3 0.3X -10 units w/ interval 2127 2134 13 0.5 2126.5 0.2X -10 units w/o interval 2045 2049 6 0.5 2045.0 0.3X -11 units w/ interval 2242 2254 13 0.4 2241.9 0.2X -11 units w/o interval 2221 2227 6 0.5 2221.1 0.2X +prepare string w/ interval 357 370 22 2.8 357.0 1.0X +prepare string w/o interval 315 333 22 3.2 314.7 1.1X +1 units w/ interval 356 380 21 2.8 355.8 1.0X +1 units w/o interval 317 326 12 3.2 317.1 1.1X +2 units w/ interval 481 488 8 2.1 480.8 0.7X +2 units w/o interval 456 464 9 2.2 456.0 0.8X +3 units w/ interval 1074 1080 5 0.9 1073.7 0.3X +3 units w/o interval 1025 1027 2 1.0 1025.4 0.3X +4 units w/ interval 1192 1196 5 0.8 1192.2 0.3X +4 units w/o interval 1219 1233 14 0.8 1218.9 0.3X +5 units w/ interval 1367 1382 23 0.7 1367.3 0.3X +5 units w/o interval 1295 1301 7 0.8 1295.1 0.3X +6 units w/ interval 1489 1525 31 0.7 1489.3 0.2X +6 units w/o interval 1496 1500 6 0.7 1495.8 0.2X +7 units w/ interval 1326 1330 4 0.8 1325.5 0.3X +7 units w/o interval 1324 1332 11 0.8 1324.1 0.3X +8 units w/ interval 1535 1547 11 0.7 1535.4 0.2X +8 units w/o interval 1542 1547 5 0.6 1542.3 0.2X +9 units w/ interval 1623 1641 18 0.6 1623.0 0.2X +9 units w/o interval 1615 1619 3 0.6 1615.3 0.2X +10 units w/ interval 1845 1861 16 0.5 1844.7 0.2X +10 units w/o interval 1858 1868 9 0.5 1857.8 0.2X +11 units w/ interval 1919 1925 11 0.5 1918.7 0.2X +11 units w/o interval 1973 1995 23 0.5 1972.8 0.2X diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index 886e242460547..5806d9288572f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -260,3 +260,7 @@ from interval_arithmetic; -- control characters as white spaces select interval '\t interval 1 day'; +select interval 'interval \t 1\tday'; +select interval 'interval\t1\tday'; +select interval '1\t' day; +select interval '1 ' day; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 56628f615c3dc..a2d03064afdd4 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 131 +-- Number of queries: 135 -- !query 0 @@ -1124,34 +1124,66 @@ struct -- !query 119 -select 1 year 2 days +select interval 'interval \t 1\tday' -- !query 119 schema -struct +struct -- !query 119 output -1 years 2 days +1 days -- !query 120 -select '10-9' year to month +select interval 'interval\t1\tday' -- !query 120 schema -struct +struct -- !query 120 output -10 years 9 months +1 days -- !query 121 -select '20 15:40:32.99899999' day to second +select interval '1\t' day -- !query 121 schema -struct +struct -- !query 121 output -20 days 15 hours 40 minutes 32.998999 seconds +1 days -- !query 122 -select 30 day day +select interval '1 ' day -- !query 122 schema -struct<> +struct -- !query 122 output +1 days + + +-- !query 123 +select 1 year 2 days +-- !query 123 schema +struct +-- !query 123 output +1 years 2 days + + +-- !query 124 +select '10-9' year to month +-- !query 124 schema +struct +-- !query 124 output +10 years 9 months + + +-- !query 125 +select '20 15:40:32.99899999' day to second +-- !query 125 schema +struct +-- !query 125 output +20 days 15 hours 40 minutes 32.998999 seconds + + +-- !query 126 +select 30 day day +-- !query 126 schema +struct<> +-- !query 126 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 14) @@ -1161,27 +1193,27 @@ select 30 day day --------------^^^ --- !query 123 +-- !query 127 select date'2012-01-01' - '2-2' year to month --- !query 123 schema +-- !query 127 schema struct --- !query 123 output +-- !query 127 output 2009-11-01 --- !query 124 +-- !query 128 select 1 month - 1 day --- !query 124 schema +-- !query 128 schema struct --- !query 124 output +-- !query 128 output 1 months -1 days --- !query 125 +-- !query 129 select 1 year to month --- !query 125 schema +-- !query 129 schema struct<> --- !query 125 output +-- !query 129 output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 7) @@ -1191,11 +1223,11 @@ select 1 year to month -------^^^ --- !query 126 +-- !query 130 select '1' year to second --- !query 126 schema +-- !query 130 schema struct<> --- !query 126 output +-- !query 130 output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 7) @@ -1205,11 +1237,11 @@ select '1' year to second -------^^^ --- !query 127 +-- !query 131 select 1 year '2-1' year to month --- !query 127 schema +-- !query 131 schema struct<> --- !query 127 output +-- !query 131 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 14) @@ -1219,11 +1251,11 @@ select 1 year '2-1' year to month --------------^^^ --- !query 128 +-- !query 132 select (-30) day --- !query 128 schema +-- !query 132 schema struct<> --- !query 128 output +-- !query 132 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 13) @@ -1233,11 +1265,11 @@ select (-30) day -------------^^^ --- !query 129 +-- !query 133 select (a + 1) day --- !query 129 schema +-- !query 133 schema struct<> --- !query 129 output +-- !query 133 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 15) @@ -1247,11 +1279,11 @@ select (a + 1) day ---------------^^^ --- !query 130 +-- !query 134 select 30 day day day --- !query 130 schema +-- !query 134 schema struct<> --- !query 130 output +-- !query 134 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 14) diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index bf7b2184b68fc..69a23bb9f1833 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 119 +-- Number of queries: 123 -- !query 0 @@ -1105,3 +1105,35 @@ select interval '\t interval 1 day' struct -- !query 118 output 1 days + + +-- !query 119 +select interval 'interval \t 1\tday' +-- !query 119 schema +struct +-- !query 119 output +1 days + + +-- !query 120 +select interval 'interval\t1\tday' +-- !query 120 schema +struct +-- !query 120 output +1 days + + +-- !query 121 +select interval '1\t' day +-- !query 121 schema +struct +-- !query 121 output +1 days + + +-- !query 122 +select interval '1 ' day +-- !query 122 schema +struct +-- !query 122 output +1 days From e23c135e568d4401a5659bc1b5ae8fc8bf147693 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 26 Nov 2019 09:59:19 -0800 Subject: [PATCH 0732/1387] [SPARK-29293][BUILD] Move scalafmt to Scala 2.12 profile; bump to 0.12 ### What changes were proposed in this pull request? Move scalafmt to Scala 2.12 profile; bump to 0.12. ### Why are the changes needed? To facilitate a future Scala 2.13 build. ### Does this PR introduce any user-facing change? None. ### How was this patch tested? This isn't covered by tests, it's a convenience for contributors. Closes #26655 from srowen/SPARK-29293. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- dev/scalafmt | 2 +- pom.xml | 46 +++++++++++++++++++++++++++------------------- 2 files changed, 28 insertions(+), 20 deletions(-) diff --git a/dev/scalafmt b/dev/scalafmt index 76f688a2f5b88..0541a4058467b 100755 --- a/dev/scalafmt +++ b/dev/scalafmt @@ -20,4 +20,4 @@ # by default, format only files that differ from git master params="${@:---diff}" -./build/mvn mvn-scalafmt_2.12:format -Dscalafmt.skip=false -Dscalafmt.parameters="$params" +./build/mvn -Pscala-2.12 mvn-scalafmt_2.12:format -Dscalafmt.skip=false -Dscalafmt.parameters="$params" \ No newline at end of file diff --git a/pom.xml b/pom.xml index 59c0c11703b1f..94214a53c3acb 100644 --- a/pom.xml +++ b/pom.xml @@ -2843,24 +2843,6 @@ - - org.antipathy - mvn-scalafmt_2.12 - 0.9_1.5.1 - - ${scalafmt.parameters} - ${scalafmt.skip} - dev/.scalafmt.conf - - - - validate - - format - - - - scala-2.12 + + + + + + org.antipathy + mvn-scalafmt_2.12 + 0.12_1.5.1 + + ${scalafmt.parameters} + ${scalafmt.skip} + ${scalafmt.skip} + ${scalafmt.skip} + dev/.scalafmt.conf + + + + validate + + format + + + + + + + From fd2bf55abaab08798a428d4e47d4050ba2b82a95 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 26 Nov 2019 11:02:25 -0800 Subject: [PATCH 0733/1387] [SPARK-27651][CORE] Avoid the network when shuffle blocks are fetched from the same host MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Before this PR `ShuffleBlockFetcherIterator` was partitioning the block fetches into two distinct sets: local reads and remote fetches. Within this PR (when the feature is enabled by "spark.shuffle.readHostLocalDisk.enabled") a new category is introduced: host-local reads. They are shuffle block fetches where although the block manager is different they are running on the same host along with the requester. Moreover to get the local directories of the other executors/block managers a new RPC message is introduced `GetLocalDirs` which is sent the the block manager master where it is answered as `BlockManagerLocalDirs`. In `BlockManagerMasterEndpoint` for answering this request the `localDirs` is extracted from the `BlockManagerInfo` and stored separately in a hash map called `executorIdLocalDirs`. Because the earlier used `blockManagerInfo` contains data for the alive block managers (see `org.apache.spark.storage.BlockManagerMasterEndpoint#removeBlockManager`). Now `executorIdLocalDirs` knows all the local dirs up to the application start (like the external shuffle service does) so in case of an RDD recalculation both host-local shuffle blocks and disk persisted RDD blocks on the same host can be served by reading the files behind the blocks directly. ## How was this patch tested? ### Unit tests `ExternalShuffleServiceSuite`: - "SPARK-27651: host local disk reading avoids external shuffle service on the same node" `ShuffleBlockFetcherIteratorSuite`: - "successful 3 local reads + 4 host local reads + 2 remote reads" And with extending existing suites where shuffle metrics was tested. ### Manual tests Running Spark on YARN in a 4 nodes cluster with 6 executors and having 12 shuffle blocks. ``` $ grep host-local experiment.log 19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks 19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_2_1, shuffle_0_6_1 19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 38 ms 19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks 19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_0_0, shuffle_0_8_0 19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 35 ms ``` Closes #25299 from attilapiros/SPARK-27651. Authored-by: “attilapiros” Signed-off-by: Marcelo Vanzin --- .../network/shuffle/ExternalBlockHandler.java | 6 + .../shuffle/ExternalBlockStoreClient.java | 49 +++- .../shuffle/ExternalShuffleBlockResolver.java | 15 ++ .../protocol/BlockTransferMessage.java | 4 +- .../shuffle/protocol/BlocksRemoved.java | 2 +- .../shuffle/protocol/ExecutorShuffleInfo.java | 2 +- .../protocol/GetLocalDirsForExecutors.java | 81 ++++++ .../protocol/LocalDirsForExecutors.java | 117 +++++++++ .../shuffle/BlockTransferMessagesSuite.java | 24 +- .../scala/org/apache/spark/SparkContext.scala | 8 + .../spark/internal/config/package.scala | 18 ++ .../spark/network/BlockDataManager.scala | 12 +- .../network/netty/NettyBlockRpcServer.scala | 6 +- .../shuffle/IndexShuffleBlockResolver.scala | 41 ++- .../spark/shuffle/ShuffleBlockResolver.scala | 10 +- .../apache/spark/storage/BlockManager.scala | 70 ++++- .../storage/BlockManagerMasterEndpoint.scala | 26 +- .../storage/ShuffleBlockFetcherIterator.scala | 248 +++++++++++++----- .../spark/ExternalShuffleServiceSuite.scala | 49 +++- .../NettyBlockTransferSecuritySuite.scala | 2 +- .../BlockStoreShuffleReaderSuite.scala | 3 +- .../spark/storage/BlockManagerInfoSuite.scala | 1 - .../spark/storage/BlockManagerSuite.scala | 4 +- .../ShuffleBlockFetcherIteratorSuite.scala | 220 ++++++++++++---- 24 files changed, 856 insertions(+), 162 deletions(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index b886fce9be21a..8c05288fb4111 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -150,6 +150,12 @@ protected void handleMessage( int numRemovedBlocks = blockManager.removeBlocks(msg.appId, msg.execId, msg.blockIds); callback.onSuccess(new BlocksRemoved(numRemovedBlocks).toByteBuffer()); + } else if (msgObj instanceof GetLocalDirsForExecutors) { + GetLocalDirsForExecutors msg = (GetLocalDirsForExecutors) msgObj; + checkAuth(client, msg.appId); + Map localDirs = blockManager.getLocalDirs(msg.appId, msg.execIds); + callback.onSuccess(new LocalDirsForExecutors(localDirs).toByteBuffer()); + } else { throw new UnsupportedOperationException("Unexpected message: " + msgObj); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java index 85d278138c2b4..d6185f089d3c0 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java @@ -21,20 +21,21 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import com.codahale.metrics.MetricSet; import com.google.common.collect.Lists; import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.client.TransportClientFactory; import org.apache.spark.network.shuffle.protocol.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.TransportContext; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportClientBootstrap; -import org.apache.spark.network.client.TransportClientFactory; import org.apache.spark.network.crypto.AuthClientBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.NoOpRpcHandler; @@ -182,7 +183,7 @@ public void onSuccess(ByteBuffer response) { @Override public void onFailure(Throwable e) { logger.warn("Error trying to remove RDD blocks " + Arrays.toString(blockIds) + - " via external shuffle service from executor: " + execId, e); + " via external shuffle service from executor: " + execId, e); numRemovedBlocksFuture.complete(0); client.close(); } @@ -190,6 +191,46 @@ public void onFailure(Throwable e) { return numRemovedBlocksFuture; } + public void getHostLocalDirs( + String host, + int port, + String[] execIds, + CompletableFuture> hostLocalDirsCompletable) { + checkInit(); + GetLocalDirsForExecutors getLocalDirsMessage = new GetLocalDirsForExecutors(appId, execIds); + try { + TransportClient client = clientFactory.createClient(host, port); + client.sendRpc(getLocalDirsMessage.toByteBuffer(), new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + try { + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response); + hostLocalDirsCompletable.complete( + ((LocalDirsForExecutors) msgObj).getLocalDirsByExec()); + } catch (Throwable t) { + logger.warn("Error trying to get the host local dirs for " + + Arrays.toString(getLocalDirsMessage.execIds) + " via external shuffle service", + t.getCause()); + hostLocalDirsCompletable.completeExceptionally(t); + } finally { + client.close(); + } + } + + @Override + public void onFailure(Throwable t) { + logger.warn("Error trying to get the host local dirs for " + + Arrays.toString(getLocalDirsMessage.execIds) + " via external shuffle service", + t.getCause()); + hostLocalDirsCompletable.completeExceptionally(t); + client.close(); + } + }); + } catch (IOException | InterruptedException e) { + hostLocalDirsCompletable.completeExceptionally(e); + } + } + @Override public void close() { checkInit(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index beca5d6e5a788..657774c1b468f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -25,7 +25,9 @@ import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; @@ -369,6 +371,19 @@ public int removeBlocks(String appId, String execId, String[] blockIds) { return numRemovedBlocks; } + public Map getLocalDirs(String appId, String[] execIds) { + return Arrays.stream(execIds) + .map(exec -> { + ExecutorShuffleInfo info = executors.get(new AppExecId(appId, exec)); + if (info == null) { + throw new RuntimeException( + String.format("Executor is not registered (appId=%s, execId=%s)", appId, exec)); + } + return Pair.of(exec, info.localDirs); + }) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + } + /** Simply encodes an executor's full ID, which is appId + execId. */ public static class AppExecId { public final String appId; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index 41dd55847ebdb..89d8dfe8716b8 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -47,7 +47,7 @@ public abstract class BlockTransferMessage implements Encodable { public enum Type { OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4), HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6), REMOVE_BLOCKS(7), BLOCKS_REMOVED(8), - FETCH_SHUFFLE_BLOCKS(9); + FETCH_SHUFFLE_BLOCKS(9), GET_LOCAL_DIRS_FOR_EXECUTORS(10), LOCAL_DIRS_FOR_EXECUTORS(11); private final byte id; @@ -76,6 +76,8 @@ public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { case 7: return RemoveBlocks.decode(buf); case 8: return BlocksRemoved.decode(buf); case 9: return FetchShuffleBlocks.decode(buf); + case 10: return GetLocalDirsForExecutors.decode(buf); + case 11: return LocalDirsForExecutors.decode(buf); default: throw new IllegalArgumentException("Unknown message type: " + type); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java index 3f04443871b68..723b2f75c6fc7 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java @@ -50,7 +50,7 @@ public String toString() { public boolean equals(Object other) { if (other != null && other instanceof BlocksRemoved) { BlocksRemoved o = (BlocksRemoved) other; - return Objects.equal(numRemovedBlocks, o.numRemovedBlocks); + return numRemovedBlocks == o.numRemovedBlocks; } return false; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java index 93758bdc58fb0..540ecd09a7e30 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java @@ -65,7 +65,7 @@ public boolean equals(Object other) { if (other != null && other instanceof ExecutorShuffleInfo) { ExecutorShuffleInfo o = (ExecutorShuffleInfo) other; return Arrays.equals(localDirs, o.localDirs) - && Objects.equal(subDirsPerLocalDir, o.subDirsPerLocalDir) + && subDirsPerLocalDir == o.subDirsPerLocalDir && Objects.equal(shuffleManager, o.shuffleManager); } return false; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java new file mode 100644 index 0000000000000..90c416acc69ad --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** Request to get the local dirs for the given executors. */ +public class GetLocalDirsForExecutors extends BlockTransferMessage { + public final String appId; + public final String[] execIds; + + public GetLocalDirsForExecutors(String appId, String[] execIds) { + this.appId = appId; + this.execIds = execIds; + } + + @Override + protected Type type() { return Type.GET_LOCAL_DIRS_FOR_EXECUTORS; } + + @Override + public int hashCode() { + return Objects.hashCode(appId) * 41 + Arrays.hashCode(execIds); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execIds", Arrays.toString(execIds)) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof GetLocalDirsForExecutors) { + GetLocalDirsForExecutors o = (GetLocalDirsForExecutors) other; + return appId.equals(o.appId) && Arrays.equals(execIds, o.execIds); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + Encoders.StringArrays.encodedLength(execIds); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.StringArrays.encode(buf, execIds); + } + + public static GetLocalDirsForExecutors decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String[] execIds = Encoders.StringArrays.decode(buf); + return new GetLocalDirsForExecutors(appId, execIds); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java new file mode 100644 index 0000000000000..0c3aa6a461148 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java @@ -0,0 +1,117 @@ +/* + * 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.network.shuffle.protocol; + +import java.util.*; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** The reply to get local dirs giving back the dirs for each of the requested executors. */ +public class LocalDirsForExecutors extends BlockTransferMessage { + private final String[] execIds; + private final int[] numLocalDirsByExec; + private final String[] allLocalDirs; + + public LocalDirsForExecutors(Map localDirsByExec) { + this.execIds = new String[localDirsByExec.size()]; + this.numLocalDirsByExec = new int[localDirsByExec.size()]; + ArrayList localDirs = new ArrayList<>(); + int index = 0; + for (Map.Entry e: localDirsByExec.entrySet()) { + execIds[index] = e.getKey(); + numLocalDirsByExec[index] = e.getValue().length; + Collections.addAll(localDirs, e.getValue()); + index++; + } + this.allLocalDirs = localDirs.toArray(new String[0]); + } + + private LocalDirsForExecutors(String[] execIds, int[] numLocalDirsByExec, String[] allLocalDirs) { + this.execIds = execIds; + this.numLocalDirsByExec = numLocalDirsByExec; + this.allLocalDirs = allLocalDirs; + } + + @Override + protected Type type() { return Type.LOCAL_DIRS_FOR_EXECUTORS; } + + @Override + public int hashCode() { + return Arrays.hashCode(execIds); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("execIds", Arrays.toString(execIds)) + .add("numLocalDirsByExec", Arrays.toString(numLocalDirsByExec)) + .add("allLocalDirs", Arrays.toString(allLocalDirs)) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof LocalDirsForExecutors) { + LocalDirsForExecutors o = (LocalDirsForExecutors) other; + return Arrays.equals(execIds, o.execIds) + && Arrays.equals(numLocalDirsByExec, o.numLocalDirsByExec) + && Arrays.equals(allLocalDirs, o.allLocalDirs); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.StringArrays.encodedLength(execIds) + + Encoders.IntArrays.encodedLength(numLocalDirsByExec) + + Encoders.StringArrays.encodedLength(allLocalDirs); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.StringArrays.encode(buf, execIds); + Encoders.IntArrays.encode(buf, numLocalDirsByExec); + Encoders.StringArrays.encode(buf, allLocalDirs); + } + + public static LocalDirsForExecutors decode(ByteBuf buf) { + String[] execIds = Encoders.StringArrays.decode(buf); + int[] numLocalDirsByExec = Encoders.IntArrays.decode(buf); + String[] allLocalDirs = Encoders.StringArrays.decode(buf); + return new LocalDirsForExecutors(execIds, numLocalDirsByExec, allLocalDirs); + } + + public Map getLocalDirsByExec() { + Map localDirsByExec = new HashMap<>(); + int index = 0; + int localDirsIndex = 0; + for (int length: numLocalDirsByExec) { + localDirsByExec.put(execIds[index], + Arrays.copyOfRange(allLocalDirs, localDirsIndex, localDirsIndex + length)); + localDirsIndex += length; + index++; + } + return localDirsByExec; + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java index fd2c67a3a270a..67229371c3a4a 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java @@ -21,6 +21,9 @@ import static org.junit.Assert.*; +import java.util.HashMap; +import java.util.Map; + import org.apache.spark.network.shuffle.protocol.*; /** Verifies that all BlockTransferMessages can be serialized correctly. */ @@ -41,10 +44,29 @@ public void serializeOpenShuffleBlocks() { checkSerializeDeserialize(new StreamHandle(12345, 16)); } - private void checkSerializeDeserialize(BlockTransferMessage msg) { + @Test + public void testLocalDirsMessages() { + checkSerializeDeserialize( + new GetLocalDirsForExecutors("app-1", new String[]{"exec-1", "exec-2"})); + + Map map = new HashMap<>(); + map.put("exec-1", new String[]{"loc1.1"}); + map.put("exec-22", new String[]{"loc2.1", "loc2.2"}); + LocalDirsForExecutors localDirsForExecs = new LocalDirsForExecutors(map); + Map resultMap = + ((LocalDirsForExecutors)checkSerializeDeserialize(localDirsForExecs)).getLocalDirsByExec(); + assertEquals(resultMap.size(), map.keySet().size()); + for (Map.Entry e: map.entrySet()) { + assertTrue(resultMap.containsKey(e.getKey())); + assertArrayEquals(e.getValue(), resultMap.get(e.getKey())); + } + } + + private BlockTransferMessage checkSerializeDeserialize(BlockTransferMessage msg) { BlockTransferMessage msg2 = BlockTransferMessage.Decoder.fromByteBuffer(msg.toByteBuffer()); assertEquals(msg, msg2); assertEquals(msg.hashCode(), msg2.hashCode()); assertEquals(msg.toString(), msg2.toString()); + return msg2; } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fc9d1f04cd7b5..069450120d5a7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2884,6 +2884,14 @@ object SparkContext extends Logging { memoryPerSlaveInt, sc.executorMemory)) } + // For host local mode setting the default of SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED + // to false because this mode is intended to be used for testing and in this case all the + // executors are running on the same host. So if host local reading was enabled here then + // testing of the remote fetching would be secondary as setting this config explicitly to + // false would be required in most of the unit test (despite the fact that remote fetching + // is much more frequent in production). + sc.conf.setIfMissing(SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED, false) + val scheduler = new TaskSchedulerImpl(sc) val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt, sc.conf) 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 543b8f67731e5..0f402e0bf2bb8 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 @@ -1089,6 +1089,24 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED = + ConfigBuilder("spark.shuffle.readHostLocalDisk.enabled") + .doc("If enabled, shuffle blocks requested from those block managers which are running on " + + "the same host are read from the disk directly instead of being fetched as remote blocks " + + "over the network.") + .booleanConf + .createWithDefault(true) + + private[spark] val STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE = + ConfigBuilder("spark.storage.localDiskByExecutors.cacheSize") + .doc("The max number of executors for which the local dirs are stored. This size is " + + "both applied for the driver and both for the executors side to avoid having an " + + "unbounded store. This cache will be used to avoid the network in case of fetching disk " + + "persisted RDD blocks or shuffle blocks (when `spark.shuffle.readHostLocalDisk.enabled` " + + "is set) from the same host.") + .intConf + .createWithDefault(1000) + private[spark] val SHUFFLE_SYNC = ConfigBuilder("spark.shuffle.sync") .doc("Whether to force outstanding writes to disk.") diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index 4993519aa3843..0bd5774b632bf 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -22,16 +22,22 @@ import scala.reflect.ClassTag import org.apache.spark.TaskContext import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.StreamCallbackWithID -import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, ShuffleBlockId, StorageLevel} private[spark] trait BlockDataManager { + /** + * Interface to get host-local shuffle block data. Throws an exception if the block cannot be + * found or cannot be read successfully. + */ + def getHostLocalShuffleData(blockId: BlockId, dirs: Array[String]): ManagedBuffer + /** * Interface to get local block data. Throws an exception if the block cannot be found or * cannot be read successfully. */ - def getBlockData(blockId: BlockId): ManagedBuffer + def getLocalBlockData(blockId: BlockId): ManagedBuffer /** * Put the block locally, using the given storage level. @@ -57,7 +63,7 @@ trait BlockDataManager { classTag: ClassTag[_]): StreamCallbackWithID /** - * Release locks acquired by [[putBlockData()]] and [[getBlockData()]]. + * Release locks acquired by [[putBlockData()]] and [[getLocalBlockData()]]. */ def releaseLock(blockId: BlockId, taskContext: Option[TaskContext]): Unit } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 3a41c5f73c0ac..91910b936e7cf 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -57,7 +57,7 @@ class NettyBlockRpcServer( case openBlocks: OpenBlocks => val blocksNum = openBlocks.blockIds.length val blocks = for (i <- (0 until blocksNum).view) - yield blockManager.getBlockData(BlockId.apply(openBlocks.blockIds(i))) + yield blockManager.getLocalBlockData(BlockId.apply(openBlocks.blockIds(i))) val streamId = streamManager.registerStream(appId, blocks.iterator.asJava, client.getChannel) logTrace(s"Registered streamId $streamId with $blocksNum buffers") @@ -67,7 +67,7 @@ class NettyBlockRpcServer( val blocks = fetchShuffleBlocks.mapIds.zipWithIndex.flatMap { case (mapId, index) => if (!fetchShuffleBlocks.batchFetchEnabled) { fetchShuffleBlocks.reduceIds(index).map { reduceId => - blockManager.getBlockData( + blockManager.getLocalBlockData( ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapId, reduceId)) } } else { @@ -76,7 +76,7 @@ class NettyBlockRpcServer( throw new IllegalStateException(s"Invalid shuffle fetch request when batch mode " + s"is enabled: $fetchShuffleBlocks") } - Array(blockManager.getBlockData( + Array(blockManager.getLocalBlockData( ShuffleBlockBatchId( fetchShuffleBlocks.shuffleId, mapId, startAndEndId(0), startAndEndId(1)))) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 8b3993e21f077..af2c82e771970 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -26,6 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.ExecutorDiskUtils import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -51,12 +52,36 @@ private[spark] class IndexShuffleBlockResolver( private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - def getDataFile(shuffleId: Int, mapId: Long): File = { - blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) + + def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, mapId, None) + + /** + * Get the shuffle data file. + * + * When the dirs parameter is None then use the disk manager's local directories. Otherwise, + * read from the specified directories. + */ + def getDataFile(shuffleId: Int, mapId: Long, dirs: Option[Array[String]]): File = { + val blockId = ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + dirs + .map(ExecutorDiskUtils.getFile(_, blockManager.subDirsPerLocalDir, blockId.name)) + .getOrElse(blockManager.diskBlockManager.getFile(blockId)) } - private def getIndexFile(shuffleId: Int, mapId: Long): File = { - blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) + /** + * Get the shuffle index file. + * + * When the dirs parameter is None then use the disk manager's local directories. Otherwise, + * read from the specified directories. + */ + private def getIndexFile( + shuffleId: Int, + mapId: Long, + dirs: Option[Array[String]] = None): File = { + val blockId = ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID) + dirs + .map(ExecutorDiskUtils.getFile(_, blockManager.subDirsPerLocalDir, blockId.name)) + .getOrElse(blockManager.diskBlockManager.getFile(blockId)) } /** @@ -190,7 +215,9 @@ private[spark] class IndexShuffleBlockResolver( } } - override def getBlockData(blockId: BlockId): ManagedBuffer = { + override def getBlockData( + blockId: BlockId, + dirs: Option[Array[String]]): ManagedBuffer = { val (shuffleId, mapId, startReduceId, endReduceId) = blockId match { case id: ShuffleBlockId => (id.shuffleId, id.mapId, id.reduceId, id.reduceId + 1) @@ -201,7 +228,7 @@ private[spark] class IndexShuffleBlockResolver( } // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index - val indexFile = getIndexFile(shuffleId, mapId) + val indexFile = getIndexFile(shuffleId, mapId, dirs) // SPARK-22982: if this FileInputStream's position is seeked forward by another piece of code // which is incorrectly using our file descriptor then this code will fetch the wrong offsets @@ -224,7 +251,7 @@ private[spark] class IndexShuffleBlockResolver( } new FileSegmentManagedBuffer( transportConf, - getDataFile(shuffleId, mapId), + getDataFile(shuffleId, mapId, dirs), startOffset, endOffset - startOffset) } finally { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala index c50789658d61a..5485cf955f11a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala @@ -31,10 +31,14 @@ trait ShuffleBlockResolver { type ShuffleId = Int /** - * Retrieve the data for the specified block. If the data for that block is not available, - * throws an unspecified exception. + * Retrieve the data for the specified block. + * + * When the dirs parameter is None then use the disk manager's local directories. Otherwise, + * read from the specified directories. + * + * If the data for that block is not available, throws an unspecified exception. */ - def getBlockData(blockId: BlockId): ManagedBuffer + def getBlockData(blockId: BlockId, dirs: Option[Array[String]] = None): ManagedBuffer def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c869a7078a1e9..cc28f9b77da30 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -22,17 +22,18 @@ import java.lang.ref.{ReferenceQueue => JReferenceQueue, WeakReference} import java.nio.ByteBuffer import java.nio.channels.Channels import java.util.Collections -import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.concurrent.{CompletableFuture, ConcurrentHashMap, TimeUnit} import scala.collection.mutable import scala.collection.mutable.HashMap import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ import scala.reflect.ClassTag -import scala.util.Random +import scala.util.{Failure, Random, Success, Try} import scala.util.control.NonFatal import com.codahale.metrics.{MetricRegistry, MetricSet} +import com.google.common.cache.CacheBuilder import org.apache.commons.io.IOUtils import org.apache.spark._ @@ -113,6 +114,47 @@ private[spark] class ByteBufferBlockData( } +private[spark] class HostLocalDirManager( + futureExecutionContext: ExecutionContext, + cacheSize: Int, + externalBlockStoreClient: ExternalBlockStoreClient, + host: String, + externalShuffleServicePort: Int) extends Logging { + + private val executorIdToLocalDirsCache = + CacheBuilder + .newBuilder() + .maximumSize(cacheSize) + .build[String, Array[String]]() + + private[spark] def getCachedHostLocalDirs() + : scala.collection.Map[String, Array[String]] = executorIdToLocalDirsCache.synchronized { + import scala.collection.JavaConverters._ + return executorIdToLocalDirsCache.asMap().asScala + } + + private[spark] def getHostLocalDirs( + executorIds: Array[String])( + callback: Try[java.util.Map[String, Array[String]]] => Unit): Unit = { + val hostLocalDirsCompletable = new CompletableFuture[java.util.Map[String, Array[String]]] + externalBlockStoreClient.getHostLocalDirs( + host, + externalShuffleServicePort, + executorIds, + hostLocalDirsCompletable) + hostLocalDirsCompletable.whenComplete { (hostLocalDirs, throwable) => + if (hostLocalDirs != null) { + callback(Success(hostLocalDirs)) + executorIdToLocalDirsCache.synchronized { + executorIdToLocalDirsCache.putAll(hostLocalDirs) + } + } else { + callback(Failure(throwable)) + } + } + } +} + /** * Manager running on every node (driver and executors) which provides interfaces for putting and * retrieving blocks both locally and remotely into various stores (memory, disk, and off-heap). @@ -206,6 +248,8 @@ private[spark] class BlockManager( new BlockManager.RemoteBlockDownloadFileManager(this) private val maxRemoteBlockToMem = conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) + var hostLocalDirManager: Option[HostLocalDirManager] = None + /** * Abstraction for storing blocks from bytes, whether they start in memory or on disk. * @@ -433,6 +477,20 @@ private[spark] class BlockManager( registerWithExternalShuffleServer() } + hostLocalDirManager = + if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED)) { + externalBlockStoreClient.map { blockStoreClient => + new HostLocalDirManager( + futureExecutionContext, + conf.get(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE), + blockStoreClient, + blockManagerId.host, + externalShuffleServicePort) + } + } else { + None + } + logInfo(s"Initialized BlockManager: $blockManagerId") } @@ -542,11 +600,17 @@ private[spark] class BlockManager( } } + override def getHostLocalShuffleData( + blockId: BlockId, + dirs: Array[String]): ManagedBuffer = { + shuffleManager.shuffleBlockResolver.getBlockData(blockId, Some(dirs)) + } + /** * Interface to get local block data. Throws an exception if the block cannot be found or * cannot be read successfully. */ - override def getBlockData(blockId: BlockId): ManagedBuffer = { + override def getLocalBlockData(blockId: BlockId): ManagedBuffer = { if (blockId.isShuffle) { shuffleManager.shuffleBlockResolver.getBlockData(blockId) } else { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 7e2027701c33a..41ef1909cd4c2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -26,6 +26,8 @@ import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import scala.util.Random +import com.google.common.cache.CacheBuilder + import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.{config, Logging} @@ -49,6 +51,13 @@ class BlockManagerMasterEndpoint( blockManagerInfo: mutable.Map[BlockManagerId, BlockManagerInfo]) extends IsolatedRpcEndpoint with Logging { + // Mapping from executor id to the block manager's local disk directories. + private val executorIdToLocalDirs = + CacheBuilder + .newBuilder() + .maximumSize(conf.get(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE)) + .build[String, Array[String]]() + // Mapping from external shuffle service block manager id to the block statuses. private val blockStatusByShuffleService = new mutable.HashMap[BlockManagerId, JHashMap[BlockId, BlockStatus]] @@ -393,6 +402,7 @@ class BlockManagerMasterEndpoint( topologyMapper.getTopologyForHost(idWithoutTopologyInfo.host)) val time = System.currentTimeMillis() + executorIdToLocalDirs.put(id.executorId, localDirs) if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { case Some(oldId) => @@ -416,7 +426,7 @@ class BlockManagerMasterEndpoint( None } - blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), localDirs, + blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint, externalShuffleServiceBlockStatus) } listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxOnHeapMemSize + maxOffHeapMemSize, @@ -496,15 +506,16 @@ class BlockManagerMasterEndpoint( if (locations.nonEmpty && status.isDefined) { val localDirs = locations.find { loc => - if (loc.port != externalShuffleServicePort && loc.host == requesterHost) { + // When the external shuffle service running on the same host is found among the block + // locations then the block must be persisted on the disk. In this case the executorId + // can be used to access this block even when the original executor is already stopped. + loc.host == requesterHost && + (loc.port == externalShuffleServicePort || blockManagerInfo .get(loc) .flatMap(_.getStatus(blockId).map(_.storageLevel.useDisk)) - .getOrElse(false) - } else { - false - } - }.map(blockManagerInfo(_).localDirs) + .getOrElse(false)) + }.flatMap { bmId => Option(executorIdToLocalDirs.getIfPresent(bmId.executorId)) } Some(BlockLocationsAndStatus(locations, status.get, localDirs)) } else { None @@ -556,7 +567,6 @@ object BlockStatus { private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, - val localDirs: Array[String], val maxOnHeapMem: Long, val maxOffHeapMem: Long, val slaveEndpoint: RpcEndpointRef, diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index f8aa97267cf1b..8fa7e68815a91 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -23,7 +23,8 @@ import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} import javax.annotation.concurrent.GuardedBy import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, LinkedHashMap, Queue} +import scala.util.{Failure, Success} import org.apache.commons.io.IOUtils @@ -84,11 +85,14 @@ final class ShuffleBlockFetcherIterator( import ShuffleBlockFetcherIterator._ + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + private val targetRemoteRequestSize = math.max(maxBytesInFlight / 5, 1L) + /** * Total number of blocks to fetch. This should be equal to the total number of blocks * in [[blocksByAddress]] because we already filter out zero-sized blocks in [[blocksByAddress]]. - * - * This should equal localBlocks.size + remoteBlocks.size. */ private[this] var numBlocksToFetch = 0 @@ -103,8 +107,12 @@ final class ShuffleBlockFetcherIterator( /** Local blocks to fetch, excluding zero-sized blocks. */ private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]() - /** Remote blocks to fetch, excluding zero-sized blocks. */ - private[this] val remoteBlocks = new HashSet[BlockId]() + /** Host local blockIds to fetch by executors, excluding zero-sized blocks. */ + private[this] val hostLocalBlocksByExecutor = + LinkedHashMap[BlockManagerId, Seq[(BlockId, Long, Int)]]() + + /** Host local blocks to fetch, excluding zero-sized blocks. */ + private[this] val hostLocalBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]() /** * A queue to hold our results. This turns the asynchronous model provided by @@ -272,73 +280,91 @@ final class ShuffleBlockFetcherIterator( } } - private[this] def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { - // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) - logDebug("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize - + ", maxBlocksInFlightPerAddress: " + maxBlocksInFlightPerAddress) - - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] + private[this] def partitionBlocksByFetchMode(): ArrayBuffer[FetchRequest] = { + logDebug(s"maxBytesInFlight: $maxBytesInFlight, targetRemoteRequestSize: " + + s"$targetRemoteRequestSize, maxBlocksInFlightPerAddress: $maxBlocksInFlightPerAddress") + + // Partition to local, host-local and remote blocks. Remote blocks are further split into + // FetchRequests of size at most maxBytesInFlight in order to limit the amount of data in flight + val collectedRemoteRequests = new ArrayBuffer[FetchRequest] var localBlockBytes = 0L + var hostLocalBlockBytes = 0L var remoteBlockBytes = 0L + var numRemoteBlocks = 0 + + val hostLocalDirReadingEnabled = + blockManager.hostLocalDirManager != null && blockManager.hostLocalDirManager.isDefined for ((address, blockInfos) <- blocksByAddress) { if (address.executorId == blockManager.blockManagerId.executorId) { - blockInfos.find(_._2 <= 0) match { - case Some((blockId, size, _)) if size < 0 => - throw new BlockException(blockId, "Negative block size " + size) - case Some((blockId, size, _)) if size == 0 => - throw new BlockException(blockId, "Zero-sized blocks should be excluded.") - case None => // do nothing. - } + checkBlockSizes(blockInfos) val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded( blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)).to[ArrayBuffer]) localBlocks ++= mergedBlockInfos.map(info => (info.blockId, info.mapIndex)) localBlockBytes += mergedBlockInfos.map(_.size).sum + } else if (hostLocalDirReadingEnabled && address.host == blockManager.blockManagerId.host) { + checkBlockSizes(blockInfos) + val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded( + blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)).to[ArrayBuffer]) + val blocksForAddress = + mergedBlockInfos.map(info => (info.blockId, info.size, info.mapIndex)) + hostLocalBlocksByExecutor += address -> blocksForAddress + hostLocalBlocks ++= blocksForAddress.map(info => (info._1, info._3)) + hostLocalBlockBytes += mergedBlockInfos.map(_.size).sum } else { - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[FetchBlockInfo] - while (iterator.hasNext) { - val (blockId, size, mapIndex) = iterator.next() - remoteBlockBytes += size - if (size < 0) { - throw new BlockException(blockId, "Negative block size " + size) - } else if (size == 0) { - throw new BlockException(blockId, "Zero-sized blocks should be excluded.") - } else { - curBlocks += FetchBlockInfo(blockId, size, mapIndex) - curRequestSize += size - } - if (curRequestSize >= targetRequestSize || - curBlocks.size >= maxBlocksInFlightPerAddress) { - // Add this FetchRequest - val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) - remoteBlocks ++= mergedBlocks.map(_.blockId) - remoteRequests += new FetchRequest(address, mergedBlocks) - logDebug(s"Creating fetch request of $curRequestSize at $address " - + s"with ${mergedBlocks.size} blocks") - curBlocks = new ArrayBuffer[FetchBlockInfo] - curRequestSize = 0 - } - } - // Add in the final request - if (curBlocks.nonEmpty) { - val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) - remoteBlocks ++= mergedBlocks.map(_.blockId) - remoteRequests += new FetchRequest(address, mergedBlocks) - } + numRemoteBlocks += blockInfos.size + remoteBlockBytes += blockInfos.map(_._2).sum + collectFetchRequests(address, blockInfos, collectedRemoteRequests) } } val totalBytes = localBlockBytes + remoteBlockBytes logInfo(s"Getting $numBlocksToFetch (${Utils.bytesToString(totalBytes)}) non-empty blocks " + - s"including ${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local blocks and " + - s"${remoteBlocks.size} (${Utils.bytesToString(remoteBlockBytes)}) remote blocks") - remoteRequests + s"including ${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local and " + + s"${hostLocalBlocks.size} (${Utils.bytesToString(hostLocalBlockBytes)}) " + + s"host-local and $numRemoteBlocks (${Utils.bytesToString(remoteBlockBytes)}) remote blocks") + collectedRemoteRequests + } + + private def collectFetchRequests( + address: BlockManagerId, + blockInfos: Seq[(BlockId, Long, Int)], + collectedRemoteRequests: ArrayBuffer[FetchRequest]): Unit = { + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[FetchBlockInfo] + while (iterator.hasNext) { + val (blockId, size, mapIndex) = iterator.next() + assertPositiveBlockSize(blockId, size) + curBlocks += FetchBlockInfo(blockId, size, mapIndex) + curRequestSize += size + if (curRequestSize >= targetRemoteRequestSize || + curBlocks.size >= maxBlocksInFlightPerAddress) { + // Add this FetchRequest + val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) + collectedRemoteRequests += new FetchRequest(address, mergedBlocks) + logDebug(s"Creating fetch request of $curRequestSize at $address " + + s"with ${mergedBlocks.size} blocks") + curBlocks = new ArrayBuffer[FetchBlockInfo] + curRequestSize = 0 + } + } + // Add in the final request + if (curBlocks.nonEmpty) { + val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) + collectedRemoteRequests += new FetchRequest(address, mergedBlocks) + } + } + + private def assertPositiveBlockSize(blockId: BlockId, blockSize: Long): Unit = { + if (blockSize < 0) { + throw BlockException(blockId, "Negative block size " + size) + } else if (blockSize == 0) { + throw BlockException(blockId, "Zero-sized blocks should be excluded.") + } + } + + private def checkBlockSizes(blockInfos: Seq[(BlockId, Long, Int)]): Unit = { + blockInfos.foreach { case (blockId, size, _) => assertPositiveBlockSize(blockId, size) } } private[this] def mergeContinuousShuffleBlockIdsIfNeeded( @@ -397,7 +423,7 @@ final class ShuffleBlockFetcherIterator( while (iter.hasNext) { val (blockId, mapIndex) = iter.next() try { - val buf = blockManager.getBlockData(blockId) + val buf = blockManager.getLocalBlockData(blockId) shuffleMetrics.incLocalBlocksFetched(1) shuffleMetrics.incLocalBytesRead(buf.size) buf.retain() @@ -420,12 +446,89 @@ final class ShuffleBlockFetcherIterator( } } + private[this] def fetchHostLocalBlock( + blockId: BlockId, + mapIndex: Int, + localDirs: Array[String], + blockManagerId: BlockManagerId): Boolean = { + try { + val buf = blockManager.getHostLocalShuffleData(blockId, localDirs) + buf.retain() + results.put(SuccessFetchResult(blockId, mapIndex, blockManagerId, buf.size(), buf, + isNetworkReqDone = false)) + true + } catch { + case e: Exception => + // If we see an exception, stop immediately. + logError(s"Error occurred while fetching local blocks", e) + results.put(FailureFetchResult(blockId, mapIndex, blockManagerId, e)) + false + } + } + + /** + * Fetch the host-local blocks while we are fetching remote blocks. This is ok because + * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we + * track in-memory are the ManagedBuffer references themselves. + */ + private[this] def fetchHostLocalBlocks(hostLocalDirManager: HostLocalDirManager): Unit = { + val cachedDirsByExec = hostLocalDirManager.getCachedHostLocalDirs() + val (hostLocalBlocksWithCachedDirs, hostLocalBlocksWithMissingDirs) = + hostLocalBlocksByExecutor + .map { case (hostLocalBmId, bmInfos) => + (hostLocalBmId, bmInfos, cachedDirsByExec.get(hostLocalBmId.executorId)) + }.partition(_._3.isDefined) + val bmId = blockManager.blockManagerId + val immutableHostLocalBlocksWithoutDirs = + hostLocalBlocksWithMissingDirs.map { case (hostLocalBmId, bmInfos, _) => + hostLocalBmId -> bmInfos + }.toMap + if (immutableHostLocalBlocksWithoutDirs.nonEmpty) { + logDebug(s"Asynchronous fetching host-local blocks without cached executors' dir: " + + s"${immutableHostLocalBlocksWithoutDirs.mkString(", ")}") + val execIdsWithoutDirs = immutableHostLocalBlocksWithoutDirs.keys.map(_.executorId).toArray + hostLocalDirManager.getHostLocalDirs(execIdsWithoutDirs) { + case Success(dirs) => + immutableHostLocalBlocksWithoutDirs.foreach { case (hostLocalBmId, blockInfos) => + blockInfos.takeWhile { case (blockId, _, mapIndex) => + fetchHostLocalBlock( + blockId, + mapIndex, + dirs.get(hostLocalBmId.executorId), + hostLocalBmId) + } + } + logDebug(s"Got host-local blocks (without cached executors' dir) in " + + s"${Utils.getUsedTimeNs(startTimeNs)}") + + case Failure(throwable) => + logError(s"Error occurred while fetching host local blocks", throwable) + val (hostLocalBmId, blockInfoSeq) = immutableHostLocalBlocksWithoutDirs.head + val (blockId, _, mapIndex) = blockInfoSeq.head + results.put(FailureFetchResult(blockId, mapIndex, hostLocalBmId, throwable)) + } + } + if (hostLocalBlocksWithCachedDirs.nonEmpty) { + logDebug(s"Synchronous fetching host-local blocks with cached executors' dir: " + + s"${hostLocalBlocksWithCachedDirs.mkString(", ")}") + hostLocalBlocksWithCachedDirs.foreach { case (_, blockInfos, localDirs) => + blockInfos.foreach { case (blockId, _, mapIndex) => + if (!fetchHostLocalBlock(blockId, mapIndex, localDirs.get, bmId)) { + return + } + } + } + logDebug(s"Got host-local blocks (with cached executors' dir) in " + + s"${Utils.getUsedTimeNs(startTimeNs)}") + } + } + private[this] def initialize(): Unit = { // Add a task completion callback (called in both success case and failure case) to cleanup. context.addTaskCompletionListener(onCompleteCallback) - // Split local and remote blocks. - val remoteRequests = splitLocalRemoteBlocks() + // Partition blocks by the different fetch modes: local, host-local and remote blocks. + val remoteRequests = partitionBlocksByFetchMode() // Add the remote requests into our queue in a random order fetchRequests ++= Utils.randomize(remoteRequests) assert ((0 == reqsInFlight) == (0 == bytesInFlight), @@ -441,6 +544,10 @@ final class ShuffleBlockFetcherIterator( // Get Local Blocks fetchLocalBlocks() logDebug(s"Got local blocks in ${Utils.getUsedTimeNs(startTimeNs)}") + + if (hostLocalBlocks.nonEmpty) { + blockManager.hostLocalDirManager.foreach(fetchHostLocalBlocks) + } } override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch @@ -476,15 +583,18 @@ final class ShuffleBlockFetcherIterator( result match { case r @ SuccessFetchResult(blockId, mapIndex, address, size, buf, isNetworkReqDone) => if (address != blockManager.blockManagerId) { - numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 - shuffleMetrics.incRemoteBytesRead(buf.size) - if (buf.isInstanceOf[FileSegmentManagedBuffer]) { - shuffleMetrics.incRemoteBytesReadToDisk(buf.size) + if (hostLocalBlocks.contains(blockId -> mapIndex)) { + shuffleMetrics.incLocalBlocksFetched(1) + shuffleMetrics.incLocalBytesRead(buf.size) + } else { + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + shuffleMetrics.incRemoteBytesRead(buf.size) + if (buf.isInstanceOf[FileSegmentManagedBuffer]) { + shuffleMetrics.incRemoteBytesReadToDisk(buf.size) + } + shuffleMetrics.incRemoteBlocksFetched(1) + bytesInFlight -= size } - shuffleMetrics.incRemoteBlocksFetched(1) - } - if (!localBlocks.contains((blockId, mapIndex))) { - bytesInFlight -= size } if (isNetworkReqDone) { reqsInFlight -= 1 diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 8844a0598ccb8..c217419f4092e 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -68,6 +68,7 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi // This test ensures that the external shuffle service is actually in use for the other tests. test("using external shuffle service") { sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) + sc.getConf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) should equal(false) sc.env.blockManager.externalShuffleServiceEnabled should equal(true) sc.env.blockManager.blockStoreClient.getClass should equal(classOf[ExternalBlockStoreClient]) @@ -79,7 +80,9 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi // Therefore, we should wait until all slaves are up TestUtils.waitUntilExecutorsUp(sc, 2, 60000) - val rdd = sc.parallelize(0 until 1000, 10).map(i => (i, 1)).reduceByKey(_ + _) + val rdd = sc.parallelize(0 until 1000, 10) + .map { i => (i, 1) } + .reduceByKey(_ + _) rdd.count() rdd.count() @@ -96,6 +99,50 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll wi e.getMessage should include ("Fetch failure will not retry stage due to testing config") } + test("SPARK-27651: read host local shuffle blocks from disk and avoid network remote fetches") { + val confWithHostLocalRead = + conf.clone.set(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED, true) + confWithHostLocalRead.set(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE, 5) + sc = new SparkContext("local-cluster[2,1,1024]", "test", confWithHostLocalRead) + sc.getConf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) should equal(true) + sc.env.blockManager.externalShuffleServiceEnabled should equal(true) + sc.env.blockManager.hostLocalDirManager.isDefined should equal(true) + sc.env.blockManager.blockStoreClient.getClass should equal(classOf[ExternalBlockStoreClient]) + + // In a slow machine, one slave may register hundreds of milliseconds ahead of the other one. + // If we don't wait for all slaves, it's possible that only one executor runs all jobs. Then + // all shuffle blocks will be in this executor, ShuffleBlockFetcherIterator will directly fetch + // local blocks from the local BlockManager and won't send requests to ExternalShuffleService. + // In this case, we won't receive FetchFailed. And it will make this test fail. + // Therefore, we should wait until all slaves are up + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) + + val rdd = sc.parallelize(0 until 1000, 10) + .map { i => (i, 1) } + .reduceByKey(_ + _) + + rdd.count() + rdd.count() + + val cachedExecutors = rdd.mapPartitions { _ => + SparkEnv.get.blockManager.hostLocalDirManager.map { localDirManager => + localDirManager.getCachedHostLocalDirs().keySet.iterator + }.getOrElse(Iterator.empty) + }.collect().toSet + + // both executors are caching the dirs of the other one + cachedExecutors should equal(sc.getExecutorIds().toSet) + + // Invalidate the registered executors, disallowing access to their shuffle blocks (without + // deleting the actual shuffle files, so we could access them without the shuffle service). + // As directories are already cached there is no request to external shuffle service. + rpcHandler.applicationRemoved(sc.conf.getAppId, false /* cleanupLocalDirs */) + + // Now Spark will not receive FetchFailed as host local blocks are read from the cached local + // disk directly + rdd.collect().map(_._2).sum should equal(1000) + } + test("SPARK-25888: using external shuffle service fetching disk persisted blocks") { val confWithRddFetchEnabled = conf.clone.set(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, true) sc = new SparkContext("local-cluster[1,1,1024]", "test", confWithRddFetchEnabled) diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index e05fad19567ae..c726329ce8a84 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -122,7 +122,7 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi val blockString = "Hello, world!" val blockBuffer = new NioManagedBuffer(ByteBuffer.wrap( blockString.getBytes(StandardCharsets.UTF_8))) - when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) + when(blockManager.getLocalBlockData(blockId)).thenReturn(blockBuffer) val securityManager0 = new SecurityManager(conf0) val exec0 = new NettyBlockTransferService(conf0, securityManager0, "localhost", "localhost", 0, diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 3f9536e224ded..a82f86a11c77e 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.shuffle import java.io.{ByteArrayOutputStream, InputStream} import java.nio.ByteBuffer +import org.mockito.ArgumentMatchers.{eq => meq} import org.mockito.Mockito.{mock, when} import org.apache.spark._ @@ -95,7 +96,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Setup the blockManager mock so the buffer gets returned when the shuffle code tries to // fetch shuffle data. val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) - when(blockManager.getBlockData(shuffleBlockId)).thenReturn(managedBuffer) + when(blockManager.getLocalBlockData(meq(shuffleBlockId))).thenReturn(managedBuffer) managedBuffer } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala index 49cbd66cccb86..01e3d6a46e709 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -31,7 +31,6 @@ class BlockManagerInfoSuite extends SparkFunSuite { val bmInfo = new BlockManagerInfo( BlockManagerId("executor0", "host", 1234, None), timeMs = 300, - Array(), maxOnHeapMem = 10000, maxOffHeapMem = 20000, slaveEndpoint = null, diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 8595f73fe5dd5..89f00b5a9d902 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -654,7 +654,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // check getRemoteBytes val bytesViaStore1 = cleanBm.getRemoteBytes(blockId) assert(bytesViaStore1.isDefined) - val expectedContent = sameHostBm.getBlockData(blockId).nioByteBuffer().array() + val expectedContent = sameHostBm.getLocalBlockData(blockId).nioByteBuffer().array() assert(bytesViaStore1.get.toArray === expectedContent) // check getRemoteValues @@ -1095,7 +1095,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val blockStatus = blockStatusOption.get assert((blockStatus.diskSize > 0) === !storageLevel.useMemory) assert((blockStatus.memSize > 0) === storageLevel.useMemory) - assert(blockManager.getBlockData(blockId).nioByteBuffer().array() === ser) + assert(blockManager.getLocalBlockData(blockId).nioByteBuffer().array() === ser) } Seq( diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 85b1a865603ad..cf4c292d4ca9e 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import java.io._ import java.nio.ByteBuffer import java.util.UUID -import java.util.concurrent.Semaphore +import java.util.concurrent.{CompletableFuture, Semaphore} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future @@ -33,7 +33,7 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkFunSuite, TaskContext} import org.apache.spark.network._ import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} -import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExternalBlockStoreClient} import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.Utils @@ -65,6 +65,29 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT transfer } + private def initHostLocalDirManager( + blockManager: BlockManager, + hostLocalDirs: Map[String, Array[String]]): Unit = { + val mockExternalBlockStoreClient = mock(classOf[ExternalBlockStoreClient]) + val hostLocalDirManager = new HostLocalDirManager( + futureExecutionContext = global, + cacheSize = 1, + externalBlockStoreClient = mockExternalBlockStoreClient, + host = "localhost", + externalShuffleServicePort = 7337) + + when(blockManager.hostLocalDirManager).thenReturn(Some(hostLocalDirManager)) + when(mockExternalBlockStoreClient.getHostLocalDirs(any(), any(), any(), any())) + .thenAnswer { invocation => + val completableFuture = invocation.getArguments()(3) + .asInstanceOf[CompletableFuture[java.util.Map[String, Array[String]]]] + import scala.collection.JavaConverters._ + completableFuture.complete(hostLocalDirs.asJava) + } + + blockManager.hostLocalDirManager = Some(hostLocalDirManager) + } + // Create a mock managed buffer for testing def createMockManagedBuffer(size: Int = 1): ManagedBuffer = { val mockManagedBuffer = mock(classOf[ManagedBuffer]) @@ -76,9 +99,24 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT mockManagedBuffer } - test("successful 3 local reads + 2 remote reads") { + def verifyBufferRelease(buffer: ManagedBuffer, inputStream: InputStream): Unit = { + // Note: ShuffleBlockFetcherIterator wraps input streams in a BufferReleasingInputStream + val wrappedInputStream = inputStream.asInstanceOf[BufferReleasingInputStream] + verify(buffer, times(0)).release() + val delegateAccess = PrivateMethod[InputStream](Symbol("delegate")) + + verify(wrappedInputStream.invokePrivate(delegateAccess()), times(0)).close() + wrappedInputStream.close() + verify(buffer, times(1)).release() + verify(wrappedInputStream.invokePrivate(delegateAccess()), times(1)).close() + wrappedInputStream.close() // close should be idempotent + verify(buffer, times(1)).release() + verify(wrappedInputStream.invokePrivate(delegateAccess()), times(1)).close() + } + + test("successful 3 local + 4 host local + 2 remote reads") { val blockManager = mock(classOf[BlockManager]) - val localBmId = BlockManagerId("test-client", "test-client", 1) + val localBmId = BlockManagerId("test-local-client", "test-local-host", 1) doReturn(localBmId).when(blockManager).blockManagerId // Make sure blockManager.getBlockData would return the blocks @@ -87,20 +125,38 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(), ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer()) localBlocks.foreach { case (blockId, buf) => - doReturn(buf).when(blockManager).getBlockData(meq(blockId)) + doReturn(buf).when(blockManager).getLocalBlockData(meq(blockId)) } // Make sure remote blocks would return - val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val remoteBmId = BlockManagerId("test-remote-client-1", "test-remote-host", 2) val remoteBlocks = Map[BlockId, ManagedBuffer]( ShuffleBlockId(0, 3, 0) -> createMockManagedBuffer(), ShuffleBlockId(0, 4, 0) -> createMockManagedBuffer()) val transfer = createMockTransfer(remoteBlocks) + // Create a block manager running on the same host (host-local) + val hostLocalBmId = BlockManagerId("test-host-local-client-1", "test-local-host", 3) + val hostLocalBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 5, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 6, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 7, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 8, 0) -> createMockManagedBuffer()) + + hostLocalBlocks.foreach { case (blockId, buf) => + doReturn(buf) + .when(blockManager) + .getHostLocalShuffleData(meq(blockId.asInstanceOf[ShuffleBlockId]), any()) + } + val hostLocalDirs = Map("test-host-local-client-1" -> Array("local-dir")) + // returning local dir for hostLocalBmId + initHostLocalDirManager(blockManager, hostLocalDirs) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( (localBmId, localBlocks.keys.map(blockId => (blockId, 1L, 0)).toSeq), - (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1L, 1)).toSeq) + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1L, 1)).toSeq), + (hostLocalBmId, hostLocalBlocks.keys.map(blockId => (blockId, 1L, 1)).toSeq) ).toIterator val taskContext = TaskContext.empty() @@ -121,37 +177,86 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT false) // 3 local blocks fetched in initialization - verify(blockManager, times(3)).getBlockData(any()) + verify(blockManager, times(3)).getLocalBlockData(any()) - for (i <- 0 until 5) { - assert(iterator.hasNext, s"iterator should have 5 elements but actually has $i elements") + val allBlocks = localBlocks ++ remoteBlocks ++ hostLocalBlocks + for (i <- 0 until allBlocks.size) { + assert(iterator.hasNext, + s"iterator should have ${allBlocks.size} elements but actually has $i elements") val (blockId, inputStream) = iterator.next() // Make sure we release buffers when a wrapped input stream is closed. - val mockBuf = localBlocks.getOrElse(blockId, remoteBlocks(blockId)) - // Note: ShuffleBlockFetcherIterator wraps input streams in a BufferReleasingInputStream - val wrappedInputStream = inputStream.asInstanceOf[BufferReleasingInputStream] - verify(mockBuf, times(0)).release() - val delegateAccess = PrivateMethod[InputStream](Symbol("delegate")) - - verify(wrappedInputStream.invokePrivate(delegateAccess()), times(0)).close() - wrappedInputStream.close() - verify(mockBuf, times(1)).release() - verify(wrappedInputStream.invokePrivate(delegateAccess()), times(1)).close() - wrappedInputStream.close() // close should be idempotent - verify(mockBuf, times(1)).release() - verify(wrappedInputStream.invokePrivate(delegateAccess()), times(1)).close() + val mockBuf = allBlocks(blockId) + verifyBufferRelease(mockBuf, inputStream) } - // 3 local blocks, and 2 remote blocks - // (but from the same block manager so one call to fetchBlocks) - verify(blockManager, times(3)).getBlockData(any()) + // 4 host-local locks fetched + verify(blockManager, times(4)) + .getHostLocalShuffleData(any(), meq(Array("local-dir"))) + + // 2 remote blocks are read from the same block manager verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any(), any()) + assert(blockManager.hostLocalDirManager.get.getCachedHostLocalDirs().size === 1) } - test("fetch continuous blocks in batch successful 3 local reads + 2 remote reads") { + test("error during accessing host local dirs for executors") { val blockManager = mock(classOf[BlockManager]) - val localBmId = BlockManagerId("test-client", "test-client", 1) + val localBmId = BlockManagerId("test-local-client", "test-local-host", 1) + doReturn(localBmId).when(blockManager).blockManagerId + val hostLocalBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer()) + + hostLocalBlocks.foreach { case (blockId, buf) => + doReturn(buf) + .when(blockManager) + .getHostLocalShuffleData(meq(blockId.asInstanceOf[ShuffleBlockId]), any()) + } + val hostLocalBmId = BlockManagerId("test-host-local-client-1", "test-local-host", 3) + + val mockExternalBlockStoreClient = mock(classOf[ExternalBlockStoreClient]) + val hostLocalDirManager = new HostLocalDirManager( + futureExecutionContext = global, + cacheSize = 1, + externalBlockStoreClient = mockExternalBlockStoreClient, + host = "localhost", + externalShuffleServicePort = 7337) + + when(blockManager.hostLocalDirManager).thenReturn(Some(hostLocalDirManager)) + when(mockExternalBlockStoreClient.getHostLocalDirs(any(), any(), any(), any())) + .thenAnswer { invocation => + val completableFuture = invocation.getArguments()(3) + .asInstanceOf[CompletableFuture[java.util.Map[String, Array[String]]]] + completableFuture.completeExceptionally(new Throwable("failed fetch")) + } + + blockManager.hostLocalDirManager = Some(hostLocalDirManager) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (hostLocalBmId, hostLocalBlocks.keys.map(blockId => (blockId, 1L, 1)).toSeq) + ).toIterator + + val transfer = createMockTransfer(Map()) + val taskContext = TaskContext.empty() + val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() + val iterator = new ShuffleBlockFetcherIterator( + taskContext, + transfer, + blockManager, + blocksByAddress, + (_, in) => in, + 48 * 1024 * 1024, + Int.MaxValue, + Int.MaxValue, + Int.MaxValue, + true, + false, + metrics, + false) + intercept[FetchFailedException] { iterator.next() } + } + + test("fetch continuous blocks in batch successful 3 local + 4 host local + 2 remote reads") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-local-host", 1) doReturn(localBmId).when(blockManager).blockManagerId // Make sure blockManager.getBlockData would return the merged block @@ -162,7 +267,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val mergedLocalBlocks = Map[BlockId, ManagedBuffer]( ShuffleBlockBatchId(0, 0, 0, 3) -> createMockManagedBuffer()) mergedLocalBlocks.foreach { case (blockId, buf) => - doReturn(buf).when(blockManager).getBlockData(meq(blockId)) + doReturn(buf).when(blockManager).getLocalBlockData(meq(blockId)) } // Make sure remote blocks would return the merged block @@ -174,9 +279,29 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT ShuffleBlockBatchId(0, 3, 0, 2) -> createMockManagedBuffer()) val transfer = createMockTransfer(mergedRemoteBlocks) + // Create a block manager running on the same host (host-local) + val hostLocalBmId = BlockManagerId("test-host-local-client-1", "test-local-host", 3) + val hostLocalBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 4, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 4, 1) -> createMockManagedBuffer(), + ShuffleBlockId(0, 4, 2) -> createMockManagedBuffer(), + ShuffleBlockId(0, 4, 3) -> createMockManagedBuffer()) + val mergedHostLocalBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockBatchId(0, 4, 0, 4) -> createMockManagedBuffer()) + + mergedHostLocalBlocks.foreach { case (blockId, buf) => + doReturn(buf) + .when(blockManager) + .getHostLocalShuffleData(meq(blockId.asInstanceOf[ShuffleBlockBatchId]), any()) + } + val hostLocalDirs = Map("test-host-local-client-1" -> Array("local-dir")) + // returning local dir for hostLocalBmId + initHostLocalDirManager(blockManager, hostLocalDirs) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( (localBmId, localBlocks.map(blockId => (blockId, 1L, 0))), - (remoteBmId, remoteBlocks.map(blockId => (blockId, 1L, 1))) + (remoteBmId, remoteBlocks.map(blockId => (blockId, 1L, 1))), + (hostLocalBmId, hostLocalBlocks.keys.map(blockId => (blockId, 1L, 1)).toSeq) ).toIterator val taskContext = TaskContext.empty() @@ -197,32 +322,23 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT true) // 3 local blocks batch fetched in initialization - verify(blockManager, times(1)).getBlockData(any()) + verify(blockManager, times(1)).getLocalBlockData(any()) - for (i <- 0 until 2) { - assert(iterator.hasNext, s"iterator should have 2 elements but actually has $i elements") + val allBlocks = mergedLocalBlocks ++ mergedRemoteBlocks ++ mergedHostLocalBlocks + for (i <- 0 until 3) { + assert(iterator.hasNext, s"iterator should have 3 elements but actually has $i elements") val (blockId, inputStream) = iterator.next() - + verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any(), any()) // Make sure we release buffers when a wrapped input stream is closed. - val mockBuf = mergedLocalBlocks.getOrElse(blockId, mergedRemoteBlocks(blockId)) - // Note: ShuffleBlockFetcherIterator wraps input streams in a BufferReleasingInputStream - val wrappedInputStream = inputStream.asInstanceOf[BufferReleasingInputStream] - verify(mockBuf, times(0)).release() - val delegateAccess = PrivateMethod[InputStream]('delegate) - - verify(wrappedInputStream.invokePrivate(delegateAccess()), times(0)).close() - wrappedInputStream.close() - verify(mockBuf, times(1)).release() - verify(wrappedInputStream.invokePrivate(delegateAccess()), times(1)).close() - wrappedInputStream.close() // close should be idempotent - verify(mockBuf, times(1)).release() - verify(wrappedInputStream.invokePrivate(delegateAccess()), times(1)).close() + val mockBuf = allBlocks(blockId) + verifyBufferRelease(mockBuf, inputStream) } - // 2 remote blocks batch fetched - // (but from the same block manager so one call to fetchBlocks) - verify(blockManager, times(1)).getBlockData(any()) - verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any(), any()) + // 4 host-local locks fetched + verify(blockManager, times(1)) + .getHostLocalShuffleData(any(), meq(Array("local-dir"))) + + assert(blockManager.hostLocalDirManager.get.getCachedHostLocalDirs().size === 1) } test("release current unexhausted buffer in case the task completes early") { @@ -546,7 +662,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val blockManager = mock(classOf[BlockManager]) val localBmId = BlockManagerId("test-client", "test-client", 1) doReturn(localBmId).when(blockManager).blockManagerId - doReturn(managedBuffer).when(blockManager).getBlockData(ShuffleBlockId(0, 0, 0)) + doReturn(managedBuffer).when(blockManager).getLocalBlockData(meq(ShuffleBlockId(0, 0, 0))) val localBlockLengths = Seq[Tuple3[BlockId, Long, Int]]( (ShuffleBlockId(0, 0, 0), 10000, 0) ) From bdf0c606b62505a1def7fa8af42f4e4904e02131 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 26 Nov 2019 14:58:20 -0800 Subject: [PATCH 0734/1387] [SPARK-28752][BUILD][FOLLOWUP] Fix to install `rouge` instead of `rogue` ### What changes were proposed in this pull request? This PR aims to fix a type; `rogue` -> `rouge` . This is a follow-up of https://github.com/apache/spark/pull/26521. ### Why are the changes needed? To support `Python 3`, we upgraded from `pygments` to `rouge`. ### Does this PR introduce any user-facing change? No. (This is for only document generation.) ### How was this patch tested? Manually. ``` $ docker build -t test dev/create-release/spark-rm/ ... 1 gem installed Successfully installed rouge-3.13.0 Parsing documentation for rouge-3.13.0 Installing ri documentation for rouge-3.13.0 Done installing documentation for rouge after 4 seconds 1 gem installed Removing intermediate container 9bd8707d9e84 ---> a18b2f6b0bb9 ... ``` Closes #26686 from dongjoon-hyun/SPARK-28752. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/spark-rm/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index fde75495b4428..cc7da152c7b2c 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -80,7 +80,7 @@ RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ $APT_INSTALL ruby2.3 ruby2.3-dev mkdocs && \ gem install jekyll --no-rdoc --no-ri -v 3.8.6 && \ gem install jekyll-redirect-from && \ - gem install rogue + gem install rouge WORKDIR /opt/spark-rm/output From 5b628f8b17e1c9a1f992d4b4fa926efc953f1861 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 26 Nov 2019 18:36:08 -0800 Subject: [PATCH 0735/1387] Revert "[SPARK-26081][SPARK-29999]" ### What changes were proposed in this pull request? This reverts commit 31c4fab (#23052) to make sure the partition calling `ManifestFileCommitProtocol.newTaskTempFile` creates actual file. This also reverts part of commit 0d3d46d (#26639) since the commit fixes the issue raised from 31c4fab and we're reverting back. The reason of partial revert is that we found the UT be worth to keep as it is, preventing regression - given the UT can detect the issue on empty partition -> no actual file. This makes one more change to UT; moved intentionally to test both DSv1 and DSv2. ### Why are the changes needed? After the changes in SPARK-26081 (commit 31c4fab / #23052), CSV/JSON/TEXT don't create actual file if the partition is empty. This optimization causes a problem in `ManifestFileCommitProtocol`: the API `newTaskTempFile` is called without actual file creation. Then `fs.getFileStatus` throws `FileNotFoundException` since the file is not created. SPARK-29999 (commit 0d3d46d / #26639) fixes the problem. But it is too costly to check file existence on each task commit. We should simply restore the behavior before SPARK-26081. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Jenkins build will follow. Closes #26671 from HeartSaVioR/revert-SPARK-26081-SPARK-29999. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Gengliang Wang --- .../datasources/csv/CsvOutputWriter.scala | 22 ++--- .../datasources/json/JsonOutputWriter.scala | 18 ++-- .../datasources/text/TextOutputWriter.scala | 16 +-- .../ManifestFileCommitProtocol.scala | 23 ++--- .../execution/datasources/csv/CSVSuite.scala | 9 -- .../datasources/json/JsonSuite.scala | 9 -- .../datasources/text/TextSuite.scala | 9 -- .../sql/streaming/FileStreamSinkSuite.scala | 98 +++++++++---------- 8 files changed, 73 insertions(+), 131 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CsvOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CsvOutputWriter.scala index 3ff36bfde3cca..2b549536ae486 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CsvOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CsvOutputWriter.scala @@ -33,25 +33,17 @@ class CsvOutputWriter( context: TaskAttemptContext, params: CSVOptions) extends OutputWriter with Logging { - private var univocityGenerator: Option[UnivocityGenerator] = None + private val charset = Charset.forName(params.charset) + + private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path), charset) + + private val gen = new UnivocityGenerator(dataSchema, writer, params) if (params.headerFlag) { - val gen = getGen() gen.writeHeaders() } - private def getGen(): UnivocityGenerator = univocityGenerator.getOrElse { - val charset = Charset.forName(params.charset) - val os = CodecStreams.createOutputStreamWriter(context, new Path(path), charset) - val newGen = new UnivocityGenerator(dataSchema, os, params) - univocityGenerator = Some(newGen) - newGen - } - - override def write(row: InternalRow): Unit = { - val gen = getGen() - gen.write(row) - } + override def write(row: InternalRow): Unit = gen.write(row) - override def close(): Unit = univocityGenerator.foreach(_.close()) + override def close(): Unit = gen.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala index b3cd570cfb1cf..dfd84e344eb2a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala @@ -44,20 +44,18 @@ class JsonOutputWriter( " which can be read back by Spark only if multiLine is enabled.") } - private var jacksonGenerator: Option[JacksonGenerator] = None + private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path), encoding) - override def write(row: InternalRow): Unit = { - val gen = jacksonGenerator.getOrElse { - val os = CodecStreams.createOutputStreamWriter(context, new Path(path), encoding) - // create the Generator without separator inserted between 2 records - val newGen = new JacksonGenerator(dataSchema, os, options) - jacksonGenerator = Some(newGen) - newGen - } + // create the Generator without separator inserted between 2 records + private[this] val gen = new JacksonGenerator(dataSchema, writer, options) + override def write(row: InternalRow): Unit = { gen.write(row) gen.writeLineEnding() } - override def close(): Unit = jacksonGenerator.foreach(_.close()) + override def close(): Unit = { + gen.close() + writer.close() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOutputWriter.scala index faf6e573105f2..2b1b81f60ceb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOutputWriter.scala @@ -16,8 +16,6 @@ */ package org.apache.spark.sql.execution.datasources.text -import java.io.OutputStream - import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.TaskAttemptContext @@ -32,23 +30,17 @@ class TextOutputWriter( context: TaskAttemptContext) extends OutputWriter { - private var outputStream: Option[OutputStream] = None + private val writer = CodecStreams.createOutputStream(context, new Path(path)) override def write(row: InternalRow): Unit = { - val os = outputStream.getOrElse { - val newStream = CodecStreams.createOutputStream(context, new Path(path)) - outputStream = Some(newStream) - newStream - } - if (!row.isNullAt(0)) { val utf8string = row.getUTF8String(0) - utf8string.writeTo(os) + utf8string.writeTo(writer) } - os.write(lineSeparator) + writer.write(lineSeparator) } override def close(): Unit = { - outputStream.foreach(_.close()) + writer.close() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index be7973b9d9300..f6cc8116c6c4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -22,7 +22,7 @@ import java.util.UUID import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.spark.internal.Logging @@ -89,7 +89,9 @@ class ManifestFileCommitProtocol(jobId: String, path: String) try { val fs = path.getFileSystem(jobContext.getConfiguration) // this is to make sure the file can be seen from driver as well - deleteIfExists(fs, path) + if (fs.exists(path)) { + fs.delete(path, false) + } } catch { case e: IOException => logWarning(s"Fail to remove temporary file $path, continue removing next.", e) @@ -137,14 +139,7 @@ class ManifestFileCommitProtocol(jobId: String, path: String) if (addedFiles.nonEmpty) { val fs = new Path(addedFiles.head).getFileSystem(taskContext.getConfiguration) val statuses: Seq[SinkFileStatus] = - addedFiles.flatMap { f => - val path = new Path(f) - if (fs.exists(path)) { - Some(SinkFileStatus(fs.getFileStatus(path))) - } else { - None - } - } + addedFiles.map(f => SinkFileStatus(fs.getFileStatus(new Path(f)))) new TaskCommitMessage(statuses) } else { new TaskCommitMessage(Seq.empty[SinkFileStatus]) @@ -155,13 +150,7 @@ class ManifestFileCommitProtocol(jobId: String, path: String) // best effort cleanup of incomplete files if (addedFiles.nonEmpty) { val fs = new Path(addedFiles.head).getFileSystem(taskContext.getConfiguration) - addedFiles.foreach { file => deleteIfExists(fs, new Path(file)) } - } - } - - private def deleteIfExists(fs: FileSystem, path: Path, recursive: Boolean = false): Unit = { - if (fs.exists(path)) { - fs.delete(path, recursive) + addedFiles.foreach { file => fs.delete(new Path(file), false) } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index a6c3a51858aed..a6e58cec10369 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2073,15 +2073,6 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } } - test("do not produce empty files for empty partitions") { - withTempPath { dir => - val path = dir.getCanonicalPath - spark.emptyDataset[String].write.csv(path) - val files = new File(path).listFiles() - assert(!files.exists(_.getName.endsWith("csv"))) - } - } - test("Do not reuse last good value for bad input field") { val schema = StructType( StructField("col1", StringType) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 3574aa266b35f..e3e0195f08e3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2471,15 +2471,6 @@ class JsonSuite extends QueryTest with SharedSparkSession with TestJsonData { emptyString(BinaryType, "".getBytes(StandardCharsets.UTF_8)) } - test("do not produce empty files for empty partitions") { - withTempPath { dir => - val path = dir.getCanonicalPath - spark.emptyDataset[String].write.json(path) - val files = new File(path).listFiles() - assert(!files.exists(_.getName.endsWith("json"))) - } - } - test("return partial result for bad records") { val schema = "a double, b array, c string, _corrupt_record string" val badRecords = Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 62a779528cec1..539ff0d0e905c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -233,13 +233,4 @@ class TextSuite extends QueryTest with SharedSparkSession { assert(data(3) == Row("\"doh\"")) assert(data.length == 4) } - - test("do not produce empty files for empty partitions") { - withTempPath { dir => - val path = dir.getCanonicalPath - spark.emptyDataset[String].write.text(path) - val files = new File(path).listFiles() - assert(!files.exists(_.getName.endsWith("txt"))) - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index f04da8bfc4485..9bce7f3568e81 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -525,6 +525,54 @@ abstract class FileStreamSinkSuite extends StreamTest { } } } + + test("Handle FileStreamSink metadata correctly for empty partition") { + Seq("parquet", "orc", "text", "json").foreach { format => + val inputData = MemoryStream[String] + val df = inputData.toDF() + + withTempDir { outputDir => + withTempDir { checkpointDir => + var query: StreamingQuery = null + try { + // repartition to more than the input to leave empty partitions + query = + df.repartition(10) + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format(format) + .start(outputDir.getCanonicalPath) + + inputData.addData("1", "2", "3") + inputData.addData("4", "5") + + failAfter(streamingTimeout) { + query.processAllAvailable() + } + } finally { + if (query != null) { + query.stop() + } + } + + val fs = new Path(outputDir.getCanonicalPath).getFileSystem( + spark.sessionState.newHadoopConf()) + val sinkLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, spark, + outputDir.getCanonicalPath) + + val allFiles = sinkLog.allFiles() + // only files from non-empty partition should be logged + assert(allFiles.length < 10) + assert(allFiles.forall(file => fs.exists(new Path(file.path)))) + + // the query should be able to read all rows correctly with metadata log + val outputDf = spark.read.format(format).load(outputDir.getCanonicalPath) + .selectExpr("CAST(value AS INT)").as[Int] + checkDatasetUnorderly(outputDf, 1, 2, 3, 4, 5) + } + } + } + } } object PendingCommitFilesTrackingManifestFileCommitProtocol { @@ -600,61 +648,11 @@ class FileStreamSinkV1Suite extends FileStreamSinkSuite { } class FileStreamSinkV2Suite extends FileStreamSinkSuite { - import testImplicits._ - override protected def sparkConf: SparkConf = super .sparkConf .set(SQLConf.USE_V1_SOURCE_LIST, "") - test("SPARK-29999 Handle FileStreamSink metadata correctly for empty partition") { - Seq("parquet", "orc", "text", "json").foreach { format => - val inputData = MemoryStream[String] - val df = inputData.toDF() - - withTempDir { outputDir => - withTempDir { checkpointDir => - var query: StreamingQuery = null - try { - // repartition to more than the input to leave empty partitions - query = - df.repartition(10) - .writeStream - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .format(format) - .start(outputDir.getCanonicalPath) - - inputData.addData("1", "2", "3") - inputData.addData("4", "5") - - failAfter(streamingTimeout) { - query.processAllAvailable() - } - } finally { - if (query != null) { - query.stop() - } - } - - val fs = new Path(outputDir.getCanonicalPath).getFileSystem( - spark.sessionState.newHadoopConf()) - val sinkLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, spark, - outputDir.getCanonicalPath) - - val allFiles = sinkLog.allFiles() - // only files from non-empty partition should be logged - assert(allFiles.length < 10) - assert(allFiles.forall(file => fs.exists(new Path(file.path)))) - - // the query should be able to read all rows correctly with metadata log - val outputDf = spark.read.format(format).load(outputDir.getCanonicalPath) - .selectExpr("CAST(value AS INT)").as[Int] - checkDatasetUnorderly(outputDf, 1, 2, 3, 4, 5) - } - } - } - } - override def checkQueryExecution(df: DataFrame): Unit = { // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has // been inferred From 08e2a39df277df1385c828bbed77b8ee46c2e796 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 26 Nov 2019 19:38:46 -0800 Subject: [PATCH 0736/1387] [SPARK-29997][WEBUI] Show job name for empty jobs in WebUI ### What changes were proposed in this pull request? In current implementation, job name for empty jobs is not shown so I've made change to show it. ### Why are the changes needed? To make debug easier. ### Does this PR introduce any user-facing change? Yes. Before applying my change, the `Job Page` will show as follows as the result of submitting a job which contains no partitions. ![fix-ui-for-empty-job-before](https://user-images.githubusercontent.com/4736016/69410847-33bfb280-0d4f-11ea-9878-d67638cbe4cb.png) After applying my change, the page will show a display like a following screenshot. ![fix-ui-for-empty-job-after](https://user-images.githubusercontent.com/4736016/69411021-86996a00-0d4f-11ea-8dea-bb8456159d18.png) ### How was this patch tested? Manual test. Closes #26637 from sarutak/fix-ui-for-empty-job. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/scheduler/DAGScheduler.scala | 12 +++++++++++- .../scala/org/apache/spark/scheduler/StageInfo.scala | 3 +++ .../org/apache/spark/status/AppStatusListener.scala | 3 ++- .../scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 6 +++++- 4 files changed, 21 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index fe3a48440991a..851419132ff2e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -696,8 +696,18 @@ private[spark] class DAGScheduler( val jobId = nextJobId.getAndIncrement() if (partitions.isEmpty) { val time = clock.getTimeMillis() + val dummyStageInfo = + new StageInfo( + StageInfo.INVALID_STAGE_ID, + StageInfo.INVALID_ATTEMPT_ID, + callSite.shortForm, + 0, + Seq.empty[RDDInfo], + Seq.empty[Int], + "") listenerBus.post( - SparkListenerJobStart(jobId, time, Seq[StageInfo](), Utils.cloneProperties(properties))) + SparkListenerJobStart( + jobId, time, Seq[StageInfo](dummyStageInfo), Utils.cloneProperties(properties))) listenerBus.post( SparkListenerJobEnd(jobId, time, JobSucceeded)) // Return immediately if the job is running 0 tasks diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index fdc50328b43d8..c2cb95771b27c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -107,4 +107,7 @@ private[spark] object StageInfo { taskLocalityPreferences, shuffleDepId) } + + final val INVALID_STAGE_ID = -1 + final val INVALID_ATTEMPT_ID = -1 } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index a5850fc2ac4b9..df140ba7d1547 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -414,7 +414,8 @@ private[spark] class AppStatusListener( val e = it.next() if (job.stageIds.contains(e.getKey()._1)) { val stage = e.getValue() - if (v1.StageStatus.PENDING.equals(stage.status)) { + // If a stage has no partitions(tasks), the stage should not be marked as skipped. + if (v1.StageStatus.PENDING.equals(stage.status) && stage.info.numTasks > 0) { stage.status = v1.StageStatus.SKIPPED job.skippedStages += stage.info.stageId job.skippedTasks += stage.info.numTasks diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 91e9caeae05db..49f57062dd9fc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -629,7 +629,11 @@ private[ui] class JobPagedTable( {jobTableRow.formattedDuration} - {job.numCompletedStages}/{job.stageIds.size - job.numSkippedStages} + {job.numCompletedStages}/{ + // A job contains at least 1 stage but if a job has no partitions(tasks), + // the stage is not submitted so the total stage should be regarded as 0. + if (job.numTasks > 0) job.stageIds.size - job.numSkippedStages else 0 + } {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} From 4fd585d2c5c5624c2d0355311077e0ffd776494b Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 26 Nov 2019 20:40:21 -0800 Subject: [PATCH 0737/1387] [SPARK-30008][SQL] The dataType of collect_list/collect_set aggs should be ArrayType(_, false) ### What changes were proposed in this pull request? ```scala // Do not allow null values. We follow the semantics of Hive's collect_list/collect_set here. // See: org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMkCollectionEvaluator ``` These two functions do not allow null values as they are defined, so their elements should not contain null. ### Why are the changes needed? Casting collect_list(a) to ArrayType(_, false) fails before this fix. ### Does this PR introduce any user-facing change? no ### How was this patch tested? add ut Closes #26651 from yaooqinn/SPARK-30008. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/expressions/aggregate/collect.scala | 2 +- .../spark/sql/catalyst/expressions/CastSuite.scala | 8 ++++++++ .../apache/spark/sql/DataFrameAggregateSuite.scala | 11 ++++++++++- 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 7cc43dfdfac33..a302726af9839 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -40,7 +40,7 @@ abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImper override def nullable: Boolean = true - override def dataType: DataType = ArrayType(child.dataType) + override def dataType: DataType = ArrayType(child.dataType, false) // Both `CollectList` and `CollectSet` are non-deterministic since their results depend on the // actual order of input rows. 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 12ca3e798b13d..a2359d655a1ef 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 @@ -27,6 +27,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCoercion.numericPrecedence +import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectList, CollectSet} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ @@ -1214,6 +1215,13 @@ class CastSuite extends CastSuiteBase { checkEvaluation(Cast(Literal(134.12), DecimalType(3, 2)), null) } } + + test("collect_list/collect_set can cast to ArrayType not containsNull") { + val list = CollectList(Literal(1)) + assert(Cast.canCast(list.dataType, ArrayType(IntegerType, false))) + val set = CollectSet(Literal(1)) + assert(Cast.canCast(set.dataType, ArrayType(StringType, false))) + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index ec7b636c8f695..fb666e9272e56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.DecimalData -import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, IntegerType} + case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) @@ -546,6 +547,14 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { ) } + test("collect functions should be able to cast to array type with no null values") { + val df = Seq(1, 2).toDF("a") + checkAnswer(df.select(collect_list("a") cast ArrayType(IntegerType, false)), + Seq(Row(Seq(1, 2)))) + checkAnswer(df.select(collect_set("a") cast ArrayType(FloatType, false)), + Seq(Row(Seq(1.0, 2.0)))) + } + test("SPARK-14664: Decimal sum/avg over window should work.") { checkAnswer( spark.sql("select sum(a) over () from values 1.0, 2.0, 3.0 T(a)"), From a58d91b1594f454e436885906798f3d6e1f781c9 Mon Sep 17 00:00:00 2001 From: wuyi Date: Wed, 27 Nov 2019 15:37:01 +0800 Subject: [PATCH 0738/1387] [SPARK-29768][SQL] Column pruning through nondeterministic expressions ### What changes were proposed in this pull request? Support columnar pruning through non-deterministic expressions. ### Why are the changes needed? In some cases, columns can still be pruned even though nondeterministic expressions appears. e.g. for the plan `Filter('a = 1, Project(Seq('a, rand() as 'r), LogicalRelation('a, 'b)))`, we shall still prune column b while non-deterministic expression appears. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added a new test file: `ScanOperationSuite`. Added test in `FileSourceStrategySuite` to verify the right prune behavior for both DS v1 and v2. Closes #26629 from Ngone51/SPARK-29768. Authored-by: wuyi Signed-off-by: Wenchen Fan --- .../sql/catalyst/planning/patterns.scala | 101 ++++++++++++++--- .../planning/ScanOperationSuite.scala | 104 ++++++++++++++++++ .../datasources/DataSourceStrategy.scala | 8 +- .../datasources/FileSourceStrategy.scala | 4 +- .../v2/V2ScanRelationPushDown.scala | 4 +- .../datasources/FileSourceStrategySuite.scala | 35 +++++- .../spark/sql/sources/PrunedScanSuite.scala | 4 + 7 files changed, 235 insertions(+), 25 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/ScanOperationSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index c2a12eda19137..4e790b1dd3f36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -26,6 +26,28 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +trait OperationHelper { + type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan) + + protected def collectAliases(fields: Seq[Expression]): AttributeMap[Expression] = + AttributeMap(fields.collect { + case a: Alias => (a.toAttribute, a.child) + }) + + protected def substitute(aliases: AttributeMap[Expression])(expr: Expression): Expression = { + expr.transform { + case a @ Alias(ref: AttributeReference, name) => + aliases.get(ref) + .map(Alias(_, name)(a.exprId, a.qualifier)) + .getOrElse(a) + + case a: AttributeReference => + aliases.get(a) + .map(Alias(_, a.name)(a.exprId, a.qualifier)).getOrElse(a) + } + } +} + /** * A pattern that matches any number of project or filter operations on top of another relational * operator. All filter operators are collected and their conditions are broken up and returned @@ -33,8 +55,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ * [[org.apache.spark.sql.catalyst.expressions.Alias Aliases]] are in-lined/substituted if * necessary. */ -object PhysicalOperation extends PredicateHelper { - type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan) +object PhysicalOperation extends OperationHelper with PredicateHelper { def unapply(plan: LogicalPlan): Option[ReturnType] = { val (fields, filters, child, _) = collectProjectsAndFilters(plan) @@ -74,22 +95,72 @@ object PhysicalOperation extends PredicateHelper { case other => (None, Nil, other, AttributeMap(Seq())) } +} - private def collectAliases(fields: Seq[Expression]): AttributeMap[Expression] = - AttributeMap(fields.collect { - case a: Alias => (a.toAttribute, a.child) - }) +/** + * A variant of [[PhysicalOperation]]. It matches any number of project or filter + * operations even if they are non-deterministic, as long as they satisfy the + * requirement of CollapseProject and CombineFilters. + */ +object ScanOperation extends OperationHelper with PredicateHelper { + type ScanReturnType = Option[(Option[Seq[NamedExpression]], + Seq[Expression], LogicalPlan, AttributeMap[Expression])] - private def substitute(aliases: AttributeMap[Expression])(expr: Expression): Expression = { - expr.transform { - case a @ Alias(ref: AttributeReference, name) => - aliases.get(ref) - .map(Alias(_, name)(a.exprId, a.qualifier)) - .getOrElse(a) + def unapply(plan: LogicalPlan): Option[ReturnType] = { + collectProjectsAndFilters(plan) match { + case Some((fields, filters, child, _)) => + Some((fields.getOrElse(child.output), filters, child)) + case None => None + } + } - case a: AttributeReference => - aliases.get(a) - .map(Alias(_, a.name)(a.exprId, a.qualifier)).getOrElse(a) + private def hasCommonNonDeterministic( + expr: Seq[Expression], + aliases: AttributeMap[Expression]): Boolean = { + expr.exists(_.collect { + case a: AttributeReference if aliases.contains(a) => aliases(a) + }.exists(!_.deterministic)) + } + + private def collectProjectsAndFilters(plan: LogicalPlan): ScanReturnType = { + plan match { + case Project(fields, child) => + collectProjectsAndFilters(child) match { + case Some((_, filters, other, aliases)) => + // Follow CollapseProject and only keep going if the collected Projects + // do not have common non-deterministic expressions. + if (!hasCommonNonDeterministic(fields, aliases)) { + val substitutedFields = + fields.map(substitute(aliases)).asInstanceOf[Seq[NamedExpression]] + Some((Some(substitutedFields), filters, other, collectAliases(substitutedFields))) + } else { + None + } + case None => None + } + + case Filter(condition, child) => + collectProjectsAndFilters(child) match { + case Some((fields, filters, other, aliases)) => + // Follow CombineFilters and only keep going if the collected Filters + // are all deterministic and this filter doesn't have common non-deterministic + // expressions with lower Project. + if (filters.forall(_.deterministic) && + !hasCommonNonDeterministic(Seq(condition), aliases)) { + val substitutedCondition = substitute(aliases)(condition) + Some((fields, filters ++ splitConjunctivePredicates(substitutedCondition), + other, aliases)) + } else { + None + } + case None => None + } + + case h: ResolvedHint => + collectProjectsAndFilters(h.child) + + case other => + Some((None, Nil, other, AttributeMap(Seq()))) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/ScanOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/ScanOperationSuite.scala new file mode 100644 index 0000000000000..7790f467a890b --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/ScanOperationSuite.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.sql.catalyst.planning + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.TestRelations +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.types.DoubleType + +class ScanOperationSuite extends SparkFunSuite { + private val relation = TestRelations.testRelation2 + private val colA = relation.output(0) + private val colB = relation.output(1) + private val aliasR = Alias(Rand(1), "r")() + private val aliasId = Alias(MonotonicallyIncreasingID(), "id")() + private val colR = AttributeReference("r", DoubleType)(aliasR.exprId, aliasR.qualifier) + + test("Project with a non-deterministic field and a deterministic child Filter") { + val project1 = Project(Seq(colB, aliasR), Filter(EqualTo(colA, Literal(1)), relation)) + project1 match { + case ScanOperation(projects, filters, _: LocalRelation) => + assert(projects.size === 2) + assert(projects(0) === colB) + assert(projects(1) === aliasR) + assert(filters.size === 1) + } + } + + test("Project with all deterministic fields but a non-deterministic child Filter") { + val project2 = Project(Seq(colA, colB), Filter(EqualTo(aliasR, Literal(1)), relation)) + project2 match { + case ScanOperation(projects, filters, _: LocalRelation) => + assert(projects.size === 2) + assert(projects(0) === colA) + assert(projects(1) === colB) + assert(filters.size === 1) + } + } + + test("Project which has the same non-deterministic expression with its child Project") { + val project3 = Project(Seq(colA, colR), Project(Seq(colA, aliasR), relation)) + assert(ScanOperation.unapply(project3).isEmpty) + } + + test("Project which has different non-deterministic expressions with its child Project") { + val project4 = Project(Seq(colA, aliasId), Project(Seq(colA, aliasR), relation)) + project4 match { + case ScanOperation(projects, _, _: LocalRelation) => + assert(projects.size === 2) + assert(projects(0) === colA) + assert(projects(1) === aliasId) + } + } + + test("Filter which has the same non-deterministic expression with its child Project") { + val filter1 = Filter(EqualTo(colR, Literal(1)), Project(Seq(colA, aliasR), relation)) + assert(ScanOperation.unapply(filter1).isEmpty) + } + + test("Deterministic filter with a child Project with a non-deterministic expression") { + val filter2 = Filter(EqualTo(colA, Literal(1)), Project(Seq(colA, aliasR), relation)) + filter2 match { + case ScanOperation(projects, filters, _: LocalRelation) => + assert(projects.size === 2) + assert(projects(0) === colA) + assert(projects(1) === aliasR) + assert(filters.size === 1) + } + } + + test("Filter which has different non-deterministic expressions with its child Project") { + val filter3 = Filter(EqualTo(MonotonicallyIncreasingID(), Literal(1)), + Project(Seq(colA, aliasR), relation)) + filter3 match { + case ScanOperation(projects, filters, _: LocalRelation) => + assert(projects.size === 2) + assert(projects(0) === colA) + assert(projects(1) === aliasR) + assert(filters.size === 1) + } + } + + + test("Deterministic filter which has a non-deterministic child Filter") { + val filter4 = Filter(EqualTo(colA, Literal(1)), Filter(EqualTo(aliasR, Literal(1)), relation)) + assert(ScanOperation.unapply(filter4).isEmpty) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 211642d78aabe..46444f0a05605 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} @@ -264,7 +264,7 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with import DataSourceStrategy._ def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _, _)) => + case ScanOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _, _)) => pruneFilterProjectRaw( l, projects, @@ -272,7 +272,7 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with (requestedColumns, allPredicates, _) => toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, allPredicates))) :: Nil - case PhysicalOperation(projects, filters, + case ScanOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _, _, _)) => pruneFilterProject( l, @@ -280,7 +280,7 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with filters, (a, f) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _, _)) => + case ScanOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _, _)) => pruneFilterProject( l, projects, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index afc9bfeda84a6..bd342c7f404fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.util.collection.BitSet @@ -137,7 +137,7 @@ object FileSourceStrategy extends Strategy with Logging { } def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case PhysicalOperation(projects, filters, + case ScanOperation(projects, filters, l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _)) => // Filters on this relation fall into four categories based on where we can use them to avoid // reading unneeded data: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index 6aa8d989583d1..239e3e8f82f18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.expressions.{And, SubqueryExpression} -import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.DataSourceStrategy @@ -27,7 +27,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] { import DataSourceV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { - case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => + case ScanOperation(project, filters, relation: DataSourceV2Relation) => val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options) val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index fa8111407665a..812305ba24403 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -31,12 +31,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet, PredicateHelper} import org.apache.spark.sql.catalyst.util -import org.apache.spark.sql.execution.{DataSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.{DataSourceScanExec, FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.{IntegerType, LongType, StructField, StructType} import org.apache.spark.util.Utils class FileSourceStrategySuite extends QueryTest with SharedSparkSession with PredicateHelper { @@ -497,6 +498,36 @@ class FileSourceStrategySuite extends QueryTest with SharedSparkSession with Pre } } + test("SPARK-29768: Column pruning through non-deterministic expressions") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + withTempPath { path => + spark.range(10) + .selectExpr("id as key", "id * 3 as s1", "id * 5 as s2") + .write.format("parquet").save(path.getAbsolutePath) + val df1 = spark.read.parquet(path.getAbsolutePath) + val df2 = df1.selectExpr("key", "rand()").where("key > 5") + val plan = df2.queryExecution.sparkPlan + val scan = plan.collect { case scan: FileSourceScanExec => scan } + assert(scan.size === 1) + assert(scan.head.requiredSchema == StructType(StructField("key", LongType) :: Nil)) + } + } + + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + withTempPath { path => + spark.range(10) + .selectExpr("id as key", "id * 3 as s1", "id * 5 as s2") + .write.format("parquet").save(path.getAbsolutePath) + val df1 = spark.read.parquet(path.getAbsolutePath) + val df2 = df1.selectExpr("key", "rand()").where("key > 5") + val plan = df2.queryExecution.optimizedPlan + val scan = plan.collect { case r: DataSourceV2ScanRelation => r } + assert(scan.size === 1) + assert(scan.head.scan.readSchema() == StructType(StructField("key", LongType) :: Nil)) + } + } + } + // Helpers for checking the arguments passed to the FileFormat. protected val checkPartitionSchema = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index d99c605b2e478..237717a3ad196 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -115,6 +115,10 @@ class PrunedScanSuite extends DataSourceTest with SharedSparkSession { testPruning("SELECT b, b FROM oneToTenPruned", "b") testPruning("SELECT a FROM oneToTenPruned", "a") testPruning("SELECT b FROM oneToTenPruned", "b") + testPruning("SELECT a, rand() FROM oneToTenPruned WHERE a > 5", "a") + testPruning("SELECT a FROM oneToTenPruned WHERE rand() > 5", "a") + testPruning("SELECT a, rand() FROM oneToTenPruned WHERE rand() > 5", "a") + testPruning("SELECT a, rand() FROM oneToTenPruned WHERE b > 5", "a", "b") def testPruning(sqlString: String, expectedColumns: String*): Unit = { test(s"Columns output ${expectedColumns.mkString(",")}: $sqlString") { From 19af1fe3a2b604a653c9f736d11648b79b93bb17 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 27 Nov 2019 18:13:33 +0900 Subject: [PATCH 0739/1387] [SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column ## What changes were proposed in this pull request? [HIVE-12063](https://issues.apache.org/jira/browse/HIVE-12063) improved pad decimal numbers with trailing zeros to the scale of the column. The following description is copied from the description of HIVE-12063. > HIVE-7373 was to address the problems of trimming tailing zeros by Hive, which caused many problems including treating 0.0, 0.00 and so on as 0, which has different precision/scale. Please refer to HIVE-7373 description. However, HIVE-7373 was reverted by HIVE-8745 while the underlying problems remained. HIVE-11835 was resolved recently to address one of the problems, where 0.0, 0.00, and so on cannot be read into decimal(1,1). However, HIVE-11835 didn't address the problem of showing as 0 in query result for any decimal values such as 0.0, 0.00, etc. This causes confusion as 0 and 0.0 have different precision/scale than 0. The proposal here is to pad zeros for query result to the type's scale. This not only removes the confusion described above, but also aligns with many other DBs. Internal decimal number representation doesn't change, however. **Spark SQL**: ```sql // bin/spark-sql spark-sql> select cast(1 as decimal(38, 18)); 1 spark-sql> // bin/beeline 0: jdbc:hive2://localhost:10000/default> select cast(1 as decimal(38, 18)); +----------------------------+--+ | CAST(1 AS DECIMAL(38,18)) | +----------------------------+--+ | 1.000000000000000000 | +----------------------------+--+ // bin/spark-shell scala> spark.sql("select cast(1 as decimal(38, 18))").show(false) +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ |1.000000000000000000 | +-------------------------+ // bin/pyspark >>> spark.sql("select cast(1 as decimal(38, 18))").show() +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ | 1.000000000000000000| +-------------------------+ // bin/sparkR > showDF(sql("SELECT cast(1 as decimal(38, 18))")) +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ | 1.000000000000000000| +-------------------------+ ``` **PostgreSQL**: ```sql postgres=# select cast(1 as decimal(38, 18)); numeric ---------------------- 1.000000000000000000 (1 row) ``` **Presto**: ```sql presto> select cast(1 as decimal(38, 18)); _col0 ---------------------- 1.000000000000000000 (1 row) ``` ## How was this patch tested? unit tests and manual test: ```sql spark-sql> select cast(1 as decimal(38, 18)); 1.000000000000000000 ``` Spark SQL Upgrading Guide: ![image](https://user-images.githubusercontent.com/5399861/69649620-4405c380-10a8-11ea-84b1-6ee675663b98.png) Closes #25214 from wangyum/SPARK-28461. Authored-by: Yuming Wang Signed-off-by: HyukjinKwon --- docs/sql-migration-guide.md | 26 ++ .../aggregate/ApproximatePercentile.scala | 2 +- .../expressions/conditionalExpressions.scala | 4 +- .../spark/sql/execution/HiveResult.scala | 10 +- .../ansi/decimalArithmeticOperations.sql.out | 18 +- .../sql-tests/results/ansi/literals.sql.out | 4 +- .../decimalArithmeticOperations.sql.out | 46 +-- .../sql-tests/results/literals.sql.out | 4 +- .../results/order-by-nulls-ordering.sql.out | 48 +-- .../results/postgreSQL/create_view.sql.out | 2 +- .../sql-tests/results/postgreSQL/date.sql.out | 4 +- .../sql-tests/results/postgreSQL/int2.sql.out | 14 +- .../sql-tests/results/postgreSQL/int4.sql.out | 14 +- .../sql-tests/results/postgreSQL/int8.sql.out | 14 +- .../results/postgreSQL/numeric.sql.out | 384 +++++++++--------- .../results/postgreSQL/select.sql.out | 20 +- .../results/postgreSQL/timestamp.sql.out | 20 +- .../results/postgreSQL/union.sql.out | 16 +- .../results/postgreSQL/window_part4.sql.out | 8 +- ...in-unit-tests-multi-column-literal.sql.out | 4 +- .../not-in-unit-tests-multi-column.sql.out | 10 +- ...n-unit-tests-single-column-literal.sql.out | 2 +- .../not-in-unit-tests-single-column.sql.out | 18 +- .../sql-tests/results/table-aliases.sql.out | 2 +- .../native/decimalPrecision.sql.out | 80 ++-- .../typeCoercion/native/division.sql.out | 18 +- .../sql-tests/results/udf/udf-union.sql.out | 12 +- .../resources/sql-tests/results/union.sql.out | 12 +- .../spark/sql/execution/HiveResultSuite.scala | 15 + .../sql/hive/thriftserver/CliSuite.scala | 7 + ...cimal_4-6-693c2e345731f9b2b547c3b75218458e | 72 ++-- ...cimal_4-7-f1eb45492510cb76cf6b452121af8531 | 72 ++-- ..._regex-10-c5b3ec90419a40660e5f83736241c429 | 74 ++-- ...nistic)-2-1e88e0ba414a00195f7ebf6b8600ac04 | 192 ++++----- ...istic) 2-0-81bb7f49a55385878637c8aac4d08e5 | 242 +++++------ ...stic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 | 164 ++++---- 36 files changed, 847 insertions(+), 807 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 74ba694800d2d..12d734fcbdf52 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -226,6 +226,32 @@ license: | - Since Spark 3.0, when casting string value to date, timestamp and interval values, the leading and trailing white spaces(<= ACSII 32) will be trimmed before casing, e.g. `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, only the trailing space will be removed, thus, the result is `null`. + - Since Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example: + + + + + + + + + + + +
    + Query + + Spark 2.4 or Prior + + Spark 3.0 +
    + SELECT CAST(1 AS decimal(38, 18)); + + 1 + + 1.000000000000000000 +
    + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index 59481ce049165..ea0ed2e8fa11b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -63,7 +63,7 @@ import org.apache.spark.sql.types._ > SELECT _FUNC_(10.0, array(0.5, 0.4, 0.1), 100); [10.0,10.0,10.0] > SELECT _FUNC_(10.0, 0.5, 100); - 10 + 10.0 """, since = "2.1.0") case class ApproximatePercentile( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 3a2dc40862bb8..6c6210994954c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -111,9 +111,9 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi examples = """ Examples: > SELECT CASE WHEN 1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END; - 1 + 1.0 > SELECT CASE WHEN 1 < 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END; - 2 + 2.0 > SELECT CASE WHEN 1 < 0 THEN 1 WHEN 2 < 0 THEN 2.0 END; NULL """) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index d4e10b3ffc733..f7f7e08462fe1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -59,14 +59,6 @@ object HiveResult { result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) } - private def formatDecimal(d: java.math.BigDecimal): String = { - if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { - java.math.BigDecimal.ZERO.toPlainString - } else { - d.stripTrailingZeros().toPlainString // Hive strips trailing zeros - } - } - private val primitiveTypes = Seq( StringType, IntegerType, @@ -127,7 +119,7 @@ object HiveResult { case (t: Timestamp, TimestampType) => DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) - case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) + case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString case (interval: CalendarInterval, CalendarIntervalType) => SQLConf.get.intervalOutputStyle match { case SQL_STANDARD => toSqlStandardString(interval) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index e12f409f6d47e..67cb0a2b36a92 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -24,10 +24,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 2 schema struct -- !query 2 output -1 1000 99.9 -2 123451.23 1234.5123 -3 1.234567891011 123.41 -4 1234567891234567890 0.112345678912345679 +1 1000.000000000000000 99.900000000000000000 +2 123451.230000000000000 1234.512300000000000000 +3 1.234567891011000 123.410000000000000000 +4 1234567891234567890.000000000000000 0.112345678912345679 -- !query 3 @@ -35,7 +35,7 @@ select 10.3 * 3.0 -- !query 3 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 3 output -30.9 +30.90 -- !query 4 @@ -43,7 +43,7 @@ select 10.3000 * 3.0 -- !query 4 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 4 output -30.9 +30.90000 -- !query 5 @@ -51,7 +51,7 @@ select 10.30000 * 30.0 -- !query 5 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 5 output -309 +309.000000 -- !query 6 @@ -59,7 +59,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 6 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> -- !query 6 output -30.9 +30.9000000000000000000000000000000000 -- !query 7 @@ -67,7 +67,7 @@ select 10.300000000000000000 * 3.0000000000000000000 -- !query 7 schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> -- !query 7 output -30.9 +30.9000000000000000000000000000000000 -- !query 8 diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index c1488eaf2aa36..5d8a893d334c2 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -169,7 +169,7 @@ select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query 17 schema struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> -- !query 17 output -1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90 90 +1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90.0 90 -- !query 18 @@ -339,7 +339,7 @@ select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD -- !query 35 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> -- !query 35 output -90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 +90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 -- !query 36 diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index cbf44548b3cce..6f035a1904ea3 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -56,9 +56,9 @@ select id, a+b, a-b, a*b, a/b from decimals_test order by id -- !query 6 schema struct -- !query 6 output -1 1099 -899 99900 0.1001 -2 24690.246 0 152402061.885129 1 -3 1234.2234567891011 -1233.9765432108989 152.358023 0.0001 +1 1099.00000000000000000 -899.00000000000000000 99900.000000 0.100100 +2 24690.24600000000000000 0.00000000000000000 152402061.885129 1.000000 +3 1234.22345678910110000 -1233.97654321089890000 152.358023 0.000100 4 123456789123456790.12345678912345679 123456789123456787.87654321087654321 138698367904130467.515623 109890109097814272.043109 @@ -67,10 +67,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 7 schema struct -- !query 7 output -1 1000 99.9 -2 123451.23 1234.5123 -3 1.234567891011 123.41 -4 1234567891234567890 0.112345678912345679 +1 1000.000000000000000 99.900000000000000000 +2 123451.230000000000000 1234.512300000000000000 +3 1.234567891011000 123.410000000000000000 +4 1234567891234567890.000000000000000 0.112345678912345679 -- !query 8 @@ -78,7 +78,7 @@ select 10.3 * 3.0 -- !query 8 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 8 output -30.9 +30.90 -- !query 9 @@ -86,7 +86,7 @@ select 10.3000 * 3.0 -- !query 9 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 9 output -30.9 +30.90000 -- !query 10 @@ -94,7 +94,7 @@ select 10.30000 * 30.0 -- !query 10 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 10 output -309 +309.000000 -- !query 11 @@ -102,7 +102,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 11 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> -- !query 11 output -30.9 +30.9000000000000000000000000000000000 -- !query 12 @@ -110,7 +110,7 @@ select 10.300000000000000000 * 3.0000000000000000000 -- !query 12 schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> -- !query 12 output -30.9 +30.9000000000000000000000000000000000 -- !query 13 @@ -198,9 +198,9 @@ select id, a+b, a-b, a*b, a/b from decimals_test order by id -- !query 23 schema struct -- !query 23 output -1 1099 -899 NULL 0.1001001001001001 -2 24690.246 0 NULL 1 -3 1234.2234567891011 -1233.9765432108989 NULL 0.000100037913541123 +1 1099.000000000000000000 -899.000000000000000000 NULL 0.100100100100100100 +2 24690.246000000000000000 0.000000000000000000 NULL 1.000000000000000000 +3 1234.223456789101100000 -1233.976543210898900000 NULL 0.000100037913541123 4 123456789123456790.123456789123456789 123456789123456787.876543210876543211 NULL 109890109097814272.043109406191131436 @@ -209,10 +209,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 24 schema struct -- !query 24 output -1 1000 99.9 -2 123451.23 1234.5123 -3 1.234567891011 123.41 -4 1234567891234567890 0.1123456789123456789 +1 1000.000000000000000000 99.9000000000000000000 +2 123451.230000000000000000 1234.5123000000000000000 +3 1.234567891011000000 123.4100000000000000000 +4 1234567891234567890.000000000000000000 0.1123456789123456789 -- !query 25 @@ -220,7 +220,7 @@ select 10.3 * 3.0 -- !query 25 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 25 output -30.9 +30.90 -- !query 26 @@ -228,7 +228,7 @@ select 10.3000 * 3.0 -- !query 26 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 26 output -30.9 +30.90000 -- !query 27 @@ -236,7 +236,7 @@ select 10.30000 * 30.0 -- !query 27 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 27 output -309 +309.000000 -- !query 28 @@ -244,7 +244,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 28 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,36)> -- !query 28 output -30.9 +30.900000000000000000000000000000000000 -- !query 29 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index c1488eaf2aa36..5d8a893d334c2 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -169,7 +169,7 @@ select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query 17 schema struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> -- !query 17 output -1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90 90 +1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90.0 90 -- !query 18 @@ -339,7 +339,7 @@ select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD -- !query 35 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> -- !query 35 output -90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 +90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 -- !query 36 diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out index c1b63dfb8caef..e1289dd8c8bde 100644 --- a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out @@ -206,14 +206,14 @@ select * from spark_10747_mix order by col1 nulls last, col5 nulls last -- !query 13 schema struct -- !query 13 output -a 1 1.0 1 NULL -b 2 1.0 1 10 -c 3 2.0 2 15.1 -c 3 2.0 2 NULL -d 3 2.0 3 0 -d 3 0.0 3 1 -d 3 NULL 4 1 -NULL 3 0.0 3 1 +a 1 1.0 1.00 NULL +b 2 1.0 1.00 10.0 +c 3 2.0 2.00 15.1 +c 3 2.0 2.00 NULL +d 3 2.0 3.00 0.0 +d 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +NULL 3 0.0 3.00 1.0 -- !query 14 @@ -221,14 +221,14 @@ select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls fi -- !query 14 schema struct -- !query 14 output -NULL 3 0.0 3 1 -d 3 0.0 3 1 -d 3 NULL 4 1 -d 3 2.0 3 0 -c 3 2.0 2 NULL -c 3 2.0 2 15.1 -b 2 1.0 1 10 -a 1 1.0 1 NULL +NULL 3 0.0 3.00 1.0 +d 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +d 3 2.0 3.00 0.0 +c 3 2.0 2.00 NULL +c 3 2.0 2.00 15.1 +b 2 1.0 1.00 10.0 +a 1 1.0 1.00 NULL -- !query 15 @@ -236,14 +236,14 @@ select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls la -- !query 15 schema struct -- !query 15 output -c 3 2.0 2 NULL -a 1 1.0 1 NULL -c 3 2.0 2 15.1 -b 2 1.0 1 10 -d 3 0.0 3 1 -NULL 3 0.0 3 1 -d 3 NULL 4 1 -d 3 2.0 3 0 +c 3 2.0 2.00 NULL +a 1 1.0 1.00 NULL +c 3 2.0 2.00 15.1 +b 2 1.0 1.00 10.0 +d 3 0.0 3.00 1.0 +NULL 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +d 3 2.0 3.00 0.0 -- !query 16 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index 031918961df4c..cd4d3e64b8da1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -956,7 +956,7 @@ SELECT * FROM tt1 struct -- !query 83 output 0123456789 abc 42.12 abc -abc 0123456789 42 abcd +abc 0123456789 42.00 abcd -- !query 84 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index 68ff2a9f68899..5371e07423c28 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -576,7 +576,7 @@ SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') -- !query 54 schema struct -- !query 54 output -0 +0.000000 -- !query 55 @@ -584,7 +584,7 @@ SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 55 schema struct -- !query 55 output -0 +0.000000 -- !query 56 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out index 8f41accfc8c49..d0a14618a5163 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -346,13 +346,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 33 schema struct -- !query 33 output --0.5 0 --1.5 -1 --2.5 -2 -0 0 -0.5 0 -1.5 1 -2.5 2 +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 -- !query 34 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index 6b1a8eb4a5883..30afd6e695842 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -495,13 +495,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 51 schema struct -- !query 51 output --0.5 0 --1.5 -1 --2.5 -2 -0 0 -0.5 0 -1.5 1 -2.5 2 +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 -- !query 52 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index c4d8affe75267..7a8e706cb8d85 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -836,13 +836,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 83 schema struct -- !query 83 output --0.5 0 --1.5 -1 --2.5 -2 -0 0 -0.5 0 -1.5 1 -2.5 2 +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 -- !query 84 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 5ffa29a93d5de..e245af28508c1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -3691,16 +3691,16 @@ SELECT * FROM num_data -- !query 460 schema struct -- !query 460 output -0 0 -1 0 -2 -34338492.215397047 -3 4.31 -4 7799461.4119 -5 16397.038491 -6 93901.57763026 -7 -83028485 -8 74881 -9 -24926804.04504742 +0 0.0000000000 +1 0.0000000000 +2 -34338492.2153970470 +3 4.3100000000 +4 7799461.4119000000 +5 16397.0384910000 +6 93901.5776302600 +7 -83028485.0000000000 +8 74881.0000000000 +9 -24926804.0450474200 -- !query 461 @@ -3869,51 +3869,51 @@ SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected -- !query 478 schema struct -- !query 478 output -2 2 1179132047626883.596862 1179132047626883.5968621359 -2 3 -147998901.448361 -147998901.4483612726 -2 4 -267821744976817.811114 -267821744976817.8111137107 -2 5 -563049578578.769243 -563049578578.7692425067 -2 6 -3224438592470.184498 -3224438592470.1844981193 -2 7 2851072985828710.485884 2851072985828710.485883795 -2 8 -2571300635581.146276 -2571300635581.146276407 -2 9 855948866655588.453742 855948866655588.4537415092 -3 2 -147998901.448361 -147998901.4483612726 -3 5 70671.235896 70671.23589621 -3 6 404715.799586 404715.7995864206 -3 9 -107434525.434154 -107434525.4341543802 -4 2 -267821744976817.811114 -267821744976817.8111137107 -4 4 60831598315717.141462 60831598315717.14146161 -4 5 127888068979.993505 127888068979.9935054429 -4 6 732381731243.745116 732381731243.7451157641 -4 9 -194415646271340.181596 -194415646271340.1815956523 -5 2 -563049578578.769243 -563049578578.7692425067 -5 3 70671.235896 70671.23589621 -5 4 127888068979.993505 127888068979.9935054429 -5 5 268862871.275336 268862871.2753355571 -5 6 1539707782.768998 1539707782.7689977863 -5 9 -408725765384.257044 -408725765384.2570436602 -6 2 -3224438592470.184498 -3224438592470.1844981193 -6 3 404715.799586 404715.7995864206 -6 4 732381731243.745116 732381731243.7451157641 -6 5 1539707782.768998 1539707782.7689977863 -6 6 8817506281.451745 8817506281.4517452373 -6 7 -7796505729750.377956 -7796505729750.3779561 -6 8 7031444034.531499 7031444034.53149906 -6 9 -2340666225110.299295 -2340666225110.2992952129 -7 2 2851072985828710.485884 2851072985828710.485883795 -7 6 -7796505729750.377956 -7796505729750.3779561 -7 9 2069634775752159.035759 2069634775752159.0357587 -8 2 -2571300635581.146276 -2571300635581.146276407 -8 6 7031444034.531499 7031444034.53149906 -8 9 -1866544013697.195857 -1866544013697.19585702 -9 2 855948866655588.453742 855948866655588.4537415092 -9 3 -107434525.434154 -107434525.4341543802 -9 4 -194415646271340.181596 -194415646271340.1815956523 -9 5 -408725765384.257044 -408725765384.2570436602 -9 6 -2340666225110.299295 -2340666225110.2992952129 -9 7 2069634775752159.035759 2069634775752159.0357587 -9 8 -1866544013697.195857 -1866544013697.19585702 -9 9 621345559900192.420121 621345559900192.42012063 +2 2 1179132047626883.5968620000 1179132047626883.5968621359 +2 3 -147998901.4483610000 -147998901.4483612726 +2 4 -267821744976817.8111140000 -267821744976817.8111137107 +2 5 -563049578578.7692430000 -563049578578.7692425067 +2 6 -3224438592470.1844980000 -3224438592470.1844981193 +2 7 2851072985828710.4858840000 2851072985828710.4858837950 +2 8 -2571300635581.1462760000 -2571300635581.1462764070 +2 9 855948866655588.4537420000 855948866655588.4537415092 +3 2 -147998901.4483610000 -147998901.4483612726 +3 5 70671.2358960000 70671.2358962100 +3 6 404715.7995860000 404715.7995864206 +3 9 -107434525.4341540000 -107434525.4341543802 +4 2 -267821744976817.8111140000 -267821744976817.8111137107 +4 4 60831598315717.1414620000 60831598315717.1414616100 +4 5 127888068979.9935050000 127888068979.9935054429 +4 6 732381731243.7451160000 732381731243.7451157641 +4 9 -194415646271340.1815960000 -194415646271340.1815956523 +5 2 -563049578578.7692430000 -563049578578.7692425067 +5 3 70671.2358960000 70671.2358962100 +5 4 127888068979.9935050000 127888068979.9935054429 +5 5 268862871.2753360000 268862871.2753355571 +5 6 1539707782.7689980000 1539707782.7689977863 +5 9 -408725765384.2570440000 -408725765384.2570436602 +6 2 -3224438592470.1844980000 -3224438592470.1844981193 +6 3 404715.7995860000 404715.7995864206 +6 4 732381731243.7451160000 732381731243.7451157641 +6 5 1539707782.7689980000 1539707782.7689977863 +6 6 8817506281.4517450000 8817506281.4517452373 +6 7 -7796505729750.3779560000 -7796505729750.3779561000 +6 8 7031444034.5314990000 7031444034.5314990600 +6 9 -2340666225110.2992950000 -2340666225110.2992952129 +7 2 2851072985828710.4858840000 2851072985828710.4858837950 +7 6 -7796505729750.3779560000 -7796505729750.3779561000 +7 9 2069634775752159.0357590000 2069634775752159.0357587000 +8 2 -2571300635581.1462760000 -2571300635581.1462764070 +8 6 7031444034.5314990000 7031444034.5314990600 +8 9 -1866544013697.1958570000 -1866544013697.1958570200 +9 2 855948866655588.4537420000 855948866655588.4537415092 +9 3 -107434525.4341540000 -107434525.4341543802 +9 4 -194415646271340.1815960000 -194415646271340.1815956523 +9 5 -408725765384.2570440000 -408725765384.2570436602 +9 6 -2340666225110.2992950000 -2340666225110.2992952129 +9 7 2069634775752159.0357590000 2069634775752159.0357587000 +9 8 -1866544013697.1958570000 -1866544013697.1958570200 +9 9 621345559900192.4201210000 621345559900192.4201206300 -- !query 479 @@ -3942,62 +3942,62 @@ SELECT t1.id1, t1.id2, t1.result, t2.expected -- !query 481 schema struct -- !query 481 output -2 3 -7967167.567378 -7967167.5673775051 -2 4 -4.402675 -4.4026748005 -2 5 -2094.188669 -2094.1886691456 -2 6 -365.685999 -365.6859989148 -2 7 0.413575 0.4135748378 -2 8 -458.574167 -458.5741672173 -2 9 1.377573 1.3775729995 -3 2 0 -0.0000001255 -3 4 0.000001 0.0000005526 -3 5 0.000263 0.0002628523 -3 6 0.000046 0.0000458991 -3 7 0 -0.0000000519 -3 8 0.000058 0.000057558 -3 9 0 -0.0000001729 -4 2 -0.227135 -0.22713465 -4 3 1809619.817146 1809619.8171461717 -4 5 475.66281 475.6628104631 -4 6 83.059961 83.0599613844 -4 7 -0.093937 -0.093937176 -4 8 104.158083 104.1580829837 -4 9 -0.312895 -0.3128945611 -5 2 -0.000478 -0.0004775119 -5 3 3804.417283 3804.4172832947 -5 4 0.002102 0.0021023296 -5 6 0.174619 0.1746194143 -5 7 -0.000197 -0.0001974869 -5 8 0.218975 0.2189746196 -5 9 -0.000658 -0.0006578075 -6 2 -0.002735 -0.0027345865 -6 3 21786.908963 21786.9089629374 -6 4 0.012039 0.0120394951 -6 5 5.72674 5.7267400867 -6 7 -0.001131 -0.0011309562 -6 8 1.254011 1.2540107321 -6 9 -0.003767 -0.0037670925 -7 2 2.417942 2.4179420715 -7 3 -19264149.651972 -19264149.6519721578 -7 4 -10.645413 -10.6454126273 -7 5 -5063.626889 -5063.6268888173 -7 6 -884.207562 -884.2075617401 -7 8 -1108.805772 -1108.8057718246 -7 9 3.330892 3.330891712 -8 2 -0.002181 -0.0021806723 -8 3 17373.781903 17373.7819025522 -8 4 0.009601 0.0096007911 -8 5 4.566739 4.5667392951 -8 6 0.797441 0.7974413411 -8 7 -0.000902 -0.0009018712 -8 9 -0.003004 -0.0030040353 -9 2 0.725914 0.7259143438 -9 3 -5783481.216948 -5783481.2169483573 -9 4 -3.195965 -3.1959647889 -9 5 -1520.201594 -1520.2015936432 -9 6 -265.456712 -265.4567119543 -9 7 0.30022 0.300219907 -9 8 -332.885566 -332.8855656982 +2 3 -7967167.5673780000 -7967167.5673775051 +2 4 -4.4026750000 -4.4026748005 +2 5 -2094.1886690000 -2094.1886691456 +2 6 -365.6859990000 -365.6859989148 +2 7 0.4135750000 0.4135748378 +2 8 -458.5741670000 -458.5741672173 +2 9 1.3775730000 1.3775729995 +3 2 0.0000000000 -0.0000001255 +3 4 0.0000010000 0.0000005526 +3 5 0.0002630000 0.0002628523 +3 6 0.0000460000 0.0000458991 +3 7 0.0000000000 -0.0000000519 +3 8 0.0000580000 0.0000575580 +3 9 0.0000000000 -0.0000001729 +4 2 -0.2271350000 -0.2271346500 +4 3 1809619.8171460000 1809619.8171461717 +4 5 475.6628100000 475.6628104631 +4 6 83.0599610000 83.0599613844 +4 7 -0.0939370000 -0.0939371760 +4 8 104.1580830000 104.1580829837 +4 9 -0.3128950000 -0.3128945611 +5 2 -0.0004780000 -0.0004775119 +5 3 3804.4172830000 3804.4172832947 +5 4 0.0021020000 0.0021023296 +5 6 0.1746190000 0.1746194143 +5 7 -0.0001970000 -0.0001974869 +5 8 0.2189750000 0.2189746196 +5 9 -0.0006580000 -0.0006578075 +6 2 -0.0027350000 -0.0027345865 +6 3 21786.9089630000 21786.9089629374 +6 4 0.0120390000 0.0120394951 +6 5 5.7267400000 5.7267400867 +6 7 -0.0011310000 -0.0011309562 +6 8 1.2540110000 1.2540107321 +6 9 -0.0037670000 -0.0037670925 +7 2 2.4179420000 2.4179420715 +7 3 -19264149.6519720000 -19264149.6519721578 +7 4 -10.6454130000 -10.6454126273 +7 5 -5063.6268890000 -5063.6268888173 +7 6 -884.2075620000 -884.2075617401 +7 8 -1108.8057720000 -1108.8057718246 +7 9 3.3308920000 3.3308917120 +8 2 -0.0021810000 -0.0021806723 +8 3 17373.7819030000 17373.7819025522 +8 4 0.0096010000 0.0096007911 +8 5 4.5667390000 4.5667392951 +8 6 0.7974410000 0.7974413411 +8 7 -0.0009020000 -0.0009018712 +8 9 -0.0030040000 -0.0030040353 +9 2 0.7259140000 0.7259143438 +9 3 -5783481.2169480000 -5783481.2169483573 +9 4 -3.1959650000 -3.1959647889 +9 5 -1520.2015940000 -1520.2015936432 +9 6 -265.4567120000 -265.4567119543 +9 7 0.3002200000 0.3002199070 +9 8 -332.8855660000 -332.8855656982 -- !query 482 @@ -4026,62 +4026,62 @@ SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected -- !query 484 schema struct -- !query 484 output -2 3 -7967167.567378 -7967167.5673775051 -2 4 -4.402675 -4.4026748005 -2 5 -2094.188669 -2094.1886691456 -2 6 -365.685999 -365.6859989148 -2 7 0.413575 0.4135748378 -2 8 -458.574167 -458.5741672173 -2 9 1.377573 1.3775729995 -3 2 0 -0.0000001255 -3 4 0.000001 0.0000005526 -3 5 0.000263 0.0002628523 -3 6 0.000046 0.0000458991 -3 7 0 -0.0000000519 -3 8 0.000058 0.000057558 -3 9 0 -0.0000001729 -4 2 -0.227135 -0.22713465 -4 3 1809619.817146 1809619.8171461717 -4 5 475.66281 475.6628104631 -4 6 83.059961 83.0599613844 -4 7 -0.093937 -0.093937176 -4 8 104.158083 104.1580829837 -4 9 -0.312895 -0.3128945611 -5 2 -0.000478 -0.0004775119 -5 3 3804.417283 3804.4172832947 -5 4 0.002102 0.0021023296 -5 6 0.174619 0.1746194143 -5 7 -0.000197 -0.0001974869 -5 8 0.218975 0.2189746196 -5 9 -0.000658 -0.0006578075 -6 2 -0.002735 -0.0027345865 -6 3 21786.908963 21786.9089629374 -6 4 0.012039 0.0120394951 -6 5 5.72674 5.7267400867 -6 7 -0.001131 -0.0011309562 -6 8 1.254011 1.2540107321 -6 9 -0.003767 -0.0037670925 -7 2 2.417942 2.4179420715 -7 3 -19264149.651972 -19264149.6519721578 -7 4 -10.645413 -10.6454126273 -7 5 -5063.626889 -5063.6268888173 -7 6 -884.207562 -884.2075617401 -7 8 -1108.805772 -1108.8057718246 -7 9 3.330892 3.330891712 -8 2 -0.002181 -0.0021806723 -8 3 17373.781903 17373.7819025522 -8 4 0.009601 0.0096007911 -8 5 4.566739 4.5667392951 -8 6 0.797441 0.7974413411 -8 7 -0.000902 -0.0009018712 -8 9 -0.003004 -0.0030040353 -9 2 0.725914 0.7259143438 -9 3 -5783481.216948 -5783481.2169483573 -9 4 -3.195965 -3.1959647889 -9 5 -1520.201594 -1520.2015936432 -9 6 -265.456712 -265.4567119543 -9 7 0.30022 0.300219907 -9 8 -332.885566 -332.8855656982 +2 3 -7967167.5673780000 -7967167.5673775051 +2 4 -4.4026750000 -4.4026748005 +2 5 -2094.1886690000 -2094.1886691456 +2 6 -365.6859990000 -365.6859989148 +2 7 0.4135750000 0.4135748378 +2 8 -458.5741670000 -458.5741672173 +2 9 1.3775730000 1.3775729995 +3 2 0.0000000000 -0.0000001255 +3 4 0.0000010000 0.0000005526 +3 5 0.0002630000 0.0002628523 +3 6 0.0000460000 0.0000458991 +3 7 0.0000000000 -0.0000000519 +3 8 0.0000580000 0.0000575580 +3 9 0.0000000000 -0.0000001729 +4 2 -0.2271350000 -0.2271346500 +4 3 1809619.8171460000 1809619.8171461717 +4 5 475.6628100000 475.6628104631 +4 6 83.0599610000 83.0599613844 +4 7 -0.0939370000 -0.0939371760 +4 8 104.1580830000 104.1580829837 +4 9 -0.3128950000 -0.3128945611 +5 2 -0.0004780000 -0.0004775119 +5 3 3804.4172830000 3804.4172832947 +5 4 0.0021020000 0.0021023296 +5 6 0.1746190000 0.1746194143 +5 7 -0.0001970000 -0.0001974869 +5 8 0.2189750000 0.2189746196 +5 9 -0.0006580000 -0.0006578075 +6 2 -0.0027350000 -0.0027345865 +6 3 21786.9089630000 21786.9089629374 +6 4 0.0120390000 0.0120394951 +6 5 5.7267400000 5.7267400867 +6 7 -0.0011310000 -0.0011309562 +6 8 1.2540110000 1.2540107321 +6 9 -0.0037670000 -0.0037670925 +7 2 2.4179420000 2.4179420715 +7 3 -19264149.6519720000 -19264149.6519721578 +7 4 -10.6454130000 -10.6454126273 +7 5 -5063.6268890000 -5063.6268888173 +7 6 -884.2075620000 -884.2075617401 +7 8 -1108.8057720000 -1108.8057718246 +7 9 3.3308920000 3.3308917120 +8 2 -0.0021810000 -0.0021806723 +8 3 17373.7819030000 17373.7819025522 +8 4 0.0096010000 0.0096007911 +8 5 4.5667390000 4.5667392951 +8 6 0.7974410000 0.7974413411 +8 7 -0.0009020000 -0.0009018712 +8 9 -0.0030040000 -0.0030040353 +9 2 0.7259140000 0.7259143438 +9 3 -5783481.2169480000 -5783481.2169483573 +9 4 -3.1959650000 -3.1959647889 +9 5 -1520.2015940000 -1520.2015936432 +9 6 -265.4567120000 -265.4567119543 +9 7 0.3002200000 0.3002199070 +9 8 -332.8855660000 -332.8855656982 -- !query 485 @@ -4196,12 +4196,12 @@ SELECT t1.id1, t1.result, t2.expected -- !query 496 schema struct -- !query 496 output -2 224790267919917440 224790267919917955.1326161858 -4 7405685069595001 7405685069594999.0773399947 -5 5068226527.321263 5068226527.3212726541 -6 281839893606.99365 281839893606.9937234336 -8 167361463828.0749 167361463828.0749132007 -9 107511333880051872 107511333880052007.0414112467 +2 224790267919917440.0000000000 224790267919917955.1326161858 +4 7405685069595001.0000000000 7405685069594999.0773399947 +5 5068226527.3212630000 5068226527.3212726541 +6 281839893606.9936500000 281839893606.9937234336 +8 167361463828.0749000000 167361463828.0749132007 +9 107511333880051872.0000000000 107511333880052007.0414112467 -- !query 497 @@ -4209,7 +4209,7 @@ SELECT AVG(val) FROM num_data -- !query 497 schema struct -- !query 497 output --13430913.5922423207 +-13430913.59224232070000 -- !query 498 @@ -4273,11 +4273,11 @@ SELECT * FROM fract_only -- !query 505 schema struct -- !query 505 output -1 0 -2 0.1 +1 0.0000 +2 0.1000 4 -0.9999 5 0.9999 -7 0 +7 0.0000 8 0.0002 @@ -4406,13 +4406,13 @@ SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round -- !query 521 schema struct -- !query 521 output --0.000001 0 0 -1 0 --5.499999 -5 -5 -6 -5 --5.5 -5 -5 -6 -6 -0 0 0 0 0 -0.0000001 1 1 0 0 -9.4999999 10 10 9 9 -9.5 10 10 9 10 +-0.000001000000000000 0 0 -1 0 +-5.499999000000000000 -5 -5 -6 -5 +-5.500000000000000000 -5 -5 -6 -6 +0.000000000000000000 0 0 0 0 +0.000000100000000000 1 1 0 0 +9.499999900000000000 10 10 9 9 +9.500000000000000000 10 10 9 10 -- !query 522 @@ -4476,11 +4476,11 @@ SELECT * FROM num_input_test -- !query 529 schema struct -- !query 529 output --555.5 --93853 -123 -3245874 -555.5 +-555.500000000000000000 +-93853.000000000000000000 +123.000000000000000000 +3245874.000000000000000000 +555.500000000000000000 -- !query 530 @@ -4488,7 +4488,7 @@ select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 -- !query 530 schema struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) / CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,6)> -- !query 530 output -1 +1.000000 -- !query 531 @@ -4536,7 +4536,7 @@ select mod (70.0,70) -- !query 536 schema struct<(CAST(70.0 AS DECIMAL(3,1)) % CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(3,1)> -- !query 536 output -0 +0.0 -- !query 537 @@ -4552,7 +4552,7 @@ select 70.0 / 70 -- !query 538 schema struct<(CAST(70.0 AS DECIMAL(3,1)) / CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(8,6)> -- !query 538 output -1 +1.000000 -- !query 539 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out index e54de1d6fdbdc..a106e33c2a595 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out @@ -336,8 +336,8 @@ VALUES (1,2), (3,4+4), (7,77.7) -- !query 17 schema struct -- !query 17 output -1 2 -3 8 +1 2.0 +3 8.0 7 77.7 @@ -350,14 +350,14 @@ TABLE int8_tbl -- !query 18 schema struct -- !query 18 output -1 2 -123 456 -123 4567890123456789 -3 8 -4 57 -4567890123456789 -4567890123456789 -4567890123456789 123 -4567890123456789 4567890123456789 +1 2.0 +123 456.0 +123 4567890123456789.0 +3 8.0 +4 57.0 +4567890123456789 -4567890123456789.0 +4567890123456789 123.0 +4567890123456789 4567890123456789.0 7 77.7 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out index 6cb3690a464e1..460b6d1fdd02f 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out @@ -258,11 +258,11 @@ SELECT '' AS `54`, d1 as `timestamp`, -- !query 27 schema struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:decimal(8,6)> -- !query 27 output - 1969-12-31 16:00:00 1969 12 31 16 0 0 - 1997-01-02 00:00:00 1997 1 2 0 0 0 - 1997-01-02 03:04:05 1997 1 2 3 4 5 - 1997-02-10 17:32:01 1997 2 10 17 32 1 - 2001-09-22 18:19:20 2001 9 22 18 19 20 + 1969-12-31 16:00:00 1969 12 31 16 0 0.000000 + 1997-01-02 00:00:00 1997 1 2 0 0 0.000000 + 1997-01-02 03:04:05 1997 1 2 3 4 5.000000 + 1997-02-10 17:32:01 1997 2 10 17 32 1.000000 + 2001-09-22 18:19:20 2001 9 22 18 19 20.000000 -- !query 28 @@ -273,11 +273,11 @@ SELECT '' AS `54`, d1 as `timestamp`, -- !query 28 schema struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> -- !query 28 output - 1969-12-31 16:00:00 4 0 0 - 1997-01-02 00:00:00 1 0 0 - 1997-01-02 03:04:05 1 5000 5000000 - 1997-02-10 17:32:01 1 1000 1000000 - 2001-09-22 18:19:20 3 20000 20000000 + 1969-12-31 16:00:00 4 0.000 0 + 1997-01-02 00:00:00 1 0.000 0 + 1997-01-02 03:04:05 1 5000.000 5000000 + 1997-02-10 17:32:01 1 1000.000 1000000 + 2001-09-22 18:19:20 3 20000.000 20000000 -- !query 29 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out index 05dedc547086e..703463e6d6b5a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out @@ -117,7 +117,7 @@ SELECT 1.1 AS two UNION SELECT 2 ORDER BY 1 struct -- !query 11 output 1.1 -2 +2.0 -- !query 12 @@ -125,7 +125,7 @@ SELECT 1 AS two UNION SELECT 2.2 ORDER BY 1 -- !query 12 schema struct -- !query 12 output -1 +1.0 2.2 @@ -143,7 +143,7 @@ SELECT 1.1 AS two UNION ALL SELECT 2 ORDER BY 1 struct -- !query 14 output 1.1 -2 +2.0 -- !query 15 @@ -161,8 +161,8 @@ SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 struct -- !query 16 output 1.1 -2 -3 +2.0 +3.0 -- !query 17 @@ -180,8 +180,8 @@ SELECT 1.1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 struct -- !query 18 output 1.1 -2 -2 +2.0 +2.0 -- !query 19 @@ -190,7 +190,7 @@ SELECT 1.1 AS two UNION (SELECT 2 UNION ALL SELECT 2) ORDER BY 1 struct -- !query 19 output 1.1 -2 +2.0 -- !query 20 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index f4ada1e18a90a..54ceacd3b3b3e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -44,8 +44,8 @@ SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWIN -- !query 3 schema struct -- !query 3 output -1 2 -2 2.5 +1 2.00000 +2 2.50000 3 NULL 4 NULL @@ -104,8 +104,8 @@ SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) -- !query 8 schema struct -- !query 8 output -3 -5 +3.00 +5.00 6.01 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out index f02f760727976..3045cff1de19b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out @@ -23,7 +23,7 @@ WHERE b = 1.0 -- Matches (null, 1.0) -- !query 1 schema struct -- !query 1 output -NULL 1 +NULL 1.0 -- !query 2 @@ -45,4 +45,4 @@ WHERE b = 5.0 -- Matches (4, 5.0) -- !query 3 schema struct -- !query 3 output -4 5 +4 5.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out index a27a66e3f27f5..5ccea3846cd36 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out @@ -37,9 +37,9 @@ WHERE (a, b) NOT IN (SELECT * -- !query 2 schema struct -- !query 2 output -2 3 -4 5 -NULL 1 +2 3.0 +4 5.0 +NULL 1.0 NULL NULL @@ -91,7 +91,7 @@ WHERE b = 1.0 -- Matches (null, 1.0) -- !query 6 schema struct -- !query 6 output -NULL 1 +NULL 1.0 -- !query 7 @@ -117,4 +117,4 @@ WHERE b = 5.0 -- Matches (4, 5.0) -- !query 8 schema struct -- !query 8 output -4 5 +4 5.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out index cf8f03eaa9311..9190e9c1e7a4b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out @@ -54,4 +54,4 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 4 schema struct -- !query 4 output -2 3 +2 3.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out index d07981cfd11e5..d7473e1ada151 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out @@ -35,9 +35,9 @@ WHERE a NOT IN (SELECT c -- !query 2 schema struct -- !query 2 output -2 3 -4 5 -NULL 1 +2 3.0 +4 5.0 +NULL 1.0 -- !query 3 @@ -88,7 +88,7 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 6 schema struct -- !query 6 output -2 3 +2 3.0 -- !query 7 @@ -100,9 +100,9 @@ WHERE a NOT IN (SELECT c -- !query 7 schema struct -- !query 7 output -2 3 -4 5 -NULL 1 +2 3.0 +4 5.0 +NULL 1.0 -- !query 8 @@ -115,7 +115,7 @@ WHERE b = 1.0 -- Only matches (null, 1.0) -- !query 8 schema struct -- !query 8 output -NULL 1 +NULL 1.0 -- !query 9 @@ -128,4 +128,4 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 9 schema struct -- !query 9 output -2 3 +2 3.0 diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index 1a2bd5ea91cde..653dc1ca357f8 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -93,5 +93,5 @@ SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) struct -- !query 10 output 1 a 1 8.5 -2 b 2 1 +2 b 2 1.0 3 c 3 3.2 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index 6ee7f59d69877..e479a6d73a182 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -2211,7 +2211,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t -- !query 265 schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) / CAST(1 AS DECIMAL(3,0))):decimal(9,6)> -- !query 265 output -1 +1.000000 -- !query 266 @@ -2219,7 +2219,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t -- !query 266 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> -- !query 266 output -1 +1.000000 -- !query 267 @@ -2227,7 +2227,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t -- !query 267 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 267 output -1 +1.00000000000 -- !query 268 @@ -2235,7 +2235,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t -- !query 268 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> -- !query 268 output -1 +1.000000000000000000000 -- !query 269 @@ -2243,7 +2243,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t -- !query 269 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> -- !query 269 output -1 +1.000000 -- !query 270 @@ -2251,7 +2251,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t -- !query 270 schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) / CAST(1 AS DECIMAL(5,0))):decimal(11,6)> -- !query 270 output -1 +1.000000 -- !query 271 @@ -2259,7 +2259,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t -- !query 271 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 271 output -1 +1.00000000000 -- !query 272 @@ -2267,7 +2267,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t -- !query 272 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> -- !query 272 output -1 +1.000000000000000000000 -- !query 273 @@ -2275,7 +2275,7 @@ SELECT cast(1 as int) / cast(1 as decimal(3, 0)) FROM t -- !query 273 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 273 output -1 +1.000000 -- !query 274 @@ -2283,7 +2283,7 @@ SELECT cast(1 as int) / cast(1 as decimal(5, 0)) FROM t -- !query 274 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 274 output -1 +1.000000 -- !query 275 @@ -2291,7 +2291,7 @@ SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t -- !query 275 schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 275 output -1 +1.00000000000 -- !query 276 @@ -2299,7 +2299,7 @@ SELECT cast(1 as int) / cast(1 as decimal(20, 0)) FROM t -- !query 276 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 276 output -1 +1.000000000000000000000 -- !query 277 @@ -2307,7 +2307,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t -- !query 277 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 277 output -1 +1.000000 -- !query 278 @@ -2315,7 +2315,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t -- !query 278 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 278 output -1 +1.000000 -- !query 279 @@ -2323,7 +2323,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t -- !query 279 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 279 output -1 +1.00000000000 -- !query 280 @@ -2331,7 +2331,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t -- !query 280 schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) / CAST(1 AS DECIMAL(20,0))):decimal(38,18)> -- !query 280 output -1 +1.000000000000000000 -- !query 281 @@ -2403,7 +2403,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t -- !query 289 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 289 output -1 +1.000000 -- !query 290 @@ -2411,7 +2411,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t -- !query 290 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 290 output -1 +1.000000 -- !query 291 @@ -2419,7 +2419,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 291 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 291 output -1 +1.00000000000 -- !query 292 @@ -2427,7 +2427,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t -- !query 292 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 292 output -1 +1.000000000000000000000 -- !query 293 @@ -2543,7 +2543,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t -- !query 305 schema struct<(CAST(1 AS DECIMAL(3,0)) / CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(9,6)> -- !query 305 output -1 +1.000000 -- !query 306 @@ -2551,7 +2551,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t -- !query 306 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> -- !query 306 output -1 +1.000000 -- !query 307 @@ -2559,7 +2559,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t -- !query 307 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 307 output -1 +1.000000 -- !query 308 @@ -2567,7 +2567,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t -- !query 308 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 308 output -1 +1.000000 -- !query 309 @@ -2575,7 +2575,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t -- !query 309 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> -- !query 309 output -1 +1.000000 -- !query 310 @@ -2583,7 +2583,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t -- !query 310 schema struct<(CAST(1 AS DECIMAL(5,0)) / CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(11,6)> -- !query 310 output -1 +1.000000 -- !query 311 @@ -2591,7 +2591,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t -- !query 311 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 311 output -1 +1.000000 -- !query 312 @@ -2599,7 +2599,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t -- !query 312 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 312 output -1 +1.000000 -- !query 313 @@ -2607,7 +2607,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as int) FROM t -- !query 313 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 313 output -1 +1.00000000000 -- !query 314 @@ -2615,7 +2615,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as int) FROM t -- !query 314 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 314 output -1 +1.00000000000 -- !query 315 @@ -2623,7 +2623,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t -- !query 315 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> -- !query 315 output -1 +1.00000000000 -- !query 316 @@ -2631,7 +2631,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as int) FROM t -- !query 316 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 316 output -1 +1.00000000000 -- !query 317 @@ -2639,7 +2639,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t -- !query 317 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> -- !query 317 output -1 +1.000000000000000000000 -- !query 318 @@ -2647,7 +2647,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t -- !query 318 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> -- !query 318 output -1 +1.000000000000000000000 -- !query 319 @@ -2655,7 +2655,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t -- !query 319 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 319 output -1 +1.000000000000000000000 -- !query 320 @@ -2663,7 +2663,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t -- !query 320 schema struct<(CAST(1 AS DECIMAL(20,0)) / CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(38,18)> -- !query 320 output -1 +1.000000000000000000 -- !query 321 @@ -2735,7 +2735,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 329 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 329 output -1 +1.00000000000 -- !query 330 @@ -2743,7 +2743,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 330 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 330 output -1 +1.00000000000 -- !query 331 @@ -2751,7 +2751,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 331 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 331 output -1 +1.00000000000 -- !query 332 @@ -2759,7 +2759,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 332 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 332 output -1 +1.00000000000 -- !query 333 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out index 017e0fea30e90..97cdf01763a70 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out @@ -63,7 +63,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t -- !query 7 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 7 output -1 +1.00000000000 -- !query 8 @@ -163,7 +163,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t -- !query 19 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 19 output -1 +1.00000000000 -- !query 20 @@ -263,7 +263,7 @@ SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t -- !query 31 schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 31 output -1 +1.00000000000 -- !query 32 @@ -363,7 +363,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t -- !query 43 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 43 output -1 +1.00000000000 -- !query 44 @@ -615,7 +615,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t -- !query 73 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 73 output -1 +1.000000 -- !query 74 @@ -623,7 +623,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t -- !query 74 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 74 output -1 +1.000000 -- !query 75 @@ -631,7 +631,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t -- !query 75 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> -- !query 75 output -1 +1.00000000000 -- !query 76 @@ -639,7 +639,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t -- !query 76 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 76 output -1 +1.000000000000000000000 -- !query 77 @@ -663,7 +663,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 79 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 79 output -1 +1.00000000000 -- !query 80 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out index 84b5e10dbeb8e..835abe3d32775 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out @@ -42,12 +42,12 @@ FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 -- !query 3 schema struct -- !query 3 output -1 1 -1 1 -1 a -2 4 -2 4 -2 b +1.0 1 +1.0 1 +1.0 a +2.0 4 +2.0 4 +2.0 b -- !query 4 diff --git a/sql/core/src/test/resources/sql-tests/results/union.sql.out b/sql/core/src/test/resources/sql-tests/results/union.sql.out index b023df825d814..95a10f0b0a8c9 100644 --- a/sql/core/src/test/resources/sql-tests/results/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/union.sql.out @@ -42,12 +42,12 @@ FROM (SELECT * FROM t1 -- !query 3 schema struct -- !query 3 output -1 1 -1 1 -1 a -2 4 -2 4 -2 b +1.0 1 +1.0 1 +1.0 a +2.0 4 +2.0 4 +2.0 b -- !query 4 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index 104cf4c58d617..4d1bb470e4e2e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -43,4 +43,19 @@ class HiveResultSuite extends SharedSparkSession { val tpe = new ExamplePointUDT() assert(HiveResult.toHiveString((point, tpe)) === "(50.0, 50.0)") } + + test("decimal formatting in hive result") { + val df = Seq(new java.math.BigDecimal("1")).toDS() + Seq(2, 6, 18).foreach { scala => + val executedPlan = + df.selectExpr(s"CAST(value AS decimal(38, $scala))").queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan) + assert(result.head.split("\\.").last.length === scala) + } + + val executedPlan = Seq(java.math.BigDecimal.ZERO).toDS() + .selectExpr(s"CAST(value AS decimal(38, 8))").queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan) + assert(result.head === "0.00000000") + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 04e7f579ff712..856102bc900a8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -393,4 +393,11 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { "select concat('Test4', ';');" -> "Test4;" ) } + + test("Pad Decimal numbers with trailing zeros to the scale of the column") { + runCliWithin(1.minute)( + "SELECT CAST(1 AS DECIMAL(38, 18));" + -> "1.000000000000000000" + ) + } } diff --git a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e index f59549a6e4a46..a298a84cb2c5a 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e +++ b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e @@ -1,38 +1,38 @@ NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -1 --1.12 -1 --0.333 0 --0.33 0 --0.3 0 -0 0 -0 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 +-1234567890.1234567890000000000000000 -1234567890 +-4400.0000000000000000000000000 4400 +-1255.4900000000000000000000000 -1255 +-1.1220000000000000000000000 -11 +-1.1200000000000000000000000 -1 +-1.1200000000000000000000000 -1 +-0.3330000000000000000000000 0 +-0.3300000000000000000000000 0 +-0.3000000000000000000000000 0 +0.0000000000000000000000000 0 +0.0000000000000000000000000 0 +0.0000000000000000000000000 0 +0.0100000000000000000000000 0 +0.0200000000000000000000000 0 +0.1000000000000000000000000 0 +0.2000000000000000000000000 0 +0.3000000000000000000000000 0 +0.3300000000000000000000000 0 +0.3330000000000000000000000 0 0.9999999999999999999999999 1 -1 1 -1 1 -1.12 1 -1.122 1 -2 2 -2 2 -3.14 3 -3.14 3 -3.14 3 -3.14 4 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 +1.0000000000000000000000000 1 +1.0000000000000000000000000 1 +1.1200000000000000000000000 1 +1.1220000000000000000000000 1 +2.0000000000000000000000000 2 +2.0000000000000000000000000 2 +3.1400000000000000000000000 3 +3.1400000000000000000000000 3 +3.1400000000000000000000000 3 +3.1400000000000000000000000 4 +10.0000000000000000000000000 10 +20.0000000000000000000000000 20 +100.0000000000000000000000000 100 +124.0000000000000000000000000 124 +125.2000000000000000000000000 125 +200.0000000000000000000000000 200 +1234567890.1234567800000000000000000 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 index 6bada475c6d3d..60df68a2e3ab5 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 +++ b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 @@ -1,38 +1,38 @@ NULL NULL --1234567890.123456789 -3703703670.370370367 --4400 -13200 --1255.49 -3766.47 --1.122 -3.366 --1.12 -3.36 --1.12 -3.36 --0.333 -0.999 --0.33 -0.99 --0.3 -0.9 -0 0 -0 0 -0 0 -0.01 0.03 -0.02 0.06 -0.1 0.3 -0.2 0.6 -0.3 0.9 -0.33 0.99 -0.333 0.999 +-1234567890.1234567890000000000000000 -3703703670.3703703670000000000000000 +-4400.0000000000000000000000000 -13200.0000000000000000000000000 +-1255.4900000000000000000000000 -3766.4700000000000000000000000 +-1.1220000000000000000000000 -3.3660000000000000000000000 +-1.1200000000000000000000000 -3.3600000000000000000000000 +-1.1200000000000000000000000 -3.3600000000000000000000000 +-0.3330000000000000000000000 -0.9990000000000000000000000 +-0.3300000000000000000000000 -0.9900000000000000000000000 +-0.3000000000000000000000000 -0.9000000000000000000000000 +0.0000000000000000000000000 0.0000000000000000000000000 +0.0000000000000000000000000 0.0000000000000000000000000 +0.0000000000000000000000000 0.0000000000000000000000000 +0.0100000000000000000000000 0.0300000000000000000000000 +0.0200000000000000000000000 0.0600000000000000000000000 +0.1000000000000000000000000 0.3000000000000000000000000 +0.2000000000000000000000000 0.6000000000000000000000000 +0.3000000000000000000000000 0.9000000000000000000000000 +0.3300000000000000000000000 0.9900000000000000000000000 +0.3330000000000000000000000 0.9990000000000000000000000 0.9999999999999999999999999 2.9999999999999999999999997 -1 3 -1 3 -1.12 3.36 -1.122 3.366 -2 6 -2 6 -3.14 9.42 -3.14 9.42 -3.14 9.42 -3.14 9.42 -10 30 -20 60 -100 300 -124 372 -125.2 375.6 -200 600 -1234567890.12345678 3703703670.37037034 +1.0000000000000000000000000 3.0000000000000000000000000 +1.0000000000000000000000000 3.0000000000000000000000000 +1.1200000000000000000000000 3.3600000000000000000000000 +1.1220000000000000000000000 3.3660000000000000000000000 +2.0000000000000000000000000 6.0000000000000000000000000 +2.0000000000000000000000000 6.0000000000000000000000000 +3.1400000000000000000000000 9.4200000000000000000000000 +3.1400000000000000000000000 9.4200000000000000000000000 +3.1400000000000000000000000 9.4200000000000000000000000 +3.1400000000000000000000000 9.4200000000000000000000000 +10.0000000000000000000000000 30.0000000000000000000000000 +20.0000000000000000000000000 60.0000000000000000000000000 +100.0000000000000000000000000 300.0000000000000000000000000 +124.0000000000000000000000000 372.0000000000000000000000000 +125.2000000000000000000000000 375.6000000000000000000000000 +200.0000000000000000000000000 600.0000000000000000000000000 +1234567890.1234567800000000000000000 3703703670.3703703400000000000000000 diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 index 93cdc5c85645c..a26c8b7d12886 100644 --- a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 +++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 @@ -1,38 +1,38 @@ NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -1 --1.12 -1 --0.333 0 --0.33 0 --0.3 0 -0 0 -0 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 -1 1 -1 1 -1 1 -1.12 1 -1.122 1 -2 2 -2 2 -3.14 3 -3.14 3 -3.14 3 -3.14 4 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 +-1234567890.123456789000000000 -1234567890 +-4400.000000000000000000 4400 +-1255.490000000000000000 -1255 +-1.122000000000000000 -11 +-1.120000000000000000 -1 +-1.120000000000000000 -1 +-0.333000000000000000 0 +-0.330000000000000000 0 +-0.300000000000000000 0 +0.000000000000000000 0 +0.000000000000000000 0 +0.000000000000000000 0 +0.010000000000000000 0 +0.020000000000000000 0 +0.100000000000000000 0 +0.200000000000000000 0 +0.300000000000000000 0 +0.330000000000000000 0 +0.333000000000000000 0 +1.000000000000000000 1 +1.000000000000000000 1 +1.000000000000000000 1 +1.120000000000000000 1 +1.122000000000000000 1 +2.000000000000000000 2 +2.000000000000000000 2 +3.140000000000000000 3 +3.140000000000000000 3 +3.140000000000000000 3 +3.140000000000000000 4 +10.000000000000000000 10 +20.000000000000000000 20 +100.000000000000000000 100 +124.000000000000000000 124 +125.200000000000000000 125 +200.000000000000000000 200 +1234567890.123456780000000000 1234567890 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 index 62d71abc6fc7d..33ea4edf780a6 100644 --- a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 @@ -3,7 +3,7 @@ 65536 32.68 65536 33.45 65536 58.86 -65536 75.7 +65536 75.70 65536 83.48 65537 NULL 65537 4.49 @@ -57,9 +57,9 @@ 65548 75.39 65548 77.24 65549 NULL -65549 13.3 +65549 13.30 65549 28.93 -65549 50.6 +65549 50.60 65549 55.04 65549 64.91 65549 76.06 @@ -70,7 +70,7 @@ 65550 33.01 65550 57.63 65550 91.38 -65550 96.9 +65550 96.90 65551 NULL 65551 39.43 65551 73.93 @@ -99,7 +99,7 @@ 65559 29.55 65559 56.06 65559 73.94 -65559 83.5 +65559 83.50 65560 NULL 65560 16.86 65560 21.81 @@ -128,7 +128,7 @@ 65565 NULL 65565 81.72 65566 NULL -65566 7.8 +65566 7.80 65567 NULL 65568 NULL 65568 21.79 @@ -136,14 +136,14 @@ 65569 NULL 65570 NULL 65570 17.09 -65570 18.2 +65570 18.20 65570 25.57 65570 45.23 -65570 76.8 +65570 76.80 65571 NULL 65571 26.64 65571 40.68 -65571 82.5 +65571 82.50 65572 NULL 65572 22.64 65572 43.49 @@ -156,9 +156,9 @@ 65574 31.28 65574 38.54 65575 NULL -65575 17 +65575 17.00 65575 32.85 -65575 83.4 +65575 83.40 65576 NULL 65576 2.04 65576 4.88 @@ -166,7 +166,7 @@ 65577 NULL 65578 NULL 65578 16.01 -65578 41.1 +65578 41.10 65578 51.36 65578 54.35 65578 58.78 @@ -188,7 +188,7 @@ 65582 NULL 65582 1.23 65582 9.35 -65582 96.6 +65582 96.60 65583 NULL 65583 28.07 65583 50.57 @@ -218,7 +218,7 @@ 65588 98.33 65589 NULL 65589 49.49 -65589 72.3 +65589 72.30 65589 74.83 65589 94.73 65590 NULL @@ -240,8 +240,8 @@ 65595 NULL 65595 8.76 65595 67.56 -65595 72.7 -65595 89.6 +65595 72.70 +65595 89.60 65595 90.24 65596 NULL 65596 12.72 @@ -252,7 +252,7 @@ 65597 37.41 65597 69.05 65598 NULL -65598 63.3 +65598 63.30 65599 NULL 65599 0.56 65599 4.93 @@ -283,7 +283,7 @@ 65605 NULL 65606 NULL 65606 7.51 -65606 24.8 +65606 24.80 65606 57.69 65606 67.94 65606 87.16 @@ -294,9 +294,9 @@ 65607 75.86 65607 91.52 65608 NULL -65608 48.9 +65608 48.90 65608 69.42 -65608 87.9 +65608 87.90 65609 NULL 65610 NULL 65610 7.59 @@ -309,7 +309,7 @@ 65611 64.89 65612 NULL 65612 16.05 -65612 25.1 +65612 25.10 65612 52.64 65613 NULL 65614 NULL @@ -317,17 +317,17 @@ 65614 94.47 65615 NULL 65615 10.79 -65615 39.4 +65615 39.40 65615 99.88 65616 NULL -65616 75.2 +65616 75.20 65617 NULL 65617 18.51 65617 47.45 -65617 64.9 +65617 64.90 65618 NULL 65618 10.06 -65618 16.6 +65618 16.60 65618 81.99 65618 88.38 65619 NULL @@ -348,20 +348,20 @@ 65622 28.37 65622 50.08 65622 74.31 -65622 88.6 -65622 93.7 +65622 88.60 +65622 93.70 65623 NULL 65623 30.83 65623 31.22 65623 39.74 65623 48.51 65623 95.58 -65623 97.2 +65623 97.20 65624 NULL 65624 58.02 65624 65.31 65624 70.08 -65624 93.3 +65624 93.30 65625 NULL 65625 20.61 65625 42.86 @@ -377,13 +377,13 @@ 65628 NULL 65628 14.83 65628 30.43 -65628 37.8 +65628 37.80 65628 74.31 65628 83.26 65629 NULL 65629 19.33 65629 58.81 -65629 72.9 +65629 72.90 65630 NULL 65630 72.13 65631 NULL @@ -412,7 +412,7 @@ 65637 48.88 65637 93.41 65638 NULL -65638 11.2 +65638 11.20 65638 19.13 65639 NULL 65640 NULL @@ -477,20 +477,20 @@ 65654 26.73 65654 29.85 65654 37.74 -65654 37.8 +65654 37.80 65654 53.55 65654 88.23 65655 NULL 65655 77.41 65656 NULL -65656 14 +65656 14.00 65656 14.96 65656 53.27 65656 64.44 65656 82.67 65657 NULL 65657 11.93 -65657 26.4 +65657 26.40 65657 64.39 65657 65.01 65658 NULL @@ -506,8 +506,8 @@ 65659 NULL 65659 8.95 65659 46.57 -65659 53.8 -65659 94.3 +65659 53.80 +65659 94.30 65659 94.69 65659 95.71 65659 99.87 @@ -517,7 +517,7 @@ 65661 NULL 65661 5.24 65661 8.06 -65661 26.8 +65661 26.80 65661 68.98 65662 NULL 65662 59.92 @@ -531,10 +531,10 @@ 65663 94.16 65664 NULL 65664 11.46 -65664 27.6 +65664 27.60 65664 34.71 65664 38.42 -65664 45.4 +65664 45.40 65664 55.82 65664 97.64 65665 NULL @@ -543,13 +543,13 @@ 65666 83.95 65667 NULL 65667 13.96 -65667 63.9 +65667 63.90 65667 97.87 65668 NULL 65669 NULL 65669 1.76 65669 16.95 -65669 38.6 +65669 38.60 65669 54.25 65669 93.79 65670 NULL @@ -561,12 +561,12 @@ 65671 8.65 65671 52.05 65672 NULL -65672 52.6 -65672 58.1 +65672 52.60 +65672 58.10 65672 64.09 65672 75.27 65673 NULL -65673 0.9 +65673 0.90 65673 33.27 65673 43.81 65673 87.78 @@ -576,7 +576,7 @@ 65675 24.19 65675 35.33 65675 35.78 -65675 79.9 +65675 79.90 65675 83.09 65675 87.36 65676 NULL @@ -591,19 +591,19 @@ 65677 87.67 65678 NULL 65678 8.72 -65678 33.9 +65678 33.90 65679 NULL 65679 64.15 65680 NULL 65680 1.01 65680 34.08 65680 54.11 -65680 55.3 +65680 55.30 65680 65.88 65681 NULL 65681 35.45 65681 41.57 -65681 61.3 +65681 61.30 65681 71.17 65681 75.85 65682 NULL @@ -641,7 +641,7 @@ 65691 28.47 65691 56.02 65691 58.01 -65691 69.8 +65691 69.80 65691 76.98 65692 NULL 65692 54.76 @@ -655,19 +655,19 @@ 65694 NULL 65694 58.23 65694 82.24 -65694 88.5 +65694 88.50 65695 NULL 65695 57.33 65695 59.96 65695 77.09 65696 NULL 65696 17.35 -65696 40.3 +65696 40.30 65696 54.02 65697 NULL 65697 3.18 65697 50.01 -65697 67.9 +65697 67.90 65697 86.79 65697 90.16 65698 NULL @@ -685,9 +685,9 @@ 65701 1.81 65701 6.35 65702 NULL -65702 37.6 +65702 37.60 65702 55.68 -65702 79.5 +65702 79.50 65703 NULL 65703 37.18 65703 40.81 @@ -708,23 +708,23 @@ 65706 55.94 65706 72.87 65707 NULL -65707 76.2 +65707 76.20 65708 NULL 65708 1.29 65709 NULL 65709 5.64 65709 49.79 65710 NULL -65710 86.7 +65710 86.70 65711 NULL 65711 8.66 65711 50.26 65711 71.89 65711 78.69 -65711 96.1 +65711 96.10 65712 NULL 65712 30.27 -65712 34.7 +65712 34.70 65712 49.69 65712 53.65 65713 NULL @@ -739,11 +739,11 @@ 65715 39.62 65715 54.79 65715 81.28 -65715 89.4 +65715 89.40 65716 NULL -65716 9 +65716 9.00 65716 10.07 -65716 33.4 +65716 33.40 65716 71.53 65716 85.93 65717 NULL @@ -758,10 +758,10 @@ 65719 NULL 65719 51.13 65719 66.85 -65719 82.1 +65719 82.10 65720 NULL 65720 2.72 -65720 18.8 +65720 18.80 65720 22.34 65720 62.04 65721 NULL @@ -775,7 +775,7 @@ 65722 1.76 65722 38.82 65723 NULL -65723 39.9 +65723 39.90 65724 NULL 65724 10.52 65724 36.05 @@ -784,7 +784,7 @@ 65724 85.52 65725 NULL 65726 NULL -65726 6 +65726 6.00 65726 60.46 65727 NULL 65727 19.81 @@ -796,7 +796,7 @@ 65729 NULL 65730 NULL 65730 1.35 -65730 30.6 +65730 30.60 65730 81.44 65731 NULL 65731 24.48 @@ -810,14 +810,14 @@ 65733 20.72 65733 88.46 65733 93.45 -65733 99.8 +65733 99.80 65734 NULL 65734 31.71 65735 NULL 65735 12.67 65735 61.16 65736 NULL -65736 28.9 +65736 28.90 65736 48.54 65736 86.51 65737 NULL @@ -828,10 +828,10 @@ 65738 NULL 65738 30.94 65738 82.32 -65738 95.1 +65738 95.10 65739 NULL 65739 74.77 -65739 92.4 +65739 92.40 65740 NULL 65740 7.49 65740 58.65 @@ -840,9 +840,9 @@ 65742 6.61 65742 43.84 65743 NULL -65743 26.6 +65743 26.60 65743 52.65 -65743 62 +65743 62.00 65744 NULL 65744 46.98 65745 NULL @@ -853,11 +853,11 @@ 65746 36.74 65746 93.21 65746 97.52 -65746 98.1 +65746 98.10 65747 NULL 65747 11.16 65747 15.07 -65747 21.8 +65747 21.80 65747 39.77 65747 52.77 65747 71.87 @@ -865,7 +865,7 @@ 65748 29.49 65749 NULL 65749 15.14 -65749 45 +65749 45.00 65749 65.49 65749 73.24 65750 NULL @@ -888,12 +888,12 @@ 65755 NULL 65755 11.23 65755 22.44 -65755 64 +65755 64.00 65755 67.54 65755 76.75 65755 81.44 65755 90.08 -65755 96.8 +65755 96.80 65756 NULL 65756 1.45 65756 11.81 @@ -907,10 +907,10 @@ 65758 25.62 65758 56.56 65758 60.88 -65758 94.9 +65758 94.90 65759 NULL 65759 10.63 -65759 14.1 +65759 14.10 65759 47.54 65759 92.81 65760 NULL @@ -920,17 +920,17 @@ 65761 NULL 65762 NULL 65762 5.49 -65762 45.7 +65762 45.70 65762 77.96 -65762 87.5 +65762 87.50 65763 NULL 65763 0.72 -65763 43.8 +65763 43.80 65763 86.43 65763 87.99 65764 NULL 65764 31.41 -65764 57.1 +65764 57.10 65765 NULL 65765 88.52 65765 88.56 @@ -938,7 +938,7 @@ 65766 37.06 65766 66.34 65766 86.53 -65766 98.9 +65766 98.90 65767 NULL 65767 90.88 65767 95.57 @@ -950,14 +950,14 @@ 65769 70.52 65769 91.49 65770 NULL -65770 51.9 +65770 51.90 65771 NULL 65771 6.15 -65771 7.5 +65771 7.50 65772 NULL 65773 NULL 65773 3.81 -65773 18.2 +65773 18.20 65773 30.49 65773 47.09 65773 53.09 @@ -966,7 +966,7 @@ 65774 NULL 65774 45.74 65774 45.97 -65774 48.8 +65774 48.80 65774 56.84 65774 94.77 65775 NULL @@ -975,7 +975,7 @@ 65775 66.68 65775 98.43 65776 NULL -65776 18.7 +65776 18.70 65776 28.47 65776 49.73 65776 98.87 @@ -993,7 +993,7 @@ 65778 95.69 65779 NULL 65779 11.87 -65779 28.2 +65779 28.20 65779 39.48 65779 45.61 65779 64.41 @@ -1008,15 +1008,15 @@ 65782 30.24 65782 34.31 65782 76.14 -65782 81.9 +65782 81.90 65783 NULL 65783 46.34 65783 51.08 65783 52.43 65783 62.58 -65783 77.4 +65783 77.40 65784 NULL -65784 15.7 +65784 15.70 65784 31.35 65784 68.18 65784 93.95 @@ -1032,7 +1032,7 @@ 65787 31.19 65787 64.88 65788 NULL -65788 16.1 +65788 16.10 65788 21.81 65788 25.77 65789 NULL @@ -1041,7 +1041,7 @@ 65789 52.49 65789 83.18 65789 92.74 -65789 96.9 +65789 96.90 65790 NULL 65790 46.91 65790 84.87 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 index 9091a9156134c..207dababa0a50 100644 --- a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 @@ -18,12 +18,12 @@ 2013-03-01 09:11:58.703073 10.07 1 2013-03-01 09:11:58.703073 10.07 1 2013-03-01 09:11:58.703073 10.07 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 2013-03-01 09:11:58.703075 5.64 1 2013-03-01 09:11:58.703075 5.64 1 2013-03-01 09:11:58.703075 5.64 1 @@ -59,11 +59,11 @@ 2013-03-01 09:11:58.70308 1.76 1 2013-03-01 09:11:58.70308 1.76 1 2013-03-01 09:11:58.70308 1.76 1 -2013-03-01 09:11:58.703081 67.9 1 -2013-03-01 09:11:58.703081 67.9 1 -2013-03-01 09:11:58.703081 67.9 1 -2013-03-01 09:11:58.703081 67.9 1 -2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.90 1 2013-03-01 09:11:58.703082 37.25 1 2013-03-01 09:11:58.703082 37.25 1 2013-03-01 09:11:58.703082 37.25 1 @@ -148,9 +148,9 @@ 2013-03-01 09:11:58.703096 11.64 1 2013-03-01 09:11:58.703096 11.64 1 2013-03-01 09:11:58.703096 11.64 1 -2013-03-01 09:11:58.703097 0.9 1 -2013-03-01 09:11:58.703097 0.9 1 -2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.90 1 2013-03-01 09:11:58.703098 1.35 1 2013-03-01 09:11:58.703098 1.35 1 2013-03-01 09:11:58.703098 1.35 1 @@ -210,27 +210,27 @@ 2013-03-01 09:11:58.70311 8.16 1 2013-03-01 09:11:58.70311 8.16 1 2013-03-01 09:11:58.70311 8.16 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 2013-03-01 09:11:58.703114 73.94 1 2013-03-01 09:11:58.703114 73.94 1 2013-03-01 09:11:58.703114 73.94 1 @@ -256,14 +256,14 @@ 2013-03-01 09:11:58.703118 8.69 1 2013-03-01 09:11:58.703119 58.02 1 2013-03-01 09:11:58.703119 58.02 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.703121 96.9 1 -2013-03-01 09:11:58.703121 96.9 1 -2013-03-01 09:11:58.703121 96.9 1 -2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.703121 96.90 1 +2013-03-01 09:11:58.703121 96.90 1 +2013-03-01 09:11:58.703121 96.90 1 +2013-03-01 09:11:58.703121 96.90 1 2013-03-01 09:11:58.703122 53.56 1 2013-03-01 09:11:58.703122 53.56 1 2013-03-01 09:11:58.703122 53.56 1 @@ -310,11 +310,11 @@ 2013-03-01 09:11:58.703133 27.34 1 2013-03-01 09:11:58.703133 27.34 1 2013-03-01 09:11:58.703133 27.34 1 -2013-03-01 09:11:58.703134 98.9 1 -2013-03-01 09:11:58.703134 98.9 1 -2013-03-01 09:11:58.703134 98.9 1 -2013-03-01 09:11:58.703134 98.9 1 -2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.90 1 2013-03-01 09:11:58.703135 29.14 1 2013-03-01 09:11:58.703135 29.14 1 2013-03-01 09:11:58.703135 29.14 1 @@ -467,12 +467,12 @@ 2013-03-01 09:11:58.703162 3.51 1 2013-03-01 09:11:58.703162 3.51 1 2013-03-01 09:11:58.703162 3.51 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 2013-03-01 09:11:58.703164 30.27 1 2013-03-01 09:11:58.703164 30.27 1 2013-03-01 09:11:58.703164 30.27 1 @@ -482,9 +482,9 @@ 2013-03-01 09:11:58.703165 8.38 1 2013-03-01 09:11:58.703165 8.38 1 2013-03-01 09:11:58.703165 8.38 1 -2013-03-01 09:11:58.703166 16.6 1 -2013-03-01 09:11:58.703166 16.6 1 -2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.60 1 +2013-03-01 09:11:58.703166 16.60 1 +2013-03-01 09:11:58.703166 16.60 1 2013-03-01 09:11:58.703167 17.66 1 2013-03-01 09:11:58.703167 17.66 1 2013-03-01 09:11:58.703167 17.66 1 @@ -537,11 +537,11 @@ 2013-03-01 09:11:58.703175 33.37 1 2013-03-01 09:11:58.703175 33.37 1 2013-03-01 09:11:58.703175 33.37 1 -2013-03-01 09:11:58.703176 28.2 1 -2013-03-01 09:11:58.703176 28.2 1 -2013-03-01 09:11:58.703176 28.2 1 -2013-03-01 09:11:58.703176 28.2 1 -2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.20 1 2013-03-01 09:11:58.703177 11.43 1 2013-03-01 09:11:58.703177 11.43 1 2013-03-01 09:11:58.703177 11.43 1 @@ -567,13 +567,13 @@ 2013-03-01 09:11:58.70318 10.28 1 2013-03-01 09:11:58.70318 10.28 1 2013-03-01 09:11:58.70318 10.28 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 2013-03-01 09:11:58.703182 1.23 1 2013-03-01 09:11:58.703182 1.23 1 2013-03-01 09:11:58.703182 1.23 1 @@ -647,10 +647,10 @@ 2013-03-01 09:11:58.703197 16.01 1 2013-03-01 09:11:58.703197 16.01 1 2013-03-01 09:11:58.703197 16.01 1 -2013-03-01 09:11:58.703198 30.6 1 -2013-03-01 09:11:58.703198 30.6 1 -2013-03-01 09:11:58.703198 30.6 1 -2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.60 1 +2013-03-01 09:11:58.703198 30.60 1 +2013-03-01 09:11:58.703198 30.60 1 +2013-03-01 09:11:58.703198 30.60 1 2013-03-01 09:11:58.703199 45.69 1 2013-03-01 09:11:58.703199 45.69 1 2013-03-01 09:11:58.703199 45.69 1 @@ -669,11 +669,11 @@ 2013-03-01 09:11:58.703203 11.63 1 2013-03-01 09:11:58.703203 11.63 1 2013-03-01 09:11:58.703203 11.63 1 -2013-03-01 09:11:58.703205 35.8 1 -2013-03-01 09:11:58.703205 35.8 1 -2013-03-01 09:11:58.703205 35.8 1 -2013-03-01 09:11:58.703205 35.8 1 -2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.80 1 2013-03-01 09:11:58.703206 6.61 1 2013-03-01 09:11:58.703206 6.61 1 2013-03-01 09:11:58.703206 6.61 1 @@ -824,9 +824,9 @@ 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 -2013-03-01 09:11:58.703234 44.1 1 -2013-03-01 09:11:58.703234 44.1 1 -2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.10 1 +2013-03-01 09:11:58.703234 44.10 1 +2013-03-01 09:11:58.703234 44.10 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 @@ -834,11 +834,11 @@ 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 -2013-03-01 09:11:58.703236 37.8 1 -2013-03-01 09:11:58.703236 37.8 1 -2013-03-01 09:11:58.703236 37.8 1 -2013-03-01 09:11:58.703236 37.8 1 -2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.80 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 @@ -847,17 +847,17 @@ 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 -2013-03-01 09:11:58.703238 6 1 -2013-03-01 09:11:58.703238 6 1 -2013-03-01 09:11:58.703238 6 1 -2013-03-01 09:11:58.703238 6 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.70324 5.1 1 -2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.703238 6.00 1 +2013-03-01 09:11:58.703238 6.00 1 +2013-03-01 09:11:58.703238 6.00 1 +2013-03-01 09:11:58.703238 6.00 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.70324 5.10 1 +2013-03-01 09:11:58.70324 5.10 1 2013-03-01 09:11:58.703241 19.33 1 2013-03-01 09:11:58.703241 19.33 1 2013-03-01 09:11:58.703241 19.33 1 @@ -973,16 +973,16 @@ 2013-03-01 09:11:58.703262 1.81 1 2013-03-01 09:11:58.703262 1.81 1 2013-03-01 09:11:58.703262 1.81 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 @@ -1068,12 +1068,12 @@ 2013-03-01 09:11:58.703281 19.95 1 2013-03-01 09:11:58.703281 19.95 1 2013-03-01 09:11:58.703281 19.95 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 2013-03-01 09:11:58.703283 17.62 1 2013-03-01 09:11:58.703283 17.62 1 2013-03-01 09:11:58.703283 17.62 1 @@ -1153,12 +1153,12 @@ 2013-03-01 09:11:58.703297 25.67 1 2013-03-01 09:11:58.703297 25.67 1 2013-03-01 09:11:58.703297 25.67 1 -2013-03-01 09:11:58.703298 8.8 1 -2013-03-01 09:11:58.703298 8.8 1 -2013-03-01 09:11:58.703298 8.8 1 -2013-03-01 09:11:58.703298 8.8 1 -2013-03-01 09:11:58.703299 9 1 -2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.703298 8.80 1 +2013-03-01 09:11:58.703298 8.80 1 +2013-03-01 09:11:58.703298 8.80 1 +2013-03-01 09:11:58.703298 8.80 1 +2013-03-01 09:11:58.703299 9.00 1 +2013-03-01 09:11:58.703299 9.00 1 2013-03-01 09:11:58.7033 7.51 1 2013-03-01 09:11:58.7033 7.51 1 2013-03-01 09:11:58.7033 7.51 1 @@ -1217,12 +1217,12 @@ 2013-03-01 09:11:58.703311 7.38 1 2013-03-01 09:11:58.703311 7.38 1 2013-03-01 09:11:58.703311 7.38 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 2013-03-01 09:11:58.703313 9.35 1 2013-03-01 09:11:58.703313 9.35 1 2013-03-01 09:11:58.703313 9.35 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 index d02ca48857b5f..a1628c7e1c0c5 100644 --- a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 @@ -46,9 +46,9 @@ 2013-03-01 09:11:58.703092 54.02 1 2013-03-01 09:11:58.703092 54.02 1 2013-03-01 09:11:58.703096 87.84 1 -2013-03-01 09:11:58.703097 0.9 1 -2013-03-01 09:11:58.703097 0.9 1 -2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.90 1 2013-03-01 09:11:58.703098 21.29 1 2013-03-01 09:11:58.703098 21.29 1 2013-03-01 09:11:58.703098 21.29 1 @@ -88,10 +88,10 @@ 2013-03-01 09:11:58.703113 58.65 1 2013-03-01 09:11:58.703118 8.69 1 2013-03-01 09:11:58.703118 8.69 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 2013-03-01 09:11:58.703125 78.52 1 2013-03-01 09:11:58.703125 78.52 1 2013-03-01 09:11:58.703125 78.52 1 @@ -119,11 +119,11 @@ 2013-03-01 09:11:58.703136 27.89 1 2013-03-01 09:11:58.703136 27.89 1 2013-03-01 09:11:58.703136 27.89 1 -2013-03-01 09:11:58.703138 86.7 1 -2013-03-01 09:11:58.703138 86.7 1 -2013-03-01 09:11:58.703138 86.7 1 -2013-03-01 09:11:58.703138 86.7 1 -2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.70 1 2013-03-01 09:11:58.703139 43.53 1 2013-03-01 09:11:58.703139 43.53 1 2013-03-01 09:11:58.703139 43.53 1 @@ -167,13 +167,13 @@ 2013-03-01 09:11:58.703179 60.94 1 2013-03-01 09:11:58.703179 60.94 1 2013-03-01 09:11:58.703179 60.94 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 2013-03-01 09:11:58.703184 73.93 1 2013-03-01 09:11:58.703184 73.93 1 2013-03-01 09:11:58.703184 73.93 1 @@ -202,12 +202,12 @@ 2013-03-01 09:11:58.703189 37.74 1 2013-03-01 09:11:58.703189 37.74 1 2013-03-01 09:11:58.703189 37.74 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 2013-03-01 09:11:58.703198 97.18 1 2013-03-01 09:11:58.703198 97.18 1 2013-03-01 09:11:58.703198 97.18 1 @@ -233,10 +233,10 @@ 2013-03-01 09:11:58.70321 37.12 1 2013-03-01 09:11:58.70321 37.12 1 2013-03-01 09:11:58.70321 37.12 1 -2013-03-01 09:11:58.703213 48.8 1 -2013-03-01 09:11:58.703213 48.8 1 -2013-03-01 09:11:58.703213 48.8 1 -2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.80 1 +2013-03-01 09:11:58.703213 48.80 1 +2013-03-01 09:11:58.703213 48.80 1 +2013-03-01 09:11:58.703213 48.80 1 2013-03-01 09:11:58.703219 32.73 1 2013-03-01 09:11:58.703219 32.73 1 2013-03-01 09:11:58.703219 32.73 1 @@ -253,30 +253,30 @@ 2013-03-01 09:11:58.703221 26.64 1 2013-03-01 09:11:58.703221 26.64 1 2013-03-01 09:11:58.703221 26.64 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 @@ -295,24 +295,24 @@ 2013-03-01 09:11:58.703244 25.67 1 2013-03-01 09:11:58.703244 25.67 1 2013-03-01 09:11:58.703244 25.67 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 2013-03-01 09:11:58.703246 72.87 1 2013-03-01 09:11:58.703246 72.87 1 2013-03-01 09:11:58.703248 81.28 1 2013-03-01 09:11:58.703248 81.28 1 2013-03-01 09:11:58.703248 81.28 1 -2013-03-01 09:11:58.703249 93.3 1 -2013-03-01 09:11:58.703249 93.3 1 -2013-03-01 09:11:58.703249 93.3 1 -2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.30 1 +2013-03-01 09:11:58.703249 93.30 1 +2013-03-01 09:11:58.703249 93.30 1 +2013-03-01 09:11:58.703249 93.30 1 2013-03-01 09:11:58.70325 93.79 1 2013-03-01 09:11:58.70325 93.79 1 2013-03-01 09:11:58.70325 93.79 1 @@ -324,12 +324,12 @@ 2013-03-01 09:11:58.703254 0.32 1 2013-03-01 09:11:58.703254 0.32 1 2013-03-01 09:11:58.703254 0.32 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 2013-03-01 09:11:58.703258 21.21 1 2013-03-01 09:11:58.703258 21.21 1 2013-03-01 09:11:58.703258 21.21 1 @@ -342,16 +342,16 @@ 2013-03-01 09:11:58.703262 78.56 1 2013-03-01 09:11:58.703262 78.56 1 2013-03-01 09:11:58.703262 78.56 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 @@ -438,10 +438,10 @@ 2013-03-01 09:11:58.703299 23.19 1 2013-03-01 09:11:58.703299 23.19 1 2013-03-01 09:11:58.703299 23.19 1 -2013-03-01 09:11:58.703309 89.4 1 -2013-03-01 09:11:58.703309 89.4 1 -2013-03-01 09:11:58.703309 89.4 1 -2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.40 1 +2013-03-01 09:11:58.703309 89.40 1 +2013-03-01 09:11:58.703309 89.40 1 +2013-03-01 09:11:58.703309 89.40 1 2013-03-01 09:11:58.70331 69.26 1 2013-03-01 09:11:58.70331 69.26 1 2013-03-01 09:11:58.70331 69.26 1 From 7c0ce28501f9009e240ab0a1be206054243607d0 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 27 Nov 2019 18:19:21 +0900 Subject: [PATCH 0740/1387] [SPARK-30056][INFRA] Skip building test artifacts in `dev/make-distribution.sh` ### What changes were proposed in this pull request? This PR aims to skip building test artifacts in `dev/make-distribution.sh`. Since Apache Spark 3.0.0, we need to build additional binary distribution, this helps the release process by speeding up building multiple binary distributions. ### Why are the changes needed? Since the generated binary artifacts are irrelevant to the test jars, we can skip this. **BEFORE** ``` $ time dev/make-distribution.sh 726.86 real 2526.04 user 45.63 sys ``` **AFTER** ``` $ time dev/make-distribution.sh 305.54 real 1099.99 user 26.52 sys ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually check `dev/make-distribution.sh` result and time. Closes #26689 from dongjoon-hyun/SPARK-30056. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- dev/make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index cd18b6870e07c..16fbb71be429e 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -165,7 +165,7 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=1g}" # Store the command as an array because $MVN variable might have spaces in it. # Normal quoting tricks don't work. # See: http://mywiki.wooledge.org/BashFAQ/050 -BUILD_COMMAND=("$MVN" clean package -DskipTests $@) +BUILD_COMMAND=("$MVN" clean package -DskipTests -Dmaven.test.skip=true $@) # Actually build the jar echo -e "\nBuilding with..." From d075b3344e39eec74cbc953d461b32325be23c4b Mon Sep 17 00:00:00 2001 From: wuyi Date: Wed, 27 Nov 2019 20:34:22 +0800 Subject: [PATCH 0741/1387] [SPARK-28366][CORE][FOLLOW-UP] Improve the conf IO_WARNING_LARGEFILETHRESHOLD ### What changes were proposed in this pull request? Improve conf `IO_WARNING_LARGEFILETHRESHOLD` (a.k.a `spark.io.warning.largeFileThreshold`): * reword documentation * change type from `long` to `bytes` ### Why are the changes needed? Improvements according to https://github.com/apache/spark/pull/25134#discussion_r350570804 & https://github.com/apache/spark/pull/25134#discussion_r350570917. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass Jenkins. Closes #26691 from Ngone51/SPARK-28366-followup. Authored-by: wuyi Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/internal/config/package.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 0f402e0bf2bb8..489c5f8fa6b76 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 @@ -1325,9 +1325,9 @@ package object config { private[spark] val IO_WARNING_LARGEFILETHRESHOLD = ConfigBuilder("spark.io.warning.largeFileThreshold") .internal() - .doc("When spark loading one single large file, if file size exceed this " + - "threshold, then log warning with possible reasons.") - .longConf + .doc("If the size in bytes of a file loaded by Spark exceeds this threshold, " + + "a warning is logged with the possible reasons.") + .bytesConf(ByteUnit.BYTE) .createWithDefault(1024 * 1024 * 1024) private[spark] val EVENT_LOG_COMPRESSION_CODEC = From 16da714ea5af9105a98e03d71321949d18590506 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Wed, 27 Nov 2019 23:16:53 +0800 Subject: [PATCH 0742/1387] [SPARK-29979][SQL][FOLLOW-UP] improve the output of DesribeTableExec ### What changes were proposed in this pull request? refine the output of "DESC TABLE" command. After this PR, the output of "DESC TABLE" command is like below : ``` id bigint data string # Partitioning Part 0 id # Detailed Table Information Name testca.table_name Comment this is a test table Location /tmp/testcat/table_name Provider foo Table Properties [bar=baz] ``` ### Why are the changes needed? Currently, "DESC TABLE" will show reserved properties (eg. location, comment) in the "Table Property" section. Since reserved properties are different from common properties, displaying reserved properties together with other table detailed information and displaying other properties in single field should be reasonable, and it is consistent with hive and DescribeTableCommand action. ### Does this PR introduce any user-facing change? yes, the output of "DESC TABLE" command is refined as above. ### How was this patch tested? Update existing unit tests. Closes #26677 from fuwhu/SPARK-29979-FOLLOWUP-1. Authored-by: fuwhu Signed-off-by: Wenchen Fan --- .../sql/connector/catalog/TableCatalog.java | 12 ++++++ .../sql/connector/catalog/CatalogV2Util.scala | 5 ++- .../apache/spark/sql/DataFrameWriter.scala | 4 +- .../apache/spark/sql/DataFrameWriterV2.scala | 4 +- .../datasources/v2/DescribeTableExec.scala | 37 ++++++++++++------- .../datasources/v2/V2SessionCatalog.scala | 2 +- .../sql/connector/DataSourceV2SQLSuite.scala | 22 +++++++---- 7 files changed, 58 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index a349480359e0e..32c6f8f2cde16 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -24,6 +24,8 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; import org.apache.spark.sql.types.StructType; +import java.util.Arrays; +import java.util.List; import java.util.Map; /** @@ -49,6 +51,16 @@ public interface TableCatalog extends CatalogPlugin { */ String PROP_COMMENT = "comment"; + /** + * A property to specify the provider of the table. + */ + String PROP_PROVIDER = "provider"; + + /** + * The list of reserved table properties. + */ + List RESERVED_PROPERTIES = Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_PROVIDER); + /** * List the tables in a namespace from the catalog. *

    diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index fa45d72062423..0dcd595ded191 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -252,7 +252,8 @@ private[sql] object CatalogV2Util { s"are both used to set the table comment, you can only specify one of them.") } - if (options.contains("provider") || properties.contains("provider")) { + if (options.contains(TableCatalog.PROP_PROVIDER) + || properties.contains(TableCatalog.PROP_PROVIDER)) { throw new AnalysisException( "USING and option/property 'provider' are both used to set the provider implementation, " + "you can only specify one of them.") @@ -266,7 +267,7 @@ private[sql] object CatalogV2Util { tableProperties ++= filteredOptions // convert USING, LOCATION, and COMMENT clauses to table properties - tableProperties += ("provider" -> provider) + tableProperties += (TableCatalog.PROP_PROVIDER -> provider) comment.map(text => tableProperties += (TableCatalog.PROP_COMMENT -> text)) location.orElse(options.get("path")).map( loc => tableProperties += (TableCatalog.PROP_LOCATION -> loc)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 42f145d5673d3..92515a0210c67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -535,7 +535,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { ident, partitionTransforms, df.queryExecution.analyzed, - Map("provider" -> source) ++ getLocationIfExists, + Map(TableCatalog.PROP_PROVIDER -> source) ++ getLocationIfExists, extraOptions.toMap, orCreate = true) // Create the table if it doesn't exist @@ -548,7 +548,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { ident, partitionTransforms, df.queryExecution.analyzed, - Map("provider" -> source) ++ getLocationIfExists, + Map(TableCatalog.PROP_PROVIDER -> source) ++ getLocationIfExists, extraOptions.toMap, ignoreIfExists = other == SaveMode.Ignore) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index 8e601b50fdd4c..cf534ab6b9e36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -24,6 +24,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.{Attribute, Bucket, Days, Hours, Literal, Months, Years} import org.apache.spark.sql.catalyst.plans.logical.{AppendData, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect} +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.connector.expressions.{LogicalExpressions, NamedReference, Transform} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation @@ -128,7 +129,8 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) identifier, partitioning.getOrElse(Seq.empty), logicalPlan, - properties = provider.map(p => properties + ("provider" -> p)).getOrElse(properties).toMap, + properties = provider.map(p => properties + (TableCatalog.PROP_PROVIDER -> p)) + .getOrElse(properties).toMap, writeOptions = options.toMap, ignoreIfExists = false) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index a032f5c9d5684..2815b0ac131f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} -import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.catalog.{Table, TableCatalog} import org.apache.spark.sql.types.StructType case class DescribeTableExec( @@ -36,14 +36,33 @@ case class DescribeTableExec( override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() addSchema(rows) + addPartitioning(rows) if (isExtended) { - addPartitioning(rows) - addProperties(rows) + addTableDetails(rows) } rows } + private def addTableDetails(rows: ArrayBuffer[InternalRow]): Unit = { + rows += emptyRow() + rows += toCatalystRow("# Detailed Table Information", "", "") + rows += toCatalystRow("Name", table.name(), "") + + TableCatalog.RESERVED_PROPERTIES.asScala.toList.foreach(propKey => { + if (table.properties.containsKey(propKey)) { + rows += toCatalystRow(propKey.capitalize, table.properties.get(propKey), "") + } + }) + val properties = + table.properties.asScala.toList + .filter(kv => !TableCatalog.RESERVED_PROPERTIES.contains(kv._1)) + .sortBy(_._1).map { + case (key, value) => key + "=" + value + }.mkString("[", ",", "]") + rows += toCatalystRow("Table Properties", properties, "") + } + private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { rows ++= table.schema.map{ column => toCatalystRow( @@ -53,8 +72,7 @@ case class DescribeTableExec( private def addPartitioning(rows: ArrayBuffer[InternalRow]): Unit = { rows += emptyRow() - rows += toCatalystRow(" Partitioning", "", "") - rows += toCatalystRow("--------------", "", "") + rows += toCatalystRow("# Partitioning", "", "") if (table.partitioning.isEmpty) { rows += toCatalystRow("Not partitioned", "", "") } else { @@ -64,15 +82,6 @@ case class DescribeTableExec( } } - private def addProperties(rows: ArrayBuffer[InternalRow]): Unit = { - rows += emptyRow() - rows += toCatalystRow(" Table Property", " Value", "") - rows += toCatalystRow("----------------", "-------", "") - rows ++= table.properties.asScala.toList.sortBy(_._1).map { - case (key, value) => toCatalystRow(key, value, "") - } - } - private def emptyRow(): InternalRow = toCatalystRow("", "", "") private def toCatalystRow(strs: String*): InternalRow = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index f452201a8d3ba..8d9957fe898d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -83,7 +83,7 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) properties: util.Map[String, String]): Table = { val (partitionColumns, maybeBucketSpec) = V2SessionCatalog.convertTransforms(partitions) - val provider = properties.getOrDefault("provider", conf.defaultDataSourceName) + val provider = properties.getOrDefault(TableCatalog.PROP_PROVIDER, conf.defaultDataSourceName) val tableProperties = properties.asScala val location = Option(properties.get(TableCatalog.PROP_LOCATION)) val storage = DataSource.buildStorageFormatFromOptions(tableProperties.toMap) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 9279d2e882997..629fd28414c54 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -112,7 +112,10 @@ class DataSourceV2SQLSuite val description = descriptionDf.collect() assert(description === Seq( Row("id", "bigint", ""), - Row("data", "string", ""))) + Row("data", "string", ""), + Row("", "", ""), + Row("# Partitioning", "", ""), + Row("Part 0", "id", ""))) } test("DescribeTable with v2 catalog when table does not exist.") { @@ -125,7 +128,9 @@ class DataSourceV2SQLSuite spark.sql("CREATE TABLE testcat.table_name (id bigint, data string)" + " USING foo" + " PARTITIONED BY (id)" + - " TBLPROPERTIES ('bar'='baz')") + " TBLPROPERTIES ('bar'='baz')" + + " COMMENT 'this is a test table'" + + " LOCATION '/tmp/testcat/table_name'") val descriptionDf = spark.sql("DESCRIBE TABLE EXTENDED testcat.table_name") assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( @@ -138,14 +143,15 @@ class DataSourceV2SQLSuite Array("id", "bigint", ""), Array("data", "string", ""), Array("", "", ""), - Array("Partitioning", "", ""), - Array("--------------", "", ""), + Array("# Partitioning", "", ""), Array("Part 0", "id", ""), Array("", "", ""), - Array("Table Property", "Value", ""), - Array("----------------", "-------", ""), - Array("bar", "baz", ""), - Array("provider", "foo", ""))) + Array("# Detailed Table Information", "", ""), + Array("Name", "testcat.table_name", ""), + Array("Comment", "this is a test table", ""), + Array("Location", "/tmp/testcat/table_name", ""), + Array("Provider", "foo", ""), + Array("Table Properties", "[bar=baz]", ""))) } From 9cd174a7c9cc4c6b782a17c7710650d7eb11389f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 27 Nov 2019 11:07:08 -0800 Subject: [PATCH 0743/1387] Revert "[SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column" This reverts commit 19af1fe3a2b604a653c9f736d11648b79b93bb17. --- docs/sql-migration-guide.md | 26 -- .../aggregate/ApproximatePercentile.scala | 2 +- .../expressions/conditionalExpressions.scala | 4 +- .../spark/sql/execution/HiveResult.scala | 10 +- .../ansi/decimalArithmeticOperations.sql.out | 18 +- .../sql-tests/results/ansi/literals.sql.out | 4 +- .../decimalArithmeticOperations.sql.out | 46 +-- .../sql-tests/results/literals.sql.out | 4 +- .../results/order-by-nulls-ordering.sql.out | 48 +-- .../results/postgreSQL/create_view.sql.out | 2 +- .../sql-tests/results/postgreSQL/date.sql.out | 4 +- .../sql-tests/results/postgreSQL/int2.sql.out | 14 +- .../sql-tests/results/postgreSQL/int4.sql.out | 14 +- .../sql-tests/results/postgreSQL/int8.sql.out | 14 +- .../results/postgreSQL/numeric.sql.out | 384 +++++++++--------- .../results/postgreSQL/select.sql.out | 20 +- .../results/postgreSQL/timestamp.sql.out | 20 +- .../results/postgreSQL/union.sql.out | 16 +- .../results/postgreSQL/window_part4.sql.out | 8 +- ...in-unit-tests-multi-column-literal.sql.out | 4 +- .../not-in-unit-tests-multi-column.sql.out | 10 +- ...n-unit-tests-single-column-literal.sql.out | 2 +- .../not-in-unit-tests-single-column.sql.out | 18 +- .../sql-tests/results/table-aliases.sql.out | 2 +- .../native/decimalPrecision.sql.out | 80 ++-- .../typeCoercion/native/division.sql.out | 18 +- .../sql-tests/results/udf/udf-union.sql.out | 12 +- .../resources/sql-tests/results/union.sql.out | 12 +- .../spark/sql/execution/HiveResultSuite.scala | 15 - .../sql/hive/thriftserver/CliSuite.scala | 7 - ...cimal_4-6-693c2e345731f9b2b547c3b75218458e | 72 ++-- ...cimal_4-7-f1eb45492510cb76cf6b452121af8531 | 72 ++-- ..._regex-10-c5b3ec90419a40660e5f83736241c429 | 74 ++-- ...nistic)-2-1e88e0ba414a00195f7ebf6b8600ac04 | 192 ++++----- ...istic) 2-0-81bb7f49a55385878637c8aac4d08e5 | 242 +++++------ ...stic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 | 164 ++++---- 36 files changed, 807 insertions(+), 847 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 12d734fcbdf52..74ba694800d2d 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -226,32 +226,6 @@ license: | - Since Spark 3.0, when casting string value to date, timestamp and interval values, the leading and trailing white spaces(<= ACSII 32) will be trimmed before casing, e.g. `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, only the trailing space will be removed, thus, the result is `null`. - - Since Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example: - - - - - - - - - - - -
    - Query - - Spark 2.4 or Prior - - Spark 3.0 -
    - SELECT CAST(1 AS decimal(38, 18)); - - 1 - - 1.000000000000000000 -
    - ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index ea0ed2e8fa11b..59481ce049165 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -63,7 +63,7 @@ import org.apache.spark.sql.types._ > SELECT _FUNC_(10.0, array(0.5, 0.4, 0.1), 100); [10.0,10.0,10.0] > SELECT _FUNC_(10.0, 0.5, 100); - 10.0 + 10 """, since = "2.1.0") case class ApproximatePercentile( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 6c6210994954c..3a2dc40862bb8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -111,9 +111,9 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi examples = """ Examples: > SELECT CASE WHEN 1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END; - 1.0 + 1 > SELECT CASE WHEN 1 < 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END; - 2.0 + 2 > SELECT CASE WHEN 1 < 0 THEN 1 WHEN 2 < 0 THEN 2.0 END; NULL """) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index f7f7e08462fe1..d4e10b3ffc733 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -59,6 +59,14 @@ object HiveResult { result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) } + private def formatDecimal(d: java.math.BigDecimal): String = { + if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { + java.math.BigDecimal.ZERO.toPlainString + } else { + d.stripTrailingZeros().toPlainString // Hive strips trailing zeros + } + } + private val primitiveTypes = Seq( StringType, IntegerType, @@ -119,7 +127,7 @@ object HiveResult { case (t: Timestamp, TimestampType) => DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) - case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString + case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) case (interval: CalendarInterval, CalendarIntervalType) => SQLConf.get.intervalOutputStyle match { case SQL_STANDARD => toSqlStandardString(interval) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index 67cb0a2b36a92..e12f409f6d47e 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -24,10 +24,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 2 schema struct -- !query 2 output -1 1000.000000000000000 99.900000000000000000 -2 123451.230000000000000 1234.512300000000000000 -3 1.234567891011000 123.410000000000000000 -4 1234567891234567890.000000000000000 0.112345678912345679 +1 1000 99.9 +2 123451.23 1234.5123 +3 1.234567891011 123.41 +4 1234567891234567890 0.112345678912345679 -- !query 3 @@ -35,7 +35,7 @@ select 10.3 * 3.0 -- !query 3 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 3 output -30.90 +30.9 -- !query 4 @@ -43,7 +43,7 @@ select 10.3000 * 3.0 -- !query 4 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 4 output -30.90000 +30.9 -- !query 5 @@ -51,7 +51,7 @@ select 10.30000 * 30.0 -- !query 5 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 5 output -309.000000 +309 -- !query 6 @@ -59,7 +59,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 6 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> -- !query 6 output -30.9000000000000000000000000000000000 +30.9 -- !query 7 @@ -67,7 +67,7 @@ select 10.300000000000000000 * 3.0000000000000000000 -- !query 7 schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> -- !query 7 output -30.9000000000000000000000000000000000 +30.9 -- !query 8 diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 5d8a893d334c2..c1488eaf2aa36 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -169,7 +169,7 @@ select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query 17 schema struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> -- !query 17 output -1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90.0 90 +1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90 90 -- !query 18 @@ -339,7 +339,7 @@ select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD -- !query 35 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> -- !query 35 output -90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 +90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 -- !query 36 diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index 6f035a1904ea3..cbf44548b3cce 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -56,9 +56,9 @@ select id, a+b, a-b, a*b, a/b from decimals_test order by id -- !query 6 schema struct -- !query 6 output -1 1099.00000000000000000 -899.00000000000000000 99900.000000 0.100100 -2 24690.24600000000000000 0.00000000000000000 152402061.885129 1.000000 -3 1234.22345678910110000 -1233.97654321089890000 152.358023 0.000100 +1 1099 -899 99900 0.1001 +2 24690.246 0 152402061.885129 1 +3 1234.2234567891011 -1233.9765432108989 152.358023 0.0001 4 123456789123456790.12345678912345679 123456789123456787.87654321087654321 138698367904130467.515623 109890109097814272.043109 @@ -67,10 +67,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 7 schema struct -- !query 7 output -1 1000.000000000000000 99.900000000000000000 -2 123451.230000000000000 1234.512300000000000000 -3 1.234567891011000 123.410000000000000000 -4 1234567891234567890.000000000000000 0.112345678912345679 +1 1000 99.9 +2 123451.23 1234.5123 +3 1.234567891011 123.41 +4 1234567891234567890 0.112345678912345679 -- !query 8 @@ -78,7 +78,7 @@ select 10.3 * 3.0 -- !query 8 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 8 output -30.90 +30.9 -- !query 9 @@ -86,7 +86,7 @@ select 10.3000 * 3.0 -- !query 9 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 9 output -30.90000 +30.9 -- !query 10 @@ -94,7 +94,7 @@ select 10.30000 * 30.0 -- !query 10 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 10 output -309.000000 +309 -- !query 11 @@ -102,7 +102,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 11 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> -- !query 11 output -30.9000000000000000000000000000000000 +30.9 -- !query 12 @@ -110,7 +110,7 @@ select 10.300000000000000000 * 3.0000000000000000000 -- !query 12 schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> -- !query 12 output -30.9000000000000000000000000000000000 +30.9 -- !query 13 @@ -198,9 +198,9 @@ select id, a+b, a-b, a*b, a/b from decimals_test order by id -- !query 23 schema struct -- !query 23 output -1 1099.000000000000000000 -899.000000000000000000 NULL 0.100100100100100100 -2 24690.246000000000000000 0.000000000000000000 NULL 1.000000000000000000 -3 1234.223456789101100000 -1233.976543210898900000 NULL 0.000100037913541123 +1 1099 -899 NULL 0.1001001001001001 +2 24690.246 0 NULL 1 +3 1234.2234567891011 -1233.9765432108989 NULL 0.000100037913541123 4 123456789123456790.123456789123456789 123456789123456787.876543210876543211 NULL 109890109097814272.043109406191131436 @@ -209,10 +209,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 24 schema struct -- !query 24 output -1 1000.000000000000000000 99.9000000000000000000 -2 123451.230000000000000000 1234.5123000000000000000 -3 1.234567891011000000 123.4100000000000000000 -4 1234567891234567890.000000000000000000 0.1123456789123456789 +1 1000 99.9 +2 123451.23 1234.5123 +3 1.234567891011 123.41 +4 1234567891234567890 0.1123456789123456789 -- !query 25 @@ -220,7 +220,7 @@ select 10.3 * 3.0 -- !query 25 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 25 output -30.90 +30.9 -- !query 26 @@ -228,7 +228,7 @@ select 10.3000 * 3.0 -- !query 26 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 26 output -30.90000 +30.9 -- !query 27 @@ -236,7 +236,7 @@ select 10.30000 * 30.0 -- !query 27 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 27 output -309.000000 +309 -- !query 28 @@ -244,7 +244,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 28 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,36)> -- !query 28 output -30.900000000000000000000000000000000000 +30.9 -- !query 29 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 5d8a893d334c2..c1488eaf2aa36 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -169,7 +169,7 @@ select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query 17 schema struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> -- !query 17 output -1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90.0 90 +1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90 90 -- !query 18 @@ -339,7 +339,7 @@ select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD -- !query 35 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> -- !query 35 output -90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 +90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 -- !query 36 diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out index e1289dd8c8bde..c1b63dfb8caef 100644 --- a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out @@ -206,14 +206,14 @@ select * from spark_10747_mix order by col1 nulls last, col5 nulls last -- !query 13 schema struct -- !query 13 output -a 1 1.0 1.00 NULL -b 2 1.0 1.00 10.0 -c 3 2.0 2.00 15.1 -c 3 2.0 2.00 NULL -d 3 2.0 3.00 0.0 -d 3 0.0 3.00 1.0 -d 3 NULL 4.00 1.0 -NULL 3 0.0 3.00 1.0 +a 1 1.0 1 NULL +b 2 1.0 1 10 +c 3 2.0 2 15.1 +c 3 2.0 2 NULL +d 3 2.0 3 0 +d 3 0.0 3 1 +d 3 NULL 4 1 +NULL 3 0.0 3 1 -- !query 14 @@ -221,14 +221,14 @@ select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls fi -- !query 14 schema struct -- !query 14 output -NULL 3 0.0 3.00 1.0 -d 3 0.0 3.00 1.0 -d 3 NULL 4.00 1.0 -d 3 2.0 3.00 0.0 -c 3 2.0 2.00 NULL -c 3 2.0 2.00 15.1 -b 2 1.0 1.00 10.0 -a 1 1.0 1.00 NULL +NULL 3 0.0 3 1 +d 3 0.0 3 1 +d 3 NULL 4 1 +d 3 2.0 3 0 +c 3 2.0 2 NULL +c 3 2.0 2 15.1 +b 2 1.0 1 10 +a 1 1.0 1 NULL -- !query 15 @@ -236,14 +236,14 @@ select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls la -- !query 15 schema struct -- !query 15 output -c 3 2.0 2.00 NULL -a 1 1.0 1.00 NULL -c 3 2.0 2.00 15.1 -b 2 1.0 1.00 10.0 -d 3 0.0 3.00 1.0 -NULL 3 0.0 3.00 1.0 -d 3 NULL 4.00 1.0 -d 3 2.0 3.00 0.0 +c 3 2.0 2 NULL +a 1 1.0 1 NULL +c 3 2.0 2 15.1 +b 2 1.0 1 10 +d 3 0.0 3 1 +NULL 3 0.0 3 1 +d 3 NULL 4 1 +d 3 2.0 3 0 -- !query 16 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index cd4d3e64b8da1..031918961df4c 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -956,7 +956,7 @@ SELECT * FROM tt1 struct -- !query 83 output 0123456789 abc 42.12 abc -abc 0123456789 42.00 abcd +abc 0123456789 42 abcd -- !query 84 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index 5371e07423c28..68ff2a9f68899 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -576,7 +576,7 @@ SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') -- !query 54 schema struct -- !query 54 output -0.000000 +0 -- !query 55 @@ -584,7 +584,7 @@ SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 55 schema struct -- !query 55 output -0.000000 +0 -- !query 56 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out index d0a14618a5163..8f41accfc8c49 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -346,13 +346,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 33 schema struct -- !query 33 output --0.500000000000000000 0 --1.500000000000000000 -1 --2.500000000000000000 -2 -0.000000000000000000 0 -0.500000000000000000 0 -1.500000000000000000 1 -2.500000000000000000 2 +-0.5 0 +-1.5 -1 +-2.5 -2 +0 0 +0.5 0 +1.5 1 +2.5 2 -- !query 34 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index 30afd6e695842..6b1a8eb4a5883 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -495,13 +495,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 51 schema struct -- !query 51 output --0.500000000000000000 0 --1.500000000000000000 -1 --2.500000000000000000 -2 -0.000000000000000000 0 -0.500000000000000000 0 -1.500000000000000000 1 -2.500000000000000000 2 +-0.5 0 +-1.5 -1 +-2.5 -2 +0 0 +0.5 0 +1.5 1 +2.5 2 -- !query 52 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index 7a8e706cb8d85..c4d8affe75267 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -836,13 +836,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 83 schema struct -- !query 83 output --0.500000000000000000 0 --1.500000000000000000 -1 --2.500000000000000000 -2 -0.000000000000000000 0 -0.500000000000000000 0 -1.500000000000000000 1 -2.500000000000000000 2 +-0.5 0 +-1.5 -1 +-2.5 -2 +0 0 +0.5 0 +1.5 1 +2.5 2 -- !query 84 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index e245af28508c1..5ffa29a93d5de 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -3691,16 +3691,16 @@ SELECT * FROM num_data -- !query 460 schema struct -- !query 460 output -0 0.0000000000 -1 0.0000000000 -2 -34338492.2153970470 -3 4.3100000000 -4 7799461.4119000000 -5 16397.0384910000 -6 93901.5776302600 -7 -83028485.0000000000 -8 74881.0000000000 -9 -24926804.0450474200 +0 0 +1 0 +2 -34338492.215397047 +3 4.31 +4 7799461.4119 +5 16397.038491 +6 93901.57763026 +7 -83028485 +8 74881 +9 -24926804.04504742 -- !query 461 @@ -3869,51 +3869,51 @@ SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected -- !query 478 schema struct -- !query 478 output -2 2 1179132047626883.5968620000 1179132047626883.5968621359 -2 3 -147998901.4483610000 -147998901.4483612726 -2 4 -267821744976817.8111140000 -267821744976817.8111137107 -2 5 -563049578578.7692430000 -563049578578.7692425067 -2 6 -3224438592470.1844980000 -3224438592470.1844981193 -2 7 2851072985828710.4858840000 2851072985828710.4858837950 -2 8 -2571300635581.1462760000 -2571300635581.1462764070 -2 9 855948866655588.4537420000 855948866655588.4537415092 -3 2 -147998901.4483610000 -147998901.4483612726 -3 5 70671.2358960000 70671.2358962100 -3 6 404715.7995860000 404715.7995864206 -3 9 -107434525.4341540000 -107434525.4341543802 -4 2 -267821744976817.8111140000 -267821744976817.8111137107 -4 4 60831598315717.1414620000 60831598315717.1414616100 -4 5 127888068979.9935050000 127888068979.9935054429 -4 6 732381731243.7451160000 732381731243.7451157641 -4 9 -194415646271340.1815960000 -194415646271340.1815956523 -5 2 -563049578578.7692430000 -563049578578.7692425067 -5 3 70671.2358960000 70671.2358962100 -5 4 127888068979.9935050000 127888068979.9935054429 -5 5 268862871.2753360000 268862871.2753355571 -5 6 1539707782.7689980000 1539707782.7689977863 -5 9 -408725765384.2570440000 -408725765384.2570436602 -6 2 -3224438592470.1844980000 -3224438592470.1844981193 -6 3 404715.7995860000 404715.7995864206 -6 4 732381731243.7451160000 732381731243.7451157641 -6 5 1539707782.7689980000 1539707782.7689977863 -6 6 8817506281.4517450000 8817506281.4517452373 -6 7 -7796505729750.3779560000 -7796505729750.3779561000 -6 8 7031444034.5314990000 7031444034.5314990600 -6 9 -2340666225110.2992950000 -2340666225110.2992952129 -7 2 2851072985828710.4858840000 2851072985828710.4858837950 -7 6 -7796505729750.3779560000 -7796505729750.3779561000 -7 9 2069634775752159.0357590000 2069634775752159.0357587000 -8 2 -2571300635581.1462760000 -2571300635581.1462764070 -8 6 7031444034.5314990000 7031444034.5314990600 -8 9 -1866544013697.1958570000 -1866544013697.1958570200 -9 2 855948866655588.4537420000 855948866655588.4537415092 -9 3 -107434525.4341540000 -107434525.4341543802 -9 4 -194415646271340.1815960000 -194415646271340.1815956523 -9 5 -408725765384.2570440000 -408725765384.2570436602 -9 6 -2340666225110.2992950000 -2340666225110.2992952129 -9 7 2069634775752159.0357590000 2069634775752159.0357587000 -9 8 -1866544013697.1958570000 -1866544013697.1958570200 -9 9 621345559900192.4201210000 621345559900192.4201206300 +2 2 1179132047626883.596862 1179132047626883.5968621359 +2 3 -147998901.448361 -147998901.4483612726 +2 4 -267821744976817.811114 -267821744976817.8111137107 +2 5 -563049578578.769243 -563049578578.7692425067 +2 6 -3224438592470.184498 -3224438592470.1844981193 +2 7 2851072985828710.485884 2851072985828710.485883795 +2 8 -2571300635581.146276 -2571300635581.146276407 +2 9 855948866655588.453742 855948866655588.4537415092 +3 2 -147998901.448361 -147998901.4483612726 +3 5 70671.235896 70671.23589621 +3 6 404715.799586 404715.7995864206 +3 9 -107434525.434154 -107434525.4341543802 +4 2 -267821744976817.811114 -267821744976817.8111137107 +4 4 60831598315717.141462 60831598315717.14146161 +4 5 127888068979.993505 127888068979.9935054429 +4 6 732381731243.745116 732381731243.7451157641 +4 9 -194415646271340.181596 -194415646271340.1815956523 +5 2 -563049578578.769243 -563049578578.7692425067 +5 3 70671.235896 70671.23589621 +5 4 127888068979.993505 127888068979.9935054429 +5 5 268862871.275336 268862871.2753355571 +5 6 1539707782.768998 1539707782.7689977863 +5 9 -408725765384.257044 -408725765384.2570436602 +6 2 -3224438592470.184498 -3224438592470.1844981193 +6 3 404715.799586 404715.7995864206 +6 4 732381731243.745116 732381731243.7451157641 +6 5 1539707782.768998 1539707782.7689977863 +6 6 8817506281.451745 8817506281.4517452373 +6 7 -7796505729750.377956 -7796505729750.3779561 +6 8 7031444034.531499 7031444034.53149906 +6 9 -2340666225110.299295 -2340666225110.2992952129 +7 2 2851072985828710.485884 2851072985828710.485883795 +7 6 -7796505729750.377956 -7796505729750.3779561 +7 9 2069634775752159.035759 2069634775752159.0357587 +8 2 -2571300635581.146276 -2571300635581.146276407 +8 6 7031444034.531499 7031444034.53149906 +8 9 -1866544013697.195857 -1866544013697.19585702 +9 2 855948866655588.453742 855948866655588.4537415092 +9 3 -107434525.434154 -107434525.4341543802 +9 4 -194415646271340.181596 -194415646271340.1815956523 +9 5 -408725765384.257044 -408725765384.2570436602 +9 6 -2340666225110.299295 -2340666225110.2992952129 +9 7 2069634775752159.035759 2069634775752159.0357587 +9 8 -1866544013697.195857 -1866544013697.19585702 +9 9 621345559900192.420121 621345559900192.42012063 -- !query 479 @@ -3942,62 +3942,62 @@ SELECT t1.id1, t1.id2, t1.result, t2.expected -- !query 481 schema struct -- !query 481 output -2 3 -7967167.5673780000 -7967167.5673775051 -2 4 -4.4026750000 -4.4026748005 -2 5 -2094.1886690000 -2094.1886691456 -2 6 -365.6859990000 -365.6859989148 -2 7 0.4135750000 0.4135748378 -2 8 -458.5741670000 -458.5741672173 -2 9 1.3775730000 1.3775729995 -3 2 0.0000000000 -0.0000001255 -3 4 0.0000010000 0.0000005526 -3 5 0.0002630000 0.0002628523 -3 6 0.0000460000 0.0000458991 -3 7 0.0000000000 -0.0000000519 -3 8 0.0000580000 0.0000575580 -3 9 0.0000000000 -0.0000001729 -4 2 -0.2271350000 -0.2271346500 -4 3 1809619.8171460000 1809619.8171461717 -4 5 475.6628100000 475.6628104631 -4 6 83.0599610000 83.0599613844 -4 7 -0.0939370000 -0.0939371760 -4 8 104.1580830000 104.1580829837 -4 9 -0.3128950000 -0.3128945611 -5 2 -0.0004780000 -0.0004775119 -5 3 3804.4172830000 3804.4172832947 -5 4 0.0021020000 0.0021023296 -5 6 0.1746190000 0.1746194143 -5 7 -0.0001970000 -0.0001974869 -5 8 0.2189750000 0.2189746196 -5 9 -0.0006580000 -0.0006578075 -6 2 -0.0027350000 -0.0027345865 -6 3 21786.9089630000 21786.9089629374 -6 4 0.0120390000 0.0120394951 -6 5 5.7267400000 5.7267400867 -6 7 -0.0011310000 -0.0011309562 -6 8 1.2540110000 1.2540107321 -6 9 -0.0037670000 -0.0037670925 -7 2 2.4179420000 2.4179420715 -7 3 -19264149.6519720000 -19264149.6519721578 -7 4 -10.6454130000 -10.6454126273 -7 5 -5063.6268890000 -5063.6268888173 -7 6 -884.2075620000 -884.2075617401 -7 8 -1108.8057720000 -1108.8057718246 -7 9 3.3308920000 3.3308917120 -8 2 -0.0021810000 -0.0021806723 -8 3 17373.7819030000 17373.7819025522 -8 4 0.0096010000 0.0096007911 -8 5 4.5667390000 4.5667392951 -8 6 0.7974410000 0.7974413411 -8 7 -0.0009020000 -0.0009018712 -8 9 -0.0030040000 -0.0030040353 -9 2 0.7259140000 0.7259143438 -9 3 -5783481.2169480000 -5783481.2169483573 -9 4 -3.1959650000 -3.1959647889 -9 5 -1520.2015940000 -1520.2015936432 -9 6 -265.4567120000 -265.4567119543 -9 7 0.3002200000 0.3002199070 -9 8 -332.8855660000 -332.8855656982 +2 3 -7967167.567378 -7967167.5673775051 +2 4 -4.402675 -4.4026748005 +2 5 -2094.188669 -2094.1886691456 +2 6 -365.685999 -365.6859989148 +2 7 0.413575 0.4135748378 +2 8 -458.574167 -458.5741672173 +2 9 1.377573 1.3775729995 +3 2 0 -0.0000001255 +3 4 0.000001 0.0000005526 +3 5 0.000263 0.0002628523 +3 6 0.000046 0.0000458991 +3 7 0 -0.0000000519 +3 8 0.000058 0.000057558 +3 9 0 -0.0000001729 +4 2 -0.227135 -0.22713465 +4 3 1809619.817146 1809619.8171461717 +4 5 475.66281 475.6628104631 +4 6 83.059961 83.0599613844 +4 7 -0.093937 -0.093937176 +4 8 104.158083 104.1580829837 +4 9 -0.312895 -0.3128945611 +5 2 -0.000478 -0.0004775119 +5 3 3804.417283 3804.4172832947 +5 4 0.002102 0.0021023296 +5 6 0.174619 0.1746194143 +5 7 -0.000197 -0.0001974869 +5 8 0.218975 0.2189746196 +5 9 -0.000658 -0.0006578075 +6 2 -0.002735 -0.0027345865 +6 3 21786.908963 21786.9089629374 +6 4 0.012039 0.0120394951 +6 5 5.72674 5.7267400867 +6 7 -0.001131 -0.0011309562 +6 8 1.254011 1.2540107321 +6 9 -0.003767 -0.0037670925 +7 2 2.417942 2.4179420715 +7 3 -19264149.651972 -19264149.6519721578 +7 4 -10.645413 -10.6454126273 +7 5 -5063.626889 -5063.6268888173 +7 6 -884.207562 -884.2075617401 +7 8 -1108.805772 -1108.8057718246 +7 9 3.330892 3.330891712 +8 2 -0.002181 -0.0021806723 +8 3 17373.781903 17373.7819025522 +8 4 0.009601 0.0096007911 +8 5 4.566739 4.5667392951 +8 6 0.797441 0.7974413411 +8 7 -0.000902 -0.0009018712 +8 9 -0.003004 -0.0030040353 +9 2 0.725914 0.7259143438 +9 3 -5783481.216948 -5783481.2169483573 +9 4 -3.195965 -3.1959647889 +9 5 -1520.201594 -1520.2015936432 +9 6 -265.456712 -265.4567119543 +9 7 0.30022 0.300219907 +9 8 -332.885566 -332.8855656982 -- !query 482 @@ -4026,62 +4026,62 @@ SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected -- !query 484 schema struct -- !query 484 output -2 3 -7967167.5673780000 -7967167.5673775051 -2 4 -4.4026750000 -4.4026748005 -2 5 -2094.1886690000 -2094.1886691456 -2 6 -365.6859990000 -365.6859989148 -2 7 0.4135750000 0.4135748378 -2 8 -458.5741670000 -458.5741672173 -2 9 1.3775730000 1.3775729995 -3 2 0.0000000000 -0.0000001255 -3 4 0.0000010000 0.0000005526 -3 5 0.0002630000 0.0002628523 -3 6 0.0000460000 0.0000458991 -3 7 0.0000000000 -0.0000000519 -3 8 0.0000580000 0.0000575580 -3 9 0.0000000000 -0.0000001729 -4 2 -0.2271350000 -0.2271346500 -4 3 1809619.8171460000 1809619.8171461717 -4 5 475.6628100000 475.6628104631 -4 6 83.0599610000 83.0599613844 -4 7 -0.0939370000 -0.0939371760 -4 8 104.1580830000 104.1580829837 -4 9 -0.3128950000 -0.3128945611 -5 2 -0.0004780000 -0.0004775119 -5 3 3804.4172830000 3804.4172832947 -5 4 0.0021020000 0.0021023296 -5 6 0.1746190000 0.1746194143 -5 7 -0.0001970000 -0.0001974869 -5 8 0.2189750000 0.2189746196 -5 9 -0.0006580000 -0.0006578075 -6 2 -0.0027350000 -0.0027345865 -6 3 21786.9089630000 21786.9089629374 -6 4 0.0120390000 0.0120394951 -6 5 5.7267400000 5.7267400867 -6 7 -0.0011310000 -0.0011309562 -6 8 1.2540110000 1.2540107321 -6 9 -0.0037670000 -0.0037670925 -7 2 2.4179420000 2.4179420715 -7 3 -19264149.6519720000 -19264149.6519721578 -7 4 -10.6454130000 -10.6454126273 -7 5 -5063.6268890000 -5063.6268888173 -7 6 -884.2075620000 -884.2075617401 -7 8 -1108.8057720000 -1108.8057718246 -7 9 3.3308920000 3.3308917120 -8 2 -0.0021810000 -0.0021806723 -8 3 17373.7819030000 17373.7819025522 -8 4 0.0096010000 0.0096007911 -8 5 4.5667390000 4.5667392951 -8 6 0.7974410000 0.7974413411 -8 7 -0.0009020000 -0.0009018712 -8 9 -0.0030040000 -0.0030040353 -9 2 0.7259140000 0.7259143438 -9 3 -5783481.2169480000 -5783481.2169483573 -9 4 -3.1959650000 -3.1959647889 -9 5 -1520.2015940000 -1520.2015936432 -9 6 -265.4567120000 -265.4567119543 -9 7 0.3002200000 0.3002199070 -9 8 -332.8855660000 -332.8855656982 +2 3 -7967167.567378 -7967167.5673775051 +2 4 -4.402675 -4.4026748005 +2 5 -2094.188669 -2094.1886691456 +2 6 -365.685999 -365.6859989148 +2 7 0.413575 0.4135748378 +2 8 -458.574167 -458.5741672173 +2 9 1.377573 1.3775729995 +3 2 0 -0.0000001255 +3 4 0.000001 0.0000005526 +3 5 0.000263 0.0002628523 +3 6 0.000046 0.0000458991 +3 7 0 -0.0000000519 +3 8 0.000058 0.000057558 +3 9 0 -0.0000001729 +4 2 -0.227135 -0.22713465 +4 3 1809619.817146 1809619.8171461717 +4 5 475.66281 475.6628104631 +4 6 83.059961 83.0599613844 +4 7 -0.093937 -0.093937176 +4 8 104.158083 104.1580829837 +4 9 -0.312895 -0.3128945611 +5 2 -0.000478 -0.0004775119 +5 3 3804.417283 3804.4172832947 +5 4 0.002102 0.0021023296 +5 6 0.174619 0.1746194143 +5 7 -0.000197 -0.0001974869 +5 8 0.218975 0.2189746196 +5 9 -0.000658 -0.0006578075 +6 2 -0.002735 -0.0027345865 +6 3 21786.908963 21786.9089629374 +6 4 0.012039 0.0120394951 +6 5 5.72674 5.7267400867 +6 7 -0.001131 -0.0011309562 +6 8 1.254011 1.2540107321 +6 9 -0.003767 -0.0037670925 +7 2 2.417942 2.4179420715 +7 3 -19264149.651972 -19264149.6519721578 +7 4 -10.645413 -10.6454126273 +7 5 -5063.626889 -5063.6268888173 +7 6 -884.207562 -884.2075617401 +7 8 -1108.805772 -1108.8057718246 +7 9 3.330892 3.330891712 +8 2 -0.002181 -0.0021806723 +8 3 17373.781903 17373.7819025522 +8 4 0.009601 0.0096007911 +8 5 4.566739 4.5667392951 +8 6 0.797441 0.7974413411 +8 7 -0.000902 -0.0009018712 +8 9 -0.003004 -0.0030040353 +9 2 0.725914 0.7259143438 +9 3 -5783481.216948 -5783481.2169483573 +9 4 -3.195965 -3.1959647889 +9 5 -1520.201594 -1520.2015936432 +9 6 -265.456712 -265.4567119543 +9 7 0.30022 0.300219907 +9 8 -332.885566 -332.8855656982 -- !query 485 @@ -4196,12 +4196,12 @@ SELECT t1.id1, t1.result, t2.expected -- !query 496 schema struct -- !query 496 output -2 224790267919917440.0000000000 224790267919917955.1326161858 -4 7405685069595001.0000000000 7405685069594999.0773399947 -5 5068226527.3212630000 5068226527.3212726541 -6 281839893606.9936500000 281839893606.9937234336 -8 167361463828.0749000000 167361463828.0749132007 -9 107511333880051872.0000000000 107511333880052007.0414112467 +2 224790267919917440 224790267919917955.1326161858 +4 7405685069595001 7405685069594999.0773399947 +5 5068226527.321263 5068226527.3212726541 +6 281839893606.99365 281839893606.9937234336 +8 167361463828.0749 167361463828.0749132007 +9 107511333880051872 107511333880052007.0414112467 -- !query 497 @@ -4209,7 +4209,7 @@ SELECT AVG(val) FROM num_data -- !query 497 schema struct -- !query 497 output --13430913.59224232070000 +-13430913.5922423207 -- !query 498 @@ -4273,11 +4273,11 @@ SELECT * FROM fract_only -- !query 505 schema struct -- !query 505 output -1 0.0000 -2 0.1000 +1 0 +2 0.1 4 -0.9999 5 0.9999 -7 0.0000 +7 0 8 0.0002 @@ -4406,13 +4406,13 @@ SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round -- !query 521 schema struct -- !query 521 output --0.000001000000000000 0 0 -1 0 --5.499999000000000000 -5 -5 -6 -5 --5.500000000000000000 -5 -5 -6 -6 -0.000000000000000000 0 0 0 0 -0.000000100000000000 1 1 0 0 -9.499999900000000000 10 10 9 9 -9.500000000000000000 10 10 9 10 +-0.000001 0 0 -1 0 +-5.499999 -5 -5 -6 -5 +-5.5 -5 -5 -6 -6 +0 0 0 0 0 +0.0000001 1 1 0 0 +9.4999999 10 10 9 9 +9.5 10 10 9 10 -- !query 522 @@ -4476,11 +4476,11 @@ SELECT * FROM num_input_test -- !query 529 schema struct -- !query 529 output --555.500000000000000000 --93853.000000000000000000 -123.000000000000000000 -3245874.000000000000000000 -555.500000000000000000 +-555.5 +-93853 +123 +3245874 +555.5 -- !query 530 @@ -4488,7 +4488,7 @@ select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 -- !query 530 schema struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) / CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,6)> -- !query 530 output -1.000000 +1 -- !query 531 @@ -4536,7 +4536,7 @@ select mod (70.0,70) -- !query 536 schema struct<(CAST(70.0 AS DECIMAL(3,1)) % CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(3,1)> -- !query 536 output -0.0 +0 -- !query 537 @@ -4552,7 +4552,7 @@ select 70.0 / 70 -- !query 538 schema struct<(CAST(70.0 AS DECIMAL(3,1)) / CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(8,6)> -- !query 538 output -1.000000 +1 -- !query 539 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out index a106e33c2a595..e54de1d6fdbdc 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out @@ -336,8 +336,8 @@ VALUES (1,2), (3,4+4), (7,77.7) -- !query 17 schema struct -- !query 17 output -1 2.0 -3 8.0 +1 2 +3 8 7 77.7 @@ -350,14 +350,14 @@ TABLE int8_tbl -- !query 18 schema struct -- !query 18 output -1 2.0 -123 456.0 -123 4567890123456789.0 -3 8.0 -4 57.0 -4567890123456789 -4567890123456789.0 -4567890123456789 123.0 -4567890123456789 4567890123456789.0 +1 2 +123 456 +123 4567890123456789 +3 8 +4 57 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 7 77.7 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out index 460b6d1fdd02f..6cb3690a464e1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out @@ -258,11 +258,11 @@ SELECT '' AS `54`, d1 as `timestamp`, -- !query 27 schema struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:decimal(8,6)> -- !query 27 output - 1969-12-31 16:00:00 1969 12 31 16 0 0.000000 - 1997-01-02 00:00:00 1997 1 2 0 0 0.000000 - 1997-01-02 03:04:05 1997 1 2 3 4 5.000000 - 1997-02-10 17:32:01 1997 2 10 17 32 1.000000 - 2001-09-22 18:19:20 2001 9 22 18 19 20.000000 + 1969-12-31 16:00:00 1969 12 31 16 0 0 + 1997-01-02 00:00:00 1997 1 2 0 0 0 + 1997-01-02 03:04:05 1997 1 2 3 4 5 + 1997-02-10 17:32:01 1997 2 10 17 32 1 + 2001-09-22 18:19:20 2001 9 22 18 19 20 -- !query 28 @@ -273,11 +273,11 @@ SELECT '' AS `54`, d1 as `timestamp`, -- !query 28 schema struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> -- !query 28 output - 1969-12-31 16:00:00 4 0.000 0 - 1997-01-02 00:00:00 1 0.000 0 - 1997-01-02 03:04:05 1 5000.000 5000000 - 1997-02-10 17:32:01 1 1000.000 1000000 - 2001-09-22 18:19:20 3 20000.000 20000000 + 1969-12-31 16:00:00 4 0 0 + 1997-01-02 00:00:00 1 0 0 + 1997-01-02 03:04:05 1 5000 5000000 + 1997-02-10 17:32:01 1 1000 1000000 + 2001-09-22 18:19:20 3 20000 20000000 -- !query 29 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out index 703463e6d6b5a..05dedc547086e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out @@ -117,7 +117,7 @@ SELECT 1.1 AS two UNION SELECT 2 ORDER BY 1 struct -- !query 11 output 1.1 -2.0 +2 -- !query 12 @@ -125,7 +125,7 @@ SELECT 1 AS two UNION SELECT 2.2 ORDER BY 1 -- !query 12 schema struct -- !query 12 output -1.0 +1 2.2 @@ -143,7 +143,7 @@ SELECT 1.1 AS two UNION ALL SELECT 2 ORDER BY 1 struct -- !query 14 output 1.1 -2.0 +2 -- !query 15 @@ -161,8 +161,8 @@ SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 struct -- !query 16 output 1.1 -2.0 -3.0 +2 +3 -- !query 17 @@ -180,8 +180,8 @@ SELECT 1.1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 struct -- !query 18 output 1.1 -2.0 -2.0 +2 +2 -- !query 19 @@ -190,7 +190,7 @@ SELECT 1.1 AS two UNION (SELECT 2 UNION ALL SELECT 2) ORDER BY 1 struct -- !query 19 output 1.1 -2.0 +2 -- !query 20 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index 54ceacd3b3b3e..f4ada1e18a90a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -44,8 +44,8 @@ SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWIN -- !query 3 schema struct -- !query 3 output -1 2.00000 -2 2.50000 +1 2 +2 2.5 3 NULL 4 NULL @@ -104,8 +104,8 @@ SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) -- !query 8 schema struct -- !query 8 output -3.00 -5.00 +3 +5 6.01 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out index 3045cff1de19b..f02f760727976 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out @@ -23,7 +23,7 @@ WHERE b = 1.0 -- Matches (null, 1.0) -- !query 1 schema struct -- !query 1 output -NULL 1.0 +NULL 1 -- !query 2 @@ -45,4 +45,4 @@ WHERE b = 5.0 -- Matches (4, 5.0) -- !query 3 schema struct -- !query 3 output -4 5.0 +4 5 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out index 5ccea3846cd36..a27a66e3f27f5 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out @@ -37,9 +37,9 @@ WHERE (a, b) NOT IN (SELECT * -- !query 2 schema struct -- !query 2 output -2 3.0 -4 5.0 -NULL 1.0 +2 3 +4 5 +NULL 1 NULL NULL @@ -91,7 +91,7 @@ WHERE b = 1.0 -- Matches (null, 1.0) -- !query 6 schema struct -- !query 6 output -NULL 1.0 +NULL 1 -- !query 7 @@ -117,4 +117,4 @@ WHERE b = 5.0 -- Matches (4, 5.0) -- !query 8 schema struct -- !query 8 output -4 5.0 +4 5 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out index 9190e9c1e7a4b..cf8f03eaa9311 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out @@ -54,4 +54,4 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 4 schema struct -- !query 4 output -2 3.0 +2 3 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out index d7473e1ada151..d07981cfd11e5 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out @@ -35,9 +35,9 @@ WHERE a NOT IN (SELECT c -- !query 2 schema struct -- !query 2 output -2 3.0 -4 5.0 -NULL 1.0 +2 3 +4 5 +NULL 1 -- !query 3 @@ -88,7 +88,7 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 6 schema struct -- !query 6 output -2 3.0 +2 3 -- !query 7 @@ -100,9 +100,9 @@ WHERE a NOT IN (SELECT c -- !query 7 schema struct -- !query 7 output -2 3.0 -4 5.0 -NULL 1.0 +2 3 +4 5 +NULL 1 -- !query 8 @@ -115,7 +115,7 @@ WHERE b = 1.0 -- Only matches (null, 1.0) -- !query 8 schema struct -- !query 8 output -NULL 1.0 +NULL 1 -- !query 9 @@ -128,4 +128,4 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 9 schema struct -- !query 9 output -2 3.0 +2 3 diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index 653dc1ca357f8..1a2bd5ea91cde 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -93,5 +93,5 @@ SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) struct -- !query 10 output 1 a 1 8.5 -2 b 2 1.0 +2 b 2 1 3 c 3 3.2 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index e479a6d73a182..6ee7f59d69877 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -2211,7 +2211,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t -- !query 265 schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) / CAST(1 AS DECIMAL(3,0))):decimal(9,6)> -- !query 265 output -1.000000 +1 -- !query 266 @@ -2219,7 +2219,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t -- !query 266 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> -- !query 266 output -1.000000 +1 -- !query 267 @@ -2227,7 +2227,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t -- !query 267 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 267 output -1.00000000000 +1 -- !query 268 @@ -2235,7 +2235,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t -- !query 268 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> -- !query 268 output -1.000000000000000000000 +1 -- !query 269 @@ -2243,7 +2243,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t -- !query 269 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> -- !query 269 output -1.000000 +1 -- !query 270 @@ -2251,7 +2251,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t -- !query 270 schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) / CAST(1 AS DECIMAL(5,0))):decimal(11,6)> -- !query 270 output -1.000000 +1 -- !query 271 @@ -2259,7 +2259,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t -- !query 271 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 271 output -1.00000000000 +1 -- !query 272 @@ -2267,7 +2267,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t -- !query 272 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> -- !query 272 output -1.000000000000000000000 +1 -- !query 273 @@ -2275,7 +2275,7 @@ SELECT cast(1 as int) / cast(1 as decimal(3, 0)) FROM t -- !query 273 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 273 output -1.000000 +1 -- !query 274 @@ -2283,7 +2283,7 @@ SELECT cast(1 as int) / cast(1 as decimal(5, 0)) FROM t -- !query 274 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 274 output -1.000000 +1 -- !query 275 @@ -2291,7 +2291,7 @@ SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t -- !query 275 schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 275 output -1.00000000000 +1 -- !query 276 @@ -2299,7 +2299,7 @@ SELECT cast(1 as int) / cast(1 as decimal(20, 0)) FROM t -- !query 276 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 276 output -1.000000000000000000000 +1 -- !query 277 @@ -2307,7 +2307,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t -- !query 277 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 277 output -1.000000 +1 -- !query 278 @@ -2315,7 +2315,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t -- !query 278 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 278 output -1.000000 +1 -- !query 279 @@ -2323,7 +2323,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t -- !query 279 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 279 output -1.00000000000 +1 -- !query 280 @@ -2331,7 +2331,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t -- !query 280 schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) / CAST(1 AS DECIMAL(20,0))):decimal(38,18)> -- !query 280 output -1.000000000000000000 +1 -- !query 281 @@ -2403,7 +2403,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t -- !query 289 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 289 output -1.000000 +1 -- !query 290 @@ -2411,7 +2411,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t -- !query 290 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 290 output -1.000000 +1 -- !query 291 @@ -2419,7 +2419,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 291 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 291 output -1.00000000000 +1 -- !query 292 @@ -2427,7 +2427,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t -- !query 292 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 292 output -1.000000000000000000000 +1 -- !query 293 @@ -2543,7 +2543,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t -- !query 305 schema struct<(CAST(1 AS DECIMAL(3,0)) / CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(9,6)> -- !query 305 output -1.000000 +1 -- !query 306 @@ -2551,7 +2551,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t -- !query 306 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> -- !query 306 output -1.000000 +1 -- !query 307 @@ -2559,7 +2559,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t -- !query 307 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 307 output -1.000000 +1 -- !query 308 @@ -2567,7 +2567,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t -- !query 308 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 308 output -1.000000 +1 -- !query 309 @@ -2575,7 +2575,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t -- !query 309 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> -- !query 309 output -1.000000 +1 -- !query 310 @@ -2583,7 +2583,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t -- !query 310 schema struct<(CAST(1 AS DECIMAL(5,0)) / CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(11,6)> -- !query 310 output -1.000000 +1 -- !query 311 @@ -2591,7 +2591,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t -- !query 311 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 311 output -1.000000 +1 -- !query 312 @@ -2599,7 +2599,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t -- !query 312 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 312 output -1.000000 +1 -- !query 313 @@ -2607,7 +2607,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as int) FROM t -- !query 313 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 313 output -1.00000000000 +1 -- !query 314 @@ -2615,7 +2615,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as int) FROM t -- !query 314 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 314 output -1.00000000000 +1 -- !query 315 @@ -2623,7 +2623,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t -- !query 315 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> -- !query 315 output -1.00000000000 +1 -- !query 316 @@ -2631,7 +2631,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as int) FROM t -- !query 316 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 316 output -1.00000000000 +1 -- !query 317 @@ -2639,7 +2639,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t -- !query 317 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> -- !query 317 output -1.000000000000000000000 +1 -- !query 318 @@ -2647,7 +2647,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t -- !query 318 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> -- !query 318 output -1.000000000000000000000 +1 -- !query 319 @@ -2655,7 +2655,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t -- !query 319 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 319 output -1.000000000000000000000 +1 -- !query 320 @@ -2663,7 +2663,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t -- !query 320 schema struct<(CAST(1 AS DECIMAL(20,0)) / CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(38,18)> -- !query 320 output -1.000000000000000000 +1 -- !query 321 @@ -2735,7 +2735,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 329 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 329 output -1.00000000000 +1 -- !query 330 @@ -2743,7 +2743,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 330 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 330 output -1.00000000000 +1 -- !query 331 @@ -2751,7 +2751,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 331 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 331 output -1.00000000000 +1 -- !query 332 @@ -2759,7 +2759,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 332 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 332 output -1.00000000000 +1 -- !query 333 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out index 97cdf01763a70..017e0fea30e90 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out @@ -63,7 +63,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t -- !query 7 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 7 output -1.00000000000 +1 -- !query 8 @@ -163,7 +163,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t -- !query 19 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 19 output -1.00000000000 +1 -- !query 20 @@ -263,7 +263,7 @@ SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t -- !query 31 schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 31 output -1.00000000000 +1 -- !query 32 @@ -363,7 +363,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t -- !query 43 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 43 output -1.00000000000 +1 -- !query 44 @@ -615,7 +615,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t -- !query 73 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 73 output -1.000000 +1 -- !query 74 @@ -623,7 +623,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t -- !query 74 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 74 output -1.000000 +1 -- !query 75 @@ -631,7 +631,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t -- !query 75 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> -- !query 75 output -1.00000000000 +1 -- !query 76 @@ -639,7 +639,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t -- !query 76 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 76 output -1.000000000000000000000 +1 -- !query 77 @@ -663,7 +663,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 79 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 79 output -1.00000000000 +1 -- !query 80 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out index 835abe3d32775..84b5e10dbeb8e 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out @@ -42,12 +42,12 @@ FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 -- !query 3 schema struct -- !query 3 output -1.0 1 -1.0 1 -1.0 a -2.0 4 -2.0 4 -2.0 b +1 1 +1 1 +1 a +2 4 +2 4 +2 b -- !query 4 diff --git a/sql/core/src/test/resources/sql-tests/results/union.sql.out b/sql/core/src/test/resources/sql-tests/results/union.sql.out index 95a10f0b0a8c9..b023df825d814 100644 --- a/sql/core/src/test/resources/sql-tests/results/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/union.sql.out @@ -42,12 +42,12 @@ FROM (SELECT * FROM t1 -- !query 3 schema struct -- !query 3 output -1.0 1 -1.0 1 -1.0 a -2.0 4 -2.0 4 -2.0 b +1 1 +1 1 +1 a +2 4 +2 4 +2 b -- !query 4 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index 4d1bb470e4e2e..104cf4c58d617 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -43,19 +43,4 @@ class HiveResultSuite extends SharedSparkSession { val tpe = new ExamplePointUDT() assert(HiveResult.toHiveString((point, tpe)) === "(50.0, 50.0)") } - - test("decimal formatting in hive result") { - val df = Seq(new java.math.BigDecimal("1")).toDS() - Seq(2, 6, 18).foreach { scala => - val executedPlan = - df.selectExpr(s"CAST(value AS decimal(38, $scala))").queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan) - assert(result.head.split("\\.").last.length === scala) - } - - val executedPlan = Seq(java.math.BigDecimal.ZERO).toDS() - .selectExpr(s"CAST(value AS decimal(38, 8))").queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan) - assert(result.head === "0.00000000") - } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 856102bc900a8..04e7f579ff712 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -393,11 +393,4 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { "select concat('Test4', ';');" -> "Test4;" ) } - - test("Pad Decimal numbers with trailing zeros to the scale of the column") { - runCliWithin(1.minute)( - "SELECT CAST(1 AS DECIMAL(38, 18));" - -> "1.000000000000000000" - ) - } } diff --git a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e index a298a84cb2c5a..f59549a6e4a46 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e +++ b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e @@ -1,38 +1,38 @@ NULL 0 --1234567890.1234567890000000000000000 -1234567890 --4400.0000000000000000000000000 4400 --1255.4900000000000000000000000 -1255 --1.1220000000000000000000000 -11 --1.1200000000000000000000000 -1 --1.1200000000000000000000000 -1 --0.3330000000000000000000000 0 --0.3300000000000000000000000 0 --0.3000000000000000000000000 0 -0.0000000000000000000000000 0 -0.0000000000000000000000000 0 -0.0000000000000000000000000 0 -0.0100000000000000000000000 0 -0.0200000000000000000000000 0 -0.1000000000000000000000000 0 -0.2000000000000000000000000 0 -0.3000000000000000000000000 0 -0.3300000000000000000000000 0 -0.3330000000000000000000000 0 +-1234567890.123456789 -1234567890 +-4400 4400 +-1255.49 -1255 +-1.122 -11 +-1.12 -1 +-1.12 -1 +-0.333 0 +-0.33 0 +-0.3 0 +0 0 +0 0 +0 0 +0.01 0 +0.02 0 +0.1 0 +0.2 0 +0.3 0 +0.33 0 +0.333 0 0.9999999999999999999999999 1 -1.0000000000000000000000000 1 -1.0000000000000000000000000 1 -1.1200000000000000000000000 1 -1.1220000000000000000000000 1 -2.0000000000000000000000000 2 -2.0000000000000000000000000 2 -3.1400000000000000000000000 3 -3.1400000000000000000000000 3 -3.1400000000000000000000000 3 -3.1400000000000000000000000 4 -10.0000000000000000000000000 10 -20.0000000000000000000000000 20 -100.0000000000000000000000000 100 -124.0000000000000000000000000 124 -125.2000000000000000000000000 125 -200.0000000000000000000000000 200 -1234567890.1234567800000000000000000 1234567890 +1 1 +1 1 +1.12 1 +1.122 1 +2 2 +2 2 +3.14 3 +3.14 3 +3.14 3 +3.14 4 +10 10 +20 20 +100 100 +124 124 +125.2 125 +200 200 +1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 index 60df68a2e3ab5..6bada475c6d3d 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 +++ b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 @@ -1,38 +1,38 @@ NULL NULL --1234567890.1234567890000000000000000 -3703703670.3703703670000000000000000 --4400.0000000000000000000000000 -13200.0000000000000000000000000 --1255.4900000000000000000000000 -3766.4700000000000000000000000 --1.1220000000000000000000000 -3.3660000000000000000000000 --1.1200000000000000000000000 -3.3600000000000000000000000 --1.1200000000000000000000000 -3.3600000000000000000000000 --0.3330000000000000000000000 -0.9990000000000000000000000 --0.3300000000000000000000000 -0.9900000000000000000000000 --0.3000000000000000000000000 -0.9000000000000000000000000 -0.0000000000000000000000000 0.0000000000000000000000000 -0.0000000000000000000000000 0.0000000000000000000000000 -0.0000000000000000000000000 0.0000000000000000000000000 -0.0100000000000000000000000 0.0300000000000000000000000 -0.0200000000000000000000000 0.0600000000000000000000000 -0.1000000000000000000000000 0.3000000000000000000000000 -0.2000000000000000000000000 0.6000000000000000000000000 -0.3000000000000000000000000 0.9000000000000000000000000 -0.3300000000000000000000000 0.9900000000000000000000000 -0.3330000000000000000000000 0.9990000000000000000000000 +-1234567890.123456789 -3703703670.370370367 +-4400 -13200 +-1255.49 -3766.47 +-1.122 -3.366 +-1.12 -3.36 +-1.12 -3.36 +-0.333 -0.999 +-0.33 -0.99 +-0.3 -0.9 +0 0 +0 0 +0 0 +0.01 0.03 +0.02 0.06 +0.1 0.3 +0.2 0.6 +0.3 0.9 +0.33 0.99 +0.333 0.999 0.9999999999999999999999999 2.9999999999999999999999997 -1.0000000000000000000000000 3.0000000000000000000000000 -1.0000000000000000000000000 3.0000000000000000000000000 -1.1200000000000000000000000 3.3600000000000000000000000 -1.1220000000000000000000000 3.3660000000000000000000000 -2.0000000000000000000000000 6.0000000000000000000000000 -2.0000000000000000000000000 6.0000000000000000000000000 -3.1400000000000000000000000 9.4200000000000000000000000 -3.1400000000000000000000000 9.4200000000000000000000000 -3.1400000000000000000000000 9.4200000000000000000000000 -3.1400000000000000000000000 9.4200000000000000000000000 -10.0000000000000000000000000 30.0000000000000000000000000 -20.0000000000000000000000000 60.0000000000000000000000000 -100.0000000000000000000000000 300.0000000000000000000000000 -124.0000000000000000000000000 372.0000000000000000000000000 -125.2000000000000000000000000 375.6000000000000000000000000 -200.0000000000000000000000000 600.0000000000000000000000000 -1234567890.1234567800000000000000000 3703703670.3703703400000000000000000 +1 3 +1 3 +1.12 3.36 +1.122 3.366 +2 6 +2 6 +3.14 9.42 +3.14 9.42 +3.14 9.42 +3.14 9.42 +10 30 +20 60 +100 300 +124 372 +125.2 375.6 +200 600 +1234567890.12345678 3703703670.37037034 diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 index a26c8b7d12886..93cdc5c85645c 100644 --- a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 +++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 @@ -1,38 +1,38 @@ NULL 0 --1234567890.123456789000000000 -1234567890 --4400.000000000000000000 4400 --1255.490000000000000000 -1255 --1.122000000000000000 -11 --1.120000000000000000 -1 --1.120000000000000000 -1 --0.333000000000000000 0 --0.330000000000000000 0 --0.300000000000000000 0 -0.000000000000000000 0 -0.000000000000000000 0 -0.000000000000000000 0 -0.010000000000000000 0 -0.020000000000000000 0 -0.100000000000000000 0 -0.200000000000000000 0 -0.300000000000000000 0 -0.330000000000000000 0 -0.333000000000000000 0 -1.000000000000000000 1 -1.000000000000000000 1 -1.000000000000000000 1 -1.120000000000000000 1 -1.122000000000000000 1 -2.000000000000000000 2 -2.000000000000000000 2 -3.140000000000000000 3 -3.140000000000000000 3 -3.140000000000000000 3 -3.140000000000000000 4 -10.000000000000000000 10 -20.000000000000000000 20 -100.000000000000000000 100 -124.000000000000000000 124 -125.200000000000000000 125 -200.000000000000000000 200 -1234567890.123456780000000000 1234567890 +-1234567890.123456789 -1234567890 +-4400 4400 +-1255.49 -1255 +-1.122 -11 +-1.12 -1 +-1.12 -1 +-0.333 0 +-0.33 0 +-0.3 0 +0 0 +0 0 +0 0 +0.01 0 +0.02 0 +0.1 0 +0.2 0 +0.3 0 +0.33 0 +0.333 0 +1 1 +1 1 +1 1 +1.12 1 +1.122 1 +2 2 +2 2 +3.14 3 +3.14 3 +3.14 3 +3.14 4 +10 10 +20 20 +100 100 +124 124 +125.2 125 +200 200 +1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 index 33ea4edf780a6..62d71abc6fc7d 100644 --- a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 @@ -3,7 +3,7 @@ 65536 32.68 65536 33.45 65536 58.86 -65536 75.70 +65536 75.7 65536 83.48 65537 NULL 65537 4.49 @@ -57,9 +57,9 @@ 65548 75.39 65548 77.24 65549 NULL -65549 13.30 +65549 13.3 65549 28.93 -65549 50.60 +65549 50.6 65549 55.04 65549 64.91 65549 76.06 @@ -70,7 +70,7 @@ 65550 33.01 65550 57.63 65550 91.38 -65550 96.90 +65550 96.9 65551 NULL 65551 39.43 65551 73.93 @@ -99,7 +99,7 @@ 65559 29.55 65559 56.06 65559 73.94 -65559 83.50 +65559 83.5 65560 NULL 65560 16.86 65560 21.81 @@ -128,7 +128,7 @@ 65565 NULL 65565 81.72 65566 NULL -65566 7.80 +65566 7.8 65567 NULL 65568 NULL 65568 21.79 @@ -136,14 +136,14 @@ 65569 NULL 65570 NULL 65570 17.09 -65570 18.20 +65570 18.2 65570 25.57 65570 45.23 -65570 76.80 +65570 76.8 65571 NULL 65571 26.64 65571 40.68 -65571 82.50 +65571 82.5 65572 NULL 65572 22.64 65572 43.49 @@ -156,9 +156,9 @@ 65574 31.28 65574 38.54 65575 NULL -65575 17.00 +65575 17 65575 32.85 -65575 83.40 +65575 83.4 65576 NULL 65576 2.04 65576 4.88 @@ -166,7 +166,7 @@ 65577 NULL 65578 NULL 65578 16.01 -65578 41.10 +65578 41.1 65578 51.36 65578 54.35 65578 58.78 @@ -188,7 +188,7 @@ 65582 NULL 65582 1.23 65582 9.35 -65582 96.60 +65582 96.6 65583 NULL 65583 28.07 65583 50.57 @@ -218,7 +218,7 @@ 65588 98.33 65589 NULL 65589 49.49 -65589 72.30 +65589 72.3 65589 74.83 65589 94.73 65590 NULL @@ -240,8 +240,8 @@ 65595 NULL 65595 8.76 65595 67.56 -65595 72.70 -65595 89.60 +65595 72.7 +65595 89.6 65595 90.24 65596 NULL 65596 12.72 @@ -252,7 +252,7 @@ 65597 37.41 65597 69.05 65598 NULL -65598 63.30 +65598 63.3 65599 NULL 65599 0.56 65599 4.93 @@ -283,7 +283,7 @@ 65605 NULL 65606 NULL 65606 7.51 -65606 24.80 +65606 24.8 65606 57.69 65606 67.94 65606 87.16 @@ -294,9 +294,9 @@ 65607 75.86 65607 91.52 65608 NULL -65608 48.90 +65608 48.9 65608 69.42 -65608 87.90 +65608 87.9 65609 NULL 65610 NULL 65610 7.59 @@ -309,7 +309,7 @@ 65611 64.89 65612 NULL 65612 16.05 -65612 25.10 +65612 25.1 65612 52.64 65613 NULL 65614 NULL @@ -317,17 +317,17 @@ 65614 94.47 65615 NULL 65615 10.79 -65615 39.40 +65615 39.4 65615 99.88 65616 NULL -65616 75.20 +65616 75.2 65617 NULL 65617 18.51 65617 47.45 -65617 64.90 +65617 64.9 65618 NULL 65618 10.06 -65618 16.60 +65618 16.6 65618 81.99 65618 88.38 65619 NULL @@ -348,20 +348,20 @@ 65622 28.37 65622 50.08 65622 74.31 -65622 88.60 -65622 93.70 +65622 88.6 +65622 93.7 65623 NULL 65623 30.83 65623 31.22 65623 39.74 65623 48.51 65623 95.58 -65623 97.20 +65623 97.2 65624 NULL 65624 58.02 65624 65.31 65624 70.08 -65624 93.30 +65624 93.3 65625 NULL 65625 20.61 65625 42.86 @@ -377,13 +377,13 @@ 65628 NULL 65628 14.83 65628 30.43 -65628 37.80 +65628 37.8 65628 74.31 65628 83.26 65629 NULL 65629 19.33 65629 58.81 -65629 72.90 +65629 72.9 65630 NULL 65630 72.13 65631 NULL @@ -412,7 +412,7 @@ 65637 48.88 65637 93.41 65638 NULL -65638 11.20 +65638 11.2 65638 19.13 65639 NULL 65640 NULL @@ -477,20 +477,20 @@ 65654 26.73 65654 29.85 65654 37.74 -65654 37.80 +65654 37.8 65654 53.55 65654 88.23 65655 NULL 65655 77.41 65656 NULL -65656 14.00 +65656 14 65656 14.96 65656 53.27 65656 64.44 65656 82.67 65657 NULL 65657 11.93 -65657 26.40 +65657 26.4 65657 64.39 65657 65.01 65658 NULL @@ -506,8 +506,8 @@ 65659 NULL 65659 8.95 65659 46.57 -65659 53.80 -65659 94.30 +65659 53.8 +65659 94.3 65659 94.69 65659 95.71 65659 99.87 @@ -517,7 +517,7 @@ 65661 NULL 65661 5.24 65661 8.06 -65661 26.80 +65661 26.8 65661 68.98 65662 NULL 65662 59.92 @@ -531,10 +531,10 @@ 65663 94.16 65664 NULL 65664 11.46 -65664 27.60 +65664 27.6 65664 34.71 65664 38.42 -65664 45.40 +65664 45.4 65664 55.82 65664 97.64 65665 NULL @@ -543,13 +543,13 @@ 65666 83.95 65667 NULL 65667 13.96 -65667 63.90 +65667 63.9 65667 97.87 65668 NULL 65669 NULL 65669 1.76 65669 16.95 -65669 38.60 +65669 38.6 65669 54.25 65669 93.79 65670 NULL @@ -561,12 +561,12 @@ 65671 8.65 65671 52.05 65672 NULL -65672 52.60 -65672 58.10 +65672 52.6 +65672 58.1 65672 64.09 65672 75.27 65673 NULL -65673 0.90 +65673 0.9 65673 33.27 65673 43.81 65673 87.78 @@ -576,7 +576,7 @@ 65675 24.19 65675 35.33 65675 35.78 -65675 79.90 +65675 79.9 65675 83.09 65675 87.36 65676 NULL @@ -591,19 +591,19 @@ 65677 87.67 65678 NULL 65678 8.72 -65678 33.90 +65678 33.9 65679 NULL 65679 64.15 65680 NULL 65680 1.01 65680 34.08 65680 54.11 -65680 55.30 +65680 55.3 65680 65.88 65681 NULL 65681 35.45 65681 41.57 -65681 61.30 +65681 61.3 65681 71.17 65681 75.85 65682 NULL @@ -641,7 +641,7 @@ 65691 28.47 65691 56.02 65691 58.01 -65691 69.80 +65691 69.8 65691 76.98 65692 NULL 65692 54.76 @@ -655,19 +655,19 @@ 65694 NULL 65694 58.23 65694 82.24 -65694 88.50 +65694 88.5 65695 NULL 65695 57.33 65695 59.96 65695 77.09 65696 NULL 65696 17.35 -65696 40.30 +65696 40.3 65696 54.02 65697 NULL 65697 3.18 65697 50.01 -65697 67.90 +65697 67.9 65697 86.79 65697 90.16 65698 NULL @@ -685,9 +685,9 @@ 65701 1.81 65701 6.35 65702 NULL -65702 37.60 +65702 37.6 65702 55.68 -65702 79.50 +65702 79.5 65703 NULL 65703 37.18 65703 40.81 @@ -708,23 +708,23 @@ 65706 55.94 65706 72.87 65707 NULL -65707 76.20 +65707 76.2 65708 NULL 65708 1.29 65709 NULL 65709 5.64 65709 49.79 65710 NULL -65710 86.70 +65710 86.7 65711 NULL 65711 8.66 65711 50.26 65711 71.89 65711 78.69 -65711 96.10 +65711 96.1 65712 NULL 65712 30.27 -65712 34.70 +65712 34.7 65712 49.69 65712 53.65 65713 NULL @@ -739,11 +739,11 @@ 65715 39.62 65715 54.79 65715 81.28 -65715 89.40 +65715 89.4 65716 NULL -65716 9.00 +65716 9 65716 10.07 -65716 33.40 +65716 33.4 65716 71.53 65716 85.93 65717 NULL @@ -758,10 +758,10 @@ 65719 NULL 65719 51.13 65719 66.85 -65719 82.10 +65719 82.1 65720 NULL 65720 2.72 -65720 18.80 +65720 18.8 65720 22.34 65720 62.04 65721 NULL @@ -775,7 +775,7 @@ 65722 1.76 65722 38.82 65723 NULL -65723 39.90 +65723 39.9 65724 NULL 65724 10.52 65724 36.05 @@ -784,7 +784,7 @@ 65724 85.52 65725 NULL 65726 NULL -65726 6.00 +65726 6 65726 60.46 65727 NULL 65727 19.81 @@ -796,7 +796,7 @@ 65729 NULL 65730 NULL 65730 1.35 -65730 30.60 +65730 30.6 65730 81.44 65731 NULL 65731 24.48 @@ -810,14 +810,14 @@ 65733 20.72 65733 88.46 65733 93.45 -65733 99.80 +65733 99.8 65734 NULL 65734 31.71 65735 NULL 65735 12.67 65735 61.16 65736 NULL -65736 28.90 +65736 28.9 65736 48.54 65736 86.51 65737 NULL @@ -828,10 +828,10 @@ 65738 NULL 65738 30.94 65738 82.32 -65738 95.10 +65738 95.1 65739 NULL 65739 74.77 -65739 92.40 +65739 92.4 65740 NULL 65740 7.49 65740 58.65 @@ -840,9 +840,9 @@ 65742 6.61 65742 43.84 65743 NULL -65743 26.60 +65743 26.6 65743 52.65 -65743 62.00 +65743 62 65744 NULL 65744 46.98 65745 NULL @@ -853,11 +853,11 @@ 65746 36.74 65746 93.21 65746 97.52 -65746 98.10 +65746 98.1 65747 NULL 65747 11.16 65747 15.07 -65747 21.80 +65747 21.8 65747 39.77 65747 52.77 65747 71.87 @@ -865,7 +865,7 @@ 65748 29.49 65749 NULL 65749 15.14 -65749 45.00 +65749 45 65749 65.49 65749 73.24 65750 NULL @@ -888,12 +888,12 @@ 65755 NULL 65755 11.23 65755 22.44 -65755 64.00 +65755 64 65755 67.54 65755 76.75 65755 81.44 65755 90.08 -65755 96.80 +65755 96.8 65756 NULL 65756 1.45 65756 11.81 @@ -907,10 +907,10 @@ 65758 25.62 65758 56.56 65758 60.88 -65758 94.90 +65758 94.9 65759 NULL 65759 10.63 -65759 14.10 +65759 14.1 65759 47.54 65759 92.81 65760 NULL @@ -920,17 +920,17 @@ 65761 NULL 65762 NULL 65762 5.49 -65762 45.70 +65762 45.7 65762 77.96 -65762 87.50 +65762 87.5 65763 NULL 65763 0.72 -65763 43.80 +65763 43.8 65763 86.43 65763 87.99 65764 NULL 65764 31.41 -65764 57.10 +65764 57.1 65765 NULL 65765 88.52 65765 88.56 @@ -938,7 +938,7 @@ 65766 37.06 65766 66.34 65766 86.53 -65766 98.90 +65766 98.9 65767 NULL 65767 90.88 65767 95.57 @@ -950,14 +950,14 @@ 65769 70.52 65769 91.49 65770 NULL -65770 51.90 +65770 51.9 65771 NULL 65771 6.15 -65771 7.50 +65771 7.5 65772 NULL 65773 NULL 65773 3.81 -65773 18.20 +65773 18.2 65773 30.49 65773 47.09 65773 53.09 @@ -966,7 +966,7 @@ 65774 NULL 65774 45.74 65774 45.97 -65774 48.80 +65774 48.8 65774 56.84 65774 94.77 65775 NULL @@ -975,7 +975,7 @@ 65775 66.68 65775 98.43 65776 NULL -65776 18.70 +65776 18.7 65776 28.47 65776 49.73 65776 98.87 @@ -993,7 +993,7 @@ 65778 95.69 65779 NULL 65779 11.87 -65779 28.20 +65779 28.2 65779 39.48 65779 45.61 65779 64.41 @@ -1008,15 +1008,15 @@ 65782 30.24 65782 34.31 65782 76.14 -65782 81.90 +65782 81.9 65783 NULL 65783 46.34 65783 51.08 65783 52.43 65783 62.58 -65783 77.40 +65783 77.4 65784 NULL -65784 15.70 +65784 15.7 65784 31.35 65784 68.18 65784 93.95 @@ -1032,7 +1032,7 @@ 65787 31.19 65787 64.88 65788 NULL -65788 16.10 +65788 16.1 65788 21.81 65788 25.77 65789 NULL @@ -1041,7 +1041,7 @@ 65789 52.49 65789 83.18 65789 92.74 -65789 96.90 +65789 96.9 65790 NULL 65790 46.91 65790 84.87 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 index 207dababa0a50..9091a9156134c 100644 --- a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 @@ -18,12 +18,12 @@ 2013-03-01 09:11:58.703073 10.07 1 2013-03-01 09:11:58.703073 10.07 1 2013-03-01 09:11:58.703073 10.07 1 -2013-03-01 09:11:58.703074 37.80 1 -2013-03-01 09:11:58.703074 37.80 1 -2013-03-01 09:11:58.703074 37.80 1 -2013-03-01 09:11:58.703074 37.80 1 -2013-03-01 09:11:58.703074 37.80 1 -2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.8 1 2013-03-01 09:11:58.703075 5.64 1 2013-03-01 09:11:58.703075 5.64 1 2013-03-01 09:11:58.703075 5.64 1 @@ -59,11 +59,11 @@ 2013-03-01 09:11:58.70308 1.76 1 2013-03-01 09:11:58.70308 1.76 1 2013-03-01 09:11:58.70308 1.76 1 -2013-03-01 09:11:58.703081 67.90 1 -2013-03-01 09:11:58.703081 67.90 1 -2013-03-01 09:11:58.703081 67.90 1 -2013-03-01 09:11:58.703081 67.90 1 -2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.9 1 2013-03-01 09:11:58.703082 37.25 1 2013-03-01 09:11:58.703082 37.25 1 2013-03-01 09:11:58.703082 37.25 1 @@ -148,9 +148,9 @@ 2013-03-01 09:11:58.703096 11.64 1 2013-03-01 09:11:58.703096 11.64 1 2013-03-01 09:11:58.703096 11.64 1 -2013-03-01 09:11:58.703097 0.90 1 -2013-03-01 09:11:58.703097 0.90 1 -2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 2013-03-01 09:11:58.703098 1.35 1 2013-03-01 09:11:58.703098 1.35 1 2013-03-01 09:11:58.703098 1.35 1 @@ -210,27 +210,27 @@ 2013-03-01 09:11:58.70311 8.16 1 2013-03-01 09:11:58.70311 8.16 1 2013-03-01 09:11:58.70311 8.16 1 -2013-03-01 09:11:58.703111 18.80 1 -2013-03-01 09:11:58.703111 18.80 1 -2013-03-01 09:11:58.703111 18.80 1 -2013-03-01 09:11:58.703111 18.80 1 -2013-03-01 09:11:58.703111 18.80 1 -2013-03-01 09:11:58.703111 18.80 1 -2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.8 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 -2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.8 1 2013-03-01 09:11:58.703114 73.94 1 2013-03-01 09:11:58.703114 73.94 1 2013-03-01 09:11:58.703114 73.94 1 @@ -256,14 +256,14 @@ 2013-03-01 09:11:58.703118 8.69 1 2013-03-01 09:11:58.703119 58.02 1 2013-03-01 09:11:58.703119 58.02 1 -2013-03-01 09:11:58.70312 52.60 1 -2013-03-01 09:11:58.70312 52.60 1 -2013-03-01 09:11:58.70312 52.60 1 -2013-03-01 09:11:58.70312 52.60 1 -2013-03-01 09:11:58.703121 96.90 1 -2013-03-01 09:11:58.703121 96.90 1 -2013-03-01 09:11:58.703121 96.90 1 -2013-03-01 09:11:58.703121 96.90 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.703121 96.9 1 2013-03-01 09:11:58.703122 53.56 1 2013-03-01 09:11:58.703122 53.56 1 2013-03-01 09:11:58.703122 53.56 1 @@ -310,11 +310,11 @@ 2013-03-01 09:11:58.703133 27.34 1 2013-03-01 09:11:58.703133 27.34 1 2013-03-01 09:11:58.703133 27.34 1 -2013-03-01 09:11:58.703134 98.90 1 -2013-03-01 09:11:58.703134 98.90 1 -2013-03-01 09:11:58.703134 98.90 1 -2013-03-01 09:11:58.703134 98.90 1 -2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.9 1 2013-03-01 09:11:58.703135 29.14 1 2013-03-01 09:11:58.703135 29.14 1 2013-03-01 09:11:58.703135 29.14 1 @@ -467,12 +467,12 @@ 2013-03-01 09:11:58.703162 3.51 1 2013-03-01 09:11:58.703162 3.51 1 2013-03-01 09:11:58.703162 3.51 1 -2013-03-01 09:11:58.703163 15.70 1 -2013-03-01 09:11:58.703163 15.70 1 -2013-03-01 09:11:58.703163 15.70 1 -2013-03-01 09:11:58.703163 15.70 1 -2013-03-01 09:11:58.703163 15.70 1 -2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.7 1 2013-03-01 09:11:58.703164 30.27 1 2013-03-01 09:11:58.703164 30.27 1 2013-03-01 09:11:58.703164 30.27 1 @@ -482,9 +482,9 @@ 2013-03-01 09:11:58.703165 8.38 1 2013-03-01 09:11:58.703165 8.38 1 2013-03-01 09:11:58.703165 8.38 1 -2013-03-01 09:11:58.703166 16.60 1 -2013-03-01 09:11:58.703166 16.60 1 -2013-03-01 09:11:58.703166 16.60 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.6 1 2013-03-01 09:11:58.703167 17.66 1 2013-03-01 09:11:58.703167 17.66 1 2013-03-01 09:11:58.703167 17.66 1 @@ -537,11 +537,11 @@ 2013-03-01 09:11:58.703175 33.37 1 2013-03-01 09:11:58.703175 33.37 1 2013-03-01 09:11:58.703175 33.37 1 -2013-03-01 09:11:58.703176 28.20 1 -2013-03-01 09:11:58.703176 28.20 1 -2013-03-01 09:11:58.703176 28.20 1 -2013-03-01 09:11:58.703176 28.20 1 -2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.2 1 2013-03-01 09:11:58.703177 11.43 1 2013-03-01 09:11:58.703177 11.43 1 2013-03-01 09:11:58.703177 11.43 1 @@ -567,13 +567,13 @@ 2013-03-01 09:11:58.70318 10.28 1 2013-03-01 09:11:58.70318 10.28 1 2013-03-01 09:11:58.70318 10.28 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 2013-03-01 09:11:58.703182 1.23 1 2013-03-01 09:11:58.703182 1.23 1 2013-03-01 09:11:58.703182 1.23 1 @@ -647,10 +647,10 @@ 2013-03-01 09:11:58.703197 16.01 1 2013-03-01 09:11:58.703197 16.01 1 2013-03-01 09:11:58.703197 16.01 1 -2013-03-01 09:11:58.703198 30.60 1 -2013-03-01 09:11:58.703198 30.60 1 -2013-03-01 09:11:58.703198 30.60 1 -2013-03-01 09:11:58.703198 30.60 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.6 1 2013-03-01 09:11:58.703199 45.69 1 2013-03-01 09:11:58.703199 45.69 1 2013-03-01 09:11:58.703199 45.69 1 @@ -669,11 +669,11 @@ 2013-03-01 09:11:58.703203 11.63 1 2013-03-01 09:11:58.703203 11.63 1 2013-03-01 09:11:58.703203 11.63 1 -2013-03-01 09:11:58.703205 35.80 1 -2013-03-01 09:11:58.703205 35.80 1 -2013-03-01 09:11:58.703205 35.80 1 -2013-03-01 09:11:58.703205 35.80 1 -2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.8 1 2013-03-01 09:11:58.703206 6.61 1 2013-03-01 09:11:58.703206 6.61 1 2013-03-01 09:11:58.703206 6.61 1 @@ -824,9 +824,9 @@ 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 -2013-03-01 09:11:58.703234 44.10 1 -2013-03-01 09:11:58.703234 44.10 1 -2013-03-01 09:11:58.703234 44.10 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.1 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 @@ -834,11 +834,11 @@ 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 -2013-03-01 09:11:58.703236 37.80 1 -2013-03-01 09:11:58.703236 37.80 1 -2013-03-01 09:11:58.703236 37.80 1 -2013-03-01 09:11:58.703236 37.80 1 -2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.8 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 @@ -847,17 +847,17 @@ 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 -2013-03-01 09:11:58.703238 6.00 1 -2013-03-01 09:11:58.703238 6.00 1 -2013-03-01 09:11:58.703238 6.00 1 -2013-03-01 09:11:58.703238 6.00 1 -2013-03-01 09:11:58.703239 24.80 1 -2013-03-01 09:11:58.703239 24.80 1 -2013-03-01 09:11:58.703239 24.80 1 -2013-03-01 09:11:58.703239 24.80 1 -2013-03-01 09:11:58.703239 24.80 1 -2013-03-01 09:11:58.70324 5.10 1 -2013-03-01 09:11:58.70324 5.10 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703238 6 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.703239 24.8 1 +2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.70324 5.1 1 2013-03-01 09:11:58.703241 19.33 1 2013-03-01 09:11:58.703241 19.33 1 2013-03-01 09:11:58.703241 19.33 1 @@ -973,16 +973,16 @@ 2013-03-01 09:11:58.703262 1.81 1 2013-03-01 09:11:58.703262 1.81 1 2013-03-01 09:11:58.703262 1.81 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 @@ -1068,12 +1068,12 @@ 2013-03-01 09:11:58.703281 19.95 1 2013-03-01 09:11:58.703281 19.95 1 2013-03-01 09:11:58.703281 19.95 1 -2013-03-01 09:11:58.703282 7.50 1 -2013-03-01 09:11:58.703282 7.50 1 -2013-03-01 09:11:58.703282 7.50 1 -2013-03-01 09:11:58.703282 7.50 1 -2013-03-01 09:11:58.703282 7.50 1 -2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.5 1 2013-03-01 09:11:58.703283 17.62 1 2013-03-01 09:11:58.703283 17.62 1 2013-03-01 09:11:58.703283 17.62 1 @@ -1153,12 +1153,12 @@ 2013-03-01 09:11:58.703297 25.67 1 2013-03-01 09:11:58.703297 25.67 1 2013-03-01 09:11:58.703297 25.67 1 -2013-03-01 09:11:58.703298 8.80 1 -2013-03-01 09:11:58.703298 8.80 1 -2013-03-01 09:11:58.703298 8.80 1 -2013-03-01 09:11:58.703298 8.80 1 -2013-03-01 09:11:58.703299 9.00 1 -2013-03-01 09:11:58.703299 9.00 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703298 8.8 1 +2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.703299 9 1 2013-03-01 09:11:58.7033 7.51 1 2013-03-01 09:11:58.7033 7.51 1 2013-03-01 09:11:58.7033 7.51 1 @@ -1217,12 +1217,12 @@ 2013-03-01 09:11:58.703311 7.38 1 2013-03-01 09:11:58.703311 7.38 1 2013-03-01 09:11:58.703311 7.38 1 -2013-03-01 09:11:58.703312 18.20 1 -2013-03-01 09:11:58.703312 18.20 1 -2013-03-01 09:11:58.703312 18.20 1 -2013-03-01 09:11:58.703312 18.20 1 -2013-03-01 09:11:58.703312 18.20 1 -2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.2 1 2013-03-01 09:11:58.703313 9.35 1 2013-03-01 09:11:58.703313 9.35 1 2013-03-01 09:11:58.703313 9.35 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 index a1628c7e1c0c5..d02ca48857b5f 100644 --- a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 @@ -46,9 +46,9 @@ 2013-03-01 09:11:58.703092 54.02 1 2013-03-01 09:11:58.703092 54.02 1 2013-03-01 09:11:58.703096 87.84 1 -2013-03-01 09:11:58.703097 0.90 1 -2013-03-01 09:11:58.703097 0.90 1 -2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.9 1 2013-03-01 09:11:58.703098 21.29 1 2013-03-01 09:11:58.703098 21.29 1 2013-03-01 09:11:58.703098 21.29 1 @@ -88,10 +88,10 @@ 2013-03-01 09:11:58.703113 58.65 1 2013-03-01 09:11:58.703118 8.69 1 2013-03-01 09:11:58.703118 8.69 1 -2013-03-01 09:11:58.70312 52.60 1 -2013-03-01 09:11:58.70312 52.60 1 -2013-03-01 09:11:58.70312 52.60 1 -2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.6 1 2013-03-01 09:11:58.703125 78.52 1 2013-03-01 09:11:58.703125 78.52 1 2013-03-01 09:11:58.703125 78.52 1 @@ -119,11 +119,11 @@ 2013-03-01 09:11:58.703136 27.89 1 2013-03-01 09:11:58.703136 27.89 1 2013-03-01 09:11:58.703136 27.89 1 -2013-03-01 09:11:58.703138 86.70 1 -2013-03-01 09:11:58.703138 86.70 1 -2013-03-01 09:11:58.703138 86.70 1 -2013-03-01 09:11:58.703138 86.70 1 -2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.7 1 2013-03-01 09:11:58.703139 43.53 1 2013-03-01 09:11:58.703139 43.53 1 2013-03-01 09:11:58.703139 43.53 1 @@ -167,13 +167,13 @@ 2013-03-01 09:11:58.703179 60.94 1 2013-03-01 09:11:58.703179 60.94 1 2013-03-01 09:11:58.703179 60.94 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 -2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.6 1 2013-03-01 09:11:58.703184 73.93 1 2013-03-01 09:11:58.703184 73.93 1 2013-03-01 09:11:58.703184 73.93 1 @@ -202,12 +202,12 @@ 2013-03-01 09:11:58.703189 37.74 1 2013-03-01 09:11:58.703189 37.74 1 2013-03-01 09:11:58.703189 37.74 1 -2013-03-01 09:11:58.703195 82.50 1 -2013-03-01 09:11:58.703195 82.50 1 -2013-03-01 09:11:58.703195 82.50 1 -2013-03-01 09:11:58.703195 82.50 1 -2013-03-01 09:11:58.703195 82.50 1 -2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.5 1 2013-03-01 09:11:58.703198 97.18 1 2013-03-01 09:11:58.703198 97.18 1 2013-03-01 09:11:58.703198 97.18 1 @@ -233,10 +233,10 @@ 2013-03-01 09:11:58.70321 37.12 1 2013-03-01 09:11:58.70321 37.12 1 2013-03-01 09:11:58.70321 37.12 1 -2013-03-01 09:11:58.703213 48.80 1 -2013-03-01 09:11:58.703213 48.80 1 -2013-03-01 09:11:58.703213 48.80 1 -2013-03-01 09:11:58.703213 48.80 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.8 1 2013-03-01 09:11:58.703219 32.73 1 2013-03-01 09:11:58.703219 32.73 1 2013-03-01 09:11:58.703219 32.73 1 @@ -253,30 +253,30 @@ 2013-03-01 09:11:58.703221 26.64 1 2013-03-01 09:11:58.703221 26.64 1 2013-03-01 09:11:58.703221 26.64 1 -2013-03-01 09:11:58.703223 57.10 1 -2013-03-01 09:11:58.703223 57.10 1 -2013-03-01 09:11:58.703223 57.10 1 -2013-03-01 09:11:58.703223 57.10 1 -2013-03-01 09:11:58.703223 57.10 1 -2013-03-01 09:11:58.703223 57.10 1 -2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.1 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 -2013-03-01 09:11:58.703226 68.30 1 -2013-03-01 09:11:58.703226 68.30 1 -2013-03-01 09:11:58.703226 68.30 1 -2013-03-01 09:11:58.703226 68.30 1 -2013-03-01 09:11:58.703226 68.30 1 -2013-03-01 09:11:58.703226 68.30 1 -2013-03-01 09:11:58.703231 18.70 1 -2013-03-01 09:11:58.703231 18.70 1 -2013-03-01 09:11:58.703231 18.70 1 -2013-03-01 09:11:58.703231 18.70 1 -2013-03-01 09:11:58.703231 18.70 1 -2013-03-01 09:11:58.703231 18.70 1 -2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703226 68.3 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703231 18.7 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 @@ -295,24 +295,24 @@ 2013-03-01 09:11:58.703244 25.67 1 2013-03-01 09:11:58.703244 25.67 1 2013-03-01 09:11:58.703244 25.67 1 -2013-03-01 09:11:58.703245 32.30 1 -2013-03-01 09:11:58.703245 32.30 1 -2013-03-01 09:11:58.703245 32.30 1 -2013-03-01 09:11:58.703245 32.30 1 -2013-03-01 09:11:58.703245 32.30 1 -2013-03-01 09:11:58.703245 32.30 1 -2013-03-01 09:11:58.703245 32.30 1 -2013-03-01 09:11:58.703245 32.30 1 -2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.3 1 2013-03-01 09:11:58.703246 72.87 1 2013-03-01 09:11:58.703246 72.87 1 2013-03-01 09:11:58.703248 81.28 1 2013-03-01 09:11:58.703248 81.28 1 2013-03-01 09:11:58.703248 81.28 1 -2013-03-01 09:11:58.703249 93.30 1 -2013-03-01 09:11:58.703249 93.30 1 -2013-03-01 09:11:58.703249 93.30 1 -2013-03-01 09:11:58.703249 93.30 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.3 1 2013-03-01 09:11:58.70325 93.79 1 2013-03-01 09:11:58.70325 93.79 1 2013-03-01 09:11:58.70325 93.79 1 @@ -324,12 +324,12 @@ 2013-03-01 09:11:58.703254 0.32 1 2013-03-01 09:11:58.703254 0.32 1 2013-03-01 09:11:58.703254 0.32 1 -2013-03-01 09:11:58.703256 43.80 1 -2013-03-01 09:11:58.703256 43.80 1 -2013-03-01 09:11:58.703256 43.80 1 -2013-03-01 09:11:58.703256 43.80 1 -2013-03-01 09:11:58.703256 43.80 1 -2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.8 1 2013-03-01 09:11:58.703258 21.21 1 2013-03-01 09:11:58.703258 21.21 1 2013-03-01 09:11:58.703258 21.21 1 @@ -342,16 +342,16 @@ 2013-03-01 09:11:58.703262 78.56 1 2013-03-01 09:11:58.703262 78.56 1 2013-03-01 09:11:58.703262 78.56 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 -2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.4 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 @@ -438,10 +438,10 @@ 2013-03-01 09:11:58.703299 23.19 1 2013-03-01 09:11:58.703299 23.19 1 2013-03-01 09:11:58.703299 23.19 1 -2013-03-01 09:11:58.703309 89.40 1 -2013-03-01 09:11:58.703309 89.40 1 -2013-03-01 09:11:58.703309 89.40 1 -2013-03-01 09:11:58.703309 89.40 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.4 1 2013-03-01 09:11:58.70331 69.26 1 2013-03-01 09:11:58.70331 69.26 1 2013-03-01 09:11:58.70331 69.26 1 From 9459833eae7fae887af560f3127997e023c51d00 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 27 Nov 2019 15:55:52 -0800 Subject: [PATCH 0744/1387] [SPARK-29989][INFRA] Add `hadoop-2.7/hive-2.3` pre-built distribution ### What changes were proposed in this pull request? This PR aims to add another pre-built binary distribution with `-Phadoop-2.7 -Phive-1.2` at `Apache Spark 3.0.0`. **PRE-BUILT BINARY DISTRIBUTION** ``` spark-3.0.0-SNAPSHOT-bin-hadoop2.7-hive1.2.tgz spark-3.0.0-SNAPSHOT-bin-hadoop2.7-hive1.2.tgz.asc spark-3.0.0-SNAPSHOT-bin-hadoop2.7-hive1.2.tgz.sha512 ``` **CONTENTS (snippet)** ``` $ ls *hadoop-* hadoop-annotations-2.7.4.jar hadoop-mapreduce-client-shuffle-2.7.4.jar hadoop-auth-2.7.4.jar hadoop-yarn-api-2.7.4.jar hadoop-client-2.7.4.jar hadoop-yarn-client-2.7.4.jar hadoop-common-2.7.4.jar hadoop-yarn-common-2.7.4.jar hadoop-hdfs-2.7.4.jar hadoop-yarn-server-common-2.7.4.jar hadoop-mapreduce-client-app-2.7.4.jar hadoop-yarn-server-web-proxy-2.7.4.jar hadoop-mapreduce-client-common-2.7.4.jar parquet-hadoop-1.10.1.jar hadoop-mapreduce-client-core-2.7.4.jar parquet-hadoop-bundle-1.6.0.jar hadoop-mapreduce-client-jobclient-2.7.4.jar $ ls *hive-* hive-beeline-1.2.1.spark2.jar hive-jdbc-1.2.1.spark2.jar hive-cli-1.2.1.spark2.jar hive-metastore-1.2.1.spark2.jar hive-exec-1.2.1.spark2.jar spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar ``` ### Why are the changes needed? Since Apache Spark switched to use `-Phive-2.3` by default, all pre-built binary distribution will use `-Phive-2.3`. This PR adds `hadoop-2.7/hive-1.2` distribution to provide a similar combination like `Apache Spark 2.4` line. ### Does this PR introduce any user-facing change? Yes. This is additional distribution which resembles to `Apache Spark 2.4` line in terms of `hive` version. ### How was this patch tested? Manual. Please note that we need a dry-run mode, but the AS-IS release script do not generate additional combinations including this in `dry-run` mode. Closes #26688 from dongjoon-hyun/SPARK-29989. Authored-by: Dongjoon Hyun Signed-off-by: Xiao Li --- dev/create-release/release-build.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 99c4b20102929..d93382b516cef 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -282,6 +282,7 @@ if [[ "$1" == "package" ]]; then if [[ $SPARK_VERSION < "3.0." ]]; then BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" else + BINARY_PKGS_ARGS["hadoop2.7-hive1.2"]="-Phadoop-2.7 -Phive-1.2 $HIVE_PROFILES" BINARY_PKGS_ARGS["hadoop3.2"]="-Phadoop-3.2 $HIVE_PROFILES" fi fi From dde0d2fcadbdf59a6ec696da12bd72bdfc968bc5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 27 Nov 2019 21:14:40 -0800 Subject: [PATCH 0745/1387] [SPARK-29991][INFRA] Support `test-hive1.2` in PR Builder ### What changes were proposed in this pull request? Currently, Apache Spark PR Builder using `hive-1.2` for `hadoop-2.7` and `hive-2.3` for `hadoop-3.2`. This PR aims to support `[test-hive1.2]` in PR Builder in order to cut the correlation between `hive-1.2/2.3` to `hadoop-2.7/3.2`. After this PR, the PR Builder will use `hive-2.3` by default for all profiles (if there is no `test-hive1.2`.) ### Why are the changes needed? This new tag allows us more flexibility. ### Does this PR introduce any user-facing change? No. (This is a dev-only change.) ### How was this patch tested? Check the Jenkins triggers in this PR. **BEFORE** ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark using SBT with these arguments: -Phadoop-2.7 -Phive-1.2 -Pyarn -Pkubernetes -Phive -Phadoop-cloud -Pspark-ganglia-lgpl -Phive-thriftserver -Pkinesis-asl -Pmesos test:package streaming-kinesis-asl-assembly/assembly ``` **AFTER** 1. Title: [[SPARK-29991][INFRA][test-hive1.2] Support `test-hive1.2` in PR Builder](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/114550/testReport) ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark using SBT with these arguments: -Phadoop-2.7 -Phive-1.2 -Pkinesis-asl -Phadoop-cloud -Pyarn -Phive -Pmesos -Pspark-ganglia-lgpl -Pkubernetes -Phive-thriftserver test:package streaming-kinesis-asl-assembly/assembly ``` 2. Title: [[SPARK-29991][INFRA] Support `test hive1.2` in PR Builder](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/114551/testReport) - Note that I removed the hyphen intentionally from `test-hive1.2`. ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark using SBT with these arguments: -Phadoop-2.7 -Phive-thriftserver -Pkubernetes -Pspark-ganglia-lgpl -Phadoop-cloud -Phive -Pmesos -Pyarn -Pkinesis-asl test:package streaming-kinesis-asl-assembly/assembly ``` Closes #26695 from dongjoon-hyun/SPARK-29991. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/run-tests.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 2d52ead06a041..13a2960049a54 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -283,12 +283,16 @@ def get_hadoop_profiles(hadoop_version): """ sbt_maven_hadoop_profiles = { - "hadoop2.7": ["-Phadoop-2.7", "-Phive-1.2"], - "hadoop3.2": ["-Phadoop-3.2", "-Phive-2.3"], + "hadoop2.7": ["-Phadoop-2.7"], + "hadoop3.2": ["-Phadoop-3.2"], } if hadoop_version in sbt_maven_hadoop_profiles: - return sbt_maven_hadoop_profiles[hadoop_version] + if ("ghprbPullTitle" in os.environ and + "test-hive1.2" in os.environ["ghprbPullTitle"].lower()): + return sbt_maven_hadoop_profiles[hadoop_version] + ["-Phive-1.2"] + else: + return sbt_maven_hadoop_profiles[hadoop_version] else: print("[error] Could not find", hadoop_version, "in the list. Valid options", " are", sbt_maven_hadoop_profiles.keys()) From 43556e46e9f741ec51fd3cb05f54fe06fedc6408 Mon Sep 17 00:00:00 2001 From: joanfontanals Date: Thu, 28 Nov 2019 08:36:54 -0600 Subject: [PATCH 0746/1387] [SPARK-29877][GRAPHX] static PageRank allow checkPoint from previous computations ### What changes were proposed in this pull request? Add an optional parameter to the staticPageRank computation with the result of a previous PageRank computation. This would make the algorithm start from a different starting point closer to the convergence configuration ### Why are the changes needed? https://issues.apache.org/jira/browse/SPARK-29877 It would be really helpful to have the possibility, when computing staticPageRank to use a previous computation as a checkpoint to continue the iterations. ### Does this PR introduce any user-facing change? Yes, it allows to start the static page Rank computation from the point where an earlier one finished. Example: Compute 10 iteration first, and continue for 3 more iterations ```scala val partialPageRank = graph.ops.staticPageRank(numIter=10, resetProb=0.15) val continuationPageRank = graph.ops.staticPageRank(numIter=3, resetProb=0.15, Some(partialPageRank)) ``` ### How was this patch tested? Yes, some tests were added. Testing was done as follow: - Check how many iterations it takes for a static Page Rank computation to converge - Run the static Page Rank computation for half of these iterations and take result as checkpoint - Restart computation and check that number of iterations it takes to converge. It never has to be larger than the original one and in most of the cases it is much smaller. Due to the presence of sinks and the normalization done in [[SPARK-18847]] it is not exactly equivalent to compute static page rank for 2 iterations, take the result at checkpoint and run for 2 more iterations than to compute directly for 4 iterations. However this checkpointing can give the algorithm a hint about the true distribution of pageRanks in the graph Closes #26608 from JoanFM/pageRank_checkPoint. Authored-by: joanfontanals Signed-off-by: Sean Owen --- .../org/apache/spark/graphx/GraphOps.scala | 12 ++ .../apache/spark/graphx/lib/PageRank.scala | 120 ++++++++++++++---- .../spark/graphx/lib/PageRankSuite.scala | 102 +++++++++++++++ 3 files changed, 212 insertions(+), 22 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 49e8487dbe935..a07c2e16b1132 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -424,6 +424,18 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali PageRank.run(graph, numIter, resetProb) } + /** + * Run PageRank for a fixed number of iterations returning a graph with vertex attributes + * containing the PageRank and edge attributes the normalized edge weight, optionally including + * including a previous pageRank computation to be used as a start point for the new iterations + * + * @see [[org.apache.spark.graphx.lib.PageRank$#runWithOptionsWithPreviousPageRank]] + */ + def staticPageRank(numIter: Int, resetProb: Double, + prePageRank: Graph[Double, Double]): Graph[Double, Double] = { + PageRank.runWithOptionsWithPreviousPageRank(graph, numIter, resetProb, None, prePageRank) + } + /** * Compute the connected component membership of each vertex and return a graph with the vertex * value containing the lowest vertex id in the connected component containing that vertex. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 105ad57d4ed86..102dc2d2dd4b0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -79,10 +79,46 @@ object PageRank extends Logging { * @return the graph containing with each vertex containing the PageRank and each edge * containing the normalized weight. */ - def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int, - resetProb: Double = 0.15): Graph[Double, Double] = + def run[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { - runWithOptions(graph, numIter, resetProb) + runWithOptions(graph, numIter, resetProb, None) + } + + /** + * Run an update pass of PageRank algorithm. Update the values of every node in the + * pageRank + * + * @param rankGraph the current PageRank + * @param personalized True if personalized pageRank + * @param resetProb the random reset probability (alpha) + * @param src the source vertex for a Personalized Page Rank + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight after a single update step. + * + */ + private def runUpdate(rankGraph: Graph[Double, Double], personalized: Boolean, + resetProb: Double, src: VertexId): Graph[Double, Double] = { + + def delta(u: VertexId, v: VertexId): Double = { if (u == v) 1.0 else 0.0 } + // Compute the outgoing rank contributions of each vertex, perform local preaggregation, and + // do the final aggregation at the receiving vertices. Requires a shuffle for aggregation. + val rankUpdates = rankGraph.aggregateMessages[Double]( + ctx => ctx.sendToDst(ctx.srcAttr * ctx.attr), _ + _, TripletFields.Src) + + // Apply the final rank updates to get the new ranks, using join to preserve ranks of vertices + // that didn't receive a message. Requires a shuffle for broadcasting updated ranks to the + // edge partitions. + val rPrb = if (personalized) { + (src: VertexId, id: VertexId) => resetProb * delta(src, id) + } else { + (src: VertexId, id: VertexId) => resetProb + } + + rankGraph.outerJoinVertices(rankUpdates) { + (id, oldRank, msgSumOpt) => rPrb(src, id) + (1.0 - resetProb) * msgSumOpt.getOrElse(0.0) + } } /** @@ -128,37 +164,77 @@ object PageRank extends Logging { if (!(id != src && personalized)) 1.0 else 0.0 } - def delta(u: VertexId, v: VertexId): Double = { if (u == v) 1.0 else 0.0 } - var iteration = 0 var prevRankGraph: Graph[Double, Double] = null while (iteration < numIter) { rankGraph.cache() + prevRankGraph = rankGraph - // Compute the outgoing rank contributions of each vertex, perform local preaggregation, and - // do the final aggregation at the receiving vertices. Requires a shuffle for aggregation. - val rankUpdates = rankGraph.aggregateMessages[Double]( - ctx => ctx.sendToDst(ctx.srcAttr * ctx.attr), _ + _, TripletFields.Src) + rankGraph = runUpdate(rankGraph, personalized, resetProb, src) + rankGraph.cache() + rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices + logInfo(s"PageRank finished iteration $iteration.") + prevRankGraph.vertices.unpersist() + prevRankGraph.edges.unpersist() + iteration += 1 + } - // Apply the final rank updates to get the new ranks, using join to preserve ranks of vertices - // that didn't receive a message. Requires a shuffle for broadcasting updated ranks to the - // edge partitions. - prevRankGraph = rankGraph - val rPrb = if (personalized) { - (src: VertexId, id: VertexId) => resetProb * delta(src, id) - } else { - (src: VertexId, id: VertexId) => resetProb - } + // SPARK-18847 If the graph has sinks (vertices with no outgoing edges) correct the sum of ranks + normalizeRankSum(rankGraph, personalized) + } - rankGraph = rankGraph.outerJoinVertices(rankUpdates) { - (id, oldRank, msgSumOpt) => rPrb(src, id) + (1.0 - resetProb) * msgSumOpt.getOrElse(0.0) - }.cache() + /** + * Run PageRank for a fixed number of iterations returning a graph + * with vertex attributes containing the PageRank and edge + * attributes the normalized edge weight. + * + * @tparam VD the original vertex attribute (not used) + * @tparam ED the original edge attribute (not used) + * + * @param graph the graph on which to compute PageRank + * @param numIter the number of iterations of PageRank to run + * @param resetProb the random reset probability (alpha) + * @param srcId the source vertex for a Personalized Page Rank (optional) + * @param preRankGraph PageRank graph from which to keep iterating + * + * @return the graph containing with each vertex containing the PageRank and each edge + * containing the normalized weight. + * + */ + def runWithOptionsWithPreviousPageRank[VD: ClassTag, ED: ClassTag]( + graph: Graph[VD, ED], numIter: Int, resetProb: Double, srcId: Option[VertexId], + preRankGraph: Graph[Double, Double]): Graph[Double, Double] = { + require(numIter > 0, s"Number of iterations must be greater than 0," + + s" but got ${numIter}") + require(resetProb >= 0 && resetProb <= 1, s"Random reset probability must belong" + + s" to [0, 1], but got ${resetProb}") + val graphVertices = graph.numVertices + val prePageRankVertices = preRankGraph.numVertices + require(graphVertices == prePageRankVertices, s"Graph and previous pageRankGraph" + + s" must have the same number of vertices but got ${graphVertices} and ${prePageRankVertices}") + val personalized = srcId.isDefined + val src: VertexId = srcId.getOrElse(-1L) + + // Initialize the PageRank graph with each edge attribute having + // weight 1/outDegree and each vertex with attribute 1.0. + // When running personalized pagerank, only the source vertex + // has an attribute 1.0. All others are set to 0. + var rankGraph: Graph[Double, Double] = preRankGraph + + var iteration = 0 + var prevRankGraph: Graph[Double, Double] = null + + while (iteration < numIter) { + rankGraph.cache() + prevRankGraph = rankGraph + + rankGraph = runUpdate(rankGraph, personalized, resetProb, src) + rankGraph.cache() rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices logInfo(s"PageRank finished iteration $iteration.") prevRankGraph.vertices.unpersist() prevRankGraph.edges.unpersist() - iteration += 1 } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala index d8f1c497718bb..a5e2fc5c9a74f 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala @@ -64,6 +64,38 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { .map { case (id, error) => error }.sum() } + def convergenceIterations[VD, ED](graph: Graph[VD, ED], resetProb: Double, + tol: Double, errorTol: Double): (Int, Int) = { + val dynamicRanks = graph.ops.pageRank(tol, resetProb).vertices.cache() + + // Compute how many iterations it takes to converge + var iter = 1 + var staticGraphRank = graph.ops.staticPageRank(iter, resetProb).vertices.cache() + while (!(compareRanks(staticGraphRank, dynamicRanks) < errorTol)) { + iter += 1 + staticGraphRank = graph.ops.staticPageRank(iter, resetProb).vertices.cache() + } + val convergenceIter = iter + val checkPointIter = convergenceIter / 2 + + // CheckPoint the graph computed at half of these iterations + val staticGraphRankPartial = graph.ops.staticPageRank(checkPointIter, resetProb) + + // Compute how many iterations it takes to converge when a checkPoint is provided + var iterWithCheckPoint = 1 + var staticGraphRankWithCheckPoint = graph.ops.staticPageRank(iterWithCheckPoint, + resetProb, staticGraphRankPartial).vertices.cache() + while (compareRanks(staticGraphRankWithCheckPoint, dynamicRanks) >= errorTol) { + iterWithCheckPoint += 1 + staticGraphRankWithCheckPoint = graph.ops.staticPageRank(iterWithCheckPoint, + resetProb, staticGraphRankPartial).vertices.cache() + } + + val convergenceIterWithCheckPoint = iterWithCheckPoint + + (convergenceIterWithCheckPoint, convergenceIter) + } + test("Star PageRank") { withSpark { sc => val nVertices = 100 @@ -184,6 +216,25 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { } } // end of Grid PageRank + test("Grid PageRank with checkpoint") { + withSpark { sc => + // Check that checkPointing helps the static PageRank to converge in less iterations + val rows = 10 + val cols = 10 + val resetProb = 0.15 + val errorTol = 1.0e-5 + val tol = 0.0001 + val gridGraph = GraphGenerators.gridGraph(sc, rows, cols).cache() + + val (iterAfterHalfCheckPoint, totalIters) = + convergenceIterations(gridGraph, resetProb, tol, errorTol) + + // In this case checkPoint does not help much + assert(totalIters == 19) + assert(iterAfterHalfCheckPoint == 18) + } + } // end of Grid PageRank + test("Chain PageRank") { withSpark { sc => val chain1 = (0 until 9).map(x => (x, x + 1)) @@ -201,6 +252,24 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { } } + test("Chain PageRank with checkpoint") { + withSpark { sc => + val chain1 = (0 until 9).map(x => (x, x + 1)) + val rawEdges = sc.parallelize(chain1, 1).map { case (s, d) => (s.toLong, d.toLong) } + val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val resetProb = 0.15 + val errorTol = 1.0e-5 + val tol = 0.0001 + + val (iterAfterHalfCheckPoint, totalIters) = + convergenceIterations(chain, resetProb, tol, errorTol) + + // In this case checkPoint does not help but it does not take more iterations + assert(totalIters == 10) + assert(iterAfterHalfCheckPoint == 10) + } + } // end of Grid PageRank + test("Chain PersonalizedPageRank") { withSpark { sc => // Check that implementation can handle large vertexIds, SPARK-25149 @@ -264,7 +333,23 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { val ranks = VertexRDD(sc.parallelize(1L to 4L zip igraphPR)) assert(compareRanks(staticRanks, ranks) < errorTol) assert(compareRanks(dynamicRanks, ranks) < errorTol) + } + } + + test("Loop with source PageRank with checkpoint") { + withSpark { sc => + val edges = sc.parallelize((1L, 2L) :: (2L, 3L) :: (3L, 4L) :: (4L, 2L) :: Nil) + val g = Graph.fromEdgeTuples(edges, 1) + val resetProb = 0.15 + val tol = 0.0001 + val errorTol = 1.0e-5 + val (iterAfterHalfCheckPoint, totalIters) = + convergenceIterations(g, resetProb, tol, errorTol) + + // In this case checkPoint helps a lot + assert(totalIters == 34) + assert(iterAfterHalfCheckPoint == 17) } } @@ -313,4 +398,21 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { } } + + test("Loop with sink PageRank with checkpoint") { + withSpark { sc => + val edges = sc.parallelize((1L, 2L) :: (2L, 3L) :: (3L, 1L) :: (1L, 4L) :: Nil) + val g = Graph.fromEdgeTuples(edges, 1) + val resetProb = 0.15 + val tol = 0.0001 + val errorTol = 1.0e-5 + + val (iterAfterHalfCheckPoint, totalIters) = + convergenceIterations(g, resetProb, tol, errorTol) + + // In this case checkPoint helps a lot + assert(totalIters == 15) + assert(iterAfterHalfCheckPoint == 9) + } + } } From 9351e3e76fb11e9fdaf39aef5aea86fdeccd6f28 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 29 Nov 2019 13:23:22 +0900 Subject: [PATCH 0747/1387] Revert "[SPARK-29991][INFRA] Support `test-hive1.2` in PR Builder" This reverts commit dde0d2fcadbdf59a6ec696da12bd72bdfc968bc5. --- dev/run-tests.py | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 13a2960049a54..2d52ead06a041 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -283,16 +283,12 @@ def get_hadoop_profiles(hadoop_version): """ sbt_maven_hadoop_profiles = { - "hadoop2.7": ["-Phadoop-2.7"], - "hadoop3.2": ["-Phadoop-3.2"], + "hadoop2.7": ["-Phadoop-2.7", "-Phive-1.2"], + "hadoop3.2": ["-Phadoop-3.2", "-Phive-2.3"], } if hadoop_version in sbt_maven_hadoop_profiles: - if ("ghprbPullTitle" in os.environ and - "test-hive1.2" in os.environ["ghprbPullTitle"].lower()): - return sbt_maven_hadoop_profiles[hadoop_version] + ["-Phive-1.2"] - else: - return sbt_maven_hadoop_profiles[hadoop_version] + return sbt_maven_hadoop_profiles[hadoop_version] else: print("[error] Could not find", hadoop_version, "in the list. Valid options", " are", sbt_maven_hadoop_profiles.keys()) From b182ed83f6c8d06c54ce169754c31c170088d08c Mon Sep 17 00:00:00 2001 From: shahid Date: Fri, 29 Nov 2019 19:44:31 -0800 Subject: [PATCH 0748/1387] [SPARK-29724][SPARK-29726][WEBUI][SQL] Support JDBC/ODBC tab for HistoryServer WebUI ### What changes were proposed in this pull request? Support JDBC/ODBC tab for HistoryServer WebUI. Currently from Historyserver we can't access the JDBC/ODBC tab for thrift server applications. In this PR, I am doing 2 main changes 1. Refactor existing thrift server listener to support kvstore 2. Add history server plugin for thrift server listener and tab. ### Why are the changes needed? Users can access Thriftserver tab from History server for both running and finished applications, ### Does this PR introduce any user-facing change? Support for JDBC/ODBC tab for the WEBUI from History server ### How was this patch tested? Add UT and Manual tests 1. Start Thriftserver and Historyserver ``` sbin/stop-thriftserver.sh sbin/stop-historyserver.sh sbin/start-thriftserver.sh sbin/start-historyserver.sh ``` 2. Launch beeline `bin/beeline -u jdbc:hive2://localhost:10000` 3. Run queries Go to the JDBC/ODBC page of the WebUI from History server ![image](https://user-images.githubusercontent.com/23054875/68365501-cf013700-0156-11ea-84b4-fda8008c92c4.png) Closes #26378 from shahidki31/ThriftKVStore. Authored-by: shahid Signed-off-by: Gengliang Wang --- .../deploy/history/FsHistoryProvider.scala | 5 +- .../spark/status/AppHistoryServerPlugin.scala | 5 + .../execution/ui/SQLHistoryServerPlugin.scala | 3 + ...apache.spark.status.AppHistoryServerPlugin | 1 + .../hive/thriftserver/HiveThriftServer2.scala | 203 +---------- .../SparkExecuteStatementOperation.scala | 19 +- .../SparkGetCatalogsOperation.scala | 10 +- .../SparkGetColumnsOperation.scala | 10 +- .../SparkGetFunctionsOperation.scala | 10 +- .../SparkGetSchemasOperation.scala | 10 +- .../SparkGetTableTypesOperation.scala | 10 +- .../SparkGetTablesOperation.scala | 10 +- .../SparkGetTypeInfoOperation.scala | 10 +- .../thriftserver/SparkSQLSessionManager.scala | 4 +- .../ui/HiveThriftServer2AppStatusStore.scala | 132 ++++++++ .../ui/HiveThriftServer2EventManager.scala | 113 +++++++ ...HiveThriftServer2HistoryServerPlugin.scala | 40 +++ .../ui/HiveThriftServer2Listener.scala | 315 ++++++++++++++++++ .../thriftserver/ui/ThriftServerPage.scala | 30 +- .../ui/ThriftServerSessionPage.scala | 13 +- .../thriftserver/ui/ThriftServerTab.scala | 15 +- .../ui/HiveThriftServer2ListenerSuite.scala | 164 +++++++++ .../ui/ThriftServerPageSuite.scala | 68 ++-- 23 files changed, 913 insertions(+), 287 deletions(-) create mode 100644 sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index e2f3314bc8595..a3776b3ad756d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -352,10 +352,11 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val ui = SparkUI.create(None, new HistoryAppStatusStore(conf, kvstore), conf, secManager, app.info.name, HistoryServer.getAttemptURI(appId, attempt.info.attemptId), attempt.info.startTime.getTime(), attempt.info.appSparkVersion) - loadPlugins().foreach(_.setupUI(ui)) - val loadedUI = LoadedAppUI(ui) + // place the tab in UI based on the display order + loadPlugins().toSeq.sortBy(_.displayOrder).foreach(_.setupUI(ui)) + val loadedUI = LoadedAppUI(ui) synchronized { activeUIs((appId, attemptId)) = loadedUI } diff --git a/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala index d144a0e998fa1..2e9a31d5ac69c 100644 --- a/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala +++ b/core/src/main/scala/org/apache/spark/status/AppHistoryServerPlugin.scala @@ -35,4 +35,9 @@ private[spark] trait AppHistoryServerPlugin { * Sets up UI of this plugin to rebuild the history UI. */ def setupUI(ui: SparkUI): Unit + + /** + * The position of a plugin tab relative to the other plugin tabs in the history UI. + */ + def displayOrder: Int = Integer.MAX_VALUE } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala index 522d0cf79bffa..5bf1ce5eb8a90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLHistoryServerPlugin.scala @@ -33,4 +33,7 @@ class SQLHistoryServerPlugin extends AppHistoryServerPlugin { new SQLTab(sqlStatusStore, ui) } } + + override def displayOrder: Int = 0 } + diff --git a/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin b/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin new file mode 100644 index 0000000000000..96d990372ee4c --- /dev/null +++ b/sql/hive-thriftserver/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin @@ -0,0 +1 @@ +org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2HistoryServerPlugin diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 9517a599be633..f15193b0dc3cc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -20,9 +20,6 @@ package org.apache.spark.sql.hive.thriftserver import java.util.Locale import java.util.concurrent.atomic.AtomicBoolean -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} @@ -32,12 +29,11 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.internal.config.UI.UI_ENABLED -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.hive.thriftserver.ui._ +import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.{ShutdownHookManager, Utils} /** @@ -47,6 +43,7 @@ import org.apache.spark.util.{ShutdownHookManager, Utils} object HiveThriftServer2 extends Logging { var uiTab: Option[ThriftServerTab] = None var listener: HiveThriftServer2Listener = _ + var eventManager: HiveThriftServer2EventManager = _ /** * :: DeveloperApi :: @@ -62,14 +59,21 @@ object HiveThriftServer2 extends Logging { server.init(executionHive.conf) server.start() - listener = new HiveThriftServer2Listener(server, sqlContext.conf) - sqlContext.sparkContext.addSparkListener(listener) - uiTab = if (sqlContext.sparkContext.getConf.get(UI_ENABLED)) { - Some(new ThriftServerTab(sqlContext.sparkContext)) + createListenerAndUI(server, sqlContext.sparkContext) + server + } + + private def createListenerAndUI(server: HiveThriftServer2, sc: SparkContext): Unit = { + val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] + eventManager = new HiveThriftServer2EventManager(sc) + listener = new HiveThriftServer2Listener(kvStore, sc.conf, Some(server)) + sc.listenerBus.addToStatusQueue(listener) + uiTab = if (sc.getConf.get(UI_ENABLED)) { + Some(new ThriftServerTab(new HiveThriftServer2AppStatusStore(kvStore, Some(listener)), + ThriftServerTab.getSparkUI(sc))) } else { None } - server } def main(args: Array[String]): Unit = { @@ -101,13 +105,7 @@ object HiveThriftServer2 extends Logging { server.init(executionHive.conf) server.start() logInfo("HiveThriftServer2 started") - listener = new HiveThriftServer2Listener(server, SparkSQLEnv.sqlContext.conf) - SparkSQLEnv.sparkContext.addSparkListener(listener) - uiTab = if (SparkSQLEnv.sparkContext.getConf.get(UI_ENABLED)) { - Some(new ThriftServerTab(SparkSQLEnv.sparkContext)) - } else { - None - } + createListenerAndUI(server, SparkSQLEnv.sparkContext) // If application was killed before HiveThriftServer2 start successfully then SparkSubmit // process can not exit, so check whether if SparkContext was stopped. if (SparkSQLEnv.sparkContext.stopped.get()) { @@ -121,179 +119,10 @@ object HiveThriftServer2 extends Logging { } } - private[thriftserver] class SessionInfo( - val sessionId: String, - val startTimestamp: Long, - val ip: String, - val userName: String) { - var finishTimestamp: Long = 0L - var totalExecution: Int = 0 - def totalTime: Long = { - if (finishTimestamp == 0L) { - System.currentTimeMillis - startTimestamp - } else { - finishTimestamp - startTimestamp - } - } - } - private[thriftserver] object ExecutionState extends Enumeration { val STARTED, COMPILED, CANCELED, FAILED, FINISHED, CLOSED = Value type ExecutionState = Value } - - private[thriftserver] class ExecutionInfo( - val statement: String, - val sessionId: String, - val startTimestamp: Long, - val userName: String) { - var finishTimestamp: Long = 0L - var closeTimestamp: Long = 0L - var executePlan: String = "" - var detail: String = "" - var state: ExecutionState.Value = ExecutionState.STARTED - val jobId: ArrayBuffer[String] = ArrayBuffer[String]() - var groupId: String = "" - def totalTime(endTime: Long): Long = { - if (endTime == 0L) { - System.currentTimeMillis - startTimestamp - } else { - endTime - startTimestamp - } - } - } - - - /** - * An inner sparkListener called in sc.stop to clean up the HiveThriftServer2 - */ - private[thriftserver] class HiveThriftServer2Listener( - val server: HiveServer2, - val conf: SQLConf) extends SparkListener { - - override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { - server.stop() - } - private val sessionList = new mutable.LinkedHashMap[String, SessionInfo] - private val executionList = new mutable.LinkedHashMap[String, ExecutionInfo] - private val retainedStatements = conf.getConf(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT) - private val retainedSessions = conf.getConf(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT) - - def getOnlineSessionNum: Int = synchronized { - sessionList.count(_._2.finishTimestamp == 0) - } - - def isExecutionActive(execInfo: ExecutionInfo): Boolean = { - !(execInfo.state == ExecutionState.FAILED || - execInfo.state == ExecutionState.CANCELED || - execInfo.state == ExecutionState.CLOSED) - } - - /** - * When an error or a cancellation occurs, we set the finishTimestamp of the statement. - * Therefore, when we count the number of running statements, we need to exclude errors and - * cancellations and count all statements that have not been closed so far. - */ - def getTotalRunning: Int = synchronized { - executionList.count { - case (_, v) => isExecutionActive(v) - } - } - - def getSessionList: Seq[SessionInfo] = synchronized { sessionList.values.toSeq } - - def getSession(sessionId: String): Option[SessionInfo] = synchronized { - sessionList.get(sessionId) - } - - def getExecutionList: Seq[ExecutionInfo] = synchronized { executionList.values.toSeq } - - override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { - for { - props <- Option(jobStart.properties) - groupId <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) - (_, info) <- executionList if info.groupId == groupId - } { - info.jobId += jobStart.jobId.toString - info.groupId = groupId - } - } - - def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { - synchronized { - val info = new SessionInfo(sessionId, System.currentTimeMillis, ip, userName) - sessionList.put(sessionId, info) - trimSessionIfNecessary() - } - } - - def onSessionClosed(sessionId: String): Unit = synchronized { - sessionList(sessionId).finishTimestamp = System.currentTimeMillis - trimSessionIfNecessary() - } - - def onStatementStart( - id: String, - sessionId: String, - statement: String, - groupId: String, - userName: String = "UNKNOWN"): Unit = synchronized { - val info = new ExecutionInfo(statement, sessionId, System.currentTimeMillis, userName) - info.state = ExecutionState.STARTED - executionList.put(id, info) - trimExecutionIfNecessary() - sessionList(sessionId).totalExecution += 1 - executionList(id).groupId = groupId - } - - def onStatementParsed(id: String, executionPlan: String): Unit = synchronized { - executionList(id).executePlan = executionPlan - executionList(id).state = ExecutionState.COMPILED - } - - def onStatementCanceled(id: String): Unit = synchronized { - executionList(id).finishTimestamp = System.currentTimeMillis - executionList(id).state = ExecutionState.CANCELED - trimExecutionIfNecessary() - } - - def onStatementError(id: String, errorMsg: String, errorTrace: String): Unit = synchronized { - executionList(id).finishTimestamp = System.currentTimeMillis - executionList(id).detail = errorMsg - executionList(id).state = ExecutionState.FAILED - trimExecutionIfNecessary() - } - - def onStatementFinish(id: String): Unit = synchronized { - executionList(id).finishTimestamp = System.currentTimeMillis - executionList(id).state = ExecutionState.FINISHED - trimExecutionIfNecessary() - } - - def onOperationClosed(id: String): Unit = synchronized { - executionList(id).closeTimestamp = System.currentTimeMillis - executionList(id).state = ExecutionState.CLOSED - } - - private def trimExecutionIfNecessary() = { - if (executionList.size > retainedStatements) { - val toRemove = math.max(retainedStatements / 10, 1) - executionList.filter(_._2.finishTimestamp != 0).take(toRemove).foreach { s => - executionList.remove(s._1) - } - } - } - - private def trimSessionIfNecessary() = { - if (sessionList.size > retainedSessions) { - val toRemove = math.max(retainedSessions / 10, 1) - sessionList.filter(_._2.finishTimestamp != 0).take(toRemove).foreach { s => - sessionList.remove(s._1) - } - } - - } - } } private[hive] class HiveThriftServer2(sqlContext: SQLContext) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 68197a9de8566..76d07848f79a9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -77,7 +77,7 @@ private[hive] class SparkExecuteStatementOperation( // RDDs will be cleaned automatically upon garbage collection. logInfo(s"Close statement with $statementId") cleanup(OperationState.CLOSED) - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int): Unit = { @@ -195,7 +195,7 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.PENDING) statementId = UUID.randomUUID().toString logInfo(s"Submitting query '$statement' with $statementId") - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, statement, @@ -245,14 +245,14 @@ private[hive] class SparkExecuteStatementOperation( case rejected: RejectedExecutionException => logError("Error submitting query in background, query rejected", rejected) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, rejected.getMessage, SparkUtils.exceptionString(rejected)) throw new HiveSQLException("The background threadpool cannot accept" + " new task for execution, please retry the operation", rejected) case NonFatal(e) => logError(s"Error executing query in background", e) setState(OperationState.ERROR) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, e.getMessage, SparkUtils.exceptionString(e)) throw new HiveSQLException(e) } @@ -284,7 +284,8 @@ private[hive] class SparkExecuteStatementOperation( "in this session.") case _ => } - HiveThriftServer2.listener.onStatementParsed(statementId, result.queryExecution.toString()) + HiveThriftServer2.eventManager.onStatementParsed(statementId, + result.queryExecution.toString()) iter = { if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { resultList = None @@ -315,12 +316,12 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error running query: " + root.toString, root) } @@ -329,7 +330,7 @@ private[hive] class SparkExecuteStatementOperation( synchronized { if (!getStatus.getState.isTerminal) { setState(OperationState.FINISHED) - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } sqlContext.sparkContext.clearJobGroup() @@ -341,7 +342,7 @@ private[hive] class SparkExecuteStatementOperation( if (!getStatus.getState.isTerminal) { logInfo(s"Cancel query with $statementId") cleanup(OperationState.CANCELED) - HiveThriftServer2.listener.onStatementCanceled(statementId) + HiveThriftServer2.eventManager.onStatementCanceled(statementId) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index 6c8a5b00992da..2945cfd200e46 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -44,7 +44,7 @@ private[hive] class SparkGetCatalogsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -56,7 +56,7 @@ private[hive] class SparkGetCatalogsOperation( val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -74,16 +74,16 @@ private[hive] class SparkGetCatalogsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting catalogs: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index f845a2285b9a3..ff7cbfeae13be 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -63,7 +63,7 @@ private[hive] class SparkGetColumnsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -78,7 +78,7 @@ private[hive] class SparkGetColumnsOperation( val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -135,17 +135,17 @@ private[hive] class SparkGetColumnsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting columns: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } private def addToRowSet( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index 1cdd8918421bb..d9c12b6ca9e64 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -54,7 +54,7 @@ private[hive] class SparkGetFunctionsOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -81,7 +81,7 @@ private[hive] class SparkGetFunctionsOperation( authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -110,16 +110,16 @@ private[hive] class SparkGetFunctionsOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting functions: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 928610a6bcff9..db19880d1b99f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -50,7 +50,7 @@ private[hive] class SparkGetSchemasOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -68,7 +68,7 @@ private[hive] class SparkGetSchemasOperation( authorizeMetaGets(HiveOperationType.GET_TABLES, null, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -93,16 +93,16 @@ private[hive] class SparkGetSchemasOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting schemas: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index ec03f1e148e69..b4093e58d3c07 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -45,7 +45,7 @@ private[hive] class SparkGetTableTypesOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -61,7 +61,7 @@ private[hive] class SparkGetTableTypesOperation( authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -80,16 +80,16 @@ private[hive] class SparkGetTableTypesOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting table types: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index bf9cf7ad46d95..45c6d980aac47 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -59,7 +59,7 @@ private[hive] class SparkGetTablesOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -85,7 +85,7 @@ private[hive] class SparkGetTablesOperation( authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -124,17 +124,17 @@ private[hive] class SparkGetTablesOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting tables: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } private def addToRowSet( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index 0d263b09d57d3..dd5668a93f82d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -44,7 +44,7 @@ private[hive] class SparkGetTypeInfoOperation( override def close(): Unit = { super.close() - HiveThriftServer2.listener.onOperationClosed(statementId) + HiveThriftServer2.eventManager.onOperationClosed(statementId) } override def runInternal(): Unit = { @@ -60,7 +60,7 @@ private[hive] class SparkGetTypeInfoOperation( authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null) } - HiveThriftServer2.listener.onStatementStart( + HiveThriftServer2.eventManager.onStatementStart( statementId, parentSession.getSessionHandle.getSessionId.toString, logMsg, @@ -98,16 +98,16 @@ private[hive] class SparkGetTypeInfoOperation( setState(OperationState.ERROR) e match { case hiveException: HiveSQLException => - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, hiveException.getMessage, SparkUtils.exceptionString(hiveException)) throw hiveException case _ => val root = ExceptionUtils.getRootCause(e) - HiveThriftServer2.listener.onStatementError( + HiveThriftServer2.eventManager.onStatementError( statementId, root.getMessage, SparkUtils.exceptionString(root)) throw new HiveSQLException("Error getting type info: " + root.toString, root) } } - HiveThriftServer2.listener.onStatementFinish(statementId) + HiveThriftServer2.eventManager.onStatementFinish(statementId) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 41b324d70c315..b3171897141c2 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -55,7 +55,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: super.openSession(protocol, username, passwd, ipAddress, sessionConf, withImpersonation, delegationToken) val session = super.getSession(sessionHandle) - HiveThriftServer2.listener.onSessionCreated( + HiveThriftServer2.eventManager.onSessionCreated( session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) val ctx = if (sqlContext.conf.hiveThriftServerSingleSession) { sqlContext @@ -74,7 +74,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: } override def closeSession(sessionHandle: SessionHandle): Unit = { - HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) + HiveThriftServer2.eventManager.onSessionClosed(sessionHandle.getSessionId.toString) val ctx = sparkSqlOperationManager.sessionToContexts.getOrDefault(sessionHandle, sqlContext) ctx.sparkSession.sessionState.catalog.getTempViewNames().foreach(ctx.uncacheTable) super.closeSession(sessionHandle) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala new file mode 100644 index 0000000000000..5cb78f6e64650 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2AppStatusStore.scala @@ -0,0 +1,132 @@ +/* + * 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.hive.thriftserver.ui + +import com.fasterxml.jackson.annotation.JsonIgnore +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState +import org.apache.spark.status.KVUtils.KVIndexParam +import org.apache.spark.util.kvstore.{KVIndex, KVStore} + +/** + * Provides a view of a KVStore with methods that make it easy to query SQL-specific state. There's + * no state kept in this class, so it's ok to have multiple instances of it in an application. + */ +class HiveThriftServer2AppStatusStore( + store: KVStore, + val listener: Option[HiveThriftServer2Listener] = None) { + + def getSessionList: Seq[SessionInfo] = { + store.view(classOf[SessionInfo]).asScala.toSeq + } + + def getExecutionList: Seq[ExecutionInfo] = { + store.view(classOf[ExecutionInfo]).asScala.toSeq + } + + def getOnlineSessionNum: Int = { + store.view(classOf[SessionInfo]).asScala.count(_.finishTimestamp == 0) + } + + def getSession(sessionId: String): Option[SessionInfo] = { + try { + Some(store.read(classOf[SessionInfo], sessionId)) + } catch { + case _: NoSuchElementException => None + } + } + + def getExecution(executionId: String): Option[ExecutionInfo] = { + try { + Some(store.read(classOf[ExecutionInfo], executionId)) + } catch { + case _: NoSuchElementException => None + } + } + + /** + * When an error or a cancellation occurs, we set the finishTimestamp of the statement. + * Therefore, when we count the number of running statements, we need to exclude errors and + * cancellations and count all statements that have not been closed so far. + */ + def getTotalRunning: Int = { + store.view(classOf[ExecutionInfo]).asScala.count(_.isExecutionActive) + } + + def getSessionCount: Long = { + store.count(classOf[SessionInfo]) + } + + def getExecutionCount: Long = { + store.count(classOf[ExecutionInfo]) + } +} + +private[thriftserver] class SessionInfo( + @KVIndexParam val sessionId: String, + val startTimestamp: Long, + val ip: String, + val userName: String, + val finishTimestamp: Long, + val totalExecution: Long) { + @JsonIgnore @KVIndex("finishTime") + private def finishTimeIndex: Long = if (finishTimestamp > 0L ) finishTimestamp else -1L + def totalTime: Long = { + if (finishTimestamp == 0L) { + System.currentTimeMillis - startTimestamp + } else { + finishTimestamp - startTimestamp + } + } +} + +private[thriftserver] class ExecutionInfo( + @KVIndexParam val execId: String, + val statement: String, + val sessionId: String, + val startTimestamp: Long, + val userName: String, + val finishTimestamp: Long, + val closeTimestamp: Long, + val executePlan: String, + val detail: String, + val state: ExecutionState.Value, + val jobId: ArrayBuffer[String], + val groupId: String) { + @JsonIgnore @KVIndex("finishTime") + private def finishTimeIndex: Long = if (finishTimestamp > 0L && !isExecutionActive) { + finishTimestamp + } else -1L + + @JsonIgnore @KVIndex("isExecutionActive") + def isExecutionActive: Boolean = { + !(state == ExecutionState.FAILED || + state == ExecutionState.CANCELED || + state == ExecutionState.CLOSED) + } + + def totalTime(endTime: Long): Long = { + if (endTime == 0L) { + System.currentTimeMillis - startTimestamp + } else { + endTime - startTimestamp + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.scala new file mode 100644 index 0000000000000..fa04c67896a69 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2EventManager.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.hive.thriftserver.ui + +import org.apache.spark.SparkContext +import org.apache.spark.scheduler.SparkListenerEvent + +/** + * This class manages events generated by the thriftserver application. It converts the + * operation and session events to listener events and post it into the live listener bus. + */ +private[thriftserver] class HiveThriftServer2EventManager(sc: SparkContext) { + + def postLiveListenerBus(event: SparkListenerEvent): Unit = { + sc.listenerBus.post(event) + } + + def onSessionCreated(ip: String, sessionId: String, userName: String = "UNKNOWN"): Unit = { + postLiveListenerBus(SparkListenerThriftServerSessionCreated(ip, sessionId, + userName, System.currentTimeMillis())) + } + + def onSessionClosed(sessionId: String): Unit = { + postLiveListenerBus(SparkListenerThriftServerSessionClosed(sessionId, + System.currentTimeMillis())) + } + + def onStatementStart( + id: String, + sessionId: String, + statement: String, + groupId: String, + userName: String = "UNKNOWN"): Unit = { + postLiveListenerBus(SparkListenerThriftServerOperationStart(id, sessionId, statement, groupId, + System.currentTimeMillis(), userName)) + } + + def onStatementParsed(id: String, executionPlan: String): Unit = { + postLiveListenerBus(SparkListenerThriftServerOperationParsed(id, executionPlan)) + } + + def onStatementCanceled(id: String): Unit = { + postLiveListenerBus(SparkListenerThriftServerOperationCanceled(id, System.currentTimeMillis())) + } + + def onStatementError(id: String, errorMsg: String, errorTrace: String): Unit = { + postLiveListenerBus(SparkListenerThriftServerOperationError(id, errorMsg, errorTrace, + System.currentTimeMillis())) + } + + def onStatementFinish(id: String): Unit = { + postLiveListenerBus(SparkListenerThriftServerOperationFinish(id, System.currentTimeMillis())) + + } + + def onOperationClosed(id: String): Unit = { + postLiveListenerBus(SparkListenerThriftServerOperationClosed(id, System.currentTimeMillis())) + } +} + +private[thriftserver] case class SparkListenerThriftServerSessionCreated( + ip: String, + sessionId: String, + userName: String, + startTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerThriftServerSessionClosed( + sessionId: String, finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerThriftServerOperationStart( + id: String, + sessionId: String, + statement: String, + groupId: String, + startTime: Long, + userName: String = "UNKNOWN") extends SparkListenerEvent + +private[thriftserver] case class SparkListenerThriftServerOperationParsed( + id: String, + executionPlan: String) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerThriftServerOperationCanceled( + id: String, finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerThriftServerOperationError( + id: String, + errorMsg: String, + errorTrace: String, + finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerThriftServerOperationFinish( + id: String, + finishTime: Long) extends SparkListenerEvent + +private[thriftserver] case class SparkListenerThriftServerOperationClosed( + id: String, + closeTime: Long) extends SparkListenerEvent + + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala new file mode 100644 index 0000000000000..aec4125801f68 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2HistoryServerPlugin.scala @@ -0,0 +1,40 @@ +/* + * 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.hive.thriftserver.ui + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SparkListener +import org.apache.spark.status.{AppHistoryServerPlugin, ElementTrackingStore} +import org.apache.spark.ui.SparkUI + +class HiveThriftServer2HistoryServerPlugin extends AppHistoryServerPlugin { + + override def createListeners(conf: SparkConf, store: ElementTrackingStore): Seq[SparkListener] = { + Seq(new HiveThriftServer2Listener(store, conf, None, false)) + } + + override def setupUI(ui: SparkUI): Unit = { + val store = new HiveThriftServer2AppStatusStore(ui.store.store) + if (store.getSessionCount > 0) { + new ThriftServerTab(store, ui) + } + } + + override def displayOrder: Int = 1 +} + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala new file mode 100644 index 0000000000000..6d0a506fa94dc --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2Listener.scala @@ -0,0 +1,315 @@ +/* + * 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.hive.thriftserver.ui + +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.hive.service.server.HiveServer2 + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.config.Status.LIVE_ENTITY_UPDATE_PERIOD +import org.apache.spark.scheduler._ +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.ExecutionState +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} + +/** + * An inner sparkListener called in sc.stop to clean up the HiveThriftServer2 + */ +private[thriftserver] class HiveThriftServer2Listener( + kvstore: ElementTrackingStore, + sparkConf: SparkConf, + server: Option[HiveServer2], + live: Boolean = true) extends SparkListener { + + private val sessionList = new ConcurrentHashMap[String, LiveSessionData]() + private val executionList = new ConcurrentHashMap[String, LiveExecutionData]() + + private val (retainedStatements: Int, retainedSessions: Int) = { + (sparkConf.get(SQLConf.THRIFTSERVER_UI_STATEMENT_LIMIT), + sparkConf.get(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT)) + } + + // How often to update live entities. -1 means "never update" when replaying applications, + // meaning only the last write will happen. For live applications, this avoids a few + // operations that we can live without when rapidly processing incoming events. + private val liveUpdatePeriodNs = if (live) sparkConf.get(LIVE_ENTITY_UPDATE_PERIOD) else -1L + + // Returns true if this listener has no live data. Exposed for tests only. + private[thriftserver] def noLiveData(): Boolean = { + sessionList.isEmpty && executionList.isEmpty + } + + kvstore.addTrigger(classOf[SessionInfo], retainedSessions) { count => + cleanupSession(count) + } + + kvstore.addTrigger(classOf[ExecutionInfo], retainedStatements) { count => + cleanupExecutions(count) + } + + kvstore.onFlush { + if (!live) { + flush((entity: LiveEntity) => updateStoreWithTriggerEnabled(entity)) + } + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { + if (live) { + server.foreach(_.stop()) + } + } + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + val properties = jobStart.properties + if (properties != null) { + val groupId = properties.getProperty(SparkContext.SPARK_JOB_GROUP_ID) + if (groupId != null) { + updateJobDetails(jobStart.jobId.toString, groupId) + } + } + } + + private def updateJobDetails(jobId: String, groupId: String): Unit = { + val execList = executionList.values().asScala.filter(_.groupId == groupId).toSeq + if (execList.nonEmpty) { + execList.foreach { exec => + exec.jobId += jobId.toString + updateLiveStore(exec) + } + } else { + // It may possible that event reordering happens, such a way that JobStart event come after + // Execution end event (Refer SPARK-27019). To handle that situation, if occurs in + // Thriftserver, following code will take care. Here will come only if JobStart event comes + // after Execution End event. + val storeExecInfo = kvstore.view(classOf[ExecutionInfo]).asScala.filter(_.groupId == groupId) + storeExecInfo.foreach { exec => + val liveExec = getOrCreateExecution(exec.execId, exec.statement, exec.sessionId, + exec.startTimestamp, exec.userName) + liveExec.jobId += jobId.toString + updateStoreWithTriggerEnabled(liveExec) + executionList.remove(liveExec.execId) + } + } + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: SparkListenerThriftServerSessionCreated => onSessionCreated(e) + case e: SparkListenerThriftServerSessionClosed => onSessionClosed(e) + case e: SparkListenerThriftServerOperationStart => onOperationStart(e) + case e: SparkListenerThriftServerOperationParsed => onOperationParsed(e) + case e: SparkListenerThriftServerOperationCanceled => onOperationCanceled(e) + case e: SparkListenerThriftServerOperationError => onOperationError(e) + case e: SparkListenerThriftServerOperationFinish => onOperationFinished(e) + case e: SparkListenerThriftServerOperationClosed => onOperationClosed(e) + case _ => // Ignore + } + } + + private def onSessionCreated(e: SparkListenerThriftServerSessionCreated): Unit = { + val session = getOrCreateSession(e.sessionId, e.startTime, e.ip, e.userName) + sessionList.put(e.sessionId, session) + updateLiveStore(session) + } + + private def onSessionClosed(e: SparkListenerThriftServerSessionClosed): Unit = { + val session = sessionList.get(e.sessionId) + session.finishTimestamp = e.finishTime + updateStoreWithTriggerEnabled(session) + sessionList.remove(e.sessionId) + } + + private def onOperationStart(e: SparkListenerThriftServerOperationStart): Unit = { + val info = getOrCreateExecution( + e.id, + e.statement, + e.sessionId, + e.startTime, + e.userName) + + info.state = ExecutionState.STARTED + executionList.put(e.id, info) + sessionList.get(e.sessionId).totalExecution += 1 + executionList.get(e.id).groupId = e.groupId + updateLiveStore(executionList.get(e.id)) + updateLiveStore(sessionList.get(e.sessionId)) + } + + private def onOperationParsed(e: SparkListenerThriftServerOperationParsed): Unit = { + executionList.get(e.id).executePlan = e.executionPlan + executionList.get(e.id).state = ExecutionState.COMPILED + updateLiveStore(executionList.get(e.id)) + } + + private def onOperationCanceled(e: SparkListenerThriftServerOperationCanceled): Unit = { + executionList.get(e.id).finishTimestamp = e.finishTime + executionList.get(e.id).state = ExecutionState.CANCELED + updateLiveStore(executionList.get(e.id)) + } + + private def onOperationError(e: SparkListenerThriftServerOperationError): Unit = { + executionList.get(e.id).finishTimestamp = e.finishTime + executionList.get(e.id).detail = e.errorMsg + executionList.get(e.id).state = ExecutionState.FAILED + updateLiveStore(executionList.get(e.id)) + } + + private def onOperationFinished(e: SparkListenerThriftServerOperationFinish): Unit = { + executionList.get(e.id).finishTimestamp = e.finishTime + executionList.get(e.id).state = ExecutionState.FINISHED + updateLiveStore(executionList.get(e.id)) + } + + private def onOperationClosed(e: SparkListenerThriftServerOperationClosed): Unit = { + executionList.get(e.id).closeTimestamp = e.closeTime + executionList.get(e.id).state = ExecutionState.CLOSED + updateStoreWithTriggerEnabled(executionList.get(e.id)) + executionList.remove(e.id) + } + + // Update both live and history stores. Trigger is enabled by default, hence + // it will cleanup the entity which exceeds the threshold. + def updateStoreWithTriggerEnabled(entity: LiveEntity): Unit = { + entity.write(kvstore, System.nanoTime(), checkTriggers = true) + } + + // Update only live stores. If trigger is enabled, it will cleanup entity + // which exceeds the threshold. + def updateLiveStore(entity: LiveEntity, trigger: Boolean = false): Unit = { + val now = System.nanoTime() + if (live && liveUpdatePeriodNs >= 0 && now - entity.lastWriteTime > liveUpdatePeriodNs) { + entity.write(kvstore, now, checkTriggers = trigger) + } + } + + /** Go through all `LiveEntity`s and use `entityFlushFunc(entity)` to flush them. */ + private def flush(entityFlushFunc: LiveEntity => Unit): Unit = { + sessionList.values.asScala.foreach(entityFlushFunc) + executionList.values.asScala.foreach(entityFlushFunc) + } + + private def getOrCreateSession( + sessionId: String, + startTime: Long, + ip: String, + username: String): LiveSessionData = { + sessionList.computeIfAbsent(sessionId, + (_: String) => new LiveSessionData(sessionId, startTime, ip, username)) + } + + private def getOrCreateExecution( + execId: String, statement: String, + sessionId: String, startTimestamp: Long, + userName: String): LiveExecutionData = { + executionList.computeIfAbsent(execId, + (_: String) => new LiveExecutionData(execId, statement, sessionId, startTimestamp, userName)) + } + + private def cleanupExecutions(count: Long): Unit = { + val countToDelete = calculateNumberToRemove(count, retainedStatements) + if (countToDelete <= 0L) { + return + } + val view = kvstore.view(classOf[ExecutionInfo]).index("finishTime").first(0L) + val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j => + j.finishTimestamp != 0 + } + toDelete.foreach { j => kvstore.delete(j.getClass, j.execId) } + } + + private def cleanupSession(count: Long): Unit = { + val countToDelete = calculateNumberToRemove(count, retainedSessions) + if (countToDelete <= 0L) { + return + } + val view = kvstore.view(classOf[SessionInfo]).index("finishTime").first(0L) + val toDelete = KVUtils.viewToSeq(view, countToDelete.toInt) { j => + j.finishTimestamp != 0L + } + + toDelete.foreach { j => kvstore.delete(j.getClass, j.sessionId) } + } + + /** + * Remove at least (retainedSize / 10) items to reduce friction. Because tracking may be done + * asynchronously, this method may return 0 in case enough items have been deleted already. + */ + private def calculateNumberToRemove(dataSize: Long, retainedSize: Long): Long = { + if (dataSize > retainedSize) { + math.max(retainedSize / 10L, dataSize - retainedSize) + } else { + 0L + } + } +} + +private[thriftserver] class LiveExecutionData( + val execId: String, + val statement: String, + val sessionId: String, + val startTimestamp: Long, + val userName: String) extends LiveEntity { + + var finishTimestamp: Long = 0L + var closeTimestamp: Long = 0L + var executePlan: String = "" + var detail: String = "" + var state: ExecutionState.Value = ExecutionState.STARTED + val jobId: ArrayBuffer[String] = ArrayBuffer[String]() + var groupId: String = "" + + override protected def doUpdate(): Any = { + new ExecutionInfo( + execId, + statement, + sessionId, + startTimestamp, + userName, + finishTimestamp, + closeTimestamp, + executePlan, + detail, + state, + jobId, + groupId) + } +} + +private[thriftserver] class LiveSessionData( + val sessionId: String, + val startTimeStamp: Long, + val ip: String, + val username: String) extends LiveEntity { + + var finishTimestamp: Long = 0L + var totalExecution: Int = 0 + + override protected def doUpdate(): Any = { + new SessionInfo( + sessionId, + startTimeStamp, + ip, + username, + finishTimestamp, + totalExecution) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index d3351f3d6ca14..adfda0c56585f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -28,7 +28,6 @@ import scala.xml.{Node, Unparsed} import org.apache.commons.text.StringEscapeUtils import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, SessionInfo} import org.apache.spark.sql.hive.thriftserver.ui.ToolTips._ import org.apache.spark.ui._ import org.apache.spark.ui.UIUtils._ @@ -36,23 +35,24 @@ import org.apache.spark.util.Utils /** Page for Spark Web UI that shows statistics of the thrift server */ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("") with Logging { - - private val listener = parent.listener - private val startTime = Calendar.getInstance().getTime() + private val store = parent.store + private val startTime = parent.startTime /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { - val content = - listener.synchronized { // make sure all parts in this page are consistent - generateBasicStats() ++ -
    ++ + val content = store.synchronized { // make sure all parts in this page are consistent + generateBasicStats() ++ +
    ++

    - {listener.getOnlineSessionNum} session(s) are online, - running {listener.getTotalRunning} SQL statement(s) + {store.getOnlineSessionNum} + session(s) are online, + running + {store.getTotalRunning} + SQL statement(s)

    ++ generateSessionStatsTable(request) ++ generateSQLStatsTable(request) - } + } UIUtils.headerSparkPage(request, "JDBC/ODBC Server", content, parent) } @@ -72,7 +72,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" /** Generate stats of batch statements of the thrift server program */ private def generateSQLStatsTable(request: HttpServletRequest): Seq[Node] = { - val numStatement = listener.getExecutionList.size + val numStatement = store.getExecutionList.size val table = if (numStatement > 0) { @@ -103,7 +103,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" Some(new SqlStatsPagedTable( request, parent, - listener.getExecutionList, + store.getExecutionList, "sqlserver", UIUtils.prependBaseUri(request, parent.basePath), parameterOtherTable, @@ -141,7 +141,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" /** Generate stats of batch sessions of the thrift server program */ private def generateSessionStatsTable(request: HttpServletRequest): Seq[Node] = { - val numSessions = listener.getSessionList.size + val numSessions = store.getSessionList.size val table = if (numSessions > 0) { val sessionTableTag = "sessionstat" @@ -171,7 +171,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" Some(new SessionStatsPagedTable( request, parent, - listener.getSessionList, + store.getSessionList, "sqlserver", UIUtils.prependBaseUri(request, parent.basePath), parameterOtherTable, diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index aa2f495db5651..c46c3d6b68a43 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive.thriftserver.ui -import java.util.Calendar import javax.servlet.http.HttpServletRequest import scala.collection.JavaConverters._ @@ -31,18 +30,16 @@ import org.apache.spark.util.Utils /** Page for Spark Web UI that shows statistics of jobs running in the thrift server */ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) extends WebUIPage("session") with Logging { - - private val listener = parent.listener - private val startTime = Calendar.getInstance().getTime() + val store = parent.store + private val startTime = parent.startTime /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val content = - listener.synchronized { // make sure all parts in this page are consistent - val sessionStat = listener.getSession(parameterId).getOrElse(null) + val content = store.synchronized { // make sure all parts in this page are consistent + val sessionStat = store.getSession(parameterId).getOrElse(null) require(sessionStat != null, "Invalid sessionID[" + parameterId + "]") generateBasicStats() ++ @@ -73,7 +70,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) /** Generate stats of batch statements of the thrift server program */ private def generateSQLStatsTable(request: HttpServletRequest, sessionID: String): Seq[Node] = { - val executionList = listener.getExecutionList + val executionList = store.getExecutionList .filter(_.sessionId == sessionID) val numStatement = executionList.size val table = if (numStatement > 0) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala index 8efb2c3311cfe..6d783b1c555a7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala @@ -19,28 +19,25 @@ package org.apache.spark.sql.hive.thriftserver.ui import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 -import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab._ import org.apache.spark.ui.{SparkUI, SparkUITab} /** * Spark Web UI tab that shows statistics of jobs running in the thrift server. * This assumes the given SparkContext has enabled its SparkUI. */ -private[thriftserver] class ThriftServerTab(sparkContext: SparkContext) - extends SparkUITab(getSparkUI(sparkContext), "sqlserver") with Logging { - +private[thriftserver] class ThriftServerTab( + val store: HiveThriftServer2AppStatusStore, + sparkUI: SparkUI) extends SparkUITab(sparkUI, "sqlserver") with Logging { override val name = "JDBC/ODBC Server" - val parent = getSparkUI(sparkContext) - val listener = HiveThriftServer2.listener + val parent = sparkUI + val startTime = sparkUI.store.applicationInfo().attempts.head.startTime attachPage(new ThriftServerPage(this)) attachPage(new ThriftServerSessionPage(this)) parent.attachTab(this) - def detach(): Unit = { - getSparkUI(sparkContext).detachTab(this) + sparkUI.detachTab(this) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala new file mode 100644 index 0000000000000..075032fa5d099 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/HiveThriftServer2ListenerSuite.scala @@ -0,0 +1,164 @@ +/* + * 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.hive.thriftserver.ui + +import java.util.Properties + +import org.mockito.Mockito.{mock, RETURNS_SMART_NULLS} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.config.Status.{ASYNC_TRACKING_ENABLED, LIVE_ENTITY_UPDATE_PERIOD} +import org.apache.spark.scheduler.SparkListenerJobStart +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.InMemoryStore + +class HiveThriftServer2ListenerSuite extends SparkFunSuite with BeforeAndAfter { + + private var kvstore: ElementTrackingStore = _ + + after { + if (kvstore != null) { + kvstore.close() + kvstore = null + } + } + + Seq(true, false).foreach { live => + test(s"listener events should store successfully (live = $live)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(live) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", + "user", System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", + "dummy query", "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) + listener.onJobStart(SparkListenerJobStart( + 0, + System.currentTimeMillis(), + Nil, + createProperties)) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) + + if (live) { + assert(statusStore.getOnlineSessionNum === 1) + } + + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId", + System.currentTimeMillis())) + + if (!live) { + // To update history store + kvstore.close(false) + } + assert(statusStore.getOnlineSessionNum === 0) + assert(statusStore.getExecutionList.size === 1) + + val storeExecData = statusStore.getExecutionList.head + + assert(storeExecData.execId === "id") + assert(storeExecData.sessionId === "sessionId") + assert(storeExecData.executePlan === "dummy plan") + assert(storeExecData.jobId === Seq("0")) + assert(listener.noLiveData()) + } + } + + Seq(true, false).foreach { live => + test(s"cleanup session if exceeds the threshold (live = $live)") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(true) + var time = 0 + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId1", + "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId2", + "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId1", time)) + time += 1 + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId2", time)) + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId3", + "user", time)) + time += 1 + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId3", time)) + + if (!live) { + kvstore.close(false) + } + assert(statusStore.getOnlineSessionNum === 0) + assert(statusStore.getSessionCount === 1) + assert(statusStore.getSession("sessionId1") === None) + assert(listener.noLiveData()) + } + } + + test("update execution info when jobstart event come after execution end event") { + val (statusStore: HiveThriftServer2AppStatusStore, + listener: HiveThriftServer2Listener) = createAppStatusStore(true) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionId", "user", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionId", "dummy query", + "groupId", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) + listener.onJobStart(SparkListenerJobStart( + 0, + System.currentTimeMillis(), + Nil, + createProperties)) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionId", + System.currentTimeMillis())) + val exec = statusStore.getExecution("id") + assert(exec.isDefined) + assert(exec.get.jobId === Seq("0")) + assert(listener.noLiveData()) + } + + private def createProperties: Properties = { + val properties = new Properties() + properties.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "groupId") + properties + } + + private def createAppStatusStore(live: Boolean) = { + val sparkConf = new SparkConf() + sparkConf.set(ASYNC_TRACKING_ENABLED, false) + .set(SQLConf.THRIFTSERVER_UI_SESSION_LIMIT, 1) + .set(LIVE_ENTITY_UPDATE_PERIOD, 0L) + kvstore = new ElementTrackingStore(new InMemoryStore, sparkConf) + if (live) { + val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) + val listener = new HiveThriftServer2Listener(kvstore, sparkConf, Some(server)) + (new HiveThriftServer2AppStatusStore(kvstore, Some(listener)), listener) + } else { + (new HiveThriftServer2AppStatusStore(kvstore), + new HiveThriftServer2Listener(kvstore, sparkConf, None, false)) + } + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala index 6f040d3f1095b..9f3c2957a182d 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPageSuite.scala @@ -17,39 +17,64 @@ package org.apache.spark.sql.hive.thriftserver.ui -import java.util.Locale +import java.util.{Calendar, Locale} import javax.servlet.http.HttpServletRequest import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.scalatest.BeforeAndAfter -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.scheduler.SparkListenerJobStart -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.HiveThriftServer2Listener -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.hive.thriftserver._ +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.InMemoryStore -class ThriftServerPageSuite extends SparkFunSuite { + +class ThriftServerPageSuite extends SparkFunSuite with BeforeAndAfter { + + private var kvstore: ElementTrackingStore = _ + + after { + if (kvstore != null) { + kvstore.close() + kvstore = null + } + } /** - * Run a dummy session and return the listener + * Run a dummy session and return the store */ - private def getListener: HiveThriftServer2Listener = { - val listener = new HiveThriftServer2Listener(mock(classOf[HiveThriftServer2]), new SQLConf) - - listener.onSessionCreated("localhost", "sessionid", "user") - listener.onStatementStart("id", "sessionid", "dummy query", "groupid", "user") - listener.onStatementParsed("id", "dummy plan") - listener.onJobStart(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) - listener.onStatementFinish("id") - listener.onOperationClosed("id") - listener.onSessionClosed("sessionid") - listener + private def getStatusStore: HiveThriftServer2AppStatusStore = { + kvstore = new ElementTrackingStore(new InMemoryStore, new SparkConf()) + val server = mock(classOf[HiveThriftServer2], RETURNS_SMART_NULLS) + val sparkConf = new SparkConf + + val listener = new HiveThriftServer2Listener(kvstore, sparkConf, Some(server)) + val statusStore = new HiveThriftServer2AppStatusStore(kvstore, Some(listener)) + + listener.onOtherEvent(SparkListenerThriftServerSessionCreated("localhost", "sessionid", "user", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationStart("id", "sessionid", + "dummy query", "groupid", System.currentTimeMillis(), "user")) + listener.onOtherEvent(SparkListenerThriftServerOperationParsed("id", "dummy plan")) + listener.onOtherEvent(SparkListenerJobStart(0, System.currentTimeMillis(), Seq())) + listener.onOtherEvent(SparkListenerThriftServerOperationFinish("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerOperationClosed("id", + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerThriftServerSessionClosed("sessionid", + System.currentTimeMillis())) + + statusStore } test("thriftserver page should load successfully") { + val store = getStatusStore + val request = mock(classOf[HttpServletRequest]) val tab = mock(classOf[ThriftServerTab], RETURNS_SMART_NULLS) - when(tab.listener).thenReturn(getListener) + when(tab.startTime).thenReturn(Calendar.getInstance().getTime) + when(tab.store).thenReturn(store) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) val page = new ThriftServerPage(tab) @@ -70,10 +95,13 @@ class ThriftServerPageSuite extends SparkFunSuite { } test("thriftserver session page should load successfully") { + val store = getStatusStore + val request = mock(classOf[HttpServletRequest]) when(request.getParameter("id")).thenReturn("sessionid") val tab = mock(classOf[ThriftServerTab], RETURNS_SMART_NULLS) - when(tab.listener).thenReturn(getListener) + when(tab.startTime).thenReturn(Calendar.getInstance().getTime) + when(tab.store).thenReturn(store) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) val page = new ThriftServerSessionPage(tab) From 4a73bed3180aeb79c92bb19aea2ac5a97899731a Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 30 Nov 2019 12:48:15 +0900 Subject: [PATCH 0749/1387] [SPARK-29991][INFRA] Support Hive 1.2 and Hive 2.3 (default) in PR builder ### What changes were proposed in this pull request? Currently, Apache Spark PR Builder using `hive-1.2` for `hadoop-2.7` and `hive-2.3` for `hadoop-3.2`. This PR aims to support - `[test-hive1.2]` in PR builder - `[test-hive2.3]` in PR builder to be consistent and independent of the default profile - After this PR, all PR builders will use Hive 2.3 by default (because Spark uses Hive 2.3 by default as of https://github.com/apache/spark/commit/c98e5eb3396a6db92f2420e743afa9ddff319ca2) - Use default profile in AppVeyor build. Note that this was reverted due to unexpected test failure at `ThriftServerPageSuite`, which was investigated in https://github.com/apache/spark/pull/26706 . This PR fixed it by letting it use their own forked JVM. There is no explicit evidence for this fix and it was just my speculation, and thankfully it fixed at least. ### Why are the changes needed? This new tag allows us more flexibility. ### Does this PR introduce any user-facing change? No. (This is a dev-only change.) ### How was this patch tested? Check the Jenkins triggers in this PR. Default: ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark using SBT with these arguments: -Phadoop-2.7 -Phive-2.3 -Phive-thriftserver -Pmesos -Pspark-ganglia-lgpl -Phadoop-cloud -Phive -Pkubernetes -Pkinesis-asl -Pyarn test:package streaming-kinesis-asl-assembly/assembly ``` `[test-hive1.2][test-hadoop3.2]`: ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark using SBT with these arguments: -Phadoop-3.2 -Phive-1.2 -Phadoop-cloud -Pyarn -Pspark-ganglia-lgpl -Phive -Phive-thriftserver -Pmesos -Pkubernetes -Pkinesis-asl test:package streaming-kinesis-asl-assembly/assembly ``` `[test-maven][test-hive-2.3]`: ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark using Maven with these arguments: -Phadoop-2.7 -Phive-2.3 -Pspark-ganglia-lgpl -Pyarn -Phive -Phadoop-cloud -Pkinesis-asl -Pmesos -Pkubernetes -Phive-thriftserver clean package -DskipTests ``` Closes #26710 from HyukjinKwon/SPARK-29991. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- appveyor.yml | 2 +- dev/run-tests-jenkins.py | 5 +++ dev/run-tests.py | 78 +++++++++++++++++++++++++--------------- project/SparkBuild.scala | 3 +- 4 files changed, 58 insertions(+), 30 deletions(-) diff --git a/appveyor.yml b/appveyor.yml index 325fd67abc674..00c688ba18eb6 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -53,7 +53,7 @@ install: build_script: # '-Djna.nosys=true' is required to avoid kernel32.dll load failure. # See SPARK-28759. - - cmd: mvn -DskipTests -Psparkr -Phive -Phive-1.2 -Djna.nosys=true package + - cmd: mvn -DskipTests -Psparkr -Phive -Djna.nosys=true package environment: NOT_CRAN: true diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 5429aeba8ea13..18aabc3f63abe 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -182,6 +182,11 @@ def main(): os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop2.7" if "test-hadoop3.2" in ghprb_pull_title: os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop3.2" + # Switch the Hive profile based on the PR title: + if "test-hive1.2" in ghprb_pull_title: + os.environ["AMPLAB_JENKINS_BUILD_HIVE_PROFILE"] = "hive1.2" + if "test-hive2.3" in ghprb_pull_title: + os.environ["AMPLAB_JENKINS_BUILD_HIVE_PROFILE"] = "hive2.3" build_display_name = os.environ["BUILD_DISPLAY_NAME"] build_url = os.environ["BUILD_URL"] diff --git a/dev/run-tests.py b/dev/run-tests.py index 2d52ead06a041..5255a77ec2081 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -179,7 +179,8 @@ def run_apache_rat_checks(): run_cmd([os.path.join(SPARK_HOME, "dev", "check-license")]) -def run_scala_style_checks(build_profiles): +def run_scala_style_checks(extra_profiles): + build_profiles = extra_profiles + modules.root.build_profile_flags set_title_and_block("Running Scala style checks", "BLOCK_SCALA_STYLE") profiles = " ".join(build_profiles) print("[info] Checking Scala style using SBT with these profiles: ", profiles) @@ -283,8 +284,8 @@ def get_hadoop_profiles(hadoop_version): """ sbt_maven_hadoop_profiles = { - "hadoop2.7": ["-Phadoop-2.7", "-Phive-1.2"], - "hadoop3.2": ["-Phadoop-3.2", "-Phive-2.3"], + "hadoop2.7": ["-Phadoop-2.7"], + "hadoop3.2": ["-Phadoop-3.2"], } if hadoop_version in sbt_maven_hadoop_profiles: @@ -295,9 +296,28 @@ def get_hadoop_profiles(hadoop_version): sys.exit(int(os.environ.get("CURRENT_BLOCK", 255))) -def build_spark_maven(hadoop_version): +def get_hive_profiles(hive_version): + """ + For the given Hive version tag, return a list of Maven/SBT profile flags for + building and testing against that Hive version. + """ + + sbt_maven_hive_profiles = { + "hive1.2": ["-Phive-1.2"], + "hive2.3": ["-Phive-2.3"], + } + + if hive_version in sbt_maven_hive_profiles: + return sbt_maven_hive_profiles[hive_version] + else: + print("[error] Could not find", hive_version, "in the list. Valid options", + " are", sbt_maven_hive_profiles.keys()) + sys.exit(int(os.environ.get("CURRENT_BLOCK", 255))) + + +def build_spark_maven(extra_profiles): # Enable all of the profiles for the build: - build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags + build_profiles = extra_profiles + modules.root.build_profile_flags mvn_goals = ["clean", "package", "-DskipTests"] profiles_and_goals = build_profiles + mvn_goals @@ -306,9 +326,9 @@ def build_spark_maven(hadoop_version): exec_maven(profiles_and_goals) -def build_spark_sbt(hadoop_version): +def build_spark_sbt(extra_profiles): # Enable all of the profiles for the build: - build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags + build_profiles = extra_profiles + modules.root.build_profile_flags sbt_goals = ["test:package", # Build test jars as some tests depend on them "streaming-kinesis-asl-assembly/assembly"] profiles_and_goals = build_profiles + sbt_goals @@ -318,10 +338,10 @@ def build_spark_sbt(hadoop_version): exec_sbt(profiles_and_goals) -def build_spark_unidoc_sbt(hadoop_version): +def build_spark_unidoc_sbt(extra_profiles): set_title_and_block("Building Unidoc API Documentation", "BLOCK_DOCUMENTATION") # Enable all of the profiles for the build: - build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags + build_profiles = extra_profiles + modules.root.build_profile_flags sbt_goals = ["unidoc"] profiles_and_goals = build_profiles + sbt_goals @@ -331,9 +351,9 @@ def build_spark_unidoc_sbt(hadoop_version): exec_sbt(profiles_and_goals) -def build_spark_assembly_sbt(hadoop_version, checkstyle=False): +def build_spark_assembly_sbt(extra_profiles, checkstyle=False): # Enable all of the profiles for the build: - build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags + build_profiles = extra_profiles + modules.root.build_profile_flags sbt_goals = ["assembly/package"] profiles_and_goals = build_profiles + sbt_goals print("[info] Building Spark assembly using SBT with these arguments: ", @@ -343,25 +363,25 @@ def build_spark_assembly_sbt(hadoop_version, checkstyle=False): if checkstyle: run_java_style_checks(build_profiles) - build_spark_unidoc_sbt(hadoop_version) + build_spark_unidoc_sbt(extra_profiles) -def build_apache_spark(build_tool, hadoop_version): - """Will build Spark against Hive v1.2.1 given the passed in build tool (either `sbt` or - `maven`). Defaults to using `sbt`.""" +def build_apache_spark(build_tool, extra_profiles): + """Will build Spark with the extra profiles and the passed in build tool + (either `sbt` or `maven`). Defaults to using `sbt`.""" set_title_and_block("Building Spark", "BLOCK_BUILD") rm_r("lib_managed") if build_tool == "maven": - build_spark_maven(hadoop_version) + build_spark_maven(extra_profiles) else: - build_spark_sbt(hadoop_version) + build_spark_sbt(extra_profiles) -def detect_binary_inop_with_mima(hadoop_version): - build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags +def detect_binary_inop_with_mima(extra_profiles): + build_profiles = extra_profiles + modules.root.build_profile_flags set_title_and_block("Detecting binary incompatibilities with MiMa", "BLOCK_MIMA") profiles = " ".join(build_profiles) print("[info] Detecting binary incompatibilities with MiMa using SBT with these profiles: ", @@ -395,14 +415,14 @@ def run_scala_tests_sbt(test_modules, test_profiles): exec_sbt(profiles_and_goals) -def run_scala_tests(build_tool, hadoop_version, test_modules, excluded_tags): +def run_scala_tests(build_tool, extra_profiles, test_modules, excluded_tags): """Function to properly execute all tests passed in as a set from the `determine_test_suites` function""" set_title_and_block("Running Spark unit tests", "BLOCK_SPARK_UNIT_TESTS") test_modules = set(test_modules) - test_profiles = get_hadoop_profiles(hadoop_version) + \ + test_profiles = extra_profiles + \ list(set(itertools.chain.from_iterable(m.build_profile_flags for m in test_modules))) if excluded_tags: @@ -555,6 +575,7 @@ def main(): # to reflect the environment settings build_tool = os.environ.get("AMPLAB_JENKINS_BUILD_TOOL", "sbt") hadoop_version = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE", "hadoop2.7") + hive_version = os.environ.get("AMPLAB_JENKINS_BUILD_HIVE_PROFILE", "hive2.3") test_env = "amplab_jenkins" # add path for Python3 in Jenkins if we're calling from a Jenkins machine # TODO(sknapp): after all builds are ported to the ubuntu workers, change this to be: @@ -564,10 +585,12 @@ def main(): # else we're running locally and can use local settings build_tool = "sbt" hadoop_version = os.environ.get("HADOOP_PROFILE", "hadoop2.7") + hive_version = os.environ.get("HIVE_PROFILE", "hive2.3") test_env = "local" print("[info] Using build tool", build_tool, "with Hadoop profile", hadoop_version, - "under environment", test_env) + "and Hive profile", hive_version, "under environment", test_env) + extra_profiles = get_hadoop_profiles(hadoop_version) + get_hive_profiles(hive_version) changed_modules = None changed_files = None @@ -601,8 +624,7 @@ def main(): if not changed_files or any(f.endswith(".scala") or f.endswith("scalastyle-config.xml") for f in changed_files): - build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags - run_scala_style_checks(build_profiles) + run_scala_style_checks(extra_profiles) should_run_java_style_checks = False if not changed_files or any(f.endswith(".java") or f.endswith("checkstyle.xml") @@ -630,18 +652,18 @@ def main(): run_build_tests() # spark build - build_apache_spark(build_tool, hadoop_version) + build_apache_spark(build_tool, extra_profiles) # backwards compatibility checks if build_tool == "sbt": # Note: compatibility tests only supported in sbt for now - detect_binary_inop_with_mima(hadoop_version) + detect_binary_inop_with_mima(extra_profiles) # Since we did not build assembly/package before running dev/mima, we need to # do it here because the tests still rely on it; see SPARK-13294 for details. - build_spark_assembly_sbt(hadoop_version, should_run_java_style_checks) + build_spark_assembly_sbt(extra_profiles, should_run_java_style_checks) # run the test suites - run_scala_tests(build_tool, hadoop_version, test_modules, excluded_tags) + run_scala_tests(build_tool, extra_profiles, test_modules, excluded_tags) modules_with_python_tests = [m for m in test_modules if m.python_test_goals] if modules_with_python_tests: diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8dda5809fa374..5a9cf11f94f11 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -476,7 +476,8 @@ object SparkParallelTestGrouping { "org.apache.spark.ml.classification.LinearSVCSuite", "org.apache.spark.sql.SQLQueryTestSuite", "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite", - "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite" + "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", + "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite" ) private val DEFAULT_TEST_GROUP = "default_test_group" From 32af7004a254107340df908e8dd91a0bc22e068e Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 30 Nov 2019 12:49:14 +0900 Subject: [PATCH 0750/1387] [SPARK-25016][INFRA][FOLLOW-UP] Remove leftover for dropping Hadoop 2.6 in Jenkins's test script ### What changes were proposed in this pull request? This PR proposes to remove the leftover. After https://github.com/apache/spark/pull/22615, we don't have Hadoop 2.6 profile anymore in master. ### Why are the changes needed? Using "test-hadoop2.6" against master branch in a PR wouldn't work. ### Does this PR introduce any user-facing change? No (dev only). ### How was this patch tested? Manually tested at https://github.com/apache/spark/pull/26707 and Jenkins build will test. Without this fix, and hadoop2.6 in the pr title, it shows as below: ``` ======================================================================== Building Spark ======================================================================== [error] Could not find hadoop2.6 in the list. Valid options are dict_keys(['hadoop2.7', 'hadoop3.2']) Attempting to post to Github... ``` Closes #26708 from HyukjinKwon/SPARK-25016. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/run-tests-jenkins.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 18aabc3f63abe..72e32d4e16e14 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -176,8 +176,6 @@ def main(): if "test-maven" in ghprb_pull_title: os.environ["AMPLAB_JENKINS_BUILD_TOOL"] = "maven" # Switch the Hadoop profile based on the PR title: - if "test-hadoop2.6" in ghprb_pull_title: - os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop2.6" if "test-hadoop2.7" in ghprb_pull_title: os.environ["AMPLAB_JENKINS_BUILD_PROFILE"] = "hadoop2.7" if "test-hadoop3.2" in ghprb_pull_title: From 91b83de4173839d559bee582f5569dc9857ab648 Mon Sep 17 00:00:00 2001 From: shahid Date: Sat, 30 Nov 2019 20:30:04 +0900 Subject: [PATCH 0751/1387] [SPARK-30086][SQL][TESTS] Run HiveThriftServer2ListenerSuite on a dedicated JVM to fix flakiness ### What changes were proposed in this pull request? This PR tries to fix flakiness in `HiveThriftServer2ListenerSuite` by using a dedicated JVM (after we switch to Hive 2.3 by default in PR builders). Likewise in https://github.com/apache/spark/commit/4a73bed3180aeb79c92bb19aea2ac5a97899731a, there's no explicit evidence for this fix. See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/114653/testReport/org.apache.spark.sql.hive.thriftserver.ui/HiveThriftServer2ListenerSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/ ``` sbt.ForkMain$ForkError: sbt.ForkMain$ForkError: java.lang.LinkageError: loader constraint violation: loader (instance of net/bytebuddy/dynamic/loading/MultipleParentClassLoader) previously initiated loading for a different type with name "org/apache/hive/service/ServiceStateChangeListener" at org.mockito.codegen.HiveThriftServer2$MockitoMock$1974707245.(Unknown Source) at sun.reflect.GeneratedSerializationConstructorAccessor164.newInstance(Unknown Source) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.objenesis.instantiator.sun.SunReflectionFactoryInstantiator.newInstance(SunReflectionFactoryInstantiator.java:48) at org.objenesis.ObjenesisBase.newInstance(ObjenesisBase.java:73) at org.mockito.internal.creation.instance.ObjenesisInstantiator.newInstance(ObjenesisInstantiator.java:19) at org.mockito.internal.creation.bytebuddy.SubclassByteBuddyMockMaker.createMock(SubclassByteBuddyMockMaker.java:47) at org.mockito.internal.creation.bytebuddy.ByteBuddyMockMaker.createMock(ByteBuddyMockMaker.java:25) at org.mockito.internal.util.MockUtil.createMock(MockUtil.java:35) at org.mockito.internal.MockitoCore.mock(MockitoCore.java:62) at org.mockito.Mockito.mock(Mockito.java:1908) at org.mockito.Mockito.mock(Mockito.java:1880) at org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite.createAppStatusStore(HiveThriftServer2ListenerSuite.scala:156) at org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite.$anonfun$new$3(HiveThriftServer2ListenerSuite.scala:47) at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) ``` ### Why are the changes needed? To make test cases more robust. ### Does this PR introduce any user-facing change? No (dev only). ### How was this patch tested? Jenkins build. Closes #26720 from shahidki31/mock. Authored-by: shahid Signed-off-by: HyukjinKwon --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5a9cf11f94f11..f91fb4df4fd43 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -477,7 +477,8 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.SQLQueryTestSuite", "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite", "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", - "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite" + "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", + "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite" ) private val DEFAULT_TEST_GROUP = "default_test_group" From f22177c9576dd148d958383759f9e812413e455d Mon Sep 17 00:00:00 2001 From: "Liu,Linhong" Date: Sat, 30 Nov 2019 08:43:34 -0600 Subject: [PATCH 0752/1387] [SPARK-29486][SQL][FOLLOWUP] Document the reason to add days field ### What changes were proposed in this pull request? Follow up of #26134 to document the reason to add days filed and explain how do we use it ### Why are the changes needed? only comment ### Does this PR introduce any user-facing change? no ### How was this patch tested? no need test Closes #26701 from LinhongLiu/spark-29486-followup. Authored-by: Liu,Linhong Signed-off-by: Sean Owen --- .../java/org/apache/spark/unsafe/types/CalendarInterval.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index d2abb36b0727c..9f0bcd6ad82fe 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -34,6 +34,10 @@ public final class CalendarInterval implements Serializable, Comparable Date: Sat, 30 Nov 2019 09:07:01 -0600 Subject: [PATCH 0753/1387] [SPARK-30057][DOCS] Add a statement of platforms Spark runs on Closes #26690 from huangtianhua/add-note-spark-runs-on-arm64. Authored-by: huangtianhua Signed-off-by: Sean Owen --- docs/index.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/index.md b/docs/index.md index 5dd8d7816bddd..d86471d56d5d9 100644 --- a/docs/index.md +++ b/docs/index.md @@ -42,9 +42,7 @@ If you'd like to build Spark from source, visit [Building Spark](building-spark.html). -Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy to run -locally on one machine --- all you need is to have `java` installed on your system `PATH`, -or the `JAVA_HOME` environment variable pointing to a Java installation. +Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS), and it should run on any platform that runs a supported version of Java. This should include JVMs on x86_64 and ARM64. It's easy to run locally on one machine --- all you need is to have `java` installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. Spark runs on Java 8/11, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. Java 8 prior to version 8u92 support is deprecated as of Spark 3.0.0. From f32ca4b2799d028620d010f8c9c4f7c09f4ad864 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Sat, 30 Nov 2019 12:23:11 -0600 Subject: [PATCH 0754/1387] [SPARK-30076][BUILD][TESTS] Upgrade Mockito to 3.1.0 ### What changes were proposed in this pull request? We used 2.28.2 of Mockito as of https://github.com/apache/spark/pull/25139 because 3.0.0 might be unstable. Now 3.1.0 is released. See release notes - https://github.com/mockito/mockito/blob/v3.1.0/doc/release-notes/official.md ### Why are the changes needed? To bring the fixes made in the dependency. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Jenkins will test. Closes #26707 from HyukjinKwon/upgrade-Mockito. Authored-by: HyukjinKwon Signed-off-by: Sean Owen --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 94214a53c3acb..bd30a16be47a4 100644 --- a/pom.xml +++ b/pom.xml @@ -853,7 +853,7 @@ org.mockito mockito-core - 2.28.2 + 3.1.0 test From 708ab57f377bfd8e71183cfead918bae5b811946 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 2 Dec 2019 09:02:39 +0900 Subject: [PATCH 0755/1387] [SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column ## What changes were proposed in this pull request? [HIVE-12063](https://issues.apache.org/jira/browse/HIVE-12063) improved pad decimal numbers with trailing zeros to the scale of the column. The following description is copied from the description of HIVE-12063. > HIVE-7373 was to address the problems of trimming tailing zeros by Hive, which caused many problems including treating 0.0, 0.00 and so on as 0, which has different precision/scale. Please refer to HIVE-7373 description. However, HIVE-7373 was reverted by HIVE-8745 while the underlying problems remained. HIVE-11835 was resolved recently to address one of the problems, where 0.0, 0.00, and so on cannot be read into decimal(1,1). However, HIVE-11835 didn't address the problem of showing as 0 in query result for any decimal values such as 0.0, 0.00, etc. This causes confusion as 0 and 0.0 have different precision/scale than 0. The proposal here is to pad zeros for query result to the type's scale. This not only removes the confusion described above, but also aligns with many other DBs. Internal decimal number representation doesn't change, however. **Spark SQL**: ```sql // bin/spark-sql spark-sql> select cast(1 as decimal(38, 18)); 1 spark-sql> // bin/beeline 0: jdbc:hive2://localhost:10000/default> select cast(1 as decimal(38, 18)); +----------------------------+--+ | CAST(1 AS DECIMAL(38,18)) | +----------------------------+--+ | 1.000000000000000000 | +----------------------------+--+ // bin/spark-shell scala> spark.sql("select cast(1 as decimal(38, 18))").show(false) +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ |1.000000000000000000 | +-------------------------+ // bin/pyspark >>> spark.sql("select cast(1 as decimal(38, 18))").show() +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ | 1.000000000000000000| +-------------------------+ // bin/sparkR > showDF(sql("SELECT cast(1 as decimal(38, 18))")) +-------------------------+ |CAST(1 AS DECIMAL(38,18))| +-------------------------+ | 1.000000000000000000| +-------------------------+ ``` **PostgreSQL**: ```sql postgres=# select cast(1 as decimal(38, 18)); numeric ---------------------- 1.000000000000000000 (1 row) ``` **Presto**: ```sql presto> select cast(1 as decimal(38, 18)); _col0 ---------------------- 1.000000000000000000 (1 row) ``` ## How was this patch tested? unit tests and manual test: ```sql spark-sql> select cast(1 as decimal(38, 18)); 1.000000000000000000 ``` Spark SQL Upgrading Guide: ![image](https://user-images.githubusercontent.com/5399861/69649620-4405c380-10a8-11ea-84b1-6ee675663b98.png) Closes #26697 from wangyum/SPARK-28461. Authored-by: Yuming Wang Signed-off-by: HyukjinKwon --- docs/sql-migration-guide.md | 26 ++ .../aggregate/ApproximatePercentile.scala | 2 +- .../expressions/conditionalExpressions.scala | 4 +- .../spark/sql/execution/HiveResult.scala | 10 +- .../ansi/decimalArithmeticOperations.sql.out | 18 +- .../sql-tests/results/ansi/literals.sql.out | 4 +- .../decimalArithmeticOperations.sql.out | 46 +-- .../sql-tests/results/literals.sql.out | 4 +- .../results/order-by-nulls-ordering.sql.out | 48 +-- .../results/postgreSQL/create_view.sql.out | 2 +- .../sql-tests/results/postgreSQL/date.sql.out | 4 +- .../sql-tests/results/postgreSQL/int2.sql.out | 14 +- .../sql-tests/results/postgreSQL/int4.sql.out | 14 +- .../sql-tests/results/postgreSQL/int8.sql.out | 14 +- .../results/postgreSQL/numeric.sql.out | 384 +++++++++--------- .../results/postgreSQL/select.sql.out | 20 +- .../results/postgreSQL/timestamp.sql.out | 20 +- .../results/postgreSQL/union.sql.out | 16 +- .../results/postgreSQL/window_part4.sql.out | 8 +- ...in-unit-tests-multi-column-literal.sql.out | 4 +- .../not-in-unit-tests-multi-column.sql.out | 10 +- ...n-unit-tests-single-column-literal.sql.out | 2 +- .../not-in-unit-tests-single-column.sql.out | 18 +- .../sql-tests/results/table-aliases.sql.out | 2 +- .../native/decimalPrecision.sql.out | 80 ++-- .../typeCoercion/native/division.sql.out | 18 +- .../sql-tests/results/udf/udf-union.sql.out | 12 +- .../resources/sql-tests/results/union.sql.out | 12 +- .../spark/sql/execution/HiveResultSuite.scala | 15 + .../sql/hive/thriftserver/CliSuite.scala | 7 + ...cimal_1_1-3-ac24b36077314acab595ada14e598e | 24 +- ...mal_1_1-4-128804f8dfe7dbb23be0498b91647ba3 | 24 +- ...cimal_4-6-693c2e345731f9b2b547c3b75218458e | 72 ++-- ...cimal_4-7-f1eb45492510cb76cf6b452121af8531 | 72 ++-- ..._regex-10-c5b3ec90419a40660e5f83736241c429 | 74 ++-- ...nistic)-2-1e88e0ba414a00195f7ebf6b8600ac04 | 192 ++++----- ...istic) 2-0-81bb7f49a55385878637c8aac4d08e5 | 242 +++++------ ...stic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 | 164 ++++---- 38 files changed, 871 insertions(+), 831 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 74ba694800d2d..12d734fcbdf52 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -226,6 +226,32 @@ license: | - Since Spark 3.0, when casting string value to date, timestamp and interval values, the leading and trailing white spaces(<= ACSII 32) will be trimmed before casing, e.g. `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, only the trailing space will be removed, thus, the result is `null`. + - Since Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example: + + + + + + + + + + + +
    + Query + + Spark 2.4 or Prior + + Spark 3.0 +
    + SELECT CAST(1 AS decimal(38, 18)); + + 1 + + 1.000000000000000000 +
    + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index 59481ce049165..ea0ed2e8fa11b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -63,7 +63,7 @@ import org.apache.spark.sql.types._ > SELECT _FUNC_(10.0, array(0.5, 0.4, 0.1), 100); [10.0,10.0,10.0] > SELECT _FUNC_(10.0, 0.5, 100); - 10 + 10.0 """, since = "2.1.0") case class ApproximatePercentile( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 3a2dc40862bb8..6c6210994954c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -111,9 +111,9 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi examples = """ Examples: > SELECT CASE WHEN 1 > 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END; - 1 + 1.0 > SELECT CASE WHEN 1 < 0 THEN 1 WHEN 2 > 0 THEN 2.0 ELSE 1.2 END; - 2 + 2.0 > SELECT CASE WHEN 1 < 0 THEN 1 WHEN 2 < 0 THEN 2.0 END; NULL """) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index d4e10b3ffc733..f7f7e08462fe1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -59,14 +59,6 @@ object HiveResult { result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) } - private def formatDecimal(d: java.math.BigDecimal): String = { - if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { - java.math.BigDecimal.ZERO.toPlainString - } else { - d.stripTrailingZeros().toPlainString // Hive strips trailing zeros - } - } - private val primitiveTypes = Seq( StringType, IntegerType, @@ -127,7 +119,7 @@ object HiveResult { case (t: Timestamp, TimestampType) => DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) - case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) + case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString case (interval: CalendarInterval, CalendarIntervalType) => SQLConf.get.intervalOutputStyle match { case SQL_STANDARD => toSqlStandardString(interval) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index e12f409f6d47e..67cb0a2b36a92 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -24,10 +24,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 2 schema struct -- !query 2 output -1 1000 99.9 -2 123451.23 1234.5123 -3 1.234567891011 123.41 -4 1234567891234567890 0.112345678912345679 +1 1000.000000000000000 99.900000000000000000 +2 123451.230000000000000 1234.512300000000000000 +3 1.234567891011000 123.410000000000000000 +4 1234567891234567890.000000000000000 0.112345678912345679 -- !query 3 @@ -35,7 +35,7 @@ select 10.3 * 3.0 -- !query 3 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 3 output -30.9 +30.90 -- !query 4 @@ -43,7 +43,7 @@ select 10.3000 * 3.0 -- !query 4 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 4 output -30.9 +30.90000 -- !query 5 @@ -51,7 +51,7 @@ select 10.30000 * 30.0 -- !query 5 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 5 output -309 +309.000000 -- !query 6 @@ -59,7 +59,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 6 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> -- !query 6 output -30.9 +30.9000000000000000000000000000000000 -- !query 7 @@ -67,7 +67,7 @@ select 10.300000000000000000 * 3.0000000000000000000 -- !query 7 schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> -- !query 7 output -30.9 +30.9000000000000000000000000000000000 -- !query 8 diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index c1488eaf2aa36..5d8a893d334c2 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -169,7 +169,7 @@ select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query 17 schema struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> -- !query 17 output -1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90 90 +1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90.0 90 -- !query 18 @@ -339,7 +339,7 @@ select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD -- !query 35 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> -- !query 35 output -90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 +90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 -- !query 36 diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index cbf44548b3cce..6f035a1904ea3 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -56,9 +56,9 @@ select id, a+b, a-b, a*b, a/b from decimals_test order by id -- !query 6 schema struct -- !query 6 output -1 1099 -899 99900 0.1001 -2 24690.246 0 152402061.885129 1 -3 1234.2234567891011 -1233.9765432108989 152.358023 0.0001 +1 1099.00000000000000000 -899.00000000000000000 99900.000000 0.100100 +2 24690.24600000000000000 0.00000000000000000 152402061.885129 1.000000 +3 1234.22345678910110000 -1233.97654321089890000 152.358023 0.000100 4 123456789123456790.12345678912345679 123456789123456787.87654321087654321 138698367904130467.515623 109890109097814272.043109 @@ -67,10 +67,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 7 schema struct -- !query 7 output -1 1000 99.9 -2 123451.23 1234.5123 -3 1.234567891011 123.41 -4 1234567891234567890 0.112345678912345679 +1 1000.000000000000000 99.900000000000000000 +2 123451.230000000000000 1234.512300000000000000 +3 1.234567891011000 123.410000000000000000 +4 1234567891234567890.000000000000000 0.112345678912345679 -- !query 8 @@ -78,7 +78,7 @@ select 10.3 * 3.0 -- !query 8 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 8 output -30.9 +30.90 -- !query 9 @@ -86,7 +86,7 @@ select 10.3000 * 3.0 -- !query 9 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 9 output -30.9 +30.90000 -- !query 10 @@ -94,7 +94,7 @@ select 10.30000 * 30.0 -- !query 10 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 10 output -309 +309.000000 -- !query 11 @@ -102,7 +102,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 11 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> -- !query 11 output -30.9 +30.9000000000000000000000000000000000 -- !query 12 @@ -110,7 +110,7 @@ select 10.300000000000000000 * 3.0000000000000000000 -- !query 12 schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> -- !query 12 output -30.9 +30.9000000000000000000000000000000000 -- !query 13 @@ -198,9 +198,9 @@ select id, a+b, a-b, a*b, a/b from decimals_test order by id -- !query 23 schema struct -- !query 23 output -1 1099 -899 NULL 0.1001001001001001 -2 24690.246 0 NULL 1 -3 1234.2234567891011 -1233.9765432108989 NULL 0.000100037913541123 +1 1099.000000000000000000 -899.000000000000000000 NULL 0.100100100100100100 +2 24690.246000000000000000 0.000000000000000000 NULL 1.000000000000000000 +3 1234.223456789101100000 -1233.976543210898900000 NULL 0.000100037913541123 4 123456789123456790.123456789123456789 123456789123456787.876543210876543211 NULL 109890109097814272.043109406191131436 @@ -209,10 +209,10 @@ select id, a*10, b/10 from decimals_test order by id -- !query 24 schema struct -- !query 24 output -1 1000 99.9 -2 123451.23 1234.5123 -3 1.234567891011 123.41 -4 1234567891234567890 0.1123456789123456789 +1 1000.000000000000000000 99.9000000000000000000 +2 123451.230000000000000000 1234.5123000000000000000 +3 1.234567891011000000 123.4100000000000000000 +4 1234567891234567890.000000000000000000 0.1123456789123456789 -- !query 25 @@ -220,7 +220,7 @@ select 10.3 * 3.0 -- !query 25 schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> -- !query 25 output -30.9 +30.90 -- !query 26 @@ -228,7 +228,7 @@ select 10.3000 * 3.0 -- !query 26 schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> -- !query 26 output -30.9 +30.90000 -- !query 27 @@ -236,7 +236,7 @@ select 10.30000 * 30.0 -- !query 27 schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> -- !query 27 output -309 +309.000000 -- !query 28 @@ -244,7 +244,7 @@ select 10.300000000000000000 * 3.000000000000000000 -- !query 28 schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,36)> -- !query 28 output -30.9 +30.900000000000000000000000000000000000 -- !query 29 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index c1488eaf2aa36..5d8a893d334c2 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -169,7 +169,7 @@ select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query 17 schema struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> -- !query 17 output -1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90 90 +1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90.0 90 -- !query 18 @@ -339,7 +339,7 @@ select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD -- !query 35 schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> -- !query 35 output -90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 +90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 -- !query 36 diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out index c1b63dfb8caef..e1289dd8c8bde 100644 --- a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out @@ -206,14 +206,14 @@ select * from spark_10747_mix order by col1 nulls last, col5 nulls last -- !query 13 schema struct -- !query 13 output -a 1 1.0 1 NULL -b 2 1.0 1 10 -c 3 2.0 2 15.1 -c 3 2.0 2 NULL -d 3 2.0 3 0 -d 3 0.0 3 1 -d 3 NULL 4 1 -NULL 3 0.0 3 1 +a 1 1.0 1.00 NULL +b 2 1.0 1.00 10.0 +c 3 2.0 2.00 15.1 +c 3 2.0 2.00 NULL +d 3 2.0 3.00 0.0 +d 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +NULL 3 0.0 3.00 1.0 -- !query 14 @@ -221,14 +221,14 @@ select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls fi -- !query 14 schema struct -- !query 14 output -NULL 3 0.0 3 1 -d 3 0.0 3 1 -d 3 NULL 4 1 -d 3 2.0 3 0 -c 3 2.0 2 NULL -c 3 2.0 2 15.1 -b 2 1.0 1 10 -a 1 1.0 1 NULL +NULL 3 0.0 3.00 1.0 +d 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +d 3 2.0 3.00 0.0 +c 3 2.0 2.00 NULL +c 3 2.0 2.00 15.1 +b 2 1.0 1.00 10.0 +a 1 1.0 1.00 NULL -- !query 15 @@ -236,14 +236,14 @@ select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls la -- !query 15 schema struct -- !query 15 output -c 3 2.0 2 NULL -a 1 1.0 1 NULL -c 3 2.0 2 15.1 -b 2 1.0 1 10 -d 3 0.0 3 1 -NULL 3 0.0 3 1 -d 3 NULL 4 1 -d 3 2.0 3 0 +c 3 2.0 2.00 NULL +a 1 1.0 1.00 NULL +c 3 2.0 2.00 15.1 +b 2 1.0 1.00 10.0 +d 3 0.0 3.00 1.0 +NULL 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +d 3 2.0 3.00 0.0 -- !query 16 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index 031918961df4c..cd4d3e64b8da1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -956,7 +956,7 @@ SELECT * FROM tt1 struct -- !query 83 output 0123456789 abc 42.12 abc -abc 0123456789 42 abcd +abc 0123456789 42.00 abcd -- !query 84 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index 68ff2a9f68899..5371e07423c28 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -576,7 +576,7 @@ SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') -- !query 54 schema struct -- !query 54 output -0 +0.000000 -- !query 55 @@ -584,7 +584,7 @@ SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') -- !query 55 schema struct -- !query 55 output -0 +0.000000 -- !query 56 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out index 8f41accfc8c49..d0a14618a5163 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -346,13 +346,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 33 schema struct -- !query 33 output --0.5 0 --1.5 -1 --2.5 -2 -0 0 -0.5 0 -1.5 1 -2.5 2 +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 -- !query 34 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index 6b1a8eb4a5883..30afd6e695842 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -495,13 +495,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 51 schema struct -- !query 51 output --0.5 0 --1.5 -1 --2.5 -2 -0 0 -0.5 0 -1.5 1 -2.5 2 +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 -- !query 52 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index c4d8affe75267..7a8e706cb8d85 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -836,13 +836,13 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), -- !query 83 schema struct -- !query 83 output --0.5 0 --1.5 -1 --2.5 -2 -0 0 -0.5 0 -1.5 1 -2.5 2 +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 -- !query 84 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 5ffa29a93d5de..e245af28508c1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -3691,16 +3691,16 @@ SELECT * FROM num_data -- !query 460 schema struct -- !query 460 output -0 0 -1 0 -2 -34338492.215397047 -3 4.31 -4 7799461.4119 -5 16397.038491 -6 93901.57763026 -7 -83028485 -8 74881 -9 -24926804.04504742 +0 0.0000000000 +1 0.0000000000 +2 -34338492.2153970470 +3 4.3100000000 +4 7799461.4119000000 +5 16397.0384910000 +6 93901.5776302600 +7 -83028485.0000000000 +8 74881.0000000000 +9 -24926804.0450474200 -- !query 461 @@ -3869,51 +3869,51 @@ SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected -- !query 478 schema struct -- !query 478 output -2 2 1179132047626883.596862 1179132047626883.5968621359 -2 3 -147998901.448361 -147998901.4483612726 -2 4 -267821744976817.811114 -267821744976817.8111137107 -2 5 -563049578578.769243 -563049578578.7692425067 -2 6 -3224438592470.184498 -3224438592470.1844981193 -2 7 2851072985828710.485884 2851072985828710.485883795 -2 8 -2571300635581.146276 -2571300635581.146276407 -2 9 855948866655588.453742 855948866655588.4537415092 -3 2 -147998901.448361 -147998901.4483612726 -3 5 70671.235896 70671.23589621 -3 6 404715.799586 404715.7995864206 -3 9 -107434525.434154 -107434525.4341543802 -4 2 -267821744976817.811114 -267821744976817.8111137107 -4 4 60831598315717.141462 60831598315717.14146161 -4 5 127888068979.993505 127888068979.9935054429 -4 6 732381731243.745116 732381731243.7451157641 -4 9 -194415646271340.181596 -194415646271340.1815956523 -5 2 -563049578578.769243 -563049578578.7692425067 -5 3 70671.235896 70671.23589621 -5 4 127888068979.993505 127888068979.9935054429 -5 5 268862871.275336 268862871.2753355571 -5 6 1539707782.768998 1539707782.7689977863 -5 9 -408725765384.257044 -408725765384.2570436602 -6 2 -3224438592470.184498 -3224438592470.1844981193 -6 3 404715.799586 404715.7995864206 -6 4 732381731243.745116 732381731243.7451157641 -6 5 1539707782.768998 1539707782.7689977863 -6 6 8817506281.451745 8817506281.4517452373 -6 7 -7796505729750.377956 -7796505729750.3779561 -6 8 7031444034.531499 7031444034.53149906 -6 9 -2340666225110.299295 -2340666225110.2992952129 -7 2 2851072985828710.485884 2851072985828710.485883795 -7 6 -7796505729750.377956 -7796505729750.3779561 -7 9 2069634775752159.035759 2069634775752159.0357587 -8 2 -2571300635581.146276 -2571300635581.146276407 -8 6 7031444034.531499 7031444034.53149906 -8 9 -1866544013697.195857 -1866544013697.19585702 -9 2 855948866655588.453742 855948866655588.4537415092 -9 3 -107434525.434154 -107434525.4341543802 -9 4 -194415646271340.181596 -194415646271340.1815956523 -9 5 -408725765384.257044 -408725765384.2570436602 -9 6 -2340666225110.299295 -2340666225110.2992952129 -9 7 2069634775752159.035759 2069634775752159.0357587 -9 8 -1866544013697.195857 -1866544013697.19585702 -9 9 621345559900192.420121 621345559900192.42012063 +2 2 1179132047626883.5968620000 1179132047626883.5968621359 +2 3 -147998901.4483610000 -147998901.4483612726 +2 4 -267821744976817.8111140000 -267821744976817.8111137107 +2 5 -563049578578.7692430000 -563049578578.7692425067 +2 6 -3224438592470.1844980000 -3224438592470.1844981193 +2 7 2851072985828710.4858840000 2851072985828710.4858837950 +2 8 -2571300635581.1462760000 -2571300635581.1462764070 +2 9 855948866655588.4537420000 855948866655588.4537415092 +3 2 -147998901.4483610000 -147998901.4483612726 +3 5 70671.2358960000 70671.2358962100 +3 6 404715.7995860000 404715.7995864206 +3 9 -107434525.4341540000 -107434525.4341543802 +4 2 -267821744976817.8111140000 -267821744976817.8111137107 +4 4 60831598315717.1414620000 60831598315717.1414616100 +4 5 127888068979.9935050000 127888068979.9935054429 +4 6 732381731243.7451160000 732381731243.7451157641 +4 9 -194415646271340.1815960000 -194415646271340.1815956523 +5 2 -563049578578.7692430000 -563049578578.7692425067 +5 3 70671.2358960000 70671.2358962100 +5 4 127888068979.9935050000 127888068979.9935054429 +5 5 268862871.2753360000 268862871.2753355571 +5 6 1539707782.7689980000 1539707782.7689977863 +5 9 -408725765384.2570440000 -408725765384.2570436602 +6 2 -3224438592470.1844980000 -3224438592470.1844981193 +6 3 404715.7995860000 404715.7995864206 +6 4 732381731243.7451160000 732381731243.7451157641 +6 5 1539707782.7689980000 1539707782.7689977863 +6 6 8817506281.4517450000 8817506281.4517452373 +6 7 -7796505729750.3779560000 -7796505729750.3779561000 +6 8 7031444034.5314990000 7031444034.5314990600 +6 9 -2340666225110.2992950000 -2340666225110.2992952129 +7 2 2851072985828710.4858840000 2851072985828710.4858837950 +7 6 -7796505729750.3779560000 -7796505729750.3779561000 +7 9 2069634775752159.0357590000 2069634775752159.0357587000 +8 2 -2571300635581.1462760000 -2571300635581.1462764070 +8 6 7031444034.5314990000 7031444034.5314990600 +8 9 -1866544013697.1958570000 -1866544013697.1958570200 +9 2 855948866655588.4537420000 855948866655588.4537415092 +9 3 -107434525.4341540000 -107434525.4341543802 +9 4 -194415646271340.1815960000 -194415646271340.1815956523 +9 5 -408725765384.2570440000 -408725765384.2570436602 +9 6 -2340666225110.2992950000 -2340666225110.2992952129 +9 7 2069634775752159.0357590000 2069634775752159.0357587000 +9 8 -1866544013697.1958570000 -1866544013697.1958570200 +9 9 621345559900192.4201210000 621345559900192.4201206300 -- !query 479 @@ -3942,62 +3942,62 @@ SELECT t1.id1, t1.id2, t1.result, t2.expected -- !query 481 schema struct -- !query 481 output -2 3 -7967167.567378 -7967167.5673775051 -2 4 -4.402675 -4.4026748005 -2 5 -2094.188669 -2094.1886691456 -2 6 -365.685999 -365.6859989148 -2 7 0.413575 0.4135748378 -2 8 -458.574167 -458.5741672173 -2 9 1.377573 1.3775729995 -3 2 0 -0.0000001255 -3 4 0.000001 0.0000005526 -3 5 0.000263 0.0002628523 -3 6 0.000046 0.0000458991 -3 7 0 -0.0000000519 -3 8 0.000058 0.000057558 -3 9 0 -0.0000001729 -4 2 -0.227135 -0.22713465 -4 3 1809619.817146 1809619.8171461717 -4 5 475.66281 475.6628104631 -4 6 83.059961 83.0599613844 -4 7 -0.093937 -0.093937176 -4 8 104.158083 104.1580829837 -4 9 -0.312895 -0.3128945611 -5 2 -0.000478 -0.0004775119 -5 3 3804.417283 3804.4172832947 -5 4 0.002102 0.0021023296 -5 6 0.174619 0.1746194143 -5 7 -0.000197 -0.0001974869 -5 8 0.218975 0.2189746196 -5 9 -0.000658 -0.0006578075 -6 2 -0.002735 -0.0027345865 -6 3 21786.908963 21786.9089629374 -6 4 0.012039 0.0120394951 -6 5 5.72674 5.7267400867 -6 7 -0.001131 -0.0011309562 -6 8 1.254011 1.2540107321 -6 9 -0.003767 -0.0037670925 -7 2 2.417942 2.4179420715 -7 3 -19264149.651972 -19264149.6519721578 -7 4 -10.645413 -10.6454126273 -7 5 -5063.626889 -5063.6268888173 -7 6 -884.207562 -884.2075617401 -7 8 -1108.805772 -1108.8057718246 -7 9 3.330892 3.330891712 -8 2 -0.002181 -0.0021806723 -8 3 17373.781903 17373.7819025522 -8 4 0.009601 0.0096007911 -8 5 4.566739 4.5667392951 -8 6 0.797441 0.7974413411 -8 7 -0.000902 -0.0009018712 -8 9 -0.003004 -0.0030040353 -9 2 0.725914 0.7259143438 -9 3 -5783481.216948 -5783481.2169483573 -9 4 -3.195965 -3.1959647889 -9 5 -1520.201594 -1520.2015936432 -9 6 -265.456712 -265.4567119543 -9 7 0.30022 0.300219907 -9 8 -332.885566 -332.8855656982 +2 3 -7967167.5673780000 -7967167.5673775051 +2 4 -4.4026750000 -4.4026748005 +2 5 -2094.1886690000 -2094.1886691456 +2 6 -365.6859990000 -365.6859989148 +2 7 0.4135750000 0.4135748378 +2 8 -458.5741670000 -458.5741672173 +2 9 1.3775730000 1.3775729995 +3 2 0.0000000000 -0.0000001255 +3 4 0.0000010000 0.0000005526 +3 5 0.0002630000 0.0002628523 +3 6 0.0000460000 0.0000458991 +3 7 0.0000000000 -0.0000000519 +3 8 0.0000580000 0.0000575580 +3 9 0.0000000000 -0.0000001729 +4 2 -0.2271350000 -0.2271346500 +4 3 1809619.8171460000 1809619.8171461717 +4 5 475.6628100000 475.6628104631 +4 6 83.0599610000 83.0599613844 +4 7 -0.0939370000 -0.0939371760 +4 8 104.1580830000 104.1580829837 +4 9 -0.3128950000 -0.3128945611 +5 2 -0.0004780000 -0.0004775119 +5 3 3804.4172830000 3804.4172832947 +5 4 0.0021020000 0.0021023296 +5 6 0.1746190000 0.1746194143 +5 7 -0.0001970000 -0.0001974869 +5 8 0.2189750000 0.2189746196 +5 9 -0.0006580000 -0.0006578075 +6 2 -0.0027350000 -0.0027345865 +6 3 21786.9089630000 21786.9089629374 +6 4 0.0120390000 0.0120394951 +6 5 5.7267400000 5.7267400867 +6 7 -0.0011310000 -0.0011309562 +6 8 1.2540110000 1.2540107321 +6 9 -0.0037670000 -0.0037670925 +7 2 2.4179420000 2.4179420715 +7 3 -19264149.6519720000 -19264149.6519721578 +7 4 -10.6454130000 -10.6454126273 +7 5 -5063.6268890000 -5063.6268888173 +7 6 -884.2075620000 -884.2075617401 +7 8 -1108.8057720000 -1108.8057718246 +7 9 3.3308920000 3.3308917120 +8 2 -0.0021810000 -0.0021806723 +8 3 17373.7819030000 17373.7819025522 +8 4 0.0096010000 0.0096007911 +8 5 4.5667390000 4.5667392951 +8 6 0.7974410000 0.7974413411 +8 7 -0.0009020000 -0.0009018712 +8 9 -0.0030040000 -0.0030040353 +9 2 0.7259140000 0.7259143438 +9 3 -5783481.2169480000 -5783481.2169483573 +9 4 -3.1959650000 -3.1959647889 +9 5 -1520.2015940000 -1520.2015936432 +9 6 -265.4567120000 -265.4567119543 +9 7 0.3002200000 0.3002199070 +9 8 -332.8855660000 -332.8855656982 -- !query 482 @@ -4026,62 +4026,62 @@ SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected -- !query 484 schema struct -- !query 484 output -2 3 -7967167.567378 -7967167.5673775051 -2 4 -4.402675 -4.4026748005 -2 5 -2094.188669 -2094.1886691456 -2 6 -365.685999 -365.6859989148 -2 7 0.413575 0.4135748378 -2 8 -458.574167 -458.5741672173 -2 9 1.377573 1.3775729995 -3 2 0 -0.0000001255 -3 4 0.000001 0.0000005526 -3 5 0.000263 0.0002628523 -3 6 0.000046 0.0000458991 -3 7 0 -0.0000000519 -3 8 0.000058 0.000057558 -3 9 0 -0.0000001729 -4 2 -0.227135 -0.22713465 -4 3 1809619.817146 1809619.8171461717 -4 5 475.66281 475.6628104631 -4 6 83.059961 83.0599613844 -4 7 -0.093937 -0.093937176 -4 8 104.158083 104.1580829837 -4 9 -0.312895 -0.3128945611 -5 2 -0.000478 -0.0004775119 -5 3 3804.417283 3804.4172832947 -5 4 0.002102 0.0021023296 -5 6 0.174619 0.1746194143 -5 7 -0.000197 -0.0001974869 -5 8 0.218975 0.2189746196 -5 9 -0.000658 -0.0006578075 -6 2 -0.002735 -0.0027345865 -6 3 21786.908963 21786.9089629374 -6 4 0.012039 0.0120394951 -6 5 5.72674 5.7267400867 -6 7 -0.001131 -0.0011309562 -6 8 1.254011 1.2540107321 -6 9 -0.003767 -0.0037670925 -7 2 2.417942 2.4179420715 -7 3 -19264149.651972 -19264149.6519721578 -7 4 -10.645413 -10.6454126273 -7 5 -5063.626889 -5063.6268888173 -7 6 -884.207562 -884.2075617401 -7 8 -1108.805772 -1108.8057718246 -7 9 3.330892 3.330891712 -8 2 -0.002181 -0.0021806723 -8 3 17373.781903 17373.7819025522 -8 4 0.009601 0.0096007911 -8 5 4.566739 4.5667392951 -8 6 0.797441 0.7974413411 -8 7 -0.000902 -0.0009018712 -8 9 -0.003004 -0.0030040353 -9 2 0.725914 0.7259143438 -9 3 -5783481.216948 -5783481.2169483573 -9 4 -3.195965 -3.1959647889 -9 5 -1520.201594 -1520.2015936432 -9 6 -265.456712 -265.4567119543 -9 7 0.30022 0.300219907 -9 8 -332.885566 -332.8855656982 +2 3 -7967167.5673780000 -7967167.5673775051 +2 4 -4.4026750000 -4.4026748005 +2 5 -2094.1886690000 -2094.1886691456 +2 6 -365.6859990000 -365.6859989148 +2 7 0.4135750000 0.4135748378 +2 8 -458.5741670000 -458.5741672173 +2 9 1.3775730000 1.3775729995 +3 2 0.0000000000 -0.0000001255 +3 4 0.0000010000 0.0000005526 +3 5 0.0002630000 0.0002628523 +3 6 0.0000460000 0.0000458991 +3 7 0.0000000000 -0.0000000519 +3 8 0.0000580000 0.0000575580 +3 9 0.0000000000 -0.0000001729 +4 2 -0.2271350000 -0.2271346500 +4 3 1809619.8171460000 1809619.8171461717 +4 5 475.6628100000 475.6628104631 +4 6 83.0599610000 83.0599613844 +4 7 -0.0939370000 -0.0939371760 +4 8 104.1580830000 104.1580829837 +4 9 -0.3128950000 -0.3128945611 +5 2 -0.0004780000 -0.0004775119 +5 3 3804.4172830000 3804.4172832947 +5 4 0.0021020000 0.0021023296 +5 6 0.1746190000 0.1746194143 +5 7 -0.0001970000 -0.0001974869 +5 8 0.2189750000 0.2189746196 +5 9 -0.0006580000 -0.0006578075 +6 2 -0.0027350000 -0.0027345865 +6 3 21786.9089630000 21786.9089629374 +6 4 0.0120390000 0.0120394951 +6 5 5.7267400000 5.7267400867 +6 7 -0.0011310000 -0.0011309562 +6 8 1.2540110000 1.2540107321 +6 9 -0.0037670000 -0.0037670925 +7 2 2.4179420000 2.4179420715 +7 3 -19264149.6519720000 -19264149.6519721578 +7 4 -10.6454130000 -10.6454126273 +7 5 -5063.6268890000 -5063.6268888173 +7 6 -884.2075620000 -884.2075617401 +7 8 -1108.8057720000 -1108.8057718246 +7 9 3.3308920000 3.3308917120 +8 2 -0.0021810000 -0.0021806723 +8 3 17373.7819030000 17373.7819025522 +8 4 0.0096010000 0.0096007911 +8 5 4.5667390000 4.5667392951 +8 6 0.7974410000 0.7974413411 +8 7 -0.0009020000 -0.0009018712 +8 9 -0.0030040000 -0.0030040353 +9 2 0.7259140000 0.7259143438 +9 3 -5783481.2169480000 -5783481.2169483573 +9 4 -3.1959650000 -3.1959647889 +9 5 -1520.2015940000 -1520.2015936432 +9 6 -265.4567120000 -265.4567119543 +9 7 0.3002200000 0.3002199070 +9 8 -332.8855660000 -332.8855656982 -- !query 485 @@ -4196,12 +4196,12 @@ SELECT t1.id1, t1.result, t2.expected -- !query 496 schema struct -- !query 496 output -2 224790267919917440 224790267919917955.1326161858 -4 7405685069595001 7405685069594999.0773399947 -5 5068226527.321263 5068226527.3212726541 -6 281839893606.99365 281839893606.9937234336 -8 167361463828.0749 167361463828.0749132007 -9 107511333880051872 107511333880052007.0414112467 +2 224790267919917440.0000000000 224790267919917955.1326161858 +4 7405685069595001.0000000000 7405685069594999.0773399947 +5 5068226527.3212630000 5068226527.3212726541 +6 281839893606.9936500000 281839893606.9937234336 +8 167361463828.0749000000 167361463828.0749132007 +9 107511333880051872.0000000000 107511333880052007.0414112467 -- !query 497 @@ -4209,7 +4209,7 @@ SELECT AVG(val) FROM num_data -- !query 497 schema struct -- !query 497 output --13430913.5922423207 +-13430913.59224232070000 -- !query 498 @@ -4273,11 +4273,11 @@ SELECT * FROM fract_only -- !query 505 schema struct -- !query 505 output -1 0 -2 0.1 +1 0.0000 +2 0.1000 4 -0.9999 5 0.9999 -7 0 +7 0.0000 8 0.0002 @@ -4406,13 +4406,13 @@ SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round -- !query 521 schema struct -- !query 521 output --0.000001 0 0 -1 0 --5.499999 -5 -5 -6 -5 --5.5 -5 -5 -6 -6 -0 0 0 0 0 -0.0000001 1 1 0 0 -9.4999999 10 10 9 9 -9.5 10 10 9 10 +-0.000001000000000000 0 0 -1 0 +-5.499999000000000000 -5 -5 -6 -5 +-5.500000000000000000 -5 -5 -6 -6 +0.000000000000000000 0 0 0 0 +0.000000100000000000 1 1 0 0 +9.499999900000000000 10 10 9 9 +9.500000000000000000 10 10 9 10 -- !query 522 @@ -4476,11 +4476,11 @@ SELECT * FROM num_input_test -- !query 529 schema struct -- !query 529 output --555.5 --93853 -123 -3245874 -555.5 +-555.500000000000000000 +-93853.000000000000000000 +123.000000000000000000 +3245874.000000000000000000 +555.500000000000000000 -- !query 530 @@ -4488,7 +4488,7 @@ select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 -- !query 530 schema struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) / CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,6)> -- !query 530 output -1 +1.000000 -- !query 531 @@ -4536,7 +4536,7 @@ select mod (70.0,70) -- !query 536 schema struct<(CAST(70.0 AS DECIMAL(3,1)) % CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(3,1)> -- !query 536 output -0 +0.0 -- !query 537 @@ -4552,7 +4552,7 @@ select 70.0 / 70 -- !query 538 schema struct<(CAST(70.0 AS DECIMAL(3,1)) / CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(8,6)> -- !query 538 output -1 +1.000000 -- !query 539 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out index e54de1d6fdbdc..a106e33c2a595 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out @@ -336,8 +336,8 @@ VALUES (1,2), (3,4+4), (7,77.7) -- !query 17 schema struct -- !query 17 output -1 2 -3 8 +1 2.0 +3 8.0 7 77.7 @@ -350,14 +350,14 @@ TABLE int8_tbl -- !query 18 schema struct -- !query 18 output -1 2 -123 456 -123 4567890123456789 -3 8 -4 57 -4567890123456789 -4567890123456789 -4567890123456789 123 -4567890123456789 4567890123456789 +1 2.0 +123 456.0 +123 4567890123456789.0 +3 8.0 +4 57.0 +4567890123456789 -4567890123456789.0 +4567890123456789 123.0 +4567890123456789 4567890123456789.0 7 77.7 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out index 6cb3690a464e1..460b6d1fdd02f 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out @@ -258,11 +258,11 @@ SELECT '' AS `54`, d1 as `timestamp`, -- !query 27 schema struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:decimal(8,6)> -- !query 27 output - 1969-12-31 16:00:00 1969 12 31 16 0 0 - 1997-01-02 00:00:00 1997 1 2 0 0 0 - 1997-01-02 03:04:05 1997 1 2 3 4 5 - 1997-02-10 17:32:01 1997 2 10 17 32 1 - 2001-09-22 18:19:20 2001 9 22 18 19 20 + 1969-12-31 16:00:00 1969 12 31 16 0 0.000000 + 1997-01-02 00:00:00 1997 1 2 0 0 0.000000 + 1997-01-02 03:04:05 1997 1 2 3 4 5.000000 + 1997-02-10 17:32:01 1997 2 10 17 32 1.000000 + 2001-09-22 18:19:20 2001 9 22 18 19 20.000000 -- !query 28 @@ -273,11 +273,11 @@ SELECT '' AS `54`, d1 as `timestamp`, -- !query 28 schema struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> -- !query 28 output - 1969-12-31 16:00:00 4 0 0 - 1997-01-02 00:00:00 1 0 0 - 1997-01-02 03:04:05 1 5000 5000000 - 1997-02-10 17:32:01 1 1000 1000000 - 2001-09-22 18:19:20 3 20000 20000000 + 1969-12-31 16:00:00 4 0.000 0 + 1997-01-02 00:00:00 1 0.000 0 + 1997-01-02 03:04:05 1 5000.000 5000000 + 1997-02-10 17:32:01 1 1000.000 1000000 + 2001-09-22 18:19:20 3 20000.000 20000000 -- !query 29 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out index 05dedc547086e..703463e6d6b5a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out @@ -117,7 +117,7 @@ SELECT 1.1 AS two UNION SELECT 2 ORDER BY 1 struct -- !query 11 output 1.1 -2 +2.0 -- !query 12 @@ -125,7 +125,7 @@ SELECT 1 AS two UNION SELECT 2.2 ORDER BY 1 -- !query 12 schema struct -- !query 12 output -1 +1.0 2.2 @@ -143,7 +143,7 @@ SELECT 1.1 AS two UNION ALL SELECT 2 ORDER BY 1 struct -- !query 14 output 1.1 -2 +2.0 -- !query 15 @@ -161,8 +161,8 @@ SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 struct -- !query 16 output 1.1 -2 -3 +2.0 +3.0 -- !query 17 @@ -180,8 +180,8 @@ SELECT 1.1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 struct -- !query 18 output 1.1 -2 -2 +2.0 +2.0 -- !query 19 @@ -190,7 +190,7 @@ SELECT 1.1 AS two UNION (SELECT 2 UNION ALL SELECT 2) ORDER BY 1 struct -- !query 19 output 1.1 -2 +2.0 -- !query 20 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index f4ada1e18a90a..54ceacd3b3b3e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -44,8 +44,8 @@ SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWIN -- !query 3 schema struct -- !query 3 output -1 2 -2 2.5 +1 2.00000 +2 2.50000 3 NULL 4 NULL @@ -104,8 +104,8 @@ SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) -- !query 8 schema struct -- !query 8 output -3 -5 +3.00 +5.00 6.01 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out index f02f760727976..3045cff1de19b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out @@ -23,7 +23,7 @@ WHERE b = 1.0 -- Matches (null, 1.0) -- !query 1 schema struct -- !query 1 output -NULL 1 +NULL 1.0 -- !query 2 @@ -45,4 +45,4 @@ WHERE b = 5.0 -- Matches (4, 5.0) -- !query 3 schema struct -- !query 3 output -4 5 +4 5.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out index a27a66e3f27f5..5ccea3846cd36 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out @@ -37,9 +37,9 @@ WHERE (a, b) NOT IN (SELECT * -- !query 2 schema struct -- !query 2 output -2 3 -4 5 -NULL 1 +2 3.0 +4 5.0 +NULL 1.0 NULL NULL @@ -91,7 +91,7 @@ WHERE b = 1.0 -- Matches (null, 1.0) -- !query 6 schema struct -- !query 6 output -NULL 1 +NULL 1.0 -- !query 7 @@ -117,4 +117,4 @@ WHERE b = 5.0 -- Matches (4, 5.0) -- !query 8 schema struct -- !query 8 output -4 5 +4 5.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out index cf8f03eaa9311..9190e9c1e7a4b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out @@ -54,4 +54,4 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 4 schema struct -- !query 4 output -2 3 +2 3.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out index d07981cfd11e5..d7473e1ada151 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out @@ -35,9 +35,9 @@ WHERE a NOT IN (SELECT c -- !query 2 schema struct -- !query 2 output -2 3 -4 5 -NULL 1 +2 3.0 +4 5.0 +NULL 1.0 -- !query 3 @@ -88,7 +88,7 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 6 schema struct -- !query 6 output -2 3 +2 3.0 -- !query 7 @@ -100,9 +100,9 @@ WHERE a NOT IN (SELECT c -- !query 7 schema struct -- !query 7 output -2 3 -4 5 -NULL 1 +2 3.0 +4 5.0 +NULL 1.0 -- !query 8 @@ -115,7 +115,7 @@ WHERE b = 1.0 -- Only matches (null, 1.0) -- !query 8 schema struct -- !query 8 output -NULL 1 +NULL 1.0 -- !query 9 @@ -128,4 +128,4 @@ WHERE b = 3.0 -- Only matches (2, 3.0) -- !query 9 schema struct -- !query 9 output -2 3 +2 3.0 diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index 1a2bd5ea91cde..653dc1ca357f8 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -93,5 +93,5 @@ SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) struct -- !query 10 output 1 a 1 8.5 -2 b 2 1 +2 b 2 1.0 3 c 3 3.2 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index 6ee7f59d69877..e479a6d73a182 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -2211,7 +2211,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t -- !query 265 schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) / CAST(1 AS DECIMAL(3,0))):decimal(9,6)> -- !query 265 output -1 +1.000000 -- !query 266 @@ -2219,7 +2219,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t -- !query 266 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> -- !query 266 output -1 +1.000000 -- !query 267 @@ -2227,7 +2227,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t -- !query 267 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 267 output -1 +1.00000000000 -- !query 268 @@ -2235,7 +2235,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t -- !query 268 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> -- !query 268 output -1 +1.000000000000000000000 -- !query 269 @@ -2243,7 +2243,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t -- !query 269 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> -- !query 269 output -1 +1.000000 -- !query 270 @@ -2251,7 +2251,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t -- !query 270 schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) / CAST(1 AS DECIMAL(5,0))):decimal(11,6)> -- !query 270 output -1 +1.000000 -- !query 271 @@ -2259,7 +2259,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t -- !query 271 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 271 output -1 +1.00000000000 -- !query 272 @@ -2267,7 +2267,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t -- !query 272 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> -- !query 272 output -1 +1.000000000000000000000 -- !query 273 @@ -2275,7 +2275,7 @@ SELECT cast(1 as int) / cast(1 as decimal(3, 0)) FROM t -- !query 273 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 273 output -1 +1.000000 -- !query 274 @@ -2283,7 +2283,7 @@ SELECT cast(1 as int) / cast(1 as decimal(5, 0)) FROM t -- !query 274 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 274 output -1 +1.000000 -- !query 275 @@ -2291,7 +2291,7 @@ SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t -- !query 275 schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 275 output -1 +1.00000000000 -- !query 276 @@ -2299,7 +2299,7 @@ SELECT cast(1 as int) / cast(1 as decimal(20, 0)) FROM t -- !query 276 schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 276 output -1 +1.000000000000000000000 -- !query 277 @@ -2307,7 +2307,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t -- !query 277 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 277 output -1 +1.000000 -- !query 278 @@ -2315,7 +2315,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t -- !query 278 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 278 output -1 +1.000000 -- !query 279 @@ -2323,7 +2323,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t -- !query 279 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 279 output -1 +1.00000000000 -- !query 280 @@ -2331,7 +2331,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t -- !query 280 schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) / CAST(1 AS DECIMAL(20,0))):decimal(38,18)> -- !query 280 output -1 +1.000000000000000000 -- !query 281 @@ -2403,7 +2403,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t -- !query 289 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 289 output -1 +1.000000 -- !query 290 @@ -2411,7 +2411,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t -- !query 290 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 290 output -1 +1.000000 -- !query 291 @@ -2419,7 +2419,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 291 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 291 output -1 +1.00000000000 -- !query 292 @@ -2427,7 +2427,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t -- !query 292 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 292 output -1 +1.000000000000000000000 -- !query 293 @@ -2543,7 +2543,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t -- !query 305 schema struct<(CAST(1 AS DECIMAL(3,0)) / CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(9,6)> -- !query 305 output -1 +1.000000 -- !query 306 @@ -2551,7 +2551,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t -- !query 306 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> -- !query 306 output -1 +1.000000 -- !query 307 @@ -2559,7 +2559,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t -- !query 307 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 307 output -1 +1.000000 -- !query 308 @@ -2567,7 +2567,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t -- !query 308 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 308 output -1 +1.000000 -- !query 309 @@ -2575,7 +2575,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t -- !query 309 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> -- !query 309 output -1 +1.000000 -- !query 310 @@ -2583,7 +2583,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t -- !query 310 schema struct<(CAST(1 AS DECIMAL(5,0)) / CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(11,6)> -- !query 310 output -1 +1.000000 -- !query 311 @@ -2591,7 +2591,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t -- !query 311 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 311 output -1 +1.000000 -- !query 312 @@ -2599,7 +2599,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t -- !query 312 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> -- !query 312 output -1 +1.000000 -- !query 313 @@ -2607,7 +2607,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as int) FROM t -- !query 313 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 313 output -1 +1.00000000000 -- !query 314 @@ -2615,7 +2615,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as int) FROM t -- !query 314 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 314 output -1 +1.00000000000 -- !query 315 @@ -2623,7 +2623,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t -- !query 315 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> -- !query 315 output -1 +1.00000000000 -- !query 316 @@ -2631,7 +2631,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as int) FROM t -- !query 316 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 316 output -1 +1.00000000000 -- !query 317 @@ -2639,7 +2639,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t -- !query 317 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> -- !query 317 output -1 +1.000000000000000000000 -- !query 318 @@ -2647,7 +2647,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t -- !query 318 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> -- !query 318 output -1 +1.000000000000000000000 -- !query 319 @@ -2655,7 +2655,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t -- !query 319 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 319 output -1 +1.000000000000000000000 -- !query 320 @@ -2663,7 +2663,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t -- !query 320 schema struct<(CAST(1 AS DECIMAL(20,0)) / CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(38,18)> -- !query 320 output -1 +1.000000000000000000 -- !query 321 @@ -2735,7 +2735,7 @@ SELECT cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 329 schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 329 output -1 +1.00000000000 -- !query 330 @@ -2743,7 +2743,7 @@ SELECT cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 330 schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 330 output -1 +1.00000000000 -- !query 331 @@ -2751,7 +2751,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 331 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 331 output -1 +1.00000000000 -- !query 332 @@ -2759,7 +2759,7 @@ SELECT cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 332 schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 332 output -1 +1.00000000000 -- !query 333 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out index 017e0fea30e90..97cdf01763a70 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out @@ -63,7 +63,7 @@ SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t -- !query 7 schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> -- !query 7 output -1 +1.00000000000 -- !query 8 @@ -163,7 +163,7 @@ SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t -- !query 19 schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> -- !query 19 output -1 +1.00000000000 -- !query 20 @@ -263,7 +263,7 @@ SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t -- !query 31 schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 31 output -1 +1.00000000000 -- !query 32 @@ -363,7 +363,7 @@ SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t -- !query 43 schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> -- !query 43 output -1 +1.00000000000 -- !query 44 @@ -615,7 +615,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t -- !query 73 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 73 output -1 +1.000000 -- !query 74 @@ -623,7 +623,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t -- !query 74 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> -- !query 74 output -1 +1.000000 -- !query 75 @@ -631,7 +631,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t -- !query 75 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> -- !query 75 output -1 +1.00000000000 -- !query 76 @@ -639,7 +639,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t -- !query 76 schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> -- !query 76 output -1 +1.000000000000000000000 -- !query 77 @@ -663,7 +663,7 @@ SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t -- !query 79 schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> -- !query 79 output -1 +1.00000000000 -- !query 80 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out index 84b5e10dbeb8e..835abe3d32775 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out @@ -42,12 +42,12 @@ FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 -- !query 3 schema struct -- !query 3 output -1 1 -1 1 -1 a -2 4 -2 4 -2 b +1.0 1 +1.0 1 +1.0 a +2.0 4 +2.0 4 +2.0 b -- !query 4 diff --git a/sql/core/src/test/resources/sql-tests/results/union.sql.out b/sql/core/src/test/resources/sql-tests/results/union.sql.out index b023df825d814..95a10f0b0a8c9 100644 --- a/sql/core/src/test/resources/sql-tests/results/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/union.sql.out @@ -42,12 +42,12 @@ FROM (SELECT * FROM t1 -- !query 3 schema struct -- !query 3 output -1 1 -1 1 -1 a -2 4 -2 4 -2 b +1.0 1 +1.0 1 +1.0 a +2.0 4 +2.0 4 +2.0 b -- !query 4 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index 104cf4c58d617..4d1bb470e4e2e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -43,4 +43,19 @@ class HiveResultSuite extends SharedSparkSession { val tpe = new ExamplePointUDT() assert(HiveResult.toHiveString((point, tpe)) === "(50.0, 50.0)") } + + test("decimal formatting in hive result") { + val df = Seq(new java.math.BigDecimal("1")).toDS() + Seq(2, 6, 18).foreach { scala => + val executedPlan = + df.selectExpr(s"CAST(value AS decimal(38, $scala))").queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan) + assert(result.head.split("\\.").last.length === scala) + } + + val executedPlan = Seq(java.math.BigDecimal.ZERO).toDS() + .selectExpr(s"CAST(value AS decimal(38, 8))").queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan) + assert(result.head === "0.00000000") + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 04e7f579ff712..856102bc900a8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -393,4 +393,11 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { "select concat('Test4', ';');" -> "Test4;" ) } + + test("Pad Decimal numbers with trailing zeros to the scale of the column") { + runCliWithin(1.minute)( + "SELECT CAST(1 AS DECIMAL(38, 18));" + -> "1.000000000000000000" + ) + } } diff --git a/sql/hive/src/test/resources/golden/decimal_1_1-3-ac24b36077314acab595ada14e598e b/sql/hive/src/test/resources/golden/decimal_1_1-3-ac24b36077314acab595ada14e598e index 6944273be927c..f360c8c73ad1d 100644 --- a/sql/hive/src/test/resources/golden/decimal_1_1-3-ac24b36077314acab595ada14e598e +++ b/sql/hive/src/test/resources/golden/decimal_1_1-3-ac24b36077314acab595ada14e598e @@ -3,18 +3,18 @@ -0.3 -0.9 -0.9 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 0.1 0.2 0.3 diff --git a/sql/hive/src/test/resources/golden/decimal_1_1-4-128804f8dfe7dbb23be0498b91647ba3 b/sql/hive/src/test/resources/golden/decimal_1_1-4-128804f8dfe7dbb23be0498b91647ba3 index f4bf1446459a9..0fa3b15120f86 100644 --- a/sql/hive/src/test/resources/golden/decimal_1_1-4-128804f8dfe7dbb23be0498b91647ba3 +++ b/sql/hive/src/test/resources/golden/decimal_1_1-4-128804f8dfe7dbb23be0498b91647ba3 @@ -3,18 +3,18 @@ 0.3 0.2 0.1 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 +0.0 -0.1 -0.2 -0.3 diff --git a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e index f59549a6e4a46..a298a84cb2c5a 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e +++ b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e @@ -1,38 +1,38 @@ NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -1 --1.12 -1 --0.333 0 --0.33 0 --0.3 0 -0 0 -0 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 +-1234567890.1234567890000000000000000 -1234567890 +-4400.0000000000000000000000000 4400 +-1255.4900000000000000000000000 -1255 +-1.1220000000000000000000000 -11 +-1.1200000000000000000000000 -1 +-1.1200000000000000000000000 -1 +-0.3330000000000000000000000 0 +-0.3300000000000000000000000 0 +-0.3000000000000000000000000 0 +0.0000000000000000000000000 0 +0.0000000000000000000000000 0 +0.0000000000000000000000000 0 +0.0100000000000000000000000 0 +0.0200000000000000000000000 0 +0.1000000000000000000000000 0 +0.2000000000000000000000000 0 +0.3000000000000000000000000 0 +0.3300000000000000000000000 0 +0.3330000000000000000000000 0 0.9999999999999999999999999 1 -1 1 -1 1 -1.12 1 -1.122 1 -2 2 -2 2 -3.14 3 -3.14 3 -3.14 3 -3.14 4 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 +1.0000000000000000000000000 1 +1.0000000000000000000000000 1 +1.1200000000000000000000000 1 +1.1220000000000000000000000 1 +2.0000000000000000000000000 2 +2.0000000000000000000000000 2 +3.1400000000000000000000000 3 +3.1400000000000000000000000 3 +3.1400000000000000000000000 3 +3.1400000000000000000000000 4 +10.0000000000000000000000000 10 +20.0000000000000000000000000 20 +100.0000000000000000000000000 100 +124.0000000000000000000000000 124 +125.2000000000000000000000000 125 +200.0000000000000000000000000 200 +1234567890.1234567800000000000000000 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 index 6bada475c6d3d..60df68a2e3ab5 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 +++ b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 @@ -1,38 +1,38 @@ NULL NULL --1234567890.123456789 -3703703670.370370367 --4400 -13200 --1255.49 -3766.47 --1.122 -3.366 --1.12 -3.36 --1.12 -3.36 --0.333 -0.999 --0.33 -0.99 --0.3 -0.9 -0 0 -0 0 -0 0 -0.01 0.03 -0.02 0.06 -0.1 0.3 -0.2 0.6 -0.3 0.9 -0.33 0.99 -0.333 0.999 +-1234567890.1234567890000000000000000 -3703703670.3703703670000000000000000 +-4400.0000000000000000000000000 -13200.0000000000000000000000000 +-1255.4900000000000000000000000 -3766.4700000000000000000000000 +-1.1220000000000000000000000 -3.3660000000000000000000000 +-1.1200000000000000000000000 -3.3600000000000000000000000 +-1.1200000000000000000000000 -3.3600000000000000000000000 +-0.3330000000000000000000000 -0.9990000000000000000000000 +-0.3300000000000000000000000 -0.9900000000000000000000000 +-0.3000000000000000000000000 -0.9000000000000000000000000 +0.0000000000000000000000000 0.0000000000000000000000000 +0.0000000000000000000000000 0.0000000000000000000000000 +0.0000000000000000000000000 0.0000000000000000000000000 +0.0100000000000000000000000 0.0300000000000000000000000 +0.0200000000000000000000000 0.0600000000000000000000000 +0.1000000000000000000000000 0.3000000000000000000000000 +0.2000000000000000000000000 0.6000000000000000000000000 +0.3000000000000000000000000 0.9000000000000000000000000 +0.3300000000000000000000000 0.9900000000000000000000000 +0.3330000000000000000000000 0.9990000000000000000000000 0.9999999999999999999999999 2.9999999999999999999999997 -1 3 -1 3 -1.12 3.36 -1.122 3.366 -2 6 -2 6 -3.14 9.42 -3.14 9.42 -3.14 9.42 -3.14 9.42 -10 30 -20 60 -100 300 -124 372 -125.2 375.6 -200 600 -1234567890.12345678 3703703670.37037034 +1.0000000000000000000000000 3.0000000000000000000000000 +1.0000000000000000000000000 3.0000000000000000000000000 +1.1200000000000000000000000 3.3600000000000000000000000 +1.1220000000000000000000000 3.3660000000000000000000000 +2.0000000000000000000000000 6.0000000000000000000000000 +2.0000000000000000000000000 6.0000000000000000000000000 +3.1400000000000000000000000 9.4200000000000000000000000 +3.1400000000000000000000000 9.4200000000000000000000000 +3.1400000000000000000000000 9.4200000000000000000000000 +3.1400000000000000000000000 9.4200000000000000000000000 +10.0000000000000000000000000 30.0000000000000000000000000 +20.0000000000000000000000000 60.0000000000000000000000000 +100.0000000000000000000000000 300.0000000000000000000000000 +124.0000000000000000000000000 372.0000000000000000000000000 +125.2000000000000000000000000 375.6000000000000000000000000 +200.0000000000000000000000000 600.0000000000000000000000000 +1234567890.1234567800000000000000000 3703703670.3703703400000000000000000 diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 index 93cdc5c85645c..a26c8b7d12886 100644 --- a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 +++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 @@ -1,38 +1,38 @@ NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -1 --1.12 -1 --0.333 0 --0.33 0 --0.3 0 -0 0 -0 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 -1 1 -1 1 -1 1 -1.12 1 -1.122 1 -2 2 -2 2 -3.14 3 -3.14 3 -3.14 3 -3.14 4 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 +-1234567890.123456789000000000 -1234567890 +-4400.000000000000000000 4400 +-1255.490000000000000000 -1255 +-1.122000000000000000 -11 +-1.120000000000000000 -1 +-1.120000000000000000 -1 +-0.333000000000000000 0 +-0.330000000000000000 0 +-0.300000000000000000 0 +0.000000000000000000 0 +0.000000000000000000 0 +0.000000000000000000 0 +0.010000000000000000 0 +0.020000000000000000 0 +0.100000000000000000 0 +0.200000000000000000 0 +0.300000000000000000 0 +0.330000000000000000 0 +0.333000000000000000 0 +1.000000000000000000 1 +1.000000000000000000 1 +1.000000000000000000 1 +1.120000000000000000 1 +1.122000000000000000 1 +2.000000000000000000 2 +2.000000000000000000 2 +3.140000000000000000 3 +3.140000000000000000 3 +3.140000000000000000 3 +3.140000000000000000 4 +10.000000000000000000 10 +20.000000000000000000 20 +100.000000000000000000 100 +124.000000000000000000 124 +125.200000000000000000 125 +200.000000000000000000 200 +1234567890.123456780000000000 1234567890 diff --git a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 index 62d71abc6fc7d..33ea4edf780a6 100644 --- a/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 +++ b/sql/hive/src/test/resources/golden/windowing_navfn.q (deterministic)-2-1e88e0ba414a00195f7ebf6b8600ac04 @@ -3,7 +3,7 @@ 65536 32.68 65536 33.45 65536 58.86 -65536 75.7 +65536 75.70 65536 83.48 65537 NULL 65537 4.49 @@ -57,9 +57,9 @@ 65548 75.39 65548 77.24 65549 NULL -65549 13.3 +65549 13.30 65549 28.93 -65549 50.6 +65549 50.60 65549 55.04 65549 64.91 65549 76.06 @@ -70,7 +70,7 @@ 65550 33.01 65550 57.63 65550 91.38 -65550 96.9 +65550 96.90 65551 NULL 65551 39.43 65551 73.93 @@ -99,7 +99,7 @@ 65559 29.55 65559 56.06 65559 73.94 -65559 83.5 +65559 83.50 65560 NULL 65560 16.86 65560 21.81 @@ -128,7 +128,7 @@ 65565 NULL 65565 81.72 65566 NULL -65566 7.8 +65566 7.80 65567 NULL 65568 NULL 65568 21.79 @@ -136,14 +136,14 @@ 65569 NULL 65570 NULL 65570 17.09 -65570 18.2 +65570 18.20 65570 25.57 65570 45.23 -65570 76.8 +65570 76.80 65571 NULL 65571 26.64 65571 40.68 -65571 82.5 +65571 82.50 65572 NULL 65572 22.64 65572 43.49 @@ -156,9 +156,9 @@ 65574 31.28 65574 38.54 65575 NULL -65575 17 +65575 17.00 65575 32.85 -65575 83.4 +65575 83.40 65576 NULL 65576 2.04 65576 4.88 @@ -166,7 +166,7 @@ 65577 NULL 65578 NULL 65578 16.01 -65578 41.1 +65578 41.10 65578 51.36 65578 54.35 65578 58.78 @@ -188,7 +188,7 @@ 65582 NULL 65582 1.23 65582 9.35 -65582 96.6 +65582 96.60 65583 NULL 65583 28.07 65583 50.57 @@ -218,7 +218,7 @@ 65588 98.33 65589 NULL 65589 49.49 -65589 72.3 +65589 72.30 65589 74.83 65589 94.73 65590 NULL @@ -240,8 +240,8 @@ 65595 NULL 65595 8.76 65595 67.56 -65595 72.7 -65595 89.6 +65595 72.70 +65595 89.60 65595 90.24 65596 NULL 65596 12.72 @@ -252,7 +252,7 @@ 65597 37.41 65597 69.05 65598 NULL -65598 63.3 +65598 63.30 65599 NULL 65599 0.56 65599 4.93 @@ -283,7 +283,7 @@ 65605 NULL 65606 NULL 65606 7.51 -65606 24.8 +65606 24.80 65606 57.69 65606 67.94 65606 87.16 @@ -294,9 +294,9 @@ 65607 75.86 65607 91.52 65608 NULL -65608 48.9 +65608 48.90 65608 69.42 -65608 87.9 +65608 87.90 65609 NULL 65610 NULL 65610 7.59 @@ -309,7 +309,7 @@ 65611 64.89 65612 NULL 65612 16.05 -65612 25.1 +65612 25.10 65612 52.64 65613 NULL 65614 NULL @@ -317,17 +317,17 @@ 65614 94.47 65615 NULL 65615 10.79 -65615 39.4 +65615 39.40 65615 99.88 65616 NULL -65616 75.2 +65616 75.20 65617 NULL 65617 18.51 65617 47.45 -65617 64.9 +65617 64.90 65618 NULL 65618 10.06 -65618 16.6 +65618 16.60 65618 81.99 65618 88.38 65619 NULL @@ -348,20 +348,20 @@ 65622 28.37 65622 50.08 65622 74.31 -65622 88.6 -65622 93.7 +65622 88.60 +65622 93.70 65623 NULL 65623 30.83 65623 31.22 65623 39.74 65623 48.51 65623 95.58 -65623 97.2 +65623 97.20 65624 NULL 65624 58.02 65624 65.31 65624 70.08 -65624 93.3 +65624 93.30 65625 NULL 65625 20.61 65625 42.86 @@ -377,13 +377,13 @@ 65628 NULL 65628 14.83 65628 30.43 -65628 37.8 +65628 37.80 65628 74.31 65628 83.26 65629 NULL 65629 19.33 65629 58.81 -65629 72.9 +65629 72.90 65630 NULL 65630 72.13 65631 NULL @@ -412,7 +412,7 @@ 65637 48.88 65637 93.41 65638 NULL -65638 11.2 +65638 11.20 65638 19.13 65639 NULL 65640 NULL @@ -477,20 +477,20 @@ 65654 26.73 65654 29.85 65654 37.74 -65654 37.8 +65654 37.80 65654 53.55 65654 88.23 65655 NULL 65655 77.41 65656 NULL -65656 14 +65656 14.00 65656 14.96 65656 53.27 65656 64.44 65656 82.67 65657 NULL 65657 11.93 -65657 26.4 +65657 26.40 65657 64.39 65657 65.01 65658 NULL @@ -506,8 +506,8 @@ 65659 NULL 65659 8.95 65659 46.57 -65659 53.8 -65659 94.3 +65659 53.80 +65659 94.30 65659 94.69 65659 95.71 65659 99.87 @@ -517,7 +517,7 @@ 65661 NULL 65661 5.24 65661 8.06 -65661 26.8 +65661 26.80 65661 68.98 65662 NULL 65662 59.92 @@ -531,10 +531,10 @@ 65663 94.16 65664 NULL 65664 11.46 -65664 27.6 +65664 27.60 65664 34.71 65664 38.42 -65664 45.4 +65664 45.40 65664 55.82 65664 97.64 65665 NULL @@ -543,13 +543,13 @@ 65666 83.95 65667 NULL 65667 13.96 -65667 63.9 +65667 63.90 65667 97.87 65668 NULL 65669 NULL 65669 1.76 65669 16.95 -65669 38.6 +65669 38.60 65669 54.25 65669 93.79 65670 NULL @@ -561,12 +561,12 @@ 65671 8.65 65671 52.05 65672 NULL -65672 52.6 -65672 58.1 +65672 52.60 +65672 58.10 65672 64.09 65672 75.27 65673 NULL -65673 0.9 +65673 0.90 65673 33.27 65673 43.81 65673 87.78 @@ -576,7 +576,7 @@ 65675 24.19 65675 35.33 65675 35.78 -65675 79.9 +65675 79.90 65675 83.09 65675 87.36 65676 NULL @@ -591,19 +591,19 @@ 65677 87.67 65678 NULL 65678 8.72 -65678 33.9 +65678 33.90 65679 NULL 65679 64.15 65680 NULL 65680 1.01 65680 34.08 65680 54.11 -65680 55.3 +65680 55.30 65680 65.88 65681 NULL 65681 35.45 65681 41.57 -65681 61.3 +65681 61.30 65681 71.17 65681 75.85 65682 NULL @@ -641,7 +641,7 @@ 65691 28.47 65691 56.02 65691 58.01 -65691 69.8 +65691 69.80 65691 76.98 65692 NULL 65692 54.76 @@ -655,19 +655,19 @@ 65694 NULL 65694 58.23 65694 82.24 -65694 88.5 +65694 88.50 65695 NULL 65695 57.33 65695 59.96 65695 77.09 65696 NULL 65696 17.35 -65696 40.3 +65696 40.30 65696 54.02 65697 NULL 65697 3.18 65697 50.01 -65697 67.9 +65697 67.90 65697 86.79 65697 90.16 65698 NULL @@ -685,9 +685,9 @@ 65701 1.81 65701 6.35 65702 NULL -65702 37.6 +65702 37.60 65702 55.68 -65702 79.5 +65702 79.50 65703 NULL 65703 37.18 65703 40.81 @@ -708,23 +708,23 @@ 65706 55.94 65706 72.87 65707 NULL -65707 76.2 +65707 76.20 65708 NULL 65708 1.29 65709 NULL 65709 5.64 65709 49.79 65710 NULL -65710 86.7 +65710 86.70 65711 NULL 65711 8.66 65711 50.26 65711 71.89 65711 78.69 -65711 96.1 +65711 96.10 65712 NULL 65712 30.27 -65712 34.7 +65712 34.70 65712 49.69 65712 53.65 65713 NULL @@ -739,11 +739,11 @@ 65715 39.62 65715 54.79 65715 81.28 -65715 89.4 +65715 89.40 65716 NULL -65716 9 +65716 9.00 65716 10.07 -65716 33.4 +65716 33.40 65716 71.53 65716 85.93 65717 NULL @@ -758,10 +758,10 @@ 65719 NULL 65719 51.13 65719 66.85 -65719 82.1 +65719 82.10 65720 NULL 65720 2.72 -65720 18.8 +65720 18.80 65720 22.34 65720 62.04 65721 NULL @@ -775,7 +775,7 @@ 65722 1.76 65722 38.82 65723 NULL -65723 39.9 +65723 39.90 65724 NULL 65724 10.52 65724 36.05 @@ -784,7 +784,7 @@ 65724 85.52 65725 NULL 65726 NULL -65726 6 +65726 6.00 65726 60.46 65727 NULL 65727 19.81 @@ -796,7 +796,7 @@ 65729 NULL 65730 NULL 65730 1.35 -65730 30.6 +65730 30.60 65730 81.44 65731 NULL 65731 24.48 @@ -810,14 +810,14 @@ 65733 20.72 65733 88.46 65733 93.45 -65733 99.8 +65733 99.80 65734 NULL 65734 31.71 65735 NULL 65735 12.67 65735 61.16 65736 NULL -65736 28.9 +65736 28.90 65736 48.54 65736 86.51 65737 NULL @@ -828,10 +828,10 @@ 65738 NULL 65738 30.94 65738 82.32 -65738 95.1 +65738 95.10 65739 NULL 65739 74.77 -65739 92.4 +65739 92.40 65740 NULL 65740 7.49 65740 58.65 @@ -840,9 +840,9 @@ 65742 6.61 65742 43.84 65743 NULL -65743 26.6 +65743 26.60 65743 52.65 -65743 62 +65743 62.00 65744 NULL 65744 46.98 65745 NULL @@ -853,11 +853,11 @@ 65746 36.74 65746 93.21 65746 97.52 -65746 98.1 +65746 98.10 65747 NULL 65747 11.16 65747 15.07 -65747 21.8 +65747 21.80 65747 39.77 65747 52.77 65747 71.87 @@ -865,7 +865,7 @@ 65748 29.49 65749 NULL 65749 15.14 -65749 45 +65749 45.00 65749 65.49 65749 73.24 65750 NULL @@ -888,12 +888,12 @@ 65755 NULL 65755 11.23 65755 22.44 -65755 64 +65755 64.00 65755 67.54 65755 76.75 65755 81.44 65755 90.08 -65755 96.8 +65755 96.80 65756 NULL 65756 1.45 65756 11.81 @@ -907,10 +907,10 @@ 65758 25.62 65758 56.56 65758 60.88 -65758 94.9 +65758 94.90 65759 NULL 65759 10.63 -65759 14.1 +65759 14.10 65759 47.54 65759 92.81 65760 NULL @@ -920,17 +920,17 @@ 65761 NULL 65762 NULL 65762 5.49 -65762 45.7 +65762 45.70 65762 77.96 -65762 87.5 +65762 87.50 65763 NULL 65763 0.72 -65763 43.8 +65763 43.80 65763 86.43 65763 87.99 65764 NULL 65764 31.41 -65764 57.1 +65764 57.10 65765 NULL 65765 88.52 65765 88.56 @@ -938,7 +938,7 @@ 65766 37.06 65766 66.34 65766 86.53 -65766 98.9 +65766 98.90 65767 NULL 65767 90.88 65767 95.57 @@ -950,14 +950,14 @@ 65769 70.52 65769 91.49 65770 NULL -65770 51.9 +65770 51.90 65771 NULL 65771 6.15 -65771 7.5 +65771 7.50 65772 NULL 65773 NULL 65773 3.81 -65773 18.2 +65773 18.20 65773 30.49 65773 47.09 65773 53.09 @@ -966,7 +966,7 @@ 65774 NULL 65774 45.74 65774 45.97 -65774 48.8 +65774 48.80 65774 56.84 65774 94.77 65775 NULL @@ -975,7 +975,7 @@ 65775 66.68 65775 98.43 65776 NULL -65776 18.7 +65776 18.70 65776 28.47 65776 49.73 65776 98.87 @@ -993,7 +993,7 @@ 65778 95.69 65779 NULL 65779 11.87 -65779 28.2 +65779 28.20 65779 39.48 65779 45.61 65779 64.41 @@ -1008,15 +1008,15 @@ 65782 30.24 65782 34.31 65782 76.14 -65782 81.9 +65782 81.90 65783 NULL 65783 46.34 65783 51.08 65783 52.43 65783 62.58 -65783 77.4 +65783 77.40 65784 NULL -65784 15.7 +65784 15.70 65784 31.35 65784 68.18 65784 93.95 @@ -1032,7 +1032,7 @@ 65787 31.19 65787 64.88 65788 NULL -65788 16.1 +65788 16.10 65788 21.81 65788 25.77 65789 NULL @@ -1041,7 +1041,7 @@ 65789 52.49 65789 83.18 65789 92.74 -65789 96.9 +65789 96.90 65790 NULL 65790 46.91 65790 84.87 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 index 9091a9156134c..207dababa0a50 100644 --- a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 2-0-81bb7f49a55385878637c8aac4d08e5 @@ -18,12 +18,12 @@ 2013-03-01 09:11:58.703073 10.07 1 2013-03-01 09:11:58.703073 10.07 1 2013-03-01 09:11:58.703073 10.07 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 -2013-03-01 09:11:58.703074 37.8 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 +2013-03-01 09:11:58.703074 37.80 1 2013-03-01 09:11:58.703075 5.64 1 2013-03-01 09:11:58.703075 5.64 1 2013-03-01 09:11:58.703075 5.64 1 @@ -59,11 +59,11 @@ 2013-03-01 09:11:58.70308 1.76 1 2013-03-01 09:11:58.70308 1.76 1 2013-03-01 09:11:58.70308 1.76 1 -2013-03-01 09:11:58.703081 67.9 1 -2013-03-01 09:11:58.703081 67.9 1 -2013-03-01 09:11:58.703081 67.9 1 -2013-03-01 09:11:58.703081 67.9 1 -2013-03-01 09:11:58.703081 67.9 1 +2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.90 1 +2013-03-01 09:11:58.703081 67.90 1 2013-03-01 09:11:58.703082 37.25 1 2013-03-01 09:11:58.703082 37.25 1 2013-03-01 09:11:58.703082 37.25 1 @@ -148,9 +148,9 @@ 2013-03-01 09:11:58.703096 11.64 1 2013-03-01 09:11:58.703096 11.64 1 2013-03-01 09:11:58.703096 11.64 1 -2013-03-01 09:11:58.703097 0.9 1 -2013-03-01 09:11:58.703097 0.9 1 -2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.90 1 2013-03-01 09:11:58.703098 1.35 1 2013-03-01 09:11:58.703098 1.35 1 2013-03-01 09:11:58.703098 1.35 1 @@ -210,27 +210,27 @@ 2013-03-01 09:11:58.70311 8.16 1 2013-03-01 09:11:58.70311 8.16 1 2013-03-01 09:11:58.70311 8.16 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 -2013-03-01 09:11:58.703111 18.8 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 +2013-03-01 09:11:58.703111 18.80 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 2013-03-01 09:11:58.703112 13.29 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 -2013-03-01 09:11:58.703113 21.8 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 +2013-03-01 09:11:58.703113 21.80 1 2013-03-01 09:11:58.703114 73.94 1 2013-03-01 09:11:58.703114 73.94 1 2013-03-01 09:11:58.703114 73.94 1 @@ -256,14 +256,14 @@ 2013-03-01 09:11:58.703118 8.69 1 2013-03-01 09:11:58.703119 58.02 1 2013-03-01 09:11:58.703119 58.02 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.703121 96.9 1 -2013-03-01 09:11:58.703121 96.9 1 -2013-03-01 09:11:58.703121 96.9 1 -2013-03-01 09:11:58.703121 96.9 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.703121 96.90 1 +2013-03-01 09:11:58.703121 96.90 1 +2013-03-01 09:11:58.703121 96.90 1 +2013-03-01 09:11:58.703121 96.90 1 2013-03-01 09:11:58.703122 53.56 1 2013-03-01 09:11:58.703122 53.56 1 2013-03-01 09:11:58.703122 53.56 1 @@ -310,11 +310,11 @@ 2013-03-01 09:11:58.703133 27.34 1 2013-03-01 09:11:58.703133 27.34 1 2013-03-01 09:11:58.703133 27.34 1 -2013-03-01 09:11:58.703134 98.9 1 -2013-03-01 09:11:58.703134 98.9 1 -2013-03-01 09:11:58.703134 98.9 1 -2013-03-01 09:11:58.703134 98.9 1 -2013-03-01 09:11:58.703134 98.9 1 +2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.90 1 +2013-03-01 09:11:58.703134 98.90 1 2013-03-01 09:11:58.703135 29.14 1 2013-03-01 09:11:58.703135 29.14 1 2013-03-01 09:11:58.703135 29.14 1 @@ -467,12 +467,12 @@ 2013-03-01 09:11:58.703162 3.51 1 2013-03-01 09:11:58.703162 3.51 1 2013-03-01 09:11:58.703162 3.51 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 -2013-03-01 09:11:58.703163 15.7 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 +2013-03-01 09:11:58.703163 15.70 1 2013-03-01 09:11:58.703164 30.27 1 2013-03-01 09:11:58.703164 30.27 1 2013-03-01 09:11:58.703164 30.27 1 @@ -482,9 +482,9 @@ 2013-03-01 09:11:58.703165 8.38 1 2013-03-01 09:11:58.703165 8.38 1 2013-03-01 09:11:58.703165 8.38 1 -2013-03-01 09:11:58.703166 16.6 1 -2013-03-01 09:11:58.703166 16.6 1 -2013-03-01 09:11:58.703166 16.6 1 +2013-03-01 09:11:58.703166 16.60 1 +2013-03-01 09:11:58.703166 16.60 1 +2013-03-01 09:11:58.703166 16.60 1 2013-03-01 09:11:58.703167 17.66 1 2013-03-01 09:11:58.703167 17.66 1 2013-03-01 09:11:58.703167 17.66 1 @@ -537,11 +537,11 @@ 2013-03-01 09:11:58.703175 33.37 1 2013-03-01 09:11:58.703175 33.37 1 2013-03-01 09:11:58.703175 33.37 1 -2013-03-01 09:11:58.703176 28.2 1 -2013-03-01 09:11:58.703176 28.2 1 -2013-03-01 09:11:58.703176 28.2 1 -2013-03-01 09:11:58.703176 28.2 1 -2013-03-01 09:11:58.703176 28.2 1 +2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.20 1 +2013-03-01 09:11:58.703176 28.20 1 2013-03-01 09:11:58.703177 11.43 1 2013-03-01 09:11:58.703177 11.43 1 2013-03-01 09:11:58.703177 11.43 1 @@ -567,13 +567,13 @@ 2013-03-01 09:11:58.70318 10.28 1 2013-03-01 09:11:58.70318 10.28 1 2013-03-01 09:11:58.70318 10.28 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 2013-03-01 09:11:58.703182 1.23 1 2013-03-01 09:11:58.703182 1.23 1 2013-03-01 09:11:58.703182 1.23 1 @@ -647,10 +647,10 @@ 2013-03-01 09:11:58.703197 16.01 1 2013-03-01 09:11:58.703197 16.01 1 2013-03-01 09:11:58.703197 16.01 1 -2013-03-01 09:11:58.703198 30.6 1 -2013-03-01 09:11:58.703198 30.6 1 -2013-03-01 09:11:58.703198 30.6 1 -2013-03-01 09:11:58.703198 30.6 1 +2013-03-01 09:11:58.703198 30.60 1 +2013-03-01 09:11:58.703198 30.60 1 +2013-03-01 09:11:58.703198 30.60 1 +2013-03-01 09:11:58.703198 30.60 1 2013-03-01 09:11:58.703199 45.69 1 2013-03-01 09:11:58.703199 45.69 1 2013-03-01 09:11:58.703199 45.69 1 @@ -669,11 +669,11 @@ 2013-03-01 09:11:58.703203 11.63 1 2013-03-01 09:11:58.703203 11.63 1 2013-03-01 09:11:58.703203 11.63 1 -2013-03-01 09:11:58.703205 35.8 1 -2013-03-01 09:11:58.703205 35.8 1 -2013-03-01 09:11:58.703205 35.8 1 -2013-03-01 09:11:58.703205 35.8 1 -2013-03-01 09:11:58.703205 35.8 1 +2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.80 1 +2013-03-01 09:11:58.703205 35.80 1 2013-03-01 09:11:58.703206 6.61 1 2013-03-01 09:11:58.703206 6.61 1 2013-03-01 09:11:58.703206 6.61 1 @@ -824,9 +824,9 @@ 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 -2013-03-01 09:11:58.703234 44.1 1 -2013-03-01 09:11:58.703234 44.1 1 -2013-03-01 09:11:58.703234 44.1 1 +2013-03-01 09:11:58.703234 44.10 1 +2013-03-01 09:11:58.703234 44.10 1 +2013-03-01 09:11:58.703234 44.10 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 @@ -834,11 +834,11 @@ 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 2013-03-01 09:11:58.703235 6.35 1 -2013-03-01 09:11:58.703236 37.8 1 -2013-03-01 09:11:58.703236 37.8 1 -2013-03-01 09:11:58.703236 37.8 1 -2013-03-01 09:11:58.703236 37.8 1 -2013-03-01 09:11:58.703236 37.8 1 +2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.80 1 +2013-03-01 09:11:58.703236 37.80 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 @@ -847,17 +847,17 @@ 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 2013-03-01 09:11:58.703237 0.24 1 -2013-03-01 09:11:58.703238 6 1 -2013-03-01 09:11:58.703238 6 1 -2013-03-01 09:11:58.703238 6 1 -2013-03-01 09:11:58.703238 6 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.703239 24.8 1 -2013-03-01 09:11:58.70324 5.1 1 -2013-03-01 09:11:58.70324 5.1 1 +2013-03-01 09:11:58.703238 6.00 1 +2013-03-01 09:11:58.703238 6.00 1 +2013-03-01 09:11:58.703238 6.00 1 +2013-03-01 09:11:58.703238 6.00 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.703239 24.80 1 +2013-03-01 09:11:58.70324 5.10 1 +2013-03-01 09:11:58.70324 5.10 1 2013-03-01 09:11:58.703241 19.33 1 2013-03-01 09:11:58.703241 19.33 1 2013-03-01 09:11:58.703241 19.33 1 @@ -973,16 +973,16 @@ 2013-03-01 09:11:58.703262 1.81 1 2013-03-01 09:11:58.703262 1.81 1 2013-03-01 09:11:58.703262 1.81 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 @@ -1068,12 +1068,12 @@ 2013-03-01 09:11:58.703281 19.95 1 2013-03-01 09:11:58.703281 19.95 1 2013-03-01 09:11:58.703281 19.95 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 -2013-03-01 09:11:58.703282 7.5 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 +2013-03-01 09:11:58.703282 7.50 1 2013-03-01 09:11:58.703283 17.62 1 2013-03-01 09:11:58.703283 17.62 1 2013-03-01 09:11:58.703283 17.62 1 @@ -1153,12 +1153,12 @@ 2013-03-01 09:11:58.703297 25.67 1 2013-03-01 09:11:58.703297 25.67 1 2013-03-01 09:11:58.703297 25.67 1 -2013-03-01 09:11:58.703298 8.8 1 -2013-03-01 09:11:58.703298 8.8 1 -2013-03-01 09:11:58.703298 8.8 1 -2013-03-01 09:11:58.703298 8.8 1 -2013-03-01 09:11:58.703299 9 1 -2013-03-01 09:11:58.703299 9 1 +2013-03-01 09:11:58.703298 8.80 1 +2013-03-01 09:11:58.703298 8.80 1 +2013-03-01 09:11:58.703298 8.80 1 +2013-03-01 09:11:58.703298 8.80 1 +2013-03-01 09:11:58.703299 9.00 1 +2013-03-01 09:11:58.703299 9.00 1 2013-03-01 09:11:58.7033 7.51 1 2013-03-01 09:11:58.7033 7.51 1 2013-03-01 09:11:58.7033 7.51 1 @@ -1217,12 +1217,12 @@ 2013-03-01 09:11:58.703311 7.38 1 2013-03-01 09:11:58.703311 7.38 1 2013-03-01 09:11:58.703311 7.38 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 -2013-03-01 09:11:58.703312 18.2 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 +2013-03-01 09:11:58.703312 18.20 1 2013-03-01 09:11:58.703313 9.35 1 2013-03-01 09:11:58.703313 9.35 1 2013-03-01 09:11:58.703313 9.35 1 diff --git a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 index d02ca48857b5f..a1628c7e1c0c5 100644 --- a/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 +++ b/sql/hive/src/test/resources/golden/windowing_rank.q (deterministic) 4-0-12cc78f3953c3e6b5411ddc729541bf0 @@ -46,9 +46,9 @@ 2013-03-01 09:11:58.703092 54.02 1 2013-03-01 09:11:58.703092 54.02 1 2013-03-01 09:11:58.703096 87.84 1 -2013-03-01 09:11:58.703097 0.9 1 -2013-03-01 09:11:58.703097 0.9 1 -2013-03-01 09:11:58.703097 0.9 1 +2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.90 1 +2013-03-01 09:11:58.703097 0.90 1 2013-03-01 09:11:58.703098 21.29 1 2013-03-01 09:11:58.703098 21.29 1 2013-03-01 09:11:58.703098 21.29 1 @@ -88,10 +88,10 @@ 2013-03-01 09:11:58.703113 58.65 1 2013-03-01 09:11:58.703118 8.69 1 2013-03-01 09:11:58.703118 8.69 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 -2013-03-01 09:11:58.70312 52.6 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 +2013-03-01 09:11:58.70312 52.60 1 2013-03-01 09:11:58.703125 78.52 1 2013-03-01 09:11:58.703125 78.52 1 2013-03-01 09:11:58.703125 78.52 1 @@ -119,11 +119,11 @@ 2013-03-01 09:11:58.703136 27.89 1 2013-03-01 09:11:58.703136 27.89 1 2013-03-01 09:11:58.703136 27.89 1 -2013-03-01 09:11:58.703138 86.7 1 -2013-03-01 09:11:58.703138 86.7 1 -2013-03-01 09:11:58.703138 86.7 1 -2013-03-01 09:11:58.703138 86.7 1 -2013-03-01 09:11:58.703138 86.7 1 +2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.70 1 +2013-03-01 09:11:58.703138 86.70 1 2013-03-01 09:11:58.703139 43.53 1 2013-03-01 09:11:58.703139 43.53 1 2013-03-01 09:11:58.703139 43.53 1 @@ -167,13 +167,13 @@ 2013-03-01 09:11:58.703179 60.94 1 2013-03-01 09:11:58.703179 60.94 1 2013-03-01 09:11:58.703179 60.94 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 -2013-03-01 09:11:58.703181 26.6 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 +2013-03-01 09:11:58.703181 26.60 1 2013-03-01 09:11:58.703184 73.93 1 2013-03-01 09:11:58.703184 73.93 1 2013-03-01 09:11:58.703184 73.93 1 @@ -202,12 +202,12 @@ 2013-03-01 09:11:58.703189 37.74 1 2013-03-01 09:11:58.703189 37.74 1 2013-03-01 09:11:58.703189 37.74 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 -2013-03-01 09:11:58.703195 82.5 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 +2013-03-01 09:11:58.703195 82.50 1 2013-03-01 09:11:58.703198 97.18 1 2013-03-01 09:11:58.703198 97.18 1 2013-03-01 09:11:58.703198 97.18 1 @@ -233,10 +233,10 @@ 2013-03-01 09:11:58.70321 37.12 1 2013-03-01 09:11:58.70321 37.12 1 2013-03-01 09:11:58.70321 37.12 1 -2013-03-01 09:11:58.703213 48.8 1 -2013-03-01 09:11:58.703213 48.8 1 -2013-03-01 09:11:58.703213 48.8 1 -2013-03-01 09:11:58.703213 48.8 1 +2013-03-01 09:11:58.703213 48.80 1 +2013-03-01 09:11:58.703213 48.80 1 +2013-03-01 09:11:58.703213 48.80 1 +2013-03-01 09:11:58.703213 48.80 1 2013-03-01 09:11:58.703219 32.73 1 2013-03-01 09:11:58.703219 32.73 1 2013-03-01 09:11:58.703219 32.73 1 @@ -253,30 +253,30 @@ 2013-03-01 09:11:58.703221 26.64 1 2013-03-01 09:11:58.703221 26.64 1 2013-03-01 09:11:58.703221 26.64 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 -2013-03-01 09:11:58.703223 57.1 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 +2013-03-01 09:11:58.703223 57.10 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 2013-03-01 09:11:58.703224 42.93 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703226 68.3 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 -2013-03-01 09:11:58.703231 18.7 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703226 68.30 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 +2013-03-01 09:11:58.703231 18.70 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 2013-03-01 09:11:58.703233 40.81 1 @@ -295,24 +295,24 @@ 2013-03-01 09:11:58.703244 25.67 1 2013-03-01 09:11:58.703244 25.67 1 2013-03-01 09:11:58.703244 25.67 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 -2013-03-01 09:11:58.703245 32.3 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 +2013-03-01 09:11:58.703245 32.30 1 2013-03-01 09:11:58.703246 72.87 1 2013-03-01 09:11:58.703246 72.87 1 2013-03-01 09:11:58.703248 81.28 1 2013-03-01 09:11:58.703248 81.28 1 2013-03-01 09:11:58.703248 81.28 1 -2013-03-01 09:11:58.703249 93.3 1 -2013-03-01 09:11:58.703249 93.3 1 -2013-03-01 09:11:58.703249 93.3 1 -2013-03-01 09:11:58.703249 93.3 1 +2013-03-01 09:11:58.703249 93.30 1 +2013-03-01 09:11:58.703249 93.30 1 +2013-03-01 09:11:58.703249 93.30 1 +2013-03-01 09:11:58.703249 93.30 1 2013-03-01 09:11:58.70325 93.79 1 2013-03-01 09:11:58.70325 93.79 1 2013-03-01 09:11:58.70325 93.79 1 @@ -324,12 +324,12 @@ 2013-03-01 09:11:58.703254 0.32 1 2013-03-01 09:11:58.703254 0.32 1 2013-03-01 09:11:58.703254 0.32 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 -2013-03-01 09:11:58.703256 43.8 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 +2013-03-01 09:11:58.703256 43.80 1 2013-03-01 09:11:58.703258 21.21 1 2013-03-01 09:11:58.703258 21.21 1 2013-03-01 09:11:58.703258 21.21 1 @@ -342,16 +342,16 @@ 2013-03-01 09:11:58.703262 78.56 1 2013-03-01 09:11:58.703262 78.56 1 2013-03-01 09:11:58.703262 78.56 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 -2013-03-01 09:11:58.703263 14.4 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 +2013-03-01 09:11:58.703263 14.40 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 2013-03-01 09:11:58.703264 52.49 1 @@ -438,10 +438,10 @@ 2013-03-01 09:11:58.703299 23.19 1 2013-03-01 09:11:58.703299 23.19 1 2013-03-01 09:11:58.703299 23.19 1 -2013-03-01 09:11:58.703309 89.4 1 -2013-03-01 09:11:58.703309 89.4 1 -2013-03-01 09:11:58.703309 89.4 1 -2013-03-01 09:11:58.703309 89.4 1 +2013-03-01 09:11:58.703309 89.40 1 +2013-03-01 09:11:58.703309 89.40 1 +2013-03-01 09:11:58.703309 89.40 1 +2013-03-01 09:11:58.703309 89.40 1 2013-03-01 09:11:58.70331 69.26 1 2013-03-01 09:11:58.70331 69.26 1 2013-03-01 09:11:58.70331 69.26 1 From 87ebfaf003fcd05a7f6d23b3ecd4661409ce5f2f Mon Sep 17 00:00:00 2001 From: wuyi Date: Mon, 2 Dec 2019 11:34:56 +0800 Subject: [PATCH 0756/1387] [SPARK-29956][SQL] A literal number with an exponent should be parsed to Double ### What changes were proposed in this pull request? For a literal number with an exponent(e.g. 1e-45, 1E2), we'd parse it to Double by default rather than Decimal. And user could still use `spark.sql.legacy.exponentLiteralToDecimal.enabled=true` to fall back to previous behavior. ### Why are the changes needed? According to ANSI standard of SQL, we see that the (part of) definition of `literal` : ``` ::= E ``` which indicates that a literal number with an exponent should be approximate numeric(e.g. Double) rather than exact numeric(e.g. Decimal). And when we test Presto, we found that Presto also conforms to this standard: ``` presto:default> select typeof(1E2); _col0 -------- double (1 row) ``` ``` presto:default> select typeof(1.2); _col0 -------------- decimal(2,1) (1 row) ``` We also find that, actually, literals like `1E2` are parsed as Double before Spark2.1, but changed to Decimal after #14828 due to *The difference between the two confuses most users* as it said. But we also see support(from DB2 test) of original behavior at #14828 (comment). Although, we also see that PostgreSQL has its own implementation: ``` postgres=# select pg_typeof(1E2); pg_typeof ----------- numeric (1 row) postgres=# select pg_typeof(1.2); pg_typeof ----------- numeric (1 row) ``` We still think that Spark should also conform to this standard while considering SQL standard and Spark own history and majority DBMS and also user experience. ### Does this PR introduce any user-facing change? Yes. For `1E2`, before this PR: ``` scala> spark.sql("select 1E2") res0: org.apache.spark.sql.DataFrame = [1E+2: decimal(1,-2)] ``` After this PR: ``` scala> spark.sql("select 1E2") res0: org.apache.spark.sql.DataFrame = [100.0: double] ``` And for `1E-45`, before this PR: ``` org.apache.spark.sql.catalyst.parser.ParseException: decimal can only support precision up to 38 == SQL == select 1E-45 at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:131) at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48) at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76) at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605) ... 47 elided ``` after this PR: ``` scala> spark.sql("select 1E-45"); res1: org.apache.spark.sql.DataFrame = [1.0E-45: double] ``` And before this PR, user may feel super weird to see that `select 1e40` works but `select 1e-40 fails`. And now, both of them work well. ### How was this patch tested? updated `literals.sql.out` and `ansi/literals.sql.out` Closes #26595 from Ngone51/SPARK-29956. Authored-by: wuyi Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + .../spark/sql/catalyst/parser/SqlBase.g4 | 18 ++++- .../sql/catalyst/parser/AstBuilder.scala | 42 +++++++++-- .../sql/catalyst/parser/ParseDriver.scala | 2 + .../apache/spark/sql/internal/SQLConf.scala | 11 +++ .../parser/ExpressionParserSuite.scala | 31 ++++++-- .../ansi/decimalArithmeticOperations.sql | 6 +- .../inputs/decimalArithmeticOperations.sql | 16 ++-- .../subquery/in-subquery/in-group-by.sql | 74 +++++++++---------- .../inputs/subquery/in-subquery/in-limit.sql | 74 +++++++++---------- .../subquery/in-subquery/in-order-by.sql | 74 +++++++++---------- .../in-subquery/in-set-operations.sql | 74 +++++++++---------- .../inputs/subquery/in-subquery/simple-in.sql | 74 +++++++++---------- .../scalar-subquery-predicate.sql | 74 +++++++++---------- .../ansi/decimalArithmeticOperations.sql.out | 6 +- .../sql-tests/results/ansi/literals.sql.out | 22 ++++-- .../decimalArithmeticOperations.sql.out | 24 +++--- .../sql-tests/results/literals.sql.out | 22 ++++-- .../sql-tests/results/operators.sql.out | 2 +- .../results/postgreSQL/numeric.sql.out | 10 +-- .../subquery/in-subquery/in-group-by.sql.out | 74 +++++++++---------- .../subquery/in-subquery/in-limit.sql.out | 74 +++++++++---------- .../subquery/in-subquery/in-order-by.sql.out | 74 +++++++++---------- .../in-subquery/in-set-operations.sql.out | 74 +++++++++---------- .../subquery/in-subquery/simple-in.sql.out | 74 +++++++++---------- .../scalar-subquery-predicate.sql.out | 74 +++++++++---------- 26 files changed, 592 insertions(+), 510 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 12d734fcbdf52..d40fb8a181424 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -225,6 +225,8 @@ license: | - Since Spark 3.0, when casting string value to integral types, including tinyint, smallint, int and bigint type, the leading and trailing white spaces(<= ACSII 32) will be trimmed before convert to integral values, e.g. `cast(' 1 ' as int)` results `1`. In Spark version 2.4 and earlier, the result will be `null`. - Since Spark 3.0, when casting string value to date, timestamp and interval values, the leading and trailing white spaces(<= ACSII 32) will be trimmed before casing, e.g. `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, only the trailing space will be removed, thus, the result is `null`. + + - Since Spark 3.0, numbers written in scientific notation(e.g. `1E2`) would be parsed as Double. In Spark version 2.4 and earlier, they're parsed as Decimal. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.exponentLiteralAsDecimal.enabled` to `true`. - Since Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example: diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 8cff591938753..33900c9a9da38 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -23,6 +23,12 @@ grammar SqlBase; */ public boolean legacy_setops_precedence_enbled = false; + /** + * When false, a literal with an exponent would be converted into + * double type rather than decimal type. + */ + public boolean legacy_exponent_literal_as_decimal_enabled = false; + /** * Verify whether current token is a valid decimal token (which contains dot). * Returns true if the character that follows the token is not a digit or letter or underscore. @@ -948,7 +954,9 @@ quotedIdentifier ; number - : MINUS? DECIMAL_VALUE #decimalLiteral + : {!legacy_exponent_literal_as_decimal_enabled}? MINUS? EXPONENT_VALUE #exponentLiteral + | {!legacy_exponent_literal_as_decimal_enabled}? MINUS? DECIMAL_VALUE #decimalLiteral + | {legacy_exponent_literal_as_decimal_enabled}? MINUS? (EXPONENT_VALUE | DECIMAL_VALUE) #legacyDecimalLiteral | MINUS? INTEGER_VALUE #integerLiteral | MINUS? BIGINT_LITERAL #bigIntLiteral | MINUS? SMALLINT_LITERAL #smallIntLiteral @@ -1754,9 +1762,13 @@ INTEGER_VALUE : DIGIT+ ; -DECIMAL_VALUE +EXPONENT_VALUE : DIGIT+ EXPONENT - | DECIMAL_DIGITS EXPONENT? {isValidDecimal()}? + | DECIMAL_DIGITS EXPONENT {isValidDecimal()}? + ; + +DECIMAL_VALUE + : DECIMAL_DIGITS {isValidDecimal()}? ; DOUBLE_LITERAL diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 3974efea9e352..df827967e4b2a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1949,11 +1949,29 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Literal(BigDecimal(ctx.getText).underlying()) } + /** + * Create a decimal literal for a regular decimal number or a scientific decimal number. + */ + override def visitLegacyDecimalLiteral( + ctx: LegacyDecimalLiteralContext): Literal = withOrigin(ctx) { + Literal(BigDecimal(ctx.getText).underlying()) + } + + /** + * Create a double literal for number with an exponent, e.g. 1E-30 + */ + override def visitExponentLiteral(ctx: ExponentLiteralContext): Literal = { + numericLiteral(ctx, ctx.getText, /* exponent values don't have a suffix */ + Double.MinValue, Double.MaxValue, DoubleType.simpleString)(_.toDouble) + } + /** Create a numeric literal expression. */ - private def numericLiteral - (ctx: NumberContext, minValue: BigDecimal, maxValue: BigDecimal, typeName: String) - (converter: String => Any): Literal = withOrigin(ctx) { - val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + private def numericLiteral( + ctx: NumberContext, + rawStrippedQualifier: String, + minValue: BigDecimal, + maxValue: BigDecimal, + typeName: String)(converter: String => Any): Literal = withOrigin(ctx) { try { val rawBigDecimal = BigDecimal(rawStrippedQualifier) if (rawBigDecimal < minValue || rawBigDecimal > maxValue) { @@ -1971,28 +1989,36 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a Byte Literal expression. */ override def visitTinyIntLiteral(ctx: TinyIntLiteralContext): Literal = { - numericLiteral(ctx, Byte.MinValue, Byte.MaxValue, ByteType.simpleString)(_.toByte) + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Byte.MinValue, Byte.MaxValue, ByteType.simpleString)(_.toByte) } /** * Create a Short Literal expression. */ override def visitSmallIntLiteral(ctx: SmallIntLiteralContext): Literal = { - numericLiteral(ctx, Short.MinValue, Short.MaxValue, ShortType.simpleString)(_.toShort) + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Short.MinValue, Short.MaxValue, ShortType.simpleString)(_.toShort) } /** * Create a Long Literal expression. */ override def visitBigIntLiteral(ctx: BigIntLiteralContext): Literal = { - numericLiteral(ctx, Long.MinValue, Long.MaxValue, LongType.simpleString)(_.toLong) + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Long.MinValue, Long.MaxValue, LongType.simpleString)(_.toLong) } /** * Create a Double Literal expression. */ override def visitDoubleLiteral(ctx: DoubleLiteralContext): Literal = { - numericLiteral(ctx, Double.MinValue, Double.MaxValue, DoubleType.simpleString)(_.toDouble) + val rawStrippedQualifier = ctx.getText.substring(0, ctx.getText.length - 1) + numericLiteral(ctx, rawStrippedQualifier, + Double.MinValue, Double.MaxValue, DoubleType.simpleString)(_.toDouble) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 30c36598d81d6..160c7ef80cab9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -100,6 +100,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced + lexer.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled lexer.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior val tokenStream = new CommonTokenStream(lexer) @@ -108,6 +109,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced + parser.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled parser.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e9f8a60978220..dc1f65f62c71f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1943,6 +1943,14 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_EXPONENT_LITERAL_AS_DECIMAL_ENABLED = + buildConf("spark.sql.legacy.exponentLiteralAsDecimal.enabled") + .internal() + .doc("When set to true, a literal with an exponent (e.g. 1E-30) would be parsed " + + "as Decimal rather than Double.") + .booleanConf + .createWithDefault(false) + val LEGACY_INTEGRALDIVIDE_RETURN_LONG = buildConf("spark.sql.legacy.integralDivide.returnBigint") .doc("If it is set to true, the div operator returns always a bigint. This behavior was " + "inherited from Hive. Otherwise, the return type is the data type of the operands.") @@ -2557,6 +2565,9 @@ class SQLConf extends Serializable with Logging { def setOpsPrecedenceEnforced: Boolean = getConf(SQLConf.LEGACY_SETOPS_PRECEDENCE_ENABLED) + def exponentLiteralAsDecimalEnabled: Boolean = + getConf(SQLConf.LEGACY_EXPONENT_LITERAL_AS_DECIMAL_ENABLED) + def integralDivideReturnLong: Boolean = getConf(SQLConf.LEGACY_INTEGRALDIVIDE_RETURN_LONG) def nameNonStructGroupingKeyAsValue: Boolean = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 371b702722a69..6c1175658a03e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -473,13 +473,15 @@ class ExpressionParserSuite extends AnalysisTest { // Decimal testDecimal("7873247234798249279371.2334") - // Scientific Decimal - testDecimal("9.0e1") - testDecimal(".9e+2") - testDecimal("0.9e+2") - testDecimal("900e-1") - testDecimal("900.0E-1") - testDecimal("9.e+1") + // SPARK-29956: Scientific Decimal is parsed as Double by default. + assertEqual("9.0e1", Literal(90.toDouble)) + assertEqual(".9e+2", Literal(90.toDouble)) + assertEqual("0.9e+2", Literal(90.toDouble)) + + // Scientific Decimal with suffix BD should still be parsed as Decimal + assertEqual("900e-1BD", Literal(BigDecimal("900e-1").underlying())) + assertEqual("900.0E-1BD", Literal(BigDecimal("900.0E-1").underlying())) + assertEqual("9.e+1BD", Literal(BigDecimal("9.e+1").underlying())) intercept(".e3") // Tiny Int Literal @@ -508,6 +510,21 @@ class ExpressionParserSuite extends AnalysisTest { intercept("1.20E-38BD", "decimal can only support precision up to 38") } + test("SPARK-29956: scientific decimal should be parsed as Decimal in legacy mode") { + def testDecimal(value: String, parser: ParserInterface): Unit = { + assertEqual(value, Literal(BigDecimal(value).underlying), parser) + } + val conf = new SQLConf() + conf.setConf(SQLConf.LEGACY_EXPONENT_LITERAL_AS_DECIMAL_ENABLED, true) + val parser = new CatalystSqlParser(conf) + testDecimal("9e1", parser) + testDecimal("9e-1", parser) + testDecimal("-9e1", parser) + testDecimal("9.0e1", parser) + testDecimal(".9e+2", parser) + testDecimal("0.9e+2", parser) + } + test("strings") { Seq(true, false).foreach { escape => val conf = new SQLConf() diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql index 0de4e2b91e7e9..d190f38345d6b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql @@ -19,10 +19,10 @@ select 10.300000000000000000 * 3.000000000000000000; select 10.300000000000000000 * 3.0000000000000000000; -- arithmetic operations causing an overflow throw exception -select (5e36 + 0.1) + 5e36; -select (-4e36 - 0.1) - 7e36; +select (5e36BD + 0.1) + 5e36BD; +select (-4e36BD - 0.1) - 7e36BD; select 12345678901234567890.0 * 12345678901234567890.0; -select 1e35 / 0.1; +select 1e35BD / 0.1; -- arithmetic operations causing a precision loss throw exception select 123456789123456789.1234567890 * 1.123456789123456789; diff --git a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql index 28a0e20c0f495..a3bc282cd6ae8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql @@ -43,11 +43,11 @@ select 10.300000000000000000 * 3.0000000000000000000; select 2.35E10 * 1.0; -- arithmetic operations causing an overflow return NULL -select (5e36 + 0.1) + 5e36; -select (-4e36 - 0.1) - 7e36; +select (5e36BD + 0.1) + 5e36BD; +select (-4e36BD - 0.1) - 7e36BD; select 12345678901234567890.0 * 12345678901234567890.0; -select 1e35 / 0.1; -select 1.2345678901234567890E30 * 1.2345678901234567890E25; +select 1e35BD / 0.1; +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD; -- arithmetic operations causing a precision loss are truncated select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345; @@ -72,11 +72,11 @@ select 10.300000000000000000 * 3.0000000000000000000; select 2.35E10 * 1.0; -- arithmetic operations causing an overflow return NULL -select (5e36 + 0.1) + 5e36; -select (-4e36 - 0.1) - 7e36; +select (5e36BD + 0.1) + 5e36BD; +select (-4e36BD - 0.1) - 7e36BD; select 12345678901234567890.0 * 12345678901234567890.0; -select 1e35 / 0.1; -select 1.2345678901234567890E30 * 1.2345678901234567890E25; +select 1e35BD / 0.1; +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD; -- arithmetic operations causing a precision loss return NULL select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql index b06e1cccca5ab..496285e3514ea 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql @@ -7,49 +7,49 @@ --CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN create temporary view t1 as select * from values - ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("t1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("t1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); create temporary view t2 as select * from values - ("t2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("t1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("t1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("t2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("t1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("t1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("t1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("t1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); create temporary view t3 as select * from values - ("t3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("t3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("t3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("t3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("t1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("t3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); -- correlated IN subquery diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql index a40ee082ba3b9..481b5e8cc7700 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql @@ -2,49 +2,49 @@ -- It includes correlated cases. create temporary view t1 as select * from values - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); create temporary view t2 as select * from values - ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); create temporary view t3 as select * from values - ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); -- correlated IN subquery diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql index 042966b0a4e26..001c49c460b06 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql @@ -7,49 +7,49 @@ --CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN create temporary view t1 as select * from values - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); create temporary view t2 as select * from values - ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); create temporary view t3 as select * from values - ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); -- correlated IN subquery diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql index 5c371d2305ac8..b81dd7dce7ff5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql @@ -2,49 +2,49 @@ -- It includes correlated cases. create temporary view t1 as select * from values - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); create temporary view t2 as select * from values - ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); create temporary view t3 as select * from values - ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); -- correlated IN subquery diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql index f19567d2fac20..2748a959cbef8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql @@ -2,49 +2,49 @@ -- It includes correlated cases. create temporary view t1 as select * from values - ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("t1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("t1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); create temporary view t2 as select * from values - ("t2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("t1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("t1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("t2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("t1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("t1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("t1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("t1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); create temporary view t3 as select * from values - ("t3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("t3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("t3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("t3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("t1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("t3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); -- correlated IN subquery diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql index 1661209093fc4..17e44a96492b8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql @@ -22,49 +22,49 @@ AND c.cv = (SELECT max(avg) GROUP BY c1.cv)); create temporary view t1 as select * from values - ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), - ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); create temporary view t2 as select * from values - ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); create temporary view t3 as select * from values - ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); -- Group 1: scalar subquery in predicate context diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index 67cb0a2b36a92..eada0f5697bee 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -71,7 +71,7 @@ struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.00000000000000000 -- !query 8 -select (5e36 + 0.1) + 5e36 +select (5e36BD + 0.1) + 5e36BD -- !query 8 schema struct<> -- !query 8 output @@ -80,7 +80,7 @@ Decimal(expanded,10000000000000000000000000000000000000.1,39,1}) cannot be repre -- !query 9 -select (-4e36 - 0.1) - 7e36 +select (-4e36BD - 0.1) - 7e36BD -- !query 9 schema struct<> -- !query 9 output @@ -98,7 +98,7 @@ Decimal(expanded,1.5241578753238836750190519987501905210E+38,38,-1}) cannot be r -- !query 11 -select 1e35 / 0.1 +select 1e35BD / 0.1 -- !query 11 schema struct<> -- !query 11 output diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 5d8a893d334c2..e2910548aef5b 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -167,17 +167,17 @@ select 1234567890123456789012345678901234567890.0 -- !query 17 select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query 17 schema -struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> +struct<1.0:double,1.2:double,1.0E10:double,150000.0:double,0.1:double,0.1:double,10000.0:double,90.0:double,90.0:double,90.0:double,90.0:double> -- !query 17 output -1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90.0 90 +1.0 1.2 1.0E10 150000.0 0.1 0.1 10000.0 90.0 90.0 90.0 90.0 -- !query 18 select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 -- !query 18 schema -struct<-1.0:double,-1.2:double,-1E+10:decimal(1,-10),-1.5E+5:decimal(2,-4),-0.1:double,-0.1:double,-1E+4:decimal(1,-4)> +struct<-1.0:double,-1.2:double,-1.0E10:double,-150000.0:double,-0.1:double,-0.1:double,-10000.0:double> -- !query 18 output --1.0 -1.2 -10000000000 -150000 -0.1 -0.1 -10000 +-1.0 -1.2 -1.0E10 -150000.0 -0.1 -0.1 -10000.0 -- !query 19 @@ -197,9 +197,15 @@ select .e3 -- !query 20 select 1E309, -1E309 -- !query 20 schema -struct<1E+309:decimal(1,-309),-1E+309:decimal(1,-309)> +struct<> -- !query 20 output -1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 -1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal 1E309 does not fit in range [-1.7976931348623157E+308, 1.7976931348623157E+308] for type double(line 1, pos 7) + +== SQL == +select 1E309, -1E309 +-------^^^ -- !query 21 @@ -381,9 +387,9 @@ select X'XuZ' -- !query 39 SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 -- !query 39 schema -struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10),-3.14E+8:decimal(3,-6),-3.14E-8:decimal(10,10),3.14E+8:decimal(3,-6),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10)> +struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E8:double,3.14E-8:double,-3.14E8:double,-3.14E-8:double,3.14E8:double,3.14E8:double,3.14E-8:double> -- !query 39 output -3.14 -3.14 314000000 0.0000000314 -314000000 -0.0000000314 314000000 314000000 0.0000000314 +3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 -- !query 40 diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index 6f035a1904ea3..bf23833c8e3e4 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -116,13 +116,13 @@ struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.00000000000000000 -- !query 13 select 2.35E10 * 1.0 -- !query 13 schema -struct<(CAST(2.35E+10 AS DECIMAL(12,1)) * CAST(1.0 AS DECIMAL(12,1))):decimal(6,-7)> +struct<(2.35E10 * CAST(1.0 AS DOUBLE)):double> -- !query 13 output -23500000000 +2.35E10 -- !query 14 -select (5e36 + 0.1) + 5e36 +select (5e36BD + 0.1) + 5e36BD -- !query 14 schema struct<(CAST((CAST(5E+36 AS DECIMAL(38,1)) + CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) + CAST(5E+36 AS DECIMAL(38,1))):decimal(38,1)> -- !query 14 output @@ -130,7 +130,7 @@ NULL -- !query 15 -select (-4e36 - 0.1) - 7e36 +select (-4e36BD - 0.1) - 7e36BD -- !query 15 schema struct<(CAST((CAST(-4E+36 AS DECIMAL(38,1)) - CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) - CAST(7E+36 AS DECIMAL(38,1))):decimal(38,1)> -- !query 15 output @@ -146,7 +146,7 @@ NULL -- !query 17 -select 1e35 / 0.1 +select 1e35BD / 0.1 -- !query 17 schema struct<(CAST(1E+35 AS DECIMAL(37,1)) / CAST(0.1 AS DECIMAL(37,1))):decimal(38,6)> -- !query 17 output @@ -154,7 +154,7 @@ NULL -- !query 18 -select 1.2345678901234567890E30 * 1.2345678901234567890E25 +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD -- !query 18 schema struct<(CAST(1.2345678901234567890E+30 AS DECIMAL(25,-6)) * CAST(1.2345678901234567890E+25 AS DECIMAL(25,-6))):decimal(38,-17)> -- !query 18 output @@ -258,13 +258,13 @@ NULL -- !query 30 select 2.35E10 * 1.0 -- !query 30 schema -struct<(CAST(2.35E+10 AS DECIMAL(12,1)) * CAST(1.0 AS DECIMAL(12,1))):decimal(6,-7)> +struct<(2.35E10 * CAST(1.0 AS DOUBLE)):double> -- !query 30 output -23500000000 +2.35E10 -- !query 31 -select (5e36 + 0.1) + 5e36 +select (5e36BD + 0.1) + 5e36BD -- !query 31 schema struct<(CAST((CAST(5E+36 AS DECIMAL(38,1)) + CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) + CAST(5E+36 AS DECIMAL(38,1))):decimal(38,1)> -- !query 31 output @@ -272,7 +272,7 @@ NULL -- !query 32 -select (-4e36 - 0.1) - 7e36 +select (-4e36BD - 0.1) - 7e36BD -- !query 32 schema struct<(CAST((CAST(-4E+36 AS DECIMAL(38,1)) - CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) - CAST(7E+36 AS DECIMAL(38,1))):decimal(38,1)> -- !query 32 output @@ -288,7 +288,7 @@ NULL -- !query 34 -select 1e35 / 0.1 +select 1e35BD / 0.1 -- !query 34 schema struct<(CAST(1E+35 AS DECIMAL(37,1)) / CAST(0.1 AS DECIMAL(37,1))):decimal(38,3)> -- !query 34 output @@ -296,7 +296,7 @@ NULL -- !query 35 -select 1.2345678901234567890E30 * 1.2345678901234567890E25 +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD -- !query 35 schema struct<(CAST(1.2345678901234567890E+30 AS DECIMAL(25,-6)) * CAST(1.2345678901234567890E+25 AS DECIMAL(25,-6))):decimal(38,-17)> -- !query 35 output diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 5d8a893d334c2..e2910548aef5b 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -167,17 +167,17 @@ select 1234567890123456789012345678901234567890.0 -- !query 17 select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 -- !query 17 schema -struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> +struct<1.0:double,1.2:double,1.0E10:double,150000.0:double,0.1:double,0.1:double,10000.0:double,90.0:double,90.0:double,90.0:double,90.0:double> -- !query 17 output -1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90.0 90 +1.0 1.2 1.0E10 150000.0 0.1 0.1 10000.0 90.0 90.0 90.0 90.0 -- !query 18 select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 -- !query 18 schema -struct<-1.0:double,-1.2:double,-1E+10:decimal(1,-10),-1.5E+5:decimal(2,-4),-0.1:double,-0.1:double,-1E+4:decimal(1,-4)> +struct<-1.0:double,-1.2:double,-1.0E10:double,-150000.0:double,-0.1:double,-0.1:double,-10000.0:double> -- !query 18 output --1.0 -1.2 -10000000000 -150000 -0.1 -0.1 -10000 +-1.0 -1.2 -1.0E10 -150000.0 -0.1 -0.1 -10000.0 -- !query 19 @@ -197,9 +197,15 @@ select .e3 -- !query 20 select 1E309, -1E309 -- !query 20 schema -struct<1E+309:decimal(1,-309),-1E+309:decimal(1,-309)> +struct<> -- !query 20 output -1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 -1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal 1E309 does not fit in range [-1.7976931348623157E+308, 1.7976931348623157E+308] for type double(line 1, pos 7) + +== SQL == +select 1E309, -1E309 +-------^^^ -- !query 21 @@ -381,9 +387,9 @@ select X'XuZ' -- !query 39 SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 -- !query 39 schema -struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10),-3.14E+8:decimal(3,-6),-3.14E-8:decimal(10,10),3.14E+8:decimal(3,-6),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10)> +struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E8:double,3.14E-8:double,-3.14E8:double,-3.14E-8:double,3.14E8:double,3.14E8:double,3.14E-8:double> -- !query 39 output -3.14 -3.14 314000000 0.0000000314 -314000000 -0.0000000314 314000000 314000000 0.0000000314 +3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 -- !query 40 diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index e0cbd575bc346..82248ce921fd2 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -29,7 +29,7 @@ struct<-5.2:decimal(2,1)> -- !query 3 select +6.8e0 -- !query 3 schema -struct<6.8:decimal(2,1)> +struct<6.8:double> -- !query 3 output 6.8 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index e245af28508c1..74babe2580a58 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -4654,7 +4654,7 @@ struct -- !query 550 select ln(1.2345678e-28) -- !query 550 schema -struct +struct -- !query 550 output -64.26166165451762 @@ -4694,7 +4694,7 @@ struct -- !query 555 select ln(5.80397490724e5) -- !query 555 schema -struct +struct -- !query 555 output 13.271468476626518 @@ -4702,7 +4702,7 @@ struct -- !query 556 select ln(9.342536355e34) -- !query 556 schema -struct +struct -- !query 556 output 80.52247093552418 @@ -4750,7 +4750,7 @@ struct -- !query 562 select log(0.99923, 4.58934e34) -- !query 562 schema -struct +struct -- !query 562 output -103611.55579543479 @@ -4758,7 +4758,7 @@ struct -- !query 563 select log(1.000016, 8.452010e18) -- !query 563 schema -struct +struct -- !query 563 output 2723830.287707013 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out index a159aa81eff1c..6067d95490621 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out @@ -4,18 +4,18 @@ -- !query 0 create temporary view t1 as select * from values - ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("t1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("t1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) -- !query 0 schema struct<> @@ -25,19 +25,19 @@ struct<> -- !query 1 create temporary view t2 as select * from values - ("t2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("t1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("t1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("t2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("t1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("t1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("t1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("t1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) -- !query 1 schema struct<> @@ -47,18 +47,18 @@ struct<> -- !query 2 create temporary view t3 as select * from values - ("t3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("t3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("t3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("t3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("t1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("t3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) -- !query 2 schema struct<> diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out index 71ca1f8649475..42f611a4c829f 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out @@ -4,18 +4,18 @@ -- !query 0 create temporary view t1 as select * from values - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) -- !query 0 schema struct<> @@ -25,19 +25,19 @@ struct<> -- !query 1 create temporary view t2 as select * from values - ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) -- !query 1 schema struct<> @@ -47,18 +47,18 @@ struct<> -- !query 2 create temporary view t3 as select * from values - ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) -- !query 2 schema struct<> diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out index 4bebd9622c3c5..bff386cbf1715 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out @@ -4,18 +4,18 @@ -- !query 0 create temporary view t1 as select * from values - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) -- !query 0 schema struct<> @@ -25,19 +25,19 @@ struct<> -- !query 1 create temporary view t2 as select * from values - ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) -- !query 1 schema struct<> @@ -47,18 +47,18 @@ struct<> -- !query 2 create temporary view t3 as select * from values - ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) -- !query 2 schema struct<> diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out index e06f9206d3401..c4160c19ce342 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out @@ -4,18 +4,18 @@ -- !query 0 create temporary view t1 as select * from values - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) -- !query 0 schema struct<> @@ -25,19 +25,19 @@ struct<> -- !query 1 create temporary view t2 as select * from values - ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) -- !query 1 schema struct<> @@ -47,18 +47,18 @@ struct<> -- !query 2 create temporary view t3 as select * from values - ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) -- !query 2 schema struct<> diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out index d69b4bcf185c3..a07d9975fa7df 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out @@ -4,18 +4,18 @@ -- !query 0 create temporary view t1 as select * from values - ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), - ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ("t1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ("t1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) -- !query 0 schema struct<> @@ -25,19 +25,19 @@ struct<> -- !query 1 create temporary view t2 as select * from values - ("t2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ("t1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ("t1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ("t2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("t1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ("t1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ("t1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ("t1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) -- !query 1 schema struct<> @@ -47,18 +47,18 @@ struct<> -- !query 2 create temporary view t3 as select * from values - ("t3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ("t3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ("t3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ("t3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ("t1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ("t3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ("t3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) -- !query 2 schema struct<> diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index dd82efba0dde1..7af20fb6dba0b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -48,18 +48,18 @@ struct -- !query 4 create temporary view t1 as select * from values - ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), - ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), - ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), - ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), - ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), - ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), - ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), - ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), - ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) -- !query 4 schema struct<> @@ -69,19 +69,19 @@ struct<> -- !query 5 create temporary view t2 as select * from values - ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), - ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), - ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), - ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), - ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), - ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), - ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), - ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), - ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), - ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), - ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) -- !query 5 schema struct<> @@ -91,18 +91,18 @@ struct<> -- !query 6 create temporary view t3 as select * from values - ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), - ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), - ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), - ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), - ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), - ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), - ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), - ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), - ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) -- !query 6 schema struct<> From 5a1896adcb87e1611559c55fc76f32063e1c7c1b Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 2 Dec 2019 12:25:28 +0800 Subject: [PATCH 0757/1387] [SPARK-30065][SQL] DataFrameNaFunctions.drop should handle duplicate columns ### What changes were proposed in this pull request? `DataFrameNaFunctions.drop` doesn't handle duplicate columns even when column names are not specified. ```Scala val left = Seq(("1", null), ("3", "4")).toDF("col1", "col2") val right = Seq(("1", "2"), ("3", null)).toDF("col1", "col2") val df = left.join(right, Seq("col1")) df.printSchema df.na.drop("any").show ``` produces ``` root |-- col1: string (nullable = true) |-- col2: string (nullable = true) |-- col2: string (nullable = true) org.apache.spark.sql.AnalysisException: Reference 'col2' is ambiguous, could be: col2, col2.; at org.apache.spark.sql.catalyst.expressions.package$AttributeSeq.resolve(package.scala:240) ``` The reason for the above failure is that columns are resolved by name and if there are multiple columns with the same name, it will fail due to ambiguity. This PR updates `DataFrameNaFunctions.drop` such that if the columns to drop are not specified, it will resolve ambiguity gracefully by applying `drop` to all the eligible columns. (Note that if the user specifies the columns, it will still continue to fail due to ambiguity). ### Why are the changes needed? If column names are not specified, `drop` should not fail due to ambiguity since it should still be able to apply `drop` to the eligible columns. ### Does this PR introduce any user-facing change? Yes, now all the rows with nulls are dropped in the above example: ``` scala> df.na.drop("any").show +----+----+----+ |col1|col2|col2| +----+----+----+ +----+----+----+ ``` ### How was this patch tested? Added new unit tests. Closes #26700 from imback82/na_drop. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../spark/sql/DataFrameNaFunctions.scala | 32 ++++++++++++------- .../spark/sql/DataFrameNaFunctionsSuite.scala | 25 +++++++++++++-- 2 files changed, 43 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 07b0a54ba0772..8447ada88a704 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -40,7 +40,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def drop(): DataFrame = drop("any", df.columns) + def drop(): DataFrame = drop0("any", outputAttributes) /** * Returns a new `DataFrame` that drops rows containing null or NaN values. @@ -50,7 +50,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def drop(how: String): DataFrame = drop(how, df.columns) + def drop(how: String): DataFrame = drop0(how, outputAttributes) /** * Returns a new `DataFrame` that drops rows containing any null or NaN values @@ -89,11 +89,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * @since 1.3.1 */ def drop(how: String, cols: Seq[String]): DataFrame = { - how.toLowerCase(Locale.ROOT) match { - case "any" => drop(cols.size, cols) - case "all" => drop(1, cols) - case _ => throw new IllegalArgumentException(s"how ($how) must be 'any' or 'all'") - } + drop0(how, toAttributes(cols)) } /** @@ -119,10 +115,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * @since 1.3.1 */ def drop(minNonNulls: Int, cols: Seq[String]): DataFrame = { - // Filtering condition: - // only keep the row if it has at least `minNonNulls` non-null and non-NaN values. - val predicate = AtLeastNNonNulls(minNonNulls, cols.map(name => df.resolve(name))) - df.filter(Column(predicate)) + drop0(minNonNulls, toAttributes(cols)) } /** @@ -487,6 +480,23 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { df.queryExecution.analyzed.output } + private def drop0(how: String, cols: Seq[Attribute]): DataFrame = { + how.toLowerCase(Locale.ROOT) match { + case "any" => drop0(cols.size, cols) + case "all" => drop0(1, cols) + case _ => throw new IllegalArgumentException(s"how ($how) must be 'any' or 'all'") + } + } + + private def drop0(minNonNulls: Int, cols: Seq[Attribute]): DataFrame = { + // Filtering condition: + // only keep the row if it has at least `minNonNulls` non-null and non-NaN values. + val predicate = AtLeastNNonNulls( + minNonNulls, + outputAttributes.filter{ col => cols.exists(_.semanticEquals(col)) }) + df.filter(Column(predicate)) + } + /** * Returns a new `DataFrame` that replaces null or NaN values in the specified * columns. If a specified column is not a numeric, string or boolean column, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index 1afe733b855b7..6cb35656835af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -267,13 +267,14 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { assert(message.contains("Reference 'f2' is ambiguous")) } - test("fill with col(*)") { + test("fill/drop with col(*)") { val df = createDF() // If columns are specified with "*", they are ignored. checkAnswer(df.na.fill("new name", Seq("*")), df.collect()) + checkAnswer(df.na.drop("any", Seq("*")), df.collect()) } - test("fill with nested columns") { + test("fill/drop with nested columns") { val schema = new StructType() .add("c1", new StructType() .add("c1-1", StringType) @@ -290,8 +291,9 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df.select("c1.c1-1"), Row(null) :: Row("b1") :: Row(null) :: Nil) - // Nested columns are ignored for fill(). + // Nested columns are ignored for fill() and drop(). checkAnswer(df.na.fill("a1", Seq("c1.c1-1")), data) + checkAnswer(df.na.drop("any", Seq("c1.c1-1")), data) } test("replace") { @@ -385,4 +387,21 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { df.na.fill("hello"), Row("1", "hello", "2") :: Row("3", "4", "hello") :: Nil) } + + test("SPARK-30065: duplicate names are allowed for drop() if column names are not specified.") { + val left = Seq(("1", null), ("3", "4"), ("5", "6")).toDF("col1", "col2") + val right = Seq(("1", "2"), ("3", null), ("5", "6")).toDF("col1", "col2") + val df = left.join(right, Seq("col1")) + + // If column names are specified, the following fails due to ambiguity. + val exception = intercept[AnalysisException] { + df.na.drop("any", Seq("col2")) + } + assert(exception.getMessage.contains("Reference 'col2' is ambiguous")) + + // If column names are not specified, drop() is applied to all the eligible rows. + checkAnswer( + df.na.drop("any"), + Row("5", "6", "6") :: Nil) + } } From d1465a1b0dea690fcfbf75edb73ff9f8a015c0dd Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 2 Dec 2019 12:37:06 +0800 Subject: [PATCH 0758/1387] [SPARK-30074][SQL] The maxNumPostShufflePartitions config should obey reducePostShufflePartitions enabled ### What changes were proposed in this pull request? 1. Make maxNumPostShufflePartitions config obey reducePostShfflePartitions config. 2. Update the description for all the SQLConf affected by `spark.sql.adaptive.enabled`. ### Why are the changes needed? Make the relation between these confs clearer. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing UT. Closes #26664 from xuanyuanking/SPARK-9853-follow. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 49 +++++++++++-------- .../exchange/EnsureRequirements.scala | 2 +- 2 files changed, 30 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index dc1f65f62c71f..9479fc21b5994 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -353,41 +353,49 @@ object SQLConf { .booleanConf .createWithDefault(false) - val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE = - buildConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize") - .doc("The target post-shuffle input size in bytes of a task.") - .bytesConf(ByteUnit.BYTE) - .createWithDefault(64 * 1024 * 1024) + val REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED = + buildConf("spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled") + .doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is enabled, this enables reducing " + + "the number of post-shuffle partitions based on map output statistics.") + .booleanConf + .createWithDefault(true) val FETCH_SHUFFLE_BLOCKS_IN_BATCH_ENABLED = buildConf("spark.sql.adaptive.shuffle.fetchShuffleBlocksInBatch.enabled") .doc("Whether to fetch the continuous shuffle blocks in batch. Instead of fetching blocks " + "one by one, fetching continuous shuffle blocks for the same map task in batch can " + - "reduce IO and improve performance. Note, this feature also depends on a relocatable " + - "serializer and the concatenation support codec in use.") + "reduce IO and improve performance. Note, multiple continuous blocks exist in single " + + s"fetch request only happen when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " + + s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled, this feature also depends " + + "on a relocatable serializer and the concatenation support codec in use.") .booleanConf .createWithDefault(true) - val REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED = - buildConf("spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled") - .doc("When true and adaptive execution is enabled, this enables reducing the number of " + - "post-shuffle partitions based on map output statistics.") - .booleanConf - .createWithDefault(true) - val SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.shuffle.minNumPostShufflePartitions") - .doc("The advisory minimum number of post-shuffle partitions used in adaptive execution.") + .doc("The advisory minimum number of post-shuffle partitions used when " + + s"'${ADAPTIVE_EXECUTION_ENABLED.key}' and " + + s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled.") .intConf .checkValue(_ > 0, "The minimum shuffle partition number " + "must be a positive integer.") .createWithDefault(1) + val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE = + buildConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize") + .doc("The target post-shuffle input size in bytes of a task. This configuration only has " + + s"an effect when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " + + s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(64 * 1024 * 1024) + val SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions") .doc("The advisory maximum number of post-shuffle partitions used in adaptive execution. " + "This is used as the initial number of pre-shuffle partitions. By default it equals to " + - "spark.sql.shuffle.partitions") + "spark.sql.shuffle.partitions. This configuration only has an effect when " + + s"'${ADAPTIVE_EXECUTION_ENABLED.key}' and " + + s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled.") .intConf .checkValue(_ > 0, "The maximum shuffle partition number " + "must be a positive integer.") @@ -395,9 +403,9 @@ object SQLConf { val LOCAL_SHUFFLE_READER_ENABLED = buildConf("spark.sql.adaptive.shuffle.localShuffleReader.enabled") - .doc("When true and adaptive execution is enabled, this enables the optimization of" + - " converting the shuffle reader to local shuffle reader for the shuffle exchange" + - " of the broadcast hash join in probe side.") + .doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is enabled, this enables the " + + "optimization of converting the shuffle reader to local shuffle reader for the shuffle " + + "exchange of the broadcast hash join in probe side.") .booleanConf .createWithDefault(true) @@ -405,7 +413,8 @@ object SQLConf { buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") .doc("The relation with a non-empty partition ratio lower than this config will not be " + "considered as the build side of a broadcast-hash join in adaptive execution regardless " + - "of its size.") + "of its size.This configuration only has an effect when " + + s"'${ADAPTIVE_EXECUTION_ENABLED.key}' is enabled.") .doubleConf .checkValue(_ >= 0, "The non-empty partition ratio must be positive number.") .createWithDefault(0.2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 866b382a1d808..068e0164443dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.internal.SQLConf */ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { private def defaultNumPreShufflePartitions: Int = - if (conf.adaptiveExecutionEnabled) { + if (conf.adaptiveExecutionEnabled && conf.reducePostShufflePartitionsEnabled) { conf.maxNumPostShufflePartitions } else { conf.numShufflePartitions From 51e69feb495dfc63023ff673da30a3198081cfb6 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 2 Dec 2019 13:40:00 +0900 Subject: [PATCH 0759/1387] [SPARK-29851][SQL][FOLLOW-UP] Use foreach instead of misusing map ### What changes were proposed in this pull request? This PR proposes to use foreach instead of misusing map as a small followup of #26476. This could cause some weird errors potentially and it's not a good practice anyway. See also SPARK-16694 ### Why are the changes needed? To avoid potential issues like SPARK-16694 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests should cover. Closes #26729 from HyukjinKwon/SPARK-29851. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../org/apache/spark/sql/connector/InMemoryTableCatalog.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala index fd1beb49b9e07..57c83ec68a649 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala @@ -180,8 +180,8 @@ class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamesp } override def dropNamespace(namespace: Array[String]): Boolean = { - listNamespaces(namespace).map(dropNamespace) - listTables(namespace).map(dropTable) + listNamespaces(namespace).foreach(dropNamespace) + listTables(namespace).foreach(dropTable) Option(namespaces.remove(namespace.toList)).isDefined } } From 85cb388ae3f25b0e6a7fc1a2d78fd1c3ec03f341 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 2 Dec 2019 13:40:11 +0800 Subject: [PATCH 0760/1387] [SPARK-30050][SQL] analyze table and rename table should not erase hive table bucketing info ### What changes were proposed in this pull request? This patch adds Hive provider into table metadata in `HiveExternalCatalog.alterTableStats`. When we call `HiveClient.alterTable`, `alterTable` will erase if it can not find hive provider in given table metadata. Rename table also has this issue. ### Why are the changes needed? Because running `ANALYZE TABLE` on a Hive table, if the table has bucketing info, will erase existing bucket info. ### Does this PR introduce any user-facing change? Yes. After this PR, running `ANALYZE TABLE` on Hive table, won't erase existing bucketing info. ### How was this patch tested? Unit test. Closes #26685 from viirya/fix-hive-bucket. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan --- .../sql/hive/client/HiveClientImpl.scala | 4 +-- .../sql/hive/HiveExternalCatalogSuite.scala | 26 +++++++++++++++++++ 2 files changed, 28 insertions(+), 2 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 12c9a972c1aff..59f2356a3464c 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 @@ -55,7 +55,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.HiveExternalCatalog.{DATASOURCE_SCHEMA, DATASOURCE_SCHEMA_NUMPARTS, DATASOURCE_SCHEMA_PART_PREFIX} import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.client.HiveClientImpl._ @@ -1059,7 +1059,7 @@ private[hive] object HiveClientImpl { } table.bucketSpec match { - case Some(bucketSpec) if DDLUtils.isHiveTable(table) => + case Some(bucketSpec) if !HiveExternalCatalog.isDatasourceTable(table) => hiveTable.setNumBuckets(bucketSpec.numBuckets) hiveTable.setBucketCols(bucketSpec.bucketColumnNames.toList.asJava) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 46623000405cb..79e569b51ca1d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -108,6 +108,32 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { assert(bucketString.contains("10")) } + test("SPARK-30050: analyze/rename table should not erase the bucketing metadata at hive side") { + val catalog = newBasicCatalog() + externalCatalog.client.runSqlHive( + """ + |CREATE TABLE db1.t(a string, b string) + |CLUSTERED BY (a, b) SORTED BY (a, b) INTO 10 BUCKETS + |STORED AS PARQUET + """.stripMargin) + + val bucketString1 = externalCatalog.client.runSqlHive("DESC FORMATTED db1.t") + .filter(_.contains("Num Buckets")).head + assert(bucketString1.contains("10")) + + catalog.alterTableStats("db1", "t", None) + + val bucketString2 = externalCatalog.client.runSqlHive("DESC FORMATTED db1.t") + .filter(_.contains("Num Buckets")).head + assert(bucketString2.contains("10")) + + catalog.renameTable("db1", "t", "t2") + + val bucketString3 = externalCatalog.client.runSqlHive("DESC FORMATTED db1.t2") + .filter(_.contains("Num Buckets")).head + assert(bucketString3.contains("10")) + } + test("SPARK-23001: NullPointerException when running desc database") { val catalog = newBasicCatalog() catalog.createDatabase(newDb("dbWithNullDesc").copy(description = null), ignoreIfExists = false) From 03ac1b799cf1e48489e8246a1b97110c80344160 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Mon, 2 Dec 2019 14:44:31 +0800 Subject: [PATCH 0761/1387] [SPARK-29959][ML][PYSPARK] Summarizer support more metrics ### What changes were proposed in this pull request? Summarizer support more metrics: sum, std ### Why are the changes needed? Those metrics are widely used, it will be convenient to directly obtain them other than a conversion. in `NaiveBayes`: we want the sum of vectors, mean & weightSum need to computed then multiplied in `StandardScaler`,`AFTSurvivalRegression`,`LinearRegression`,`LinearSVC`,`LogisticRegression`: we need to obtain `variance` and then sqrt it to get std ### Does this PR introduce any user-facing change? yes, new metrics are exposed to end users ### How was this patch tested? added testsuites Closes #26596 from zhengruifeng/summarizer_add_metrics. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- docs/ml-statistics.md | 3 +- .../spark/ml/classification/LinearSVC.scala | 4 +- .../classification/LogisticRegression.scala | 4 +- .../spark/ml/classification/NaiveBayes.scala | 11 +-- .../spark/ml/feature/StandardScaler.scala | 8 +- .../ml/regression/AFTSurvivalRegression.scala | 4 +- .../ml/regression/LinearRegression.scala | 8 +- .../org/apache/spark/ml/stat/Summarizer.scala | 95 +++++++++++++++---- .../spark/ml/stat/SummarizerSuite.scala | 68 ++++++++++++- python/pyspark/ml/stat.py | 18 ++++ 10 files changed, 180 insertions(+), 43 deletions(-) diff --git a/docs/ml-statistics.md b/docs/ml-statistics.md index c404b628117de..997c57244b2ef 100644 --- a/docs/ml-statistics.md +++ b/docs/ml-statistics.md @@ -109,7 +109,8 @@ Refer to the [`ChiSquareTest` Python docs](api/python/index.html#pyspark.ml.stat ## Summarizer We provide vector column summary statistics for `Dataframe` through `Summarizer`. -Available metrics are the column-wise max, min, mean, variance, and number of nonzeros, as well as the total count. +Available metrics are the column-wise max, min, mean, sum, variance, std, and number of nonzeros, +as well as the total count.
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 45114f6ee80b7..4aab796d42ec3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -170,7 +170,7 @@ class LinearSVC @Since("2.2.0") ( regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth) val (summarizer, labelSummarizer) = instances.treeAggregate( - (createSummarizerBuffer("mean", "variance", "count"), new MultiClassSummarizer))( + (createSummarizerBuffer("mean", "std", "count"), new MultiClassSummarizer))( seqOp = (c: (SummarizerBuffer, MultiClassSummarizer), instance: Instance) => (c._1.add(instance.features, instance.weight), c._2.add(instance.label, instance.weight)), combOp = (c1: (SummarizerBuffer, MultiClassSummarizer), @@ -207,7 +207,7 @@ class LinearSVC @Since("2.2.0") ( throw new SparkException(msg) } - val featuresStd = summarizer.variance.toArray.map(math.sqrt) + val featuresStd = summarizer.std.toArray val getFeaturesStd = (j: Int) => featuresStd(j) val regParamL2 = $(regParam) val bcFeaturesStd = instances.context.broadcast(featuresStd) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 1cb591523968e..7f31a81940a98 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -501,7 +501,7 @@ class LogisticRegression @Since("1.2.0") ( fitIntercept) val (summarizer, labelSummarizer) = instances.treeAggregate( - (createSummarizerBuffer("mean", "variance", "count"), new MultiClassSummarizer))( + (createSummarizerBuffer("mean", "std", "count"), new MultiClassSummarizer))( seqOp = (c: (SummarizerBuffer, MultiClassSummarizer), instance: Instance) => (c._1.add(instance.features, instance.weight), c._2.add(instance.label, instance.weight)), combOp = (c1: (SummarizerBuffer, MultiClassSummarizer), @@ -582,7 +582,7 @@ class LogisticRegression @Since("1.2.0") ( } val featuresMean = summarizer.mean.toArray - val featuresStd = summarizer.variance.toArray.map(math.sqrt) + val featuresStd = summarizer.std.toArray if (!$(fitIntercept) && (0 until numFeatures).exists { i => featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 52a0f4d9b9828..7da5703f23410 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -186,16 +186,12 @@ class NaiveBayes @Since("1.5.0") ( } // Aggregates term frequencies per label. - // TODO: Summarizer directly returns sum vector. val aggregated = dataset.groupBy(col($(labelCol))) - .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "count") + .agg(sum(w).as("weightSum"), Summarizer.metrics("sum", "count") .summary(validateUDF(col($(featuresCol))), w).as("summary")) - .select($(labelCol), "weightSum", "summary.mean", "summary.count") + .select($(labelCol), "weightSum", "summary.sum", "summary.count") .as[(Double, Double, Vector, Long)] - .map { case (label, weightSum, mean, count) => - BLAS.scal(weightSum, mean) - (label, weightSum, mean, count) - }.collect().sortBy(_._1) + .collect().sortBy(_._1) val numFeatures = aggregated.head._3.size instr.logNumFeatures(numFeatures) @@ -269,7 +265,6 @@ class NaiveBayes @Since("1.5.0") ( } // Aggregates mean vector and square-sum vector per label. - // TODO: Summarizer directly returns square-sum vector. val aggregated = dataset.groupBy(col($(labelCol))) .agg(sum(w).as("weightSum"), Summarizer.metrics("mean", "normL2") .summary(col($(featuresCol)), w).as("summary")) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 41e319c1fb1fc..8d4d4197e7db2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -108,13 +108,11 @@ class StandardScaler @Since("1.4.0") ( override def fit(dataset: Dataset[_]): StandardScalerModel = { transformSchema(dataset.schema, logging = true) - val Row(mean: Vector, variance: Vector) = dataset - .select(Summarizer.metrics("mean", "variance").summary(col($(inputCol))).as("summary")) - .select("summary.mean", "summary.variance") + val Row(mean: Vector, std: Vector) = dataset + .select(Summarizer.metrics("mean", "std").summary(col($(inputCol))).as("summary")) + .select("summary.mean", "summary.std") .first() - val std = Vectors.dense(variance.toArray.map(math.sqrt)) - copyValues(new StandardScalerModel(uid, std.compressed, mean.compressed).setParent(this)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index fe89aed9a928c..faf77252cb738 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -215,13 +215,13 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) val featuresSummarizer = instances.treeAggregate( - createSummarizerBuffer("mean", "variance", "count"))( + createSummarizerBuffer("mean", "std", "count"))( seqOp = (c: SummarizerBuffer, v: AFTPoint) => c.add(v.features), combOp = (c1: SummarizerBuffer, c2: SummarizerBuffer) => c1.merge(c2), depth = $(aggregationDepth) ) - val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val featuresStd = featuresSummarizer.std.toArray val numFeatures = featuresStd.size instr.logPipelineStage(this) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index dad1080cea39c..04629436bd425 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -358,8 +358,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) val (featuresSummarizer, ySummarizer) = instances.treeAggregate( - (createSummarizerBuffer("mean", "variance"), - createSummarizerBuffer("mean", "variance", "count")))( + (createSummarizerBuffer("mean", "std"), + createSummarizerBuffer("mean", "std", "count")))( seqOp = (c: (SummarizerBuffer, SummarizerBuffer), instance: Instance) => (c._1.add(instance.features, instance.weight), c._2.add(Vectors.dense(instance.label), instance.weight)), @@ -370,7 +370,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String ) val yMean = ySummarizer.mean(0) - val rawYStd = math.sqrt(ySummarizer.variance(0)) + val rawYStd = ySummarizer.std(0) instr.logNumExamples(ySummarizer.count) instr.logNamedValue(Instrumentation.loggerTags.meanOfLabels, yMean) @@ -421,7 +421,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) val featuresMean = featuresSummarizer.mean.toArray - val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val featuresStd = featuresSummarizer.std.toArray val bcFeaturesMean = instances.context.broadcast(featuresMean) val bcFeaturesStd = instances.context.broadcast(featuresStd) diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala index 6463a7abdd0a8..044fe6e3bbad8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala @@ -89,7 +89,9 @@ object Summarizer extends Logging { * * The following metrics are accepted (case sensitive): * - mean: a vector that contains the coefficient-wise mean. + * - sum: a vector that contains the coefficient-wise sum. * - variance: a vector tha contains the coefficient-wise variance. + * - std: a vector tha contains the coefficient-wise standard deviation. * - count: the count of all vectors seen. * - numNonzeros: a vector with the number of non-zeros for each coefficients * - max: the maximum for each coefficient. @@ -106,7 +108,7 @@ object Summarizer extends Logging { @Since("2.3.0") @scala.annotation.varargs def metrics(metrics: String*): SummaryBuilder = { - require(metrics.size >= 1, "Should include at least one metric") + require(metrics.nonEmpty, "Should include at least one metric") val (typedMetrics, computeMetrics) = getRelevantMetrics(metrics) new SummaryBuilderImpl(typedMetrics, computeMetrics) } @@ -119,6 +121,14 @@ object Summarizer extends Logging { @Since("2.3.0") def mean(col: Column): Column = mean(col, lit(1.0)) + @Since("3.0.0") + def sum(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "sum") + } + + @Since("3.0.0") + def sum(col: Column): Column = sum(col, lit(1.0)) + @Since("2.3.0") def variance(col: Column, weightCol: Column): Column = { getSingleMetric(col, weightCol, "variance") @@ -127,6 +137,14 @@ object Summarizer extends Logging { @Since("2.3.0") def variance(col: Column): Column = variance(col, lit(1.0)) + @Since("3.0.0") + def std(col: Column, weightCol: Column): Column = { + getSingleMetric(col, weightCol, "std") + } + + @Since("3.0.0") + def std(col: Column): Column = std(col, lit(1.0)) + @Since("2.3.0") def count(col: Column, weightCol: Column): Column = { getSingleMetric(col, weightCol, "count") @@ -245,7 +263,9 @@ private[ml] object SummaryBuilderImpl extends Logging { */ private val allMetrics: Seq[(String, Metric, DataType, Seq[ComputeMetric])] = Seq( ("mean", Mean, vectorUDT, Seq(ComputeMean, ComputeWeightSum)), + ("sum", Sum, vectorUDT, Seq(ComputeMean, ComputeWeightSum)), ("variance", Variance, vectorUDT, Seq(ComputeWeightSum, ComputeMean, ComputeM2n)), + ("std", Std, vectorUDT, Seq(ComputeWeightSum, ComputeMean, ComputeM2n)), ("count", Count, LongType, Seq()), ("numNonZeros", NumNonZeros, vectorUDT, Seq(ComputeNNZ)), ("max", Max, vectorUDT, Seq(ComputeMax, ComputeNNZ)), @@ -259,7 +279,9 @@ private[ml] object SummaryBuilderImpl extends Logging { */ sealed trait Metric extends Serializable private[stat] case object Mean extends Metric + private[stat] case object Sum extends Metric private[stat] case object Variance extends Metric + private[stat] case object Std extends Metric private[stat] case object Count extends Metric private[stat] case object NumNonZeros extends Metric private[stat] case object Max extends Metric @@ -295,14 +317,15 @@ private[ml] object SummaryBuilderImpl extends Logging { private var totalCnt: Long = 0 private var totalWeightSum: Double = 0.0 private var weightSquareSum: Double = 0.0 - private var weightSum: Array[Double] = null + private var currWeightSum: Array[Double] = null private var nnz: Array[Long] = null private var currMax: Array[Double] = null private var currMin: Array[Double] = null def this() { this( - Seq(Mean, Variance, Count, NumNonZeros, Max, Min, NormL2, NormL1), + Seq(Mean, Sum, Variance, Std, Count, NumNonZeros, + Max, Min, NormL2, NormL1), Seq(ComputeMean, ComputeM2n, ComputeM2, ComputeL1, ComputeWeightSum, ComputeNNZ, ComputeMax, ComputeMin) ) @@ -323,7 +346,9 @@ private[ml] object SummaryBuilderImpl extends Logging { if (requestedCompMetrics.contains(ComputeM2n)) { currM2n = Array.ofDim[Double](n) } if (requestedCompMetrics.contains(ComputeM2)) { currM2 = Array.ofDim[Double](n) } if (requestedCompMetrics.contains(ComputeL1)) { currL1 = Array.ofDim[Double](n) } - if (requestedCompMetrics.contains(ComputeWeightSum)) { weightSum = Array.ofDim[Double](n) } + if (requestedCompMetrics.contains(ComputeWeightSum)) { + currWeightSum = Array.ofDim[Double](n) + } if (requestedCompMetrics.contains(ComputeNNZ)) { nnz = Array.ofDim[Long](n) } if (requestedCompMetrics.contains(ComputeMax)) { currMax = Array.fill[Double](n)(Double.MinValue) @@ -340,7 +365,7 @@ private[ml] object SummaryBuilderImpl extends Logging { val localCurrM2n = currM2n val localCurrM2 = currM2 val localCurrL1 = currL1 - val localWeightSum = weightSum + val localCurrWeightSum = currWeightSum val localNumNonzeros = nnz val localCurrMax = currMax val localCurrMin = currMin @@ -353,17 +378,18 @@ private[ml] object SummaryBuilderImpl extends Logging { localCurrMin(index) = value } - if (localWeightSum != null) { + if (localCurrWeightSum != null) { if (localCurrMean != null) { val prevMean = localCurrMean(index) val diff = value - prevMean - localCurrMean(index) = prevMean + weight * diff / (localWeightSum(index) + weight) + localCurrMean(index) = prevMean + + weight * diff / (localCurrWeightSum(index) + weight) if (localCurrM2n != null) { localCurrM2n(index) += weight * (value - localCurrMean(index)) * diff } } - localWeightSum(index) += weight + localCurrWeightSum(index) += weight } if (localCurrM2 != null) { @@ -402,9 +428,9 @@ private[ml] object SummaryBuilderImpl extends Logging { weightSquareSum += other.weightSquareSum var i = 0 while (i < n) { - if (weightSum != null) { - val thisWeightSum = weightSum(i) - val otherWeightSum = other.weightSum(i) + if (currWeightSum != null) { + val thisWeightSum = currWeightSum(i) + val otherWeightSum = other.currWeightSum(i) val totalWeightSum = thisWeightSum + otherWeightSum if (totalWeightSum != 0.0) { @@ -420,7 +446,7 @@ private[ml] object SummaryBuilderImpl extends Logging { } } } - weightSum(i) = totalWeightSum + currWeightSum(i) = totalWeightSum } // merge m2 together @@ -442,7 +468,7 @@ private[ml] object SummaryBuilderImpl extends Logging { this.totalCnt = other.totalCnt this.totalWeightSum = other.totalWeightSum this.weightSquareSum = other.weightSquareSum - if (other.weightSum != null) { this.weightSum = other.weightSum.clone() } + if (other.currWeightSum != null) { this.currWeightSum = other.currWeightSum.clone() } if (other.nnz != null) { this.nnz = other.nnz.clone() } if (other.currMax != null) { this.currMax = other.currMax.clone() } if (other.currMin != null) { this.currMin = other.currMin.clone() } @@ -460,12 +486,28 @@ private[ml] object SummaryBuilderImpl extends Logging { val realMean = Array.ofDim[Double](n) var i = 0 while (i < n) { - realMean(i) = currMean(i) * (weightSum(i) / totalWeightSum) + realMean(i) = currMean(i) * (currWeightSum(i) / totalWeightSum) i += 1 } Vectors.dense(realMean) } + /** + * Sum of each dimension. + */ + def sum: Vector = { + require(requestedMetrics.contains(Sum)) + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") + + val realSum = Array.ofDim[Double](n) + var i = 0 + while (i < n) { + realSum(i) = currMean(i) * currWeightSum(i) + i += 1 + } + Vectors.dense(realSum) + } + /** * Unbiased estimate of sample variance of each dimension. */ @@ -473,8 +515,23 @@ private[ml] object SummaryBuilderImpl extends Logging { require(requestedMetrics.contains(Variance)) require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") - val realVariance = Array.ofDim[Double](n) + val realVariance = computeVariance + Vectors.dense(realVariance) + } + + /** + * Unbiased estimate of standard deviation of each dimension. + */ + def std: Vector = { + require(requestedMetrics.contains(Std)) + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") + val realVariance = computeVariance + Vectors.dense(realVariance.map(math.sqrt)) + } + + private def computeVariance: Array[Double] = { + val realVariance = Array.ofDim[Double](n) val denominator = totalWeightSum - (weightSquareSum / totalWeightSum) // Sample variance is computed, if the denominator is less than 0, the variance is just 0. @@ -484,12 +541,12 @@ private[ml] object SummaryBuilderImpl extends Logging { val len = currM2n.length while (i < len) { // We prevent variance from negative value caused by numerical error. - realVariance(i) = math.max((currM2n(i) + deltaMean(i) * deltaMean(i) * weightSum(i) * - (totalWeightSum - weightSum(i)) / totalWeightSum) / denominator, 0.0) + realVariance(i) = math.max((currM2n(i) + deltaMean(i) * deltaMean(i) * currWeightSum(i) * + (totalWeightSum - currWeightSum(i)) / totalWeightSum) / denominator, 0.0) i += 1 } } - Vectors.dense(realVariance) + realVariance } /** @@ -579,7 +636,9 @@ private[ml] object SummaryBuilderImpl extends Logging { override def eval(state: SummarizerBuffer): Any = { val metrics = requestedMetrics.map { case Mean => vectorUDT.serialize(state.mean) + case Sum => vectorUDT.serialize(state.sum) case Variance => vectorUDT.serialize(state.variance) + case Std => vectorUDT.serialize(state.std) case Count => state.count case NumNonZeros => vectorUDT.serialize(state.numNonzeros) case Max => vectorUDT.serialize(state.max) diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala index 5e4f402989697..68ba57c0d5fc8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.stat import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, Statistics} @@ -83,6 +83,28 @@ class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { Row(Row(summarizerWithoutWeight.mean), expWithoutWeight.mean)) } + registerTest(s"$name - sum only") { + val (df, c, w) = wrappedInit() + val weightSum = summarizer.weightSum + val expected1 = summarizer.mean.asML.copy + BLAS.scal(weightSum, expected1) + val expected2 = exp.mean.copy + BLAS.scal(weightSum, expected2) + compareRow(df.select(metrics("sum").summary(c, w), sum(c, w)).first(), + Row(Row(expected1), expected2)) + } + + registerTest(s"$name - sum only w/o weight") { + val (df, c, _) = wrappedInit() + val weightSum = summarizerWithoutWeight.weightSum + val expected1 = summarizerWithoutWeight.mean.asML.copy + BLAS.scal(weightSum, expected1) + val expected2 = expWithoutWeight.mean.copy + BLAS.scal(weightSum, expected2) + compareRow(df.select(metrics("sum").summary(c), sum(c)).first(), + Row(Row(expected1), expected2)) + } + registerTest(s"$name - variance only") { val (df, c, w) = wrappedInit() compareRow(df.select(metrics("variance").summary(c, w), variance(c, w)).first(), @@ -95,6 +117,22 @@ class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { Row(Row(summarizerWithoutWeight.variance), expWithoutWeight.variance)) } + registerTest(s"$name - std only") { + val (df, c, w) = wrappedInit() + val expected1 = Vectors.dense(summarizer.variance.toArray.map(math.sqrt)) + val expected2 = Vectors.dense(exp.variance.toArray.map(math.sqrt)) + compareRow(df.select(metrics("std").summary(c, w), std(c, w)).first(), + Row(Row(expected1), expected2)) + } + + registerTest(s"$name - std only w/o weight") { + val (df, c, _) = wrappedInit() + val expected1 = Vectors.dense(summarizerWithoutWeight.variance.toArray.map(math.sqrt)) + val expected2 = Vectors.dense(expWithoutWeight.variance.toArray.map(math.sqrt)) + compareRow(df.select(metrics("std").summary(c), std(c)).first(), + Row(Row(expected1), expected2)) + } + registerTest(s"$name - count only") { val (df, c, w) = wrappedInit() compareRow(df.select(metrics("count").summary(c, w), count(c, w)).first(), @@ -192,8 +230,12 @@ class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { assert(v1 ~== v2 absTol 1e-4) case (v1: Vector, v2: OldVector) => assert(v1 ~== v2.asML absTol 1e-4) + case (i1: Int, i2: Int) => + assert(i1 === i2) case (l1: Long, l2: Long) => assert(l1 === l2) + case (d1: Double, d2: Double) => + assert(d1 ~== d2 absTol 1e-4) case (r1: Row, r2: Row) => compareRow(r1, r2) case (x1: Any, x2: Any) => @@ -531,6 +573,30 @@ class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { assert(summarizer3.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) } + test("support new metrics: sum, std, numFeatures, sumL2, weightSum") { + val summarizer1 = new SummarizerBuffer() + .add(Vectors.dense(10.0, -10.0), 1e10) + .add(Vectors.dense(0.0, 0.0), 1e-7) + + val summarizer2 = new SummarizerBuffer() + summarizer2.add(Vectors.dense(10.0, -10.0), 1e10) + for (i <- 1 to 100) { + summarizer2.add(Vectors.dense(0.0, 0.0), 1e-7) + } + + val summarizer3 = new SummarizerBuffer() + for (i <- 1 to 100) { + summarizer3.add(Vectors.dense(0.0, 0.0), 1e-7) + } + summarizer3.add(Vectors.dense(10.0, -10.0), 1e10) + + Seq(summarizer1, summarizer2, summarizer3).foreach { summarizer => + val variance = summarizer.variance + val expectedStd = Vectors.dense(variance.toArray.map(math.sqrt)) + assert(summarizer.std ~== expectedStd relTol 1e-14) + } + } + ignore("performance test") { /* Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.12 diff --git a/python/pyspark/ml/stat.py b/python/pyspark/ml/stat.py index a84b67476609b..8f2eadda2946e 100644 --- a/python/pyspark/ml/stat.py +++ b/python/pyspark/ml/stat.py @@ -243,6 +243,14 @@ def mean(col, weightCol=None): """ return Summarizer._get_single_metric(col, weightCol, "mean") + @staticmethod + @since("3.0.0") + def sum(col, weightCol=None): + """ + return a column of sum summary + """ + return Summarizer._get_single_metric(col, weightCol, "sum") + @staticmethod @since("2.4.0") def variance(col, weightCol=None): @@ -251,6 +259,14 @@ def variance(col, weightCol=None): """ return Summarizer._get_single_metric(col, weightCol, "variance") + @staticmethod + @since("3.0.0") + def std(col, weightCol=None): + """ + return a column of std summary + """ + return Summarizer._get_single_metric(col, weightCol, "std") + @staticmethod @since("2.4.0") def count(col, weightCol=None): @@ -323,7 +339,9 @@ def metrics(*metrics): The following metrics are accepted (case sensitive): - mean: a vector that contains the coefficient-wise mean. + - sum: a vector that contains the coefficient-wise sum. - variance: a vector tha contains the coefficient-wise variance. + - std: a vector tha contains the coefficient-wise standard deviation. - count: the count of all vectors seen. - numNonzeros: a vector with the number of non-zeros for each coefficients - max: the maximum for each coefficient. From 169415ffac3050a86934011525ea00eef7fca35c Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 2 Dec 2019 15:59:12 +0800 Subject: [PATCH 0762/1387] [SPARK-30025][CORE] Continuous shuffle block fetching should be disabled by default when the old fetch protocol is used ### What changes were proposed in this pull request? Disable continuous shuffle block fetching when the old fetch protocol in use. ### Why are the changes needed? The new feature of continuous shuffle block fetching depends on the latest version of the shuffle fetch protocol. We should keep this constraint in `BlockStoreShuffleReader.fetchContinuousBlocksInBatch`. ### Does this PR introduce any user-facing change? Users will not get the exception related to continuous shuffle block fetching when old version of the external shuffle service is used. ### How was this patch tested? Existing UT. Closes #26663 from xuanyuanking/SPARK-30025. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../apache/spark/network/netty/NettyBlockRpcServer.scala | 8 ++++++-- .../apache/spark/shuffle/BlockStoreShuffleReader.scala | 6 ++++-- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 3 ++- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 91910b936e7cf..b3904f3362e8e 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -56,8 +56,12 @@ class NettyBlockRpcServer( message match { case openBlocks: OpenBlocks => val blocksNum = openBlocks.blockIds.length - val blocks = for (i <- (0 until blocksNum).view) - yield blockManager.getLocalBlockData(BlockId.apply(openBlocks.blockIds(i))) + val blocks = (0 until blocksNum).map { i => + val blockId = BlockId.apply(openBlocks.blockIds(i)) + assert(!blockId.isInstanceOf[ShuffleBlockBatchId], + "Continuous shuffle block fetching only works for new fetch protocol.") + blockManager.getLocalBlockData(blockId) + } val streamId = streamManager.registerStream(appId, blocks.iterator.asJava, client.getChannel) logTrace(s"Registered streamId $streamId with $blocksNum buffers") diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index d5a66db23301a..bc2a0fbc36d5b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -50,14 +50,16 @@ private[spark] class BlockStoreShuffleReader[K, C]( } else { true } + val useOldFetchProtocol = conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) val doBatchFetch = shouldBatchFetch && serializerRelocatable && - (!compressed || codecConcatenation) + (!compressed || codecConcatenation) && !useOldFetchProtocol if (shouldBatchFetch && !doBatchFetch) { logDebug("The feature tag of continuous shuffle block fetching is set to true, but " + "we can not enable the feature because other conditions are not satisfied. " + s"Shuffle compress: $compressed, serializer relocatable: $serializerRelocatable, " + - s"codec concatenation: $codecConcatenation.") + s"codec concatenation: $codecConcatenation, use old shuffle fetch protocol: " + + s"$useOldFetchProtocol.") } doBatchFetch } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9479fc21b5994..105b2a857e5af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -367,7 +367,8 @@ object SQLConf { "reduce IO and improve performance. Note, multiple continuous blocks exist in single " + s"fetch request only happen when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " + s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled, this feature also depends " + - "on a relocatable serializer and the concatenation support codec in use.") + "on a relocatable serializer, the concatenation support codec in use and the new version" + + "shuffle fetch protocol.") .booleanConf .createWithDefault(true) From 04a5b8f5f80ee746bdc16267e44a993a9941d335 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Mon, 2 Dec 2019 16:11:58 +0800 Subject: [PATCH 0763/1387] [SPARK-29839][SQL] Supporting STORED AS in CREATE TABLE LIKE ### What changes were proposed in this pull request? In SPARK-29421 (#26097) , we can specify a different table provider for `CREATE TABLE LIKE` via `USING provider`. Hive support `STORED AS` new file format syntax: ```sql CREATE TABLE tbl(a int) STORED AS TEXTFILE; CREATE TABLE tbl2 LIKE tbl STORED AS PARQUET; ``` For Hive compatibility, we should also support `STORED AS` in `CREATE TABLE LIKE`. ### Why are the changes needed? See https://github.com/apache/spark/pull/26097#issue-327424759 ### Does this PR introduce any user-facing change? Add a new syntax based on current CTL: CREATE TABLE tbl2 LIKE tbl [STORED AS hiveFormat]; ### How was this patch tested? Add UTs. Closes #26466 from LantaoJin/SPARK-29839. Authored-by: LantaoJin Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 1 + .../spark/sql/catalyst/parser/SqlBase.g4 | 7 +- .../spark/sql/execution/SparkSqlParser.scala | 43 +++++++- .../spark/sql/execution/command/tables.scala | 41 +++++-- .../execution/command/DDLParserSuite.scala | 54 +++++----- .../sql/hive/execution/HiveDDLSuite.scala | 100 ++++++++++++++++++ 6 files changed, 207 insertions(+), 39 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index d40fb8a181424..3ea4839a8175a 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -878,6 +878,7 @@ Spark SQL supports the vast majority of Hive features, such as: * All Hive DDL Functions, including: * `CREATE TABLE` * `CREATE TABLE AS SELECT` + * `CREATE TABLE LIKE` * `ALTER TABLE` * Most Hive Data types, including: * `TINYINT` diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 33900c9a9da38..abaaecf69f619 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -121,7 +121,12 @@ statement (TBLPROPERTIES tableProps=tablePropertyList))* (AS? query)? #createHiveTable | CREATE TABLE (IF NOT EXISTS)? target=tableIdentifier - LIKE source=tableIdentifier tableProvider? locationSpec? #createTableLike + LIKE source=tableIdentifier + (tableProvider | + rowFormat | + createFileFormat | + locationSpec | + (TBLPROPERTIES tableProps=tablePropertyList))* #createTableLike | replaceTableHeader ('(' colTypeList ')')? tableProvider ((OPTIONS options=tablePropertyList) | (PARTITIONED BY partitioning=transformList) | diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 128ba82dd471b..b150b40049964 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -540,15 +540,50 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * For example: * {{{ * CREATE TABLE [IF NOT EXISTS] [db_name.]table_name - * LIKE [other_db_name.]existing_table_name [USING provider] [locationSpec] + * LIKE [other_db_name.]existing_table_name + * [USING provider | + * [ + * [ROW FORMAT row_format] + * [STORED AS file_format] [WITH SERDEPROPERTIES (...)] + * ] + * ] + * [locationSpec] + * [TBLPROPERTIES (property_name=property_value, ...)] * }}} */ override def visitCreateTableLike(ctx: CreateTableLikeContext): LogicalPlan = withOrigin(ctx) { val targetTable = visitTableIdentifier(ctx.target) val sourceTable = visitTableIdentifier(ctx.source) - val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText) - val location = Option(ctx.locationSpec).map(visitLocationSpec) - CreateTableLikeCommand(targetTable, sourceTable, provider, location, ctx.EXISTS != null) + checkDuplicateClauses(ctx.tableProvider, "PROVIDER", ctx) + checkDuplicateClauses(ctx.createFileFormat, "STORED AS/BY", ctx) + checkDuplicateClauses(ctx.rowFormat, "ROW FORMAT", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) + val provider = ctx.tableProvider.asScala.headOption.map(_.multipartIdentifier.getText) + val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) + // rowStorage used to determine CatalogStorageFormat.serde and + // CatalogStorageFormat.properties in STORED AS clause. + val rowStorage = ctx.rowFormat.asScala.headOption.map(visitRowFormat) + .getOrElse(CatalogStorageFormat.empty) + val fileFormat = ctx.createFileFormat.asScala.headOption.map(visitCreateFileFormat) match { + case Some(f) => + if (provider.isDefined) { + throw new ParseException("'STORED AS hiveFormats' and 'USING provider' " + + "should not be specified both", ctx) + } + f.copy( + locationUri = location.map(CatalogUtils.stringToURI), + serde = rowStorage.serde.orElse(f.serde), + properties = rowStorage.properties ++ f.properties) + case None => + if (rowStorage.serde.isDefined) { + throw new ParseException("'ROW FORMAT' must be used with 'STORED AS'", ctx) + } + CatalogStorageFormat.empty.copy(locationUri = location.map(CatalogUtils.stringToURI)) + } + val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) + CreateTableLikeCommand( + targetTable, sourceTable, fileFormat, provider, properties, ctx.EXISTS != null) } /** 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 44ce746b143c0..71500c304bd48 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 @@ -53,28 +53,37 @@ import org.apache.spark.sql.util.SchemaUtils * are identical to the ones defined in the source table. * * The CatalogTable attributes copied from the source table are storage(inputFormat, outputFormat, - * serde, compressed, properties), schema, provider, partitionColumnNames, bucketSpec. + * serde, compressed, properties), schema, provider, partitionColumnNames, bucketSpec by default. * - * Use "CREATE TABLE t1 LIKE t2 USING file_format" - * to specify new file format for t1 from a data source table t2. + * Use "CREATE TABLE t1 LIKE t2 USING file_format" to specify new provider for t1. + * For Hive compatibility, use "CREATE TABLE t1 LIKE t2 STORED AS hiveFormat" + * to specify new file storage format (inputFormat, outputFormat, serde) for t1. * * The syntax of using this command in SQL is: * {{{ * CREATE TABLE [IF NOT EXISTS] [db_name.]table_name - * LIKE [other_db_name.]existing_table_name [USING provider] [locationSpec] + * LIKE [other_db_name.]existing_table_name + * [USING provider | + * [ + * [ROW FORMAT row_format] + * [STORED AS file_format] [WITH SERDEPROPERTIES (...)] + * ] + * ] + * [locationSpec] + * [TBLPROPERTIES (property_name=property_value, ...)] * }}} */ case class CreateTableLikeCommand( targetTable: TableIdentifier, sourceTable: TableIdentifier, + fileFormat: CatalogStorageFormat, provider: Option[String], - location: Option[String], + properties: Map[String, String] = Map.empty, ifNotExists: Boolean) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val sourceTableDesc = catalog.getTempViewOrPermanentTableMetadata(sourceTable) - val newProvider = if (provider.isDefined) { if (!DDLUtils.isHiveTable(provider)) { // check the validation of provider input, invalid provider will throw @@ -84,24 +93,36 @@ case class CreateTableLikeCommand( provider } else if (sourceTableDesc.tableType == CatalogTableType.VIEW) { Some(sparkSession.sessionState.conf.defaultDataSourceName) + } else if (fileFormat.inputFormat.isDefined) { + Some(DDLUtils.HIVE_PROVIDER) } else { sourceTableDesc.provider } + val newStorage = if (fileFormat.inputFormat.isDefined) { + fileFormat + } else { + sourceTableDesc.storage.copy(locationUri = fileFormat.locationUri) + } + // If the location is specified, we create an external table internally. // Otherwise create a managed table. - val tblType = if (location.isEmpty) CatalogTableType.MANAGED else CatalogTableType.EXTERNAL + val tblType = if (newStorage.locationUri.isEmpty) { + CatalogTableType.MANAGED + } else { + CatalogTableType.EXTERNAL + } val newTableDesc = CatalogTable( identifier = targetTable, tableType = tblType, - storage = sourceTableDesc.storage.copy( - locationUri = location.map(CatalogUtils.stringToURI(_))), + storage = newStorage, schema = sourceTableDesc.schema, provider = newProvider, partitionColumnNames = sourceTableDesc.partitionColumnNames, - bucketSpec = sourceTableDesc.bucketSpec) + bucketSpec = sourceTableDesc.bucketSpec, + properties = properties) catalog.createTable(newTableDesc, ifNotExists) Seq.empty[Row] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 813cae29d2683..6b829171dd6e2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -974,75 +974,81 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { test("create table like") { val v1 = "CREATE TABLE table1 LIKE table2" - val (target, source, provider, location, exists) = parser.parsePlan(v1).collect { - case CreateTableLikeCommand(t, s, p, l, allowExisting) => (t, s, p, l, allowExisting) - }.head + val (target, source, fileFormat, provider, properties, exists) = + parser.parsePlan(v1).collect { + case CreateTableLikeCommand(t, s, f, p, pr, e) => (t, s, f, p, pr, e) + }.head assert(exists == false) assert(target.database.isEmpty) assert(target.table == "table1") assert(source.database.isEmpty) assert(source.table == "table2") - assert(location.isEmpty) + assert(fileFormat.locationUri.isEmpty) assert(provider.isEmpty) val v2 = "CREATE TABLE IF NOT EXISTS table1 LIKE table2" - val (target2, source2, provider2, location2, exists2) = parser.parsePlan(v2).collect { - case CreateTableLikeCommand(t, s, p, l, allowExisting) => (t, s, p, l, allowExisting) - }.head + val (target2, source2, fileFormat2, provider2, properties2, exists2) = + parser.parsePlan(v2).collect { + case CreateTableLikeCommand(t, s, f, p, pr, e) => (t, s, f, p, pr, e) + }.head assert(exists2) assert(target2.database.isEmpty) assert(target2.table == "table1") assert(source2.database.isEmpty) assert(source2.table == "table2") - assert(location2.isEmpty) + assert(fileFormat2.locationUri.isEmpty) assert(provider2.isEmpty) val v3 = "CREATE TABLE table1 LIKE table2 LOCATION '/spark/warehouse'" - val (target3, source3, provider3, location3, exists3) = parser.parsePlan(v3).collect { - case CreateTableLikeCommand(t, s, p, l, allowExisting) => (t, s, p, l, allowExisting) - }.head + val (target3, source3, fileFormat3, provider3, properties3, exists3) = + parser.parsePlan(v3).collect { + case CreateTableLikeCommand(t, s, f, p, pr, e) => (t, s, f, p, pr, e) + }.head assert(!exists3) assert(target3.database.isEmpty) assert(target3.table == "table1") assert(source3.database.isEmpty) assert(source3.table == "table2") - assert(location3 == Some("/spark/warehouse")) + assert(fileFormat3.locationUri.map(_.toString) == Some("/spark/warehouse")) assert(provider3.isEmpty) val v4 = "CREATE TABLE IF NOT EXISTS table1 LIKE table2 LOCATION '/spark/warehouse'" - val (target4, source4, provider4, location4, exists4) = parser.parsePlan(v4).collect { - case CreateTableLikeCommand(t, s, p, l, allowExisting) => (t, s, p, l, allowExisting) - }.head + val (target4, source4, fileFormat4, provider4, properties4, exists4) = + parser.parsePlan(v4).collect { + case CreateTableLikeCommand(t, s, f, p, pr, e) => (t, s, f, p, pr, e) + }.head assert(exists4) assert(target4.database.isEmpty) assert(target4.table == "table1") assert(source4.database.isEmpty) assert(source4.table == "table2") - assert(location4 == Some("/spark/warehouse")) + assert(fileFormat4.locationUri.map(_.toString) == Some("/spark/warehouse")) assert(provider4.isEmpty) val v5 = "CREATE TABLE IF NOT EXISTS table1 LIKE table2 USING parquet" - val (target5, source5, provider5, location5, exists5) = parser.parsePlan(v5).collect { - case CreateTableLikeCommand(t, s, p, l, allowExisting) => (t, s, p, l, allowExisting) - }.head + val (target5, source5, fileFormat5, provider5, properties5, exists5) = + parser.parsePlan(v5).collect { + case CreateTableLikeCommand(t, s, f, p, pr, e) => (t, s, f, p, pr, e) + }.head assert(exists5) assert(target5.database.isEmpty) assert(target5.table == "table1") assert(source5.database.isEmpty) assert(source5.table == "table2") - assert(location5.isEmpty) + assert(fileFormat5.locationUri.isEmpty) assert(provider5 == Some("parquet")) val v6 = "CREATE TABLE IF NOT EXISTS table1 LIKE table2 USING ORC" - val (target6, source6, provider6, location6, exists6) = parser.parsePlan(v6).collect { - case CreateTableLikeCommand(t, s, p, l, allowExisting) => (t, s, p, l, allowExisting) - }.head + val (target6, source6, fileFormat6, provider6, properties6, exists6) = + parser.parsePlan(v6).collect { + case CreateTableLikeCommand(t, s, f, p, pr, e) => (t, s, f, p, pr, e) + }.head assert(exists6) assert(target6.database.isEmpty) assert(target6.table == "table1") assert(source6.database.isEmpty) assert(source6.table == "table2") - assert(location6.isEmpty) + assert(fileFormat6.locationUri.isEmpty) assert(provider6 == Some("ORC")) } } 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 21d995b3e427f..7a7618d4125ad 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 @@ -2564,4 +2564,104 @@ class HiveDDLSuite } } } + + test("Create Table LIKE STORED AS Hive Format") { + val catalog = spark.sessionState.catalog + withTable("s") { + sql("CREATE TABLE s(a INT, b INT) STORED AS ORC") + hiveFormats.foreach { tableType => + val expectedSerde = HiveSerDe.sourceToSerDe(tableType) + withTable("t") { + sql(s"CREATE TABLE t LIKE s STORED AS $tableType") + val table = catalog.getTableMetadata(TableIdentifier("t")) + assert(table.provider == Some("hive")) + assert(table.storage.serde == expectedSerde.get.serde) + assert(table.storage.inputFormat == expectedSerde.get.inputFormat) + assert(table.storage.outputFormat == expectedSerde.get.outputFormat) + } + } + } + } + + test("Create Table LIKE with specified TBLPROPERTIES") { + val catalog = spark.sessionState.catalog + withTable("s", "t") { + sql("CREATE TABLE s(a INT, b INT) USING hive TBLPROPERTIES('a'='apple')") + val source = catalog.getTableMetadata(TableIdentifier("s")) + assert(source.properties("a") == "apple") + sql("CREATE TABLE t LIKE s STORED AS parquet TBLPROPERTIES('f'='foo', 'b'='bar')") + val table = catalog.getTableMetadata(TableIdentifier("t")) + assert(table.properties.get("a") === None) + assert(table.properties("f") == "foo") + assert(table.properties("b") == "bar") + } + } + + test("Create Table LIKE with row format") { + val catalog = spark.sessionState.catalog + withTable("sourceHiveTable", "sourceDsTable", "targetHiveTable1", "targetHiveTable2") { + sql("CREATE TABLE sourceHiveTable(a INT, b INT) STORED AS PARQUET") + sql("CREATE TABLE sourceDsTable(a INT, b INT) USING PARQUET") + + // row format doesn't work in create targetDsTable + var e = intercept[AnalysisException] { + spark.sql( + """ + |CREATE TABLE targetDsTable LIKE sourceHiveTable USING PARQUET + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + """.stripMargin) + }.getMessage + assert(e.contains("'ROW FORMAT' must be used with 'STORED AS'")) + + // row format doesn't work with provider hive + e = intercept[AnalysisException] { + spark.sql( + """ + |CREATE TABLE targetHiveTable LIKE sourceHiveTable USING hive + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ('test' = 'test') + """.stripMargin) + }.getMessage + assert(e.contains("'ROW FORMAT' must be used with 'STORED AS'")) + + // row format doesn't work without 'STORED AS' + e = intercept[AnalysisException] { + spark.sql( + """ + |CREATE TABLE targetDsTable LIKE sourceDsTable + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ('test' = 'test') + """.stripMargin) + }.getMessage + assert(e.contains("'ROW FORMAT' must be used with 'STORED AS'")) + + // row format works with STORED AS hive format (from hive table) + spark.sql( + """ + |CREATE TABLE targetHiveTable1 LIKE sourceHiveTable STORED AS PARQUET + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ('test' = 'test') + """.stripMargin) + var table = catalog.getTableMetadata(TableIdentifier("targetHiveTable1")) + assert(table.provider === Some("hive")) + assert(table.storage.inputFormat === + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(table.storage.serde === Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + assert(table.storage.properties("test") == "test") + + // row format works with STORED AS hive format (from datasource table) + spark.sql( + """ + |CREATE TABLE targetHiveTable2 LIKE sourceDsTable STORED AS PARQUET + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ('test' = 'test') + """.stripMargin) + table = catalog.getTableMetadata(TableIdentifier("targetHiveTable2")) + assert(table.provider === Some("hive")) + assert(table.storage.inputFormat === + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(table.storage.serde === Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + assert(table.storage.properties("test") == "test") + } + } } From 4e073f3c5093e136518e456d0a3a7437ad9867a3 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 2 Dec 2019 20:47:23 +0800 Subject: [PATCH 0764/1387] [SPARK-30047][SQL] Support interval types in UnsafeRow ### What changes were proposed in this pull request? Optimize aggregates on interval values from sort-based to hash-based, and we can use the `org.apache.spark.sql.catalyst.expressions.RowBasedKeyValueBatch` for better performance. ### Why are the changes needed? improve aggerates ### Does this PR introduce any user-facing change? no ### How was this patch tested? add ut and existing ones Closes #26680 from yaooqinn/SPARK-30047. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/UnsafeRow.java | 23 +++++++++++++- .../expressions/codegen/UnsafeWriter.java | 16 ++++++---- .../spark/sql/catalyst/InternalRow.scala | 4 ++- .../expressions/codegen/CodeGenerator.scala | 9 ++++-- .../codegen/GenerateUnsafeProjection.scala | 1 + .../expressions/UnsafeRowConverterSuite.scala | 30 +++++++++++++++++++ .../codegen/UnsafeRowWriterSuite.scala | 12 ++++++++ .../vectorized/MutableColumnarRow.java | 8 +++++ .../vectorized/WritableColumnVector.java | 7 +++++ .../aggregate/HashAggregateExec.scala | 7 +++-- .../aggregate/RowBasedHashMapGenerator.scala | 3 +- .../spark/sql/DataFrameAggregateSuite.scala | 17 +++++++++-- .../vectorized/ColumnarBatchSuite.scala | 30 +++++-------------- 13 files changed, 128 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index dc6fa30e14f4d..23e7d1f07e4a3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -103,7 +103,8 @@ public static boolean isFixedLength(DataType dt) { } public static boolean isMutable(DataType dt) { - return mutableFieldTypes.contains(dt) || dt instanceof DecimalType; + return mutableFieldTypes.contains(dt) || dt instanceof DecimalType || + dt instanceof CalendarIntervalType; } ////////////////////////////////////////////////////////////////////////////// @@ -297,6 +298,26 @@ public void setDecimal(int ordinal, Decimal value, int precision) { } } + @Override + public void setInterval(int ordinal, CalendarInterval value) { + assertIndexIsValid(ordinal); + long cursor = getLong(ordinal) >>> 32; + assert cursor > 0 : "invalid cursor " + cursor; + if (value == null) { + setNullAt(ordinal); + // zero-out the bytes + Platform.putLong(baseObject, baseOffset + cursor, 0L); + Platform.putLong(baseObject, baseOffset + cursor + 8, 0L); + // keep the offset for future update + Platform.putLong(baseObject, getFieldOffset(ordinal), (cursor << 32) | 16L); + } else { + Platform.putInt(baseObject, baseOffset + cursor, value.months); + Platform.putInt(baseObject, baseOffset + cursor + 4, value.days); + Platform.putLong(baseObject, baseOffset + cursor + 8, value.microseconds); + setLong(ordinal, (cursor << 32) | 16L); + } + } + @Override public Object get(int ordinal, DataType dataType) { return SpecializedGettersReader.read(this, ordinal, dataType, true, true); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java index 4e4392734184a..84b2b29479414 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java @@ -22,6 +22,7 @@ import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; +import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -134,13 +135,16 @@ public final void write(int ordinal, CalendarInterval input) { // grow the global buffer before writing data. grow(16); - // Write the months, days and microseconds fields of Interval to the variable length portion. - Platform.putInt(getBuffer(), cursor(), input.months); - Platform.putInt(getBuffer(), cursor() + 4, input.days); - Platform.putLong(getBuffer(), cursor() + 8, input.microseconds); - + if (input == null) { + BitSetMethods.set(getBuffer(), startingOffset, ordinal); + } else { + // Write the months, days and microseconds fields of interval to the variable length portion. + Platform.putInt(getBuffer(), cursor(), input.months); + Platform.putInt(getBuffer(), cursor() + 4, input.days); + Platform.putLong(getBuffer(), cursor() + 8, input.microseconds); + } + // we need to reserve the space so that we can update it later. setOffsetAndSize(ordinal, 16); - // move the cursor forward. increaseCursor(16); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala index 1bb13e3368c02..57cdd75887153 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** * An abstract class for row used internally in Spark SQL, which only contains the columns as @@ -58,6 +58,8 @@ abstract class InternalRow extends SpecializedGetters with Serializable { */ def setDecimal(i: Int, value: Decimal, precision: Int): Unit = update(i, value) + def setInterval(i: Int, value: CalendarInterval): Unit = update(i, value) + /** * Make a copy of the current [[InternalRow]] object. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index acd3858431e62..7b8e92d665d0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -605,6 +605,7 @@ class CodegenContext extends Logging { s"((java.lang.Double.isNaN($c1) && java.lang.Double.isNaN($c2)) || $c1 == $c2)" case dt: DataType if isPrimitiveType(dt) => s"$c1 == $c2" case dt: DataType if dt.isInstanceOf[AtomicType] => s"$c1.equals($c2)" + case CalendarIntervalType => s"$c1.equals($c2)" case array: ArrayType => genComp(array, c1, c2) + " == 0" case struct: StructType => genComp(struct, c1, c2) + " == 0" case udt: UserDefinedType[_] => genEqual(udt.sqlType, c1, c2) @@ -1579,6 +1580,7 @@ object CodeGenerator extends Logging { val jt = javaType(dataType) dataType match { case _ if isPrimitiveType(jt) => s"$row.set${primitiveTypeName(jt)}($ordinal, $value)" + case CalendarIntervalType => s"$row.setInterval($ordinal, $value)" case t: DecimalType => s"$row.setDecimal($ordinal, $value, ${t.precision})" case udt: UserDefinedType[_] => setColumn(row, udt.sqlType, ordinal, value) // The UTF8String, InternalRow, ArrayData and MapData may came from UnsafeRow, we should copy @@ -1602,8 +1604,10 @@ object CodeGenerator extends Logging { nullable: Boolean, isVectorized: Boolean = false): String = { if (nullable) { - // Can't call setNullAt on DecimalType, because we need to keep the offset - if (!isVectorized && dataType.isInstanceOf[DecimalType]) { + // Can't call setNullAt on DecimalType/CalendarIntervalType, because we need to keep the + // offset + if (!isVectorized && (dataType.isInstanceOf[DecimalType] || + dataType.isInstanceOf[CalendarIntervalType])) { s""" |if (!${ev.isNull}) { | ${setColumn(row, dataType, ordinal, ev.value)}; @@ -1634,6 +1638,7 @@ object CodeGenerator extends Logging { case _ if isPrimitiveType(jt) => s"$vector.put${primitiveTypeName(jt)}($rowId, $value);" case t: DecimalType => s"$vector.putDecimal($rowId, $value, ${t.precision});" + case CalendarIntervalType => s"$vector.putInterval($rowId, $value);" case t: StringType => s"$vector.putByteArray($rowId, $value.getBytes());" case _ => throw new IllegalArgumentException(s"cannot generate code for unsupported type: $dataType") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 8da7f65bdeee6..459c1d9a8ba11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -111,6 +111,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro case t: DecimalType if t.precision > Decimal.MAX_LONG_DIGITS => // Can't call setNullAt() for DecimalType with precision larger than 18. s"$rowWriter.write($index, (Decimal) null, ${t.precision}, ${t.scale});" + case CalendarIntervalType => s"$rowWriter.write($index, (CalendarInterval) null);" case _ => s"$rowWriter.setNullAt($index);" } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index b80b30a4e07ae..0149f0deb8d2b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -124,6 +124,36 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB (Timestamp.valueOf("2015-06-22 08:10:25")) } + testBothCodegenAndInterpreted( + "basic conversion with primitive, string and interval types") { + val factory = UnsafeProjection + val fieldTypes: Array[DataType] = Array(LongType, StringType, CalendarIntervalType) + val converter = factory.create(fieldTypes) + + val row = new SpecificInternalRow(fieldTypes) + row.setLong(0, 0) + row.update(1, UTF8String.fromString("Hello")) + val interval1 = new CalendarInterval(3, 1, 1000L) + row.update(2, interval1) + + val unsafeRow: UnsafeRow = converter.apply(row) + assert(unsafeRow.getSizeInBytes === + 8 + 8 * 3 + roundedSize("Hello".getBytes(StandardCharsets.UTF_8).length) + 16) + + assert(unsafeRow.getLong(0) === 0) + assert(unsafeRow.getString(1) === "Hello") + assert(unsafeRow.getInterval(2) === interval1) + + val interval2 = new CalendarInterval(1, 2, 3L) + unsafeRow.setInterval(2, interval2) + assert(unsafeRow.getInterval(2) === interval2) + + val offset = unsafeRow.getLong(2) >>> 32 + unsafeRow.setInterval(2, null) + assert(unsafeRow.getInterval(2) === null) + assert(unsafeRow.getLong(2) >>> 32 === offset) + } + testBothCodegenAndInterpreted("null handling") { val factory = UnsafeProjection val fieldTypes: Array[DataType] = Array( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala index 86b8fa54c0fd4..eaed279679251 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.SparkFunSuite import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.CalendarInterval class UnsafeRowWriterSuite extends SparkFunSuite { @@ -49,4 +50,15 @@ class UnsafeRowWriterSuite extends SparkFunSuite { // The two rows should be the equal assert(res1 == res2) } + + test("write and get calendar intervals through UnsafeRowWriter") { + val rowWriter = new UnsafeRowWriter(2) + rowWriter.resetRowWriter() + rowWriter.write(0, null.asInstanceOf[CalendarInterval]) + assert(rowWriter.getRow.isNullAt(0)) + assert(rowWriter.getRow.getInterval(0) === null) + val interval = new CalendarInterval(0, 1, 0) + rowWriter.write(1, interval) + assert(rowWriter.getRow.getInterval(1) === interval) + } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java index fca7e36859126..f4fdf50692c11 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java @@ -212,6 +212,8 @@ public void update(int ordinal, Object value) { DecimalType t = (DecimalType) dt; Decimal d = Decimal.apply((BigDecimal) value, t.precision(), t.scale()); setDecimal(ordinal, d, t.precision()); + } else if (dt instanceof CalendarIntervalType) { + setInterval(ordinal, (CalendarInterval) value); } else { throw new UnsupportedOperationException("Datatype not supported " + dt); } @@ -270,4 +272,10 @@ public void setDecimal(int ordinal, Decimal value, int precision) { columns[ordinal].putNotNull(rowId); columns[ordinal].putDecimal(rowId, value, precision); } + + @Override + public void setInterval(int ordinal, CalendarInterval value) { + columns[ordinal].putNotNull(rowId); + columns[ordinal].putInterval(rowId, value); + } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java index 951ca39f1bed0..f04b0707ff7b5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java @@ -27,6 +27,7 @@ import org.apache.spark.sql.vectorized.ColumnarArray; import org.apache.spark.sql.vectorized.ColumnarMap; import org.apache.spark.unsafe.array.ByteArrayMethods; +import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; /** @@ -372,6 +373,12 @@ public void putDecimal(int rowId, Decimal value, int precision) { } } + public void putInterval(int rowId, CalendarInterval value) { + getChild(0).putInt(rowId, value.months); + getChild(1).putInt(rowId, value.days); + getChild(2).putLong(rowId, value.microseconds); + } + @Override public UTF8String getUTF8String(int rowId) { if (isNullAt(rowId)) return null; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index ad8976c77b16a..b79d3a278bb3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DecimalType, StringType, StructType} +import org.apache.spark.sql.types.{CalendarIntervalType, DecimalType, StringType, StructType} import org.apache.spark.unsafe.KVIterator import org.apache.spark.util.Utils @@ -643,7 +643,8 @@ case class HashAggregateExec( private def checkIfFastHashMapSupported(ctx: CodegenContext): Boolean = { val isSupported = (groupingKeySchema ++ bufferSchema).forall(f => CodeGenerator.isPrimitiveType(f.dataType) || - f.dataType.isInstanceOf[DecimalType] || f.dataType.isInstanceOf[StringType]) && + f.dataType.isInstanceOf[DecimalType] || f.dataType.isInstanceOf[StringType] || + f.dataType.isInstanceOf[CalendarIntervalType]) && bufferSchema.nonEmpty && modes.forall(mode => mode == Partial || mode == PartialMerge) // For vectorized hash map, We do not support byte array based decimal type for aggregate values @@ -655,7 +656,7 @@ case class HashAggregateExec( val isNotByteArrayDecimalType = bufferSchema.map(_.dataType).filter(_.isInstanceOf[DecimalType]) .forall(!DecimalType.isByteArrayDecimalType(_)) - isSupported && isNotByteArrayDecimalType + isSupported && isNotByteArrayDecimalType } private def enableTwoLevelHashMap(ctx: CodegenContext): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala index 56cf78d8b7fc1..44d19ad60d49f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/RowBasedHashMapGenerator.scala @@ -127,7 +127,8 @@ class RowBasedHashMapGenerator( case t: DecimalType => s"agg_rowWriter.write(${ordinal}, ${key.name}, ${t.precision}, ${t.scale})" case t: DataType => - if (!t.isInstanceOf[StringType] && !CodeGenerator.isPrimitiveType(t)) { + if (!t.isInstanceOf[StringType] && !t.isInstanceOf[CalendarIntervalType] && + !CodeGenerator.isPrimitiveType(t)) { throw new IllegalArgumentException(s"cannot generate code for unsupported type: $t") } s"agg_rowWriter.write(${ordinal}, ${key.name})" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index fb666e9272e56..944e4212b1bfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -29,8 +29,8 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.DecimalData -import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, IntegerType} - +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) @@ -951,4 +951,17 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { assert(error.message.contains("function count_if requires boolean type")) } } + + test("calendar interval agg support hash aggregate") { + val df1 = Seq((1, "1 day"), (2, "2 day"), (3, "3 day"), (3, null)).toDF("a", "b") + val df2 = df1.select(avg('b cast CalendarIntervalType)) + checkAnswer(df2, Row(new CalendarInterval(0, 2, 0)) :: Nil) + assert(df2.queryExecution.executedPlan.find(_.isInstanceOf[HashAggregateExec]).isDefined) + val df3 = df1.groupBy('a).agg(avg('b cast CalendarIntervalType)) + checkAnswer(df3, + Row(1, new CalendarInterval(0, 1, 0)) :: + Row(2, new CalendarInterval(0, 2, 0)) :: + Row(3, new CalendarInterval(0, 3, 0)) :: Nil) + assert(df3.queryExecution.executedPlan.find(_.isInstanceOf[HashAggregateExec]).isDefined) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index 7c7b5b70323b8..37d028d6a713f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -647,29 +647,13 @@ class ColumnarBatchSuite extends SparkFunSuite { assert(days.dataType() == IntegerType) assert(microseconds.dataType() == LongType) - months.putInt(0, 1) - days.putInt(0, 10) - microseconds.putLong(0, 100) - reference += new CalendarInterval(1, 10, 100) - - months.putInt(1, 0) - days.putInt(1, 0) - microseconds.putLong(1, 2000) - reference += new CalendarInterval(0, 0, 2000) - - column.putNull(2) - assert(column.getInterval(2) == null) - reference += null - - months.putInt(3, 20) - days.putInt(3, 0) - microseconds.putLong(3, 0) - reference += new CalendarInterval(20, 0, 0) - - months.putInt(4, 0) - days.putInt(4, 200) - microseconds.putLong(4, 0) - reference += new CalendarInterval(0, 200, 0) + Seq(new CalendarInterval(1, 10, 100), + new CalendarInterval(0, 0, 2000), + new CalendarInterval(20, 0, 0), + new CalendarInterval(0, 200, 0)).zipWithIndex.foreach { case (v, i) => + column.putInterval(i, v) + reference += v + } reference.zipWithIndex.foreach { case (v, i) => val errMsg = "VectorType=" + column.getClass.getSimpleName From e271664a01fd7dee63391890514d76262cad1bc1 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 2 Dec 2019 21:05:06 +0800 Subject: [PATCH 0765/1387] [MINOR][SQL] Rename config name to spark.sql.analyzer.failAmbiguousSelfJoin.enabled ### What changes were proposed in this pull request? add `.enabled` postfix to `spark.sql.analyzer.failAmbiguousSelfJoin`. ### Why are the changes needed? to follow the existing naming style ### Does this PR introduce any user-facing change? no ### How was this patch tested? not needed Closes #26694 from cloud-fan/conf. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 +- .../org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- .../scala/org/apache/spark/sql/Dataset.scala | 4 ++-- .../analysis/DetectAmbiguousSelfJoin.scala | 4 ++-- .../spark/sql/DataFrameSelfJoinSuite.scala | 16 ++++++++-------- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 3ea4839a8175a..5c2e9ba11989d 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -111,7 +111,7 @@ license: | - The result of `java.lang.Math`'s `log`, `log1p`, `exp`, `expm1`, and `pow` may vary across platforms. In Spark 3.0, the result of the equivalent SQL functions (including related SQL functions like `LOG10`) return values consistent with `java.lang.StrictMath`. In virtually all cases this makes no difference in the return value, and the difference is very small, but may not exactly match `java.lang.Math` on x86 platforms in cases like, for example, `log(3.0)`, whose value varies between `Math.log()` and `StrictMath.log()`. - - Since Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin` to `false`. + - Since Spark 3.0, Dataset query fails if it contains ambiguous column reference that is caused by self join. A typical example: `val df1 = ...; val df2 = df1.filter(...);`, then `df1.join(df2, df1("a") > df2("a"))` returns an empty result which is quite confusing. This is because Spark cannot resolve Dataset column references that point to tables being self joined, and `df1("a")` is exactly the same as `df2("a")` in Spark. To restore the behavior before Spark 3.0, you can set `spark.sql.analyzer.failAmbiguousSelfJoin.enabled` to `false`. - Since Spark 3.0, `Cast` function processes string literals such as 'Infinity', '+Infinity', '-Infinity', 'NaN', 'Inf', '+Inf', '-Inf' in case insensitive manner when casting the literals to `Double` or `Float` type to ensure greater compatibility with other database systems. This behaviour change is illustrated in the table below:
    diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 105b2a857e5af..e1b8192fba213 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -875,8 +875,8 @@ object SQLConf { .booleanConf .createWithDefault(true) - val FAIL_AMBIGUOUS_SELF_JOIN = - buildConf("spark.sql.analyzer.failAmbiguousSelfJoin") + val FAIL_AMBIGUOUS_SELF_JOIN_ENABLED = + buildConf("spark.sql.analyzer.failAmbiguousSelfJoin.enabled") .doc("When true, fail the Dataset query if it contains ambiguous self-join.") .internal() .booleanConf 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 e1bca44dfccf5..1b75fccbdb7b2 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 @@ -229,7 +229,7 @@ class Dataset[T] private[sql]( case _ => queryExecution.analyzed } - if (sparkSession.sessionState.conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN)) { + if (sparkSession.sessionState.conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED)) { plan.setTagValue(Dataset.DATASET_ID_TAG, id) } plan @@ -1337,7 +1337,7 @@ class Dataset[T] private[sql]( private def addDataFrameIdToCol(expr: NamedExpression): NamedExpression = { val newExpr = expr transform { case a: AttributeReference - if sparkSession.sessionState.conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN) => + if sparkSession.sessionState.conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED) => val metadata = new MetadataBuilder() .withMetadata(a.metadata) .putLong(Dataset.DATASET_ID_KEY, id) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala index 5c3c735f0346c..614d6c2846bfa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/analysis/DetectAmbiguousSelfJoin.scala @@ -71,7 +71,7 @@ class DetectAmbiguousSelfJoin(conf: SQLConf) extends Rule[LogicalPlan] { } override def apply(plan: LogicalPlan): LogicalPlan = { - if (!conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN)) return plan + if (!conf.getConf(SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED)) return plan // We always remove the special metadata from `AttributeReference` at the end of this rule, so // Dataset column reference only exists in the root node via Dataset transformations like @@ -149,7 +149,7 @@ class DetectAmbiguousSelfJoin(conf: SQLConf) extends Rule[LogicalPlan] { "to figure out which one. Please alias the Datasets with different names via " + "`Dataset.as` before joining them, and specify the column using qualified name, e.g. " + """`df.as("a").join(df.as("b"), $"a.id" > $"b.id")`. You can also set """ + - s"${SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key} to false to disable this check.") + s"${SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key} to false to disable this check.") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala index 92f1e4306c5b1..59b5dacb10fdb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala @@ -96,7 +96,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { val df2 = df1.filter($"id" > 0) withSQLConf( - SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key -> "false", + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "false", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { // `df1("id") > df2("id")` is always false. checkAnswer(df1.join(df2, df1("id") > df2("id")), Nil) @@ -110,7 +110,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { } withSQLConf( - SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key -> "true", + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { assertAmbiguousSelfJoin(df1.join(df2, df1("id") > df2("id"))) } @@ -121,7 +121,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { val df2 = df1.filter($"id" > 0) withSQLConf( - SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key -> "true", + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { assertAmbiguousSelfJoin(df1.join(df2, df1.colRegex("id") > df2.colRegex("id"))) } @@ -132,7 +132,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { val df2 = df1.filter($"a.b" > 0) withSQLConf( - SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key -> "true", + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { assertAmbiguousSelfJoin(df1.join(df2, df1("a.b") > df2("a.c"))) } @@ -143,7 +143,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { val df2 = df1.filter($"id" > 0) withSQLConf( - SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key -> "false", + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "false", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { // `df2("id")` actually points to the column of `df1`. checkAnswer(df1.join(df2).select(df2("id")), Seq(0, 0, 1, 1, 2, 2).map(Row(_))) @@ -157,7 +157,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { } withSQLConf( - SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key -> "true", + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { assertAmbiguousSelfJoin(df1.join(df2).select(df2("id"))) } @@ -170,7 +170,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { val df4 = spark.range(1) withSQLConf( - SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key -> "false", + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "false", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { // `df2("id") < df3("id")` is always false checkAnswer(df1.join(df2).join(df3, df2("id") < df3("id")), Nil) @@ -196,7 +196,7 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { } withSQLConf( - SQLConf.FAIL_AMBIGUOUS_SELF_JOIN.key -> "true", + SQLConf.FAIL_AMBIGUOUS_SELF_JOIN_ENABLED.key -> "true", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { assertAmbiguousSelfJoin(df1.join(df2).join(df3, df2("id") < df3("id"))) assertAmbiguousSelfJoin(df1.join(df4).join(df2).select(df2("id"))) From 54edaee58654bdc3c961906a8390088f35460ae9 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 2 Dec 2019 09:01:45 -0600 Subject: [PATCH 0766/1387] [MINOR][SS] Add implementation note on overriding serialize/deserialize in HDFSMetadataLog methods' scaladoc ### What changes were proposed in this pull request? The patch adds scaladoc on `HDFSMetadataLog.serialize` and `HDFSMetadataLog.deserialize` for adding implementation note when overriding - HDFSMetadataLog calls `serialize` and `deserialize` inside try-finally and caller will do the resource (input stream, output stream) cleanup, so resource cleanup should not be performed in these methods, but there's no note on this (only code comment, not scaladoc) which is easy to be missed. ### Why are the changes needed? Contributors who are unfamiliar with the intention seem to think it as a bug if the resource is not cleaned up in serialize/deserialize of subclass of HDFSMetadataLog, and they couldn't know about the intention without reading the code of HDFSMetadataLog. Adding the note as scaladoc would expand the visibility. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Just a doc change. Closes #26732 from HeartSaVioR/MINOR-SS-HDFSMetadataLog-serde-scaladoc. Lead-authored-by: Jungtaek Lim (HeartSaVioR) Co-authored-by: dz <953396112@qq.com> Signed-off-by: Sean Owen --- .../sql/execution/streaming/HDFSMetadataLog.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 5c9249fb16343..ed0c44da08c5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -90,13 +90,21 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: } } + /** + * Serialize the metadata and write to the output stream. If this method is overridden in a + * subclass, the overriding method should not close the given output stream, as it will be closed + * in the caller. + */ protected def serialize(metadata: T, out: OutputStream): Unit = { - // called inside a try-finally where the underlying stream is closed in the caller Serialization.write(metadata, out) } + /** + * Read and deserialize the metadata from input stream. If this method is overridden in a + * subclass, the overriding method should not close the given input stream, as it will be closed + * in the caller. + */ protected def deserialize(in: InputStream): T = { - // called inside a try-finally where the underlying stream is closed in the caller val reader = new InputStreamReader(in, StandardCharsets.UTF_8) Serialization.read[T](reader) } From e842033accf12190f1bf3962546065613656410f Mon Sep 17 00:00:00 2001 From: huangtianhua Date: Mon, 2 Dec 2019 09:04:00 -0600 Subject: [PATCH 0767/1387] [SPARK-27721][BUILD] Switch to use right leveldbjni according to the platforms This change adds a profile to switch to use the right leveldbjni package according to the platforms: aarch64 uses org.openlabtesting.leveldbjni:leveldbjni-all.1.8, and other platforms use the old one org.fusesource.leveldbjni:leveldbjni-all.1.8. And because some hadoop dependencies packages are also depend on org.fusesource.leveldbjni:leveldbjni-all, but hadoop merge the similar change on trunk, details see https://issues.apache.org/jira/browse/HADOOP-16614, so exclude the dependency of org.fusesource.leveldbjni for these hadoop packages related. Then Spark can build/test on aarch64 platform successfully. Closes #26636 from huangtianhua/add-aarch64-leveldbjni. Authored-by: huangtianhua Signed-off-by: Sean Owen --- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- pom.xml | 23 ++++++++++++++++++++++- 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f042a12fda3d2..a1c8a8e6582eb 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -45,7 +45,7 @@ guava - org.fusesource.leveldbjni + ${leveldbjni.group} leveldbjni-all diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2ee17800c10e4..163c250054e4d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -52,7 +52,7 @@ - org.fusesource.leveldbjni + ${leveldbjni.group} leveldbjni-all 1.8 diff --git a/pom.xml b/pom.xml index bd30a16be47a4..0ce87233a9d5f 100644 --- a/pom.xml +++ b/pom.xml @@ -203,6 +203,8 @@ and ./python/setup.py too. --> 0.15.1 + + org.fusesource.leveldbjni ${java.home} @@ -531,7 +533,7 @@ ${commons.httpcore.version} - org.fusesource.leveldbjni + ${leveldbjni.group} leveldbjni-all 1.8 @@ -958,6 +960,10 @@ ${hadoop.version} ${hadoop.deps.scope} + + org.fusesource.leveldbjni + leveldbjni-all + asm asm @@ -1204,6 +1210,10 @@ tests test + + org.fusesource.leveldbjni + leveldbjni-all + asm asm @@ -1244,6 +1254,10 @@ ${yarn.version} ${hadoop.deps.scope} + + org.fusesource.leveldbjni + leveldbjni-all + asm asm @@ -3118,5 +3132,12 @@ sparkr + + + aarch64 + + org.openlabtesting.leveldbjni + + From babefdee1c133c6b35ff026d5deacb292a0b85aa Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 2 Dec 2019 09:05:40 -0600 Subject: [PATCH 0768/1387] [SPARK-30085][SQL][DOC] Standardize sql reference ### What changes were proposed in this pull request? Standardize sql reference ### Why are the changes needed? To have consistent docs ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Tested using jykyll build --serve Closes #26721 from huaxingao/spark-30085. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- docs/sql-ref-syntax-aux-analyze-table.md | 23 +++-- docs/sql-ref-syntax-aux-cache-cache-table.md | 13 ++- docs/sql-ref-syntax-aux-cache-clear-cache.md | 1 + .../sql-ref-syntax-aux-cache-uncache-table.md | 13 ++- docs/sql-ref-syntax-aux-cache.md | 1 + docs/sql-ref-syntax-aux-describe-database.md | 2 +- docs/sql-ref-syntax-aux-describe-function.md | 4 +- docs/sql-ref-syntax-aux-describe-query.md | 2 +- docs/sql-ref-syntax-aux-describe-table.md | 8 +- docs/sql-ref-syntax-aux-refresh-table.md | 6 +- docs/sql-ref-syntax-aux-show-columns.md | 6 +- docs/sql-ref-syntax-aux-show-create-table.md | 12 ++- docs/sql-ref-syntax-aux-show-databases.md | 2 +- docs/sql-ref-syntax-aux-show-functions.md | 2 +- docs/sql-ref-syntax-aux-show-partitions.md | 25 ++++-- docs/sql-ref-syntax-aux-show-table.md | 13 ++- docs/sql-ref-syntax-aux-show-tables.md | 4 +- docs/sql-ref-syntax-aux-show-tblproperties.md | 4 +- docs/sql-ref-syntax-ddl-alter-database.md | 3 +- docs/sql-ref-syntax-ddl-alter-table.md | 84 ++++++++++++------- docs/sql-ref-syntax-ddl-alter-view.md | 16 ++-- docs/sql-ref-syntax-ddl-create-database.md | 6 +- docs/sql-ref-syntax-ddl-create-function.md | 4 +- docs/sql-ref-syntax-ddl-create-view.md | 23 +++-- docs/sql-ref-syntax-ddl-drop-database.md | 4 +- docs/sql-ref-syntax-ddl-drop-function.md | 2 +- docs/sql-ref-syntax-ddl-drop-table.md | 14 ++-- docs/sql-ref-syntax-ddl-drop-view.md | 14 ++-- docs/sql-ref-syntax-ddl-repair-table.md | 12 ++- docs/sql-ref-syntax-ddl-truncate-table.md | 24 ++++-- docs/sql-ref-syntax-dml-insert-into.md | 24 ++++-- ...l-ref-syntax-dml-insert-overwrite-table.md | 24 ++++-- docs/sql-ref-syntax-dml-load.md | 24 ++++-- 33 files changed, 277 insertions(+), 142 deletions(-) diff --git a/docs/sql-ref-syntax-aux-analyze-table.md b/docs/sql-ref-syntax-aux-analyze-table.md index 887f225f89bcc..e448ca38c849e 100644 --- a/docs/sql-ref-syntax-aux-analyze-table.md +++ b/docs/sql-ref-syntax-aux-analyze-table.md @@ -25,20 +25,33 @@ The `ANALYZE TABLE` statement collects statistics about the table to be used by ### Syntax {% highlight sql %} -ANALYZE TABLE table_name [ PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) ] +ANALYZE TABLE table_identifier [ partition_spec ] COMPUTE STATISTICS [ NOSCAN | FOR COLUMNS col [ , ... ] | FOR ALL COLUMNS ] {% endhighlight %} ### Parameters
    -
    table_name
    -
    The name of an existing table.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    -
    PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] )
    -
    Specifies one or more partition column and value pairs. The partition value is optional.
    +
    partition_spec
    +
    + An optional parameter that specifies a comma separated list of key and value pairs + for partitions. When specified, partition statistics is returned.

    + Syntax: + + PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) + +
    diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md index 224b6631618fb..ed6ef973466dd 100644 --- a/docs/sql-ref-syntax-aux-cache-cache-table.md +++ b/docs/sql-ref-syntax-aux-cache-cache-table.md @@ -24,7 +24,7 @@ license: | ### Syntax {% highlight sql %} -CACHE [ LAZY ] TABLE table_name +CACHE [ LAZY ] TABLE table_identifier [ OPTIONS ( 'storageLevel' [ = ] value ) ] [ [ AS ] query ] {% endhighlight %} @@ -35,8 +35,14 @@ CACHE [ LAZY ] TABLE table_name
    -
    table_name
    -
    The name of the table to be cached.
    +
    table_identifier
    +
    + Specifies the table or view name to be cached. The table or view name may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    @@ -80,4 +86,5 @@ CACHE TABLE testCache OPTIONS ('storageLevel' 'DISK_ONLY') SELECT * FROM testDat ### Related Statements * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) + * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) diff --git a/docs/sql-ref-syntax-aux-cache-clear-cache.md b/docs/sql-ref-syntax-aux-cache-clear-cache.md index d8e451a230a71..57ba77d59ea7b 100644 --- a/docs/sql-ref-syntax-aux-cache-clear-cache.md +++ b/docs/sql-ref-syntax-aux-cache-clear-cache.md @@ -35,4 +35,5 @@ CLEAR CACHE; ### Related Statements * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) * [UNCACHE TABLE](sql-ref-syntax-aux-cache-uncache-table.html) + * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md index a6cb4d6807b22..e0581d0d213df 100644 --- a/docs/sql-ref-syntax-aux-cache-uncache-table.md +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -24,12 +24,18 @@ license: | underlying entries should already have been brought to cache by previous `CACHE TABLE` operation. `UNCACHE TABLE` on a non-existent table throws Exception if `IF EXISTS` is not specified. ### Syntax {% highlight sql %} -UNCACHE TABLE [ IF EXISTS ] table_name +UNCACHE TABLE [ IF EXISTS ] table_identifier {% endhighlight %} ### Parameters
    -
    table_name
    -
    The name of the table or view to be uncached.
    +
    table_identifier
    +
    + Specifies the table or view name to be uncached. The table or view name may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    ### Examples {% highlight sql %} @@ -38,3 +44,4 @@ UNCACHE TABLE t1; ### Related Statements * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) + * [REFRESH TABLE](sql-ref-syntax-aux-refresh-table.html) diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md index c3dcb276a7e0f..e213bc3dff4e1 100644 --- a/docs/sql-ref-syntax-aux-cache.md +++ b/docs/sql-ref-syntax-aux-cache.md @@ -22,3 +22,4 @@ license: | * [CACHE TABLE statement](sql-ref-syntax-aux-cache-cache-table.html) * [UNCACHE TABLE statement](sql-ref-syntax-aux-cache-uncache-table.html) * [CLEAR CACHE statement](sql-ref-syntax-aux-cache-clear-cache.html) +* [REFRESH TABLE statement](sql-ref-syntax-aux-refresh-table.html) diff --git a/docs/sql-ref-syntax-aux-describe-database.md b/docs/sql-ref-syntax-aux-describe-database.md index b8e76fd1ee18b..05a64ab2060b4 100644 --- a/docs/sql-ref-syntax-aux-describe-database.md +++ b/docs/sql-ref-syntax-aux-describe-database.md @@ -27,7 +27,7 @@ interchangeable. ### Syntax {% highlight sql %} -{DESC | DESCRIBE} DATABASE [EXTENDED] db_name +{ DESC | DESCRIBE } DATABASE [ EXTENDED ] db_name {% endhighlight %} ### Parameters diff --git a/docs/sql-ref-syntax-aux-describe-function.md b/docs/sql-ref-syntax-aux-describe-function.md index d3dc1920642f6..f3c9c625b97b8 100644 --- a/docs/sql-ref-syntax-aux-describe-function.md +++ b/docs/sql-ref-syntax-aux-describe-function.md @@ -27,7 +27,7 @@ metadata information is returned along with the extended usage information. ### Syntax {% highlight sql %} -{DESC | DESCRIBE} FUNCTION [EXTENDED] function_name +{ DESC | DESCRIBE } FUNCTION [ EXTENDED ] function_name {% endhighlight %} ### Parameters @@ -40,7 +40,7 @@ metadata information is returned along with the extended usage information. it is resolved from the current database.

    Syntax: - [database_name.]function_name + [ database_name. ] function_name
    diff --git a/docs/sql-ref-syntax-aux-describe-query.md b/docs/sql-ref-syntax-aux-describe-query.md index aa872955eb943..b07ebe78193d1 100644 --- a/docs/sql-ref-syntax-aux-describe-query.md +++ b/docs/sql-ref-syntax-aux-describe-query.md @@ -26,7 +26,7 @@ describe the query output. ### Syntax {% highlight sql %} -{DESC | DESCRIBE} [QUERY] input_statement +{ DESC | DESCRIBE } [ QUERY ] input_statement {% endhighlight %} ### Parameters diff --git a/docs/sql-ref-syntax-aux-describe-table.md b/docs/sql-ref-syntax-aux-describe-table.md index e38fd32377c51..4e6aeb5b6f349 100644 --- a/docs/sql-ref-syntax-aux-describe-table.md +++ b/docs/sql-ref-syntax-aux-describe-table.md @@ -26,7 +26,7 @@ to return the metadata pertaining to a partition or column respectively. ### Syntax {% highlight sql %} -{DESC | DESCRIBE} [TABLE] [format] table_identifier [partition_spec] [col_name] +{ DESC | DESCRIBE } [ TABLE ] [ format ] table_identifier [ partition_spec ] [ col_name ] {% endhighlight %} ### Parameters @@ -42,7 +42,7 @@ to return the metadata pertaining to a partition or column respectively. Specifies a table name, which may be optionally qualified with a database name.

    Syntax: - [database_name.]table_name + [ database_name. ] table_name
    partition_spec
    @@ -51,7 +51,7 @@ to return the metadata pertaining to a partition or column respectively. for partitions. When specified, additional partition metadata is returned.

    Syntax: - PARTITION (partition_col_name = partition_col_val [ , ... ]) + PARTITION ( partition_col_name = partition_col_val [ , ... ] )
    col_name
    @@ -63,7 +63,7 @@ to return the metadata pertaining to a partition or column respectively. Syntax: - [database_name.][table_name.]column_name + [ database_name. ] [ table_name. ] column_name diff --git a/docs/sql-ref-syntax-aux-refresh-table.md b/docs/sql-ref-syntax-aux-refresh-table.md index 262382a467073..4b3fcdd65ba75 100644 --- a/docs/sql-ref-syntax-aux-refresh-table.md +++ b/docs/sql-ref-syntax-aux-refresh-table.md @@ -26,17 +26,17 @@ lazy manner when the cached table or the query associated with it is executed ag ### Syntax {% highlight sql %} -REFRESH [TABLE] tableIdentifier +REFRESH [TABLE] table_identifier {% endhighlight %} ### Parameters
    -
    tableIdentifier
    +
    table_identifier
    Specifies a table name, which is either a qualified or unqualified name that designates a table/view. If no database identifier is provided, it refers to a temporary view or a table/view in the current database.

    Syntax: - [database_name.]table_name + [ database_name. ] table_name
    diff --git a/docs/sql-ref-syntax-aux-show-columns.md b/docs/sql-ref-syntax-aux-show-columns.md index 1e6947b160bfe..0c8aba83a8403 100644 --- a/docs/sql-ref-syntax-aux-show-columns.md +++ b/docs/sql-ref-syntax-aux-show-columns.md @@ -23,18 +23,18 @@ Return the list of columns in a table. If the table does not exist, an exception ### Syntax {% highlight sql %} -SHOW COLUMNS table [ database ] +SHOW COLUMNS table_identifier [ database ] {% endhighlight %} ### Parameters
    -
    table
    +
    table_identifier
    Specifies the table name of an existing table. The table may be optionally qualified with a database name.

    Syntax: - { IN | FROM } [database_name.]table_name + { IN | FROM } [ database_name . ] table_name

    Note: Keywords IN and FROM are interchangeable. diff --git a/docs/sql-ref-syntax-aux-show-create-table.md b/docs/sql-ref-syntax-aux-show-create-table.md index 7871d30b5b186..24aba602ab3cf 100644 --- a/docs/sql-ref-syntax-aux-show-create-table.md +++ b/docs/sql-ref-syntax-aux-show-create-table.md @@ -24,13 +24,19 @@ license: | ### Syntax {% highlight sql %} -SHOW CREATE TABLE name +SHOW CREATE TABLE table_identifier {% endhighlight %} ### Parameters
    -
    name
    -
    The name of the table or view to be used for SHOW CREATE TABLE.
    +
    table_identifier
    +
    + Specifies a table or view name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    ### Examples diff --git a/docs/sql-ref-syntax-aux-show-databases.md b/docs/sql-ref-syntax-aux-show-databases.md index efa32f2915143..9d4be21daeabc 100644 --- a/docs/sql-ref-syntax-aux-show-databases.md +++ b/docs/sql-ref-syntax-aux-show-databases.md @@ -27,7 +27,7 @@ and mean the same thing. ### Syntax {% highlight sql %} -SHOW {DATABASES|SCHEMAS} [LIKE string_pattern] +SHOW { DATABASES | SCHEMAS } [ LIKE string_pattern ] {% endhighlight %} ### Parameters diff --git a/docs/sql-ref-syntax-aux-show-functions.md b/docs/sql-ref-syntax-aux-show-functions.md index db026071e6a59..701d427039aa5 100644 --- a/docs/sql-ref-syntax-aux-show-functions.md +++ b/docs/sql-ref-syntax-aux-show-functions.md @@ -28,7 +28,7 @@ clause is optional and supported only for compatibility with other systems. ### Syntax {% highlight sql %} -SHOW [ function_kind ] FUNCTIONS ([LIKE] function_name | regex_pattern) +SHOW [ function_kind ] FUNCTIONS ( [ LIKE ] function_name | regex_pattern ) {% endhighlight %} ### Parameters diff --git a/docs/sql-ref-syntax-aux-show-partitions.md b/docs/sql-ref-syntax-aux-show-partitions.md index 216f3f0d679ec..6c8401e8e4718 100644 --- a/docs/sql-ref-syntax-aux-show-partitions.md +++ b/docs/sql-ref-syntax-aux-show-partitions.md @@ -26,19 +26,30 @@ partition spec. ### Syntax {% highlight sql %} -SHOW PARTITIONS table_name - [ PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) ] +SHOW PARTITIONS table_identifier [ partition_spec ] {% endhighlight %} ### Parameters
    -
    table_name
    -
    The name of an existing table.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    -
    PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] )
    -
    An optional parameter that specifies a comma separated list of key and value pairs for - partitions. When specified, the partitions that match the partition spec are returned.
    +
    partition_spec
    +
    + An optional parameter that specifies a comma separated list of key and value pairs + for partitions. When specified, the partitions that match the partition spec are returned.

    + Syntax: + + PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) + +
    ### Examples diff --git a/docs/sql-ref-syntax-aux-show-table.md b/docs/sql-ref-syntax-aux-show-table.md index 1d881a73c8112..49696585ba581 100644 --- a/docs/sql-ref-syntax-aux-show-table.md +++ b/docs/sql-ref-syntax-aux-show-table.md @@ -31,7 +31,8 @@ cannot be used with a partition specification. ### Syntax {% highlight sql %} -SHOW TABLE EXTENDED [IN|FROM database_name] LIKE 'identifier_with_wildcards' [PARTITION(partition_spec)]; +SHOW TABLE EXTENDED [ IN | FROM database_name ] LIKE 'identifier_with_wildcards' + [ partition_spec ] {% endhighlight %} ### Parameters @@ -50,10 +51,14 @@ SHOW TABLE EXTENDED [IN|FROM database_name] LIKE 'identifier_with_wildcards' [PA
  • The leading and trailing blanks are trimmed in the input pattern before processing.
  • -
    PARTITION(partition_spec)
    +
    partition_spec
    - Specifies partition column and its value which is exists in the table. Note that a table regex - cannot be used with a partition specification.. + An optional parameter that specifies a comma separated list of key and value pairs + for partitions. Note that a table regex cannot be used with a partition specification.

    + Syntax: + + PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) +
    ### Examples diff --git a/docs/sql-ref-syntax-aux-show-tables.md b/docs/sql-ref-syntax-aux-show-tables.md index 46cfe40dc2600..41af8e7d607d0 100644 --- a/docs/sql-ref-syntax-aux-show-tables.md +++ b/docs/sql-ref-syntax-aux-show-tables.md @@ -27,12 +27,12 @@ current database. ### Syntax {% highlight sql %} -SHOW TABLES [{FROM|IN} database_name] [LIKE 'regex_pattern'] +SHOW TABLES [ { FROM | IN } database_name ] [ LIKE 'regex_pattern' ] {% endhighlight %} ### Parameters
    -
    {FROM|IN} database_name
    +
    { FROM | IN } database_name
    Specifies the database name from which tables are listed.
    diff --git a/docs/sql-ref-syntax-aux-show-tblproperties.md b/docs/sql-ref-syntax-aux-show-tblproperties.md index 212db05017ad4..451fd4abc85bb 100644 --- a/docs/sql-ref-syntax-aux-show-tblproperties.md +++ b/docs/sql-ref-syntax-aux-show-tblproperties.md @@ -37,7 +37,7 @@ SHOW TBLPROPERTIES table_identifier with a database name.

    Syntax: - [database_name.]table_name + [ database_name. ] table_name
    unquoted_property_key
    @@ -46,7 +46,7 @@ SHOW TBLPROPERTIES table_identifier parts separated by dot.

    Syntax: - [key_part1][.key_part2][...] + [ key_part1 ] [ .key_part2 ] [ ... ]
    property_key_as_string_literal
    diff --git a/docs/sql-ref-syntax-ddl-alter-database.md b/docs/sql-ref-syntax-ddl-alter-database.md index f241688baf71a..a32343674feb0 100644 --- a/docs/sql-ref-syntax-ddl-alter-database.md +++ b/docs/sql-ref-syntax-ddl-alter-database.md @@ -27,7 +27,8 @@ for a database and may be used for auditing purposes. ### Syntax {% highlight sql %} -ALTER {DATABASE | SCHEMA} database_name SET DBPROPERTIES (property_name=property_value, ...); +ALTER { DATABASE | SCHEMA } database_name + SET DBPROPERTIES ( property_name = property_value, ... ) {% endhighlight %} ### Parameters diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index a9971498f520b..1d7ace74231e6 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -27,30 +27,33 @@ license: | #### Syntax {% highlight sql %} -ALTER TABLE [db_name.]old_table_name RENAME TO [db_name.]new_table_name +ALTER TABLE table_identifier RENAME TO table_identifier -ALTER TABLE table_name PARTITION partition_spec RENAME TO PARTITION partition_spec; +ALTER TABLE table_identifier partition_spec RENAME TO partition_spec {% endhighlight %} #### Parameters
    -
    old_table_name
    -
    Name of an existing table.
    -
    -
    -
    db_name
    -
    Name of the existing database.
    -
    - -
    -
    new_table_name
    -
    New name using which the table has to be renamed.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    partition_spec
    -
    Partition to be renamed.
    +
    + Partition to be renamed.

    + Syntax: + + PARTITION ( partition_col_name = partition_col_val [ , ... ] ) + +
    @@ -59,18 +62,23 @@ ALTER TABLE table_name PARTITION partition_spec RENAME TO PARTITION partition_sp #### Syntax {% highlight sql %} -ALTER TABLE table_name ADD COLUMNS (col_spec[, col_spec ...]) +ALTER TABLE table_identifier ADD COLUMNS ( col_spec [ , col_spec ... ] ) {% endhighlight %} #### Parameters
    -
    table_name
    -
    The name of an existing table.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    -
    -
    COLUMNS (col_spec)
    +
    COLUMNS ( col_spec )
    Specifies the columns to be added to be renamed.
    @@ -87,10 +95,10 @@ this overrides the old value with the new one. {% highlight sql %} --Set Table Properties -ALTER TABLE table_name SET TBLPROPERTIES (key1=val1, key2=val2, ...) +ALTER TABLE table_identifier SET TBLPROPERTIES ( key1 = val1, key2 = val2, ... ) --Unset Table Properties -ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] (key1, key2, ...) +ALTER TABLE table_identifier UNSET TBLPROPERTIES [ IF EXISTS ] ( key1, key2, ... ) {% endhighlight %} @@ -102,11 +110,11 @@ this overrides the old value with the new one. {% highlight sql %} --Set SERDE Properties -ALTER TABLE table_name [PARTITION part_spec] - SET SERDEPROPERTIES (key1=val1, key2=val2, ...) +ALTER TABLE table_identifier [ partition_spec ] + SET SERDEPROPERTIES ( key1 = val1, key2 = val2, ... ) -ALTER TABLE table_name [PARTITION part_spec] SET SERDE serde_class_name - [WITH SERDEPROPERTIES (key1=val1, key2=val2, ...)] +ALTER TABLE table_identifier [ partition_spec ] SET SERDE serde_class_name + [ WITH SERDEPROPERTIES ( key1 = val1, key2 = val2, ... ) ] {% endhighlight %} @@ -118,26 +126,38 @@ exsisting tables. {% highlight sql %} --Changing File Format -ALTER TABLE table_name [PARTITION partition_spec] SET FILEFORMAT file_format; +ALTER TABLE table_identifier [ partition_spec ] SET FILEFORMAT file_format --Changing File Location -ALTER TABLE table_name [PARTITION partition_spec] SET LOCATION 'new_location'; +ALTER TABLE table_identifier [ partition_spec ] SET LOCATION 'new_location' {% endhighlight %} #### Parameters
    -
    table_name
    -
    The name of an existing table.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    -
    PARTITION (part_spec)
    -
    Specifies the partition on which the property has to be set.
    +
    partition_spec
    +
    + Specifies the partition on which the property has to be set.

    + Syntax: + + PARTITION ( partition_col_name = partition_col_val [ , ... ] ) + +
    -
    SERDEPROPERTIES (key1=val1, key2=val2, ...)
    +
    SERDEPROPERTIES ( key1 = val1, key2 = val2, ... )
    Specifies the SERDE properties to be set.
    diff --git a/docs/sql-ref-syntax-ddl-alter-view.md b/docs/sql-ref-syntax-ddl-alter-view.md index d5cbb86d48821..d42d040522cae 100644 --- a/docs/sql-ref-syntax-ddl-alter-view.md +++ b/docs/sql-ref-syntax-ddl-alter-view.md @@ -40,7 +40,7 @@ ALTER VIEW view_identifier RENAME TO view_identifier Specifies a view name, which may be optionally qualified with a database name.

    Syntax: - [database_name.]view_name + [ database_name. ] view_name
    @@ -52,7 +52,7 @@ the properties. #### Syntax {% highlight sql %} -ALTER VIEW view_identifier SET TBLPROPERTIES (property_key=property_val [, ...]) +ALTER VIEW view_identifier SET TBLPROPERTIES ( property_key = property_val [ , ... ] ) {% endhighlight %} #### Parameters @@ -62,7 +62,7 @@ ALTER VIEW view_identifier SET TBLPROPERTIES (property_key=property_val [, ...]) Specifies a view name, which may be optionally qualified with a database name.

    Syntax: - [database_name.]view_name + [ database_name. ] view_name
    property_key
    @@ -70,7 +70,7 @@ ALTER VIEW view_identifier SET TBLPROPERTIES (property_key=property_val [, ...]) Specifies the property key. The key may consists of multiple parts separated by dot.

    Syntax: - [key_part1][.key_part2][...] + [ key_part1 ] [ .key_part2 ] [ ... ] @@ -81,7 +81,7 @@ Drop one or more properties of an existing view. If the specified keys do not ex #### Syntax {% highlight sql %} -ALTER VIEW view_identifier UNSET TBLPROPERTIES [IF EXISTS] (property_key [, ...]) +ALTER VIEW view_identifier UNSET TBLPROPERTIES [ IF EXISTS ] ( property_key [ , ... ] ) {% endhighlight %} #### Parameters @@ -91,7 +91,7 @@ ALTER VIEW view_identifier UNSET TBLPROPERTIES [IF EXISTS] (property_key [, ...] Specifies a view name, which may be optionally qualified with a database name.

    Syntax: - [database_name.]view_name + [ database_name. ] view_name
    property_key
    @@ -99,7 +99,7 @@ ALTER VIEW view_identifier UNSET TBLPROPERTIES [IF EXISTS] (property_key [, ...] Specifies the property key. The key may consists of multiple parts separated by dot.

    Syntax: - [key_part1][.key_part2][...] + [ key_part1 ] [ .key_part2 ] [ ... ] @@ -120,7 +120,7 @@ ALTER VIEW view_identifier AS select_statement Specifies a view name, which may be optionally qualified with a database name.

    Syntax: - [database_name.]view_name + [ database_name. ] view_name
    select_statement
    diff --git a/docs/sql-ref-syntax-ddl-create-database.md b/docs/sql-ref-syntax-ddl-create-database.md index ed0bbf629b027..4d2211c650953 100644 --- a/docs/sql-ref-syntax-ddl-create-database.md +++ b/docs/sql-ref-syntax-ddl-create-database.md @@ -24,10 +24,10 @@ Creates a database with the specified name. If database with the same name alrea ### Syntax {% highlight sql %} -CREATE {DATABASE | SCHEMA} [ IF NOT EXISTS ] database_name +CREATE { DATABASE | SCHEMA } [ IF NOT EXISTS ] database_name [ COMMENT database_comment ] [ LOCATION database_directory ] - [ WITH DBPROPERTIES (property_name=property_value [ , ...]) ] + [ WITH DBPROPERTIES ( property_name = property_value [ , ... ] ) ] {% endhighlight %} ### Parameters @@ -44,7 +44,7 @@ CREATE {DATABASE | SCHEMA} [ IF NOT EXISTS ] database_name
    database_comment
    Specifies the description for the database.
    -
    WITH DBPROPERTIES (property_name=property_value [ , ...])
    +
    WITH DBPROPERTIES ( property_name=property_value [ , ... ] )
    Specifies the properties for the database in key-value pairs.
    diff --git a/docs/sql-ref-syntax-ddl-create-function.md b/docs/sql-ref-syntax-ddl-create-function.md index 8b4aaebfbbabd..1f94bf6d25aa5 100644 --- a/docs/sql-ref-syntax-ddl-create-function.md +++ b/docs/sql-ref-syntax-ddl-create-function.md @@ -64,7 +64,7 @@ CREATE [ OR REPLACE ] [ TEMPORARY ] FUNCTION [ IF NOT EXISTS ] optionally qualified with a database name.

    Syntax: - [database_name.]function_name + [ database_name. ] function_name
    class_name
    @@ -84,7 +84,7 @@ CREATE [ OR REPLACE ] [ TEMPORARY ] FUNCTION [ IF NOT EXISTS ] along with its dependencies.

    Syntax: - USING { { (JAR | FILE ) resource_uri} , ...} + USING { { (JAR | FILE ) resource_uri } , ... } diff --git a/docs/sql-ref-syntax-ddl-create-view.md b/docs/sql-ref-syntax-ddl-create-view.md index 16310caca0bd9..67060d70f0790 100644 --- a/docs/sql-ref-syntax-ddl-create-view.md +++ b/docs/sql-ref-syntax-ddl-create-view.md @@ -26,9 +26,8 @@ a virtual table that has no physical data therefore other operations like ### Syntax {% highlight sql %} -CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name - create_view_clauses - AS query; +CREATE [ OR REPLACE ] [ [ GLOBAL ] TEMPORARY ] VIEW [ IF NOT EXISTS ] view_identifier + create_view_clauses AS query {% endhighlight %} ### Parameters @@ -37,7 +36,7 @@ CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_nam
    If a view of same name already exists, it will be replaced.
    -
    [GLOBAL] TEMPORARY
    +
    [ GLOBAL ] TEMPORARY
    TEMPORARY views are session-scoped and will be dropped when session ends because it skips persisting the definition in the underlying metastore, if any. GLOBAL TEMPORARY views are tied to a system preserved temporary database `global_temp`.
    @@ -46,13 +45,23 @@ CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_nam
    IF NOT EXISTS
    Creates a view if it does not exists.
    +
    +
    view_identifier
    +
    + Specifies a view name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] view_name + +
    +
    create_view_clauses
    These clauses are optional and order insensitive. It can be of following formats.
      -
    • [(column_name [COMMENT column_comment], ...) ] to specify column-level comments.
    • -
    • [COMMENT view_comment] to specify view-level comments.
    • -
    • [TBLPROPERTIES (property_name = property_value, ...)] to add metadata key-value pairs.
    • +
    • [ ( column_name [ COMMENT column_comment ], ... ) ] to specify column-level comments.
    • +
    • [ COMMENT view_comment ] to specify view-level comments.
    • +
    • [ TBLPROPERTIES ( property_name = property_value, ... ) ] to add metadata key-value pairs.
    diff --git a/docs/sql-ref-syntax-ddl-drop-database.md b/docs/sql-ref-syntax-ddl-drop-database.md index f3cdbf91a8d2a..68244648c9a5b 100644 --- a/docs/sql-ref-syntax-ddl-drop-database.md +++ b/docs/sql-ref-syntax-ddl-drop-database.md @@ -27,14 +27,14 @@ exception will be thrown if the database does not exist in the system. ### Syntax {% highlight sql %} -DROP (DATABASE|SCHEMA) [IF EXISTS] dbname [RESTRICT|CASCADE]; +DROP ( DATABASE | SCHEMA ) [ IF EXISTS ] dbname [ RESTRICT | CASCADE ] {% endhighlight %} ### Parameters
    -
    DATABASE|SCHEMA
    +
    DATABASE | SCHEMA
    `DATABASE` and `SCHEMA` mean the same thing, either of them can be used.
    diff --git a/docs/sql-ref-syntax-ddl-drop-function.md b/docs/sql-ref-syntax-ddl-drop-function.md index f1b54c675cef5..a059960920bef 100644 --- a/docs/sql-ref-syntax-ddl-drop-function.md +++ b/docs/sql-ref-syntax-ddl-drop-function.md @@ -25,7 +25,7 @@ The `DROP FUNCTION` statement drops a temporary or user defined function (UDF). ### Syntax {% highlight sql %} -DROP [TEMPORARY] FUNCTION [IF EXISTS] [db_name.]function_name; +DROP [ TEMPORARY ] FUNCTION [ IF EXISTS ] [ db_name. ] function_name {% endhighlight %} diff --git a/docs/sql-ref-syntax-ddl-drop-table.md b/docs/sql-ref-syntax-ddl-drop-table.md index f9129d5114fa7..d1d8534efe7a2 100644 --- a/docs/sql-ref-syntax-ddl-drop-table.md +++ b/docs/sql-ref-syntax-ddl-drop-table.md @@ -28,7 +28,7 @@ In case of an external table, only the associated metadata information is remove ### Syntax {% highlight sql %} -DROP TABLE [IF EXISTS] [database_name.]table_name +DROP TABLE [ IF EXISTS ] table_identifier {% endhighlight %} ### Parameter @@ -37,13 +37,13 @@ DROP TABLE [IF EXISTS] [database_name.]table_name
    If specified, no exception is thrown when the table does not exists.
    -
    database_name
    +
    table_identifier
    - Specifies the database name where table is present. -
    -
    table_name
    -
    - Specifies the table name to be dropped. + Specifies the table name to be dropped. The table name may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name +
    diff --git a/docs/sql-ref-syntax-ddl-drop-view.md b/docs/sql-ref-syntax-ddl-drop-view.md index bf99150048d49..bf9e497b3a418 100644 --- a/docs/sql-ref-syntax-ddl-drop-view.md +++ b/docs/sql-ref-syntax-ddl-drop-view.md @@ -24,7 +24,7 @@ license: | ### Syntax {% highlight sql %} -DROP VIEW [IF EXISTS] [database_name.]view_name +DROP VIEW [ IF EXISTS ] view_identifier {% endhighlight %} ### Parameter @@ -33,13 +33,13 @@ DROP VIEW [IF EXISTS] [database_name.]view_name
    If specified, no exception is thrown when the view does not exists.
    -
    database_name
    +
    view_identifier
    - Specifies the database name where view is present. -
    -
    view_name
    -
    - Specifies the view name to be dropped. + Specifies the view name to be dropped. The view name may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] view_name +
    diff --git a/docs/sql-ref-syntax-ddl-repair-table.md b/docs/sql-ref-syntax-ddl-repair-table.md index 2816d0bba3c9f..daa6a46fca58f 100644 --- a/docs/sql-ref-syntax-ddl-repair-table.md +++ b/docs/sql-ref-syntax-ddl-repair-table.md @@ -24,13 +24,19 @@ license: | ### Syntax {% highlight sql %} -MSCK REPAIR TABLE table_name +MSCK REPAIR TABLE table_identifier {% endhighlight %} ### Parameters
    -
    table_name
    -
    Specifies the name of the table to be repaired.
    +
    table_identifier
    +
    + Specifies the name of the table to be repaired. The table name may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    ### Examples diff --git a/docs/sql-ref-syntax-ddl-truncate-table.md b/docs/sql-ref-syntax-ddl-truncate-table.md index 7baaefe317ba1..3a0569e809d84 100644 --- a/docs/sql-ref-syntax-ddl-truncate-table.md +++ b/docs/sql-ref-syntax-ddl-truncate-table.md @@ -26,18 +26,30 @@ in `partition_spec`. If no `partition_spec` is specified it will remove all part ### Syntax {% highlight sql %} -TRUNCATE TABLE table_name [PARTITION partition_spec]; +TRUNCATE TABLE table_identifier [ partition_spec ] {% endhighlight %} ### Parameters
    -
    table_name
    -
    The name of an existing table.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    -
    -
    PARTITION ( partition_spec :[ partition_column = partition_col_value, partition_column = partition_col_value, ...] )
    -
    Specifies one or more partition column and value pairs. The partition value is optional.
    +
    partition_spec
    +
    + An optional parameter that specifies a comma separated list of key and value pairs + for partitions.

    + Syntax: + + PARTITION ( partition_col_name = partition_col_val [ , ... ] ) + +
    diff --git a/docs/sql-ref-syntax-dml-insert-into.md b/docs/sql-ref-syntax-dml-insert-into.md index 890e30bdbd3e1..3060e2d1a6562 100644 --- a/docs/sql-ref-syntax-dml-insert-into.md +++ b/docs/sql-ref-syntax-dml-insert-into.md @@ -25,20 +25,32 @@ The `INSERT INTO` statement inserts new rows into a table. The inserted rows can ### Syntax {% highlight sql %} -INSERT INTO [ TABLE ] table_name - [ PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) ] +INSERT INTO [ TABLE ] table_identifier [ partition_spec ] { { VALUES ( { value | NULL } [ , ... ] ) [ , ( ... ) ] } | query } {% endhighlight %} ### Parameters
    -
    table_name
    -
    The name of an existing table.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    -
    PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] )
    -
    Specifies one or more partition column and value pairs. The partition value is optional.
    +
    partition_spec
    +
    + An optional parameter that specifies a comma separated list of key and value pairs + for partitions.

    + Syntax: + + PARTITION (partition_col_name = partition_col_val [ , ... ]) + +
    diff --git a/docs/sql-ref-syntax-dml-insert-overwrite-table.md b/docs/sql-ref-syntax-dml-insert-overwrite-table.md index 9372329b08286..1f32342642b2a 100644 --- a/docs/sql-ref-syntax-dml-insert-overwrite-table.md +++ b/docs/sql-ref-syntax-dml-insert-overwrite-table.md @@ -25,20 +25,32 @@ The `INSERT OVERWRITE` statement overwrites the existing data in the table using ### Syntax {% highlight sql %} -INSERT OVERWRITE [ TABLE ] table_name - [ PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) [ IF NOT EXISTS ] ] +INSERT OVERWRITE [ TABLE ] table_identifier [ partition_spec [ IF NOT EXISTS ] ] { { VALUES ( { value | NULL } [ , ... ] ) [ , ( ... ) ] } | query } {% endhighlight %} ### Parameters
    -
    table_name
    -
    The name of an existing table.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    -
    PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] )
    -
    Specifies one or more partition column and value pairs. The partition value is optional.
    +
    partition_spec
    +
    + An optional parameter that specifies a comma separated list of key and value pairs + for partitions.

    + Syntax: + + PARTITION ( partition_col_name [ = partition_col_val ] [ , ... ] ) + +
    diff --git a/docs/sql-ref-syntax-dml-load.md b/docs/sql-ref-syntax-dml-load.md index c2a6102db4aad..aadfd708e274e 100644 --- a/docs/sql-ref-syntax-dml-load.md +++ b/docs/sql-ref-syntax-dml-load.md @@ -24,8 +24,7 @@ license: | ### Syntax {% highlight sql %} -LOAD DATA [ LOCAL ] INPATH path [ OVERWRITE ] INTO TABLE table_name - [ PARTITION ( partition_col_name = partition_col_val [ , ... ] ) ] +LOAD DATA [ LOCAL ] INPATH path [ OVERWRITE ] INTO TABLE table_identifier [ partition_spec ] {% endhighlight %} ### Parameters @@ -35,13 +34,26 @@ LOAD DATA [ LOCAL ] INPATH path [ OVERWRITE ] INTO TABLE table_name
    -
    table_name
    -
    The name of an existing table.
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    -
    PARTITION ( partition_col_name = partition_col_val [ , ... ] )
    -
    Specifies one or more partition column and value pairs.
    +
    partition_spec
    +
    + An optional parameter that specifies a comma separated list of key and value pairs + for partitions.

    + Syntax: + + PARTITION ( partition_col_name = partition_col_val [ , ... ] ) + +
    From e04a63437b8f31db90ca1669ee98289f4ba633e1 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 2 Dec 2019 09:06:37 -0600 Subject: [PATCH 0769/1387] [SPARK-30075][CORE][TESTS] Fix the hashCode implementation of ArrayKeyIndexType correctly ### What changes were proposed in this pull request? This patch fixes the bug on ArrayKeyIndexType.hashCode() as it is simply calling Array.hashCode() which in turn calls Object.hashCode(). That should be Arrays.hashCode() to reflect the elements in the array. ### Why are the changes needed? I've encountered the bug in #25811 while adding test codes for #25811, and I've split the fix into individual PR to speed up reviewing. Without this patch, ArrayKeyIndexType would bring various issues when it's used as type of collections. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? I've skipped adding UT as ArrayKeyIndexType is in test and the patch is pretty simple one-liner. Closes #26709 from HeartSaVioR/SPARK-30075. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Sean Owen --- .../java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java index 32030fb4115c3..dd53fdf0b1b4c 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java @@ -38,7 +38,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return key.hashCode(); + return Arrays.hashCode(key) ^ Arrays.hashCode(id); } } From 68034a805607ced50dbedca73dfc7eaf0102dde8 Mon Sep 17 00:00:00 2001 From: Ali Afroozeh Date: Mon, 2 Dec 2019 20:56:40 +0100 Subject: [PATCH 0770/1387] [SPARK-30072][SQL] Create dedicated planner for subqueries ### What changes were proposed in this pull request? This PR changes subquery planning by calling the planner and plan preparation rules on the subquery plan directly. Before we were creating a `QueryExecution` instance for subqueries to get the executedPlan. This would re-run analysis and optimization on the subqueries plan. Running the analysis again on an optimized query plan can have unwanted consequences, as some rules, for example `DecimalPrecision`, are not idempotent. As an example, consider the expression `1.7 * avg(a)` which after applying the `DecimalPrecision` rule becomes: ``` promote_precision(1.7) * promote_precision(avg(a)) ``` After the optimization, more specifically the constant folding rule, this expression becomes: ``` 1.7 * promote_precision(avg(a)) ``` Now if we run the analyzer on this optimized query again, we will get: ``` promote_precision(1.7) * promote_precision(promote_precision(avg(a))) ``` Which will later optimized as: ``` 1.7 * promote_precision(promote_precision(avg(a))) ``` As can be seen, re-running the analysis and optimization on this expression results in an expression with extra nested promote_preceision nodes. Adding unneeded nodes to the plan is problematic because it can eliminate situations where we can reuse the plan. We opted to introduce dedicated planners for subuqueries, instead of making the DecimalPrecision rule idempotent, because this eliminates this entire category of problems. Another benefit is that planning time for subqueries is reduced. ### How was this patch tested? Unit tests Closes #26705 from dbaliafroozeh/CreateDedicatedPlannerForSubqueries. Authored-by: Ali Afroozeh Signed-off-by: herman --- .../sql/catalyst/optimizer/Optimizer.scala | 6 +- .../plans/logical/basicLogicalOperators.scala | 10 +- .../RemoveRedundantAliasAndProjectSuite.scala | 8 +- .../sql/dynamicpruning/PartitionPruning.scala | 2 +- .../PlanDynamicPruningFilters.scala | 10 +- .../spark/sql/execution/QueryExecution.scala | 93 ++++++++++++++----- .../adaptive/AdaptiveSparkPlanExec.scala | 16 ++-- .../adaptive/InsertAdaptiveSparkPlan.scala | 15 ++- .../execution/python/ExtractPythonUDFs.scala | 2 +- .../apache/spark/sql/execution/subquery.scala | 4 +- .../spark/sql/execution/PlannerSuite.scala | 24 +++++ 11 files changed, 134 insertions(+), 56 deletions(-) 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 9d0bd358aa24c..05fd5e35e22ad 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 @@ -246,7 +246,7 @@ abstract class Optimizer(catalogManager: CatalogManager) } def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case s: SubqueryExpression => - val Subquery(newPlan) = Optimizer.this.execute(Subquery(s.plan)) + val Subquery(newPlan, _) = Optimizer.this.execute(Subquery.fromExpression(s)) // At this point we have an optimized subquery plan that we are going to attach // to this subquery expression. Here we can safely remove any top level sort // in the plan as tuples produced by a subquery are un-ordered. @@ -377,8 +377,8 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { plan match { // We want to keep the same output attributes for subqueries. This means we cannot remove // the aliases that produce these attributes - case Subquery(child) => - Subquery(removeRedundantAliases(child, blacklist ++ child.outputSet)) + case Subquery(child, correlated) => + Subquery(removeRedundantAliases(child, blacklist ++ child.outputSet), correlated) // A join has to be treated differently, because the left and the right side of the join are // not allowed to use the same attributes. We use a blacklist to prevent us from creating a 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 75aafefa86636..93b314d4e54a5 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 @@ -45,11 +45,19 @@ case class ReturnAnswer(child: LogicalPlan) extends UnaryNode { /** * This node is inserted at the top of a subquery when it is optimized. This makes sure we can * recognize a subquery as such, and it allows us to write subquery aware transformations. + * + * @param correlated flag that indicates the subquery is correlated, and will be rewritten into a + * join during analysis. */ -case class Subquery(child: LogicalPlan) extends OrderPreservingUnaryNode { +case class Subquery(child: LogicalPlan, correlated: Boolean) extends OrderPreservingUnaryNode { override def output: Seq[Attribute] = child.output } +object Subquery { + def fromExpression(s: SubqueryExpression): Subquery = + Subquery(s.plan, SubqueryExpression.hasCorrelatedSubquery(s)) +} + case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends OrderPreservingUnaryNode { override def output: Seq[Attribute] = projectList.map(_.toAttribute) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala index 3802dbf5d6e06..2e0ab7f64f4d6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala @@ -119,9 +119,13 @@ class RemoveRedundantAliasAndProjectSuite extends PlanTest with PredicateHelper test("do not remove output attributes from a subquery") { val relation = LocalRelation('a.int, 'b.int) - val query = Subquery(relation.select('a as "a", 'b as "b").where('b < 10).select('a).analyze) + val query = Subquery( + relation.select('a as "a", 'b as "b").where('b < 10).select('a).analyze, + correlated = false) val optimized = Optimize.execute(query) - val expected = Subquery(relation.select('a as "a", 'b).where('b < 10).select('a).analyze) + val expected = Subquery( + relation.select('a as "a", 'b).where('b < 10).select('a).analyze, + correlated = false) comparePlans(optimized, expected) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala index d64e11136e673..78fd17c73fb61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala @@ -252,7 +252,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { override def apply(plan: LogicalPlan): LogicalPlan = plan match { // Do not rewrite subqueries. - case _: Subquery => plan + case s: Subquery if s.correlated => plan case _ if !SQLConf.get.dynamicPartitionPruningEnabled => plan case _ => prune(plan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PlanDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PlanDynamicPruningFilters.scala index 031c3b1aa0d50..1398dc049dd99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PlanDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PlanDynamicPruningFilters.scala @@ -55,22 +55,24 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession) plan transformAllExpressions { case DynamicPruningSubquery( value, buildPlan, buildKeys, broadcastKeyIndex, onlyInBroadcast, exprId) => - val qe = new QueryExecution(sparkSession, buildPlan) + val sparkPlan = QueryExecution.createSparkPlan( + sparkSession, sparkSession.sessionState.planner, buildPlan) // Using `sparkPlan` is a little hacky as it is based on the assumption that this rule is // the first to be applied (apart from `InsertAdaptiveSparkPlan`). val canReuseExchange = reuseBroadcast && buildKeys.nonEmpty && plan.find { case BroadcastHashJoinExec(_, _, _, BuildLeft, _, left, _) => - left.sameResult(qe.sparkPlan) + left.sameResult(sparkPlan) case BroadcastHashJoinExec(_, _, _, BuildRight, _, _, right) => - right.sameResult(qe.sparkPlan) + right.sameResult(sparkPlan) case _ => false }.isDefined if (canReuseExchange) { val mode = broadcastMode(buildKeys, buildPlan) + val executedPlan = QueryExecution.prepareExecutedPlan(sparkSession, sparkPlan) // plan a broadcast exchange of the build side of the join - val exchange = BroadcastExchangeExec(mode, qe.executedPlan) + val exchange = BroadcastExchangeExec(mode, executedPlan) val name = s"dynamicpruning#${exprId.id}" // place the broadcast adaptor for reusing the broadcast results on the probe side val broadcastValues = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 258f9cea05b82..190c74297e9f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -81,11 +81,8 @@ class QueryExecution( } lazy val sparkPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { - SparkSession.setActiveSession(sparkSession) - // TODO: We use next(), i.e. take the first plan returned by the planner, here for now, - // but we will implement to choose the best plan. // Clone the logical plan here, in case the planner rules change the states of the logical plan. - planner.plan(ReturnAnswer(optimizedPlan.clone())).next() + QueryExecution.createSparkPlan(sparkSession, planner, optimizedPlan.clone()) } // executedPlan should not be used to initialize any SparkPlan. It should be @@ -93,7 +90,7 @@ class QueryExecution( lazy val executedPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { // clone the plan to avoid sharing the plan instance between different stages like analyzing, // optimizing and planning. - prepareForExecution(sparkPlan.clone()) + QueryExecution.prepareForExecution(preparations, sparkPlan.clone()) } /** @@ -109,28 +106,10 @@ class QueryExecution( lazy val toRdd: RDD[InternalRow] = new SQLExecutionRDD( executedPlan.execute(), sparkSession.sessionState.conf) - /** - * Prepares a planned [[SparkPlan]] for execution by inserting shuffle operations and internal - * row format conversions as needed. - */ - protected def prepareForExecution(plan: SparkPlan): SparkPlan = { - preparations.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } + protected def preparations: Seq[Rule[SparkPlan]] = { + QueryExecution.preparations(sparkSession) } - /** A sequence of rules that will be applied in order to the physical plan before execution. */ - protected def preparations: Seq[Rule[SparkPlan]] = Seq( - // `AdaptiveSparkPlanExec` is a leaf node. If inserted, all the following rules will be no-op - // as the original plan is hidden behind `AdaptiveSparkPlanExec`. - InsertAdaptiveSparkPlan(sparkSession, this), - PlanDynamicPruningFilters(sparkSession), - PlanSubqueries(sparkSession), - EnsureRequirements(sparkSession.sessionState.conf), - ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, - sparkSession.sessionState.columnarRules), - CollapseCodegenStages(sparkSession.sessionState.conf), - ReuseExchange(sparkSession.sessionState.conf), - ReuseSubquery(sparkSession.sessionState.conf)) - def simpleString: String = simpleString(false) def simpleString(formatted: Boolean): String = withRedaction { @@ -248,3 +227,67 @@ class QueryExecution( } } } + +object QueryExecution { + /** + * Construct a sequence of rules that are used to prepare a planned [[SparkPlan]] for execution. + * These rules will make sure subqueries are planned, make use the data partitioning and ordering + * are correct, insert whole stage code gen, and try to reduce the work done by reusing exchanges + * and subqueries. + */ + private[execution] def preparations(sparkSession: SparkSession): Seq[Rule[SparkPlan]] = + Seq( + // `AdaptiveSparkPlanExec` is a leaf node. If inserted, all the following rules will be no-op + // as the original plan is hidden behind `AdaptiveSparkPlanExec`. + InsertAdaptiveSparkPlan(sparkSession), + PlanDynamicPruningFilters(sparkSession), + PlanSubqueries(sparkSession), + EnsureRequirements(sparkSession.sessionState.conf), + ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, + sparkSession.sessionState.columnarRules), + CollapseCodegenStages(sparkSession.sessionState.conf), + ReuseExchange(sparkSession.sessionState.conf), + ReuseSubquery(sparkSession.sessionState.conf) + ) + + /** + * Prepares a planned [[SparkPlan]] for execution by inserting shuffle operations and internal + * row format conversions as needed. + */ + private[execution] def prepareForExecution( + preparations: Seq[Rule[SparkPlan]], + plan: SparkPlan): SparkPlan = { + preparations.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } + } + + /** + * Transform a [[LogicalPlan]] into a [[SparkPlan]]. + * + * Note that the returned physical plan still needs to be prepared for execution. + */ + def createSparkPlan( + sparkSession: SparkSession, + planner: SparkPlanner, + plan: LogicalPlan): SparkPlan = { + SparkSession.setActiveSession(sparkSession) + // TODO: We use next(), i.e. take the first plan returned by the planner, here for now, + // but we will implement to choose the best plan. + planner.plan(ReturnAnswer(plan)).next() + } + + /** + * Prepare the [[SparkPlan]] for execution. + */ + def prepareExecutedPlan(spark: SparkSession, plan: SparkPlan): SparkPlan = { + prepareForExecution(preparations(spark), plan) + } + + /** + * Transform the subquery's [[LogicalPlan]] into a [[SparkPlan]] and prepare the resulting + * [[SparkPlan]] for execution. + */ + def prepareExecutedPlan(spark: SparkSession, plan: LogicalPlan): SparkPlan = { + val sparkPlan = createSparkPlan(spark, spark.sessionState.planner, plan.clone()) + prepareExecutedPlan(spark, sparkPlan) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index b4eea620b93a6..8617220a0d0b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -63,7 +63,7 @@ case class AdaptiveSparkPlanExec( @transient preprocessingRules: Seq[Rule[SparkPlan]], @transient subqueryCache: TrieMap[SparkPlan, BaseSubqueryExec], @transient stageCache: TrieMap[SparkPlan, QueryStageExec], - @transient queryExecution: QueryExecution) + @transient isSubquery: Boolean) extends LeafExecNode { @transient private val lock = new Object() @@ -128,14 +128,12 @@ case class AdaptiveSparkPlanExec( private def getFinalPhysicalPlan(): SparkPlan = lock.synchronized { if (!isFinalPlan) { - // Make sure we only update Spark UI if this plan's `QueryExecution` object matches the one - // retrieved by the `sparkContext`'s current execution ID. Note that sub-queries do not have - // their own execution IDs and therefore rely on the main query to update UI. - val executionId = Option( - session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).flatMap { idStr => - val id = idStr.toLong - val qe = SQLExecution.getQueryExecution(id) - if (qe.eq(queryExecution)) Some(id) else None + // Subqueries do not have their own execution IDs and therefore rely on the main query to + // update UI. + val executionId = if (isSubquery) { + None + } else { + Option(session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) } var currentLogicalPlan = currentPhysicalPlan.logicalLink.get var result = createQueryStages(currentPhysicalPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 23eadfd6f3e5e..554d7e4dc9402 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -35,9 +35,7 @@ import org.apache.spark.sql.internal.SQLConf * * Note that this rule is stateful and thus should not be reused across query executions. */ -case class InsertAdaptiveSparkPlan( - session: SparkSession, - queryExecution: QueryExecution) extends Rule[SparkPlan] { +case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan] { private val conf = session.sessionState.conf @@ -47,9 +45,9 @@ case class InsertAdaptiveSparkPlan( // Exchange-reuse is shared across the entire query, including sub-queries. private val stageCache = new TrieMap[SparkPlan, QueryStageExec]() - override def apply(plan: SparkPlan): SparkPlan = applyInternal(plan, queryExecution) + override def apply(plan: SparkPlan): SparkPlan = applyInternal(plan, false) - private def applyInternal(plan: SparkPlan, qe: QueryExecution): SparkPlan = plan match { + private def applyInternal(plan: SparkPlan, isSubquery: Boolean): SparkPlan = plan match { case _: ExecutedCommandExec => plan case _ if conf.adaptiveExecutionEnabled && supportAdaptive(plan) => try { @@ -62,7 +60,8 @@ case class InsertAdaptiveSparkPlan( // Run pre-processing rules. val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preprocessingRules) logDebug(s"Adaptive execution enabled for plan: $plan") - AdaptiveSparkPlanExec(newPlan, session, preprocessingRules, subqueryCache, stageCache, qe) + AdaptiveSparkPlanExec(newPlan, session, preprocessingRules, + subqueryCache, stageCache, isSubquery) } catch { case SubqueryAdaptiveNotSupportedException(subquery) => logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + @@ -110,10 +109,10 @@ case class InsertAdaptiveSparkPlan( } def compileSubquery(plan: LogicalPlan): SparkPlan = { - val queryExec = new QueryExecution(session, plan) // Apply the same instance of this rule to sub-queries so that sub-queries all share the // same `stageCache` for Exchange reuse. - this.applyInternal(queryExec.sparkPlan, queryExec) + this.applyInternal( + QueryExecution.createSparkPlan(session, session.sessionState.planner, plan.clone()), true) } private def verifyAdaptivePlan(plan: SparkPlan, logicalPlan: LogicalPlan): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index d49d790d7888b..7bc8b95cfb03b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -205,7 +205,7 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan match { // SPARK-26293: A subquery will be rewritten into join later, and will go through this rule // eventually. Here we skip subquery, as Python UDF only needs to be extracted once. - case _: Subquery => plan + case s: Subquery if s.correlated => plan case _ => plan transformUp { // A safe guard. `ExtractPythonUDFs` only runs once, so we will not hit `BatchEvalPython` and diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index adb4956c6d3cf..c2270c57eb941 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -178,7 +178,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = { plan.transformAllExpressions { case subquery: expressions.ScalarSubquery => - val executedPlan = new QueryExecution(sparkSession, subquery.plan).executedPlan + val executedPlan = QueryExecution.prepareExecutedPlan(sparkSession, subquery.plan) ScalarSubquery( SubqueryExec(s"scalar-subquery#${subquery.exprId.id}", executedPlan), subquery.exprId) @@ -192,7 +192,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { } ) } - val executedPlan = new QueryExecution(sparkSession, query).executedPlan + val executedPlan = QueryExecution.prepareExecutedPlan(sparkSession, query) InSubqueryExec(expr, SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 90ce6765013b4..3dea0b1ce937c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -856,6 +856,30 @@ class PlannerSuite extends SharedSparkSession { StructField("f2", StringType, nullable = true), StructField("f3", StringType, nullable = false)))) } + + test("Do not analyze subqueries twice") { + // Analyzing the subquery twice will result in stacked + // CheckOverflow & PromotePrecision expressions. + val df = sql( + """ + |SELECT id, + | (SELECT 1.3000000 * AVG(CAST(id AS DECIMAL(10, 3))) FROM range(13)) AS ref + |FROM range(5) + |""".stripMargin) + + val Seq(subquery) = df.queryExecution.executedPlan.subqueriesAll + subquery.foreach { node => + node.expressions.foreach { expression => + expression.foreach { + case PromotePrecision(_: PromotePrecision) => + fail(s"$expression contains stacked PromotePrecision expressions.") + case CheckOverflow(_: CheckOverflow, _, _) => + fail(s"$expression contains stacked CheckOverflow expressions.") + case _ => // Ok + } + } + } + } } // Used for unit-testing EnsureRequirements From 332e593093460c34fae303a913a862b1b579c83f Mon Sep 17 00:00:00 2001 From: sychen Date: Tue, 3 Dec 2019 10:07:09 +0900 Subject: [PATCH 0771/1387] [SPARK-29943][SQL] Improve error messages for unsupported data type ### What changes were proposed in this pull request? Improve error messages for unsupported data type. ### Why are the changes needed? When the spark reads the hive table and encounters an unsupported field type, the exception message has only one unsupported type, and the user cannot know which field of which table. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? ```create view t AS SELECT STRUCT('a' AS `$a`, 1 AS b) as q;``` current: org.apache.spark.SparkException: Cannot recognize hive type string: struct<$a:string,b:int> change: org.apache.spark.SparkException: Cannot recognize hive type string: struct<$a:string,b:int>, column: q ```select * from t,t_normal_1,t_normal_2``` current: org.apache.spark.SparkException: Cannot recognize hive type string: struct<$a:string,b:int> change: org.apache.spark.SparkException: Cannot recognize hive type string: struct<$a:string,b:int>, column: q, db: default, table: t Closes #26577 from cxzl25/unsupport_data_type_msg. Authored-by: sychen Signed-off-by: HyukjinKwon --- .../spark/sql/hive/client/HiveClientImpl.scala | 12 +++++++++--- 1 file changed, 9 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 59f2356a3464c..d4342e427192c 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 @@ -440,8 +440,13 @@ private[hive] class HiveClientImpl( private def convertHiveTableToCatalogTable(h: HiveTable): CatalogTable = { // Note: Hive separates partition columns and the schema, but for us the // partition columns are part of the schema - val cols = h.getCols.asScala.map(fromHiveColumn) - val partCols = h.getPartCols.asScala.map(fromHiveColumn) + val (cols, partCols) = try { + (h.getCols.asScala.map(fromHiveColumn), h.getPartCols.asScala.map(fromHiveColumn)) + } catch { + case ex: SparkException => + throw new SparkException( + s"${ex.getMessage}, db: ${h.getDbName}, table: ${h.getTableName}", ex) + } val schema = StructType(cols ++ partCols) val bucketSpec = if (h.getNumBuckets > 0) { @@ -982,7 +987,8 @@ private[hive] object HiveClientImpl { CatalystSqlParser.parseDataType(hc.getType) } catch { case e: ParseException => - throw new SparkException("Cannot recognize hive type string: " + hc.getType, e) + throw new SparkException( + s"Cannot recognize hive type string: ${hc.getType}, column: ${hc.getName}", e) } } From 4021354b73dd86ee765f50ff90ab777edfc21bdb Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Tue, 3 Dec 2019 10:02:23 +0800 Subject: [PATCH 0772/1387] [SPARK-30044][ML] MNB/CNB/BNB use empty sigma matrix instead of null ### What changes were proposed in this pull request? MNB/CNB/BNB use empty sigma matrix instead of null ### Why are the changes needed? 1,Using empty sigma matrix will simplify the impl 2,I am reviewing FM impl these days, FMModels have optional bias and linear part. It seems more reasonable to set optional part an empty vector/matrix or zero value than `null` ### Does this PR introduce any user-facing change? yes, sigma from `null` to empty matrix ### How was this patch tested? updated testsuites Closes #26679 from zhengruifeng/nb_use_empty_sigma. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../spark/ml/classification/NaiveBayes.scala | 28 +++++++------------ .../ml/classification/NaiveBayesSuite.scala | 14 +++++----- python/pyspark/ml/classification.py | 8 +++--- 3 files changed, 21 insertions(+), 29 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 7da5703f23410..88eb5f4b48554 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -19,7 +19,6 @@ package org.apache.spark.ml.classification import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats -import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Since import org.apache.spark.ml.PredictorParams @@ -243,12 +242,12 @@ class NaiveBayes @Since("1.5.0") ( $(modelType) match { case Multinomial | Bernoulli => val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) - new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, Matrices.zeros(0, 0)) .setOldLabels(labelArray) case Complement => // Since the CNB compute the coefficient in a complement way. val theta = new DenseMatrix(numLabels, numFeatures, thetaArray.map(v => -v), true) - new NaiveBayesModel(uid, pi.compressed, theta.compressed, null) + new NaiveBayesModel(uid, pi.compressed, theta.compressed, Matrices.zeros(0, 0)) } } @@ -575,8 +574,7 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { private[NaiveBayesModel] class NaiveBayesModelWriter(instance: NaiveBayesModel) extends MLWriter { import NaiveBayes._ - private case class Data(pi: Vector, theta: Matrix) - private case class GaussianData(pi: Vector, theta: Matrix, sigma: Matrix) + private case class Data(pi: Vector, theta: Matrix, sigma: Matrix) override protected def saveImpl(path: String): Unit = { // Save metadata and Params @@ -585,21 +583,17 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { instance.getModelType match { case Multinomial | Bernoulli | Complement => - // Save model data: pi, theta - require(instance.sigma == null) - val data = Data(instance.pi, instance.theta) - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - + require(instance.sigma.numRows == 0 && instance.sigma.numCols == 0) case Gaussian => - require(instance.sigma != null) - val data = GaussianData(instance.pi, instance.theta, instance.sigma) - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + require(instance.sigma.numRows != 0 && instance.sigma.numCols != 0) } + + val data = Data(instance.pi, instance.theta, instance.sigma) + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } } private class NaiveBayesModelReader extends MLReader[NaiveBayesModel] { - import NaiveBayes._ /** Checked against metadata when loading model */ private val className = classOf[NaiveBayesModel].getName @@ -608,19 +602,17 @@ object NaiveBayesModel extends MLReadable[NaiveBayesModel] { implicit val format = DefaultFormats val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion) - val modelTypeJson = metadata.getParamValue("modelType") - val modelType = Param.jsonDecode[String](compact(render(modelTypeJson))) val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) val vecConverted = MLUtils.convertVectorColumnsToML(data, "pi") - val model = if (major.toInt < 3 || modelType != Gaussian) { + val model = if (major.toInt < 3) { val Row(pi: Vector, theta: Matrix) = MLUtils.convertMatrixColumnsToML(vecConverted, "theta") .select("pi", "theta") .head() - new NaiveBayesModel(metadata.uid, pi, theta, null) + new NaiveBayesModel(metadata.uid, pi, theta, Matrices.zeros(0, 0)) } else { val Row(pi: Vector, theta: Matrix, sigma: Matrix) = MLUtils.convertMatrixColumnsToML(vecConverted, "theta", "sigma") diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 4a555ad3ed071..2e74128d5456a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -96,8 +96,8 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { assert(Vectors.dense(model.pi.toArray.map(math.exp)) ~== Vectors.dense(piData.toArray.map(math.exp)) absTol 0.05, "pi mismatch") assert(model.theta.map(math.exp) ~== thetaData.map(math.exp) absTol 0.05, "theta mismatch") - if (sigmaData == null) { - assert(model.sigma == null, "sigma mismatch") + if (sigmaData === Matrices.zeros(0, 0)) { + assert(model.sigma === Matrices.zeros(0, 0), "sigma mismatch") } else { assert(model.sigma.map(math.exp) ~== sigmaData.map(math.exp) absTol 0.05, "sigma mismatch") @@ -166,7 +166,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { ParamsSuite.checkParams(new NaiveBayes) val model = new NaiveBayesModel("nb", pi = Vectors.dense(Array(0.2, 0.8)), theta = new DenseMatrix(2, 3, Array(0.1, 0.2, 0.3, 0.4, 0.6, 0.4)), - sigma = null) + sigma = Matrices.zeros(0, 0)) ParamsSuite.checkParams(model) } @@ -195,7 +195,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("multinomial") val model = nb.fit(testDataset) - validateModelFit(pi, theta, null, model) + validateModelFit(pi, theta, Matrices.zeros(0, 0), model) assert(model.hasParent) MLTestingUtils.checkCopyAndUids(nb, model) @@ -281,7 +281,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { val nb = new NaiveBayes().setSmoothing(1.0).setModelType("bernoulli") val model = nb.fit(testDataset) - validateModelFit(pi, theta, null, model) + validateModelFit(pi, theta, Matrices.zeros(0, 0), model) assert(model.hasParent) val validationDataset = @@ -512,7 +512,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { if (model.getModelType == "gaussian") { assert(model.sigma === model2.sigma) } else { - assert(model.sigma === null && model2.sigma === null) + assert(model.sigma === Matrices.zeros(0, 0) && model2.sigma === Matrices.zeros(0, 0)) } } val nb = new NaiveBayes() @@ -531,7 +531,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { nb, spark) { (expected, actual) => assert(expected.pi === actual.pi) assert(expected.theta === actual.theta) - assert(expected.sigma === null && actual.sigma === null) + assert(expected.sigma === Matrices.zeros(0, 0) && actual.sigma === Matrices.zeros(0, 0)) } } } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d6fe26dc69da8..8e2b2396be14d 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1934,8 +1934,8 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, DenseVector([-0.81..., -0.58...]) >>> model.theta DenseMatrix(2, 2, [-0.91..., -0.51..., -0.40..., -1.09...], 1) - >>> model.sigma == None - True + >>> model.sigma + DenseMatrix(0, 0, [...], ...) >>> test0 = sc.parallelize([Row(features=Vectors.dense([1.0, 0.0]))]).toDF() >>> model.predict(test0.head().features) 1.0 @@ -1978,8 +1978,8 @@ class NaiveBayes(JavaProbabilisticClassifier, _NaiveBayesParams, HasThresholds, 'complement' >>> model5.theta DenseMatrix(2, 2, [...], 1) - >>> model5.sigma == None - True + >>> model5.sigma + DenseMatrix(0, 0, [...], ...) .. versionadded:: 1.5.0 """ From 075ae1eeaf198792650287cd5b3f607a05c574bf Mon Sep 17 00:00:00 2001 From: wuyi Date: Tue, 3 Dec 2019 17:02:50 +0800 Subject: [PATCH 0773/1387] [SPARK-29537][SQL] throw exception when user defined a wrong base path ### What changes were proposed in this pull request? When user defined a base path which is not an ancestor directory for all the input paths, throw exception immediately. ### Why are the changes needed? Assuming that we have a DataFrame[c1, c2] be written out in parquet and partitioned by c1. When using `spark.read.parquet("/path/to/data/c1=1")` to read the data, we'll have a DataFrame with column c2 only. But if we use `spark.read.option("basePath", "/path/from").parquet("/path/to/data/c1=1")` to read the data, we'll have a DataFrame with column c1 and c2. This's happens because a wrong base path does not actually work in `parsePartition()`, so paring would continue until it reaches a directory without "=". And I think the result of the second read way doesn't make sense. ### Does this PR introduce any user-facing change? Yes, with this change, user would hit `IllegalArgumentException ` when given a wrong base path while previous behavior doesn't. ### How was this patch tested? Added UT. Closes #26195 from Ngone51/dev-wrong-basePath. Lead-authored-by: wuyi Co-authored-by: wuyi Signed-off-by: Wenchen Fan --- .../PartitioningAwareFileIndex.scala | 10 +++++++++- .../datasources/FileIndexSuite.scala | 20 +++++++++++++++++++ .../sql/test/DataFrameReaderWriterSuite.scala | 15 ++++++++++++++ 3 files changed, 44 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index 21ddeb6491155..2e09c729529a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -221,7 +221,15 @@ abstract class PartitioningAwareFileIndex( if (!fs.isDirectory(userDefinedBasePath)) { throw new IllegalArgumentException(s"Option '$BASE_PATH_PARAM' must be a directory") } - Set(fs.makeQualified(userDefinedBasePath)) + val qualifiedBasePath = fs.makeQualified(userDefinedBasePath) + val qualifiedBasePathStr = qualifiedBasePath.toString + rootPaths + .find(!fs.makeQualified(_).toString.startsWith(qualifiedBasePathStr)) + .foreach { rp => + throw new IllegalArgumentException( + s"Wrong basePath $userDefinedBasePath for the root path: $rp") + } + Set(qualifiedBasePath) case None => rootPaths.map { path => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index a7a2349a1dfb9..553773e2555cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -352,6 +352,26 @@ class FileIndexSuite extends SharedSparkSession { "driver side must not be negative")) } + test ("SPARK-29537: throw exception when user defined a wrong base path") { + withTempDir { dir => + val partitionDirectory = new File(dir, "a=foo") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val path = new Path(dir.getCanonicalPath) + val wrongBasePath = new File(dir, "unknown") + // basePath must be a directory + wrongBasePath.mkdir() + val parameters = Map("basePath" -> wrongBasePath.getCanonicalPath) + val fileIndex = new InMemoryFileIndex(spark, Seq(path), parameters, None) + val msg = intercept[IllegalArgumentException] { + // trigger inferPartitioning() + fileIndex.partitionSpec() + }.getMessage + assert(msg === s"Wrong basePath ${wrongBasePath.getCanonicalPath} for the root path: $path") + } + } + test("refresh for InMemoryFileIndex with FileStatusCache") { withTempDir { dir => val fileStatusCache = FileStatusCache.getOrCreate(spark) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index cef0e5ab47568..55a60940a7750 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -234,6 +234,21 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with assert(DataSourceUtils.decodePartitioningColumns(partColumns) === Seq("col1", "col2")) } + test ("SPARK-29537: throw exception when user defined a wrong base path") { + withTempPath { p => + val path = new Path(p.toURI).toString + Seq((1, 1), (2, 2)).toDF("c1", "c2") + .write.partitionBy("c1").mode(SaveMode.Overwrite).parquet(path) + val wrongBasePath = new File(p, "unknown") + // basePath must be a directory + wrongBasePath.mkdir() + val msg = intercept[IllegalArgumentException] { + spark.read.option("basePath", wrongBasePath.getCanonicalPath).parquet(path) + }.getMessage + assert(msg === s"Wrong basePath ${wrongBasePath.getCanonicalPath} for the root path: $path") + } + } + test("save mode") { spark.range(10).write .format("org.apache.spark.sql.test") From d7b268ab3264b892c4477cf8af30fb78c2694748 Mon Sep 17 00:00:00 2001 From: herman Date: Tue, 3 Dec 2019 11:25:49 +0100 Subject: [PATCH 0774/1387] [SPARK-29348][SQL] Add observable Metrics for Streaming queries ### What changes were proposed in this pull request? Observable metrics are named arbitrary aggregate functions that can be defined on a query (Dataframe). As soon as the execution of a Dataframe reaches a completion point (e.g. finishes batch query or reaches streaming epoch) a named event is emitted that contains the metrics for the data processed since the last completion point. A user can observe these metrics by attaching a listener to spark session, it depends on the execution mode which listener to attach: - Batch: `QueryExecutionListener`. This will be called when the query completes. A user can access the metrics by using the `QueryExecution.observedMetrics` map. - (Micro-batch) Streaming: `StreamingQueryListener`. This will be called when the streaming query completes an epoch. A user can access the metrics by using the `StreamingQueryProgress.observedMetrics` map. Please note that we currently do not support continuous execution streaming. ### Why are the changes needed? This enabled observable metrics. ### Does this PR introduce any user-facing change? Yes. It adds the `observe` method to `Dataset`. ### How was this patch tested? - Added unit tests for the `CollectMetrics` logical node to the `AnalysisSuite`. - Added unit tests for `StreamingProgress` JSON serialization to the `StreamingQueryStatusAndProgressSuite`. - Added integration tests for streaming to the `StreamingQueryListenerSuite`. - Added integration tests for batch to the `DataFrameCallbackSuite`. Closes #26127 from hvanhovell/SPARK-29348. Authored-by: herman Signed-off-by: herman --- project/MimaExcludes.scala | 5 +- .../sql/catalyst/analysis/Analyzer.scala | 10 ++ .../sql/catalyst/analysis/CheckAnalysis.scala | 71 +++++++++++++- .../catalyst/plans/logical/PlanHelper.scala | 4 +- .../plans/logical/basicLogicalOperators.scala | 22 +++++ .../apache/spark/sql/internal/SQLConf.scala | 7 +- .../org/apache/spark/sql/types/DataType.scala | 35 +++++++ .../sql/catalyst/analysis/AnalysisSuite.scala | 87 ++++++++++++++++- .../scala/org/apache/spark/sql/Dataset.scala | 48 ++++++++++ .../execution/AggregatingAccumulator.scala | 15 ++- .../sql/execution/CollectMetricsExec.scala | 95 +++++++++++++++++++ .../spark/sql/execution/QueryExecution.scala | 5 +- .../spark/sql/execution/SparkStrategies.scala | 2 + .../streaming/ProgressReporter.scala | 16 +++- .../apache/spark/sql/streaming/progress.scala | 10 +- .../AggregatingAccumulatorSuite.scala | 9 +- .../StreamingQueryListenerSuite.scala | 63 +++++++++++- ...StreamingQueryStatusAndProgressSuite.scala | 52 +++++++++- .../sql/util/DataFrameCallbackSuite.scala | 50 +++++++++- 19 files changed, 586 insertions(+), 20 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 617eb173f4f49..259d30a259196 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -458,7 +458,10 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.this"), // [SPARK-28957][SQL] Copy any "spark.hive.foo=bar" spark properties into hadoop conf as "hive.foo=bar" - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.appendS3AndSparkHadoopConfigurations") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.appendS3AndSparkHadoopConfigurations"), + + // [SPARK-29348] Add observable metrics. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryProgress.this") ) // Exclude rules for 2.4.x diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 625ef2153c711..83fa405e521ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2432,6 +2432,10 @@ class Analyzer( nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e) }.copy(child = newChild) + // Don't touch collect metrics. Top-level metrics are not supported (check analysis will fail) + // and we want to retain them inside the aggregate functions. + case m: CollectMetrics => m + // todo: It's hard to write a general rule to pull out nondeterministic expressions // from LogicalPlan, currently we only do it for UnaryNode which has same output // schema with its child. @@ -2932,6 +2936,12 @@ object CleanupAliases extends Rule[LogicalPlan] { Window(cleanedWindowExprs, partitionSpec.map(trimAliases), orderSpec.map(trimAliases(_).asInstanceOf[SortOrder]), child) + case CollectMetrics(name, metrics, child) => + val cleanedMetrics = metrics.map { + e => trimNonTopLevelAliases(e).asInstanceOf[NamedExpression] + } + CollectMetrics(name, cleanedMetrics, child) + // Operators that operate on objects should only have expressions from encoders, which should // never have extra aliases. case o: ObjectConsumer => o diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 72612d1dc76c9..cfb16233b3940 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.spark.sql.catalyst.analysis +import scala.collection.mutable + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ @@ -280,6 +281,41 @@ trait CheckAnalysis extends PredicateHelper { groupingExprs.foreach(checkValidGroupingExprs) aggregateExprs.foreach(checkValidAggregateExpression) + case CollectMetrics(name, metrics, _) => + if (name == null || name.isEmpty) { + operator.failAnalysis(s"observed metrics should be named: $operator") + } + // Check if an expression is a valid metric. A metric must meet the following criteria: + // - Is not a window function; + // - Is not nested aggregate function; + // - Is not a distinct aggregate function; + // - Has only non-deterministic functions that are nested inside an aggregate function; + // - Has only attributes that are nested inside an aggregate function. + def checkMetric(s: Expression, e: Expression, seenAggregate: Boolean = false): Unit = { + e match { + case _: WindowExpression => + e.failAnalysis( + "window expressions are not allowed in observed metrics, but found: " + s.sql) + case _ if !e.deterministic && !seenAggregate => + e.failAnalysis(s"non-deterministic expression ${s.sql} can only be used " + + "as an argument to an aggregate function.") + case a: AggregateExpression if seenAggregate => + e.failAnalysis( + "nested aggregates are not allowed in observed metrics, but found: " + s.sql) + case a: AggregateExpression if a.isDistinct => + e.failAnalysis( + "distinct aggregates are not allowed in observed metrics, but found: " + s.sql) + case _: Attribute if !seenAggregate => + e.failAnalysis (s"attribute ${s.sql} can only be used as an argument to an " + + "aggregate function.") + case _: AggregateExpression => + e.children.foreach(checkMetric (s, _, seenAggregate = true)) + case _ => + e.children.foreach(checkMetric (s, _, seenAggregate)) + } + } + metrics.foreach(m => checkMetric(m, m)) + case Sort(orders, _, _) => orders.foreach { order => if (!RowOrdering.isOrderable(order.dataType)) { @@ -534,6 +570,7 @@ trait CheckAnalysis extends PredicateHelper { case _ => // Analysis successful! } } + checkCollectedMetrics(plan) extendedCheckRules.foreach(_(plan)) plan.foreachUp { case o if !o.resolved => @@ -627,6 +664,38 @@ trait CheckAnalysis extends PredicateHelper { checkCorrelationsInSubquery(expr.plan) } + /** + * Validate that collected metrics names are unique. The same name cannot be used for metrics + * with different results. However multiple instances of metrics with with same result and name + * are allowed (e.g. self-joins). + */ + private def checkCollectedMetrics(plan: LogicalPlan): Unit = { + val metricsMap = mutable.Map.empty[String, LogicalPlan] + def check(plan: LogicalPlan): Unit = plan.foreach { node => + node match { + case metrics @ CollectMetrics(name, _, _) => + metricsMap.get(name) match { + case Some(other) => + // Exact duplicates are allowed. They can be the result + // of a CTE that is used multiple times or a self join. + if (!metrics.sameResult(other)) { + failAnalysis( + s"Multiple definitions of observed metrics named '$name': $plan") + } + case None => + metricsMap.put(name, metrics) + } + case _ => + } + node.expressions.foreach(_.foreach { + case subquery: SubqueryExpression => + check(subquery.plan) + case _ => + }) + } + check(plan) + } + /** * Validates to make sure the outer references appearing inside the subquery * are allowed. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/PlanHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/PlanHelper.scala index 4a28d879d1145..63348f766a5b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/PlanHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/PlanHelper.scala @@ -43,7 +43,9 @@ object PlanHelper { case e: WindowExpression if !plan.isInstanceOf[Window] => e case e: AggregateExpression - if !(plan.isInstanceOf[Aggregate] || plan.isInstanceOf[Window]) => e + if !(plan.isInstanceOf[Aggregate] || + plan.isInstanceOf[Window] || + plan.isInstanceOf[CollectMetrics]) => e case e: Generator if !plan.isInstanceOf[Generate] => e } 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 93b314d4e54a5..67438a47e8daa 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 @@ -971,3 +971,25 @@ case class Deduplicate( * This is used to whitelist such commands in the subquery-related checks. */ trait SupportsSubquery extends LogicalPlan + +/** + * Collect arbitrary (named) metrics from a dataset. As soon as the query reaches a completion + * point (batch query completes or streaming query epoch completes) an event is emitted on the + * driver which can be observed by attaching a listener to the spark session. The metrics are named + * so we can collect metrics at multiple places in a single dataset. + * + * This node behaves like a global aggregate. All the metrics collected must be aggregate functions + * or be literals. + */ +case class CollectMetrics( + name: String, + metrics: Seq[NamedExpression], + child: LogicalPlan) + extends UnaryNode { + + override lazy val resolved: Boolean = { + name.nonEmpty && metrics.nonEmpty && metrics.forall(_.resolved) && childrenResolved + } + + override def output: Seq[Attribute] = child.output +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e1b8192fba213..426ec6efb6ad6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -160,7 +160,12 @@ object SQLConf { confGetter.get()() } } else { - confGetter.get()() + val conf = existingConf.get() + if (conf != null) { + conf + } else { + confGetter.get()() + } } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index ad1d6b62ef3a1..de062f7efd8d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -21,9 +21,14 @@ import java.util.Locale import scala.util.control.NonFatal +import com.fasterxml.jackson.core.{JsonGenerator, JsonParser} +import com.fasterxml.jackson.databind.{DeserializationContext, JsonDeserializer, JsonSerializer, SerializerProvider} +import com.fasterxml.jackson.databind.`type`.TypeFactory +import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize} import org.json4s._ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ +import org.json4s.jackson.{JValueDeserializer, JValueSerializer} import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Stable @@ -40,7 +45,10 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ + @Stable +@JsonSerialize(using = classOf[DataTypeJsonSerializer]) +@JsonDeserialize(using = classOf[DataTypeJsonDeserializer]) abstract class DataType extends AbstractDataType { /** * Enables matching against DataType for expressions: @@ -475,3 +483,30 @@ object DataType { } } } + +/** + * Jackson serializer for [[DataType]]. Internally this delegates to json4s based serialization. + */ +class DataTypeJsonSerializer extends JsonSerializer[DataType] { + private val delegate = new JValueSerializer + override def serialize( + value: DataType, + gen: JsonGenerator, + provider: SerializerProvider): Unit = { + delegate.serialize(value.jsonValue, gen, provider) + } +} + +/** + * Jackson deserializer for [[DataType]]. Internally this delegates to json4s based deserialization. + */ +class DataTypeJsonDeserializer extends JsonDeserializer[DataType] { + private val delegate = new JValueDeserializer(classOf[Any]) + + override def deserialize( + jsonParser: JsonParser, + deserializationContext: DeserializationContext): DataType = { + val json = delegate.deserialize(jsonParser, deserializationContext) + DataType.parseDataType(json.asInstanceOf[JValue]) + } +} 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 74445a111e4d7..ae474cac9f748 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 @@ -29,11 +29,11 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable 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.aggregate.{Count, Sum} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan 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, RoundRobinPartitioning} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf @@ -650,4 +650,87 @@ class AnalysisSuite extends AnalysisTest with Matchers { assertAnalysisError(parsePlan("INSERT INTO test VALUES (1)"), Seq("Table not found: test")) } + + test("check CollectMetrics resolved") { + val a = testRelation.output.head + val sum = Sum(a).toAggregateExpression().as("sum") + val random_sum = Sum(Rand(1L)).toAggregateExpression().as("rand_sum") + val literal = Literal(1).as("lit") + + // Ok + assert(CollectMetrics("event", literal :: sum :: random_sum :: Nil, testRelation).resolved) + + // Bad name + assert(!CollectMetrics("", sum :: Nil, testRelation).resolved) + assertAnalysisError(CollectMetrics("", sum :: Nil, testRelation), + "observed metrics should be named" :: Nil) + + // No columns + assert(!CollectMetrics("evt", Nil, testRelation).resolved) + + def checkAnalysisError(exprs: Seq[NamedExpression], errors: String*): Unit = { + assertAnalysisError(CollectMetrics("event", exprs, testRelation), errors) + } + + // Unwrapped attribute + checkAnalysisError( + a :: Nil, + "Attribute", "can only be used as an argument to an aggregate function") + + // Unwrapped non-deterministic expression + checkAnalysisError( + Rand(10).as("rnd") :: Nil, + "non-deterministic expression", "can only be used as an argument to an aggregate function") + + // Distinct aggregate + checkAnalysisError( + Sum(a).toAggregateExpression(isDistinct = true).as("sum") :: Nil, + "distinct aggregates are not allowed in observed metrics, but found") + + // Nested aggregate + checkAnalysisError( + Sum(Sum(a).toAggregateExpression()).toAggregateExpression().as("sum") :: Nil, + "nested aggregates are not allowed in observed metrics, but found") + + // Windowed aggregate + val windowExpr = WindowExpression( + RowNumber(), + WindowSpecDefinition(Nil, a.asc :: Nil, + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))) + checkAnalysisError( + windowExpr.as("rn") :: Nil, + "window expressions are not allowed in observed metrics, but found") + } + + test("check CollectMetrics duplicates") { + val a = testRelation.output.head + val sum = Sum(a).toAggregateExpression().as("sum") + val count = Count(Literal(1)).toAggregateExpression().as("cnt") + + // Same result - duplicate names are allowed + assertAnalysisSuccess(Union( + CollectMetrics("evt1", count :: Nil, testRelation) :: + CollectMetrics("evt1", count :: Nil, testRelation) :: Nil)) + + // Same children, structurally different metrics - fail + assertAnalysisError(Union( + CollectMetrics("evt1", count :: Nil, testRelation) :: + CollectMetrics("evt1", sum :: Nil, testRelation) :: Nil), + "Multiple definitions of observed metrics" :: "evt1" :: Nil) + + // Different children, same metrics - fail + val b = 'b.string + val tblB = LocalRelation(b) + assertAnalysisError(Union( + CollectMetrics("evt1", count :: Nil, testRelation) :: + CollectMetrics("evt1", count :: Nil, tblB) :: Nil), + "Multiple definitions of observed metrics" :: "evt1" :: Nil) + + // Subquery different tree - fail + val subquery = Aggregate(Nil, sum :: Nil, CollectMetrics("evt1", count :: Nil, testRelation)) + val query = Project( + b :: ScalarSubquery(subquery, Nil).as("sum") :: Nil, + CollectMetrics("evt1", count :: Nil, tblB)) + assertAnalysisError(query, "Multiple definitions of observed metrics" :: "evt1" :: Nil) + } } 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 1b75fccbdb7b2..635f5f87b5a2f 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 @@ -1848,6 +1848,54 @@ class Dataset[T] private[sql]( @scala.annotation.varargs def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs : _*) + /** + * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset + * that returns the same result as the input, with the following guarantees: + * - It will compute the defined aggregates (metrics) on all the data that is flowing through the + * Dataset at that point. + * - It will report the value of the defined aggregate columns as soon as we reach a completion + * point. A completion point is either the end of a query (batch mode) or the end of a streaming + * epoch. The value of the aggregates only reflects the data processed since the previous + * completion point. + * Please note that continuous execution is currently not supported. + * + * The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or + * more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that + * contain references to the input Dataset's columns must always be wrapped in an aggregate + * function. + * + * A user can observe these metrics by either adding + * [[org.apache.spark.sql.streaming.StreamingQueryListener]] or a + * [[org.apache.spark.sql.util.QueryExecutionListener]] to the spark session. + * + * {{{ + * // Observe row count (rc) and error row count (erc) in the streaming Dataset + * val observed_ds = ds.observe("my_event", count(lit(1)).as("rc"), count($"error").as("erc")) + * observed_ds.writeStream.format("...").start() + * + * // Monitor the metrics using a listener. + * spark.streams.addListener(new StreamingQueryListener() { + * override def onQueryProgress(event: QueryProgressEvent): Unit = { + * event.progress.observedMetrics.get("my_event").foreach { row => + * // Trigger if the number of errors exceeds 5 percent + * val num_rows = row.getAs[Long]("rc") + * val num_error_rows = row.getAs[Long]("erc") + * val ratio = num_error_rows.toDouble / num_rows + * if (ratio > 0.05) { + * // Trigger alert + * } + * } + * } + * }) + * }}} + * + * @group typedrel + * @since 3.0.0 + */ + def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = withTypedPlan { + CollectMetrics(name, (expr +: exprs).map(_.named), logicalPlan) + } + /** * Returns a new Dataset by taking the first `n` rows. The difference between this function * and `head` is that `head` is an action and returns an array (by triggering query execution) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala index fa5ba1a691cd6..9aab5b390fe13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/AggregatingAccumulator.scala @@ -43,7 +43,8 @@ class AggregatingAccumulator private( assert(bufferSchema.size == updateExpressions.size) assert(mergeExpressions == null || bufferSchema.size == mergeExpressions.size) - private[this] var joinedRow: JoinedRow = _ + @transient + private var joinedRow: JoinedRow = _ private var buffer: SpecificInternalRow = _ @@ -184,7 +185,6 @@ class AggregatingAccumulator private( resultProjection(input) } - /** * Get the output schema of the aggregating accumulator. */ @@ -194,6 +194,17 @@ class AggregatingAccumulator private( case (e, i) => StructField(s"c_$i", e.dataType, e.nullable) }) } + + /** + * Set the state of the accumulator to the state of another accumulator. This is used in cases + * where we only want to publish the state of the accumulator when the task completes, see + * [[CollectMetricsExec]] for an example. + */ + private[execution] def setState(other: AggregatingAccumulator): Unit = { + assert(buffer == null || (buffer eq other.buffer)) + buffer = other.buffer + joinedRow = other.joinedRow + } } object AggregatingAccumulator { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala new file mode 100644 index 0000000000000..e482bc9941ea9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CollectMetricsExec.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import scala.collection.mutable + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.types.StructType + +/** + * Collect arbitrary (named) metrics from a [[SparkPlan]]. + */ +case class CollectMetricsExec( + name: String, + metricExpressions: Seq[NamedExpression], + child: SparkPlan) + extends UnaryExecNode { + + private lazy val accumulator: AggregatingAccumulator = { + val acc = AggregatingAccumulator(metricExpressions, child.output) + acc.register(sparkContext, Option("Collected metrics")) + acc + } + + val metricsSchema: StructType = { + StructType.fromAttributes(metricExpressions.map(_.toAttribute)) + } + + // This is not used very frequently (once a query); it is not useful to use code generation here. + private lazy val toRowConverter: InternalRow => Row = { + CatalystTypeConverters.createToScalaConverter(metricsSchema) + .asInstanceOf[InternalRow => Row] + } + + def collectedMetrics: Row = toRowConverter(accumulator.value) + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override protected def doExecute(): RDD[InternalRow] = { + val collector = accumulator + collector.reset() + child.execute().mapPartitions { rows => + // Only publish the value of the accumulator when the task has completed. This is done by + // updating a task local accumulator ('updater') which will be merged with the actual + // accumulator as soon as the task completes. This avoids the following problems during the + // heartbeat: + // - Correctness issues due to partially completed/visible updates. + // - Performance issues due to excessive serialization. + val updater = collector.copyAndReset() + TaskContext.get().addTaskCompletionListener[Unit] { _ => + collector.setState(updater) + } + + rows.map { r => + updater.add(r) + r + } + } + } +} + +object CollectMetricsExec { + /** + * Recursively collect all collected metrics from a query tree. + */ + def collect(plan: SparkPlan): Map[String, Row] = { + val metrics = plan.collectInPlanAndSubqueries { + case collector: CollectMetricsExec => collector.name -> collector.collectedMetrics + } + metrics.toMap + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 190c74297e9f2..28bbe4fb4993d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -22,7 +22,7 @@ import java.io.{BufferedWriter, OutputStreamWriter} import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.expressions.codegen.ByteCodeStats @@ -106,6 +106,9 @@ class QueryExecution( lazy val toRdd: RDD[InternalRow] = new SQLExecutionRDD( executedPlan.execute(), sparkSession.sessionState.conf) + /** Get the metrics observed during the execution of the query plan. */ + def observedMetrics: Map[String, Row] = CollectMetricsExec.collect(executedPlan) + protected def preparations: Seq[Rule[SparkPlan]] = { QueryExecution.preparations(sparkSession) } 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 76af81abdb0d4..8eb0e2262e670 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 @@ -750,6 +750,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { throw new UnsupportedOperationException(s"UPDATE TABLE is not supported temporarily.") case _: MergeIntoTable => throw new UnsupportedOperationException(s"MERGE INTO TABLE is not supported temporarily.") + case logical.CollectMetrics(name, metrics, child) => + execution.CollectMetricsExec(name, metrics, planLater(child)) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 4f42992126c49..71bcd53435850 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_SECOND import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -173,6 +173,7 @@ trait ProgressReporter extends Logging { val sinkProgress = SinkProgress( sink.toString, sinkCommitProgress.map(_.numOutputRows)) + val observedMetrics = extractObservedMetrics(hasNewData, lastExecution) val newProgress = new StreamingQueryProgress( id = id, @@ -184,7 +185,8 @@ trait ProgressReporter extends Logging { eventTime = new java.util.HashMap(executionStats.eventTimeStats.asJava), stateOperators = executionStats.stateOperators.toArray, sources = sourceProgress.toArray, - sink = sinkProgress) + sink = sinkProgress, + observedMetrics = new java.util.HashMap(observedMetrics.asJava)) if (hasNewData) { // Reset noDataEventTimestamp if we processed any data @@ -323,6 +325,16 @@ trait ProgressReporter extends Logging { } } + /** Extracts observed metrics from the most recent query execution. */ + private def extractObservedMetrics( + hasNewData: Boolean, + lastExecution: QueryExecution): Map[String, Row] = { + if (!hasNewData || lastExecution == null) { + return Map.empty + } + lastExecution.observedMetrics + } + /** Records the duration of running `body` for the next query progress update. */ protected def reportTimeTaken[T](triggerDetailKey: String)(body: => T): T = { val startTime = triggerClock.getTimeMillis() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index e2fea8c9dd6ab..a9681dbd0c676 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -24,12 +24,15 @@ import java.util.UUID import scala.collection.JavaConverters._ import scala.util.control.NonFatal +import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.json4s._ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.streaming.SinkProgress.DEFAULT_NUM_OUTPUT_ROWS /** @@ -106,7 +109,9 @@ class StreamingQueryProgress private[sql]( val eventTime: ju.Map[String, String], val stateOperators: Array[StateOperatorProgress], val sources: Array[SourceProgress], - val sink: SinkProgress) extends Serializable { + val sink: SinkProgress, + @JsonDeserialize(contentAs = classOf[GenericRowWithSchema]) + val observedMetrics: ju.Map[String, Row]) extends Serializable { /** The aggregate (across all sources) number of records processed in a trigger. */ def numInputRows: Long = sources.map(_.numInputRows).sum @@ -149,7 +154,8 @@ class StreamingQueryProgress private[sql]( ("eventTime" -> safeMapToJValue[String](eventTime, s => JString(s))) ~ ("stateOperators" -> JArray(stateOperators.map(_.jsonValue).toList)) ~ ("sources" -> JArray(sources.map(_.jsonValue).toList)) ~ - ("sink" -> sink.jsonValue) + ("sink" -> sink.jsonValue) ~ + ("observedMetrics" -> safeMapToJValue[Row](observedMetrics, row => row.jsonValue)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala index aaec6a9761d63..a33b9fad7ff4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala @@ -73,9 +73,12 @@ class AggregatingAccumulatorSuite // Idempotency of result checkResult(acc1.value, InternalRow(73L, str("baz"), 3L), expectedSchema, false) - // A few updates to the copied accumulator - acc2.add(InternalRow(-2L, str("qwerty"), -6773.9d)) - acc2.add(InternalRow(-35L, str("zzz-top"), -323.9d)) + // A few updates to the copied accumulator using an updater + val updater = acc2.copyAndReset() + updater.add(InternalRow(-2L, str("qwerty"), -6773.9d)) + updater.add(InternalRow(-35L, str("zzz-top"), -323.9d)) + assert(acc2.isZero) + acc2.setState(updater) checkResult(acc2.value, InternalRow(-36L, str("zzz-top"), 2L), expectedSchema, false) // Merge accumulators diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 4c58cb85c4d36..2f66dd3255b11 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -29,7 +29,7 @@ import org.scalatest.concurrent.Waiters.Waiter import org.apache.spark.SparkException import org.apache.spark.scheduler._ -import org.apache.spark.sql.{Encoder, SparkSession} +import org.apache.spark.sql.{Encoder, Row, SparkSession} import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf @@ -404,6 +404,63 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.2.txt") } + test("listener propagates observable metrics") { + import org.apache.spark.sql.functions._ + val clock = new StreamManualClock + val inputData = new MemoryStream[Int](0, sqlContext) + val df = inputData.toDF() + .observe( + name = "my_event", + min($"value").as("min_val"), + max($"value").as("max_val"), + sum($"value").as("sum_val"), + count(when($"value" % 2 === 0, 1)).as("num_even")) + .observe( + name = "other_event", + avg($"value").cast("int").as("avg_val")) + val listener = new EventCollector + def checkMetrics(f: java.util.Map[String, Row] => Unit): StreamAction = { + AssertOnQuery { _ => + eventually(Timeout(streamingTimeout)) { + assert(listener.allProgressEvents.nonEmpty) + f(listener.allProgressEvents.last.observedMetrics) + true + } + } + } + + try { + spark.streams.addListener(listener) + testStream(df, OutputMode.Append)( + StartStream(Trigger.ProcessingTime(100), triggerClock = clock), + // Batch 1 + AddData(inputData, 1, 2), + AdvanceManualClock(100), + checkMetrics { metrics => + assert(metrics.get("my_event") === Row(1, 2, 3L, 1L)) + assert(metrics.get("other_event") === Row(1)) + }, + + // Batch 2 + AddData(inputData, 10, 30, -10, 5), + AdvanceManualClock(100), + checkMetrics { metrics => + assert(metrics.get("my_event") === Row(-10, 30, 35L, 3L)) + assert(metrics.get("other_event") === Row(8)) + }, + + // Batch 3 - no data + AdvanceManualClock(100), + checkMetrics { metrics => + assert(metrics.isEmpty) + }, + StopStream + ) + } finally { + spark.streams.removeListener(listener) + } + } + private def testReplayListenerBusWithBorkenEventJsons(fileName: String): Unit = { val input = getClass.getResourceAsStream(s"/structured-streaming/$fileName") val events = mutable.ArrayBuffer[SparkListenerEvent]() @@ -454,6 +511,10 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { _progressEvents.filter(_.numInputRows > 0) } + def allProgressEvents: Seq[StreamingQueryProgress] = _progressEvents.synchronized { + _progressEvents.clone() + } + def reset(): Unit = { startEvent = null terminationEvent = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index da291f490b76c..b6a6be2bb0312 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -27,12 +27,15 @@ import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQueryStatusAndProgressSuite._ import org.apache.spark.sql.streaming.StreamingQuerySuite.clock import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.sql.types.StructType class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { test("StreamingQueryProgress - prettyJson") { @@ -77,6 +80,17 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "sink" : { | "description" : "sink", | "numOutputRows" : -1 + | }, + | "observedMetrics" : { + | "event1" : { + | "c1" : 1, + | "c2" : 3.0 + | }, + | "event2" : { + | "rc" : 1, + | "min_q" : "hello", + | "max_q" : "world" + | } | } |} """.stripMargin.trim) @@ -110,6 +124,22 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "sink" : { | "description" : "sink", | "numOutputRows" : -1 + | }, + | "observedMetrics" : { + | "event_a" : { + | "c1" : null, + | "c2" : -20.7 + | }, + | "event_b1" : { + | "rc" : 33, + | "min_q" : "foo", + | "max_q" : "bar" + | }, + | "event_b2" : { + | "rc" : 200, + | "min_q" : "fzo", + | "max_q" : "baz" + | } | } |} """.stripMargin.trim) @@ -265,6 +295,17 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { } object StreamingQueryStatusAndProgressSuite { + private val schema1 = new StructType() + .add("c1", "long") + .add("c2", "double") + private val schema2 = new StructType() + .add("rc", "long") + .add("min_q", "string") + .add("max_q", "string") + private def row(schema: StructType, elements: Any*): Row = { + new GenericRowWithSchema(elements.toArray, schema) + } + val testProgress1 = new StreamingQueryProgress( id = UUID.randomUUID, runId = UUID.randomUUID, @@ -293,7 +334,10 @@ object StreamingQueryStatusAndProgressSuite { processedRowsPerSecond = Double.PositiveInfinity // should not be present in the json ) ), - sink = SinkProgress("sink", None) + sink = SinkProgress("sink", None), + observedMetrics = new java.util.HashMap(Map( + "event1" -> row(schema1, 1L, 3.0d), + "event2" -> row(schema2, 1L, "hello", "world")).asJava) ) val testProgress2 = new StreamingQueryProgress( @@ -317,7 +361,11 @@ object StreamingQueryStatusAndProgressSuite { processedRowsPerSecond = Double.NegativeInfinity // should not be present in the json ) ), - sink = SinkProgress("sink", None) + sink = SinkProgress("sink", None), + observedMetrics = new java.util.HashMap(Map( + "event_a" -> row(schema1, null, -20.7d), + "event_b1" -> row(schema2, 33L, "foo", "bar"), + "event_b2" -> row(schema2, 200L, "fzo", "baz")).asJava) ) val testStatus = new StreamingQueryStatus("active", true, false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index 083b40d0680aa..f4ab232af28b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.util import scala.collection.mutable.ArrayBuffer import org.apache.spark._ -import org.apache.spark.sql.{functions, AnalysisException, QueryTest} +import org.apache.spark.sql.{functions, AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} @@ -226,4 +226,52 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { assert(errors.head._2 == e) } } + + test("get observable metrics by callback") { + val metricMaps = ArrayBuffer.empty[Map[String, Row]] + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + metricMaps += qe.observedMetrics + } + + override def onFailure(funcName: String, qe: QueryExecution, exception: Throwable): Unit = { + // No-op + } + } + spark.listenerManager.register(listener) + try { + val df = spark.range(100) + .observe( + name = "my_event", + min($"id").as("min_val"), + max($"id").as("max_val"), + sum($"id").as("sum_val"), + count(when($"id" % 2 === 0, 1)).as("num_even")) + .observe( + name = "other_event", + avg($"id").cast("int").as("avg_val")) + + def checkMetrics(metrics: Map[String, Row]): Unit = { + assert(metrics.size === 2) + assert(metrics("my_event") === Row(0L, 99L, 4950L, 50L)) + assert(metrics("other_event") === Row(49)) + } + + // First run + df.collect() + sparkContext.listenerBus.waitUntilEmpty() + assert(metricMaps.size === 1) + checkMetrics(metricMaps.head) + metricMaps.clear() + + // Second run should produce the same result as the first run. + df.collect() + sparkContext.listenerBus.waitUntilEmpty() + assert(metricMaps.size === 1) + checkMetrics(metricMaps.head) + + } finally { + spark.listenerManager.unregister(listener) + } + } } From 39291cff951639a7ae4b487ea2c606affa5ff76f Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 3 Dec 2019 18:40:14 +0800 Subject: [PATCH 0775/1387] [SPARK-30048][SQL] Enable aggregates with interval type values for RelationalGroupedDataset ### What changes were proposed in this pull request? Now the min/max/sum/avg are support for intervals, we should also enable it in RelationalGroupedDataset ### Why are the changes needed? API consistency improvement ### Does this PR introduce any user-facing change? yes, Dataset support min/max/sum/avg(mean) on intervals ### How was this patch tested? add ut Closes #26681 from yaooqinn/SPARK-30048. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../spark/sql/types/AbstractDataType.scala | 4 +- .../scala/org/apache/spark/sql/Dataset.scala | 6 +-- .../spark/sql/RelationalGroupedDataset.scala | 39 ++++++++++--------- .../spark/sql/DataFrameAggregateSuite.scala | 20 ++++++++++ 4 files changed, 45 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index 21ac32adca6e9..25303475a73ca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -79,8 +79,8 @@ private[sql] class TypeCollection(private val types: Seq[AbstractDataType]) private[sql] object TypeCollection { /** - * Types that include numeric types and interval type. They are only used in unary_minus, - * unary_positive, add and subtract operations. + * Types that include numeric types and interval type, which support numeric type calculations, + * i.e. unary_minus, unary_positive, sum, avg, min, max, add and subtract operations. */ val NumericAndInterval = TypeCollection(NumericType, CalendarIntervalType) 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 635f5f87b5a2f..be3f2aed25936 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 @@ -268,9 +268,9 @@ class Dataset[T] private[sql]( } } - private[sql] def numericColumns: Seq[Expression] = { - schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => - queryExecution.analyzed.resolveQuoted(n.name, sparkSession.sessionState.analyzer.resolver).get + private[sql] def numericCalculationSupportedColumns: Seq[Expression] = { + queryExecution.analyzed.output.filter { attr => + TypeCollection.NumericAndInterval.acceptsType(attr.dataType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index b1ba7d4538732..52bd0ecb1fffd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{NumericType, StructType} +import org.apache.spark.sql.types.{StructType, TypeCollection} /** * A set of methods for aggregations on a `DataFrame`, created by [[Dataset#groupBy groupBy]], @@ -88,20 +88,20 @@ class RelationalGroupedDataset protected[sql]( case expr: Expression => Alias(expr, toPrettySQL(expr))() } - private[this] def aggregateNumericColumns(colNames: String*)(f: Expression => AggregateFunction) - : DataFrame = { + private[this] def aggregateNumericOrIntervalColumns( + colNames: String*)(f: Expression => AggregateFunction): DataFrame = { val columnExprs = if (colNames.isEmpty) { - // No columns specified. Use all numeric columns. - df.numericColumns + // No columns specified. Use all numeric calculation supported columns. + df.numericCalculationSupportedColumns } else { - // Make sure all specified columns are numeric. + // Make sure all specified columns are numeric calculation supported columns. colNames.map { colName => val namedExpr = df.resolve(colName) - if (!namedExpr.dataType.isInstanceOf[NumericType]) { + if (!TypeCollection.NumericAndInterval.acceptsType(namedExpr.dataType)) { throw new AnalysisException( - s""""$colName" is not a numeric column. """ + - "Aggregation function can only be applied on a numeric column.") + s""""$colName" is not a numeric or calendar interval column. """ + + "Aggregation function can only be applied on a numeric or calendar interval column.") } namedExpr } @@ -269,7 +269,8 @@ class RelationalGroupedDataset protected[sql]( def count(): DataFrame = toDF(Seq(Alias(Count(Literal(1)).toAggregateExpression(), "count")())) /** - * Compute the average value for each numeric columns for each group. This is an alias for `avg`. + * Compute the average value for each numeric or calender interval columns for each group. This + * is an alias for `avg`. * The resulting `DataFrame` will also contain the grouping columns. * When specified columns are given, only compute the average values for them. * @@ -277,11 +278,11 @@ class RelationalGroupedDataset protected[sql]( */ @scala.annotation.varargs def mean(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Average) + aggregateNumericOrIntervalColumns(colNames : _*)(Average) } /** - * Compute the max value for each numeric columns for each group. + * Compute the max value for each numeric calender interval columns for each group. * The resulting `DataFrame` will also contain the grouping columns. * When specified columns are given, only compute the max values for them. * @@ -289,11 +290,11 @@ class RelationalGroupedDataset protected[sql]( */ @scala.annotation.varargs def max(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Max) + aggregateNumericOrIntervalColumns(colNames : _*)(Max) } /** - * Compute the mean value for each numeric columns for each group. + * Compute the mean value for each numeric calender interval columns for each group. * The resulting `DataFrame` will also contain the grouping columns. * When specified columns are given, only compute the mean values for them. * @@ -301,11 +302,11 @@ class RelationalGroupedDataset protected[sql]( */ @scala.annotation.varargs def avg(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Average) + aggregateNumericOrIntervalColumns(colNames : _*)(Average) } /** - * Compute the min value for each numeric column for each group. + * Compute the min value for each numeric calender interval column for each group. * The resulting `DataFrame` will also contain the grouping columns. * When specified columns are given, only compute the min values for them. * @@ -313,11 +314,11 @@ class RelationalGroupedDataset protected[sql]( */ @scala.annotation.varargs def min(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Min) + aggregateNumericOrIntervalColumns(colNames : _*)(Min) } /** - * Compute the sum for each numeric columns for each group. + * Compute the sum for each numeric calender interval columns for each group. * The resulting `DataFrame` will also contain the grouping columns. * When specified columns are given, only compute the sum for them. * @@ -325,7 +326,7 @@ class RelationalGroupedDataset protected[sql]( */ @scala.annotation.varargs def sum(colNames: String*): DataFrame = { - aggregateNumericColumns(colNames : _*)(Sum) + aggregateNumericOrIntervalColumns(colNames : _*)(Sum) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 944e4212b1bfb..a08ef19c3ac5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -964,4 +964,24 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { Row(3, new CalendarInterval(0, 3, 0)) :: Nil) assert(df3.queryExecution.executedPlan.find(_.isInstanceOf[HashAggregateExec]).isDefined) } + + test("Dataset agg functions support calendar intervals") { + val df1 = Seq((1, "1 day"), (2, "2 day"), (3, "3 day"), (3, null)).toDF("a", "b") + val df2 = df1.select('a, 'b cast CalendarIntervalType).groupBy('a % 2) + checkAnswer(df2.sum("b"), + Row(0, new CalendarInterval(0, 2, 0)) :: + Row(1, new CalendarInterval(0, 4, 0)) :: Nil) + checkAnswer(df2.avg("b"), + Row(0, new CalendarInterval(0, 2, 0)) :: + Row(1, new CalendarInterval(0, 2, 0)) :: Nil) + checkAnswer(df2.mean("b"), + Row(0, new CalendarInterval(0, 2, 0)) :: + Row(1, new CalendarInterval(0, 2, 0)) :: Nil) + checkAnswer(df2.max("b"), + Row(0, new CalendarInterval(0, 2, 0)) :: + Row(1, new CalendarInterval(0, 3, 0)) :: Nil) + checkAnswer(df2.min("b"), + Row(0, new CalendarInterval(0, 2, 0)) :: + Row(1, new CalendarInterval(0, 1, 0)) :: Nil) + } } From 65552a81d1c12c7133e85695f6681799954ff6b1 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 3 Dec 2019 23:42:21 +0800 Subject: [PATCH 0776/1387] [SPARK-30083][SQL] visitArithmeticUnary should wrap PLUS case with UnaryPositive for type checking ### What changes were proposed in this pull request? `UnaryPositive` only accepts numeric and interval as we defined, but what we do for this in `AstBuider.visitArithmeticUnary` is just bypassing it. This should not be omitted for the type checking requirement. ### Why are the changes needed? bug fix, you can find a pre-discussion here https://github.com/apache/spark/pull/26578#discussion_r347350398 ### Does this PR introduce any user-facing change? yes, +non-numeric-or-interval is now invalid. ``` -- !query 14 select +date '1900-01-01' -- !query 14 schema struct -- !query 14 output 1900-01-01 -- !query 15 select +timestamp '1900-01-01' -- !query 15 schema struct -- !query 15 output 1900-01-01 00:00:00 -- !query 16 select +map(1, 2) -- !query 16 schema struct> -- !query 16 output {1:2} -- !query 17 select +array(1,2) -- !query 17 schema struct> -- !query 17 output [1,2] -- !query 18 select -'1' -- !query 18 schema struct<(- CAST(1 AS DOUBLE)):double> -- !query 18 output -1.0 -- !query 19 select -X'1' -- !query 19 schema struct<> -- !query 19 output org.apache.spark.sql.AnalysisException cannot resolve '(- X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7 -- !query 20 select +X'1' -- !query 20 schema struct -- !query 20 output ``` ### How was this patch tested? add ut check Closes #26716 from yaooqinn/SPARK-30083. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + .../spark/sql/catalyst/dsl/package.scala | 1 + .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../parser/ExpressionParserSuite.scala | 4 +- .../resources/sql-tests/inputs/literals.sql | 5 ++ .../sql-tests/results/ansi/interval.sql.out | 4 +- .../sql-tests/results/ansi/literals.sql.out | 72 +++++++++++++++---- .../sql-tests/results/interval.sql.out | 4 +- .../sql-tests/results/literals.sql.out | 72 +++++++++++++++---- .../sql-tests/results/operators.sql.out | 20 +++--- .../native/promoteStrings.sql.out | 4 +- 11 files changed, 145 insertions(+), 45 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 5c2e9ba11989d..5ccc3fd248219 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -254,6 +254,8 @@ license: |
    + - Since Spark 3.0, the unary arithmetic operator plus(`+`) only accepts string, numeric and interval type values as inputs. Besides, `+` with a integral string representation will be coerced to double value, e.g. `+'1'` results `1.0`. In Spark version 2.4 and earlier, this operator is ignored. There is no type checking for it, thus, all type values with a `+` prefix are valid, e.g. `+ array(1, 2)` is valid and results `[1, 2]`. Besides, there is no type coercion for it at all, e.g. in Spark 2.4, the result of `+'1'` is string `1`. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index c1cef013eab1c..18753f29b32c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -64,6 +64,7 @@ package object dsl { trait ImplicitOperators { def expr: Expression + def unary_+ : Expression = UnaryPositive(expr) def unary_- : Expression = UnaryMinus(expr) def unary_! : Predicate = Not(expr) def unary_~ : Expression = BitwiseNot(expr) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index df827967e4b2a..047cc22fe5641 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1461,7 +1461,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val value = expression(ctx.valueExpression) ctx.operator.getType match { case SqlBaseParser.PLUS => - value + UnaryPositive(value) case SqlBaseParser.MINUS => UnaryMinus(value) case SqlBaseParser.TILDE => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 6c1175658a03e..9f6e083d0030b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -226,10 +226,10 @@ class ExpressionParserSuite extends AnalysisTest { } test("unary arithmetic expressions") { - assertEqual("+a", 'a) + assertEqual("+a", +'a) assertEqual("-a", -'a) assertEqual("~a", ~'a) - assertEqual("-+~~a", -(~(~'a))) + assertEqual("-+~~a", -( +(~(~'a)))) } test("cast expressions") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 61b02d86bb51b..738b8411759c2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -110,6 +110,11 @@ select -integer '7'; select +integer '7'; select +date '1999-01-01'; select +timestamp '1999-01-01'; +select +interval '1 day'; +select +map(1, 2); +select +array(1,2); +select +named_struct('a', 1, 'b', 'spark'); +select +X'1'; -- can't negate date/timestamp/binary select -date '1999-01-01'; select -timestamp '1999-01-01'; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index a2d03064afdd4..2709fb42488cc 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -255,7 +255,7 @@ struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 31 select +interval '-1 month 1 day -1 second' -- !query 31 schema -struct +struct<(+ INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 31 output -1 months 1 days -1 seconds @@ -263,7 +263,7 @@ struct -- !query 32 select +interval -1 month 1 day -1 second -- !query 32 schema -struct +struct<(+ INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 32 output -1 months 1 days -1 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index e2910548aef5b..96425cf6219ed 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 50 +-- Number of queries: 55 -- !query 0 @@ -433,7 +433,7 @@ struct<(- 7):int> -- !query 44 select +integer '7' -- !query 44 schema -struct<7:int> +struct<(+ 7):int> -- !query 44 output 7 @@ -441,41 +441,87 @@ struct<7:int> -- !query 45 select +date '1999-01-01' -- !query 45 schema -struct +struct<> -- !query 45 output -1999-01-01 +org.apache.spark.sql.AnalysisException +cannot resolve '(+ DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 -- !query 46 select +timestamp '1999-01-01' -- !query 46 schema -struct +struct<> -- !query 46 output -1999-01-01 00:00:00 +org.apache.spark.sql.AnalysisException +cannot resolve '(+ TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 -- !query 47 -select -date '1999-01-01' +select +interval '1 day' -- !query 47 schema -struct<> +struct<(+ INTERVAL '1 days'):interval> -- !query 47 output -org.apache.spark.sql.AnalysisException -cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 +1 days -- !query 48 -select -timestamp '1999-01-01' +select +map(1, 2) -- !query 48 schema struct<> -- !query 48 output org.apache.spark.sql.AnalysisException -cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 +cannot resolve '(+ map(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'map(1, 2)' is of map type.; line 1 pos 7 -- !query 49 -select -x'2379ACFe' +select +array(1,2) -- !query 49 schema struct<> -- !query 49 output org.apache.spark.sql.AnalysisException +cannot resolve '(+ array(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'array(1, 2)' is of array type.; line 1 pos 7 + + +-- !query 50 +select +named_struct('a', 1, 'b', 'spark') +-- !query 50 schema +struct<> +-- !query 50 output +org.apache.spark.sql.AnalysisException +cannot resolve '(+ named_struct('a', 1, 'b', 'spark'))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'named_struct('a', 1, 'b', 'spark')' is of struct type.; line 1 pos 7 + + +-- !query 51 +select +X'1' +-- !query 51 schema +struct<> +-- !query 51 output +org.apache.spark.sql.AnalysisException +cannot resolve '(+ X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7 + + +-- !query 52 +select -date '1999-01-01' +-- !query 52 schema +struct<> +-- !query 52 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 + + +-- !query 53 +select -timestamp '1999-01-01' +-- !query 53 schema +struct<> +-- !query 53 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 + + +-- !query 54 +select -x'2379ACFe' +-- !query 54 schema +struct<> +-- !query 54 output +org.apache.spark.sql.AnalysisException cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 69a23bb9f1833..601521ba43227 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -255,7 +255,7 @@ struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 31 select +interval '-1 month 1 day -1 second' -- !query 31 schema -struct +struct<(+ INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 31 output -1 months 1 days -1 seconds @@ -263,7 +263,7 @@ struct -- !query 32 select +interval -1 month 1 day -1 second -- !query 32 schema -struct +struct<(+ INTERVAL '-1 months 1 days -1 seconds'):interval> -- !query 32 output -1 months 1 days -1 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index e2910548aef5b..96425cf6219ed 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 50 +-- Number of queries: 55 -- !query 0 @@ -433,7 +433,7 @@ struct<(- 7):int> -- !query 44 select +integer '7' -- !query 44 schema -struct<7:int> +struct<(+ 7):int> -- !query 44 output 7 @@ -441,41 +441,87 @@ struct<7:int> -- !query 45 select +date '1999-01-01' -- !query 45 schema -struct +struct<> -- !query 45 output -1999-01-01 +org.apache.spark.sql.AnalysisException +cannot resolve '(+ DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 -- !query 46 select +timestamp '1999-01-01' -- !query 46 schema -struct +struct<> -- !query 46 output -1999-01-01 00:00:00 +org.apache.spark.sql.AnalysisException +cannot resolve '(+ TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 -- !query 47 -select -date '1999-01-01' +select +interval '1 day' -- !query 47 schema -struct<> +struct<(+ INTERVAL '1 days'):interval> -- !query 47 output -org.apache.spark.sql.AnalysisException -cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 +1 days -- !query 48 -select -timestamp '1999-01-01' +select +map(1, 2) -- !query 48 schema struct<> -- !query 48 output org.apache.spark.sql.AnalysisException -cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 +cannot resolve '(+ map(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'map(1, 2)' is of map type.; line 1 pos 7 -- !query 49 -select -x'2379ACFe' +select +array(1,2) -- !query 49 schema struct<> -- !query 49 output org.apache.spark.sql.AnalysisException +cannot resolve '(+ array(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'array(1, 2)' is of array type.; line 1 pos 7 + + +-- !query 50 +select +named_struct('a', 1, 'b', 'spark') +-- !query 50 schema +struct<> +-- !query 50 output +org.apache.spark.sql.AnalysisException +cannot resolve '(+ named_struct('a', 1, 'b', 'spark'))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'named_struct('a', 1, 'b', 'spark')' is of struct type.; line 1 pos 7 + + +-- !query 51 +select +X'1' +-- !query 51 schema +struct<> +-- !query 51 output +org.apache.spark.sql.AnalysisException +cannot resolve '(+ X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7 + + +-- !query 52 +select -date '1999-01-01' +-- !query 52 schema +struct<> +-- !query 52 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 + + +-- !query 53 +select -timestamp '1999-01-01' +-- !query 53 schema +struct<> +-- !query 53 output +org.apache.spark.sql.AnalysisException +cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 + + +-- !query 54 +select -x'2379ACFe' +-- !query 54 schema +struct<> +-- !query 54 output +org.apache.spark.sql.AnalysisException cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index 82248ce921fd2..89dbf53c3c2ef 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -13,7 +13,7 @@ struct<-100:int> -- !query 1 select +230 -- !query 1 schema -struct<230:int> +struct<(+ 230):int> -- !query 1 output 230 @@ -29,7 +29,7 @@ struct<-5.2:decimal(2,1)> -- !query 3 select +6.8e0 -- !query 3 schema -struct<6.8:double> +struct<(+ 6.8):double> -- !query 3 output 6.8 @@ -37,7 +37,7 @@ struct<6.8:double> -- !query 4 select -key, +key from testdata where key = 2 -- !query 4 schema -struct<(- key):int,key:int> +struct<(- key):int,(+ key):int> -- !query 4 output -2 2 @@ -45,7 +45,7 @@ struct<(- key):int,key:int> -- !query 5 select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1 -- !query 5 schema -struct<(- (key + 1)):int,((- key) + 1):int,(key + 5):int> +struct<(- (key + 1)):int,((- key) + 1):int,(+ (key + 5)):int> -- !query 5 output -2 0 6 @@ -53,7 +53,7 @@ struct<(- (key + 1)):int,((- key) + 1):int,(key + 5):int> -- !query 6 select -max(key), +max(key) from testdata -- !query 6 schema -struct<(- max(key)):int,max(key):int> +struct<(- max(key)):int,(+ max(key)):int> -- !query 6 output -100 100 @@ -69,7 +69,7 @@ struct<(- -10):int> -- !query 8 select + (-key) from testdata where key = 32 -- !query 8 schema -struct<(- key):int> +struct<(+ (- key)):int> -- !query 8 output -32 @@ -77,7 +77,7 @@ struct<(- key):int> -- !query 9 select - (+max(key)) from testdata -- !query 9 schema -struct<(- max(key)):int> +struct<(- (+ max(key))):int> -- !query 9 output -100 @@ -93,7 +93,7 @@ struct<(- -3):int> -- !query 11 select - + 20 -- !query 11 schema -struct<(- 20):int> +struct<(- (+ 20)):int> -- !query 11 output -20 @@ -101,7 +101,7 @@ struct<(- 20):int> -- !query 12 select + + 100 -- !query 12 schema -struct<100:int> +struct<(+ (+ 100)):int> -- !query 12 output 100 @@ -117,7 +117,7 @@ struct<(- (- max(key))):int> -- !query 14 select + - key from testdata where key = 33 -- !query 14 schema -struct<(- key):int> +struct<(+ (- key)):int> -- !query 14 output -33 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out index c54ceba85ce79..8557f92f25115 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out @@ -2541,9 +2541,9 @@ struct<(- CAST(1 AS DOUBLE)):double> -- !query 311 SELECT + '1' FROM t -- !query 311 schema -struct<1:string> +struct<(+ CAST(1 AS DOUBLE)):double> -- !query 311 output -1 +1.0 -- !query 312 From 8c2849a6954a9f4d2a7e6dbf5ac34bb5e5c63271 Mon Sep 17 00:00:00 2001 From: John Ayad Date: Wed, 4 Dec 2019 00:04:55 +0800 Subject: [PATCH 0777/1387] [SPARK-30082][SQL] Do not replace Zeros when replacing NaNs ### What changes were proposed in this pull request? Do not cast `NaN` to an `Integer`, `Long`, `Short` or `Byte`. This is because casting `NaN` to those types results in a `0` which erroneously replaces `0`s while only `NaN`s should be replaced. ### Why are the changes needed? This Scala code snippet: ``` import scala.math; println(Double.NaN.toLong) ``` returns `0` which is problematic as if you run the following Spark code, `0`s get replaced as well: ``` >>> df = spark.createDataFrame([(1.0, 0), (0.0, 3), (float('nan'), 0)], ("index", "value")) >>> df.show() +-----+-----+ |index|value| +-----+-----+ | 1.0| 0| | 0.0| 3| | NaN| 0| +-----+-----+ >>> df.replace(float('nan'), 2).show() +-----+-----+ |index|value| +-----+-----+ | 1.0| 2| | 0.0| 3| | 2.0| 2| +-----+-----+ ``` ### Does this PR introduce any user-facing change? Yes, after the PR, running the same above code snippet returns the correct expected results: ``` >>> df = spark.createDataFrame([(1.0, 0), (0.0, 3), (float('nan'), 0)], ("index", "value")) >>> df.show() +-----+-----+ |index|value| +-----+-----+ | 1.0| 0| | 0.0| 3| | NaN| 0| +-----+-----+ >>> df.replace(float('nan'), 2).show() +-----+-----+ |index|value| +-----+-----+ | 1.0| 0| | 0.0| 3| | 2.0| 0| +-----+-----+ ``` ### How was this patch tested? Added unit tests to verify replacing `NaN` only affects columns of type `Float` and `Double` Closes #26738 from johnhany97/SPARK-30082. Lead-authored-by: John Ayad Co-authored-by: John Ayad Signed-off-by: Wenchen Fan --- .../spark/sql/DataFrameNaFunctions.scala | 2 +- .../spark/sql/DataFrameNaFunctionsSuite.scala | 44 +++++++++++++++++++ 2 files changed, 45 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 8447ada88a704..2a86b65b8f79f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -456,7 +456,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { val keyExpr = df.col(col.name).expr def buildExpr(v: Any) = Cast(Literal(v), keyExpr.dataType) val branches = replacementMap.flatMap { case (source, target) => - Seq(buildExpr(source), buildExpr(target)) + Seq(Literal(source), buildExpr(target)) }.toSeq new Column(CaseKeyWhen(keyExpr, branches :+ keyExpr)).as(col.name) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index 6cb35656835af..fb1ca69b6f73f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -37,6 +37,14 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { ).toDF("name", "age", "height") } + def createNaNDF(): DataFrame = { + Seq[(java.lang.Integer, java.lang.Long, java.lang.Short, + java.lang.Byte, java.lang.Float, java.lang.Double)]( + (1, 1L, 1.toShort, 1.toByte, 1.0f, 1.0), + (0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) + ).toDF("int", "long", "short", "byte", "float", "double") + } + test("drop") { val input = createDF() val rows = input.collect() @@ -404,4 +412,40 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { df.na.drop("any"), Row("5", "6", "6") :: Nil) } + + test("replace nan with float") { + checkAnswer( + createNaNDF().na.replace("*", Map( + Float.NaN -> 10.0f + )), + Row(1, 1L, 1.toShort, 1.toByte, 1.0f, 1.0) :: + Row(0, 0L, 0.toShort, 0.toByte, 10.0f, 10.0) :: Nil) + } + + test("replace nan with double") { + checkAnswer( + createNaNDF().na.replace("*", Map( + Double.NaN -> 10.0 + )), + Row(1, 1L, 1.toShort, 1.toByte, 1.0f, 1.0) :: + Row(0, 0L, 0.toShort, 0.toByte, 10.0f, 10.0) :: Nil) + } + + test("replace float with nan") { + checkAnswer( + createNaNDF().na.replace("*", Map( + 1.0f -> Float.NaN + )), + Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: + Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil) + } + + test("replace double with nan") { + checkAnswer( + createNaNDF().na.replace("*", Map( + 1.0 -> Double.NaN + )), + Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: + Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil) + } } From a3394e49a7bcc66dad551458376aa33c55ca9861 Mon Sep 17 00:00:00 2001 From: root1 Date: Tue, 3 Dec 2019 10:45:49 -0600 Subject: [PATCH 0778/1387] [SPARK-29477] Improve tooltip for Streaming tab ### What changes were proposed in this pull request? Added tooltip for duration columns in the batch table of streaming tab of Web UI. ### Why are the changes needed? Tooltips will help users in understanding columns of batch table of streaming tab. ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Manually tested. Closes #26467 from iRakson/streaming_tab_tooltip. Authored-by: root1 Signed-off-by: Sean Owen --- .../scala/org/apache/spark/streaming/ui/BatchPage.scala | 9 ++++++--- .../org/apache/spark/streaming/UISeleniumSuite.scala | 7 ++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index f3d2e478e9b2d..2c85d26738757 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -37,10 +37,13 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { private def columns: Seq[Node] = { Output Op Id Description - Output Op Duration + Output Op Duration {SparkUIUtils.tooltip("Time taken for all the jobs of this batch to" + + " finish processing from the time they were submitted.", + "top")} Status Job Id - Job Duration + Job Duration {SparkUIUtils.tooltip("Time taken from submission time to completion " + + "time of the job", "top")} Stages: Succeeded/Total Tasks (for all stages): Succeeded/Total Error @@ -381,7 +384,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { Input - Metadata + Metadata {SparkUIUtils.tooltip("Batch Input Details", "right")} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index ca9d62f50b5ab..bdc9e9ee2aed1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -151,8 +151,9 @@ class UISeleniumSuite summaryText should contain ("Total delay:") findAll(cssSelector("""#batch-job-table th""")).map(_.text).toSeq should be { - List("Output Op Id", "Description", "Output Op Duration", "Status", "Job Id", - "Job Duration", "Stages: Succeeded/Total", "Tasks (for all stages): Succeeded/Total", + List("Output Op Id", "Description", "Output Op Duration (?)", "Status", "Job Id", + "Job Duration (?)", "Stages: Succeeded/Total", + "Tasks (for all stages): Succeeded/Total", "Error") } @@ -163,7 +164,7 @@ class UISeleniumSuite // Check job ids val jobIdCells = findAll(cssSelector( """#batch-job-table a""")).toSeq - jobIdCells.map(_.text) should be (List("0", "1", "2", "3")) + jobIdCells.map(_.text).filter(_.forall(_.isDigit)) should be (List("0", "1", "2", "3")) val jobLinks = jobIdCells.flatMap(_.attribute("href")) jobLinks.size should be (4) From 4193d2f4cc2bb100625b073e3a2e8599c3b4cb7c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 3 Dec 2019 08:59:43 -0800 Subject: [PATCH 0779/1387] [SPARK-30012][CORE][SQL] Change classes extending scala collection classes to work with 2.13 ### What changes were proposed in this pull request? Move some classes extending Scala collections into parallel source trees, to support 2.13; other minor collection-related modifications. Modify some classes extending Scala collections to work with 2.13 as well as 2.12. In many cases, this means introducing parallel source trees, as the type hierarchy changed in ways that one class can't support both. ### Why are the changes needed? To support building for Scala 2.13 in the future. ### Does this PR introduce any user-facing change? There should be no behavior change. ### How was this patch tested? Existing tests. Note that the 2.13 changes are not tested by the PR builder, of course. They compile in 2.13 but can't even be tested locally. Later, once the project can be compiled for 2.13, thus tested, it's possible the 2.13 implementations will need updates. Closes #26728 from srowen/SPARK-30012. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- core/pom.xml | 3 +- .../spark/util/BoundedPriorityQueue.scala | 2 + .../spark/util/TimeStampedHashMap.scala | 2 + .../spark/util/BoundedPriorityQueue.scala | 73 +++++++++ .../spark/util/TimeStampedHashMap.scala | 143 ++++++++++++++++++ .../spark/util/collection/CompactBuffer.scala | 2 - .../collection/ExternalAppendOnlyMap.scala | 2 +- dev/change-scala-version.sh | 2 +- repl/pom.xml | 6 +- sql/catalyst/pom.xml | 18 +++ .../catalyst/expressions/AttributeMap.scala | 2 + .../catalyst/expressions/ExpressionSet.scala | 2 + .../catalyst/util/CaseInsensitiveMap.scala | 2 + .../catalyst/expressions/AttributeMap.scala | 46 ++++++ .../catalyst/expressions/ExpressionSet.scala | 100 ++++++++++++ .../catalyst/util/CaseInsensitiveMap.scala | 59 ++++++++ sql/core/pom.xml | 1 + .../execution/streaming/StreamProgress.scala | 2 + .../execution/streaming/StreamProgress.scala | 53 +++++++ 19 files changed, 510 insertions(+), 10 deletions(-) rename core/src/main/{scala => scala-2.12}/org/apache/spark/util/BoundedPriorityQueue.scala (95%) rename core/src/main/{scala => scala-2.12}/org/apache/spark/util/TimeStampedHashMap.scala (98%) create mode 100644 core/src/main/scala-2.13/org/apache/spark/util/BoundedPriorityQueue.scala create mode 100644 core/src/main/scala-2.13/org/apache/spark/util/TimeStampedHashMap.scala rename sql/catalyst/src/main/{scala => scala-2.12}/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala (94%) rename sql/catalyst/src/main/{scala => scala-2.12}/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala (97%) rename sql/catalyst/src/main/{scala => scala-2.12}/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala (96%) create mode 100644 sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala create mode 100644 sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala create mode 100644 sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala rename sql/core/src/main/{scala => scala-2.12}/org/apache/spark/sql/execution/streaming/StreamProgress.scala (95%) create mode 100644 sql/core/src/main/scala-2.13/org/apache/spark/sql/execution/streaming/StreamProgress.scala diff --git a/core/pom.xml b/core/pom.xml index 24d61bf7a1912..9d54d21b95ba3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -32,7 +32,6 @@ core - src/main/scala-${scala.binary.version} @@ -530,7 +529,7 @@ - ${extra.source.dir} + src/main/scala-${scala.binary.version} diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala-2.12/org/apache/spark/util/BoundedPriorityQueue.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala rename to core/src/main/scala-2.12/org/apache/spark/util/BoundedPriorityQueue.scala index 0827395fea0bb..a241023723444 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala-2.12/org/apache/spark/util/BoundedPriorityQueue.scala @@ -31,6 +31,8 @@ import scala.collection.generic.Growable private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) extends Iterable[A] with Growable[A] with Serializable { + // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. + private val underlying = new JPriorityQueue[A](maxSize, ord) override def iterator: Iterator[A] = underlying.iterator.asScala diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala-2.12/org/apache/spark/util/TimeStampedHashMap.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala rename to core/src/main/scala-2.12/org/apache/spark/util/TimeStampedHashMap.scala index 550e0674a14e0..da12582a5083a 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala-2.12/org/apache/spark/util/TimeStampedHashMap.scala @@ -40,6 +40,8 @@ private[spark] case class TimeStampedValue[V](value: V, timestamp: Long) private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) extends mutable.Map[A, B]() with Logging { + // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. + private val internalMap = new ConcurrentHashMap[A, TimeStampedValue[B]]() def get(key: A): Option[B] = { diff --git a/core/src/main/scala-2.13/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala-2.13/org/apache/spark/util/BoundedPriorityQueue.scala new file mode 100644 index 0000000000000..bc55a44fc3c2e --- /dev/null +++ b/core/src/main/scala-2.13/org/apache/spark/util/BoundedPriorityQueue.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.Serializable +import java.util.{PriorityQueue => JPriorityQueue} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.Growable + +/** + * Bounded priority queue. This class wraps the original PriorityQueue + * class and modifies it such that only the top K elements are retained. + * The top K elements are defined by an implicit Ordering[A]. + */ +private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) + extends Iterable[A] with Growable[A] with Serializable { + + // Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation. + + private val underlying = new JPriorityQueue[A](maxSize, ord) + + override def iterator: Iterator[A] = underlying.iterator.asScala + + override def size: Int = underlying.size + + override def knownSize: Int = size + + override def addAll(xs: IterableOnce[A]): this.type = { + xs.foreach { this += _ } + this + } + + override def addOne(elem: A): this.type = { + if (size < maxSize) { + underlying.offer(elem) + } else { + maybeReplaceLowest(elem) + } + this + } + + def poll(): A = { + underlying.poll() + } + + override def clear(): Unit = { underlying.clear() } + + private def maybeReplaceLowest(a: A): Boolean = { + val head = underlying.peek() + if (head != null && ord.gt(a, head)) { + underlying.poll() + underlying.offer(a) + } else { + false + } + } +} diff --git a/core/src/main/scala-2.13/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala-2.13/org/apache/spark/util/TimeStampedHashMap.scala new file mode 100644 index 0000000000000..9c860061b5862 --- /dev/null +++ b/core/src/main/scala-2.13/org/apache/spark/util/TimeStampedHashMap.scala @@ -0,0 +1,143 @@ +/* + * 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.util + +import java.util.Map.Entry +import java.util.Set +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.internal.Logging + +private[spark] case class TimeStampedValue[V](value: V, timestamp: Long) + +/** + * This is a custom implementation of scala.collection.mutable.Map which stores the insertion + * timestamp along with each key-value pair. If specified, the timestamp of each pair can be + * updated every time it is accessed. Key-value pairs whose timestamp are older than a particular + * threshold time can then be removed using the clearOldValues method. This is intended to + * be a drop-in replacement of scala.collection.mutable.HashMap. + * + * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed + */ +private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) + extends mutable.Map[A, B]() with Logging { + + // Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation. + + private val internalMap = new ConcurrentHashMap[A, TimeStampedValue[B]]() + + def get(key: A): Option[B] = { + val value = internalMap.get(key) + if (value != null && updateTimeStampOnGet) { + internalMap.replace(key, value, TimeStampedValue(value.value, currentTime)) + } + Option(value).map(_.value) + } + + def iterator: Iterator[(A, B)] = { + getEntrySet.iterator.asScala.map(kv => (kv.getKey, kv.getValue.value)) + } + + def getEntrySet: Set[Entry[A, TimeStampedValue[B]]] = internalMap.entrySet + + override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { + val newMap = new TimeStampedHashMap[A, B1] + val oldInternalMap = this.internalMap.asInstanceOf[ConcurrentHashMap[A, TimeStampedValue[B1]]] + newMap.internalMap.putAll(oldInternalMap) + kv match { case (a, b) => newMap.internalMap.put(a, TimeStampedValue(b, currentTime)) } + newMap + } + + override def addOne(kv: (A, B)): this.type = { + kv match { case (a, b) => internalMap.put(a, TimeStampedValue(b, currentTime)) } + this + } + + override def subtractOne(key: A): this.type = { + internalMap.remove(key) + this + } + + override def update(key: A, value: B): Unit = { + this += ((key, value)) + } + + override def apply(key: A): B = { + get(key).getOrElse { throw new NoSuchElementException() } + } + + override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = { + internalMap.asScala.map { case (k, TimeStampedValue(v, t)) => (k, v) }.filter(p) + } + + override def empty: mutable.Map[A, B] = new TimeStampedHashMap[A, B]() + + override def size: Int = internalMap.size + + override def foreach[U](f: ((A, B)) => U): Unit = { + val it = getEntrySet.iterator + while(it.hasNext) { + val entry = it.next() + val kv = (entry.getKey, entry.getValue.value) + f(kv) + } + } + + def putIfAbsent(key: A, value: B): Option[B] = { + val prev = internalMap.putIfAbsent(key, TimeStampedValue(value, currentTime)) + Option(prev).map(_.value) + } + + def putAll(map: Map[A, B]): Unit = { + map.foreach { case (k, v) => update(k, v) } + } + + def toMap: Map[A, B] = iterator.toMap + + def clearOldValues(threshTime: Long, f: (A, B) => Unit): Unit = { + val it = getEntrySet.iterator + while (it.hasNext) { + val entry = it.next() + if (entry.getValue.timestamp < threshTime) { + f(entry.getKey, entry.getValue.value) + logDebug("Removing key " + entry.getKey) + it.remove() + } + } + } + + /** Removes old key-value pairs that have timestamp earlier than `threshTime`. */ + def clearOldValues(threshTime: Long): Unit = { + clearOldValues(threshTime, (_, _) => ()) + } + + private def currentTime: Long = System.currentTimeMillis + + // For testing + + def getTimeStampedValue(key: A): Option[TimeStampedValue[B]] = { + Option(internalMap.get(key)) + } + + def getTimestamp(key: A): Option[Long] = { + getTimeStampedValue(key).map(_.timestamp) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala index 5d3693190cc1f..9d5f1aac3391b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala @@ -112,8 +112,6 @@ private[spark] class CompactBuffer[T: ClassTag] extends Seq[T] with Serializable override def length: Int = curSize - override def size: Int = curSize - override def iterator: Iterator[T] = new Iterator[T] { private var pos = 0 override def hasNext: Boolean = pos < curSize diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 14409c3661baa..7f40b469a95e9 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -367,7 +367,7 @@ class ExternalAppendOnlyMap[K, V, C]( private def removeFromBuffer[T](buffer: ArrayBuffer[T], index: Int): T = { val elem = buffer(index) buffer(index) = buffer(buffer.size - 1) // This also works if index == buffer.size - 1 - buffer.reduceToSize(buffer.size - 1) + buffer.trimEnd(1) elem } diff --git a/dev/change-scala-version.sh b/dev/change-scala-version.sh index 4054d530d065e..06411b9b12a0d 100755 --- a/dev/change-scala-version.sh +++ b/dev/change-scala-version.sh @@ -19,7 +19,7 @@ set -e -VALID_VERSIONS=( 2.12 ) +VALID_VERSIONS=( 2.12 2.13 ) usage() { echo "Usage: $(basename $0) [-h|--help] diff --git a/repl/pom.xml b/repl/pom.xml index c7de67e41ca94..3aee53cc9599a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -32,8 +32,6 @@ repl - src/main/scala-${scala.binary.version} - src/test/scala-${scala.binary.version} @@ -146,7 +144,7 @@ - ${extra.source.dir} + src/main/scala-${scala.binary.version} @@ -158,7 +156,7 @@ - ${extra.testsource.dir} + src/test/scala-${scala.binary.version} diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index b416994195d01..304e3a7457fcc 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -167,6 +167,24 @@ true + + org.codehaus.mojo + build-helper-maven-plugin + + + add-sources + generate-sources + + add-source + + + + src/main/scala-${scala.binary.version} + + + + + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala similarity index 94% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala rename to sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala index 9f4a0f2b7017a..75a8bec018a1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -31,6 +31,8 @@ object AttributeMap { class AttributeMap[A](val baseMap: Map[ExprId, (Attribute, A)]) extends Map[Attribute, A] with Serializable { + // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. + override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2) override def contains(k: Attribute): Boolean = get(k).isDefined diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala similarity index 97% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala rename to sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala index 7e8e7b8cd5f18..5198c6dd2fdf0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala @@ -57,6 +57,8 @@ class ExpressionSet protected( protected val originals: mutable.Buffer[Expression] = new ArrayBuffer) extends Set[Expression] { + // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. + protected def add(e: Expression): Unit = { if (!e.deterministic) { originals += e diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala similarity index 96% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala rename to sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index 06f95989f2e3a..699b42bd6d60d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -30,6 +30,8 @@ import java.util.Locale class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Map[String, T] with Serializable { + // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. + val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(Locale.ROOT))) override def get(k: String): Option[T] = keyLowerCasedMap.get(k.toLowerCase(Locale.ROOT)) diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala new file mode 100644 index 0000000000000..4caa3d0461875 --- /dev/null +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -0,0 +1,46 @@ +/* + * 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 + +/** + * Builds a map that is keyed by an Attribute's expression id. Using the expression id allows values + * to be looked up even when the attributes used differ cosmetically (i.e., the capitalization + * of the name, or the expected nullability). + */ +object AttributeMap { + def apply[A](kvs: Seq[(Attribute, A)]): AttributeMap[A] = { + new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap) + } +} + +class AttributeMap[A](val baseMap: Map[ExprId, (Attribute, A)]) + extends Map[Attribute, A] with Serializable { + + // Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation. + + override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2) + + override def contains(k: Attribute): Boolean = get(k).isDefined + + override def updated[B1 >: A](key: Attribute, value: B1): Map[Attribute, B1] = + baseMap.values.toMap + (key -> value) + + override def iterator: Iterator[(Attribute, A)] = baseMap.valuesIterator + + override def removed(key: Attribute): Map[Attribute, A] = baseMap.values.toMap - key +} diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala new file mode 100644 index 0000000000000..cf74b98b31b0c --- /dev/null +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala @@ -0,0 +1,100 @@ +/* + * 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 scala.collection.mutable.ArrayBuffer + +object ExpressionSet { + /** Constructs a new [[ExpressionSet]] by applying [[Canonicalize]] to `expressions`. */ + def apply(expressions: TraversableOnce[Expression]): ExpressionSet = { + val set = new ExpressionSet() + expressions.foreach(set.add) + set + } +} + +/** + * A [[Set]] where membership is determined based on determinacy and a canonical representation of + * an [[Expression]] (i.e. one that attempts to ignore cosmetic differences). + * See [[Canonicalize]] for more details. + * + * Internally this set uses the canonical representation, but keeps also track of the original + * expressions to ease debugging. Since different expressions can share the same canonical + * representation, this means that operations that extract expressions from this set are only + * guaranteed to see at least one such expression. For example: + * + * {{{ + * val set = ExpressionSet(a + 1, 1 + a) + * + * set.iterator => Iterator(a + 1) + * set.contains(a + 1) => true + * set.contains(1 + a) => true + * set.contains(a + 2) => false + * }}} + * + * For non-deterministic expressions, they are always considered as not contained in the [[Set]]. + * On adding a non-deterministic expression, simply append it to the original expressions. + * This is consistent with how we define `semanticEquals` between two expressions. + */ +class ExpressionSet protected( + protected val baseSet: mutable.Set[Expression] = new mutable.HashSet, + protected val originals: mutable.Buffer[Expression] = new ArrayBuffer) + extends Set[Expression] { + + // Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation. + + protected def add(e: Expression): Unit = { + if (!e.deterministic) { + originals += e + } else if (!baseSet.contains(e.canonicalized) ) { + baseSet.add(e.canonicalized) + originals += e + } + } + + override def contains(elem: Expression): Boolean = baseSet.contains(elem.canonicalized) + + override def incl(elem: Expression): ExpressionSet = { + val newSet = new ExpressionSet(baseSet.clone(), originals.clone()) + newSet.add(elem) + newSet + } + + override def excl(elem: Expression): ExpressionSet = { + if (elem.deterministic) { + val newBaseSet = baseSet.clone().filterNot(_ == elem.canonicalized) + val newOriginals = originals.clone().filterNot(_.canonicalized == elem.canonicalized) + new ExpressionSet(newBaseSet, newOriginals) + } else { + new ExpressionSet(baseSet.clone(), originals.clone()) + } + } + + override def iterator: Iterator[Expression] = originals.iterator + + /** + * Returns a string containing both the post [[Canonicalize]] expressions and the original + * expressions in this set. + */ + def toDebugString: String = + s""" + |baseSet: ${baseSet.mkString(", ")} + |originals: ${originals.mkString(", ")} + """.stripMargin +} diff --git a/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala new file mode 100644 index 0000000000000..1f8cc6c0f83a2 --- /dev/null +++ b/sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -0,0 +1,59 @@ +/* + * 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.util + +import java.util.Locale + +/** + * Builds a map in which keys are case insensitive. Input map can be accessed for cases where + * case-sensitive information is required. The primary constructor is marked private to avoid + * nested case-insensitive map creation, otherwise the keys in the original map will become + * case-insensitive in this scenario. + * Note: CaseInsensitiveMap is serializable. However, after transformation, e.g. `filterKeys()`, + * it may become not serializable. + */ +class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Map[String, T] + with Serializable { + + // Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation. + + val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(Locale.ROOT))) + + override def get(k: String): Option[T] = keyLowerCasedMap.get(k.toLowerCase(Locale.ROOT)) + + override def contains(k: String): Boolean = + keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) + + override def updated[B1 >: T](key: String, value: B1): Map[String, B1] = { + new CaseInsensitiveMap[B1](originalMap + (key -> value)) + } + + override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator + + override def removed(key: String): Map[String, T] = { + new CaseInsensitiveMap(originalMap.filter(!_._1.equalsIgnoreCase(key))) + } +} + +object CaseInsensitiveMap { + def apply[T](params: Map[String, T]): CaseInsensitiveMap[T] = params match { + case caseSensitiveMap: CaseInsensitiveMap[T] => caseSensitiveMap + case _ => new CaseInsensitiveMap(params) + } +} + diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 87874e2a0eb07..0e664eca6a820 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -198,6 +198,7 @@ v${hive.version.short}/src/main/scala v${hive.version.short}/src/main/java + src/main/scala-${scala.binary.version} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala-2.12/org/apache/spark/sql/execution/streaming/StreamProgress.scala similarity index 95% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala rename to sql/core/src/main/scala-2.12/org/apache/spark/sql/execution/streaming/StreamProgress.scala index c6634f51d0382..9e5bb8e061ccb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala +++ b/sql/core/src/main/scala-2.12/org/apache/spark/sql/execution/streaming/StreamProgress.scala @@ -29,6 +29,8 @@ class StreamProgress( new immutable.HashMap[SparkDataStream, OffsetV2]) extends scala.collection.immutable.Map[SparkDataStream, OffsetV2] { + // Note: this class supports Scala 2.12. A parallel source tree has a 2.13 implementation. + def toOffsetSeq(source: Seq[SparkDataStream], metadata: OffsetSeqMetadata): OffsetSeq = { OffsetSeq(source.map(get), Some(metadata)) } diff --git a/sql/core/src/main/scala-2.13/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala-2.13/org/apache/spark/sql/execution/streaming/StreamProgress.scala new file mode 100644 index 0000000000000..0aa29640899c6 --- /dev/null +++ b/sql/core/src/main/scala-2.13/org/apache/spark/sql/execution/streaming/StreamProgress.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.streaming + +import scala.collection.immutable + +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} + +/** + * A helper class that looks like a Map[Source, Offset]. + */ +class StreamProgress( + val baseMap: immutable.Map[SparkDataStream, OffsetV2] = + new immutable.HashMap[SparkDataStream, OffsetV2]) + extends scala.collection.immutable.Map[SparkDataStream, OffsetV2] { + + // Note: this class supports Scala 2.13. A parallel source tree has a 2.12 implementation. + + def toOffsetSeq(source: Seq[SparkDataStream], metadata: OffsetSeqMetadata): OffsetSeq = { + OffsetSeq(source.map(get), Some(metadata)) + } + + override def toString: String = + baseMap.map { case (k, v) => s"$k: $v"}.mkString("{", ",", "}") + + override def updated[B1 >: OffsetV2](key: SparkDataStream, value: B1): Map[SparkDataStream, B1] = + baseMap + (key -> value) + + override def get(key: SparkDataStream): Option[OffsetV2] = baseMap.get(key) + + override def iterator: Iterator[(SparkDataStream, OffsetV2)] = baseMap.iterator + + override def removed(key: SparkDataStream): Map[SparkDataStream, OffsetV2] = baseMap - key + + def ++(updates: IterableOnce[(SparkDataStream, OffsetV2)]): StreamProgress = { + new StreamProgress(baseMap ++ updates) + } +} From 196ea936c39d621a605740ea026cd18974da1112 Mon Sep 17 00:00:00 2001 From: xiaodeshan Date: Tue, 3 Dec 2019 14:27:48 -0800 Subject: [PATCH 0780/1387] [SPARK-30106][SQL][TEST] Fix the test of DynamicPartitionPruningSuite ### What changes were proposed in this pull request? Changed the test **DPP triggers only for certain types of query** in **DynamicPartitionPruningSuite**. ### Why are the changes needed? The sql has no partition key. The description "no predicate on the dimension table" is not right. So fix it. ``` Given("no predicate on the dimension table") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { val df = sql( """ |SELECT * FROM fact_sk f |JOIN dim_store s |ON f.date_id = s.store_id """.stripMargin) ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? Updated UT Closes #26744 from deshanxiao/30106. Authored-by: xiaodeshan Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/DynamicPartitionPruningSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index e522aa1809d29..e0ea36157e56a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -423,7 +423,7 @@ class DynamicPartitionPruningSuite """ |SELECT * FROM fact_sk f |JOIN dim_store s - |ON f.date_id = s.store_id + |ON f.store_id = s.store_id """.stripMargin) checkPartitionPruningPredicate(df, false, false) From 60f20e5ea2000ab8f4a593b5e4217fd5637c5e22 Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Tue, 3 Dec 2019 14:31:06 -0800 Subject: [PATCH 0781/1387] [SPARK-30060][CORE] Rename metrics enable/disable configs ### What changes were proposed in this pull request? This proposes to introduce a naming convention for Spark metrics configuration parameters used to enable/disable metrics source reporting using the Dropwizard metrics library: `spark.metrics.sourceNameCamelCase.enabled` and update 2 parameters to use this naming convention. ### Why are the changes needed? Currently Spark has a few parameters to enable/disable metrics reporting. Their naming pattern is not uniform and this can create confusion. Currently we have: `spark.metrics.static.sources.enabled` `spark.app.status.metrics.enabled` `spark.sql.streaming.metricsEnabled` ### Does this PR introduce any user-facing change? Update parameters for enabling/disabling metrics reporting new in Spark 3.0: `spark.metrics.static.sources.enabled` -> `spark.metrics.staticSources.enabled`, `spark.app.status.metrics.enabled` -> `spark.metrics.appStatusSource.enabled`. Note: `spark.sql.streaming.metricsEnabled` is left unchanged as it is already in use in Spark 2.x. ### How was this patch tested? Manually tested Closes #26692 from LucaCanali/uniformNamingMetricsEnableParameters. Authored-by: Luca Canali Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/internal/config/Status.scala | 4 ++-- .../org/apache/spark/internal/config/package.scala | 2 +- .../org/apache/spark/status/AppStatusSource.scala | 4 ++-- docs/monitoring.md | 12 ++++++------ 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/Status.scala b/core/src/main/scala/org/apache/spark/internal/config/Status.scala index 3e6a4e9810664..3cc00a6f094cf 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Status.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Status.scala @@ -55,8 +55,8 @@ private[spark] object Status { .intConf .createWithDefault(Int.MaxValue) - val APP_STATUS_METRICS_ENABLED = - ConfigBuilder("spark.app.status.metrics.enabled") + val METRICS_APP_STATUS_SOURCE_ENABLED = + ConfigBuilder("spark.metrics.appStatusSource.enabled") .doc("Whether Dropwizard/Codahale metrics " + "will be reported for the status of the running spark app.") .booleanConf 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 489c5f8fa6b76..039911f260955 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 @@ -638,7 +638,7 @@ package object config { .createOptional private[spark] val METRICS_STATIC_SOURCES_ENABLED = - ConfigBuilder("spark.metrics.static.sources.enabled") + ConfigBuilder("spark.metrics.staticSources.enabled") .doc("Whether to register static sources with the metrics system.") .booleanConf .createWithDefault(true) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala index f6a21578ff499..20f171bd3c375 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala @@ -22,7 +22,7 @@ import AppStatusSource.getCounter import com.codahale.metrics.{Counter, Gauge, MetricRegistry} import org.apache.spark.SparkConf -import org.apache.spark.internal.config.Status.APP_STATUS_METRICS_ENABLED +import org.apache.spark.internal.config.Status.METRICS_APP_STATUS_SOURCE_ENABLED import org.apache.spark.metrics.source.Source private [spark] class JobDuration(val value: AtomicLong) extends Gauge[Long] { @@ -71,7 +71,7 @@ private[spark] object AppStatusSource { } def createSource(conf: SparkConf): Option[AppStatusSource] = { - Option(conf.get(APP_STATUS_METRICS_ENABLED)) + Option(conf.get(METRICS_APP_STATUS_SOURCE_ENABLED)) .filter(identity) .map { _ => new AppStatusSource() } } diff --git a/docs/monitoring.md b/docs/monitoring.md index f094d0ba0dfc3..2aed452adb05f 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -924,7 +924,7 @@ This is the component with the largest amount of instrumented metrics - namespace=HiveExternalCatalog - **note:**: these metrics are conditional to a configuration parameter: - `spark.metrics.static.sources.enabled` (default is true) + `spark.metrics.staticSources.enabled` (default is true) - fileCacheHits.count - filesDiscovered.count - hiveClientCalls.count @@ -933,7 +933,7 @@ This is the component with the largest amount of instrumented metrics - namespace=CodeGenerator - **note:**: these metrics are conditional to a configuration parameter: - `spark.metrics.static.sources.enabled` (default is true) + `spark.metrics.staticSources.enabled` (default is true) - compilationTime (histogram) - generatedClassSize (histogram) - generatedMethodSize (histogram) @@ -962,8 +962,8 @@ This is the component with the largest amount of instrumented metrics - queue.executorManagement.listenerProcessingTime (timer) - namespace=appStatus (all metrics of type=counter) - - **note:** Introduced in Spark 3.0. Conditional to configuration parameter: - `spark.app.status.metrics.enabled=true` (default is false) + - **note:** Introduced in Spark 3.0. Conditional to a configuration parameter: + `spark.metrics.appStatusSource.enabled` (default is false) - stages.failedStages.count - stages.skippedStages.count - stages.completedStages.count @@ -1057,7 +1057,7 @@ when running in local mode. - namespace=HiveExternalCatalog - **note:**: these metrics are conditional to a configuration parameter: - `spark.metrics.static.sources.enabled` (default is true) + `spark.metrics.staticSources.enabled` (default is true) - fileCacheHits.count - filesDiscovered.count - hiveClientCalls.count @@ -1066,7 +1066,7 @@ when running in local mode. - namespace=CodeGenerator - **note:**: these metrics are conditional to a configuration parameter: - `spark.metrics.static.sources.enabled` (default is true) + `spark.metrics.staticSources.enabled` (default is true) - compilationTime (histogram) - generatedClassSize (histogram) - generatedMethodSize (histogram) From f3abee377d1b86826498a1be329a1c82203162f5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 3 Dec 2019 14:33:36 -0800 Subject: [PATCH 0782/1387] [SPARK-30051][BUILD] Clean up hadoop-3.2 dependency ### What changes were proposed in this pull request? This PR aims to cut `org.eclipse.jetty:jetty-webapp`and `org.eclipse.jetty:jetty-xml` transitive dependency from `hadoop-common`. ### Why are the changes needed? This will simplify our dependency management by the removal of unused dependencies. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the GitHub Action with all combinations and the Jenkins UT with (Hadoop-3.2). Closes #26742 from dongjoon-hyun/SPARK-30051. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 -- pom.xml | 10 ++++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index b2c1777ee2ad0..99552febd1989 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -135,8 +135,6 @@ jersey-container-servlet-core-2.29.jar jersey-hk2-2.29.jar jersey-media-jaxb-2.29.jar jersey-server-2.29.jar -jetty-webapp-9.4.18.v20190429.jar -jetty-xml-9.4.18.v20190429.jar jline-2.14.6.jar joda-time-2.10.5.jar jodd-core-3.5.2.jar diff --git a/pom.xml b/pom.xml index 0ce87233a9d5f..53f1c50e63eea 100644 --- a/pom.xml +++ b/pom.xml @@ -406,6 +406,12 @@ ${jetty.version} provided
    + + org.eclipse.jetty + jetty-webapp + ${jetty.version} + provided + com.google.guava guava @@ -1030,6 +1036,10 @@ javax.ws.rs jsr311-api + + org.eclipse.jetty + jetty-webapp + From 3dd3a623f293bc7fd4937c95f06b967fa187b0f1 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 4 Dec 2019 10:10:30 +0900 Subject: [PATCH 0783/1387] [SPARK-27990][SPARK-29903][PYTHON] Add recursiveFileLookup option to Python DataFrameReader MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? As a follow-up to #24830, this PR adds the `recursiveFileLookup` option to the Python DataFrameReader API. ### Why are the changes needed? This PR maintains Python feature parity with Scala. ### Does this PR introduce any user-facing change? Yes. Before this PR, you'd only be able to use this option as follows: ```python spark.read.option("recursiveFileLookup", True).text("test-data").show() ``` With this PR, you can reference the option from within the format-specific method: ```python spark.read.text("test-data", recursiveFileLookup=True).show() ``` This option now also shows up in the Python API docs. ### How was this patch tested? I tested this manually by creating the following directories with dummy data: ``` test-data ├── 1.txt └── nested └── 2.txt test-parquet ├── nested │ ├── _SUCCESS │ ├── part-00000-...-.parquet ├── _SUCCESS ├── part-00000-...-.parquet ``` I then ran the following tests and confirmed the output looked good: ```python spark.read.parquet("test-parquet", recursiveFileLookup=True).show() spark.read.text("test-data", recursiveFileLookup=True).show() spark.read.csv("test-data", recursiveFileLookup=True).show() ``` `python/pyspark/sql/tests/test_readwriter.py` seems pretty sparse. I'm happy to add my tests there, though it seems we have been deferring testing like this to the Scala side of things. Closes #26718 from nchammas/SPARK-27990-recursiveFileLookup-python. Authored-by: Nicholas Chammas Signed-off-by: HyukjinKwon --- python/pyspark/sql/readwriter.py | 36 +++++++++++++++++++++++++------- python/pyspark/sql/streaming.py | 36 +++++++++++++++++++++++++------- 2 files changed, 56 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index a392b29bb379f..3f8a3a7595458 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -171,7 +171,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, samplingRatio=None, - dropFieldIfAllNull=None, encoding=None, locale=None): + dropFieldIfAllNull=None, encoding=None, locale=None, recursiveFileLookup=None): """ Loads JSON files and returns the results as a :class:`DataFrame`. @@ -247,6 +247,10 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param locale: sets a locale as language tag in IETF BCP 47 format. If None is set, it uses the default value, ``en-US``. For instance, ``locale`` is used while parsing dates and timestamps. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. + + .. _partition discovery: /sql-data-sources-parquet.html#partition-discovery >>> df1 = spark.read.json('python/test_support/sql/people.json') >>> df1.dtypes @@ -266,7 +270,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, samplingRatio=samplingRatio, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, - locale=locale) + locale=locale, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -300,9 +304,12 @@ def table(self, tableName): return self._df(self._jreader.table(tableName)) @since(1.4) - def parquet(self, *paths): + def parquet(self, *paths, **options): """Loads Parquet files, returning the result as a :class:`DataFrame`. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. + You can set the following Parquet-specific option(s) for reading Parquet files: * ``mergeSchema``: sets whether we should merge schemas collected from all \ Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. \ @@ -312,11 +319,13 @@ def parquet(self, *paths): >>> df.dtypes [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')] """ + recursiveFileLookup = options.get('recursiveFileLookup', None) + self._set_opts(recursiveFileLookup=recursiveFileLookup) return self._df(self._jreader.parquet(_to_seq(self._spark._sc, paths))) @ignore_unicode_prefix @since(1.6) - def text(self, paths, wholetext=False, lineSep=None): + def text(self, paths, wholetext=False, lineSep=None, recursiveFileLookup=None): """ Loads text files and returns a :class:`DataFrame` whose schema starts with a string column named "value", and followed by partitioned columns if there @@ -329,6 +338,8 @@ def text(self, paths, wholetext=False, lineSep=None): :param wholetext: if true, read each file from input path(s) as a single row. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. >>> df = spark.read.text('python/test_support/sql/text-test.txt') >>> df.collect() @@ -337,7 +348,8 @@ def text(self, paths, wholetext=False, lineSep=None): >>> df.collect() [Row(value=u'hello\\nthis')] """ - self._set_opts(wholetext=wholetext, lineSep=lineSep) + self._set_opts( + wholetext=wholetext, lineSep=lineSep, recursiveFileLookup=recursiveFileLookup) if isinstance(paths, basestring): paths = [paths] return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(paths))) @@ -349,7 +361,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None, maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, - samplingRatio=None, enforceSchema=None, emptyValue=None, locale=None, lineSep=None): + samplingRatio=None, enforceSchema=None, emptyValue=None, locale=None, lineSep=None, + recursiveFileLookup=None): r"""Loads a CSV file and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -457,6 +470,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. Maximum length is 1 character. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. >>> df = spark.read.csv('python/test_support/sql/ages.csv') >>> df.dtypes @@ -476,7 +491,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine, charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, samplingRatio=samplingRatio, - enforceSchema=enforceSchema, emptyValue=emptyValue, locale=locale, lineSep=lineSep) + enforceSchema=enforceSchema, emptyValue=emptyValue, locale=locale, lineSep=lineSep, + recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -504,13 +520,17 @@ def func(iterator): raise TypeError("path can be only string, list or RDD") @since(1.5) - def orc(self, path): + def orc(self, path, recursiveFileLookup=None): """Loads ORC files, returning the result as a :class:`DataFrame`. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. + >>> df = spark.read.orc('python/test_support/sql/orc_partitioned') >>> df.dtypes [('a', 'bigint'), ('b', 'int'), ('c', 'int')] """ + self._set_opts(recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): path = [path] return self._df(self._jreader.orc(_to_seq(self._spark._sc, path))) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index f224526a4cc74..93b4c78953860 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -411,7 +411,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, locale=None, - dropFieldIfAllNull=None, encoding=None): + dropFieldIfAllNull=None, encoding=None, recursiveFileLookup=None): """ Loads a JSON file stream and returns the results as a :class:`DataFrame`. @@ -487,6 +487,10 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, the JSON files. For example UTF-16BE, UTF-32LE. If None is set, the encoding of input JSON will be detected automatically when the multiLine option is set to ``true``. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. + + .. _partition discovery: /sql-data-sources-parquet.html#partition-discovery >>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema) >>> json_sdf.isStreaming @@ -502,33 +506,41 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale, - dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding) + dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, + recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.json(path)) else: raise TypeError("path can be only a single string") @since(2.3) - def orc(self, path): + def orc(self, path, recursiveFileLookup=None): """Loads a ORC file stream, returning the result as a :class:`DataFrame`. .. note:: Evolving. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. + >>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp()) >>> orc_sdf.isStreaming True >>> orc_sdf.schema == sdf_schema True """ + self._set_opts(recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.orc(path)) else: raise TypeError("path can be only a single string") @since(2.0) - def parquet(self, path): + def parquet(self, path, recursiveFileLookup=None): """Loads a Parquet file stream, returning the result as a :class:`DataFrame`. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. + You can set the following Parquet-specific option(s) for reading Parquet files: * ``mergeSchema``: sets whether we should merge schemas collected from all \ Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. \ @@ -542,6 +554,7 @@ def parquet(self, path): >>> parquet_sdf.schema == sdf_schema True """ + self._set_opts(recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.parquet(path)) else: @@ -549,7 +562,7 @@ def parquet(self, path): @ignore_unicode_prefix @since(2.0) - def text(self, path, wholetext=False, lineSep=None): + def text(self, path, wholetext=False, lineSep=None, recursiveFileLookup=None): """ Loads a text file stream and returns a :class:`DataFrame` whose schema starts with a string column named "value", and followed by partitioned columns if there @@ -564,6 +577,8 @@ def text(self, path, wholetext=False, lineSep=None): :param wholetext: if true, read each file from input path(s) as a single row. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. >>> text_sdf = spark.readStream.text(tempfile.mkdtemp()) >>> text_sdf.isStreaming @@ -571,7 +586,8 @@ def text(self, path, wholetext=False, lineSep=None): >>> "value" in str(text_sdf.schema) True """ - self._set_opts(wholetext=wholetext, lineSep=lineSep) + self._set_opts( + wholetext=wholetext, lineSep=lineSep, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.text(path)) else: @@ -584,7 +600,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None, maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, - enforceSchema=None, emptyValue=None, locale=None, lineSep=None): + enforceSchema=None, emptyValue=None, locale=None, lineSep=None, + recursiveFileLookup=None): r"""Loads a CSV file stream and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -687,6 +704,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. Maximum length is 1 character. + :param recursiveFileLookup: recursively scan a directory for files. Using this option + disables `partition discovery`_. >>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema) >>> csv_sdf.isStreaming @@ -704,7 +723,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine, charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema, - emptyValue=emptyValue, locale=locale, lineSep=lineSep) + emptyValue=emptyValue, locale=locale, lineSep=lineSep, + recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.csv(path)) else: From 5496e980e9a4dc20e84db1fa6c4b5426dce60b19 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Wed, 4 Dec 2019 09:50:00 +0800 Subject: [PATCH 0784/1387] [SPARK-30109][ML] PCA use BLAS.gemv for sparse vectors ### What changes were proposed in this pull request? When PCA was first impled in [SPARK-5521](https://issues.apache.org/jira/browse/SPARK-5521), at that time Matrix.multiply(BLAS.gemv internally) did not support sparse vector. So worked around it by applying a sparse matrix multiplication. Since [SPARK-7681](https://issues.apache.org/jira/browse/SPARK-7681), BLAS.gemv supported sparse vector. So we can directly use Matrix.multiply now. ### Why are the changes needed? for simplity ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing testsuites Closes #26745 from zhengruifeng/pca_mul. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../org/apache/spark/ml/feature/PCA.scala | 18 ++---------------- .../org/apache/spark/mllib/feature/PCA.scala | 13 +------------ 2 files changed, 3 insertions(+), 28 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 69dcacbb0c183..27a3854d39b47 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -147,22 +147,8 @@ class PCAModel private[ml] ( override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val func = { vector: Vector => - vector match { - case dv: DenseVector => - pc.transpose.multiply(dv) - case SparseVector(size, indices, values) => - /* SparseVector -> single row SparseMatrix */ - val sm = Matrices.sparse(size, 1, Array(0, indices.length), indices, values).transpose - val projection = sm.multiply(pc) - Vectors.dense(projection.values) - case _ => - throw new IllegalArgumentException("Unsupported vector format. Expected " + - s"SparseVector or DenseVector. Instead got: ${vector.getClass}") - } - } - - val transformer = udf(func) + val transposed = pc.transpose + val transformer = udf { vector: Vector => transposed.multiply(vector) } dataset.withColumn($(outputCol), transformer(col($(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala index 2fc517cad12db..78a783a787724 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala @@ -111,18 +111,7 @@ class PCAModel private[spark] ( */ @Since("1.4.0") override def transform(vector: Vector): Vector = { - vector match { - case dv: DenseVector => - pc.transpose.multiply(dv) - case SparseVector(size, indices, values) => - /* SparseVector -> single row SparseMatrix */ - val sm = Matrices.sparse(size, 1, Array(0, indices.length), indices, values).transpose - val projection = sm.multiply(pc) - Vectors.dense(projection.values) - case _ => - throw new IllegalArgumentException("Unsupported vector format. Expected " + - s"SparseVector or DenseVector. Instead got: ${vector.getClass}") - } + pc.transpose.multiply(vector) } } From 708cf16be9bb131ba8980c494ca497a4d187e160 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 3 Dec 2019 17:59:02 -0800 Subject: [PATCH 0785/1387] [SPARK-30111][K8S] Apt-get update to fix debian issues ### What changes were proposed in this pull request? Added apt-get update as per [docker best-practices](https://docs.docker.com/develop/develop-images/dockerfile_best-practices/#apt-get) ### Why are the changes needed? Builder is failing because: Without doing apt-get update, the APT lists get outdated and begins referring to package versions that no longer exist, hence the 404 trying to download them (Debian does not keep old versions in the archive when a package is updated). ### Does this PR introduce any user-facing change? no ### How was this patch tested? k8s builder Closes #26753 from ifilonenko/SPARK-30111. Authored-by: Ilan Filonenko Signed-off-by: shane knapp --- .../docker/src/main/dockerfiles/spark/bindings/R/Dockerfile | 2 +- .../src/main/dockerfiles/spark/bindings/python/Dockerfile | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile index c65824c3f9a88..59f375b707ca7 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile @@ -25,7 +25,7 @@ USER 0 RUN mkdir ${SPARK_HOME}/R -RUN apt install -y r-base r-base-dev && rm -rf /var/cache/apt/* +RUN apt-get update && apt install -y r-base r-base-dev && rm -rf /var/cache/apt/* COPY R ${SPARK_HOME}/R ENV R_HOME /usr/lib/R diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile index a7dce048710d8..8dfc5f7ff60c5 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -25,7 +25,8 @@ USER 0 RUN mkdir ${SPARK_HOME}/python # TODO: Investigate running both pip and pip3 via virtualenvs -RUN apt install -y python python-pip && \ +RUN apt-get update && \ + apt install -y python python-pip && \ apt install -y python3 python3-pip && \ # We remove ensurepip since it adds no functionality since pip is # installed on the image and it just takes up 1.6MB on the image From e766a323bc3462763b03f9d892a0b3fdf2cb29db Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 4 Dec 2019 11:31:57 +0900 Subject: [PATCH 0786/1387] [SPARK-30091][SQL][PYTHON] Document mergeSchema option directly in the PySpark Parquet APIs ### What changes were proposed in this pull request? This change properly documents the `mergeSchema` option directly in the Python APIs for reading Parquet data. ### Why are the changes needed? The docstring for `DataFrameReader.parquet()` mentions `mergeSchema` but doesn't show it in the API. It seems like a simple oversight. Before this PR, you'd have to do this to use `mergeSchema`: ```python spark.read.option('mergeSchema', True).parquet('test-parquet').show() ``` After this PR, you can use the option as (I believe) it was intended to be used: ```python spark.read.parquet('test-parquet', mergeSchema=True).show() ``` ### Does this PR introduce any user-facing change? Yes, this PR changes the signatures of `DataFrameReader.parquet()` and `DataStreamReader.parquet()` to match their docstrings. ### How was this patch tested? Testing the `mergeSchema` option directly seems to be left to the Scala side of the codebase. I tested my change manually to confirm the API works. I also confirmed that setting `spark.sql.parquet.mergeSchema` at the session does not get overridden by leaving `mergeSchema` at its default when calling `parquet()`: ``` >>> spark.conf.set('spark.sql.parquet.mergeSchema', True) >>> spark.range(3).write.parquet('test-parquet/id') >>> spark.range(3).withColumnRenamed('id', 'name').write.parquet('test-parquet/name') >>> spark.read.option('recursiveFileLookup', True).parquet('test-parquet').show() +----+----+ | id|name| +----+----+ |null| 1| |null| 2| |null| 0| | 1|null| | 2|null| | 0|null| +----+----+ >>> spark.read.option('recursiveFileLookup', True).parquet('test-parquet', mergeSchema=False).show() +----+ | id| +----+ |null| |null| |null| | 1| | 2| | 0| +----+ ``` Closes #26730 from nchammas/parquet-merge-schema. Authored-by: Nicholas Chammas Signed-off-by: HyukjinKwon --- python/pyspark/sql/readwriter.py | 14 +++++++------- python/pyspark/sql/streaming.py | 19 +++++++++---------- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 3f8a3a7595458..153776d8b4171 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -305,22 +305,22 @@ def table(self, tableName): @since(1.4) def parquet(self, *paths, **options): - """Loads Parquet files, returning the result as a :class:`DataFrame`. + """ + Loads Parquet files, returning the result as a :class:`DataFrame`. + :param mergeSchema: sets whether we should merge schemas collected from all + Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. + The default value is specified in ``spark.sql.parquet.mergeSchema``. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. - You can set the following Parquet-specific option(s) for reading Parquet files: - * ``mergeSchema``: sets whether we should merge schemas collected from all \ - Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. \ - The default value is specified in ``spark.sql.parquet.mergeSchema``. - >>> df = spark.read.parquet('python/test_support/sql/parquet_partitioned') >>> df.dtypes [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')] """ + mergeSchema = options.get('mergeSchema', None) recursiveFileLookup = options.get('recursiveFileLookup', None) - self._set_opts(recursiveFileLookup=recursiveFileLookup) + self._set_opts(mergeSchema=mergeSchema, recursiveFileLookup=recursiveFileLookup) return self._df(self._jreader.parquet(_to_seq(self._spark._sc, paths))) @ignore_unicode_prefix diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 93b4c78953860..6359c31ba5655 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -535,26 +535,25 @@ def orc(self, path, recursiveFileLookup=None): raise TypeError("path can be only a single string") @since(2.0) - def parquet(self, path, recursiveFileLookup=None): - """Loads a Parquet file stream, returning the result as a :class:`DataFrame`. + def parquet(self, path, mergeSchema=None, recursiveFileLookup=None): + """ + Loads a Parquet file stream, returning the result as a :class:`DataFrame`. + + .. note:: Evolving. + :param mergeSchema: sets whether we should merge schemas collected from all + Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. + The default value is specified in ``spark.sql.parquet.mergeSchema``. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. - You can set the following Parquet-specific option(s) for reading Parquet files: - * ``mergeSchema``: sets whether we should merge schemas collected from all \ - Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. \ - The default value is specified in ``spark.sql.parquet.mergeSchema``. - - .. note:: Evolving. - >>> parquet_sdf = spark.readStream.schema(sdf_schema).parquet(tempfile.mkdtemp()) >>> parquet_sdf.isStreaming True >>> parquet_sdf.schema == sdf_schema True """ - self._set_opts(recursiveFileLookup=recursiveFileLookup) + self._set_opts(mergeSchema=mergeSchema, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.parquet(path)) else: From c8922d9145a9bc60c0f423a6c1b7d4f0bfa2e585 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 4 Dec 2019 11:44:24 +0900 Subject: [PATCH 0787/1387] [SPARK-30113][SQL][PYTHON] Expose mergeSchema option in PySpark's ORC APIs ### What changes were proposed in this pull request? This PR is a follow-up to #24043 and cousin of #26730. It exposes the `mergeSchema` option directly in the ORC APIs. ### Why are the changes needed? So the Python API matches the Scala API. ### Does this PR introduce any user-facing change? Yes, it adds a new option directly in the ORC reader method signatures. ### How was this patch tested? I tested this manually as follows: ``` >>> spark.range(3).write.orc('test-orc') >>> spark.range(3).withColumnRenamed('id', 'name').write.orc('test-orc/nested') >>> spark.read.orc('test-orc', recursiveFileLookup=True, mergeSchema=True) DataFrame[id: bigint, name: bigint] >>> spark.read.orc('test-orc', recursiveFileLookup=True, mergeSchema=False) DataFrame[id: bigint] >>> spark.conf.set('spark.sql.orc.mergeSchema', True) >>> spark.read.orc('test-orc', recursiveFileLookup=True) DataFrame[id: bigint, name: bigint] >>> spark.read.orc('test-orc', recursiveFileLookup=True, mergeSchema=False) DataFrame[id: bigint] ``` Closes #26755 from nchammas/SPARK-30113-ORC-mergeSchema. Authored-by: Nicholas Chammas Signed-off-by: HyukjinKwon --- python/pyspark/sql/readwriter.py | 9 ++++++--- python/pyspark/sql/streaming.py | 9 ++++++--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 153776d8b4171..198313b10ec14 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -520,17 +520,20 @@ def func(iterator): raise TypeError("path can be only string, list or RDD") @since(1.5) - def orc(self, path, recursiveFileLookup=None): + def orc(self, path, mergeSchema=None, recursiveFileLookup=None): """Loads ORC files, returning the result as a :class:`DataFrame`. + :param mergeSchema: sets whether we should merge schemas collected from all + ORC part-files. This will override ``spark.sql.orc.mergeSchema``. + The default value is specified in ``spark.sql.orc.mergeSchema``. :param recursiveFileLookup: recursively scan a directory for files. Using this option - disables `partition discovery`_. + disables `partition discovery`_. >>> df = spark.read.orc('python/test_support/sql/orc_partitioned') >>> df.dtypes [('a', 'bigint'), ('b', 'int'), ('c', 'int')] """ - self._set_opts(recursiveFileLookup=recursiveFileLookup) + self._set_opts(mergeSchema=mergeSchema, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): path = [path] return self._df(self._jreader.orc(_to_seq(self._spark._sc, path))) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 6359c31ba5655..2937bc0bda70a 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -514,13 +514,16 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, raise TypeError("path can be only a single string") @since(2.3) - def orc(self, path, recursiveFileLookup=None): + def orc(self, path, mergeSchema=None, recursiveFileLookup=None): """Loads a ORC file stream, returning the result as a :class:`DataFrame`. .. note:: Evolving. + :param mergeSchema: sets whether we should merge schemas collected from all + ORC part-files. This will override ``spark.sql.orc.mergeSchema``. + The default value is specified in ``spark.sql.orc.mergeSchema``. :param recursiveFileLookup: recursively scan a directory for files. Using this option - disables `partition discovery`_. + disables `partition discovery`_. >>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp()) >>> orc_sdf.isStreaming @@ -528,7 +531,7 @@ def orc(self, path, recursiveFileLookup=None): >>> orc_sdf.schema == sdf_schema True """ - self._set_opts(recursiveFileLookup=recursiveFileLookup) + self._set_opts(mergeSchema=mergeSchema, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.orc(path)) else: From 55132ae9c996bed6a8231680a6da4de924bee100 Mon Sep 17 00:00:00 2001 From: Aman Omer Date: Wed, 4 Dec 2019 13:51:40 +0800 Subject: [PATCH 0788/1387] [SPARK-30099][SQL] Improve Analyzed Logical Plan ### What changes were proposed in this pull request? Avoid duplicate error message in Analyzed Logical plan. ### Why are the changes needed? Currently, when any query throws `AnalysisException`, same error message will be repeated because of following code segment. https://github.com/apache/spark/blob/04a5b8f5f80ee746bdc16267e44a993a9941d335/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala#L157-L166 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually. Result of `explain extended select * from wrong;` BEFORE > == Parsed Logical Plan == > 'Project [*] > +- 'UnresolvedRelation [wrong] > > == Analyzed Logical Plan == > org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31; > 'Project [*] > +- 'UnresolvedRelation [wrong] > > org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31; > 'Project [*] > +- 'UnresolvedRelation [wrong] > > == Optimized Logical Plan == > org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31; > 'Project [*] > +- 'UnresolvedRelation [wrong] > > == Physical Plan == > org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31; > 'Project [*] > +- 'UnresolvedRelation [wrong] > AFTER > == Parsed Logical Plan == > 'Project [*] > +- 'UnresolvedRelation [wrong] > > == Analyzed Logical Plan == > org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31; > 'Project [*] > +- 'UnresolvedRelation [wrong] > > == Optimized Logical Plan == > org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31; > 'Project [*] > +- 'UnresolvedRelation [wrong] > > == Physical Plan == > org.apache.spark.sql.AnalysisException: Table or view not found: wrong; line 1 pos 31; > 'Project [*] > +- 'UnresolvedRelation [wrong] > Closes #26734 from amanomer/cor_APlan. Authored-by: Aman Omer Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/QueryExecution.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 28bbe4fb4993d..2935e1cbaf15d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -137,14 +137,15 @@ class QueryExecution( append("== Parsed Logical Plan ==\n") QueryPlan.append(logical, append, verbose, addSuffix, maxFields) append("\n== Analyzed Logical Plan ==\n") - val analyzedOutput = try { - truncatedString( - analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields) + try { + append( + truncatedString( + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields) + ) + append("\n") } catch { - case e: AnalysisException => e.toString + case _: AnalysisException => } - append(analyzedOutput) - append("\n") QueryPlan.append(analyzed, append, verbose, addSuffix, maxFields) append("\n== Optimized Logical Plan ==\n") QueryPlan.append(optimizedPlan, append, verbose, addSuffix, maxFields) From 710ddab39e20f49e917311c3e27d142b5a2bcc71 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Wed, 4 Dec 2019 16:39:57 +0800 Subject: [PATCH 0789/1387] [SPARK-29914][ML] ML models attach metadata in `transform`/`transformSchema` ### What changes were proposed in this pull request? 1, `predictionCol` in `ml.classification` & `ml.clustering` add `NominalAttribute` 2, `rawPredictionCol` in `ml.classification` add `AttributeGroup` containing vectorsize=`numClasses` 3, `probabilityCol` in `ml.classification` & `ml.clustering` add `AttributeGroup` containing vectorsize=`numClasses`/`k` 4, `leafCol` in GBT/RF add `AttributeGroup` containing vectorsize=`numTrees` 5, `leafCol` in DecisionTree add `NominalAttribute` 6, `outputCol` in models in `ml.feature` add `AttributeGroup` containing vectorsize 7, `outputCol` in `UnaryTransformer`s in `ml.feature` add `AttributeGroup` containing vectorsize ### Why are the changes needed? Appened metadata can be used in downstream ops, like `Classifier.getNumClasses` There are many impls (like `Binarizer`/`Bucketizer`/`VectorAssembler`/`OneHotEncoder`/`FeatureHasher`/`HashingTF`/`VectorSlicer`/...) in `.ml` that append appropriate metadata in `transform`/`transformSchema` method. However there are also many impls return no metadata in transformation, even some metadata like `vector.size`/`numAttrs`/`attrs` can be ealily inferred. ### Does this PR introduce any user-facing change? Yes, add some metadatas in transformed dataset. ### How was this patch tested? existing testsuites and added testsuites Closes #26547 from zhengruifeng/add_output_vecSize. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../scala/org/apache/spark/ml/Predictor.scala | 17 ++-- .../org/apache/spark/ml/Transformer.scala | 5 +- .../spark/ml/classification/Classifier.scala | 32 +++++-- .../DecisionTreeClassifier.scala | 15 +++- .../ml/classification/GBTClassifier.scala | 15 +++- .../spark/ml/classification/OneVsRest.scala | 15 +++- .../ProbabilisticClassifier.scala | 30 +++++-- .../RandomForestClassifier.scala | 15 +++- .../spark/ml/clustering/BisectingKMeans.scala | 12 ++- .../spark/ml/clustering/GaussianMixture.scala | 22 +++-- .../apache/spark/ml/clustering/KMeans.scala | 12 ++- .../org/apache/spark/ml/clustering/LDA.scala | 12 ++- .../apache/spark/ml/feature/Binarizer.scala | 9 +- .../spark/ml/feature/ChiSqSelector.scala | 3 +- .../spark/ml/feature/CountVectorizer.scala | 15 ++-- .../org/apache/spark/ml/feature/DCT.scala | 15 +++- .../spark/ml/feature/ElementwiseProduct.scala | 13 ++- .../org/apache/spark/ml/feature/IDF.scala | 14 ++- .../spark/ml/feature/MaxAbsScaler.scala | 12 ++- .../spark/ml/feature/MinMaxScaler.scala | 12 ++- .../apache/spark/ml/feature/Normalizer.scala | 16 +++- .../org/apache/spark/ml/feature/PCA.scala | 18 ++-- .../spark/ml/feature/RobustScaler.scala | 12 ++- .../spark/ml/feature/StandardScaler.scala | 12 ++- .../spark/ml/feature/VectorSlicer.scala | 3 +- .../apache/spark/ml/feature/Word2Vec.scala | 15 +++- .../ml/regression/AFTSurvivalRegression.scala | 14 ++- .../ml/regression/DecisionTreeRegressor.scala | 19 +++- .../spark/ml/regression/GBTRegressor.scala | 14 ++- .../GeneralizedLinearRegression.scala | 17 ++-- .../ml/regression/IsotonicRegression.scala | 11 ++- .../ml/regression/RandomForestRegressor.scala | 14 ++- .../org/apache/spark/ml/tree/treeModels.scala | 17 ++++ .../apache/spark/ml/util/SchemaUtils.scala | 86 +++++++++++++++++++ .../DecisionTreeClassifierSuite.scala | 7 ++ .../classification/GBTClassifierSuite.scala | 7 ++ .../ml/classification/LinearSVCSuite.scala | 7 +- .../LogisticRegressionSuite.scala | 8 +- .../ml/classification/OneVsRestSuite.scala | 2 + .../RandomForestClassifierSuite.scala | 7 ++ .../ml/clustering/BisectingKMeansSuite.scala | 2 + .../ml/clustering/GaussianMixtureSuite.scala | 5 ++ .../spark/ml/clustering/KMeansSuite.scala | 3 + .../apache/spark/ml/feature/DCTSuite.scala | 20 +++++ .../apache/spark/ml/feature/IDFSuite.scala | 3 + .../spark/ml/feature/MaxAbsScalerSuite.scala | 3 + .../spark/ml/feature/MinMaxScalerSuite.scala | 3 + .../spark/ml/feature/NormalizerSuite.scala | 18 ++++ .../apache/spark/ml/feature/PCASuite.scala | 2 + .../spark/ml/feature/Word2VecSuite.scala | 3 + .../org/apache/spark/ml/util/MLTest.scala | 35 +++++++- 51 files changed, 593 insertions(+), 105 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index 9eac8ed22a3f6..98dd692cbe55d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -88,8 +88,9 @@ private[ml] trait PredictorParams extends Params * and put it in an RDD with strong types. * Validate the output instances with the given function. */ - protected def extractInstances(dataset: Dataset[_], - validateInstance: Instance => Unit): RDD[Instance] = { + protected def extractInstances( + dataset: Dataset[_], + validateInstance: Instance => Unit): RDD[Instance] = { extractInstances(dataset).map { instance => validateInstance(instance) instance @@ -222,7 +223,11 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, protected def featuresDataType: DataType = new VectorUDT override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema, fitting = false, featuresDataType) + var outputSchema = validateAndTransformSchema(schema, fitting = false, featuresDataType) + if ($(predictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumeric(outputSchema, $(predictionCol)) + } + outputSchema } /** @@ -244,10 +249,12 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, } protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Any) => + val outputSchema = transformSchema(dataset.schema, logging = true) + val predictUDF = udf { features: Any => predict(features.asInstanceOf[FeaturesType]) } - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol))), + outputSchema($(predictionCol)).metadata) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index a3a2b55adc25d..7874fc29db6c8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -117,9 +117,10 @@ abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] } override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val transformUDF = udf(this.createTransformFunc, outputDataType) - dataset.withColumn($(outputCol), transformUDF(dataset($(inputCol)))) + dataset.withColumn($(outputCol), transformUDF(dataset($(inputCol))), + outputSchema($(outputCol)).metadata) } override def copy(extra: ParamMap): T = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index 3bff236677e6b..be552e6be0b50 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -48,8 +48,9 @@ private[spark] trait ClassifierParams * and put it in an RDD with strong types. * Validates the label on the classifier is a valid integer in the range [0, numClasses). */ - protected def extractInstances(dataset: Dataset[_], - numClasses: Int): RDD[Instance] = { + protected def extractInstances( + dataset: Dataset[_], + numClasses: Int): RDD[Instance] = { val validateInstance = (instance: Instance) => { val label = instance.label require(label.toLong == label && label >= 0 && label < numClasses, s"Classifier was given" + @@ -183,6 +184,19 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur /** Number of classes (values which the label can take). */ def numClasses: Int + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(predictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumValues(schema, + $(predictionCol), numClasses) + } + if ($(rawPredictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(rawPredictionCol), numClasses) + } + outputSchema + } + /** * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by * parameters: @@ -193,29 +207,31 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * @return transformed dataset */ override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) // Output selected columns only. // This is a bit complicated since it tries to avoid repeated computation. var outputData = dataset var numColsOutput = 0 if (getRawPredictionCol != "") { - val predictRawUDF = udf { (features: Any) => + val predictRawUDF = udf { features: Any => predictRaw(features.asInstanceOf[FeaturesType]) } - outputData = outputData.withColumn(getRawPredictionCol, predictRawUDF(col(getFeaturesCol))) + outputData = outputData.withColumn(getRawPredictionCol, predictRawUDF(col(getFeaturesCol)), + outputSchema($(rawPredictionCol)).metadata) numColsOutput += 1 } if (getPredictionCol != "") { - val predUDF = if (getRawPredictionCol != "") { + val predCol = if (getRawPredictionCol != "") { udf(raw2prediction _).apply(col(getRawPredictionCol)) } else { - val predictUDF = udf { (features: Any) => + val predictUDF = udf { features: Any => predict(features.asInstanceOf[FeaturesType]) } predictUDF(col(getFeaturesCol)) } - outputData = outputData.withColumn(getPredictionCol, predUDF) + outputData = outputData.withColumn(getPredictionCol, predCol, + outputSchema($(predictionCol)).metadata) numColsOutput += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index e02109375373e..d10f684f0dcf1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -36,6 +36,7 @@ import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeMo import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} +import org.apache.spark.sql.types.StructType /** * Decision tree learning algorithm (http://en.wikipedia.org/wiki/Decision_tree_learning) @@ -202,13 +203,23 @@ class DecisionTreeClassificationModel private[ml] ( rootNode.predictImpl(features).prediction } + @Since("3.0.0") + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(leafCol).nonEmpty) { + outputSchema = SchemaUtils.updateField(outputSchema, getLeafField($(leafCol))) + } + outputSchema + } + override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val outputData = super.transform(dataset) if ($(leafCol).nonEmpty) { val leafUDF = udf { features: Vector => predictLeaf(features) } - outputData.withColumn($(leafCol), leafUDF(col($(featuresCol)))) + outputData.withColumn($(leafCol), leafUDF(col($(featuresCol))), + outputSchema($(leafCol)).metadata) } else { outputData } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index e1f5338f34899..6e54e0f15b85c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -36,6 +36,7 @@ import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.StructType /** * Gradient-Boosted Trees (GBTs) (http://en.wikipedia.org/wiki/Gradient_boosting) @@ -291,13 +292,23 @@ class GBTClassificationModel private[ml]( @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights + @Since("1.6.0") + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(leafCol).nonEmpty) { + outputSchema = SchemaUtils.updateField(outputSchema, getLeafField($(leafCol))) + } + outputSchema + } + override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val outputData = super.transform(dataset) if ($(leafCol).nonEmpty) { val leafUDF = udf { features: Vector => predictLeaf(features) } - outputData.withColumn($(leafCol), leafUDF(col($(featuresCol)))) + outputData.withColumn($(leafCol), leafUDF(col($(featuresCol))), + outputSchema($(leafCol)).metadata) } else { outputData } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 51a624795cdd4..bbf8e8fc90ad5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -161,13 +161,23 @@ final class OneVsRestModel private[ml] ( @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema, fitting = false, getClassifier.featuresDataType) + var outputSchema = validateAndTransformSchema(schema, fitting = false, + getClassifier.featuresDataType) + if ($(predictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumValues(outputSchema, + $(predictionCol), numClasses) + } + if ($(rawPredictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(rawPredictionCol), numClasses) + } + outputSchema } @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { // Check schema - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) if (getPredictionCol.isEmpty && getRawPredictionCol.isEmpty) { logWarning(s"$uid: OneVsRestModel.transform() does nothing" + @@ -230,6 +240,7 @@ final class OneVsRestModel private[ml] ( predictionColNames :+= getRawPredictionCol predictionColumns :+= rawPredictionUDF(col(accColName)) + .as($(rawPredictionCol), outputSchema($(rawPredictionCol)).metadata) } if (getPredictionCol.nonEmpty) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index 2171ac335e7b8..2e4d69330d132 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -90,6 +90,15 @@ abstract class ProbabilisticClassificationModel[ set(thresholds, value).asInstanceOf[M] } + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(probabilityCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(probabilityCol), numClasses) + } + outputSchema + } + /** * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by * parameters: @@ -101,7 +110,7 @@ abstract class ProbabilisticClassificationModel[ * @return transformed dataset */ override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + ".transform() called with non-matching numClasses and thresholds.length." + @@ -113,36 +122,39 @@ abstract class ProbabilisticClassificationModel[ var outputData = dataset var numColsOutput = 0 if ($(rawPredictionCol).nonEmpty) { - val predictRawUDF = udf { (features: Any) => + val predictRawUDF = udf { features: Any => predictRaw(features.asInstanceOf[FeaturesType]) } - outputData = outputData.withColumn(getRawPredictionCol, predictRawUDF(col(getFeaturesCol))) + outputData = outputData.withColumn(getRawPredictionCol, predictRawUDF(col(getFeaturesCol)), + outputSchema($(rawPredictionCol)).metadata) numColsOutput += 1 } if ($(probabilityCol).nonEmpty) { - val probUDF = if ($(rawPredictionCol).nonEmpty) { + val probCol = if ($(rawPredictionCol).nonEmpty) { udf(raw2probability _).apply(col($(rawPredictionCol))) } else { - val probabilityUDF = udf { (features: Any) => + val probabilityUDF = udf { features: Any => predictProbability(features.asInstanceOf[FeaturesType]) } probabilityUDF(col($(featuresCol))) } - outputData = outputData.withColumn($(probabilityCol), probUDF) + outputData = outputData.withColumn($(probabilityCol), probCol, + outputSchema($(probabilityCol)).metadata) numColsOutput += 1 } if ($(predictionCol).nonEmpty) { - val predUDF = if ($(rawPredictionCol).nonEmpty) { + val predCol = if ($(rawPredictionCol).nonEmpty) { udf(raw2prediction _).apply(col($(rawPredictionCol))) } else if ($(probabilityCol).nonEmpty) { udf(probability2prediction _).apply(col($(probabilityCol))) } else { - val predictUDF = udf { (features: Any) => + val predictUDF = udf { features: Any => predict(features.asInstanceOf[FeaturesType]) } predictUDF(col($(featuresCol))) } - outputData = outputData.withColumn($(predictionCol), predUDF) + outputData = outputData.withColumn($(predictionCol), predCol, + outputSchema($(predictionCol)).metadata) numColsOutput += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index bc28d783ed962..f88fc2a6a0914 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -36,6 +36,7 @@ import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestMo import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} +import org.apache.spark.sql.types.StructType /** * Random Forest learning algorithm for @@ -210,13 +211,23 @@ class RandomForestClassificationModel private[ml] ( @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights + @Since("1.4.0") + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(leafCol).nonEmpty) { + outputSchema = SchemaUtils.updateField(outputSchema, getLeafField($(leafCol))) + } + outputSchema + } + override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val outputData = super.transform(dataset) if ($(leafCol).nonEmpty) { val leafUDF = udf { features: Vector => predictLeaf(features) } - outputData.withColumn($(leafCol), leafUDF(col($(featuresCol)))) + outputData.withColumn($(leafCol), leafUDF(col($(featuresCol))), + outputSchema($(leafCol)).metadata) } else { outputData } diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 5f2316fa7ce18..79760d69489c6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -110,15 +110,21 @@ class BisectingKMeansModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val predictUDF = udf((vector: Vector) => predict(vector)) dataset.withColumn($(predictionCol), - predictUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol))) + predictUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)), + outputSchema($(predictionCol)).metadata) } @Since("2.0.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(predictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumValues(outputSchema, + $(predictionCol), parentModel.k) + } + outputSchema } @Since("3.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 916f326ab5615..9d00d6a8bcbe4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -112,7 +112,7 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val vectorCol = DatasetUtils.columnToVector(dataset, $(featuresCol)) var outputData = dataset @@ -120,17 +120,20 @@ class GaussianMixtureModel private[ml] ( if ($(probabilityCol).nonEmpty) { val probUDF = udf((vector: Vector) => predictProbability(vector)) - outputData = outputData.withColumn($(probabilityCol), probUDF(vectorCol)) + outputData = outputData.withColumn($(probabilityCol), probUDF(vectorCol), + outputSchema($(probabilityCol)).metadata) numColsOutput += 1 } if ($(predictionCol).nonEmpty) { if ($(probabilityCol).nonEmpty) { val predUDF = udf((vector: Vector) => vector.argmax) - outputData = outputData.withColumn($(predictionCol), predUDF(col($(probabilityCol)))) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(probabilityCol))), + outputSchema($(predictionCol)).metadata) } else { val predUDF = udf((vector: Vector) => predict(vector)) - outputData = outputData.withColumn($(predictionCol), predUDF(vectorCol)) + outputData = outputData.withColumn($(predictionCol), predUDF(vectorCol), + outputSchema($(predictionCol)).metadata) } numColsOutput += 1 } @@ -144,7 +147,16 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(predictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumValues(outputSchema, + $(predictionCol), weights.length) + } + if ($(probabilityCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(probabilityCol), weights.length) + } + outputSchema } @Since("3.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index caeded400f9aa..5cbba6c77f9fb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -127,17 +127,23 @@ class KMeansModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val predictUDF = udf((vector: Vector) => predict(vector)) dataset.withColumn($(predictionCol), - predictUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol))) + predictUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)), + outputSchema($(predictionCol)).metadata) } @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(predictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumValues(outputSchema, + $(predictionCol), parentModel.k) + } + outputSchema } @Since("3.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 9b0005b3747dc..e30be8c20dcc3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -459,12 +459,13 @@ abstract class LDAModel private[ml] ( */ @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val func = getTopicDistributionMethod val transformer = udf(func) dataset.withColumn($(topicDistributionCol), - transformer(DatasetUtils.columnToVector(dataset, getFeaturesCol))) + transformer(DatasetUtils.columnToVector(dataset, getFeaturesCol)), + outputSchema($(topicDistributionCol)).metadata) } /** @@ -504,7 +505,12 @@ abstract class LDAModel private[ml] ( @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(topicDistributionCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(topicDistributionCol), oldLocalModel.k) + } + outputSchema } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 07a4f91443bc5..381ab6eb7355b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer -import org.apache.spark.ml.attribute.BinaryAttribute +import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ @@ -193,7 +193,12 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) case DoubleType => BinaryAttribute.defaultAttr.withName(outputColName).toStructField() case _: VectorUDT => - StructField(outputColName, new VectorUDT) + val size = AttributeGroup.fromStructField(schema(inputColName)).size + if (size < 0) { + StructField(outputColName, new VectorUDT) + } else { + new AttributeGroup(outputColName, numAttributes = size).toStructField() + } case _ => throw new IllegalArgumentException(s"Data type $inputType is not supported.") } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 9103e4feac454..76f4f944f11d5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -289,8 +289,7 @@ final class ChiSqSelectorModel private[ml] ( override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) val newField = prepOutputField(schema) - val outputFields = schema.fields :+ newField - StructType(outputFields) + SchemaUtils.appendColumn(schema, newField) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index c58d44d492342..7ba6f640b1e49 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -300,7 +300,7 @@ class CountVectorizerModel( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) if (broadcastDict.isEmpty) { val dict = vocabulary.zipWithIndex.toMap broadcastDict = Some(dataset.sparkSession.sparkContext.broadcast(dict)) @@ -326,14 +326,19 @@ class CountVectorizerModel( Vectors.sparse(dictBr.value.size, effectiveCounts) } - val attrs = vocabulary.map(_ => new NumericAttribute).asInstanceOf[Array[Attribute]] - val metadata = new AttributeGroup($(outputCol), attrs).toMetadata() - dataset.withColumn($(outputCol), vectorizer(col($(inputCol))), metadata) + dataset.withColumn($(outputCol), vectorizer(col($(inputCol))), + outputSchema($(outputCol)).metadata) } @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(outputCol).nonEmpty) { + val attrs: Array[Attribute] = vocabulary.map(_ => new NumericAttribute) + val field = new AttributeGroup($(outputCol), attrs).toStructField() + outputSchema = SchemaUtils.updateField(outputSchema, field) + } + outputSchema } @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index e2167f01281da..d057e5a62e507 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -21,10 +21,11 @@ import org.jtransforms.dct._ import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.BooleanParam import org.apache.spark.ml.util._ -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types._ /** * A feature transformer that takes the 1D discrete cosine transform of a real vector. No zero @@ -75,6 +76,18 @@ class DCT @Since("1.5.0") (@Since("1.5.0") override val uid: String) override protected def outputDataType: DataType = new VectorUDT + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(inputCol).nonEmpty && $(outputCol).nonEmpty) { + val size = AttributeGroup.fromStructField(schema($(inputCol))).size + if (size >= 0) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), size) + } + } + outputSchema + } + @Since("3.0.0") override def toString: String = { s"DCT: uid=$uid, inverse=$inverse" diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 227c13d60fd8f..3b328f2fd8cee 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -21,10 +21,10 @@ import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.Param -import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} +import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature.{ElementwiseProduct => OldElementwiseProduct} import org.apache.spark.mllib.linalg.{Vectors => OldVectors} -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types._ /** * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a @@ -82,6 +82,15 @@ class ElementwiseProduct @Since("1.4.0") (@Since("1.4.0") override val uid: Stri override protected def outputDataType: DataType = new VectorUDT() + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(outputCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), $(scalingVec).size) + } + outputSchema + } + @Since("3.0.0") override def toString: String = { s"ElementwiseProduct: uid=$uid" + diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 5f4103abcf50f..e6f124ef7d666 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -131,7 +131,7 @@ class IDFModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val func = { vector: Vector => vector match { @@ -149,12 +149,18 @@ class IDFModel private[ml] ( } val transformer = udf(func) - dataset.withColumn($(outputCol), transformer(col($(inputCol)))) + dataset.withColumn($(outputCol), transformer(col($(inputCol))), + outputSchema($(outputCol)).metadata) } @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(outputCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), idf.size) + } + outputSchema } @Since("1.4.1") @@ -180,7 +186,7 @@ class IDFModel private[ml] ( @Since("3.0.0") override def toString: String = { - s"IDFModel: uid=$uid, numDocs=$numDocs" + s"IDFModel: uid=$uid, numDocs=$numDocs, numFeatures=${idf.size}" } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala index 6bab70e502ed7..2d48a5f9f4915 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -117,19 +117,25 @@ class MaxAbsScalerModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val scale = maxAbs.toArray.map { v => if (v == 0) 1.0 else 1 / v } val func = StandardScalerModel.getTransformFunc( Array.empty, scale, false, true) val transformer = udf(func) - dataset.withColumn($(outputCol), transformer(col($(inputCol)))) + dataset.withColumn($(outputCol), transformer(col($(inputCol))), + outputSchema($(outputCol)).metadata) } @Since("2.0.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(outputCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), maxAbs.size) + } + outputSchema } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index e381a0435e9eb..c84892c974b90 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -174,7 +174,7 @@ class MinMaxScalerModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val numFeatures = originalMax.size val scale = $(max) - $(min) @@ -210,12 +210,18 @@ class MinMaxScalerModel private[ml] ( Vectors.dense(values).compressed } - dataset.withColumn($(outputCol), transformer(col($(inputCol)))) + dataset.withColumn($(outputCol), transformer(col($(inputCol))), + outputSchema($(outputCol)).metadata) } @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(outputCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), originalMin.size) + } + outputSchema } @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index d129c2b2c2dc1..4c7583b8381dc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -19,12 +19,13 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.attribute.AttributeGroup import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamValidators} import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vectors => OldVectors} -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types._ /** * Normalize a vector to have unit norm using the given p-norm. @@ -66,6 +67,19 @@ class Normalizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) override protected def outputDataType: DataType = new VectorUDT() + @Since("1.4.0") + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(inputCol).nonEmpty && $(outputCol).nonEmpty) { + val size = AttributeGroup.fromStructField(schema($(inputCol))).size + if (size >= 0) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), size) + } + } + outputSchema + } + @Since("3.0.0") override def toString: String = { s"Normalizer: uid=$uid, p=${$(p)}" diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 27a3854d39b47..9eeb4c8ca2506 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -29,7 +29,7 @@ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, Vectors => OldVectors} import org.apache.spark.sql._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.util.VersionUtils.majorVersion /** @@ -52,10 +52,8 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) require(!schema.fieldNames.contains($(outputCol)), s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + SchemaUtils.updateAttributeGroupSize(schema, $(outputCol), $(k)) } - } /** @@ -145,16 +143,22 @@ class PCAModel private[ml] ( */ @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val transposed = pc.transpose val transformer = udf { vector: Vector => transposed.multiply(vector) } - dataset.withColumn($(outputCol), transformer(col($(inputCol)))) + dataset.withColumn($(outputCol), transformer(col($(inputCol))), + outputSchema($(outputCol)).metadata) } @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(outputCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), $(k)) + } + outputSchema } @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala index 1b9b8082931a5..f02ef0dfc70f8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RobustScaler.scala @@ -227,7 +227,7 @@ class RobustScalerModel private[ml] ( def setOutputCol(value: String): this.type = set(outputCol, value) override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val shift = if ($(withCentering)) median.toArray else Array.emptyDoubleArray val scale = if ($(withScaling)) { @@ -238,11 +238,17 @@ class RobustScalerModel private[ml] ( shift, scale, $(withCentering), $(withScaling)) val transformer = udf(func) - dataset.withColumn($(outputCol), transformer(col($(inputCol)))) + dataset.withColumn($(outputCol), transformer(col($(inputCol))), + outputSchema($(outputCol)).metadata) } override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(outputCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), median.size) + } + outputSchema } override def copy(extra: ParamMap): RobustScalerModel = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 8d4d4197e7db2..c6b1b29a6d9bc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -157,7 +157,7 @@ class StandardScalerModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val shift = if ($(withMean)) mean.toArray else Array.emptyDoubleArray val scale = if ($(withStd)) { std.toArray.map { v => if (v == 0) 0.0 else 1.0 / v } @@ -166,12 +166,18 @@ class StandardScalerModel private[ml] ( val func = getTransformFunc(shift, scale, $(withMean), $(withStd)) val transformer = udf(func) - dataset.withColumn($(outputCol), transformer(col($(inputCol)))) + dataset.withColumn($(outputCol), transformer(col($(inputCol))), + outputSchema($(outputCol)).metadata) } @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(outputCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), mean.size) + } + outputSchema } @Since("1.4.1") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index b84b8af4e8a94..45bb4b8e6e65d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -153,8 +153,7 @@ final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: Stri } val numFeaturesSelected = $(indices).length + $(names).length val outputAttr = new AttributeGroup($(outputCol), numFeaturesSelected) - val outputFields = schema.fields :+ outputAttr.toStructField() - StructType(outputFields) + SchemaUtils.appendColumn(schema, outputAttr.toStructField) } @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 81dde0315c190..bbfcbfbe038ef 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -288,15 +288,16 @@ class Word2VecModel private[ml] ( */ @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val vectors = wordVectors.getVectors .mapValues(vv => Vectors.dense(vv.map(_.toDouble))) .map(identity) // mapValues doesn't return a serializable map (SI-7005) val bVectors = dataset.sparkSession.sparkContext.broadcast(vectors) val d = $(vectorSize) + val emptyVec = Vectors.sparse(d, Array.emptyIntArray, Array.emptyDoubleArray) val word2Vec = udf { sentence: Seq[String] => if (sentence.isEmpty) { - Vectors.sparse(d, Array.empty[Int], Array.empty[Double]) + emptyVec } else { val sum = Vectors.zeros(d) sentence.foreach { word => @@ -308,12 +309,18 @@ class Word2VecModel private[ml] ( sum } } - dataset.withColumn($(outputCol), word2Vec(col($(inputCol)))) + dataset.withColumn($(outputCol), word2Vec(col($(inputCol))), + outputSchema($(outputCol)).metadata) } @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema) + var outputSchema = validateAndTransformSchema(schema) + if ($(outputCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(outputCol), $(vectorSize)) + } + outputSchema } @Since("1.4.1") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index faf77252cb738..7079ac89dcc93 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -349,7 +349,7 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) var predictionColNames = Seq.empty[String] var predictionColumns = Seq.empty[Column] @@ -358,12 +358,14 @@ class AFTSurvivalRegressionModel private[ml] ( val predictUDF = udf { features: Vector => predict(features) } predictionColNames :+= $(predictionCol) predictionColumns :+= predictUDF(col($(featuresCol))) + .as($(predictionCol), outputSchema($(predictionCol)).metadata) } if (hasQuantilesCol) { val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} predictionColNames :+= $(quantilesCol) predictionColumns :+= predictQuantilesUDF(col($(featuresCol))) + .as($(quantilesCol), outputSchema($(quantilesCol)).metadata) } if (predictionColNames.nonEmpty) { @@ -377,7 +379,15 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema, fitting = false) + var outputSchema = validateAndTransformSchema(schema, fitting = false) + if ($(predictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumeric(outputSchema, $(predictionCol)) + } + if (isDefined(quantilesCol) && $(quantilesCol).nonEmpty) { + outputSchema = SchemaUtils.updateAttributeGroupSize(outputSchema, + $(quantilesCol), $(quantileProbabilities).length) + } + outputSchema } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 4a97997a1deb8..447e6f90a44e7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -36,7 +36,7 @@ import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeMo import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Column, DataFrame, Dataset} import org.apache.spark.sql.functions._ - +import org.apache.spark.sql.types.StructType /** * Decision tree @@ -202,9 +202,21 @@ class DecisionTreeRegressionModel private[ml] ( rootNode.predictImpl(features).impurityStats.calculate() } + @Since("1.4.0") + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if (isDefined(varianceCol) && $(varianceCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumeric(outputSchema, $(varianceCol)) + } + if ($(leafCol).nonEmpty) { + outputSchema = SchemaUtils.updateField(outputSchema, getLeafField($(leafCol))) + } + outputSchema + } + @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) var predictionColNames = Seq.empty[String] var predictionColumns = Seq.empty[Column] @@ -213,18 +225,21 @@ class DecisionTreeRegressionModel private[ml] ( val predictUDF = udf { features: Vector => predict(features) } predictionColNames :+= $(predictionCol) predictionColumns :+= predictUDF(col($(featuresCol))) + .as($(predictionCol), outputSchema($(predictionCol)).metadata) } if (isDefined(varianceCol) && $(varianceCol).nonEmpty) { val predictVarianceUDF = udf { features: Vector => predictVariance(features) } predictionColNames :+= $(varianceCol) predictionColumns :+= predictVarianceUDF(col($(featuresCol))) + .as($(varianceCol), outputSchema($(varianceCol)).metadata) } if ($(leafCol).nonEmpty) { val leafUDF = udf { features: Vector => predictLeaf(features) } predictionColNames :+= $(leafCol) predictionColumns :+= leafUDF(col($(featuresCol))) + .as($(leafCol), outputSchema($(leafCol)).metadata) } if (predictionColNames.nonEmpty) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 700f7a2075a91..eb0f2362af570 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -35,6 +35,7 @@ import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.sql.{Column, DataFrame, Dataset} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.StructType /** * Gradient-Boosted Trees (GBTs) @@ -255,8 +256,17 @@ class GBTRegressionModel private[ml]( @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights + @Since("1.4.0") + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(leafCol).nonEmpty) { + outputSchema = SchemaUtils.updateField(outputSchema, getLeafField($(leafCol))) + } + outputSchema + } + override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) var predictionColNames = Seq.empty[String] var predictionColumns = Seq.empty[Column] @@ -267,12 +277,14 @@ class GBTRegressionModel private[ml]( val predictUDF = udf { features: Vector => bcastModel.value.predict(features) } predictionColNames :+= $(predictionCol) predictionColumns :+= predictUDF(col($(featuresCol))) + .as($(featuresCol), outputSchema($(featuresCol)).metadata) } if ($(leafCol).nonEmpty) { val leafUDF = udf { features: Vector => bcastModel.value.predictLeaf(features) } predictionColNames :+= $(leafCol) predictionColumns :+= leafUDF(col($(featuresCol))) + .as($(leafCol), outputSchema($(leafCol)).metadata) } if (predictionColNames.nonEmpty) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 53b29102f01be..f24eeff682110 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.PredictorParams -import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.attribute._ import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.optim._ @@ -213,7 +213,9 @@ private[regression] trait GeneralizedLinearRegressionBase extends PredictorParam } if (hasLinkPredictionCol) { - SchemaUtils.appendColumn(newSchema, $(linkPredictionCol), DoubleType) + val attr = NumericAttribute.defaultAttr + .withName($(linkPredictionCol)) + SchemaUtils.appendColumn(newSchema, attr.toStructField()) } else { newSchema } @@ -1043,6 +1045,8 @@ class GeneralizedLinearRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { + val outputSchema = transformSchema(dataset.schema, logging = true) + val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) var outputData = dataset var numColsOutput = 0 @@ -1050,17 +1054,20 @@ class GeneralizedLinearRegressionModel private[ml] ( if (hasLinkPredictionCol) { val predLinkUDF = udf((features: Vector, offset: Double) => predictLink(features, offset)) outputData = outputData - .withColumn($(linkPredictionCol), predLinkUDF(col($(featuresCol)), offset)) + .withColumn($(linkPredictionCol), predLinkUDF(col($(featuresCol)), offset), + outputSchema($(linkPredictionCol)).metadata) numColsOutput += 1 } if ($(predictionCol).nonEmpty) { if (hasLinkPredictionCol) { val predUDF = udf((eta: Double) => familyAndLink.fitted(eta)) - outputData = outputData.withColumn($(predictionCol), predUDF(col($(linkPredictionCol)))) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(linkPredictionCol))), + outputSchema($(predictionCol)).metadata) } else { val predUDF = udf((features: Vector, offset: Double) => predict(features, offset)) - outputData = outputData.withColumn($(predictionCol), predUDF(col($(featuresCol)), offset)) + outputData = outputData.withColumn($(predictionCol), predUDF(col($(featuresCol)), offset), + outputSchema($(predictionCol)).metadata) } numColsOutput += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index 47f9e4bfb8333..d12e5daabebbf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -240,7 +240,7 @@ class IsotonicRegressionModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) val predict = dataset.schema($(featuresCol)).dataType match { case DoubleType => udf { feature: Double => oldModel.predict(feature) } @@ -248,12 +248,17 @@ class IsotonicRegressionModel private[ml] ( val idx = $(featureIndex) udf { features: Vector => oldModel.predict(features(idx)) } } - dataset.withColumn($(predictionCol), predict(col($(featuresCol)))) + dataset.withColumn($(predictionCol), predict(col($(featuresCol))), + outputSchema($(predictionCol)).metadata) } @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - validateAndTransformSchema(schema, fitting = false) + var outputSchema = validateAndTransformSchema(schema, fitting = false) + if ($(predictionCol).nonEmpty) { + outputSchema = SchemaUtils.updateNumeric(outputSchema, $(predictionCol)) + } + outputSchema } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index c3afab57a49c7..fa4dbbb47079f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -33,6 +33,7 @@ import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.sql.{Column, DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} +import org.apache.spark.sql.types.StructType /** * Random Forest @@ -192,8 +193,17 @@ class RandomForestRegressionModel private[ml] ( @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights + @Since("1.4.0") + override def transformSchema(schema: StructType): StructType = { + var outputSchema = super.transformSchema(schema) + if ($(leafCol).nonEmpty) { + outputSchema = SchemaUtils.updateField(outputSchema, getLeafField($(leafCol))) + } + outputSchema + } + override def transform(dataset: Dataset[_]): DataFrame = { - transformSchema(dataset.schema, logging = true) + val outputSchema = transformSchema(dataset.schema, logging = true) var predictionColNames = Seq.empty[String] var predictionColumns = Seq.empty[Column] @@ -204,12 +214,14 @@ class RandomForestRegressionModel private[ml] ( val predictUDF = udf { features: Vector => bcastModel.value.predict(features) } predictionColNames :+= $(predictionCol) predictionColumns :+= predictUDF(col($(featuresCol))) + .as($(predictionCol), outputSchema($(predictionCol)).metadata) } if ($(leafCol).nonEmpty) { val leafUDF = udf { features: Vector => bcastModel.value.predictLeaf(features) } predictionColNames :+= $(leafCol) predictionColumns :+= leafUDF(col($(featuresCol))) + .as($(leafCol), outputSchema($(leafCol)).metadata) } if (predictionColNames.nonEmpty) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index 10895d4fd11d9..3009b733d4fb7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path import org.json4s._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.{Param, Params} import org.apache.spark.ml.tree.DecisionTreeModelReadWrite.NodeData @@ -89,6 +90,18 @@ private[spark] trait DecisionTreeModel { } } + private[ml] lazy val numLeave: Int = + leafIterator(rootNode).size + + private[ml] lazy val leafAttr = { + NominalAttribute.defaultAttr + .withNumValues(numLeave) + } + + private[ml] def getLeafField(leafCol: String) = { + leafAttr.withName(leafCol).toStructField() + } + @transient private lazy val leafIndices: Map[LeafNode, Int] = { leafIterator(rootNode).zipWithIndex.toMap } @@ -146,6 +159,10 @@ private[ml] trait TreeEnsembleModel[M <: DecisionTreeModel] { val indices = trees.map(_.predictLeaf(features)) Vectors.dense(indices) } + + private[ml] def getLeafField(leafCol: String) = { + new AttributeGroup(leafCol, attrs = trees.map(_.leafAttr)).toStructField() + } } private[ml] object TreeEnsembleModel { diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala index c3894ebdd1785..752069daf8910 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.ml.util +import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types._ @@ -106,6 +107,91 @@ private[spark] object SchemaUtils { StructType(schema.fields :+ col) } + /** + * Update the size of a ML Vector column. If this column do not exist, append it. + * @param schema input schema + * @param colName column name + * @param size number of features + * @return new schema + */ + def updateAttributeGroupSize( + schema: StructType, + colName: String, + size: Int): StructType = { + require(size > 0) + val attrGroup = new AttributeGroup(colName, size) + val field = attrGroup.toStructField + updateField(schema, field, true) + } + + /** + * Update the number of values of an existing column. If this column do not exist, append it. + * @param schema input schema + * @param colName column name + * @param numValues number of values. + * @return new schema + */ + def updateNumValues( + schema: StructType, + colName: String, + numValues: Int): StructType = { + val attr = NominalAttribute.defaultAttr + .withName(colName) + .withNumValues(numValues) + val field = attr.toStructField + updateField(schema, field, true) + } + + /** + * Update the numeric meta of an existing column. If this column do not exist, append it. + * @param schema input schema + * @param colName column name + * @return new schema + */ + def updateNumeric( + schema: StructType, + colName: String): StructType = { + val attr = NumericAttribute.defaultAttr + .withName(colName) + val field = attr.toStructField + updateField(schema, field, true) + } + + /** + * Update the metadata of an existing column. If this column do not exist, append it. + * @param schema input schema + * @param field struct field + * @param overwriteMetadata whether to overwrite the metadata. If true, the metadata in the + * schema will be overwritten. If false, the metadata in `field` + * and `schema` will be merged to generate output metadata. + * @return new schema + */ + def updateField( + schema: StructType, + field: StructField, + overwriteMetadata: Boolean = true): StructType = { + if (schema.fieldNames.contains(field.name)) { + val newFields = schema.fields.map { f => + if (f.name == field.name) { + if (overwriteMetadata) { + field + } else { + val newMeta = new MetadataBuilder() + .withMetadata(field.metadata) + .withMetadata(f.metadata) + .build() + StructField(field.name, field.dataType, field.nullable, newMeta) + } + } else { + f + } + } + StructType(newFields) + } else { + appendColumn(schema, field) + } + } + /** * Check whether the given column in the schema is one of the supporting vector type: Vector, * Array[Float]. Array[Double] diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index 3ebf8a83a892c..fd5af5b954150 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -249,6 +249,13 @@ class DecisionTreeClassifierSuite extends MLTest with DefaultReadWriteTest { val newData: DataFrame = TreeTests.setMetadata(rdd, categoricalFeatures, numClasses) val newTree = dt.fit(newData) + newTree.setLeafCol("predictedLeafId") + + val transformed = newTree.transform(newData) + checkNominalOnDF(transformed, "prediction", newTree.numClasses) + checkNominalOnDF(transformed, "predictedLeafId", newTree.numLeave) + checkVectorSizeOnDF(transformed, "rawPrediction", newTree.numClasses) + checkVectorSizeOnDF(transformed, "probability", newTree.numClasses) MLTestingUtils.checkCopyAndUids(dt, newTree) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index fdca71f8911c6..ffd4b5e6d3055 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -473,6 +473,13 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { .setCheckpointInterval(5) .setSeed(123) val model = gbt.fit(df) + model.setLeafCol("predictedLeafId") + + val transformed = model.transform(df) + checkNominalOnDF(transformed, "prediction", model.numClasses) + checkVectorSizeOnDF(transformed, "predictedLeafId", model.trees.length) + checkVectorSizeOnDF(transformed, "rawPrediction", model.numClasses) + checkVectorSizeOnDF(transformed, "probability", model.numClasses) model.trees.foreach (i => { assert(i.getMaxDepth === model.getMaxDepth) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala index dc38f17d296f2..b23b4f4ac0d26 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -112,8 +112,13 @@ class LinearSVCSuite extends MLTest with DefaultReadWriteTest { assert(lsvc.getFeaturesCol === "features") assert(lsvc.getPredictionCol === "prediction") assert(lsvc.getRawPredictionCol === "rawPrediction") + val model = lsvc.setMaxIter(5).fit(smallBinaryDataset) - model.transform(smallBinaryDataset) + val transformed = model.transform(smallBinaryDataset) + checkNominalOnDF(transformed, "prediction", model.numClasses) + checkVectorSizeOnDF(transformed, "rawPrediction", model.numClasses) + + transformed .select("label", "prediction", "rawPrediction") .collect() assert(model.getThreshold === 0.0) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 60c9cce6a4879..38bdfded9693e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -155,8 +155,14 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(!lr.isDefined(lr.weightCol)) assert(lr.getFitIntercept) assert(lr.getStandardization) + val model = lr.fit(smallBinaryDataset) - model.transform(smallBinaryDataset) + val transformed = model.transform(smallBinaryDataset) + checkNominalOnDF(transformed, "prediction", model.numClasses) + checkVectorSizeOnDF(transformed, "rawPrediction", model.numClasses) + checkVectorSizeOnDF(transformed, "probability", model.numClasses) + + transformed .select("label", "probability", "prediction", "rawPrediction") .collect() assert(model.getThreshold === 0.5) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index adffd83ab1bd1..024a3870d8bca 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -81,6 +81,8 @@ class OneVsRestSuite extends MLTest with DefaultReadWriteTest { assert(ovaModel.numClasses === numClasses) val transformedDataset = ovaModel.transform(dataset) + checkNominalOnDF(transformedDataset, "prediction", ovaModel.numClasses) + checkVectorSizeOnDF(transformedDataset, "rawPrediction", ovaModel.numClasses) // check for label metadata in prediction col val predictionColSchema = transformedDataset.schema(ovaModel.getPredictionCol) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index 5958bfcf5ea6d..379d3bd128a5d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -242,6 +242,13 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { val df: DataFrame = TreeTests.setMetadata(rdd, categoricalFeatures, numClasses) val model = rf.fit(df) + model.setLeafCol("predictedLeafId") + + val transformed = model.transform(df) + checkNominalOnDF(transformed, "prediction", model.numClasses) + checkVectorSizeOnDF(transformed, "predictedLeafId", model.trees.length) + checkVectorSizeOnDF(transformed, "rawPrediction", model.numClasses) + checkVectorSizeOnDF(transformed, "probability", model.numClasses) model.trees.foreach (i => { assert(i.getMaxDepth === model.getMaxDepth) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index 5288595d2e239..7ac7b64adfdab 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -174,6 +174,8 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { .setSeed(1) .fit(df) val predictionDf = model.transform(df) + checkNominalOnDF(predictionDf, "prediction", model.getK) + assert(predictionDf.select("prediction").distinct().count() == 3) val predictionsMap = predictionDf.collect().map(row => row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala index 133536f763f4e..e570693c90e6e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala @@ -71,10 +71,15 @@ class GaussianMixtureSuite extends MLTest with DefaultReadWriteTest { assert(gm.getK === 2) assert(gm.getFeaturesCol === "features") assert(gm.getPredictionCol === "prediction") + assert(gm.getProbabilityCol === "probability") assert(gm.getMaxIter === 100) assert(gm.getTol === 0.01) val model = gm.setMaxIter(1).fit(dataset) + val transformed = model.transform(dataset) + checkNominalOnDF(transformed, "prediction", model.weights.length) + checkVectorSizeOnDF(transformed, "probability", model.weights.length) + MLTestingUtils.checkCopyAndUids(gm, model) assert(model.hasSummary) val copiedModel = model.copy(ParamMap.empty) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index e3c82fafca218..f6b1a8e9d6df3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -60,6 +60,9 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes assert(kmeans.getDistanceMeasure === DistanceMeasure.EUCLIDEAN) val model = kmeans.setMaxIter(1).fit(dataset) + val transformed = model.transform(dataset) + checkNominalOnDF(transformed, "prediction", model.clusterCenters.length) + MLTestingUtils.checkCopyAndUids(kmeans, model) assert(model.hasSummary) val copiedModel = model.copy(ParamMap.empty) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala index 079dabb3665be..19645b517d79c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.ml.feature import org.jtransforms.dct.DoubleDCT_1D +import org.scalatest.exceptions.TestFailedException import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} @@ -74,5 +75,24 @@ class DCTSuite extends MLTest with DefaultReadWriteTest { case Row(resultVec: Vector, wantedVec: Vector) => assert(Vectors.sqdist(resultVec, wantedVec) < 1e-6) } + + val vectorSize = dataset + .select("vec") + .map { case Row(vec: Vector) => vec.size } + .head() + + // Can not infer size of ouput vector, since no metadata is provided + intercept[TestFailedException] { + val transformed = transformer.transform(dataset) + checkVectorSizeOnDF(transformed, "resultVec", vectorSize) + } + + val dataset2 = new VectorSizeHint() + .setSize(vectorSize) + .setInputCol("vec") + .transform(dataset) + + val transformed2 = transformer.transform(dataset2) + checkVectorSizeOnDF(transformed2, "resultVec", vectorSize) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala index 73b2b82daaf43..b4e144ea5ba5e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala @@ -68,6 +68,9 @@ class IDFSuite extends MLTest with DefaultReadWriteTest { .setOutputCol("idfValue") val idfModel = idfEst.fit(df) + val transformed = idfModel.transform(df) + checkVectorSizeOnDF(transformed, "idfValue", idfModel.idf.size) + MLTestingUtils.checkCopyAndUids(idfEst, idfModel) testTransformer[(Vector, Vector)](df, idfModel, "idfValue", "expected") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala index 8dd0f0cb91e37..5de938fa40c4d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala @@ -44,6 +44,9 @@ class MaxAbsScalerSuite extends MLTest with DefaultReadWriteTest { .setOutputCol("scaled") val model = scaler.fit(df) + val transformed = model.transform(df) + checkVectorSizeOnDF(transformed, "scaled", model.maxAbs.size) + testTransformer[(Vector, Vector)](df, model, "expected", "scaled") { case Row(expectedVec: Vector, actualVec: Vector) => assert(expectedVec === actualVec, diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala index 2d965f2ca2c54..9b2b0c48f4f61 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala @@ -46,6 +46,9 @@ class MinMaxScalerSuite extends MLTest with DefaultReadWriteTest { .setMax(5) val model = scaler.fit(df) + val transformed = model.transform(df) + checkVectorSizeOnDF(transformed, "scaled", model.originalMin.size) + testTransformer[(Vector, Vector)](df, model, "expected", "scaled") { case Row(vector1: Vector, vector2: Vector) => assert(vector1 === vector2, "Transformed vector is different with expected.") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala index eff57f1223af4..d97df0050d74e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.feature +import org.scalatest.exceptions.TestFailedException + import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.ml.util.TestingUtils._ @@ -81,6 +83,22 @@ class NormalizerSuite extends MLTest with DefaultReadWriteTest { assertTypeOfVector(normalized, features) assertValues(normalized, expected) } + + val vectorSize = data.head.size + + // Can not infer size of output vector, since no metadata is provided + intercept[TestFailedException] { + val transformed = normalizer.transform(dataFrame) + checkVectorSizeOnDF(transformed, "normalized", vectorSize) + } + + val dataFrame2 = new VectorSizeHint() + .setSize(vectorSize) + .setInputCol("features") + .transform(dataFrame) + + val transformed2 = normalizer.transform(dataFrame2) + checkVectorSizeOnDF(transformed2, "normalized", vectorSize) } test("Normalization with setter") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala index 531b1d7c4d9f7..88c9867337e7c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala @@ -58,6 +58,8 @@ class PCASuite extends MLTest with DefaultReadWriteTest { .setK(3) val pcaModel = pca.fit(df) + val transformed = pcaModel.transform(df) + checkVectorSizeOnDF(transformed, "pca_features", pcaModel.getK) MLTestingUtils.checkCopyAndUids(pca, pcaModel) testTransformer[(Vector, Vector)](df, pcaModel, "pca_features", "expected") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index d28f1f4240ad0..11e1847ef235e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -61,6 +61,9 @@ class Word2VecSuite extends MLTest with DefaultReadWriteTest { .setSeed(42L) val model = w2v.fit(docDF) + val transformed = model.transform(docDF) + checkVectorSizeOnDF(transformed, "result", model.getVectorSize) + MLTestingUtils.checkCopyAndUids(w2v, model) // These expectations are just magic values, characterizing the current diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala index 978a3cbe54c1e..3e1e2ad6a7f55 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala @@ -24,10 +24,11 @@ import org.scalatest.Suite import org.apache.spark.{DebugFilesystem, SparkConf, SparkContext, TestUtils} import org.apache.spark.internal.config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK import org.apache.spark.ml.{Model, PredictionModel, Transformer} +import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.{DataFrame, Dataset, Encoder, Row} import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.test.TestSparkSession @@ -64,6 +65,38 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => } } + private[ml] def checkVectorSizeOnDF( + dataframe: DataFrame, + vecColName: String, + vecSize: Int): Unit = { + import dataframe.sparkSession.implicits._ + val group = AttributeGroup.fromStructField(dataframe.schema(vecColName)) + assert(group.size === vecSize, + s"the vector size obtained from schema should be $vecSize, but got ${group.size}") + val sizeUDF = udf { vector: Vector => vector.size } + assert(dataframe.select(sizeUDF(col(vecColName))) + .as[Int] + .collect() + .forall(_ === vecSize)) + } + + private[ml] def checkNominalOnDF( + dataframe: DataFrame, + colName: String, + numValues: Int): Unit = { + import dataframe.sparkSession.implicits._ + val n = Attribute.fromStructField(dataframe.schema(colName)) match { + case binAttr: BinaryAttribute => Some(2) + case nomAttr: NominalAttribute => nomAttr.getNumValues + } + assert(n.isDefined && n.get === numValues, + s"the number of values obtained from schema should be $numValues, but got $n") + assert(dataframe.select(colName) + .as[Double] + .collect() + .forall(v => v === v.toInt && v >= 0 && v < numValues)) + } + private[util] def testTransformerOnStreamData[A : Encoder]( dataframe: DataFrame, transformer: Transformer, From a2102c81ee10527cae2ed675f50de3a1972a4cce Mon Sep 17 00:00:00 2001 From: 07ARB Date: Wed, 4 Dec 2019 12:33:43 -0600 Subject: [PATCH 0790/1387] [SPARK-29453][WEBUI] Improve tooltips information for SQL tab ### What changes were proposed in this pull request? Adding tooltip to SQL tab for better usability. ### Why are the changes needed? There are a few common points of confusion in the UI that could be clarified with tooltips. We should add tooltips to explain. ### Does this PR introduce any user-facing change? yes. ![Screenshot 2019-11-23 at 9 47 41 AM](https://user-images.githubusercontent.com/8948111/69472963-aaec5980-0dd6-11ea-881a-fe6266171054.png) ### How was this patch tested? Manual test. Closes #26641 from 07ARB/SPARK-29453. Authored-by: 07ARB Signed-off-by: Sean Owen --- .../spark/sql/execution/ui/AllExecutionsPage.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala index ec0577283265d..e1ff90a2c20e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala @@ -324,7 +324,15 @@ private[ui] class ExecutionPagedTable( - {header} + {if (header == "Duration") { + + {header} + + } else { + {header} + }} } else { From 2ceed6f32c51a0c5e535fcff37336e7d8cd87537 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 4 Dec 2019 15:03:26 -0800 Subject: [PATCH 0791/1387] [SPARK-29392][CORE][SQL][FOLLOWUP] Avoid deprecated (in 2.13) Symbol syntax 'foo in favor of simpler expression, where it generated deprecation warnings ### What changes were proposed in this pull request? Where it generates a deprecation warning in Scala 2.13, replace Symbol shorthand syntax `'foo` with an equivalent. ### Why are the changes needed? Symbol syntax `'foo` is deprecated in Scala 2.13. The lines changed below otherwise generate about 440 warnings when building for 2.13. The previous PR directly replaced many usages with `Symbol("foo")`. But it's also used to specify Columns via implicit conversion (`.select('foo)`) or even where simple Strings are used (`.as('foo)`), as it's kind of an abstraction for interned Strings. While I find this syntax confusing and would like to deprecate it, here I just replaced it where it generates a build warning (not sure why all occurrences don't): `$"foo"` or just `"foo"`. ### Does this PR introduce any user-facing change? Should not change behavior. ### How was this patch tested? Existing tests. Closes #26748 from srowen/SPARK-29392.2. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../ml/classification/LinearSVCSuite.scala | 6 +- .../PowerIterationClusteringSuite.scala | 6 +- .../sql/catalyst/DistributionSuite.scala | 122 +++---- .../apache/spark/sql/CachedTableSuite.scala | 18 +- .../spark/sql/ColumnExpressionSuite.scala | 18 +- .../spark/sql/ConfigBehaviorSuite.scala | 2 +- .../apache/spark/sql/CsvFunctionsSuite.scala | 2 +- .../spark/sql/DataFrameAggregateSuite.scala | 100 +++--- .../datasources/orc/OrcFilterSuite.scala | 298 ++++++++--------- .../datasources/orc/OrcFilterSuite.scala | 299 +++++++++--------- .../sql/hive/HiveMetastoreCatalogSuite.scala | 6 +- .../sql/hive/HiveParquetSourceSuite.scala | 2 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 8 +- .../apache/spark/sql/hive/InsertSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 12 +- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- .../hive/execution/HiveTableScanSuite.scala | 4 +- .../execution/ObjectHashAggregateSuite.scala | 2 +- .../PruneFileSourcePartitionsSuite.scala | 3 +- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- .../sql/hive/orc/HiveOrcFilterSuite.scala | 269 ++++++++-------- 21 files changed, 594 insertions(+), 589 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala index b23b4f4ac0d26..8fb2343afb61a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -59,9 +59,9 @@ class LinearSVCSuite extends MLTest with DefaultReadWriteTest { // Dataset for testing SparseVector val toSparse: Vector => SparseVector = _.asInstanceOf[DenseVector].toSparse val sparse = udf(toSparse) - smallSparseBinaryDataset = smallBinaryDataset.withColumn("features", sparse('features)) - smallSparseValidationDataset = smallValidationDataset.withColumn("features", sparse('features)) - + smallSparseBinaryDataset = smallBinaryDataset.withColumn("features", sparse($"features")) + smallSparseValidationDataset = + smallValidationDataset.withColumn("features", sparse($"features")) } /** diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala index d3b8575327a87..6b3970def12ec 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala @@ -161,14 +161,14 @@ class PowerIterationClusteringSuite extends SparkFunSuite } test("test default weight") { - val dataWithoutWeight = data.sample(0.5, 1L).select('src, 'dst) + val dataWithoutWeight = data.sample(0.5, 1L).select("src", "dst") val assignments = new PowerIterationClustering() .setK(2) .setMaxIter(40) .assignClusters(dataWithoutWeight) val localAssignments = assignments - .select('id, 'cluster) + .select("id", "cluster") .as[(Long, Int)].collect().toSet val dataWithWeightOne = dataWithoutWeight.withColumn("weight", lit(1.0)) @@ -178,7 +178,7 @@ class PowerIterationClusteringSuite extends SparkFunSuite .setMaxIter(40) .assignClusters(dataWithWeightOne) val localAssignments2 = assignments2 - .select('id, 'cluster) + .select("id", "cluster") .as[(Long, Int)].collect().toSet assert(localAssignments === localAssignments2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 5c69ef95ffe17..4862f923e4fba 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -59,12 +59,12 @@ class DistributionSuite extends SparkFunSuite { true) checkSatisfied( - HashPartitioning(Seq('a), 10), + HashPartitioning(Seq($"a"), 10), UnspecifiedDistribution, true) checkSatisfied( - RangePartitioning(Seq('a.asc), 10), + RangePartitioning(Seq($"a".asc), 10), UnspecifiedDistribution, true) @@ -101,22 +101,22 @@ class DistributionSuite extends SparkFunSuite { true) checkSatisfied( - HashPartitioning(Seq('a), 1), + HashPartitioning(Seq($"a"), 1), AllTuples, true) checkSatisfied( - HashPartitioning(Seq('a), 10), + HashPartitioning(Seq($"a"), 10), AllTuples, false) checkSatisfied( - RangePartitioning(Seq('a.asc), 1), + RangePartitioning(Seq($"a".asc), 1), AllTuples, true) checkSatisfied( - RangePartitioning(Seq('a.asc), 10), + RangePartitioning(Seq($"a".asc), 10), AllTuples, false) @@ -130,17 +130,17 @@ class DistributionSuite extends SparkFunSuite { // SinglePartition can satisfy all the distributions except `BroadcastDistribution` checkSatisfied( SinglePartition, - ClusteredDistribution(Seq('a, 'b, 'c)), + ClusteredDistribution(Seq($"a", $"b", $"c")), true) checkSatisfied( SinglePartition, - HashClusteredDistribution(Seq('a, 'b, 'c)), + HashClusteredDistribution(Seq($"a", $"b", $"c")), true) checkSatisfied( SinglePartition, - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), true) checkSatisfied( @@ -153,56 +153,56 @@ class DistributionSuite extends SparkFunSuite { // HashPartitioning can satisfy ClusteredDistribution iff its hash expressions are a subset of // the required clustering expressions. checkSatisfied( - HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c")), true) checkSatisfied( - HashPartitioning(Seq('b, 'c), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq($"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c")), true) checkSatisfied( - HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('b, 'c)), + HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"b", $"c")), false) checkSatisfied( - HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('d, 'e)), + HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"d", $"e")), false) // HashPartitioning can satisfy HashClusteredDistribution iff its hash expressions are exactly // same with the required hash clustering expressions. checkSatisfied( - HashPartitioning(Seq('a, 'b, 'c), 10), - HashClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq($"a", $"b", $"c"), 10), + HashClusteredDistribution(Seq($"a", $"b", $"c")), true) checkSatisfied( - HashPartitioning(Seq('c, 'b, 'a), 10), - HashClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq($"c", $"b", $"a"), 10), + HashClusteredDistribution(Seq($"a", $"b", $"c")), false) checkSatisfied( - HashPartitioning(Seq('a, 'b), 10), - HashClusteredDistribution(Seq('a, 'b, 'c)), + HashPartitioning(Seq($"a", $"b"), 10), + HashClusteredDistribution(Seq($"a", $"b", $"c")), false) // HashPartitioning cannot satisfy OrderedDistribution checkSatisfied( - HashPartitioning(Seq('a, 'b, 'c), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioning(Seq($"a", $"b", $"c"), 10), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), false) checkSatisfied( - HashPartitioning(Seq('a, 'b, 'c), 1), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioning(Seq($"a", $"b", $"c"), 1), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), false) // TODO: this can be relaxed. checkSatisfied( - HashPartitioning(Seq('b, 'c), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + HashPartitioning(Seq($"b", $"c"), 10), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), false) } @@ -210,18 +210,18 @@ class DistributionSuite extends SparkFunSuite { // RangePartitioning can satisfy OrderedDistribution iff its ordering is a prefix // of the required ordering, or the required ordering is a prefix of its ordering. checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc)), true) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + OrderedDistribution(Seq($"a".asc, $"b".asc)), true) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc, 'd.desc)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc, 'd.desc)), true) // TODO: We can have an optimization to first sort the dataset @@ -229,78 +229,78 @@ class DistributionSuite extends SparkFunSuite { // should tradeoff the benefit of a less number of Exchange operators // and the parallelism. checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + OrderedDistribution(Seq($"a".asc, $"b".desc, $"c".asc)), false) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('b.asc, 'a.asc)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + OrderedDistribution(Seq($"b".asc, $"a".asc)), false) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - OrderedDistribution(Seq('a.asc, 'b.asc, 'd.desc)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + OrderedDistribution(Seq($"a".asc, $"b".asc, 'd.desc)), false) // RangePartitioning can satisfy ClusteredDistribution iff its ordering expressions are a subset // of the required clustering expressions. checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('a, 'b, 'c)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + ClusteredDistribution(Seq($"a", $"b", $"c")), true) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('c, 'b, 'a)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + ClusteredDistribution(Seq($"c", $"b", $"a")), true) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('b, 'c, 'a, 'd)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + ClusteredDistribution(Seq($"b", $"c", $"a", $"d")), true) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('a, 'b)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + ClusteredDistribution(Seq($"a", $"b")), false) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('c, 'd)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + ClusteredDistribution(Seq($"c", $"d")), false) // RangePartitioning cannot satisfy HashClusteredDistribution checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - HashClusteredDistribution(Seq('a, 'b, 'c)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + HashClusteredDistribution(Seq($"a", $"b", $"c")), false) } test("Partitioning.numPartitions must match Distribution.requiredNumPartitions to satisfy it") { checkSatisfied( SinglePartition, - ClusteredDistribution(Seq('a, 'b, 'c), Some(10)), + ClusteredDistribution(Seq($"a", $"b", $"c"), Some(10)), false) checkSatisfied( SinglePartition, - HashClusteredDistribution(Seq('a, 'b, 'c), Some(10)), + HashClusteredDistribution(Seq($"a", $"b", $"c"), Some(10)), false) checkSatisfied( - HashPartitioning(Seq('a, 'b, 'c), 10), - ClusteredDistribution(Seq('a, 'b, 'c), Some(5)), + HashPartitioning(Seq($"a", $"b", $"c"), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"), Some(5)), false) checkSatisfied( - HashPartitioning(Seq('a, 'b, 'c), 10), - HashClusteredDistribution(Seq('a, 'b, 'c), Some(5)), + HashPartitioning(Seq($"a", $"b", $"c"), 10), + HashClusteredDistribution(Seq($"a", $"b", $"c"), Some(5)), false) checkSatisfied( - RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), - ClusteredDistribution(Seq('a, 'b, 'c), Some(5)), + RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), + ClusteredDistribution(Seq($"a", $"b", $"c"), Some(5)), false) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index edafdaa54945a..d3f4159d6471f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -96,7 +96,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi test("cache temp table") { withTempView("tempTable") { - testData.select('key).createOrReplaceTempView("tempTable") + testData.select("key").createOrReplaceTempView("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) spark.catalog.cacheTable("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable")) @@ -127,8 +127,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi } test("uncaching temp table") { - testData.select('key).createOrReplaceTempView("tempTable1") - testData.select('key).createOrReplaceTempView("tempTable2") + testData.select("key").createOrReplaceTempView("tempTable1") + testData.select("key").createOrReplaceTempView("tempTable2") spark.catalog.cacheTable("tempTable1") assertCached(sql("SELECT COUNT(*) FROM tempTable1")) @@ -361,15 +361,15 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi } test("Drops temporary table") { - testData.select('key).createOrReplaceTempView("t1") + testData.select("key").createOrReplaceTempView("t1") spark.table("t1") spark.catalog.dropTempView("t1") intercept[AnalysisException](spark.table("t1")) } test("Drops cached temporary table") { - testData.select('key).createOrReplaceTempView("t1") - testData.select('key).createOrReplaceTempView("t2") + testData.select("key").createOrReplaceTempView("t1") + testData.select("key").createOrReplaceTempView("t2") spark.catalog.cacheTable("t1") assert(spark.catalog.isCached("t1")) @@ -859,7 +859,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi test("SPARK-23880 table cache should be lazy and don't trigger any jobs") { val cachedData = checkIfNoJobTriggered { - spark.range(1002).filter('id > 1000).orderBy('id.desc).cache() + spark.range(1002).filter($"id" > 1000).orderBy($"id".desc).cache() } assert(cachedData.collect === Seq(1001)) } @@ -891,7 +891,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi test("SPARK-24596 Non-cascading Cache Invalidation - drop persistent view") { withTable("t") { - spark.range(1, 10).toDF("key").withColumn("value", 'key * 2) + spark.range(1, 10).toDF("key").withColumn("value", $"key" * 2) .write.format("json").saveAsTable("t") withView("t1") { withTempView("t2") { @@ -911,7 +911,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi test("SPARK-24596 Non-cascading Cache Invalidation - uncache table") { withTable("t") { - spark.range(1, 10).toDF("key").withColumn("value", 'key * 2) + spark.range(1, 10).toDF("key").withColumn("value", $"key" * 2) .write.format("json").saveAsTable("t") withTempView("t1", "t2") { sql("CACHE TABLE t") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index c346c8946a972..a9ee25b10dc02 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -537,12 +537,12 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("sqrt") { checkAnswer( - testData.select(sqrt('key)).orderBy('key.asc), + testData.select(sqrt($"key")).orderBy($"key".asc), (1 to 100).map(n => Row(math.sqrt(n))) ) checkAnswer( - testData.select(sqrt('value), 'key).orderBy('key.asc, 'value.asc), + testData.select(sqrt($"value"), $"key").orderBy($"key".asc, $"value".asc), (1 to 100).map(n => Row(math.sqrt(n), n)) ) @@ -554,12 +554,12 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("upper") { checkAnswer( - lowerCaseData.select(upper('l)), + lowerCaseData.select(upper($"l")), ('a' to 'd').map(c => Row(c.toString.toUpperCase(Locale.ROOT))) ) checkAnswer( - testData.select(upper('value), 'key), + testData.select(upper($"value"), $"key"), (1 to 100).map(n => Row(n.toString, n)) ) @@ -575,12 +575,12 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { test("lower") { checkAnswer( - upperCaseData.select(lower('L)), + upperCaseData.select(lower($"L")), ('A' to 'F').map(c => Row(c.toString.toLowerCase(Locale.ROOT))) ) checkAnswer( - testData.select(lower('value), 'key), + testData.select(lower($"value"), $"key"), (1 to 100).map(n => Row(n.toString, n)) ) @@ -753,8 +753,8 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("columns can be compared") { - assert('key.desc == 'key.desc) - assert('key.desc != 'key.asc) + assert($"key".desc == $"key".desc) + assert($"key".desc != $"key".asc) } test("alias with metadata") { @@ -817,7 +817,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { } test("randn") { - val randCol = testData.select('key, randn(5L).as("rand")) + val randCol = testData.select($"key", randn(5L).as("rand")) randCol.columns.length should be (2) val rows = randCol.collect() rows.foreach { row => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala index 431e797e1686e..56ae904e83fdb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala @@ -41,7 +41,7 @@ class ConfigBehaviorSuite extends QueryTest with SharedSparkSession { // Trigger a sort // Range has range partitioning in its output now. To have a range shuffle, we // need to run a repartition first. - val data = spark.range(0, n, 1, 1).repartition(10).sort('id.desc) + val data = spark.range(0, n, 1, 1).repartition(10).sort($"id".desc) .selectExpr("SPARK_PARTITION_ID() pid", "id").as[(Int, Long)].collect() // Compute histogram for the number of records per partition post sort diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala index d34e50518348a..61f0e138cc358 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -153,7 +153,7 @@ class CsvFunctionsSuite extends QueryTest with SharedSparkSession { test("infers schemas of a CSV string and pass to to from_csv") { val in = Seq("""0.123456789,987654321,"San Francisco"""").toDS() val options = Map.empty[String, String].asJava - val out = in.select(from_csv('value, schema_of_csv("0.1,1,a"), options) as "parsed") + val out = in.select(from_csv($"value", schema_of_csv("0.1,1,a"), options) as "parsed") val expected = StructType(Seq(StructField( "parsed", StructType(Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index a08ef19c3ac5a..7065f076e8afd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -45,7 +45,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { Seq(Row(1, 3), Row(2, 3), Row(3, 3)) ) checkAnswer( - testData2.groupBy("a").agg(sum($"b").as("totB")).agg(sum('totB)), + testData2.groupBy("a").agg(sum($"b").as("totB")).agg(sum($"totB")), Row(9) ) checkAnswer( @@ -111,7 +111,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { val df = Seq(("some[thing]", "random-string")).toDF("key", "val") checkAnswer( - df.groupBy(regexp_extract('key, "([a-z]+)\\[", 1)).count(), + df.groupBy(regexp_extract($"key", "([a-z]+)\\[", 1)).count(), Row("some", 1) :: Nil ) } @@ -277,7 +277,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("agg without groups") { checkAnswer( - testData2.agg(sum('b)), + testData2.agg(sum($"b")), Row(9) ) } @@ -291,52 +291,53 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("average") { checkAnswer( - testData2.agg(avg('a), mean('a)), + testData2.agg(avg($"a"), mean($"a")), Row(2.0, 2.0)) checkAnswer( - testData2.agg(avg('a), sumDistinct('a)), // non-partial + testData2.agg(avg($"a"), sumDistinct($"a")), // non-partial Row(2.0, 6.0) :: Nil) checkAnswer( - decimalData.agg(avg('a)), + decimalData.agg(avg($"a")), Row(new java.math.BigDecimal(2))) checkAnswer( - decimalData.agg(avg('a), sumDistinct('a)), // non-partial + decimalData.agg(avg($"a"), sumDistinct($"a")), // non-partial Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) checkAnswer( - decimalData.agg(avg('a cast DecimalType(10, 2))), + decimalData.agg(avg($"a" cast DecimalType(10, 2))), Row(new java.math.BigDecimal(2))) // non-partial checkAnswer( - decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), + decimalData.agg( + avg($"a" cast DecimalType(10, 2)), sumDistinct($"a" cast DecimalType(10, 2))), Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) } test("null average") { checkAnswer( - testData3.agg(avg('b)), + testData3.agg(avg($"b")), Row(2.0)) checkAnswer( - testData3.agg(avg('b), countDistinct('b)), + testData3.agg(avg($"b"), countDistinct($"b")), Row(2.0, 1)) checkAnswer( - testData3.agg(avg('b), sumDistinct('b)), // non-partial + testData3.agg(avg($"b"), sumDistinct($"b")), // non-partial Row(2.0, 2.0)) } test("zero average") { val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") checkAnswer( - emptyTableData.agg(avg('a)), + emptyTableData.agg(avg($"a")), Row(null)) checkAnswer( - emptyTableData.agg(avg('a), sumDistinct('b)), // non-partial + emptyTableData.agg(avg($"a"), sumDistinct($"b")), // non-partial Row(null, null)) } @@ -344,28 +345,29 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { assert(testData2.count() === testData2.rdd.map(_ => 1).count()) checkAnswer( - testData2.agg(count('a), sumDistinct('a)), // non-partial + testData2.agg(count($"a"), sumDistinct($"a")), // non-partial Row(6, 6.0)) } test("null count") { checkAnswer( - testData3.groupBy('a).agg(count('b)), + testData3.groupBy($"a").agg(count($"b")), Seq(Row(1, 0), Row(2, 1)) ) checkAnswer( - testData3.groupBy('a).agg(count('a + 'b)), + testData3.groupBy($"a").agg(count($"a" + $"b")), Seq(Row(1, 0), Row(2, 1)) ) checkAnswer( - testData3.agg(count('a), count('b), count(lit(1)), countDistinct('a), countDistinct('b)), + testData3.agg( + count($"a"), count($"b"), count(lit(1)), countDistinct($"a"), countDistinct($"b")), Row(2, 1, 2, 2, 1) ) checkAnswer( - testData3.agg(count('b), countDistinct('b), sumDistinct('b)), // non-partial + testData3.agg(count($"b"), countDistinct($"b"), sumDistinct($"b")), // non-partial Row(1, 1, 2) ) } @@ -380,17 +382,17 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { .toDF("key1", "key2", "key3") checkAnswer( - df1.agg(countDistinct('key1, 'key2)), + df1.agg(countDistinct($"key1", $"key2")), Row(3) ) checkAnswer( - df1.agg(countDistinct('key1, 'key2, 'key3)), + df1.agg(countDistinct($"key1", $"key2", $"key3")), Row(3) ) checkAnswer( - df1.groupBy('key1).agg(countDistinct('key2, 'key3)), + df1.groupBy($"key1").agg(countDistinct($"key2", $"key3")), Seq(Row("a", 2), Row("x", 1)) ) } @@ -398,14 +400,14 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("zero count") { val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") checkAnswer( - emptyTableData.agg(count('a), sumDistinct('a)), // non-partial + emptyTableData.agg(count($"a"), sumDistinct($"a")), // non-partial Row(0, null)) } test("stddev") { val testData2ADev = math.sqrt(4.0 / 5.0) checkAnswer( - testData2.agg(stddev('a), stddev_pop('a), stddev_samp('a)), + testData2.agg(stddev($"a"), stddev_pop($"a"), stddev_samp($"a")), Row(testData2ADev, math.sqrt(4 / 6.0), testData2ADev)) checkAnswer( testData2.agg(stddev("a"), stddev_pop("a"), stddev_samp("a")), @@ -415,47 +417,47 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("zero stddev") { val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") checkAnswer( - emptyTableData.agg(stddev('a), stddev_pop('a), stddev_samp('a)), + emptyTableData.agg(stddev($"a"), stddev_pop($"a"), stddev_samp($"a")), Row(null, null, null)) } test("zero sum") { val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") checkAnswer( - emptyTableData.agg(sum('a)), + emptyTableData.agg(sum($"a")), Row(null)) } test("zero sum distinct") { val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") checkAnswer( - emptyTableData.agg(sumDistinct('a)), + emptyTableData.agg(sumDistinct($"a")), Row(null)) } test("moments") { - val sparkVariance = testData2.agg(variance('a)) + val sparkVariance = testData2.agg(variance($"a")) checkAggregatesWithTol(sparkVariance, Row(4.0 / 5.0), absTol) - val sparkVariancePop = testData2.agg(var_pop('a)) + val sparkVariancePop = testData2.agg(var_pop($"a")) checkAggregatesWithTol(sparkVariancePop, Row(4.0 / 6.0), absTol) - val sparkVarianceSamp = testData2.agg(var_samp('a)) + val sparkVarianceSamp = testData2.agg(var_samp($"a")) checkAggregatesWithTol(sparkVarianceSamp, Row(4.0 / 5.0), absTol) - val sparkSkewness = testData2.agg(skewness('a)) + val sparkSkewness = testData2.agg(skewness($"a")) checkAggregatesWithTol(sparkSkewness, Row(0.0), absTol) - val sparkKurtosis = testData2.agg(kurtosis('a)) + val sparkKurtosis = testData2.agg(kurtosis($"a")) checkAggregatesWithTol(sparkKurtosis, Row(-1.5), absTol) } test("zero moments") { val input = Seq((1, 2)).toDF("a", "b") checkAnswer( - input.agg(stddev('a), stddev_samp('a), stddev_pop('a), variance('a), - var_samp('a), var_pop('a), skewness('a), kurtosis('a)), + input.agg(stddev($"a"), stddev_samp($"a"), stddev_pop($"a"), variance($"a"), + var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN)) @@ -475,8 +477,8 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("null moments") { val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") - checkAnswer( - emptyTableData.agg(variance('a), var_samp('a), var_pop('a), skewness('a), kurtosis('a)), + checkAnswer(emptyTableData.agg( + variance($"a"), var_samp($"a"), var_pop($"a"), skewness($"a"), kurtosis($"a")), Row(null, null, null, null, null)) checkAnswer( @@ -566,7 +568,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("SQL decimal test (used for catching certain decimal handling bugs in aggregates)") { checkAnswer( - decimalData.groupBy('a cast DecimalType(10, 2)).agg(avg('b cast DecimalType(10, 2))), + decimalData.groupBy($"a" cast DecimalType(10, 2)).agg(avg($"b" cast DecimalType(10, 2))), Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal("1.5")), Row(new java.math.BigDecimal(2), new java.math.BigDecimal("1.5")), Row(new java.math.BigDecimal(3), new java.math.BigDecimal("1.5")))) @@ -653,7 +655,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { testData2.groupBy(lit(3), lit(4)).agg(lit(6), lit(7), sum("b")), Seq(Row(3, 4, 6, 7, 9))) checkAnswer( - testData2.groupBy(lit(3), lit(4)).agg(lit(6), 'b, sum("b")), + testData2.groupBy(lit(3), lit(4)).agg(lit(6), $"b", sum("b")), Seq(Row(3, 4, 6, 1, 3), Row(3, 4, 6, 2, 6))) checkAnswer( @@ -716,14 +718,14 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { assert(thrownException.message.contains("not allowed to use a window function")) } - checkWindowError(testData2.select(min(avg('b).over(Window.partitionBy('a))))) - checkWindowError(testData2.agg(sum('b), max(rank().over(Window.orderBy('a))))) - checkWindowError(testData2.groupBy('a).agg(sum('b), max(rank().over(Window.orderBy('b))))) - checkWindowError(testData2.groupBy('a).agg(max(sum(sum('b)).over(Window.orderBy('a))))) - checkWindowError( - testData2.groupBy('a).agg(sum('b).as("s"), max(count("*").over())).where('s === 3)) - checkAnswer( - testData2.groupBy('a).agg(max('b), sum('b).as("s"), count("*").over()).where('s === 3), + checkWindowError(testData2.select(min(avg($"b").over(Window.partitionBy($"a"))))) + checkWindowError(testData2.agg(sum($"b"), max(rank().over(Window.orderBy($"a"))))) + checkWindowError(testData2.groupBy($"a").agg(sum($"b"), max(rank().over(Window.orderBy($"b"))))) + checkWindowError(testData2.groupBy($"a").agg(max(sum(sum($"b")).over(Window.orderBy($"a"))))) + checkWindowError(testData2.groupBy($"a").agg( + sum($"b").as("s"), max(count("*").over())).where($"s" === 3)) + checkAnswer(testData2.groupBy($"a").agg( + max($"b"), sum($"b").as("s"), count("*").over()).where($"s" === 3), Row(1, 2, 3, 3) :: Row(2, 2, 3, 3) :: Row(3, 2, 3, 3) :: Nil) checkWindowError(sql("SELECT MIN(AVG(b) OVER(PARTITION BY a)) FROM testData2")) @@ -739,7 +741,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("SPARK-24788: RelationalGroupedDataset.toString with unresolved exprs should not fail") { // Checks if these raise no exception - assert(testData.groupBy('key).toString.contains( + assert(testData.groupBy($"key").toString.contains( "[grouping expressions: [key], value: [key: int, value: string], type: GroupBy]")) assert(testData.groupBy(col("key")).toString.contains( "[grouping expressions: [key], value: [key: int, value: string], type: GroupBy]")) @@ -954,10 +956,10 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("calendar interval agg support hash aggregate") { val df1 = Seq((1, "1 day"), (2, "2 day"), (3, "3 day"), (3, null)).toDF("a", "b") - val df2 = df1.select(avg('b cast CalendarIntervalType)) + val df2 = df1.select(avg($"b" cast CalendarIntervalType)) checkAnswer(df2, Row(new CalendarInterval(0, 2, 0)) :: Nil) assert(df2.queryExecution.executedPlan.find(_.isInstanceOf[HashAggregateExec]).isDefined) - val df3 = df1.groupBy('a).agg(avg('b cast CalendarIntervalType)) + val df3 = df1.groupBy($"a").agg(avg($"b" cast CalendarIntervalType)) checkAnswer(df3, Row(1, new CalendarInterval(0, 1, 0)) :: Row(2, new CalendarInterval(0, 2, 0)) :: diff --git a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 80e330b3f2ae2..d09236a934337 100644 --- a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -87,154 +87,154 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { test("filter pushdown - integer") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - long") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - float") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - double") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - string") { withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - boolean") { withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - decimal") { withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - checkFilterPredicate('_1 === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate($"_1" === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate('_1 < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === '_1, PredicateLeaf.Operator.EQUALS) + Literal(BigDecimal.valueOf(1)) === $"_1", PredicateLeaf.Operator.EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + Literal(BigDecimal.valueOf(1)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > '_1, PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(2)) > $"_1", PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(3)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(1)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(4)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } @@ -245,46 +245,46 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { new Timestamp(milliseconds) } withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(timestamps(0)) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(timestamps(0)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(timestamps(2)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(timestamps(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - combinations with logical operators") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => checkFilterPredicate( - '_1.isNotNull, + $"_1".isNotNull, "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" ) checkFilterPredicate( - '_1 =!= 1, + $"_1" =!= 1, "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - !('_1 < 4), + !($"_1" < 4), "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - '_1 < 2 || '_1 > 3, + $"_1" < 2 || $"_1" > 3, "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + "expr = (or leaf-0 (not leaf-1))" ) checkFilterPredicate( - '_1 < 2 && '_1 > 3, + $"_1" < 2 && $"_1" > 3, "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + "expr = (and (not leaf-0) leaf-1 (not leaf-2))" ) @@ -296,22 +296,22 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { Date.valueOf(day) } withOrcDataFrame(dates.map(Tuple1(_))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === dates(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> dates(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < dates(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > dates(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= dates(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= dates(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(dates(0)) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(dates(0)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(dates(1)) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(dates(2)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(dates(0)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(dates(3)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === dates(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> dates(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < dates(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > dates(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= dates(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= dates(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(dates(0)) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(dates(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(dates(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(dates(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(dates(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(dates(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } @@ -321,15 +321,15 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } // ArrayType withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate('_1.isNull, noneSupported = true) + checkNoFilterPredicate($"_1".isNull, noneSupported = true) } // BinaryType withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate('_1 <=> 1.b, noneSupported = true) + checkNoFilterPredicate($"_1" <=> 1.b, noneSupported = true) } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate('_1.isNotNull, noneSupported = true) + checkNoFilterPredicate($"_1".isNotNull, noneSupported = true) } } diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 5e51b8c5aed6a..b95a32ef85ddf 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -88,154 +88,154 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { test("filter pushdown - integer") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - long") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - float") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - double") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - string") { withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - boolean") { withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - decimal") { withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - checkFilterPredicate('_1 === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate($"_1" === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate('_1 < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === '_1, PredicateLeaf.Operator.EQUALS) + Literal(BigDecimal.valueOf(1)) === $"_1", PredicateLeaf.Operator.EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + Literal(BigDecimal.valueOf(1)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > '_1, PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(2)) > $"_1", PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(3)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(1)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(4)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } @@ -246,46 +246,47 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { new Timestamp(milliseconds) } withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(timestamps(0)) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(timestamps(0)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(timestamps(2)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(timestamps(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - combinations with logical operators") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => checkFilterPredicate( - '_1.isNotNull, + $"_1".isNotNull, "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" ) checkFilterPredicate( - '_1 =!= 1, + $"_1" =!= 1, "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - !('_1 < 4), + !($"_1" < 4), "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - '_1 < 2 || '_1 > 3, + $"_1" < 2 || $"_1" > 3, "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + "expr = (or leaf-0 (not leaf-1))" ) checkFilterPredicate( - '_1 < 2 && '_1 > 3, + $"_1" < 2 && $"_1" > 3, "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + "expr = (and (not leaf-0) leaf-1 (not leaf-2))" ) @@ -297,22 +298,22 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { Date.valueOf(day) } withOrcDataFrame(dates.map(Tuple1(_))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === dates(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> dates(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < dates(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > dates(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= dates(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= dates(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(dates(0)) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(dates(0)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(dates(1)) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(dates(2)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(dates(0)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(dates(3)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === dates(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> dates(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < dates(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > dates(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= dates(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= dates(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(dates(0)) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(dates(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(dates(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(dates(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(dates(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(dates(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } @@ -322,15 +323,15 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } // ArrayType withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate('_1.isNull, noneSupported = true) + checkNoFilterPredicate($"_1".isNull, noneSupported = true) } // BinaryType withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate('_1 <=> 1.b, noneSupported = true) + checkNoFilterPredicate($"_1" <=> 1.b, noneSupported = true) } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate('_1.isNotNull, noneSupported = true) + checkNoFilterPredicate($"_1".isNotNull, noneSupported = true) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 9f2906df19bff..20bafd832d0da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -46,7 +46,7 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton with SQLTestUtils { test("duplicated metastore relations") { val df = spark.sql("SELECT * FROM src") logInfo(df.queryExecution.toString) - df.as('a).join(df.as('b), $"a.key" === $"b.key") + df.as("a").join(df.as("b"), $"a.key" === $"b.key") } test("should not truncate struct type catalog string") { @@ -142,8 +142,8 @@ class DataSourceWithHiveMetastoreCatalogSuite import testImplicits._ private val testDF = range(1, 3).select( - ('id + 0.1) cast DecimalType(10, 3) as 'd1, - 'id cast StringType as 'd2 + ($"id" + 0.1) cast DecimalType(10, 3) as "d1", + $"id" cast StringType as "d2" ).coalesce(1) override def beforeAll(): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSourceSuite.scala index de588768cfdee..071f0fec97663 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSourceSuite.scala @@ -212,7 +212,7 @@ class HiveParquetSourceSuite extends ParquetPartitioningTest { val filePath2 = new File(tempDir, "testParquet2").getCanonicalPath val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") - val df2 = df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").max("y.int") + val df2 = df.as("x").join(df.as("y"), $"x.str" === $"y.str").groupBy("y.str").max("y.int") intercept[Throwable](df2.write.parquet(filePath)) val df3 = df2.toDF("str", "max_int") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 8056deefa07c9..31ff62ed0a530 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -700,7 +700,7 @@ object SPARK_9757 extends QueryTest { val df = hiveContext .range(10) - .select(('id + 0.1) cast DecimalType(10, 3) as 'dec) + .select(($"id" + 0.1) cast DecimalType(10, 3) as "dec") df.write.option("path", dir.getCanonicalPath).mode("overwrite").saveAsTable("t") checkAnswer(hiveContext.table("t"), df) } @@ -709,7 +709,7 @@ object SPARK_9757 extends QueryTest { val df = hiveContext .range(10) - .select(callUDF("struct", ('id + 0.2) cast DecimalType(10, 3)) as 'dec_struct) + .select(callUDF("struct", ($"id" + 0.2) cast DecimalType(10, 3)) as "dec_struct") df.write.option("path", dir.getCanonicalPath).mode("overwrite").saveAsTable("t") checkAnswer(hiveContext.table("t"), df) } @@ -771,8 +771,8 @@ object SPARK_14244 extends QueryTest { import hiveContext.implicits._ try { - val window = Window.orderBy('id) - val df = spark.range(2).select(cume_dist().over(window).as('cdist)).orderBy('cdist) + val window = Window.orderBy("id") + val df = spark.range(2).select(cume_dist().over(window).as("cdist")).orderBy("cdist") checkAnswer(df, Seq(Row(0.5D), Row(1.0D))) } finally { sparkContext.stop() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 8f1cd8089458f..d0852b22d1854 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -461,7 +461,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter // Columns `c + 1` and `d + 1` are resolved by position, and thus mapped to partition // columns `b` and `c` of the target table. val df = Seq((1, 2, 3, 4)).toDF("a", "b", "c", "d") - df.select('a + 1, 'b + 1, 'c + 1, 'd + 1).write.insertInto(tableName) + df.select($"a" + 1, $"b" + 1, $"c" + 1, $"d" + 1).write.insertInto(tableName) checkAnswer( sql(s"SELECT a, b, c, d FROM $tableName"), 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 7a7618d4125ad..56f424d9784e7 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 @@ -1253,7 +1253,7 @@ class HiveDDLSuite val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED withTempView(sourceViewName) { withTable(targetTabName) { - spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + spark.range(10).select($"id" as "a", $"id" as "b", $"id" as "c", $"id" as "d") .createTempView(sourceViewName) val locationClause = if (location.nonEmpty) s"LOCATION '${location.getOrElse("")}'" else "" @@ -1287,7 +1287,7 @@ class HiveDDLSuite val targetTabName = "tab2" val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED withTable(sourceTabName, targetTabName) { - spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + spark.range(10).select($"id" as "a", $"id" as "b", $"id" as "c", $"id" as "d") .write.format("json").saveAsTable(sourceTabName) val locationClause = if (location.nonEmpty) s"LOCATION '${location.getOrElse("")}'" else "" @@ -1327,7 +1327,7 @@ class HiveDDLSuite withTable(sourceTabName, targetTabName) { withTempPath { dir => val path = dir.getCanonicalPath - spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + spark.range(10).select($"id" as "a", $"id" as "b", $"id" as "c", $"id" as "d") .write.format("parquet").save(path) sql(s"CREATE TABLE $sourceTabName USING parquet OPTIONS (PATH '${dir.toURI}')") @@ -1456,7 +1456,7 @@ class HiveDDLSuite val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED withTable(sourceTabName, targetTabName) { withView(sourceViewName) { - spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + spark.range(10).select($"id" as "a", $"id" as "b", $"id" as "c", $"id" as "d") .write.format("json").saveAsTable(sourceTabName) sql(s"CREATE VIEW $sourceViewName AS SELECT * FROM $sourceTabName") @@ -2456,12 +2456,12 @@ class HiveDDLSuite test("SPARK-22252: FileFormatWriter should respect the input query schema in HIVE") { withTable("t1", "t2", "t3", "t4") { - spark.range(1).select('id as 'col1, 'id as 'col2).write.saveAsTable("t1") + spark.range(1).select($"id" as "col1", $"id" as "col2").write.saveAsTable("t1") spark.sql("select COL1, COL2 from t1").write.format("hive").saveAsTable("t2") checkAnswer(spark.table("t2"), Row(0, 0)) // Test picking part of the columns when writing. - spark.range(1).select('id, 'id as 'col1, 'id as 'col2).write.saveAsTable("t3") + spark.range(1).select($"id", $"id" as "col1", $"id" as "col2").write.saveAsTable("t3") spark.sql("select COL1, COL2 from t3").write.format("hive").saveAsTable("t4") checkAnswer(spark.table("t4"), Row(0, 0)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 5e84d9513adf5..5a8365017a5ba 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -711,7 +711,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd } def isExplanation(result: DataFrame): Boolean = { - val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } + val explanation = result.select("plan").collect().map { case Row(plan: String) => plan } explanation.head.startsWith("== Physical Plan ==") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 3f9bb8de42e09..67d7ed0841abb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -85,8 +85,8 @@ class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestH sql("create table spark_4959 (col1 string)") sql("""insert into table spark_4959 select "hi" from src limit 1""") table("spark_4959").select( - 'col1.as("CaseSensitiveColName"), - 'col1.as("CaseSensitiveColName2")).createOrReplaceTempView("spark_4959_2") + $"col1".as("CaseSensitiveColName"), + $"col1".as("CaseSensitiveColName2")).createOrReplaceTempView("spark_4959_2") assert(sql("select CaseSensitiveColName from spark_4959_2").head() === Row("hi")) assert(sql("select casesensitivecolname from spark_4959_2").head() === Row("hi")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala index 2391106cfb253..930f801467497 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala @@ -156,7 +156,7 @@ class ObjectHashAggregateSuite ) checkAnswer( - df.groupBy($"id" % 4 as 'mod).agg(aggFunctions.head, aggFunctions.tail: _*), + df.groupBy($"id" % 4 as "mod").agg(aggFunctions.head, aggFunctions.tail: _*), data.groupBy(_.getInt(0) % 4).map { case (key, value) => key -> Row.fromSeq(value.map(_.toSeq).transpose.map(_.count(_ != null): Long)) }.toSeq.map { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 6b2d0c656b371..c9c36992906a8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -65,7 +65,8 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te options = Map.empty)(sparkSession = spark) val logicalRelation = LogicalRelation(relation, tableMeta) - val query = Project(Seq('i, 'p), Filter('p === 1, logicalRelation)).analyze + val query = Project(Seq(Symbol("i"), Symbol("p")), + Filter(Symbol("p") === 1, logicalRelation)).analyze val optimized = Optimize.execute(query) assert(optimized.missingInput.isEmpty) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 3e5834f33ea52..d35c9336821ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -2375,7 +2375,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { }) spark .range(5) - .select(badUDF('id).as("a")) + .select(badUDF($"id").as("a")) .createOrReplaceTempView("test") val scriptFilePath = getTestResourcePath("data") val e = intercept[SparkException] { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala index b5e50915c7c89..5fc41067f661d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala @@ -124,154 +124,154 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { test("filter pushdown - integer") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - long") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - float") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - double") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - string") { withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - boolean") { withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - decimal") { withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - checkFilterPredicate('_1 === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate($"_1" === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate('_1 < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === '_1, PredicateLeaf.Operator.EQUALS) + Literal(BigDecimal.valueOf(1)) === $"_1", PredicateLeaf.Operator.EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + Literal(BigDecimal.valueOf(1)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > '_1, PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(2)) > $"_1", PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(3)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(1)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(4)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } @@ -282,22 +282,23 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { new Timestamp(milliseconds) } withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => - checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate('_1 === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate('_1 <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate('_1 < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate('_1 > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate('_1 >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(timestamps(0)) === '_1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(timestamps(0)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > '_1, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(timestamps(2)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(timestamps(0)) <=> $"_1", + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } @@ -309,30 +310,30 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { // to produce string expression and then compare it to given string expression below. // This might have to be changed after Hive version is upgraded. checkFilterPredicateWithDiffHiveVersion( - '_1.isNotNull, + $"_1".isNotNull, """leaf-0 = (IS_NULL _1) |expr = (not leaf-0)""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - '_1 =!= 1, + $"_1" =!= 1, """leaf-0 = (IS_NULL _1) |leaf-1 = (EQUALS _1 1) |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - !('_1 < 4), + !($"_1" < 4), """leaf-0 = (IS_NULL _1) |leaf-1 = (LESS_THAN _1 4) |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - '_1 < 2 || '_1 > 3, + $"_1" < 2 || $"_1" > 3, """leaf-0 = (LESS_THAN _1 2) |leaf-1 = (LESS_THAN_EQUALS _1 3) |expr = (or leaf-0 (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - '_1 < 2 && '_1 > 3, + $"_1" < 2 && $"_1" > 3, """leaf-0 = (IS_NULL _1) |leaf-1 = (LESS_THAN _1 2) |leaf-2 = (LESS_THAN_EQUALS _1 3) @@ -347,22 +348,22 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { } // ArrayType withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate('_1.isNull) + checkNoFilterPredicate($"_1".isNull) } // BinaryType withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate('_1 <=> 1.b) + checkNoFilterPredicate($"_1" <=> 1.b) } // DateType if (!HiveUtils.isHive23) { val stringDate = "2015-01-01" withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate)))) { implicit df => - checkNoFilterPredicate('_1 === Date.valueOf(stringDate)) + checkNoFilterPredicate($"_1" === Date.valueOf(stringDate)) } } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate('_1.isNotNull) + checkNoFilterPredicate($"_1".isNotNull) } } From 29e09a83b7c6babdd2cc5688ce27982de4c3cd46 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 4 Dec 2019 17:31:23 -0600 Subject: [PATCH 0792/1387] [SPARK-30084][DOCS] Document how to trigger Jekyll build on Python API doc changes ### What changes were proposed in this pull request? This PR adds a note to the docs README showing how to get Jekyll to automatically pick up changes to the Python API docs. ### Why are the changes needed? `jekyll serve --watch` doesn't watch for changes to the API docs. Without the technique documented in this note, or something equivalent, developers have to manually retrigger a Jekyll build any time they update the Python API docs. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? I tested this PR manually by making changes to Python docstrings and confirming that Jekyll automatically picks them up and serves them locally. Closes #26719 from nchammas/SPARK-30084-watch-api-docs. Authored-by: Nicholas Chammas Signed-off-by: Sean Owen --- .gitignore | 3 ++- docs/README.md | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index ae20c85ebe351..798e8acc4d43b 100644 --- a/.gitignore +++ b/.gitignore @@ -45,7 +45,7 @@ dev/create-release/*final dev/create-release/*txt dev/pr-deps/ dist/ -docs/_site +docs/_site/ docs/api sql/docs sql/site @@ -63,6 +63,7 @@ project/plugins/target/ python/lib/pyspark.zip python/.eggs/ python/deps +python/docs/_site/ python/test_coverage/coverage_data python/test_coverage/htmlcov python/pyspark/python diff --git a/docs/README.md b/docs/README.md index 0bb1ada7167df..ef849d53daf75 100644 --- a/docs/README.md +++ b/docs/README.md @@ -102,3 +102,17 @@ using [MkDocs](https://www.mkdocs.org/). NOTE: To skip the step of building and copying over the Scala, Java, Python, R and SQL API docs, run `SKIP_API=1 jekyll build`. In addition, `SKIP_SCALADOC=1`, `SKIP_PYTHONDOC=1`, `SKIP_RDOC=1` and `SKIP_SQLDOC=1` can be used to skip a single step of the corresponding language. `SKIP_SCALADOC` indicates skipping both the Scala and Java docs. + +### Automatically Rebuilding API Docs + +`jekyll serve --watch` will only watch what's in `docs/`, and it won't follow symlinks. That means it won't monitor your API docs under `python/docs` or elsewhere. + +To work around this limitation for Python, install [`entr`](http://eradman.com/entrproject/) and run the following in a separate shell: + +```sh +cd "$SPARK_HOME/python/docs" +find .. -type f -name '*.py' \ +| entr -s 'make html && cp -r _build/html/. ../../docs/api/python' +``` + +Whenever there is a change to your Python code, `entr` will automatically rebuild the Python API docs and copy them to `docs/`, thus triggering a Jekyll update. From c5f312a6ac9099a4a51e4ee923435408918ef310 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 4 Dec 2019 17:11:50 -0800 Subject: [PATCH 0793/1387] [SPARK-30129][CORE] Set client's id in TransportClient after successful auth The new auth code was missing this bit, so it was not possible to know which app a client belonged to when auth was on. I also refactored the SASL test that checks for this so it also checks the new protocol (test failed before the fix, passes now). Closes #26760 from vanzin/SPARK-30129. Authored-by: Marcelo Vanzin Signed-off-by: Dongjoon Hyun --- .../network/crypto/AuthClientBootstrap.java | 1 + .../spark/network/crypto/AuthRpcHandler.java | 1 + .../network/sasl/SaslIntegrationSuite.java | 106 ---------- .../network/shuffle/AppIsolationSuite.java | 184 ++++++++++++++++++ 4 files changed, 186 insertions(+), 106 deletions(-) create mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/AppIsolationSuite.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java index 77b167d15e911..4428f0f295d6e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java @@ -78,6 +78,7 @@ public void doBootstrap(TransportClient client, Channel channel) { try { doSparkAuth(client, channel); + client.setClientId(appId); } catch (GeneralSecurityException | IOException e) { throw Throwables.propagate(e); } catch (RuntimeException e) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java index fb44dbbb0953b..821cc7a849504 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java @@ -125,6 +125,7 @@ public void receive(TransportClient client, ByteBuffer message, RpcResponseCallb response.encode(responseData); callback.onSuccess(responseData.nioBuffer()); engine.sessionCipher().addToChannel(channel); + client.setClientId(challenge.appId); } catch (Exception e) { // This is a fatal error: authentication has failed. Close the channel explicitly. LOG.debug("Authentication failed for client {}, closing channel.", channel.remoteAddress()); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index e8e766d3fb3ab..96dfc3b7cae61 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -21,8 +21,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicReference; import org.junit.After; import org.junit.AfterClass; @@ -34,8 +32,6 @@ import org.apache.spark.network.TestUtils; import org.apache.spark.network.TransportContext; -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.client.ChunkReceivedCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientFactory; @@ -44,15 +40,6 @@ import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; -import org.apache.spark.network.shuffle.BlockFetchingListener; -import org.apache.spark.network.shuffle.ExternalBlockHandler; -import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver; -import org.apache.spark.network.shuffle.OneForOneBlockFetcher; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.shuffle.protocol.OpenBlocks; -import org.apache.spark.network.shuffle.protocol.RegisterExecutor; -import org.apache.spark.network.shuffle.protocol.StreamHandle; import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -165,93 +152,6 @@ public void testNoSaslServer() { } } - /** - * This test is not actually testing SASL behavior, but testing that the shuffle service - * performs correct authorization checks based on the SASL authentication data. - */ - @Test - public void testAppIsolation() throws Exception { - // Start a new server with the correct RPC handler to serve block data. - ExternalShuffleBlockResolver blockResolver = mock(ExternalShuffleBlockResolver.class); - ExternalBlockHandler blockHandler = new ExternalBlockHandler( - new OneForOneStreamManager(), blockResolver); - TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, secretKeyHolder); - - try ( - TransportContext blockServerContext = new TransportContext(conf, blockHandler); - TransportServer blockServer = blockServerContext.createServer(Arrays.asList(bootstrap)); - // Create a client, and make a request to fetch blocks from a different app. - TransportClientFactory clientFactory1 = blockServerContext.createClientFactory( - Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); - TransportClient client1 = clientFactory1.createClient( - TestUtils.getLocalHost(), blockServer.getPort())) { - - AtomicReference exception = new AtomicReference<>(); - - CountDownLatch blockFetchLatch = new CountDownLatch(1); - BlockFetchingListener listener = new BlockFetchingListener() { - @Override - public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { - blockFetchLatch.countDown(); - } - @Override - public void onBlockFetchFailure(String blockId, Throwable t) { - exception.set(t); - blockFetchLatch.countDown(); - } - }; - - String[] blockIds = { "shuffle_0_1_2", "shuffle_0_3_4" }; - OneForOneBlockFetcher fetcher = - new OneForOneBlockFetcher(client1, "app-2", "0", blockIds, listener, conf); - fetcher.start(); - blockFetchLatch.await(); - checkSecurityException(exception.get()); - - // Register an executor so that the next steps work. - ExecutorShuffleInfo executorInfo = new ExecutorShuffleInfo( - new String[] { System.getProperty("java.io.tmpdir") }, 1, - "org.apache.spark.shuffle.sort.SortShuffleManager"); - RegisterExecutor regmsg = new RegisterExecutor("app-1", "0", executorInfo); - client1.sendRpcSync(regmsg.toByteBuffer(), TIMEOUT_MS); - - // Make a successful request to fetch blocks, which creates a new stream. But do not actually - // fetch any blocks, to keep the stream open. - OpenBlocks openMessage = new OpenBlocks("app-1", "0", blockIds); - ByteBuffer response = client1.sendRpcSync(openMessage.toByteBuffer(), TIMEOUT_MS); - StreamHandle stream = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response); - long streamId = stream.streamId; - - try ( - // Create a second client, authenticated with a different app ID, and try to read from - // the stream created for the previous app. - TransportClientFactory clientFactory2 = blockServerContext.createClientFactory( - Arrays.asList(new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); - TransportClient client2 = clientFactory2.createClient( - TestUtils.getLocalHost(), blockServer.getPort()) - ) { - CountDownLatch chunkReceivedLatch = new CountDownLatch(1); - ChunkReceivedCallback callback = new ChunkReceivedCallback() { - @Override - public void onSuccess(int chunkIndex, ManagedBuffer buffer) { - chunkReceivedLatch.countDown(); - } - - @Override - public void onFailure(int chunkIndex, Throwable t) { - exception.set(t); - chunkReceivedLatch.countDown(); - } - }; - - exception.set(null); - client2.fetchChunk(streamId, 0, callback); - chunkReceivedLatch.await(); - checkSecurityException(exception.get()); - } - } - } - /** RPC handler which simply responds with the message it received. */ public static class TestRpcHandler extends RpcHandler { @Override @@ -264,10 +164,4 @@ public StreamManager getStreamManager() { return new OneForOneStreamManager(); } } - - private static void checkSecurityException(Throwable t) { - assertNotNull("No exception was caught.", t); - assertTrue("Expected SecurityException.", - t.getMessage().contains(SecurityException.class.getName())); - } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/AppIsolationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/AppIsolationSuite.java new file mode 100644 index 0000000000000..92e75222d0391 --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/AppIsolationSuite.java @@ -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.network.shuffle; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.Supplier; + +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.crypto.AuthClientBootstrap; +import org.apache.spark.network.crypto.AuthServerBootstrap; +import org.apache.spark.network.sasl.SaslClientBootstrap; +import org.apache.spark.network.sasl.SaslServerBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.shuffle.protocol.OpenBlocks; +import org.apache.spark.network.shuffle.protocol.RegisterExecutor; +import org.apache.spark.network.shuffle.protocol.StreamHandle; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class AppIsolationSuite { + + // Use a long timeout to account for slow / overloaded build machines. In the normal case, + // tests should finish way before the timeout expires. + private static final long TIMEOUT_MS = 10_000; + + private static SecretKeyHolder secretKeyHolder; + private static TransportConf conf; + + @BeforeClass + public static void beforeAll() { + Map confMap = new HashMap<>(); + confMap.put("spark.network.crypto.enabled", "true"); + confMap.put("spark.network.crypto.saslFallback", "false"); + conf = new TransportConf("shuffle", new MapConfigProvider(confMap)); + + secretKeyHolder = mock(SecretKeyHolder.class); + when(secretKeyHolder.getSaslUser(eq("app-1"))).thenReturn("app-1"); + when(secretKeyHolder.getSecretKey(eq("app-1"))).thenReturn("app-1"); + when(secretKeyHolder.getSaslUser(eq("app-2"))).thenReturn("app-2"); + when(secretKeyHolder.getSecretKey(eq("app-2"))).thenReturn("app-2"); + } + + @Test + public void testSaslAppIsolation() throws Exception { + testAppIsolation( + () -> new SaslServerBootstrap(conf, secretKeyHolder), + appId -> new SaslClientBootstrap(conf, appId, secretKeyHolder)); + } + + @Test + public void testAuthEngineAppIsolation() throws Exception { + testAppIsolation( + () -> new AuthServerBootstrap(conf, secretKeyHolder), + appId -> new AuthClientBootstrap(conf, appId, secretKeyHolder)); + } + + private void testAppIsolation( + Supplier serverBootstrap, + Function clientBootstrapFactory) throws Exception { + // Start a new server with the correct RPC handler to serve block data. + ExternalShuffleBlockResolver blockResolver = mock(ExternalShuffleBlockResolver.class); + ExternalBlockHandler blockHandler = new ExternalBlockHandler( + new OneForOneStreamManager(), blockResolver); + TransportServerBootstrap bootstrap = serverBootstrap.get(); + + try ( + TransportContext blockServerContext = new TransportContext(conf, blockHandler); + TransportServer blockServer = blockServerContext.createServer(Arrays.asList(bootstrap)); + // Create a client, and make a request to fetch blocks from a different app. + TransportClientFactory clientFactory1 = blockServerContext.createClientFactory( + Arrays.asList(clientBootstrapFactory.apply("app-1"))); + TransportClient client1 = clientFactory1.createClient( + TestUtils.getLocalHost(), blockServer.getPort())) { + + AtomicReference exception = new AtomicReference<>(); + + CountDownLatch blockFetchLatch = new CountDownLatch(1); + BlockFetchingListener listener = new BlockFetchingListener() { + @Override + public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { + blockFetchLatch.countDown(); + } + @Override + public void onBlockFetchFailure(String blockId, Throwable t) { + exception.set(t); + blockFetchLatch.countDown(); + } + }; + + String[] blockIds = { "shuffle_0_1_2", "shuffle_0_3_4" }; + OneForOneBlockFetcher fetcher = + new OneForOneBlockFetcher(client1, "app-2", "0", blockIds, listener, conf); + fetcher.start(); + blockFetchLatch.await(); + checkSecurityException(exception.get()); + + // Register an executor so that the next steps work. + ExecutorShuffleInfo executorInfo = new ExecutorShuffleInfo( + new String[] { System.getProperty("java.io.tmpdir") }, 1, + "org.apache.spark.shuffle.sort.SortShuffleManager"); + RegisterExecutor regmsg = new RegisterExecutor("app-1", "0", executorInfo); + client1.sendRpcSync(regmsg.toByteBuffer(), TIMEOUT_MS); + + // Make a successful request to fetch blocks, which creates a new stream. But do not actually + // fetch any blocks, to keep the stream open. + OpenBlocks openMessage = new OpenBlocks("app-1", "0", blockIds); + ByteBuffer response = client1.sendRpcSync(openMessage.toByteBuffer(), TIMEOUT_MS); + StreamHandle stream = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response); + long streamId = stream.streamId; + + try ( + // Create a second client, authenticated with a different app ID, and try to read from + // the stream created for the previous app. + TransportClientFactory clientFactory2 = blockServerContext.createClientFactory( + Arrays.asList(clientBootstrapFactory.apply("app-2"))); + TransportClient client2 = clientFactory2.createClient( + TestUtils.getLocalHost(), blockServer.getPort()) + ) { + CountDownLatch chunkReceivedLatch = new CountDownLatch(1); + ChunkReceivedCallback callback = new ChunkReceivedCallback() { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + chunkReceivedLatch.countDown(); + } + + @Override + public void onFailure(int chunkIndex, Throwable t) { + exception.set(t); + chunkReceivedLatch.countDown(); + } + }; + + exception.set(null); + client2.fetchChunk(streamId, 0, callback); + chunkReceivedLatch.await(); + checkSecurityException(exception.get()); + } + } + } + + private static void checkSecurityException(Throwable t) { + assertNotNull("No exception was caught.", t); + assertTrue("Expected SecurityException.", + t.getMessage().contains(SecurityException.class.getName())); + } +} From ebd83a544e0eb9fe03e9c1c879e00b50d947a761 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 5 Dec 2019 11:27:25 +0800 Subject: [PATCH 0794/1387] [SPARK-30009][CORE][SQL][FOLLOWUP] Remove OrderingUtil and Utils.nanSafeCompare{Doubles,Floats} and use java.lang.{Double,Float}.compare directly ### What changes were proposed in this pull request? Follow up on https://github.com/apache/spark/pull/26654#discussion_r353826162 Instead of OrderingUtil or Utils.nanSafeCompare{Doubles,Floats}, just use java.lang.{Double,Float}.compare directly. All work identically w.r.t. NaN when used to `compare`. ### Why are the changes needed? Simplification of the previous change, which existed to support Scala 2.13 migration. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests Closes #26761 from srowen/SPARK-30009.2. Authored-by: Sean Owen Signed-off-by: Wenchen Fan --- .../org/apache/spark/util/OrderingUtil.scala | 33 ------------------ .../org/apache/spark/util/OrderingUtil.scala | 34 ------------------- .../scala/org/apache/spark/util/Utils.scala | 28 --------------- .../org/apache/spark/util/UtilsSuite.scala | 30 ---------------- .../spark/util/collection/SorterSuite.scala | 3 +- .../expressions/codegen/CodeGenerator.scala | 4 +-- .../apache/spark/sql/types/DoubleType.scala | 2 +- .../apache/spark/sql/types/FloatType.scala | 2 +- .../org/apache/spark/sql/types/numerics.scala | 5 ++- 9 files changed, 7 insertions(+), 134 deletions(-) delete mode 100644 core/src/main/scala-2.12/org/apache/spark/util/OrderingUtil.scala delete mode 100644 core/src/main/scala-2.13/org/apache/spark/util/OrderingUtil.scala diff --git a/core/src/main/scala-2.12/org/apache/spark/util/OrderingUtil.scala b/core/src/main/scala-2.12/org/apache/spark/util/OrderingUtil.scala deleted file mode 100644 index 193ae36f3c089..0000000000000 --- a/core/src/main/scala-2.12/org/apache/spark/util/OrderingUtil.scala +++ /dev/null @@ -1,33 +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.util - -/** - * This class only exists to bridge the difference between Scala 2.12 and Scala 2.13's - * support for floating-point ordering. It is implemented separately for both as there - * is no method that exists in both for comparison. - * - * It functions like Ordering.Double in Scala 2.12. - */ -private[spark] object OrderingUtil { - - def compareDouble(x: Double, y: Double): Int = Ordering.Double.compare(x, y) - - def compareFloat(x: Float, y: Float): Int = Ordering.Float.compare(x, y) - -} diff --git a/core/src/main/scala-2.13/org/apache/spark/util/OrderingUtil.scala b/core/src/main/scala-2.13/org/apache/spark/util/OrderingUtil.scala deleted file mode 100644 index e861506672706..0000000000000 --- a/core/src/main/scala-2.13/org/apache/spark/util/OrderingUtil.scala +++ /dev/null @@ -1,34 +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.util - -/** - * This class only exists to bridge the difference between Scala 2.12 and Scala 2.13's - * support for floating-point ordering. It is implemented separately for both as there - * is no method that exists in both for comparison. - * - * It functions like Ordering.Double.TotalOrdering in Scala 2.13, which matches java.lang.Double - * rather than Scala 2.12's Ordering.Double in handling of NaN. - */ -private[spark] object OrderingUtil { - - def compareDouble(x: Double, y: Double): Int = Ordering.Double.TotalOrdering.compare(x, y) - - def compareFloat(x: Float, y: Float): Int = Ordering.Float.TotalOrdering.compare(x, y) - -} 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 723fbdf73f8d6..317d4b6407420 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1744,34 +1744,6 @@ private[spark] object Utils extends Logging { hashAbs } - /** - * NaN-safe version of `java.lang.Double.compare()` which allows NaN values to be compared - * according to semantics where NaN == NaN and NaN is greater than any non-NaN double. - */ - def nanSafeCompareDoubles(x: Double, y: Double): Int = { - val xIsNan: Boolean = java.lang.Double.isNaN(x) - val yIsNan: Boolean = java.lang.Double.isNaN(y) - if ((xIsNan && yIsNan) || (x == y)) 0 - else if (xIsNan) 1 - else if (yIsNan) -1 - else if (x > y) 1 - else -1 - } - - /** - * NaN-safe version of `java.lang.Float.compare()` which allows NaN values to be compared - * according to semantics where NaN == NaN and NaN is greater than any non-NaN float. - */ - def nanSafeCompareFloats(x: Float, y: Float): Int = { - val xIsNan: Boolean = java.lang.Float.isNaN(x) - val yIsNan: Boolean = java.lang.Float.isNaN(y) - if ((xIsNan && yIsNan) || (x == y)) 0 - else if (xIsNan) 1 - else if (yIsNan) -1 - else if (x > y) 1 - else -1 - } - /** * Returns the system properties map that is thread-safe to iterator over. It gets the * properties which have been set explicitly, as well as those for which only a default value 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 7e4a40b60aac5..8f8902e497d49 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -849,36 +849,6 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(buffer.toString === "st circular test circular") } - test("nanSafeCompareDoubles") { - def shouldMatchDefaultOrder(a: Double, b: Double): Unit = { - assert(Utils.nanSafeCompareDoubles(a, b) === JDouble.compare(a, b)) - assert(Utils.nanSafeCompareDoubles(b, a) === JDouble.compare(b, a)) - } - shouldMatchDefaultOrder(0d, 0d) - shouldMatchDefaultOrder(0d, 1d) - shouldMatchDefaultOrder(Double.MinValue, Double.MaxValue) - assert(Utils.nanSafeCompareDoubles(Double.NaN, Double.NaN) === 0) - assert(Utils.nanSafeCompareDoubles(Double.NaN, Double.PositiveInfinity) === 1) - assert(Utils.nanSafeCompareDoubles(Double.NaN, Double.NegativeInfinity) === 1) - assert(Utils.nanSafeCompareDoubles(Double.PositiveInfinity, Double.NaN) === -1) - assert(Utils.nanSafeCompareDoubles(Double.NegativeInfinity, Double.NaN) === -1) - } - - test("nanSafeCompareFloats") { - def shouldMatchDefaultOrder(a: Float, b: Float): Unit = { - assert(Utils.nanSafeCompareFloats(a, b) === JFloat.compare(a, b)) - assert(Utils.nanSafeCompareFloats(b, a) === JFloat.compare(b, a)) - } - shouldMatchDefaultOrder(0f, 0f) - shouldMatchDefaultOrder(1f, 1f) - shouldMatchDefaultOrder(Float.MinValue, Float.MaxValue) - assert(Utils.nanSafeCompareFloats(Float.NaN, Float.NaN) === 0) - assert(Utils.nanSafeCompareFloats(Float.NaN, Float.PositiveInfinity) === 1) - assert(Utils.nanSafeCompareFloats(Float.NaN, Float.NegativeInfinity) === 1) - assert(Utils.nanSafeCompareFloats(Float.PositiveInfinity, Float.NaN) === -1) - assert(Utils.nanSafeCompareFloats(Float.NegativeInfinity, Float.NaN) === -1) - } - test("isDynamicAllocationEnabled") { val conf = new SparkConf() conf.set("spark.master", "yarn") diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index bd1921d7b770e..bb03f0d3cdc20 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -23,7 +23,6 @@ import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging -import org.apache.spark.util.OrderingUtil import org.apache.spark.util.Utils.timeIt import org.apache.spark.util.random.XORShiftRandom @@ -60,7 +59,7 @@ class SorterSuite extends SparkFunSuite with Logging { Arrays.sort(keys) new Sorter(new KVArraySortDataFormat[Double, Number]) - .sort(keyValueArray, 0, keys.length, OrderingUtil.compareDouble) + .sort(keyValueArray, 0, keys.length, (x, y) => java.lang.Double.compare(x, y)) keys.zipWithIndex.foreach { case (k, i) => assert(k === keyValueArray(2 * i)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 7b8e92d665d0d..88f3c9e92d644 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -625,8 +625,8 @@ class CodegenContext extends Logging { def genComp(dataType: DataType, c1: String, c2: String): String = dataType match { // java boolean doesn't support > or < operator case BooleanType => s"($c1 == $c2 ? 0 : ($c1 ? 1 : -1))" - case DoubleType => s"org.apache.spark.util.Utils.nanSafeCompareDoubles($c1, $c2)" - case FloatType => s"org.apache.spark.util.Utils.nanSafeCompareFloats($c1, $c2)" + case DoubleType => s"java.lang.Double.compare($c1, $c2)" + case FloatType => s"java.lang.Float.compare($c1, $c2)" // use c1 - c2 may overflow case dt: DataType if isPrimitiveType(dt) => s"($c1 > $c2 ? 1 : $c1 < $c2 ? -1 : 0)" case BinaryType => s"org.apache.spark.sql.catalyst.util.TypeUtils.compareBinary($c1, $c2)" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala index ba0fe34a8c283..8a5cf63b12ed7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala @@ -39,7 +39,7 @@ class DoubleType private() extends FractionalType { private[sql] val numeric = implicitly[Numeric[Double]] private[sql] val fractional = implicitly[Fractional[Double]] private[sql] val ordering = - (x: Double, y: Double) => Utils.nanSafeCompareDoubles(x, y) + (x: Double, y: Double) => java.lang.Double.compare(x, y) private[sql] val asIntegral = DoubleAsIfIntegral override private[sql] def exactNumeric = DoubleExactNumeric diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala index 9bd216ceb6f30..738291eaf7762 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -39,7 +39,7 @@ class FloatType private() extends FractionalType { private[sql] val numeric = implicitly[Numeric[Float]] private[sql] val fractional = implicitly[Fractional[Float]] private[sql] val ordering = - (x: Float, y: Float) => Utils.nanSafeCompareFloats(x, y) + (x: Float, y: Float) => java.lang.Float.compare(x, y) private[sql] val asIntegral = FloatAsIfIntegral override private[sql] def exactNumeric = FloatExactNumeric diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala index 71fed3e69b651..1ac85360f944f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala @@ -21,7 +21,6 @@ import scala.math.Numeric._ import scala.math.Ordering import org.apache.spark.sql.types.Decimal.DecimalIsConflicted -import org.apache.spark.util.OrderingUtil object ByteExactNumeric extends ByteIsIntegral with Ordering.ByteOrdering { private def checkOverflow(res: Int, x: Byte, y: Byte, op: String): Unit = { @@ -149,7 +148,7 @@ object FloatExactNumeric extends FloatIsFractional { } } - override def compare(x: Float, y: Float): Int = OrderingUtil.compareFloat(x, y) + override def compare(x: Float, y: Float): Int = java.lang.Float.compare(x, y) } object DoubleExactNumeric extends DoubleIsFractional { @@ -177,7 +176,7 @@ object DoubleExactNumeric extends DoubleIsFractional { } } - override def compare(x: Double, y: Double): Int = OrderingUtil.compareDouble(x, y) + override def compare(x: Double, y: Double): Int = java.lang.Double.compare(x, y) } object DecimalExactNumeric extends DecimalIsConflicted { From 0bd8b995d6bad9e4fe26975fc81cff24c56ba9e6 Mon Sep 17 00:00:00 2001 From: Aman Omer Date: Thu, 5 Dec 2019 15:28:07 +0800 Subject: [PATCH 0795/1387] [SPARK-30093][SQL] Improve error message for creating view ### What changes were proposed in this pull request? Improved error message while creating views. ### Why are the changes needed? Error message should suggest user to use TEMPORARY keyword while creating permanent view referred by temporary view. https://github.com/apache/spark/pull/26317#discussion_r352377363 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Updated test case. Closes #26731 from amanomer/imp_err_msg. Authored-by: Aman Omer Signed-off-by: Wenchen Fan --- .../spark/sql/execution/command/views.scala | 3 ++- .../results/postgreSQL/create_view.sql.out | 26 +++++++++---------- .../spark/sql/execution/SQLViewSuite.scala | 3 ++- 3 files changed, 17 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index a8dbd57e42700..2cc462fa6c1bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -162,7 +162,8 @@ case class CreateViewCommand( if sparkSession.sessionState.catalog.isTemporaryTable(ident) => // temporary views are only stored in the session catalog throw new AnalysisException(s"Not allowed to create a permanent view $name by " + - s"referencing a temporary view $ident") + s"referencing a temporary view $ident. " + + "Please create a temp view instead by CREATE TEMP VIEW") case other if !other.resolved => other.expressions.flatMap(_.collect { // Traverse subquery plan for any unresolved relations. case e: SubqueryExpression => verify(e.plan) diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index cd4d3e64b8da1..cd79e4253aac9 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -267,7 +267,7 @@ CREATE VIEW v1_temp AS SELECT * FROM temp_table struct<> -- !query 27 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `v1_temp` by referencing a temporary view `temp_table`; +Not allowed to create a permanent view `v1_temp` by referencing a temporary view `temp_table`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 28 @@ -323,7 +323,7 @@ CREATE VIEW temp_view_test.v3_temp AS SELECT * FROM temp_table struct<> -- !query 32 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `temp_view_test`.`v3_temp` by referencing a temporary view `temp_table`; +Not allowed to create a permanent view `temp_view_test`.`v3_temp` by referencing a temporary view `temp_table`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 33 @@ -372,7 +372,7 @@ CREATE VIEW v4_temp AS struct<> -- !query 35 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `v4_temp` by referencing a temporary view `temp_table`; +Not allowed to create a permanent view `v4_temp` by referencing a temporary view `temp_table`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 36 @@ -384,7 +384,7 @@ CREATE VIEW v5_temp AS struct<> -- !query 36 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `v5_temp` by referencing a temporary view `temp_table`; +Not allowed to create a permanent view `v5_temp` by referencing a temporary view `temp_table`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 37 @@ -543,7 +543,7 @@ CREATE VIEW v6_temp AS SELECT * FROM base_table WHERE id IN (SELECT id FROM temp struct<> -- !query 47 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `v6_temp` by referencing a temporary view `temp_table`; +Not allowed to create a permanent view `v6_temp` by referencing a temporary view `temp_table`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 48 @@ -552,7 +552,7 @@ CREATE VIEW v7_temp AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM tem struct<> -- !query 48 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `v7_temp` by referencing a temporary view `temp_table`; +Not allowed to create a permanent view `v7_temp` by referencing a temporary view `temp_table`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 49 @@ -561,7 +561,7 @@ CREATE VIEW v8_temp AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM temp struct<> -- !query 49 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `v8_temp` by referencing a temporary view `temp_table`; +Not allowed to create a permanent view `v8_temp` by referencing a temporary view `temp_table`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 50 @@ -570,7 +570,7 @@ CREATE VIEW v9_temp AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM struct<> -- !query 50 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `v9_temp` by referencing a temporary view `temp_table`; +Not allowed to create a permanent view `v9_temp` by referencing a temporary view `temp_table`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 51 @@ -679,7 +679,7 @@ CREATE VIEW temporal1 AS SELECT * FROM t1 CROSS JOIN tt struct<> -- !query 61 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `temporal1` by referencing a temporary view `tt`; +Not allowed to create a permanent view `temporal1` by referencing a temporary view `tt`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 62 @@ -720,7 +720,7 @@ CREATE VIEW temporal2 AS SELECT * FROM t1 INNER JOIN tt ON t1.num = tt.num2 struct<> -- !query 64 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `temporal2` by referencing a temporary view `tt`; +Not allowed to create a permanent view `temporal2` by referencing a temporary view `tt`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 65 @@ -761,7 +761,7 @@ CREATE VIEW temporal3 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 struct<> -- !query 67 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `temporal3` by referencing a temporary view `tt`; +Not allowed to create a permanent view `temporal3` by referencing a temporary view `tt`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 68 @@ -802,7 +802,7 @@ CREATE VIEW temporal4 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 AND t struct<> -- !query 70 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `temporal4` by referencing a temporary view `tt`; +Not allowed to create a permanent view `temporal4` by referencing a temporary view `tt`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 71 @@ -811,7 +811,7 @@ CREATE VIEW temporal5 AS SELECT * FROM t1 WHERE num IN (SELECT num FROM t1 WHERE struct<> -- !query 71 output org.apache.spark.sql.AnalysisException -Not allowed to create a permanent view `temporal5` by referencing a temporary view `tt`; +Not allowed to create a permanent view `temporal5` by referencing a temporary view `tt`. Please create a temp view instead by CREATE TEMP VIEW; -- !query 72 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 918e1960dbd55..84ab8d8c2358f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -80,7 +80,8 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql("CREATE VIEW jtv1 AS SELECT * FROM temp_jtv1 WHERE id < 6") }.getMessage assert(e.contains("Not allowed to create a permanent view `jtv1` by " + - "referencing a temporary view `temp_jtv1`")) + "referencing a temporary view `temp_jtv1`. " + + "Please create a temp view instead by CREATE TEMP VIEW")) val globalTempDB = spark.sharedState.globalTempViewManager.database sql("CREATE GLOBAL TEMP VIEW global_temp_jtv1 AS SELECT * FROM jt WHERE id > 0") From 0ab922c1ebed12b3edb50f5d9669e3c823bdcdc9 Mon Sep 17 00:00:00 2001 From: turbofei Date: Thu, 5 Dec 2019 16:00:16 +0800 Subject: [PATCH 0796/1387] [SPARK-29860][SQL] Fix dataType mismatch issue for InSubquery ### What changes were proposed in this pull request? There is an issue for InSubquery expression. For example, there are two tables `ta` and `tb` created by the below statements. ``` sql("create table ta(id Decimal(18,0)) using parquet") sql("create table tb(id Decimal(19,0)) using parquet") ``` This statement below would thrown dataType mismatch exception. ``` sql("select * from ta where id in (select id from tb)").show() ``` However, this similar statement could execute successfully. ``` sql("select * from ta where id in ((select id from tb))").show() ``` The root cause is that, for `InSubquery` expression, it does not find a common type for two decimalType like `In` expression. Besides that, for `InSubquery` expression, it also does not find a common type for DecimalType and double/float/bigInt. In this PR, I fix this issue by finding widerType for `InSubquery` expression when DecimalType is involved. ### Why are the changes needed? Some InSubquery would throw dataType mismatch exception. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test. Closes #26485 from turboFei/SPARK-29860-in-subquery. Authored-by: turbofei Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/TypeCoercion.scala | 3 +-- .../negative-cases/subq-input-typecheck.sql | 2 +- .../subq-input-typecheck.sql.out | 6 +++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 21 +++++++++++++++++++ 4 files changed, 26 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 83c76c2d4e2bc..162f48d420b32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -470,8 +470,7 @@ object TypeCoercion { val rhs = sub.output val commonTypes = lhs.zip(rhs).flatMap { case (l, r) => - findCommonTypeForBinaryComparison(l.dataType, r.dataType, conf) - .orElse(findTightestCommonType(l.dataType, r.dataType)) + findWiderTypeForTwo(l.dataType, r.dataType) } // The number of columns/expressions must match between LHS and RHS of an diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql index 95b115a8dd094..98ce1354a1355 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql @@ -18,7 +18,7 @@ CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES AS t1(t4a, t4b, t4c); CREATE TEMPORARY VIEW t5 AS SELECT * FROM VALUES - (CAST(1 AS DECIMAL(18, 0)), CAST(2 AS STRING), CAST(3 AS BIGINT)) + (CAST('2011-01-01 01:01:01' AS TIMESTAMP), CAST(2 AS STRING), CAST(3 AS BIGINT)) AS t1(t5a, t5b, t5c); -- TC 01.01 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out index dcd30055bca19..ea7585716de83 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -44,7 +44,7 @@ struct<> -- !query 4 CREATE TEMPORARY VIEW t5 AS SELECT * FROM VALUES - (CAST(1 AS DECIMAL(18, 0)), CAST(2 AS STRING), CAST(3 AS BIGINT)) + (CAST('2011-01-01 01:01:01' AS TIMESTAMP), CAST(2 AS STRING), CAST(3 AS BIGINT)) AS t1(t5a, t5b, t5c) -- !query 4 schema struct<> @@ -139,8 +139,8 @@ cannot resolve '(named_struct('t4a', t4.`t4a`, 't4b', t4.`t4b`, 't4c', t4.`t4c`) The data type of one or more elements in the left hand side of an IN subquery is not compatible with the data type of the output of the subquery Mismatched columns: -[(t4.`t4a`:double, t5.`t5a`:decimal(18,0)), (t4.`t4c`:string, t5.`t5c`:bigint)] +[(t4.`t4a`:double, t5.`t5a`:timestamp), (t4.`t4c`:string, t5.`t5c`:bigint)] Left side: [double, string, string]. Right side: -[decimal(18,0), string, bigint].; +[timestamp, string, bigint].; 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 5b0de1baa553b..37d98f7c87420 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 @@ -3313,6 +3313,27 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { cubeDF.join(cubeDF, "nums"), Row(1, 0, 0) :: Row(2, 0, 0) :: Row(3, 0, 0) :: Nil) } + + test("SPARK-29860: Fix dataType mismatch issue for InSubquery") { + withTempView("ta", "tb", "tc", "td", "te", "tf") { + sql("CREATE TEMPORARY VIEW ta AS SELECT * FROM VALUES(CAST(1 AS DECIMAL(8, 0))) AS ta(id)") + sql("CREATE TEMPORARY VIEW tb AS SELECT * FROM VALUES(CAST(1 AS DECIMAL(7, 2))) AS tb(id)") + sql("CREATE TEMPORARY VIEW tc AS SELECT * FROM VALUES(CAST(1 AS DOUBLE)) AS tc(id)") + sql("CREATE TEMPORARY VIEW td AS SELECT * FROM VALUES(CAST(1 AS FLOAT)) AS td(id)") + sql("CREATE TEMPORARY VIEW te AS SELECT * FROM VALUES(CAST(1 AS BIGINT)) AS te(id)") + sql("CREATE TEMPORARY VIEW tf AS SELECT * FROM VALUES(CAST(1 AS DECIMAL(38, 38))) AS tf(id)") + val df1 = sql("SELECT id FROM ta WHERE id IN (SELECT id FROM tb)") + checkAnswer(df1, Row(new java.math.BigDecimal(1))) + val df2 = sql("SELECT id FROM ta WHERE id IN (SELECT id FROM tc)") + checkAnswer(df2, Row(new java.math.BigDecimal(1))) + val df3 = sql("SELECT id FROM ta WHERE id IN (SELECT id FROM td)") + checkAnswer(df3, Row(new java.math.BigDecimal(1))) + val df4 = sql("SELECT id FROM ta WHERE id IN (SELECT id FROM te)") + checkAnswer(df4, Row(new java.math.BigDecimal(1))) + val df5 = sql("SELECT id FROM ta WHERE id IN (SELECT id FROM tf)") + checkAnswer(df5, Array.empty[Row]) + } + } } case class Foo(bar: Option[String]) From 332e252a1448a27cfcfc1d1d794f7979e6cd331a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 5 Dec 2019 16:14:27 +0800 Subject: [PATCH 0797/1387] [SPARK-29425][SQL] The ownership of a database should be respected ### What changes were proposed in this pull request? Keep the owner of a database when executing alter database commands ### Why are the changes needed? Spark will inadvertently delete the owner of a database for executing databases ddls ### Does this PR introduce any user-facing change? NO ### How was this patch tested? add and modify uts Closes #26080 from yaooqinn/SPARK-29425. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../connector/catalog/SupportsNamespaces.java | 10 ++++ .../catalog/ExternalCatalogSuite.scala | 6 +- .../catalog/SessionCatalogSuite.scala | 8 ++- .../spark/sql/execution/command/ddl.scala | 16 ++++-- .../sql/execution/command/DDLSuite.scala | 19 ++++--- .../sql/hive/client/HiveClientImpl.scala | 43 ++++++++------ .../spark/sql/hive/client/HiveShim.scala | 57 ++++++++++++++++++- .../spark/sql/hive/client/VersionsSuite.scala | 29 ++++++++++ .../sql/hive/execution/HiveDDLSuite.scala | 40 ++++++++++++- 9 files changed, 191 insertions(+), 37 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java index 171777777dbac..2e60487287f2d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java @@ -53,6 +53,16 @@ public interface SupportsNamespaces extends CatalogPlugin { */ String PROP_COMMENT = "comment"; + /** + * A property to specify the owner of the namespace. + */ + String PROP_OWNER_NAME = "ownerName"; + + /** + * A property to specify the type of the namespace's owner. + */ + String PROP_OWNER_TYPE = "ownerType"; + /** * The list of reserved namespace properties. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 6b1c35094e4a4..98a53b59fa031 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, N import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -143,8 +144,9 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac // Note: alter properties here because Hive does not support altering other fields catalog.alterDatabase(db1.copy(properties = Map("k" -> "v3", "good" -> "true"))) val newDb1 = catalog.getDatabase("db1") - assert(db1.properties.isEmpty) - assert(newDb1.properties.size == 2) + val reversedProperties = Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE) + assert((db1.properties -- reversedProperties).isEmpty) + assert((newDb1.properties -- reversedProperties).size == 2) assert(newDb1.properties.get("k") == Some("v3")) assert(newDb1.properties.get("good") == Some("true")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index f334ba5690af0..a21aaa24297d3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{AliasIdentifier, FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View} +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -207,8 +208,9 @@ abstract class SessionCatalogSuite extends AnalysisTest { // Note: alter properties here because Hive does not support altering other fields catalog.alterDatabase(db1.copy(properties = Map("k" -> "v3", "good" -> "true"))) val newDb1 = catalog.getDatabaseMetadata("db1") - assert(db1.properties.isEmpty) - assert(newDb1.properties.size == 2) + val reversedProperties = Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE) + assert((db1.properties -- reversedProperties).isEmpty) + assert((newDb1.properties -- reversedProperties).size == 2) assert(newDb1.properties.get("k") == Some("v3")) assert(newDb1.properties.get("good") == Some("true")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 3645d38b3b558..bdba10eb488d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -21,6 +21,7 @@ import java.util.Locale import java.util.concurrent.TimeUnit._ import scala.collection.{GenMap, GenSeq} +import scala.collection.JavaConverters._ import scala.collection.parallel.ForkJoinTaskSupport import scala.collection.parallel.immutable.ParVector import scala.util.control.NonFatal @@ -37,6 +38,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter @@ -172,19 +174,23 @@ case class DescribeDatabaseCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val dbMetadata: CatalogDatabase = sparkSession.sessionState.catalog.getDatabaseMetadata(databaseName) + val allDbProperties = dbMetadata.properties val result = Row("Database Name", dbMetadata.name) :: Row("Description", dbMetadata.description) :: - Row("Location", CatalogUtils.URIToString(dbMetadata.locationUri)) :: Nil + Row("Location", CatalogUtils.URIToString(dbMetadata.locationUri)):: + Row("Owner Name", allDbProperties.getOrElse(PROP_OWNER_NAME, "")) :: + Row("Owner Type", allDbProperties.getOrElse(PROP_OWNER_TYPE, "")) :: Nil if (extended) { - val properties = - if (dbMetadata.properties.isEmpty) { + val properties = allDbProperties -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE) + val propertiesStr = + if (properties.isEmpty) { "" } else { - dbMetadata.properties.toSeq.mkString("(", ", ", ")") + properties.toSeq.mkString("(", ", ", ")") } - result :+ Row("Properties", properties) + result :+ Row("Properties", propertiesStr) } else { result } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index f777fa44b5009..c762f25d62eed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -22,7 +22,6 @@ import java.net.URI import java.util.Locale import org.apache.hadoop.fs.Path -import org.scalatest.BeforeAndAfterEach import org.apache.spark.internal.config import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD @@ -31,6 +30,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -185,6 +185,8 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { abstract class DDLSuite extends QueryTest with SQLTestUtils { + protected val reversedProperties = Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE) + protected def isUsingHiveMetastore: Boolean = { spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive" } @@ -328,7 +330,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { try { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbName) - assert(db1 == CatalogDatabase( + assert(db1.copy(properties = db1.properties -- reversedProperties) == CatalogDatabase( dbName, "", getDBPath(dbName), @@ -351,7 +353,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"CREATE DATABASE $dbName Location '$path'") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) val expPath = makeQualifiedPath(tmpDir.toString) - assert(db1 == CatalogDatabase( + assert(db1.copy(properties = db1.properties -- reversedProperties) == CatalogDatabase( dbNameWithoutBackTicks, "", expPath, @@ -374,7 +376,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - assert(db1 == CatalogDatabase( + assert(db1.copy(properties = db1.properties -- reversedProperties) == CatalogDatabase( dbNameWithoutBackTicks, "", getDBPath(dbNameWithoutBackTicks), @@ -747,7 +749,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"CREATE DATABASE $dbName") checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") + .where("key not like 'Owner%'"), // filter for consistency with in-memory catalog Row("Database Name", dbNameWithoutBackTicks) :: Row("Description", "") :: Row("Location", CatalogUtils.URIToString(location)) :: @@ -756,7 +759,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") + .where("key not like 'Owner%'"), // filter for consistency with in-memory catalog Row("Database Name", dbNameWithoutBackTicks) :: Row("Description", "") :: Row("Location", CatalogUtils.URIToString(location)) :: @@ -765,7 +769,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") + .where("key not like 'Owner%'"), // filter for consistency with in-memory catalog Row("Database Name", dbNameWithoutBackTicks) :: Row("Description", "") :: Row("Location", CatalogUtils.URIToString(location)) :: 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 d4342e427192c..700c0884dd17c 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 @@ -32,8 +32,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.{IMetaStoreClient, TableType => HiveTableType} -import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, Table => MetaStoreApiTable} -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Order, SerDeInfo, StorageDescriptor} +import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, Table => MetaStoreApiTable, _} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC @@ -54,6 +53,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} +import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.HiveExternalCatalog.{DATASOURCE_SCHEMA, DATASOURCE_SCHEMA_NUMPARTS, DATASOURCE_SCHEMA_PART_PREFIX} @@ -355,13 +355,8 @@ private[hive] class HiveClientImpl( override def createDatabase( database: CatalogDatabase, ignoreIfExists: Boolean): Unit = withHiveState { - client.createDatabase( - new HiveDatabase( - database.name, - database.description, - CatalogUtils.URIToString(database.locationUri), - Option(database.properties).map(_.asJava).orNull), - ignoreIfExists) + val hiveDb = toHiveDatabase(database, true) + client.createDatabase(hiveDb, ignoreIfExists) } override def dropDatabase( @@ -379,22 +374,38 @@ private[hive] class HiveClientImpl( s"Hive ${version.fullVersion} does not support altering database location") } } - client.alterDatabase( + val hiveDb = toHiveDatabase(database, false) + client.alterDatabase(database.name, hiveDb) + } + + private def toHiveDatabase(database: CatalogDatabase, isCreate: Boolean): HiveDatabase = { + val props = database.properties + val hiveDb = new HiveDatabase( database.name, - new HiveDatabase( - database.name, - database.description, - CatalogUtils.URIToString(database.locationUri), - Option(database.properties).map(_.asJava).orNull)) + database.description, + CatalogUtils.URIToString(database.locationUri), + (props -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE)).asJava) + props.get(PROP_OWNER_NAME).orElse(if (isCreate) Some(userName) else None).foreach { ownerName => + shim.setDatabaseOwnerName(hiveDb, ownerName) + } + props.get(PROP_OWNER_TYPE).orElse(if (isCreate) Some(PrincipalType.USER.name) else None) + .foreach { ownerType => + shim.setDatabaseOwnerType(hiveDb, ownerType) + } + hiveDb } override def getDatabase(dbName: String): CatalogDatabase = withHiveState { Option(client.getDatabase(dbName)).map { d => + val paras = Option(d.getParameters).map(_.asScala.toMap).getOrElse(Map()) ++ + Map(PROP_OWNER_NAME -> shim.getDatabaseOwnerName(d), + PROP_OWNER_TYPE -> shim.getDatabaseOwnerType(d)) + CatalogDatabase( name = d.getName, description = Option(d.getDescription).getOrElse(""), locationUri = CatalogUtils.stringToURI(d.getLocationUri), - properties = Option(d.getParameters).map(_.asScala.toMap).orNull) + properties = paras) }.getOrElse(throw new NoSuchDatabaseException(dbName)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 586fbbefade46..01ddaf2ee8ac1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -29,8 +29,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.IMetaStoreClient -import org.apache.hadoop.hive.metastore.api.{EnvironmentContext, Function => HiveFunction, FunctionType} -import org.apache.hadoop.hive.metastore.api.{MetaException, PrincipalType, ResourceType, ResourceUri} +import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, MetaException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.io.AcidUtils import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} @@ -154,6 +153,14 @@ private[client] sealed abstract class Shim { deleteData: Boolean, purge: Boolean): Unit + def getDatabaseOwnerName(db: Database): String + + def setDatabaseOwnerName(db: Database, owner: String): Unit + + def getDatabaseOwnerType(db: Database): String + + def setDatabaseOwnerType(db: Database, ownerType: String): Unit + protected def findStaticMethod(klass: Class[_], name: String, args: Class[_]*): Method = { val method = findMethod(klass, name, args: _*) require(Modifier.isStatic(method.getModifiers()), @@ -456,6 +463,14 @@ private[client] class Shim_v0_12 extends Shim with Logging { def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { Seq.empty[String] } + + override def getDatabaseOwnerName(db: Database): String = "" + + override def setDatabaseOwnerName(db: Database, owner: String): Unit = {} + + override def getDatabaseOwnerType(db: Database): String = "" + + override def setDatabaseOwnerType(db: Database, ownerType: String): Unit = {} } private[client] class Shim_v0_13 extends Shim_v0_12 { @@ -493,6 +508,28 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { "getResults", classOf[JList[Object]]) + private lazy val getDatabaseOwnerNameMethod = + findMethod( + classOf[Database], + "getOwnerName") + + private lazy val setDatabaseOwnerNameMethod = + findMethod( + classOf[Database], + "setOwnerName", + classOf[String]) + + private lazy val getDatabaseOwnerTypeMethod = + findMethod( + classOf[Database], + "getOwnerType") + + private lazy val setDatabaseOwnerTypeMethod = + findMethod( + classOf[Database], + "setOwnerType", + classOf[PrincipalType]) + override def setCurrentSessionState(state: SessionState): Unit = setCurrentSessionStateMethod.invoke(null, state) @@ -809,6 +846,22 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } } + override def getDatabaseOwnerName(db: Database): String = { + Option(getDatabaseOwnerNameMethod.invoke(db)).map(_.asInstanceOf[String]).getOrElse("") + } + + override def setDatabaseOwnerName(db: Database, owner: String): Unit = { + setDatabaseOwnerNameMethod.invoke(db, owner) + } + + override def getDatabaseOwnerType(db: Database): String = { + Option(getDatabaseOwnerTypeMethod.invoke(db)) + .map(_.asInstanceOf[PrincipalType].name()).getOrElse("") + } + + override def setDatabaseOwnerType(db: Database, ownerType: String): Unit = { + setDatabaseOwnerTypeMethod.invoke(db, PrincipalType.valueOf(ownerType)) + } } private[client] class Shim_v0_14 extends Shim_v0_13 { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index e3797041883ac..e07978ae180e4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.mapred.TextInputFormat +import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging @@ -170,6 +171,34 @@ class VersionsSuite extends SparkFunSuite with Logging { client.createDatabase(tempDB, ignoreIfExists = true) } + test(s"$version: create/get/alter database should pick right user name as owner") { + if (version != "0.12") { + val currentUser = UserGroupInformation.getCurrentUser.getUserName + val ownerName = "SPARK_29425" + val db1 = "SPARK_29425_1" + val db2 = "SPARK_29425_2" + val ownerProps = Map("ownerName" -> ownerName) + + // create database with owner + val dbWithOwner = CatalogDatabase(db1, "desc", Utils.createTempDir().toURI, ownerProps) + client.createDatabase(dbWithOwner, ignoreIfExists = true) + val getDbWithOwner = client.getDatabase(db1) + assert(getDbWithOwner.properties("ownerName") === ownerName) + // alter database without owner + client.alterDatabase(getDbWithOwner.copy(properties = Map())) + assert(client.getDatabase(db1).properties("ownerName") === "") + + // create database without owner + val dbWithoutOwner = CatalogDatabase(db2, "desc", Utils.createTempDir().toURI, Map()) + client.createDatabase(dbWithoutOwner, ignoreIfExists = true) + val getDbWithoutOwner = client.getDatabase(db2) + assert(getDbWithoutOwner.properties("ownerName") === currentUser) + // alter database with owner + client.alterDatabase(getDbWithoutOwner.copy(properties = ownerProps)) + assert(client.getDatabase(db2).properties("ownerName") === ownerName) + } + } + test(s"$version: createDatabase with null description") { withTempDir { tmpDir => val dbWithNullDesc = 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 56f424d9784e7..0684d66558c61 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 @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.HiveExternalCatalog @@ -372,12 +373,45 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA assert(table.provider == Some("org.apache.spark.sql.hive.orc")) } } + + private def checkOwner(db: String, expected: String): Unit = { + val owner = sql(s"DESCRIBE DATABASE EXTENDED $db") + .where("database_description_item='Owner Name'") + .collect().head.getString(1) + assert(owner === expected) + } + + test("Database Ownership") { + val catalog = spark.sessionState.catalog + try { + val db1 = "spark_29425_1" + val db2 = "spark_29425_2" + val owner = "spark_29425" + + sql(s"CREATE DATABASE $db1") + checkOwner(db1, Utils.getCurrentUserName()) + sql(s"ALTER DATABASE $db1 SET DBPROPERTIES ('a'='a')") + checkOwner(db1, Utils.getCurrentUserName()) + + // TODO: Specify ownership should be forbidden after we implement `SET OWNER` syntax + sql(s"CREATE DATABASE $db2 WITH DBPROPERTIES('ownerName'='$owner')") + checkOwner(db2, owner) + sql(s"ALTER DATABASE $db2 SET DBPROPERTIES ('a'='a')") + checkOwner(db2, owner) + // TODO: Changing ownership should be forbidden after we implement `SET OWNER` syntax + sql(s"ALTER DATABASE $db2 SET DBPROPERTIES ('ownerName'='a')") + checkOwner(db2, "a") + } finally { + catalog.reset() + } + } } class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { import testImplicits._ val hiveFormats = Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO") + private val reversedProperties = Seq("ownerName", "ownerType") override def afterEach(): Unit = { try { @@ -1112,7 +1146,8 @@ class HiveDDLSuite sql(s"CREATE DATABASE $dbName Location '${tmpDir.toURI.getPath.stripSuffix("/")}'") val db1 = catalog.getDatabaseMetadata(dbName) val dbPath = new URI(tmpDir.toURI.toString.stripSuffix("/")) - assert(db1 == CatalogDatabase(dbName, "", dbPath, Map.empty)) + assert(db1.copy(properties = db1.properties -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE)) === + CatalogDatabase(dbName, "", dbPath, Map.empty)) sql("USE db1") sql(s"CREATE TABLE $tabName as SELECT 1") @@ -1150,7 +1185,8 @@ class HiveDDLSuite val expectedDBLocation = s"file:${dbPath.toUri.getPath.stripSuffix("/")}/$dbName.db" val expectedDBUri = CatalogUtils.stringToURI(expectedDBLocation) val db1 = catalog.getDatabaseMetadata(dbName) - assert(db1 == CatalogDatabase( + assert(db1.copy(properties = db1.properties -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE)) == + CatalogDatabase( dbName, "", expectedDBUri, From b9cae37750480ed9c00f46a592cd3910b58b081f Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 5 Dec 2019 22:03:44 +0800 Subject: [PATCH 0798/1387] [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres # What changes were proposed in this pull request? Add an analyzer rule to convert unresolved `Add`, `Subtract`, etc. to `TimeAdd`, `DateAdd`, etc. according to the following policy: ```scala /** * For [[Add]]: * 1. if both side are interval, stays the same; * 2. else if one side is interval, turns it to [[TimeAdd]]; * 3. else if one side is date, turns it to [[DateAdd]] ; * 4. else stays the same. * * For [[Subtract]]: * 1. if both side are interval, stays the same; * 2. else if the right side is an interval, turns it to [[TimeSub]]; * 3. else if one side is timestamp, turns it to [[SubtractTimestamps]]; * 4. else if the right side is date, turns it to [[DateDiff]]/[[SubtractDates]]; * 5. else if the left side is date, turns it to [[DateSub]]; * 6. else turns it to stays the same. * * For [[Multiply]]: * 1. If one side is interval, turns it to [[MultiplyInterval]]; * 2. otherwise, stays the same. * * For [[Divide]]: * 1. If the left side is interval, turns it to [[DivideInterval]]; * 2. otherwise, stays the same. */ ``` Besides, we change datetime functions from implicit cast types to strict ones, all available type coercions happen in `DateTimeOperations` coercion rule. ### Why are the changes needed? Feature Parity between PostgreSQL and Spark, and make the null semantic consistent with Spark. ### Does this PR introduce any user-facing change? 1. date_add/date_sub functions only accept int/tinynit/smallint as the second arg, double/string etc, are forbidden like hive, which produce weird results. ### How was this patch tested? add ut Closes #26412 from yaooqinn/SPARK-29774. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 60 ++++ .../sql/catalyst/analysis/TypeCoercion.scala | 61 +--- .../expressions/datetimeExpressions.scala | 20 +- .../catalyst/analysis/TypeCoercionSuite.scala | 59 ---- .../expressions/DateExpressionsSuite.scala | 16 + .../resources/sql-tests/inputs/datetime.sql | 48 ++- .../sql-tests/results/datetime.sql.out | 280 ++++++++++++++++-- .../native/dateTimeOperations.sql.out | 54 ++-- .../native/decimalPrecision.sql.out | 48 +-- .../native/promoteStrings.sql.out | 15 +- 10 files changed, 457 insertions(+), 204 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 83fa405e521ce..ee1bb9a4c9d98 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -228,6 +228,7 @@ class Analyzer( ResolveLambdaVariables(conf) :: ResolveTimeZone(conf) :: ResolveRandomSeed :: + ResolveBinaryArithmetic(conf) :: TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), Batch("PostgreSQL Dialect", Once, PostgreSQLDialect.postgreSQLDialectRules: _*), @@ -246,6 +247,65 @@ class Analyzer( CleanupAliases) ) + /** + * For [[Add]]: + * 1. if both side are interval, stays the same; + * 2. else if one side is interval, turns it to [[TimeAdd]]; + * 3. else if one side is date, turns it to [[DateAdd]] ; + * 4. else stays the same. + * + * For [[Subtract]]: + * 1. if both side are interval, stays the same; + * 2. else if the right side is an interval, turns it to [[TimeSub]]; + * 3. else if one side is timestamp, turns it to [[SubtractTimestamps]]; + * 4. else if the right side is date, turns it to [[DateDiff]]/[[SubtractDates]]; + * 5. else if the left side is date, turns it to [[DateSub]]; + * 6. else turns it to stays the same. + * + * For [[Multiply]]: + * 1. If one side is interval, turns it to [[MultiplyInterval]]; + * 2. otherwise, stays the same. + * + * For [[Divide]]: + * 1. If the left side is interval, turns it to [[DivideInterval]]; + * 2. otherwise, stays the same. + */ + case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case p: LogicalPlan => p.transformExpressionsUp { + case a @ Add(l, r) if a.childrenResolved => (l.dataType, r.dataType) match { + case (CalendarIntervalType, CalendarIntervalType) => a + case (_, CalendarIntervalType) => Cast(TimeAdd(l, r), l.dataType) + case (CalendarIntervalType, _) => Cast(TimeAdd(r, l), r.dataType) + case (DateType, _) => DateAdd(l, r) + case (_, DateType) => DateAdd(r, l) + case _ => a + } + case s @ Subtract(l, r) if s.childrenResolved => (l.dataType, r.dataType) match { + case (CalendarIntervalType, CalendarIntervalType) => s + case (_, CalendarIntervalType) => Cast(TimeSub(l, r), l.dataType) + case (TimestampType, _) => SubtractTimestamps(l, r) + case (_, TimestampType) => SubtractTimestamps(l, r) + case (_, DateType) => if (conf.usePostgreSQLDialect) { + DateDiff(l, r) + } else { + SubtractDates(l, r) + } + case (DateType, _) => DateSub(l, r) + case _ => s + } + case m @ Multiply(l, r) if m.childrenResolved => (l.dataType, r.dataType) match { + case (CalendarIntervalType, _) => MultiplyInterval(l, r) + case (_, CalendarIntervalType) => MultiplyInterval(r, l) + case _ => m + } + case d @ Divide(l, r) if d.childrenResolved => (l.dataType, r.dataType) match { + case (CalendarIntervalType, _) => DivideInterval(l, r) + case _ => d + } + } + } + } /** * Substitute child plan with WindowSpecDefinitions. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 162f48d420b32..e76193fd94222 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -821,52 +821,24 @@ object TypeCoercion { } } - /** - * 1. Turns Add/Subtract of DateType/TimestampType/StringType and CalendarIntervalType - * to TimeAdd/TimeSub. - * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType - * and TimestampType/IntegerType/DateType to DateAdd/DateSub/SubtractDates and - * to SubtractTimestamps. - * 3. Turns Multiply/Divide of CalendarIntervalType and NumericType - * to MultiplyInterval/DivideInterval - */ object DateTimeOperations extends Rule[LogicalPlan] { - - private val acceptedTypes = Seq(DateType, TimestampType, StringType) - - def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e - - case Add(l @ CalendarIntervalType(), r) if acceptedTypes.contains(r.dataType) => - Cast(TimeAdd(r, l), r.dataType) - case Add(l, r @ CalendarIntervalType()) if acceptedTypes.contains(l.dataType) => - Cast(TimeAdd(l, r), l.dataType) - case Subtract(l, r @ CalendarIntervalType()) if acceptedTypes.contains(l.dataType) => - Cast(TimeSub(l, r), l.dataType) - case Multiply(l @ CalendarIntervalType(), r @ NumericType()) => - MultiplyInterval(l, r) - case Multiply(l @ NumericType(), r @ CalendarIntervalType()) => - MultiplyInterval(r, l) - case Divide(l @ CalendarIntervalType(), r @ NumericType()) => - DivideInterval(l, r) - - case b @ BinaryOperator(l @ CalendarIntervalType(), r @ NullType()) => - b.withNewChildren(Seq(l, Cast(r, CalendarIntervalType))) - case b @ BinaryOperator(l @ NullType(), r @ CalendarIntervalType()) => - b.withNewChildren(Seq(Cast(l, CalendarIntervalType), r)) - - case Add(l @ DateType(), r @ IntegerType()) => DateAdd(l, r) - case Add(l @ IntegerType(), r @ DateType()) => DateAdd(r, l) - case Subtract(l @ DateType(), r @ IntegerType()) => DateSub(l, r) - case Subtract(l @ DateType(), r @ DateType()) => - if (SQLConf.get.usePostgreSQLDialect) DateDiff(l, r) else SubtractDates(l, r) - case Subtract(l @ TimestampType(), r @ TimestampType()) => - SubtractTimestamps(l, r) - case Subtract(l @ TimestampType(), r @ DateType()) => - SubtractTimestamps(l, Cast(r, TimestampType)) - case Subtract(l @ DateType(), r @ TimestampType()) => - SubtractTimestamps(Cast(l, TimestampType), r) + case d @ DateAdd(TimestampType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) + case d @ DateAdd(StringType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) + case d @ DateSub(TimestampType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) + case d @ DateSub(StringType(), _) => d.copy(startDate = Cast(d.startDate, DateType)) + + case s @ SubtractTimestamps(DateType(), _) => + s.copy(endTimestamp = Cast(s.endTimestamp, TimestampType)) + case s @ SubtractTimestamps(_, DateType()) => + s.copy(startTimestamp = Cast(s.startTimestamp, TimestampType)) + + case t @ TimeAdd(DateType(), _, _) => t.copy(start = Cast(t.start, TimestampType)) + case t @ TimeAdd(StringType(), _, _) => t.copy(start = Cast(t.start, TimestampType)) + case t @ TimeSub(DateType(), _, _) => t.copy(start = Cast(t.start, TimestampType)) + case t @ TimeSub(StringType(), _, _) => t.copy(start = Cast(t.start, TimestampType)) } } @@ -880,11 +852,8 @@ object TypeCoercion { case e if !e.childrenResolved => e // If DecimalType operands are involved, DecimalPrecision will handle it - // If CalendarIntervalType operands are involved, DateTimeOperations will handle it case b @ BinaryOperator(left, right) if !left.dataType.isInstanceOf[DecimalType] && !right.dataType.isInstanceOf[DecimalType] && - !left.dataType.isInstanceOf[CalendarIntervalType] && - !right.dataType.isInstanceOf[CalendarIntervalType] && left.dataType != right.dataType => findTightestCommonType(left.dataType, right.dataType).map { commonType => if (b.inputType.acceptsType(commonType)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index f5023a10baa62..b3aeda852a4dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -151,17 +151,18 @@ case class CurrentBatchTimestamp( """, since = "1.5.0") case class DateAdd(startDate: Expression, days: Expression) - extends BinaryExpression with ImplicitCastInputTypes { + extends BinaryExpression with ExpectsInputTypes { override def left: Expression = startDate override def right: Expression = days - override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType) + override def inputTypes: Seq[AbstractDataType] = + Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType)) override def dataType: DataType = DateType override def nullSafeEval(start: Any, d: Any): Any = { - start.asInstanceOf[Int] + d.asInstanceOf[Int] + start.asInstanceOf[Int] + d.asInstanceOf[Number].intValue() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -185,16 +186,17 @@ case class DateAdd(startDate: Expression, days: Expression) """, since = "1.5.0") case class DateSub(startDate: Expression, days: Expression) - extends BinaryExpression with ImplicitCastInputTypes { + extends BinaryExpression with ExpectsInputTypes { override def left: Expression = startDate override def right: Expression = days - override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType) + override def inputTypes: Seq[AbstractDataType] = + Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType)) override def dataType: DataType = DateType override def nullSafeEval(start: Any, d: Any): Any = { - start.asInstanceOf[Int] - d.asInstanceOf[Int] + start.asInstanceOf[Int] - d.asInstanceOf[Number].intValue() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -1072,7 +1074,7 @@ case class NextDay(startDate: Expression, dayOfWeek: Expression) * Adds an interval to timestamp. */ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + extends BinaryExpression with TimeZoneAwareExpression with ExpectsInputTypes { def this(start: Expression, interval: Expression) = this(start, interval, None) @@ -1187,7 +1189,7 @@ case class FromUTCTimestamp(left: Expression, right: Expression) * Subtracts an interval from timestamp. */ case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[String] = None) - extends BinaryExpression with TimeZoneAwareExpression with ImplicitCastInputTypes { + extends BinaryExpression with TimeZoneAwareExpression with ExpectsInputTypes { def this(start: Expression, interval: Expression) = this(start, interval, None) @@ -2127,7 +2129,7 @@ case class DatePart(field: Expression, source: Expression, child: Expression) * between the given timestamps. */ case class SubtractTimestamps(endTimestamp: Expression, startTimestamp: Expression) - extends BinaryExpression with ImplicitCastInputTypes { + extends BinaryExpression with ExpectsInputTypes { override def left: Expression = endTimestamp override def right: Expression = startTimestamp diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 567cf5ec8ebe6..f35617b374c99 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1401,44 +1401,6 @@ class TypeCoercionSuite extends AnalysisTest { } } - test("rule for date/timestamp operations") { - val dateTimeOperations = TypeCoercion.DateTimeOperations - val date = Literal(new java.sql.Date(0L)) - val timestamp = Literal(new Timestamp(0L)) - val interval = Literal(new CalendarInterval(0, 0, 0)) - val str = Literal("2015-01-01") - val intValue = Literal(0, IntegerType) - - ruleTest(dateTimeOperations, Add(date, interval), Cast(TimeAdd(date, interval), DateType)) - ruleTest(dateTimeOperations, Add(interval, date), Cast(TimeAdd(date, interval), DateType)) - ruleTest(dateTimeOperations, Add(timestamp, interval), - Cast(TimeAdd(timestamp, interval), TimestampType)) - ruleTest(dateTimeOperations, Add(interval, timestamp), - Cast(TimeAdd(timestamp, interval), TimestampType)) - ruleTest(dateTimeOperations, Add(str, interval), Cast(TimeAdd(str, interval), StringType)) - ruleTest(dateTimeOperations, Add(interval, str), Cast(TimeAdd(str, interval), StringType)) - - ruleTest(dateTimeOperations, Subtract(date, interval), Cast(TimeSub(date, interval), DateType)) - ruleTest(dateTimeOperations, Subtract(timestamp, interval), - Cast(TimeSub(timestamp, interval), TimestampType)) - ruleTest(dateTimeOperations, Subtract(str, interval), Cast(TimeSub(str, interval), StringType)) - - // interval operations should not be effected - ruleTest(dateTimeOperations, Add(interval, interval), Add(interval, interval)) - ruleTest(dateTimeOperations, Subtract(interval, interval), Subtract(interval, interval)) - - ruleTest(dateTimeOperations, Add(date, intValue), DateAdd(date, intValue)) - ruleTest(dateTimeOperations, Add(intValue, date), DateAdd(date, intValue)) - ruleTest(dateTimeOperations, Subtract(date, intValue), DateSub(date, intValue)) - ruleTest(dateTimeOperations, Subtract(date, date), SubtractDates(date, date)) - ruleTest(dateTimeOperations, Subtract(timestamp, timestamp), - SubtractTimestamps(timestamp, timestamp)) - ruleTest(dateTimeOperations, Subtract(timestamp, date), - SubtractTimestamps(timestamp, Cast(date, TimestampType))) - ruleTest(dateTimeOperations, Subtract(date, timestamp), - SubtractTimestamps(Cast(date, TimestampType), timestamp)) - } - /** * There are rules that need to not fire before child expressions get resolved. * We use this test to make sure those rules do not fire early. @@ -1586,27 +1548,6 @@ class TypeCoercionSuite extends AnalysisTest { Multiply(CaseWhen(Seq((EqualTo(1, 2), Cast(1, DecimalType(34, 24)))), Cast(100, DecimalType(34, 24))), Cast(1, IntegerType))) } - - test("rule for interval operations") { - val dateTimeOperations = TypeCoercion.DateTimeOperations - val interval = Literal(new CalendarInterval(0, 0, 0)) - - Seq( - Literal(10.toByte, ByteType), - Literal(10.toShort, ShortType), - Literal(10, IntegerType), - Literal(10L, LongType), - Literal(Decimal(10), DecimalType.SYSTEM_DEFAULT), - Literal(10.5.toFloat, FloatType), - Literal(10.5, DoubleType)).foreach { num => - ruleTest(dateTimeOperations, Multiply(interval, num), - MultiplyInterval(interval, num)) - ruleTest(dateTimeOperations, Multiply(num, interval), - MultiplyInterval(interval, num)) - ruleTest(dateTimeOperations, Divide(interval, num), - DivideInterval(interval, num)) - } - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 5cd4d11e32f7a..7d015aa478df0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -332,6 +332,12 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("date_add") { + checkEvaluation( + DateAdd(Literal(Date.valueOf("2016-02-28")), Literal(1.toByte)), + DateTimeUtils.fromJavaDate(Date.valueOf("2016-02-29"))) + checkEvaluation( + DateAdd(Literal(Date.valueOf("2016-02-28")), Literal(1.toShort)), + DateTimeUtils.fromJavaDate(Date.valueOf("2016-02-29"))) checkEvaluation( DateAdd(Literal(Date.valueOf("2016-02-28")), Literal(1)), DateTimeUtils.fromJavaDate(Date.valueOf("2016-02-29"))) @@ -347,10 +353,18 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { DateAdd(Literal(Date.valueOf("2016-02-28")), positiveIntLit), 49627) checkEvaluation( DateAdd(Literal(Date.valueOf("2016-02-28")), negativeIntLit), -15910) + checkConsistencyBetweenInterpretedAndCodegen(DateAdd, DateType, ByteType) + checkConsistencyBetweenInterpretedAndCodegen(DateAdd, DateType, ShortType) checkConsistencyBetweenInterpretedAndCodegen(DateAdd, DateType, IntegerType) } test("date_sub") { + checkEvaluation( + DateSub(Literal(Date.valueOf("2015-01-01")), Literal(1.toByte)), + DateTimeUtils.fromJavaDate(Date.valueOf("2014-12-31"))) + checkEvaluation( + DateSub(Literal(Date.valueOf("2015-01-01")), Literal(1.toShort)), + DateTimeUtils.fromJavaDate(Date.valueOf("2014-12-31"))) checkEvaluation( DateSub(Literal(Date.valueOf("2015-01-01")), Literal(1)), DateTimeUtils.fromJavaDate(Date.valueOf("2014-12-31"))) @@ -366,6 +380,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { DateSub(Literal(Date.valueOf("2016-02-28")), positiveIntLit), -15909) checkEvaluation( DateSub(Literal(Date.valueOf("2016-02-28")), negativeIntLit), 49628) + checkConsistencyBetweenInterpretedAndCodegen(DateSub, DateType, ByteType) + checkConsistencyBetweenInterpretedAndCodegen(DateSub, DateType, ShortType) checkConsistencyBetweenInterpretedAndCodegen(DateSub, DateType, IntegerType) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index de2040c651230..b14778b91510e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -30,12 +30,48 @@ select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), week select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01'); -select date '2001-09-28' + 7; -select 7 + date '2001-09-28'; -select date '2001-10-01' - 7; -select date '2001-10-01' - date '2001-09-28'; -select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'; -select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'; select date '2019-01-01\t'; select timestamp '2019-01-01\t'; + +-- time add/sub +select timestamp'2011-11-11 11:11:11' + interval '2' day; +select timestamp'2011-11-11 11:11:11' - interval '2' day; +select date'2011-11-11 11:11:11' + interval '2' second; +select date'2011-11-11 11:11:11' - interval '2' second; +select '2011-11-11' - interval '2' day; +select '2011-11-11 11:11:11' - interval '2' second; +select '1' - interval '2' second; +select 1 - interval '2' second; + +-- subtract timestamps +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'; +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'; +select timestamp'2019-10-06 10:11:12.345678' - null; +select null - timestamp'2019-10-06 10:11:12.345678'; + +-- date add/sub +select date_add('2011-11-11', 1Y); +select date_add('2011-11-11', 1S); +select date_add('2011-11-11', 1); +select date_add('2011-11-11', 1L); +select date_add('2011-11-11', 1.0); +select date_add('2011-11-11', 1E1); +select date_add('2011-11-11', '1'); +select date_add(date'2011-11-11', 1); +select date_add(timestamp'2011-11-11', 1); +select date_sub(date'2011-11-11', 1); +select date_sub(timestamp'2011-11-11', 1); +select date_sub(null, 1); +select date_sub(date'2011-11-11', null); +select date'2011-11-11' + 1E1; +select null + date '2001-09-28'; +select date '2001-09-28' + 7Y; +select 7S + date '2001-09-28'; +select date '2001-10-01' - 7; +select date '2001-09-28' + null; +select date '2001-09-28' - null; + +-- subtract dates +select null - date '2019-10-06'; +select date '2001-10-01' - date '2001-09-28'; diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index a269420a9a8b3..09584550e3f4c 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 19 +-- Number of queries: 47 -- !query 0 @@ -100,64 +100,294 @@ struct +struct -- !query 11 output -2001-10-05 +2019-01-01 -- !query 12 -select 7 + date '2001-09-28' +select timestamp '2019-01-01\t' -- !query 12 schema -struct +struct -- !query 12 output -2001-10-05 +2019-01-01 00:00:00 -- !query 13 -select date '2001-10-01' - 7 +select timestamp'2011-11-11 11:11:11' + interval '2' day -- !query 13 schema -struct +struct -- !query 13 output -2001-09-24 +2011-11-13 11:11:11 -- !query 14 -select date '2001-10-01' - date '2001-09-28' +select timestamp'2011-11-11 11:11:11' - interval '2' day -- !query 14 schema -struct +struct -- !query 14 output -3 days +2011-11-09 11:11:11 -- !query 15 -select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +select date'2011-11-11 11:11:11' + interval '2' second -- !query 15 schema -struct +struct -- !query 15 output -2078 hours 48 minutes 47.654322 seconds +2011-11-11 -- !query 16 -select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +select date'2011-11-11 11:11:11' - interval '2' second -- !query 16 schema -struct +struct -- !query 16 output --2078 hours -48 minutes -47.654322 seconds +2011-11-10 -- !query 17 -select date '2019-01-01\t' +select '2011-11-11' - interval '2' day -- !query 17 schema -struct +struct -- !query 17 output -2019-01-01 +2011-11-09 00:00:00 -- !query 18 -select timestamp '2019-01-01\t' +select '2011-11-11 11:11:11' - interval '2' second -- !query 18 schema -struct +struct -- !query 18 output -2019-01-01 00:00:00 +2011-11-11 11:11:09 + + +-- !query 19 +select '1' - interval '2' second +-- !query 19 schema +struct +-- !query 19 output +NULL + + +-- !query 20 +select 1 - interval '2' second +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +cannot resolve '1 - INTERVAL '2 seconds'' due to data type mismatch: argument 1 requires timestamp type, however, '1' is of int type.; line 1 pos 7 + + +-- !query 21 +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query 21 schema +struct +-- !query 21 output +2078 hours 48 minutes 47.654322 seconds + + +-- !query 22 +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query 22 schema +struct +-- !query 22 output +-2078 hours -48 minutes -47.654322 seconds + + +-- !query 23 +select timestamp'2019-10-06 10:11:12.345678' - null +-- !query 23 schema +struct +-- !query 23 output +NULL + + +-- !query 24 +select null - timestamp'2019-10-06 10:11:12.345678' +-- !query 24 schema +struct +-- !query 24 output +NULL + + +-- !query 25 +select date_add('2011-11-11', 1Y) +-- !query 25 schema +struct +-- !query 25 output +2011-11-12 + + +-- !query 26 +select date_add('2011-11-11', 1S) +-- !query 26 schema +struct +-- !query 26 output +2011-11-12 + + +-- !query 27 +select date_add('2011-11-11', 1) +-- !query 27 schema +struct +-- !query 27 output +2011-11-12 + + +-- !query 28 +select date_add('2011-11-11', 1L) +-- !query 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7 + + +-- !query 29 +select date_add('2011-11-11', 1.0) +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7 + + +-- !query 30 +select date_add('2011-11-11', 1E1) +-- !query 30 schema +struct<> +-- !query 30 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 + + +-- !query 31 +select date_add('2011-11-11', '1') +-- !query 31 schema +struct<> +-- !query 31 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(CAST('2011-11-11' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 + + +-- !query 32 +select date_add(date'2011-11-11', 1) +-- !query 32 schema +struct +-- !query 32 output +2011-11-12 + + +-- !query 33 +select date_add(timestamp'2011-11-11', 1) +-- !query 33 schema +struct +-- !query 33 output +2011-11-12 + + +-- !query 34 +select date_sub(date'2011-11-11', 1) +-- !query 34 schema +struct +-- !query 34 output +2011-11-10 + + +-- !query 35 +select date_sub(timestamp'2011-11-11', 1) +-- !query 35 schema +struct +-- !query 35 output +2011-11-10 + + +-- !query 36 +select date_sub(null, 1) +-- !query 36 schema +struct +-- !query 36 output +NULL + + +-- !query 37 +select date_sub(date'2011-11-11', null) +-- !query 37 schema +struct +-- !query 37 output +NULL + + +-- !query 38 +select date'2011-11-11' + 1E1 +-- !query 38 schema +struct<> +-- !query 38 output +org.apache.spark.sql.AnalysisException +cannot resolve 'date_add(DATE '2011-11-11', 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 + + +-- !query 39 +select null + date '2001-09-28' +-- !query 39 schema +struct +-- !query 39 output +NULL + + +-- !query 40 +select date '2001-09-28' + 7Y +-- !query 40 schema +struct +-- !query 40 output +2001-10-05 + + +-- !query 41 +select 7S + date '2001-09-28' +-- !query 41 schema +struct +-- !query 41 output +2001-10-05 + + +-- !query 42 +select date '2001-10-01' - 7 +-- !query 42 schema +struct +-- !query 42 output +2001-09-24 + + +-- !query 43 +select date '2001-09-28' + null +-- !query 43 schema +struct +-- !query 43 output +NULL + + +-- !query 44 +select date '2001-09-28' - null +-- !query 44 schema +struct +-- !query 44 output +NULL + + +-- !query 45 +select null - date '2019-10-06' +-- !query 45 schema +struct +-- !query 45 output +NULL + + +-- !query 46 +select date '2001-10-01' - date '2001-09-28' +-- !query 46 schema +struct +-- !query 46 output +3 days diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out index 8608fa494d6cd..07bdfe560a425 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out @@ -16,7 +16,7 @@ select cast(1 as tinyint) + interval 2 day struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) + INTERVAL '2 days')' (tinyint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 -- !query 2 @@ -25,7 +25,7 @@ select cast(1 as smallint) + interval 2 day struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) + INTERVAL '2 days')' (smallint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 -- !query 3 @@ -34,7 +34,7 @@ select cast(1 as int) + interval 2 day struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS INT) + INTERVAL '2 days')' (int and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 -- !query 4 @@ -43,7 +43,7 @@ select cast(1 as bigint) + interval 2 day struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) + INTERVAL '2 days')' (bigint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 -- !query 5 @@ -52,7 +52,7 @@ select cast(1 as float) + interval 2 day struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) + INTERVAL '2 days')' (float and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 -- !query 6 @@ -61,7 +61,7 @@ select cast(1 as double) + interval 2 day struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) + INTERVAL '2 days')' (double and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 -- !query 7 @@ -70,7 +70,7 @@ select cast(1 as decimal(10, 0)) + interval 2 day struct<> -- !query 7 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days')' (decimal(10,0) and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 8 @@ -95,7 +95,7 @@ select cast('1' as binary) + interval 2 day struct<> -- !query 10 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + INTERVAL '2 days')' (binary and interval).; line 1 pos 7 +cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 -- !query 11 @@ -104,7 +104,7 @@ select cast(1 as boolean) + interval 2 day struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) + INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + INTERVAL '2 days')' (boolean and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 -- !query 12 @@ -129,7 +129,7 @@ select interval 2 day + cast(1 as tinyint) struct<> -- !query 14 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS TINYINT))' (interval and tinyint).; line 1 pos 7 +cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 -- !query 15 @@ -138,7 +138,7 @@ select interval 2 day + cast(1 as smallint) struct<> -- !query 15 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS SMALLINT))' (interval and smallint).; line 1 pos 7 +cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 -- !query 16 @@ -147,7 +147,7 @@ select interval 2 day + cast(1 as int) struct<> -- !query 16 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS INT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS INT))' (interval and int).; line 1 pos 7 +cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 -- !query 17 @@ -156,7 +156,7 @@ select interval 2 day + cast(1 as bigint) struct<> -- !query 17 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS BIGINT))' (interval and bigint).; line 1 pos 7 +cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 -- !query 18 @@ -165,7 +165,7 @@ select interval 2 day + cast(1 as float) struct<> -- !query 18 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS FLOAT))' (interval and float).; line 1 pos 7 +cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 -- !query 19 @@ -174,7 +174,7 @@ select interval 2 day + cast(1 as double) struct<> -- !query 19 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS DOUBLE))' (interval and double).; line 1 pos 7 +cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 -- !query 20 @@ -183,7 +183,7 @@ select interval 2 day + cast(1 as decimal(10, 0)) struct<> -- !query 20 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS DECIMAL(10,0)))' (interval and decimal(10,0)).; line 1 pos 7 +cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 21 @@ -208,7 +208,7 @@ select interval 2 day + cast('1' as binary) struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST('1' AS BINARY))' (interval and binary).; line 1 pos 7 +cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 -- !query 24 @@ -217,7 +217,7 @@ select interval 2 day + cast(1 as boolean) struct<> -- !query 24 output org.apache.spark.sql.AnalysisException -cannot resolve '(INTERVAL '2 days' + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(INTERVAL '2 days' + CAST(1 AS BOOLEAN))' (interval and boolean).; line 1 pos 7 +cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 -- !query 25 @@ -242,7 +242,7 @@ select cast(1 as tinyint) - interval 2 day struct<> -- !query 27 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) - INTERVAL '2 days')' (tinyint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS TINYINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 -- !query 28 @@ -251,7 +251,7 @@ select cast(1 as smallint) - interval 2 day struct<> -- !query 28 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) - INTERVAL '2 days')' (smallint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS SMALLINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 -- !query 29 @@ -260,7 +260,7 @@ select cast(1 as int) - interval 2 day struct<> -- !query 29 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS INT) - INTERVAL '2 days')' (int and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS INT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 -- !query 30 @@ -269,7 +269,7 @@ select cast(1 as bigint) - interval 2 day struct<> -- !query 30 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) - INTERVAL '2 days')' (bigint and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS BIGINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 -- !query 31 @@ -278,7 +278,7 @@ select cast(1 as float) - interval 2 day struct<> -- !query 31 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) - INTERVAL '2 days')' (float and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS FLOAT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 -- !query 32 @@ -287,7 +287,7 @@ select cast(1 as double) - interval 2 day struct<> -- !query 32 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) - INTERVAL '2 days')' (double and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS DOUBLE) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 -- !query 33 @@ -296,7 +296,7 @@ select cast(1 as decimal(10, 0)) - interval 2 day struct<> -- !query 33 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days')' (decimal(10,0) and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 34 @@ -321,7 +321,7 @@ select cast('1' as binary) - interval 2 day struct<> -- !query 36 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - INTERVAL '2 days')' (binary and interval).; line 1 pos 7 +cannot resolve 'CAST('1' AS BINARY) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 -- !query 37 @@ -330,7 +330,7 @@ select cast(1 as boolean) - interval 2 day struct<> -- !query 37 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) - INTERVAL '2 days')' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - INTERVAL '2 days')' (boolean and interval).; line 1 pos 7 +cannot resolve 'CAST(1 AS BOOLEAN) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 -- !query 38 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index e479a6d73a182..ec6f3d7696b03 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -312,7 +312,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t struct<> -- !query 37 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 38 @@ -321,7 +321,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t struct<> -- !query 38 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 39 @@ -330,7 +330,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t struct<> -- !query 39 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 40 @@ -339,7 +339,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 41 @@ -712,7 +712,7 @@ SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 85 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 86 @@ -721,7 +721,7 @@ SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 86 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 87 @@ -730,7 +730,7 @@ SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 87 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 88 @@ -739,7 +739,7 @@ SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 88 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 89 @@ -1008,7 +1008,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FRO struct<> -- !query 121 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 122 @@ -1017,7 +1017,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FRO struct<> -- !query 122 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 123 @@ -1026,7 +1026,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FR struct<> -- !query 123 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 124 @@ -1035,7 +1035,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FR struct<> -- !query 124 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 125 @@ -1044,7 +1044,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t struct<> -- !query 125 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 126 @@ -1053,7 +1053,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t struct<> -- !query 126 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 127 @@ -1062,7 +1062,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t struct<> -- !query 127 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 128 @@ -1071,7 +1071,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t struct<> -- !query 128 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 129 @@ -1408,7 +1408,7 @@ SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR struct<> -- !query 169 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 170 @@ -1417,7 +1417,7 @@ SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR struct<> -- !query 170 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 171 @@ -1426,7 +1426,7 @@ SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR struct<> -- !query 171 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 172 @@ -1435,7 +1435,7 @@ SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FR struct<> -- !query 172 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 173 @@ -1444,7 +1444,7 @@ SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 173 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 +cannot resolve 'subtractdates(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 -- !query 174 @@ -1453,7 +1453,7 @@ SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 174 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 +cannot resolve 'subtractdates(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 -- !query 175 @@ -1462,7 +1462,7 @@ SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 175 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 +cannot resolve 'subtractdates(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 -- !query 176 @@ -1471,7 +1471,7 @@ SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 176 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 +cannot resolve 'subtractdates(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 -- !query 177 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out index 8557f92f25115..af0e348ae8ef1 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out @@ -107,7 +107,7 @@ SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t struct<> -- !query 12 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS DOUBLE) + CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) + CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 -- !query 13 @@ -198,16 +198,15 @@ SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS DOUBLE) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) - CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 +cannot resolve 'subtracttimestamps('1', CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, ''1'' is of string type.; line 1 pos 7 -- !query 24 SELECT '1' - cast('2017-12-11 09:30:00' as date) FROM t -- !query 24 schema -struct<> +struct -- !query 24 output -org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS DOUBLE) - CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) - CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 +NULL -- !query 25 @@ -699,7 +698,7 @@ SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t struct<> -- !query 83 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) + CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) + CAST('1' AS DOUBLE))' (date and double).; line 1 pos 7 +cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 -- !query 84 @@ -782,7 +781,7 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t struct<> -- !query 93 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) - CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7 +cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), '1')' due to data type mismatch: argument 2 requires timestamp type, however, ''1'' is of string type.; line 1 pos 7 -- !query 94 @@ -791,7 +790,7 @@ SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t struct<> -- !query 94 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) - CAST('1' AS DOUBLE))' (date and double).; line 1 pos 7 +cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 -- !query 95 From 35bab339848cba7e518478cb87591bff6a95ff07 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 5 Dec 2019 11:50:55 -0600 Subject: [PATCH 0799/1387] [SPARK-30121][BUILD] Fix memory usage in sbt build script ### What changes were proposed in this pull request? 1. the default memory setting is missing in usage instructions ``` build/sbt -h ``` before ``` -mem set memory options (default: , which is -Xms2048m -Xmx2048m -XX:ReservedCodeCacheSize=256m) ``` after ``` -mem set memory options (default: 2048, which is -Xms2048m -Xmx2048m -XX:ReservedCodeCacheSize=256m) ``` 2. the Perm space is not needed anymore, since java7 is removed. the changes in this pr are based on the main sbt script of the newest stable version 1.3.4. ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? no ### How was this patch tested? manually Closes #26757 from yaooqinn/SPARK-30121. Authored-by: Kent Yao Signed-off-by: Sean Owen --- build/sbt | 2 +- build/sbt-launch-lib.bash | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/build/sbt b/build/sbt index 7d8d0993e57d8..475dfd3b20b43 100755 --- a/build/sbt +++ b/build/sbt @@ -66,7 +66,7 @@ Usage: $script_name [options] -sbt-dir path to global settings/plugins directory (default: ~/.sbt) -sbt-boot path to shared boot directory (default: ~/.sbt/boot in 0.11 series) -ivy path to local Ivy repository (default: ~/.ivy2) - -mem set memory options (default: $sbt_mem, which is $(get_mem_opts $sbt_mem)) + -mem set memory options (default: $sbt_default_mem, which is $(get_mem_opts $sbt_default_mem)) -no-share use all local caches; no sharing -no-global uses global caches, but does not use global ~/.sbt directory. -jvm-debug Turn on JVM debugging, open at the given port. diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 0ed6f8b6d737b..162bfbf2257c7 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -17,6 +17,7 @@ declare -a java_args declare -a scalac_args declare -a sbt_commands declare -a maven_profiles +declare sbt_default_mem=2048 if test -x "$JAVA_HOME/bin/java"; then echo -e "Using $JAVA_HOME as default JAVA_HOME." @@ -111,11 +112,10 @@ addDebugger () { # a ham-fisted attempt to move some memory settings in concert # so they need not be dicked around with individually. get_mem_opts () { - local mem=${1:-2048} - local perm=$(( $mem / 4 )) - (( $perm > 256 )) || perm=256 - (( $perm < 4096 )) || perm=4096 - local codecache=$(( $perm / 2 )) + local mem=${1:-$sbt_default_mem} + local codecache=$(( $mem / 8 )) + (( $codecache > 128 )) || codecache=128 + (( $codecache < 2048 )) || codecache=2048 echo "-Xms${mem}m -Xmx${mem}m -XX:ReservedCodeCacheSize=${codecache}m" } From 5892bbf447f195d73f89b9ec64eb8abf671672f9 Mon Sep 17 00:00:00 2001 From: Aman Omer Date: Thu, 5 Dec 2019 11:54:45 -0600 Subject: [PATCH 0800/1387] [SPARK-30124][MLLIB] unnecessary persist in PythonMLLibAPI.scala ### What changes were proposed in this pull request? Removed unnecessary persist. ### Why are the changes needed? Persist in `PythonMLLibAPI.scala` is unnecessary because later in `run()` of `gmmAlg` is caching the data. https://github.com/apache/spark/blob/710ddab39e20f49e917311c3e27d142b5a2bcc71/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala#L167-L171 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually Closes #26758 from amanomer/improperPersist. Authored-by: Aman Omer Signed-off-by: Sean Owen --- .../org/apache/spark/mllib/api/python/PythonMLLibAPI.scala | 6 +----- .../org/apache/spark/mllib/clustering/GaussianMixture.scala | 1 + 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index bafaafb720ed8..259ecb3a1762f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -407,11 +407,7 @@ private[python] class PythonMLLibAPI extends Serializable { if (seed != null) gmmAlg.setSeed(seed) - try { - new GaussianMixtureModelWrapper(gmmAlg.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK))) - } finally { - data.rdd.unpersist() - } + new GaussianMixtureModelWrapper(gmmAlg.run(data.rdd)) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index a9ed36e90974e..cde23096b40a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -234,6 +234,7 @@ class GaussianMixture private ( iter += 1 compute.destroy() } + breezeData.unpersist() new GaussianMixtureModel(weights, gaussians) } From 7782b61a31ba49cdeffc35a942bd365bb71b026d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 5 Dec 2019 13:48:29 -0800 Subject: [PATCH 0801/1387] [SPARK-29392][CORE][SQL][FOLLOWUP] Avoid deprecated (in 2.13) Symbol syntax 'foo in favor of simpler expression, where it generated deprecation warnings TL;DR - this is more of the same change in https://github.com/apache/spark/pull/26748 I told you it'd be iterative! Closes #26765 from srowen/SPARK-29392.3. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/DistributionSuite.scala | 4 +- .../analysis/AnalysisErrorSuite.scala | 56 +++++----- .../sql/catalyst/analysis/AnalysisSuite.scala | 80 ++++++------- .../spark/sql/DataFrameAggregateSuite.scala | 2 +- .../spark/sql/DataFrameFunctionsSuite.scala | 50 +++++---- .../apache/spark/sql/DataFrameJoinSuite.scala | 8 +- .../spark/sql/DataFramePivotSuite.scala | 4 +- .../spark/sql/DataFrameSelfJoinSuite.scala | 8 +- .../sql/DataFrameSetOperationsSuite.scala | 8 +- .../apache/spark/sql/DataFrameStatSuite.scala | 4 +- .../org/apache/spark/sql/DataFrameSuite.scala | 105 +++++++++--------- .../sql/sources/HadoopFsRelationTest.scala | 26 ++--- .../ParquetHadoopFsRelationSuite.scala | 4 +- 13 files changed, 183 insertions(+), 176 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 4862f923e4fba..e94c1204a2be7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -221,7 +221,7 @@ class DistributionSuite extends SparkFunSuite { checkSatisfied( RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), - OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc, 'd.desc)), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"c".asc, $"d".desc)), true) // TODO: We can have an optimization to first sort the dataset @@ -240,7 +240,7 @@ class DistributionSuite extends SparkFunSuite { checkSatisfied( RangePartitioning(Seq($"a".asc, $"b".asc, $"c".asc), 10), - OrderedDistribution(Seq($"a".asc, $"b".asc, 'd.desc)), + OrderedDistribution(Seq($"a".asc, $"b".asc, $"d".desc)), false) // RangePartitioning can satisfy ClusteredDistribution iff its ordering expressions are a subset diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index e0fa1f2ecb88e..646abb6b85916 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -112,24 +112,24 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "scalar subquery with 2 columns", testRelation.select( - (ScalarSubquery(testRelation.select('a, dateLit.as('b))) + Literal(1)).as('a)), + (ScalarSubquery(testRelation.select($"a", dateLit.as("b"))) + Literal(1)).as("a")), "Scalar subquery must return only one column, but got 2" :: Nil) errorTest( "scalar subquery with no column", - testRelation.select(ScalarSubquery(LocalRelation()).as('a)), + testRelation.select(ScalarSubquery(LocalRelation()).as("a")), "Scalar subquery must return only one column, but got 0" :: Nil) errorTest( "single invalid type, single arg", - testRelation.select(TestFunction(dateLit :: Nil, IntegerType :: Nil).as('a)), + testRelation.select(TestFunction(dateLit :: Nil, IntegerType :: Nil).as("a")), "cannot resolve" :: "testfunction(CAST(NULL AS DATE))" :: "argument 1" :: "requires int type" :: "'CAST(NULL AS DATE)' is of date type" :: Nil) errorTest( "single invalid type, second arg", testRelation.select( - TestFunction(dateLit :: dateLit :: Nil, DateType :: IntegerType :: Nil).as('a)), + TestFunction(dateLit :: dateLit :: Nil, DateType :: IntegerType :: Nil).as("a")), "cannot resolve" :: "testfunction(CAST(NULL AS DATE), CAST(NULL AS DATE))" :: "argument 2" :: "requires int type" :: "'CAST(NULL AS DATE)' is of date type" :: Nil) @@ -137,7 +137,7 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "multiple invalid type", testRelation.select( - TestFunction(dateLit :: dateLit :: Nil, IntegerType :: IntegerType :: Nil).as('a)), + TestFunction(dateLit :: dateLit :: Nil, IntegerType :: IntegerType :: Nil).as("a")), "cannot resolve" :: "testfunction(CAST(NULL AS DATE), CAST(NULL AS DATE))" :: "argument 1" :: "argument 2" :: "requires int type" :: "'CAST(NULL AS DATE)' is of date type" :: Nil) @@ -150,7 +150,7 @@ class AnalysisErrorSuite extends AnalysisTest { WindowSpecDefinition( UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, - UnspecifiedFrame)).as('window)), + UnspecifiedFrame)).as("window")), "not supported within a window function" :: Nil) errorTest( @@ -161,7 +161,7 @@ class AnalysisErrorSuite extends AnalysisTest { WindowSpecDefinition( UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, - UnspecifiedFrame)).as('window)), + UnspecifiedFrame)).as("window")), "Distinct window functions are not supported" :: Nil) errorTest( @@ -176,7 +176,7 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "nested aggregate functions", - testRelation.groupBy('a)( + testRelation.groupBy($"a")( AggregateExpression( Max(AggregateExpression(Count(Literal(1)), Complete, isDistinct = false)), Complete, @@ -192,39 +192,39 @@ class AnalysisErrorSuite extends AnalysisTest { WindowSpecDefinition( UnresolvedAttribute("a") :: Nil, SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, - SpecifiedWindowFrame(RangeFrame, Literal(1), Literal(2)))).as('window)), + SpecifiedWindowFrame(RangeFrame, Literal(1), Literal(2)))).as("window")), "window frame" :: "must match the required frame" :: Nil) errorTest( "too many generators", - listRelation.select(Explode('list).as('a), Explode('list).as('b)), + listRelation.select(Explode($"list").as("a"), Explode($"list").as("b")), "only one generator" :: "explode" :: Nil) errorTest( "unresolved attributes", - testRelation.select('abcd), + testRelation.select($"abcd"), "cannot resolve" :: "abcd" :: Nil) errorTest( "unresolved attributes with a generated name", - testRelation2.groupBy('a)(max('b)) - .where(sum('b) > 0) - .orderBy('havingCondition.asc), + testRelation2.groupBy($"a")(max($"b")) + .where(sum($"b") > 0) + .orderBy($"havingCondition".asc), "cannot resolve" :: "havingCondition" :: Nil) errorTest( "unresolved star expansion in max", - testRelation2.groupBy('a)(sum(UnresolvedStar(None))), + testRelation2.groupBy($"a")(sum(UnresolvedStar(None))), "Invalid usage of '*'" :: "in expression 'sum'" :: Nil) errorTest( "sorting by unsupported column types", - mapRelation.orderBy('map.asc), + mapRelation.orderBy($"map".asc), "sort" :: "type" :: "map" :: Nil) errorTest( "sorting by attributes are not from grouping expressions", - testRelation2.groupBy('a, 'c)('a, 'c, count('a).as("a3")).orderBy('b.asc), + testRelation2.groupBy($"a", $"c")($"a", $"c", count($"a").as("a3")).orderBy($"b".asc), "cannot resolve" :: "'`b`'" :: "given input columns" :: "[a, a3, c]" :: Nil) errorTest( @@ -239,7 +239,7 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "missing group by", - testRelation2.groupBy('a)('b), + testRelation2.groupBy($"a")($"b"), "'`b`'" :: "group by" :: Nil ) @@ -317,7 +317,7 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "SPARK-9955: correct error message for aggregate", // When parse SQL string, we will wrap aggregate expressions with UnresolvedAlias. - testRelation2.where('bad_column > 1).groupBy('a)(UnresolvedAlias(max('b))), + testRelation2.where($"bad_column" > 1).groupBy($"a")(UnresolvedAlias(max($"b"))), "cannot resolve '`bad_column`'" :: Nil) errorTest( @@ -385,14 +385,14 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "generator nested in expressions", - listRelation.select(Explode('list) + 1), + listRelation.select(Explode($"list") + 1), "Generators are not supported when it's nested in expressions, but got: (explode(list) + 1)" :: Nil ) errorTest( "generator appears in operator which is not Project", - listRelation.sortBy(Explode('list).asc), + listRelation.sortBy(Explode($"list").asc), "Generators are not supported outside the SELECT clause, but got: Sort" :: Nil ) @@ -410,7 +410,7 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "more than one generators in SELECT", - listRelation.select(Explode('list), Explode('list)), + listRelation.select(Explode($"list"), Explode($"list")), "Only one generator allowed per select clause but found 2: explode(list), explode(list)" :: Nil ) @@ -510,20 +510,20 @@ class AnalysisErrorSuite extends AnalysisTest { } test("Join can work on binary types but can't work on map types") { - val left = LocalRelation('a.binary, 'b.map(StringType, StringType)) - val right = LocalRelation('c.binary, 'd.map(StringType, StringType)) + val left = LocalRelation(Symbol("a").binary, Symbol("b").map(StringType, StringType)) + val right = LocalRelation(Symbol("c").binary, Symbol("d").map(StringType, StringType)) val plan1 = left.join( right, joinType = Cross, - condition = Some('a === 'c)) + condition = Some(Symbol("a") === Symbol("c"))) assertAnalysisSuccess(plan1) val plan2 = left.join( right, joinType = Cross, - condition = Some('b === 'd)) + condition = Some(Symbol("b") === Symbol("d"))) assertAnalysisError(plan2, "EqualTo does not support ordering on type map" :: Nil) } @@ -596,7 +596,7 @@ class AnalysisErrorSuite extends AnalysisTest { val plan5 = Filter( Exists( Sample(0.0, 0.5, false, 1L, - Filter(EqualTo(UnresolvedAttribute("a"), b), LocalRelation(b))).select('b) + Filter(EqualTo(UnresolvedAttribute("a"), b), LocalRelation(b))).select("b") ), LocalRelation(a)) assertAnalysisError(plan5, @@ -606,7 +606,7 @@ class AnalysisErrorSuite extends AnalysisTest { test("Error on filter condition containing aggregate expressions") { val a = AttributeReference("a", IntegerType)() val b = AttributeReference("b", IntegerType)() - val plan = Filter('a === UnresolvedFunction("max", Seq(b), true), LocalRelation(a, b)) + val plan = Filter(Symbol("a") === UnresolvedFunction("max", Seq(b), true), LocalRelation(a, b)) assertAnalysisError(plan, "Aggregate/Window/Generate expressions are not valid in where clause of the query" :: Nil) } 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 ae474cac9f748..5405009c9e208 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 @@ -47,7 +47,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { val plan = (1 to 120) .map(_ => testRelation) .fold[LogicalPlan](testRelation) { (a, b) => - a.select(UnresolvedStar(None)).select('a).union(b.select(UnresolvedStar(None))) + a.select(UnresolvedStar(None)).select($"a").union(b.select(UnresolvedStar(None))) } assertAnalysisSuccess(plan) @@ -99,9 +99,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { // Case 1: one missing attribute is in the leaf node and another is in the unary node val plan1 = testRelation2 - .where('a > "str").select('a, 'b) - .where('b > "str").select('a) - .sortBy('b.asc, 'c.desc) + .where($"a" > "str").select($"a", $"b") + .where($"b" > "str").select($"a") + .sortBy($"b".asc, $"c".desc) val expected1 = testRelation2 .where(a > "str").select(a, b, c) .where(b > "str").select(a, b, c) @@ -111,9 +111,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { // Case 2: all the missing attributes are in the leaf node val plan2 = testRelation2 - .where('a > "str").select('a) - .where('a > "str").select('a) - .sortBy('b.asc, 'c.desc) + .where($"a" > "str").select($"a") + .where($"a" > "str").select($"a") + .sortBy($"b".asc, $"c".desc) val expected2 = testRelation2 .where(a > "str").select(a, b, c) .where(a > "str").select(a, b, c) @@ -130,8 +130,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { // Case: join itself can resolve all the missing attributes val plan = testRelation2.join(testRelation3) - .where('a > "str").select('a, 'b) - .sortBy('c.desc, 'h.asc) + .where($"a" > "str").select($"a", $"b") + .sortBy($"c".desc, $"h".asc) val expected = testRelation2.join(testRelation3) .where(a > "str").select(a, b, c, h) .sortBy(c.desc, h.asc) @@ -149,9 +149,9 @@ class AnalysisSuite extends AnalysisTest with Matchers { // Case 1: when the child of Sort is not Aggregate, // the sort reference is handled by the rule ResolveSortReferences val plan1 = testRelation2 - .groupBy('a, 'c, 'b)('a, 'c, count('a).as("a3")) - .select('a, 'c, 'a3) - .orderBy('b.asc) + .groupBy($"a", $"c", $"b")($"a", $"c", count($"a").as("a3")) + .select($"a", $"c", $"a3") + .orderBy($"b".asc) val expected1 = testRelation2 .groupBy(a, c, b)(a, c, alias_a3, b) @@ -164,8 +164,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { // Case 2: when the child of Sort is Aggregate, // the sort reference is handled by the rule ResolveAggregateFunctions val plan2 = testRelation2 - .groupBy('a, 'c, 'b)('a, 'c, count('a).as("a3")) - .orderBy('b.asc) + .groupBy($"a", $"c", $"b")($"a", $"c", count($"a").as("a3")) + .orderBy($"b".asc) val expected2 = testRelation2 .groupBy(a, c, b)(a, c, alias_a3, alias_b) @@ -187,11 +187,11 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("divide should be casted into fractional types") { val plan = caseInsensitiveAnalyzer.execute( testRelation2.select( - 'a / Literal(2) as 'div1, - 'a / 'b as 'div2, - 'a / 'c as 'div3, - 'a / 'd as 'div4, - 'e / 'e as 'div5)) + $"a" / Literal(2) as "div1", + $"a" / $"b" as "div2", + $"a" / $"c" as "div3", + $"a" / $"d" as "div4", + $"e" / $"e" as "div5")) val pl = plan.asInstanceOf[Project].projectList assert(pl(0).dataType == DoubleType) @@ -259,7 +259,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { CreateNamedStruct(Seq( Literal(att1.name), att1, Literal("a_plus_1"), (att1 + 1))), - 'col.struct(prevPlan.output(0).dataType.asInstanceOf[StructType]).notNull + Symbol("col").struct(prevPlan.output(0).dataType.asInstanceOf[StructType]).notNull )).as("arr") ) @@ -270,7 +270,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { val a = testRelation2.output(0) val c = testRelation2.output(2) - val plan = testRelation2.select('c).orderBy(Floor('a).asc) + val plan = testRelation2.select($"c").orderBy(Floor($"a").asc) val expected = testRelation2.select(c, a) .orderBy(Floor(Cast(a, DoubleType, Option(TimeZone.getDefault().getID))).asc).select(c) @@ -374,8 +374,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { val alias3 = count(a).as("a3") val plan = testRelation2 - .groupBy('a, 'c)('a.as("a1"), 'c.as("a2"), count('a).as("a3")) - .orderBy('a1.asc, 'c.asc) + .groupBy($"a", $"c")($"a".as("a1"), $"c".as("a2"), count($"a").as("a3")) + .orderBy($"a1".asc, $"c".asc) val expected = testRelation2 .groupBy(a, c)(alias1, alias2, alias3) @@ -391,13 +391,15 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-12102: Ignore nullablity when comparing two sides of case") { - val relation = LocalRelation('a.struct('x.int), 'b.struct('x.int.withNullability(false))) - val plan = relation.select(CaseWhen(Seq((Literal(true), 'a.attr)), 'b).as("val")) + val relation = LocalRelation(Symbol("a").struct(Symbol("x").int), + Symbol("b").struct(Symbol("x").int.withNullability(false))) + val plan = relation.select( + CaseWhen(Seq((Literal(true), Symbol("a").attr)), Symbol("b")).as("val")) assertAnalysisSuccess(plan) } test("Keep attribute qualifiers after dedup") { - val input = LocalRelation('key.int, 'value.string) + val input = LocalRelation(Symbol("key").int, Symbol("value").string) val query = Project(Seq($"x.key", $"y.key"), @@ -524,8 +526,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-20963 Support aliases for join relations in FROM clause") { def joinRelationWithAliases(outputNames: Seq[String]): LogicalPlan = { - val src1 = LocalRelation('id.int, 'v1.string).as("s1") - val src2 = LocalRelation('id.int, 'v2.string).as("s2") + val src1 = LocalRelation(Symbol("id").int, Symbol("v1").string).as("s1") + val src2 = LocalRelation(Symbol("id").int, Symbol("v2").string).as("s2") UnresolvedSubqueryColumnAliases( outputNames, SubqueryAlias( @@ -553,12 +555,13 @@ class AnalysisSuite extends AnalysisTest with Matchers { } checkPartitioning[HashPartitioning](numPartitions = 10, exprs = Literal(20)) - checkPartitioning[HashPartitioning](numPartitions = 10, exprs = 'a.attr, 'b.attr) + checkPartitioning[HashPartitioning](numPartitions = 10, + exprs = Symbol("a").attr, Symbol("b").attr) checkPartitioning[RangePartitioning](numPartitions = 10, exprs = SortOrder(Literal(10), Ascending)) checkPartitioning[RangePartitioning](numPartitions = 10, - exprs = SortOrder('a.attr, Ascending), SortOrder('b.attr, Descending)) + exprs = SortOrder(Symbol("a").attr, Ascending), SortOrder(Symbol("b").attr, Descending)) checkPartitioning[RoundRobinPartitioning](numPartitions = 10, exprs = Seq.empty: _*) @@ -569,7 +572,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkPartitioning(numPartitions = -1, exprs = Literal(20)) } intercept[IllegalArgumentException] { - checkPartitioning(numPartitions = 10, exprs = SortOrder('a.attr, Ascending), 'b.attr) + checkPartitioning(numPartitions = 10, exprs = + SortOrder(Symbol("a").attr, Ascending), Symbol("b").attr) } } @@ -592,10 +596,10 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-24488 Generator with multiple aliases") { assertAnalysisSuccess( - listRelation.select(Explode('list).as("first_alias").as("second_alias"))) + listRelation.select(Explode($"list").as("first_alias").as("second_alias"))) assertAnalysisSuccess( listRelation.select(MultiAlias(MultiAlias( - PosExplode('list), Seq("first_pos", "first_val")), Seq("second_pos", "second_val")))) + PosExplode($"list"), Seq("first_pos", "first_val")), Seq("second_pos", "second_val")))) } test("SPARK-24151: CURRENT_DATE, CURRENT_TIMESTAMP should be case insensitive") { @@ -618,18 +622,18 @@ class AnalysisSuite extends AnalysisTest with Matchers { object ViewAnalyzer extends RuleExecutor[LogicalPlan] { val batches = Batch("View", Once, EliminateView) :: Nil } - val relation = LocalRelation('a.int.notNull, 'b.string) + val relation = LocalRelation(Symbol("a").int.notNull, Symbol("b").string) val view = View(CatalogTable( identifier = TableIdentifier("v1"), tableType = CatalogTableType.VIEW, storage = CatalogStorageFormat.empty, schema = StructType(Seq(StructField("a", IntegerType), StructField("b", StringType)))), - output = Seq('a.int, 'b.string), + output = Seq(Symbol("a").int, Symbol("b").string), child = relation) val tz = Option(conf.sessionLocalTimeZone) val expected = Project(Seq( - Alias(Cast('a.int.notNull, IntegerType, tz), "a")(), - Alias(Cast('b.string, StringType, tz), "b")()), + Alias(Cast(Symbol("a").int.notNull, IntegerType, tz), "a")(), + Alias(Cast(Symbol("b").string, StringType, tz), "b")()), relation) val res = ViewAnalyzer.execute(view) comparePlans(res, expected) @@ -719,7 +723,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { "Multiple definitions of observed metrics" :: "evt1" :: Nil) // Different children, same metrics - fail - val b = 'b.string + val b = Symbol("b").string val tblB = LocalRelation(b) assertAnalysisError(Union( CollectMetrics("evt1", count :: Nil, testRelation) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 7065f076e8afd..49e259ff0242f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -969,7 +969,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { test("Dataset agg functions support calendar intervals") { val df1 = Seq((1, "1 day"), (2, "2 day"), (3, "3 day"), (3, null)).toDF("a", "b") - val df2 = df1.select('a, 'b cast CalendarIntervalType).groupBy('a % 2) + val df2 = df1.select($"a", $"b" cast CalendarIntervalType).groupBy($"a" % 2) checkAnswer(df2.sum("b"), Row(0, new CalendarInterval(0, 2, 0)) :: Row(1, new CalendarInterval(0, 4, 0)) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 6c6d29a539197..a346377cd1bc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -278,15 +278,15 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { test("pmod") { val intData = Seq((7, 3), (-7, 3)).toDF("a", "b") checkAnswer( - intData.select(pmod('a, 'b)), + intData.select(pmod($"a", $"b")), Seq(Row(1), Row(2)) ) checkAnswer( - intData.select(pmod('a, lit(3))), + intData.select(pmod($"a", lit(3))), Seq(Row(1), Row(2)) ) checkAnswer( - intData.select(pmod(lit(-7), 'b)), + intData.select(pmod(lit(-7), $"b")), Seq(Row(2), Row(2)) ) checkAnswer( @@ -303,7 +303,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ) val doubleData = Seq((7.2, 4.1)).toDF("a", "b") checkAnswer( - doubleData.select(pmod('a, 'b)), + doubleData.select(pmod($"a", $"b")), Seq(Row(3.1000000000000005)) // same as hive ) checkAnswer( @@ -600,7 +600,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ) def testPrimitiveType(): Unit = { - checkAnswer(idf.select(map_entries('m)), iExpected) + checkAnswer(idf.select(map_entries($"m")), iExpected) checkAnswer(idf.selectExpr("map_entries(m)"), iExpected) checkAnswer(idf.selectExpr("map_entries(map(1, null, 2, null))"), Seq.fill(iExpected.length)(Row(Seq(Row(1, null), Row(2, null))))) @@ -627,7 +627,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ) def testNonPrimitiveType(): Unit = { - checkAnswer(sdf.select(map_entries('m)), sExpected) + checkAnswer(sdf.select(map_entries($"m")), sExpected) checkAnswer(sdf.selectExpr("map_entries(m)"), sExpected) } @@ -652,7 +652,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ) checkAnswer(df1.selectExpr("map_concat(map1, map2)"), expected1a) - checkAnswer(df1.select(map_concat('map1, 'map2)), expected1a) + checkAnswer(df1.select(map_concat($"map1", $"map2")), expected1a) val expected1b = Seq( Row(Map(1 -> 100, 2 -> 200)), @@ -661,7 +661,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ) checkAnswer(df1.selectExpr("map_concat(map1)"), expected1b) - checkAnswer(df1.select(map_concat('map1)), expected1b) + checkAnswer(df1.select(map_concat($"map1")), expected1b) val df2 = Seq( ( @@ -693,7 +693,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ) checkAnswer(df3.selectExpr("map_concat(map1, map2)"), expected3) - checkAnswer(df3.select(map_concat('map1, 'map2)), expected3) + checkAnswer(df3.select(map_concat($"map1", $"map2")), expected3) val expectedMessage1 = "input to function map_concat should all be the same type" @@ -702,7 +702,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { }.getMessage().contains(expectedMessage1)) assert(intercept[AnalysisException] { - df2.select(map_concat('map1, 'map2)).collect() + df2.select(map_concat($"map1", $"map2")).collect() }.getMessage().contains(expectedMessage1)) val expectedMessage2 = "input to function map_concat should all be of type map" @@ -712,7 +712,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { }.getMessage().contains(expectedMessage2)) assert(intercept[AnalysisException] { - df2.select(map_concat('map1, lit(12))).collect() + df2.select(map_concat($"map1", lit(12))).collect() }.getMessage().contains(expectedMessage2)) } @@ -731,7 +731,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(null)) def testPrimitiveType(): Unit = { - checkAnswer(idf.select(map_from_entries('a)), iExpected) + checkAnswer(idf.select(map_from_entries($"a")), iExpected) checkAnswer(idf.selectExpr("map_from_entries(a)"), iExpected) checkAnswer(idf.selectExpr("map_from_entries(array(struct(1, null), struct(2, null)))"), Seq.fill(iExpected.length)(Row(Map(1 -> null, 2 -> null)))) @@ -759,7 +759,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Row(null)) def testNonPrimitiveType(): Unit = { - checkAnswer(sdf.select(map_from_entries('a)), sExpected) + checkAnswer(sdf.select(map_from_entries($"a")), sExpected) checkAnswer(sdf.selectExpr("map_from_entries(a)"), sExpected) } @@ -979,8 +979,10 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { } test("sequence") { - checkAnswer(Seq((-2, 2)).toDF().select(sequence('_1, '_2)), Seq(Row(Array(-2, -1, 0, 1, 2)))) - checkAnswer(Seq((7, 2, -2)).toDF().select(sequence('_1, '_2, '_3)), Seq(Row(Array(7, 5, 3)))) + checkAnswer(Seq((-2, 2)).toDF().select(sequence($"_1", $"_2")), + Seq(Row(Array(-2, -1, 0, 1, 2)))) + checkAnswer(Seq((7, 2, -2)).toDF().select(sequence($"_1", $"_2", $"_3")), + Seq(Row(Array(7, 5, 3)))) checkAnswer( spark.sql("select sequence(" + @@ -1006,7 +1008,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { // test type coercion checkAnswer( - Seq((1.toByte, 3L, 1)).toDF().select(sequence('_1, '_2, '_3)), + Seq((1.toByte, 3L, 1)).toDF().select(sequence($"_1", $"_2", $"_3")), Seq(Row(Array(1L, 2L, 3L)))) checkAnswer( @@ -1034,9 +1036,9 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { test("reverse function - string") { val oneRowDF = Seq(("Spark", 3215)).toDF("s", "i") def testString(): Unit = { - checkAnswer(oneRowDF.select(reverse('s)), Seq(Row("krapS"))) + checkAnswer(oneRowDF.select(reverse($"s")), Seq(Row("krapS"))) checkAnswer(oneRowDF.selectExpr("reverse(s)"), Seq(Row("krapS"))) - checkAnswer(oneRowDF.select(reverse('i)), Seq(Row("5123"))) + checkAnswer(oneRowDF.select(reverse($"i")), Seq(Row("5123"))) checkAnswer(oneRowDF.selectExpr("reverse(i)"), Seq(Row("5123"))) checkAnswer(oneRowDF.selectExpr("reverse(null)"), Seq(Row(null))) } @@ -1058,7 +1060,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { def testArrayOfPrimitiveTypeNotContainsNull(): Unit = { checkAnswer( - idfNotContainsNull.select(reverse('i)), + idfNotContainsNull.select(reverse($"i")), Seq(Row(Seq(7, 8, 9, 1)), Row(Seq(2, 7, 9, 8, 5)), Row(Seq.empty), Row(null)) ) checkAnswer( @@ -1084,7 +1086,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { def testArrayOfPrimitiveTypeContainsNull(): Unit = { checkAnswer( - idfContainsNull.select(reverse('i)), + idfContainsNull.select(reverse($"i")), Seq(Row(Seq(7, null, 8, 9, 1)), Row(Seq(2, 7, 9, 8, 5, null)), Row(Seq.empty), Row(null)) ) checkAnswer( @@ -1110,7 +1112,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { def testArrayOfNonPrimitiveType(): Unit = { checkAnswer( - sdf.select(reverse('s)), + sdf.select(reverse($"s")), Seq(Row(Seq("b", "a", "c")), Row(Seq(null, "c", null, "b")), Row(Seq.empty), Row(null)) ) checkAnswer( @@ -1815,7 +1817,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ).toDF("i") def testArrayOfPrimitiveTypeNotContainsNull(): Unit = { - checkShuffleResult(idfNotContainsNull.select(shuffle('i))) + checkShuffleResult(idfNotContainsNull.select(shuffle($"i"))) checkShuffleResult(idfNotContainsNull.selectExpr("shuffle(i)")) } @@ -1835,7 +1837,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ).toDF("i") def testArrayOfPrimitiveTypeContainsNull(): Unit = { - checkShuffleResult(idfContainsNull.select(shuffle('i))) + checkShuffleResult(idfContainsNull.select(shuffle($"i"))) checkShuffleResult(idfContainsNull.selectExpr("shuffle(i)")) } @@ -1855,7 +1857,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ).toDF("s") def testNonPrimitiveType(): Unit = { - checkShuffleResult(sdf.select(shuffle('s))) + checkShuffleResult(sdf.select(shuffle($"s"))) checkShuffleResult(sdf.selectExpr("shuffle(s)")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 8e0cb839fbab5..ddc06603e45aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -47,13 +47,13 @@ class DataFrameJoinSuite extends QueryTest with SharedSparkSession { } test("join - sorted columns not in join's outputSet") { - val df = Seq((1, 2, "1"), (3, 4, "3")).toDF("int", "int2", "str_sort").as('df1) - val df2 = Seq((1, 3, "1"), (5, 6, "5")).toDF("int", "int2", "str").as('df2) - val df3 = Seq((1, 3, "1"), (5, 6, "5")).toDF("int", "int2", "str").as('df3) + val df = Seq((1, 2, "1"), (3, 4, "3")).toDF("int", "int2", "str_sort").as("df1") + val df2 = Seq((1, 3, "1"), (5, 6, "5")).toDF("int", "int2", "str").as("df2") + val df3 = Seq((1, 3, "1"), (5, 6, "5")).toDF("int", "int2", "str").as("df3") checkAnswer( df.join(df2, $"df1.int" === $"df2.int", "outer").select($"df1.int", $"df2.int2") - .orderBy('str_sort.asc, 'str.asc), + .orderBy(Symbol("str_sort").asc, Symbol("str").asc), Row(null, 6) :: Row(1, 3) :: Row(3, null) :: Nil) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala index bcd0c3f0d64a7..51c6a835d58d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -46,7 +46,7 @@ class DataFramePivotSuite extends QueryTest with SharedSparkSession { courseSales.groupBy("course").pivot("year", Seq(2012, 2013)).agg(sum($"earnings")), expected) checkAnswer( - courseSales.groupBy('course).pivot('year, Seq(2012, 2013)).agg(sum('earnings)), + courseSales.groupBy($"course").pivot($"year", Seq(2012, 2013)).agg(sum($"earnings")), expected) } @@ -206,7 +206,7 @@ class DataFramePivotSuite extends QueryTest with SharedSparkSession { complexData.groupBy().pivot("b", Seq(true, false)).agg(max("a")), expected) checkAnswer( - complexData.groupBy().pivot('b, Seq(true, false)).agg(max('a)), + complexData.groupBy().pivot($"b", Seq(true, false)).agg(max("a")), expected) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala index 59b5dacb10fdb..250ec7dc0ba5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala @@ -34,8 +34,8 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { } test("join - self join") { - val df1 = testData.select(testData("key")).as('df1) - val df2 = testData.select(testData("key")).as('df2) + val df1 = testData.select(testData("key")).as("df1") + val df2 = testData.select(testData("key")).as("df2") checkAnswer( df1.join(df2, $"df1.key" === $"df2.key"), @@ -57,11 +57,11 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession { test("join - using aliases after self join") { val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") checkAnswer( - df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count(), + df.as("x").join(df.as("y"), $"x.str" === $"y.str").groupBy("x.str").count(), Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) checkAnswer( - df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("y.str").count(), + df.as("x").join(df.as("y"), $"x.str" === $"y.str").groupBy("y.str").count(), Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala index fbb7e903c3450..bd3f48078374d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala @@ -307,7 +307,7 @@ class DataFrameSetOperationsSuite extends QueryTest with SharedSparkSession { val union = df1.union(df2) checkAnswer( - union.filter('i < rand(7) * 10), + union.filter($"i" < rand(7) * 10), expected(union) ) checkAnswer( @@ -321,13 +321,13 @@ class DataFrameSetOperationsSuite extends QueryTest with SharedSparkSession { val intersect = df1.intersect(df2) checkAnswer( - intersect.filter('i < rand(7) * 10), + intersect.filter($"i" < rand(7) * 10), expected(intersect) ) val except = df1.except(df2) checkAnswer( - except.filter('i < rand(7) * 10), + except.filter($"i" < rand(7) * 10), expected(except) ) } @@ -375,7 +375,7 @@ class DataFrameSetOperationsSuite extends QueryTest with SharedSparkSession { case j: Union if j.children.size == 5 => j }.size === 1) checkAnswer( - unionDF.agg(avg('key), max('key), min('key), sum('key)), + unionDF.agg(avg("key"), max("key"), min("key"), sum("key")), Row(50.5, 100, 1, 25250) :: Nil ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index 3b57173bd246b..394bad751b5ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -443,9 +443,9 @@ class DataFrameStatSuite extends QueryTest with SharedSparkSession { assert(sketch4.confidence() === 0.99 +- 5e-3) intercept[IllegalArgumentException] { - df.select('id cast DoubleType as 'id) + df.select($"id" cast DoubleType as "id") .stat - .countMinSketch('id, depth = 10, width = 20, seed = 42) + .countMinSketch($"id", depth = 10, width = 20, seed = 42) } } 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 5983fe63c79ec..13c2f9a810dcb 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 @@ -49,9 +49,9 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { import testImplicits._ test("analysis error should be eagerly reported") { - intercept[Exception] { testData.select('nonExistentName) } + intercept[Exception] { testData.select("nonExistentName") } intercept[Exception] { - testData.groupBy('key).agg(Map("nonExistentName" -> "sum")) + testData.groupBy("key").agg(Map("nonExistentName" -> "sum")) } intercept[Exception] { testData.groupBy("nonExistentName").agg(Map("key" -> "sum")) @@ -248,12 +248,12 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("repartition") { intercept[IllegalArgumentException] { - testData.select('key).repartition(0) + testData.select("key").repartition(0) } checkAnswer( - testData.select('key).repartition(10).select('key), - testData.select('key).collect().toSeq) + testData.select("key").repartition(10).select("key"), + testData.select("key").collect().toSeq) } test("repartition with SortOrder") { @@ -315,14 +315,14 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("coalesce") { intercept[IllegalArgumentException] { - testData.select('key).coalesce(0) + testData.select("key").coalesce(0) } - assert(testData.select('key).coalesce(1).rdd.partitions.size === 1) + assert(testData.select("key").coalesce(1).rdd.partitions.size === 1) checkAnswer( - testData.select('key).coalesce(1).select('key), - testData.select('key).collect().toSeq) + testData.select("key").coalesce(1).select("key"), + testData.select("key").collect().toSeq) assert(spark.emptyDataFrame.coalesce(1).rdd.partitions.size === 1) } @@ -335,7 +335,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("convert Scala Symbol 'attrname into unresolved attribute") { checkAnswer( - testData.where('key === lit(1)).select('value), + testData.where($"key" === lit(1)).select("value"), Row("1")) } @@ -347,17 +347,17 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("simple select") { checkAnswer( - testData.where('key === lit(1)).select('value), + testData.where($"key" === lit(1)).select("value"), Row("1")) } test("select with functions") { checkAnswer( - testData.select(sum('value), avg('value), count(lit(1))), + testData.select(sum("value"), avg("value"), count(lit(1))), Row(5050.0, 50.5, 100)) checkAnswer( - testData2.select('a + 'b, 'a < 'b), + testData2.select($"a" + $"b", $"a" < $"b"), Seq( Row(2, false), Row(3, true), @@ -367,31 +367,31 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { Row(5, false))) checkAnswer( - testData2.select(sumDistinct('a)), + testData2.select(sumDistinct($"a")), Row(6)) } test("sorting with null ordering") { val data = Seq[java.lang.Integer](2, 1, null).toDF("key") - checkAnswer(data.orderBy('key.asc), Row(null) :: Row(1) :: Row(2) :: Nil) + checkAnswer(data.orderBy($"key".asc), Row(null) :: Row(1) :: Row(2) :: Nil) checkAnswer(data.orderBy(asc("key")), Row(null) :: Row(1) :: Row(2) :: Nil) - checkAnswer(data.orderBy('key.asc_nulls_first), Row(null) :: Row(1) :: Row(2) :: Nil) + checkAnswer(data.orderBy($"key".asc_nulls_first), Row(null) :: Row(1) :: Row(2) :: Nil) checkAnswer(data.orderBy(asc_nulls_first("key")), Row(null) :: Row(1) :: Row(2) :: Nil) - checkAnswer(data.orderBy('key.asc_nulls_last), Row(1) :: Row(2) :: Row(null) :: Nil) + checkAnswer(data.orderBy($"key".asc_nulls_last), Row(1) :: Row(2) :: Row(null) :: Nil) checkAnswer(data.orderBy(asc_nulls_last("key")), Row(1) :: Row(2) :: Row(null) :: Nil) - checkAnswer(data.orderBy('key.desc), Row(2) :: Row(1) :: Row(null) :: Nil) + checkAnswer(data.orderBy($"key".desc), Row(2) :: Row(1) :: Row(null) :: Nil) checkAnswer(data.orderBy(desc("key")), Row(2) :: Row(1) :: Row(null) :: Nil) - checkAnswer(data.orderBy('key.desc_nulls_first), Row(null) :: Row(2) :: Row(1) :: Nil) + checkAnswer(data.orderBy($"key".desc_nulls_first), Row(null) :: Row(2) :: Row(1) :: Nil) checkAnswer(data.orderBy(desc_nulls_first("key")), Row(null) :: Row(2) :: Row(1) :: Nil) - checkAnswer(data.orderBy('key.desc_nulls_last), Row(2) :: Row(1) :: Row(null) :: Nil) + checkAnswer(data.orderBy($"key".desc_nulls_last), Row(2) :: Row(1) :: Row(null) :: Nil) checkAnswer(data.orderBy(desc_nulls_last("key")), Row(2) :: Row(1) :: Row(null) :: Nil) } test("global sorting") { checkAnswer( - testData2.orderBy('a.asc, 'b.asc), + testData2.orderBy($"a".asc, $"b".asc), Seq(Row(1, 1), Row(1, 2), Row(2, 1), Row(2, 2), Row(3, 1), Row(3, 2))) checkAnswer( @@ -399,31 +399,31 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { Seq(Row(1, 2), Row(1, 1), Row(2, 2), Row(2, 1), Row(3, 2), Row(3, 1))) checkAnswer( - testData2.orderBy('a.asc, 'b.desc), + testData2.orderBy($"a".asc, $"b".desc), Seq(Row(1, 2), Row(1, 1), Row(2, 2), Row(2, 1), Row(3, 2), Row(3, 1))) checkAnswer( - testData2.orderBy('a.desc, 'b.desc), + testData2.orderBy($"a".desc, $"b".desc), Seq(Row(3, 2), Row(3, 1), Row(2, 2), Row(2, 1), Row(1, 2), Row(1, 1))) checkAnswer( - testData2.orderBy('a.desc, 'b.asc), + testData2.orderBy($"a".desc, $"b".asc), Seq(Row(3, 1), Row(3, 2), Row(2, 1), Row(2, 2), Row(1, 1), Row(1, 2))) checkAnswer( - arrayData.toDF().orderBy('data.getItem(0).asc), + arrayData.toDF().orderBy($"data".getItem(0).asc), arrayData.toDF().collect().sortBy(_.getAs[Seq[Int]](0)(0)).toSeq) checkAnswer( - arrayData.toDF().orderBy('data.getItem(0).desc), + arrayData.toDF().orderBy($"data".getItem(0).desc), arrayData.toDF().collect().sortBy(_.getAs[Seq[Int]](0)(0)).reverse.toSeq) checkAnswer( - arrayData.toDF().orderBy('data.getItem(1).asc), + arrayData.toDF().orderBy($"data".getItem(1).asc), arrayData.toDF().collect().sortBy(_.getAs[Seq[Int]](0)(1)).toSeq) checkAnswer( - arrayData.toDF().orderBy('data.getItem(1).desc), + arrayData.toDF().orderBy($"data".getItem(1).desc), arrayData.toDF().collect().sortBy(_.getAs[Seq[Int]](0)(1)).reverse.toSeq) } @@ -452,7 +452,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { checkAnswer( // SELECT *, foo(key, value) FROM testData - testData.select($"*", foo('key, 'value)).limit(3), + testData.select($"*", foo($"key", $"value")).limit(3), Row(1, "1", "11") :: Row(2, "2", "22") :: Row(3, "3", "33") :: Nil ) } @@ -796,7 +796,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("apply on query results (SPARK-5462)") { val df = testData.sparkSession.sql("select key from testData") - checkAnswer(df.select(df("key")), testData.select('key).collect().toSeq) + checkAnswer(df.select(df("key")), testData.select("key").collect().toSeq) } test("inputFiles") { @@ -1199,7 +1199,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { } test("SPARK-6899: type should match when using codegen") { - checkAnswer(decimalData.agg(avg('a)), Row(new java.math.BigDecimal(2))) + checkAnswer(decimalData.agg(avg("a")), Row(new java.math.BigDecimal(2))) } test("SPARK-7133: Implement struct, array, and map field accessor") { @@ -1401,7 +1401,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("Sorting columns are not in Filter and Project") { checkAnswer( - upperCaseData.filter('N > 1).select('N).filter('N < 6).orderBy('L.asc), + upperCaseData.filter($"N" > 1).select("N").filter($"N" < 6).orderBy($"L".asc), Row(2) :: Row(3) :: Row(4) :: Row(5) :: Nil) } @@ -1444,17 +1444,17 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("Alias uses internally generated names 'aggOrder' and 'havingCondition'") { val df = Seq(1 -> 2).toDF("i", "j") - val query1 = df.groupBy('i) - .agg(max('j).as("aggOrder")) - .orderBy(sum('j)) + val query1 = df.groupBy("i") + .agg(max("j").as("aggOrder")) + .orderBy(sum("j")) checkAnswer(query1, Row(1, 2)) // In the plan, there are two attributes having the same name 'havingCondition' // One is a user-provided alias name; another is an internally generated one. - val query2 = df.groupBy('i) - .agg(max('j).as("havingCondition")) - .where(sum('j) > 0) - .orderBy('havingCondition.asc) + val query2 = df.groupBy("i") + .agg(max("j").as("havingCondition")) + .where(sum("j") > 0) + .orderBy($"havingCondition".asc) checkAnswer(query2, Row(1, 2)) } @@ -1463,7 +1463,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { (1 to 10).toDF("id").write.mode(SaveMode.Overwrite).json(dir.getCanonicalPath) val input = spark.read.json(dir.getCanonicalPath) - val df = input.select($"id", rand(0).as('r)) + val df = input.select($"id", rand(0).as("r")) df.as("a").join(df.filter($"r" < 0.5).as("b"), $"a.id" === $"b.id").collect().foreach { row => assert(row.getDouble(1) - row.getDouble(3) === 0.0 +- 0.001) } @@ -1751,7 +1751,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("assertAnalyzed shouldn't replace original stack trace") { val e = intercept[AnalysisException] { - spark.range(1).select('id as 'a, 'id as 'b).groupBy('a).agg('b) + spark.range(1).select($"id" as "a", $"id" as "b").groupBy("a").agg($"b") } assert(e.getStackTrace.head.getClassName != classOf[QueryExecution].getName) @@ -1984,14 +1984,14 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("order-by ordinal.") { checkAnswer( - testData2.select(lit(7), 'a, 'b).orderBy(lit(1), lit(2), lit(3)), + testData2.select(lit(7), $"a", $"b").orderBy(lit(1), lit(2), lit(3)), Seq(Row(7, 1, 1), Row(7, 1, 2), Row(7, 2, 1), Row(7, 2, 2), Row(7, 3, 1), Row(7, 3, 2))) } test("SPARK-22271: mean overflows and returns null for some decimal variables") { val d = 0.034567890 val df = Seq(d, d, d, d, d, d, d, d, d, d).toDF("DecimalCol") - val result = df.select('DecimalCol cast DecimalType(38, 33)) + val result = df.select($"DecimalCol" cast DecimalType(38, 33)) .select(col("DecimalCol")).describe() val mean = result.select("DecimalCol").where($"summary" === "mean") assert(mean.collect().toSet === Set(Row("0.0345678900000000000000000000000000000"))) @@ -2027,24 +2027,25 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { val sourceDF = spark.createDataFrame(rows, schema) def structWhenDF: DataFrame = sourceDF - .select(when('cond, struct(lit("a").as("val1"), lit(10).as("val2"))).otherwise('s) as "res") - .select('res.getField("val1")) + .select(when($"cond", + struct(lit("a").as("val1"), lit(10).as("val2"))).otherwise($"s") as "res") + .select($"res".getField("val1")) def arrayWhenDF: DataFrame = sourceDF - .select(when('cond, array(lit("a"), lit("b"))).otherwise('a) as "res") - .select('res.getItem(0)) + .select(when($"cond", array(lit("a"), lit("b"))).otherwise($"a") as "res") + .select($"res".getItem(0)) def mapWhenDF: DataFrame = sourceDF - .select(when('cond, map(lit(0), lit("a"))).otherwise('m) as "res") - .select('res.getItem(0)) + .select(when($"cond", map(lit(0), lit("a"))).otherwise($"m") as "res") + .select($"res".getItem(0)) def structIfDF: DataFrame = sourceDF .select(expr("if(cond, struct('a' as val1, 10 as val2), s)") as "res") - .select('res.getField("val1")) + .select($"res".getField("val1")) def arrayIfDF: DataFrame = sourceDF .select(expr("if(cond, array('a', 'b'), a)") as "res") - .select('res.getItem(0)) + .select($"res".getItem(0)) def mapIfDF: DataFrame = sourceDF .select(expr("if(cond, map(0, 'a'), m)") as "res") - .select('res.getItem(0)) + .select($"res".getItem(0)) def checkResult(): Unit = { checkAnswer(structWhenDF, Seq(Row("a"), Row(null))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 5db83c698ff15..4ada5077aec7f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -73,22 +73,22 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes // Simple filtering and partition pruning checkAnswer( - df.filter('a > 1 && 'p1 === 2), + df.filter($"a" > 1 && $"p1" === 2), for (i <- 2 to 3; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", 2, p2)) // Simple projection and filtering checkAnswer( - df.filter('a > 1).select('b, 'a + 1), + df.filter($"a" > 1).select($"b", $"a" + 1), for (i <- 2 to 3; _ <- 1 to 2; _ <- Seq("foo", "bar")) yield Row(s"val_$i", i + 1)) // Simple projection and partition pruning checkAnswer( - df.filter('a > 1 && 'p1 < 2).select('b, 'p1), + df.filter($"a" > 1 && $"p1" < 2).select($"b", $"p1"), for (i <- 2 to 3; _ <- Seq("foo", "bar")) yield Row(s"val_$i", 1)) // Project many copies of columns with different types (reproduction for SPARK-7858) checkAnswer( - df.filter('a > 1 && 'p1 < 2).select('b, 'b, 'b, 'b, 'p1, 'p1, 'p1, 'p1), + df.filter($"a" > 1 && $"p1" < 2).select($"b", $"b", $"b", $"b", $"p1", $"p1", $"p1", $"p1"), for (i <- 2 to 3; _ <- Seq("foo", "bar")) yield Row(s"val_$i", s"val_$i", s"val_$i", s"val_$i", 1, 1, 1, 1)) @@ -384,12 +384,12 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes test("saveAsTable()/load() - partitioned table - boolean type") { spark.range(2) - .select('id, ('id % 2 === 0).as("b")) + .select($"id", ($"id" % 2 === 0).as("b")) .write.partitionBy("b").saveAsTable("t") withTable("t") { checkAnswer( - spark.table("t").sort('id), + spark.table("t").sort($"id"), Row(0, true) :: Row(1, false) :: Nil ) } @@ -731,12 +731,12 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } yield (i, s"val_$i", 1.0d, p2, 123, 123.123f)).toDF("a", "b", "p1", "p2", "p3", "f") val input = df.select( - 'a, - 'b, - 'p1.cast(StringType).as('ps1), - 'p2, - 'p3.cast(FloatType).as('pf1), - 'f) + $"a", + $"b", + $"p1".cast(StringType).as("ps1"), + $"p2", + $"p3".cast(FloatType).as("pf1"), + $"f") withTempView("t") { input @@ -770,7 +770,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes .saveAsTable("t") withTable("t") { - checkAnswer(spark.table("t").select('b, 'c, 'a), df.select('b, 'c, 'a).collect()) + checkAnswer(spark.table("t").select("b", "c", "a"), df.select("b", "c", "a").collect()) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala index 6ebc1d145848c..2e6b86206a631 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala @@ -152,8 +152,8 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { withTempPath { dir => val path = dir.getCanonicalPath - spark.range(2).select('id as 'a, 'id as 'b).write.partitionBy("b").parquet(path) - val df = spark.read.parquet(path).filter('a === 0).select('b) + spark.range(2).select($"id" as "a", $"id" as "b").write.partitionBy("b").parquet(path) + val df = spark.read.parquet(path).filter($"a" === 0).select("b") val physicalPlan = df.queryExecution.sparkPlan assert(physicalPlan.collect { case p: execution.ProjectExec => p }.length === 1) From 755d8894485396b0a21304568c8ec5a55030f2fd Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 5 Dec 2019 16:32:33 -0800 Subject: [PATCH 0802/1387] [SPARK-24666][ML] Fix infinity vectors produced by Word2Vec when numIterations are large ### What changes were proposed in this pull request? This patch adds normalization to word vectors when fitting dataset in Word2Vec. ### Why are the changes needed? Running Word2Vec on some datasets, when numIterations is large, can produce infinity word vectors. ### Does this PR introduce any user-facing change? Yes. After this patch, Word2Vec won't produce infinity word vectors. ### How was this patch tested? Manually. This issue is not always reproducible on any dataset. The dataset known to reproduce it is too large (925M) to upload. ```scala case class Sentences(name: String, words: Array[String]) val dataset = spark.read .option("header", "true").option("sep", "\t") .option("quote", "").option("nullValue", "\\N") .csv("/tmp/title.akas.tsv") .filter("region = 'US' or language = 'en'") .select("title") .as[String] .map(s => Sentences(s, s.split(' '))) .persist() println("Training model...") val word2Vec = new Word2Vec() .setInputCol("words") .setOutputCol("vector") .setVectorSize(64) .setWindowSize(4) .setNumPartitions(50) .setMinCount(5) .setMaxIter(30) val model = word2Vec.fit(dataset) model.getVectors.show() ``` Before: ``` Training model... +-------------+--------------------+ | word| vector| +-------------+--------------------+ | Unspoken|[-Infinity,-Infin...| | Talent|[-Infinity,Infini...| | Hourglass|[2.02805806500023...| |Nickelodeon's|[-4.2918617120906...| | Priests|[-1.3570403355926...| | Religion:|[-6.7049072282803...| | Bu|[5.05591774315586...| | Totoro:|[-1.0539840178632...| | Trouble,|[-3.5363592836003...| | Hatter|[4.90413981352826...| | '79|[7.50436471285412...| | Vile|[-2.9147142985312...| | 9/11|[-Infinity,Infini...| | Santino|[1.30005911270850...| | Motives|[-1.2538958306253...| | '13|[-4.5040152427657...| | Fierce|[Infinity,Infinit...| | Stover|[-2.6326895394029...| | 'It|[1.66574533864436...| | Butts|[Infinity,Infinit...| +-------------+--------------------+ only showing top 20 rows ``` After: ``` Training model... +-------------+--------------------+ | word| vector| +-------------+--------------------+ | Unspoken|[-0.0454501919448...| | Talent|[-0.2657704949378...| | Hourglass|[-0.1399687677621...| |Nickelodeon's|[-0.1767119318246...| | Priests|[-0.0047509293071...| | Religion:|[-0.0411605164408...| | Bu|[0.11837736517190...| | Totoro:|[0.05258282646536...| | Trouble,|[0.09482011198997...| | Hatter|[0.06040831282734...| | '79|[0.04783720895648...| | Vile|[-0.0017210749210...| | 9/11|[-0.0713915303349...| | Santino|[-0.0412711687386...| | Motives|[-0.0492418706417...| | '13|[-0.0073119504377...| | Fierce|[-0.0565455369651...| | Stover|[0.06938160210847...| | 'It|[0.01117012929171...| | Butts|[0.05374567210674...| +-------------+--------------------+ only showing top 20 rows ``` Closes #26722 from viirya/SPARK-24666-2. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- .../apache/spark/mllib/feature/Word2Vec.scala | 17 ++++++++++++++--- .../apache/spark/ml/feature/Word2VecSuite.scala | 8 -------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 7888a8001d6b7..640412f601b50 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -439,9 +439,20 @@ class Word2Vec extends Serializable with Logging { } }.flatten } - val synAgg = partial.reduceByKey { case (v1, v2) => - blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) - v1 + // SPARK-24666: do normalization for aggregating weights from partitions. + // Original Word2Vec either single-thread or multi-thread which do Hogwild-style aggregation. + // Our approach needs to do extra normalization, otherwise adding weights continuously may + // cause overflow on float and lead to infinity/-infinity weights. + val synAgg = partial.mapPartitions { iter => + iter.map { case (id, vec) => + (id, (vec, 1)) + } + }.reduceByKey { case ((v1, count1), (v2, count2)) => + blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) + (v1, count1 + count2) + }.map { case (id, (vec, count)) => + blas.sscal(vectorSize, 1.0f / count, vec, 1) + (id, vec) }.collect() var i = 0 while (i < synAgg.length) { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index 11e1847ef235e..b00624b46670f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -78,14 +78,6 @@ class Word2VecSuite extends MLTest with DefaultReadWriteTest { test("getVectors") { val sentence = "a b " * 100 + "a c " * 10 val doc = sc.parallelize(Seq(sentence, sentence)).map(line => line.split(" ")) - - val codes = Map( - "a" -> Array(-0.2811822295188904, -0.6356269121170044, -0.3020961284637451), - "b" -> Array(1.0309048891067505, -1.29472815990448, 0.22276712954044342), - "c" -> Array(-0.08456747233867645, 0.5137411952018738, 0.11731560528278351) - ) - val expectedVectors = codes.toSeq.sortBy(_._1).map { case (w, v) => Vectors.dense(v) } - val docDF = doc.zip(doc).toDF("text", "alsotext") val model = new Word2Vec() From 25431d79f7daf2a68298701154eb505c2a4add80 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 5 Dec 2019 21:46:28 -0800 Subject: [PATCH 0803/1387] [SPARK-29953][SS] Don't clean up source files for FileStreamSource if the files belong to the output of FileStreamSink ### What changes were proposed in this pull request? This patch prevents the cleanup operation in FileStreamSource if the source files belong to the FileStreamSink. This is needed because the output of FileStreamSink can be read with multiple Spark queries and queries will read the files based on the metadata log, which won't reflect the cleanup. To simplify the logic, the patch only takes care of the case of when the source path without glob pattern refers to the output directory of FileStreamSink, via checking FileStreamSource to see whether it leverages metadata directory or not to list the source files. ### Why are the changes needed? Without this patch, if end users turn on cleanup option with the path which is the output of FileStreamSink, there may be out of sync between metadata and available files which may break other queries reading the path. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added UT. Closes #26590 from HeartSaVioR/SPARK-29953. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Shixiong Zhu --- .../structured-streaming-programming-guide.md | 2 +- .../streaming/FileStreamSource.scala | 17 +++- .../sql/streaming/FileStreamSourceSuite.scala | 83 +++++++++++++++---- 3 files changed, 81 insertions(+), 21 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 01679e5defe12..b91b93066d258 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -551,7 +551,7 @@ Here are the details of all the sources in Spark. When "archive" is provided, additional option sourceArchiveDir must be provided as well. The value of "sourceArchiveDir" must have 2 subdirectories (so depth of directory is greater than 2). e.g. /archived/here. This will ensure archived files are never included as new source files.
    Spark will move source files respecting their own path. For example, if the path of source file is /a/b/dataset.txt and the path of archive directory is /archived/here, file will be moved to /archived/here/a/b/dataset.txt.
    NOTE: Both archiving (via moving) or deleting completed files will introduce overhead (slow down) in each micro-batch, so you need to understand the cost for each operation in your file system before enabling this option. On the other hand, enabling this option will reduce the cost to list source files which can be an expensive operation.
    - NOTE 2: The source path should not be used from multiple sources or queries when enabling this option.
    + NOTE 2: The source path should not be used from multiple sources or queries when enabling this option. Similarly, you must ensure the source path doesn't match to any files in output directory of file stream sink.
    NOTE 3: Both delete and move actions are best effort. Failing to delete or move files will not fail the streaming query.

    For file-format-specific options, see the related methods in DataStreamReader diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 35d486c7c7437..f31fb32634a4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -206,6 +206,17 @@ class FileStreamSource( CaseInsensitiveMap(options), None).allFiles() } + private def setSourceHasMetadata(newValue: Option[Boolean]): Unit = newValue match { + case Some(true) => + if (sourceCleaner.isDefined) { + throw new UnsupportedOperationException("Clean up source files is not supported when" + + " reading from the output directory of FileStreamSink.") + } + sourceHasMetadata = Some(true) + case _ => + sourceHasMetadata = newValue + } + /** * Returns a list of files found, sorted by their timestamp. */ @@ -216,7 +227,7 @@ class FileStreamSource( sourceHasMetadata match { case None => if (FileStreamSink.hasMetadata(Seq(path), hadoopConf, sparkSession.sessionState.conf)) { - sourceHasMetadata = Some(true) + setSourceHasMetadata(Some(true)) allFiles = allFilesUsingMetadataLogFileIndex() } else { allFiles = allFilesUsingInMemoryFileIndex() @@ -228,10 +239,10 @@ class FileStreamSource( // metadata log and data files are only generated after the previous // `FileStreamSink.hasMetadata` check if (FileStreamSink.hasMetadata(Seq(path), hadoopConf, sparkSession.sessionState.conf)) { - sourceHasMetadata = Some(true) + setSourceHasMetadata(Some(true)) allFiles = allFilesUsingMetadataLogFileIndex() } else { - sourceHasMetadata = Some(false) + setSourceHasMetadata(Some(false)) // `allFiles` have already been fetched using InMemoryFileIndex in this round } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 1ef0ae878ec07..b8dac13b3842d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -29,6 +29,7 @@ import org.apache.hadoop.util.Progressable import org.scalatest.PrivateMethodTester import org.scalatest.time.SpanSugar._ +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ @@ -149,6 +150,20 @@ abstract class FileStreamSourceTest } } + case class AddFilesToFileStreamSinkLog( + fs: FileSystem, + srcDir: Path, + sinkLog: FileStreamSinkLog, + batchId: Int)( + pathFilter: Path => Boolean) extends ExternalAction { + override def runAction(): Unit = { + val statuses = fs.listStatus(srcDir, new PathFilter { + override def accept(path: Path): Boolean = pathFilter(path) + }) + sinkLog.add(batchId, statuses.map(SinkFileStatus(_))) + } + } + /** Use `format` and `path` to create FileStreamSource via DataFrameReader */ def createFileStream( format: String, @@ -1617,14 +1632,6 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } test("remove completed files when remove option is enabled") { - def assertFileIsRemoved(files: Array[String], fileName: String): Unit = { - assert(!files.exists(_.startsWith(fileName))) - } - - def assertFileIsNotRemoved(files: Array[String], fileName: String): Unit = { - assert(files.exists(_.startsWith(fileName))) - } - withTempDirs { case (src, tmp) => withSQLConf( SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "2", @@ -1642,28 +1649,24 @@ class FileStreamSourceSuite extends FileStreamSourceTest { CheckAnswer("keep1"), AssertOnQuery("input file removed") { _: StreamExecution => // it doesn't rename any file yet - assertFileIsNotRemoved(src.list(), "keep1") + assertFileIsNotRemoved(src, "keep1") true }, AddTextFileData("keep2", src, tmp, tmpFilePrefix = "ke ep2 %"), CheckAnswer("keep1", "keep2"), AssertOnQuery("input file removed") { _: StreamExecution => - val files = src.list() - // it renames input file for first batch, but not for second batch yet - assertFileIsRemoved(files, "keep1") - assertFileIsNotRemoved(files, "ke ep2 %") + assertFileIsRemoved(src, "keep1") + assertFileIsNotRemoved(src, "ke ep2 %") true }, AddTextFileData("keep3", src, tmp, tmpFilePrefix = "keep3"), CheckAnswer("keep1", "keep2", "keep3"), AssertOnQuery("input file renamed") { _: StreamExecution => - val files = src.list() - // it renames input file for second batch, but not third batch yet - assertFileIsRemoved(files, "ke ep2 %") - assertFileIsNotRemoved(files, "keep3") + assertFileIsRemoved(src, "ke ep2 %") + assertFileIsNotRemoved(src, "keep3") true } @@ -1739,6 +1742,44 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } + Seq("delete", "archive").foreach { cleanOption => + test(s"Throw UnsupportedOperationException on configuring $cleanOption when source path" + + " refers the output dir of FileStreamSink") { + withThreeTempDirs { case (src, tmp, archiveDir) => + withSQLConf( + SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "2", + // Force deleting the old logs + SQLConf.FILE_SOURCE_LOG_CLEANUP_DELAY.key -> "1" + ) { + val option = Map("latestFirst" -> "false", "maxFilesPerTrigger" -> "1", + "cleanSource" -> cleanOption, "sourceArchiveDir" -> archiveDir.getAbsolutePath) + + val fileStream = createFileStream("text", src.getCanonicalPath, options = option) + val filtered = fileStream.filter($"value" contains "keep") + + // create FileStreamSinkLog under source directory + val sinkLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, spark, + new File(src, FileStreamSink.metadataDir).getCanonicalPath) + val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) + val srcPath = new Path(src.getCanonicalPath) + val fileSystem = srcPath.getFileSystem(hadoopConf) + + // Here we will just check whether the source file is removed or not, as we cover + // functionality test of "archive" in other UT. + testStream(filtered)( + AddTextFileData("keep1", src, tmp, tmpFilePrefix = "keep1"), + AddFilesToFileStreamSinkLog(fileSystem, srcPath, sinkLog, 0) { path => + path.getName.startsWith("keep1") + }, + ExpectFailure[UnsupportedOperationException]( + t => assert(t.getMessage.startsWith("Clean up source files is not supported")), + isFatalError = false) + ) + } + } + } + } + class FakeFileSystem(scheme: String) extends FileSystem { override def exists(f: Path): Boolean = true @@ -1797,6 +1838,14 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } + private def assertFileIsRemoved(sourceDir: File, fileName: String): Unit = { + assert(!sourceDir.list().exists(_.startsWith(fileName))) + } + + private def assertFileIsNotRemoved(sourceDir: File, fileName: String): Unit = { + assert(sourceDir.list().exists(_.startsWith(fileName))) + } + private def assertFileIsNotMoved(sourceDir: File, expectedDir: File, filePrefix: String): Unit = { assert(sourceDir.exists()) assert(sourceDir.list().exists(_.startsWith(filePrefix))) From da27f91560df1d30258fe41cd568c4dbf0606254 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Thu, 5 Dec 2019 23:12:45 -0800 Subject: [PATCH 0804/1387] [SPARK-29957][TEST] Reset MiniKDC's default enctypes to fit jdk8/jdk11 ### What changes were proposed in this pull request? Hadoop jira: https://issues.apache.org/jira/browse/HADOOP-12911 In this jira, the author said to replace origin Apache Directory project which is not maintained (but not said it won't work well in jdk11) to Apache Kerby which is java binding(fit java version). And in Flink: https://github.com/apache/flink/pull/9622 Author show the reason why hadoop-2.7.2's `MminiKdc` failed with jdk11. Because new encryption types of `es128-cts-hmac-sha256-128` and `aes256-cts-hmac-sha384-192` (for Kerberos 5) enabled by default were added in Java 11. Spark with `hadoop-2.7's MiniKdc`does not support these encryption types and does not work well when these encryption types are enabled, which results in the authentication failure. And when I test hadoop-2.7.2's minikdc in local, the kerberos 's debug error message is read message stream failed, message can't match. ### Why are the changes needed? Support jdk11 with hadoop-2.7 ### Does this PR introduce any user-facing change? NO ### How was this patch tested? Existed UT Closes #26594 from AngersZhuuuu/minikdc-3.2.0. Lead-authored-by: angerszhu Co-authored-by: AngersZhuuuu Signed-off-by: Dongjoon Hyun --- .../spark/sql/kafka010/KafkaTestUtils.scala | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 6e97c37af7df0..47ae7be85ce02 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -25,6 +25,7 @@ import java.util.concurrent.TimeUnit import javax.security.auth.login.Configuration import scala.collection.JavaConverters._ +import scala.io.Source import scala.util.Random import com.google.common.io.Files @@ -136,9 +137,44 @@ class KafkaTestUtils( kdcConf.setProperty(MiniKdc.DEBUG, "true") kdc = new MiniKdc(kdcConf, kdcDir) kdc.start() + // TODO https://issues.apache.org/jira/browse/SPARK-30037 + // Need to build spark's own MiniKDC and customize krb5.conf like Kafka + rewriteKrb5Conf() kdcReady = true } + /** + * In this method we rewrite krb5.conf to make kdc and client use the same enctypes + */ + private def rewriteKrb5Conf(): Unit = { + val krb5Conf = Source.fromFile(kdc.getKrb5conf, "UTF-8").getLines() + var rewritten = false + val addedConfig = + addedKrb5Config("default_tkt_enctypes", "aes128-cts-hmac-sha1-96") + + addedKrb5Config("default_tgs_enctypes", "aes128-cts-hmac-sha1-96") + val rewriteKrb5Conf = krb5Conf.map(s => + if (s.contains("libdefaults")) { + rewritten = true + s + addedConfig + } else { + s + }).filter(!_.trim.startsWith("#")).mkString(System.lineSeparator()) + + val krb5confStr = if (!rewritten) { + "[libdefaults]" + addedConfig + System.lineSeparator() + + System.lineSeparator() + rewriteKrb5Conf + } else { + rewriteKrb5Conf + } + + kdc.getKrb5conf.delete() + Files.write(krb5confStr, kdc.getKrb5conf, StandardCharsets.UTF_8) + } + + private def addedKrb5Config(key: String, value: String): String = { + System.lineSeparator() + s" $key=$value" + } + private def createKeytabsAndJaasConfigFile(): String = { assert(kdcReady, "KDC should be set up beforehand") val baseDir = Utils.createTempDir() @@ -171,6 +207,7 @@ class KafkaTestUtils( | useKeyTab=true | storeKey=true | useTicketCache=false + | refreshKrb5Config=true | keyTab="${zkServerKeytabFile.getAbsolutePath()}" | principal="$zkServerUser@$realm"; |}; @@ -180,6 +217,7 @@ class KafkaTestUtils( | useKeyTab=true | storeKey=true | useTicketCache=false + | refreshKrb5Config=true | keyTab="${zkClientKeytabFile.getAbsolutePath()}" | principal="$zkClientUser@$realm"; |}; From a5ccbced8c1d6f41e5287269294642f32e445976 Mon Sep 17 00:00:00 2001 From: madianjun Date: Thu, 5 Dec 2019 23:39:49 -0800 Subject: [PATCH 0805/1387] [SPARK-30067][CORE] Fix fragment offset comparison in getBlockHosts MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? A bug fixed about the code in getBlockHosts() function. In the case "The fragment ends at a position within this block", the end of fragment should be before the end of block,where the "end of block" means `b.getOffset + b.getLength`,not `b.getLength`. ### Why are the changes needed? When comparing the fragment end and the block end,we should use fragment's `offset + length`,and then compare to the block's `b.getOffset + b.getLength`, not the block's length. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? No test. Closes #26650 from mdianjun/fix-getBlockHosts. Authored-by: madianjun Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/execution/PartitionedFileUtil.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala index 3196624f7c7c3..7dece29eb0212 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala @@ -64,13 +64,14 @@ object PartitionedFileUtil { offset: Long, length: Long): Array[String] = { val candidates = blockLocations.map { - // The fragment starts from a position within this block + // The fragment starts from a position within this block. It handles the case where the + // fragment is fully contained in the block. case b if b.getOffset <= offset && offset < b.getOffset + b.getLength => b.getHosts -> (b.getOffset + b.getLength - offset).min(length) // The fragment ends at a position within this block - case b if offset <= b.getOffset && offset + length < b.getLength => - b.getHosts -> (offset + length - b.getOffset).min(length) + case b if b.getOffset < offset + length && offset + length < b.getOffset + b.getLength => + b.getHosts -> (offset + length - b.getOffset) // The fragment fully contains this block case b if offset <= b.getOffset && b.getOffset + b.getLength <= offset + length => From b86d4bb9318b0ad89dc14e7a39bd0fcf1ddb2af6 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 6 Dec 2019 15:45:13 +0800 Subject: [PATCH 0806/1387] [SPARK-30001][SQL] ResolveRelations should handle both V1 and V2 tables ### What changes were proposed in this pull request? This PR makes `Analyzer.ResolveRelations` responsible for looking up both v1 and v2 tables from the session catalog and create an appropriate relation. ### Why are the changes needed? Currently there are two issues: 1. As described in [SPARK-29966](https://issues.apache.org/jira/browse/SPARK-29966), the logic for resolving relation can load a table twice, which is a perf regression (e.g., Hive metastore can be accessed twice). 2. As described in [SPARK-30001](https://issues.apache.org/jira/browse/SPARK-30001), if a catalog name is specified for v1 tables, the query fails: ``` scala> sql("create table t using csv as select 1 as i") res2: org.apache.spark.sql.DataFrame = [] scala> sql("select * from t").show +---+ | i| +---+ | 1| +---+ scala> sql("select * from spark_catalog.t").show org.apache.spark.sql.AnalysisException: Table or view not found: spark_catalog.t; line 1 pos 14; 'Project [*] +- 'UnresolvedRelation [spark_catalog, t] ``` ### Does this PR introduce any user-facing change? Yes. Now the catalog name is resolved correctly: ``` scala> sql("create table t using csv as select 1 as i") res0: org.apache.spark.sql.DataFrame = [] scala> sql("select * from t").show +---+ | i| +---+ | 1| +---+ scala> sql("select * from spark_catalog.t").show +---+ | i| +---+ | 1| +---+ ``` ### How was this patch tested? Added new tests. Closes #26684 from imback82/resolve_relation. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 167 +++++++++--------- .../sql/catalyst/catalog/SessionCatalog.scala | 36 ++-- .../sql/connector/catalog/LookupCatalog.scala | 22 +++ .../sql/connector/DataSourceV2SQLSuite.scala | 12 ++ .../command/PlanResolutionSuite.scala | 3 +- 5 files changed, 143 insertions(+), 97 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index ee1bb9a4c9d98..659e4a5c86ec1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -199,6 +199,7 @@ class Analyzer( new ResolveCatalogs(catalogManager) :: ResolveInsertInto :: ResolveRelations :: + ResolveTables :: ResolveReferences :: ResolveCreateNamedStruct :: ResolveDeserializer :: @@ -725,21 +726,29 @@ class Analyzer( } /** - * Resolve relations to temp views. This is not an actual rule, and is only called by - * [[ResolveTables]]. + * Resolve relations to temp views. This is not an actual rule, and is called by + * [[ResolveTables]] and [[ResolveRelations]]. */ object ResolveTempViews extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case u @ UnresolvedRelation(Seq(part1)) => - v1SessionCatalog.lookupTempView(part1).getOrElse(u) - case u @ UnresolvedRelation(Seq(part1, part2)) => - v1SessionCatalog.lookupGlobalTempView(part1, part2).getOrElse(u) + case u @ UnresolvedRelation(ident) => + lookupTempView(ident).getOrElse(u) + case i @ InsertIntoStatement(UnresolvedRelation(ident), _, _, _, _) => + lookupTempView(ident) + .map(view => i.copy(table = view)) + .getOrElse(i) } + + def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = + identifier match { + case Seq(part1) => v1SessionCatalog.lookupTempView(part1) + case Seq(part1, part2) => v1SessionCatalog.lookupGlobalTempView(part1, part2) + case _ => None + } } /** - * Resolve table relations with concrete relations from v2 catalog. This is not an actual rule, - * and is only called by [[ResolveRelations]]. + * Resolve table relations with concrete relations from v2 catalog. * * [[ResolveRelations]] still resolves v1 tables. */ @@ -772,6 +781,19 @@ class Analyzer( case u: UnresolvedV2Relation => CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) } + + /** + * Performs the lookup of DataSourceV2 Tables from v2 catalog. + */ + private def lookupV2Relation(identifier: Seq[String]): Option[DataSourceV2Relation] = + identifier match { + case NonSessionCatalogAndIdentifier(catalog, ident) => + CatalogV2Util.loadTable(catalog, ident) match { + case Some(table) => Some(DataSourceV2Relation.create(table)) + case None => None + } + case _ => None + } } /** @@ -779,9 +801,9 @@ class Analyzer( */ object ResolveRelations extends Rule[LogicalPlan] { - // If the unresolved relation is running directly on files, we just return the original - // UnresolvedRelation, the plan will get resolved later. Else we look up the table from catalog - // and change the default database name(in AnalysisContext) if it is a view. + // If an unresolved relation is given, it is looked up from the session catalog and either v1 + // or v2 relation is returned. Otherwise, we look up the table from catalog + // and change the default database name (in AnalysisContext) if it is a view. // We usually look up a table from the default database if the table identifier has an empty // database part, for a view the default database should be the currentDb when the view was // created. When the case comes to resolving a nested view, the view may have different default @@ -806,14 +828,8 @@ class Analyzer( // Note this is compatible with the views defined by older versions of Spark(before 2.2), which // have empty defaultDatabase and all the relations in viewText have database part defined. def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match { - case u @ UnresolvedRelation(AsTableIdentifier(ident)) if !isRunningDirectlyOnFiles(ident) => - val defaultDatabase = AnalysisContext.get.defaultDatabase - val foundRelation = lookupTableFromCatalog(ident, u, defaultDatabase) - if (foundRelation != u) { - resolveRelation(foundRelation) - } else { - u - } + case u @ UnresolvedRelation(SessionCatalogAndIdentifier(catalog, ident)) => + lookupRelation(catalog, ident, recurse = true).getOrElse(u) // The view's child should be a logical plan parsed from the `desc.viewText`, the variable // `viewText` should be defined, or else we throw an error on the generation of the View @@ -836,47 +852,65 @@ class Analyzer( case _ => plan } - def apply(plan: LogicalPlan): LogicalPlan = ResolveTables(plan).resolveOperatorsUp { - case i @ InsertIntoStatement(u @ UnresolvedRelation(AsTableIdentifier(ident)), _, child, _, _) - if child.resolved => - EliminateSubqueryAliases(lookupTableFromCatalog(ident, u)) match { + def apply(plan: LogicalPlan): LogicalPlan = ResolveTempViews(plan).resolveOperatorsUp { + case i @ InsertIntoStatement(table, _, _, _, _) if i.query.resolved => + val relation = table match { + case u @ UnresolvedRelation(SessionCatalogAndIdentifier(catalog, ident)) => + lookupRelation(catalog, ident, recurse = false).getOrElse(u) + case other => other + } + + EliminateSubqueryAliases(relation) match { case v: View => - u.failAnalysis(s"Inserting into a view is not allowed. View: ${v.desc.identifier}.") + table.failAnalysis(s"Inserting into a view is not allowed. View: ${v.desc.identifier}.") case other => i.copy(table = other) } + case u: UnresolvedRelation => resolveRelation(u) } - // Look up the table with the given name from catalog. The database we used is decided by the - // precedence: - // 1. Use the database part of the table identifier, if it is defined; - // 2. Use defaultDatabase, if it is defined(In this case, no temporary objects can be used, - // and the default database is only used to look up a view); - // 3. Use the currentDb of the SessionCatalog. - private def lookupTableFromCatalog( - tableIdentifier: TableIdentifier, - u: UnresolvedRelation, - defaultDatabase: Option[String] = None): LogicalPlan = { - val tableIdentWithDb = tableIdentifier.copy( - database = tableIdentifier.database.orElse(defaultDatabase)) - try { - v1SessionCatalog.lookupRelation(tableIdentWithDb) - } catch { - case _: NoSuchTableException | _: NoSuchDatabaseException => - u + // Look up a relation from the given session catalog with the following logic: + // 1) If a relation is not found in the catalog, return None. + // 2) If a v1 table is found, create a v1 relation. Otherwise, create a v2 relation. + // If recurse is set to true, it will call `resolveRelation` recursively to resolve + // relations with the correct database scope. + private def lookupRelation( + catalog: CatalogPlugin, + ident: Identifier, + recurse: Boolean): Option[LogicalPlan] = { + val newIdent = withNewNamespace(ident) + assert(newIdent.namespace.size == 1) + + CatalogV2Util.loadTable(catalog, newIdent) match { + case Some(v1Table: V1Table) => + val tableIdent = TableIdentifier(newIdent.name, newIdent.namespace.headOption) + val relation = v1SessionCatalog.getRelation(v1Table.v1Table) + if (recurse) { + Some(resolveRelation(relation)) + } else { + Some(relation) + } + case Some(table) => + Some(DataSourceV2Relation.create(table)) + case None => None } } - // If the database part is specified, and we support running SQL directly on files, and - // it's not a temporary view, and the table does not exist, then let's just return the - // original UnresolvedRelation. It is possible we are matching a query like "select * - // from parquet.`/path/to/query`". The plan will get resolved in the rule `ResolveDataSource`. - // Note that we are testing (!db_exists || !table_exists) because the catalog throws - // an exception from tableExists if the database does not exist. - private def isRunningDirectlyOnFiles(table: TableIdentifier): Boolean = { - table.database.isDefined && conf.runSQLonFile && !v1SessionCatalog.isTemporaryTable(table) && - (!v1SessionCatalog.databaseExists(table.database.get) - || !v1SessionCatalog.tableExists(table)) + // The namespace used for lookup is decided by the following precedence: + // 1. Use the existing namespace if it is defined. + // 2. Use defaultDatabase fom AnalysisContext, if it is defined. In this case, no temporary + // objects can be used, and the default database is only used to look up a view. + // 3. Use the current namespace of the session catalog. + private def withNewNamespace(ident: Identifier): Identifier = { + if (ident.namespace.nonEmpty) { + ident + } else { + val defaultNamespace = AnalysisContext.get.defaultDatabase match { + case Some(db) => Array(db) + case None => Array(v1SessionCatalog.getCurrentDatabase) + } + Identifier.of(defaultNamespace, ident.name) + } } } @@ -2898,37 +2932,6 @@ class Analyzer( } } } - - /** - * Performs the lookup of DataSourceV2 Tables. The order of resolution is: - * 1. Check if this relation is a temporary table. - * 2. Check if it has a catalog identifier. Here we try to load the table. - * If we find the table, return the v2 relation and catalog. - * 3. Try resolving the relation using the V2SessionCatalog if that is defined. - * If the V2SessionCatalog returns a V1 table definition, - * return `None` so that we can fallback to the V1 code paths. - * If the V2SessionCatalog returns a V2 table, return the v2 relation and V2SessionCatalog. - */ - private def lookupV2RelationAndCatalog( - identifier: Seq[String]): Option[(DataSourceV2Relation, CatalogPlugin, Identifier)] = - identifier match { - case CatalogObjectIdentifier(catalog, ident) if !CatalogV2Util.isSessionCatalog(catalog) => - CatalogV2Util.loadTable(catalog, ident) match { - case Some(table) => Some((DataSourceV2Relation.create(table), catalog, ident)) - case None => None - } - case CatalogObjectIdentifier(catalog, ident) if CatalogV2Util.isSessionCatalog(catalog) => - CatalogV2Util.loadTable(catalog, ident) match { - case Some(_: V1Table) => None - case Some(table) => - Some((DataSourceV2Relation.create(table), catalog, ident)) - case None => None - } - case _ => None - } - - private def lookupV2Relation(identifier: Seq[String]): Option[DataSourceV2Relation] = - lookupV2RelationAndCatalog(identifier).map(_._1) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 96ca1ac73e043..c05886a407ec3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -747,26 +747,34 @@ class SessionCatalog( }.getOrElse(throw new NoSuchTableException(db, table)) } else if (name.database.isDefined || !tempViews.contains(table)) { val metadata = externalCatalog.getTable(db, table) - if (metadata.tableType == CatalogTableType.VIEW) { - val viewText = metadata.viewText.getOrElse(sys.error("Invalid view without text.")) - logDebug(s"'$viewText' will be used for the view($table).") - // The relation is a view, so we wrap the relation by: - // 1. Add a [[View]] operator over the relation to keep track of the view desc; - // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the name of the view. - val child = View( - desc = metadata, - output = metadata.schema.toAttributes, - child = parser.parsePlan(viewText)) - SubqueryAlias(table, db, child) - } else { - SubqueryAlias(table, db, UnresolvedCatalogRelation(metadata)) - } + getRelation(metadata) } else { SubqueryAlias(table, tempViews(table)) } } } + def getRelation(metadata: CatalogTable): LogicalPlan = { + val name = metadata.identifier + val db = formatDatabaseName(name.database.getOrElse(currentDb)) + val table = formatTableName(name.table) + + if (metadata.tableType == CatalogTableType.VIEW) { + val viewText = metadata.viewText.getOrElse(sys.error("Invalid view without text.")) + logDebug(s"'$viewText' will be used for the view($table).") + // The relation is a view, so we wrap the relation by: + // 1. Add a [[View]] operator over the relation to keep track of the view desc; + // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the name of the view. + val child = View( + desc = metadata, + output = metadata.schema.toAttributes, + child = parser.parsePlan(viewText)) + SubqueryAlias(table, db, child) + } else { + SubqueryAlias(table, db, UnresolvedCatalogRelation(metadata)) + } + } + def lookupTempView(table: String): Option[SubqueryAlias] = { val formattedTable = formatTableName(table) getTempView(formattedTable).map { view => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 613c0d1797cc6..a37d37fd27a96 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -65,6 +65,28 @@ private[sql] trait LookupCatalog extends Logging { } } + /** + * Extract session catalog and identifier from a multi-part identifier. + */ + object SessionCatalogAndIdentifier { + def unapply(parts: Seq[String]): Option[(CatalogPlugin, Identifier)] = parts match { + case CatalogObjectIdentifier(catalog, ident) if CatalogV2Util.isSessionCatalog(catalog) => + Some(catalog, ident) + case _ => None + } + } + + /** + * Extract non-session catalog and identifier from a multi-part identifier. + */ + object NonSessionCatalogAndIdentifier { + def unapply(parts: Seq[String]): Option[(CatalogPlugin, Identifier)] = parts match { + case CatalogObjectIdentifier(catalog, ident) if !CatalogV2Util.isSessionCatalog(catalog) => + Some(catalog, ident) + case _ => None + } + } + /** * Extract catalog and namespace from a multi-part identifier with the current catalog if needed. * Catalog name takes precedence over namespaces. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 629fd28414c54..e2f8776682bb8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1813,6 +1813,18 @@ class DataSourceV2SQLSuite } } + test("SPARK-30001: session catalog name can be specified in SQL statements") { + // unset this config to use the default v2 session catalog. + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + + withTable("t") { + sql("CREATE TABLE t USING json AS SELECT 1 AS i") + checkAnswer(sql("select * from t"), Row(1)) + checkAnswer(sql("select * from spark_catalog.t"), Row(1)) + checkAnswer(sql("select * from spark_catalog.default.t"), Row(1)) + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index d5c65a7c1e1b6..49cacbea6dd17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -141,7 +141,8 @@ class PlanResolutionSuite extends AnalysisTest { CTESubstitution, new ResolveCatalogs(catalogManager), new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), - analyzer.ResolveTables) + analyzer.ResolveTables, + analyzer.ResolveRelations) rules.foldLeft(parsePlan(query)) { case (plan, rule) => rule.apply(plan) } From 187f3c17733f94aa3372caca355ad18ec1198f2f Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Fri, 6 Dec 2019 00:07:38 -0800 Subject: [PATCH 0807/1387] [SPARK-28083][SQL] Support LIKE ... ESCAPE syntax ## What changes were proposed in this pull request? The syntax 'LIKE predicate: ESCAPE clause' is a ANSI SQL. For example: ``` select 'abcSpark_13sd' LIKE '%Spark\\_%'; //true select 'abcSpark_13sd' LIKE '%Spark/_%'; //false select 'abcSpark_13sd' LIKE '%Spark"_%'; //false select 'abcSpark_13sd' LIKE '%Spark/_%' ESCAPE '/'; //true select 'abcSpark_13sd' LIKE '%Spark"_%' ESCAPE '"'; //true select 'abcSpark%13sd' LIKE '%Spark\\%%'; //true select 'abcSpark%13sd' LIKE '%Spark/%%'; //false select 'abcSpark%13sd' LIKE '%Spark"%%'; //false select 'abcSpark%13sd' LIKE '%Spark/%%' ESCAPE '/'; //true select 'abcSpark%13sd' LIKE '%Spark"%%' ESCAPE '"'; //true select 'abcSpark\\13sd' LIKE '%Spark\\\\_%'; //true select 'abcSpark/13sd' LIKE '%Spark//_%'; //false select 'abcSpark"13sd' LIKE '%Spark""_%'; //false select 'abcSpark/13sd' LIKE '%Spark//_%' ESCAPE '/'; //true select 'abcSpark"13sd' LIKE '%Spark""_%' ESCAPE '"'; //true ``` But Spark SQL only supports 'LIKE predicate'. Note: If the input string or pattern string is null, then the result is null too. There are some mainstream database support the syntax. **PostgreSQL:** https://www.postgresql.org/docs/11/functions-matching.html **Vertica:** https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Predicates/LIKE-predicate.htm?zoom_highlight=like%20escape **MySQL:** https://dev.mysql.com/doc/refman/5.6/en/string-comparison-functions.html **Oracle:** https://docs.oracle.com/en/database/oracle/oracle-database/19/jjdbc/JDBC-reference-information.html#GUID-5D371A5B-D7F6-42EB-8C0D-D317F3C53708 https://docs.oracle.com/en/database/oracle/oracle-database/19/sqlrf/Pattern-matching-Conditions.html#GUID-0779657B-06A8-441F-90C5-044B47862A0A ## How was this patch tested? Exists UT and new UT. This PR merged to my production environment and runs above sql: ``` spark-sql> select 'abcSpark_13sd' LIKE '%Spark\\_%'; true Time taken: 0.119 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark_13sd' LIKE '%Spark/_%'; false Time taken: 0.103 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark_13sd' LIKE '%Spark"_%'; false Time taken: 0.096 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark_13sd' LIKE '%Spark/_%' ESCAPE '/'; true Time taken: 0.096 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark_13sd' LIKE '%Spark"_%' ESCAPE '"'; true Time taken: 0.092 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark\\%%'; true Time taken: 0.109 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark/%%'; false Time taken: 0.1 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark"%%'; false Time taken: 0.081 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark/%%' ESCAPE '/'; true Time taken: 0.095 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark%13sd' LIKE '%Spark"%%' ESCAPE '"'; true Time taken: 0.113 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark\\13sd' LIKE '%Spark\\\\_%'; true Time taken: 0.078 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark/13sd' LIKE '%Spark//_%'; false Time taken: 0.067 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark"13sd' LIKE '%Spark""_%'; false Time taken: 0.084 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark/13sd' LIKE '%Spark//_%' ESCAPE '/'; true Time taken: 0.091 seconds, Fetched 1 row(s) spark-sql> select 'abcSpark"13sd' LIKE '%Spark""_%' ESCAPE '"'; true Time taken: 0.091 seconds, Fetched 1 row(s) ``` I create a table and its schema is: ``` spark-sql> desc formatted gja_test; key string NULL value string NULL other string NULL # Detailed Table Information Database test Table gja_test Owner test Created Time Wed Apr 10 11:06:15 CST 2019 Last Access Thu Jan 01 08:00:00 CST 1970 Created By Spark 2.4.1-SNAPSHOT Type MANAGED Provider hive Table Properties [transient_lastDdlTime=1563443838] Statistics 26 bytes Location hdfs://namenode.xxx:9000/home/test/hive/warehouse/test.db/gja_test Serde Library org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe InputFormat org.apache.hadoop.mapred.TextInputFormat OutputFormat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat Storage Properties [field.delim= , serialization.format= ] Partition Provider Catalog Time taken: 0.642 seconds, Fetched 21 row(s) ``` Table `gja_test` exists three rows of data. ``` spark-sql> select * from gja_test; a A ao b B bo "__ """__ " Time taken: 0.665 seconds, Fetched 3 row(s) ``` At finally, I test this function: ``` spark-sql> select * from gja_test where key like value escape '"'; "__ """__ " Time taken: 0.687 seconds, Fetched 1 row(s) ``` Closes #25001 from beliefer/ansi-sql-like. Lead-authored-by: gengjiaan Co-authored-by: Jiaan Geng Signed-off-by: Gengliang Wang --- docs/sql-keywords.md | 1 + .../spark/sql/catalyst/parser/SqlBase.g4 | 5 +- .../spark/sql/catalyst/dsl/package.scala | 3 +- .../expressions/regexpExpressions.scala | 33 +++++--- .../sql/catalyst/optimizer/expressions.scala | 5 +- .../sql/catalyst/parser/AstBuilder.scala | 9 ++- .../spark/sql/catalyst/util/StringUtils.scala | 10 ++- .../expressions/RegexpExpressionsSuite.scala | 78 +++++++++++++++++++ .../parser/ExpressionParserSuite.scala | 12 +++ .../parser/TableIdentifierParserSuite.scala | 2 + .../sql/catalyst/util/StringUtilsSuite.scala | 35 +++++++-- .../sql/dynamicpruning/PartitionPruning.scala | 2 +- 12 files changed, 167 insertions(+), 28 deletions(-) diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 3117ee40a8c9b..9ad6984913a27 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -104,6 +104,7 @@ Below is a list of all the keywords in Spark SQL. DROPnon-reservednon-reservedreserved ELSEreservednon-reservedreserved ENDreservednon-reservedreserved + ESCAPEreservednon-reservedreserved ESCAPEDnon-reservednon-reservednon-reserved EXCEPTreservedstrict-non-reservedreserved EXCHANGEnon-reservednon-reservednon-reserved diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index abaaecf69f619..21c38b604199f 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -724,7 +724,8 @@ predicate : NOT? kind=BETWEEN lower=valueExpression AND upper=valueExpression | NOT? kind=IN '(' expression (',' expression)* ')' | NOT? kind=IN '(' query ')' - | NOT? kind=(RLIKE | LIKE) pattern=valueExpression + | NOT? kind=RLIKE pattern=valueExpression + | NOT? kind=LIKE pattern=valueExpression (ESCAPE escapeChar=STRING)? | IS NOT? kind=NULL | IS NOT? kind=(TRUE | FALSE | UNKNOWN) | IS NOT? kind=DISTINCT FROM right=valueExpression @@ -1265,6 +1266,7 @@ nonReserved | DROP | ELSE | END + | ESCAPE | ESCAPED | EXCHANGE | EXISTS @@ -1525,6 +1527,7 @@ DISTRIBUTE: 'DISTRIBUTE'; DROP: 'DROP'; ELSE: 'ELSE'; END: 'END'; +ESCAPE: 'ESCAPE'; ESCAPED: 'ESCAPED'; EXCEPT: 'EXCEPT'; EXCHANGE: 'EXCHANGE'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 18753f29b32c0..b4a8bafe22dfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -98,7 +98,8 @@ package object dsl { case _ => In(expr, list) } - def like(other: Expression): Expression = Like(expr, other) + def like(other: Expression, escapeChar: Char = '\\'): Expression = + Like(expr, other, escapeChar) def rlike(other: Expression): Expression = RLike(expr, other) def contains(other: Expression): Expression = Contains(expr, other) def startsWith(other: Expression): Expression = StartsWith(expr, other) 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 f52e903e75094..6ff6dc729795d 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 @@ -70,8 +70,8 @@ abstract class StringRegexExpression extends BinaryExpression * Simple RegEx pattern matching function */ @ExpressionDescription( - usage = "str _FUNC_ pattern - Returns true if str matches pattern, " + - "null if any arguments are null, false otherwise.", + usage = "str _FUNC_ pattern[ ESCAPE escape] - Returns true if str matches `pattern` with " + + "`escape`, null if any arguments are null, false otherwise.", arguments = """ Arguments: * str - a string expression @@ -83,16 +83,15 @@ abstract class StringRegexExpression extends BinaryExpression % matches zero or more characters in the input (similar to .* in posix regular expressions) - The escape character is '\'. If an escape character precedes a special symbol or another - escape character, the following character is matched literally. It is invalid to escape - any other character. - Since Spark 2.0, string literals are unescaped in our SQL parser. For example, in order to match "\abc", the pattern should be "\\abc". When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, it fallbacks to Spark 1.6 behavior regarding string literal parsing. For example, if the config is enabled, the pattern to match "\abc" should be "\abc". + * escape - an character added since Spark 3.0. The default escape character is the '\'. + If an escape character precedes a special symbol or another escape character, the + following character is matched literally. It is invalid to escape any other character. """, examples = """ Examples: @@ -104,19 +103,25 @@ abstract class StringRegexExpression extends BinaryExpression spark.sql.parser.escapedStringLiterals false > SELECT '%SystemDrive%\\Users\\John' _FUNC_ '\%SystemDrive\%\\\\Users%'; true + > SELECT '%SystemDrive%/Users/John' _FUNC_ '/%SystemDrive/%//Users%' ESCAPE '/'; + true """, note = """ Use RLIKE to match with standard regular expressions. """, since = "1.0.0") // scalastyle:on line.contains.tab -case class Like(left: Expression, right: Expression) extends StringRegexExpression { +case class Like(left: Expression, right: Expression, escapeChar: Char = '\\') + extends StringRegexExpression { - override def escape(v: String): String = StringUtils.escapeLikeRegex(v) + override def escape(v: String): String = StringUtils.escapeLikeRegex(v, escapeChar) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() - override def toString: String = s"$left LIKE $right" + override def toString: String = escapeChar match { + case '\\' => s"$left LIKE $right" + case c => s"$left LIKE $right ESCAPE '$c'" + } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val patternClass = classOf[Pattern].getName @@ -149,10 +154,18 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi } else { val pattern = ctx.freshName("pattern") val rightStr = ctx.freshName("rightStr") + // We need double escape to avoid org.codehaus.commons.compiler.CompileException. + // '\\' will cause exception 'Single quote must be backslash-escaped in character literal'. + // '\"' will cause exception 'Line break in literal not allowed'. + val newEscapeChar = if (escapeChar == '\"' || escapeChar == '\\') { + s"""\\\\\\$escapeChar""" + } else { + escapeChar + } nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" String $rightStr = $eval2.toString(); - $patternClass $pattern = $patternClass.compile($escapeFunc($rightStr)); + $patternClass $pattern = $patternClass.compile($escapeFunc($rightStr, '$newEscapeChar')); ${ev.value} = $pattern.matcher($eval1.toString()).matches(); """ }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 36ad796c08a38..98acad8a7413a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -484,7 +484,7 @@ object LikeSimplification extends Rule[LogicalPlan] { private val equalTo = "([^_%]*)".r def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case Like(input, Literal(pattern, StringType)) => + case Like(input, Literal(pattern, StringType), escapeChar) => if (pattern == null) { // If pattern is null, return null value directly, since "col like null" == null. Literal(null, BooleanType) @@ -503,8 +503,7 @@ object LikeSimplification extends Rule[LogicalPlan] { Contains(input, Literal(infix)) case equalTo(str) => EqualTo(input, Literal(str)) - case _ => - Like(input, Literal.create(pattern, StringType)) + case _ => Like(input, Literal.create(pattern, StringType), escapeChar) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 047cc22fe5641..78164279d8f48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1386,7 +1386,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case SqlBaseParser.IN => invertIfNotDefined(In(e, ctx.expression.asScala.map(expression))) case SqlBaseParser.LIKE => - invertIfNotDefined(Like(e, expression(ctx.pattern))) + val escapeChar = Option(ctx.escapeChar).map(string).map { str => + if (str.length != 1) { + throw new ParseException("Invalid escape string." + + "Escape string must contains only one character.", ctx) + } + str.charAt(0) + }.getOrElse('\\') + invertIfNotDefined(Like(e, expression(ctx.pattern), escapeChar)) case SqlBaseParser.RLIKE => invertIfNotDefined(RLike(e, expression(ctx.pattern))) case SqlBaseParser.NULL if ctx.NOT != null => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index 3bda9a2a1fc48..d6a649b791fb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -39,9 +39,10 @@ object StringUtils extends Logging { * throw an [[AnalysisException]]. * * @param pattern the SQL pattern to convert + * @param escapeStr the escape string contains one character. * @return the equivalent Java regular expression of the pattern */ - def escapeLikeRegex(pattern: String): String = { + def escapeLikeRegex(pattern: String, escapeChar: Char): String = { val in = pattern.toIterator val out = new StringBuilder() @@ -50,13 +51,14 @@ object StringUtils extends Logging { while (in.hasNext) { in.next match { - case '\\' if in.hasNext => + case c1 if c1 == escapeChar && in.hasNext => val c = in.next c match { - case '_' | '%' | '\\' => out ++= Pattern.quote(Character.toString(c)) + case '_' | '%' => out ++= Pattern.quote(Character.toString(c)) + case c if c == escapeChar => out ++= Pattern.quote(Character.toString(c)) case _ => fail(s"the escape character is not allowed to precede '$c'") } - case '\\' => fail("it is not allowed to end with the escape character") + case c if c == escapeChar => fail("it is not allowed to end with the escape character") case '_' => out ++= "." case '%' => out ++= ".*" case c => out ++= Pattern.quote(Character.toString(c)) 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 06fb73ad83923..2c8794f083dbb 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 @@ -118,6 +118,84 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkLiteralRow("""%SystemDrive%\Users\John""" like _, """\%SystemDrive\%\\Users%""", true) } + Seq('/', '#', '\"').foreach { escapeChar => + test(s"LIKE Pattern ESCAPE '$escapeChar'") { + // null handling + checkLiteralRow(Literal.create(null, StringType).like(_, escapeChar), "a", null) + checkEvaluation( + Literal.create("a", StringType).like(Literal.create(null, StringType), escapeChar), null) + checkEvaluation( + Literal.create(null, StringType).like(Literal.create(null, StringType), escapeChar), null) + checkEvaluation(Literal.create("a", StringType).like( + NonFoldableLiteral.create("a", StringType), escapeChar), true) + checkEvaluation(Literal.create("a", StringType).like( + NonFoldableLiteral.create(null, StringType), escapeChar), null) + checkEvaluation(Literal.create(null, StringType).like( + NonFoldableLiteral.create("a", StringType), escapeChar), null) + checkEvaluation(Literal.create(null, StringType).like( + NonFoldableLiteral.create(null, StringType), escapeChar), null) + + // simple patterns + checkLiteralRow("abdef" like(_, escapeChar), "abdef", true) + checkLiteralRow("a_%b" like(_, escapeChar), s"a${escapeChar}__b", true) + checkLiteralRow("addb" like(_, escapeChar), "a_%b", true) + checkLiteralRow("addb" like(_, escapeChar), s"a${escapeChar}__b", false) + checkLiteralRow("addb" like(_, escapeChar), s"a%$escapeChar%b", false) + checkLiteralRow("a_%b" like(_, escapeChar), s"a%$escapeChar%b", true) + checkLiteralRow("addb" like(_, escapeChar), "a%", true) + checkLiteralRow("addb" like(_, escapeChar), "**", false) + checkLiteralRow("abc" like(_, escapeChar), "a%", true) + checkLiteralRow("abc" like(_, escapeChar), "b%", false) + checkLiteralRow("abc" like(_, escapeChar), "bc%", false) + checkLiteralRow("a\nb" like(_, escapeChar), "a_b", true) + checkLiteralRow("ab" like(_, escapeChar), "a%b", true) + checkLiteralRow("a\nb" like(_, escapeChar), "a%b", true) + + // empty input + checkLiteralRow("" like(_, escapeChar), "", true) + checkLiteralRow("a" like(_, escapeChar), "", false) + checkLiteralRow("" like(_, escapeChar), "a", false) + + // SI-17647 double-escaping backslash + checkLiteralRow(s"""$escapeChar$escapeChar$escapeChar$escapeChar""" like(_, escapeChar), + s"""%$escapeChar$escapeChar%""", true) + checkLiteralRow("""%%""" like(_, escapeChar), """%%""", true) + checkLiteralRow(s"""${escapeChar}__""" like(_, escapeChar), + s"""$escapeChar$escapeChar${escapeChar}__""", true) + checkLiteralRow(s"""$escapeChar$escapeChar${escapeChar}__""" like(_, escapeChar), + s"""%$escapeChar$escapeChar%$escapeChar%""", false) + checkLiteralRow(s"""_$escapeChar$escapeChar$escapeChar%""" like(_, escapeChar), + s"""%$escapeChar${escapeChar}""", false) + + // unicode + // scalastyle:off nonascii + checkLiteralRow("a\u20ACa" like(_, escapeChar), "_\u20AC_", true) + checkLiteralRow("a€a" like(_, escapeChar), "_€_", true) + checkLiteralRow("a€a" like(_, escapeChar), "_\u20AC_", true) + checkLiteralRow("a\u20ACa" like(_, escapeChar), "_€_", true) + // scalastyle:on nonascii + + // invalid escaping + val invalidEscape = intercept[AnalysisException] { + evaluateWithoutCodegen("""a""" like(s"""${escapeChar}a""", escapeChar)) + } + assert(invalidEscape.getMessage.contains("pattern")) + val endEscape = intercept[AnalysisException] { + evaluateWithoutCodegen("""a""" like(s"""a$escapeChar""", escapeChar)) + } + assert(endEscape.getMessage.contains("pattern")) + + // case + checkLiteralRow("A" like(_, escapeChar), "a%", false) + checkLiteralRow("a" like(_, escapeChar), "A%", false) + checkLiteralRow("AaA" like(_, escapeChar), "_a_", true) + + // example + checkLiteralRow(s"""%SystemDrive%${escapeChar}Users${escapeChar}John""" like(_, escapeChar), + s"""$escapeChar%SystemDrive$escapeChar%$escapeChar${escapeChar}Users%""", true) + } + } + test("RLIKE Regular Expression") { checkLiteralRow(Literal.create(null, StringType) rlike _, "abdef", null) checkEvaluation("abdef" rlike Literal.create(null, StringType), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 9f6e083d0030b..0b694ea954156 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -186,6 +186,18 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%")) } + test("like escape expressions") { + val message = "Escape string must contains only one character." + assertEqual("a like 'pattern%' escape '#'", 'a.like("pattern%", '#')) + assertEqual("a like 'pattern%' escape '\"'", 'a.like("pattern%", '\"')) + intercept("a like 'pattern%' escape '##'", message) + intercept("a like 'pattern%' escape ''", message) + assertEqual("a not like 'pattern%' escape '#'", !('a.like("pattern%", '#'))) + assertEqual("a not like 'pattern%' escape '\"'", !('a.like("pattern%", '\"'))) + intercept("a not like 'pattern%' escape '\"/'", message) + intercept("a not like 'pattern%' escape ''", message) + } + test("like expressions with ESCAPED_STRING_LITERALS = true") { val conf = new SQLConf() conf.setConfString(SQLConf.ESCAPED_STRING_LITERALS.key, "true") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 9560aec944d9a..904ae384154c8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -367,6 +367,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "drop", "else", "end", + "escape", "escaped", "except", "exchange", @@ -581,6 +582,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "distinct", "else", "end", + "escape", "except", "false", "fetch", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala index 63d3831404d47..67bc4bc81cb92 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala @@ -23,13 +23,34 @@ import org.apache.spark.sql.catalyst.util.StringUtils._ class StringUtilsSuite extends SparkFunSuite { test("escapeLikeRegex") { - assert(escapeLikeRegex("abdef") === "(?s)\\Qa\\E\\Qb\\E\\Qd\\E\\Qe\\E\\Qf\\E") - assert(escapeLikeRegex("a\\__b") === "(?s)\\Qa\\E\\Q_\\E.\\Qb\\E") - assert(escapeLikeRegex("a_%b") === "(?s)\\Qa\\E..*\\Qb\\E") - assert(escapeLikeRegex("a%\\%b") === "(?s)\\Qa\\E.*\\Q%\\E\\Qb\\E") - assert(escapeLikeRegex("a%") === "(?s)\\Qa\\E.*") - assert(escapeLikeRegex("**") === "(?s)\\Q*\\E\\Q*\\E") - assert(escapeLikeRegex("a_b") === "(?s)\\Qa\\E.\\Qb\\E") + val expectedEscapedStrOne = "(?s)\\Qa\\E\\Qb\\E\\Qd\\E\\Qe\\E\\Qf\\E" + val expectedEscapedStrTwo = "(?s)\\Qa\\E\\Q_\\E.\\Qb\\E" + val expectedEscapedStrThree = "(?s)\\Qa\\E..*\\Qb\\E" + val expectedEscapedStrFour = "(?s)\\Qa\\E.*\\Q%\\E\\Qb\\E" + val expectedEscapedStrFive = "(?s)\\Qa\\E.*" + val expectedEscapedStrSix = "(?s)\\Q*\\E\\Q*\\E" + val expectedEscapedStrSeven = "(?s)\\Qa\\E.\\Qb\\E" + assert(escapeLikeRegex("abdef", '\\') === expectedEscapedStrOne) + assert(escapeLikeRegex("abdef", '/') === expectedEscapedStrOne) + assert(escapeLikeRegex("abdef", '\"') === expectedEscapedStrOne) + assert(escapeLikeRegex("a\\__b", '\\') === expectedEscapedStrTwo) + assert(escapeLikeRegex("a/__b", '/') === expectedEscapedStrTwo) + assert(escapeLikeRegex("a\"__b", '\"') === expectedEscapedStrTwo) + assert(escapeLikeRegex("a_%b", '\\') === expectedEscapedStrThree) + assert(escapeLikeRegex("a_%b", '/') === expectedEscapedStrThree) + assert(escapeLikeRegex("a_%b", '\"') === expectedEscapedStrThree) + assert(escapeLikeRegex("a%\\%b", '\\') === expectedEscapedStrFour) + assert(escapeLikeRegex("a%/%b", '/') === expectedEscapedStrFour) + assert(escapeLikeRegex("a%\"%b", '\"') === expectedEscapedStrFour) + assert(escapeLikeRegex("a%", '\\') === expectedEscapedStrFive) + assert(escapeLikeRegex("a%", '/') === expectedEscapedStrFive) + assert(escapeLikeRegex("a%", '\"') === expectedEscapedStrFive) + assert(escapeLikeRegex("**", '\\') === expectedEscapedStrSix) + assert(escapeLikeRegex("**", '/') === expectedEscapedStrSix) + assert(escapeLikeRegex("**", '\"') === expectedEscapedStrSix) + assert(escapeLikeRegex("a_b", '\\') === expectedEscapedStrSeven) + assert(escapeLikeRegex("a_b", '/') === expectedEscapedStrSeven) + assert(escapeLikeRegex("a_b", '\"') === expectedEscapedStrSeven) } test("filter pattern") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala index 78fd17c73fb61..48ba8618f272e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala @@ -159,7 +159,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { case Not(expr) => isLikelySelective(expr) case And(l, r) => isLikelySelective(l) || isLikelySelective(r) case Or(l, r) => isLikelySelective(l) && isLikelySelective(r) - case Like(_, _) => true + case Like(_, _, _) => true case _: BinaryComparison => true case _: In | _: InSet => true case _: StringPredicate => true From 1595e46a4e586b78d75d90b4e8dbac21b90a80c9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 6 Dec 2019 23:41:59 +0900 Subject: [PATCH 0808/1387] [SPARK-30142][TEST-MAVEN][BUILD] Upgrade Maven to 3.6.3 ### What changes were proposed in this pull request? This PR aims to upgrade Maven from 3.6.2 to 3.6.3. ### Why are the changes needed? This will bring bug fixes like the following. - MNG-6759 Maven fails to use section from dependency when resolving transitive dependencies in some cases - MNG-6760 ExclusionArtifactFilter result invalid when wildcard exclusion is followed by other exclusions The following is the full release note. - https://maven.apache.org/docs/3.6.3/release-notes.html ### Does this PR introduce any user-facing change? No. (This is a dev-environment change.) ### How was this patch tested? Pass the Jenkins with both SBT and Maven. Closes #26770 from dongjoon-hyun/SPARK-30142. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- dev/appveyor-install-dependencies.ps1 | 2 +- docs/building-spark.md | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index e0976e66db29f..9d471c8dcbc16 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -81,7 +81,7 @@ if (!(Test-Path $tools)) { # ========================== Maven Push-Location $tools -$mavenVer = "3.6.2" +$mavenVer = "3.6.3" Start-FileDownload "https://archive.apache.org/dist/maven/maven-3/$mavenVer/binaries/apache-maven-$mavenVer-bin.zip" "maven.zip" # extract diff --git a/docs/building-spark.md b/docs/building-spark.md index 13f848eff88db..898f904461a6d 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -27,7 +27,7 @@ license: | ## Apache Maven The Maven-based build is the build of reference for Apache Spark. -Building Spark using Maven requires Maven 3.6.2 and Java 8. +Building Spark using Maven requires Maven 3.6.3 and Java 8. Spark requires Scala 2.12; support for Scala 2.11 was removed in Spark 3.0.0. ### Setting up Maven's Memory Usage diff --git a/pom.xml b/pom.xml index 53f1c50e63eea..16a7eb544bba8 100644 --- a/pom.xml +++ b/pom.xml @@ -118,7 +118,7 @@ 1.8 ${java.version} ${java.version} - 3.6.2 + 3.6.3 spark 1.7.16 1.2.17 From c8ed71b3cd397d07881f651956934cd10287ea9e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 6 Dec 2019 08:15:38 -0800 Subject: [PATCH 0809/1387] [SPARK-30011][SQL] Inline 2.12 "AsIfIntegral" classes, not present in 2.13 ### What changes were proposed in this pull request? Classes like DoubleAsIfIntegral are not found in Scala 2.13, but used in the current build. This change 'inlines' the 2.12 implementation and makes it work with both 2.12 and 2.13. ### Why are the changes needed? To cross-compile with 2.13. ### Does this PR introduce any user-facing change? It should not as it copies in 2.12's existing behavior. ### How was this patch tested? Existing tests. Closes #26769 from srowen/SPARK-30011. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/types/Decimal.scala | 6 ++- .../apache/spark/sql/types/DoubleType.scala | 37 +++++++++++++++++-- .../apache/spark/sql/types/FloatType.scala | 36 ++++++++++++++++-- 3 files changed, 70 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 24edbb2c82a32..f9ccebc4f8a04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.types import java.lang.{Long => JLong} import java.math.{BigInteger, MathContext, RoundingMode} +import scala.util.Try + import org.apache.spark.annotation.Unstable -import org.apache.spark.sql.AnalysisException /** * A mutable implementation of BigDecimal that can hold a Long if values are small enough. @@ -622,6 +623,9 @@ object Decimal { override def toLong(x: Decimal): Long = x.toLong override def fromInt(x: Int): Decimal = new Decimal().set(x) override def compare(x: Decimal, y: Decimal): Int = x.compare(y) + // Added from Scala 2.13; don't override to work in 2.12 + // TODO revisit once Scala 2.12 support is dropped + def parseString(str: String): Option[Decimal] = Try(Decimal(str)).toOption } /** A [[scala.math.Fractional]] evidence parameter for Decimals. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala index 8a5cf63b12ed7..01268a9ff1667 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.types import scala.math.{Fractional, Numeric} -import scala.math.Numeric.DoubleAsIfIntegral import scala.reflect.runtime.universe.typeTag +import scala.util.Try import org.apache.spark.annotation.Stable -import org.apache.spark.util.Utils /** * The data type representing `Double` values. Please use the singleton `DataTypes.DoubleType`. @@ -40,7 +39,7 @@ class DoubleType private() extends FractionalType { private[sql] val fractional = implicitly[Fractional[Double]] private[sql] val ordering = (x: Double, y: Double) => java.lang.Double.compare(x, y) - private[sql] val asIntegral = DoubleAsIfIntegral + private[sql] val asIntegral = DoubleType.DoubleAsIfIntegral override private[sql] def exactNumeric = DoubleExactNumeric @@ -56,4 +55,34 @@ class DoubleType private() extends FractionalType { * @since 1.3.0 */ @Stable -case object DoubleType extends DoubleType +case object DoubleType extends DoubleType { + + // Traits below copied from Scala 2.12; not present in 2.13 + // TODO: SPARK-30011 revisit once Scala 2.12 support is dropped + trait DoubleIsConflicted extends Numeric[Double] { + def plus(x: Double, y: Double): Double = x + y + def minus(x: Double, y: Double): Double = x - y + def times(x: Double, y: Double): Double = x * y + def negate(x: Double): Double = -x + def fromInt(x: Int): Double = x.toDouble + def toInt(x: Double): Int = x.toInt + def toLong(x: Double): Long = x.toLong + def toFloat(x: Double): Float = x.toFloat + def toDouble(x: Double): Double = x + // logic in Numeric base trait mishandles abs(-0.0) + override def abs(x: Double): Double = math.abs(x) + // Added from Scala 2.13; don't override to work in 2.12 + def parseString(str: String): Option[Double] = + Try(java.lang.Double.parseDouble(str)).toOption + + } + + trait DoubleAsIfIntegral extends DoubleIsConflicted with Integral[Double] { + def quot(x: Double, y: Double): Double = (BigDecimal(x) quot BigDecimal(y)).doubleValue + def rem(x: Double, y: Double): Double = (BigDecimal(x) remainder BigDecimal(y)).doubleValue + } + + object DoubleAsIfIntegral extends DoubleAsIfIntegral { + override def compare(x: Double, y: Double): Int = java.lang.Double.compare(x, y) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala index 738291eaf7762..402c5e1472c32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.types import scala.math.{Fractional, Numeric} -import scala.math.Numeric.FloatAsIfIntegral import scala.reflect.runtime.universe.typeTag +import scala.util.Try import org.apache.spark.annotation.Stable -import org.apache.spark.util.Utils /** * The data type representing `Float` values. Please use the singleton `DataTypes.FloatType`. @@ -40,7 +39,7 @@ class FloatType private() extends FractionalType { private[sql] val fractional = implicitly[Fractional[Float]] private[sql] val ordering = (x: Float, y: Float) => java.lang.Float.compare(x, y) - private[sql] val asIntegral = FloatAsIfIntegral + private[sql] val asIntegral = FloatType.FloatAsIfIntegral override private[sql] def exactNumeric = FloatExactNumeric @@ -57,4 +56,33 @@ class FloatType private() extends FractionalType { * @since 1.3.0 */ @Stable -case object FloatType extends FloatType +case object FloatType extends FloatType { + + // Traits below copied from Scala 2.12; not present in 2.13 + // TODO: SPARK-30011 revisit once Scala 2.12 support is dropped + trait FloatIsConflicted extends Numeric[Float] { + def plus(x: Float, y: Float): Float = x + y + def minus(x: Float, y: Float): Float = x - y + def times(x: Float, y: Float): Float = x * y + def negate(x: Float): Float = -x + def fromInt(x: Int): Float = x.toFloat + def toInt(x: Float): Int = x.toInt + def toLong(x: Float): Long = x.toLong + def toFloat(x: Float): Float = x + def toDouble(x: Float): Double = x.toDouble + // logic in Numeric base trait mishandles abs(-0.0f) + override def abs(x: Float): Float = math.abs(x) + // Added from Scala 2.13; don't override to work in 2.12 + def parseString(str: String): Option[Float] = + Try(java.lang.Float.parseFloat(str)).toOption + } + + trait FloatAsIfIntegral extends FloatIsConflicted with Integral[Float] { + def quot(x: Float, y: Float): Float = (BigDecimal(x) quot BigDecimal(y)).floatValue + def rem(x: Float, y: Float): Float = (BigDecimal(x) remainder BigDecimal(y)).floatValue + } + + object FloatAsIfIntegral extends FloatAsIfIntegral { + override def compare(x: Float, y: Float): Int = java.lang.Float.compare(x, y) + } +} From c1a5f94973213b1cad15388f3ef8a488424c34a7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 6 Dec 2019 09:22:16 -0800 Subject: [PATCH 0810/1387] [SPARK-30112][SQL] Allow insert overwrite same table if using dynamic partition overwrite ### What changes were proposed in this pull request? This patch proposes to allow insert overwrite same table if using dynamic partition overwrite. ### Why are the changes needed? Currently, Insert overwrite cannot overwrite to same table even it is dynamic partition overwrite. But for dynamic partition overwrite, we do not delete partition directories ahead. We write to staging directories and move data to final partition directories. We should be able to insert overwrite to same table under dynamic partition overwrite. This enables users to read data from a table and insert overwrite to same table by using dynamic partition overwrite. Because this is not allowed for now, users need to write to other temporary location and move it back to the table. ### Does this PR introduce any user-facing change? Yes. Users can insert overwrite same table if using dynamic partition overwrite. ### How was this patch tested? Unit test. Closes #26752 from viirya/dynamic-overwrite-same-table. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../datasources/DataSourceStrategy.scala | 12 +++-- .../InsertIntoHadoopFsRelationCommand.scala | 29 ++++++----- .../spark/sql/sources/InsertSuite.scala | 49 +++++++++++++++++++ 3 files changed, 74 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 46444f0a05605..8ddeb5edf9431 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -188,15 +188,13 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } val outputPath = t.location.rootPaths.head - if (overwrite) DDLUtils.verifyNotReadPath(actualQuery, outputPath) - val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append val partitionSchema = actualQuery.resolve( t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver) val staticPartitions = parts.filter(_._2.nonEmpty).map { case (k, v) => k -> v.get } - InsertIntoHadoopFsRelationCommand( + val insertCommand = InsertIntoHadoopFsRelationCommand( outputPath, staticPartitions, i.ifPartitionNotExists, @@ -209,6 +207,14 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast table, Some(t.location), actualQuery.output.map(_.name)) + + // For dynamic partition overwrite, we do not delete partition directories ahead. + // We write to staging directories and move to final partition directories after writing + // job is done. So it is ok to have outputPath try to overwrite inputpath. + if (overwrite && !insertCommand.dynamicPartitionOverwrite) { + DDLUtils.verifyNotReadPath(actualQuery, outputPath) + } + insertCommand } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index fbe874b3e8bc5..f11972115e09f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.util.SchemaUtils @@ -60,6 +61,21 @@ case class InsertIntoHadoopFsRelationCommand( extends DataWritingCommand { import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName + private lazy val parameters = CaseInsensitiveMap(options) + + private[sql] lazy val dynamicPartitionOverwrite: Boolean = { + val partitionOverwriteMode = parameters.get("partitionOverwriteMode") + // scalastyle:off caselocale + .map(mode => PartitionOverwriteMode.withName(mode.toUpperCase)) + // scalastyle:on caselocale + .getOrElse(SQLConf.get.partitionOverwriteMode) + val enableDynamicOverwrite = partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC + // This config only makes sense when we are overwriting a partitioned dataset with dynamic + // partition columns. + enableDynamicOverwrite && mode == SaveMode.Overwrite && + staticPartitions.size < partitionColumns.length + } + override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { // Most formats don't do well with duplicate columns, so lets not allow that SchemaUtils.checkColumnNameDuplication( @@ -90,19 +106,6 @@ case class InsertIntoHadoopFsRelationCommand( fs, catalogTable.get, qualifiedOutputPath, matchingPartitions) } - val parameters = CaseInsensitiveMap(options) - - val partitionOverwriteMode = parameters.get("partitionOverwriteMode") - // scalastyle:off caselocale - .map(mode => PartitionOverwriteMode.withName(mode.toUpperCase)) - // scalastyle:on caselocale - .getOrElse(sparkSession.sessionState.conf.partitionOverwriteMode) - val enableDynamicOverwrite = partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC - // This config only makes sense when we are overwriting a partitioned dataset with dynamic - // partition columns. - val dynamicPartitionOverwrite = enableDynamicOverwrite && mode == SaveMode.Overwrite && - staticPartitions.size < partitionColumns.length - val committer = FileCommitProtocol.instantiate( sparkSession.sessionState.conf.fileCommitProtocolClass, jobId = java.util.UUID.randomUUID().toString, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index fbde38322fca8..871cb1ff151ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -270,6 +270,55 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { "INSERT OVERWRITE to a table while querying it should not be allowed.") } + test("SPARK-30112: it is allowed to write to a table while querying it for " + + "dynamic partition overwrite.") { + Seq(PartitionOverwriteMode.DYNAMIC.toString, + PartitionOverwriteMode.STATIC.toString).foreach { mode => + withSQLConf(SQLConf.PARTITION_OVERWRITE_MODE.key -> mode) { + withTable("insertTable") { + sql( + """ + |CREATE TABLE insertTable(i int, part1 int, part2 int) USING PARQUET + |PARTITIONED BY (part1, part2) + """.stripMargin) + + sql("INSERT INTO TABLE insertTable PARTITION(part1=1, part2=1) SELECT 1") + checkAnswer(spark.table("insertTable"), Row(1, 1, 1)) + sql("INSERT OVERWRITE TABLE insertTable PARTITION(part1=1, part2=2) SELECT 2") + checkAnswer(spark.table("insertTable"), Row(1, 1, 1) :: Row(2, 1, 2) :: Nil) + + if (mode == PartitionOverwriteMode.DYNAMIC.toString) { + sql( + """ + |INSERT OVERWRITE TABLE insertTable PARTITION(part1=1, part2) + |SELECT i + 1, part2 FROM insertTable + """.stripMargin) + checkAnswer(spark.table("insertTable"), Row(2, 1, 1) :: Row(3, 1, 2) :: Nil) + + sql( + """ + |INSERT OVERWRITE TABLE insertTable PARTITION(part1=1, part2) + |SELECT i + 1, part2 + 1 FROM insertTable + """.stripMargin) + checkAnswer(spark.table("insertTable"), + Row(2, 1, 1) :: Row(3, 1, 2) :: Row(4, 1, 3) :: Nil) + } else { + val message = intercept[AnalysisException] { + sql( + """ + |INSERT OVERWRITE TABLE insertTable PARTITION(part1=1, part2) + |SELECT i + 1, part2 FROM insertTable + """.stripMargin) + }.getMessage + assert( + message.contains("Cannot overwrite a path that is also being read from."), + "INSERT OVERWRITE to a table while querying it should not be allowed.") + } + } + } + } + } + test("Caching") { // write something to the jsonTable sql( From 58be82ad4b98fc17e821e916e69e77a6aa36209d Mon Sep 17 00:00:00 2001 From: wuyi Date: Sat, 7 Dec 2019 02:15:25 +0800 Subject: [PATCH 0811/1387] [SPARK-30098][SQL] Use default datasource as provider for CREATE TABLE syntax ### What changes were proposed in this pull request? In this PR, we propose to use the value of `spark.sql.source.default` as the provider for `CREATE TABLE` syntax instead of `hive` in Spark 3.0. And to help the migration, we introduce a legacy conf `spark.sql.legacy.respectHiveDefaultProvider.enabled` and set its default to `false`. ### Why are the changes needed? 1. Currently, `CREATE TABLE` syntax use hive provider to create table while `DataFrameWriter.saveAsTable` API using the value of `spark.sql.source.default` as a provider to create table. It would be better to make them consistent. 2. User may gets confused in some cases. For example: ``` CREATE TABLE t1 (c1 INT) USING PARQUET; CREATE TABLE t2 (c1 INT); ``` In these two DDLs, use may think that `t2` should also use parquet as default provider since Spark always advertise parquet as the default format. However, it's hive in this case. On the other hand, if we omit the USING clause in a CTAS statement, we do pick parquet by default if `spark.sql.hive.convertCATS=true`: ``` CREATE TABLE t3 USING PARQUET AS SELECT 1 AS VALUE; CREATE TABLE t4 AS SELECT 1 AS VALUE; ``` And these two cases together can be really confusing. 3. Now, Spark SQL is very independent and popular. We do not need to be fully consistent with Hive's behavior. ### Does this PR introduce any user-facing change? Yes, before this PR, using `CREATE TABLE` syntax will use hive provider. But now, it use the value of `spark.sql.source.default` as its provider. ### How was this patch tested? Added tests in `DDLParserSuite` and `HiveDDlSuite`. Closes #26736 from Ngone51/dev-create-table-using-parquet-by-default. Lead-authored-by: wuyi Co-authored-by: yi.wu Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + .../spark/sql/catalyst/parser/SqlBase.g4 | 36 ++-- .../sql/catalyst/parser/AstBuilder.scala | 66 ++++---- .../sql/catalyst/parser/ParseDriver.scala | 2 + .../apache/spark/sql/internal/SQLConf.scala | 12 ++ .../sql/catalyst/parser/DDLParserSuite.scala | 21 +++ .../spark/sql/execution/SparkSqlParser.scala | 13 +- .../results/postgreSQL/create_view.sql.out | 5 +- .../sql/execution/SparkSqlParserSuite.scala | 5 +- .../execution/command/DDLParserSuite.scala | 154 +++++++++--------- .../sql/execution/command/DDLSuite.scala | 6 +- .../sources/CreateTableAsSelectSuite.scala | 2 +- .../sql/hive/thriftserver/CliSuite.scala | 8 +- .../HiveThriftServer2Suites.scala | 22 +-- .../execution/HiveCompatibilitySuite.scala | 4 + .../HiveWindowFunctionQuerySuite.scala | 2 +- .../sql/hive/HiveShowCreateTableSuite.scala | 4 +- .../spark/sql/hive/StatisticsSuite.scala | 4 +- .../sql/hive/execution/HiveCommandSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 39 ++++- .../sql/hive/execution/HiveExplainSuite.scala | 5 +- .../sql/hive/execution/HiveSerDeSuite.scala | 9 +- .../sql/hive/execution/SQLQuerySuite.scala | 18 +- .../sql/hive/execution/WindowQuerySuite.scala | 2 +- .../apache/spark/sql/hive/test/TestHive.scala | 4 +- 25 files changed, 261 insertions(+), 186 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 5ccc3fd248219..e22e4158f2490 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -253,6 +253,8 @@ license: | + + - Since Spark 3.0, CREATE TABLE without a specific provider will use the value of `spark.sql.sources.default` as its provider. In Spark version 2.4 and earlier, it was hive. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createHiveTableByDefault.enabled` to `true`. - Since Spark 3.0, the unary arithmetic operator plus(`+`) only accepts string, numeric and interval type values as inputs. Besides, `+` with a integral string representation will be coerced to double value, e.g. `+'1'` results `1.0`. In Spark version 2.4 and earlier, this operator is ignored. There is no type checking for it, thus, all type values with a `+` prefix are valid, e.g. `+ array(1, 2)` is valid and results `[1, 2]`. Besides, there is no type coercion for it at all, e.g. in Spark 2.4, the result of `+'1'` is string `1`. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 21c38b604199f..f894836d64233 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -29,6 +29,12 @@ grammar SqlBase; */ public boolean legacy_exponent_literal_as_decimal_enabled = false; + /** + * When false, CREATE TABLE syntax without a provider will use + * the value of spark.sql.sources.default as its provider. + */ + public boolean legacy_create_hive_table_by_default_enabled = false; + /** * Verify whether current token is a valid decimal token (which contains dot). * Returns true if the character that follows the token is not a digit or letter or underscore. @@ -101,13 +107,13 @@ statement (RESTRICT | CASCADE)? #dropNamespace | SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)? (LIKE? pattern=STRING)? #showNamespaces - | createTableHeader ('(' colTypeList ')')? tableProvider - ((OPTIONS options=tablePropertyList) | - (PARTITIONED BY partitioning=transformList) | - bucketSpec | - locationSpec | - (COMMENT comment=STRING) | - (TBLPROPERTIES tableProps=tablePropertyList))* + | {!legacy_create_hive_table_by_default_enabled}? + createTableHeader ('(' colTypeList ')')? tableProvider? + createTableClauses + (AS? query)? #createTable + | {legacy_create_hive_table_by_default_enabled}? + createTableHeader ('(' colTypeList ')')? tableProvider + createTableClauses (AS? query)? #createTable | createTableHeader ('(' columns=colTypeList ')')? ((COMMENT comment=STRING) | @@ -128,12 +134,7 @@ statement locationSpec | (TBLPROPERTIES tableProps=tablePropertyList))* #createTableLike | replaceTableHeader ('(' colTypeList ')')? tableProvider - ((OPTIONS options=tablePropertyList) | - (PARTITIONED BY partitioning=transformList) | - bucketSpec | - locationSpec | - (COMMENT comment=STRING) | - (TBLPROPERTIES tableProps=tablePropertyList))* + createTableClauses (AS? query)? #replaceTable | ANALYZE TABLE multipartIdentifier partitionSpec? COMPUTE STATISTICS (identifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze @@ -352,6 +353,15 @@ tableProvider : USING multipartIdentifier ; +createTableClauses + :((OPTIONS options=tablePropertyList) | + (PARTITIONED BY partitioning=transformList) | + bucketSpec | + locationSpec | + (COMMENT comment=STRING) | + (TBLPROPERTIES tableProps=tablePropertyList))* + ; + tablePropertyList : '(' tableProperty (',' tableProperty)* ')' ; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 78164279d8f48..310746585289a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2379,6 +2379,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ type TableHeader = (Seq[String], Boolean, Boolean, Boolean) + /** + * Type to keep track of table clauses: + * (partitioning, bucketSpec, options, locationSpec, properties, comment). + */ + type TableClauses = (Seq[Transform], Option[BucketSpec], Map[String, String], + Map[String, String], Option[String], Option[String]) + /** * Validate a create table statement and return the [[TableIdentifier]]. */ @@ -2614,6 +2621,24 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx.EXTENDED != null) } + override def visitCreateTableClauses(ctx: CreateTableClausesContext): TableClauses = { + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) + checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) + checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + + val partitioning: Seq[Transform] = + Option(ctx.partitioning).map(visitTransformList).getOrElse(Nil) + val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) + val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) + val comment = Option(ctx.comment).map(string) + (partitioning, bucketSpec, properties, options, location, comment) + } + /** * Create a table, returning a [[CreateTableStatement]] logical plan. * @@ -2639,26 +2664,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = withOrigin(ctx) { val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) if (external) { - operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) + operationNotAllowed("CREATE EXTERNAL TABLE ...", ctx) } - - checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) - checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) - checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) - checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) - checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) - checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) - val schema = Option(ctx.colTypeList()).map(createSchema) - val partitioning: Seq[Transform] = - Option(ctx.partitioning).map(visitTransformList).getOrElse(Nil) - val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) - val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - - val provider = ctx.tableProvider.multipartIdentifier.getText - val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) - val comment = Option(ctx.comment).map(string) + val defaultProvider = conf.defaultDataSourceName + val provider = + Option(ctx.tableProvider).map(_.multipartIdentifier.getText).getOrElse(defaultProvider) + val (partitioning, bucketSpec, properties, options, location, comment) = + visitCreateTableClauses(ctx.createTableClauses()) Option(ctx.query).map(plan) match { case Some(_) if temp => @@ -2713,23 +2726,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging operationNotAllowed("REPLACE EXTERNAL TABLE ... USING", ctx) } - checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) - checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) - checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) - checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) - checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) - checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) - + val (partitioning, bucketSpec, properties, options, location, comment) = + visitCreateTableClauses(ctx.createTableClauses()) val schema = Option(ctx.colTypeList()).map(createSchema) - val partitioning: Seq[Transform] = - Option(ctx.partitioning).map(visitTransformList).getOrElse(Nil) - val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) - val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - val provider = ctx.tableProvider.multipartIdentifier.getText - val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) - val comment = Option(ctx.comment).map(string) val orCreate = ctx.replaceTableHeader().CREATE() != null Option(ctx.query).map(plan) match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 160c7ef80cab9..b968848224c54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -101,6 +101,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced lexer.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled + lexer.legacy_create_hive_table_by_default_enabled = conf.createHiveTableByDefaultEnabled lexer.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior val tokenStream = new CommonTokenStream(lexer) @@ -110,6 +111,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced parser.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled + parser.legacy_create_hive_table_by_default_enabled = conf.createHiveTableByDefaultEnabled parser.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 426ec6efb6ad6..f535792489fb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1966,6 +1966,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED = + buildConf("spark.sql.legacy.createHiveTableByDefault.enabled") + .internal() + .doc("When set to true, CREATE TABLE syntax without a provider will use hive " + + s"instead of the value of ${DEFAULT_DATA_SOURCE_NAME.key}.") + .booleanConf + .createWithDefault(false) + + val LEGACY_INTEGRALDIVIDE_RETURN_LONG = buildConf("spark.sql.legacy.integralDivide.returnBigint") .doc("If it is set to true, the div operator returns always a bigint. This behavior was " + "inherited from Hive. Otherwise, the return type is the data type of the operands.") @@ -2583,6 +2592,9 @@ class SQLConf extends Serializable with Logging { def exponentLiteralAsDecimalEnabled: Boolean = getConf(SQLConf.LEGACY_EXPONENT_LITERAL_AS_DECIMAL_ENABLED) + def createHiveTableByDefaultEnabled: Boolean = + getConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED) + def integralDivideReturnLong: Boolean = getConf(SQLConf.LEGACY_INTEGRALDIVIDE_RETURN_LONG) def nameNonStructGroupingKeyAsValue: Boolean = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index aa2cde2e201b4..fb965b5bbf1e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -48,6 +49,26 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsePlan(sql), expected, checkAnalysis = false) } + test("SPARK-30098: create table without provider should " + + "use default data source under non-legacy mode") { + val createSql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING)" + val defaultProvider = conf.defaultDataSourceName + val expectedPlan = CreateTableStatement( + Seq("my_tab"), + new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType), + Seq.empty[Transform], + None, + Map.empty[String, String], + defaultProvider, + Map.empty[String, String], + None, + None, + false) + parseCompare(createSql, expectedPlan) + } + test("create/replace table using - schema") { val createSql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" val replaceSql = "REPLACE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index b150b40049964..20e0ccdf0e72d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -187,22 +187,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { if (external) { operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) } - - checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) - checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) - checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) - checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) - checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) - checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) - if (ifNotExists) { // Unlike CREATE TEMPORARY VIEW USING, CREATE TEMPORARY TABLE USING does not support // IF NOT EXISTS. Users are not allowed to replace the existing temp table. operationNotAllowed("CREATE TEMPORARY TABLE IF NOT EXISTS", ctx) } - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - val provider = ctx.tableProvider.multipartIdentifier.getText + val (_, _, _, options, _, _) = visitCreateTableClauses(ctx.createTableClauses()) + val provider = Option(ctx.tableProvider).map(_.multipartIdentifier.getText).getOrElse( + throw new ParseException("CREATE TEMPORARY TABLE without a provider is not allowed.", ctx)) val schema = Option(ctx.colTypeList()).map(createSchema) logWarning(s"CREATE TEMPORARY TABLE ... USING ... is deprecated, please use " + diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index cd79e4253aac9..cd3f456cb9738 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -46,8 +46,7 @@ CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) -- !query 4 schema struct<> -- !query 4 output -org.apache.spark.sql.AnalysisException -Hive support is required to CREATE Hive TABLE (AS SELECT); + -- !query 5 @@ -57,7 +56,7 @@ CREATE VIEW key_dependent_view AS struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -Table or view not found: view_base_table; line 2 pos 17 +expression 'default.view_base_table.`data`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; -- !query 6 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 0c766429742df..7e9b0d824fb9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -159,7 +159,7 @@ class SparkSqlParserSuite extends AnalysisTest { } test("create table - schema") { - assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING)", + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) STORED AS textfile", createTable( table = "my_tab", schema = (new StructType) @@ -179,7 +179,8 @@ class SparkSqlParserSuite extends AnalysisTest { partitionColumnNames = Seq("c", "d") ) ) - assertEqual("CREATE TABLE my_tab(id BIGINT, nested STRUCT)", + assertEqual("CREATE TABLE my_tab(id BIGINT, nested STRUCT) " + + "STORED AS textfile", createTable( table = "my_tab", schema = (new StructType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 6b829171dd6e2..e486df930e9fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.dsl.plans import org.apache.spark.sql.catalyst.dsl.plans.DslLogicalPlan import org.apache.spark.sql.catalyst.expressions.JsonTuple import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan, Project, ScriptTransformation} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform} import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -74,6 +75,12 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { }.head } + private def withCreateTableStatement(sql: String)(prediction: CreateTableStatement => Unit) + : Unit = { + val statement = parser.parsePlan(sql).asInstanceOf[CreateTableStatement] + prediction(statement) + } + test("alter database - property values must be set") { assertUnsupported( sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')", @@ -277,9 +284,9 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { test("create hive external table - location must be specified") { assertUnsupported( - sql = "CREATE EXTERNAL TABLE my_tab", + sql = "CREATE EXTERNAL TABLE my_tab STORED AS parquet", containsThesePhrases = Seq("create external table", "location")) - val query = "CREATE EXTERNAL TABLE my_tab LOCATION '/something/anything'" + val query = "CREATE EXTERNAL TABLE my_tab STORED AS parquet LOCATION '/something/anything'" val ct = parseAs[CreateTable](query) assert(ct.tableDesc.tableType == CatalogTableType.EXTERNAL) assert(ct.tableDesc.storage.locationUri == Some(new URI("/something/anything"))) @@ -287,7 +294,8 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { test("create hive table - property values must be set") { assertUnsupported( - sql = "CREATE TABLE my_tab TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + sql = "CREATE TABLE my_tab STORED AS parquet " + + "TBLPROPERTIES('key_without_value', 'key_with_value'='x')", containsThesePhrases = Seq("key_without_value")) assertUnsupported( sql = "CREATE TABLE my_tab ROW FORMAT SERDE 'serde' " + @@ -296,7 +304,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { } test("create hive table - location implies external") { - val query = "CREATE TABLE my_tab LOCATION '/something/anything'" + val query = "CREATE TABLE my_tab STORED AS parquet LOCATION '/something/anything'" val ct = parseAs[CreateTable](query) assert(ct.tableDesc.tableType == CatalogTableType.EXTERNAL) assert(ct.tableDesc.storage.locationUri == Some(new URI("/something/anything"))) @@ -599,22 +607,17 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { test("Test CTAS #3") { val s3 = """CREATE TABLE page_view AS SELECT * FROM src""" - val (desc, exists) = extractTableDesc(s3) - assert(exists == false) - assert(desc.identifier.database == None) - assert(desc.identifier.table == "page_view") - assert(desc.tableType == CatalogTableType.MANAGED) - assert(desc.storage.locationUri == None) - assert(desc.schema.isEmpty) - assert(desc.viewText == None) // TODO will be SQLText - assert(desc.viewDefaultDatabase.isEmpty) - assert(desc.viewQueryColumnNames.isEmpty) - assert(desc.storage.properties == Map()) - assert(desc.storage.inputFormat == Some("org.apache.hadoop.mapred.TextInputFormat")) - assert(desc.storage.outputFormat == - Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")) - assert(desc.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) - assert(desc.properties == Map()) + val statement = parser.parsePlan(s3).asInstanceOf[CreateTableAsSelectStatement] + assert(statement.tableName(0) == "page_view") + assert(statement.asSelect == parser.parsePlan("SELECT * FROM src")) + assert(statement.partitioning.isEmpty) + assert(statement.bucketSpec.isEmpty) + assert(statement.properties.isEmpty) + assert(statement.provider == conf.defaultDataSourceName) + assert(statement.options.isEmpty) + assert(statement.location.isEmpty) + assert(statement.comment.isEmpty) + assert(!statement.ifNotExists) } test("Test CTAS #4") { @@ -774,68 +777,60 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { test("create table - basic") { val query = "CREATE TABLE my_table (id int, name string)" - val (desc, allowExisting) = extractTableDesc(query) - assert(!allowExisting) - assert(desc.identifier.database.isEmpty) - assert(desc.identifier.table == "my_table") - assert(desc.tableType == CatalogTableType.MANAGED) - assert(desc.schema == new StructType().add("id", "int").add("name", "string")) - assert(desc.partitionColumnNames.isEmpty) - assert(desc.bucketSpec.isEmpty) - assert(desc.viewText.isEmpty) - assert(desc.viewDefaultDatabase.isEmpty) - assert(desc.viewQueryColumnNames.isEmpty) - assert(desc.storage.locationUri.isEmpty) - assert(desc.storage.inputFormat == - Some("org.apache.hadoop.mapred.TextInputFormat")) - assert(desc.storage.outputFormat == - Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")) - assert(desc.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) - assert(desc.storage.properties.isEmpty) - assert(desc.properties.isEmpty) - assert(desc.comment.isEmpty) + withCreateTableStatement(query) { state => + assert(state.tableName(0) == "my_table") + assert(state.tableSchema == new StructType().add("id", "int").add("name", "string")) + assert(state.partitioning.isEmpty) + assert(state.bucketSpec.isEmpty) + assert(state.properties.isEmpty) + assert(state.provider == conf.defaultDataSourceName) + assert(state.options.isEmpty) + assert(state.location.isEmpty) + assert(state.comment.isEmpty) + assert(!state.ifNotExists) + } } test("create table - with database name") { val query = "CREATE TABLE dbx.my_table (id int, name string)" - val (desc, _) = extractTableDesc(query) - assert(desc.identifier.database == Some("dbx")) - assert(desc.identifier.table == "my_table") + withCreateTableStatement(query) { state => + assert(state.tableName(0) == "dbx") + assert(state.tableName(1) == "my_table") + } } test("create table - temporary") { val query = "CREATE TEMPORARY TABLE tab1 (id int, name string)" val e = intercept[ParseException] { parser.parsePlan(query) } - assert(e.message.contains("CREATE TEMPORARY TABLE is not supported yet")) + assert(e.message.contains("CREATE TEMPORARY TABLE without a provider is not allowed.")) } test("create table - external") { val query = "CREATE EXTERNAL TABLE tab1 (id int, name string) LOCATION '/path/to/nowhere'" - val (desc, _) = extractTableDesc(query) - assert(desc.tableType == CatalogTableType.EXTERNAL) - assert(desc.storage.locationUri == Some(new URI("/path/to/nowhere"))) + val e = intercept[ParseException] { parser.parsePlan(query) } + assert(e.message.contains("Operation not allowed: CREATE EXTERNAL TABLE ...")) } test("create table - if not exists") { val query = "CREATE TABLE IF NOT EXISTS tab1 (id int, name string)" - val (_, allowExisting) = extractTableDesc(query) - assert(allowExisting) + withCreateTableStatement(query) { state => + assert(state.ifNotExists) + } } test("create table - comment") { val query = "CREATE TABLE my_table (id int, name string) COMMENT 'its hot as hell below'" - val (desc, _) = extractTableDesc(query) - assert(desc.comment == Some("its hot as hell below")) + withCreateTableStatement(query) { state => + assert(state.comment == Some("its hot as hell below")) + } } test("create table - partitioned columns") { - val query = "CREATE TABLE my_table (id int, name string) PARTITIONED BY (month int)" - val (desc, _) = extractTableDesc(query) - assert(desc.schema == new StructType() - .add("id", "int") - .add("name", "string") - .add("month", "int")) - assert(desc.partitionColumnNames == Seq("month")) + val query = "CREATE TABLE my_table (id int, name string) PARTITIONED BY (id)" + withCreateTableStatement(query) { state => + val transform = IdentityTransform(FieldReference(Seq("id"))) + assert(state.partitioning == Seq(transform)) + } } test("create table - clustered by") { @@ -851,23 +846,25 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { """ val query1 = s"$baseQuery INTO $numBuckets BUCKETS" - val (desc1, _) = extractTableDesc(query1) - assert(desc1.bucketSpec.isDefined) - val bucketSpec1 = desc1.bucketSpec.get - assert(bucketSpec1.numBuckets == numBuckets) - assert(bucketSpec1.bucketColumnNames.head.equals(bucketedColumn)) - assert(bucketSpec1.sortColumnNames.isEmpty) + withCreateTableStatement(query1) { state => + assert(state.bucketSpec.isDefined) + val bucketSpec = state.bucketSpec.get + assert(bucketSpec.numBuckets == numBuckets) + assert(bucketSpec.bucketColumnNames.head.equals(bucketedColumn)) + assert(bucketSpec.sortColumnNames.isEmpty) + } val query2 = s"$baseQuery SORTED BY($sortColumn) INTO $numBuckets BUCKETS" - val (desc2, _) = extractTableDesc(query2) - assert(desc2.bucketSpec.isDefined) - val bucketSpec2 = desc2.bucketSpec.get - assert(bucketSpec2.numBuckets == numBuckets) - assert(bucketSpec2.bucketColumnNames.head.equals(bucketedColumn)) - assert(bucketSpec2.sortColumnNames.head.equals(sortColumn)) + withCreateTableStatement(query2) { state => + assert(state.bucketSpec.isDefined) + val bucketSpec = state.bucketSpec.get + assert(bucketSpec.numBuckets == numBuckets) + assert(bucketSpec.bucketColumnNames.head.equals(bucketedColumn)) + assert(bucketSpec.sortColumnNames.head.equals(sortColumn)) + } } - test("create table - skewed by") { + test("create table(hive) - skewed by") { val baseQuery = "CREATE TABLE my_table (id int, name string) SKEWED BY" val query1 = s"$baseQuery(id) ON (1, 10, 100)" val query2 = s"$baseQuery(id, name) ON ((1, 'x'), (2, 'y'), (3, 'z'))" @@ -880,7 +877,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { assert(e3.getMessage.contains("Operation not allowed")) } - test("create table - row format") { + test("create table(hive) - row format") { val baseQuery = "CREATE TABLE my_table (id int, name string) ROW FORMAT" val query1 = s"$baseQuery SERDE 'org.apache.poof.serde.Baff'" val query2 = s"$baseQuery SERDE 'org.apache.poof.serde.Baff' WITH SERDEPROPERTIES ('k1'='v1')" @@ -908,7 +905,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { "mapkey.delim" -> "b")) } - test("create table - file format") { + test("create table(hive) - file format") { val baseQuery = "CREATE TABLE my_table (id int, name string) STORED AS" val query1 = s"$baseQuery INPUTFORMAT 'winput' OUTPUTFORMAT 'wowput'" val query2 = s"$baseQuery ORC" @@ -922,7 +919,7 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { assert(desc2.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) } - test("create table - storage handler") { + test("create table(hive) - storage handler") { val baseQuery = "CREATE TABLE my_table (id int, name string) STORED BY" val query1 = s"$baseQuery 'org.papachi.StorageHandler'" val query2 = s"$baseQuery 'org.mamachi.StorageHandler' WITH SERDEPROPERTIES ('k1'='v1')" @@ -934,11 +931,12 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { test("create table - properties") { val query = "CREATE TABLE my_table (id int, name string) TBLPROPERTIES ('k1'='v1', 'k2'='v2')" - val (desc, _) = extractTableDesc(query) - assert(desc.properties == Map("k1" -> "v1", "k2" -> "v2")) + withCreateTableStatement(query) { state => + assert(state.properties == Map("k1" -> "v1", "k2" -> "v2")) + } } - test("create table - everything!") { + test("create table(hive) - everything!") { val query = """ |CREATE EXTERNAL TABLE IF NOT EXISTS dbx.my_table (id int, name string) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index c762f25d62eed..10873678e05f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -82,7 +82,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { val tabName = "tbl" withTable(tabName) { val e = intercept[AnalysisException] { - sql(s"CREATE TABLE $tabName (i INT, j STRING)") + sql(s"CREATE TABLE $tabName (i INT, j STRING) STORED AS parquet") }.getMessage assert(e.contains("Hive support is required to CREATE Hive TABLE")) } @@ -110,13 +110,13 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { import testImplicits._ withTable("t", "t1") { var e = intercept[AnalysisException] { - sql("CREATE TABLE t SELECT 1 as a, 1 as b") + sql("CREATE TABLE t STORED AS parquet SELECT 1 as a, 1 as b") }.getMessage assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)")) spark.range(1).select('id as 'a, 'id as 'b).write.saveAsTable("t1") e = intercept[AnalysisException] { - sql("CREATE TABLE t SELECT a, b from t1") + sql("CREATE TABLE t STORED AS parquet SELECT a, b from t1") }.getMessage assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 08f0865c1e128..983209051c8ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -183,7 +183,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession { }.getMessage assert(error.contains("Operation not allowed") && - error.contains("CREATE EXTERNAL TABLE ... USING")) + error.contains("CREATE EXTERNAL TABLE ...")) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 856102bc900a8..6609701be0ede 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -164,7 +164,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") runCliWithin(3.minute)( - "CREATE TABLE hive_test(key INT, val STRING);" + "CREATE TABLE hive_test(key INT, val STRING) USING hive;" -> "", "SHOW TABLES;" -> "hive_test", @@ -211,7 +211,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; """.stripMargin -> "", - "CREATE TABLE sourceTable (key INT, val STRING);" + "CREATE TABLE sourceTable (key INT, val STRING) USING hive;" -> "", s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE sourceTable;" -> "", @@ -237,7 +237,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; """.stripMargin -> "", - "CREATE TABLE sourceTableForWithHiveAux (key INT, val STRING);" + "CREATE TABLE sourceTableForWithHiveAux (key INT, val STRING) USING hive;" -> "", s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE sourceTableForWithHiveAux;" -> "", @@ -370,7 +370,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; """.stripMargin -> "", - "CREATE TABLE sourceTableForWithSQL(key INT, val STRING);" + "CREATE TABLE sourceTableForWithSQL(key INT, val STRING) USING hive;" -> "", s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE sourceTableForWithSQL;" -> "", diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 8754177f86b8e..108fec4ae035c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -101,7 +101,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { withJdbcStatement("test_16563") { statement => val queries = Seq( - "CREATE TABLE test_16563(key INT, val STRING)", + "CREATE TABLE test_16563(key INT, val STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_16563") queries.foreach(statement.execute) @@ -165,7 +165,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { withJdbcStatement("test") { statement => val queries = Seq( "SET spark.sql.shuffle.partitions=3", - "CREATE TABLE test(key INT, val STRING)", + "CREATE TABLE test(key INT, val STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", "CACHE TABLE test") @@ -191,7 +191,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { test("SPARK-3004 regression: result set containing NULL") { withJdbcStatement("test_null") { statement => val queries = Seq( - "CREATE TABLE test_null(key INT, val STRING)", + "CREATE TABLE test_null(key INT, val STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKvWithNull}' OVERWRITE INTO TABLE test_null") queries.foreach(statement.execute) @@ -211,7 +211,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { test("SPARK-4292 regression: result set iterator issue") { withJdbcStatement("test_4292") { statement => val queries = Seq( - "CREATE TABLE test_4292(key INT, val STRING)", + "CREATE TABLE test_4292(key INT, val STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_4292") queries.foreach(statement.execute) @@ -228,7 +228,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { test("SPARK-4309 regression: Date type support") { withJdbcStatement("test_date") { statement => val queries = Seq( - "CREATE TABLE test_date(key INT, value STRING)", + "CREATE TABLE test_date(key INT, value STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_date") queries.foreach(statement.execute) @@ -245,7 +245,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { test("SPARK-4407 regression: Complex type support") { withJdbcStatement("test_map") { statement => val queries = Seq( - "CREATE TABLE test_map(key INT, value STRING)", + "CREATE TABLE test_map(key INT, value STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map") queries.foreach(statement.execute) @@ -268,7 +268,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { test("SPARK-12143 regression: Binary type support") { withJdbcStatement("test_binary") { statement => val queries = Seq( - "CREATE TABLE test_binary(key INT, value STRING)", + "CREATE TABLE test_binary(key INT, value STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_binary") queries.foreach(statement.execute) @@ -294,7 +294,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { { statement => val queries = Seq( - "CREATE TABLE test_map(key INT, value STRING)", + "CREATE TABLE test_map(key INT, value STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map", "CACHE TABLE test_table AS SELECT key FROM test_map ORDER BY key DESC", "CREATE DATABASE db1") @@ -501,7 +501,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { { statement => val queries = Seq( - "CREATE TABLE smallKV(key INT, val STRING)", + "CREATE TABLE smallKV(key INT, val STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE smallKV", """CREATE TABLE addJar(key string) |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' @@ -598,7 +598,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val dataPath = "../hive/src/test/resources/data/files/kv1.txt" Seq( - "CREATE TABLE test_udtf(key INT, value STRING)", + "CREATE TABLE test_udtf(key INT, value STRING) USING hive", s"LOAD DATA LOCAL INPATH '$dataPath' OVERWRITE INTO TABLE test_udtf" ).foreach(statement.execute) @@ -879,7 +879,7 @@ class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { withJdbcStatement("test") { statement => val queries = Seq( "SET spark.sql.shuffle.partitions=3", - "CREATE TABLE test(key INT, val STRING)", + "CREATE TABLE test(key INT, val STRING) USING hive", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", "CACHE TABLE test") diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 36c19c680d165..29825e5116ef9 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -42,6 +42,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled private val originalSessionLocalTimeZone = TestHive.conf.sessionLocalTimeZone + private val originalCreateHiveTable = TestHive.conf.createHiveTableByDefaultEnabled def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -65,6 +66,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests // (timestamp_*) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") + TestHive.setConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED, true) RuleExecutor.resetMetrics() } @@ -77,6 +79,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, originalSessionLocalTimeZone) + TestHive.setConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED, + originalCreateHiveTable) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index b0cf25c3a7813..ed23f65815917 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -58,7 +58,7 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte | p_size INT, | p_container STRING, | p_retailprice DOUBLE, - | p_comment STRING) + | p_comment STRING) USING hive """.stripMargin) val testData1 = TestHive.getHiveFile("data/files/part_tiny.txt").getCanonicalPath sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala index 0386dc79804c6..6663193b40a4d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala @@ -178,14 +178,14 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet test("SPARK-24911: keep quotes for nested fields in hive") { withTable("t1") { - val createTable = "CREATE TABLE `t1`(`a` STRUCT<`b`: STRING>)" + val createTable = "CREATE TABLE `t1`(`a` STRUCT<`b`: STRING>) USING hive" sql(createTable) val shownDDL = sql(s"SHOW CREATE TABLE t1") .head() .getString(0) .split("\n") .head - assert(shownDDL == createTable) + assert(shownDDL == createTable.dropRight(" USING hive".length)) checkCreateTable("t1") } 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 40581066c62bb..99a45d6b45b89 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 @@ -158,7 +158,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto // Non-partitioned table val nonPartTable = "non_part_table" withTable(nonPartTable) { - sql(s"CREATE TABLE $nonPartTable (key STRING, value STRING)") + sql(s"CREATE TABLE $nonPartTable (key STRING, value STRING) USING hive") sql(s"INSERT INTO TABLE $nonPartTable SELECT * FROM src") sql(s"INSERT INTO TABLE $nonPartTable SELECT * FROM src") @@ -840,7 +840,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto test("alter table should not have the side effect to store statistics in Spark side") { val table = "alter_table_side_effect" withTable(table) { - sql(s"CREATE TABLE $table (i string, j string)") + sql(s"CREATE TABLE $table (i string, j string) USING hive") sql(s"INSERT INTO TABLE $table SELECT 'a', 'b'") val catalogTable1 = getCatalogTable(table) val hiveSize1 = BigInt(catalogTable1.ignoredProperties(StatsSetupConst.TOTAL_SIZE)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index b2bcdeaba6cf0..07cff1e7d99f0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -58,7 +58,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto |STORED AS PARQUET |TBLPROPERTIES('prop1Key'="prop1Val", '`prop2Key`'="prop2Val") """.stripMargin) - sql("CREATE TABLE parquet_tab3(col1 int, `col 2` int)") + sql("CREATE TABLE parquet_tab3(col1 int, `col 2` int) USING hive") sql("CREATE TABLE parquet_tab4 (price int, qty int) partitioned by (year int, month int)") sql("INSERT INTO parquet_tab4 PARTITION(year = 2015, month = 1) SELECT 1, 1") sql("INSERT INTO parquet_tab4 PARTITION(year = 2015, month = 2) SELECT 2, 2") 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 0684d66558c61..af9b6771b74f3 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 @@ -180,8 +180,8 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA test("SPARK-22431: illegal nested type") { val queries = Seq( - "CREATE TABLE t AS SELECT STRUCT('a' AS `$a`, 1 AS b) q", - "CREATE TABLE t(q STRUCT<`$a`:INT, col2:STRING>, i1 INT)", + "CREATE TABLE t USING hive AS SELECT STRUCT('a' AS `$a`, 1 AS b) q", + "CREATE TABLE t(q STRUCT<`$a`:INT, col2:STRING>, i1 INT) USING hive", "CREATE VIEW t AS SELECT STRUCT('a' AS `$a`, 1 AS b) q") queries.foreach(query => { @@ -252,7 +252,7 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA test("SPARK-22431: negative alter table tests with nested types") { withTable("t1") { - spark.sql("CREATE TABLE t1 (q STRUCT, i1 INT)") + spark.sql("CREATE TABLE t1 (q STRUCT, i1 INT) USING hive") val err = intercept[SparkException] { spark.sql("ALTER TABLE t1 ADD COLUMNS (newcol1 STRUCT<`$col1`:STRING, col2:Int>)") }.getMessage @@ -463,7 +463,7 @@ class HiveDDLSuite "create the table `default`.`tab1`")) e = intercept[AnalysisException] { - sql(s"CREATE TABLE tab2 location '${tempDir.getCanonicalPath}'") + sql(s"CREATE TABLE tab2 USING hive location '${tempDir.getCanonicalPath}'") }.getMessage assert(e.contains("Unable to infer the schema. The schema specification is required to " + "create the table `default`.`tab2`")) @@ -1626,7 +1626,7 @@ class HiveDDLSuite assert(spark.catalog.getTable("default", indexTabName).name === indexTabName) intercept[TableAlreadyExistsException] { - sql(s"CREATE TABLE $indexTabName(b int)") + sql(s"CREATE TABLE $indexTabName(b int) USING hive") } intercept[TableAlreadyExistsException] { sql(s"ALTER TABLE $tabName RENAME TO $indexTabName") @@ -2483,7 +2483,7 @@ class HiveDDLSuite test("load command for non local invalid path validation") { withTable("tbl") { - sql("CREATE TABLE tbl(i INT, j STRING)") + sql("CREATE TABLE tbl(i INT, j STRING) USING hive") val e = intercept[AnalysisException]( sql("load data inpath '/doesnotexist.csv' into table tbl")) assert(e.message.contains("LOAD DATA input path does not exist")) @@ -2700,4 +2700,31 @@ class HiveDDLSuite assert(table.storage.properties("test") == "test") } } + + test("SPARK-30098: create table without provider should " + + "use default data source under non-legacy mode") { + val catalog = spark.sessionState.catalog + withSQLConf( + SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED.key -> "false") { + withTable("s") { + val defaultProvider = conf.defaultDataSourceName + sql("CREATE TABLE s(a INT, b INT)") + val table = catalog.getTableMetadata(TableIdentifier("s")) + assert(table.provider === Some(defaultProvider)) + } + } + } + + test("SPARK-30098: create table without provider should " + + "use hive under legacy mode") { + val catalog = spark.sessionState.catalog + withSQLConf( + SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED.key -> "true") { + withTable("s") { + sql("CREATE TABLE s(a INT, b INT)") + val table = catalog.getTableMetadata(TableIdentifier("s")) + assert(table.provider === Some("hive")) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 68ccee5e6623a..921b46edc0a20 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -97,13 +97,14 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("explain create table command") { - checkKeywordsExist(sql("explain create table temp__b as select * from src limit 2"), + checkKeywordsExist(sql("explain create table temp__b using hive as select * from src limit 2"), "== Physical Plan ==", "InsertIntoHiveTable", "Limit", "src") - checkKeywordsExist(sql("explain extended create table temp__b as select * from src limit 2"), + checkKeywordsExist( + sql("explain extended create table temp__b using hive as select * from src limit 2"), "== Parsed Logical Plan ==", "== Analyzed Logical Plan ==", "== Optimized Logical Plan ==", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index ed4304b9aa57b..9a1190af02fac 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -83,15 +83,18 @@ class HiveSerDeSuite extends HiveComparisonTest with PlanTest with BeforeAndAfte } test("Test the default fileformat for Hive-serde tables") { - withSQLConf("hive.default.fileformat" -> "orc") { - val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + withSQLConf("hive.default.fileformat" -> "orc", + SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED.key -> "true") { + val (desc, exists) = extractTableDesc( + "CREATE TABLE IF NOT EXISTS fileformat_test (id int)") assert(exists) assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) assert(desc.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) } - withSQLConf("hive.default.fileformat" -> "parquet") { + withSQLConf("hive.default.fileformat" -> "parquet", + SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED.key -> "true") { val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") assert(exists) val input = desc.storage.inputFormat diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index d35c9336821ff..7dcd9dfac2861 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -775,7 +775,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql("CREATE TABLE test2 (key INT, value STRING)") testData.write.mode(SaveMode.Append).insertInto("test2") testData.write.mode(SaveMode.Append).insertInto("test2") - sql("CREATE TABLE test AS SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key") + sql("CREATE TABLE test USING hive AS " + + "SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key") checkAnswer( table("test"), sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq) @@ -936,7 +937,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { read.json(ds).createOrReplaceTempView("data") withSQLConf(SQLConf.CONVERT_CTAS.key -> "false") { - sql("CREATE TABLE explodeTest (key bigInt)") + sql("CREATE TABLE explodeTest (key bigInt) USING hive") table("explodeTest").queryExecution.analyzed match { case SubqueryAlias(_, r: HiveTableRelation) => // OK case _ => @@ -1890,7 +1891,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } withTable("load_t") { - sql("CREATE TABLE load_t (a STRING)") + sql("CREATE TABLE load_t (a STRING) USING hive") sql(s"LOAD DATA LOCAL INPATH '$path/*part-r*' INTO TABLE load_t") checkAnswer(sql("SELECT * FROM load_t"), Seq(Row("1"), Row("2"), Row("3"))) @@ -1910,7 +1911,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Files.write(s"$i", new File(dirPath, s"part-r-0000 $i"), StandardCharsets.UTF_8) } withTable("load_t") { - sql("CREATE TABLE load_t (a STRING)") + sql("CREATE TABLE load_t (a STRING) USING hive") sql(s"LOAD DATA LOCAL INPATH '$path/part-r-0000 1' INTO TABLE load_t") checkAnswer(sql("SELECT * FROM load_t"), Seq(Row("1"))) } @@ -1925,7 +1926,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Files.write(s"$i", new File(dirPath, s"part-r-0000$i"), StandardCharsets.UTF_8) } withTable("load_t_folder_wildcard") { - sql("CREATE TABLE load_t (a STRING)") + sql("CREATE TABLE load_t (a STRING) USING hive") sql(s"LOAD DATA LOCAL INPATH '${ path.substring(0, path.length - 1) .concat("*") @@ -1949,7 +1950,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Files.write(s"$i", new File(dirPath, s"part-r-0000$i"), StandardCharsets.UTF_8) } withTable("load_t1") { - sql("CREATE TABLE load_t1 (a STRING)") + sql("CREATE TABLE load_t1 (a STRING) USING hive") sql(s"LOAD DATA LOCAL INPATH '$path/part-r-0000?' INTO TABLE load_t1") checkAnswer(sql("SELECT * FROM load_t1"), Seq(Row("1"), Row("2"), Row("3"))) } @@ -1964,7 +1965,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Files.write(s"$i", new File(dirPath, s"part-r-0000$i"), StandardCharsets.UTF_8) } withTable("load_t2") { - sql("CREATE TABLE load_t2 (a STRING)") + sql("CREATE TABLE load_t2 (a STRING) USING hive") sql(s"LOAD DATA LOCAL INPATH '$path/?art-r-00001' INTO TABLE load_t2") checkAnswer(sql("SELECT * FROM load_t2"), Seq(Row("1"))) } @@ -2090,7 +2091,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { withTable("t") { df.createTempView("tempView") val e = intercept[AnalysisException] { - sql("CREATE TABLE t AS SELECT key, get_json_object(jstring, '$.f1') FROM tempView") + sql("CREATE TABLE t USING hive AS " + + "SELECT key, get_json_object(jstring, '$.f1') FROM tempView") }.getMessage assert(e.contains(expectedMsg)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala index 3f9485dd018b1..15712a18ce751 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala @@ -41,7 +41,7 @@ class WindowQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleto | p_size INT, | p_container STRING, | p_retailprice DOUBLE, - | p_comment STRING) + | p_comment STRING) USING hive """.stripMargin) val testData1 = TestHive.getHiveFile("data/files/part_tiny.txt").toURI sql( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 5fceab3105e83..cc4592a5caf68 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -328,10 +328,10 @@ private[hive] class TestHiveSparkSession( @transient val hiveQTestUtilTables: Seq[TestTable] = Seq( TestTable("src", - "CREATE TABLE src (key INT, value STRING)".cmd, + "CREATE TABLE src (key INT, value STRING) STORED AS TEXTFILE".cmd, s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), TestTable("src1", - "CREATE TABLE src1 (key INT, value STRING)".cmd, + "CREATE TABLE src1 (key INT, value STRING) STORED AS TEXTFILE".cmd, s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), TestTable("srcpart", () => { "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)" From 81996f9e4d8a17c3475a33af0c9c3d32cd70865f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 6 Dec 2019 12:01:36 -0800 Subject: [PATCH 0812/1387] [SPARK-30152][INFRA] Enable Hadoop-2.7/JDK11 build at GitHub Action ### What changes were proposed in this pull request? This PR enables JDK11 build with `hadoop-2.7` profile at `GitHub Action`. **BEFORE (6 jobs including one JDK11 job)** ![before](https://user-images.githubusercontent.com/9700541/70342731-7763f300-180a-11ea-859f-69038b88451f.png) **AFTER (7 jobs including two JDK11 jobs)** ![after](https://user-images.githubusercontent.com/9700541/70342658-54d1da00-180a-11ea-9fba-507fc087dc62.png) ### Why are the changes needed? SPARK-29957 makes JDK11 test work with `hadoop-2.7` profile. We need to protect it. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This is `GitHub Action` only PR. See the result of `GitHub Action` on this PR. Closes #26782 from dongjoon-hyun/SPARK-GHA-HADOOP-2.7. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 9338dc7bcbd5e..b884a182cce97 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -18,8 +18,6 @@ jobs: hadoop: [ 'hadoop-2.7', 'hadoop-3.2' ] hive: [ 'hive-1.2', 'hive-2.3' ] exclude: - - java: '11' - hadoop: 'hadoop-2.7' - java: '11' hive: 'hive-1.2' - hadoop: 'hadoop-3.2' From a30ec19a7358f18849944ecfab1d2b14e733614c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 6 Dec 2019 16:16:28 -0800 Subject: [PATCH 0813/1387] [SPARK-30155][SQL] Rename parse() to parseString() to avoid conflict in Scala 2.13 ### What changes were proposed in this pull request? Rename internal method LegacyTypeStringParser.parse() to parseString(). ### Why are the changes needed? In Scala 2.13, the parse() definition clashes with supertype declarations. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #26784 from srowen/SPARK-30155. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/LegacyTypeStringParser.scala | 2 +- .../main/scala/org/apache/spark/sql/types/StructType.scala | 2 +- .../execution/datasources/parquet/ParquetFileFormat.scala | 5 ++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/LegacyTypeStringParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/LegacyTypeStringParser.scala index 60d7361242c69..058136a213e9e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/LegacyTypeStringParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/LegacyTypeStringParser.scala @@ -84,7 +84,7 @@ object LegacyTypeStringParser extends RegexParsers { /** * Parses a string representation of a DataType. */ - def parse(asString: String): DataType = parseAll(dataType, asString) match { + def parseString(asString: String): DataType = parseAll(dataType, asString) match { case Success(result, _) => result case failure: NoSuccess => throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 236f73ba3832c..67d9e36f77755 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -490,7 +490,7 @@ object StructType extends AbstractDataType { override private[sql] def simpleString: String = "struct" private[sql] def fromString(raw: String): StructType = { - Try(DataType.fromJson(raw)).getOrElse(LegacyTypeStringParser.parse(raw)) match { + Try(DataType.fromJson(raw)).getOrElse(LegacyTypeStringParser.parseString(raw)) match { case t: StructType => t case _ => throw new RuntimeException(s"Failed parsing ${StructType.simpleString}: $raw") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index fb5daa4bfae3a..f52aaf0140e1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -27,7 +27,6 @@ import scala.util.{Failure, Try} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.input.FileSplit import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi @@ -399,7 +398,7 @@ object ParquetFileFormat extends Logging { logInfo( "Serialized Spark schema in Parquet key-value metadata is not in JSON format, " + "falling back to the deprecated DataType.fromCaseClassString parser.") - LegacyTypeStringParser.parse(serializedSchema.get) + LegacyTypeStringParser.parseString(serializedSchema.get) } .recover { case cause: Throwable => logWarning( @@ -510,7 +509,7 @@ object ParquetFileFormat extends Logging { logInfo( "Serialized Spark schema in Parquet key-value metadata is not in JSON format, " + "falling back to the deprecated DataType.fromCaseClassString parser.") - LegacyTypeStringParser.parse(schemaString).asInstanceOf[StructType] + LegacyTypeStringParser.parseString(schemaString).asInstanceOf[StructType] }.recoverWith { case cause: Throwable => logWarning( From 51aa7a920ec097ed2a797687de8382e21691f18c Mon Sep 17 00:00:00 2001 From: Aman Omer Date: Sat, 7 Dec 2019 10:58:02 +0900 Subject: [PATCH 0814/1387] [SPARK-30148][SQL] Optimize writing plans if there is an analysis exception ### What changes were proposed in this pull request? Optimized QueryExecution.scala#writePlans(). ### Why are the changes needed? If any query fails in Analysis phase and gets AnalysisException, there is no need to execute further phases since those will return a same result i.e, AnalysisException. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually Closes #26778 from amanomer/optExplain. Authored-by: Aman Omer Signed-off-by: HyukjinKwon --- .../apache/spark/sql/execution/QueryExecution.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 2935e1cbaf15d..a0ad1c72806f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -143,14 +143,14 @@ class QueryExecution( analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields) ) append("\n") + QueryPlan.append(analyzed, append, verbose, addSuffix, maxFields) + append("\n== Optimized Logical Plan ==\n") + QueryPlan.append(optimizedPlan, append, verbose, addSuffix, maxFields) + append("\n== Physical Plan ==\n") + QueryPlan.append(executedPlan, append, verbose, addSuffix, maxFields) } catch { - case _: AnalysisException => + case e: AnalysisException => append(e.toString) } - QueryPlan.append(analyzed, append, verbose, addSuffix, maxFields) - append("\n== Optimized Logical Plan ==\n") - QueryPlan.append(optimizedPlan, append, verbose, addSuffix, maxFields) - append("\n== Physical Plan ==\n") - QueryPlan.append(executedPlan, append, verbose, addSuffix, maxFields) } override def toString: String = withRedaction { From 1e0037b5e9ff077bdb59ad4536b7e5081a963089 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 7 Dec 2019 10:59:10 +0900 Subject: [PATCH 0815/1387] [SPARK-30157][BUILD][TEST-HADOOP3.2][TEST-JAVA11] Upgrade Apache HttpCore from 4.4.10 to 4.4.12 ### What changes were proposed in this pull request? This PR aims to upgrade `Apache HttpCore` from 4.4.10 to 4.4.12. ### Why are the changes needed? `Apache HttpCore v4.4.11` is the first official release for JDK11. > This is a maintenance release that corrects a number of defects in non-blocking SSL session code that caused compatibility issues with TLSv1.3 protocol implementation shipped with Java 11. For the full release note, please see the following. - https://www.apache.org/dist/httpcomponents/httpcore/RELEASE_NOTES-4.4.x.txt ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #26786 from dongjoon-hyun/SPARK-30157. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 2 +- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 371270702c56e..9f3c78f913b8f 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -85,7 +85,7 @@ hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.6.jar -httpcore-4.4.10.jar +httpcore-4.4.12.jar istack-commons-runtime-3.0.8.jar ivy-2.4.0.jar jackson-annotations-2.10.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index a2b603b33c0e3..dab39e39c3c1a 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -99,7 +99,7 @@ hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.6.jar -httpcore-4.4.10.jar +httpcore-4.4.12.jar istack-commons-runtime-3.0.8.jar ivy-2.4.0.jar jackson-annotations-2.10.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 99552febd1989..6800ebfd40aac 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -98,7 +98,7 @@ hk2-locator-2.5.0.jar hk2-utils-2.5.0.jar htrace-core4-4.1.0-incubating.jar httpclient-4.5.6.jar -httpcore-4.4.10.jar +httpcore-4.4.12.jar istack-commons-runtime-3.0.8.jar ivy-2.4.0.jar jackson-annotations-2.10.0.jar diff --git a/pom.xml b/pom.xml index 16a7eb544bba8..cdf7704506a8f 100644 --- a/pom.xml +++ b/pom.xml @@ -158,7 +158,7 @@ 0.12.8 4.5.6 - 4.4.10 + 4.4.12 3.1 3.4.1 From afc4fa02bd2b7eb835e5c5dcbe0cbd1303910b42 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 6 Dec 2019 18:49:43 -0800 Subject: [PATCH 0816/1387] [SPARK-30156][BUILD] Upgrade Jersey from 2.29 to 2.29.1 ### What changes were proposed in this pull request? This PR aims to upgrade `Jersey` from 2.29 to 2.29.1. ### Why are the changes needed? This will bring several bug fixes and important dependency upgrades. - https://eclipse-ee4j.github.io/jersey.github.io/release-notes/2.29.1.html ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #26785 from dongjoon-hyun/SPARK-30156. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 30 ++++++++++++------------- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 30 ++++++++++++------------- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 30 ++++++++++++------------- pom.xml | 2 +- 4 files changed, 46 insertions(+), 46 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 9f3c78f913b8f..5927709407f3a 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -10,7 +10,7 @@ antlr-2.7.7.jar antlr-runtime-3.4.jar antlr4-runtime-4.7.1.jar aopalliance-1.0.jar -aopalliance-repackaged-2.5.0.jar +aopalliance-repackaged-2.6.1.jar apache-log4j-extras-1.2.17.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar @@ -80,9 +80,9 @@ hadoop-yarn-client-2.7.4.jar hadoop-yarn-common-2.7.4.jar hadoop-yarn-server-common-2.7.4.jar hadoop-yarn-server-web-proxy-2.7.4.jar -hk2-api-2.5.0.jar -hk2-locator-2.5.0.jar -hk2-utils-2.5.0.jar +hk2-api-2.6.1.jar +hk2-locator-2.6.1.jar +hk2-utils-2.6.1.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.12.jar @@ -100,9 +100,10 @@ jackson-module-paranamer-2.10.0.jar jackson-module-scala_2.12-2.10.0.jar jackson-xc-1.9.13.jar jakarta.activation-api-1.2.1.jar -jakarta.annotation-api-1.3.4.jar -jakarta.inject-2.5.0.jar -jakarta.ws.rs-api-2.1.5.jar +jakarta.annotation-api-1.3.5.jar +jakarta.inject-2.6.1.jar +jakarta.validation-api-2.0.2.jar +jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api-2.3.2.jar janino-3.0.15.jar javassist-3.22.0-CR2.jar @@ -113,13 +114,13 @@ jaxb-api-2.2.2.jar jaxb-runtime-2.3.2.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-2.29.jar -jersey-common-2.29.jar -jersey-container-servlet-2.29.jar -jersey-container-servlet-core-2.29.jar -jersey-hk2-2.29.jar -jersey-media-jaxb-2.29.jar -jersey-server-2.29.jar +jersey-client-2.29.1.jar +jersey-common-2.29.1.jar +jersey-container-servlet-2.29.1.jar +jersey-container-servlet-core-2.29.1.jar +jersey-hk2-2.29.1.jar +jersey-media-jaxb-2.29.1.jar +jersey-server-2.29.1.jar jetty-6.1.26.jar jetty-sslengine-6.1.26.jar jetty-util-6.1.26.jar @@ -199,7 +200,6 @@ stream-2.9.6.jar stringtemplate-3.2.1.jar super-csv-2.2.0.jar univocity-parsers-2.8.3.jar -validation-api-2.0.1.Final.jar xbean-asm7-shaded-4.15.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index dab39e39c3c1a..4b63fa0f4076c 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -9,7 +9,7 @@ algebra_2.12-2.0.0-M2.jar antlr-runtime-3.5.2.jar antlr4-runtime-4.7.1.jar aopalliance-1.0.jar -aopalliance-repackaged-2.5.0.jar +aopalliance-repackaged-2.6.1.jar apacheds-i18n-2.0.0-M15.jar apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar @@ -94,9 +94,9 @@ hive-shims-common-2.3.6.jar hive-shims-scheduler-2.3.6.jar hive-storage-api-2.6.0.jar hive-vector-code-gen-2.3.6.jar -hk2-api-2.5.0.jar -hk2-locator-2.5.0.jar -hk2-utils-2.5.0.jar +hk2-api-2.6.1.jar +hk2-locator-2.6.1.jar +hk2-utils-2.6.1.jar htrace-core-3.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.12.jar @@ -114,9 +114,10 @@ jackson-module-paranamer-2.10.0.jar jackson-module-scala_2.12-2.10.0.jar jackson-xc-1.9.13.jar jakarta.activation-api-1.2.1.jar -jakarta.annotation-api-1.3.4.jar -jakarta.inject-2.5.0.jar -jakarta.ws.rs-api-2.1.5.jar +jakarta.annotation-api-1.3.5.jar +jakarta.inject-2.6.1.jar +jakarta.validation-api-2.0.2.jar +jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api-2.3.2.jar janino-3.0.15.jar javassist-3.22.0-CR2.jar @@ -128,13 +129,13 @@ jaxb-api-2.2.2.jar jaxb-runtime-2.3.2.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-2.29.jar -jersey-common-2.29.jar -jersey-container-servlet-2.29.jar -jersey-container-servlet-core-2.29.jar -jersey-hk2-2.29.jar -jersey-media-jaxb-2.29.jar -jersey-server-2.29.jar +jersey-client-2.29.1.jar +jersey-common-2.29.1.jar +jersey-container-servlet-2.29.1.jar +jersey-container-servlet-core-2.29.1.jar +jersey-hk2-2.29.1.jar +jersey-media-jaxb-2.29.1.jar +jersey-server-2.29.1.jar jetty-6.1.26.jar jetty-sslengine-6.1.26.jar jetty-util-6.1.26.jar @@ -213,7 +214,6 @@ stream-2.9.6.jar super-csv-2.2.0.jar transaction-api-1.1.jar univocity-parsers-2.8.3.jar -validation-api-2.0.1.Final.jar velocity-1.5.jar xbean-asm7-shaded-4.15.jar xercesImpl-2.9.1.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 6800ebfd40aac..979c6f9fd5ccd 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -10,7 +10,7 @@ algebra_2.12-2.0.0-M2.jar antlr-runtime-3.5.2.jar antlr4-runtime-4.7.1.jar aopalliance-1.0.jar -aopalliance-repackaged-2.5.0.jar +aopalliance-repackaged-2.6.1.jar arpack_combined_all-0.1.jar arrow-format-0.15.1.jar arrow-memory-0.15.1.jar @@ -93,9 +93,9 @@ hive-shims-common-2.3.6.jar hive-shims-scheduler-2.3.6.jar hive-storage-api-2.6.0.jar hive-vector-code-gen-2.3.6.jar -hk2-api-2.5.0.jar -hk2-locator-2.5.0.jar -hk2-utils-2.5.0.jar +hk2-api-2.6.1.jar +hk2-locator-2.6.1.jar +hk2-utils-2.6.1.jar htrace-core4-4.1.0-incubating.jar httpclient-4.5.6.jar httpcore-4.4.12.jar @@ -113,9 +113,10 @@ jackson-module-jaxb-annotations-2.10.0.jar jackson-module-paranamer-2.10.0.jar jackson-module-scala_2.12-2.10.0.jar jakarta.activation-api-1.2.1.jar -jakarta.annotation-api-1.3.4.jar -jakarta.inject-2.5.0.jar -jakarta.ws.rs-api-2.1.5.jar +jakarta.annotation-api-1.3.5.jar +jakarta.inject-2.6.1.jar +jakarta.validation-api-2.0.2.jar +jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api-2.3.2.jar janino-3.0.15.jar javassist-3.22.0-CR2.jar @@ -128,13 +129,13 @@ jaxb-runtime-2.3.2.jar jcip-annotations-1.0-1.jar jcl-over-slf4j-1.7.16.jar jdo-api-3.0.1.jar -jersey-client-2.29.jar -jersey-common-2.29.jar -jersey-container-servlet-2.29.jar -jersey-container-servlet-core-2.29.jar -jersey-hk2-2.29.jar -jersey-media-jaxb-2.29.jar -jersey-server-2.29.jar +jersey-client-2.29.1.jar +jersey-common-2.29.1.jar +jersey-container-servlet-2.29.1.jar +jersey-container-servlet-core-2.29.1.jar +jersey-hk2-2.29.1.jar +jersey-media-jaxb-2.29.1.jar +jersey-server-2.29.1.jar jline-2.14.6.jar joda-time-2.10.5.jar jodd-core-3.5.2.jar @@ -230,7 +231,6 @@ super-csv-2.2.0.jar token-provider-1.0.1.jar transaction-api-1.1.jar univocity-parsers-2.8.3.jar -validation-api-2.0.1.Final.jar velocity-1.5.jar woodstox-core-5.0.3.jar xbean-asm7-shaded-4.15.jar diff --git a/pom.xml b/pom.xml index cdf7704506a8f..617e0c36a93a3 100644 --- a/pom.xml +++ b/pom.xml @@ -183,7 +183,7 @@ 2.6.2 4.1.17 3.0.15 - 2.29 + 2.29.1 2.10.5 3.5.2 3.0.0 From e88d74052bf40eabab9e3388fa09e52097ffa3aa Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 7 Dec 2019 15:03:51 +0900 Subject: [PATCH 0817/1387] [SPARK-30147][SQL] Trim the string when cast string type to booleans MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Now, we trim the string when casting string value to those `canCast` types values, e.g. int, double, decimal, interval, date, timestamps, except for boolean. This behavior makes type cast and coercion inconsistency in Spark. Not fitting ANSI SQL standard either. ``` If TD is boolean, then Case: a) If SD is character string, then SV is replaced by TRIM ( BOTH ' ' FROM VE ) Case: i) If the rules for literal in Subclause 5.3, “literal”, can be applied to SV to determine a valid value of the data type TD, then let TV be that value. ii) Otherwise, an exception condition is raised: data exception — invalid character value for cast. b) If SD is boolean, then TV is SV ``` In this pull request, we trim all the whitespaces from both ends of the string before converting it to a bool value. This behavior is as same as others, but a bit different from sql standard, which trim only spaces. ### Why are the changes needed? Type cast/coercion consistency ### Does this PR introduce any user-facing change? yes, string with whitespaces in both ends will be trimmed before converted to booleans. e.g. `select cast('\t true' as boolean)` results `true` now, before this pr it's `null` ### How was this patch tested? add unit tests Closes #26776 from yaooqinn/SPARK-30147. Authored-by: Kent Yao Signed-off-by: Takeshi Yamamuro --- docs/sql-migration-guide.md | 4 +-- .../spark/sql/catalyst/util/StringUtils.scala | 5 ++-- .../sql/catalyst/expressions/CastSuite.scala | 2 ++ .../test/resources/sql-tests/inputs/cast.sql | 7 ++++- .../resources/sql-tests/results/cast.sql.out | 29 ++++++++++++++++++- 5 files changed, 39 insertions(+), 8 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index e22e4158f2490..9bcd36ce41271 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -222,10 +222,8 @@ license: | - Since Spark 3.0, when casting interval values to string type, there is no "interval" prefix, e.g. `1 days 2 hours`. In Spark version 2.4 and earlier, the string contains the "interval" prefix like `interval 1 days 2 hours`. - - Since Spark 3.0, when casting string value to integral types, including tinyint, smallint, int and bigint type, the leading and trailing white spaces(<= ACSII 32) will be trimmed before convert to integral values, e.g. `cast(' 1 ' as int)` results `1`. In Spark version 2.4 and earlier, the result will be `null`. + - Since Spark 3.0, when casting string value to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing whitespaces(<= ACSII 32) will be trimmed before converted to these type values, e.g. `cast(' 1\t' as int)` results `1`, `cast(' 1\t' as boolean)` results `true`, `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, while casting string to integrals and booleans, it will not trim the whitespaces from both ends, the foregoing results will be `null`, while to datetimes, only the trailing spaces(= ASCII 32) will be removed. - - Since Spark 3.0, when casting string value to date, timestamp and interval values, the leading and trailing white spaces(<= ACSII 32) will be trimmed before casing, e.g. `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, only the trailing space will be removed, thus, the result is `null`. - - Since Spark 3.0, numbers written in scientific notation(e.g. `1E2`) would be parsed as Double. In Spark version 2.4 and earlier, they're parsed as Decimal. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.exponentLiteralAsDecimal.enabled` to `true`. - Since Spark 3.0, we pad decimal numbers with trailing zeros to the scale of the column for `spark-sql` interface, for example: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index d6a649b791fb5..51ab3df0e0bfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.util -import java.util.concurrent.atomic.AtomicBoolean import java.util.regex.{Pattern, PatternSyntaxException} import scala.collection.mutable.ArrayBuffer @@ -74,9 +73,9 @@ object StringUtils extends Logging { Set("f", "false", "n", "no", "0").map(UTF8String.fromString) // scalastyle:off caselocale - def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s.toLowerCase) + def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s.trimAll().toLowerCase) - def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s.toLowerCase) + def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s.trimAll().toLowerCase) // scalastyle:on caselocale /** 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 a2359d655a1ef..dde25ad994c5b 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 @@ -685,8 +685,10 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkCast("y", true) checkCast("yes", true) checkCast("1", true) + checkCast("1 ", true) checkCast("f", false) + checkCast("f\t", false) checkCast("false", false) checkCast("FAlsE", false) checkCast("n", false) diff --git a/sql/core/src/test/resources/sql-tests/inputs/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/cast.sql index 3c1702e6f837e..972ebdd01f61e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cast.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cast.sql @@ -69,4 +69,9 @@ select cast(' 1' as INT); select cast(' 1' as bigint); select cast(' 1' as float); select cast(' 1 ' as DOUBLE); -select cast('1.0 ' as DEC); \ No newline at end of file +select cast('1.0 ' as DEC); + +-- trim string before cast to boolean +select cast('\t\t true \n\r ' as boolean); +select cast('\t\n false \t\r' as boolean); +select cast('\t\n xyz \t\r' as boolean); diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 112826a3d3158..343f9748c5ba8 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 43 +-- Number of queries: 46 -- !query 0 @@ -351,3 +351,30 @@ select cast('1.0 ' as DEC) struct -- !query 42 output 1 + + +-- !query 43 +select cast('\t\t true \n\r ' as boolean) +-- !query 43 schema +struct +-- !query 43 output +true + + +-- !query 44 +select cast('\t\n false \t\r' as boolean) +-- !query 44 schema +struct +-- !query 44 output +false + + +-- !query 45 +select cast('\t\n xyz \t\r' as boolean) +-- !query 45 schema +struct +-- !query 45 output +NULL From 1068b8b24910eec8122bf7fa4748a101becf0d2b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 7 Dec 2019 12:04:10 -0800 Subject: [PATCH 0818/1387] [SPARK-30163][INFRA] Use Google Maven mirror in GitHub Action ### What changes were proposed in this pull request? This PR aims to use [Google Maven mirror](https://cloudplatform.googleblog.com/2015/11/faster-builds-for-Java-developers-with-Maven-Central-mirror.html) in `GitHub Action` jobs to improve the stability. ```xml google-maven-central GCS Maven Central mirror https://maven-central.storage-download.googleapis.com/repos/central/data/ central ``` ### Why are the changes needed? Although we added Maven cache inside `GitHub Action`, the timeouts happen too frequently during access `artifact descriptor`. ``` [ERROR] Failed to execute goal on project spark-mllib_2.12: ... Failed to read artifact descriptor for ... ... Connection timed out (Read failed) -> [Help 1] ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This PR is irrelevant to Jenkins. This is tested on the personal repository first. `GitHub Action` of this PR should pass. - https://github.com/dongjoon-hyun/spark/pull/11 Closes #26793 from dongjoon-hyun/SPARK-30163. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index b884a182cce97..2624e0bb9ca14 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -65,6 +65,7 @@ jobs: run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" + echo "google-maven-centralGCS Maven Central mirrorhttps://maven-central.storage-download.googleapis.com/repos/central/data/central" > ~/.m2/settings.xml ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -P${{ matrix.hive }} -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install rm -rf ~/.m2/repository/org/apache/spark From 16f1b23d75c0b44aac61111bfb2ae9bb0f3fab68 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 7 Dec 2019 12:58:00 -0800 Subject: [PATCH 0819/1387] [SPARK-30163][INFRA][FOLLOWUP] Make `.m2` directory for cold start without cache ### What changes were proposed in this pull request? This PR is a follow-up of https://github.com/apache/spark/pull/26793 and aims to initialize `~/.m2` directory. ### Why are the changes needed? In case of cache reset, `~/.m2` directory doesn't exist. It causes a failure. - `master` branch has a cache as of now. So, we missed this. - `branch-2.4` has no cache as of now, and we hit this failure. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This PR is tested against personal `branch-2.4`. - https://github.com/dongjoon-hyun/spark/pull/12 Closes #26794 from dongjoon-hyun/SPARK-30163-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/master.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 2624e0bb9ca14..7524ac64f3b92 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -65,6 +65,7 @@ jobs: run: | export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" + mkdir -p ~/.m2 echo "google-maven-centralGCS Maven Central mirrorhttps://maven-central.storage-download.googleapis.com/repos/central/data/central" > ~/.m2/settings.xml ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -P${{ matrix.hive }} -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install rm -rf ~/.m2/repository/org/apache/spark From bca9de66847dab562d44d65a284bf75e7ede6421 Mon Sep 17 00:00:00 2001 From: Pablo Langa Date: Sun, 8 Dec 2019 20:15:09 -0800 Subject: [PATCH 0820/1387] [SPARK-29922][SQL] SHOW FUNCTIONS should do multi-catalog resolution MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add ShowFunctionsStatement and make SHOW FUNCTIONS go through the same catalog/table resolution framework of v2 commands. We don’t have this methods in the catalog to implement an V2 command * catalog.listFunctions ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing `SHOW FUNCTIONS LIKE namespace.function` ### Does this PR introduce any user-facing change? Yes. When running SHOW FUNCTIONS LIKE namespace.function Spark fails the command if the current catalog is set to a v2 catalog. ### How was this patch tested? Unit tests. Closes #26667 from planga82/feature/SPARK-29922_ShowFunctions_V2Catalog. Authored-by: Pablo Langa Signed-off-by: Liang-Chi Hsieh --- .../sql/catalyst/parser/AstBuilder.scala | 16 ++++++++++++ .../catalyst/plans/logical/statements.scala | 9 +++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 26 +++++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 17 ++++++++++++ .../spark/sql/execution/SparkSqlParser.scala | 24 ----------------- .../sql/connector/DataSourceV2SQLSuite.scala | 9 +++++++ .../sql/execution/SparkSqlParserSuite.scala | 15 ----------- .../hive/execution/HiveComparisonTest.scala | 2 +- 8 files changed, 78 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 310746585289a..858870a161417 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3358,4 +3358,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging visitMultipartIdentifier(ctx.table), Option(ctx.key).map(visitTablePropertyKey)) } + + /** + * Create a plan for a SHOW FUNCTIONS command. + */ + override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { + val (userScope, systemScope) = Option(ctx.identifier) + .map(_.getText.toLowerCase(Locale.ROOT)) match { + case None | Some("all") => (true, true) + case Some("system") => (false, true) + case Some("user") => (true, false) + case Some(x) => throw new ParseException(s"SHOW $x FUNCTIONS not supported", ctx) + } + val pattern = Option(ctx.pattern).map(string(_)) + val functionName = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) + ShowFunctionsStatement(userScope, systemScope, pattern, functionName) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 5db099e1de631..13356bfd04ffd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -485,3 +485,12 @@ case class ShowCurrentNamespaceStatement() extends ParsedStatement case class ShowTablePropertiesStatement( tableName: Seq[String], propertyKey: Option[String]) extends ParsedStatement + +/** + * SHOW FUNCTIONS statement, as parsed from SQL + */ +case class ShowFunctionsStatement( + userScope: Boolean, + systemScope: Boolean, + pattern: Option[String], + functionName: Option[Seq[String]]) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index fb965b5bbf1e8..b0d9a00d653ce 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1799,6 +1799,32 @@ class DDLParserSuite extends AnalysisTest { ShowTablePropertiesStatement(Seq("a", "b", "c"), Some("propKey1"))) } + test("SHOW FUNCTIONS") { + comparePlans( + parsePlan("SHOW FUNCTIONS"), + ShowFunctionsStatement(true, true, None, None)) + comparePlans( + parsePlan("SHOW USER FUNCTIONS"), + ShowFunctionsStatement(true, false, None, None)) + comparePlans( + parsePlan("SHOW user FUNCTIONS"), + ShowFunctionsStatement(true, false, None, None)) + comparePlans( + parsePlan("SHOW SYSTEM FUNCTIONS"), + ShowFunctionsStatement(false, true, None, None)) + comparePlans( + parsePlan("SHOW ALL FUNCTIONS"), + ShowFunctionsStatement(true, true, None, None)) + comparePlans( + parsePlan("SHOW FUNCTIONS LIKE 'funct*'"), + ShowFunctionsStatement(true, true, Some("funct*"), None)) + comparePlans( + parsePlan("SHOW FUNCTIONS LIKE a.b.c"), + ShowFunctionsStatement(true, true, None, Some(Seq("a", "b", "c")))) + val sql = "SHOW other FUNCTIONS" + intercept(sql, s"$sql not supported") + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 4cc701274a1f0..c7ef878cb8c92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -471,6 +471,23 @@ class ResolveSessionCatalog( ShowTablePropertiesCommand( tableName.asTableIdentifier, propertyKey) + + case ShowFunctionsStatement(userScope, systemScope, pattern, fun) => + val (database, function) = fun match { + case Some(CatalogAndIdentifierParts(catalog, functionName)) => + if (isSessionCatalog(catalog)) { + functionName match { + case Seq(db, fn) => (Some(db), Some(fn)) + case Seq(fn) => (None, Some(fn)) + case _ => + throw new AnalysisException(s"Unsupported function name '${functionName.quoted}'") + } + } else { + throw new AnalysisException ("SHOW FUNCTIONS is only supported in v1 catalog") + } + case None => (None, pattern) + } + ShowFunctionsCommand(database, function, userScope, systemScope) } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 20e0ccdf0e72d..37edad6b5a30d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -236,30 +236,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { DescribeFunctionCommand(functionName, EXTENDED != null) } - /** - * Create a plan for a SHOW FUNCTIONS command. - */ - override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) { - import ctx._ - val (user, system) = Option(ctx.identifier).map(_.getText.toLowerCase(Locale.ROOT)) match { - case None | Some("all") => (true, true) - case Some("system") => (false, true) - case Some("user") => (true, false) - case Some(x) => throw new ParseException(s"SHOW $x FUNCTIONS not supported", ctx) - } - - val (db, pat) = if (multipartIdentifier != null) { - val name = visitFunctionName(multipartIdentifier) - (name.database, Some(name.funcName)) - } else if (pattern != null) { - (None, Some(string(pattern))) - } else { - (None, None) - } - - ShowFunctionsCommand(db, pat, user, system) - } - /** * Create a [[CreateFunctionCommand]] command. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index e2f8776682bb8..ebaf753696fd5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1799,6 +1799,15 @@ class DataSourceV2SQLSuite } } + test("SHOW FUNCTIONS not valid v1 namespace") { + val function = "testcat.ns1.ns2.fun" + + val e = intercept[AnalysisException] { + sql(s"SHOW FUNCTIONS LIKE $function") + } + assert(e.message.contains("SHOW FUNCTIONS is only supported in v1 catalog")) + } + test("global temp view should not be masked by v2 catalog") { val globalTempDB = spark.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) spark.conf.set(s"spark.sql.catalog.$globalTempDB", classOf[InMemoryTableCatalog].getName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 7e9b0d824fb9d..ddf3a1c312ea2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -80,21 +80,6 @@ class SparkSqlParserSuite extends AnalysisTest { intercept("REFRESH", "Resource paths cannot be empty in REFRESH statements") } - test("show functions") { - assertEqual("show functions", ShowFunctionsCommand(None, None, true, true)) - assertEqual("show all functions", ShowFunctionsCommand(None, None, true, true)) - assertEqual("show user functions", ShowFunctionsCommand(None, None, true, false)) - assertEqual("show system functions", ShowFunctionsCommand(None, None, false, true)) - intercept("show special functions", "SHOW special FUNCTIONS") - assertEqual("show functions foo", - ShowFunctionsCommand(None, Some("foo"), true, true)) - assertEqual("show functions foo.bar", - ShowFunctionsCommand(Some("foo"), Some("bar"), true, true)) - assertEqual("show functions 'foo\\\\.*'", - ShowFunctionsCommand(None, Some("foo\\.*"), true, true)) - intercept("show functions foo.bar.baz", "Unsupported function name") - } - test("describe function") { assertEqual("describe function bar", DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = false)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 8bf2e19b929e7..b506c107d73d5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -373,7 +373,7 @@ abstract class HiveComparisonTest // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && - (!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctionsStatement]) && (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && (!hiveQuery.logical.isInstanceOf[DescribeTableStatement]) && From a57bbf2ee02e30053e67f62a0afd6f525bba5c66 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 9 Dec 2019 13:15:49 +0900 Subject: [PATCH 0821/1387] [SPARK-30164][TESTS][DOCS] Exclude Hive domain in Unidoc build explicitly ### What changes were proposed in this pull request? This PR proposes to exclude Unidoc checking in Hive domain. We don't publish this as a part of Spark documentation (see also https://github.com/apache/spark/blob/master/docs/_plugins/copy_api_dirs.rb#L30) and most of them are copy of Hive thrift server so that we can officially use Hive 2.3 release. It doesn't much make sense to check the documentation generation against another domain, and that we don't use in documentation publish. ### Why are the changes needed? To avoid unnecessary computation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? By Jenkins: ``` ======================================================================== Building Spark ======================================================================== [info] Building Spark using SBT with these arguments: -Phadoop-2.7 -Phive-2.3 -Phive -Pmesos -Pkubernetes -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pspark-ganglia-lgpl -Pyarn test:package streaming-kinesis-asl-assembly/assembly ... ======================================================================== Building Unidoc API Documentation ======================================================================== [info] Building Spark unidoc using SBT with these arguments: -Phadoop-2.7 -Phive-2.3 -Phive -Pmesos -Pkubernetes -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pspark-ganglia-lgpl -Pyarn unidoc ... [info] Main Java API documentation successful. ... [info] Main Scala API documentation successful. ``` Closes #26800 from HyukjinKwon/do-not-merge. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- project/SparkBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f91fb4df4fd43..8c3b5f49bcc3f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -834,6 +834,7 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/internal"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/hive/test"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalog/v2/utils"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hive"))) } private def ignoreClasspaths(classpaths: Seq[Classpath]): Seq[Classpath] = { From dcea7a4c9a04190dffec184eb286e9709faf3272 Mon Sep 17 00:00:00 2001 From: Aman Omer Date: Mon, 9 Dec 2019 13:23:16 +0800 Subject: [PATCH 0822/1387] [SPARK-29883][SQL] Implement a helper method for aliasing bool_and() and bool_or() ### What changes were proposed in this pull request? This PR introduces a method `expressionWithAlias` in class `FunctionRegistry` which is used to register function's constructor. Currently, `expressionWithAlias` is used to register `BoolAnd` & `BoolOr`. ### Why are the changes needed? Error message is wrong when alias name is used for `BoolAnd` & `BoolOr`. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Tested manually. For query, `select every('true');` Output before this PR, > Error in query: cannot resolve 'bool_and('true')' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [string].; line 1 pos 7; After this PR, > Error in query: cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7; Closes #26712 from amanomer/29883. Authored-by: Aman Omer Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 70 ++++++++++++++----- .../aggregate/UnevaluableAggs.scala | 8 +-- .../catalyst/optimizer/finishAnalysis.scala | 4 +- .../ExpressionTypeCheckingSuite.scala | 4 +- .../sql-tests/results/group-by.sql.out | 26 +++---- .../results/udf/udf-group-by.sql.out | 26 +++---- .../org/apache/spark/sql/ExplainSuite.scala | 4 +- 7 files changed, 89 insertions(+), 53 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 7cc64d43858c9..f8a0c528bff0a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -313,11 +313,11 @@ object FunctionRegistry { expression[CollectList]("collect_list"), expression[CollectSet]("collect_set"), expression[CountMinSketchAgg]("count_min_sketch"), - expression[BoolAnd]("every"), - expression[BoolAnd]("bool_and"), - expression[BoolOr]("any"), - expression[BoolOr]("some"), - expression[BoolOr]("bool_or"), + expressionWithAlias[BoolAnd]("every"), + expressionWithAlias[BoolAnd]("bool_and"), + expressionWithAlias[BoolOr]("any"), + expressionWithAlias[BoolOr]("some"), + expressionWithAlias[BoolOr]("bool_or"), // string functions expression[Ascii]("ascii"), @@ -590,12 +590,12 @@ object FunctionRegistry { val builder = (expressions: Seq[Expression]) => { if (varargCtor.isDefined) { // If there is an apply method that accepts Seq[Expression], use that one. - Try(varargCtor.get.newInstance(expressions).asInstanceOf[Expression]) match { - case Success(e) => e - case Failure(e) => - // the exception is an invocation exception. To get a meaningful message, we need the - // cause. - throw new AnalysisException(e.getCause.getMessage) + try { + varargCtor.get.newInstance(expressions).asInstanceOf[Expression] + } catch { + // the exception is an invocation exception. To get a meaningful message, we need the + // cause. + case e: Exception => throw new AnalysisException(e.getCause.getMessage) } } else { // Otherwise, find a constructor method that matches the number of arguments, and use that. @@ -618,12 +618,12 @@ object FunctionRegistry { } throw new AnalysisException(invalidArgumentsMsg) } - Try(f.newInstance(expressions : _*).asInstanceOf[Expression]) match { - case Success(e) => e - case Failure(e) => - // the exception is an invocation exception. To get a meaningful message, we need the - // cause. - throw new AnalysisException(e.getCause.getMessage) + try { + f.newInstance(expressions : _*).asInstanceOf[Expression] + } catch { + // the exception is an invocation exception. To get a meaningful message, we need the + // cause. + case e: Exception => throw new AnalysisException(e.getCause.getMessage) } } } @@ -631,6 +631,42 @@ object FunctionRegistry { (name, (expressionInfo[T](name), builder)) } + private def expressionWithAlias[T <: Expression](name: String) + (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = { + val constructors = tag.runtimeClass.getConstructors + .filter(_.getParameterTypes.head == classOf[String]) + assert(constructors.length == 1) + val builder = (expressions: Seq[Expression]) => { + val params = classOf[String] +: Seq.fill(expressions.size)(classOf[Expression]) + val f = constructors.find(_.getParameterTypes.toSeq == params).getOrElse { + val validParametersCount = constructors + .filter(_.getParameterTypes.tail.forall(_ == classOf[Expression])) + .map(_.getParameterCount - 1).distinct.sorted + val invalidArgumentsMsg = if (validParametersCount.length == 0) { + s"Invalid arguments for function $name" + } else { + val expectedNumberOfParameters = if (validParametersCount.length == 1) { + validParametersCount.head.toString + } else { + validParametersCount.init.mkString("one of ", ", ", " and ") + + validParametersCount.last + } + s"Invalid number of arguments for function $name. " + + s"Expected: $expectedNumberOfParameters; Found: ${expressions.size}" + } + throw new AnalysisException(invalidArgumentsMsg) + } + try { + f.newInstance(name.toString +: expressions: _*).asInstanceOf[Expression] + } catch { + // the exception is an invocation exception. To get a meaningful message, we need the + // cause. + case e: Exception => throw new AnalysisException(e.getCause.getMessage) + } + } + (name, (expressionInfo[T](name), builder)) + } + /** * Creates a function registry lookup entry for cast aliases (SPARK-16730). * For example, if name is "int", and dataType is IntegerType, this means int(x) would become diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala index c559fefe3a80c..acb0af0248a74 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala @@ -52,8 +52,8 @@ abstract class UnevaluableBooleanAggBase(arg: Expression) false """, since = "3.0.0") -case class BoolAnd(arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = "bool_and" +case class BoolAnd(funcName: String, arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = funcName } @ExpressionDescription( @@ -68,6 +68,6 @@ case class BoolAnd(arg: Expression) extends UnevaluableBooleanAggBase(arg) { false """, since = "3.0.0") -case class BoolOr(arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = "bool_or" +case class BoolOr(funcName: String, arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = funcName } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index f64b6e00373f6..c330274341524 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -47,8 +47,8 @@ object ReplaceExpressions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case e: RuntimeReplaceable => e.child case CountIf(predicate) => Count(new NullIf(predicate, Literal.FalseLiteral)) - case BoolOr(arg) => Max(arg) - case BoolAnd(arg) => Min(arg) + case BoolOr(_, arg) => Max(arg) + case BoolAnd(_, arg) => Min(arg) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index feb927264ba68..c83759e8f4c14 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -144,8 +144,8 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertSuccess(Sum('stringField)) assertSuccess(Average('stringField)) assertSuccess(Min('arrayField)) - assertSuccess(new BoolAnd('booleanField)) - assertSuccess(new BoolOr('booleanField)) + assertSuccess(new BoolAnd("bool_and", 'booleanField)) + assertSuccess(new BoolOr("bool_or", 'booleanField)) assertError(Min('mapField), "min does not support ordering on type") assertError(Max('mapField), "max does not support ordering on type") diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 8b6e370a9867a..ed5ced8c8c0fe 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -293,7 +293,7 @@ struct<> -- !query 31 SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output NULL NULL NULL NULL NULL @@ -301,7 +301,7 @@ NULL NULL NULL NULL NULL -- !query 32 SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output NULL NULL NULL NULL NULL @@ -309,7 +309,7 @@ NULL NULL NULL NULL NULL -- !query 33 SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output false true true false true @@ -317,7 +317,7 @@ false true true false true -- !query 34 SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k -- !query 34 schema -struct +struct -- !query 34 output 1 false true true false true 2 true true true true true @@ -329,7 +329,7 @@ struct +struct -- !query 35 output 1 false 3 false @@ -339,7 +339,7 @@ struct -- !query 36 SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL -- !query 36 schema -struct +struct -- !query 36 output 4 NULL @@ -380,7 +380,7 @@ SELECT every(1) struct<> -- !query 39 output org.apache.spark.sql.AnalysisException -cannot resolve 'bool_and(1)' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [int].; line 1 pos 7 +cannot resolve 'every(1)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 -- !query 40 @@ -389,7 +389,7 @@ SELECT some(1S) struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve 'bool_or(1S)' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [smallint].; line 1 pos 7 +cannot resolve 'some(1S)' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 @@ -398,7 +398,7 @@ SELECT any(1L) struct<> -- !query 41 output org.apache.spark.sql.AnalysisException -cannot resolve 'bool_or(1L)' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [bigint].; line 1 pos 7 +cannot resolve 'any(1L)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 -- !query 42 @@ -407,7 +407,7 @@ SELECT every("true") struct<> -- !query 42 output org.apache.spark.sql.AnalysisException -cannot resolve 'bool_and('true')' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [string].; line 1 pos 7 +cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7 -- !query 43 @@ -431,7 +431,7 @@ cannot resolve 'bool_or(1.0D)' due to data type mismatch: Input to function 'boo -- !query 45 SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg -- !query 45 schema -struct +struct -- !query 45 output 1 false false 1 true false @@ -448,7 +448,7 @@ struct +struct -- !query 46 output 1 false false 1 true true @@ -465,7 +465,7 @@ struct +struct -- !query 47 output 1 false false 1 true true diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index ea2cab703eaae..bdacd184158a5 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -293,7 +293,7 @@ struct<> -- !query 31 SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output NULL NULL NULL @@ -301,7 +301,7 @@ NULL NULL NULL -- !query 32 SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output NULL NULL NULL @@ -309,7 +309,7 @@ NULL NULL NULL -- !query 33 SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output false true true @@ -317,7 +317,7 @@ false true true -- !query 34 SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) -- !query 34 schema -struct +struct -- !query 34 output 1 false true true 2 true true true @@ -329,7 +329,7 @@ struct +struct -- !query 35 output 1 false 3 false @@ -339,7 +339,7 @@ struct -- !query 36 SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL -- !query 36 schema -struct +struct -- !query 36 output 4 NULL @@ -380,7 +380,7 @@ SELECT every(udf(1)) struct<> -- !query 39 output org.apache.spark.sql.AnalysisException -cannot resolve 'bool_and(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [int].; line 1 pos 7 +cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 -- !query 40 @@ -389,7 +389,7 @@ SELECT some(udf(1S)) struct<> -- !query 40 output org.apache.spark.sql.AnalysisException -cannot resolve 'bool_or(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [smallint].; line 1 pos 7 +cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 -- !query 41 @@ -398,7 +398,7 @@ SELECT any(udf(1L)) struct<> -- !query 41 output org.apache.spark.sql.AnalysisException -cannot resolve 'bool_or(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [bigint].; line 1 pos 7 +cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 -- !query 42 @@ -407,13 +407,13 @@ SELECT udf(every("true")) struct<> -- !query 42 output org.apache.spark.sql.AnalysisException -cannot resolve 'bool_and('true')' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [string].; line 1 pos 11 +cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11 -- !query 43 SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg -- !query 43 schema -struct +struct -- !query 43 output 1 false false 1 true false @@ -430,7 +430,7 @@ struct +struct -- !query 44 output 1 false false 1 true true @@ -447,7 +447,7 @@ struct +struct -- !query 45 output 1 false false 1 true true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index d5c16a30ade5b..f968fbb27d4f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -95,8 +95,8 @@ class ExplainSuite extends QueryTest with SharedSparkSession { // plan should show the rewritten aggregate expression. val df = sql("SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k") checkKeywordsExistsInExplain(df, - "Aggregate [k#x], [k#x, min(v#x) AS bool_and(v)#x, max(v#x) AS bool_or(v)#x, " + - "max(v#x) AS bool_or(v)#x]") + "Aggregate [k#x], [k#x, min(v#x) AS every(v)#x, max(v#x) AS some(v)#x, " + + "max(v#x) AS any(v)#x]") } } From c2f29d5ea58eb4565cc5602937d6d0bb75558513 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Mon, 9 Dec 2019 19:43:32 +0900 Subject: [PATCH 0823/1387] [SPARK-30138][SQL] Separate configuration key of max iterations for analyzer and optimizer ### What changes were proposed in this pull request? separate the configuration keys "spark.sql.optimizer.maxIterations" and "spark.sql.analyzer.maxIterations". ### Why are the changes needed? Currently, both Analyzer and Optimizer use conf "spark.sql.optimizer.maxIterations" to set the max iterations to run, which is a little confusing. It is clearer to add a new conf "spark.sql.analyzer.maxIterations" for analyzer max iterations. ### Does this PR introduce any user-facing change? no ### How was this patch tested? Existing unit tests. Closes #26766 from fuwhu/SPARK-30138. Authored-by: fuwhu Signed-off-by: Takeshi Yamamuro --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 ++-- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 10 +++++++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 659e4a5c86ec1..f15476cb3a589 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -134,11 +134,11 @@ class Analyzer( this( new CatalogManager(conf, FakeV2SessionCatalog, catalog), conf, - conf.optimizerMaxIterations) + conf.analyzerMaxIterations) } def this(catalogManager: CatalogManager, conf: SQLConf) = { - this(catalogManager, conf, conf.optimizerMaxIterations) + this(catalogManager, conf, conf.analyzerMaxIterations) } def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f535792489fb4..1ff23f697c4db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -170,6 +170,12 @@ object SQLConf { } } + val ANALYZER_MAX_ITERATIONS = buildConf("spark.sql.analyzer.maxIterations") + .internal() + .doc("The max number of iterations the analyzer runs.") + .intConf + .createWithDefault(100) + val OPTIMIZER_EXCLUDED_RULES = buildConf("spark.sql.optimizer.excludedRules") .doc("Configures a list of rules to be disabled in the optimizer, in which the rules are " + "specified by their rule names and separated by comma. It is not guaranteed that all the " + @@ -180,7 +186,7 @@ object SQLConf { val OPTIMIZER_MAX_ITERATIONS = buildConf("spark.sql.optimizer.maxIterations") .internal() - .doc("The max number of iterations the optimizer and analyzer runs.") + .doc("The max number of iterations the optimizer runs.") .intConf .createWithDefault(100) @@ -2126,6 +2132,8 @@ class SQLConf extends Serializable with Logging { /** ************************ Spark SQL Params/Hints ******************* */ + def analyzerMaxIterations: Int = getConf(ANALYZER_MAX_ITERATIONS) + def optimizerExcludedRules: Option[String] = getConf(OPTIMIZER_EXCLUDED_RULES) def optimizerMaxIterations: Int = getConf(OPTIMIZER_MAX_ITERATIONS) From a717d219a66d0e7b18b8ff392e1e03cd2781c457 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Mon, 9 Dec 2019 22:19:08 +0900 Subject: [PATCH 0824/1387] [SPARK-30159][SQL][TESTS] Fix the method calls of `QueryTest.checkAnswer` ### What changes were proposed in this pull request? Before this PR, the method `checkAnswer` in Object `QueryTest` returns an optional string. It doesn't throw exceptions when errors happen. The actual exceptions are thrown in the trait `QueryTest`. However, there are some test suites(`StreamSuite`, `SessionStateSuite`, `BinaryFileFormatSuite`, etc.) that use the no-op method `QueryTest.checkAnswer` and expect it to fail test cases when the execution results don't match the expected answers. After this PR: 1. the method `checkAnswer` in Object `QueryTest` will fail tests on errors or unexpected results. 2. add a new method `getErrorMessageInCheckAnswer`, which is exactly the same as the previous version of `checkAnswer`. There are some test suites use this one to customize the test failure message. 3. for the test suites that extend the trait `QueryTest`, we should use the method `checkAnswer` directly, instead of calling the method from Object `QueryTest`. ### Why are the changes needed? We should fix these method calls to perform actual validations in test suites. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing unit tests. Closes #26788 from gengliangwang/fixCheckAnswer. Authored-by: Gengliang Wang Signed-off-by: HyukjinKwon --- .../sql/avro/JavaAvroFunctionsSuite.java | 9 +---- .../apache/spark/sql/JavaSaveLoadSuite.java | 5 +-- .../org/apache/spark/sql/QueryTest.scala | 40 +++++++++++++------ .../ReduceNumShufflePartitionsSuite.scala | 25 +++++------- .../binaryfile/BinaryFileFormatSuite.scala | 6 +-- .../spark/sql/streaming/StreamSuite.scala | 6 +-- .../spark/sql/hive/JavaDataFrameSuite.java | 5 +-- .../hive/JavaMetastoreDataSourcesSuite.java | 9 +---- .../execution/AggregationQuerySuite.scala | 2 +- 9 files changed, 47 insertions(+), 60 deletions(-) diff --git a/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java b/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java index a448583dddfb7..cf4bba0f7f312 100644 --- a/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java +++ b/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java @@ -44,13 +44,6 @@ public void tearDown() { spark.stop(); } - private static void checkAnswer(Dataset actual, Dataset expected) { - String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected.collectAsList()); - if (errorMessage != null) { - Assert.fail(errorMessage); - } - } - @Test public void testToAvroFromAvro() { Dataset rangeDf = spark.range(10); @@ -69,6 +62,6 @@ public void testToAvroFromAvro() { from_avro(avroDF.col("a"), avroTypeLong), from_avro(avroDF.col("b"), avroTypeStr)); - checkAnswer(actual, df); + QueryTest$.MODULE$.checkAnswer(actual, df.collectAsList()); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java index 127d272579a62..875cb913ed7cd 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java @@ -43,10 +43,7 @@ public class JavaSaveLoadSuite { Dataset df; private static void checkAnswer(Dataset actual, List expected) { - String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected); - if (errorMessage != null) { - Assert.fail(errorMessage); - } + QueryTest$.MODULE$.checkAnswer(actual, expected); } @Before diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 3039a4ccb677c..4a21ae9242039 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -21,6 +21,9 @@ import java.util.{Locale, TimeZone} import scala.collection.JavaConverters._ +import org.junit.Assert +import org.scalatest.Assertions + import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.SQLExecution @@ -150,10 +153,7 @@ abstract class QueryTest extends PlanTest { assertEmptyMissingInput(analyzedDF) - QueryTest.checkAnswer(analyzedDF, expectedAnswer) match { - case Some(errorMessage) => fail(errorMessage) - case None => - } + QueryTest.checkAnswer(analyzedDF, expectedAnswer) } protected def checkAnswer(df: => DataFrame, expectedAnswer: Row): Unit = { @@ -235,18 +235,32 @@ abstract class QueryTest extends PlanTest { } } -object QueryTest { +object QueryTest extends Assertions { + /** + * Runs the plan and makes sure the answer matches the expected result. + * + * @param df the DataFrame to be executed + * @param expectedAnswer the expected result in a Seq of Rows. + * @param checkToRDD whether to verify deserialization to an RDD. This runs the query twice. + */ + def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row], checkToRDD: Boolean = true): Unit = { + getErrorMessageInCheckAnswer(df, expectedAnswer, checkToRDD) match { + case Some(errorMessage) => fail(errorMessage) + case None => + } + } + /** * Runs the plan and makes sure the answer matches the expected result. * If there was exception during the execution or the contents of the DataFrame does not - * match the expected result, an error message will be returned. Otherwise, a [[None]] will + * match the expected result, an error message will be returned. Otherwise, a None will * be returned. * - * @param df the [[DataFrame]] to be executed - * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param df the DataFrame to be executed + * @param expectedAnswer the expected result in a Seq of Rows. * @param checkToRDD whether to verify deserialization to an RDD. This runs the query twice. */ - def checkAnswer( + def getErrorMessageInCheckAnswer( df: DataFrame, expectedAnswer: Seq[Row], checkToRDD: Boolean = true): Option[String] = { @@ -408,10 +422,10 @@ object QueryTest { } } - def checkAnswer(df: DataFrame, expectedAnswer: java.util.List[Row]): String = { - checkAnswer(df, expectedAnswer.asScala) match { - case Some(errorMessage) => errorMessage - case None => null + def checkAnswer(df: DataFrame, expectedAnswer: java.util.List[Row]): Unit = { + getErrorMessageInCheckAnswer(df, expectedAnswer.asScala) match { + case Some(errorMessage) => Assert.fail(errorMessage) + case None => } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 21ec1ac9bda08..fe07b1ff109b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -258,13 +258,6 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val numInputPartitions: Int = 10 - def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = { - QueryTest.checkAnswer(actual, expectedAnswer) match { - case Some(errorMessage) => fail(errorMessage) - case None => - } - } - def withSparkSession( f: SparkSession => Unit, targetPostShuffleInputSize: Int, @@ -309,7 +302,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val agg = df.groupBy("key").count() // Check the answer first. - checkAnswer( + QueryTest.checkAnswer( agg, spark.range(0, 20).selectExpr("id", "50 as cnt").collect()) @@ -356,7 +349,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA .range(0, 1000) .selectExpr("id % 500 as key", "id as value") .union(spark.range(0, 1000).selectExpr("id % 500 as key", "id as value")) - checkAnswer( + QueryTest.checkAnswer( join, expectedAnswer.collect()) @@ -408,7 +401,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA spark .range(0, 500) .selectExpr("id", "2 as cnt") - checkAnswer( + QueryTest.checkAnswer( join, expectedAnswer.collect()) @@ -460,7 +453,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA spark .range(0, 1000) .selectExpr("id % 500 as key", "2 as cnt", "id as value") - checkAnswer( + QueryTest.checkAnswer( join, expectedAnswer.collect()) @@ -504,7 +497,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // Check the answer first. val expectedAnswer = spark.range(0, 500).selectExpr("id % 500", "id as value") .union(spark.range(500, 1000).selectExpr("id % 500", "id as value")) - checkAnswer( + QueryTest.checkAnswer( join, expectedAnswer.collect()) @@ -534,7 +527,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA // ReusedQueryStage 0 // ReusedQueryStage 0 val resultDf = df.join(df, "key").join(df, "key") - checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) + QueryTest.checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) val finalPlan = resultDf.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan assert(finalPlan.collect { case p: ReusedQueryStageExec => p }.length == 2) @@ -550,7 +543,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val grouped = df.groupBy("key").agg(max("value").as("value")) val resultDf2 = grouped.groupBy(col("key") + 1).max("value") .union(grouped.groupBy(col("key") + 2).max("value")) - checkAnswer(resultDf2, Row(1, 0) :: Row(2, 0) :: Nil) + QueryTest.checkAnswer(resultDf2, Row(1, 0) :: Row(2, 0) :: Nil) val finalPlan2 = resultDf2.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -580,7 +573,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val ds = spark.range(3) val resultDf = ds.repartition(2, ds.col("id")).toDF() - checkAnswer(resultDf, + QueryTest.checkAnswer(resultDf, Seq(0, 1, 2).map(i => Row(i))) val finalPlan = resultDf.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -596,7 +589,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA val resultDf = df1.union(df2) - checkAnswer(resultDf, Seq((0), (1), (2), (3)).map(i => Row(i))) + QueryTest.checkAnswer(resultDf, Seq((0), (1), (2), (3)).map(i => Row(i))) val finalPlan = resultDf.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala index 70ec9bbf4819d..2cd142f913072 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala @@ -352,15 +352,15 @@ class BinaryFileFormatSuite extends QueryTest with SharedSparkSession { .select(CONTENT) } val expected = Seq(Row(content)) - QueryTest.checkAnswer(readContent(), expected) + checkAnswer(readContent(), expected) withSQLConf(SOURCES_BINARY_FILE_MAX_LENGTH.key -> content.length.toString) { - QueryTest.checkAnswer(readContent(), expected) + checkAnswer(readContent(), expected) } // Disable read. If the implementation attempts to read, the exception would be different. file.setReadable(false) val caught = intercept[SparkException] { withSQLConf(SOURCES_BINARY_FILE_MAX_LENGTH.key -> (content.length - 1).toString) { - QueryTest.checkAnswer(readContent(), expected) + checkAnswer(readContent(), expected) } } assert(caught.getMessage.contains("exceeds the max length allowed")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 75ad041ccb801..a637b42c6b036 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -755,9 +755,9 @@ class StreamSuite extends StreamTest { inputData.addData(9) streamingQuery.processAllAvailable() - QueryTest.checkAnswer(spark.table("counts").toDF(), - Row("1", 1) :: Row("2", 1) :: Row("3", 2) :: Row("4", 2) :: - Row("5", 2) :: Row("6", 2) :: Row("7", 1) :: Row("8", 1) :: Row("9", 1) :: Nil) + checkAnswer(spark.table("counts").toDF(), + Row(1, 1L) :: Row(2, 1L) :: Row(3, 2L) :: Row(4, 2L) :: + Row(5, 2L) :: Row(6, 2L) :: Row(7, 1L) :: Row(8, 1L) :: Row(9, 1L) :: Nil) } finally { if (streamingQuery ne null) { streamingQuery.stop() diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java index 636ce10da3734..ffbccc90b45a0 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java @@ -39,10 +39,7 @@ public class JavaDataFrameSuite { Dataset df; private static void checkAnswer(Dataset actual, List expected) { - String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected); - if (errorMessage != null) { - Assert.fail(errorMessage); - } + QueryTest$.MODULE$.checkAnswer(actual, expected); } @Before diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index 5304052b45a41..ba58b92ddb027 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -51,13 +51,6 @@ public class JavaMetastoreDataSourcesSuite { FileSystem fs; Dataset df; - private static void checkAnswer(Dataset actual, List expected) { - String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected); - if (errorMessage != null) { - Assert.fail(errorMessage); - } - } - @Before public void setUp() throws IOException { sqlContext = TestHive$.MODULE$; @@ -100,7 +93,7 @@ public void saveTableAndQueryIt() { .options(options) .saveAsTable("javaSavedTable"); - checkAnswer( + QueryTest$.MODULE$.checkAnswer( sqlContext.sql("SELECT * FROM javaSavedTable"), df.collectAsList()); } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 4a3277f5a7e49..f84b854048e8a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -1063,7 +1063,7 @@ class HashAggregationQueryWithControlledFallbackSuite extends AggregationQuerySu // todo: remove it? val newActual = Dataset.ofRows(spark, actual.logicalPlan) - QueryTest.checkAnswer(newActual, expectedAnswer) match { + QueryTest.getErrorMessageInCheckAnswer(newActual, expectedAnswer) match { case Some(errorMessage) => val newErrorMessage = s""" From 729f43f499f3dd2718c0b28d73f2ca29cc811eac Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Mon, 9 Dec 2019 08:55:30 -0600 Subject: [PATCH 0825/1387] [SPARK-27189][CORE] Add Executor metrics and memory usage instrumentation to the metrics system ## What changes were proposed in this pull request? This PR proposes to add instrumentation of memory usage via the Spark Dropwizard/Codahale metrics system. Memory usage metrics are available via the Executor metrics, recently implemented as detailed in https://issues.apache.org/jira/browse/SPARK-23206. Additional notes: This takes advantage of the metrics poller introduced in #23767. ## Why are the changes needed? Executor metrics bring have many useful insights on memory usage, in particular on the usage of storage memory and executor memory. This is useful for troubleshooting. Having the information in the metrics systems allows to add those metrics to Spark performance dashboards and study memory usage as a function of time, as in the example graph https://issues.apache.org/jira/secure/attachment/12962810/Example_dashboard_Spark_Memory_Metrics.PNG ## Does this PR introduce any user-facing change? Adds `ExecutorMetrics` source to publish executor metrics via the Dropwizard metrics system. Details of the available metrics in docs/monitoring.md Adds configuration parameter `spark.metrics.executormetrics.source.enabled` ## How was this patch tested? Tested on YARN cluster and with an existing setup for a Spark dashboard based on InfluxDB and Grafana. Closes #24132 from LucaCanali/memoryMetricsSource. Authored-by: Luca Canali Signed-off-by: Imran Rashid --- .../scala/org/apache/spark/SparkContext.scala | 16 ++++- .../org/apache/spark/executor/Executor.scala | 11 +++- .../executor/ExecutorMetricsPoller.scala | 4 +- .../executor/ExecutorMetricsSource.scala | 65 +++++++++++++++++++ .../spark/internal/config/package.scala | 6 ++ .../metrics/source/SourceConfigSuite.scala | 30 ++++++++- docs/monitoring.md | 41 ++++++++++++ 7 files changed, 167 insertions(+), 6 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 069450120d5a7..96ca12b5f6c60 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -42,7 +42,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.deploy.StandaloneResourceUtils._ -import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.executor.{ExecutorMetrics, ExecutorMetricsSource} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -551,9 +551,16 @@ class SparkContext(config: SparkConf) extends Logging { _dagScheduler = new DAGScheduler(this) _heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet) + val _executorMetricsSource = + if (_conf.get(METRICS_EXECUTORMETRICS_SOURCE_ENABLED)) { + Some(new ExecutorMetricsSource) + } else { + None + } + // create and start the heartbeater for collecting memory metrics _heartbeater = new Heartbeater( - () => SparkContext.this.reportHeartBeat(), + () => SparkContext.this.reportHeartBeat(_executorMetricsSource), "driver-heartbeater", conf.get(EXECUTOR_HEARTBEAT_INTERVAL)) _heartbeater.start() @@ -622,6 +629,7 @@ class SparkContext(config: SparkConf) extends Logging { _env.metricsSystem.registerSource(_dagScheduler.metricsSource) _env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager)) _env.metricsSystem.registerSource(new JVMCPUSource()) + _executorMetricsSource.foreach(_.register(_env.metricsSystem)) _executorAllocationManager.foreach { e => _env.metricsSystem.registerSource(e.executorAllocationManagerSource) } @@ -2473,8 +2481,10 @@ class SparkContext(config: SparkConf) extends Logging { } /** Reports heartbeat metrics for the driver. */ - private def reportHeartBeat(): Unit = { + private def reportHeartBeat(executorMetricsSource: Option[ExecutorMetricsSource]): Unit = { val currentMetrics = ExecutorMetrics.getCurrentMetrics(env.memoryManager) + executorMetricsSource.foreach(_.updateMetricsSnapshot(currentMetrics)) + val driverUpdates = new HashMap[(Int, Int), ExecutorMetrics] // In the driver, we do not track per-stage metrics, so use a dummy stage for the key driverUpdates.put(EventLoggingListener.DRIVER_STAGE_KEY, new ExecutorMetrics(currentMetrics)) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 8cd98e47b8a4f..6eb1525a0db5a 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -113,10 +113,18 @@ private[spark] class Executor( // create. The map key is a task id. private val taskReaperForTask: HashMap[Long, TaskReaper] = HashMap[Long, TaskReaper]() + val executorMetricsSource = + if (conf.get(METRICS_EXECUTORMETRICS_SOURCE_ENABLED)) { + Some(new ExecutorMetricsSource) + } else { + None + } + if (!isLocal) { env.blockManager.initialize(conf.getAppId) env.metricsSystem.registerSource(executorSource) env.metricsSystem.registerSource(new JVMCPUSource()) + executorMetricsSource.foreach(_.register(env.metricsSystem)) env.metricsSystem.registerSource(env.blockManager.shuffleMetricsSource) } @@ -181,7 +189,8 @@ private[spark] class Executor( // Poller for the memory metrics. Visible for testing. private[executor] val metricsPoller = new ExecutorMetricsPoller( env.memoryManager, - METRICS_POLLING_INTERVAL_MS) + METRICS_POLLING_INTERVAL_MS, + executorMetricsSource) // Executor for the heartbeat task. private val heartbeater = new Heartbeater( diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala index 805b0f729b122..1c1a1ca8035d0 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -48,7 +48,8 @@ import org.apache.spark.util.{ThreadUtils, Utils} */ private[spark] class ExecutorMetricsPoller( memoryManager: MemoryManager, - pollingInterval: Long) extends Logging { + pollingInterval: Long, + executorMetricsSource: Option[ExecutorMetricsSource]) extends Logging { type StageKey = (Int, Int) // Task Count and Metric Peaks @@ -79,6 +80,7 @@ private[spark] class ExecutorMetricsPoller( // get the latest values for the metrics val latestMetrics = ExecutorMetrics.getCurrentMetrics(memoryManager) + executorMetricsSource.foreach(_.updateMetricsSnapshot(latestMetrics)) def updatePeaks(metrics: AtomicLongArray): Unit = { (0 until metrics.length).foreach { i => diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala new file mode 100644 index 0000000000000..b052e43d4665b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} +import org.apache.spark.metrics.source.Source + +/** + * Expose executor metrics from [[ExecutorMetricsType]] using the Dropwizard metrics system. + * + * Metrics related to the memory system can be expensive to gather, therefore + * we implement some optimizations: + * (1) Metrics values are cached, updated at each heartbeat (default period is 10 seconds). + * An alternative faster polling mechanism is used, only if activated, by setting + * spark.executor.metrics.pollingInterval=. + * (2) Procfs metrics are gathered all in one-go and only conditionally: + * if the /proc filesystem exists + * and spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true + * and spark.eventLog.logStageExecutorMetrics.enabled=true. + */ +private[spark] class ExecutorMetricsSource extends Source { + + override val metricRegistry = new MetricRegistry() + override val sourceName = "ExecutorMetrics" + @volatile var metricsSnapshot: Array[Long] = Array.fill(ExecutorMetricType.numMetrics)(0L) + + // called by ExecutorMetricsPoller + def updateMetricsSnapshot(metricsUpdates: Array[Long]): Unit = { + metricsSnapshot = metricsUpdates + } + + private class ExecutorMetricGauge(idx: Int) extends Gauge[Long] { + def getValue: Long = metricsSnapshot(idx) + } + + def register(metricsSystem: MetricsSystem): Unit = { + val gauges: IndexedSeq[ExecutorMetricGauge] = (0 until ExecutorMetricType.numMetrics).map { + idx => new ExecutorMetricGauge(idx) + }.toIndexedSeq + + ExecutorMetricType.metricToOffset.foreach { + case (name, idx) => + metricRegistry.register(MetricRegistry.name(name), gauges(idx)) + } + + metricsSystem.registerSource(this) + } +} 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 039911f260955..25dc4c6efa26b 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 @@ -637,6 +637,12 @@ package object config { .stringConf .createOptional + private[spark] val METRICS_EXECUTORMETRICS_SOURCE_ENABLED = + ConfigBuilder("spark.metrics.executorMetricsSource.enabled") + .doc("Whether to register the ExecutorMetrics source with the metrics system.") + .booleanConf + .createWithDefault(true) + private[spark] val METRICS_STATIC_SOURCES_ENABLED = ConfigBuilder("spark.metrics.staticSources.enabled") .doc("Whether to register static sources with the metrics system.") diff --git a/core/src/test/scala/org/apache/spark/metrics/source/SourceConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/source/SourceConfigSuite.scala index 76c568056aee4..8f5ab7419d4f7 100644 --- a/core/src/test/scala/org/apache/spark/metrics/source/SourceConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/source/SourceConfigSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.metrics.source import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.internal.config.METRICS_STATIC_SOURCES_ENABLED +import org.apache.spark.internal.config.{METRICS_EXECUTORMETRICS_SOURCE_ENABLED, METRICS_STATIC_SOURCES_ENABLED} class SourceConfigSuite extends SparkFunSuite with LocalSparkContext { @@ -52,4 +52,32 @@ class SourceConfigSuite extends SparkFunSuite with LocalSparkContext { } } + test("Test configuration for adding ExecutorMetrics source registration") { + val conf = new SparkConf() + conf.set(METRICS_EXECUTORMETRICS_SOURCE_ENABLED, true) + val sc = new SparkContext("local", "test", conf) + try { + val metricsSystem = sc.env.metricsSystem + + // ExecutorMetrics source should be registered + assert (metricsSystem.getSourcesByName("ExecutorMetrics").nonEmpty) + } finally { + sc.stop() + } + } + + test("Test configuration for skipping ExecutorMetrics source registration") { + val conf = new SparkConf() + conf.set(METRICS_EXECUTORMETRICS_SOURCE_ENABLED, false) + val sc = new SparkContext("local", "test", conf) + try { + val metricsSystem = sc.env.metricsSystem + + // ExecutorMetrics source should not be registered + assert (metricsSystem.getSourcesByName("ExecutorMetrics").isEmpty) + } finally { + sc.stop() + } + } + } diff --git a/docs/monitoring.md b/docs/monitoring.md index 2aed452adb05f..cff1126481f94 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -995,6 +995,12 @@ This is the component with the largest amount of instrumented metrics - namespace=JVMCPU - jvmCpuTime +- namespace=ExecutorMetrics + - **note:** these metrics are conditional to a configuration parameter: + `spark.metrics.executorMetricsSource.enabled` (default is true) + - This source contains memory-related metrics. A full list of available metrics in this + namespace can be found in the corresponding entry for the Executor component instance. + - namespace=plugin.\ - Optional namespace(s). Metrics in this namespace are defined by user-supplied code, and configured using the Spark plugin API. See "Advanced Instrumentation" below for how to load @@ -1046,6 +1052,41 @@ when running in local mode. - threadpool.maxPool_size - threadpool.startedTasks +- namespace=ExecutorMetrics + - **notes:** + - These metrics are conditional to a configuration parameter: + `spark.metrics.executorMetricsSource.enabled` (default value is true) + - ExecutorMetrics are updated as part of heartbeat processes scheduled + for the executors and for the driver at regular intervals: `spark.executor.heartbeatInterval` (default value is 10 seconds) + - An optional faster polling mechanism is available for executor memory metrics, + it can be activated by setting a polling interval (in milliseconds) using the configuration parameter `spark.executor.metrics.pollingInterval` + - JVMHeapMemory + - JVMOffHeapMemory + - OnHeapExecutionMemory + - OnHeapStorageMemory + - OnHeapUnifiedMemory + - OffHeapExecutionMemory + - OffHeapStorageMemory + - OffHeapUnifiedMemory + - DirectPoolMemory + - MappedPoolMemory + - MinorGCCount + - MinorGCTime + - MajorGCCount + - MajorGCTime + - "ProcessTree*" metric counters: + - ProcessTreeJVMVMemory + - ProcessTreeJVMRSSMemory + - ProcessTreePythonVMemory + - ProcessTreePythonRSSMemory + - ProcessTreeOtherVMemory + - ProcessTreeOtherRSSMemory + - **note:** "ProcessTree*" metrics are collected only under certain conditions. + The conditions are the logical AND of the following: `/proc` filesystem exists, + `spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true`, + `spark.eventLog.logStageExecutorMetrics.enabled=true`. + "ProcessTree*" metrics report 0 when those conditions are not met. + - namespace=JVMCPU - jvmCpuTime From 538b8d101cf06b059288f013579dafaafa388bdc Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 9 Dec 2019 08:57:20 -0800 Subject: [PATCH 0826/1387] [SPARK-30159][SQL][FOLLOWUP] Fix lint-java via removing unnecessary imports ### What changes were proposed in this pull request? This patch fixes the Java code style violations in SPARK-30159 (#26788) which are caught by lint-java (Github Action caught it and I can reproduce it locally). Looks like Jenkins build may have different policy on checking Java style check or less accurate. ### Why are the changes needed? Java linter starts complaining. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? lint-java passed locally This closes #26819 Closes #26818 from HeartSaVioR/SPARK-30159-FOLLOWUP. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java | 1 - .../test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java | 1 - .../test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java | 1 - .../org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java | 1 - 4 files changed, 4 deletions(-) diff --git a/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java b/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java index cf4bba0f7f312..94188ecf57c86 100644 --- a/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java +++ b/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java @@ -18,7 +18,6 @@ package org.apache.spark.sql.avro; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java index 875cb913ed7cd..e2a69d55337bc 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java @@ -25,7 +25,6 @@ import java.util.Map; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java index ffbccc90b45a0..2b532389bafb6 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java @@ -22,7 +22,6 @@ import java.util.List; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index ba58b92ddb027..d433386a6c19a 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; From 8a9cccf1f3f4365e40f682bb111ec6c15cbc9be4 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 9 Dec 2019 13:39:33 -0600 Subject: [PATCH 0827/1387] [SPARK-30146][ML][PYSPARK] Add setWeightCol to GBTs in PySpark ### What changes were proposed in this pull request? add ```setWeightCol``` and ```setMinWeightFractionPerNode``` in Python side of ```GBTClassifier``` and ```GBTRegressor``` ### Why are the changes needed? https://github.com/apache/spark/pull/25926 added ```setWeightCol``` and ```setMinWeightFractionPerNode``` in GBTs on scala side. This PR will add ```setWeightCol``` and ```setMinWeightFractionPerNode``` in GBTs on python side ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? doc test Closes #26774 from huaxingao/spark-30146. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- python/pyspark/ml/classification.py | 27 +++++++++++++++++++++++---- python/pyspark/ml/regression.py | 28 ++++++++++++++++++++++++---- 2 files changed, 47 insertions(+), 8 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 8e2b2396be14d..a615ca2cde94a 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1617,6 +1617,8 @@ class GBTClassifier(JavaProbabilisticClassifier, _GBTClassifierParams, ... leafCol="leafId") >>> gbt.setMaxIter(5) GBTClassifier... + >>> gbt.setMinWeightFractionPerNode(0.049) + GBTClassifier... >>> gbt.getMaxIter() 5 >>> gbt.getFeatureSubsetStrategy() @@ -1684,14 +1686,15 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, lossType="logistic", maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, impurity="variance", featureSubsetStrategy="all", validationTol=0.01, validationIndicatorCol=None, - leafCol="", minWeightFractionPerNode=0.0): + leafCol="", minWeightFractionPerNode=0.0, weightCol=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ lossType="logistic", maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, \ impurity="variance", featureSubsetStrategy="all", validationTol=0.01, \ - validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0) + validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0, \ + weightCol=None) """ super(GBTClassifier, self).__init__() self._java_obj = self._new_java_obj( @@ -1711,14 +1714,16 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, lossType="logistic", maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, impurity="variance", featureSubsetStrategy="all", validationTol=0.01, - validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0): + validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0, + weightCol=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ lossType="logistic", maxIter=20, stepSize=0.1, seed=None, subsamplingRate=1.0, \ impurity="variance", featureSubsetStrategy="all", validationTol=0.01, \ - validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0) + validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0, \ + weightCol=None) Sets params for Gradient Boosted Tree Classification. """ kwargs = self._input_kwargs @@ -1826,6 +1831,20 @@ def setStepSize(self, value): """ return self._set(stepSize=value) + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + @since("3.0.0") + def setMinWeightFractionPerNode(self, value): + """ + Sets the value of :py:attr:`minWeightFractionPerNode`. + """ + return self._set(minWeightFractionPerNode=value) + class GBTClassificationModel(_TreeEnsembleModel, JavaProbabilisticClassificationModel, _GBTClassifierParams, JavaMLWritable, JavaMLReadable): diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index fdb04bb5115c5..0ea906d2688a1 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1226,6 +1226,8 @@ class GBTRegressor(JavaPredictor, _GBTRegressorParams, JavaMLWritable, JavaMLRea >>> gbt = GBTRegressor(maxDepth=2, seed=42, leafCol="leafId") >>> gbt.setMaxIter(5) GBTRegressor... + >>> gbt.setMinWeightFractionPerNode(0.049) + GBTRegressor... >>> gbt.getMaxIter() 5 >>> print(gbt.getImpurity()) @@ -1285,14 +1287,16 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, seed=None, impurity="variance", featureSubsetStrategy="all", validationTol=0.01, - validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0): + validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0, + weightCol=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, \ checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, seed=None, \ impurity="variance", featureSubsetStrategy="all", validationTol=0.01, \ - validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0) + validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0, + weightCol=None) """ super(GBTRegressor, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.regression.GBTRegressor", self.uid) @@ -1311,14 +1315,16 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, seed=None, impuriy="variance", featureSubsetStrategy="all", validationTol=0.01, - validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0): + validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0, + weightCol=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, subsamplingRate=1.0, \ checkpointInterval=10, lossType="squared", maxIter=20, stepSize=0.1, seed=None, \ impurity="variance", featureSubsetStrategy="all", validationTol=0.01, \ - validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0) + validationIndicatorCol=None, leafCol="", minWeightFractionPerNode=0.0, \ + weightCol=None) Sets params for Gradient Boosted Tree Regression. """ kwargs = self._input_kwargs @@ -1432,6 +1438,20 @@ def setStepSize(self, value): """ return self._set(stepSize=value) + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + @since("3.0.0") + def setMinWeightFractionPerNode(self, value): + """ + Sets the value of :py:attr:`minWeightFractionPerNode`. + """ + return self._set(minWeightFractionPerNode=value) + class GBTRegressionModel(_TreeEnsembleModel, _GBTRegressorParams, JavaMLWritable, JavaMLReadable): """ From 36fa1980c24c5c697982b107c8f9714f3eb57f36 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 9 Dec 2019 14:41:48 -0600 Subject: [PATCH 0828/1387] [SPARK-30158][SQL][CORE] Seq -> Array for sc.parallelize for 2.13 compatibility; remove WrappedArray ### What changes were proposed in this pull request? Use Seq instead of Array in sc.parallelize, with reference types. Remove usage of WrappedArray. ### Why are the changes needed? These both enable building on Scala 2.13. ### Does this PR introduce any user-facing change? None ### How was this patch tested? Existing tests Closes #26787 from srowen/SPARK-30158. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../mllib/ElementwiseProductExample.scala | 2 +- .../spark/mllib/pmml/PMMLExportable.scala | 2 +- .../ml/clustering/BisectingKMeansSuite.scala | 4 +- .../spark/ml/clustering/KMeansSuite.scala | 4 +- .../spark/ml/recommendation/ALSSuite.scala | 4 +- .../clustering/GaussianMixtureSuite.scala | 4 +- .../spark/mllib/clustering/KMeansSuite.scala | 4 +- .../spark/mllib/clustering/LDASuite.scala | 4 +- .../apache/spark/mllib/feature/PCASuite.scala | 2 +- .../sql/JavaHigherOrderFunctionsSuite.java | 73 ++++++++++++++----- .../org/apache/spark/sql/JavaTestUtils.java | 47 ------------ .../ParquetPartitionDiscoverySuite.scala | 16 ++-- .../parquet/ParquetQuerySuite.scala | 2 +- 13 files changed, 78 insertions(+), 90 deletions(-) delete mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaTestUtils.java diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala index 1e4e3543194e2..1855058bb4f3c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala @@ -34,7 +34,7 @@ object ElementwiseProductExample { // $example on$ // Create some vector data; also works for sparse vectors - val data = sc.parallelize(Array(Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(4.0, 5.0, 6.0))) + val data = sc.parallelize(Seq(Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(4.0, 5.0, 6.0))) val transformingVector = Vectors.dense(0.0, 1.0, 2.0) val transformer = new ElementwiseProduct(transformingVector) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index 5d61796f1de60..340386c7e7bec 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -58,7 +58,7 @@ trait PMMLExportable { @Since("1.4.0") def toPMML(sc: SparkContext, path: String): Unit = { val pmml = toPMML() - sc.parallelize(Array(pmml), 1).saveAsTextFile(path) + sc.parallelize(Seq(pmml), 1).saveAsTextFile(path) } /** diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index 7ac7b64adfdab..9984451b08ce1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -149,7 +149,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { test("BisectingKMeans with cosine distance is not supported for 0-length vectors") { val model = new BisectingKMeans().setK(2).setDistanceMeasure(DistanceMeasure.COSINE).setSeed(1) - val df = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df = spark.createDataFrame(spark.sparkContext.parallelize(Seq( Vectors.dense(0.0, 0.0), Vectors.dense(10.0, 10.0), Vectors.dense(1.0, 0.5) @@ -160,7 +160,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { } test("BisectingKMeans with cosine distance") { - val df = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df = spark.createDataFrame(spark.sparkContext.parallelize(Seq( Vectors.dense(1.0, 1.0), Vectors.dense(10.0, 10.0), Vectors.dense(1.0, 0.5), diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index f6b1a8e9d6df3..173305269c63f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -158,7 +158,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes } test("KMeans using cosine distance") { - val df = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df = spark.createDataFrame(spark.sparkContext.parallelize(Seq( Vectors.dense(1.0, 1.0), Vectors.dense(10.0, 10.0), Vectors.dense(1.0, 0.5), @@ -191,7 +191,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes test("KMeans with cosine distance is not supported for 0-length vectors") { val model = new KMeans().setDistanceMeasure(DistanceMeasure.COSINE).setK(2) - val df = spark.createDataFrame(spark.sparkContext.parallelize(Array( + val df = spark.createDataFrame(spark.sparkContext.parallelize(Seq( Vectors.dense(0.0, 0.0), Vectors.dense(10.0, 10.0), Vectors.dense(1.0, 0.5) diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index aeb5e41eca085..b7bb127adb94a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -488,8 +488,8 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { } test("implicit feedback regression") { - val trainingWithNeg = sc.parallelize(Array(Rating(0, 0, 1), Rating(1, 1, 1), Rating(0, 1, -3))) - val trainingWithZero = sc.parallelize(Array(Rating(0, 0, 1), Rating(1, 1, 1), Rating(0, 1, 0))) + val trainingWithNeg = sc.parallelize(Seq(Rating(0, 0, 1), Rating(1, 1, 1), Rating(0, 1, -3))) + val trainingWithZero = sc.parallelize(Seq(Rating(0, 0, 1), Rating(1, 1, 1), Rating(0, 1, 0))) val modelWithNeg = trainALS(trainingWithNeg, rank = 1, maxIter = 5, regParam = 0.01, implicitPrefs = true) val modelWithZero = diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala index 11189d8bd4776..54ed30799e7b9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala @@ -40,7 +40,7 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext { } test("single cluster") { - val data = sc.parallelize(Array( + val data = sc.parallelize(Seq( Vectors.dense(6.0, 9.0), Vectors.dense(5.0, 10.0), Vectors.dense(4.0, 11.0) @@ -105,7 +105,7 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext { } test("single cluster with sparse data") { - val data = sc.parallelize(Array( + val data = sc.parallelize(Seq( Vectors.sparse(3, Array(0, 2), Array(4.0, 2.0)), Vectors.sparse(3, Array(0, 2), Array(2.0, 4.0)), Vectors.sparse(3, Array(1), Array(6.0)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 149a525a58ff6..e63ca70961858 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -34,7 +34,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { private val seed = 42 test("single cluster") { - val data = sc.parallelize(Array( + val data = sc.parallelize(Seq( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0) @@ -64,7 +64,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { test("fewer distinct points than clusters") { val data = sc.parallelize( - Array( + Seq( Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(1.0, 2.0, 3.0)), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 321df05e272db..56d41403f74cc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -279,10 +279,10 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { test("LocalLDAModel logLikelihood") { val ldaModel: LocalLDAModel = toyModel - val docsSingleWord = sc.parallelize(Array(Vectors.sparse(6, Array(0), Array(1))) + val docsSingleWord = sc.parallelize(Seq(Vectors.sparse(6, Array(0), Array(1))) .zipWithIndex .map { case (wordCounts, docId) => (docId.toLong, wordCounts) }) - val docsRepeatedWord = sc.parallelize(Array(Vectors.sparse(6, Array(0), Array(5))) + val docsRepeatedWord = sc.parallelize(Seq(Vectors.sparse(6, Array(0), Array(5))) .zipWithIndex .map { case (wordCounts, docId) => (docId.toLong, wordCounts) }) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala index e478f14906ba8..d0c8de0e75d53 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala @@ -56,7 +56,7 @@ class PCASuite extends SparkFunSuite with MLlibTestSparkContext { } test("number of features more than 65535") { - val data1 = sc.parallelize(Array( + val data1 = sc.parallelize(Seq( Vectors.dense(Array.fill(100000)(2.0)), Vectors.dense(Array.fill(100000)(0.0)) ), 2) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaHigherOrderFunctionsSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaHigherOrderFunctionsSuite.java index e240326bee63a..de0acc295b5ea 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaHigherOrderFunctionsSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaHigherOrderFunctionsSuite.java @@ -17,33 +17,68 @@ package test.org.apache.spark.sql; +import java.util.Arrays; import java.util.HashMap; import java.util.List; +import static java.util.stream.Collectors.toList; import static scala.collection.JavaConverters.mapAsScalaMap; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.types.*; -import static org.apache.spark.sql.types.DataTypes.*; +import org.apache.spark.sql.RowFactory; import static org.apache.spark.sql.functions.*; import org.apache.spark.sql.test.TestSparkSession; -import static test.org.apache.spark.sql.JavaTestUtils.*; +import org.apache.spark.sql.types.*; +import static org.apache.spark.sql.types.DataTypes.*; public class JavaHigherOrderFunctionsSuite { private transient TestSparkSession spark; private Dataset arrDf; private Dataset mapDf; + private void checkAnswer(Dataset actualDS, List expected) throws Exception { + List actual = actualDS.collectAsList(); + Assert.assertEquals(expected.size(), actual.size()); + for (int i = 0; i < expected.size(); i++) { + Row expectedRow = expected.get(i); + Row actualRow = actual.get(i); + Assert.assertEquals(expectedRow.size(), actualRow.size()); + for (int j = 0; j < expectedRow.size(); j++) { + Object expectedValue = expectedRow.get(j); + Object actualValue = actualRow.get(j); + if (expectedValue != null && expectedValue.getClass().isArray()) { + actualValue = actualValue.getClass().getMethod("array").invoke(actualValue); + Assert.assertArrayEquals((Object[]) expectedValue, (Object[]) actualValue); + } else { + Assert.assertEquals(expectedValue, actualValue); + } + } + } + } + + @SafeVarargs + private static List toRows(T... objs) { + return Arrays.stream(objs) + .map(RowFactory::create) + .collect(toList()); + } + + @SafeVarargs + private static T[] makeArray(T... ts) { + return ts; + } + private void setUpArrDf() { List data = toRows( makeArray(1, 9, 8, 7), makeArray(5, 8, 9, 7, 2), - JavaTestUtils.makeArray(), + JavaHigherOrderFunctionsSuite.makeArray(), null ); StructType schema = new StructType() @@ -78,13 +113,13 @@ public void tearDown() { } @Test - public void testTransform() { + public void testTransform() throws Exception { checkAnswer( arrDf.select(transform(col("x"), x -> x.plus(1))), toRows( makeArray(2, 10, 9, 8), makeArray(6, 9, 10, 8, 3), - JavaTestUtils.makeArray(), + JavaHigherOrderFunctionsSuite.makeArray(), null ) ); @@ -93,20 +128,20 @@ public void testTransform() { toRows( makeArray(1, 10, 10, 10), makeArray(5, 9, 11, 10, 6), - JavaTestUtils.makeArray(), + JavaHigherOrderFunctionsSuite.makeArray(), null ) ); } @Test - public void testFilter() { + public void testFilter() throws Exception { checkAnswer( arrDf.select(filter(col("x"), x -> x.plus(1).equalTo(10))), toRows( makeArray(9), makeArray(9), - JavaTestUtils.makeArray(), + JavaHigherOrderFunctionsSuite.makeArray(), null ) ); @@ -115,14 +150,14 @@ public void testFilter() { toRows( makeArray(9, 8, 7), makeArray(7), - JavaTestUtils.makeArray(), + JavaHigherOrderFunctionsSuite.makeArray(), null ) ); } @Test - public void testExists() { + public void testExists() throws Exception { checkAnswer( arrDf.select(exists(col("x"), x -> x.plus(1).equalTo(10))), toRows( @@ -135,7 +170,7 @@ public void testExists() { } @Test - public void testForall() { + public void testForall() throws Exception { checkAnswer( arrDf.select(forall(col("x"), x -> x.plus(1).equalTo(10))), toRows( @@ -148,7 +183,7 @@ public void testForall() { } @Test - public void testAggregate() { + public void testAggregate() throws Exception { checkAnswer( arrDf.select(aggregate(col("x"), lit(0), (acc, x) -> acc.plus(x))), toRows( @@ -170,20 +205,20 @@ public void testAggregate() { } @Test - public void testZipWith() { + public void testZipWith() throws Exception { checkAnswer( arrDf.select(zip_with(col("x"), col("x"), (a, b) -> lit(42))), toRows( makeArray(42, 42, 42, 42), makeArray(42, 42, 42, 42, 42), - JavaTestUtils.makeArray(), + JavaHigherOrderFunctionsSuite.makeArray(), null ) ); } @Test - public void testTransformKeys() { + public void testTransformKeys() throws Exception { checkAnswer( mapDf.select(transform_keys(col("x"), (k, v) -> k.plus(v))), toRows( @@ -197,7 +232,7 @@ public void testTransformKeys() { } @Test - public void testTransformValues() { + public void testTransformValues() throws Exception { checkAnswer( mapDf.select(transform_values(col("x"), (k, v) -> k.plus(v))), toRows( @@ -211,7 +246,7 @@ public void testTransformValues() { } @Test - public void testMapFilter() { + public void testMapFilter() throws Exception { checkAnswer( mapDf.select(map_filter(col("x"), (k, v) -> lit(false))), toRows( @@ -222,7 +257,7 @@ public void testMapFilter() { } @Test - public void testMapZipWith() { + public void testMapZipWith() throws Exception { checkAnswer( mapDf.select(map_zip_with(col("x"), col("x"), (k, v1, v2) -> lit(false))), toRows( diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaTestUtils.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaTestUtils.java deleted file mode 100644 index 7fc6460e7352c..0000000000000 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaTestUtils.java +++ /dev/null @@ -1,47 +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 test.org.apache.spark.sql; - -import java.util.Arrays; -import java.util.List; -import static java.util.stream.Collectors.toList; - -import scala.collection.mutable.WrappedArray; - -import static org.junit.Assert.assertEquals; - -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; - -public class JavaTestUtils { - public static void checkAnswer(Dataset actual, List expected) { - assertEquals(expected, actual.collectAsList()); - } - - public static List toRows(Object... objs) { - return Arrays.asList(objs) - .stream() - .map(RowFactory::create) - .collect(toList()); - } - - public static WrappedArray makeArray(T... ts) { - return WrappedArray.make(ts); - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 138336dc7e333..6497a1ceb5c0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -215,14 +215,14 @@ abstract class ParquetPartitionDiscoverySuite check("file://path/a=10", Some { PartitionValues( - ArrayBuffer("a"), - ArrayBuffer(Literal.create(10, IntegerType))) + Seq("a"), + Seq(Literal.create(10, IntegerType))) }) check("file://path/a=10/b=hello/c=1.5", Some { PartitionValues( - ArrayBuffer("a", "b", "c"), - ArrayBuffer( + Seq("a", "b", "c"), + Seq( Literal.create(10, IntegerType), Literal.create("hello", StringType), Literal.create(1.5, DoubleType))) @@ -230,8 +230,8 @@ abstract class ParquetPartitionDiscoverySuite check("file://path/a=10/b_hello/c=1.5", Some { PartitionValues( - ArrayBuffer("c"), - ArrayBuffer(Literal.create(1.5, DoubleType))) + Seq("c"), + Seq(Literal.create(1.5, DoubleType))) }) check("file:///", None) @@ -272,8 +272,8 @@ abstract class ParquetPartitionDiscoverySuite assert(partitionSpec2 == Option(PartitionValues( - ArrayBuffer("a"), - ArrayBuffer(Literal.create(10, IntegerType))))) + Seq("a"), + Seq(Literal.create(10, IntegerType))))) } test("parse partitions") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index f38973f7dffd1..61e93a7acd3f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -391,7 +391,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS withTempPath { dir => val basePath = dir.getCanonicalPath val schema = StructType(Array(StructField("name", DecimalType(10, 5), false))) - val rowRDD = sparkContext.parallelize(Array(Row(Decimal("67123.45")))) + val rowRDD = sparkContext.parallelize(Seq(Row(Decimal("67123.45")))) val df = spark.createDataFrame(rowRDD, schema) df.write.parquet(basePath) From 3d98c9f9854c6078d0784d3aa5cc1bb4b5e6a8e8 Mon Sep 17 00:00:00 2001 From: Luan Date: Tue, 10 Dec 2019 10:57:32 +0900 Subject: [PATCH 0829/1387] [SPARK-30179][SQL][TESTS] Improve test in SingleSessionSuite ### What changes were proposed in this pull request? improve the temporary functions test in SingleSessionSuite by verifying the result in a query ### Why are the changes needed? ### Does this PR introduce any user-facing change? ### How was this patch tested? Closes #26812 from leoluan2009/SPARK-30179. Authored-by: Luan Signed-off-by: HyukjinKwon --- .../hive/thriftserver/HiveThriftServer2Suites.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 108fec4ae035c..84eed7b2eda22 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -790,6 +790,8 @@ class SingleSessionSuite extends HiveThriftJdbcTest { Seq( "SET foo=bar", s"ADD JAR $jarURL", + "CREATE TABLE test_udtf(key INT, value STRING) USING hive", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_udtf", s"""CREATE TEMPORARY FUNCTION udtf_count2 |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' """.stripMargin @@ -816,6 +818,16 @@ class SingleSessionSuite extends HiveThriftJdbcTest { assert(rs2.next()) assert(rs2.getString(1) === "Usage: N/A.") + + val rs3 = statement.executeQuery( + "SELECT key, cc FROM test_udtf LATERAL VIEW udtf_count2(value) dd AS cc") + assert(rs3.next()) + assert(rs3.getInt(1) === 165) + assert(rs3.getInt(2) === 5) + + assert(rs3.next()) + assert(rs3.getInt(1) === 165) + assert(rs3.getInt(2) === 5) } finally { statement.executeQuery("DROP TEMPORARY FUNCTION udtf_count2") } From be867e8a9ee8fc5e4831521770f51793e9265550 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 10 Dec 2019 12:22:03 +0900 Subject: [PATCH 0830/1387] [SPARK-30196][BUILD] Bump lz4-java version to 1.7.0 ### What changes were proposed in this pull request? This pr intends to upgrade lz4-java from 1.6.0 to 1.7.0. ### Why are the changes needed? This release includes a performance bug (https://github.com/lz4/lz4-java/pull/143) fixed by JoshRosen and some improvements (e.g., LZ4 binary update). You can see the link below for the changes; https://github.com/lz4/lz4-java/blob/master/CHANGES.md#170 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #26823 from maropu/LZ4_1_7_0. Authored-by: Takeshi Yamamuro Signed-off-by: HyukjinKwon --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 2 +- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 5927709407f3a..5fe29374f83ea 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -145,7 +145,7 @@ libfb303-0.9.3.jar libthrift-0.12.0.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar -lz4-java-1.6.0.jar +lz4-java-1.7.0.jar machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 4b63fa0f4076c..4028718d8d17b 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -161,7 +161,7 @@ libfb303-0.9.3.jar libthrift-0.12.0.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar -lz4-java-1.6.0.jar +lz4-java-1.7.0.jar machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 979c6f9fd5ccd..244062c3ac239 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -173,7 +173,7 @@ libfb303-0.9.3.jar libthrift-0.12.0.jar log4j-1.2.17.jar logging-interceptor-3.12.0.jar -lz4-java-1.6.0.jar +lz4-java-1.7.0.jar machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar mesos-1.4.0-shaded-protobuf.jar diff --git a/pom.xml b/pom.xml index 617e0c36a93a3..9907e5a18d3ab 100644 --- a/pom.xml +++ b/pom.xml @@ -627,7 +627,7 @@ org.lz4 lz4-java - 1.6.0 + 1.7.0 com.github.luben From aa9da9365ff31948e42ab4c6dcc6cb4cec5fd852 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 10 Dec 2019 20:56:21 +0800 Subject: [PATCH 0831/1387] [SPARK-30151][SQL] Issue better error message when user-specified schema mismatched ### What changes were proposed in this pull request? Issue better error message when user-specified schema and not match relation schema ### Why are the changes needed? Inspired by https://github.com/apache/spark/pull/25248#issuecomment-559594305, user could get a weird error message when type mapping behavior change between Spark schema and datasource schema(e.g. JDBC). Instead of saying "SomeProvider does not allow user-specified schemas.", we'd better tell user what is really happening here to make user be more clearly about the error. ### Does this PR introduce any user-facing change? Yes, user will see error message changes. ### How was this patch tested? Updated existed tests. Closes #26781 from Ngone51/dev-mismatch-schema. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../spark/sql/execution/datasources/DataSource.scala | 7 ++++++- .../org/apache/spark/sql/sources/TableScanSuite.scala | 5 +++-- .../apache/spark/sql/test/DataFrameReaderWriterSuite.scala | 5 ++--- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index e9b8fae7cd735..3615afcf86c7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -343,7 +343,12 @@ case class DataSource( val baseRelation = dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions) if (baseRelation.schema != schema) { - throw new AnalysisException(s"$className does not allow user-specified schemas.") + throw new AnalysisException( + "The user-specified schema doesn't match the actual schema: " + + s"user-specified: ${schema.toDDL}, actual: ${baseRelation.schema.toDDL}. If " + + "you're using DataFrameReader.schema API or creating a table, please do not " + + "specify the schema. Or if you're scanning an existed table, please drop " + + "it and re-create it.") } baseRelation diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index d4e117953942e..9a95bf770772e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -358,7 +358,7 @@ class TableScanSuite extends DataSourceTest with SharedSparkSession { // Make sure we do throw correct exception when users use a relation provider that // only implements the RelationProvider or the SchemaRelationProvider. Seq("TEMPORARY VIEW", "TABLE").foreach { tableType => - val schemaNotAllowed = intercept[Exception] { + val schemaNotMatch = intercept[Exception] { sql( s""" |CREATE $tableType relationProvierWithSchema (i int) @@ -369,7 +369,8 @@ class TableScanSuite extends DataSourceTest with SharedSparkSession { |) """.stripMargin) } - assert(schemaNotAllowed.getMessage.contains("does not allow user-specified schemas")) + assert(schemaNotMatch.getMessage.contains( + "The user-specified schema doesn't match the actual schema")) val schemaNeeded = intercept[Exception] { sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 55a60940a7750..fb939007697c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -489,11 +489,10 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with // when users do not specify the schema checkAnswer(dfReader.load(), spark.range(1, 11).toDF()) - // when users specify the schema + // when users specify a wrong schema val inputSchema = new StructType().add("s", IntegerType, nullable = false) val e = intercept[AnalysisException] { dfReader.schema(inputSchema).load() } - assert(e.getMessage.contains( - "org.apache.spark.sql.sources.SimpleScanSource does not allow user-specified schemas")) + assert(e.getMessage.contains("The user-specified schema doesn't match the actual schema")) } test("read a data source that does not extend RelationProvider") { From 1cac9b2cc669b9cc20a07a97f3caba48a3b30f01 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 10 Dec 2019 09:33:06 -0600 Subject: [PATCH 0832/1387] [SPARK-29967][ML][PYTHON] KMeans support instance weighting ### What changes were proposed in this pull request? add weight support in KMeans ### Why are the changes needed? KMeans should support weighting ### Does this PR introduce any user-facing change? Yes. ```KMeans.setWeightCol``` ### How was this patch tested? Unit Tests Closes #26739 from huaxingao/spark-29967. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../apache/spark/ml/clustering/KMeans.scala | 34 ++- .../spark/ml/util/Instrumentation.scala | 5 + .../mllib/clustering/DistanceMeasure.scala | 20 +- .../spark/mllib/clustering/KMeans.scala | 65 +++-- .../spark/ml/clustering/KMeansSuite.scala | 226 +++++++++++++++++- python/pyspark/ml/clustering.py | 27 ++- 6 files changed, 332 insertions(+), 45 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 5cbba6c77f9fb..a42c920e24987 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -31,9 +31,10 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.clustering.{DistanceMeasure, KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} -import org.apache.spark.sql.functions.udf -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.VersionUtils.majorVersion @@ -41,7 +42,7 @@ import org.apache.spark.util.VersionUtils.majorVersion * Common params for KMeans and KMeansModel */ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFeaturesCol - with HasSeed with HasPredictionCol with HasTol with HasDistanceMeasure { + with HasSeed with HasPredictionCol with HasTol with HasDistanceMeasure with HasWeightCol { /** * The number of clusters to create (k). Must be > 1. Note that it is possible for fewer than @@ -319,12 +320,31 @@ class KMeans @Since("1.5.0") ( @Since("1.5.0") def setSeed(value: Long): this.type = set(seed, value) + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + @Since("2.0.0") override def fit(dataset: Dataset[_]): KMeansModel = instrumented { instr => transformSchema(dataset.schema, logging = true) val handlePersistence = dataset.storageLevel == StorageLevel.NONE - val instances = DatasetUtils.columnToOldVector(dataset, getFeaturesCol) + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + val instances: RDD[(OldVector, Double)] = dataset + .select(DatasetUtils.columnToVector(dataset, getFeaturesCol), w).rdd.map { + case Row(point: Vector, weight: Double) => (OldVectors.fromML(point), weight) + } if (handlePersistence) { instances.persist(StorageLevel.MEMORY_AND_DISK) @@ -333,7 +353,7 @@ class KMeans @Since("1.5.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, featuresCol, predictionCol, k, initMode, initSteps, distanceMeasure, - maxIter, seed, tol) + maxIter, seed, tol, weightCol) val algo = new MLlibKMeans() .setK($(k)) .setInitializationMode($(initMode)) @@ -342,7 +362,7 @@ class KMeans @Since("1.5.0") ( .setSeed($(seed)) .setEpsilon($(tol)) .setDistanceMeasure($(distanceMeasure)) - val parentModel = algo.run(instances, Option(instr)) + val parentModel = algo.runWithWeight(instances, Option(instr)) val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) val summary = new KMeansSummary( model.transform(dataset), diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala index 58a763257af20..d4b39e11fd1d7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala @@ -123,6 +123,10 @@ private[spark] class Instrumentation private () extends Logging with MLEvents { logNamedValue(Instrumentation.loggerTags.numExamples, num) } + def logSumOfWeights(num: Double): Unit = { + logNamedValue(Instrumentation.loggerTags.sumOfWeights, num) + } + /** * Logs the value with customized name field. */ @@ -179,6 +183,7 @@ private[spark] object Instrumentation { val numExamples = "numExamples" val meanOfLabels = "meanOfLabels" val varianceOfLabels = "varianceOfLabels" + val sumOfWeights = "sumOfWeights" } def instrumented[T](body: (Instrumentation => T)): T = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala index 683360efabc76..4d89d1fb9aa92 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala @@ -84,8 +84,8 @@ private[spark] abstract class DistanceMeasure extends Serializable { * @param point a `VectorWithNorm` to be added to `sum` of a cluster * @param sum the `sum` for a cluster to be updated */ - def updateClusterSum(point: VectorWithNorm, sum: Vector): Unit = { - axpy(1.0, point.vector, sum) + def updateClusterSum(point: VectorWithNorm, sum: Vector, weight: Double = 1.0): Unit = { + axpy(weight, point.vector, sum) } /** @@ -100,6 +100,18 @@ private[spark] abstract class DistanceMeasure extends Serializable { new VectorWithNorm(sum) } + /** + * Returns a centroid for a cluster given its `sum` vector and the weightSum of points. + * + * @param sum the `sum` for a cluster + * @param weightSum the weightSum of points in the cluster + * @return the centroid of the cluster + */ + def centroid(sum: Vector, weightSum: Double): VectorWithNorm = { + scal(1.0 / weightSum, sum) + new VectorWithNorm(sum) + } + /** * Returns two new centroids symmetric to the specified centroid applying `noise` with the * with the specified `level`. @@ -249,9 +261,9 @@ private[spark] class CosineDistanceMeasure extends DistanceMeasure { * @param point a `VectorWithNorm` to be added to `sum` of a cluster * @param sum the `sum` for a cluster to be updated */ - override def updateClusterSum(point: VectorWithNorm, sum: Vector): Unit = { + override def updateClusterSum(point: VectorWithNorm, sum: Vector, weight: Double = 1.0): Unit = { assert(point.norm > 0, "Cosine distance is not defined for zero-length vectors.") - axpy(1.0 / point.norm, point.vector, sum) + axpy(weight / point.norm, point.vector, sum) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index ac85fbc235c93..f456dd6395080 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.ml.util.Instrumentation -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.axpy import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -209,11 +209,14 @@ class KMeans private ( */ @Since("0.8.0") def run(data: RDD[Vector]): KMeansModel = { - run(data, None) + val instances: RDD[(Vector, Double)] = data.map { + case (point) => (point, 1.0) + } + runWithWeight(instances, None) } - private[spark] def run( - data: RDD[Vector], + private[spark] def runWithWeight( + data: RDD[(Vector, Double)], instr: Option[Instrumentation]): KMeansModel = { if (data.getStorageLevel == StorageLevel.NONE) { @@ -222,12 +225,15 @@ class KMeans private ( } // Compute squared norms and cache them. - val norms = data.map(Vectors.norm(_, 2.0)) - val zippedData = data.zip(norms).map { case (v, norm) => - new VectorWithNorm(v, norm) + val norms = data.map { case (v, _) => + Vectors.norm(v, 2.0) + } + + val zippedData = data.zip(norms).map { case ((v, w), norm) => + (new VectorWithNorm(v, norm), w) } zippedData.persist() - val model = runAlgorithm(zippedData, instr) + val model = runAlgorithmWithWeight(zippedData, instr) zippedData.unpersist() // Warn at the end of the run as well, for increased visibility. @@ -241,8 +247,8 @@ class KMeans private ( /** * Implementation of K-Means algorithm. */ - private def runAlgorithm( - data: RDD[VectorWithNorm], + private def runAlgorithmWithWeight( + data: RDD[(VectorWithNorm, Double)], instr: Option[Instrumentation]): KMeansModel = { val sc = data.sparkContext @@ -251,14 +257,17 @@ class KMeans private ( val distanceMeasureInstance = DistanceMeasure.decodeFromString(this.distanceMeasure) + val dataVectorWithNorm = data.map(d => d._1) + val weights = data.map(d => d._2) + val centers = initialModel match { case Some(kMeansCenters) => kMeansCenters.clusterCenters.map(new VectorWithNorm(_)) case None => if (initializationMode == KMeans.RANDOM) { - initRandom(data) + initRandom(dataVectorWithNorm) } else { - initKMeansParallel(data, distanceMeasureInstance) + initKMeansParallel(dataVectorWithNorm, distanceMeasureInstance) } } val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 @@ -275,35 +284,43 @@ class KMeans private ( // Execute iterations of Lloyd's algorithm until converged while (iteration < maxIterations && !converged) { val costAccum = sc.doubleAccumulator + val countAccum = sc.longAccumulator val bcCenters = sc.broadcast(centers) // Find the new centers - val collected = data.mapPartitions { points => + val collected = data.mapPartitions { pointsAndWeights => val thisCenters = bcCenters.value val dims = thisCenters.head.vector.size val sums = Array.fill(thisCenters.length)(Vectors.zeros(dims)) - val counts = Array.fill(thisCenters.length)(0L) - points.foreach { point => + // clusterWeightSum is needed to calculate cluster center + // cluster center = + // sample1 * weight1/clusterWeightSum + sample2 * weight2/clusterWeightSum + ... + val clusterWeightSum = Array.ofDim[Double](thisCenters.length) + + pointsAndWeights.foreach { case (point, weight) => val (bestCenter, cost) = distanceMeasureInstance.findClosest(thisCenters, point) - costAccum.add(cost) - distanceMeasureInstance.updateClusterSum(point, sums(bestCenter)) - counts(bestCenter) += 1 + costAccum.add(cost * weight) + countAccum.add(1) + distanceMeasureInstance.updateClusterSum(point, sums(bestCenter), weight) + clusterWeightSum(bestCenter) += weight } - counts.indices.filter(counts(_) > 0).map(j => (j, (sums(j), counts(j)))).iterator - }.reduceByKey { case ((sum1, count1), (sum2, count2)) => + clusterWeightSum.indices.filter(clusterWeightSum(_) > 0) + .map(j => (j, (sums(j), clusterWeightSum(j)))).iterator + }.reduceByKey { case ((sum1, clusterWeightSum1), (sum2, clusterWeightSum2)) => axpy(1.0, sum2, sum1) - (sum1, count1 + count2) + (sum1, clusterWeightSum1 + clusterWeightSum2) }.collectAsMap() if (iteration == 0) { - instr.foreach(_.logNumExamples(collected.values.map(_._2).sum)) + instr.foreach(_.logNumExamples(countAccum.value)) + instr.foreach(_.logSumOfWeights(collected.values.map(_._2).sum)) } - val newCenters = collected.mapValues { case (sum, count) => - distanceMeasureInstance.centroid(sum, count) + val newCenters = collected.mapValues { case (sum, weightSum) => + distanceMeasureInstance.centroid(sum, weightSum) } bcCenters.destroy() diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 173305269c63f..584594436267f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -217,7 +217,6 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes assert(trueCost ~== floatArrayCost absTol 1e-6) } - test("read/write") { def checkModelData(model: KMeansModel, model2: KMeansModel): Unit = { assert(model.clusterCenters === model2.clusterCenters) @@ -254,6 +253,231 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes testClusteringModelSinglePrediction(model, model.predict, dataset, model.getFeaturesCol, model.getPredictionCol) } + + test("compare with weightCol and without weightCol") { + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + Vectors.dense(1.0, 1.0), + Vectors.dense(10.0, 10.0), Vectors.dense(10.0, 10.0), + Vectors.dense(1.0, 0.5), + Vectors.dense(10.0, 4.4), Vectors.dense(10.0, 4.4), + Vectors.dense(-1.0, 1.0), + Vectors.dense(-100.0, 90.0), Vectors.dense(-100.0, 90.0) + )).map(v => TestRow(v))) + + val model1 = new KMeans() + .setK(3) + .setSeed(42) + .setInitMode(MLlibKMeans.RANDOM) + .setTol(1e-6) + .setDistanceMeasure(DistanceMeasure.COSINE) + .fit(df1) + + val predictionDf1 = model1.transform(df1) + assert(predictionDf1.select("prediction").distinct().count() == 3) + val predictionsMap1 = predictionDf1.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap1(Vectors.dense(1.0, 1.0)) == + predictionsMap1(Vectors.dense(10.0, 10.0))) + assert(predictionsMap1(Vectors.dense(1.0, 0.5)) == + predictionsMap1(Vectors.dense(10.0, 4.4))) + assert(predictionsMap1(Vectors.dense(-1.0, 1.0)) == + predictionsMap1(Vectors.dense(-100.0, 90.0))) + + model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + (Vectors.dense(1.0, 1.0), 1.0), + (Vectors.dense(10.0, 10.0), 2.0), + (Vectors.dense(1.0, 0.5), 1.0), + (Vectors.dense(10.0, 4.4), 2.0), + (Vectors.dense(-1.0, 1.0), 1.0), + (Vectors.dense(-100.0, 90.0), 2.0)))).toDF("features", "weightCol") + + val model2 = new KMeans() + .setK(3) + .setSeed(42) + .setInitMode(MLlibKMeans.RANDOM) + .setTol(1e-6) + .setDistanceMeasure(DistanceMeasure.COSINE) + .setWeightCol("weightCol") + .fit(df2) + + val predictionDf2 = model2.transform(df2) + assert(predictionDf2.select("prediction").distinct().count() == 3) + val predictionsMap2 = predictionDf2.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap2(Vectors.dense(1.0, 1.0)) == + predictionsMap2(Vectors.dense(10.0, 10.0))) + assert(predictionsMap2(Vectors.dense(1.0, 0.5)) == + predictionsMap2(Vectors.dense(10.0, 4.4))) + assert(predictionsMap2(Vectors.dense(-1.0, 1.0)) == + predictionsMap2(Vectors.dense(-100.0, 90.0))) + + model2.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + // compare if model1 and model2 have the same cluster centers + assert(model1.clusterCenters.length === model2.clusterCenters.length) + assert(model1.clusterCenters.toSet.subsetOf((model2.clusterCenters.toSet))) + } + + test("Two centers with weightCol") { + // use the same weight for all samples. + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + (Vectors.dense(0.0, 0.0), 2.0), + (Vectors.dense(0.0, 0.1), 2.0), + (Vectors.dense(0.1, 0.0), 2.0), + (Vectors.dense(9.0, 0.0), 2.0), + (Vectors.dense(9.0, 0.2), 2.0), + (Vectors.dense(9.2, 0.0), 2.0)))).toDF("features", "weightCol") + + val model1 = new KMeans() + .setK(2) + .setInitMode(MLlibKMeans.RANDOM) + .setWeightCol("weightCol") + .setMaxIter(10) + .fit(df1) + + val predictionDf1 = model1.transform(df1) + assert(predictionDf1.select("prediction").distinct().count() == 2) + val predictionsMap1 = predictionDf1.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap1(Vectors.dense(0.0, 0.0)) == + predictionsMap1(Vectors.dense(0.0, 0.1))) + assert(predictionsMap1(Vectors.dense(0.0, 0.0)) == + predictionsMap1(Vectors.dense(0.1, 0.0))) + assert(predictionsMap1(Vectors.dense(9.0, 0.0)) == + predictionsMap1(Vectors.dense(9.0, 0.2))) + assert(predictionsMap1(Vectors.dense(9.0, 0.2)) == + predictionsMap1(Vectors.dense(9.2, 0.0))) + + model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + // center 1: + // total weights in cluster 1: 2.0 + 2.0 + 2.0 = 6.0 + // x: 9.0 * (2.0/6.0) + 9.0 * (2.0/6.0) + 9.2 * (2.0/6.0) = 9.066666666666666 + // y: 0.0 * (2.0/6.0) + 0.2 * (2.0/6.0) + 0.0 * (2.0/6.0) = 0.06666666666666667 + // center 2: + // total weights in cluster 2: 2.0 + 2.0 + 2.0 = 6.0 + // x: 0.0 * (2.0/6.0) + 0.0 * (2.0/6.0) + 0.1 * (2.0/6.0) = 0.03333333333333333 + // y: 0.0 * (2.0/6.0) + 0.1 * (2.0/6.0) + 0.0 * (2.0/6.0) = 0.03333333333333333 + val model1_center1 = Vectors.dense(9.066666666666666, 0.06666666666666667) + val model1_center2 = Vectors.dense(0.03333333333333333, 0.03333333333333333) + assert(model1.clusterCenters(0) === model1_center1) + assert(model1.clusterCenters(1) === model1_center2) + + // use different weight + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + (Vectors.dense(0.0, 0.0), 1.0), + (Vectors.dense(0.0, 0.1), 2.0), + (Vectors.dense(0.1, 0.0), 3.0), + (Vectors.dense(9.0, 0.0), 2.5), + (Vectors.dense(9.0, 0.2), 1.0), + (Vectors.dense(9.2, 0.0), 2.0)))).toDF("features", "weightCol") + + val model2 = new KMeans() + .setK(2) + .setInitMode(MLlibKMeans.RANDOM) + .setWeightCol("weightCol") + .setMaxIter(10) + .fit(df2) + + val predictionDf2 = model2.transform(df2) + assert(predictionDf2.select("prediction").distinct().count() == 2) + val predictionsMap2 = predictionDf2.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap2(Vectors.dense(0.0, 0.0)) == + predictionsMap2(Vectors.dense(0.0, 0.1))) + assert(predictionsMap2(Vectors.dense(0.0, 0.0)) == + predictionsMap2(Vectors.dense(0.1, 0.0))) + assert(predictionsMap2(Vectors.dense(9.0, 0.0)) == + predictionsMap2(Vectors.dense(9.0, 0.2))) + assert(predictionsMap2(Vectors.dense(9.0, 0.2)) == + predictionsMap2(Vectors.dense(9.2, 0.0))) + + model2.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + // center 1: + // total weights in cluster 1: 2.5 + 1.0 + 2.0 = 5.5 + // x: 9.0 * (2.5/5.5) + 9.0 * (1.0/5.5) + 9.2 * (2.0/5.5) = 9.072727272727272 + // y: 0.0 * (2.5/5.5) + 0.2 * (1.0/5.5) + 0.0 * (2.0/5.5) = 0.03636363636363637 + // center 2: + // total weights in cluster 2: 1.0 + 2.0 + 3.0 = 6.0 + // x: 0.0 * (1.0/6.0) + 0.0 * (2.0/6.0) + 0.1 * (3.0/6.0) = 0.05 + // y: 0.0 * (1.0/6.0) + 0.1 * (2.0/6.0) + 0.0 * (3.0/6.0) = 0.03333333333333333 + val model2_center1 = Vectors.dense(9.072727272727272, 0.03636363636363637) + val model2_center2 = Vectors.dense(0.05, 0.03333333333333333) + assert(model2.clusterCenters(0) === model2_center1) + assert(model2.clusterCenters(1) === model2_center2) + } + + test("Four centers with weightCol") { + // no weight + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + Vectors.dense(0.1, 0.1), + Vectors.dense(5.0, 0.2), + Vectors.dense(10.0, 0.0), + Vectors.dense(15.0, 0.5), + Vectors.dense(32.0, 18.0), + Vectors.dense(30.1, 20.0), + Vectors.dense(-6.0, -6.0), + Vectors.dense(-10.0, -10.0))).map(v => TestRow(v))) + + val model1 = new KMeans() + .setK(4) + .setInitMode(MLlibKMeans.K_MEANS_PARALLEL) + .setMaxIter(10) + .fit(df1) + + val predictionDf1 = model1.transform(df1) + assert(predictionDf1.select("prediction").distinct().count() == 4) + val predictionsMap1 = predictionDf1.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap1(Vectors.dense(0.1, 0.1)) == + predictionsMap1(Vectors.dense(5.0, 0.2)) ) + assert(predictionsMap1(Vectors.dense(10.0, 0.0)) == + predictionsMap1(Vectors.dense(15.0, 0.5)) ) + assert(predictionsMap1(Vectors.dense(32.0, 18.0)) == + predictionsMap1(Vectors.dense(30.1, 20.0))) + assert(predictionsMap1(Vectors.dense(-6.0, -6.0)) == + predictionsMap1(Vectors.dense(-10.0, -10.0))) + + model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + // use same weight, should have the same result as no weight + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + (Vectors.dense(0.1, 0.1), 2.0), + (Vectors.dense(5.0, 0.2), 2.0), + (Vectors.dense(10.0, 0.0), 2.0), + (Vectors.dense(15.0, 0.5), 2.0), + (Vectors.dense(32.0, 18.0), 2.0), + (Vectors.dense(30.1, 20.0), 2.0), + (Vectors.dense(-6.0, -6.0), 2.0), + (Vectors.dense(-10.0, -10.0), 2.0)))).toDF("features", "weightCol") + + val model2 = new KMeans() + .setK(4) + .setInitMode(MLlibKMeans.K_MEANS_PARALLEL) + .setWeightCol("weightCol") + .setMaxIter(10) + .fit(df2) + + val predictionDf2 = model2.transform(df2) + assert(predictionDf2.select("prediction").distinct().count() == 4) + val predictionsMap2 = predictionDf2.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap2(Vectors.dense(0.1, 0.1)) == + predictionsMap2(Vectors.dense(5.0, 0.2))) + assert(predictionsMap2(Vectors.dense(10.0, 0.0)) == + predictionsMap2(Vectors.dense(15.0, 0.5))) + assert(predictionsMap2(Vectors.dense(32.0, 18.0)) == + predictionsMap2(Vectors.dense(30.1, 20.0))) + assert(predictionsMap2(Vectors.dense(-6.0, -6.0)) == + predictionsMap2(Vectors.dense(-10.0, -10.0))) + + model2.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + assert(model1.clusterCenters === model2.clusterCenters) + } } object KMeansSuite { diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 5aab7a3f5077b..99c38765be9fb 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -423,7 +423,7 @@ def trainingCost(self): @inherit_doc class _KMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionCol, HasTol, - HasDistanceMeasure): + HasDistanceMeasure, HasWeightCol): """ Params for :py:class:`KMeans` and :py:class:`KMeansModel`. @@ -517,12 +517,14 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): (the k-means|| algorithm by Bahmani et al). >>> from pyspark.ml.linalg import Vectors - >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), - ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] - >>> df = spark.createDataFrame(data, ["features"]) + >>> data = [(Vectors.dense([0.0, 0.0]), 2.0), (Vectors.dense([1.0, 1.0]), 2.0), + ... (Vectors.dense([9.0, 8.0]), 2.0), (Vectors.dense([8.0, 9.0]), 2.0)] + >>> df = spark.createDataFrame(data, ["features", "weighCol"]) >>> kmeans = KMeans(k=2) >>> kmeans.setSeed(1) KMeans... + >>> kmeans.setWeightCol("weighCol") + KMeans... >>> kmeans.setMaxIter(10) KMeans... >>> kmeans.getMaxIter() @@ -552,7 +554,7 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): >>> summary.clusterSizes [2, 2] >>> summary.trainingCost - 2.0 + 4.0 >>> kmeans_path = temp_path + "/kmeans" >>> kmeans.save(kmeans_path) >>> kmeans2 = KMeans.load(kmeans_path) @@ -574,11 +576,11 @@ class KMeans(JavaEstimator, _KMeansParams, JavaMLWritable, JavaMLReadable): @keyword_only def __init__(self, featuresCol="features", predictionCol="prediction", k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, seed=None, - distanceMeasure="euclidean"): + distanceMeasure="euclidean", weightCol=None): """ __init__(self, featuresCol="features", predictionCol="prediction", k=2, \ initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, seed=None, \ - distanceMeasure="euclidean") + distanceMeasure="euclidean", weightCol=None) """ super(KMeans, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.KMeans", self.uid) @@ -594,11 +596,11 @@ def _create_model(self, java_model): @since("1.5.0") def setParams(self, featuresCol="features", predictionCol="prediction", k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, seed=None, - distanceMeasure="euclidean"): + distanceMeasure="euclidean", weightCol=None): """ setParams(self, featuresCol="features", predictionCol="prediction", k=2, \ initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, seed=None, \ - distanceMeasure="euclidean") + distanceMeasure="euclidean", weightCol=None) Sets params for KMeans. """ @@ -668,6 +670,13 @@ def setTol(self, value): """ return self._set(tol=value) + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + @inherit_doc class _BisectingKMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionCol, From a9f1809a2a1ea84b5c96bc7fd22cda052a270b41 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 10 Dec 2019 07:49:22 -0800 Subject: [PATCH 0833/1387] [SPARK-30206][SQL] Rename normalizeFilters in DataSourceStrategy to be generic ### What changes were proposed in this pull request? This PR renames `normalizeFilters` in `DataSourceStrategy` to be more generic as the logic is not specific to filters. ### Why are the changes needed? These changes are needed to support PR #26751. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26830 from aokolnychyi/rename-normalize-exprs. Authored-by: Anton Okolnychyi Signed-off-by: Dongjoon Hyun --- .../execution/datasources/DataSourceStrategy.scala | 12 ++++++------ .../execution/datasources/FileSourceStrategy.scala | 2 +- .../datasources/PruneFileSourcePartitions.scala | 2 +- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../datasources/v2/V2ScanRelationPushDown.scala | 2 +- .../datasources/DataSourceStrategySuite.scala | 2 +- 6 files changed, 11 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 8ddeb5edf9431..d44cb11e28762 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -429,14 +429,14 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with object DataSourceStrategy { /** - * The attribute name of predicate could be different than the one in schema in case of - * case insensitive, we should change them to match the one in schema, so we do not need to - * worry about case sensitivity anymore. + * The attribute name may differ from the one in the schema if the query analyzer + * is case insensitive. We should change attribute names to match the ones in the schema, + * so we do not need to worry about case sensitivity anymore. */ - protected[sql] def normalizeFilters( - filters: Seq[Expression], + protected[sql] def normalizeExprs( + exprs: Seq[Expression], attributes: Seq[AttributeReference]): Seq[Expression] = { - filters.map { e => + exprs.map { e => e transform { case a: AttributeReference => a.withName(attributes.find(_.semanticEquals(a)).getOrElse(a).name) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index bd342c7f404fb..8385db9f78653 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -147,7 +147,7 @@ object FileSourceStrategy extends Strategy with Logging { // - filters that need to be evaluated again after the scan val filterSet = ExpressionSet(filters) - val normalizedFilters = DataSourceStrategy.normalizeFilters(filters, l.output) + val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, l.output) val partitionColumns = l.resolve( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 4725b19763930..02d629721327d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -39,7 +39,7 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { _, _)) if filters.nonEmpty && fsRelation.partitionSchemaOption.isDefined => - val normalizedFilters = DataSourceStrategy.normalizeFilters( + val normalizedFilters = DataSourceStrategy.normalizeExprs( filters.filterNot(SubqueryExpression.hasSubquery), logicalRelation.output) val sparkSession = fsRelation.sparkSession diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index a0d10f1d09e63..44de8f275fea6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -166,7 +166,7 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { } // fail if any filter cannot be converted. // correctness depends on removing all matching data. - val filters = DataSourceStrategy.normalizeFilters(condition.toSeq, output) + val filters = DataSourceStrategy.normalizeExprs(condition.toSeq, output) .flatMap(splitConjunctivePredicates(_).map { f => DataSourceStrategy.translateFilter(f).getOrElse( throw new AnalysisException(s"Exec update failed:" + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index 239e3e8f82f18..76bb3c5bd8457 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -31,7 +31,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] { val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options) val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) - val normalizedFilters = DataSourceStrategy.normalizeFilters( + val normalizedFilters = DataSourceStrategy.normalizeExprs( withoutSubquery, relation.output) // `pushedFilters` will be pushed down and evaluated in the underlying data sources. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala index a44a94aaa4f94..b76db70494cf8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala @@ -222,7 +222,7 @@ class DataSourceStrategySuite extends PlanTest with SharedSparkSession { test("SPARK-26865 DataSourceV2Strategy should push normalized filters") { val attrInt = 'cint.int assertResult(Seq(IsNotNull(attrInt))) { - DataSourceStrategy.normalizeFilters(Seq(IsNotNull(attrInt.withName("CiNt"))), Seq(attrInt)) + DataSourceStrategy.normalizeExprs(Seq(IsNotNull(attrInt.withName("CiNt"))), Seq(attrInt)) } } From d9b30694122f8716d3acb448638ef1e2b96ebc7a Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 11 Dec 2019 01:22:34 +0800 Subject: [PATCH 0834/1387] [SPARK-30125][SQL] Remove PostgreSQL dialect ### What changes were proposed in this pull request? Reprocess all PostgreSQL dialect related PRs, listing in order: - #25158: PostgreSQL integral division support [revert] - #25170: UT changes for the integral division support [revert] - #25458: Accept "true", "yes", "1", "false", "no", "0", and unique prefixes as input and trim input for the boolean data type. [revert] - #25697: Combine below 2 feature tags into "spark.sql.dialect" [revert] - #26112: Date substraction support [keep the ANSI-compliant part] - #26444: Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled" [revert] - #26463: Cast to boolean support for PostgreSQL dialect [revert] - #26584: Make the behavior of Postgre dialect independent of ansi mode config [keep the ANSI-compliant part] ### Why are the changes needed? As the discussion in http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-PostgreSQL-dialect-td28417.html, we need to remove PostgreSQL dialect form code base for several reasons: 1. The current approach makes the codebase complicated and hard to maintain. 2. Fully migrating PostgreSQL workloads to Spark SQL is not our focus for now. ### Does this PR introduce any user-facing change? Yes, the config `spark.sql.dialect` will be removed. ### How was this patch tested? Existing UT. Closes #26763 from xuanyuanking/SPARK-30125. Lead-authored-by: Yuanjian Li Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 4 +- .../sql/catalyst/analysis/Analyzer.scala | 7 +- .../catalyst/analysis/PostgreSQLDialect.scala | 49 ----------- .../sql/catalyst/analysis/TypeCoercion.scala | 12 +-- .../spark/sql/catalyst/expressions/Cast.scala | 14 +-- .../sql/catalyst/expressions/arithmetic.scala | 2 +- .../postgreSQL/PostgreCastToBoolean.scala | 83 ----------------- .../sql/catalyst/parser/ParseDriver.scala | 12 +-- .../spark/sql/catalyst/util/StringUtils.scala | 2 +- .../util/postgreSQL/StringUtils.scala | 33 ------- .../apache/spark/sql/internal/SQLConf.scala | 47 ++-------- .../catalyst/analysis/TypeCoercionSuite.scala | 27 +----- .../encoders/ExpressionEncoderSuite.scala | 2 +- .../catalyst/encoders/RowEncoderSuite.scala | 4 +- .../ArithmeticExpressionSuite.scala | 24 ++--- .../sql/catalyst/expressions/CastSuite.scala | 22 ++--- .../expressions/DecimalExpressionSuite.scala | 4 +- .../catalyst/expressions/ScalaUDFSuite.scala | 4 +- .../expressions/postgreSQL/CastSuite.scala | 73 --------------- .../parser/ExpressionParserSuite.scala | 10 +-- .../parser/TableIdentifierParserSuite.scala | 2 +- .../sql-tests/inputs/postgreSQL/boolean.sql | 1 - .../sql-tests/results/datetime.sql.out | 0 .../results/postgreSQL/boolean.sql.out | 66 ++++++-------- .../sql-tests/results/postgreSQL/case.sql.out | 18 ++-- .../sql-tests/results/postgreSQL/date.sql.out | 88 +++++++++---------- .../sql-tests/results/postgreSQL/int2.sql.out | 24 ++--- .../sql-tests/results/postgreSQL/int4.sql.out | 32 +++---- .../sql-tests/results/postgreSQL/int8.sql.out | 78 ++++++++-------- .../postgreSQL/select_implicit.sql.out | 55 +++++++----- .../results/postgreSQL/window_part1.sql.out | 2 +- .../results/udf/postgreSQL/udf-case.sql.out | 18 ++-- .../postgreSQL/udf-select_implicit.sql.out | 55 +++++++----- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../sql/PostgreSQLDialectQuerySuite.scala | 42 --------- .../apache/spark/sql/SQLQueryTestSuite.scala | 4 +- .../ThriftServerQueryTestSuite.scala | 10 +-- 37 files changed, 286 insertions(+), 646 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/datetime.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 9ad6984913a27..4f50ba6d440cb 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -When `spark.sql.dialect=PostgreSQL` or keep default `spark.sql.dialect=Spark` with setting `spark.sql.dialect.spark.ansi.enabled` to true, Spark SQL will use the ANSI mode parser. +When `spark.sql.ansi.enabled` is true, Spark SQL will use the ANSI mode parser. In this mode, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. @@ -28,7 +28,7 @@ When the ANSI mode is disabled, Spark SQL has two kinds of keywords: * Non-reserved keywords: Same definition as the one when the ANSI mode enabled. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. -By default `spark.sql.dialect.spark.ansi.enabled` is false. +By default `spark.sql.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f15476cb3a589..2438ef9218224 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -232,7 +232,6 @@ class Analyzer( ResolveBinaryArithmetic(conf) :: TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), - Batch("PostgreSQL Dialect", Once, PostgreSQLDialect.postgreSQLDialectRules: _*), Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*), Batch("Remove Unresolved Hints", Once, new ResolveHints.RemoveAllHints(conf)), @@ -287,11 +286,7 @@ class Analyzer( case (_, CalendarIntervalType) => Cast(TimeSub(l, r), l.dataType) case (TimestampType, _) => SubtractTimestamps(l, r) case (_, TimestampType) => SubtractTimestamps(l, r) - case (_, DateType) => if (conf.usePostgreSQLDialect) { - DateDiff(l, r) - } else { - SubtractDates(l, r) - } + case (_, DateType) => SubtractDates(l, r) case (DateType, _) => DateSub(l, r) case _ => s } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala deleted file mode 100644 index e7f0e571804d3..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala +++ /dev/null @@ -1,49 +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.sql.catalyst.analysis - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.Cast -import org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, StringType} - -object PostgreSQLDialect { - val postgreSQLDialectRules: List[Rule[LogicalPlan]] = - CastToBoolean :: - Nil - - object CastToBoolean extends Rule[LogicalPlan] with Logging { - override def apply(plan: LogicalPlan): LogicalPlan = { - // The SQL configuration `spark.sql.dialect` can be changed in runtime. - // To make sure the configuration is effective, we have to check it during rule execution. - val conf = SQLConf.get - if (conf.usePostgreSQLDialect) { - plan.transformExpressions { - case Cast(child, dataType, timeZoneId) - if child.dataType != BooleanType && dataType == BooleanType => - PostgreCastToBoolean(child, timeZoneId) - } - } else { - plan - } - } - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index e76193fd94222..dd174ef28f309 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -59,7 +59,7 @@ object TypeCoercion { CaseWhenCoercion :: IfCoercion :: StackCoercion :: - Division(conf) :: + Division :: ImplicitTypeCasts :: DateTimeOperations :: WindowFrameCoercion :: @@ -662,7 +662,7 @@ object TypeCoercion { * Hive only performs integral division with the DIV operator. The arguments to / are always * converted to fractional types. */ - case class Division(conf: SQLConf) extends TypeCoercionRule { + object Division extends TypeCoercionRule { override protected def coerceTypes( plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who has not been resolved yet, @@ -673,13 +673,7 @@ object TypeCoercion { case d: Divide if d.dataType == DoubleType => d case d: Divide if d.dataType.isInstanceOf[DecimalType] => d case Divide(left, right) if isNumericOrNull(left) && isNumericOrNull(right) => - val preferIntegralDivision = conf.usePostgreSQLDialect - (left.dataType, right.dataType) match { - case (_: IntegralType, _: IntegralType) if preferIntegralDivision => - IntegralDivide(left, right) - case _ => - Divide(Cast(left, DoubleType), Cast(right, DoubleType)) - } + Divide(Cast(left, DoubleType), Cast(right, DoubleType)) } private def isNumericOrNull(ex: Expression): Boolean = { 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 a871a746d64ff..fa27a48419dbb 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 @@ -276,7 +276,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def needsTimeZone: Boolean = Cast.needsTimeZone(child.dataType, dataType) // [[func]] assumes the input is no longer null because eval already does the null check. - @inline protected def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) + @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) @@ -387,7 +387,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } // UDFToBoolean - protected[this] def castToBoolean(from: DataType): Any => Any = from match { + private[this] def castToBoolean(from: DataType): Any => Any = from match { case StringType => buildCast[UTF8String](_, s => { if (StringUtils.isTrueString(s)) { @@ -602,7 +602,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit * Change the precision / scale in a given decimal to those set in `decimalType` (if any), * modifying `value` in-place and returning it if successful. If an overflow occurs, it * either returns null or throws an exception according to the value set for - * `spark.sql.dialect.spark.ansi.enabled`. + * `spark.sql.ansi.enabled`. * * NOTE: this modifies `value` in-place, so don't call it on external data. */ @@ -621,7 +621,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit /** * Create new `Decimal` with precision and scale given in `decimalType` (if any). - * If overflow occurs, if `spark.sql.dialect.spark.ansi.enabled` is false, null is returned; + * If overflow occurs, if `spark.sql.ansi.enabled` is false, null is returned; * otherwise, an `ArithmeticException` is thrown. */ private[this] def toPrecision(value: Decimal, decimalType: DecimalType): Decimal = @@ -794,7 +794,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = child.genCode(ctx) val nullSafeCast = nullSafeCastFunction(child.dataType, dataType, ctx) @@ -804,7 +804,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // The function arguments are: `input`, `result` and `resultIsNull`. We don't need `inputIsNull` // in parameter list, because the returned code will be put in null safe evaluation region. - protected type CastFunction = (ExprValue, ExprValue, ExprValue) => Block + private[this] type CastFunction = (ExprValue, ExprValue, ExprValue) => Block private[this] def nullSafeCastFunction( from: DataType, @@ -1254,7 +1254,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def timestampToDoubleCode(ts: ExprValue): Block = code"$ts / (double)$MICROS_PER_SECOND" - protected[this] def castToBooleanCode(from: DataType): CastFunction = from match { + private[this] def castToBooleanCode(from: DataType): CastFunction = from match { case StringType => val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" (c, evPrim, evNull) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 7650fb07a61cd..82a8e6d80a0bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -150,7 +150,7 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { sys.error("BinaryArithmetics must override either calendarIntervalMethod or genCode") // Name of the function for the exact version of this expression in [[Math]]. - // If the option "spark.sql.dialect.spark.ansi.enabled" is enabled and there is corresponding + // If the option "spark.sql.ansi.enabled" is enabled and there is corresponding // function in [[Math]], the exact function will be called instead of evaluation with [[symbol]]. def exactMathMethod: Option[String] = None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala deleted file mode 100644 index 02bc6f0d0d8bf..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala +++ /dev/null @@ -1,83 +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.sql.catalyst.expressions.postgreSQL - -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions.{CastBase, Expression, TimeZoneAwareExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.util.postgreSQL.StringUtils -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -case class PostgreCastToBoolean(child: Expression, timeZoneId: Option[String]) - extends CastBase { - - override protected def ansiEnabled = - throw new UnsupportedOperationException("PostgreSQL dialect doesn't support ansi mode") - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) - - override def checkInputDataTypes(): TypeCheckResult = child.dataType match { - case StringType | IntegerType | NullType => - TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"cannot cast type ${child.dataType} to boolean") - } - - override def castToBoolean(from: DataType): Any => Any = from match { - case StringType => - buildCast[UTF8String](_, str => { - val s = str.trimAll().toLowerCase() - if (StringUtils.isTrueString(s)) { - true - } else if (StringUtils.isFalseString(s)) { - false - } else { - throw new IllegalArgumentException(s"invalid input syntax for type boolean: $s") - } - }) - case IntegerType => - super.castToBoolean(from) - } - - override def castToBooleanCode(from: DataType): CastFunction = from match { - case StringType => - val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" - (c, evPrim, evNull) => - code""" - if ($stringUtils.isTrueString($c.trim().toLowerCase())) { - $evPrim = true; - } else if ($stringUtils.isFalseString($c.trim().toLowerCase())) { - $evPrim = false; - } else { - throw new IllegalArgumentException("invalid input syntax for type boolean: $c"); - } - """ - - case IntegerType => - super.castToBooleanCode(from) - } - - override def dataType: DataType = BooleanType - - override def nullable: Boolean = child.nullable - - override def toString: String = s"PostgreCastToBoolean($child as ${dataType.simpleString})" - - override def sql: String = s"CAST(${child.sql} AS ${dataType.sql})" -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index b968848224c54..c96f2da9a5289 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.Dialect import org.apache.spark.sql.types.{DataType, StructType} /** @@ -89,20 +88,13 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = { logDebug(s"Parsing command: $command") - // When we use PostgreSQL dialect or use Spark dialect with setting - // `spark.sql.dialect.spark.ansi.enabled=true`, the parser will use ANSI SQL standard keywords. - val SQLStandardKeywordBehavior = conf.dialect match { - case Dialect.POSTGRESQL => true - case Dialect.SPARK => conf.dialectSparkAnsiEnabled - } - val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced lexer.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled lexer.legacy_create_hive_table_by_default_enabled = conf.createHiveTableByDefaultEnabled - lexer.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior + lexer.SQL_standard_keyword_behavior = conf.ansiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -112,7 +104,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced parser.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled parser.legacy_create_hive_table_by_default_enabled = conf.createHiveTableByDefaultEnabled - parser.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior + parser.SQL_standard_keyword_behavior = conf.ansiEnabled try { try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index 51ab3df0e0bfb..b42ae4e45366e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -38,7 +38,7 @@ object StringUtils extends Logging { * throw an [[AnalysisException]]. * * @param pattern the SQL pattern to convert - * @param escapeStr the escape string contains one character. + * @param escapeChar the escape string contains one character. * @return the equivalent Java regular expression of the pattern */ def escapeLikeRegex(pattern: String, escapeChar: Char): String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala deleted file mode 100644 index 1ae15df29d6e7..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala +++ /dev/null @@ -1,33 +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.sql.catalyst.util.postgreSQL - -import org.apache.spark.unsafe.types.UTF8String - -object StringUtils { - // "true", "yes", "1", "false", "no", "0", and unique prefixes of these strings are accepted. - private[this] val trueStrings = - Set("true", "tru", "tr", "t", "yes", "ye", "y", "on", "1").map(UTF8String.fromString) - - private[this] val falseStrings = - Set("false", "fals", "fal", "fa", "f", "no", "n", "off", "of", "0").map(UTF8String.fromString) - - def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s) - - def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s) -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1ff23f697c4db..f7c1d4a798ecb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1675,37 +1675,6 @@ object SQLConf { .booleanConf .createWithDefault(false) - object Dialect extends Enumeration { - val SPARK, POSTGRESQL = Value - } - - val DIALECT = - buildConf("spark.sql.dialect") - .doc("The specific features of the SQL language to be adopted, which are available when " + - "accessing the given database. Currently, Spark supports two database dialects, `Spark` " + - "and `PostgreSQL`. With `PostgreSQL` dialect, Spark will: " + - "1. perform integral division with the / operator if both sides are integral types; " + - "2. accept \"true\", \"yes\", \"1\", \"false\", \"no\", \"0\", and unique prefixes as " + - "input and trim input for the boolean data type.") - .stringConf - .transform(_.toUpperCase(Locale.ROOT)) - .checkValues(Dialect.values.map(_.toString)) - .createWithDefault(Dialect.SPARK.toString) - - val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") - .internal() - .doc("This configuration is deprecated and will be removed in the future releases." + - "It is replaced by spark.sql.dialect.spark.ansi.enabled.") - .booleanConf - .createWithDefault(false) - - val DIALECT_SPARK_ANSI_ENABLED = buildConf("spark.sql.dialect.spark.ansi.enabled") - .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + - "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + - "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + - "the SQL parser.") - .fallbackConf(ANSI_ENABLED) - val VALIDATE_PARTITION_COLUMNS = buildConf("spark.sql.sources.validatePartitionColumns") .internal() @@ -1826,6 +1795,14 @@ object SQLConf { .checkValues(IntervalStyle.values.map(_.toString)) .createWithDefault(IntervalStyle.MULTI_UNITS.toString) + val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") + .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + + "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + + "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + + "the SQL parser.") + .booleanConf + .createWithDefault(false) + val SORT_BEFORE_REPARTITION = buildConf("spark.sql.execution.sortBeforeRepartition") .internal() @@ -2563,13 +2540,7 @@ class SQLConf extends Serializable with Logging { def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE)) - def dialect: Dialect.Value = Dialect.withName(getConf(DIALECT)) - - def usePostgreSQLDialect: Boolean = dialect == Dialect.POSTGRESQL - - def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) - - def ansiEnabled: Boolean = usePostgreSQLDialect || dialectSparkAnsiEnabled + def ansiEnabled: Boolean = getConf(ANSI_ENABLED) def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index f35617b374c99..0d6f9bcedb6a2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1425,7 +1425,7 @@ class TypeCoercionSuite extends AnalysisTest { test("SPARK-15776 Divide expression's dataType should be casted to Double or Decimal " + "in aggregation function like sum") { - val rules = Seq(FunctionArgumentConversion, Division(conf)) + val rules = Seq(FunctionArgumentConversion, Division) // Casts Integer to Double ruleTest(rules, sum(Divide(4, 3)), sum(Divide(Cast(4, DoubleType), Cast(3, DoubleType)))) // Left expression is Double, right expression is Int. Another rule ImplicitTypeCasts will @@ -1444,35 +1444,12 @@ class TypeCoercionSuite extends AnalysisTest { } test("SPARK-17117 null type coercion in divide") { - val rules = Seq(FunctionArgumentConversion, Division(conf), ImplicitTypeCasts) + val rules = Seq(FunctionArgumentConversion, Division, ImplicitTypeCasts) val nullLit = Literal.create(null, NullType) ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), Cast(nullLit, DoubleType))) ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), Cast(1L, DoubleType))) } - test("SPARK-28395 Division operator support integral division") { - val rules = Seq(FunctionArgumentConversion, Division(conf)) - Seq(SQLConf.Dialect.SPARK, SQLConf.Dialect.POSTGRESQL).foreach { dialect => - withSQLConf(SQLConf.DIALECT.key -> dialect.toString) { - val result1 = if (dialect == SQLConf.Dialect.POSTGRESQL) { - IntegralDivide(1L, 1L) - } else { - Divide(Cast(1L, DoubleType), Cast(1L, DoubleType)) - } - ruleTest(rules, Divide(1L, 1L), result1) - val result2 = if (dialect == SQLConf.Dialect.POSTGRESQL) { - IntegralDivide(1, Cast(1, ShortType)) - } else { - Divide(Cast(1, DoubleType), Cast(Cast(1, ShortType), DoubleType)) - } - ruleTest(rules, Divide(1, Cast(1, ShortType)), result2) - - ruleTest(rules, Divide(1L, 1D), Divide(Cast(1L, DoubleType), Cast(1D, DoubleType))) - ruleTest(rules, Divide(Decimal(1.1), 1L), Divide(Decimal(1.1), 1L)) - } - } - } - test("binary comparison with string promotion") { val rule = TypeCoercion.PromoteStrings(conf) ruleTest(rule, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 62e688e4d4bd6..c1f1be3b30e4b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -436,7 +436,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes testAndVerifyNotLeakingReflectionObjects( s"overflowing $testName, ansiEnabled=$ansiEnabled") { withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> ansiEnabled.toString + SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString ) { // Need to construct Encoder here rather than implicitly resolving it // so that SQLConf changes are respected. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index fe068f7a5f6c2..1a1cab823d4f3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -169,7 +169,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } private def testDecimalOverflow(schema: StructType, row: Row): Unit = { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { val encoder = RowEncoder(schema).resolveAndBind() intercept[Exception] { encoder.toRow(row) @@ -182,7 +182,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val encoder = RowEncoder(schema).resolveAndBind() assert(encoder.fromRow(encoder.toRow(row)).get(0) == null) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 6e3fc438e41ea..ad8b1a1673679 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -61,7 +61,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Add(positiveLongLit, negativeLongLit), -1L) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Add, tpe, tpe) } @@ -80,7 +80,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(UnaryMinus(Literal(Int.MinValue)), Int.MinValue) checkEvaluation(UnaryMinus(Literal(Short.MinValue)), Short.MinValue) checkEvaluation(UnaryMinus(Literal(Byte.MinValue)), Byte.MinValue) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkExceptionInExpression[ArithmeticException]( UnaryMinus(Literal(Long.MinValue)), "overflow") checkExceptionInExpression[ArithmeticException]( @@ -122,7 +122,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Subtract(positiveLongLit, negativeLongLit), positiveLong - negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Subtract, tpe, tpe) } @@ -144,7 +144,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Multiply(positiveLongLit, negativeLongLit), positiveLong * negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Multiply, tpe, tpe) } @@ -445,12 +445,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minLongLiteral, minLongLiteral) val e5 = Subtract(minLongLiteral, maxLongLiteral) val e6 = Multiply(minLongLiteral, minLongLiteral) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Long.MinValue) checkEvaluation(e2, Long.MinValue) checkEvaluation(e3, -2L) @@ -469,12 +469,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minIntLiteral, minIntLiteral) val e5 = Subtract(minIntLiteral, maxIntLiteral) val e6 = Multiply(minIntLiteral, minIntLiteral) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Int.MinValue) checkEvaluation(e2, Int.MinValue) checkEvaluation(e3, -2) @@ -493,12 +493,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minShortLiteral, minShortLiteral) val e5 = Subtract(minShortLiteral, maxShortLiteral) val e6 = Multiply(minShortLiteral, minShortLiteral) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Short.MinValue) checkEvaluation(e2, Short.MinValue) checkEvaluation(e3, (-2).toShort) @@ -517,12 +517,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minByteLiteral, minByteLiteral) val e5 = Subtract(minByteLiteral, maxByteLiteral) val e6 = Multiply(minByteLiteral, minByteLiteral) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Byte.MinValue) checkEvaluation(e2, Byte.MinValue) checkEvaluation(e3, (-2).toByte) 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 dde25ad994c5b..2d8f22c34ade7 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 @@ -685,10 +685,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkCast("y", true) checkCast("yes", true) checkCast("1", true) - checkCast("1 ", true) - checkCast("f", false) - checkCast("f\t", false) checkCast("false", false) checkCast("FAlsE", false) checkCast("n", false) @@ -696,8 +693,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkCast("0", false) checkEvaluation(cast("abc", BooleanType), null) - checkEvaluation(cast("tru", BooleanType), null) - checkEvaluation(cast("fla", BooleanType), null) checkEvaluation(cast("", BooleanType), null) } @@ -894,8 +889,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to decimal type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { checkExceptionInExpression[ArithmeticException]( cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( @@ -961,8 +955,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to byte type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ByteType) Seq(Byte.MaxValue + 1, Byte.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ByteType), "overflow") @@ -987,8 +980,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to short type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ShortType) Seq(Short.MaxValue + 1, Short.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ShortType), "overflow") @@ -1013,8 +1005,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to int type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key ->requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(IntegerType) testLongMaxAndMin(IntegerType) @@ -1031,8 +1022,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to long type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testLongMaxAndMin(LongType) Seq(Long.MaxValue, 0, Long.MinValue).foreach { value => @@ -1209,7 +1199,7 @@ class CastSuite extends CastSuiteBase { } test("SPARK-28470: Cast should honor nullOnOverflow property") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(Cast(Literal("134.12"), DecimalType(3, 2)), null) checkEvaluation( Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala index 8609d888b7bc9..36bc3db580400 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala @@ -32,7 +32,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("MakeDecimal") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) @@ -41,7 +41,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { evaluateWithoutCodegen(overflowExpr, null) checkEvaluationWithUnsafeProjection(overflowExpr, null) } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index cf6ebfb0ecefb..c5ffc381b58e2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -57,7 +57,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-28369: honor nullOnOverflow config for ScalaUDF") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, @@ -69,7 +69,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } assert(e2.getCause.isInstanceOf[ArithmeticException]) } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala deleted file mode 100644 index 6c5218b379f31..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala +++ /dev/null @@ -1,73 +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.sql.catalyst.expressions.postgreSQL - -import java.sql.{Date, Timestamp} - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} - -class CastSuite extends SparkFunSuite with ExpressionEvalHelper { - private def checkPostgreCastToBoolean(v: Any, expected: Any): Unit = { - checkEvaluation(PostgreCastToBoolean(Literal(v), None), expected) - } - - test("cast string to boolean") { - checkPostgreCastToBoolean("true", true) - checkPostgreCastToBoolean("tru", true) - checkPostgreCastToBoolean("tr", true) - checkPostgreCastToBoolean("t", true) - checkPostgreCastToBoolean("tRUe", true) - checkPostgreCastToBoolean(" tRue ", true) - checkPostgreCastToBoolean(" tRu ", true) - checkPostgreCastToBoolean("yes", true) - checkPostgreCastToBoolean("ye", true) - checkPostgreCastToBoolean("y", true) - checkPostgreCastToBoolean("1", true) - checkPostgreCastToBoolean("on", true) - - checkPostgreCastToBoolean("false", false) - checkPostgreCastToBoolean("fals", false) - checkPostgreCastToBoolean("fal", false) - checkPostgreCastToBoolean("fa", false) - checkPostgreCastToBoolean("f", false) - checkPostgreCastToBoolean(" fAlse ", false) - checkPostgreCastToBoolean(" fAls ", false) - checkPostgreCastToBoolean(" FAlsE ", false) - checkPostgreCastToBoolean("no", false) - checkPostgreCastToBoolean("n", false) - checkPostgreCastToBoolean("0", false) - checkPostgreCastToBoolean("off", false) - checkPostgreCastToBoolean("of", false) - - intercept[IllegalArgumentException](PostgreCastToBoolean(Literal("o"), None).eval()) - intercept[IllegalArgumentException](PostgreCastToBoolean(Literal("abc"), None).eval()) - intercept[IllegalArgumentException](PostgreCastToBoolean(Literal(""), None).eval()) - } - - test("unsupported data types to cast to boolean") { - assert(PostgreCastToBoolean(Literal(new Timestamp(1)), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(new Date(1)), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toLong), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toShort), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toByte), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(BigDecimal(1.0)), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toDouble), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toFloat), None).checkInputDataTypes().isFailure) - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 0b694ea954156..90bb487ec6168 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -645,7 +645,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual(s"${sign}interval $intervalValue", expectedLiteral) // SPARK-23264 Support interval values without INTERVAL clauses if ANSI SQL enabled - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } } @@ -732,12 +732,12 @@ class ExpressionParserSuite extends AnalysisTest { test("SPARK-23264 Interval Compatibility tests") { def checkIntervals(intervalValue: String, expected: Literal): Unit = { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } // Compatibility tests: If ANSI SQL disabled, `intervalValue` should be parsed as an alias - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val aliases = defaultParser.parseExpression(intervalValue).collect { case a @ Alias(_: Literal, name) if intervalUnits.exists { unit => name.startsWith(unit.toString) } => a @@ -835,12 +835,12 @@ class ExpressionParserSuite extends AnalysisTest { } test("current date/timestamp braceless expressions") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual("current_date", CurrentDate()) assertEqual("current_timestamp", CurrentTimestamp()) } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { assertEqual("current_date", UnresolvedAttribute.quoted("current_date")) assertEqual("current_timestamp", UnresolvedAttribute.quoted("current_timestamp")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 904ae384154c8..23063bbab7aa2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -660,7 +660,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { } test("table identifier - reserved/non-reserved keywords if ANSI mode enabled") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { reservedKeywordsInAnsiMode.foreach { keyword => val errMsg = intercept[ParseException] { parseTableIdentifier(keyword) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql index 178823bcfe9d6..3a949c834deb5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql @@ -98,7 +98,6 @@ SELECT boolean('f') <= boolean('t') AS true; -- explicit casts to/from text SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false`; - SELECT boolean(string(' true ')) AS true, boolean(string(' FALSE')) AS `false`; SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false`; diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out old mode 100644 new mode 100755 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out index e5f3425efc458..39f239d7dbf2e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out @@ -53,10 +53,9 @@ true -- !query 6 SELECT boolean('test') AS error -- !query 6 schema -struct<> +struct -- !query 6 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: test +NULL -- !query 7 @@ -70,10 +69,9 @@ false -- !query 8 SELECT boolean('foo') AS error -- !query 8 schema -struct<> +struct -- !query 8 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: foo +NULL -- !query 9 @@ -95,10 +93,9 @@ true -- !query 11 SELECT boolean('yeah') AS error -- !query 11 schema -struct<> +struct -- !query 11 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: yeah +NULL -- !query 12 @@ -120,10 +117,9 @@ false -- !query 14 SELECT boolean('nay') AS error -- !query 14 schema -struct<> +struct -- !query 14 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: nay +NULL -- !query 15 @@ -131,7 +127,7 @@ SELECT boolean('on') AS true -- !query 15 schema struct -- !query 15 output -true +NULL -- !query 16 @@ -139,7 +135,7 @@ SELECT boolean('off') AS `false` -- !query 16 schema struct -- !query 16 output -false +NULL -- !query 17 @@ -147,34 +143,31 @@ SELECT boolean('of') AS `false` -- !query 17 schema struct -- !query 17 output -false +NULL -- !query 18 SELECT boolean('o') AS error -- !query 18 schema -struct<> +struct -- !query 18 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: o +NULL -- !query 19 SELECT boolean('on_') AS error -- !query 19 schema -struct<> +struct -- !query 19 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: on_ +NULL -- !query 20 SELECT boolean('off_') AS error -- !query 20 schema -struct<> +struct -- !query 20 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: off_ +NULL -- !query 21 @@ -188,10 +181,9 @@ true -- !query 22 SELECT boolean('11') AS error -- !query 22 schema -struct<> +struct -- !query 22 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: 11 +NULL -- !query 23 @@ -205,19 +197,17 @@ false -- !query 24 SELECT boolean('000') AS error -- !query 24 schema -struct<> +struct -- !query 24 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: 000 +NULL -- !query 25 SELECT boolean('') AS error -- !query 25 schema -struct<> +struct -- !query 25 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: +NULL -- !query 26 @@ -320,19 +310,17 @@ true false -- !query 38 SELECT boolean(string(' tru e ')) AS invalid -- !query 38 schema -struct<> +struct -- !query 38 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: tru e +NULL -- !query 39 SELECT boolean(string('')) AS invalid -- !query 39 schema -struct<> +struct -- !query 39 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: +NULL -- !query 40 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out index 348198b060238..a3410684e85dd 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out @@ -176,28 +176,28 @@ struct -- !query 18 SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END -- !query 18 schema -struct +struct -- !query 18 output -1 +1.0 -- !query 19 SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct +struct -- !query 19 output -1 +1.0 -- !query 20 SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl -- !query 20 schema -struct 100) THEN (1 div 0) ELSE 0 END:int> +struct 100) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) ELSE CAST(0 AS DOUBLE) END:double> -- !query 20 output -0 -0 -0 -0 +0.0 +0.0 +0.0 +0.0 -- !query 21 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out old mode 100644 new mode 100755 index 5371e07423c28..700476e91eaac --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -482,93 +482,93 @@ SELECT date '5874898-01-01' -- !query 46 SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL -- !query 46 schema -struct +struct -- !query 46 output --1035 --1036 --1037 --1400 --1401 --1402 --1403 --15542 --15607 -13977 -14343 -14710 -91 -92 -93 +-2 years -10 months +-2 years -10 months -1 days +-2 years -9 months -30 days +-3 years -10 months +-3 years -10 months -1 days +-3 years -10 months -2 days +-3 years -9 months -30 days +-42 years -6 months -18 days +-42 years -8 months -22 days +3 months +3 months 1 days +3 months 2 days +38 years 3 months 7 days +39 years 3 months 8 days +40 years 3 months 9 days -- !query 47 SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL -- !query 47 schema -struct +struct -- !query 47 output --4585 --4650 -11048 -11049 -11050 -24934 -25300 -25667 -9554 -9555 -9556 -9557 -9920 -9921 -9922 +-12 years -6 months -18 days +-12 years -8 months -22 days +26 years 1 months 27 days +26 years 1 months 28 days +26 years 2 months +26 years 2 months 1 days +27 years 1 months 27 days +27 years 2 months +27 years 2 months 1 days +30 years 3 months +30 years 3 months 1 days +30 years 3 months 2 days +68 years 3 months 7 days +69 years 3 months 8 days +70 years 3 months 9 days -- !query 48 SELECT date 'yesterday' - date 'today' AS `One day` -- !query 48 schema -struct +struct -- !query 48 output --1 +-1 days -- !query 49 SELECT date 'today' - date 'tomorrow' AS `One day` -- !query 49 schema -struct +struct -- !query 49 output --1 +-1 days -- !query 50 SELECT date 'yesterday' - date 'tomorrow' AS `Two days` -- !query 50 schema -struct +struct -- !query 50 output --2 +-2 days -- !query 51 SELECT date 'tomorrow' - date 'today' AS `One day` -- !query 51 schema -struct +struct -- !query 51 output -1 +1 days -- !query 52 SELECT date 'today' - date 'yesterday' AS `One day` -- !query 52 schema -struct +struct -- !query 52 output -1 +1 days -- !query 53 SELECT date 'tomorrow' - date 'yesterday' AS `Two days` -- !query 53 schema -struct +struct -- !query 53 output -2 +2 days -- !query 54 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out old mode 100644 new mode 100755 index d0a14618a5163..d6e9f7ec79d68 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -268,25 +268,25 @@ struct -- !query 27 SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i -- !query 27 schema -struct +struct -- !query 27 output - -1234 -617 - -32767 -16383 - 0 0 - 1234 617 - 32767 16383 + -1234 -617.0 + -32767 -16383.5 + 0 0.0 + 1234 617.0 + 32767 16383.5 -- !query 28 SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i -- !query 28 schema -struct +struct -- !query 28 output - -1234 -617 - -32767 -16383 - 0 0 - 1234 617 - 32767 16383 + -1234 -617.0 + -32767 -16383.5 + 0 0.0 + 1234 617.0 + 32767 16383.5 -- !query 29 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out old mode 100644 new mode 100755 index 30afd6e695842..e1cb96d09ec75 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -321,25 +321,25 @@ struct -- !query 33 SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i -- !query 33 schema -struct +struct -- !query 33 output - -123456 -61728 - -2147483647 -1073741823 - 0 0 - 123456 61728 - 2147483647 1073741823 + -123456 -61728.0 + -2147483647 -1.0737418235E9 + 0 0.0 + 123456 61728.0 + 2147483647 1.0737418235E9 -- !query 34 SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i -- !query 34 schema -struct +struct -- !query 34 output - -123456 -61728 - -2147483647 -1073741823 - 0 0 - 123456 61728 - 2147483647 1073741823 + -123456 -61728.0 + -2147483647 -1.0737418235E9 + 0 0.0 + 123456 61728.0 + 2147483647 1.0737418235E9 -- !query 35 @@ -417,17 +417,17 @@ struct -- !query 44 SELECT 2 + 2 / 2 AS three -- !query 44 schema -struct +struct -- !query 44 output -3 +3.0 -- !query 45 SELECT (2 + 2) / 2 AS two -- !query 45 schema -struct +struct -- !query 45 output -2 +2.0 -- !query 46 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out old mode 100644 new mode 100755 index 7a8e706cb8d85..8e2700caa3345 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -409,13 +409,13 @@ struct -- !query 42 SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL -- !query 42 schema -struct +struct -- !query 42 output - 123 456 0 123 - 123 4567890123456789 0 123 - 4567890123456789 -4567890123456789 -1 0 - 4567890123456789 123 37137318076884 57 - 4567890123456789 4567890123456789 1 0 + 123 456 0.26973684210526316 123 + 123 4567890123456789 2.6927092525360204E-14 123 + 4567890123456789 -4567890123456789 -1.0 0 + 4567890123456789 123 3.713731807688446E13 57 + 4567890123456789 4567890123456789 1.0 0 -- !query 43 @@ -493,49 +493,49 @@ struct -- !query 49 SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL -- !query 49 schema -struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:bigint> +struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:double> -- !query 49 output -165 81 5166 2 -165 81 5166 2 -4567890123456831 4567890123456747 191851385185185138 108759288653733 -4567890123456831 4567890123456747 191851385185185138 108759288653733 -4567890123456831 4567890123456747 191851385185185138 108759288653733 +165 81 5166 2.9285714285714284 +165 81 5166 2.9285714285714284 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 -- !query 50 SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL -- !query 50 schema -struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:bigint> +struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:double> -- !query 50 output -369 123 30258 2 -369 123 30258 2 -4567890123457035 -4567890123456543 1123700970370370094 0 -4567890123457035 -4567890123456543 1123700970370370094 0 -4567890123457035 -4567890123456543 1123700970370370094 0 +369 123 30258 2.0 +369 123 30258 2.0 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 -- !query 51 SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL -- !query 51 schema -struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:bigint> +struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:double> -- !query 51 output -165 81 5166 2 -165 81 5166 2 -4567890123456831 4567890123456747 191851385185185138 108759288653733 -4567890123456831 4567890123456747 191851385185185138 108759288653733 -4567890123456831 4567890123456747 191851385185185138 108759288653733 +165 81 5166 2.9285714285714284 +165 81 5166 2.9285714285714284 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 -- !query 52 SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL -- !query 52 schema -struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:bigint> +struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:double> -- !query 52 output -369 123 30258 2 -369 123 30258 2 -4567890123457035 -4567890123456543 1123700970370370094 0 -4567890123457035 -4567890123456543 1123700970370370094 0 -4567890123457035 -4567890123456543 1123700970370370094 0 +369 123 30258 2.0 +369 123 30258 2.0 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 -- !query 53 @@ -569,7 +569,7 @@ struct -- !query 56 select bigint('9223372036854775800') / bigint('0') -- !query 56 schema -struct<(CAST(9223372036854775800 AS BIGINT) div CAST(0 AS BIGINT)):bigint> +struct<(CAST(CAST(9223372036854775800 AS BIGINT) AS DOUBLE) / CAST(CAST(0 AS BIGINT) AS DOUBLE)):double> -- !query 56 output NULL @@ -577,7 +577,7 @@ NULL -- !query 57 select bigint('-9223372036854775808') / smallint('0') -- !query 57 schema -struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(0 AS SMALLINT) AS BIGINT)):bigint> +struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(0 AS SMALLINT) AS DOUBLE)):double> -- !query 57 output NULL @@ -585,7 +585,7 @@ NULL -- !query 58 select smallint('100') / bigint('0') -- !query 58 schema -struct<(CAST(CAST(100 AS SMALLINT) AS BIGINT) div CAST(0 AS BIGINT)):bigint> +struct<(CAST(CAST(100 AS SMALLINT) AS DOUBLE) / CAST(CAST(0 AS BIGINT) AS DOUBLE)):double> -- !query 58 output NULL @@ -740,9 +740,9 @@ long overflow -- !query 74 SELECT bigint((-9223372036854775808)) / bigint((-1)) -- !query 74 schema -struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(-1 AS BIGINT)):bigint> +struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS BIGINT) AS DOUBLE)):double> -- !query 74 output --9223372036854775808 +9.223372036854776E18 -- !query 75 @@ -765,9 +765,9 @@ long overflow -- !query 77 SELECT bigint((-9223372036854775808)) / int((-1)) -- !query 77 schema -struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS INT) AS DOUBLE)):double> -- !query 77 output --9223372036854775808 +9.223372036854776E18 -- !query 78 @@ -790,9 +790,9 @@ long overflow -- !query 80 SELECT bigint((-9223372036854775808)) / smallint((-1)) -- !query 80 schema -struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS SMALLINT) AS DOUBLE)):double> -- !query 80 output --9223372036854775808 +9.223372036854776E18 -- !query 81 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out old mode 100644 new mode 100755 index 0675820b381da..e9ba62801d6a2 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out @@ -239,31 +239,36 @@ struct SELECT a/2, a/2 FROM test_missing_target ORDER BY a/2 -- !query 23 schema -struct<(a div 2):int,(a div 2):int> +struct<(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double,(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double> -- !query 23 output -0 0 -0 0 -1 1 -1 1 -2 2 -2 2 -3 3 -3 3 -4 4 -4 4 +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 -- !query 24 SELECT a/2, a/2 FROM test_missing_target GROUP BY a/2 ORDER BY a/2 -- !query 24 schema -struct<(a div 2):int,(a div 2):int> +struct<(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double,(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double> -- !query 24 output -0 0 -1 1 -2 2 -3 3 -4 4 +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 -- !query 25 @@ -331,7 +336,8 @@ SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2 struct -- !query 30 output 1 -5 +2 +3 4 @@ -370,8 +376,10 @@ SELECT count(b) FROM test_missing_target -- !query 33 schema struct -- !query 33 output -7 +4 3 +2 +1 -- !query 34 @@ -390,11 +398,12 @@ SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y WHERE x.a = y.a GROUP BY x.b/2 ORDER BY x.b/2 -- !query 35 schema -struct<(b div 2):int,count(b):bigint> +struct<(CAST(b AS DOUBLE) / CAST(2 AS DOUBLE)):double,count(b):bigint> -- !query 35 output -0 1 -1 5 -2 4 +0.5 1 +1.0 2 +1.5 3 +2.0 4 -- !query 36 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out old mode 100644 new mode 100755 index 45bc98ae97640..61b86b556e1a8 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out @@ -387,7 +387,7 @@ struct -- !query 23 SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 -- !query 23 schema -struct +struct -- !query 23 output 0.0 0.0 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out old mode 100644 new mode 100755 index 44a764ce4e6dd..7012b15213662 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out @@ -176,28 +176,28 @@ struct -- !query 18 SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END -- !query 18 schema -struct +struct -- !query 18 output -1 +1.0 -- !query 19 SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct +struct -- !query 19 output -1 +1.0 -- !query 20 SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl -- !query 20 schema -struct 100) THEN CAST(udf(cast((1 div 0) as string)) AS INT) ELSE CAST(udf(cast(0 as string)) AS INT) END:int> +struct 100) THEN CAST(udf(cast((cast(1 as double) / cast(0 as double)) as string)) AS DOUBLE) ELSE CAST(CAST(udf(cast(0 as string)) AS INT) AS DOUBLE) END:double> -- !query 20 output -0 -0 -0 -0 +0.0 +0.0 +0.0 +0.0 -- !query 21 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out index a60cbf33b9b24..2540f0260f09c 100755 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out @@ -242,31 +242,36 @@ struct +struct -- !query 23 output -0 0 -0 0 -1 1 -1 1 -2 2 -2 2 -3 3 -3 3 -4 4 -4 4 +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 -- !query 24 SELECT udf(a/2), udf(a/2) FROM test_missing_target GROUP BY udf(a/2) ORDER BY udf(a/2) -- !query 24 schema -struct +struct -- !query 24 output -0 0 -1 1 -2 2 -3 3 -4 4 +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 -- !query 25 @@ -334,7 +339,8 @@ SELECT udf(count(b)) FROM test_missing_target GROUP BY udf(b/2) ORDER BY udf(b/2 struct -- !query 30 output 1 -5 +2 +3 4 @@ -373,8 +379,10 @@ SELECT udf(count(b)) FROM test_missing_target -- !query 33 schema struct -- !query 33 output -7 +4 3 +2 +1 -- !query 34 @@ -394,11 +402,12 @@ test_missing_target y WHERE udf(x.a) = udf(y.a) GROUP BY udf(x.b/2) ORDER BY udf(x.b/2) -- !query 35 schema -struct +struct -- !query 35 output -0 1 -1 5 -2 4 +0.5 1 +1.0 2 +1.5 3 +2.0 4 -- !query 36 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 13c2f9a810dcb..5269da1651e76 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 @@ -164,7 +164,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { DecimalData(BigDecimal("9"* 20 + ".123"), BigDecimal("9"* 20 + ".123")) :: Nil).toDF() Seq(true, false).foreach { ansiEnabled => - withSQLConf((SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, ansiEnabled.toString)) { + withSQLConf((SQLConf.ANSI_ENABLED.key, ansiEnabled.toString)) { val structDf = largeDecimals.select("a").agg(sum("a")) if (!ansiEnabled) { checkAnswer(structDf, Row(null)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala deleted file mode 100644 index 7056f483609a9..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala +++ /dev/null @@ -1,42 +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.sql - -import org.apache.spark.SparkConf -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession - -class PostgreSQLDialectQuerySuite extends QueryTest with SharedSparkSession { - - override def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) - - test("cast string to boolean") { - Seq("true", "tru", "tr", "t", " tRue ", " tRu ", "yes", "ye", - "y", "1", "on").foreach { input => - checkAnswer(sql(s"select cast('$input' as boolean)"), Row(true)) - } - Seq("false", "fals", "fal", "fa", "f", " fAlse ", " fAls ", "no", "n", - "0", "off", "of").foreach { input => - checkAnswer(sql(s"select cast('$input' as boolean)"), Row(false)) - } - - Seq("o", "abc", "").foreach { input => - intercept[IllegalArgumentException](sql(s"select cast('$input' as boolean)").collect()) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index e6dcf0b86308a..9169b3819f0a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -332,9 +332,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _: AnsiTest => - localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index b564bb2d24005..7d2963f3c21f0 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -117,14 +117,10 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } testCase match { - case _: PgSQLTest => - statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") - case _: AnsiTest => - statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") - statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") + case _: PgSQLTest | _: AnsiTest => + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") case _ => - statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") - statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = false") + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = false") } // Run the SQL queries preparing them for comparison. From 6103cf196081ab3e63713b623fe2ca3704420616 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 10 Dec 2019 09:51:29 -0800 Subject: [PATCH 0835/1387] [SPARK-30200][SQL] Add ExplainMode for Dataset.explain ### What changes were proposed in this pull request? This pr intends to add `ExplainMode` for explaining `Dataset/DataFrame` with a given format mode (`ExplainMode`). `ExplainMode` has four types along with the SQL EXPLAIN command: `Simple`, `Extended`, `Codegen`, `Cost`, and `Formatted`. For example, this pr enables users to explain DataFrame/Dataset with the `FORMATTED` format implemented in #24759; ``` scala> spark.range(10).groupBy("id").count().explain(ExplainMode.Formatted) == Physical Plan == * HashAggregate (3) +- * HashAggregate (2) +- * Range (1) (1) Range [codegen id : 1] Output: [id#0L] (2) HashAggregate [codegen id : 1] Input: [id#0L] (3) HashAggregate [codegen id : 1] Input: [id#0L, count#8L] ``` This comes from [the cloud-fan suggestion.](https://github.com/apache/spark/pull/24759#issuecomment-560211270) ### Why are the changes needed? To follow the SQL EXPLAIN command. ### Does this PR introduce any user-facing change? No, this is just for a new API in Dataset. ### How was this patch tested? Add tests in `ExplainSuite`. Closes #26829 from maropu/DatasetExplain. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/ExplainMode.java | 64 ++++++++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 40 ++++++++--- .../sql/execution/command/commands.scala | 7 +- .../org/apache/spark/sql/ExplainSuite.scala | 67 ++++++++++++++++--- 4 files changed, 156 insertions(+), 22 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java b/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java new file mode 100644 index 0000000000000..018ea12db55f1 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql; + +import org.apache.spark.annotation.Unstable; + +/** + * ExplainMode is used to specify the expected output format of plans (logical and physical) + * for debugging purpose. + * + * @since 3.0.0 + */ +@Unstable +public enum ExplainMode { + /** + * Simple mode means that when printing explain for a DataFrame, only a physical plan is + * expected to be printed to the console. + * + * @since 3.0.0 + */ + Simple, + /** + * Extended mode means that when printing explain for a DataFrame, both logical and physical + * plans are expected to be printed to the console. + * + * @since 3.0.0 + */ + Extended, + /** + * Extended mode means that when printing explain for a DataFrame, if generated codes are + * available, a physical plan and the generated codes are expected to be printed to the console. + * + * @since 3.0.0 + */ + Codegen, + /** + * Extended mode means that when printing explain for a DataFrame, if plan node statistics are + * available, a logical plan and the statistics are expected to be printed to the console. + * + * @since 3.0.0 + */ + Cost, + /** + * Formatted mode means that when printing explain for a DataFrame, explain output is + * expected to be split into two sections: a physical plan outline and node details. + * + * @since 3.0.0 + */ + Formatted +} 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 be3f2aed25936..aa641f0a3fa4e 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 @@ -522,36 +522,58 @@ class Dataset[T] private[sql]( // scalastyle:on println /** - * Prints the plans (logical and physical) to the console for debugging purposes. + * Prints the plans (logical and physical) with a format specified by a given explain mode. * * @group basic - * @since 1.6.0 + * @since 3.0.0 */ - def explain(extended: Boolean): Unit = { + def explain(mode: ExplainMode): Unit = { // Because temporary views are resolved during analysis when we create a Dataset, and // `ExplainCommand` analyzes input query plan and resolves temporary views again. Using // `ExplainCommand` here will probably output different query plans, compared to the results // of evaluation of the Dataset. So just output QueryExecution's query plans here. val qe = ExplainCommandUtil.explainedQueryExecution(sparkSession, logicalPlan, queryExecution) - val outputString = - if (extended) { - qe.toString - } else { + val outputString = mode match { + case ExplainMode.Simple => qe.simpleString - } + case ExplainMode.Extended => + qe.toString + case ExplainMode.Codegen => + try { + org.apache.spark.sql.execution.debug.codegenString(queryExecution.executedPlan) + } catch { + case e: AnalysisException => e.toString + } + case ExplainMode.Cost => + qe.stringWithStats + case ExplainMode.Formatted => + qe.simpleString(formatted = true) + } // scalastyle:off println println(outputString) // scalastyle:on println } + /** + * Prints the plans (logical and physical) to the console for debugging purposes. + * + * @group basic + * @since 1.6.0 + */ + def explain(extended: Boolean): Unit = if (extended) { + explain(ExplainMode.Extended) + } else { + explain(ExplainMode.Simple) + } + /** * Prints the physical plan to the console for debugging purposes. * * @group basic * @since 1.6.0 */ - def explain(): Unit = explain(extended = false) + def explain(): Unit = explain(ExplainMode.Simple) /** * Returns all column names and their data types as an array. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index bb641bcb430ea..888c4419458d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.execution.command import java.util.UUID import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} @@ -132,13 +131,15 @@ case class DataWritingCommandExec(cmd: DataWritingCommand, child: SparkPlan) * (but do NOT actually execute it). * * {{{ - * EXPLAIN (EXTENDED | CODEGEN) SELECT * FROM ... + * EXPLAIN (EXTENDED | CODEGEN | COST | FORMATTED) SELECT * FROM ... * }}} * * @param logicalPlan plan to explain * @param extended whether to do extended explain or not * @param codegen whether to output generated code from whole-stage codegen or not * @param cost whether to show cost information for operators. + * @param formatted whether to split explain output into two sections: a physical plan outline + * and node details. */ case class ExplainCommand( logicalPlan: LogicalPlan, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index f968fbb27d4f4..4e5d0f58be444 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -25,16 +25,19 @@ import org.apache.spark.sql.types.StructType class ExplainSuite extends QueryTest with SharedSparkSession { import testImplicits._ - /** - * Get the explain from a DataFrame and run the specified action on it. - */ - private def withNormalizedExplain(df: DataFrame, extended: Boolean)(f: String => Unit) = { + private def getNormalizedExplain(df: DataFrame, mode: ExplainMode): String = { val output = new java.io.ByteArrayOutputStream() Console.withOut(output) { - df.explain(extended = extended) + df.explain(mode) } - val normalizedOutput = output.toString.replaceAll("#\\d+", "#x") - f(normalizedOutput) + output.toString.replaceAll("#\\d+", "#x") + } + + /** + * Get the explain from a DataFrame and run the specified action on it. + */ + private def withNormalizedExplain(df: DataFrame, mode: ExplainMode)(f: String => Unit) = { + f(getNormalizedExplain(df, mode)) } /** @@ -53,14 +56,19 @@ class ExplainSuite extends QueryTest with SharedSparkSession { /** * Runs the plan and makes sure the plans contains all of the keywords. */ - private def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { - withNormalizedExplain(df, extended = true) { normalizedOutput => + private def checkKeywordsExistsInExplain( + df: DataFrame, mode: ExplainMode, keywords: String*): Unit = { + withNormalizedExplain(df, mode) { normalizedOutput => for (key <- keywords) { assert(normalizedOutput.contains(key)) } } } + private def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { + checkKeywordsExistsInExplain(df, ExplainMode.Extended, keywords: _*) + } + test("SPARK-23034 show rdd names in RDD scan nodes (Dataset)") { val rddWithName = spark.sparkContext.parallelize(Row(1, "abc") :: Nil).setName("testRdd") val df = spark.createDataFrame(rddWithName, StructType.fromDDL("c0 int, c1 string")) @@ -209,7 +217,7 @@ class ExplainSuite extends QueryTest with SharedSparkSession { test("SPARK-26659: explain of DataWritingCommandExec should not contain duplicate cmd.nodeName") { withTable("temptable") { val df = sql("create table temptable using parquet as select * from range(2)") - withNormalizedExplain(df, extended = false) { normalizedOutput => + withNormalizedExplain(df, ExplainMode.Simple) { normalizedOutput => assert("Create\\w*?TableAsSelectCommand".r.findAllMatchIn(normalizedOutput).length == 1) } } @@ -262,6 +270,45 @@ class ExplainSuite extends QueryTest with SharedSparkSession { } } } + + test("Support ExplainMode in Dataset.explain") { + val df1 = Seq((1, 2), (2, 3)).toDF("k", "v1") + val df2 = Seq((2, 3), (1, 1)).toDF("k", "v2") + val testDf = df1.join(df2, "k").groupBy("k").agg(count("v1"), sum("v1"), avg("v2")) + + val simpleExplainOutput = getNormalizedExplain(testDf, ExplainMode.Simple) + assert(simpleExplainOutput.startsWith("== Physical Plan ==")) + Seq("== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==").foreach { planType => + assert(!simpleExplainOutput.contains(planType)) + } + checkKeywordsExistsInExplain( + testDf, + ExplainMode.Extended, + "== Parsed Logical Plan ==" :: + "== Analyzed Logical Plan ==" :: + "== Optimized Logical Plan ==" :: + "== Physical Plan ==" :: + Nil: _*) + checkKeywordsExistsInExplain( + testDf, + ExplainMode.Cost, + "Statistics(sizeInBytes=" :: + Nil: _*) + checkKeywordsExistsInExplain( + testDf, + ExplainMode.Codegen, + "WholeStageCodegen subtrees" :: + "Generated code:" :: + Nil: _*) + checkKeywordsExistsInExplain( + testDf, + ExplainMode.Formatted, + "* LocalTableScan (1)" :: + "(1) LocalTableScan [codegen id :" :: + Nil: _*) + } } case class ExplainSingleData(id: Int) From 24c4ce1e6497a7ad80803babd9f11ee54607f7d1 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 10 Dec 2019 09:54:50 -0800 Subject: [PATCH 0836/1387] [SPARK-28351][SQL][FOLLOWUP] Remove 'DELETE FROM' from unsupportedHiveNativeCommands ### What changes were proposed in this pull request? Minor change, rm `DELETE FROM` from unsupported hive native operation, because it is supported in parser. ### Why are the changes needed? clear ambiguous ambiguous ### Does this PR introduce any user-facing change? no ### How was this patch tested? no Closes #26836 from yaooqinn/SPARK-28351. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index f894836d64233..cd9748eaa6f28 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -274,7 +274,6 @@ unsupportedHiveNativeCommands | kw1=COMMIT | kw1=ROLLBACK | kw1=DFS - | kw1=DELETE kw2=FROM ; createTableHeader From 8f0eb7dc868f59db6bee4f009bc148c09cf0df57 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 11 Dec 2019 02:22:08 +0800 Subject: [PATCH 0837/1387] [SPARK-29587][SQL] Support SQL Standard type real as float(4) numeric as decimal ### What changes were proposed in this pull request? The types decimal and numeric are equivalent. Both types are part of the SQL standard. the real type is 4 bytes, variable-precision, inexact, 6 decimal digits precision, same as our float, part of the SQL standard. ### Why are the changes needed? improve sql standard support other dbs https://www.postgresql.org/docs/9.3/datatype-numeric.html https://prestodb.io/docs/current/language/types.html#floating-point http://www.sqlservertutorial.net/sql-server-basics/sql-server-data-types/ MySQL treats REAL as a synonym for DOUBLE PRECISION (a nonstandard variation), unless the REAL_AS_FLOAT SQL mode is enabled. In MySQL, NUMERIC is implemented as DECIMAL, so the following remarks about DECIMAL apply equally to NUMERIC. ### Does this PR introduce any user-facing change? no ### How was this patch tested? add ut Closes #26537 from yaooqinn/SPARK-29587. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/AstBuilder.scala | 9 ++++--- .../sql-tests/inputs/show-create-table.sql | 5 ++++ .../results/show-create-table.sql.out | 27 ++++++++++++++++++- 3 files changed, 36 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 858870a161417..1beadc5e37801 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2179,7 +2179,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case ("smallint" | "short", Nil) => ShortType case ("int" | "integer", Nil) => IntegerType case ("bigint" | "long", Nil) => LongType - case ("float", Nil) => FloatType + case ("float" | "real", Nil) => FloatType case ("double", Nil) => DoubleType case ("date", Nil) => DateType case ("timestamp", Nil) => TimestampType @@ -2187,9 +2187,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case ("character" | "char", length :: Nil) => CharType(length.getText.toInt) case ("varchar", length :: Nil) => VarcharType(length.getText.toInt) case ("binary", Nil) => BinaryType - case ("decimal" | "dec", Nil) => DecimalType.USER_DEFAULT - case ("decimal" | "dec", precision :: Nil) => DecimalType(precision.getText.toInt, 0) - case ("decimal" | "dec", precision :: scale :: Nil) => + case ("decimal" | "dec" | "numeric", Nil) => DecimalType.USER_DEFAULT + case ("decimal" | "dec" | "numeric", precision :: Nil) => + DecimalType(precision.getText.toInt, 0) + case ("decimal" | "dec" | "numeric", precision :: scale :: Nil) => DecimalType(precision.getText.toInt, scale.getText.toInt) case ("interval", Nil) => CalendarIntervalType case (dt, params) => diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql b/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql index 852bfbd63847d..3118371af39b5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql @@ -59,3 +59,8 @@ TBLPROPERTIES ('a' = '1'); SHOW CREATE TABLE tbl; DROP TABLE tbl; + +-- float alias real and decimal alias numeric +CREATE TABLE tbl (a REAL, b NUMERIC, c NUMERIC(10), d NUMERIC(10,1)) USING parquet; +SHOW CREATE TABLE tbl; +DROP TABLE tbl; diff --git a/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out index 1faf16cc30509..fc1b62ca61801 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 24 +-- Number of queries: 27 -- !query 0 @@ -220,3 +220,28 @@ DROP TABLE tbl struct<> -- !query 23 output + + +-- !query 24 +CREATE TABLE tbl (a REAL, b NUMERIC, c NUMERIC(10), d NUMERIC(10,1)) USING parquet +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +SHOW CREATE TABLE tbl +-- !query 25 schema +struct +-- !query 25 output +CREATE TABLE `tbl` (`a` FLOAT, `b` DECIMAL(10,0), `c` DECIMAL(10,0), `d` DECIMAL(10,1)) +USING parquet + + +-- !query 26 +DROP TABLE tbl +-- !query 26 schema +struct<> +-- !query 26 output + From aec1d95f3b43a9bf349006ea5655d61fad740dd0 Mon Sep 17 00:00:00 2001 From: Karthikeyan Singaravelan Date: Tue, 10 Dec 2019 11:08:13 -0800 Subject: [PATCH 0838/1387] [SPARK-30205][PYSPARK] Import ABCs from collections.abc to remove deprecation warnings ### What changes were proposed in this pull request? This PR aims to remove deprecation warnings by importing ABCs from `collections.abc` instead of `collections`. - https://github.com/python/cpython/pull/10596 ### Why are the changes needed? This will remove deprecation warnings in Python 3.7 and 3.8. ``` $ python -V Python 3.7.5 $ python python/pyspark/resultiterable.py python/pyspark/resultiterable.py:23: DeprecationWarning: Using or importing the ABCs from 'collections' instead of from 'collections.abc' is deprecated since Python 3.3,and in 3.9 it will stop working class ResultIterable(collections.Iterable): ``` ### Does this PR introduce any user-facing change? No, this doesn't introduce user-facing change ### How was this patch tested? Manually because this is about deprecation warning messages. Closes #26835 from tirkarthi/spark-30205-fix-abc-warnings. Authored-by: Karthikeyan Singaravelan Signed-off-by: Dongjoon Hyun --- python/pyspark/resultiterable.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index 1ab5ce14c3531..c867b51877ffe 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -15,12 +15,16 @@ # limitations under the License. # -import collections +try: + from collections.abc import Iterable +except ImportError: + from collections import Iterable + __all__ = ["ResultIterable"] -class ResultIterable(collections.Iterable): +class ResultIterable(Iterable): """ A special result iterable. This is used because the standard From ad238a2238a9d0da89be4424574436cbfaee579d Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Tue, 10 Dec 2019 14:43:26 -0600 Subject: [PATCH 0839/1387] [SPARK-29976][CORE] Trigger speculation for stages with too few tasks ### What changes were proposed in this pull request? This PR add an optional spark conf for speculation to allow speculative runs for stages where there are only a few tasks. ``` spark.speculation.task.duration.threshold ``` If provided, tasks would be speculatively run if the TaskSet contains less tasks than the number of slots on a single executor and the task is taking longer time than the threshold. ### Why are the changes needed? This change helps avoid scenarios where there is single executor that could hang forever due to disk issue and we unfortunately assigned the single task in a TaskSet to that executor and cause the whole job to hang forever. ### Does this PR introduce any user-facing change? yes. If the new config `spark.speculation.task.duration.threshold` is provided and the TaskSet contains less tasks than the number of slots on a single executor and the task is taking longer time than the threshold, then speculative tasks would be submitted for the running tasks in the TaskSet. ### How was this patch tested? Unit tests are added to TaskSetManagerSuite. Closes #26614 from yuchenhuo/SPARK-29976. Authored-by: Yuchen Huo Signed-off-by: Thomas Graves --- .../spark/internal/config/package.scala | 12 +++ .../spark/scheduler/TaskSetManager.scala | 60 +++++++++--- .../spark/scheduler/TaskSetManagerSuite.scala | 98 +++++++++++++++++++ docs/configuration.md | 13 +++ 4 files changed, 167 insertions(+), 16 deletions(-) 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 25dc4c6efa26b..9d7b31aa30f0d 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 @@ -1499,6 +1499,18 @@ package object config { .doubleConf .createWithDefault(0.75) + private[spark] val SPECULATION_TASK_DURATION_THRESHOLD = + ConfigBuilder("spark.speculation.task.duration.threshold") + .doc("Task duration after which scheduler would try to speculative run the task. If " + + "provided, tasks would be speculatively run if current stage contains less tasks " + + "than or equal to the number of slots on a single executor and the task is taking " + + "longer time than the threshold. This config helps speculate stage with very few " + + "tasks. Regular speculation configs may also apply if the executor slots are " + + "large enough. E.g. tasks might be re-launched if there are enough successful runs " + + "even though the threshold hasn't been reached.") + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir") .doc("Staging directory used while submitting applications.") .stringConf diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 5c0bc497dd1b3..e026e90ee648e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -81,6 +81,13 @@ private[spark] class TaskSetManager( val speculationQuantile = conf.get(SPECULATION_QUANTILE) val speculationMultiplier = conf.get(SPECULATION_MULTIPLIER) val minFinishedForSpeculation = math.max((speculationQuantile * numTasks).floor.toInt, 1) + // User provided threshold for speculation regardless of whether the quantile has been reached + val speculationTaskDurationThresOpt = conf.get(SPECULATION_TASK_DURATION_THRESHOLD) + // SPARK-29976: Only when the total number of tasks in the stage is less than or equal to the + // number of slots on a single executor, would the task manager speculative run the tasks if + // their duration is longer than the given threshold. In this way, we wouldn't speculate too + // aggressively but still handle basic cases. + val speculationTasksLessEqToSlots = numTasks <= (conf.get(EXECUTOR_CORES) / sched.CPUS_PER_TASK) // For each task, tracks whether a copy of the task has succeeded. A task will also be // marked as "succeeded" if it failed with a fetch failure, in which case it should not @@ -957,15 +964,41 @@ private[spark] class TaskSetManager( recomputeLocality() } + /** + * Check if the task associated with the given tid has past the time threshold and should be + * speculative run. + */ + private def checkAndSubmitSpeculatableTask( + tid: Long, + currentTimeMillis: Long, + threshold: Double): Boolean = { + val info = taskInfos(tid) + val index = info.index + if (!successful(index) && copiesRunning(index) == 1 && + info.timeRunning(currentTimeMillis) > threshold && !speculatableTasks.contains(index)) { + addPendingTask(index, speculatable = true) + logInfo( + ("Marking task %d in stage %s (on %s) as speculatable because it ran more" + + " than %.0f ms(%d speculatable tasks in this taskset now)") + .format(index, taskSet.id, info.host, threshold, speculatableTasks.size + 1)) + speculatableTasks += index + sched.dagScheduler.speculativeTaskSubmitted(tasks(index)) + true + } else { + false + } + } + /** * Check for tasks to be speculated and return true if there are any. This is called periodically * by the TaskScheduler. * */ override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = { - // Can't speculate if we only have one task, and no need to speculate if the task set is a - // zombie or is from a barrier stage. - if (isZombie || isBarrier || numTasks == 1) { + // No need to speculate if the task set is zombie or is from a barrier stage. If there is only + // one task we don't speculate since we don't have metrics to decide whether it's taking too + // long or not, unless a task duration threshold is explicitly provided. + if (isZombie || isBarrier || (numTasks == 1 && !speculationTaskDurationThresOpt.isDefined)) { return false } var foundTasks = false @@ -983,19 +1016,14 @@ private[spark] class TaskSetManager( // bound based on that. logDebug("Task length threshold for speculation: " + threshold) for (tid <- runningTasksSet) { - val info = taskInfos(tid) - val index = info.index - if (!successful(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && - !speculatableTasks.contains(index)) { - addPendingTask(index, speculatable = true) - logInfo( - ("Marking task %d in stage %s (on %s) as speculatable because it ran more" + - " than %.0f ms(%d speculatable tasks in this taskset now)") - .format(index, taskSet.id, info.host, threshold, speculatableTasks.size + 1)) - speculatableTasks += index - sched.dagScheduler.speculativeTaskSubmitted(tasks(index)) - foundTasks = true - } + foundTasks |= checkAndSubmitSpeculatableTask(tid, time, threshold) + } + } else if (speculationTaskDurationThresOpt.isDefined && speculationTasksLessEqToSlots) { + val time = clock.getTimeMillis() + val threshold = speculationTaskDurationThresOpt.get + logDebug(s"Tasks taking longer time than provided speculation threshold: $threshold") + for (tid <- runningTasksSet) { + foundTasks |= checkAndSubmitSpeculatableTask(tid, time, threshold) } } foundTasks diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 34bcae8abd512..1d648320fc80c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1778,6 +1778,104 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) } + private def testSpeculationDurationSetup( + speculationThresholdOpt: Option[String], + speculationQuantile: Double, + numTasks: Int, + numSlots: Int): (TaskSetManager, ManualClock) = { + sc = new SparkContext("local", "test") + sc.conf.set(config.SPECULATION_ENABLED, true) + sc.conf.set(config.SPECULATION_QUANTILE.key, speculationQuantile.toString) + // Set the number of slots per executor + sc.conf.set(config.EXECUTOR_CORES.key, numSlots.toString) + sc.conf.set(config.CPUS_PER_TASK.key, "1") + if (speculationThresholdOpt.isDefined) { + sc.conf.set(config.SPECULATION_TASK_DURATION_THRESHOLD.key, speculationThresholdOpt.get) + } + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + // Create a task set with the given number of tasks + val taskSet = FakeTask.createTaskSet(numTasks) + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + manager.isZombie = false + + // Offer resources for the task to start + for (i <- 1 to numTasks) { + manager.resourceOffer(s"exec$i", s"host$i", NO_PREF) + } + (manager, clock) + } + + private def testSpeculationDurationThreshold( + speculationThresholdProvided: Boolean, + numTasks: Int, + numSlots: Int): Unit = { + val (manager, clock) = testSpeculationDurationSetup( + // Set the threshold to be 60 minutes + if (speculationThresholdProvided) Some("60min") else None, + // Set the quantile to be 1.0 so that regular speculation would not be triggered + 1.0, + numTasks, + numSlots + ) + + // if the time threshold has not been exceeded, no speculative run should be triggered + clock.advance(1000*60*60) + assert(!manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.size == 0) + + // Now the task should have been running for 60 minutes and 1 second + clock.advance(1) + if (speculationThresholdProvided && numSlots >= numTasks) { + assert(manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.size == numTasks) + // Should not submit duplicated tasks + assert(!manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.size == numTasks) + } else { + // If the feature flag is turned off, or the stage contains too many tasks + assert(!manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.size == 0) + } + } + + Seq(1, 2).foreach { numTasks => + test("SPARK-29976 when a speculation time threshold is provided, should speculative " + + s"run the task even if there are not enough successful runs, total tasks: $numTasks") { + testSpeculationDurationThreshold(true, numTasks, numTasks) + } + + test("SPARK-29976: when the speculation time threshold is not provided," + + s"don't speculative run if there are not enough successful runs, total tasks: $numTasks") { + testSpeculationDurationThreshold(false, numTasks, numTasks) + } + } + + test("SPARK-29976 when a speculation time threshold is provided, should not speculative " + + "if there are too many tasks in the stage even though time threshold is provided") { + testSpeculationDurationThreshold(true, 2, 1) + } + + test("SPARK-29976 Regular speculation configs should still take effect even when a " + + "threshold is provided") { + val (manager, clock) = testSpeculationDurationSetup( + Some("60min"), + speculationQuantile = 0.5, + numTasks = 2, + numSlots = 2 + ) + + // Task duration can't be 0, advance 1 sec + clock.advance(1000) + // Mark one of the task succeeded, which should satisfy the quantile + manager.handleSuccessfulTask(0, createTaskResult(0)) + // Advance 1 more second so the remaining task takes longer than medium but doesn't satisfy the + // duration threshold yet + clock.advance(1000) + assert(manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.size == 1) + } + test("TaskOutputFileAlreadyExistException lead to task set abortion") { sc = new SparkContext("local", "test") sched = new FakeTaskScheduler(sc, ("exec1", "host1")) diff --git a/docs/configuration.md b/docs/configuration.md index 8cd285cf83ca7..93758967a29d9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2033,6 +2033,19 @@ Apart from these, the following properties are also available, and may be useful Fraction of tasks which must be complete before speculation is enabled for a particular stage. + + spark.speculation.task.duration.threshold + None + + Task duration after which scheduler would try to speculative run the task. If provided, tasks + would be speculatively run if current stage contains less tasks than or equal to the number of + slots on a single executor and the task is taking longer time than the threshold. This config + helps speculate stage with very few tasks. Regular speculation configs may also apply if the + executor slots are large enough. E.g. tasks might be re-launched if there are enough successful + runs even though the threshold hasn't been reached. + Default unit is bytes, unless otherwise specified. + + spark.task.cpus 1 From cfd7ca9a06161f7622b5179a777f965c11892afa Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 10 Dec 2019 11:21:46 -0800 Subject: [PATCH 0840/1387] Revert "[SPARK-21869][SS] Apply Apache Commons Pool to Kafka producer" This reverts commit 3641c3dd69b2bd2beae028d52356450cc41f69ed. --- .../sql/kafka010/CachedKafkaProducer.scala | 118 ++++++---- .../kafka010/InternalKafkaConnectorPool.scala | 210 ------------------ .../kafka010/InternalKafkaConsumerPool.scala | 210 +++++++++++++++--- .../kafka010/InternalKafkaProducerPool.scala | 68 ------ .../sql/kafka010/KafkaDataConsumer.scala | 7 +- .../spark/sql/kafka010/KafkaDataWriter.scala | 34 +-- .../spark/sql/kafka010/KafkaWriteTask.scala | 20 +- .../apache/spark/sql/kafka010/package.scala | 34 +-- .../kafka010/CachedKafkaProducerSuite.scala | 154 ++++--------- ...a => InternalKafkaConsumerPoolSuite.scala} | 8 +- .../sql/kafka010/KafkaDataConsumerSuite.scala | 6 +- .../apache/spark/sql/kafka010/KafkaTest.scala | 10 +- .../kafka010/KafkaDataConsumerSuite.scala | 7 + 13 files changed, 332 insertions(+), 554 deletions(-) delete mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConnectorPool.scala delete mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaProducerPool.scala rename external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/{InternalKafkaConnectorPoolSuite.scala => InternalKafkaConsumerPoolSuite.scala} (96%) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala index 907440ab3731b..fc177cdc9037e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala @@ -18,68 +18,60 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.io.Closeable +import java.util.concurrent.{ConcurrentMap, ExecutionException, TimeUnit} +import com.google.common.cache._ +import com.google.common.util.concurrent.{ExecutionError, UncheckedExecutionException} +import org.apache.kafka.clients.producer.KafkaProducer import scala.collection.JavaConverters._ import scala.util.control.NonFatal -import org.apache.kafka.clients.producer.{Callback, KafkaProducer, ProducerRecord} - import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaRedactionUtil} -import org.apache.spark.sql.kafka010.InternalKafkaProducerPool._ -import org.apache.spark.util.ShutdownHookManager -private[kafka010] class CachedKafkaProducer(val kafkaParams: ju.Map[String, Object]) - extends Closeable with Logging { +private[kafka010] object CachedKafkaProducer extends Logging { private type Producer = KafkaProducer[Array[Byte], Array[Byte]] - private val producer = createProducer() + private val defaultCacheExpireTimeout = TimeUnit.MINUTES.toMillis(10) - private def createProducer(): Producer = { - val producer: Producer = new Producer(kafkaParams) - if (log.isDebugEnabled()) { - val redactedParamsSeq = KafkaRedactionUtil.redactParams(toCacheKey(kafkaParams)) - logDebug(s"Created a new instance of kafka producer for $redactedParamsSeq.") + private lazy val cacheExpireTimeout: Long = Option(SparkEnv.get) + .map(_.conf.get(PRODUCER_CACHE_TIMEOUT)) + .getOrElse(defaultCacheExpireTimeout) + + private val cacheLoader = new CacheLoader[Seq[(String, Object)], Producer] { + override def load(config: Seq[(String, Object)]): Producer = { + createKafkaProducer(config) } - producer } - override def close(): Unit = { - try { - if (log.isInfoEnabled()) { - val redactedParamsSeq = KafkaRedactionUtil.redactParams(toCacheKey(kafkaParams)) - logInfo(s"Closing the KafkaProducer with params: ${redactedParamsSeq.mkString("\n")}.") + private val removalListener = new RemovalListener[Seq[(String, Object)], Producer]() { + override def onRemoval( + notification: RemovalNotification[Seq[(String, Object)], Producer]): Unit = { + val paramsSeq: Seq[(String, Object)] = notification.getKey + val producer: Producer = notification.getValue + if (log.isDebugEnabled()) { + val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) + logDebug(s"Evicting kafka producer $producer params: $redactedParamsSeq, " + + s"due to ${notification.getCause}") } - producer.close() - } catch { - case NonFatal(e) => logWarning("Error while closing kafka producer.", e) + close(paramsSeq, producer) } } - def send(record: ProducerRecord[Array[Byte], Array[Byte]], callback: Callback): Unit = { - producer.send(record, callback) - } - - def flush(): Unit = { - producer.flush() - } -} + private lazy val guavaCache: LoadingCache[Seq[(String, Object)], Producer] = + CacheBuilder.newBuilder().expireAfterAccess(cacheExpireTimeout, TimeUnit.MILLISECONDS) + .removalListener(removalListener) + .build[Seq[(String, Object)], Producer](cacheLoader) -private[kafka010] object CachedKafkaProducer extends Logging { - - private val sparkConf = SparkEnv.get.conf - private val producerPool = new InternalKafkaProducerPool(sparkConf) - - ShutdownHookManager.addShutdownHook { () => - try { - producerPool.close() - } catch { - case e: Throwable => - logWarning("Ignoring exception while shutting down pool from shutdown hook", e) + private def createKafkaProducer(paramsSeq: Seq[(String, Object)]): Producer = { + val kafkaProducer: Producer = new Producer(paramsSeq.toMap.asJava) + if (log.isDebugEnabled()) { + val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) + logDebug(s"Created a new instance of KafkaProducer for $redactedParamsSeq.") } + kafkaProducer } /** @@ -87,20 +79,50 @@ private[kafka010] object CachedKafkaProducer extends Logging { * exist, a new KafkaProducer will be created. KafkaProducer is thread safe, it is best to keep * one instance per specified kafkaParams. */ - def acquire(kafkaParams: ju.Map[String, Object]): CachedKafkaProducer = { - val updatedKafkaParams = + private[kafka010] def getOrCreate(kafkaParams: ju.Map[String, Object]): Producer = { + val updatedKafkaProducerConfiguration = KafkaConfigUpdater("executor", kafkaParams.asScala.toMap) .setAuthenticationConfigIfNeeded() .build() - val key = toCacheKey(updatedKafkaParams) - producerPool.borrowObject(key, updatedKafkaParams) + val paramsSeq: Seq[(String, Object)] = paramsToSeq(updatedKafkaProducerConfiguration) + try { + guavaCache.get(paramsSeq) + } catch { + case e @ (_: ExecutionException | _: UncheckedExecutionException | _: ExecutionError) + if e.getCause != null => + throw e.getCause + } + } + + private def paramsToSeq(kafkaParams: ju.Map[String, Object]): Seq[(String, Object)] = { + val paramsSeq: Seq[(String, Object)] = kafkaParams.asScala.toSeq.sortBy(x => x._1) + paramsSeq + } + + /** For explicitly closing kafka producer */ + private[kafka010] def close(kafkaParams: ju.Map[String, Object]): Unit = { + val paramsSeq = paramsToSeq(kafkaParams) + guavaCache.invalidate(paramsSeq) } - def release(producer: CachedKafkaProducer): Unit = { - producerPool.returnObject(producer) + /** Auto close on cache evict */ + private def close(paramsSeq: Seq[(String, Object)], producer: Producer): Unit = { + try { + if (log.isInfoEnabled()) { + val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) + logInfo(s"Closing the KafkaProducer with params: ${redactedParamsSeq.mkString("\n")}.") + } + producer.close() + } catch { + case NonFatal(e) => logWarning("Error while closing kafka producer.", e) + } } private[kafka010] def clear(): Unit = { - producerPool.reset() + logInfo("Cleaning up guava cache.") + guavaCache.invalidateAll() } + + // Intended for testing purpose only. + private def getAsMap: ConcurrentMap[Seq[(String, Object)], Producer] = guavaCache.asMap() } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConnectorPool.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConnectorPool.scala deleted file mode 100644 index 0fb250e8d05ae..0000000000000 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConnectorPool.scala +++ /dev/null @@ -1,210 +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.sql.kafka010 - -import java.{util => ju} -import java.io.Closeable -import java.util.concurrent.ConcurrentHashMap - -import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener} -import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig} - -import org.apache.spark.internal.Logging - -/** - * Provides object pool for objects which is grouped by a key. - * - * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on - * the class, and same contract applies: after using the borrowed object, you must either call - * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object - * should be destroyed. - * - * The soft capacity of pool is determined by "poolConfig.capacity" config value, - * and the pool will have reasonable default value if the value is not provided. - * (The instance will do its best effort to respect soft capacity but it can exceed when there's - * a borrowing request and there's neither free space nor idle object to clear.) - * - * This class guarantees that no caller will get pooled object once the object is borrowed and - * not yet returned, hence provide thread-safety usage of non-thread-safe objects unless caller - * shares the object to multiple threads. - */ -private[kafka010] abstract class InternalKafkaConnectorPool[K, V <: Closeable]( - objectFactory: ObjectFactory[K, V], - poolConfig: PoolConfig[V], - swallowedExceptionListener: SwallowedExceptionListener) extends Logging { - - // the class is intended to have only soft capacity - assert(poolConfig.getMaxTotal < 0) - - private val pool = { - val internalPool = new GenericKeyedObjectPool[K, V](objectFactory, poolConfig) - internalPool.setSwallowedExceptionListener(swallowedExceptionListener) - internalPool - } - - /** - * Borrows object from the pool. If there's no idle object for the key, - * the pool will create the object. - * - * If the pool doesn't have idle object for the key and also exceeds the soft capacity, - * pool will try to clear some of idle objects. - * - * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise - * the object will be kept in pool as active object. - */ - def borrowObject(key: K, kafkaParams: ju.Map[String, Object]): V = { - updateKafkaParamForKey(key, kafkaParams) - - if (size >= poolConfig.softMaxSize) { - logWarning("Pool exceeds its soft max size, cleaning up idle objects...") - pool.clearOldest() - } - - pool.borrowObject(key) - } - - /** Returns borrowed object to the pool. */ - def returnObject(connector: V): Unit = { - pool.returnObject(createKey(connector), connector) - } - - /** Invalidates (destroy) borrowed object to the pool. */ - def invalidateObject(connector: V): Unit = { - pool.invalidateObject(createKey(connector), connector) - } - - /** Invalidates all idle values for the key */ - def invalidateKey(key: K): Unit = { - pool.clear(key) - } - - /** - * Closes the keyed object pool. Once the pool is closed, - * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject - * will continue to work, with returned objects destroyed on return. - * - * Also destroys idle instances in the pool. - */ - def close(): Unit = { - pool.close() - } - - def reset(): Unit = { - // this is the best-effort of clearing up. otherwise we should close the pool and create again - // but we don't want to make it "var" only because of tests. - pool.clear() - } - - def numIdle: Int = pool.getNumIdle - - def numIdle(key: K): Int = pool.getNumIdle(key) - - def numActive: Int = pool.getNumActive - - def numActive(key: K): Int = pool.getNumActive(key) - - def size: Int = numIdle + numActive - - def size(key: K): Int = numIdle(key) + numActive(key) - - private def updateKafkaParamForKey(key: K, kafkaParams: ju.Map[String, Object]): Unit = { - // We can assume that kafkaParam should not be different for same cache key, - // otherwise we can't reuse the cached object and cache key should contain kafkaParam. - // So it should be safe to put the key/value pair only when the key doesn't exist. - val oldKafkaParams = objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams) - require(oldKafkaParams == null || kafkaParams == oldKafkaParams, "Kafka parameters for same " + - s"cache key should be equal. old parameters: $oldKafkaParams new parameters: $kafkaParams") - } - - protected def createKey(connector: V): K -} - -private[kafka010] abstract class PoolConfig[V] extends GenericKeyedObjectPoolConfig[V] { - - init() - - def softMaxSize: Int - - def jmxEnabled: Boolean - - def minEvictableIdleTimeMillis: Long - - def evictorThreadRunIntervalMillis: Long - - def jmxNamePrefix: String - - def init(): Unit = { - // NOTE: Below lines define the behavior, so do not modify unless you know what you are - // doing, and update the class doc accordingly if necessary when you modify. - - // 1. Set min idle objects per key to 0 to avoid creating unnecessary object. - // 2. Set max idle objects per key to 3 but set total objects per key to infinite - // which ensures borrowing per key is not restricted. - // 3. Set max total objects to infinite which ensures all objects are managed in this pool. - setMinIdlePerKey(0) - setMaxIdlePerKey(3) - setMaxTotalPerKey(-1) - setMaxTotal(-1) - - // Set minimum evictable idle time which will be referred from evictor thread - setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis) - setSoftMinEvictableIdleTimeMillis(-1) - - // evictor thread will run test with ten idle objects - setTimeBetweenEvictionRunsMillis(evictorThreadRunIntervalMillis) - setNumTestsPerEvictionRun(10) - setEvictionPolicy(new DefaultEvictionPolicy[V]()) - - // Immediately fail on exhausted pool while borrowing - setBlockWhenExhausted(false) - - setJmxEnabled(jmxEnabled) - setJmxNamePrefix(jmxNamePrefix) - } -} - -private[kafka010] abstract class ObjectFactory[K, V <: Closeable] - extends BaseKeyedPooledObjectFactory[K, V] { - val keyToKafkaParams = new ConcurrentHashMap[K, ju.Map[String, Object]]() - - override def create(key: K): V = { - Option(keyToKafkaParams.get(key)) match { - case Some(kafkaParams) => createValue(key, kafkaParams) - case None => throw new IllegalStateException("Kafka params should be set before " + - "borrowing object.") - } - } - - override def wrap(value: V): PooledObject[V] = { - new DefaultPooledObject[V](value) - } - - override def destroyObject(key: K, p: PooledObject[V]): Unit = { - p.getObject.close() - } - - protected def createValue(key: K, kafkaParams: ju.Map[String, Object]): V -} - -private[kafka010] class CustomSwallowedExceptionListener(connectorType: String) - extends SwallowedExceptionListener with Logging { - - override def onSwallowException(e: Exception): Unit = { - logWarning(s"Error closing Kafka $connectorType", e) - } -} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala index a8e6045646451..276a942742b8e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPool.scala @@ -18,46 +18,204 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} +import java.util.concurrent.ConcurrentHashMap -import org.apache.commons.pool2.PooledObject +import org.apache.commons.pool2.{BaseKeyedPooledObjectFactory, PooledObject, SwallowedExceptionListener} +import org.apache.commons.pool2.impl.{DefaultEvictionPolicy, DefaultPooledObject, GenericKeyedObjectPool, GenericKeyedObjectPoolConfig} import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.sql.kafka010.InternalKafkaConsumerPool._ import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey -// TODO: revisit the relation between CacheKey and kafkaParams - for now it looks a bit weird -// as we force all consumers having same (groupId, topicPartition) to have same kafkaParams -// which might be viable in performance perspective (kafkaParams might be too huge to use -// as a part of key), but there might be the case kafkaParams could be different - -// cache key should be differentiated for both kafkaParams. +/** + * Provides object pool for [[InternalKafkaConsumer]] which is grouped by [[CacheKey]]. + * + * This class leverages [[GenericKeyedObjectPool]] internally, hence providing methods based on + * the class, and same contract applies: after using the borrowed object, you must either call + * returnObject() if the object is healthy to return to pool, or invalidateObject() if the object + * should be destroyed. + * + * The soft capacity of pool is determined by "spark.kafka.consumer.cache.capacity" config value, + * and the pool will have reasonable default value if the value is not provided. + * (The instance will do its best effort to respect soft capacity but it can exceed when there's + * a borrowing request and there's neither free space nor idle object to clear.) + * + * This class guarantees that no caller will get pooled object once the object is borrowed and + * not yet returned, hence provide thread-safety usage of non-thread-safe [[InternalKafkaConsumer]] + * unless caller shares the object to multiple threads. + */ private[kafka010] class InternalKafkaConsumerPool( - objectFactory: ConsumerObjectFactory, - poolConfig: ConsumerPoolConfig) - extends InternalKafkaConnectorPool[CacheKey, InternalKafkaConsumer]( - objectFactory, - poolConfig, - new CustomSwallowedExceptionListener("consumer")) { + objectFactory: ObjectFactory, + poolConfig: PoolConfig) extends Logging { def this(conf: SparkConf) = { - this(new ConsumerObjectFactory, new ConsumerPoolConfig(conf)) + this(new ObjectFactory, new PoolConfig(conf)) + } + + // the class is intended to have only soft capacity + assert(poolConfig.getMaxTotal < 0) + + private val pool = { + val internalPool = new GenericKeyedObjectPool[CacheKey, InternalKafkaConsumer]( + objectFactory, poolConfig) + internalPool.setSwallowedExceptionListener(CustomSwallowedExceptionListener) + internalPool + } + + /** + * Borrows [[InternalKafkaConsumer]] object from the pool. If there's no idle object for the key, + * the pool will create the [[InternalKafkaConsumer]] object. + * + * If the pool doesn't have idle object for the key and also exceeds the soft capacity, + * pool will try to clear some of idle objects. + * + * Borrowed object must be returned by either calling returnObject or invalidateObject, otherwise + * the object will be kept in pool as active object. + */ + def borrowObject(key: CacheKey, kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = { + updateKafkaParamForKey(key, kafkaParams) + + if (size >= poolConfig.softMaxSize) { + logWarning("Pool exceeds its soft max size, cleaning up idle objects...") + pool.clearOldest() + } + + pool.borrowObject(key) + } + + /** Returns borrowed object to the pool. */ + def returnObject(consumer: InternalKafkaConsumer): Unit = { + pool.returnObject(extractCacheKey(consumer), consumer) + } + + /** Invalidates (destroy) borrowed object to the pool. */ + def invalidateObject(consumer: InternalKafkaConsumer): Unit = { + pool.invalidateObject(extractCacheKey(consumer), consumer) + } + + /** Invalidates all idle consumers for the key */ + def invalidateKey(key: CacheKey): Unit = { + pool.clear(key) + } + + /** + * Closes the keyed object pool. Once the pool is closed, + * borrowObject will fail with [[IllegalStateException]], but returnObject and invalidateObject + * will continue to work, with returned objects destroyed on return. + * + * Also destroys idle instances in the pool. + */ + def close(): Unit = { + pool.close() + } + + def reset(): Unit = { + // this is the best-effort of clearing up. otherwise we should close the pool and create again + // but we don't want to make it "var" only because of tests. + pool.clear() } - override protected def createKey(consumer: InternalKafkaConsumer): CacheKey = { + def numIdle: Int = pool.getNumIdle + + def numIdle(key: CacheKey): Int = pool.getNumIdle(key) + + def numActive: Int = pool.getNumActive + + def numActive(key: CacheKey): Int = pool.getNumActive(key) + + def size: Int = numIdle + numActive + + def size(key: CacheKey): Int = numIdle(key) + numActive(key) + + // TODO: revisit the relation between CacheKey and kafkaParams - for now it looks a bit weird + // as we force all consumers having same (groupId, topicPartition) to have same kafkaParams + // which might be viable in performance perspective (kafkaParams might be too huge to use + // as a part of key), but there might be the case kafkaParams could be different - + // cache key should be differentiated for both kafkaParams. + private def updateKafkaParamForKey(key: CacheKey, kafkaParams: ju.Map[String, Object]): Unit = { + // We can assume that kafkaParam should not be different for same cache key, + // otherwise we can't reuse the cached object and cache key should contain kafkaParam. + // So it should be safe to put the key/value pair only when the key doesn't exist. + val oldKafkaParams = objectFactory.keyToKafkaParams.putIfAbsent(key, kafkaParams) + require(oldKafkaParams == null || kafkaParams == oldKafkaParams, "Kafka parameters for same " + + s"cache key should be equal. old parameters: $oldKafkaParams new parameters: $kafkaParams") + } + + private def extractCacheKey(consumer: InternalKafkaConsumer): CacheKey = { new CacheKey(consumer.topicPartition, consumer.kafkaParams) } } -private class ConsumerPoolConfig(conf: SparkConf) extends PoolConfig[InternalKafkaConsumer] { - def softMaxSize: Int = conf.get(CONSUMER_CACHE_CAPACITY) - def jmxEnabled: Boolean = conf.get(CONSUMER_CACHE_JMX_ENABLED) - def minEvictableIdleTimeMillis: Long = conf.get(CONSUMER_CACHE_TIMEOUT) - def evictorThreadRunIntervalMillis: Long = conf.get(CONSUMER_CACHE_EVICTOR_THREAD_RUN_INTERVAL) - def jmxNamePrefix: String = "kafka010-cached-simple-kafka-consumer-pool" -} +private[kafka010] object InternalKafkaConsumerPool { + object CustomSwallowedExceptionListener extends SwallowedExceptionListener with Logging { + override def onSwallowException(e: Exception): Unit = { + logError(s"Error closing Kafka consumer", e) + } + } + + class PoolConfig(conf: SparkConf) extends GenericKeyedObjectPoolConfig[InternalKafkaConsumer] { + private var _softMaxSize = Int.MaxValue + + def softMaxSize: Int = _softMaxSize + + init() + + def init(): Unit = { + _softMaxSize = conf.get(CONSUMER_CACHE_CAPACITY) + + val jmxEnabled = conf.get(CONSUMER_CACHE_JMX_ENABLED) + val minEvictableIdleTimeMillis = conf.get(CONSUMER_CACHE_TIMEOUT) + val evictorThreadRunIntervalMillis = conf.get( + CONSUMER_CACHE_EVICTOR_THREAD_RUN_INTERVAL) -private class ConsumerObjectFactory extends ObjectFactory[CacheKey, InternalKafkaConsumer] { - override protected def createValue( - key: CacheKey, - kafkaParams: ju.Map[String, Object]): InternalKafkaConsumer = { - new InternalKafkaConsumer(key.topicPartition, kafkaParams) + // NOTE: Below lines define the behavior, so do not modify unless you know what you are + // doing, and update the class doc accordingly if necessary when you modify. + + // 1. Set min idle objects per key to 0 to avoid creating unnecessary object. + // 2. Set max idle objects per key to 3 but set total objects per key to infinite + // which ensures borrowing per key is not restricted. + // 3. Set max total objects to infinite which ensures all objects are managed in this pool. + setMinIdlePerKey(0) + setMaxIdlePerKey(3) + setMaxTotalPerKey(-1) + setMaxTotal(-1) + + // Set minimum evictable idle time which will be referred from evictor thread + setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis) + setSoftMinEvictableIdleTimeMillis(-1) + + // evictor thread will run test with ten idle objects + setTimeBetweenEvictionRunsMillis(evictorThreadRunIntervalMillis) + setNumTestsPerEvictionRun(10) + setEvictionPolicy(new DefaultEvictionPolicy[InternalKafkaConsumer]()) + + // Immediately fail on exhausted pool while borrowing + setBlockWhenExhausted(false) + + setJmxEnabled(jmxEnabled) + setJmxNamePrefix("kafka010-cached-simple-kafka-consumer-pool") + } + } + + class ObjectFactory extends BaseKeyedPooledObjectFactory[CacheKey, InternalKafkaConsumer] { + val keyToKafkaParams = new ConcurrentHashMap[CacheKey, ju.Map[String, Object]]() + + override def create(key: CacheKey): InternalKafkaConsumer = { + Option(keyToKafkaParams.get(key)) match { + case Some(kafkaParams) => new InternalKafkaConsumer(key.topicPartition, kafkaParams) + case None => throw new IllegalStateException("Kafka params should be set before " + + "borrowing object.") + } + } + + override def wrap(value: InternalKafkaConsumer): PooledObject[InternalKafkaConsumer] = { + new DefaultPooledObject[InternalKafkaConsumer](value) + } + + override def destroyObject(key: CacheKey, p: PooledObject[InternalKafkaConsumer]): Unit = { + p.getObject.close() + } } } + diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaProducerPool.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaProducerPool.scala deleted file mode 100644 index 165b64313abb5..0000000000000 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaProducerPool.scala +++ /dev/null @@ -1,68 +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.sql.kafka010 - -import java.{util => ju} - -import scala.collection.JavaConverters._ - -import org.apache.commons.pool2.PooledObject - -import org.apache.spark.SparkConf -import org.apache.spark.sql.kafka010.InternalKafkaProducerPool.CacheKey - -private[kafka010] class InternalKafkaProducerPool( - objectFactory: ProducerObjectFactory, - poolConfig: ProducerPoolConfig) - extends InternalKafkaConnectorPool[CacheKey, CachedKafkaProducer]( - objectFactory, - poolConfig, - new CustomSwallowedExceptionListener("producer")) { - - def this(conf: SparkConf) = { - this(new ProducerObjectFactory, new ProducerPoolConfig(conf)) - } - - override protected def createKey(producer: CachedKafkaProducer): CacheKey = { - InternalKafkaProducerPool.toCacheKey(producer.kafkaParams) - } -} - -private class ProducerPoolConfig(conf: SparkConf) extends PoolConfig[CachedKafkaProducer] { - def softMaxSize: Int = conf.get(PRODUCER_CACHE_CAPACITY) - def jmxEnabled: Boolean = conf.get(PRODUCER_CACHE_JMX_ENABLED) - def minEvictableIdleTimeMillis: Long = conf.get(PRODUCER_CACHE_TIMEOUT) - def evictorThreadRunIntervalMillis: Long = conf.get(PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL) - def jmxNamePrefix: String = "kafka010-cached-simple-kafka-producer-pool" -} - -private class ProducerObjectFactory extends ObjectFactory[CacheKey, CachedKafkaProducer] { - override protected def createValue( - key: CacheKey, - kafkaParams: ju.Map[String, Object]): CachedKafkaProducer = { - new CachedKafkaProducer(kafkaParams) - } -} - -private[kafka010] object InternalKafkaProducerPool { - type CacheKey = Seq[(String, Object)] - - def toCacheKey(params: ju.Map[String, Object]): CacheKey = { - params.asScala.toSeq.sortBy(x => x._1) - } -} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala index f2dad945719e6..ca82c908f441b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala @@ -613,7 +613,7 @@ private[kafka010] object KafkaDataConsumer extends Logging { consumerPool.close() } catch { case e: Throwable => - logWarning("Ignoring exception while shutting down pools from shutdown hook", e) + logWarning("Ignoring Exception while shutting down pools from shutdown hook", e) } } @@ -639,11 +639,6 @@ private[kafka010] object KafkaDataConsumer extends Logging { new KafkaDataConsumer(topicPartition, kafkaParams, consumerPool, fetchedDataPool) } - private[kafka010] def clear(): Unit = { - consumerPool.reset() - fetchedDataPool.reset() - } - private def reportDataLoss0( failOnDataLoss: Boolean, finalMessage: String, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala index 870ed7aef4deb..3f8d3d2da5797 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala @@ -44,7 +44,7 @@ private[kafka010] class KafkaDataWriter( inputSchema: Seq[Attribute]) extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] { - private var producer = CachedKafkaProducer.acquire(producerParams) + private lazy val producer = CachedKafkaProducer.getOrCreate(producerParams) def write(row: InternalRow): Unit = { checkForErrors() @@ -55,36 +55,20 @@ private[kafka010] class KafkaDataWriter( // Send is asynchronous, but we can't commit until all rows are actually in Kafka. // This requires flushing and then checking that no callbacks produced errors. // We also check for errors before to fail as soon as possible - the check is cheap. - try { - checkForErrors() - producer.flush() - checkForErrors() - } finally { - releaseProducer() - } + checkForErrors() + producer.flush() + checkForErrors() KafkaDataWriterCommitMessage } - def abort(): Unit = { - close() - } + def abort(): Unit = {} def close(): Unit = { - try { - checkForErrors() - if (producer != null) { - producer.flush() - checkForErrors() - } - } finally { - releaseProducer() - } - } - - private def releaseProducer(): Unit = { + checkForErrors() if (producer != null) { - CachedKafkaProducer.release(producer) - producer = null + producer.flush() + checkForErrors() + CachedKafkaProducer.close(producerParams) } } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index cfe3e1642dc5e..8b907065af1d0 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -39,13 +39,13 @@ private[kafka010] class KafkaWriteTask( inputSchema: Seq[Attribute], topic: Option[String]) extends KafkaRowWriter(inputSchema, topic) { // used to synchronize with Kafka callbacks - private var producer: CachedKafkaProducer = _ + private var producer: KafkaProducer[Array[Byte], Array[Byte]] = _ /** * Writes key value data out to topics. */ def execute(iterator: Iterator[InternalRow]): Unit = { - producer = CachedKafkaProducer.acquire(producerConfiguration) + producer = CachedKafkaProducer.getOrCreate(producerConfiguration) while (iterator.hasNext && failedWrite == null) { val currentRow = iterator.next() sendRow(currentRow, producer) @@ -53,17 +53,11 @@ private[kafka010] class KafkaWriteTask( } def close(): Unit = { - try { + checkForErrors() + if (producer != null) { + producer.flush() checkForErrors() - if (producer != null) { - producer.flush() - checkForErrors() - } - } finally { - if (producer != null) { - CachedKafkaProducer.release(producer) - producer = null - } + producer = null } } } @@ -89,7 +83,7 @@ private[kafka010] abstract class KafkaRowWriter( * assuming the row is in Kafka. */ protected def sendRow( - row: InternalRow, producer: CachedKafkaProducer): Unit = { + row: InternalRow, producer: KafkaProducer[Array[Byte], Array[Byte]]): Unit = { val projectedRow = projection(row) val topic = projectedRow.getUTF8String(0) val key = projectedRow.getBinary(1) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala index f103b5b69b583..6f6ae55fc4971 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala @@ -26,6 +26,12 @@ package object kafka010 { // scalastyle:ignore // ^^ scalastyle:ignore is for ignoring warnings about digits in package name type PartitionOffsetMap = Map[TopicPartition, Long] + private[kafka010] val PRODUCER_CACHE_TIMEOUT = + ConfigBuilder("spark.kafka.producer.cache.timeout") + .doc("The expire time to remove the unused producers.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("10m") + private[kafka010] val CONSUMER_CACHE_CAPACITY = ConfigBuilder("spark.kafka.consumer.cache.capacity") .doc("The maximum number of consumers cached. Please note it's a soft limit" + @@ -68,32 +74,4 @@ package object kafka010 { // scalastyle:ignore "When non-positive, no idle evictor thread will be run.") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("1m") - - private[kafka010] val PRODUCER_CACHE_CAPACITY = - ConfigBuilder("spark.kafka.producer.cache.capacity") - .doc("The maximum number of producers cached. Please note it's a soft limit" + - " (check Structured Streaming Kafka integration guide for further details).") - .intConf - .createWithDefault(64) - - private[kafka010] val PRODUCER_CACHE_JMX_ENABLED = - ConfigBuilder("spark.kafka.producer.cache.jmx.enable") - .doc("Enable or disable JMX for pools created with this configuration instance.") - .booleanConf - .createWithDefault(false) - - private[kafka010] val PRODUCER_CACHE_TIMEOUT = - ConfigBuilder("spark.kafka.producer.cache.timeout") - .doc("The minimum amount of time a producer may sit idle in the pool before " + - "it is eligible for eviction by the evictor. " + - "When non-positive, no producers will be evicted from the pool due to idle time alone.") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("5m") - - private[kafka010] val PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL = - ConfigBuilder("spark.kafka.producer.cache.evictorThreadRunInterval") - .doc("The interval of time between runs of the idle evictor thread for producer pool. " + - "When non-positive, no idle evictor thread will be run.") - .timeConf(TimeUnit.MILLISECONDS) - .createWithDefaultString("1m") } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala index 4506a4029d88d..7425a74315e1a 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala @@ -17,133 +17,61 @@ package org.apache.spark.sql.kafka010 -import java.util.concurrent.{Executors, TimeUnit} +import java.{util => ju} +import java.util.concurrent.ConcurrentMap -import scala.collection.JavaConverters._ -import scala.util.Random - -import org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -import org.apache.kafka.clients.producer.{Callback, ProducerRecord, RecordMetadata} -import org.apache.kafka.clients.producer.ProducerConfig.{KEY_SERIALIZER_CLASS_CONFIG, VALUE_SERIALIZER_CLASS_CONFIG} +import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.serialization.ByteArraySerializer import org.scalatest.PrivateMethodTester -import org.apache.spark.{TaskContext, TaskContextImpl} -import org.apache.spark.sql.kafka010.InternalKafkaProducerPool._ import org.apache.spark.sql.test.SharedSparkSession class CachedKafkaProducerSuite extends SharedSparkSession with PrivateMethodTester with KafkaTest { - private var testUtils: KafkaTestUtils = _ - private val topic = "topic" + Random.nextInt() - private var producerPool: InternalKafkaProducerPool = _ - - override def beforeAll(): Unit = { - super.beforeAll() - testUtils = new KafkaTestUtils(Map[String, Object]()) - testUtils.setup() - } - - override def afterAll(): Unit = { - if (testUtils != null) { - testUtils.teardown() - testUtils = null - } - super.afterAll() - } + type KP = KafkaProducer[Array[Byte], Array[Byte]] - override def beforeEach(): Unit = { + protected override def beforeEach(): Unit = { super.beforeEach() - - producerPool = { - val internalKafkaConsumerPoolMethod = PrivateMethod[InternalKafkaProducerPool]('producerPool) - CachedKafkaProducer.invokePrivate(internalKafkaConsumerPoolMethod()) - } - - producerPool.reset() + CachedKafkaProducer.clear() } - private def getKafkaParams(acks: Int = 0) = Map[String, Object]( - "acks" -> acks.toString, + test("Should return the cached instance on calling getOrCreate with same params.") { + val kafkaParams = new ju.HashMap[String, Object]() + kafkaParams.put("acks", "0") // Here only host should be resolvable, it does not need a running instance of kafka server. - BOOTSTRAP_SERVERS_CONFIG -> testUtils.brokerAddress, - KEY_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName, - VALUE_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName - ).asJava - - test("acquire should return the cached instance with same params") { - val kafkaParams = getKafkaParams() - - val producer1 = CachedKafkaProducer.acquire(kafkaParams) - CachedKafkaProducer.release(producer1) - val producer2 = CachedKafkaProducer.acquire(kafkaParams) - CachedKafkaProducer.release(producer2) - - assert(producer1 === producer2) - assert(producerPool.size(toCacheKey(kafkaParams)) === 1) + kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") + kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) + kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) + val producer = CachedKafkaProducer.getOrCreate(kafkaParams) + val producer2 = CachedKafkaProducer.getOrCreate(kafkaParams) + assert(producer == producer2) + + val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]](Symbol("getAsMap")) + val map = CachedKafkaProducer.invokePrivate(cacheMap()) + assert(map.size == 1) } - test("acquire should return a new instance with different params") { - val kafkaParams1 = getKafkaParams() - val kafkaParams2 = getKafkaParams(1) - - val producer1 = CachedKafkaProducer.acquire(kafkaParams1) - CachedKafkaProducer.release(producer1) - val producer2 = CachedKafkaProducer.acquire(kafkaParams2) - CachedKafkaProducer.release(producer2) - - assert(producer1 !== producer2) - assert(producerPool.size(toCacheKey(kafkaParams1)) === 1) - assert(producerPool.size(toCacheKey(kafkaParams2)) === 1) - } - - test("Concurrent use of CachedKafkaProducer") { - val data = (1 to 1000).map(_.toString) - testUtils.createTopic(topic, 1) - - val kafkaParams = getKafkaParams() - val numThreads = 100 - val numProducerUsages = 500 - - @volatile var error: Throwable = null - - val callback = new Callback() { - override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { - if (error == null && e != null) { - error = e - } - } - } - - def produce(): Unit = { - val taskContext = if (Random.nextBoolean) { - new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), null, null, null) - } else { - null - } - TaskContext.setTaskContext(taskContext) - val producer = CachedKafkaProducer.acquire(kafkaParams) - try { - data.foreach { d => - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, null, d.getBytes) - producer.send(record, callback) - } - } finally { - CachedKafkaProducer.release(producer) - } - } - - val threadpool = Executors.newFixedThreadPool(numThreads) - try { - val futures = (1 to numProducerUsages).map { i => - threadpool.submit(new Runnable { - override def run(): Unit = { produce() } - }) - } - futures.foreach(_.get(1, TimeUnit.MINUTES)) - assert(error == null) - } finally { - threadpool.shutdown() - } + test("Should close the correct kafka producer for the given kafkaPrams.") { + val kafkaParams = new ju.HashMap[String, Object]() + kafkaParams.put("acks", "0") + kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") + kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) + kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) + val producer: KP = CachedKafkaProducer.getOrCreate(kafkaParams) + kafkaParams.put("acks", "1") + val producer2: KP = CachedKafkaProducer.getOrCreate(kafkaParams) + // With updated conf, a new producer instance should be created. + assert(producer != producer2) + + val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]](Symbol("getAsMap")) + val map = CachedKafkaProducer.invokePrivate(cacheMap()) + assert(map.size == 2) + + CachedKafkaProducer.close(kafkaParams) + val map2 = CachedKafkaProducer.invokePrivate(cacheMap()) + assert(map2.size == 1) + import scala.collection.JavaConverters._ + val (seq: Seq[(String, Object)], _producer: KP) = map2.asScala.toArray.apply(0) + assert(_producer == producer) } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/InternalKafkaConnectorPoolSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPoolSuite.scala similarity index 96% rename from external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/InternalKafkaConnectorPoolSuite.scala rename to external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPoolSuite.scala index 3143429abd711..78d7feef58519 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/InternalKafkaConnectorPoolSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/InternalKafkaConsumerPoolSuite.scala @@ -29,13 +29,7 @@ import org.apache.spark.SparkConf import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey import org.apache.spark.sql.test.SharedSparkSession -/* - * There are multiple implementations of [[InternalKafkaConnectorPool]] but they don't differ - * significantly. Because of that only [[InternalKafkaConsumerPool]] used to test all the - * functionality. If the behavior of implementations starts to differ it worth to add further - * tests but for now it would be mainly copy-paste. - */ -class InternalKafkaConnectorPoolSuite extends SharedSparkSession { +class InternalKafkaConsumerPoolSuite extends SharedSparkSession { test("basic multiple borrows and returns for single key") { val pool = new InternalKafkaConsumerPool(new SparkConf()) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala index 6e1f10e7f6d74..d22955180d050 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDataConsumerSuite.scala @@ -195,7 +195,7 @@ class KafkaDataConsumerSuite @volatile var error: Throwable = null - def consume(): Unit = { + def consume(i: Int): Unit = { val taskContext = if (Random.nextBoolean) { new TaskContextImpl(0, 0, 0, 0, attemptNumber = Random.nextInt(2), null, null, null) } else { @@ -233,9 +233,9 @@ class KafkaDataConsumerSuite val threadpool = Executors.newFixedThreadPool(numThreads) try { - val futures = (1 to numConsumerUsages).map { _ => + val futures = (1 to numConsumerUsages).map { i => threadpool.submit(new Runnable { - override def run(): Unit = { consume() } + override def run(): Unit = { consume(i) } }) } futures.foreach(_.get(1, TimeUnit.MINUTES)) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTest.scala index 2900322b947be..19acda95c707c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTest.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTest.scala @@ -21,16 +21,12 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite -/** A trait to clean cached Kafka connector in `afterAll` */ +/** A trait to clean cached Kafka producers in `afterAll` */ trait KafkaTest extends BeforeAndAfterAll { self: SparkFunSuite => override def afterAll(): Unit = { - try { - KafkaDataConsumer.clear() - CachedKafkaProducer.clear() - } finally { - super.afterAll() - } + super.afterAll() + CachedKafkaProducer.clear() } } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala index 246672bcbacfc..82913cf416a5f 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala @@ -121,6 +121,8 @@ class KafkaDataConsumerSuite extends SparkFunSuite with MockitoSugar with Before val numThreads = 100 val numConsumerUsages = 500 + @volatile var error: Throwable = null + def consume(i: Int): Unit = { val useCache = Random.nextBoolean val taskContext = if (Random.nextBoolean) { @@ -136,6 +138,10 @@ class KafkaDataConsumerSuite extends SparkFunSuite with MockitoSugar with Before new String(bytes) } assert(rcvd == data) + } catch { + case e: Throwable => + error = e + throw e } finally { consumer.release() } @@ -149,6 +155,7 @@ class KafkaDataConsumerSuite extends SparkFunSuite with MockitoSugar with Before }) } futures.foreach(_.get(1, TimeUnit.MINUTES)) + assert(error == null) } finally { threadPool.shutdown() } From d7843dde0f82551f0481885feb15acd63dd554c0 Mon Sep 17 00:00:00 2001 From: root1 Date: Tue, 10 Dec 2019 14:23:51 -0800 Subject: [PATCH 0841/1387] [SPARK-29152][CORE] Executor Plugin shutdown when dynamic allocation is enabled ### What changes were proposed in this pull request? Added `shutdownHook` for shutdown method of executor plugin. This will ensure that shutdown method will be called always. ### Why are the changes needed? Whenever executors are not going down gracefully, i.e getting killed due to idle time or getting killed forcefully, shutdown method of executors plugin is not getting called. Shutdown method can be used to release any resources that plugin has acquired during its initialisation. So its important to make sure that every time a executor goes down shutdown method of plugin gets called. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Tested Manually. Closes #26810 from iRakson/Executor_Plugin. Authored-by: root1 Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/executor/Executor.scala | 47 +++++++++++-------- 1 file changed, 27 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 6eb1525a0db5a..0ea16d0dcb159 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -24,6 +24,7 @@ import java.net.{URI, URL} import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent._ +import java.util.concurrent.atomic.AtomicBoolean import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ @@ -65,6 +66,10 @@ private[spark] class Executor( logInfo(s"Starting executor ID $executorId on host $executorHostname") + private val executorShutdown = new AtomicBoolean(false) + ShutdownHookManager.addShutdownHook( + () => stop() + ) // Application dependencies (added through SparkContext) that we've fetched so far on this node. // Each map holds the master's timestamp for the version of that file or JAR we got. private val currentFiles: HashMap[String, Long] = new HashMap[String, Long]() @@ -258,27 +263,29 @@ private[spark] class Executor( } def stop(): Unit = { - env.metricsSystem.report() - try { - metricsPoller.stop() - } catch { - case NonFatal(e) => - logWarning("Unable to stop executor metrics poller", e) - } - try { - heartbeater.stop() - } catch { - case NonFatal(e) => - logWarning("Unable to stop heartbeater", e) - } - threadPool.shutdown() + if (!executorShutdown.getAndSet(true)) { + env.metricsSystem.report() + try { + metricsPoller.stop() + } catch { + case NonFatal(e) => + logWarning("Unable to stop executor metrics poller", e) + } + try { + heartbeater.stop() + } catch { + case NonFatal(e) => + logWarning("Unable to stop heartbeater", e) + } + threadPool.shutdown() - // Notify plugins that executor is shutting down so they can terminate cleanly - Utils.withContextClassLoader(replClassLoader) { - plugins.foreach(_.shutdown()) - } - if (!isLocal) { - env.stop() + // Notify plugins that executor is shutting down so they can terminate cleanly + Utils.withContextClassLoader(replClassLoader) { + plugins.foreach(_.shutdown()) + } + if (!isLocal) { + env.stop() + } } } From 3cc55f6a0a560782f6e20296ac716ef68a412d26 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 10 Dec 2019 19:41:24 -0800 Subject: [PATCH 0842/1387] [SPARK-29392][CORE][SQL][FOLLOWUP] More removal of 'foo Symbol syntax for Scala 2.13 ### What changes were proposed in this pull request? Another continuation of https://github.com/apache/spark/pull/26748 ### Why are the changes needed? To cleanly cross compile with Scala 2.13. ### Does this PR introduce any user-facing change? None. ### How was this patch tested? Existing tests Closes #26842 from srowen/SPARK-29392.4. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/DSLHintSuite.scala | 6 +- .../ExpressionTypeCheckingSuite.scala | 193 ++++++++++-------- .../sql/DataFrameWindowFramesSuite.scala | 4 +- .../sql/DataFrameWindowFunctionsSuite.scala | 22 +- .../apache/spark/sql/DatasetCacheSuite.scala | 18 +- .../org/apache/spark/sql/DatasetSuite.scala | 14 +- .../sql/DynamicPartitionPruningSuite.scala | 5 +- .../spark/sql/GeneratorFunctionSuite.scala | 37 ++-- .../org/apache/spark/sql/JoinHintSuite.scala | 22 +- 9 files changed, 168 insertions(+), 153 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala index 388eb238db0c0..c316e0406b1b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala @@ -22,9 +22,9 @@ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical._ class DSLHintSuite extends AnalysisTest { - lazy val a = 'a.int - lazy val b = 'b.string - lazy val c = 'c.string + lazy val a = Symbol("a").int + lazy val b = Symbol("b").string + lazy val c = Symbol("c").string lazy val r1 = LocalRelation(a, b, c) test("various hint parameters") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index c83759e8f4c14..f944b4ad87e40 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -29,12 +29,12 @@ import org.apache.spark.sql.types._ class ExpressionTypeCheckingSuite extends SparkFunSuite { val testRelation = LocalRelation( - 'intField.int, - 'stringField.string, - 'booleanField.boolean, - 'decimalField.decimal(8, 0), - 'arrayField.array(StringType), - 'mapField.map(StringType, LongType)) + Symbol("intField").int, + Symbol("stringField").string, + Symbol("booleanField").boolean, + Symbol("decimalField").decimal(8, 0), + Symbol("arrayField").array(StringType), + Symbol("mapField").map(StringType, LongType)) def assertError(expr: Expression, errorMessage: String): Unit = { val e = intercept[AnalysisException] { @@ -56,83 +56,92 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { } test("check types for unary arithmetic") { - assertError(BitwiseNot('stringField), "requires integral type") + assertError(BitwiseNot(Symbol("stringField")), "requires integral type") } test("check types for binary arithmetic") { // We will cast String to Double for binary arithmetic - assertSuccess(Add('intField, 'stringField)) - assertSuccess(Subtract('intField, 'stringField)) - assertSuccess(Multiply('intField, 'stringField)) - assertSuccess(Divide('intField, 'stringField)) - assertSuccess(Remainder('intField, 'stringField)) - // checkAnalysis(BitwiseAnd('intField, 'stringField)) - - assertErrorForDifferingTypes(Add('intField, 'booleanField)) - assertErrorForDifferingTypes(Subtract('intField, 'booleanField)) - assertErrorForDifferingTypes(Multiply('intField, 'booleanField)) - assertErrorForDifferingTypes(Divide('intField, 'booleanField)) - assertErrorForDifferingTypes(Remainder('intField, 'booleanField)) - assertErrorForDifferingTypes(BitwiseAnd('intField, 'booleanField)) - assertErrorForDifferingTypes(BitwiseOr('intField, 'booleanField)) - assertErrorForDifferingTypes(BitwiseXor('intField, 'booleanField)) - - assertError(Add('booleanField, 'booleanField), "requires (numeric or interval) type") - assertError(Subtract('booleanField, 'booleanField), + assertSuccess(Add(Symbol("intField"), Symbol("stringField"))) + assertSuccess(Subtract(Symbol("intField"), Symbol("stringField"))) + assertSuccess(Multiply(Symbol("intField"), Symbol("stringField"))) + assertSuccess(Divide(Symbol("intField"), Symbol("stringField"))) + assertSuccess(Remainder(Symbol("intField"), Symbol("stringField"))) + // checkAnalysis(BitwiseAnd(Symbol("intField"), Symbol("stringField"))) + + assertErrorForDifferingTypes(Add(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(Subtract(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(Multiply(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(Divide(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(Remainder(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(BitwiseAnd(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(BitwiseOr(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(BitwiseXor(Symbol("intField"), Symbol("booleanField"))) + + assertError(Add(Symbol("booleanField"), Symbol("booleanField")), "requires (numeric or interval) type") - assertError(Multiply('booleanField, 'booleanField), "requires numeric type") - assertError(Divide('booleanField, 'booleanField), "requires (double or decimal) type") - assertError(Remainder('booleanField, 'booleanField), "requires numeric type") - - assertError(BitwiseAnd('booleanField, 'booleanField), "requires integral type") - assertError(BitwiseOr('booleanField, 'booleanField), "requires integral type") - assertError(BitwiseXor('booleanField, 'booleanField), "requires integral type") + assertError(Subtract(Symbol("booleanField"), Symbol("booleanField")), + "requires (numeric or interval) type") + assertError(Multiply(Symbol("booleanField"), Symbol("booleanField")), "requires numeric type") + assertError(Divide(Symbol("booleanField"), Symbol("booleanField")), + "requires (double or decimal) type") + assertError(Remainder(Symbol("booleanField"), Symbol("booleanField")), "requires numeric type") + + assertError(BitwiseAnd(Symbol("booleanField"), Symbol("booleanField")), + "requires integral type") + assertError(BitwiseOr(Symbol("booleanField"), Symbol("booleanField")), "requires integral type") + assertError(BitwiseXor(Symbol("booleanField"), Symbol("booleanField")), + "requires integral type") } test("check types for predicates") { // We will cast String to Double for binary comparison - assertSuccess(EqualTo('intField, 'stringField)) - assertSuccess(EqualNullSafe('intField, 'stringField)) - assertSuccess(LessThan('intField, 'stringField)) - assertSuccess(LessThanOrEqual('intField, 'stringField)) - assertSuccess(GreaterThan('intField, 'stringField)) - assertSuccess(GreaterThanOrEqual('intField, 'stringField)) + assertSuccess(EqualTo(Symbol("intField"), Symbol("stringField"))) + assertSuccess(EqualNullSafe(Symbol("intField"), Symbol("stringField"))) + assertSuccess(LessThan(Symbol("intField"), Symbol("stringField"))) + assertSuccess(LessThanOrEqual(Symbol("intField"), Symbol("stringField"))) + assertSuccess(GreaterThan(Symbol("intField"), Symbol("stringField"))) + assertSuccess(GreaterThanOrEqual(Symbol("intField"), Symbol("stringField"))) // We will transform EqualTo with numeric and boolean types to CaseKeyWhen - assertSuccess(EqualTo('intField, 'booleanField)) - assertSuccess(EqualNullSafe('intField, 'booleanField)) - - assertErrorForDifferingTypes(EqualTo('intField, 'mapField)) - assertErrorForDifferingTypes(EqualNullSafe('intField, 'mapField)) - assertErrorForDifferingTypes(LessThan('intField, 'booleanField)) - assertErrorForDifferingTypes(LessThanOrEqual('intField, 'booleanField)) - assertErrorForDifferingTypes(GreaterThan('intField, 'booleanField)) - assertErrorForDifferingTypes(GreaterThanOrEqual('intField, 'booleanField)) - - assertError(EqualTo('mapField, 'mapField), "EqualTo does not support ordering on type map") - assertError(EqualNullSafe('mapField, 'mapField), + assertSuccess(EqualTo(Symbol("intField"), Symbol("booleanField"))) + assertSuccess(EqualNullSafe(Symbol("intField"), Symbol("booleanField"))) + + assertErrorForDifferingTypes(EqualTo(Symbol("intField"), Symbol("mapField"))) + assertErrorForDifferingTypes(EqualNullSafe(Symbol("intField"), Symbol("mapField"))) + assertErrorForDifferingTypes(LessThan(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(LessThanOrEqual(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(GreaterThan(Symbol("intField"), Symbol("booleanField"))) + assertErrorForDifferingTypes(GreaterThanOrEqual(Symbol("intField"), Symbol("booleanField"))) + + assertError(EqualTo(Symbol("mapField"), Symbol("mapField")), + "EqualTo does not support ordering on type map") + assertError(EqualNullSafe(Symbol("mapField"), Symbol("mapField")), "EqualNullSafe does not support ordering on type map") - assertError(LessThan('mapField, 'mapField), + assertError(LessThan(Symbol("mapField"), Symbol("mapField")), "LessThan does not support ordering on type map") - assertError(LessThanOrEqual('mapField, 'mapField), + assertError(LessThanOrEqual(Symbol("mapField"), Symbol("mapField")), "LessThanOrEqual does not support ordering on type map") - assertError(GreaterThan('mapField, 'mapField), + assertError(GreaterThan(Symbol("mapField"), Symbol("mapField")), "GreaterThan does not support ordering on type map") - assertError(GreaterThanOrEqual('mapField, 'mapField), + assertError(GreaterThanOrEqual(Symbol("mapField"), Symbol("mapField")), "GreaterThanOrEqual does not support ordering on type map") - assertError(If('intField, 'stringField, 'stringField), + assertError(If(Symbol("intField"), Symbol("stringField"), Symbol("stringField")), "type of predicate expression in If should be boolean") - assertErrorForDifferingTypes(If('booleanField, 'intField, 'booleanField)) + assertErrorForDifferingTypes( + If(Symbol("booleanField"), Symbol("intField"), Symbol("booleanField"))) assertError( - CaseWhen(Seq(('booleanField.attr, 'intField.attr), ('booleanField.attr, 'mapField.attr))), + CaseWhen(Seq((Symbol("booleanField").attr, Symbol("intField").attr), + (Symbol("booleanField").attr, Symbol("mapField").attr))), "THEN and ELSE expressions should all be same type or coercible to a common type") assertError( - CaseKeyWhen('intField, Seq('intField, 'stringField, 'intField, 'mapField)), + CaseKeyWhen(Symbol("intField"), Seq(Symbol("intField"), Symbol("stringField"), + Symbol("intField"), Symbol("mapField"))), "THEN and ELSE expressions should all be same type or coercible to a common type") assertError( - CaseWhen(Seq(('booleanField.attr, 'intField.attr), ('intField.attr, 'intField.attr))), + CaseWhen(Seq((Symbol("booleanField").attr, Symbol("intField").attr), + (Symbol("intField").attr, Symbol("intField").attr))), "WHEN expressions in CaseWhen should all be boolean type") } @@ -141,29 +150,29 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { // instead of from AggregateExpression, which is the wrapper of an AggregateFunction. // We will cast String to Double for sum and average - assertSuccess(Sum('stringField)) - assertSuccess(Average('stringField)) - assertSuccess(Min('arrayField)) - assertSuccess(new BoolAnd("bool_and", 'booleanField)) - assertSuccess(new BoolOr("bool_or", 'booleanField)) - - assertError(Min('mapField), "min does not support ordering on type") - assertError(Max('mapField), "max does not support ordering on type") - assertError(Sum('booleanField), "requires (numeric or interval) type") - assertError(Average('booleanField), "requires (numeric or interval) type") + assertSuccess(Sum(Symbol("stringField"))) + assertSuccess(Average(Symbol("stringField"))) + assertSuccess(Min(Symbol("arrayField"))) + assertSuccess(new BoolAnd("bool_and", Symbol("booleanField"))) + assertSuccess(new BoolOr("bool_or", Symbol("booleanField"))) + + assertError(Min(Symbol("mapField")), "min does not support ordering on type") + assertError(Max(Symbol("mapField")), "max does not support ordering on type") + assertError(Sum(Symbol("booleanField")), "requires (numeric or interval) type") + assertError(Average(Symbol("booleanField")), "requires (numeric or interval) type") } test("check types for others") { - assertError(CreateArray(Seq('intField, 'booleanField)), + assertError(CreateArray(Seq(Symbol("intField"), Symbol("booleanField"))), "input to function array should all be the same type") - assertError(Coalesce(Seq('intField, 'booleanField)), + assertError(Coalesce(Seq(Symbol("intField"), Symbol("booleanField"))), "input to function coalesce should all be the same type") assertError(Coalesce(Nil), "function coalesce requires at least one argument") assertError(new Murmur3Hash(Nil), "function hash requires at least one argument") assertError(new XxHash64(Nil), "function xxhash64 requires at least one argument") - assertError(Explode('intField), + assertError(Explode(Symbol("intField")), "input to function explode should be array or map type") - assertError(PosExplode('intField), + assertError(PosExplode(Symbol("intField")), "input to function explode should be array or map type") } @@ -174,7 +183,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { CreateNamedStruct(Seq(1, "a", "b", 2.0)), "Only foldable string expressions are allowed to appear at odd position") assertError( - CreateNamedStruct(Seq('a.string.at(0), "a", "b", 2.0)), + CreateNamedStruct(Seq(Symbol("a").string.at(0), "a", "b", 2.0)), "Only foldable string expressions are allowed to appear at odd position") assertError( CreateNamedStruct(Seq(Literal.create(null, StringType), "a")), @@ -184,36 +193,42 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { test("check types for CreateMap") { assertError(CreateMap(Seq("a", "b", 2.0)), "even number of arguments") assertError( - CreateMap(Seq('intField, 'stringField, 'booleanField, 'stringField)), + CreateMap(Seq(Symbol("intField"), Symbol("stringField"), + Symbol("booleanField"), Symbol("stringField"))), "keys of function map should all be the same type") assertError( - CreateMap(Seq('stringField, 'intField, 'stringField, 'booleanField)), + CreateMap(Seq(Symbol("stringField"), Symbol("intField"), + Symbol("stringField"), Symbol("booleanField"))), "values of function map should all be the same type") } test("check types for ROUND/BROUND") { assertSuccess(Round(Literal(null), Literal(null))) - assertSuccess(Round('intField, Literal(1))) + assertSuccess(Round(Symbol("intField"), Literal(1))) - assertError(Round('intField, 'intField), "Only foldable Expression is allowed") - assertError(Round('intField, 'booleanField), "requires int type") - assertError(Round('intField, 'mapField), "requires int type") - assertError(Round('booleanField, 'intField), "requires numeric type") + assertError(Round(Symbol("intField"), Symbol("intField")), + "Only foldable Expression is allowed") + assertError(Round(Symbol("intField"), Symbol("booleanField")), "requires int type") + assertError(Round(Symbol("intField"), Symbol("mapField")), "requires int type") + assertError(Round(Symbol("booleanField"), Symbol("intField")), "requires numeric type") assertSuccess(BRound(Literal(null), Literal(null))) - assertSuccess(BRound('intField, Literal(1))) + assertSuccess(BRound(Symbol("intField"), Literal(1))) - assertError(BRound('intField, 'intField), "Only foldable Expression is allowed") - assertError(BRound('intField, 'booleanField), "requires int type") - assertError(BRound('intField, 'mapField), "requires int type") - assertError(BRound('booleanField, 'intField), "requires numeric type") + assertError(BRound(Symbol("intField"), Symbol("intField")), + "Only foldable Expression is allowed") + assertError(BRound(Symbol("intField"), Symbol("booleanField")), "requires int type") + assertError(BRound(Symbol("intField"), Symbol("mapField")), "requires int type") + assertError(BRound(Symbol("booleanField"), Symbol("intField")), "requires numeric type") } test("check types for Greatest/Least") { for (operator <- Seq[(Seq[Expression] => Expression)](Greatest, Least)) { - assertError(operator(Seq('booleanField)), "requires at least two arguments") - assertError(operator(Seq('intField, 'stringField)), "should all have the same type") - assertError(operator(Seq('mapField, 'mapField)), "does not support ordering") + assertError(operator(Seq(Symbol("booleanField"))), "requires at least two arguments") + assertError(operator(Seq(Symbol("intField"), Symbol("stringField"))), + "should all have the same type") + assertError(operator(Seq(Symbol("mapField"), Symbol("mapField"))), + "does not support ordering") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala index fbd399917e390..8c998290b5044 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala @@ -111,7 +111,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { checkAnswer( df.select( - 'key, + $"key", first("value").over( window.rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), first("value").over( @@ -226,7 +226,7 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession { checkAnswer( df.select( - 'key, + $"key", sum("value").over(window. rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), sum("value").over(window. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 835630bff7099..696b056a682b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -633,20 +633,20 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSparkSession { assert(thrownException.message.contains("window functions inside WHERE and HAVING clauses")) } - checkAnalysisError(testData2.select('a).where(rank().over(Window.orderBy('b)) === 1)) - checkAnalysisError(testData2.where('b === 2 && rank().over(Window.orderBy('b)) === 1)) + checkAnalysisError(testData2.select("a").where(rank().over(Window.orderBy($"b")) === 1)) + checkAnalysisError(testData2.where($"b" === 2 && rank().over(Window.orderBy($"b")) === 1)) checkAnalysisError( - testData2.groupBy('a) - .agg(avg('b).as("avgb")) - .where('a > 'avgb && rank().over(Window.orderBy('a)) === 1)) + testData2.groupBy($"a") + .agg(avg($"b").as("avgb")) + .where($"a" > $"avgb" && rank().over(Window.orderBy($"a")) === 1)) checkAnalysisError( - testData2.groupBy('a) - .agg(max('b).as("maxb"), sum('b).as("sumb")) - .where(rank().over(Window.orderBy('a)) === 1)) + testData2.groupBy($"a") + .agg(max($"b").as("maxb"), sum($"b").as("sumb")) + .where(rank().over(Window.orderBy($"a")) === 1)) checkAnalysisError( - testData2.groupBy('a) - .agg(max('b).as("maxb"), sum('b).as("sumb")) - .where('sumb === 5 && rank().over(Window.orderBy('a)) === 1)) + testData2.groupBy($"a") + .agg(max($"b").as("maxb"), sum($"b").as("sumb")) + .where($"sumb" === 5 && rank().over(Window.orderBy($"a")) === 1)) checkAnalysisError(sql("SELECT a FROM testData2 WHERE RANK() OVER(ORDER BY b) = 1")) checkAnalysisError(sql("SELECT * FROM testData2 WHERE b = 2 AND RANK() OVER(ORDER BY b) = 1")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 33d9def0b44e5..5dba9aa5b1fee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -158,8 +158,8 @@ class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimit test("SPARK-24596 Non-cascading Cache Invalidation") { val df = Seq(("a", 1), ("b", 2)).toDF("s", "i") - val df2 = df.filter('i > 1) - val df3 = df.filter('i < 2) + val df2 = df.filter($"i" > 1) + val df3 = df.filter($"i" < 2) df2.cache() df.cache() @@ -178,8 +178,8 @@ class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimit val expensiveUDF = udf({ x: Int => Thread.sleep(5000); x }) val df = spark.range(0, 5).toDF("a") val df1 = df.withColumn("b", expensiveUDF($"a")) - val df2 = df1.groupBy('a).agg(sum('b)) - val df3 = df.agg(sum('a)) + val df2 = df1.groupBy($"a").agg(sum($"b")) + val df3 = df.agg(sum($"a")) df1.cache() df2.cache() @@ -192,16 +192,16 @@ class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimit // df1 un-cached; df2's cache plan stays the same assert(df1.storageLevel == StorageLevel.NONE) - assertCacheDependency(df1.groupBy('a).agg(sum('b))) + assertCacheDependency(df1.groupBy($"a").agg(sum($"b"))) - val df4 = df1.groupBy('a).agg(sum('b)).agg(sum("sum(b)")) + val df4 = df1.groupBy($"a").agg(sum($"b")).agg(sum("sum(b)")) assertCached(df4) // reuse loaded cache failAfter(3.seconds) { checkDataset(df4, Row(10)) } - val df5 = df.agg(sum('a)).filter($"sum(a)" > 1) + val df5 = df.agg(sum($"a")).filter($"sum(a)" > 1) assertCached(df5) // first time use, load cache checkDataset(df5, Row(10)) @@ -209,8 +209,8 @@ class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimit test("SPARK-26708 Cache data and cached plan should stay consistent") { val df = spark.range(0, 5).toDF("a") - val df1 = df.withColumn("b", 'a + 1) - val df2 = df.filter('a > 1) + val df1 = df.withColumn("b", $"a" + 1) + val df2 = df.filter($"a" > 1) df.cache() // Add df1 to the CacheManager; the buffer is currently empty. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 9a367d48d2144..91d4a1c3dc73e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -980,7 +980,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { } test("SPARK-14554: Dataset.map may generate wrong java code for wide table") { - val wideDF = spark.range(10).select(Seq.tabulate(1000) {i => ('id + i).as(s"c$i")} : _*) + val wideDF = spark.range(10).select(Seq.tabulate(1000) {i => ($"id" + i).as(s"c$i")} : _*) // Make sure the generated code for this plan can compile and execute. checkDataset(wideDF.map(_.getLong(0)), 0L until 10 : _*) } @@ -1390,7 +1390,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { } testCheckpointing("basic") { - val ds = spark.range(10).repartition('id % 2).filter('id > 5).orderBy('id.desc) + val ds = spark.range(10).repartition($"id" % 2).filter($"id" > 5).orderBy($"id".desc) val cp = if (reliable) ds.checkpoint(eager) else ds.localCheckpoint(eager) val logicalRDD = cp.logicalPlan match { @@ -1425,10 +1425,10 @@ class DatasetSuite extends QueryTest with SharedSparkSession { } testCheckpointing("should preserve partitioning information") { - val ds = spark.range(10).repartition('id % 2) + val ds = spark.range(10).repartition($"id" % 2) val cp = if (reliable) ds.checkpoint(eager) else ds.localCheckpoint(eager) - val agg = cp.groupBy('id % 2).agg(count('id)) + val agg = cp.groupBy($"id" % 2).agg(count($"id")) agg.queryExecution.executedPlan.collectFirst { case ShuffleExchangeExec(_, _: RDDScanExec, _) => @@ -1440,7 +1440,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { ) } - checkAnswer(agg, ds.groupBy('id % 2).agg(count('id))) + checkAnswer(agg, ds.groupBy($"id" % 2).agg(count($"id"))) } } } @@ -1538,11 +1538,9 @@ class DatasetSuite extends QueryTest with SharedSparkSession { checkAnswer(df.sort("id"), expected) checkAnswer(df.sort(col("id")), expected) checkAnswer(df.sort($"id"), expected) - checkAnswer(df.sort('id), expected) checkAnswer(df.orderBy("id"), expected) checkAnswer(df.orderBy(col("id")), expected) checkAnswer(df.orderBy($"id"), expected) - checkAnswer(df.orderBy('id), expected) } test("SPARK-21567: Dataset should work with type alias") { @@ -1697,7 +1695,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { test("SPARK-24571: filtering of string values by char literal") { val df = Seq("Amsterdam", "San Francisco", "X").toDF("city") - checkAnswer(df.where('city === 'X'), Seq(Row("X"))) + checkAnswer(df.where($"city" === 'X'), Seq(Row("X"))) checkAnswer( df.where($"city".contains(java.lang.Character.valueOf('A'))), Seq(Row("Amsterdam"))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index e0ea36157e56a..a54528f376d1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -97,7 +97,8 @@ class DynamicPartitionPruningSuite (6, 60) ) - spark.range(1000).select('id as 'product_id, ('id % 10) as 'store_id, ('id + 1) as 'code) + spark.range(1000) + .select($"id" as "product_id", ($"id" % 10) as "store_id", ($"id" + 1) as "code") .write .format(tableFormat) .mode("overwrite") @@ -1037,7 +1038,7 @@ class DynamicPartitionPruningSuite test("no partition pruning when the build side is a stream") { withTable("fact") { val input = MemoryStream[Int] - val stream = input.toDF.select('value as "one", ('value * 3) as "code") + val stream = input.toDF.select($"value" as "one", ($"value" * 3) as "code") spark.range(100).select( $"id", ($"id" + 1).as("one"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala index 4edce3b0811e0..96a0eb3e32e9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -88,28 +88,28 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { test("single explode") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") checkAnswer( - df.select(explode('intList)), + df.select(explode($"intList")), Row(1) :: Row(2) :: Row(3) :: Nil) } test("single explode_outer") { val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") checkAnswer( - df.select(explode_outer('intList)), + df.select(explode_outer($"intList")), Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) } test("single posexplode") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") checkAnswer( - df.select(posexplode('intList)), + df.select(posexplode($"intList")), Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Nil) } test("single posexplode_outer") { val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") checkAnswer( - df.select(posexplode_outer('intList)), + df.select(posexplode_outer($"intList")), Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Row(null, null) :: Nil) } @@ -117,13 +117,13 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") checkAnswer( - df.select($"a", explode('intList)), + df.select($"a", explode($"intList")), Row(1, 1) :: Row(1, 2) :: Row(1, 3) :: Nil) checkAnswer( - df.select($"*", explode('intList)), + df.select($"*", explode($"intList")), Row(1, Seq(1, 2, 3), 1) :: Row(1, Seq(1, 2, 3), 2) :: Row(1, Seq(1, 2, 3), 3) :: Nil) @@ -133,7 +133,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") checkAnswer( - df.select($"a", explode_outer('intList)), + df.select($"a", explode_outer($"intList")), Row(1, 1) :: Row(1, 2) :: Row(1, 3) :: @@ -141,7 +141,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { Nil) checkAnswer( - df.select($"*", explode_outer('intList)), + df.select($"*", explode_outer($"intList")), Row(1, Seq(1, 2, 3), 1) :: Row(1, Seq(1, 2, 3), 2) :: Row(1, Seq(1, 2, 3), 3) :: @@ -153,11 +153,11 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") checkAnswer( - df.select(explode('intList).as('int)).select('int), + df.select(explode($"intList").as("int")).select($"int"), Row(1) :: Row(2) :: Row(3) :: Nil) checkAnswer( - df.select(explode('intList).as('int)).select(sum('int)), + df.select(explode($"intList").as("int")).select(sum($"int")), Row(6) :: Nil) } @@ -165,11 +165,11 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") checkAnswer( - df.select(explode_outer('intList).as('int)).select('int), + df.select(explode_outer($"intList").as("int")).select($"int"), Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) checkAnswer( - df.select(explode('intList).as('int)).select(sum('int)), + df.select(explode($"intList").as("int")).select(sum($"int")), Row(6) :: Nil) } @@ -177,7 +177,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") checkAnswer( - df.select(explode('map)), + df.select(explode($"map")), Row("a", "b")) } @@ -186,7 +186,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { (3, Map("c" -> "d"))).toDF("a", "map") checkAnswer( - df.select(explode_outer('map)), + df.select(explode_outer($"map")), Row("a", "b") :: Row(null, null) :: Row("c", "d") :: Nil) } @@ -194,7 +194,7 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") checkAnswer( - df.select(explode('map).as("key1" :: "value1" :: Nil)).select("key1", "value1"), + df.select(explode($"map").as("key1" :: "value1" :: Nil)).select("key1", "value1"), Row("a", "b")) } @@ -202,13 +202,13 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { val df = Seq((3, None), (1, Some(Map("a" -> "b")))).toDF("a", "map") checkAnswer( - df.select(explode_outer('map).as("key1" :: "value1" :: Nil)).select("key1", "value1"), + df.select(explode_outer($"map").as("key1" :: "value1" :: Nil)).select("key1", "value1"), Row("a", "b") :: Row(null, null) :: Nil) } test("self join explode") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - val exploded = df.select(explode('intList).as('i)) + val exploded = df.select(explode($"intList").as("i")) checkAnswer( exploded.join(exploded, exploded("i") === exploded("i")).agg(count("*")), @@ -277,7 +277,8 @@ class GeneratorFunctionSuite extends QueryTest with SharedSparkSession { test("inline_outer") { val df = Seq((1, "2"), (3, "4"), (5, "6")).toDF("col1", "col2") - val df2 = df.select(when('col1 === 1, null).otherwise(array(struct('col1, 'col2))).as("col1")) + val df2 = df.select( + when($"col1" === 1, null).otherwise(array(struct($"col1", $"col2"))).as("col1")) checkAnswer( df2.selectExpr("inline(col1)"), Row(3, "4") :: Row(5, "6") :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index 6b154253e6e6c..e405864584d07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -99,7 +99,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { test("multiple joins") { verifyJoinHint( - df1.join(df2.hint("broadcast").join(df3, 'b1 === 'c1).hint("broadcast"), 'a1 === 'c1), + df1.join(df2.hint("broadcast").join(df3, $"b1" === $"c1").hint("broadcast"), $"a1" === $"c1"), JoinHint( None, Some(HintInfo(strategy = Some(BROADCAST)))) :: @@ -108,7 +108,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { None) :: Nil ) verifyJoinHint( - df1.hint("broadcast").join(df2, 'a1 === 'b1).hint("broadcast").join(df3, 'a1 === 'c1), + df1.hint("broadcast").join(df2, $"a1" === $"b1").hint("broadcast").join(df3, $"a1" === $"c1"), JoinHint( Some(HintInfo(strategy = Some(BROADCAST))), None) :: @@ -180,8 +180,8 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { ) verifyJoinHint( - df1.join(df2, 'a1 === 'b1 && 'a1 > 5).hint("broadcast") - .join(df3, 'b1 === 'c1 && 'a1 < 10), + df1.join(df2, $"a1" === $"b1" && $"a1" > 5).hint("broadcast") + .join(df3, $"b1" === $"c1" && $"a1" < 10), JoinHint( Some(HintInfo(strategy = Some(BROADCAST))), None) :: @@ -189,9 +189,9 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { ) verifyJoinHint( - df1.join(df2, 'a1 === 'b1 && 'a1 > 5).hint("broadcast") - .join(df3, 'b1 === 'c1 && 'a1 < 10) - .join(df, 'b1 === 'id), + df1.join(df2, $"a1" === $"b1" && $"a1" > 5).hint("broadcast") + .join(df3, $"b1" === $"c1" && $"a1" < 10) + .join(df, $"b1" === $"id"), JoinHint.NONE :: JoinHint( Some(HintInfo(strategy = Some(BROADCAST))), @@ -222,7 +222,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { test("hint merge") { verifyJoinHintWithWarnings( - df.hint("broadcast").filter('id > 2).hint("broadcast").join(df, "id"), + df.hint("broadcast").filter($"id" > 2).hint("broadcast").join(df, "id"), JoinHint( Some(HintInfo(strategy = Some(BROADCAST))), None) :: Nil, @@ -236,7 +236,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { Nil ) verifyJoinHintWithWarnings( - df.hint("merge").filter('id > 2).hint("shuffle_hash").join(df, "id").hint("broadcast"), + df.hint("merge").filter($"id" > 2).hint("shuffle_hash").join(df, "id").hint("broadcast"), JoinHint( Some(HintInfo(strategy = Some(SHUFFLE_HASH))), None) :: Nil, @@ -312,13 +312,13 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { test("nested hint") { verifyJoinHint( - df.hint("broadcast").hint("broadcast").filter('id > 2).join(df, "id"), + df.hint("broadcast").hint("broadcast").filter($"id" > 2).join(df, "id"), JoinHint( Some(HintInfo(strategy = Some(BROADCAST))), None) :: Nil ) verifyJoinHint( - df.hint("shuffle_hash").hint("broadcast").hint("merge").filter('id > 2).join(df, "id"), + df.hint("shuffle_hash").hint("broadcast").hint("merge").filter($"id" > 2).join(df, "id"), JoinHint( Some(HintInfo(strategy = Some(SHUFFLE_MERGE))), None) :: Nil From eb509968a72831c5bcab510b9b49ff5f3a48a4bb Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 10 Dec 2019 23:30:12 -0800 Subject: [PATCH 0843/1387] [SPARK-30211][INFRA] Use python3 in make-distribution.sh ### What changes were proposed in this pull request? This PR switches python to python3 in `make-distribution.sh`. ### Why are the changes needed? SPARK-29672 changed this - https://github.com/apache/spark/pull/26330/files#diff-8cf6167d58ce775a08acafcfe6f40966 ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #26844 from wangyum/SPARK-30211. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- dev/make-distribution.sh | 2 +- docs/building-spark.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 16fbb71be429e..8ba93aa56f00a 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -233,7 +233,7 @@ if [ "$MAKE_PIP" == "true" ]; then pushd "$SPARK_HOME/python" > /dev/null # Delete the egg info file if it exists, this can cache older setup files. rm -rf pyspark.egg-info || echo "No existing egg info file, skipping deletion" - python setup.py sdist + python3 setup.py sdist popd > /dev/null else echo "Skipping building python distribution package" diff --git a/docs/building-spark.md b/docs/building-spark.md index 898f904461a6d..891d6225bbdd5 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -66,7 +66,7 @@ with Maven profile settings and so on like the direct Maven build. Example: ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes -This will build Spark distribution along with Python pip and R packages. For more information on usage, run `./dev/make-distribution.sh --help` +This will build Spark distribution along with Python pip and R packages. (Note that build with Python pip package requires Python 3.6). For more information on usage, run `./dev/make-distribution.sh --help` ## Specifying the Hadoop Version and Enabling YARN From beae14d5ed4c6f2f81949b852f990fc8b801b3e4 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 11 Dec 2019 16:56:42 +0800 Subject: [PATCH 0844/1387] [SPARK-30104][SQL] Fix catalog resolution for 'global_temp' ### What changes were proposed in this pull request? `global_temp` is used as a database name to access global temp views. The current catalog lookup logic considers only the first element of multi-part name when it resolves a catalog. This results in using the session catalog even `global_temp` is used as a table name under v2 catalog. This PR addresses this by making sure multi-part name has two elements before using the session catalog. ### Why are the changes needed? Currently, 'global_temp' can be used as a table name in certain commands (CREATE) but not in others (DESCRIBE): ``` // Assume "spark.sql.globalTempDatabase" is set to "global_temp". sql(s"CREATE TABLE testcat.t (id bigint, data string) USING foo") sql(s"CREATE TABLE testcat.global_temp (id bigint, data string) USING foo") sql("USE testcat") sql(s"DESCRIBE TABLE t").show +---------------+---------+-------+ | col_name|data_type|comment| +---------------+---------+-------+ | id| bigint| | | data| string| | | | | | | # Partitioning| | | |Not partitioned| | | +---------------+---------+-------+ sql(s"DESCRIBE TABLE global_temp").show org.apache.spark.sql.AnalysisException: Table not found: global_temp;; 'DescribeTable 'UnresolvedV2Relation [global_temp], org.apache.spark.sql.connector.InMemoryTableSessionCatalog2f1af64f, `global_temp`, false at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:47) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:46) at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:122) ``` ### Does this PR introduce any user-facing change? Yes, `sql(s"DESCRIBE TABLE global_temp").show` in the above example now displays: ``` +---------------+---------+-------+ | col_name|data_type|comment| +---------------+---------+-------+ | id| bigint| | | data| string| | | | | | | # Partitioning| | | |Not partitioned| | | +---------------+---------+-------+ ``` instead of throwing an exception. ### How was this patch tested? Added new tests. Closes #26741 from imback82/global_temp. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../catalyst/analysis/ResolveCatalogs.scala | 124 ++++---- .../catalog/CatalogV2Implicits.scala | 2 + .../sql/connector/catalog/LookupCatalog.scala | 82 ++--- .../analysis/ResolveSessionCatalog.scala | 280 +++++++++--------- .../sql/connector/DataSourceV2SQLSuite.scala | 20 ++ 5 files changed, 277 insertions(+), 231 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 896b2830d524e..8183aa36a5b90 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -33,96 +33,98 @@ class ResolveCatalogs(val catalogManager: CatalogManager) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case AlterTableAddColumnsStatement( - nameParts @ NonSessionCatalog(catalog, tableName), cols) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => val changes = cols.map { col => TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) } - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) case AlterTableAlterColumnStatement( - nameParts @ NonSessionCatalog(catalog, tableName), colName, dataType, comment) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), colName, dataType, comment) => val typeChange = dataType.map { newDataType => TableChange.updateColumnType(colName.toArray, newDataType, true) } val commentChange = comment.map { newComment => TableChange.updateColumnComment(colName.toArray, newComment) } - createAlterTable(nameParts, catalog, tableName, typeChange.toSeq ++ commentChange) + createAlterTable(nameParts, catalog, tbl, typeChange.toSeq ++ commentChange) case AlterTableRenameColumnStatement( - nameParts @ NonSessionCatalog(catalog, tableName), col, newName) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), col, newName) => val changes = Seq(TableChange.renameColumn(col.toArray, newName)) - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) case AlterTableDropColumnsStatement( - nameParts @ NonSessionCatalog(catalog, tableName), cols) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) case AlterTableSetPropertiesStatement( - nameParts @ NonSessionCatalog(catalog, tableName), props) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), props) => val changes = props.map { case (key, value) => TableChange.setProperty(key, value) }.toSeq - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) // TODO: v2 `UNSET TBLPROPERTIES` should respect the ifExists flag. case AlterTableUnsetPropertiesStatement( - nameParts @ NonSessionCatalog(catalog, tableName), keys, _) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), keys, _) => val changes = keys.map(key => TableChange.removeProperty(key)) - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) case AlterTableSetLocationStatement( - nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, newLoc) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), partitionSpec, newLoc) => if (partitionSpec.nonEmpty) { throw new AnalysisException( "ALTER TABLE SET LOCATION does not support partition for v2 tables.") } val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, newLoc)) - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) case AlterViewSetPropertiesStatement( - NonSessionCatalog(catalog, tableName), props) => + NonSessionCatalogAndTable(catalog, tbl), props) => throw new AnalysisException( - s"Can not specify catalog `${catalog.name}` for view ${tableName.quoted} " + + s"Can not specify catalog `${catalog.name}` for view ${tbl.quoted} " + s"because view support in catalog has not been implemented yet") case AlterViewUnsetPropertiesStatement( - NonSessionCatalog(catalog, tableName), keys, ifExists) => + NonSessionCatalogAndTable(catalog, tbl), keys, ifExists) => throw new AnalysisException( - s"Can not specify catalog `${catalog.name}` for view ${tableName.quoted} " + + s"Can not specify catalog `${catalog.name}` for view ${tbl.quoted} " + s"because view support in catalog has not been implemented yet") - case AlterNamespaceSetPropertiesStatement(NonSessionCatalog(catalog, nameParts), properties) => - AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts, properties) + case AlterNamespaceSetPropertiesStatement( + NonSessionCatalogAndNamespace(catalog, ns), properties) => + AlterNamespaceSetProperties(catalog.asNamespaceCatalog, ns, properties) - case AlterNamespaceSetLocationStatement(NonSessionCatalog(catalog, nameParts), location) => - AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts, + case AlterNamespaceSetLocationStatement( + NonSessionCatalogAndNamespace(catalog, ns), location) => + AlterNamespaceSetProperties(catalog.asNamespaceCatalog, ns, Map(SupportsNamespaces.PROP_LOCATION -> location)) - case RenameTableStatement(NonSessionCatalog(catalog, oldName), newNameParts, isView) => + case RenameTableStatement(NonSessionCatalogAndTable(catalog, oldName), newNameParts, isView) => if (isView) { throw new AnalysisException("Renaming view is not supported in v2 catalogs.") } RenameTable(catalog.asTableCatalog, oldName.asIdentifier, newNameParts.asIdentifier) case DescribeTableStatement( - nameParts @ NonSessionCatalog(catalog, tableName), partitionSpec, isExtended) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), partitionSpec, isExtended) => if (partitionSpec.nonEmpty) { throw new AnalysisException("DESCRIBE TABLE does not support partition for v2 tables.") } - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) + val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tbl.asIdentifier) DescribeTable(r, isExtended) case DescribeColumnStatement( - NonSessionCatalog(catalog, tableName), colNameParts, isExtended) => + NonSessionCatalogAndTable(catalog, tbl), colNameParts, isExtended) => throw new AnalysisException("Describing columns is not supported for v2 tables.") case c @ CreateTableStatement( - NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => CreateV2Table( catalog.asTableCatalog, - tableName.asIdentifier, + tbl.asIdentifier, c.tableSchema, // convert the bucket spec and add it as a transform c.partitioning ++ c.bucketSpec.map(_.asTransform), @@ -130,10 +132,10 @@ class ResolveCatalogs(val catalogManager: CatalogManager) ignoreIfExists = c.ifNotExists) case c @ CreateTableAsSelectStatement( - NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => CreateTableAsSelect( catalog.asTableCatalog, - tableName.asIdentifier, + tbl.asIdentifier, // convert the bucket spec and add it as a transform c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, @@ -141,14 +143,14 @@ class ResolveCatalogs(val catalogManager: CatalogManager) writeOptions = c.options.filterKeys(_ != "path"), ignoreIfExists = c.ifNotExists) - case RefreshTableStatement(NonSessionCatalog(catalog, tableName)) => - RefreshTable(catalog.asTableCatalog, tableName.asIdentifier) + case RefreshTableStatement(NonSessionCatalogAndTable(catalog, tbl)) => + RefreshTable(catalog.asTableCatalog, tbl.asIdentifier) case c @ ReplaceTableStatement( - NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => ReplaceTable( catalog.asTableCatalog, - tableName.asIdentifier, + tbl.asIdentifier, c.tableSchema, // convert the bucket spec and add it as a transform c.partitioning ++ c.bucketSpec.map(_.asTransform), @@ -156,10 +158,10 @@ class ResolveCatalogs(val catalogManager: CatalogManager) orCreate = c.orCreate) case c @ ReplaceTableAsSelectStatement( - NonSessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => ReplaceTableAsSelect( catalog.asTableCatalog, - tableName.asIdentifier, + tbl.asIdentifier, // convert the bucket spec and add it as a transform c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, @@ -167,35 +169,32 @@ class ResolveCatalogs(val catalogManager: CatalogManager) writeOptions = c.options.filterKeys(_ != "path"), orCreate = c.orCreate) - case DropTableStatement(NonSessionCatalog(catalog, tableName), ifExists, _) => - DropTable(catalog.asTableCatalog, tableName.asIdentifier, ifExists) + case DropTableStatement(NonSessionCatalogAndTable(catalog, tbl), ifExists, _) => + DropTable(catalog.asTableCatalog, tbl.asIdentifier, ifExists) - case DropViewStatement(NonSessionCatalog(catalog, viewName), _) => + case DropViewStatement(NonSessionCatalogAndTable(catalog, viewName), _) => throw new AnalysisException( s"Can not specify catalog `${catalog.name}` for view ${viewName.quoted} " + s"because view support in catalog has not been implemented yet") - case c @ CreateNamespaceStatement(NonSessionCatalog(catalog, nameParts), _, _) => - CreateNamespace( - catalog.asNamespaceCatalog, - nameParts, - c.ifNotExists, - c.properties) + case c @ CreateNamespaceStatement(NonSessionCatalogAndNamespace(catalog, ns), _, _) => + CreateNamespace(catalog.asNamespaceCatalog, ns, c.ifNotExists, c.properties) - case DropNamespaceStatement(NonSessionCatalog(catalog, nameParts), ifExists, cascade) => - DropNamespace(catalog, nameParts, ifExists, cascade) + case DropNamespaceStatement(NonSessionCatalogAndNamespace(catalog, ns), ifExists, cascade) => + DropNamespace(catalog, ns, ifExists, cascade) - case DescribeNamespaceStatement(NonSessionCatalog(catalog, nameParts), extended) => - DescribeNamespace(catalog.asNamespaceCatalog, nameParts, extended) + case DescribeNamespaceStatement(NonSessionCatalogAndNamespace(catalog, ns), extended) => + DescribeNamespace(catalog.asNamespaceCatalog, ns, extended) - case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => + case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, ns)), pattern) => + val namespace = if (ns.nonEmpty) Some(ns) else None ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) case ShowNamespacesStatement(None, pattern) => ShowNamespaces(currentCatalog.asNamespaceCatalog, None, pattern) - case ShowTablesStatement(Some(NonSessionCatalog(catalog, nameParts)), pattern) => - ShowTables(catalog.asTableCatalog, nameParts, pattern) + case ShowTablesStatement(Some(NonSessionCatalogAndNamespace(catalog, ns)), pattern) => + ShowTables(catalog.asTableCatalog, ns, pattern) case ShowTablesStatement(None, pattern) if !isSessionCatalog(currentCatalog) => ShowTables(currentCatalog.asTableCatalog, catalogManager.currentNamespace, pattern) @@ -204,7 +203,8 @@ class ResolveCatalogs(val catalogManager: CatalogManager) if (isNamespaceSet) { SetCatalogAndNamespace(catalogManager, None, Some(nameParts)) } else { - val CatalogAndNamespace(catalog, namespace) = nameParts + val CatalogAndNamespace(catalog, ns) = nameParts + val namespace = if (ns.nonEmpty) Some(ns) else None SetCatalogAndNamespace(catalogManager, Some(catalog.name()), namespace) } @@ -212,15 +212,23 @@ class ResolveCatalogs(val catalogManager: CatalogManager) ShowCurrentNamespace(catalogManager) case ShowTablePropertiesStatement( - nameParts @ NonSessionCatalog(catalog, tableName), propertyKey) => - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) + nameParts @ NonSessionCatalogAndTable(catalog, tbl), propertyKey) => + val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tbl.asIdentifier) ShowTableProperties(r, propertyKey) } - object NonSessionCatalog { + object NonSessionCatalogAndTable { def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { - case CatalogAndIdentifierParts(catalog, parts) if !isSessionCatalog(catalog) => - Some(catalog -> parts) + case CatalogAndIdentifier(catalog, ident) if !isSessionCatalog(catalog) => + Some(catalog -> ident.asMultipartIdentifier) + case _ => None + } + } + + object NonSessionCatalogAndNamespace { + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { + case CatalogAndNamespace(catalog, ns) if !isSessionCatalog(catalog) => + Some(catalog -> ns) case _ => None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index aa4bab4080f60..882e968f34b59 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -96,6 +96,8 @@ private[sql] object CatalogV2Implicits { quote(ident.name) } } + + def asMultipartIdentifier: Seq[String] = ident.namespace :+ ident.name } implicit class MultipartIdentifierHelper(parts: Seq[String]) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index a37d37fd27a96..4d3aff22741ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -38,7 +38,7 @@ private[sql] trait LookupCatalog extends Logging { * * This does not substitute the default catalog if no catalog is set in the identifier. */ - private object CatalogAndIdentifier { + private object CatalogAndMultipartIdentifier { def unapply(parts: Seq[String]): Some[(Option[CatalogPlugin], Seq[String])] = parts match { case Seq(_) => Some((None, parts)) @@ -57,7 +57,7 @@ private[sql] trait LookupCatalog extends Logging { */ object CatalogObjectIdentifier { def unapply(parts: Seq[String]): Some[(CatalogPlugin, Identifier)] = parts match { - case CatalogAndIdentifier(maybeCatalog, nameParts) => + case CatalogAndMultipartIdentifier(maybeCatalog, nameParts) => Some(( maybeCatalog.getOrElse(currentCatalog), Identifier.of(nameParts.init.toArray, nameParts.last) @@ -88,19 +88,51 @@ private[sql] trait LookupCatalog extends Logging { } /** - * Extract catalog and namespace from a multi-part identifier with the current catalog if needed. + * Extract catalog and namespace from a multi-part name with the current catalog if needed. * Catalog name takes precedence over namespaces. */ object CatalogAndNamespace { - def unapply(parts: Seq[String]): Some[(CatalogPlugin, Option[Seq[String]])] = parts match { - case Seq(catalogName, tail @ _*) => + def unapply(nameParts: Seq[String]): Some[(CatalogPlugin, Seq[String])] = { + assert(nameParts.nonEmpty) + try { + Some((catalogManager.catalog(nameParts.head), nameParts.tail)) + } catch { + case _: CatalogNotFoundException => + Some((currentCatalog, nameParts)) + } + } + } + + /** + * Extract catalog and identifier from a multi-part name with the current catalog if needed. + * Catalog name takes precedence over identifier, but for a single-part name, identifier takes + * precedence over catalog name. + */ + object CatalogAndIdentifier { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper + + private val globalTempDB = SQLConf.get.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Identifier)] = { + assert(nameParts.nonEmpty) + if (nameParts.length == 1) { + Some((currentCatalog, Identifier.of(Array(), nameParts.head))) + } else if (nameParts.length == 2 && nameParts.head.equalsIgnoreCase(globalTempDB)) { + // Conceptually global temp views are in a special reserved catalog. However, the v2 catalog + // API does not support view yet, and we have to use v1 commands to deal with global temp + // views. To simplify the implementation, we put global temp views in a special namespace + // in the session catalog. The special namespace has higher priority during name resolution. + // For example, if the name of a custom catalog is the same with `GLOBAL_TEMP_DATABASE`, + // this custom catalog can't be accessed. + Some((catalogManager.v2SessionCatalog, nameParts.asIdentifier)) + } else { try { - Some( - (catalogManager.catalog(catalogName), if (tail.isEmpty) { None } else { Some(tail) })) + Some((catalogManager.catalog(nameParts.head), nameParts.tail.asIdentifier)) } catch { case _: CatalogNotFoundException => - Some((currentCatalog, Some(parts))) + Some((currentCatalog, nameParts.asIdentifier)) } + } } } @@ -111,7 +143,8 @@ private[sql] trait LookupCatalog extends Logging { */ object AsTableIdentifier { def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { - case CatalogAndIdentifier(None, names) if CatalogV2Util.isSessionCatalog(currentCatalog) => + case CatalogAndMultipartIdentifier(None, names) + if CatalogV2Util.isSessionCatalog(currentCatalog) => names match { case Seq(name) => Some(TableIdentifier(name)) @@ -130,39 +163,12 @@ private[sql] trait LookupCatalog extends Logging { */ object AsTemporaryViewIdentifier { def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { - case CatalogAndIdentifier(None, Seq(table)) => + case CatalogAndMultipartIdentifier(None, Seq(table)) => Some(TableIdentifier(table)) - case CatalogAndIdentifier(None, Seq(database, table)) => + case CatalogAndMultipartIdentifier(None, Seq(database, table)) => Some(TableIdentifier(table, Some(database))) case _ => None } } - - /** - * Extract catalog and the rest name parts from a multi-part identifier. - */ - object CatalogAndIdentifierParts { - private val globalTempDB = SQLConf.get.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) - - def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = { - assert(nameParts.nonEmpty) - try { - // Conceptually global temp views are in a special reserved catalog. However, the v2 catalog - // API does not support view yet, and we have to use v1 commands to deal with global temp - // views. To simplify the implementation, we put global temp views in a special namespace - // in the session catalog. The special namespace has higher priority during name resolution. - // For example, if the name of a custom catalog is the same with `GLOBAL_TEMP_DATABASE`, - // this custom catalog can't be accessed. - if (nameParts.head.equalsIgnoreCase(globalTempDB)) { - Some((catalogManager.v2SessionCatalog, nameParts)) - } else { - Some((catalogManager.catalog(nameParts.head), nameParts.tail)) - } - } catch { - case _: CatalogNotFoundException => - Some((currentCatalog, nameParts)) - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index c7ef878cb8c92..53eb7dae2ca0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -48,21 +48,21 @@ class ResolveSessionCatalog( override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case AlterTableAddColumnsStatement( - nameParts @ SessionCatalog(catalog, tableName), cols) => - loadTable(catalog, tableName.asIdentifier).collect { + nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => cols.foreach(c => assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand")) - AlterTableAddColumnsCommand(tableName.asTableIdentifier, cols.map(convertToStructField)) + AlterTableAddColumnsCommand(tbl.asTableIdentifier, cols.map(convertToStructField)) }.getOrElse { val changes = cols.map { col => TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) } - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) } case AlterTableAlterColumnStatement( - nameParts @ SessionCatalog(catalog, tableName), colName, dataType, comment) => - loadTable(catalog, tableName.asIdentifier).collect { + nameParts @ SessionCatalogAndTable(catalog, tbl), colName, dataType, comment) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => if (colName.length > 1) { throw new AnalysisException( @@ -85,7 +85,7 @@ class ResolveSessionCatalog( cleanedDataType, nullable = true, builder.build()) - AlterTableChangeColumnCommand(tableName.asTableIdentifier, colName(0), newColumn) + AlterTableChangeColumnCommand(tbl.asTableIdentifier, colName(0), newColumn) }.getOrElse { val typeChange = dataType.map { newDataType => TableChange.updateColumnType(colName.toArray, newDataType, true) @@ -93,122 +93,124 @@ class ResolveSessionCatalog( val commentChange = comment.map { newComment => TableChange.updateColumnComment(colName.toArray, newComment) } - createAlterTable(nameParts, catalog, tableName, typeChange.toSeq ++ commentChange) + createAlterTable(nameParts, catalog, tbl, typeChange.toSeq ++ commentChange) } case AlterTableRenameColumnStatement( - nameParts @ SessionCatalog(catalog, tableName), col, newName) => - loadTable(catalog, tableName.asIdentifier).collect { + nameParts @ SessionCatalogAndTable(catalog, tbl), col, newName) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => throw new AnalysisException("RENAME COLUMN is only supported with v2 tables.") }.getOrElse { val changes = Seq(TableChange.renameColumn(col.toArray, newName)) - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) } case AlterTableDropColumnsStatement( - nameParts @ SessionCatalog(catalog, tableName), cols) => - loadTable(catalog, tableName.asIdentifier).collect { + nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => throw new AnalysisException("DROP COLUMN is only supported with v2 tables.") }.getOrElse { val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) } case AlterTableSetPropertiesStatement( - nameParts @ SessionCatalog(catalog, tableName), props) => - loadTable(catalog, tableName.asIdentifier).collect { + nameParts @ SessionCatalogAndTable(catalog, tbl), props) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => - AlterTableSetPropertiesCommand(tableName.asTableIdentifier, props, isView = false) + AlterTableSetPropertiesCommand(tbl.asTableIdentifier, props, isView = false) }.getOrElse { val changes = props.map { case (key, value) => TableChange.setProperty(key, value) }.toSeq - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) } case AlterTableUnsetPropertiesStatement( - nameParts @ SessionCatalog(catalog, tableName), keys, ifExists) => - loadTable(catalog, tableName.asIdentifier).collect { + nameParts @ SessionCatalogAndTable(catalog, tbl), keys, ifExists) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => AlterTableUnsetPropertiesCommand( - tableName.asTableIdentifier, keys, ifExists, isView = false) + tbl.asTableIdentifier, keys, ifExists, isView = false) }.getOrElse { val changes = keys.map(key => TableChange.removeProperty(key)) - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) } case AlterTableSetLocationStatement( - nameParts @ SessionCatalog(catalog, tableName), partitionSpec, newLoc) => - loadTable(catalog, tableName.asIdentifier).collect { + nameParts @ SessionCatalogAndTable(catalog, tbl), partitionSpec, newLoc) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => - AlterTableSetLocationCommand(tableName.asTableIdentifier, partitionSpec, newLoc) + AlterTableSetLocationCommand(tbl.asTableIdentifier, partitionSpec, newLoc) }.getOrElse { if (partitionSpec.nonEmpty) { throw new AnalysisException( "ALTER TABLE SET LOCATION does not support partition for v2 tables.") } val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, newLoc)) - createAlterTable(nameParts, catalog, tableName, changes) + createAlterTable(nameParts, catalog, tbl, changes) } // ALTER VIEW should always use v1 command if the resolved catalog is session catalog. - case AlterViewSetPropertiesStatement(SessionCatalog(catalog, tableName), props) => - AlterTableSetPropertiesCommand(tableName.asTableIdentifier, props, isView = true) + case AlterViewSetPropertiesStatement(SessionCatalogAndTable(_, tbl), props) => + AlterTableSetPropertiesCommand(tbl.asTableIdentifier, props, isView = true) - case AlterViewUnsetPropertiesStatement(SessionCatalog(catalog, tableName), keys, ifExists) => - AlterTableUnsetPropertiesCommand(tableName.asTableIdentifier, keys, ifExists, isView = true) + case AlterViewUnsetPropertiesStatement(SessionCatalogAndTable(_, tbl), keys, ifExists) => + AlterTableUnsetPropertiesCommand(tbl.asTableIdentifier, keys, ifExists, isView = true) - case d @ DescribeNamespaceStatement(SessionCatalog(_, nameParts), _) => - if (nameParts.length != 1) { + case d @ DescribeNamespaceStatement(SessionCatalogAndNamespace(_, ns), _) => + if (ns.length != 1) { throw new AnalysisException( - s"The database name is not valid: ${nameParts.quoted}") + s"The database name is not valid: ${ns.quoted}") } - DescribeDatabaseCommand(nameParts.head, d.extended) + DescribeDatabaseCommand(ns.head, d.extended) - case AlterNamespaceSetPropertiesStatement(SessionCatalog(_, nameParts), properties) => - if (nameParts.length != 1) { + case AlterNamespaceSetPropertiesStatement( + SessionCatalogAndNamespace(_, ns), properties) => + if (ns.length != 1) { throw new AnalysisException( - s"The database name is not valid: ${nameParts.quoted}") + s"The database name is not valid: ${ns.quoted}") } - AlterDatabasePropertiesCommand(nameParts.head, properties) + AlterDatabasePropertiesCommand(ns.head, properties) - case AlterNamespaceSetLocationStatement(SessionCatalog(_, nameParts), location) => - if (nameParts.length != 1) { + case AlterNamespaceSetLocationStatement(SessionCatalogAndNamespace(_, ns), location) => + if (ns.length != 1) { throw new AnalysisException( - s"The database name is not valid: ${nameParts.quoted}") + s"The database name is not valid: ${ns.quoted}") } - AlterDatabaseSetLocationCommand(nameParts.head, location) + AlterDatabaseSetLocationCommand(ns.head, location) - case RenameTableStatement(SessionCatalog(_, oldName), newNameParts, isView) => + case RenameTableStatement(SessionCatalogAndTable(_, oldName), newNameParts, isView) => AlterTableRenameCommand(oldName.asTableIdentifier, newNameParts.asTableIdentifier, isView) case DescribeTableStatement( - nameParts @ SessionCatalog(catalog, tableName), partitionSpec, isExtended) => - loadTable(catalog, tableName.asIdentifier).collect { + nameParts @ SessionCatalogAndTable(catalog, tbl), partitionSpec, isExtended) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => - DescribeTableCommand(tableName.asTableIdentifier, partitionSpec, isExtended) + DescribeTableCommand(tbl.asTableIdentifier, partitionSpec, isExtended) }.getOrElse { // The v1 `DescribeTableCommand` can describe view as well. - if (isView(tableName)) { - DescribeTableCommand(tableName.asTableIdentifier, partitionSpec, isExtended) + if (isView(tbl)) { + DescribeTableCommand(tbl.asTableIdentifier, partitionSpec, isExtended) } else { if (partitionSpec.nonEmpty) { throw new AnalysisException("DESCRIBE TABLE does not support partition for v2 tables.") } - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tableName.asIdentifier) + val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tbl.asIdentifier) DescribeTable(r, isExtended) } } - case DescribeColumnStatement(SessionCatalog(catalog, tableName), colNameParts, isExtended) => - loadTable(catalog, tableName.asIdentifier).collect { + case DescribeColumnStatement( + SessionCatalogAndTable(catalog, tbl), colNameParts, isExtended) => + loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => - DescribeColumnCommand(tableName.asTableIdentifier, colNameParts, isExtended) + DescribeColumnCommand(tbl.asTableIdentifier, colNameParts, isExtended) }.getOrElse { - if (isView(tableName)) { - DescribeColumnCommand(tableName.asTableIdentifier, colNameParts, isExtended) + if (isView(tbl)) { + DescribeColumnCommand(tbl.asTableIdentifier, colNameParts, isExtended) } else { throw new AnalysisException("Describing columns is not supported for v2 tables.") } @@ -217,7 +219,7 @@ class ResolveSessionCatalog( // For CREATE TABLE [AS SELECT], we should use the v1 command if the catalog is resolved to the // session catalog and the table provider is not v2. case c @ CreateTableStatement( - SessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => if (!isV2Provider(c.provider)) { val tableDesc = buildCatalogTable(c.tableName.asTableIdentifier, c.tableSchema, c.partitioning, c.bucketSpec, c.properties, c.provider, c.options, c.location, @@ -227,7 +229,7 @@ class ResolveSessionCatalog( } else { CreateV2Table( catalog.asTableCatalog, - tableName.asIdentifier, + tbl.asIdentifier, c.tableSchema, // convert the bucket spec and add it as a transform c.partitioning ++ c.bucketSpec.map(_.asTransform), @@ -236,7 +238,7 @@ class ResolveSessionCatalog( } case c @ CreateTableAsSelectStatement( - SessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => if (!isV2Provider(c.provider)) { val tableDesc = buildCatalogTable(c.tableName.asTableIdentifier, new StructType, c.partitioning, c.bucketSpec, c.properties, c.provider, c.options, c.location, @@ -246,7 +248,7 @@ class ResolveSessionCatalog( } else { CreateTableAsSelect( catalog.asTableCatalog, - tableName.asIdentifier, + tbl.asIdentifier, // convert the bucket spec and add it as a transform c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, @@ -255,19 +257,19 @@ class ResolveSessionCatalog( ignoreIfExists = c.ifNotExists) } - case RefreshTableStatement(SessionCatalog(_, tableName)) => - RefreshTable(tableName.asTableIdentifier) + case RefreshTableStatement(SessionCatalogAndTable(_, tbl)) => + RefreshTable(tbl.asTableIdentifier) // For REPLACE TABLE [AS SELECT], we should fail if the catalog is resolved to the // session catalog and the table provider is not v2. case c @ ReplaceTableStatement( - SessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => if (!isV2Provider(c.provider)) { throw new AnalysisException("REPLACE TABLE is only supported with v2 tables.") } else { ReplaceTable( catalog.asTableCatalog, - tableName.asIdentifier, + tbl.asIdentifier, c.tableSchema, // convert the bucket spec and add it as a transform c.partitioning ++ c.bucketSpec.map(_.asTransform), @@ -276,13 +278,13 @@ class ResolveSessionCatalog( } case c @ ReplaceTableAsSelectStatement( - SessionCatalog(catalog, tableName), _, _, _, _, _, _, _, _, _) => + SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => if (!isV2Provider(c.provider)) { throw new AnalysisException("REPLACE TABLE AS SELECT is only supported with v2 tables.") } else { ReplaceTableAsSelect( catalog.asTableCatalog, - tableName.asIdentifier, + tbl.asIdentifier, // convert the bucket spec and add it as a transform c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, @@ -291,69 +293,70 @@ class ResolveSessionCatalog( orCreate = c.orCreate) } - case d @ DropTableStatement(SessionCatalog(catalog, tableName), ifExists, purge) => + case d @ DropTableStatement(SessionCatalogAndTable(catalog, tbl), ifExists, purge) => DropTableCommand(d.tableName.asTableIdentifier, ifExists, isView = false, purge = purge) - case DropViewStatement(SessionCatalog(catalog, viewName), ifExists) => + case DropViewStatement(SessionCatalogAndTable(catalog, viewName), ifExists) => DropTableCommand(viewName.asTableIdentifier, ifExists, isView = true, purge = false) - case c @ CreateNamespaceStatement(SessionCatalog(catalog, nameParts), _, _) => - if (nameParts.length != 1) { + case c @ CreateNamespaceStatement(SessionCatalogAndNamespace(_, ns), _, _) => + if (ns.length != 1) { throw new AnalysisException( - s"The database name is not valid: ${nameParts.quoted}") + s"The database name is not valid: ${ns.quoted}") } val comment = c.properties.get(SupportsNamespaces.PROP_COMMENT) val location = c.properties.get(SupportsNamespaces.PROP_LOCATION) val newProperties = c.properties -- SupportsNamespaces.RESERVED_PROPERTIES.asScala - CreateDatabaseCommand(nameParts.head, c.ifNotExists, location, comment, newProperties) + CreateDatabaseCommand(ns.head, c.ifNotExists, location, comment, newProperties) - case d @ DropNamespaceStatement(SessionCatalog(_, nameParts), _, _) => - if (nameParts.length != 1) { + case d @ DropNamespaceStatement(SessionCatalogAndNamespace(_, ns), _, _) => + if (ns.length != 1) { throw new AnalysisException( - s"The database name is not valid: ${nameParts.quoted}") + s"The database name is not valid: ${ns.quoted}") } - DropDatabaseCommand(nameParts.head, d.ifExists, d.cascade) + DropDatabaseCommand(ns.head, d.ifExists, d.cascade) - case ShowTablesStatement(Some(SessionCatalog(catalog, nameParts)), pattern) => - if (nameParts.length != 1) { - throw new AnalysisException( - s"The database name is not valid: ${nameParts.quoted}") + case ShowTablesStatement(Some(CatalogAndNamespace(catalog, ns)), pattern) + if isSessionCatalog(catalog) => + if (ns.length != 1) { + throw new AnalysisException( + s"The database name is not valid: ${ns.quoted}") } - ShowTablesCommand(Some(nameParts.head), pattern) + ShowTablesCommand(Some(ns.head), pattern) case ShowTablesStatement(None, pattern) if isSessionCatalog(currentCatalog) => ShowTablesCommand(None, pattern) - case ShowTableStatement(namespace, pattern, partitionsSpec) => - val db = namespace match { - case Some(namespace) if namespace.length != 1 => + case ShowTableStatement(ns, pattern, partitionsSpec) => + val db = ns match { + case Some(ns) if ns.length != 1 => throw new AnalysisException( - s"The database name is not valid: ${namespace.quoted}") - case _ => namespace.map(_.head) + s"The database name is not valid: ${ns.quoted}") + case _ => ns.map(_.head) } ShowTablesCommand(db, Some(pattern), true, partitionsSpec) - case AnalyzeTableStatement(tableName, partitionSpec, noScan) => - val v1TableName = parseV1Table(tableName, "ANALYZE TABLE") + case AnalyzeTableStatement(tbl, partitionSpec, noScan) => + val v1TableName = parseV1Table(tbl, "ANALYZE TABLE") if (partitionSpec.isEmpty) { AnalyzeTableCommand(v1TableName.asTableIdentifier, noScan) } else { AnalyzePartitionCommand(v1TableName.asTableIdentifier, partitionSpec, noScan) } - case AnalyzeColumnStatement(tableName, columnNames, allColumns) => - val v1TableName = parseV1Table(tableName, "ANALYZE TABLE") + case AnalyzeColumnStatement(tbl, columnNames, allColumns) => + val v1TableName = parseV1Table(tbl, "ANALYZE TABLE") AnalyzeColumnCommand(v1TableName.asTableIdentifier, columnNames, allColumns) - case RepairTableStatement(tableName) => - val v1TableName = parseV1Table(tableName, "MSCK REPAIR TABLE") + case RepairTableStatement(tbl) => + val v1TableName = parseV1Table(tbl, "MSCK REPAIR TABLE") AlterTableRecoverPartitionsCommand( v1TableName.asTableIdentifier, "MSCK REPAIR TABLE") - case LoadDataStatement(tableName, path, isLocal, isOverwrite, partition) => - val v1TableName = parseV1Table(tableName, "LOAD DATA") + case LoadDataStatement(tbl, path, isLocal, isOverwrite, partition) => + val v1TableName = parseV1Table(tbl, "LOAD DATA") LoadDataCommand( v1TableName.asTableIdentifier, path, @@ -361,73 +364,73 @@ class ResolveSessionCatalog( isOverwrite, partition) - case ShowCreateTableStatement(tableName) => - val v1TableName = parseV1Table(tableName, "SHOW CREATE TABLE") + case ShowCreateTableStatement(tbl) => + val v1TableName = parseV1Table(tbl, "SHOW CREATE TABLE") ShowCreateTableCommand(v1TableName.asTableIdentifier) - case CacheTableStatement(tableName, plan, isLazy, options) => - val v1TableName = parseV1Table(tableName, "CACHE TABLE") + case CacheTableStatement(tbl, plan, isLazy, options) => + val v1TableName = parseV1Table(tbl, "CACHE TABLE") CacheTableCommand(v1TableName.asTableIdentifier, plan, isLazy, options) - case UncacheTableStatement(tableName, ifExists) => - val v1TableName = parseV1Table(tableName, "UNCACHE TABLE") + case UncacheTableStatement(tbl, ifExists) => + val v1TableName = parseV1Table(tbl, "UNCACHE TABLE") UncacheTableCommand(v1TableName.asTableIdentifier, ifExists) - case TruncateTableStatement(tableName, partitionSpec) => - val v1TableName = parseV1Table(tableName, "TRUNCATE TABLE") + case TruncateTableStatement(tbl, partitionSpec) => + val v1TableName = parseV1Table(tbl, "TRUNCATE TABLE") TruncateTableCommand( v1TableName.asTableIdentifier, partitionSpec) - case ShowPartitionsStatement(tableName, partitionSpec) => - val v1TableName = parseV1Table(tableName, "SHOW PARTITIONS") + case ShowPartitionsStatement(tbl, partitionSpec) => + val v1TableName = parseV1Table(tbl, "SHOW PARTITIONS") ShowPartitionsCommand( v1TableName.asTableIdentifier, partitionSpec) - case ShowColumnsStatement(table, namespace) => + case ShowColumnsStatement(tbl, ns) => val sql = "SHOW COLUMNS" - val v1TableName = parseV1Table(table, sql).asTableIdentifier + val v1TableName = parseV1Table(tbl, sql).asTableIdentifier val resolver = conf.resolver - val db = namespace match { + val db = ns match { case Some(db) if (v1TableName.database.exists(!resolver(_, db.head))) => throw new AnalysisException( s"SHOW COLUMNS with conflicting databases: " + s"'${db.head}' != '${v1TableName.database.get}'") - case _ => namespace.map(_.head) + case _ => ns.map(_.head) } - if (namespace.isDefined && namespace.get.length > 1) { + if (ns.isDefined && ns.get.length > 1) { throw new AnalysisException( - s"Namespace name should have only one part if specified: ${namespace.get.quoted}") + s"Namespace name should have only one part if specified: ${ns.get.quoted}") } - if (table.length > 2) { + if (tbl.length > 2) { throw new AnalysisException( - s"Table name should have at most two parts: ${table.quoted}") + s"Table name should have at most two parts: ${tbl.quoted}") } ShowColumnsCommand(db, v1TableName) - case AlterTableRecoverPartitionsStatement(tableName) => - val v1TableName = parseV1Table(tableName, "ALTER TABLE RECOVER PARTITIONS") + case AlterTableRecoverPartitionsStatement(tbl) => + val v1TableName = parseV1Table(tbl, "ALTER TABLE RECOVER PARTITIONS") AlterTableRecoverPartitionsCommand( v1TableName.asTableIdentifier, "ALTER TABLE RECOVER PARTITIONS") - case AlterTableAddPartitionStatement(tableName, partitionSpecsAndLocs, ifNotExists) => - val v1TableName = parseV1Table(tableName, "ALTER TABLE ADD PARTITION") + case AlterTableAddPartitionStatement(tbl, partitionSpecsAndLocs, ifNotExists) => + val v1TableName = parseV1Table(tbl, "ALTER TABLE ADD PARTITION") AlterTableAddPartitionCommand( v1TableName.asTableIdentifier, partitionSpecsAndLocs, ifNotExists) - case AlterTableRenamePartitionStatement(tableName, from, to) => - val v1TableName = parseV1Table(tableName, "ALTER TABLE RENAME PARTITION") + case AlterTableRenamePartitionStatement(tbl, from, to) => + val v1TableName = parseV1Table(tbl, "ALTER TABLE RENAME PARTITION") AlterTableRenamePartitionCommand( v1TableName.asTableIdentifier, from, to) - case AlterTableDropPartitionStatement(tableName, specs, ifExists, purge, retainData) => - val v1TableName = parseV1Table(tableName, "ALTER TABLE DROP PARTITION") + case AlterTableDropPartitionStatement(tbl, specs, ifExists, purge, retainData) => + val v1TableName = parseV1Table(tbl, "ALTER TABLE DROP PARTITION") AlterTableDropPartitionCommand( v1TableName.asTableIdentifier, specs, @@ -435,27 +438,26 @@ class ResolveSessionCatalog( purge, retainData) - case AlterTableSerDePropertiesStatement( - tableName, serdeClassName, serdeProperties, partitionSpec) => - val v1TableName = parseV1Table(tableName, "ALTER TABLE SerDe Properties") + case AlterTableSerDePropertiesStatement(tbl, serdeClassName, serdeProperties, partitionSpec) => + val v1TableName = parseV1Table(tbl, "ALTER TABLE SerDe Properties") AlterTableSerDePropertiesCommand( v1TableName.asTableIdentifier, serdeClassName, serdeProperties, partitionSpec) - case AlterViewAsStatement(tableName, originalText, query) => - val v1TableName = parseV1Table(tableName, "ALTER VIEW QUERY") + case AlterViewAsStatement(tbl, originalText, query) => + val v1TableName = parseV1Table(tbl, "ALTER VIEW QUERY") AlterViewAsCommand( v1TableName.asTableIdentifier, originalText, query) case CreateViewStatement( - tableName, userSpecifiedColumns, comment, properties, + tbl, userSpecifiedColumns, comment, properties, originalText, child, allowExisting, replace, viewType) => - val v1TableName = parseV1Table(tableName, "CREATE VIEW") + val v1TableName = parseV1Table(tbl, "CREATE VIEW") CreateViewCommand( v1TableName.asTableIdentifier, userSpecifiedColumns, @@ -467,20 +469,20 @@ class ResolveSessionCatalog( replace, viewType) - case ShowTablePropertiesStatement(SessionCatalog(_, tableName), propertyKey) => + case ShowTablePropertiesStatement(SessionCatalogAndTable(_, tbl), propertyKey) => ShowTablePropertiesCommand( - tableName.asTableIdentifier, + tbl.asTableIdentifier, propertyKey) case ShowFunctionsStatement(userScope, systemScope, pattern, fun) => val (database, function) = fun match { - case Some(CatalogAndIdentifierParts(catalog, functionName)) => + case Some(CatalogAndIdentifier(catalog, functionIdent)) => if (isSessionCatalog(catalog)) { - functionName match { + functionIdent.asMultipartIdentifier match { case Seq(db, fn) => (Some(db), Some(fn)) case Seq(fn) => (None, Some(fn)) case _ => - throw new AnalysisException(s"Unsupported function name '${functionName.quoted}'") + throw new AnalysisException(s"Unsupported function name '${functionIdent.quoted}'") } } else { throw new AnalysisException ("SHOW FUNCTIONS is only supported in v1 catalog") @@ -491,11 +493,11 @@ class ResolveSessionCatalog( } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { - val CatalogAndIdentifierParts(catalog, parts) = tableName + val CatalogAndIdentifier(catalog, ident) = tableName if (!isSessionCatalog(catalog)) { throw new AnalysisException(s"$sql is only supported with v1 tables.") } - parts + ident.asMultipartIdentifier } private def buildCatalogTable( @@ -536,10 +538,18 @@ class ResolveSessionCatalog( comment = comment) } - object SessionCatalog { + object SessionCatalogAndTable { + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { + case CatalogAndIdentifier(catalog, ident) if isSessionCatalog(catalog) => + Some(catalog -> ident.asMultipartIdentifier) + case _ => None + } + } + + object SessionCatalogAndNamespace { def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { - case CatalogAndIdentifierParts(catalog, parts) if isSessionCatalog(catalog) => - Some(catalog -> parts) + case CatalogAndNamespace(catalog, ns) if isSessionCatalog(catalog) => + Some(catalog -> ns) case _ => None } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index ebaf753696fd5..6675636c0e62f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1822,6 +1822,26 @@ class DataSourceV2SQLSuite } } + test("SPARK-30104: global temp db is used as a table name under v2 catalog") { + val globalTempDB = spark.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + val t = s"testcat.$globalTempDB" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + sql("USE testcat") + // The following should not throw AnalysisException, but should use `testcat.$globalTempDB`. + sql(s"DESCRIBE TABLE $globalTempDB") + } + } + + test("table name same as catalog can be used") { + withTable("testcat.testcat") { + sql(s"CREATE TABLE testcat.testcat (id bigint, data string) USING foo") + sql("USE testcat") + // The following should not throw AnalysisException. + sql(s"DESCRIBE TABLE testcat") + } + } + test("SPARK-30001: session catalog name can be specified in SQL statements") { // unset this config to use the default v2 session catalog. spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) From 99ea324b6f22e979d2b4238eef0effa3709d03bd Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 11 Dec 2019 01:26:29 -0800 Subject: [PATCH 0845/1387] [SPARK-27506][SQL] Allow deserialization of Avro data using compatible schemas Follow up of https://github.com/apache/spark/pull/24405 ### What changes were proposed in this pull request? The current implementation of _from_avro_ and _AvroDataToCatalyst_ doesn't allow doing schema evolution since it requires the deserialization of an Avro record with the exact same schema with which it was serialized. The proposed change is to add a new option `actualSchema` to allow passing the schema used to serialize the records. This allows using a different compatible schema for reading by passing both schemas to _GenericDatumReader_. If no writer's schema is provided, nothing changes from before. ### Why are the changes needed? Consider the following example. ``` // schema ID: 1 val schema1 = """ { "type": "record", "name": "MySchema", "fields": [ {"name": "col1", "type": "int"}, {"name": "col2", "type": "string"} ] } """ // schema ID: 2 val schema2 = """ { "type": "record", "name": "MySchema", "fields": [ {"name": "col1", "type": "int"}, {"name": "col2", "type": "string"}, {"name": "col3", "type": "string", "default": ""} ] } """ ``` The two schemas are compatible - i.e. you can use `schema2` to deserialize events serialized with `schema1`, in which case there will be the field `col3` with the default value. Now imagine that you have two dataframes (read from batch or streaming), one with Avro events from schema1 and the other with events from schema2. **We want to combine them into one dataframe** for storing or further processing. With the current `from_avro` function we can only decode each of them with the corresponding schema: ``` scalaval df1 = ... // Avro events created with schema1 df1: org.apache.spark.sql.DataFrame = [eventBytes: binary] scalaval decodedDf1 = df1.select(from_avro('eventBytes, schema1) as "decoded") decodedDf1: org.apache.spark.sql.DataFrame = [decoded: struct] scalaval df2= ... // Avro events created with schema2 df2: org.apache.spark.sql.DataFrame = [eventBytes: binary] scalaval decodedDf2 = df2.select(from_avro('eventBytes, schema2) as "decoded") decodedDf2: org.apache.spark.sql.DataFrame = [decoded: struct] ``` but then `decodedDf1` and `decodedDf2` have different Spark schemas and we can't union them. Instead, with the proposed change we can decode `df1` in the following way: ``` scalaimport scala.collection.JavaConverters._ scalaval decodedDf1 = df1.select(from_avro(data = 'eventBytes, jsonFormatSchema = schema2, options = Map("actualSchema" -> schema1).asJava) as "decoded") decodedDf1: org.apache.spark.sql.DataFrame = [decoded: struct] ``` so that both dataframes have the same schemas and can be merged. ### Does this PR introduce any user-facing change? This PR allows users to pass a new configuration but it doesn't affect current code. ### How was this patch tested? A new unit test was added. Closes #26780 from Fokko/SPARK-27506. Lead-authored-by: Fokko Driesprong Co-authored-by: Gianluca Amori Signed-off-by: Gengliang Wang --- docs/sql-data-sources-avro.md | 9 ++++ .../spark/sql/avro/AvroDataToCatalyst.scala | 9 +++- .../apache/spark/sql/avro/AvroOptions.scala | 8 ++++ .../org/apache/spark/sql/avro/functions.scala | 7 +-- .../spark/sql/avro/AvroFunctionsSuite.scala | 45 ++++++++++++++++++- python/pyspark/sql/avro/functions.py | 7 +-- 6 files changed, 76 insertions(+), 9 deletions(-) diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md index 726db2ec09ad6..dda7af1dc0f31 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/sql-data-sources-avro.md @@ -240,6 +240,15 @@ Data source options of Avro can be set via: function from_avro + + actualSchema + None + Optional Avro schema (in JSON format) that was used to serialize the data. This should be set if the schema provided + for deserialization is compatible with - but not the same as - the one used to originally convert the data to Avro. + For more information on Avro's schema evolution and compatability, please refer to the [documentation of Confluent](https://docs.confluent.io/current/schema-registry/avro.html). + + function from_avro + ## Configuration diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 5656ac7f38e1b..8570828aa1c5c 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -51,9 +51,14 @@ case class AvroDataToCatalyst( override def nullable: Boolean = true + private lazy val avroOptions = AvroOptions(options) + @transient private lazy val avroSchema = new Schema.Parser().parse(jsonFormatSchema) - @transient private lazy val reader = new GenericDatumReader[Any](avroSchema) + @transient private lazy val reader = avroOptions.actualSchema + .map(actualSchema => + new GenericDatumReader[Any](new Schema.Parser().parse(actualSchema), avroSchema)) + .getOrElse(new GenericDatumReader[Any](avroSchema)) @transient private lazy val deserializer = new AvroDeserializer(avroSchema, dataType) @@ -62,7 +67,7 @@ case class AvroDataToCatalyst( @transient private var result: Any = _ @transient private lazy val parseMode: ParseMode = { - val mode = AvroOptions(options).parseMode + val mode = avroOptions.parseMode if (mode != PermissiveMode && mode != FailFastMode) { throw new AnalysisException(unacceptableModeMessage(mode.name)) } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala index 338244aa9e53b..b133a84f18829 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala @@ -40,6 +40,14 @@ class AvroOptions( */ val schema: Option[String] = parameters.get("avroSchema") + /** + * Optional Avro schema (in JSON format) that was used to serialize the data. + * This should be set if the schema provided for deserialization is compatible + * with - but not the same as - the one used to originally convert the data to Avro. + * See SPARK-27506 for more details. + */ + val actualSchema: Option[String] = parameters.get("actualSchema") + /** * Top level record name in write result, which is required in Avro spec. * See https://avro.apache.org/docs/1.8.2/spec.html#schema_record . diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala index a6ae3906c6d80..03fcd20c52ded 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala @@ -45,9 +45,10 @@ object functions { } /** - * Converts a binary column of avro format into its corresponding catalyst value. The specified - * schema must match the read data, otherwise the behavior is undefined: it may fail or return - * arbitrary result. + * Converts a binary column of Avro format into its corresponding catalyst value. If a schema is + * provided via the option actualSchema, a different (but compatible) schema can be used for + * reading. If no actualSchema option is provided, the specified schema must match the read data, + * otherwise the behavior is undefined: it may fail or return arbitrary result. * * @param data the binary column. * @param jsonFormatSchema the avro schema in JSON string format. diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala index c4f995015dfca..f0083e95fc2ce 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala @@ -28,7 +28,7 @@ import org.apache.avro.io.EncoderFactory import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.execution.LocalTableScanExec -import org.apache.spark.sql.functions.{col, struct} +import org.apache.spark.sql.functions.{col, lit, struct} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -153,4 +153,47 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { assert(df.collect().map(_.get(0)) === Seq(Row("one"), Row("two"), Row("three"), Row("four"))) } } + + test("SPARK-27506: roundtrip in to_avro and from_avro with different compatible schemas") { + val df = spark.range(10).select( + struct('id.as("col1"), 'id.cast("string").as("col2")).as("struct") + ) + val avroStructDF = df.select(functions.to_avro('struct).as("avro")) + val actualAvroSchema = + s""" + |{ + | "type": "record", + | "name": "struct", + | "fields": [ + | {"name": "col1", "type": "int"}, + | {"name": "col2", "type": "string"} + | ] + |} + |""".stripMargin + + val evolvedAvroSchema = + s""" + |{ + | "type": "record", + | "name": "struct", + | "fields": [ + | {"name": "col1", "type": "int"}, + | {"name": "col2", "type": "string"}, + | {"name": "col3", "type": "string", "default": ""} + | ] + |} + |""".stripMargin + + val expected = spark.range(10).select( + struct('id.as("col1"), 'id.cast("string").as("col2"), lit("").as("col3")).as("struct") + ) + + checkAnswer( + avroStructDF.select( + functions.from_avro( + 'avro, + evolvedAvroSchema, + Map("actualSchema" -> actualAvroSchema).asJava)), + expected) + } } diff --git a/python/pyspark/sql/avro/functions.py b/python/pyspark/sql/avro/functions.py index 711de6532e28c..3ecdb877bdf8b 100644 --- a/python/pyspark/sql/avro/functions.py +++ b/python/pyspark/sql/avro/functions.py @@ -30,9 +30,10 @@ @since(3.0) def from_avro(data, jsonFormatSchema, options={}): """ - Converts a binary column of avro format into its corresponding catalyst value. The specified - schema must match the read data, otherwise the behavior is undefined: it may fail or return - arbitrary result. + Converts a binary column of Avro format into its corresponding catalyst value. If a schema is + provided via the option actualSchema, a different (but compatible) schema can be used for + reading. If no actualSchema option is provided, the specified schema must match the read data, + otherwise the behavior is undefined: it may fail or return arbitrary result. Note: Avro is built-in but external data source module since Spark 2.4. Please deploy the application as per the deployment section of "Apache Avro Data Source Guide". From 82418b419cfc89c8e2ade6a21b4a3b336c07bb51 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 11 Dec 2019 20:41:07 +0800 Subject: [PATCH 0846/1387] [SPARK-30207][SQL][DOCS] Enhance the SQL NULL Semantics document ### What changes were proposed in this pull request? Enhancement of the SQL NULL Semantics document: sql-ref-null-semantics.html. ### Why are the changes needed? Clarify the behavior of `UNKNOWN` for both `EXIST` and `IN` operation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Doc changes only. Closes #26837 from xuanyuanking/SPARK-30207. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- docs/sql-ref-null-semantics.md | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-null-semantics.md b/docs/sql-ref-null-semantics.md index a67b3993a31c0..fd467d224ffd5 100644 --- a/docs/sql-ref-null-semantics.md +++ b/docs/sql-ref-null-semantics.md @@ -604,13 +604,14 @@ SELECT name, age FROM unknown_age; ### EXISTS/NOT EXISTS Subquery In Spark, EXISTS and NOT EXISTS expressions are allowed inside a WHERE clause. These are boolean expressions which return either `TRUE` or -`FALSE`. In otherwords, EXISTS is a membership condition and returns `TRUE` +`FALSE`. In other words, EXISTS is a membership condition and returns `TRUE` when the subquery it refers to returns one or more rows. Similary, NOT EXISTS is a non-membership condition and returns TRUE when no rows or zero rows are returned from the subquery. These two expressions are not affected by presence of NULL in the result of -the subquery. +the subquery. They are normally faster because they can be converted to +semijoins / anti-semijoins without special provisions for null awareness. #### Examples {% highlight sql %} @@ -668,6 +669,10 @@ To summarize, below are the rules for computing the result of an `IN` expression list does not contain NULL values - UNKNOWN is returned when the value is `NULL`, or the non-NULL value is not found in the list and the list contains at least one `NULL` value + +NOT IN always returns UNKNOWN when the list contains `NULL`, regardless of the input value. +This is because IN returns UNKNOWN if the value is not in the list containing `NULL`, +and because NOT UNKNOWN is again UNKNOWN. #### Examples {% highlight sql %} From d46c03c3d383eb3eaf9c80db87d48a20c7bcd24d Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Wed, 11 Dec 2019 09:39:39 -0600 Subject: [PATCH 0847/1387] [SPARK-29460][WEBUI] Add tooltip for Jobs page ### What changes were proposed in this pull request? Adding tooltip for jobs tab column - Job Id (Job Group), Description ,Submitted, Duration, Stages, Tasks Before: ![Screenshot from 2019-11-04 11-31-02](https://user-images.githubusercontent.com/51401130/68102467-e8a54300-fef8-11e9-9f9e-48dd1b393ac8.png) After: ![Screenshot from 2019-11-04 11-30-53](https://user-images.githubusercontent.com/51401130/68102478-f3f86e80-fef8-11e9-921a-357678229cb4.png) ### Why are the changes needed? Jobs tab do not have any tooltip for the columns, Some page provide tooltip , inorder to resolve the inconsistency and for better user experience. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual Closes #26384 from PavithraRamachandran/jobTab_tooltip. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- .../apache/spark/ui/jobs/AllJobsPage.scala | 55 +++++++++++++++---- 1 file changed, 43 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 49f57062dd9fc..6300e639633c5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -541,12 +541,15 @@ private[ui] class JobPagedTable( override def headers: Seq[Node] = { // Information for each header: title, cssClass, and sortable - val jobHeadersAndCssClasses: Seq[(String, String, Boolean)] = + val jobHeadersAndCssClasses: Seq[(String, String, Boolean, Option[String])] = Seq( - (jobIdTitle, "", true), - ("Description", "", true), ("Submitted", "", true), ("Duration", "", true), - ("Stages: Succeeded/Total", "", false), - ("Tasks (for all stages): Succeeded/Total", "", false) + (jobIdTitle, "", true, None), + ("Description", "", true, None), + ("Submitted", "", true, None), + ("Duration", "", true, Some("Elapsed time since the job was submitted " + + "until execution completion of all its stages.")), + ("Stages: Succeeded/Total", "", false, None), + ("Tasks (for all stages): Succeeded/Total", "", false, None) ) if (!jobHeadersAndCssClasses.filter(_._3).map(_._1).contains(sortColumn)) { @@ -554,7 +557,7 @@ private[ui] class JobPagedTable( } val headerRow: Seq[Node] = { - jobHeadersAndCssClasses.map { case (header, cssClass, sortable) => + jobHeadersAndCssClasses.map { case (header, cssClass, sortable, tooltip) => if (header == sortColumn) { val headerLink = Unparsed( parameterPath + @@ -566,9 +569,17 @@ private[ui] class JobPagedTable( - {header} -  {Unparsed(arrow)} - + { + if (tooltip.nonEmpty) { + + {header} {Unparsed(arrow)} + + } else { + + {header} {Unparsed(arrow)} + + } + } } else { @@ -581,12 +592,32 @@ private[ui] class JobPagedTable( - {header} - + { + if (tooltip.nonEmpty) { + + {header} + + } else { + + {header} + + } + } + } else { - {header} + { + if (tooltip.nonEmpty) { + + {header} + + } else { + + {header} + + } + } } } From a59cb13cda73b0d05f68181c66558d33298600c6 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 11 Dec 2019 08:17:53 -0800 Subject: [PATCH 0848/1387] [SPARK-30200][SQL][FOLLOWUP] Fix typo in ExplainMode ### What changes were proposed in this pull request? This pr is a follow-up of #26829 to fix typos in ExplainMode. ### Why are the changes needed? For better docs. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #26851 from maropu/SPARK-30200-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java b/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java index 018ea12db55f1..50e11e94731e4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java +++ b/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java @@ -41,14 +41,14 @@ public enum ExplainMode { */ Extended, /** - * Extended mode means that when printing explain for a DataFrame, if generated codes are + * Codegen mode means that when printing explain for a DataFrame, if generated codes are * available, a physical plan and the generated codes are expected to be printed to the console. * * @since 3.0.0 */ Codegen, /** - * Extended mode means that when printing explain for a DataFrame, if plan node statistics are + * Cost mode means that when printing explain for a DataFrame, if plan node statistics are * available, a logical plan and the statistics are expected to be printed to the console. * * @since 3.0.0 From e933539cdd557297daf97ff5e532a3f098896979 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 12 Dec 2019 01:08:53 +0800 Subject: [PATCH 0849/1387] [SPARK-29864][SPARK-29920][SQL] Strict parsing of day-time strings to intervals ### What changes were proposed in this pull request? In the PR, I propose new implementation of `fromDayTimeString` which strictly parses strings in day-time formats to intervals. New implementation accepts only strings that match to a pattern defined by the `from` and `to`. Here is the mapping of user's bounds and patterns: - `[+|-]D+ H[H]:m[m]:s[s][.SSSSSSSSS]` for **DAY TO SECOND** - `[+|-]D+ H[H]:m[m]` for **DAY TO MINUTE** - `[+|-]D+ H[H]` for **DAY TO HOUR** - `[+|-]H[H]:m[m]s[s][.SSSSSSSSS]` for **HOUR TO SECOND** - `[+|-]H[H]:m[m]` for **HOUR TO MINUTE** - `[+|-]m[m]:s[s][.SSSSSSSSS]` for **MINUTE TO SECOND** Closes #26327 Closes #26358 ### Why are the changes needed? - Improve user experience with Spark SQL, and respect to the bound specified by users. - Behave the same as other broadly used DBMS - Oracle and MySQL. ### Does this PR introduce any user-facing change? Yes, before: ```sql spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE; interval 1 weeks 3 days 11 hours 12 minutes ``` After: ```sql spark-sql> SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE; Error in query: requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 10 11:12:13.123(line 1, pos 16) == SQL == SELECT INTERVAL '10 11:12:13.123' HOUR TO MINUTE ----------------^^^ ``` ### How was this patch tested? - Added tests to `IntervalUtilsSuite` - By `ExpressionParserSuite` - Updated `literals.sql` Closes #26473 from MaxGekk/strict-from-daytime-string. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + .../sql/catalyst/util/IntervalUtils.scala | 128 ++++- .../apache/spark/sql/internal/SQLConf.scala | 11 + .../parser/ExpressionParserSuite.scala | 2 +- .../catalyst/util/IntervalUtilsSuite.scala | 137 +++-- .../resources/sql-tests/inputs/interval.sql | 15 +- .../sql-tests/results/ansi/interval.sql.out | 468 ++++++++++-------- .../sql-tests/results/interval.sql.out | 394 ++++++++------- .../results/postgreSQL/interval.sql.out | 100 +++- 9 files changed, 818 insertions(+), 439 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 9bcd36ce41271..ca78f3ca46ea5 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -256,6 +256,8 @@ license: | - Since Spark 3.0, the unary arithmetic operator plus(`+`) only accepts string, numeric and interval type values as inputs. Besides, `+` with a integral string representation will be coerced to double value, e.g. `+'1'` results `1.0`. In Spark version 2.4 and earlier, this operator is ignored. There is no type checking for it, thus, all type values with a `+` prefix are valid, e.g. `+ array(1, 2)` is valid and results `[1, 2]`. Besides, there is no type coercion for it at all, e.g. in Spark 2.4, the result of `+'1'` is string `1`. + - Since Spark 3.0, day-time interval strings are converted to intervals with respect to the `from` and `to` bounds. If an input string does not match to the pattern defined by specified bounds, the `ParseException` exception is thrown. For example, `interval '2 10:20' hour to minute` raises the exception because the expected format is `[+|-]h[h]:[m]m`. In Spark version 2.4, the `from` bound was not taken into account, and the `to` bound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted to `interval 10 hours 20 minutes`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.fromDayTimeString.enabled` to `true`. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 6e33c5b24f478..2a60cfd52ca99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import scala.util.control.NonFatal import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -155,9 +156,6 @@ object IntervalUtils { fromDayTimeString(s, DAY, SECOND) } - private val dayTimePattern = - "^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$".r - /** * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn * @@ -168,9 +166,35 @@ object IntervalUtils { * - MINUTE TO SECOND */ def fromDayTimeString(input: String, from: IntervalUnit, to: IntervalUnit): CalendarInterval = { + if (SQLConf.get.getConf(SQLConf.LEGACY_FROM_DAYTIME_STRING)) { + parseDayTimeLegacy(input, from, to) + } else { + parseDayTime(input, from, to) + } + } + + private val dayTimePatternLegacy = + "^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$".r + + /** + * Legacy method of parsing a string in a day-time format. It ignores the `from` bound, + * and takes into account only the `to` bound by truncating the result. For example, + * if the input string is "2 12:30:15", `from` is "hour" and `to` is "second", the result + * is "2 days 12 hours 30 minutes". + * + * @param input The day-time string + * @param from The interval units from which the input strings begins + * @param to The interval units at which the input string ends + * @return an instance of `CalendarInterval` if parsing completes successfully otherwise + * the exception `IllegalArgumentException` is raised. + */ + private def parseDayTimeLegacy( + input: String, + from: IntervalUnit, + to: IntervalUnit): CalendarInterval = { require(input != null, "Interval day-time string must be not null") assert(input.length == input.trim.length) - val m = dayTimePattern.pattern.matcher(input) + val m = dayTimePatternLegacy.pattern.matcher(input) require(m.matches, s"Interval string must match day-time format of 'd h:m:s.n': $input") try { @@ -222,6 +246,78 @@ object IntervalUtils { } } + private val signRe = "(?[+|-])" + private val dayRe = "(?\\d+)" + private val hourRe = "(?\\d{1,2})" + private val minuteRe = "(?\\d{1,2})" + private val secondRe = "(?(\\d{1,2})(\\.(\\d{1,9}))?)" + + private val dayTimePattern = Map( + (MINUTE, SECOND) -> s"^$signRe?$minuteRe:$secondRe$$".r, + (HOUR, MINUTE) -> s"^$signRe?$hourRe:$minuteRe$$".r, + (HOUR, SECOND) -> s"^$signRe?$hourRe:$minuteRe:$secondRe$$".r, + (DAY, HOUR) -> s"^$signRe?$dayRe $hourRe$$".r, + (DAY, MINUTE) -> s"^$signRe?$dayRe $hourRe:$minuteRe$$".r, + (DAY, SECOND) -> s"^$signRe?$dayRe $hourRe:$minuteRe:$secondRe$$".r + ) + + private def unitsRange(start: IntervalUnit, end: IntervalUnit): Seq[IntervalUnit] = { + (start.id to end.id).map(IntervalUnit(_)) + } + + /** + * Parses an input string in the day-time format defined by the `from` and `to` bounds. + * It supports the following formats: + * - [+|-]D+ H[H]:m[m]:s[s][.SSSSSSSSS] for DAY TO SECOND + * - [+|-]D+ H[H]:m[m] for DAY TO MINUTE + * - [+|-]D+ H[H] for DAY TO HOUR + * - [+|-]H[H]:m[m]s[s][.SSSSSSSSS] for HOUR TO SECOND + * - [+|-]H[H]:m[m] for HOUR TO MINUTE + * - [+|-]m[m]:s[s][.SSSSSSSSS] for MINUTE TO SECOND + * + * Note: the seconds fraction is truncated to microseconds. + * + * @param input The input string to parse. + * @param from The interval unit from which the input string begins. + * @param to The interval unit at where the input string ends. + * @return an instance of `CalendarInterval` if the input string was parsed successfully + * otherwise throws an exception. + * @throws IllegalArgumentException The input string has incorrect format and cannot be parsed. + * @throws ArithmeticException An interval unit value is out of valid range or the resulted + * interval fields `days` or `microseconds` are out of the valid + * ranges. + */ + private def parseDayTime( + input: String, + from: IntervalUnit, + to: IntervalUnit): CalendarInterval = { + require(input != null, "Interval day-time string must be not null") + val regexp = dayTimePattern.get(from -> to) + require(regexp.isDefined, s"Cannot support (interval '$input' $from to $to) expression") + val pattern = regexp.get.pattern + val m = pattern.matcher(input) + require(m.matches, s"Interval string must match day-time format of '$pattern': $input") + var micros: Long = 0L + var days: Int = 0 + unitsRange(to, from).foreach { + case unit @ DAY => + days = toLongWithRange(unit, m.group(unit.toString), 0, Int.MaxValue).toInt + case unit @ HOUR => + val parsed = toLongWithRange(unit, m.group(unit.toString), 0, 23) + micros = Math.addExact(micros, parsed * MICROS_PER_HOUR) + case unit @ MINUTE => + val parsed = toLongWithRange(unit, m.group(unit.toString), 0, 59) + micros = Math.addExact(micros, parsed * MICROS_PER_MINUTE) + case unit @ SECOND => + micros = Math.addExact(micros, parseSecondNano(m.group(unit.toString))) + case _ => + throw new IllegalArgumentException( + s"Cannot support (interval '$input' $from to $to) expression") + } + val sign = if (m.group("sign") != null && m.group("sign") == "-") -1 else 1 + new CalendarInterval(0, sign * days, sign * micros) + } + // Parses a string with nanoseconds, truncates the result and returns microseconds private def parseNanos(nanosStr: String, isNegative: Boolean): Long = { if (nanosStr != null) { @@ -237,6 +333,30 @@ object IntervalUtils { } } + /** + * Parse second_nano string in ss.nnnnnnnnn format to microseconds + */ + private def parseSecondNano(secondNano: String): Long = { + def parseSeconds(secondsStr: String): Long = { + toLongWithRange( + SECOND, + secondsStr, + Long.MinValue / MICROS_PER_SECOND, + Long.MaxValue / MICROS_PER_SECOND) * MICROS_PER_SECOND + } + + secondNano.split("\\.") match { + case Array(secondsStr) => parseSeconds(secondsStr) + case Array("", nanosStr) => parseNanos(nanosStr, false) + case Array(secondsStr, nanosStr) => + val seconds = parseSeconds(secondsStr) + Math.addExact(seconds, parseNanos(nanosStr, seconds < 0)) + case _ => + throw new IllegalArgumentException( + "Interval string does not match second-nano format of ss.nnnnnnnnn") + } + } + /** * Gets interval duration * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f7c1d4a798ecb..c54008cf91942 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2087,6 +2087,17 @@ object SQLConf { .stringConf .createWithDefault( "https://maven-central.storage-download.googleapis.com/repos/central/data/") + + val LEGACY_FROM_DAYTIME_STRING = + buildConf("spark.sql.legacy.fromDayTimeString.enabled") + .internal() + .doc("When true, the `from` bound is not taken into account in conversion of " + + "a day-time string to an interval, and the `to` bound is used to skip" + + "all interval units out of the specified range. If it is set to `false`, " + + "`ParseException` is thrown if the input does not match to the pattern " + + "defined by `from` and `to`.") + .booleanConf + .createWithDefault(false) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 90bb487ec6168..4dccada3d5546 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -716,7 +716,7 @@ class ExpressionParserSuite extends AnalysisTest { "0:0:0", "0:0:1") hourTimeValues.foreach { value => - val result = Literal(IntervalUtils.fromDayTimeString(value)) + val result = Literal(IntervalUtils.fromDayTimeString(value, HOUR, SECOND)) checkIntervals(s"'$value' hour to second", result) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index c98bb61b6ee7b..15ba5f03d0505 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -20,12 +20,14 @@ package org.apache.spark.sql.catalyst.util import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -class IntervalUtilsSuite extends SparkFunSuite { +class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { private def checkFromString(input: String, expected: CalendarInterval): Unit = { assert(stringToInterval(UTF8String.fromString(input)) === expected) @@ -160,43 +162,45 @@ class IntervalUtilsSuite extends SparkFunSuite { } } - test("from day-time string") { - assert(fromDayTimeString("5 12:40:30.999999999") === - new CalendarInterval( - 0, - 5, - 12 * MICROS_PER_HOUR + - 40 * MICROS_PER_MINUTE + - 30 * MICROS_PER_SECOND + 999999L)) - assert(fromDayTimeString("10 0:12:0.888") === - new CalendarInterval( - 0, - 10, - 12 * MICROS_PER_MINUTE + 888 * MICROS_PER_MILLIS)) - assert(fromDayTimeString("-3 0:0:0") === new CalendarInterval(0, -3, 0L)) - - try { - fromDayTimeString("5 30:12:20") - fail("Expected to throw an exception for the invalid input") - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("hour 30 outside range")) - } - - try { - fromDayTimeString("5 30-12") - fail("Expected to throw an exception for the invalid input") - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("must match day-time format")) - } - - try { - fromDayTimeString("5 1:12:20", HOUR, MICROSECOND) - fail("Expected to throw an exception for the invalid convention type") - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("Cannot support (interval")) + test("from day-time string - legacy") { + withSQLConf(SQLConf.LEGACY_FROM_DAYTIME_STRING.key -> "true") { + assert(fromDayTimeString("5 12:40:30.999999999") === + new CalendarInterval( + 0, + 5, + 12 * MICROS_PER_HOUR + + 40 * MICROS_PER_MINUTE + + 30 * MICROS_PER_SECOND + 999999L)) + assert(fromDayTimeString("10 0:12:0.888") === + new CalendarInterval( + 0, + 10, + 12 * MICROS_PER_MINUTE + 888 * MICROS_PER_MILLIS)) + assert(fromDayTimeString("-3 0:0:0") === new CalendarInterval(0, -3, 0L)) + + try { + fromDayTimeString("5 30:12:20") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("hour 30 outside range")) + } + + try { + fromDayTimeString("5 30-12") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("must match day-time format")) + } + + try { + fromDayTimeString("5 1:12:20", HOUR, MICROSECOND) + fail("Expected to throw an exception for the invalid convention type") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("Cannot support (interval")) + } } } @@ -384,4 +388,61 @@ class IntervalUtilsSuite extends SparkFunSuite { val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) assert(IntervalUtils.toMultiUnitsString(i9) === "-3000 hours") } + + test("from day-time string") { + def check(input: String, from: IntervalUnit, to: IntervalUnit, expected: String): Unit = { + withClue(s"from = $from, to = $to") { + val expectedUtf8 = UTF8String.fromString(expected) + assert(fromDayTimeString(input, from, to) === safeStringToInterval(expectedUtf8)) + } + } + def checkFail( + input: String, + from: IntervalUnit, + to: IntervalUnit, + errMsg: String): Unit = { + try { + fromDayTimeString(input, from, to) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains(errMsg)) + } + } + + check("12:40", HOUR, MINUTE, "12 hours 40 minutes") + check("+12:40", HOUR, MINUTE, "12 hours 40 minutes") + check("-12:40", HOUR, MINUTE, "-12 hours -40 minutes") + checkFail("5 12:40", HOUR, MINUTE, "must match day-time format") + + check("12:40:30.999999999", HOUR, SECOND, "12 hours 40 minutes 30.999999 seconds") + check("+12:40:30.123456789", HOUR, SECOND, "12 hours 40 minutes 30.123456 seconds") + check("-12:40:30.123456789", HOUR, SECOND, "-12 hours -40 minutes -30.123456 seconds") + checkFail("5 12:40:30", HOUR, SECOND, "must match day-time format") + checkFail("12:40:30.0123456789", HOUR, SECOND, "must match day-time format") + + check("40:30.123456789", MINUTE, SECOND, "40 minutes 30.123456 seconds") + check("+40:30.123456789", MINUTE, SECOND, "40 minutes 30.123456 seconds") + check("-40:30.123456789", MINUTE, SECOND, "-40 minutes -30.123456 seconds") + checkFail("12:40:30", MINUTE, SECOND, "must match day-time format") + + check("5 12", DAY, HOUR, "5 days 12 hours") + check("+5 12", DAY, HOUR, "5 days 12 hours") + check("-5 12", DAY, HOUR, "-5 days -12 hours") + checkFail("5 12:30", DAY, HOUR, "must match day-time format") + + check("5 12:40", DAY, MINUTE, "5 days 12 hours 40 minutes") + check("+5 12:40", DAY, MINUTE, "5 days 12 hours 40 minutes") + check("-5 12:40", DAY, MINUTE, "-5 days -12 hours -40 minutes") + checkFail("5 12", DAY, MINUTE, "must match day-time format") + + check("5 12:40:30.123", DAY, SECOND, "5 days 12 hours 40 minutes 30.123 seconds") + check("+5 12:40:30.123456", DAY, SECOND, "5 days 12 hours 40 minutes 30.123456 seconds") + check("-5 12:40:30.123456789", DAY, SECOND, "-5 days -12 hours -40 minutes -30.123456 seconds") + checkFail("5 12", DAY, SECOND, "must match day-time format") + + checkFail("5 30:12:20", DAY, SECOND, "hour 30 outside range") + checkFail("5 30-12", DAY, SECOND, "must match day-time format") + checkFail("5 1:12:20", HOUR, MICROSECOND, "Cannot support (interval") + } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index 5806d9288572f..5b5eaab225d28 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -97,17 +97,22 @@ select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisec select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; select interval '0 0:0:0.1' day to second; select interval '10-9' year to month; +select interval '20 15' day to hour; +select interval '20 15:40' day to minute; +select interval '20 15:40:32.99899999' day to second; +select interval '15:40' hour to minute; +select interval '15:40:32.99899999' hour to second; +select interval '40:32.99899999' minute to second; +select interval '40:32' minute to second; +select interval 30 day day; + +-- invalid day-time string intervals select interval '20 15:40:32.99899999' day to hour; select interval '20 15:40:32.99899999' day to minute; -select interval '20 15:40:32.99899999' day to second; select interval '15:40:32.99899999' hour to minute; select interval '15:40.99899999' hour to second; select interval '15:40' hour to second; -select interval '15:40:32.99899999' hour to second; select interval '20 40:32.99899999' minute to second; -select interval '40:32.99899999' minute to second; -select interval '40:32' minute to second; -select interval 30 day day; -- ns is not supported select interval 10 nanoseconds; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 2709fb42488cc..8c65c5ece7e54 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 135 +-- Number of queries: 138 -- !query 0 @@ -517,7 +517,7 @@ struct -- !query 64 -select interval '20 15:40:32.99899999' day to hour +select interval '20 15' day to hour -- !query 64 schema struct -- !query 64 output @@ -525,7 +525,7 @@ struct -- !query 65 -select interval '20 15:40:32.99899999' day to minute +select interval '20 15:40' day to minute -- !query 65 schema struct -- !query 65 output @@ -541,7 +541,7 @@ struct -- !query 67 -select interval '15:40:32.99899999' hour to minute +select interval '15:40' hour to minute -- !query 67 schema struct -- !query 67 output @@ -549,74 +549,134 @@ struct -- !query 68 -select interval '15:40.99899999' hour to second +select interval '15:40:32.99899999' hour to second -- !query 68 schema -struct +struct -- !query 68 output -15 minutes 40.998999 seconds +15 hours 40 minutes 32.998999 seconds -- !query 69 -select interval '15:40' hour to second +select interval '40:32.99899999' minute to second -- !query 69 schema -struct +struct -- !query 69 output -15 hours 40 minutes +40 minutes 32.998999 seconds -- !query 70 -select interval '15:40:32.99899999' hour to second +select interval '40:32' minute to second -- !query 70 schema -struct +struct -- !query 70 output -15 hours 40 minutes 32.998999 seconds +40 minutes 32 seconds -- !query 71 -select interval '20 40:32.99899999' minute to second +select interval 30 day day -- !query 71 schema -struct +struct<> -- !query 71 output -20 days 40 minutes 32.998999 seconds +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 23) + +== SQL == +select interval 30 day day +-----------------------^^^ -- !query 72 -select interval '40:32.99899999' minute to second +select interval '20 15:40:32.99899999' day to hour -- !query 72 schema -struct +struct<> -- !query 72 output -40 minutes 32.998999 seconds +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) + +== SQL == +select interval '20 15:40:32.99899999' day to hour +----------------^^^ -- !query 73 -select interval '40:32' minute to second +select interval '20 15:40:32.99899999' day to minute -- !query 73 schema -struct +struct<> -- !query 73 output -40 minutes 32 seconds +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) + +== SQL == +select interval '20 15:40:32.99899999' day to minute +----------------^^^ -- !query 74 -select interval 30 day day +select interval '15:40:32.99899999' hour to minute -- !query 74 schema struct<> -- !query 74 output org.apache.spark.sql.catalyst.parser.ParseException -no viable alternative at input 'day'(line 1, pos 23) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 15:40:32.99899999(line 1, pos 16) == SQL == -select interval 30 day day ------------------------^^^ +select interval '15:40:32.99899999' hour to minute +----------------^^^ -- !query 75 -select interval 10 nanoseconds +select interval '15:40.99899999' hour to second -- !query 75 schema struct<> -- !query 75 output org.apache.spark.sql.catalyst.parser.ParseException +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40.99899999(line 1, pos 16) + +== SQL == +select interval '15:40.99899999' hour to second +----------------^^^ + + +-- !query 76 +select interval '15:40' hour to second +-- !query 76 schema +struct<> +-- !query 76 output +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40(line 1, pos 16) + +== SQL == +select interval '15:40' hour to second +----------------^^^ + + +-- !query 77 +select interval '20 40:32.99899999' minute to second +-- !query 77 schema +struct<> +-- !query 77 output +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 20 40:32.99899999(line 1, pos 16) + +== SQL == +select interval '20 40:32.99899999' minute to second +----------------^^^ + + +-- !query 78 +select interval 10 nanoseconds +-- !query 78 schema +struct<> +-- !query 78 output +org.apache.spark.sql.catalyst.parser.ParseException + no viable alternative at input '10 nanoseconds'(line 1, pos 19) == SQL == @@ -624,35 +684,35 @@ select interval 10 nanoseconds -------------------^^^ --- !query 76 +-- !query 79 select map(1, interval 1 day, 2, interval 3 week) --- !query 76 schema +-- !query 79 schema struct> --- !query 76 output +-- !query 79 output {1:1 days,2:21 days} --- !query 77 +-- !query 80 select interval 'interval 3 year 1 hour' --- !query 77 schema +-- !query 80 schema struct --- !query 77 output +-- !query 80 output 3 years 1 hours --- !query 78 +-- !query 81 select interval '3 year 1 hour' --- !query 78 schema +-- !query 81 schema struct --- !query 78 output +-- !query 81 output 3 years 1 hours --- !query 79 +-- !query 82 select interval --- !query 79 schema +-- !query 82 schema struct<> --- !query 79 output +-- !query 82 output org.apache.spark.sql.catalyst.parser.ParseException at least one time unit should be given for interval literal(line 1, pos 7) @@ -662,11 +722,11 @@ select interval -------^^^ --- !query 80 +-- !query 83 select interval 1 fake_unit --- !query 80 schema +-- !query 83 schema struct<> --- !query 80 output +-- !query 83 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input '1 fake_unit'(line 1, pos 18) @@ -676,11 +736,11 @@ select interval 1 fake_unit ------------------^^^ --- !query 81 +-- !query 84 select interval 1 year to month --- !query 81 schema +-- !query 84 schema struct<> --- !query 81 output +-- !query 84 output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 16) @@ -690,11 +750,11 @@ select interval 1 year to month ----------------^^^ --- !query 82 +-- !query 85 select interval '1' year to second --- !query 82 schema +-- !query 85 schema struct<> --- !query 82 output +-- !query 85 output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 16) @@ -704,11 +764,11 @@ select interval '1' year to second ----------------^^^ --- !query 83 +-- !query 86 select interval '10-9' year to month '2-1' year to month --- !query 83 schema +-- !query 86 schema struct<> --- !query 83 output +-- !query 86 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -718,11 +778,11 @@ select interval '10-9' year to month '2-1' year to month -------------------------------------^^^ --- !query 84 +-- !query 87 select interval '10-9' year to month '12:11:10' hour to second --- !query 84 schema +-- !query 87 schema struct<> --- !query 84 output +-- !query 87 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -732,11 +792,11 @@ select interval '10-9' year to month '12:11:10' hour to second -------------------------------------^^^ --- !query 85 +-- !query 88 select interval '1 15:11' day to minute '12:11:10' hour to second --- !query 85 schema +-- !query 88 schema struct<> --- !query 85 output +-- !query 88 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) @@ -746,11 +806,11 @@ select interval '1 15:11' day to minute '12:11:10' hour to second ----------------------------------------^^^ --- !query 86 +-- !query 89 select interval 1 year '2-1' year to month --- !query 86 schema +-- !query 89 schema struct<> --- !query 86 output +-- !query 89 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -760,11 +820,11 @@ select interval 1 year '2-1' year to month -----------------------^^^ --- !query 87 +-- !query 90 select interval 1 year '12:11:10' hour to second --- !query 87 schema +-- !query 90 schema struct<> --- !query 87 output +-- !query 90 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -774,11 +834,11 @@ select interval 1 year '12:11:10' hour to second -----------------------^^^ --- !query 88 +-- !query 91 select interval '10-9' year to month '1' year --- !query 88 schema +-- !query 91 schema struct<> --- !query 88 output +-- !query 91 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -788,11 +848,11 @@ select interval '10-9' year to month '1' year -------------------------------------^^^ --- !query 89 +-- !query 92 select interval '12:11:10' hour to second '1' year --- !query 89 schema +-- !query 92 schema struct<> --- !query 89 output +-- !query 92 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) @@ -802,11 +862,11 @@ select interval '12:11:10' hour to second '1' year ------------------------------------------^^^ --- !query 90 +-- !query 93 select interval (-30) day --- !query 90 schema +-- !query 93 schema struct<> --- !query 90 output +-- !query 93 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 22) @@ -816,11 +876,11 @@ select interval (-30) day ----------------------^^^ --- !query 91 +-- !query 94 select interval (a + 1) day --- !query 91 schema +-- !query 94 schema struct<> --- !query 91 output +-- !query 94 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 24) @@ -830,11 +890,11 @@ select interval (a + 1) day ------------------------^^^ --- !query 92 +-- !query 95 select interval 30 day day day --- !query 92 schema +-- !query 95 schema struct<> --- !query 92 output +-- !query 95 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 23) @@ -844,192 +904,192 @@ select interval 30 day day day -----------------------^^^ --- !query 93 +-- !query 96 select sum(cast(null as interval)) --- !query 93 schema +-- !query 96 schema struct --- !query 93 output +-- !query 96 output NULL --- !query 94 +-- !query 97 select sum(cast(v as interval)) from VALUES ('1 seconds') t(v) where 1=0 --- !query 94 schema +-- !query 97 schema struct --- !query 94 output +-- !query 97 output NULL --- !query 95 +-- !query 98 select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 95 schema +-- !query 98 schema struct --- !query 95 output +-- !query 98 output 3 seconds --- !query 96 +-- !query 99 select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 96 schema +-- !query 99 schema struct --- !query 96 output +-- !query 99 output 1 seconds --- !query 97 +-- !query 100 select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 97 schema +-- !query 100 schema struct --- !query 97 output +-- !query 100 output -3 seconds --- !query 98 +-- !query 101 select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 98 schema +-- !query 101 schema struct --- !query 98 output +-- !query 101 output -7 days 2 seconds --- !query 99 +-- !query 102 select i, sum(cast(v as interval)) from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) group by i --- !query 99 schema +-- !query 102 schema struct --- !query 99 output +-- !query 102 output 1 -2 days 2 2 seconds 3 NULL --- !query 100 +-- !query 103 select sum(cast(v as interval)) as sv from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) having sv is not null --- !query 100 schema +-- !query 103 schema struct --- !query 100 output +-- !query 103 output -2 days 2 seconds --- !query 101 +-- !query 104 SELECT i, sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM VALUES(1, '1 seconds'), (1, '2 seconds'), (2, NULL), (2, NULL) t(i,v) --- !query 101 schema +-- !query 104 schema struct --- !query 101 output +-- !query 104 output 1 2 seconds 1 3 seconds 2 NULL 2 NULL --- !query 102 +-- !query 105 select avg(cast(v as interval)) from VALUES (null) t(v) --- !query 102 schema +-- !query 105 schema struct --- !query 102 output +-- !query 105 output NULL --- !query 103 +-- !query 106 select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0 --- !query 103 schema +-- !query 106 schema struct --- !query 103 output +-- !query 106 output NULL --- !query 104 +-- !query 107 select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 104 schema +-- !query 107 schema struct --- !query 104 output +-- !query 107 output 1.5 seconds --- !query 105 +-- !query 108 select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 105 schema +-- !query 108 schema struct --- !query 105 output +-- !query 108 output 0.5 seconds --- !query 106 +-- !query 109 select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 106 schema +-- !query 109 schema struct --- !query 106 output +-- !query 109 output -1.5 seconds --- !query 107 +-- !query 110 select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 107 schema +-- !query 110 schema struct --- !query 107 output +-- !query 110 output -3 days -11 hours -59 minutes -59 seconds --- !query 108 +-- !query 111 select i, avg(cast(v as interval)) from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) group by i --- !query 108 schema +-- !query 111 schema struct --- !query 108 output +-- !query 111 output 1 -1 days 2 2 seconds 3 NULL --- !query 109 +-- !query 112 select avg(cast(v as interval)) as sv from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) having sv is not null --- !query 109 schema +-- !query 112 schema struct --- !query 109 output +-- !query 112 output -15 hours -59 minutes -59.333333 seconds --- !query 110 +-- !query 113 SELECT i, avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v) --- !query 110 schema +-- !query 113 schema struct --- !query 110 output +-- !query 113 output 1 1.5 seconds 1 2 seconds 2 NULL 2 NULL --- !query 111 +-- !query 114 create temporary view interval_arithmetic as select CAST(dateval AS date), CAST(tsval AS timestamp) from values ('2012-01-01', '2012-01-01') as interval_arithmetic(dateval, tsval) --- !query 111 schema +-- !query 114 schema struct<> --- !query 111 output +-- !query 114 output --- !query 112 +-- !query 115 select dateval, dateval - interval '2-2' year to month, @@ -1039,13 +1099,13 @@ select - interval '2-2' year to month + dateval, interval '2-2' year to month + dateval from interval_arithmetic --- !query 112 schema +-- !query 115 schema struct --- !query 112 output +-- !query 115 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 --- !query 113 +-- !query 116 select tsval, tsval - interval '2-2' year to month, @@ -1055,24 +1115,24 @@ select - interval '2-2' year to month + tsval, interval '2-2' year to month + tsval from interval_arithmetic --- !query 113 schema +-- !query 116 schema struct --- !query 113 output +-- !query 116 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 --- !query 114 +-- !query 117 select interval '2-2' year to month + interval '3-3' year to month, interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic --- !query 114 schema +-- !query 117 schema struct<(INTERVAL '2 years 2 months' + INTERVAL '3 years 3 months'):interval,(INTERVAL '2 years 2 months' - INTERVAL '3 years 3 months'):interval> --- !query 114 output +-- !query 117 output 5 years 5 months -1 years -1 months --- !query 115 +-- !query 118 select dateval, dateval - interval '99 11:22:33.123456789' day to second, @@ -1082,13 +1142,13 @@ select -interval '99 11:22:33.123456789' day to second + dateval, interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic --- !query 115 schema +-- !query 118 schema struct --- !query 115 output +-- !query 118 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 --- !query 116 +-- !query 119 select tsval, tsval - interval '99 11:22:33.123456789' day to second, @@ -1098,92 +1158,92 @@ select -interval '99 11:22:33.123456789' day to second + tsval, interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic --- !query 116 schema +-- !query 119 schema struct --- !query 116 output +-- !query 119 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 --- !query 117 +-- !query 120 select interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic --- !query 117 schema +-- !query 120 schema struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> --- !query 117 output +-- !query 120 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds --- !query 118 +-- !query 121 select interval '\t interval 1 day' --- !query 118 schema +-- !query 121 schema struct --- !query 118 output +-- !query 121 output 1 days --- !query 119 +-- !query 122 select interval 'interval \t 1\tday' --- !query 119 schema +-- !query 122 schema struct --- !query 119 output +-- !query 122 output 1 days --- !query 120 +-- !query 123 select interval 'interval\t1\tday' --- !query 120 schema +-- !query 123 schema struct --- !query 120 output +-- !query 123 output 1 days --- !query 121 +-- !query 124 select interval '1\t' day --- !query 121 schema +-- !query 124 schema struct --- !query 121 output +-- !query 124 output 1 days --- !query 122 +-- !query 125 select interval '1 ' day --- !query 122 schema +-- !query 125 schema struct --- !query 122 output +-- !query 125 output 1 days --- !query 123 +-- !query 126 select 1 year 2 days --- !query 123 schema +-- !query 126 schema struct --- !query 123 output +-- !query 126 output 1 years 2 days --- !query 124 +-- !query 127 select '10-9' year to month --- !query 124 schema +-- !query 127 schema struct --- !query 124 output +-- !query 127 output 10 years 9 months --- !query 125 +-- !query 128 select '20 15:40:32.99899999' day to second --- !query 125 schema +-- !query 128 schema struct --- !query 125 output +-- !query 128 output 20 days 15 hours 40 minutes 32.998999 seconds --- !query 126 +-- !query 129 select 30 day day --- !query 126 schema +-- !query 129 schema struct<> --- !query 126 output +-- !query 129 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 14) @@ -1193,27 +1253,27 @@ select 30 day day --------------^^^ --- !query 127 +-- !query 130 select date'2012-01-01' - '2-2' year to month --- !query 127 schema +-- !query 130 schema struct --- !query 127 output +-- !query 130 output 2009-11-01 --- !query 128 +-- !query 131 select 1 month - 1 day --- !query 128 schema +-- !query 131 schema struct --- !query 128 output +-- !query 131 output 1 months -1 days --- !query 129 +-- !query 132 select 1 year to month --- !query 129 schema +-- !query 132 schema struct<> --- !query 129 output +-- !query 132 output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 7) @@ -1223,11 +1283,11 @@ select 1 year to month -------^^^ --- !query 130 +-- !query 133 select '1' year to second --- !query 130 schema +-- !query 133 schema struct<> --- !query 130 output +-- !query 133 output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 7) @@ -1237,11 +1297,11 @@ select '1' year to second -------^^^ --- !query 131 +-- !query 134 select 1 year '2-1' year to month --- !query 131 schema +-- !query 134 schema struct<> --- !query 131 output +-- !query 134 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 14) @@ -1251,11 +1311,11 @@ select 1 year '2-1' year to month --------------^^^ --- !query 132 +-- !query 135 select (-30) day --- !query 132 schema +-- !query 135 schema struct<> --- !query 132 output +-- !query 135 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 13) @@ -1265,11 +1325,11 @@ select (-30) day -------------^^^ --- !query 133 +-- !query 136 select (a + 1) day --- !query 133 schema +-- !query 136 schema struct<> --- !query 133 output +-- !query 136 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 15) @@ -1279,11 +1339,11 @@ select (a + 1) day ---------------^^^ --- !query 134 +-- !query 137 select 30 day day day --- !query 134 schema +-- !query 137 schema struct<> --- !query 134 output +-- !query 137 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 14) diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 601521ba43227..ff0a3ff74f1ec 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 123 +-- Number of queries: 126 -- !query 0 @@ -517,7 +517,7 @@ struct -- !query 64 -select interval '20 15:40:32.99899999' day to hour +select interval '20 15' day to hour -- !query 64 schema struct -- !query 64 output @@ -525,7 +525,7 @@ struct -- !query 65 -select interval '20 15:40:32.99899999' day to minute +select interval '20 15:40' day to minute -- !query 65 schema struct -- !query 65 output @@ -541,7 +541,7 @@ struct -- !query 67 -select interval '15:40:32.99899999' hour to minute +select interval '15:40' hour to minute -- !query 67 schema struct -- !query 67 output @@ -549,68 +549,128 @@ struct -- !query 68 -select interval '15:40.99899999' hour to second +select interval '15:40:32.99899999' hour to second -- !query 68 schema -struct +struct -- !query 68 output -15 minutes 40.998999 seconds +15 hours 40 minutes 32.998999 seconds -- !query 69 -select interval '15:40' hour to second +select interval '40:32.99899999' minute to second -- !query 69 schema -struct +struct -- !query 69 output -15 hours 40 minutes +40 minutes 32.998999 seconds -- !query 70 -select interval '15:40:32.99899999' hour to second +select interval '40:32' minute to second -- !query 70 schema -struct +struct -- !query 70 output -15 hours 40 minutes 32.998999 seconds +40 minutes 32 seconds -- !query 71 -select interval '20 40:32.99899999' minute to second +select interval 30 day day -- !query 71 schema -struct +struct -- !query 71 output -20 days 40 minutes 32.998999 seconds +30 days -- !query 72 -select interval '40:32.99899999' minute to second +select interval '20 15:40:32.99899999' day to hour -- !query 72 schema -struct +struct<> -- !query 72 output -40 minutes 32.998999 seconds +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) + +== SQL == +select interval '20 15:40:32.99899999' day to hour +----------------^^^ -- !query 73 -select interval '40:32' minute to second +select interval '20 15:40:32.99899999' day to minute -- !query 73 schema -struct +struct<> -- !query 73 output -40 minutes 32 seconds +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) + +== SQL == +select interval '20 15:40:32.99899999' day to minute +----------------^^^ -- !query 74 -select interval 30 day day +select interval '15:40:32.99899999' hour to minute -- !query 74 schema -struct +struct<> -- !query 74 output -30 days +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 15:40:32.99899999(line 1, pos 16) + +== SQL == +select interval '15:40:32.99899999' hour to minute +----------------^^^ -- !query 75 -select interval 10 nanoseconds +select interval '15:40.99899999' hour to second -- !query 75 schema struct<> -- !query 75 output org.apache.spark.sql.catalyst.parser.ParseException +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40.99899999(line 1, pos 16) + +== SQL == +select interval '15:40.99899999' hour to second +----------------^^^ + + +-- !query 76 +select interval '15:40' hour to second +-- !query 76 schema +struct<> +-- !query 76 output +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40(line 1, pos 16) + +== SQL == +select interval '15:40' hour to second +----------------^^^ + + +-- !query 77 +select interval '20 40:32.99899999' minute to second +-- !query 77 schema +struct<> +-- !query 77 output +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 20 40:32.99899999(line 1, pos 16) + +== SQL == +select interval '20 40:32.99899999' minute to second +----------------^^^ + + +-- !query 78 +select interval 10 nanoseconds +-- !query 78 schema +struct<> +-- !query 78 output +org.apache.spark.sql.catalyst.parser.ParseException + no viable alternative at input '10 nanoseconds'(line 1, pos 19) == SQL == @@ -618,35 +678,35 @@ select interval 10 nanoseconds -------------------^^^ --- !query 76 +-- !query 79 select map(1, interval 1 day, 2, interval 3 week) --- !query 76 schema +-- !query 79 schema struct> --- !query 76 output +-- !query 79 output {1:1 days,2:21 days} --- !query 77 +-- !query 80 select interval 'interval 3 year 1 hour' --- !query 77 schema +-- !query 80 schema struct --- !query 77 output +-- !query 80 output 3 years 1 hours --- !query 78 +-- !query 81 select interval '3 year 1 hour' --- !query 78 schema +-- !query 81 schema struct --- !query 78 output +-- !query 81 output 3 years 1 hours --- !query 79 +-- !query 82 select interval --- !query 79 schema +-- !query 82 schema struct<> --- !query 79 output +-- !query 82 output org.apache.spark.sql.catalyst.parser.ParseException at least one time unit should be given for interval literal(line 1, pos 7) @@ -656,11 +716,11 @@ select interval -------^^^ --- !query 80 +-- !query 83 select interval 1 fake_unit --- !query 80 schema +-- !query 83 schema struct<> --- !query 80 output +-- !query 83 output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input '1 fake_unit'(line 1, pos 18) @@ -670,11 +730,11 @@ select interval 1 fake_unit ------------------^^^ --- !query 81 +-- !query 84 select interval 1 year to month --- !query 81 schema +-- !query 84 schema struct<> --- !query 81 output +-- !query 84 output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 16) @@ -684,11 +744,11 @@ select interval 1 year to month ----------------^^^ --- !query 82 +-- !query 85 select interval '1' year to second --- !query 82 schema +-- !query 85 schema struct<> --- !query 82 output +-- !query 85 output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 16) @@ -698,11 +758,11 @@ select interval '1' year to second ----------------^^^ --- !query 83 +-- !query 86 select interval '10-9' year to month '2-1' year to month --- !query 83 schema +-- !query 86 schema struct<> --- !query 83 output +-- !query 86 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -712,11 +772,11 @@ select interval '10-9' year to month '2-1' year to month -------------------------------------^^^ --- !query 84 +-- !query 87 select interval '10-9' year to month '12:11:10' hour to second --- !query 84 schema +-- !query 87 schema struct<> --- !query 84 output +-- !query 87 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -726,11 +786,11 @@ select interval '10-9' year to month '12:11:10' hour to second -------------------------------------^^^ --- !query 85 +-- !query 88 select interval '1 15:11' day to minute '12:11:10' hour to second --- !query 85 schema +-- !query 88 schema struct<> --- !query 85 output +-- !query 88 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) @@ -740,11 +800,11 @@ select interval '1 15:11' day to minute '12:11:10' hour to second ----------------------------------------^^^ --- !query 86 +-- !query 89 select interval 1 year '2-1' year to month --- !query 86 schema +-- !query 89 schema struct<> --- !query 86 output +-- !query 89 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -754,11 +814,11 @@ select interval 1 year '2-1' year to month -----------------------^^^ --- !query 87 +-- !query 90 select interval 1 year '12:11:10' hour to second --- !query 87 schema +-- !query 90 schema struct<> --- !query 87 output +-- !query 90 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -768,11 +828,11 @@ select interval 1 year '12:11:10' hour to second -----------------------^^^ --- !query 88 +-- !query 91 select interval '10-9' year to month '1' year --- !query 88 schema +-- !query 91 schema struct<> --- !query 88 output +-- !query 91 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -782,11 +842,11 @@ select interval '10-9' year to month '1' year -------------------------------------^^^ --- !query 89 +-- !query 92 select interval '12:11:10' hour to second '1' year --- !query 89 schema +-- !query 92 schema struct<> --- !query 89 output +-- !query 92 output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) @@ -796,29 +856,29 @@ select interval '12:11:10' hour to second '1' year ------------------------------------------^^^ --- !query 90 +-- !query 93 select interval (-30) day --- !query 90 schema +-- !query 93 schema struct<> --- !query 90 output +-- !query 93 output org.apache.spark.sql.AnalysisException Undefined function: 'interval'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 --- !query 91 +-- !query 94 select interval (a + 1) day --- !query 91 schema +-- !query 94 schema struct<> --- !query 91 output +-- !query 94 output org.apache.spark.sql.AnalysisException Undefined function: 'interval'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 --- !query 92 +-- !query 95 select interval 30 day day day --- !query 92 schema +-- !query 95 schema struct<> --- !query 92 output +-- !query 95 output org.apache.spark.sql.catalyst.parser.ParseException extraneous input 'day' expecting (line 1, pos 27) @@ -828,192 +888,192 @@ select interval 30 day day day ---------------------------^^^ --- !query 93 +-- !query 96 select sum(cast(null as interval)) --- !query 93 schema +-- !query 96 schema struct --- !query 93 output +-- !query 96 output NULL --- !query 94 +-- !query 97 select sum(cast(v as interval)) from VALUES ('1 seconds') t(v) where 1=0 --- !query 94 schema +-- !query 97 schema struct --- !query 94 output +-- !query 97 output NULL --- !query 95 +-- !query 98 select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 95 schema +-- !query 98 schema struct --- !query 95 output +-- !query 98 output 3 seconds --- !query 96 +-- !query 99 select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 96 schema +-- !query 99 schema struct --- !query 96 output +-- !query 99 output 1 seconds --- !query 97 +-- !query 100 select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 97 schema +-- !query 100 schema struct --- !query 97 output +-- !query 100 output -3 seconds --- !query 98 +-- !query 101 select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 98 schema +-- !query 101 schema struct --- !query 98 output +-- !query 101 output -7 days 2 seconds --- !query 99 +-- !query 102 select i, sum(cast(v as interval)) from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) group by i --- !query 99 schema +-- !query 102 schema struct --- !query 99 output +-- !query 102 output 1 -2 days 2 2 seconds 3 NULL --- !query 100 +-- !query 103 select sum(cast(v as interval)) as sv from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) having sv is not null --- !query 100 schema +-- !query 103 schema struct --- !query 100 output +-- !query 103 output -2 days 2 seconds --- !query 101 +-- !query 104 SELECT i, sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM VALUES(1, '1 seconds'), (1, '2 seconds'), (2, NULL), (2, NULL) t(i,v) --- !query 101 schema +-- !query 104 schema struct --- !query 101 output +-- !query 104 output 1 2 seconds 1 3 seconds 2 NULL 2 NULL --- !query 102 +-- !query 105 select avg(cast(v as interval)) from VALUES (null) t(v) --- !query 102 schema +-- !query 105 schema struct --- !query 102 output +-- !query 105 output NULL --- !query 103 +-- !query 106 select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0 --- !query 103 schema +-- !query 106 schema struct --- !query 103 output +-- !query 106 output NULL --- !query 104 +-- !query 107 select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 104 schema +-- !query 107 schema struct --- !query 104 output +-- !query 107 output 1.5 seconds --- !query 105 +-- !query 108 select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 105 schema +-- !query 108 schema struct --- !query 105 output +-- !query 108 output 0.5 seconds --- !query 106 +-- !query 109 select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 106 schema +-- !query 109 schema struct --- !query 106 output +-- !query 109 output -1.5 seconds --- !query 107 +-- !query 110 select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 107 schema +-- !query 110 schema struct --- !query 107 output +-- !query 110 output -3 days -11 hours -59 minutes -59 seconds --- !query 108 +-- !query 111 select i, avg(cast(v as interval)) from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) group by i --- !query 108 schema +-- !query 111 schema struct --- !query 108 output +-- !query 111 output 1 -1 days 2 2 seconds 3 NULL --- !query 109 +-- !query 112 select avg(cast(v as interval)) as sv from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) having sv is not null --- !query 109 schema +-- !query 112 schema struct --- !query 109 output +-- !query 112 output -15 hours -59 minutes -59.333333 seconds --- !query 110 +-- !query 113 SELECT i, avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v) --- !query 110 schema +-- !query 113 schema struct --- !query 110 output +-- !query 113 output 1 1.5 seconds 1 2 seconds 2 NULL 2 NULL --- !query 111 +-- !query 114 create temporary view interval_arithmetic as select CAST(dateval AS date), CAST(tsval AS timestamp) from values ('2012-01-01', '2012-01-01') as interval_arithmetic(dateval, tsval) --- !query 111 schema +-- !query 114 schema struct<> --- !query 111 output +-- !query 114 output --- !query 112 +-- !query 115 select dateval, dateval - interval '2-2' year to month, @@ -1023,13 +1083,13 @@ select - interval '2-2' year to month + dateval, interval '2-2' year to month + dateval from interval_arithmetic --- !query 112 schema +-- !query 115 schema struct --- !query 112 output +-- !query 115 output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 --- !query 113 +-- !query 116 select tsval, tsval - interval '2-2' year to month, @@ -1039,24 +1099,24 @@ select - interval '2-2' year to month + tsval, interval '2-2' year to month + tsval from interval_arithmetic --- !query 113 schema +-- !query 116 schema struct --- !query 113 output +-- !query 116 output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 --- !query 114 +-- !query 117 select interval '2-2' year to month + interval '3-3' year to month, interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic --- !query 114 schema +-- !query 117 schema struct<(INTERVAL '2 years 2 months' + INTERVAL '3 years 3 months'):interval,(INTERVAL '2 years 2 months' - INTERVAL '3 years 3 months'):interval> --- !query 114 output +-- !query 117 output 5 years 5 months -1 years -1 months --- !query 115 +-- !query 118 select dateval, dateval - interval '99 11:22:33.123456789' day to second, @@ -1066,13 +1126,13 @@ select -interval '99 11:22:33.123456789' day to second + dateval, interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic --- !query 115 schema +-- !query 118 schema struct --- !query 115 output +-- !query 118 output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 --- !query 116 +-- !query 119 select tsval, tsval - interval '99 11:22:33.123456789' day to second, @@ -1082,58 +1142,58 @@ select -interval '99 11:22:33.123456789' day to second + tsval, interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic --- !query 116 schema +-- !query 119 schema struct --- !query 116 output +-- !query 119 output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 --- !query 117 +-- !query 120 select interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic --- !query 117 schema +-- !query 120 schema struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> --- !query 117 output +-- !query 120 output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds --- !query 118 +-- !query 121 select interval '\t interval 1 day' --- !query 118 schema +-- !query 121 schema struct --- !query 118 output +-- !query 121 output 1 days --- !query 119 +-- !query 122 select interval 'interval \t 1\tday' --- !query 119 schema +-- !query 122 schema struct --- !query 119 output +-- !query 122 output 1 days --- !query 120 +-- !query 123 select interval 'interval\t1\tday' --- !query 120 schema +-- !query 123 schema struct --- !query 120 output +-- !query 123 output 1 days --- !query 121 +-- !query 124 select interval '1\t' day --- !query 121 schema +-- !query 124 schema struct --- !query 121 output +-- !query 124 output 1 days --- !query 122 +-- !query 125 select interval '1 ' day --- !query 122 schema +-- !query 125 schema struct --- !query 122 output +-- !query 125 output 1 days diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index f23415a108ff0..bf911dd3fbe6b 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -125,17 +125,29 @@ struct -- !query 15 SELECT interval '1 2:03' day to hour -- !query 15 schema -struct +struct<> -- !query 15 output -1 days 2 hours +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 1 2:03(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03' day to hour +----------------^^^ -- !query 16 SELECT interval '1 2:03:04' day to hour -- !query 16 schema -struct +struct<> -- !query 16 output -1 days 2 hours +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 1 2:03:04(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03:04' day to hour +----------------^^^ -- !query 17 @@ -149,17 +161,29 @@ struct -- !query 18 SELECT interval '1 2:03:04' day to minute -- !query 18 schema -struct +struct<> -- !query 18 output -1 days 2 hours 3 minutes +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 1 2:03:04(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03:04' day to minute +----------------^^^ -- !query 19 SELECT interval '1 2:03' day to second -- !query 19 schema -struct +struct<> -- !query 19 output -1 days 2 hours 3 minutes +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03' day to second +----------------^^^ -- !query 20 @@ -173,49 +197,85 @@ struct -- !query 21 SELECT interval '1 2:03' hour to minute -- !query 21 schema -struct +struct<> -- !query 21 output -1 days 2 hours 3 minutes +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 1 2:03(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03' hour to minute +----------------^^^ -- !query 22 SELECT interval '1 2:03:04' hour to minute -- !query 22 schema -struct +struct<> -- !query 22 output -1 days 2 hours 3 minutes +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 1 2:03:04(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03:04' hour to minute +----------------^^^ -- !query 23 SELECT interval '1 2:03' hour to second -- !query 23 schema -struct +struct<> -- !query 23 output -1 days 2 hours 3 minutes +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03' hour to second +----------------^^^ -- !query 24 SELECT interval '1 2:03:04' hour to second -- !query 24 schema -struct +struct<> -- !query 24 output -1 days 2 hours 3 minutes 4 seconds +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03:04(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03:04' hour to second +----------------^^^ -- !query 25 SELECT interval '1 2:03' minute to second -- !query 25 schema -struct +struct<> -- !query 25 output -1 days 2 minutes 3 seconds +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03' minute to second +----------------^^^ -- !query 26 SELECT interval '1 2:03:04' minute to second -- !query 26 schema -struct +struct<> -- !query 26 output -1 days 2 hours 3 minutes 4 seconds +org.apache.spark.sql.catalyst.parser.ParseException + +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03:04(line 1, pos 16) + +== SQL == +SELECT interval '1 2:03:04' minute to second +----------------^^^ -- !query 27 From e39bb4c9fdeba05ee16c363f2183421fa49578c2 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 11 Dec 2019 09:23:39 -0800 Subject: [PATCH 0850/1387] [MINOR][SS][DOC] Fix the ss-kafka doc for availability of 'minPartitions' option ### What changes were proposed in this pull request? This patch fixes the availability of `minPartitions` option for Kafka source, as it is only supported by micro-batch for now. There's a WIP PR for batch (#25436) as well but there's no progress on the PR so far, so safer to fix the doc first, and let it be added later when we address it with batch case as well. ### Why are the changes needed? The doc is wrong and misleading. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Just a doc change. Closes #26849 from HeartSaVioR/MINOR-FIX-minPartition-availability-doc. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- docs/structured-streaming-kafka-integration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 8c17de92f3487..eb9e186e961a9 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -469,7 +469,7 @@ The following configurations are optional: minPartitions int none - streaming and batch + streaming Desired minimum number of partitions to read from Kafka. By default, Spark has a 1-1 mapping of topicPartitions to Spark partitions consuming from Kafka. If you set this option to a value greater than your topicPartitions, Spark will divvy up large From 33f53cb2d51b62f4c294c8640dc069e42f36d686 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 11 Dec 2019 12:33:58 -0800 Subject: [PATCH 0851/1387] [SPARK-30195][SQL][CORE][ML] Change some function, import definitions to work with stricter compiler in Scala 2.13 ### What changes were proposed in this pull request? See https://issues.apache.org/jira/browse/SPARK-30195 for the background; I won't repeat it here. This is sort of a grab-bag of related issues. ### Why are the changes needed? To cross-compile with Scala 2.13 later. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests for 2.12. I've been manually checking that this actually resolves the compile problems in 2.13 separately. Closes #26826 from srowen/SPARK-30195. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../spark/storage/BlockManagerMaster.scala | 4 ++-- .../apache/spark/InternalAccumulatorSuite.scala | 2 +- .../scheduler/OutputCommitCoordinatorSuite.scala | 2 +- .../apache/spark/util/ClosureCleanerSuite.scala | 2 +- .../org/apache/spark/ml/linalg/Matrices.scala | 15 +++++++-------- .../spark/ml/feature/CountVectorizer.scala | 4 ++-- .../scala/org/apache/spark/ml/fpm/FPGrowth.scala | 2 +- .../spark/mllib/clustering/GaussianMixture.scala | 5 ++++- .../apache/spark/mllib/clustering/KMeans.scala | 6 +++--- .../apache/spark/mllib/feature/Word2Vec.scala | 6 +++--- .../org/apache/spark/mllib/fpm/PrefixSpan.scala | 6 +++--- .../org/apache/spark/mllib/linalg/Matrices.scala | 13 ++++++------- .../expressions/aggregate/PivotFirst.scala | 2 +- .../catalyst/expressions/objects/objects.scala | 6 +++--- .../apache/spark/sql/DataFrameNaFunctions.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 8 ++++---- .../datasources/orc/OrcFileFormat.scala | 2 +- .../sql/execution/ui/SQLAppStatusListener.scala | 2 +- .../apache/spark/sql/expressions/Window.scala | 2 +- .../org/apache/spark/sql/DatasetCacheSuite.scala | 2 +- .../apache/spark/sql/DatasetPrimitiveSuite.scala | 4 ++-- .../org/apache/spark/sql/DatasetSuite.scala | 16 ++++++++-------- .../scala/org/apache/spark/sql/JoinSuite.scala | 4 ++-- .../compression/IntegralDeltaSuite.scala | 8 +++----- .../compression/PassThroughEncodingSuite.scala | 6 ++---- .../spark/streaming/ui/StreamingPage.scala | 3 +-- .../spark/streaming/StreamingContextSuite.scala | 2 +- 27 files changed, 66 insertions(+), 70 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 9678c917882cd..e335eb6ddb761 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -17,8 +17,8 @@ package org.apache.spark.storage -import scala.collection.Iterable import scala.collection.generic.CanBuildFrom +import scala.collection.immutable.Iterable import scala.concurrent.Future import org.apache.spark.{SparkConf, SparkException} @@ -201,7 +201,7 @@ class BlockManagerMaster( Option[BlockStatus], Iterable[Option[BlockStatus]]]] val blockStatus = timeout.awaitResult( - Future.sequence[Option[BlockStatus], Iterable](futures)(cbf, ThreadUtils.sameThread)) + Future.sequence(futures)(cbf, ThreadUtils.sameThread)) if (blockStatus == null) { throw new SparkException("BlockManager returned null for BlockStatus query: " + blockId) } diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index b982626ef0f7a..5399d868f46f1 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -90,7 +90,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { TaskContext.get().taskMetrics().testAccum.get.add(1) iter } - .reduceByKey { case (x, y) => x + y } + .reduceByKey { (x, y) => x + y } .mapPartitions { iter => TaskContext.get().taskMetrics().testAccum.get.add(10) iter diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 1db1a7fe8e27f..728b9d65054ec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -251,7 +251,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { // stage so that we can check the state of the output committer. val retriedStage = sc.parallelize(1 to 100, 10) .map { i => (i % 10, i) } - .reduceByKey { case (_, _) => + .reduceByKey { (_, _) => val ctx = TaskContext.get() if (ctx.stageAttemptNumber() == 0) { throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 1, 1L, 1, 1, diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 5e08a3dc1181d..b0520c7ab1b1f 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -299,7 +299,7 @@ private object TestUserClosuresActuallyCleaned { rdd.aggregateByKey(0)({ case (_, _) => return; 1 }, { case (_, _) => return; 1 }).count() } def testFoldByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.foldByKey(0) { case (_, _) => return; 1 } } - def testReduceByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.reduceByKey { case (_, _) => return; 1 } } + def testReduceByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.reduceByKey { (_, _) => return; 1 } } def testReduceByKeyLocally(rdd: RDD[(Int, Int)]): Unit = { rdd.reduceByKeyLocally { case (_, _) => return; 1 } } diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala index 14428c6f45cce..61d35c8f7e303 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -1175,10 +1175,10 @@ object Matrices { numCols += mat.numCols } if (!hasSparse) { - new DenseMatrix(numRows, numCols, matrices.flatMap(_.toArray)) + new DenseMatrix(numRows, numCols, matrices.flatMap { m: Matrix => m.toArray }) } else { var startCol = 0 - val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat => + val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat: Matrix => val nCols = mat.numCols mat match { case spMat: SparseMatrix => @@ -1189,7 +1189,7 @@ object Matrices { cnt += 1 } startCol += nCols - data + data.toSeq case dnMat: DenseMatrix => val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => @@ -1198,7 +1198,7 @@ object Matrices { } } startCol += nCols - data + data.toSeq } } SparseMatrix.fromCOO(numRows, numCols, entries) @@ -1237,7 +1237,6 @@ object Matrices { val allValues = new Array[Double](numRows * numCols) var startRow = 0 matrices.foreach { mat => - var j = 0 val nRows = mat.numRows mat.foreachActive { (i, j, v) => val indStart = j * numRows + startRow @@ -1248,7 +1247,7 @@ object Matrices { new DenseMatrix(numRows, numCols, allValues) } else { var startRow = 0 - val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat => + val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat: Matrix => val nRows = mat.numRows mat match { case spMat: SparseMatrix => @@ -1259,7 +1258,7 @@ object Matrices { cnt += 1 } startRow += nRows - data + data.toSeq case dnMat: DenseMatrix => val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => @@ -1268,7 +1267,7 @@ object Matrices { } } startRow += nRows - data + data.toSeq } } SparseMatrix.fromCOO(numRows, numCols, entries) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index 7ba6f640b1e49..6d39f18df60ea 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -215,8 +215,8 @@ class CountVectorizer @Since("1.5.0") (@Since("1.5.0") override val uid: String) wc.changeValue(w, 1L, _ + 1L) } wc.map { case (word, count) => (word, (count, 1)) } - }.reduceByKey { case ((wc1, df1), (wc2, df2)) => - (wc1 + wc2, df1 + df2) + }.reduceByKey { (wcdf1, wcdf2) => + (wcdf1._1 + wcdf2._1, wcdf1._2 + wcdf2._2) } val filteringRequired = isSet(minDF) || isSet(maxDF) diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala index a9592dbfca068..4d001c159eda0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -286,7 +286,7 @@ class FPGrowthModel private[ml] ( val dt = dataset.schema($(itemsCol)).dataType // For each rule, examine the input items and summarize the consequents - val predictUDF = udf((items: Seq[_]) => { + val predictUDF = udf((items: Seq[Any]) => { if (items != null) { val itemset = items.toSet brRules.value.filter(_._1.forall(itemset.contains)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index cde23096b40a4..4d98ba41bbb7b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -272,7 +272,10 @@ class GaussianMixture private ( private def initCovariance(x: IndexedSeq[BV[Double]]): BreezeMatrix[Double] = { val mu = vectorMean(x) val ss = BDV.zeros[Double](x(0).length) - x.foreach(xi => ss += (xi - mu) ^:^ 2.0) + x.foreach { xi => + val d: BV[Double] = xi - mu + ss += d ^:^ 2.0 + } diag(ss / x.length.toDouble) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index f456dd6395080..c4d9bb669ee24 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -309,9 +309,9 @@ class KMeans private ( clusterWeightSum.indices.filter(clusterWeightSum(_) > 0) .map(j => (j, (sums(j), clusterWeightSum(j)))).iterator - }.reduceByKey { case ((sum1, clusterWeightSum1), (sum2, clusterWeightSum2)) => - axpy(1.0, sum2, sum1) - (sum1, clusterWeightSum1 + clusterWeightSum2) + }.reduceByKey { (sumweight1, sumweight2) => + axpy(1.0, sumweight2._1, sumweight1._1) + (sumweight1._1, sumweight1._2 + sumweight2._2) }.collectAsMap() if (iteration == 0) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 640412f601b50..13899fa8296f6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -447,9 +447,9 @@ class Word2Vec extends Serializable with Logging { iter.map { case (id, vec) => (id, (vec, 1)) } - }.reduceByKey { case ((v1, count1), (v2, count2)) => - blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) - (v1, count1 + count2) + }.reduceByKey { (vc1, vc2) => + blas.saxpy(vectorSize, 1.0f, vc2._1, 1, vc1._1, 1) + (vc1._1, vc1._2 + vc2._2) }.map { case (id, (vec, count)) => blas.sscal(vectorSize, 1.0f / count, vec, 1) (id, vec) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index 69e4b76b2d8c2..ac2b576f4ac4e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -316,9 +316,9 @@ object PrefixSpan extends Logging { ((prefix.id, item), (1L, postfixSize)) } } - }.reduceByKey { case ((c0, s0), (c1, s1)) => - (c0 + c1, s0 + s1) - }.filter { case (_, (c, _)) => c >= minCount } + }.reduceByKey { (cs0, cs1) => + (cs0._1 + cs1._1, cs0._2 + cs1._2) + }.filter { case (_, cs) => cs._1 >= minCount } .collect() val newLargePrefixes = mutable.Map.empty[Int, Prefix] freqPrefixes.foreach { case ((id, item), (count, projDBSize)) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 0304fd88dcd9f..0a15e20934c37 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -1130,7 +1130,7 @@ object Matrices { new DenseMatrix(numRows, numCols, matrices.flatMap(_.toArray)) } else { var startCol = 0 - val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat => + val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat: Matrix => val nCols = mat.numCols mat match { case spMat: SparseMatrix => @@ -1141,7 +1141,7 @@ object Matrices { cnt += 1 } startCol += nCols - data + data.toSeq case dnMat: DenseMatrix => val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => @@ -1150,7 +1150,7 @@ object Matrices { } } startCol += nCols - data + data.toSeq } } SparseMatrix.fromCOO(numRows, numCols, entries) @@ -1189,7 +1189,6 @@ object Matrices { val allValues = new Array[Double](numRows * numCols) var startRow = 0 matrices.foreach { mat => - var j = 0 val nRows = mat.numRows mat.foreachActive { (i, j, v) => val indStart = j * numRows + startRow @@ -1200,7 +1199,7 @@ object Matrices { new DenseMatrix(numRows, numCols, allValues) } else { var startRow = 0 - val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat => + val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat: Matrix => val nRows = mat.numRows mat match { case spMat: SparseMatrix => @@ -1211,7 +1210,7 @@ object Matrices { cnt += 1 } startRow += nRows - data + data.toSeq case dnMat: DenseMatrix => val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => @@ -1220,7 +1219,7 @@ object Matrices { } } startRow += nRows - data + data.toSeq } } SparseMatrix.fromCOO(numRows, numCols, entries) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala index 33bc5b5821b36..17471535873fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PivotFirst.scala @@ -83,7 +83,7 @@ case class PivotFirst( override val dataType: DataType = ArrayType(valueDataType) - val pivotIndex = if (pivotColumn.dataType.isInstanceOf[AtomicType]) { + val pivotIndex: Map[Any, Int] = if (pivotColumn.dataType.isInstanceOf[AtomicType]) { HashMap(pivotColumnValues.zipWithIndex: _*) } else { TreeMap(pivotColumnValues.zipWithIndex: _*)( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 871aba67cf132..3732bdcdc2a9d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -916,7 +916,7 @@ case class MapObjects private( ${classOf[Builder[_, _]].getName} $builder = $getBuilder; $builder.sizeHint($dataLength); """, - genValue => s"$builder.$$plus$$eq($genValue);", + (genValue: String) => s"$builder.$$plus$$eq($genValue);", s"(${cls.getName}) $builder.result();" ) case Some(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) => @@ -930,7 +930,7 @@ case class MapObjects private( val param = Try(cls.getConstructor(Integer.TYPE)).map(_ => dataLength).getOrElse("") s"${cls.getName} $builder = new ${cls.getName}($param);" }, - genValue => s"$builder.add($genValue);", + (genValue: String) => s"$builder.add($genValue);", s"$builder;" ) case None => @@ -940,7 +940,7 @@ case class MapObjects private( $convertedType[] $convertedArray = null; $convertedArray = $arrayConstructor; """, - genValue => s"$convertedArray[$loopIndex] = $genValue;", + (genValue: String) => s"$convertedArray[$loopIndex] = $genValue;", s"new ${classOf[GenericArrayData].getName}($convertedArray);" ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 2a86b65b8f79f..953db806258a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -452,7 +452,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * TODO: This can be optimized to use broadcast join when replacementMap is large. */ - private def replaceCol(col: StructField, replacementMap: Map[_, _]): Column = { + private def replaceCol[K, V](col: StructField, replacementMap: Map[K, V]): Column = { val keyExpr = df.col(col.name).expr def buildExpr(v: Any) = Cast(Literal(v), keyExpr.dataType) val branches = replacementMap.flatMap { case (source, target) => 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 8eb0e2262e670..2c706da44b914 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._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes -import org.apache.spark.sql.execution.adaptive.LogicalQueryStage +import org.apache.spark.sql.execution.aggregate.AggUtils import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec @@ -422,7 +422,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } - aggregate.AggUtils.planStreamingAggregation( + AggUtils.planStreamingAggregation( normalizedGroupingExpressions, aggregateExpressions.map(expr => expr.asInstanceOf[AggregateExpression]), rewrittenResultExpressions, @@ -517,13 +517,13 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { val aggregateOperator = if (functionsWithDistinct.isEmpty) { - aggregate.AggUtils.planAggregateWithoutDistinct( + AggUtils.planAggregateWithoutDistinct( normalizedGroupingExpressions, aggregateExpressions, resultExpressions, planLater(child)) } else { - aggregate.AggUtils.planAggregateWithOneDistinct( + AggUtils.planAggregateWithOneDistinct( normalizedGroupingExpressions, functionsWithDistinct, functionsWithoutDistinct, 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 12c81a1dcab56..fd791ce7c5e19 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 @@ -26,7 +26,7 @@ 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.{OrcUtils => _, _} import org.apache.orc.OrcConf.{COMPRESS, MAPRED_OUTPUT_SCHEMA} import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index f207d352573de..e1100c3e3a579 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -53,7 +53,7 @@ class SQLAppStatusListener( liveExecutions.isEmpty && stageMetrics.isEmpty } - kvstore.addTrigger(classOf[SQLExecutionUIData], conf.get(UI_RETAINED_EXECUTIONS)) { count => + kvstore.addTrigger(classOf[SQLExecutionUIData], conf.get[Int](UI_RETAINED_EXECUTIONS)) { count => cleanupExecutions(count) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index cd1c198ddebf0..d13baaedbaeff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.expressions import org.apache.spark.annotation.Stable import org.apache.spark.sql.Column -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.{WindowSpec => _, _} /** * Utility functions for defining window in DataFrames. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 5dba9aa5b1fee..dc6df13514976 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -97,7 +97,7 @@ class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimit test("persist and then groupBy columns asKey, map") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() val grouped = ds.groupByKey(_._1) - val agged = grouped.mapGroups { case (g, iter) => (g, iter.map(_._2).sum) } + val agged = grouped.mapGroups { (g, iter) => (g, iter.map(_._2).sum) } agged.persist() checkDataset( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index 91a8f0a26b360..124b58483d24f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -171,7 +171,7 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSparkSession { test("groupBy function, map") { val ds = Seq(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11).toDS() val grouped = ds.groupByKey(_ % 2) - val agged = grouped.mapGroups { case (g, iter) => + val agged = grouped.mapGroups { (g, iter) => val name = if (g == 0) "even" else "odd" (name, iter.size) } @@ -184,7 +184,7 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSparkSession { test("groupBy function, flatMap") { val ds = Seq("a", "b", "c", "xyz", "hello").toDS() val grouped = ds.groupByKey(_.length) - val agged = grouped.flatMapGroups { case (g, iter) => Iterator(g.toString, iter.mkString) } + val agged = grouped.flatMapGroups { (g, iter) => Iterator(g.toString, iter.mkString) } checkDatasetUnorderly( agged, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 91d4a1c3dc73e..a54c24c44997f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -520,7 +520,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { test("groupBy function, map") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() val grouped = ds.groupByKey(v => (v._1, "word")) - val agged = grouped.mapGroups { case (g, iter) => (g._1, iter.map(_._2).sum) } + val agged = grouped.mapGroups { (g, iter) => (g._1, iter.map(_._2).sum) } checkDatasetUnorderly( agged, @@ -530,7 +530,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { test("groupBy function, flatMap") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() val grouped = ds.groupByKey(v => (v._1, "word")) - val agged = grouped.flatMapGroups { case (g, iter) => + val agged = grouped.flatMapGroups { (g, iter) => Iterator(g._1, iter.map(_._2).sum.toString) } @@ -542,11 +542,11 @@ class DatasetSuite extends QueryTest with SharedSparkSession { test("groupBy function, mapValues, flatMap") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() val keyValue = ds.groupByKey(_._1).mapValues(_._2) - val agged = keyValue.mapGroups { case (g, iter) => (g, iter.sum) } + val agged = keyValue.mapGroups { (g, iter) => (g, iter.sum) } checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) val keyValue1 = ds.groupByKey(t => (t._1, "key")).mapValues(t => (t._2, "value")) - val agged1 = keyValue1.mapGroups { case (g, iter) => (g._1, iter.map(_._1).sum) } + val agged1 = keyValue1.mapGroups { (g, iter) => (g._1, iter.map(_._1).sum) } checkDataset(agged1, ("a", 30), ("b", 3), ("c", 1)) } @@ -907,7 +907,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { test("grouping key and grouped value has field with same name") { val ds = Seq(ClassData("a", 1), ClassData("a", 2)).toDS() val agged = ds.groupByKey(d => ClassNullableData(d.a, null)).mapGroups { - case (key, values) => key.a + values.map(_.b).sum + (key, values) => key.a + values.map(_.b).sum } checkDataset(agged, "a3") @@ -1002,7 +1002,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { .select("user", "item") .as[(Int, Int)] .groupByKey(_._1) - .mapGroups { case (src, ids) => (src, ids.map(_._2).toArray) } + .mapGroups { (src, ids) => (src, ids.map(_._2).toArray) } .toDF("id", "actual") dataset.join(actual, dataset("user") === actual("id")).collect() @@ -1269,10 +1269,10 @@ class DatasetSuite extends QueryTest with SharedSparkSession { checkDataset( df.withColumn("b", lit(0)).as[ClassData] - .groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() }) + .groupByKey(_.a).flatMapGroups { (_, _) => List[Int]() }) checkDataset( df.withColumn("b", expr("0")).as[ClassData] - .groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() }) + .groupByKey(_.a).flatMapGroups { (_, _) => List[Int]() }) } test("SPARK-18125: Spark generated code causes CompileException") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index bf6b1cab55e9e..ac7976090ef89 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -522,10 +522,10 @@ class JoinSuite extends QueryTest with SharedSparkSession { SQLConf.CROSS_JOINS_ENABLED.key -> "true") { assert(statisticSizeInByte(spark.table("testData2")) > - spark.conf.get(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD)) + spark.conf.get[Long](SQLConf.AUTO_BROADCASTJOIN_THRESHOLD)) assert(statisticSizeInByte(spark.table("testData")) < - spark.conf.get(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD)) + spark.conf.get[Long](SQLConf.AUTO_BROADCASTJOIN_THRESHOLD)) Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala index 294feb72ea802..b5630488b3667 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala @@ -34,7 +34,7 @@ class IntegralDeltaSuite extends SparkFunSuite { columnType: NativeColumnType[I], scheme: CompressionScheme): Unit = { - def skeleton(input: Seq[I#InternalType]): Unit = { + def skeleton(input: Seq[Any]): Unit = { // ------------- // Tests encoder // ------------- @@ -52,7 +52,7 @@ class IntegralDeltaSuite extends SparkFunSuite { input.foreach { value => val row = new GenericInternalRow(1) - columnType.setField(row, 0, value) + columnType.setField(row, 0, value.asInstanceOf[I#InternalType]) builder.appendFrom(row, 0) } @@ -173,9 +173,7 @@ class IntegralDeltaSuite extends SparkFunSuite { } test(s"$scheme: long random series") { - // Have to workaround with `Any` since no `ClassTag[I#JvmType]` available here. - val input = Array.fill[Any](10000)(makeRandomValue(columnType)) - skeleton(input.map(_.asInstanceOf[I#InternalType])) + skeleton(Seq.fill[I#InternalType](10000)(makeRandomValue(columnType))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala index 01dac2d1ad74c..f946a6779ec95 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala @@ -160,8 +160,7 @@ class PassThroughSuite extends SparkFunSuite { } test(s"$PassThrough with $typeName: long random series") { - val input = Array.fill[Any](10000)(makeRandomValue(columnType)) - skeleton(input.map(_.asInstanceOf[T#InternalType])) + skeleton(Seq.fill[T#InternalType](10000)(makeRandomValue(columnType))) } test(s"$PassThrough with $typeName: empty column for decompress()") { @@ -169,8 +168,7 @@ class PassThroughSuite extends SparkFunSuite { } test(s"$PassThrough with $typeName: long random series for decompress()") { - val input = Array.fill[Any](10000)(makeRandomValue(columnType)) - skeletonForDecompress(input.map(_.asInstanceOf[T#InternalType])) + skeletonForDecompress(Seq.fill[T#InternalType](10000)(makeRandomValue(columnType))) } test(s"$PassThrough with $typeName: simple case with null for decompress()") { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index f28fc21538066..31ebb4ce53c17 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -24,8 +24,7 @@ import scala.collection.mutable.ArrayBuffer import scala.xml.{Node, Unparsed} import org.apache.spark.internal.Logging -import org.apache.spark.ui._ -import org.apache.spark.ui.{UIUtils => SparkUIUtils} +import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} /** * A helper class to generate JavaScript and HTML for both timeline and histogram graphs. diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 48d278dd1c7b8..1d6637861511f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -984,7 +984,7 @@ object SlowTestReceiver { } /** Streaming application for testing DStream and RDD creation sites */ -package object testPackage extends Assertions { +object testPackage extends Assertions { def test(): Unit = { val conf = new SparkConf().setMaster("local").setAppName("CreationSite test") val ssc = new StreamingContext(conf, Milliseconds(100)) From 9cf9304e171aa03166957d2fc5dd3d2f14c94f9e Mon Sep 17 00:00:00 2001 From: Pablo Langa Date: Wed, 11 Dec 2019 14:02:58 -0800 Subject: [PATCH 0852/1387] [SPARK-30038][SQL] DESCRIBE FUNCTION should do multi-catalog resolution ### What changes were proposed in this pull request? Add DescribeFunctionsStatement and make DESCRIBE FUNCTIONS go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing DESCRIBE FUNCTIONS namespace.function ### Does this PR introduce any user-facing change? Yes. When running DESCRIBE FUNCTIONS namespace.function Spark fails the command if the current catalog is set to a v2 catalog. ### How was this patch tested? Unit tests. Closes #26840 from planga82/feature/SPARK-30038_DescribeFunction_V2Catalog. Authored-by: Pablo Langa Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/parser/AstBuilder.scala | 17 +++++++++++++++++ .../sql/catalyst/plans/logical/statements.scala | 7 +++++++ .../sql/catalyst/parser/DDLParserSuite.scala | 15 +++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 15 ++++++++++++++- .../spark/sql/execution/SparkSqlParser.scala | 16 ---------------- .../sql/connector/DataSourceV2SQLSuite.scala | 12 ++++++++++++ .../sql/execution/SparkSqlParserSuite.scala | 12 ------------ .../sql/hive/execution/HiveComparisonTest.scala | 2 +- 8 files changed, 66 insertions(+), 30 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 1beadc5e37801..d3df7e03962e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3360,6 +3360,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.key).map(visitTablePropertyKey)) } + /** + * Create a plan for a DESCRIBE FUNCTION statement. + */ + override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) { + import ctx._ + val functionName = + if (describeFuncName.STRING() != null) { + Seq(string(describeFuncName.STRING())) + } else if (describeFuncName.qualifiedName() != null) { + visitQualifiedName(describeFuncName.qualifiedName) + } else { + Seq(describeFuncName.getText) + } + DescribeFunctionStatement(functionName, EXTENDED != null) + } + + /** * Create a plan for a SHOW FUNCTIONS command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 13356bfd04ffd..9f9181b136cc6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -486,6 +486,13 @@ case class ShowTablePropertiesStatement( tableName: Seq[String], propertyKey: Option[String]) extends ParsedStatement +/** + * A DESCRIBE FUNCTION statement, as parsed from SQL + */ +case class DescribeFunctionStatement( + functionName: Seq[String], + isExtended: Boolean) extends ParsedStatement + /** * SHOW FUNCTIONS statement, as parsed from SQL */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index b0d9a00d653ce..2c2533ed88d0f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1799,6 +1799,21 @@ class DDLParserSuite extends AnalysisTest { ShowTablePropertiesStatement(Seq("a", "b", "c"), Some("propKey1"))) } + test("DESCRIBE FUNCTION") { + comparePlans( + parsePlan("DESC FUNCTION a"), + DescribeFunctionStatement(Seq("a"), false)) + comparePlans( + parsePlan("DESCRIBE FUNCTION a"), + DescribeFunctionStatement(Seq("a"), false)) + comparePlans( + parsePlan("DESCRIBE FUNCTION a.b.c"), + DescribeFunctionStatement(Seq("a", "b", "c"), false)) + comparePlans( + parsePlan("DESCRIBE FUNCTION EXTENDED a.b.c"), + DescribeFunctionStatement(Seq("a", "b", "c"), true)) + } + test("SHOW FUNCTIONS") { comparePlans( parsePlan("SHOW FUNCTIONS"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 53eb7dae2ca0a..b9bbc70c71ce6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -474,6 +474,19 @@ class ResolveSessionCatalog( tbl.asTableIdentifier, propertyKey) + case DescribeFunctionStatement(CatalogAndIdentifier(catalog, functionIdent), extended) => + val functionIdentifier = if (isSessionCatalog(catalog)) { + functionIdent.asMultipartIdentifier match { + case Seq(db, fn) => FunctionIdentifier(fn, Some(db)) + case Seq(fn) => FunctionIdentifier(fn, None) + case _ => + throw new AnalysisException(s"Unsupported function name '${functionIdent.quoted}'") + } + } else { + throw new AnalysisException ("DESCRIBE FUNCTION is only supported in v1 catalog") + } + DescribeFunctionCommand(functionIdentifier, extended) + case ShowFunctionsStatement(userScope, systemScope, pattern, fun) => val (database, function) = fun match { case Some(CatalogAndIdentifier(catalog, functionIdent)) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 37edad6b5a30d..88ed672c8105c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -220,22 +220,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) } - /** - * Create a plan for a DESCRIBE FUNCTION command. - */ - override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) { - import ctx._ - val functionName = - if (describeFuncName.STRING() != null) { - FunctionIdentifier(string(describeFuncName.STRING()), database = None) - } else if (describeFuncName.qualifiedName() != null) { - visitFunctionName(describeFuncName.qualifiedName) - } else { - FunctionIdentifier(describeFuncName.getText, database = None) - } - DescribeFunctionCommand(functionName, EXTENDED != null) - } - /** * Create a [[CreateFunctionCommand]] command. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 6675636c0e62f..50ec0d775ba6e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1799,6 +1799,18 @@ class DataSourceV2SQLSuite } } + test("DESCRIBE FUNCTION: only support session catalog") { + val e = intercept[AnalysisException] { + sql("DESCRIBE FUNCTION testcat.ns1.ns2.fun") + } + assert(e.message.contains("DESCRIBE FUNCTION is only supported in v1 catalog")) + + val e1 = intercept[AnalysisException] { + sql("DESCRIBE FUNCTION default.ns1.ns2.fun") + } + assert(e1.message.contains("Unsupported function name 'default.ns1.ns2.fun'")) + } + test("SHOW FUNCTIONS not valid v1 namespace") { val function = "testcat.ns1.ns2.fun" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index ddf3a1c312ea2..53263f6f5169b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -80,18 +80,6 @@ class SparkSqlParserSuite extends AnalysisTest { intercept("REFRESH", "Resource paths cannot be empty in REFRESH statements") } - test("describe function") { - assertEqual("describe function bar", - DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = false)) - assertEqual("describe function extended bar", - DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = true)) - assertEqual("describe function foo.bar", - DescribeFunctionCommand( - FunctionIdentifier("bar", database = Some("foo")), isExtended = false)) - assertEqual("describe function extended f.bar", - DescribeFunctionCommand(FunctionIdentifier("bar", database = Some("f")), isExtended = true)) - } - private def createTableUsing( table: String, database: Option[String] = None, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index b506c107d73d5..e1615f17a7bae 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -374,7 +374,7 @@ abstract class HiveComparisonTest // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && (!hiveQuery.logical.isInstanceOf[ShowFunctionsStatement]) && - (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunctionStatement]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && (!hiveQuery.logical.isInstanceOf[DescribeTableStatement]) && (!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) && From b4aeaf906fe1ece886a730ae7291384e297a3bfb Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 11 Dec 2019 14:58:21 -0800 Subject: [PATCH 0853/1387] [SPARK-30198][CORE] BytesToBytesMap does not grow internal long array as expected ### What changes were proposed in this pull request? This patch changes the condition to check if BytesToBytesMap should grow up its internal array. Specifically, it changes to compare by the capacity of the array, instead of its size. ### Why are the changes needed? One Spark job on our cluster hangs forever at BytesToBytesMap.safeLookup. After inspecting, the long array size is 536870912. Currently in BytesToBytesMap.append, we only grow the internal array if the size of the array is less than its MAX_CAPACITY that is 536870912. So in above case, the array can not be grown up, and safeLookup can not find an empty slot forever. But it is wrong because we use two array entries per key, so the array size is twice the capacity. We should compare the current capacity of the array, instead of its size. ### Does this PR introduce any user-facing change? No ### How was this patch tested? This issue only happens when loading big number of values into BytesToBytesMap, so it is hard to do unit test. This is tested manually with internal Spark job. Closes #26828 from viirya/fix-bytemap. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../java/org/apache/spark/unsafe/map/BytesToBytesMap.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index b15365fe54ad6..7590b72112c54 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -741,7 +741,9 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff longArray.set(pos * 2 + 1, keyHashcode); isDefined = true; - if (numKeys >= growthThreshold && longArray.size() < MAX_CAPACITY) { + // We use two array entries per key, so the array size is twice the capacity. + // We should compare the current capacity of the array, instead of its size. + if (numKeys >= growthThreshold && longArray.size() / 2 < MAX_CAPACITY) { try { growAndRehash(); } catch (SparkOutOfMemoryError oom) { From 40b9c895a4c64546b258e0079fc896baf4e78da7 Mon Sep 17 00:00:00 2001 From: Aaruna Date: Wed, 11 Dec 2019 18:21:36 -0800 Subject: [PATCH 0854/1387] [SPARK-30199][DSTREAM] Recover `spark.(ui|blockManager).port` from checkpoint ### What changes were proposed in this pull request? This PR aims to recover `spark.ui.port` and `spark.blockManager.port` from checkpoint like `spark.driver.port`. ### Why are the changes needed? When the user configures these values, we can respect them. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the newly added test cases. Closes #26827 from dongjoon-hyun/SPARK-30199. Authored-by: Aaruna Signed-off-by: Dongjoon Hyun --- .../apache/spark/streaming/Checkpoint.scala | 4 +++ .../spark/streaming/CheckpointSuite.scala | 27 +++++++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 8617434d4d740..5d81d36dfe357 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -55,6 +55,8 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) "spark.driver.bindAddress", "spark.driver.port", "spark.master", + "spark.ui.port", + "spark.blockManager.port", "spark.kubernetes.driver.pod.name", "spark.kubernetes.executor.podNamePrefix", "spark.yarn.jars", @@ -69,6 +71,8 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) .remove("spark.driver.host") .remove("spark.driver.bindAddress") .remove("spark.driver.port") + .remove("spark.ui.port") + .remove("spark.blockManager.port") .remove("spark.kubernetes.driver.pod.name") .remove("spark.kubernetes.executor.podNamePrefix") val newReloadConf = new SparkConf(loadDefaults = true) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 25c0796585366..238ef1e2367a0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -420,6 +420,33 @@ class CheckpointSuite extends TestSuiteBase with LocalStreamingContext with DStr assert(restoredConf1.get("spark.driver.port") !== "9999") } + test("SPARK-30199 get ui port and blockmanager port") { + val conf = Map("spark.ui.port" -> "30001", "spark.blockManager.port" -> "30002") + conf.foreach { case (k, v) => System.setProperty(k, v) } + ssc = new StreamingContext(master, framework, batchDuration) + conf.foreach { case (k, v) => assert(ssc.conf.get(k) === v) } + + val cp = new Checkpoint(ssc, Time(1000)) + ssc.stop() + + // Serialize/deserialize to simulate write to storage and reading it back + val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) + + val newCpConf = newCp.createSparkConf() + conf.foreach { case (k, v) => assert(newCpConf.contains(k) && newCpConf.get(k) === v) } + + // Check if all the parameters have been restored + ssc = new StreamingContext(null, newCp, null) + conf.foreach { case (k, v) => assert(ssc.conf.get(k) === v) } + ssc.stop() + + // If port numbers are not set in system property, these parameters should not be presented + // in the newly recovered conf. + conf.foreach(kv => System.clearProperty(kv._1)) + val newCpConf1 = newCp.createSparkConf() + conf.foreach { case (k, _) => assert(!newCpConf1.contains(k)) } + } + // This tests whether the system can recover from a master failure with simple // non-stateful operations. This assumes as reliable, replayable input // source - TestInputDStream. From 1ced6c15448503a899be07afdb7f605a01bd70d1 Mon Sep 17 00:00:00 2001 From: jiake Date: Wed, 11 Dec 2019 19:39:31 -0800 Subject: [PATCH 0855/1387] [SPARK-30213][SQL] Remove the mutable status in ShuffleQueryStageExec ### What changes were proposed in this pull request? Currently `ShuffleQueryStageExec `contain the mutable status, eg `mapOutputStatisticsFuture `variable. So It is not easy to pass when we copy `ShuffleQueryStageExec`. This PR will put the `mapOutputStatisticsFuture ` variable from `ShuffleQueryStageExec` to `ShuffleExchangeExec`. And then we can pass the value of `mapOutputStatisticsFuture ` when copying. ### Why are the changes needed? In order to remove the mutable status in `ShuffleQueryStageExec` ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing uts Closes #26846 from JkSelf/removeMutableVariable. Authored-by: jiake Signed-off-by: Dongjoon Hyun --- .../sql/execution/adaptive/QueryStageExec.scala | 15 ++++----------- .../adaptive/ReduceNumShufflePartitions.scala | 2 +- .../execution/exchange/ShuffleExchangeExec.scala | 11 +++++++++++ 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index aa4e77bf628d4..d8ee972442b79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -129,21 +129,14 @@ case class ShuffleQueryStageExec( override val id: Int, override val plan: ShuffleExchangeExec) extends QueryStageExec { - @transient lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { - if (plan.inputRDD.getNumPartitions == 0) { - Future.successful(null) - } else { - sparkContext.submitMapStage(plan.shuffleDependency) - } - } - override def doMaterialize(): Future[Any] = { - mapOutputStatisticsFuture + plan.mapOutputStatisticsFuture } override def cancel(): Unit = { - mapOutputStatisticsFuture match { - case action: FutureAction[MapOutputStatistics] if !mapOutputStatisticsFuture.isCompleted => + plan.mapOutputStatisticsFuture match { + case action: FutureAction[MapOutputStatistics] + if !plan.mapOutputStatisticsFuture.isCompleted => action.cancel() case _ => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index c7e29cc355925..0acbeaa251f78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -77,7 +77,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { plan } else { val shuffleMetrics = shuffleStages.map { stage => - val metricsFuture = stage.mapOutputStatisticsFuture + val metricsFuture = stage.plan.mapOutputStatisticsFuture assert(metricsFuture.isCompleted, "ShuffleQueryStageExec should already be ready") ThreadUtils.awaitResult(metricsFuture, Duration.Zero) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index b876183c78ec2..4281f01e2756a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.exchange import java.util.Random import java.util.function.Supplier +import scala.concurrent.Future + import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.rdd.RDD @@ -64,6 +66,15 @@ case class ShuffleExchangeExec( @transient lazy val inputRDD: RDD[InternalRow] = child.execute() + // 'mapOutputStatisticsFuture' is only needed when enable AQE. + @transient lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { + if (inputRDD.getNumPartitions == 0) { + Future.successful(null) + } else { + sparkContext.submitMapStage(shuffleDependency) + } + } + /** * A [[ShuffleDependency]] that will partition rows of its child based on * the partitioning scheme defined in `newPartitioning`. Those partitions of From b709091b4f488d4f08b0121e1a4c46e461ea032e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 12 Dec 2019 14:16:32 +0900 Subject: [PATCH 0856/1387] [SPARK-30228][BUILD] Update zstd-jni to 1.4.4-3 ### What changes were proposed in this pull request? This PR aims to update zstd-jni library to 1.4.4-3. ### Why are the changes needed? This will bring the latest bug fixes in zstd itself and some performance improvement. - https://github.com/facebook/zstd/releases/tag/v1.4.4 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins. Closes #26856 from dongjoon-hyun/SPARK-ZSTD-144. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 2 +- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 5fe29374f83ea..bd6ef3512c9ce 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -206,4 +206,4 @@ xmlenc-0.52.jar xz-1.5.jar zjsonpatch-0.3.0.jar zookeeper-3.4.14.jar -zstd-jni-1.4.3-1.jar +zstd-jni-1.4.4-3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 4028718d8d17b..d08acecb4e2b3 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -221,4 +221,4 @@ xmlenc-0.52.jar xz-1.5.jar zjsonpatch-0.3.0.jar zookeeper-3.4.14.jar -zstd-jni-1.4.3-1.jar +zstd-jni-1.4.4-3.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 244062c3ac239..8953d1e6cd5f0 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -237,4 +237,4 @@ xbean-asm7-shaded-4.15.jar xz-1.5.jar zjsonpatch-0.3.0.jar zookeeper-3.4.14.jar -zstd-jni-1.4.3-1.jar +zstd-jni-1.4.4-3.jar diff --git a/pom.xml b/pom.xml index 9907e5a18d3ab..5abad0b80e9ec 100644 --- a/pom.xml +++ b/pom.xml @@ -632,7 +632,7 @@ com.github.luben zstd-jni - 1.4.3-1 + 1.4.4-3 com.clearspring.analytics From 3741a36ebf326b56956289e06922d178982e4879 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 12 Dec 2019 14:47:20 +0800 Subject: [PATCH 0857/1387] [SPARK-30104][SQL][FOLLOWUP] V2 catalog named 'global_temp' should always be masked ### What changes were proposed in this pull request? This is a follow up to #26741 to address the following: 1. V2 catalog named `global_temp` should always be masked. 2. #26741 introduces `CatalogAndIdentifer` that supersedes `CatalogObjectIdentfier`. This PR removes `CatalogObjectIdentfier` and its usages and replace them with `CatalogAndIdentifer`. 3. `CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog)` and `CatalogObjectIdentifier(catalog, ident) if isSessionCatalog(catalog)` are replaced with `NonSessionCatalogAndIdentifier` and `SessionCatalogAndIdentifier` respectively. ### Why are the changes needed? To fix an existing with handling v2 catalog named `global_temp` and to simplify the code base. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added new tests. Closes #26853 from imback82/lookup_table. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/connector/catalog/LookupCatalog.scala | 21 ++++--------------- .../catalog/LookupCatalogSuite.scala | 14 +++++++++---- .../apache/spark/sql/DataFrameWriter.scala | 17 +++++++-------- .../apache/spark/sql/DataFrameWriterV2.scala | 4 ++-- .../sql/connector/DataSourceV2SQLSuite.scala | 12 +++++++++++ 5 files changed, 36 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 4d3aff22741ef..59e7805547c84 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -52,25 +52,12 @@ private[sql] trait LookupCatalog extends Logging { } } - /** - * Extract catalog and identifier from a multi-part identifier with the current catalog if needed. - */ - object CatalogObjectIdentifier { - def unapply(parts: Seq[String]): Some[(CatalogPlugin, Identifier)] = parts match { - case CatalogAndMultipartIdentifier(maybeCatalog, nameParts) => - Some(( - maybeCatalog.getOrElse(currentCatalog), - Identifier.of(nameParts.init.toArray, nameParts.last) - )) - } - } - /** * Extract session catalog and identifier from a multi-part identifier. */ object SessionCatalogAndIdentifier { def unapply(parts: Seq[String]): Option[(CatalogPlugin, Identifier)] = parts match { - case CatalogObjectIdentifier(catalog, ident) if CatalogV2Util.isSessionCatalog(catalog) => + case CatalogAndIdentifier(catalog, ident) if CatalogV2Util.isSessionCatalog(catalog) => Some(catalog, ident) case _ => None } @@ -81,7 +68,7 @@ private[sql] trait LookupCatalog extends Logging { */ object NonSessionCatalogAndIdentifier { def unapply(parts: Seq[String]): Option[(CatalogPlugin, Identifier)] = parts match { - case CatalogObjectIdentifier(catalog, ident) if !CatalogV2Util.isSessionCatalog(catalog) => + case CatalogAndIdentifier(catalog, ident) if !CatalogV2Util.isSessionCatalog(catalog) => Some(catalog, ident) case _ => None } @@ -117,7 +104,7 @@ private[sql] trait LookupCatalog extends Logging { assert(nameParts.nonEmpty) if (nameParts.length == 1) { Some((currentCatalog, Identifier.of(Array(), nameParts.head))) - } else if (nameParts.length == 2 && nameParts.head.equalsIgnoreCase(globalTempDB)) { + } else if (nameParts.head.equalsIgnoreCase(globalTempDB)) { // Conceptually global temp views are in a special reserved catalog. However, the v2 catalog // API does not support view yet, and we have to use v1 commands to deal with global temp // views. To simplify the implementation, we put global temp views in a special namespace @@ -139,7 +126,7 @@ private[sql] trait LookupCatalog extends Logging { /** * Extract legacy table identifier from a multi-part identifier. * - * For legacy support only. Please use [[CatalogObjectIdentifier]] instead on DSv2 code paths. + * For legacy support only. Please use [[CatalogAndIdentifier]] instead on DSv2 code paths. */ object AsTableIdentifier { def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index 513f7e0348d09..a576e66236abf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.FakeV2SessionCatalog import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.util.CaseInsensitiveStringMap private case class DummyCatalogPlugin(override val name: String) extends CatalogPlugin { @@ -36,7 +37,9 @@ private case class DummyCatalogPlugin(override val name: String) extends Catalog class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { import CatalystSqlParser._ - private val catalogs = Seq("prod", "test").map(x => x -> DummyCatalogPlugin(x)).toMap + private val globalTempDB = SQLConf.get.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + private val catalogs = + Seq("prod", "test", globalTempDB).map(x => x -> DummyCatalogPlugin(x)).toMap private val sessionCatalog = FakeV2SessionCatalog override val catalogManager: CatalogManager = { @@ -46,13 +49,16 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) }) when(manager.currentCatalog).thenReturn(sessionCatalog) + when(manager.v2SessionCatalog).thenReturn(sessionCatalog) manager } - test("catalog object identifier") { + test("catalog and identifier") { Seq( ("tbl", sessionCatalog, Seq.empty, "tbl"), ("db.tbl", sessionCatalog, Seq("db"), "tbl"), + (s"$globalTempDB.tbl", sessionCatalog, Seq(globalTempDB), "tbl"), + (s"$globalTempDB.ns1.ns2.tbl", sessionCatalog, Seq(globalTempDB, "ns1", "ns2"), "tbl"), ("prod.func", catalogs("prod"), Seq.empty, "func"), ("ns1.ns2.tbl", sessionCatalog, Seq("ns1", "ns2"), "tbl"), ("prod.db.tbl", catalogs("prod"), Seq("db"), "tbl"), @@ -64,7 +70,7 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { case (sql, expectedCatalog, namespace, name) => inside(parseMultipartIdentifier(sql)) { - case CatalogObjectIdentifier(catalog, ident) => + case CatalogAndIdentifier(catalog, ident) => catalog shouldEqual expectedCatalog ident shouldEqual Identifier.of(namespace.toArray, name) } @@ -156,7 +162,7 @@ class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog wit Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { case (sql, expectedCatalog, namespace, name) => inside(parseMultipartIdentifier(sql)) { - case CatalogObjectIdentifier(catalog, ident) => + case CatalogAndIdentifier(catalog, ident) => catalog shouldEqual expectedCatalog ident shouldEqual Identifier.of(namespace.toArray, name) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 92515a0210c67..2b124ae260ca4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -339,7 +339,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def insertInto(tableName: String): Unit = { - import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} + import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, NonSessionCatalogAndIdentifier, SessionCatalogAndIdentifier} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.CatalogV2Util._ @@ -357,11 +357,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val canUseV2 = lookupV2Provider().isDefined session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { - case CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog) => + case NonSessionCatalogAndIdentifier(catalog, ident) => insertInto(catalog, ident) - case CatalogObjectIdentifier(catalog, ident) - if isSessionCatalog(catalog) && canUseV2 && ident.namespace().length <= 1 => + case SessionCatalogAndIdentifier(catalog, ident) + if canUseV2 && ident.namespace().length <= 1 => insertInto(catalog, ident) case AsTableIdentifier(tableIdentifier) => @@ -479,19 +479,18 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * @since 1.4.0 */ def saveAsTable(tableName: String): Unit = { - import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, CatalogObjectIdentifier} + import df.sparkSession.sessionState.analyzer.{AsTableIdentifier, NonSessionCatalogAndIdentifier, SessionCatalogAndIdentifier} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - import org.apache.spark.sql.connector.catalog.CatalogV2Util._ val session = df.sparkSession val canUseV2 = lookupV2Provider().isDefined session.sessionState.sqlParser.parseMultipartIdentifier(tableName) match { - case CatalogObjectIdentifier(catalog, ident) if !isSessionCatalog(catalog) => + case NonSessionCatalogAndIdentifier(catalog, ident) => saveAsTable(catalog.asTableCatalog, ident) - case CatalogObjectIdentifier(catalog, ident) - if isSessionCatalog(catalog) && canUseV2 && ident.namespace().length <= 1 => + case SessionCatalogAndIdentifier(catalog, ident) + if canUseV2 && ident.namespace().length <= 1 => saveAsTable(catalog.asTableCatalog, ident) case AsTableIdentifier(tableIdentifier) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index cf534ab6b9e36..f0758809bd749 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -41,7 +41,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.catalog.CatalogV2Util._ - import df.sparkSession.sessionState.analyzer.CatalogObjectIdentifier + import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier private val df: DataFrame = ds.toDF() @@ -52,7 +52,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) private val (catalog, identifier) = { - val CatalogObjectIdentifier(catalog, identifier) = tableName + val CatalogAndIdentifier(catalog, identifier) = tableName (catalog.asTableCatalog, identifier) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 50ec0d775ba6e..4c0a472edbd9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1845,6 +1845,18 @@ class DataSourceV2SQLSuite } } + test("SPARK-30104: v2 catalog named global_temp will be masked") { + val globalTempDB = spark.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) + spark.conf.set(s"spark.sql.catalog.$globalTempDB", classOf[InMemoryTableCatalog].getName) + + val e = intercept[AnalysisException] { + // Since the following multi-part name starts with `globalTempDB`, it is resolved to + // the session catalog, not the `gloabl_temp` v2 catalog. + sql(s"CREATE TABLE $globalTempDB.ns1.ns2.tbl (id bigint, data string) USING json") + } + assert(e.message.contains("global_temp.ns1.ns2.tbl is not a valid TableIdentifier")) + } + test("table name same as catalog can be used") { withTable("testcat.testcat") { sql(s"CREATE TABLE testcat.testcat (id bigint, data string) USING foo") From 2936507f949030547cbe2bb310012b0f20f5e4da Mon Sep 17 00:00:00 2001 From: root1 Date: Thu, 12 Dec 2019 17:11:21 +0800 Subject: [PATCH 0858/1387] [SPARK-30150][SQL] ADD FILE, ADD JAR, LIST FILE & LIST JAR Command do not accept quoted path ### What changes were proposed in this pull request? `add file "abc.txt"` and `add file 'abc.txt'` are not supported. For these two spark sql gives `FileNotFoundException`. Only `add file abc.txt` is supported currently. After these changes path can be given as quoted text for ADD FILE, ADD JAR, LIST FILE, LIST JAR commands in spark-sql ### Why are the changes needed? In many of the spark-sql commands (like create table ,etc )we write path in quoted format only. To maintain this consistency we should support quoted format with this command as well. ### Does this PR introduce any user-facing change? Yes. Now users can write path with quotes. ### How was this patch tested? Manually tested. Closes #26779 from iRakson/SPARK-30150. Authored-by: root1 Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 7 ++++++- .../sql/execution/SparkSqlParserSuite.scala | 18 ++++++++++++++++++ 3 files changed, 25 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index cd9748eaa6f28..be482a641ccc6 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -222,7 +222,7 @@ statement multipartIdentifier partitionSpec? #loadData | TRUNCATE TABLE multipartIdentifier partitionSpec? #truncateTable | MSCK REPAIR TABLE multipartIdentifier #repairTable - | op=(ADD | LIST) identifier .*? #manageResource + | op=(ADD | LIST) identifier (STRING | .*?) #manageResource | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 88ed672c8105c..ce930e90c1ce9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -308,9 +308,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * ADD (FILE[S] | JAR[S] ) * LIST (FILE[S] [filepath ...] | JAR[S] [jarpath ...]) * }}} + * + * Note that filepath/jarpath can be given as follows; + * - /path/to/fileOrJar + * - "/path/to/fileOrJar" + * - '/path/to/fileOrJar' */ override def visitManageResource(ctx: ManageResourceContext): LogicalPlan = withOrigin(ctx) { - val mayebePaths = remainder(ctx.identifier).trim + val mayebePaths = if (ctx.STRING != null) string(ctx.STRING) else remainder(ctx.identifier).trim ctx.op.getType match { case SqlBaseParser.ADD => ctx.identifier.getText.toLowerCase(Locale.ROOT) match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 53263f6f5169b..56dceba06d157 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -233,4 +233,22 @@ class SparkSqlParserSuite extends AnalysisTest { parser.parsePlan("ALTER SCHEMA foo SET DBPROPERTIES ('x' = 'y')")) assertEqual("DESC DATABASE foo", parser.parsePlan("DESC SCHEMA foo")) } + + test("manage resources") { + assertEqual("ADD FILE abc.txt", AddFileCommand("abc.txt")) + assertEqual("ADD FILE 'abc.txt'", AddFileCommand("abc.txt")) + assertEqual("ADD FILE \"/path/to/abc.txt\"", AddFileCommand("/path/to/abc.txt")) + assertEqual("LIST FILE abc.txt", ListFilesCommand(Array("abc.txt"))) + assertEqual("LIST FILE '/path//abc.txt'", ListFilesCommand(Array("/path//abc.txt"))) + assertEqual("LIST FILE \"/path2/abc.txt\"", ListFilesCommand(Array("/path2/abc.txt"))) + assertEqual("ADD JAR /path2/_2/abc.jar", AddJarCommand("/path2/_2/abc.jar")) + assertEqual("ADD JAR '/test/path_2/jar/abc.jar'", AddJarCommand("/test/path_2/jar/abc.jar")) + assertEqual("ADD JAR \"abc.jar\"", AddJarCommand("abc.jar")) + assertEqual("LIST JAR /path-with-dash/abc.jar", + ListJarsCommand(Array("/path-with-dash/abc.jar"))) + assertEqual("LIST JAR 'abc.jar'", ListJarsCommand(Array("abc.jar"))) + assertEqual("LIST JAR \"abc.jar\"", ListJarsCommand(Array("abc.jar"))) + assertEqual("ADD FILE /path with space/abc.txt", AddFileCommand("/path with space/abc.txt")) + assertEqual("ADD JAR /path with space/abc.jar", AddJarCommand("/path with space/abc.jar")) + } } From 8e9bfea1070052ebdd20f4a19b53534533bed909 Mon Sep 17 00:00:00 2001 From: David Date: Thu, 12 Dec 2019 20:49:10 +0900 Subject: [PATCH 0859/1387] [SPARK-29188][PYTHON] toPandas (without Arrow) gets wrong dtypes when applied on empty DF ### What changes were proposed in this pull request? An empty Spark DataFrame converted to a Pandas DataFrame wouldn't have the right column types. Several type mappings were missing. ### Why are the changes needed? Empty Spark DataFrames can be used to write unit tests, and verified by converting them to Pandas first. But this can fail when the column types are wrong. ### Does this PR introduce any user-facing change? Yes; the error reported in the JIRA issue should not happen anymore. ### How was this patch tested? Through unit tests in `pyspark.sql.tests.test_dataframe.DataFrameTests#test_to_pandas_from_empty_dataframe` Closes #26747 from dlindelof/SPARK-29188. Authored-by: David Signed-off-by: HyukjinKwon --- python/pyspark/sql/dataframe.py | 14 +++++ python/pyspark/sql/tests/test_dataframe.py | 67 ++++++++++++++++++++++ 2 files changed, 81 insertions(+) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 4683c74277447..77c2bf7f5cd69 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -2110,6 +2110,7 @@ def toPandas(self): from pyspark.sql.utils import require_minimum_pandas_version require_minimum_pandas_version() + import numpy as np import pandas as pd if self.sql_ctx._conf.pandasRespectSessionTimeZone(): @@ -2190,6 +2191,11 @@ def toPandas(self): not(isinstance(field.dataType, IntegralType) and field.nullable and pdf[field.name].isnull().any()): dtype[field.name] = pandas_type + # Ensure we fall back to nullable numpy types, even when whole column is null: + if isinstance(field.dataType, IntegralType) and pdf[field.name].isnull().any(): + dtype[field.name] = np.float64 + if isinstance(field.dataType, BooleanType) and pdf[field.name].isnull().any(): + dtype[field.name] = np.object for f, t in dtype.items(): pdf[f] = pdf[f].astype(t, copy=False) @@ -2311,8 +2317,16 @@ def _to_corrected_pandas_type(dt): return np.int16 elif type(dt) == IntegerType: return np.int32 + elif type(dt) == LongType: + return np.int64 elif type(dt) == FloatType: return np.float32 + elif type(dt) == DoubleType: + return np.float64 + elif type(dt) == BooleanType: + return np.bool + elif type(dt) == TimestampType: + return np.datetime64 else: return None diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 90a5415c15c27..433a26d42ae98 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -547,6 +547,73 @@ def test_to_pandas_avoid_astype(self): self.assertEquals(types[1], np.object) self.assertEquals(types[2], np.float64) + @unittest.skipIf(not have_pandas, pandas_requirement_message) + def test_to_pandas_from_empty_dataframe(self): + # SPARK-29188 test that toPandas() on an empty dataframe has the correct dtypes + import numpy as np + sql = """ + SELECT CAST(1 AS TINYINT) AS tinyint, + CAST(1 AS SMALLINT) AS smallint, + CAST(1 AS INT) AS int, + CAST(1 AS BIGINT) AS bigint, + CAST(0 AS FLOAT) AS float, + CAST(0 AS DOUBLE) AS double, + CAST(1 AS BOOLEAN) AS boolean, + CAST('foo' AS STRING) AS string, + CAST('2019-01-01' AS TIMESTAMP) AS timestamp + """ + dtypes_when_nonempty_df = self.spark.sql(sql).toPandas().dtypes + dtypes_when_empty_df = self.spark.sql(sql).filter("False").toPandas().dtypes + self.assertTrue(np.all(dtypes_when_empty_df == dtypes_when_nonempty_df)) + + @unittest.skipIf(not have_pandas, pandas_requirement_message) + def test_to_pandas_from_null_dataframe(self): + # SPARK-29188 test that toPandas() on a dataframe with only nulls has correct dtypes + import numpy as np + sql = """ + SELECT CAST(NULL AS TINYINT) AS tinyint, + CAST(NULL AS SMALLINT) AS smallint, + CAST(NULL AS INT) AS int, + CAST(NULL AS BIGINT) AS bigint, + CAST(NULL AS FLOAT) AS float, + CAST(NULL AS DOUBLE) AS double, + CAST(NULL AS BOOLEAN) AS boolean, + CAST(NULL AS STRING) AS string, + CAST(NULL AS TIMESTAMP) AS timestamp + """ + pdf = self.spark.sql(sql).toPandas() + types = pdf.dtypes + self.assertEqual(types[0], np.float64) + self.assertEqual(types[1], np.float64) + self.assertEqual(types[2], np.float64) + self.assertEqual(types[3], np.float64) + self.assertEqual(types[4], np.float32) + self.assertEqual(types[5], np.float64) + self.assertEqual(types[6], np.object) + self.assertEqual(types[7], np.object) + self.assertTrue(np.can_cast(np.datetime64, types[8])) + + @unittest.skipIf(not have_pandas, pandas_requirement_message) + def test_to_pandas_from_mixed_dataframe(self): + # SPARK-29188 test that toPandas() on a dataframe with some nulls has correct dtypes + import numpy as np + sql = """ + SELECT CAST(col1 AS TINYINT) AS tinyint, + CAST(col2 AS SMALLINT) AS smallint, + CAST(col3 AS INT) AS int, + CAST(col4 AS BIGINT) AS bigint, + CAST(col5 AS FLOAT) AS float, + CAST(col6 AS DOUBLE) AS double, + CAST(col7 AS BOOLEAN) AS boolean, + CAST(col8 AS STRING) AS string, + CAST(col9 AS TIMESTAMP) AS timestamp + FROM VALUES (1, 1, 1, 1, 1, 1, 1, 1, 1), + (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL) + """ + pdf_with_some_nulls = self.spark.sql(sql).toPandas() + pdf_with_only_nulls = self.spark.sql(sql).filter('tinyint is null').toPandas() + self.assertTrue(np.all(pdf_with_only_nulls.dtypes == pdf_with_some_nulls.dtypes)) + def test_create_dataframe_from_array_of_long(self): import array data = [Row(longarray=array.array('l', [-9223372036854775808, 0, 9223372036854775807]))] From ce61ee89416ea2816f29e7feadd369424db0ff38 Mon Sep 17 00:00:00 2001 From: 07ARB Date: Thu, 12 Dec 2019 20:30:47 +0800 Subject: [PATCH 0860/1387] [SPARK-30126][CORE] support space in file path and name for addFile and addJar function ### What changes were proposed in this pull request? sparkContext.addFile and sparkContext.addJar fails when file path contains spaces ### Why are the changes needed? When uploading a file to the spark context via the addFile and addJar function, an exception is thrown when file path contains a space character. Escaping the space with %20 or or + doesn't change the result. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add test case. Closes #26773 from 07ARB/SPARK-30126. Authored-by: 07ARB Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/SparkContext.scala | 26 +++++---- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../org/apache/spark/SparkContextSuite.scala | 54 +++++++++++++++++++ 3 files changed, 70 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 96ca12b5f6c60..94a0ce78c9826 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1533,17 +1533,17 @@ class SparkContext(config: SparkConf) extends Logging { */ def addFile(path: String, recursive: Boolean): Unit = { val uri = new Path(path).toUri - val schemeCorrectedPath = uri.getScheme match { - case null => new File(path).getCanonicalFile.toURI.toString + val schemeCorrectedURI = uri.getScheme match { + case null => new File(path).getCanonicalFile.toURI case "local" => logWarning("File with 'local' scheme is not supported to add to file server, since " + "it is already available on every node.") return - case _ => path + case _ => uri } - val hadoopPath = new Path(schemeCorrectedPath) - val scheme = new URI(schemeCorrectedPath).getScheme + val hadoopPath = new Path(schemeCorrectedURI) + val scheme = schemeCorrectedURI.getScheme if (!Array("http", "https", "ftp").contains(scheme)) { val fs = hadoopPath.getFileSystem(hadoopConfiguration) val isDir = fs.getFileStatus(hadoopPath).isDirectory @@ -1563,7 +1563,11 @@ class SparkContext(config: SparkConf) extends Logging { val key = if (!isLocal && scheme == "file") { env.rpcEnv.fileServer.addFile(new File(uri.getPath)) } else { - schemeCorrectedPath + if (uri.getScheme == null) { + schemeCorrectedURI.toString + } else { + path + } } val timestamp = System.currentTimeMillis if (addedFiles.putIfAbsent(key, timestamp).isEmpty) { @@ -1856,7 +1860,7 @@ class SparkContext(config: SparkConf) extends Logging { def checkRemoteJarFile(path: String): String = { val hadoopPath = new Path(path) - val scheme = new URI(path).getScheme + val scheme = hadoopPath.toUri.getScheme if (!Array("http", "https", "ftp").contains(scheme)) { try { val fs = hadoopPath.getFileSystem(hadoopConfiguration) @@ -1878,21 +1882,21 @@ class SparkContext(config: SparkConf) extends Logging { } } - if (path == null) { - logWarning("null specified as parameter to addJar") + if (path == null || path.isEmpty) { + logWarning("null or empty path specified as parameter to addJar") } else { val key = if (path.contains("\\")) { // For local paths with backslashes on Windows, URI throws an exception addLocalJarFile(new File(path)) } else { - val uri = new URI(path) + val uri = new Path(path).toUri // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies Utils.validateURL(uri) uri.getScheme match { // A JAR file which exists only on the driver node case null => // SPARK-22585 path without schema is not url encoded - addLocalJarFile(new File(uri.getRawPath)) + addLocalJarFile(new File(uri.getPath)) // A JAR file which exists only on the driver node case "file" => addLocalJarFile(new File(uri.getPath)) // A JAR file which exists locally on every worker node 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 317d4b6407420..6b42815aa25ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -731,7 +731,7 @@ private[spark] object Utils extends Logging { case "file" => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. - val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) + val sourceFile = if (uri.isAbsolute) new File(uri) else new File(uri.getPath) copyFile(url, sourceFile, targetFile, fileOverwrite) case _ => val fs = getHadoopFileSystem(uri, hadoopConf) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 712ed9b539fc6..fe0a80fda544d 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -233,6 +233,42 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } + test("SPARK-30126: addFile when file path contains spaces with recursive works") { + withTempDir { dir => + try { + val sep = File.separator + val tmpDir = Utils.createTempDir(dir.getAbsolutePath + sep + "test space") + val tmpConfFile1 = File.createTempFile("test file", ".conf", tmpDir) + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(tmpConfFile1.getAbsolutePath, true) + + assert(sc.listFiles().size == 1) + assert(sc.listFiles().head.contains(new Path(tmpConfFile1.getName).toUri.toString)) + } finally { + sc.stop() + } + } + } + + test("SPARK-30126: addFile when file path contains spaces without recursive works") { + withTempDir { dir => + try { + val sep = File.separator + val tmpDir = Utils.createTempDir(dir.getAbsolutePath + sep + "test space") + val tmpConfFile2 = File.createTempFile("test file", ".conf", tmpDir) + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(tmpConfFile2.getAbsolutePath) + + assert(sc.listFiles().size == 1) + assert(sc.listFiles().head.contains(new Path(tmpConfFile2.getName).toUri.toString)) + } finally { + sc.stop() + } + } + } + test("addFile recursive can't add directories by default") { withTempDir { dir => try { @@ -294,6 +330,24 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } + test("SPARK-30126: add jar when path contains spaces") { + withTempDir { dir => + try { + val sep = File.separator + val tmpDir = Utils.createTempDir(dir.getAbsolutePath + sep + "test space") + val tmpJar = File.createTempFile("test", ".jar", tmpDir) + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(tmpJar.getAbsolutePath) + + assert(sc.listJars().size == 1) + assert(sc.listJars().head.contains(tmpJar.getName)) + } finally { + sc.stop() + } + } + } + test("add jar with invalid path") { withTempDir { tmpDir => val tmpJar = File.createTempFile("test", ".jar", tmpDir) From 25de90e762500e4dbb30e9e1262ec513c3756c62 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 12 Dec 2019 08:38:15 -0600 Subject: [PATCH 0861/1387] [SPARK-30170][SQL][MLLIB][TESTS] Eliminate compilation warnings: part 1 ### What changes were proposed in this pull request? - Replace `Seq[String]` by `Seq[_]` in `StopWordsRemoverSuite` because `String` type is unchecked due erasure. - Throw an exception for default case in `MLTest.checkNominalOnDF` because we don't expect other attribute types currently. - Explicitly cast float to double in `BigDecimal(y)`. This is what the `apply()` method does for `float`s. - Replace deprecated `verifyZeroInteractions` by `verifyNoInteractions`. - Equivalent replacement of `\0` by `\u0000` in `CSVExprUtilsSuite` - Import `scala.language.implicitConversions` in `CollectionExpressionsSuite`, `HashExpressionsSuite` and in `ExpressionParserSuite`. ### Why are the changes needed? The changes fix compiler warnings showed in the JIRA ticket https://issues.apache.org/jira/browse/SPARK-30170 . Eliminating the warning highlights other warnings which could take more attention to real problems. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites `StopWordsRemoverSuite`, `AnalysisExternalCatalogSuite`, `CSVExprUtilsSuite`, `CollectionExpressionsSuite`, `HashExpressionsSuite`, `ExpressionParserSuite` and sub-tests of `MLTest`. Closes #26799 from MaxGekk/eliminate-warning-2. Authored-by: Maxim Gekk Signed-off-by: Sean Owen --- .../apache/spark/ml/feature/StopWordsRemoverSuite.scala | 4 ++-- .../src/test/scala/org/apache/spark/ml/util/MLTest.scala | 2 ++ .../main/scala/org/apache/spark/sql/types/FloatType.scala | 8 ++++++-- .../catalyst/analysis/AnalysisExternalCatalogSuite.scala | 4 ++-- .../apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala | 2 +- .../catalyst/expressions/CollectionExpressionsSuite.scala | 1 + .../sql/catalyst/expressions/HashExpressionsSuite.scala | 1 + .../spark/sql/catalyst/parser/ExpressionParserSuite.scala | 2 ++ 8 files changed, 17 insertions(+), 7 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala index c142f83e05956..dc6fb31a1f8e4 100755 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala @@ -242,7 +242,7 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { remover.transform(df) .select("filtered1", "expected1", "filtered2", "expected2") .collect().foreach { - case Row(r1: Seq[String], e1: Seq[String], r2: Seq[String], e2: Seq[String]) => + case Row(r1: Seq[_], e1: Seq[_], r2: Seq[_], e2: Seq[_]) => assert(r1 === e1, s"The result value is not correct after bucketing. Expected $e1 but found $r1") assert(r2 === e2, @@ -268,7 +268,7 @@ class StopWordsRemoverSuite extends MLTest with DefaultReadWriteTest { remover.transform(df) .select("filtered1", "expected1", "filtered2", "expected2") .collect().foreach { - case Row(r1: Seq[String], e1: Seq[String], r2: Seq[String], e2: Seq[String]) => + case Row(r1: Seq[_], e1: Seq[_], r2: Seq[_], e2: Seq[_]) => assert(r1 === e1, s"The result value is not correct after bucketing. Expected $e1 but found $r1") assert(r2 === e2, diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala index 3e1e2ad6a7f55..410fbf297846f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala @@ -88,6 +88,8 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => val n = Attribute.fromStructField(dataframe.schema(colName)) match { case binAttr: BinaryAttribute => Some(2) case nomAttr: NominalAttribute => nomAttr.getNumValues + case unknown => + throw new IllegalArgumentException(s"Attribute type: ${unknown.getClass.getName}") } assert(n.isDefined && n.get === numValues, s"the number of values obtained from schema should be $numValues, but got $n") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala index 402c5e1472c32..1491f5904baef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -78,8 +78,12 @@ case object FloatType extends FloatType { } trait FloatAsIfIntegral extends FloatIsConflicted with Integral[Float] { - def quot(x: Float, y: Float): Float = (BigDecimal(x) quot BigDecimal(y)).floatValue - def rem(x: Float, y: Float): Float = (BigDecimal(x) remainder BigDecimal(y)).floatValue + def quot(x: Float, y: Float): Float = { + (BigDecimal(x.toDouble) quot BigDecimal(y.toDouble)).floatValue + } + def rem(x: Float, y: Float): Float = { + (BigDecimal(x.toDouble) remainder BigDecimal(y.toDouble)).floatValue + } } object FloatAsIfIntegral extends FloatAsIfIntegral { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala index 9b6e8841a9807..a85ac3fc4d0b7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala @@ -59,7 +59,7 @@ class AnalysisExternalCatalogSuite extends AnalysisTest with Matchers { Alias(UnresolvedFunction("sum", Seq(UnresolvedAttribute("a")), isDistinct = false), "s")() val plan = Project(Seq(func), testRelation) analyzer.execute(plan) - verifyZeroInteractions(catalog) + verifyNoInteractions(catalog) } } @@ -73,7 +73,7 @@ class AnalysisExternalCatalogSuite extends AnalysisTest with Matchers { ignoreIfExists = false) reset(externCatalog) catalog.functionExists(FunctionIdentifier("sum")) - verifyZeroInteractions(externCatalog) + verifyNoInteractions(externCatalog) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index 4696a578f2f91..06ffb4231062c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -78,7 +78,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { // null character, expressed in Unicode literal syntax ("""\u0000""", Some("\u0000"), None), // and specified directly - ("\0", Some("\u0000"), None) + ("\u0000", Some("\u0000"), None) ) test("should correctly produce separator strings, or exceptions, from input") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index b4343b648110f..cc9ebfe409426 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.util.TimeZone +import scala.language.implicitConversions import scala.util.Random import org.apache.spark.SparkFunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index 3a68847ecb1f4..7dc072722352f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets import java.time.{ZoneId, ZoneOffset} import scala.collection.mutable.ArrayBuffer +import scala.language.implicitConversions import org.apache.commons.codec.digest.DigestUtils import org.scalatest.exceptions.TestFailedException diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 4dccada3d5546..1bea1c254c0fc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -20,6 +20,8 @@ import java.sql.{Date, Timestamp} import java.time.LocalDateTime import java.util.concurrent.TimeUnit +import scala.language.implicitConversions + import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ From fd39b6db346d8cfe592fb97653cb68df4f6d6434 Mon Sep 17 00:00:00 2001 From: Aaron Lau Date: Thu, 12 Dec 2019 08:42:18 -0600 Subject: [PATCH 0862/1387] [SQL] Typo in HashedRelation error ### What changes were proposed in this pull request? Fixed typo in exception message of HashedRelations ### Why are the changes needed? Better exception messages ### Does this PR introduce any user-facing change? No ### How was this patch tested? No tests needed Closes #26822 from aaron-lau/master. Authored-by: Aaron Lau Signed-off-by: Sean Owen --- .../org/apache/spark/sql/execution/joins/HashedRelation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index efe03e0f9ab46..4001338662d53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -335,7 +335,7 @@ private[joins] object UnsafeHashedRelation { if (!success) { binaryMap.free() // scalastyle:off throwerror - throw new SparkOutOfMemoryError("There is no enough memory to build hash map") + throw new SparkOutOfMemoryError("There is not enough memory to build hash map") // scalastyle:on throwerror } } From cc087a3ac5591c43d6b861b69b10647594d21b89 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 12 Dec 2019 08:33:33 -0800 Subject: [PATCH 0863/1387] [SPARK-30162][SQL] Add PushedFilters to metadata in Parquet DSv2 implementation ### What changes were proposed in this pull request? This PR proposes to add `PushedFilters` into metadata to show the pushed filters in Parquet DSv2 implementation. In case of ORC, it is already added at https://github.com/apache/spark/pull/24719/files#diff-0fc82694b20da3cd2cbb07206920eef7R62-R64 ### Why are the changes needed? In order for users to be able to debug, and to match with ORC. ### Does this PR introduce any user-facing change? ```scala spark.range(10).write.mode("overwrite").parquet("/tmp/foo") spark.read.parquet("/tmp/foo").filter("5 > id").explain() ``` **Before:** ``` == Physical Plan == *(1) Project [id#20L] +- *(1) Filter (isnotnull(id#20L) AND (5 > id#20L)) +- *(1) ColumnarToRow +- BatchScan[id#20L] ParquetScan Location: InMemoryFileIndex[file:/tmp/foo], ReadSchema: struct ``` **After:** ``` == Physical Plan == *(1) Project [id#13L] +- *(1) Filter (isnotnull(id#13L) AND (5 > id#13L)) +- *(1) ColumnarToRow +- BatchScan[id#13L] ParquetScan Location: InMemoryFileIndex[file:/tmp/foo], ReadSchema: struct, PushedFilters: [IsNotNull(id), LessThan(id,5)] ``` ### How was this patch tested? Unittest were added and manually tested. Closes #26857 from HyukjinKwon/SPARK-30162. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../datasources/v2/parquet/ParquetScan.scala | 4 ++++ .../DataSourceScanExecRedactionSuite.scala | 24 ++++++++++++------- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala index 7e6ea41cf0b8c..cf16a174d9e22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala @@ -87,4 +87,8 @@ case class ParquetScan( } override def hashCode(): Int = getClass.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala index f1a3092a193f4..073aed8206ed7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala @@ -150,15 +150,21 @@ class DataSourceV2ScanExecRedactionSuite extends DataSourceScanRedactionTest { } test("FileScan description") { - withTempPath { path => - val dir = path.getCanonicalPath - spark.range(0, 10).write.orc(dir) - val df = spark.read.orc(dir) - - assert(isIncluded(df.queryExecution, "ReadSchema")) - assert(isIncluded(df.queryExecution, "BatchScan")) - assert(isIncluded(df.queryExecution, "PushedFilters")) - assert(isIncluded(df.queryExecution, "Location")) + Seq("json", "orc", "parquet").foreach { format => + withTempPath { path => + val dir = path.getCanonicalPath + spark.range(0, 10).write.format(format).save(dir) + val df = spark.read.format(format).load(dir) + + withClue(s"Source '$format':") { + assert(isIncluded(df.queryExecution, "ReadSchema")) + assert(isIncluded(df.queryExecution, "BatchScan")) + if (Seq("orc", "parquet").contains(format)) { + assert(isIncluded(df.queryExecution, "PushedFilters")) + } + assert(isIncluded(df.queryExecution, "Location")) + } + } } } } From 39c0696a393e9cc1e3c4d56d3e69cb4bdc529be7 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 12 Dec 2019 11:04:01 -0600 Subject: [PATCH 0864/1387] [MINOR] Fix google style guide address ### What changes were proposed in this pull request? This PR update google style guide address to `https://google.github.io/styleguide/javaguide.html`. ### Why are the changes needed? `https://google-styleguide.googlecode.com/svn-history/r130/trunk/javaguide.html` **404**: ![image](https://user-images.githubusercontent.com/5399861/70717915-431c9500-1d2a-11ea-895b-024be953a116.png) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Closes #26865 from wangyum/fix-google-styleguide. Authored-by: Yuming Wang Signed-off-by: Sean Owen --- dev/checkstyle.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml index 32ce96f5034d8..483fc7c01bd8f 100644 --- a/dev/checkstyle.xml +++ b/dev/checkstyle.xml @@ -24,7 +24,7 @@ Checkstyle configuration based on the Google coding conventions from: - Google Java Style - https://google-styleguide.googlecode.com/svn-history/r130/trunk/javaguide.html + https://google.github.io/styleguide/javaguide.html with Spark-specific changes from: From cada5beef72530fa699b5ec13d67261be37730e4 Mon Sep 17 00:00:00 2001 From: ulysses Date: Thu, 12 Dec 2019 09:52:27 -0800 Subject: [PATCH 0865/1387] [SPARK-30230][SQL] Like ESCAPE syntax can not use '_' and '%' ### What changes were proposed in this pull request? Since [25001](https://github.com/apache/spark/pull/25001), spark support like escape syntax. But '%' and '_' is the reserve char in `Like` expression. We can not use them as escape char. ### Why are the changes needed? Avoid some unexpect problem when using like escape syntax. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Add UT. Closes #26860 from ulysses-you/SPARK-30230. Authored-by: ulysses Signed-off-by: Gengliang Wang --- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 4 ++++ .../spark/sql/catalyst/parser/ExpressionParserSuite.scala | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d3df7e03962e0..671bbbc2e37bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1393,6 +1393,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } str.charAt(0) }.getOrElse('\\') + if ('%' == escapeChar || '_' == escapeChar) { + throw new ParseException("Invalid escape string." + + "Escape string can not be '%', '_'.", ctx) + } invertIfNotDefined(Like(e, expression(ctx.pattern), escapeChar)) case SqlBaseParser.RLIKE => invertIfNotDefined(RLike(e, expression(ctx.pattern))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 1bea1c254c0fc..9f11cecf26ae3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -198,6 +198,10 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("a not like 'pattern%' escape '\"'", !('a.like("pattern%", '\"'))) intercept("a not like 'pattern%' escape '\"/'", message) intercept("a not like 'pattern%' escape ''", message) + + val message2 = "Escape string can not be '%', '_'." + intercept("a like 'pattern%' escape '_'", message2) + intercept("a like 'pattern%' escape '%'", message2) } test("like expressions with ESCAPED_STRING_LITERALS = true") { From 982f72f4c3c6f5ebd939753b50f44038fd6a83ca Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 12 Dec 2019 13:07:20 -0800 Subject: [PATCH 0866/1387] [SPARK-30238][SQL] hive partition pruning can only support string and integral types ### What changes were proposed in this pull request? Check the partition column data type and only allow string and integral types in hive partition pruning. ### Why are the changes needed? Currently we only support string and integral types in hive partition pruning, but the check is done for literals. If the predicate is `InSet`, then there is no literal and we may pass an unsupported partition predicate to Hive and cause problems. ### Does this PR introduce any user-facing change? yes. fix a bug. A query fails before and can run now. ### How was this patch tested? a new test Closes #26871 from cloud-fan/bug. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../spark/sql/hive/client/HiveShim.scala | 14 ++++++++------ .../sql/hive/execution/SQLQuerySuite.scala | 18 ++++++++++++++++++ 2 files changed, 26 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 01ddaf2ee8ac1..d8078a534a874 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -703,7 +703,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } } - object NonVarcharAttribute { + object SupportedAttribute { // hive varchar is treated as catalyst string, but hive varchar can't be pushed down. private val varcharKeys = table.getPartitionKeys.asScala .filter(col => col.getType.startsWith(serdeConstants.VARCHAR_TYPE_NAME) || @@ -713,8 +713,10 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { def unapply(attr: Attribute): Option[String] = { if (varcharKeys.contains(attr.name)) { None - } else { + } else if (attr.dataType.isInstanceOf[IntegralType] || attr.dataType == StringType) { Some(attr.name) + } else { + None } } } @@ -737,20 +739,20 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } def convert(expr: Expression): Option[String] = expr match { - case In(ExtractAttribute(NonVarcharAttribute(name)), ExtractableLiterals(values)) + case In(ExtractAttribute(SupportedAttribute(name)), ExtractableLiterals(values)) if useAdvanced => Some(convertInToOr(name, values)) - case InSet(ExtractAttribute(NonVarcharAttribute(name)), ExtractableValues(values)) + case InSet(ExtractAttribute(SupportedAttribute(name)), ExtractableValues(values)) if useAdvanced => Some(convertInToOr(name, values)) case op @ SpecialBinaryComparison( - ExtractAttribute(NonVarcharAttribute(name)), ExtractableLiteral(value)) => + ExtractAttribute(SupportedAttribute(name)), ExtractableLiteral(value)) => Some(s"$name ${op.symbol} $value") case op @ SpecialBinaryComparison( - ExtractableLiteral(value), ExtractAttribute(NonVarcharAttribute(name))) => + ExtractableLiteral(value), ExtractAttribute(SupportedAttribute(name))) => Some(s"$value ${op.symbol} $name") case And(expr1, expr2) if useAdvanced => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 7dcd9dfac2861..b0adb04d44614 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -2473,4 +2473,22 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } } + + test("partition pruning should handle date correctly") { + withSQLConf(SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "2") { + withTable("t") { + sql("CREATE TABLE t (i INT) PARTITIONED BY (j DATE)") + sql("INSERT INTO t PARTITION(j='1990-11-11') SELECT 1") + checkAnswer(sql("SELECT i, CAST(j AS STRING) FROM t"), Row(1, "1990-11-11")) + checkAnswer( + sql( + """ + |SELECT i, CAST(j AS STRING) + |FROM t + |WHERE j IN (DATE'1990-11-10', DATE'1990-11-11', DATE'1990-11-12') + |""".stripMargin), + Row(1, "1990-11-11")) + } + } + } } From 5114389aef2cacaacc82e6025696b33d6d20b2a6 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 12 Dec 2019 13:40:46 -0800 Subject: [PATCH 0867/1387] [SPARK-30107][SQL] Expose nested schema pruning to all V2 sources ### What changes were proposed in this pull request? This PR exposes the existing logic for nested schema pruning to all sources, which is in line with the description of `SupportsPushDownRequiredColumns` . Right now, `SchemaPruning` (rule, not helper utility) is applied in the optimizer directly on certain instances of `Table` ignoring `SupportsPushDownRequiredColumns` that is part of `ScanBuilder`. I think it would be cleaner to perform schema pruning and filter push-down in one place. Therefore, this PR moves all the logic into `V2ScanRelationPushDown`. ### Why are the changes needed? This change allows all V2 data sources to benefit from nested column pruning (if they support it). ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This PR mostly relies on existing tests. On top, it adds one test to verify that top-level schema pruning works as well as one test for predicates with subqueries. Closes #26751 from aokolnychyi/nested-schema-pruning-ds-v2. Authored-by: Anton Okolnychyi Signed-off-by: Gengliang Wang --- .../execution/datasources/SchemaPruning.scala | 36 --------------- .../datasources/v2/FileScanBuilder.scala | 8 +++- .../datasources/v2/PushDownUtils.scala | 36 +++++++++++---- .../v2/V2ScanRelationPushDown.scala | 31 +++++++++---- .../datasources/v2/orc/OrcScanBuilder.scala | 2 + .../v2/parquet/ParquetScanBuilder.scala | 2 + .../spark/sql/FileBasedDataSourceSuite.scala | 17 +++++++ .../datasources/SchemaPruningSuite.scala | 44 +++++++++++++++++++ .../execution/datasources/csv/CSVSuite.scala | 18 ++++++++ 9 files changed, 140 insertions(+), 54 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala index c2211cccb501c..61e0154a0ffe8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala @@ -17,16 +17,12 @@ package org.apache.spark.sql.execution.datasources -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} -import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable -import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} @@ -58,21 +54,6 @@ object SchemaPruning extends Rule[LogicalPlan] { hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession) buildPrunedRelation(l, prunedHadoopRelation) }).getOrElse(op) - - case op @ PhysicalOperation(projects, filters, - d @ DataSourceV2Relation(table: FileTable, output, _)) if canPruneTable(table) => - - prunePhysicalColumns(output, projects, filters, table.dataSchema, - prunedDataSchema => { - val prunedFileTable = table match { - case o: OrcTable => o.copy(userSpecifiedSchema = Some(prunedDataSchema)) - case p: ParquetTable => p.copy(userSpecifiedSchema = Some(prunedDataSchema)) - case _ => - val message = s"${table.formatName} data source doesn't support schema pruning." - throw new AnalysisException(message) - } - buildPrunedRelationV2(d, prunedFileTable) - }).getOrElse(op) } /** @@ -119,12 +100,6 @@ object SchemaPruning extends Rule[LogicalPlan] { fsRelation.fileFormat.isInstanceOf[ParquetFileFormat] || fsRelation.fileFormat.isInstanceOf[OrcFileFormat] - /** - * Checks to see if the given [[FileTable]] can be pruned. Currently we support ORC v2. - */ - private def canPruneTable(table: FileTable) = - table.isInstanceOf[OrcTable] || table.isInstanceOf[ParquetTable] - /** * Normalizes the names of the attribute references in the given projects and filters to reflect * the names in the given logical relation. This makes it possible to compare attributes and @@ -191,17 +166,6 @@ object SchemaPruning extends Rule[LogicalPlan] { outputRelation.copy(relation = prunedBaseRelation, output = prunedOutput) } - /** - * Builds a pruned data source V2 relation from the output of the relation and the schema - * of the pruned [[FileTable]]. - */ - private def buildPrunedRelationV2( - outputRelation: DataSourceV2Relation, - prunedFileTable: FileTable) = { - val prunedOutput = getPrunedOutput(outputRelation.output, prunedFileTable.schema) - outputRelation.copy(table = prunedFileTable, output = prunedOutput) - } - // Prune the given output to make it consistent with `requiredSchema`. private def getPrunedOutput( output: Seq[AttributeReference], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala index 21bc14c577bdc..97874e8f4932e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala @@ -27,15 +27,21 @@ abstract class FileScanBuilder( dataSchema: StructType) extends ScanBuilder with SupportsPushDownRequiredColumns { private val partitionSchema = fileIndex.partitionSchema private val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + protected val supportsNestedSchemaPruning = false protected var requiredSchema = StructType(dataSchema.fields ++ partitionSchema.fields) override def pruneColumns(requiredSchema: StructType): Unit = { + // [SPARK-30107] While `requiredSchema` might have pruned nested columns, + // the actual data schema of this scan is determined in `readDataSchema`. + // File formats that don't support nested schema pruning, + // use `requiredSchema` as a reference and prune only top-level columns. this.requiredSchema = requiredSchema } protected def readDataSchema(): StructType = { val requiredNameSet = createRequiredNameSet() - val fields = dataSchema.fields.filter { field => + val schema = if (supportsNestedSchemaPruning) requiredSchema else dataSchema + val fields = schema.fields.filter { field => val colName = PartitioningUtils.getColName(field, isCaseSensitive) requiredNameSet.contains(colName) && !partitionNameSet.contains(colName) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala index 634ecfdf7e1d5..09a8a7ebb6ddc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, NamedExpression, PredicateHelper, SchemaPruning} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType object PushDownUtils extends PredicateHelper { /** @@ -76,23 +78,31 @@ object PushDownUtils extends PredicateHelper { * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), * and new output attributes after column pruning. */ - // TODO: nested column pruning. def pruneColumns( scanBuilder: ScanBuilder, relation: DataSourceV2Relation, - exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { + projects: Seq[NamedExpression], + filters: Seq[Expression]): (Scan, Seq[AttributeReference]) = { scanBuilder match { + case r: SupportsPushDownRequiredColumns if SQLConf.get.nestedSchemaPruningEnabled => + val rootFields = SchemaPruning.identifyRootFields(projects, filters) + val prunedSchema = if (rootFields.nonEmpty) { + SchemaPruning.pruneDataSchema(relation.schema, rootFields) + } else { + new StructType() + } + r.pruneColumns(prunedSchema) + val scan = r.build() + scan -> toOutputAttrs(scan.readSchema(), relation) + case r: SupportsPushDownRequiredColumns => + val exprs = projects ++ filters val requiredColumns = AttributeSet(exprs.flatMap(_.references)) val neededOutput = relation.output.filter(requiredColumns.contains) if (neededOutput != relation.output) { r.pruneColumns(neededOutput.toStructType) val scan = r.build() - val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap - scan -> scan.readSchema().toAttributes.map { - // We have to keep the attribute id during transformation. - a => a.withExprId(nameToAttr(a.name).exprId) - } + scan -> toOutputAttrs(scan.readSchema(), relation) } else { r.build() -> relation.output } @@ -100,4 +110,14 @@ object PushDownUtils extends PredicateHelper { case _ => scanBuilder.build() -> relation.output } } + + private def toOutputAttrs( + schema: StructType, + relation: DataSourceV2Relation): Seq[AttributeReference] = { + val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap + schema.toAttributes.map { + // we have to keep the attribute id during transformation + a => a.withExprId(nameToAttr(a.name).exprId) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index 76bb3c5bd8457..92acd3ba8d902 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.sql.catalyst.expressions.{And, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, ProjectionOverSchema, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule @@ -30,18 +30,22 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] { case ScanOperation(project, filters, relation: DataSourceV2Relation) => val scanBuilder = relation.table.asReadable.newScanBuilder(relation.options) - val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) - val normalizedFilters = DataSourceStrategy.normalizeExprs( - withoutSubquery, relation.output) + val normalizedFilters = DataSourceStrategy.normalizeExprs(filters, relation.output) + val (normalizedFiltersWithSubquery, normalizedFiltersWithoutSubquery) = + normalizedFilters.partition(SubqueryExpression.hasSubquery) // `pushedFilters` will be pushed down and evaluated in the underlying data sources. // `postScanFilters` need to be evaluated after the scan. // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. val (pushedFilters, postScanFiltersWithoutSubquery) = PushDownUtils.pushFilters( - scanBuilder, normalizedFilters) - val postScanFilters = postScanFiltersWithoutSubquery ++ withSubquery + scanBuilder, normalizedFiltersWithoutSubquery) + val postScanFilters = postScanFiltersWithoutSubquery ++ normalizedFiltersWithSubquery + + val normalizedProjects = DataSourceStrategy + .normalizeExprs(project, relation.output) + .asInstanceOf[Seq[NamedExpression]] val (scan, output) = PushDownUtils.pruneColumns( - scanBuilder, relation, project ++ postScanFilters) + scanBuilder, relation, normalizedProjects, postScanFilters) logInfo( s""" |Pushing operators to ${relation.name} @@ -52,11 +56,20 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] { val scanRelation = DataSourceV2ScanRelation(relation.table, scan, output) + val projectionOverSchema = ProjectionOverSchema(output.toStructType) + val projectionFunc = (expr: Expression) => expr transformDown { + case projectionOverSchema(newExpr) => newExpr + } + val filterCondition = postScanFilters.reduceLeftOption(And) - val withFilter = filterCondition.map(Filter(_, scanRelation)).getOrElse(scanRelation) + val newFilterCondition = filterCondition.map(projectionFunc) + val withFilter = newFilterCondition.map(Filter(_, scanRelation)).getOrElse(scanRelation) val withProjection = if (withFilter.output != project) { - Project(project, withFilter) + val newProjects = normalizedProjects + .map(projectionFunc) + .asInstanceOf[Seq[NamedExpression]] + Project(newProjects, withFilter) } else { withFilter } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 8d1d4ec45915b..1421ffd8b6de4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -43,6 +43,8 @@ case class OrcScanBuilder( sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) } + override protected val supportsNestedSchemaPruning: Boolean = true + override def build(): Scan = { OrcScan(sparkSession, hadoopConf, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options, pushedFilters()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala index 87db00077e798..2f861356e9499 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala @@ -56,6 +56,8 @@ case class ParquetScanBuilder( parquetFilters.convertibleFilters(this.filters).toArray } + override protected val supportsNestedSchemaPruning: Boolean = true + private var filters: Array[Filter] = Array.empty override def pushFilters(filters: Array[Filter]): Array[Filter] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index a7f3e81904de0..d4f76858af95f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -657,6 +657,23 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { } } + test("Return correct results when data columns overlap with partition columns (nested data)") { + Seq("parquet", "orc", "json").foreach { format => + withSQLConf(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + withTempPath { path => + val tablePath = new File(s"${path.getCanonicalPath}/c3=c/c1=a/c5=e") + + val inputDF = sql("SELECT 1 c1, 2 c2, 3 c3, named_struct('c4_1', 2, 'c4_2', 3) c4, 5 c5") + inputDF.write.format(format).save(tablePath.getCanonicalPath) + + val resultDF = spark.read.format(format).load(path.getCanonicalPath) + .select("c1", "c4.c4_1", "c5", "c3") + checkAnswer(resultDF, Row("a", 2, "e", "c")) + } + } + } + } + test("sizeInBytes should be the total size of all files") { Seq("orc", "").foreach { useV1SourceReaderList => withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> useV1SourceReaderList) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 8cb5c12d76c97..328a10704109c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -90,6 +90,36 @@ abstract class SchemaPruningSuite briefContacts.map { case BriefContact(id, name, address) => BriefContactWithDataPartitionColumn(id, name, address, 2) } + testSchemaPruning("select only top-level fields") { + val query = sql("select address from contacts") + checkScan(query, "struct") + checkAnswer(query.orderBy("id"), + Row("123 Main Street") :: + Row("321 Wall Street") :: + Row("567 Maple Drive") :: + Row("6242 Ash Street") :: + Nil) + } + + testSchemaPruning("select a single complex field with disabled nested schema pruning") { + withSQLConf(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> "false") { + val query = sql("select name.middle from contacts") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) + } + } + + testSchemaPruning("select only input_file_name()") { + val query = sql("select input_file_name() from contacts") + checkScan(query, "struct<>") + } + + testSchemaPruning("select only expressions without references") { + val query = sql("select count(*) from contacts") + checkScan(query, "struct<>") + checkAnswer(query, Row(4)) + } + testSchemaPruning("select a single complex field") { val query = sql("select name.middle from contacts") checkScan(query, "struct>") @@ -378,6 +408,20 @@ abstract class SchemaPruningSuite checkAnswer(query.orderBy("id"), Row(1) :: Nil) } + testMixedCaseQueryPruning("subquery filter with different-case column names") { + withTempView("temp") { + val spark = this.spark + import spark.implicits._ + + val df = Seq(2).toDF("col2") + df.createOrReplaceTempView("temp") + + val query = sql("select id from mixedcase where Col2.b IN (select col2 from temp)") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), Row(1) :: Nil) + } + } + // Tests schema pruning for a query whose column and field names are exactly the same as the table // schema's column and field names. N.B. this implies that `testThunk` should pass using either a // case-sensitive or case-insensitive query parser diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index a6e58cec10369..f6adc7acb2772 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2186,4 +2186,22 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { checkAnswer(readback, Row(timestamp)) } } + + test("return correct results when data columns overlap with partition columns") { + withTempPath { path => + val tablePath = new File(s"${path.getCanonicalPath}/cOl3=c/cOl1=a/cOl5=e") + + val inputDF = Seq((1, 2, 3, 4, 5)).toDF("cOl1", "cOl2", "cOl3", "cOl4", "cOl5") + inputDF.write + .option("header", "true") + .csv(tablePath.getCanonicalPath) + + val resultDF = spark.read + .option("header", "true") + .option("inferSchema", "true") + .csv(path.getCanonicalPath) + .select("CoL1", "Col2", "CoL5", "CoL3") + checkAnswer(resultDF, Row("a", 2, "e", "c")) + } + } } From cb6d2b3f836744b2b71e085949dd0ef485a4fa1a Mon Sep 17 00:00:00 2001 From: Pablo Langa Date: Thu, 12 Dec 2019 15:15:54 -0800 Subject: [PATCH 0868/1387] [SPARK-30040][SQL] DROP FUNCTION should do multi-catalog resolution ### What changes were proposed in this pull request? Add DropFunctionStatement and make DROP FUNCTION go through the same catalog/table resolution framework of v2 commands. ### Why are the changes needed? It's important to make all the commands have the same table resolution behavior, to avoid confusing DROP FUNCTION namespace.function ### Does this PR introduce any user-facing change? Yes. When running DROP FUNCTION namespace.function Spark fails the command if the current catalog is set to a v2 catalog. ### How was this patch tested? Unit tests. Closes #26854 from planga82/feature/SPARK-30040_DropFunctionV2Catalog. Authored-by: Pablo Langa Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/parser/AstBuilder.scala | 17 ++++++++- .../catalyst/plans/logical/statements.scala | 8 ++++ .../sql/catalyst/parser/DDLParserSuite.scala | 19 ++++++++++ .../analysis/ResolveSessionCatalog.scala | 13 +++++++ .../spark/sql/execution/SparkSqlParser.scala | 17 --------- .../sql/connector/DataSourceV2SQLSuite.scala | 12 ++++++ .../execution/command/DDLParserSuite.scala | 38 ------------------- 7 files changed, 68 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 671bbbc2e37bf..6816980947fe9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3380,7 +3380,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging DescribeFunctionStatement(functionName, EXTENDED != null) } - /** * Create a plan for a SHOW FUNCTIONS command. */ @@ -3396,4 +3395,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val functionName = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) ShowFunctionsStatement(userScope, systemScope, pattern, functionName) } + + /** + * Create a DROP FUNCTION statement. + * + * For example: + * {{{ + * DROP [TEMPORARY] FUNCTION [IF EXISTS] function; + * }}} + */ + override def visitDropFunction(ctx: DropFunctionContext): LogicalPlan = withOrigin(ctx) { + val functionName = visitMultipartIdentifier(ctx.multipartIdentifier) + DropFunctionStatement( + functionName, + ctx.EXISTS != null, + ctx.TEMPORARY != null) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 9f9181b136cc6..145a15cf3086e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -501,3 +501,11 @@ case class ShowFunctionsStatement( systemScope: Boolean, pattern: Option[String], functionName: Option[Seq[String]]) extends ParsedStatement + +/** + * DROP FUNCTION statement, as parsed from SQL + */ +case class DropFunctionStatement( + functionName: Seq[String], + ifExists: Boolean, + isTemp: Boolean) extends ParsedStatement diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 2c2533ed88d0f..e6d503b3a58f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1840,6 +1840,25 @@ class DDLParserSuite extends AnalysisTest { intercept(sql, s"$sql not supported") } + test("DROP FUNCTION") { + comparePlans( + parsePlan("DROP FUNCTION a"), + DropFunctionStatement(Seq("a"), false, false)) + comparePlans( + parsePlan("DROP FUNCTION a.b.c"), + DropFunctionStatement(Seq("a", "b", "c"), false, false)) + comparePlans( + parsePlan("DROP TEMPORARY FUNCTION a.b.c"), + DropFunctionStatement(Seq("a", "b", "c"), false, true)) + comparePlans( + parsePlan("DROP FUNCTION IF EXISTS a.b.c"), + DropFunctionStatement(Seq("a", "b", "c"), true, false)) + comparePlans( + parsePlan("DROP TEMPORARY FUNCTION IF EXISTS a.b.c"), + DropFunctionStatement(Seq("a", "b", "c"), true, true)) + + } + private case class TableSpec( name: Seq[String], schema: Option[StructType], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index b9bbc70c71ce6..6fa6e99d636ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -503,6 +503,19 @@ class ResolveSessionCatalog( case None => (None, pattern) } ShowFunctionsCommand(database, function, userScope, systemScope) + + case DropFunctionStatement(CatalogAndIdentifier(catalog, functionIdent), ifExists, isTemp) => + if (isSessionCatalog(catalog)) { + val (database, function) = functionIdent.asMultipartIdentifier match { + case Seq(db, fn) => (Some(db), fn) + case Seq(fn) => (None, fn) + case _ => + throw new AnalysisException(s"Unsupported function name '${functionIdent.quoted}'") + } + DropFunctionCommand(database, function, ifExists, isTemp) + } else { + throw new AnalysisException("DROP FUNCTION is only supported in v1 catalog") + } } private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index ce930e90c1ce9..6fcf62d26ad88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -252,23 +252,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ctx.REPLACE != null) } - /** - * Create a [[DropFunctionCommand]] command. - * - * For example: - * {{{ - * DROP [TEMPORARY] FUNCTION [IF EXISTS] function; - * }}} - */ - override def visitDropFunction(ctx: DropFunctionContext): LogicalPlan = withOrigin(ctx) { - val functionIdentifier = visitFunctionName(ctx.multipartIdentifier) - DropFunctionCommand( - functionIdentifier.database, - functionIdentifier.funcName, - ctx.EXISTS != null, - ctx.TEMPORARY != null) - } - /** * Convert a nested constants list into a sequence of string sequences. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 4c0a472edbd9f..d0c67ad63c37f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1820,6 +1820,18 @@ class DataSourceV2SQLSuite assert(e.message.contains("SHOW FUNCTIONS is only supported in v1 catalog")) } + test("DROP FUNCTION: only support session catalog") { + val e = intercept[AnalysisException] { + sql("DROP FUNCTION testcat.ns1.ns2.fun") + } + assert(e.message.contains("DROP FUNCTION is only supported in v1 catalog")) + + val e1 = intercept[AnalysisException] { + sql("DESCRIBE FUNCTION default.ns1.ns2.fun") + } + assert(e1.message.contains("Unsupported function name 'default.ns1.ns2.fun'")) + } + test("global temp view should not be masked by v2 catalog") { val globalTempDB = spark.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE) spark.conf.set(s"spark.sql.catalog.$globalTempDB", classOf[InMemoryTableCatalog].getName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index e486df930e9fa..823ba5d4892ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -170,44 +170,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { comparePlans(parsed5, expected5) } - test("drop function") { - val sql1 = "DROP TEMPORARY FUNCTION helloworld" - val sql2 = "DROP TEMPORARY FUNCTION IF EXISTS helloworld" - val sql3 = "DROP FUNCTION hello.world" - val sql4 = "DROP FUNCTION IF EXISTS hello.world" - - val parsed1 = parser.parsePlan(sql1) - val parsed2 = parser.parsePlan(sql2) - val parsed3 = parser.parsePlan(sql3) - val parsed4 = parser.parsePlan(sql4) - - val expected1 = DropFunctionCommand( - None, - "helloworld", - ifExists = false, - isTemp = true) - val expected2 = DropFunctionCommand( - None, - "helloworld", - ifExists = true, - isTemp = true) - val expected3 = DropFunctionCommand( - Some("hello"), - "world", - ifExists = false, - isTemp = false) - val expected4 = DropFunctionCommand( - Some("hello"), - "world", - ifExists = true, - isTemp = false) - - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - comparePlans(parsed3, expected3) - comparePlans(parsed4, expected4) - } - test("create hive table - table file format") { val allSources = Seq("parquet", "parquetfile", "orc", "orcfile", "avro", "avrofile", "sequencefile", "rcfile", "textfile") From 94eb66593a328dd3fcecc5f5f1772d82843ec14f Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 13 Dec 2019 16:12:41 +0800 Subject: [PATCH 0869/1387] [SPARK-30227][SQL] Add close() on DataWriter interface ### What changes were proposed in this pull request? This patch adds close() method to the DataWriter interface, which will become the place to cleanup the resource. ### Why are the changes needed? The lifecycle of DataWriter instance ends at either commit() or abort(). That makes datasource implementors to feel they can place resource cleanup in both sides, but abort() can be called when commit() fails; so they have to ensure they don't do double-cleanup if cleanup is not idempotent. ### Does this PR introduce any user-facing change? Depends on the definition of user; if they're developers of custom DSv2 source, they have to add close() in their DataWriter implementations. It's OK to just add close() with empty content as they should have already dealt with resource cleanup in commit/abort, but they would love to migrate the resource cleanup logic to close() as it avoids double cleanup. If they're just end users using the provided DSv2 source (regardless of built-in/3rd party), no change. ### How was this patch tested? Existing tests. Closes #26855 from HeartSaVioR/SPARK-30227. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Wenchen Fan --- .../spark/sql/kafka010/KafkaDataWriter.scala | 5 ++++- .../spark/sql/kafka010/KafkaSinkSuite.scala | 2 +- .../spark/sql/connector/write/DataWriter.java | 8 +++++--- .../spark/sql/connector/InMemoryTable.scala | 2 ++ .../datasources/FileFormatDataWriter.scala | 2 ++ .../datasources/FileFormatWriter.scala | 2 ++ .../datasources/noop/NoopDataSource.scala | 1 + .../v2/WriteToDataSourceV2Exec.scala | 2 ++ .../continuous/ContinuousWriteRDD.scala | 2 ++ .../streaming/sources/ForeachWriterTable.scala | 17 ++++++++--------- .../sources/PackedRowWriterFactory.scala | 10 ++++++---- .../execution/streaming/sources/memory.scala | 2 ++ .../connector/SimpleWritableDataSource.scala | 2 ++ 13 files changed, 39 insertions(+), 18 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala index 3f8d3d2da5797..9a2b369933616 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala @@ -63,7 +63,10 @@ private[kafka010] class KafkaDataWriter( def abort(): Unit = {} - def close(): Unit = { + def close(): Unit = {} + + /** explicitly invalidate producer from pool. only for testing. */ + private[kafka010] def invalidateProducer(): Unit = { checkForErrors() if (producer != null) { producer.flush() diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index e2dcd62005310..ac242ba3d1356 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -370,7 +370,7 @@ class KafkaContinuousSinkSuite extends KafkaSinkStreamingSuiteBase { iter.foreach(writeTask.write(_)) writeTask.commit() } finally { - writeTask.close() + writeTask.invalidateProducer() } } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java index eefe784dede43..59c69a18292d8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/DataWriter.java @@ -17,6 +17,7 @@ package org.apache.spark.sql.connector.write; +import java.io.Closeable; import java.io.IOException; import org.apache.spark.annotation.Evolving; @@ -31,8 +32,9 @@ * the {@link #write(Object)}, {@link #abort()} is called afterwards and the remaining records will * not be processed. If all records are successfully written, {@link #commit()} is called. * - * Once a data writer returns successfully from {@link #commit()} or {@link #abort()}, its lifecycle - * is over and Spark will not use it again. + * Once a data writer returns successfully from {@link #commit()} or {@link #abort()}, Spark will + * call {@link #close()} to let DataWriter doing resource cleanup. After calling {@link #close()}, + * its lifecycle is over and Spark will not use it again. * * If this data writer succeeds(all records are successfully written and {@link #commit()} * succeeds), a {@link WriterCommitMessage} will be sent to the driver side and pass to @@ -56,7 +58,7 @@ * Note that, Currently the type `T` can only be {@link org.apache.spark.sql.catalyst.InternalRow}. */ @Evolving -public interface DataWriter { +public interface DataWriter extends Closeable { /** * Writes one record. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index 201860e5135ba..09c4b9ab9d535 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -252,4 +252,6 @@ private class BufferWriter extends DataWriter[InternalRow] { override def commit(): WriterCommitMessage = buffer override def abort(): Unit = {} + + override def close(): Unit = {} } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index c1ebc98fb1ddd..50c4f6cd57a96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -86,6 +86,8 @@ abstract class FileFormatDataWriter( committer.abortTask(taskAttemptContext) } } + + override def close(): Unit = {} } /** FileFormatWriteTask for empty partitions */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index f02d9e92acb88..219c778b9164a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -277,6 +277,8 @@ object FileFormatWriter extends Logging { // If there is an error, abort the task dataWriter.abort() logError(s"Job $jobId aborted.") + }, finallyBlock = { + dataWriter.close() }) } catch { case e: FetchFailedException => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala index 03e5f43a2a0af..dd44651050e6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -72,6 +72,7 @@ private[noop] object NoopWriter extends DataWriter[InternalRow] { override def write(record: InternalRow): Unit = {} override def commit(): WriterCommitMessage = null override def abort(): Unit = {} + override def close(): Unit = {} } private[noop] object NoopStreamingWrite extends StreamingWrite { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 7d8a115c126eb..f4c70f7593b55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -467,6 +467,8 @@ object DataWritingSparkTask extends Logging { dataWriter.abort() logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " + s"stage $stageId.$stageAttempt)") + }, finallyBlock = { + dataWriter.close() }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala index 558b4313d6d88..909dda57ee586 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala @@ -80,6 +80,8 @@ class ContinuousWriteRDD(var prev: RDD[InternalRow], writerFactory: StreamingDat logError(s"Writer for partition ${context.partitionId()} is aborting.") if (dataWriter != null) dataWriter.abort() logError(s"Writer for partition ${context.partitionId()} aborted.") + }, finallyBlock = { + dataWriter.close() }) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala index 53d4bca1a5f7e..4793cb9a9b79c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala @@ -135,7 +135,7 @@ class ForeachDataWriter[T]( // If open returns false, we should skip writing rows. private val opened = writer.open(partitionId, epochId) - private var closeCalled: Boolean = false + private var errorOrNull: Throwable = _ override def write(record: InternalRow): Unit = { if (!opened) return @@ -144,25 +144,24 @@ class ForeachDataWriter[T]( writer.process(rowConverter(record)) } catch { case t: Throwable => - closeWriter(t) + errorOrNull = t throw t } + } override def commit(): WriterCommitMessage = { - closeWriter(null) ForeachWriterCommitMessage } override def abort(): Unit = { - closeWriter(new SparkException("Foreach writer has been aborted due to a task failure")) + if (errorOrNull == null) { + errorOrNull = new SparkException("Foreach writer has been aborted due to a task failure") + } } - private def closeWriter(errorOrNull: Throwable): Unit = { - if (!closeCalled) { - closeCalled = true - writer.close(errorOrNull) - } + override def close(): Unit = { + writer.close(errorOrNull) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala index 53f56edc27681..507f860e0452a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala @@ -56,10 +56,12 @@ class PackedRowDataWriter() extends DataWriter[InternalRow] with Logging { override def write(row: InternalRow): Unit = data.append(row.copy()) override def commit(): PackedRowCommitMessage = { - val msg = PackedRowCommitMessage(data.toArray) - data.clear() - msg + PackedRowCommitMessage(data.toArray) } - override def abort(): Unit = data.clear() + override def abort(): Unit = {} + + override def close(): Unit = { + data.clear() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala index a976876b4d8e4..0cc067fc76757 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala @@ -191,6 +191,8 @@ class MemoryDataWriter(partition: Int, schema: StructType) } override def abort(): Unit = {} + + override def close(): Unit = {} } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala index 306da996e2caa..a0f1a9f9f53f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala @@ -240,4 +240,6 @@ class CSVDataWriter(fs: FileSystem, file: Path) extends DataWriter[InternalRow] fs.delete(file, false) } } + + override def close(): Unit = {} } From 64c7b94d6471fca444fc06763defd8f12a1e1251 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 13 Dec 2019 17:44:23 +0900 Subject: [PATCH 0870/1387] [SPARK-30231][SQL][PYTHON] Support explain mode in PySpark df.explain ### What changes were proposed in this pull request? This pr intends to support explain modes implemented in #26829 for PySpark. ### Why are the changes needed? For better debugging info. in PySpark dataframes. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added UTs. Closes #26861 from maropu/ExplainModeInPython. Authored-by: Takeshi Yamamuro Signed-off-by: HyukjinKwon --- python/pyspark/sql/dataframe.py | 54 +++++++++++++++++-- .../scala/org/apache/spark/sql/Dataset.scala | 34 ++++++++---- .../org/apache/spark/sql/ExplainSuite.scala | 18 +++++++ 3 files changed, 92 insertions(+), 14 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 77c2bf7f5cd69..9f86086dad83d 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -253,10 +253,18 @@ def printSchema(self): print(self._jdf.schema().treeString()) @since(1.3) - def explain(self, extended=False): + def explain(self, extended=None, mode=None): """Prints the (logical and physical) plans to the console for debugging purpose. :param extended: boolean, default ``False``. If ``False``, prints only the physical plan. + :param mode: specifies the expected output format of plans. + + * ``simple``: Print only a physical plan. + * ``extended``: Print both logical and physical plans. + * ``codegen``: Print a physical plan and generated codes if they are available. + * ``cost``: Print a logical plan and statistics if they are available. + * ``formatted``: Split explain output into two sections: a physical plan outline \ + and node details. >>> df.explain() == Physical Plan == @@ -271,11 +279,47 @@ def explain(self, extended=False): ... == Physical Plan == ... + + >>> df.explain(mode="formatted") + == Physical Plan == + * Scan ExistingRDD (1) + (1) Scan ExistingRDD [codegen id : 1] + Output: [age#0, name#1] + + .. versionchanged:: 3.0.0 + Added optional argument `mode` to specify the expected output format of plans. """ - if extended: - print(self._jdf.queryExecution().toString()) - else: - print(self._jdf.queryExecution().simpleString()) + + if extended is not None and mode is not None: + raise Exception("extended and mode can not be specified simultaneously") + + # For the no argument case: df.explain() + is_no_argument = extended is None and mode is None + + # For the cases below: + # explain(True) + # explain(extended=False) + is_extended_case = extended is not None and isinstance(extended, bool) + + # For the mode specified: df.explain(mode="formatted") + is_mode_case = mode is not None and isinstance(mode, basestring) + + if not is_no_argument and not (is_extended_case or is_mode_case): + argtypes = [ + str(type(arg)) for arg in [extended, mode] if arg is not None] + raise TypeError( + "extended (optional) and mode (optional) should be a bool and str; " + "however, got [%s]." % ", ".join(argtypes)) + + # Sets an explain mode depending on a given argument + if is_no_argument: + explainMode = "simple" + elif is_extended_case: + explainMode = "extended" if extended else "simple" + elif is_mode_case: + explainMode = mode + + print(self._jdf.toExplainString(explainMode)) @since(2.4) def exceptAll(self, other): 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 aa641f0a3fa4e..42338bd8accac 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql import java.io.{ByteArrayOutputStream, CharArrayWriter, DataOutputStream} +import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -521,20 +522,14 @@ class Dataset[T] private[sql]( def printSchema(level: Int): Unit = println(schema.treeString(level)) // scalastyle:on println - /** - * Prints the plans (logical and physical) with a format specified by a given explain mode. - * - * @group basic - * @since 3.0.0 - */ - def explain(mode: ExplainMode): Unit = { + private def toExplainString(mode: ExplainMode): String = { // Because temporary views are resolved during analysis when we create a Dataset, and // `ExplainCommand` analyzes input query plan and resolves temporary views again. Using // `ExplainCommand` here will probably output different query plans, compared to the results // of evaluation of the Dataset. So just output QueryExecution's query plans here. val qe = ExplainCommandUtil.explainedQueryExecution(sparkSession, logicalPlan, queryExecution) - val outputString = mode match { + mode match { case ExplainMode.Simple => qe.simpleString case ExplainMode.Extended => @@ -550,8 +545,29 @@ class Dataset[T] private[sql]( case ExplainMode.Formatted => qe.simpleString(formatted = true) } + } + + private[sql] def toExplainString(mode: String): String = { + mode.toLowerCase(Locale.ROOT) match { + case "simple" => toExplainString(ExplainMode.Simple) + case "extended" => toExplainString(ExplainMode.Extended) + case "codegen" => toExplainString(ExplainMode.Codegen) + case "cost" => toExplainString(ExplainMode.Cost) + case "formatted" => toExplainString(ExplainMode.Formatted) + case _ => throw new IllegalArgumentException(s"Unknown explain mode: $mode. Accepted " + + "explain modes are 'simple', 'extended', 'codegen', 'cost', 'formatted'.") + } + } + + /** + * Prints the plans (logical and physical) with a format specified by a given explain mode. + * + * @group basic + * @since 3.0.0 + */ + def explain(mode: ExplainMode): Unit = { // scalastyle:off println - println(outputString) + println(toExplainString(mode)) // scalastyle:on println } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index 4e5d0f58be444..ecb4aa273d8f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -309,6 +309,24 @@ class ExplainSuite extends QueryTest with SharedSparkSession { "(1) LocalTableScan [codegen id :" :: Nil: _*) } + + test("Dataset.toExplainString has mode as string") { + val df = spark.range(10).toDF + def assertExplainOutput(mode: ExplainMode): Unit = { + assert(df.toExplainString(mode.toString).replaceAll("#\\d+", "#x").trim === + getNormalizedExplain(df, mode).trim) + } + assertExplainOutput(ExplainMode.Simple) + assertExplainOutput(ExplainMode.Extended) + assertExplainOutput(ExplainMode.Codegen) + assertExplainOutput(ExplainMode.Cost) + assertExplainOutput(ExplainMode.Formatted) + + val errMsg = intercept[IllegalArgumentException] { + df.toExplainString("unknown") + }.getMessage + assert(errMsg.contains("Unknown explain mode: unknown")) + } } case class ExplainSingleData(id: Int) From ac9b1881a281d33730d2bfb82ab2fb4bc04cc0a0 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 13 Dec 2019 21:45:35 +0800 Subject: [PATCH 0871/1387] [SPARK-30248][SQL] Fix DROP TABLE behavior when session catalog name is provided in the identifier ### What changes were proposed in this pull request? If a table name is qualified with session catalog name `spark_catalog`, the `DROP TABLE` command fails. For example, the following ``` sql("CREATE TABLE tbl USING json AS SELECT 1 AS i") sql("DROP TABLE spark_catalog.tbl") ``` fails with: ``` org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database 'spark_catalog' not found; at org.apache.spark.sql.catalyst.catalog.ExternalCatalog.requireDbExists(ExternalCatalog.scala:42) at org.apache.spark.sql.catalyst.catalog.ExternalCatalog.requireDbExists$(ExternalCatalog.scala:40) at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.requireDbExists(InMemoryCatalog.scala:45) at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.tableExists(InMemoryCatalog.scala:336) ``` This PR correctly resolves `spark_catalog` as a catalog. ### Why are the changes needed? It's fixing a bug. ### Does this PR introduce any user-facing change? Yes, now, the `spark_catalog.tbl` in the above example is dropped as expected. ### How was this patch tested? Added a test. Closes #26878 from imback82/fix_drop_table. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 4 ++-- .../apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 8 ++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 6fa6e99d636ec..afe15efdbf7ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -293,8 +293,8 @@ class ResolveSessionCatalog( orCreate = c.orCreate) } - case d @ DropTableStatement(SessionCatalogAndTable(catalog, tbl), ifExists, purge) => - DropTableCommand(d.tableName.asTableIdentifier, ifExists, isView = false, purge = purge) + case DropTableStatement(SessionCatalogAndTable(catalog, tbl), ifExists, purge) => + DropTableCommand(tbl.asTableIdentifier, ifExists, isView = false, purge = purge) case DropViewStatement(SessionCatalogAndTable(catalog, viewName), ifExists) => DropTableCommand(viewName.asTableIdentifier, ifExists, isView = true, purge = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index d0c67ad63c37f..35822b99e3aba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -569,6 +569,14 @@ class DataSourceV2SQLSuite assert(catalog("testcat").asTableCatalog.tableExists(ident) === false) } + test("DropTable: table qualified with the session catalog name") { + val ident = Identifier.of(Array(), "tbl") + sql("CREATE TABLE tbl USING json AS SELECT 1 AS i") + assert(catalog("spark_catalog").asTableCatalog.tableExists(ident) === true) + sql("DROP TABLE spark_catalog.tbl") + assert(catalog("spark_catalog").asTableCatalog.tableExists(ident) === false) + } + test("DropTable: if exists") { intercept[NoSuchTableException] { sql(s"DROP TABLE testcat.db.notbl") From cc276f8a6e9d438d2b9b1c35c9cb7aca2e6178a1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 13 Dec 2019 08:25:51 -0800 Subject: [PATCH 0872/1387] [SPARK-30243][BUILD][K8S] Upgrade K8s client dependency to 4.6.4 ### What changes were proposed in this pull request? This PR aims to upgrade K8s client library from 4.6.1 to 4.6.4 for `3.0.0-preview2`. ### Why are the changes needed? This will bring the latest bug fixes. - https://github.com/fabric8io/kubernetes-client/releases/tag/v4.6.4 - https://github.com/fabric8io/kubernetes-client/releases/tag/v4.6.3 - https://github.com/fabric8io/kubernetes-client/releases/tag/v4.6.2 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with K8s integration test. Closes #26874 from dongjoon-hyun/SPARK-30243. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 10 +++++----- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 10 +++++----- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 10 +++++----- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- 5 files changed, 17 insertions(+), 17 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index bd6ef3512c9ce..00808893ccc7b 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -137,14 +137,14 @@ jsr305-3.0.0.jar jta-1.1.jar jul-to-slf4j-1.7.16.jar kryo-shaded-4.0.2.jar -kubernetes-client-4.6.1.jar -kubernetes-model-4.6.1.jar -kubernetes-model-common-4.6.1.jar +kubernetes-client-4.6.4.jar +kubernetes-model-4.6.4.jar +kubernetes-model-common-4.6.4.jar leveldbjni-all-1.8.jar libfb303-0.9.3.jar libthrift-0.12.0.jar log4j-1.2.17.jar -logging-interceptor-3.12.0.jar +logging-interceptor-3.12.6.jar lz4-java-1.7.0.jar machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar @@ -158,7 +158,7 @@ minlog-1.3.0.jar netty-all-4.1.42.Final.jar objenesis-2.5.1.jar okapi-shade-0.4.2.jar -okhttp-3.12.0.jar +okhttp-3.12.6.jar okio-1.15.0.jar opencsv-2.3.jar orc-core-1.5.8-nohive.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index d08acecb4e2b3..7d7d4ed447362 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -153,14 +153,14 @@ jsr305-3.0.0.jar jta-1.1.jar jul-to-slf4j-1.7.16.jar kryo-shaded-4.0.2.jar -kubernetes-client-4.6.1.jar -kubernetes-model-4.6.1.jar -kubernetes-model-common-4.6.1.jar +kubernetes-client-4.6.4.jar +kubernetes-model-4.6.4.jar +kubernetes-model-common-4.6.4.jar leveldbjni-all-1.8.jar libfb303-0.9.3.jar libthrift-0.12.0.jar log4j-1.2.17.jar -logging-interceptor-3.12.0.jar +logging-interceptor-3.12.6.jar lz4-java-1.7.0.jar machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar @@ -174,7 +174,7 @@ minlog-1.3.0.jar netty-all-4.1.42.Final.jar objenesis-2.5.1.jar okapi-shade-0.4.2.jar -okhttp-3.12.0.jar +okhttp-3.12.6.jar okio-1.15.0.jar opencsv-2.3.jar orc-core-1.5.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 8953d1e6cd5f0..997fa356decd8 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -165,14 +165,14 @@ kerby-pkix-1.0.1.jar kerby-util-1.0.1.jar kerby-xdr-1.0.1.jar kryo-shaded-4.0.2.jar -kubernetes-client-4.6.1.jar -kubernetes-model-4.6.1.jar -kubernetes-model-common-4.6.1.jar +kubernetes-client-4.6.4.jar +kubernetes-model-4.6.4.jar +kubernetes-model-common-4.6.4.jar leveldbjni-all-1.8.jar libfb303-0.9.3.jar libthrift-0.12.0.jar log4j-1.2.17.jar -logging-interceptor-3.12.0.jar +logging-interceptor-3.12.6.jar lz4-java-1.7.0.jar machinist_2.12-0.6.8.jar macro-compat_2.12-1.1.1.jar @@ -189,7 +189,7 @@ nimbus-jose-jwt-4.41.1.jar objenesis-2.5.1.jar okapi-shade-0.4.2.jar okhttp-2.7.5.jar -okhttp-3.12.0.jar +okhttp-3.12.6.jar okio-1.15.0.jar opencsv-2.3.jar orc-core-1.5.8.jar diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 487876b12ce91..8c46738b259bc 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -29,7 +29,7 @@ Spark Project Kubernetes kubernetes - 4.6.1 + 4.6.4 diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index c79caff164ce8..8e1043f77db6d 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -29,7 +29,7 @@ 1.3.0 1.4.0 - 4.6.1 + 4.6.4 3.2.2 1.0 kubernetes-integration-tests From 4da9780bc0a12672b45ffdcc28e594593bc68350 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 13 Dec 2019 11:23:55 -0800 Subject: [PATCH 0873/1387] Revert "[SPARK-30230][SQL] Like ESCAPE syntax can not use '_' and '%'" This reverts commit cada5beef72530fa699b5ec13d67261be37730e4. Closes #26883 from gengliangwang/revert. Authored-by: Gengliang Wang Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 4 ---- .../spark/sql/catalyst/parser/ExpressionParserSuite.scala | 4 ---- 2 files changed, 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 6816980947fe9..9f8c1d2c0ef95 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1393,10 +1393,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } str.charAt(0) }.getOrElse('\\') - if ('%' == escapeChar || '_' == escapeChar) { - throw new ParseException("Invalid escape string." + - "Escape string can not be '%', '_'.", ctx) - } invertIfNotDefined(Like(e, expression(ctx.pattern), escapeChar)) case SqlBaseParser.RLIKE => invertIfNotDefined(RLike(e, expression(ctx.pattern))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 9f11cecf26ae3..1bea1c254c0fc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -198,10 +198,6 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("a not like 'pattern%' escape '\"'", !('a.like("pattern%", '\"'))) intercept("a not like 'pattern%' escape '\"/'", message) intercept("a not like 'pattern%' escape ''", message) - - val message2 = "Escape string can not be '%', '_'." - intercept("a like 'pattern%' escape '_'", message2) - intercept("a like 'pattern%' escape '%'", message2) } test("like expressions with ESCAPED_STRING_LITERALS = true") { From e1ee3fb72f8d2486a6e874ceb0b7f9dbf59e5399 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Fri, 13 Dec 2019 11:31:31 -0800 Subject: [PATCH 0874/1387] [SPARK-30216][INFRA] Use python3 in Docker release image ### What changes were proposed in this pull request? - Reverts commit 1f94bf4 and d6be46e - Switches python to python3 in Docker release image. ### Why are the changes needed? `dev/make-distribution.sh` and `python/setup.py` are use python3. https://github.com/apache/spark/pull/26844/files#diff-ba2c046d92a1d2b5b417788bfb5cb5f8L236 https://github.com/apache/spark/pull/26330/files#diff-8cf6167d58ce775a08acafcfe6f40966 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test: ``` yumwangubuntu-3513086:~/spark$ dev/create-release/do-release-docker.sh -n -d /home/yumwang/spark-release Output directory already exists. Overwrite and continue? [y/n] y Branch [branch-2.4]: master Current branch version is 3.0.0-SNAPSHOT. Release [3.0.0]: 3.0.0-preview2 RC # [1]: This is a dry run. Please confirm the ref that will be built for testing. Ref [master]: ASF user [yumwang]: Full name [Yuming Wang]: GPG key [yumwangapache.org]: DBD447010C1B4F7DAD3F7DFD6E1B4122F6A3A338 ================ Release details: BRANCH: master VERSION: 3.0.0-preview2 TAG: v3.0.0-preview2-rc1 NEXT: 3.0.1-SNAPSHOT ASF USER: yumwang GPG KEY: DBD447010C1B4F7DAD3F7DFD6E1B4122F6A3A338 FULL NAME: Yuming Wang E-MAIL: yumwangapache.org ================ Is this info correct [y/n]? y GPG passphrase: ======================== = Building spark-rm image with tag latest... Command: docker build -t spark-rm:latest --build-arg UID=110302528 /home/yumwang/spark/dev/create-release/spark-rm Log file: docker-build.log Building v3.0.0-preview2-rc1; output will be at /home/yumwang/spark-release/output gpg: directory '/home/spark-rm/.gnupg' created gpg: keybox '/home/spark-rm/.gnupg/pubring.kbx' created gpg: /home/spark-rm/.gnupg/trustdb.gpg: trustdb created gpg: key 6E1B4122F6A3A338: public key "Yuming Wang " imported gpg: key 6E1B4122F6A3A338: secret key imported gpg: Total number processed: 1 gpg: imported: 1 gpg: secret keys read: 1 gpg: secret keys imported: 1 ======================== = Creating release tag v3.0.0-preview2-rc1... Command: /opt/spark-rm/release-tag.sh Log file: tag.log It may take some time for the tag to be synchronized to github. Press enter when you've verified that the new tag (v3.0.0-preview2-rc1) is available. ======================== = Building Spark... Command: /opt/spark-rm/release-build.sh package Log file: build.log ======================== = Building documentation... Command: /opt/spark-rm/release-build.sh docs Log file: docs.log ======================== = Publishing release Command: /opt/spark-rm/release-build.sh publish-release Log file: publish.log ``` Generated doc: ![image](https://user-images.githubusercontent.com/5399861/70693075-a7723100-1cf7-11ea-9f88-9356a02349a1.png) Closes #26848 from wangyum/SPARK-30216. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- dev/create-release/do-release-docker.sh | 3 -- dev/create-release/spark-rm/Dockerfile | 17 ++++++------ docs/building-spark.md | 2 +- python/docs/Makefile | 37 ++++--------------------- 4 files changed, 15 insertions(+), 44 deletions(-) diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index f643c060eb321..694a87bf78084 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -136,9 +136,6 @@ if [ -n "$JAVA" ]; then JAVA_VOL="--volume $JAVA:/opt/spark-java" fi -# SPARK-24530: Sphinx must work with python 3 to generate doc correctly. -echo "SPHINXPYTHON=/opt/p35/bin/python" >> $ENVFILE - echo "Building $RELEASE_TAG; output will be at $WORKDIR/output" docker run -ti \ --env-file "$ENVFILE" \ diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index cc7da152c7b2c..12195f5561076 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -33,7 +33,7 @@ ENV DEBCONF_NONINTERACTIVE_SEEN true # These arguments are just for reuse and not really meant to be customized. ARG APT_INSTALL="apt-get install --no-install-recommends -y" -ARG BASE_PIP_PKGS="setuptools wheel virtualenv" +ARG BASE_PIP_PKGS="setuptools wheel" ARG PIP_PKGS="pyopenssl pypandoc numpy sphinx" # Install extra needed repos and refresh. @@ -62,14 +62,13 @@ RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ curl -sL https://deb.nodesource.com/setup_11.x | bash && \ $APT_INSTALL nodejs && \ # Install needed python packages. Use pip for installing packages (for consistency). - $APT_INSTALL libpython2.7-dev libpython3-dev python-pip python3-pip && \ - pip install $BASE_PIP_PKGS && \ - pip install $PIP_PKGS && \ - cd && \ - virtualenv -p python3 /opt/p35 && \ - . /opt/p35/bin/activate && \ - pip install $BASE_PIP_PKGS && \ - pip install $PIP_PKGS && \ + $APT_INSTALL libpython3-dev python3-pip && \ + # Change default python version to python3. + update-alternatives --install /usr/bin/python python /usr/bin/python2.7 1 && \ + update-alternatives --install /usr/bin/python python /usr/bin/python3.6 2 && \ + update-alternatives --set python /usr/bin/python3.6 && \ + pip3 install $BASE_PIP_PKGS && \ + pip3 install $PIP_PKGS && \ # Install R packages and dependencies used when building. # R depends on pandoc*, libssl (which are installed above). $APT_INSTALL r-base r-base-dev && \ diff --git a/docs/building-spark.md b/docs/building-spark.md index 891d6225bbdd5..898f904461a6d 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -66,7 +66,7 @@ with Maven profile settings and so on like the direct Maven build. Example: ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes -This will build Spark distribution along with Python pip and R packages. (Note that build with Python pip package requires Python 3.6). For more information on usage, run `./dev/make-distribution.sh --help` +This will build Spark distribution along with Python pip and R packages. For more information on usage, run `./dev/make-distribution.sh --help` ## Specifying the Hadoop Version and Enabling YARN diff --git a/python/docs/Makefile b/python/docs/Makefile index 4767fd9f1c038..66d3fc425daa3 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -1,44 +1,19 @@ # Makefile for Sphinx documentation # -ifndef SPHINXBUILD -ifndef SPHINXPYTHON -SPHINXBUILD = sphinx-build -endif -endif - -ifdef SPHINXBUILD -# User-friendly check for sphinx-build. -ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) -$(error The '$(SPHINXBUILD)' command was not found. Make sure you have Sphinx installed, then set the SPHINXBUILD environment variable to point to the full path of the '$(SPHINXBUILD)' executable. Alternatively you can add the directory with the executable to your PATH. If you don't have Sphinx installed, grab it from http://sphinx-doc.org/) -endif -else -# Note that there is an issue with Python version and Sphinx in PySpark documentation generation. -# Please remove this check below when this issue is fixed. See SPARK-24530 for more details. -PYTHON_VERSION_CHECK = $(shell $(SPHINXPYTHON) -c 'import sys; print(sys.version_info < (3, 0, 0))') -ifeq ($(PYTHON_VERSION_CHECK), True) -$(error Note that Python 3 is required to generate PySpark documentation correctly for now. Current Python executable was less than Python 3. See SPARK-24530. To force Sphinx to use a specific Python executable, please set SPHINXPYTHON to point to the Python 3 executable.) -endif -# Check if Sphinx is installed. -ifeq ($(shell $(SPHINXPYTHON) -c 'import sphinx' >/dev/null 2>&1; echo $$?), 1) -$(error Python executable '$(SPHINXPYTHON)' did not have Sphinx installed. Make sure you have Sphinx installed, then set the SPHINXPYTHON environment variable to point to the Python executable having Sphinx installed. If you don't have Sphinx installed, grab it from http://sphinx-doc.org/) -endif -# Use 'SPHINXPYTHON -msphinx' instead of 'sphinx-build'. See https://github.com/sphinx-doc/sphinx/pull/3523 for more details. -SPHINXBUILD = $(SPHINXPYTHON) -msphinx -endif - # You can set these variables from the command line. SPHINXOPTS ?= +SPHINXBUILD ?= sphinx-build PAPER ?= BUILDDIR ?= _build -# You can set SPHINXBUILD to specify Sphinx build executable or SPHINXPYTHON to specify the Python executable used in Sphinx. -# They follow: -# 1. if SPHINXPYTHON is set, use Python. If SPHINXBUILD is set, use sphinx-build. -# 2. If both are set, SPHINXBUILD has a higher priority over SPHINXPYTHON -# 3. By default, SPHINXBUILD is used as 'sphinx-build'. export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.8.1-src.zip) +# User-friendly check for sphinx-build +ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) +$(error The '$(SPHINXBUILD)' command was not found. Make sure you have Sphinx installed, then set the SPHINXBUILD environment variable to point to the full path of the '$(SPHINXBUILD)' executable. Alternatively you can add the directory with the executable to your PATH. If you don't have Sphinx installed, grab it from http://sphinx-doc.org/) +endif + # Internal variables. PAPEROPT_a4 = -D latex_paper_size=a4 PAPEROPT_letter = -D latex_paper_size=letter From ec26dde36b17656a276ac537f44db270b3ad269a Mon Sep 17 00:00:00 2001 From: "sharan.gk" Date: Fri, 13 Dec 2019 11:35:00 -0800 Subject: [PATCH 0875/1387] [SPARK-29455][WEBUI] Improve tooltip information for Stages ### What changes were proposed in this pull request? Adding tooltip to Stages tab for better usability. ### Why are the changes needed? There are a few common points of confusion in the UI that could be clarified with tooltips. We should add tooltips to explain. ### Does this PR introduce any user-facing change? Yes ![image](https://user-images.githubusercontent.com/29914590/70693889-5a389400-1ce4-11ea-91bb-ee1e997a5c35.png) ### How was this patch tested? Manual Closes #26859 from sharangk/tooltip1. Authored-by: sharan.gk Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/ui/ToolTips.scala | 3 +++ core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala | 4 +++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 07403c11998cb..aefd001e573f9 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -99,4 +99,7 @@ private[spark] object ToolTips { dynamic allocation is enabled. The number of granted executors may exceed the limit ephemerally when executors are being killed. """ + + val DURATION = + "Elapsed time since the stage was submitted until execution completion of all its tasks." } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 09a215ba9f03d..ac431c90e14f7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -183,7 +183,9 @@ private[ui] class StagePagedTable( Seq(("Stage Id", null, true)) ++ {if (isFairScheduler) {Seq(("Pool Name", null, true))} else Seq.empty} ++ Seq( - ("Description", null, true), ("Submitted", null, true), ("Duration", null, true), + ("Description", null, true), + ("Submitted", null, true), + ("Duration", ToolTips.DURATION, true), ("Tasks: Succeeded/Total", null, false), ("Input", ToolTips.INPUT, true), ("Output", ToolTips.OUTPUT, true), From 61ebc8118665531b4c11a31f2b3d459dd201b097 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 13 Dec 2019 14:30:11 -0800 Subject: [PATCH 0876/1387] [SPARK-30167][REPL] Log4j configuration for REPL can't override the root logger properly MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? In the current implementation of `SparkShellLoggingFilter`, if the log level of the root logger and the log level of a message are different, whether a message should logged is decided based on log4j's configuration but whether the message should be output to the REPL's console is not cared. So, if the log level of the root logger is `DEBUG`, the log level of REPL's logger is `WARN` and the log level of a message is `INFO`, the message will output to the REPL's console even though `INFO < WARN`. https://github.com/apache/spark/pull/26798/files#diff-bfd5810d8aa78ad90150e806d830bb78L237 The ideal behavior should be like as follows and implemented them in this change. 1. If the log level of a message is greater than or equal to the log level of the root logger, the message should be logged but whether the message is output to the REPL's console should be decided based on whether the log level of the message is greater than or equal to the log level of the REPL's logger. 2. If a log level or custom appenders are explicitly defined for a category, whether a log message via the logger corresponding to the category is logged and output to the REPL's console should be decided baed on the log level of the category. We can confirm whether a log level or appenders are explicitly set to a logger for a category by `Logger#getLevel` and `Logger#getAllAppenders.hasMoreElements`. ### Why are the changes needed? This is a bug breaking a compatibility. #9816 enabled REPL's log4j configuration to override root logger but #23675 seems to have broken the feature. You can see one example when you modifies the default log4j configuration like as follows. ``` # Change the log level for rootCategory to DEBUG log4j.rootCategory=DEBUG, console ... # The log level for repl.Main remains WARN log4j.logger.org.apache.spark.repl.Main=WARN ``` If you launch REPL with the configuration, INFO level logs appear even though the log level for REPL is WARN. ``` ・・・ 19/12/08 23:31:38 INFO Utils: Successfully started service 'sparkDriver' on port 33083. 19/12/08 23:31:38 INFO SparkEnv: Registering MapOutputTracker 19/12/08 23:31:38 INFO SparkEnv: Registering BlockManagerMaster 19/12/08 23:31:38 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information 19/12/08 23:31:38 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up 19/12/08 23:31:38 INFO SparkEnv: Registering BlockManagerMasterHeartbeat ・・・ ``` Before #23675 was applied, those INFO level logs are not shown with the same log4j.properties. ### Does this PR introduce any user-facing change? Yes. The logging behavior for REPL is fixed. ### How was this patch tested? Manual test and newly added unit test. Closes #26798 from sarutak/fix-spark-shell-loglevel. Authored-by: Kousuke Saruta Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/internal/Logging.scala | 28 +++-- .../apache/spark/internal/LoggingSuite.scala | 12 +- .../org/apache/spark/repl/ReplSuite.scala | 114 +++++++++++++++++- 3 files changed, 133 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala index edfe9446094c8..2e4846bec2db4 100644 --- a/core/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -116,6 +116,11 @@ trait Logging { false } + // For testing + def initializeForcefully(isInterpreter: Boolean, silent: Boolean): Unit = { + initializeLogging(isInterpreter, silent) + } + private def initializeLogging(isInterpreter: Boolean, silent: Boolean): Unit = { // Don't use a logger in here, as this is itself occurring during initialization of a logger // If Log4j 1.2 is being used, but is not initialized, load a default properties file @@ -230,19 +235,18 @@ private class SparkShellLoggingFilter extends Filter { */ def decide(loggingEvent: LoggingEvent): Int = { if (Logging.sparkShellThresholdLevel == null) { - return Filter.NEUTRAL - } - val rootLevel = LogManager.getRootLogger().getLevel() - if (!loggingEvent.getLevel().eq(rootLevel)) { - return Filter.NEUTRAL - } - var logger = loggingEvent.getLogger() - while (logger.getParent() != null) { - if (logger.getLevel() != null) { - return Filter.NEUTRAL + Filter.NEUTRAL + } else if (loggingEvent.getLevel.isGreaterOrEqual(Logging.sparkShellThresholdLevel)) { + Filter.NEUTRAL + } else { + var logger = loggingEvent.getLogger() + while (logger.getParent() != null) { + if (logger.getLevel != null || logger.getAllAppenders.hasMoreElements) { + return Filter.NEUTRAL + } + logger = logger.getParent() } - logger = logger.getParent() + Filter.DENY } - return Filter.DENY } } diff --git a/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala b/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala index 250ac3dafcabc..6b7cc304a1baa 100644 --- a/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala @@ -33,18 +33,14 @@ class LoggingSuite extends SparkFunSuite { val originalThreshold = Logging.sparkShellThresholdLevel Logging.sparkShellThresholdLevel = Level.WARN try { - val logger = Logger.getLogger("a.b.c.D") - val logEvent = new LoggingEvent(logger.getName(), logger, Level.INFO, "Test", null) - assert(ssf.decide(logEvent) === Filter.DENY) - - // log level is less than threshold level but different from root level - val logEvent1 = new LoggingEvent(logger.getName(), logger, Level.DEBUG, "Test", null) - assert(ssf.decide(logEvent1) != Filter.DENY) + val logger1 = Logger.getLogger("a.b.c.D") + val logEvent1 = new LoggingEvent(logger1.getName(), logger1, Level.INFO, "Test", null) + assert(ssf.decide(logEvent1) == Filter.DENY) // custom log level configured val parentLogger = Logger.getLogger("a.b.c") parentLogger.setLevel(Level.INFO) - assert(ssf.decide(logEvent) != Filter.DENY) + assert(ssf.decide(logEvent1) != Filter.DENY) // log level is greater than or equal to threshold level val logger2 = Logger.getLogger("a.b.E") 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 ea8254e36dd9b..1e92b36c336d8 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -18,13 +18,15 @@ package org.apache.spark.repl import java.io._ +import java.nio.file.Files import scala.tools.nsc.interpreter.SimpleReader -import org.apache.log4j.{Level, LogManager} +import org.apache.log4j.{Level, LogManager, PropertyConfigurator} import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION @@ -297,4 +299,114 @@ class ReplSuite extends SparkFunSuite with BeforeAndAfterAll { assertContains("successful", output) } + test("SPARK-30167: Log4j configuration for REPL should override root logger properly") { + val testConfiguration = + """ + |# Set everything to be logged to the console + |log4j.rootCategory=INFO, console + |log4j.appender.console=org.apache.log4j.ConsoleAppender + |log4j.appender.console.target=System.err + |log4j.appender.console.layout=org.apache.log4j.PatternLayout + |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + | + |# Set the log level for this class to WARN same as the default setting. + |log4j.logger.org.apache.spark.repl.Main=ERROR + |""".stripMargin + + val log4jprops = Files.createTempFile("log4j.properties.d", "log4j.properties") + Files.write(log4jprops, testConfiguration.getBytes) + + val originalRootLogger = LogManager.getRootLogger + val originalRootAppender = originalRootLogger.getAppender("file") + val originalStderr = System.err + val originalReplThresholdLevel = Logging.sparkShellThresholdLevel + + val replLoggerLogMessage = "Log level for REPL: " + val warnLogMessage1 = "warnLogMessage1 should not be output" + val errorLogMessage1 = "errorLogMessage1 should be output" + val infoLogMessage1 = "infoLogMessage2 should be output" + val infoLogMessage2 = "infoLogMessage3 should be output" + + val out = try { + PropertyConfigurator.configure(log4jprops.toAbsolutePath.toString) + + // Re-initialization is needed to set SparkShellLoggingFilter to ConsoleAppender + Main.initializeForcefully(true, false) + runInterpreter("local", + s""" + |import java.io.{ByteArrayOutputStream, PrintStream} + | + |import org.apache.log4j.{ConsoleAppender, Level, LogManager} + | + |val replLogger = LogManager.getLogger("${Main.getClass.getName.stripSuffix("$")}") + | + |// Log level for REPL is expected to be ERROR + |"$replLoggerLogMessage" + replLogger.getLevel() + | + |val bout = new ByteArrayOutputStream() + | + |// Configure stderr to let log messages output to ByteArrayOutputStream. + |val defaultErrStream: PrintStream = System.err + |try { + | System.setErr(new PrintStream(bout)) + | + | // Reconfigure ConsoleAppender to reflect the stderr setting. + | val consoleAppender = + | LogManager.getRootLogger.getAllAppenders.nextElement.asInstanceOf[ConsoleAppender] + | consoleAppender.activateOptions() + | + | // customLogger1 is not explicitly configured neither its log level nor appender + | // so this inherits the settings of rootLogger + | // but ConsoleAppender can use a different log level. + | val customLogger1 = LogManager.getLogger("customLogger1") + | customLogger1.warn("$warnLogMessage1") + | customLogger1.error("$errorLogMessage1") + | + | // customLogger2 is explicitly configured its log level as INFO + | // so info level messages logged via customLogger2 should be output. + | val customLogger2 = LogManager.getLogger("customLogger2") + | customLogger2.setLevel(Level.INFO) + | customLogger2.info("$infoLogMessage1") + | + | // customLogger2 is explicitly configured its log level + | // so its child should inherit the settings. + | val customLogger3 = LogManager.getLogger("customLogger2.child") + | customLogger3.info("$infoLogMessage2") + | + | // echo log messages + | bout.toString + |} finally { + | System.setErr(defaultErrStream) + |} + |""".stripMargin) + } finally { + // Restore log4j settings for this suite + val log4jproperties = Thread.currentThread() + .getContextClassLoader.getResource("log4j.properties") + LogManager.resetConfiguration() + PropertyConfigurator.configure(log4jproperties) + Logging.sparkShellThresholdLevel = originalReplThresholdLevel + } + + // Ensure stderr configuration is successfully restored. + assert(originalStderr eq System.err) + + // Ensure log4j settings are successfully restored. + val restoredRootLogger = LogManager.getRootLogger + val restoredRootAppender = restoredRootLogger.getAppender("file") + assert(originalRootAppender.getClass == restoredRootAppender.getClass) + assert(originalRootLogger.getLevel == restoredRootLogger.getLevel) + + // Ensure loggers added in this test case are successfully removed. + assert(LogManager.getLogger("customLogger2").getLevel == null) + assert(LogManager.getLogger("customLogger2.child").getLevel == null) + + // Ensure log level threshold for REPL is ERROR. + assertContains(replLoggerLogMessage + "ERROR", out) + + assertDoesNotContain(warnLogMessage1, out) + assertContains(errorLogMessage1, out) + assertContains(infoLogMessage1, out) + assertContains(infoLogMessage2, out) + } } From 4c37a8a3f4a489b52f1919d2db84f6e32c6a05cd Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 13 Dec 2019 15:16:00 -0800 Subject: [PATCH 0877/1387] [SPARK-30143][SS] Add a timeout on stopping a streaming query ### What changes were proposed in this pull request? Add a timeout configuration for StreamingQuery.stop() ### Why are the changes needed? The stop() method on a Streaming Query awaits the termination of the stream execution thread. However, the stream execution thread may block forever depending on the streaming source implementation (like in Kafka, which runs UninterruptibleThreads). This causes control flow applications to hang indefinitely as well. We'd like to introduce a timeout to stop the execution thread, so that the control flow thread can decide to do an action if a timeout is hit. ### Does this PR introduce any user-facing change? By default, no. If the timeout configuration is set, then a TimeoutException will be thrown if a stream cannot be stopped within the given timeout. ### How was this patch tested? Unit tests Closes #26771 from brkyvz/stopTimeout. Lead-authored-by: Burak Yavuz Co-authored-by: Burak Yavuz Signed-off-by: Burak Yavuz --- .../apache/spark/sql/internal/SQLConf.scala | 7 + .../streaming/MicroBatchExecution.scala | 3 +- .../execution/streaming/StreamExecution.scala | 26 +++- .../continuous/ContinuousExecution.scala | 3 +- .../sql/streaming/DataStreamWriter.scala | 11 +- .../spark/sql/streaming/StreamingQuery.scala | 12 +- .../sql/streaming/StreamingQueryManager.scala | 3 +- .../JavaDataStreamReaderWriterSuite.java | 5 +- .../spark/sql/streaming/StreamSuite.scala | 35 ++++- .../streaming/util/BlockOnStopSource.scala | 132 ++++++++++++++++++ 10 files changed, 224 insertions(+), 13 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c54008cf91942..91347cfe80366 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1298,6 +1298,13 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(10L) + val STREAMING_STOP_TIMEOUT = + buildConf("spark.sql.streaming.stopTimeout") + .doc("How long to wait for the streaming execution thread to stop when calling the " + + "streaming query's stop() method in milliseconds. 0 or negative values wait indefinitely.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(0L) + val STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL = buildConf("spark.sql.streaming.noDataProgressEventInterval") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 5fe1f92e396c9..872c36764104f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -150,8 +150,7 @@ class MicroBatchExecution( state.set(TERMINATED) if (queryExecutionThread.isAlive) { sparkSession.sparkContext.cancelJobGroup(runId.toString) - queryExecutionThread.interrupt() - queryExecutionThread.join() + interruptAndAwaitExecutionThreadTermination() // microBatchThread may spawn new jobs, so we need to cancel again to prevent a leak sparkSession.sparkContext.cancelJobGroup(runId.toString) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index f470ad3f9c690..1cb395517e128 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming import java.io.{InterruptedIOException, IOException, UncheckedIOException} import java.nio.channels.ClosedByInterruptException import java.util.UUID -import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} +import java.util.concurrent.{CountDownLatch, ExecutionException, TimeoutException, TimeUnit} import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.locks.ReentrantLock @@ -434,6 +434,30 @@ abstract class StreamExecution( } } + /** + * Interrupts the query execution thread and awaits its termination until until it exceeds the + * timeout. The timeout can be set on "spark.sql.streaming.stopTimeout". + * + * @throws TimeoutException If the thread cannot be stopped within the timeout + */ + @throws[TimeoutException] + protected def interruptAndAwaitExecutionThreadTermination(): Unit = { + val timeout = math.max( + sparkSession.sessionState.conf.getConf(SQLConf.STREAMING_STOP_TIMEOUT), 0) + queryExecutionThread.interrupt() + queryExecutionThread.join(timeout) + if (queryExecutionThread.isAlive) { + val stackTraceException = new SparkException("The stream thread was last executing:") + stackTraceException.setStackTrace(queryExecutionThread.getStackTrace) + val timeoutException = new TimeoutException( + s"Stream Execution thread failed to stop within $timeout milliseconds (specified by " + + s"${SQLConf.STREAMING_STOP_TIMEOUT.key}). See the cause on what was " + + "being executed in the streaming query thread.") + timeoutException.initCause(stackTraceException) + throw timeoutException + } + } + /** * Blocks the current thread until processing for data from the given `source` has reached at * least the given `Offset`. This method is intended for use primarily when writing tests. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 8c7371e75b539..481552a2e4a0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -427,8 +427,7 @@ class ContinuousExecution( if (queryExecutionThread.isAlive) { // The query execution thread will clean itself up in the finally clause of runContinuous. // We just need to interrupt the long running job. - queryExecutionThread.interrupt() - queryExecutionThread.join() + interruptAndAwaitExecutionThreadTermination() } logInfo(s"Query $prettyIdString was stopped") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 74170b1b5d77e..62a1add8b6d94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming import java.util.Locale +import java.util.concurrent.TimeoutException import scala.collection.JavaConverters._ @@ -238,10 +239,18 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** * Starts the execution of the streaming query, which will continually output results to the given * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with - * the stream. + * the stream. Throws a `TimeoutException` if the following conditions are met: + * - Another run of the same streaming query, that is a streaming query + * sharing the same checkpoint location, is already active on the same + * Spark Driver + * - The SQL configuration `spark.sql.streaming.stopActiveRunOnRestart` + * is enabled + * - The active run cannot be stopped within the timeout controlled by + * the SQL configuration `spark.sql.streaming.stopTimeout` * * @since 2.0.0 */ + @throws[TimeoutException] def start(): StreamingQuery = { if (source.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { throw new AnalysisException("Hive data source can only be used with tables, you can not " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index 47ddc88e964e8..85d980e5d6733 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming import java.util.UUID +import java.util.concurrent.TimeoutException import org.apache.spark.annotation.Evolving import org.apache.spark.sql.SparkSession @@ -142,10 +143,17 @@ trait StreamingQuery { def processAllAvailable(): Unit /** - * Stops the execution of this query if it is running. This method blocks until the threads - * performing execution has stopped. + * Stops the execution of this query if it is running. This waits until the termination of the + * query execution threads or until a timeout is hit. + * + * By default stop will block indefinitely. You can configure a timeout by the configuration + * `spark.sql.streaming.stopTimeout`. A timeout of 0 (or negative) milliseconds will block + * indefinitely. If a `TimeoutException` is thrown, users can retry stopping the stream. If the + * issue persists, it is advisable to kill the Spark application. + * * @since 2.0.0 */ + @throws[TimeoutException] def stop(): Unit /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index e64f67cc755fa..810f4a182fca6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.streaming import java.util.{ConcurrentModificationException, UUID} -import java.util.concurrent.TimeUnit +import java.util.concurrent.{TimeoutException, TimeUnit} import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ @@ -321,6 +321,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo * @param trigger [[Trigger]] for the query. * @param triggerClock [[Clock]] to use for the triggering. */ + @throws[TimeoutException] private[sql] def startQuery( userSpecifiedName: Option[String], userSpecifiedCheckpointLocation: Option[String], diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java index 48cdb2642d830..5903623847f52 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java @@ -18,6 +18,7 @@ package test.org.apache.spark.sql.streaming; import java.io.File; +import java.util.concurrent.TimeoutException; import org.junit.After; import org.junit.Before; @@ -52,7 +53,7 @@ public void tearDown() { } @Test - public void testForeachBatchAPI() { + public void testForeachBatchAPI() throws TimeoutException { StreamingQuery query = spark .readStream() .textFile(input) @@ -66,7 +67,7 @@ public void call(Dataset v1, Long v2) throws Exception {} } @Test - public void testForeachAPI() { + public void testForeachAPI() throws TimeoutException { StreamingQuery query = spark .readStream() .textFile(input) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index a637b42c6b036..297d6c7ca2171 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.streaming import java.io.{File, InterruptedIOException, IOException, UncheckedIOException} import java.nio.channels.ClosedByInterruptException -import java.util.concurrent.{CountDownLatch, ExecutionException, TimeoutException, TimeUnit} +import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} +import scala.concurrent.TimeoutException import scala.reflect.ClassTag import scala.util.control.ControlThrowable @@ -42,7 +43,7 @@ import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreCon import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.sql.streaming.util.{BlockOnStopSourceProvider, StreamManualClock} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils @@ -1125,6 +1126,36 @@ class StreamSuite extends StreamTest { } ) } + + // ProcessingTime trigger generates MicroBatchExecution, and ContinuousTrigger starts a + // ContinuousExecution + Seq(Trigger.ProcessingTime("1 second"), Trigger.Continuous("1 second")).foreach { trigger => + test(s"SPARK-30143: stop waits until timeout if blocked - trigger: $trigger") { + BlockOnStopSourceProvider.enableBlocking() + val sq = spark.readStream.format(classOf[BlockOnStopSourceProvider].getName) + .load() + .writeStream + .format("console") + .trigger(trigger) + .start() + failAfter(60.seconds) { + val startTime = System.nanoTime() + withSQLConf(SQLConf.STREAMING_STOP_TIMEOUT.key -> "2000") { + intercept[TimeoutException] { + sq.stop() + } + } + val duration = (System.nanoTime() - startTime) / 1e6 + assert(duration >= 2000, + s"Should have waited more than 2000 millis, but waited $duration millis") + + BlockOnStopSourceProvider.disableBlocking() + withSQLConf(SQLConf.STREAMING_STOP_TIMEOUT.key -> "0") { + sq.stop() + } + } + } + } } abstract class FakeSource extends StreamSourceProvider { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala new file mode 100644 index 0000000000000..f25758c520691 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala @@ -0,0 +1,132 @@ +/* + * 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.streaming.util + +import java.util +import java.util.concurrent.CountDownLatch + +import scala.collection.JavaConverters._ + +import org.apache.zookeeper.KeeperException.UnimplementedException + +import org.apache.spark.sql.{DataFrame, Row, SparkSession, SQLContext} +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.TableCapability.CONTINUOUS_READ +import org.apache.spark.sql.connector.read.{streaming, InputPartition, Scan, ScanBuilder} +import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, PartitionOffset} +import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, Source} +import org.apache.spark.sql.sources.StreamSourceProvider +import org.apache.spark.sql.types.{LongType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +/** The V1 and V2 provider of a streaming source, which blocks indefinitely on the call of stop() */ +object BlockOnStopSourceProvider { + private var _latch: CountDownLatch = _ + val schema: StructType = new StructType().add("id", LongType) + + /** Set the latch that we will use to block the streaming query thread. */ + def enableBlocking(): Unit = { + if (_latch == null || _latch.getCount == 0) { + _latch = new CountDownLatch(1) + } + } + + def disableBlocking(): Unit = { + if (_latch != null) { + _latch.countDown() + _latch = null + } + } +} + +class BlockOnStopSourceProvider extends StreamSourceProvider with TableProvider { + override def getTable(options: CaseInsensitiveStringMap): Table = { + new BlockOnStopSourceTable(BlockOnStopSourceProvider._latch) + } + + override def sourceSchema( + sqlContext: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + "blockingSource" -> BlockOnStopSourceProvider.schema + } + + override def createSource( + sqlContext: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + new BlockOnStopSource(sqlContext.sparkSession, BlockOnStopSourceProvider._latch) + } +} + +/** A V1 Streaming Source which blocks on stop(). It does not produce any data. */ +class BlockOnStopSource(spark: SparkSession, latch: CountDownLatch) extends Source { + // Blocks until latch countdowns + override def stop(): Unit = latch.await() + + // Boiler-plate + override val schema: StructType = BlockOnStopSourceProvider.schema + override def getOffset: Option[Offset] = Some(LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + spark.createDataFrame(spark.sparkContext.emptyRDD[Row], schema) + } +} + +/** A V2 Table, which can create a blocking streaming source for ContinuousExecution. */ +class BlockOnStopSourceTable(latch: CountDownLatch) extends Table with SupportsRead { + override def schema(): StructType = BlockOnStopSourceProvider.schema + + override def name(): String = "blockingSource" + + override def capabilities(): util.Set[TableCapability] = Set(CONTINUOUS_READ).asJava + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new ScanBuilder { + override def build(): Scan = new Scan { + override def readSchema(): StructType = schema() + + override def toContinuousStream(checkpointLocation: String): ContinuousStream = { + new BlockOnStopContinuousStream(latch) + } + } + } + } +} + +/** + * A V2 Streaming Source which blocks on stop(). It does not produce any data. We use this for + * testing stopping in ContinuousExecution. + */ +class BlockOnStopContinuousStream(latch: CountDownLatch) extends ContinuousStream { + + // Blocks until latch countdowns + override def stop(): Unit = latch.await() + + // Boiler-plate + override def planInputPartitions(start: streaming.Offset): Array[InputPartition] = Array.empty + override def mergeOffsets(offsets: Array[PartitionOffset]): streaming.Offset = LongOffset(0L) + override def deserializeOffset(json: String): streaming.Offset = LongOffset(0L) + override def initialOffset(): Offset = LongOffset(0) + override def commit(end: streaming.Offset): Unit = {} + override def createContinuousReaderFactory(): ContinuousPartitionReaderFactory = { + throw new UnimplementedException + } +} From f197204f0376c5bca5a4ebb7111b1282cd3f8199 Mon Sep 17 00:00:00 2001 From: John Ayad Date: Sat, 14 Dec 2019 13:08:15 -0800 Subject: [PATCH 0878/1387] [SPARK-30236][SQL][DOCS] Clarify date and time patterns supported in docs ### What changes were proposed in this pull request? Link to appropriate Java Class with list of date/time patterns supported ### Why are the changes needed? Avoid confusion on the end-user's side of things, as seen in questions like [this](https://stackoverflow.com/questions/54496878/date-format-conversion-is-adding-1-year-to-the-border-dates) on StackOverflow ### Does this PR introduce any user-facing change? Yes, Docs are updated. ### How was this patch tested? `date_format`: ![image](https://user-images.githubusercontent.com/2394761/70796647-b5c55900-1d9a-11ea-89f9-7a8661641c09.png) `to_unix_timestamp`: ![image](https://user-images.githubusercontent.com/2394761/70796664-c07fee00-1d9a-11ea-9029-e82d899e3f59.png) `unix_timestamp`: ![image](https://user-images.githubusercontent.com/2394761/70796688-caa1ec80-1d9a-11ea-8868-a18c437a5d49.png) `from_unixtime`: ![image](https://user-images.githubusercontent.com/2394761/70796703-d4c3eb00-1d9a-11ea-85fe-3c672e0cda28.png) `to_date`: ![image](https://user-images.githubusercontent.com/2394761/70796718-dd1c2600-1d9a-11ea-81f4-a0966eeb0f1d.png) `to_timestamp`: ![image](https://user-images.githubusercontent.com/2394761/70796735-e6a58e00-1d9a-11ea-8ef7-d3e1d9b5370f.png) Closes #26864 from johnhany97/SPARK-30236. Authored-by: John Ayad Signed-off-by: Dongjoon Hyun --- .../expressions/datetimeExpressions.scala | 46 +++++++++++++++++-- 1 file changed, 42 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index b3aeda852a4dd..a9186ba514c42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -596,6 +596,12 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa // scalastyle:off line.size.limit @ExpressionDescription( usage = "_FUNC_(timestamp, fmt) - Converts `timestamp` to a value of string in the format specified by the date format `fmt`.", + arguments = """ + Arguments: + * timestamp - A date/timestamp or string to be converted to the given format. + * fmt - Date/time format pattern to follow. See `java.time.format.DateTimeFormatter` for valid date + and time format patterns. + """, examples = """ Examples: > SELECT _FUNC_('2016-04-08', 'y'); @@ -654,7 +660,14 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti * Deterministic version of [[UnixTimestamp]], must have at least one parameter. */ @ExpressionDescription( - usage = "_FUNC_(expr[, pattern]) - Returns the UNIX timestamp of the given time.", + usage = "_FUNC_(timeExp[, format]) - Returns the UNIX timestamp of the given time.", + arguments = """ + Arguments: + * timeExp - A date/timestamp or string which is returned as a UNIX timestamp. + * format - Date/time format pattern to follow. Ignored if `timeExp` is not a string. + Default value is "uuuu-MM-dd HH:mm:ss". See `java.time.format.DateTimeFormatter` + for valid date and time format patterns. + """, examples = """ Examples: > SELECT _FUNC_('2016-04-08', 'yyyy-MM-dd'); @@ -692,7 +705,14 @@ case class ToUnixTimestamp( * second parameter. */ @ExpressionDescription( - usage = "_FUNC_([expr[, pattern]]) - Returns the UNIX timestamp of current or specified time.", + usage = "_FUNC_([timeExp[, format]]) - Returns the UNIX timestamp of current or specified time.", + arguments = """ + Arguments: + * timeExp - A date/timestamp or string. If not provided, this defaults to current time. + * format - Date/time format pattern to follow. Ignored if `timeExp` is not a string. + Default value is "uuuu-MM-dd HH:mm:ss". See `java.time.format.DateTimeFormatter` + for valid date and time format patterns. + """, examples = """ Examples: > SELECT _FUNC_(); @@ -866,6 +886,12 @@ abstract class UnixTime extends ToTimestamp { */ @ExpressionDescription( usage = "_FUNC_(unix_time, format) - Returns `unix_time` in the specified `format`.", + arguments = """ + Arguments: + * unix_time - UNIX Timestamp to be converted to the provided format. + * format - Date/time format pattern to follow. See `java.time.format.DateTimeFormatter` + for valid date and time format patterns. + """, examples = """ Examples: > SELECT _FUNC_(0, 'yyyy-MM-dd HH:mm:ss'); @@ -1408,6 +1434,12 @@ case class ToUTCTimestamp(left: Expression, right: Expression) a date. Returns null with invalid input. By default, it follows casting rules to a date if the `fmt` is omitted. """, + arguments = """ + Arguments: + * date_str - A string to be parsed to date. + * fmt - Date format pattern to follow. See `java.time.format.DateTimeFormatter` for valid + date and time format patterns. + """, examples = """ Examples: > SELECT _FUNC_('2009-07-30 04:17:52'); @@ -1446,10 +1478,16 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr */ @ExpressionDescription( usage = """ - _FUNC_(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to - a timestamp. Returns null with invalid input. By default, it follows casting rules to + _FUNC_(timestamp_str[, fmt]) - Parses the `timestamp_str` expression with the `fmt` expression + to a timestamp. Returns null with invalid input. By default, it follows casting rules to a timestamp if the `fmt` is omitted. """, + arguments = """ + Arguments: + * timestamp_str - A string to be parsed to timestamp. + * fmt - Timestamp format pattern to follow. See `java.time.format.DateTimeFormatter` for valid + date and time format patterns. + """, examples = """ Examples: > SELECT _FUNC_('2016-12-31 00:12:00'); From d3ec8b173558075535133869c5ae3b19b403265c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 14 Dec 2019 13:10:46 -0800 Subject: [PATCH 0879/1387] [SPARK-30066][SQL] Support columnar execution on interval types ### What changes were proposed in this pull request? Columnar execution support for interval types ### Why are the changes needed? support cache tables with interval columns improve performance too ### Does this PR introduce any user-facing change? Yes cache table with accept interval columns ### How was this patch tested? add ut Closes #26699 from yaooqinn/SPARK-30066. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../spark/unsafe/types/CalendarInterval.java | 12 ++++++ .../spark/sql/catalyst/InternalRow.scala | 2 + .../execution/columnar/ColumnAccessor.scala | 5 +++ .../execution/columnar/ColumnBuilder.scala | 4 ++ .../sql/execution/columnar/ColumnStats.scala | 22 ++++++++++- .../sql/execution/columnar/ColumnType.scala | 34 ++++++++++++++++- .../columnar/GenerateColumnAccessor.scala | 6 +++ .../apache/spark/sql/CachedTableSuite.scala | 15 ++++++++ .../execution/columnar/ColumnStatsSuite.scala | 38 +++++++++++++++++++ .../execution/columnar/ColumnTypeSuite.scala | 6 ++- .../columnar/ColumnarTestUtils.scala | 4 +- .../NullableColumnAccessorSuite.scala | 3 +- .../columnar/NullableColumnBuilderSuite.scala | 3 +- 13 files changed, 148 insertions(+), 6 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 9f0bcd6ad82fe..f9f78d575e834 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -133,4 +133,16 @@ private void appendUnit(StringBuilder sb, long value, String unit) { * @throws ArithmeticException if a numeric overflow occurs */ public Duration extractAsDuration() { return Duration.of(microseconds, ChronoUnit.MICROS); } + + /** + * A constant holding the minimum value an {@code CalendarInterval} can have. + */ + public static CalendarInterval MIN_VALUE = + new CalendarInterval(Integer.MIN_VALUE, Integer.MIN_VALUE, Long.MIN_VALUE); + + /** + * A constant holding the maximum value an {@code CalendarInterval} can have. + */ + public static CalendarInterval MAX_VALUE = + new CalendarInterval(Integer.MAX_VALUE, Integer.MAX_VALUE, Long.MAX_VALUE); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala index 57cdd75887153..f98b59edd4226 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala @@ -171,6 +171,8 @@ object InternalRow { case LongType | TimestampType => (input, v) => input.setLong(ordinal, v.asInstanceOf[Long]) case FloatType => (input, v) => input.setFloat(ordinal, v.asInstanceOf[Float]) case DoubleType => (input, v) => input.setDouble(ordinal, v.asInstanceOf[Double]) + case CalendarIntervalType => + (input, v) => input.setInterval(ordinal, v.asInstanceOf[CalendarInterval]) case DecimalType.Fixed(precision, _) => (input, v) => input.setDecimal(ordinal, v.asInstanceOf[Decimal], precision) case udt: UserDefinedType[_] => getWriter(ordinal, udt.sqlType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala index a43dc0b41ecca..06f411dec158d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeMapData import org.apache.spark.sql.execution.columnar.compression.CompressibleColumnAccessor import org.apache.spark.sql.execution.vectorized.WritableColumnVector import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is @@ -104,6 +105,10 @@ private[columnar] class BinaryColumnAccessor(buffer: ByteBuffer) extends BasicColumnAccessor[Array[Byte]](buffer, BINARY) with NullableColumnAccessor +private[columnar] class IntervalColumnAccessor(buffer: ByteBuffer, dataType: CalendarIntervalType) + extends BasicColumnAccessor[CalendarInterval](buffer, CALENDAR_INTERVAL) + with NullableColumnAccessor + private[columnar] class CompactDecimalColumnAccessor(buffer: ByteBuffer, dataType: DecimalType) extends NativeColumnAccessor(buffer, COMPACT_DECIMAL(dataType)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala index d30655e0c4a20..3d94681a2fb31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala @@ -125,6 +125,9 @@ class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STR private[columnar] class BinaryColumnBuilder extends ComplexColumnBuilder(new BinaryColumnStats, BINARY) +private[columnar] +class IntervalColumnBuilder extends ComplexColumnBuilder(new IntervalColumnStats, CALENDAR_INTERVAL) + private[columnar] class CompactDecimalColumnBuilder(dataType: DecimalType) extends NativeColumnBuilder(new DecimalColumnStats(dataType), COMPACT_DECIMAL(dataType)) @@ -176,6 +179,7 @@ private[columnar] object ColumnBuilder { case DoubleType => new DoubleColumnBuilder case StringType => new StringColumnBuilder case BinaryType => new BinaryColumnBuilder + case CalendarIntervalType => new IntervalColumnBuilder case dt: DecimalType if dt.precision <= Decimal.MAX_LONG_DIGITS => new CompactDecimalColumnBuilder(dt) case dt: DecimalType => new DecimalColumnBuilder(dt) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala index bc7e73ae1ba87..c0e61b381f49b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializable { val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() @@ -295,6 +295,26 @@ private[columnar] final class BinaryColumnStats extends ColumnStats { Array[Any](null, null, nullCount, count, sizeInBytes) } +private[columnar] final class IntervalColumnStats extends ColumnStats { + protected var upper: CalendarInterval = CalendarInterval.MIN_VALUE + protected var lower: CalendarInterval = CalendarInterval.MAX_VALUE + + override def gatherStats(row: InternalRow, ordinal: Int): Unit = { + if (!row.isNullAt(ordinal)) { + val value = row.getInterval(ordinal) + if (value.compareTo(upper) > 0) upper = value + if (value.compareTo(lower) < 0) lower = value + sizeInBytes += CALENDAR_INTERVAL.actualSize(row, ordinal) + count += 1 + } else { + gatherNullStats + } + } + + override def collectedStatistics: Array[Any] = + Array[Any](lower, upper, nullCount, count, sizeInBytes) +} + private[columnar] final class DecimalColumnStats(precision: Int, scale: Int) extends ColumnStats { def this(dt: DecimalType) = this(dt.precision, dt.scale) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index 059e1447e31d9..5e84e16646338 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** @@ -705,6 +705,37 @@ private[columnar] case class MAP(dataType: MapType) override def clone(v: UnsafeMapData): UnsafeMapData = v.copy() } +private[columnar] object CALENDAR_INTERVAL extends ColumnType[CalendarInterval] + with DirectCopyColumnType[CalendarInterval] { + + override def dataType: DataType = CalendarIntervalType + + override def defaultSize: Int = 16 + + override def actualSize(row: InternalRow, ordinal: Int): Int = 20 + + override def getField(row: InternalRow, ordinal: Int): CalendarInterval = row.getInterval(ordinal) + + override def setField(row: InternalRow, ordinal: Int, value: CalendarInterval): Unit = { + row.setInterval(ordinal, value) + } + + override def extract(buffer: ByteBuffer): CalendarInterval = { + ByteBufferHelper.getInt(buffer) + val months = ByteBufferHelper.getInt(buffer) + val days = ByteBufferHelper.getInt(buffer) + val microseconds = ByteBufferHelper.getLong(buffer) + new CalendarInterval(months, days, microseconds) + } + + override def append(v: CalendarInterval, buffer: ByteBuffer): Unit = { + ByteBufferHelper.putInt(buffer, 16) + ByteBufferHelper.putInt(buffer, v.months) + ByteBufferHelper.putInt(buffer, v.days) + ByteBufferHelper.putLong(buffer, v.microseconds) + } +} + private[columnar] object ColumnType { @tailrec def apply(dataType: DataType): ColumnType[_] = { @@ -719,6 +750,7 @@ private[columnar] object ColumnType { case DoubleType => DOUBLE case StringType => STRING case BinaryType => BINARY + case i: CalendarIntervalType => CALENDAR_INTERVAL case dt: DecimalType if dt.precision <= Decimal.MAX_LONG_DIGITS => COMPACT_DECIMAL(dt) case dt: DecimalType => LARGE_DECIMAL(dt) case arr: ArrayType => ARRAY(arr) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala index 2d699e8a9d088..bd2d06665a910 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeFormatter, CodeGenerator, UnsafeRowWriter} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval /** * An Iterator to walk through the InternalRows from a CachedBatch @@ -51,6 +52,10 @@ class MutableUnsafeRow(val writer: UnsafeRowWriter) extends BaseGenericInternalR // the writer will be used directly to avoid creating wrapper objects override def setDecimal(i: Int, v: Decimal, precision: Int): Unit = throw new UnsupportedOperationException + + override def setInterval(i: Int, value: CalendarInterval): Unit = + throw new UnsupportedOperationException + override def update(i: Int, v: Any): Unit = throw new UnsupportedOperationException // all other methods inherited from GenericMutableRow are not need @@ -81,6 +86,7 @@ object GenerateColumnAccessor extends CodeGenerator[Seq[DataType], ColumnarItera case DoubleType => classOf[DoubleColumnAccessor].getName case StringType => classOf[StringColumnAccessor].getName case BinaryType => classOf[BinaryColumnAccessor].getName + case CalendarIntervalType => classOf[IntervalColumnAccessor].getName case dt: DecimalType if dt.precision <= Decimal.MAX_LONG_DIGITS => classOf[CompactDecimalColumnAccessor].getName case dt: DecimalType => classOf[DecimalColumnAccessor].getName diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index d3f4159d6471f..2c97ec07bc577 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Join, JoinStrategyHint, SHUFFLE_HASH} +import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.execution.{RDDScanExec, SparkPlan} import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec @@ -36,6 +37,7 @@ import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.storage.StorageLevel.{MEMORY_AND_DISK_2, MEMORY_ONLY} +import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.{AccumulatorContext, Utils} private case class BigData(s: String) @@ -1094,4 +1096,17 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi } } } + + test("cache supports for intervals") { + withTable("interval_cache") { + Seq((1, "1 second"), (2, "2 seconds"), (2, null)) + .toDF("k", "v").write.saveAsTable("interval_cache") + sql("CACHE TABLE t1 AS SELECT k, cast(v as interval) FROM interval_cache") + assert(spark.catalog.isCached("t1")) + checkAnswer(sql("SELECT * FROM t1 WHERE k = 1"), + Row(1, new CalendarInterval(0, 0, DateTimeConstants.MICROS_PER_SECOND))) + sql("UNCACHE TABLE t1") + assert(!spark.catalog.isCached("t1")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala index 3121b7e99c99d..eb76316f487ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark.SparkFunSuite import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval class ColumnStatsSuite extends SparkFunSuite { testColumnStats(classOf[BooleanColumnStats], BOOLEAN, Array(true, false, 0)) @@ -30,6 +31,7 @@ class ColumnStatsSuite extends SparkFunSuite { testColumnStats(classOf[DoubleColumnStats], DOUBLE, Array(Double.MaxValue, Double.MinValue, 0)) testColumnStats(classOf[StringColumnStats], STRING, Array(null, null, 0)) testDecimalColumnStats(Array(null, null, 0)) + testIntervalColumnStats(Array(CalendarInterval.MAX_VALUE, CalendarInterval.MIN_VALUE, 0)) def testColumnStats[T <: AtomicType, U <: ColumnStats]( columnStatsClass: Class[U], @@ -103,4 +105,40 @@ class ColumnStatsSuite extends SparkFunSuite { } } } + + def testIntervalColumnStats[T <: AtomicType, U <: ColumnStats]( + initialStatistics: Array[Any]): Unit = { + + val columnStatsName = classOf[IntervalColumnStats].getSimpleName + val columnType = CALENDAR_INTERVAL + + test(s"$columnStatsName: empty") { + val columnStats = new IntervalColumnStats + columnStats.collectedStatistics.zip(initialStatistics).foreach { + case (actual, expected) => assert(actual === expected) + } + } + + test(s"$columnStatsName: non-empty") { + import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ + + val columnStats = new IntervalColumnStats + val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) + rows.foreach(columnStats.gatherStats(_, 0)) + + val values = rows.take(10).map(_.get(0, columnType.dataType)) + val ordering = CalendarIntervalType.ordering.asInstanceOf[Ordering[Any]] + val stats = columnStats.collectedStatistics + + assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) + assertResult(values.max(ordering), "Wrong upper bound")(stats(1)) + assertResult(10, "Wrong null count")(stats(2)) + assertResult(20, "Wrong row count")(stats(3)) + assertResult(stats(4), "Wrong size in bytes") { + rows.map { row => + if (row.isNullAt(0)) 4 else columnType.actualSize(row, 0) + }.sum + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index ff05049551dc8..731db232ed61c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval class ColumnTypeSuite extends SparkFunSuite with Logging { private val DEFAULT_BUFFER_SIZE = 512 @@ -38,7 +39,8 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { val checks = Map( NULL -> 0, BOOLEAN -> 1, BYTE -> 1, SHORT -> 2, INT -> 4, LONG -> 8, FLOAT -> 4, DOUBLE -> 8, COMPACT_DECIMAL(15, 10) -> 8, LARGE_DECIMAL(20, 10) -> 12, - STRING -> 8, BINARY -> 16, STRUCT_TYPE -> 20, ARRAY_TYPE -> 28, MAP_TYPE -> 68) + STRING -> 8, BINARY -> 16, STRUCT_TYPE -> 20, ARRAY_TYPE -> 28, MAP_TYPE -> 68, + CALENDAR_INTERVAL -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -76,6 +78,7 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { checkActualSize(ARRAY_TYPE, Array[Any](1), 4 + 8 + 8 + 8) checkActualSize(MAP_TYPE, Map(1 -> "a"), 4 + (8 + 8 + 8 + 8) + (8 + 8 + 8 + 8)) checkActualSize(STRUCT_TYPE, Row("hello"), 28) + checkActualSize(CALENDAR_INTERVAL, CalendarInterval.MAX_VALUE, 4 + 4 + 4 + 8) } testNativeColumnType(BOOLEAN) @@ -94,6 +97,7 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { testColumnType(STRUCT_TYPE) testColumnType(ARRAY_TYPE) testColumnType(MAP_TYPE) + testColumnType(CALENDAR_INTERVAL) def testNativeColumnType[T <: AtomicType](columnType: NativeColumnType[T]): Unit = { testColumnType[T#InternalType](columnType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala index 686c8fa6f5fa9..fee3329030e66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.types.{AtomicType, Decimal} -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} object ColumnarTestUtils { def makeNullRow(length: Int): GenericInternalRow = { @@ -51,6 +51,8 @@ object ColumnarTestUtils { case DOUBLE => Random.nextDouble() case STRING => UTF8String.fromString(Random.nextString(Random.nextInt(32))) case BINARY => randomBytes(Random.nextInt(32)) + case CALENDAR_INTERVAL => + new CalendarInterval(Random.nextInt(), Random.nextInt(), Random.nextLong()) case COMPACT_DECIMAL(precision, scale) => Decimal(Random.nextLong() % 100, precision, scale) case LARGE_DECIMAL(precision, scale) => Decimal(Random.nextLong(), precision, scale) case STRUCT(_) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala index 8f4ca3cea77a5..92d9d84d9fac6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala @@ -44,7 +44,8 @@ class NullableColumnAccessorSuite extends SparkFunSuite { NULL, BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, STRING, BINARY, COMPACT_DECIMAL(15, 10), LARGE_DECIMAL(20, 10), STRUCT(StructType(StructField("a", StringType) :: Nil)), - ARRAY(ArrayType(IntegerType)), MAP(MapType(IntegerType, StringType))) + ARRAY(ArrayType(IntegerType)), MAP(MapType(IntegerType, StringType)), + CALENDAR_INTERVAL) .foreach { testNullableColumnAccessor(_) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala index b2b6e92e9a056..7e295b4dc31c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala @@ -42,7 +42,8 @@ class NullableColumnBuilderSuite extends SparkFunSuite { BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, STRING, BINARY, COMPACT_DECIMAL(15, 10), LARGE_DECIMAL(20, 10), STRUCT(StructType(StructField("a", StringType) :: Nil)), - ARRAY(ArrayType(IntegerType)), MAP(MapType(IntegerType, StringType))) + ARRAY(ArrayType(IntegerType)), MAP(MapType(IntegerType, StringType)), + CALENDAR_INTERVAL) .foreach { testNullableColumnBuilder(_) } From 46e950bea883b98cd3beb7bd637bffe522656435 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 14 Dec 2019 13:13:54 -0800 Subject: [PATCH 0880/1387] [SPARK-30263][CORE] Don't log potentially sensitive value of non-Spark properties ignored in spark-submit ### What changes were proposed in this pull request? The value of non-Spark config properties ignored in spark-submit is no longer logged. ### Why are the changes needed? The value isn't really needed in the logs, and could contain potentially sensitive info. While we can redact the values selectively too, I figured it's more robust to just not log them at all here, as the values aren't important in this log statement. ### Does this PR introduce any user-facing change? Other than the change to logging above, no. ### How was this patch tested? Existing tests Closes #26893 from srowen/SPARK-30263. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 f1ed4f2d62d70..3f7cfea778ac6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -137,10 +137,10 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S * Remove keys that don't start with "spark." from `sparkProperties`. */ private def ignoreNonSparkProperties(): Unit = { - sparkProperties.foreach { case (k, v) => + sparkProperties.keys.foreach { k => if (!k.startsWith("spark.")) { sparkProperties -= k - logWarning(s"Ignoring non-spark config property: $k=$v") + logWarning(s"Ignoring non-Spark config property: $k") } } } From f483a13d4aee2b2a98a9e336d4832eefaac2053b Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sat, 14 Dec 2019 14:26:50 -0800 Subject: [PATCH 0881/1387] [SPARK-30231][SQL][PYTHON][FOLLOWUP] Make error messages clear in PySpark df.explain ### What changes were proposed in this pull request? This pr is a followup of #26861 to address minor comments from viirya. ### Why are the changes needed? For better error messages. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested. Closes #26886 from maropu/SPARK-30231-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/dataframe.py | 11 ++++++----- .../src/main/scala/org/apache/spark/sql/Dataset.scala | 1 + 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 9f86086dad83d..43dd30b8ca8ee 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -305,11 +305,12 @@ def explain(self, extended=None, mode=None): is_mode_case = mode is not None and isinstance(mode, basestring) if not is_no_argument and not (is_extended_case or is_mode_case): - argtypes = [ - str(type(arg)) for arg in [extended, mode] if arg is not None] - raise TypeError( - "extended (optional) and mode (optional) should be a bool and str; " - "however, got [%s]." % ", ".join(argtypes)) + if extended is not None: + errMsg = "extended (optional) should be provided as bool" \ + ", got {0}".format(type(extended)) + else: # For mode case + errMsg = "mode (optional) should be provided as str, got {0}".format(type(mode)) + raise TypeError(errMsg) # Sets an explain mode depending on a given argument if is_no_argument: 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 42338bd8accac..3fb82420a515c 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 @@ -547,6 +547,7 @@ class Dataset[T] private[sql]( } } + // This method intends to be called from PySpark DataFrame private[sql] def toExplainString(mode: String): String = { mode.toLowerCase(Locale.ROOT) match { case "simple" => toExplainString(ExplainMode.Simple) From 4cbef8988ea1e2f3e7c24a4043f1be9ff18973e8 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Sat, 14 Dec 2019 15:36:14 -0800 Subject: [PATCH 0882/1387] [SPARK-30259][SQL] Fix CREATE TABLE behavior when session catalog is specified explicitly MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Fix bug : CREATE TABLE throw error when session catalog specified explicitly. ### Why are the changes needed? Currently, Spark throw error when the session catalog is specified explicitly in "CREATE TABLE" and "CREATE TABLE AS SELECT" command, eg.  > CREATE TABLE spark_catalog.tbl USING json AS SELECT 1 AS i; the error message is like below: > 19/12/14 10:56:08 INFO HiveMetaStore: 0: get_table : db=spark_catalog tbl=tbl > 19/12/14 10:56:08 INFO audit: ugi=fuwhu ip=unknown-ip-addr cmd=get_table : db=spark_catalog tbl=tbl > 19/12/14 10:56:08 INFO HiveMetaStore: 0: get_database: spark_catalog > 19/12/14 10:56:08 INFO audit: ugi=fuwhu ip=unknown-ip-addr cmd=get_database: spark_catalog > 19/12/14 10:56:08 WARN ObjectStore: Failed to get database spark_catalog, returning NoSuchObjectException > Error in query: Database 'spark_catalog' not found; ### Does this PR introduce any user-facing change? Yes, after this PR, "CREATE TALBE" and "CREATE TABLE AS SELECT" can complete successfully when session catalog "spark_catalog" specified explicitly. ### How was this patch tested? New unit tests added. Closes #26887 from fuwhu/SPARK-30259. Authored-by: fuwhu Signed-off-by: Dongjoon Hyun --- .../analysis/ResolveSessionCatalog.scala | 4 ++-- .../sql/connector/DataSourceV2SQLSuite.scala | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index afe15efdbf7ec..1dedf6e777391 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -221,7 +221,7 @@ class ResolveSessionCatalog( case c @ CreateTableStatement( SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => if (!isV2Provider(c.provider)) { - val tableDesc = buildCatalogTable(c.tableName.asTableIdentifier, c.tableSchema, + val tableDesc = buildCatalogTable(tbl.asTableIdentifier, c.tableSchema, c.partitioning, c.bucketSpec, c.properties, c.provider, c.options, c.location, c.comment, c.ifNotExists) val mode = if (c.ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists @@ -240,7 +240,7 @@ class ResolveSessionCatalog( case c @ CreateTableAsSelectStatement( SessionCatalogAndTable(catalog, tbl), _, _, _, _, _, _, _, _, _) => if (!isV2Provider(c.provider)) { - val tableDesc = buildCatalogTable(c.tableName.asTableIdentifier, new StructType, + val tableDesc = buildCatalogTable(tbl.asTableIdentifier, new StructType, c.partitioning, c.bucketSpec, c.properties, c.provider, c.options, c.location, c.comment, c.ifNotExists) val mode = if (c.ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 35822b99e3aba..cfcd8c7a73d5c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1898,6 +1898,22 @@ class DataSourceV2SQLSuite } } + test("SPARK-30259: session catalog can be specified in CREATE TABLE AS SELECT command") { + withTable("tbl") { + val ident = Identifier.of(Array(), "tbl") + sql("CREATE TABLE spark_catalog.tbl USING json AS SELECT 1 AS i") + assert(catalog("spark_catalog").asTableCatalog.tableExists(ident) === true) + } + } + + test("SPARK-30259: session catalog can be specified in CREATE TABLE command") { + withTable("tbl") { + val ident = Identifier.of(Array(), "tbl") + sql("CREATE TABLE spark_catalog.tbl (col string) USING json") + assert(catalog("spark_catalog").asTableCatalog.tableExists(ident) === true) + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") From fb2f5a49061f4593648a9822bacaea8bfd046505 Mon Sep 17 00:00:00 2001 From: xiaodeshan Date: Sat, 14 Dec 2019 17:15:30 -0800 Subject: [PATCH 0883/1387] [SPARK-25100][CORE] Register TaskCommitMessage to KyroSerializer ## What changes were proposed in this pull request? Fix the bug when invoking saveAsNewAPIHadoopDataset to store data, the job will fail because the class TaskCommitMessage hasn't be registered if serializer is KryoSerializer and spark.kryo.registrationRequired is true ## How was this patch tested? UT Closes #26714 from deshanxiao/SPARK-25100. Authored-by: xiaodeshan Signed-off-by: Dongjoon Hyun --- .../spark/serializer/KryoSerializer.scala | 4 ++- .../scala/org/apache/spark/FileSuite.scala | 30 +++++++++++++++++++ .../serializer/KryoSerializerSuite.scala | 23 ++++++++++++++ 3 files changed, 56 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 6efb8b35733ef..cdaab599e2a0b 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -40,6 +40,7 @@ import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.internal.io.FileCommitProtocol._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ @@ -469,7 +470,8 @@ private[serializer] object KryoSerializer { classOf[Array[String]], classOf[Array[Array[String]]], classOf[BoundedPriorityQueue[_]], - classOf[SparkConf] + classOf[SparkConf], + classOf[TaskCommitMessage] ) private val toRegisterSerializer = Map[Class[_], KryoClassSerializer[_]]( diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 0368d77e3d5f1..ed11653c810d7 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutput import org.apache.spark.internal.config._ import org.apache.spark.rdd.{HadoopRDD, NewHadoopRDD} +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -700,4 +701,33 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { assert(collectRDDAndDeleteFileBeforeCompute(true).isEmpty) } + + test("SPARK-25100: Using KryoSerializer and" + + "setting registrationRequired true can lead job failed") { + val inputFile = new File(tempDir, "/input").getAbsolutePath + val textFileOutputDir = new File(tempDir, "/out1").getAbsolutePath + val dataSetDir = new File(tempDir, "/out2").getAbsolutePath + + Utils.tryWithResource(new PrintWriter(new File(inputFile))) { writer => + for (i <- 1 to 100) { + writer.print(i) + writer.write('\n') + } + } + + val conf = new SparkConf(false).setMaster("local"). + set("spark.kryo.registrationRequired", "true").setAppName("test") + conf.set("spark.serializer", classOf[KryoSerializer].getName) + + val jobConf = new JobConf() + jobConf.setOutputKeyClass(classOf[IntWritable]) + jobConf.setOutputValueClass(classOf[IntWritable]) + jobConf.set("mapred.output.dir", dataSetDir) + + sc = new SparkContext(conf) + val pairRDD = sc.textFile(inputFile).map(x => (x, 1)) + + pairRDD.saveAsTextFile(textFileOutputDir) + pairRDD.saveAsNewAPIHadoopDataset(jobConf) + } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index d7c151209fcac..c55efe9821541 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -34,6 +34,7 @@ import org.roaringbitmap.RoaringBitmap import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ import org.apache.spark.storage.BlockManagerId @@ -358,6 +359,28 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } } + test("registration of TaskCommitMessage") { + val conf = new SparkConf(false) + conf.set(KRYO_REGISTRATION_REQUIRED, true) + + val ser = new KryoSerializer(conf).newInstance() + // In HadoopMapReduceCommitProtocol#commitTask + val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map() + addedAbsPathFiles.put("test1", "test1") + addedAbsPathFiles.put("test2", "test2") + + val partitionPaths: mutable.Set[String] = mutable.Set() + partitionPaths.add("test3") + + val taskCommitMessage1 = new TaskCommitMessage(addedAbsPathFiles.toMap -> partitionPaths.toSet) + val taskCommitMessage2 = new TaskCommitMessage(Map.empty -> Set.empty) + Seq(taskCommitMessage1, taskCommitMessage2).foreach { taskCommitMessage => + val obj1 = ser.deserialize[TaskCommitMessage](ser.serialize(taskCommitMessage)).obj + val obj2 = taskCommitMessage.obj + assert(obj1 == obj2) + } + } + test("serialization buffer overflow reporting") { import org.apache.spark.SparkException val kryoBufferMaxProperty = KRYO_SERIALIZER_MAX_BUFFER_SIZE.key From a9fbd310300e57ed58818d7347f3c3172701c491 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sat, 14 Dec 2019 17:39:06 -0800 Subject: [PATCH 0884/1387] [SPARK-30240][CORE] Support HTTP redirects directly to a proxy server ### What changes were proposed in this pull request? The PR adds a new config option to configure an address for the proxy server, and a new handler that intercepts redirects and replaces the URL with one pointing at the proxy server. This is needed on top of the "proxy base path" support because redirects use full URLs, not just absolute paths from the server's root. ### Why are the changes needed? Spark's web UI has support for generating links to paths with a prefix, to support a proxy server, but those do not apply when the UI is responding with redirects. In that case, Spark is sending its own URL back to the client, and if it's behind a dumb proxy server that doesn't do rewriting (like when using stunnel for HTTPS support) then the client will see the wrong URL and may fail. ### Does this PR introduce any user-facing change? Yes. It's a new UI option. ### How was this patch tested? Tested with added unit test, with Spark behind stunnel, and in a more complicated app using a different HTTPS proxy. Closes #26873 from vanzin/SPARK-30240. Authored-by: Marcelo Vanzin Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/TestUtils.scala | 12 +++- .../org/apache/spark/internal/config/UI.scala | 5 ++ .../org/apache/spark/ui/JettyUtils.scala | 60 ++++++++++++++++++- .../scala/org/apache/spark/ui/UISuite.scala | 53 +++++++++++++++- docs/configuration.md | 12 ++++ 5 files changed, 136 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 1f0636433f573..d459627930f4c 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -247,6 +247,16 @@ private[spark] object TestUtils { url: URL, method: String = "GET", headers: Seq[(String, String)] = Nil): Int = { + withHttpConnection(url, method, headers = headers) { connection => + connection.getResponseCode() + } + } + + def withHttpConnection[T]( + url: URL, + method: String = "GET", + headers: Seq[(String, String)] = Nil) + (fn: HttpURLConnection => T): T = { val connection = url.openConnection().asInstanceOf[HttpURLConnection] connection.setRequestMethod(method) headers.foreach { case (k, v) => connection.setRequestProperty(k, v) } @@ -271,7 +281,7 @@ private[spark] object TestUtils { try { connection.connect() - connection.getResponseCode() + fn(connection) } finally { connection.disconnect() } diff --git a/core/src/main/scala/org/apache/spark/internal/config/UI.scala b/core/src/main/scala/org/apache/spark/internal/config/UI.scala index 1a8268161160b..60d985713d30e 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/UI.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/UI.scala @@ -150,6 +150,11 @@ private[spark] object UI { .stringConf .createWithDefault("org.apache.spark.security.ShellBasedGroupsMappingProvider") + val PROXY_REDIRECT_URI = ConfigBuilder("spark.ui.proxyRedirectUri") + .doc("Proxy address to use when responding with HTTP redirects.") + .stringConf + .createOptional + val CUSTOM_EXECUTOR_LOG_URL = ConfigBuilder("spark.ui.custom.executor.log.url") .doc("Specifies custom spark executor log url for supporting external log service instead of " + "using cluster managers' application log urls in the Spark UI. Spark will support " + diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index cd4104731d400..94c99d48e773c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui import java.net.{URI, URL} import java.util.EnumSet import javax.servlet.DispatcherType -import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} +import javax.servlet.http._ import scala.language.implicitConversions import scala.xml.Node @@ -259,7 +259,15 @@ private[spark] object JettyUtils extends Logging { server.addBean(errorHandler) val collection = new ContextHandlerCollection - server.setHandler(collection) + conf.get(PROXY_REDIRECT_URI) match { + case Some(proxyUri) => + val proxyHandler = new ProxyRedirectHandler(proxyUri) + proxyHandler.setHandler(collection) + server.setHandler(proxyHandler) + + case _ => + server.setHandler(collection) + } // Executor used to create daemon threads for the Jetty connectors. val serverExecutor = new ScheduledExecutorScheduler(s"$serverName-JettyScheduler", true) @@ -526,3 +534,51 @@ private[spark] case class ServerInfo( } } + +/** + * A Jetty handler to handle redirects to a proxy server. It intercepts redirects and rewrites the + * location to point to the proxy server. + * + * The handler needs to be set as the server's handler, because Jetty sometimes generates redirects + * before invoking any servlet handlers or filters. One of such cases is when asking for the root of + * a servlet context without the trailing slash (e.g. "/jobs") - Jetty will send a redirect to the + * same URL, but with a trailing slash. + */ +private class ProxyRedirectHandler(_proxyUri: String) extends HandlerWrapper { + + private val proxyUri = _proxyUri.stripSuffix("/") + + override def handle( + target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + super.handle(target, baseRequest, request, new ResponseWrapper(request, response)) + } + + private class ResponseWrapper( + req: HttpServletRequest, + res: HttpServletResponse) + extends HttpServletResponseWrapper(res) { + + override def sendRedirect(location: String): Unit = { + val newTarget = if (location != null) { + val target = new URI(location) + val path = if (target.getPath().startsWith("/")) { + target.getPath() + } else { + req.getRequestURI().stripSuffix("/") + "/" + target.getPath() + } + // The target path should already be encoded, so don't re-encode it, just the + // proxy address part. + val proxyBase = UIUtils.uiRoot(req) + val proxyPrefix = if (proxyBase.nonEmpty) s"$proxyUri$proxyBase" else proxyUri + s"${res.encodeURL(proxyPrefix)}${target.getPath()}" + } else { + null + } + super.sendRedirect(newTarget) + } + } + +} diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 34fd218437f87..2ad4a634cd9a7 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -32,7 +32,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.LocalSparkContext._ -import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.internal.config.UI import org.apache.spark.util.Utils class UISuite extends SparkFunSuite { @@ -45,7 +45,7 @@ class UISuite extends SparkFunSuite { val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set(UI_ENABLED, true) + .set(UI.UI_ENABLED, true) val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc @@ -273,7 +273,6 @@ class UISuite extends SparkFunSuite { val (_, testContext) = newContext("/test2") serverInfo.addHandler(testContext, securityMgr) - testContext.start() val httpPort = serverInfo.boundPort @@ -318,6 +317,54 @@ class UISuite extends SparkFunSuite { } } + test("redirect with proxy server support") { + val proxyRoot = "https://proxy.example.com:443/prefix" + val (conf, securityMgr, sslOptions) = sslDisabledConf() + conf.set(UI.PROXY_REDIRECT_URI, proxyRoot) + + val serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, conf) + try { + val serverAddr = s"http://localhost:${serverInfo.boundPort}" + + val (_, ctx) = newContext("/ctx1") + serverInfo.addHandler(ctx, securityMgr) + + val redirect = JettyUtils.createRedirectHandler("/src", "/dst") + serverInfo.addHandler(redirect, securityMgr) + + // Test Jetty's built-in redirect to add the trailing slash to the context path. + TestUtils.withHttpConnection(new URL(s"$serverAddr/ctx1")) { conn => + assert(conn.getResponseCode() === HttpServletResponse.SC_FOUND) + val location = Option(conn.getHeaderFields().get("Location")) + .map(_.get(0)).orNull + assert(location === s"$proxyRoot/ctx1/") + } + + // Test with a URL handled by the added redirect handler, and also including a path prefix. + val headers = Seq("X-Forwarded-Context" -> "/prefix") + TestUtils.withHttpConnection( + new URL(s"$serverAddr/src/"), + headers = headers) { conn => + assert(conn.getResponseCode() === HttpServletResponse.SC_FOUND) + val location = Option(conn.getHeaderFields().get("Location")) + .map(_.get(0)).orNull + assert(location === s"$proxyRoot/prefix/dst") + } + + // Not really used by Spark, but test with a relative redirect. + val relative = JettyUtils.createRedirectHandler("/rel", "root") + serverInfo.addHandler(relative, securityMgr) + TestUtils.withHttpConnection(new URL(s"$serverAddr/rel/")) { conn => + assert(conn.getResponseCode() === HttpServletResponse.SC_FOUND) + val location = Option(conn.getHeaderFields().get("Location")) + .map(_.get(0)).orNull + assert(location === s"$proxyRoot/rel/root") + } + } finally { + stopServer(serverInfo) + } + } + /** * Create a new context handler for the given path, with a single servlet that responds to * requests in `$path/root`. diff --git a/docs/configuration.md b/docs/configuration.md index 93758967a29d9..497a2ad36b67c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1106,6 +1106,18 @@ Apart from these, the following properties are also available, and may be useful This is the URL where your proxy is running. This URL is for proxy which is running in front of Spark Master. This is useful when running proxy for authentication e.g. OAuth proxy. Make sure this is a complete URL including scheme (http/https) and port to reach your proxy. + + spark.ui.proxyRedirectUri + + + Where to address redirects when Spark is running behind a proxy. This will make Spark + modify redirect responses so they point to the proxy server, instead of the Spark UI's own + address. This should be only the address of the server, without any prefix paths for the + application; the prefix should be set either by the proxy server itself (by adding the + X-Forwarded-Context request header), or by setting the proxy base in the Spark + app's configuration. + + spark.ui.showConsoleProgress false From 58b29392f85411eee98631ff30461254ff7bf595 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sun, 15 Dec 2019 08:42:16 -0600 Subject: [PATCH 0885/1387] [SPARK-30173][INFRA] Automatically close stale PRs ### What changes were proposed in this pull request? This PR adds [a GitHub workflow to automatically close stale PRs](https://github.com/marketplace/actions/close-stale-issues). ### Why are the changes needed? This will help cut down the number of open but stale PRs and keep the PR queue manageable. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? I'm not sure how to test this PR without impacting real PRs on the repo. See: https://github.com/actions/stale/issues/32 Closes #26877 from nchammas/SPARK-30173-stale-prs. Authored-by: Nicholas Chammas Signed-off-by: Sean Owen --- .github/workflows/stale.yml | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 .github/workflows/stale.yml diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml new file mode 100644 index 0000000000000..0201eadd27928 --- /dev/null +++ b/.github/workflows/stale.yml @@ -0,0 +1,23 @@ +name: Close stale PRs + +on: + schedule: + - cron: "0 0 * * *" + +jobs: + stale: + runs-on: ubuntu-latest + steps: + - uses: actions/stale@v1.1.0 + with: + repo-token: ${{ secrets.GITHUB_TOKEN }} + stale-pr-message: | + We're closing this PR because it hasn't been updated in a while. + This isn't a judgement on the merit of the PR in any way. It's just + a way of keeping the PR queue manageable. + + If you'd like to revive this PR, please reopen it! + days-before-stale: 100 + # Setting this to 0 is the same as setting it to 1. + # See: https://github.com/actions/stale/issues/28 + days-before-close: 0 From 67b644c3d74b0587dd5d498b903383ac4de932fe Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 15 Dec 2019 08:45:57 -0600 Subject: [PATCH 0886/1387] [SPARK-30166][SQL] Eliminate compilation warnings in JSONOptions ### What changes were proposed in this pull request? In the PR, I propose to replace `setJacksonOptions()` in `JSONOptions` by `buildJsonFactory()` which builds `JsonFactory` using `JsonFactoryBuilder`. This allows to avoid using **deprecated** feature configurations from `JsonParser.Feature`. ### Why are the changes needed? - The changes eliminate the following compilation warnings in `sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala`: ``` Warning:Warning:line (137)Java enum ALLOW_NUMERIC_LEADING_ZEROS in Java enum Feature is deprecated: see corresponding Javadoc for more information. factory.configure(JsonParser.Feature.ALLOW_NUMERIC_LEADING_ZEROS, allowNumericLeadingZeros) Warning:Warning:line (138)Java enum ALLOW_NON_NUMERIC_NUMBERS in Java enum Feature is deprecated: see corresponding Javadoc for more information. factory.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, allowNonNumericNumbers) Warning:Warning:line (139)Java enum ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER in Java enum Feature is deprecated: see corresponding Javadoc for more information. factory.configure(JsonParser.Feature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER, Warning:Warning:line (141)Java enum ALLOW_UNQUOTED_CONTROL_CHARS in Java enum Feature is deprecated: see corresponding Javadoc for more information. factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS, allowUnquotedControlChars) ``` - This put together building JsonFactory and set options from JSONOptions. So, we will not forget to call `setJacksonOptions` in the future. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By `JsonSuite`, `JsonFunctionsSuite`, `JsonExpressionsSuite`. Closes #26797 from MaxGekk/eliminate-warning. Authored-by: Maxim Gekk Signed-off-by: Sean Owen --- .../expressions/jsonExpressions.scala | 15 ++++------ .../spark/sql/catalyst/json/JSONOptions.scala | 28 +++++++++++-------- .../sql/catalyst/json/JacksonParser.scala | 3 +- .../sql/catalyst/json/JsonInferSchema.scala | 3 +- .../catalyst/json/JsonInferSchemaSuite.scala | 3 +- .../json/JsonParsingOptionsSuite.scala | 2 +- 6 files changed, 26 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index de7e1160185dc..a1e3a84bd045d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -22,6 +22,7 @@ import java.io._ import scala.util.parsing.combinator.RegexParsers import com.fasterxml.jackson.core._ +import com.fasterxml.jackson.core.json.JsonReadFeature import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult @@ -99,10 +100,10 @@ private[this] object JsonPathParser extends RegexParsers { } private[this] object SharedFactory { - val jsonFactory = new JsonFactory() - - // Enabled for Hive compatibility - jsonFactory.enable(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS) + val jsonFactory = new JsonFactoryBuilder() + // Enabled for Hive compatibility + .enable(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS) + .build() } /** @@ -756,11 +757,7 @@ case class SchemaOfJson( private lazy val jsonOptions = new JSONOptions(options, "UTC") @transient - private lazy val jsonFactory = { - val factory = new JsonFactory() - jsonOptions.setJacksonOptions(factory) - factory - } + private lazy val jsonFactory = jsonOptions.buildJsonFactory() @transient private lazy val jsonInferSchema = new JsonInferSchema(jsonOptions) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index 4952540f1132d..cdf4b4689e821 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -21,7 +21,8 @@ import java.nio.charset.{Charset, StandardCharsets} import java.time.ZoneId import java.util.Locale -import com.fasterxml.jackson.core.{JsonFactory, JsonParser} +import com.fasterxml.jackson.core.{JsonFactory, JsonFactoryBuilder} +import com.fasterxml.jackson.core.json.JsonReadFeature import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ @@ -30,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf /** * Options for parsing JSON data into Spark SQL rows. * - * Most of these map directly to Jackson's internal options, specified in [[JsonParser.Feature]]. + * Most of these map directly to Jackson's internal options, specified in [[JsonReadFeature]]. */ private[sql] class JSONOptions( @transient val parameters: CaseInsensitiveMap[String], @@ -129,16 +130,19 @@ private[sql] class JSONOptions( */ val inferTimestamp: Boolean = parameters.get("inferTimestamp").map(_.toBoolean).getOrElse(true) - /** Sets config options on a Jackson [[JsonFactory]]. */ - def setJacksonOptions(factory: JsonFactory): Unit = { - factory.configure(JsonParser.Feature.ALLOW_COMMENTS, allowComments) - factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, allowUnquotedFieldNames) - factory.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, allowSingleQuotes) - factory.configure(JsonParser.Feature.ALLOW_NUMERIC_LEADING_ZEROS, allowNumericLeadingZeros) - factory.configure(JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS, allowNonNumericNumbers) - factory.configure(JsonParser.Feature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER, - allowBackslashEscapingAnyCharacter) - factory.configure(JsonParser.Feature.ALLOW_UNQUOTED_CONTROL_CHARS, allowUnquotedControlChars) + /** Build a Jackson [[JsonFactory]] using JSON options. */ + def buildJsonFactory(): JsonFactory = { + new JsonFactoryBuilder() + .configure(JsonReadFeature.ALLOW_JAVA_COMMENTS, allowComments) + .configure(JsonReadFeature.ALLOW_UNQUOTED_FIELD_NAMES, allowUnquotedFieldNames) + .configure(JsonReadFeature.ALLOW_SINGLE_QUOTES, allowSingleQuotes) + .configure(JsonReadFeature.ALLOW_LEADING_ZEROS_FOR_NUMBERS, allowNumericLeadingZeros) + .configure(JsonReadFeature.ALLOW_NON_NUMERIC_NUMBERS, allowNonNumericNumbers) + .configure( + JsonReadFeature.ALLOW_BACKSLASH_ESCAPING_ANY_CHARACTER, + allowBackslashEscapingAnyCharacter) + .configure(JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS, allowUnquotedControlChars) + .build() } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index b534b5a3d2d64..ead26665bd6ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -52,8 +52,7 @@ class JacksonParser( // `ValueConverter`s for the root schema for all fields in the schema private val rootConverter = makeRootConverter(schema) - private val factory = new JsonFactory() - options.setJacksonOptions(factory) + private val factory = options.buildJsonFactory() private val timestampFormatter = TimestampFormatter( options.timestampFormat, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index c5a97c7b8835a..f030955ee6e7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -57,8 +57,7 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { // In each RDD partition, perform schema inference on each row and merge afterwards. val typeMerger = JsonInferSchema.compatibleRootType(columnNameOfCorruptRecord, parseMode) val mergedTypesFromPartitions = json.mapPartitions { iter => - val factory = new JsonFactory() - options.setJacksonOptions(factory) + val factory = options.buildJsonFactory() iter.flatMap { row => try { Utils.tryWithResource(createParser(factory, row)) { parser => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala index 8ce45f06ba65d..a48e61861c158 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala @@ -28,8 +28,7 @@ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { def checkType(options: Map[String, String], json: String, dt: DataType): Unit = { val jsonOptions = new JSONOptions(options, "UTC", "") val inferSchema = new JsonInferSchema(jsonOptions) - val factory = new JsonFactory() - jsonOptions.setJacksonOptions(factory) + val factory = jsonOptions.buildJsonFactory() val parser = CreateJacksonParser.string(factory, json) parser.nextToken() val expectedType = StructType(Seq(StructField("a", dt, true))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index bafb6769af69c..7592809d7c85b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -103,7 +103,7 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession { } // The following two tests are not really working - need to look into Jackson's - // JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS. + // JsonReadFeature.ALLOW_NON_NUMERIC_NUMBERS. ignore("allowNonNumericNumbers off") { val str = """{"age": NaN}""" val df = spark.read.json(Seq(str).toDS()) From 26b658f6fbcc94031cf1ac38f5e8b090ea4856ee Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 15 Dec 2019 19:44:29 -0700 Subject: [PATCH 0887/1387] [SPARK-30253][INFRA] Do not add commits when releasing preview version ### What changes were proposed in this pull request? This PR add support do not add commits to master branch when releasing preview version. ### Why are the changes needed? We need manual revert this change, example: ![image](https://user-images.githubusercontent.com/5399861/70788945-f9d15180-1dcc-11ea-81f5-c0d89c28440a.png) ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test Closes #26879 from wangyum/SPARK-30253. Authored-by: Yuming Wang Signed-off-by: Yuming Wang --- dev/create-release/release-tag.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index 8024440759eb5..f8b730b06136b 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -104,7 +104,11 @@ git commit -a -m "Preparing development version $NEXT_VERSION" if ! is_dry_run; then # Push changes git push origin $RELEASE_TAG - git push origin HEAD:$GIT_BRANCH + if [[ $RELEASE_VERSION != *"preview"* ]]; then + git push origin HEAD:$GIT_BRANCH + else + echo "It's preview release. We only push $RELEASE_TAG to remote." + fi cd .. rm -rf spark From 0a2afcec7dac9b18e876f10072e7615f190def88 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 16 Dec 2019 14:42:35 +0900 Subject: [PATCH 0888/1387] [SPARK-30200][SQL][FOLLOW-UP] Expose only explain(mode: String) in Scala side, and clean up related codes ### What changes were proposed in this pull request? This PR mainly targets: 1. Expose only explain(mode: String) in Scala side 2. Clean up related codes - Hide `ExplainMode` under private `execution` package. No particular reason but just because `ExplainUtils` exists there - Use `case object` + `trait` pattern in `ExplainMode` to look after `ParseMode`. - Move `Dataset.toExplainString` to `QueryExecution.explainString` to look after `QueryExecution.simpleString`, and deduplicate the codes at `ExplainCommand`. - Use `ExplainMode` in `ExplainCommand` too. - Add `explainString` to `PythonSQLUtils` to avoid unexpected test failure of PySpark during refactoring Scala codes side. ### Why are the changes needed? To minimised exposed APIs, deduplicate, and clean up. ### Does this PR introduce any user-facing change? `Dataset.explain(mode: ExplainMode)` will be removed (which only exists in master). ### How was this patch tested? Manually tested and existing tests should cover. Closes #26898 from HyukjinKwon/SPARK-30200-followup. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/dataframe.py | 14 ++-- .../org/apache/spark/sql/ExplainMode.java | 64 ----------------- .../scala/org/apache/spark/sql/Dataset.scala | 54 +++----------- .../spark/sql/api/python/PythonSQLUtils.scala | 7 +- .../spark/sql/execution/ExplainMode.scala | 72 +++++++++++++++++++ .../spark/sql/execution/QueryExecution.scala | 32 +++++++++ .../spark/sql/execution/SparkSqlParser.scala | 11 +-- .../sql/execution/command/commands.scala | 50 ++----------- .../org/apache/spark/sql/ExplainSuite.scala | 31 ++++---- .../scala/org/apache/spark/sql/UDFSuite.scala | 4 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 8 +-- .../spark/sql/streaming/StreamSuite.scala | 5 +- 12 files changed, 163 insertions(+), 189 deletions(-) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainMode.scala diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 43dd30b8ca8ee..2fa90d67880c3 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -306,21 +306,21 @@ def explain(self, extended=None, mode=None): if not is_no_argument and not (is_extended_case or is_mode_case): if extended is not None: - errMsg = "extended (optional) should be provided as bool" \ + err_msg = "extended (optional) should be provided as bool" \ ", got {0}".format(type(extended)) else: # For mode case - errMsg = "mode (optional) should be provided as str, got {0}".format(type(mode)) - raise TypeError(errMsg) + err_msg = "mode (optional) should be provided as str, got {0}".format(type(mode)) + raise TypeError(err_msg) # Sets an explain mode depending on a given argument if is_no_argument: - explainMode = "simple" + explain_mode = "simple" elif is_extended_case: - explainMode = "extended" if extended else "simple" + explain_mode = "extended" if extended else "simple" elif is_mode_case: - explainMode = mode + explain_mode = mode - print(self._jdf.toExplainString(explainMode)) + print(self._sc._jvm.PythonSQLUtils.explainString(self._jdf.queryExecution(), explain_mode)) @since(2.4) def exceptAll(self, other): diff --git a/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java b/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java deleted file mode 100644 index 50e11e94731e4..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/ExplainMode.java +++ /dev/null @@ -1,64 +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.sql; - -import org.apache.spark.annotation.Unstable; - -/** - * ExplainMode is used to specify the expected output format of plans (logical and physical) - * for debugging purpose. - * - * @since 3.0.0 - */ -@Unstable -public enum ExplainMode { - /** - * Simple mode means that when printing explain for a DataFrame, only a physical plan is - * expected to be printed to the console. - * - * @since 3.0.0 - */ - Simple, - /** - * Extended mode means that when printing explain for a DataFrame, both logical and physical - * plans are expected to be printed to the console. - * - * @since 3.0.0 - */ - Extended, - /** - * Codegen mode means that when printing explain for a DataFrame, if generated codes are - * available, a physical plan and the generated codes are expected to be printed to the console. - * - * @since 3.0.0 - */ - Codegen, - /** - * Cost mode means that when printing explain for a DataFrame, if plan node statistics are - * available, a logical plan and the statistics are expected to be printed to the console. - * - * @since 3.0.0 - */ - Cost, - /** - * Formatted mode means that when printing explain for a DataFrame, explain output is - * expected to be split into two sections: a physical plan outline and node details. - * - * @since 3.0.0 - */ - Formatted -} 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 3fb82420a515c..09f7578596bd7 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 @@ -18,7 +18,6 @@ package org.apache.spark.sql import java.io.{ByteArrayOutputStream, CharArrayWriter, DataOutputStream} -import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -522,53 +521,20 @@ class Dataset[T] private[sql]( def printSchema(level: Int): Unit = println(schema.treeString(level)) // scalastyle:on println - private def toExplainString(mode: ExplainMode): String = { - // Because temporary views are resolved during analysis when we create a Dataset, and - // `ExplainCommand` analyzes input query plan and resolves temporary views again. Using - // `ExplainCommand` here will probably output different query plans, compared to the results - // of evaluation of the Dataset. So just output QueryExecution's query plans here. - val qe = ExplainCommandUtil.explainedQueryExecution(sparkSession, logicalPlan, queryExecution) - - mode match { - case ExplainMode.Simple => - qe.simpleString - case ExplainMode.Extended => - qe.toString - case ExplainMode.Codegen => - try { - org.apache.spark.sql.execution.debug.codegenString(queryExecution.executedPlan) - } catch { - case e: AnalysisException => e.toString - } - case ExplainMode.Cost => - qe.stringWithStats - case ExplainMode.Formatted => - qe.simpleString(formatted = true) - } - } - - // This method intends to be called from PySpark DataFrame - private[sql] def toExplainString(mode: String): String = { - mode.toLowerCase(Locale.ROOT) match { - case "simple" => toExplainString(ExplainMode.Simple) - case "extended" => toExplainString(ExplainMode.Extended) - case "codegen" => toExplainString(ExplainMode.Codegen) - case "cost" => toExplainString(ExplainMode.Cost) - case "formatted" => toExplainString(ExplainMode.Formatted) - case _ => throw new IllegalArgumentException(s"Unknown explain mode: $mode. Accepted " + - "explain modes are 'simple', 'extended', 'codegen', 'cost', 'formatted'.") - } - } - /** * Prints the plans (logical and physical) with a format specified by a given explain mode. * * @group basic * @since 3.0.0 */ - def explain(mode: ExplainMode): Unit = { + def explain(mode: String): Unit = { + // Because temporary views are resolved during analysis when we create a Dataset, and + // `ExplainCommand` analyzes input query plan and resolves temporary views again. Using + // `ExplainCommand` here will probably output different query plans, compared to the results + // of evaluation of the Dataset. So just output QueryExecution's query plans here. + // scalastyle:off println - println(toExplainString(mode)) + println(queryExecution.explainString(ExplainMode.fromString(mode))) // scalastyle:on println } @@ -579,9 +545,9 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def explain(extended: Boolean): Unit = if (extended) { - explain(ExplainMode.Extended) + explain(ExtendedMode.name) } else { - explain(ExplainMode.Simple) + explain(SimpleMode.name) } /** @@ -590,7 +556,7 @@ class Dataset[T] private[sql]( * @group basic * @since 1.6.0 */ - def explain(): Unit = explain(ExplainMode.Simple) + def explain(): Unit = explain(SimpleMode.name) /** * Returns all column names and their data types as an array. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index 482e2bfeb7098..b232aa18c816e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -23,10 +23,11 @@ import java.nio.channels.Channels import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.PythonRDDServer import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.{ExplainMode, QueryExecution} import org.apache.spark.sql.execution.arrow.ArrowConverters import org.apache.spark.sql.types.DataType @@ -56,6 +57,10 @@ private[sql] object PythonSQLUtils { sqlContext: SQLContext): DataFrame = { ArrowConverters.toDataFrame(arrowBatchRDD, schemaString, sqlContext) } + + def explainString(queryExecution: QueryExecution, mode: String): String = { + queryExecution.explainString(ExplainMode.fromString(mode)) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainMode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainMode.scala new file mode 100644 index 0000000000000..0ceafe99cdfcf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExplainMode.scala @@ -0,0 +1,72 @@ +/* + * 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 + +import java.util.Locale + +sealed trait ExplainMode { + /** + * String name of the explain mode. + */ + def name: String +} + +/** + * Simple mode means that when printing explain for a DataFrame, only a physical plan is + * expected to be printed to the console. + */ +case object SimpleMode extends ExplainMode { val name = "simple" } + +/** + * Extended mode means that when printing explain for a DataFrame, both logical and physical + * plans are expected to be printed to the console. + */ +case object ExtendedMode extends ExplainMode { val name = "extended" } + +/** + * Codegen mode means that when printing explain for a DataFrame, if generated codes are + * available, a physical plan and the generated codes are expected to be printed to the console. + */ +case object CodegenMode extends ExplainMode { val name = "codegen" } + +/** + * Cost mode means that when printing explain for a DataFrame, if plan node statistics are + * available, a logical plan and the statistics are expected to be printed to the console. + */ +case object CostMode extends ExplainMode { val name = "cost" } + +/** + * Formatted mode means that when printing explain for a DataFrame, explain output is + * expected to be split into two sections: a physical plan outline and node details. + */ +case object FormattedMode extends ExplainMode { val name = "formatted" } + +object ExplainMode { + /** + * Returns the explain mode from the given string. + */ + def fromString(mode: String): ExplainMode = mode.toLowerCase(Locale.ROOT) match { + case SimpleMode.name => SimpleMode + case ExtendedMode.name => ExtendedMode + case CodegenMode.name => CodegenMode + case CostMode.name => CostMode + case FormattedMode.name => FormattedMode + case _ => throw new IllegalArgumentException(s"Unknown explain mode: $mode. Accepted " + + "explain modes are 'simple', 'extended', 'codegen', 'cost', 'formatted'.") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index a0ad1c72806f5..4467a1b1b1a7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import java.io.{BufferedWriter, OutputStreamWriter} +import java.util.UUID import org.apache.hadoop.fs.Path @@ -34,7 +35,9 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.dynamicpruning.PlanDynamicPruningFilters import org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} +import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.util.Utils /** @@ -132,6 +135,35 @@ class QueryExecution( concat.toString } + def explainString(mode: ExplainMode): String = { + val queryExecution = if (logical.isStreaming) { + // This is used only by explaining `Dataset/DataFrame` created by `spark.readStream`, so the + // output mode does not matter since there is no `Sink`. + new IncrementalExecution( + sparkSession, logical, OutputMode.Append(), "", + UUID.randomUUID, UUID.randomUUID, 0, OffsetSeqMetadata(0, 0)) + } else { + this + } + + mode match { + case SimpleMode => + queryExecution.simpleString + case ExtendedMode => + queryExecution.toString + case CodegenMode => + try { + org.apache.spark.sql.execution.debug.codegenString(queryExecution.executedPlan) + } catch { + case e: AnalysisException => e.toString + } + case CostMode => + queryExecution.stringWithStats + case FormattedMode => + queryExecution.simpleString(formatted = true) + } + } + private def writePlans(append: String => Unit, maxFields: Int): Unit = { val (verbose, addSuffix) = (true, false) append("== Parsed Logical Plan ==\n") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 6fcf62d26ad88..1eaef68506e44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -136,10 +136,13 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } else { ExplainCommand( logicalPlan = statement, - extended = ctx.EXTENDED != null, - codegen = ctx.CODEGEN != null, - cost = ctx.COST != null, - formatted = ctx.FORMATTED != null) + mode = { + if (ctx.EXTENDED != null) ExtendedMode + else if (ctx.CODEGEN != null) CodegenMode + else if (ctx.COST != null) CostMode + else if (ctx.FORMATTED != null) FormattedMode + else SimpleMode + }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 888c4419458d7..2e0eda5adb19b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} -import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{ExplainMode, LeafExecNode, QueryExecution, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.debug._ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} @@ -135,18 +135,11 @@ case class DataWritingCommandExec(cmd: DataWritingCommand, child: SparkPlan) * }}} * * @param logicalPlan plan to explain - * @param extended whether to do extended explain or not - * @param codegen whether to output generated code from whole-stage codegen or not - * @param cost whether to show cost information for operators. - * @param formatted whether to split explain output into two sections: a physical plan outline - * and node details. + * @param mode explain mode */ case class ExplainCommand( logicalPlan: LogicalPlan, - extended: Boolean = false, - codegen: Boolean = false, - cost: Boolean = false, - formatted: Boolean = false) + mode: ExplainMode) extends RunnableCommand { override val output: Seq[Attribute] = @@ -154,48 +147,13 @@ case class ExplainCommand( // Run through the optimizer to generate the physical plan. override def run(sparkSession: SparkSession): Seq[Row] = try { - val queryExecution = ExplainCommandUtil.explainedQueryExecution(sparkSession, logicalPlan, - sparkSession.sessionState.executePlan(logicalPlan)) - val outputString = - if (codegen) { - try { - codegenString(queryExecution.executedPlan) - } catch { - case e: AnalysisException => e.toString - } - } else if (extended) { - queryExecution.toString - } else if (cost) { - queryExecution.stringWithStats - } else if (formatted) { - queryExecution.simpleString(formatted = true) - } else { - queryExecution.simpleString - } + val outputString = sparkSession.sessionState.executePlan(logicalPlan).explainString(mode) Seq(Row(outputString)) } catch { case cause: TreeNodeException[_] => ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_)) } } -object ExplainCommandUtil { - // Returns `QueryExecution` which is used to explain a logical plan. - def explainedQueryExecution( - sparkSession: SparkSession, - logicalPlan: LogicalPlan, - queryExecution: => QueryExecution): QueryExecution = { - if (logicalPlan.isStreaming) { - // This is used only by explaining `Dataset/DataFrame` created by `spark.readStream`, so the - // output mode does not matter since there is no `Sink`. - new IncrementalExecution( - sparkSession, logicalPlan, OutputMode.Append(), "", - UUID.randomUUID, UUID.randomUUID, 0, OffsetSeqMetadata(0, 0)) - } else { - queryExecution - } - } -} - /** An explain command for users to see how a streaming batch is executed. */ case class StreamingExplainCommand( queryExecution: IncrementalExecution, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index ecb4aa273d8f5..27f9368bb50ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.execution._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -28,7 +29,7 @@ class ExplainSuite extends QueryTest with SharedSparkSession { private def getNormalizedExplain(df: DataFrame, mode: ExplainMode): String = { val output = new java.io.ByteArrayOutputStream() Console.withOut(output) { - df.explain(mode) + df.explain(mode.name) } output.toString.replaceAll("#\\d+", "#x") } @@ -66,7 +67,7 @@ class ExplainSuite extends QueryTest with SharedSparkSession { } private def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { - checkKeywordsExistsInExplain(df, ExplainMode.Extended, keywords: _*) + checkKeywordsExistsInExplain(df, ExtendedMode, keywords: _*) } test("SPARK-23034 show rdd names in RDD scan nodes (Dataset)") { @@ -217,7 +218,7 @@ class ExplainSuite extends QueryTest with SharedSparkSession { test("SPARK-26659: explain of DataWritingCommandExec should not contain duplicate cmd.nodeName") { withTable("temptable") { val df = sql("create table temptable using parquet as select * from range(2)") - withNormalizedExplain(df, ExplainMode.Simple) { normalizedOutput => + withNormalizedExplain(df, SimpleMode) { normalizedOutput => assert("Create\\w*?TableAsSelectCommand".r.findAllMatchIn(normalizedOutput).length == 1) } } @@ -276,7 +277,7 @@ class ExplainSuite extends QueryTest with SharedSparkSession { val df2 = Seq((2, 3), (1, 1)).toDF("k", "v2") val testDf = df1.join(df2, "k").groupBy("k").agg(count("v1"), sum("v1"), avg("v2")) - val simpleExplainOutput = getNormalizedExplain(testDf, ExplainMode.Simple) + val simpleExplainOutput = getNormalizedExplain(testDf, SimpleMode) assert(simpleExplainOutput.startsWith("== Physical Plan ==")) Seq("== Parsed Logical Plan ==", "== Analyzed Logical Plan ==", @@ -285,7 +286,7 @@ class ExplainSuite extends QueryTest with SharedSparkSession { } checkKeywordsExistsInExplain( testDf, - ExplainMode.Extended, + ExtendedMode, "== Parsed Logical Plan ==" :: "== Analyzed Logical Plan ==" :: "== Optimized Logical Plan ==" :: @@ -293,18 +294,18 @@ class ExplainSuite extends QueryTest with SharedSparkSession { Nil: _*) checkKeywordsExistsInExplain( testDf, - ExplainMode.Cost, + CostMode, "Statistics(sizeInBytes=" :: Nil: _*) checkKeywordsExistsInExplain( testDf, - ExplainMode.Codegen, + CodegenMode, "WholeStageCodegen subtrees" :: "Generated code:" :: Nil: _*) checkKeywordsExistsInExplain( testDf, - ExplainMode.Formatted, + FormattedMode, "* LocalTableScan (1)" :: "(1) LocalTableScan [codegen id :" :: Nil: _*) @@ -313,17 +314,17 @@ class ExplainSuite extends QueryTest with SharedSparkSession { test("Dataset.toExplainString has mode as string") { val df = spark.range(10).toDF def assertExplainOutput(mode: ExplainMode): Unit = { - assert(df.toExplainString(mode.toString).replaceAll("#\\d+", "#x").trim === + assert(df.queryExecution.explainString(mode).replaceAll("#\\d+", "#x").trim === getNormalizedExplain(df, mode).trim) } - assertExplainOutput(ExplainMode.Simple) - assertExplainOutput(ExplainMode.Extended) - assertExplainOutput(ExplainMode.Codegen) - assertExplainOutput(ExplainMode.Cost) - assertExplainOutput(ExplainMode.Formatted) + assertExplainOutput(SimpleMode) + assertExplainOutput(ExtendedMode) + assertExplainOutput(CodegenMode) + assertExplainOutput(CostMode) + assertExplainOutput(FormattedMode) val errMsg = intercept[IllegalArgumentException] { - df.toExplainString("unknown") + ExplainMode.fromString("unknown") }.getMessage assert(errMsg.contains("Unknown explain mode: unknown")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 268fb65144e72..49e6e8f2cb8e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -22,7 +22,7 @@ import java.math.BigDecimal import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.plans.logical.Project -import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.{QueryExecution, SimpleMode} import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, ExplainCommand} import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand @@ -309,7 +309,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { test("SPARK-19338 Provide identical names for UDFs in the EXPLAIN output") { def explainStr(df: DataFrame): String = { - val explain = ExplainCommand(df.queryExecution.logical, extended = false) + val explain = ExplainCommand(df.queryExecution.logical, SimpleMode) val sparkPlan = spark.sessionState.executePlan(explain).executedPlan sparkPlan.executeCollect().map(_.getString(0).trim).headOption.getOrElse("") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 43f6381c19790..66ddc6ee83d06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeTestUtils} -import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.{DataSourceScanExec, ExtendedMode} import org.apache.spark.sql.execution.command.{ExplainCommand, ShowCreateTableCommand} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCPartition, JDBCRDD, JDBCRelation, JdbcUtils} @@ -974,7 +974,7 @@ class JDBCSuite extends QueryTest test("test credentials in the properties are not in plan output") { val df = sql("SELECT * FROM parts") - val explain = ExplainCommand(df.queryExecution.logical, extended = true) + val explain = ExplainCommand(df.queryExecution.logical, ExtendedMode) spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { r => assert(!List("testPass", "testUser").exists(r.toString.contains)) } @@ -987,7 +987,7 @@ class JDBCSuite extends QueryTest test("test credentials in the connection url are not in the plan output") { val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) - val explain = ExplainCommand(df.queryExecution.logical, extended = true) + val explain = ExplainCommand(df.queryExecution.logical, ExtendedMode) spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { r => assert(!List("testPass", "testUser").exists(r.toString.contains)) } @@ -1009,7 +1009,7 @@ class JDBCSuite extends QueryTest | password '$password') """.stripMargin) - val explain = ExplainCommand(df.queryExecution.logical, extended = true) + val explain = ExplainCommand(df.queryExecution.logical, ExtendedMode) spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { r => assert(!r.toString.contains(password)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 297d6c7ca2171..bf80962d62c65 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.SimpleMode import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{ContinuousMemoryStream, MemorySink} @@ -472,7 +473,7 @@ class StreamSuite extends StreamTest { val df = inputData.toDS().map(_ + "foo").groupBy("value").agg(count("*")) // Test `df.explain` - val explain = ExplainCommand(df.queryExecution.logical, extended = false) + val explain = ExplainCommand(df.queryExecution.logical, SimpleMode) val explainString = spark.sessionState .executePlan(explain) @@ -524,7 +525,7 @@ class StreamSuite extends StreamTest { val df = inputData.toDS().map(_ * 2).filter(_ > 5) // Test `df.explain` - val explain = ExplainCommand(df.queryExecution.logical, extended = false) + val explain = ExplainCommand(df.queryExecution.logical, SimpleMode) val explainString = spark.sessionState .executePlan(explain) From 1fc353d51a62cb554e6af23dbc9a613e214e3af1 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 15 Dec 2019 23:16:17 -0700 Subject: [PATCH 0889/1387] Revert "[SPARK-30056][INFRA] Skip building test artifacts in `dev/make-distribution.sh` ### What changes were proposed in this pull request? This reverts commit 7c0ce285. ### Why are the changes needed? Failed to make distribution: ``` [INFO] -----------------< org.apache.spark:spark-sketch_2.12 >----------------- [INFO] Building Spark Project Sketch 3.0.0-preview2 [3/33] [INFO] --------------------------------[ jar ]--------------------------------- [INFO] Downloading from central: https://repo.maven.apache.org/maven2/org/apache/spark/spark-tags_2.12/3.0.0-preview2/spark-tags_2.12-3.0.0-preview2-tests.jar [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-preview2: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 26.513 s] [INFO] Spark Project Tags ................................. SUCCESS [ 48.393 s] [INFO] Spark Project Sketch ............................... FAILURE [ 0.034 s] [INFO] Spark Project Local DB ............................. SKIPPED [INFO] Spark Project Networking ........................... SKIPPED [INFO] Spark Project Shuffle Streaming Service ............ SKIPPED [INFO] Spark Project Unsafe ............................... SKIPPED [INFO] Spark Project Launcher ............................. SKIPPED [INFO] Spark Project Core ................................. SKIPPED [INFO] Spark Project ML Local Library ..................... SKIPPED [INFO] Spark Project GraphX ............................... SKIPPED [INFO] Spark Project Streaming ............................ SKIPPED [INFO] Spark Project Catalyst ............................. SKIPPED [INFO] Spark Project SQL .................................. SKIPPED [INFO] Spark Project ML Library ........................... SKIPPED [INFO] Spark Project Tools ................................ SKIPPED [INFO] Spark Project Hive ................................. SKIPPED [INFO] Spark Project Graph API ............................ SKIPPED [INFO] Spark Project Cypher ............................... SKIPPED [INFO] Spark Project Graph ................................ SKIPPED [INFO] Spark Project REPL ................................. SKIPPED [INFO] Spark Project YARN Shuffle Service ................. SKIPPED [INFO] Spark Project YARN ................................. SKIPPED [INFO] Spark Project Mesos ................................ SKIPPED [INFO] Spark Project Kubernetes ........................... SKIPPED [INFO] Spark Project Hive Thrift Server ................... SKIPPED [INFO] Spark Project Assembly ............................. SKIPPED [INFO] Kafka 0.10+ Token Provider for Streaming ........... SKIPPED [INFO] Spark Integration for Kafka 0.10 ................... SKIPPED [INFO] Kafka 0.10+ Source for Structured Streaming ........ SKIPPED [INFO] Spark Project Examples ............................. SKIPPED [INFO] Spark Integration for Kafka 0.10 Assembly .......... SKIPPED [INFO] Spark Avro ......................................... SKIPPED [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ [INFO] Total time: 01:15 min [INFO] Finished at: 2019-12-16T05:29:43Z [INFO] ------------------------------------------------------------------------ [ERROR] Failed to execute goal on project spark-sketch_2.12: Could not resolve dependencies for project org.apache.spark:spark-sketch_2.12:jar:3.0.0-preview2: Could not find artifact org.apache.spark:spark-tags_2.12:jar:tests:3.0.0-preview2 in central (https://repo.maven.apache.org/maven2) -> [Help 1] [ERROR] ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test. Closes #26902 from wangyum/SPARK-30056. Authored-by: Yuming Wang Signed-off-by: Yuming Wang --- dev/make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 8ba93aa56f00a..0b30eec76bb53 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -165,7 +165,7 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=1g}" # Store the command as an array because $MVN variable might have spaces in it. # Normal quoting tricks don't work. # See: http://mywiki.wooledge.org/BashFAQ/050 -BUILD_COMMAND=("$MVN" clean package -DskipTests -Dmaven.test.skip=true $@) +BUILD_COMMAND=("$MVN" clean package -DskipTests $@) # Actually build the jar echo -e "\nBuilding with..." From 3bf5498b4a58ebf39662ee717d3538af8b838e2c Mon Sep 17 00:00:00 2001 From: Boris Boutkov Date: Mon, 16 Dec 2019 16:29:09 +0900 Subject: [PATCH 0890/1387] [MINOR][DOCS] Fix documentation for slide function ### What changes were proposed in this pull request? This PR proposes to fix documentation for slide function. Fixed the spacing issue and added some parameter related info. ### Why are the changes needed? Documentation improvement ### Does this PR introduce any user-facing change? No (doc-only change). ### How was this patch tested? Manually tested by documentation build. Closes #26896 from bboutkov/pyspark_doc_fix. Authored-by: Boris Boutkov Signed-off-by: HyukjinKwon --- R/pkg/R/functions.R | 4 ++-- python/pyspark/sql/functions.py | 5 +++++ sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 5 +++++ 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index eec221c2be4bf..177e0353c52a2 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -3620,8 +3620,8 @@ setMethod("size", #' (array indices start at 1, or from the end if start is negative) with the specified length. #' #' @rdname column_collection_functions -#' @param start an index indicating the first element occurring in the result. -#' @param length a number of consecutive elements chosen to the result. +#' @param start the starting index +#' @param length the length of the slice #' @aliases slice slice,Column-method #' @note slice since 2.4.0 setMethod("slice", diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index bd0766508dc06..2cd91ec2b1aef 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1973,6 +1973,11 @@ def slice(x, start, length): """ Collection function: returns an array containing all the elements in `x` from index `start` (array indices start at 1, or from the end if `start` is negative) with the specified `length`. + + :param x: the array to be sliced + :param start: the starting index + :param length: the length of the slice + >>> df = spark.createDataFrame([([1, 2, 3],), ([4, 5],)], ['x']) >>> df.select(slice(df.x, 2, 2).alias("sliced")).collect() [Row(sliced=[2, 3]), Row(sliced=[5])] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 72e9e337c4258..9911972d0f1ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3267,6 +3267,11 @@ object functions { /** * Returns an array containing all the elements in `x` from index `start` (or starting from the * end if `start` is negative) with the specified `length`. + * + * @param x the array column to be sliced + * @param start the starting index + * @param length the length of the slice + * * @group collection_funcs * @since 2.4.0 */ From 72f5597ce28025dfb0b726ba5df259d508adfaf3 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 16 Dec 2019 17:43:01 +0800 Subject: [PATCH 0891/1387] [SPARK-30104][SQL][FOLLOWUP] Remove LookupCatalog.AsTemporaryViewIdentifier ### What changes were proposed in this pull request? As discussed in https://github.com/apache/spark/pull/26741#discussion_r357504518, `LookupCatalog.AsTemporaryViewIdentifier` is no longer used and can be removed. ### Why are the changes needed? Code clean up ### Does this PR introduce any user-facing change? No ### How was this patch tested? Removed tests that were testing solely `AsTemporaryViewIdentifier` extractor. Closes #26897 from imback82/30104-followup. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/connector/catalog/LookupCatalog.scala | 14 ----- .../catalog/LookupCatalogSuite.scala | 56 ------------------- 2 files changed, 70 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 59e7805547c84..eaded966d9c7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -144,18 +144,4 @@ private[sql] trait LookupCatalog extends Logging { None } } - - /** - * For temp views, extract a table identifier from a multi-part identifier if it has no catalog. - */ - object AsTemporaryViewIdentifier { - def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { - case CatalogAndMultipartIdentifier(None, Seq(table)) => - Some(TableIdentifier(table)) - case CatalogAndMultipartIdentifier(None, Seq(database, table)) => - Some(TableIdentifier(table, Some(database))) - case _ => - None - } - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index a576e66236abf..bd467ed878454 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -102,34 +102,6 @@ class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { } } } - - test("temporary table identifier") { - Seq( - ("tbl", TableIdentifier("tbl")), - ("db.tbl", TableIdentifier("tbl", Some("db"))), - ("`db.tbl`", TableIdentifier("db.tbl")), - ("parquet.`file:/tmp/db.tbl`", TableIdentifier("file:/tmp/db.tbl", Some("parquet"))), - ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", - TableIdentifier("s3://buck/tmp/abc.json", Some("org.apache.spark.sql.json")))).foreach { - case (sqlIdent: String, expectedTableIdent: TableIdentifier) => - // when there is no catalog and the namespace has one part, the rule should match - inside(parseMultipartIdentifier(sqlIdent)) { - case AsTemporaryViewIdentifier(ident) => - ident shouldEqual expectedTableIdent - } - } - - Seq("prod.func", "prod.db.tbl", "test.db.tbl", "ns1.ns2.tbl", "test.ns1.ns2.ns3.tbl") - .foreach { sqlIdent => - inside(parseMultipartIdentifier(sqlIdent)) { - case AsTemporaryViewIdentifier(_) => - fail("AsTemporaryViewIdentifier should not match when " + - "the catalog is set or the namespace has multiple parts") - case _ => - // expected - } - } - } } class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog with Inside { @@ -186,32 +158,4 @@ class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog wit } } } - - test("temporary table identifier") { - Seq( - ("tbl", TableIdentifier("tbl")), - ("db.tbl", TableIdentifier("tbl", Some("db"))), - ("`db.tbl`", TableIdentifier("db.tbl")), - ("parquet.`file:/tmp/db.tbl`", TableIdentifier("file:/tmp/db.tbl", Some("parquet"))), - ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", - TableIdentifier("s3://buck/tmp/abc.json", Some("org.apache.spark.sql.json")))).foreach { - case (sqlIdent: String, expectedTableIdent: TableIdentifier) => - // when there is no catalog and the namespace has one part, the rule should match - inside(parseMultipartIdentifier(sqlIdent)) { - case AsTemporaryViewIdentifier(ident) => - ident shouldEqual expectedTableIdent - } - } - - Seq("prod.func", "prod.db.tbl", "test.db.tbl", "ns1.ns2.tbl", "test.ns1.ns2.ns3.tbl") - .foreach { sqlIdent => - inside(parseMultipartIdentifier(sqlIdent)) { - case AsTemporaryViewIdentifier(_) => - fail("AsTemporaryViewIdentifier should not match when " + - "the catalog is set or the namespace has multiple parts") - case _ => - // expected - } - } - } } From fdcd0e71b9bdf64bc73c27cdb7cce69d349cd641 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 16 Dec 2019 18:55:17 +0800 Subject: [PATCH 0892/1387] [SPARK-30192][SQL] support column position in DS v2 ### What changes were proposed in this pull request? update DS v2 API to support add/alter column with column position ### Why are the changes needed? We have a parser rule for column position, but we fail the query if it's specified, because the builtin catalog can't support add/alter column with column position. Since we have the catalog plugin API now, we should let the catalog implementation to decide if it supports column position or not. ### Does this PR introduce any user-facing change? not yet ### How was this patch tested? new tests Closes #26817 from cloud-fan/parser. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/connector/catalog/IdentifierImpl.java | 19 +- .../sql/connector/catalog/TableChange.java | 178 +++++++++++++++++- .../catalyst/analysis/ResolveCatalogs.scala | 23 ++- .../sql/catalyst/parser/AstBuilder.scala | 27 +-- .../catalyst/plans/logical/statements.scala | 10 +- .../catalog/CatalogV2Implicits.scala | 2 +- .../sql/connector/catalog/CatalogV2Util.scala | 60 ++++-- .../sql/catalyst/parser/DDLParserSuite.scala | 70 ++++--- .../analysis/ResolveSessionCatalog.scala | 27 ++- .../sql-tests/results/change-column.sql.out | 20 +- .../spark/sql/connector/AlterTableTests.scala | 98 ++++++++++ .../sql/connector/DataSourceV2SQLSuite.scala | 4 +- .../sql/execution/command/DDLSuite.scala | 10 + 14 files changed, 444 insertions(+), 106 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index be482a641ccc6..1f9d18cbbfd54 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -855,7 +855,7 @@ intervalUnit ; colPosition - : FIRST | AFTER multipartIdentifier + : position=FIRST | position=AFTER afterCol=errorCapturingIdentifier ; dataType diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java index 56d13ef742cea..a56007b2a5ab8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/IdentifierImpl.java @@ -17,14 +17,15 @@ package org.apache.spark.sql.connector.catalog; -import com.google.common.base.Preconditions; -import org.apache.spark.annotation.Experimental; - import java.util.Arrays; import java.util.Objects; import java.util.stream.Collectors; import java.util.stream.Stream; +import com.google.common.base.Preconditions; + +import org.apache.spark.annotation.Experimental; + /** * An {@link Identifier} implementation. */ @@ -51,19 +52,11 @@ public String name() { return name; } - private String escapeQuote(String part) { - if (part.contains("`")) { - return part.replace("`", "``"); - } else { - return part; - } - } - @Override public String toString() { return Stream.concat(Stream.of(namespace), Stream.of(name)) - .map(part -> '`' + escapeQuote(part) + '`') - .collect(Collectors.joining(".")); + .map(CatalogV2Implicits::quote) + .collect(Collectors.joining(".")); } @Override diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java index 20c22388b0ef9..783439935c8d2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java @@ -17,11 +17,12 @@ package org.apache.spark.sql.connector.catalog; -import org.apache.spark.annotation.Experimental; -import org.apache.spark.sql.types.DataType; - import java.util.Arrays; import java.util.Objects; +import javax.annotation.Nullable; + +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.types.DataType; /** * TableChange subclasses represent requested changes to a table. These are passed to @@ -76,7 +77,7 @@ static TableChange removeProperty(String property) { * @return a TableChange for the addition */ static TableChange addColumn(String[] fieldNames, DataType dataType) { - return new AddColumn(fieldNames, dataType, true, null); + return new AddColumn(fieldNames, dataType, true, null, null); } /** @@ -92,7 +93,7 @@ static TableChange addColumn(String[] fieldNames, DataType dataType) { * @return a TableChange for the addition */ static TableChange addColumn(String[] fieldNames, DataType dataType, boolean isNullable) { - return new AddColumn(fieldNames, dataType, isNullable, null); + return new AddColumn(fieldNames, dataType, isNullable, null, null); } /** @@ -113,7 +114,30 @@ static TableChange addColumn( DataType dataType, boolean isNullable, String comment) { - return new AddColumn(fieldNames, dataType, isNullable, comment); + return new AddColumn(fieldNames, dataType, isNullable, comment, null); + } + + /** + * Create a TableChange for adding a column. + *

    + * If the field already exists, the change will result in an {@link IllegalArgumentException}. + * If the new field is nested and its parent does not exist or is not a struct, the change will + * result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the new column + * @param dataType the new column's data type + * @param isNullable whether the new column can contain null + * @param comment the new field's comment string + * @param position the new columns's position + * @return a TableChange for the addition + */ + static TableChange addColumn( + String[] fieldNames, + DataType dataType, + boolean isNullable, + String comment, + ColumnPosition position) { + return new AddColumn(fieldNames, dataType, isNullable, comment, position); } /** @@ -180,6 +204,21 @@ static TableChange updateColumnComment(String[] fieldNames, String newComment) { return new UpdateColumnComment(fieldNames, newComment); } + /** + * Create a TableChange for updating the position of a field. + *

    + * The name is used to find the field to update. + *

    + * If the field does not exist, the change will result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the column to update + * @param newPosition the new position + * @return a TableChange for the update + */ + static TableChange updateColumnPosition(String[] fieldNames, ColumnPosition newPosition) { + return new UpdateColumnPosition(fieldNames, newPosition); + } + /** * Create a TableChange for deleting a field. *

    @@ -259,6 +298,69 @@ public int hashCode() { } } + interface ColumnPosition { + + static ColumnPosition first() { + return First.SINGLETON; + } + + static ColumnPosition after(String column) { + return new After(column); + } + } + + /** + * Column position FIRST means the specified column should be the first column. + * Note that, the specified column may be a nested field, and then FIRST means this field should + * be the first one within the struct. + */ + final class First implements ColumnPosition { + private static final First SINGLETON = new First(); + + private First() {} + + @Override + public String toString() { + return "FIRST"; + } + } + + /** + * Column position AFTER means the specified column should be put after the given `column`. + * Note that, the specified column may be a nested field, and then the given `column` refers to + * a field in the same struct. + */ + final class After implements ColumnPosition { + private final String column; + + private After(String column) { + assert column != null; + this.column = column; + } + + public String column() { + return column; + } + + @Override + public String toString() { + return "AFTER " + column; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + After after = (After) o; + return column.equals(after.column); + } + + @Override + public int hashCode() { + return Objects.hash(column); + } + } + interface ColumnChange extends TableChange { String[] fieldNames(); } @@ -275,12 +377,19 @@ final class AddColumn implements ColumnChange { private final DataType dataType; private final boolean isNullable; private final String comment; - - private AddColumn(String[] fieldNames, DataType dataType, boolean isNullable, String comment) { + private final ColumnPosition position; + + private AddColumn( + String[] fieldNames, + DataType dataType, + boolean isNullable, + String comment, + ColumnPosition position) { this.fieldNames = fieldNames; this.dataType = dataType; this.isNullable = isNullable; this.comment = comment; + this.position = position; } @Override @@ -296,10 +405,16 @@ public boolean isNullable() { return isNullable; } + @Nullable public String comment() { return comment; } + @Nullable + public ColumnPosition position() { + return position; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -308,12 +423,13 @@ public boolean equals(Object o) { return isNullable == addColumn.isNullable && Arrays.equals(fieldNames, addColumn.fieldNames) && dataType.equals(addColumn.dataType) && - comment.equals(addColumn.comment); + Objects.equals(comment, addColumn.comment) && + Objects.equals(position, addColumn.position); } @Override public int hashCode() { - int result = Objects.hash(dataType, isNullable, comment); + int result = Objects.hash(dataType, isNullable, comment, position); result = 31 * result + Arrays.hashCode(fieldNames); return result; } @@ -453,6 +569,48 @@ public int hashCode() { } } + /** + * A TableChange to update the position of a field. + *

    + * The field names are used to find the field to update. + *

    + * If the field does not exist, the change must result in an {@link IllegalArgumentException}. + */ + final class UpdateColumnPosition implements ColumnChange { + private final String[] fieldNames; + private final ColumnPosition position; + + private UpdateColumnPosition(String[] fieldNames, ColumnPosition position) { + this.fieldNames = fieldNames; + this.position = position; + } + + @Override + public String[] fieldNames() { + return fieldNames; + } + + public ColumnPosition position() { + return position; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + UpdateColumnPosition that = (UpdateColumnPosition) o; + return Arrays.equals(fieldNames, that.fieldNames) && + position.equals(that.position); + } + + @Override + public int hashCode() { + int result = Objects.hash(position); + result = 31 * result + Arrays.hashCode(fieldNames); + return result; + } + } + /** * A TableChange to delete a field. *

    diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 8183aa36a5b90..3361173c9962f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -35,19 +35,32 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case AlterTableAddColumnsStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => val changes = cols.map { col => - TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) + TableChange.addColumn( + col.name.toArray, + col.dataType, + true, + col.comment.orNull, + col.position.orNull) } createAlterTable(nameParts, catalog, tbl, changes) case AlterTableAlterColumnStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), colName, dataType, comment) => + nameParts @ NonSessionCatalogAndTable(catalog, tbl), colName, dataType, comment, pos) => + val colNameArray = colName.toArray val typeChange = dataType.map { newDataType => - TableChange.updateColumnType(colName.toArray, newDataType, true) + TableChange.updateColumnType(colNameArray, newDataType, true) } val commentChange = comment.map { newComment => - TableChange.updateColumnComment(colName.toArray, newComment) + TableChange.updateColumnComment(colNameArray, newComment) } - createAlterTable(nameParts, catalog, tbl, typeChange.toSeq ++ commentChange) + val positionChange = pos.map { newPosition => + TableChange.updateColumnPosition(colNameArray, newPosition) + } + createAlterTable( + nameParts, + catalog, + tbl, + typeChange.toSeq ++ commentChange ++ positionChange) case AlterTableRenameColumnStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), col, newName) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 9f8c1d2c0ef95..8f3827391ab0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit import org.apache.spark.sql.connector.catalog.SupportsNamespaces +import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -2803,19 +2804,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) } + override def visitColPosition(ctx: ColPositionContext): ColumnPosition = { + ctx.position.getType match { + case SqlBaseParser.FIRST => ColumnPosition.first() + case SqlBaseParser.AFTER => ColumnPosition.after(ctx.afterCol.getText) + } + } + /** * Parse new column info from ADD COLUMN into a QualifiedColType. */ override def visitQualifiedColTypeWithPosition( ctx: QualifiedColTypeWithPositionContext): QualifiedColType = withOrigin(ctx) { - if (ctx.colPosition != null) { - operationNotAllowed("ALTER TABLE table ADD COLUMN ... FIRST | AFTER otherCol", ctx) - } - QualifiedColType( typedVisit[Seq[String]](ctx.name), typedVisit[DataType](ctx.dataType), - Option(ctx.comment).map(string)) + Option(ctx.comment).map(string), + Option(ctx.colPosition).map(typedVisit[ColumnPosition])) } /** @@ -2863,19 +2868,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitAlterTableColumn( ctx: AlterTableColumnContext): LogicalPlan = withOrigin(ctx) { val verb = if (ctx.CHANGE != null) "CHANGE" else "ALTER" - if (ctx.colPosition != null) { - operationNotAllowed(s"ALTER TABLE table $verb COLUMN ... FIRST | AFTER otherCol", ctx) - } - - if (ctx.dataType == null && ctx.comment == null) { - operationNotAllowed(s"ALTER TABLE table $verb COLUMN requires a TYPE or a COMMENT", ctx) + if (ctx.dataType == null && ctx.comment == null && ctx.colPosition == null) { + operationNotAllowed( + s"ALTER TABLE table $verb COLUMN requires a TYPE or a COMMENT or a FIRST/AFTER", ctx) } AlterTableAlterColumnStatement( visitMultipartIdentifier(ctx.table), typedVisit[Seq[String]](ctx.column), Option(ctx.dataType).map(typedVisit[DataType]), - Option(ctx.comment).map(string)) + Option(ctx.comment).map(string), + Option(ctx.colPosition).map(typedVisit[ColumnPosition])) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 145a15cf3086e..e205dd4e28993 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.ViewType import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.{DataType, StructType} @@ -141,7 +142,11 @@ case class ReplaceTableAsSelectStatement( /** * Column data as parsed by ALTER TABLE ... ADD COLUMNS. */ -case class QualifiedColType(name: Seq[String], dataType: DataType, comment: Option[String]) +case class QualifiedColType( + name: Seq[String], + dataType: DataType, + comment: Option[String], + position: Option[ColumnPosition]) /** * ALTER TABLE ... ADD COLUMNS command, as parsed from SQL. @@ -157,7 +162,8 @@ case class AlterTableAlterColumnStatement( tableName: Seq[String], column: Seq[String], dataType: Option[DataType], - comment: Option[String]) extends ParsedStatement + comment: Option[String], + position: Option[ColumnPosition]) extends ParsedStatement /** * ALTER TABLE ... RENAME COLUMN command, as parsed from SQL. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 882e968f34b59..86e5894b369aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -118,7 +118,7 @@ private[sql] object CatalogV2Implicits { def quoted: String = parts.map(quote).mkString(".") } - private def quote(part: String): String = { + def quote(part: String): String = { if (part.contains(".") || part.contains("`")) { s"`${part.replace("`", "``")}`" } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 0dcd595ded191..2f4914dd7db30 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -104,26 +104,16 @@ private[sql] object CatalogV2Util { case add: AddColumn => add.fieldNames match { case Array(name) => - val newField = StructField(name, add.dataType, nullable = add.isNullable) - Option(add.comment) match { - case Some(comment) => - schema.add(newField.withComment(comment)) - case _ => - schema.add(newField) - } + val field = StructField(name, add.dataType, nullable = add.isNullable) + val newField = Option(add.comment).map(field.withComment).getOrElse(field) + addField(schema, newField, add.position()) case names => replace(schema, names.init, parent => parent.dataType match { case parentType: StructType => val field = StructField(names.last, add.dataType, nullable = add.isNullable) - val newParentType = Option(add.comment) match { - case Some(comment) => - parentType.add(field.withComment(comment)) - case None => - parentType.add(field) - } - - Some(StructField(parent.name, newParentType, parent.nullable, parent.metadata)) + val newField = Option(add.comment).map(field.withComment).getOrElse(field) + Some(parent.copy(dataType = addField(parentType, newField, add.position()))) case _ => throw new IllegalArgumentException(s"Not a struct: ${names.init.last}") @@ -147,6 +137,27 @@ private[sql] object CatalogV2Util { replace(schema, update.fieldNames, field => Some(field.withComment(update.newComment))) + case update: UpdateColumnPosition => + def updateFieldPos(struct: StructType, name: String): StructType = { + val oldField = struct.fields.find(_.name == name).getOrElse { + throw new IllegalArgumentException("Field not found: " + name) + } + val withFieldRemoved = StructType(struct.fields.filter(_ != oldField)) + addField(withFieldRemoved, oldField, update.position()) + } + + update.fieldNames() match { + case Array(name) => + updateFieldPos(schema, name) + case names => + replace(schema, names.init, parent => parent.dataType match { + case parentType: StructType => + Some(parent.copy(dataType = updateFieldPos(parentType, names.last))) + case _ => + throw new IllegalArgumentException(s"Not a struct: ${names.init.last}") + }) + } + case delete: DeleteColumn => replace(schema, delete.fieldNames, _ => None) @@ -157,6 +168,25 @@ private[sql] object CatalogV2Util { } } + private def addField( + schema: StructType, + field: StructField, + position: ColumnPosition): StructType = { + if (position == null) { + schema.add(field) + } else if (position.isInstanceOf[First]) { + StructType(field +: schema.fields) + } else { + val afterCol = position.asInstanceOf[After].column() + val fieldIndex = schema.fields.indexWhere(_.name == afterCol) + if (fieldIndex == -1) { + throw new IllegalArgumentException("AFTER column not found: " + afterCol) + } + val (before, after) = schema.fields.splitAt(fieldIndex + 1) + StructType(before ++ (field +: after)) + } + } + private def replace( struct: StructType, fieldNames: Seq[String], diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index e6d503b3a58f0..ec3a731b38a89 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, Loc import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition.{after, first} import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} @@ -492,7 +493,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int"), AlterTableAddColumnsStatement(Seq("table_name"), Seq( - QualifiedColType(Seq("x"), IntegerType, None) + QualifiedColType(Seq("x"), IntegerType, None, None) ))) } @@ -500,8 +501,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS x int, y string"), AlterTableAddColumnsStatement(Seq("table_name"), Seq( - QualifiedColType(Seq("x"), IntegerType, None), - QualifiedColType(Seq("y"), StringType, None) + QualifiedColType(Seq("x"), IntegerType, None, None), + QualifiedColType(Seq("y"), StringType, None, None) ))) } @@ -509,7 +510,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS x int"), AlterTableAddColumnsStatement(Seq("table_name"), Seq( - QualifiedColType(Seq("x"), IntegerType, None) + QualifiedColType(Seq("x"), IntegerType, None, None) ))) } @@ -517,7 +518,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS (x int)"), AlterTableAddColumnsStatement(Seq("table_name"), Seq( - QualifiedColType(Seq("x"), IntegerType, None) + QualifiedColType(Seq("x"), IntegerType, None, None) ))) } @@ -525,7 +526,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS (x int COMMENT 'doc')"), AlterTableAddColumnsStatement(Seq("table_name"), Seq( - QualifiedColType(Seq("x"), IntegerType, Some("doc")) + QualifiedColType(Seq("x"), IntegerType, Some("doc"), None) ))) } @@ -533,7 +534,21 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int COMMENT 'doc'"), AlterTableAddColumnsStatement(Seq("table_name"), Seq( - QualifiedColType(Seq("x"), IntegerType, Some("doc")) + QualifiedColType(Seq("x"), IntegerType, Some("doc"), None) + ))) + } + + test("alter table: add column with position") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMN x int FIRST"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x"), IntegerType, None, Some(first())) + ))) + + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMN x int AFTER y"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x"), IntegerType, None, Some(after("y"))) ))) } @@ -541,25 +556,19 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc'"), AlterTableAddColumnsStatement(Seq("table_name"), Seq( - QualifiedColType(Seq("x", "y", "z"), IntegerType, Some("doc")) + QualifiedColType(Seq("x", "y", "z"), IntegerType, Some("doc"), None) ))) } test("alter table: add multiple columns with nested column name") { comparePlans( - parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc', a.b string"), + parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc', a.b string FIRST"), AlterTableAddColumnsStatement(Seq("table_name"), Seq( - QualifiedColType(Seq("x", "y", "z"), IntegerType, Some("doc")), - QualifiedColType(Seq("a", "b"), StringType, None) + QualifiedColType(Seq("x", "y", "z"), IntegerType, Some("doc"), None), + QualifiedColType(Seq("a", "b"), StringType, None, Some(first())) ))) } - test("alter table: add column at position (not supported)") { - assertUnsupported("ALTER TABLE table_name ADD COLUMNS name bigint COMMENT 'doc' FIRST, a.b int") - assertUnsupported("ALTER TABLE table_name ADD COLUMN name bigint COMMENT 'doc' FIRST") - assertUnsupported("ALTER TABLE table_name ADD COLUMN name string AFTER a.b") - } - test("alter table: set location") { comparePlans( parsePlan("ALTER TABLE a.b.c SET LOCATION 'new location'"), @@ -589,6 +598,7 @@ class DDLParserSuite extends AnalysisTest { Seq("table_name"), Seq("a", "b", "c"), Some(LongType), + None, None)) } @@ -599,6 +609,7 @@ class DDLParserSuite extends AnalysisTest { Seq("table_name"), Seq("a", "b", "c"), Some(LongType), + None, None)) } @@ -609,22 +620,31 @@ class DDLParserSuite extends AnalysisTest { Seq("table_name"), Seq("a", "b", "c"), None, - Some("new comment"))) + Some("new comment"), + None)) } - test("alter table: update column type and comment") { + test("alter table: update column position") { comparePlans( - parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c TYPE bigint COMMENT 'new comment'"), + parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c FIRST"), AlterTableAlterColumnStatement( Seq("table_name"), Seq("a", "b", "c"), - Some(LongType), - Some("new comment"))) + None, + None, + Some(first()))) } - test("alter table: change column position (not supported)") { - assertUnsupported("ALTER TABLE table_name CHANGE COLUMN name COMMENT 'doc' FIRST") - assertUnsupported("ALTER TABLE table_name CHANGE COLUMN name TYPE INT AFTER other_col") + test("alter table: update column type, comment and position") { + comparePlans( + parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c " + + "TYPE bigint COMMENT 'new comment' AFTER d"), + AlterTableAlterColumnStatement( + Seq("table_name"), + Seq("a", "b", "c"), + Some(LongType), + Some("new comment"), + Some(after("d")))) } test("alter table: drop column") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 1dedf6e777391..834e99c265e9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -55,13 +55,18 @@ class ResolveSessionCatalog( AlterTableAddColumnsCommand(tbl.asTableIdentifier, cols.map(convertToStructField)) }.getOrElse { val changes = cols.map { col => - TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) + TableChange.addColumn( + col.name.toArray, + col.dataType, + true, + col.comment.orNull, + col.position.orNull) } createAlterTable(nameParts, catalog, tbl, changes) } case AlterTableAlterColumnStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), colName, dataType, comment) => + nameParts @ SessionCatalogAndTable(catalog, tbl), colName, dataType, comment, pos) => loadTable(catalog, tbl.asIdentifier).collect { case v1Table: V1Table => if (colName.length > 1) { @@ -72,6 +77,10 @@ class ResolveSessionCatalog( throw new AnalysisException( "ALTER COLUMN with v1 tables must specify new data type.") } + if (pos.isDefined) { + throw new AnalysisException("" + + "ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.") + } val builder = new MetadataBuilder // Add comment to metadata comment.map(c => builder.putString("comment", c)) @@ -87,13 +96,21 @@ class ResolveSessionCatalog( builder.build()) AlterTableChangeColumnCommand(tbl.asTableIdentifier, colName(0), newColumn) }.getOrElse { + val colNameArray = colName.toArray val typeChange = dataType.map { newDataType => - TableChange.updateColumnType(colName.toArray, newDataType, true) + TableChange.updateColumnType(colNameArray, newDataType, true) } val commentChange = comment.map { newComment => - TableChange.updateColumnComment(colName.toArray, newComment) + TableChange.updateColumnComment(colNameArray, newComment) + } + val positionChange = pos.map { newPosition => + TableChange.updateColumnPosition(colNameArray, newPosition) } - createAlterTable(nameParts, catalog, tbl, typeChange.toSeq ++ commentChange) + createAlterTable( + nameParts, + catalog, + tbl, + typeChange.toSeq ++ commentChange ++ positionChange) } case AlterTableRenameColumnStatement( diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 21a344c071bc4..82326346b361c 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -27,7 +27,7 @@ struct<> -- !query 2 output org.apache.spark.sql.catalyst.parser.ParseException -Operation not allowed: ALTER TABLE table CHANGE COLUMN requires a TYPE or a COMMENT(line 1, pos 0) +Operation not allowed: ALTER TABLE table CHANGE COLUMN requires a TYPE or a COMMENT or a FIRST/AFTER(line 1, pos 0) == SQL == ALTER TABLE test_change CHANGE a @@ -87,13 +87,8 @@ ALTER TABLE test_change CHANGE a TYPE INT AFTER b -- !query 8 schema struct<> -- !query 8 output -org.apache.spark.sql.catalyst.parser.ParseException - -Operation not allowed: ALTER TABLE table CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) - -== SQL == -ALTER TABLE test_change CHANGE a TYPE INT AFTER b -^^^ +org.apache.spark.sql.AnalysisException +ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.; -- !query 9 @@ -101,13 +96,8 @@ ALTER TABLE test_change CHANGE b TYPE STRING FIRST -- !query 9 schema struct<> -- !query 9 output -org.apache.spark.sql.catalyst.parser.ParseException - -Operation not allowed: ALTER TABLE table CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) - -== SQL == -ALTER TABLE test_change CHANGE b TYPE STRING FIRST -^^^ +org.apache.spark.sql.AnalysisException +ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.; -- !query 10 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 7392850f276cc..2ba3c99dfbefd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -101,6 +101,49 @@ trait AlterTableTests extends SharedSparkSession { } } + test("AlterTable: add column with position") { + val t = s"${catalogAndNamespace}table_name" + withTable(t) { + sql(s"CREATE TABLE $t (point struct) USING $v2Format") + + sql(s"ALTER TABLE $t ADD COLUMN a string FIRST") + assert(getTableMetadata(t).schema == new StructType() + .add("a", StringType) + .add("point", new StructType().add("x", IntegerType))) + + sql(s"ALTER TABLE $t ADD COLUMN b string AFTER point") + assert(getTableMetadata(t).schema == new StructType() + .add("a", StringType) + .add("point", new StructType().add("x", IntegerType)) + .add("b", StringType)) + + val e1 = intercept[SparkException]( + sql(s"ALTER TABLE $t ADD COLUMN c string AFTER non_exist")) + assert(e1.getMessage().contains("AFTER column not found")) + + sql(s"ALTER TABLE $t ADD COLUMN point.y int FIRST") + assert(getTableMetadata(t).schema == new StructType() + .add("a", StringType) + .add("point", new StructType() + .add("y", IntegerType) + .add("x", IntegerType)) + .add("b", StringType)) + + sql(s"ALTER TABLE $t ADD COLUMN point.z int AFTER x") + assert(getTableMetadata(t).schema == new StructType() + .add("a", StringType) + .add("point", new StructType() + .add("y", IntegerType) + .add("x", IntegerType) + .add("z", IntegerType)) + .add("b", StringType)) + + val e2 = intercept[SparkException]( + sql(s"ALTER TABLE $t ADD COLUMN point.x2 int AFTER non_exist")) + assert(e2.getMessage().contains("AFTER column not found")) + } + } + test("AlterTable: add multiple columns") { val t = s"${catalogAndNamespace}table_name" withTable(t) { @@ -471,6 +514,61 @@ trait AlterTableTests extends SharedSparkSession { } } + test("AlterTable: update column position") { + val t = s"${catalogAndNamespace}table_name" + withTable(t) { + sql(s"CREATE TABLE $t (a int, b int, point struct) USING $v2Format") + + sql(s"ALTER TABLE $t ALTER COLUMN b FIRST") + assert(getTableMetadata(t).schema == new StructType() + .add("b", IntegerType) + .add("a", IntegerType) + .add("point", new StructType() + .add("x", IntegerType) + .add("y", IntegerType) + .add("z", IntegerType))) + + sql(s"ALTER TABLE $t ALTER COLUMN b AFTER point") + assert(getTableMetadata(t).schema == new StructType() + .add("a", IntegerType) + .add("point", new StructType() + .add("x", IntegerType) + .add("y", IntegerType) + .add("z", IntegerType)) + .add("b", IntegerType)) + + val e1 = intercept[SparkException]( + sql(s"ALTER TABLE $t ALTER COLUMN b AFTER non_exist")) + assert(e1.getMessage.contains("AFTER column not found")) + + sql(s"ALTER TABLE $t ALTER COLUMN point.y FIRST") + assert(getTableMetadata(t).schema == new StructType() + .add("a", IntegerType) + .add("point", new StructType() + .add("y", IntegerType) + .add("x", IntegerType) + .add("z", IntegerType)) + .add("b", IntegerType)) + + sql(s"ALTER TABLE $t ALTER COLUMN point.y AFTER z") + assert(getTableMetadata(t).schema == new StructType() + .add("a", IntegerType) + .add("point", new StructType() + .add("x", IntegerType) + .add("z", IntegerType) + .add("y", IntegerType)) + .add("b", IntegerType)) + + val e2 = intercept[SparkException]( + sql(s"ALTER TABLE $t ALTER COLUMN point.y AFTER non_exist")) + assert(e2.getMessage.contains("AFTER column not found")) + + // `AlterTable.resolved` checks column existence. + intercept[AnalysisException]( + sql(s"ALTER TABLE $t ALTER COLUMN a.y AFTER x")) + } + } + test("AlterTable: update column type and comment") { val t = s"${catalogAndNamespace}table_name" withTable(t) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index cfcd8c7a73d5c..15381e05d7250 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1199,7 +1199,7 @@ class DataSourceV2SQLSuite } test("tableCreation: duplicate column names in the table definition") { - val errorMsg = "Found duplicate column(s) in the table definition of `t`" + val errorMsg = "Found duplicate column(s) in the table definition of t" Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisError( @@ -1223,7 +1223,7 @@ class DataSourceV2SQLSuite } test("tableCreation: duplicate nested column names in the table definition") { - val errorMsg = "Found duplicate column(s) in the table definition of `t`" + val errorMsg = "Found duplicate column(s) in the table definition of t" Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { assertAnalysisError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 10873678e05f2..2bb121b27e7d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -181,6 +181,16 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { assert(e.contains("Hive built-in ORC data source must be used with Hive support enabled")) } } + + test("ALTER TABLE ALTER COLUMN with position is not supported") { + withTable("t") { + sql("CREATE TABLE t(i INT) USING parquet") + val e = intercept[AnalysisException] { + sql("ALTER TABLE t ALTER COLUMN i TYPE INT FIRST") + } + assert(e.message.contains("ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables")) + } + } } abstract class DDLSuite extends QueryTest with SQLTestUtils { From ba0f59bfafa78159b10508b3e584060b2cbdf37c Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 16 Dec 2019 04:54:12 -0700 Subject: [PATCH 0893/1387] [SPARK-30265][INFRA] Do not change R version when releasing preview versions ### What changes were proposed in this pull request? This PR makes it do not change R version when releasing preview versions. ### Why are the changes needed? Failed to make Spark binary distribution: ``` ++ . /opt/spark-rm/output/spark-3.0.0-preview2-bin-hadoop2.7/R/find-r.sh +++ '[' -z /usr/bin ']' ++ /usr/bin/Rscript -e ' if("devtools" %in% rownames(installed.packages())) { library(devtools); devtools::document(pkg="./pkg", roclets=c("rd")) }' Loading required package: usethis Updating SparkR documentation First time using roxygen2. Upgrading automatically... Loading SparkR Invalid DESCRIPTION: Malformed package version. See section 'The DESCRIPTION file' in the 'Writing R Extensions' manual. Error: invalid version specification '3.0.0-preview2' In addition: Warning message: roxygen2 requires Encoding: UTF-8 Execution halted [ERROR] Command execution failed. org.apache.commons.exec.ExecuteException: Process exited with an error: 1 (Exit value: 1) at org.apache.commons.exec.DefaultExecutor.executeInternal (DefaultExecutor.java:404) at org.apache.commons.exec.DefaultExecutor.execute (DefaultExecutor.java:166) at org.codehaus.mojo.exec.ExecMojo.executeCommandLine (ExecMojo.java:804) at org.codehaus.mojo.exec.ExecMojo.executeCommandLine (ExecMojo.java:751) at org.codehaus.mojo.exec.ExecMojo.execute (ExecMojo.java:313) at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo (DefaultBuildPluginManager.java:137) at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:210) at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:156) at org.apache.maven.lifecycle.internal.MojoExecutor.execute (MojoExecutor.java:148) at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject (LifecycleModuleBuilder.java:117) at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject (LifecycleModuleBuilder.java:81) at org.apache.maven.lifecycle.internal.builder.singlethreaded.SingleThreadedBuilder.build (SingleThreadedBuilder.java:56) at org.apache.maven.lifecycle.internal.LifecycleStarter.execute (LifecycleStarter.java:128) at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:305) at org.apache.maven.DefaultMaven.doExecute (DefaultMaven.java:192) at org.apache.maven.DefaultMaven.execute (DefaultMaven.java:105) at org.apache.maven.cli.MavenCli.execute (MavenCli.java:957) at org.apache.maven.cli.MavenCli.doMain (MavenCli.java:289) at org.apache.maven.cli.MavenCli.main (MavenCli.java:193) at sun.reflect.NativeMethodAccessorImpl.invoke0 (Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke (NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke (DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke (Method.java:498) at org.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced (Launcher.java:282) at org.codehaus.plexus.classworlds.launcher.Launcher.launch (Launcher.java:225) at org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode (Launcher.java:406) at org.codehaus.plexus.classworlds.launcher.Launcher.main (Launcher.java:347) [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 3.0.0-preview2: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 18.619 s] [INFO] Spark Project Tags ................................. SUCCESS [ 13.652 s] [INFO] Spark Project Sketch ............................... SUCCESS [ 5.673 s] [INFO] Spark Project Local DB ............................. SUCCESS [ 2.081 s] [INFO] Spark Project Networking ........................... SUCCESS [ 3.509 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 0.993 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 7.556 s] [INFO] Spark Project Launcher ............................. SUCCESS [ 5.522 s] [INFO] Spark Project Core ................................. FAILURE [01:06 min] [INFO] Spark Project ML Local Library ..................... SKIPPED [INFO] Spark Project GraphX ............................... SKIPPED [INFO] Spark Project Streaming ............................ SKIPPED [INFO] Spark Project Catalyst ............................. SKIPPED [INFO] Spark Project SQL .................................. SKIPPED [INFO] Spark Project ML Library ........................... SKIPPED [INFO] Spark Project Tools ................................ SKIPPED [INFO] Spark Project Hive ................................. SKIPPED [INFO] Spark Project Graph API ............................ SKIPPED [INFO] Spark Project Cypher ............................... SKIPPED [INFO] Spark Project Graph ................................ SKIPPED [INFO] Spark Project REPL ................................. SKIPPED [INFO] Spark Project Assembly ............................. SKIPPED [INFO] Kafka 0.10+ Token Provider for Streaming ........... SKIPPED [INFO] Spark Integration for Kafka 0.10 ................... SKIPPED [INFO] Kafka 0.10+ Source for Structured Streaming ........ SKIPPED [INFO] Spark Project Examples ............................. SKIPPED [INFO] Spark Integration for Kafka 0.10 Assembly .......... SKIPPED [INFO] Spark Avro ......................................... SKIPPED [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ [INFO] Total time: 02:04 min [INFO] Finished at: 2019-12-16T08:02:45Z [INFO] ------------------------------------------------------------------------ [ERROR] Failed to execute goal org.codehaus.mojo:exec-maven-plugin:1.6.0:exec (sparkr-pkg) on project spark-core_2.12: Command execution failed.: Process exited with an error: 1 (Exit value: 1) -> [Help 1] [ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch. [ERROR] Re-run Maven using the -X switch to enable full debug logging. [ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles: [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException [ERROR] [ERROR] After correcting the problems, you can resume the build with the command [ERROR] mvn -rf :spark-core_2.12 ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test: ```diff diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index cdb59093781..b648c51e010 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R -336,8 +336,8 sparkR.session <- function( # Check if version number of SparkSession matches version number of SparkR package jvmVersion <- callJMethod(sparkSession, "version") - # Remove -SNAPSHOT from jvm versions - jvmVersionStrip <- gsub("-SNAPSHOT", "", jvmVersion) + # Remove -preview2 from jvm versions + jvmVersionStrip <- gsub("-preview2", "", jvmVersion) rPackageVersion <- paste0(packageVersion("SparkR")) if (jvmVersionStrip != rPackageVersion) { ``` Closes #26904 from wangyum/SPARK-30265. Authored-by: Yuming Wang Signed-off-by: Yuming Wang --- dev/create-release/release-tag.sh | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index f8b730b06136b..39856a9955955 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -73,8 +73,12 @@ git config user.email $GIT_EMAIL # Create release version $MVN versions:set -DnewVersion=$RELEASE_VERSION | grep -v "no value" # silence logs -# Set the release version in R/pkg/DESCRIPTION -sed -i".tmp1" 's/Version.*$/Version: '"$RELEASE_VERSION"'/g' R/pkg/DESCRIPTION +if [[ $RELEASE_VERSION != *"preview"* ]]; then + # Set the release version in R/pkg/DESCRIPTION + sed -i".tmp1" 's/Version.*$/Version: '"$RELEASE_VERSION"'/g' R/pkg/DESCRIPTION +else + sed -i".tmp1" 's/-SNAPSHOT/'"-$(cut -d "-" -f 2 <<< $RELEASE_VERSION)"'/g' R/pkg/R/sparkR.R +fi # Set the release version in docs sed -i".tmp1" 's/SPARK_VERSION:.*$/SPARK_VERSION: '"$RELEASE_VERSION"'/g' docs/_config.yml sed -i".tmp2" 's/SPARK_VERSION_SHORT:.*$/SPARK_VERSION_SHORT: '"$RELEASE_VERSION"'/g' docs/_config.yml From 23b1312324a173b75a3dd1564eda6eb32776b857 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 16 Dec 2019 21:35:37 +0900 Subject: [PATCH 0894/1387] [SPARK-30200][DOCS][FOLLOW-UP] Add documentation for explain(mode: String) ### What changes were proposed in this pull request? This PR adds the documentation of the new `mode` added to `Dataset.explain`. ### Why are the changes needed? To let users know the new modes. ### Does this PR introduce any user-facing change? No (doc-only change). ### How was this patch tested? Manually built the doc: ![Screen Shot 2019-12-16 at 3 34 28 PM](https://user-images.githubusercontent.com/6477701/70884617-d64f1680-2019-11ea-9336-247ade7f8768.png) Closes #26903 from HyukjinKwon/SPARK-30200-doc. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../main/scala/org/apache/spark/sql/Dataset.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) 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 09f7578596bd7..77b3a2c40baaa 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 @@ -524,6 +524,16 @@ class Dataset[T] private[sql]( /** * Prints the plans (logical and physical) with a format specified by a given explain mode. * + * @param mode specifies the expected output format of plans. + *

      + *
    • `simple` Print only a physical plan.
    • + *
    • `extended`: Print both logical and physical plans.
    • + *
    • `codegen`: Print a physical plan and generated codes if they are + * available.
    • + *
    • `cost`: Print a logical plan and statistics if they are available.
    • + *
    • `formatted`: Split explain output into two sections: a physical plan outline + * and node details.
    • + *
    * @group basic * @since 3.0.0 */ @@ -541,6 +551,8 @@ class Dataset[T] private[sql]( /** * Prints the plans (logical and physical) to the console for debugging purposes. * + * @param extended default `false`. If `false`, prints only the physical plan. + * * @group basic * @since 1.6.0 */ From b573f23ed18a19e5ca2b51e3a452d2d5f716729d Mon Sep 17 00:00:00 2001 From: Shahin Shakeri Date: Mon, 16 Dec 2019 10:11:50 -0800 Subject: [PATCH 0895/1387] [SPARK-29574][K8S] Add SPARK_DIST_CLASSPATH to the executor class path ### What changes were proposed in this pull request? Include `$SPARK_DIST_CLASSPATH` in class path when launching `CoarseGrainedExecutorBackend` on Kubernetes executors using the provided `entrypoint.sh` ### Why are the changes needed? For user provided Hadoop, `$SPARK_DIST_CLASSPATH` contains the required jars. ### Does this PR introduce any user-facing change? no ### How was this patch tested? Kubernetes 1.14, Spark 2.4.4, Hadoop 3.2.1. Adding $SPARK_DIST_CLASSPATH to `-cp ` param of entrypoint.sh enables launching the executors correctly. Closes #26493 from sshakeri/master. Authored-by: Shahin Shakeri Signed-off-by: Marcelo Vanzin --- docs/hadoop-provided.md | 22 +++++++++++++++++++ .../src/main/dockerfiles/spark/entrypoint.sh | 8 ++++++- 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/docs/hadoop-provided.md b/docs/hadoop-provided.md index 37cdaa6150d36..6442e947abcd0 100644 --- a/docs/hadoop-provided.md +++ b/docs/hadoop-provided.md @@ -39,3 +39,25 @@ export SPARK_DIST_CLASSPATH=$(/path/to/hadoop/bin/hadoop classpath) export SPARK_DIST_CLASSPATH=$(hadoop --config /path/to/configs classpath) {% endhighlight %} + +# Hadoop Free Build Setup for Spark on Kubernetes +To run the Hadoop free build of Spark on Kubernetes, the executor image must have the appropriate version of Hadoop binaries and the correct `SPARK_DIST_CLASSPATH` value set. See the example below for the relevant changes needed in the executor Dockerfile: + +{% highlight bash %} +### Set environment variables in the executor dockerfile ### + +ENV SPARK_HOME="/opt/spark" +ENV HADOOP_HOME="/opt/hadoop" +ENV PATH="$SPARK_HOME/bin:$HADOOP_HOME/bin:$PATH" +... + +#Copy your target hadoop binaries to the executor hadoop home + +COPY /opt/hadoop3 $HADOOP_HOME +... + +#Copy and use the Spark provided entrypoint.sh. It sets your SPARK_DIST_CLASSPATH using the hadoop binary in $HADOOP_HOME and starts the executor. If you choose to customize the value of SPARK_DIST_CLASSPATH here, the value will be retained in entrypoint.sh + +ENTRYPOINT [ "/opt/entrypoint.sh" ] +... +{% endhighlight %} diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 4fe8df61ef569..6ee3523c8edab 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -56,6 +56,12 @@ elif [ "$PYSPARK_MAJOR_PYTHON_VERSION" == "3" ]; then export PYSPARK_DRIVER_PYTHON="python3" fi +# If HADOOP_HOME is set and SPARK_DIST_CLASSPATH is not set, set it here so Hadoop jars are available to the executor. +# It does not set SPARK_DIST_CLASSPATH if already set, to avoid overriding customizations of this value from elsewhere e.g. Docker/K8s. +if [ -n ${HADOOP_HOME} ] && [ -z ${SPARK_DIST_CLASSPATH} ]; then + export SPARK_DIST_CLASSPATH=$($HADOOP_HOME/bin/hadoop classpath) +fi + if ! [ -z ${HADOOP_CONF_DIR+x} ]; then SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; fi @@ -77,7 +83,7 @@ case "$1" in "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY - -cp "$SPARK_CLASSPATH" + -cp "$SPARK_CLASSPATH:$SPARK_DIST_CLASSPATH" org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID From dddfeca175bdce5294debe00d4a993daef92ca60 Mon Sep 17 00:00:00 2001 From: Niranjan Artal Date: Mon, 16 Dec 2019 15:27:34 -0600 Subject: [PATCH 0896/1387] [SPARK-30209][SQL][WEB-UI] Display stageId, attemptId and taskId for max metrics in Spark UI ### What changes were proposed in this pull request? SPARK-30209 discusses about adding additional metrics such as stageId, attempId and taskId for max metrics. We have the data required to display in LiveStageMetrics. Need to capture and pass these metrics to display on the UI. To minimize memory used for variables, we are saving maximum of each metric id per stage. So per stage additional memory usage is (#metrics * 4 * sizeof(Long)). Then max is calculated for each metric id among all stages which is passed in the stringValue method. Memory used is minimal. Ran the benchmark for runtime. Stage.Proc time has increased to around 1.5-2.5x but the Aggregate time has decreased. ### Why are the changes needed? These additional metrics stageId, attemptId and taskId could help in debugging the jobs quicker. For a given operator, it will be easy to identify the task which is taking maximum time to complete from the SQL tab itself. ### Does this PR introduce any user-facing change? Yes. stageId, attemptId and taskId is shown only for executor side metrics. For driver metrics, "(driver)" is displayed on UI. ![image (3)](https://user-images.githubusercontent.com/50492963/70763041-929d9980-1d07-11ea-940f-88ac6bdce9b5.png) "Driver" ![image (4)](https://user-images.githubusercontent.com/50492963/70763043-94675d00-1d07-11ea-95ab-3478728cb435.png) ### How was this patch tested? Manually tested, ran benchmark script for runtime. Closes #26843 from nartal1/SPARK-30209. Authored-by: Niranjan Artal Signed-off-by: Thomas Graves --- .../sql/execution/metric/SQLMetrics.scala | 52 ++++++++++----- .../execution/ui/SQLAppStatusListener.scala | 63 +++++++++++++++---- .../execution/metric/SQLMetricsSuite.scala | 39 ++++++++---- .../metric/SQLMetricsTestUtils.scala | 26 ++++++-- .../ui/SQLAppStatusListenerSuite.scala | 5 +- 5 files changed, 137 insertions(+), 48 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index b7f0ab2969e45..45b1c86130db5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -111,7 +111,8 @@ object SQLMetrics { // data size total (min, med, max): // 100GB (100MB, 1GB, 10GB) val acc = new SQLMetric(SIZE_METRIC, -1) - acc.register(sc, name = Some(s"$name total (min, med, max)"), countFailedValues = false) + acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), + countFailedValues = false) acc } @@ -120,14 +121,16 @@ object SQLMetrics { // duration(min, med, max): // 5s (800ms, 1s, 2s) val acc = new SQLMetric(TIMING_METRIC, -1) - acc.register(sc, name = Some(s"$name total (min, med, max)"), countFailedValues = false) + acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), + countFailedValues = false) acc } def createNanoTimingMetric(sc: SparkContext, name: String): SQLMetric = { // Same with createTimingMetric, just normalize the unit of time to millisecond. val acc = new SQLMetric(NS_TIMING_METRIC, -1) - acc.register(sc, name = Some(s"$name total (min, med, max)"), countFailedValues = false) + acc.register(sc, name = Some(s"$name total (min, med, max (stageId (attemptId): taskId))"), + countFailedValues = false) acc } @@ -142,31 +145,46 @@ object SQLMetrics { // probe avg (min, med, max): // (1.2, 2.2, 6.3) val acc = new SQLMetric(AVERAGE_METRIC) - acc.register(sc, name = Some(s"$name (min, med, max)"), countFailedValues = false) + acc.register(sc, name = Some(s"$name (min, med, max (stageId (attemptId): taskId))"), + countFailedValues = false) acc } + private def toNumberFormat(value: Long): String = { + val numberFormat = NumberFormat.getNumberInstance(Locale.US) + numberFormat.format(value.toDouble / baseForAvgMetric) + } + + def metricNeedsMax(metricsType: String): Boolean = { + metricsType != SUM_METRIC + } + /** * A function that defines how we aggregate the final accumulator results among all tasks, * and represent it in string for a SQL physical operator. - */ - def stringValue(metricsType: String, values: Array[Long]): String = { + */ + def stringValue(metricsType: String, values: Array[Long], maxMetrics: Array[Long]): String = { + // stringMetric = "(driver)" OR (stage $stageId (attempt $attemptId): task $taskId)) + val stringMetric = if (maxMetrics.isEmpty) { + "(driver)" + } else { + s"(stage ${maxMetrics(1)} (attempt ${maxMetrics(2)}): task ${maxMetrics(3)})" + } if (metricsType == SUM_METRIC) { val numberFormat = NumberFormat.getIntegerInstance(Locale.US) numberFormat.format(values.sum) } else if (metricsType == AVERAGE_METRIC) { - val numberFormat = NumberFormat.getNumberInstance(Locale.US) - val validValues = values.filter(_ > 0) val Seq(min, med, max) = { val metric = if (validValues.isEmpty) { - Seq.fill(3)(0L) + val zeros = Seq.fill(3)(0L) + zeros.map(v => toNumberFormat(v)) } else { Arrays.sort(validValues) - Seq(validValues(0), validValues(validValues.length / 2), - validValues(validValues.length - 1)) + Seq(toNumberFormat(validValues(0)), toNumberFormat(validValues(validValues.length / 2)), + s"${toNumberFormat(validValues(validValues.length - 1))} $stringMetric") } - metric.map(v => numberFormat.format(v.toDouble / baseForAvgMetric)) + metric } s"\n($min, $med, $max)" } else { @@ -183,13 +201,15 @@ object SQLMetrics { val validValues = values.filter(_ >= 0) val Seq(sum, min, med, max) = { val metric = if (validValues.isEmpty) { - Seq.fill(4)(0L) + val zeros = Seq.fill(4)(0L) + zeros.map(v => strFormat(v)) } else { Arrays.sort(validValues) - Seq(validValues.sum, validValues(0), validValues(validValues.length / 2), - validValues(validValues.length - 1)) + Seq(strFormat(validValues.sum), strFormat(validValues(0)), + strFormat(validValues(validValues.length / 2)), + s"${strFormat(validValues(validValues.length - 1))} $stringMetric") } - metric.map(strFormat) + metric } s"\n$sum ($min, $med, $max)" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index e1100c3e3a579..64d2f33be5df0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -103,12 +103,13 @@ class SQLAppStatusListener( } }.getOrElse(getOrCreateExecution(executionId)) - // Record the accumulator IDs for the stages of this job, so that the code that keeps - // track of the metrics knows which accumulators to look at. - val accumIds = exec.metrics.map(_.accumulatorId).toSet - if (accumIds.nonEmpty) { + // Record the accumulator IDs and metric types for the stages of this job, so that the code + // that keeps track of the metrics knows which accumulators to look at. + val accumIdsAndType = exec.metrics.map { m => (m.accumulatorId, m.metricType) }.toMap + if (accumIdsAndType.nonEmpty) { event.stageInfos.foreach { stage => - stageMetrics.put(stage.stageId, new LiveStageMetrics(0, stage.numTasks, accumIds)) + stageMetrics.put(stage.stageId, new LiveStageMetrics(stage.stageId, 0, + stage.numTasks, accumIdsAndType)) } } @@ -126,7 +127,8 @@ class SQLAppStatusListener( Option(stageMetrics.get(event.stageInfo.stageId)).foreach { stage => if (stage.attemptId != event.stageInfo.attemptNumber) { stageMetrics.put(event.stageInfo.stageId, - new LiveStageMetrics(event.stageInfo.attemptNumber, stage.numTasks, stage.accumulatorIds)) + new LiveStageMetrics(event.stageInfo.stageId, event.stageInfo.attemptNumber, + stage.numTasks, stage.accumIdsToMetricType)) } } } @@ -198,12 +200,17 @@ class SQLAppStatusListener( private def aggregateMetrics(exec: LiveExecutionData): Map[Long, String] = { val metricTypes = exec.metrics.map { m => (m.accumulatorId, m.metricType) }.toMap - val taskMetrics = exec.stages.toSeq + val liveStageMetrics = exec.stages.toSeq .flatMap { stageId => Option(stageMetrics.get(stageId)) } - .flatMap(_.metricValues()) + + val taskMetrics = liveStageMetrics.flatMap(_.metricValues()) + + val maxMetrics = liveStageMetrics.flatMap(_.maxMetricValues()) val allMetrics = new mutable.HashMap[Long, Array[Long]]() + val maxMetricsFromAllStages = new mutable.HashMap[Long, Array[Long]]() + taskMetrics.foreach { case (id, values) => val prev = allMetrics.getOrElse(id, null) val updated = if (prev != null) { @@ -214,6 +221,18 @@ class SQLAppStatusListener( allMetrics(id) = updated } + // Find the max for each metric id between all stages. + maxMetrics.foreach { case (id, value, taskId, stageId, attemptId) => + val updated = maxMetricsFromAllStages.getOrElse(id, Array(value, stageId, attemptId, taskId)) + if (value > updated(0)) { + updated(0) = value + updated(1) = stageId + updated(2) = attemptId + updated(3) = taskId + } + maxMetricsFromAllStages(id) = updated + } + exec.driverAccumUpdates.foreach { case (id, value) => if (metricTypes.contains(id)) { val prev = allMetrics.getOrElse(id, null) @@ -229,7 +248,8 @@ class SQLAppStatusListener( } val aggregatedMetrics = allMetrics.map { case (id, values) => - id -> SQLMetrics.stringValue(metricTypes(id), values) + id -> SQLMetrics.stringValue(metricTypes(id), values, maxMetricsFromAllStages.getOrElse(id, + Array.empty[Long])) }.toMap // Check the execution again for whether the aggregated metrics data has been calculated. @@ -440,9 +460,10 @@ private class LiveExecutionData(val executionId: Long) extends LiveEntity { } private class LiveStageMetrics( + val stageId: Int, val attemptId: Int, val numTasks: Int, - val accumulatorIds: Set[Long]) { + val accumIdsToMetricType: Map[Long, String]) { /** * Mapping of task IDs to their respective index. Note this may contain more elements than the @@ -461,6 +482,8 @@ private class LiveStageMetrics( */ private val taskMetrics = new ConcurrentHashMap[Long, Array[Long]]() + private val metricsIdToMaxTaskValue = new ConcurrentHashMap[Long, Array[Long]]() + def registerTask(taskId: Long, taskIdx: Int): Unit = { taskIndices.update(taskId, taskIdx) } @@ -487,7 +510,7 @@ private class LiveStageMetrics( } accumUpdates - .filter { acc => acc.update.isDefined && accumulatorIds.contains(acc.id) } + .filter { acc => acc.update.isDefined && accumIdsToMetricType.contains(acc.id) } .foreach { acc => // In a live application, accumulators have Long values, but when reading from event // logs, they have String values. For now, assume all accumulators are Long and convert @@ -500,14 +523,30 @@ private class LiveStageMetrics( val metricValues = taskMetrics.computeIfAbsent(acc.id, _ => new Array(numTasks)) metricValues(taskIdx) = value - } + if (SQLMetrics.metricNeedsMax(accumIdsToMetricType(acc.id))) { + val maxMetricsTaskId = metricsIdToMaxTaskValue.computeIfAbsent(acc.id, _ => Array(value, + taskId)) + + if (value > maxMetricsTaskId.head) { + maxMetricsTaskId(0) = value + maxMetricsTaskId(1) = taskId + } + } + } if (finished) { completedIndices += taskIdx } } def metricValues(): Seq[(Long, Array[Long])] = taskMetrics.asScala.toSeq + + // Return Seq of metric id, value, taskId, stageId, attemptId for this stage + def maxMetricValues(): Seq[(Long, Long, Long, Int, Int)] = { + metricsIdToMaxTaskValue.asScala.toSeq.map { case (id, maxMetrics) => (id, maxMetrics(0), + maxMetrics(1), stageId, attemptId) + } + } } private object SQLAppStatusListener { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index fbf97e9a3240c..fcb089eba80a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -84,8 +84,9 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { val ds = spark.range(10).filter('id < 5) testSparkPlanMetricsWithPredicates(ds.toDF(), 1, Map( 0L -> (("WholeStageCodegen (1)", Map( - "duration total (min, med, max)" -> {_.toString.matches(timingMetricPattern)}))) - ), true) + "duration total (min, med, max (stageId (attemptId): taskId))" -> { + _.toString.matches(timingMetricPattern) + })))), true) } test("Aggregate metrics") { @@ -95,9 +96,11 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { val df = testData2.groupBy().count() // 2 partitions val expected1 = Seq( Map("number of output rows" -> 2L, - "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)"), + "avg hash probe bucket list iters (min, med, max (stageId (attemptId): taskId))" -> + aggregateMetricsPattern), Map("number of output rows" -> 1L, - "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)")) + "avg hash probe bucket list iters (min, med, max (stageId (attemptId): taskId))" -> + aggregateMetricsPattern)) val shuffleExpected1 = Map( "records read" -> 2L, "local blocks read" -> 2L, @@ -113,9 +116,12 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { val df2 = testData2.groupBy('a).count() val expected2 = Seq( Map("number of output rows" -> 4L, - "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)"), + "avg hash probe bucket list iters (min, med, max (stageId (attemptId): taskId))" -> + aggregateMetricsPattern), Map("number of output rows" -> 3L, - "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)")) + "avg hash probe bucket list iters (min, med, max (stageId (attemptId): taskId))" -> + aggregateMetricsPattern)) + val shuffleExpected2 = Map( "records read" -> 4L, "local blocks read" -> 4L, @@ -161,9 +167,12 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { } val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe bucket list iters (min, med, max)") - probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => - assert(probe.toDouble > 1.0) + val probes = metrics(nodeId)._2("avg hash probe bucket list iters (min, med, max (stageId" + + " (attemptId): taskId))") + // Extract min, med, max from the string and strip off everthing else. + val index = probes.toString.stripPrefix("\n(").stripSuffix(")").indexOf(" (", 0) + probes.toString.stripPrefix("\n(").stripSuffix(")").slice(0, index).split(", ").foreach { + probe => assert(probe.toDouble > 1.0) } } } @@ -208,9 +217,15 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { val df = Seq(1, 3, 2).toDF("id").sort('id) testSparkPlanMetricsWithPredicates(df, 2, Map( 0L -> (("Sort", Map( - "sort time total (min, med, max)" -> {_.toString.matches(timingMetricPattern)}, - "peak memory total (min, med, max)" -> {_.toString.matches(sizeMetricPattern)}, - "spill size total (min, med, max)" -> {_.toString.matches(sizeMetricPattern)}))) + "sort time total (min, med, max (stageId (attemptId): taskId))" -> { + _.toString.matches(timingMetricPattern) + }, + "peak memory total (min, med, max (stageId (attemptId): taskId))" -> { + _.toString.matches(sizeMetricPattern) + }, + "spill size total (min, med, max (stageId (attemptId): taskId))" -> { + _.toString.matches(sizeMetricPattern) + }))) )) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala index 57731e5f4920c..0c1148f7b82e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -41,16 +41,28 @@ trait SQLMetricsTestUtils extends SQLTestUtils { protected def statusStore: SQLAppStatusStore = spark.sharedState.statusStore - // Pattern of size SQLMetric value, e.g. "\n96.2 MiB (32.1 MiB, 32.1 MiB, 32.1 MiB)" + // Pattern of size SQLMetric value, e.g. "\n96.2 MiB (32.1 MiB, 32.1 MiB, 32.1 MiB (stage 0 + // (attempt 0): task 4))" OR "\n96.2 MiB (32.1 MiB, 32.1 MiB, 32.1 MiB)" protected val sizeMetricPattern = { val bytes = "([0-9]+(\\.[0-9]+)?) (EiB|PiB|TiB|GiB|MiB|KiB|B)" - s"\\n$bytes \\($bytes, $bytes, $bytes\\)" + val maxMetrics = "\\(stage ([0-9])+ \\(attempt ([0-9])+\\)\\: task ([0-9])+\\)" + s"\\n$bytes \\($bytes, $bytes, $bytes( $maxMetrics)?\\)" } - // Pattern of timing SQLMetric value, e.g. "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms)" + // Pattern of timing SQLMetric value, e.g. "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms (stage 3 (attempt + // 0): task 217))" OR "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms)" protected val timingMetricPattern = { val duration = "([0-9]+(\\.[0-9]+)?) (ms|s|m|h)" - s"\\n$duration \\($duration, $duration, $duration\\)" + val maxMetrics = "\\(stage ([0-9])+ \\(attempt ([0-9])+\\)\\: task ([0-9])+\\)" + s"\\n$duration \\($duration, $duration, $duration( $maxMetrics)?\\)" + } + + // Pattern of size SQLMetric value for Aggregate tests. + // e.g "\n(1, 1, 0.9 (stage 1 (attempt 0): task 8)) OR "\n(1, 1, 0.9 )" + protected val aggregateMetricsPattern = { + val iters = "([0-9]+(\\.[0-9]+)?)" + val maxMetrics = "\\(stage ([0-9])+ \\(attempt ([0-9])+\\)\\: task ([0-9])+\\)" + s"\\n\\($iters, $iters, $iters( $maxMetrics)?\\)" } /** @@ -86,7 +98,7 @@ trait SQLMetricsTestUtils extends SQLTestUtils { } val totalNumBytesMetric = executedNode.metrics.find( - _.name == "written output total (min, med, max)").get + _.name == "written output total (min, med, max (stageId (attemptId): taskId))").get val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "") .split(" ").head.trim.toDouble assert(totalNumBytes > 0) @@ -205,7 +217,9 @@ trait SQLMetricsTestUtils extends SQLTestUtils { expectedMetrics: Map[Long, (String, Map[String, Any])]): Unit = { val expectedMetricsPredicates = expectedMetrics.mapValues { case (nodeName, nodeMetrics) => (nodeName, nodeMetrics.mapValues(expectedMetricValue => - (actualMetricValue: Any) => expectedMetricValue.toString === actualMetricValue)) + (actualMetricValue: Any) => { + actualMetricValue.toString.matches(expectedMetricValue.toString) + })) } testSparkPlanMetricsWithPredicates(df, expectedNumOfJobs, expectedMetricsPredicates) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index a8b77b8202123..4113c2c5d296d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -524,9 +524,10 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils val metrics = statusStore.executionMetrics(execId) val driverMetric = physicalPlan.metrics("dummy") val driverMetric2 = physicalPlan.metrics("dummy2") - val expectedValue = SQLMetrics.stringValue(driverMetric.metricType, Array(expectedAccumValue)) + val expectedValue = SQLMetrics.stringValue(driverMetric.metricType, + Array(expectedAccumValue), Array.empty[Long]) val expectedValue2 = SQLMetrics.stringValue(driverMetric2.metricType, - Array(expectedAccumValue2)) + Array(expectedAccumValue2), Array.empty[Long]) assert(metrics.contains(driverMetric.id)) assert(metrics(driverMetric.id) === expectedValue) From 5954311739ff170b9a28cf8b20d1fbce1a13795a Mon Sep 17 00:00:00 2001 From: turbofei Date: Mon, 16 Dec 2019 14:40:07 -0800 Subject: [PATCH 0897/1387] [SPARK-29043][CORE] Improve the concurrent performance of History Server Even we set spark.history.fs.numReplayThreads to a large number, such as 30. The history server still replays logs slowly. We found that, if there is a straggler in a batch of replay tasks, all the other threads will wait for this straggler. In this PR, we create processing to save the logs which are being replayed. So that the replay tasks can execute Asynchronously. It can accelerate the speed to replay logs for history server. No. UT. Closes #25797 from turboFei/SPARK-29043. Authored-by: turbofei Signed-off-by: Marcelo Vanzin --- .../deploy/history/FsHistoryProvider.scala | 121 ++++++++++++------ .../history/FsHistoryProviderSuite.scala | 39 +++++- 2 files changed, 118 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index a3776b3ad756d..f560b7e9157b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -20,13 +20,12 @@ package org.apache.spark.deploy.history import java.io.{File, FileNotFoundException, IOException} import java.lang.{Long => JLong} import java.nio.file.Files -import java.util.{Date, ServiceLoader} +import java.util.{Date, NoSuchElementException, ServiceLoader} import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Future, TimeUnit} import java.util.zip.ZipOutputStream import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.concurrent.ExecutionException import scala.io.Source import scala.xml.Node @@ -160,6 +159,26 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) new HistoryServerDiskManager(conf, path, listing, clock) } + // Used to store the paths, which are being processed. This enable the replay log tasks execute + // asynchronously and make sure that checkForLogs would not process a path repeatedly. + private val processing = ConcurrentHashMap.newKeySet[String] + + private def isProcessing(path: Path): Boolean = { + processing.contains(path.getName) + } + + private def isProcessing(info: LogInfo): Boolean = { + processing.contains(info.logPath.split("/").last) + } + + private def processing(path: Path): Unit = { + processing.add(path.getName) + } + + private def endProcessing(path: Path): Unit = { + processing.remove(path.getName) + } + private val blacklist = new ConcurrentHashMap[String, Long] // Visible for testing @@ -440,6 +459,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val updated = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) .filter { entry => !isBlacklisted(entry.getPath) } + .filter { entry => !isProcessing(entry.getPath) } .flatMap { entry => EventLogFileReader(fs, entry) } .filter { reader => try { @@ -512,43 +532,17 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logDebug(s"New/updated attempts found: ${updated.size} ${updated.map(_.rootPath)}") } - val tasks = updated.flatMap { entry => + updated.foreach { entry => + processing(entry.rootPath) try { - val task: Future[Unit] = replayExecutor.submit( - () => mergeApplicationListing(entry, newLastScanTime, true)) - Some(task -> entry.rootPath) + val task: Runnable = () => mergeApplicationListing(entry, newLastScanTime, true) + replayExecutor.submit(task) } catch { // let the iteration over the updated entries break, since an exception on // replayExecutor.submit (..) indicates the ExecutorService is unable // to take any more submissions at this time case e: Exception => logError(s"Exception while submitting event log for replay", e) - None - } - } - - pendingReplayTasksCount.addAndGet(tasks.size) - - // Wait for all tasks to finish. This makes sure that checkForLogs - // is not scheduled again while some tasks are already running in - // the replayExecutor. - tasks.foreach { case (task, path) => - try { - task.get() - } catch { - case e: InterruptedException => - throw e - case e: ExecutionException if e.getCause.isInstanceOf[AccessControlException] => - // We don't have read permissions on the log file - logWarning(s"Unable to read log $path", e.getCause) - blacklist(path) - // SPARK-28157 We should remove this blacklisted entry from the KVStore - // to handle permission-only changes with the same file sizes later. - listing.delete(classOf[LogInfo], path.toString) - case e: Exception => - logError("Exception while merging application listings", e) - } finally { - pendingReplayTasksCount.decrementAndGet() } } @@ -563,7 +557,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) .last(newLastScanTime - 1) .asScala .toList - stale.foreach { log => + stale.filterNot(isProcessing).foreach { log => log.appId.foreach { appId => cleanAppData(appId, log.attemptId, log.logPath) listing.delete(classOf[LogInfo], log.logPath) @@ -664,10 +658,39 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + private def mergeApplicationListing( + reader: EventLogFileReader, + scanTime: Long, + enableOptimizations: Boolean): Unit = { + try { + pendingReplayTasksCount.incrementAndGet() + doMergeApplicationListing(reader, scanTime, enableOptimizations) + if (conf.get(CLEANER_ENABLED)) { + checkAndCleanLog(reader.rootPath.toString) + } + } catch { + case e: InterruptedException => + throw e + case e: AccessControlException => + // We don't have read permissions on the log file + logWarning(s"Unable to read log ${reader.rootPath}", e) + blacklist(reader.rootPath) + // SPARK-28157 We should remove this blacklisted entry from the KVStore + // to handle permission-only changes with the same file sizes later. + listing.delete(classOf[LogInfo], reader.rootPath.toString) + case e: Exception => + logError("Exception while merging application listings", e) + } finally { + endProcessing(reader.rootPath) + pendingReplayTasksCount.decrementAndGet() + } + } + /** * Replay the given log file, saving the application in the listing db. + * Visable for testing */ - protected def mergeApplicationListing( + private[history] def doMergeApplicationListing( reader: EventLogFileReader, scanTime: Long, enableOptimizations: Boolean): Unit = { @@ -773,7 +796,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // mean the end event is before the configured threshold, so call the method again to // re-parse the whole log. logInfo(s"Reparsing $logPath since end event was not found.") - mergeApplicationListing(reader, scanTime, enableOptimizations = false) + doMergeApplicationListing(reader, scanTime, enableOptimizations = false) case _ => // If the app hasn't written down its app ID to the logs, still record the entry in the @@ -798,6 +821,30 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + /** + * Check and delete specified event log according to the max log age defined by the user. + */ + private[history] def checkAndCleanLog(logPath: String): Unit = Utils.tryLog { + val maxTime = clock.getTimeMillis() - conf.get(MAX_LOG_AGE_S) * 1000 + val log = listing.read(classOf[LogInfo], logPath) + + if (log.lastProcessed <= maxTime && log.appId.isEmpty) { + logInfo(s"Deleting invalid / corrupt event log ${log.logPath}") + deleteLog(fs, new Path(log.logPath)) + listing.delete(classOf[LogInfo], log.logPath) + } + + log.appId.foreach { appId => + val app = listing.read(classOf[ApplicationInfoWrapper], appId) + if (app.oldestAttempt() <= maxTime) { + val (remaining, toDelete) = app.attempts.partition { attempt => + attempt.info.lastUpdated.getTime() >= maxTime + } + deleteAttemptLogs(app, remaining, toDelete) + } + } + } + /** * Delete event logs from the log directory according to the clean policy defined by the user. */ @@ -827,7 +874,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) .asScala .filter { l => l.logType == null || l.logType == LogType.EventLogs } .toList - stale.foreach { log => + stale.filterNot(isProcessing).foreach { log => if (log.appId.isEmpty) { logInfo(s"Deleting invalid / corrupt event log ${log.logPath}") deleteLog(fs, new Path(log.logPath)) @@ -935,7 +982,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) .asScala .filter { l => l.logType != null && l.logType == LogType.DriverLogs } .toList - stale.foreach { log => + stale.filterNot(isProcessing).foreach { log => logInfo(s"Deleting invalid driver log ${log.logPath}") listing.delete(classOf[LogInfo], log.logPath) deleteLog(driverLogFs, new Path(log.logPath)) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index ed195dd44e917..a96667ffacd26 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -160,13 +160,13 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { assume(!Utils.isWindows) class TestFsHistoryProvider extends FsHistoryProvider(createTestConf()) { - var mergeApplicationListingCall = 0 - override protected def mergeApplicationListing( + var doMergeApplicationListingCall = 0 + override private[history] def doMergeApplicationListing( reader: EventLogFileReader, lastSeen: Long, enableSkipToEnd: Boolean): Unit = { - super.mergeApplicationListing(reader, lastSeen, enableSkipToEnd) - mergeApplicationListingCall += 1 + super.doMergeApplicationListing(reader, lastSeen, enableSkipToEnd) + doMergeApplicationListingCall += 1 } } val provider = new TestFsHistoryProvider @@ -187,7 +187,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { list.size should be (1) } - provider.mergeApplicationListingCall should be (1) + provider.doMergeApplicationListingCall should be (1) } test("history file is renamed from inprogress to completed") { @@ -1321,6 +1321,35 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { assertSerDe(serializer, attemptInfoWithIndex) } + test("SPARK-29043: clean up specified event log") { + val clock = new ManualClock() + val conf = createTestConf().set(MAX_LOG_AGE_S, 0L).set(CLEANER_ENABLED, true) + val provider = new FsHistoryProvider(conf, clock) + + // create an invalid application log file + val inValidLogFile = newLogFile("inValidLogFile", None, inProgress = true) + inValidLogFile.createNewFile() + writeFile(inValidLogFile, None, + SparkListenerApplicationStart(inValidLogFile.getName, None, 1L, "test", None)) + inValidLogFile.setLastModified(clock.getTimeMillis()) + + // create a valid application log file + val validLogFile = newLogFile("validLogFile", None, inProgress = true) + validLogFile.createNewFile() + writeFile(validLogFile, None, + SparkListenerApplicationStart(validLogFile.getName, Some("local_123"), 1L, "test", None)) + validLogFile.setLastModified(clock.getTimeMillis()) + + provider.checkForLogs() + // The invalid application log file would be cleaned by checkAndCleanLog(). + assert(new File(testDir.toURI).listFiles().size === 1) + + clock.advance(1) + // cleanLogs() would clean the valid application log file. + provider.cleanLogs() + assert(new File(testDir.toURI).listFiles().size === 0) + } + private def assertOptionAfterSerde(opt: Option[Long], expected: Option[Long]): Unit = { if (expected.isEmpty) { assert(opt.isEmpty) From dd217e10fc0408831c2c658fc3f52d2917f1a6a2 Mon Sep 17 00:00:00 2001 From: shahid Date: Mon, 16 Dec 2019 15:02:34 -0800 Subject: [PATCH 0898/1387] [SPARK-25392][CORE][WEBUI] Prevent error page when accessing pools page from history server ### What changes were proposed in this pull request? ### Why are the changes needed? Currently from history server, we will not able to access the pool info, as we aren't writing pool information to the event log other than pool name. Already spark is hiding pool table when accessing from history server. But from the pool column in the stage table will redirect to the pools table, and that will throw error when accessing the pools page. To prevent error page, we need to hide the pool column also in the stage table ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual test Before change: ![Screenshot 2019-11-21 at 6 49 40 AM](https://user-images.githubusercontent.com/23054875/69293868-219b2280-0c30-11ea-9b9a-17140d024d3a.png) ![Screenshot 2019-11-21 at 6 48 51 AM](https://user-images.githubusercontent.com/23054875/69293834-147e3380-0c30-11ea-9dec-d5f67665486d.png) After change: ![Screenshot 2019-11-21 at 7 29 01 AM](https://user-images.githubusercontent.com/23054875/69293991-9cfcd400-0c30-11ea-98a0-7a6268a4e5ab.png) Closes #26616 from shahidki31/poolHistory. Authored-by: shahid Signed-off-by: Marcelo Vanzin --- .../main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala | 3 +-- core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala | 2 ++ core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala | 2 ++ 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index f672ce0ec6a68..d8a93adbbe90a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -30,7 +30,6 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { private val sc = parent.sc private val subPath = "stages" - private def isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { // For now, pool information is only accessible in live UIs @@ -57,7 +56,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
    - val poolsDescription = if (sc.isDefined && isFairScheduler) { + val poolsDescription = if (parent.isFairScheduler) {

    diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index c2644a8eea157..dc3106400dd2b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -32,7 +32,9 @@ private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore) val sc = parent.sc val killEnabled = parent.killEnabled + // Show pool information for only live UI. def isFairScheduler: Boolean = { + sc.isDefined && store .environmentInfo() .sparkProperties diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 2d222b842be55..b59dd333da19e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -37,7 +37,9 @@ private[ui] class StagesTab(val parent: SparkUI, val store: AppStatusStore) attachPage(new StagePage(this, store)) attachPage(new PoolPage(this)) + // Show pool information for only live UI. def isFairScheduler: Boolean = { + sc.isDefined && store .environmentInfo() .sparkProperties From 5ed72a194076c7890672dfe38ccf687bc0eab03c Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 16 Dec 2019 18:15:40 -0600 Subject: [PATCH 0899/1387] [SPARK-30247][PYSPARK] GaussianMixtureModel in py side should expose gaussian ### What changes were proposed in this pull request? expose gaussian in PySpark ### Why are the changes needed? A ```GaussianMixtureModel``` contains two parts of coefficients: ```weights``` & ```gaussians```. However, ```gaussians``` is not exposed on Python side. ### Does this PR introduce any user-facing change? Yes. ```GaussianMixtureModel.gaussians``` is exposed in PySpark. ### How was this patch tested? add doctest Closes #26882 from huaxingao/spark-30247. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- python/pyspark/ml/clustering.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 99c38765be9fb..f784b8f208dc5 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -154,6 +154,15 @@ def weights(self): """ return self._call_java("weights") + @property + @since("3.0.0") + def gaussians(self): + """ + Array of :py:class:`MultivariateGaussian` where gaussians[i] represents + the Multivariate Gaussian (Normal) Distribution for Gaussian i + """ + return self._call_java("gaussians") + @property @since("2.0.0") def gaussiansDF(self): @@ -251,6 +260,9 @@ class GaussianMixture(JavaEstimator, _GaussianMixtureParams, JavaMLWritable, Jav >>> weights = model.weights >>> len(weights) 3 + >>> gaussians = model.gaussians + >>> len(gaussians) + 3 >>> model.gaussiansDF.select("mean").head() Row(mean=DenseVector([0.825, 0.8675])) >>> model.gaussiansDF.select("cov").head() @@ -273,6 +285,8 @@ class GaussianMixture(JavaEstimator, _GaussianMixtureParams, JavaMLWritable, Jav False >>> model2.weights == model.weights True + >>> model2.gaussians == model.gaussians + True >>> model2.gaussiansDF.select("mean").head() Row(mean=DenseVector([0.825, 0.8675])) >>> model2.gaussiansDF.select("cov").head() From b03ce63c058fad62285293f5a640d0a44db6cda9 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 16 Dec 2019 18:24:32 -0600 Subject: [PATCH 0900/1387] [SPARK-30258][TESTS] Eliminate warnings of deprecated Spark APIs in tests ### What changes were proposed in this pull request? In the PR, I propose to move all tests that use deprecated Spark APIs to separate test classes, and add the annotation: ```scala deprecated("This test suite will be removed.", "3.0.0") ``` The annotation suppress warnings from already deprecated methods and classes. ### Why are the changes needed? The warnings about deprecated Spark APIs in tests does not indicate any issues because the tests use such APIs intentionally. Eliminating the warnings allows to highlight other warnings that could show real problems. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites and by - DeprecatedAvroFunctionsSuite - DeprecatedDateFunctionsSuite - DeprecatedDatasetAggregatorSuite - DeprecatedStreamingAggregationSuite - DeprecatedWholeStageCodegenSuite Closes #26885 from MaxGekk/eliminate-deprecate-warnings. Authored-by: Maxim Gekk Signed-off-by: Sean Owen --- .../spark/sql/avro/AvroFunctionsSuite.scala | 23 ++-- .../avro/DeprecatedAvroFunctionsSuite.scala | 123 ++++++++++++++++++ .../spark/sql/DatasetAggregatorSuite.scala | 61 --------- .../apache/spark/sql/DateFunctionsSuite.scala | 87 +------------ .../DeprecatedDatasetAggregatorSuite.scala | 77 +++++++++++ .../sql/DeprecatedDateFunctionsSuite.scala | 113 ++++++++++++++++ .../org/apache/spark/sql/JoinSuite.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 +- .../DeprecatedWholeStageCodegenSuite.scala | 40 ++++++ .../execution/WholeStageCodegenSuite.scala | 14 -- .../DeprecatedStreamingAggregationSuite.scala | 62 +++++++++ .../streaming/StreamingAggregationSuite.scala | 11 -- 12 files changed, 438 insertions(+), 185 deletions(-) create mode 100644 external/avro/src/test/scala/org/apache/spark/sql/avro/DeprecatedAvroFunctionsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDatasetAggregatorSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDateFunctionsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.scala diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala index f0083e95fc2ce..d6c22c00af7b3 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala @@ -38,7 +38,9 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { test("roundtrip in to_avro and from_avro - int and string") { val df = spark.range(10).select('id, 'id.cast("string").as("str")) - val avroDF = df.select(to_avro('id).as("a"), to_avro('str).as("b")) + val avroDF = df.select( + functions.to_avro('id).as("a"), + functions.to_avro('str).as("b")) val avroTypeLong = s""" |{ | "type": "int", @@ -51,12 +53,14 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { | "name": "str" |} """.stripMargin - checkAnswer(avroDF.select(from_avro('a, avroTypeLong), from_avro('b, avroTypeStr)), df) + checkAnswer(avroDF.select( + functions.from_avro('a, avroTypeLong), + functions.from_avro('b, avroTypeStr)), df) } test("roundtrip in to_avro and from_avro - struct") { val df = spark.range(10).select(struct('id, 'id.cast("string").as("str")).as("struct")) - val avroStructDF = df.select(to_avro('struct).as("avro")) + val avroStructDF = df.select(functions.to_avro('struct).as("avro")) val avroTypeStruct = s""" |{ | "type": "record", @@ -67,13 +71,14 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { | ] |} """.stripMargin - checkAnswer(avroStructDF.select(from_avro('avro, avroTypeStruct)), df) + checkAnswer(avroStructDF.select( + functions.from_avro('avro, avroTypeStruct)), df) } test("handle invalid input in from_avro") { val count = 10 val df = spark.range(count).select(struct('id, 'id.as("id2")).as("struct")) - val avroStructDF = df.select(to_avro('struct).as("avro")) + val avroStructDF = df.select(functions.to_avro('struct).as("avro")) val avroTypeStruct = s""" |{ | "type": "record", @@ -87,7 +92,7 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { intercept[SparkException] { avroStructDF.select( - org.apache.spark.sql.avro.functions.from_avro( + functions.from_avro( 'avro, avroTypeStruct, Map("mode" -> "FAILFAST").asJava)).collect() } @@ -95,7 +100,7 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { val expected = (0 until count).map(_ => Row(Row(null, null))) checkAnswer( avroStructDF.select( - org.apache.spark.sql.avro.functions.from_avro( + functions.from_avro( 'avro, avroTypeStruct, Map("mode" -> "PERMISSIVE").asJava)), expected) } @@ -115,8 +120,8 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession { | }, "null" ] |}, "null" ] """.stripMargin - val readBackOne = dfOne.select(to_avro($"array").as("avro")) - .select(from_avro($"avro", avroTypeArrStruct).as("array")) + val readBackOne = dfOne.select(functions.to_avro($"array").as("avro")) + .select(functions.from_avro($"avro", avroTypeArrStruct).as("array")) checkAnswer(dfOne, readBackOne) } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/DeprecatedAvroFunctionsSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/DeprecatedAvroFunctionsSuite.scala new file mode 100644 index 0000000000000..cdfa1b118b18d --- /dev/null +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/DeprecatedAvroFunctionsSuite.scala @@ -0,0 +1,123 @@ +/* + * 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.avro + +import java.io.ByteArrayOutputStream + +import org.apache.avro.Schema +import org.apache.avro.generic.{GenericDatumWriter, GenericRecord, GenericRecordBuilder} +import org.apache.avro.io.EncoderFactory + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.execution.LocalTableScanExec +import org.apache.spark.sql.functions.{col, struct} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +@deprecated("This test suite will be removed.", "3.0.0") +class DeprecatedAvroFunctionsSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + test("roundtrip in to_avro and from_avro - int and string") { + val df = spark.range(10).select('id, 'id.cast("string").as("str")) + + val avroDF = df.select(to_avro('id).as("a"), to_avro('str).as("b")) + val avroTypeLong = s""" + |{ + | "type": "int", + | "name": "id" + |} + """.stripMargin + val avroTypeStr = s""" + |{ + | "type": "string", + | "name": "str" + |} + """.stripMargin + checkAnswer(avroDF.select(from_avro('a, avroTypeLong), from_avro('b, avroTypeStr)), df) + } + + test("roundtrip in to_avro and from_avro - struct") { + val df = spark.range(10).select(struct('id, 'id.cast("string").as("str")).as("struct")) + val avroStructDF = df.select(to_avro('struct).as("avro")) + val avroTypeStruct = s""" + |{ + | "type": "record", + | "name": "struct", + | "fields": [ + | {"name": "col1", "type": "long"}, + | {"name": "col2", "type": "string"} + | ] + |} + """.stripMargin + checkAnswer(avroStructDF.select(from_avro('avro, avroTypeStruct)), df) + } + + test("roundtrip in to_avro and from_avro - array with null") { + val dfOne = Seq(Tuple1(Tuple1(1) :: Nil), Tuple1(null :: Nil)).toDF("array") + val avroTypeArrStruct = s""" + |[ { + | "type" : "array", + | "items" : [ { + | "type" : "record", + | "name" : "x", + | "fields" : [ { + | "name" : "y", + | "type" : "int" + | } ] + | }, "null" ] + |}, "null" ] + """.stripMargin + val readBackOne = dfOne.select(to_avro($"array").as("avro")) + .select(from_avro($"avro", avroTypeArrStruct).as("array")) + checkAnswer(dfOne, readBackOne) + } + + test("SPARK-27798: from_avro produces same value when converted to local relation") { + val simpleSchema = + """ + |{ + | "type": "record", + | "name" : "Payload", + | "fields" : [ {"name" : "message", "type" : "string" } ] + |} + """.stripMargin + + def generateBinary(message: String, avroSchema: String): Array[Byte] = { + val schema = new Schema.Parser().parse(avroSchema) + val out = new ByteArrayOutputStream() + val writer = new GenericDatumWriter[GenericRecord](schema) + val encoder = EncoderFactory.get().binaryEncoder(out, null) + val rootRecord = new GenericRecordBuilder(schema).set("message", message).build() + writer.write(rootRecord, encoder) + encoder.flush() + out.toByteArray + } + + // This bug is hit when the rule `ConvertToLocalRelation` is run. But the rule was excluded + // in `SharedSparkSession`. + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "") { + val df = Seq("one", "two", "three", "four").map(generateBinary(_, simpleSchema)) + .toDF() + .withColumn("value", from_avro(col("value"), simpleSchema)) + + assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec]) + assert(df.collect().map(_.get(0)) === Seq(Row("one"), Row("two"), Row("three"), Row("four"))) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index 817387b2845f5..6ffe133ee652b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -19,12 +19,10 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.expressions.Aggregator -import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{BooleanType, IntegerType, StringType, StructType} - object ComplexResultAgg extends Aggregator[(String, Int), (Long, Long), (Long, Long)] { override def zero: (Long, Long) = (0, 0) override def reduce(countAndSum: (Long, Long), input: (String, Int)): (Long, Long) = { @@ -226,25 +224,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { private implicit val ordering = Ordering.by((c: AggData) => c.a -> c.b) - test("typed aggregation: TypedAggregator") { - val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - - checkDataset( - ds.groupByKey(_._1).agg(typed.sum(_._2)), - ("a", 30.0), ("b", 3.0), ("c", 1.0)) - } - - test("typed aggregation: TypedAggregator, expr, expr") { - val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - - checkDataset( - ds.groupByKey(_._1).agg( - typed.sum(_._2), - expr("sum(_2)").as[Long], - count("*")), - ("a", 30.0, 30L, 2L), ("b", 3.0, 3L, 2L), ("c", 1.0, 1L, 1L)) - } - test("typed aggregation: complex result type") { val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS() @@ -255,17 +234,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { ("a", 2.0, (2L, 4L)), ("b", 3.0, (1L, 3L))) } - test("typed aggregation: in project list") { - val ds = Seq(1, 3, 2, 5).toDS() - - checkDataset( - ds.select(typed.sum((i: Int) => i)), - 11.0) - checkDataset( - ds.select(typed.sum((i: Int) => i), typed.sum((i: Int) => i * 2)), - 11.0 -> 22.0) - } - test("typed aggregation: class input") { val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS() @@ -315,14 +283,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { ("one", 1), ("two", 1)) } - test("typed aggregate: avg, count, sum") { - val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS() - checkDataset( - ds.groupByKey(_._1).agg( - typed.avg(_._2), typed.count(_._2), typed.sum(_._2), typed.sumLong(_._2)), - ("a", 2.0, 2L, 4.0, 4L), ("b", 3.0, 1L, 3.0, 3L)) - } - test("generic typed sum") { val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS() checkDataset( @@ -366,18 +326,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { checkAnswer(df2.agg(RowAgg.toColumn as "b").select("b"), Row(6) :: Nil) } - test("spark-15114 shorter system generated alias names") { - val ds = Seq(1, 3, 2, 5).toDS() - assert(ds.select(typed.sum((i: Int) => i)).columns.head === "TypedSumDouble(int)") - val ds2 = ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i)) - assert(ds2.columns.head === "TypedSumDouble(int)") - assert(ds2.columns.last === "TypedAverage(int)") - val df = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") - assert(df.groupBy($"j").agg(RowAgg.toColumn).columns.last == - "RowAgg(org.apache.spark.sql.Row)") - assert(df.groupBy($"j").agg(RowAgg.toColumn as "agg1").columns.last == "agg1") - } - test("SPARK-15814 Aggregator can return null result") { val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS() checkDatasetUnorderly( @@ -390,15 +338,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession { checkDataset(ds.select(MapTypeBufferAgg.toColumn), 1) } - test("SPARK-15204 improve nullability inference for Aggregator") { - val ds1 = Seq(1, 3, 2, 5).toDS() - assert(ds1.select(typed.sum((i: Int) => i)).schema.head.nullable === false) - val ds2 = Seq(AggData(1, "a"), AggData(2, "a")).toDS() - assert(ds2.select(SeqAgg.toColumn).schema.head.nullable) - val ds3 = sql("SELECT 'Some String' AS b, 1279869254 AS a").as[AggData] - assert(ds3.select(NameAgg.toColumn).schema.head.nullable) - } - test("SPARK-18147: very complex aggregator result type") { val df = Seq(1 -> "a", 2 -> "b", 2 -> "c").toDF("i", "j") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index c80e675b149d0..7b6b935496678 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{DoubleType, StructField, StructType} +import org.apache.spark.sql.types.DoubleType import org.apache.spark.unsafe.types.CalendarInterval class DateFunctionsSuite extends QueryTest with SharedSparkSession { @@ -704,91 +704,6 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df.selectExpr("datediff(a, d)"), Seq(Row(1), Row(1))) } - test("from_utc_timestamp with literal zone") { - val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") - ).toDF("a", "b") - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - checkAnswer( - df.select(from_utc_timestamp(col("a"), "PST")), - Seq( - Row(Timestamp.valueOf("2015-07-23 17:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - checkAnswer( - df.select(from_utc_timestamp(col("b"), "PST")), - Seq( - Row(Timestamp.valueOf("2015-07-23 17:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - } - val msg = intercept[AnalysisException] { - df.select(from_utc_timestamp(col("a"), "PST")).collect() - }.getMessage - assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) - } - - test("from_utc_timestamp with column zone") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "CET"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "PST") - ).toDF("a", "b", "c") - checkAnswer( - df.select(from_utc_timestamp(col("a"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 02:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - checkAnswer( - df.select(from_utc_timestamp(col("b"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 02:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - } - } - - test("to_utc_timestamp with literal zone") { - val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") - ).toDF("a", "b") - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - checkAnswer( - df.select(to_utc_timestamp(col("a"), "PST")), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-25 07:00:00")))) - checkAnswer( - df.select(to_utc_timestamp(col("b"), "PST")), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-25 07:00:00")))) - } - val msg = intercept[AnalysisException] { - df.select(to_utc_timestamp(col("a"), "PST")).collect() - }.getMessage - assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) - } - - test("to_utc_timestamp with column zone") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "PST"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "CET") - ).toDF("a", "b", "c") - checkAnswer( - df.select(to_utc_timestamp(col("a"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-24 22:00:00")))) - checkAnswer( - df.select(to_utc_timestamp(col("b"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-24 22:00:00")))) - } - } - - test("to_timestamp with microseconds precision") { withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { val timestamp = "1970-01-01T00:00:00.123456Z" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDatasetAggregatorSuite.scala new file mode 100644 index 0000000000000..b1d5e80f8563f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDatasetAggregatorSuite.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.expressions.scalalang.typed +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSparkSession + +@deprecated("This test suite will be removed.", "3.0.0") +class DeprecatedDatasetAggregatorSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + test("typed aggregation: TypedAggregator") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDataset( + ds.groupByKey(_._1).agg(typed.sum(_._2)), + ("a", 30.0), ("b", 3.0), ("c", 1.0)) + } + + test("typed aggregation: TypedAggregator, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDataset( + ds.groupByKey(_._1).agg( + typed.sum(_._2), + expr("sum(_2)").as[Long], + count("*")), + ("a", 30.0, 30L, 2L), ("b", 3.0, 3L, 2L), ("c", 1.0, 1L, 1L)) + } + + test("typed aggregation: in project list") { + val ds = Seq(1, 3, 2, 5).toDS() + + checkDataset( + ds.select(typed.sum((i: Int) => i)), + 11.0) + checkDataset( + ds.select(typed.sum((i: Int) => i), typed.sum((i: Int) => i * 2)), + 11.0 -> 22.0) + } + + test("typed aggregate: avg, count, sum") { + val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS() + checkDataset( + ds.groupByKey(_._1).agg( + typed.avg(_._2), typed.count(_._2), typed.sum(_._2), typed.sumLong(_._2)), + ("a", 2.0, 2L, 4.0, 4L), ("b", 3.0, 1L, 3.0, 3L)) + } + + test("spark-15114 shorter system generated alias names") { + val ds = Seq(1, 3, 2, 5).toDS() + assert(ds.select(typed.sum((i: Int) => i)).columns.head === "TypedSumDouble(int)") + val ds2 = ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i)) + assert(ds2.columns.head === "TypedSumDouble(int)") + assert(ds2.columns.last === "TypedAverage(int)") + val df = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") + assert(df.groupBy($"j").agg(RowAgg.toColumn).columns.last == + "RowAgg(org.apache.spark.sql.Row)") + assert(df.groupBy($"j").agg(RowAgg.toColumn as "agg1").columns.last == "agg1") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDateFunctionsSuite.scala new file mode 100644 index 0000000000000..bef83ee199cc3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDateFunctionsSuite.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 + +import java.sql.Timestamp + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +@deprecated("This test suite will be removed.", "3.0.0") +class DeprecatedDateFunctionsSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + test("from_utc_timestamp with literal zone") { + val df = Seq( + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") + ).toDF("a", "b") + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + checkAnswer( + df.select(from_utc_timestamp(col("a"), "PST")), + Seq( + Row(Timestamp.valueOf("2015-07-23 17:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + checkAnswer( + df.select(from_utc_timestamp(col("b"), "PST")), + Seq( + Row(Timestamp.valueOf("2015-07-23 17:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + } + val msg = intercept[AnalysisException] { + df.select(from_utc_timestamp(col("a"), "PST")).collect() + }.getMessage + assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) + } + + test("from_utc_timestamp with column zone") { + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + val df = Seq( + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "CET"), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "PST") + ).toDF("a", "b", "c") + checkAnswer( + df.select(from_utc_timestamp(col("a"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 02:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + checkAnswer( + df.select(from_utc_timestamp(col("b"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 02:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + } + } + + test("to_utc_timestamp with literal zone") { + val df = Seq( + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") + ).toDF("a", "b") + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + checkAnswer( + df.select(to_utc_timestamp(col("a"), "PST")), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-25 07:00:00")))) + checkAnswer( + df.select(to_utc_timestamp(col("b"), "PST")), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-25 07:00:00")))) + } + val msg = intercept[AnalysisException] { + df.select(to_utc_timestamp(col("a"), "PST")).collect() + }.getMessage + assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) + } + + test("to_utc_timestamp with column zone") { + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + val df = Seq( + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "PST"), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "CET") + ).toDF("a", "b", "c") + checkAnswer( + df.select(to_utc_timestamp(col("a"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-24 22:00:00")))) + checkAnswer( + df.select(to_utc_timestamp(col("b"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-24 22:00:00")))) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index ac7976090ef89..068ea05ead351 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -27,7 +27,7 @@ import org.mockito.Mockito._ import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.Filter import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec, SparkPlan} import org.apache.spark.sql.execution.joins._ @@ -238,7 +238,9 @@ class JoinSuite extends QueryTest with SharedSparkSession { checkAnswer( bigDataX.join(bigDataY).where($"x.key" === $"y.key"), - testData.rdd.flatMap(row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq) + testData.rdd.flatMap { row => + Seq.fill(16)(new GenericRow(Seq(row, row).flatMap(_.toSeq).toArray)) + }.collect().toSeq) } test("cartesian product join") { 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 37d98f7c87420..06309bfef7e9a 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 @@ -26,6 +26,7 @@ import scala.collection.parallel.immutable.ParVector import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.HiveResult.hiveResultString @@ -783,8 +784,9 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { | SELECT * FROM testData UNION ALL | SELECT * FROM testData) y |WHERE x.key = y.key""".stripMargin), - testData.rdd.flatMap( - row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq) + testData.rdd.flatMap { row => + Seq.fill(16)(new GenericRow(Seq(row, row).flatMap(_.toSeq).toArray)) + }.collect().toSeq) } test("cartesian product join") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala new file mode 100644 index 0000000000000..c198978f5888d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala @@ -0,0 +1,40 @@ +/* + * 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 + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.expressions.scalalang.typed +import org.apache.spark.sql.test.SharedSparkSession + +@deprecated("This test suite will be removed.", "3.0.0") +class DeprecatedWholeStageCodegenSuite extends QueryTest with SharedSparkSession { + + test("simple typed UDAF should be included in WholeStageCodegen") { + import testImplicits._ + + val ds = Seq(("a", 10), ("b", 1), ("b", 2), ("c", 1)).toDS() + .groupByKey(_._1).agg(typed.sum(_._2)) + + val plan = ds.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) + assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index afe9eb5c151d7..572932fc2750b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec -import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -107,19 +106,6 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { assert(ds.collect() === Array(0, 6)) } - test("simple typed UDAF should be included in WholeStageCodegen") { - import testImplicits._ - - val ds = Seq(("a", 10), ("b", 1), ("b", 2), ("c", 1)).toDS() - .groupByKey(_._1).agg(typed.sum(_._2)) - - val plan = ds.queryExecution.executedPlan - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) - assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) - } - test("cache for primitive type should be in WholeStageCodegen with InMemoryTableScanExec") { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.scala new file mode 100644 index 0000000000000..99f7e32d4df72 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.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.sql.streaming + +import org.scalatest.Assertions + +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.state.StreamingAggregationStateManager +import org.apache.spark.sql.expressions.scalalang.typed +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.OutputMode._ + +@deprecated("This test suite will be removed.", "3.0.0") +class DeprecatedStreamingAggregationSuite extends StateStoreMetricsTest with Assertions { + + import testImplicits._ + + def executeFuncWithStateVersionSQLConf( + stateVersion: Int, + confPairs: Seq[(String, String)], + func: => Any): Unit = { + withSQLConf(confPairs ++ + Seq(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString): _*) { + func + } + } + + def testWithAllStateVersions(name: String, confPairs: (String, String)*) + (func: => Any): Unit = { + for (version <- StreamingAggregationStateManager.supportedVersions) { + test(s"$name - state format version $version") { + executeFuncWithStateVersionSQLConf(version, confPairs, func) + } + } + } + + + testWithAllStateVersions("typed aggregators") { + val inputData = MemoryStream[(String, Int)] + val aggregated = inputData.toDS().groupByKey(_._1).agg(typed.sumLong(_._2)) + + testStream(aggregated, Update)( + AddData(inputData, ("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)), + CheckLastBatch(("a", 30), ("b", 3), ("c", 1)) + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 6dbf4ff283af9..9779635df3ef1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.execution.streaming.state.StreamingAggregationStateManager -import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode._ @@ -280,16 +279,6 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { ) } - testWithAllStateVersions("typed aggregators") { - val inputData = MemoryStream[(String, Int)] - val aggregated = inputData.toDS().groupByKey(_._1).agg(typed.sumLong(_._2)) - - testStream(aggregated, Update)( - AddData(inputData, ("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)), - CheckLastBatch(("a", 30), ("b", 3), ("c", 1)) - ) - } - testWithAllStateVersions("prune results by current_time, complete mode") { import testImplicits._ val clock = new StreamManualClock From 5de5e46624b3c8733b6c3c29053b611c279022e2 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 17 Dec 2019 10:22:29 +0900 Subject: [PATCH 0901/1387] [SPARK-30268][INFRA] Fix incorrect pyspark version when releasing preview versions ### What changes were proposed in this pull request? This PR fix incorrect pyspark version when releasing preview versions. ### Why are the changes needed? Failed to make Spark binary distribution: ``` cp: cannot stat 'spark-3.0.0-preview2-bin-hadoop2.7/python/dist/pyspark-3.0.0.dev02.tar.gz': No such file or directory gpg: can't open 'pyspark-3.0.0.dev02.tar.gz': No such file or directory gpg: signing failed: No such file or directory gpg: pyspark-3.0.0.dev02.tar.gz: No such file or directory ``` ``` yumwangubuntu-3513086:~/spark-release/output$ ll spark-3.0.0-preview2-bin-hadoop2.7/python/dist/ total 214140 drwxr-xr-x 2 yumwang stack 4096 Dec 16 06:17 ./ drwxr-xr-x 9 yumwang stack 4096 Dec 16 06:17 ../ -rw-r--r-- 1 yumwang stack 219267173 Dec 16 06:17 pyspark-3.0.0.dev2.tar.gz ``` ``` /usr/local/lib/python3.6/dist-packages/setuptools/dist.py:476: UserWarning: Normalizing '3.0.0.dev02' to '3.0.0.dev2' normalized_version, ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test: ``` LM-SHC-16502798:spark yumwang$ SPARK_VERSION=3.0.0-preview2 LM-SHC-16502798:spark yumwang$ echo "$SPARK_VERSION" | sed -e "s/-/./" -e "s/SNAPSHOT/dev0/" -e "s/preview/dev/" 3.0.0.dev2 ``` Closes #26909 from wangyum/SPARK-30268. Authored-by: Yuming Wang Signed-off-by: HyukjinKwon --- dev/create-release/release-build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index d93382b516cef..022d3af95c05d 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -220,7 +220,7 @@ if [[ "$1" == "package" ]]; then # Write out the VERSION to PySpark version info we rewrite the - into a . and SNAPSHOT # to dev0 to be closer to PEP440. - PYSPARK_VERSION=`echo "$SPARK_VERSION" | sed -e "s/-/./" -e "s/SNAPSHOT/dev0/" -e "s/preview/dev0/"` + PYSPARK_VERSION=`echo "$SPARK_VERSION" | sed -e "s/-/./" -e "s/SNAPSHOT/dev0/" -e "s/preview/dev/"` echo "__version__='$PYSPARK_VERSION'" > python/pyspark/version.py # Get maven home set by MVN From c216ef1d0371170847b007249c33f6f9f18700a0 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 17 Dec 2019 01:57:21 +0000 Subject: [PATCH 0902/1387] Preparing Spark release v3.0.0-preview2-rc2 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10-token-provider/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graph/api/pom.xml | 2 +- graph/cypher/pom.xml | 2 +- graph/graph/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 42 files changed, 43 insertions(+), 43 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 95d3e52bef3a9..2d34a0c3e2afa 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 3.0.0 +Version: 3.0.0-preview2 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ef916fb99a04c..715a1124b87c0 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index a1c8a8e6582eb..965c1f37db594 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 163c250054e4d..557bd7a94b0f1 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index a6d99813a8501..8c718d84bcf0c 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 76a402bb2bd31..65f491c5e006f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 3c3c0d2d96a1c..2bc7cd84a6081 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 883b73a69c9de..7826c2d16d4b1 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 93a4f67fd23f2..3d8b039ebeb00 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 9d54d21b95ba3..d2e09ae75e51b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index a888620139207..81424e26090c7 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.0.0-SNAPSHOT -SPARK_VERSION_SHORT: 3.0.0 +SPARK_VERSION: 3.0.0-preview2 +SPARK_VERSION_SHORT: 3.0.0-preview2 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index a099f1e042e99..5f0564575b510 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index ba6f20bfdbf58..d5deb746a8a38 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index aff79b8b8e642..04cf2e0a820e9 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index f2dcf5d217a89..55d9f72551211 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 693820da6af6b..e785d7a012f1c 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index 01ca96b4f8f40..ce4abf902b8af 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index d11569d709b23..b32d24d399a30 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 0ce922349ea66..c4b419a12d001 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 7d69764b77de7..8e7431b2a1dc6 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index db64b201abc2c..3fbb2966e7710 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/graph/api/pom.xml b/graph/api/pom.xml index a18c5290195fe..1d8ae4e46d952 100644 --- a/graph/api/pom.xml +++ b/graph/api/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 12b3832d9777a..20e91d2342aa2 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml index 7438ee5e54fe4..1bec2205e18b8 100644 --- a/graph/graph/pom.xml +++ b/graph/graph/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 444568a03d6c7..2d069806e3c4e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 02734e82ed26e..8375c8e35b91f 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e75e8345cd51d..2950313135ef2 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 2eab868ac0dc8..122b20374493d 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 11769ef548d7c..c8ed9742aa03c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/pom.xml b/pom.xml index 5abad0b80e9ec..c1171bc46dd1a 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 1abc41279ebe8..fd37862f7ee88 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "3.0.0.dev0" +__version__ = "3.0.0-preview2" diff --git a/repl/pom.xml b/repl/pom.xml index 3aee53cc9599a..e989926b00355 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 8c46738b259bc..99d672b725249 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 8e1043f77db6d..20a266e7361a0 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 107ba365aa775..ca04ba42473b2 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 0e5df14e060da..1dc573e16c1bd 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 304e3a7457fcc..e803bbacbf0fc 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 0e664eca6a820..1d289f7a58d14 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 75c7f77942396..982f1c8321daf 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c37582386347b..ecad1d103b613 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 87af6388e1118..8ef572f3d1311 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index e380e869f55c7..a28163393dc4b 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.0.0-preview2 ../pom.xml From 56dcd79992ff1eb67ffab337bf0ea68d9641ed2b Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 17 Dec 2019 01:57:27 +0000 Subject: [PATCH 0903/1387] Preparing development version 3.0.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10-token-provider/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graph/api/pom.xml | 2 +- graph/cypher/pom.xml | 2 +- graph/graph/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 42 files changed, 43 insertions(+), 43 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 2d34a0c3e2afa..392afec58382c 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 3.0.0-preview2 +Version: 3.0.1 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 715a1124b87c0..e8a296d002711 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 965c1f37db594..fc1441d80b7bf 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 557bd7a94b0f1..de2a6fb18d69b 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 8c718d84bcf0c..6c0c016fdf94f 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 65f491c5e006f..b8df19152a606 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 2bc7cd84a6081..8119709a6aeff 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 7826c2d16d4b1..96ac525880213 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 3d8b039ebeb00..f385f07f951ce 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d2e09ae75e51b..5774da7f92fd7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 81424e26090c7..db2f6721e0cc4 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.0.0-preview2 -SPARK_VERSION_SHORT: 3.0.0-preview2 +SPARK_VERSION: 3.0.1-SNAPSHOT +SPARK_VERSION_SHORT: 3.0.1 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 5f0564575b510..1b59c629a8c73 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index d5deb746a8a38..87183e72f5de3 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 04cf2e0a820e9..9e669769ec2af 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 55d9f72551211..9426db5e11354 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index e785d7a012f1c..d77364d0210fe 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index ce4abf902b8af..70a3406701315 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index b32d24d399a30..936c342fef028 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index c4b419a12d001..599ae172671e5 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 8e7431b2a1dc6..9fa18d9f82833 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 3fbb2966e7710..acfc2c5a36b1a 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/graph/api/pom.xml b/graph/api/pom.xml index 1d8ae4e46d952..3af3f15bbe820 100644 --- a/graph/api/pom.xml +++ b/graph/api/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 20e91d2342aa2..e803ee084f3a8 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml index 1bec2205e18b8..53327b4033c0c 100644 --- a/graph/graph/pom.xml +++ b/graph/graph/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 2d069806e3c4e..f0666536bb706 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 8375c8e35b91f..48f559e4822f3 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 2950313135ef2..0759ce5c114b3 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 122b20374493d..ed34a370a8274 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c8ed9742aa03c..b8ae344e63e65 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index c1171bc46dd1a..f538ecc39d9b7 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index fd37862f7ee88..906298ff9964a 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "3.0.0-preview2" +__version__ = "3.0.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index e989926b00355..b826b87f53efc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 99d672b725249..9a48bf37b2cd5 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 20a266e7361a0..5206eebad999a 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index ca04ba42473b2..e0c997f869aba 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 1dc573e16c1bd..95299d2066ea3 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index e803bbacbf0fc..6667c7bd089dc 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 1d289f7a58d14..921781ace1783 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 982f1c8321daf..f60820dd45f34 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ecad1d103b613..e8c1598b3669c 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8ef572f3d1311..1e99b6c21522b 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index a28163393dc4b..5430a3e600365 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-preview2 + 3.0.1-SNAPSHOT ../pom.xml From 696288f623671c1f43ecf3f12183f73b8bbe0b2e Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 16 Dec 2019 19:57:44 -0700 Subject: [PATCH 0904/1387] [INFRA] Reverts commit 56dcd79 and c216ef1 ### What changes were proposed in this pull request? 1. Revert "Preparing development version 3.0.1-SNAPSHOT": 56dcd79 2. Revert "Preparing Spark release v3.0.0-preview2-rc2": c216ef1 ### Why are the changes needed? Shouldn't change master. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? manual test: https://github.com/apache/spark/compare/5de5e46..wangyum:revert-master Closes #26915 from wangyum/revert-master. Authored-by: Yuming Wang Signed-off-by: Yuming Wang --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10-token-provider/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graph/api/pom.xml | 2 +- graph/cypher/pom.xml | 2 +- graph/graph/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 42 files changed, 43 insertions(+), 43 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 392afec58382c..95d3e52bef3a9 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 3.0.1 +Version: 3.0.0 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index e8a296d002711..ef916fb99a04c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index fc1441d80b7bf..a1c8a8e6582eb 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index de2a6fb18d69b..163c250054e4d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 6c0c016fdf94f..a6d99813a8501 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index b8df19152a606..76a402bb2bd31 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 8119709a6aeff..3c3c0d2d96a1c 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 96ac525880213..883b73a69c9de 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index f385f07f951ce..93a4f67fd23f2 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 5774da7f92fd7..9d54d21b95ba3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index db2f6721e0cc4..a888620139207 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.0.1-SNAPSHOT -SPARK_VERSION_SHORT: 3.0.1 +SPARK_VERSION: 3.0.0-SNAPSHOT +SPARK_VERSION_SHORT: 3.0.0 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 1b59c629a8c73..a099f1e042e99 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 87183e72f5de3..ba6f20bfdbf58 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 9e669769ec2af..aff79b8b8e642 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 9426db5e11354..f2dcf5d217a89 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index d77364d0210fe..693820da6af6b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index 70a3406701315..01ca96b4f8f40 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 936c342fef028..d11569d709b23 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 599ae172671e5..0ce922349ea66 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 9fa18d9f82833..7d69764b77de7 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index acfc2c5a36b1a..db64b201abc2c 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/api/pom.xml b/graph/api/pom.xml index 3af3f15bbe820..a18c5290195fe 100644 --- a/graph/api/pom.xml +++ b/graph/api/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index e803ee084f3a8..12b3832d9777a 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml index 53327b4033c0c..7438ee5e54fe4 100644 --- a/graph/graph/pom.xml +++ b/graph/graph/pom.xml @@ -23,7 +23,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index f0666536bb706..444568a03d6c7 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 48f559e4822f3..02734e82ed26e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 0759ce5c114b3..e75e8345cd51d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index ed34a370a8274..2eab868ac0dc8 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index b8ae344e63e65..11769ef548d7c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index f538ecc39d9b7..5abad0b80e9ec 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 906298ff9964a..1abc41279ebe8 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "3.0.1.dev0" +__version__ = "3.0.0.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index b826b87f53efc..3aee53cc9599a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 9a48bf37b2cd5..8c46738b259bc 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 5206eebad999a..8e1043f77db6d 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e0c997f869aba..107ba365aa775 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 95299d2066ea3..0e5df14e060da 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6667c7bd089dc..304e3a7457fcc 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 921781ace1783..0e664eca6a820 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index f60820dd45f34..75c7f77942396 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index e8c1598b3669c..c37582386347b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 1e99b6c21522b..87af6388e1118 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 5430a3e600365..e380e869f55c7 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.0.1-SNAPSHOT + 3.0.0-SNAPSHOT ../pom.xml From e75d9afb2f282ce79c9fd8bce031287739326a4f Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 17 Dec 2019 11:13:27 +0800 Subject: [PATCH 0905/1387] [SPARK-30094][SQL] Apply current namespace for the single-part table name ### What changes were proposed in this pull request? This PR applies the current namespace for the single-part table name if the current catalog is a non-session catalog. Note that the reason the current namespace is not applied for the session catalog is that the single-part name could be referencing a temp view which doesn't belong to any namespaces. The empty namespace for a table inside the session catalog is resolved by the session catalog implementation. ### Why are the changes needed? It's fixing the following bug where the current namespace is not respected: ``` sql("CREATE TABLE testcat.ns.t USING foo AS SELECT 1 AS id") sql("USE testcat.ns") sql("SHOW CURRENT NAMESPACE").show +-------+---------+ |catalog|namespace| +-------+---------+ |testcat| ns| +-------+---------+ // `t` is not resolved since the current namespace `ns` is not used. sql("DESCRIBE t").show Failed to analyze query: org.apache.spark.sql.AnalysisException: Table not found: t;; ``` ### Does this PR introduce any user-facing change? Yes, the above `DESCRIBE` command will succeed. ### How was this patch tested? Added tests. Closes #26894 from imback82/current_namespace. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/ResolveCatalogs.scala | 2 +- .../sql/connector/catalog/LookupCatalog.scala | 11 ++++++++++- .../connector/catalog/LookupCatalogSuite.scala | 3 ++- .../catalyst/analysis/ResolveSessionCatalog.scala | 2 +- .../sql/connector/DataSourceV2SQLSuite.scala | 15 +++++++++++++++ .../execution/command/PlanResolutionSuite.scala | 1 + 6 files changed, 30 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 3361173c9962f..4487a9f1b6b8e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -232,7 +232,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) object NonSessionCatalogAndTable { def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { - case CatalogAndIdentifier(catalog, ident) if !isSessionCatalog(catalog) => + case NonSessionCatalogAndIdentifier(catalog, ident) => Some(catalog -> ident.asMultipartIdentifier) case _ => None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index eaded966d9c7d..080ddf1d027e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -103,7 +103,16 @@ private[sql] trait LookupCatalog extends Logging { def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Identifier)] = { assert(nameParts.nonEmpty) if (nameParts.length == 1) { - Some((currentCatalog, Identifier.of(Array(), nameParts.head))) + // If the current catalog is session catalog, the current namespace is not used because + // the single-part name could be referencing a temp view, which doesn't belong to any + // namespaces. An empty namespace will be resolved inside the session catalog + // implementation when a relation is looked up. + val ns = if (CatalogV2Util.isSessionCatalog(currentCatalog)) { + Array.empty[String] + } else { + catalogManager.currentNamespace + } + Some((currentCatalog, Identifier.of(ns, nameParts.head))) } else if (nameParts.head.equalsIgnoreCase(globalTempDB)) { // Conceptually global temp views are in a special reserved catalog. However, the v2 catalog // API does not support view yet, and we have to use v1 commands to deal with global temp diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala index bd467ed878454..b2f27e4740cbe 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/LookupCatalogSuite.scala @@ -116,10 +116,11 @@ class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog wit catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) }) when(manager.currentCatalog).thenReturn(catalogs("prod")) + when(manager.currentNamespace).thenReturn(Array.empty[String]) manager } - test("catalog object identifier") { + test("catalog and identifier") { Seq( ("tbl", catalogs("prod"), Seq.empty, "tbl"), ("db.tbl", catalogs("prod"), Seq("db"), "tbl"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 834e99c265e9d..5abde31466d0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -583,7 +583,7 @@ class ResolveSessionCatalog( object SessionCatalogAndTable { def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, Seq[String])] = nameParts match { - case CatalogAndIdentifier(catalog, ident) if isSessionCatalog(catalog) => + case SessionCatalogAndIdentifier(catalog, ident) => Some(catalog -> ident.asMultipartIdentifier) case _ => None } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 15381e05d7250..b55e1b92cf60a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1914,6 +1914,21 @@ class DataSourceV2SQLSuite } } + test("SPARK-30094: current namespace is used during table resolution") { + // unset this config to use the default v2 session catalog. + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + + withTable("spark_catalog.t", "testcat.ns.t") { + sql("CREATE TABLE t USING parquet AS SELECT 1") + sql("CREATE TABLE testcat.ns.t USING parquet AS SELECT 2") + + checkAnswer(sql("SELECT * FROM t"), Row(1)) + + sql("USE testcat.ns") + checkAnswer(sql("SELECT * FROM t"), Row(2)) + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 49cacbea6dd17..4576c4b25bf52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -111,6 +111,7 @@ class PlanResolutionSuite extends AnalysisTest { } }) when(manager.currentCatalog).thenReturn(testCat) + when(manager.currentNamespace).thenReturn(Array.empty[String]) when(manager.v1SessionCatalog).thenReturn(v1SessionCatalog) manager } From 1da7e8295cfcee15f14274b5c72ebcc997b4f85e Mon Sep 17 00:00:00 2001 From: ulysses Date: Tue, 17 Dec 2019 12:15:53 +0800 Subject: [PATCH 0906/1387] [SPARK-30201][SQL] HiveOutputWriter standardOI should use ObjectInspectorCopyOption.DEFAULT ### What changes were proposed in this pull request? Now spark use `ObjectInspectorCopyOption.JAVA` as oi option which will convert any string to UTF-8 string. When write non UTF-8 code data, then `EFBFBD` will appear. We should use `ObjectInspectorCopyOption.DEFAULT` to support pass the bytes. ### Why are the changes needed? Here is the way to reproduce: 1. make a file contains 16 radix 'AABBCC' which is not the UTF-8 code. 2. create table test1 (c string) location '$file_path'; 3. select hex(c) from test1; // AABBCC 4. craete table test2 (c string) as select c from test1; 5. select hex(c) from test2; // EFBFBDEFBFBDEFBFBD ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Closes #26831 from ulysses-you/SPARK-30201. Authored-by: ulysses Signed-off-by: Wenchen Fan --- .../spark/sql/hive/HiveInspectors.scala | 9 +++++-- .../sql/hive/execution/HiveFileFormat.scala | 7 +++++- .../apache/spark/sql/hive/InsertSuite.scala | 24 +++++++++++++++++++ 3 files changed, 37 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 5b627b8164130..0cd9b3641bd4a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -305,12 +305,17 @@ private[hive] trait HiveInspectors { withNullSafe(o => getByteWritable(o)) case _: ByteObjectInspector => withNullSafe(o => o.asInstanceOf[java.lang.Byte]) - case _: JavaHiveVarcharObjectInspector => + // To spark HiveVarchar and HiveChar are same as string + case _: HiveVarcharObjectInspector if x.preferWritable() => + withNullSafe(o => getStringWritable(o)) + case _: HiveVarcharObjectInspector => withNullSafe { o => val s = o.asInstanceOf[UTF8String].toString new HiveVarchar(s, s.length) } - case _: JavaHiveCharObjectInspector => + case _: HiveCharObjectInspector if x.preferWritable() => + withNullSafe(o => getStringWritable(o)) + case _: HiveCharObjectInspector => withNullSafe { o => val s = o.asInstanceOf[UTF8String].toString new HiveChar(s, s.length) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 27071075b4165..c51c521cacba0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -130,10 +130,15 @@ class HiveOutputWriter( new Path(path), Reporter.NULL) + /** + * Since SPARK-30201 ObjectInspectorCopyOption.JAVA change to ObjectInspectorCopyOption.DEFAULT. + * The reason is DEFAULT option can convert `UTF8String` to `Text` with bytes and + * we can compatible with non UTF-8 code bytes during write. + */ private val standardOI = ObjectInspectorUtils .getStandardObjectInspector( tableDesc.getDeserializer(jobConf).getObjectInspector, - ObjectInspectorCopyOption.JAVA) + ObjectInspectorCopyOption.DEFAULT) .asInstanceOf[StructObjectInspector] private val fieldOIs = diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index d0852b22d1854..421dcb499bd6a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import java.io.File +import com.google.common.io.Files import org.apache.hadoop.fs.Path import org.scalatest.{BeforeAndAfter, PrivateMethodTester} @@ -823,4 +824,27 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } } + + test("SPARK-30201 HiveOutputWriter standardOI should use ObjectInspectorCopyOption.DEFAULT") { + withTable("t1", "t2") { + withTempDir { dir => + val file = new File(dir, "test.hex") + val hex = "AABBCC" + val bs = org.apache.commons.codec.binary.Hex.decodeHex(hex.toCharArray) + Files.write(bs, file) + val path = file.getParent + sql(s"create table t1 (c string) STORED AS TEXTFILE location '$path'") + checkAnswer( + sql("select hex(c) from t1"), + Row(hex) + ) + + sql("create table t2 as select c from t1") + checkAnswer( + sql("select hex(c) from t2"), + Row(hex) + ) + } + } + } } From 1c714befd882ebb3d561d7b660c85a2c48b0802c Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Mon, 16 Dec 2019 21:11:15 -0800 Subject: [PATCH 0907/1387] [SPARK-25100][TEST][FOLLOWUP] Refactor test cases in `FileSuite` and `KryoSerializerSuite` ### What changes were proposed in this pull request? Refactor test cases added by https://github.com/apache/spark/pull/26714, to improve code compactness. ### How was this patch tested? Tested locally. Closes #26916 from jiangxb1987/SPARK-25100. Authored-by: Xingbo Jiang Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/FileSuite.scala | 41 +++++++++++-------- .../serializer/KryoSerializerSuite.scala | 14 +++---- 2 files changed, 30 insertions(+), 25 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index ed11653c810d7..e9ee6b5dfb665 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -702,32 +702,39 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { assert(collectRDDAndDeleteFileBeforeCompute(true).isEmpty) } - test("SPARK-25100: Using KryoSerializer and" + - "setting registrationRequired true can lead job failed") { - val inputFile = new File(tempDir, "/input").getAbsolutePath - val textFileOutputDir = new File(tempDir, "/out1").getAbsolutePath - val dataSetDir = new File(tempDir, "/out2").getAbsolutePath - - Utils.tryWithResource(new PrintWriter(new File(inputFile))) { writer => - for (i <- 1 to 100) { + test("SPARK-25100: Support commit tasks when Kyro registration is required") { + // Prepare the input file + val inputFilePath = new File(tempDir, "/input").getAbsolutePath + Utils.tryWithResource(new PrintWriter(new File(inputFilePath))) { writer => + for (i <- 1 to 3) { writer.print(i) writer.write('\n') } } - val conf = new SparkConf(false).setMaster("local"). - set("spark.kryo.registrationRequired", "true").setAppName("test") - conf.set("spark.serializer", classOf[KryoSerializer].getName) + // Start a new SparkContext + val conf = new SparkConf(false) + .setMaster("local") + .setAppName("test") + .set("spark.kryo.registrationRequired", "true") + .set("spark.serializer", classOf[KryoSerializer].getName) + sc = new SparkContext(conf) + + // Prepare the input RDD + val pairRDD = sc.textFile(inputFilePath).map(x => (x, x)) + // Test saveAsTextFile() + val outputFilePath1 = new File(tempDir, "/out1").getAbsolutePath + pairRDD.saveAsTextFile(outputFilePath1) + assert(sc.textFile(outputFilePath1).collect() === Array("(1,1)", "(2,2)", "(3,3)")) + + // Test saveAsNewAPIHadoopDataset() + val outputFilePath2 = new File(tempDir, "/out2").getAbsolutePath val jobConf = new JobConf() jobConf.setOutputKeyClass(classOf[IntWritable]) jobConf.setOutputValueClass(classOf[IntWritable]) - jobConf.set("mapred.output.dir", dataSetDir) - - sc = new SparkContext(conf) - val pairRDD = sc.textFile(inputFile).map(x => (x, 1)) - - pairRDD.saveAsTextFile(textFileOutputDir) + jobConf.set("mapred.output.dir", outputFilePath2) pairRDD.saveAsNewAPIHadoopDataset(jobConf) + assert(sc.textFile(outputFilePath2).collect() === Array("1\t1", "2\t2", "3\t3")) } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index c55efe9821541..4c47a67ee9ffc 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -363,16 +363,14 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val conf = new SparkConf(false) conf.set(KRYO_REGISTRATION_REQUIRED, true) - val ser = new KryoSerializer(conf).newInstance() - // In HadoopMapReduceCommitProtocol#commitTask - val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map() - addedAbsPathFiles.put("test1", "test1") - addedAbsPathFiles.put("test2", "test2") + // HadoopMapReduceCommitProtocol.commitTask() returns a TaskCommitMessage containing a complex + // structure. - val partitionPaths: mutable.Set[String] = mutable.Set() - partitionPaths.add("test3") + val ser = new KryoSerializer(conf).newInstance() + val addedAbsPathFiles = Map("test1" -> "test1", "test2" -> "test2") + val partitionPaths = Set("test3") - val taskCommitMessage1 = new TaskCommitMessage(addedAbsPathFiles.toMap -> partitionPaths.toSet) + val taskCommitMessage1 = new TaskCommitMessage(addedAbsPathFiles -> partitionPaths) val taskCommitMessage2 = new TaskCommitMessage(Map.empty -> Set.empty) Seq(taskCommitMessage1, taskCommitMessage2).foreach { taskCommitMessage => val obj1 = ser.deserialize[TaskCommitMessage](ser.serialize(taskCommitMessage)).obj From bf7215c510e0eb2afa0a8f1337c535b4091f08f9 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 17 Dec 2019 15:36:21 +0900 Subject: [PATCH 0908/1387] [SPARK-30066][SQL][FOLLOWUP] Remove size field for interval column cache ### What changes were proposed in this pull request? A followup for #26699, clear the size field for interval column cache, which is needless and can reduce the memory cost. ### Why are the changes needed? followup ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing ut. Closes #26906 from yaooqinn/SPARK-30066-f. Authored-by: Kent Yao Signed-off-by: Takeshi Yamamuro --- .../sql/execution/columnar/ColumnType.scala | 19 +++++++++++++------ .../execution/columnar/ColumnTypeSuite.scala | 2 +- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index 5e84e16646338..d3c8e9251cefd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -705,15 +705,12 @@ private[columnar] case class MAP(dataType: MapType) override def clone(v: UnsafeMapData): UnsafeMapData = v.copy() } -private[columnar] object CALENDAR_INTERVAL extends ColumnType[CalendarInterval] - with DirectCopyColumnType[CalendarInterval] { +private[columnar] object CALENDAR_INTERVAL extends ColumnType[CalendarInterval] { override def dataType: DataType = CalendarIntervalType override def defaultSize: Int = 16 - override def actualSize(row: InternalRow, ordinal: Int): Int = 20 - override def getField(row: InternalRow, ordinal: Int): CalendarInterval = row.getInterval(ordinal) override def setField(row: InternalRow, ordinal: Int, value: CalendarInterval): Unit = { @@ -721,15 +718,25 @@ private[columnar] object CALENDAR_INTERVAL extends ColumnType[CalendarInterval] } override def extract(buffer: ByteBuffer): CalendarInterval = { - ByteBufferHelper.getInt(buffer) val months = ByteBufferHelper.getInt(buffer) val days = ByteBufferHelper.getInt(buffer) val microseconds = ByteBufferHelper.getLong(buffer) new CalendarInterval(months, days, microseconds) } + // copy the bytes from ByteBuffer to UnsafeRow + override def extract(buffer: ByteBuffer, row: InternalRow, ordinal: Int): Unit = { + if (row.isInstanceOf[MutableUnsafeRow]) { + val cursor = buffer.position() + buffer.position(cursor + defaultSize) + row.asInstanceOf[MutableUnsafeRow].writer.write(ordinal, buffer.array(), + buffer.arrayOffset() + cursor, defaultSize) + } else { + setField(row, ordinal, extract(buffer)) + } + } + override def append(v: CalendarInterval, buffer: ByteBuffer): Unit = { - ByteBufferHelper.putInt(buffer, 16) ByteBufferHelper.putInt(buffer, v.months) ByteBufferHelper.putInt(buffer, v.days) ByteBufferHelper.putLong(buffer, v.microseconds) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 731db232ed61c..56d5bfbb9675d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -78,7 +78,7 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { checkActualSize(ARRAY_TYPE, Array[Any](1), 4 + 8 + 8 + 8) checkActualSize(MAP_TYPE, Map(1 -> "a"), 4 + (8 + 8 + 8 + 8) + (8 + 8 + 8 + 8)) checkActualSize(STRUCT_TYPE, Row("hello"), 28) - checkActualSize(CALENDAR_INTERVAL, CalendarInterval.MAX_VALUE, 4 + 4 + 4 + 8) + checkActualSize(CALENDAR_INTERVAL, CalendarInterval.MAX_VALUE, 4 + 4 + 8) } testNativeColumnType(BOOLEAN) From 18431c7baaba72539603814ef1757650000943d5 Mon Sep 17 00:00:00 2001 From: Zhenhua Wang Date: Tue, 17 Dec 2019 22:21:26 +0900 Subject: [PATCH 0909/1387] [SPARK-30269][SQL] Should use old partition stats to decide whether to update stats when analyzing partition ### What changes were proposed in this pull request? It's an obvious bug: currently when analyzing partition stats, we use old table stats to compare with newly computed stats to decide whether it should update stats or not. ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? no ### How was this patch tested? add new tests Closes #26908 from wzhfy/failto_update_part_stats. Authored-by: Zhenhua Wang Signed-off-by: HyukjinKwon --- .../command/AnalyzePartitionCommand.scala | 2 +- .../sql/StatisticsCollectionTestBase.scala | 7 ++- .../spark/sql/hive/StatisticsSuite.scala | 52 +++++++++++++++++-- 3 files changed, 54 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala index 18fefa0a6f19f..9ddc470fbd33c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala @@ -110,7 +110,7 @@ case class AnalyzePartitionCommand( val newTotalSize = CommandUtils.calculateLocationSize( sessionState, tableMeta.identifier, p.storage.locationUri) val newRowCount = rowCounts.get(p.spec) - val newStats = CommandUtils.compareAndGetNewStats(tableMeta.stats, newTotalSize, newRowCount) + val newStats = CommandUtils.compareAndGetNewStats(p.stats, newTotalSize, newRowCount) newStats.map(_ => p.copy(stats = newStats)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala index 915f66526c3e6..fde8ddf491bd1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala @@ -27,6 +27,7 @@ import scala.util.Random import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogColumnStat, CatalogStatistics, CatalogTable, HiveTableRelation} +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Histogram, HistogramBin, HistogramSerializer, LogicalPlan} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -238,10 +239,14 @@ abstract class StatisticsCollectionTestBase extends QueryTest with SQLTestUtils getTableFromCatalogCache(tableName) != null } - def getCatalogStatistics(tableName: String): CatalogStatistics = { + def getTableStats(tableName: String): CatalogStatistics = { getCatalogTable(tableName).stats.get } + def getPartitionStats(tableName: String, partSpec: TablePartitionSpec): CatalogStatistics = { + spark.sessionState.catalog.getPartition(TableIdentifier(tableName), partSpec).stats.get + } + def checkTableStats( tableName: String, hasSizeInBytes: Boolean, 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 99a45d6b45b89..488175a22bad7 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 @@ -54,7 +54,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto Seq(dsTbl, hiveTbl).foreach { tbl => sql(s"ANALYZE TABLE $tbl COMPUTE STATISTICS") - val catalogStats = getCatalogStatistics(tbl) + val catalogStats = getTableStats(tbl) withSQLConf(SQLConf.CBO_ENABLED.key -> "false") { val relationStats = spark.table(tbl).queryExecution.optimizedPlan.stats assert(relationStats.sizeInBytes == catalogStats.sizeInBytes) @@ -312,7 +312,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") - assert(getCatalogStatistics(tableName).sizeInBytes === BigInt(17436)) + assert(getTableStats(tableName).sizeInBytes === BigInt(17436)) } } } @@ -353,11 +353,11 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto // Analyze original table - expect 3 partitions sql(s"ANALYZE TABLE $sourceTableName COMPUTE STATISTICS noscan") - assert(getCatalogStatistics(sourceTableName).sizeInBytes === BigInt(3 * 5812)) + assert(getTableStats(sourceTableName).sizeInBytes === BigInt(3 * 5812)) // Analyze partial-copy table - expect only 1 partition sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") - assert(getCatalogStatistics(tableName).sizeInBytes === BigInt(5812)) + assert(getTableStats(tableName).sizeInBytes === BigInt(5812)) } } } @@ -1204,7 +1204,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto assert(tsHistogramProps.size == 1) // Validate histogram after deserialization. - val cs = getCatalogStatistics(tableName).colStats + val cs = getTableStats(tableName).colStats val intHistogram = cs("cint").histogram.get val tsHistogram = cs("ctimestamp").histogram.get assert(intHistogram.bins.length == spark.sessionState.conf.histogramNumBins) @@ -1514,4 +1514,46 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } } } + + test("SPARK-30269 failed to update partition stats if it's equal to table's old stats") { + val tbl = "SPARK_30269" + val ext_tbl = "SPARK_30269_external" + withTempDir { dir => + withTable(tbl, ext_tbl) { + sql(s"CREATE TABLE $tbl (key INT, value STRING, ds STRING) PARTITIONED BY (ds)") + sql( + s""" + | CREATE TABLE $ext_tbl (key INT, value STRING, ds STRING) + | PARTITIONED BY (ds) + | LOCATION '${dir.toURI}' + """.stripMargin) + + Seq(tbl, ext_tbl).foreach { tblName => + sql(s"INSERT INTO $tblName VALUES (1, 'a', '2019-12-13')") + + // analyze table + sql(s"ANALYZE TABLE $tblName COMPUTE STATISTICS NOSCAN") + var tableStats = getTableStats(tblName) + assert(tableStats.sizeInBytes == 601) + assert(tableStats.rowCount.isEmpty) + + sql(s"ANALYZE TABLE $tblName COMPUTE STATISTICS") + tableStats = getTableStats(tblName) + assert(tableStats.sizeInBytes == 601) + assert(tableStats.rowCount.get == 1) + + // analyze a single partition + sql(s"ANALYZE TABLE $tblName PARTITION (ds='2019-12-13') COMPUTE STATISTICS NOSCAN") + var partStats = getPartitionStats(tblName, Map("ds" -> "2019-12-13")) + assert(partStats.sizeInBytes == 601) + assert(partStats.rowCount.isEmpty) + + sql(s"ANALYZE TABLE $tblName PARTITION (ds='2019-12-13') COMPUTE STATISTICS") + partStats = getPartitionStats(tblName, Map("ds" -> "2019-12-13")) + assert(partStats.sizeInBytes == 601) + assert(partStats.rowCount.get == 1) + } + } + } + } } From fac6b9bde82380af4338740eae13e854f59ebb83 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 17 Dec 2019 09:06:23 -0800 Subject: [PATCH 0910/1387] Revert [SPARK-27300][GRAPH] Add Spark Graph modules and dependencies This reverts commit 709387d66003b9f9c847488c101156a71231eddf. See https://issues.apache.org/jira/browse/SPARK-27300?focusedCommentId=16990048&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16990048 and previous mailing list discussions. ### What changes were proposed in this pull request? Revert the addition of skeleton graph API modules for Spark 3.0. ### Why are the changes needed? It does not appear that content will be added to the module for Spark 3, so I propose avoiding committing to the modules, which are no-ops now, in the upcoming major 3.0 release. ### Does this PR introduce any user-facing change? No, the modules were not released. ### How was this patch tested? Existing tests, but mostly N/A. Closes #26928 from srowen/Revert27300. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- LICENSE-binary | 1 - assembly/pom.xml | 5 -- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 1 - dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 1 - dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 1 - graph/api/pom.xml | 55 ------------- graph/cypher/pom.xml | 100 ------------------------ graph/graph/pom.xml | 50 ------------ pom.xml | 4 - project/SparkBuild.scala | 7 +- 10 files changed, 2 insertions(+), 223 deletions(-) delete mode 100644 graph/api/pom.xml delete mode 100644 graph/cypher/pom.xml delete mode 100644 graph/graph/pom.xml diff --git a/LICENSE-binary b/LICENSE-binary index 0b45686f9ba39..e19cc9784b63d 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -393,7 +393,6 @@ org.eclipse.jetty:jetty-util org.eclipse.jetty:jetty-webapp org.eclipse.jetty:jetty-xml org.scala-lang.modules:scala-xml_2.12 -org.opencypher:okapi-shade com.github.joshelser:dropwizard-metrics-hadoop-metrics2-reporter com.zaxxer.HikariCP org.apache.hive:hive-beeline diff --git a/assembly/pom.xml b/assembly/pom.xml index ef916fb99a04c..68ebfadb668ab 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -64,11 +64,6 @@ spark-graphx_${scala.binary.version} ${project.version} - - org.apache.spark - spark-graph_${scala.binary.version} - ${project.version} - org.apache.spark spark-sql_${scala.binary.version} diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 00808893ccc7b..04c6e5f94fdb7 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -157,7 +157,6 @@ metrics-jvm-4.1.1.jar minlog-1.3.0.jar netty-all-4.1.42.Final.jar objenesis-2.5.1.jar -okapi-shade-0.4.2.jar okhttp-3.12.6.jar okio-1.15.0.jar opencsv-2.3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 7d7d4ed447362..300c3ee931471 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -173,7 +173,6 @@ metrics-jvm-4.1.1.jar minlog-1.3.0.jar netty-all-4.1.42.Final.jar objenesis-2.5.1.jar -okapi-shade-0.4.2.jar okhttp-3.12.6.jar okio-1.15.0.jar opencsv-2.3.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 997fa356decd8..aa1a8ccc42e33 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -187,7 +187,6 @@ mssql-jdbc-6.2.1.jre7.jar netty-all-4.1.42.Final.jar nimbus-jose-jwt-4.41.1.jar objenesis-2.5.1.jar -okapi-shade-0.4.2.jar okhttp-2.7.5.jar okhttp-3.12.6.jar okio-1.15.0.jar diff --git a/graph/api/pom.xml b/graph/api/pom.xml deleted file mode 100644 index a18c5290195fe..0000000000000 --- a/graph/api/pom.xml +++ /dev/null @@ -1,55 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.12 - 3.0.0-SNAPSHOT - ../../pom.xml - - - spark-graph-api_2.12 - jar - Spark Project Graph API - - graph-api - - http://spark.apache.org/ - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-sql_${scala.binary.version} - ${project.version} - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml deleted file mode 100644 index 12b3832d9777a..0000000000000 --- a/graph/cypher/pom.xml +++ /dev/null @@ -1,100 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.12 - 3.0.0-SNAPSHOT - ../../pom.xml - - - spark-cypher_2.12 - jar - Spark Project Cypher - - cypher - - http://spark.apache.org/ - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-sql_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-graph-api_${scala.binary.version} - ${project.version} - - - org.opencypher - okapi-shade - ${okapi.version} - - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-sql_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} - test-jar - test - - - org.opencypher - okapi-tck - ${okapi.version} - test - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml deleted file mode 100644 index 7438ee5e54fe4..0000000000000 --- a/graph/graph/pom.xml +++ /dev/null @@ -1,50 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.12 - 3.0.0-SNAPSHOT - ../../pom.xml - - - spark-graph_2.12 - jar - Spark Project Graph - - graph - - http://spark.apache.org/ - - - - org.apache.spark - spark-cypher_${scala.binary.version} - ${project.version} - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/pom.xml b/pom.xml index 5abad0b80e9ec..4e41b82f548be 100644 --- a/pom.xml +++ b/pom.xml @@ -106,9 +106,6 @@ external/kafka-0-10-assembly external/kafka-0-10-sql external/avro - graph/api - graph/cypher - graph/graph @@ -127,7 +124,6 @@ ${hadoop.version} 3.4.14 2.7.1 - 0.4.2 org.apache.hive core diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8c3b5f49bcc3f..4a9382b28444b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -49,15 +49,12 @@ object BuildCommons { val streamingProjects@Seq(streaming, streamingKafka010) = Seq("streaming", "streaming-kafka-0-10").map(ProjectRef(buildLocation, _)) - val graphProjects@Seq(graph, graphApi, cypher) = - Seq("graph", "graph-api", "cypher").map(ProjectRef(buildLocation, _)) - val allProjects@Seq( core, graphx, mllib, mllibLocal, repl, networkCommon, networkShuffle, launcher, unsafe, tags, sketch, kvstore, _* ) = Seq( "core", "graphx", "mllib", "mllib-local", "repl", "network-common", "network-shuffle", "launcher", "unsafe", "tags", "sketch", "kvstore" - ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects ++ graphProjects + ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn, sparkGangliaLgpl, streamingKinesisAsl, @@ -336,7 +333,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags, tokenProviderKafka010, sqlKafka010, kvstore, avro, graph, graphApi, cypher + unsafe, tags, tokenProviderKafka010, sqlKafka010, kvstore, avro ).contains(x) } From 297f406425d410e5c450a9fbe24679b49f00a553 Mon Sep 17 00:00:00 2001 From: Aman Omer Date: Wed, 18 Dec 2019 01:30:28 +0800 Subject: [PATCH 0911/1387] [SPARK-29600][SQL] ArrayContains function may return incorrect result for DecimalType ### What changes were proposed in this pull request? Use `TypeCoercion.findWiderTypeForTwo()` instead of `TypeCoercion.findTightestCommonType()` while preprocessing `inputTypes` in `ArrayContains`. ### Why are the changes needed? `TypeCoercion.findWiderTypeForTwo()` also handles cases for DecimalType. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Test cases to be added. Closes #26811 from amanomer/29600. Authored-by: Aman Omer Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/TypeCoercion.scala | 2 +- .../expressions/collectionOperations.scala | 2 +- .../spark/sql/DataFrameFunctionsSuite.scala | 19 ++++++++++++++++++- 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index dd174ef28f309..f416e8efe17ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -243,7 +243,7 @@ object TypeCoercion { * string. If the wider decimal type exceeds system limitation, this rule will truncate * the decimal type before return it. */ - private[analysis] def findWiderTypeWithoutStringPromotionForTwo( + private[catalyst] def findWiderTypeWithoutStringPromotionForTwo( t1: DataType, t2: DataType): Option[DataType] = { findTightestCommonType(t1, t2) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index d5d42510842ed..6ed68e47ce7a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -1081,7 +1081,7 @@ case class ArrayContains(left: Expression, right: Expression) (left.dataType, right.dataType) match { case (_, NullType) => Seq.empty case (ArrayType(e1, hasNull), e2) => - TypeCoercion.findTightestCommonType(e1, e2) match { + TypeCoercion.findWiderTypeWithoutStringPromotionForTwo(e1, e2) match { case Some(dt) => Seq(ArrayType(dt, hasNull), dt) case _ => Seq.empty } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index a346377cd1bc1..584768eff700b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -850,7 +850,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { val errorMsg1 = s""" |Input to function array_contains should have been array followed by a - |value with same element type, but it's [array, decimal(29,29)]. + |value with same element type, but it's [array, decimal(38,29)]. """.stripMargin.replace("\n", " ").trim() assert(e1.message.contains(errorMsg1)) @@ -865,6 +865,23 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { assert(e2.message.contains(errorMsg2)) } + test("SPARK-29600: ArrayContains function may return incorrect result for DecimalType") { + checkAnswer( + sql("select array_contains(array(1.10), 1.1)"), + Seq(Row(true)) + ) + + checkAnswer( + sql("SELECT array_contains(array(1.1), 1.10)"), + Seq(Row(true)) + ) + + checkAnswer( + sql("SELECT array_contains(array(1.11), 1.1)"), + Seq(Row(false)) + ) + } + test("arrays_overlap function") { val df = Seq( (Seq[Option[Int]](Some(1), Some(2)), Seq[Option[Int]](Some(-1), Some(10))), From cdc8fc6233450ed040f2f0272d06510c1eedbefb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 17 Dec 2019 10:32:15 -0800 Subject: [PATCH 0912/1387] [SPARK-30235][CORE] Switching off host local disk reading of shuffle blocks in case of useOldFetchProtocol MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? When `spark.shuffle.useOldFetchProtocol` is enabled then switching off the direct disk reading of host-local shuffle blocks and falling back to remote block fetching (and this way avoiding the `GetLocalDirsForExecutors` block transfer message which is introduced from Spark 3.0.0). ### Why are the changes needed? In `[SPARK-27651][Core] Avoid the network when shuffle blocks are fetched from the same host` a new block transfer message is introduced, `GetLocalDirsForExecutors`. This new message could be sent to the external shuffle service and as it is not supported by the previous version of external shuffle service it should be avoided when `spark.shuffle.useOldFetchProtocol` is true. In the migration guide I changed the exception type as `org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Decoder#fromByteBuffer` throws a IllegalArgumentException with the given text and uses the message type which is just a simple number (byte). I have checked and this is true for version 2.4.4 too. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? This specific case (considering one extra boolean to switch off host local disk reading feature) is not tested but existing tests were run. Closes #26869 from attilapiros/SPARK-30235. Authored-by: “attilapiros” Signed-off-by: Marcelo Vanzin --- .../apache/spark/internal/config/package.scala | 16 ++++++++-------- .../org/apache/spark/storage/BlockManager.scala | 3 ++- docs/core-migration-guide.md | 2 ++ docs/sql-migration-guide.md | 2 -- 4 files changed, 12 insertions(+), 11 deletions(-) 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 9d7b31aa30f0d..2078965e88246 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 @@ -1095,14 +1095,6 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED = - ConfigBuilder("spark.shuffle.readHostLocalDisk.enabled") - .doc("If enabled, shuffle blocks requested from those block managers which are running on " + - "the same host are read from the disk directly instead of being fetched as remote blocks " + - "over the network.") - .booleanConf - .createWithDefault(true) - private[spark] val STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE = ConfigBuilder("spark.storage.localDiskByExecutors.cacheSize") .doc("The max number of executors for which the local dirs are stored. This size is " + @@ -1148,6 +1140,14 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED = + ConfigBuilder("spark.shuffle.readHostLocalDisk.enabled") + .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled), shuffle " + + "blocks requested from those block managers which are running on the same host are read " + + "from the disk directly instead of being fetched as remote blocks over the network.") + .booleanConf + .createWithDefault(true) + private[spark] val MEMORY_MAP_LIMIT_FOR_TESTS = ConfigBuilder("spark.storage.memoryMapLimitForTests") .internal() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index cc28f9b77da30..c47901314f53a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -478,7 +478,8 @@ private[spark] class BlockManager( } hostLocalDirManager = - if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED)) { + if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) && + !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { externalBlockStoreClient.map { blockStoreClient => new HostLocalDirManager( futureExecutionContext, diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 17d071d0779b2..fdb0afad6af9b 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -36,3 +36,5 @@ license: | - Deprecated method `AccumulableInfo.apply` have been removed because creating `AccumulableInfo` is disallowed. - Event log file will be written as UTF-8 encoding, and Spark History Server will replay event log files as UTF-8 encoding. Previously Spark writes event log file as default charset of driver JVM process, so Spark History Server of Spark 2.x is needed to read the old event log files in case of incompatible encoding. + +- A new protocol for fetching shuffle blocks is used. It's recommended that external shuffle services be upgraded when running Spark 3.0 apps. Old external shuffle services can still be used by setting the configuration `spark.shuffle.useOldFetchProtocol` to `true`. Otherwise, Spark may run into errors with messages like `IllegalArgumentException: Unexpected message type: `. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index ca78f3ca46ea5..73a4d19a1b15f 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -97,8 +97,6 @@ license: | - Since Spark 3.0, when Avro files are written with user provided non-nullable schema, even the catalyst schema is nullable, Spark is still able to write the files. However, Spark will throw runtime NPE if any of the records contains null. - - Since Spark 3.0, we use a new protocol for fetching shuffle blocks, for external shuffle service users, we need to upgrade the server correspondingly. Otherwise, we'll get the error message `UnsupportedOperationException: Unexpected message: FetchShuffleBlocks`. If it is hard to upgrade the shuffle service right now, you can still use the old protocol by setting `spark.shuffle.useOldFetchProtocol` to `true`. - - Since Spark 3.0, a higher-order function `exists` follows the three-valued boolean logic, i.e., if the `predicate` returns any `null`s and no `true` is obtained, then `exists` will return `null` instead of `false`. For example, `exists(array(1, null, 3), x -> x % 2 == 0)` will be `null`. The previous behaviour can be restored by setting `spark.sql.legacy.arrayExistsFollowsThreeValuedLogic` to `false`. - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. From b2baaa2fccceaa69f69a76f534cfbc50e6471cbe Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 17 Dec 2019 11:37:05 -0800 Subject: [PATCH 0913/1387] [SPARK-30274][CORE] Avoid BytesToBytesMap lookup hang forever when holding keys reaching max capacity ### What changes were proposed in this pull request? We should not append keys to BytesToBytesMap to be its max capacity. ### Why are the changes needed? BytesToBytesMap.append allows to append keys until the number of keys reaches MAX_CAPACITY. But once the the pointer array in the map holds MAX_CAPACITY keys, next time call of lookup will hang forever. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually test by: ```java Test public void testCapacity() { TestMemoryManager memoryManager2 = new TestMemoryManager( new SparkConf() .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), true) .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 25600 * 1024 * 1024L) .set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false) .set(package$.MODULE$.SHUFFLE_COMPRESS(), false)); TaskMemoryManager taskMemoryManager2 = new TaskMemoryManager(memoryManager2, 0); final long pageSizeBytes = 8000000 + 8; // 8 bytes for end-of-page marker final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager2, 1024, pageSizeBytes); try { for (long i = 0; i < BytesToBytesMap.MAX_CAPACITY + 1; i++) { final long[] value = new long[]{i}; boolean succeed = map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).append( value, Platform.LONG_ARRAY_OFFSET, 8, value, Platform.LONG_ARRAY_OFFSET, 8); } map.free(); } finally { map.free(); } } ``` Once the map was appended to 536870912 keys (MAX_CAPACITY), the next lookup will hang. Closes #26914 from viirya/fix-bytemap2. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../java/org/apache/spark/unsafe/map/BytesToBytesMap.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 7590b72112c54..7bdd89488d119 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -694,7 +694,10 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff assert (vlen % 8 == 0); assert (longArray != null); - if (numKeys == MAX_CAPACITY + // We should not increase number of keys to be MAX_CAPACITY. The usage pattern of this map is + // lookup + append. If we append key until the number of keys to be MAX_CAPACITY, next time + // the call of lookup will hang forever because it cannot find an empty slot. + if (numKeys == MAX_CAPACITY - 1 // The map could be reused from last spill (because of no enough memory to grow), // then we don't try to grow again if hit the `growthThreshold`. || !canGrowArray && numKeys >= growthThreshold) { From cc7f1eb874a7efaf5938def9b5e5577cadfc640d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 18 Dec 2019 12:36:41 +0800 Subject: [PATCH 0914/1387] [SPARK-29774][SQL][FOLLOWUP] Add a migration guide for date_add and date_sub ### What changes were proposed in this pull request? add a migration guide for date_add and date_sub to indicates their behavior change. It a followup for #26412 ### Why are the changes needed? add a migration guide ### Does this PR introduce any user-facing change? yes, doc change ### How was this patch tested? no Closes #26932 from yaooqinn/SPARK-29774-f. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 73a4d19a1b15f..9825e2ff48933 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -256,6 +256,8 @@ license: | - Since Spark 3.0, day-time interval strings are converted to intervals with respect to the `from` and `to` bounds. If an input string does not match to the pattern defined by specified bounds, the `ParseException` exception is thrown. For example, `interval '2 10:20' hour to minute` raises the exception because the expected format is `[+|-]h[h]:[m]m`. In Spark version 2.4, the `from` bound was not taken into account, and the `to` bound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted to `interval 10 hours 20 minutes`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.fromDayTimeString.enabled` to `true`. + - Since Spark 3.0, the `date_add` and `date_sub` functions only accepts int, smallint, tinyint as the 2nd argument, fractional and string types are not valid anymore, e.g. `date_add(cast('1964-05-23' as date), '12.34')` will cause `AnalysisException`. In Spark version 2.4 and earlier, if the 2nd argument is fractional or string value, it will be coerced to int value, and the result will be a date value of `1964-06-04`. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was From d38f8167483d4d79e8360f24a8c0bffd51460659 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 18 Dec 2019 15:25:40 +0900 Subject: [PATCH 0915/1387] [MINOR][SQL][DOC] Fix some format issues in Dataset API Doc ### What changes were proposed in this pull request? fix listing up format issues in Dataset API Doc (scala & java) ### Why are the changes needed? improve doc ### Does this PR introduce any user-facing change? yes, API doc changing ### How was this patch tested? no Closes #26922 from yaooqinn/datasetdoc. Authored-by: Kent Yao Signed-off-by: HyukjinKwon --- .../scala/org/apache/spark/sql/Dataset.scala | 50 +++++++++++-------- 1 file changed, 28 insertions(+), 22 deletions(-) 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 77b3a2c40baaa..52012be32e9e2 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 @@ -450,12 +450,14 @@ class Dataset[T] private[sql]( /** * Returns a new Dataset where each record has been mapped on to the specified type. The * method used to map columns depend on the type of `U`: - * - When `U` is a class, fields for the class will be mapped to columns of the same name - * (case sensitivity is determined by `spark.sql.caseSensitive`). - * - When `U` is a tuple, the columns will be mapped by ordinal (i.e. the first column will - * be assigned to `_1`). - * - When `U` is a primitive type (i.e. String, Int, etc), then the first column of the - * `DataFrame` will be used. + *
      + *
    • When `U` is a class, fields for the class will be mapped to columns of the same name + * (case sensitivity is determined by `spark.sql.caseSensitive`).
    • + *
    • When `U` is a tuple, the columns will be mapped by ordinal (i.e. the first column will + * be assigned to `_1`).
    • + *
    • When `U` is a primitive type (i.e. String, Int, etc), then the first column of the + * `DataFrame` will be used.
    • + *
    * * If the schema of the Dataset does not match the desired `U` type, you can use `select` * along with `alias` or `as` to rearrange or rename as required. @@ -718,11 +720,12 @@ class Dataset[T] private[sql]( * before which we assume no more late data is going to arrive. * * Spark will use this watermark for several purposes: - * - To know when a given time window aggregation can be finalized and thus can be emitted when - * using output modes that do not allow updates. - * - To minimize the amount of state that we need to keep for on-going aggregations, - * `mapGroupsWithState` and `dropDuplicates` operators. - * + *
      + *
    • To know when a given time window aggregation can be finalized and thus can be emitted + * when using output modes that do not allow updates.
    • + *
    • To minimize the amount of state that we need to keep for on-going aggregations, + * `mapGroupsWithState` and `dropDuplicates` operators.
    • + *
    * The current watermark is computed by looking at the `MAX(eventTime)` seen across * all of the partitions in the query minus a user specified `delayThreshold`. Due to the cost * of coordinating this value across partitions, the actual watermark used is only guaranteed @@ -1868,12 +1871,14 @@ class Dataset[T] private[sql]( /** * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset * that returns the same result as the input, with the following guarantees: - * - It will compute the defined aggregates (metrics) on all the data that is flowing through the - * Dataset at that point. - * - It will report the value of the defined aggregate columns as soon as we reach a completion + *
      + *
    • It will compute the defined aggregates (metrics) on all the data that is flowing through + * the Dataset at that point.
    • + *
    • It will report the value of the defined aggregate columns as soon as we reach a completion * point. A completion point is either the end of a query (batch mode) or the end of a streaming * epoch. The value of the aggregates only reflects the data processed since the previous - * completion point. + * completion point.
    • + *
    * Please note that continuous execution is currently not supported. * * The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or @@ -2511,13 +2516,14 @@ class Dataset[T] private[sql]( /** * Computes specified statistics for numeric and string columns. Available statistics are: - * - * - count - * - mean - * - stddev - * - min - * - max - * - arbitrary approximate percentiles specified as a percentage (eg, 75%) + *
      + *
    • count
    • + *
    • mean
    • + *
    • stddev
    • + *
    • min
    • + *
    • max
    • + *
    • arbitrary approximate percentiles specified as a percentage (e.g. 75%)
    • + *
    * * If no statistics are given, this function computes count, mean, stddev, min, * approximate quartiles (percentiles at 25%, 50%, and 75%), and max. From f15eee18cc1f37babcda161d705900e9a94e57ea Mon Sep 17 00:00:00 2001 From: Jalpan Randeri Date: Wed, 18 Dec 2019 23:59:27 +0900 Subject: [PATCH 0916/1387] [SPARK-29493][SQL] Arrow MapType support ### What changes were proposed in this pull request? This pull request add support for Arrow MapType into Spark SQL. ### Why are the changes needed? Without this change User's of spark are not able to query data in spark if one of columns is stored as map and Apache Arrow execution mode is preferred by user. More info: https://issues.apache.org/jira/projects/SPARK/issues/SPARK-29493 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Introduced few unit tests around map type in existing arrow test suit Closes #26512 from jalpan-randeri/feature-arrow-java-map-type. Authored-by: Jalpan Randeri Signed-off-by: HyukjinKwon --- .../sql/vectorized/ArrowColumnVector.java | 32 ++++- .../apache/spark/sql/util/ArrowUtils.scala | 16 +++ .../sql/execution/arrow/ArrowWriter.scala | 40 ++++++ .../arrow/ArrowConvertersSuite.scala | 8 +- .../execution/arrow/ArrowWriterSuite.scala | 120 +++++++++++++++++- .../apache/spark/sql/test/SQLTestData.scala | 9 ++ 6 files changed, 217 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 541818331a0bd..d2220dc2668f9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -123,7 +123,8 @@ public ColumnarArray getArray(int rowId) { @Override public ColumnarMap getMap(int rowId) { - throw new UnsupportedOperationException(); + if (isNullAt(rowId)) return null; + return accessor.getMap(rowId); } @Override @@ -156,6 +157,9 @@ public ArrowColumnVector(ValueVector vector) { accessor = new DateAccessor((DateDayVector) vector); } else if (vector instanceof TimeStampMicroTZVector) { accessor = new TimestampAccessor((TimeStampMicroTZVector) vector); + } else if (vector instanceof MapVector) { + MapVector mapVector = (MapVector) vector; + accessor = new MapAccessor(mapVector); } else if (vector instanceof ListVector) { ListVector listVector = (ListVector) vector; accessor = new ArrayAccessor(listVector); @@ -236,6 +240,10 @@ byte[] getBinary(int rowId) { ColumnarArray getArray(int rowId) { throw new UnsupportedOperationException(); } + + ColumnarMap getMap(int rowId) { + throw new UnsupportedOperationException(); + } } private static class BooleanAccessor extends ArrowVectorAccessor { @@ -472,4 +480,26 @@ private static class StructAccessor extends ArrowVectorAccessor { super(vector); } } + + private static class MapAccessor extends ArrowVectorAccessor { + private final MapVector accessor; + private final ArrowColumnVector keys; + private final ArrowColumnVector values; + + MapAccessor(MapVector vector) { + super(vector); + this.accessor = vector; + StructVector entries = (StructVector) vector.getDataVector(); + this.keys = new ArrowColumnVector(entries.getChild(MapVector.KEY_NAME)); + this.values = new ArrowColumnVector(entries.getChild(MapVector.VALUE_NAME)); + } + + @Override + final ColumnarMap getMap(int rowId) { + int index = rowId * MapVector.OFFSET_WIDTH; + int offset = accessor.getOffsetBuffer().getInt(index); + int length = accessor.getInnerValueCountAt(rowId); + return new ColumnarMap(keys, values, offset, length); + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index 62546a322d3c9..317eb635ba412 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.util import scala.collection.JavaConverters._ import org.apache.arrow.memory.RootAllocator +import org.apache.arrow.vector.complex.MapVector import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, TimeUnit} import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} @@ -88,6 +89,16 @@ object ArrowUtils { fields.map { field => toArrowField(field.name, field.dataType, field.nullable, timeZoneId) }.toSeq.asJava) + case MapType(keyType, valueType, valueContainsNull) => + val mapType = new FieldType(nullable, new ArrowType.Map(false), null) + // Note: Map Type struct can not be null, Struct Type key field can not be null + new Field(name, mapType, + Seq(toArrowField(MapVector.DATA_VECTOR_NAME, + new StructType() + .add(MapVector.KEY_NAME, keyType, nullable = false) + .add(MapVector.VALUE_NAME, valueType, nullable = valueContainsNull), + nullable = false, + timeZoneId)).asJava) case dataType => val fieldType = new FieldType(nullable, toArrowType(dataType, timeZoneId), null) new Field(name, fieldType, Seq.empty[Field].asJava) @@ -96,6 +107,11 @@ object ArrowUtils { def fromArrowField(field: Field): DataType = { field.getType match { + case _: ArrowType.Map => + val elementField = field.getChildren.get(0) + val keyType = fromArrowField(elementField.getChildren.get(0)) + val valueType = fromArrowField(elementField.getChildren.get(1)) + MapType(keyType, valueType, elementField.getChildren.get(1).isNullable) case ArrowType.List.INSTANCE => val elementField = field.getChildren().get(0) val elementType = fromArrowField(elementField) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala index 6147d6fefd52a..501e1c460f9c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala @@ -62,6 +62,11 @@ object ArrowWriter { case (ArrayType(_, _), vector: ListVector) => val elementVector = createFieldWriter(vector.getDataVector()) new ArrayWriter(vector, elementVector) + case (MapType(_, _, _), vector: MapVector) => + val entryWriter = createFieldWriter(vector.getDataVector).asInstanceOf[StructWriter] + val keyWriter = createFieldWriter(entryWriter.valueVector.getChild(MapVector.KEY_NAME)) + val valueWriter = createFieldWriter(entryWriter.valueVector.getChild(MapVector.VALUE_NAME)) + new MapWriter(vector, keyWriter, valueWriter) case (StructType(_), vector: StructVector) => val children = (0 until vector.size()).map { ordinal => createFieldWriter(vector.getChildByOrdinal(ordinal)) @@ -343,3 +348,38 @@ private[arrow] class StructWriter( children.foreach(_.reset()) } } + +private[arrow] class MapWriter( + val valueVector: MapVector, + val keyWriter: ArrowFieldWriter, + val valueWriter: ArrowFieldWriter) extends ArrowFieldWriter { + + override def setNull(): Unit = {} + + override def setValue(input: SpecializedGetters, ordinal: Int): Unit = { + val map = input.getMap(ordinal) + valueVector.startNewValue(count) + val keys = map.keyArray() + val values = map.valueArray() + var i = 0 + while (i < map.numElements()) { + keyWriter.write(keys, i) + valueWriter.write(values, i) + i += 1 + } + + valueVector.endValue(count, map.numElements()) + } + + override def finish(): Unit = { + super.finish() + keyWriter.finish() + valueWriter.finish() + } + + override def reset(): Unit = { + super.reset() + keyWriter.reset() + valueWriter.reset() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala index 2eb4ac52aca90..fdb23d5be78a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala @@ -1210,15 +1210,13 @@ class ArrowConvertersSuite extends SharedSparkSession { testQuietly("unsupported types") { def runUnsupported(block: => Unit): Unit = { - val msg = intercept[SparkException] { + val msg = intercept[UnsupportedOperationException] { block } - assert(msg.getMessage.contains("Unsupported data type")) - assert(msg.getCause.getClass === classOf[UnsupportedOperationException]) + assert(msg.getMessage.contains("is not supported")) } - runUnsupported { mapData.toDF().toArrowBatchRdd.collect() } - runUnsupported { complexData.toArrowBatchRdd.collect() } + runUnsupported { calenderIntervalData.toDF().toArrowBatchRdd.collect() } } test("test Arrow Validator") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala index 92506032ab2e5..bdc3b5eed7d8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.arrow import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized.ArrowColumnVector +import org.apache.spark.sql.vectorized._ import org.apache.spark.unsafe.types.UTF8String class ArrowWriterSuite extends SparkFunSuite { @@ -267,4 +267,120 @@ class ArrowWriterSuite extends SparkFunSuite { writer.root.close() } + + test("map") { + val schema = new StructType() + .add("map", MapType(IntegerType, StringType), nullable = true) + val writer = ArrowWriter.create(schema, null) + assert(writer.schema == schema) + + writer.write(InternalRow(ArrayBasedMapData( + keys = Array(1, 2, 3), + values = Array( + UTF8String.fromString("v2"), + UTF8String.fromString("v3"), + UTF8String.fromString("v4") + ) + ))) + writer.write(InternalRow(ArrayBasedMapData(Array(43), + Array(UTF8String.fromString("v5")) + ))) + writer.write(InternalRow(ArrayBasedMapData(Array(43), Array(null)))) + writer.write(InternalRow(null)) + + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors.get(0)) + val map0 = reader.getMap(0) + assert(map0.numElements() == 3) + assert(map0.keyArray().array().mkString(",") == Array(1, 2, 3).mkString(",")) + assert(map0.valueArray().array().mkString(",") == Array("v2", "v3", "v4").mkString(",")) + + val map1 = reader.getMap(1) + assert(map1.numElements() == 1) + assert(map1.keyArray().array().mkString(",") == Array(43).mkString(",")) + assert(map1.valueArray().array().mkString(",") == Array("v5").mkString(",")) + + val map2 = reader.getMap(2) + assert(map2.numElements() == 1) + assert(map2.keyArray().array().mkString(",") == Array(43).mkString(",")) + assert(map2.valueArray().array().mkString(",") == Array(null).mkString(",")) + + val map3 = reader.getMap(3) + assert(map3 == null) + writer.root.close() + } + + test("empty map") { + val schema = new StructType() + .add("map", MapType(IntegerType, StringType), nullable = true) + val writer = ArrowWriter.create(schema, null) + assert(writer.schema == schema) + writer.write(InternalRow(ArrayBasedMapData(Array(), Array()))) + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors.get(0)) + + val map0 = reader.getMap(0) + assert(map0.numElements() == 0) + writer.root.close() + } + + test("nested map") { + val valueSchema = new StructType() + .add("name", StringType) + .add("age", IntegerType) + + val schema = new StructType() + .add("map", + MapType( + keyType = IntegerType, + valueType = valueSchema + ), + nullable = true) + val writer = ArrowWriter.create(schema, null) + assert(writer.schema == schema) + + writer.write(InternalRow( + ArrayBasedMapData( + keys = Array(1), + values = Array(InternalRow(UTF8String.fromString("jon"), 20)) + ))) + + writer.write(InternalRow( + ArrayBasedMapData( + keys = Array(1), + values = Array(InternalRow(UTF8String.fromString("alice"), 30)) + ))) + + writer.write(InternalRow( + ArrayBasedMapData( + keys = Array(1), + values = Array(InternalRow(UTF8String.fromString("bob"), 40)) + ))) + + + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors.get(0)) + + def stringRepr(map: ColumnarMap): String = { + map.valueArray().getStruct(0, 2).toSeq(valueSchema).mkString(",") + } + + val map0 = reader.getMap(0) + assert(map0.numElements() == 1) + assert(map0.keyArray().array().mkString(",") == Array(1).mkString(",")) + assert(stringRepr(map0) == Array("jon", "20").mkString(",")) + + val map1 = reader.getMap(1) + assert(map1.numElements() == 1) + assert(map1.keyArray().array().mkString(",") == Array(1).mkString(",")) + assert(stringRepr(map1) == Array("alice", "30").mkString(",")) + + val map2 = reader.getMap(2) + assert(map2.numElements() == 1) + assert(map2.keyArray().array().mkString(",") == Array(1).mkString(",")) + assert(stringRepr(map2) == Array("bob", "40").mkString(",")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 615923fe02d6c..c51faaf10f5dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext, SQLImplicits} +import org.apache.spark.unsafe.types.CalendarInterval /** * A collection of sample data used in SQL tests. @@ -168,6 +169,13 @@ private[sql] trait SQLTestData { self => rdd } + protected lazy val calenderIntervalData: RDD[IntervalData] = { + val rdd = spark.sparkContext.parallelize( + IntervalData(new CalendarInterval(1, 1, 1)) :: Nil) + rdd.toDF().createOrReplaceTempView("calenderIntervalData") + rdd + } + protected lazy val repeatedData: RDD[StringData] = { val rdd = spark.sparkContext.parallelize(List.fill(2)(StringData("test"))) rdd.toDF().createOrReplaceTempView("repeatedData") @@ -335,4 +343,5 @@ private[sql] object SQLTestData { case class ComplexData(m: Map[String, Int], s: TestData, a: Seq[Int], b: Boolean) case class CourseSales(course: String, year: Int, earnings: Double) case class TrainingSales(training: String, sales: CourseSales) + case class IntervalData(data: CalendarInterval) } From 094563384478a402c36415edf04ee7b884a34fc9 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 18 Dec 2019 10:27:31 -0800 Subject: [PATCH 0917/1387] [SPARK-29997][WEBUI][FOLLOWUP] Refactor code for job description of empty jobs ### What changes were proposed in this pull request? Refactor the code brought by #26637 . No more dummy StageInfo and its side-effects are needed at all. This change also enable users to set job description to empty jobs though. ### Why are the changes needed? The previous approach introduced dummy StageInfo and this causes side-effects. ### Does this PR introduce any user-facing change? Yes. Description set by user will be shown in the AllJobsPage. ![](https://user-images.githubusercontent.com/4736016/70788638-acf17900-1dd4-11ea-95f9-6d6739b24083.png) ### How was this patch tested? Manual test and newly added unit test. Closes #26703 from sarutak/fix-ui-for-empty-job2. Lead-authored-by: Kousuke Saruta Co-authored-by: Dongjoon Hyun Co-authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../apache/spark/scheduler/DAGScheduler.scala | 16 +++++----------- .../org/apache/spark/scheduler/StageInfo.scala | 3 --- .../spark/status/AppStatusListener.scala | 3 +-- .../org/apache/spark/ui/jobs/AllJobsPage.scala | 17 ++++++++++------- .../spark/scheduler/DAGSchedulerSuite.scala | 5 +++++ .../org/apache/spark/ui/UISeleniumSuite.scala | 18 +++++++++++++++++- 6 files changed, 38 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 851419132ff2e..7bf363dd71c1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -695,19 +695,13 @@ private[spark] class DAGScheduler( val jobId = nextJobId.getAndIncrement() if (partitions.isEmpty) { + val clonedProperties = Utils.cloneProperties(properties) + if (sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) == null) { + clonedProperties.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, callSite.shortForm) + } val time = clock.getTimeMillis() - val dummyStageInfo = - new StageInfo( - StageInfo.INVALID_STAGE_ID, - StageInfo.INVALID_ATTEMPT_ID, - callSite.shortForm, - 0, - Seq.empty[RDDInfo], - Seq.empty[Int], - "") listenerBus.post( - SparkListenerJobStart( - jobId, time, Seq[StageInfo](dummyStageInfo), Utils.cloneProperties(properties))) + SparkListenerJobStart(jobId, time, Seq.empty, clonedProperties)) listenerBus.post( SparkListenerJobEnd(jobId, time, JobSucceeded)) // Return immediately if the job is running 0 tasks diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index c2cb95771b27c..fdc50328b43d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -107,7 +107,4 @@ private[spark] object StageInfo { taskLocalityPreferences, shuffleDepId) } - - final val INVALID_STAGE_ID = -1 - final val INVALID_ATTEMPT_ID = -1 } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index df140ba7d1547..a5850fc2ac4b9 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -414,8 +414,7 @@ private[spark] class AppStatusListener( val e = it.next() if (job.stageIds.contains(e.getKey()._1)) { val stage = e.getValue() - // If a stage has no partitions(tasks), the stage should not be marked as skipped. - if (v1.StageStatus.PENDING.equals(stage.status) && stage.info.numTasks > 0) { + if (v1.StageStatus.PENDING.equals(stage.status)) { stage.status = v1.StageStatus.SKIPPED job.skippedStages += stage.info.stageId job.skippedTasks += stage.info.numTasks diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 6300e639633c5..f53e67ff5cc98 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -71,7 +71,10 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We val jobId = job.jobId val status = job.status val (_, lastStageDescription) = lastStageNameAndDescription(store, job) - val jobDescription = UIUtils.makeDescription(lastStageDescription, "", plainText = true).text + val jobDescription = UIUtils.makeDescription( + job.description.getOrElse(lastStageDescription), + "", + plainText = true).text val submissionTime = job.submissionTime.get.getTime() val completionTime = job.completionTime.map(_.getTime()).getOrElse(System.currentTimeMillis()) @@ -449,7 +452,11 @@ private[ui] class JobDataSource( val formattedSubmissionTime = submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") val (lastStageName, lastStageDescription) = lastStageNameAndDescription(store, jobData) - val jobDescription = UIUtils.makeDescription(lastStageDescription, basePath, plainText = false) + val jobDescription = + UIUtils.makeDescription( + jobData.description.getOrElse(lastStageDescription), + basePath, + plainText = false) val detailUrl = "%s/jobs/job/?id=%s".format(basePath, jobData.jobId) @@ -660,11 +667,7 @@ private[ui] class JobPagedTable( {jobTableRow.formattedDuration} - {job.numCompletedStages}/{ - // A job contains at least 1 stage but if a job has no partitions(tasks), - // the stage is not submitted so the total stage should be regarded as 0. - if (job.numTasks > 0) job.stageIds.size - job.numSkippedStages else 0 - } + {job.numCompletedStages}/{job.stageIds.size - job.numSkippedStages} {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 1e3b59f7e97d8..101e60c73e9f8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -826,6 +826,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val listener = new SparkListener() { override def onJobStart(event: SparkListenerJobStart): Unit = { try { + // spark.job.description can be implicitly set for 0 partition jobs. + // So event.properties and properties can be different. See SPARK-29997. + event.properties.remove(SparkContext.SPARK_JOB_DESCRIPTION) + properties.remove(SparkContext.SPARK_JOB_DESCRIPTION) + assert(event.properties.equals(properties), "Expected same content of properties, " + s"but got properties with different content. props in caller ${properties} /" + s" props in event ${event.properties}") diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index f0736348940ca..9f0cdeac9ca39 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui -import java.net.{HttpURLConnection, URL} +import java.net.URL import java.util.Locale import javax.servlet.http.{HttpServletRequest, HttpServletResponse} @@ -756,6 +756,22 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } } + test("description for empty jobs") { + withSpark(newSparkContext()) { sc => + sc.emptyRDD[Int].collect + val description = "This is my job" + sc.setJobDescription(description) + sc.emptyRDD[Int].collect + + eventually(timeout(10.seconds), interval(50.milliseconds)) { + goToUi(sc, "/jobs") + val descriptions = findAll(className("description-input")).toArray + descriptions(0).text should be (description) + descriptions(1).text should include ("collect") + } + } + } + def goToUi(sc: SparkContext, path: String): Unit = { goToUi(sc.ui.get, path) } From abfc267f0cc38d792f68923946a83877d07dee27 Mon Sep 17 00:00:00 2001 From: chenliang Date: Wed, 18 Dec 2019 15:12:32 -0800 Subject: [PATCH 0918/1387] [SPARK-30262][SQL] Avoid NumberFormatException when totalSize is empty MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? We could get the Partitions Statistics Info.But in some specail case, The Info like totalSize,rawDataSize,rowCount maybe empty. When we do some ddls like `desc formatted partition` ,the NumberFormatException is showed as below: ``` spark-sql> desc formatted table1 partition(year='2019', month='10', day='17', hour='23'); 19/10/19 00:02:40 ERROR SparkSQLDriver: Failed in [desc formatted table1 partition(year='2019', month='10', day='17', hour='23')] java.lang.NumberFormatException: Zero length BigInteger at java.math.BigInteger.(BigInteger.java:411) at java.math.BigInteger.(BigInteger.java:597) at scala.math.BigInt$.apply(BigInt.scala:77) at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$31.apply(HiveClientImpl.scala:1056) ``` Although we can use 'Analyze table partition ' to update the totalSize,rawDataSize or rowCount, it's unresonable for normal SQL to throw NumberFormatException for Empty totalSize.We should fix the empty case when readHiveStats. ### Why are the changes needed? This is a related to the robustness of the code and may lead to unexpected exception in some unpredictable situation.Here is the case: image ### Does this PR introduce any user-facing change? No ### How was this patch tested? manual Closes #26892 from southernriver/SPARK-30262. Authored-by: chenliang Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 7 ++++--- 1 file changed, 4 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 700c0884dd17c..f196e94a83f97 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 @@ -1189,9 +1189,10 @@ private[hive] object HiveClientImpl { * Note that this statistics could be overridden by Spark's statistics if that's available. */ private def readHiveStats(properties: Map[String, String]): Option[CatalogStatistics] = { - 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(_)) + val totalSize = properties.get(StatsSetupConst.TOTAL_SIZE).filter(_.nonEmpty).map(BigInt(_)) + val rawDataSize = properties.get(StatsSetupConst.RAW_DATA_SIZE).filter(_.nonEmpty) + .map(BigInt(_)) + val rowCount = properties.get(StatsSetupConst.ROW_COUNT).filter(_.nonEmpty).map(BigInt(_)) // 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. // Currently, only totalSize, rawDataSize, and rowCount are used to build the field `stats` From 1e48b43a0e8846c686c162073370ee889a66af65 Mon Sep 17 00:00:00 2001 From: ulysses Date: Wed, 18 Dec 2019 15:54:30 -0800 Subject: [PATCH 0919/1387] [SPARK-30254][SQL] Fix LikeSimplification optimizer to use a given escapeChar Since [25001](https://github.com/apache/spark/pull/25001), spark support like escape syntax. We should also sync the escape used by `LikeSimplification`. Avoid optimize failed. No. Add UT. Closes #26880 from ulysses-you/SPARK-30254. Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/optimizer/expressions.scala | 7 ++++--- .../catalyst/optimizer/LikeSimplificationSuite.scala | 10 ++++++++++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 98acad8a7413a..73df8e6df2bab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -489,17 +489,18 @@ object LikeSimplification extends Rule[LogicalPlan] { // If pattern is null, return null value directly, since "col like null" == null. Literal(null, BooleanType) } else { + val escapeStr = String.valueOf(escapeChar) pattern.toString match { - case startsWith(prefix) if !prefix.endsWith("\\") => + case startsWith(prefix) if !prefix.endsWith(escapeStr) => StartsWith(input, Literal(prefix)) case endsWith(postfix) => EndsWith(input, Literal(postfix)) // 'a%a' pattern is basically same with 'a%' && '%a'. // However, the additional `Length` condition is required to prevent 'a' match 'a%a'. - case startsAndEndsWith(prefix, postfix) if !prefix.endsWith("\\") => + case startsAndEndsWith(prefix, postfix) if !prefix.endsWith(escapeStr) => And(GreaterThanOrEqual(Length(input), Literal(prefix.length + postfix.length)), And(StartsWith(input, Literal(prefix)), EndsWith(input, Literal(postfix)))) - case contains(infix) if !infix.endsWith("\\") => + case contains(infix) if !infix.endsWith(escapeStr) => Contains(input, Literal(infix)) case equalTo(str) => EqualTo(input, Literal(str)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala index f64d10357ce2f..436f62e4225c8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala @@ -106,4 +106,14 @@ class LikeSimplificationSuite extends PlanTest { val optimized = Optimize.execute(originalQuery) comparePlans(optimized, testRelation.where(Literal(null, BooleanType)).analyze) } + + test("test like escape syntax") { + val originalQuery1 = testRelation.where('a.like("abc#%", '#')) + val optimized1 = Optimize.execute(originalQuery1.analyze) + comparePlans(optimized1, originalQuery1.analyze) + + val originalQuery2 = testRelation.where('a.like("abc#%abc", '#')) + val optimized2 = Optimize.execute(originalQuery2.analyze) + comparePlans(optimized2, originalQuery2.analyze) + } } From ab8eb86a77543769a7e7c2c312e90f1d38134ac5 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 19 Dec 2019 16:34:27 +0900 Subject: [PATCH 0920/1387] Revert "[SPARK-29629][SQL] Support typed integer literal expression" This reverts commit 8e667db5d801bd1c74a75abbea62f2a0007c950b. Closes #26940 from gengliangwang/revert_Spark_29629. Authored-by: Gengliang Wang Signed-off-by: HyukjinKwon --- .../sql/catalyst/parser/AstBuilder.scala | 12 +- .../resources/sql-tests/inputs/literals.sql | 7 -- .../sql-tests/results/ansi/literals.sql.out | 106 +++++------------- .../sql-tests/results/literals.sql.out | 106 +++++------------- 4 files changed, 61 insertions(+), 170 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 8f3827391ab0d..cf6dac944bf4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1866,7 +1866,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * {{{ * [TYPE] '[VALUE]' * }}} - * Currently Date, Timestamp, Interval, Binary and INTEGER typed literals are supported. + * Currently Date, Timestamp, Interval and Binary typed literals are supported. */ override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { val value = string(ctx.STRING) @@ -1897,16 +1897,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case "X" => val padding = if (value.length % 2 != 0) "0" else "" Literal(DatatypeConverter.parseHexBinary(padding + value)) - case "INTEGER" => - val i = try { - value.toInt - } catch { - case e: NumberFormatException => - val ex = new ParseException(s"Cannot parse the Int value: $value, $e", ctx) - ex.setStackTrace(e.getStackTrace) - throw ex - } - Literal(i, IntegerType) case other => throw new ParseException(s"Literals of type '$other' are currently not" + " supported.", ctx) diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql index 738b8411759c2..108cfd766af2c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/literals.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -100,14 +100,7 @@ select X'XuZ'; -- Hive literal_double test. SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8; --- typed integer expression -select integer '7'; -select integer'7'; -select integer '2147483648'; - -- awareness of the negative/positive sign before type -select -integer '7'; -select +integer '7'; select +date '1999-01-01'; select +timestamp '1999-01-01'; select +interval '1 day'; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 96425cf6219ed..382294010914f 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 55 +-- Number of queries: 50 -- !query 0 @@ -393,135 +393,89 @@ struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E8:double,3.14E-8:double,-3.14E8 -- !query 40 -select integer '7' --- !query 40 schema -struct<7:int> --- !query 40 output -7 - - --- !query 41 -select integer'7' --- !query 41 schema -struct<7:int> --- !query 41 output -7 - - --- !query 42 -select integer '2147483648' --- !query 42 schema -struct<> --- !query 42 output -org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the Int value: 2147483648, java.lang.NumberFormatException: For input string: "2147483648"(line 1, pos 7) - -== SQL == -select integer '2147483648' --------^^^ - - --- !query 43 -select -integer '7' --- !query 43 schema -struct<(- 7):int> --- !query 43 output --7 - - --- !query 44 -select +integer '7' --- !query 44 schema -struct<(+ 7):int> --- !query 44 output -7 - - --- !query 45 select +date '1999-01-01' --- !query 45 schema +-- !query 40 schema struct<> --- !query 45 output +-- !query 40 output org.apache.spark.sql.AnalysisException cannot resolve '(+ DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 --- !query 46 +-- !query 41 select +timestamp '1999-01-01' --- !query 46 schema +-- !query 41 schema struct<> --- !query 46 output +-- !query 41 output org.apache.spark.sql.AnalysisException cannot resolve '(+ TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 --- !query 47 +-- !query 42 select +interval '1 day' --- !query 47 schema +-- !query 42 schema struct<(+ INTERVAL '1 days'):interval> --- !query 47 output +-- !query 42 output 1 days --- !query 48 +-- !query 43 select +map(1, 2) --- !query 48 schema +-- !query 43 schema struct<> --- !query 48 output +-- !query 43 output org.apache.spark.sql.AnalysisException cannot resolve '(+ map(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'map(1, 2)' is of map type.; line 1 pos 7 --- !query 49 +-- !query 44 select +array(1,2) --- !query 49 schema +-- !query 44 schema struct<> --- !query 49 output +-- !query 44 output org.apache.spark.sql.AnalysisException cannot resolve '(+ array(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'array(1, 2)' is of array type.; line 1 pos 7 --- !query 50 +-- !query 45 select +named_struct('a', 1, 'b', 'spark') --- !query 50 schema +-- !query 45 schema struct<> --- !query 50 output +-- !query 45 output org.apache.spark.sql.AnalysisException cannot resolve '(+ named_struct('a', 1, 'b', 'spark'))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'named_struct('a', 1, 'b', 'spark')' is of struct type.; line 1 pos 7 --- !query 51 +-- !query 46 select +X'1' --- !query 51 schema +-- !query 46 schema struct<> --- !query 51 output +-- !query 46 output org.apache.spark.sql.AnalysisException cannot resolve '(+ X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7 --- !query 52 +-- !query 47 select -date '1999-01-01' --- !query 52 schema +-- !query 47 schema struct<> --- !query 52 output +-- !query 47 output org.apache.spark.sql.AnalysisException cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 --- !query 53 +-- !query 48 select -timestamp '1999-01-01' --- !query 53 schema +-- !query 48 schema struct<> --- !query 53 output +-- !query 48 output org.apache.spark.sql.AnalysisException cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 --- !query 54 +-- !query 49 select -x'2379ACFe' --- !query 54 schema +-- !query 49 schema struct<> --- !query 54 output +-- !query 49 output org.apache.spark.sql.AnalysisException cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 96425cf6219ed..382294010914f 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 55 +-- Number of queries: 50 -- !query 0 @@ -393,135 +393,89 @@ struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E8:double,3.14E-8:double,-3.14E8 -- !query 40 -select integer '7' --- !query 40 schema -struct<7:int> --- !query 40 output -7 - - --- !query 41 -select integer'7' --- !query 41 schema -struct<7:int> --- !query 41 output -7 - - --- !query 42 -select integer '2147483648' --- !query 42 schema -struct<> --- !query 42 output -org.apache.spark.sql.catalyst.parser.ParseException - -Cannot parse the Int value: 2147483648, java.lang.NumberFormatException: For input string: "2147483648"(line 1, pos 7) - -== SQL == -select integer '2147483648' --------^^^ - - --- !query 43 -select -integer '7' --- !query 43 schema -struct<(- 7):int> --- !query 43 output --7 - - --- !query 44 -select +integer '7' --- !query 44 schema -struct<(+ 7):int> --- !query 44 output -7 - - --- !query 45 select +date '1999-01-01' --- !query 45 schema +-- !query 40 schema struct<> --- !query 45 output +-- !query 40 output org.apache.spark.sql.AnalysisException cannot resolve '(+ DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 --- !query 46 +-- !query 41 select +timestamp '1999-01-01' --- !query 46 schema +-- !query 41 schema struct<> --- !query 46 output +-- !query 41 output org.apache.spark.sql.AnalysisException cannot resolve '(+ TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 --- !query 47 +-- !query 42 select +interval '1 day' --- !query 47 schema +-- !query 42 schema struct<(+ INTERVAL '1 days'):interval> --- !query 47 output +-- !query 42 output 1 days --- !query 48 +-- !query 43 select +map(1, 2) --- !query 48 schema +-- !query 43 schema struct<> --- !query 48 output +-- !query 43 output org.apache.spark.sql.AnalysisException cannot resolve '(+ map(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'map(1, 2)' is of map type.; line 1 pos 7 --- !query 49 +-- !query 44 select +array(1,2) --- !query 49 schema +-- !query 44 schema struct<> --- !query 49 output +-- !query 44 output org.apache.spark.sql.AnalysisException cannot resolve '(+ array(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'array(1, 2)' is of array type.; line 1 pos 7 --- !query 50 +-- !query 45 select +named_struct('a', 1, 'b', 'spark') --- !query 50 schema +-- !query 45 schema struct<> --- !query 50 output +-- !query 45 output org.apache.spark.sql.AnalysisException cannot resolve '(+ named_struct('a', 1, 'b', 'spark'))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'named_struct('a', 1, 'b', 'spark')' is of struct type.; line 1 pos 7 --- !query 51 +-- !query 46 select +X'1' --- !query 51 schema +-- !query 46 schema struct<> --- !query 51 output +-- !query 46 output org.apache.spark.sql.AnalysisException cannot resolve '(+ X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7 --- !query 52 +-- !query 47 select -date '1999-01-01' --- !query 52 schema +-- !query 47 schema struct<> --- !query 52 output +-- !query 47 output org.apache.spark.sql.AnalysisException cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 --- !query 53 +-- !query 48 select -timestamp '1999-01-01' --- !query 53 schema +-- !query 48 schema struct<> --- !query 53 output +-- !query 48 output org.apache.spark.sql.AnalysisException cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 --- !query 54 +-- !query 49 select -x'2379ACFe' --- !query 54 schema +-- !query 49 schema struct<> --- !query 54 output +-- !query 49 output org.apache.spark.sql.AnalysisException cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 From 2af5237fe889ddc0926ca7af8eab0fec5622ecbb Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Thu, 19 Dec 2019 17:01:40 +0800 Subject: [PATCH 0921/1387] [SPARK-29918][SQL][FOLLOWUP][TEST] Fix arrayOffset in `RecordBinaryComparatorSuite` ### What changes were proposed in this pull request? As mentioned in https://github.com/apache/spark/pull/26548#pullrequestreview-334345333, some test cases in `RecordBinaryComparatorSuite` use a fixed arrayOffset when writing to long arrays, this could lead to weird stuff including crashing with a SIGSEGV. This PR fix the problem by computing the arrayOffset based on `Platform.LONG_ARRAY_OFFSET`. ### How was this patch tested? Tested locally. Previously, when we try to add `System.gc()` between write into long array and compare by RecordBinaryComparator, there is a chance to hit JVM crash with SIGSEGV like: ``` # # A fatal error has been detected by the Java Runtime Environment: # # SIGSEGV (0xb) at pc=0x00007efc66970bcb, pid=11831, tid=0x00007efc0f9f9700 # # JRE version: OpenJDK Runtime Environment (8.0_222-b10) (build 1.8.0_222-8u222-b10-1ubuntu1~16.04.1-b10) # Java VM: OpenJDK 64-Bit Server VM (25.222-b10 mixed mode linux-amd64 compressed oops) # Problematic frame: # V [libjvm.so+0x5fbbcb] # # Core dump written. Default location: /home/jenkins/workspace/sql/core/core or core.11831 # # An error report file with more information is saved as: # /home/jenkins/workspace/sql/core/hs_err_pid11831.log # # If you would like to submit a bug report, please visit: # http://bugreport.java.com/bugreport/crash.jsp # ``` After the fix those test cases didn't crash the JVM anymore. Closes #26939 from jiangxb1987/rbc. Authored-by: Xingbo Jiang Signed-off-by: Wenchen Fan --- .../spark/sql/execution/sort/RecordBinaryComparatorSuite.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java index 4b23615275871..564e76737ecde 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java @@ -325,7 +325,7 @@ public void testBinaryComparatorWhenOnlyTheLastColumnDiffers() throws Exception @Test public void testCompareLongsAsLittleEndian() { - long arrayOffset = 12; + long arrayOffset = Platform.LONG_ARRAY_OFFSET + 4; long[] arr1 = new long[2]; Platform.putLong(arr1, arrayOffset, 0x0100000000000000L); @@ -347,7 +347,7 @@ public void testCompareLongsAsLittleEndian() { @Test public void testCompareLongsAsUnsigned() { - long arrayOffset = 12; + long arrayOffset = Platform.LONG_ARRAY_OFFSET + 4; long[] arr1 = new long[2]; Platform.putLong(arr1, arrayOffset + 4, 0xa000000000000000L); From ab87bfd087e5154d1e50c6f9c5be4b64d14eec64 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 19 Dec 2019 18:20:41 +0900 Subject: [PATCH 0922/1387] [SPARK-29450][SS] Measure the number of output rows for streaming aggregation with append mode ### What changes were proposed in this pull request? This patch addresses missing metric, the number of output rows for streaming aggregation with append mode. Other modes are correctly measuring it. ### Why are the changes needed? Without the patch, the value for such metric is always 0. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test added. Also manually tested with below query: > query ``` import spark.implicits._ spark.conf.set("spark.sql.shuffle.partitions", "5") val df = spark.readStream .format("rate") .option("rowsPerSecond", 1000) .load() .withWatermark("timestamp", "5 seconds") .selectExpr("timestamp", "mod(value, 100) as mod", "value") .groupBy(window($"timestamp", "10 seconds"), $"mod") .agg(max("value").as("max_value"), min("value").as("min_value"), avg("value").as("avg_value")) val query = df .writeStream .format("memory") .option("queryName", "test") .outputMode("append") .start() query.awaitTermination() ``` > before the patch ![screenshot-before-SPARK-29450](https://user-images.githubusercontent.com/1317309/69023217-58d7bc80-0a01-11ea-8cac-40f1cced6d16.png) > after the patch ![screenshot-after-SPARK-29450](https://user-images.githubusercontent.com/1317309/69023221-5c6b4380-0a01-11ea-8a66-7bf1b7d09fc7.png) Closes #26104 from HeartSaVioR/SPARK-29450. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../streaming/statefulOperators.scala | 1 + .../streaming/StreamingAggregationSuite.scala | 65 ++++++++++++++++++- 2 files changed, 65 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index 01b309c3cf345..1bec924ba219a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -353,6 +353,7 @@ case class StateStoreSaveExec( finished = true null } else { + numOutputRows += 1 removedValueRow } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 9779635df3ef1..22caa48686c7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.streaming import java.io.File import java.util.Locale +import scala.collection.mutable + import org.apache.commons.io.FileUtils import org.scalatest.Assertions @@ -183,7 +185,68 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { ) } - testWithAllStateVersions("state metrics") { + testWithAllStateVersions("state metrics - append mode") { + val inputData = MemoryStream[Int] + val aggWithWatermark = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + implicit class RichStreamExecution(query: StreamExecution) { + // this could be either empty row batch or actual batch + def stateNodes: Seq[SparkPlan] = { + query.lastExecution.executedPlan.collect { + case p if p.isInstanceOf[StateStoreSaveExec] => p + } + } + + def stateOperatorProgresses: Seq[StateOperatorProgress] = { + val operatorProgress = mutable.ArrayBuffer[StateOperatorProgress]() + var progress = query.recentProgress.last + + operatorProgress ++= progress.stateOperators.map { op => op.copy(op.numRowsUpdated) } + if (progress.numInputRows == 0) { + // empty batch, merge metrics from previous batch as well + progress = query.recentProgress.takeRight(2).head + operatorProgress.zipWithIndex.foreach { case (sop, index) => + // "numRowsUpdated" should be merged, as it could be updated in both batches. + // (for now it is only updated from previous batch, but things can be changed.) + // other metrics represent current status of state so picking up the latest values. + val newOperatorProgress = sop.copy( + sop.numRowsUpdated + progress.stateOperators(index).numRowsUpdated) + operatorProgress(index) = newOperatorProgress + } + } + + operatorProgress + } + } + + testStream(aggWithWatermark)( + AddData(inputData, 15), + CheckAnswer(), // watermark = 5 + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 1 }, + AddData(inputData, 10, 12, 14), + CheckAnswer(), // watermark = 5 + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 0 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 2 }, + AddData(inputData, 25), + CheckAnswer((10, 3)), // watermark = 15 + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsUpdated === 1 }, + AssertOnQuery { _.stateOperatorProgresses.head.numRowsTotal === 2 } + ) + } + + testWithAllStateVersions("state metrics - update/complete mode") { val inputData = MemoryStream[Int] val aggregated = From dea18231d499d57a717f7662ca3ce0eac7375425 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 19 Dec 2019 12:30:34 -0800 Subject: [PATCH 0923/1387] [SPARK-30309][SQL] Mark `Filter` as a `sealed` class ### What changes were proposed in this pull request? Added the `sealed` keyword to the `Filter` class ### Why are the changes needed? To do not miss handling of new filters in a datasource in the future. For example, `AlwaysTrue` and `AlwaysFalse` were added recently by https://github.com/apache/spark/pull/23606 ### Does this PR introduce any user-facing change? Should not. ### How was this patch tested? By existing tests. Closes #26950 from MaxGekk/sealed-filter. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/sql/sources/filters.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala index a1ab55a7185ce..fc7d33e823a99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -29,7 +29,7 @@ import org.apache.spark.annotation.{Evolving, Stable} * @since 1.3.0 */ @Stable -abstract class Filter { +sealed abstract class Filter { /** * List of columns that are referenced by this filter. * @since 2.1.0 From 726f6d3e3cb6ec24ca39da93cdb16df0a03d8771 Mon Sep 17 00:00:00 2001 From: Aman Omer Date: Fri, 20 Dec 2019 12:49:16 +0800 Subject: [PATCH 0924/1387] [SPARK-30184][SQL] Implement a helper method for aliasing functions ### What changes were proposed in this pull request? This PR is to use `expressionWithAlias` for remaining functions for which alias name can be used. Remaining functions are: `Average, First, Last, ApproximatePercentile, StddevSamp, VarianceSamp` PR https://github.com/apache/spark/pull/26712 introduced `expressionWithAlias` ### Why are the changes needed? Error message is wrong when alias name is used for above mentioned functions. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually Closes #26808 from amanomer/fncAlias. Lead-authored-by: Aman Omer Co-authored-by: Aman Omer <40591404+amanomer@users.noreply.github.com> Signed-off-by: Wenchen Fan --- .../catalyst/analysis/FunctionRegistry.scala | 70 +++++-------------- .../aggregate/ApproximatePercentile.scala | 6 +- .../expressions/aggregate/Average.scala | 4 +- .../aggregate/CentralMomentAgg.scala | 6 +- .../expressions/aggregate/First.scala | 6 +- .../catalyst/expressions/aggregate/Last.scala | 6 +- .../aggregate/UnevaluableAggs.scala | 10 +-- .../catalyst/optimizer/finishAnalysis.scala | 4 +- .../ExpressionTypeCheckingSuite.scala | 4 +- .../sql-tests/results/group-by.sql.out | 2 +- .../results/postgreSQL/window_part4.sql.out | 16 ++--- .../results/udf/udf-group-by.sql.out | 12 ++-- 12 files changed, 61 insertions(+), 85 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index f8a0c528bff0a..7a8b88e5264a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -22,7 +22,6 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import scala.reflect.ClassTag -import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException @@ -31,6 +30,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.xml._ +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.types._ @@ -193,6 +193,8 @@ object FunctionRegistry { type FunctionBuilder = Seq[Expression] => Expression + val FUNC_ALIAS = TreeNodeTag[String]("functionAliasName") + // Note: Whenever we add a new entry here, make sure we also update ExpressionToSQLSuite val expressions: Map[String, (ExpressionInfo, FunctionBuilder)] = Map( // misc non-aggregate functions @@ -289,35 +291,35 @@ object FunctionRegistry { expression[CovPopulation]("covar_pop"), expression[CovSample]("covar_samp"), expression[First]("first"), - expression[First]("first_value"), + expression[First]("first_value", true), expression[Kurtosis]("kurtosis"), expression[Last]("last"), - expression[Last]("last_value"), + expression[Last]("last_value", true), expression[Max]("max"), expression[MaxBy]("max_by"), - expression[Average]("mean"), + expression[Average]("mean", true), expression[Min]("min"), expression[MinBy]("min_by"), expression[Percentile]("percentile"), expression[Skewness]("skewness"), expression[ApproximatePercentile]("percentile_approx"), - expression[ApproximatePercentile]("approx_percentile"), - expression[StddevSamp]("std"), - expression[StddevSamp]("stddev"), + expression[ApproximatePercentile]("approx_percentile", true), + expression[StddevSamp]("std", true), + expression[StddevSamp]("stddev", true), expression[StddevPop]("stddev_pop"), expression[StddevSamp]("stddev_samp"), expression[Sum]("sum"), - expression[VarianceSamp]("variance"), + expression[VarianceSamp]("variance", true), expression[VariancePop]("var_pop"), expression[VarianceSamp]("var_samp"), expression[CollectList]("collect_list"), expression[CollectSet]("collect_set"), expression[CountMinSketchAgg]("count_min_sketch"), - expressionWithAlias[BoolAnd]("every"), - expressionWithAlias[BoolAnd]("bool_and"), - expressionWithAlias[BoolOr]("any"), - expressionWithAlias[BoolOr]("some"), - expressionWithAlias[BoolOr]("bool_or"), + expression[BoolAnd]("every", true), + expression[BoolAnd]("bool_and"), + expression[BoolOr]("any", true), + expression[BoolOr]("some", true), + expression[BoolOr]("bool_or"), // string functions expression[Ascii]("ascii"), @@ -573,7 +575,7 @@ object FunctionRegistry { val functionSet: Set[FunctionIdentifier] = builtin.listFunction().toSet /** See usage above. */ - private def expression[T <: Expression](name: String) + private def expression[T <: Expression](name: String, setAlias: Boolean = false) (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = { // For `RuntimeReplaceable`, skip the constructor with most arguments, which is the main @@ -619,7 +621,9 @@ object FunctionRegistry { throw new AnalysisException(invalidArgumentsMsg) } try { - f.newInstance(expressions : _*).asInstanceOf[Expression] + val exp = f.newInstance(expressions : _*).asInstanceOf[Expression] + if (setAlias) exp.setTagValue(FUNC_ALIAS, name) + exp } catch { // the exception is an invocation exception. To get a meaningful message, we need the // cause. @@ -631,42 +635,6 @@ object FunctionRegistry { (name, (expressionInfo[T](name), builder)) } - private def expressionWithAlias[T <: Expression](name: String) - (implicit tag: ClassTag[T]): (String, (ExpressionInfo, FunctionBuilder)) = { - val constructors = tag.runtimeClass.getConstructors - .filter(_.getParameterTypes.head == classOf[String]) - assert(constructors.length == 1) - val builder = (expressions: Seq[Expression]) => { - val params = classOf[String] +: Seq.fill(expressions.size)(classOf[Expression]) - val f = constructors.find(_.getParameterTypes.toSeq == params).getOrElse { - val validParametersCount = constructors - .filter(_.getParameterTypes.tail.forall(_ == classOf[Expression])) - .map(_.getParameterCount - 1).distinct.sorted - val invalidArgumentsMsg = if (validParametersCount.length == 0) { - s"Invalid arguments for function $name" - } else { - val expectedNumberOfParameters = if (validParametersCount.length == 1) { - validParametersCount.head.toString - } else { - validParametersCount.init.mkString("one of ", ", ", " and ") + - validParametersCount.last - } - s"Invalid number of arguments for function $name. " + - s"Expected: $expectedNumberOfParameters; Found: ${expressions.size}" - } - throw new AnalysisException(invalidArgumentsMsg) - } - try { - f.newInstance(name.toString +: expressions: _*).asInstanceOf[Expression] - } catch { - // the exception is an invocation exception. To get a meaningful message, we need the - // cause. - case e: Exception => throw new AnalysisException(e.getCause.getMessage) - } - } - (name, (expressionInfo[T](name), builder)) - } - /** * Creates a function registry lookup entry for cast aliases (SPARK-16730). * For example, if name is "int", and dataType is IntegerType, this means int(x) would become diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index ea0ed2e8fa11b..b143ddef6a6d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import com.google.common.primitives.{Doubles, Ints, Longs} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.PercentileDigest @@ -185,7 +185,8 @@ case class ApproximatePercentile( if (returnPercentileArray) ArrayType(child.dataType, false) else child.dataType } - override def prettyName: String = "percentile_approx" + override def prettyName: String = + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("percentile_approx") override def serialize(obj: PercentileDigest): Array[Byte] = { ApproximatePercentile.serializer.serialize(obj) @@ -321,4 +322,5 @@ object ApproximatePercentile { } val serializer: PercentileDigestSerializer = new PercentileDigestSerializer + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala index aaad3c7bcefa8..9bb048a9851e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.{DecimalPrecision, TypeCheckResult} +import org.apache.spark.sql.catalyst.analysis.{DecimalPrecision, FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.TypeUtils @@ -37,7 +37,7 @@ import org.apache.spark.sql.types._ since = "1.0.0") case class Average(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes { - override def prettyName: String = "avg" + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("avg") override def children: Seq[Expression] = child :: Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index 8ce8dfa19c017..bf402807d62d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -174,7 +175,8 @@ case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { If(n === 1.0, Double.NaN, sqrt(m2 / (n - 1.0)))) } - override def prettyName: String = "stddev_samp" + override def prettyName: String = + getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("stddev_samp") } // Compute the population variance of a column @@ -215,7 +217,7 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { If(n === 1.0, Double.NaN, m2 / (n - 1.0))) } - override def prettyName: String = "var_samp" + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("var_samp") } @ExpressionDescription( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala index 9f351395846e4..8de866ed9fb1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/First.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -113,5 +113,7 @@ case class First(child: Expression, ignoreNullsExpr: Expression) override lazy val evaluateExpression: AttributeReference = first - override def toString: String = s"first($child)${if (ignoreNulls) " ignore nulls"}" + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("first") + + override def toString: String = s"$prettyName($child)${if (ignoreNulls) " ignore nulls"}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala index 405719faaeb58..f8af0cd1f3033 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Last.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -111,5 +111,7 @@ case class Last(child: Expression, ignoreNullsExpr: Expression) override lazy val evaluateExpression: AttributeReference = last - override def toString: String = s"last($child)${if (ignoreNulls) " ignore nulls"}" + override def prettyName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("last") + + override def toString: String = s"$prettyName($child)${if (ignoreNulls) " ignore nulls"}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala index acb0af0248a74..a1cd4a77d0445 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -52,8 +52,8 @@ abstract class UnevaluableBooleanAggBase(arg: Expression) false """, since = "3.0.0") -case class BoolAnd(funcName: String, arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = funcName +case class BoolAnd(arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("bool_and") } @ExpressionDescription( @@ -68,6 +68,6 @@ case class BoolAnd(funcName: String, arg: Expression) extends UnevaluableBoolean false """, since = "3.0.0") -case class BoolOr(funcName: String, arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = funcName +case class BoolOr(arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse("bool_or") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index c330274341524..f64b6e00373f6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -47,8 +47,8 @@ object ReplaceExpressions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case e: RuntimeReplaceable => e.child case CountIf(predicate) => Count(new NullIf(predicate, Literal.FalseLiteral)) - case BoolOr(_, arg) => Max(arg) - case BoolAnd(_, arg) => Min(arg) + case BoolOr(arg) => Max(arg) + case BoolAnd(arg) => Min(arg) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index f944b4ad87e40..86a1f1fb58a07 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -153,8 +153,8 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertSuccess(Sum(Symbol("stringField"))) assertSuccess(Average(Symbol("stringField"))) assertSuccess(Min(Symbol("arrayField"))) - assertSuccess(new BoolAnd("bool_and", Symbol("booleanField"))) - assertSuccess(new BoolOr("bool_or", Symbol("booleanField"))) + assertSuccess(new BoolAnd(Symbol("booleanField"))) + assertSuccess(new BoolOr(Symbol("booleanField"))) assertError(Min(Symbol("mapField")), "min does not support ordering on type") assertError(Max(Symbol("mapField")), "max does not support ordering on type") diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index ed5ced8c8c0fe..62a1666497084 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -128,7 +128,7 @@ NULL 1 SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM testData -- !query 13 schema -struct +struct -- !query 13 output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index 54ceacd3b3b3e..4721ceb03a966 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -241,7 +241,7 @@ NaN SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) -- !query 19 schema -struct +struct -- !query 19 output 16900.0 18491.666666666668 @@ -254,7 +254,7 @@ NaN SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) -- !query 20 schema -struct +struct -- !query 20 output 16900.0 18491.666666666668 @@ -267,7 +267,7 @@ NaN SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) -- !query 21 schema -struct +struct -- !query 21 output 16900.0 18491.666666666668 @@ -280,7 +280,7 @@ NaN SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) -- !query 22 schema -struct +struct -- !query 22 output 16900.0 18491.666666666668 @@ -405,7 +405,7 @@ NaN SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) -- !query 31 schema -struct +struct -- !query 31 output 130.0 135.9840676942217 @@ -419,7 +419,7 @@ NaN SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) -- !query 32 schema -struct +struct -- !query 32 output 130.0 135.9840676942217 @@ -433,7 +433,7 @@ NaN SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) -- !query 33 schema -struct +struct -- !query 33 output 130.0 135.9840676942217 @@ -447,7 +447,7 @@ NaN SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) -- !query 34 schema -struct +struct -- !query 34 output 130.0 135.9840676942217 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index bdacd184158a5..a835740a6a86d 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -128,7 +128,7 @@ NULL 1 SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) FROM testData -- !query 13 schema -struct +struct -- !query 13 output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 @@ -293,7 +293,7 @@ struct<> -- !query 31 SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 -- !query 31 schema -struct +struct -- !query 31 output NULL NULL NULL @@ -301,7 +301,7 @@ NULL NULL NULL -- !query 32 SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 -- !query 32 schema -struct +struct -- !query 32 output NULL NULL NULL @@ -309,7 +309,7 @@ NULL NULL NULL -- !query 33 SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 -- !query 33 schema -struct +struct -- !query 33 output false true true @@ -317,7 +317,7 @@ false true true -- !query 34 SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) -- !query 34 schema -struct +struct -- !query 34 output 1 false true true 2 true true true @@ -339,7 +339,7 @@ struct -- !query 36 SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL -- !query 36 schema -struct +struct -- !query 36 output 4 NULL From 18e8d1d5b2bab3194cb2118d3da16109922a578d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 19 Dec 2019 20:56:06 -0800 Subject: [PATCH 0925/1387] [SPARK-30307][SQL] remove ReusedQueryStageExec ### What changes were proposed in this pull request? When we reuse exchanges in AQE, what we produce is `ReuseQueryStage(QueryStage(Exchange))`. This PR changes it to `QueryStage(ReusedExchange(Exchange))`. This PR also fixes an issue in `LocalShuffleReaderExec.outputPartitioning`. We can only preserve the partitioning if we read one mapper per task. ### Why are the changes needed? `QueryStage` is light-weighted and we don't need to reuse its instance. What we really care is to reuse the exchange instance, which has heavy states (e.g. broadcasted valued, submitted map stage). To simplify the framework, we should use the existing `ReusedExchange` node to do the reuse work, instead of creating a new node. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26952 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: Xiao Li --- .../adaptive/AdaptiveSparkPlanExec.scala | 17 +-- .../adaptive/DemoteBroadcastHashJoin.scala | 4 +- .../adaptive/LogicalQueryStageStrategy.scala | 4 +- .../adaptive/OptimizeLocalShuffleReader.scala | 56 +++++---- .../execution/adaptive/QueryStageExec.scala | 116 +++++++----------- .../adaptive/ReduceNumShufflePartitions.scala | 22 ++-- .../sql/execution/exchange/Exchange.scala | 2 +- .../ReduceNumShufflePartitionsSuite.scala | 9 +- .../adaptive/AdaptiveQueryExecSuite.scala | 9 +- 9 files changed, 108 insertions(+), 131 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 8617220a0d0b2..1f1b020ced125 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -281,11 +281,12 @@ case class AdaptiveSparkPlanExec( // First have a quick check in the `stageCache` without having to traverse down the node. stageCache.get(e.canonicalized) match { case Some(existingStage) if conf.exchangeReuseEnabled => - val reusedStage = reuseQueryStage(existingStage, e) - // When reusing a stage, we treat it a new stage regardless of whether the existing stage - // has been materialized or not. Thus we won't skip re-optimization for a reused stage. - CreateStageResult(newPlan = reusedStage, - allChildStagesMaterialized = false, newStages = Seq(reusedStage)) + val stage = reuseQueryStage(existingStage, e) + // This is a leaf stage and is not materialized yet even if the reused exchange may has + // been completed. It will trigger re-optimization later and stage materialization will + // finish in instant if the underlying exchange is already completed. + CreateStageResult( + newPlan = stage, allChildStagesMaterialized = false, newStages = Seq(stage)) case _ => val result = createQueryStages(e.child) @@ -341,10 +342,10 @@ case class AdaptiveSparkPlanExec( queryStage } - private def reuseQueryStage(s: QueryStageExec, e: Exchange): QueryStageExec = { - val queryStage = ReusedQueryStageExec(currentStageId, s, e.output) + private def reuseQueryStage(existing: QueryStageExec, exchange: Exchange): QueryStageExec = { + val queryStage = existing.newReuseInstance(currentStageId, exchange.output) currentStageId += 1 - setLogicalLinkForNewQueryStage(queryStage, e) + setLogicalLinkForNewQueryStage(queryStage, exchange) queryStage } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala index 711b49e2d20a9..09efcb712b5ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala @@ -29,9 +29,7 @@ import org.apache.spark.sql.internal.SQLConf case class DemoteBroadcastHashJoin(conf: SQLConf) extends Rule[LogicalPlan] { private def shouldDemote(plan: LogicalPlan): Boolean = plan match { - case LogicalQueryStage(_, stage: QueryStageExec) - if stage.resultOption.isDefined && - stage.resultOption.get.isInstanceOf[MapOutputStatistics] => + case LogicalQueryStage(_, stage: ShuffleQueryStageExec) if stage.resultOption.isDefined => val mapOutputStatistics = stage.resultOption.get.asInstanceOf[MapOutputStatistics] val partitionCnt = mapOutputStatistics.bytesByPartitionId.length val nonZeroCnt = mapOutputStatistics.bytesByPartitionId.count(_ > 0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala index a0d07a68ab0f4..d60c3ca72f6f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala @@ -36,9 +36,7 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNes object LogicalQueryStageStrategy extends Strategy with PredicateHelper { private def isBroadcastStage(plan: LogicalPlan): Boolean = plan match { - case LogicalQueryStage(_, physicalPlan) - if BroadcastQueryStageExec.isBroadcastQueryStageExec(physicalPlan) => - true + case LogicalQueryStage(_, _: BroadcastQueryStageExec) => true case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 176e5ec8312e1..0659a89d2f808 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchangeExec} +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.internal.SQLConf @@ -69,23 +69,20 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { private def createLocalReader(plan: SparkPlan): LocalShuffleReaderExec = { plan match { - case c @ CoalescedShuffleReaderExec(q: QueryStageExec, _) => + case c @ CoalescedShuffleReaderExec(s: ShuffleQueryStageExec, _) => LocalShuffleReaderExec( - q, getPartitionStartIndices(q, Some(c.partitionStartIndices.length))) - case q: QueryStageExec => - LocalShuffleReaderExec(q, getPartitionStartIndices(q, None)) + s, getPartitionStartIndices(s, Some(c.partitionStartIndices.length))) + case s: ShuffleQueryStageExec => + LocalShuffleReaderExec(s, getPartitionStartIndices(s, None)) } } // TODO: this method assumes all shuffle blocks are the same data size. We should calculate the // partition start indices based on block size to avoid data skew. private def getPartitionStartIndices( - shuffle: QueryStageExec, + shuffleStage: ShuffleQueryStageExec, advisoryParallelism: Option[Int]): Array[Array[Int]] = { - val shuffleDep = shuffle match { - case s: ShuffleQueryStageExec => s.plan.shuffleDependency - case ReusedQueryStageExec(_, s: ShuffleQueryStageExec, _) => s.plan.shuffleDependency - } + val shuffleDep = shuffleStage.shuffle.shuffleDependency val numReducers = shuffleDep.partitioner.numPartitions val expectedParallelism = advisoryParallelism.getOrElse(numReducers) val numMappers = shuffleDep.rdd.getNumPartitions @@ -140,7 +137,7 @@ object OptimizeLocalShuffleReader { } def canUseLocalShuffleReader(plan: SparkPlan): Boolean = { - ShuffleQueryStageExec.isShuffleQueryStageExec(plan) || + plan.isInstanceOf[ShuffleQueryStageExec] || plan.isInstanceOf[CoalescedShuffleReaderExec] } } @@ -150,8 +147,8 @@ object OptimizeLocalShuffleReader { * shuffle files written by one mapper. By doing this, it's very likely to read the shuffle files * locally, as the shuffle files that a reduce task needs to read are in one node. * - * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the - * shuffle exchange node during canonicalization. + * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange node during + * canonicalization. * @param partitionStartIndicesPerMapper A mapper usually writes many shuffle blocks, and it's * better to launch multiple tasks to read shuffle blocks of * one mapper. This array contains the partition start @@ -163,11 +160,24 @@ case class LocalShuffleReaderExec( override def output: Seq[Attribute] = child.output - override def outputPartitioning: Partitioning = child match { - case stage: ShuffleQueryStageExec => - stage.plan.child.outputPartitioning - case r @ ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - r.updatePartitioning(stage.plan.child.outputPartitioning) + override lazy val outputPartitioning: Partitioning = { + // when we read one mapper per task, then the output partitioning is the same as the plan + // before shuffle. + if (partitionStartIndicesPerMapper.forall(_.length == 1)) { + child match { + case ShuffleQueryStageExec(_, s: ShuffleExchangeExec) => + s.child.outputPartitioning + case ShuffleQueryStageExec(_, r @ ReusedExchangeExec(_, s: ShuffleExchangeExec)) => + s.child.outputPartitioning match { + case e: Expression => r.updateAttr(e).asInstanceOf[Partitioning] + case other => other + } + case _ => + throw new IllegalStateException("operating on canonicalization plan") + } + } else { + UnknownPartitioning(partitionStartIndicesPerMapper.map(_.length).sum) + } } private var cachedShuffleRDD: RDD[InternalRow] = null @@ -176,9 +186,9 @@ case class LocalShuffleReaderExec( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createLocalShuffleRDD(partitionStartIndicesPerMapper) - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createLocalShuffleRDD(partitionStartIndicesPerMapper) + stage.shuffle.createLocalShuffleRDD(partitionStartIndicesPerMapper) + case _ => + throw new IllegalStateException("operating on canonicalization plan") } } cachedShuffleRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala index d8ee972442b79..f2079a38da651 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange._ - /** * A query stage is an independent subgraph of the query plan. Query stage materializes its output * before proceeding with further operators of the query plan. The data statistics of the @@ -74,12 +73,19 @@ abstract class QueryStageExec extends LeafExecNode { doMaterialize() } + def newReuseInstance(newStageId: Int, newOutput: Seq[Attribute]): QueryStageExec + /** * Compute the statistics of the query stage if executed, otherwise None. */ def computeStats(): Option[Statistics] = resultOption.map { _ => // Metrics `dataSize` are available in both `ShuffleExchangeExec` and `BroadcastExchangeExec`. - Statistics(sizeInBytes = plan.metrics("dataSize").value) + val exchange = plan match { + case r: ReusedExchangeExec => r.child + case e: Exchange => e + case _ => throw new IllegalStateException("wrong plan for query stage:\n " + plan.treeString) + } + Statistics(sizeInBytes = exchange.metrics("dataSize").value) } @transient @@ -123,20 +129,33 @@ abstract class QueryStageExec extends LeafExecNode { } /** - * A shuffle query stage whose child is a [[ShuffleExchangeExec]]. + * A shuffle query stage whose child is a [[ShuffleExchangeExec]] or [[ReusedExchangeExec]]. */ case class ShuffleQueryStageExec( override val id: Int, - override val plan: ShuffleExchangeExec) extends QueryStageExec { + override val plan: SparkPlan) extends QueryStageExec { + + @transient val shuffle = plan match { + case s: ShuffleExchangeExec => s + case ReusedExchangeExec(_, s: ShuffleExchangeExec) => s + case _ => + throw new IllegalStateException("wrong plan for shuffle stage:\n " + plan.treeString) + } override def doMaterialize(): Future[Any] = { - plan.mapOutputStatisticsFuture + shuffle.mapOutputStatisticsFuture + } + + override def newReuseInstance(newStageId: Int, newOutput: Seq[Attribute]): QueryStageExec = { + ShuffleQueryStageExec( + newStageId, + ReusedExchangeExec(newOutput, shuffle)) } override def cancel(): Unit = { - plan.mapOutputStatisticsFuture match { + shuffle.mapOutputStatisticsFuture match { case action: FutureAction[MapOutputStatistics] - if !plan.mapOutputStatisticsFuture.isCompleted => + if !shuffle.mapOutputStatisticsFuture.isCompleted => action.cancel() case _ => } @@ -144,84 +163,33 @@ case class ShuffleQueryStageExec( } /** - * A broadcast query stage whose child is a [[BroadcastExchangeExec]]. + * A broadcast query stage whose child is a [[BroadcastExchangeExec]] or [[ReusedExchangeExec]]. */ case class BroadcastQueryStageExec( override val id: Int, - override val plan: BroadcastExchangeExec) extends QueryStageExec { - - override def doMaterialize(): Future[Any] = { - plan.completionFuture - } - - override def cancel(): Unit = { - if (!plan.relationFuture.isDone) { - sparkContext.cancelJobGroup(plan.runId.toString) - plan.relationFuture.cancel(true) - } - } -} - -object ShuffleQueryStageExec { - /** - * Returns true if the plan is a [[ShuffleQueryStageExec]] or a reused [[ShuffleQueryStageExec]]. - */ - def isShuffleQueryStageExec(plan: SparkPlan): Boolean = plan match { - case r: ReusedQueryStageExec => isShuffleQueryStageExec(r.plan) - case _: ShuffleQueryStageExec => true - case _ => false - } -} + override val plan: SparkPlan) extends QueryStageExec { -object BroadcastQueryStageExec { - /** - * Returns true if the plan is a [[BroadcastQueryStageExec]] or a reused - * [[BroadcastQueryStageExec]]. - */ - def isBroadcastQueryStageExec(plan: SparkPlan): Boolean = plan match { - case r: ReusedQueryStageExec => isBroadcastQueryStageExec(r.plan) - case _: BroadcastQueryStageExec => true - case _ => false + @transient val broadcast = plan match { + case b: BroadcastExchangeExec => b + case ReusedExchangeExec(_, b: BroadcastExchangeExec) => b + case _ => + throw new IllegalStateException("wrong plan for broadcast stage:\n " + plan.treeString) } -} - -/** - * A wrapper for reused query stage to have different output. - */ -case class ReusedQueryStageExec( - override val id: Int, - override val plan: QueryStageExec, - override val output: Seq[Attribute]) extends QueryStageExec { override def doMaterialize(): Future[Any] = { - plan.materialize() + broadcast.completionFuture } - override def cancel(): Unit = { - plan.cancel() + override def newReuseInstance(newStageId: Int, newOutput: Seq[Attribute]): QueryStageExec = { + BroadcastQueryStageExec( + newStageId, + ReusedExchangeExec(newOutput, broadcast)) } - // `ReusedQueryStageExec` can have distinct set of output attribute ids from its child, we need - // to update the attribute ids in `outputPartitioning` and `outputOrdering`. - private lazy val updateAttr: Expression => Expression = { - val originalAttrToNewAttr = AttributeMap(plan.output.zip(output)) - e => e.transform { - case attr: Attribute => originalAttrToNewAttr.getOrElse(attr, attr) + override def cancel(): Unit = { + if (!broadcast.relationFuture.isDone) { + sparkContext.cancelJobGroup(broadcast.runId.toString) + broadcast.relationFuture.cancel(true) } } - - private[sql] def updatePartitioning(p: Partitioning): Partitioning = p match { - case e: Expression => updateAttr(e).asInstanceOf[Partitioning] - case other => other - } - - override def outputPartitioning: Partitioning = { - updatePartitioning(plan.outputPartitioning) - } - - override def outputOrdering: Seq[SortOrder] = { - plan.outputOrdering.map(updateAttr(_).asInstanceOf[SortOrder]) - } - - override def computeStats(): Option[Statistics] = plan.computeStats() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 0acbeaa251f78..3767e6cae46e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable.ArrayBuffer -import scala.concurrent.duration.Duration import org.apache.spark.MapOutputStatistics import org.apache.spark.rdd.RDD @@ -28,7 +27,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartit import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.util.ThreadUtils /** * A rule to adjust the post shuffle partitions based on the map output statistics. @@ -66,20 +64,18 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil case stage: ShuffleQueryStageExec => Seq(stage) - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => Seq(stage) case _ => plan.children.flatMap(collectShuffleStages) } val shuffleStages = collectShuffleStages(plan) // ShuffleExchanges introduced by repartition do not support changing the number of partitions. // We change the number of partitions in the stage only if all the ShuffleExchanges support it. - if (!shuffleStages.forall(_.plan.canChangeNumPartitions)) { + if (!shuffleStages.forall(_.shuffle.canChangeNumPartitions)) { plan } else { val shuffleMetrics = shuffleStages.map { stage => - val metricsFuture = stage.plan.mapOutputStatisticsFuture - assert(metricsFuture.isCompleted, "ShuffleQueryStageExec should already be ready") - ThreadUtils.awaitResult(metricsFuture, Duration.Zero) + assert(stage.resultOption.isDefined, "ShuffleQueryStageExec should already be ready") + stage.resultOption.get.asInstanceOf[MapOutputStatistics] } // `ShuffleQueryStageExec` gives null mapOutputStatistics when the input RDD has 0 partitions, @@ -97,7 +93,7 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // even for shuffle exchange whose input RDD has 0 partition, we should still update its // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same // number of output partitions. - case stage: QueryStageExec if ShuffleQueryStageExec.isShuffleQueryStageExec(stage) => + case stage: ShuffleQueryStageExec => CoalescedShuffleReaderExec(stage, partitionStartIndices) } } else { @@ -185,8 +181,8 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { * A wrapper of shuffle query stage, which submits fewer reduce task as one reduce task may read * multiple shuffle partitions. This can avoid many small reduce tasks that hurt performance. * - * @param child It's usually `ShuffleQueryStageExec` or `ReusedQueryStageExec`, but can be the - * shuffle exchange node during canonicalization. + * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange node during + * canonicalization. */ case class CoalescedShuffleReaderExec( child: SparkPlan, @@ -204,9 +200,9 @@ case class CoalescedShuffleReaderExec( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.plan.createShuffledRDD(Some(partitionStartIndices)) - case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => - stage.plan.createShuffledRDD(Some(partitionStartIndices)) + stage.shuffle.createShuffledRDD(Some(partitionStartIndices)) + case _ => + throw new IllegalStateException("operating on canonicalization plan") } } cachedShuffleRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala index a1dde415d6e8b..849ff384c130a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/Exchange.scala @@ -71,7 +71,7 @@ case class ReusedExchangeExec(override val output: Seq[Attribute], child: Exchan // `ReusedExchangeExec` can have distinct set of output attribute ids from its child, we need // to update the attribute ids in `outputPartitioning` and `outputOrdering`. - private lazy val updateAttr: Expression => Expression = { + private[sql] lazy val updateAttr: Expression => Expression = { val originalAttrToNewAttr = AttributeMap(child.output.zip(output)) e => e.transform { case attr: Attribute => originalAttrToNewAttr.getOrElse(attr, attr) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index fe07b1ff109b6..70d5a8a615ad3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.adaptive._ import org.apache.spark.sql.execution.adaptive.{CoalescedShuffleReaderExec, ReduceNumShufflePartitions} +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -530,7 +531,9 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA QueryTest.checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) val finalPlan = resultDf.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec].executedPlan - assert(finalPlan.collect { case p: ReusedQueryStageExec => p }.length == 2) + assert(finalPlan.collect { + case ShuffleQueryStageExec(_, r: ReusedExchangeExec) => r + }.length == 2) assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) @@ -561,7 +564,9 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA assert(leafStages.length == 2) val reusedStages = level1Stages.flatMap { stage => - stage.plan.collect { case r: ReusedQueryStageExec => r } + stage.plan.collect { + case ShuffleQueryStageExec(_, r: ReusedExchangeExec) => r + } } assert(reusedStages.length == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 4fa4dadf7f654..847280b78e469 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} -import org.apache.spark.sql.execution.exchange.Exchange +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildRight, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -65,9 +65,10 @@ class AdaptiveQueryExecSuite } } - private def findReusedExchange(plan: SparkPlan): Seq[ReusedQueryStageExec] = { + private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { collectInPlanAndSubqueries(plan) { - case e: ReusedQueryStageExec => e + case ShuffleQueryStageExec(_, e: ReusedExchangeExec) => e + case BroadcastQueryStageExec(_, e: ReusedExchangeExec) => e } } @@ -428,7 +429,7 @@ class AdaptiveQueryExecSuite // Even with local shuffle reader, the query stage reuse can also work. val ex = findReusedExchange(adaptivePlan) assert(ex.nonEmpty) - assert(ex.head.plan.isInstanceOf[BroadcastQueryStageExec]) + assert(ex.head.child.isInstanceOf[BroadcastExchangeExec]) val sub = findReusedSubquery(adaptivePlan) assert(sub.isEmpty) } From a296d15235566a6592c5ddbe459f0a6fff578aee Mon Sep 17 00:00:00 2001 From: jiake Date: Fri, 20 Dec 2019 00:23:26 -0800 Subject: [PATCH 0926/1387] [SPARK-30291] catch the exception when doing materialize in AQE ### What changes were proposed in this pull request? AQE need catch the exception when doing materialize. And then user can get more information about the exception when enable AQE. ### Why are the changes needed? provide more cause about the exception when doing materialize ### Does this PR introduce any user-facing change? Before this PR, the error in the added unit test is java.lang.RuntimeException: Invalid bucket file file:///${SPARK_HOME}/assembly/spark-warehouse/org.apache.spark.sql.execution.adaptive.AdaptiveQueryExecSuite/bucketed_table/part-00000-3551343c-d003-4ada-82c8-45c712a72efe-c000.snappy.parquet After this PR, the error in the added unit test is: org.apache.spark.SparkException: Adaptive execution failed due to stage materialization failures. ### How was this patch tested? Add a new ut Closes #26931 from JkSelf/catchMoreException. Authored-by: jiake Signed-off-by: Xiao Li --- .../adaptive/AdaptiveSparkPlanExec.scala | 18 +++++++------ .../adaptive/AdaptiveQueryExecSuite.scala | 25 +++++++++++++++++++ 2 files changed, 36 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 1f1b020ced125..9e2cbbabe3046 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -148,13 +148,17 @@ case class AdaptiveSparkPlanExec( // Start materialization of all new stages. result.newStages.foreach { stage => - stage.materialize().onComplete { res => - if (res.isSuccess) { - events.offer(StageSuccess(stage, res.get)) - } else { - events.offer(StageFailure(stage, res.failed.get)) - } - }(AdaptiveSparkPlanExec.executionContext) + try { + stage.materialize().onComplete { res => + if (res.isSuccess) { + events.offer(StageSuccess(stage, res.get)) + } else { + events.offer(StageFailure(stage, res.failed.get)) + } + }(AdaptiveSparkPlanExec.executionContext) + } catch { + case e: Throwable => events.offer(StageFailure(stage, e)) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 847280b78e469..8af9b8f004265 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.execution.adaptive +import java.io.File +import java.net.URI + import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} @@ -24,6 +27,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildRight, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.util.Utils class AdaptiveQueryExecSuite extends QueryTest @@ -553,4 +557,25 @@ class AdaptiveQueryExecSuite spark.sparkContext.removeSparkListener(listener) } } + + test("SPARK-30291: AQE should catch the exceptions when doing materialize") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withTable("bucketed_table") { + val df1 = + (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k").as("df1") + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") + val warehouseFilePath = new URI(spark.sessionState.conf.warehousePath).getPath + val tableDir = new File(warehouseFilePath, "bucketed_table") + Utils.deleteRecursively(tableDir) + df1.write.parquet(tableDir.getAbsolutePath) + + val agged = spark.table("bucketed_table").groupBy("i").count() + val error = intercept[Exception] { + agged.count() + } + assert(error.getCause().toString contains "Failed to materialize query stage") + } + } + } } From 12249fcdc7534c8be67b9331b1a4dfdeb7724d63 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 20 Dec 2019 19:21:43 +0800 Subject: [PATCH 0927/1387] [SPARK-30301][SQL] Fix wrong results when datetimes as fields of complex types MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? When date and timestamp values are fields of arrays, maps, etc, we convert them to hive string using `toString`. This makes the result wrong before the default transition ’1582-10-15‘. https://bugs.openjdk.java.net/browse/JDK-8061577?focusedCommentId=13566712&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-13566712 cases to reproduce: ```sql +-- !query 47 +select array(cast('1582-10-13' as date), date '1582-10-14', date '1582-10-15', null) +-- !query 47 schema +struct> +-- !query 47 output +[1582-10-03,1582-10-04,1582-10-15,null] + + +-- !query 48 +select cast('1582-10-13' as date), date '1582-10-14', date '1582-10-15' +-- !query 48 schema +struct +-- !query 48 output +1582-10-13 1582-10-14 1582-10-15 ``` other refencences https://github.com/h2database/h2database/issues/831 ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? yes, complex types containing datetimes in `spark-sql `script and thrift server can result same as self-contained spark app or `spark-shell` script ### How was this patch tested? add uts Closes #26942 from yaooqinn/SPARK-30301. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../catalyst/expressions/csvExpressions.scala | 2 +- .../expressions/jsonExpressions.scala | 2 +- .../spark/sql/execution/HiveResult.scala | 75 +++++-------------- .../resources/sql-tests/results/array.sql.out | 4 +- .../sql-tests/results/csv-functions.sql.out | 2 +- .../sql-tests/results/inline-table.sql.out | 2 +- .../sql-tests/results/json-functions.sql.out | 2 +- .../typeCoercion/native/concat.sql.out | 2 +- .../typeCoercion/native/mapZipWith.sql.out | 2 +- .../typeCoercion/native/mapconcat.sql.out | 2 +- .../results/udf/udf-inline-table.sql.out | 2 +- .../spark/sql/execution/HiveResultSuite.scala | 30 +++++--- 12 files changed, 50 insertions(+), 77 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala index 67c24f687af08..73d329b4f5829 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -42,7 +42,7 @@ import org.apache.spark.unsafe.types.UTF8String > SELECT _FUNC_('1, 0.8', 'a INT, b DOUBLE'); {"a":1,"b":0.8} > SELECT _FUNC_('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); - {"time":2015-08-26 00:00:00.0} + {"time":2015-08-26 00:00:00} """, since = "3.0.0") // scalastyle:on line.size.limit diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index a1e3a84bd045d..3c08d866444d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -505,7 +505,7 @@ case class JsonTuple(children: Seq[Expression]) > SELECT _FUNC_('{"a":1, "b":0.8}', 'a INT, b DOUBLE'); {"a":1,"b":0.8} > SELECT _FUNC_('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); - {"time":2015-08-26 00:00:00.0} + {"time":2015-08-26 00:00:00} """, since = "2.2.0") // scalastyle:on line.size.limit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index f7f7e08462fe1..c92b10cc03645 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -56,78 +56,41 @@ object HiveResult { // We need the types so we can output struct field names val types = executedPlan.output.map(_.dataType) // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) + result.map(_.zip(types).map(e => toHiveString(e))) + .map(_.mkString("\t")) } - private val primitiveTypes = Seq( - StringType, - IntegerType, - LongType, - DoubleType, - FloatType, - BooleanType, - ByteType, - ShortType, - DateType, - TimestampType, - BinaryType) - private lazy val zoneId = DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone) private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) - /** Hive outputs fields of structs slightly differently than top level attributes. */ - private def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (decimal, DecimalType()) => decimal.toString - case (interval: CalendarInterval, CalendarIntervalType) => - SQLConf.get.intervalOutputStyle match { - case SQL_STANDARD => toSqlStandardString(interval) - case ISO_8601 => toIso8601String(interval) - case MULTI_UNITS => toMultiUnitsString(interval) - } - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - /** Formats a datum (based on the given data type) and returns the string representation. */ - def toHiveString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" + def toHiveString(a: (Any, DataType), nested: Boolean = false): String = a match { + case (null, _) => if (nested) "null" else "NULL" + case (b, BooleanType) => b.toString case (d: Date, DateType) => dateFormatter.format(DateTimeUtils.fromJavaDate(d)) case (t: Timestamp, TimestampType) => - DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t)) + timestampFormatter.format(DateTimeUtils.fromJavaTimestamp(t)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString + case (n, _: NumericType) => n.toString + case (s: String, StringType) => if (nested) "\"" + s + "\"" else s case (interval: CalendarInterval, CalendarIntervalType) => SQLConf.get.intervalOutputStyle match { case SQL_STANDARD => toSqlStandardString(interval) case ISO_8601 => toIso8601String(interval) case MULTI_UNITS => toMultiUnitsString(interval) } - case (interval, CalendarIntervalType) => interval.toString - case (other, _ : UserDefinedType[_]) => other.toString - case (other, tpe) if primitiveTypes.contains(tpe) => other.toString + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(e => toHiveString(e, true)).mkString("[", ",", "]") + case (m: Map[_, _], MapType(kType, vType, _)) => + m.map { case (key, value) => + toHiveString((key, kType), true) + ":" + toHiveString((value, vType), true) + }.toSeq.sorted.mkString("{", ",", "}") + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { case (v, t) => + s""""${t.name}":${toHiveString((v, t.dataType), true)}""" + }.mkString("{", ",", "}") + case (other, _: UserDefinedType[_]) => other.toString } } diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 4195205e275ae..31987de5da9bd 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -75,7 +75,7 @@ select * from primitive_arrays -- !query 5 schema struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,date_array:array,timestamp_array:array> -- !query 5 output -[true] [2,1] [2,1] [2,1] [2,1] [9223372036854775809,9223372036854775808] [2.0,1.0] [2.0,1.0] [2016-03-14,2016-03-13] [2016-11-15 20:54:00.0,2016-11-12 20:54:00.0] +[true] [2,1] [2,1] [2,1] [2,1] [9223372036854775809,9223372036854775808] [2.0,1.0] [2.0,1.0] [2016-03-14,2016-03-13] [2016-11-15 20:54:00,2016-11-12 20:54:00] -- !query 6 @@ -122,7 +122,7 @@ from primitive_arrays -- !query 8 schema struct,sort_array(tinyint_array, true):array,sort_array(smallint_array, true):array,sort_array(int_array, true):array,sort_array(bigint_array, true):array,sort_array(decimal_array, true):array,sort_array(double_array, true):array,sort_array(float_array, true):array,sort_array(date_array, true):array,sort_array(timestamp_array, true):array> -- !query 8 output -[true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00.0,2016-11-15 20:54:00.0] +[true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00,2016-11-15 20:54:00] -- !query 9 diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out index 03d4bfffa8923..6cae948f1b4b7 100644 --- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -15,7 +15,7 @@ select from_csv('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yy -- !query 1 schema struct> -- !query 1 output -{"time":2015-08-26 00:00:00.0} +{"time":2015-08-26 00:00:00} -- !query 2 diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out index 4e80f0bda5513..bf5a560ec0722 100644 --- a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -150,4 +150,4 @@ select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991- -- !query 16 schema struct> -- !query 16 output -1991-12-06 00:00:00 [1991-12-06 01:00:00.0,1991-12-06 12:00:00.0] +1991-12-06 00:00:00 [1991-12-06 01:00:00,1991-12-06 12:00:00] diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index ca0cd90d94fa7..1f6e5e78ca226 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -106,7 +106,7 @@ select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat -- !query 12 schema struct> -- !query 12 output -{"time":2015-08-26 00:00:00.0} +{"time":2015-08-26 00:00:00} -- !query 13 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out index 6c6d3110d7d0d..d7ebc3c77ed5d 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out @@ -298,7 +298,7 @@ FROM various_arrays -- !query 12 schema struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,data_array:array,timestamp_array:array,string_array:array,array_array:array>,struct_array:array>,map_array:array>> -- !query 12 output -[true,false,true] [2,1,3,4] [2,1,3,4] [2,1,3,4] [2,1,3,4] [9223372036854775809,9223372036854775808,9223372036854775808,9223372036854775809] [2.0,1.0,3.0,4.0] [2.0,1.0,3.0,4.0] [2016-03-14,2016-03-13,2016-03-12,2016-03-11] [2016-11-15 20:54:00.0,2016-11-12 20:54:00.0,2016-11-11 20:54:00.0] ["a","b","c","d"] [["a","b"],["c","d"],["e"],["f"]] [{"col1":"a","col2":1},{"col1":"b","col2":2},{"col1":"c","col2":3},{"col1":"d","col2":4}] [{"a":1},{"b":2},{"c":3},{"d":4}] +[true,false,true] [2,1,3,4] [2,1,3,4] [2,1,3,4] [2,1,3,4] [9223372036854775809,9223372036854775808,9223372036854775808,9223372036854775809] [2.0,1.0,3.0,4.0] [2.0,1.0,3.0,4.0] [2016-03-14,2016-03-13,2016-03-12,2016-03-11] [2016-11-15 20:54:00,2016-11-12 20:54:00,2016-11-11 20:54:00] ["a","b","c","d"] [["a","b"],["c","d"],["e"],["f"]] [{"col1":"a","col2":1},{"col1":"b","col2":2},{"col1":"c","col2":3},{"col1":"d","col2":4}] [{"a":1},{"b":2},{"c":3},{"d":4}] -- !query 13 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out index 86a578ca013df..2fdaf63cd3bf1 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out @@ -149,7 +149,7 @@ FROM various_maps -- !query 12 schema struct>> -- !query 12 output -{"2016-11-15 20:54:00":{"k":"2016-11-15 20:54:00","v1":2016-11-12 20:54:00.0,"v2":null},"2016-11-15 20:54:00.000":{"k":"2016-11-15 20:54:00.000","v1":null,"v2":"2016-11-12 20:54:00.000"}} +{"2016-11-15 20:54:00":{"k":"2016-11-15 20:54:00","v1":2016-11-12 20:54:00,"v2":null},"2016-11-15 20:54:00.000":{"k":"2016-11-15 20:54:00.000","v1":null,"v2":"2016-11-12 20:54:00.000"}} -- !query 13 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out index 79e00860e4c05..ee6d62b483884 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out @@ -65,7 +65,7 @@ FROM various_maps -- !query 1 schema struct,tinyint_map:map,smallint_map:map,int_map:map,bigint_map:map,decimal_map:map,float_map:map,double_map:map,date_map:map,timestamp_map:map,string_map:map,array_map:map,array>,struct_map:map,struct>,string_int_map:map,int_string_map:map> -- !query 1 output -{false:true,true:false} {1:2,3:4} {1:2,3:4} {4:6,7:8} {6:7,8:9} {9223372036854775808:9223372036854775809,9223372036854775809:9223372036854775808} {1.0:2.0,3.0:4.0} {1.0:2.0,3.0:4.0} {2016-03-12:2016-03-11,2016-03-14:2016-03-13} {2016-11-11 20:54:00.0:2016-11-09 20:54:00.0,2016-11-15 20:54:00.0:2016-11-12 20:54:00.0} {"a":"b","c":"d"} {["a","b"]:["c","d"],["e"]:["f"]} {{"col1":"a","col2":1}:{"col1":"b","col2":2},{"col1":"c","col2":3}:{"col1":"d","col2":4}} {"a":1,"c":2} {1:"a",2:"c"} +{false:true,true:false} {1:2,3:4} {1:2,3:4} {4:6,7:8} {6:7,8:9} {9223372036854775808:9223372036854775809,9223372036854775809:9223372036854775808} {1.0:2.0,3.0:4.0} {1.0:2.0,3.0:4.0} {2016-03-12:2016-03-11,2016-03-14:2016-03-13} {2016-11-11 20:54:00:2016-11-09 20:54:00,2016-11-15 20:54:00:2016-11-12 20:54:00} {"a":"b","c":"d"} {["a","b"]:["c","d"],["e"]:["f"]} {{"col1":"a","col2":1}:{"col1":"b","col2":2},{"col1":"c","col2":3}:{"col1":"d","col2":4}} {"a":1,"c":2} {1:"a",2:"c"} -- !query 2 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out index 2cf24e50c80a5..9203c2b31dc11 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out @@ -150,4 +150,4 @@ select udf(a), b from values (timestamp('1991-12-06 00:00:00.0'), array(timestam -- !query 16 schema struct> -- !query 16 output -1991-12-06 00:00:00 [1991-12-06 01:00:00.0,1991-12-06 12:00:00.0] +1991-12-06 00:00:00 [1991-12-06 01:00:00,1991-12-06 12:00:00] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index 4d1bb470e4e2e..bb59b12e6f350 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -17,25 +17,35 @@ package org.apache.spark.sql.execution -import java.sql.{Date, Timestamp} - import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} class HiveResultSuite extends SharedSparkSession { import testImplicits._ test("date formatting in hive result") { - val date = "2018-12-28" - val executedPlan = Seq(Date.valueOf(date)).toDS().queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan) - assert(result.head == date) + val dates = Seq("2018-12-28", "1582-10-13", "1582-10-14", "1582-10-15") + val df = dates.toDF("a").selectExpr("cast(a as date) as b") + val executedPlan1 = df.queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan1) + assert(result == dates) + val executedPlan2 = df.selectExpr("array(b)").queryExecution.executedPlan + val result2 = HiveResult.hiveResultString(executedPlan2) + assert(result2 == dates.map(x => s"[$x]")) } test("timestamp formatting in hive result") { - val timestamp = "2018-12-28 01:02:03" - val executedPlan = Seq(Timestamp.valueOf(timestamp)).toDS().queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan) - assert(result.head == timestamp) + val timestamps = Seq( + "2018-12-28 01:02:03", + "1582-10-13 01:02:03", + "1582-10-14 01:02:03", + "1582-10-15 01:02:03") + val df = timestamps.toDF("a").selectExpr("cast(a as timestamp) as b") + val executedPlan1 = df.queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan1) + assert(result == timestamps) + val executedPlan2 = df.selectExpr("array(b)").queryExecution.executedPlan + val result2 = HiveResult.hiveResultString(executedPlan2) + assert(result2 == timestamps.map(x => s"[$x]")) } test("toHiveString correctly handles UDTs") { From 0d2ef3ae2b2d0b66f763d6bb2e490a667c83f9f2 Mon Sep 17 00:00:00 2001 From: Niranjan Artal Date: Fri, 20 Dec 2019 07:29:28 -0600 Subject: [PATCH 0928/1387] [SPARK-30300][SQL][WEB-UI] Fix updating the UI max value string when driver updates the same metric id as the tasks ### What changes were proposed in this pull request? In this PR, For a given metrics id we are checking if the driver side accumulator's value is greater than max of all stages value. If it's true, then we are removing that entry from the Hashmap. By doing this, for this metrics, "driver" would be displayed on the UI(As the driver would have the maximum value) ### Why are the changes needed? This PR fixes https://issues.apache.org/jira/browse/SPARK-30300. Currently driver's metric value is not compared while caluculating the max. ### Does this PR introduce any user-facing change? For the metrics where driver's value is greater than max of all stages, this is the change. Previous : (min, median, max (stageId 0( attemptId 1): taskId 2)) Now: (min, median, max (driver)) ### How was this patch tested? Ran unit tests. Closes #26941 from nartal1/SPARK-30300. Authored-by: Niranjan Artal Signed-off-by: Thomas Graves --- .../spark/sql/execution/ui/SQLAppStatusListener.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index 64d2f33be5df0..d5bb36e8a0c95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -237,6 +237,12 @@ class SQLAppStatusListener( if (metricTypes.contains(id)) { val prev = allMetrics.getOrElse(id, null) val updated = if (prev != null) { + // If the driver updates same metrics as tasks and has higher value then remove + // that entry from maxMetricsFromAllStage. This would make stringValue function default + // to "driver" that would be displayed on UI. + if (maxMetricsFromAllStages.contains(id) && value > maxMetricsFromAllStages(id)(0)) { + maxMetricsFromAllStages.remove(id) + } val _copy = Arrays.copyOf(prev, prev.length + 1) _copy(prev.length) = value _copy From 07b04c4c72ef0a1c6631afc2dc6d9be9817e3190 Mon Sep 17 00:00:00 2001 From: Prakhar Jain Date: Fri, 20 Dec 2019 08:54:14 -0600 Subject: [PATCH 0929/1387] [SPARK-29938][SQL] Add batching support in Alter table add partition flow ### What changes were proposed in this pull request? Add batching support in Alter table add partition flow. Also calculate new partition sizes faster by doing listing in parallel. ### Why are the changes needed? This PR split the the single createPartitions() call AlterTableAddPartition flow into smaller batches, which could prevent - SocketTimeoutException: Adding thousand of partitions in Hive metastore itself takes lot of time. Because of this hive client fails with SocketTimeoutException. - Hive metastore from OOM (caused by millions of partitions). It will also try to gather stats (total size of all files in all new partitions) faster by parallely listing the new partition paths. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added UT. Also tested on a cluster in HDI with 15000 partitions with remote metastore server. Without batching - operation fails with SocketTimeoutException, With batching it finishes in 25 mins. Closes #26569 from prakharjain09/add_partition_batching_r1. Authored-by: Prakhar Jain Signed-off-by: Sean Owen --- .../spark/sql/execution/command/ddl.scala | 22 ++++++++++++++----- .../spark/sql/sources/InsertSuite.scala | 14 ++++++++++++ 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index bdba10eb488d7..f8299eee8669a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -476,14 +476,26 @@ case class AlterTableAddPartitionCommand( CatalogTablePartition(normalizedSpec, table.storage.copy( locationUri = location.map(CatalogUtils.stringToURI))) } - catalog.createPartitions(table.identifier, parts, ignoreIfExists = ifNotExists) + + // Hive metastore may not have enough memory to handle millions of partitions in single RPC. + // Also the request to metastore times out when adding lot of partitions in one shot. + // we should split them into smaller batches + val batchSize = 100 + parts.toIterator.grouped(batchSize).foreach { batch => + catalog.createPartitions(table.identifier, batch, ignoreIfExists = ifNotExists) + } if (table.stats.nonEmpty) { if (sparkSession.sessionState.conf.autoSizeUpdateEnabled) { - val addedSize = parts.map { part => - CommandUtils.calculateLocationSize(sparkSession.sessionState, table.identifier, - part.storage.locationUri) - }.sum + def calculatePartSize(part: CatalogTablePartition) = CommandUtils.calculateLocationSize( + sparkSession.sessionState, table.identifier, part.storage.locationUri) + val threshold = sparkSession.sparkContext.conf.get(RDD_PARALLEL_LISTING_THRESHOLD) + val partSizes = if (parts.length > threshold) { + ThreadUtils.parmap(parts, "gatheringNewPartitionStats", 8)(calculatePartSize) + } else { + parts.map(calculatePartSize) + } + val addedSize = partSizes.sum if (addedSize > 0) { val newStats = CatalogStatistics(sizeInBytes = table.stats.get.sizeInBytes + addedSize) catalog.alterTableStats(table.identifier, Some(newStats)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 871cb1ff151ba..bcff30a51c3f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -521,6 +521,20 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } } + test("new partitions should be added to catalog after writing to catalog table") { + val table = "partitioned_catalog_table" + val numParts = 210 + withTable(table) { + val df = (1 to numParts).map(i => (i, i)).toDF("part", "col1") + val tempTable = "partitioned_catalog_temp_table" + df.createOrReplaceTempView(tempTable) + sql(s"CREATE TABLE $table (part Int, col1 Int) USING parquet PARTITIONED BY (part)") + sql(s"INSERT INTO TABLE $table SELECT * from $tempTable") + val partitions = spark.sessionState.catalog.listPartitionNames(TableIdentifier(table)) + assert(partitions.size == numParts) + } + } + test("SPARK-20236: dynamic partition overwrite without catalog table") { withSQLConf(SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString) { withTempPath { path => From 7dff3b125de23a4d6ce834217ee08973b259414c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 20 Dec 2019 08:55:04 -0600 Subject: [PATCH 0930/1387] [SPARK-30272][SQL][CORE] Remove usage of Guava that breaks in 27; replace with workalikes ### What changes were proposed in this pull request? Remove usages of Guava that no longer work in Guava 27, and replace with workalikes. I'll comment on key types of changes below. ### Why are the changes needed? Hadoop 3.2.1 uses Guava 27, so this helps us avoid problems running on Hadoop 3.2.1+ and generally lowers our exposure to Guava. ### Does this PR introduce any user-facing change? Should not be, but see notes below on hash codes and toString. ### How was this patch tested? Existing tests will verify whether these changes break anything for Guava 14. I manually built with an updated version and it compiles with Guava 27; tests running manually locally now. Closes #26911 from srowen/SPARK-30272. Authored-by: Sean Owen Signed-off-by: Sean Owen --- .../spark/util/kvstore/CustomType1.java | 9 +- .../buffer/FileSegmentManagedBuffer.java | 11 +-- .../network/buffer/NettyManagedBuffer.java | 7 +- .../network/buffer/NioManagedBuffer.java | 7 +- .../spark/network/client/TransportClient.java | 11 +-- .../network/protocol/ChunkFetchFailure.java | 13 +-- .../network/protocol/ChunkFetchRequest.java | 7 +- .../network/protocol/ChunkFetchSuccess.java | 13 +-- .../spark/network/protocol/OneWayMessage.java | 9 +- .../spark/network/protocol/RpcFailure.java | 13 +-- .../spark/network/protocol/RpcRequest.java | 13 +-- .../spark/network/protocol/RpcResponse.java | 13 +-- .../spark/network/protocol/StreamChunkId.java | 13 +-- .../spark/network/protocol/StreamFailure.java | 13 +-- .../spark/network/protocol/StreamRequest.java | 9 +- .../network/protocol/StreamResponse.java | 15 ++-- .../spark/network/protocol/UploadStream.java | 9 +- .../shuffle/ExternalShuffleBlockResolver.java | 13 +-- .../shuffle/protocol/BlocksRemoved.java | 9 +- .../shuffle/protocol/ExecutorShuffleInfo.java | 16 ++-- .../shuffle/protocol/FetchShuffleBlocks.java | 17 ++-- .../protocol/GetLocalDirsForExecutors.java | 10 ++- .../protocol/LocalDirsForExecutors.java | 11 +-- .../network/shuffle/protocol/OpenBlocks.java | 18 ++-- .../shuffle/protocol/RegisterExecutor.java | 21 +++-- .../shuffle/protocol/RemoveBlocks.java | 23 ++--- .../shuffle/protocol/StreamHandle.java | 17 ++-- .../network/shuffle/protocol/UploadBlock.java | 24 +++--- .../shuffle/protocol/UploadBlockStream.java | 12 +-- ...anupNonShuffleServiceServedFilesSuite.java | 3 +- .../shuffle/ExternalShuffleCleanupSuite.java | 3 +- .../network/yarn/YarnShuffleService.java | 10 ++- .../scala/org/apache/spark/SparkEnv.scala | 5 +- .../deploy/history/FsHistoryProvider.scala | 3 +- .../org/apache/spark/rdd/HadoopRDD.scala | 4 +- .../spark/status/ElementTrackingStore.scala | 8 +- .../org/apache/spark/util/ThreadUtils.scala | 85 +++++++++++++++++-- .../scala/org/apache/spark/util/Utils.scala | 6 +- .../sort/UnsafeShuffleWriterSuite.java | 3 +- .../scheduler/TaskResultGetterSuite.scala | 5 +- .../kinesis/KPLBasedKinesisTestUtils.scala | 7 +- .../sql/catalyst/JavaTypeInference.scala | 23 +++-- .../sql/JavaBeanDeserializationSuite.java | 36 ++++---- 43 files changed, 360 insertions(+), 217 deletions(-) diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java index 92b643b0cb928..ebb5c2c5ed55c 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java @@ -17,8 +17,6 @@ package org.apache.spark.util.kvstore; -import com.google.common.base.Objects; - public class CustomType1 { @KVIndex @@ -52,12 +50,7 @@ public int hashCode() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("key", key) - .add("id", id) - .add("name", name) - .add("num", num) - .toString(); + return "CustomType1[key=" + key + ",id=" + id + ",name=" + name + ",num=" + num; } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 45fee541a4f5d..66566b67870f3 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -26,9 +26,10 @@ import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; -import com.google.common.base.Objects; import com.google.common.io.ByteStreams; import io.netty.channel.DefaultFileRegion; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.LimitedInputStream; @@ -144,10 +145,10 @@ public Object convertToNetty() throws IOException { @Override public String toString() { - return Objects.toStringHelper(this) - .add("file", file) - .add("offset", offset) - .add("length", length) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("file", file) + .append("offset", offset) + .append("length", length) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java index acc49d968c186..b42977c7cb7f6 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java @@ -21,9 +21,10 @@ import java.io.InputStream; import java.nio.ByteBuffer; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufInputStream; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}. @@ -69,8 +70,8 @@ public Object convertToNetty() throws IOException { @Override public String toString() { - return Objects.toStringHelper(this) - .add("buf", buf) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("buf", buf) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java index 631d767715256..084f89d2611cf 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java @@ -21,9 +21,10 @@ import java.io.InputStream; import java.nio.ByteBuffer; -import com.google.common.base.Objects; import io.netty.buffer.ByteBufInputStream; import io.netty.buffer.Unpooled; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * A {@link ManagedBuffer} backed by {@link ByteBuffer}. @@ -67,8 +68,8 @@ public Object convertToNetty() throws IOException { @Override public String toString() { - return Objects.toStringHelper(this) - .add("buf", buf) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("buf", buf) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index b018197deaf2e..6dcc703e92669 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -27,13 +27,14 @@ import javax.annotation.Nullable; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.util.concurrent.SettableFuture; import io.netty.channel.Channel; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -301,10 +302,10 @@ public void close() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("remoteAdress", channel.remoteAddress()) - .add("clientId", clientId) - .add("isActive", isActive()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("remoteAdress", channel.remoteAddress()) + .append("clientId", clientId) + .append("isActive", isActive()) .toString(); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java index a7afbfa8621c8..0f1781cbf1f2c 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * Response to {@link ChunkFetchRequest} when there is an error fetching the chunk. @@ -54,7 +57,7 @@ public static ChunkFetchFailure decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(streamChunkId, errorString); + return Objects.hash(streamChunkId, errorString); } @Override @@ -68,9 +71,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamChunkId", streamChunkId) - .add("errorString", errorString) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkId) + .append("errorString", errorString) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java index fe54fcc50dc86..7b034d5c2f595 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java @@ -17,8 +17,9 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * Request to fetch a sequence of a single chunk of a stream. This will correspond to a single @@ -64,8 +65,8 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamChunkId", streamChunkId) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkId) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java index d5c9a9b3202fb..eaad143fc3f5f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NettyManagedBuffer; @@ -67,7 +70,7 @@ public static ChunkFetchSuccess decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(streamChunkId, body()); + return Objects.hash(streamChunkId, body()); } @Override @@ -81,9 +84,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamChunkId", streamChunkId) - .add("buffer", body()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkId) + .append("buffer", body()) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java index 1632fb9e03687..719f6c64c5dee 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NettyManagedBuffer; @@ -72,8 +75,8 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("body", body()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("body", body()) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java index 61061903de23f..6e4f5687d16cd 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** Response to {@link RpcRequest} for a failed RPC. */ public final class RpcFailure extends AbstractMessage implements ResponseMessage { @@ -52,7 +55,7 @@ public static RpcFailure decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(requestId, errorString); + return Objects.hash(requestId, errorString); } @Override @@ -66,9 +69,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("errorString", errorString) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("errorString", errorString) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java index cc1bb95d2d566..f2609ce2dbdb3 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NettyManagedBuffer; @@ -64,7 +67,7 @@ public static RpcRequest decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(requestId, body()); + return Objects.hash(requestId, body()); } @Override @@ -78,9 +81,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("body", body()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("body", body()) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java index c03291e9c0b23..51b36ea183362 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NettyManagedBuffer; @@ -64,7 +67,7 @@ public static RpcResponse decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(requestId, body()); + return Objects.hash(requestId, body()); } @Override @@ -78,9 +81,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("body", body()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("body", body()) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java index d46a263884807..75c6d630b9c33 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * Encapsulates a request for a particular chunk of a stream. @@ -51,7 +54,7 @@ public static StreamChunkId decode(ByteBuf buffer) { @Override public int hashCode() { - return Objects.hashCode(streamId, chunkIndex); + return Objects.hash(streamId, chunkIndex); } @Override @@ -65,9 +68,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamId", streamId) - .add("chunkIndex", chunkIndex) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("chunkIndex", chunkIndex) .toString(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java index 68fcfa7748611..06836f5eea390 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * Message indicating an error when transferring a stream. @@ -54,7 +57,7 @@ public static StreamFailure decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(streamId, error); + return Objects.hash(streamId, error); } @Override @@ -68,9 +71,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamId", streamId) - .add("error", error) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("error", error) .toString(); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java index 1b135af752bd8..3d035e5c94f23 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; /** * Request to stream data from the remote end. @@ -67,8 +70,8 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamId", streamId) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) .toString(); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java index 568108c4fe5e8..f30605ce836fc 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java @@ -17,8 +17,11 @@ package org.apache.spark.network.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.buffer.ManagedBuffer; @@ -67,7 +70,7 @@ public static StreamResponse decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(byteCount, streamId); + return Objects.hash(byteCount, streamId); } @Override @@ -81,10 +84,10 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamId", streamId) - .add("byteCount", byteCount) - .add("body", body()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("byteCount", byteCount) + .append("body", body()) .toString(); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java index 7d21151e01074..fb50801a51ba3 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java @@ -20,8 +20,9 @@ import java.io.IOException; import java.nio.ByteBuffer; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NettyManagedBuffer; @@ -99,9 +100,9 @@ public boolean equals(Object other) { @Override public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("body", body()) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("body", body()) .toString(); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 657774c1b468f..ba1a17bf7e5ea 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -27,12 +27,13 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.commons.lang3.tuple.Pair; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Objects; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; @@ -401,19 +402,19 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; AppExecId appExecId = (AppExecId) o; - return Objects.equal(appId, appExecId.appId) && Objects.equal(execId, appExecId.execId); + return Objects.equals(appId, appExecId.appId) && Objects.equals(execId, appExecId.execId); } @Override public int hashCode() { - return Objects.hashCode(appId, execId); + return Objects.hash(appId, execId); } @Override public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execId", execId) .toString(); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java index 723b2f75c6fc7..a4d6035df807c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java @@ -17,8 +17,11 @@ package org.apache.spark.network.shuffle.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; // Needed by ScalaDoc. See SPARK-7726 import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; @@ -41,8 +44,8 @@ public int hashCode() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("numRemovedBlocks", numRemovedBlocks) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("numRemovedBlocks", numRemovedBlocks) .toString(); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java index 540ecd09a7e30..b4e7bc409d3b8 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java @@ -18,11 +18,13 @@ package org.apache.spark.network.shuffle.protocol; import java.util.Arrays; +import java.util.Objects; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.protocol.Encodable; import org.apache.spark.network.protocol.Encoders; @@ -48,15 +50,15 @@ public ExecutorShuffleInfo( @Override public int hashCode() { - return Objects.hashCode(subDirsPerLocalDir, shuffleManager) * 41 + Arrays.hashCode(localDirs); + return Objects.hash(subDirsPerLocalDir, shuffleManager) * 41 + Arrays.hashCode(localDirs); } @Override public String toString() { - return Objects.toStringHelper(this) - .add("localDirs", Arrays.toString(localDirs)) - .add("subDirsPerLocalDir", subDirsPerLocalDir) - .add("shuffleManager", shuffleManager) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("localDirs", Arrays.toString(localDirs)) + .append("subDirsPerLocalDir", subDirsPerLocalDir) + .append("shuffleManager", shuffleManager) .toString(); } @@ -66,7 +68,7 @@ public boolean equals(Object other) { ExecutorShuffleInfo o = (ExecutorShuffleInfo) other; return Arrays.equals(localDirs, o.localDirs) && subDirsPerLocalDir == o.subDirsPerLocalDir - && Objects.equal(shuffleManager, o.shuffleManager); + && Objects.equals(shuffleManager, o.shuffleManager); } return false; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java index c0f307af042e2..98057d58f7ab5 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -19,8 +19,9 @@ import java.util.Arrays; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.protocol.Encoders; @@ -68,13 +69,13 @@ public FetchShuffleBlocks( @Override public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) - .add("shuffleId", shuffleId) - .add("mapIds", Arrays.toString(mapIds)) - .add("reduceIds", Arrays.deepToString(reduceIds)) - .add("batchFetchEnabled", batchFetchEnabled) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execId", execId) + .append("shuffleId", shuffleId) + .append("mapIds", Arrays.toString(mapIds)) + .append("reduceIds", Arrays.deepToString(reduceIds)) + .append("batchFetchEnabled", batchFetchEnabled) .toString(); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java index 90c416acc69ad..47f617c5e0a0a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/GetLocalDirsForExecutors.java @@ -18,9 +18,11 @@ package org.apache.spark.network.shuffle.protocol; import java.util.Arrays; +import java.util.Objects; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.protocol.Encoders; @@ -47,9 +49,9 @@ public int hashCode() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execIds", Arrays.toString(execIds)) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execIds", Arrays.toString(execIds)) .toString(); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java index 0c3aa6a461148..9e2f0668cbd24 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/LocalDirsForExecutors.java @@ -19,8 +19,9 @@ import java.util.*; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.protocol.Encoders; @@ -63,10 +64,10 @@ public int hashCode() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("execIds", Arrays.toString(execIds)) - .add("numLocalDirsByExec", Arrays.toString(numLocalDirsByExec)) - .add("allLocalDirs", Arrays.toString(allLocalDirs)) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("execIds", Arrays.toString(execIds)) + .append("numLocalDirsByExec", Arrays.toString(numLocalDirsByExec)) + .append("allLocalDirs", Arrays.toString(allLocalDirs)) .toString(); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java index ce954b8a289e4..771e17b3233ec 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java @@ -18,9 +18,11 @@ package org.apache.spark.network.shuffle.protocol; import java.util.Arrays; +import java.util.Objects; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.protocol.Encoders; @@ -44,15 +46,15 @@ public OpenBlocks(String appId, String execId, String[] blockIds) { @Override public int hashCode() { - return Objects.hashCode(appId, execId) * 41 + Arrays.hashCode(blockIds); + return Objects.hash(appId, execId) * 41 + Arrays.hashCode(blockIds); } @Override public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) - .add("blockIds", Arrays.toString(blockIds)) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execId", execId) + .append("blockIds", Arrays.toString(blockIds)) .toString(); } @@ -60,8 +62,8 @@ public String toString() { public boolean equals(Object other) { if (other != null && other instanceof OpenBlocks) { OpenBlocks o = (OpenBlocks) other; - return Objects.equal(appId, o.appId) - && Objects.equal(execId, o.execId) + return Objects.equals(appId, o.appId) + && Objects.equals(execId, o.execId) && Arrays.equals(blockIds, o.blockIds); } return false; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java index 167ef33104227..f6af755cd9cd5 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java @@ -17,8 +17,11 @@ package org.apache.spark.network.shuffle.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.protocol.Encoders; @@ -48,15 +51,15 @@ public RegisterExecutor( @Override public int hashCode() { - return Objects.hashCode(appId, execId, executorInfo); + return Objects.hash(appId, execId, executorInfo); } @Override public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) - .add("executorInfo", executorInfo) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execId", execId) + .append("executorInfo", executorInfo) .toString(); } @@ -64,9 +67,9 @@ public String toString() { public boolean equals(Object other) { if (other != null && other instanceof RegisterExecutor) { RegisterExecutor o = (RegisterExecutor) other; - return Objects.equal(appId, o.appId) - && Objects.equal(execId, o.execId) - && Objects.equal(executorInfo, o.executorInfo); + return Objects.equals(appId, o.appId) + && Objects.equals(execId, o.execId) + && Objects.equals(executorInfo, o.executorInfo); } return false; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java index 1c718d307753f..ade838bd4286c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java @@ -17,11 +17,14 @@ package org.apache.spark.network.shuffle.protocol; -import com.google.common.base.Objects; +import java.util.Arrays; +import java.util.Objects; + import io.netty.buffer.ByteBuf; -import org.apache.spark.network.protocol.Encoders; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; -import java.util.Arrays; +import org.apache.spark.network.protocol.Encoders; // Needed by ScalaDoc. See SPARK-7726 import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; @@ -43,15 +46,15 @@ public RemoveBlocks(String appId, String execId, String[] blockIds) { @Override public int hashCode() { - return Objects.hashCode(appId, execId) * 41 + Arrays.hashCode(blockIds); + return Objects.hash(appId, execId) * 41 + Arrays.hashCode(blockIds); } @Override public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) - .add("blockIds", Arrays.toString(blockIds)) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execId", execId) + .append("blockIds", Arrays.toString(blockIds)) .toString(); } @@ -59,8 +62,8 @@ public String toString() { public boolean equals(Object other) { if (other != null && other instanceof RemoveBlocks) { RemoveBlocks o = (RemoveBlocks) other; - return Objects.equal(appId, o.appId) - && Objects.equal(execId, o.execId) + return Objects.equals(appId, o.appId) + && Objects.equals(execId, o.execId) && Arrays.equals(blockIds, o.blockIds); } return false; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java index 1915295aa6cc2..dd7715a4e82d4 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java @@ -17,8 +17,11 @@ package org.apache.spark.network.shuffle.protocol; -import com.google.common.base.Objects; +import java.util.Objects; + import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; // Needed by ScalaDoc. See SPARK-7726 import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; @@ -41,14 +44,14 @@ public StreamHandle(long streamId, int numChunks) { @Override public int hashCode() { - return Objects.hashCode(streamId, numChunks); + return Objects.hash(streamId, numChunks); } @Override public String toString() { - return Objects.toStringHelper(this) - .add("streamId", streamId) - .add("numChunks", numChunks) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("numChunks", numChunks) .toString(); } @@ -56,8 +59,8 @@ public String toString() { public boolean equals(Object other) { if (other != null && other instanceof StreamHandle) { StreamHandle o = (StreamHandle) other; - return Objects.equal(streamId, o.streamId) - && Objects.equal(numChunks, o.numChunks); + return Objects.equals(streamId, o.streamId) + && Objects.equals(numChunks, o.numChunks); } return false; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java index 3caed59d508fd..a5bc3f7009b46 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java @@ -18,9 +18,11 @@ package org.apache.spark.network.shuffle.protocol; import java.util.Arrays; +import java.util.Objects; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.protocol.Encoders; @@ -60,18 +62,18 @@ public UploadBlock( @Override public int hashCode() { - int objectsHashCode = Objects.hashCode(appId, execId, blockId); + int objectsHashCode = Objects.hash(appId, execId, blockId); return (objectsHashCode * 41 + Arrays.hashCode(metadata)) * 41 + Arrays.hashCode(blockData); } @Override public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) - .add("blockId", blockId) - .add("metadata size", metadata.length) - .add("block size", blockData.length) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) + .append("execId", execId) + .append("blockId", blockId) + .append("metadata size", metadata.length) + .append("block size", blockData.length) .toString(); } @@ -79,9 +81,9 @@ public String toString() { public boolean equals(Object other) { if (other != null && other instanceof UploadBlock) { UploadBlock o = (UploadBlock) other; - return Objects.equal(appId, o.appId) - && Objects.equal(execId, o.execId) - && Objects.equal(blockId, o.blockId) + return Objects.equals(appId, o.appId) + && Objects.equals(execId, o.execId) + && Objects.equals(blockId, o.blockId) && Arrays.equals(metadata, o.metadata) && Arrays.equals(blockData, o.blockData); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlockStream.java index 9df30967d5bb2..958a84e516c81 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlockStream.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlockStream.java @@ -18,9 +18,11 @@ package org.apache.spark.network.shuffle.protocol; import java.util.Arrays; +import java.util.Objects; -import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.spark.network.protocol.Encoders; @@ -53,9 +55,9 @@ public int hashCode() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("blockId", blockId) - .add("metadata size", metadata.length) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("blockId", blockId) + .append("metadata size", metadata.length) .toString(); } @@ -63,7 +65,7 @@ public String toString() { public boolean equals(Object other) { if (other != null && other instanceof UploadBlockStream) { UploadBlockStream o = (UploadBlockStream) other; - return Objects.equal(blockId, o.blockId) + return Objects.equals(blockId, o.blockId) && Arrays.equals(metadata, o.metadata); } return false; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java index e38442327e22d..b37d8620a57f4 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -30,7 +30,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.util.concurrent.MoreExecutors; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -42,7 +41,7 @@ public class CleanupNonShuffleServiceServedFilesSuite { // Same-thread Executor used to ensure cleanup happens synchronously in test thread. - private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); + private Executor sameThreadExecutor = Runnable::run; private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 47c087088a8a2..48b73e32216ce 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -24,7 +24,6 @@ import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; -import com.google.common.util.concurrent.MoreExecutors; import org.junit.Test; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -35,7 +34,7 @@ public class ExternalShuffleCleanupSuite { // Same-thread Executor used to ensure cleanup happens synchronously in test thread. - private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); + private Executor sameThreadExecutor = Runnable::run; private TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index c170f99b112c0..815a56d765b6a 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -23,14 +23,16 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Objects; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -417,7 +419,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; AppId appExecId = (AppId) o; - return Objects.equal(appId, appExecId.appId); + return Objects.equals(appId, appExecId.appId); } @Override @@ -427,8 +429,8 @@ public int hashCode() { @Override public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("appId", appId) .toString(); } } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5fe1c663affa5..8ba1739831803 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -26,7 +26,7 @@ import scala.collection.concurrent import scala.collection.mutable import scala.util.Properties -import com.google.common.collect.MapMaker +import com.google.common.cache.CacheBuilder import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.DeveloperApi @@ -76,7 +76,8 @@ class SparkEnv ( // A general, soft-reference map for metadata needed during HadoopRDD split computation // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). - private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() + private[spark] val hadoopJobMetadata = + CacheBuilder.newBuilder().softValues().build[String, AnyRef]().asMap() private[spark] var driverTmpDir: Option[String] = None diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index f560b7e9157b5..54c50006c33b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -31,7 +31,6 @@ import scala.xml.Node import com.fasterxml.jackson.annotation.JsonIgnore import com.fasterxml.jackson.databind.annotation.JsonDeserialize -import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.hdfs.protocol.HdfsConstants @@ -214,7 +213,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) if (!Utils.isTesting) { ThreadUtils.newDaemonFixedThreadPool(NUM_PROCESSING_THREADS, "log-replay-executor") } else { - MoreExecutors.sameThreadExecutor() + ThreadUtils.sameThreadExecutorService } } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index ff4928dae6bf8..9742d12cfe01e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -405,9 +405,9 @@ private[spark] object HadoopRDD extends Logging { * The three methods below are helpers for accessing the local map, a property of the SparkEnv of * the local process. */ - def getCachedMetadata(key: String): Any = SparkEnv.get.hadoopJobMetadata.get(key) + def getCachedMetadata(key: String): AnyRef = SparkEnv.get.hadoopJobMetadata.get(key) - private def putCachedMetadata(key: String, value: Any): Unit = + private def putCachedMetadata(key: String, value: AnyRef): Unit = SparkEnv.get.hadoopJobMetadata.put(key, value) /** Add Hadoop configuration specific to a single partition and attempt. */ diff --git a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala index 38cb030297c81..1b8dc9c8275ad 100644 --- a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala +++ b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala @@ -18,14 +18,12 @@ package org.apache.spark.status import java.util.Collection -import java.util.concurrent.TimeUnit +import java.util.concurrent.{ExecutorService, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} -import com.google.common.util.concurrent.MoreExecutors - import org.apache.spark.SparkConf import org.apache.spark.internal.config.Status._ import org.apache.spark.status.ElementTrackingStore._ @@ -72,10 +70,10 @@ private[spark] class ElementTrackingStore(store: KVStore, conf: SparkConf) exten private val triggers = new HashMap[Class[_], LatchedTriggers]() private val flushTriggers = new ListBuffer[() => Unit]() - private val executor = if (conf.get(ASYNC_TRACKING_ENABLED)) { + private val executor: ExecutorService = if (conf.get(ASYNC_TRACKING_ENABLED)) { ThreadUtils.newDaemonSingleThreadExecutor("element-tracking-store-worker") } else { - MoreExecutors.sameThreadExecutor() + ThreadUtils.sameThreadExecutorService } @volatile private var stopped = false diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index 9ed95f6b7bd53..de39e4b410f25 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -18,22 +18,97 @@ package org.apache.spark.util import java.util.concurrent._ +import java.util.concurrent.locks.ReentrantLock -import scala.collection.generic.CanBuildFrom -import scala.language.higherKinds - -import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor, Future} import scala.concurrent.duration.{Duration, FiniteDuration} +import scala.language.higherKinds import scala.util.control.NonFatal +import com.google.common.util.concurrent.ThreadFactoryBuilder + import org.apache.spark.SparkException import org.apache.spark.rpc.RpcAbortException private[spark] object ThreadUtils { private val sameThreadExecutionContext = - ExecutionContext.fromExecutorService(MoreExecutors.sameThreadExecutor()) + ExecutionContext.fromExecutorService(sameThreadExecutorService()) + + // Inspired by Guava MoreExecutors.sameThreadExecutor; inlined and converted + // to Scala here to avoid Guava version issues + def sameThreadExecutorService(): ExecutorService = new AbstractExecutorService { + private val lock = new ReentrantLock() + private val termination = lock.newCondition() + private var runningTasks = 0 + private var serviceIsShutdown = false + + override def shutdown(): Unit = { + lock.lock() + try { + serviceIsShutdown = true + } finally { + lock.unlock() + } + } + + override def shutdownNow(): java.util.List[Runnable] = { + shutdown() + java.util.Collections.emptyList() + } + + override def isShutdown: Boolean = { + lock.lock() + try { + serviceIsShutdown + } finally { + lock.unlock() + } + } + + override def isTerminated: Boolean = synchronized { + lock.lock() + try { + serviceIsShutdown && runningTasks == 0 + } finally { + lock.unlock() + } + } + + override def awaitTermination(timeout: Long, unit: TimeUnit): Boolean = { + var nanos = unit.toNanos(timeout) + lock.lock() + try { + while (nanos > 0 && !isTerminated()) { + nanos = termination.awaitNanos(nanos) + } + isTerminated() + } finally { + lock.unlock() + } + } + + override def execute(command: Runnable): Unit = { + lock.lock() + try { + if (isShutdown()) throw new RejectedExecutionException("Executor already shutdown") + runningTasks += 1 + } finally { + lock.unlock() + } + try { + command.run() + } finally { + lock.lock() + try { + runningTasks -= 1 + if (isTerminated()) termination.signalAll() + } finally { + lock.unlock() + } + } + } + } /** * An `ExecutionContextExecutor` that runs each task in the thread that invokes `execute/submit`. 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 6b42815aa25ab..5af6a020d5175 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -28,7 +28,7 @@ import java.nio.channels.{Channels, FileChannel, WritableByteChannel} import java.nio.charset.StandardCharsets import java.nio.file.Files import java.security.SecureRandom -import java.util.{Locale, Properties, Random, UUID} +import java.util.{Arrays, Locale, Properties, Random, UUID} import java.util.concurrent._ import java.util.concurrent.TimeUnit.NANOSECONDS import java.util.zip.GZIPInputStream @@ -45,9 +45,9 @@ import scala.util.matching.Regex import _root_.io.netty.channel.unix.Errors.NativeIoException import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} -import com.google.common.hash.HashCodes import com.google.common.io.{ByteStreams, Files => GFiles} import com.google.common.net.InetAddresses +import org.apache.commons.codec.binary.Hex import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} @@ -2810,7 +2810,7 @@ private[spark] object Utils extends Logging { val rnd = new SecureRandom() val secretBytes = new Array[Byte](bits / JByte.SIZE) rnd.nextBytes(secretBytes) - HashCodes.fromBytes(secretBytes).toString() + Hex.encodeHexString(secretBytes) } /** diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 10e6936eb3799..ee8e38c24b47f 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -30,7 +30,6 @@ import scala.collection.Iterator; import com.google.common.collect.HashMultiset; -import com.google.common.collect.Iterators; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -249,7 +248,7 @@ class BadRecords extends scala.collection.AbstractIterator writer = createWriter(true); - writer.write(Iterators.emptyIterator()); + writer.write(new ArrayList>().iterator()); final Option mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 406bd9244870e..2efe6da5e986f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -25,7 +25,6 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.util.control.NonFatal -import com.google.common.util.concurrent.MoreExecutors import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{spy, times, verify} @@ -38,7 +37,7 @@ import org.apache.spark.TaskState.TaskState import org.apache.spark.TestUtils.JavaSourceFromString import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.storage.TaskResultBlockId -import org.apache.spark.util.{MutableURLClassLoader, RpcUtils, Utils} +import org.apache.spark.util.{MutableURLClassLoader, RpcUtils, ThreadUtils, Utils} /** @@ -99,7 +98,7 @@ private class MyTaskResultGetter(env: SparkEnv, scheduler: TaskSchedulerImpl) extends TaskResultGetter(env, scheduler) { // Use the current thread so we can access its results synchronously - protected override val getTaskResultExecutor = MoreExecutors.sameThreadExecutor() + protected override val getTaskResultExecutor = ThreadUtils.sameThreadExecutorService // DirectTaskResults that we receive from the executors private val _taskResults = new ArrayBuffer[DirectTaskResult[_]] diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala index 2ee3224b3c286..af84498d5e47e 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala @@ -22,9 +22,12 @@ import java.nio.charset.StandardCharsets import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import com.amazonaws.services.kinesis.producer.{KinesisProducer => KPLProducer, KinesisProducerConfiguration, UserRecordResult} +import com.amazonaws.services.kinesis.producer.{KinesisProducer => KPLProducer, + KinesisProducerConfiguration, UserRecordResult} import com.google.common.util.concurrent.{FutureCallback, Futures} +import org.apache.spark.util.ThreadUtils + private[kinesis] class KPLBasedKinesisTestUtils(streamShardCount: Int = 2) extends KinesisTestUtils(streamShardCount) { override protected def getProducer(aggregate: Boolean): KinesisDataGenerator = { @@ -66,7 +69,7 @@ private[kinesis] class KPLDataGenerator(regionName: String) extends KinesisDataG sentSeqNumbers += ((num, seqNumber)) } } - Futures.addCallback(future, kinesisCallBack) + Futures.addCallback(future, kinesisCallBack, ThreadUtils.sameThreadExecutorService) } producer.flushSync() shardIdToSeqNumbers.toMap diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index c5be3efc6371e..91ddf0f28ad80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -47,6 +47,17 @@ object JavaTypeInference { private val keySetReturnType = classOf[JMap[_, _]].getMethod("keySet").getGenericReturnType private val valuesReturnType = classOf[JMap[_, _]].getMethod("values").getGenericReturnType + // Guava changed the name of this method; this tries to stay compatible with both + // TODO replace with isSupertypeOf when Guava 14 support no longer needed for Hadoop + private val ttIsAssignableFrom: (TypeToken[_], TypeToken[_]) => Boolean = { + val ttMethods = classOf[TypeToken[_]].getMethods. + filter(_.getParameterCount == 1). + filter(_.getParameterTypes.head == classOf[TypeToken[_]]) + val isAssignableFromMethod = ttMethods.find(_.getName == "isSupertypeOf").getOrElse( + ttMethods.find(_.getName == "isAssignableFrom").get) + (a: TypeToken[_], b: TypeToken[_]) => isAssignableFromMethod.invoke(a, b).asInstanceOf[Boolean] + } + /** * Infers the corresponding SQL data type of a JavaBean class. * @param beanClass Java type @@ -111,11 +122,11 @@ object JavaTypeInference { val (dataType, nullable) = inferDataType(typeToken.getComponentType, seenTypeSet) (ArrayType(dataType, nullable), true) - case _ if iterableType.isAssignableFrom(typeToken) => + case _ if ttIsAssignableFrom(iterableType, typeToken) => val (dataType, nullable) = inferDataType(elementType(typeToken), seenTypeSet) (ArrayType(dataType, nullable), true) - case _ if mapType.isAssignableFrom(typeToken) => + case _ if ttIsAssignableFrom(mapType, typeToken) => val (keyType, valueType) = mapKeyValueType(typeToken) val (keyDataType, _) = inferDataType(keyType, seenTypeSet) val (valueDataType, nullable) = inferDataType(valueType, seenTypeSet) @@ -273,7 +284,7 @@ object JavaTypeInference { } Invoke(arrayData, methodName, ObjectType(c)) - case c if listType.isAssignableFrom(typeToken) => + case c if ttIsAssignableFrom(listType, typeToken) => val et = elementType(typeToken) val newTypePath = walkedTypePath.recordArray(et.getType.getTypeName) val (dataType, elementNullable) = inferDataType(et) @@ -289,7 +300,7 @@ object JavaTypeInference { UnresolvedMapObjects(mapFunction, path, customCollectionCls = Some(c)) - case _ if mapType.isAssignableFrom(typeToken) => + case _ if ttIsAssignableFrom(mapType, typeToken) => val (keyType, valueType) = mapKeyValueType(typeToken) val newTypePath = walkedTypePath.recordMap(keyType.getType.getTypeName, valueType.getType.getTypeName) @@ -404,10 +415,10 @@ object JavaTypeInference { case _ if typeToken.isArray => toCatalystArray(inputObject, typeToken.getComponentType) - case _ if listType.isAssignableFrom(typeToken) => + case _ if ttIsAssignableFrom(listType, typeToken) => toCatalystArray(inputObject, elementType(typeToken)) - case _ if mapType.isAssignableFrom(typeToken) => + case _ if ttIsAssignableFrom(mapType, typeToken) => val (keyType, valueType) = mapKeyValueType(typeToken) createSerializerForMap( diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java index c8b5555a135d3..5603cb988b8e7 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java @@ -22,6 +22,10 @@ import java.time.LocalDate; import java.util.*; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; +import org.junit.*; + import org.apache.spark.sql.*; import org.apache.spark.sql.catalyst.expressions.GenericRow; import org.apache.spark.sql.catalyst.util.DateTimeUtils; @@ -29,7 +33,6 @@ import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; -import org.junit.*; import org.apache.spark.sql.test.TestSparkSession; @@ -486,17 +489,17 @@ public int hashCode() { @Override public String toString() { - return com.google.common.base.Objects.toStringHelper(this) - .add("shortField", shortField) - .add("intField", intField) - .add("longField", longField) - .add("floatField", floatField) - .add("doubleField", doubleField) - .add("stringField", stringField) - .add("booleanField", booleanField) - .add("timestampField", timestampField) - .add("nullIntField", nullIntField) - .toString(); + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("shortField", shortField) + .append("intField", intField) + .append("longField", longField) + .append("floatField", floatField) + .append("doubleField", doubleField) + .append("stringField", stringField) + .append("booleanField", booleanField) + .append("timestampField", timestampField) + .append("nullIntField", nullIntField) + .toString(); } } @@ -584,11 +587,12 @@ public int hashCode() { @Override public String toString() { - return com.google.common.base.Objects.toStringHelper(this) - .add("localDateField", localDateField) - .add("instantField", instantField) - .toString(); + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("localDateField", localDateField) + .append("instantField", instantField) + .toString(); } + } private static Row createLocalDateInstantRow(Long index) { From c72f88b0ba20727e831ba9755d9628d0347ee3cb Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Fri, 20 Dec 2019 10:39:26 -0800 Subject: [PATCH 0931/1387] [SPARK-17398][SQL] Fix ClassCastException when querying partitioned JSON table ### What changes were proposed in this pull request? When querying a partitioned table with format `org.apache.hive.hcatalog.data.JsonSerDe` and more than one task runs in each executor concurrently, the following exception is encountered: `java.lang.ClassCastException: java.util.ArrayList cannot be cast to org.apache.hive.hcatalog.data.HCatRecord` The exception occurs in `HadoopTableReader.fillObject`. `org.apache.hive.hcatalog.data.JsonSerDe#initialize` populates a `cachedObjectInspector` field by calling `HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector`, which is not thread-safe; this `cachedObjectInspector` is returned by `JsonSerDe#getObjectInspector`. We protect against this Hive bug by synchronizing on an object when we need to call `initialize` on `org.apache.hadoop.hive.serde2.Deserializer` instances (which may be `JsonSerDe` instances). By doing so, the `ObjectInspector` for the `Deserializer` of the partitions of the JSON table and that of the table `SerDe` are the same cached `ObjectInspector` and `HadoopTableReader.fillObject` then works correctly. (If the `ObjectInspector`s are different, then a bug in `HCatRecordObjectInspector` causes an `ArrayList` to be created instead of an `HCatRecord`, resulting in the `ClassCastException` that is seen.) ### Why are the changes needed? To avoid HIVE-15773 / HIVE-21752. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Tested manually on a cluster with a partitioned JSON table and running a query using more than one core per executor. Before this change, the ClassCastException happens consistently. With this change it does not happen. Closes #26895 from wypoon/SPARK-17398. Authored-by: Wing Yew Poon Signed-off-by: Marcelo Vanzin --- .../apache/spark/sql/hive/TableReader.scala | 23 ++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index b1182b271912f..4d18eb6289418 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -132,7 +132,9 @@ class HadoopTableReader( val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHadoopConf.value.value val deserializer = deserializerClass.getConstructor().newInstance() - deserializer.initialize(hconf, localTableDesc.getProperties) + DeserializerLock.synchronized { + deserializer.initialize(hconf, localTableDesc.getProperties) + } HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) } @@ -252,10 +254,14 @@ class HadoopTableReader( partProps.asScala.foreach { case (key, value) => props.setProperty(key, value) } - deserializer.initialize(hconf, props) + DeserializerLock.synchronized { + deserializer.initialize(hconf, props) + } // get the table deserializer val tableSerDe = localTableDesc.getDeserializerClass.getConstructor().newInstance() - tableSerDe.initialize(hconf, localTableDesc.getProperties) + DeserializerLock.synchronized { + tableSerDe.initialize(hconf, tableProperties) + } // fill the non partition key attributes HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, @@ -370,6 +376,17 @@ private[hive] object HiveTableUtil { } } +/** + * Object to synchronize on when calling org.apache.hadoop.hive.serde2.Deserializer#initialize. + * + * [SPARK-17398] org.apache.hive.hcatalog.data.JsonSerDe#initialize calls the non-thread-safe + * HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector, the results of which are + * returned by JsonSerDe#getObjectInspector. + * To protect against this bug in Hive (HIVE-15773/HIVE-21752), we synchronize on this object + * when calling initialize on Deserializer instances that could be JsonSerDe instances. + */ +private[hive] object DeserializerLock + private[hive] object HadoopTableReader extends HiveInspectors with Logging { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to From cd84400271bdd6f1c3ac00ac7c110d010c243817 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 21 Dec 2019 09:56:15 -0800 Subject: [PATCH 0932/1387] [SPARK-29906][SQL][FOLLOWUP] Update the final plan in UI for AQE ### What changes were proposed in this pull request? a followup of https://github.com/apache/spark/pull/26576, which mistakenly removes the UI update of the final plan. ### Why are the changes needed? fix mistake. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26968 from cloud-fan/fix. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 9e2cbbabe3046..213205605aeeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -209,6 +209,7 @@ case class AdaptiveSparkPlanExec( // Run the final plan when there's no more unfinished stages. currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) + executionId.foreach(onUpdatePlan) isFinalPlan = true logDebug(s"Final plan: $currentPhysicalPlan") } From fa47b7faf733f4e439e00dd95c1ee4b90d857d2d Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 21 Dec 2019 10:51:28 -0800 Subject: [PATCH 0933/1387] [SPARK-30280][DOC] Update docs for make Hive 2.3 dependency by default ### What changes were proposed in this pull request? This PR update document for make Hive 2.3 dependency by default. ### Why are the changes needed? The documentation is incorrect. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #26919 from wangyum/SPARK-30280. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- docs/building-spark.md | 7 ++----- docs/sql-data-sources-hive-tables.md | 8 ++++---- docs/sql-distributed-sql-engine.md | 2 +- docs/sql-migration-guide.md | 2 +- 4 files changed, 8 insertions(+), 11 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 898f904461a6d..580f982086731 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -83,13 +83,10 @@ Example: To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `-Phive-thriftserver` profiles to your existing build options. -By default, Spark will use Hive 1.2.1 with the `hadoop-2.7` profile, and Hive 2.3.6 with the `hadoop-3.2` profile. - - # With Hive 1.2.1 support - ./build/mvn -Pyarn -Phive -Phive-thriftserver -DskipTests clean package +By default Spark will build with Hive 2.3.6. # With Hive 2.3.6 support - ./build/mvn -Pyarn -Phive -Phive-thriftserver -Phadoop-3.2 -DskipTests clean package + ./build/mvn -Pyarn -Phive -Phive-thriftserver -DskipTests clean package ## Packaging without Hadoop Dependencies for YARN diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md index f99b064949347..0054d466db987 100644 --- a/docs/sql-data-sources-hive-tables.md +++ b/docs/sql-data-sources-hive-tables.md @@ -119,7 +119,7 @@ One of the most important pieces of Spark SQL's Hive support is interaction with which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL -will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc). +will compile against built-in Hive and use those classes for internal execution (serdes, UDFs, UDAFs, etc). The following options can be used to configure the version of Hive that is used to retrieve metadata: @@ -127,7 +127,7 @@ The following options can be used to configure the version of Hive that is used Property NameDefaultMeaning spark.sql.hive.metastore.version - 1.2.1 + 2.3.6 Version of the Hive metastore. Available options are 0.12.0 through 2.3.6 and 3.0.0 through 3.1.2. @@ -141,9 +141,9 @@ The following options can be used to configure the version of Hive that is used property can be one of three options:
    1. builtin
    2. - Use Hive 1.2.1, which is bundled with the Spark assembly when -Phive is + Use Hive 2.3.6, which is bundled with the Spark assembly when -Phive is enabled. When this option is chosen, spark.sql.hive.metastore.version must be - either 1.2.1 or not defined. + either 2.3.6 or not defined.
    3. maven
    4. Use Hive jars of specified version downloaded from Maven repositories. This configuration is not generally recommended for production deployments. diff --git a/docs/sql-distributed-sql-engine.md b/docs/sql-distributed-sql-engine.md index 13be6d51a7ece..8d47a672985d3 100644 --- a/docs/sql-distributed-sql-engine.md +++ b/docs/sql-distributed-sql-engine.md @@ -29,7 +29,7 @@ without the need to write any code. ## Running the Thrift JDBC/ODBC server The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) -in Hive 1.2.1. You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1. +in built-in Hive. You can test the JDBC server with the beeline script that comes with either Spark or compatible Hive. To start the JDBC/ODBC server, run the following in the Spark directory: diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 9825e2ff48933..1db2a7d41082b 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -819,7 +819,7 @@ Python UDF registration is unchanged. ## Compatibility with Apache Hive Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. -Currently, Hive SerDes and UDFs are based on Hive 1.2.1, +Currently, Hive SerDes and UDFs are based on built-in Hive, and Spark SQL can be connected to different versions of Hive Metastore (from 0.12.0 to 2.3.6 and 3.0.0 to 3.1.2. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)). From 8384ff4c9dca41ba3c5e2a1f3fe52c066c0984b7 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 21 Dec 2019 14:01:25 -0800 Subject: [PATCH 0934/1387] [SPARK-28144][SPARK-29294][SS] Upgrade Kafka to 2.4.0 ### What changes were proposed in this pull request? This patch upgrades the version of Kafka to 2.4, which supports Scala 2.13. There're some incompatible changes in Kafka 2.4 which the patch addresses as well: * `ZkUtils` is removed -> Replaced with `KafkaZkClient` * Majority of methods are removed in `AdminUtils` -> Replaced with `AdminZkClient` * Method signature of `Scheduler.schedule` is changed (return type) -> leverage `DeterministicScheduler` to avoid implementing `ScheduledFuture` ### Why are the changes needed? * Kafka 2.4 supports Scala 2.13 ### Does this PR introduce any user-facing change? No, as Kafka API is known to be compatible across versions. ### How was this patch tested? Existing UTs Closes #26960 from HeartSaVioR/SPARK-29294. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../spark/sql/kafka010/KafkaTestUtils.scala | 48 ++++++++--------- external/kafka-0-10/pom.xml | 5 ++ .../streaming/kafka010/KafkaTestUtils.scala | 36 ++++++++----- .../kafka010/mocks/MockScheduler.scala | 53 ++++++------------- .../streaming/kafka010/mocks/MockTime.scala | 2 +- pom.xml | 2 +- 6 files changed, 68 insertions(+), 78 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index 47ae7be85ce02..7b972fede96e5 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -30,9 +30,9 @@ import scala.util.Random import com.google.common.io.Files import kafka.api.Request -import kafka.server.{KafkaConfig, KafkaServer} +import kafka.server.{HostedPartition, KafkaConfig, KafkaServer} import kafka.server.checkpoints.OffsetCheckpointFile -import kafka.utils.ZkUtils +import kafka.zk.KafkaZkClient import org.apache.hadoop.minikdc.MiniKdc import org.apache.hadoop.security.UserGroupInformation import org.apache.kafka.clients.CommonClientConfigs @@ -44,6 +44,7 @@ import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol.{PLAINTEXT, SASL_PLAINTEXT} import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} +import org.apache.kafka.common.utils.SystemTime import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} import org.apache.zookeeper.server.auth.SASLAuthenticationProvider import org.scalatest.Assertions._ @@ -81,7 +82,7 @@ class KafkaTestUtils( private val zkSessionTimeout = 10000 private var zookeeper: EmbeddedZookeeper = _ - private var zkUtils: ZkUtils = _ + private var zkClient: KafkaZkClient = _ // Kafka broker related configurations private val brokerHost = localCanonicalHostName @@ -115,9 +116,9 @@ class KafkaTestUtils( s"$brokerHost:$brokerPort" } - def zookeeperClient: ZkUtils = { + def zookeeperClient: KafkaZkClient = { assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") - Option(zkUtils).getOrElse( + Option(zkClient).getOrElse( throw new IllegalStateException("Zookeeper client is not yet initialized")) } @@ -243,7 +244,8 @@ class KafkaTestUtils( zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") // Get the actual zookeeper binding port zkPort = zookeeper.actualPort - zkUtils = ZkUtils(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, false) + zkClient = KafkaZkClient(s"$zkHost:$zkPort", isSecure = false, zkSessionTimeout, + zkConnectionTimeout, 1, new SystemTime()) zkReady = true } @@ -288,7 +290,7 @@ class KafkaTestUtils( setupEmbeddedZookeeper() setupEmbeddedKafkaServer() eventually(timeout(1.minute)) { - assert(zkUtils.getAllBrokersInCluster().nonEmpty, "Broker was not up in 60 seconds") + assert(zkClient.getAllBrokersInCluster.nonEmpty, "Broker was not up in 60 seconds") } } @@ -335,9 +337,9 @@ class KafkaTestUtils( } } - if (zkUtils != null) { - zkUtils.close() - zkUtils = null + if (zkClient != null) { + zkClient.close() + zkClient = null } if (zookeeper != null) { @@ -367,7 +369,7 @@ class KafkaTestUtils( var created = false while (!created) { try { - val newTopic = new NewTopic(topic, partitions, 1) + val newTopic = new NewTopic(topic, partitions, 1.shortValue()) adminClient.createTopics(Collections.singleton(newTopic)) created = true } catch { @@ -384,7 +386,7 @@ class KafkaTestUtils( } def getAllTopicsAndPartitionSize(): Seq[(String, Int)] = { - zkUtils.getPartitionsForTopics(zkUtils.getAllTopics()).mapValues(_.size).toSeq + zkClient.getPartitionsForTopics(zkClient.getAllTopicsInCluster).mapValues(_.size).toSeq } /** Create a Kafka topic and wait until it is propagated to the whole cluster */ @@ -394,9 +396,9 @@ class KafkaTestUtils( /** Delete a Kafka topic and wait until it is propagated to the whole cluster */ def deleteTopic(topic: String): Unit = { - val partitions = zkUtils.getPartitionsForTopics(Seq(topic))(topic).size + val partitions = zkClient.getPartitionsForTopics(Set(topic))(topic).size adminClient.deleteTopics(Collections.singleton(topic)) - verifyTopicDeletionWithRetries(zkUtils, topic, partitions, List(this.server)) + verifyTopicDeletionWithRetries(topic, partitions, List(this.server)) } /** Add new partitions to a Kafka topic */ @@ -575,15 +577,12 @@ class KafkaTestUtils( servers: Seq[KafkaServer]): Unit = { val topicAndPartitions = (0 until numPartitions).map(new TopicPartition(topic, _)) - import ZkUtils._ // wait until admin path for delete topic is deleted, signaling completion of topic deletion - assert( - !zkUtils.pathExists(getDeleteTopicPath(topic)), - s"${getDeleteTopicPath(topic)} still exists") - assert(!zkUtils.pathExists(getTopicPath(topic)), s"${getTopicPath(topic)} still exists") + assert(!zkClient.isTopicMarkedForDeletion(topic), "topic is still marked for deletion") + assert(!zkClient.topicExists(topic), "topic still exists") // ensure that the topic-partition has been deleted from all brokers' replica managers assert(servers.forall(server => topicAndPartitions.forall(tp => - server.replicaManager.getPartition(tp) == None)), + server.replicaManager.getPartition(tp) == HostedPartition.None)), s"topic $topic still exists in the replica manager") // ensure that logs from all replicas are deleted if delete topic is marked successful assert(servers.forall(server => topicAndPartitions.forall(tp => @@ -598,13 +597,12 @@ class KafkaTestUtils( }), s"checkpoint for topic $topic still exists") // ensure the topic is gone assert( - !zkUtils.getAllTopics().contains(topic), + !zkClient.getAllTopicsInCluster.contains(topic), s"topic $topic still exists on zookeeper") } /** Verify topic is deleted. Retry to delete the topic if not. */ private def verifyTopicDeletionWithRetries( - zkUtils: ZkUtils, topic: String, numPartitions: Int, servers: Seq[KafkaServer]): Unit = { @@ -626,9 +624,9 @@ class KafkaTestUtils( def isPropagated = server.dataPlaneRequestProcessor.metadataCache .getPartitionInfo(topic, partition) match { case Some(partitionState) => - zkUtils.getLeaderForPartition(topic, partition).isDefined && - Request.isValidBrokerId(partitionState.basePartitionState.leader) && - !partitionState.basePartitionState.replicas.isEmpty + zkClient.getLeaderForPartition(new TopicPartition(topic, partition)).isDefined && + Request.isValidBrokerId(partitionState.leader) && + !partitionState.replicas.isEmpty case _ => false diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index d11569d709b23..ad7a8b7e23f1d 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -111,6 +111,11 @@ org.apache.spark spark-tags_${scala.binary.version} + + org.jmock + jmock-junit4 + test + 2.3 - 2.3.1 + 2.4.0 10.12.1.1 1.10.1 1.5.8 From f31d9a629b94f5db57bdaf96eae90f165c588da6 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Sat, 21 Dec 2019 14:08:58 -0800 Subject: [PATCH 0935/1387] [MINOR][DOC][SQL][CORE] Fix typo in document and comments ### What changes were proposed in this pull request? Fixed typo in `docs` directory and in other directories 1. Find typo in `docs` and apply fixes to files in all directories 2. Fix `the the` -> `the` ### Why are the changes needed? Better readability of documents ### Does this PR introduce any user-facing change? No ### How was this patch tested? No test needed Closes #26976 from kiszk/typo_20191221. Authored-by: Kazuaki Ishizaki Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/util/Clock.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 4 ++-- .../ExternalAppendOnlyMapSuite.scala | 2 +- docs/_data/menu-sql.yaml | 2 +- docs/configuration.md | 6 +++--- docs/ml-classification-regression.md | 2 +- docs/ml-migration-guide.md | 6 +++--- docs/monitoring.md | 2 +- docs/sql-data-sources-avro.md | 2 +- docs/sql-migration-guide.md | 2 +- docs/sql-pyspark-pandas-with-arrow.md | 2 +- docs/sql-ref-null-semantics.md | 20 +++++++++---------- docs/sql-ref-syntax-ddl-alter-table.md | 2 +- docs/sql-ref-syntax-dml-insert-into.md | 4 ++-- docs/streaming-kafka-0-10-integration.md | 4 ++-- .../structured-streaming-kafka-integration.md | 2 +- docs/web-ui.md | 2 +- .../kafka010/KafkaOffsetRangeCalculator.scala | 2 +- .../ml/tree/impl/DecisionTreeMetadata.scala | 2 +- .../InterpretedUnsafeProjection.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../plans/logical/AnalysisHelperSuite.scala | 2 +- .../aggregate/ObjectAggregationIterator.scala | 2 +- .../python/PythonForeachWriter.scala | 2 +- .../StreamingSymmetricHashJoinExec.scala | 2 +- .../streaming/sources/ConsoleWrite.scala | 2 +- .../spark/sql/streaming/GroupState.scala | 2 +- .../NestedSchemaPruningBenchmark.scala | 2 +- 28 files changed, 44 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Clock.scala b/core/src/main/scala/org/apache/spark/util/Clock.scala index d2674d4f47224..226f15d3d38c2 100644 --- a/core/src/main/scala/org/apache/spark/util/Clock.scala +++ b/core/src/main/scala/org/apache/spark/util/Clock.scala @@ -42,7 +42,7 @@ private[spark] trait Clock { * * TL;DR: on modern (2.6.32+) Linux kernels with modern (AMD K8+) CPUs, the values returned by * `System.nanoTime()` are consistent across CPU cores *and* packages, and provide always - * increasing values (although it may not be completely monotonic when the the system clock is + * increasing values (although it may not be completely monotonic when the system clock is * adjusted by NTP daemons using time slew). */ // scalastyle:on line.size.limit diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 7221623f89e1b..a0da3ca5b5f3b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -83,7 +83,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } // Five messages should be marked as received and queued, but no messages should be posted to - // listeners yet because the the listener bus hasn't been started. + // listeners yet because the listener bus hasn't been started. assert(bus.metrics.numEventsPosted.getCount === 5) assert(bus.queuedEvents.size === 5) @@ -206,7 +206,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match assert(sharedQueueSize(bus) === 1) assert(numDroppedEvents(bus) === 1) - // Allow the the remaining events to be processed so we can stop the listener bus: + // Allow the remaining events to be processed so we can stop the listener bus: listenerWait.release(2) bus.stop() } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 2b5993a352cb0..0b4e1494bf300 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -436,7 +436,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite val it = map.iterator assert(it.isInstanceOf[CompletionIterator[_, _]]) // org.apache.spark.util.collection.AppendOnlyMap.destructiveSortedIterator returns - // an instance of an annonymous Iterator class. + // an instance of an anonymous Iterator class. val underlyingMapRef = WeakReference(map.currentMap) diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 36e0b99a07ffd..1149e4704be2e 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -233,5 +233,5 @@ url: sql-ref-functions-udf-scalar.html - text: Aggregate functions url: sql-ref-functions-udf-aggregate.html - - text: Arthmetic operations + - text: Arithmetic operations url: sql-ref-arithmetic-ops.html diff --git a/docs/configuration.md b/docs/configuration.md index 497a2ad36b67c..a02733fdbee89 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2423,7 +2423,7 @@ showDF(properties, numRows = 200, truncate = FALSE) Interval at which data received by Spark Streaming receivers is chunked into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the performance - tuning section in the Spark Streaming programing guide for more details. + tuning section in the Spark Streaming programming guide for more details. @@ -2434,7 +2434,7 @@ showDF(properties, numRows = 200, truncate = FALSE) Effectively, each stream will consume at most this number of records per second. Setting this configuration to 0 or a negative number will put no limit on the rate. See the deployment guide - in the Spark Streaming programing guide for mode details. + in the Spark Streaming programming guide for mode details. @@ -2444,7 +2444,7 @@ showDF(properties, numRows = 200, truncate = FALSE) Enable write-ahead logs for receivers. All the input data received through receivers will be saved to write-ahead logs that will allow it to be recovered after driver failures. See the deployment guide - in the Spark Streaming programing guide for more details. + in the Spark Streaming programming guide for more details. diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index 05c688960f04c..6a81aff1476d9 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -670,7 +670,7 @@ others. Gamma Continuous - Inverse*, Idenity, Log + Inverse*, Identity, Log Tweedie diff --git a/docs/ml-migration-guide.md b/docs/ml-migration-guide.md index 9e8cd3e07b1ee..49f701b2156b3 100644 --- a/docs/ml-migration-guide.md +++ b/docs/ml-migration-guide.md @@ -254,7 +254,7 @@ Deprecations in the `spark.mllib` and `spark.ml` packages include: We move all functionality in overridden methods to the corresponding `transformSchema`. * [SPARK-14829](https://issues.apache.org/jira/browse/SPARK-14829): In `spark.mllib` package, `LinearRegressionWithSGD`, `LassoWithSGD`, `RidgeRegressionWithSGD` and `LogisticRegressionWithSGD` have been deprecated. - We encourage users to use `spark.ml.regression.LinearRegresson` and `spark.ml.classification.LogisticRegresson`. + We encourage users to use `spark.ml.regression.LinearRegression` and `spark.ml.classification.LogisticRegression`. * [SPARK-14900](https://issues.apache.org/jira/browse/SPARK-14900): In `spark.mllib.evaluation.MulticlassMetrics`, the parameters `precision`, `recall` and `fMeasure` have been deprecated in favor of `accuracy`. * [SPARK-15644](https://issues.apache.org/jira/browse/SPARK-15644): @@ -266,12 +266,12 @@ Deprecations in the `spark.mllib` and `spark.ml` packages include: Changes of behavior in the `spark.mllib` and `spark.ml` packages include: * [SPARK-7780](https://issues.apache.org/jira/browse/SPARK-7780): - `spark.mllib.classification.LogisticRegressionWithLBFGS` directly calls `spark.ml.classification.LogisticRegresson` for binary classification now. + `spark.mllib.classification.LogisticRegressionWithLBFGS` directly calls `spark.ml.classification.LogisticRegression` for binary classification now. This will introduce the following behavior changes for `spark.mllib.classification.LogisticRegressionWithLBFGS`: * The intercept will not be regularized when training binary classification model with L1/L2 Updater. * If users set without regularization, training with or without feature scaling will return the same solution by the same convergence rate. * [SPARK-13429](https://issues.apache.org/jira/browse/SPARK-13429): - In order to provide better and consistent result with `spark.ml.classification.LogisticRegresson`, + In order to provide better and consistent result with `spark.ml.classification.LogisticRegression`, the default value of `spark.mllib.classification.LogisticRegressionWithLBFGS`: `convergenceTol` has been changed from 1E-4 to 1E-6. * [SPARK-12363](https://issues.apache.org/jira/browse/SPARK-12363): Fix a bug of `PowerIterationClustering` which will likely change its result. diff --git a/docs/monitoring.md b/docs/monitoring.md index cff1126481f94..090178f5b37eb 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -640,7 +640,7 @@ A list of the available metrics, with a short description: ### Executor Metrics -Executor-level metrics are sent from each executor to the driver as part of the Heartbeat to describe the performance metrics of Executor itself like JVM heap memory, GC infomation. Metrics `peakExecutorMetrics.*` are only enabled if `spark.eventLog.logStageExecutorMetrics.enabled` is true. +Executor-level metrics are sent from each executor to the driver as part of the Heartbeat to describe the performance metrics of Executor itself like JVM heap memory, GC information. Metrics `peakExecutorMetrics.*` are only enabled if `spark.eventLog.logStageExecutorMetrics.enabled` is true. A list of the available metrics, with a short description: diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md index dda7af1dc0f31..53be8709e91cc 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/sql-data-sources-avro.md @@ -245,7 +245,7 @@ Data source options of Avro can be set via: diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 1db2a7d41082b..674621f3fdfaf 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -220,7 +220,7 @@ license: | - Since Spark 3.0, when casting interval values to string type, there is no "interval" prefix, e.g. `1 days 2 hours`. In Spark version 2.4 and earlier, the string contains the "interval" prefix like `interval 1 days 2 hours`. - - Since Spark 3.0, when casting string value to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing whitespaces(<= ACSII 32) will be trimmed before converted to these type values, e.g. `cast(' 1\t' as int)` results `1`, `cast(' 1\t' as boolean)` results `true`, `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, while casting string to integrals and booleans, it will not trim the whitespaces from both ends, the foregoing results will be `null`, while to datetimes, only the trailing spaces(= ASCII 32) will be removed. + - Since Spark 3.0, when casting string value to integral types(tinyint, smallint, int and bigint), datetime types(date, timestamp and interval) and boolean type, the leading and trailing whitespaces (<= ASCII 32) will be trimmed before converted to these type values, e.g. `cast(' 1\t' as int)` results `1`, `cast(' 1\t' as boolean)` results `true`, `cast('2019-10-10\t as date)` results the date value `2019-10-10`. In Spark version 2.4 and earlier, while casting string to integrals and booleans, it will not trim the whitespaces from both ends, the foregoing results will be `null`, while to datetimes, only the trailing spaces (= ASCII 32) will be removed. - Since Spark 3.0, numbers written in scientific notation(e.g. `1E2`) would be parsed as Double. In Spark version 2.4 and earlier, they're parsed as Decimal. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.exponentLiteralAsDecimal.enabled` to `true`. diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md index d638278b42355..7eb8a74547f70 100644 --- a/docs/sql-pyspark-pandas-with-arrow.md +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -255,7 +255,7 @@ different than a Pandas timestamp. It is recommended to use Pandas time series f working with timestamps in `pandas_udf`s to get the best performance, see [here](https://pandas.pydata.org/pandas-docs/stable/timeseries.html) for details. -### Compatibiliy Setting for PyArrow >= 0.15.0 and Spark 2.3.x, 2.4.x +### Compatibility Setting for PyArrow >= 0.15.0 and Spark 2.3.x, 2.4.x Since Arrow 0.15.0, a change in the binary IPC format requires an environment variable to be compatible with previous versions of Arrow <= 0.14.1. This is only necessary to do for PySpark diff --git a/docs/sql-ref-null-semantics.md b/docs/sql-ref-null-semantics.md index fd467d224ffd5..3cbc15c600cee 100644 --- a/docs/sql-ref-null-semantics.md +++ b/docs/sql-ref-null-semantics.md @@ -25,14 +25,14 @@ A column is associated with a data type and represents a specific attribute of an entity (for example, `age` is a column of an entity called `person`). Sometimes, the value of a column specific to a row is not known at the time the row comes into existence. -In `SQL`, such values are represnted as `NULL`. This section details the +In `SQL`, such values are represented as `NULL`. This section details the semantics of `NULL` values handling in various operators, expressions and other `SQL` constructs. 1. [Null handling in comparison operators](#comp-operators) 2. [Null handling in Logical operators](#logical-operators) 3. [Null handling in Expressions](#expressions) - 1. [Null handling in null-in-tolerant expressions](#null-in-tolerant) + 1. [Null handling in null-intolerant expressions](#null-intolerant) 2. [Null handling Expressions that can process null value operands](#can-process-null) 3. [Null handling in built-in aggregate expressions](#built-in-aggregate) 4. [Null handling in WHERE, HAVING and JOIN conditions](#condition-expressions) @@ -61,10 +61,10 @@ the `age` column and this table will be used in various examples in the sections
      None Optional Avro schema (in JSON format) that was used to serialize the data. This should be set if the schema provided for deserialization is compatible with - but not the same as - the one used to originally convert the data to Avro. - For more information on Avro's schema evolution and compatability, please refer to the [documentation of Confluent](https://docs.confluent.io/current/schema-registry/avro.html). + For more information on Avro's schema evolution and compatibility, please refer to the [documentation of Confluent](https://docs.confluent.io/current/schema-registry/avro.html). function from_avro
      700Dan50
      -### Comparision operators +### Comparison operators Apache spark supports the standard comparison operators such as '>', '>=', '=', '<' and '<='. -The result of these operators is unknown or `NULL` when one of the operarands or both the operands are +The result of these operators is unknown or `NULL` when one of the operands or both the operands are unknown or `NULL`. In order to compare the `NULL` values for equality, Spark provides a null-safe equal operator ('<=>'), which returns `False` when one of the operand is `NULL` and returns 'True` when both the operands are `NULL`. The following table illustrates the behaviour of comparison operators when @@ -152,7 +152,7 @@ SELECT NULL <=> NULL; Spark supports standard logical operators such as `AND`, `OR` and `NOT`. These operators take `Boolean` expressions as the arguments and return a `Boolean` value. -The following tables illustrate the behavior of logical opeators when one or both operands are `NULL`. +The following tables illustrate the behavior of logical operators when one or both operands are `NULL`. @@ -236,12 +236,12 @@ The comparison operators and logical operators are treated as expressions in Spark. Other than these two kinds of expressions, Spark supports other form of expressions such as function expressions, cast expressions, etc. The expressions in Spark can be broadly classified as : -- Null in-tolerent expressions +- Null intolerant expressions - Expressions that can process `NULL` value operands - The result of these expressions depends on the expression itself. -#### Null in-tolerant expressions -Null in-tolerant expressions return `NULL` when one or more arguments of +#### Null intolerant expressions +Null intolerant expressions return `NULL` when one or more arguments of expression are `NULL` and most of the expressions fall in this category. ##### Examples @@ -297,7 +297,7 @@ SELECT isnull(null) AS expression_output; |true | +-----------------+ --- Returns the first occurence of non `NULL` value. +-- Returns the first occurrence of non `NULL` value. SELECT coalesce(null, null, 3, null) AS expression_output; +-----------------+ |expression_output| @@ -460,7 +460,7 @@ WHERE p1.age <=> p2.age {% endhighlight %} ### Aggregate operator (GROUP BY, DISTINCT) -As discussed in the previous section [comparison operator](sql-ref-null-semantics.html#comparision-operators), +As discussed in the previous section [comparison operator](sql-ref-null-semantics.html#comparison-operators), two `NULL` values are not equal. However, for the purpose of grouping and distinct processing, the two or more values with `NULL data`are grouped together into the same bucket. This behaviour is conformant with SQL standard and with other enterprise database management systems. diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index 1d7ace74231e6..a921478daa470 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -120,7 +120,7 @@ ALTER TABLE table_identifier [ partition_spec ] SET SERDE serde_class_name #### SET LOCATION And SET FILE FORMAT `ALTER TABLE SET` command can also be used for changing the file location and file format for -exsisting tables. +existing tables. ##### Syntax {% highlight sql %} diff --git a/docs/sql-ref-syntax-dml-insert-into.md b/docs/sql-ref-syntax-dml-insert-into.md index 3060e2d1a6562..715f43c9b80ea 100644 --- a/docs/sql-ref-syntax-dml-insert-into.md +++ b/docs/sql-ref-syntax-dml-insert-into.md @@ -55,7 +55,7 @@ INSERT INTO [ TABLE ] table_identifier [ partition_spec ]
      VALUES ( { value | NULL } [ , ... ] ) [ , ( ... ) ]
      -
      Specifies the values to be inserted. Either an explicitly specified value or a NULL can be inserted. A comma must be used to seperate each value in the clause. More than one set of values can be specified to insert multiple rows.
      +
      Specifies the values to be inserted. Either an explicitly specified value or a NULL can be inserted. A comma must be used to separate each value in the clause. More than one set of values can be specified to insert multiple rows.
      @@ -215,4 +215,4 @@ INSERT INTO [ TABLE ] table_identifier [ partition_spec ] ### Related Statements * [INSERT OVERWRITE statement](sql-ref-syntax-dml-insert-overwrite-table.html) * [INSERT OVERWRITE DIRECTORY statement](sql-ref-syntax-dml-insert-overwrite-directory.html) - * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) \ No newline at end of file + * [INSERT OVERWRITE DIRECTORY with Hive format statement](sql-ref-syntax-dml-insert-overwrite-directory-hive.html) diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index d8fd6724e91bd..0f5964786fbce 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -233,7 +233,7 @@ For data stores that support transactions, saving offsets in the same transactio {% highlight scala %} // The details depend on your data store, but the general idea looks like this -// begin from the the offsets committed to the database +// begin from the offsets committed to the database val fromOffsets = selectOffsetsFromYourDatabase.map { resultSet => new TopicPartition(resultSet.string("topic"), resultSet.int("partition")) -> resultSet.long("offset") }.toMap @@ -263,7 +263,7 @@ stream.foreachRDD { rdd => {% highlight java %} // The details depend on your data store, but the general idea looks like this -// begin from the the offsets committed to the database +// begin from the offsets committed to the database Map fromOffsets = new HashMap<>(); for (resultSet : selectOffsetsFromYourDatabase) fromOffsets.put(new TopicPartition(resultSet.string("topic"), resultSet.int("partition")), resultSet.long("offset")); diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index eb9e186e961a9..7d10c6d709539 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -405,7 +405,7 @@ The following configurations are optional:
      - - + From 240840fe92c6724a025d423d8322d61f909b003a Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 15 Jan 2020 21:52:43 +0900 Subject: [PATCH 1109/1387] [SPARK-30515][SQL] Refactor SimplifyBinaryComparison to reduce the time complexity ### What changes were proposed in this pull request? The changes in the rule `SimplifyBinaryComparison` from https://github.com/apache/spark/pull/27008 could bring performance regression in the optimizer when there are a large set of filter conditions. We need to improve the implementation and reduce the time complexity. ### Why are the changes needed? Need to fix the potential performance regression in the optimizer. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing unit tests. Also run a micor benchmark in `BinaryComparisonSimplificationSuite` ``` object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Constant Folding", FixedPoint(50), SimplifyBinaryComparison) :: Nil } test("benchmark") { val a = Symbol("a") val condition = (1 to 500).map(i => EqualTo(a, a)).reduceLeft(And) val finalCondition = And(condition, IsNotNull(a)) val plan = nullableRelation.where(finalCondition).analyze val start = System.nanoTime() Optimize.execute(plan) println((System.nanoTime() - start) /1000000) } ``` Before the changes: 2507ms After the changes: 3ms Closes #27212 from gengliangwang/SimplifyBinaryComparison. Authored-by: Gengliang Wang Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/optimizer/expressions.scala | 49 ++++++++++--------- 1 file changed, 25 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 41a7dfb092aba..bd400f86ea2c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -410,37 +410,38 @@ object SimplifyBinaryComparison extends Rule[LogicalPlan] with PredicateHelper with ConstraintHelper { private def canSimplifyComparison( - plan: LogicalPlan, left: Expression, right: Expression): Boolean = { + left: Expression, + right: Expression, + notNullExpressions: => ExpressionSet): Boolean = { if (left.semanticEquals(right)) { - if (!left.nullable && !right.nullable) { - true - } else { - // We do more checks for non-nullable cases - plan match { - case Filter(fc, _) => - splitConjunctivePredicates(fc).exists { condition => - condition.semanticEquals(IsNotNull(left)) - } - case _ => false - } - } + (!left.nullable && !right.nullable) || notNullExpressions.contains(left) } else { false } } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case l: LogicalPlan => l transformExpressionsUp { - // True with equality - case a EqualNullSafe b if a.semanticEquals(b) => TrueLiteral - case a EqualTo b if canSimplifyComparison(l, a, b) => TrueLiteral - case a GreaterThanOrEqual b if canSimplifyComparison(l, a, b) => TrueLiteral - case a LessThanOrEqual b if canSimplifyComparison(l, a, b) => TrueLiteral - - // False with inequality - case a GreaterThan b if canSimplifyComparison(l, a, b) => FalseLiteral - case a LessThan b if canSimplifyComparison(l, a, b) => FalseLiteral - } + case l: LogicalPlan => + lazy val notNullExpressions = ExpressionSet(l match { + case Filter(fc, _) => + splitConjunctivePredicates(fc).collect { + case i: IsNotNull => i.child + } + case _ => Seq.empty + }) + + l transformExpressionsUp { + // True with equality + case a EqualNullSafe b if a.semanticEquals(b) => TrueLiteral + case a EqualTo b if canSimplifyComparison(a, b, notNullExpressions) => TrueLiteral + case a GreaterThanOrEqual b if canSimplifyComparison(a, b, notNullExpressions) => + TrueLiteral + case a LessThanOrEqual b if canSimplifyComparison(a, b, notNullExpressions) => TrueLiteral + + // False with inequality + case a GreaterThan b if canSimplifyComparison(a, b, notNullExpressions) => FalseLiteral + case a LessThan b if canSimplifyComparison(a, b, notNullExpressions) => FalseLiteral + } } } From 5f6cd61913bd8bcb149cc48ed10a9b71412c871f Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 15 Jan 2020 22:02:16 +0900 Subject: [PATCH 1110/1387] [SPARK-29708][SQL] Correct aggregated values when grouping sets are duplicated ### What changes were proposed in this pull request? This pr intends to fix wrong aggregated values in `GROUPING SETS` when there are duplicated grouping sets in a query (e.g., `GROUPING SETS ((k1),(k1))`). For example; ``` scala> spark.table("t").show() +---+---+---+ | k1| k2| v| +---+---+---+ | 0| 0| 3| +---+---+---+ scala> sql("""select grouping_id(), k1, k2, sum(v) from t group by grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2))""").show() +-------------+---+----+------+ |grouping_id()| k1| k2|sum(v)| +-------------+---+----+------+ | 0| 0| 0| 9| <---- wrong aggregate value and the correct answer is `3` | 1| 0|null| 3| +-------------+---+----+------+ // PostgreSQL case postgres=# select k1, k2, sum(v) from t group by grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2)); k1 | k2 | sum ----+------+----- 0 | 0 | 3 0 | 0 | 3 0 | 0 | 3 0 | NULL | 3 (4 rows) // Hive case hive> select GROUPING__ID, k1, k2, sum(v) from t group by k1, k2 grouping sets ((k1),(k1,k2),(k2,k1),(k1,k2)); 1 0 NULL 3 0 0 0 3 ``` [MS SQL Server has the same behaviour with PostgreSQL](https://github.com/apache/spark/pull/26961#issuecomment-573638442). This pr follows the behaviour of PostgreSQL/SQL server; it adds one more virtual attribute in `Expand` for avoiding wrongly grouping rows with the same grouping ID. ### Why are the changes needed? To fix bugs. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? The existing tests. Closes #26961 from maropu/SPARK-29708. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 30 +++++++++--- .../sql-tests/inputs/grouping_set.sql | 6 +++ .../inputs/postgreSQL/groupingsets.sql | 1 - .../sql-tests/results/grouping_set.sql.out | 47 +++++++++++++++++-- .../results/postgreSQL/groupingsets.sql.out | 6 ++- 6 files changed, 78 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4ce8fcc3b10cf..adea15ded1be7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -454,7 +454,7 @@ class Analyzer( gid: Attribute): LogicalPlan = { // Change the nullability of group by aliases if necessary. For example, if we have // GROUPING SETS ((a,b), a), we do not need to change the nullability of a, but we - // should change the nullabilty of b to be TRUE. + // should change the nullability of b to be TRUE. // TODO: For Cube/Rollup just set nullability to be `true`. val expandedAttributes = groupByAliases.map { alias => if (selectedGroupByExprs.exists(!_.contains(alias.child))) { 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 5853ae2a97726..40db8b6f49dc4 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 @@ -17,17 +17,16 @@ package org.apache.spark.sql.catalyst.plans.logical +import scala.collection.mutable + import org.apache.spark.sql.catalyst.AliasIdentifier -import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier, SupportsNamespaces, TableCatalog, TableChange} -import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} -import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler @@ -641,11 +640,14 @@ object Expand { child: LogicalPlan): Expand = { val attrMap = groupByAttrs.zipWithIndex.toMap + val hasDuplicateGroupingSets = groupingSetsAttrs.size != + groupingSetsAttrs.map(_.map(_.exprId).toSet).distinct.size + // Create an array of Projections for the child projection, and replace the projections' // expressions which equal GroupBy expressions with Literal(null), if those expressions // are not set for this grouping set. - val projections = groupingSetsAttrs.map { groupingSetAttrs => - child.output ++ groupByAttrs.map { attr => + val projections = groupingSetsAttrs.zipWithIndex.map { case (groupingSetAttrs, i) => + val projAttrs = child.output ++ groupByAttrs.map { attr => if (!groupingSetAttrs.contains(attr)) { // if the input attribute in the Invalid Grouping Expression set of for this group // replace it with constant null @@ -655,11 +657,25 @@ object Expand { } // groupingId is the last output, here we use the bit mask as the concrete value for it. } :+ Literal.create(buildBitmask(groupingSetAttrs, attrMap), IntegerType) + + if (hasDuplicateGroupingSets) { + // If `groupingSetsAttrs` has duplicate entries (e.g., GROUPING SETS ((key), (key))), + // we add one more virtual grouping attribute (`_gen_grouping_pos`) to avoid + // wrongly grouping rows with the same grouping ID. + projAttrs :+ Literal.create(i, IntegerType) + } else { + projAttrs + } } // the `groupByAttrs` has different meaning in `Expand.output`, it could be the original // grouping expression or null, so here we create new instance of it. - val output = child.output ++ groupByAttrs.map(_.newInstance) :+ gid + val output = if (hasDuplicateGroupingSets) { + val gpos = AttributeReference("_gen_grouping_pos", IntegerType, false)() + child.output ++ groupByAttrs.map(_.newInstance) :+ gid :+ gpos + } else { + child.output ++ groupByAttrs.map(_.newInstance) :+ gid + } Expand(projections, output, Project(child.output ++ groupByAliases, child)) } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/grouping_set.sql b/sql/core/src/test/resources/sql-tests/inputs/grouping_set.sql index 6bbde9f38d657..d30914fdd92df 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/grouping_set.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/grouping_set.sql @@ -51,3 +51,9 @@ SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE; SELECT c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS (()); +-- duplicate entries in grouping sets +SELECT k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); + +SELECT grouping__id, k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); + +SELECT grouping(k1), k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql index c9ee83eb75eba..fc54d179f742c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/groupingsets.sql @@ -385,7 +385,6 @@ select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) -- from gstest1 group by cube(a,b); -- unsortable cases --- [SPARK-29708] Different answers in aggregates of multiple grouping sets select unsortable_col, count(*) from gstest4 group by grouping sets ((unsortable_col),(unsortable_col)) order by string(unsortable_col); diff --git a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out index 34ab09c5e3bba..9b81a55c7de6f 100644 --- a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 15 +-- Number of queries: 18 -- !query 0 @@ -110,8 +110,10 @@ SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUP -- !query 10 schema struct<(a + b):int,b:int,sum(c):bigint> -- !query 10 output -2 NULL 2 -4 NULL 4 +2 NULL 1 +2 NULL 1 +4 NULL 2 +4 NULL 2 NULL 1 1 NULL 2 2 @@ -164,3 +166,42 @@ struct<> -- !query 14 output org.apache.spark.sql.AnalysisException expression '`c1`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 15 +SELECT k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query 15 schema +struct +-- !query 15 output +1 1 1.0 +1 1 1.0 +1 NULL 1.0 +2 2 2.0 +2 2 2.0 +2 NULL 2.0 + + +-- !query 16 +SELECT grouping__id, k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query 16 schema +struct +-- !query 16 output +0 1 1 1.0 +0 1 1 1.0 +0 2 2 2.0 +0 2 2 2.0 +1 1 NULL 1.0 +1 2 NULL 2.0 + + +-- !query 17 +SELECT grouping(k1), k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query 17 schema +struct +-- !query 17 output +0 1 1 1.0 +0 1 1 1.0 +0 1 NULL 1.0 +0 2 2 2.0 +0 2 2 2.0 +0 2 NULL 2.0 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out index 87adfc5552810..6e77ab207c94a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out @@ -510,8 +510,10 @@ select unsortable_col, count(*) -- !query 37 schema struct -- !query 37 output -1 8 -2 8 +1 4 +1 4 +2 4 +2 4 -- !query 38 From 525c5695f8fd7358e80ef6ed4854ea1af1d5dc63 Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 15 Jan 2020 08:42:24 -0600 Subject: [PATCH 1111/1387] [SPARK-30504][PYTHON][ML] Set weightCol in OneVsRest(Model) _to_java and _from_java ### What changes were proposed in this pull request? This PR adjusts `_to_java` and `_from_java` of `OneVsRest` and `OneVsRestModel` to preserve `weightCol`. ### Why are the changes needed? Currently both `Params` don't preserve `weightCol` `Params` when data is saved / loaded: ```python from pyspark.ml.classification import LogisticRegression, OneVsRest, OneVsRestModel from pyspark.ml.linalg import DenseVector df = spark.createDataFrame([(0, 1, DenseVector([1.0, 0.0])), (0, 1, DenseVector([1.0, 0.0]))], ("label", "w", "features")) ovr = OneVsRest(classifier=LogisticRegression()).setWeightCol("w") ovrm = ovr.fit(df) ovr.getWeightCol() ## 'w' ovrm.getWeightCol() ## 'w' ovr.write().overwrite().save("/tmp/ovr") ovr_ = OneVsRest.load("/tmp/ovr") ovr_.getWeightCol() ## KeyError ## ... ## KeyError: Param(parent='OneVsRest_5145d56b6bd1', name='weightCol', doc='weight column name. ...) ovrm.write().overwrite().save("/tmp/ovrm") ovrm_ = OneVsRestModel.load("/tmp/ovrm") ovrm_ .getWeightCol() ## KeyError ## ... ## KeyError: Param(parent='OneVsRestModel_598c6d900fad', name='weightCol', doc='weight column name ... ``` ### Does this PR introduce any user-facing change? After this PR is merged, loaded objects will have `weightCol` `Param` set. ### How was this patch tested? - Manual testing. - Extension of existing persistence tests. Closes #27190 from zero323/SPARK-30504. Authored-by: zero323 Signed-off-by: Sean Owen --- python/pyspark/ml/classification.py | 8 ++++++ python/pyspark/ml/tests/test_persistence.py | 32 ++++++++++++--------- 2 files changed, 27 insertions(+), 13 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d3b7fdd15b452..689d8068cde35 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -2571,6 +2571,8 @@ def _from_java(cls, java_stage): py_stage = cls(featuresCol=featuresCol, labelCol=labelCol, predictionCol=predictionCol, rawPredictionCol=rawPredictionCol, classifier=classifier, parallelism=parallelism) + if java_stage.isDefined(java_stage.getParam("weightCol")): + py_stage.setWeightCol(java_stage.getWeightCol()) py_stage._resetUid(java_stage.uid()) return py_stage @@ -2587,6 +2589,8 @@ def _to_java(self): _java_obj.setFeaturesCol(self.getFeaturesCol()) _java_obj.setLabelCol(self.getLabelCol()) _java_obj.setPredictionCol(self.getPredictionCol()) + if (self.isDefined(self.weightCol) and self.getWeightCol()): + _java_obj.setWeightCol(self.getWeightCol()) _java_obj.setRawPredictionCol(self.getRawPredictionCol()) return _java_obj @@ -2765,6 +2769,8 @@ def _from_java(cls, java_stage): py_stage = cls(models=models).setPredictionCol(predictionCol)\ .setFeaturesCol(featuresCol) py_stage._set(labelCol=labelCol) + if java_stage.isDefined(java_stage.getParam("weightCol")): + py_stage._set(weightCol=java_stage.getWeightCol()) py_stage._set(classifier=classifier) py_stage._resetUid(java_stage.uid()) return py_stage @@ -2786,6 +2792,8 @@ def _to_java(self): _java_obj.set("featuresCol", self.getFeaturesCol()) _java_obj.set("labelCol", self.getLabelCol()) _java_obj.set("predictionCol", self.getPredictionCol()) + if (self.isDefined(self.weightCol) and self.getWeightCol()): + _java_obj.set("weightCol", self.getWeightCol()) return _java_obj diff --git a/python/pyspark/ml/tests/test_persistence.py b/python/pyspark/ml/tests/test_persistence.py index 69f59a776296b..d4edcc26e17ac 100644 --- a/python/pyspark/ml/tests/test_persistence.py +++ b/python/pyspark/ml/tests/test_persistence.py @@ -269,21 +269,27 @@ def test_python_transformer_pipeline_persistence(self): def test_onevsrest(self): temp_path = tempfile.mkdtemp() - df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), - (1.0, Vectors.sparse(2, [], [])), - (2.0, Vectors.dense(0.5, 0.5))] * 10, - ["label", "features"]) + df = self.spark.createDataFrame([(0.0, 0.5, Vectors.dense(1.0, 0.8)), + (1.0, 0.5, Vectors.sparse(2, [], [])), + (2.0, 1.0, Vectors.dense(0.5, 0.5))] * 10, + ["label", "wt", "features"]) + lr = LogisticRegression(maxIter=5, regParam=0.01) ovr = OneVsRest(classifier=lr) - model = ovr.fit(df) - ovrPath = temp_path + "/ovr" - ovr.save(ovrPath) - loadedOvr = OneVsRest.load(ovrPath) - self._compare_pipelines(ovr, loadedOvr) - modelPath = temp_path + "/ovrModel" - model.save(modelPath) - loadedModel = OneVsRestModel.load(modelPath) - self._compare_pipelines(model, loadedModel) + + def reload_and_compare(ovr, suffix): + model = ovr.fit(df) + ovrPath = temp_path + "/{}".format(suffix) + ovr.save(ovrPath) + loadedOvr = OneVsRest.load(ovrPath) + self._compare_pipelines(ovr, loadedOvr) + modelPath = temp_path + "/{}Model".format(suffix) + model.save(modelPath) + loadedModel = OneVsRestModel.load(modelPath) + self._compare_pipelines(model, loadedModel) + + reload_and_compare(OneVsRest(classifier=lr), "ovr") + reload_and_compare(OneVsRest(classifier=lr).setWeightCol("wt"), "ovrw") def test_decisiontree_classifier(self): dt = DecisionTreeClassifier(maxDepth=1) From 990a2be27fc05ed81bbe42cf4f0059c486ab3557 Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 15 Jan 2020 08:43:36 -0600 Subject: [PATCH 1112/1387] [SPARK-30378][ML][PYSPARK][FOLLOWUP] Remove Param fields provided by _FactorizationMachinesParams ### What changes were proposed in this pull request? Removal of following `Param` fields: - `factorSize` - `fitLinear` - `miniBatchFraction` - `initStd` - `solver` from `FMClassifier` and `FMRegressor` ### Why are the changes needed? This `Param` members are already provided by `_FactorizationMachinesParams` https://github.com/apache/spark/blob/0f3d744c3f19750ab03eeae3606e122dcffae5df/python/pyspark/ml/regression.py#L2303-L2318 which is mixed into `FMRegressor`: https://github.com/apache/spark/blob/0f3d744c3f19750ab03eeae3606e122dcffae5df/python/pyspark/ml/regression.py#L2350 and `FMClassifier`: https://github.com/apache/spark/blob/0f3d744c3f19750ab03eeae3606e122dcffae5df/python/pyspark/ml/classification.py#L2793 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual testing. Closes #27205 from zero323/SPARK-30378-FOLLOWUP. Authored-by: zero323 Signed-off-by: Sean Owen --- python/pyspark/ml/classification.py | 17 ----------------- python/pyspark/ml/regression.py | 17 ----------------- 2 files changed, 34 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 689d8068cde35..d587782db88b0 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -2848,23 +2848,6 @@ class FMClassifier(JavaProbabilisticClassifier, _FactorizationMachinesParams, Ja .. versionadded:: 3.0.0 """ - factorSize = Param(Params._dummy(), "factorSize", "Dimensionality of the factor vectors, " + - "which are used to get pairwise interactions between variables", - typeConverter=TypeConverters.toInt) - - fitLinear = Param(Params._dummy(), "fitLinear", "whether to fit linear term (aka 1-way term)", - typeConverter=TypeConverters.toBoolean) - - miniBatchFraction = Param(Params._dummy(), "miniBatchFraction", "fraction of the input data " + - "set that should be used for one iteration of gradient descent", - typeConverter=TypeConverters.toFloat) - - initStd = Param(Params._dummy(), "initStd", "standard deviation of initial coefficients", - typeConverter=TypeConverters.toFloat) - - solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + - "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", probabilityCol="probability", rawPredictionCol="rawPrediction", diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 6f01d54368353..d96a6f43c0596 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -2395,23 +2395,6 @@ class FMRegressor(JavaPredictor, _FactorizationMachinesParams, JavaMLWritable, J .. versionadded:: 3.0.0 """ - factorSize = Param(Params._dummy(), "factorSize", "Dimensionality of the factor vectors, " + - "which are used to get pairwise interactions between variables", - typeConverter=TypeConverters.toInt) - - fitLinear = Param(Params._dummy(), "fitLinear", "whether to fit linear term (aka 1-way term)", - typeConverter=TypeConverters.toBoolean) - - miniBatchFraction = Param(Params._dummy(), "miniBatchFraction", "fraction of the input data " + - "set that should be used for one iteration of gradient descent", - typeConverter=TypeConverters.toFloat) - - initStd = Param(Params._dummy(), "initStd", "standard deviation of initial coefficients", - typeConverter=TypeConverters.toFloat) - - solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + - "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, From e751bc66a02997aaca792cd06fa6c65a8792425c Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 15 Jan 2020 10:47:31 -0800 Subject: [PATCH 1113/1387] [SPARK-30479][SQL] Apply compaction of event log to SQL events ### What changes were proposed in this pull request? This patch addresses adding event filter to handle SQL related events. This patch is next task of SPARK-29779 (#27085), please refer the description of PR #27085 to see overall rationalization of this patch. Below functionalities will be addressed in later parts: * integrate compaction into FsHistoryProvider * documentation about new configuration ### Why are the changes needed? One of major goal of SPARK-28594 is to prevent the event logs to become too huge, and SPARK-29779 achieves the goal. We've got another approach in prior, but the old approach required models in both KVStore and live entities to guarantee compatibility, while they're not designed to do so. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added UTs. Closes #27164 from HeartSaVioR/SPARK-30479. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../status/ListenerEventsTestHelper.scala | 47 ++++++ ...he.spark.deploy.history.EventFilterBuilder | 1 + .../history/SQLEventFilterBuilder.scala | 147 ++++++++++++++++++ .../history/SQLEventFilterBuilderSuite.scala | 107 +++++++++++++ .../SQLLiveEntitiesEventFilterSuite.scala | 135 ++++++++++++++++ 5 files changed, 437 insertions(+) create mode 100644 sql/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala index 585c8cc2ae6d4..4b3fbacc47f9c 100644 --- a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala @@ -137,6 +137,53 @@ object ListenerEventsTestHelper { SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorUpdates) } + case class JobInfo( + stageIds: Seq[Int], + stageToTaskIds: Map[Int, Seq[Long]], + stageToRddIds: Map[Int, Seq[Int]]) + + def pushJobEventsWithoutJobEnd( + listener: SparkListener, + jobId: Int, + jobProps: Properties, + execIds: Array[String], + time: Long): JobInfo = { + // Start a job with 1 stage / 4 tasks each + val rddsForStage = createRdds(2) + val stage = createStage(rddsForStage, Nil) + + listener.onJobStart(SparkListenerJobStart(jobId, time, Seq(stage), jobProps)) + + // Submit stage + stage.submissionTime = Some(time) + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, jobProps)) + + // Start tasks from stage + val s1Tasks = createTasks(4, execIds, time) + s1Tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stage.stageId, + stage.attemptNumber(), task)) + } + + // Succeed all tasks in stage. + val s1Metrics = TaskMetrics.empty + s1Metrics.setExecutorCpuTime(2L) + s1Metrics.setExecutorRunTime(4L) + + s1Tasks.foreach { task => + task.markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, + "taskType", Success, task, new ExecutorMetrics, s1Metrics)) + } + + // End stage. + stage.completionTime = Some(time) + listener.onStageCompleted(SparkListenerStageCompleted(stage)) + + JobInfo(Seq(stage.stageId), Map(stage.stageId -> s1Tasks.map(_.taskId)), + Map(stage.stageId -> rddsForStage.map(_.id))) + } + private def nextTaskId(): Long = { taskIdTracker += 1 taskIdTracker diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder b/sql/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder new file mode 100644 index 0000000000000..5025616b752d1 --- /dev/null +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.deploy.history.EventFilterBuilder @@ -0,0 +1 @@ +org.apache.spark.sql.execution.history.SQLEventFilterBuilder \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala new file mode 100644 index 0000000000000..fbd729b9d7c37 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -0,0 +1,147 @@ +/* + * 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.history + +import scala.collection.mutable + +import org.apache.spark.deploy.history.{EventFilter, EventFilterBuilder, JobEventFilter} +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.ui._ +import org.apache.spark.sql.streaming.StreamingQueryListener + +/** + * This class tracks live SQL executions, and pass the list to the [[SQLLiveEntitiesEventFilter]] + * to help SQLLiveEntitiesEventFilter to accept live SQL executions as well as relevant + * jobs (+ stages/tasks/RDDs). + * + * Note that this class only tracks the jobs which are relevant to SQL executions - cannot classify + * between finished job and live job without relation of SQL execution. + */ +private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { + private val _liveExecutionToJobs = new mutable.HashMap[Long, mutable.Set[Int]] + private val _jobToStages = new mutable.HashMap[Int, Set[Int]] + private val _stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] + private val _stageToRDDs = new mutable.HashMap[Int, Set[Int]] + private val stages = new mutable.HashSet[Int] + + def liveSQLExecutions: Set[Long] = _liveExecutionToJobs.keySet.toSet + def liveJobs: Set[Int] = _liveExecutionToJobs.values.flatten.toSet + def liveStages: Set[Int] = _stageToRDDs.keySet.toSet + def liveTasks: Set[Long] = _stageToTasks.values.flatten.toSet + def liveRDDs: Set[Int] = _stageToRDDs.values.flatten.toSet + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + val executionIdString = jobStart.properties.getProperty(SQLExecution.EXECUTION_ID_KEY) + if (executionIdString == null) { + // This is not a job created by SQL + return + } + + val executionId = executionIdString.toLong + val jobId = jobStart.jobId + + val jobsForExecution = _liveExecutionToJobs.getOrElseUpdate(executionId, + mutable.HashSet[Int]()) + jobsForExecution += jobId + + _jobToStages += jobStart.jobId -> jobStart.stageIds.toSet + stages ++= jobStart.stageIds + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + val stageId = stageSubmitted.stageInfo.stageId + if (stages.contains(stageId)) { + _stageToRDDs.put(stageId, stageSubmitted.stageInfo.rddInfos.map(_.id).toSet) + _stageToTasks.getOrElseUpdate(stageId, new mutable.HashSet[Long]()) + } + } + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + _stageToTasks.get(taskStart.stageId).foreach { tasks => + tasks += taskStart.taskInfo.taskId + } + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case e: SparkListenerSQLExecutionStart => onExecutionStart(e) + case e: SparkListenerSQLExecutionEnd => onExecutionEnd(e) + case _ => // Ignore + } + + private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { + _liveExecutionToJobs += event.executionId -> mutable.HashSet[Int]() + } + + private def onExecutionEnd(event: SparkListenerSQLExecutionEnd): Unit = { + _liveExecutionToJobs.remove(event.executionId).foreach { jobs => + val stagesToDrop = _jobToStages.filter(kv => jobs.contains(kv._1)).values.flatten + _jobToStages --= jobs + stages --= stagesToDrop + _stageToTasks --= stagesToDrop + _stageToRDDs --= stagesToDrop + } + } + + override def createFilter(): EventFilter = { + new SQLLiveEntitiesEventFilter(liveSQLExecutions, liveJobs, liveStages, liveTasks, liveRDDs) + } +} + +/** + * This class accepts events which are related to the live SQL executions based on the given + * information. + * + * Note that acceptFn will not match the event ("Don't mind") instead of returning false on + * job related events, because it cannot determine whether the job is related to the finished + * SQL executions, or job is NOT related to the SQL executions. For this case, it just gives up + * the decision and let other filters decide it. + */ +private[spark] class SQLLiveEntitiesEventFilter( + liveSQLExecutions: Set[Long], + _liveJobs: Set[Int], + _liveStages: Set[Int], + _liveTasks: Set[Long], + _liveRDDs: Set[Int]) + extends JobEventFilter(None, _liveJobs, _liveStages, _liveTasks, _liveRDDs) with Logging { + + logDebug(s"live SQL executions : $liveSQLExecutions") + + private val _acceptFn: PartialFunction[SparkListenerEvent, Boolean] = { + case e: SparkListenerSQLExecutionStart => + liveSQLExecutions.contains(e.executionId) + case e: SparkListenerSQLAdaptiveExecutionUpdate => + liveSQLExecutions.contains(e.executionId) + case e: SparkListenerSQLExecutionEnd => + liveSQLExecutions.contains(e.executionId) + case e: SparkListenerDriverAccumUpdates => + liveSQLExecutions.contains(e.executionId) + + case e if acceptFnForJobEvents.lift(e).contains(true) => + // NOTE: if acceptFnForJobEvents(e) returns false, we should leave it to "unmatched" + // because we don't know whether the job has relevant SQL execution which is finished, + // or the job is not related to the SQL execution. + true + + // these events are for finished batches so safer to ignore + case _: StreamingQueryListener.QueryProgressEvent => false + } + + override def acceptFn(): PartialFunction[SparkListenerEvent, Boolean] = _acceptFn +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala new file mode 100644 index 0000000000000..5f3d750e8f271 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala @@ -0,0 +1,107 @@ +/* + * 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.history + +import java.util.Properties + +import org.apache.spark.SparkFunSuite +import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} +import org.apache.spark.status.ListenerEventsTestHelper + +class SQLEventFilterBuilderSuite extends SparkFunSuite { + import ListenerEventsTestHelper._ + + override protected def beforeEach(): Unit = { + ListenerEventsTestHelper.reset() + } + + test("track live SQL executions") { + var time = 0L + + val listener = new SQLEventFilterBuilder + + listener.onOtherEvent(SparkListenerLogStart("TestSparkVersion")) + + // Start the application. + time += 1 + listener.onApplicationStart(SparkListenerApplicationStart( + "name", + Some("id"), + time, + "user", + Some("attempt"), + None)) + + // Start a couple of executors. + time += 1 + val execIds = Array("1", "2") + execIds.foreach { id => + listener.onExecutorAdded(createExecutorAddedEvent(id, time)) + } + + // Start SQL Execution + listener.onOtherEvent(SparkListenerSQLExecutionStart(1, "desc1", "details1", "plan", + new SparkPlanInfo("node", "str", Seq.empty, Map.empty, Seq.empty), time)) + + time += 1 + + // job 1, 2: coupled with SQL execution 1, finished + val jobProp = createJobProps() + val jobPropWithSqlExecution = new Properties(jobProp) + jobPropWithSqlExecution.setProperty(SQLExecution.EXECUTION_ID_KEY, "1") + val jobInfoForJob1 = pushJobEventsWithoutJobEnd(listener, 1, jobPropWithSqlExecution, + execIds, time) + listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded)) + + val jobInfoForJob2 = pushJobEventsWithoutJobEnd(listener, 2, jobPropWithSqlExecution, + execIds, time) + listener.onJobEnd(SparkListenerJobEnd(2, time, JobSucceeded)) + + // job 3: not coupled with SQL execution 1, finished + pushJobEventsWithoutJobEnd(listener, 3, jobProp, execIds, time) + listener.onJobEnd(SparkListenerJobEnd(3, time, JobSucceeded)) + + // job 4: not coupled with SQL execution 1, not finished + pushJobEventsWithoutJobEnd(listener, 4, jobProp, execIds, time) + listener.onJobEnd(SparkListenerJobEnd(4, time, JobSucceeded)) + + assert(listener.liveSQLExecutions === Set(1)) + + // only SQL executions related jobs are tracked + assert(listener.liveJobs === Set(1, 2)) + assert(listener.liveStages === + (jobInfoForJob1.stageIds ++ jobInfoForJob2.stageIds).toSet) + assert(listener.liveTasks === + (jobInfoForJob1.stageToTaskIds.values.flatten ++ + jobInfoForJob2.stageToTaskIds.values.flatten).toSet) + assert(listener.liveRDDs === + (jobInfoForJob1.stageToRddIds.values.flatten ++ + jobInfoForJob2.stageToRddIds.values.flatten).toSet) + + // End SQL execution + listener.onOtherEvent(SparkListenerSQLExecutionEnd(1, 0)) + + assert(listener.liveSQLExecutions.isEmpty) + assert(listener.liveJobs.isEmpty) + assert(listener.liveStages.isEmpty) + assert(listener.liveTasks.isEmpty) + assert(listener.liveRDDs.isEmpty) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala new file mode 100644 index 0000000000000..46fdaba413c6e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala @@ -0,0 +1,135 @@ +/* + * 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.history + +import org.apache.spark.{SparkFunSuite, Success, TaskState} +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.ui.{SparkListenerDriverAccumUpdates, SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} +import org.apache.spark.status.ListenerEventsTestHelper.{createRddsWithId, createStage, createTasks} + +class SQLLiveEntitiesEventFilterSuite extends SparkFunSuite { + test("filter in events for jobs related to live SQL execution") { + // assume finished job 1 with stage 1, task (1, 2), rdds (1, 2) and finished sql execution id 1 + // live job 2 with stages 2, tasks (3, 4), rdds (3, 4) and job 2 belongs to the live + // sql execution id 2 + + val liveSQLExecutions = Set(2L) + val liveJobs = Set(2) + val liveStages = Set(2, 3) + val liveTasks = Set(3L, 4L, 5L, 6L) + val liveRDDs = Set(3, 4, 5, 6) + val liveExecutors: Set[String] = Set("1", "2") + + val filter = new SQLLiveEntitiesEventFilter(liveSQLExecutions, liveJobs, liveStages, liveTasks, + liveRDDs) + val acceptFn = filter.acceptFn().lift + + // Verifying with finished SQL execution 1 + assert(Some(false) === acceptFn(SparkListenerSQLExecutionStart(1, "description1", "details1", + "plan", null, 0))) + assert(Some(false) === acceptFn(SparkListenerSQLExecutionEnd(1, 0))) + assert(Some(false) === acceptFn(SparkListenerSQLAdaptiveExecutionUpdate(1, "plan", null))) + assert(Some(false) === acceptFn(SparkListenerDriverAccumUpdates(1, Seq.empty))) + + // Verifying with finished job 1 + val rddsForStage1 = createRddsWithId(1 to 2) + val stage1 = createStage(1, rddsForStage1, Nil) + val tasksForStage1 = createTasks(Seq(1L, 2L), liveExecutors.toArray, 0) + tasksForStage1.foreach { task => task.markFinished(TaskState.FINISHED, 5) } + + val jobStartEventForJob1 = SparkListenerJobStart(1, 0, Seq(stage1)) + val jobEndEventForJob1 = SparkListenerJobEnd(1, 0, JobSucceeded) + val stageSubmittedEventsForJob1 = SparkListenerStageSubmitted(stage1) + val stageCompletedEventsForJob1 = SparkListenerStageCompleted(stage1) + val unpersistRDDEventsForJob1 = (1 to 2).map(SparkListenerUnpersistRDD) + + // job events for finished job should be considered as "don't know" + assert(None === acceptFn(jobStartEventForJob1)) + assert(None === acceptFn(jobEndEventForJob1)) + + // stage events for finished job should be considered as "don't know" + assert(None === acceptFn(stageSubmittedEventsForJob1)) + assert(None === acceptFn(stageCompletedEventsForJob1)) + unpersistRDDEventsForJob1.foreach { event => + assert(None === acceptFn(event)) + } + + val taskSpeculativeTaskSubmittedEvent = SparkListenerSpeculativeTaskSubmitted(stage1.stageId, + stageAttemptId = 1) + assert(None === acceptFn(taskSpeculativeTaskSubmittedEvent)) + + // task events for finished job should be considered as "don't know" + tasksForStage1.foreach { task => + val taskStartEvent = SparkListenerTaskStart(stage1.stageId, 0, task) + assert(None === acceptFn(taskStartEvent)) + + val taskGettingResultEvent = SparkListenerTaskGettingResult(task) + assert(None === acceptFn(taskGettingResultEvent)) + + val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", + Success, task, new ExecutorMetrics, null) + assert(None === acceptFn(taskEndEvent)) + } + + // Verifying with live SQL execution 2 + assert(Some(true) === acceptFn(SparkListenerSQLExecutionStart(2, "description2", "details2", + "plan", null, 0))) + assert(Some(true) === acceptFn(SparkListenerSQLExecutionEnd(2, 0))) + assert(Some(true) === acceptFn(SparkListenerSQLAdaptiveExecutionUpdate(2, "plan", null))) + assert(Some(true) === acceptFn(SparkListenerDriverAccumUpdates(2, Seq.empty))) + + // Verifying with live job 2 + val rddsForStage2 = createRddsWithId(3 to 4) + val stage2 = createStage(2, rddsForStage2, Nil) + val tasksForStage2 = createTasks(Seq(3L, 4L), liveExecutors.toArray, 0) + tasksForStage1.foreach { task => task.markFinished(TaskState.FINISHED, 5) } + + val jobStartEventForJob2 = SparkListenerJobStart(2, 0, Seq(stage2)) + val stageSubmittedEventsForJob2 = SparkListenerStageSubmitted(stage2) + val stageCompletedEventsForJob2 = SparkListenerStageCompleted(stage2) + val unpersistRDDEventsForJob2 = rddsForStage2.map { rdd => SparkListenerUnpersistRDD(rdd.id) } + + // job events for live job should be accepted + assert(Some(true) === acceptFn(jobStartEventForJob2)) + + // stage events for live job should be accepted + assert(Some(true) === acceptFn(stageSubmittedEventsForJob2)) + assert(Some(true) === acceptFn(stageCompletedEventsForJob2)) + unpersistRDDEventsForJob2.foreach { event => + assert(Some(true) === acceptFn(event)) + } + + val taskSpeculativeTaskSubmittedEvent2 = SparkListenerSpeculativeTaskSubmitted(stage2.stageId, + stageAttemptId = 1) + assert(Some(true) === acceptFn(taskSpeculativeTaskSubmittedEvent2)) + + // task events for live job should be accepted + tasksForStage2.foreach { task => + val taskStartEvent = SparkListenerTaskStart(stage2.stageId, 0, task) + assert(Some(true) === acceptFn(taskStartEvent)) + + val taskGettingResultEvent = SparkListenerTaskGettingResult(task) + assert(Some(true) === acceptFn(taskGettingResultEvent)) + + val taskEndEvent = SparkListenerTaskEnd(stage1.stageId, 0, "taskType", + Success, task, new ExecutorMetrics, null) + assert(Some(true) === acceptFn(taskEndEvent)) + } + } +} From 6c178a5d16dc9e7b0ba9e96e01ee66a8e2b5a21c Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Wed, 15 Jan 2020 11:46:34 -0800 Subject: [PATCH 1114/1387] [SPARK-30495][SS] Consider spark.security.credentials.kafka.enabled and cluster configuration when checking latest delegation token ### What changes were proposed in this pull request? Spark SQL Kafka consumer connector considers delegation token usage even if the user configures `sasl.jaas.config` manually. In this PR I've added `spark.security.credentials.kafka.enabled` and cluster configuration check to the condition. ### Why are the changes needed? Now it's not possible to configure `sasl.jaas.config` manually. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing + additional unit tests. Closes #27191 from gaborgsomogyi/SPARK-30495. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- .../HadoopDelegationTokenManager.scala | 61 ++++++++-------- .../kafka010/consumer/KafkaDataConsumer.scala | 2 +- .../spark/kafka010/KafkaTokenUtil.scala | 14 ++-- .../spark/kafka010/KafkaTokenUtilSuite.scala | 69 +++++++++++-------- 4 files changed, 83 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index f769ce468e49c..3168c763df4df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -65,11 +65,6 @@ private[spark] class HadoopDelegationTokenManager( protected val hadoopConf: Configuration, protected val schedulerRef: RpcEndpointRef) extends Logging { - private val deprecatedProviderEnabledConfigs = List( - "spark.yarn.security.tokens.%s.enabled", - "spark.yarn.security.credentials.%s.enabled") - private val providerEnabledConfig = "spark.security.credentials.%s.enabled" - private val principal = sparkConf.get(PRINCIPAL).orNull // The keytab can be a local: URI for cluster mode, so translate it to a regular path. If it is @@ -181,29 +176,6 @@ private[spark] class HadoopDelegationTokenManager( delegationTokenProviders.contains(serviceName) } - protected def isServiceEnabled(serviceName: String): Boolean = { - val key = providerEnabledConfig.format(serviceName) - - deprecatedProviderEnabledConfigs.foreach { pattern => - val deprecatedKey = pattern.format(serviceName) - if (sparkConf.contains(deprecatedKey)) { - logWarning(s"${deprecatedKey} is deprecated. Please use ${key} instead.") - } - } - - val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern => - sparkConf - .getOption(pattern.format(serviceName)) - .map(_.toBoolean) - .getOrElse(true) - } - - sparkConf - .getOption(key) - .map(_.toBoolean) - .getOrElse(isEnabledDeprecated) - } - private def scheduleRenewal(delay: Long): Unit = { val _delay = math.max(0, delay) logInfo(s"Scheduling renewal in ${UIUtils.formatDuration(delay)}.") @@ -299,8 +271,39 @@ private[spark] class HadoopDelegationTokenManager( // Filter out providers for which spark.security.credentials.{service}.enabled is false. providers - .filter { p => isServiceEnabled(p.serviceName) } + .filter { p => HadoopDelegationTokenManager.isServiceEnabled(sparkConf, p.serviceName) } .map { p => (p.serviceName, p) } .toMap } } + +private[spark] object HadoopDelegationTokenManager extends Logging { + private val providerEnabledConfig = "spark.security.credentials.%s.enabled" + + private val deprecatedProviderEnabledConfigs = List( + "spark.yarn.security.tokens.%s.enabled", + "spark.yarn.security.credentials.%s.enabled") + + def isServiceEnabled(sparkConf: SparkConf, serviceName: String): Boolean = { + val key = providerEnabledConfig.format(serviceName) + + deprecatedProviderEnabledConfigs.foreach { pattern => + val deprecatedKey = pattern.format(serviceName) + if (sparkConf.contains(deprecatedKey)) { + logWarning(s"${deprecatedKey} is deprecated. Please use ${key} instead.") + } + } + + val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern => + sparkConf + .getOption(pattern.format(serviceName)) + .map(_.toBoolean) + .getOrElse(true) + } + + sparkConf + .getOption(key) + .map(_.toBoolean) + .getOrElse(isEnabledDeprecated) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala index 7aa386f74c1c3..5f23029d9fed3 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/consumer/KafkaDataConsumer.scala @@ -528,7 +528,7 @@ private[kafka010] class KafkaDataConsumer( retrieveConsumer() } require(_consumer.isDefined, "Consumer must be defined") - if (!KafkaTokenUtil.isConnectorUsingCurrentToken(_consumer.get.kafkaParamsWithSecurity, + if (KafkaTokenUtil.needTokenUpdate(SparkEnv.get.conf, _consumer.get.kafkaParamsWithSecurity, _consumer.get.clusterConfig)) { logDebug("Cached consumer uses an old delegation token, invalidating.") releaseConsumer() diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala index 0ebe98330b4ae..49109d363ba8f 100644 --- a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala @@ -38,6 +38,7 @@ import org.apache.kafka.common.security.token.delegation.DelegationToken import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.util.Utils @@ -290,16 +291,17 @@ private[spark] object KafkaTokenUtil extends Logging { params } - def isConnectorUsingCurrentToken( + def needTokenUpdate( + sparkConf: SparkConf, params: ju.Map[String, Object], clusterConfig: Option[KafkaTokenClusterConf]): Boolean = { - if (params.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) { + if (HadoopDelegationTokenManager.isServiceEnabled(sparkConf, "kafka") && + clusterConfig.isDefined && params.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) { logDebug("Delegation token used by connector, checking if uses the latest token.") - val consumerJaasParams = params.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[String] - require(clusterConfig.isDefined, "Delegation token must exist for this connector.") - getTokenJaasParams(clusterConfig.get) == consumerJaasParams + val connectorJaasParams = params.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[String] + getTokenJaasParams(clusterConfig.get) != connectorJaasParams } else { - true + false } } } diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala index 6fa1b56bff977..94f7853003bd9 100644 --- a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala @@ -233,46 +233,61 @@ class KafkaTokenUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { assert(jaasParams.contains(tokenPassword1)) } - test("isConnectorUsingCurrentToken without security should return true") { - val kafkaParams = Map[String, Object]().asJava + test("needTokenUpdate without security credentials enabled should return false") { + sparkConf.set(s"spark.security.credentials.kafka.enabled", "false") + val kafkaParams = getKafkaParams(addJaasConfig = true, Some("custom_jaas_config")) - assert(KafkaTokenUtil.isConnectorUsingCurrentToken(kafkaParams, None)) + assert(!KafkaTokenUtil.needTokenUpdate(sparkConf, kafkaParams, None)) } - test("isConnectorUsingCurrentToken with same token should return true") { - setSparkEnv( - Map( - s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers - ) - ) + test("needTokenUpdate without cluster config should return false") { + val kafkaParams = getKafkaParams(addJaasConfig = true, Some("custom_jaas_config")) + + assert(!KafkaTokenUtil.needTokenUpdate(sparkConf, kafkaParams, None)) + } + + test("needTokenUpdate without jaas config should return false") { + setSparkEnv(Map.empty) + val kafkaParams = getKafkaParams(addJaasConfig = false) + + assert(!KafkaTokenUtil.needTokenUpdate(SparkEnv.get.conf, kafkaParams, None)) + } + + test("needTokenUpdate with same token should return false") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", bootStrapServers) addTokenToUGI(tokenService1, tokenId1, tokenPassword1) - val kafkaParams = getKafkaParams() - val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig(SparkEnv.get.conf, + val kafkaParams = getKafkaParams(addJaasConfig = true) + val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig(sparkConf, kafkaParams.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG).asInstanceOf[String]) - assert(KafkaTokenUtil.isConnectorUsingCurrentToken(kafkaParams, clusterConfig)) + assert(!KafkaTokenUtil.needTokenUpdate(sparkConf, kafkaParams, clusterConfig)) } - test("isConnectorUsingCurrentToken with different token should return false") { - setSparkEnv( - Map( - s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers - ) - ) + test("needTokenUpdate with different token should return true") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", bootStrapServers) addTokenToUGI(tokenService1, tokenId1, tokenPassword1) - val kafkaParams = getKafkaParams() + val kafkaParams = getKafkaParams(addJaasConfig = true) addTokenToUGI(tokenService1, tokenId2, tokenPassword2) - val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig(SparkEnv.get.conf, + val clusterConfig = KafkaTokenUtil.findMatchingTokenClusterConfig(sparkConf, kafkaParams.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG).asInstanceOf[String]) - assert(!KafkaTokenUtil.isConnectorUsingCurrentToken(kafkaParams, clusterConfig)) + assert(KafkaTokenUtil.needTokenUpdate(sparkConf, kafkaParams, clusterConfig)) } - private def getKafkaParams(): ju.Map[String, Object] = { - val clusterConf = createClusterConf(identifier1, SASL_SSL.name) - Map[String, Object]( - CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers, - SaslConfigs.SASL_JAAS_CONFIG -> KafkaTokenUtil.getTokenJaasParams(clusterConf) - ).asJava + private def getKafkaParams( + addJaasConfig: Boolean, + jaasConfig: Option[String] = None): ju.Map[String, Object] = { + var params = Map[String, Object]( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers + ) + if (addJaasConfig) { + params ++= Map[String, Object]( + SaslConfigs.SASL_JAAS_CONFIG -> jaasConfig.getOrElse { + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + KafkaTokenUtil.getTokenJaasParams(clusterConf) + } + ) + } + params.asJava } } From d42cf4566a9d4438fd1cae88674f0d02f3dbf5c9 Mon Sep 17 00:00:00 2001 From: Henrique Goulart Date: Wed, 15 Jan 2020 13:27:15 -0800 Subject: [PATCH 1115/1387] [SPARK-30246][CORE] OneForOneStreamManager might leak memory in connectionTerminated ### What changes were proposed in this pull request? Ensure that all StreamStates are removed from OneForOneStreamManager memory map even if there's an error trying to release buffers ### Why are the changes needed? OneForOneStreamManager may not remove all StreamStates from memory map when a connection is terminated. A RuntimeException might be thrown in StreamState$buffers.next() by one of ExternalShuffleBlockResolver$getBlockData... **breaking the loop through streams.entrySet(), keeping StreamStates in memory forever leaking memory.** That may happen when an application is terminated abruptly and executors removed before the connection is terminated or if shuffleIndexCache fails to get ShuffleIndexInformation References: https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java#L319 https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java#L357 https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L195 https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L208 https://github.com/apache/spark/blob/ee050ddbc6eb6bc08c7751a0eb00e7a05b011b52/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java#L330 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test added Closes #27064 from hensg/SPARK-30246. Lead-authored-by: Henrique Goulart Co-authored-by: Henrique Goulart Signed-off-by: Marcelo Vanzin --- .../server/OneForOneStreamManager.java | 24 +++++++++--- .../server/OneForOneStreamManagerSuite.java | 39 +++++++++++++++++++ 2 files changed, 58 insertions(+), 5 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index cf3547a68bffc..1a902a937a176 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -117,21 +117,35 @@ public static Pair parseStreamChunkId(String streamChunkId) { @Override public void connectionTerminated(Channel channel) { + RuntimeException failedToReleaseBufferException = null; + // Close all streams which have been associated with the channel. for (Map.Entry entry: streams.entrySet()) { StreamState state = entry.getValue(); if (state.associatedChannel == channel) { streams.remove(entry.getKey()); - // Release all remaining buffers. - while (state.buffers.hasNext()) { - ManagedBuffer buffer = state.buffers.next(); - if (buffer != null) { - buffer.release(); + try { + // Release all remaining buffers. + while (state.buffers.hasNext()) { + ManagedBuffer buffer = state.buffers.next(); + if (buffer != null) { + buffer.release(); + } + } + } catch (RuntimeException e) { + if (failedToReleaseBufferException == null) { + failedToReleaseBufferException = e; + } else { + logger.error("Exception trying to release remaining StreamState buffers", e); } } } } + + if (failedToReleaseBufferException != null) { + throw failedToReleaseBufferException; + } } @Override diff --git a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java index fb3503b783e54..45e1836da641f 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -18,6 +18,7 @@ package org.apache.spark.network.server; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import io.netty.channel.Channel; @@ -96,4 +97,42 @@ public void managedBuffersAreFreedWhenConnectionIsClosed() { Mockito.verify(buffer2, Mockito.times(1)).release(); Assert.assertEquals(0, manager.numStreamStates()); } + + @Test + public void streamStatesAreFreedWhenConnectionIsClosedEvenIfBufferIteratorThrowsException() { + OneForOneStreamManager manager = new OneForOneStreamManager(); + + Iterator buffers = Mockito.mock(Iterator.class); + Mockito.when(buffers.hasNext()).thenReturn(true); + Mockito.when(buffers.next()).thenThrow(RuntimeException.class); + + ManagedBuffer mockManagedBuffer = Mockito.mock(ManagedBuffer.class); + + Iterator buffers2 = Mockito.mock(Iterator.class); + Mockito.when(buffers2.hasNext()).thenReturn(true).thenReturn(true); + Mockito.when(buffers2.next()).thenReturn(mockManagedBuffer).thenThrow(RuntimeException.class); + + Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); + manager.registerStream("appId", buffers, dummyChannel); + manager.registerStream("appId", buffers2, dummyChannel); + + Assert.assertEquals(2, manager.numStreamStates()); + + try { + manager.connectionTerminated(dummyChannel); + Assert.fail("connectionTerminated should throw exception when fails to release all buffers"); + + } catch (RuntimeException e) { + + Mockito.verify(buffers, Mockito.times(1)).hasNext(); + Mockito.verify(buffers, Mockito.times(1)).next(); + + Mockito.verify(buffers2, Mockito.times(2)).hasNext(); + Mockito.verify(buffers2, Mockito.times(2)).next(); + + Mockito.verify(mockManagedBuffer, Mockito.times(1)).release(); + + Assert.assertEquals(0, manager.numStreamStates()); + } + } } From 8a926e448f12aed02bc1191fd16ea968e15c138b Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 16 Jan 2020 08:36:22 +0900 Subject: [PATCH 1116/1387] [SPARK-26736][SQL] Partition pruning through nondeterministic expressions in Hive tables ### What changes were proposed in this pull request? This PR intends to improve partition pruning for nondeterministic expressions in Hive tables: Before this PR: ``` scala> sql("""create table test(id int) partitioned by (dt string)""") scala> sql("""select * from test where dt='20190101' and rand() < 0.5""").explain() == Physical Plan == *(1) Filter ((isnotnull(dt#19) AND (dt#19 = 20190101)) AND (rand(6515336563966543616) < 0.5)) +- Scan hive default.test [id#18, dt#19], HiveTableRelation `default`.`test`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#18], [dt#19], Statistics(sizeInBytes=8.0 EiB) ``` After this PR: ``` == Physical Plan == *(1) Filter (rand(-9163956883277176328) < 0.5) +- Scan hive default.test [id#0, dt#1], HiveTableRelation `default`.`test`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [id#0], [dt#1], Statistics(sizeInBytes=8.0 EiB), [isnotnull(dt#1), (dt#1 = 20190101)] ``` This PR is the rework of #24118. ### Why are the changes needed? For better performance. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit tests added. Closes #27219 from maropu/SPARK-26736. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../spark/sql/hive/HiveStrategies.scala | 2 +- ...ry test-0-56a1c59bd13c2a83a91eb0ec658fcecc | 500 ++++++++++++++++++ .../sql/hive/execution/PruningSuite.scala | 7 + 3 files changed, 508 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/Partition pruning - with filter containing non-deterministic condition - query test-0-56a1c59bd13c2a83a91eb0ec658fcecc diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 33ca1889e944d..b9c98f4ea15e9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -252,7 +252,7 @@ private[hive] trait HiveStrategies { */ object HiveTableScans extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case PhysicalOperation(projectList, predicates, relation: HiveTableRelation) => + case ScanOperation(projectList, predicates, relation: HiveTableRelation) => // Filter out all predicates that only deal with partition keys, these are given to the // hive table scan operator to be used for partition pruning. val partitionKeyIds = AttributeSet(relation.partitionCols) diff --git a/sql/hive/src/test/resources/golden/Partition pruning - with filter containing non-deterministic condition - query test-0-56a1c59bd13c2a83a91eb0ec658fcecc b/sql/hive/src/test/resources/golden/Partition pruning - with filter containing non-deterministic condition - query test-0-56a1c59bd13c2a83a91eb0ec658fcecc new file mode 100644 index 0000000000000..0fe6b905e7781 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Partition pruning - with filter containing non-deterministic condition - query test-0-56a1c59bd13c2a83a91eb0ec658fcecc @@ -0,0 +1,500 @@ +val_238 11 +val_86 11 +val_311 11 +val_27 11 +val_165 11 +val_409 11 +val_255 11 +val_278 11 +val_98 11 +val_484 11 +val_265 11 +val_193 11 +val_401 11 +val_150 11 +val_273 11 +val_224 11 +val_369 11 +val_66 11 +val_128 11 +val_213 11 +val_146 11 +val_406 11 +val_429 11 +val_374 11 +val_152 11 +val_469 11 +val_145 11 +val_495 11 +val_37 11 +val_327 11 +val_281 11 +val_277 11 +val_209 11 +val_15 11 +val_82 11 +val_403 11 +val_166 11 +val_417 11 +val_430 11 +val_252 11 +val_292 11 +val_219 11 +val_287 11 +val_153 11 +val_193 11 +val_338 11 +val_446 11 +val_459 11 +val_394 11 +val_237 11 +val_482 11 +val_174 11 +val_413 11 +val_494 11 +val_207 11 +val_199 11 +val_466 11 +val_208 11 +val_174 11 +val_399 11 +val_396 11 +val_247 11 +val_417 11 +val_489 11 +val_162 11 +val_377 11 +val_397 11 +val_309 11 +val_365 11 +val_266 11 +val_439 11 +val_342 11 +val_367 11 +val_325 11 +val_167 11 +val_195 11 +val_475 11 +val_17 11 +val_113 11 +val_155 11 +val_203 11 +val_339 11 +val_0 11 +val_455 11 +val_128 11 +val_311 11 +val_316 11 +val_57 11 +val_302 11 +val_205 11 +val_149 11 +val_438 11 +val_345 11 +val_129 11 +val_170 11 +val_20 11 +val_489 11 +val_157 11 +val_378 11 +val_221 11 +val_92 11 +val_111 11 +val_47 11 +val_72 11 +val_4 11 +val_280 11 +val_35 11 +val_427 11 +val_277 11 +val_208 11 +val_356 11 +val_399 11 +val_169 11 +val_382 11 +val_498 11 +val_125 11 +val_386 11 +val_437 11 +val_469 11 +val_192 11 +val_286 11 +val_187 11 +val_176 11 +val_54 11 +val_459 11 +val_51 11 +val_138 11 +val_103 11 +val_239 11 +val_213 11 +val_216 11 +val_430 11 +val_278 11 +val_176 11 +val_289 11 +val_221 11 +val_65 11 +val_318 11 +val_332 11 +val_311 11 +val_275 11 +val_137 11 +val_241 11 +val_83 11 +val_333 11 +val_180 11 +val_284 11 +val_12 11 +val_230 11 +val_181 11 +val_67 11 +val_260 11 +val_404 11 +val_384 11 +val_489 11 +val_353 11 +val_373 11 +val_272 11 +val_138 11 +val_217 11 +val_84 11 +val_348 11 +val_466 11 +val_58 11 +val_8 11 +val_411 11 +val_230 11 +val_208 11 +val_348 11 +val_24 11 +val_463 11 +val_431 11 +val_179 11 +val_172 11 +val_42 11 +val_129 11 +val_158 11 +val_119 11 +val_496 11 +val_0 11 +val_322 11 +val_197 11 +val_468 11 +val_393 11 +val_454 11 +val_100 11 +val_298 11 +val_199 11 +val_191 11 +val_418 11 +val_96 11 +val_26 11 +val_165 11 +val_327 11 +val_230 11 +val_205 11 +val_120 11 +val_131 11 +val_51 11 +val_404 11 +val_43 11 +val_436 11 +val_156 11 +val_469 11 +val_468 11 +val_308 11 +val_95 11 +val_196 11 +val_288 11 +val_481 11 +val_457 11 +val_98 11 +val_282 11 +val_197 11 +val_187 11 +val_318 11 +val_318 11 +val_409 11 +val_470 11 +val_137 11 +val_369 11 +val_316 11 +val_169 11 +val_413 11 +val_85 11 +val_77 11 +val_0 11 +val_490 11 +val_87 11 +val_364 11 +val_179 11 +val_118 11 +val_134 11 +val_395 11 +val_282 11 +val_138 11 +val_238 11 +val_419 11 +val_15 11 +val_118 11 +val_72 11 +val_90 11 +val_307 11 +val_19 11 +val_435 11 +val_10 11 +val_277 11 +val_273 11 +val_306 11 +val_224 11 +val_309 11 +val_389 11 +val_327 11 +val_242 11 +val_369 11 +val_392 11 +val_272 11 +val_331 11 +val_401 11 +val_242 11 +val_452 11 +val_177 11 +val_226 11 +val_5 11 +val_497 11 +val_402 11 +val_396 11 +val_317 11 +val_395 11 +val_58 11 +val_35 11 +val_336 11 +val_95 11 +val_11 11 +val_168 11 +val_34 11 +val_229 11 +val_233 11 +val_143 11 +val_472 11 +val_322 11 +val_498 11 +val_160 11 +val_195 11 +val_42 11 +val_321 11 +val_430 11 +val_119 11 +val_489 11 +val_458 11 +val_78 11 +val_76 11 +val_41 11 +val_223 11 +val_492 11 +val_149 11 +val_449 11 +val_218 11 +val_228 11 +val_138 11 +val_453 11 +val_30 11 +val_209 11 +val_64 11 +val_468 11 +val_76 11 +val_74 11 +val_342 11 +val_69 11 +val_230 11 +val_33 11 +val_368 11 +val_103 11 +val_296 11 +val_113 11 +val_216 11 +val_367 11 +val_344 11 +val_167 11 +val_274 11 +val_219 11 +val_239 11 +val_485 11 +val_116 11 +val_223 11 +val_256 11 +val_263 11 +val_70 11 +val_487 11 +val_480 11 +val_401 11 +val_288 11 +val_191 11 +val_5 11 +val_244 11 +val_438 11 +val_128 11 +val_467 11 +val_432 11 +val_202 11 +val_316 11 +val_229 11 +val_469 11 +val_463 11 +val_280 11 +val_2 11 +val_35 11 +val_283 11 +val_331 11 +val_235 11 +val_80 11 +val_44 11 +val_193 11 +val_321 11 +val_335 11 +val_104 11 +val_466 11 +val_366 11 +val_175 11 +val_403 11 +val_483 11 +val_53 11 +val_105 11 +val_257 11 +val_406 11 +val_409 11 +val_190 11 +val_406 11 +val_401 11 +val_114 11 +val_258 11 +val_90 11 +val_203 11 +val_262 11 +val_348 11 +val_424 11 +val_12 11 +val_396 11 +val_201 11 +val_217 11 +val_164 11 +val_431 11 +val_454 11 +val_478 11 +val_298 11 +val_125 11 +val_431 11 +val_164 11 +val_424 11 +val_187 11 +val_382 11 +val_5 11 +val_70 11 +val_397 11 +val_480 11 +val_291 11 +val_24 11 +val_351 11 +val_255 11 +val_104 11 +val_70 11 +val_163 11 +val_438 11 +val_119 11 +val_414 11 +val_200 11 +val_491 11 +val_237 11 +val_439 11 +val_360 11 +val_248 11 +val_479 11 +val_305 11 +val_417 11 +val_199 11 +val_444 11 +val_120 11 +val_429 11 +val_169 11 +val_443 11 +val_323 11 +val_325 11 +val_277 11 +val_230 11 +val_478 11 +val_178 11 +val_468 11 +val_310 11 +val_317 11 +val_333 11 +val_493 11 +val_460 11 +val_207 11 +val_249 11 +val_265 11 +val_480 11 +val_83 11 +val_136 11 +val_353 11 +val_172 11 +val_214 11 +val_462 11 +val_233 11 +val_406 11 +val_133 11 +val_175 11 +val_189 11 +val_454 11 +val_375 11 +val_401 11 +val_421 11 +val_407 11 +val_384 11 +val_256 11 +val_26 11 +val_134 11 +val_67 11 +val_384 11 +val_379 11 +val_18 11 +val_462 11 +val_492 11 +val_100 11 +val_298 11 +val_9 11 +val_341 11 +val_498 11 +val_146 11 +val_458 11 +val_362 11 +val_186 11 +val_285 11 +val_348 11 +val_167 11 +val_18 11 +val_273 11 +val_183 11 +val_281 11 +val_344 11 +val_97 11 +val_469 11 +val_315 11 +val_84 11 +val_28 11 +val_37 11 +val_448 11 +val_152 11 +val_348 11 +val_307 11 +val_194 11 +val_414 11 +val_477 11 +val_222 11 +val_126 11 +val_90 11 +val_169 11 +val_403 11 +val_400 11 +val_200 11 +val_97 11 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index cc592cf6ca629..390181ce5d46e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -141,6 +141,13 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { Seq("2008-04-08", "11"), Seq("2008-04-09", "11"))) + createPruningTest("Partition pruning - with filter containing non-deterministic condition", + "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08' AND hr < 12 AND rand() < 1", + Seq("value", "hr"), + Seq("value", "hr"), + Seq( + Seq("2008-04-08", "11"))) + def createPruningTest( testCaseName: String, sql: String, From 883ae331c36aa2279b163b87571997c8ffebfb6c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 15 Jan 2020 17:38:52 -0800 Subject: [PATCH 1117/1387] [SPARK-30497][SQL] migrate DESCRIBE TABLE to the new framework ### What changes were proposed in this pull request? Use the new framework to resolve the DESCRIBE TABLE command. The v1 DESCRIBE TABLE command supports both table and view. Checked with Hive and Presto, they don't have DESCRIBE TABLE syntax but only DESCRIBE, which supports both table and view: 1. https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-DescribeTable/View/MaterializedView/Column 2. https://prestodb.io/docs/current/sql/describe.html We should make it clear that DESCRIBE support both table and view, by renaming the command to `DescribeRelation`. This PR also tunes the framework a little bit to support the case that a command accepts both table and view. ### Why are the changes needed? This is a part of effort to make the relation lookup behavior consistent: SPARK-29900. Note that I make a separate PR here instead of #26921, as I need to update the framework to support a new use case: accept both table and view. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #27187 from cloud-fan/describe. Authored-by: Wenchen Fan Signed-off-by: Xiao Li --- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 45 +++++++---- .../sql/catalyst/analysis/CheckAnalysis.scala | 10 +-- .../catalyst/analysis/ResolveCatalogs.scala | 8 -- .../sql/catalyst/analysis/namespace.scala | 33 -------- .../spark/sql/catalyst/analysis/table.scala | 33 -------- .../catalyst/analysis/v2ResolutionPlans.scala | 76 +++++++++++++++++++ .../sql/catalyst/parser/AstBuilder.scala | 8 +- .../catalyst/plans/logical/statements.scala | 8 -- .../catalyst/plans/logical/v2Commands.scala | 12 +-- .../catalog/CatalogV2Implicits.scala | 8 ++ .../sql/catalyst/parser/DDLParserSuite.scala | 10 +-- .../analysis/ResolveSessionCatalog.scala | 25 ++---- .../datasources/v2/DataSourceV2Strategy.scala | 9 ++- .../datasources/v2/V2SessionCatalog.scala | 2 +- .../sql-tests/results/describe.sql.out | 3 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 2 +- .../sql/connector/DataSourceV2SQLSuite.scala | 5 ++ .../sql/execution/SparkSqlParserSuite.scala | 4 +- .../command/PlanResolutionSuite.scala | 62 +++++++-------- .../hive/execution/HiveComparisonTest.scala | 2 +- 21 files changed, 186 insertions(+), 181 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/namespace.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/table.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 0532bc163b5de..645d0d709e063 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -218,7 +218,7 @@ statement | (DESC | DESCRIBE) namespace EXTENDED? multipartIdentifier #describeNamespace | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? - multipartIdentifier partitionSpec? describeColName? #describeTable + multipartIdentifier partitionSpec? describeColName? #describeRelation | (DESC | DESCRIBE) QUERY? query #describeQuery | COMMENT ON namespace multipartIdentifier IS comment=(STRING | NULL) #commentNamespace diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index adea15ded1be7..6e0e257038fa0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -759,6 +759,8 @@ class Analyzer( u.failAnalysis(s"${ident.quoted} is a temp view not table.") } u + case u @ UnresolvedTableOrView(ident) => + lookupTempView(ident).map(_ => ResolvedView(ident.asIdentifier)).getOrElse(u) } def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = { @@ -803,16 +805,16 @@ class Analyzer( .map(ResolvedTable(catalog.asTableCatalog, ident, _)) .getOrElse(u) + case u @ UnresolvedTableOrView(NonSessionCatalogAndIdentifier(catalog, ident)) => + CatalogV2Util.loadTable(catalog, ident) + .map(ResolvedTable(catalog.asTableCatalog, ident, _)) + .getOrElse(u) + case i @ InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) if i.query.resolved => lookupV2Relation(u.multipartIdentifier) .map(v2Relation => i.copy(table = v2Relation)) .getOrElse(i) - case desc @ DescribeTable(u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.tableName) - .map(rel => desc.copy(table = rel)) - .getOrElse(desc) - case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => CatalogV2Util.loadRelation(u.catalog, u.tableName) .map(rel => alter.copy(table = rel)) @@ -889,17 +891,28 @@ class Analyzer( case u: UnresolvedRelation => lookupRelation(u.multipartIdentifier).map(resolveViews).getOrElse(u) - case u @ UnresolvedTable(identifier: Seq[String]) => - expandRelationName(identifier) match { - case SessionCatalogAndIdentifier(catalog, ident) => - CatalogV2Util.loadTable(catalog, ident) match { - case Some(v1Table: V1Table) if v1Table.v1Table.tableType == CatalogTableType.VIEW => - u.failAnalysis(s"$ident is a view not table.") - case Some(table) => ResolvedTable(catalog.asTableCatalog, ident, table) - case None => u - } - case _ => u - } + case u @ UnresolvedTable(identifier) => + lookupTableOrView(identifier).map { + case v: ResolvedView => + u.failAnalysis(s"${v.identifier.quoted} is a view not table.") + case table => table + }.getOrElse(u) + + case u @ UnresolvedTableOrView(identifier) => + lookupTableOrView(identifier).getOrElse(u) + } + + private def lookupTableOrView(identifier: Seq[String]): Option[LogicalPlan] = { + expandRelationName(identifier) match { + case SessionCatalogAndIdentifier(catalog, ident) => + CatalogV2Util.loadTable(catalog, ident).map { + case v1Table: V1Table if v1Table.v1Table.tableType == CatalogTableType.VIEW => + ResolvedView(ident) + case table => + ResolvedTable(catalog.asTableCatalog, ident, table) + } + case _ => None + } } // Look up a relation from the session catalog with the following logic: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index bd50a6b36efc2..1d44c84f3123f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -98,6 +98,9 @@ trait CheckAnalysis extends PredicateHelper { case u: UnresolvedTable => u.failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") + case u: UnresolvedTableOrView => + u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") + case u: UnresolvedRelation => u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") @@ -118,13 +121,6 @@ trait CheckAnalysis extends PredicateHelper { case AlterTable(_, _, u: UnresolvedV2Relation, _) => failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") - case DescribeTable(u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => - u.failAnalysis( - s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") - - case DescribeTable(u: UnresolvedV2Relation, _) => - failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") - case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. // If the arguments of the higher-order functions are resolved but the type check fails, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 034e30d5a234f..b73ba01389067 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -115,14 +115,6 @@ class ResolveCatalogs(val catalogManager: CatalogManager) } RenameTable(catalog.asTableCatalog, oldName.asIdentifier, newNameParts.asIdentifier) - case DescribeTableStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), partitionSpec, isExtended) => - if (partitionSpec.nonEmpty) { - throw new AnalysisException("DESCRIBE TABLE does not support partition for v2 tables.") - } - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tbl.asIdentifier) - DescribeTable(r, isExtended) - case DescribeColumnStatement( NonSessionCatalogAndTable(catalog, tbl), colNameParts, isExtended) => throw new AnalysisException("Describing columns is not supported for v2 tables.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/namespace.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/namespace.scala deleted file mode 100644 index da6955d84991e..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/namespace.scala +++ /dev/null @@ -1,33 +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.sql.catalyst.analysis - -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LeafNode -import org.apache.spark.sql.connector.catalog.SupportsNamespaces - -case class ResolvedNamespace(catalog: SupportsNamespaces, namespace: Seq[String]) - extends LeafNode { - override def output: Seq[Attribute] = Nil -} - -case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNode { - override lazy val resolved: Boolean = false - - override def output: Seq[Attribute] = Nil -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/table.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/table.scala deleted file mode 100644 index 855e707d9014b..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/table.scala +++ /dev/null @@ -1,33 +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.sql.catalyst.analysis - -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LeafNode -import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} - -case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: Table) - extends LeafNode { - override def output: Seq[Attribute] = Nil -} - -case class UnresolvedTable(multipartIdentifier: Seq[String]) extends LeafNode { - override lazy val resolved: Boolean = false - - override def output: Seq[Attribute] = Nil -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala new file mode 100644 index 0000000000000..239f987e97a76 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces, Table, TableCatalog} + +/** + * Holds the name of a namespace that has yet to be looked up in a catalog. It will be resolved to + * [[ResolvedNamespace]] during analysis. + */ +case class UnresolvedNamespace(multipartIdentifier: Seq[String]) extends LeafNode { + override lazy val resolved: Boolean = false + + override def output: Seq[Attribute] = Nil +} + +/** + * Holds the name of a table that has yet to be looked up in a catalog. It will be resolved to + * [[ResolvedTable]] during analysis. + */ +case class UnresolvedTable(multipartIdentifier: Seq[String]) extends LeafNode { + override lazy val resolved: Boolean = false + + override def output: Seq[Attribute] = Nil +} + +/** + * Holds the name of a table or view that has yet to be looked up in a catalog. It will + * be resolved to [[ResolvedTable]] or [[ResolvedView]] during analysis. + */ +case class UnresolvedTableOrView(multipartIdentifier: Seq[String]) extends LeafNode { + override lazy val resolved: Boolean = false + override def output: Seq[Attribute] = Nil +} + +/** + * A plan containing resolved namespace. + */ +case class ResolvedNamespace(catalog: SupportsNamespaces, namespace: Seq[String]) + extends LeafNode { + override def output: Seq[Attribute] = Nil +} + +/** + * A plan containing resolved table. + */ +case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: Table) + extends LeafNode { + override def output: Seq[Attribute] = Nil +} + +/** + * A plan containing resolved (temp) views. + */ +// TODO: create a generic representation for temp view, v1 view and v2 view, after we add view +// support to v2 catalog. For now we only need the identifier to fallback to v1 command. +case class ResolvedView(identifier: Identifier) extends LeafNode { + override def output: Seq[Attribute] = Nil +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 7ffb282626473..e326e48da6581 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3062,9 +3062,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Create a [[DescribeColumnStatement]] or [[DescribeTableStatement]] commands. + * Create a [[DescribeColumnStatement]] or [[DescribeRelation]] commands. */ - override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) { + override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) { val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null if (ctx.describeColName != null) { if (ctx.partitionSpec != null) { @@ -3086,8 +3086,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } else { Map.empty[String, String] } - DescribeTableStatement( - visitMultipartIdentifier(ctx.multipartIdentifier()), + DescribeRelation( + UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())), partitionSpec, isExtended) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 476ecf5bfdfce..1e097899602a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -292,14 +292,6 @@ case class DropViewStatement( viewName: Seq[String], ifExists: Boolean) extends ParsedStatement -/** - * A DESCRIBE TABLE tbl_name statement, as parsed from SQL. - */ -case class DescribeTableStatement( - tableName: Seq[String], - partitionSpec: TablePartitionSpec, - isExtended: Boolean) extends ParsedStatement - /** * A DESCRIBE TABLE tbl_name col_name statement, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index f1946dc7bb09d..e98b2cf7abfcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.analysis.{NamedRelation, UnresolvedException} +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Unevaluable} import org.apache.spark.sql.catalyst.plans.DescribeTableSchema import org.apache.spark.sql.connector.catalog._ @@ -314,12 +315,13 @@ case class ShowNamespaces( } /** - * The logical plan of the DESCRIBE TABLE command that works for v2 tables. + * The logical plan of the DESCRIBE relation_name command that works for v2 tables. */ -case class DescribeTable(table: NamedRelation, isExtended: Boolean) extends Command { - - override lazy val resolved: Boolean = table.resolved - +case class DescribeRelation( + relation: LogicalPlan, + partitionSpec: TablePartitionSpec, + isExtended: Boolean) extends Command { + override def children: Seq[LogicalPlan] = Seq(relation) override def output: Seq[Attribute] = DescribeTableSchema.describeTableAttributes() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 86e5894b369aa..16aec23521f9f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -98,6 +98,14 @@ private[sql] object CatalogV2Implicits { } def asMultipartIdentifier: Seq[String] = ident.namespace :+ ident.name + + def asTableIdentifier: TableIdentifier = ident.namespace match { + case ns if ns.isEmpty => TableIdentifier(ident.name) + case Array(dbName) => TableIdentifier(ident.name, Some(dbName)) + case _ => + throw new AnalysisException( + s"$quoted is not a valid TableIdentifier as it has more than 2 name parts.") + } } implicit class MultipartIdentifierHelper(parts: Seq[String]) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 53007a23553e4..3a4c08235731e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, PersistedView, UnresolvedAttribute, UnresolvedNamespace, UnresolvedRelation, UnresolvedStar, UnresolvedTable, UnresolvedTableOrView} import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, BucketSpec, FileResource, FunctionResource, FunctionResourceType, JarResource} import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ @@ -791,13 +791,13 @@ class DDLParserSuite extends AnalysisTest { test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), - DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) + DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = false)) comparePlans(parsePlan("describe table t"), - DescribeTableStatement(Seq("t"), Map.empty, isExtended = false)) + DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = false)) comparePlans(parsePlan("describe table extended t"), - DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) + DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = true)) comparePlans(parsePlan("describe table formatted t"), - DescribeTableStatement(Seq("t"), Map.empty, isExtended = true)) + DescribeRelation(UnresolvedTableOrView(Seq("t")), Map.empty, isExtended = true)) } test("insert table: basic append") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index c4325289507b6..f0c87812daff8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} @@ -214,23 +214,12 @@ class ResolveSessionCatalog( case RenameTableStatement(SessionCatalogAndTable(_, oldName), newNameParts, isView) => AlterTableRenameCommand(oldName.asTableIdentifier, newNameParts.asTableIdentifier, isView) - case DescribeTableStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), partitionSpec, isExtended) => - loadTable(catalog, tbl.asIdentifier).collect { - case v1Table: V1Table => - DescribeTableCommand(tbl.asTableIdentifier, partitionSpec, isExtended) - }.getOrElse { - // The v1 `DescribeTableCommand` can describe view as well. - if (isView(tbl)) { - DescribeTableCommand(tbl.asTableIdentifier, partitionSpec, isExtended) - } else { - if (partitionSpec.nonEmpty) { - throw new AnalysisException("DESCRIBE TABLE does not support partition for v2 tables.") - } - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tbl.asIdentifier) - DescribeTable(r, isExtended) - } - } + case DescribeRelation(ResolvedTable(_, ident, _: V1Table), partitionSpec, isExtended) => + DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) + + // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. + case DescribeRelation(ResolvedView(ident), partitionSpec, isExtended) => + DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) case DescribeColumnStatement( SessionCatalogAndTable(catalog, tbl), colNameParts, isExtended) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 430f76e15b273..b452b66e03813 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.{AnalysisException, Strategy} import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable} import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetLocation, AlterNamespaceSetOwner, AlterNamespaceSetProperties, AlterTable, AppendData, CommentOnNamespace, CommentOnTable, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, RenameTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables} -import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, TableChange} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, TableChange} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy @@ -197,7 +197,10 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case desc @ DescribeNamespace(ResolvedNamespace(catalog, ns), extended) => DescribeNamespaceExec(desc.output, catalog, ns, extended) :: Nil - case desc @ DescribeTable(DataSourceV2Relation(table, _, _), isExtended) => + case desc @ DescribeRelation(ResolvedTable(_, _, table), partitionSpec, isExtended) => + if (partitionSpec.nonEmpty) { + throw new AnalysisException("DESCRIBE does not support partition for v2 tables.") + } DescribeTableExec(desc.output, table, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 67d24536a5a46..ddb2926eb6c9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap */ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) extends TableCatalog with SupportsNamespaces { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import V2SessionCatalog._ override val defaultNamespace: Array[String] = Array("default") diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 01eff0888e444..1d83717f60b8d 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -539,7 +539,8 @@ EXPLAIN EXTENDED DESC t struct -- !query 34 output == Parsed Logical Plan == -'DescribeTableStatement [t], false +'DescribeRelation false ++- 'UnresolvedTableOrView [t] == Analyzed Logical Plan == col_name: string, data_type: string, comment: string diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 03d0aa999f5d1..75a630c200cce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -460,7 +460,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false case _: DescribeCommandBase | _: DescribeColumnCommand - | _: DescribeTableStatement + | _: DescribeRelation | _: DescribeColumnStatement => true case PhysicalOperation(_, _, Sort(_, true, _)) => true case _ => plan.children.iterator.exists(isSorted) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index e9ec42a8c3b25..5a7774abf0926 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -120,6 +120,11 @@ class DataSourceV2SQLSuite Row("", "", ""), Row("# Partitioning", "", ""), Row("Part 0", "id", ""))) + + val e = intercept[AnalysisException] { + sql("DESCRIBE TABLE testcat.table_name PARTITION (id = 1)") + } + assert(e.message.contains("DESCRIBE does not support partition for v2 tables")) } test("DescribeTable with v2 catalog when table does not exist.") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 56dceba06d157..06574a9f8fd2c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} -import org.apache.spark.sql.catalyst.plans.logical.{DescribeColumnStatement, DescribeTableStatement, LogicalPlan, Project, RepartitionByExpression, Sort} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 3f95e92e95b64..0901c66cccceb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -26,11 +26,11 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolvedTable, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{EqualTo, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeTable, DropTable, InsertAction, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SubqueryAlias, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.InMemoryTableProvider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.execution.datasources.CreateTable @@ -49,22 +49,20 @@ class PlanResolutionSuite extends AnalysisTest { t } - private val table1: Table = { - val t = mock(classOf[Table]) - when(t.schema()).thenReturn(new StructType().add("i", "int")) - t + private val v1Table: V1Table = { + val t = mock(classOf[CatalogTable]) + when(t.schema).thenReturn(new StructType().add("i", "int")) + when(t.tableType).thenReturn(CatalogTableType.MANAGED) + V1Table(t) } private val testCat: TableCatalog = { val newCatalog = mock(classOf[TableCatalog]) when(newCatalog.loadTable(any())).thenAnswer((invocation: InvocationOnMock) => { invocation.getArgument[Identifier](0).name match { - case "tab" => - table - case "tab1" => - table1 - case name => - throw new NoSuchTableException(name) + case "tab" => table + case "tab1" => table + case name => throw new NoSuchTableException(name) } }) when(newCatalog.name()).thenReturn("testcat") @@ -75,20 +73,11 @@ class PlanResolutionSuite extends AnalysisTest { val newCatalog = mock(classOf[TableCatalog]) when(newCatalog.loadTable(any())).thenAnswer((invocation: InvocationOnMock) => { invocation.getArgument[Identifier](0).name match { - case "v1Table" => - val v1Table = mock(classOf[V1Table]) - when(v1Table.schema).thenReturn(new StructType().add("i", "int")) - v1Table - case "v1Table1" => - val v1Table1 = mock(classOf[V1Table]) - when(v1Table1.schema).thenReturn(new StructType().add("i", "int")) - v1Table1 - case "v2Table" => - table - case "v2Table1" => - table1 - case name => - throw new NoSuchTableException(name) + case "v1Table" => v1Table + case "v1Table1" => v1Table + case "v2Table" => table + case "v2Table1" => table + case name => throw new NoSuchTableException(name) } }) when(newCatalog.name()).thenReturn(CatalogManager.SESSION_CATALOG_NAME) @@ -99,6 +88,7 @@ class PlanResolutionSuite extends AnalysisTest { new InMemoryCatalog, EmptyFunctionRegistry, new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) + v1SessionCatalog.createTempView("v", LocalRelation(Nil), false) private val catalogManagerWithDefault = { val manager = mock(classOf[CatalogManager]) @@ -140,10 +130,10 @@ class PlanResolutionSuite extends AnalysisTest { val analyzer = new Analyzer(catalogManager, conf) val rules = Seq( CTESubstitution, + analyzer.ResolveRelations, new ResolveCatalogs(catalogManager), new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), - analyzer.ResolveTables, - analyzer.ResolveRelations) + analyzer.ResolveTables) rules.foldLeft(parsePlan(query)) { case (plan, rule) => rule.apply(plan) } @@ -812,7 +802,7 @@ class PlanResolutionSuite extends AnalysisTest { } } - test("DESCRIBE TABLE") { + test("DESCRIBE relation") { Seq("v1Table" -> true, "v2Table" -> false, "testcat.tab" -> false).foreach { case (tblName, useV1Command) => val sql1 = s"DESC TABLE $tblName" @@ -827,27 +817,31 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed2, expected2) } else { parsed1 match { - case DescribeTable(_: DataSourceV2Relation, isExtended) => + case DescribeRelation(_: ResolvedTable, _, isExtended) => assert(!isExtended) case _ => fail("Expect DescribeTable, but got:\n" + parsed1.treeString) } parsed2 match { - case DescribeTable(_: DataSourceV2Relation, isExtended) => + case DescribeRelation(_: ResolvedTable, _, isExtended) => assert(isExtended) case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) } } val sql3 = s"DESC TABLE $tblName PARTITION(a=1)" + val parsed3 = parseAndResolve(sql3) if (useV1Command) { - val parsed3 = parseAndResolve(sql3) val expected3 = DescribeTableCommand( TableIdentifier(tblName, None), Map("a" -> "1"), false) comparePlans(parsed3, expected3) } else { - val e = intercept[AnalysisException](parseAndResolve(sql3)) - assert(e.message.contains("DESCRIBE TABLE does not support partition for v2 tables")) + parsed3 match { + case DescribeRelation(_: ResolvedTable, partitionSpec, isExtended) => + assert(!isExtended) + assert(partitionSpec == Map("a" -> "1")) + case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index e1615f17a7bae..28e1db961f611 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -376,7 +376,7 @@ abstract class HiveComparisonTest (!hiveQuery.logical.isInstanceOf[ShowFunctionsStatement]) && (!hiveQuery.logical.isInstanceOf[DescribeFunctionStatement]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && - (!hiveQuery.logical.isInstanceOf[DescribeTableStatement]) && + (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && (!hiveQuery.logical.isInstanceOf[DescribeColumnStatement]) && preparedHive != catalyst) { From a3a42b30d04009282e770c289b043ca5941e32e5 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 16 Jan 2020 11:11:36 +0900 Subject: [PATCH 1118/1387] [SPARK-27986][SQL][FOLLOWUP] Respect filter in sql/toString of AggregateExpression ### What changes were proposed in this pull request? This pr intends to add filter information in the explain output of an aggregate (This is a follow-up of #26656). Without this pr: ``` scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").explain(true) == Parsed Logical Plan == 'Aggregate ['k], ['k, unresolvedalias('SUM('v, ('v > 3)), None)] +- 'UnresolvedRelation [t] == Analyzed Logical Plan == k: int, sum(v): bigint Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) AS sum(v)#3L] +- SubqueryAlias `default`.`t` +- Relation[k#0,v#1] parquet == Optimized Logical Plan == Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) AS sum(v)#3L] +- Relation[k#0,v#1] parquet == Physical Plan == HashAggregate(keys=[k#0], functions=[sum(cast(v#1 as bigint))], output=[k#0, sum(v)#3L]) +- Exchange hashpartitioning(k#0, 200), true, [id=#20] +- HashAggregate(keys=[k#0], functions=[partial_sum(cast(v#1 as bigint))], output=[k#0, sum#7L]) +- *(1) ColumnarToRow +- FileScan parquet default.t[k#0,v#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-master/spark-warehouse/t], PartitionFilters: [], PushedFilters: [], ReadSchema: struct scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").show() +---+------+ | k|sum(v)| +---+------+ +---+------+ ``` With this pr: ``` scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").explain(true) == Parsed Logical Plan == 'Aggregate ['k], ['k, unresolvedalias('SUM('v, ('v > 3)), None)] +- 'UnresolvedRelation [t] == Analyzed Logical Plan == k: int, sum(v) FILTER (v > 3): bigint Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) filter (v#1 > 3) AS sum(v) FILTER (v > 3)#5L] +- SubqueryAlias `default`.`t` +- Relation[k#0,v#1] parquet == Optimized Logical Plan == Aggregate [k#0], [k#0, sum(cast(v#1 as bigint)) filter (v#1 > 3) AS sum(v) FILTER (v > 3)#5L] +- Relation[k#0,v#1] parquet == Physical Plan == HashAggregate(keys=[k#0], functions=[sum(cast(v#1 as bigint))], output=[k#0, sum(v) FILTER (v > 3)#5L]) +- Exchange hashpartitioning(k#0, 200), true, [id=#20] +- HashAggregate(keys=[k#0], functions=[partial_sum(cast(v#1 as bigint)) filter (v#1 > 3)], output=[k#0, sum#9L]) +- *(1) ColumnarToRow +- FileScan parquet default.t[k#0,v#1] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/maropu/Repositories/spark/spark-master/spark-warehouse/t], PartitionFilters: [], PushedFilters: [], ReadSchema: struct scala> sql("select k, SUM(v) filter (where v > 3) from t group by k").show() +---+---------------------+ | k|sum(v) FILTER (v > 3)| +---+---------------------+ +---+---------------------+ ``` ### Why are the changes needed? For better usability. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually. Closes #27198 from maropu/SPARK-27986-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- .../expressions/aggregate/interfaces.scala | 19 ++++-- .../sql/execution/aggregate/AggUtils.scala | 23 +++++-- .../sql-tests/results/group-by-filter.sql.out | 62 +++++++++---------- .../postgreSQL/aggregates_part3.sql.out | 4 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 33 +++++----- 5 files changed, 86 insertions(+), 55 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index 1f85b07d8385d..24731c4b8577c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -137,10 +137,11 @@ case class AggregateExpression( @transient override lazy val references: AttributeSet = { - mode match { - case Partial | Complete => aggregateFunction.references ++ filterAttributes + val aggAttributes = mode match { + case Partial | Complete => aggregateFunction.references case PartialMerge | Final => AttributeSet(aggregateFunction.aggBufferAttributes) } + aggAttributes ++ filterAttributes } override def toString: String = { @@ -149,10 +150,20 @@ case class AggregateExpression( case PartialMerge => "merge_" case Final | Complete => "" } - prefix + aggregateFunction.toAggString(isDistinct) + val aggFuncStr = prefix + aggregateFunction.toAggString(isDistinct) + filter match { + case Some(predicate) => s"$aggFuncStr FILTER (WHERE $predicate)" + case _ => aggFuncStr + } } - override def sql: String = aggregateFunction.sql(isDistinct) + override def sql: String = { + val aggFuncStr = aggregateFunction.sql(isDistinct) + filter match { + case Some(predicate) => s"$aggFuncStr FILTER (WHERE ${predicate.sql})" + case _ => aggFuncStr + } + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index e729fa278e9f3..56a287d4d0279 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.optimizer.NormalizeFloatingNumbers import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.streaming.{StateStoreRestoreExec, StateStoreSaveExec} @@ -27,6 +26,22 @@ import org.apache.spark.sql.execution.streaming.{StateStoreRestoreExec, StateSto * Utility functions used by the query planner to convert our plan to new aggregation code path. */ object AggUtils { + + private def mayRemoveAggFilters(exprs: Seq[AggregateExpression]): Seq[AggregateExpression] = { + exprs.map { ae => + if (ae.filter.isDefined) { + ae.mode match { + // Aggregate filters are applicable only in partial/complete modes; + // this method filters out them, otherwise. + case Partial | Complete => ae + case _ => ae.copy(filter = None) + } + } else { + ae + } + } + } + private def createAggregate( requiredChildDistributionExpressions: Option[Seq[Expression]] = None, groupingExpressions: Seq[NamedExpression] = Nil, @@ -41,7 +56,7 @@ object AggUtils { HashAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, groupingExpressions = groupingExpressions, - aggregateExpressions = aggregateExpressions, + aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, initialInputBufferOffset = initialInputBufferOffset, resultExpressions = resultExpressions, @@ -54,7 +69,7 @@ object AggUtils { ObjectHashAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, groupingExpressions = groupingExpressions, - aggregateExpressions = aggregateExpressions, + aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, initialInputBufferOffset = initialInputBufferOffset, resultExpressions = resultExpressions, @@ -63,7 +78,7 @@ object AggUtils { SortAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, groupingExpressions = groupingExpressions, - aggregateExpressions = aggregateExpressions, + aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, initialInputBufferOffset = initialInputBufferOffset, resultExpressions = resultExpressions, diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out index 5d266c980a49a..fbb66878f891f 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -51,13 +51,13 @@ SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) AS `count(b)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; +grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) FILTER (WHERE (testdata.`a` >= 2)) AS `count(b) FILTER (WHERE (a >= 2))`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; -- !query 4 SELECT COUNT(a) FILTER (WHERE a = 1), COUNT(b) FILTER (WHERE a > 1) FROM testData -- !query 4 schema -struct +struct 1)):bigint> -- !query 4 output 2 4 @@ -65,7 +65,7 @@ struct -- !query 5 SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp -- !query 5 schema -struct +struct -- !query 5 output 2 @@ -73,7 +73,7 @@ struct -- !query 6 SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp -- !query 6 schema -struct +struct -- !query 6 output 2 @@ -81,7 +81,7 @@ struct -- !query 7 SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp -- !query 7 schema -struct +struct -- !query 7 output 2 @@ -89,7 +89,7 @@ struct -- !query 8 SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp -- !query 8 schema -struct +struct -- !query 8 output 2 @@ -97,7 +97,7 @@ struct -- !query 9 SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a -- !query 9 schema -struct +struct= 2)):bigint> -- !query 9 output 1 0 2 2 @@ -117,7 +117,7 @@ expression 'testdata.`a`' is neither present in the group by, nor is it an aggre -- !query 11 SELECT COUNT(a) FILTER (WHERE a >= 0), COUNT(b) FILTER (WHERE a >= 3) FROM testData GROUP BY a -- !query 11 schema -struct +struct= 0)):bigint,count(b) FILTER (WHERE (a >= 3)):bigint> -- !query 11 output 0 0 2 0 @@ -128,7 +128,7 @@ struct -- !query 12 SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp GROUP BY dept_id -- !query 12 schema -struct +struct DATE '2003-01-01')):double> -- !query 12 output 10 200.0 100 400.0 @@ -141,7 +141,7 @@ NULL NULL -- !query 13 SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_date("2003-01-01")) FROM emp GROUP BY dept_id -- !query 13 schema -struct +struct to_date('2003-01-01'))):double> -- !query 13 output 10 200.0 100 400.0 @@ -154,7 +154,7 @@ NULL NULL -- !query 14 SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_timestamp("2003-01-01 00:00:00")) FROM emp GROUP BY dept_id -- !query 14 schema -struct +struct to_timestamp('2003-01-01 00:00:00'))):double> -- !query 14 output 10 200.0 100 400.0 @@ -167,7 +167,7 @@ NULL NULL -- !query 15 SELECT dept_id, SUM(salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2003-01-01") FROM emp GROUP BY dept_id -- !query 15 schema -struct +struct 2003-01-01)):double> -- !query 15 output 10 200.0 100 400.0 @@ -180,7 +180,7 @@ NULL NULL -- !query 16 SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1 -- !query 16 schema -struct +struct -- !query 16 output foo 6 @@ -188,7 +188,7 @@ foo 6 -- !query 17 SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= date "2003-01-01") FROM emp GROUP BY 1 -- !query 17 schema -struct +struct= DATE '2003-01-01')):double> -- !query 17 output foo 1350.0 @@ -196,7 +196,7 @@ foo 1350.0 -- !query 18 SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_date("2003-01-01")) FROM emp GROUP BY 1 -- !query 18 schema -struct +struct= to_date('2003-01-01'))):double> -- !query 18 output foo 1350.0 @@ -204,7 +204,7 @@ foo 1350.0 -- !query 19 SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_timestamp("2003-01-01")) FROM emp GROUP BY 1 -- !query 19 schema -struct +struct= to_timestamp('2003-01-01'))):double> -- !query 19 output foo 1350.0 @@ -212,7 +212,7 @@ foo 1350.0 -- !query 20 select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id -- !query 20 schema -struct +struct 200)):double> -- !query 20 output 10 2 2 400.0 NULL 100 2 2 800.0 800.0 @@ -225,7 +225,7 @@ NULL 1 1 400.0 400.0 -- !query 21 select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id -- !query 21 schema -struct +struct 500)):double> -- !query 21 output 10 2 2 400.0 NULL 100 2 2 800.0 800.0 @@ -238,7 +238,7 @@ NULL 1 1 400.0 NULL -- !query 22 select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id -- !query 22 schema -struct +struct 200)):double> -- !query 22 output 10 2 2 400.0 NULL 100 2 2 NULL 800.0 @@ -251,7 +251,7 @@ NULL 1 1 NULL 400.0 -- !query 23 select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id -- !query 23 schema -struct +struct 500)):double> -- !query 23 output 10 2 2 400.0 NULL 100 2 2 NULL 800.0 @@ -264,7 +264,7 @@ NULL 1 1 NULL NULL -- !query 24 SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1 -- !query 24 schema -struct +struct= 0)):bigint> -- !query 24 output @@ -272,7 +272,7 @@ struct -- !query 25 SELECT 'foo', MAX(STRUCT(a)) FILTER (WHERE b >= 1) FROM testData WHERE a = 0 GROUP BY 1 -- !query 25 schema -struct> +struct= 1)):struct> -- !query 25 output @@ -280,7 +280,7 @@ struct> -- !query 26 SELECT a + b, COUNT(b) FILTER (WHERE b >= 2) FROM testData GROUP BY a + b -- !query 26 schema -struct<(a + b):int,count(b):bigint> +struct<(a + b):int,count(b) FILTER (WHERE (b >= 2)):bigint> -- !query 26 output 2 0 3 1 @@ -301,7 +301,7 @@ expression 'testdata.`a`' is neither present in the group by, nor is it an aggre -- !query 28 SELECT a + 1 + 1, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY a + 1 -- !query 28 schema -struct<((a + 1) + 1):int,count(b):bigint> +struct<((a + 1) + 1):int,count(b) FILTER (WHERE (b > 0)):bigint> -- !query 28 output 3 2 4 2 @@ -312,7 +312,7 @@ NULL 1 -- !query 29 SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k -- !query 29 schema -struct +struct 0)):bigint> -- !query 29 output 1 2 2 2 @@ -327,7 +327,7 @@ SELECT emp.dept_id, FROM emp GROUP BY dept_id -- !query 30 schema -struct +struct scalarsubquery())):double> -- !query 30 output 10 133.33333333333334 NULL 100 400.0 400.0 @@ -344,7 +344,7 @@ SELECT emp.dept_id, FROM emp GROUP BY dept_id -- !query 31 schema -struct +struct -- !query 31 output 10 133.33333333333334 133.33333333333334 100 400.0 NULL @@ -366,7 +366,7 @@ GROUP BY dept_id struct<> -- !query 32 output org.apache.spark.sql.AnalysisException -IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) AS avg(salary)#x] +IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE exists#x [dept_id#x]) AS avg(salary) FILTER (WHERE exists(dept_id))#x] : +- Project [state#x] : +- Filter (dept_id#x = outer(dept_id#x)) : +- SubqueryAlias `dept` @@ -392,7 +392,7 @@ GROUP BY dept_id struct<> -- !query 33 output org.apache.spark.sql.AnalysisException -IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) AS sum(salary)#x] +IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT exists#x [dept_id#x]) AS sum(salary) FILTER (WHERE (NOT exists(dept_id)))#x] : +- Project [state#x] : +- Filter (dept_id#x = outer(dept_id#x)) : +- SubqueryAlias `dept` @@ -417,7 +417,7 @@ GROUP BY dept_id struct<> -- !query 34 output org.apache.spark.sql.AnalysisException -IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) AS avg(salary)#x] +IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE dept_id#x IN (list#x [])) AS avg(salary) FILTER (WHERE (dept_id IN (listquery())))#x] : +- Distinct : +- Project [dept_id#x] : +- SubqueryAlias `dept` @@ -442,7 +442,7 @@ GROUP BY dept_id struct<> -- !query 35 output org.apache.spark.sql.AnalysisException -IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) AS sum(salary)#x] +IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT dept_id#x IN (list#x [])) AS sum(salary) FILTER (WHERE (NOT (dept_id IN (listquery()))))#x] : +- Distinct : +- Project [dept_id#x] : +- SubqueryAlias `dept` diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out index 9678b2e8966bc..d2ab138efcdae 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out @@ -14,7 +14,7 @@ It is not allowed to use an aggregate function in the argument of another aggreg -- !query 1 select min(unique1) filter (where unique1 > 100) from tenk1 -- !query 1 schema -struct +struct 100)):int> -- !query 1 output 101 @@ -22,7 +22,7 @@ struct -- !query 2 select sum(1/ten) filter (where ten > 0) from tenk1 -- !query 2 schema -struct +struct 0)):double> -- !query 2 output 2828.9682539682954 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 9e629844eb496..7af7b678f8c30 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 @@ -27,6 +27,7 @@ import scala.collection.parallel.immutable.ParVector import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, Partial} import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.HiveResult.hiveResultString @@ -2843,16 +2844,18 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark val query = s"SELECT ${funcToResult._1} FILTER (WHERE b > 1) FROM testData2" val df = sql(query) val physical = df.queryExecution.sparkPlan - val aggregateExpressions = physical.collectFirst { + val aggregateExpressions = physical.collect { case agg: HashAggregateExec => agg.aggregateExpressions case agg: ObjectHashAggregateExec => agg.aggregateExpressions + }.flatten + aggregateExpressions.foreach { expr => + if (expr.mode == Complete || expr.mode == Partial) { + assert(expr.filter.isDefined) + } else { + assert(expr.filter.isEmpty) + } } - assert(aggregateExpressions.isDefined) - assert(aggregateExpressions.get.size == 1) - aggregateExpressions.get.foreach { expr => - assert(expr.filter.isDefined) - } - checkAnswer(df, Row(funcToResult._2) :: Nil) + checkAnswer(df, Row(funcToResult._2)) } } @@ -2860,15 +2863,17 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { val df = sql("SELECT PERCENTILE(a, 1) FILTER (WHERE b > 1) FROM testData2") val physical = df.queryExecution.sparkPlan - val aggregateExpressions = physical.collectFirst { + val aggregateExpressions = physical.collect { case agg: SortAggregateExec => agg.aggregateExpressions + }.flatten + aggregateExpressions.foreach { expr => + if (expr.mode == Complete || expr.mode == Partial) { + assert(expr.filter.isDefined) + } else { + assert(expr.filter.isEmpty) + } } - assert(aggregateExpressions.isDefined) - assert(aggregateExpressions.get.size == 1) - aggregateExpressions.get.foreach { expr => - assert(expr.filter.isDefined) - } - checkAnswer(df, Row(3) :: Nil) + checkAnswer(df, Row(3)) } } From 5a55a5a0d08928300ac92e0e07a9203eba48dadc Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 16 Jan 2020 11:14:43 +0900 Subject: [PATCH 1119/1387] [SPARK-30518][SQL] Precision and scale should be same for values between -1.0 and 1.0 in Decimal ### What changes were proposed in this pull request? For decimal values between -1.0 and 1.0, it should has same precision and scale in `Decimal`, in order to make it be consistent with `DecimalType`. ### Why are the changes needed? Currently, for values between -1.0 and 1.0, precision and scale is inconsistent between `Decimal` and `DecimalType`. For example, for numbers like 0.3, it will have (precision, scale) as (2, 1) in `Decimal`, but (1, 1) in `DecimalType`: ``` scala> Literal(new BigDecimal("0.3")).dataType.asInstanceOf[DecimalType].precision res3: Int = 1 scala> Literal(new BigDecimal("0.3")).value.asInstanceOf[Decimal].precision res4: Int = 2 ``` We should make `Decimal` be consistent with `DecimalType`. And, here, we change it to only count precision digits after dot for values between -1.0 and 1.0 as other DBMS does, like hive: ``` hive> create table testrel as select 0.3; hive> describe testrel; OK _c0 decimal(1,1) ``` This could bring larger scale for values between -1.0 and 1.0. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Updated existed tests. Closes #27217 from Ngone51/set-decimal-from-javadecimal. Authored-by: yi.wu Signed-off-by: Takeshi Yamamuro --- .../scala/org/apache/spark/sql/types/Decimal.scala | 8 +++++--- .../expressions/ArithmeticExpressionSuite.scala | 8 ++++---- .../org/apache/spark/sql/types/DecimalSuite.scala | 14 +++++++------- 3 files changed, 16 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index f9ccebc4f8a04..533cddb556a9b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -31,6 +31,8 @@ import org.apache.spark.annotation.Unstable * - _precision and _scale represent the SQL precision and scale we are looking for * - If decimalVal is set, it represents the whole decimal value * - Otherwise, the decimal value is longVal / (10 ** _scale) + * + * Note, for values between -1.0 and 1.0, precision digits are only counted after dot. */ @Unstable final class Decimal extends Ordered[Decimal] with Serializable { @@ -128,12 +130,12 @@ final class Decimal extends Ordered[Decimal] with Serializable { def set(decimal: BigDecimal): Decimal = { this.decimalVal = decimal this.longVal = 0L - if (decimal.precision <= decimal.scale) { + if (decimal.precision < decimal.scale) { // For Decimal, we expect the precision is equal to or large than the scale, however, // in BigDecimal, the digit count starts from the leftmost nonzero digit of the exact // result. For example, the precision of 0.01 equals to 1 based on the definition, but - // the scale is 2. The expected precision should be 3. - this._precision = decimal.scale + 1 + // the scale is 2. The expected precision should be 2. + this._precision = decimal.scale } else { this._precision = decimal.precision } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index ad8b1a1673679..b4a1ae20c3e11 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -409,9 +409,9 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper test("SPARK-28322: IntegralDivide supports decimal type") { withSQLConf(SQLConf.LEGACY_INTEGRALDIVIDE_RETURN_LONG.key -> "false") { checkEvaluation(IntegralDivide(Literal(Decimal(1)), Literal(Decimal(2))), Decimal(0)) - checkEvaluation(IntegralDivide(Literal(Decimal(1.4)), Literal(Decimal(0.6))), Decimal(2)) + checkEvaluation(IntegralDivide(Literal(Decimal(2.4)), Literal(Decimal(1.1))), Decimal(2)) checkEvaluation(IntegralDivide(Literal(Decimal(1.2)), Literal(Decimal(1.1))), Decimal(1)) - checkEvaluation(IntegralDivide(Literal(Decimal(1.2)), Literal(Decimal(0.0))), null) + checkEvaluation(IntegralDivide(Literal(Decimal(0.2)), Literal(Decimal(0.0))), null) checkEvaluation(DecimalPrecision.decimalAndDecimal.apply(IntegralDivide( Literal(Decimal("99999999999999999999999999999999999")), Literal(Decimal(0.001)))), BigDecimal("99999999999999999999999999999999999000")) @@ -422,9 +422,9 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper } withSQLConf(SQLConf.LEGACY_INTEGRALDIVIDE_RETURN_LONG.key -> "true") { checkEvaluation(IntegralDivide(Literal(Decimal(1)), Literal(Decimal(2))), 0L) - checkEvaluation(IntegralDivide(Literal(Decimal(1.4)), Literal(Decimal(0.6))), 2L) + checkEvaluation(IntegralDivide(Literal(Decimal(2.4)), Literal(Decimal(1.1))), 2L) checkEvaluation(IntegralDivide(Literal(Decimal(1.2)), Literal(Decimal(1.1))), 1L) - checkEvaluation(IntegralDivide(Literal(Decimal(1.2)), Literal(Decimal(0.0))), null) + checkEvaluation(IntegralDivide(Literal(Decimal(0.2)), Literal(Decimal(0.0))), null) // overflows long and so returns a wrong result checkEvaluation(DecimalPrecision.decimalAndDecimal.apply(IntegralDivide( Literal(Decimal("99999999999999999999999999999999999")), Literal(Decimal(0.001)))), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index bf41b711c9473..0c01d32421274 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -32,15 +32,15 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { test("creating decimals") { checkDecimal(new Decimal(), "0", 1, 0) - checkDecimal(Decimal(BigDecimal("0.09")), "0.09", 3, 2) - checkDecimal(Decimal(BigDecimal("0.9")), "0.9", 2, 1) - checkDecimal(Decimal(BigDecimal("0.90")), "0.90", 3, 2) - checkDecimal(Decimal(BigDecimal("0.0")), "0.0", 2, 1) + checkDecimal(Decimal(BigDecimal("0.09")), "0.09", 2, 2) + checkDecimal(Decimal(BigDecimal("0.9")), "0.9", 1, 1) + checkDecimal(Decimal(BigDecimal("0.90")), "0.90", 2, 2) + checkDecimal(Decimal(BigDecimal("0.0")), "0.0", 1, 1) checkDecimal(Decimal(BigDecimal("0")), "0", 1, 0) checkDecimal(Decimal(BigDecimal("1.0")), "1.0", 2, 1) - checkDecimal(Decimal(BigDecimal("-0.09")), "-0.09", 3, 2) - checkDecimal(Decimal(BigDecimal("-0.9")), "-0.9", 2, 1) - checkDecimal(Decimal(BigDecimal("-0.90")), "-0.90", 3, 2) + checkDecimal(Decimal(BigDecimal("-0.09")), "-0.09", 2, 2) + checkDecimal(Decimal(BigDecimal("-0.9")), "-0.9", 1, 1) + checkDecimal(Decimal(BigDecimal("-0.90")), "-0.90", 2, 2) checkDecimal(Decimal(BigDecimal("-1.0")), "-1.0", 2, 1) checkDecimal(Decimal(BigDecimal("10.030")), "10.030", 5, 3) checkDecimal(Decimal(BigDecimal("10.030"), 4, 1), "10.0", 4, 1) From aec55cd1cae98d06a86f300e694de9c0cfe9b234 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 16 Jan 2020 11:01:30 +0800 Subject: [PATCH 1120/1387] [SPARK-30502][ML][CORE] PeriodicRDDCheckpointer support storageLevel ### What changes were proposed in this pull request? 1, add field `storageLevel` in `PeriodicRDDCheckpointer` 2, for ml.GBT/ml.RF set storageLevel=`StorageLevel.MEMORY_AND_DISK` ### Why are the changes needed? Intermediate RDDs in ML are cached with storageLevel=StorageLevel.MEMORY_AND_DISK. PeriodicRDDCheckpointer & PeriodicGraphCheckpointer now store RDD with storageLevel=StorageLevel.MEMORY_ONLY, it maybe nice to set the storageLevel of checkpointer. ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing testsuites Closes #27189 from zhengruifeng/checkpointer_storage. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../apache/spark/rdd/util/PeriodicRDDCheckpointer.scala | 9 +++++++-- .../apache/spark/ml/tree/impl/GradientBoostedTrees.scala | 4 ++-- .../org/apache/spark/ml/tree/impl/RandomForest.scala | 2 +- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala b/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala index 4a6106984a495..e460542f0319e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala +++ b/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala @@ -76,8 +76,13 @@ import org.apache.spark.util.PeriodicCheckpointer */ private[spark] class PeriodicRDDCheckpointer[T]( checkpointInterval: Int, - sc: SparkContext) + sc: SparkContext, + storageLevel: StorageLevel) extends PeriodicCheckpointer[RDD[T]](checkpointInterval, sc) { + require(storageLevel != StorageLevel.NONE) + + def this(checkpointInterval: Int, sc: SparkContext) = + this(checkpointInterval, sc, StorageLevel.MEMORY_ONLY) override protected def checkpoint(data: RDD[T]): Unit = data.checkpoint() @@ -85,7 +90,7 @@ private[spark] class PeriodicRDDCheckpointer[T]( override protected def persist(data: RDD[T]): Unit = { if (data.getStorageLevel == StorageLevel.NONE) { - data.persist() + data.persist(storageLevel) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index 18068e9483f2b..51aad3f08c96a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -316,7 +316,7 @@ private[spark] object GradientBoostedTrees extends Logging { // Prepare periodic checkpointers // Note: this is checkpointing the unweighted training error val predErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)]( - treeStrategy.getCheckpointInterval, sc) + treeStrategy.getCheckpointInterval, sc, StorageLevel.MEMORY_AND_DISK) timer.stop("init") @@ -392,7 +392,7 @@ private[spark] object GradientBoostedTrees extends Logging { validatePredError = computeInitialPredictionAndError( validationTreePoints, firstTreeWeight, firstTreeModel, loss, bcSplits) validatePredErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)]( - treeStrategy.getCheckpointInterval, sc) + treeStrategy.getCheckpointInterval, sc, StorageLevel.MEMORY_AND_DISK) validatePredErrorCheckpointer.update(validatePredError) bestValidateError = computeWeightedError(validationTreePoints, validatePredError) timer.stop("init validation") diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index a91a7b8f8e748..b3848c1695a99 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -163,7 +163,7 @@ private[spark] object RandomForest extends Logging with Serializable { // At first, all the rows belong to the root nodes (node Id == 1). nodeIds = baggedInput.map { _ => Array.fill(numTrees)(1) } nodeIdCheckpointer = new PeriodicRDDCheckpointer[Array[Int]]( - strategy.getCheckpointInterval, sc) + strategy.getCheckpointInterval, sc, StorageLevel.MEMORY_AND_DISK) nodeIdCheckpointer.update(nodeIds) } From 0a95eb08003a115f59495b30aacaaa832940e977 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 16 Jan 2020 12:39:44 +0900 Subject: [PATCH 1121/1387] [SPARK-30434][FOLLOW-UP][PYTHON][SQL] Make the parameter list consistent in createDataFrame ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/27109. It should match the parameter lists in `createDataFrame`. ### Why are the changes needed? To pass parameters supposed to pass. ### Does this PR introduce any user-facing change? No (it's only in master) ### How was this patch tested? Manually tested and existing tests should cover. Closes #27225 from HyukjinKwon/SPARK-30434-followup. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/pandas/conversion.py | 2 +- python/pyspark/sql/session.py | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/pandas/conversion.py b/python/pyspark/sql/pandas/conversion.py index 1c957a1665a7e..42374100da28b 100644 --- a/python/pyspark/sql/pandas/conversion.py +++ b/python/pyspark/sql/pandas/conversion.py @@ -267,7 +267,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr warnings.warn(msg) raise data = self._convert_from_pandas(data, schema, timezone) - return self._create_dataframe(data, schema, samplingRatio, samplingRatio) + return self._create_dataframe(data, schema, samplingRatio, verifySchema) def _convert_from_pandas(self, pdf, schema, timezone): """ diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index bf858bcf310fa..233f4927389d2 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -598,10 +598,10 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr if has_pandas and isinstance(data, pandas.DataFrame): # Create a DataFrame from pandas DataFrame. return super(SparkSession, self).createDataFrame( - data, schema, verifySchema, samplingRatio) - return self._create_dataframe(data, schema, verifySchema, samplingRatio) + data, schema, samplingRatio, verifySchema) + return self._create_dataframe(data, schema, samplingRatio, verifySchema) - def _create_dataframe(self, data, schema, verifySchema, samplingRatio): + def _create_dataframe(self, data, schema, samplingRatio, verifySchema): if isinstance(schema, StructType): verify_func = _make_type_verifier(schema) if verifySchema else lambda _: True From be4d825872b41e04e190066e550217362b82061e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Jan 2020 20:09:12 -0800 Subject: [PATCH 1122/1387] [SPARK-30312][SQL][FOLLOWUP] Rename conf by adding `.enabled` ### What changes were proposed in this pull request? Based on the [comment](https://github.com/apache/spark/pull/26956#discussion_r366680558), this patch changes the SQL config name from `spark.sql.truncateTable.ignorePermissionAcl` to `spark.sql.truncateTable.ignorePermissionAcl.enabled`. ### Why are the changes needed? Make this config consistent other SQL configs. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test. Closes #27210 from viirya/truncate-table-permission-followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6d45d30a787dc..53136ac244b27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2006,7 +2006,7 @@ object SQLConf { .createWithDefault(false) val TRUNCATE_TABLE_IGNORE_PERMISSION_ACL = - buildConf("spark.sql.truncateTable.ignorePermissionAcl") + buildConf("spark.sql.truncateTable.ignorePermissionAcl.enabled") .internal() .doc("When set to true, TRUNCATE TABLE command will not try to set back original " + "permission and ACLs when re-creating the table/partition paths.") From 4e50f0291f032b4a5c0b46ed01fdef14e4cbb050 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 16 Jan 2020 13:10:08 +0900 Subject: [PATCH 1123/1387] [SPARK-30323][SQL] Support filters pushdown in CSV datasource ### What changes were proposed in this pull request? In the PR, I propose to support pushed down filters in CSV datasource. The reason of pushing a filter up to `UnivocityParser` is to apply the filter as soon as all its attributes become available i.e. converted from CSV fields to desired values according to the schema. This allows to skip conversions of other values if the filter returns `false`. This can improve performance when pushed filters are highly selective and conversion of CSV string fields to desired values are comparably expensive ( for example, conversion to `TIMESTAMP` values). Here are details of the implementation: - `UnivocityParser.convert()` converts parsed CSV tokens one-by-one sequentially starting from index 0 up to `parsedSchema.length - 1`. At current index `i`, it applies filters that refer to attributes at row fields indexes `0..i`. If any filter returns `false`, it skips conversions of other input tokens. - Pushed filters are converted to expressions. The expressions are bound to row positions according to `requiredSchema`. The expressions are compiled to predicates via generating Java code. - To be able to apply predicates to partially initialized rows, the predicates are grouped, and combined via the `And` expression. Final predicate at index `N` can refer to row fields at the positions `0..N`, and can be applied to a row even if other fields at the positions `N+1..requiredSchema.lenght-1` are not set. ### Why are the changes needed? The changes improve performance on synthetic benchmarks more **than 9 times** (on JDK 8 & 11): ``` OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 Intel(R) Core(TM) i7-4850HQ CPU 2.30GHz Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ w/o filters 11889 11945 52 0.0 118893.1 1.0X pushdown disabled 11790 11860 115 0.0 117902.3 1.0X w/ filters 1240 1278 33 0.1 12400.8 9.6X ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? - Added new test suite `CSVFiltersSuite` - Added tests to `CSVSuite` and `UnivocityParserSuite` Closes #26973 from MaxGekk/csv-filters-pushdown. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../spark/sql/catalyst/csv/CSVFilters.scala | 191 ++++++++++++++++++ .../sql/catalyst/csv/UnivocityParser.scala | 58 ++++-- .../catalyst/expressions/csvExpressions.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 7 + .../sql/catalyst/csv/CSVFiltersSuite.scala | 134 ++++++++++++ .../catalyst/csv/UnivocityParserSuite.scala | 50 +++++ .../benchmarks/CSVBenchmark-jdk11-results.txt | 8 + sql/core/benchmarks/CSVBenchmark-results.txt | 8 + .../apache/spark/sql/DataFrameReader.scala | 2 +- .../datasources/csv/CSVFileFormat.scala | 11 +- .../v2/csv/CSVPartitionReaderFactory.scala | 7 +- .../datasources/v2/csv/CSVScan.scala | 12 +- .../datasources/v2/csv/CSVScanBuilder.scala | 26 ++- .../datasources/csv/CSVBenchmark.scala | 46 +++++ .../execution/datasources/csv/CSVSuite.scala | 77 ++++++- 15 files changed, 608 insertions(+), 31 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala new file mode 100644 index 0000000000000..b50a76a496556 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -0,0 +1,191 @@ +/* + * 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.csv + +import scala.util.Try + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.{BooleanType, StructType} + +/** + * An instance of the class compiles filters to predicates and allows to + * apply the predicates to an internal row with partially initialized values + * converted from parsed CSV fields. + * + * @param filters The filters pushed down to CSV datasource. + * @param requiredSchema The schema with only fields requested by the upper layer. + */ +class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) { + /** + * Converted filters to predicates and grouped by maximum field index + * in the read schema. For example, if an filter refers to 2 attributes + * attrA with field index 5 and attrB with field index 10 in the read schema: + * 0 === $"attrA" or $"attrB" < 100 + * the filter is compiled to a predicate, and placed to the `predicates` + * array at the position 10. In this way, if there is a row with initialized + * fields from the 0 to 10 index, the predicate can be applied to the row + * to check that the row should be skipped or not. + * Multiple predicates with the same maximum reference index are combined + * by the `And` expression. + */ + private val predicates: Array[BasePredicate] = { + val len = requiredSchema.fields.length + val groupedPredicates = Array.fill[BasePredicate](len)(null) + if (SQLConf.get.csvFilterPushDown) { + val groupedFilters = Array.fill(len)(Seq.empty[sources.Filter]) + for (filter <- filters) { + val refs = filter.references + val index = if (refs.isEmpty) { + // For example, AlwaysTrue and AlwaysFalse doesn't have any references + // Filters w/o refs always return the same result. Taking into account + // that predicates are combined via And, we can apply such filters only + // once at the position 0. + 0 + } else { + // readSchema must contain attributes of all filters. + // Accordingly, fieldIndex() returns a valid index always. + refs.map(requiredSchema.fieldIndex).max + } + groupedFilters(index) :+= filter + } + if (len > 0 && !groupedFilters(0).isEmpty) { + // We assume that filters w/o refs like AlwaysTrue and AlwaysFalse + // can be evaluated faster that others. We put them in front of others. + val (literals, others) = groupedFilters(0).partition(_.references.isEmpty) + groupedFilters(0) = literals ++ others + } + for (i <- 0 until len) { + if (!groupedFilters(i).isEmpty) { + val reducedExpr = groupedFilters(i) + .flatMap(CSVFilters.filterToExpression(_, toRef)) + .reduce(And) + groupedPredicates(i) = Predicate.create(reducedExpr) + } + } + } + groupedPredicates + } + + /** + * Applies all filters that refer to row fields at the positions from 0 to index. + * @param row The internal row to check. + * @param index Maximum field index. The function assumes that all fields + * from 0 to index position are set. + * @return false iff row fields at the position from 0 to index pass filters + * or there are no applicable filters + * otherwise false if at least one of the filters returns false. + */ + def skipRow(row: InternalRow, index: Int): Boolean = { + val predicate = predicates(index) + predicate != null && !predicate.eval(row) + } + + // Finds a filter attribute in the read schema and converts it to a `BoundReference` + private def toRef(attr: String): Option[BoundReference] = { + requiredSchema.getFieldIndex(attr).map { index => + val field = requiredSchema(index) + BoundReference(requiredSchema.fieldIndex(attr), field.dataType, field.nullable) + } + } +} + +object CSVFilters { + private def checkFilterRefs(filter: sources.Filter, schema: StructType): Boolean = { + val fieldNames = schema.fields.map(_.name).toSet + filter.references.forall(fieldNames.contains(_)) + } + + /** + * Returns the filters currently supported by CSV datasource. + * @param filters The filters pushed down to CSV datasource. + * @param schema data schema of CSV files. + * @return a sub-set of `filters` that can be handled by CSV datasource. + */ + def pushedFilters(filters: Array[sources.Filter], schema: StructType): Array[sources.Filter] = { + filters.filter(checkFilterRefs(_, schema)) + } + + private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = { + a.zip(b).headOption + } + + private def toLiteral(value: Any): Option[Literal] = { + Try(Literal(value)).toOption + } + + /** + * Converts a filter to an expression and binds it to row positions. + * + * @param filter The filter to convert. + * @param toRef The function converts a filter attribute to a bound reference. + * @return some expression with resolved attributes or None if the conversion + * of the given filter to an expression is impossible. + */ + def filterToExpression( + filter: sources.Filter, + toRef: String => Option[BoundReference]): Option[Expression] = { + def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { + zip(toRef(name), toLiteral(value)) + } + def translate(filter: sources.Filter): Option[Expression] = filter match { + case sources.And(left, right) => + zip(translate(left), translate(right)).map(And.tupled) + case sources.Or(left, right) => + zip(translate(left), translate(right)).map(Or.tupled) + case sources.Not(child) => + translate(child).map(Not) + case sources.EqualTo(attribute, value) => + zipAttributeAndValue(attribute, value).map(EqualTo.tupled) + case sources.EqualNullSafe(attribute, value) => + zipAttributeAndValue(attribute, value).map(EqualNullSafe.tupled) + case sources.IsNull(attribute) => + toRef(attribute).map(IsNull) + case sources.IsNotNull(attribute) => + toRef(attribute).map(IsNotNull) + case sources.In(attribute, values) => + val literals = values.toSeq.flatMap(toLiteral) + if (literals.length == values.length) { + toRef(attribute).map(In(_, literals)) + } else { + None + } + case sources.GreaterThan(attribute, value) => + zipAttributeAndValue(attribute, value).map(GreaterThan.tupled) + case sources.GreaterThanOrEqual(attribute, value) => + zipAttributeAndValue(attribute, value).map(GreaterThanOrEqual.tupled) + case sources.LessThan(attribute, value) => + zipAttributeAndValue(attribute, value).map(LessThan.tupled) + case sources.LessThanOrEqual(attribute, value) => + zipAttributeAndValue(attribute, value).map(LessThanOrEqual.tupled) + case sources.StringContains(attribute, value) => + zipAttributeAndValue(attribute, value).map(Contains.tupled) + case sources.StringStartsWith(attribute, value) => + zipAttributeAndValue(attribute, value).map(StartsWith.tupled) + case sources.StringEndsWith(attribute, value) => + zipAttributeAndValue(attribute, value).map(EndsWith.tupled) + case sources.AlwaysTrue() => + Some(Literal(true, BooleanType)) + case sources.AlwaysFalse() => + Some(Literal(false, BooleanType)) + } + translate(filter) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 661525a65294d..288179fc480da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -27,6 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -39,15 +40,20 @@ import org.apache.spark.unsafe.types.UTF8String * @param requiredSchema The schema of the data that should be output for each row. This should be a * subset of the columns in dataSchema. * @param options Configuration options for a CSV parser. + * @param filters The pushdown filters that should be applied to converted values. */ class UnivocityParser( dataSchema: StructType, requiredSchema: StructType, - val options: CSVOptions) extends Logging { + val options: CSVOptions, + filters: Seq[Filter]) extends Logging { require(requiredSchema.toSet.subsetOf(dataSchema.toSet), s"requiredSchema (${requiredSchema.catalogString}) should be the subset of " + s"dataSchema (${dataSchema.catalogString}).") + def this(dataSchema: StructType, requiredSchema: StructType, options: CSVOptions) = { + this(dataSchema, requiredSchema, options, Seq.empty) + } def this(schema: StructType, options: CSVOptions) = this(schema, schema, options) // A `ValueConverter` is responsible for converting the given value to a desired type. @@ -72,7 +78,11 @@ class UnivocityParser( new CsvParser(parserSetting) } - private val row = new GenericInternalRow(requiredSchema.length) + // Pre-allocated Seq to avoid the overhead of the seq builder. + private val requiredRow = Seq(new GenericInternalRow(requiredSchema.length)) + // Pre-allocated empty sequence returned when the parsed row cannot pass filters. + // We preallocate it avoid unnecessary invokes of the seq builder. + private val noRows = Seq.empty[InternalRow] private val timestampFormatter = TimestampFormatter( options.timestampFormat, @@ -83,6 +93,8 @@ class UnivocityParser( options.zoneId, options.locale) + private val csvFilters = new CSVFilters(filters, requiredSchema) + // Retrieve the raw record string. private def getCurrentInput: UTF8String = { UTF8String.fromString(tokenizer.getContext.currentParsedContent().stripLineEnd) @@ -194,7 +206,7 @@ class UnivocityParser( private val doParse = if (options.columnPruning && requiredSchema.isEmpty) { // If `columnPruning` enabled and partition attributes scanned only, // `schema` gets empty. - (_: String) => InternalRow.empty + (_: String) => Seq(InternalRow.empty) } else { // parse if the columnPruning is disabled or requiredSchema is nonEmpty (input: String) => convert(tokenizer.parseLine(input)) @@ -204,7 +216,7 @@ class UnivocityParser( * Parses a single CSV string and turns it into either one resulting row or no row (if the * the record is malformed). */ - def parse(input: String): InternalRow = doParse(input) + def parse(input: String): Seq[InternalRow] = doParse(input) private val getToken = if (options.columnPruning) { (tokens: Array[String], index: Int) => tokens(index) @@ -212,7 +224,7 @@ class UnivocityParser( (tokens: Array[String], index: Int) => tokens(tokenIndexArr(index)) } - private def convert(tokens: Array[String]): InternalRow = { + private def convert(tokens: Array[String]): Seq[InternalRow] = { if (tokens == null) { throw BadRecordException( () => getCurrentInput, @@ -229,7 +241,7 @@ class UnivocityParser( } def getPartialResult(): Option[InternalRow] = { try { - Some(convert(checkedTokens)) + convert(checkedTokens).headOption } catch { case _: BadRecordException => None } @@ -242,12 +254,24 @@ class UnivocityParser( new RuntimeException("Malformed CSV record")) } else { // When the length of the returned tokens is identical to the length of the parsed schema, - // we just need to convert the tokens that correspond to the required columns. - var badRecordException: Option[Throwable] = None + // we just need to: + // 1. Convert the tokens that correspond to the required schema. + // 2. Apply the pushdown filters to `requiredRow`. var i = 0 + val row = requiredRow.head + var skipRow = false + var badRecordException: Option[Throwable] = None while (i < requiredSchema.length) { try { - row(i) = valueConverters(i).apply(getToken(tokens, i)) + if (!skipRow) { + row(i) = valueConverters(i).apply(getToken(tokens, i)) + if (csvFilters.skipRow(row, i)) { + skipRow = true + } + } + if (skipRow) { + row.setNullAt(i) + } } catch { case NonFatal(e) => badRecordException = badRecordException.orElse(Some(e)) @@ -255,11 +279,15 @@ class UnivocityParser( } i += 1 } - - if (badRecordException.isEmpty) { - row + if (skipRow) { + noRows } else { - throw BadRecordException(() => getCurrentInput, () => Some(row), badRecordException.get) + if (badRecordException.isDefined) { + throw BadRecordException( + () => getCurrentInput, () => requiredRow.headOption, badRecordException.get) + } else { + requiredRow + } } } } @@ -291,7 +319,7 @@ private[sql] object UnivocityParser { schema: StructType): Iterator[InternalRow] = { val tokenizer = parser.tokenizer val safeParser = new FailureSafeParser[Array[String]]( - input => Seq(parser.convert(input)), + input => parser.convert(input), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) @@ -344,7 +372,7 @@ private[sql] object UnivocityParser { val filteredLines: Iterator[String] = CSVExprUtils.filterCommentAndEmpty(lines, options) val safeParser = new FailureSafeParser[String]( - input => Seq(parser.parse(input)), + input => parser.parse(input), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala index 73d329b4f5829..54af314fe4171 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -114,7 +114,7 @@ case class CsvToStructs( StructType(nullableSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) val rawParser = new UnivocityParser(actualSchema, actualSchema, parsedOptions) new FailureSafeParser[String]( - input => Seq(rawParser.parse(input)), + input => rawParser.parse(input), mode, nullableSchema, parsedOptions.columnNameOfCorruptRecord) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 53136ac244b27..d63f4daf00e26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2153,6 +2153,11 @@ object SQLConf { .booleanConf .createWithDefault(false) + val CSV_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.csv.filterPushdown.enabled") + .doc("When true, enable filter pushdown to CSV datasource.") + .booleanConf + .createWithDefault(true) + /** * Holds information about keys that have been deprecated. * @@ -2722,6 +2727,8 @@ class SQLConf extends Serializable with Logging { def ignoreDataLocality: Boolean = getConf(SQLConf.IGNORE_DATA_LOCALITY) + def csvFilterPushDown: Boolean = getConf(CSV_FILTER_PUSHDOWN_ENABLED) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala new file mode 100644 index 0000000000000..499bbaf452aee --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.csv + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.sources +import org.apache.spark.sql.sources.{AlwaysFalse, AlwaysTrue, Filter} +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.unsafe.types.UTF8String + +class CSVFiltersSuite extends SparkFunSuite { + test("filter to expression conversion") { + val ref = BoundReference(0, IntegerType, true) + def check(f: Filter, expr: Expression): Unit = { + assert(CSVFilters.filterToExpression(f, _ => Some(ref)).get === expr) + } + + check(sources.AlwaysTrue, Literal(true)) + check(sources.AlwaysFalse, Literal(false)) + check(sources.IsNull("a"), IsNull(ref)) + check(sources.Not(sources.IsNull("a")), Not(IsNull(ref))) + check(sources.IsNotNull("a"), IsNotNull(ref)) + check(sources.EqualTo("a", "b"), EqualTo(ref, Literal("b"))) + check(sources.EqualNullSafe("a", "b"), EqualNullSafe(ref, Literal("b"))) + check(sources.StringStartsWith("a", "b"), StartsWith(ref, Literal("b"))) + check(sources.StringEndsWith("a", "b"), EndsWith(ref, Literal("b"))) + check(sources.StringContains("a", "b"), Contains(ref, Literal("b"))) + check(sources.LessThanOrEqual("a", 1), LessThanOrEqual(ref, Literal(1))) + check(sources.LessThan("a", 1), LessThan(ref, Literal(1))) + check(sources.GreaterThanOrEqual("a", 1), GreaterThanOrEqual(ref, Literal(1))) + check(sources.GreaterThan("a", 1), GreaterThan(ref, Literal(1))) + check(sources.And(sources.AlwaysTrue, sources.AlwaysTrue), And(Literal(true), Literal(true))) + check(sources.Or(sources.AlwaysTrue, sources.AlwaysTrue), Or(Literal(true), Literal(true))) + check(sources.In("a", Array(1)), In(ref, Seq(Literal(1)))) + } + + private def getSchema(str: String): StructType = str match { + case "" => new StructType() + case _ => StructType.fromDDL(str) + } + + test("skipping rows") { + def check( + requiredSchema: String = "i INTEGER, d DOUBLE", + filters: Seq[Filter], + row: InternalRow, + pos: Int, + skip: Boolean): Unit = { + val csvFilters = new CSVFilters(filters, getSchema(requiredSchema)) + assert(csvFilters.skipRow(row, pos) === skip) + } + + check(filters = Seq(), row = InternalRow(3.14), pos = 0, skip = false) + check(filters = Seq(AlwaysTrue), row = InternalRow(1), pos = 0, skip = false) + check(filters = Seq(AlwaysFalse), row = InternalRow(1), pos = 0, skip = true) + check( + filters = Seq(sources.EqualTo("i", 1), sources.LessThan("d", 10), sources.AlwaysFalse), + row = InternalRow(1, 3.14), + pos = 0, + skip = true) + check( + filters = Seq(sources.EqualTo("i", 10)), + row = InternalRow(10, 3.14), + pos = 0, + skip = false) + check( + filters = Seq(sources.IsNotNull("d"), sources.GreaterThanOrEqual("d", 2.96)), + row = InternalRow(3.14), + pos = 0, + skip = false) + check( + filters = Seq(sources.In("i", Array(10, 20)), sources.LessThanOrEqual("d", 2.96)), + row = InternalRow(10, 3.14), + pos = 1, + skip = true) + val filters1 = Seq( + sources.Or( + sources.AlwaysTrue, + sources.And( + sources.Not(sources.IsNull("i")), + sources.Not( + sources.And( + sources.StringEndsWith("s", "ab"), + sources.StringEndsWith("s", "cd") + ) + ) + ) + ), + sources.GreaterThan("d", 0), + sources.LessThan("i", 500) + ) + val filters2 = Seq( + sources.And( + sources.StringContains("s", "abc"), + sources.And( + sources.Not(sources.IsNull("i")), + sources.And( + sources.StringEndsWith("s", "ab"), + sources.StringEndsWith("s", "bc") + ) + ) + ), + sources.GreaterThan("d", 100), + sources.LessThan("i", 0) + ) + Seq(filters1 -> false, filters2 -> true).foreach { case (filters, skip) => + for (p <- 0 until 3) { + check( + requiredSchema = "i INTEGER, d DOUBLE, s STRING", + filters = filters, + row = InternalRow(10, 3.14, UTF8String.fromString("abc")), + pos = p, + skip = skip) + } + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index 31601f787f1a9..bd4b2529f8b92 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -24,9 +24,11 @@ import java.util.{Locale, TimeZone} import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.sources.{EqualTo, Filter, StringStartsWith} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -267,4 +269,52 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { assert(convertedValue.isInstanceOf[UTF8String]) assert(convertedValue == expected) } + + test("skipping rows using pushdown filters") { + def check( + input: String = "1,a", + dataSchema: StructType = StructType.fromDDL("i INTEGER, s STRING"), + requiredSchema: StructType = StructType.fromDDL("i INTEGER"), + filters: Seq[Filter], + expected: Seq[InternalRow]): Unit = { + Seq(false, true).foreach { columnPruning => + val options = new CSVOptions(Map.empty[String, String], columnPruning, "GMT") + val parser = new UnivocityParser(dataSchema, requiredSchema, options, filters) + val actual = parser.parse(input) + assert(actual === expected) + } + } + + check(filters = Seq(), expected = Seq(InternalRow(1))) + check(filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow(1))) + check(filters = Seq(EqualTo("i", 2)), expected = Seq()) + check( + requiredSchema = StructType.fromDDL("s STRING"), + filters = Seq(StringStartsWith("s", "b")), + expected = Seq()) + check( + requiredSchema = StructType.fromDDL("i INTEGER, s STRING"), + filters = Seq(StringStartsWith("s", "a")), + expected = Seq(InternalRow(1, UTF8String.fromString("a")))) + check( + input = "1,a,3.14", + dataSchema = StructType.fromDDL("i INTEGER, s STRING, d DOUBLE"), + requiredSchema = StructType.fromDDL("i INTEGER, d DOUBLE"), + filters = Seq(EqualTo("d", 3.14)), + expected = Seq(InternalRow(1, 3.14))) + + val errMsg = intercept[IllegalArgumentException] { + check(filters = Seq(EqualTo("invalid attr", 1)), expected = Seq()) + }.getMessage + assert(errMsg.contains("invalid attr does not exist")) + + val errMsg2 = intercept[IllegalArgumentException] { + check( + dataSchema = new StructType(), + requiredSchema = new StructType(), + filters = Seq(EqualTo("i", 1)), + expected = Seq(InternalRow.empty)) + }.getMessage + assert(errMsg2.contains("i does not exist")) + } } diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index 2d24a273f7575..d8071e7bbdb35 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -56,4 +56,12 @@ parse dates from Dataset[String] 51026 51447 5 from_csv(timestamp) 60738 61818 936 0.2 6073.8 0.0X from_csv(date) 46012 46278 370 0.2 4601.2 0.1X +OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 11889 11945 52 0.0 118893.1 1.0X +pushdown disabled 11790 11860 115 0.0 117902.3 1.0X +w/ filters 1240 1278 33 0.1 12400.8 9.6X + diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index 0777549efc5f5..b3ba69c9eb6b1 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -56,4 +56,12 @@ parse dates from Dataset[String] 48728 49071 3 from_csv(timestamp) 62294 62493 260 0.2 6229.4 0.0X from_csv(date) 44581 44665 117 0.2 4458.1 0.1X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 12557 12634 78 0.0 125572.9 1.0X +pushdown disabled 12449 12509 65 0.0 124486.4 1.0X +w/ filters 1372 1393 18 0.1 13724.8 9.1X + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index a8b352407be8c..1af4931c553ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -557,7 +557,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val parsed = linesWithoutHeader.mapPartitions { iter => val rawParser = new UnivocityParser(actualSchema, parsedOptions) val parser = new FailureSafeParser[String]( - input => Seq(rawParser.parse(input)), + input => rawParser.parse(input), parsedOptions.parseMode, schema, parsedOptions.columnNameOfCorruptRecord) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 8abc6fcacd4c5..cbf9d2bac7ceb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -17,16 +17,13 @@ package org.apache.spark.sql.execution.datasources.csv -import java.nio.charset.Charset - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce._ -import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityGenerator, UnivocityParser} +import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.util.CompressionCodecs import org.apache.spark.sql.execution.datasources._ @@ -134,7 +131,11 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) val actualRequiredSchema = StructType( requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) - val parser = new UnivocityParser(actualDataSchema, actualRequiredSchema, parsedOptions) + val parser = new UnivocityParser( + actualDataSchema, + actualRequiredSchema, + parsedOptions, + filters) val schema = if (columnPruning) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index a20b0f1560a1d..31d31bd43f453 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -43,7 +44,8 @@ case class CSVPartitionReaderFactory( dataSchema: StructType, readDataSchema: StructType, partitionSchema: StructType, - parsedOptions: CSVOptions) extends FilePartitionReaderFactory { + parsedOptions: CSVOptions, + filters: Seq[Filter]) extends FilePartitionReaderFactory { private val columnPruning = sqlConf.csvColumnPruning override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { @@ -55,7 +57,8 @@ case class CSVPartitionReaderFactory( val parser = new UnivocityParser( actualDataSchema, actualReadDataSchema, - parsedOptions) + parsedOptions, + filters) val schema = if (columnPruning) actualReadDataSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index 78b04aa811e09..690d66908e613 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2.{FileScan, TextBasedFileScan} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -38,6 +39,7 @@ case class CSVScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, + pushedFilters: Array[Filter], partitionFilters: Seq[Expression] = Seq.empty) extends TextBasedFileScan(sparkSession, options) { @@ -86,17 +88,21 @@ case class CSVScan( // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. CSVPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, parsedOptions) + dataSchema, readDataSchema, readPartitionSchema, parsedOptions, pushedFilters) } override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters) override def equals(obj: Any): Boolean = obj match { - case c: CSVScan => super.equals(c) && dataSchema == c.dataSchema && options == c.options - + case c: CSVScan => super.equals(c) && dataSchema == c.dataSchema && options == c.options && + equivalentFilters(pushedFilters, c.pushedFilters) case _ => false } override def hashCode(): Int = super.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala index 8b486d0344506..81a234e254000 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.catalyst.csv.CSVFilters +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -30,9 +32,27 @@ case class CSVScanBuilder( schema: StructType, dataSchema: StructType, options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) with SupportsPushDownFilters { override def build(): Scan = { - CSVScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + CSVScan( + sparkSession, + fileIndex, + dataSchema, + readDataSchema(), + readPartitionSchema(), + options, + pushedFilters()) } + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + if (sparkSession.sessionState.conf.csvFilterPushDown) { + _pushedFilters = CSVFilters.pushedFilters(filters, dataSchema) + } + filters + } + + override def pushedFilters(): Array[Filter] = _pushedFilters } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala index ad80afa441de1..e2abb39c986a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala @@ -23,6 +23,7 @@ import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{Column, Dataset, Row} import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -293,6 +294,50 @@ object CSVBenchmark extends SqlBasedBenchmark { } } + private def filtersPushdownBenchmark(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark(s"Filters pushdown", rowsNum, output = output) + val colsNum = 100 + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", TimestampType)) + val schema = StructType(StructField("key", IntegerType) +: fields) + def columns(): Seq[Column] = { + val ts = Seq.tabulate(colsNum) { i => + lit(Instant.ofEpochSecond(i * 12345678)).as(s"col$i") + } + ($"id" % 1000).as("key") +: ts + } + withTempPath { path => + spark.range(rowsNum).select(columns(): _*) + .write.option("header", true) + .csv(path.getAbsolutePath) + def readback = { + spark.read + .option("header", true) + .schema(schema) + .csv(path.getAbsolutePath) + } + + benchmark.addCase(s"w/o filters", numIters) { _ => + readback.noop() + } + + def withFilter(configEnabled: Boolean): Unit = { + withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> configEnabled.toString()) { + readback.filter($"key" === 0).noop() + } + } + + benchmark.addCase(s"pushdown disabled", numIters) { _ => + withFilter(configEnabled = false) + } + + benchmark.addCase(s"w/ filters", numIters) { _ => + withFilter(configEnabled = true) + } + + benchmark.run() + } + } + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Benchmark to measure CSV read/write performance") { val numIters = 3 @@ -300,6 +345,7 @@ object CSVBenchmark extends SqlBasedBenchmark { multiColumnsBenchmark(rowsNum = 1000 * 1000, numIters) countBenchmark(rowsNum = 10 * 1000 * 1000, numIters) datetimeBenchmark(rowsNum = 10 * 1000 * 1000, numIters) + filtersPushdownBenchmark(rowsNum = 100 * 1000, numIters) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index ae9aaf15aae9f..846b5c594d42e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec import org.apache.spark.{SparkException, TestUtils} -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -2195,4 +2195,79 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { checkAnswer(resultDF, Row("a", 2, "e", "c")) } } + + test("filters push down") { + Seq(true, false).foreach { filterPushdown => + Seq(true, false).foreach { columnPruning => + withSQLConf( + SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString, + SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> columnPruning.toString) { + + withTempPath { path => + val t = "2019-12-17 00:01:02" + Seq( + "c0,c1,c2", + "abc,1,2019-11-14 20:35:30", + s"def,2,$t").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + Seq(true, false).foreach { multiLine => + Seq("PERMISSIVE", "DROPMALFORMED", "FAILFAST").foreach { mode => + val readback = spark.read + .option("mode", mode) + .option("header", true) + .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .option("multiLine", multiLine) + .schema("c0 string, c1 integer, c2 timestamp") + .csv(path.getAbsolutePath) + .where($"c1" === 2) + .select($"c2") + // count() pushes empty schema. This checks handling of a filter + // which refers to not existed field. + assert(readback.count() === 1) + checkAnswer(readback, Row(Timestamp.valueOf(t))) + } + } + } + } + } + } + } + + test("filters push down - malformed input in PERMISSIVE mode") { + val invalidTs = "2019-123-14 20:35:30" + val invalidRow = s"0,$invalidTs,999" + val validTs = "2019-12-14 20:35:30" + Seq(true, false).foreach { filterPushdown => + withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString) { + withTempPath { path => + Seq( + "c0,c1,c2", + invalidRow, + s"1,$validTs,999").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + def checkReadback(condition: Column, expected: Seq[Row]): Unit = { + val readback = spark.read + .option("mode", "PERMISSIVE") + .option("columnNameOfCorruptRecord", "c3") + .option("header", true) + .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .schema("c0 integer, c1 timestamp, c2 integer, c3 string") + .csv(path.getAbsolutePath) + .where(condition) + .select($"c0", $"c1", $"c3") + checkAnswer(readback, expected) + } + + checkReadback( + condition = $"c2" === 999, + expected = Seq(Row(0, null, invalidRow), Row(1, Timestamp.valueOf(validTs), null))) + checkReadback( + condition = $"c2" === 999 && $"c1" > "1970-01-01 00:00:00", + expected = Seq(Row(1, Timestamp.valueOf(validTs), null))) + } + } + } + } } From f88874194ac8099a6fa8067a7d2b7a98fca6a849 Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Wed, 15 Jan 2020 20:19:44 -0800 Subject: [PATCH 1124/1387] [SPARK-30491][INFRA] Enable dependency audit files to tell dependency classifier ### What changes were proposed in this pull request? Enable dependency audit files to tell the value of artifact id, version, and classifier of a dependency. For example, `avro-mapred-1.8.2-hadoop2.jar` should be expanded to `avro-mapred/1.8.2/hadoop2/avro-mapred-1.8.2-hadoop2.jar` where `avro-mapred` is the artifact id, `1.8.2` is the version, and `haddop2` is the classifier. ### Why are the changes needed? Dependency audit files are expected to be consumed by automated tests or downstream tools. However, current dependency audit files under `dev/deps` only show jar names. And there isn't a simple rule on how to parse the jar name to get the values of different fields. For example, `hadoop2` is the classifier of `avro-mapred-1.8.2-hadoop2.jar`, in contrast, `incubating` is the version of `htrace-core-3.1.0-incubating.jar`. Reference: There is a good example of the downstream tool that would be enabled as yhuai suggested, > Say we have a Spark application that depends on a third-party dependency `foo`, which pulls in `jackson` as a transient dependency. Unfortunately, `foo` depends on a different version of `jackson` than Spark. So, in the pom of this Spark application, we use the dependency management section to pin the version of `jackson`. By doing this, we are lifting `jackson` to the top-level dependency of my application and I want to have a way to keep tracking what Spark uses. What we can do is to cross-check my Spark application's classpath with what Spark uses. Then, with a test written in my code base, whenever my application bumps Spark version, this test will check what we define in the application and what Spark has, and then remind us to change our application's pom if needed. In my case, I am fine to directly access git to get these audit files. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Code changes are verified by generated dependency audit files naturally. Thus, there are no tests added. Closes #27177 from mengCareers/depsOptimize. Lead-authored-by: Xinrong Meng Co-authored-by: mengCareers Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 416 ++++++++++----------- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 446 +++++++++++----------- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 478 ++++++++++++------------ dev/test-dependencies.sh | 20 +- 4 files changed, 688 insertions(+), 672 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 04c6e5f94fdb7..af702e8283826 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -1,208 +1,208 @@ -JLargeArrays-1.5.jar -JTransforms-3.1.jar -JavaEWAH-0.3.2.jar -RoaringBitmap-0.7.45.jar -ST4-4.0.4.jar -activation-1.1.1.jar -aircompressor-0.10.jar -algebra_2.12-2.0.0-M2.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar -antlr4-runtime-4.7.1.jar -aopalliance-1.0.jar -aopalliance-repackaged-2.6.1.jar -apache-log4j-extras-1.2.17.jar -apacheds-i18n-2.0.0-M15.jar -apacheds-kerberos-codec-2.0.0-M15.jar -api-asn1-api-1.0.0-M20.jar -api-util-1.0.0-M20.jar -arpack_combined_all-0.1.jar -arrow-format-0.15.1.jar -arrow-memory-0.15.1.jar -arrow-vector-0.15.1.jar -audience-annotations-0.5.0.jar -automaton-1.11-8.jar -avro-1.8.2.jar -avro-ipc-1.8.2.jar -avro-mapred-1.8.2-hadoop2.jar -bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-1.0.jar -breeze_2.12-1.0.jar -cats-kernel_2.12-2.0.0-M4.jar -chill-java-0.9.3.jar -chill_2.12-0.9.3.jar -commons-beanutils-1.9.4.jar -commons-cli-1.2.jar -commons-codec-1.10.jar -commons-collections-3.2.2.jar -commons-compiler-3.0.15.jar -commons-compress-1.8.1.jar -commons-configuration-1.6.jar -commons-crypto-1.0.0.jar -commons-dbcp-1.4.jar -commons-digester-1.8.jar -commons-httpclient-3.1.jar -commons-io-2.4.jar -commons-lang-2.6.jar -commons-lang3-3.9.jar -commons-logging-1.1.3.jar -commons-math3-3.4.1.jar -commons-net-3.1.jar -commons-pool-1.5.4.jar -commons-text-1.6.jar -compress-lzf-1.0.3.jar -core-1.1.2.jar -curator-client-2.7.1.jar -curator-framework-2.7.1.jar -curator-recipes-2.7.1.jar -datanucleus-api-jdo-3.2.6.jar -datanucleus-core-3.2.10.jar -datanucleus-rdbms-3.2.9.jar -derby-10.12.1.1.jar -flatbuffers-java-1.9.0.jar -generex-1.0.2.jar -gson-2.2.4.jar -guava-14.0.1.jar -guice-3.0.jar -guice-servlet-3.0.jar -hadoop-annotations-2.7.4.jar -hadoop-auth-2.7.4.jar -hadoop-client-2.7.4.jar -hadoop-common-2.7.4.jar -hadoop-hdfs-2.7.4.jar -hadoop-mapreduce-client-app-2.7.4.jar -hadoop-mapreduce-client-common-2.7.4.jar -hadoop-mapreduce-client-core-2.7.4.jar -hadoop-mapreduce-client-jobclient-2.7.4.jar -hadoop-mapreduce-client-shuffle-2.7.4.jar -hadoop-yarn-api-2.7.4.jar -hadoop-yarn-client-2.7.4.jar -hadoop-yarn-common-2.7.4.jar -hadoop-yarn-server-common-2.7.4.jar -hadoop-yarn-server-web-proxy-2.7.4.jar -hk2-api-2.6.1.jar -hk2-locator-2.6.1.jar -hk2-utils-2.6.1.jar -htrace-core-3.1.0-incubating.jar -httpclient-4.5.6.jar -httpcore-4.4.12.jar -istack-commons-runtime-3.0.8.jar -ivy-2.4.0.jar -jackson-annotations-2.10.0.jar -jackson-core-2.10.0.jar -jackson-core-asl-1.9.13.jar -jackson-databind-2.10.0.jar -jackson-dataformat-yaml-2.10.0.jar -jackson-jaxrs-1.9.13.jar -jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.10.0.jar -jackson-module-paranamer-2.10.0.jar -jackson-module-scala_2.12-2.10.0.jar -jackson-xc-1.9.13.jar -jakarta.activation-api-1.2.1.jar -jakarta.annotation-api-1.3.5.jar -jakarta.inject-2.6.1.jar -jakarta.validation-api-2.0.2.jar -jakarta.ws.rs-api-2.1.6.jar -jakarta.xml.bind-api-2.3.2.jar -janino-3.0.15.jar -javassist-3.22.0-CR2.jar -javax.inject-1.jar -javax.servlet-api-3.1.0.jar -javolution-5.5.1.jar -jaxb-api-2.2.2.jar -jaxb-runtime-2.3.2.jar -jcl-over-slf4j-1.7.16.jar -jdo-api-3.0.1.jar -jersey-client-2.29.1.jar -jersey-common-2.29.1.jar -jersey-container-servlet-2.29.1.jar -jersey-container-servlet-core-2.29.1.jar -jersey-hk2-2.29.1.jar -jersey-media-jaxb-2.29.1.jar -jersey-server-2.29.1.jar -jetty-6.1.26.jar -jetty-sslengine-6.1.26.jar -jetty-util-6.1.26.jar -jline-2.14.6.jar -joda-time-2.10.5.jar -jodd-core-3.5.2.jar -jpam-1.1.jar -json4s-ast_2.12-3.6.6.jar -json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12-3.6.6.jar -jsp-api-2.1.jar -jsr305-3.0.0.jar -jta-1.1.jar -jul-to-slf4j-1.7.16.jar -kryo-shaded-4.0.2.jar -kubernetes-client-4.6.4.jar -kubernetes-model-4.6.4.jar -kubernetes-model-common-4.6.4.jar -leveldbjni-all-1.8.jar -libfb303-0.9.3.jar -libthrift-0.12.0.jar -log4j-1.2.17.jar -logging-interceptor-3.12.6.jar -lz4-java-1.7.0.jar -machinist_2.12-0.6.8.jar -macro-compat_2.12-1.1.1.jar -mesos-1.4.0-shaded-protobuf.jar -metrics-core-4.1.1.jar -metrics-graphite-4.1.1.jar -metrics-jmx-4.1.1.jar -metrics-json-4.1.1.jar -metrics-jvm-4.1.1.jar -minlog-1.3.0.jar -netty-all-4.1.42.Final.jar -objenesis-2.5.1.jar -okhttp-3.12.6.jar -okio-1.15.0.jar -opencsv-2.3.jar -orc-core-1.5.8-nohive.jar -orc-mapreduce-1.5.8-nohive.jar -orc-shims-1.5.8.jar -oro-2.0.8.jar -osgi-resource-locator-1.0.3.jar -paranamer-2.8.jar -parquet-column-1.10.1.jar -parquet-common-1.10.1.jar -parquet-encoding-1.10.1.jar -parquet-format-2.4.0.jar -parquet-hadoop-1.10.1.jar -parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.10.1.jar -protobuf-java-2.5.0.jar -py4j-0.10.8.1.jar -pyrolite-4.30.jar -scala-collection-compat_2.12-2.1.1.jar -scala-compiler-2.12.10.jar -scala-library-2.12.10.jar -scala-parser-combinators_2.12-1.1.2.jar -scala-reflect-2.12.10.jar -scala-xml_2.12-1.2.0.jar -shapeless_2.12-2.3.3.jar -shims-0.7.45.jar -slf4j-api-1.7.16.jar -slf4j-log4j12-1.7.16.jar -snakeyaml-1.24.jar -snappy-0.2.jar -snappy-java-1.1.7.3.jar -spire-macros_2.12-0.17.0-M1.jar -spire-platform_2.12-0.17.0-M1.jar -spire-util_2.12-0.17.0-M1.jar -spire_2.12-0.17.0-M1.jar -stax-api-1.0-2.jar -stax-api-1.0.1.jar -stream-2.9.6.jar -stringtemplate-3.2.1.jar -super-csv-2.2.0.jar -univocity-parsers-2.8.3.jar -xbean-asm7-shaded-4.15.jar -xercesImpl-2.9.1.jar -xmlenc-0.52.jar -xz-1.5.jar -zjsonpatch-0.3.0.jar -zookeeper-3.4.14.jar -zstd-jni-1.4.4-3.jar +JLargeArrays/1.5//JLargeArrays-1.5.jar +JTransforms/3.1//JTransforms-3.1.jar +JavaEWAH/0.3.2//JavaEWAH-0.3.2.jar +RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +ST4/4.0.4//ST4-4.0.4.jar +activation/1.1.1//activation-1.1.1.jar +aircompressor/0.10//aircompressor-0.10.jar +algebra_2.12/2.0.0-M2//algebra_2.12-2.0.0-M2.jar +antlr-runtime/3.4//antlr-runtime-3.4.jar +antlr/2.7.7//antlr-2.7.7.jar +antlr4-runtime/4.7.1//antlr4-runtime-4.7.1.jar +aopalliance-repackaged/2.6.1//aopalliance-repackaged-2.6.1.jar +aopalliance/1.0//aopalliance-1.0.jar +apache-log4j-extras/1.2.17//apache-log4j-extras-1.2.17.jar +apacheds-i18n/2.0.0-M15//apacheds-i18n-2.0.0-M15.jar +apacheds-kerberos-codec/2.0.0-M15//apacheds-kerberos-codec-2.0.0-M15.jar +api-asn1-api/1.0.0-M20//api-asn1-api-1.0.0-M20.jar +api-util/1.0.0-M20//api-util-1.0.0-M20.jar +arpack_combined_all/0.1//arpack_combined_all-0.1.jar +arrow-format/0.15.1//arrow-format-0.15.1.jar +arrow-memory/0.15.1//arrow-memory-0.15.1.jar +arrow-vector/0.15.1//arrow-vector-0.15.1.jar +audience-annotations/0.5.0//audience-annotations-0.5.0.jar +automaton/1.11-8//automaton-1.11-8.jar +avro-ipc/1.8.2//avro-ipc-1.8.2.jar +avro-mapred/1.8.2/hadoop2/avro-mapred-1.8.2-hadoop2.jar +avro/1.8.2//avro-1.8.2.jar +bonecp/0.8.0.RELEASE//bonecp-0.8.0.RELEASE.jar +breeze-macros_2.12/1.0//breeze-macros_2.12-1.0.jar +breeze_2.12/1.0//breeze_2.12-1.0.jar +cats-kernel_2.12/2.0.0-M4//cats-kernel_2.12-2.0.0-M4.jar +chill-java/0.9.3//chill-java-0.9.3.jar +chill_2.12/0.9.3//chill_2.12-0.9.3.jar +commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar +commons-cli/1.2//commons-cli-1.2.jar +commons-codec/1.10//commons-codec-1.10.jar +commons-collections/3.2.2//commons-collections-3.2.2.jar +commons-compiler/3.0.15//commons-compiler-3.0.15.jar +commons-compress/1.8.1//commons-compress-1.8.1.jar +commons-configuration/1.6//commons-configuration-1.6.jar +commons-crypto/1.0.0//commons-crypto-1.0.0.jar +commons-dbcp/1.4//commons-dbcp-1.4.jar +commons-digester/1.8//commons-digester-1.8.jar +commons-httpclient/3.1//commons-httpclient-3.1.jar +commons-io/2.4//commons-io-2.4.jar +commons-lang/2.6//commons-lang-2.6.jar +commons-lang3/3.9//commons-lang3-3.9.jar +commons-logging/1.1.3//commons-logging-1.1.3.jar +commons-math3/3.4.1//commons-math3-3.4.1.jar +commons-net/3.1//commons-net-3.1.jar +commons-pool/1.5.4//commons-pool-1.5.4.jar +commons-text/1.6//commons-text-1.6.jar +compress-lzf/1.0.3//compress-lzf-1.0.3.jar +core/1.1.2//core-1.1.2.jar +curator-client/2.7.1//curator-client-2.7.1.jar +curator-framework/2.7.1//curator-framework-2.7.1.jar +curator-recipes/2.7.1//curator-recipes-2.7.1.jar +datanucleus-api-jdo/3.2.6//datanucleus-api-jdo-3.2.6.jar +datanucleus-core/3.2.10//datanucleus-core-3.2.10.jar +datanucleus-rdbms/3.2.9//datanucleus-rdbms-3.2.9.jar +derby/10.12.1.1//derby-10.12.1.1.jar +flatbuffers-java/1.9.0//flatbuffers-java-1.9.0.jar +generex/1.0.2//generex-1.0.2.jar +gson/2.2.4//gson-2.2.4.jar +guava/14.0.1//guava-14.0.1.jar +guice-servlet/3.0//guice-servlet-3.0.jar +guice/3.0//guice-3.0.jar +hadoop-annotations/2.7.4//hadoop-annotations-2.7.4.jar +hadoop-auth/2.7.4//hadoop-auth-2.7.4.jar +hadoop-client/2.7.4//hadoop-client-2.7.4.jar +hadoop-common/2.7.4//hadoop-common-2.7.4.jar +hadoop-hdfs/2.7.4//hadoop-hdfs-2.7.4.jar +hadoop-mapreduce-client-app/2.7.4//hadoop-mapreduce-client-app-2.7.4.jar +hadoop-mapreduce-client-common/2.7.4//hadoop-mapreduce-client-common-2.7.4.jar +hadoop-mapreduce-client-core/2.7.4//hadoop-mapreduce-client-core-2.7.4.jar +hadoop-mapreduce-client-jobclient/2.7.4//hadoop-mapreduce-client-jobclient-2.7.4.jar +hadoop-mapreduce-client-shuffle/2.7.4//hadoop-mapreduce-client-shuffle-2.7.4.jar +hadoop-yarn-api/2.7.4//hadoop-yarn-api-2.7.4.jar +hadoop-yarn-client/2.7.4//hadoop-yarn-client-2.7.4.jar +hadoop-yarn-common/2.7.4//hadoop-yarn-common-2.7.4.jar +hadoop-yarn-server-common/2.7.4//hadoop-yarn-server-common-2.7.4.jar +hadoop-yarn-server-web-proxy/2.7.4//hadoop-yarn-server-web-proxy-2.7.4.jar +hk2-api/2.6.1//hk2-api-2.6.1.jar +hk2-locator/2.6.1//hk2-locator-2.6.1.jar +hk2-utils/2.6.1//hk2-utils-2.6.1.jar +htrace-core/3.1.0-incubating//htrace-core-3.1.0-incubating.jar +httpclient/4.5.6//httpclient-4.5.6.jar +httpcore/4.4.12//httpcore-4.4.12.jar +istack-commons-runtime/3.0.8//istack-commons-runtime-3.0.8.jar +ivy/2.4.0//ivy-2.4.0.jar +jackson-annotations/2.10.0//jackson-annotations-2.10.0.jar +jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar +jackson-core/2.10.0//jackson-core-2.10.0.jar +jackson-databind/2.10.0//jackson-databind-2.10.0.jar +jackson-dataformat-yaml/2.10.0//jackson-dataformat-yaml-2.10.0.jar +jackson-jaxrs/1.9.13//jackson-jaxrs-1.9.13.jar +jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations/2.10.0//jackson-module-jaxb-annotations-2.10.0.jar +jackson-module-paranamer/2.10.0//jackson-module-paranamer-2.10.0.jar +jackson-module-scala_2.12/2.10.0//jackson-module-scala_2.12-2.10.0.jar +jackson-xc/1.9.13//jackson-xc-1.9.13.jar +jakarta.activation-api/1.2.1//jakarta.activation-api-1.2.1.jar +jakarta.annotation-api/1.3.5//jakarta.annotation-api-1.3.5.jar +jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar +jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar +jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar +jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar +janino/3.0.15//janino-3.0.15.jar +javassist/3.22.0-CR2//javassist-3.22.0-CR2.jar +javax.inject/1//javax.inject-1.jar +javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar +javolution/5.5.1//javolution-5.5.1.jar +jaxb-api/2.2.2//jaxb-api-2.2.2.jar +jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar +jcl-over-slf4j/1.7.16//jcl-over-slf4j-1.7.16.jar +jdo-api/3.0.1//jdo-api-3.0.1.jar +jersey-client/2.29.1//jersey-client-2.29.1.jar +jersey-common/2.29.1//jersey-common-2.29.1.jar +jersey-container-servlet-core/2.29.1//jersey-container-servlet-core-2.29.1.jar +jersey-container-servlet/2.29.1//jersey-container-servlet-2.29.1.jar +jersey-hk2/2.29.1//jersey-hk2-2.29.1.jar +jersey-media-jaxb/2.29.1//jersey-media-jaxb-2.29.1.jar +jersey-server/2.29.1//jersey-server-2.29.1.jar +jetty-sslengine/6.1.26//jetty-sslengine-6.1.26.jar +jetty-util/6.1.26//jetty-util-6.1.26.jar +jetty/6.1.26//jetty-6.1.26.jar +jline/2.14.6//jline-2.14.6.jar +joda-time/2.10.5//joda-time-2.10.5.jar +jodd-core/3.5.2//jodd-core-3.5.2.jar +jpam/1.1//jpam-1.1.jar +json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar +json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar +json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar +json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +jsp-api/2.1//jsp-api-2.1.jar +jsr305/3.0.0//jsr305-3.0.0.jar +jta/1.1//jta-1.1.jar +jul-to-slf4j/1.7.16//jul-to-slf4j-1.7.16.jar +kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar +kubernetes-client/4.6.4//kubernetes-client-4.6.4.jar +kubernetes-model-common/4.6.4//kubernetes-model-common-4.6.4.jar +kubernetes-model/4.6.4//kubernetes-model-4.6.4.jar +leveldbjni-all/1.8//leveldbjni-all-1.8.jar +libfb303/0.9.3//libfb303-0.9.3.jar +libthrift/0.12.0//libthrift-0.12.0.jar +log4j/1.2.17//log4j-1.2.17.jar +logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar +lz4-java/1.7.0//lz4-java-1.7.0.jar +machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar +macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar +mesos/1.4.0/shaded-protobuf/mesos-1.4.0-shaded-protobuf.jar +metrics-core/4.1.1//metrics-core-4.1.1.jar +metrics-graphite/4.1.1//metrics-graphite-4.1.1.jar +metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar +metrics-json/4.1.1//metrics-json-4.1.1.jar +metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar +minlog/1.3.0//minlog-1.3.0.jar +netty-all/4.1.42.Final//netty-all-4.1.42.Final.jar +objenesis/2.5.1//objenesis-2.5.1.jar +okhttp/3.12.6//okhttp-3.12.6.jar +okio/1.15.0//okio-1.15.0.jar +opencsv/2.3//opencsv-2.3.jar +orc-core/1.5.8/nohive/orc-core-1.5.8-nohive.jar +orc-mapreduce/1.5.8/nohive/orc-mapreduce-1.5.8-nohive.jar +orc-shims/1.5.8//orc-shims-1.5.8.jar +oro/2.0.8//oro-2.0.8.jar +osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar +paranamer/2.8//paranamer-2.8.jar +parquet-column/1.10.1//parquet-column-1.10.1.jar +parquet-common/1.10.1//parquet-common-1.10.1.jar +parquet-encoding/1.10.1//parquet-encoding-1.10.1.jar +parquet-format/2.4.0//parquet-format-2.4.0.jar +parquet-hadoop-bundle/1.6.0//parquet-hadoop-bundle-1.6.0.jar +parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar +parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar +protobuf-java/2.5.0//protobuf-java-2.5.0.jar +py4j/0.10.8.1//py4j-0.10.8.1.jar +pyrolite/4.30//pyrolite-4.30.jar +scala-collection-compat_2.12/2.1.1//scala-collection-compat_2.12-2.1.1.jar +scala-compiler/2.12.10//scala-compiler-2.12.10.jar +scala-library/2.12.10//scala-library-2.12.10.jar +scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar +scala-reflect/2.12.10//scala-reflect-2.12.10.jar +scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar +shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar +shims/0.7.45//shims-0.7.45.jar +slf4j-api/1.7.16//slf4j-api-1.7.16.jar +slf4j-log4j12/1.7.16//slf4j-log4j12-1.7.16.jar +snakeyaml/1.24//snakeyaml-1.24.jar +snappy-java/1.1.7.3//snappy-java-1.1.7.3.jar +snappy/0.2//snappy-0.2.jar +spire-macros_2.12/0.17.0-M1//spire-macros_2.12-0.17.0-M1.jar +spire-platform_2.12/0.17.0-M1//spire-platform_2.12-0.17.0-M1.jar +spire-util_2.12/0.17.0-M1//spire-util_2.12-0.17.0-M1.jar +spire_2.12/0.17.0-M1//spire_2.12-0.17.0-M1.jar +stax-api/1.0-2//stax-api-1.0-2.jar +stax-api/1.0.1//stax-api-1.0.1.jar +stream/2.9.6//stream-2.9.6.jar +stringtemplate/3.2.1//stringtemplate-3.2.1.jar +super-csv/2.2.0//super-csv-2.2.0.jar +univocity-parsers/2.8.3//univocity-parsers-2.8.3.jar +xbean-asm7-shaded/4.15//xbean-asm7-shaded-4.15.jar +xercesImpl/2.9.1//xercesImpl-2.9.1.jar +xmlenc/0.52//xmlenc-0.52.jar +xz/1.5//xz-1.5.jar +zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar +zookeeper/3.4.14//zookeeper-3.4.14.jar +zstd-jni/1.4.4-3//zstd-jni-1.4.4-3.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 300c3ee931471..9bfd8014de487 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -1,223 +1,223 @@ -HikariCP-2.5.1.jar -JLargeArrays-1.5.jar -JTransforms-3.1.jar -RoaringBitmap-0.7.45.jar -ST4-4.0.4.jar -activation-1.1.1.jar -aircompressor-0.10.jar -algebra_2.12-2.0.0-M2.jar -antlr-runtime-3.5.2.jar -antlr4-runtime-4.7.1.jar -aopalliance-1.0.jar -aopalliance-repackaged-2.6.1.jar -apacheds-i18n-2.0.0-M15.jar -apacheds-kerberos-codec-2.0.0-M15.jar -api-asn1-api-1.0.0-M20.jar -api-util-1.0.0-M20.jar -arpack_combined_all-0.1.jar -arrow-format-0.15.1.jar -arrow-memory-0.15.1.jar -arrow-vector-0.15.1.jar -audience-annotations-0.5.0.jar -automaton-1.11-8.jar -avro-1.8.2.jar -avro-ipc-1.8.2.jar -avro-mapred-1.8.2-hadoop2.jar -bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-1.0.jar -breeze_2.12-1.0.jar -cats-kernel_2.12-2.0.0-M4.jar -chill-java-0.9.3.jar -chill_2.12-0.9.3.jar -commons-beanutils-1.9.4.jar -commons-cli-1.2.jar -commons-codec-1.10.jar -commons-collections-3.2.2.jar -commons-compiler-3.0.15.jar -commons-compress-1.8.1.jar -commons-configuration-1.6.jar -commons-crypto-1.0.0.jar -commons-dbcp-1.4.jar -commons-digester-1.8.jar -commons-httpclient-3.1.jar -commons-io-2.4.jar -commons-lang-2.6.jar -commons-lang3-3.9.jar -commons-logging-1.1.3.jar -commons-math3-3.4.1.jar -commons-net-3.1.jar -commons-pool-1.5.4.jar -commons-text-1.6.jar -compress-lzf-1.0.3.jar -core-1.1.2.jar -curator-client-2.7.1.jar -curator-framework-2.7.1.jar -curator-recipes-2.7.1.jar -datanucleus-api-jdo-4.2.4.jar -datanucleus-core-4.1.17.jar -datanucleus-rdbms-4.1.19.jar -derby-10.12.1.1.jar -dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar -flatbuffers-java-1.9.0.jar -generex-1.0.2.jar -gson-2.2.4.jar -guava-14.0.1.jar -guice-3.0.jar -guice-servlet-3.0.jar -hadoop-annotations-2.7.4.jar -hadoop-auth-2.7.4.jar -hadoop-client-2.7.4.jar -hadoop-common-2.7.4.jar -hadoop-hdfs-2.7.4.jar -hadoop-mapreduce-client-app-2.7.4.jar -hadoop-mapreduce-client-common-2.7.4.jar -hadoop-mapreduce-client-core-2.7.4.jar -hadoop-mapreduce-client-jobclient-2.7.4.jar -hadoop-mapreduce-client-shuffle-2.7.4.jar -hadoop-yarn-api-2.7.4.jar -hadoop-yarn-client-2.7.4.jar -hadoop-yarn-common-2.7.4.jar -hadoop-yarn-server-common-2.7.4.jar -hadoop-yarn-server-web-proxy-2.7.4.jar -hive-beeline-2.3.6.jar -hive-cli-2.3.6.jar -hive-common-2.3.6.jar -hive-exec-2.3.6-core.jar -hive-jdbc-2.3.6.jar -hive-llap-common-2.3.6.jar -hive-metastore-2.3.6.jar -hive-serde-2.3.6.jar -hive-service-rpc-2.3.6.jar -hive-shims-0.23-2.3.6.jar -hive-shims-2.3.6.jar -hive-shims-common-2.3.6.jar -hive-shims-scheduler-2.3.6.jar -hive-storage-api-2.6.0.jar -hive-vector-code-gen-2.3.6.jar -hk2-api-2.6.1.jar -hk2-locator-2.6.1.jar -hk2-utils-2.6.1.jar -htrace-core-3.1.0-incubating.jar -httpclient-4.5.6.jar -httpcore-4.4.12.jar -istack-commons-runtime-3.0.8.jar -ivy-2.4.0.jar -jackson-annotations-2.10.0.jar -jackson-core-2.10.0.jar -jackson-core-asl-1.9.13.jar -jackson-databind-2.10.0.jar -jackson-dataformat-yaml-2.10.0.jar -jackson-jaxrs-1.9.13.jar -jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.10.0.jar -jackson-module-paranamer-2.10.0.jar -jackson-module-scala_2.12-2.10.0.jar -jackson-xc-1.9.13.jar -jakarta.activation-api-1.2.1.jar -jakarta.annotation-api-1.3.5.jar -jakarta.inject-2.6.1.jar -jakarta.validation-api-2.0.2.jar -jakarta.ws.rs-api-2.1.6.jar -jakarta.xml.bind-api-2.3.2.jar -janino-3.0.15.jar -javassist-3.22.0-CR2.jar -javax.inject-1.jar -javax.jdo-3.2.0-m3.jar -javax.servlet-api-3.1.0.jar -javolution-5.5.1.jar -jaxb-api-2.2.2.jar -jaxb-runtime-2.3.2.jar -jcl-over-slf4j-1.7.16.jar -jdo-api-3.0.1.jar -jersey-client-2.29.1.jar -jersey-common-2.29.1.jar -jersey-container-servlet-2.29.1.jar -jersey-container-servlet-core-2.29.1.jar -jersey-hk2-2.29.1.jar -jersey-media-jaxb-2.29.1.jar -jersey-server-2.29.1.jar -jetty-6.1.26.jar -jetty-sslengine-6.1.26.jar -jetty-util-6.1.26.jar -jline-2.14.6.jar -joda-time-2.10.5.jar -jodd-core-3.5.2.jar -jpam-1.1.jar -json-1.8.jar -json4s-ast_2.12-3.6.6.jar -json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12-3.6.6.jar -jsp-api-2.1.jar -jsr305-3.0.0.jar -jta-1.1.jar -jul-to-slf4j-1.7.16.jar -kryo-shaded-4.0.2.jar -kubernetes-client-4.6.4.jar -kubernetes-model-4.6.4.jar -kubernetes-model-common-4.6.4.jar -leveldbjni-all-1.8.jar -libfb303-0.9.3.jar -libthrift-0.12.0.jar -log4j-1.2.17.jar -logging-interceptor-3.12.6.jar -lz4-java-1.7.0.jar -machinist_2.12-0.6.8.jar -macro-compat_2.12-1.1.1.jar -mesos-1.4.0-shaded-protobuf.jar -metrics-core-4.1.1.jar -metrics-graphite-4.1.1.jar -metrics-jmx-4.1.1.jar -metrics-json-4.1.1.jar -metrics-jvm-4.1.1.jar -minlog-1.3.0.jar -netty-all-4.1.42.Final.jar -objenesis-2.5.1.jar -okhttp-3.12.6.jar -okio-1.15.0.jar -opencsv-2.3.jar -orc-core-1.5.8.jar -orc-mapreduce-1.5.8.jar -orc-shims-1.5.8.jar -oro-2.0.8.jar -osgi-resource-locator-1.0.3.jar -paranamer-2.8.jar -parquet-column-1.10.1.jar -parquet-common-1.10.1.jar -parquet-encoding-1.10.1.jar -parquet-format-2.4.0.jar -parquet-hadoop-1.10.1.jar -parquet-jackson-1.10.1.jar -protobuf-java-2.5.0.jar -py4j-0.10.8.1.jar -pyrolite-4.30.jar -scala-collection-compat_2.12-2.1.1.jar -scala-compiler-2.12.10.jar -scala-library-2.12.10.jar -scala-parser-combinators_2.12-1.1.2.jar -scala-reflect-2.12.10.jar -scala-xml_2.12-1.2.0.jar -shapeless_2.12-2.3.3.jar -shims-0.7.45.jar -slf4j-api-1.7.16.jar -slf4j-log4j12-1.7.16.jar -snakeyaml-1.24.jar -snappy-java-1.1.7.3.jar -spire-macros_2.12-0.17.0-M1.jar -spire-platform_2.12-0.17.0-M1.jar -spire-util_2.12-0.17.0-M1.jar -spire_2.12-0.17.0-M1.jar -stax-api-1.0-2.jar -stax-api-1.0.1.jar -stream-2.9.6.jar -super-csv-2.2.0.jar -transaction-api-1.1.jar -univocity-parsers-2.8.3.jar -velocity-1.5.jar -xbean-asm7-shaded-4.15.jar -xercesImpl-2.9.1.jar -xmlenc-0.52.jar -xz-1.5.jar -zjsonpatch-0.3.0.jar -zookeeper-3.4.14.jar -zstd-jni-1.4.4-3.jar +HikariCP/2.5.1//HikariCP-2.5.1.jar +JLargeArrays/1.5//JLargeArrays-1.5.jar +JTransforms/3.1//JTransforms-3.1.jar +RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +ST4/4.0.4//ST4-4.0.4.jar +activation/1.1.1//activation-1.1.1.jar +aircompressor/0.10//aircompressor-0.10.jar +algebra_2.12/2.0.0-M2//algebra_2.12-2.0.0-M2.jar +antlr-runtime/3.5.2//antlr-runtime-3.5.2.jar +antlr4-runtime/4.7.1//antlr4-runtime-4.7.1.jar +aopalliance-repackaged/2.6.1//aopalliance-repackaged-2.6.1.jar +aopalliance/1.0//aopalliance-1.0.jar +apacheds-i18n/2.0.0-M15//apacheds-i18n-2.0.0-M15.jar +apacheds-kerberos-codec/2.0.0-M15//apacheds-kerberos-codec-2.0.0-M15.jar +api-asn1-api/1.0.0-M20//api-asn1-api-1.0.0-M20.jar +api-util/1.0.0-M20//api-util-1.0.0-M20.jar +arpack_combined_all/0.1//arpack_combined_all-0.1.jar +arrow-format/0.15.1//arrow-format-0.15.1.jar +arrow-memory/0.15.1//arrow-memory-0.15.1.jar +arrow-vector/0.15.1//arrow-vector-0.15.1.jar +audience-annotations/0.5.0//audience-annotations-0.5.0.jar +automaton/1.11-8//automaton-1.11-8.jar +avro-ipc/1.8.2//avro-ipc-1.8.2.jar +avro-mapred/1.8.2/hadoop2/avro-mapred-1.8.2-hadoop2.jar +avro/1.8.2//avro-1.8.2.jar +bonecp/0.8.0.RELEASE//bonecp-0.8.0.RELEASE.jar +breeze-macros_2.12/1.0//breeze-macros_2.12-1.0.jar +breeze_2.12/1.0//breeze_2.12-1.0.jar +cats-kernel_2.12/2.0.0-M4//cats-kernel_2.12-2.0.0-M4.jar +chill-java/0.9.3//chill-java-0.9.3.jar +chill_2.12/0.9.3//chill_2.12-0.9.3.jar +commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar +commons-cli/1.2//commons-cli-1.2.jar +commons-codec/1.10//commons-codec-1.10.jar +commons-collections/3.2.2//commons-collections-3.2.2.jar +commons-compiler/3.0.15//commons-compiler-3.0.15.jar +commons-compress/1.8.1//commons-compress-1.8.1.jar +commons-configuration/1.6//commons-configuration-1.6.jar +commons-crypto/1.0.0//commons-crypto-1.0.0.jar +commons-dbcp/1.4//commons-dbcp-1.4.jar +commons-digester/1.8//commons-digester-1.8.jar +commons-httpclient/3.1//commons-httpclient-3.1.jar +commons-io/2.4//commons-io-2.4.jar +commons-lang/2.6//commons-lang-2.6.jar +commons-lang3/3.9//commons-lang3-3.9.jar +commons-logging/1.1.3//commons-logging-1.1.3.jar +commons-math3/3.4.1//commons-math3-3.4.1.jar +commons-net/3.1//commons-net-3.1.jar +commons-pool/1.5.4//commons-pool-1.5.4.jar +commons-text/1.6//commons-text-1.6.jar +compress-lzf/1.0.3//compress-lzf-1.0.3.jar +core/1.1.2//core-1.1.2.jar +curator-client/2.7.1//curator-client-2.7.1.jar +curator-framework/2.7.1//curator-framework-2.7.1.jar +curator-recipes/2.7.1//curator-recipes-2.7.1.jar +datanucleus-api-jdo/4.2.4//datanucleus-api-jdo-4.2.4.jar +datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar +datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar +derby/10.12.1.1//derby-10.12.1.1.jar +dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar +flatbuffers-java/1.9.0//flatbuffers-java-1.9.0.jar +generex/1.0.2//generex-1.0.2.jar +gson/2.2.4//gson-2.2.4.jar +guava/14.0.1//guava-14.0.1.jar +guice-servlet/3.0//guice-servlet-3.0.jar +guice/3.0//guice-3.0.jar +hadoop-annotations/2.7.4//hadoop-annotations-2.7.4.jar +hadoop-auth/2.7.4//hadoop-auth-2.7.4.jar +hadoop-client/2.7.4//hadoop-client-2.7.4.jar +hadoop-common/2.7.4//hadoop-common-2.7.4.jar +hadoop-hdfs/2.7.4//hadoop-hdfs-2.7.4.jar +hadoop-mapreduce-client-app/2.7.4//hadoop-mapreduce-client-app-2.7.4.jar +hadoop-mapreduce-client-common/2.7.4//hadoop-mapreduce-client-common-2.7.4.jar +hadoop-mapreduce-client-core/2.7.4//hadoop-mapreduce-client-core-2.7.4.jar +hadoop-mapreduce-client-jobclient/2.7.4//hadoop-mapreduce-client-jobclient-2.7.4.jar +hadoop-mapreduce-client-shuffle/2.7.4//hadoop-mapreduce-client-shuffle-2.7.4.jar +hadoop-yarn-api/2.7.4//hadoop-yarn-api-2.7.4.jar +hadoop-yarn-client/2.7.4//hadoop-yarn-client-2.7.4.jar +hadoop-yarn-common/2.7.4//hadoop-yarn-common-2.7.4.jar +hadoop-yarn-server-common/2.7.4//hadoop-yarn-server-common-2.7.4.jar +hadoop-yarn-server-web-proxy/2.7.4//hadoop-yarn-server-web-proxy-2.7.4.jar +hive-beeline/2.3.6//hive-beeline-2.3.6.jar +hive-cli/2.3.6//hive-cli-2.3.6.jar +hive-common/2.3.6//hive-common-2.3.6.jar +hive-exec/2.3.6/core/hive-exec-2.3.6-core.jar +hive-jdbc/2.3.6//hive-jdbc-2.3.6.jar +hive-llap-common/2.3.6//hive-llap-common-2.3.6.jar +hive-metastore/2.3.6//hive-metastore-2.3.6.jar +hive-serde/2.3.6//hive-serde-2.3.6.jar +hive-service-rpc/2.3.6//hive-service-rpc-2.3.6.jar +hive-shims-0.23/2.3.6//hive-shims-0.23-2.3.6.jar +hive-shims-common/2.3.6//hive-shims-common-2.3.6.jar +hive-shims-scheduler/2.3.6//hive-shims-scheduler-2.3.6.jar +hive-shims/2.3.6//hive-shims-2.3.6.jar +hive-storage-api/2.6.0//hive-storage-api-2.6.0.jar +hive-vector-code-gen/2.3.6//hive-vector-code-gen-2.3.6.jar +hk2-api/2.6.1//hk2-api-2.6.1.jar +hk2-locator/2.6.1//hk2-locator-2.6.1.jar +hk2-utils/2.6.1//hk2-utils-2.6.1.jar +htrace-core/3.1.0-incubating//htrace-core-3.1.0-incubating.jar +httpclient/4.5.6//httpclient-4.5.6.jar +httpcore/4.4.12//httpcore-4.4.12.jar +istack-commons-runtime/3.0.8//istack-commons-runtime-3.0.8.jar +ivy/2.4.0//ivy-2.4.0.jar +jackson-annotations/2.10.0//jackson-annotations-2.10.0.jar +jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar +jackson-core/2.10.0//jackson-core-2.10.0.jar +jackson-databind/2.10.0//jackson-databind-2.10.0.jar +jackson-dataformat-yaml/2.10.0//jackson-dataformat-yaml-2.10.0.jar +jackson-jaxrs/1.9.13//jackson-jaxrs-1.9.13.jar +jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations/2.10.0//jackson-module-jaxb-annotations-2.10.0.jar +jackson-module-paranamer/2.10.0//jackson-module-paranamer-2.10.0.jar +jackson-module-scala_2.12/2.10.0//jackson-module-scala_2.12-2.10.0.jar +jackson-xc/1.9.13//jackson-xc-1.9.13.jar +jakarta.activation-api/1.2.1//jakarta.activation-api-1.2.1.jar +jakarta.annotation-api/1.3.5//jakarta.annotation-api-1.3.5.jar +jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar +jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar +jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar +jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar +janino/3.0.15//janino-3.0.15.jar +javassist/3.22.0-CR2//javassist-3.22.0-CR2.jar +javax.inject/1//javax.inject-1.jar +javax.jdo/3.2.0-m3//javax.jdo-3.2.0-m3.jar +javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar +javolution/5.5.1//javolution-5.5.1.jar +jaxb-api/2.2.2//jaxb-api-2.2.2.jar +jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar +jcl-over-slf4j/1.7.16//jcl-over-slf4j-1.7.16.jar +jdo-api/3.0.1//jdo-api-3.0.1.jar +jersey-client/2.29.1//jersey-client-2.29.1.jar +jersey-common/2.29.1//jersey-common-2.29.1.jar +jersey-container-servlet-core/2.29.1//jersey-container-servlet-core-2.29.1.jar +jersey-container-servlet/2.29.1//jersey-container-servlet-2.29.1.jar +jersey-hk2/2.29.1//jersey-hk2-2.29.1.jar +jersey-media-jaxb/2.29.1//jersey-media-jaxb-2.29.1.jar +jersey-server/2.29.1//jersey-server-2.29.1.jar +jetty-sslengine/6.1.26//jetty-sslengine-6.1.26.jar +jetty-util/6.1.26//jetty-util-6.1.26.jar +jetty/6.1.26//jetty-6.1.26.jar +jline/2.14.6//jline-2.14.6.jar +joda-time/2.10.5//joda-time-2.10.5.jar +jodd-core/3.5.2//jodd-core-3.5.2.jar +jpam/1.1//jpam-1.1.jar +json/1.8//json-1.8.jar +json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar +json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar +json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar +json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +jsp-api/2.1//jsp-api-2.1.jar +jsr305/3.0.0//jsr305-3.0.0.jar +jta/1.1//jta-1.1.jar +jul-to-slf4j/1.7.16//jul-to-slf4j-1.7.16.jar +kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar +kubernetes-client/4.6.4//kubernetes-client-4.6.4.jar +kubernetes-model-common/4.6.4//kubernetes-model-common-4.6.4.jar +kubernetes-model/4.6.4//kubernetes-model-4.6.4.jar +leveldbjni-all/1.8//leveldbjni-all-1.8.jar +libfb303/0.9.3//libfb303-0.9.3.jar +libthrift/0.12.0//libthrift-0.12.0.jar +log4j/1.2.17//log4j-1.2.17.jar +logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar +lz4-java/1.7.0//lz4-java-1.7.0.jar +machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar +macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar +mesos/1.4.0/shaded-protobuf/mesos-1.4.0-shaded-protobuf.jar +metrics-core/4.1.1//metrics-core-4.1.1.jar +metrics-graphite/4.1.1//metrics-graphite-4.1.1.jar +metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar +metrics-json/4.1.1//metrics-json-4.1.1.jar +metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar +minlog/1.3.0//minlog-1.3.0.jar +netty-all/4.1.42.Final//netty-all-4.1.42.Final.jar +objenesis/2.5.1//objenesis-2.5.1.jar +okhttp/3.12.6//okhttp-3.12.6.jar +okio/1.15.0//okio-1.15.0.jar +opencsv/2.3//opencsv-2.3.jar +orc-core/1.5.8//orc-core-1.5.8.jar +orc-mapreduce/1.5.8//orc-mapreduce-1.5.8.jar +orc-shims/1.5.8//orc-shims-1.5.8.jar +oro/2.0.8//oro-2.0.8.jar +osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar +paranamer/2.8//paranamer-2.8.jar +parquet-column/1.10.1//parquet-column-1.10.1.jar +parquet-common/1.10.1//parquet-common-1.10.1.jar +parquet-encoding/1.10.1//parquet-encoding-1.10.1.jar +parquet-format/2.4.0//parquet-format-2.4.0.jar +parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar +parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar +protobuf-java/2.5.0//protobuf-java-2.5.0.jar +py4j/0.10.8.1//py4j-0.10.8.1.jar +pyrolite/4.30//pyrolite-4.30.jar +scala-collection-compat_2.12/2.1.1//scala-collection-compat_2.12-2.1.1.jar +scala-compiler/2.12.10//scala-compiler-2.12.10.jar +scala-library/2.12.10//scala-library-2.12.10.jar +scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar +scala-reflect/2.12.10//scala-reflect-2.12.10.jar +scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar +shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar +shims/0.7.45//shims-0.7.45.jar +slf4j-api/1.7.16//slf4j-api-1.7.16.jar +slf4j-log4j12/1.7.16//slf4j-log4j12-1.7.16.jar +snakeyaml/1.24//snakeyaml-1.24.jar +snappy-java/1.1.7.3//snappy-java-1.1.7.3.jar +spire-macros_2.12/0.17.0-M1//spire-macros_2.12-0.17.0-M1.jar +spire-platform_2.12/0.17.0-M1//spire-platform_2.12-0.17.0-M1.jar +spire-util_2.12/0.17.0-M1//spire-util_2.12-0.17.0-M1.jar +spire_2.12/0.17.0-M1//spire_2.12-0.17.0-M1.jar +stax-api/1.0-2//stax-api-1.0-2.jar +stax-api/1.0.1//stax-api-1.0.1.jar +stream/2.9.6//stream-2.9.6.jar +super-csv/2.2.0//super-csv-2.2.0.jar +transaction-api/1.1//transaction-api-1.1.jar +univocity-parsers/2.8.3//univocity-parsers-2.8.3.jar +velocity/1.5//velocity-1.5.jar +xbean-asm7-shaded/4.15//xbean-asm7-shaded-4.15.jar +xercesImpl/2.9.1//xercesImpl-2.9.1.jar +xmlenc/0.52//xmlenc-0.52.jar +xz/1.5//xz-1.5.jar +zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar +zookeeper/3.4.14//zookeeper-3.4.14.jar +zstd-jni/1.4.4-3//zstd-jni-1.4.4-3.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index aa1a8ccc42e33..da2020bbb7171 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -1,239 +1,239 @@ -HikariCP-2.5.1.jar -JLargeArrays-1.5.jar -JTransforms-3.1.jar -RoaringBitmap-0.7.45.jar -ST4-4.0.4.jar -accessors-smart-1.2.jar -activation-1.1.1.jar -aircompressor-0.10.jar -algebra_2.12-2.0.0-M2.jar -antlr-runtime-3.5.2.jar -antlr4-runtime-4.7.1.jar -aopalliance-1.0.jar -aopalliance-repackaged-2.6.1.jar -arpack_combined_all-0.1.jar -arrow-format-0.15.1.jar -arrow-memory-0.15.1.jar -arrow-vector-0.15.1.jar -audience-annotations-0.5.0.jar -automaton-1.11-8.jar -avro-1.8.2.jar -avro-ipc-1.8.2.jar -avro-mapred-1.8.2-hadoop2.jar -bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-1.0.jar -breeze_2.12-1.0.jar -cats-kernel_2.12-2.0.0-M4.jar -chill-java-0.9.3.jar -chill_2.12-0.9.3.jar -commons-beanutils-1.9.4.jar -commons-cli-1.2.jar -commons-codec-1.10.jar -commons-collections-3.2.2.jar -commons-compiler-3.0.15.jar -commons-compress-1.8.1.jar -commons-configuration2-2.1.1.jar -commons-crypto-1.0.0.jar -commons-daemon-1.0.13.jar -commons-dbcp-1.4.jar -commons-httpclient-3.1.jar -commons-io-2.4.jar -commons-lang-2.6.jar -commons-lang3-3.9.jar -commons-logging-1.1.3.jar -commons-math3-3.4.1.jar -commons-net-3.1.jar -commons-pool-1.5.4.jar -commons-text-1.6.jar -compress-lzf-1.0.3.jar -core-1.1.2.jar -curator-client-2.13.0.jar -curator-framework-2.13.0.jar -curator-recipes-2.13.0.jar -datanucleus-api-jdo-4.2.4.jar -datanucleus-core-4.1.17.jar -datanucleus-rdbms-4.1.19.jar -derby-10.12.1.1.jar -dnsjava-2.1.7.jar -dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar -ehcache-3.3.1.jar -flatbuffers-java-1.9.0.jar -generex-1.0.2.jar -geronimo-jcache_1.0_spec-1.0-alpha-1.jar -gson-2.2.4.jar -guava-14.0.1.jar -guice-4.0.jar -guice-servlet-4.0.jar -hadoop-annotations-3.2.0.jar -hadoop-auth-3.2.0.jar -hadoop-client-3.2.0.jar -hadoop-common-3.2.0.jar -hadoop-hdfs-client-3.2.0.jar -hadoop-mapreduce-client-common-3.2.0.jar -hadoop-mapreduce-client-core-3.2.0.jar -hadoop-mapreduce-client-jobclient-3.2.0.jar -hadoop-yarn-api-3.2.0.jar -hadoop-yarn-client-3.2.0.jar -hadoop-yarn-common-3.2.0.jar -hadoop-yarn-registry-3.2.0.jar -hadoop-yarn-server-common-3.2.0.jar -hadoop-yarn-server-web-proxy-3.2.0.jar -hive-beeline-2.3.6.jar -hive-cli-2.3.6.jar -hive-common-2.3.6.jar -hive-exec-2.3.6-core.jar -hive-jdbc-2.3.6.jar -hive-llap-common-2.3.6.jar -hive-metastore-2.3.6.jar -hive-serde-2.3.6.jar -hive-service-rpc-2.3.6.jar -hive-shims-0.23-2.3.6.jar -hive-shims-2.3.6.jar -hive-shims-common-2.3.6.jar -hive-shims-scheduler-2.3.6.jar -hive-storage-api-2.6.0.jar -hive-vector-code-gen-2.3.6.jar -hk2-api-2.6.1.jar -hk2-locator-2.6.1.jar -hk2-utils-2.6.1.jar -htrace-core4-4.1.0-incubating.jar -httpclient-4.5.6.jar -httpcore-4.4.12.jar -istack-commons-runtime-3.0.8.jar -ivy-2.4.0.jar -jackson-annotations-2.10.0.jar -jackson-core-2.10.0.jar -jackson-core-asl-1.9.13.jar -jackson-databind-2.10.0.jar -jackson-dataformat-yaml-2.10.0.jar -jackson-jaxrs-base-2.9.5.jar -jackson-jaxrs-json-provider-2.9.5.jar -jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.10.0.jar -jackson-module-paranamer-2.10.0.jar -jackson-module-scala_2.12-2.10.0.jar -jakarta.activation-api-1.2.1.jar -jakarta.annotation-api-1.3.5.jar -jakarta.inject-2.6.1.jar -jakarta.validation-api-2.0.2.jar -jakarta.ws.rs-api-2.1.6.jar -jakarta.xml.bind-api-2.3.2.jar -janino-3.0.15.jar -javassist-3.22.0-CR2.jar -javax.inject-1.jar -javax.jdo-3.2.0-m3.jar -javax.servlet-api-3.1.0.jar -javolution-5.5.1.jar -jaxb-api-2.2.11.jar -jaxb-runtime-2.3.2.jar -jcip-annotations-1.0-1.jar -jcl-over-slf4j-1.7.16.jar -jdo-api-3.0.1.jar -jersey-client-2.29.1.jar -jersey-common-2.29.1.jar -jersey-container-servlet-2.29.1.jar -jersey-container-servlet-core-2.29.1.jar -jersey-hk2-2.29.1.jar -jersey-media-jaxb-2.29.1.jar -jersey-server-2.29.1.jar -jline-2.14.6.jar -joda-time-2.10.5.jar -jodd-core-3.5.2.jar -jpam-1.1.jar -json-1.8.jar -json-smart-2.3.jar -json4s-ast_2.12-3.6.6.jar -json4s-core_2.12-3.6.6.jar -json4s-jackson_2.12-3.6.6.jar -json4s-scalap_2.12-3.6.6.jar -jsp-api-2.1.jar -jsr305-3.0.0.jar -jta-1.1.jar -jul-to-slf4j-1.7.16.jar -kerb-admin-1.0.1.jar -kerb-client-1.0.1.jar -kerb-common-1.0.1.jar -kerb-core-1.0.1.jar -kerb-crypto-1.0.1.jar -kerb-identity-1.0.1.jar -kerb-server-1.0.1.jar -kerb-simplekdc-1.0.1.jar -kerb-util-1.0.1.jar -kerby-asn1-1.0.1.jar -kerby-config-1.0.1.jar -kerby-pkix-1.0.1.jar -kerby-util-1.0.1.jar -kerby-xdr-1.0.1.jar -kryo-shaded-4.0.2.jar -kubernetes-client-4.6.4.jar -kubernetes-model-4.6.4.jar -kubernetes-model-common-4.6.4.jar -leveldbjni-all-1.8.jar -libfb303-0.9.3.jar -libthrift-0.12.0.jar -log4j-1.2.17.jar -logging-interceptor-3.12.6.jar -lz4-java-1.7.0.jar -machinist_2.12-0.6.8.jar -macro-compat_2.12-1.1.1.jar -mesos-1.4.0-shaded-protobuf.jar -metrics-core-4.1.1.jar -metrics-graphite-4.1.1.jar -metrics-jmx-4.1.1.jar -metrics-json-4.1.1.jar -metrics-jvm-4.1.1.jar -minlog-1.3.0.jar -mssql-jdbc-6.2.1.jre7.jar -netty-all-4.1.42.Final.jar -nimbus-jose-jwt-4.41.1.jar -objenesis-2.5.1.jar -okhttp-2.7.5.jar -okhttp-3.12.6.jar -okio-1.15.0.jar -opencsv-2.3.jar -orc-core-1.5.8.jar -orc-mapreduce-1.5.8.jar -orc-shims-1.5.8.jar -oro-2.0.8.jar -osgi-resource-locator-1.0.3.jar -paranamer-2.8.jar -parquet-column-1.10.1.jar -parquet-common-1.10.1.jar -parquet-encoding-1.10.1.jar -parquet-format-2.4.0.jar -parquet-hadoop-1.10.1.jar -parquet-jackson-1.10.1.jar -protobuf-java-2.5.0.jar -py4j-0.10.8.1.jar -pyrolite-4.30.jar -re2j-1.1.jar -scala-collection-compat_2.12-2.1.1.jar -scala-compiler-2.12.10.jar -scala-library-2.12.10.jar -scala-parser-combinators_2.12-1.1.2.jar -scala-reflect-2.12.10.jar -scala-xml_2.12-1.2.0.jar -shapeless_2.12-2.3.3.jar -shims-0.7.45.jar -slf4j-api-1.7.16.jar -slf4j-log4j12-1.7.16.jar -snakeyaml-1.24.jar -snappy-java-1.1.7.3.jar -spire-macros_2.12-0.17.0-M1.jar -spire-platform_2.12-0.17.0-M1.jar -spire-util_2.12-0.17.0-M1.jar -spire_2.12-0.17.0-M1.jar -stax-api-1.0.1.jar -stax2-api-3.1.4.jar -stream-2.9.6.jar -super-csv-2.2.0.jar -token-provider-1.0.1.jar -transaction-api-1.1.jar -univocity-parsers-2.8.3.jar -velocity-1.5.jar -woodstox-core-5.0.3.jar -xbean-asm7-shaded-4.15.jar -xz-1.5.jar -zjsonpatch-0.3.0.jar -zookeeper-3.4.14.jar -zstd-jni-1.4.4-3.jar +HikariCP/2.5.1//HikariCP-2.5.1.jar +JLargeArrays/1.5//JLargeArrays-1.5.jar +JTransforms/3.1//JTransforms-3.1.jar +RoaringBitmap/0.7.45//RoaringBitmap-0.7.45.jar +ST4/4.0.4//ST4-4.0.4.jar +accessors-smart/1.2//accessors-smart-1.2.jar +activation/1.1.1//activation-1.1.1.jar +aircompressor/0.10//aircompressor-0.10.jar +algebra_2.12/2.0.0-M2//algebra_2.12-2.0.0-M2.jar +antlr-runtime/3.5.2//antlr-runtime-3.5.2.jar +antlr4-runtime/4.7.1//antlr4-runtime-4.7.1.jar +aopalliance-repackaged/2.6.1//aopalliance-repackaged-2.6.1.jar +aopalliance/1.0//aopalliance-1.0.jar +arpack_combined_all/0.1//arpack_combined_all-0.1.jar +arrow-format/0.15.1//arrow-format-0.15.1.jar +arrow-memory/0.15.1//arrow-memory-0.15.1.jar +arrow-vector/0.15.1//arrow-vector-0.15.1.jar +audience-annotations/0.5.0//audience-annotations-0.5.0.jar +automaton/1.11-8//automaton-1.11-8.jar +avro-ipc/1.8.2//avro-ipc-1.8.2.jar +avro-mapred/1.8.2/hadoop2/avro-mapred-1.8.2-hadoop2.jar +avro/1.8.2//avro-1.8.2.jar +bonecp/0.8.0.RELEASE//bonecp-0.8.0.RELEASE.jar +breeze-macros_2.12/1.0//breeze-macros_2.12-1.0.jar +breeze_2.12/1.0//breeze_2.12-1.0.jar +cats-kernel_2.12/2.0.0-M4//cats-kernel_2.12-2.0.0-M4.jar +chill-java/0.9.3//chill-java-0.9.3.jar +chill_2.12/0.9.3//chill_2.12-0.9.3.jar +commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar +commons-cli/1.2//commons-cli-1.2.jar +commons-codec/1.10//commons-codec-1.10.jar +commons-collections/3.2.2//commons-collections-3.2.2.jar +commons-compiler/3.0.15//commons-compiler-3.0.15.jar +commons-compress/1.8.1//commons-compress-1.8.1.jar +commons-configuration2/2.1.1//commons-configuration2-2.1.1.jar +commons-crypto/1.0.0//commons-crypto-1.0.0.jar +commons-daemon/1.0.13//commons-daemon-1.0.13.jar +commons-dbcp/1.4//commons-dbcp-1.4.jar +commons-httpclient/3.1//commons-httpclient-3.1.jar +commons-io/2.4//commons-io-2.4.jar +commons-lang/2.6//commons-lang-2.6.jar +commons-lang3/3.9//commons-lang3-3.9.jar +commons-logging/1.1.3//commons-logging-1.1.3.jar +commons-math3/3.4.1//commons-math3-3.4.1.jar +commons-net/3.1//commons-net-3.1.jar +commons-pool/1.5.4//commons-pool-1.5.4.jar +commons-text/1.6//commons-text-1.6.jar +compress-lzf/1.0.3//compress-lzf-1.0.3.jar +core/1.1.2//core-1.1.2.jar +curator-client/2.13.0//curator-client-2.13.0.jar +curator-framework/2.13.0//curator-framework-2.13.0.jar +curator-recipes/2.13.0//curator-recipes-2.13.0.jar +datanucleus-api-jdo/4.2.4//datanucleus-api-jdo-4.2.4.jar +datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar +datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar +derby/10.12.1.1//derby-10.12.1.1.jar +dnsjava/2.1.7//dnsjava-2.1.7.jar +dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar +ehcache/3.3.1//ehcache-3.3.1.jar +flatbuffers-java/1.9.0//flatbuffers-java-1.9.0.jar +generex/1.0.2//generex-1.0.2.jar +geronimo-jcache_1.0_spec/1.0-alpha-1//geronimo-jcache_1.0_spec-1.0-alpha-1.jar +gson/2.2.4//gson-2.2.4.jar +guava/14.0.1//guava-14.0.1.jar +guice-servlet/4.0//guice-servlet-4.0.jar +guice/4.0//guice-4.0.jar +hadoop-annotations/3.2.0//hadoop-annotations-3.2.0.jar +hadoop-auth/3.2.0//hadoop-auth-3.2.0.jar +hadoop-client/3.2.0//hadoop-client-3.2.0.jar +hadoop-common/3.2.0//hadoop-common-3.2.0.jar +hadoop-hdfs-client/3.2.0//hadoop-hdfs-client-3.2.0.jar +hadoop-mapreduce-client-common/3.2.0//hadoop-mapreduce-client-common-3.2.0.jar +hadoop-mapreduce-client-core/3.2.0//hadoop-mapreduce-client-core-3.2.0.jar +hadoop-mapreduce-client-jobclient/3.2.0//hadoop-mapreduce-client-jobclient-3.2.0.jar +hadoop-yarn-api/3.2.0//hadoop-yarn-api-3.2.0.jar +hadoop-yarn-client/3.2.0//hadoop-yarn-client-3.2.0.jar +hadoop-yarn-common/3.2.0//hadoop-yarn-common-3.2.0.jar +hadoop-yarn-registry/3.2.0//hadoop-yarn-registry-3.2.0.jar +hadoop-yarn-server-common/3.2.0//hadoop-yarn-server-common-3.2.0.jar +hadoop-yarn-server-web-proxy/3.2.0//hadoop-yarn-server-web-proxy-3.2.0.jar +hive-beeline/2.3.6//hive-beeline-2.3.6.jar +hive-cli/2.3.6//hive-cli-2.3.6.jar +hive-common/2.3.6//hive-common-2.3.6.jar +hive-exec/2.3.6/core/hive-exec-2.3.6-core.jar +hive-jdbc/2.3.6//hive-jdbc-2.3.6.jar +hive-llap-common/2.3.6//hive-llap-common-2.3.6.jar +hive-metastore/2.3.6//hive-metastore-2.3.6.jar +hive-serde/2.3.6//hive-serde-2.3.6.jar +hive-service-rpc/2.3.6//hive-service-rpc-2.3.6.jar +hive-shims-0.23/2.3.6//hive-shims-0.23-2.3.6.jar +hive-shims-common/2.3.6//hive-shims-common-2.3.6.jar +hive-shims-scheduler/2.3.6//hive-shims-scheduler-2.3.6.jar +hive-shims/2.3.6//hive-shims-2.3.6.jar +hive-storage-api/2.6.0//hive-storage-api-2.6.0.jar +hive-vector-code-gen/2.3.6//hive-vector-code-gen-2.3.6.jar +hk2-api/2.6.1//hk2-api-2.6.1.jar +hk2-locator/2.6.1//hk2-locator-2.6.1.jar +hk2-utils/2.6.1//hk2-utils-2.6.1.jar +htrace-core4/4.1.0-incubating//htrace-core4-4.1.0-incubating.jar +httpclient/4.5.6//httpclient-4.5.6.jar +httpcore/4.4.12//httpcore-4.4.12.jar +istack-commons-runtime/3.0.8//istack-commons-runtime-3.0.8.jar +ivy/2.4.0//ivy-2.4.0.jar +jackson-annotations/2.10.0//jackson-annotations-2.10.0.jar +jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar +jackson-core/2.10.0//jackson-core-2.10.0.jar +jackson-databind/2.10.0//jackson-databind-2.10.0.jar +jackson-dataformat-yaml/2.10.0//jackson-dataformat-yaml-2.10.0.jar +jackson-jaxrs-base/2.9.5//jackson-jaxrs-base-2.9.5.jar +jackson-jaxrs-json-provider/2.9.5//jackson-jaxrs-json-provider-2.9.5.jar +jackson-mapper-asl/1.9.13//jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations/2.10.0//jackson-module-jaxb-annotations-2.10.0.jar +jackson-module-paranamer/2.10.0//jackson-module-paranamer-2.10.0.jar +jackson-module-scala_2.12/2.10.0//jackson-module-scala_2.12-2.10.0.jar +jakarta.activation-api/1.2.1//jakarta.activation-api-1.2.1.jar +jakarta.annotation-api/1.3.5//jakarta.annotation-api-1.3.5.jar +jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar +jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar +jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar +jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar +janino/3.0.15//janino-3.0.15.jar +javassist/3.22.0-CR2//javassist-3.22.0-CR2.jar +javax.inject/1//javax.inject-1.jar +javax.jdo/3.2.0-m3//javax.jdo-3.2.0-m3.jar +javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar +javolution/5.5.1//javolution-5.5.1.jar +jaxb-api/2.2.11//jaxb-api-2.2.11.jar +jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar +jcip-annotations/1.0-1//jcip-annotations-1.0-1.jar +jcl-over-slf4j/1.7.16//jcl-over-slf4j-1.7.16.jar +jdo-api/3.0.1//jdo-api-3.0.1.jar +jersey-client/2.29.1//jersey-client-2.29.1.jar +jersey-common/2.29.1//jersey-common-2.29.1.jar +jersey-container-servlet-core/2.29.1//jersey-container-servlet-core-2.29.1.jar +jersey-container-servlet/2.29.1//jersey-container-servlet-2.29.1.jar +jersey-hk2/2.29.1//jersey-hk2-2.29.1.jar +jersey-media-jaxb/2.29.1//jersey-media-jaxb-2.29.1.jar +jersey-server/2.29.1//jersey-server-2.29.1.jar +jline/2.14.6//jline-2.14.6.jar +joda-time/2.10.5//joda-time-2.10.5.jar +jodd-core/3.5.2//jodd-core-3.5.2.jar +jpam/1.1//jpam-1.1.jar +json-smart/2.3//json-smart-2.3.jar +json/1.8//json-1.8.jar +json4s-ast_2.12/3.6.6//json4s-ast_2.12-3.6.6.jar +json4s-core_2.12/3.6.6//json4s-core_2.12-3.6.6.jar +json4s-jackson_2.12/3.6.6//json4s-jackson_2.12-3.6.6.jar +json4s-scalap_2.12/3.6.6//json4s-scalap_2.12-3.6.6.jar +jsp-api/2.1//jsp-api-2.1.jar +jsr305/3.0.0//jsr305-3.0.0.jar +jta/1.1//jta-1.1.jar +jul-to-slf4j/1.7.16//jul-to-slf4j-1.7.16.jar +kerb-admin/1.0.1//kerb-admin-1.0.1.jar +kerb-client/1.0.1//kerb-client-1.0.1.jar +kerb-common/1.0.1//kerb-common-1.0.1.jar +kerb-core/1.0.1//kerb-core-1.0.1.jar +kerb-crypto/1.0.1//kerb-crypto-1.0.1.jar +kerb-identity/1.0.1//kerb-identity-1.0.1.jar +kerb-server/1.0.1//kerb-server-1.0.1.jar +kerb-simplekdc/1.0.1//kerb-simplekdc-1.0.1.jar +kerb-util/1.0.1//kerb-util-1.0.1.jar +kerby-asn1/1.0.1//kerby-asn1-1.0.1.jar +kerby-config/1.0.1//kerby-config-1.0.1.jar +kerby-pkix/1.0.1//kerby-pkix-1.0.1.jar +kerby-util/1.0.1//kerby-util-1.0.1.jar +kerby-xdr/1.0.1//kerby-xdr-1.0.1.jar +kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar +kubernetes-client/4.6.4//kubernetes-client-4.6.4.jar +kubernetes-model-common/4.6.4//kubernetes-model-common-4.6.4.jar +kubernetes-model/4.6.4//kubernetes-model-4.6.4.jar +leveldbjni-all/1.8//leveldbjni-all-1.8.jar +libfb303/0.9.3//libfb303-0.9.3.jar +libthrift/0.12.0//libthrift-0.12.0.jar +log4j/1.2.17//log4j-1.2.17.jar +logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar +lz4-java/1.7.0//lz4-java-1.7.0.jar +machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar +macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar +mesos/1.4.0/shaded-protobuf/mesos-1.4.0-shaded-protobuf.jar +metrics-core/4.1.1//metrics-core-4.1.1.jar +metrics-graphite/4.1.1//metrics-graphite-4.1.1.jar +metrics-jmx/4.1.1//metrics-jmx-4.1.1.jar +metrics-json/4.1.1//metrics-json-4.1.1.jar +metrics-jvm/4.1.1//metrics-jvm-4.1.1.jar +minlog/1.3.0//minlog-1.3.0.jar +mssql-jdbc/6.2.1.jre7//mssql-jdbc-6.2.1.jre7.jar +netty-all/4.1.42.Final//netty-all-4.1.42.Final.jar +nimbus-jose-jwt/4.41.1//nimbus-jose-jwt-4.41.1.jar +objenesis/2.5.1//objenesis-2.5.1.jar +okhttp/2.7.5//okhttp-2.7.5.jar +okhttp/3.12.6//okhttp-3.12.6.jar +okio/1.15.0//okio-1.15.0.jar +opencsv/2.3//opencsv-2.3.jar +orc-core/1.5.8//orc-core-1.5.8.jar +orc-mapreduce/1.5.8//orc-mapreduce-1.5.8.jar +orc-shims/1.5.8//orc-shims-1.5.8.jar +oro/2.0.8//oro-2.0.8.jar +osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar +paranamer/2.8//paranamer-2.8.jar +parquet-column/1.10.1//parquet-column-1.10.1.jar +parquet-common/1.10.1//parquet-common-1.10.1.jar +parquet-encoding/1.10.1//parquet-encoding-1.10.1.jar +parquet-format/2.4.0//parquet-format-2.4.0.jar +parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar +parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar +protobuf-java/2.5.0//protobuf-java-2.5.0.jar +py4j/0.10.8.1//py4j-0.10.8.1.jar +pyrolite/4.30//pyrolite-4.30.jar +re2j/1.1//re2j-1.1.jar +scala-collection-compat_2.12/2.1.1//scala-collection-compat_2.12-2.1.1.jar +scala-compiler/2.12.10//scala-compiler-2.12.10.jar +scala-library/2.12.10//scala-library-2.12.10.jar +scala-parser-combinators_2.12/1.1.2//scala-parser-combinators_2.12-1.1.2.jar +scala-reflect/2.12.10//scala-reflect-2.12.10.jar +scala-xml_2.12/1.2.0//scala-xml_2.12-1.2.0.jar +shapeless_2.12/2.3.3//shapeless_2.12-2.3.3.jar +shims/0.7.45//shims-0.7.45.jar +slf4j-api/1.7.16//slf4j-api-1.7.16.jar +slf4j-log4j12/1.7.16//slf4j-log4j12-1.7.16.jar +snakeyaml/1.24//snakeyaml-1.24.jar +snappy-java/1.1.7.3//snappy-java-1.1.7.3.jar +spire-macros_2.12/0.17.0-M1//spire-macros_2.12-0.17.0-M1.jar +spire-platform_2.12/0.17.0-M1//spire-platform_2.12-0.17.0-M1.jar +spire-util_2.12/0.17.0-M1//spire-util_2.12-0.17.0-M1.jar +spire_2.12/0.17.0-M1//spire_2.12-0.17.0-M1.jar +stax-api/1.0.1//stax-api-1.0.1.jar +stax2-api/3.1.4//stax2-api-3.1.4.jar +stream/2.9.6//stream-2.9.6.jar +super-csv/2.2.0//super-csv-2.2.0.jar +token-provider/1.0.1//token-provider-1.0.1.jar +transaction-api/1.1//transaction-api-1.1.jar +univocity-parsers/2.8.3//univocity-parsers-2.8.3.jar +velocity/1.5//velocity-1.5.jar +woodstox-core/5.0.3//woodstox-core-5.0.3.jar +xbean-asm7-shaded/4.15//xbean-asm7-shaded-4.15.jar +xz/1.5//xz-1.5.jar +zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar +zookeeper/3.4.14//zookeeper-3.4.14.jar +zstd-jni/1.4.4-3//zstd-jni-1.4.4-3.jar diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index ad5cc75d72d10..936ac00f6b9e7 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -88,8 +88,24 @@ for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do mkdir -p dev/pr-deps $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE -P$HIVE_PROFILE dependency:build-classpath -pl assembly -am \ | grep "Dependencies classpath:" -A 1 \ - | tail -n 1 | tr ":" "\n" | rev | cut -d "/" -f 1 | rev | sort \ - | grep -v spark > dev/pr-deps/spark-deps-$HADOOP_HIVE_PROFILE + | tail -n 1 | tr ":" "\n" | awk -F '/' '{ + # For each dependency classpath, we fetch the last three parts split by "/": artifact id, version, and jar name. + # Since classifier, if exists, always sits between "artifact_id-version-" and ".jar" suffix in the jar name, + # we extract classifier and put it right before the jar name explicitly. + # For example, `orc-core/1.5.5/nohive/orc-core-1.5.5-nohive.jar` + # ^^^^^^ + # extracted classifier + # `okio/1.15.0//okio-1.15.0.jar` + # ^ + # empty for dependencies without classifier + artifact_id=$(NF-2); + version=$(NF-1); + jar_name=$NF; + classifier_start_index=length(artifact_id"-"version"-") + 1; + classifier_end_index=index(jar_name, ".jar") - 1; + classifier=substr(jar_name, classifier_start_index, classifier_end_index - classifier_start_index + 1); + print artifact_id"/"version"/"classifier"/"jar_name + }' | sort | grep -v spark > dev/pr-deps/spark-deps-$HADOOP_HIVE_PROFILE done if [[ $@ == **replace-manifest** ]]; then From 018bdcc53c925072b07956de0600452ad255b9c7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 16 Jan 2020 13:36:28 +0900 Subject: [PATCH 1125/1387] [SPARK-30521][SQL][TESTS] Eliminate deprecation warnings for ExpressionInfo ### What changes were proposed in this pull request? In the PR, I propose to use non-deprecated constructor of `ExpressionInfo` in `SparkSessionExtensionSuite`, and pass valid strings as `examples`, `note`, `since` and `deprecated` parameters. ### Why are the changes needed? Using another constructor allows to eliminate the following deprecation warnings while compiling Spark: ``` Warning:(335, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information. new ExpressionInfo("noClass", "myDb", "myFunction", "usage", "extended usage"), Warning:(732, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information. new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"), Warning:(751, 5) constructor ExpressionInfo in class ExpressionInfo is deprecated: see corresponding Javadoc for more information. new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"), ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? By compiling and running `SparkSessionExtensionSuite`. Closes #27221 from MaxGekk/eliminate-expr-info-warnings. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../sql/SparkSessionExtensionSuite.scala | 45 +++++++++++++++++-- 1 file changed, 42 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 8c204bc47c6c7..99ea95089d71c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -332,7 +332,20 @@ case class MyParser(spark: SparkSession, delegate: ParserInterface) extends Pars object MyExtensions { val myFunction = (FunctionIdentifier("myFunction"), - new ExpressionInfo("noClass", "myDb", "myFunction", "usage", "extended usage"), + new ExpressionInfo( + "noClass", + "myDb", + "myFunction", + "usage", + "extended usage", + " Examples:", + """ + note + """, + "3.0.0", + """ + deprecated + """), (_: Seq[Expression]) => Literal(5, IntegerType)) } @@ -729,7 +742,20 @@ case class MySparkStrategy2(spark: SparkSession) extends SparkStrategy { object MyExtensions2 { val myFunction = (FunctionIdentifier("myFunction2"), - new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"), + new ExpressionInfo( + "noClass", + "myDb", + "myFunction2", + "usage", + "extended usage", + " Examples:", + """ + note + """, + "3.0.0", + """ + deprecated + """), (_: Seq[Expression]) => Literal(5, IntegerType)) } @@ -748,7 +774,20 @@ class MyExtensions2 extends (SparkSessionExtensions => Unit) { object MyExtensions2Duplicate { val myFunction = (FunctionIdentifier("myFunction2"), - new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"), + new ExpressionInfo( + "noClass", + "myDb", + "myFunction2", + "usage", + "extended usage", + " Examples:", + """ + note + """, + "3.0.0", + """ + deprecated + """), (_: Seq[Expression]) => Literal(5, IntegerType)) } From 1ef1d6caf259da3bc417444f63cb6cc782bc8cb9 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 16 Jan 2020 19:23:10 +0800 Subject: [PATCH 1126/1387] [SPARK-29565][FOLLOWUP] add setInputCol/setOutputCol in OHEModel ### What changes were proposed in this pull request? add setInputCol/setOutputCol in OHEModel ### Why are the changes needed? setInputCol/setOutputCol should be in OHEModel too. ### Does this PR introduce any user-facing change? Yes. ```OHEModel.setInputCol``` ```OHEModel.setOutputCol``` ### How was this patch tested? Manually tested. Closes #27228 from huaxingao/spark-29565. Authored-by: Huaxin Gao Signed-off-by: zhengruifeng --- .../apache/spark/ml/feature/OneHotEncoder.scala | 8 ++++++++ python/pyspark/ml/feature.py | 14 ++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 6ba7054688061..0ef092f6be463 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -287,6 +287,14 @@ class OneHotEncoderModel private[ml] ( } } + /** @group setParam */ + @Since("3.0.0") + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + @Since("3.0.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ @Since("3.0.0") def setInputCols(values: Array[String]): this.type = set(inputCols, values) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index dc6cc7d674179..f9206e957796a 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2541,6 +2541,20 @@ def setOutputCols(self, value): """ return self._set(outputCols=value) + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @since("3.0.0") def setHandleInvalid(self, value): """ From 82f25f585539900d8405d3bdd3b0054608b60047 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 16 Jan 2020 21:46:07 +0800 Subject: [PATCH 1127/1387] [SPARK-30507][SQL] TableCalalog reserved properties shoudn't be changed via options or tblpropeties ### What changes were proposed in this pull request? TableCatalog reserves some properties, e,g `provider`, `location` for internal usage. Some of them are static once create, some of them need specific syntax to modify. Instead of using `OPTIONS (k='v')` or TBLPROPERTIES (k='v'), if k is a reserved TableCatalog property, we should use its specific syntax to add/modify/delete it. e.g. `provider` is a reserved property, we should use the `USING` clause to specify it, and should not allow `ALTER TABLE ... UNSET TBLPROPERTIES('provider')` to delete it. Also, there are two paths for v1/v2 catalog tables to resolve these properties, e.g. the v1 session catalog tables will only use the `USING` clause to decide `provider` but v2 tables will also lookup OPTION/TBLPROPERTIES(although there is a bug prohibit it). Additionally, 'path' is not reserved but holds special meaning for `LOCATION` and it is used in `CREATE/REPLACE TABLE`'s `OPTIONS` sub-clause. Now for the session catalog tables, the `path` is case-insensitive, but for the non-session catalog tables, it is case-sensitive, we should make it both case insensitive for disambiguation. ### Why are the changes needed? prevent reserved properties from being modified unexpectedly unify the property resolution for v1/v2. fix some bugs. ### Does this PR introduce any user-facing change? yes 1 . `location` and `provider` (case sensitive) cannot be used in `CREATE/REPLACE TABLE ... OPTIONS/TBLPROPETIES` and `ALTER TABLE ... SET TBLPROPERTIES (...)`, if legacy on, they will be ignored to let the command success without having side effects 3. Once `path` in `CREATE/REPLACE TABLE ... OPTIONS` is case insensitive for v1 but sensitive for v2, but now we change it case insensitive for both kinds of tables, then v2 tables will also fail if `LOCATION` and `OPTIONS('PaTh' ='abc')` are both specified or will pick `PaTh`'s value as table location if `LOCATION` is missing. 4. Now we will detect if there are two different case `path` keys or more in `CREATE/REPLACE TABLE ... OPTIONS`, once it is a kind of unexpected last-win policy for v1, and v2 is case sensitive. ### How was this patch tested? add ut Closes #27197 from yaooqinn/SPARK-30507. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 74 ++++++++++++++- .../catalyst/analysis/ResolveCatalogs.scala | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 84 +++++++++++----- .../sql/connector/catalog/CatalogV2Util.scala | 41 +------- .../analysis/ResolveSessionCatalog.scala | 21 ++-- .../spark/sql/execution/command/ddl.scala | 7 +- .../sql/connector/DataSourceV2SQLSuite.scala | 95 +++++++++++++++++-- 7 files changed, 242 insertions(+), 84 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 120b115721803..bb086ff657e41 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -262,7 +262,79 @@ license: | - Since Spark 3.0, the function `percentile_approx` and its alias `approx_percentile` only accept integral value with range in `[1, 2147483647]` as its 3rd argument `accuracy`, fractional and string types are disallowed, e.g. `percentile_approx(10.0, 0.2, 1.8D)` will cause `AnalysisException`. In Spark version 2.4 and earlier, if `accuracy` is fractional or string value, it will be coerced to an int value, `percentile_approx(10.0, 0.2, 1.8D)` is operated as `percentile_approx(10.0, 0.2, 1)` which results in `10.0`. - - Since Spark 3.0, `location` and `comment` become reserved database properties, Commands will fail if we specify reserved properties in `CREATE DATABASE ... WITH DBPROPERTIES` and `ALTER DATABASE ... SET DBPROPERTIES`. We need their specific clauses to specify them, e.g. `CREATE DATABASE test COMMENT 'any comment' LOCATION 'some path'`. We can set `spark.sql.legacy.property.nonReserved` to `true` to ignore the `ParseException`, in this case, these properties will be silently removed, e.g `SET DBPROTERTIES('location'='/tmp')` will affect nothing. In Spark version 2.4 and earlier, these properties are neither reserved nor have side effects, e.g. `SET DBPROTERTIES('location'='/tmp')` will not change the location of the database but only create a headless property just like `'a'='b'`. + - Since Spark 3.0, the properties listing below become reserved, commands will fail if we specify reserved properties in places like `CREATE DATABASE ... WITH DBPROPERTIES` and `ALTER TABLE ... SET TBLPROPERTIES`. We need their specific clauses to specify them, e.g. `CREATE DATABASE test COMMENT 'any comment' LOCATION 'some path'`. We can set `spark.sql.legacy.property.nonReserved` to `true` to ignore the `ParseException`, in this case, these properties will be silently removed, e.g `SET DBPROTERTIES('location'='/tmp')` will affect nothing. In Spark version 2.4 and earlier, these properties are neither reserved nor have side effects, e.g. `SET DBPROTERTIES('location'='/tmp')` will not change the location of the database but only create a headless property just like `'a'='b'`. +
      latest batch queryThe end point when a batch query is ended, a json string specifying an ending timesamp for each TopicPartition. + The end point when a batch query is ended, a json string specifying an ending timestamp for each TopicPartition. The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to the given timestamp in the corresponding partition. If the matched offset doesn't exist, the offset will be set to latest.

      diff --git a/docs/web-ui.md b/docs/web-ui.md index f94e81ca67961..6f1afb2ed46f5 100644 --- a/docs/web-ui.md +++ b/docs/web-ui.md @@ -444,7 +444,7 @@ The third section has the SQL statistics of the submitted operations. * _Canceled_, final state when the execution is canceled. * _Finished_ processing and waiting to fetch results. * _Closed_, final state when client closed the statement. -* **Detail** of the execution plan with parsed logical plan, analyzed logical plan, optimized logical plan and physical plan or errors in the the SQL statement. +* **Detail** of the execution plan with parsed logical plan, analyzed logical plan, optimized logical plan and physical plan or errors in the SQL statement.

      JDBC/ODBC SQL Statistics diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala index 61ffe31edfd04..ead45423d673f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap /** - * Class to calculate offset ranges to process based on the the from and until offsets, and + * Class to calculate offset ranges to process based on the from and until offsets, and * the configured `minPartitions`. */ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int]) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala index 6c194902a750b..a9c2941ef3a53 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala @@ -148,7 +148,7 @@ private[spark] object DecisionTreeMetadata extends Logging { require(maxCategoriesPerFeature <= maxPossibleBins, s"DecisionTree requires maxBins (= $maxPossibleBins) to be at least as large as the " + s"number of values in each categorical feature, but categorical feature $maxCategory " + - s"has $maxCategoriesPerFeature values. Considering remove this and other categorical " + + s"has $maxCategoriesPerFeature values. Consider removing this and other categorical " + "features with a large number of values, or add more training examples.") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala index 55a5bd380859e..39a16e917c4a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedUnsafeProjection.scala @@ -132,7 +132,7 @@ object InterpretedUnsafeProjection { dt: DataType, nullable: Boolean): (SpecializedGetters, Int) => Unit = { - // Create the the basic writer. + // Create the basic writer. val unsafeWriter: (SpecializedGetters, Int) => Unit = dt match { case BooleanType => (v, i) => writer.write(i, v.getBoolean(i)) 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 05fd5e35e22ad..810c28116de47 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 @@ -406,7 +406,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { // Create the attribute mapping. Note that the currentNextAttrPairs can contain duplicate // keys in case of Union (this is caused by the PushProjectionThroughUnion rule); in this - // case we use the the first mapping (which should be provided by the first child). + // case we use the first mapping (which should be provided by the first child). val mapping = AttributeMap(currentNextAttrPairs) // Create a an expression cleaning function for nodes that can actually produce redundant diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelperSuite.scala index 9100e10ca0c09..0a3f86ebf6808 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelperSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelperSuite.scala @@ -118,7 +118,7 @@ class AnalysisHelperSuite extends SparkFunSuite { test("do not allow transform in analyzer") { val plan = Project(Nil, LocalRelation()) - // These should be OK since we are not in the analzyer + // These should be OK since we are not in the analyzer plan.transform { case p: Project => p } plan.transformUp { case p: Project => p } plan.transformDown { case p: Project => p } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala index 1f325c11c9e44..2ec99c4b9ade3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectAggregationIterator.scala @@ -158,7 +158,7 @@ class ObjectAggregationIterator( val buffer: InternalRow = getAggregationBufferByKey(hashMap, groupingKey) processRow(buffer, newInput) - // The the hash map gets too large, makes a sorted spill and clear the map. + // The hash map gets too large, makes a sorted spill and clear the map. if (hashMap.size >= fallbackCountThreshold) { logInfo( s"Aggregation hash map size ${hashMap.size} reaches threshold " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonForeachWriter.scala index a4e9b3305052f..2a799bab1eb81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonForeachWriter.scala @@ -78,7 +78,7 @@ object PythonForeachWriter { * * Internally, it uses a [[HybridRowQueue]] to buffer the rows in a practically unlimited queue * across memory and local disk. However, HybridRowQueue is designed to be used only with - * EvalPythonExec where the reader is always behind the the writer, that is, the reader does not + * EvalPythonExec where the reader is always behind the writer, that is, the reader does not * try to read n+1 rows if the writer has only written n rows at any point of time. This * assumption is not true for PythonForeachWriter where rows may be added at a different rate as * they are consumed by the python worker. Hence, to maintain the invariant of the reader being diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index f1bfe97610fed..3c45f22815db8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -110,7 +110,7 @@ import org.apache.spark.util.{CompletionIterator, SerializableConfiguration} * * 3. When both window in join key and time range conditions are present, case 1 + 2. * In this case, since window equality is a stricter condition than the time range, we can - * use the the State Key Watermark = event time watermark to discard state (similar to case 1). + * use the State Key Watermark = event time watermark to discard state (similar to case 1). * * @param leftKeys Expression to generate key rows for joining from left input * @param rightKeys Expression to generate key rows for joining from right input diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala index ad5c7cf24caf7..dc25289aa1e2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactor import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap -/** Common methods used to create writes for the the console sink */ +/** Common methods used to create writes for the console sink */ class ConsoleWrite(schema: StructType, options: CaseInsensitiveStringMap) extends StreamingWrite with Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala index ab68eba81b843..af08a53e465b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala @@ -93,7 +93,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState * any trigger and timeout function call will not occur until there is data. * - Since the processing time timeout is based on the clock time, it is affected by the * variations in the system clock (i.e. time zone changes, clock skew, etc.). - * - With `EventTimeTimeout`, the user also has to specify the the the event time watermark in + * - With `EventTimeTimeout`, the user also has to specify the event time watermark in * the query using `Dataset.withWatermark()`. With this setting, data that is older than the * watermark are filtered out. The timeout can be set for a group by setting a timeout timestamp * using`GroupState.setTimeoutTimestamp()`, and the timeout would occur when the watermark diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala index 4b6da5a02eac3..0734c6e18deea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala @@ -36,7 +36,7 @@ abstract class NestedSchemaPruningBenchmark extends SqlBasedBenchmark { // We use `col1 BIGINT, col2 STRUCT<_1: BIGINT, _2: STRING>, // col3 ARRAY>` as a test schema. - // col1, col2._1 and col3._1 are used for comparision. col2._2 and col3._2 mimics the burden + // col1, col2._1 and col3._1 are used for comparison. col2._2 and col3._2 mimics the burden // for the other columns private val df = spark .range(N * 10) From a38bf7e051e98058399529248750ff4eddfe2ade Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Sat, 21 Dec 2019 14:40:07 -0800 Subject: [PATCH 0936/1387] [SPARK-28083][SQL][TEST][FOLLOW-UP] Enable LIKE ... ESCAPE test cases ### What changes were proposed in this pull request? This PR is a follow-up to https://github.com/apache/spark/pull/25001 ### Why are the changes needed? No ### Does this PR introduce any user-facing change? No ### How was this patch tested? Pass the Jenkins with the newly update test files. Closes #26949 from beliefer/uncomment-like-escape-tests. Authored-by: gengjiaan Signed-off-by: Dongjoon Hyun --- .../sql-tests/inputs/postgreSQL/strings.sql | 61 +- .../results/postgreSQL/strings.sql.out | 594 +++++++++++++----- 2 files changed, 451 insertions(+), 204 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/strings.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/strings.sql index 05841af27dd2e..541ff0bdad745 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/strings.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/strings.sql @@ -270,55 +270,54 @@ SELECT 'indio' NOT LIKE 'in__o' AS `false`; SELECT 'indio' LIKE 'in_o' AS `false`; SELECT 'indio' NOT LIKE 'in_o' AS `true`; --- [SPARK-28083] ANSI SQL: LIKE predicate: ESCAPE clause -- unused escape character --- SELECT 'hawkeye' LIKE 'h%' ESCAPE '#' AS "true"; --- SELECT 'hawkeye' NOT LIKE 'h%' ESCAPE '#' AS "false"; +SELECT 'hawkeye' LIKE 'h%' ESCAPE '#' AS `true`; +SELECT 'hawkeye' NOT LIKE 'h%' ESCAPE '#' AS `false`; --- SELECT 'indio' LIKE 'ind_o' ESCAPE '$' AS "true"; --- SELECT 'indio' NOT LIKE 'ind_o' ESCAPE '$' AS "false"; +SELECT 'indio' LIKE 'ind_o' ESCAPE '$' AS `true`; +SELECT 'indio' NOT LIKE 'ind_o' ESCAPE '$' AS `false`; -- escape character -- E061-05 like predicate with escape clause --- SELECT 'h%' LIKE 'h#%' ESCAPE '#' AS "true"; --- SELECT 'h%' NOT LIKE 'h#%' ESCAPE '#' AS "false"; +SELECT 'h%' LIKE 'h#%' ESCAPE '#' AS `true`; +SELECT 'h%' NOT LIKE 'h#%' ESCAPE '#' AS `false`; --- SELECT 'h%wkeye' LIKE 'h#%' ESCAPE '#' AS "false"; --- SELECT 'h%wkeye' NOT LIKE 'h#%' ESCAPE '#' AS "true"; +SELECT 'h%wkeye' LIKE 'h#%' ESCAPE '#' AS `false`; +SELECT 'h%wkeye' NOT LIKE 'h#%' ESCAPE '#' AS `true`; --- SELECT 'h%wkeye' LIKE 'h#%%' ESCAPE '#' AS "true"; --- SELECT 'h%wkeye' NOT LIKE 'h#%%' ESCAPE '#' AS "false"; +SELECT 'h%wkeye' LIKE 'h#%%' ESCAPE '#' AS `true`; +SELECT 'h%wkeye' NOT LIKE 'h#%%' ESCAPE '#' AS `false`; --- SELECT 'h%awkeye' LIKE 'h#%a%k%e' ESCAPE '#' AS "true"; --- SELECT 'h%awkeye' NOT LIKE 'h#%a%k%e' ESCAPE '#' AS "false"; +SELECT 'h%awkeye' LIKE 'h#%a%k%e' ESCAPE '#' AS `true`; +SELECT 'h%awkeye' NOT LIKE 'h#%a%k%e' ESCAPE '#' AS `false`; --- SELECT 'indio' LIKE '_ndio' ESCAPE '$' AS "true"; --- SELECT 'indio' NOT LIKE '_ndio' ESCAPE '$' AS "false"; +SELECT 'indio' LIKE '_ndio' ESCAPE '$' AS `true`; +SELECT 'indio' NOT LIKE '_ndio' ESCAPE '$' AS `false`; --- SELECT 'i_dio' LIKE 'i$_d_o' ESCAPE '$' AS "true"; --- SELECT 'i_dio' NOT LIKE 'i$_d_o' ESCAPE '$' AS "false"; +SELECT 'i_dio' LIKE 'i$_d_o' ESCAPE '$' AS `true`; +SELECT 'i_dio' NOT LIKE 'i$_d_o' ESCAPE '$' AS `false`; --- SELECT 'i_dio' LIKE 'i$_nd_o' ESCAPE '$' AS "false"; --- SELECT 'i_dio' NOT LIKE 'i$_nd_o' ESCAPE '$' AS "true"; +SELECT 'i_dio' LIKE 'i$_nd_o' ESCAPE '$' AS `false`; +SELECT 'i_dio' NOT LIKE 'i$_nd_o' ESCAPE '$' AS `true`; --- SELECT 'i_dio' LIKE 'i$_d%o' ESCAPE '$' AS "true"; --- SELECT 'i_dio' NOT LIKE 'i$_d%o' ESCAPE '$' AS "false"; +SELECT 'i_dio' LIKE 'i$_d%o' ESCAPE '$' AS `true`; +SELECT 'i_dio' NOT LIKE 'i$_d%o' ESCAPE '$' AS `false`; -- escape character same as pattern character --- SELECT 'maca' LIKE 'm%aca' ESCAPE '%' AS "true"; --- SELECT 'maca' NOT LIKE 'm%aca' ESCAPE '%' AS "false"; +SELECT 'maca' LIKE 'm%aca' ESCAPE '%' AS `true`; +SELECT 'maca' NOT LIKE 'm%aca' ESCAPE '%' AS `false`; --- SELECT 'ma%a' LIKE 'm%a%%a' ESCAPE '%' AS "true"; --- SELECT 'ma%a' NOT LIKE 'm%a%%a' ESCAPE '%' AS "false"; +SELECT 'ma%a' LIKE 'm%a%%a' ESCAPE '%' AS `true`; +SELECT 'ma%a' NOT LIKE 'm%a%%a' ESCAPE '%' AS `false`; --- SELECT 'bear' LIKE 'b_ear' ESCAPE '_' AS "true"; --- SELECT 'bear' NOT LIKE 'b_ear' ESCAPE '_' AS "false"; +SELECT 'bear' LIKE 'b_ear' ESCAPE '_' AS `true`; +SELECT 'bear' NOT LIKE 'b_ear' ESCAPE '_' AS `false`; --- SELECT 'be_r' LIKE 'b_e__r' ESCAPE '_' AS "true"; --- SELECT 'be_r' NOT LIKE 'b_e__r' ESCAPE '_' AS "false"; +SELECT 'be_r' LIKE 'b_e__r' ESCAPE '_' AS `true`; +SELECT 'be_r' NOT LIKE 'b_e__r' ESCAPE '_' AS `false`; --- SELECT 'be_r' LIKE '__e__r' ESCAPE '_' AS "false"; --- SELECT 'be_r' NOT LIKE '__e__r' ESCAPE '_' AS "true"; +SELECT 'be_r' LIKE '__e__r' ESCAPE '_' AS `false`; +SELECT 'be_r' NOT LIKE '__e__r' ESCAPE '_' AS `true`; -- [SPARK-28448] Implement ILIKE operator -- diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out index 1e4e6e5021de8..4619f0ded7d92 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 92 +-- Number of queries: 122 -- !query 0 @@ -281,470 +281,718 @@ true -- !query 34 -SELECT 'foo' LIKE '_%' as t, 'f' LIKE '_%' as t, '' LIKE '_%' as f +SELECT 'hawkeye' LIKE 'h%' ESCAPE '#' AS `true` -- !query 34 schema -struct +struct -- !query 34 output -true true false +true -- !query 35 -SELECT 'foo' LIKE '%_' as t, 'f' LIKE '%_' as t, '' LIKE '%_' as f +SELECT 'hawkeye' NOT LIKE 'h%' ESCAPE '#' AS `false` -- !query 35 schema -struct +struct -- !query 35 output -true true false +false -- !query 36 -SELECT 'foo' LIKE '__%' as t, 'foo' LIKE '___%' as t, 'foo' LIKE '____%' as f +SELECT 'indio' LIKE 'ind_o' ESCAPE '$' AS `true` -- !query 36 schema -struct +struct -- !query 36 output -true true false +true -- !query 37 -SELECT 'foo' LIKE '%__' as t, 'foo' LIKE '%___' as t, 'foo' LIKE '%____' as f +SELECT 'indio' NOT LIKE 'ind_o' ESCAPE '$' AS `false` -- !query 37 schema -struct +struct -- !query 37 output -true true false +false -- !query 38 -SELECT 'jack' LIKE '%____%' AS t +SELECT 'h%' LIKE 'h#%' ESCAPE '#' AS `true` -- !query 38 schema -struct +struct -- !query 38 output true -- !query 39 -SELECT 'unknown' || ' and unknown' AS `Concat unknown types` +SELECT 'h%' NOT LIKE 'h#%' ESCAPE '#' AS `false` -- !query 39 schema -struct +struct -- !query 39 output -unknown and unknown +false -- !query 40 -SELECT string('text') || ' and unknown' AS `Concat text to unknown type` +SELECT 'h%wkeye' LIKE 'h#%' ESCAPE '#' AS `false` -- !query 40 schema -struct +struct -- !query 40 output -text and unknown +false -- !query 41 -CREATE TABLE toasttest(f1 string) USING parquet +SELECT 'h%wkeye' NOT LIKE 'h#%' ESCAPE '#' AS `true` -- !query 41 schema -struct<> +struct -- !query 41 output - +true -- !query 42 -insert into toasttest values(repeat('1234567890',10000)) +SELECT 'h%wkeye' LIKE 'h#%%' ESCAPE '#' AS `true` -- !query 42 schema -struct<> +struct -- !query 42 output - +true -- !query 43 -insert into toasttest values(repeat('1234567890',10000)) +SELECT 'h%wkeye' NOT LIKE 'h#%%' ESCAPE '#' AS `false` -- !query 43 schema -struct<> +struct -- !query 43 output - +false -- !query 44 -insert into toasttest values(repeat('1234567890',10000)) +SELECT 'h%awkeye' LIKE 'h#%a%k%e' ESCAPE '#' AS `true` -- !query 44 schema -struct<> +struct -- !query 44 output - +true -- !query 45 -insert into toasttest values(repeat('1234567890',10000)) +SELECT 'h%awkeye' NOT LIKE 'h#%a%k%e' ESCAPE '#' AS `false` -- !query 45 schema -struct<> +struct -- !query 45 output - +false -- !query 46 -SELECT substr(f1, 99995) from toasttest +SELECT 'indio' LIKE '_ndio' ESCAPE '$' AS `true` -- !query 46 schema -struct +struct -- !query 46 output +true + + +-- !query 47 +SELECT 'indio' NOT LIKE '_ndio' ESCAPE '$' AS `false` +-- !query 47 schema +struct +-- !query 47 output +false + + +-- !query 48 +SELECT 'i_dio' LIKE 'i$_d_o' ESCAPE '$' AS `true` +-- !query 48 schema +struct +-- !query 48 output +true + + +-- !query 49 +SELECT 'i_dio' NOT LIKE 'i$_d_o' ESCAPE '$' AS `false` +-- !query 49 schema +struct +-- !query 49 output +false + + +-- !query 50 +SELECT 'i_dio' LIKE 'i$_nd_o' ESCAPE '$' AS `false` +-- !query 50 schema +struct +-- !query 50 output +false + + +-- !query 51 +SELECT 'i_dio' NOT LIKE 'i$_nd_o' ESCAPE '$' AS `true` +-- !query 51 schema +struct +-- !query 51 output +true + + +-- !query 52 +SELECT 'i_dio' LIKE 'i$_d%o' ESCAPE '$' AS `true` +-- !query 52 schema +struct +-- !query 52 output +true + + +-- !query 53 +SELECT 'i_dio' NOT LIKE 'i$_d%o' ESCAPE '$' AS `false` +-- !query 53 schema +struct +-- !query 53 output +false + + +-- !query 54 +SELECT 'maca' LIKE 'm%aca' ESCAPE '%' AS `true` +-- !query 54 schema +struct<> +-- !query 54 output +org.apache.spark.sql.AnalysisException +the pattern 'm%aca' is invalid, the escape character is not allowed to precede 'a'; + + +-- !query 55 +SELECT 'maca' NOT LIKE 'm%aca' ESCAPE '%' AS `false` +-- !query 55 schema +struct<> +-- !query 55 output +org.apache.spark.sql.AnalysisException +the pattern 'm%aca' is invalid, the escape character is not allowed to precede 'a'; + + +-- !query 56 +SELECT 'ma%a' LIKE 'm%a%%a' ESCAPE '%' AS `true` +-- !query 56 schema +struct<> +-- !query 56 output +org.apache.spark.sql.AnalysisException +the pattern 'm%a%%a' is invalid, the escape character is not allowed to precede 'a'; + + +-- !query 57 +SELECT 'ma%a' NOT LIKE 'm%a%%a' ESCAPE '%' AS `false` +-- !query 57 schema +struct<> +-- !query 57 output +org.apache.spark.sql.AnalysisException +the pattern 'm%a%%a' is invalid, the escape character is not allowed to precede 'a'; + + +-- !query 58 +SELECT 'bear' LIKE 'b_ear' ESCAPE '_' AS `true` +-- !query 58 schema +struct<> +-- !query 58 output +org.apache.spark.sql.AnalysisException +the pattern 'b_ear' is invalid, the escape character is not allowed to precede 'e'; + + +-- !query 59 +SELECT 'bear' NOT LIKE 'b_ear' ESCAPE '_' AS `false` +-- !query 59 schema +struct<> +-- !query 59 output +org.apache.spark.sql.AnalysisException +the pattern 'b_ear' is invalid, the escape character is not allowed to precede 'e'; + + +-- !query 60 +SELECT 'be_r' LIKE 'b_e__r' ESCAPE '_' AS `true` +-- !query 60 schema +struct<> +-- !query 60 output +org.apache.spark.sql.AnalysisException +the pattern 'b_e__r' is invalid, the escape character is not allowed to precede 'e'; + + +-- !query 61 +SELECT 'be_r' NOT LIKE 'b_e__r' ESCAPE '_' AS `false` +-- !query 61 schema +struct<> +-- !query 61 output +org.apache.spark.sql.AnalysisException +the pattern 'b_e__r' is invalid, the escape character is not allowed to precede 'e'; + + +-- !query 62 +SELECT 'be_r' LIKE '__e__r' ESCAPE '_' AS `false` +-- !query 62 schema +struct +-- !query 62 output +false + + +-- !query 63 +SELECT 'be_r' NOT LIKE '__e__r' ESCAPE '_' AS `true` +-- !query 63 schema +struct +-- !query 63 output +true + + +-- !query 64 +SELECT 'foo' LIKE '_%' as t, 'f' LIKE '_%' as t, '' LIKE '_%' as f +-- !query 64 schema +struct +-- !query 64 output +true true false + + +-- !query 65 +SELECT 'foo' LIKE '%_' as t, 'f' LIKE '%_' as t, '' LIKE '%_' as f +-- !query 65 schema +struct +-- !query 65 output +true true false + + +-- !query 66 +SELECT 'foo' LIKE '__%' as t, 'foo' LIKE '___%' as t, 'foo' LIKE '____%' as f +-- !query 66 schema +struct +-- !query 66 output +true true false + + +-- !query 67 +SELECT 'foo' LIKE '%__' as t, 'foo' LIKE '%___' as t, 'foo' LIKE '%____' as f +-- !query 67 schema +struct +-- !query 67 output +true true false + + +-- !query 68 +SELECT 'jack' LIKE '%____%' AS t +-- !query 68 schema +struct +-- !query 68 output +true + + +-- !query 69 +SELECT 'unknown' || ' and unknown' AS `Concat unknown types` +-- !query 69 schema +struct +-- !query 69 output +unknown and unknown + + +-- !query 70 +SELECT string('text') || ' and unknown' AS `Concat text to unknown type` +-- !query 70 schema +struct +-- !query 70 output +text and unknown + + +-- !query 71 +CREATE TABLE toasttest(f1 string) USING parquet +-- !query 71 schema +struct<> +-- !query 71 output + + + +-- !query 72 +insert into toasttest values(repeat('1234567890',10000)) +-- !query 72 schema +struct<> +-- !query 72 output + + + +-- !query 73 +insert into toasttest values(repeat('1234567890',10000)) +-- !query 73 schema +struct<> +-- !query 73 output + + + +-- !query 74 +insert into toasttest values(repeat('1234567890',10000)) +-- !query 74 schema +struct<> +-- !query 74 output + + + +-- !query 75 +insert into toasttest values(repeat('1234567890',10000)) +-- !query 75 schema +struct<> +-- !query 75 output + + + +-- !query 76 +SELECT substr(f1, 99995) from toasttest +-- !query 76 schema +struct +-- !query 76 output 567890 567890 567890 567890 --- !query 47 +-- !query 77 SELECT substr(f1, 99995, 10) from toasttest --- !query 47 schema +-- !query 77 schema struct --- !query 47 output +-- !query 77 output 567890 567890 567890 567890 --- !query 48 +-- !query 78 SELECT length('abcdef') AS `length_6` --- !query 48 schema +-- !query 78 schema struct --- !query 48 output +-- !query 78 output 6 --- !query 49 +-- !query 79 SELECT position('cd', 'abcdef') AS `pos_3` --- !query 49 schema +-- !query 79 schema struct --- !query 49 output +-- !query 79 output 3 --- !query 50 +-- !query 80 SELECT position('xy', 'abcdef') AS `pos_0` --- !query 50 schema +-- !query 80 schema struct --- !query 50 output +-- !query 80 output 0 --- !query 51 +-- !query 81 SELECT replace('abcdef', 'de', '45') AS `abc45f` --- !query 51 schema +-- !query 81 schema struct --- !query 51 output +-- !query 81 output abc45f --- !query 52 +-- !query 82 SELECT replace('yabadabadoo', 'ba', '123') AS `ya123da123doo` --- !query 52 schema +-- !query 82 schema struct --- !query 52 output +-- !query 82 output ya123da123doo --- !query 53 +-- !query 83 SELECT replace('yabadoo', 'bad', '') AS `yaoo` --- !query 53 schema +-- !query 83 schema struct --- !query 53 output +-- !query 83 output yaoo --- !query 54 +-- !query 84 select hex(256*256*256 - 1) AS `ffffff` --- !query 54 schema +-- !query 84 schema struct --- !query 54 output +-- !query 84 output FFFFFF --- !query 55 +-- !query 85 select hex(bigint(bigint(bigint(bigint(256)*256)*256)*256) - 1) AS `ffffffff` --- !query 55 schema +-- !query 85 schema struct --- !query 55 output +-- !query 85 output FFFFFFFF --- !query 56 +-- !query 86 select md5('') = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` --- !query 56 schema +-- !query 86 schema struct --- !query 56 output +-- !query 86 output true --- !query 57 +-- !query 87 select md5('a') = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` --- !query 57 schema +-- !query 87 schema struct --- !query 57 output +-- !query 87 output true --- !query 58 +-- !query 88 select md5('abc') = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` --- !query 58 schema +-- !query 88 schema struct --- !query 58 output +-- !query 88 output true --- !query 59 +-- !query 89 select md5('message digest') = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` --- !query 59 schema +-- !query 89 schema struct --- !query 59 output +-- !query 89 output true --- !query 60 +-- !query 90 select md5('abcdefghijklmnopqrstuvwxyz') = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` --- !query 60 schema +-- !query 90 schema struct --- !query 60 output +-- !query 90 output true --- !query 61 +-- !query 91 select md5('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789') = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` --- !query 61 schema +-- !query 91 schema struct --- !query 61 output +-- !query 91 output true --- !query 62 +-- !query 92 select md5('12345678901234567890123456789012345678901234567890123456789012345678901234567890') = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` --- !query 62 schema +-- !query 92 schema struct --- !query 62 output +-- !query 92 output true --- !query 63 +-- !query 93 select md5(binary('')) = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` --- !query 63 schema +-- !query 93 schema struct --- !query 63 output +-- !query 93 output true --- !query 64 +-- !query 94 select md5(binary('a')) = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` --- !query 64 schema +-- !query 94 schema struct --- !query 64 output +-- !query 94 output true --- !query 65 +-- !query 95 select md5(binary('abc')) = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` --- !query 65 schema +-- !query 95 schema struct --- !query 65 output +-- !query 95 output true --- !query 66 +-- !query 96 select md5(binary('message digest')) = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` --- !query 66 schema +-- !query 96 schema struct --- !query 66 output +-- !query 96 output true --- !query 67 +-- !query 97 select md5(binary('abcdefghijklmnopqrstuvwxyz')) = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` --- !query 67 schema +-- !query 97 schema struct --- !query 67 output +-- !query 97 output true --- !query 68 +-- !query 98 select md5(binary('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789')) = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` --- !query 68 schema +-- !query 98 schema struct --- !query 68 output +-- !query 98 output true --- !query 69 +-- !query 99 select md5(binary('12345678901234567890123456789012345678901234567890123456789012345678901234567890')) = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` --- !query 69 schema +-- !query 99 schema struct --- !query 69 output +-- !query 99 output true --- !query 70 +-- !query 100 SELECT initcap('hi THOMAS') --- !query 70 schema +-- !query 100 schema struct --- !query 70 output +-- !query 100 output Hi Thomas --- !query 71 +-- !query 101 SELECT lpad('hi', 5, 'xy') --- !query 71 schema +-- !query 101 schema struct --- !query 71 output +-- !query 101 output xyxhi --- !query 72 +-- !query 102 SELECT lpad('hi', 5) --- !query 72 schema +-- !query 102 schema struct --- !query 72 output +-- !query 102 output hi --- !query 73 +-- !query 103 SELECT lpad('hi', -5, 'xy') --- !query 73 schema +-- !query 103 schema struct --- !query 73 output +-- !query 103 output --- !query 74 +-- !query 104 SELECT lpad('hello', 2) --- !query 74 schema +-- !query 104 schema struct --- !query 74 output +-- !query 104 output he --- !query 75 +-- !query 105 SELECT lpad('hi', 5, '') --- !query 75 schema +-- !query 105 schema struct --- !query 75 output +-- !query 105 output hi --- !query 76 +-- !query 106 SELECT rpad('hi', 5, 'xy') --- !query 76 schema +-- !query 106 schema struct --- !query 76 output +-- !query 106 output hixyx --- !query 77 +-- !query 107 SELECT rpad('hi', 5) --- !query 77 schema +-- !query 107 schema struct --- !query 77 output +-- !query 107 output hi --- !query 78 +-- !query 108 SELECT rpad('hi', -5, 'xy') --- !query 78 schema +-- !query 108 schema struct --- !query 78 output +-- !query 108 output --- !query 79 +-- !query 109 SELECT rpad('hello', 2) --- !query 79 schema +-- !query 109 schema struct --- !query 79 output +-- !query 109 output he --- !query 80 +-- !query 110 SELECT rpad('hi', 5, '') --- !query 80 schema +-- !query 110 schema struct --- !query 80 output +-- !query 110 output hi --- !query 81 +-- !query 111 SELECT ltrim('zzzytrim', 'xyz') --- !query 81 schema +-- !query 111 schema struct --- !query 81 output +-- !query 111 output trim --- !query 82 +-- !query 112 SELECT translate('', '14', 'ax') --- !query 82 schema +-- !query 112 schema struct --- !query 82 output +-- !query 112 output --- !query 83 +-- !query 113 SELECT translate('12345', '14', 'ax') --- !query 83 schema +-- !query 113 schema struct --- !query 83 output +-- !query 113 output a23x5 --- !query 84 +-- !query 114 SELECT ascii('x') --- !query 84 schema +-- !query 114 schema struct --- !query 84 output +-- !query 114 output 120 --- !query 85 +-- !query 115 SELECT ascii('') --- !query 85 schema +-- !query 115 schema struct --- !query 85 output +-- !query 115 output 0 --- !query 86 +-- !query 116 SELECT chr(65) --- !query 86 schema +-- !query 116 schema struct --- !query 86 output +-- !query 116 output A --- !query 87 +-- !query 117 SELECT chr(0) --- !query 87 schema +-- !query 117 schema struct --- !query 87 output +-- !query 117 output --- !query 88 +-- !query 118 SELECT repeat('Pg', 4) --- !query 88 schema +-- !query 118 schema struct --- !query 88 output +-- !query 118 output PgPgPgPg --- !query 89 +-- !query 119 SELECT repeat('Pg', -4) --- !query 89 schema +-- !query 119 schema struct --- !query 89 output +-- !query 119 output --- !query 90 +-- !query 120 SELECT trim(binary('\\000') from binary('\\000Tom\\000')) --- !query 90 schema +-- !query 120 schema struct --- !query 90 output +-- !query 120 output Tom --- !query 91 +-- !query 121 DROP TABLE toasttest --- !query 91 schema +-- !query 121 schema struct<> --- !query 91 output +-- !query 121 output From e5abbab0ed5d9d70522c1e19c53e95c631dd1565 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 23 Dec 2019 09:57:42 +0900 Subject: [PATCH 0937/1387] [SPARK-30128][DOCS][PYTHON][SQL] Document/promote 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC ### What changes were proposed in this pull request? This PR adds and exposes the options, 'recursiveFileLookup' and 'pathGlobFilter' in file sources 'mergeSchema' in ORC, into documentation. - `recursiveFileLookup` at file sources: https://github.com/apache/spark/pull/24830 ([SPARK-27627](https://issues.apache.org/jira/browse/SPARK-27627)) - `pathGlobFilter` at file sources: https://github.com/apache/spark/pull/24518 ([SPARK-27990](https://issues.apache.org/jira/browse/SPARK-27990)) - `mergeSchema` at ORC: https://github.com/apache/spark/pull/24043 ([SPARK-11412](https://issues.apache.org/jira/browse/SPARK-11412)) **Note that** `timeZone` option was not moved from `DataFrameReader.options` as I assume it will likely affect other datasources as well once DSv2 is complete. ### Why are the changes needed? To document available options in sources properly. ### Does this PR introduce any user-facing change? In PySpark, `pathGlobFilter` can be set via `DataFrameReader.(text|orc|parquet|json|csv)` and `DataStreamReader.(text|orc|parquet|json|csv)`. ### How was this patch tested? Manually built the doc and checked the output. Option setting in PySpark is rather a logical change. I manually tested one only: ```bash $ ls -al tmp ... -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 aa -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ab -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 ac -rw-r--r-- 1 hyukjin.kwon staff 3 Dec 20 12:19 cc ``` ```python >>> spark.read.text("tmp", pathGlobFilter="*c").show() ``` ``` +-----+ |value| +-----+ | ac| | cc| +-----+ ``` Closes #26958 from HyukjinKwon/doc-followup. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/readwriter.py | 50 ++++++++++++----- python/pyspark/sql/streaming.py | 55 ++++++++++++------- .../apache/spark/sql/DataFrameReader.scala | 50 +++++++++++------ .../sql/streaming/DataStreamReader.scala | 47 ++++++++++------ 4 files changed, 134 insertions(+), 68 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 198313b10ec14..3d3280dbd9943 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -171,7 +171,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, samplingRatio=None, - dropFieldIfAllNull=None, encoding=None, locale=None, recursiveFileLookup=None): + dropFieldIfAllNull=None, encoding=None, locale=None, pathGlobFilter=None, + recursiveFileLookup=None): """ Loads JSON files and returns the results as a :class:`DataFrame`. @@ -247,6 +248,9 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, :param locale: sets a locale as language tag in IETF BCP 47 format. If None is set, it uses the default value, ``en-US``. For instance, ``locale`` is used while parsing dates and timestamps. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -270,7 +274,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, samplingRatio=samplingRatio, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, - locale=locale, recursiveFileLookup=recursiveFileLookup) + locale=locale, pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -309,8 +313,12 @@ def parquet(self, *paths, **options): Loads Parquet files, returning the result as a :class:`DataFrame`. :param mergeSchema: sets whether we should merge schemas collected from all - Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. - The default value is specified in ``spark.sql.parquet.mergeSchema``. + Parquet part-files. This will override + ``spark.sql.parquet.mergeSchema``. The default value is specified in + ``spark.sql.parquet.mergeSchema``. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -319,13 +327,16 @@ def parquet(self, *paths, **options): [('name', 'string'), ('year', 'int'), ('month', 'int'), ('day', 'int')] """ mergeSchema = options.get('mergeSchema', None) + pathGlobFilter = options.get('pathGlobFilter', None) recursiveFileLookup = options.get('recursiveFileLookup', None) - self._set_opts(mergeSchema=mergeSchema, recursiveFileLookup=recursiveFileLookup) + self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, + recursiveFileLookup=recursiveFileLookup) return self._df(self._jreader.parquet(_to_seq(self._spark._sc, paths))) @ignore_unicode_prefix @since(1.6) - def text(self, paths, wholetext=False, lineSep=None, recursiveFileLookup=None): + def text(self, paths, wholetext=False, lineSep=None, pathGlobFilter=None, + recursiveFileLookup=None): """ Loads text files and returns a :class:`DataFrame` whose schema starts with a string column named "value", and followed by partitioned columns if there @@ -338,6 +349,9 @@ def text(self, paths, wholetext=False, lineSep=None, recursiveFileLookup=None): :param wholetext: if true, read each file from input path(s) as a single row. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -349,7 +363,8 @@ def text(self, paths, wholetext=False, lineSep=None, recursiveFileLookup=None): [Row(value=u'hello\\nthis')] """ self._set_opts( - wholetext=wholetext, lineSep=lineSep, recursiveFileLookup=recursiveFileLookup) + wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter, + recursiveFileLookup=recursiveFileLookup) if isinstance(paths, basestring): paths = [paths] return self._df(self._jreader.text(self._spark._sc._jvm.PythonUtils.toSeq(paths))) @@ -362,7 +377,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, samplingRatio=None, enforceSchema=None, emptyValue=None, locale=None, lineSep=None, - recursiveFileLookup=None): + pathGlobFilter=None, recursiveFileLookup=None): r"""Loads a CSV file and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -470,6 +485,9 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. Maximum length is 1 character. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -492,7 +510,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine, charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, samplingRatio=samplingRatio, enforceSchema=enforceSchema, emptyValue=emptyValue, locale=locale, lineSep=lineSep, - recursiveFileLookup=recursiveFileLookup) + pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -520,20 +538,24 @@ def func(iterator): raise TypeError("path can be only string, list or RDD") @since(1.5) - def orc(self, path, mergeSchema=None, recursiveFileLookup=None): + def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None): """Loads ORC files, returning the result as a :class:`DataFrame`. :param mergeSchema: sets whether we should merge schemas collected from all - ORC part-files. This will override ``spark.sql.orc.mergeSchema``. - The default value is specified in ``spark.sql.orc.mergeSchema``. + ORC part-files. This will override ``spark.sql.orc.mergeSchema``. + The default value is specified in ``spark.sql.orc.mergeSchema``. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option - disables `partition discovery`_. + disables `partition discovery`_. >>> df = spark.read.orc('python/test_support/sql/orc_partitioned') >>> df.dtypes [('a', 'bigint'), ('b', 'int'), ('c', 'int')] """ - self._set_opts(mergeSchema=mergeSchema, recursiveFileLookup=recursiveFileLookup) + self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, + recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): path = [path] return self._df(self._jreader.orc(_to_seq(self._spark._sc, path))) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 2937bc0bda70a..f17a52f6b3dc8 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -341,9 +341,6 @@ def option(self, key, value): * ``timeZone``: sets the string that indicates a timezone to be used to parse timestamps in the JSON/CSV datasources or partition values. If it isn't set, it uses the default value, session local timezone. - * ``pathGlobFilter``: an optional glob pattern to only include files with paths matching - the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. - It does not change the behavior of partition discovery. .. note:: Evolving. @@ -360,9 +357,6 @@ def options(self, **options): * ``timeZone``: sets the string that indicates a timezone to be used to parse timestamps in the JSON/CSV datasources or partition values. If it isn't set, it uses the default value, session local timezone. - * ``pathGlobFilter``: an optional glob pattern to only include files with paths matching - the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. - It does not change the behavior of partition discovery. .. note:: Evolving. @@ -411,7 +405,8 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None, multiLine=None, allowUnquotedControlChars=None, lineSep=None, locale=None, - dropFieldIfAllNull=None, encoding=None, recursiveFileLookup=None): + dropFieldIfAllNull=None, encoding=None, pathGlobFilter=None, + recursiveFileLookup=None): """ Loads a JSON file stream and returns the results as a :class:`DataFrame`. @@ -487,6 +482,9 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, the JSON files. For example UTF-16BE, UTF-32LE. If None is set, the encoding of input JSON will be detected automatically when the multiLine option is set to ``true``. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -507,21 +505,24 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, timestampFormat=timestampFormat, multiLine=multiLine, allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale, dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding, - recursiveFileLookup=recursiveFileLookup) + pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.json(path)) else: raise TypeError("path can be only a single string") @since(2.3) - def orc(self, path, mergeSchema=None, recursiveFileLookup=None): + def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None): """Loads a ORC file stream, returning the result as a :class:`DataFrame`. .. note:: Evolving. :param mergeSchema: sets whether we should merge schemas collected from all - ORC part-files. This will override ``spark.sql.orc.mergeSchema``. - The default value is specified in ``spark.sql.orc.mergeSchema``. + ORC part-files. This will override ``spark.sql.orc.mergeSchema``. + The default value is specified in ``spark.sql.orc.mergeSchema``. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -531,22 +532,27 @@ def orc(self, path, mergeSchema=None, recursiveFileLookup=None): >>> orc_sdf.schema == sdf_schema True """ - self._set_opts(mergeSchema=mergeSchema, recursiveFileLookup=recursiveFileLookup) + self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, + recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.orc(path)) else: raise TypeError("path can be only a single string") @since(2.0) - def parquet(self, path, mergeSchema=None, recursiveFileLookup=None): + def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None): """ Loads a Parquet file stream, returning the result as a :class:`DataFrame`. .. note:: Evolving. :param mergeSchema: sets whether we should merge schemas collected from all - Parquet part-files. This will override ``spark.sql.parquet.mergeSchema``. - The default value is specified in ``spark.sql.parquet.mergeSchema``. + Parquet part-files. This will override + ``spark.sql.parquet.mergeSchema``. The default value is specified in + ``spark.sql.parquet.mergeSchema``. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -556,7 +562,8 @@ def parquet(self, path, mergeSchema=None, recursiveFileLookup=None): >>> parquet_sdf.schema == sdf_schema True """ - self._set_opts(mergeSchema=mergeSchema, recursiveFileLookup=recursiveFileLookup) + self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter, + recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.parquet(path)) else: @@ -564,7 +571,8 @@ def parquet(self, path, mergeSchema=None, recursiveFileLookup=None): @ignore_unicode_prefix @since(2.0) - def text(self, path, wholetext=False, lineSep=None, recursiveFileLookup=None): + def text(self, path, wholetext=False, lineSep=None, pathGlobFilter=None, + recursiveFileLookup=None): """ Loads a text file stream and returns a :class:`DataFrame` whose schema starts with a string column named "value", and followed by partitioned columns if there @@ -579,6 +587,9 @@ def text(self, path, wholetext=False, lineSep=None, recursiveFileLookup=None): :param wholetext: if true, read each file from input path(s) as a single row. :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -589,7 +600,8 @@ def text(self, path, wholetext=False, lineSep=None, recursiveFileLookup=None): True """ self._set_opts( - wholetext=wholetext, lineSep=lineSep, recursiveFileLookup=recursiveFileLookup) + wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter, + recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.text(path)) else: @@ -603,7 +615,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, enforceSchema=None, emptyValue=None, locale=None, lineSep=None, - recursiveFileLookup=None): + pathGlobFilter=None, recursiveFileLookup=None): r"""Loads a CSV file stream and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -706,6 +718,9 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non :param lineSep: defines the line separator that should be used for parsing. If None is set, it covers all ``\\r``, ``\\r\\n`` and ``\\n``. Maximum length is 1 character. + :param pathGlobFilter: an optional glob pattern to only include files with paths matching + the pattern. The syntax follows `org.apache.hadoop.fs.GlobFilter`. + It does not change the behavior of `partition discovery`_. :param recursiveFileLookup: recursively scan a directory for files. Using this option disables `partition discovery`_. @@ -726,7 +741,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine, charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema, emptyValue=emptyValue, locale=locale, lineSep=lineSep, - recursiveFileLookup=recursiveFileLookup) + pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup) if isinstance(path, basestring): return self._df(self._jreader.csv(path)) else: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 8570e4640feea..2d303b03ecd3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -98,9 +98,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *

        *
      • `timeZone` (default session local timezone): sets the string that indicates a timezone * to be used to parse timestamps in the JSON/CSV datasources or partition values.
      • - *
      • `pathGlobFilter`: an optional glob pattern to only include files with paths matching - * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. - * It does not change the behavior of partition discovery.
      • *
      * * @since 1.4.0 @@ -138,9 +135,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
        *
      • `timeZone` (default session local timezone): sets the string that indicates a timezone * to be used to parse timestamps in the JSON/CSV datasources or partition values.
      • - *
      • `pathGlobFilter`: an optional glob pattern to only include files with paths matching - * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. - * It does not change the behavior of partition discovery.
      • *
      * * @since 1.4.0 @@ -157,9 +151,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
        *
      • `timeZone` (default session local timezone): sets the string that indicates a timezone * to be used to parse timestamps in the JSON/CSV datasources or partition values.
      • - *
      • `pathGlobFilter`: an optional glob pattern to only include files with paths matching - * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. - * It does not change the behavior of partition discovery.
      • *
      * * @since 1.4.0 @@ -403,6 +394,11 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * empty array/struct during schema inference. *
    5. `locale` (default is `en-US`): sets a locale as language tag in IETF BCP 47 format. * For instance, this is used while parsing dates and timestamps.
    6. + *
    7. `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
    8. + *
    9. `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
    10. * * * @since 2.0.0 @@ -640,6 +636,11 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * For instance, this is used while parsing dates and timestamps. *
    11. `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing. Maximum length is 1 character.
    12. + *
    13. `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
    14. + *
    15. `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
    16. * * * @since 2.0.0 @@ -666,7 +667,13 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    17. `mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets * whether we should merge schemas collected from all Parquet part-files. This will override * `spark.sql.parquet.mergeSchema`.
    18. + *
    19. `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
    20. + *
    21. `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
    22. * + * * @since 1.4.0 */ @scala.annotation.varargs @@ -688,6 +695,18 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Loads ORC files and returns the result as a `DataFrame`. * + * You can set the following ORC-specific option(s) for reading ORC files: + *
        + *
      • `mergeSchema` (default is the value specified in `spark.sql.orc.mergeSchema`): sets whether + * we should merge schemas collected from all ORC part-files. This will override + * `spark.sql.orc.mergeSchema`.
      • + *
      • `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
      • + *
      • `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
      • + *
      + * * @param paths input paths * @since 2.0.0 */ @@ -736,6 +755,11 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * *
    23. `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing.
    24. + *
    25. `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
    26. + *
    27. `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
    28. * * * @param paths input paths @@ -771,13 +795,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * spark.read().textFile("/path/to/spark/README.md") * }}} * - * You can set the following textFile-specific option(s) for reading text files: - *
        - *
      • `wholetext` (default `false`): If true, read a file as a single row and not split by "\n". - *
      • - *
      • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator - * that should be used for parsing.
      • - *
      + * You can set the text-specific options as specified in `DataFrameReader.text`. * * @param paths input path * @since 2.0.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 4a6516d325ddd..cfe6192e7d5c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -83,9 +83,6 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
        *
      • `timeZone` (default session local timezone): sets the string that indicates a timezone * to be used to parse timestamps in the JSON/CSV datasources or partition values.
      • - *
      • `pathGlobFilter`: an optional glob pattern to only include files with paths matching - * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. - * It does not change the behavior of partition discovery.
      • *
      * * @since 2.0.0 @@ -123,9 +120,6 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
        *
      • `timeZone` (default session local timezone): sets the string that indicates a timezone * to be used to parse timestamps in the JSON/CSV data sources or partition values.
      • - *
      • `pathGlobFilter`: an optional glob pattern to only include files with paths matching - * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. - * It does not change the behavior of partition discovery.
      • *
      * * @since 2.0.0 @@ -142,9 +136,6 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
        *
      • `timeZone` (default session local timezone): sets the string that indicates a timezone * to be used to parse timestamps in the JSON/CSV data sources or partition values.
      • - *
      • `pathGlobFilter`: an optional glob pattern to only include files with paths matching - * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. - * It does not change the behavior of partition discovery.
      • *
      * * @since 2.0.0 @@ -277,6 +268,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * empty array/struct during schema inference. *
    29. `locale` (default is `en-US`): sets a locale as language tag in IETF BCP 47 format. * For instance, this is used while parsing dates and timestamps.
    30. + *
    31. `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
    32. + *
    33. `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
    34. * * * @since 2.0.0 @@ -357,6 +353,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * For instance, this is used while parsing dates and timestamps. *
    35. `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing. Maximum length is 1 character.
    36. + *
    37. `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
    38. + *
    39. `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
    40. * * * @since 2.0.0 @@ -370,6 +371,14 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
        *
      • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
      • + *
      • `mergeSchema` (default is the value specified in `spark.sql.orc.mergeSchema`): sets whether + * we should merge schemas collected from all ORC part-files. This will override + * `spark.sql.orc.mergeSchema`.
      • + *
      • `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
      • + *
      • `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
      • *
      * * @since 2.3.0 @@ -389,6 +398,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * whether we should merge schemas collected from all * Parquet part-files. This will override * `spark.sql.parquet.mergeSchema`. + *
    41. `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
    42. + *
    43. `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
    44. * * * @since 2.0.0 @@ -419,6 +433,11 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * *
    45. `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator * that should be used for parsing.
    46. + *
    47. `pathGlobFilter`: an optional glob pattern to only include files with paths matching + * the pattern. The syntax follows org.apache.hadoop.fs.GlobFilter. + * It does not change the behavior of partition discovery.
    48. + *
    49. `recursiveFileLookup`: recursively scan a directory for files. Using this option + * disables partition discovery
    50. * * * @since 2.0.0 @@ -442,15 +461,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * spark.readStream().textFile("/path/to/spark/README.md") * }}} * - * You can set the following text-specific options to deal with text files: - *
        - *
      • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be - * considered in every trigger.
      • - *
      • `wholetext` (default `false`): If true, read a file as a single row and not split by "\n". - *
      • - *
      • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator - * that should be used for parsing.
      • - *
      + * You can set the text-specific options as specified in `DataStreamReader.text`. * * @param path input path * @since 2.1.0 From 640dcc435b3136035adb8b6320b29efa59f7f65e Mon Sep 17 00:00:00 2001 From: "wangguangxin.cn" Date: Mon, 23 Dec 2019 13:13:35 +0800 Subject: [PATCH 0938/1387] [SPARK-28332][SQL] Reserve init value -1 only when do min max statistics in SQLMetrics ### What changes were proposed in this pull request? This is an alternative solution to https://github.com/apache/spark/pull/25095. SQLMetrics use -1 as init value as a work around for [SPARK-11013](https://issues.apache.org/jira/browse/SPARK-11013.) However, it may bring out some badcases as https://github.com/apache/spark/pull/26726 reporting. In fact, we only need to reserve -1 when doing min max statistics in `SQLMetrics.stringValue` so that we can filter out those not initialized accumulators. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing UTs Closes #26899 from WangGuangxin/sqlmetrics. Authored-by: wangguangxin.cn Signed-off-by: Wenchen Fan --- .../sql/execution/metric/SQLMetrics.scala | 11 +++++--- .../adaptive/AdaptiveQueryExecSuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 26 +++++++++++++++++++ 3 files changed, 35 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 45b1c86130db5..65aabe004d75b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -50,14 +50,19 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato override def reset(): Unit = _value = _zeroValue override def merge(other: AccumulatorV2[Long, Long]): Unit = other match { - case o: SQLMetric => _value += o.value + case o: SQLMetric => + if (_value < 0) _value = 0 + if (o.value > 0) _value += o.value case _ => throw new UnsupportedOperationException( s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") } override def isZero(): Boolean = _value == _zeroValue - override def add(v: Long): Unit = _value += v + override def add(v: Long): Unit = { + if (_value < 0) _value = 0 + _value += v + } // We can set a double value to `SQLMetric` which stores only long value, if it is // average metrics. @@ -65,7 +70,7 @@ class SQLMetric(val metricType: String, initValue: Long = 0L) extends Accumulato def set(v: Long): Unit = _value = v - def +=(v: Long): Unit = _value += v + def +=(v: Long): Unit = add(v) override def value: Long = _value diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 8af9b8f004265..8ea595bd87c39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -495,7 +495,7 @@ class AdaptiveQueryExecSuite withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.LOCAL_SHUFFLE_READER_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30") { + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "40") { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( """ |SELECT * FROM testData t1 join testData2 t2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index fcb089eba80a6..206bd78c01a87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -613,4 +614,29 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { Map(1L -> (("InMemoryTableScan", Map.empty))) ) } + + test("SPARK-28332: SQLMetric merge should handle -1 properly") { + def checkSparkPlanMetrics(plan: SparkPlan, expected: Map[String, Long]): Unit = { + expected.foreach { case (metricName: String, metricValue: Long) => + assert(plan.metrics.contains(metricName), s"The query plan should have metric $metricName") + val actualMetric = plan.metrics.get(metricName).get + assert(actualMetric.value == metricValue, + s"The query plan metric $metricName did not match, " + + s"expected:$metricValue, actual:${actualMetric.value}") + } + } + + val df = testData.join(testData2.filter('b === 0), $"key" === $"a", "left_outer") + df.collect() + val plan = df.queryExecution.executedPlan + + val exchanges = plan.collect { + case s: ShuffleExchangeExec => s + } + + assert(exchanges.size == 2, "The query plan should have two shuffle exchanges") + + checkSparkPlanMetrics(exchanges(0), Map("dataSize" -> 3200, "shuffleRecordsWritten" -> 100)) + checkSparkPlanMetrics(exchanges(1), Map("dataSize" -> 0, "shuffleRecordsWritten" -> 0)) + } } From c6ab7165dd11a0a7b8aea4c805409088e9a41a74 Mon Sep 17 00:00:00 2001 From: zhanjf Date: Mon, 23 Dec 2019 10:11:09 -0600 Subject: [PATCH 0939/1387] [SPARK-29224][ML] Implement Factorization Machines as a ml-pipeline component MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Implement Factorization Machines as a ml-pipeline component 1. loss function supports: logloss, mse 2. optimizer: GD, adamW ### Why are the changes needed? Factorization Machines is widely used in advertising and recommendation system to estimate CTR(click-through rate). Advertising and recommendation system usually has a lot of data, so we need Spark to estimate the CTR, and Factorization Machines are common ml model to estimate CTR. References: 1. S. Rendle, “Factorization machines,” in Proceedings of IEEE International Conference on Data Mining (ICDM), pp. 995–1000, 2010. https://www.csie.ntu.edu.tw/~b97053/paper/Rendle2010FM.pdf ### Does this PR introduce any user-facing change? No ### How was this patch tested? run unit tests Closes #26124 from mob-ai/ml/fm. Authored-by: zhanjf Signed-off-by: Sean Owen --- docs/ml-classification-regression.md | 107 +++ .../examples/ml/JavaFMClassifierExample.java | 105 +++ .../examples/ml/JavaFMRegressorExample.java | 90 ++ .../main/python/ml/fm_classifier_example.py | 77 ++ .../main/python/ml/fm_regressor_example.py | 74 ++ .../examples/ml/FMClassifierExample.scala | 96 +++ .../examples/ml/FMRegressorExample.scala | 84 ++ .../ml/classification/FMClassifier.scala | 332 +++++++ .../spark/ml/regression/FMRegressor.scala | 815 ++++++++++++++++++ .../ml/classification/FMClassifierSuite.scala | 242 ++++++ .../ml/regression/FMRegressorSuite.scala | 240 ++++++ python/pyspark/ml/classification.py | 166 +++- python/pyspark/ml/regression.py | 164 +++- 13 files changed, 2590 insertions(+), 2 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaFMClassifierExample.java create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaFMRegressorExample.java create mode 100644 examples/src/main/python/ml/fm_classifier_example.py create mode 100644 examples/src/main/python/ml/fm_regressor_example.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/regression/FMRegressorSuite.scala diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index 6a81aff1476d9..630a15d8535f4 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -530,6 +530,42 @@ Refer to the [R API docs](api/R/spark.naiveBayes.html) for more details. +## Factorization machines classifier + +For more background and more details about the implementation of factorization machines, +refer to the [Factorization Machines section](ml-classification-regression.html#factorization-machines). + +**Examples** + +The following examples load a dataset in LibSVM format, split it into training and test sets, +train on the first dataset, and then evaluate on the held-out test set. +We scale features to be between 0 and 1 to prevent the exploding gradient problem. + +
      +
      + +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.classification.FMClassifier) for more details. + +{% include_example scala/org/apache/spark/examples/ml/FMClassifierExample.scala %} +
      + +
      + +Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/FMClassifier.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaFMClassifierExample.java %} +
      + +
      + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classification.FMClassifier) for more details. + +{% include_example python/ml/fm_classifier_example.py %} +
      + +
      + + # Regression ## Linear regression @@ -1015,6 +1051,43 @@ Refer to the [`IsotonicRegression` R API docs](api/R/spark.isoreg.html) for more + +## Factorization machines regressor + +For more background and more details about the implementation of factorization machines, +refer to the [Factorization Machines section](ml-classification-regression.html#factorization-machines). + +**Examples** + +The following examples load a dataset in LibSVM format, split it into training and test sets, +train on the first dataset, and then evaluate on the held-out test set. +We scale features to be between 0 and 1 to prevent the exploding gradient problem. + +
      +
      + +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.FMRegressor) for more details. + +{% include_example scala/org/apache/spark/examples/ml/FMRegressorExample.scala %} +
      + +
      + +Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/FMRegressor.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaFMRegressorExample.java %} +
      + +
      + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.FMRegressor) for more details. + +{% include_example python/ml/fm_regressor_example.py %} +
      + +
      + + # Linear methods We implement popular linear methods such as logistic @@ -1044,6 +1117,40 @@ regression](http://en.wikipedia.org/wiki/Tikhonov_regularization) model. We implement Pipelines API for both linear regression and logistic regression with elastic net regularization. +# Factorization Machines + +[Factorization Machines](https://www.csie.ntu.edu.tw/~b97053/paper/Rendle2010FM.pdf) are able to estimate interactions +between features even in problems with huge sparsity (like advertising and recommendation system). +The `spark.ml` implementation supports factorization machines for binary classification and for regression. + +Factorization machines formula is: + +$$ +\hat{y} = w_0 + \sum\limits^n_{i-1} w_i x_i + + \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j +$$ + +The first two terms denote intercept and linear term (same as in linear regression), +and the last term denotes pairwise interactions term. $$v_i$$ describes the i-th variable +with k factors. + +FM can be used for regression and optimization criterion is mean square error. FM also can be used for +binary classification through sigmoid function. The optimization criterion is logistic loss. + +The pairwise interactions can be reformulated: + +$$ +\sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j + = \frac{1}{2}\sum\limits^k_{f=1} + \left(\left( \sum\limits^n_{i=1}v_{i,f}x_i \right)^2 - + \sum\limits^n_{i=1}v_{i,f}^2x_i^2 \right) +$$ + +This equation has only linear complexity in both k and n - i.e. its computation is in $$O(kn)$$. + +In general, in order to prevent the exploding gradient problem, it is best to scale continuous features to be between 0 and 1, +or bin the continuous features and one-hot encode them. + # Decision trees [Decision trees](http://en.wikipedia.org/wiki/Decision_tree_learning) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaFMClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaFMClassifierExample.java new file mode 100644 index 0000000000000..2f11082e0dd69 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaFMClassifierExample.java @@ -0,0 +1,105 @@ +/* + * 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.examples.ml; + +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.FMClassificationModel; +import org.apache.spark.ml.classification.FMClassifier; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.ml.feature.*; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaFMClassifierExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaFMClassifierExample") + .getOrCreate(); + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + Dataset data = spark + .read() + .format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); + + // Index labels, adding metadata to the label column. + // Fit on whole dataset to include all labels in index. + StringIndexerModel labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data); + // Scale features. + MinMaxScalerModel featureScaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + .fit(data); + + // Split the data into training and test sets (30% held out for testing) + Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); + Dataset trainingData = splits[0]; + Dataset testData = splits[1]; + + // Train a FM model. + FMClassifier fm = new FMClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("scaledFeatures") + .setStepSize(0.001); + + // Convert indexed labels back to original labels. + IndexToString labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labelsArray()[0]); + + // Create a Pipeline. + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {labelIndexer, featureScaler, fm, labelConverter}); + + // Train model. + PipelineModel model = pipeline.fit(trainingData); + + // Make predictions. + Dataset predictions = model.transform(testData); + + // Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5); + + // Select (prediction, true label) and compute test accuracy. + MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("accuracy"); + double accuracy = evaluator.evaluate(predictions); + System.out.println("Test Accuracy = " + accuracy); + + FMClassificationModel fmModel = (FMClassificationModel)(model.stages()[2]); + System.out.println("Factors: " + fmModel.factors()); + System.out.println("Linear: " + fmModel.linear()); + System.out.println("Intercept: " + fmModel.intercept()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaFMRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaFMRegressorExample.java new file mode 100644 index 0000000000000..3ade40f3d39c2 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaFMRegressorExample.java @@ -0,0 +1,90 @@ +/* + * 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.examples.ml; + +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.feature.MinMaxScaler; +import org.apache.spark.ml.feature.MinMaxScalerModel; +import org.apache.spark.ml.regression.FMRegressionModel; +import org.apache.spark.ml.regression.FMRegressor; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaFMRegressorExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaFMRegressorExample") + .getOrCreate(); + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + + // Scale features. + MinMaxScalerModel featureScaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + .fit(data); + + // Split the data into training and test sets (30% held out for testing). + Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); + Dataset trainingData = splits[0]; + Dataset testData = splits[1]; + + // Train a FM model. + FMRegressor fm = new FMRegressor() + .setLabelCol("label") + .setFeaturesCol("scaledFeatures") + .setStepSize(0.001); + + // Create a Pipeline. + Pipeline pipeline = new Pipeline().setStages(new PipelineStage[] {featureScaler, fm}); + + // Train model. + PipelineModel model = pipeline.fit(trainingData); + + // Make predictions. + Dataset predictions = model.transform(testData); + + // Select example rows to display. + predictions.select("prediction", "label", "features").show(5); + + // Select (prediction, true label) and compute test error. + RegressionEvaluator evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse"); + double rmse = evaluator.evaluate(predictions); + System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); + + FMRegressionModel fmModel = (FMRegressionModel)(model.stages()[1]); + System.out.println("Factors: " + fmModel.factors()); + System.out.println("Linear: " + fmModel.linear()); + System.out.println("Intercept: " + fmModel.intercept()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/python/ml/fm_classifier_example.py b/examples/src/main/python/ml/fm_classifier_example.py new file mode 100644 index 0000000000000..6e7c2ccf021ed --- /dev/null +++ b/examples/src/main/python/ml/fm_classifier_example.py @@ -0,0 +1,77 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +FMClassifier Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.classification import FMClassifier +from pyspark.ml.feature import MinMaxScaler, StringIndexer +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("FMClassifierExample") \ + .getOrCreate() + + # $example on$ + # Load and parse the data file, converting it to a DataFrame. + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + # Index labels, adding metadata to the label column. + # Fit on whole dataset to include all labels in index. + labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) + # Scale features. + featureScaler = MinMaxScaler(inputCol="features", outputCol="scaledFeatures").fit(data) + + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a FM model. + fm = FMClassifier(labelCol="indexedLabel", featuresCol="scaledFeatures", stepSize=0.001) + + # Create a Pipeline. + pipeline = Pipeline(stages=[labelIndexer, featureScaler, fm]) + + # Train model. + model = pipeline.fit(trainingData) + + # Make predictions. + predictions = model.transform(testData) + + # Select example rows to display. + predictions.select("prediction", "indexedLabel", "features").show(5) + + # Select (prediction, true label) and compute test accuracy + evaluator = MulticlassClassificationEvaluator( + labelCol="indexedLabel", predictionCol="prediction", metricName="accuracy") + accuracy = evaluator.evaluate(predictions) + print("Test set accuracy = %g" % accuracy) + + fmModel = model.stages[2] + print("Factors: " + str(fmModel.factors)) + print("Linear: " + str(fmModel.linear)) + print("Intercept: " + str(fmModel.intercept)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/fm_regressor_example.py b/examples/src/main/python/ml/fm_regressor_example.py new file mode 100644 index 0000000000000..afd76396800b7 --- /dev/null +++ b/examples/src/main/python/ml/fm_regressor_example.py @@ -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. +# + +""" +FMRegressor Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.regression import FMRegressor +from pyspark.ml.feature import MinMaxScaler +from pyspark.ml.evaluation import RegressionEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("FMRegressorExample") \ + .getOrCreate() + + # $example on$ + # Load and parse the data file, converting it to a DataFrame. + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + # Scale features. + featureScaler = MinMaxScaler(inputCol="features", outputCol="scaledFeatures").fit(data) + + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a FM model. + fm = FMRegressor(featuresCol="scaledFeatures", stepSize=0.001) + + # Create a Pipeline. + pipeline = Pipeline(stages=[featureScaler, fm]) + + # Train model. + model = pipeline.fit(trainingData) + + # Make predictions. + predictions = model.transform(testData) + + # Select example rows to display. + predictions.select("prediction", "label", "features").show(5) + + # Select (prediction, true label) and compute test error + evaluator = RegressionEvaluator( + labelCol="label", predictionCol="prediction", metricName="rmse") + rmse = evaluator.evaluate(predictions) + print("Root Mean Squared Error (RMSE) on test data = %g" % rmse) + + fmModel = model.stages[1] + print("Factors: " + str(fmModel.factors)) + print("Linear: " + str(fmModel.linear)) + print("Intercept: " + str(fmModel.intercept)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala new file mode 100644 index 0000000000000..612a76fd125c3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala @@ -0,0 +1,96 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.{FMClassificationModel, FMClassifier} +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +import org.apache.spark.ml.feature.{IndexToString, MinMaxScaler, StringIndexer} +// $example off$ +import org.apache.spark.sql.SparkSession + +object FMClassifierExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("FMClassifierExample") + .getOrCreate() + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Index labels, adding metadata to the label column. + // Fit on whole dataset to include all labels in index. + val labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data) + // Scale features. + val featureScaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + .fit(data) + + // Split the data into training and test sets (30% held out for testing). + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + + // Train a FM model. + val fm = new FMClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("scaledFeatures") + .setStepSize(0.001) + + // Convert indexed labels back to original labels. + val labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labelsArray(0)) + + // Create a Pipeline. + val pipeline = new Pipeline() + .setStages(Array(labelIndexer, featureScaler, fm, labelConverter)) + + // Train model. + val model = pipeline.fit(trainingData) + + // Make predictions. + val predictions = model.transform(testData) + + // Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5) + + // Select (prediction, true label) and compute test accuracy. + val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("accuracy") + val accuracy = evaluator.evaluate(predictions) + println(s"Test set accuracy = $accuracy") + + val fmModel = model.stages(2).asInstanceOf[FMClassificationModel] + println(s"Factors: ${fmModel.factors} Linear: ${fmModel.linear} " + + s"Intercept: ${fmModel.intercept}") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala new file mode 100644 index 0000000000000..6bb06eab33d9c --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.feature.MinMaxScaler +import org.apache.spark.ml.regression.{FMRegressionModel, FMRegressor} +// $example off$ +import org.apache.spark.sql.SparkSession + +object FMRegressorExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("FMRegressorExample") + .getOrCreate() + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Scale features. + val featureScaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + .fit(data) + + // Split the data into training and test sets (30% held out for testing). + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + + // Train a FM model. + val fm = new FMRegressor() + .setLabelCol("label") + .setFeaturesCol("scaledFeatures") + .setStepSize(0.001) + + // Create a Pipeline. + val pipeline = new Pipeline() + .setStages(Array(featureScaler, fm)) + + // Train model. + val model = pipeline.fit(trainingData) + + // Make predictions. + val predictions = model.transform(testData) + + // Select example rows to display. + predictions.select("prediction", "label", "features").show(5) + + // Select (prediction, true label) and compute test error. + val evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse") + val rmse = evaluator.evaluate(predictions) + println(s"Root Mean Squared Error (RMSE) on test data = $rmse") + + val fmModel = model.stages(1).asInstanceOf[FMRegressionModel] + println(s"Factors: ${fmModel.factors} Linear: ${fmModel.linear} " + + s"Intercept: ${fmModel.intercept}") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala new file mode 100644 index 0000000000000..9cb296cacb108 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala @@ -0,0 +1,332 @@ +/* + * 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.ml.classification + +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.regression.{FactorizationMachines, FactorizationMachinesParams} +import org.apache.spark.ml.regression.FactorizationMachines._ +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.Instrumentation.instrumented +import org.apache.spark.mllib.linalg.{Vector => OldVector} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.col +import org.apache.spark.storage.StorageLevel + +/** + * Params for FMClassifier. + */ +private[classification] trait FMClassifierParams extends ProbabilisticClassifierParams + with FactorizationMachinesParams { +} + +/** + * Factorization Machines learning algorithm for classification. + * It supports normal gradient descent and AdamW solver. + * + * The implementation is based upon: + * + * S. Rendle. "Factorization machines" 2010. + * + * FM is able to estimate interactions even in problems with huge sparsity + * (like advertising and recommendation system). + * FM formula is: + * {{{ + * y = \sigma\left( w_0 + \sum\limits^n_{i-1} w_i x_i + + * \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j \right) + * }}} + * First two terms denote global bias and linear term (as same as linear regression), + * and last term denotes pairwise interactions term. {{{v_i}}} describes the i-th variable + * with k factors. + * + * FM classification model uses logistic loss which can be solved by gradient descent method, and + * regularization terms like L2 are usually added to the loss function to prevent overfitting. + * + * @note Multiclass labels are not currently supported. + */ +@Since("3.0.0") +class FMClassifier @Since("3.0.0") ( + @Since("3.0.0") override val uid: String) + extends ProbabilisticClassifier[Vector, FMClassifier, FMClassificationModel] + with FactorizationMachines with FMClassifierParams with DefaultParamsWritable with Logging { + + @Since("3.0.0") + def this() = this(Identifiable.randomUID("fmc")) + + /** + * Set the dimensionality of the factors. + * Default is 8. + * + * @group setParam + */ + @Since("3.0.0") + def setFactorSize(value: Int): this.type = set(factorSize, value) + setDefault(factorSize -> 8) + + /** + * Set whether to fit intercept term. + * Default is true. + * + * @group setParam + */ + @Since("3.0.0") + def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) + setDefault(fitIntercept -> true) + + /** + * Set whether to fit linear term. + * Default is true. + * + * @group setParam + */ + @Since("3.0.0") + def setFitLinear(value: Boolean): this.type = set(fitLinear, value) + setDefault(fitLinear -> true) + + /** + * Set the L2 regularization parameter. + * Default is 0.0. + * + * @group setParam + */ + @Since("3.0.0") + def setRegParam(value: Double): this.type = set(regParam, value) + setDefault(regParam -> 0.0) + + /** + * Set the mini-batch fraction parameter. + * Default is 1.0. + * + * @group setParam + */ + @Since("3.0.0") + def setMiniBatchFraction(value: Double): this.type = set(miniBatchFraction, value) + setDefault(miniBatchFraction -> 1.0) + + /** + * Set the standard deviation of initial coefficients. + * Default is 0.01. + * + * @group setParam + */ + @Since("3.0.0") + def setInitStd(value: Double): this.type = set(initStd, value) + setDefault(initStd -> 0.01) + + /** + * Set the maximum number of iterations. + * Default is 100. + * + * @group setParam + */ + @Since("3.0.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + setDefault(maxIter -> 100) + + /** + * Set the initial step size for the first step (like learning rate). + * Default is 1.0. + * + * @group setParam + */ + @Since("3.0.0") + def setStepSize(value: Double): this.type = set(stepSize, value) + setDefault(stepSize -> 1.0) + + /** + * Set the convergence tolerance of iterations. + * Default is 1E-6. + * + * @group setParam + */ + @Since("3.0.0") + def setTol(value: Double): this.type = set(tol, value) + setDefault(tol -> 1E-6) + + /** + * Set the solver algorithm used for optimization. + * Supported options: "gd", "adamW". + * Default: "adamW" + * + * @group setParam + */ + @Since("3.0.0") + def setSolver(value: String): this.type = set(solver, value) + setDefault(solver -> AdamW) + + /** + * Set the random seed for weight initialization. + * + * @group setParam + */ + @Since("3.0.0") + def setSeed(value: Long): this.type = set(seed, value) + + override protected[spark] def train( + dataset: Dataset[_] + ): FMClassificationModel = instrumented { instr => + + val handlePersistence = dataset.storageLevel == StorageLevel.NONE + val data: RDD[(Double, OldVector)] = + dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => + require(label == 0 || label == 1, s"FMClassifier was given" + + s" dataset with invalid label $label. Labels must be in {0,1}; note that" + + s" FMClassifier currently only supports binary classification.") + (label, features) + } + + if (handlePersistence) data.persist(StorageLevel.MEMORY_AND_DISK) + + val numClasses = 2 + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + + instr.logPipelineStage(this) + instr.logDataset(dataset) + instr.logParams(this, factorSize, fitIntercept, fitLinear, regParam, + miniBatchFraction, initStd, maxIter, stepSize, tol, solver) + instr.logNumClasses(numClasses) + + val numFeatures = data.first()._2.size + instr.logNumFeatures(numFeatures) + + val coefficients = trainImpl(data, numFeatures, LogisticLoss) + + val (intercept, linear, factors) = splitCoefficients( + coefficients, numFeatures, $(factorSize), $(fitIntercept), $(fitLinear)) + + if (handlePersistence) data.unpersist() + + copyValues(new FMClassificationModel(uid, intercept, linear, factors)) + } + + @Since("3.0.0") + override def copy(extra: ParamMap): FMClassifier = defaultCopy(extra) +} + +@Since("3.0.0") +object FMClassifier extends DefaultParamsReadable[FMClassifier] { + + @Since("3.0.0") + override def load(path: String): FMClassifier = super.load(path) +} + +/** + * Model produced by [[FMClassifier]] + */ +@Since("3.0.0") +class FMClassificationModel private[classification] ( + @Since("3.0.0") override val uid: String, + @Since("3.0.0") val intercept: Double, + @Since("3.0.0") val linear: Vector, + @Since("3.0.0") val factors: Matrix) + extends ProbabilisticClassificationModel[Vector, FMClassificationModel] + with FMClassifierParams with MLWritable { + + @Since("3.0.0") + override val numClasses: Int = 2 + + @Since("3.0.0") + override val numFeatures: Int = linear.size + + override protected def predictRaw(features: Vector): Vector = { + val rawPrediction = getRawPrediction(features, intercept, linear, factors) + Vectors.dense(Array(-rawPrediction, rawPrediction)) + } + + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { + rawPrediction match { + case dv: DenseVector => + dv.values(1) = 1.0 / (1.0 + math.exp(-dv.values(1))) + dv.values(0) = 1.0 - dv.values(1) + dv + case sv: SparseVector => + throw new RuntimeException("Unexpected error in FMClassificationModel:" + + " raw2probabilityInPlace encountered SparseVector") + } + } + + @Since("3.0.0") + override def copy(extra: ParamMap): FMClassificationModel = { + copyValues(new FMClassificationModel(uid, intercept, linear, factors), extra) + } + + @Since("3.0.0") + override def write: MLWriter = + new FMClassificationModel.FMClassificationModelWriter(this) + + override def toString: String = { + s"FMClassificationModel: " + + s"uid=${super.toString}, numClasses=$numClasses, numFeatures=$numFeatures, " + + s"factorSize=${$(factorSize)}, fitLinear=${$(fitLinear)}, fitIntercept=${$(fitIntercept)}" + } +} + +@Since("3.0.0") +object FMClassificationModel extends MLReadable[FMClassificationModel] { + + @Since("3.0.0") + override def read: MLReader[FMClassificationModel] = new FMClassificationModelReader + + @Since("3.0.0") + override def load(path: String): FMClassificationModel = super.load(path) + + /** [[MLWriter]] instance for [[FMClassificationModel]] */ + private[FMClassificationModel] class FMClassificationModelWriter( + instance: FMClassificationModel) extends MLWriter with Logging { + + private case class Data( + intercept: Double, + linear: Vector, + factors: Matrix) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.intercept, instance.linear, instance.factors) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class FMClassificationModelReader extends MLReader[FMClassificationModel] { + + private val className = classOf[FMClassificationModel].getName + + override def load(path: String): FMClassificationModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.format("parquet").load(dataPath) + + val Row(intercept: Double, linear: Vector, factors: Matrix) = + data.select("intercept", "linear", "factors").head() + val model = new FMClassificationModel(metadata.uid, intercept, linear, factors) + metadata.getAndSetParams(model) + model + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala new file mode 100644 index 0000000000000..3627946becfb9 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala @@ -0,0 +1,815 @@ +/* + * 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.ml.regression + +import scala.util.Random + +import breeze.linalg.{axpy => brzAxpy, norm => brzNorm, Vector => BV} +import breeze.numerics.{sqrt => brzSqrt} +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.linalg.BLAS._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.regression.FactorizationMachines._ +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.Instrumentation.instrumented +import org.apache.spark.mllib.{linalg => OldLinalg} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.optimization.{Gradient, GradientDescent, SquaredL2Updater, Updater} +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.col +import org.apache.spark.storage.StorageLevel + +/** + * Params for Factorization Machines + */ +private[ml] trait FactorizationMachinesParams extends PredictorParams + with HasMaxIter with HasStepSize with HasTol with HasSolver with HasSeed + with HasFitIntercept with HasRegParam { + + /** + * Param for dimensionality of the factors (>= 0) + * @group param + */ + @Since("3.0.0") + final val factorSize: IntParam = new IntParam(this, "factorSize", + "Dimensionality of the factor vectors, " + + "which are used to get pairwise interactions between variables", + ParamValidators.gt(0)) + + /** @group getParam */ + @Since("3.0.0") + final def getFactorSize: Int = $(factorSize) + + /** + * Param for whether to fit linear term (aka 1-way term) + * @group param + */ + @Since("3.0.0") + final val fitLinear: BooleanParam = new BooleanParam(this, "fitLinear", + "whether to fit linear term (aka 1-way term)") + + /** @group getParam */ + @Since("3.0.0") + final def getFitLinear: Boolean = $(fitLinear) + + /** + * Param for mini-batch fraction, must be in range (0, 1] + * @group param + */ + @Since("3.0.0") + final val miniBatchFraction: DoubleParam = new DoubleParam(this, "miniBatchFraction", + "fraction of the input data set that should be used for one iteration of gradient descent", + ParamValidators.inRange(0, 1, false, true)) + + /** @group getParam */ + @Since("3.0.0") + final def getMiniBatchFraction: Double = $(miniBatchFraction) + + /** + * Param for standard deviation of initial coefficients + * @group param + */ + @Since("3.0.0") + final val initStd: DoubleParam = new DoubleParam(this, "initStd", + "standard deviation of initial coefficients", ParamValidators.gt(0)) + + /** @group getParam */ + @Since("3.0.0") + final def getInitStd: Double = $(initStd) + + /** + * The solver algorithm for optimization. + * Supported options: "gd", "adamW". + * Default: "adamW" + * + * @group param + */ + @Since("3.0.0") + final override val solver: Param[String] = new Param[String](this, "solver", + "The solver algorithm for optimization. Supported options: " + + s"${supportedSolvers.mkString(", ")}. (Default adamW)", + ParamValidators.inArray[String](supportedSolvers)) +} + +private[ml] trait FactorizationMachines extends FactorizationMachinesParams { + + private[ml] def initCoefficients(numFeatures: Int): OldVector = { + val rnd = new Random($(seed)) + val initialCoefficients = + OldVectors.dense( + Array.fill($(factorSize) * numFeatures)(rnd.nextGaussian() * $(initStd)) ++ + (if ($(fitLinear)) new Array[Double](numFeatures) else Array.emptyDoubleArray) ++ + (if ($(fitIntercept)) new Array[Double](1) else Array.emptyDoubleArray)) + initialCoefficients + } + + private[ml] def trainImpl( + data: RDD[(Double, OldVector)], + numFeatures: Int, + loss: String + ): Vector = { + + // initialize coefficients + val initialCoefficients = initCoefficients(numFeatures) + val coefficientsSize = initialCoefficients.size + + // optimize coefficients with gradient descent + val gradient = parseLoss(loss, $(factorSize), $(fitIntercept), $(fitLinear), numFeatures) + + val updater = parseSolver($(solver), coefficientsSize) + + val optimizer = new GradientDescent(gradient, updater) + .setStepSize($(stepSize)) + .setNumIterations($(maxIter)) + .setRegParam($(regParam)) + .setMiniBatchFraction($(miniBatchFraction)) + .setConvergenceTol($(tol)) + val coefficients = optimizer.optimize(data, initialCoefficients) + coefficients.asML + } +} + +private[ml] object FactorizationMachines { + + /** String name for "gd". */ + val GD = "gd" + + /** String name for "adamW". */ + val AdamW = "adamW" + + /** Set of solvers that FactorizationMachines supports. */ + val supportedSolvers = Array(GD, AdamW) + + /** String name for "logisticLoss". */ + val LogisticLoss = "logisticLoss" + + /** String name for "squaredError". */ + val SquaredError = "squaredError" + + /** Set of loss function names that FactorizationMachines supports. */ + val supportedRegressorLosses = Array(SquaredError) + val supportedClassifierLosses = Array(LogisticLoss) + val supportedLosses = supportedRegressorLosses ++ supportedClassifierLosses + + def parseSolver(solver: String, coefficientsSize: Int): Updater = { + solver match { + case GD => new SquaredL2Updater() + case AdamW => new AdamWUpdater(coefficientsSize) + } + } + + def parseLoss( + lossFunc: String, + factorSize: Int, + fitIntercept: Boolean, + fitLinear: Boolean, + numFeatures: Int + ): BaseFactorizationMachinesGradient = { + + lossFunc match { + case LogisticLoss => + new LogisticFactorizationMachinesGradient(factorSize, fitIntercept, fitLinear, numFeatures) + case SquaredError => + new MSEFactorizationMachinesGradient(factorSize, fitIntercept, fitLinear, numFeatures) + case _ => throw new IllegalArgumentException(s"loss function type $lossFunc is invalidation") + } + } + + def splitCoefficients( + coefficients: Vector, + numFeatures: Int, + factorSize: Int, + fitIntercept: Boolean, + fitLinear: Boolean + ): (Double, Vector, Matrix) = { + + val coefficientsSize = numFeatures * factorSize + + (if (fitLinear) numFeatures else 0) + (if (fitIntercept) 1 else 0) + require(coefficientsSize == coefficients.size, + s"coefficients.size did not match the excepted size ${coefficientsSize}") + + val intercept = if (fitIntercept) coefficients(coefficients.size - 1) else 0.0 + val linear: Vector = if (fitLinear) { + new DenseVector(coefficients.toArray.slice( + numFeatures * factorSize, numFeatures * factorSize + numFeatures)) + } else { + Vectors.sparse(numFeatures, Seq.empty) + } + val factors = new DenseMatrix(numFeatures, factorSize, + coefficients.toArray.slice(0, numFeatures * factorSize), true) + (intercept, linear, factors) + } + + def combineCoefficients( + intercept: Double, + linear: Vector, + factors: Matrix, + fitIntercept: Boolean, + fitLinear: Boolean + ): Vector = { + + val coefficients = factors.toDense.values ++ + (if (fitLinear) linear.toArray else Array.emptyDoubleArray) ++ + (if (fitIntercept) Array(intercept) else Array.emptyDoubleArray) + new DenseVector(coefficients) + } + + def getRawPrediction( + features: Vector, + intercept: Double, + linear: Vector, + factors: Matrix + ): Double = { + var rawPrediction = intercept + features.dot(linear) + (0 until factors.numCols).foreach { f => + var sumSquare = 0.0 + var sum = 0.0 + features.foreachActive { case (index, value) => + val vx = factors(index, f) * value + sumSquare += vx * vx + sum += vx + } + rawPrediction += 0.5 * (sum * sum - sumSquare) + } + + rawPrediction + } +} + +/** + * Params for FMRegressor + */ +private[regression] trait FMRegressorParams extends FactorizationMachinesParams { +} + +/** + * Factorization Machines learning algorithm for regression. + * It supports normal gradient descent and AdamW solver. + * + * The implementation is based upon: + * + * S. Rendle. "Factorization machines" 2010. + * + * FM is able to estimate interactions even in problems with huge sparsity + * (like advertising and recommendation system). + * FM formula is: + * {{{ + * y = w_0 + \sum\limits^n_{i-1} w_i x_i + + * \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j + * }}} + * First two terms denote global bias and linear term (as same as linear regression), + * and last term denotes pairwise interactions term. {{{v_i}}} describes the i-th variable + * with k factors. + * + * FM regression model uses MSE loss which can be solved by gradient descent method, and + * regularization terms like L2 are usually added to the loss function to prevent overfitting. + */ +@Since("3.0.0") +class FMRegressor @Since("3.0.0") ( + @Since("3.0.0") override val uid: String) + extends Predictor[Vector, FMRegressor, FMRegressionModel] + with FactorizationMachines with FMRegressorParams with DefaultParamsWritable with Logging { + + @Since("3.0.0") + def this() = this(Identifiable.randomUID("fmr")) + + /** + * Set the dimensionality of the factors. + * Default is 8. + * + * @group setParam + */ + @Since("3.0.0") + def setFactorSize(value: Int): this.type = set(factorSize, value) + setDefault(factorSize -> 8) + + /** + * Set whether to fit intercept term. + * Default is true. + * + * @group setParam + */ + @Since("3.0.0") + def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) + setDefault(fitIntercept -> true) + + /** + * Set whether to fit linear term. + * Default is true. + * + * @group setParam + */ + @Since("3.0.0") + def setFitLinear(value: Boolean): this.type = set(fitLinear, value) + setDefault(fitLinear -> true) + + /** + * Set the L2 regularization parameter. + * Default is 0.0. + * + * @group setParam + */ + @Since("3.0.0") + def setRegParam(value: Double): this.type = set(regParam, value) + setDefault(regParam -> 0.0) + + /** + * Set the mini-batch fraction parameter. + * Default is 1.0. + * + * @group setParam + */ + @Since("3.0.0") + def setMiniBatchFraction(value: Double): this.type = set(miniBatchFraction, value) + setDefault(miniBatchFraction -> 1.0) + + /** + * Set the standard deviation of initial coefficients. + * Default is 0.01. + * + * @group setParam + */ + @Since("3.0.0") + def setInitStd(value: Double): this.type = set(initStd, value) + setDefault(initStd -> 0.01) + + /** + * Set the maximum number of iterations. + * Default is 100. + * + * @group setParam + */ + @Since("3.0.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + setDefault(maxIter -> 100) + + /** + * Set the initial step size for the first step (like learning rate). + * Default is 1.0. + * + * @group setParam + */ + @Since("3.0.0") + def setStepSize(value: Double): this.type = set(stepSize, value) + setDefault(stepSize -> 1.0) + + /** + * Set the convergence tolerance of iterations. + * Default is 1E-6. + * + * @group setParam + */ + @Since("3.0.0") + def setTol(value: Double): this.type = set(tol, value) + setDefault(tol -> 1E-6) + + /** + * Set the solver algorithm used for optimization. + * Supported options: "gd", "adamW". + * Default: "adamW" + * + * @group setParam + */ + @Since("3.0.0") + def setSolver(value: String): this.type = set(solver, value) + setDefault(solver -> AdamW) + + /** + * Set the random seed for weight initialization. + * + * @group setParam + */ + @Since("3.0.0") + def setSeed(value: Long): this.type = set(seed, value) + + override protected[spark] def train( + dataset: Dataset[_] + ): FMRegressionModel = instrumented { instr => + + val handlePersistence = dataset.storageLevel == StorageLevel.NONE + val data: RDD[(Double, OldVector)] = + dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => + (label, features) + } + + if (handlePersistence) data.persist(StorageLevel.MEMORY_AND_DISK) + + instr.logPipelineStage(this) + instr.logDataset(dataset) + instr.logParams(this, factorSize, fitIntercept, fitLinear, regParam, + miniBatchFraction, initStd, maxIter, stepSize, tol, solver) + + val numFeatures = data.first()._2.size + instr.logNumFeatures(numFeatures) + + val coefficients = trainImpl(data, numFeatures, SquaredError) + + val (intercept, linear, factors) = splitCoefficients( + coefficients, numFeatures, $(factorSize), $(fitIntercept), $(fitLinear)) + + if (handlePersistence) data.unpersist() + + copyValues(new FMRegressionModel(uid, intercept, linear, factors)) + } + + @Since("3.0.0") + override def copy(extra: ParamMap): FMRegressor = defaultCopy(extra) +} + +@Since("3.0.0") +object FMRegressor extends DefaultParamsReadable[FMRegressor] { + + @Since("3.0.0") + override def load(path: String): FMRegressor = super.load(path) +} + +/** + * Model produced by [[FMRegressor]]. + */ +@Since("3.0.0") +class FMRegressionModel private[regression] ( + @Since("3.0.0") override val uid: String, + @Since("3.0.0") val intercept: Double, + @Since("3.0.0") val linear: Vector, + @Since("3.0.0") val factors: Matrix) + extends PredictionModel[Vector, FMRegressionModel] + with FMRegressorParams with MLWritable { + + @Since("3.0.0") + override val numFeatures: Int = linear.size + + override def predict(features: Vector): Double = { + getRawPrediction(features, intercept, linear, factors) + } + + @Since("3.0.0") + override def copy(extra: ParamMap): FMRegressionModel = { + copyValues(new FMRegressionModel(uid, intercept, linear, factors), extra) + } + + @Since("3.0.0") + override def write: MLWriter = + new FMRegressionModel.FMRegressionModelWriter(this) + + override def toString: String = { + s"FMRegressionModel: " + + s"uid=${super.toString}, numFeatures=$numFeatures, " + + s"factorSize=${$(factorSize)}, fitLinear=${$(fitLinear)}, fitIntercept=${$(fitIntercept)}" + } +} + +@Since("3.0.0") +object FMRegressionModel extends MLReadable[FMRegressionModel] { + + @Since("3.0.0") + override def read: MLReader[FMRegressionModel] = new FMRegressionModelReader + + @Since("3.0.0") + override def load(path: String): FMRegressionModel = super.load(path) + + /** [[MLWriter]] instance for [[FMRegressionModel]] */ + private[FMRegressionModel] class FMRegressionModelWriter( + instance: FMRegressionModel) extends MLWriter with Logging { + + private case class Data( + intercept: Double, + linear: Vector, + factors: Matrix) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.intercept, instance.linear, instance.factors) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class FMRegressionModelReader extends MLReader[FMRegressionModel] { + + private val className = classOf[FMRegressionModel].getName + + override def load(path: String): FMRegressionModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.format("parquet").load(dataPath) + + val Row(intercept: Double, linear: Vector, factors: Matrix) = data + .select("intercept", "linear", "factors").head() + val model = new FMRegressionModel(metadata.uid, intercept, linear, factors) + metadata.getAndSetParams(model) + model + } + } +} + +/** + * Factorization Machines base gradient class + * Implementing the raw FM formula, include raw prediction and raw gradient, + * then inherit the base class to implement special gradient class(like logloss, mse). + * + * Factorization Machines raw formula: + * {{{ + * y_{fm} = w_0 + \sum\limits^n_{i-1} w_i x_i + + * \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j + * }}} + * the pairwise interactions (2-way term) can be reformulated: + * {{{ + * \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j + * = \frac{1}{2}\sum\limits^k_{f=1} + * \left(\left( \sum\limits^n_{i=1}v_{i,f}x_i \right)^2 - + * \sum\limits^n_{i=1}v_{i,f}^2x_i^2 \right) + * }}} + * and the gradients are: + * {{{ + * \frac{\partial}{\partial\theta}y_{fm} = \left\{ + * \begin{align} + * &1, & if\ \theta\ is\ w_0 \\ + * &x_i, & if\ \theta\ is\ w_i \\ + * &x_i{\sum}^n_{j=1}v_{j,f}x_j - v_{i,f}x_i^2, & if\ \theta\ is\ v_{i,j} \\ + * \end{align} + * \right. + * }}} + * + * Factorization Machines formula with prediction task: + * {{{ + * \hat{y} = p\left( y_{fm} \right) + * }}} + * p is the prediction function, for binary classification task is sigmoid. + * The loss funcation gradient formula: + * {{{ + * \frac{\partial}{\partial\theta} l\left( \hat{y},y \right) = + * \frac{\partial}{\partial\theta} l\left( p\left( y_{fm} \right),y \right) = + * \frac{\partial l}{\partial \hat{y}} \cdot + * \frac{\partial \hat{y}}{\partial y_{fm}} \cdot + * \frac{\partial y_{fm}}{\partial\theta} + * }}} + * Last term is same for all task, so be implemented in base gradient class. + * last term named rawGradient in following code, and first two term named multiplier. + */ +private[ml] abstract class BaseFactorizationMachinesGradient( + factorSize: Int, + fitIntercept: Boolean, + fitLinear: Boolean, + numFeatures: Int) extends Gradient { + + override def compute( + data: OldVector, + label: Double, + weights: OldVector, + cumGradient: OldVector): Double = { + val (rawPrediction, sumVX) = getRawPrediction(data, weights) + val rawGradient = getRawGradient(data, weights, sumVX) + val multiplier = getMultiplier(rawPrediction, label) + axpy(multiplier, rawGradient, cumGradient) + val loss = getLoss(rawPrediction, label) + loss + } + + def getPrediction(rawPrediction: Double): Double + + protected def getMultiplier(rawPrediction: Double, label: Double): Double + + protected def getLoss(rawPrediction: Double, label: Double): Double + + def getRawPrediction(data: OldVector, weights: OldVector): (Double, Array[Double]) = { + val sumVX = new Array[Double](factorSize) + var rawPrediction = 0.0 + val vWeightsSize = numFeatures * factorSize + + if (fitIntercept) rawPrediction += weights(weights.size - 1) + if (fitLinear) { + data.foreachActive { case (index, value) => + rawPrediction += weights(vWeightsSize + index) * value + } + } + (0 until factorSize).foreach { f => + var sumSquare = 0.0 + var sum = 0.0 + data.foreachActive { case (index, value) => + val vx = weights(index * factorSize + f) * value + sumSquare += vx * vx + sum += vx + } + sumVX(f) = sum + rawPrediction += 0.5 * (sum * sum - sumSquare) + } + + (rawPrediction, sumVX) + } + + private def getRawGradient( + data: OldVector, + weights: OldVector, + sumVX: Array[Double] + ): OldVector = { + data match { + // Usually Factorization Machines is used, there will be a lot of sparse features. + // So need to optimize the gradient descent of sparse vector. + case data: OldLinalg.SparseVector => + val gardSize = data.indices.length * factorSize + + (if (fitLinear) data.indices.length else 0) + + (if (fitIntercept) 1 else 0) + val gradIndex = Array.fill(gardSize)(0) + val gradValue = Array.fill(gardSize)(0.0) + var gradI = 0 + val vWeightsSize = numFeatures * factorSize + + data.foreachActive { case (index, value) => + (0 until factorSize).foreach { f => + gradIndex(gradI) = index * factorSize + f + gradValue(gradI) = value * sumVX(f) - weights(index * factorSize + f) * value * value + gradI += 1 + } + } + if (fitLinear) { + data.foreachActive { case (index, value) => + gradIndex(gradI) = vWeightsSize + index + gradValue(gradI) = value + gradI += 1 + } + } + if (fitIntercept) { + gradIndex(gradI) = weights.size - 1 + gradValue(gradI) = 1.0 + } + + OldVectors.sparse(weights.size, gradIndex, gradValue) + case data: OldLinalg.DenseVector => + val gradient = Array.fill(weights.size)(0.0) + val vWeightsSize = numFeatures * factorSize + + if (fitIntercept) gradient(weights.size - 1) += 1.0 + if (fitLinear) { + data.foreachActive { case (index, value) => + gradient(vWeightsSize + index) += value + } + } + (0 until factorSize).foreach { f => + data.foreachActive { case (index, value) => + gradient(index * factorSize + f) += + value * sumVX(f) - weights(index * factorSize + f) * value * value + } + } + + OldVectors.dense(gradient) + } + } +} + +/** + * FM with logistic loss + * prediction formula: + * {{{ + * \hat{y} = \sigmoid(y_{fm}) + * }}} + * loss formula: + * {{{ + * - y * log(\hat{y}) - (1 - y) * log(1 - \hat{y}) + * }}} + * multiplier formula: + * {{{ + * \frac{\partial l}{\partial \hat{y}} \cdot + * \frac{\partial \hat{y}}{\partial y_{fm}} = + * \hat{y} - y + * }}} + */ +private[ml] class LogisticFactorizationMachinesGradient( + factorSize: Int, + fitIntercept: Boolean, + fitLinear: Boolean, + numFeatures: Int) + extends BaseFactorizationMachinesGradient( + factorSize: Int, + fitIntercept: Boolean, + fitLinear: Boolean, + numFeatures: Int) with Logging { + + override def getPrediction(rawPrediction: Double): Double = { + 1.0 / (1.0 + math.exp(-rawPrediction)) + } + + override protected def getMultiplier(rawPrediction: Double, label: Double): Double = { + getPrediction(rawPrediction) - label + } + + override protected def getLoss(rawPrediction: Double, label: Double): Double = { + if (label > 0) MLUtils.log1pExp(-rawPrediction) + else MLUtils.log1pExp(rawPrediction) + } +} + +/** + * FM with mse + * prediction formula: + * {{{ + * \hat{y} = y_{fm} + * }}} + * loss formula: + * {{{ + * (\hat{y} - y) ^ 2 + * }}} + * multiplier formula: + * {{{ + * \frac{\partial l}{\partial \hat{y}} \cdot + * \frac{\partial \hat{y}}{\partial y_{fm}} = + * 2 * (\hat{y} - y) + * }}} + */ +private[ml] class MSEFactorizationMachinesGradient( + factorSize: Int, + fitIntercept: Boolean, + fitLinear: Boolean, + numFeatures: Int) + extends BaseFactorizationMachinesGradient( + factorSize: Int, + fitIntercept: Boolean, + fitLinear: Boolean, + numFeatures: Int) with Logging { + + override def getPrediction(rawPrediction: Double): Double = { + rawPrediction + } + + override protected def getMultiplier(rawPrediction: Double, label: Double): Double = { + 2 * (rawPrediction - label) + } + + override protected def getLoss(rawPrediction: Double, label: Double): Double = { + (rawPrediction - label) * (rawPrediction - label) + } +} + +/** + * AdamW optimizer. + * + * The implementation is based upon: + * + * Loshchilov I, Hutter F. "DECOUPLED WEIGHT DECAY REGULARIZATION" 2019. + * + * The main contribution of this paper is to improve regularization in Adam + * by decoupling the weight decay from the gradient-based update. + * This paper proposed a simple modification to recover the original formulation of + * weight decay regularization by decoupling the weight decay from the optimization steps + * taken w.r.t. the loss function. + */ +private[ml] class AdamWUpdater(weightSize: Int) extends Updater with Logging { + val beta1: Double = 0.9 + val beta2: Double = 0.999 + val epsilon: Double = 1e-8 + + val m: BV[Double] = BV.zeros[Double](weightSize).toDenseVector + val v: BV[Double] = BV.zeros[Double](weightSize).toDenseVector + var beta1T: Double = 1.0 + var beta2T: Double = 1.0 + + override def compute( + weightsOld: OldVector, + gradient: OldVector, + stepSize: Double, + iter: Int, + regParam: Double + ): (OldVector, Double) = { + val w: BV[Double] = weightsOld.asBreeze.toDenseVector + val lr = stepSize // learning rate + if (stepSize > 0) { + val g: BV[Double] = gradient.asBreeze.toDenseVector + m *= beta1 + brzAxpy(1 - beta1, g, m) + v *= beta2 + brzAxpy(1 - beta2, g * g, v) + beta1T *= beta1 + beta2T *= beta2 + val mHat = m / (1 - beta1T) + val vHat = v / (1 - beta2T) + w -= lr * mHat / (brzSqrt(vHat) + epsilon) + regParam * w + } + val norm = brzNorm(w, 2.0) + + (Vectors.fromBreeze(w), 0.5 * regParam * norm * norm) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala new file mode 100644 index 0000000000000..d477049824b19 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala @@ -0,0 +1,242 @@ +/* + * 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.ml.classification + +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.regression.FMRegressorSuite._ +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.sql.{DataFrame, Row} + +class FMClassifierSuite extends MLTest with DefaultReadWriteTest { + + import testImplicits._ + + private val seed = 42 + @transient var smallBinaryDataset: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + smallBinaryDataset = generateLogisticInput(1.0, 1.0, nPoints = 100, seed = seed).toDF() + } + + test("params") { + ParamsSuite.checkParams(new FMClassifier) + val model = new FMClassificationModel("fmc_test", 0.0, Vectors.dense(0.0), + new DenseMatrix(1, 8, new Array[Double](8))) + ParamsSuite.checkParams(model) + } + + test("FMClassifier: Predictor, Classifier methods") { + val sqlContext = smallBinaryDataset.sqlContext + import sqlContext.implicits._ + val fm = new FMClassifier() + + val model = fm.fit(smallBinaryDataset) + assert(model.numClasses === 2) + val numFeatures = smallBinaryDataset.select("features").first().getAs[Vector](0).size + assert(model.numFeatures === numFeatures) + + testTransformer[(Double, Vector)](smallBinaryDataset.toDF(), + model, "rawPrediction", "probability", "prediction") { + case Row(raw: Vector, prob: Vector, pred: Double) => + // Compare rawPrediction with probability + assert(raw.size === 2) + assert(prob.size === 2) + val probFromRaw1 = 1.0 / (1.0 + math.exp(-raw(1))) + assert(prob(1) ~== probFromRaw1 relTol 1E-6) + assert(prob(0) ~== 1.0 - probFromRaw1 relTol 1E-6) + // Compare prediction with probability + val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 + assert(pred == predFromProb) + } + + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, FMClassificationModel](this, model, smallBinaryDataset) + } + + def logLoss(modelRes: DataFrame): Double = { + modelRes.select("label", "probability").rdd.map { + case Row(label: Double, probability: DenseVector) => + if (label > 0) -math.log(probability(1)) + else -math.log(probability(0)) + }.mean() + } + + test("check logisticLoss with AdamW") { + // This testcase only tests whether the FM logloss part is valid and does not test the + // pairwise interaction logic. The pairwise interaction logic be tested in FMRegressor. + // When there is only one feature, FM will degenerate into LR. So two models will get + // almost same loss value. + + val fm = new FMClassifier().setMaxIter(50) + val fmModel = fm.fit(smallBinaryDataset) + val fmRes = fmModel.transform(smallBinaryDataset) + val fmLogLoss = logLoss(fmRes) + + /* + Use following code to fit the dataset, the resulting logloss is 0.4756465459065247. + val lr = new LogisticRegression() + val lrModel = lr.fit(smallBinaryDataset) + val lrRes = lrModel.transform(smallBinaryDataset) + val lrLogLoss = logLoss(lrRes) + */ + assert(fmLogLoss ~== 0.4756465459065247 absTol 1E-3) + } + + test("check logisticLoss with GD") { + val fm = new FMClassifier().setSolver("gd") + val fmModel = fm.fit(smallBinaryDataset) + val fmRes = fmModel.transform(smallBinaryDataset) + val fmLogLoss = logLoss(fmRes) + assert(fmLogLoss ~== 0.4756465459065247 absTol 1E-2) + } + + test("sparse datasets") { + // test sparse input will not throw exception + val dataset = spark.createDataFrame(Array( + (1.0, Vectors.dense(Array(1.0, 2.0, 3.0))), + (0.0, Vectors.sparse(3, Array(0, 2), Array(-1.0, 2.0))), + (0.0, Vectors.sparse(3, Array.emptyIntArray, Array.emptyDoubleArray)), + (1.0, Vectors.sparse(3, Array(0, 1), Array(2.0, 3.0))) + )).toDF("label", "features") + val fm = new FMClassifier().setMaxIter(10) + fm.fit(dataset) + } + + test("setThreshold, getThreshold") { + val fm = new FMClassifier() + + // default + withClue("FMClassifier should not have thresholds set by default.") { + intercept[NoSuchElementException] { + fm.getThresholds + } + } + + // Set via thresholds + val fm2 = new FMClassifier() + val threshold = Array(0.3, 0.7) + fm2.setThresholds(threshold) + assert(fm2.getThresholds === threshold) + } + + test("thresholds prediction") { + val fm = new FMClassifier() + val df = smallBinaryDataset.toDF() + val fmModel = fm.fit(df) + + // should predict all zeros + fmModel.setThresholds(Array(0.0, 1.0)) + testTransformer[(Double, Vector)](df, fmModel, "prediction") { + case Row(prediction: Double) => prediction === 0.0 + } + + // should predict all ones + fmModel.setThresholds(Array(1.0, 0.0)) + testTransformer[(Double, Vector)](df, fmModel, "prediction") { + case Row(prediction: Double) => prediction === 1.0 + } + + val fmBase = new FMClassifier() + val model = fmBase.fit(df) + val basePredictions = model.transform(df).select("prediction").collect() + + // constant threshold scaling is the same as no thresholds + fmModel.setThresholds(Array(1.0, 1.0)) + testTransformerByGlobalCheckFunc[(Double, Vector)](df, fmModel, "prediction") { + scaledPredictions: Seq[Row] => + assert(scaledPredictions.zip(basePredictions).forall { case (scaled, base) => + scaled.getDouble(0) === base.getDouble(0) + }) + } + + // force it to use the predict method + model.setRawPredictionCol("").setProbabilityCol("").setThresholds(Array(0, 1)) + testTransformer[(Double, Vector)](df, model, "prediction") { + case Row(prediction: Double) => prediction === 0.0 + } + } + + test("FMClassifier doesn't fit intercept when fitIntercept is off") { + val fm = new FMClassifier().setFitIntercept(false) + val model = fm.fit(smallBinaryDataset) + assert(model.intercept === 0.0) + } + + test("FMClassifier doesn't fit linear when fitLinear is off") { + val fm = new FMClassifier().setFitLinear(false) + val model = fm.fit(smallBinaryDataset) + assert(model.linear === Vectors.sparse(model.numFeatures, Seq.empty)) + } + + test("prediction on single instance") { + val fm = new FMClassifier() + val fmModel = fm.fit(smallBinaryDataset) + testPredictionModelSinglePrediction(fmModel, smallBinaryDataset) + } + + test("read/write") { + def checkModelData( + model: FMClassificationModel, + model2: FMClassificationModel + ): Unit = { + assert(model.intercept === model2.intercept) + assert(model.linear.toArray === model2.linear.toArray) + assert(model.factors.toArray === model2.factors.toArray) + assert(model.numFeatures === model2.numFeatures) + } + val fm = new FMClassifier() + val data = smallBinaryDataset + .withColumnRenamed("features", allParamSettings("featuresCol").toString) + .withColumnRenamed("label", allParamSettings("labelCol").toString) + testEstimatorAndModelReadWrite(fm, data, allParamSettings, + allParamSettings, checkModelData) + } +} + +object FMClassifierSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "featuresCol" -> "myFeatures", + "labelCol" -> "myLabel", + "predictionCol" -> "prediction", + "rawPredictionCol" -> "rawPrediction", + "probabilityCol" -> "probability", + "factorSize" -> 4, + "fitIntercept" -> false, + "fitLinear" -> false, + "regParam" -> 0.01, + "miniBatchFraction" -> 0.1, + "initStd" -> 0.01, + "maxIter" -> 2, + "stepSize" -> 0.1, + "tol" -> 1e-4, + "solver" -> "gd", + "seed" -> 10L, + "thresholds" -> Array(0.4, 0.6) + ) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/FMRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/FMRegressorSuite.scala new file mode 100644 index 0000000000000..372432ceb3a41 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/FMRegressorSuite.scala @@ -0,0 +1,240 @@ +/* + * 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.ml.regression + +import scala.util.Random + +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.regression.FactorizationMachines._ +import org.apache.spark.ml.regression.FMRegressorSuite._ +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.functions.{avg, col} + +class FMRegressorSuite extends MLTest with DefaultReadWriteTest { + + private val seed = 10 + @transient var crossDataset: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val (crossDatasetTmp, _) = generateFactorInteractionInput( + spark, 2, 10, 1000, seed, true, true) + crossDataset = crossDatasetTmp + } + + test("params") { + ParamsSuite.checkParams(new FMRegressor) + val model = new FMRegressionModel("fmr_test", 0.0, Vectors.dense(0.0), + new DenseMatrix(1, 8, new Array[Double](8))) + ParamsSuite.checkParams(model) + } + + test("combineCoefficients") { + val numFeatures = 2 + val factorSize = 4 + val b = 0.1 + val w = Vectors.dense(Array(0.2, 0.3)) + val v = new DenseMatrix(numFeatures, factorSize, + Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1), true) + + val expectList = Array( + (true, true, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.2, 0.3, 0.1)), + (false, true, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.2, 0.3)), + (true, false, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.1)), + (false, false, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1))) + + expectList.foreach { case (fitIntercept, fitLinear, expectCoeffs) => + assert(combineCoefficients(b, w, v, fitIntercept, fitLinear) === Vectors.dense(expectCoeffs)) + } + } + + test("splitCoefficients") { + val numFeatures = 2 + val factorSize = 4 + val b = 0.1 + val w = Vectors.dense(Array(0.2, 0.3)) + val v = new DenseMatrix(numFeatures, factorSize, + Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1), true) + val emptyB = 0.0 + val emptyW = Vectors.sparse(numFeatures, Seq.empty) + + val expectList = Array( + (true, true, b, w, v, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.2, 0.3, 0.1)), + (false, true, emptyB, w, v, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.2, 0.3)), + (true, false, b, emptyW, v, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.1)), + (false, false, emptyB, emptyW, v, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1))) + + expectList.foreach { case (fitIntercept, fitLinear, b1, w1, v1, coeffs) => + val (b2, w2, v2) = splitCoefficients(Vectors.dense(coeffs), + numFeatures, factorSize, fitIntercept, fitLinear) + assert(b1 === b2) + assert(w1 === w2) + assert(v1 === v2) + } + } + + def checkMSE(fitIntercept: Boolean, fitLinear: Boolean): Unit = { + val numFeatures = 3 + val numSamples = 200 + val factorSize = 2 + val (data, coefficients) = generateFactorInteractionInput( + spark, factorSize, numFeatures, numSamples, seed, fitIntercept, fitLinear) + val (b, w, v) = splitCoefficients(new DenseVector(coefficients), + numFeatures, factorSize, fitIntercept, fitLinear) + + val fm = new FMRegressor() + .setSolver("adamW") + .setFeaturesCol("features") + .setLabelCol("label") + .setFactorSize(factorSize) + .setFitIntercept(fitIntercept) + .setFitLinear(fitLinear) + .setInitStd(0.01) + .setMaxIter(100) + .setMiniBatchFraction(1.0) + .setStepSize(1.0) + .setRegParam(0.0) + .setTol(1E-6) + val fmModel = fm.fit(data) + val res = fmModel.transform(data) + + // check mse value + val mse = res.select((col("prediction") - col("label")).as("error")) + .select((col("error") * col("error")).as("error_square")) + .agg(avg("error_square")) + .collect()(0).getAs[Double](0) + assert(mse ~== 0.0 absTol 1E-4) + + // check coefficients + assert(b ~== fmModel.intercept absTol 1E-2) + assert(w ~== fmModel.linear absTol 1E-2) + (0 until numFeatures).foreach { i => + ((i + 1) until numFeatures).foreach { j => + // assert is same + var innerProd1 = 0.0 + var innerProd2 = 0.0 + (0 until factorSize).foreach { k => + innerProd1 += v(i, k) * v(j, k) + innerProd2 += fmModel.factors(i, k) * fmModel.factors(j, k) + } + assert(innerProd1 ~== innerProd2 absTol 1E-2) + } + } + } + + test("MSE with intercept and linear") { + checkMSE(true, true) + } + + test("MSE with intercept but without linear") { + checkMSE(true, false) + } + + test("MSE with linear but without intercept") { + checkMSE(false, true) + } + + test("MSE without intercept or linear") { + checkMSE(false, false) + } + + test("read/write") { + def checkModelData( + model: FMRegressionModel, + model2: FMRegressionModel + ): Unit = { + assert(model.intercept === model2.intercept) + assert(model.linear.toArray === model2.linear.toArray) + assert(model.factors.toArray === model2.factors.toArray) + assert(model.numFeatures === model2.numFeatures) + } + val fm = new FMRegressor() + val data = crossDataset + .withColumnRenamed("features", allParamSettings("featuresCol").toString) + .withColumnRenamed("label", allParamSettings("labelCol").toString) + testEstimatorAndModelReadWrite(fm, data, allParamSettings, + allParamSettings, checkModelData) + } +} + +object FMRegressorSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "featuresCol" -> "myFeatures", + "labelCol" -> "myLabel", + "predictionCol" -> "prediction", + "factorSize" -> 2, + "fitIntercept" -> false, + "fitLinear" -> false, + "regParam" -> 0.01, + "miniBatchFraction" -> 0.1, + "initStd" -> 0.01, + "maxIter" -> 2, + "stepSize" -> 0.1, + "tol" -> 1e-4, + "solver" -> "gd", + "seed" -> 11L + ) + + def generateFactorInteractionInput( + spark: SparkSession, + factorSize: Int, + numFeatures: Int, + numSamples: Int, + seed: Int, + fitIntercept: Boolean, + fitLinear: Boolean + ): (DataFrame, Array[Double]) = { + import spark.implicits._ + val sc = spark.sparkContext + + // generate FM coefficients randomly + val rnd = new Random(seed) + val coefficientsSize = factorSize * numFeatures + + (if (fitLinear) numFeatures else 0) + (if (fitIntercept) 1 else 0) + val coefficients = Array.fill(coefficientsSize)(rnd.nextDouble() - 0.5) + val (intercept, linear, factors) = splitCoefficients( + Vectors.dense(coefficients), numFeatures, factorSize, fitIntercept, fitLinear) + + // generate samples randomly + val X: DataFrame = sc.parallelize(0 until numSamples).map { i => + val x = new DenseVector(Array.fill(numFeatures)(rnd.nextDouble() - 0.5)) + (i, x) + }.toDF("id", "features") + + // calculate FM prediction + val fmModel = new FMRegressionModel( + "fmr_test", intercept, linear, factors) + fmModel.set(fmModel.factorSize, factorSize) + fmModel.set(fmModel.fitIntercept, fitIntercept) + fmModel.set(fmModel.fitLinear, fitLinear) + val data = fmModel.transform(X) + .withColumnRenamed("prediction", "label") + .select("features", "label") + (data, coefficients) + } +} diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index a615ca2cde94a..10155b2a287d8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -45,7 +45,8 @@ 'RandomForestClassifier', 'RandomForestClassificationModel', 'NaiveBayes', 'NaiveBayesModel', 'MultilayerPerceptronClassifier', 'MultilayerPerceptronClassificationModel', - 'OneVsRest', 'OneVsRestModel'] + 'OneVsRest', 'OneVsRestModel', + 'FMClassifier', 'FMClassificationModel'] class _JavaClassifierParams(HasRawPredictionCol, _JavaPredictorParams): @@ -2763,6 +2764,169 @@ def _to_java(self): return _java_obj +@inherit_doc +class FMClassifier(JavaProbabilisticClassifier, HasMaxIter, HasStepSize, HasTol, HasSolver, + HasSeed, HasFitIntercept, HasRegParam, JavaMLWritable, JavaMLReadable): + """ + Factorization Machines learning algorithm for classification. + + solver Supports: + + * gd (normal mini-batch gradient descent) + * adamW (default) + + >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.ml.classification import FMClassifier + >>> df = spark.createDataFrame([ + ... (1.0, Vectors.dense(1.0)), + ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) + >>> fm = FMClassifier(factorSize=2) + >>> model = fm.fit(df) + >>> test0 = spark.createDataFrame([ + ... (Vectors.dense(-1.0),), + ... (Vectors.dense(0.5),), + ... (Vectors.dense(1.0),), + ... (Vectors.dense(2.0),)], ["features"]) + >>> model.transform(test0).select("features", "probability").show(10, False) + +--------+------------------------------------------+ + |features|probability | + +--------+------------------------------------------+ + |[-1.0] |[0.9999999997574736,2.425264676902229E-10]| + |[0.5] |[0.47627851732981163,0.5237214826701884] | + |[1.0] |[5.491554426243495E-4,0.9994508445573757] | + |[2.0] |[2.005766663870645E-10,0.9999999997994233]| + +--------+------------------------------------------+ + >>> model.intercept + -7.316665276826291 + >>> model.linear + DenseVector([14.8232]) + >>> model.factors + DenseMatrix(1, 2, [0.0028, 0.0048], 1) + + .. versionadded:: 3.0.0 + """ + + factorSize = Param(Params._dummy(), "factorSize", "Dimensionality of the factor vectors, " + + "which are used to get pairwise interactions between variables", + typeConverter=TypeConverters.toInt) + + fitLinear = Param(Params._dummy(), "fitLinear", "whether to fit linear term (aka 1-way term)", + typeConverter=TypeConverters.toBoolean) + + miniBatchFraction = Param(Params._dummy(), "miniBatchFraction", "fraction of the input data " + + "set that should be used for one iteration of gradient descent", + typeConverter=TypeConverters.toFloat) + + initStd = Param(Params._dummy(), "initStd", "standard deviation of initial coefficients", + typeConverter=TypeConverters.toFloat) + + solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + + "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) + + @keyword_only + def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", + probabilityCol="probability", rawPredictionCol="rawPrediction", + factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, + tol=1e-6, solver="adamW", thresholds=None, seed=None): + """ + __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ + probabilityCol="probability", rawPredictionCol="rawPrediction", \ + factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, \ + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, \ + tol=1e-6, solver="adamW", thresholds=None, seed=None) + """ + super(FMClassifier, self).__init__() + self._java_obj = self._new_java_obj( + "org.apache.spark.ml.classification.FMClassifier", self.uid) + self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, + tol=1e-6, solver="adamW") + kwargs = self._input_kwargs + self.setParams(**kwargs) + + @keyword_only + @since("3.0.0") + def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", + probabilityCol="probability", rawPredictionCol="rawPrediction", + factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, + tol=1e-6, solver="adamW", thresholds=None, seed=None): + """ + setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ + probabilityCol="probability", rawPredictionCol="rawPrediction", \ + factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, \ + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, \ + tol=1e-6, solver="adamW", thresholds=None, seed=None) + Sets Params for FMClassifier. + """ + kwargs = self._input_kwargs + return self._set(**kwargs) + + def _create_model(self, java_model): + return FMClassificationModel(java_model) + + @since("3.0.0") + def setFactorSize(self, value): + """ + Sets the value of :py:attr:`factorSize`. + """ + return self._set(factorSize=value) + + @since("3.0.0") + def setFitLinear(self, value): + """ + Sets the value of :py:attr:`fitLinear`. + """ + return self._set(fitLinear=value) + + @since("3.0.0") + def setMiniBatchFraction(self, value): + """ + Sets the value of :py:attr:`miniBatchFraction`. + """ + return self._set(miniBatchFraction=value) + + @since("3.0.0") + def setInitStd(self, value): + """ + Sets the value of :py:attr:`initStd`. + """ + return self._set(initStd=value) + + +class FMClassificationModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable): + """ + Model fitted by :class:`FMClassifier`. + + .. versionadded:: 3.0.0 + """ + + @property + @since("3.0.0") + def intercept(self): + """ + Model intercept. + """ + return self._call_java("intercept") + + @property + @since("3.0.0") + def linear(self): + """ + Model linear term. + """ + return self._call_java("linear") + + @property + @since("3.0.0") + def factors(self): + """ + Model factor term. + """ + return self._call_java("factors") + + if __name__ == "__main__": import doctest import pyspark.ml.classification diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 0ea906d2688a1..c9f174ec24e57 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -37,7 +37,8 @@ 'IsotonicRegression', 'IsotonicRegressionModel', 'LinearRegression', 'LinearRegressionModel', 'LinearRegressionSummary', 'LinearRegressionTrainingSummary', - 'RandomForestRegressor', 'RandomForestRegressionModel'] + 'RandomForestRegressor', 'RandomForestRegressionModel', + 'FMRegressor', 'FMRegressionModel'] class _LinearRegressionParams(_JavaPredictorParams, HasRegParam, HasElasticNetParam, HasMaxIter, @@ -2296,6 +2297,167 @@ def __repr__(self): return self._call_java("toString") +@inherit_doc +class FMRegressor(JavaPredictor, HasMaxIter, HasStepSize, HasTol, HasSolver, HasSeed, + HasFitIntercept, HasRegParam, JavaMLWritable, JavaMLReadable): + """ + Factorization Machines learning algorithm for regression. + + solver Supports: + + * gd (normal mini-batch gradient descent) + * adamW (default) + + >>> from pyspark.ml.linalg import Vectors + >>> from pyspark.ml.regression import FMRegressor + >>> df = spark.createDataFrame([ + ... (2.0, Vectors.dense(2.0)), + ... (1.0, Vectors.dense(1.0)), + ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) + >>> + >>> fm = FMRegressor(factorSize=2, maxIter=1000) + >>> model = fm.fit(df) + >>> test0 = spark.createDataFrame([ + ... (Vectors.dense(-2.0),), + ... (Vectors.dense(0.5),), + ... (Vectors.dense(1.0),), + ... (Vectors.dense(4.0),)], ["features"]) + >>> model.transform(test0).show(10, False) + +--------+-------------------+ + |features|prediction | + +--------+-------------------+ + |[-2.0] |-1.9989237712341565| + |[0.5] |0.4956682219523814 | + |[1.0] |0.994586620589689 | + |[4.0] |3.9880970124135344 | + +--------+-------------------+ + >>> model.intercept + -0.0032501766849261557 + >>> model.linear + DenseVector([0.9978]) + >>> model.factors + DenseMatrix(1, 2, [0.0146, -0.0078], 1) + + .. versionadded:: 3.0.0 + """ + + factorSize = Param(Params._dummy(), "factorSize", "Dimensionality of the factor vectors, " + + "which are used to get pairwise interactions between variables", + typeConverter=TypeConverters.toInt) + + fitLinear = Param(Params._dummy(), "fitLinear", "whether to fit linear term (aka 1-way term)", + typeConverter=TypeConverters.toBoolean) + + miniBatchFraction = Param(Params._dummy(), "miniBatchFraction", "fraction of the input data " + + "set that should be used for one iteration of gradient descent", + typeConverter=TypeConverters.toFloat) + + initStd = Param(Params._dummy(), "initStd", "standard deviation of initial coefficients", + typeConverter=TypeConverters.toFloat) + + solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + + "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) + + @keyword_only + def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", + factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, + tol=1e-6, solver="adamW", seed=None): + """ + __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ + factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, \ + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, \ + tol=1e-6, solver="adamW", seed=None) + """ + super(FMRegressor, self).__init__() + self._java_obj = self._new_java_obj( + "org.apache.spark.ml.regression.FMRegressor", self.uid) + self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, + tol=1e-6, solver="adamW") + kwargs = self._input_kwargs + self.setParams(**kwargs) + + @keyword_only + @since("3.0.0") + def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", + factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, + tol=1e-6, solver="adamW", seed=None): + """ + setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ + factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, \ + miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, \ + tol=1e-6, solver="adamW", seed=None) + Sets Params for FMRegressor. + """ + kwargs = self._input_kwargs + return self._set(**kwargs) + + def _create_model(self, java_model): + return FMRegressionModel(java_model) + + @since("3.0.0") + def setFactorSize(self, value): + """ + Sets the value of :py:attr:`factorSize`. + """ + return self._set(factorSize=value) + + @since("3.0.0") + def setFitLinear(self, value): + """ + Sets the value of :py:attr:`fitLinear`. + """ + return self._set(fitLinear=value) + + @since("3.0.0") + def setMiniBatchFraction(self, value): + """ + Sets the value of :py:attr:`miniBatchFraction`. + """ + return self._set(miniBatchFraction=value) + + @since("3.0.0") + def setInitStd(self, value): + """ + Sets the value of :py:attr:`initStd`. + """ + return self._set(initStd=value) + + +class FMRegressionModel(JavaPredictionModel, JavaMLWritable, JavaMLReadable): + """ + Model fitted by :class:`FMRegressor`. + + .. versionadded:: 3.0.0 + """ + + @property + @since("3.0.0") + def intercept(self): + """ + Model intercept. + """ + return self._call_java("intercept") + + @property + @since("3.0.0") + def linear(self): + """ + Model linear term. + """ + return self._call_java("linear") + + @property + @since("3.0.0") + def factors(self): + """ + Model factor term. + """ + return self._call_java("factors") + + if __name__ == "__main__": import doctest import pyspark.ml.regression From 7bff2db9ed803e05a43c2d875c1dea819d81248a Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 23 Dec 2019 14:10:40 -0800 Subject: [PATCH 0940/1387] [SPARK-21869][SS] Revise Kafka producer pool to implement 'expire' correctly This patch revises Kafka producer pool (cache) to implement 'expire' correctly. Current implementation of Kafka producer cache leverages Guava cache, which decides cached producer instance to be expired if the instance is not "accessed" from cache. The behavior defines expiration time as "last accessed time + timeout", which is incorrect because some task may use the instance longer than timeout. There's no concept of "returning" in Guava cache as well, so it cannot be fixed with Guava cache. This patch introduces a new pool implementation which tracks "reference count" of cached instance, and defines expiration time for the instance as "last returned time + timeout" if the reference count goes 0, otherwise Long.MaxValue (effectively no expire). Expiring instances will be done with evict thread explicitly instead of evicting in part of handling acquire. (It might bring more overhead, but it ensures clearing expired instances even the pool is idle.) This patch also creates a new package `producer` under `kafka010`, to hide the details from `kafka010` package. In point of `kafka010` package's view, only acquire()/release()/reset() are available in pool, and even for CachedKafkaProducer the package cannot close the producer directly. Explained above. Yes, but only for the way of expiring cached instances. (The difference is described above.) Each executor leveraging spark-sql-kafka would have one eviction thread. New and existing UTs. Closes #26845 from HeartSaVioR/SPARK-21869-revised. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../sql/kafka010/CachedKafkaProducer.scala | 128 ----------- .../spark/sql/kafka010/KafkaDataWriter.scala | 23 +- .../spark/sql/kafka010/KafkaWriteTask.scala | 20 +- .../apache/spark/sql/kafka010/package.scala | 7 + .../producer/CachedKafkaProducer.scala | 41 ++++ .../producer/InternalKafkaProducerPool.scala | 206 ++++++++++++++++++ .../kafka010/CachedKafkaProducerSuite.scala | 77 ------- .../spark/sql/kafka010/KafkaSinkSuite.scala | 2 +- .../apache/spark/sql/kafka010/KafkaTest.scala | 3 +- .../InternalKafkaProducerPoolSuite.scala | 192 ++++++++++++++++ 10 files changed, 472 insertions(+), 227 deletions(-) delete mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/CachedKafkaProducer.scala create mode 100644 external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala delete mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPoolSuite.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala deleted file mode 100644 index fc177cdc9037e..0000000000000 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala +++ /dev/null @@ -1,128 +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.sql.kafka010 - -import java.{util => ju} -import java.util.concurrent.{ConcurrentMap, ExecutionException, TimeUnit} - -import com.google.common.cache._ -import com.google.common.util.concurrent.{ExecutionError, UncheckedExecutionException} -import org.apache.kafka.clients.producer.KafkaProducer -import scala.collection.JavaConverters._ -import scala.util.control.NonFatal - -import org.apache.spark.SparkEnv -import org.apache.spark.internal.Logging -import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaRedactionUtil} - -private[kafka010] object CachedKafkaProducer extends Logging { - - private type Producer = KafkaProducer[Array[Byte], Array[Byte]] - - private val defaultCacheExpireTimeout = TimeUnit.MINUTES.toMillis(10) - - private lazy val cacheExpireTimeout: Long = Option(SparkEnv.get) - .map(_.conf.get(PRODUCER_CACHE_TIMEOUT)) - .getOrElse(defaultCacheExpireTimeout) - - private val cacheLoader = new CacheLoader[Seq[(String, Object)], Producer] { - override def load(config: Seq[(String, Object)]): Producer = { - createKafkaProducer(config) - } - } - - private val removalListener = new RemovalListener[Seq[(String, Object)], Producer]() { - override def onRemoval( - notification: RemovalNotification[Seq[(String, Object)], Producer]): Unit = { - val paramsSeq: Seq[(String, Object)] = notification.getKey - val producer: Producer = notification.getValue - if (log.isDebugEnabled()) { - val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) - logDebug(s"Evicting kafka producer $producer params: $redactedParamsSeq, " + - s"due to ${notification.getCause}") - } - close(paramsSeq, producer) - } - } - - private lazy val guavaCache: LoadingCache[Seq[(String, Object)], Producer] = - CacheBuilder.newBuilder().expireAfterAccess(cacheExpireTimeout, TimeUnit.MILLISECONDS) - .removalListener(removalListener) - .build[Seq[(String, Object)], Producer](cacheLoader) - - private def createKafkaProducer(paramsSeq: Seq[(String, Object)]): Producer = { - val kafkaProducer: Producer = new Producer(paramsSeq.toMap.asJava) - if (log.isDebugEnabled()) { - val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) - logDebug(s"Created a new instance of KafkaProducer for $redactedParamsSeq.") - } - kafkaProducer - } - - /** - * Get a cached KafkaProducer for a given configuration. If matching KafkaProducer doesn't - * exist, a new KafkaProducer will be created. KafkaProducer is thread safe, it is best to keep - * one instance per specified kafkaParams. - */ - private[kafka010] def getOrCreate(kafkaParams: ju.Map[String, Object]): Producer = { - val updatedKafkaProducerConfiguration = - KafkaConfigUpdater("executor", kafkaParams.asScala.toMap) - .setAuthenticationConfigIfNeeded() - .build() - val paramsSeq: Seq[(String, Object)] = paramsToSeq(updatedKafkaProducerConfiguration) - try { - guavaCache.get(paramsSeq) - } catch { - case e @ (_: ExecutionException | _: UncheckedExecutionException | _: ExecutionError) - if e.getCause != null => - throw e.getCause - } - } - - private def paramsToSeq(kafkaParams: ju.Map[String, Object]): Seq[(String, Object)] = { - val paramsSeq: Seq[(String, Object)] = kafkaParams.asScala.toSeq.sortBy(x => x._1) - paramsSeq - } - - /** For explicitly closing kafka producer */ - private[kafka010] def close(kafkaParams: ju.Map[String, Object]): Unit = { - val paramsSeq = paramsToSeq(kafkaParams) - guavaCache.invalidate(paramsSeq) - } - - /** Auto close on cache evict */ - private def close(paramsSeq: Seq[(String, Object)], producer: Producer): Unit = { - try { - if (log.isInfoEnabled()) { - val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) - logInfo(s"Closing the KafkaProducer with params: ${redactedParamsSeq.mkString("\n")}.") - } - producer.close() - } catch { - case NonFatal(e) => logWarning("Error while closing kafka producer.", e) - } - } - - private[kafka010] def clear(): Unit = { - logInfo("Cleaning up guava cache.") - guavaCache.invalidateAll() - } - - // Intended for testing purpose only. - private def getAsMap: ConcurrentMap[Seq[(String, Object)], Producer] = guavaCache.asMap() -} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala index 9a2b369933616..63863a6cc6d6f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala @@ -22,6 +22,7 @@ import java.{util => ju} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.kafka010.producer.{CachedKafkaProducer, InternalKafkaProducerPool} /** * Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we @@ -44,11 +45,14 @@ private[kafka010] class KafkaDataWriter( inputSchema: Seq[Attribute]) extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] { - private lazy val producer = CachedKafkaProducer.getOrCreate(producerParams) + private var producer: Option[CachedKafkaProducer] = None def write(row: InternalRow): Unit = { checkForErrors() - sendRow(row, producer) + if (producer.isEmpty) { + producer = Some(InternalKafkaProducerPool.acquire(producerParams)) + } + producer.foreach { p => sendRow(row, p.producer) } } def commit(): WriterCommitMessage = { @@ -56,22 +60,15 @@ private[kafka010] class KafkaDataWriter( // This requires flushing and then checking that no callbacks produced errors. // We also check for errors before to fail as soon as possible - the check is cheap. checkForErrors() - producer.flush() + producer.foreach(_.producer.flush()) checkForErrors() KafkaDataWriterCommitMessage } def abort(): Unit = {} - def close(): Unit = {} - - /** explicitly invalidate producer from pool. only for testing. */ - private[kafka010] def invalidateProducer(): Unit = { - checkForErrors() - if (producer != null) { - producer.flush() - checkForErrors() - CachedKafkaProducer.close(producerParams) - } + def close(): Unit = { + producer.foreach(InternalKafkaProducerPool.release) + producer = None } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala index 8b907065af1d0..fddba3f0f9919 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.header.internals.RecordHeader import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, UnsafeProjection} +import org.apache.spark.sql.kafka010.producer.{CachedKafkaProducer, InternalKafkaProducerPool} import org.apache.spark.sql.types.BinaryType /** @@ -39,25 +40,30 @@ private[kafka010] class KafkaWriteTask( inputSchema: Seq[Attribute], topic: Option[String]) extends KafkaRowWriter(inputSchema, topic) { // used to synchronize with Kafka callbacks - private var producer: KafkaProducer[Array[Byte], Array[Byte]] = _ + private var producer: Option[CachedKafkaProducer] = None /** * Writes key value data out to topics. */ def execute(iterator: Iterator[InternalRow]): Unit = { - producer = CachedKafkaProducer.getOrCreate(producerConfiguration) + producer = Some(InternalKafkaProducerPool.acquire(producerConfiguration)) + val internalProducer = producer.get.producer while (iterator.hasNext && failedWrite == null) { val currentRow = iterator.next() - sendRow(currentRow, producer) + sendRow(currentRow, internalProducer) } } def close(): Unit = { - checkForErrors() - if (producer != null) { - producer.flush() + try { checkForErrors() - producer = null + producer.foreach { p => + p.producer.flush() + checkForErrors() + } + } finally { + producer.foreach(InternalKafkaProducerPool.release) + producer = None } } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala index 6f6ae55fc4971..460bb8bd34ec6 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package.scala @@ -32,6 +32,13 @@ package object kafka010 { // scalastyle:ignore .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("10m") + private[kafka010] val PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL = + ConfigBuilder("spark.kafka.producer.cache.evictorThreadRunInterval") + .doc("The interval of time between runs of the idle evictor thread for producer pool. " + + "When non-positive, no idle evictor thread will be run.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("1m") + private[kafka010] val CONSUMER_CACHE_CAPACITY = ConfigBuilder("spark.kafka.consumer.cache.capacity") .doc("The maximum number of consumers cached. Please note it's a soft limit" + diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/CachedKafkaProducer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/CachedKafkaProducer.scala new file mode 100644 index 0000000000000..83519de0d3b1e --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/CachedKafkaProducer.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010.producer + +import java.{util => ju} + +import scala.util.control.NonFatal + +import org.apache.kafka.clients.producer.KafkaProducer + +import org.apache.spark.internal.Logging + +private[kafka010] class CachedKafkaProducer( + val cacheKey: Seq[(String, Object)], + val producer: KafkaProducer[Array[Byte], Array[Byte]]) extends Logging { + val id: String = ju.UUID.randomUUID().toString + + private[producer] def close(): Unit = { + try { + logInfo(s"Closing the KafkaProducer with id: $id.") + producer.close() + } catch { + case NonFatal(e) => logWarning("Error while closing kafka producer.", e) + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala new file mode 100644 index 0000000000000..8d1f9b8d37f60 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala @@ -0,0 +1,206 @@ +/* + * 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.kafka010.producer + +import java.{util => ju} +import java.util.concurrent.{ScheduledExecutorService, ScheduledFuture, TimeUnit} +import javax.annotation.concurrent.GuardedBy + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.kafka.clients.producer.KafkaProducer + +import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.internal.Logging +import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaRedactionUtil} +import org.apache.spark.sql.kafka010.{PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL, PRODUCER_CACHE_TIMEOUT} +import org.apache.spark.util.{Clock, ShutdownHookManager, SystemClock, ThreadUtils, Utils} + +/** + * Provides object pool for [[CachedKafkaProducer]] which is grouped by + * [[org.apache.spark.sql.kafka010.producer.InternalKafkaProducerPool.CacheKey]]. + */ +private[producer] class InternalKafkaProducerPool( + executorService: ScheduledExecutorService, + val clock: Clock, + conf: SparkConf) extends Logging { + import InternalKafkaProducerPool._ + + def this(sparkConf: SparkConf) = { + this(ThreadUtils.newDaemonSingleThreadScheduledExecutor( + "kafka-producer-cache-evictor"), new SystemClock, sparkConf) + } + + /** exposed for testing */ + private[producer] val cacheExpireTimeoutMillis: Long = conf.get(PRODUCER_CACHE_TIMEOUT) + + @GuardedBy("this") + private val cache = new mutable.HashMap[CacheKey, CachedProducerEntry] + + private def startEvictorThread(): Option[ScheduledFuture[_]] = { + val evictorThreadRunIntervalMillis = conf.get(PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL) + if (evictorThreadRunIntervalMillis > 0) { + val future = executorService.scheduleAtFixedRate(() => { + Utils.tryLogNonFatalError(evictExpired()) + }, 0, evictorThreadRunIntervalMillis, TimeUnit.MILLISECONDS) + Some(future) + } else { + None + } + } + + private val scheduled = startEvictorThread() + + /** + * Get a cached KafkaProducer for a given configuration. If matching KafkaProducer doesn't + * exist, a new KafkaProducer will be created. KafkaProducer is thread safe, it is best to keep + * one instance per specified kafkaParams. + */ + private[producer] def acquire(kafkaParams: ju.Map[String, Object]): CachedKafkaProducer = { + val updatedKafkaProducerConfiguration = + KafkaConfigUpdater("executor", kafkaParams.asScala.toMap) + .setAuthenticationConfigIfNeeded() + .build() + val paramsSeq: Seq[(String, Object)] = paramsToSeq(updatedKafkaProducerConfiguration) + synchronized { + val entry = cache.getOrElseUpdate(paramsSeq, { + val producer = createKafkaProducer(paramsSeq) + val cachedProducer = new CachedKafkaProducer(paramsSeq, producer) + new CachedProducerEntry(cachedProducer, + TimeUnit.MILLISECONDS.toNanos(cacheExpireTimeoutMillis)) + }) + entry.handleBorrowed() + entry.producer + } + } + + private[producer] def release(producer: CachedKafkaProducer): Unit = { + synchronized { + cache.get(producer.cacheKey) match { + case Some(entry) if entry.producer.id == producer.id => + entry.handleReturned(clock.nanoTime()) + case _ => + logWarning(s"Released producer ${producer.id} is not a member of the cache. Closing.") + producer.close() + } + } + } + + private[producer] def shutdown(): Unit = { + scheduled.foreach(_.cancel(false)) + ThreadUtils.shutdown(executorService) + } + + /** exposed for testing. */ + private[producer] def reset(): Unit = synchronized { + cache.foreach { case (_, v) => v.producer.close() } + cache.clear() + } + + /** exposed for testing */ + private[producer] def getAsMap: Map[CacheKey, CachedProducerEntry] = cache.toMap + + private def evictExpired(): Unit = { + val curTimeNs = clock.nanoTime() + val producers = new mutable.ArrayBuffer[CachedProducerEntry]() + synchronized { + cache.retain { case (_, v) => + if (v.expired(curTimeNs)) { + producers += v + false + } else { + true + } + } + } + producers.foreach { _.producer.close() } + } + + private def createKafkaProducer(paramsSeq: Seq[(String, Object)]): Producer = { + val kafkaProducer: Producer = new Producer(paramsSeq.toMap.asJava) + if (log.isDebugEnabled()) { + val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) + logDebug(s"Created a new instance of KafkaProducer for $redactedParamsSeq.") + } + kafkaProducer + } + + private def paramsToSeq(kafkaParams: ju.Map[String, Object]): Seq[(String, Object)] = { + kafkaParams.asScala.toSeq.sortBy(x => x._1) + } +} + +private[kafka010] object InternalKafkaProducerPool extends Logging { + private val pool = new InternalKafkaProducerPool( + Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf())) + + private type CacheKey = Seq[(String, Object)] + private type Producer = KafkaProducer[Array[Byte], Array[Byte]] + + ShutdownHookManager.addShutdownHook { () => + try { + pool.shutdown() + } catch { + case e: Throwable => + logWarning("Ignoring Exception while shutting down pools from shutdown hook", e) + } + } + + /** + * This class is used as metadata of producer pool, and shouldn't be exposed to the public. + * This class assumes thread-safety is guaranteed by the caller. + */ + private[producer] class CachedProducerEntry( + val producer: CachedKafkaProducer, + cacheExpireTimeoutNs: Long) { + private var _refCount: Long = 0L + private var _expireAt: Long = Long.MaxValue + + /** exposed for testing */ + private[producer] def refCount: Long = _refCount + private[producer] def expireAt: Long = _expireAt + + def handleBorrowed(): Unit = { + _refCount += 1 + _expireAt = Long.MaxValue + } + + def handleReturned(curTimeNs: Long): Unit = { + require(_refCount > 0, "Reference count shouldn't become negative. Returning same producer " + + "multiple times would occur this bug. Check the logic around returning producer.") + + _refCount -= 1 + if (_refCount == 0) { + _expireAt = curTimeNs + cacheExpireTimeoutNs + } + } + + def expired(curTimeNs: Long): Boolean = _refCount == 0 && _expireAt < curTimeNs + } + + def acquire(kafkaParams: ju.Map[String, Object]): CachedKafkaProducer = { + pool.acquire(kafkaParams) + } + + def release(producer: CachedKafkaProducer): Unit = { + pool.release(producer) + } + + def reset(): Unit = pool.reset() +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala deleted file mode 100644 index 7425a74315e1a..0000000000000 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala +++ /dev/null @@ -1,77 +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.sql.kafka010 - -import java.{util => ju} -import java.util.concurrent.ConcurrentMap - -import org.apache.kafka.clients.producer.KafkaProducer -import org.apache.kafka.common.serialization.ByteArraySerializer -import org.scalatest.PrivateMethodTester - -import org.apache.spark.sql.test.SharedSparkSession - -class CachedKafkaProducerSuite extends SharedSparkSession with PrivateMethodTester with KafkaTest { - - type KP = KafkaProducer[Array[Byte], Array[Byte]] - - protected override def beforeEach(): Unit = { - super.beforeEach() - CachedKafkaProducer.clear() - } - - test("Should return the cached instance on calling getOrCreate with same params.") { - val kafkaParams = new ju.HashMap[String, Object]() - kafkaParams.put("acks", "0") - // Here only host should be resolvable, it does not need a running instance of kafka server. - kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") - kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) - kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) - val producer = CachedKafkaProducer.getOrCreate(kafkaParams) - val producer2 = CachedKafkaProducer.getOrCreate(kafkaParams) - assert(producer == producer2) - - val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]](Symbol("getAsMap")) - val map = CachedKafkaProducer.invokePrivate(cacheMap()) - assert(map.size == 1) - } - - test("Should close the correct kafka producer for the given kafkaPrams.") { - val kafkaParams = new ju.HashMap[String, Object]() - kafkaParams.put("acks", "0") - kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") - kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) - kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) - val producer: KP = CachedKafkaProducer.getOrCreate(kafkaParams) - kafkaParams.put("acks", "1") - val producer2: KP = CachedKafkaProducer.getOrCreate(kafkaParams) - // With updated conf, a new producer instance should be created. - assert(producer != producer2) - - val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]](Symbol("getAsMap")) - val map = CachedKafkaProducer.invokePrivate(cacheMap()) - assert(map.size == 2) - - CachedKafkaProducer.close(kafkaParams) - val map2 = CachedKafkaProducer.invokePrivate(cacheMap()) - assert(map2.size == 1) - import scala.collection.JavaConverters._ - val (seq: Seq[(String, Object)], _producer: KP) = map2.asScala.toArray.apply(0) - assert(_producer == producer) - } -} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index ac242ba3d1356..e2dcd62005310 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -370,7 +370,7 @@ class KafkaContinuousSinkSuite extends KafkaSinkStreamingSuiteBase { iter.foreach(writeTask.write(_)) writeTask.commit() } finally { - writeTask.invalidateProducer() + writeTask.close() } } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTest.scala index 19acda95c707c..087d938f8ed8e 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTest.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTest.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.kafka010 import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.kafka010.producer.InternalKafkaProducerPool /** A trait to clean cached Kafka producers in `afterAll` */ trait KafkaTest extends BeforeAndAfterAll { @@ -27,6 +28,6 @@ trait KafkaTest extends BeforeAndAfterAll { override def afterAll(): Unit = { super.afterAll() - CachedKafkaProducer.clear() + InternalKafkaProducerPool.reset() } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPoolSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPoolSuite.scala new file mode 100644 index 0000000000000..97885754f204c --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPoolSuite.scala @@ -0,0 +1,192 @@ +/* + * 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.kafka010.producer + +import java.{util => ju} +import java.util.concurrent.{Executors, TimeUnit} + +import scala.util.Random + +import org.apache.kafka.common.serialization.ByteArraySerializer +import org.jmock.lib.concurrent.DeterministicScheduler + +import org.apache.spark.SparkConf +import org.apache.spark.sql.kafka010.{PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL, PRODUCER_CACHE_TIMEOUT} +import org.apache.spark.sql.kafka010.producer.InternalKafkaProducerPool.CachedProducerEntry +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.util.ManualClock + +class InternalKafkaProducerPoolSuite extends SharedSparkSession { + + private var pool: InternalKafkaProducerPool = _ + + protected override def afterEach(): Unit = { + if (pool != null) { + try { + pool.shutdown() + pool = null + } catch { + // ignore as it's known issue, DeterministicScheduler doesn't support shutdown + case _: UnsupportedOperationException => + } + } + } + + test("Should return same cached instance on calling acquire with same params.") { + pool = new InternalKafkaProducerPool(new SparkConf()) + + val kafkaParams = getTestKafkaParams() + val producer = pool.acquire(kafkaParams) + val producer2 = pool.acquire(kafkaParams) + assert(producer eq producer2) + + val map = pool.getAsMap + assert(map.size === 1) + val cacheEntry = map.head._2 + assertCacheEntry(pool, cacheEntry, 2L) + + pool.release(producer) + assertCacheEntry(pool, cacheEntry, 1L) + + pool.release(producer2) + assertCacheEntry(pool, cacheEntry, 0L) + + val producer3 = pool.acquire(kafkaParams) + assertCacheEntry(pool, cacheEntry, 1L) + assert(producer eq producer3) + } + + test("Should return different cached instances on calling acquire with different params.") { + pool = new InternalKafkaProducerPool(new SparkConf()) + + val kafkaParams = getTestKafkaParams() + val producer = pool.acquire(kafkaParams) + kafkaParams.put("acks", "1") + val producer2 = pool.acquire(kafkaParams) + // With updated conf, a new producer instance should be created. + assert(producer ne producer2) + + val map = pool.getAsMap + assert(map.size === 2) + val cacheEntry = map.find(_._2.producer.id == producer.id).get._2 + assertCacheEntry(pool, cacheEntry, 1L) + val cacheEntry2 = map.find(_._2.producer.id == producer2.id).get._2 + assertCacheEntry(pool, cacheEntry2, 1L) + } + + test("expire instances") { + val minEvictableIdleTimeMillis = 2000L + val evictorThreadRunIntervalMillis = 500L + + val conf = new SparkConf() + conf.set(PRODUCER_CACHE_TIMEOUT, minEvictableIdleTimeMillis) + conf.set(PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL, evictorThreadRunIntervalMillis) + + val scheduler = new DeterministicScheduler() + val clock = new ManualClock() + pool = new InternalKafkaProducerPool(scheduler, clock, conf) + + val kafkaParams = getTestKafkaParams() + + var map = pool.getAsMap + assert(map.isEmpty) + + val producer = pool.acquire(kafkaParams) + map = pool.getAsMap + assert(map.size === 1) + + clock.advance(minEvictableIdleTimeMillis + 100) + scheduler.tick(evictorThreadRunIntervalMillis + 100, TimeUnit.MILLISECONDS) + map = pool.getAsMap + assert(map.size === 1) + + pool.release(producer) + + // This will clean up expired instance from cache. + clock.advance(minEvictableIdleTimeMillis + 100) + scheduler.tick(evictorThreadRunIntervalMillis + 100, TimeUnit.MILLISECONDS) + + map = pool.getAsMap + assert(map.size === 0) + } + + test("reference counting with concurrent access") { + pool = new InternalKafkaProducerPool(new SparkConf()) + + val kafkaParams = getTestKafkaParams() + + val numThreads = 100 + val numProducerUsages = 500 + + def produce(i: Int): Unit = { + val producer = pool.acquire(kafkaParams) + try { + val map = pool.getAsMap + assert(map.size === 1) + val cacheEntry = map.head._2 + assert(cacheEntry.refCount > 0L) + assert(cacheEntry.expireAt === Long.MaxValue) + + Thread.sleep(Random.nextInt(100)) + } finally { + pool.release(producer) + } + } + + val threadpool = Executors.newFixedThreadPool(numThreads) + try { + val futures = (1 to numProducerUsages).map { i => + threadpool.submit(new Runnable { + override def run(): Unit = { produce(i) } + }) + } + futures.foreach(_.get(1, TimeUnit.MINUTES)) + } finally { + threadpool.shutdown() + } + + val map = pool.getAsMap + assert(map.size === 1) + + val cacheEntry = map.head._2 + assertCacheEntry(pool, cacheEntry, 0L) + } + + private def getTestKafkaParams(): ju.HashMap[String, Object] = { + val kafkaParams = new ju.HashMap[String, Object]() + kafkaParams.put("acks", "0") + // Here only host should be resolvable, it does not need a running instance of kafka server. + kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") + kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) + kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) + kafkaParams + } + + private def assertCacheEntry( + pool: InternalKafkaProducerPool, + cacheEntry: CachedProducerEntry, + expectedRefCount: Long): Unit = { + val timeoutVal = TimeUnit.MILLISECONDS.toNanos(pool.cacheExpireTimeoutMillis) + assert(cacheEntry.refCount === expectedRefCount) + if (expectedRefCount > 0) { + assert(cacheEntry.expireAt === Long.MaxValue) + } else { + assert(cacheEntry.expireAt <= pool.clock.nanoTime() + timeoutVal) + } + } +} From 2164243526b5a2ebd1a77fe2aa43b7329b6a229c Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 24 Dec 2019 11:39:03 +0900 Subject: [PATCH 0941/1387] [SPARK-28144][SPARK-29294][SS][FOLLOWUP] Use SystemTime defined in Kafka Time interface ### What changes were proposed in this pull request? This patch addresses review comments in #26960 (https://github.com/apache/spark/pull/26960#discussion_r360661930 / https://github.com/apache/spark/pull/26960#discussion_r360661947) which were not addressed in the patch. Addressing these review comments will let the code less dependent on actual implementation as it only relies on `Time` interface in Kafka. ### Why are the changes needed? These were review comments in previous PR and they bring actual benefit though they're minors. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #26979 from HeartSaVioR/SPARK-29294-follow-up. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../org/apache/spark/streaming/kafka010/KafkaTestUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala index 4e6a23bd72f11..0783e591def51 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala @@ -34,7 +34,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.serialization.StringSerializer -import org.apache.kafka.common.utils.SystemTime +import org.apache.kafka.common.utils.{Time => KTime} import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} import org.apache.spark.{SparkConf, SparkException} @@ -106,7 +106,7 @@ private[kafka010] class KafkaTestUtils extends Logging { // Get the actual zookeeper binding port zkPort = zookeeper.actualPort zkClient = KafkaZkClient(s"$zkHost:$zkPort", isSecure = false, zkSessionTimeout, - zkConnectionTimeout, 1, new SystemTime()) + zkConnectionTimeout, 1, KTime.SYSTEM) admClient = new AdminZkClient(zkClient) zkReady = true } From ab0dd41ff2c05134266bd14d96676c96aa2764cd Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 24 Dec 2019 12:01:29 +0900 Subject: [PATCH 0942/1387] [SPARK-26618][SQL][FOLLOWUP] Update the SQL migration guide regarding to typed `TIMESTAMP` and `DATE` literals ### What changes were proposed in this pull request? In the PR, I propose to update the SQL migration guide and clarify semantic of string conversion to typed `TIMESTAMP` and `DATE` literals. ### Why are the changes needed? This is a follow-up of the PR https://github.com/apache/spark/pull/23541 which changed the behavior of `TIMESTAMP`/`DATE` literals, and can impact on results of user's queries. ### Does this PR introduce any user-facing change? No ### How was this patch tested? It should be checked by jenkins build. Closes #26985 from MaxGekk/timestamp-date-constructors-followup. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- docs/sql-migration-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 674621f3fdfaf..73b7df421a635 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -77,6 +77,8 @@ license: | - Formatting of `TIMESTAMP` and `DATE` literals. + - Creating of typed `TIMESTAMP` and `DATE` literals from strings. Since Spark 3.0, string conversion to typed `TIMESTAMP`/`DATE` literals is performed via casting to `TIMESTAMP`/`DATE` values. For example, `TIMESTAMP '2019-12-23 12:59:30'` is semantically equal to `CAST('2019-12-23 12:59:30' AS TIMESTAMP)`. In Spark version 2.4 and earlier, the `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()` functions are used for the same purpose. + - In Spark version 2.4 and earlier, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. Since Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`. - In Spark version 2.4 and earlier, the `current_timestamp` function returns a timestamp with millisecond resolution only. Since Spark 3.0, the function can return the result with microsecond resolution if the underlying clock available on the system offers such resolution. From ba3f6330dd2b6054988f1f6f0ffe014fc4969088 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 24 Dec 2019 14:01:27 +0800 Subject: [PATCH 0943/1387] Revert "[SPARK-29224][ML] Implement Factorization Machines as a ml-pipeline component" This reverts commit c6ab7165dd11a0a7b8aea4c805409088e9a41a74. --- docs/ml-classification-regression.md | 107 --- .../examples/ml/JavaFMClassifierExample.java | 105 --- .../examples/ml/JavaFMRegressorExample.java | 90 -- .../main/python/ml/fm_classifier_example.py | 77 -- .../main/python/ml/fm_regressor_example.py | 74 -- .../examples/ml/FMClassifierExample.scala | 96 --- .../examples/ml/FMRegressorExample.scala | 84 -- .../ml/classification/FMClassifier.scala | 332 ------- .../spark/ml/regression/FMRegressor.scala | 815 ------------------ .../ml/classification/FMClassifierSuite.scala | 242 ------ .../ml/regression/FMRegressorSuite.scala | 240 ------ python/pyspark/ml/classification.py | 166 +--- python/pyspark/ml/regression.py | 164 +--- 13 files changed, 2 insertions(+), 2590 deletions(-) delete mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaFMClassifierExample.java delete mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaFMRegressorExample.java delete mode 100644 examples/src/main/python/ml/fm_classifier_example.py delete mode 100644 examples/src/main/python/ml/fm_regressor_example.py delete mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/ml/regression/FMRegressorSuite.scala diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index 630a15d8535f4..6a81aff1476d9 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -530,42 +530,6 @@ Refer to the [R API docs](api/R/spark.naiveBayes.html) for more details. -## Factorization machines classifier - -For more background and more details about the implementation of factorization machines, -refer to the [Factorization Machines section](ml-classification-regression.html#factorization-machines). - -**Examples** - -The following examples load a dataset in LibSVM format, split it into training and test sets, -train on the first dataset, and then evaluate on the held-out test set. -We scale features to be between 0 and 1 to prevent the exploding gradient problem. - -
      -
      - -Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.classification.FMClassifier) for more details. - -{% include_example scala/org/apache/spark/examples/ml/FMClassifierExample.scala %} -
      - -
      - -Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/FMClassifier.html) for more details. - -{% include_example java/org/apache/spark/examples/ml/JavaFMClassifierExample.java %} -
      - -
      - -Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classification.FMClassifier) for more details. - -{% include_example python/ml/fm_classifier_example.py %} -
      - -
      - - # Regression ## Linear regression @@ -1051,43 +1015,6 @@ Refer to the [`IsotonicRegression` R API docs](api/R/spark.isoreg.html) for more - -## Factorization machines regressor - -For more background and more details about the implementation of factorization machines, -refer to the [Factorization Machines section](ml-classification-regression.html#factorization-machines). - -**Examples** - -The following examples load a dataset in LibSVM format, split it into training and test sets, -train on the first dataset, and then evaluate on the held-out test set. -We scale features to be between 0 and 1 to prevent the exploding gradient problem. - -
      -
      - -Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.FMRegressor) for more details. - -{% include_example scala/org/apache/spark/examples/ml/FMRegressorExample.scala %} -
      - -
      - -Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/FMRegressor.html) for more details. - -{% include_example java/org/apache/spark/examples/ml/JavaFMRegressorExample.java %} -
      - -
      - -Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.FMRegressor) for more details. - -{% include_example python/ml/fm_regressor_example.py %} -
      - -
      - - # Linear methods We implement popular linear methods such as logistic @@ -1117,40 +1044,6 @@ regression](http://en.wikipedia.org/wiki/Tikhonov_regularization) model. We implement Pipelines API for both linear regression and logistic regression with elastic net regularization. -# Factorization Machines - -[Factorization Machines](https://www.csie.ntu.edu.tw/~b97053/paper/Rendle2010FM.pdf) are able to estimate interactions -between features even in problems with huge sparsity (like advertising and recommendation system). -The `spark.ml` implementation supports factorization machines for binary classification and for regression. - -Factorization machines formula is: - -$$ -\hat{y} = w_0 + \sum\limits^n_{i-1} w_i x_i + - \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j -$$ - -The first two terms denote intercept and linear term (same as in linear regression), -and the last term denotes pairwise interactions term. $$v_i$$ describes the i-th variable -with k factors. - -FM can be used for regression and optimization criterion is mean square error. FM also can be used for -binary classification through sigmoid function. The optimization criterion is logistic loss. - -The pairwise interactions can be reformulated: - -$$ -\sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j - = \frac{1}{2}\sum\limits^k_{f=1} - \left(\left( \sum\limits^n_{i=1}v_{i,f}x_i \right)^2 - - \sum\limits^n_{i=1}v_{i,f}^2x_i^2 \right) -$$ - -This equation has only linear complexity in both k and n - i.e. its computation is in $$O(kn)$$. - -In general, in order to prevent the exploding gradient problem, it is best to scale continuous features to be between 0 and 1, -or bin the continuous features and one-hot encode them. - # Decision trees [Decision trees](http://en.wikipedia.org/wiki/Decision_tree_learning) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaFMClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaFMClassifierExample.java deleted file mode 100644 index 2f11082e0dd69..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaFMClassifierExample.java +++ /dev/null @@ -1,105 +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.examples.ml; - -// $example on$ -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.classification.FMClassificationModel; -import org.apache.spark.ml.classification.FMClassifier; -import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; -import org.apache.spark.ml.feature.*; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -// $example off$ - -public class JavaFMClassifierExample { - public static void main(String[] args) { - SparkSession spark = SparkSession - .builder() - .appName("JavaFMClassifierExample") - .getOrCreate(); - - // $example on$ - // Load and parse the data file, converting it to a DataFrame. - Dataset data = spark - .read() - .format("libsvm") - .load("data/mllib/sample_libsvm_data.txt"); - - // Index labels, adding metadata to the label column. - // Fit on whole dataset to include all labels in index. - StringIndexerModel labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data); - // Scale features. - MinMaxScalerModel featureScaler = new MinMaxScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures") - .fit(data); - - // Split the data into training and test sets (30% held out for testing) - Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); - Dataset trainingData = splits[0]; - Dataset testData = splits[1]; - - // Train a FM model. - FMClassifier fm = new FMClassifier() - .setLabelCol("indexedLabel") - .setFeaturesCol("scaledFeatures") - .setStepSize(0.001); - - // Convert indexed labels back to original labels. - IndexToString labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labelsArray()[0]); - - // Create a Pipeline. - Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {labelIndexer, featureScaler, fm, labelConverter}); - - // Train model. - PipelineModel model = pipeline.fit(trainingData); - - // Make predictions. - Dataset predictions = model.transform(testData); - - // Select example rows to display. - predictions.select("predictedLabel", "label", "features").show(5); - - // Select (prediction, true label) and compute test accuracy. - MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() - .setLabelCol("indexedLabel") - .setPredictionCol("prediction") - .setMetricName("accuracy"); - double accuracy = evaluator.evaluate(predictions); - System.out.println("Test Accuracy = " + accuracy); - - FMClassificationModel fmModel = (FMClassificationModel)(model.stages()[2]); - System.out.println("Factors: " + fmModel.factors()); - System.out.println("Linear: " + fmModel.linear()); - System.out.println("Intercept: " + fmModel.intercept()); - // $example off$ - - spark.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaFMRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaFMRegressorExample.java deleted file mode 100644 index 3ade40f3d39c2..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaFMRegressorExample.java +++ /dev/null @@ -1,90 +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.examples.ml; - -// $example on$ -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.evaluation.RegressionEvaluator; -import org.apache.spark.ml.feature.MinMaxScaler; -import org.apache.spark.ml.feature.MinMaxScalerModel; -import org.apache.spark.ml.regression.FMRegressionModel; -import org.apache.spark.ml.regression.FMRegressor; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -// $example off$ - -public class JavaFMRegressorExample { - public static void main(String[] args) { - SparkSession spark = SparkSession - .builder() - .appName("JavaFMRegressorExample") - .getOrCreate(); - - // $example on$ - // Load and parse the data file, converting it to a DataFrame. - Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); - - // Scale features. - MinMaxScalerModel featureScaler = new MinMaxScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures") - .fit(data); - - // Split the data into training and test sets (30% held out for testing). - Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); - Dataset trainingData = splits[0]; - Dataset testData = splits[1]; - - // Train a FM model. - FMRegressor fm = new FMRegressor() - .setLabelCol("label") - .setFeaturesCol("scaledFeatures") - .setStepSize(0.001); - - // Create a Pipeline. - Pipeline pipeline = new Pipeline().setStages(new PipelineStage[] {featureScaler, fm}); - - // Train model. - PipelineModel model = pipeline.fit(trainingData); - - // Make predictions. - Dataset predictions = model.transform(testData); - - // Select example rows to display. - predictions.select("prediction", "label", "features").show(5); - - // Select (prediction, true label) and compute test error. - RegressionEvaluator evaluator = new RegressionEvaluator() - .setLabelCol("label") - .setPredictionCol("prediction") - .setMetricName("rmse"); - double rmse = evaluator.evaluate(predictions); - System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); - - FMRegressionModel fmModel = (FMRegressionModel)(model.stages()[1]); - System.out.println("Factors: " + fmModel.factors()); - System.out.println("Linear: " + fmModel.linear()); - System.out.println("Intercept: " + fmModel.intercept()); - // $example off$ - - spark.stop(); - } -} diff --git a/examples/src/main/python/ml/fm_classifier_example.py b/examples/src/main/python/ml/fm_classifier_example.py deleted file mode 100644 index 6e7c2ccf021ed..0000000000000 --- a/examples/src/main/python/ml/fm_classifier_example.py +++ /dev/null @@ -1,77 +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. -# - -""" -FMClassifier Example. -""" -from __future__ import print_function - -# $example on$ -from pyspark.ml import Pipeline -from pyspark.ml.classification import FMClassifier -from pyspark.ml.feature import MinMaxScaler, StringIndexer -from pyspark.ml.evaluation import MulticlassClassificationEvaluator -# $example off$ -from pyspark.sql import SparkSession - -if __name__ == "__main__": - spark = SparkSession \ - .builder \ - .appName("FMClassifierExample") \ - .getOrCreate() - - # $example on$ - # Load and parse the data file, converting it to a DataFrame. - data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - - # Index labels, adding metadata to the label column. - # Fit on whole dataset to include all labels in index. - labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) - # Scale features. - featureScaler = MinMaxScaler(inputCol="features", outputCol="scaledFeatures").fit(data) - - # Split the data into training and test sets (30% held out for testing) - (trainingData, testData) = data.randomSplit([0.7, 0.3]) - - # Train a FM model. - fm = FMClassifier(labelCol="indexedLabel", featuresCol="scaledFeatures", stepSize=0.001) - - # Create a Pipeline. - pipeline = Pipeline(stages=[labelIndexer, featureScaler, fm]) - - # Train model. - model = pipeline.fit(trainingData) - - # Make predictions. - predictions = model.transform(testData) - - # Select example rows to display. - predictions.select("prediction", "indexedLabel", "features").show(5) - - # Select (prediction, true label) and compute test accuracy - evaluator = MulticlassClassificationEvaluator( - labelCol="indexedLabel", predictionCol="prediction", metricName="accuracy") - accuracy = evaluator.evaluate(predictions) - print("Test set accuracy = %g" % accuracy) - - fmModel = model.stages[2] - print("Factors: " + str(fmModel.factors)) - print("Linear: " + str(fmModel.linear)) - print("Intercept: " + str(fmModel.intercept)) - # $example off$ - - spark.stop() diff --git a/examples/src/main/python/ml/fm_regressor_example.py b/examples/src/main/python/ml/fm_regressor_example.py deleted file mode 100644 index afd76396800b7..0000000000000 --- a/examples/src/main/python/ml/fm_regressor_example.py +++ /dev/null @@ -1,74 +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. -# - -""" -FMRegressor Example. -""" -from __future__ import print_function - -# $example on$ -from pyspark.ml import Pipeline -from pyspark.ml.regression import FMRegressor -from pyspark.ml.feature import MinMaxScaler -from pyspark.ml.evaluation import RegressionEvaluator -# $example off$ -from pyspark.sql import SparkSession - -if __name__ == "__main__": - spark = SparkSession \ - .builder \ - .appName("FMRegressorExample") \ - .getOrCreate() - - # $example on$ - # Load and parse the data file, converting it to a DataFrame. - data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - - # Scale features. - featureScaler = MinMaxScaler(inputCol="features", outputCol="scaledFeatures").fit(data) - - # Split the data into training and test sets (30% held out for testing) - (trainingData, testData) = data.randomSplit([0.7, 0.3]) - - # Train a FM model. - fm = FMRegressor(featuresCol="scaledFeatures", stepSize=0.001) - - # Create a Pipeline. - pipeline = Pipeline(stages=[featureScaler, fm]) - - # Train model. - model = pipeline.fit(trainingData) - - # Make predictions. - predictions = model.transform(testData) - - # Select example rows to display. - predictions.select("prediction", "label", "features").show(5) - - # Select (prediction, true label) and compute test error - evaluator = RegressionEvaluator( - labelCol="label", predictionCol="prediction", metricName="rmse") - rmse = evaluator.evaluate(predictions) - print("Root Mean Squared Error (RMSE) on test data = %g" % rmse) - - fmModel = model.stages[1] - print("Factors: " + str(fmModel.factors)) - print("Linear: " + str(fmModel.linear)) - print("Intercept: " + str(fmModel.intercept)) - # $example off$ - - spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala deleted file mode 100644 index 612a76fd125c3..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/FMClassifierExample.scala +++ /dev/null @@ -1,96 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.ml - -// $example on$ -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.{FMClassificationModel, FMClassifier} -import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator -import org.apache.spark.ml.feature.{IndexToString, MinMaxScaler, StringIndexer} -// $example off$ -import org.apache.spark.sql.SparkSession - -object FMClassifierExample { - def main(args: Array[String]): Unit = { - val spark = SparkSession - .builder - .appName("FMClassifierExample") - .getOrCreate() - - // $example on$ - // Load and parse the data file, converting it to a DataFrame. - val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - - // Index labels, adding metadata to the label column. - // Fit on whole dataset to include all labels in index. - val labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data) - // Scale features. - val featureScaler = new MinMaxScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures") - .fit(data) - - // Split the data into training and test sets (30% held out for testing). - val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) - - // Train a FM model. - val fm = new FMClassifier() - .setLabelCol("indexedLabel") - .setFeaturesCol("scaledFeatures") - .setStepSize(0.001) - - // Convert indexed labels back to original labels. - val labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labelsArray(0)) - - // Create a Pipeline. - val pipeline = new Pipeline() - .setStages(Array(labelIndexer, featureScaler, fm, labelConverter)) - - // Train model. - val model = pipeline.fit(trainingData) - - // Make predictions. - val predictions = model.transform(testData) - - // Select example rows to display. - predictions.select("predictedLabel", "label", "features").show(5) - - // Select (prediction, true label) and compute test accuracy. - val evaluator = new MulticlassClassificationEvaluator() - .setLabelCol("indexedLabel") - .setPredictionCol("prediction") - .setMetricName("accuracy") - val accuracy = evaluator.evaluate(predictions) - println(s"Test set accuracy = $accuracy") - - val fmModel = model.stages(2).asInstanceOf[FMClassificationModel] - println(s"Factors: ${fmModel.factors} Linear: ${fmModel.linear} " + - s"Intercept: ${fmModel.intercept}") - // $example off$ - - spark.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala deleted file mode 100644 index 6bb06eab33d9c..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/FMRegressorExample.scala +++ /dev/null @@ -1,84 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.ml - -// $example on$ -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.evaluation.RegressionEvaluator -import org.apache.spark.ml.feature.MinMaxScaler -import org.apache.spark.ml.regression.{FMRegressionModel, FMRegressor} -// $example off$ -import org.apache.spark.sql.SparkSession - -object FMRegressorExample { - def main(args: Array[String]): Unit = { - val spark = SparkSession - .builder - .appName("FMRegressorExample") - .getOrCreate() - - // $example on$ - // Load and parse the data file, converting it to a DataFrame. - val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - - // Scale features. - val featureScaler = new MinMaxScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures") - .fit(data) - - // Split the data into training and test sets (30% held out for testing). - val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) - - // Train a FM model. - val fm = new FMRegressor() - .setLabelCol("label") - .setFeaturesCol("scaledFeatures") - .setStepSize(0.001) - - // Create a Pipeline. - val pipeline = new Pipeline() - .setStages(Array(featureScaler, fm)) - - // Train model. - val model = pipeline.fit(trainingData) - - // Make predictions. - val predictions = model.transform(testData) - - // Select example rows to display. - predictions.select("prediction", "label", "features").show(5) - - // Select (prediction, true label) and compute test error. - val evaluator = new RegressionEvaluator() - .setLabelCol("label") - .setPredictionCol("prediction") - .setMetricName("rmse") - val rmse = evaluator.evaluate(predictions) - println(s"Root Mean Squared Error (RMSE) on test data = $rmse") - - val fmModel = model.stages(1).asInstanceOf[FMRegressionModel] - println(s"Factors: ${fmModel.factors} Linear: ${fmModel.linear} " + - s"Intercept: ${fmModel.intercept}") - // $example off$ - - spark.stop() - } -} -// scalastyle:on println diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala deleted file mode 100644 index 9cb296cacb108..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/FMClassifier.scala +++ /dev/null @@ -1,332 +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.ml.classification - -import org.apache.hadoop.fs.Path - -import org.apache.spark.annotation.Since -import org.apache.spark.internal.Logging -import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param._ -import org.apache.spark.ml.regression.{FactorizationMachines, FactorizationMachinesParams} -import org.apache.spark.ml.regression.FactorizationMachines._ -import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.mllib.linalg.{Vector => OldVector} -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.col -import org.apache.spark.storage.StorageLevel - -/** - * Params for FMClassifier. - */ -private[classification] trait FMClassifierParams extends ProbabilisticClassifierParams - with FactorizationMachinesParams { -} - -/** - * Factorization Machines learning algorithm for classification. - * It supports normal gradient descent and AdamW solver. - * - * The implementation is based upon: - * - * S. Rendle. "Factorization machines" 2010. - * - * FM is able to estimate interactions even in problems with huge sparsity - * (like advertising and recommendation system). - * FM formula is: - * {{{ - * y = \sigma\left( w_0 + \sum\limits^n_{i-1} w_i x_i + - * \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j \right) - * }}} - * First two terms denote global bias and linear term (as same as linear regression), - * and last term denotes pairwise interactions term. {{{v_i}}} describes the i-th variable - * with k factors. - * - * FM classification model uses logistic loss which can be solved by gradient descent method, and - * regularization terms like L2 are usually added to the loss function to prevent overfitting. - * - * @note Multiclass labels are not currently supported. - */ -@Since("3.0.0") -class FMClassifier @Since("3.0.0") ( - @Since("3.0.0") override val uid: String) - extends ProbabilisticClassifier[Vector, FMClassifier, FMClassificationModel] - with FactorizationMachines with FMClassifierParams with DefaultParamsWritable with Logging { - - @Since("3.0.0") - def this() = this(Identifiable.randomUID("fmc")) - - /** - * Set the dimensionality of the factors. - * Default is 8. - * - * @group setParam - */ - @Since("3.0.0") - def setFactorSize(value: Int): this.type = set(factorSize, value) - setDefault(factorSize -> 8) - - /** - * Set whether to fit intercept term. - * Default is true. - * - * @group setParam - */ - @Since("3.0.0") - def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) - - /** - * Set whether to fit linear term. - * Default is true. - * - * @group setParam - */ - @Since("3.0.0") - def setFitLinear(value: Boolean): this.type = set(fitLinear, value) - setDefault(fitLinear -> true) - - /** - * Set the L2 regularization parameter. - * Default is 0.0. - * - * @group setParam - */ - @Since("3.0.0") - def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) - - /** - * Set the mini-batch fraction parameter. - * Default is 1.0. - * - * @group setParam - */ - @Since("3.0.0") - def setMiniBatchFraction(value: Double): this.type = set(miniBatchFraction, value) - setDefault(miniBatchFraction -> 1.0) - - /** - * Set the standard deviation of initial coefficients. - * Default is 0.01. - * - * @group setParam - */ - @Since("3.0.0") - def setInitStd(value: Double): this.type = set(initStd, value) - setDefault(initStd -> 0.01) - - /** - * Set the maximum number of iterations. - * Default is 100. - * - * @group setParam - */ - @Since("3.0.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) - - /** - * Set the initial step size for the first step (like learning rate). - * Default is 1.0. - * - * @group setParam - */ - @Since("3.0.0") - def setStepSize(value: Double): this.type = set(stepSize, value) - setDefault(stepSize -> 1.0) - - /** - * Set the convergence tolerance of iterations. - * Default is 1E-6. - * - * @group setParam - */ - @Since("3.0.0") - def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) - - /** - * Set the solver algorithm used for optimization. - * Supported options: "gd", "adamW". - * Default: "adamW" - * - * @group setParam - */ - @Since("3.0.0") - def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> AdamW) - - /** - * Set the random seed for weight initialization. - * - * @group setParam - */ - @Since("3.0.0") - def setSeed(value: Long): this.type = set(seed, value) - - override protected[spark] def train( - dataset: Dataset[_] - ): FMClassificationModel = instrumented { instr => - - val handlePersistence = dataset.storageLevel == StorageLevel.NONE - val data: RDD[(Double, OldVector)] = - dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => - require(label == 0 || label == 1, s"FMClassifier was given" + - s" dataset with invalid label $label. Labels must be in {0,1}; note that" + - s" FMClassifier currently only supports binary classification.") - (label, features) - } - - if (handlePersistence) data.persist(StorageLevel.MEMORY_AND_DISK) - - val numClasses = 2 - if (isDefined(thresholds)) { - require($(thresholds).length == numClasses, this.getClass.getSimpleName + - ".train() called with non-matching numClasses and thresholds.length." + - s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") - } - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, factorSize, fitIntercept, fitLinear, regParam, - miniBatchFraction, initStd, maxIter, stepSize, tol, solver) - instr.logNumClasses(numClasses) - - val numFeatures = data.first()._2.size - instr.logNumFeatures(numFeatures) - - val coefficients = trainImpl(data, numFeatures, LogisticLoss) - - val (intercept, linear, factors) = splitCoefficients( - coefficients, numFeatures, $(factorSize), $(fitIntercept), $(fitLinear)) - - if (handlePersistence) data.unpersist() - - copyValues(new FMClassificationModel(uid, intercept, linear, factors)) - } - - @Since("3.0.0") - override def copy(extra: ParamMap): FMClassifier = defaultCopy(extra) -} - -@Since("3.0.0") -object FMClassifier extends DefaultParamsReadable[FMClassifier] { - - @Since("3.0.0") - override def load(path: String): FMClassifier = super.load(path) -} - -/** - * Model produced by [[FMClassifier]] - */ -@Since("3.0.0") -class FMClassificationModel private[classification] ( - @Since("3.0.0") override val uid: String, - @Since("3.0.0") val intercept: Double, - @Since("3.0.0") val linear: Vector, - @Since("3.0.0") val factors: Matrix) - extends ProbabilisticClassificationModel[Vector, FMClassificationModel] - with FMClassifierParams with MLWritable { - - @Since("3.0.0") - override val numClasses: Int = 2 - - @Since("3.0.0") - override val numFeatures: Int = linear.size - - override protected def predictRaw(features: Vector): Vector = { - val rawPrediction = getRawPrediction(features, intercept, linear, factors) - Vectors.dense(Array(-rawPrediction, rawPrediction)) - } - - override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { - rawPrediction match { - case dv: DenseVector => - dv.values(1) = 1.0 / (1.0 + math.exp(-dv.values(1))) - dv.values(0) = 1.0 - dv.values(1) - dv - case sv: SparseVector => - throw new RuntimeException("Unexpected error in FMClassificationModel:" + - " raw2probabilityInPlace encountered SparseVector") - } - } - - @Since("3.0.0") - override def copy(extra: ParamMap): FMClassificationModel = { - copyValues(new FMClassificationModel(uid, intercept, linear, factors), extra) - } - - @Since("3.0.0") - override def write: MLWriter = - new FMClassificationModel.FMClassificationModelWriter(this) - - override def toString: String = { - s"FMClassificationModel: " + - s"uid=${super.toString}, numClasses=$numClasses, numFeatures=$numFeatures, " + - s"factorSize=${$(factorSize)}, fitLinear=${$(fitLinear)}, fitIntercept=${$(fitIntercept)}" - } -} - -@Since("3.0.0") -object FMClassificationModel extends MLReadable[FMClassificationModel] { - - @Since("3.0.0") - override def read: MLReader[FMClassificationModel] = new FMClassificationModelReader - - @Since("3.0.0") - override def load(path: String): FMClassificationModel = super.load(path) - - /** [[MLWriter]] instance for [[FMClassificationModel]] */ - private[FMClassificationModel] class FMClassificationModelWriter( - instance: FMClassificationModel) extends MLWriter with Logging { - - private case class Data( - intercept: Double, - linear: Vector, - factors: Matrix) - - override protected def saveImpl(path: String): Unit = { - DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.intercept, instance.linear, instance.factors) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } - } - - private class FMClassificationModelReader extends MLReader[FMClassificationModel] { - - private val className = classOf[FMClassificationModel].getName - - override def load(path: String): FMClassificationModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - val data = sparkSession.read.format("parquet").load(dataPath) - - val Row(intercept: Double, linear: Vector, factors: Matrix) = - data.select("intercept", "linear", "factors").head() - val model = new FMClassificationModel(metadata.uid, intercept, linear, factors) - metadata.getAndSetParams(model) - model - } - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala deleted file mode 100644 index 3627946becfb9..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala +++ /dev/null @@ -1,815 +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.ml.regression - -import scala.util.Random - -import breeze.linalg.{axpy => brzAxpy, norm => brzNorm, Vector => BV} -import breeze.numerics.{sqrt => brzSqrt} -import org.apache.hadoop.fs.Path - -import org.apache.spark.annotation.Since -import org.apache.spark.internal.Logging -import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} -import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.linalg.BLAS._ -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.regression.FactorizationMachines._ -import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.mllib.{linalg => OldLinalg} -import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.mllib.optimization.{Gradient, GradientDescent, SquaredL2Updater, Updater} -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.functions.col -import org.apache.spark.storage.StorageLevel - -/** - * Params for Factorization Machines - */ -private[ml] trait FactorizationMachinesParams extends PredictorParams - with HasMaxIter with HasStepSize with HasTol with HasSolver with HasSeed - with HasFitIntercept with HasRegParam { - - /** - * Param for dimensionality of the factors (>= 0) - * @group param - */ - @Since("3.0.0") - final val factorSize: IntParam = new IntParam(this, "factorSize", - "Dimensionality of the factor vectors, " + - "which are used to get pairwise interactions between variables", - ParamValidators.gt(0)) - - /** @group getParam */ - @Since("3.0.0") - final def getFactorSize: Int = $(factorSize) - - /** - * Param for whether to fit linear term (aka 1-way term) - * @group param - */ - @Since("3.0.0") - final val fitLinear: BooleanParam = new BooleanParam(this, "fitLinear", - "whether to fit linear term (aka 1-way term)") - - /** @group getParam */ - @Since("3.0.0") - final def getFitLinear: Boolean = $(fitLinear) - - /** - * Param for mini-batch fraction, must be in range (0, 1] - * @group param - */ - @Since("3.0.0") - final val miniBatchFraction: DoubleParam = new DoubleParam(this, "miniBatchFraction", - "fraction of the input data set that should be used for one iteration of gradient descent", - ParamValidators.inRange(0, 1, false, true)) - - /** @group getParam */ - @Since("3.0.0") - final def getMiniBatchFraction: Double = $(miniBatchFraction) - - /** - * Param for standard deviation of initial coefficients - * @group param - */ - @Since("3.0.0") - final val initStd: DoubleParam = new DoubleParam(this, "initStd", - "standard deviation of initial coefficients", ParamValidators.gt(0)) - - /** @group getParam */ - @Since("3.0.0") - final def getInitStd: Double = $(initStd) - - /** - * The solver algorithm for optimization. - * Supported options: "gd", "adamW". - * Default: "adamW" - * - * @group param - */ - @Since("3.0.0") - final override val solver: Param[String] = new Param[String](this, "solver", - "The solver algorithm for optimization. Supported options: " + - s"${supportedSolvers.mkString(", ")}. (Default adamW)", - ParamValidators.inArray[String](supportedSolvers)) -} - -private[ml] trait FactorizationMachines extends FactorizationMachinesParams { - - private[ml] def initCoefficients(numFeatures: Int): OldVector = { - val rnd = new Random($(seed)) - val initialCoefficients = - OldVectors.dense( - Array.fill($(factorSize) * numFeatures)(rnd.nextGaussian() * $(initStd)) ++ - (if ($(fitLinear)) new Array[Double](numFeatures) else Array.emptyDoubleArray) ++ - (if ($(fitIntercept)) new Array[Double](1) else Array.emptyDoubleArray)) - initialCoefficients - } - - private[ml] def trainImpl( - data: RDD[(Double, OldVector)], - numFeatures: Int, - loss: String - ): Vector = { - - // initialize coefficients - val initialCoefficients = initCoefficients(numFeatures) - val coefficientsSize = initialCoefficients.size - - // optimize coefficients with gradient descent - val gradient = parseLoss(loss, $(factorSize), $(fitIntercept), $(fitLinear), numFeatures) - - val updater = parseSolver($(solver), coefficientsSize) - - val optimizer = new GradientDescent(gradient, updater) - .setStepSize($(stepSize)) - .setNumIterations($(maxIter)) - .setRegParam($(regParam)) - .setMiniBatchFraction($(miniBatchFraction)) - .setConvergenceTol($(tol)) - val coefficients = optimizer.optimize(data, initialCoefficients) - coefficients.asML - } -} - -private[ml] object FactorizationMachines { - - /** String name for "gd". */ - val GD = "gd" - - /** String name for "adamW". */ - val AdamW = "adamW" - - /** Set of solvers that FactorizationMachines supports. */ - val supportedSolvers = Array(GD, AdamW) - - /** String name for "logisticLoss". */ - val LogisticLoss = "logisticLoss" - - /** String name for "squaredError". */ - val SquaredError = "squaredError" - - /** Set of loss function names that FactorizationMachines supports. */ - val supportedRegressorLosses = Array(SquaredError) - val supportedClassifierLosses = Array(LogisticLoss) - val supportedLosses = supportedRegressorLosses ++ supportedClassifierLosses - - def parseSolver(solver: String, coefficientsSize: Int): Updater = { - solver match { - case GD => new SquaredL2Updater() - case AdamW => new AdamWUpdater(coefficientsSize) - } - } - - def parseLoss( - lossFunc: String, - factorSize: Int, - fitIntercept: Boolean, - fitLinear: Boolean, - numFeatures: Int - ): BaseFactorizationMachinesGradient = { - - lossFunc match { - case LogisticLoss => - new LogisticFactorizationMachinesGradient(factorSize, fitIntercept, fitLinear, numFeatures) - case SquaredError => - new MSEFactorizationMachinesGradient(factorSize, fitIntercept, fitLinear, numFeatures) - case _ => throw new IllegalArgumentException(s"loss function type $lossFunc is invalidation") - } - } - - def splitCoefficients( - coefficients: Vector, - numFeatures: Int, - factorSize: Int, - fitIntercept: Boolean, - fitLinear: Boolean - ): (Double, Vector, Matrix) = { - - val coefficientsSize = numFeatures * factorSize + - (if (fitLinear) numFeatures else 0) + (if (fitIntercept) 1 else 0) - require(coefficientsSize == coefficients.size, - s"coefficients.size did not match the excepted size ${coefficientsSize}") - - val intercept = if (fitIntercept) coefficients(coefficients.size - 1) else 0.0 - val linear: Vector = if (fitLinear) { - new DenseVector(coefficients.toArray.slice( - numFeatures * factorSize, numFeatures * factorSize + numFeatures)) - } else { - Vectors.sparse(numFeatures, Seq.empty) - } - val factors = new DenseMatrix(numFeatures, factorSize, - coefficients.toArray.slice(0, numFeatures * factorSize), true) - (intercept, linear, factors) - } - - def combineCoefficients( - intercept: Double, - linear: Vector, - factors: Matrix, - fitIntercept: Boolean, - fitLinear: Boolean - ): Vector = { - - val coefficients = factors.toDense.values ++ - (if (fitLinear) linear.toArray else Array.emptyDoubleArray) ++ - (if (fitIntercept) Array(intercept) else Array.emptyDoubleArray) - new DenseVector(coefficients) - } - - def getRawPrediction( - features: Vector, - intercept: Double, - linear: Vector, - factors: Matrix - ): Double = { - var rawPrediction = intercept + features.dot(linear) - (0 until factors.numCols).foreach { f => - var sumSquare = 0.0 - var sum = 0.0 - features.foreachActive { case (index, value) => - val vx = factors(index, f) * value - sumSquare += vx * vx - sum += vx - } - rawPrediction += 0.5 * (sum * sum - sumSquare) - } - - rawPrediction - } -} - -/** - * Params for FMRegressor - */ -private[regression] trait FMRegressorParams extends FactorizationMachinesParams { -} - -/** - * Factorization Machines learning algorithm for regression. - * It supports normal gradient descent and AdamW solver. - * - * The implementation is based upon: - * - * S. Rendle. "Factorization machines" 2010. - * - * FM is able to estimate interactions even in problems with huge sparsity - * (like advertising and recommendation system). - * FM formula is: - * {{{ - * y = w_0 + \sum\limits^n_{i-1} w_i x_i + - * \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j - * }}} - * First two terms denote global bias and linear term (as same as linear regression), - * and last term denotes pairwise interactions term. {{{v_i}}} describes the i-th variable - * with k factors. - * - * FM regression model uses MSE loss which can be solved by gradient descent method, and - * regularization terms like L2 are usually added to the loss function to prevent overfitting. - */ -@Since("3.0.0") -class FMRegressor @Since("3.0.0") ( - @Since("3.0.0") override val uid: String) - extends Predictor[Vector, FMRegressor, FMRegressionModel] - with FactorizationMachines with FMRegressorParams with DefaultParamsWritable with Logging { - - @Since("3.0.0") - def this() = this(Identifiable.randomUID("fmr")) - - /** - * Set the dimensionality of the factors. - * Default is 8. - * - * @group setParam - */ - @Since("3.0.0") - def setFactorSize(value: Int): this.type = set(factorSize, value) - setDefault(factorSize -> 8) - - /** - * Set whether to fit intercept term. - * Default is true. - * - * @group setParam - */ - @Since("3.0.0") - def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) - setDefault(fitIntercept -> true) - - /** - * Set whether to fit linear term. - * Default is true. - * - * @group setParam - */ - @Since("3.0.0") - def setFitLinear(value: Boolean): this.type = set(fitLinear, value) - setDefault(fitLinear -> true) - - /** - * Set the L2 regularization parameter. - * Default is 0.0. - * - * @group setParam - */ - @Since("3.0.0") - def setRegParam(value: Double): this.type = set(regParam, value) - setDefault(regParam -> 0.0) - - /** - * Set the mini-batch fraction parameter. - * Default is 1.0. - * - * @group setParam - */ - @Since("3.0.0") - def setMiniBatchFraction(value: Double): this.type = set(miniBatchFraction, value) - setDefault(miniBatchFraction -> 1.0) - - /** - * Set the standard deviation of initial coefficients. - * Default is 0.01. - * - * @group setParam - */ - @Since("3.0.0") - def setInitStd(value: Double): this.type = set(initStd, value) - setDefault(initStd -> 0.01) - - /** - * Set the maximum number of iterations. - * Default is 100. - * - * @group setParam - */ - @Since("3.0.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - setDefault(maxIter -> 100) - - /** - * Set the initial step size for the first step (like learning rate). - * Default is 1.0. - * - * @group setParam - */ - @Since("3.0.0") - def setStepSize(value: Double): this.type = set(stepSize, value) - setDefault(stepSize -> 1.0) - - /** - * Set the convergence tolerance of iterations. - * Default is 1E-6. - * - * @group setParam - */ - @Since("3.0.0") - def setTol(value: Double): this.type = set(tol, value) - setDefault(tol -> 1E-6) - - /** - * Set the solver algorithm used for optimization. - * Supported options: "gd", "adamW". - * Default: "adamW" - * - * @group setParam - */ - @Since("3.0.0") - def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> AdamW) - - /** - * Set the random seed for weight initialization. - * - * @group setParam - */ - @Since("3.0.0") - def setSeed(value: Long): this.type = set(seed, value) - - override protected[spark] def train( - dataset: Dataset[_] - ): FMRegressionModel = instrumented { instr => - - val handlePersistence = dataset.storageLevel == StorageLevel.NONE - val data: RDD[(Double, OldVector)] = - dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { - case Row(label: Double, features: Vector) => - (label, features) - } - - if (handlePersistence) data.persist(StorageLevel.MEMORY_AND_DISK) - - instr.logPipelineStage(this) - instr.logDataset(dataset) - instr.logParams(this, factorSize, fitIntercept, fitLinear, regParam, - miniBatchFraction, initStd, maxIter, stepSize, tol, solver) - - val numFeatures = data.first()._2.size - instr.logNumFeatures(numFeatures) - - val coefficients = trainImpl(data, numFeatures, SquaredError) - - val (intercept, linear, factors) = splitCoefficients( - coefficients, numFeatures, $(factorSize), $(fitIntercept), $(fitLinear)) - - if (handlePersistence) data.unpersist() - - copyValues(new FMRegressionModel(uid, intercept, linear, factors)) - } - - @Since("3.0.0") - override def copy(extra: ParamMap): FMRegressor = defaultCopy(extra) -} - -@Since("3.0.0") -object FMRegressor extends DefaultParamsReadable[FMRegressor] { - - @Since("3.0.0") - override def load(path: String): FMRegressor = super.load(path) -} - -/** - * Model produced by [[FMRegressor]]. - */ -@Since("3.0.0") -class FMRegressionModel private[regression] ( - @Since("3.0.0") override val uid: String, - @Since("3.0.0") val intercept: Double, - @Since("3.0.0") val linear: Vector, - @Since("3.0.0") val factors: Matrix) - extends PredictionModel[Vector, FMRegressionModel] - with FMRegressorParams with MLWritable { - - @Since("3.0.0") - override val numFeatures: Int = linear.size - - override def predict(features: Vector): Double = { - getRawPrediction(features, intercept, linear, factors) - } - - @Since("3.0.0") - override def copy(extra: ParamMap): FMRegressionModel = { - copyValues(new FMRegressionModel(uid, intercept, linear, factors), extra) - } - - @Since("3.0.0") - override def write: MLWriter = - new FMRegressionModel.FMRegressionModelWriter(this) - - override def toString: String = { - s"FMRegressionModel: " + - s"uid=${super.toString}, numFeatures=$numFeatures, " + - s"factorSize=${$(factorSize)}, fitLinear=${$(fitLinear)}, fitIntercept=${$(fitIntercept)}" - } -} - -@Since("3.0.0") -object FMRegressionModel extends MLReadable[FMRegressionModel] { - - @Since("3.0.0") - override def read: MLReader[FMRegressionModel] = new FMRegressionModelReader - - @Since("3.0.0") - override def load(path: String): FMRegressionModel = super.load(path) - - /** [[MLWriter]] instance for [[FMRegressionModel]] */ - private[FMRegressionModel] class FMRegressionModelWriter( - instance: FMRegressionModel) extends MLWriter with Logging { - - private case class Data( - intercept: Double, - linear: Vector, - factors: Matrix) - - override protected def saveImpl(path: String): Unit = { - DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.intercept, instance.linear, instance.factors) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } - } - - private class FMRegressionModelReader extends MLReader[FMRegressionModel] { - - private val className = classOf[FMRegressionModel].getName - - override def load(path: String): FMRegressionModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - val data = sparkSession.read.format("parquet").load(dataPath) - - val Row(intercept: Double, linear: Vector, factors: Matrix) = data - .select("intercept", "linear", "factors").head() - val model = new FMRegressionModel(metadata.uid, intercept, linear, factors) - metadata.getAndSetParams(model) - model - } - } -} - -/** - * Factorization Machines base gradient class - * Implementing the raw FM formula, include raw prediction and raw gradient, - * then inherit the base class to implement special gradient class(like logloss, mse). - * - * Factorization Machines raw formula: - * {{{ - * y_{fm} = w_0 + \sum\limits^n_{i-1} w_i x_i + - * \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j - * }}} - * the pairwise interactions (2-way term) can be reformulated: - * {{{ - * \sum\limits^n_{i=1} \sum\limits^n_{j=i+1} \langle v_i, v_j \rangle x_i x_j - * = \frac{1}{2}\sum\limits^k_{f=1} - * \left(\left( \sum\limits^n_{i=1}v_{i,f}x_i \right)^2 - - * \sum\limits^n_{i=1}v_{i,f}^2x_i^2 \right) - * }}} - * and the gradients are: - * {{{ - * \frac{\partial}{\partial\theta}y_{fm} = \left\{ - * \begin{align} - * &1, & if\ \theta\ is\ w_0 \\ - * &x_i, & if\ \theta\ is\ w_i \\ - * &x_i{\sum}^n_{j=1}v_{j,f}x_j - v_{i,f}x_i^2, & if\ \theta\ is\ v_{i,j} \\ - * \end{align} - * \right. - * }}} - * - * Factorization Machines formula with prediction task: - * {{{ - * \hat{y} = p\left( y_{fm} \right) - * }}} - * p is the prediction function, for binary classification task is sigmoid. - * The loss funcation gradient formula: - * {{{ - * \frac{\partial}{\partial\theta} l\left( \hat{y},y \right) = - * \frac{\partial}{\partial\theta} l\left( p\left( y_{fm} \right),y \right) = - * \frac{\partial l}{\partial \hat{y}} \cdot - * \frac{\partial \hat{y}}{\partial y_{fm}} \cdot - * \frac{\partial y_{fm}}{\partial\theta} - * }}} - * Last term is same for all task, so be implemented in base gradient class. - * last term named rawGradient in following code, and first two term named multiplier. - */ -private[ml] abstract class BaseFactorizationMachinesGradient( - factorSize: Int, - fitIntercept: Boolean, - fitLinear: Boolean, - numFeatures: Int) extends Gradient { - - override def compute( - data: OldVector, - label: Double, - weights: OldVector, - cumGradient: OldVector): Double = { - val (rawPrediction, sumVX) = getRawPrediction(data, weights) - val rawGradient = getRawGradient(data, weights, sumVX) - val multiplier = getMultiplier(rawPrediction, label) - axpy(multiplier, rawGradient, cumGradient) - val loss = getLoss(rawPrediction, label) - loss - } - - def getPrediction(rawPrediction: Double): Double - - protected def getMultiplier(rawPrediction: Double, label: Double): Double - - protected def getLoss(rawPrediction: Double, label: Double): Double - - def getRawPrediction(data: OldVector, weights: OldVector): (Double, Array[Double]) = { - val sumVX = new Array[Double](factorSize) - var rawPrediction = 0.0 - val vWeightsSize = numFeatures * factorSize - - if (fitIntercept) rawPrediction += weights(weights.size - 1) - if (fitLinear) { - data.foreachActive { case (index, value) => - rawPrediction += weights(vWeightsSize + index) * value - } - } - (0 until factorSize).foreach { f => - var sumSquare = 0.0 - var sum = 0.0 - data.foreachActive { case (index, value) => - val vx = weights(index * factorSize + f) * value - sumSquare += vx * vx - sum += vx - } - sumVX(f) = sum - rawPrediction += 0.5 * (sum * sum - sumSquare) - } - - (rawPrediction, sumVX) - } - - private def getRawGradient( - data: OldVector, - weights: OldVector, - sumVX: Array[Double] - ): OldVector = { - data match { - // Usually Factorization Machines is used, there will be a lot of sparse features. - // So need to optimize the gradient descent of sparse vector. - case data: OldLinalg.SparseVector => - val gardSize = data.indices.length * factorSize + - (if (fitLinear) data.indices.length else 0) + - (if (fitIntercept) 1 else 0) - val gradIndex = Array.fill(gardSize)(0) - val gradValue = Array.fill(gardSize)(0.0) - var gradI = 0 - val vWeightsSize = numFeatures * factorSize - - data.foreachActive { case (index, value) => - (0 until factorSize).foreach { f => - gradIndex(gradI) = index * factorSize + f - gradValue(gradI) = value * sumVX(f) - weights(index * factorSize + f) * value * value - gradI += 1 - } - } - if (fitLinear) { - data.foreachActive { case (index, value) => - gradIndex(gradI) = vWeightsSize + index - gradValue(gradI) = value - gradI += 1 - } - } - if (fitIntercept) { - gradIndex(gradI) = weights.size - 1 - gradValue(gradI) = 1.0 - } - - OldVectors.sparse(weights.size, gradIndex, gradValue) - case data: OldLinalg.DenseVector => - val gradient = Array.fill(weights.size)(0.0) - val vWeightsSize = numFeatures * factorSize - - if (fitIntercept) gradient(weights.size - 1) += 1.0 - if (fitLinear) { - data.foreachActive { case (index, value) => - gradient(vWeightsSize + index) += value - } - } - (0 until factorSize).foreach { f => - data.foreachActive { case (index, value) => - gradient(index * factorSize + f) += - value * sumVX(f) - weights(index * factorSize + f) * value * value - } - } - - OldVectors.dense(gradient) - } - } -} - -/** - * FM with logistic loss - * prediction formula: - * {{{ - * \hat{y} = \sigmoid(y_{fm}) - * }}} - * loss formula: - * {{{ - * - y * log(\hat{y}) - (1 - y) * log(1 - \hat{y}) - * }}} - * multiplier formula: - * {{{ - * \frac{\partial l}{\partial \hat{y}} \cdot - * \frac{\partial \hat{y}}{\partial y_{fm}} = - * \hat{y} - y - * }}} - */ -private[ml] class LogisticFactorizationMachinesGradient( - factorSize: Int, - fitIntercept: Boolean, - fitLinear: Boolean, - numFeatures: Int) - extends BaseFactorizationMachinesGradient( - factorSize: Int, - fitIntercept: Boolean, - fitLinear: Boolean, - numFeatures: Int) with Logging { - - override def getPrediction(rawPrediction: Double): Double = { - 1.0 / (1.0 + math.exp(-rawPrediction)) - } - - override protected def getMultiplier(rawPrediction: Double, label: Double): Double = { - getPrediction(rawPrediction) - label - } - - override protected def getLoss(rawPrediction: Double, label: Double): Double = { - if (label > 0) MLUtils.log1pExp(-rawPrediction) - else MLUtils.log1pExp(rawPrediction) - } -} - -/** - * FM with mse - * prediction formula: - * {{{ - * \hat{y} = y_{fm} - * }}} - * loss formula: - * {{{ - * (\hat{y} - y) ^ 2 - * }}} - * multiplier formula: - * {{{ - * \frac{\partial l}{\partial \hat{y}} \cdot - * \frac{\partial \hat{y}}{\partial y_{fm}} = - * 2 * (\hat{y} - y) - * }}} - */ -private[ml] class MSEFactorizationMachinesGradient( - factorSize: Int, - fitIntercept: Boolean, - fitLinear: Boolean, - numFeatures: Int) - extends BaseFactorizationMachinesGradient( - factorSize: Int, - fitIntercept: Boolean, - fitLinear: Boolean, - numFeatures: Int) with Logging { - - override def getPrediction(rawPrediction: Double): Double = { - rawPrediction - } - - override protected def getMultiplier(rawPrediction: Double, label: Double): Double = { - 2 * (rawPrediction - label) - } - - override protected def getLoss(rawPrediction: Double, label: Double): Double = { - (rawPrediction - label) * (rawPrediction - label) - } -} - -/** - * AdamW optimizer. - * - * The implementation is based upon: - * - * Loshchilov I, Hutter F. "DECOUPLED WEIGHT DECAY REGULARIZATION" 2019. - * - * The main contribution of this paper is to improve regularization in Adam - * by decoupling the weight decay from the gradient-based update. - * This paper proposed a simple modification to recover the original formulation of - * weight decay regularization by decoupling the weight decay from the optimization steps - * taken w.r.t. the loss function. - */ -private[ml] class AdamWUpdater(weightSize: Int) extends Updater with Logging { - val beta1: Double = 0.9 - val beta2: Double = 0.999 - val epsilon: Double = 1e-8 - - val m: BV[Double] = BV.zeros[Double](weightSize).toDenseVector - val v: BV[Double] = BV.zeros[Double](weightSize).toDenseVector - var beta1T: Double = 1.0 - var beta2T: Double = 1.0 - - override def compute( - weightsOld: OldVector, - gradient: OldVector, - stepSize: Double, - iter: Int, - regParam: Double - ): (OldVector, Double) = { - val w: BV[Double] = weightsOld.asBreeze.toDenseVector - val lr = stepSize // learning rate - if (stepSize > 0) { - val g: BV[Double] = gradient.asBreeze.toDenseVector - m *= beta1 - brzAxpy(1 - beta1, g, m) - v *= beta2 - brzAxpy(1 - beta2, g * g, v) - beta1T *= beta1 - beta2T *= beta2 - val mHat = m / (1 - beta1T) - val vHat = v / (1 - beta2T) - w -= lr * mHat / (brzSqrt(vHat) + epsilon) + regParam * w - } - val norm = brzNorm(w, 2.0) - - (Vectors.fromBreeze(w), 0.5 * regParam * norm * norm) - } -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala deleted file mode 100644 index d477049824b19..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/FMClassifierSuite.scala +++ /dev/null @@ -1,242 +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.ml.classification - -import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput -import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.regression.FMRegressorSuite._ -import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.sql.{DataFrame, Row} - -class FMClassifierSuite extends MLTest with DefaultReadWriteTest { - - import testImplicits._ - - private val seed = 42 - @transient var smallBinaryDataset: DataFrame = _ - - override def beforeAll(): Unit = { - super.beforeAll() - - smallBinaryDataset = generateLogisticInput(1.0, 1.0, nPoints = 100, seed = seed).toDF() - } - - test("params") { - ParamsSuite.checkParams(new FMClassifier) - val model = new FMClassificationModel("fmc_test", 0.0, Vectors.dense(0.0), - new DenseMatrix(1, 8, new Array[Double](8))) - ParamsSuite.checkParams(model) - } - - test("FMClassifier: Predictor, Classifier methods") { - val sqlContext = smallBinaryDataset.sqlContext - import sqlContext.implicits._ - val fm = new FMClassifier() - - val model = fm.fit(smallBinaryDataset) - assert(model.numClasses === 2) - val numFeatures = smallBinaryDataset.select("features").first().getAs[Vector](0).size - assert(model.numFeatures === numFeatures) - - testTransformer[(Double, Vector)](smallBinaryDataset.toDF(), - model, "rawPrediction", "probability", "prediction") { - case Row(raw: Vector, prob: Vector, pred: Double) => - // Compare rawPrediction with probability - assert(raw.size === 2) - assert(prob.size === 2) - val probFromRaw1 = 1.0 / (1.0 + math.exp(-raw(1))) - assert(prob(1) ~== probFromRaw1 relTol 1E-6) - assert(prob(0) ~== 1.0 - probFromRaw1 relTol 1E-6) - // Compare prediction with probability - val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 - assert(pred == predFromProb) - } - - ProbabilisticClassifierSuite.testPredictMethods[ - Vector, FMClassificationModel](this, model, smallBinaryDataset) - } - - def logLoss(modelRes: DataFrame): Double = { - modelRes.select("label", "probability").rdd.map { - case Row(label: Double, probability: DenseVector) => - if (label > 0) -math.log(probability(1)) - else -math.log(probability(0)) - }.mean() - } - - test("check logisticLoss with AdamW") { - // This testcase only tests whether the FM logloss part is valid and does not test the - // pairwise interaction logic. The pairwise interaction logic be tested in FMRegressor. - // When there is only one feature, FM will degenerate into LR. So two models will get - // almost same loss value. - - val fm = new FMClassifier().setMaxIter(50) - val fmModel = fm.fit(smallBinaryDataset) - val fmRes = fmModel.transform(smallBinaryDataset) - val fmLogLoss = logLoss(fmRes) - - /* - Use following code to fit the dataset, the resulting logloss is 0.4756465459065247. - val lr = new LogisticRegression() - val lrModel = lr.fit(smallBinaryDataset) - val lrRes = lrModel.transform(smallBinaryDataset) - val lrLogLoss = logLoss(lrRes) - */ - assert(fmLogLoss ~== 0.4756465459065247 absTol 1E-3) - } - - test("check logisticLoss with GD") { - val fm = new FMClassifier().setSolver("gd") - val fmModel = fm.fit(smallBinaryDataset) - val fmRes = fmModel.transform(smallBinaryDataset) - val fmLogLoss = logLoss(fmRes) - assert(fmLogLoss ~== 0.4756465459065247 absTol 1E-2) - } - - test("sparse datasets") { - // test sparse input will not throw exception - val dataset = spark.createDataFrame(Array( - (1.0, Vectors.dense(Array(1.0, 2.0, 3.0))), - (0.0, Vectors.sparse(3, Array(0, 2), Array(-1.0, 2.0))), - (0.0, Vectors.sparse(3, Array.emptyIntArray, Array.emptyDoubleArray)), - (1.0, Vectors.sparse(3, Array(0, 1), Array(2.0, 3.0))) - )).toDF("label", "features") - val fm = new FMClassifier().setMaxIter(10) - fm.fit(dataset) - } - - test("setThreshold, getThreshold") { - val fm = new FMClassifier() - - // default - withClue("FMClassifier should not have thresholds set by default.") { - intercept[NoSuchElementException] { - fm.getThresholds - } - } - - // Set via thresholds - val fm2 = new FMClassifier() - val threshold = Array(0.3, 0.7) - fm2.setThresholds(threshold) - assert(fm2.getThresholds === threshold) - } - - test("thresholds prediction") { - val fm = new FMClassifier() - val df = smallBinaryDataset.toDF() - val fmModel = fm.fit(df) - - // should predict all zeros - fmModel.setThresholds(Array(0.0, 1.0)) - testTransformer[(Double, Vector)](df, fmModel, "prediction") { - case Row(prediction: Double) => prediction === 0.0 - } - - // should predict all ones - fmModel.setThresholds(Array(1.0, 0.0)) - testTransformer[(Double, Vector)](df, fmModel, "prediction") { - case Row(prediction: Double) => prediction === 1.0 - } - - val fmBase = new FMClassifier() - val model = fmBase.fit(df) - val basePredictions = model.transform(df).select("prediction").collect() - - // constant threshold scaling is the same as no thresholds - fmModel.setThresholds(Array(1.0, 1.0)) - testTransformerByGlobalCheckFunc[(Double, Vector)](df, fmModel, "prediction") { - scaledPredictions: Seq[Row] => - assert(scaledPredictions.zip(basePredictions).forall { case (scaled, base) => - scaled.getDouble(0) === base.getDouble(0) - }) - } - - // force it to use the predict method - model.setRawPredictionCol("").setProbabilityCol("").setThresholds(Array(0, 1)) - testTransformer[(Double, Vector)](df, model, "prediction") { - case Row(prediction: Double) => prediction === 0.0 - } - } - - test("FMClassifier doesn't fit intercept when fitIntercept is off") { - val fm = new FMClassifier().setFitIntercept(false) - val model = fm.fit(smallBinaryDataset) - assert(model.intercept === 0.0) - } - - test("FMClassifier doesn't fit linear when fitLinear is off") { - val fm = new FMClassifier().setFitLinear(false) - val model = fm.fit(smallBinaryDataset) - assert(model.linear === Vectors.sparse(model.numFeatures, Seq.empty)) - } - - test("prediction on single instance") { - val fm = new FMClassifier() - val fmModel = fm.fit(smallBinaryDataset) - testPredictionModelSinglePrediction(fmModel, smallBinaryDataset) - } - - test("read/write") { - def checkModelData( - model: FMClassificationModel, - model2: FMClassificationModel - ): Unit = { - assert(model.intercept === model2.intercept) - assert(model.linear.toArray === model2.linear.toArray) - assert(model.factors.toArray === model2.factors.toArray) - assert(model.numFeatures === model2.numFeatures) - } - val fm = new FMClassifier() - val data = smallBinaryDataset - .withColumnRenamed("features", allParamSettings("featuresCol").toString) - .withColumnRenamed("label", allParamSettings("labelCol").toString) - testEstimatorAndModelReadWrite(fm, data, allParamSettings, - allParamSettings, checkModelData) - } -} - -object FMClassifierSuite { - - /** - * Mapping from all Params to valid settings which differ from the defaults. - * This is useful for tests which need to exercise all Params, such as save/load. - * This excludes input columns to simplify some tests. - */ - val allParamSettings: Map[String, Any] = Map( - "featuresCol" -> "myFeatures", - "labelCol" -> "myLabel", - "predictionCol" -> "prediction", - "rawPredictionCol" -> "rawPrediction", - "probabilityCol" -> "probability", - "factorSize" -> 4, - "fitIntercept" -> false, - "fitLinear" -> false, - "regParam" -> 0.01, - "miniBatchFraction" -> 0.1, - "initStd" -> 0.01, - "maxIter" -> 2, - "stepSize" -> 0.1, - "tol" -> 1e-4, - "solver" -> "gd", - "seed" -> 10L, - "thresholds" -> Array(0.4, 0.6) - ) -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/FMRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/FMRegressorSuite.scala deleted file mode 100644 index 372432ceb3a41..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/FMRegressorSuite.scala +++ /dev/null @@ -1,240 +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.ml.regression - -import scala.util.Random - -import org.apache.spark.ml.linalg._ -import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.regression.FactorizationMachines._ -import org.apache.spark.ml.regression.FMRegressorSuite._ -import org.apache.spark.ml.util._ -import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.functions.{avg, col} - -class FMRegressorSuite extends MLTest with DefaultReadWriteTest { - - private val seed = 10 - @transient var crossDataset: DataFrame = _ - - override def beforeAll(): Unit = { - super.beforeAll() - - val (crossDatasetTmp, _) = generateFactorInteractionInput( - spark, 2, 10, 1000, seed, true, true) - crossDataset = crossDatasetTmp - } - - test("params") { - ParamsSuite.checkParams(new FMRegressor) - val model = new FMRegressionModel("fmr_test", 0.0, Vectors.dense(0.0), - new DenseMatrix(1, 8, new Array[Double](8))) - ParamsSuite.checkParams(model) - } - - test("combineCoefficients") { - val numFeatures = 2 - val factorSize = 4 - val b = 0.1 - val w = Vectors.dense(Array(0.2, 0.3)) - val v = new DenseMatrix(numFeatures, factorSize, - Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1), true) - - val expectList = Array( - (true, true, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.2, 0.3, 0.1)), - (false, true, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.2, 0.3)), - (true, false, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.1)), - (false, false, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1))) - - expectList.foreach { case (fitIntercept, fitLinear, expectCoeffs) => - assert(combineCoefficients(b, w, v, fitIntercept, fitLinear) === Vectors.dense(expectCoeffs)) - } - } - - test("splitCoefficients") { - val numFeatures = 2 - val factorSize = 4 - val b = 0.1 - val w = Vectors.dense(Array(0.2, 0.3)) - val v = new DenseMatrix(numFeatures, factorSize, - Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1), true) - val emptyB = 0.0 - val emptyW = Vectors.sparse(numFeatures, Seq.empty) - - val expectList = Array( - (true, true, b, w, v, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.2, 0.3, 0.1)), - (false, true, emptyB, w, v, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.2, 0.3)), - (true, false, b, emptyW, v, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 0.1)), - (false, false, emptyB, emptyW, v, Array(0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1))) - - expectList.foreach { case (fitIntercept, fitLinear, b1, w1, v1, coeffs) => - val (b2, w2, v2) = splitCoefficients(Vectors.dense(coeffs), - numFeatures, factorSize, fitIntercept, fitLinear) - assert(b1 === b2) - assert(w1 === w2) - assert(v1 === v2) - } - } - - def checkMSE(fitIntercept: Boolean, fitLinear: Boolean): Unit = { - val numFeatures = 3 - val numSamples = 200 - val factorSize = 2 - val (data, coefficients) = generateFactorInteractionInput( - spark, factorSize, numFeatures, numSamples, seed, fitIntercept, fitLinear) - val (b, w, v) = splitCoefficients(new DenseVector(coefficients), - numFeatures, factorSize, fitIntercept, fitLinear) - - val fm = new FMRegressor() - .setSolver("adamW") - .setFeaturesCol("features") - .setLabelCol("label") - .setFactorSize(factorSize) - .setFitIntercept(fitIntercept) - .setFitLinear(fitLinear) - .setInitStd(0.01) - .setMaxIter(100) - .setMiniBatchFraction(1.0) - .setStepSize(1.0) - .setRegParam(0.0) - .setTol(1E-6) - val fmModel = fm.fit(data) - val res = fmModel.transform(data) - - // check mse value - val mse = res.select((col("prediction") - col("label")).as("error")) - .select((col("error") * col("error")).as("error_square")) - .agg(avg("error_square")) - .collect()(0).getAs[Double](0) - assert(mse ~== 0.0 absTol 1E-4) - - // check coefficients - assert(b ~== fmModel.intercept absTol 1E-2) - assert(w ~== fmModel.linear absTol 1E-2) - (0 until numFeatures).foreach { i => - ((i + 1) until numFeatures).foreach { j => - // assert is same - var innerProd1 = 0.0 - var innerProd2 = 0.0 - (0 until factorSize).foreach { k => - innerProd1 += v(i, k) * v(j, k) - innerProd2 += fmModel.factors(i, k) * fmModel.factors(j, k) - } - assert(innerProd1 ~== innerProd2 absTol 1E-2) - } - } - } - - test("MSE with intercept and linear") { - checkMSE(true, true) - } - - test("MSE with intercept but without linear") { - checkMSE(true, false) - } - - test("MSE with linear but without intercept") { - checkMSE(false, true) - } - - test("MSE without intercept or linear") { - checkMSE(false, false) - } - - test("read/write") { - def checkModelData( - model: FMRegressionModel, - model2: FMRegressionModel - ): Unit = { - assert(model.intercept === model2.intercept) - assert(model.linear.toArray === model2.linear.toArray) - assert(model.factors.toArray === model2.factors.toArray) - assert(model.numFeatures === model2.numFeatures) - } - val fm = new FMRegressor() - val data = crossDataset - .withColumnRenamed("features", allParamSettings("featuresCol").toString) - .withColumnRenamed("label", allParamSettings("labelCol").toString) - testEstimatorAndModelReadWrite(fm, data, allParamSettings, - allParamSettings, checkModelData) - } -} - -object FMRegressorSuite { - - /** - * Mapping from all Params to valid settings which differ from the defaults. - * This is useful for tests which need to exercise all Params, such as save/load. - * This excludes input columns to simplify some tests. - */ - val allParamSettings: Map[String, Any] = Map( - "featuresCol" -> "myFeatures", - "labelCol" -> "myLabel", - "predictionCol" -> "prediction", - "factorSize" -> 2, - "fitIntercept" -> false, - "fitLinear" -> false, - "regParam" -> 0.01, - "miniBatchFraction" -> 0.1, - "initStd" -> 0.01, - "maxIter" -> 2, - "stepSize" -> 0.1, - "tol" -> 1e-4, - "solver" -> "gd", - "seed" -> 11L - ) - - def generateFactorInteractionInput( - spark: SparkSession, - factorSize: Int, - numFeatures: Int, - numSamples: Int, - seed: Int, - fitIntercept: Boolean, - fitLinear: Boolean - ): (DataFrame, Array[Double]) = { - import spark.implicits._ - val sc = spark.sparkContext - - // generate FM coefficients randomly - val rnd = new Random(seed) - val coefficientsSize = factorSize * numFeatures + - (if (fitLinear) numFeatures else 0) + (if (fitIntercept) 1 else 0) - val coefficients = Array.fill(coefficientsSize)(rnd.nextDouble() - 0.5) - val (intercept, linear, factors) = splitCoefficients( - Vectors.dense(coefficients), numFeatures, factorSize, fitIntercept, fitLinear) - - // generate samples randomly - val X: DataFrame = sc.parallelize(0 until numSamples).map { i => - val x = new DenseVector(Array.fill(numFeatures)(rnd.nextDouble() - 0.5)) - (i, x) - }.toDF("id", "features") - - // calculate FM prediction - val fmModel = new FMRegressionModel( - "fmr_test", intercept, linear, factors) - fmModel.set(fmModel.factorSize, factorSize) - fmModel.set(fmModel.fitIntercept, fitIntercept) - fmModel.set(fmModel.fitLinear, fitLinear) - val data = fmModel.transform(X) - .withColumnRenamed("prediction", "label") - .select("features", "label") - (data, coefficients) - } -} diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 10155b2a287d8..a615ca2cde94a 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -45,8 +45,7 @@ 'RandomForestClassifier', 'RandomForestClassificationModel', 'NaiveBayes', 'NaiveBayesModel', 'MultilayerPerceptronClassifier', 'MultilayerPerceptronClassificationModel', - 'OneVsRest', 'OneVsRestModel', - 'FMClassifier', 'FMClassificationModel'] + 'OneVsRest', 'OneVsRestModel'] class _JavaClassifierParams(HasRawPredictionCol, _JavaPredictorParams): @@ -2764,169 +2763,6 @@ def _to_java(self): return _java_obj -@inherit_doc -class FMClassifier(JavaProbabilisticClassifier, HasMaxIter, HasStepSize, HasTol, HasSolver, - HasSeed, HasFitIntercept, HasRegParam, JavaMLWritable, JavaMLReadable): - """ - Factorization Machines learning algorithm for classification. - - solver Supports: - - * gd (normal mini-batch gradient descent) - * adamW (default) - - >>> from pyspark.ml.linalg import Vectors - >>> from pyspark.ml.classification import FMClassifier - >>> df = spark.createDataFrame([ - ... (1.0, Vectors.dense(1.0)), - ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) - >>> fm = FMClassifier(factorSize=2) - >>> model = fm.fit(df) - >>> test0 = spark.createDataFrame([ - ... (Vectors.dense(-1.0),), - ... (Vectors.dense(0.5),), - ... (Vectors.dense(1.0),), - ... (Vectors.dense(2.0),)], ["features"]) - >>> model.transform(test0).select("features", "probability").show(10, False) - +--------+------------------------------------------+ - |features|probability | - +--------+------------------------------------------+ - |[-1.0] |[0.9999999997574736,2.425264676902229E-10]| - |[0.5] |[0.47627851732981163,0.5237214826701884] | - |[1.0] |[5.491554426243495E-4,0.9994508445573757] | - |[2.0] |[2.005766663870645E-10,0.9999999997994233]| - +--------+------------------------------------------+ - >>> model.intercept - -7.316665276826291 - >>> model.linear - DenseVector([14.8232]) - >>> model.factors - DenseMatrix(1, 2, [0.0028, 0.0048], 1) - - .. versionadded:: 3.0.0 - """ - - factorSize = Param(Params._dummy(), "factorSize", "Dimensionality of the factor vectors, " + - "which are used to get pairwise interactions between variables", - typeConverter=TypeConverters.toInt) - - fitLinear = Param(Params._dummy(), "fitLinear", "whether to fit linear term (aka 1-way term)", - typeConverter=TypeConverters.toBoolean) - - miniBatchFraction = Param(Params._dummy(), "miniBatchFraction", "fraction of the input data " + - "set that should be used for one iteration of gradient descent", - typeConverter=TypeConverters.toFloat) - - initStd = Param(Params._dummy(), "initStd", "standard deviation of initial coefficients", - typeConverter=TypeConverters.toFloat) - - solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + - "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) - - @keyword_only - def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", - probabilityCol="probability", rawPredictionCol="rawPrediction", - factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, - tol=1e-6, solver="adamW", thresholds=None, seed=None): - """ - __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ - probabilityCol="probability", rawPredictionCol="rawPrediction", \ - factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, \ - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, \ - tol=1e-6, solver="adamW", thresholds=None, seed=None) - """ - super(FMClassifier, self).__init__() - self._java_obj = self._new_java_obj( - "org.apache.spark.ml.classification.FMClassifier", self.uid) - self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, - tol=1e-6, solver="adamW") - kwargs = self._input_kwargs - self.setParams(**kwargs) - - @keyword_only - @since("3.0.0") - def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", - probabilityCol="probability", rawPredictionCol="rawPrediction", - factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, - tol=1e-6, solver="adamW", thresholds=None, seed=None): - """ - setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ - probabilityCol="probability", rawPredictionCol="rawPrediction", \ - factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, \ - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, \ - tol=1e-6, solver="adamW", thresholds=None, seed=None) - Sets Params for FMClassifier. - """ - kwargs = self._input_kwargs - return self._set(**kwargs) - - def _create_model(self, java_model): - return FMClassificationModel(java_model) - - @since("3.0.0") - def setFactorSize(self, value): - """ - Sets the value of :py:attr:`factorSize`. - """ - return self._set(factorSize=value) - - @since("3.0.0") - def setFitLinear(self, value): - """ - Sets the value of :py:attr:`fitLinear`. - """ - return self._set(fitLinear=value) - - @since("3.0.0") - def setMiniBatchFraction(self, value): - """ - Sets the value of :py:attr:`miniBatchFraction`. - """ - return self._set(miniBatchFraction=value) - - @since("3.0.0") - def setInitStd(self, value): - """ - Sets the value of :py:attr:`initStd`. - """ - return self._set(initStd=value) - - -class FMClassificationModel(JavaProbabilisticClassificationModel, JavaMLWritable, JavaMLReadable): - """ - Model fitted by :class:`FMClassifier`. - - .. versionadded:: 3.0.0 - """ - - @property - @since("3.0.0") - def intercept(self): - """ - Model intercept. - """ - return self._call_java("intercept") - - @property - @since("3.0.0") - def linear(self): - """ - Model linear term. - """ - return self._call_java("linear") - - @property - @since("3.0.0") - def factors(self): - """ - Model factor term. - """ - return self._call_java("factors") - - if __name__ == "__main__": import doctest import pyspark.ml.classification diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index c9f174ec24e57..0ea906d2688a1 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -37,8 +37,7 @@ 'IsotonicRegression', 'IsotonicRegressionModel', 'LinearRegression', 'LinearRegressionModel', 'LinearRegressionSummary', 'LinearRegressionTrainingSummary', - 'RandomForestRegressor', 'RandomForestRegressionModel', - 'FMRegressor', 'FMRegressionModel'] + 'RandomForestRegressor', 'RandomForestRegressionModel'] class _LinearRegressionParams(_JavaPredictorParams, HasRegParam, HasElasticNetParam, HasMaxIter, @@ -2297,167 +2296,6 @@ def __repr__(self): return self._call_java("toString") -@inherit_doc -class FMRegressor(JavaPredictor, HasMaxIter, HasStepSize, HasTol, HasSolver, HasSeed, - HasFitIntercept, HasRegParam, JavaMLWritable, JavaMLReadable): - """ - Factorization Machines learning algorithm for regression. - - solver Supports: - - * gd (normal mini-batch gradient descent) - * adamW (default) - - >>> from pyspark.ml.linalg import Vectors - >>> from pyspark.ml.regression import FMRegressor - >>> df = spark.createDataFrame([ - ... (2.0, Vectors.dense(2.0)), - ... (1.0, Vectors.dense(1.0)), - ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) - >>> - >>> fm = FMRegressor(factorSize=2, maxIter=1000) - >>> model = fm.fit(df) - >>> test0 = spark.createDataFrame([ - ... (Vectors.dense(-2.0),), - ... (Vectors.dense(0.5),), - ... (Vectors.dense(1.0),), - ... (Vectors.dense(4.0),)], ["features"]) - >>> model.transform(test0).show(10, False) - +--------+-------------------+ - |features|prediction | - +--------+-------------------+ - |[-2.0] |-1.9989237712341565| - |[0.5] |0.4956682219523814 | - |[1.0] |0.994586620589689 | - |[4.0] |3.9880970124135344 | - +--------+-------------------+ - >>> model.intercept - -0.0032501766849261557 - >>> model.linear - DenseVector([0.9978]) - >>> model.factors - DenseMatrix(1, 2, [0.0146, -0.0078], 1) - - .. versionadded:: 3.0.0 - """ - - factorSize = Param(Params._dummy(), "factorSize", "Dimensionality of the factor vectors, " + - "which are used to get pairwise interactions between variables", - typeConverter=TypeConverters.toInt) - - fitLinear = Param(Params._dummy(), "fitLinear", "whether to fit linear term (aka 1-way term)", - typeConverter=TypeConverters.toBoolean) - - miniBatchFraction = Param(Params._dummy(), "miniBatchFraction", "fraction of the input data " + - "set that should be used for one iteration of gradient descent", - typeConverter=TypeConverters.toFloat) - - initStd = Param(Params._dummy(), "initStd", "standard deviation of initial coefficients", - typeConverter=TypeConverters.toFloat) - - solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + - "options: gd, adamW. (Default adamW)", typeConverter=TypeConverters.toString) - - @keyword_only - def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", - factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, - tol=1e-6, solver="adamW", seed=None): - """ - __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ - factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, \ - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, \ - tol=1e-6, solver="adamW", seed=None) - """ - super(FMRegressor, self).__init__() - self._java_obj = self._new_java_obj( - "org.apache.spark.ml.regression.FMRegressor", self.uid) - self._setDefault(factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, - tol=1e-6, solver="adamW") - kwargs = self._input_kwargs - self.setParams(**kwargs) - - @keyword_only - @since("3.0.0") - def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", - factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, - tol=1e-6, solver="adamW", seed=None): - """ - setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ - factorSize=8, fitIntercept=True, fitLinear=True, regParam=0.0, \ - miniBatchFraction=1.0, initStd=0.01, maxIter=100, stepSize=1.0, \ - tol=1e-6, solver="adamW", seed=None) - Sets Params for FMRegressor. - """ - kwargs = self._input_kwargs - return self._set(**kwargs) - - def _create_model(self, java_model): - return FMRegressionModel(java_model) - - @since("3.0.0") - def setFactorSize(self, value): - """ - Sets the value of :py:attr:`factorSize`. - """ - return self._set(factorSize=value) - - @since("3.0.0") - def setFitLinear(self, value): - """ - Sets the value of :py:attr:`fitLinear`. - """ - return self._set(fitLinear=value) - - @since("3.0.0") - def setMiniBatchFraction(self, value): - """ - Sets the value of :py:attr:`miniBatchFraction`. - """ - return self._set(miniBatchFraction=value) - - @since("3.0.0") - def setInitStd(self, value): - """ - Sets the value of :py:attr:`initStd`. - """ - return self._set(initStd=value) - - -class FMRegressionModel(JavaPredictionModel, JavaMLWritable, JavaMLReadable): - """ - Model fitted by :class:`FMRegressor`. - - .. versionadded:: 3.0.0 - """ - - @property - @since("3.0.0") - def intercept(self): - """ - Model intercept. - """ - return self._call_java("intercept") - - @property - @since("3.0.0") - def linear(self): - """ - Model linear term. - """ - return self._call_java("linear") - - @property - @since("3.0.0") - def factors(self): - """ - Model factor term. - """ - return self._call_java("factors") - - if __name__ == "__main__": import doctest import pyspark.ml.regression From 57ca95246cca69e4f6a847fff08005c921430ace Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Wed, 25 Dec 2019 08:57:34 +0900 Subject: [PATCH 0944/1387] [SPARK-29505][SQL] Make DESC EXTENDED case insensitive ### What changes were proposed in this pull request? While querying using **desc** , if column name is not entered exactly as per the column name given during the table creation, the colstats are wrong. fetching of col stats has been made case insensitive. ### Why are the changes needed? functions like **analyze**, etc support case insensitive retrieval of column data. ### Does this PR introduce any user-facing change? NO ### How was this patch tested? | TaskMemoryManager |----+ | | - * +-------------+ | +-------------------+ | | +------------------+ | - * | MemConsumer |----+ | | | StorageMemPool | | - * +-------------+ +-------------------+ | | +------------------+ | - * | TaskMemoryManager |----+ | | - * +-------------------+ | | +------------------+ | - * +---->| |OnHeapExecMemPool | | - * * | | +------------------+ | - * * | | | - * +-------------+ * | | +------------------+ | - * | MemConsumer |----+ | | |OffHeapExecMemPool| | - * +-------------+ | +-------------------+ | | +------------------+ | - * +--->| TaskMemoryManager |----+ | | - * +-------------------+ +------------------------+ + * +---------------------------+ + * +-------------+ | MemoryManager | + * | MemConsumer |----+ | | + * +-------------+ | +-------------------+ | +---------------------+ | + * +--->| TaskMemoryManager |----+ | |OnHeapStorageMemPool | | + * +-------------+ | +-------------------+ | | +---------------------+ | + * | MemConsumer |----+ | | | + * +-------------+ +-------------------+ | | +---------------------+ | + * | TaskMemoryManager |----+ | |OffHeapStorageMemPool| | + * +-------------------+ | | +---------------------+ | + * +---->| | + * * | | +---------------------+ | + * * | | |OnHeapExecMemPool | | + * +-------------+ * | | +---------------------+ | + * | MemConsumer |----+ | | | + * +-------------+ | +-------------------+ | | +---------------------+ | + * +--->| TaskMemoryManager |----+ | |OffHeapExecMemPool | | + * +-------------------+ | +---------------------+ | + * | | + * +---------------------------+ * }}} * * From 65b603d597f683bd3180b5241b1b24663722d950 Mon Sep 17 00:00:00 2001 From: Neal Song Date: Sat, 11 Jan 2020 20:08:46 -0800 Subject: [PATCH 1079/1387] [SPARK-30458][WEBUI] Fix Wrong Executor Computing Time in Time Line of Stage Page ### What changes were proposed in this pull request? The Executor Computing Time in Time Line of Stage Page will be right ### Why are the changes needed? The Executor Computing Time in Time Line of Stage Page is Wrong. It includes the Scheduler Delay Time, while the Proportion excludes the Scheduler Delay Snipaste_2020-01-08_19-04-33 The right executor computing time is 1ms, but the number in UI is 3ms(include 2ms scheduler delay); the proportion is right. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual Closes #27135 from sddyljsx/SPARK-30458. Lead-authored-by: Neal Song Co-authored-by: neal_song Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index fce05e8a42fda..4dc5349cb8ad3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -288,10 +288,10 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val executorOverhead = serializationTime + deserializationTime val executorRunTime = if (taskInfo.duration.isDefined) { - totalExecutionTime - executorOverhead - gettingResultTime + math.max(totalExecutionTime - executorOverhead - gettingResultTime - schedulerDelay, 0) } else { metricsOpt.map(_.executorRunTime).getOrElse( - totalExecutionTime - executorOverhead - gettingResultTime) + math.max(totalExecutionTime - executorOverhead - gettingResultTime - schedulerDelay, 0)) } val executorComputingTime = executorRunTime - shuffleReadTime - shuffleWriteTime val executorComputingTimeProportion = From 823e3d309c51528e69893f71c4be0f5bc8552d99 Mon Sep 17 00:00:00 2001 From: ulysses Date: Sun, 12 Jan 2020 15:03:57 +0800 Subject: [PATCH 1080/1387] [SPARK-30353][SQL] Add IsNotNull check in SimplifyBinaryComparison optimization ### What changes were proposed in this pull request? Now Spark can propagate constraint during sql optimization when `spark.sql.constraintPropagation.enabled` is true, then `where c = 1` will convert to `where c = 1 and c is not null`. We also can use constraint in `SimplifyBinaryComparison`. `SimplifyBinaryComparison` will simplify expression which is not nullable and semanticEquals. And we also can simplify if one expression is infered `IsNotNull`. ### Why are the changes needed? Simplify SQL. ``` create table test (c1 string); explain extended select c1 from test where c1 = c1 limit 10; -- before GlobalLimit 10 +- LocalLimit 10 +- Filter (isnotnull(c1#20) AND (c1#20 = c1#20)) +- Relation[c1#20] -- after GlobalLimit 10 +- LocalLimit 10 +- Filter (isnotnull(c1#20) +- Relation[c1#20] explain extended select c1 from test where c1 > c1 limit 10; -- before GlobalLimit 10 +- LocalLimit 10 +- Filter (isnotnull(c1#20) && (c1#20 > c1#20)) +- Relation[c1#20] -- after LocalRelation , [c1#20] ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Add UT. Closes #27008 from ulysses-you/SPARK-30353. Authored-by: ulysses Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/expressions.scala | 37 ++++++++--- .../BinaryComparisonSimplificationSuite.scala | 65 +++++++++++++++++-- 2 files changed, 88 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 269c8500f95ce..41a7dfb092aba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -406,19 +406,40 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { * 2) Replace '=', '<=', and '>=' with 'true' literal if both operands are non-nullable. * 3) Replace '<' and '>' with 'false' literal if both operands are non-nullable. */ -object SimplifyBinaryComparison extends Rule[LogicalPlan] with PredicateHelper { +object SimplifyBinaryComparison + extends Rule[LogicalPlan] with PredicateHelper with ConstraintHelper { + + private def canSimplifyComparison( + plan: LogicalPlan, left: Expression, right: Expression): Boolean = { + if (left.semanticEquals(right)) { + if (!left.nullable && !right.nullable) { + true + } else { + // We do more checks for non-nullable cases + plan match { + case Filter(fc, _) => + splitConjunctivePredicates(fc).exists { condition => + condition.semanticEquals(IsNotNull(left)) + } + case _ => false + } + } + } else { + false + } + } + def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case q: LogicalPlan => q transformExpressionsUp { + case l: LogicalPlan => l transformExpressionsUp { // True with equality case a EqualNullSafe b if a.semanticEquals(b) => TrueLiteral - case a EqualTo b if !a.nullable && !b.nullable && a.semanticEquals(b) => TrueLiteral - case a GreaterThanOrEqual b if !a.nullable && !b.nullable && a.semanticEquals(b) => - TrueLiteral - case a LessThanOrEqual b if !a.nullable && !b.nullable && a.semanticEquals(b) => TrueLiteral + case a EqualTo b if canSimplifyComparison(l, a, b) => TrueLiteral + case a GreaterThanOrEqual b if canSimplifyComparison(l, a, b) => TrueLiteral + case a LessThanOrEqual b if canSimplifyComparison(l, a, b) => TrueLiteral // False with inequality - case a GreaterThan b if !a.nullable && !b.nullable && a.semanticEquals(b) => FalseLiteral - case a LessThan b if !a.nullable && !b.nullable && a.semanticEquals(b) => FalseLiteral + case a GreaterThan b if canSimplifyComparison(l, a, b) => FalseLiteral + case a LessThan b if canSimplifyComparison(l, a, b) => FalseLiteral } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala index 5794691a365a9..9c71cc8e0d291 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLite import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper { @@ -33,6 +34,8 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper val batches = Batch("AnalysisNodes", Once, EliminateSubqueryAliases) :: + Batch("Infer Filters", Once, + InferFiltersFromConstraints) :: Batch("Constant Folding", FixedPoint(50), NullPropagation, ConstantFolding, @@ -44,12 +47,15 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper val nullableRelation = LocalRelation('a.int.withNullability(true)) val nonNullableRelation = LocalRelation('a.int.withNullability(false)) - test("Preserve nullable exprs in general") { - for (e <- Seq('a === 'a, 'a <= 'a, 'a >= 'a, 'a < 'a, 'a > 'a)) { - val plan = nullableRelation.where(e).analyze - val actual = Optimize.execute(plan) - val correctAnswer = plan - comparePlans(actual, correctAnswer) + test("Preserve nullable exprs when constraintPropagation is false") { + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { + val a = Symbol("a") + for (e <- Seq(a === a, a <= a, a >= a, a < a, a > a)) { + val plan = nullableRelation.where(e).analyze + val actual = Optimize.execute(plan) + val correctAnswer = plan + comparePlans(actual, correctAnswer) + } } } @@ -122,4 +128,51 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper comparePlans(optimized, correctAnswer) } + + test("Simplify null and nonnull with filter constraints") { + val a = Symbol("a") + Seq(a === a, a <= a, a >= a, a < a, a > a).foreach { condition => + val plan = nonNullableRelation.where(condition).analyze + val actual = Optimize.execute(plan) + val correctAnswer = nonNullableRelation.analyze + comparePlans(actual, correctAnswer) + } + + // infer filter constraints will add IsNotNull + Seq(a === a, a <= a, a >= a).foreach { condition => + val plan = nullableRelation.where(condition).analyze + val actual = Optimize.execute(plan) + val correctAnswer = nullableRelation.where('a.isNotNull).analyze + comparePlans(actual, correctAnswer) + } + + Seq(a < a, a > a).foreach { condition => + val plan = nullableRelation.where(condition).analyze + val actual = Optimize.execute(plan) + val correctAnswer = nullableRelation.analyze + comparePlans(actual, correctAnswer) + } + } + + test("Simplify nullable without constraints propagation") { + withSQLConf(SQLConf.CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { + val a = Symbol("a") + Seq(And(a === a, a.isNotNull), + And(a <= a, a.isNotNull), + And(a >= a, a.isNotNull)).foreach { condition => + val plan = nullableRelation.where(condition).analyze + val actual = Optimize.execute(plan) + val correctAnswer = nullableRelation.where('a.isNotNull).analyze + comparePlans(actual, correctAnswer) + } + + Seq(And(a < a, a.isNotNull), And(a > a, a.isNotNull)) + .foreach { condition => + val plan = nullableRelation.where(condition).analyze + val actual = Optimize.execute(plan) + val correctAnswer = nullableRelation.analyze + comparePlans(actual, correctAnswer) + } + } + } } From 1f50a5875b46885a40668c058a1a28e736776244 Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Sun, 12 Jan 2020 15:18:30 +0800 Subject: [PATCH 1081/1387] [SPARK-27296][SQL] Allows Aggregator to be registered as a UDF ## What changes were proposed in this pull request? Defines a new subclass of UDF: `UserDefinedAggregator`. Also allows `Aggregator` to be registered as a udf. Under the hood, the implementation is based on the internal `TypedImperativeAggregate` class that spark's predefined aggregators make use of. The effect is that custom user defined aggregators are now serialized only on partition boundaries instead of being serialized and deserialized at each input row. The two new modes of using `Aggregator` are as follows: ```scala val agg: Aggregator[IN, BUF, OUT] = // typed aggregator val udaf1 = UserDefinedAggregator(agg) val udaf2 = spark.udf.register("agg", agg) ``` ## How was this patch tested? Unit testing has been added that corresponds to the testing suites for `UserDefinedAggregateFunction`. Additionally, unit tests explicitly count the number of aggregator ser/de cycles to ensure that it is governed only by the number of data partitions. To evaluate the performance impact, I did two comparisons. The code and REPL results are recorded on [this gist](https://gist.github.com/erikerlandson/b0e106a4dbaf7f80b4f4f3a21f05f892) To characterize its behavior I benchmarked both a relatively simple aggregator and then an aggregator with a complex structure (a t-digest). ### performance The following compares the new `Aggregator` based aggregation against UDAF. In this scenario, the new aggregation is about 100x faster. The difference in performance impact depends on the complexity of the aggregator. For very simple aggregators (e.g. implementing 'sum', etc), the performance impact is more like 25-30%. ```scala scala> import scala.util.Random._, org.apache.spark.sql.Row, org.apache.spark.tdigest._ import scala.util.Random._ import org.apache.spark.sql.Row import org.apache.spark.tdigest._ scala> val data = sc.parallelize(Vector.fill(50000){(nextInt(2), nextGaussian, nextGaussian.toFloat)}, 5).toDF("cat", "x1", "x2") data: org.apache.spark.sql.DataFrame = [cat: int, x1: double ... 1 more field] scala> val udaf = TDigestUDAF(0.5, 0) udaf: org.apache.spark.tdigest.TDigestUDAF = TDigestUDAF(0.5,0) scala> val bs = Benchmark.sample(10) { data.agg(udaf($"x1"), udaf($"x2")).first } bs: Array[(Double, org.apache.spark.sql.Row)] = Array((16.523,[TDigestSQL(TDigest(0.5,0,130,TDigestMap(-4.9171836327285225 -> (1.0, 1.0), -3.9615949140987685 -> (1.0, 2.0), -3.792874086327091 -> (0.7500781537109753, 2.7500781537109753), -3.720534874164185 -> (1.796754196108008, 4.546832349818983), -3.702105588052377 -> (0.4531676501810167, 5.0), -3.665883591332569 -> (2.3434687534153142, 7.343468753415314), -3.649982231368131 -> (0.6565312465846858, 8.0), -3.5914188829817744 -> (4.0, 12.0), -3.530472305581248 -> (4.0, 16.0), -3.4060489584449467 -> (2.9372251939818383, 18.93722519398184), -3.3000694035428486 -> (8.12412890252889, 27.061354096510726), -3.2250016655261877 -> (8.30564453211017, 35.3669986286209), -3.180537395623448 -> (6.001782561137285, 41.3687811... scala> bs.map(_._1) res0: Array[Double] = Array(16.523, 17.138, 17.863, 17.801, 17.769, 17.786, 17.744, 17.8, 17.939, 17.854) scala> val agg = TDigestAggregator(0.5, 0) agg: org.apache.spark.tdigest.TDigestAggregator = TDigestAggregator(0.5,0) scala> val udaa = spark.udf.register("tdigest", agg) udaa: org.apache.spark.sql.expressions.UserDefinedAggregator[Double,org.apache.spark.tdigest.TDigestSQL,org.apache.spark.tdigest.TDigestSQL] = UserDefinedAggregator(TDigestAggregator(0.5,0),None,true,true) scala> val bs = Benchmark.sample(10) { data.agg(udaa($"x1"), udaa($"x2")).first } bs: Array[(Double, org.apache.spark.sql.Row)] = Array((0.313,[TDigestSQL(TDigest(0.5,0,130,TDigestMap(-4.9171836327285225 -> (1.0, 1.0), -3.9615949140987685 -> (1.0, 2.0), -3.792874086327091 -> (0.7500781537109753, 2.7500781537109753), -3.720534874164185 -> (1.796754196108008, 4.546832349818983), -3.702105588052377 -> (0.4531676501810167, 5.0), -3.665883591332569 -> (2.3434687534153142, 7.343468753415314), -3.649982231368131 -> (0.6565312465846858, 8.0), -3.5914188829817744 -> (4.0, 12.0), -3.530472305581248 -> (4.0, 16.0), -3.4060489584449467 -> (2.9372251939818383, 18.93722519398184), -3.3000694035428486 -> (8.12412890252889, 27.061354096510726), -3.2250016655261877 -> (8.30564453211017, 35.3669986286209), -3.180537395623448 -> (6.001782561137285, 41.36878118... scala> bs.map(_._1) res1: Array[Double] = Array(0.313, 0.193, 0.175, 0.185, 0.174, 0.176, 0.16, 0.186, 0.171, 0.179) scala> ``` Closes #25024 from erikerlandson/spark-27296. Authored-by: Erik Erlandson Signed-off-by: Wenchen Fan --- .../apache/spark/sql/UDFRegistration.scala | 18 +- .../spark/sql/execution/aggregate/udaf.scala | 72 ++- .../sql/expressions/UserDefinedFunction.scala | 48 +- .../org/apache/spark/sql/functions.scala | 64 ++- .../sql/DataFrameWindowFunctionsSuite.scala | 38 +- .../sql/hive/execution/UDAQuerySuite.scala | 417 ++++++++++++++++++ 6 files changed, 643 insertions(+), 14 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index bb05c76cfee6d..a4ff09596ad8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -28,10 +28,11 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUDF} -import org.apache.spark.sql.execution.aggregate.ScalaUDAF +import org.apache.spark.sql.execution.aggregate.{ScalaAggregator, ScalaUDAF} import org.apache.spark.sql.execution.python.UserDefinedPythonFunction -import org.apache.spark.sql.expressions.{SparkUserDefinedFunction, UserDefinedAggregateFunction, UserDefinedFunction} +import org.apache.spark.sql.expressions.{Aggregator, SparkUserDefinedFunction, UserDefinedAggregateFunction, UserDefinedAggregator, UserDefinedFunction} import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils @@ -101,9 +102,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 2.2.0 */ def register(name: String, udf: UserDefinedFunction): UserDefinedFunction = { - def builder(children: Seq[Expression]) = udf.apply(children.map(Column.apply) : _*).expr - functionRegistry.createOrReplaceTempFunction(name, builder) - udf + udf match { + case udaf: UserDefinedAggregator[_, _, _] => + def builder(children: Seq[Expression]) = udaf.scalaAggregator(children) + functionRegistry.createOrReplaceTempFunction(name, builder) + udf + case _ => + def builder(children: Seq[Expression]) = udf.apply(children.map(Column.apply) : _*).expr + functionRegistry.createOrReplaceTempFunction(name, builder) + udf + } } // scalastyle:off line.size.limit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index 100486fa9850f..dfae5c07e0373 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -17,13 +17,17 @@ package org.apache.spark.sql.execution.aggregate +import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.internal.Logging -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Column, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, _} -import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection -import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} +import org.apache.spark.sql.catalyst.expressions.aggregate.{ImperativeAggregate, TypedImperativeAggregate} +import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateMutableProjection, GenerateSafeProjection} +import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction} import org.apache.spark.sql.types._ /** @@ -450,3 +454,63 @@ case class ScalaUDAF( override def nodeName: String = udaf.getClass.getSimpleName } + +case class ScalaAggregator[IN, BUF, OUT]( + children: Seq[Expression], + agg: Aggregator[IN, BUF, OUT], + inputEncoderNR: ExpressionEncoder[IN], + nullable: Boolean = true, + isDeterministic: Boolean = true, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[BUF] + with NonSQLExpression + with UserDefinedExpression + with ImplicitCastInputTypes + with Logging { + + private[this] lazy val inputEncoder = inputEncoderNR.resolveAndBind() + private[this] lazy val bufferEncoder = + agg.bufferEncoder.asInstanceOf[ExpressionEncoder[BUF]].resolveAndBind() + private[this] lazy val outputEncoder = agg.outputEncoder.asInstanceOf[ExpressionEncoder[OUT]] + + def dataType: DataType = outputEncoder.objSerializer.dataType + + def inputTypes: Seq[DataType] = inputEncoder.schema.map(_.dataType) + + override lazy val deterministic: Boolean = isDeterministic + + def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ScalaAggregator[IN, BUF, OUT] = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ScalaAggregator[IN, BUF, OUT] = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + private[this] lazy val inputProjection = UnsafeProjection.create(children) + + def createAggregationBuffer(): BUF = agg.zero + + def update(buffer: BUF, input: InternalRow): BUF = + agg.reduce(buffer, inputEncoder.fromRow(inputProjection(input))) + + def merge(buffer: BUF, input: BUF): BUF = agg.merge(buffer, input) + + def eval(buffer: BUF): Any = { + val row = outputEncoder.toRow(agg.finish(buffer)) + if (outputEncoder.isSerializedAsStruct) row else row.get(0, dataType) + } + + private[this] lazy val bufferRow = new UnsafeRow(bufferEncoder.namedExpressions.length) + + def serialize(agg: BUF): Array[Byte] = + bufferEncoder.toRow(agg).asInstanceOf[UnsafeRow].getBytes() + + def deserialize(storageFormat: Array[Byte]): BUF = { + bufferRow.pointTo(storageFormat, storageFormat.length) + bufferEncoder.fromRow(bufferRow) + } + + override def toString: String = s"""${nodeName}(${children.mkString(",")})""" + + override def nodeName: String = agg.getClass.getSimpleName +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 0c956ecbf936e..85b2cd379ba24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -17,10 +17,15 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.Stable -import org.apache.spark.sql.Column +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.annotation.{Experimental, Stable} +import org.apache.spark.sql.{Column, Encoder} import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUDF} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} +import org.apache.spark.sql.execution.aggregate.ScalaAggregator import org.apache.spark.sql.types.{AnyDataType, DataType} /** @@ -136,3 +141,42 @@ private[sql] case class SparkUserDefinedFunction( } } } + +private[sql] case class UserDefinedAggregator[IN, BUF, OUT]( + aggregator: Aggregator[IN, BUF, OUT], + inputEncoder: Encoder[IN], + name: Option[String] = None, + nullable: Boolean = true, + deterministic: Boolean = true) extends UserDefinedFunction { + + @scala.annotation.varargs + def apply(exprs: Column*): Column = { + Column(AggregateExpression(scalaAggregator(exprs.map(_.expr)), Complete, isDistinct = false)) + } + + // This is also used by udf.register(...) when it detects a UserDefinedAggregator + def scalaAggregator(exprs: Seq[Expression]): ScalaAggregator[IN, BUF, OUT] = { + val iEncoder = inputEncoder.asInstanceOf[ExpressionEncoder[IN]] + ScalaAggregator(exprs, aggregator, iEncoder, nullable, deterministic) + } + + override def withName(name: String): UserDefinedAggregator[IN, BUF, OUT] = { + copy(name = Option(name)) + } + + override def asNonNullable(): UserDefinedAggregator[IN, BUF, OUT] = { + if (!nullable) { + this + } else { + copy(nullable = false) + } + } + + override def asNondeterministic(): UserDefinedAggregator[IN, BUF, OUT] = { + if (!deterministic) { + this + } else { + copy(deterministic = false) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 59dbe3e4b397f..fde6d3e576e47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -32,12 +32,11 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, HintInfo, ResolvedHint} import org.apache.spark.sql.execution.SparkSqlParser -import org.apache.spark.sql.expressions.{SparkUserDefinedFunction, UserDefinedFunction} +import org.apache.spark.sql.expressions.{Aggregator, SparkUserDefinedFunction, UserDefinedAggregator, UserDefinedFunction} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils - /** * Commonly used functions available for DataFrame operations. Using functions defined here provides * a little bit more compile-time safety to make sure the function exists. @@ -4231,6 +4230,67 @@ object functions { // Scala UDF functions ////////////////////////////////////////////////////////////////////////////////////////////// + /** + * Obtains a `UserDefinedFunction` that wraps the given `Aggregator` + * so that it may be used with untyped Data Frames. + * {{{ + * val agg = // Aggregator[IN, BUF, OUT] + * + * // declare a UDF based on agg + * val aggUDF = udaf(agg) + * val aggData = df.agg(aggUDF($"colname")) + * + * // register agg as a named function + * spark.udf.register("myAggName", udaf(agg)) + * }}} + * + * @tparam IN the aggregator input type + * @tparam BUF the aggregating buffer type + * @tparam OUT the finalized output type + * + * @param agg the typed Aggregator + * + * @return a UserDefinedFunction that can be used as an aggregating expression. + * + * @note The input encoder is inferred from the input type IN. + */ + def udaf[IN: TypeTag, BUF, OUT](agg: Aggregator[IN, BUF, OUT]): UserDefinedFunction = { + udaf(agg, ExpressionEncoder[IN]()) + } + + /** + * Obtains a `UserDefinedFunction` that wraps the given `Aggregator` + * so that it may be used with untyped Data Frames. + * {{{ + * Aggregator agg = // custom Aggregator + * Encoder enc = // input encoder + * + * // declare a UDF based on agg + * UserDefinedFunction aggUDF = udaf(agg, enc) + * DataFrame aggData = df.agg(aggUDF($"colname")) + * + * // register agg as a named function + * spark.udf.register("myAggName", udaf(agg, enc)) + * }}} + * + * @tparam IN the aggregator input type + * @tparam BUF the aggregating buffer type + * @tparam OUT the finalized output type + * + * @param agg the typed Aggregator + * @param inputEncoder a specific input encoder to use + * + * @return a UserDefinedFunction that can be used as an aggregating expression + * + * @note This overloading takes an explicit input encoder, to support UDAF + * declarations in Java. + */ + def udaf[IN, BUF, OUT]( + agg: Aggregator[IN, BUF, OUT], + inputEncoder: Encoder[IN]): UserDefinedFunction = { + UserDefinedAggregator(agg, inputEncoder) + } + /** * Defines a Scala closure of 0 arguments as user-defined function (UDF). * The data types are automatically inferred based on the Scala closure's diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 696b056a682b3..2e37879ea1658 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.Matchers.the import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.optimizer.TransposeWindow import org.apache.spark.sql.execution.exchange.Exchange -import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window} +import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -412,6 +412,42 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSparkSession { Row("b", 2, 4, 8))) } + test("window function with aggregator") { + val agg = udaf(new Aggregator[(Long, Long), Long, Long] { + def zero: Long = 0L + def reduce(b: Long, a: (Long, Long)): Long = b + (a._1 * a._2) + def merge(b1: Long, b2: Long): Long = b1 + b2 + def finish(r: Long): Long = r + def bufferEncoder: Encoder[Long] = Encoders.scalaLong + def outputEncoder: Encoder[Long] = Encoders.scalaLong + }) + + val df = Seq( + ("a", 1, 1), + ("a", 1, 5), + ("a", 2, 10), + ("a", 2, -1), + ("b", 4, 7), + ("b", 3, 8), + ("b", 2, 4)) + .toDF("key", "a", "b") + val window = Window.partitionBy($"key").orderBy($"a").rangeBetween(Long.MinValue, 0L) + checkAnswer( + df.select( + $"key", + $"a", + $"b", + agg($"a", $"b").over(window)), + Seq( + Row("a", 1, 1, 6), + Row("a", 1, 5, 6), + Row("a", 2, 10, 24), + Row("a", 2, -1, 24), + Row("b", 4, 7, 60), + Row("b", 3, 8, 32), + Row("b", 2, 4, 8))) + } + test("null inputs") { val df = Seq(("a", 1), ("a", 1), ("a", 2), ("a", 2), ("b", 4), ("b", 3), ("b", 2)) .toDF("key", "value") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala new file mode 100644 index 0000000000000..e6856a58b0ea9 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/UDAQuerySuite.scala @@ -0,0 +1,417 @@ +/* + * 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.hive.execution + +import java.lang.{Double => jlDouble, Integer => jlInt, Long => jlLong} + +import scala.collection.JavaConverters._ +import scala.util.Random + +import test.org.apache.spark.sql.MyDoubleAvg +import test.org.apache.spark.sql.MyDoubleSum + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.expressions.{Aggregator} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ + +class MyDoubleAvgAggBase extends Aggregator[jlDouble, (Double, Long), jlDouble] { + def zero: (Double, Long) = (0.0, 0L) + def reduce(b: (Double, Long), a: jlDouble): (Double, Long) = { + if (a != null) (b._1 + a, b._2 + 1L) else b + } + def merge(b1: (Double, Long), b2: (Double, Long)): (Double, Long) = + (b1._1 + b2._1, b1._2 + b2._2) + def finish(r: (Double, Long)): jlDouble = + if (r._2 > 0L) 100.0 + (r._1 / r._2.toDouble) else null + def bufferEncoder: Encoder[(Double, Long)] = + Encoders.tuple(Encoders.scalaDouble, Encoders.scalaLong) + def outputEncoder: Encoder[jlDouble] = Encoders.DOUBLE +} + +object MyDoubleAvgAgg extends MyDoubleAvgAggBase +object MyDoubleSumAgg extends MyDoubleAvgAggBase { + override def finish(r: (Double, Long)): jlDouble = if (r._2 > 0L) r._1 else null +} + +object LongProductSumAgg extends Aggregator[(jlLong, jlLong), Long, jlLong] { + def zero: Long = 0L + def reduce(b: Long, a: (jlLong, jlLong)): Long = { + if ((a._1 != null) && (a._2 != null)) b + (a._1 * a._2) else b + } + def merge(b1: Long, b2: Long): Long = b1 + b2 + def finish(r: Long): jlLong = r + def bufferEncoder: Encoder[Long] = Encoders.scalaLong + def outputEncoder: Encoder[jlLong] = Encoders.LONG +} + +@SQLUserDefinedType(udt = classOf[CountSerDeUDT]) +case class CountSerDeSQL(nSer: Int, nDeSer: Int, sum: Int) + +class CountSerDeUDT extends UserDefinedType[CountSerDeSQL] { + def userClass: Class[CountSerDeSQL] = classOf[CountSerDeSQL] + + override def typeName: String = "count-ser-de" + + private[spark] override def asNullable: CountSerDeUDT = this + + def sqlType: DataType = StructType( + StructField("nSer", IntegerType, false) :: + StructField("nDeSer", IntegerType, false) :: + StructField("sum", IntegerType, false) :: + Nil) + + def serialize(sql: CountSerDeSQL): Any = { + val row = new GenericInternalRow(3) + row.setInt(0, 1 + sql.nSer) + row.setInt(1, sql.nDeSer) + row.setInt(2, sql.sum) + row + } + + def deserialize(any: Any): CountSerDeSQL = any match { + case row: InternalRow if (row.numFields == 3) => + CountSerDeSQL(row.getInt(0), 1 + row.getInt(1), row.getInt(2)) + case u => throw new Exception(s"failed to deserialize: $u") + } + + override def equals(obj: Any): Boolean = { + obj match { + case _: CountSerDeUDT => true + case _ => false + } + } + + override def hashCode(): Int = classOf[CountSerDeUDT].getName.hashCode() +} + +case object CountSerDeUDT extends CountSerDeUDT + +object CountSerDeAgg extends Aggregator[Int, CountSerDeSQL, CountSerDeSQL] { + def zero: CountSerDeSQL = CountSerDeSQL(0, 0, 0) + def reduce(b: CountSerDeSQL, a: Int): CountSerDeSQL = b.copy(sum = b.sum + a) + def merge(b1: CountSerDeSQL, b2: CountSerDeSQL): CountSerDeSQL = + CountSerDeSQL(b1.nSer + b2.nSer, b1.nDeSer + b2.nDeSer, b1.sum + b2.sum) + def finish(r: CountSerDeSQL): CountSerDeSQL = r + def bufferEncoder: Encoder[CountSerDeSQL] = ExpressionEncoder[CountSerDeSQL]() + def outputEncoder: Encoder[CountSerDeSQL] = ExpressionEncoder[CountSerDeSQL]() +} + +abstract class UDAQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + import testImplicits._ + + override def beforeAll(): Unit = { + super.beforeAll() + val data1 = Seq[(Integer, Integer)]( + (1, 10), + (null, -60), + (1, 20), + (1, 30), + (2, 0), + (null, -10), + (2, -1), + (2, null), + (2, null), + (null, 100), + (3, null), + (null, null), + (3, null)).toDF("key", "value") + data1.write.saveAsTable("agg1") + + val data2 = Seq[(Integer, Integer, Integer)]( + (1, 10, -10), + (null, -60, 60), + (1, 30, -30), + (1, 30, 30), + (2, 1, 1), + (null, -10, 10), + (2, -1, null), + (2, 1, 1), + (2, null, 1), + (null, 100, -10), + (3, null, 3), + (null, null, null), + (3, null, null)).toDF("key", "value1", "value2") + data2.write.saveAsTable("agg2") + + val data3 = Seq[(Seq[Integer], Integer, Integer)]( + (Seq[Integer](1, 1), 10, -10), + (Seq[Integer](null), -60, 60), + (Seq[Integer](1, 1), 30, -30), + (Seq[Integer](1), 30, 30), + (Seq[Integer](2), 1, 1), + (null, -10, 10), + (Seq[Integer](2, 3), -1, null), + (Seq[Integer](2, 3), 1, 1), + (Seq[Integer](2, 3, 4), null, 1), + (Seq[Integer](null), 100, -10), + (Seq[Integer](3), null, 3), + (null, null, null), + (Seq[Integer](3), null, null)).toDF("key", "value1", "value2") + data3.write.saveAsTable("agg3") + + val data4 = Seq[Boolean](true, false, true).toDF("boolvalues") + data4.write.saveAsTable("agg4") + + val emptyDF = spark.createDataFrame( + sparkContext.emptyRDD[Row], + StructType(StructField("key", StringType) :: StructField("value", IntegerType) :: Nil)) + emptyDF.createOrReplaceTempView("emptyTable") + + // Register UDAs + spark.udf.register("mydoublesum", udaf(MyDoubleSumAgg)) + spark.udf.register("mydoubleavg", udaf(MyDoubleAvgAgg)) + spark.udf.register("longProductSum", udaf(LongProductSumAgg)) + } + + override def afterAll(): Unit = { + try { + spark.sql("DROP TABLE IF EXISTS agg1") + spark.sql("DROP TABLE IF EXISTS agg2") + spark.sql("DROP TABLE IF EXISTS agg3") + spark.sql("DROP TABLE IF EXISTS agg4") + spark.catalog.dropTempView("emptyTable") + } finally { + super.afterAll() + } + } + + test("aggregators") { + checkAnswer( + spark.sql( + """ + |SELECT + | key, + | mydoublesum(value + 1.5 * key), + | mydoubleavg(value), + | avg(value - key), + | mydoublesum(value - 1.5 * key), + | avg(value) + |FROM agg1 + |GROUP BY key + """.stripMargin), + Row(1, 64.5, 120.0, 19.0, 55.5, 20.0) :: + Row(2, 5.0, 99.5, -2.5, -7.0, -0.5) :: + Row(3, null, null, null, null, null) :: + Row(null, null, 110.0, null, null, 10.0) :: Nil) + } + + test("non-deterministic children expressions of aggregator") { + val e = intercept[AnalysisException] { + spark.sql( + """ + |SELECT mydoublesum(value + 1.5 * key + rand()) + |FROM agg1 + |GROUP BY key + """.stripMargin) + }.getMessage + assert(Seq("nondeterministic expression", + "should not appear in the arguments of an aggregate function").forall(e.contains)) + } + + test("interpreted aggregate function") { + checkAnswer( + spark.sql( + """ + |SELECT mydoublesum(value), key + |FROM agg1 + |GROUP BY key + """.stripMargin), + Row(60.0, 1) :: Row(-1.0, 2) :: Row(null, 3) :: Row(30.0, null) :: Nil) + + checkAnswer( + spark.sql( + """ + |SELECT mydoublesum(value) FROM agg1 + """.stripMargin), + Row(89.0) :: Nil) + + checkAnswer( + spark.sql( + """ + |SELECT mydoublesum(null) + """.stripMargin), + Row(null) :: Nil) + } + + test("interpreted and expression-based aggregation functions") { + checkAnswer( + spark.sql( + """ + |SELECT mydoublesum(value), key, avg(value) + |FROM agg1 + |GROUP BY key + """.stripMargin), + Row(60.0, 1, 20.0) :: + Row(-1.0, 2, -0.5) :: + Row(null, 3, null) :: + Row(30.0, null, 10.0) :: Nil) + + checkAnswer( + spark.sql( + """ + |SELECT + | mydoublesum(value + 1.5 * key), + | avg(value - key), + | key, + | mydoublesum(value - 1.5 * key), + | avg(value) + |FROM agg1 + |GROUP BY key + """.stripMargin), + Row(64.5, 19.0, 1, 55.5, 20.0) :: + Row(5.0, -2.5, 2, -7.0, -0.5) :: + Row(null, null, 3, null, null) :: + Row(null, null, null, null, 10.0) :: Nil) + } + + test("single distinct column set") { + checkAnswer( + spark.sql( + """ + |SELECT + | mydoubleavg(distinct value1), + | avg(value1), + | avg(value2), + | key, + | mydoubleavg(value1 - 1), + | mydoubleavg(distinct value1) * 0.1, + | avg(value1 + value2) + |FROM agg2 + |GROUP BY key + """.stripMargin), + Row(120.0, 70.0/3.0, -10.0/3.0, 1, 67.0/3.0 + 100.0, 12.0, 20.0) :: + Row(100.0, 1.0/3.0, 1.0, 2, -2.0/3.0 + 100.0, 10.0, 2.0) :: + Row(null, null, 3.0, 3, null, null, null) :: + Row(110.0, 10.0, 20.0, null, 109.0, 11.0, 30.0) :: Nil) + + checkAnswer( + spark.sql( + """ + |SELECT + | key, + | mydoubleavg(distinct value1), + | mydoublesum(value2), + | mydoublesum(distinct value1), + | mydoubleavg(distinct value1), + | mydoubleavg(value1) + |FROM agg2 + |GROUP BY key + """.stripMargin), + Row(1, 120.0, -10.0, 40.0, 120.0, 70.0/3.0 + 100.0) :: + Row(2, 100.0, 3.0, 0.0, 100.0, 1.0/3.0 + 100.0) :: + Row(3, null, 3.0, null, null, null) :: + Row(null, 110.0, 60.0, 30.0, 110.0, 110.0) :: Nil) + } + + test("multiple distinct multiple columns sets") { + checkAnswer( + spark.sql( + """ + |SELECT + | key, + | count(distinct value1), + | sum(distinct value1), + | count(distinct value2), + | sum(distinct value2), + | count(distinct value1, value2), + | longProductSum(distinct value1, value2), + | count(value1), + | sum(value1), + | count(value2), + | sum(value2), + | longProductSum(value1, value2), + | count(*), + | count(1) + |FROM agg2 + |GROUP BY key + """.stripMargin), + Row(null, 3, 30, 3, 60, 3, -4700, 3, 30, 3, 60, -4700, 4, 4) :: + Row(1, 2, 40, 3, -10, 3, -100, 3, 70, 3, -10, -100, 3, 3) :: + Row(2, 2, 0, 1, 1, 1, 1, 3, 1, 3, 3, 2, 4, 4) :: + Row(3, 0, null, 1, 3, 0, 0, 0, null, 1, 3, 0, 2, 2) :: Nil) + } + + test("verify aggregator ser/de behavior") { + val data = sparkContext.parallelize((1 to 100).toSeq, 3).toDF("value1") + val agg = udaf(CountSerDeAgg) + checkAnswer( + data.agg(agg($"value1")), + Row(CountSerDeSQL(4, 4, 5050)) :: Nil) + } + + test("verify type casting failure") { + assertThrows[org.apache.spark.sql.AnalysisException] { + spark.sql( + """ + |SELECT mydoublesum(boolvalues) FROM agg4 + """.stripMargin) + } + } +} + +class HashUDAQuerySuite extends UDAQuerySuite + +class HashUDAQueryWithControlledFallbackSuite extends UDAQuerySuite { + + override protected def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = { + super.checkAnswer(actual, expectedAnswer) + Seq("true", "false").foreach { enableTwoLevelMaps => + withSQLConf("spark.sql.codegen.aggregate.map.twolevel.enabled" -> + enableTwoLevelMaps) { + (1 to 3).foreach { fallbackStartsAt => + withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" -> + s"${(fallbackStartsAt - 1).toString}, ${fallbackStartsAt.toString}") { + QueryTest.getErrorMessageInCheckAnswer(actual, expectedAnswer) match { + case Some(errorMessage) => + val newErrorMessage = + s""" + |The following aggregation query failed when using HashAggregate with + |controlled fallback (it falls back to bytes to bytes map once it has processed + |${fallbackStartsAt - 1} input rows and to sort-based aggregation once it has + |processed $fallbackStartsAt input rows). The query is ${actual.queryExecution} + | + |$errorMessage + """.stripMargin + + fail(newErrorMessage) + case None => // Success + } + } + } + } + } + } + + // Override it to make sure we call the actually overridden checkAnswer. + override protected def checkAnswer(df: => DataFrame, expectedAnswer: Row): Unit = { + checkAnswer(df, Seq(expectedAnswer)) + } + + // Override it to make sure we call the actually overridden checkAnswer. + override protected def checkAnswer(df: => DataFrame, expectedAnswer: DataFrame): Unit = { + checkAnswer(df, expectedAnswer.collect()) + } +} From f5118f81e395bde0cd8253dbef6a9e6455c3958a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 12 Jan 2020 13:18:19 -0800 Subject: [PATCH 1082/1387] [SPARK-30409][SPARK-29173][SQL][TESTS] Use `NoOp` datasource in SQL benchmarks ### What changes were proposed in this pull request? In the PR, I propose to replace `.collect()`, `.count()` and `.foreach(_ => ())` in SQL benchmarks and use the `NoOp` datasource. I added an implicit class to `SqlBasedBenchmark` with the `.noop()` method. It can be used in benchmark like: `ds.noop()`. The last one is unfolded to `ds.write.format("noop").mode(Overwrite).save()`. ### Why are the changes needed? To avoid additional overhead that `collect()` (and other actions) has. For example, `.collect()` has to convert values according to external types and pull data to the driver. This can hide actual performance regressions or improvements of benchmarked operations. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Re-run all modified benchmarks using Amazon EC2. | Item | Description | | ---- | ----| | Region | us-west-2 (Oregon) | | Instance | r3.xlarge (spot instance) | | AMI | ami-06f2f779464715dc5 (ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1) | | Java | OpenJDK8/10 | - Run `TPCDSQueryBenchmark` using instructions from the PR #26049 ``` # `spark-tpcds-datagen` needs this. (JDK8) $ git clone https://github.com/apache/spark.git -b branch-2.4 --depth 1 spark-2.4 $ export SPARK_HOME=$PWD $ ./build/mvn clean package -DskipTests # Generate data. (JDK8) $ git clone gitgithub.com:maropu/spark-tpcds-datagen.git $ cd spark-tpcds-datagen/ $ build/mvn clean package $ mkdir -p /data/tpcds $ ./bin/dsdgen --output-location /data/tpcds/s1 // This need `Spark 2.4` ``` - Other benchmarks ran by the script: ``` #!/usr/bin/env python3 import os from sparktestsupport.shellutils import run_cmd benchmarks = [ ['sql/test', 'org.apache.spark.sql.execution.benchmark.AggregateBenchmark'], ['avro/test', 'org.apache.spark.sql.execution.benchmark.AvroReadBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.BloomFilterBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.DataSourceReadBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.DateTimeBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.ExtractBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.FilterPushdownBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.InExpressionBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.IntervalBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.JoinBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.MakeDateTimeBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.MiscBenchmark'], ['hive/test', 'org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.OrcNestedSchemaPruningBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.OrcV2NestedSchemaPruningBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.ParquetNestedSchemaPruningBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.RangeBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.UDFBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.WideSchemaBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.benchmark.WideTableBenchmark'], ['hive/test', 'org.apache.spark.sql.hive.orc.OrcReadBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.datasources.csv.CSVBenchmark'], ['sql/test', 'org.apache.spark.sql.execution.datasources.json.JsonBenchmark'] ] print('Set SPARK_GENERATE_BENCHMARK_FILES=1') os.environ['SPARK_GENERATE_BENCHMARK_FILES'] = '1' for b in benchmarks: print("Run benchmark: %s" % b[1]) run_cmd(['build/sbt', '%s:runMain %s' % (b[0], b[1])]) ``` Closes #27078 from MaxGekk/noop-in-benchmarks. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../AvroReadBenchmark-jdk11-results.txt | 64 +- .../benchmarks/AvroReadBenchmark-results.txt | 64 +- .../benchmark/AvroReadBenchmark.scala | 16 +- .../AggregateBenchmark-jdk11-results.txt | 98 +- .../benchmarks/AggregateBenchmark-results.txt | 98 +- .../BloomFilterBenchmark-jdk11-results.txt | 12 +- .../BloomFilterBenchmark-results.txt | 12 +- .../benchmarks/CSVBenchmark-jdk11-results.txt | 68 +- sql/core/benchmarks/CSVBenchmark-results.txt | 68 +- .../DataSourceReadBenchmark-jdk11-results.txt | 276 ++-- .../DataSourceReadBenchmark-results.txt | 276 ++-- .../DateTimeBenchmark-jdk11-results.txt | 326 ++--- .../benchmarks/DateTimeBenchmark-results.txt | 326 ++--- .../ExtractBenchmark-jdk11-results.txt | 191 +-- .../benchmarks/ExtractBenchmark-results.txt | 204 +-- .../FilterPushdownBenchmark-jdk11-results.txt | 670 ++++++++++ .../FilterPushdownBenchmark-results.txt | 1188 ++++++++--------- .../InExpressionBenchmark-jdk11-results.txt | 740 ++++++++++ .../InExpressionBenchmark-results.txt | 630 ++++----- .../IntervalBenchmark-jdk11-results.txt | 52 +- .../benchmarks/IntervalBenchmark-results.txt | 52 +- .../JoinBenchmark-jdk11-results.txt | 60 +- sql/core/benchmarks/JoinBenchmark-results.txt | 60 +- .../JsonBenchmark-jdk11-results.txt | 112 ++ sql/core/benchmarks/JsonBenchmark-results.txt | 108 +- .../MakeDateTimeBenchmark-jdk11-results.txt | 28 +- .../MakeDateTimeBenchmark-results.txt | 28 +- .../MiscBenchmark-jdk11-results.txt | 80 +- sql/core/benchmarks/MiscBenchmark-results.txt | 80 +- ...edSchemaPruningBenchmark-jdk11-results.txt | 48 +- ...rcNestedSchemaPruningBenchmark-results.txt | 48 +- ...edSchemaPruningBenchmark-jdk11-results.txt | 48 +- ...V2NestedSchemaPruningBenchmark-results.txt | 48 +- ...edSchemaPruningBenchmark-jdk11-results.txt | 48 +- ...etNestedSchemaPruningBenchmark-results.txt | 48 +- .../RangeBenchmark-jdk11-results.txt | 12 +- .../benchmarks/RangeBenchmark-results.txt | 12 +- .../TPCDSQueryBenchmark-jdk11-results.txt | 810 +++++++++++ .../TPCDSQueryBenchmark-results.txt | 540 ++++---- .../benchmarks/UDFBenchmark-jdk11-results.txt | 42 +- sql/core/benchmarks/UDFBenchmark-results.txt | 42 +- .../WideSchemaBenchmark-jdk11-results.txt | 145 ++ .../WideSchemaBenchmark-results.txt | 192 +-- .../WideTableBenchmark-jdk11-results.txt | 16 +- .../benchmarks/WideTableBenchmark-results.txt | 16 +- .../benchmark/AggregateBenchmark.scala | 20 +- .../benchmark/BloomFilterBenchmark.scala | 4 +- .../benchmark/DataSourceReadBenchmark.scala | 96 +- .../benchmark/DateTimeBenchmark.scala | 11 +- .../benchmark/ExtractBenchmark.scala | 6 +- .../benchmark/FilterPushdownBenchmark.scala | 4 +- .../benchmark/InExpressionBenchmark.scala | 2 +- .../benchmark/IntervalBenchmark.scala | 6 +- .../execution/benchmark/JoinBenchmark.scala | 20 +- .../benchmark/MakeDateTimeBenchmark.scala | 6 +- .../execution/benchmark/MiscBenchmark.scala | 22 +- .../NestedSchemaPruningBenchmark.scala | 7 +- .../execution/benchmark/RangeBenchmark.scala | 6 +- .../benchmark/SqlBasedBenchmark.scala | 9 +- .../benchmark/TPCDSQueryBenchmark.scala | 2 +- .../execution/benchmark/UDFBenchmark.scala | 21 +- .../benchmark/WideSchemaBenchmark.scala | 6 +- .../benchmark/WideTableBenchmark.scala | 2 +- .../datasources/csv/CSVBenchmark.scala | 51 +- .../datasources/json/JsonBenchmark.scala | 67 +- ...shAggregateExecBenchmark-jdk11-results.txt | 45 + ...jectHashAggregateExecBenchmark-results.txt | 36 +- .../OrcReadBenchmark-jdk11-results.txt | 156 +++ .../benchmarks/OrcReadBenchmark-results.txt | 132 +- .../ObjectHashAggregateExecBenchmark.scala | 30 +- .../spark/sql/hive/orc/OrcReadBenchmark.scala | 48 +- 71 files changed, 5787 insertions(+), 3130 deletions(-) create mode 100644 sql/core/benchmarks/FilterPushdownBenchmark-jdk11-results.txt create mode 100644 sql/core/benchmarks/InExpressionBenchmark-jdk11-results.txt create mode 100644 sql/core/benchmarks/JsonBenchmark-jdk11-results.txt create mode 100644 sql/core/benchmarks/TPCDSQueryBenchmark-jdk11-results.txt create mode 100644 sql/core/benchmarks/WideSchemaBenchmark-jdk11-results.txt create mode 100644 sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-jdk11-results.txt create mode 100644 sql/hive/benchmarks/OrcReadBenchmark-jdk11-results.txt diff --git a/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt b/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt index 94137a691e4aa..3c1b5af0d5986 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-jdk11-results.txt @@ -2,121 +2,121 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2995 3081 121 5.3 190.4 1.0X +Sum 2689 2694 7 5.8 170.9 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2865 2881 23 5.5 182.2 1.0X +Sum 2741 2759 26 5.7 174.2 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2919 2936 23 5.4 185.6 1.0X +Sum 2736 2748 17 5.7 173.9 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3148 3262 161 5.0 200.1 1.0X +Sum 3305 3317 17 4.8 210.2 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2651 2721 99 5.9 168.5 1.0X +Sum 2904 2952 68 5.4 184.6 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2782 2854 103 5.7 176.9 1.0X +Sum 3090 3093 4 5.1 196.5 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 4531 4583 73 2.3 432.1 1.0X +Sum of columns 5351 5365 20 2.0 510.3 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 3084 3105 30 5.1 196.1 1.0X -Partition column 3143 3164 30 5.0 199.8 1.0X -Both columns 3272 3339 94 4.8 208.1 0.9X +Data column 3278 3288 14 4.8 208.4 1.0X +Partition column 3149 3193 62 5.0 200.2 1.0X +Both columns 3198 3204 7 4.9 203.4 1.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3249 3318 98 3.2 309.8 1.0X +Sum of string length 3435 3438 5 3.1 327.6 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 5308 5335 38 2.0 506.2 1.0X +Sum of string length 5634 5650 23 1.9 537.3 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 4405 4429 33 2.4 420.1 1.0X +Sum of string length 4725 4752 39 2.2 450.6 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3256 3309 75 3.2 310.5 1.0X +Sum of string length 3550 3566 23 3.0 338.6 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 5230 5290 85 0.2 4987.4 1.0X +Sum of single column 5271 5279 11 0.2 5027.0 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 10206 10329 174 0.1 9733.1 1.0X +Sum of single column 10393 10516 174 0.1 9911.3 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 15333 15365 46 0.1 14622.3 1.0X +Sum of single column 15330 15343 19 0.1 14619.6 1.0X diff --git a/external/avro/benchmarks/AvroReadBenchmark-results.txt b/external/avro/benchmarks/AvroReadBenchmark-results.txt index 7b008a312c320..0ab611a0f9a29 100644 --- a/external/avro/benchmarks/AvroReadBenchmark-results.txt +++ b/external/avro/benchmarks/AvroReadBenchmark-results.txt @@ -2,121 +2,121 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3067 3132 91 5.1 195.0 1.0X +Sum 3049 3071 32 5.2 193.8 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2927 2929 3 5.4 186.1 1.0X +Sum 2982 2992 13 5.3 189.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2928 2990 87 5.4 186.2 1.0X +Sum 2984 2989 7 5.3 189.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3374 3447 104 4.7 214.5 1.0X +Sum 3262 3353 128 4.8 207.4 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 2896 2901 7 5.4 184.1 1.0X +Sum 2716 2723 10 5.8 172.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum 3004 3006 3 5.2 191.0 1.0X +Sum 2868 2870 3 5.5 182.4 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of columns 4814 4830 22 2.2 459.1 1.0X +Sum of columns 4714 4739 35 2.2 449.6 1.0X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column 3361 3362 1 4.7 213.7 1.0X -Partition column 2999 3013 20 5.2 190.7 1.1X -Both columns 3613 3615 2 4.4 229.7 0.9X +Data column 3257 3286 41 4.8 207.1 1.0X +Partition column 3258 3277 27 4.8 207.2 1.0X +Both columns 3399 3405 9 4.6 216.1 1.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3415 3416 1 3.1 325.7 1.0X +Sum of string length 3292 3316 33 3.2 314.0 1.0X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 5535 5536 2 1.9 527.8 1.0X +Sum of string length 5450 5456 9 1.9 519.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 4567 4575 11 2.3 435.6 1.0X +Sum of string length 4410 4435 35 2.4 420.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of string length 3248 3268 29 3.2 309.7 1.0X +Sum of string length 3074 3122 68 3.4 293.2 1.0X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 5486 5497 15 0.2 5232.0 1.0X +Sum of single column 5120 5136 23 0.2 4882.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 10682 10746 90 0.1 10186.8 1.0X +Sum of single column 9952 10002 71 0.1 9490.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Sum of single column 16177 16177 0 0.1 15427.7 1.0X +Sum of single column 14973 14978 7 0.1 14279.8 1.0X diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala index a16126ae24246..dc9606f405191 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala @@ -64,7 +64,7 @@ object AvroReadBenchmark extends SqlBasedBenchmark { prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1")) benchmark.addCase("Sum") { _ => - spark.sql("SELECT sum(id) FROM avroTable").collect() + spark.sql("SELECT sum(id) FROM avroTable").noop() } benchmark.run() @@ -85,7 +85,7 @@ object AvroReadBenchmark extends SqlBasedBenchmark { spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1")) benchmark.addCase("Sum of columns") { _ => - spark.sql("SELECT sum(c1), sum(length(c2)) FROM avroTable").collect() + spark.sql("SELECT sum(c1), sum(length(c2)) FROM avroTable").noop() } benchmark.run() @@ -104,15 +104,15 @@ object AvroReadBenchmark extends SqlBasedBenchmark { prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p")) benchmark.addCase("Data column") { _ => - spark.sql("SELECT sum(id) FROM avroTable").collect() + spark.sql("SELECT sum(id) FROM avroTable").noop() } benchmark.addCase("Partition column") { _ => - spark.sql("SELECT sum(p) FROM avroTable").collect() + spark.sql("SELECT sum(p) FROM avroTable").noop() } benchmark.addCase("Both columns") { _ => - spark.sql("SELECT sum(p), sum(id) FROM avroTable").collect() + spark.sql("SELECT sum(p), sum(id) FROM avroTable").noop() } benchmark.run() @@ -130,7 +130,7 @@ object AvroReadBenchmark extends SqlBasedBenchmark { prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1")) benchmark.addCase("Sum of string length") { _ => - spark.sql("SELECT sum(length(c1)) FROM avroTable").collect() + spark.sql("SELECT sum(length(c1)) FROM avroTable").noop() } benchmark.run() @@ -155,7 +155,7 @@ object AvroReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("Sum of string length") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM avroTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").noop() } benchmark.run() @@ -178,7 +178,7 @@ object AvroReadBenchmark extends SqlBasedBenchmark { prepareTable(dir, spark.sql("SELECT * FROM t1")) benchmark.addCase("Sum of single column") { _ => - spark.sql(s"SELECT sum(c$middle) FROM avroTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM avroTable").noop() } benchmark.run() diff --git a/sql/core/benchmarks/AggregateBenchmark-jdk11-results.txt b/sql/core/benchmarks/AggregateBenchmark-jdk11-results.txt index e71d7ab0f3b6f..baa237d3142d8 100644 --- a/sql/core/benchmarks/AggregateBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-jdk11-results.txt @@ -2,142 +2,142 @@ aggregate without grouping ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz agg w/o group: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -agg w/o group wholestage off 55644 59484 NaN 37.7 26.5 1.0X -agg w/o group wholestage on 896 906 8 2340.7 0.4 62.1X +agg w/o group wholestage off 59499 62969 2375 35.2 28.4 1.0X +agg w/o group wholestage on 890 902 10 2357.2 0.4 66.9X ================================================================================================ stat functions ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz stddev: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -stddev wholestage off 8655 9022 519 12.1 82.5 1.0X -stddev wholestage on 1306 1323 13 80.3 12.5 6.6X +stddev wholestage off 8702 8870 237 12.0 83.0 1.0X +stddev wholestage on 1307 1314 9 80.2 12.5 6.7X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz kurtosis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -kurtosis wholestage off 40795 41041 349 2.6 389.0 1.0X -kurtosis wholestage on 1441 1468 22 72.8 13.7 28.3X +kurtosis wholestage off 42656 42797 198 2.5 406.8 1.0X +kurtosis wholestage on 1440 1466 32 72.8 13.7 29.6X ================================================================================================ aggregate with linear keys ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 10559 10956 562 7.9 125.9 1.0X -codegen = T hashmap = F 6533 6567 34 12.8 77.9 1.6X -codegen = T hashmap = T 1362 1377 22 61.6 16.2 7.8X +codegen = F 10963 11035 101 7.7 130.7 1.0X +codegen = T hashmap = F 6852 7080 207 12.2 81.7 1.6X +codegen = T hashmap = T 1377 1421 43 60.9 16.4 8.0X ================================================================================================ aggregate with randomized keys ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 12631 12634 3 6.6 150.6 1.0X -codegen = T hashmap = F 8434 8478 44 9.9 100.5 1.5X -codegen = T hashmap = T 2484 2598 117 33.8 29.6 5.1X +codegen = F 12240 12296 80 6.9 145.9 1.0X +codegen = T hashmap = F 8318 8376 87 10.1 99.2 1.5X +codegen = T hashmap = T 2551 2617 80 32.9 30.4 4.8X ================================================================================================ aggregate with string key ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w string key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 4173 4204 44 5.0 199.0 1.0X -codegen = T hashmap = F 2664 2721 81 7.9 127.0 1.6X -codegen = T hashmap = T 1178 1219 59 17.8 56.2 3.5X +codegen = F 4442 4504 88 4.7 211.8 1.0X +codegen = T hashmap = F 2685 2778 95 7.8 128.0 1.7X +codegen = T hashmap = T 1181 1203 15 17.8 56.3 3.8X ================================================================================================ aggregate with decimal key ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w decimal key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 3740 3747 9 5.6 178.3 1.0X -codegen = T hashmap = F 2398 2528 184 8.7 114.3 1.6X -codegen = T hashmap = T 638 644 7 32.9 30.4 5.9X +codegen = F 3487 3534 66 6.0 166.3 1.0X +codegen = T hashmap = F 2239 2353 161 9.4 106.8 1.6X +codegen = T hashmap = T 664 693 26 31.6 31.7 5.3X ================================================================================================ aggregate with multiple key types ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 6874 6896 31 3.1 327.8 1.0X -codegen = T hashmap = F 3866 3886 28 5.4 184.3 1.8X -codegen = T hashmap = T 2619 2641 31 8.0 124.9 2.6X +codegen = F 6414 6475 86 3.3 305.9 1.0X +codegen = T hashmap = F 3776 3800 34 5.6 180.1 1.7X +codegen = T hashmap = T 2824 2852 40 7.4 134.7 2.3X ================================================================================================ max function bytecode size of wholestagecodegen ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz max function bytecode size: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 793 806 19 0.8 1209.7 1.0X -codegen = T hugeMethodLimit = 10000 401 456 61 1.6 611.2 2.0X -codegen = T hugeMethodLimit = 1500 694 715 19 0.9 1059.3 1.1X +codegen = F 654 696 36 1.0 998.6 1.0X +codegen = T hugeMethodLimit = 10000 378 405 24 1.7 577.1 1.7X +codegen = T hugeMethodLimit = 1500 383 403 22 1.7 584.3 1.7X ================================================================================================ cube ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cube: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cube wholestage off 3616 3630 19 1.4 689.8 1.0X -cube wholestage on 1819 1866 41 2.9 347.0 2.0X +cube wholestage off 3443 3444 1 1.5 656.7 1.0X +cube wholestage on 1797 1815 20 2.9 342.7 1.9X ================================================================================================ hash and BytesToBytesMap ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz BytesToBytesMap: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UnsafeRowhash 321 321 1 65.4 15.3 1.0X -murmur3 hash 145 145 0 144.5 6.9 2.2X -fast hash 70 71 1 298.9 3.3 4.6X -arrayEqual 198 199 2 105.7 9.5 1.6X -Java HashMap (Long) 132 136 3 158.6 6.3 2.4X -Java HashMap (two ints) 152 156 4 138.3 7.2 2.1X -Java HashMap (UnsafeRow) 819 839 17 25.6 39.1 0.4X -LongToUnsafeRowMap (opt=false) 459 472 25 45.7 21.9 0.7X -LongToUnsafeRowMap (opt=true) 107 108 1 195.5 5.1 3.0X -BytesToBytesMap (off Heap) 1012 1019 10 20.7 48.3 0.3X -BytesToBytesMap (on Heap) 963 974 18 21.8 45.9 0.3X -Aggregate HashMap 41 43 2 515.1 1.9 7.9X +UnsafeRowhash 313 314 2 67.1 14.9 1.0X +murmur3 hash 144 145 2 145.9 6.9 2.2X +fast hash 70 72 5 301.5 3.3 4.5X +arrayEqual 189 190 0 111.0 9.0 1.7X +Java HashMap (Long) 129 132 4 162.4 6.2 2.4X +Java HashMap (two ints) 148 154 7 142.0 7.0 2.1X +Java HashMap (UnsafeRow) 839 840 1 25.0 40.0 0.4X +LongToUnsafeRowMap (opt=false) 462 464 1 45.4 22.0 0.7X +LongToUnsafeRowMap (opt=true) 108 108 1 194.2 5.2 2.9X +BytesToBytesMap (off Heap) 1027 1029 2 20.4 49.0 0.3X +BytesToBytesMap (on Heap) 999 1001 3 21.0 47.6 0.3X +Aggregate HashMap 44 44 0 477.8 2.1 7.1X diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt index ea8f113143a0f..b1c2e9d6ae9fe 100644 --- a/sql/core/benchmarks/AggregateBenchmark-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-results.txt @@ -2,142 +2,142 @@ aggregate without grouping ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz agg w/o group: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -agg w/o group wholestage off 50499 52524 2863 41.5 24.1 1.0X -agg w/o group wholestage on 1163 1205 56 1803.1 0.6 43.4X +agg w/o group wholestage off 49902 52257 NaN 42.0 23.8 1.0X +agg w/o group wholestage on 1162 1171 10 1805.2 0.6 43.0X ================================================================================================ stat functions ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz stddev: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -stddev wholestage off 13393 13403 14 7.8 127.7 1.0X -stddev wholestage on 1291 1315 25 81.2 12.3 10.4X +stddev wholestage off 8203 8243 56 12.8 78.2 1.0X +stddev wholestage on 1287 1303 10 81.5 12.3 6.4X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz kurtosis: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -kurtosis wholestage off 40131 40250 168 2.6 382.7 1.0X -kurtosis wholestage on 1435 1452 20 73.1 13.7 28.0X +kurtosis wholestage off 39557 39919 511 2.7 377.2 1.0X +kurtosis wholestage on 1398 1476 138 75.0 13.3 28.3X ================================================================================================ aggregate with linear keys ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 10008 10008 1 8.4 119.3 1.0X -codegen = T hashmap = F 5803 6158 370 14.5 69.2 1.7X -codegen = T hashmap = T 1332 1348 12 63.0 15.9 7.5X +codegen = F 11236 12182 1337 7.5 133.9 1.0X +codegen = T hashmap = F 7079 7337 250 11.9 84.4 1.6X +codegen = T hashmap = T 1278 1419 186 65.6 15.2 8.8X ================================================================================================ aggregate with randomized keys ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 11853 11988 191 7.1 141.3 1.0X -codegen = T hashmap = F 7469 7531 57 11.2 89.0 1.6X -codegen = T hashmap = T 2412 2436 24 34.8 28.8 4.9X +codegen = F 11629 11650 30 7.2 138.6 1.0X +codegen = T hashmap = F 7552 7747 169 11.1 90.0 1.5X +codegen = T hashmap = T 2414 2662 167 34.7 28.8 4.8X ================================================================================================ aggregate with string key ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w string key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 4880 4908 40 4.3 232.7 1.0X -codegen = T hashmap = F 3483 3505 26 6.0 166.1 1.4X -codegen = T hashmap = T 2362 2425 39 8.9 112.6 2.1X +codegen = F 4790 4904 162 4.4 228.4 1.0X +codegen = T hashmap = F 3439 3504 105 6.1 164.0 1.4X +codegen = T hashmap = T 2327 2365 39 9.0 111.0 2.1X ================================================================================================ aggregate with decimal key ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w decimal key: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 3475 3516 58 6.0 165.7 1.0X -codegen = T hashmap = F 1939 1958 27 10.8 92.5 1.8X -codegen = T hashmap = T 599 609 9 35.0 28.6 5.8X +codegen = F 3260 3418 223 6.4 155.5 1.0X +codegen = T hashmap = F 2316 2325 14 9.1 110.4 1.4X +codegen = T hashmap = T 605 607 2 34.7 28.8 5.4X ================================================================================================ aggregate with multiple key types ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w multiple keys: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 7649 7675 37 2.7 364.7 1.0X -codegen = T hashmap = F 4729 4759 42 4.4 225.5 1.6X -codegen = T hashmap = T 3917 3929 16 5.4 186.8 2.0X +codegen = F 7426 7473 67 2.8 354.1 1.0X +codegen = T hashmap = F 4685 4723 54 4.5 223.4 1.6X +codegen = T hashmap = T 3946 4005 83 5.3 188.2 1.9X ================================================================================================ max function bytecode size of wholestagecodegen ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz max function bytecode size: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -codegen = F 628 661 25 1.0 958.2 1.0X -codegen = T hugeMethodLimit = 10000 366 385 21 1.8 558.9 1.7X -codegen = T hugeMethodLimit = 1500 620 648 24 1.1 946.7 1.0X +codegen = F 628 672 49 1.0 958.4 1.0X +codegen = T hugeMethodLimit = 10000 357 373 12 1.8 545.3 1.8X +codegen = T hugeMethodLimit = 1500 344 356 7 1.9 525.6 1.8X ================================================================================================ cube ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cube: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cube wholestage off 3225 3273 68 1.6 615.1 1.0X -cube wholestage on 1636 1680 56 3.2 312.0 2.0X +cube wholestage off 3167 3266 140 1.7 604.1 1.0X +cube wholestage on 1549 1576 29 3.4 295.4 2.0X ================================================================================================ hash and BytesToBytesMap ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz BytesToBytesMap: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -UnsafeRowhash 331 332 0 63.3 15.8 1.0X -murmur3 hash 148 148 0 141.4 7.1 2.2X -fast hash 75 76 6 280.6 3.6 4.4X -arrayEqual 176 176 0 119.0 8.4 1.9X -Java HashMap (Long) 140 144 6 149.7 6.7 2.4X -Java HashMap (two ints) 153 157 5 137.2 7.3 2.2X -Java HashMap (UnsafeRow) 845 852 6 24.8 40.3 0.4X -LongToUnsafeRowMap (opt=false) 463 482 26 45.3 22.1 0.7X -LongToUnsafeRowMap (opt=true) 118 120 6 178.1 5.6 2.8X -BytesToBytesMap (off Heap) 935 937 2 22.4 44.6 0.4X -BytesToBytesMap (on Heap) 897 901 5 23.4 42.8 0.4X -Aggregate HashMap 57 58 0 369.9 2.7 5.8X +UnsafeRowhash 326 328 2 64.3 15.5 1.0X +murmur3 hash 147 147 0 142.7 7.0 2.2X +fast hash 74 75 1 282.3 3.5 4.4X +arrayEqual 175 175 0 119.8 8.3 1.9X +Java HashMap (Long) 138 140 4 152.1 6.6 2.4X +Java HashMap (two ints) 148 154 7 141.7 7.1 2.2X +Java HashMap (UnsafeRow) 1043 1090 66 20.1 49.8 0.3X +LongToUnsafeRowMap (opt=false) 464 466 2 45.2 22.1 0.7X +LongToUnsafeRowMap (opt=true) 104 106 8 202.3 4.9 3.1X +BytesToBytesMap (off Heap) 1140 1149 12 18.4 54.4 0.3X +BytesToBytesMap (on Heap) 1002 1132 183 20.9 47.8 0.3X +Aggregate HashMap 74 74 0 281.9 3.5 4.4X diff --git a/sql/core/benchmarks/BloomFilterBenchmark-jdk11-results.txt b/sql/core/benchmarks/BloomFilterBenchmark-jdk11-results.txt index 9cd7ad16ac2bb..cb1e96a3df236 100644 --- a/sql/core/benchmarks/BloomFilterBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/BloomFilterBenchmark-jdk11-results.txt @@ -2,23 +2,23 @@ ORC Write ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter 17367 17786 592 5.8 173.7 1.0X -With bloom filter 20556 20596 57 4.9 205.6 0.8X +Without bloom filter 19554 20736 1672 5.1 195.5 1.0X +With bloom filter 22112 22203 129 4.5 221.1 0.9X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter 2204 2226 31 45.4 22.0 1.0X -With bloom filter 1415 1465 71 70.7 14.2 1.6X +Without bloom filter 1866 1879 19 53.6 18.7 1.0X +With bloom filter 1523 1544 29 65.6 15.2 1.2X diff --git a/sql/core/benchmarks/BloomFilterBenchmark-results.txt b/sql/core/benchmarks/BloomFilterBenchmark-results.txt index b4e3e843798e6..ec629129559f4 100644 --- a/sql/core/benchmarks/BloomFilterBenchmark-results.txt +++ b/sql/core/benchmarks/BloomFilterBenchmark-results.txt @@ -2,23 +2,23 @@ ORC Write ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter 19489 19693 289 5.1 194.9 1.0X -With bloom filter 23045 23148 145 4.3 230.5 0.8X +Without bloom filter 19110 19145 50 5.2 191.1 1.0X +With bloom filter 21890 21908 25 4.6 218.9 0.9X ================================================================================================ ORC Read ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read a row from 100M rows: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Without bloom filter 2038 2084 65 49.1 20.4 1.0X -With bloom filter 1465 1475 15 68.3 14.6 1.4X +Without bloom filter 1724 1746 30 58.0 17.2 1.0X +With bloom filter 1364 1376 17 73.3 13.6 1.3X diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index b65b236fd71f2..2d24a273f7575 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -2,58 +2,58 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 56894 57106 184 0.0 1137889.9 1.0X +One quoted string 44297 44515 373 0.0 885948.7 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 220825 222234 2018 0.0 220825.5 1.0X -Select 100 columns 50507 50723 278 0.0 50506.6 4.4X -Select one column 38629 38642 16 0.0 38628.6 5.7X -count() 8549 8597 51 0.1 8549.2 25.8X -Select 100 columns, one bad input field 68309 68474 182 0.0 68309.2 3.2X -Select 100 columns, corrupt record field 74551 74701 136 0.0 74551.5 3.0X +Select 1000 columns 196720 197783 1560 0.0 196719.8 1.0X +Select 100 columns 46691 46861 219 0.0 46691.4 4.2X +Select one column 36811 36922 111 0.0 36811.3 5.3X +count() 8520 8610 106 0.1 8520.5 23.1X +Select 100 columns, one bad input field 67914 67994 136 0.0 67914.0 2.9X +Select 100 columns, corrupt record field 77272 77445 214 0.0 77272.0 2.5X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 27745 28050 276 0.4 2774.5 1.0X -Select 1 column + count() 19989 20315 319 0.5 1998.9 1.4X -count() 6091 6109 25 1.6 609.1 4.6X +Select 10 columns + count() 25965 26054 103 0.4 2596.5 1.0X +Select 1 column + count() 18591 18666 91 0.5 1859.1 1.4X +count() 6102 6119 18 1.6 610.2 4.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2235 2301 59 4.5 223.5 1.0X -to_csv(timestamp) 16033 16205 153 0.6 1603.3 0.1X -write timestamps to files 13556 13685 167 0.7 1355.6 0.2X -Create a dataset of dates 2262 2290 44 4.4 226.2 1.0X -to_csv(date) 11122 11160 33 0.9 1112.2 0.2X -write dates to files 8436 8486 76 1.2 843.6 0.3X +Create a dataset of timestamps 2142 2161 17 4.7 214.2 1.0X +to_csv(timestamp) 14744 14950 182 0.7 1474.4 0.1X +write timestamps to files 12078 12202 175 0.8 1207.8 0.2X +Create a dataset of dates 2275 2291 18 4.4 227.5 0.9X +to_csv(date) 11407 11464 51 0.9 1140.7 0.2X +write dates to files 7638 7702 90 1.3 763.8 0.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2617 2644 26 3.8 261.7 1.0X -read timestamps from files 53245 53381 149 0.2 5324.5 0.0X -infer timestamps from files 103797 104026 257 0.1 10379.7 0.0X -read date text from files 2371 2378 7 4.2 237.1 1.1X -read date from files 41808 41929 177 0.2 4180.8 0.1X -infer date from files 35069 35336 458 0.3 3506.9 0.1X -timestamp strings 3104 3127 21 3.2 310.4 0.8X -parse timestamps from Dataset[String] 61888 62132 342 0.2 6188.8 0.0X -infer timestamps from Dataset[String] 112494 114609 1949 0.1 11249.4 0.0X -date strings 3558 3603 41 2.8 355.8 0.7X -parse dates from Dataset[String] 45871 46000 120 0.2 4587.1 0.1X -from_csv(timestamp) 56975 57035 53 0.2 5697.5 0.0X -from_csv(date) 43711 43795 74 0.2 4371.1 0.1X +read timestamp text from files 2578 2590 10 3.9 257.8 1.0X +read timestamps from files 60103 60694 512 0.2 6010.3 0.0X +infer timestamps from files 107871 108268 351 0.1 10787.1 0.0X +read date text from files 2306 2310 4 4.3 230.6 1.1X +read date from files 47415 47657 367 0.2 4741.5 0.1X +infer date from files 35261 35447 164 0.3 3526.1 0.1X +timestamp strings 3045 3056 11 3.3 304.5 0.8X +parse timestamps from Dataset[String] 62221 63173 849 0.2 6222.1 0.0X +infer timestamps from Dataset[String] 118838 119629 697 0.1 11883.8 0.0X +date strings 3459 3481 19 2.9 345.9 0.7X +parse dates from Dataset[String] 51026 51447 503 0.2 5102.6 0.1X +from_csv(timestamp) 60738 61818 936 0.2 6073.8 0.0X +from_csv(date) 46012 46278 370 0.2 4601.2 0.1X diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index d2037e86a3a71..0777549efc5f5 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -2,58 +2,58 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 62603 62755 133 0.0 1252055.6 1.0X +One quoted string 51602 51659 59 0.0 1032039.4 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 225032 225919 782 0.0 225031.7 1.0X -Select 100 columns 51982 52290 286 0.0 51982.1 4.3X -Select one column 40167 40283 133 0.0 40167.4 5.6X -count() 11435 11593 176 0.1 11435.1 19.7X -Select 100 columns, one bad input field 66864 66968 174 0.0 66864.1 3.4X -Select 100 columns, corrupt record field 79570 80418 1080 0.0 79569.5 2.8X +Select 1000 columns 191926 192879 1615 0.0 191925.6 1.0X +Select 100 columns 46766 46846 69 0.0 46766.1 4.1X +Select one column 35877 35930 83 0.0 35876.8 5.3X +count() 11186 11262 65 0.1 11186.0 17.2X +Select 100 columns, one bad input field 59943 60107 232 0.0 59943.0 3.2X +Select 100 columns, corrupt record field 73062 73406 479 0.0 73062.2 2.6X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 23271 23389 103 0.4 2327.1 1.0X -Select 1 column + count() 18206 19772 NaN 0.5 1820.6 1.3X -count() 8500 8521 18 1.2 850.0 2.7X +Select 10 columns + count() 22389 22447 87 0.4 2238.9 1.0X +Select 1 column + count() 14844 14890 43 0.7 1484.4 1.5X +count() 5519 5538 18 1.8 551.9 4.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2025 2068 66 4.9 202.5 1.0X -to_csv(timestamp) 22192 22983 879 0.5 2219.2 0.1X -write timestamps to files 15949 16030 72 0.6 1594.9 0.1X -Create a dataset of dates 2200 2234 32 4.5 220.0 0.9X -to_csv(date) 18268 18341 73 0.5 1826.8 0.1X -write dates to files 10495 10722 214 1.0 1049.5 0.2X +Create a dataset of timestamps 1949 1977 25 5.1 194.9 1.0X +to_csv(timestamp) 14944 15702 714 0.7 1494.4 0.1X +write timestamps to files 12983 12998 14 0.8 1298.3 0.2X +Create a dataset of dates 2156 2164 7 4.6 215.6 0.9X +to_csv(date) 9675 9709 41 1.0 967.5 0.2X +write dates to files 7880 7897 15 1.3 788.0 0.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 6491 6503 18 1.5 649.1 1.0X -read timestamps from files 56069 56795 874 0.2 5606.9 0.1X -infer timestamps from files 113383 114203 825 0.1 11338.3 0.1X -read date text from files 6411 6419 10 1.6 641.1 1.0X -read date from files 46245 46371 138 0.2 4624.5 0.1X -infer date from files 43623 43906 291 0.2 4362.3 0.1X -timestamp strings 4951 4959 7 2.0 495.1 1.3X -parse timestamps from Dataset[String] 65786 66309 663 0.2 6578.6 0.1X -infer timestamps from Dataset[String] 130891 133861 1928 0.1 13089.1 0.0X -date strings 3814 3895 84 2.6 381.4 1.7X -parse dates from Dataset[String] 52259 52960 614 0.2 5225.9 0.1X -from_csv(timestamp) 63013 63306 291 0.2 6301.3 0.1X -from_csv(date) 49840 52352 NaN 0.2 4984.0 0.1X +read timestamp text from files 2235 2245 10 4.5 223.5 1.0X +read timestamps from files 54490 54690 283 0.2 5449.0 0.0X +infer timestamps from files 104501 104737 236 0.1 10450.1 0.0X +read date text from files 2035 2040 6 4.9 203.5 1.1X +read date from files 39650 39707 52 0.3 3965.0 0.1X +infer date from files 29235 29363 164 0.3 2923.5 0.1X +timestamp strings 3412 3426 18 2.9 341.2 0.7X +parse timestamps from Dataset[String] 66864 67804 981 0.1 6686.4 0.0X +infer timestamps from Dataset[String] 118780 119284 837 0.1 11878.0 0.0X +date strings 3730 3734 4 2.7 373.0 0.6X +parse dates from Dataset[String] 48728 49071 309 0.2 4872.8 0.0X +from_csv(timestamp) 62294 62493 260 0.2 6229.4 0.0X +from_csv(date) 44581 44665 117 0.2 4458.1 0.1X diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-jdk11-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-jdk11-results.txt index 55cb301dba174..595f69c8e1407 100644 --- a/sql/core/benchmarks/DataSourceReadBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/DataSourceReadBenchmark-jdk11-results.txt @@ -2,251 +2,251 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 27115 27169 76 0.6 1723.9 1.0X -SQL Json 9061 9124 89 1.7 576.1 3.0X -SQL Parquet Vectorized 196 232 39 80.4 12.4 138.5X -SQL Parquet MR 2187 2216 40 7.2 139.1 12.4X -SQL ORC Vectorized 335 344 5 46.9 21.3 80.9X -SQL ORC MR 1757 1786 42 9.0 111.7 15.4X +SQL CSV 28338 28589 356 0.6 1801.7 1.0X +SQL Json 9273 9332 83 1.7 589.6 3.1X +SQL Parquet Vectorized 186 217 22 84.3 11.9 152.0X +SQL Parquet MR 1951 1972 29 8.1 124.1 14.5X +SQL ORC Vectorized 256 277 22 61.4 16.3 110.6X +SQL ORC MR 1627 1717 127 9.7 103.4 17.4X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 201 205 5 78.3 12.8 1.0X -ParquetReader Vectorized -> Row 91 92 1 173.2 5.8 2.2X +ParquetReader Vectorized 208 223 13 75.8 13.2 1.0X +ParquetReader Vectorized -> Row 96 97 1 164.1 6.1 2.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 27969 27972 4 0.6 1778.2 1.0X -SQL Json 10328 10389 87 1.5 656.6 2.7X -SQL Parquet Vectorized 217 237 24 72.5 13.8 128.8X -SQL Parquet MR 2494 2567 103 6.3 158.6 11.2X -SQL ORC Vectorized 310 321 10 50.8 19.7 90.3X -SQL ORC MR 1901 1907 9 8.3 120.9 14.7X +SQL CSV 28493 28516 33 0.6 1811.5 1.0X +SQL Json 10257 10291 47 1.5 652.1 2.8X +SQL Parquet Vectorized 215 233 14 73.2 13.7 132.5X +SQL Parquet MR 2384 2388 7 6.6 151.5 12.0X +SQL ORC Vectorized 298 307 7 52.8 18.9 95.6X +SQL ORC MR 1798 1814 22 8.7 114.3 15.8X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 272 280 10 57.8 17.3 1.0X -ParquetReader Vectorized -> Row 144 185 68 109.3 9.1 1.9X +ParquetReader Vectorized 286 293 6 54.9 18.2 1.0X +ParquetReader Vectorized -> Row 154 179 57 102.3 9.8 1.9X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 29507 29532 34 0.5 1876.0 1.0X -SQL Json 10463 10474 16 1.5 665.2 2.8X -SQL Parquet Vectorized 193 204 10 81.3 12.3 152.6X -SQL Parquet MR 2948 2954 7 5.3 187.5 10.0X -SQL ORC Vectorized 268 277 9 58.7 17.0 110.1X -SQL ORC MR 1910 1950 57 8.2 121.4 15.5X +SQL CSV 30821 30902 114 0.5 1959.5 1.0X +SQL Json 10935 10944 13 1.4 695.3 2.8X +SQL Parquet Vectorized 203 213 12 77.6 12.9 152.1X +SQL Parquet MR 2334 2351 24 6.7 148.4 13.2X +SQL ORC Vectorized 281 286 4 56.0 17.9 109.6X +SQL ORC MR 1943 2022 112 8.1 123.5 15.9X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 263 278 38 59.7 16.7 1.0X -ParquetReader Vectorized -> Row 259 266 9 60.7 16.5 1.0X +ParquetReader Vectorized 284 291 9 55.5 18.0 1.0X +ParquetReader Vectorized -> Row 277 281 6 56.8 17.6 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 36696 36771 106 0.4 2333.0 1.0X -SQL Json 13496 13520 34 1.2 858.0 2.7X -SQL Parquet Vectorized 282 292 9 55.7 17.9 130.0X -SQL Parquet MR 3358 3383 36 4.7 213.5 10.9X -SQL ORC Vectorized 409 414 5 38.5 26.0 89.7X -SQL ORC MR 2250 2275 35 7.0 143.1 16.3X +SQL CSV 38264 38306 60 0.4 2432.7 1.0X +SQL Json 14369 14371 3 1.1 913.6 2.7X +SQL Parquet Vectorized 313 319 6 50.3 19.9 122.3X +SQL Parquet MR 2581 2602 30 6.1 164.1 14.8X +SQL ORC Vectorized 423 432 9 37.2 26.9 90.4X +SQL ORC MR 2108 2142 49 7.5 134.0 18.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 360 372 15 43.6 22.9 1.0X -ParquetReader Vectorized -> Row 354 357 5 44.4 22.5 1.0X +ParquetReader Vectorized 401 409 8 39.2 25.5 1.0X +ParquetReader Vectorized -> Row 392 400 15 40.2 24.9 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 30462 30466 5 0.5 1936.7 1.0X -SQL Json 12916 12948 45 1.2 821.2 2.4X -SQL Parquet Vectorized 181 185 5 86.7 11.5 168.0X -SQL Parquet MR 2810 2820 14 5.6 178.7 10.8X -SQL ORC Vectorized 426 430 4 36.9 27.1 71.6X -SQL ORC MR 2106 2112 9 7.5 133.9 14.5X +SQL CSV 36276 36300 34 0.4 2306.4 1.0X +SQL Json 13691 14374 967 1.1 870.4 2.6X +SQL Parquet Vectorized 193 198 5 81.6 12.3 188.2X +SQL Parquet MR 2361 2389 40 6.7 150.1 15.4X +SQL ORC Vectorized 430 434 4 36.6 27.3 84.4X +SQL ORC MR 2037 2072 50 7.7 129.5 17.8X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 255 261 7 61.6 16.2 1.0X -ParquetReader Vectorized -> Row 285 288 5 55.1 18.1 0.9X +ParquetReader Vectorized 277 284 10 56.8 17.6 1.0X +ParquetReader Vectorized -> Row 274 276 4 57.5 17.4 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 36950 36979 41 0.4 2349.2 1.0X -SQL Json 18794 18795 2 0.8 1194.9 2.0X -SQL Parquet Vectorized 279 295 17 56.3 17.8 132.3X -SQL Parquet MR 3933 4025 130 4.0 250.0 9.4X -SQL ORC Vectorized 521 527 6 30.2 33.2 70.9X -SQL ORC MR 2290 2326 51 6.9 145.6 16.1X +SQL CSV 39757 39761 5 0.4 2527.7 1.0X +SQL Json 20049 20052 5 0.8 1274.7 2.0X +SQL Parquet Vectorized 310 318 10 50.7 19.7 128.3X +SQL Parquet MR 2535 2571 52 6.2 161.2 15.7X +SQL ORC Vectorized 537 543 8 29.3 34.1 74.1X +SQL ORC MR 2132 2161 41 7.4 135.6 18.6X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 356 365 12 44.2 22.6 1.0X -ParquetReader Vectorized -> Row 350 352 2 45.0 22.2 1.0X +ParquetReader Vectorized 390 394 5 40.3 24.8 1.0X +ParquetReader Vectorized -> Row 389 391 5 40.5 24.7 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 26764 26810 65 0.4 2552.4 1.0X -SQL Json 12107 12195 124 0.9 1154.6 2.2X -SQL Parquet Vectorized 2202 2210 10 4.8 210.0 12.2X -SQL Parquet MR 5297 5302 6 2.0 505.2 5.1X -SQL ORC Vectorized 2356 2372 23 4.5 224.7 11.4X -SQL ORC MR 4370 4419 70 2.4 416.8 6.1X +SQL CSV 27215 27240 34 0.4 2595.5 1.0X +SQL Json 12713 12783 100 0.8 1212.4 2.1X +SQL Parquet Vectorized 2265 2269 5 4.6 216.0 12.0X +SQL Parquet MR 4477 4544 95 2.3 426.9 6.1X +SQL ORC Vectorized 2388 2404 23 4.4 227.7 11.4X +SQL ORC MR 4295 4305 15 2.4 409.6 6.3X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 19953 19966 18 0.5 1902.8 1.0X -SQL Json 7151 7220 98 1.5 681.9 2.8X -SQL Parquet Vectorized 692 695 3 15.1 66.0 28.8X -SQL Parquet MR 2859 2943 118 3.7 272.6 7.0X -SQL ORC Vectorized 535 540 5 19.6 51.0 37.3X -SQL ORC MR 2157 2162 8 4.9 205.7 9.3X +SQL CSV 17544 17580 51 0.6 1673.1 1.0X +SQL Json 8277 8328 71 1.3 789.4 2.1X +SQL Parquet Vectorized 674 682 7 15.6 64.3 26.0X +SQL Parquet MR 1960 1972 17 5.3 187.0 8.9X +SQL ORC Vectorized 551 558 11 19.0 52.6 31.8X +SQL ORC MR 2047 2052 6 5.1 195.2 8.6X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column - CSV 46775 46785 13 0.3 2973.9 1.0X -Data column - Json 13891 13893 2 1.1 883.2 3.4X -Data column - Parquet Vectorized 301 306 7 52.3 19.1 155.6X -Data column - Parquet MR 3565 3572 10 4.4 226.7 13.1X -Data column - ORC Vectorized 434 458 36 36.2 27.6 107.7X -Data column - ORC MR 2337 2354 24 6.7 148.6 20.0X -Partition column - CSV 10645 10688 61 1.5 676.8 4.4X -Partition column - Json 10912 10973 87 1.4 693.7 4.3X -Partition column - Parquet Vectorized 93 103 9 169.4 5.9 503.8X -Partition column - Parquet MR 1588 1597 13 9.9 100.9 29.5X -Partition column - ORC Vectorized 92 99 11 170.7 5.9 507.6X -Partition column - ORC MR 1714 1716 3 9.2 109.0 27.3X -Both columns - CSV 46199 46222 32 0.3 2937.3 1.0X -Both columns - Json 17279 17291 18 0.9 1098.6 2.7X -Both columns - Parquet Vectorized 346 355 13 45.4 22.0 135.0X -Both columns - Parquet MR 3883 3908 35 4.1 246.9 12.0X -Both columns - ORC Vectorized 577 618 57 27.3 36.7 81.1X -Both columns - ORC MR 2967 3024 80 5.3 188.7 15.8X +Data column - CSV 40273 40290 24 0.4 2560.5 1.0X +Data column - Json 14420 14440 28 1.1 916.8 2.8X +Data column - Parquet Vectorized 336 342 6 46.8 21.4 119.8X +Data column - Parquet MR 2651 2652 2 5.9 168.5 15.2X +Data column - ORC Vectorized 444 451 9 35.4 28.2 90.7X +Data column - ORC MR 2342 2356 20 6.7 148.9 17.2X +Partition column - CSV 11307 11310 4 1.4 718.9 3.6X +Partition column - Json 12105 12115 14 1.3 769.6 3.3X +Partition column - Parquet Vectorized 87 97 13 181.2 5.5 464.0X +Partition column - Parquet MR 1364 1368 7 11.5 86.7 29.5X +Partition column - ORC Vectorized 83 97 13 189.0 5.3 484.1X +Partition column - ORC MR 1424 1437 19 11.0 90.5 28.3X +Both columns - CSV 41896 42166 381 0.4 2663.7 1.0X +Both columns - Json 15852 15871 27 1.0 1007.8 2.5X +Both columns - Parquet Vectorized 379 383 5 41.5 24.1 106.2X +Both columns - Parquet MR 2889 2916 38 5.4 183.7 13.9X +Both columns - ORC Vectorized 581 582 2 27.1 36.9 69.3X +Both columns - ORC MR 2626 2641 22 6.0 166.9 15.3X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 23623 23731 153 0.4 2252.9 1.0X -SQL Json 13299 13432 187 0.8 1268.3 1.8X -SQL Parquet Vectorized 1464 1466 4 7.2 139.6 16.1X -SQL Parquet MR 7602 7628 37 1.4 724.9 3.1X -ParquetReader Vectorized 1032 1043 15 10.2 98.4 22.9X -SQL ORC Vectorized 1206 1211 7 8.7 115.0 19.6X -SQL ORC MR 4726 4991 374 2.2 450.7 5.0X +SQL CSV 20831 21141 439 0.5 1986.6 1.0X +SQL Json 11720 11721 1 0.9 1117.7 1.8X +SQL Parquet Vectorized 1470 1475 7 7.1 140.2 14.2X +SQL Parquet MR 3902 3902 0 2.7 372.1 5.3X +ParquetReader Vectorized 1074 1077 4 9.8 102.5 19.4X +SQL ORC Vectorized 1289 1334 64 8.1 122.9 16.2X +SQL ORC MR 3603 3612 13 2.9 343.6 5.8X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 23715 24152 619 0.4 2261.6 1.0X -SQL Json 10120 10280 226 1.0 965.1 2.3X -SQL Parquet Vectorized 1063 1072 13 9.9 101.4 22.3X -SQL Parquet MR 5460 5464 5 1.9 520.8 4.3X -ParquetReader Vectorized 934 936 4 11.2 89.0 25.4X -SQL ORC Vectorized 1094 1094 0 9.6 104.3 21.7X -SQL ORC MR 3964 4401 618 2.6 378.0 6.0X +SQL CSV 21850 21910 85 0.5 2083.8 1.0X +SQL Json 8651 8668 24 1.2 825.0 2.5X +SQL Parquet Vectorized 1079 1090 16 9.7 102.9 20.3X +SQL Parquet MR 2906 2925 27 3.6 277.1 7.5X +ParquetReader Vectorized 951 954 4 11.0 90.7 23.0X +SQL ORC Vectorized 1246 1250 5 8.4 118.8 17.5X +SQL ORC MR 3146 3162 22 3.3 300.1 6.9X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 21348 21472 175 0.5 2035.9 1.0X -SQL Json 5877 5956 112 1.8 560.5 3.6X -SQL Parquet Vectorized 244 256 22 43.0 23.2 87.6X -SQL Parquet MR 3139 3371 328 3.3 299.4 6.8X -ParquetReader Vectorized 238 245 9 44.1 22.7 89.7X -SQL ORC Vectorized 378 383 7 27.7 36.0 56.5X -SQL ORC MR 2234 2315 115 4.7 213.0 9.6X +SQL CSV 18993 19140 209 0.6 1811.3 1.0X +SQL Json 5467 5469 2 1.9 521.4 3.5X +SQL Parquet Vectorized 240 248 10 43.8 22.8 79.3X +SQL Parquet MR 1745 1753 12 6.0 166.4 10.9X +ParquetReader Vectorized 240 244 5 43.7 22.9 79.1X +SQL ORC Vectorized 496 500 4 21.1 47.3 38.3X +SQL ORC MR 1822 1827 8 5.8 173.7 10.4X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 4053 4064 16 0.3 3865.4 1.0X -SQL Json 4115 4118 4 0.3 3924.6 1.0X -SQL Parquet Vectorized 72 82 11 14.5 69.0 56.0X -SQL Parquet MR 314 325 18 3.3 299.3 12.9X -SQL ORC Vectorized 80 86 8 13.1 76.2 50.7X -SQL ORC MR 250 253 2 4.2 238.5 16.2X +SQL CSV 3907 3911 6 0.3 3726.3 1.0X +SQL Json 3755 3763 12 0.3 3581.2 1.0X +SQL Parquet Vectorized 68 71 6 15.4 64.8 57.5X +SQL Parquet MR 234 239 5 4.5 223.0 16.7X +SQL ORC Vectorized 74 77 5 14.2 70.4 52.9X +SQL ORC MR 203 204 2 5.2 193.3 19.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 50 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 7802 7849 66 0.1 7440.8 1.0X -SQL Json 16640 17481 1190 0.1 15868.8 0.5X -SQL Parquet Vectorized 106 126 31 9.9 101.0 73.7X -SQL Parquet MR 349 358 7 3.0 332.8 22.4X -SQL ORC Vectorized 108 115 10 9.7 102.7 72.5X -SQL ORC MR 284 298 20 3.7 270.5 27.5X +SQL CSV 7909 7927 25 0.1 7542.9 1.0X +SQL Json 15014 15101 123 0.1 14318.8 0.5X +SQL Parquet Vectorized 105 128 22 10.0 100.0 75.4X +SQL Parquet MR 275 283 9 3.8 261.9 28.8X +SQL ORC Vectorized 104 116 9 10.1 98.9 76.3X +SQL ORC MR 234 245 12 4.5 223.0 33.8X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 12639 12672 47 0.1 12053.5 1.0X -SQL Json 30613 30688 106 0.0 29194.8 0.4X -SQL Parquet Vectorized 145 165 21 7.2 138.3 87.2X -SQL Parquet MR 384 393 9 2.7 366.4 32.9X -SQL ORC Vectorized 129 134 5 8.1 123.2 97.8X -SQL ORC MR 280 319 66 3.7 266.9 45.2X +SQL CSV 13033 13129 136 0.1 12429.1 1.0X +SQL Json 28298 29130 1176 0.0 26987.3 0.5X +SQL Parquet Vectorized 139 151 9 7.5 132.7 93.7X +SQL Parquet MR 314 322 7 3.3 299.5 41.5X +SQL ORC Vectorized 123 143 17 8.5 117.3 106.0X +SQL ORC MR 260 272 9 4.0 248.1 50.1X diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt index 4e8bb019fba43..40e8dfc77c712 100644 --- a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt +++ b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt @@ -2,251 +2,251 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 23037 23172 191 0.7 1464.7 1.0X -SQL Json 8682 8686 5 1.8 552.0 2.7X -SQL Parquet Vectorized 183 205 32 85.9 11.6 125.8X -SQL Parquet MR 2189 2200 15 7.2 139.2 10.5X -SQL ORC Vectorized 296 306 5 53.1 18.8 77.7X -SQL ORC MR 1705 1717 18 9.2 108.4 13.5X +SQL CSV 24716 24743 38 0.6 1571.4 1.0X +SQL Json 9669 9686 25 1.6 614.7 2.6X +SQL Parquet Vectorized 172 193 21 91.2 11.0 143.4X +SQL Parquet MR 1929 1942 18 8.2 122.7 12.8X +SQL ORC Vectorized 247 266 19 63.6 15.7 99.9X +SQL ORC MR 1640 1660 29 9.6 104.3 15.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 195 200 7 80.9 12.4 1.0X -ParquetReader Vectorized -> Row 96 97 2 163.0 6.1 2.0X +ParquetReader Vectorized 197 200 4 79.9 12.5 1.0X +ParquetReader Vectorized -> Row 96 98 3 164.1 6.1 2.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 25126 25265 196 0.6 1597.5 1.0X -SQL Json 9442 9445 4 1.7 600.3 2.7X -SQL Parquet Vectorized 228 240 7 69.1 14.5 110.4X -SQL Parquet MR 2432 2445 19 6.5 154.6 10.3X -SQL ORC Vectorized 315 319 6 49.9 20.0 79.8X -SQL ORC MR 1901 1916 21 8.3 120.9 13.2X +SQL CSV 25320 25343 32 0.6 1609.8 1.0X +SQL Json 10460 10465 8 1.5 665.0 2.4X +SQL Parquet Vectorized 206 218 13 76.5 13.1 123.2X +SQL Parquet MR 2032 2036 6 7.7 129.2 12.5X +SQL ORC Vectorized 295 301 4 53.4 18.7 85.9X +SQL ORC MR 1867 1885 25 8.4 118.7 13.6X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 293 302 9 53.6 18.7 1.0X -ParquetReader Vectorized -> Row 264 266 2 59.7 16.8 1.1X +ParquetReader Vectorized 288 294 6 54.6 18.3 1.0X +ParquetReader Vectorized -> Row 252 254 4 62.3 16.0 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 27419 27443 34 0.6 1743.3 1.0X -SQL Json 9831 9836 8 1.6 625.0 2.8X -SQL Parquet Vectorized 192 198 9 81.8 12.2 142.7X -SQL Parquet MR 2696 2740 62 5.8 171.4 10.2X -SQL ORC Vectorized 329 335 8 47.9 20.9 83.4X -SQL ORC MR 1932 2006 105 8.1 122.8 14.2X +SQL CSV 27385 27423 54 0.6 1741.1 1.0X +SQL Json 10118 10133 20 1.6 643.3 2.7X +SQL Parquet Vectorized 180 189 10 87.4 11.4 152.1X +SQL Parquet MR 2548 2552 6 6.2 162.0 10.7X +SQL ORC Vectorized 306 312 8 51.4 19.4 89.5X +SQL ORC MR 1882 1927 64 8.4 119.6 14.6X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 248 253 6 63.5 15.8 1.0X -ParquetReader Vectorized -> Row 250 256 7 62.9 15.9 1.0X +ParquetReader Vectorized 255 260 7 61.7 16.2 1.0X +ParquetReader Vectorized -> Row 252 257 6 62.4 16.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 34898 34907 14 0.5 2218.7 1.0X -SQL Json 12760 12764 5 1.2 811.3 2.7X -SQL Parquet Vectorized 283 289 5 55.6 18.0 123.3X -SQL Parquet MR 3238 3240 3 4.9 205.9 10.8X -SQL ORC Vectorized 401 405 7 39.2 25.5 87.0X -SQL ORC MR 2274 2290 23 6.9 144.6 15.3X +SQL CSV 36971 37037 94 0.4 2350.5 1.0X +SQL Json 13285 13300 22 1.2 844.6 2.8X +SQL Parquet Vectorized 275 285 5 57.1 17.5 134.3X +SQL Parquet MR 2599 2603 6 6.1 165.3 14.2X +SQL ORC Vectorized 386 395 5 40.7 24.6 95.7X +SQL ORC MR 2059 2075 22 7.6 130.9 18.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 339 351 16 46.5 21.5 1.0X -ParquetReader Vectorized -> Row 342 348 13 46.0 21.8 1.0X +ParquetReader Vectorized 352 361 14 44.7 22.4 1.0X +ParquetReader Vectorized -> Row 386 392 8 40.7 24.6 0.9X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 28872 28886 20 0.5 1835.6 1.0X -SQL Json 13360 13377 24 1.2 849.4 2.2X -SQL Parquet Vectorized 181 185 6 86.8 11.5 159.3X -SQL Parquet MR 2645 2651 8 5.9 168.2 10.9X -SQL ORC Vectorized 456 459 5 34.5 29.0 63.4X -SQL ORC MR 2047 2066 26 7.7 130.2 14.1X +SQL CSV 29272 29322 71 0.5 1861.1 1.0X +SQL Json 15022 15099 108 1.0 955.1 1.9X +SQL Parquet Vectorized 172 178 6 91.5 10.9 170.2X +SQL Parquet MR 2184 2206 31 7.2 138.9 13.4X +SQL ORC Vectorized 477 485 6 32.9 30.4 61.3X +SQL ORC MR 2036 2054 26 7.7 129.4 14.4X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 240 246 10 65.5 15.3 1.0X -ParquetReader Vectorized -> Row 245 246 2 64.2 15.6 1.0X +ParquetReader Vectorized 251 255 5 62.6 16.0 1.0X +ParquetReader Vectorized -> Row 248 254 7 63.5 15.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 36298 36305 10 0.4 2307.7 1.0X -SQL Json 18250 18276 36 0.9 1160.3 2.0X -SQL Parquet Vectorized 278 285 7 56.5 17.7 130.4X -SQL Parquet MR 3144 3146 4 5.0 199.9 11.5X -SQL ORC Vectorized 533 546 16 29.5 33.9 68.1X -SQL ORC MR 2265 2302 53 6.9 144.0 16.0X +SQL CSV 38020 38024 6 0.4 2417.2 1.0X +SQL Json 20449 20463 19 0.8 1300.1 1.9X +SQL Parquet Vectorized 268 274 8 58.7 17.0 141.8X +SQL Parquet MR 2484 2493 12 6.3 157.9 15.3X +SQL ORC Vectorized 580 582 2 27.1 36.9 65.6X +SQL ORC MR 2179 2199 29 7.2 138.5 17.5X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 338 346 12 46.6 21.5 1.0X -ParquetReader Vectorized -> Row 338 344 9 46.5 21.5 1.0X +ParquetReader Vectorized 344 350 7 45.7 21.9 1.0X +ParquetReader Vectorized -> Row 346 352 12 45.5 22.0 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 24839 25273 613 0.4 2368.9 1.0X -SQL Json 11861 11869 11 0.9 1131.2 2.1X -SQL Parquet Vectorized 2298 2305 9 4.6 219.2 10.8X -SQL Parquet MR 5045 5053 10 2.1 481.2 4.9X -SQL ORC Vectorized 2391 2405 21 4.4 228.0 10.4X -SQL ORC MR 4561 4645 118 2.3 435.0 5.4X +SQL CSV 27652 28221 804 0.4 2637.1 1.0X +SQL Json 12827 12842 21 0.8 1223.3 2.2X +SQL Parquet Vectorized 2297 2311 19 4.6 219.1 12.0X +SQL Parquet MR 4207 4217 15 2.5 401.2 6.6X +SQL ORC Vectorized 2316 2342 36 4.5 220.9 11.9X +SQL ORC MR 4158 4236 110 2.5 396.5 6.7X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 14147 14244 137 0.7 1349.1 1.0X -SQL Json 7289 7306 23 1.4 695.1 1.9X -SQL Parquet Vectorized 818 821 4 12.8 78.0 17.3X -SQL Parquet MR 2562 2570 11 4.1 244.4 5.5X -SQL ORC Vectorized 571 579 8 18.3 54.5 24.8X -SQL ORC MR 2143 2164 31 4.9 204.3 6.6X +SQL CSV 19185 19343 224 0.5 1829.6 1.0X +SQL Json 7682 7692 14 1.4 732.6 2.5X +SQL Parquet Vectorized 796 805 9 13.2 75.9 24.1X +SQL Parquet MR 1880 1891 17 5.6 179.2 10.2X +SQL ORC Vectorized 553 558 5 19.0 52.7 34.7X +SQL ORC MR 2105 2128 32 5.0 200.8 9.1X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column - CSV 38652 38680 40 0.4 2457.4 1.0X -Data column - Json 12756 12760 5 1.2 811.0 3.0X -Data column - Parquet Vectorized 304 314 9 51.7 19.3 127.2X -Data column - Parquet MR 3387 3393 9 4.6 215.3 11.4X -Data column - ORC Vectorized 425 436 10 37.0 27.0 91.0X -Data column - ORC MR 2303 2330 38 6.8 146.4 16.8X -Partition column - CSV 11239 11249 14 1.4 714.5 3.4X -Partition column - Json 10477 10479 3 1.5 666.1 3.7X -Partition column - Parquet Vectorized 95 102 9 165.5 6.0 406.7X -Partition column - Parquet MR 1574 1575 1 10.0 100.1 24.6X -Partition column - ORC Vectorized 95 106 20 166.3 6.0 408.5X -Partition column - ORC MR 1682 1693 15 9.4 106.9 23.0X -Both columns - CSV 39146 39203 81 0.4 2488.8 1.0X -Both columns - Json 14675 14691 23 1.1 933.0 2.6X -Both columns - Parquet Vectorized 347 351 3 45.3 22.1 111.4X -Both columns - Parquet MR 3680 3717 52 4.3 234.0 10.5X -Both columns - ORC Vectorized 556 565 8 28.3 35.3 69.6X -Both columns - ORC MR 2909 2923 20 5.4 184.9 13.3X +Data column - CSV 43759 43811 73 0.4 2782.1 1.0X +Data column - Json 13866 13874 11 1.1 881.6 3.2X +Data column - Parquet Vectorized 292 302 10 53.9 18.5 150.1X +Data column - Parquet MR 2681 2697 23 5.9 170.5 16.3X +Data column - ORC Vectorized 416 422 12 37.8 26.4 105.2X +Data column - ORC MR 2256 2275 27 7.0 143.4 19.4X +Partition column - CSV 13909 13949 56 1.1 884.3 3.1X +Partition column - Json 11248 11252 7 1.4 715.1 3.9X +Partition column - Parquet Vectorized 83 95 13 189.4 5.3 526.9X +Partition column - Parquet MR 1531 1532 2 10.3 97.3 28.6X +Partition column - ORC Vectorized 81 97 17 193.1 5.2 537.3X +Partition column - ORC MR 1557 1570 19 10.1 99.0 28.1X +Both columns - CSV 48341 48524 259 0.3 3073.4 0.9X +Both columns - Json 13636 13652 23 1.2 866.9 3.2X +Both columns - Parquet Vectorized 341 354 16 46.1 21.7 128.2X +Both columns - Parquet MR 2806 2825 26 5.6 178.4 15.6X +Both columns - ORC Vectorized 548 554 8 28.7 34.8 79.8X +Both columns - ORC MR 2602 2632 43 6.0 165.4 16.8X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 17457 17740 401 0.6 1664.9 1.0X -SQL Json 12276 12287 16 0.9 1170.7 1.4X -SQL Parquet Vectorized 1525 1539 20 6.9 145.4 11.5X -SQL Parquet MR 5051 5098 66 2.1 481.7 3.5X -ParquetReader Vectorized 1115 1123 12 9.4 106.3 15.7X -SQL ORC Vectorized 1269 1294 37 8.3 121.0 13.8X -SQL ORC MR 3938 3951 17 2.7 375.6 4.4X +SQL CSV 22570 22687 166 0.5 2152.4 1.0X +SQL Json 11103 11129 38 0.9 1058.8 2.0X +SQL Parquet Vectorized 1508 1516 12 7.0 143.8 15.0X +SQL Parquet MR 3686 3692 9 2.8 351.5 6.1X +ParquetReader Vectorized 1117 1133 22 9.4 106.6 20.2X +SQL ORC Vectorized 1195 1212 24 8.8 114.0 18.9X +SQL ORC MR 3617 3618 3 2.9 344.9 6.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 18086 18119 47 0.6 1724.8 1.0X -SQL Json 8484 8851 520 1.2 809.1 2.1X -SQL Parquet Vectorized 1127 1131 5 9.3 107.5 16.0X -SQL Parquet MR 4120 4131 15 2.5 392.9 4.4X -ParquetReader Vectorized 984 1019 49 10.7 93.9 18.4X -SQL ORC Vectorized 1208 1211 4 8.7 115.2 15.0X -SQL ORC MR 3401 3410 13 3.1 324.4 5.3X +SQL CSV 19569 19819 354 0.5 1866.2 1.0X +SQL Json 8292 8308 22 1.3 790.8 2.4X +SQL Parquet Vectorized 1107 1136 41 9.5 105.6 17.7X +SQL Parquet MR 2784 2812 39 3.8 265.5 7.0X +ParquetReader Vectorized 990 994 5 10.6 94.4 19.8X +SQL ORC Vectorized 1198 1199 2 8.8 114.2 16.3X +SQL ORC MR 3164 3195 44 3.3 301.7 6.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 24825 24970 205 0.4 2367.5 1.0X -SQL Json 9847 9857 14 1.1 939.1 2.5X -SQL Parquet Vectorized 258 261 6 40.7 24.6 96.3X -SQL Parquet MR 3182 3242 85 3.3 303.4 7.8X -ParquetReader Vectorized 241 242 2 43.6 22.9 103.2X -SQL ORC Vectorized 453 456 4 23.1 43.2 54.8X -SQL ORC MR 1917 1927 13 5.5 182.8 12.9X +SQL CSV 15940 15969 41 0.7 1520.1 1.0X +SQL Json 4845 4845 0 2.2 462.0 3.3X +SQL Parquet Vectorized 243 249 6 43.1 23.2 65.5X +SQL Parquet MR 1732 1751 26 6.1 165.2 9.2X +ParquetReader Vectorized 241 243 3 43.4 23.0 66.0X +SQL ORC Vectorized 425 431 7 24.7 40.5 37.5X +SQL ORC MR 1713 1728 20 6.1 163.4 9.3X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 5163 5174 16 0.2 4923.5 1.0X -SQL Json 4459 4538 111 0.2 4252.7 1.2X -SQL Parquet Vectorized 78 84 8 13.4 74.7 65.9X -SQL Parquet MR 511 519 9 2.1 486.9 10.1X -SQL ORC Vectorized 86 93 11 12.2 82.1 60.0X -SQL ORC MR 350 359 7 3.0 333.6 14.8X +SQL CSV 3838 3885 66 0.3 3660.4 1.0X +SQL Json 3615 3615 0 0.3 3447.8 1.1X +SQL Parquet Vectorized 66 74 8 15.8 63.2 57.9X +SQL Parquet MR 230 237 6 4.6 219.3 16.7X +SQL ORC Vectorized 72 77 9 14.5 68.9 53.1X +SQL ORC MR 194 201 5 5.4 185.3 19.7X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 50 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 9839 9842 4 0.1 9383.4 1.0X -SQL Json 15887 15889 4 0.1 15150.7 0.6X -SQL Parquet Vectorized 115 125 11 9.1 109.9 85.4X -SQL Parquet MR 666 671 8 1.6 635.4 14.8X -SQL ORC Vectorized 115 120 6 9.1 110.1 85.2X -SQL ORC MR 455 458 3 2.3 433.7 21.6X +SQL CSV 8711 8754 60 0.1 8307.9 1.0X +SQL Json 14414 14423 12 0.1 13746.5 0.6X +SQL Parquet Vectorized 97 106 12 10.8 92.7 89.6X +SQL Parquet MR 267 274 7 3.9 254.2 32.7X +SQL ORC Vectorized 100 104 7 10.5 95.1 87.4X +SQL ORC MR 226 230 6 4.6 215.2 38.6X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -SQL CSV 15858 15891 46 0.1 15123.5 1.0X -SQL Json 30200 30256 80 0.0 28800.6 0.5X -SQL Parquet Vectorized 160 165 7 6.5 153.0 98.8X -SQL Parquet MR 682 690 7 1.5 650.3 23.3X -SQL ORC Vectorized 143 150 10 7.4 136.0 111.2X -SQL ORC MR 494 509 15 2.1 471.4 32.1X +SQL CSV 14509 14596 123 0.1 13836.8 1.0X +SQL Json 27545 27909 515 0.0 26269.1 0.5X +SQL Parquet Vectorized 141 151 13 7.4 134.8 102.7X +SQL Parquet MR 313 341 23 3.4 298.4 46.4X +SQL ORC Vectorized 121 129 15 8.7 115.4 119.9X +SQL ORC MR 252 269 33 4.2 240.3 57.6X diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt index 27cb1008fa4a1..729b112de5e75 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt @@ -2,428 +2,428 @@ Extract components ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 460 486 36 21.7 46.0 1.0X -cast to timestamp wholestage on 412 455 70 24.3 41.2 1.1X +cast to timestamp wholestage off 546 572 36 18.3 54.6 1.0X +cast to timestamp wholestage on 412 438 16 24.3 41.2 1.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 1351 1351 1 7.4 135.1 1.0X -year of timestamp wholestage on 1277 1312 47 7.8 127.7 1.1X +year of timestamp wholestage off 1240 1295 77 8.1 124.0 1.0X +year of timestamp wholestage on 1109 1130 24 9.0 110.9 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 1591 1604 19 6.3 159.1 1.0X -quarter of timestamp wholestage on 1583 1600 12 6.3 158.3 1.0X +quarter of timestamp wholestage off 1572 1574 3 6.4 157.2 1.0X +quarter of timestamp wholestage on 1386 1405 18 7.2 138.6 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 1263 1268 7 7.9 126.3 1.0X -month of timestamp wholestage on 1281 1316 46 7.8 128.1 1.0X +month of timestamp wholestage off 1194 1196 2 8.4 119.4 1.0X +month of timestamp wholestage on 1057 1069 12 9.5 105.7 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1756 1757 1 5.7 175.6 1.0X -weekofyear of timestamp wholestage on 1748 1762 15 5.7 174.8 1.0X +weekofyear of timestamp wholestage off 2070 2071 2 4.8 207.0 1.0X +weekofyear of timestamp wholestage on 1549 1555 6 6.5 154.9 1.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 1250 1253 4 8.0 125.0 1.0X -day of timestamp wholestage on 1270 1285 13 7.9 127.0 1.0X +day of timestamp wholestage off 1173 1186 18 8.5 117.3 1.0X +day of timestamp wholestage on 1056 1076 26 9.5 105.6 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 1295 1303 12 7.7 129.5 1.0X -dayofyear of timestamp wholestage on 1294 1308 16 7.7 129.4 1.0X +dayofyear of timestamp wholestage off 1207 1211 6 8.3 120.7 1.0X +dayofyear of timestamp wholestage on 1097 1108 9 9.1 109.7 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 1277 1311 48 7.8 127.7 1.0X -dayofmonth of timestamp wholestage on 1270 1288 24 7.9 127.0 1.0X +dayofmonth of timestamp wholestage off 1184 1190 8 8.4 118.4 1.0X +dayofmonth of timestamp wholestage on 1053 1060 9 9.5 105.3 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 1472 1472 0 6.8 147.2 1.0X -dayofweek of timestamp wholestage on 1434 1437 4 7.0 143.4 1.0X +dayofweek of timestamp wholestage off 1343 1362 27 7.4 134.3 1.0X +dayofweek of timestamp wholestage on 1228 1239 7 8.1 122.8 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 1363 1372 13 7.3 136.3 1.0X -weekday of timestamp wholestage on 1362 1368 3 7.3 136.2 1.0X +weekday of timestamp wholestage off 1276 1278 3 7.8 127.6 1.0X +weekday of timestamp wholestage on 1160 1181 22 8.6 116.0 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 748 753 7 13.4 74.8 1.0X -hour of timestamp wholestage on 749 756 9 13.4 74.9 1.0X +hour of timestamp wholestage off 854 862 12 11.7 85.4 1.0X +hour of timestamp wholestage on 741 748 6 13.5 74.1 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 722 725 4 13.9 72.2 1.0X -minute of timestamp wholestage on 750 754 4 13.3 75.0 1.0X +minute of timestamp wholestage off 853 854 1 11.7 85.3 1.0X +minute of timestamp wholestage on 730 737 11 13.7 73.0 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 614 620 8 16.3 61.4 1.0X -second of timestamp wholestage on 611 617 5 16.4 61.1 1.0X +second of timestamp wholestage off 726 728 2 13.8 72.6 1.0X +second of timestamp wholestage on 614 623 9 16.3 61.4 1.2X ================================================================================================ Current date and time ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 291 298 10 34.4 29.1 1.0X -current_date wholestage on 303 316 13 33.0 30.3 1.0X +current_date wholestage off 369 370 2 27.1 36.9 1.0X +current_date wholestage on 277 284 8 36.2 27.7 1.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 328 355 39 30.5 32.8 1.0X -current_timestamp wholestage on 309 497 387 32.4 30.9 1.1X +current_timestamp wholestage off 410 411 1 24.4 41.0 1.0X +current_timestamp wholestage on 283 418 259 35.4 28.3 1.5X ================================================================================================ Date arithmetic ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 1094 1103 12 9.1 109.4 1.0X -cast to date wholestage on 1105 1111 5 9.0 110.5 1.0X +cast to date wholestage off 987 992 7 10.1 98.7 1.0X +cast to date wholestage on 891 896 4 11.2 89.1 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 1258 1258 0 7.9 125.8 1.0X -last_day wholestage on 1279 1289 9 7.8 127.9 1.0X +last_day wholestage off 1179 1179 1 8.5 117.9 1.0X +last_day wholestage on 1052 1080 44 9.5 105.2 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 1148 1152 7 8.7 114.8 1.0X -next_day wholestage on 1148 1152 4 8.7 114.8 1.0X +next_day wholestage off 1073 1081 11 9.3 107.3 1.0X +next_day wholestage on 948 955 10 10.5 94.8 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 1069 1074 6 9.4 106.9 1.0X -date_add wholestage on 1087 1091 4 9.2 108.7 1.0X +date_add wholestage off 1006 1009 5 9.9 100.6 1.0X +date_add wholestage on 867 870 4 11.5 86.7 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 1076 1076 1 9.3 107.6 1.0X -date_sub wholestage on 1086 1094 7 9.2 108.6 1.0X +date_sub wholestage off 980 988 11 10.2 98.0 1.0X +date_sub wholestage on 866 873 9 11.6 86.6 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1442 1442 1 6.9 144.2 1.0X -add_months wholestage on 1406 1426 12 7.1 140.6 1.0X +add_months wholestage off 1329 1332 4 7.5 132.9 1.0X +add_months wholestage on 1199 1206 8 8.3 119.9 1.1X ================================================================================================ Formatting dates ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 5082 5179 138 2.0 508.2 1.0X -format date wholestage on 4899 4904 6 2.0 489.9 1.0X +format date wholestage off 4724 4736 18 2.1 472.4 1.0X +format date wholestage on 4550 4574 26 2.2 455.0 1.0X ================================================================================================ Formatting timestamps ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 7148 7170 31 1.4 714.8 1.0X -from_unixtime wholestage on 7228 7235 9 1.4 722.8 1.0X +from_unixtime wholestage off 7171 7183 17 1.4 717.1 1.0X +from_unixtime wholestage on 7114 7141 20 1.4 711.4 1.0X ================================================================================================ Convert timestamps ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 982 990 11 10.2 98.2 1.0X -from_utc_timestamp wholestage on 1053 1058 5 9.5 105.3 0.9X +from_utc_timestamp wholestage off 1498 1504 8 6.7 149.8 1.0X +from_utc_timestamp wholestage on 1399 1405 5 7.1 139.9 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1063 1075 16 9.4 106.3 1.0X -to_utc_timestamp wholestage on 1121 1136 15 8.9 112.1 0.9X +to_utc_timestamp wholestage off 1505 1507 2 6.6 150.5 1.0X +to_utc_timestamp wholestage on 1396 1401 5 7.2 139.6 1.1X ================================================================================================ Intervals ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 356 374 25 28.1 35.6 1.0X -cast interval wholestage on 341 353 14 29.3 34.1 1.0X +cast interval wholestage off 423 428 7 23.6 42.3 1.0X +cast interval wholestage on 300 302 2 33.3 30.0 1.4X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1883 1898 22 5.3 188.3 1.0X -datediff wholestage on 1854 1862 8 5.4 185.4 1.0X +datediff wholestage off 1626 1630 6 6.1 162.6 1.0X +datediff wholestage on 1467 1471 3 6.8 146.7 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 1989 1996 10 5.0 198.9 1.0X -months_between wholestage on 2006 2041 36 5.0 200.6 1.0X +months_between wholestage off 1988 1992 5 5.0 198.8 1.0X +months_between wholestage on 1812 1834 24 5.5 181.2 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 2761 2820 83 0.4 2761.1 1.0X -window wholestage on 47075 47156 75 0.0 47075.0 0.1X +window wholestage off 2277 2334 80 0.4 2277.1 1.0X +window wholestage on 48996 49048 67 0.0 48996.0 0.0X ================================================================================================ Truncation ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 916 919 5 10.9 91.6 1.0X -date_trunc YEAR wholestage on 902 904 2 11.1 90.2 1.0X +date_trunc YEAR wholestage off 867 870 6 11.5 86.7 1.0X +date_trunc YEAR wholestage on 815 819 6 12.3 81.5 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 909 915 9 11.0 90.9 1.0X -date_trunc YYYY wholestage on 902 909 5 11.1 90.2 1.0X +date_trunc YYYY wholestage off 866 875 13 11.5 86.6 1.0X +date_trunc YYYY wholestage on 811 813 2 12.3 81.1 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 916 925 12 10.9 91.6 1.0X -date_trunc YY wholestage on 903 908 4 11.1 90.3 1.0X +date_trunc YY wholestage off 864 867 4 11.6 86.4 1.0X +date_trunc YY wholestage on 812 824 10 12.3 81.2 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 921 928 11 10.9 92.1 1.0X -date_trunc MON wholestage on 907 912 4 11.0 90.7 1.0X +date_trunc MON wholestage off 881 884 4 11.3 88.1 1.0X +date_trunc MON wholestage on 820 826 7 12.2 82.0 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 923 928 7 10.8 92.3 1.0X -date_trunc MONTH wholestage on 902 911 9 11.1 90.2 1.0X +date_trunc MONTH wholestage off 880 881 2 11.4 88.0 1.0X +date_trunc MONTH wholestage on 819 822 4 12.2 81.9 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 922 924 3 10.8 92.2 1.0X -date_trunc MM wholestage on 908 914 7 11.0 90.8 1.0X +date_trunc MM wholestage off 889 904 21 11.2 88.9 1.0X +date_trunc MM wholestage on 818 828 8 12.2 81.8 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 644 665 30 15.5 64.4 1.0X -date_trunc DAY wholestage on 632 636 7 15.8 63.2 1.0X +date_trunc DAY wholestage off 590 593 4 16.9 59.0 1.0X +date_trunc DAY wholestage on 510 514 4 19.6 51.0 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 626 636 14 16.0 62.6 1.0X -date_trunc DD wholestage on 635 640 5 15.8 63.5 1.0X +date_trunc DD wholestage off 596 604 11 16.8 59.6 1.0X +date_trunc DD wholestage on 511 519 9 19.6 51.1 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 652 656 5 15.3 65.2 1.0X -date_trunc HOUR wholestage on 644 648 5 15.5 64.4 1.0X +date_trunc HOUR wholestage off 586 592 9 17.1 58.6 1.0X +date_trunc HOUR wholestage on 507 513 5 19.7 50.7 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 617 618 2 16.2 61.7 1.0X -date_trunc MINUTE wholestage on 621 625 4 16.1 62.1 1.0X +date_trunc MINUTE wholestage off 561 562 1 17.8 56.1 1.0X +date_trunc MINUTE wholestage on 480 485 4 20.8 48.0 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 631 631 0 15.8 63.1 1.0X -date_trunc SECOND wholestage on 625 630 8 16.0 62.5 1.0X +date_trunc SECOND wholestage off 561 561 1 17.8 56.1 1.0X +date_trunc SECOND wholestage on 479 480 2 20.9 47.9 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 760 761 1 13.2 76.0 1.0X -date_trunc WEEK wholestage on 760 766 5 13.2 76.0 1.0X +date_trunc WEEK wholestage off 725 727 2 13.8 72.5 1.0X +date_trunc WEEK wholestage on 674 684 11 14.8 67.4 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 1656 1657 1 6.0 165.6 1.0X -date_trunc QUARTER wholestage on 1648 1678 36 6.1 164.8 1.0X +date_trunc QUARTER wholestage off 1653 1659 10 6.1 165.3 1.0X +date_trunc QUARTER wholestage on 1588 1601 12 6.3 158.8 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 309 309 1 32.4 30.9 1.0X -trunc year wholestage on 310 315 6 32.3 31.0 1.0X +trunc year wholestage off 391 393 2 25.6 39.1 1.0X +trunc year wholestage on 312 316 4 32.1 31.2 1.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 310 310 1 32.3 31.0 1.0X -trunc yyyy wholestage on 310 313 2 32.3 31.0 1.0X +trunc yyyy wholestage off 390 394 5 25.6 39.0 1.0X +trunc yyyy wholestage on 316 319 4 31.6 31.6 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 307 308 2 32.6 30.7 1.0X -trunc yy wholestage on 310 313 3 32.2 31.0 1.0X +trunc yy wholestage off 387 390 5 25.8 38.7 1.0X +trunc yy wholestage on 313 316 3 31.9 31.3 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 308 317 12 32.5 30.8 1.0X -trunc mon wholestage on 309 316 7 32.3 30.9 1.0X +trunc mon wholestage off 388 395 11 25.8 38.8 1.0X +trunc mon wholestage on 314 316 2 31.8 31.4 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 306 307 2 32.7 30.6 1.0X -trunc month wholestage on 309 314 6 32.4 30.9 1.0X +trunc month wholestage off 388 390 4 25.8 38.8 1.0X +trunc month wholestage on 315 318 2 31.7 31.5 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 308 308 0 32.5 30.8 1.0X -trunc mm wholestage on 309 325 11 32.4 30.9 1.0X +trunc mm wholestage off 384 388 4 26.0 38.4 1.0X +trunc mm wholestage on 314 321 9 31.9 31.4 1.2X ================================================================================================ Parsing ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 170 170 1 5.9 170.0 1.0X -to timestamp str wholestage on 160 165 4 6.2 160.2 1.1X +to timestamp str wholestage off 178 191 18 5.6 178.5 1.0X +to timestamp str wholestage on 157 160 2 6.4 156.6 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 1727 1730 4 0.6 1727.1 1.0X -to_timestamp wholestage on 1775 1793 19 0.6 1774.9 1.0X +to_timestamp wholestage off 1790 1795 7 0.6 1790.0 1.0X +to_timestamp wholestage on 1813 1820 10 0.6 1813.1 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 1760 1765 6 0.6 1760.4 1.0X -to_unix_timestamp wholestage on 1756 1763 9 0.6 1756.0 1.0X +to_unix_timestamp wholestage off 1836 1837 1 0.5 1835.8 1.0X +to_unix_timestamp wholestage on 1786 1791 3 0.6 1785.8 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 159 160 0 6.3 159.4 1.0X -to date str wholestage on 156 162 7 6.4 156.2 1.0X +to date str wholestage off 169 169 1 5.9 168.8 1.0X +to date str wholestage on 151 153 2 6.6 151.2 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 2509 2511 3 0.4 2509.2 1.0X -to_date wholestage on 2500 2514 9 0.4 2499.8 1.0X +to_date wholestage off 2504 2512 10 0.4 2504.4 1.0X +to_date wholestage on 2522 2536 19 0.4 2522.3 1.0X ================================================================================================ Conversion from/to external types ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Timestamp 353 356 5 14.2 70.5 1.0X -Collect longs 1561 1864 476 3.2 312.2 0.2X -Collect timestamps 2010 2182 149 2.5 402.0 0.2X +From java.sql.Timestamp 346 367 35 14.5 69.2 1.0X +Collect longs 2139 2329 289 2.3 427.7 0.2X +Collect timestamps 1883 2086 303 2.7 376.5 0.2X diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 32578f6573dfd..0c30534dd71ca 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -2,428 +2,428 @@ Extract components ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 408 434 37 24.5 40.8 1.0X -cast to timestamp wholestage on 363 372 6 27.6 36.3 1.1X +cast to timestamp wholestage off 425 447 30 23.5 42.5 1.0X +cast to timestamp wholestage on 368 401 29 27.2 36.8 1.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 1317 1327 14 7.6 131.7 1.0X -year of timestamp wholestage on 1258 1308 79 8.0 125.8 1.0X +year of timestamp wholestage off 1158 1215 80 8.6 115.8 1.0X +year of timestamp wholestage on 1158 1179 31 8.6 115.8 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 1464 1490 37 6.8 146.4 1.0X -quarter of timestamp wholestage on 1389 1404 13 7.2 138.9 1.1X +quarter of timestamp wholestage off 1285 1295 15 7.8 128.5 1.0X +quarter of timestamp wholestage on 1243 1257 11 8.0 124.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 1239 1245 7 8.1 123.9 1.0X -month of timestamp wholestage on 1241 1253 8 8.1 124.1 1.0X +month of timestamp wholestage off 1076 1082 8 9.3 107.6 1.0X +month of timestamp wholestage on 1088 1098 9 9.2 108.8 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1896 1899 4 5.3 189.6 1.0X -weekofyear of timestamp wholestage on 1876 1885 6 5.3 187.6 1.0X +weekofyear of timestamp wholestage off 1649 1659 14 6.1 164.9 1.0X +weekofyear of timestamp wholestage on 1648 1656 8 6.1 164.8 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 1234 1235 1 8.1 123.4 1.0X -day of timestamp wholestage on 1221 1231 8 8.2 122.1 1.0X +day of timestamp wholestage off 1083 1084 3 9.2 108.3 1.0X +day of timestamp wholestage on 1082 1089 13 9.2 108.2 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 1269 1275 8 7.9 126.9 1.0X -dayofyear of timestamp wholestage on 1266 1277 9 7.9 126.6 1.0X +dayofyear of timestamp wholestage off 1102 1103 1 9.1 110.2 1.0X +dayofyear of timestamp wholestage on 1123 1138 14 8.9 112.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 1236 1238 2 8.1 123.6 1.0X -dayofmonth of timestamp wholestage on 1223 1235 10 8.2 122.3 1.0X +dayofmonth of timestamp wholestage off 1068 1073 7 9.4 106.8 1.0X +dayofmonth of timestamp wholestage on 1082 1095 13 9.2 108.2 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 1408 1409 2 7.1 140.8 1.0X -dayofweek of timestamp wholestage on 1393 1418 38 7.2 139.3 1.0X +dayofweek of timestamp wholestage off 1265 1294 41 7.9 126.5 1.0X +dayofweek of timestamp wholestage on 1253 1262 11 8.0 125.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 1339 1348 12 7.5 133.9 1.0X -weekday of timestamp wholestage on 1329 1341 22 7.5 132.9 1.0X +weekday of timestamp wholestage off 1189 1191 3 8.4 118.9 1.0X +weekday of timestamp wholestage on 1193 1199 6 8.4 119.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 359 366 10 27.9 35.9 1.0X -hour of timestamp wholestage on 344 354 11 29.1 34.4 1.0X +hour of timestamp wholestage off 366 368 3 27.3 36.6 1.0X +hour of timestamp wholestage on 360 364 6 27.8 36.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 343 343 0 29.1 34.3 1.0X -minute of timestamp wholestage on 336 339 6 29.8 33.6 1.0X +minute of timestamp wholestage off 348 350 2 28.7 34.8 1.0X +minute of timestamp wholestage on 355 361 9 28.1 35.5 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 367 383 23 27.2 36.7 1.0X -second of timestamp wholestage on 335 341 6 29.8 33.5 1.1X +second of timestamp wholestage off 347 352 7 28.8 34.7 1.0X +second of timestamp wholestage on 351 359 10 28.5 35.1 1.0X ================================================================================================ Current date and time ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 284 286 2 35.2 28.4 1.0X -current_date wholestage on 286 297 18 35.0 28.6 1.0X +current_date wholestage off 284 287 4 35.2 28.4 1.0X +current_date wholestage on 312 318 6 32.1 31.2 0.9X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 291 336 65 34.4 29.1 1.0X -current_timestamp wholestage on 279 290 11 35.9 27.9 1.0X +current_timestamp wholestage off 291 292 2 34.4 29.1 1.0X +current_timestamp wholestage on 297 333 40 33.6 29.7 1.0X ================================================================================================ Date arithmetic ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 1069 1069 0 9.4 106.9 1.0X -cast to date wholestage on 1037 1043 7 9.6 103.7 1.0X +cast to date wholestage off 903 903 1 11.1 90.3 1.0X +cast to date wholestage on 897 900 7 11.2 89.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 1275 1287 18 7.8 127.5 1.0X -last_day wholestage on 1257 1267 10 8.0 125.7 1.0X +last_day wholestage off 1082 1082 1 9.2 108.2 1.0X +last_day wholestage on 1107 1118 16 9.0 110.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 1114 1116 2 9.0 111.4 1.0X -next_day wholestage on 1077 1081 7 9.3 107.7 1.0X +next_day wholestage off 968 974 8 10.3 96.8 1.0X +next_day wholestage on 958 959 1 10.4 95.8 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 1056 1058 2 9.5 105.6 1.0X -date_add wholestage on 1043 1047 3 9.6 104.3 1.0X +date_add wholestage off 894 895 1 11.2 89.4 1.0X +date_add wholestage on 882 890 9 11.3 88.2 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 1045 1049 5 9.6 104.5 1.0X -date_sub wholestage on 1044 1052 16 9.6 104.4 1.0X +date_sub wholestage off 892 896 6 11.2 89.2 1.0X +date_sub wholestage on 881 888 7 11.3 88.1 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1379 1384 7 7.3 137.9 1.0X -add_months wholestage on 1356 1364 6 7.4 135.6 1.0X +add_months wholestage off 1221 1223 3 8.2 122.1 1.0X +add_months wholestage on 1212 1217 5 8.2 121.2 1.0X ================================================================================================ Formatting dates ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 5436 5538 144 1.8 543.6 1.0X -format date wholestage on 5283 5295 16 1.9 528.3 1.0X +format date wholestage off 4989 5009 29 2.0 498.9 1.0X +format date wholestage on 5037 5055 26 2.0 503.7 1.0X ================================================================================================ Formatting timestamps ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 8980 8998 26 1.1 898.0 1.0X -from_unixtime wholestage on 8921 8939 19 1.1 892.1 1.0X +from_unixtime wholestage off 9157 9164 10 1.1 915.7 1.0X +from_unixtime wholestage on 9101 9120 16 1.1 910.1 1.0X ================================================================================================ Convert timestamps ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 726 736 15 13.8 72.6 1.0X -from_utc_timestamp wholestage on 701 710 16 14.3 70.1 1.0X +from_utc_timestamp wholestage off 732 739 10 13.7 73.2 1.0X +from_utc_timestamp wholestage on 767 776 8 13.0 76.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1234 1240 8 8.1 123.4 1.0X -to_utc_timestamp wholestage on 757 770 15 13.2 75.7 1.6X +to_utc_timestamp wholestage off 802 805 3 12.5 80.2 1.0X +to_utc_timestamp wholestage on 776 781 5 12.9 77.6 1.0X ================================================================================================ Intervals ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 327 328 2 30.6 32.7 1.0X -cast interval wholestage on 299 307 13 33.5 29.9 1.1X +cast interval wholestage off 328 330 3 30.5 32.8 1.0X +cast interval wholestage on 319 326 7 31.3 31.9 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1865 1872 10 5.4 186.5 1.0X -datediff wholestage on 1815 1821 7 5.5 181.5 1.0X +datediff wholestage off 1762 1764 3 5.7 176.2 1.0X +datediff wholestage on 1495 1502 7 6.7 149.5 1.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 1524 1525 1 6.6 152.4 1.0X -months_between wholestage on 1511 1518 11 6.6 151.1 1.0X +months_between wholestage off 1338 1339 1 7.5 133.8 1.0X +months_between wholestage on 1334 1339 5 7.5 133.4 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 2441 2519 111 0.4 2440.7 1.0X -window wholestage on 46149 46165 16 0.0 46149.1 0.1X +window wholestage off 2023 2094 100 0.5 2023.2 1.0X +window wholestage on 43505 43551 33 0.0 43504.8 0.0X ================================================================================================ Truncation ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 773 777 6 12.9 77.3 1.0X -date_trunc YEAR wholestage on 720 733 21 13.9 72.0 1.1X +date_trunc YEAR wholestage off 660 661 1 15.1 66.0 1.0X +date_trunc YEAR wholestage on 589 599 7 17.0 58.9 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 753 754 1 13.3 75.3 1.0X -date_trunc YYYY wholestage on 721 725 3 13.9 72.1 1.0X +date_trunc YYYY wholestage off 656 657 1 15.2 65.6 1.0X +date_trunc YYYY wholestage on 593 604 16 16.9 59.3 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 759 762 4 13.2 75.9 1.0X -date_trunc YY wholestage on 727 731 3 13.8 72.7 1.0X +date_trunc YY wholestage off 666 669 4 15.0 66.6 1.0X +date_trunc YY wholestage on 591 603 19 16.9 59.1 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 728 731 4 13.7 72.8 1.0X -date_trunc MON wholestage on 717 728 17 13.9 71.7 1.0X +date_trunc MON wholestage off 592 592 1 16.9 59.2 1.0X +date_trunc MON wholestage on 569 580 8 17.6 56.9 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 728 729 1 13.7 72.8 1.0X -date_trunc MONTH wholestage on 716 719 2 14.0 71.6 1.0X +date_trunc MONTH wholestage off 593 594 2 16.9 59.3 1.0X +date_trunc MONTH wholestage on 575 579 4 17.4 57.5 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 725 732 9 13.8 72.5 1.0X -date_trunc MM wholestage on 721 728 6 13.9 72.1 1.0X +date_trunc MM wholestage off 589 590 2 17.0 58.9 1.0X +date_trunc MM wholestage on 569 575 4 17.6 56.9 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 515 520 7 19.4 51.5 1.0X -date_trunc DAY wholestage on 475 484 8 21.1 47.5 1.1X +date_trunc DAY wholestage off 438 442 5 22.8 43.8 1.0X +date_trunc DAY wholestage on 346 350 4 28.9 34.6 1.3X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 515 520 7 19.4 51.5 1.0X -date_trunc DD wholestage on 471 484 9 21.2 47.1 1.1X +date_trunc DD wholestage off 438 439 2 22.8 43.8 1.0X +date_trunc DD wholestage on 347 354 7 28.8 34.7 1.3X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 504 504 1 19.9 50.4 1.0X -date_trunc HOUR wholestage on 465 471 5 21.5 46.5 1.1X +date_trunc HOUR wholestage off 384 386 2 26.0 38.4 1.0X +date_trunc HOUR wholestage on 357 365 6 28.0 35.7 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 485 488 4 20.6 48.5 1.0X -date_trunc MINUTE wholestage on 452 458 6 22.1 45.2 1.1X +date_trunc MINUTE wholestage off 373 375 3 26.8 37.3 1.0X +date_trunc MINUTE wholestage on 327 331 5 30.6 32.7 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 483 484 1 20.7 48.3 1.0X -date_trunc SECOND wholestage on 447 453 5 22.4 44.7 1.1X +date_trunc SECOND wholestage off 361 363 3 27.7 36.1 1.0X +date_trunc SECOND wholestage on 335 341 8 29.9 33.5 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 615 617 2 16.3 61.5 1.0X -date_trunc WEEK wholestage on 576 586 11 17.4 57.6 1.1X +date_trunc WEEK wholestage off 515 516 2 19.4 51.5 1.0X +date_trunc WEEK wholestage on 455 459 4 22.0 45.5 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 1512 1513 1 6.6 151.2 1.0X -date_trunc QUARTER wholestage on 1482 1494 11 6.7 148.2 1.0X +date_trunc QUARTER wholestage off 1337 1341 6 7.5 133.7 1.0X +date_trunc QUARTER wholestage on 1328 1334 10 7.5 132.8 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 342 343 1 29.2 34.2 1.0X -trunc year wholestage on 301 304 2 33.3 30.1 1.1X +trunc year wholestage off 318 328 14 31.4 31.8 1.0X +trunc year wholestage on 297 308 17 33.6 29.7 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 321 323 2 31.1 32.1 1.0X -trunc yyyy wholestage on 303 305 2 33.0 30.3 1.1X +trunc yyyy wholestage off 318 319 2 31.5 31.8 1.0X +trunc yyyy wholestage on 296 302 10 33.8 29.6 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 314 325 15 31.8 31.4 1.0X -trunc yy wholestage on 299 307 10 33.4 29.9 1.1X +trunc yy wholestage off 321 345 35 31.2 32.1 1.0X +trunc yy wholestage on 297 319 45 33.6 29.7 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 311 312 2 32.2 31.1 1.0X -trunc mon wholestage on 300 307 6 33.3 30.0 1.0X +trunc mon wholestage off 318 318 0 31.5 31.8 1.0X +trunc mon wholestage on 299 306 6 33.4 29.9 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 312 314 2 32.1 31.2 1.0X -trunc month wholestage on 299 300 1 33.5 29.9 1.0X +trunc month wholestage off 316 318 3 31.6 31.6 1.0X +trunc month wholestage on 296 301 7 33.8 29.6 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 308 316 11 32.5 30.8 1.0X -trunc mm wholestage on 298 301 2 33.6 29.8 1.0X +trunc mm wholestage off 316 321 8 31.7 31.6 1.0X +trunc mm wholestage on 295 302 8 33.9 29.5 1.1X ================================================================================================ Parsing ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 228 232 5 4.4 228.3 1.0X -to timestamp str wholestage on 213 214 2 4.7 212.9 1.1X +to timestamp str wholestage off 219 220 1 4.6 219.3 1.0X +to timestamp str wholestage on 212 214 2 4.7 212.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 1850 1850 0 0.5 1850.0 1.0X -to_timestamp wholestage on 1877 1899 27 0.5 1876.8 1.0X +to_timestamp wholestage off 1852 1852 1 0.5 1851.9 1.0X +to_timestamp wholestage on 1862 1869 9 0.5 1861.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 1883 1886 4 0.5 1882.7 1.0X -to_unix_timestamp wholestage on 1817 1831 17 0.6 1817.3 1.0X +to_unix_timestamp wholestage off 1839 1842 4 0.5 1839.1 1.0X +to_unix_timestamp wholestage on 1861 1866 4 0.5 1861.2 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 212 212 1 4.7 211.8 1.0X -to date str wholestage on 206 208 2 4.8 206.3 1.0X +to date str wholestage off 222 228 9 4.5 221.6 1.0X +to date str wholestage on 210 211 2 4.8 209.5 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 2334 2335 0 0.4 2334.3 1.0X -to_date wholestage on 2224 2236 13 0.4 2224.2 1.0X +to_date wholestage off 2386 2392 8 0.4 2386.3 1.0X +to_date wholestage on 2438 2457 18 0.4 2437.7 1.0X ================================================================================================ Conversion from/to external types ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Timestamp 271 274 3 18.4 54.2 1.0X -Collect longs 1386 1925 681 3.6 277.1 0.2X -Collect timestamps 1961 2169 331 2.5 392.2 0.1X +From java.sql.Timestamp 287 291 7 17.4 57.3 1.0X +Collect longs 1903 2672 694 2.6 380.6 0.2X +Collect timestamps 1544 1644 89 3.2 308.8 0.2X diff --git a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt index 74f01f6ee460a..0572cc25bf6c2 100644 --- a/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-jdk11-results.txt @@ -1,100 +1,119 @@ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 342 456 120 29.2 34.2 1.0X -MILLENNIUM of timestamp 1462 1498 53 6.8 146.2 0.2X -CENTURY of timestamp 1229 1293 86 8.1 122.9 0.3X -DECADE of timestamp 1204 1242 56 8.3 120.4 0.3X -YEAR of timestamp 1180 1226 72 8.5 118.0 0.3X -ISOYEAR of timestamp 1279 1301 33 7.8 127.9 0.3X -QUARTER of timestamp 1357 1379 22 7.4 135.7 0.3X -MONTH of timestamp 1155 1182 24 8.7 115.5 0.3X -WEEK of timestamp 1634 1641 10 6.1 163.4 0.2X -DAY of timestamp 1167 1171 6 8.6 116.7 0.3X -DAYOFWEEK of timestamp 1335 1344 8 7.5 133.5 0.3X -DOW of timestamp 1344 1352 11 7.4 134.4 0.3X -ISODOW of timestamp 1279 1281 2 7.8 127.9 0.3X -DOY of timestamp 1172 1183 16 8.5 117.2 0.3X -HOUR of timestamp 722 731 12 13.8 72.2 0.5X -MINUTE of timestamp 725 734 9 13.8 72.5 0.5X -SECOND of timestamp 656 661 6 15.3 65.6 0.5X -MILLISECONDS of timestamp 630 637 9 15.9 63.0 0.5X -MICROSECONDS of timestamp 587 592 7 17.0 58.7 0.6X -EPOCH of timestamp 1036 1047 13 9.7 103.6 0.3X +cast to timestamp 380 456 67 26.3 38.0 1.0X +MILLENNIUM of timestamp 1274 1361 138 7.9 127.4 0.3X +CENTURY of timestamp 1119 1132 19 8.9 111.9 0.3X +DECADE of timestamp 1076 1083 6 9.3 107.6 0.4X +YEAR of timestamp 1066 1098 43 9.4 106.6 0.4X +ISOYEAR of timestamp 1190 1194 4 8.4 119.0 0.3X +QUARTER of timestamp 1269 1273 4 7.9 126.9 0.3X +MONTH of timestamp 1060 1075 22 9.4 106.0 0.4X +WEEK of timestamp 1560 1565 8 6.4 156.0 0.2X +DAY of timestamp 1039 1046 8 9.6 103.9 0.4X +DAYOFWEEK of timestamp 1248 1274 24 8.0 124.8 0.3X +DOW of timestamp 1252 1273 25 8.0 125.2 0.3X +ISODOW of timestamp 1195 1204 9 8.4 119.5 0.3X +DOY of timestamp 1081 1086 6 9.3 108.1 0.4X +HOUR of timestamp 778 781 5 12.9 77.8 0.5X +MINUTE of timestamp 779 780 1 12.8 77.9 0.5X +SECOND of timestamp 597 611 20 16.7 59.7 0.6X +MILLISECONDS of timestamp 636 642 6 15.7 63.6 0.6X +MICROSECONDS of timestamp 498 504 5 20.1 49.8 0.8X +EPOCH of timestamp 946 956 9 10.6 94.6 0.4X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1044 1081 63 9.6 104.4 1.0X -MILLENNIUM of date 1367 1375 10 7.3 136.7 0.8X -CENTURY of date 1181 1183 3 8.5 118.1 0.9X -DECADE of date 1154 1163 8 8.7 115.4 0.9X -YEAR of date 1138 1154 16 8.8 113.8 0.9X -ISOYEAR of date 1394 1401 8 7.2 139.4 0.7X -QUARTER of date 1336 1362 26 7.5 133.6 0.8X -MONTH of date 1137 1144 11 8.8 113.7 0.9X -WEEK of date 1619 1632 18 6.2 161.9 0.6X -DAY of date 1144 1158 22 8.7 114.4 0.9X -DAYOFWEEK of date 1328 1330 2 7.5 132.8 0.8X -DOW of date 1336 1339 4 7.5 133.6 0.8X -ISODOW of date 1272 1276 3 7.9 127.2 0.8X -DOY of date 1170 1174 4 8.6 117.0 0.9X -HOUR of date 1963 1986 35 5.1 196.3 0.5X -MINUTE of date 1969 1984 19 5.1 196.9 0.5X -SECOND of date 1926 1929 3 5.2 192.6 0.5X -MILLISECONDS of date 1928 1939 16 5.2 192.8 0.5X -MICROSECONDS of date 1816 1821 4 5.5 181.6 0.6X -EPOCH of date 2331 2336 4 4.3 233.1 0.4X +cast to timestamp 356 362 9 28.1 35.6 1.0X +MILLENNIUM of timestamp 1260 1297 41 7.9 126.0 0.3X +CENTURY of timestamp 1082 1085 3 9.2 108.2 0.3X +DECADE of timestamp 1056 1068 19 9.5 105.6 0.3X +YEAR of timestamp 1045 1053 13 9.6 104.5 0.3X +ISOYEAR of timestamp 1300 1316 25 7.7 130.0 0.3X +QUARTER of timestamp 1279 1280 2 7.8 127.9 0.3X +MONTH of timestamp 1037 1046 11 9.6 103.7 0.3X +WEEK of timestamp 1539 1557 28 6.5 153.9 0.2X +DAY of timestamp 1032 1038 6 9.7 103.2 0.3X +DAYOFWEEK of timestamp 1241 1244 4 8.1 124.1 0.3X +DOW of timestamp 1237 1241 7 8.1 123.7 0.3X +ISODOW of timestamp 1155 1158 3 8.7 115.5 0.3X +DOY of timestamp 1075 1080 4 9.3 107.5 0.3X +HOUR of timestamp 766 770 5 13.1 76.6 0.5X +MINUTE of timestamp 764 769 4 13.1 76.4 0.5X +SECOND of timestamp 590 592 2 16.9 59.0 0.6X +MILLISECONDS of timestamp 627 636 10 16.0 62.7 0.6X +MICROSECONDS of timestamp 493 505 15 20.3 49.3 0.7X +EPOCH of timestamp 962 966 4 10.4 96.2 0.4X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 315 320 8 31.8 31.5 1.0X -MILLENNIUM of timestamp 1365 1369 4 7.3 136.5 0.2X -CENTURY of timestamp 1176 1179 4 8.5 117.6 0.3X -DECADE of timestamp 1153 1168 14 8.7 115.3 0.3X -YEAR of timestamp 1131 1135 5 8.8 113.1 0.3X -ISOYEAR of timestamp 1388 1397 8 7.2 138.8 0.2X -QUARTER of timestamp 1344 1363 25 7.4 134.4 0.2X -MONTH of timestamp 1135 1141 6 8.8 113.5 0.3X -WEEK of timestamp 1612 1615 5 6.2 161.2 0.2X -DAY of timestamp 1138 1145 7 8.8 113.8 0.3X -DAYOFWEEK of timestamp 1322 1330 7 7.6 132.2 0.2X -DOW of timestamp 1317 1323 5 7.6 131.7 0.2X -ISODOW of timestamp 1284 1287 4 7.8 128.4 0.2X -DOY of timestamp 1163 1172 11 8.6 116.3 0.3X -HOUR of timestamp 709 709 1 14.1 70.9 0.4X -MINUTE of timestamp 713 714 1 14.0 71.3 0.4X -SECOND of timestamp 635 641 6 15.7 63.5 0.5X -MILLISECONDS of timestamp 625 629 7 16.0 62.5 0.5X -MICROSECONDS of timestamp 572 573 2 17.5 57.2 0.6X -EPOCH of timestamp 1031 1033 3 9.7 103.1 0.3X +cast to date 886 905 17 11.3 88.6 1.0X +MILLENNIUM of date 1253 1261 7 8.0 125.3 0.7X +CENTURY of date 1068 1079 10 9.4 106.8 0.8X +DECADE of date 1040 1068 25 9.6 104.0 0.9X +YEAR of date 1032 1043 11 9.7 103.2 0.9X +ISOYEAR of date 1304 1313 12 7.7 130.4 0.7X +QUARTER of date 1284 1301 16 7.8 128.4 0.7X +MONTH of date 1033 1036 4 9.7 103.3 0.9X +WEEK of date 1535 1545 8 6.5 153.5 0.6X +DAY of date 1023 1033 11 9.8 102.3 0.9X +DAYOFWEEK of date 1230 1236 6 8.1 123.0 0.7X +DOW of date 1238 1247 9 8.1 123.8 0.7X +ISODOW of date 1159 1169 17 8.6 115.9 0.8X +DOY of date 1082 1084 3 9.2 108.2 0.8X +HOUR of date 1879 1891 11 5.3 187.9 0.5X +MINUTE of date 1881 1905 21 5.3 188.1 0.5X +SECOND of date 1718 1724 5 5.8 171.8 0.5X +MILLISECONDS of date 1733 1737 6 5.8 173.3 0.5X +MICROSECONDS of date 1629 1644 23 6.1 162.9 0.5X +EPOCH of date 2085 2090 5 4.8 208.5 0.4X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 994 998 3 10.1 99.4 1.0X -MILLENNIUM of date 1361 1371 9 7.3 136.1 0.7X -CENTURY of date 1171 1180 15 8.5 117.1 0.8X -DECADE of date 1149 1153 4 8.7 114.9 0.9X -YEAR of date 1128 1130 2 8.9 112.8 0.9X -ISOYEAR of date 1384 1398 19 7.2 138.4 0.7X -QUARTER of date 1333 1345 19 7.5 133.3 0.7X -MONTH of date 1132 1138 7 8.8 113.2 0.9X -WEEK of date 1616 1624 9 6.2 161.6 0.6X -DAY of date 1132 1135 3 8.8 113.2 0.9X -DAYOFWEEK of date 1320 1322 3 7.6 132.0 0.8X -DOW of date 1317 1319 2 7.6 131.7 0.8X -ISODOW of date 1269 1272 5 7.9 126.9 0.8X -DOY of date 1171 1190 17 8.5 117.1 0.8X -HOUR of date 1960 1962 2 5.1 196.0 0.5X -MINUTE of date 1974 1975 1 5.1 197.4 0.5X -SECOND of date 1915 1922 8 5.2 191.5 0.5X -MILLISECONDS of date 1923 1924 2 5.2 192.3 0.5X -MICROSECONDS of date 1817 1819 1 5.5 181.7 0.5X -EPOCH of date 2323 2326 2 4.3 232.3 0.4X +cast to date 888 891 6 11.3 88.8 1.0X +MILLENNIUM of date 1250 1260 17 8.0 125.0 0.7X +CENTURY of date 1070 1076 6 9.3 107.0 0.8X +DECADE of date 1036 1041 5 9.6 103.6 0.9X +YEAR of date 1037 1038 1 9.6 103.7 0.9X +ISOYEAR of date 1300 1307 9 7.7 130.0 0.7X +QUARTER of date 1267 1277 9 7.9 126.7 0.7X +MONTH of date 1034 1037 4 9.7 103.4 0.9X +WEEK of date 1543 1554 10 6.5 154.3 0.6X +DAY of date 1022 1030 12 9.8 102.2 0.9X +DAYOFWEEK of date 1230 1232 4 8.1 123.0 0.7X +DOW of date 1227 1242 15 8.1 122.7 0.7X +ISODOW of date 1157 1173 20 8.6 115.7 0.8X +DOY of date 1073 1083 18 9.3 107.3 0.8X +HOUR of date 1873 1878 7 5.3 187.3 0.5X +MINUTE of date 1861 1876 14 5.4 186.1 0.5X +SECOND of date 1717 1724 6 5.8 171.7 0.5X +MILLISECONDS of date 1729 1736 7 5.8 172.9 0.5X +MICROSECONDS of date 1622 1627 5 6.2 162.2 0.5X +EPOCH of date 2066 2079 19 4.8 206.6 0.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +cast to interval 1273 1279 6 7.9 127.3 1.0X +MILLENNIUM of interval 1323 1352 36 7.6 132.3 1.0X +CENTURY of interval 1353 1356 4 7.4 135.3 0.9X +DECADE of interval 1326 1339 11 7.5 132.6 1.0X +YEAR of interval 1341 1345 3 7.5 134.1 0.9X +QUARTER of interval 1368 1372 4 7.3 136.8 0.9X +MONTH of interval 1320 1326 6 7.6 132.0 1.0X +DAY of interval 1306 1310 4 7.7 130.6 1.0X +HOUR of interval 1341 1347 8 7.5 134.1 0.9X +MINUTE of interval 1337 1349 11 7.5 133.7 1.0X +SECOND of interval 1450 1451 1 6.9 145.0 0.9X +MILLISECONDS of interval 1476 1490 23 6.8 147.6 0.9X +MICROSECONDS of interval 1316 1331 25 7.6 131.6 1.0X +EPOCH of interval 1461 1462 1 6.8 146.1 0.9X diff --git a/sql/core/benchmarks/ExtractBenchmark-results.txt b/sql/core/benchmarks/ExtractBenchmark-results.txt index 89d3616d2c205..317f107bd746c 100644 --- a/sql/core/benchmarks/ExtractBenchmark-results.txt +++ b/sql/core/benchmarks/ExtractBenchmark-results.txt @@ -1,119 +1,119 @@ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke extract for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 404 460 71 24.8 40.4 1.0X -MILLENNIUM of timestamp 1432 1580 128 7.0 143.2 0.3X -CENTURY of timestamp 1380 1390 10 7.2 138.0 0.3X -DECADE of timestamp 1254 1261 8 8.0 125.4 0.3X -YEAR of timestamp 1229 1236 9 8.1 122.9 0.3X -ISOYEAR of timestamp 1382 1415 44 7.2 138.2 0.3X -QUARTER of timestamp 1444 1458 16 6.9 144.4 0.3X -MONTH of timestamp 1212 1257 60 8.2 121.2 0.3X -WEEK of timestamp 1718 1730 12 5.8 171.8 0.2X -DAY of timestamp 1180 1201 22 8.5 118.0 0.3X -DAYOFWEEK of timestamp 1371 1427 68 7.3 137.1 0.3X -DOW of timestamp 1365 1378 18 7.3 136.5 0.3X -ISODOW of timestamp 1312 1319 7 7.6 131.2 0.3X -DOY of timestamp 1224 1238 16 8.2 122.4 0.3X -HOUR of timestamp 343 355 16 29.2 34.3 1.2X -MINUTE of timestamp 341 344 3 29.3 34.1 1.2X -SECOND of timestamp 553 556 2 18.1 55.3 0.7X -MILLISECONDS of timestamp 562 565 3 17.8 56.2 0.7X -MICROSECONDS of timestamp 435 465 49 23.0 43.5 0.9X -EPOCH of timestamp 1211 1217 9 8.3 121.1 0.3X +cast to timestamp 409 457 45 24.5 40.9 1.0X +MILLENNIUM of timestamp 1240 1295 57 8.1 124.0 0.3X +CENTURY of timestamp 1186 1240 49 8.4 118.6 0.3X +DECADE of timestamp 1083 1104 20 9.2 108.3 0.4X +YEAR of timestamp 1061 1073 15 9.4 106.1 0.4X +ISOYEAR of timestamp 1198 1213 25 8.3 119.8 0.3X +QUARTER of timestamp 1304 1322 23 7.7 130.4 0.3X +MONTH of timestamp 1052 1067 19 9.5 105.2 0.4X +WEEK of timestamp 1534 1558 25 6.5 153.4 0.3X +DAY of timestamp 1038 1057 26 9.6 103.8 0.4X +DAYOFWEEK of timestamp 1226 1239 22 8.2 122.6 0.3X +DOW of timestamp 1212 1224 13 8.3 121.2 0.3X +ISODOW of timestamp 1148 1165 24 8.7 114.8 0.4X +DOY of timestamp 1066 1075 14 9.4 106.6 0.4X +HOUR of timestamp 358 362 6 27.9 35.8 1.1X +MINUTE of timestamp 364 369 4 27.4 36.4 1.1X +SECOND of timestamp 453 471 26 22.1 45.3 0.9X +MILLISECONDS of timestamp 497 500 2 20.1 49.7 0.8X +MICROSECONDS of timestamp 360 363 4 27.8 36.0 1.1X +EPOCH of timestamp 1100 1104 5 9.1 110.0 0.4X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1030 1038 8 9.7 103.0 1.0X -MILLENNIUM of date 1302 1306 6 7.7 130.2 0.8X -CENTURY of date 1309 1315 6 7.6 130.9 0.8X -DECADE of date 1197 1202 7 8.4 119.7 0.9X -YEAR of date 1182 1192 10 8.5 118.2 0.9X -ISOYEAR of date 1421 1429 10 7.0 142.1 0.7X -QUARTER of date 1454 1468 14 6.9 145.4 0.7X -MONTH of date 1217 1230 19 8.2 121.7 0.8X -WEEK of date 1723 1725 3 5.8 172.3 0.6X -DAY of date 1188 1191 4 8.4 118.8 0.9X -DAYOFWEEK of date 1348 1355 6 7.4 134.8 0.8X -DOW of date 1344 1353 8 7.4 134.4 0.8X -ISODOW of date 1301 1311 9 7.7 130.1 0.8X -DOY of date 1237 1241 4 8.1 123.7 0.8X -HOUR of date 1465 1474 16 6.8 146.5 0.7X -MINUTE of date 1463 1471 8 6.8 146.3 0.7X -SECOND of date 1720 1725 5 5.8 172.0 0.6X -MILLISECONDS of date 1739 1749 9 5.8 173.9 0.6X -MICROSECONDS of date 1607 1611 4 6.2 160.7 0.6X -EPOCH of date 2370 2377 10 4.2 237.0 0.4X +cast to timestamp 315 317 2 31.8 31.5 1.0X +MILLENNIUM of timestamp 1155 1162 8 8.7 115.5 0.3X +CENTURY of timestamp 1146 1152 5 8.7 114.6 0.3X +DECADE of timestamp 1031 1043 11 9.7 103.1 0.3X +YEAR of timestamp 1033 1041 10 9.7 103.3 0.3X +ISOYEAR of timestamp 1274 1278 5 7.8 127.4 0.2X +QUARTER of timestamp 1326 1346 30 7.5 132.6 0.2X +MONTH of timestamp 1027 1031 7 9.7 102.7 0.3X +WEEK of timestamp 1529 1535 6 6.5 152.9 0.2X +DAY of timestamp 1024 1031 9 9.8 102.4 0.3X +DAYOFWEEK of timestamp 1197 1201 5 8.4 119.7 0.3X +DOW of timestamp 1201 1218 15 8.3 120.1 0.3X +ISODOW of timestamp 1143 1149 6 8.8 114.3 0.3X +DOY of timestamp 1074 1085 9 9.3 107.4 0.3X +HOUR of timestamp 354 354 0 28.3 35.4 0.9X +MINUTE of timestamp 358 361 5 27.9 35.8 0.9X +SECOND of timestamp 445 456 17 22.5 44.5 0.7X +MILLISECONDS of timestamp 504 514 12 19.8 50.4 0.6X +MICROSECONDS of timestamp 361 370 8 27.7 36.1 0.9X +EPOCH of timestamp 1111 1117 9 9.0 111.1 0.3X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Invoke date_part for timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +Invoke extract for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp 307 312 6 32.6 30.7 1.0X -MILLENNIUM of timestamp 1294 1298 4 7.7 129.4 0.2X -CENTURY of timestamp 1317 1321 6 7.6 131.7 0.2X -DECADE of timestamp 1202 1207 8 8.3 120.2 0.3X -YEAR of timestamp 1196 1201 8 8.4 119.6 0.3X -ISOYEAR of timestamp 1431 1440 10 7.0 143.1 0.2X -QUARTER of timestamp 1483 1487 5 6.7 148.3 0.2X -MONTH of timestamp 1205 1209 4 8.3 120.5 0.3X -WEEK of timestamp 1728 1731 4 5.8 172.8 0.2X -DAY of timestamp 1190 1193 3 8.4 119.0 0.3X -DAYOFWEEK of timestamp 1369 1371 2 7.3 136.9 0.2X -DOW of timestamp 1371 1373 3 7.3 137.1 0.2X -ISODOW of timestamp 1316 1321 8 7.6 131.6 0.2X -DOY of timestamp 1236 1241 6 8.1 123.6 0.2X -HOUR of timestamp 342 343 1 29.3 34.2 0.9X -MINUTE of timestamp 338 352 23 29.6 33.8 0.9X -SECOND of timestamp 555 556 1 18.0 55.5 0.6X -MILLISECONDS of timestamp 570 573 4 17.6 57.0 0.5X -MICROSECONDS of timestamp 436 442 5 22.9 43.6 0.7X -EPOCH of timestamp 1212 1218 7 8.3 121.2 0.3X +cast to date 849 851 3 11.8 84.9 1.0X +MILLENNIUM of date 1129 1139 11 8.9 112.9 0.8X +CENTURY of date 1136 1143 7 8.8 113.6 0.7X +DECADE of date 1039 1043 5 9.6 103.9 0.8X +YEAR of date 1030 1037 10 9.7 103.0 0.8X +ISOYEAR of date 1269 1278 9 7.9 126.9 0.7X +QUARTER of date 1323 1330 6 7.6 132.3 0.6X +MONTH of date 1021 1023 2 9.8 102.1 0.8X +WEEK of date 1541 1549 8 6.5 154.1 0.6X +DAY of date 1021 1033 13 9.8 102.1 0.8X +DAYOFWEEK of date 1196 1209 11 8.4 119.6 0.7X +DOW of date 1214 1229 13 8.2 121.4 0.7X +ISODOW of date 1148 1153 7 8.7 114.8 0.7X +DOY of date 1073 1079 5 9.3 107.3 0.8X +HOUR of date 1311 1314 4 7.6 131.1 0.6X +MINUTE of date 1311 1311 1 7.6 131.1 0.6X +SECOND of date 1420 1434 13 7.0 142.0 0.6X +MILLISECONDS of date 1426 1442 14 7.0 142.6 0.6X +MICROSECONDS of date 1312 1318 6 7.6 131.2 0.6X +EPOCH of date 2034 2050 16 4.9 203.4 0.4X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date 1017 1025 13 9.8 101.7 1.0X -MILLENNIUM of date 1314 1318 5 7.6 131.4 0.8X -CENTURY of date 1318 1334 16 7.6 131.8 0.8X -DECADE of date 1205 1210 5 8.3 120.5 0.8X -YEAR of date 1187 1190 3 8.4 118.7 0.9X -ISOYEAR of date 1409 1432 21 7.1 140.9 0.7X -QUARTER of date 1477 1482 6 6.8 147.7 0.7X -MONTH of date 1193 1199 7 8.4 119.3 0.9X -WEEK of date 1711 1719 7 5.8 171.1 0.6X -DAY of date 1183 1192 7 8.5 118.3 0.9X -DAYOFWEEK of date 1368 1371 6 7.3 136.8 0.7X -DOW of date 1361 1366 8 7.3 136.1 0.7X -ISODOW of date 1312 1315 3 7.6 131.2 0.8X -DOY of date 1230 1233 2 8.1 123.0 0.8X -HOUR of date 1480 1483 2 6.8 148.0 0.7X -MINUTE of date 1473 1489 27 6.8 147.3 0.7X -SECOND of date 1731 1737 7 5.8 173.1 0.6X -MILLISECONDS of date 1744 1749 6 5.7 174.4 0.6X -MICROSECONDS of date 1592 1594 1 6.3 159.2 0.6X -EPOCH of date 2368 2371 3 4.2 236.8 0.4X +cast to date 852 879 42 11.7 85.2 1.0X +MILLENNIUM of date 1131 1136 7 8.8 113.1 0.8X +CENTURY of date 1138 1145 6 8.8 113.8 0.7X +DECADE of date 1030 1043 13 9.7 103.0 0.8X +YEAR of date 1022 1028 8 9.8 102.2 0.8X +ISOYEAR of date 1260 1265 6 7.9 126.0 0.7X +QUARTER of date 1326 1330 7 7.5 132.6 0.6X +MONTH of date 1014 1034 26 9.9 101.4 0.8X +WEEK of date 1523 1526 5 6.6 152.3 0.6X +DAY of date 1022 1023 2 9.8 102.2 0.8X +DAYOFWEEK of date 1197 1203 9 8.4 119.7 0.7X +DOW of date 1188 1198 16 8.4 118.8 0.7X +ISODOW of date 1143 1153 9 8.8 114.3 0.7X +DOY of date 1052 1058 7 9.5 105.2 0.8X +HOUR of date 1309 1311 4 7.6 130.9 0.7X +MINUTE of date 1302 1305 6 7.7 130.2 0.7X +SECOND of date 1414 1432 16 7.1 141.4 0.6X +MILLISECONDS of date 1441 1450 11 6.9 144.1 0.6X +MICROSECONDS of date 1292 1301 8 7.7 129.2 0.7X +EPOCH of date 2030 2036 8 4.9 203.0 0.4X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.15 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Invoke date_part for interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to interval 1365 1395 31 7.3 136.5 1.0X -MILLENNIUM of interval 1620 1651 27 6.2 162.0 0.8X -CENTURY of interval 1469 1487 22 6.8 146.9 0.9X -DECADE of interval 1462 1473 17 6.8 146.2 0.9X -YEAR of interval 1438 1447 8 7.0 143.8 0.9X -QUARTER of interval 1456 1458 3 6.9 145.6 0.9X -MONTH of interval 1440 1452 16 6.9 144.0 0.9X -DAY of interval 1478 1485 6 6.8 147.8 0.9X -HOUR of interval 1579 1580 3 6.3 157.9 0.9X -MINUTE of interval 1598 1605 11 6.3 159.8 0.9X -SECOND of interval 1571 1579 10 6.4 157.1 0.9X -MILLISECONDS of interval 1570 1577 6 6.4 157.0 0.9X -MICROSECONDS of interval 1484 1488 5 6.7 148.4 0.9X -EPOCH of interval 1521 1522 1 6.6 152.1 0.9X +cast to interval 1249 1254 6 8.0 124.9 1.0X +MILLENNIUM of interval 1310 1316 9 7.6 131.0 1.0X +CENTURY of interval 1304 1315 10 7.7 130.4 1.0X +DECADE of interval 1306 1313 7 7.7 130.6 1.0X +YEAR of interval 1304 1313 11 7.7 130.4 1.0X +QUARTER of interval 1310 1317 7 7.6 131.0 1.0X +MONTH of interval 1311 1319 12 7.6 131.1 1.0X +DAY of interval 1295 1304 13 7.7 129.5 1.0X +HOUR of interval 1301 1306 8 7.7 130.1 1.0X +MINUTE of interval 1316 1319 3 7.6 131.6 0.9X +SECOND of interval 1437 1440 3 7.0 143.7 0.9X +MILLISECONDS of interval 1435 1449 16 7.0 143.5 0.9X +MICROSECONDS of interval 1304 1314 9 7.7 130.4 1.0X +EPOCH of interval 1440 1453 19 6.9 144.0 0.9X diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-jdk11-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..d292607e2cbcb --- /dev/null +++ b/sql/core/benchmarks/FilterPushdownBenchmark-jdk11-results.txt @@ -0,0 +1,670 @@ +================================================================================================ +Pushdown for many distinct value case +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11943 12023 69 1.3 759.3 1.0X +Parquet Vectorized (Pushdown) 880 934 44 17.9 55.9 13.6X +Native ORC Vectorized 7847 7896 41 2.0 498.9 1.5X +Native ORC Vectorized (Pushdown) 507 525 22 31.0 32.3 23.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 string row ('7864320' < value < '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12020 12040 13 1.3 764.2 1.0X +Parquet Vectorized (Pushdown) 819 840 24 19.2 52.0 14.7X +Native ORC Vectorized 8045 8062 14 2.0 511.5 1.5X +Native ORC Vectorized (Pushdown) 498 535 55 31.6 31.7 24.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row (value = '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12006 12028 21 1.3 763.3 1.0X +Parquet Vectorized (Pushdown) 772 800 25 20.4 49.1 15.6X +Native ORC Vectorized 8074 8091 15 1.9 513.3 1.5X +Native ORC Vectorized (Pushdown) 467 481 10 33.7 29.7 25.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row (value <=> '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11995 12021 30 1.3 762.6 1.0X +Parquet Vectorized (Pushdown) 780 792 10 20.2 49.6 15.4X +Native ORC Vectorized 8046 8066 12 2.0 511.6 1.5X +Native ORC Vectorized (Pushdown) 476 489 13 33.0 30.3 25.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row ('7864320' <= value <= '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11967 11989 12 1.3 760.9 1.0X +Parquet Vectorized (Pushdown) 775 788 11 20.3 49.3 15.4X +Native ORC Vectorized 8028 8046 25 2.0 510.4 1.5X +Native ORC Vectorized (Pushdown) 461 493 29 34.1 29.3 26.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 21435 21513 46 0.7 1362.8 1.0X +Parquet Vectorized (Pushdown) 21710 21742 33 0.7 1380.3 1.0X +Native ORC Vectorized 19324 19373 35 0.8 1228.6 1.1X +Native ORC Vectorized (Pushdown) 19535 19572 40 0.8 1242.0 1.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 int row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11008 11034 21 1.4 699.9 1.0X +Parquet Vectorized (Pushdown) 762 767 6 20.6 48.4 14.4X +Native ORC Vectorized 7156 7186 58 2.2 454.9 1.5X +Native ORC Vectorized (Pushdown) 436 447 9 36.1 27.7 25.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 int row (7864320 < value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10987 11033 38 1.4 698.5 1.0X +Parquet Vectorized (Pushdown) 766 776 11 20.5 48.7 14.4X +Native ORC Vectorized 7160 7192 35 2.2 455.2 1.5X +Native ORC Vectorized (Pushdown) 440 454 26 35.8 27.9 25.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11066 11082 19 1.4 703.6 1.0X +Parquet Vectorized (Pushdown) 749 770 15 21.0 47.6 14.8X +Native ORC Vectorized 7248 7294 57 2.2 460.8 1.5X +Native ORC Vectorized (Pushdown) 437 450 16 36.0 27.8 25.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (value <=> 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11066 11091 32 1.4 703.5 1.0X +Parquet Vectorized (Pushdown) 754 776 28 20.9 47.9 14.7X +Native ORC Vectorized 7257 7265 6 2.2 461.4 1.5X +Native ORC Vectorized (Pushdown) 449 457 7 35.0 28.6 24.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (7864320 <= value <= 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11067 11079 14 1.4 703.6 1.0X +Parquet Vectorized (Pushdown) 753 762 8 20.9 47.9 14.7X +Native ORC Vectorized 7251 7267 22 2.2 461.0 1.5X +Native ORC Vectorized (Pushdown) 427 435 6 36.8 27.2 25.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (7864319 < value < 7864321): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11055 11086 18 1.4 702.8 1.0X +Parquet Vectorized (Pushdown) 748 753 6 21.0 47.5 14.8X +Native ORC Vectorized 7227 7253 27 2.2 459.5 1.5X +Native ORC Vectorized (Pushdown) 429 435 5 36.7 27.3 25.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% int rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12060 12091 21 1.3 766.7 1.0X +Parquet Vectorized (Pushdown) 2799 2821 17 5.6 178.0 4.3X +Native ORC Vectorized 8334 8391 55 1.9 529.9 1.4X +Native ORC Vectorized (Pushdown) 2197 2209 12 7.2 139.7 5.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% int rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 15593 15619 22 1.0 991.3 1.0X +Parquet Vectorized (Pushdown) 10573 10584 10 1.5 672.2 1.5X +Native ORC Vectorized 12276 12290 16 1.3 780.5 1.3X +Native ORC Vectorized (Pushdown) 8931 8958 31 1.8 567.8 1.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% int rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 19094 19120 23 0.8 1214.0 1.0X +Parquet Vectorized (Pushdown) 18327 18342 14 0.9 1165.2 1.0X +Native ORC Vectorized 15890 15943 44 1.0 1010.2 1.2X +Native ORC Vectorized (Pushdown) 15414 15442 24 1.0 980.0 1.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 20010 20037 21 0.8 1272.2 1.0X +Parquet Vectorized (Pushdown) 20297 20318 21 0.8 1290.4 1.0X +Native ORC Vectorized 16994 17029 39 0.9 1080.5 1.2X +Native ORC Vectorized (Pushdown) 17180 17224 53 0.9 1092.3 1.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value > -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 19988 20061 96 0.8 1270.8 1.0X +Parquet Vectorized (Pushdown) 20343 20389 42 0.8 1293.4 1.0X +Native ORC Vectorized 16932 16958 22 0.9 1076.5 1.2X +Native ORC Vectorized (Pushdown) 17308 17351 31 0.9 1100.4 1.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value != -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 20167 20210 36 0.8 1282.2 1.0X +Parquet Vectorized (Pushdown) 20508 20543 44 0.8 1303.9 1.0X +Native ORC Vectorized 17038 17070 32 0.9 1083.3 1.2X +Native ORC Vectorized (Pushdown) 17250 17663 538 0.9 1096.7 1.2X + + +================================================================================================ +Pushdown for few distinct value case (use dictionary encoding) +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 distinct string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10641 10674 57 1.5 676.6 1.0X +Parquet Vectorized (Pushdown) 661 665 7 23.8 42.0 16.1X +Native ORC Vectorized 9194 9227 33 1.7 584.5 1.2X +Native ORC Vectorized (Pushdown) 798 808 18 19.7 50.7 13.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 distinct string row ('100' < value < '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10785 10810 20 1.5 685.7 1.0X +Parquet Vectorized (Pushdown) 656 666 10 24.0 41.7 16.5X +Native ORC Vectorized 9435 9457 16 1.7 599.9 1.1X +Native ORC Vectorized (Pushdown) 796 822 34 19.8 50.6 13.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 distinct string row (value = '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10639 10664 26 1.5 676.4 1.0X +Parquet Vectorized (Pushdown) 734 742 5 21.4 46.7 14.5X +Native ORC Vectorized 9346 9362 17 1.7 594.2 1.1X +Native ORC Vectorized (Pushdown) 863 869 5 18.2 54.9 12.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 distinct string row (value <=> '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10617 10672 59 1.5 675.0 1.0X +Parquet Vectorized (Pushdown) 736 747 8 21.4 46.8 14.4X +Native ORC Vectorized 9345 9445 125 1.7 594.1 1.1X +Native ORC Vectorized (Pushdown) 868 877 7 18.1 55.2 12.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 distinct string row ('100' <= value <= '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10750 10790 35 1.5 683.5 1.0X +Parquet Vectorized (Pushdown) 745 748 6 21.1 47.4 14.4X +Native ORC Vectorized 9471 9488 16 1.7 602.1 1.1X +Native ORC Vectorized (Pushdown) 857 866 6 18.3 54.5 12.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all distinct string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 21173 21253 91 0.7 1346.1 1.0X +Parquet Vectorized (Pushdown) 21369 21456 75 0.7 1358.6 1.0X +Native ORC Vectorized 20282 20397 83 0.8 1289.5 1.0X +Native ORC Vectorized (Pushdown) 20704 20768 48 0.8 1316.3 1.0X + + +================================================================================================ +Pushdown benchmark for StringStartsWith +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +StringStartsWith filter: (value like '10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12426 12474 59 1.3 790.0 1.0X +Parquet Vectorized (Pushdown) 1847 1855 8 8.5 117.4 6.7X +Native ORC Vectorized 8336 8391 68 1.9 530.0 1.5X +Native ORC Vectorized (Pushdown) 8514 8536 14 1.8 541.3 1.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +StringStartsWith filter: (value like '1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12143 12167 23 1.3 772.1 1.0X +Parquet Vectorized (Pushdown) 751 758 9 21.0 47.7 16.2X +Native ORC Vectorized 8064 8069 8 2.0 512.7 1.5X +Native ORC Vectorized (Pushdown) 8226 8254 35 1.9 523.0 1.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +StringStartsWith filter: (value like '786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12123 12142 13 1.3 770.8 1.0X +Parquet Vectorized (Pushdown) 739 743 4 21.3 47.0 16.4X +Native ORC Vectorized 8038 8052 10 2.0 511.0 1.5X +Native ORC Vectorized (Pushdown) 8211 8227 12 1.9 522.0 1.5X + + +================================================================================================ +Pushdown benchmark for decimal +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(9, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5889 5912 23 2.7 374.4 1.0X +Parquet Vectorized (Pushdown) 186 193 7 84.6 11.8 31.7X +Native ORC Vectorized 5272 5291 18 3.0 335.2 1.1X +Native ORC Vectorized (Pushdown) 156 183 36 100.9 9.9 37.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(9, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 7738 7804 111 2.0 492.0 1.0X +Parquet Vectorized (Pushdown) 3172 3188 27 5.0 201.6 2.4X +Native ORC Vectorized 7522 7528 4 2.1 478.2 1.0X +Native ORC Vectorized (Pushdown) 3390 3433 35 4.6 215.5 2.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(9, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 13990 14000 9 1.1 889.4 1.0X +Parquet Vectorized (Pushdown) 13251 13266 21 1.2 842.5 1.1X +Native ORC Vectorized 14799 14817 14 1.1 940.9 0.9X +Native ORC Vectorized (Pushdown) 14149 14195 64 1.1 899.6 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(9, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 15114 15165 44 1.0 960.9 1.0X +Parquet Vectorized (Pushdown) 15182 15222 23 1.0 965.2 1.0X +Native ORC Vectorized 16653 16683 19 0.9 1058.8 0.9X +Native ORC Vectorized (Pushdown) 16730 16743 9 0.9 1063.7 0.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(18, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6112 6127 17 2.6 388.6 1.0X +Parquet Vectorized (Pushdown) 186 190 3 84.7 11.8 32.9X +Native ORC Vectorized 5292 5359 74 3.0 336.4 1.2X +Native ORC Vectorized (Pushdown) 152 159 14 103.3 9.7 40.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(18, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 7119 7158 57 2.2 452.6 1.0X +Parquet Vectorized (Pushdown) 1765 1784 13 8.9 112.2 4.0X +Native ORC Vectorized 6389 6398 11 2.5 406.2 1.1X +Native ORC Vectorized (Pushdown) 1732 1743 13 9.1 110.1 4.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(18, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10996 11025 19 1.4 699.1 1.0X +Parquet Vectorized (Pushdown) 8047 8074 16 2.0 511.6 1.4X +Native ORC Vectorized 10620 10642 23 1.5 675.2 1.0X +Native ORC Vectorized (Pushdown) 8086 8102 14 1.9 514.1 1.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(18, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 14800 14834 28 1.1 941.0 1.0X +Parquet Vectorized (Pushdown) 14261 14268 7 1.1 906.7 1.0X +Native ORC Vectorized 14832 14880 44 1.1 943.0 1.0X +Native ORC Vectorized (Pushdown) 14402 14428 28 1.1 915.7 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(38, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8132 8178 36 1.9 517.0 1.0X +Parquet Vectorized (Pushdown) 201 209 10 78.3 12.8 40.5X +Native ORC Vectorized 5270 5290 16 3.0 335.0 1.5X +Native ORC Vectorized (Pushdown) 149 153 2 105.2 9.5 54.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(38, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9422 9465 50 1.7 599.1 1.0X +Parquet Vectorized (Pushdown) 2271 2283 13 6.9 144.4 4.1X +Native ORC Vectorized 6651 6662 10 2.4 422.9 1.4X +Native ORC Vectorized (Pushdown) 2002 2016 17 7.9 127.3 4.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(38, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 14781 14814 37 1.1 939.7 1.0X +Parquet Vectorized (Pushdown) 10675 10696 24 1.5 678.7 1.4X +Native ORC Vectorized 11887 11983 101 1.3 755.8 1.2X +Native ORC Vectorized (Pushdown) 9383 9396 11 1.7 596.5 1.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(38, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 19753 19777 28 0.8 1255.9 1.0X +Parquet Vectorized (Pushdown) 19049 19089 49 0.8 1211.1 1.0X +Native ORC Vectorized 17238 17247 7 0.9 1096.0 1.1X +Native ORC Vectorized (Pushdown) 16780 16804 21 0.9 1066.9 1.2X + + +================================================================================================ +Pushdown benchmark for InSet -> InFilters +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11101 11144 74 1.4 705.8 1.0X +Parquet Vectorized (Pushdown) 773 776 3 20.3 49.1 14.4X +Native ORC Vectorized 6973 7001 28 2.3 443.3 1.6X +Native ORC Vectorized (Pushdown) 445 451 5 35.4 28.3 25.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11098 11111 13 1.4 705.6 1.0X +Parquet Vectorized (Pushdown) 775 778 4 20.3 49.2 14.3X +Native ORC Vectorized 6980 6996 9 2.3 443.8 1.6X +Native ORC Vectorized (Pushdown) 441 448 8 35.7 28.0 25.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11089 11118 26 1.4 705.0 1.0X +Parquet Vectorized (Pushdown) 772 777 3 20.4 49.1 14.4X +Native ORC Vectorized 6973 6985 9 2.3 443.3 1.6X +Native ORC Vectorized (Pushdown) 446 448 3 35.3 28.3 24.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11113 11137 19 1.4 706.5 1.0X +Parquet Vectorized (Pushdown) 802 807 5 19.6 51.0 13.9X +Native ORC Vectorized 7034 7080 63 2.2 447.2 1.6X +Native ORC Vectorized (Pushdown) 461 469 5 34.1 29.3 24.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11122 11134 16 1.4 707.1 1.0X +Parquet Vectorized (Pushdown) 804 817 16 19.6 51.1 13.8X +Native ORC Vectorized 7027 7037 10 2.2 446.8 1.6X +Native ORC Vectorized (Pushdown) 463 470 5 34.0 29.4 24.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11097 11115 13 1.4 705.5 1.0X +Parquet Vectorized (Pushdown) 804 810 6 19.6 51.1 13.8X +Native ORC Vectorized 7010 7022 15 2.2 445.7 1.6X +Native ORC Vectorized (Pushdown) 465 468 4 33.8 29.6 23.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11378 11387 7 1.4 723.4 1.0X +Parquet Vectorized (Pushdown) 11640 11675 37 1.4 740.1 1.0X +Native ORC Vectorized 7266 7284 24 2.2 462.0 1.6X +Native ORC Vectorized (Pushdown) 612 624 12 25.7 38.9 18.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11411 11448 49 1.4 725.5 1.0X +Parquet Vectorized (Pushdown) 11670 11692 20 1.3 741.9 1.0X +Native ORC Vectorized 7284 7292 10 2.2 463.1 1.6X +Native ORC Vectorized (Pushdown) 652 659 8 24.1 41.4 17.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11399 11447 71 1.4 724.7 1.0X +Parquet Vectorized (Pushdown) 11659 11684 15 1.3 741.3 1.0X +Native ORC Vectorized 7290 7301 10 2.2 463.5 1.6X +Native ORC Vectorized (Pushdown) 655 700 77 24.0 41.7 17.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11398 11424 23 1.4 724.7 1.0X +Parquet Vectorized (Pushdown) 11660 11775 100 1.3 741.3 1.0X +Native ORC Vectorized 7292 7312 24 2.2 463.6 1.6X +Native ORC Vectorized (Pushdown) 789 796 5 19.9 50.2 14.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11399 11462 55 1.4 724.7 1.0X +Parquet Vectorized (Pushdown) 11648 11670 24 1.4 740.6 1.0X +Native ORC Vectorized 7254 7265 8 2.2 461.2 1.6X +Native ORC Vectorized (Pushdown) 851 857 5 18.5 54.1 13.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11383 11499 97 1.4 723.7 1.0X +Parquet Vectorized (Pushdown) 11694 11731 22 1.3 743.5 1.0X +Native ORC Vectorized 7244 7272 22 2.2 460.6 1.6X +Native ORC Vectorized (Pushdown) 887 896 12 17.7 56.4 12.8X + + +================================================================================================ +Pushdown benchmark for tinyint +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 tinyint row (value = CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6301 6337 24 2.5 400.6 1.0X +Parquet Vectorized (Pushdown) 254 260 4 61.9 16.2 24.8X +Native ORC Vectorized 3257 3273 12 4.8 207.1 1.9X +Native ORC Vectorized (Pushdown) 211 217 4 74.4 13.4 29.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 7169 7219 46 2.2 455.8 1.0X +Parquet Vectorized (Pushdown) 1665 1672 12 9.4 105.9 4.3X +Native ORC Vectorized 4199 4208 12 3.7 267.0 1.7X +Native ORC Vectorized (Pushdown) 1376 1394 30 11.4 87.5 5.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10703 10739 29 1.5 680.5 1.0X +Parquet Vectorized (Pushdown) 7648 7671 20 2.1 486.3 1.4X +Native ORC Vectorized 7815 7836 14 2.0 496.9 1.4X +Native ORC Vectorized (Pushdown) 6289 6295 9 2.5 399.8 1.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 14239 14284 35 1.1 905.3 1.0X +Parquet Vectorized (Pushdown) 13733 13769 28 1.1 873.1 1.0X +Native ORC Vectorized 11432 11481 32 1.4 726.8 1.2X +Native ORC Vectorized (Pushdown) 11241 11254 11 1.4 714.7 1.3X + + +================================================================================================ +Pushdown benchmark for Timestamp +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as INT96 row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6805 6823 17 2.3 432.7 1.0X +Parquet Vectorized (Pushdown) 6876 6892 11 2.3 437.2 1.0X +Native ORC Vectorized 3186 3208 33 4.9 202.5 2.1X +Native ORC Vectorized (Pushdown) 131 140 9 120.4 8.3 52.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as INT96 rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 7742 7765 21 2.0 492.2 1.0X +Parquet Vectorized (Pushdown) 7798 7853 70 2.0 495.8 1.0X +Native ORC Vectorized 4128 4138 8 3.8 262.4 1.9X +Native ORC Vectorized (Pushdown) 1392 1402 17 11.3 88.5 5.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as INT96 rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11354 11380 33 1.4 721.9 1.0X +Parquet Vectorized (Pushdown) 11408 11449 36 1.4 725.3 1.0X +Native ORC Vectorized 7752 7780 26 2.0 492.8 1.5X +Native ORC Vectorized (Pushdown) 6233 6246 12 2.5 396.3 1.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as INT96 rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 14944 14997 49 1.1 950.1 1.0X +Parquet Vectorized (Pushdown) 15066 15120 95 1.0 957.9 1.0X +Native ORC Vectorized 11422 11487 45 1.4 726.2 1.3X +Native ORC Vectorized (Pushdown) 11169 11211 30 1.4 710.1 1.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as TIMESTAMP_MICROS row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6047 6083 28 2.6 384.5 1.0X +Parquet Vectorized (Pushdown) 188 192 3 83.5 12.0 32.1X +Native ORC Vectorized 3169 3180 11 5.0 201.5 1.9X +Native ORC Vectorized (Pushdown) 127 138 15 124.0 8.1 47.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6967 7009 26 2.3 443.0 1.0X +Parquet Vectorized (Pushdown) 1705 1711 7 9.2 108.4 4.1X +Native ORC Vectorized 4092 4115 18 3.8 260.2 1.7X +Native ORC Vectorized (Pushdown) 1389 1394 9 11.3 88.3 5.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10658 10695 23 1.5 677.6 1.0X +Parquet Vectorized (Pushdown) 7760 7792 35 2.0 493.4 1.4X +Native ORC Vectorized 7780 7802 24 2.0 494.6 1.4X +Native ORC Vectorized (Pushdown) 6241 6279 24 2.5 396.8 1.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 14209 14253 53 1.1 903.4 1.0X +Parquet Vectorized (Pushdown) 13742 13764 15 1.1 873.7 1.0X +Native ORC Vectorized 11437 11463 18 1.4 727.1 1.2X +Native ORC Vectorized (Pushdown) 11172 11218 43 1.4 710.3 1.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6124 6173 46 2.6 389.4 1.0X +Parquet Vectorized (Pushdown) 187 189 3 84.2 11.9 32.8X +Native ORC Vectorized 3161 3174 13 5.0 200.9 1.9X +Native ORC Vectorized (Pushdown) 129 144 24 122.4 8.2 47.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 7068 7094 23 2.2 449.4 1.0X +Parquet Vectorized (Pushdown) 1724 1740 19 9.1 109.6 4.1X +Native ORC Vectorized 4127 4137 12 3.8 262.4 1.7X +Native ORC Vectorized (Pushdown) 1385 1393 10 11.4 88.1 5.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10767 10786 21 1.5 684.5 1.0X +Parquet Vectorized (Pushdown) 7805 7820 13 2.0 496.2 1.4X +Native ORC Vectorized 7714 7775 64 2.0 490.4 1.4X +Native ORC Vectorized (Pushdown) 6247 6266 17 2.5 397.2 1.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 14337 14392 34 1.1 911.5 1.0X +Parquet Vectorized (Pushdown) 13774 13791 13 1.1 875.7 1.0X +Native ORC Vectorized 11426 11448 16 1.4 726.4 1.3X +Native ORC Vectorized (Pushdown) 11149 11181 27 1.4 708.8 1.3X + + +================================================================================================ +Pushdown benchmark with many filters +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 row with 1 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 509 514 6 0.0 509244147.0 1.0X +Parquet Vectorized (Pushdown) 512 519 6 0.0 511810881.0 1.0X +Native ORC Vectorized 488 495 8 0.0 488404846.0 1.0X +Native ORC Vectorized (Pushdown) 489 500 16 0.0 489064429.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 row with 250 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 1704 1712 7 0.0 1703741457.0 1.0X +Parquet Vectorized (Pushdown) 1782 1794 7 0.0 1781727704.0 1.0X +Native ORC Vectorized 1668 1674 5 0.0 1668306343.0 1.0X +Native ORC Vectorized (Pushdown) 1670 1679 7 0.0 1669592413.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5011 5044 25 0.0 5011180244.0 1.0X +Parquet Vectorized (Pushdown) 5375 5397 17 0.0 5375351637.0 0.9X +Native ORC Vectorized 4979 4992 12 0.0 4979326800.0 1.0X +Native ORC Vectorized (Pushdown) 4979 5008 26 0.0 4978952429.0 1.0X + + diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt index e680ddff53dd1..1fda7bffc8e82 100644 --- a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt +++ b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt @@ -2,669 +2,669 @@ Pushdown for many distinct value case ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 string row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11405 / 11485 1.4 725.1 1.0X -Parquet Vectorized (Pushdown) 675 / 690 23.3 42.9 16.9X -Native ORC Vectorized 7127 / 7170 2.2 453.1 1.6X -Native ORC Vectorized (Pushdown) 519 / 541 30.3 33.0 22.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 string row ('7864320' < value < '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11457 / 11473 1.4 728.4 1.0X -Parquet Vectorized (Pushdown) 656 / 686 24.0 41.7 17.5X -Native ORC Vectorized 7328 / 7342 2.1 465.9 1.6X -Native ORC Vectorized (Pushdown) 539 / 565 29.2 34.2 21.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 string row (value = '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11878 / 11888 1.3 755.2 1.0X -Parquet Vectorized (Pushdown) 630 / 654 25.0 40.1 18.9X -Native ORC Vectorized 7342 / 7362 2.1 466.8 1.6X -Native ORC Vectorized (Pushdown) 519 / 537 30.3 33.0 22.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 string row (value <=> '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11423 / 11440 1.4 726.2 1.0X -Parquet Vectorized (Pushdown) 625 / 643 25.2 39.7 18.3X -Native ORC Vectorized 7315 / 7335 2.2 465.1 1.6X -Native ORC Vectorized (Pushdown) 507 / 520 31.0 32.2 22.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 string row ('7864320' <= value <= '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11440 / 11478 1.4 727.3 1.0X -Parquet Vectorized (Pushdown) 634 / 652 24.8 40.3 18.0X -Native ORC Vectorized 7311 / 7324 2.2 464.8 1.6X -Native ORC Vectorized (Pushdown) 517 / 548 30.4 32.8 22.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all string rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 20750 / 20872 0.8 1319.3 1.0X -Parquet Vectorized (Pushdown) 21002 / 21032 0.7 1335.3 1.0X -Native ORC Vectorized 16714 / 16742 0.9 1062.6 1.2X -Native ORC Vectorized (Pushdown) 16926 / 16965 0.9 1076.1 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 int row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10510 / 10532 1.5 668.2 1.0X -Parquet Vectorized (Pushdown) 642 / 665 24.5 40.8 16.4X -Native ORC Vectorized 6609 / 6618 2.4 420.2 1.6X -Native ORC Vectorized (Pushdown) 502 / 512 31.4 31.9 21.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 int row (7864320 < value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10505 / 10514 1.5 667.9 1.0X -Parquet Vectorized (Pushdown) 659 / 673 23.9 41.9 15.9X -Native ORC Vectorized 6634 / 6641 2.4 421.8 1.6X -Native ORC Vectorized (Pushdown) 513 / 526 30.7 32.6 20.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 int row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10555 / 10570 1.5 671.1 1.0X -Parquet Vectorized (Pushdown) 651 / 668 24.2 41.4 16.2X -Native ORC Vectorized 6721 / 6728 2.3 427.3 1.6X -Native ORC Vectorized (Pushdown) 508 / 519 31.0 32.3 20.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 int row (value <=> 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10556 / 10566 1.5 671.1 1.0X -Parquet Vectorized (Pushdown) 647 / 654 24.3 41.1 16.3X -Native ORC Vectorized 6716 / 6728 2.3 427.0 1.6X -Native ORC Vectorized (Pushdown) 510 / 521 30.9 32.4 20.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 int row (7864320 <= value <= 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10556 / 10565 1.5 671.1 1.0X -Parquet Vectorized (Pushdown) 649 / 654 24.2 41.3 16.3X -Native ORC Vectorized 6700 / 6712 2.3 426.0 1.6X -Native ORC Vectorized (Pushdown) 509 / 520 30.9 32.3 20.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 int row (7864319 < value < 7864321): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10547 / 10566 1.5 670.5 1.0X -Parquet Vectorized (Pushdown) 649 / 653 24.2 41.3 16.3X -Native ORC Vectorized 6703 / 6713 2.3 426.2 1.6X -Native ORC Vectorized (Pushdown) 510 / 520 30.8 32.5 20.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% int rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11478 / 11525 1.4 729.7 1.0X -Parquet Vectorized (Pushdown) 2576 / 2587 6.1 163.8 4.5X -Native ORC Vectorized 7633 / 7657 2.1 485.3 1.5X -Native ORC Vectorized (Pushdown) 2076 / 2096 7.6 132.0 5.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% int rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14785 / 14802 1.1 940.0 1.0X -Parquet Vectorized (Pushdown) 9971 / 9977 1.6 633.9 1.5X -Native ORC Vectorized 11082 / 11107 1.4 704.6 1.3X -Native ORC Vectorized (Pushdown) 8061 / 8073 2.0 512.5 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% int rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 18174 / 18214 0.9 1155.5 1.0X -Parquet Vectorized (Pushdown) 17387 / 17403 0.9 1105.5 1.0X -Native ORC Vectorized 14465 / 14492 1.1 919.7 1.3X -Native ORC Vectorized (Pushdown) 14024 / 14041 1.1 891.6 1.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all int rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 19004 / 19014 0.8 1208.2 1.0X -Parquet Vectorized (Pushdown) 19219 / 19232 0.8 1221.9 1.0X -Native ORC Vectorized 15266 / 15290 1.0 970.6 1.2X -Native ORC Vectorized (Pushdown) 15469 / 15482 1.0 983.5 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all int rows (value > -1): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 19036 / 19052 0.8 1210.3 1.0X -Parquet Vectorized (Pushdown) 19287 / 19306 0.8 1226.2 1.0X -Native ORC Vectorized 15311 / 15371 1.0 973.5 1.2X -Native ORC Vectorized (Pushdown) 15517 / 15590 1.0 986.5 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all int rows (value != -1): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 19072 / 19102 0.8 1212.6 1.0X -Parquet Vectorized (Pushdown) 19288 / 19318 0.8 1226.3 1.0X -Native ORC Vectorized 15277 / 15293 1.0 971.3 1.2X -Native ORC Vectorized (Pushdown) 15479 / 15499 1.0 984.1 1.2X +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9582 9636 49 1.6 609.2 1.0X +Parquet Vectorized (Pushdown) 759 785 40 20.7 48.3 12.6X +Native ORC Vectorized 7141 7183 60 2.2 454.0 1.3X +Native ORC Vectorized (Pushdown) 513 556 49 30.7 32.6 18.7X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 string row ('7864320' < value < '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9657 9675 22 1.6 614.0 1.0X +Parquet Vectorized (Pushdown) 707 724 26 22.2 44.9 13.7X +Native ORC Vectorized 7250 7296 49 2.2 460.9 1.3X +Native ORC Vectorized (Pushdown) 503 537 51 31.3 32.0 19.2X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row (value = '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9636 9657 20 1.6 612.7 1.0X +Parquet Vectorized (Pushdown) 679 693 13 23.2 43.2 14.2X +Native ORC Vectorized 7327 7359 25 2.1 465.8 1.3X +Native ORC Vectorized (Pushdown) 483 512 30 32.6 30.7 20.0X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row (value <=> '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9688 9710 29 1.6 615.9 1.0X +Parquet Vectorized (Pushdown) 674 692 14 23.3 42.8 14.4X +Native ORC Vectorized 7315 7343 22 2.2 465.1 1.3X +Native ORC Vectorized (Pushdown) 483 498 24 32.6 30.7 20.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row ('7864320' <= value <= '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9611 9638 21 1.6 611.1 1.0X +Parquet Vectorized (Pushdown) 670 676 4 23.5 42.6 14.3X +Native ORC Vectorized 7261 7313 31 2.2 461.6 1.3X +Native ORC Vectorized (Pushdown) 480 518 51 32.8 30.5 20.0X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 18597 18738 91 0.8 1182.4 1.0X +Parquet Vectorized (Pushdown) 18864 18915 30 0.8 1199.4 1.0X +Native ORC Vectorized 16865 16968 67 0.9 1072.3 1.1X +Native ORC Vectorized (Pushdown) 17056 17168 120 0.9 1084.4 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 int row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8923 8981 48 1.8 567.3 1.0X +Parquet Vectorized (Pushdown) 651 670 25 24.1 41.4 13.7X +Native ORC Vectorized 6477 6507 46 2.4 411.8 1.4X +Native ORC Vectorized (Pushdown) 453 470 26 34.7 28.8 19.7X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 int row (7864320 < value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8979 8989 12 1.8 570.9 1.0X +Parquet Vectorized (Pushdown) 659 674 15 23.9 41.9 13.6X +Native ORC Vectorized 6502 6522 13 2.4 413.4 1.4X +Native ORC Vectorized (Pushdown) 460 479 22 34.2 29.2 19.5X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9046 9065 23 1.7 575.1 1.0X +Parquet Vectorized (Pushdown) 654 662 9 24.1 41.6 13.8X +Native ORC Vectorized 6592 6617 24 2.4 419.1 1.4X +Native ORC Vectorized (Pushdown) 450 468 20 34.9 28.6 20.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (value <=> 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9036 9052 16 1.7 574.5 1.0X +Parquet Vectorized (Pushdown) 652 661 9 24.1 41.5 13.9X +Native ORC Vectorized 6566 6616 36 2.4 417.4 1.4X +Native ORC Vectorized (Pushdown) 456 493 47 34.5 29.0 19.8X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (7864320 <= value <= 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9024 9062 52 1.7 573.7 1.0X +Parquet Vectorized (Pushdown) 657 662 4 23.9 41.8 13.7X +Native ORC Vectorized 6575 6606 19 2.4 418.1 1.4X +Native ORC Vectorized (Pushdown) 454 464 15 34.6 28.9 19.9X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (7864319 < value < 7864321): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8992 9012 18 1.7 571.7 1.0X +Parquet Vectorized (Pushdown) 651 656 4 24.2 41.4 13.8X +Native ORC Vectorized 6615 6622 5 2.4 420.6 1.4X +Native ORC Vectorized (Pushdown) 455 467 22 34.5 29.0 19.7X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% int rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9928 9972 49 1.6 631.2 1.0X +Parquet Vectorized (Pushdown) 2416 2427 11 6.5 153.6 4.1X +Native ORC Vectorized 7620 7650 21 2.1 484.5 1.3X +Native ORC Vectorized (Pushdown) 2094 2161 119 7.5 133.1 4.7X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% int rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 13110 13154 46 1.2 833.5 1.0X +Parquet Vectorized (Pushdown) 9096 9130 34 1.7 578.3 1.4X +Native ORC Vectorized 11109 11139 21 1.4 706.3 1.2X +Native ORC Vectorized (Pushdown) 8100 8158 40 1.9 515.0 1.6X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% int rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 16372 16424 53 1.0 1040.9 1.0X +Parquet Vectorized (Pushdown) 15745 15794 51 1.0 1001.0 1.0X +Native ORC Vectorized 14642 14681 31 1.1 930.9 1.1X +Native ORC Vectorized (Pushdown) 14209 14239 19 1.1 903.4 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 17032 17078 45 0.9 1082.9 1.0X +Parquet Vectorized (Pushdown) 17495 17509 18 0.9 1112.3 1.0X +Native ORC Vectorized 15626 15720 61 1.0 993.5 1.1X +Native ORC Vectorized (Pushdown) 15871 15950 71 1.0 1009.1 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value > -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 17169 17200 20 0.9 1091.6 1.0X +Parquet Vectorized (Pushdown) 17450 17490 33 0.9 1109.4 1.0X +Native ORC Vectorized 16062 16134 81 1.0 1021.2 1.1X +Native ORC Vectorized (Pushdown) 16225 16281 48 1.0 1031.5 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value != -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 17091 17199 102 0.9 1086.6 1.0X +Parquet Vectorized (Pushdown) 17394 17480 60 0.9 1105.9 1.0X +Native ORC Vectorized 16322 16366 35 1.0 1037.7 1.0X +Native ORC Vectorized (Pushdown) 16459 16543 52 1.0 1046.4 1.0X ================================================================================================ Pushdown for few distinct value case (use dictionary encoding) ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 distinct string row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10250 / 10274 1.5 651.7 1.0X -Parquet Vectorized (Pushdown) 571 / 576 27.5 36.3 17.9X -Native ORC Vectorized 8651 / 8660 1.8 550.0 1.2X -Native ORC Vectorized (Pushdown) 909 / 933 17.3 57.8 11.3X +Select 0 distinct string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8671 8722 42 1.8 551.3 1.0X +Parquet Vectorized (Pushdown) 567 576 9 27.7 36.1 15.3X +Native ORC Vectorized 8567 8600 48 1.8 544.7 1.0X +Native ORC Vectorized (Pushdown) 846 870 30 18.6 53.8 10.2X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 distinct string row ('100' < value < '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10420 / 10426 1.5 662.5 1.0X -Parquet Vectorized (Pushdown) 574 / 579 27.4 36.5 18.2X -Native ORC Vectorized 8973 / 8982 1.8 570.5 1.2X -Native ORC Vectorized (Pushdown) 916 / 955 17.2 58.2 11.4X +Select 0 distinct string row ('100' < value < '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8852 8890 30 1.8 562.8 1.0X +Parquet Vectorized (Pushdown) 563 567 5 27.9 35.8 15.7X +Native ORC Vectorized 8816 8847 25 1.8 560.5 1.0X +Native ORC Vectorized (Pushdown) 838 869 41 18.8 53.3 10.6X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 distinct string row (value = '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10428 / 10441 1.5 663.0 1.0X -Parquet Vectorized (Pushdown) 789 / 809 19.9 50.2 13.2X -Native ORC Vectorized 9042 / 9055 1.7 574.9 1.2X -Native ORC Vectorized (Pushdown) 1130 / 1145 13.9 71.8 9.2X +Select 1 distinct string row (value = '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8764 8799 24 1.8 557.2 1.0X +Parquet Vectorized (Pushdown) 632 637 6 24.9 40.2 13.9X +Native ORC Vectorized 8688 8707 24 1.8 552.4 1.0X +Native ORC Vectorized (Pushdown) 907 1073 259 17.3 57.7 9.7X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 distinct string row (value <=> '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10402 / 10416 1.5 661.3 1.0X -Parquet Vectorized (Pushdown) 791 / 806 19.9 50.3 13.2X -Native ORC Vectorized 9042 / 9055 1.7 574.9 1.2X -Native ORC Vectorized (Pushdown) 1112 / 1145 14.1 70.7 9.4X +Select 1 distinct string row (value <=> '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8789 8802 11 1.8 558.8 1.0X +Parquet Vectorized (Pushdown) 638 646 13 24.6 40.6 13.8X +Native ORC Vectorized 8707 8728 16 1.8 553.6 1.0X +Native ORC Vectorized (Pushdown) 909 931 26 17.3 57.8 9.7X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 distinct string row ('100' <= value <= '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10548 / 10563 1.5 670.6 1.0X -Parquet Vectorized (Pushdown) 790 / 796 19.9 50.2 13.4X -Native ORC Vectorized 9144 / 9153 1.7 581.3 1.2X -Native ORC Vectorized (Pushdown) 1117 / 1148 14.1 71.0 9.4X +Select 1 distinct string row ('100' <= value <= '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8888 8901 11 1.8 565.1 1.0X +Parquet Vectorized (Pushdown) 632 639 5 24.9 40.2 14.1X +Native ORC Vectorized 8835 9125 424 1.8 561.7 1.0X +Native ORC Vectorized (Pushdown) 909 951 53 17.3 57.8 9.8X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all distinct string rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 20445 / 20469 0.8 1299.8 1.0X -Parquet Vectorized (Pushdown) 20686 / 20699 0.8 1315.2 1.0X -Native ORC Vectorized 18851 / 18953 0.8 1198.5 1.1X -Native ORC Vectorized (Pushdown) 19255 / 19268 0.8 1224.2 1.1X +Select all distinct string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 18995 19038 52 0.8 1207.7 1.0X +Parquet Vectorized (Pushdown) 19265 19339 56 0.8 1224.9 1.0X +Native ORC Vectorized 19356 19526 130 0.8 1230.6 1.0X +Native ORC Vectorized (Pushdown) 19683 19762 105 0.8 1251.4 1.0X ================================================================================================ Pushdown benchmark for StringStartsWith ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -StringStartsWith filter: (value like '10%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14265 / 15213 1.1 907.0 1.0X -Parquet Vectorized (Pushdown) 4228 / 4870 3.7 268.8 3.4X -Native ORC Vectorized 10116 / 10977 1.6 643.2 1.4X -Native ORC Vectorized (Pushdown) 10653 / 11376 1.5 677.3 1.3X +StringStartsWith filter: (value like '10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9986 10034 42 1.6 634.9 1.0X +Parquet Vectorized (Pushdown) 1539 1553 20 10.2 97.8 6.5X +Native ORC Vectorized 7520 7560 30 2.1 478.1 1.3X +Native ORC Vectorized (Pushdown) 7717 7764 37 2.0 490.6 1.3X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -StringStartsWith filter: (value like '1000%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11499 / 11539 1.4 731.1 1.0X -Parquet Vectorized (Pushdown) 669 / 672 23.5 42.5 17.2X -Native ORC Vectorized 7343 / 7363 2.1 466.8 1.6X -Native ORC Vectorized (Pushdown) 7559 / 7568 2.1 480.6 1.5X +StringStartsWith filter: (value like '1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9711 9754 31 1.6 617.4 1.0X +Parquet Vectorized (Pushdown) 647 657 12 24.3 41.1 15.0X +Native ORC Vectorized 7257 7309 42 2.2 461.4 1.3X +Native ORC Vectorized (Pushdown) 7424 7497 56 2.1 472.0 1.3X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -StringStartsWith filter: (value like '786432%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11463 / 11468 1.4 728.8 1.0X -Parquet Vectorized (Pushdown) 647 / 651 24.3 41.1 17.7X -Native ORC Vectorized 7322 / 7338 2.1 465.5 1.6X -Native ORC Vectorized (Pushdown) 7533 / 7544 2.1 478.9 1.5X +StringStartsWith filter: (value like '786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9747 9835 94 1.6 619.7 1.0X +Parquet Vectorized (Pushdown) 648 656 8 24.3 41.2 15.0X +Native ORC Vectorized 7233 7262 50 2.2 459.8 1.3X +Native ORC Vectorized (Pushdown) 7463 7539 115 2.1 474.5 1.3X ================================================================================================ Pushdown benchmark for decimal ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 decimal(9, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 5543 / 5564 2.8 352.4 1.0X -Parquet Vectorized (Pushdown) 168 / 174 93.7 10.7 33.0X -Native ORC Vectorized 4992 / 5052 3.2 317.4 1.1X -Native ORC Vectorized (Pushdown) 840 / 850 18.7 53.4 6.6X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% decimal(9, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 7312 / 7358 2.2 464.9 1.0X -Parquet Vectorized (Pushdown) 3008 / 3078 5.2 191.2 2.4X -Native ORC Vectorized 6775 / 6798 2.3 430.7 1.1X -Native ORC Vectorized (Pushdown) 6819 / 6832 2.3 433.5 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% decimal(9, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 13232 / 13241 1.2 841.3 1.0X -Parquet Vectorized (Pushdown) 12555 / 12569 1.3 798.2 1.1X -Native ORC Vectorized 12597 / 12627 1.2 800.9 1.1X -Native ORC Vectorized (Pushdown) 12677 / 12711 1.2 806.0 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% decimal(9, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14725 / 14729 1.1 936.2 1.0X -Parquet Vectorized (Pushdown) 14781 / 14800 1.1 939.7 1.0X -Native ORC Vectorized 15360 / 15453 1.0 976.5 1.0X -Native ORC Vectorized (Pushdown) 15444 / 15466 1.0 981.9 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 decimal(18, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 5746 / 5763 2.7 365.3 1.0X -Parquet Vectorized (Pushdown) 166 / 169 94.8 10.6 34.6X -Native ORC Vectorized 5007 / 5023 3.1 318.3 1.1X -Native ORC Vectorized (Pushdown) 2629 / 2640 6.0 167.1 2.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% decimal(18, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6827 / 6864 2.3 434.0 1.0X -Parquet Vectorized (Pushdown) 1809 / 1827 8.7 115.0 3.8X -Native ORC Vectorized 6287 / 6296 2.5 399.7 1.1X -Native ORC Vectorized (Pushdown) 6364 / 6377 2.5 404.6 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% decimal(18, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11315 / 11342 1.4 719.4 1.0X -Parquet Vectorized (Pushdown) 8431 / 8450 1.9 536.0 1.3X -Native ORC Vectorized 11591 / 11611 1.4 736.9 1.0X -Native ORC Vectorized (Pushdown) 11424 / 11475 1.4 726.3 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% decimal(18, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 15703 / 15712 1.0 998.4 1.0X -Parquet Vectorized (Pushdown) 14982 / 15009 1.0 952.5 1.0X -Native ORC Vectorized 16887 / 16955 0.9 1073.7 0.9X -Native ORC Vectorized (Pushdown) 16518 / 16530 1.0 1050.2 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 decimal(38, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 8101 / 8130 1.9 515.1 1.0X -Parquet Vectorized (Pushdown) 184 / 187 85.6 11.7 44.1X -Native ORC Vectorized 4998 / 5027 3.1 317.8 1.6X -Native ORC Vectorized (Pushdown) 165 / 168 95.6 10.5 49.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% decimal(38, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 9405 / 9447 1.7 597.9 1.0X -Parquet Vectorized (Pushdown) 2269 / 2275 6.9 144.2 4.1X -Native ORC Vectorized 6167 / 6203 2.6 392.1 1.5X -Native ORC Vectorized (Pushdown) 1783 / 1787 8.8 113.3 5.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% decimal(38, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14700 / 14707 1.1 934.6 1.0X -Parquet Vectorized (Pushdown) 10699 / 10712 1.5 680.2 1.4X -Native ORC Vectorized 10687 / 10703 1.5 679.5 1.4X -Native ORC Vectorized (Pushdown) 8364 / 8415 1.9 531.8 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% decimal(38, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 19780 / 19894 0.8 1257.6 1.0X -Parquet Vectorized (Pushdown) 19003 / 19025 0.8 1208.1 1.0X -Native ORC Vectorized 15385 / 15404 1.0 978.2 1.3X -Native ORC Vectorized (Pushdown) 15032 / 15060 1.0 955.7 1.3X +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(9, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4018 4049 32 3.9 255.4 1.0X +Parquet Vectorized (Pushdown) 163 178 19 96.6 10.4 24.7X +Native ORC Vectorized 4918 4969 39 3.2 312.7 0.8X +Native ORC Vectorized (Pushdown) 160 167 11 98.1 10.2 25.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(9, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5800 5837 36 2.7 368.8 1.0X +Parquet Vectorized (Pushdown) 2764 2787 19 5.7 175.7 2.1X +Native ORC Vectorized 7034 7050 13 2.2 447.2 0.8X +Native ORC Vectorized (Pushdown) 3179 3191 9 4.9 202.1 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(9, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12173 12215 33 1.3 773.9 1.0X +Parquet Vectorized (Pushdown) 11669 11723 31 1.3 741.9 1.0X +Native ORC Vectorized 13874 13893 19 1.1 882.1 0.9X +Native ORC Vectorized (Pushdown) 13241 13285 33 1.2 841.8 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(9, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 13290 13316 23 1.2 845.0 1.0X +Parquet Vectorized (Pushdown) 13335 13367 23 1.2 847.8 1.0X +Native ORC Vectorized 15485 15504 18 1.0 984.5 0.9X +Native ORC Vectorized (Pushdown) 15576 15605 18 1.0 990.3 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(18, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4154 4175 32 3.8 264.1 1.0X +Parquet Vectorized (Pushdown) 159 162 7 98.9 10.1 26.1X +Native ORC Vectorized 4902 4926 25 3.2 311.7 0.8X +Native ORC Vectorized (Pushdown) 153 162 23 102.6 9.7 27.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(18, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5090 5114 46 3.1 323.6 1.0X +Parquet Vectorized (Pushdown) 1509 1533 45 10.4 95.9 3.4X +Native ORC Vectorized 5943 5952 11 2.6 377.9 0.9X +Native ORC Vectorized (Pushdown) 1649 1686 61 9.5 104.8 3.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(18, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8887 8938 79 1.8 565.0 1.0X +Parquet Vectorized (Pushdown) 6901 6917 13 2.3 438.8 1.3X +Native ORC Vectorized 10030 10047 15 1.6 637.7 0.9X +Native ORC Vectorized (Pushdown) 7672 7689 12 2.1 487.8 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(18, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12618 12633 19 1.2 802.2 1.0X +Parquet Vectorized (Pushdown) 12296 12315 13 1.3 781.7 1.0X +Native ORC Vectorized 14260 14318 43 1.1 906.6 0.9X +Native ORC Vectorized (Pushdown) 13661 13678 12 1.2 868.5 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(38, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6470 6492 29 2.4 411.4 1.0X +Parquet Vectorized (Pushdown) 175 178 3 89.8 11.1 36.9X +Native ORC Vectorized 4928 4949 17 3.2 313.3 1.3X +Native ORC Vectorized (Pushdown) 153 166 23 103.0 9.7 42.4X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(38, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 7674 7709 60 2.0 487.9 1.0X +Parquet Vectorized (Pushdown) 1984 1994 10 7.9 126.1 3.9X +Native ORC Vectorized 6245 6259 15 2.5 397.1 1.2X +Native ORC Vectorized (Pushdown) 1926 1935 11 8.2 122.5 4.0X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(38, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12485 12497 10 1.3 793.7 1.0X +Parquet Vectorized (Pushdown) 9348 9376 26 1.7 594.3 1.3X +Native ORC Vectorized 11573 11583 12 1.4 735.8 1.1X +Native ORC Vectorized (Pushdown) 9211 9228 10 1.7 585.6 1.4X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(38, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 17198 17206 7 0.9 1093.4 1.0X +Parquet Vectorized (Pushdown) 16613 16629 12 0.9 1056.2 1.0X +Native ORC Vectorized 17081 17103 31 0.9 1086.0 1.0X +Native ORC Vectorized (Pushdown) 16659 16693 28 0.9 1059.2 1.0X ================================================================================================ Pushdown benchmark for InSet -> InFilters ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 5, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10521 / 10534 1.5 668.9 1.0X -Parquet Vectorized (Pushdown) 677 / 691 23.2 43.1 15.5X -Native ORC Vectorized 6768 / 6776 2.3 430.3 1.6X -Native ORC Vectorized (Pushdown) 501 / 512 31.4 31.8 21.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 5, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10531 / 10538 1.5 669.5 1.0X -Parquet Vectorized (Pushdown) 677 / 718 23.2 43.0 15.6X -Native ORC Vectorized 6765 / 6773 2.3 430.1 1.6X -Native ORC Vectorized (Pushdown) 499 / 507 31.5 31.7 21.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 5, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10540 / 10553 1.5 670.1 1.0X -Parquet Vectorized (Pushdown) 678 / 710 23.2 43.1 15.5X -Native ORC Vectorized 6787 / 6794 2.3 431.5 1.6X -Native ORC Vectorized (Pushdown) 501 / 509 31.4 31.9 21.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 10, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10551 / 10559 1.5 670.8 1.0X -Parquet Vectorized (Pushdown) 703 / 708 22.4 44.7 15.0X -Native ORC Vectorized 6791 / 6802 2.3 431.7 1.6X -Native ORC Vectorized (Pushdown) 519 / 526 30.3 33.0 20.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 10, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10561 / 10565 1.5 671.4 1.0X -Parquet Vectorized (Pushdown) 711 / 716 22.1 45.2 14.9X -Native ORC Vectorized 6791 / 6806 2.3 431.8 1.6X -Native ORC Vectorized (Pushdown) 529 / 537 29.8 33.6 20.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 10, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10572 / 10590 1.5 672.1 1.0X -Parquet Vectorized (Pushdown) 713 / 716 22.1 45.3 14.8X -Native ORC Vectorized 6808 / 6815 2.3 432.9 1.6X -Native ORC Vectorized (Pushdown) 530 / 541 29.7 33.7 19.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 50, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10871 / 10882 1.4 691.2 1.0X -Parquet Vectorized (Pushdown) 11104 / 11110 1.4 706.0 1.0X -Native ORC Vectorized 7088 / 7104 2.2 450.7 1.5X -Native ORC Vectorized (Pushdown) 665 / 677 23.6 42.3 16.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 50, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10861 / 10867 1.4 690.5 1.0X -Parquet Vectorized (Pushdown) 11094 / 11099 1.4 705.3 1.0X -Native ORC Vectorized 7075 / 7092 2.2 449.8 1.5X -Native ORC Vectorized (Pushdown) 718 / 733 21.9 45.6 15.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 50, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10868 / 10887 1.4 691.0 1.0X -Parquet Vectorized (Pushdown) 11100 / 11106 1.4 705.7 1.0X -Native ORC Vectorized 7087 / 7093 2.2 450.6 1.5X -Native ORC Vectorized (Pushdown) 712 / 731 22.1 45.3 15.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 100, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10850 / 10888 1.4 689.8 1.0X -Parquet Vectorized (Pushdown) 11086 / 11105 1.4 704.9 1.0X -Native ORC Vectorized 7090 / 7101 2.2 450.8 1.5X -Native ORC Vectorized (Pushdown) 867 / 882 18.1 55.1 12.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 100, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10816 / 10819 1.5 687.7 1.0X -Parquet Vectorized (Pushdown) 11052 / 11059 1.4 702.7 1.0X -Native ORC Vectorized 7037 / 7044 2.2 447.4 1.5X -Native ORC Vectorized (Pushdown) 919 / 931 17.1 58.4 11.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 100, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10807 / 10815 1.5 687.1 1.0X -Parquet Vectorized (Pushdown) 11047 / 11054 1.4 702.4 1.0X -Native ORC Vectorized 7042 / 7047 2.2 447.7 1.5X -Native ORC Vectorized (Pushdown) 950 / 961 16.6 60.4 11.4X +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8986 9024 59 1.8 571.3 1.0X +Parquet Vectorized (Pushdown) 662 676 14 23.7 42.1 13.6X +Native ORC Vectorized 6568 6589 27 2.4 417.6 1.4X +Native ORC Vectorized (Pushdown) 462 475 18 34.0 29.4 19.4X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8973 8986 21 1.8 570.5 1.0X +Parquet Vectorized (Pushdown) 663 666 5 23.7 42.1 13.5X +Native ORC Vectorized 6569 6574 6 2.4 417.6 1.4X +Native ORC Vectorized (Pushdown) 462 507 57 34.0 29.4 19.4X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9003 9037 37 1.7 572.4 1.0X +Parquet Vectorized (Pushdown) 666 671 6 23.6 42.4 13.5X +Native ORC Vectorized 6571 6581 8 2.4 417.8 1.4X +Native ORC Vectorized (Pushdown) 463 473 18 34.0 29.4 19.5X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9015 9022 8 1.7 573.2 1.0X +Parquet Vectorized (Pushdown) 687 698 13 22.9 43.7 13.1X +Native ORC Vectorized 6592 6611 13 2.4 419.1 1.4X +Native ORC Vectorized (Pushdown) 485 497 16 32.5 30.8 18.6X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9001 9008 14 1.7 572.3 1.0X +Parquet Vectorized (Pushdown) 686 692 5 22.9 43.6 13.1X +Native ORC Vectorized 6599 6619 32 2.4 419.6 1.4X +Native ORC Vectorized (Pushdown) 473 483 17 33.2 30.1 19.0X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8996 9002 8 1.7 571.9 1.0X +Parquet Vectorized (Pushdown) 690 697 6 22.8 43.9 13.0X +Native ORC Vectorized 6579 6587 8 2.4 418.3 1.4X +Native ORC Vectorized (Pushdown) 481 490 15 32.7 30.6 18.7X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9292 9304 9 1.7 590.8 1.0X +Parquet Vectorized (Pushdown) 9558 9567 9 1.6 607.7 1.0X +Native ORC Vectorized 6898 6911 10 2.3 438.6 1.3X +Native ORC Vectorized (Pushdown) 625 641 15 25.2 39.8 14.9X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9281 9298 12 1.7 590.1 1.0X +Parquet Vectorized (Pushdown) 9546 9561 17 1.6 606.9 1.0X +Native ORC Vectorized 6877 6897 18 2.3 437.2 1.3X +Native ORC Vectorized (Pushdown) 661 668 15 23.8 42.0 14.0X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9322 9335 22 1.7 592.7 1.0X +Parquet Vectorized (Pushdown) 9551 9573 18 1.6 607.2 1.0X +Native ORC Vectorized 6902 6915 13 2.3 438.8 1.4X +Native ORC Vectorized (Pushdown) 659 680 25 23.9 41.9 14.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9278 9294 18 1.7 589.9 1.0X +Parquet Vectorized (Pushdown) 9520 9560 27 1.7 605.3 1.0X +Native ORC Vectorized 6855 6870 16 2.3 435.9 1.4X +Native ORC Vectorized (Pushdown) 795 808 16 19.8 50.5 11.7X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9306 9311 4 1.7 591.6 1.0X +Parquet Vectorized (Pushdown) 9529 9551 16 1.7 605.8 1.0X +Native ORC Vectorized 6875 6882 7 2.3 437.1 1.4X +Native ORC Vectorized (Pushdown) 853 865 15 18.4 54.2 10.9X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9256 9271 9 1.7 588.5 1.0X +Parquet Vectorized (Pushdown) 9500 9520 13 1.7 604.0 1.0X +Native ORC Vectorized 6843 6857 9 2.3 435.1 1.4X +Native ORC Vectorized (Pushdown) 858 870 14 18.3 54.6 10.8X ================================================================================================ Pushdown benchmark for tinyint ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 tinyint row (value = CAST(63 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6034 / 6048 2.6 383.6 1.0X -Parquet Vectorized (Pushdown) 333 / 344 47.2 21.2 18.1X -Native ORC Vectorized 3240 / 3307 4.9 206.0 1.9X -Native ORC Vectorized (Pushdown) 330 / 341 47.6 21.0 18.3X +Select 1 tinyint row (value = CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4303 4312 8 3.7 273.6 1.0X +Parquet Vectorized (Pushdown) 208 213 6 75.5 13.2 20.7X +Native ORC Vectorized 2950 2958 9 5.3 187.6 1.5X +Native ORC Vectorized (Pushdown) 207 212 10 76.1 13.1 20.8X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6759 / 6800 2.3 429.7 1.0X -Parquet Vectorized (Pushdown) 1533 / 1537 10.3 97.5 4.4X -Native ORC Vectorized 3863 / 3874 4.1 245.6 1.7X -Native ORC Vectorized (Pushdown) 1235 / 1248 12.7 78.5 5.5X +Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5125 5162 57 3.1 325.9 1.0X +Parquet Vectorized (Pushdown) 1402 1419 16 11.2 89.2 3.7X +Native ORC Vectorized 3840 3849 8 4.1 244.1 1.3X +Native ORC Vectorized (Pushdown) 1302 1306 3 12.1 82.8 3.9X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10247 / 10289 1.5 651.5 1.0X -Parquet Vectorized (Pushdown) 7430 / 7453 2.1 472.4 1.4X -Native ORC Vectorized 6995 / 7009 2.2 444.7 1.5X -Native ORC Vectorized (Pushdown) 5561 / 5571 2.8 353.6 1.8X +Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8562 8577 12 1.8 544.4 1.0X +Parquet Vectorized (Pushdown) 6539 6564 22 2.4 415.8 1.3X +Native ORC Vectorized 7300 7320 13 2.2 464.1 1.2X +Native ORC Vectorized (Pushdown) 5944 5954 15 2.6 377.9 1.4X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 13949 / 13991 1.1 886.9 1.0X -Parquet Vectorized (Pushdown) 13486 / 13511 1.2 857.4 1.0X -Native ORC Vectorized 10149 / 10186 1.5 645.3 1.4X -Native ORC Vectorized (Pushdown) 9889 / 9905 1.6 628.7 1.4X +Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11989 12005 13 1.3 762.2 1.0X +Parquet Vectorized (Pushdown) 11637 11686 54 1.4 739.8 1.0X +Native ORC Vectorized 10795 10816 18 1.5 686.3 1.1X +Native ORC Vectorized (Pushdown) 10592 10599 5 1.5 673.4 1.1X ================================================================================================ Pushdown benchmark for Timestamp ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 timestamp stored as INT96 row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6307 / 6310 2.5 401.0 1.0X -Parquet Vectorized (Pushdown) 6360 / 6397 2.5 404.3 1.0X -Native ORC Vectorized 2912 / 2917 5.4 185.1 2.2X -Native ORC Vectorized (Pushdown) 138 / 141 114.4 8.7 45.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% timestamp stored as INT96 rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 7225 / 7233 2.2 459.4 1.0X -Parquet Vectorized (Pushdown) 7250 / 7255 2.2 461.0 1.0X -Native ORC Vectorized 3772 / 3783 4.2 239.8 1.9X -Native ORC Vectorized (Pushdown) 1277 / 1282 12.3 81.2 5.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% timestamp stored as INT96 rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10952 / 10965 1.4 696.3 1.0X -Parquet Vectorized (Pushdown) 10985 / 10998 1.4 698.4 1.0X -Native ORC Vectorized 7178 / 7227 2.2 456.3 1.5X -Native ORC Vectorized (Pushdown) 5825 / 5830 2.7 370.3 1.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% timestamp stored as INT96 rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14560 / 14583 1.1 925.7 1.0X -Parquet Vectorized (Pushdown) 14608 / 14620 1.1 928.7 1.0X -Native ORC Vectorized 10601 / 10640 1.5 674.0 1.4X -Native ORC Vectorized (Pushdown) 10392 / 10406 1.5 660.7 1.4X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 timestamp stored as TIMESTAMP_MICROS row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 5653 / 5658 2.8 359.4 1.0X -Parquet Vectorized (Pushdown) 165 / 169 95.1 10.5 34.2X -Native ORC Vectorized 2918 / 2921 5.4 185.5 1.9X -Native ORC Vectorized (Pushdown) 137 / 145 114.9 8.7 41.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6540 / 6552 2.4 415.8 1.0X -Parquet Vectorized (Pushdown) 1610 / 1614 9.8 102.3 4.1X -Native ORC Vectorized 3775 / 3788 4.2 240.0 1.7X -Native ORC Vectorized (Pushdown) 1274 / 1277 12.3 81.0 5.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10259 / 10278 1.5 652.3 1.0X -Parquet Vectorized (Pushdown) 7591 / 7601 2.1 482.6 1.4X -Native ORC Vectorized 7185 / 7194 2.2 456.8 1.4X -Native ORC Vectorized (Pushdown) 5828 / 5843 2.7 370.6 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 13850 / 13868 1.1 880.5 1.0X -Parquet Vectorized (Pushdown) 13433 / 13450 1.2 854.0 1.0X -Native ORC Vectorized 10635 / 10669 1.5 676.1 1.3X -Native ORC Vectorized (Pushdown) 10437 / 10448 1.5 663.6 1.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 5884 / 5888 2.7 374.1 1.0X -Parquet Vectorized (Pushdown) 166 / 170 94.7 10.6 35.4X -Native ORC Vectorized 2913 / 2916 5.4 185.2 2.0X -Native ORC Vectorized (Pushdown) 136 / 144 115.4 8.7 43.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6763 / 6776 2.3 430.0 1.0X -Parquet Vectorized (Pushdown) 1634 / 1638 9.6 103.9 4.1X -Native ORC Vectorized 3777 / 3785 4.2 240.1 1.8X -Native ORC Vectorized (Pushdown) 1276 / 1279 12.3 81.2 5.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10460 / 10469 1.5 665.0 1.0X -Parquet Vectorized (Pushdown) 7689 / 7698 2.0 488.9 1.4X -Native ORC Vectorized 7190 / 7197 2.2 457.1 1.5X -Native ORC Vectorized (Pushdown) 5820 / 5834 2.7 370.0 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14033 / 14039 1.1 892.2 1.0X -Parquet Vectorized (Pushdown) 13608 / 13636 1.2 865.2 1.0X -Native ORC Vectorized 10635 / 10686 1.5 676.2 1.3X -Native ORC Vectorized (Pushdown) 10420 / 10442 1.5 662.5 1.3X +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as INT96 row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4700 4716 19 3.3 298.8 1.0X +Parquet Vectorized (Pushdown) 4745 4758 11 3.3 301.7 1.0X +Native ORC Vectorized 2848 2865 22 5.5 181.1 1.7X +Native ORC Vectorized (Pushdown) 129 135 12 122.2 8.2 36.5X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as INT96 rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5575 5590 9 2.8 354.5 1.0X +Parquet Vectorized (Pushdown) 5610 5623 10 2.8 356.7 1.0X +Native ORC Vectorized 3706 3723 11 4.2 235.6 1.5X +Native ORC Vectorized (Pushdown) 1286 1287 2 12.2 81.7 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as INT96 rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 9872 9887 13 1.6 627.7 1.0X +Parquet Vectorized (Pushdown) 9932 9942 8 1.6 631.4 1.0X +Native ORC Vectorized 7238 7273 25 2.2 460.2 1.4X +Native ORC Vectorized (Pushdown) 5933 5937 6 2.7 377.2 1.7X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as INT96 rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12660 12677 12 1.2 804.9 1.0X +Parquet Vectorized (Pushdown) 12652 12704 31 1.2 804.4 1.0X +Native ORC Vectorized 10842 10870 19 1.5 689.3 1.2X +Native ORC Vectorized (Pushdown) 10630 10639 6 1.5 675.9 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as TIMESTAMP_MICROS row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4057 4065 7 3.9 257.9 1.0X +Parquet Vectorized (Pushdown) 155 159 7 101.2 9.9 26.1X +Native ORC Vectorized 2828 2840 8 5.6 179.8 1.4X +Native ORC Vectorized (Pushdown) 126 130 9 124.8 8.0 32.2X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4926 4938 9 3.2 313.2 1.0X +Parquet Vectorized (Pushdown) 1415 1422 6 11.1 90.0 3.5X +Native ORC Vectorized 3705 3714 7 4.2 235.6 1.3X +Native ORC Vectorized (Pushdown) 1279 1285 9 12.3 81.3 3.9X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8409 8413 4 1.9 534.6 1.0X +Parquet Vectorized (Pushdown) 6489 6497 6 2.4 412.5 1.3X +Native ORC Vectorized 7248 7255 10 2.2 460.8 1.2X +Native ORC Vectorized (Pushdown) 5922 5932 7 2.7 376.5 1.4X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11821 11830 8 1.3 751.5 1.0X +Parquet Vectorized (Pushdown) 11478 11490 11 1.4 729.8 1.0X +Native ORC Vectorized 10851 10856 4 1.4 689.9 1.1X +Native ORC Vectorized (Pushdown) 10620 10628 10 1.5 675.2 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4307 4341 75 3.7 273.8 1.0X +Parquet Vectorized (Pushdown) 156 161 6 101.1 9.9 27.7X +Native ORC Vectorized 2825 2836 8 5.6 179.6 1.5X +Native ORC Vectorized (Pushdown) 125 128 9 125.4 8.0 34.3X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5173 5178 3 3.0 328.9 1.0X +Parquet Vectorized (Pushdown) 1444 1451 9 10.9 91.8 3.6X +Native ORC Vectorized 3707 3715 5 4.2 235.7 1.4X +Native ORC Vectorized (Pushdown) 1275 1276 2 12.3 81.0 4.1X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8645 8655 9 1.8 549.6 1.0X +Parquet Vectorized (Pushdown) 6588 6603 11 2.4 418.9 1.3X +Native ORC Vectorized 7233 7246 13 2.2 459.9 1.2X +Native ORC Vectorized (Pushdown) 5904 5912 8 2.7 375.3 1.5X + +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12057 12069 12 1.3 766.5 1.0X +Parquet Vectorized (Pushdown) 11694 11699 5 1.3 743.5 1.0X +Native ORC Vectorized 10817 10843 28 1.5 687.7 1.1X +Native ORC Vectorized (Pushdown) 10615 10634 11 1.5 674.9 1.1X ================================================================================================ Pushdown benchmark with many filters ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 row with 1 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 319 / 323 0.0 318789986.0 1.0X -Parquet Vectorized (Pushdown) 323 / 347 0.0 322755287.0 1.0X -Native ORC Vectorized 316 / 336 0.0 315670745.0 1.0X -Native ORC Vectorized (Pushdown) 317 / 320 0.0 317392594.0 1.0X +Select 1 row with 1 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 463 468 6 0.0 462711905.0 1.0X +Parquet Vectorized (Pushdown) 468 470 3 0.0 468018831.0 1.0X +Native ORC Vectorized 447 450 5 0.0 446698170.0 1.0X +Native ORC Vectorized (Pushdown) 449 458 11 0.0 448997785.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 row with 250 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 2192 / 2218 0.0 2191883823.0 1.0X -Parquet Vectorized (Pushdown) 2675 / 2687 0.0 2675439029.0 0.8X -Native ORC Vectorized 2158 / 2162 0.0 2157646071.0 1.0X -Native ORC Vectorized (Pushdown) 2309 / 2326 0.0 2309096612.0 0.9X +Select 1 row with 250 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 1538 1549 9 0.0 1538463215.0 1.0X +Parquet Vectorized (Pushdown) 1659 1668 6 0.0 1659315980.0 0.9X +Native ORC Vectorized 1513 1517 5 0.0 1512577059.0 1.0X +Native ORC Vectorized (Pushdown) 1517 1538 12 0.0 1516938695.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 row with 500 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6219 / 6248 0.0 6218727737.0 1.0X -Parquet Vectorized (Pushdown) 7376 / 7436 0.0 7375977710.0 0.8X -Native ORC Vectorized 6252 / 6279 0.0 6252473320.0 1.0X -Native ORC Vectorized (Pushdown) 6858 / 6876 0.0 6857854486.0 0.9X +Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4497 4525 29 0.0 4497410600.0 1.0X +Parquet Vectorized (Pushdown) 4945 4955 8 0.0 4945493883.0 0.9X +Native ORC Vectorized 4466 4485 23 0.0 4466103057.0 1.0X +Native ORC Vectorized (Pushdown) 4477 4496 18 0.0 4476752574.0 1.0X diff --git a/sql/core/benchmarks/InExpressionBenchmark-jdk11-results.txt b/sql/core/benchmarks/InExpressionBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..04474566ef43d --- /dev/null +++ b/sql/core/benchmarks/InExpressionBenchmark-jdk11-results.txt @@ -0,0 +1,740 @@ +================================================================================================ +In Expression Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 163 213 53 61.4 16.3 1.0X +InSet expression 105 130 16 95.5 10.5 1.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 112 126 9 89.4 11.2 1.0X +InSet expression 78 92 11 128.4 7.8 1.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 170 181 8 58.9 17.0 1.0X +InSet expression 109 118 9 91.8 10.9 1.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 279 283 3 35.8 27.9 1.0X +InSet expression 140 148 8 71.6 14.0 2.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 461 486 28 21.7 46.1 1.0X +InSet expression 184 187 3 54.5 18.4 2.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 735 754 16 13.6 73.5 1.0X +InSet expression 317 323 5 31.6 31.7 2.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 63 68 5 157.8 6.3 1.0X +InSet expression 56 61 4 177.7 5.6 1.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 84 87 2 118.8 8.4 1.0X +InSet expression 58 62 4 171.6 5.8 1.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 139 146 10 72.0 13.9 1.0X +InSet expression 58 63 7 173.5 5.8 2.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 231 236 7 43.3 23.1 1.0X +InSet expression 59 64 8 170.5 5.9 3.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 411 414 3 24.3 41.1 1.0X +InSet expression 64 72 8 155.4 6.4 6.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 775 779 6 12.9 77.5 1.0X +InSet expression 70 74 4 142.7 7.0 11.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1129 1133 3 8.9 112.9 1.0X +InSet expression 86 91 6 116.9 8.6 13.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1574 1577 3 6.4 157.4 1.0X +InSet expression 89 92 4 112.9 8.9 17.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1865 1914 100 5.4 186.5 1.0X +InSet expression 275 279 4 36.3 27.5 6.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 63 66 4 158.5 6.3 1.0X +InSet expression 62 68 14 160.7 6.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 81 87 10 124.0 8.1 1.0X +InSet expression 70 82 15 142.4 7.0 1.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 142 145 5 70.4 14.2 1.0X +InSet expression 78 80 3 128.4 7.8 1.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 229 235 14 43.8 22.9 1.0X +InSet expression 91 94 4 109.5 9.1 2.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 409 411 2 24.4 40.9 1.0X +InSet expression 108 112 5 92.9 10.8 3.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 772 779 8 12.9 77.2 1.0X +InSet expression 126 131 4 79.6 12.6 6.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1138 1144 6 8.8 113.8 1.0X +InSet expression 136 140 6 73.5 13.6 8.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1500 1504 3 6.7 150.0 1.0X +InSet expression 148 154 6 67.7 14.8 10.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1932 1969 72 5.2 193.2 1.0X +InSet expression 275 278 3 36.3 27.5 7.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 57 59 2 174.1 5.7 1.0X +InSet expression 53 57 5 187.1 5.3 1.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 76 79 7 131.9 7.6 1.0X +InSet expression 54 57 8 186.1 5.4 1.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 128 132 3 77.9 12.8 1.0X +InSet expression 55 58 4 183.2 5.5 2.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 227 237 11 44.1 22.7 1.0X +InSet expression 56 58 2 178.0 5.6 4.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 407 413 9 24.5 40.7 1.0X +InSet expression 59 64 7 169.0 5.9 6.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 772 775 4 13.0 77.2 1.0X +InSet expression 67 70 5 148.7 6.7 11.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1128 1132 3 8.9 112.8 1.0X +InSet expression 75 77 2 133.6 7.5 15.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1502 1507 6 6.7 150.2 1.0X +InSet expression 82 84 2 121.6 8.2 18.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1860 1897 75 5.4 186.0 1.0X +InSet expression 246 249 2 40.7 24.6 7.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 46 48 3 219.3 4.6 1.0X +InSet expression 40 41 2 251.6 4.0 1.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 65 67 2 154.5 6.5 1.0X +InSet expression 45 47 2 221.2 4.5 1.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 121 124 3 82.5 12.1 1.0X +InSet expression 46 49 4 217.2 4.6 2.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 214 216 2 46.7 21.4 1.0X +InSet expression 57 59 3 175.3 5.7 3.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 400 406 6 25.0 40.0 1.0X +InSet expression 58 61 3 173.9 5.8 7.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 762 765 2 13.1 76.2 1.0X +InSet expression 69 71 3 145.5 6.9 11.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1116 1122 4 9.0 111.6 1.0X +InSet expression 84 87 2 118.5 8.4 13.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1491 1498 5 6.7 149.1 1.0X +InSet expression 88 91 2 113.4 8.8 16.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1858 1909 109 5.4 185.8 1.0X +InSet expression 247 249 2 40.5 24.7 7.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 53 54 2 190.2 5.3 1.0X +InSet expression 156 162 13 64.3 15.6 0.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 67 69 3 149.9 6.7 1.0X +InSet expression 171 174 2 58.5 17.1 0.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 120 122 4 83.0 12.0 1.0X +InSet expression 178 181 1 56.2 17.8 0.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 212 213 2 47.2 21.2 1.0X +InSet expression 222 224 3 45.0 22.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 393 398 4 25.5 39.3 1.0X +InSet expression 193 197 2 51.8 19.3 2.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 756 763 7 13.2 75.6 1.0X +InSet expression 186 188 2 53.8 18.6 4.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 92 94 4 109.1 9.2 1.0X +InSet expression 205 208 2 48.8 20.5 0.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 140 141 2 71.6 14.0 1.0X +InSet expression 232 235 2 43.0 23.2 0.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 247 250 5 40.5 24.7 1.0X +InSet expression 240 242 3 41.6 24.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 426 429 5 23.5 42.6 1.0X +InSet expression 290 293 4 34.5 29.0 1.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 785 787 2 12.7 78.5 1.0X +InSet expression 250 252 2 40.0 25.0 3.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 3101 3190 192 3.2 310.1 1.0X +InSet expression 251 252 1 39.8 25.1 12.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 89 92 4 112.1 8.9 1.0X +InSet expression 171 175 2 58.4 17.1 0.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 135 136 2 73.9 13.5 1.0X +InSet expression 191 194 1 52.4 19.1 0.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 243 245 2 41.2 24.3 1.0X +InSet expression 199 201 1 50.4 19.9 1.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 420 423 2 23.8 42.0 1.0X +InSet expression 242 244 2 41.3 24.2 1.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 774 776 2 12.9 77.4 1.0X +InSet expression 210 216 5 47.7 21.0 3.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 3627 3744 248 2.8 362.7 1.0X +InSet expression 202 206 2 49.4 20.2 17.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 52 55 4 19.2 52.1 1.0X +InSet expression 158 172 24 6.3 158.0 0.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 63 64 2 16.0 62.6 1.0X +InSet expression 160 164 4 6.2 160.1 0.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 97 99 2 10.3 97.2 1.0X +InSet expression 163 166 2 6.1 163.2 0.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 184 186 2 5.4 184.4 1.0X +InSet expression 173 176 2 5.8 173.0 1.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 472 478 9 2.1 472.4 1.0X +InSet expression 178 181 4 5.6 177.6 2.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1054 1184 282 0.9 1053.8 1.0X +InSet expression 188 191 1 5.3 187.9 5.6X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 275 282 8 3.6 274.7 1.0X +InSet expression 195 199 8 5.1 194.7 1.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 458 470 8 2.2 458.4 1.0X +InSet expression 195 200 5 5.1 195.5 2.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1093 1099 6 0.9 1093.3 1.0X +InSet expression 202 204 1 4.9 202.2 5.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 2091 2101 12 0.5 2090.6 1.0X +InSet expression 207 210 2 4.8 206.7 10.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 4164 4187 16 0.2 4164.1 1.0X +InSet expression 239 244 4 4.2 239.1 17.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 8331 8353 21 0.1 8330.7 1.0X +InSet expression 251 252 1 4.0 250.7 33.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 66 67 2 15.2 65.8 1.0X +InSet expression 81 84 6 12.3 81.0 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 70 72 2 14.3 70.1 1.0X +InSet expression 84 86 2 12.0 83.6 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 89 92 3 11.2 89.3 1.0X +InSet expression 92 94 2 10.9 92.2 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 123 125 2 8.1 123.0 1.0X +InSet expression 95 97 2 10.6 94.7 1.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 221 223 2 4.5 220.7 1.0X +InSet expression 92 94 3 10.9 91.7 2.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 753 903 327 1.3 752.9 1.0X +InSet expression 97 100 3 10.3 97.2 7.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 48 49 2 209.3 4.8 1.0X +InSet expression 178 181 2 56.1 17.8 0.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 65 67 4 154.9 6.5 1.0X +InSet expression 196 198 1 50.9 19.6 0.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 118 121 6 84.5 11.8 1.0X +InSet expression 259 261 1 38.6 25.9 0.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 199 201 1 50.2 19.9 1.0X +InSet expression 280 282 3 35.7 28.0 0.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 368 370 2 27.1 36.8 1.0X +InSet expression 252 254 2 39.7 25.2 1.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 710 715 3 14.1 71.0 1.0X +InSet expression 251 255 6 39.8 25.1 2.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 699 701 2 14.3 69.9 1.0X +InSet expression 695 695 1 14.4 69.5 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 700 703 2 14.3 70.0 1.0X +InSet expression 700 701 1 14.3 70.0 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 742 746 3 13.5 74.2 1.0X +InSet expression 704 706 1 14.2 70.4 1.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 839 840 2 11.9 83.9 1.0X +InSet expression 710 713 2 14.1 71.0 1.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1027 1030 2 9.7 102.7 1.0X +InSet expression 712 714 1 14.0 71.2 1.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1390 1411 43 7.2 139.0 1.0X +InSet expression 727 730 3 13.8 72.7 1.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1767 1774 8 5.7 176.7 1.0X +InSet expression 739 741 3 13.5 73.9 2.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 2144 2155 12 4.7 214.4 1.0X +InSet expression 758 766 7 13.2 75.8 2.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 2528 2552 22 4.0 252.8 1.0X +InSet expression 865 868 2 11.6 86.5 2.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 58 61 3 17.2 58.3 1.0X +InSet expression 113 115 2 8.9 112.7 0.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 86 89 3 11.7 85.8 1.0X +InSet expression 113 116 2 8.8 113.3 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 278 280 2 3.6 278.4 1.0X +InSet expression 141 145 5 7.1 140.6 2.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 577 704 75 1.7 576.9 1.0X +InSet expression 193 196 3 5.2 193.5 3.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 2462 2643 375 0.4 2461.6 1.0X +InSet expression 234 238 6 4.3 234.1 10.5X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 8526 9225 487 0.1 8526.5 1.0X +InSet expression 306 312 5 3.3 306.0 27.9X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 51 55 7 19.5 51.3 1.0X +InSet expression 169 172 2 5.9 168.8 0.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 62 66 6 16.2 61.6 1.0X +InSet expression 170 175 7 5.9 169.9 0.4X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 141 145 3 7.1 141.4 1.0X +InSet expression 213 220 8 4.7 212.7 0.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 335 365 67 3.0 334.9 1.0X +InSet expression 303 309 7 3.3 303.4 1.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 976 1114 286 1.0 976.4 1.0X +InSet expression 358 365 10 2.8 357.5 2.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 4162 4703 859 0.2 4162.5 1.0X +InSet expression 473 476 3 2.1 473.2 8.8X + + diff --git a/sql/core/benchmarks/InExpressionBenchmark-results.txt b/sql/core/benchmarks/InExpressionBenchmark-results.txt index f6685bfc45089..36d38ea9b162b 100644 --- a/sql/core/benchmarks/InExpressionBenchmark-results.txt +++ b/sql/core/benchmarks/InExpressionBenchmark-results.txt @@ -2,739 +2,739 @@ In Expression Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 105 148 42 94.8 10.5 1.0X -InSet expression 79 98 19 126.9 7.9 1.3X +In expression 132 172 32 75.7 13.2 1.0X +InSet expression 79 98 13 125.8 7.9 1.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 101 115 20 99.3 10.1 1.0X -InSet expression 76 84 8 131.4 7.6 1.3X +In expression 100 111 7 99.7 10.0 1.0X +InSet expression 70 78 9 143.0 7.0 1.4X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 172 176 3 58.0 17.2 1.0X -InSet expression 100 107 9 99.6 10.0 1.7X +In expression 161 170 7 62.0 16.1 1.0X +InSet expression 88 93 7 113.9 8.8 1.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 299 302 4 33.5 29.9 1.0X -InSet expression 145 149 5 69.0 14.5 2.1X +In expression 270 277 6 37.0 27.0 1.0X +InSet expression 116 123 9 86.0 11.6 2.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 518 524 11 19.3 51.8 1.0X -InSet expression 240 250 12 41.6 24.0 2.2X +In expression 450 463 13 22.2 45.0 1.0X +InSet expression 182 189 7 54.9 18.2 2.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 831 844 14 12.0 83.1 1.0X -InSet expression 425 432 4 23.5 42.5 2.0X +In expression 706 714 7 14.2 70.6 1.0X +InSet expression 302 311 7 33.1 30.2 2.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 58 62 5 171.9 5.8 1.0X -InSet expression 56 58 5 178.0 5.6 1.0X +In expression 62 64 5 162.2 6.2 1.0X +InSet expression 57 59 5 176.7 5.7 1.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 76 79 5 131.9 7.6 1.0X -InSet expression 50 55 7 198.2 5.0 1.5X +In expression 79 82 5 126.7 7.9 1.0X +InSet expression 53 54 3 188.7 5.3 1.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 129 139 23 77.3 12.9 1.0X -InSet expression 48 50 5 210.5 4.8 2.7X +In expression 132 135 5 75.8 13.2 1.0X +InSet expression 50 53 6 200.7 5.0 2.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 225 226 0 44.4 22.5 1.0X -InSet expression 52 56 7 191.2 5.2 4.3X +In expression 228 231 4 43.8 22.8 1.0X +InSet expression 51 53 3 195.5 5.1 4.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 400 406 11 25.0 40.0 1.0X -InSet expression 54 58 7 185.0 5.4 7.4X +In expression 404 404 0 24.8 40.4 1.0X +InSet expression 58 61 4 171.0 5.8 6.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 761 762 1 13.1 76.1 1.0X -InSet expression 60 61 2 167.1 6.0 12.7X +In expression 766 767 1 13.1 76.6 1.0X +InSet expression 66 68 3 151.0 6.6 11.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 300 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1118 1119 1 8.9 111.8 1.0X -InSet expression 66 67 2 152.2 6.6 17.0X +In expression 1124 1124 0 8.9 112.4 1.0X +InSet expression 74 77 4 135.3 7.4 15.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 400 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1478 1487 19 6.8 147.8 1.0X -InSet expression 71 75 11 141.7 7.1 20.9X +In expression 1566 1567 2 6.4 156.6 1.0X +InSet expression 82 84 4 121.9 8.2 19.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 500 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1836 1854 27 5.4 183.6 1.0X -InSet expression 248 253 3 40.2 24.8 7.4X +In expression 1841 1871 37 5.4 184.1 1.0X +InSet expression 314 318 5 31.8 31.4 5.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 55 68 19 180.3 5.5 1.0X -InSet expression 60 63 7 167.0 6.0 0.9X +In expression 58 62 5 171.0 5.8 1.0X +InSet expression 53 55 4 187.5 5.3 1.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 72 76 5 138.0 7.2 1.0X -InSet expression 63 68 11 157.7 6.3 1.1X +In expression 76 77 2 131.3 7.6 1.0X +InSet expression 65 66 3 154.3 6.5 1.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 133 136 5 75.0 13.3 1.0X -InSet expression 73 78 10 137.2 7.3 1.8X +In expression 138 140 5 72.7 13.8 1.0X +InSet expression 74 78 8 135.2 7.4 1.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 223 225 2 44.8 22.3 1.0X -InSet expression 81 84 14 124.1 8.1 2.8X +In expression 226 227 1 44.2 22.6 1.0X +InSet expression 83 86 7 120.8 8.3 2.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 402 404 1 24.9 40.2 1.0X -InSet expression 90 91 2 111.6 9.0 4.5X +In expression 402 403 0 24.9 40.2 1.0X +InSet expression 93 94 3 108.0 9.3 4.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 758 759 0 13.2 75.8 1.0X -InSet expression 110 119 20 91.0 11.0 6.9X +In expression 761 762 0 13.1 76.1 1.0X +InSet expression 113 116 7 88.4 11.3 6.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 300 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1121 1123 3 8.9 112.1 1.0X -InSet expression 121 122 2 82.6 12.1 9.3X +In expression 1125 1125 0 8.9 112.5 1.0X +InSet expression 136 142 11 73.5 13.6 8.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 400 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1482 1484 2 6.7 148.2 1.0X -InSet expression 134 135 2 74.6 13.4 11.1X +In expression 1486 1487 1 6.7 148.6 1.0X +InSet expression 141 142 2 70.8 14.1 10.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 500 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1838 1882 92 5.4 183.8 1.0X -InSet expression 251 254 3 39.8 25.1 7.3X +In expression 1842 1873 67 5.4 184.2 1.0X +InSet expression 315 318 3 31.7 31.5 5.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 51 52 2 197.1 5.1 1.0X -InSet expression 61 63 3 162.8 6.1 0.8X +In expression 54 55 4 186.6 5.4 1.0X +InSet expression 49 51 3 203.0 4.9 1.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 69 73 10 145.0 6.9 1.0X -InSet expression 43 46 7 231.2 4.3 1.6X +In expression 72 74 5 139.7 7.2 1.0X +InSet expression 46 48 5 218.2 4.6 1.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 123 129 19 81.4 12.3 1.0X -InSet expression 43 46 8 230.0 4.3 2.8X +In expression 125 127 5 79.9 12.5 1.0X +InSet expression 47 48 4 212.5 4.7 2.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 222 223 1 45.1 22.2 1.0X -InSet expression 49 50 2 206.2 4.9 4.6X +In expression 221 224 4 45.2 22.1 1.0X +InSet expression 48 49 3 206.3 4.8 4.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 401 402 0 24.9 40.1 1.0X -InSet expression 51 56 11 196.6 5.1 7.9X +In expression 401 404 6 25.0 40.1 1.0X +InSet expression 55 56 2 180.5 5.5 7.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 755 756 1 13.2 75.5 1.0X -InSet expression 56 57 2 179.5 5.6 13.5X +In expression 762 763 1 13.1 76.2 1.0X +InSet expression 63 69 14 159.8 6.3 12.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 300 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1115 1116 1 9.0 111.5 1.0X -InSet expression 61 62 4 165.2 6.1 18.4X +In expression 1117 1117 0 9.0 111.7 1.0X +InSet expression 70 71 2 143.3 7.0 16.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 400 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1476 1478 1 6.8 147.6 1.0X -InSet expression 66 67 2 152.2 6.6 22.5X +In expression 1557 1558 1 6.4 155.7 1.0X +InSet expression 77 78 2 129.6 7.7 20.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 500 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1834 1873 85 5.5 183.4 1.0X -InSet expression 230 233 3 43.5 23.0 8.0X +In expression 1841 1877 79 5.4 184.1 1.0X +InSet expression 320 322 2 31.2 32.0 5.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 40 42 2 247.6 4.0 1.0X -InSet expression 37 39 3 271.6 3.7 1.1X +In expression 43 44 3 231.6 4.3 1.0X +InSet expression 40 42 4 252.4 4.0 1.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 59 60 3 170.0 5.9 1.0X -InSet expression 42 44 3 237.6 4.2 1.4X +In expression 62 64 4 162.0 6.2 1.0X +InSet expression 45 47 4 222.2 4.5 1.4X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 114 116 6 87.5 11.4 1.0X -InSet expression 53 58 10 188.0 5.3 2.1X +In expression 117 119 4 85.4 11.7 1.0X +InSet expression 57 59 5 176.9 5.7 2.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 207 214 14 48.3 20.7 1.0X -InSet expression 62 63 2 162.1 6.2 3.4X +In expression 207 207 0 48.3 20.7 1.0X +InSet expression 65 66 3 153.3 6.5 3.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 385 391 6 26.0 38.5 1.0X -InSet expression 71 73 2 140.4 7.1 5.4X +In expression 390 394 4 25.7 39.0 1.0X +InSet expression 76 77 3 132.0 7.6 5.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 746 750 2 13.4 74.6 1.0X -InSet expression 101 105 8 98.5 10.1 7.4X +In expression 752 755 2 13.3 75.2 1.0X +InSet expression 111 112 3 90.2 11.1 6.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 300 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1100 1106 4 9.1 110.0 1.0X -InSet expression 109 111 2 91.6 10.9 10.1X +In expression 1106 1115 6 9.0 110.6 1.0X +InSet expression 129 130 3 77.7 12.9 8.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 400 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1470 1480 7 6.8 147.0 1.0X -InSet expression 115 116 2 87.1 11.5 12.8X +In expression 1476 1484 6 6.8 147.6 1.0X +InSet expression 129 130 3 77.3 12.9 11.4X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 500 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1838 1907 152 5.4 183.8 1.0X -InSet expression 231 233 2 43.3 23.1 8.0X +In expression 1844 1900 124 5.4 184.4 1.0X +InSet expression 321 322 2 31.2 32.1 5.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 48 52 6 206.5 4.8 1.0X -InSet expression 150 152 4 66.8 15.0 0.3X +In expression 50 53 5 199.5 5.0 1.0X +InSet expression 166 169 5 60.2 16.6 0.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 62 63 1 161.3 6.2 1.0X -InSet expression 165 168 5 60.7 16.5 0.4X +In expression 64 68 8 155.5 6.4 1.0X +InSet expression 186 188 4 53.9 18.6 0.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 116 116 0 86.1 11.6 1.0X -InSet expression 173 175 3 57.9 17.3 0.7X +In expression 118 119 2 84.7 11.8 1.0X +InSet expression 194 208 26 51.4 19.4 0.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 206 208 6 48.6 20.6 1.0X -InSet expression 212 214 2 47.1 21.2 1.0X +In expression 208 208 0 48.1 20.8 1.0X +InSet expression 240 244 5 41.7 24.0 0.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 384 386 2 26.0 38.4 1.0X -InSet expression 183 185 2 54.6 18.3 2.1X +In expression 390 397 10 25.6 39.0 1.0X +InSet expression 205 207 5 48.8 20.5 1.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 742 748 13 13.5 74.2 1.0X -InSet expression 175 177 2 57.1 17.5 4.2X +In expression 745 745 1 13.4 74.5 1.0X +InSet expression 194 197 5 51.5 19.4 3.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 88 89 1 114.2 8.8 1.0X -InSet expression 168 170 2 59.5 16.8 0.5X +In expression 90 91 1 111.2 9.0 1.0X +InSet expression 199 202 4 50.2 19.9 0.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 128 129 3 78.0 12.8 1.0X -InSet expression 187 188 2 53.6 18.7 0.7X +In expression 131 132 0 76.4 13.1 1.0X +InSet expression 221 223 2 45.2 22.1 0.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 242 242 0 41.3 24.2 1.0X -InSet expression 192 194 2 52.0 19.2 1.3X +In expression 244 245 0 40.9 24.4 1.0X +InSet expression 235 236 1 42.6 23.5 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 419 420 0 23.8 41.9 1.0X -InSet expression 235 236 1 42.5 23.5 1.8X +In expression 423 424 2 23.7 42.3 1.0X +InSet expression 284 285 2 35.3 28.4 1.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 774 775 1 12.9 77.4 1.0X -InSet expression 205 206 3 48.9 20.5 3.8X +In expression 777 778 1 12.9 77.7 1.0X +InSet expression 248 249 2 40.4 24.8 3.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 3036 3123 191 3.3 303.6 1.0X -InSet expression 197 198 1 50.8 19.7 15.4X +In expression 3032 3125 203 3.3 303.2 1.0X +InSet expression 239 241 2 41.8 23.9 12.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 83 84 2 120.9 8.3 1.0X -InSet expression 167 168 2 60.0 16.7 0.5X +In expression 84 86 3 118.8 8.4 1.0X +InSet expression 167 168 2 59.9 16.7 0.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 125 126 3 80.3 12.5 1.0X -InSet expression 186 188 2 53.7 18.6 0.7X +In expression 126 127 1 79.1 12.6 1.0X +InSet expression 183 185 2 54.6 18.3 0.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 237 238 1 42.1 23.7 1.0X -InSet expression 192 195 3 52.0 19.2 1.2X +In expression 239 240 0 41.8 23.9 1.0X +InSet expression 189 192 4 52.9 18.9 1.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 414 415 0 24.1 41.4 1.0X -InSet expression 239 242 3 41.9 23.9 1.7X +In expression 417 417 0 24.0 41.7 1.0X +InSet expression 231 234 4 43.3 23.1 1.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 769 771 3 13.0 76.9 1.0X -InSet expression 203 213 22 49.3 20.3 3.8X +In expression 770 772 3 13.0 77.0 1.0X +InSet expression 201 204 5 49.7 20.1 3.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 3757 3796 85 2.7 375.7 1.0X -InSet expression 193 194 2 51.9 19.3 19.5X +In expression 3587 3686 212 2.8 358.7 1.0X +InSet expression 196 198 3 50.9 19.6 18.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 47 48 3 21.3 47.0 1.0X -InSet expression 155 168 29 6.4 155.3 0.3X +In expression 50 51 2 20.1 49.7 1.0X +InSet expression 151 153 3 6.6 150.7 0.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 58 59 2 17.4 57.6 1.0X -InSet expression 157 160 2 6.4 157.4 0.4X +In expression 62 62 2 16.2 61.6 1.0X +InSet expression 153 155 2 6.5 153.3 0.4X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 92 92 2 10.9 91.5 1.0X -InSet expression 160 162 2 6.3 159.6 0.6X +In expression 95 96 2 10.5 95.0 1.0X +InSet expression 156 158 2 6.4 156.4 0.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 171 171 0 5.9 170.8 1.0X -InSet expression 169 172 3 5.9 169.3 1.0X +In expression 175 177 5 5.7 175.2 1.0X +InSet expression 165 167 3 6.1 164.9 1.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 429 429 0 2.3 428.6 1.0X -InSet expression 170 172 2 5.9 170.4 2.5X +In expression 440 440 0 2.3 439.7 1.0X +InSet expression 167 169 2 6.0 167.0 2.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 996 1144 328 1.0 996.3 1.0X -InSet expression 177 179 3 5.7 176.8 5.6X +In expression 1005 1154 330 1.0 1004.8 1.0X +InSet expression 180 182 2 5.5 180.2 5.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 248 251 2 4.0 248.0 1.0X -InSet expression 175 177 2 5.7 174.9 1.4X +In expression 251 258 7 4.0 251.1 1.0X +InSet expression 193 197 3 5.2 193.2 1.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 420 426 11 2.4 420.0 1.0X -InSet expression 177 180 3 5.7 176.9 2.4X +In expression 424 437 8 2.4 424.2 1.0X +InSet expression 196 198 2 5.1 195.5 2.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1005 1008 4 1.0 1004.9 1.0X -InSet expression 184 187 3 5.4 183.7 5.5X +In expression 1018 1023 5 1.0 1017.8 1.0X +InSet expression 203 206 3 4.9 202.9 5.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1922 1933 13 0.5 1922.2 1.0X -InSet expression 189 193 7 5.3 188.9 10.2X +In expression 1947 1955 7 0.5 1947.5 1.0X +InSet expression 208 211 3 4.8 208.4 9.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 3861 3871 12 0.3 3860.5 1.0X -InSet expression 213 225 30 4.7 213.5 18.1X +In expression 3886 3899 10 0.3 3885.9 1.0X +InSet expression 233 235 4 4.3 232.6 16.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 7731 7774 25 0.1 7731.5 1.0X -InSet expression 222 225 3 4.5 222.4 34.8X +In expression 7702 7793 60 0.1 7701.8 1.0X +InSet expression 243 248 6 4.1 243.4 31.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 127 133 9 7.9 126.8 1.0X -InSet expression 142 143 2 7.0 141.9 0.9X +In expression 132 135 4 7.6 132.1 1.0X +InSet expression 147 149 2 6.8 147.1 0.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 132 133 2 7.6 131.7 1.0X -InSet expression 144 146 2 6.9 144.1 0.9X +In expression 139 141 5 7.2 139.0 1.0X +InSet expression 150 151 2 6.7 149.6 0.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 153 154 2 6.5 152.9 1.0X -InSet expression 151 153 2 6.6 151.2 1.0X +In expression 160 161 2 6.3 159.6 1.0X +InSet expression 157 158 2 6.4 157.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 186 187 2 5.4 185.8 1.0X -InSet expression 154 156 3 6.5 153.7 1.2X +In expression 190 192 2 5.3 189.8 1.0X +InSet expression 160 161 2 6.2 160.5 1.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 260 263 3 3.8 260.3 1.0X -InSet expression 151 153 2 6.6 151.3 1.7X +In expression 264 265 2 3.8 263.8 1.0X +InSet expression 159 160 2 6.3 158.5 1.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 731 891 352 1.4 731.4 1.0X -InSet expression 155 157 3 6.4 155.4 4.7X +In expression 735 884 327 1.4 735.4 1.0X +InSet expression 164 166 3 6.1 163.9 4.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 42 43 2 240.1 4.2 1.0X -InSet expression 159 160 2 63.0 15.9 0.3X +In expression 45 46 3 223.6 4.5 1.0X +InSet expression 162 164 3 61.6 16.2 0.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 58 59 2 171.4 5.8 1.0X -InSet expression 174 183 21 57.5 17.4 0.3X +In expression 62 65 7 161.9 6.2 1.0X +InSet expression 179 181 2 56.0 17.9 0.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 111 113 3 90.0 11.1 1.0X -InSet expression 228 229 2 43.9 22.8 0.5X +In expression 115 116 1 86.8 11.5 1.0X +InSet expression 222 225 6 45.1 22.2 0.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 192 193 1 52.1 19.2 1.0X -InSet expression 250 250 1 40.1 25.0 0.8X +In expression 198 199 1 50.6 19.8 1.0X +InSet expression 238 239 2 42.0 23.8 0.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 373 384 12 26.8 37.3 1.0X -InSet expression 229 236 7 43.7 22.9 1.6X +In expression 363 368 5 27.6 36.3 1.0X +InSet expression 222 224 2 45.0 22.2 1.6X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 694 707 25 14.4 69.4 1.0X -InSet expression 221 226 7 45.2 22.1 3.1X +In expression 701 706 4 14.3 70.1 1.0X +InSet expression 226 228 3 44.2 22.6 3.1X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 196 198 2 50.9 19.6 1.0X -InSet expression 169 170 0 59.2 16.9 1.2X +In expression 659 665 5 15.2 65.9 1.0X +InSet expression 660 664 3 15.1 66.0 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 212 212 0 47.3 21.2 1.0X -InSet expression 197 197 0 50.8 19.7 1.1X +In expression 663 667 2 15.1 66.3 1.0X +InSet expression 659 661 2 15.2 65.9 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 266 266 1 37.7 26.6 1.0X -InSet expression 203 217 23 49.4 20.3 1.3X +In expression 700 705 4 14.3 70.0 1.0X +InSet expression 667 669 2 15.0 66.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 356 367 12 28.1 35.6 1.0X -InSet expression 212 213 1 47.1 21.2 1.7X +In expression 801 805 3 12.5 80.1 1.0X +InSet expression 675 677 1 14.8 67.5 1.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 559 573 26 17.9 55.9 1.0X -InSet expression 221 223 2 45.2 22.1 2.5X +In expression 984 987 5 10.2 98.4 1.0X +InSet expression 685 690 4 14.6 68.5 1.4X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 901 916 9 11.1 90.1 1.0X -InSet expression 238 241 9 42.1 23.8 3.8X +In expression 1350 1356 4 7.4 135.0 1.0X +InSet expression 710 712 3 14.1 71.0 1.9X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 300 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1264 1282 10 7.9 126.4 1.0X -InSet expression 253 262 15 39.5 25.3 5.0X +In expression 1716 1722 6 5.8 171.6 1.0X +InSet expression 716 719 2 14.0 71.6 2.4X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 400 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1628 1646 11 6.1 162.8 1.0X -InSet expression 264 265 1 37.8 26.4 6.2X +In expression 2083 2100 13 4.8 208.3 1.0X +InSet expression 742 744 1 13.5 74.2 2.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 500 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1993 2015 15 5.0 199.3 1.0X -InSet expression 355 368 10 28.2 35.5 5.6X +In expression 2469 2486 11 4.1 246.9 1.0X +InSet expression 829 831 2 12.1 82.9 3.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 52 63 14 19.3 51.8 1.0X -InSet expression 96 98 2 10.4 95.9 0.5X +In expression 56 58 4 17.9 55.9 1.0X +InSet expression 123 124 2 8.1 123.1 0.5X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 78 80 3 12.8 77.9 1.0X -InSet expression 97 154 48 10.3 97.1 0.8X +In expression 83 84 2 12.1 82.7 1.0X +InSet expression 124 127 7 8.1 123.8 0.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 269 279 27 3.7 268.7 1.0X -InSet expression 120 124 13 8.3 119.9 2.2X +In expression 275 281 9 3.6 275.5 1.0X +InSet expression 155 158 4 6.4 155.2 1.8X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 579 626 96 1.7 579.2 1.0X -InSet expression 165 167 3 6.1 165.1 3.5X +In expression 659 718 109 1.5 659.4 1.0X +InSet expression 217 218 2 4.6 217.0 3.0X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 2582 2775 415 0.4 2582.1 1.0X -InSet expression 196 201 10 5.1 196.0 13.2X +In expression 2488 2684 434 0.4 2488.4 1.0X +InSet expression 267 270 3 3.7 266.7 9.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 9438 9939 763 0.1 9437.9 1.0X -InSet expression 256 258 3 3.9 255.8 36.9X +In expression 9462 10091 897 0.1 9462.2 1.0X +InSet expression 347 349 2 2.9 347.4 27.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 5 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 47 48 2 21.4 46.8 1.0X -InSet expression 158 160 2 6.3 157.6 0.3X +In expression 51 55 7 19.7 50.8 1.0X +InSet expression 166 168 4 6.0 166.3 0.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 10 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 62 63 4 16.2 61.9 1.0X -InSet expression 158 161 4 6.3 158.4 0.4X +In expression 66 70 5 15.1 66.4 1.0X +InSet expression 167 170 3 6.0 167.3 0.4X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 25 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 138 140 3 7.3 137.9 1.0X -InSet expression 202 219 43 5.0 201.7 0.7X +In expression 142 144 3 7.0 142.1 1.0X +InSet expression 211 215 6 4.7 211.2 0.7X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 50 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 366 367 1 2.7 365.7 1.0X -InSet expression 286 289 4 3.5 285.6 1.3X +In expression 375 378 5 2.7 375.5 1.0X +InSet expression 297 298 2 3.4 297.5 1.3X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 100 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 1055 1212 346 0.9 1054.7 1.0X -InSet expression 348 354 6 2.9 347.9 3.0X +In expression 1122 1260 304 0.9 1122.2 1.0X +InSet expression 354 357 4 2.8 353.9 3.2X -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz 200 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -In expression 6463 6772 650 0.2 6463.3 1.0X -InSet expression 450 455 4 2.2 449.6 14.4X +In expression 5651 5989 644 0.2 5651.4 1.0X +InSet expression 471 473 2 2.1 471.0 12.0X diff --git a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt index 07dd8d5e44ea7..7d23e5467baed 100644 --- a/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-jdk11-results.txt @@ -1,29 +1,29 @@ -Java HotSpot(TM) 64-Bit Server VM 11.0.5+10-LTS on Mac OS X 10.14.6 -Intel(R) Core(TM) i5-5287U CPU @ 2.90GHz +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare string w/ interval 574 610 45 1.7 573.9 1.0X -prepare string w/o interval 518 538 27 1.9 517.7 1.1X -1 units w/ interval 425 439 16 2.4 425.3 1.3X -1 units w/o interval 385 393 10 2.6 385.2 1.5X -2 units w/ interval 553 561 11 1.8 553.1 1.0X -2 units w/o interval 531 543 11 1.9 531.0 1.1X -3 units w/ interval 1134 1159 32 0.9 1134.0 0.5X -3 units w/o interval 1121 1126 6 0.9 1121.3 0.5X -4 units w/ interval 1226 1250 21 0.8 1226.1 0.5X -4 units w/o interval 1227 1239 11 0.8 1227.1 0.5X -5 units w/ interval 1375 1447 93 0.7 1374.7 0.4X -5 units w/o interval 1335 1346 19 0.7 1335.1 0.4X -6 units w/ interval 1530 1556 24 0.7 1529.5 0.4X -6 units w/o interval 1481 1492 17 0.7 1480.7 0.4X -7 units w/ interval 1730 1745 14 0.6 1729.9 0.3X -7 units w/o interval 1788 1859 112 0.6 1788.1 0.3X -8 units w/ interval 1952 2087 117 0.5 1951.7 0.3X -8 units w/o interval 2083 2207 209 0.5 2082.5 0.3X -9 units w/ interval 2228 2291 60 0.4 2227.5 0.3X -9 units w/o interval 2130 2184 75 0.5 2130.1 0.3X -10 units w/ interval 2414 2502 81 0.4 2413.8 0.2X -10 units w/o interval 2463 2488 35 0.4 2463.1 0.2X -11 units w/ interval 2717 2755 42 0.4 2716.8 0.2X -11 units w/o interval 2578 2661 77 0.4 2577.7 0.2X +prepare string w/ interval 756 831 78 1.3 756.4 1.0X +prepare string w/o interval 622 637 15 1.6 622.4 1.2X +1 units w/ interval 506 532 22 2.0 506.2 1.5X +1 units w/o interval 472 477 9 2.1 471.6 1.6X +2 units w/ interval 699 710 11 1.4 698.5 1.1X +2 units w/o interval 670 674 6 1.5 669.9 1.1X +3 units w/ interval 1431 1437 7 0.7 1431.3 0.5X +3 units w/o interval 1418 1429 11 0.7 1418.2 0.5X +4 units w/ interval 1687 1692 8 0.6 1686.8 0.4X +4 units w/o interval 1679 1688 7 0.6 1679.4 0.5X +5 units w/ interval 1862 1864 3 0.5 1861.9 0.4X +5 units w/o interval 1847 1864 15 0.5 1846.9 0.4X +6 units w/ interval 2067 2081 12 0.5 2066.9 0.4X +6 units w/o interval 2070 2071 2 0.5 2069.6 0.4X +7 units w/ interval 2458 2468 13 0.4 2457.7 0.3X +7 units w/o interval 2450 2453 3 0.4 2450.1 0.3X +8 units w/ interval 2833 2838 8 0.4 2832.6 0.3X +8 units w/o interval 2830 2839 8 0.4 2829.8 0.3X +9 units w/ interval 2873 2880 6 0.3 2873.4 0.3X +9 units w/o interval 2860 2863 3 0.3 2860.1 0.3X +10 units w/ interval 3252 3257 5 0.3 3252.2 0.2X +10 units w/o interval 3212 3220 8 0.3 3211.6 0.2X +11 units w/ interval 3369 3376 6 0.3 3368.5 0.2X +11 units w/o interval 3384 3395 15 0.3 3384.2 0.2X diff --git a/sql/core/benchmarks/IntervalBenchmark-results.txt b/sql/core/benchmarks/IntervalBenchmark-results.txt index 8976fb590d8dd..da2f0e66dacac 100644 --- a/sql/core/benchmarks/IntervalBenchmark-results.txt +++ b/sql/core/benchmarks/IntervalBenchmark-results.txt @@ -1,29 +1,29 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1 -Intel(R) Core(TM) i9-9980HK CPU @ 2.40GHz +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast strings to intervals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare string w/ interval 357 370 22 2.8 357.0 1.0X -prepare string w/o interval 315 333 22 3.2 314.7 1.1X -1 units w/ interval 356 380 21 2.8 355.8 1.0X -1 units w/o interval 317 326 12 3.2 317.1 1.1X -2 units w/ interval 481 488 8 2.1 480.8 0.7X -2 units w/o interval 456 464 9 2.2 456.0 0.8X -3 units w/ interval 1074 1080 5 0.9 1073.7 0.3X -3 units w/o interval 1025 1027 2 1.0 1025.4 0.3X -4 units w/ interval 1192 1196 5 0.8 1192.2 0.3X -4 units w/o interval 1219 1233 14 0.8 1218.9 0.3X -5 units w/ interval 1367 1382 23 0.7 1367.3 0.3X -5 units w/o interval 1295 1301 7 0.8 1295.1 0.3X -6 units w/ interval 1489 1525 31 0.7 1489.3 0.2X -6 units w/o interval 1496 1500 6 0.7 1495.8 0.2X -7 units w/ interval 1326 1330 4 0.8 1325.5 0.3X -7 units w/o interval 1324 1332 11 0.8 1324.1 0.3X -8 units w/ interval 1535 1547 11 0.7 1535.4 0.2X -8 units w/o interval 1542 1547 5 0.6 1542.3 0.2X -9 units w/ interval 1623 1641 18 0.6 1623.0 0.2X -9 units w/o interval 1615 1619 3 0.6 1615.3 0.2X -10 units w/ interval 1845 1861 16 0.5 1844.7 0.2X -10 units w/o interval 1858 1868 9 0.5 1857.8 0.2X -11 units w/ interval 1919 1925 11 0.5 1918.7 0.2X -11 units w/o interval 1973 1995 23 0.5 1972.8 0.2X +prepare string w/ interval 648 721 94 1.5 648.3 1.0X +prepare string w/o interval 562 596 49 1.8 562.3 1.2X +1 units w/ interval 568 590 21 1.8 568.5 1.1X +1 units w/o interval 522 538 20 1.9 521.7 1.2X +2 units w/ interval 751 754 3 1.3 751.5 0.9X +2 units w/o interval 716 723 6 1.4 716.1 0.9X +3 units w/ interval 1402 1411 11 0.7 1401.6 0.5X +3 units w/o interval 1381 1387 5 0.7 1381.2 0.5X +4 units w/ interval 1591 1595 6 0.6 1591.2 0.4X +4 units w/o interval 1582 1585 3 0.6 1582.3 0.4X +5 units w/ interval 1747 1749 2 0.6 1747.3 0.4X +5 units w/o interval 1738 1746 10 0.6 1737.7 0.4X +6 units w/ interval 1929 1931 3 0.5 1929.1 0.3X +6 units w/o interval 1919 1922 2 0.5 1919.0 0.3X +7 units w/ interval 2345 2354 8 0.4 2345.0 0.3X +7 units w/o interval 2334 2336 2 0.4 2334.1 0.3X +8 units w/ interval 2533 2546 16 0.4 2533.0 0.3X +8 units w/o interval 2519 2521 1 0.4 2519.4 0.3X +9 units w/ interval 2885 2889 5 0.3 2884.5 0.2X +9 units w/o interval 2804 2813 12 0.4 2803.9 0.2X +10 units w/ interval 3041 3060 16 0.3 3041.3 0.2X +10 units w/o interval 3031 3043 15 0.3 3031.2 0.2X +11 units w/ interval 3270 3280 9 0.3 3269.9 0.2X +11 units w/o interval 3273 3280 7 0.3 3272.6 0.2X diff --git a/sql/core/benchmarks/JoinBenchmark-jdk11-results.txt b/sql/core/benchmarks/JoinBenchmark-jdk11-results.txt index 06b220586ed94..1e61017425aa3 100644 --- a/sql/core/benchmarks/JoinBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/JoinBenchmark-jdk11-results.txt @@ -2,74 +2,74 @@ Join Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w long wholestage off 4872 4888 24 4.3 232.3 1.0X -Join w long wholestage on 337 425 61 62.3 16.1 14.5X +Join w long wholestage off 4441 4572 185 4.7 211.8 1.0X +Join w long wholestage on 1409 1500 96 14.9 67.2 3.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w long duplicated wholestage off 6290 6346 79 3.3 299.9 1.0X -Join w long duplicated wholestage on 328 347 17 64.0 15.6 19.2X +Join w long duplicated wholestage off 5111 5116 7 4.1 243.7 1.0X +Join w long duplicated wholestage on 1493 1518 22 14.0 71.2 3.4X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 ints wholestage off 174475 174532 82 0.1 8319.6 1.0X -Join w 2 ints wholestage on 165490 167355 1507 0.1 7891.2 1.1X +Join w 2 ints wholestage off 171821 171906 121 0.1 8193.0 1.0X +Join w 2 ints wholestage on 166559 166975 263 0.1 7942.1 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 longs wholestage off 7025 7121 135 3.0 335.0 1.0X -Join w 2 longs wholestage on 1878 1954 103 11.2 89.5 3.7X +Join w 2 longs wholestage off 7511 7555 62 2.8 358.2 1.0X +Join w 2 longs wholestage on 3776 4119 232 5.6 180.1 2.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 longs duplicated wholestage off 20822 20879 81 1.0 992.9 1.0X -Join w 2 longs duplicated wholestage on 2454 2512 60 8.5 117.0 8.5X +Join w 2 longs duplicated wholestage off 13563 13617 77 1.5 646.7 1.0X +Join w 2 longs duplicated wholestage on 7947 8053 71 2.6 378.9 1.7X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -outer join w long wholestage off 3900 3907 10 5.4 186.0 1.0X -outer join w long wholestage on 226 235 10 92.6 10.8 17.2X +outer join w long wholestage off 3915 3923 12 5.4 186.7 1.0X +outer join w long wholestage on 1421 1461 30 14.8 67.8 2.8X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -semi join w long wholestage off 2560 2621 87 8.2 122.1 1.0X -semi join w long wholestage on 209 219 15 100.5 10.0 12.3X +semi join w long wholestage off 2310 2332 30 9.1 110.2 1.0X +semi join w long wholestage on 835 860 34 25.1 39.8 2.8X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort merge join wholestage off 1707 1712 7 1.2 814.0 1.0X -sort merge join wholestage on 1447 1482 43 1.4 690.0 1.2X +sort merge join wholestage off 1846 1886 56 1.1 880.5 1.0X +sort merge join wholestage on 1402 1654 234 1.5 668.3 1.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort merge join with duplicates wholestage off 2293 2307 19 0.9 1093.4 1.0X -sort merge join with duplicates wholestage on 2018 2061 66 1.0 962.1 1.1X +sort merge join with duplicates wholestage off 2852 2879 38 0.7 1360.0 1.0X +sort merge join with duplicates wholestage on 2645 2742 156 0.8 1261.0 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -shuffle hash join wholestage off 1424 1458 47 2.9 339.6 1.0X -shuffle hash join wholestage on 1203 1227 28 3.5 286.9 1.2X +shuffle hash join wholestage off 1506 1564 82 2.8 359.1 1.0X +shuffle hash join wholestage on 1303 1330 23 3.2 310.6 1.2X diff --git a/sql/core/benchmarks/JoinBenchmark-results.txt b/sql/core/benchmarks/JoinBenchmark-results.txt index 3b88d5eaccc7e..c19dd4f12bb32 100644 --- a/sql/core/benchmarks/JoinBenchmark-results.txt +++ b/sql/core/benchmarks/JoinBenchmark-results.txt @@ -2,74 +2,74 @@ Join Benchmark ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w long wholestage off 4685 4814 182 4.5 223.4 1.0X -Join w long wholestage on 440 524 102 47.7 21.0 10.7X +Join w long wholestage off 4531 4557 37 4.6 216.1 1.0X +Join w long wholestage on 1214 1310 95 17.3 57.9 3.7X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w long duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w long duplicated wholestage off 6266 6291 35 3.3 298.8 1.0X -Join w long duplicated wholestage on 340 359 27 61.7 16.2 18.4X +Join w long duplicated wholestage off 5200 5239 55 4.0 248.0 1.0X +Join w long duplicated wholestage on 1535 1547 11 13.7 73.2 3.4X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w 2 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 ints wholestage off 174733 174916 259 0.1 8331.9 1.0X -Join w 2 ints wholestage on 166815 167619 823 0.1 7954.4 1.0X +Join w 2 ints wholestage off 170776 170795 27 0.1 8143.2 1.0X +Join w 2 ints wholestage on 165134 165183 36 0.1 7874.2 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w 2 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 longs wholestage off 7835 7889 76 2.7 373.6 1.0X -Join w 2 longs wholestage on 2057 2148 73 10.2 98.1 3.8X +Join w 2 longs wholestage off 6561 6595 48 3.2 312.8 1.0X +Join w 2 longs wholestage on 2999 3070 85 7.0 143.0 2.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Join w 2 longs duplicated: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Join w 2 longs duplicated wholestage off 19435 19497 87 1.1 926.7 1.0X -Join w 2 longs duplicated wholestage on 2472 2528 57 8.5 117.9 7.9X +Join w 2 longs duplicated wholestage off 15731 15757 38 1.3 750.1 1.0X +Join w 2 longs duplicated wholestage on 8017 8112 80 2.6 382.3 2.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz outer join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -outer join w long wholestage off 3945 3992 66 5.3 188.1 1.0X -outer join w long wholestage on 227 235 11 92.5 10.8 17.4X +outer join w long wholestage off 3573 3577 6 5.9 170.4 1.0X +outer join w long wholestage on 1310 1325 15 16.0 62.5 2.7X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz semi join w long: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -semi join w long wholestage off 2436 2469 46 8.6 116.2 1.0X -semi join w long wholestage on 231 242 11 91.0 11.0 10.6X +semi join w long wholestage off 1893 1916 33 11.1 90.3 1.0X +semi join w long wholestage on 819 842 30 25.6 39.0 2.3X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz sort merge join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort merge join wholestage off 1820 1825 7 1.2 867.9 1.0X -sort merge join wholestage on 1507 1552 34 1.4 718.8 1.2X +sort merge join wholestage off 1302 1312 13 1.6 620.9 1.0X +sort merge join wholestage on 1168 1233 62 1.8 557.0 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz sort merge join with duplicates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort merge join with duplicates wholestage off 2319 2380 86 0.9 1105.7 1.0X -sort merge join with duplicates wholestage on 2087 2139 51 1.0 995.3 1.1X +sort merge join with duplicates wholestage off 1996 2005 12 1.1 951.7 1.0X +sort merge join with duplicates wholestage on 1766 1803 42 1.2 842.0 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz shuffle hash join: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -shuffle hash join wholestage off 1554 1591 52 2.7 370.6 1.0X -shuffle hash join wholestage on 1142 1200 51 3.7 272.2 1.4X +shuffle hash join wholestage off 1298 1300 3 3.2 309.6 1.0X +shuffle hash join wholestage on 1201 1210 10 3.5 286.4 1.1X diff --git a/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..920e0a7723e70 --- /dev/null +++ b/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt @@ -0,0 +1,112 @@ +================================================================================================ +Benchmark for performance of JSON parsing +================================================================================================ + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 84774 84927 264 1.2 847.7 1.0X +UTF-8 is set 119081 120155 1773 0.8 1190.8 0.7X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 49293 49356 70 2.0 492.9 1.0X +UTF-8 is set 80183 80211 25 1.2 801.8 0.6X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 61070 61476 536 0.2 6107.0 1.0X +UTF-8 is set 109765 109881 102 0.1 10976.5 0.6X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 176999 178163 1008 0.0 353997.9 1.0X +UTF-8 is set 201209 201641 614 0.0 402419.0 0.9X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Select 10 columns 18768 20587 496 0.5 1876.8 1.0X +Select 1 column 22642 22644 3 0.4 2264.2 0.8X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Short column without encoding 7697 7738 55 1.3 769.7 1.0X +Short column with UTF-8 14051 14189 176 0.7 1405.1 0.5X +Wide column without encoding 108999 110075 1085 0.1 10899.9 0.1X +Wide column with UTF-8 157433 157779 308 0.1 15743.3 0.0X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 644 647 4 15.5 64.4 1.0X +from_json 25859 25872 12 0.4 2585.9 0.0X +json_tuple 31679 31761 71 0.3 3167.9 0.0X +get_json_object 24772 25220 389 0.4 2477.2 0.0X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 3135 3165 52 15.9 62.7 1.0X +schema inferring 29383 29389 10 1.7 587.7 0.1X +parsing 32623 35183 NaN 1.5 652.5 0.1X + +Preparing data for benchmarking ... +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 11874 11948 82 4.2 237.5 1.0X +Schema inferring 42382 42398 23 1.2 847.6 0.3X +Parsing without charset 36410 36442 54 1.4 728.2 0.3X +Parsing with UTF-8 62412 62463 48 0.8 1248.2 0.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Create a dataset of timestamps 2191 2209 20 4.6 219.1 1.0X +to_json(timestamp) 18670 19042 565 0.5 1867.0 0.1X +write timestamps to files 11836 13156 NaN 0.8 1183.6 0.2X +Create a dataset of dates 2321 2351 33 4.3 232.1 0.9X +to_json(date) 12703 12726 24 0.8 1270.3 0.2X +write dates to files 8230 8303 76 1.2 823.0 0.3X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +read timestamp text from files 2780 2795 13 3.6 278.0 1.0X +read timestamps from files 37158 37305 137 0.3 3715.8 0.1X +infer timestamps from files 73666 73838 149 0.1 7366.6 0.0X +read date text from files 2597 2609 10 3.9 259.7 1.1X +read date from files 24439 24501 56 0.4 2443.9 0.1X +timestamp strings 3052 3064 12 3.3 305.2 0.9X +parse timestamps from Dataset[String] 43611 43665 52 0.2 4361.1 0.1X +infer timestamps from Dataset[String] 83745 84153 376 0.1 8374.5 0.0X +date strings 4068 4076 10 2.5 406.8 0.7X +parse dates from Dataset[String] 34700 34807 118 0.3 3470.0 0.1X +from_json(timestamp) 64074 64124 53 0.2 6407.4 0.0X +from_json(date) 52520 52617 101 0.2 5252.0 0.1X + + diff --git a/sql/core/benchmarks/JsonBenchmark-results.txt b/sql/core/benchmarks/JsonBenchmark-results.txt index 56d371eade685..e435f573a5d92 100644 --- a/sql/core/benchmarks/JsonBenchmark-results.txt +++ b/sql/core/benchmarks/JsonBenchmark-results.txt @@ -3,110 +3,110 @@ Benchmark for performance of JSON parsing ================================================================================================ Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 69387 69850 407 1.4 693.9 1.0X -UTF-8 is set 112131 112205 83 0.9 1121.3 0.6X +No encoding 61888 61918 27 1.6 618.9 1.0X +UTF-8 is set 109057 113663 NaN 0.9 1090.6 0.6X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 44542 44671 122 2.2 445.4 1.0X -UTF-8 is set 71793 71945 146 1.4 717.9 0.6X +No encoding 44517 44535 29 2.2 445.2 1.0X +UTF-8 is set 75722 75840 111 1.3 757.2 0.6X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 58615 59011 347 0.2 5861.5 1.0X -UTF-8 is set 102542 102719 153 0.1 10254.2 0.6X +No encoding 63677 64090 633 0.2 6367.7 1.0X +UTF-8 is set 99424 99615 185 0.1 9942.4 0.6X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -No encoding 168861 170014 1552 0.0 337722.0 1.0X -UTF-8 is set 191140 191250 112 0.0 382280.3 0.9X +No encoding 174052 174251 174 0.0 348104.1 1.0X +UTF-8 is set 189000 189098 113 0.0 378000.9 0.9X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns 28017 28066 47 0.4 2801.7 1.0X -Select 1 column 24590 24641 67 0.4 2459.0 1.1X +Select 10 columns 18387 18473 142 0.5 1838.7 1.0X +Select 1 column 25560 25571 13 0.4 2556.0 0.7X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Short column without encoding 17179 17465 487 0.6 1717.9 1.0X -Short column with UTF-8 25173 25255 139 0.4 2517.3 0.7X -Wide column without encoding 146956 147069 104 0.1 14695.6 0.1X -Wide column with UTF-8 196626 197233 549 0.1 19662.6 0.1X +Short column without encoding 9323 9384 58 1.1 932.3 1.0X +Short column with UTF-8 14016 14058 55 0.7 1401.6 0.7X +Wide column without encoding 133258 133532 382 0.1 13325.8 0.1X +Wide column with UTF-8 181212 181283 61 0.1 18121.2 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 1206 1212 5 8.3 120.6 1.0X -from_json 27641 27680 34 0.4 2764.1 0.0X -json_tuple 43404 44377 860 0.2 4340.4 0.0X -get_json_object 26821 27239 619 0.4 2682.1 0.0X +Text read 1168 1174 5 8.6 116.8 1.0X +from_json 22604 23571 883 0.4 2260.4 0.1X +json_tuple 29979 30053 91 0.3 2997.9 0.0X +get_json_object 21987 22263 241 0.5 2198.7 0.1X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 5842 5865 33 8.6 116.8 1.0X -schema inferring 69673 70082 478 0.7 1393.5 0.1X -parsing 78805 81812 NaN 0.6 1576.1 0.1X +Text read 5831 5842 14 8.6 116.6 1.0X +schema inferring 31372 31456 73 1.6 627.4 0.2X +parsing 35911 36191 254 1.4 718.2 0.2X Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Text read 10782 10801 18 4.6 215.6 1.0X -Schema inferring 76817 77205 623 0.7 1536.3 0.1X -Parsing without charset 90638 91395 794 0.6 1812.8 0.1X -Parsing with UTF-8 120085 121975 1705 0.4 2401.7 0.1X +Text read 10249 10314 77 4.9 205.0 1.0X +Schema inferring 35403 35436 40 1.4 708.1 0.3X +Parsing without charset 32875 32879 4 1.5 657.5 0.3X +Parsing with UTF-8 53444 53519 100 0.9 1068.9 0.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 4706 4717 9 2.1 470.6 1.0X -to_json(timestamp) 29447 29615 226 0.3 2944.7 0.2X -write timestamps to files 20251 20673 503 0.5 2025.1 0.2X -Create a dataset of dates 4157 4172 18 2.4 415.7 1.1X -to_json(date) 21267 21301 53 0.5 2126.7 0.2X -write dates to files 13477 13897 485 0.7 1347.7 0.3X +Create a dataset of timestamps 1909 1924 17 5.2 190.9 1.0X +to_json(timestamp) 18956 19122 208 0.5 1895.6 0.1X +write timestamps to files 13446 13472 43 0.7 1344.6 0.1X +Create a dataset of dates 2180 2200 28 4.6 218.0 0.9X +to_json(date) 12780 12899 109 0.8 1278.0 0.1X +write dates to files 7835 7865 29 1.3 783.5 0.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2666 2687 29 3.8 266.6 1.0X -read timestamps from files 46967 47354 438 0.2 4696.7 0.1X -infer timestamps from files 97693 97745 65 0.1 9769.3 0.0X -read date text from files 2594 2599 5 3.9 259.4 1.0X -read date from files 35796 36008 195 0.3 3579.6 0.1X -timestamp strings 6367 6424 84 1.6 636.7 0.4X -parse timestamps from Dataset[String] 58863 59255 340 0.2 5886.3 0.0X -infer timestamps from Dataset[String] 114148 114820 836 0.1 11414.8 0.0X -date strings 7847 7863 22 1.3 784.7 0.3X -parse dates from Dataset[String] 49085 49289 212 0.2 4908.5 0.1X -from_json(timestamp) 77030 77335 395 0.1 7703.0 0.0X -from_json(date) 63275 63290 15 0.2 6327.5 0.0X +read timestamp text from files 2467 2477 9 4.1 246.7 1.0X +read timestamps from files 40186 40342 135 0.2 4018.6 0.1X +infer timestamps from files 82005 82079 71 0.1 8200.5 0.0X +read date text from files 2243 2264 22 4.5 224.3 1.1X +read date from files 24852 24863 19 0.4 2485.2 0.1X +timestamp strings 3836 3854 16 2.6 383.6 0.6X +parse timestamps from Dataset[String] 51521 51697 242 0.2 5152.1 0.0X +infer timestamps from Dataset[String] 97300 97398 133 0.1 9730.0 0.0X +date strings 4488 4491 5 2.2 448.8 0.5X +parse dates from Dataset[String] 37918 37976 68 0.3 3791.8 0.1X +from_json(timestamp) 69611 69632 36 0.1 6961.1 0.0X +from_json(date) 56598 56974 347 0.2 5659.8 0.0X diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt index f7332d4cea16e..65faa752b94cb 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-jdk11-results.txt @@ -1,22 +1,22 @@ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 2952 3104 224 33.9 29.5 1.0X -make_date(2019, 9, 16) 2341 2585 359 42.7 23.4 1.3X -make_date(*, *, *) 4751 4808 53 21.0 47.5 0.6X +prepare make_date() 3204 3323 139 31.2 32.0 1.0X +make_date(2019, 9, 16) 2529 2604 126 39.5 25.3 1.3X +make_date(*, *, *) 5102 5113 10 19.6 51.0 0.6X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_timestamp() 3462 3585 143 0.3 3461.8 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 114 120 6 8.7 114.4 30.3X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 74 82 13 13.5 74.0 46.8X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 82 95 11 12.1 82.5 42.0X -make_timestamp(*, *, *, 3, 4, 50.123456) 293 308 14 3.4 292.9 11.8X -make_timestamp(*, *, *, *, *, 0) 278 284 9 3.6 278.1 12.4X -make_timestamp(*, *, *, *, *, 60.0) 287 289 2 3.5 287.3 12.0X -make_timestamp(2019, 1, 2, *, *, *) 3556 3574 21 0.3 3555.7 1.0X -make_timestamp(*, *, *, *, *, *) 3578 3590 16 0.3 3577.9 1.0X +prepare make_timestamp() 3484 3513 28 0.3 3484.3 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 112 131 17 9.0 111.5 31.2X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 93 102 10 10.8 92.8 37.6X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 85 88 4 11.8 84.8 41.1X +make_timestamp(*, *, *, 3, 4, 50.123456) 303 308 8 3.3 302.8 11.5X +make_timestamp(*, *, *, *, *, 0) 303 307 3 3.3 302.8 11.5X +make_timestamp(*, *, *, *, *, 60.0) 289 297 8 3.5 289.1 12.1X +make_timestamp(2019, 1, 2, *, *, *) 3576 3585 11 0.3 3576.4 1.0X +make_timestamp(*, *, *, *, *, *) 3610 3618 12 0.3 3610.0 1.0X diff --git a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt index 2b184fd26eeeb..92bcc4444e60a 100644 --- a/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/MakeDateTimeBenchmark-results.txt @@ -1,22 +1,22 @@ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz make_date(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_date() 2990 3159 267 33.4 29.9 1.0X -make_date(2019, 9, 16) 2376 2446 90 42.1 23.8 1.3X -make_date(*, *, *) 4751 4786 31 21.0 47.5 0.6X +prepare make_date() 2920 3239 278 34.3 29.2 1.0X +make_date(2019, 9, 16) 2322 2371 61 43.1 23.2 1.3X +make_date(*, *, *) 4553 4560 6 22.0 45.5 0.6X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz make_timestamp(): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -prepare make_timestamp() 3855 3936 104 0.3 3855.3 1.0X -make_timestamp(2019, 1, 2, 3, 4, 50.123456) 83 87 5 12.0 83.1 46.4X -make_timestamp(2019, 1, 2, 3, 4, 60.000000) 73 82 9 13.8 72.7 53.0X -make_timestamp(2019, 12, 31, 23, 59, 60.00) 66 74 9 15.2 65.8 58.6X -make_timestamp(*, *, *, 3, 4, 50.123456) 266 277 11 3.8 265.9 14.5X -make_timestamp(*, *, *, *, *, 0) 268 275 13 3.7 267.6 14.4X -make_timestamp(*, *, *, *, *, 60.0) 272 273 0 3.7 272.4 14.2X -make_timestamp(2019, 1, 2, *, *, *) 3940 3966 24 0.3 3940.4 1.0X -make_timestamp(*, *, *, *, *, *) 3867 3917 68 0.3 3867.4 1.0X +prepare make_timestamp() 3636 3673 38 0.3 3635.7 1.0X +make_timestamp(2019, 1, 2, 3, 4, 50.123456) 94 99 4 10.7 93.8 38.8X +make_timestamp(2019, 1, 2, 3, 4, 60.000000) 68 80 13 14.6 68.3 53.2X +make_timestamp(2019, 12, 31, 23, 59, 60.00) 65 79 19 15.3 65.3 55.7X +make_timestamp(*, *, *, 3, 4, 50.123456) 271 280 14 3.7 270.7 13.4X +make_timestamp(*, *, *, *, *, 0) 255 263 11 3.9 255.5 14.2X +make_timestamp(*, *, *, *, *, 60.0) 254 258 4 3.9 254.2 14.3X +make_timestamp(2019, 1, 2, *, *, *) 3714 3722 8 0.3 3713.9 1.0X +make_timestamp(*, *, *, *, *, *) 3736 3741 6 0.3 3736.3 1.0X diff --git a/sql/core/benchmarks/MiscBenchmark-jdk11-results.txt b/sql/core/benchmarks/MiscBenchmark-jdk11-results.txt index 991a447fe3237..281b0fe28171b 100644 --- a/sql/core/benchmarks/MiscBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/MiscBenchmark-jdk11-results.txt @@ -2,126 +2,126 @@ filter & aggregate without group ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz range/filter/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -range/filter/sum wholestage off 52841 53513 951 39.7 25.2 1.0X -range/filter/sum wholestage on 3104 3188 109 675.6 1.5 17.0X +range/filter/sum wholestage off 54616 54834 309 38.4 26.0 1.0X +range/filter/sum wholestage on 3263 3369 129 642.6 1.6 16.7X ================================================================================================ range/limit/sum ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz range/limit/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -range/limit/sum wholestage off 199 222 32 2633.5 0.4 1.0X -range/limit/sum wholestage on 131 138 7 3991.0 0.3 1.5X +range/limit/sum wholestage off 239 265 36 2190.0 0.5 1.0X +range/limit/sum wholestage on 137 162 15 3814.6 0.3 1.7X ================================================================================================ sample ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz sample with replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sample with replacement wholestage off 13015 13441 602 10.1 99.3 1.0X -sample with replacement wholestage on 7688 7745 91 17.0 58.7 1.7X +sample with replacement wholestage off 13093 13417 458 10.0 99.9 1.0X +sample with replacement wholestage on 7606 7624 13 17.2 58.0 1.7X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz sample without replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sample without replacement wholestage off 2990 2993 4 43.8 22.8 1.0X -sample without replacement wholestage on 1172 1186 13 111.9 8.9 2.6X +sample without replacement wholestage off 3031 3038 9 43.2 23.1 1.0X +sample without replacement wholestage on 1156 1177 16 113.4 8.8 2.6X ================================================================================================ collect ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz collect: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -collect 1 million 317 425 188 3.3 301.9 1.0X -collect 2 millions 575 598 26 1.8 548.2 0.6X -collect 4 millions 1350 1848 704 0.8 1287.6 0.2X +collect 1 million 335 360 20 3.1 319.0 1.0X +collect 2 millions 579 633 42 1.8 552.2 0.6X +collect 4 millions 1192 1331 196 0.9 1137.1 0.3X ================================================================================================ collect limit ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz collect limit: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -collect limit 1 million 375 387 14 2.8 358.0 1.0X -collect limit 2 millions 714 736 28 1.5 681.3 0.5X +collect limit 1 million 378 386 12 2.8 360.8 1.0X +collect limit 2 millions 724 733 12 1.4 690.3 0.5X ================================================================================================ generate explode ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate explode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate explode array wholestage off 19669 19829 227 0.9 1172.3 1.0X -generate explode array wholestage on 10983 11020 35 1.5 654.7 1.8X +generate explode array wholestage off 15880 16159 395 1.1 946.5 1.0X +generate explode array wholestage on 15690 15783 73 1.1 935.2 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate explode map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate explode map wholestage off 57756 57928 243 0.3 3442.5 1.0X -generate explode map wholestage on 47398 47703 250 0.4 2825.2 1.2X +generate explode map wholestage off 51293 51311 26 0.3 3057.3 1.0X +generate explode map wholestage on 50409 50795 388 0.3 3004.6 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate posexplode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate posexplode array wholestage off 21402 21525 174 0.8 1275.7 1.0X -generate posexplode array wholestage on 11898 11982 67 1.4 709.2 1.8X +generate posexplode array wholestage off 17246 17860 869 1.0 1027.9 1.0X +generate posexplode array wholestage on 17344 17472 88 1.0 1033.8 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate inline array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate inline array wholestage off 15570 15597 37 1.1 928.1 1.0X -generate inline array wholestage on 10044 10161 87 1.7 598.6 1.6X +generate inline array wholestage off 13698 13790 130 1.2 816.5 1.0X +generate inline array wholestage on 12995 13033 38 1.3 774.6 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate big struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate big struct array wholestage off 601 615 19 0.1 10023.5 1.0X -generate big struct array wholestage on 388 410 25 0.2 6458.7 1.6X +generate big struct array wholestage off 476 489 18 0.1 7938.6 1.0X +generate big struct array wholestage on 424 440 14 0.1 7074.9 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate big nested struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate big nested struct array wholestage off 492 525 48 0.1 8191.7 1.0X -generate big nested struct array wholestage on 468 496 30 0.1 7796.4 1.1X +generate big nested struct array wholestage off 413 433 28 0.1 6886.8 1.0X +generate big nested struct array wholestage on 394 420 39 0.2 6560.9 1.0X ================================================================================================ generate regular generator ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate stack: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate stack wholestage off 28014 28091 109 0.6 1669.8 1.0X -generate stack wholestage on 19246 19267 27 0.9 1147.2 1.5X +generate stack wholestage off 26494 26510 23 0.6 1579.2 1.0X +generate stack wholestage on 24205 24339 80 0.7 1442.8 1.1X diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt index 012e81b03c8df..8b1e728ed9cf9 100644 --- a/sql/core/benchmarks/MiscBenchmark-results.txt +++ b/sql/core/benchmarks/MiscBenchmark-results.txt @@ -2,126 +2,126 @@ filter & aggregate without group ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz range/filter/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -range/filter/sum wholestage off 45894 47528 2310 45.7 21.9 1.0X -range/filter/sum wholestage on 3193 3572 213 656.8 1.5 14.4X +range/filter/sum wholestage off 46812 48110 1836 44.8 22.3 1.0X +range/filter/sum wholestage on 3116 3656 309 673.1 1.5 15.0X ================================================================================================ range/limit/sum ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz range/limit/sum: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -range/limit/sum wholestage off 202 207 7 2593.1 0.4 1.0X -range/limit/sum wholestage on 148 161 20 3545.1 0.3 1.4X +range/limit/sum wholestage off 200 219 27 2621.6 0.4 1.0X +range/limit/sum wholestage on 117 125 7 4477.8 0.2 1.7X ================================================================================================ sample ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz sample with replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sample with replacement wholestage off 12337 12657 452 10.6 94.1 1.0X -sample with replacement wholestage on 7355 7368 14 17.8 56.1 1.7X +sample with replacement wholestage off 12963 13506 768 10.1 98.9 1.0X +sample with replacement wholestage on 7397 7742 300 17.7 56.4 1.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz sample without replacement: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sample without replacement wholestage off 3050 3060 14 43.0 23.3 1.0X -sample without replacement wholestage on 1103 1118 22 118.8 8.4 2.8X +sample without replacement wholestage off 2977 2977 0 44.0 22.7 1.0X +sample without replacement wholestage on 1087 1099 13 120.6 8.3 2.7X ================================================================================================ collect ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz collect: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -collect 1 million 324 361 41 3.2 309.4 1.0X -collect 2 millions 619 700 70 1.7 590.4 0.5X -collect 4 millions 1214 1293 111 0.9 1158.2 0.3X +collect 1 million 281 315 34 3.7 268.2 1.0X +collect 2 millions 531 564 28 2.0 506.6 0.5X +collect 4 millions 1179 1970 1118 0.9 1124.5 0.2X ================================================================================================ collect limit ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz collect limit: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -collect limit 1 million 391 396 5 2.7 373.0 1.0X -collect limit 2 millions 746 769 26 1.4 711.8 0.5X +collect limit 1 million 344 352 5 3.0 328.0 1.0X +collect limit 2 millions 656 660 6 1.6 625.2 0.5X ================================================================================================ generate explode ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate explode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate explode array wholestage off 15063 16007 1335 1.1 897.8 1.0X -generate explode array wholestage on 10909 10932 20 1.5 650.2 1.4X +generate explode array wholestage off 14664 14780 164 1.1 874.0 1.0X +generate explode array wholestage on 14789 14886 87 1.1 881.5 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate explode map: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate explode map wholestage off 55011 55580 806 0.3 3278.9 1.0X -generate explode map wholestage on 46009 46445 399 0.4 2742.3 1.2X +generate explode map wholestage off 50441 51014 811 0.3 3006.5 1.0X +generate explode map wholestage on 49164 49915 892 0.3 2930.4 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate posexplode array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate posexplode array wholestage off 17987 18866 1244 0.9 1072.1 1.0X -generate posexplode array wholestage on 11400 11444 38 1.5 679.5 1.6X +generate posexplode array wholestage off 17101 17130 40 1.0 1019.3 1.0X +generate posexplode array wholestage on 15625 15675 46 1.1 931.3 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate inline array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate inline array wholestage off 14905 14991 122 1.1 888.4 1.0X -generate inline array wholestage on 9919 10129 160 1.7 591.2 1.5X +generate inline array wholestage off 13923 14720 1127 1.2 829.9 1.0X +generate inline array wholestage on 12246 12591 695 1.4 729.9 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate big struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate big struct array wholestage off 508 577 98 0.1 8474.0 1.0X -generate big struct array wholestage on 451 485 26 0.1 7523.2 1.1X +generate big struct array wholestage off 442 456 20 0.1 7368.2 1.0X +generate big struct array wholestage on 409 429 22 0.1 6823.9 1.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate big nested struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate big nested struct array wholestage off 537 567 43 0.1 8944.4 1.0X -generate big nested struct array wholestage on 528 546 24 0.1 8804.2 1.0X +generate big nested struct array wholestage off 409 415 8 0.1 6822.4 1.0X +generate big nested struct array wholestage on 401 414 23 0.1 6687.0 1.0X ================================================================================================ generate regular generator ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz generate stack: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -generate stack wholestage off 27220 27608 548 0.6 1622.5 1.0X -generate stack wholestage on 20070 20170 117 0.8 1196.3 1.4X +generate stack wholestage off 27043 27118 106 0.6 1611.9 1.0X +generate stack wholestage on 24002 24036 34 0.7 1430.6 1.1X diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk11-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk11-results.txt index 4d35f872871f0..c286dc2da8076 100644 --- a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-jdk11-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For ORC v1 ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 150 192 23 6.7 150.1 1.0X -Nested column 1241 1289 82 0.8 1241.3 0.1X -Nested column in array 5466 5574 114 0.2 5465.6 0.0X +Top-level column 155 197 23 6.5 155.0 1.0X +Nested column 1290 1338 43 0.8 1289.9 0.1X +Nested column in array 5914 5989 57 0.2 5914.1 0.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 463 515 30 2.2 462.6 1.0X -Nested column 1920 1967 27 0.5 1920.1 0.2X -Nested column in array 6565 6685 123 0.2 6564.8 0.1X +Top-level column 463 537 53 2.2 463.2 1.0X +Nested column 1807 1874 62 0.6 1807.3 0.3X +Nested column in array 6729 6906 116 0.1 6729.4 0.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 387 431 80 2.6 387.0 1.0X -Nested column 1846 1865 13 0.5 1846.2 0.2X -Nested column in array 6458 6569 114 0.2 6458.2 0.1X +Top-level column 378 402 18 2.6 377.7 1.0X +Nested column 1765 1794 46 0.6 1765.4 0.2X +Nested column in array 6675 6718 42 0.1 6674.6 0.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 391 405 13 2.6 390.9 1.0X -Nested column 4597 4684 81 0.2 4597.4 0.1X -Nested column in array 9434 9499 54 0.1 9433.9 0.0X +Top-level column 384 399 8 2.6 383.7 1.0X +Nested column 4724 4820 72 0.2 4724.3 0.1X +Nested column in array 9256 9405 133 0.1 9255.7 0.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 131 165 27 7.6 130.9 1.0X -Nested column 1258 1298 31 0.8 1257.7 0.1X -Nested column in array 5360 5393 16 0.2 5359.7 0.0X +Top-level column 121 137 13 8.3 120.7 1.0X +Nested column 1303 1361 45 0.8 1303.2 0.1X +Nested column in array 5901 5978 49 0.2 5901.0 0.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 584 632 63 1.7 583.7 1.0X -Nested column 5211 5384 108 0.2 5211.1 0.1X -Nested column in array 10279 10397 94 0.1 10279.0 0.1X +Top-level column 651 675 30 1.5 650.8 1.0X +Nested column 5398 5527 179 0.2 5397.9 0.1X +Nested column in array 10413 10685 310 0.1 10413.1 0.1X diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt index 0dd85968b937e..0cbd855ec5e3a 100644 --- a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For ORC v1 ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 131 167 24 7.6 131.0 1.0X -Nested column 1296 1340 33 0.8 1296.0 0.1X -Nested column in array 5568 5745 243 0.2 5567.8 0.0X +Top-level column 116 148 22 8.6 115.9 1.0X +Nested column 1200 1221 23 0.8 1200.3 0.1X +Nested column in array 4786 4898 202 0.2 4785.6 0.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 432 468 33 2.3 431.6 1.0X -Nested column 1778 1828 35 0.6 1777.7 0.2X -Nested column in array 6565 6727 137 0.2 6565.1 0.1X +Top-level column 428 452 18 2.3 427.7 1.0X +Nested column 1681 1725 53 0.6 1680.7 0.3X +Nested column in array 5652 5699 47 0.2 5651.9 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 365 406 60 2.7 365.3 1.0X -Nested column 1803 1821 22 0.6 1803.2 0.2X -Nested column in array 6453 6530 111 0.2 6453.5 0.1X +Top-level column 341 351 15 2.9 340.9 1.0X +Nested column 1529 1559 31 0.7 1528.6 0.2X +Nested column in array 5468 5517 97 0.2 5468.2 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 370 390 30 2.7 369.6 1.0X -Nested column 4955 5028 61 0.2 4955.0 0.1X -Nested column in array 10490 10553 49 0.1 10490.5 0.0X +Top-level column 353 365 12 2.8 352.5 1.0X +Nested column 4135 4171 23 0.2 4135.2 0.1X +Nested column in array 8766 8816 56 0.1 8766.0 0.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 135 150 18 7.4 134.5 1.0X -Nested column 1615 1717 78 0.6 1615.0 0.1X -Nested column in array 6919 7024 53 0.1 6919.4 0.0X +Top-level column 115 133 25 8.7 115.3 1.0X +Nested column 1200 1216 13 0.8 1199.7 0.1X +Nested column in array 5296 5345 39 0.2 5296.3 0.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 580 619 28 1.7 580.4 1.0X -Nested column 5547 5752 185 0.2 5547.3 0.1X -Nested column in array 11639 11838 183 0.1 11639.4 0.0X +Top-level column 545 560 14 1.8 544.7 1.0X +Nested column 4704 4813 184 0.2 4703.8 0.1X +Nested column in array 9785 9824 42 0.1 9784.8 0.1X diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk11-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk11-results.txt index 0036510d62fc9..31970672aacd4 100644 --- a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-jdk11-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For ORC v2 ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 150 187 20 6.7 150.2 1.0X -Nested column 1370 1463 104 0.7 1369.8 0.1X -Nested column in array 6575 6656 66 0.2 6575.2 0.0X +Top-level column 176 207 20 5.7 176.2 1.0X +Nested column 1173 1212 61 0.9 1173.2 0.2X +Nested column in array 5557 5628 58 0.2 5557.5 0.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 134 161 20 7.5 133.7 1.0X -Nested column 1532 1550 15 0.7 1532.4 0.1X -Nested column in array 6601 6688 126 0.2 6601.4 0.0X +Top-level column 143 178 21 7.0 143.5 1.0X +Nested column 1233 1272 31 0.8 1233.0 0.1X +Nested column in array 5585 5691 66 0.2 5585.4 0.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 409 491 190 2.4 409.5 1.0X -Nested column 1896 1938 60 0.5 1896.2 0.2X -Nested column in array 7414 7472 43 0.1 7414.0 0.1X +Top-level column 395 416 23 2.5 395.4 1.0X +Nested column 1665 1691 16 0.6 1664.7 0.2X +Nested column in array 6194 6307 183 0.2 6194.5 0.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 408 433 37 2.5 408.1 1.0X -Nested column 4703 4795 101 0.2 4703.4 0.1X -Nested column in array 10477 10556 61 0.1 10476.6 0.0X +Top-level column 394 407 14 2.5 393.7 1.0X +Nested column 4381 4536 97 0.2 4380.6 0.1X +Nested column in array 8797 8833 34 0.1 8796.7 0.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 130 164 31 7.7 130.3 1.0X -Nested column 1471 1513 44 0.7 1470.5 0.1X -Nested column in array 6492 6551 55 0.2 6492.0 0.0X +Top-level column 136 170 24 7.4 135.6 1.0X +Nested column 1135 1171 30 0.9 1135.0 0.1X +Nested column in array 4833 4911 92 0.2 4833.3 0.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 268 283 14 3.7 268.4 1.0X -Nested column 3347 3401 77 0.3 3347.3 0.1X -Nested column in array 9297 9433 237 0.1 9296.6 0.0X +Top-level column 327 336 8 3.1 327.0 1.0X +Nested column 3321 3377 56 0.3 3321.0 0.1X +Nested column in array 8080 8161 97 0.1 8080.4 0.0X diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt index 8aff152f0c136..c7d8c8a541b59 100644 --- a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For ORC v2 ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 141 161 22 7.1 140.6 1.0X -Nested column 1425 1455 26 0.7 1424.7 0.1X -Nested column in array 5248 5300 46 0.2 5247.5 0.0X +Top-level column 121 156 27 8.3 121.1 1.0X +Nested column 1373 1406 37 0.7 1373.4 0.1X +Nested column in array 5545 5579 54 0.2 5544.8 0.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 133 163 22 7.5 132.8 1.0X -Nested column 1254 1308 40 0.8 1254.0 0.1X -Nested column in array 5303 5418 81 0.2 5303.3 0.0X +Top-level column 127 147 20 7.9 127.0 1.0X +Nested column 1280 1328 32 0.8 1280.2 0.1X +Nested column in array 5617 5696 70 0.2 5617.0 0.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 377 401 19 2.7 376.7 1.0X -Nested column 1676 1722 21 0.6 1676.1 0.2X -Nested column in array 6019 6127 109 0.2 6018.7 0.1X +Top-level column 343 356 17 2.9 342.6 1.0X +Nested column 1692 1710 14 0.6 1692.3 0.2X +Nested column in array 6128 6168 30 0.2 6128.0 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 390 447 151 2.6 390.1 1.0X -Nested column 4300 4364 60 0.2 4299.7 0.1X -Nested column in array 8832 9030 114 0.1 8832.4 0.0X +Top-level column 348 355 11 2.9 348.1 1.0X +Nested column 4350 4392 35 0.2 4349.8 0.1X +Nested column in array 8864 8901 29 0.1 8864.1 0.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 132 143 7 7.6 131.6 1.0X -Nested column 1260 1303 20 0.8 1260.2 0.1X -Nested column in array 5359 5453 74 0.2 5359.1 0.0X +Top-level column 123 143 27 8.2 122.5 1.0X +Nested column 1233 1295 29 0.8 1233.2 0.1X +Nested column in array 5534 5597 53 0.2 5533.7 0.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 288 302 20 3.5 287.6 1.0X -Nested column 3169 3242 53 0.3 3168.7 0.1X -Nested column in array 8151 8301 123 0.1 8151.3 0.0X +Top-level column 265 280 20 3.8 264.8 1.0X +Nested column 3211 3263 96 0.3 3211.2 0.1X +Nested column in array 8324 8357 42 0.1 8323.6 0.0X diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk11-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk11-results.txt index 4535220aeab0b..f6135968bb97b 100644 --- a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-jdk11-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For Parquet ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 164 191 18 6.1 163.6 1.0X -Nested column 416 444 26 2.4 415.7 0.4X -Nested column in array 1349 1398 39 0.7 1349.4 0.1X +Top-level column 150 218 36 6.6 150.5 1.0X +Nested column 294 334 39 3.4 293.8 0.5X +Nested column in array 994 1024 34 1.0 994.3 0.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 153 169 14 6.5 152.7 1.0X -Nested column 540 566 26 1.9 540.5 0.3X -Nested column in array 1378 1447 87 0.7 1378.0 0.1X +Top-level column 161 183 13 6.2 160.9 1.0X +Nested column 296 332 43 3.4 295.6 0.5X +Nested column in array 1048 1089 48 1.0 1047.9 0.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 414 433 26 2.4 414.0 1.0X -Nested column 736 777 56 1.4 736.2 0.6X -Nested column in array 1895 1954 86 0.5 1895.2 0.2X +Top-level column 393 411 16 2.5 393.1 1.0X +Nested column 550 563 9 1.8 549.9 0.7X +Nested column in array 1468 1494 20 0.7 1467.5 0.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 402 436 49 2.5 402.1 1.0X -Nested column 3190 3225 26 0.3 3190.4 0.1X -Nested column in array 3749 3806 68 0.3 3748.7 0.1X +Top-level column 404 419 15 2.5 404.2 1.0X +Nested column 2833 2875 24 0.4 2832.8 0.1X +Nested column in array 3340 3396 53 0.3 3340.3 0.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 135 159 25 7.4 134.5 1.0X -Nested column 462 497 36 2.2 462.4 0.3X -Nested column in array 1453 1508 45 0.7 1453.5 0.1X +Top-level column 142 159 12 7.0 142.3 1.0X +Nested column 308 318 8 3.2 307.8 0.5X +Nested column in array 1119 1154 32 0.9 1118.5 0.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 277 301 27 3.6 277.3 1.0X -Nested column 2057 2097 33 0.5 2057.3 0.1X -Nested column in array 3027 3134 122 0.3 3027.4 0.1X +Top-level column 338 342 2 3.0 338.0 1.0X +Nested column 1873 1971 129 0.5 1873.1 0.2X +Nested column in array 2708 2760 50 0.4 2707.9 0.1X diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt index 182a1aed497a3..9f64e0425df8d 100644 --- a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt +++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt @@ -2,52 +2,52 @@ Nested Schema Pruning Benchmark For Parquet ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 149 183 22 6.7 148.8 1.0X -Nested column 413 436 20 2.4 413.4 0.4X -Nested column in array 1309 1327 24 0.8 1308.9 0.1X +Top-level column 136 157 19 7.3 136.3 1.0X +Nested column 254 267 8 3.9 254.3 0.5X +Nested column in array 1071 1089 18 0.9 1071.1 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 160 184 17 6.3 159.5 1.0X -Nested column 436 481 77 2.3 436.5 0.4X -Nested column in array 1354 1381 26 0.7 1353.6 0.1X +Top-level column 134 147 12 7.5 134.1 1.0X +Nested column 288 295 5 3.5 287.7 0.5X +Nested column in array 1104 1135 35 0.9 1104.1 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 378 412 61 2.6 378.0 1.0X -Nested column 744 790 51 1.3 744.0 0.5X -Nested column in array 1915 1987 76 0.5 1915.0 0.2X +Top-level column 361 372 14 2.8 361.1 1.0X +Nested column 522 535 16 1.9 521.8 0.7X +Nested column in array 1540 1553 11 0.6 1539.6 0.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 389 407 16 2.6 388.6 1.0X -Nested column 3152 3217 48 0.3 3151.9 0.1X -Nested column in array 3698 3860 274 0.3 3698.2 0.1X +Top-level column 375 384 11 2.7 374.6 1.0X +Nested column 2686 2715 24 0.4 2686.2 0.1X +Nested column in array 3067 3080 13 0.3 3067.2 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 143 170 27 7.0 142.7 1.0X -Nested column 450 461 8 2.2 449.9 0.3X -Nested column in array 1411 1460 41 0.7 1411.2 0.1X +Top-level column 120 135 8 8.3 120.3 1.0X +Nested column 280 290 13 3.6 279.9 0.4X +Nested column in array 1114 1143 29 0.9 1114.2 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Top-level column 293 309 22 3.4 292.9 1.0X -Nested column 2109 2142 19 0.5 2109.1 0.1X -Nested column in array 3018 3096 79 0.3 3017.7 0.1X +Top-level column 263 277 18 3.8 263.0 1.0X +Nested column 1724 1763 38 0.6 1724.1 0.2X +Nested column in array 2530 2605 65 0.4 2529.9 0.1X diff --git a/sql/core/benchmarks/RangeBenchmark-jdk11-results.txt b/sql/core/benchmarks/RangeBenchmark-jdk11-results.txt index dc86fb686fb77..44109e406ebb6 100644 --- a/sql/core/benchmarks/RangeBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/RangeBenchmark-jdk11-results.txt @@ -2,14 +2,14 @@ range ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz range: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -full scan 20357 21595 1692 25.8 38.8 1.0X -limit after range 115 130 15 4563.9 0.2 177.2X -filter after range 1890 1906 21 277.5 3.6 10.8X -count after range 85 87 3 6200.7 0.2 240.8X -count after limit after range 101 109 9 5213.9 0.2 202.4X +full scan 11004 11558 944 47.6 21.0 1.0X +limit after range 121 135 19 4320.4 0.2 90.7X +filter after range 1529 1544 23 342.9 2.9 7.2X +count after range 123 143 21 4268.5 0.2 89.6X +count after limit after range 115 123 6 4547.6 0.2 95.4X diff --git a/sql/core/benchmarks/RangeBenchmark-results.txt b/sql/core/benchmarks/RangeBenchmark-results.txt index 7b6daf9b9c40d..e006785e7a2ca 100644 --- a/sql/core/benchmarks/RangeBenchmark-results.txt +++ b/sql/core/benchmarks/RangeBenchmark-results.txt @@ -2,14 +2,14 @@ range ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz range: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -full scan 19094 19768 459 27.5 36.4 1.0X -limit after range 90 108 19 5803.9 0.2 211.4X -filter after range 1807 1820 12 290.1 3.4 10.6X -count after range 88 93 6 5941.4 0.2 216.4X -count after limit after range 82 86 4 6398.2 0.2 233.0X +full scan 11565 11703 217 45.3 22.1 1.0X +limit after range 96 100 4 5455.9 0.2 120.3X +filter after range 1426 1432 10 367.7 2.7 8.1X +count after range 82 84 2 6412.8 0.2 141.5X +count after limit after range 72 76 3 7264.9 0.1 160.2X diff --git a/sql/core/benchmarks/TPCDSQueryBenchmark-jdk11-results.txt b/sql/core/benchmarks/TPCDSQueryBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..0e1a6d504da0b --- /dev/null +++ b/sql/core/benchmarks/TPCDSQueryBenchmark-jdk11-results.txt @@ -0,0 +1,810 @@ +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q1 1460 1941 680 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q2 2422 2665 344 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q3 566 578 8 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q4 15396 15718 456 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q5 3251 3670 592 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q6 2139 2232 131 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q7 1285 1365 113 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q8 1038 1085 67 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q9 2709 2729 28 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q10 5975 6075 140 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q11 3569 4018 635 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q12 503 541 45 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q13 2950 3044 132 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q14a 24716 25725 1427 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q14b 20165 20747 822 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q15 837 896 55 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q16 2124 2190 93 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q17 3178 3351 245 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q18 2523 2653 184 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q19 818 876 53 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q20 513 521 7 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q21 1458 1496 54 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q22 4247 4364 166 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q23a 15449 15516 95 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q23b 18832 19116 401 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q24a 3190 3852 937 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q24b 3326 3374 68 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q25 3145 3174 40 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q26 852 879 25 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q27 1392 1393 1 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q28 3913 3932 27 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q29 3145 3199 77 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q30 1263 1304 57 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q31 2182 2520 479 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q32 641 652 14 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q33 1779 1971 272 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q34 828 841 11 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q35 5293 5346 75 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q36 1222 1226 6 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q37 1432 1460 39 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q38 1927 2083 221 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q39a 2762 2911 210 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q39b 2710 2841 186 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q40 732 801 64 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q41 412 466 50 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q42 424 444 22 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q43 673 683 11 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q44 1325 1340 21 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q45 552 618 52 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q46 1097 1124 39 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q47 4876 5064 266 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q48 2709 2734 35 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q49 2172 2361 267 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q50 1467 1516 69 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q51 4681 4847 234 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q52 423 443 27 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q53 723 741 18 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q54 3656 3675 27 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q55 416 450 31 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q56 1552 1585 47 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q57 2971 3246 388 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q58 1843 2035 271 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q59 2128 2177 69 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q60 1478 1498 27 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q61 1471 1574 145 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q62 539 550 11 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q63 738 751 23 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q64 8378 9105 1028 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q65 1642 1685 61 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q66 2038 2056 24 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q67 10208 10302 133 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q68 1021 1050 42 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q69 5244 5326 116 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q70 1441 1448 9 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q71 1230 1240 13 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q72 21418 22601 1674 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q73 779 783 3 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q74 2947 3332 545 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q75 5149 5374 317 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q76 969 974 5 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q77 1928 2256 464 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q78 4871 5152 397 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q79 906 958 57 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q80 3756 4051 417 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q81 1124 1152 40 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q82 1954 1981 39 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q83 1150 1159 13 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q84 1301 1333 46 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q85 4009 4176 235 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q86 657 678 25 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q87 2230 2470 339 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q88 2772 2959 265 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q89 819 856 40 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q90 436 445 8 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q91 753 837 73 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q92 532 557 27 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q93 1280 1304 34 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q94 1034 1072 53 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q95 6223 6526 429 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q96 392 399 7 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q97 1845 1932 124 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q98 607 643 26 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q99 650 689 51 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q5a-v2.7 5008 5199 270 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q6-v2.7 1873 1930 80 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q10a-v2.7 4496 4505 13 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q11-v2.7 3597 3918 454 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q12-v2.7 405 432 22 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q14-v2.7 18204 18604 565 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q14a-v2.7 116778 117402 883 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q18a-v2.7 4616 4850 331 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q20-v2.7 449 487 46 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q22-v2.7 20882 20987 149 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q22a-v2.7 10305 10646 483 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q24-v2.7 2843 3091 350 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q27a-v2.7 2733 2857 177 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q34-v2.7 784 834 43 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q35-v2.7 4648 4900 358 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q35a-v2.7 4225 4350 177 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q36a-v2.7 2651 2830 253 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q47-v2.7 4884 5022 195 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q49-v2.7 2126 2311 262 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q51a-v2.7 30262 30597 474 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q57-v2.7 2962 3086 176 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q64-v2.7 8345 8680 474 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q67a-v2.7 18924 19300 532 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q70a-v2.7 2575 2677 144 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q72-v2.7 21513 22533 1442 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q74-v2.7 2891 3182 411 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q75-v2.7 5079 5308 324 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q77a-v2.7 4166 4419 357 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q78-v2.7 4992 5258 376 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q80a-v2.7 6027 6309 399 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q86a-v2.7 1475 1639 232 0.0 Infinity 1.0X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +q98-v2.7 596 626 34 0.0 Infinity 1.0X + diff --git a/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt b/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt index 87cf48c5a11f2..8228e191ec487 100644 --- a/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt +++ b/sql/core/benchmarks/TPCDSQueryBenchmark-results.txt @@ -1,810 +1,810 @@ -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q1 1661 1738 109 0.0 Infinity 1.0X +q1 1626 1675 69 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q2 2517 2601 119 0.0 Infinity 1.0X +q2 2166 2277 158 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q3 544 566 36 0.0 Infinity 1.0X +q3 465 505 65 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q4 15391 15676 403 0.0 Infinity 1.0X +q4 15108 15662 784 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q5 3347 3709 512 0.0 Infinity 1.0X +q5 3087 3281 274 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q6 2461 2469 11 0.0 Infinity 1.0X +q6 1780 1873 132 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q7 1195 1216 30 0.0 Infinity 1.0X +q7 1103 1137 49 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q8 1164 1207 60 0.0 Infinity 1.0X +q8 998 1019 30 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q9 2893 2932 54 0.0 Infinity 1.0X +q9 2445 2463 25 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q10 3685 3840 219 0.0 Infinity 1.0X +q10 4853 5233 537 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q11 3791 3820 41 0.0 Infinity 1.0X +q11 3370 3417 67 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q12 553 575 18 0.0 Infinity 1.0X +q12 442 538 67 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q13 2960 3157 278 0.0 Infinity 1.0X +q13 2589 2767 253 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14a 26682 27183 709 0.0 Infinity 1.0X +q14a 23714 24391 957 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14b 20780 21353 811 0.0 Infinity 1.0X +q14b 19056 19103 66 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q15 867 883 20 0.0 Infinity 1.0X +q15 771 797 24 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q16 1448 1484 52 0.0 Infinity 1.0X +q16 1658 1707 69 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q17 3104 3147 60 0.0 Infinity 1.0X +q17 2905 2979 104 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q18 2299 2555 362 0.0 Infinity 1.0X +q18 2272 2423 213 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q19 881 897 17 0.0 Infinity 1.0X +q19 707 730 35 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q20 539 583 41 0.0 Infinity 1.0X +q20 449 506 42 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q21 1257 1268 15 0.0 Infinity 1.0X +q21 1154 1167 19 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22 4142 4340 281 0.0 Infinity 1.0X +q22 4056 4476 594 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q23a 15861 16068 292 0.0 Infinity 1.0X +q23a 14557 14780 317 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q23b 19762 19835 103 0.0 Infinity 1.0X +q23b 17887 18451 799 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24a 3686 3808 173 0.0 Infinity 1.0X +q24a 2930 3193 372 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24b 3364 3396 46 0.0 Infinity 1.0X +q24b 2760 2958 280 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q25 3004 3294 410 0.0 Infinity 1.0X +q25 2913 3150 335 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q26 886 917 28 0.0 Infinity 1.0X +q26 810 819 15 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q27 1382 1382 0 0.0 Infinity 1.0X +q27 1171 1217 65 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q28 3329 3354 35 0.0 Infinity 1.0X +q28 3212 3273 86 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q29 2949 3048 141 0.0 Infinity 1.0X +q29 2917 3107 270 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q30 1443 1443 0 0.0 Infinity 1.0X +q30 1248 1277 40 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q31 2658 2859 285 0.0 Infinity 1.0X +q31 1924 2091 237 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q32 707 733 23 0.0 Infinity 1.0X +q32 559 597 26 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q33 1790 1798 12 0.0 Infinity 1.0X +q33 1314 1325 16 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q34 880 886 6 0.0 Infinity 1.0X +q34 761 773 11 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35 3152 3277 177 0.0 Infinity 1.0X +q35 4967 4984 24 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q36 1211 1229 26 0.0 Infinity 1.0X +q36 1109 1116 9 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q37 1495 1500 7 0.0 Infinity 1.0X +q37 1200 1234 48 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q38 2017 2073 79 0.0 Infinity 1.0X +q38 1898 2035 194 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q39a 2575 2603 39 0.0 Infinity 1.0X +q39a 2252 2362 155 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q39b 2388 2507 168 0.0 Infinity 1.0X +q39b 2142 2248 150 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q40 872 920 81 0.0 Infinity 1.0X +q40 654 687 57 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q41 453 518 48 0.0 Infinity 1.0X +q41 383 448 51 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q42 442 480 30 0.0 Infinity 1.0X +q42 358 383 21 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q43 683 691 7 0.0 Infinity 1.0X +q43 577 619 37 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q44 1301 1307 8 0.0 Infinity 1.0X +q44 1188 1234 65 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q45 700 717 26 0.0 Infinity 1.0X +q45 529 562 25 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q46 1198 1224 38 0.0 Infinity 1.0X +q46 993 1023 42 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q47 4915 5163 350 0.0 Infinity 1.0X +q47 4547 4741 274 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q48 2569 2585 22 0.0 Infinity 1.0X +q48 2303 2426 174 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q49 2193 2277 119 0.0 Infinity 1.0X +q49 2080 2086 9 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q50 1558 1564 9 0.0 Infinity 1.0X +q50 1371 1388 24 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q51 4375 4515 198 0.0 Infinity 1.0X +q51 4373 4513 197 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q52 441 454 15 0.0 Infinity 1.0X +q52 360 379 28 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q53 719 751 29 0.0 Infinity 1.0X +q53 661 677 23 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q54 3141 3182 58 0.0 Infinity 1.0X +q54 3454 3611 222 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q55 433 471 24 0.0 Infinity 1.0X +q55 360 383 37 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q56 1766 1811 65 0.0 Infinity 1.0X +q56 1350 1388 53 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q57 3203 3378 248 0.0 Infinity 1.0X +q57 2910 3156 349 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q58 2170 2236 94 0.0 Infinity 1.0X +q58 1733 1762 42 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q59 2318 2322 6 0.0 Infinity 1.0X +q59 2021 2044 33 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q60 1806 1842 51 0.0 Infinity 1.0X +q60 1356 1404 67 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q61 1635 1643 10 0.0 Infinity 1.0X +q61 1290 1292 3 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q62 629 648 29 0.0 Infinity 1.0X +q62 479 506 33 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q63 708 730 21 0.0 Infinity 1.0X +q63 619 647 20 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q64 8943 9106 231 0.0 Infinity 1.0X +q64 7745 8352 859 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q65 1384 1432 68 0.0 Infinity 1.0X +q65 1307 1337 43 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q66 2225 2334 154 0.0 Infinity 1.0X +q66 1879 2128 352 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q67 9937 9994 80 0.0 Infinity 1.0X +q67 9682 9703 29 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q68 1152 1155 4 0.0 Infinity 1.0X +q68 928 952 34 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q69 3333 3393 86 0.0 Infinity 1.0X +q69 4261 4330 97 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q70 1472 1498 36 0.0 Infinity 1.0X +q70 1345 1361 23 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q71 1301 1328 38 0.0 Infinity 1.0X +q71 1103 1119 22 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q72 131085 131881 1126 0.0 Infinity 1.0X +q72 20211 21316 1562 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q73 844 859 26 0.0 Infinity 1.0X +q73 680 725 49 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q74 3230 3269 55 0.0 Infinity 1.0X +q74 3007 3109 144 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q75 5169 5503 472 0.0 Infinity 1.0X +q75 4597 4942 487 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q76 1149 1170 30 0.0 Infinity 1.0X +q76 899 937 38 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q77 2225 2534 437 0.0 Infinity 1.0X +q77 1794 2086 412 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q78 5088 5280 272 0.0 Infinity 1.0X +q78 4717 4827 155 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q79 1065 1069 6 0.0 Infinity 1.0X +q79 830 880 47 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q80 3555 3754 282 0.0 Infinity 1.0X +q80 3233 3315 116 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q81 1249 1253 6 0.0 Infinity 1.0X +q81 982 1070 123 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q82 2167 2186 26 0.0 Infinity 1.0X +q82 1674 1738 89 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q83 1379 1445 94 0.0 Infinity 1.0X +q83 1067 1104 52 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q84 1375 1397 31 0.0 Infinity 1.0X +q84 1166 1210 62 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q85 4318 4621 427 0.0 Infinity 1.0X +q85 3682 3831 211 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q86 715 726 16 0.0 Infinity 1.0X +q86 616 635 18 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q87 2486 2520 49 0.0 Infinity 1.0X +q87 2101 2230 183 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q88 3302 3430 182 0.0 Infinity 1.0X +q88 2415 2523 153 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q89 854 914 93 0.0 Infinity 1.0X +q89 677 732 47 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q90 622 635 10 0.0 Infinity 1.0X +q90 414 429 19 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q91 1033 1052 26 0.0 Infinity 1.0X +q91 793 814 19 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q92 649 684 31 0.0 Infinity 1.0X +q92 508 530 24 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q93 1331 1366 49 0.0 Infinity 1.0X +q93 1138 1155 24 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q94 1033 1042 13 0.0 Infinity 1.0X +q94 979 1060 115 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q95 5701 5709 12 0.0 Infinity 1.0X +q95 5805 6024 310 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q96 438 462 20 0.0 Infinity 1.0X +q96 337 345 10 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q97 1882 1956 104 0.0 Infinity 1.0X +q97 1641 1748 152 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q98 728 765 46 0.0 Infinity 1.0X +q98 538 587 61 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q99 728 736 8 0.0 Infinity 1.0X +q99 619 633 19 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q5a-v2.7 5933 6010 108 0.0 Infinity 1.0X +q5a-v2.7 4861 4954 132 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q6-v2.7 2322 2379 80 0.0 Infinity 1.0X +q6-v2.7 1798 1861 89 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q10a-v2.7 3946 3947 2 0.0 Infinity 1.0X +q10a-v2.7 4093 4209 164 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q11-v2.7 3784 3837 75 0.0 Infinity 1.0X +q11-v2.7 3336 3404 96 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q12-v2.7 475 541 75 0.0 Infinity 1.0X +q12-v2.7 380 408 30 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14-v2.7 19842 20008 235 0.0 Infinity 1.0X +q14-v2.7 17331 17776 629 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q14a-v2.7 127135 127517 539 0.0 Infinity 1.0X +q14a-v2.7 111982 112268 404 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q18a-v2.7 5413 6082 946 0.0 Infinity 1.0X +q18a-v2.7 4063 4659 843 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q20-v2.7 535 552 18 0.0 Infinity 1.0X +q20-v2.7 420 446 29 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22-v2.7 20164 20228 91 0.0 Infinity 1.0X +q22-v2.7 18976 19164 265 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q22a-v2.7 9719 9961 342 0.0 Infinity 1.0X +q22a-v2.7 9087 9281 275 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q24-v2.7 3210 3278 97 0.0 Infinity 1.0X +q24-v2.7 2817 2834 24 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q27a-v2.7 2729 2862 188 0.0 Infinity 1.0X +q27a-v2.7 2301 2401 141 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q34-v2.7 877 901 30 0.0 Infinity 1.0X +q34-v2.7 700 731 48 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35-v2.7 2976 3126 212 0.0 Infinity 1.0X +q35-v2.7 4158 4513 503 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q35a-v2.7 3449 3544 134 0.0 Infinity 1.0X +q35a-v2.7 3904 3979 106 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q36a-v2.7 2814 2850 51 0.0 Infinity 1.0X +q36a-v2.7 2430 2534 147 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q47-v2.7 4699 5077 534 0.0 Infinity 1.0X +q47-v2.7 4502 4808 433 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q49-v2.7 2202 2330 182 0.0 Infinity 1.0X +q49-v2.7 1904 2159 360 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q51a-v2.7 28603 28823 311 0.0 Infinity 1.0X +q51a-v2.7 27939 28264 460 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q57-v2.7 3007 3239 329 0.0 Infinity 1.0X +q57-v2.7 2813 2981 237 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q64-v2.7 8686 9350 940 0.0 Infinity 1.0X +q64-v2.7 8413 8612 282 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q67a-v2.7 18643 18876 330 0.0 Infinity 1.0X +q67a-v2.7 17696 17858 230 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q70a-v2.7 2990 3028 54 0.0 Infinity 1.0X +q70a-v2.7 2511 2562 71 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q72-v2.7 127577 127803 319 0.0 Infinity 1.0X +q72-v2.7 20209 22083 2650 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q74-v2.7 3086 3144 82 0.0 Infinity 1.0X +q74-v2.7 2870 2912 60 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q75-v2.7 5158 5431 386 0.0 Infinity 1.0X +q75-v2.7 4534 4870 475 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q77a-v2.7 5109 5406 420 0.0 Infinity 1.0X +q77a-v2.7 4010 4285 388 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q78-v2.7 4743 5234 694 0.0 Infinity 1.0X +q78-v2.7 4879 4969 127 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q80a-v2.7 6636 7007 525 0.0 Infinity 1.0X +q80a-v2.7 5338 5728 552 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q86a-v2.7 1651 1661 14 0.0 Infinity 1.0X +q86a-v2.7 1370 1391 29 0.0 Infinity 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-8u222-b10-1ubuntu1~18.04.1-b10 on Linux 4.15.0-1044-aws +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz TPCDS Snappy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -q98-v2.7 637 699 93 0.0 Infinity 1.0X +q98-v2.7 577 612 31 0.0 Infinity 1.0X diff --git a/sql/core/benchmarks/UDFBenchmark-jdk11-results.txt b/sql/core/benchmarks/UDFBenchmark-jdk11-results.txt index 3454c1bccda20..79daf73a95bb4 100644 --- a/sql/core/benchmarks/UDFBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/UDFBenchmark-jdk11-results.txt @@ -2,58 +2,58 @@ UDF with mixed input types ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int/string to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int/string to string wholestage off 289 302 18 0.3 2893.0 1.0X -long/nullable int/string to string wholestage on 141 165 20 0.7 1405.4 2.1X +long/nullable int/string to string wholestage off 279 320 58 0.4 2789.3 1.0X +long/nullable int/string to string wholestage on 182 188 7 0.6 1818.0 1.5X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int/string to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int/string to option wholestage off 112 120 11 0.9 1124.6 1.0X -long/nullable int/string to option wholestage on 68 77 8 1.5 681.3 1.7X +long/nullable int/string to option wholestage off 133 137 5 0.8 1332.3 1.0X +long/nullable int/string to option wholestage on 110 118 8 0.9 1097.3 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int/string to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int/string to primitive wholestage off 74 79 6 1.3 744.3 1.0X -long/nullable int/string to primitive wholestage on 71 83 10 1.4 713.5 1.0X +long/nullable int/string to primitive wholestage off 69 83 19 1.4 694.7 1.0X +long/nullable int/string to primitive wholestage on 66 81 18 1.5 657.5 1.1X ================================================================================================ UDF with primitive types ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int to string wholestage off 94 99 6 1.1 942.4 1.0X -long/nullable int to string wholestage on 58 60 1 1.7 584.3 1.6X +long/nullable int to string wholestage off 76 80 5 1.3 761.5 1.0X +long/nullable int to string wholestage on 65 67 2 1.5 654.2 1.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int to option wholestage off 43 46 5 2.3 425.7 1.0X -long/nullable int to option wholestage on 42 49 7 2.4 423.2 1.0X +long/nullable int to option wholestage off 49 50 2 2.0 487.9 1.0X +long/nullable int to option wholestage on 50 66 12 2.0 503.6 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int to primitive wholestage off 40 41 1 2.5 401.7 1.0X -long/nullable int to primitive wholestage on 40 44 5 2.5 400.1 1.0X +long/nullable int to primitive wholestage off 48 58 14 2.1 483.5 1.0X +long/nullable int to primitive wholestage on 46 52 6 2.2 456.8 1.1X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz UDF identity overhead: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Baseline 35 40 6 2.9 348.2 1.0X -With identity UDF 38 38 0 2.6 380.4 0.9X +Baseline 39 41 3 2.6 385.8 1.0X +With identity UDF 44 46 3 2.3 444.4 0.9X diff --git a/sql/core/benchmarks/UDFBenchmark-results.txt b/sql/core/benchmarks/UDFBenchmark-results.txt index 1490b4439640c..3f17a999094a3 100644 --- a/sql/core/benchmarks/UDFBenchmark-results.txt +++ b/sql/core/benchmarks/UDFBenchmark-results.txt @@ -2,58 +2,58 @@ UDF with mixed input types ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int/string to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int/string to string wholestage off 242 308 94 0.4 2416.5 1.0X -long/nullable int/string to string wholestage on 141 179 25 0.7 1410.3 1.7X +long/nullable int/string to string wholestage off 250 327 108 0.4 2500.6 1.0X +long/nullable int/string to string wholestage on 142 157 16 0.7 1421.2 1.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int/string to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int/string to option wholestage off 91 97 9 1.1 908.1 1.0X -long/nullable int/string to option wholestage on 77 91 12 1.3 774.4 1.2X +long/nullable int/string to option wholestage off 124 125 2 0.8 1237.8 1.0X +long/nullable int/string to option wholestage on 73 93 27 1.4 730.1 1.7X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int/string to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int/string to primitive wholestage off 68 75 9 1.5 684.7 1.0X -long/nullable int/string to primitive wholestage on 62 64 3 1.6 619.8 1.1X +long/nullable int/string to primitive wholestage off 66 69 4 1.5 658.8 1.0X +long/nullable int/string to primitive wholestage on 61 67 11 1.6 611.7 1.1X ================================================================================================ UDF with primitive types ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int to string wholestage off 67 68 0 1.5 672.2 1.0X -long/nullable int to string wholestage on 66 72 11 1.5 660.5 1.0X +long/nullable int to string wholestage off 66 67 0 1.5 663.9 1.0X +long/nullable int to string wholestage on 66 68 2 1.5 664.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int to option wholestage off 53 57 5 1.9 528.1 1.0X -long/nullable int to option wholestage on 41 44 4 2.4 410.2 1.3X +long/nullable int to option wholestage off 40 42 3 2.5 402.6 1.0X +long/nullable int to option wholestage on 40 42 2 2.5 401.3 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz long/nullable int to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -long/nullable int to primitive wholestage off 46 48 4 2.2 459.1 1.0X -long/nullable int to primitive wholestage on 40 40 0 2.5 398.0 1.2X +long/nullable int to primitive wholestage off 38 39 0 2.6 384.8 1.0X +long/nullable int to primitive wholestage on 39 45 5 2.5 392.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz UDF identity overhead: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Baseline 53 58 4 1.9 525.6 1.0X -With identity UDF 38 38 0 2.7 376.3 1.4X +Baseline 32 33 1 3.1 320.8 1.0X +With identity UDF 37 40 6 2.7 369.1 0.9X diff --git a/sql/core/benchmarks/WideSchemaBenchmark-jdk11-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..19ce183820c05 --- /dev/null +++ b/sql/core/benchmarks/WideSchemaBenchmark-jdk11-results.txt @@ -0,0 +1,145 @@ +================================================================================================ +parsing large select expressions +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +parsing large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 select expressions 8 15 8 0.0 8028037.0 1.0X +100 select expressions 15 18 3 0.0 14899892.0 0.5X +2500 select expressions 237 243 8 0.0 237252523.0 0.0X + + +================================================================================================ +many column field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +many column field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 cols x 100000 rows (read in-mem) 59 72 8 1.7 591.0 1.0X +1 cols x 100000 rows (exec in-mem) 57 81 15 1.8 566.0 1.0X +1 cols x 100000 rows (read parquet) 61 78 13 1.6 614.8 1.0X +1 cols x 100000 rows (write parquet) 147 158 10 0.7 1468.5 0.4X +100 cols x 1000 rows (read in-mem) 57 62 6 1.8 565.8 1.0X +100 cols x 1000 rows (exec in-mem) 76 83 10 1.3 758.7 0.8X +100 cols x 1000 rows (read parquet) 70 79 10 1.4 700.8 0.8X +100 cols x 1000 rows (write parquet) 150 162 11 0.7 1498.8 0.4X +2500 cols x 40 rows (read in-mem) 413 424 15 0.2 4134.4 0.1X +2500 cols x 40 rows (exec in-mem) 753 772 23 0.1 7528.2 0.1X +2500 cols x 40 rows (read parquet) 304 312 8 0.3 3044.6 0.2X +2500 cols x 40 rows (write parquet) 507 520 11 0.2 5069.3 0.1X + + +================================================================================================ +wide shallowly nested struct field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +wide shallowly nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 54 63 8 1.8 540.7 1.0X +1 wide x 100000 rows (exec in-mem) 67 77 11 1.5 671.8 0.8X +1 wide x 100000 rows (read parquet) 90 97 6 1.1 901.2 0.6X +1 wide x 100000 rows (write parquet) 150 163 11 0.7 1503.9 0.4X +100 wide x 1000 rows (read in-mem) 69 75 8 1.4 689.8 0.8X +100 wide x 1000 rows (exec in-mem) 111 148 96 0.9 1111.5 0.5X +100 wide x 1000 rows (read parquet) 181 241 35 0.6 1808.7 0.3X +100 wide x 1000 rows (write parquet) 164 180 27 0.6 1636.1 0.3X +2500 wide x 40 rows (read in-mem) 78 101 84 1.3 781.0 0.7X +2500 wide x 40 rows (exec in-mem) 943 966 37 0.1 9430.9 0.1X +2500 wide x 40 rows (read parquet) 1385 1453 95 0.1 13853.3 0.0X +2500 wide x 40 rows (write parquet) 175 190 19 0.6 1745.5 0.3X + + +================================================================================================ +deeply nested struct field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +deeply nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 deep x 100000 rows (read in-mem) 44 49 6 2.3 441.1 1.0X +1 deep x 100000 rows (exec in-mem) 54 59 6 1.9 536.4 0.8X +1 deep x 100000 rows (read parquet) 65 68 6 1.5 646.1 0.7X +1 deep x 100000 rows (write parquet) 141 147 9 0.7 1413.9 0.3X +100 deep x 1000 rows (read in-mem) 459 470 11 0.2 4592.9 0.1X +100 deep x 1000 rows (exec in-mem) 1736 1740 6 0.1 17355.1 0.0X +100 deep x 1000 rows (read parquet) 1638 1643 6 0.1 16382.2 0.0X +100 deep x 1000 rows (write parquet) 555 567 12 0.2 5548.4 0.1X +250 deep x 400 rows (read in-mem) 2556 2556 1 0.0 25558.5 0.0X +250 deep x 400 rows (exec in-mem) 10410 10416 8 0.0 104102.6 0.0X +250 deep x 400 rows (read parquet) 9670 9688 26 0.0 96699.1 0.0X +250 deep x 400 rows (write parquet) 2638 2642 5 0.0 26379.7 0.0X + + +================================================================================================ +bushy struct field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +bushy struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 x 1 deep x 100000 rows (read in-mem) 39 44 6 2.6 388.2 1.0X +1 x 1 deep x 100000 rows (exec in-mem) 48 50 4 2.1 477.4 0.8X +1 x 1 deep x 100000 rows (read parquet) 47 54 9 2.1 466.1 0.8X +1 x 1 deep x 100000 rows (write parquet) 135 141 5 0.7 1350.5 0.3X +128 x 8 deep x 1000 rows (read in-mem) 45 53 9 2.2 445.2 0.9X +128 x 8 deep x 1000 rows (exec in-mem) 155 160 4 0.6 1553.0 0.2X +128 x 8 deep x 1000 rows (read parquet) 173 217 31 0.6 1729.8 0.2X +128 x 8 deep x 1000 rows (write parquet) 139 154 10 0.7 1389.9 0.3X +1024 x 11 deep x 100 rows (read in-mem) 73 77 4 1.4 730.2 0.5X +1024 x 11 deep x 100 rows (exec in-mem) 733 738 8 0.1 7326.1 0.1X +1024 x 11 deep x 100 rows (read parquet) 652 660 8 0.2 6517.6 0.1X +1024 x 11 deep x 100 rows (write parquet) 171 186 20 0.6 1706.4 0.2X + + +================================================================================================ +wide array field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +wide array field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 43 46 4 2.3 429.7 1.0X +1 wide x 100000 rows (exec in-mem) 54 57 4 1.8 542.4 0.8X +1 wide x 100000 rows (read parquet) 82 87 8 1.2 816.6 0.5X +1 wide x 100000 rows (write parquet) 137 159 19 0.7 1374.9 0.3X +100 wide x 1000 rows (read in-mem) 37 39 4 2.7 367.1 1.2X +100 wide x 1000 rows (exec in-mem) 45 50 6 2.2 451.6 1.0X +100 wide x 1000 rows (read parquet) 52 57 5 1.9 520.8 0.8X +100 wide x 1000 rows (write parquet) 125 131 8 0.8 1247.0 0.3X +2500 wide x 40 rows (read in-mem) 35 39 4 2.9 348.8 1.2X +2500 wide x 40 rows (exec in-mem) 46 49 5 2.2 456.0 0.9X +2500 wide x 40 rows (read parquet) 51 55 6 2.0 508.3 0.8X +2500 wide x 40 rows (write parquet) 129 135 6 0.8 1287.3 0.3X + + +================================================================================================ +wide map field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +wide map field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 39 48 9 2.5 394.2 1.0X +1 wide x 100000 rows (exec in-mem) 51 56 9 1.9 514.4 0.8X +1 wide x 100000 rows (read parquet) 119 124 7 0.8 1195.0 0.3X +1 wide x 100000 rows (write parquet) 130 138 8 0.8 1299.8 0.3X +100 wide x 1000 rows (read in-mem) 31 32 3 3.3 306.5 1.3X +100 wide x 1000 rows (exec in-mem) 40 42 3 2.5 402.7 1.0X +100 wide x 1000 rows (read parquet) 65 70 6 1.5 651.8 0.6X +100 wide x 1000 rows (write parquet) 123 129 6 0.8 1228.5 0.3X +2500 wide x 40 rows (read in-mem) 33 37 6 3.0 330.1 1.2X +2500 wide x 40 rows (exec in-mem) 43 44 3 2.3 426.6 0.9X +2500 wide x 40 rows (read parquet) 66 69 9 1.5 657.8 0.6X +2500 wide x 40 rows (write parquet) 123 127 2 0.8 1234.4 0.3X + + diff --git a/sql/core/benchmarks/WideSchemaBenchmark-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-results.txt index 6347a6ac6b67c..59eb181f6b115 100644 --- a/sql/core/benchmarks/WideSchemaBenchmark-results.txt +++ b/sql/core/benchmarks/WideSchemaBenchmark-results.txt @@ -2,144 +2,144 @@ parsing large select expressions ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 select expressions 6 / 13 0.0 5997373.0 1.0X -100 select expressions 7 / 10 0.0 7204596.0 0.8X -2500 select expressions 103 / 107 0.0 102962705.0 0.1X +parsing large select: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 select expressions 5 13 8 0.0 5370143.0 1.0X +100 select expressions 12 16 6 0.0 11995425.0 0.4X +2500 select expressions 211 214 4 0.0 210927791.0 0.0X ================================================================================================ many column field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 cols x 100000 rows (read in-mem) 40 / 51 2.5 396.5 1.0X -1 cols x 100000 rows (exec in-mem) 41 / 48 2.4 414.4 1.0X -1 cols x 100000 rows (read parquet) 61 / 70 1.6 610.2 0.6X -1 cols x 100000 rows (write parquet) 209 / 233 0.5 2086.1 0.2X -100 cols x 1000 rows (read in-mem) 43 / 49 2.3 433.8 0.9X -100 cols x 1000 rows (exec in-mem) 57 / 66 1.8 568.4 0.7X -100 cols x 1000 rows (read parquet) 60 / 66 1.7 599.0 0.7X -100 cols x 1000 rows (write parquet) 212 / 224 0.5 2120.6 0.2X -2500 cols x 40 rows (read in-mem) 268 / 275 0.4 2676.5 0.1X -2500 cols x 40 rows (exec in-mem) 494 / 504 0.2 4936.9 0.1X -2500 cols x 40 rows (read parquet) 132 / 139 0.8 1319.7 0.3X -2500 cols x 40 rows (write parquet) 371 / 381 0.3 3710.1 0.1X +many column field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 cols x 100000 rows (read in-mem) 44 53 6 2.3 440.3 1.0X +1 cols x 100000 rows (exec in-mem) 44 54 9 2.3 437.0 1.0X +1 cols x 100000 rows (read parquet) 53 61 10 1.9 532.4 0.8X +1 cols x 100000 rows (write parquet) 129 142 36 0.8 1291.6 0.3X +100 cols x 1000 rows (read in-mem) 49 55 7 2.0 494.9 0.9X +100 cols x 1000 rows (exec in-mem) 69 73 5 1.4 693.2 0.6X +100 cols x 1000 rows (read parquet) 60 67 8 1.7 596.3 0.7X +100 cols x 1000 rows (write parquet) 142 156 31 0.7 1417.8 0.3X +2500 cols x 40 rows (read in-mem) 391 399 13 0.3 3912.6 0.1X +2500 cols x 40 rows (exec in-mem) 743 749 8 0.1 7432.5 0.1X +2500 cols x 40 rows (read parquet) 297 310 10 0.3 2972.8 0.1X +2500 cols x 40 rows (write parquet) 485 492 16 0.2 4848.1 0.1X ================================================================================================ wide shallowly nested struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 wide x 100000 rows (read in-mem) 37 / 43 2.7 373.6 1.0X -1 wide x 100000 rows (exec in-mem) 47 / 54 2.1 472.7 0.8X -1 wide x 100000 rows (read parquet) 132 / 145 0.8 1316.5 0.3X -1 wide x 100000 rows (write parquet) 205 / 232 0.5 2046.3 0.2X -100 wide x 1000 rows (read in-mem) 68 / 79 1.5 676.3 0.6X -100 wide x 1000 rows (exec in-mem) 88 / 97 1.1 882.2 0.4X -100 wide x 1000 rows (read parquet) 197 / 234 0.5 1971.8 0.2X -100 wide x 1000 rows (write parquet) 236 / 249 0.4 2359.6 0.2X -2500 wide x 40 rows (read in-mem) 77 / 85 1.3 768.0 0.5X -2500 wide x 40 rows (exec in-mem) 386 / 393 0.3 3855.2 0.1X -2500 wide x 40 rows (read parquet) 1741 / 1765 0.1 17408.3 0.0X -2500 wide x 40 rows (write parquet) 243 / 256 0.4 2425.2 0.2X +wide shallowly nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 43 48 6 2.3 427.0 1.0X +1 wide x 100000 rows (exec in-mem) 56 63 8 1.8 557.8 0.8X +1 wide x 100000 rows (read parquet) 82 88 10 1.2 818.6 0.5X +1 wide x 100000 rows (write parquet) 134 145 21 0.7 1344.6 0.3X +100 wide x 1000 rows (read in-mem) 55 61 16 1.8 553.1 0.8X +100 wide x 1000 rows (exec in-mem) 94 101 17 1.1 941.4 0.5X +100 wide x 1000 rows (read parquet) 151 179 29 0.7 1511.7 0.3X +100 wide x 1000 rows (write parquet) 147 157 9 0.7 1470.0 0.3X +2500 wide x 40 rows (read in-mem) 66 69 9 1.5 658.9 0.6X +2500 wide x 40 rows (exec in-mem) 853 871 30 0.1 8525.7 0.1X +2500 wide x 40 rows (read parquet) 1158 1296 195 0.1 11577.8 0.0X +2500 wide x 40 rows (write parquet) 157 173 23 0.6 1569.6 0.3X ================================================================================================ deeply nested struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -deeply nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 deep x 100000 rows (read in-mem) 35 / 42 2.9 350.2 1.0X -1 deep x 100000 rows (exec in-mem) 40 / 43 2.5 399.5 0.9X -1 deep x 100000 rows (read parquet) 69 / 73 1.4 691.6 0.5X -1 deep x 100000 rows (write parquet) 203 / 224 0.5 2025.9 0.2X -100 deep x 1000 rows (read in-mem) 70 / 75 1.4 703.7 0.5X -100 deep x 1000 rows (exec in-mem) 654 / 684 0.2 6539.9 0.1X -100 deep x 1000 rows (read parquet) 10503 / 10550 0.0 105030.5 0.0X -100 deep x 1000 rows (write parquet) 235 / 243 0.4 2353.2 0.1X -250 deep x 400 rows (read in-mem) 249 / 259 0.4 2492.6 0.1X -250 deep x 400 rows (exec in-mem) 3842 / 3854 0.0 38424.8 0.0X -250 deep x 400 rows (read parquet) 153080 / 153444 0.0 1530796.1 0.0X -250 deep x 400 rows (write parquet) 434 / 441 0.2 4344.6 0.1X +deeply nested struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 deep x 100000 rows (read in-mem) 37 41 6 2.7 374.5 1.0X +1 deep x 100000 rows (exec in-mem) 47 50 6 2.1 466.9 0.8X +1 deep x 100000 rows (read parquet) 58 61 7 1.7 577.7 0.6X +1 deep x 100000 rows (write parquet) 128 134 18 0.8 1282.2 0.3X +100 deep x 1000 rows (read in-mem) 345 350 5 0.3 3447.8 0.1X +100 deep x 1000 rows (exec in-mem) 1283 1283 0 0.1 12830.5 0.0X +100 deep x 1000 rows (read parquet) 1201 1205 7 0.1 12005.2 0.0X +100 deep x 1000 rows (write parquet) 436 443 9 0.2 4361.4 0.1X +250 deep x 400 rows (read in-mem) 1882 1883 1 0.1 18819.9 0.0X +250 deep x 400 rows (exec in-mem) 7705 7709 5 0.0 77054.4 0.0X +250 deep x 400 rows (read parquet) 7052 7087 50 0.0 70517.1 0.0X +250 deep x 400 rows (write parquet) 1978 1979 1 0.1 19780.3 0.0X ================================================================================================ bushy struct field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -bushy struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 x 1 deep x 100000 rows (read in-mem) 37 / 42 2.7 370.2 1.0X -1 x 1 deep x 100000 rows (exec in-mem) 43 / 47 2.4 425.3 0.9X -1 x 1 deep x 100000 rows (read parquet) 48 / 51 2.1 478.7 0.8X -1 x 1 deep x 100000 rows (write parquet) 204 / 215 0.5 2042.0 0.2X -128 x 8 deep x 1000 rows (read in-mem) 32 / 37 3.1 318.6 1.2X -128 x 8 deep x 1000 rows (exec in-mem) 91 / 96 1.1 906.6 0.4X -128 x 8 deep x 1000 rows (read parquet) 351 / 379 0.3 3510.3 0.1X -128 x 8 deep x 1000 rows (write parquet) 199 / 203 0.5 1988.3 0.2X -1024 x 11 deep x 100 rows (read in-mem) 73 / 76 1.4 730.4 0.5X -1024 x 11 deep x 100 rows (exec in-mem) 327 / 334 0.3 3267.2 0.1X -1024 x 11 deep x 100 rows (read parquet) 2063 / 2078 0.0 20629.2 0.0X -1024 x 11 deep x 100 rows (write parquet) 248 / 266 0.4 2475.1 0.1X +bushy struct field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 x 1 deep x 100000 rows (read in-mem) 34 39 7 2.9 341.5 1.0X +1 x 1 deep x 100000 rows (exec in-mem) 42 45 5 2.4 423.4 0.8X +1 x 1 deep x 100000 rows (read parquet) 42 45 6 2.4 423.8 0.8X +1 x 1 deep x 100000 rows (write parquet) 124 132 19 0.8 1240.4 0.3X +128 x 8 deep x 1000 rows (read in-mem) 39 42 6 2.6 387.3 0.9X +128 x 8 deep x 1000 rows (exec in-mem) 134 138 6 0.7 1342.5 0.3X +128 x 8 deep x 1000 rows (read parquet) 147 164 27 0.7 1468.2 0.2X +128 x 8 deep x 1000 rows (write parquet) 130 142 34 0.8 1297.7 0.3X +1024 x 11 deep x 100 rows (read in-mem) 64 68 11 1.6 639.3 0.5X +1024 x 11 deep x 100 rows (exec in-mem) 642 652 14 0.2 6416.9 0.1X +1024 x 11 deep x 100 rows (read parquet) 527 531 5 0.2 5268.1 0.1X +1024 x 11 deep x 100 rows (write parquet) 155 166 28 0.6 1545.0 0.2X ================================================================================================ wide array field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -wide array field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 wide x 100000 rows (read in-mem) 33 / 38 3.0 328.4 1.0X -1 wide x 100000 rows (exec in-mem) 40 / 44 2.5 402.7 0.8X -1 wide x 100000 rows (read parquet) 83 / 91 1.2 826.6 0.4X -1 wide x 100000 rows (write parquet) 204 / 218 0.5 2039.1 0.2X -100 wide x 1000 rows (read in-mem) 28 / 31 3.6 277.2 1.2X -100 wide x 1000 rows (exec in-mem) 34 / 37 2.9 343.2 1.0X -100 wide x 1000 rows (read parquet) 56 / 61 1.8 556.4 0.6X -100 wide x 1000 rows (write parquet) 202 / 206 0.5 2017.3 0.2X -2500 wide x 40 rows (read in-mem) 29 / 30 3.5 286.4 1.1X -2500 wide x 40 rows (exec in-mem) 33 / 39 3.0 330.2 1.0X -2500 wide x 40 rows (read parquet) 54 / 66 1.8 544.0 0.6X -2500 wide x 40 rows (write parquet) 196 / 208 0.5 1959.2 0.2X +wide array field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 36 39 5 2.7 364.2 1.0X +1 wide x 100000 rows (exec in-mem) 46 50 7 2.2 460.4 0.8X +1 wide x 100000 rows (read parquet) 75 78 8 1.3 749.8 0.5X +1 wide x 100000 rows (write parquet) 127 133 19 0.8 1266.0 0.3X +100 wide x 1000 rows (read in-mem) 31 33 4 3.2 309.9 1.2X +100 wide x 1000 rows (exec in-mem) 40 42 4 2.5 397.3 0.9X +100 wide x 1000 rows (read parquet) 49 52 7 2.0 488.6 0.7X +100 wide x 1000 rows (write parquet) 122 135 23 0.8 1216.2 0.3X +2500 wide x 40 rows (read in-mem) 31 32 3 3.3 305.7 1.2X +2500 wide x 40 rows (exec in-mem) 39 42 5 2.6 391.9 0.9X +2500 wide x 40 rows (read parquet) 48 51 7 2.1 482.9 0.8X +2500 wide x 40 rows (write parquet) 120 130 22 0.8 1203.6 0.3X ================================================================================================ wide map field read and write ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -wide map field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -1 wide x 100000 rows (read in-mem) 31 / 34 3.3 305.7 1.0X -1 wide x 100000 rows (exec in-mem) 39 / 44 2.6 390.0 0.8X -1 wide x 100000 rows (read parquet) 125 / 132 0.8 1250.5 0.2X -1 wide x 100000 rows (write parquet) 198 / 213 0.5 1979.9 0.2X -100 wide x 1000 rows (read in-mem) 21 / 23 4.7 212.7 1.4X -100 wide x 1000 rows (exec in-mem) 28 / 32 3.5 283.3 1.1X -100 wide x 1000 rows (read parquet) 68 / 73 1.5 683.0 0.4X -100 wide x 1000 rows (write parquet) 188 / 206 0.5 1882.1 0.2X -2500 wide x 40 rows (read in-mem) 25 / 28 4.0 252.2 1.2X -2500 wide x 40 rows (exec in-mem) 32 / 34 3.1 318.5 1.0X -2500 wide x 40 rows (read parquet) 69 / 73 1.4 691.5 0.4X -2500 wide x 40 rows (write parquet) 193 / 202 0.5 1932.8 0.2X +wide map field r/w: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +1 wide x 100000 rows (read in-mem) 35 40 8 2.9 348.8 1.0X +1 wide x 100000 rows (exec in-mem) 46 47 2 2.2 461.8 0.8X +1 wide x 100000 rows (read parquet) 124 127 7 0.8 1236.1 0.3X +1 wide x 100000 rows (write parquet) 125 138 26 0.8 1245.4 0.3X +100 wide x 1000 rows (read in-mem) 26 35 8 3.8 263.1 1.3X +100 wide x 1000 rows (exec in-mem) 35 41 10 2.8 351.8 1.0X +100 wide x 1000 rows (read parquet) 59 62 8 1.7 586.7 0.6X +100 wide x 1000 rows (write parquet) 116 125 32 0.9 1158.2 0.3X +2500 wide x 40 rows (read in-mem) 27 30 5 3.7 270.2 1.3X +2500 wide x 40 rows (exec in-mem) 37 38 3 2.7 366.4 1.0X +2500 wide x 40 rows (read parquet) 58 62 8 1.7 584.3 0.6X +2500 wide x 40 rows (write parquet) 118 126 24 0.9 1176.1 0.3X diff --git a/sql/core/benchmarks/WideTableBenchmark-jdk11-results.txt b/sql/core/benchmarks/WideTableBenchmark-jdk11-results.txt index 5ba8e8ff9a62b..8f3920db0dcd9 100644 --- a/sql/core/benchmarks/WideTableBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/WideTableBenchmark-jdk11-results.txt @@ -2,16 +2,16 @@ projection on wide table ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz projection on wide table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -split threshold 10 42942 43023 108 0.0 40952.8 1.0X -split threshold 100 37236 37310 93 0.0 35510.9 1.2X -split threshold 1024 12198 12260 56 0.1 11633.1 3.5X -split threshold 2048 9789 9860 76 0.1 9335.2 4.4X -split threshold 4096 10120 10161 63 0.1 9651.0 4.2X -split threshold 8192 9961 9972 7 0.1 9499.1 4.3X -split threshold 65536 58987 59087 95 0.0 56254.5 0.7X +split threshold 10 8915 9048 180 0.1 8501.7 1.0X +split threshold 100 4419 4465 31 0.2 4214.2 2.0X +split threshold 1024 2477 2559 57 0.4 2362.4 3.6X +split threshold 2048 2314 2391 74 0.5 2206.7 3.9X +split threshold 4096 2374 2399 18 0.4 2264.2 3.8X +split threshold 8192 2831 2846 11 0.4 2699.7 3.1X +split threshold 65536 26886 26944 55 0.0 25640.7 0.3X diff --git a/sql/core/benchmarks/WideTableBenchmark-results.txt b/sql/core/benchmarks/WideTableBenchmark-results.txt index eeb8c34232382..4f64f877ac0c8 100644 --- a/sql/core/benchmarks/WideTableBenchmark-results.txt +++ b/sql/core/benchmarks/WideTableBenchmark-results.txt @@ -2,16 +2,16 @@ projection on wide table ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz projection on wide table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -split threshold 10 43737 44157 345 0.0 41711.3 1.0X -split threshold 100 33514 35770 1455 0.0 31961.3 1.3X -split threshold 1024 14959 15206 256 0.1 14265.8 2.9X -split threshold 2048 12024 12193 171 0.1 11467.2 3.6X -split threshold 4096 11378 11869 450 0.1 10851.1 3.8X -split threshold 8192 10588 10797 154 0.1 10097.8 4.1X -split threshold 65536 56801 57025 130 0.0 54169.4 0.8X +split threshold 10 6461 6535 84 0.2 6161.9 1.0X +split threshold 100 3643 3725 74 0.3 3474.5 1.8X +split threshold 1024 2217 2255 26 0.5 2113.9 2.9X +split threshold 2048 1941 2003 60 0.5 1851.5 3.3X +split threshold 4096 2195 2220 20 0.5 2093.4 2.9X +split threshold 8192 2592 2652 39 0.4 2472.4 2.5X +split threshold 65536 26324 26365 66 0.0 25104.1 0.2X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 2776bc310fefe..965d78227c335 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -48,7 +48,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { runBenchmark("aggregate without grouping") { val N = 500L << 22 codegenBenchmark("agg w/o group", N) { - spark.range(N).selectExpr("sum(id)").collect() + spark.range(N).selectExpr("sum(id)").noop() } } @@ -56,11 +56,11 @@ object AggregateBenchmark extends SqlBasedBenchmark { val N = 100L << 20 codegenBenchmark("stddev", N) { - spark.range(N).groupBy().agg("id" -> "stddev").collect() + spark.range(N).groupBy().agg("id" -> "stddev").noop() } codegenBenchmark("kurtosis", N) { - spark.range(N).groupBy().agg("id" -> "kurtosis").collect() + spark.range(N).groupBy().agg("id" -> "kurtosis").noop() } } @@ -70,7 +70,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { val benchmark = new Benchmark("Aggregate w keys", N, output = output) def f(): Unit = { - spark.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + spark.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().noop() } benchmark.addCase("codegen = F", numIters = 2) { _ => @@ -107,7 +107,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { spark.range(N).selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") - def f(): Unit = spark.sql("select k, k, sum(id) from test group by k, k").collect() + def f(): Unit = spark.sql("select k, k, sum(id) from test group by k, k").noop() benchmark.addCase("codegen = F", numIters = 2) { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { @@ -142,7 +142,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { val benchmark = new Benchmark("Aggregate w string key", N, output = output) def f(): Unit = spark.range(N).selectExpr("id", "cast(id & 1023 as string) as k") - .groupBy("k").count().collect() + .groupBy("k").count().noop() benchmark.addCase("codegen = F", numIters = 2) { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { @@ -177,7 +177,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { val benchmark = new Benchmark("Aggregate w decimal key", N, output = output) def f(): Unit = spark.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") - .groupBy("k").count().collect() + .groupBy("k").count().noop() benchmark.addCase("codegen = F") { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { @@ -222,7 +222,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { "id > 1023 as k6") .groupBy("k1", "k2", "k3", "k4", "k5", "k6") .sum() - .collect() + .noop() benchmark.addCase("codegen = F") { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { @@ -282,7 +282,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { "case when id > 1800 and id <= 1900 then 1 else 0 end as v18") .groupBy("k1", "k2", "k3") .sum() - .collect() + .noop() benchmark.addCase("codegen = F") { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { @@ -315,7 +315,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { codegenBenchmark("cube", N) { spark.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") - .cube("k1", "k2").sum("id").collect() + .cube("k1", "k2").sum("id").noop() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala index f727ebcf3fd1e..ae241b3625d02 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala @@ -70,10 +70,10 @@ object BloomFilterBenchmark extends SqlBasedBenchmark { runBenchmark(s"ORC Read") { val benchmark = new Benchmark(s"Read a row from ${scaleFactor}M rows", N, output = output) benchmark.addCase("Without bloom filter") { _ => - spark.read.orc(path + "/withoutBF").where("value = 0").count + spark.read.orc(path + "/withoutBF").where("value = 0").noop() } benchmark.addCase("With bloom filter") { _ => - spark.read.orc(path + "/withBF").where("value = 0").count + spark.read.orc(path + "/withBF").where("value = 0").noop() } benchmark.run() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala index df122977fe5f2..a084bec985510 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala @@ -122,30 +122,30 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark { prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1")) sqlBenchmark.addCase("SQL CSV") { _ => - spark.sql("select sum(id) from csvTable").collect() + spark.sql("select sum(id) from csvTable").noop() } sqlBenchmark.addCase("SQL Json") { _ => - spark.sql("select sum(id) from jsonTable").collect() + spark.sql("select sum(id) from jsonTable").noop() } sqlBenchmark.addCase("SQL Parquet Vectorized") { _ => - spark.sql("select sum(id) from parquetTable").collect() + spark.sql("select sum(id) from parquetTable").noop() } sqlBenchmark.addCase("SQL Parquet MR") { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("select sum(id) from parquetTable").collect() + spark.sql("select sum(id) from parquetTable").noop() } } sqlBenchmark.addCase("SQL ORC Vectorized") { _ => - spark.sql("SELECT sum(id) FROM orcTable").collect() + spark.sql("SELECT sum(id) FROM orcTable").noop() } sqlBenchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(id) FROM orcTable").collect() + spark.sql("SELECT sum(id) FROM orcTable").noop() } } @@ -238,30 +238,30 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark { spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1")) benchmark.addCase("SQL CSV") { _ => - spark.sql("select sum(c1), sum(length(c2)) from csvTable").collect() + spark.sql("select sum(c1), sum(length(c2)) from csvTable").noop() } benchmark.addCase("SQL Json") { _ => - spark.sql("select sum(c1), sum(length(c2)) from jsonTable").collect() + spark.sql("select sum(c1), sum(length(c2)) from jsonTable").noop() } benchmark.addCase("SQL Parquet Vectorized") { _ => - spark.sql("select sum(c1), sum(length(c2)) from parquetTable").collect() + spark.sql("select sum(c1), sum(length(c2)) from parquetTable").noop() } benchmark.addCase("SQL Parquet MR") { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("select sum(c1), sum(length(c2)) from parquetTable").collect() + spark.sql("select sum(c1), sum(length(c2)) from parquetTable").noop() } } benchmark.addCase("SQL ORC Vectorized") { _ => - spark.sql("SELECT sum(c1), sum(length(c2)) FROM orcTable").collect() + spark.sql("SELECT sum(c1), sum(length(c2)) FROM orcTable").noop() } benchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(c1), sum(length(c2)) FROM orcTable").collect() + spark.sql("SELECT sum(c1), sum(length(c2)) FROM orcTable").noop() } } @@ -283,30 +283,30 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark { spark.sql("select cast((value % 200) + 10000 as STRING) as c1 from t1")) benchmark.addCase("SQL CSV") { _ => - spark.sql("select sum(length(c1)) from csvTable").collect() + spark.sql("select sum(length(c1)) from csvTable").noop() } benchmark.addCase("SQL Json") { _ => - spark.sql("select sum(length(c1)) from jsonTable").collect() + spark.sql("select sum(length(c1)) from jsonTable").noop() } benchmark.addCase("SQL Parquet Vectorized") { _ => - spark.sql("select sum(length(c1)) from parquetTable").collect() + spark.sql("select sum(length(c1)) from parquetTable").noop() } benchmark.addCase("SQL Parquet MR") { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("select sum(length(c1)) from parquetTable").collect() + spark.sql("select sum(length(c1)) from parquetTable").noop() } } benchmark.addCase("SQL ORC Vectorized") { _ => - spark.sql("select sum(length(c1)) from orcTable").collect() + spark.sql("select sum(length(c1)) from orcTable").noop() } benchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("select sum(length(c1)) from orcTable").collect() + spark.sql("select sum(length(c1)) from orcTable").noop() } } @@ -326,86 +326,86 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark { prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p")) benchmark.addCase("Data column - CSV") { _ => - spark.sql("select sum(id) from csvTable").collect() + spark.sql("select sum(id) from csvTable").noop() } benchmark.addCase("Data column - Json") { _ => - spark.sql("select sum(id) from jsonTable").collect() + spark.sql("select sum(id) from jsonTable").noop() } benchmark.addCase("Data column - Parquet Vectorized") { _ => - spark.sql("select sum(id) from parquetTable").collect() + spark.sql("select sum(id) from parquetTable").noop() } benchmark.addCase("Data column - Parquet MR") { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("select sum(id) from parquetTable").collect() + spark.sql("select sum(id) from parquetTable").noop() } } benchmark.addCase("Data column - ORC Vectorized") { _ => - spark.sql("SELECT sum(id) FROM orcTable").collect() + spark.sql("SELECT sum(id) FROM orcTable").noop() } benchmark.addCase("Data column - ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(id) FROM orcTable").collect() + spark.sql("SELECT sum(id) FROM orcTable").noop() } } benchmark.addCase("Partition column - CSV") { _ => - spark.sql("select sum(p) from csvTable").collect() + spark.sql("select sum(p) from csvTable").noop() } benchmark.addCase("Partition column - Json") { _ => - spark.sql("select sum(p) from jsonTable").collect() + spark.sql("select sum(p) from jsonTable").noop() } benchmark.addCase("Partition column - Parquet Vectorized") { _ => - spark.sql("select sum(p) from parquetTable").collect() + spark.sql("select sum(p) from parquetTable").noop() } benchmark.addCase("Partition column - Parquet MR") { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("select sum(p) from parquetTable").collect() + spark.sql("select sum(p) from parquetTable").noop() } } benchmark.addCase("Partition column - ORC Vectorized") { _ => - spark.sql("SELECT sum(p) FROM orcTable").collect() + spark.sql("SELECT sum(p) FROM orcTable").noop() } benchmark.addCase("Partition column - ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(p) FROM orcTable").collect() + spark.sql("SELECT sum(p) FROM orcTable").noop() } } benchmark.addCase("Both columns - CSV") { _ => - spark.sql("select sum(p), sum(id) from csvTable").collect() + spark.sql("select sum(p), sum(id) from csvTable").noop() } benchmark.addCase("Both columns - Json") { _ => - spark.sql("select sum(p), sum(id) from jsonTable").collect() + spark.sql("select sum(p), sum(id) from jsonTable").noop() } benchmark.addCase("Both columns - Parquet Vectorized") { _ => - spark.sql("select sum(p), sum(id) from parquetTable").collect() + spark.sql("select sum(p), sum(id) from parquetTable").noop() } benchmark.addCase("Both columns - Parquet MR") { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("select sum(p), sum(id) from parquetTable").collect + spark.sql("select sum(p), sum(id) from parquetTable").noop() } } benchmark.addCase("Both columns - ORC Vectorized") { _ => - spark.sql("SELECT sum(p), sum(id) FROM orcTable").collect() + spark.sql("SELECT sum(p), sum(id) FROM orcTable").noop() } benchmark.addCase("Both columns - ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(p), sum(id) FROM orcTable").collect() + spark.sql("SELECT sum(p), sum(id) FROM orcTable").noop() } } @@ -431,23 +431,23 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("SQL CSV") { _ => spark.sql("select sum(length(c2)) from csvTable where c1 is " + - "not NULL and c2 is not NULL").collect() + "not NULL and c2 is not NULL").noop() } benchmark.addCase("SQL Json") { _ => spark.sql("select sum(length(c2)) from jsonTable where c1 is " + - "not NULL and c2 is not NULL").collect() + "not NULL and c2 is not NULL").noop() } benchmark.addCase("SQL Parquet Vectorized") { _ => spark.sql("select sum(length(c2)) from parquetTable where c1 is " + - "not NULL and c2 is not NULL").collect() + "not NULL and c2 is not NULL").noop() } benchmark.addCase("SQL Parquet MR") { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("select sum(length(c2)) from parquetTable where c1 is " + - "not NULL and c2 is not NULL").collect() + "not NULL and c2 is not NULL").noop() } } @@ -478,13 +478,13 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("SQL ORC Vectorized") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM orcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").noop() } benchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT SUM(LENGTH(c2)) FROM orcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").noop() } } @@ -510,30 +510,30 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark { prepareTable(dir, spark.sql("SELECT * FROM t1")) benchmark.addCase("SQL CSV") { _ => - spark.sql(s"SELECT sum(c$middle) FROM csvTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM csvTable").noop() } benchmark.addCase("SQL Json") { _ => - spark.sql(s"SELECT sum(c$middle) FROM jsonTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM jsonTable").noop() } benchmark.addCase("SQL Parquet Vectorized") { _ => - spark.sql(s"SELECT sum(c$middle) FROM parquetTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM parquetTable").noop() } benchmark.addCase("SQL Parquet MR") { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql(s"SELECT sum(c$middle) FROM parquetTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM parquetTable").noop() } } benchmark.addCase("SQL ORC Vectorized") { _ => - spark.sql(s"SELECT sum(c$middle) FROM orcTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM orcTable").noop() } benchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql(s"SELECT sum(c$middle) FROM orcTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM orcTable").noop() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala index 9cfa9070aca67..92dadccde207c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.benchmark import java.sql.Timestamp import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.internal.SQLConf /** @@ -39,10 +38,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { private def doBenchmark(cardinality: Int, exprs: String*): Unit = { spark.range(cardinality) .selectExpr(exprs: _*) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } private def run(cardinality: Int, name: String, exprs: String*): Unit = { @@ -138,10 +134,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { benchmark.addCase("From java.sql.Timestamp", numIters) { _ => spark.range(rowsNum) .map(millis => new Timestamp(millis)) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } benchmark.addCase("Collect longs", numIters) { _ => spark.range(0, rowsNum, 1, 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index 941649df6b727..de23132284dc8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.benchmark import java.time.Instant import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.internal.SQLConf /** @@ -43,10 +42,7 @@ object ExtractBenchmark extends SqlBasedBenchmark { spark .range(sinceSecond, sinceSecond + cardinality, 1, 1) .selectExpr(exprs: _*) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 50ba50176c7f5..444ffa4f99697 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -119,7 +119,7 @@ object FilterPushdownBenchmark extends SqlBasedBenchmark { val name = s"Parquet Vectorized ${if (pushDownEnabled) s"(Pushdown)" else ""}" benchmark.addCase(name) { _ => withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> s"$pushDownEnabled") { - spark.sql(s"SELECT $selectExpr FROM parquetTable WHERE $whereExpr").collect() + spark.sql(s"SELECT $selectExpr FROM parquetTable WHERE $whereExpr").noop() } } } @@ -128,7 +128,7 @@ object FilterPushdownBenchmark extends SqlBasedBenchmark { val name = s"Native ORC Vectorized ${if (pushDownEnabled) s"(Pushdown)" else ""}" benchmark.addCase(name) { _ => withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> s"$pushDownEnabled") { - spark.sql(s"SELECT $selectExpr FROM orcTable WHERE $whereExpr").collect() + spark.sql(s"SELECT $selectExpr FROM orcTable WHERE $whereExpr").noop() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala index 6a6851ab60e5c..caf3387875813 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala @@ -167,7 +167,7 @@ object InExpressionBenchmark extends SqlBasedBenchmark { def testClosure(): Unit = { val df = spark.sql(s"SELECT * FROM t WHERE id IN (${values.mkString(",")})") - df.queryExecution.toRdd.foreach(_ => ()) + df.noop() } benchmark.addCase("In expression") { _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala index b9bb6f5febd7f..94e763459a111 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/IntervalBenchmark.scala @@ -21,7 +21,6 @@ import scala.collection.mutable.ListBuffer import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.Column -import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -45,10 +44,7 @@ object IntervalBenchmark extends SqlBasedBenchmark { spark .range(0, cardinality, 1, 1) .select(exprs: _*) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala index f4786368bd9e4..1cc92892fe122 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala @@ -46,7 +46,7 @@ object JoinBenchmark extends SqlBasedBenchmark { codegenBenchmark("Join w long", N) { val df = spark.range(N).join(dim, (col("id") % M) === col("k")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) - df.count() + df.noop() } } @@ -57,7 +57,7 @@ object JoinBenchmark extends SqlBasedBenchmark { codegenBenchmark("Join w long duplicated", N) { val df = spark.range(N).join(dim, (col("id") % M) === col("k")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) - df.count() + df.noop() } } @@ -72,7 +72,7 @@ object JoinBenchmark extends SqlBasedBenchmark { (col("id") % M).cast(IntegerType) === col("k1") && (col("id") % M).cast(IntegerType) === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) - df.count() + df.noop() } } @@ -86,7 +86,7 @@ object JoinBenchmark extends SqlBasedBenchmark { val df = spark.range(N).join(dim3, (col("id") % M) === col("k1") && (col("id") % M) === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) - df.count() + df.noop() } } @@ -100,7 +100,7 @@ object JoinBenchmark extends SqlBasedBenchmark { val df = spark.range(N).join(dim4, (col("id") bitwiseAND M) === col("k1") && (col("id") bitwiseAND M) === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) - df.count() + df.noop() } } @@ -111,7 +111,7 @@ object JoinBenchmark extends SqlBasedBenchmark { codegenBenchmark("outer join w long", N) { val df = spark.range(N).join(dim, (col("id") % M) === col("k"), "left") assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) - df.count() + df.noop() } } @@ -122,7 +122,7 @@ object JoinBenchmark extends SqlBasedBenchmark { codegenBenchmark("semi join w long", N) { val df = spark.range(N).join(dim, (col("id") % M) === col("k"), "leftsemi") assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) - df.count() + df.noop() } } @@ -133,7 +133,7 @@ object JoinBenchmark extends SqlBasedBenchmark { val df2 = spark.range(N).selectExpr(s"id * 3 as k2") val df = df1.join(df2, col("k1") === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined) - df.count() + df.noop() } } @@ -146,7 +146,7 @@ object JoinBenchmark extends SqlBasedBenchmark { .selectExpr(s"(id * 15485867) % ${N*10} as k2") val df = df1.join(df2, col("k1") === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined) - df.count() + df.noop() } } @@ -161,7 +161,7 @@ object JoinBenchmark extends SqlBasedBenchmark { val df2 = spark.range(N / 3).selectExpr(s"id * 3 as k2") val df = df1.join(df2, col("k1") === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[ShuffledHashJoinExec]).isDefined) - df.count() + df.noop() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala index 9e5aca70ac628..c92098c93aa1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.internal.SQLConf /** @@ -40,10 +39,7 @@ object MakeDateTimeBenchmark extends SqlBasedBenchmark { spark .range(0, cardinality, 1, 1) .selectExpr(exprs: _*) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala index bafc0337bdc0e..2aecf553d75a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala @@ -35,7 +35,7 @@ object MiscBenchmark extends SqlBasedBenchmark { def filterAndAggregateWithoutGroup(numRows: Long): Unit = { runBenchmark("filter & aggregate without group") { codegenBenchmark("range/filter/sum", numRows) { - spark.range(numRows).filter("(id & 1) = 1").groupBy().sum().collect() + spark.range(numRows).filter("(id & 1) = 1").groupBy().sum().noop() } } } @@ -43,7 +43,7 @@ object MiscBenchmark extends SqlBasedBenchmark { def limitAndAggregateWithoutGroup(numRows: Long): Unit = { runBenchmark("range/limit/sum") { codegenBenchmark("range/limit/sum", numRows) { - spark.range(numRows).limit(1000000).groupBy().sum().collect() + spark.range(numRows).limit(1000000).groupBy().sum().noop() } } } @@ -51,11 +51,11 @@ object MiscBenchmark extends SqlBasedBenchmark { def sample(numRows: Int): Unit = { runBenchmark("sample") { codegenBenchmark("sample with replacement", numRows) { - spark.range(numRows).sample(withReplacement = true, 0.01).groupBy().sum().collect() + spark.range(numRows).sample(withReplacement = true, 0.01).groupBy().sum().noop() } codegenBenchmark("sample without replacement", numRows) { - spark.range(numRows).sample(withReplacement = false, 0.01).groupBy().sum().collect() + spark.range(numRows).sample(withReplacement = false, 0.01).groupBy().sum().noop() } } } @@ -95,28 +95,28 @@ object MiscBenchmark extends SqlBasedBenchmark { val df = spark.range(numRows).selectExpr( "id as key", "array(rand(), rand(), rand(), rand(), rand()) as values") - df.selectExpr("key", "explode(values) value").count() + df.selectExpr("key", "explode(values) value").noop() } codegenBenchmark("generate explode map", numRows) { val df = spark.range(numRows).selectExpr( "id as key", "map('a', rand(), 'b', rand(), 'c', rand(), 'd', rand(), 'e', rand()) pairs") - df.selectExpr("key", "explode(pairs) as (k, v)").count() + df.selectExpr("key", "explode(pairs) as (k, v)").noop() } codegenBenchmark("generate posexplode array", numRows) { val df = spark.range(numRows).selectExpr( "id as key", "array(rand(), rand(), rand(), rand(), rand()) as values") - df.selectExpr("key", "posexplode(values) as (idx, value)").count() + df.selectExpr("key", "posexplode(values) as (idx, value)").noop() } codegenBenchmark("generate inline array", numRows) { val df = spark.range(numRows).selectExpr( "id as key", "array((rand(), rand()), (rand(), rand()), (rand(), 0.0d)) as values") - df.selectExpr("key", "inline(values) as (r1, r2)").count() + df.selectExpr("key", "inline(values) as (r1, r2)").noop() } val M = 60000 @@ -129,7 +129,7 @@ object MiscBenchmark extends SqlBasedBenchmark { })))).toDF("col", "arr") df.selectExpr("*", "explode(arr) as arr_col") - .select("col", "arr_col.*").count + .select("col", "arr_col.*").noop() } withSQLConf(SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> "true") { @@ -142,7 +142,7 @@ object MiscBenchmark extends SqlBasedBenchmark { })))).toDF("col", "arr") .selectExpr("col", "struct(col, arr) as st") .selectExpr("col", "st.col as col1", "explode(st.arr) as arr_col") - df.collect() + df.noop() } } } @@ -158,7 +158,7 @@ object MiscBenchmark extends SqlBasedBenchmark { "id % 5 as t3", "id % 7 as t4", "id % 13 as t5") - df.selectExpr("key", "stack(4, t1, t2, t3, t4, t5)").count() + df.selectExpr("key", "stack(4, t1, t2, t3, t4, t5)").noop() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala index 0734c6e18deea..90fad7f36b862 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.internal.SQLConf /** @@ -48,11 +47,7 @@ abstract class NestedSchemaPruningBenchmark extends SqlBasedBenchmark { private def addCase(benchmark: Benchmark, name: String, sql: String): Unit = { benchmark.addCase(name) { _ => - spark.sql(sql) - .write - .format("noop") - .mode(Overwrite) - .save() + spark.sql(sql).noop() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala index a9f873f9094ba..e566f5d5adee6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala @@ -40,15 +40,15 @@ object RangeBenchmark extends SqlBasedBenchmark { val benchmark = new Benchmark("range", N, output = output) benchmark.addCase("full scan", numIters = 4) { _ => - spark.range(N).queryExecution.toRdd.foreach(_ => ()) + spark.range(N).noop() } benchmark.addCase("limit after range", numIters = 4) { _ => - spark.range(N).limit(100).queryExecution.toRdd.foreach(_ => ()) + spark.range(N).limit(100).noop() } benchmark.addCase("filter after range", numIters = 4) { _ => - spark.range(N).filter('id % 100 === 0).queryExecution.toRdd.foreach(_ => ()) + spark.range(N).filter('id % 100 === 0).noop() } benchmark.addCase("count after range", numIters = 4) { _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala index e95e5a960246b..ee7a03e5e0542 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.internal.SQLConf @@ -57,4 +58,10 @@ trait SqlBasedBenchmark extends BenchmarkBase with SQLHelper { benchmark.run() } + + implicit class DatasetToBenchmark(ds: Dataset[_]) { + def noop(): Unit = { + ds.write.format("noop").mode(Overwrite).save() + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala index 5ff33b9cfbfc9..be02447db2e55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -91,7 +91,7 @@ object TPCDSQueryBenchmark extends SqlBasedBenchmark { val numRows = queryRelations.map(tableSizes.getOrElse(_, 0L)).sum val benchmark = new Benchmark(s"TPCDS Snappy", numRows, 2, output = output) benchmark.addCase(s"$name$nameSuffix") { _ => - spark.sql(queryString).collect() + spark.sql(queryString).noop() } benchmark.run() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala index 04c1b5ade12cb..ee8a6e787c36c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ @@ -45,10 +44,7 @@ object UDFBenchmark extends SqlBasedBenchmark { val stringCol = idCol.cast(StringType) spark.range(cardinality) .select(udf(idCol, nullableIntCol, stringCol)) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } private def doRunBenchmarkWithPrimitiveTypes( @@ -58,10 +54,7 @@ object UDFBenchmark extends SqlBasedBenchmark { idCol % 2 === 0, idCol.cast(IntegerType)).otherwise(Literal(null, IntegerType)) spark.range(cardinality) .select(udf(idCol, nullableIntCol)) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { @@ -116,10 +109,7 @@ object UDFBenchmark extends SqlBasedBenchmark { benchmark.addCase(s"Baseline", numIters = 5) { _ => spark.range(cardinality) .select(col("id"), col("id") * 2, col("id") * 3) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } val identityUDF = udf { x: Long => x } @@ -129,10 +119,7 @@ object UDFBenchmark extends SqlBasedBenchmark { identityUDF(col("id")), identityUDF(col("id") * 2), identityUDF(col("id") * 3)) - .write - .format("noop") - .mode(Overwrite) - .save() + .noop() } benchmark.run() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala index 683d398faeeab..77dc3a10f8033 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -70,14 +70,14 @@ object WideSchemaBenchmark extends SqlBasedBenchmark { desc: String, selector: String): Unit = { benchmark.addCase(desc + " (read in-mem)") { iter => - df.selectExpr(s"sum($selector)").collect() + df.selectExpr(s"sum($selector)").noop() } benchmark.addCase(desc + " (exec in-mem)") { iter => - df.selectExpr("*", s"hash($selector) as f").selectExpr(s"sum($selector)", "sum(f)").collect() + df.selectExpr("*", s"hash($selector) as f").selectExpr(s"sum($selector)", "sum(f)").noop() } val parquet = saveAsParquet(df) benchmark.addCase(desc + " (read parquet)") { iter => - parquet.selectExpr(s"sum($selector) as f").collect() + parquet.selectExpr(s"sum($selector) as f").noop() } benchmark.addCase(desc + " (write parquet)") { iter => saveAsParquet(df.selectExpr(s"sum($selector) as f")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideTableBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideTableBenchmark.scala index 52426d81bd1a7..ba79c12c461c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideTableBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideTableBenchmark.scala @@ -42,7 +42,7 @@ object WideTableBenchmark extends SqlBasedBenchmark { Seq("10", "100", "1024", "2048", "4096", "8192", "65536").foreach { n => benchmark.addCase(s"split threshold $n", numIters = 5) { iter => withSQLConf(SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> n) { - df.selectExpr(columns: _*).foreach(_ => ()) + df.selectExpr(columns: _*).noop() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala index a4cffedaf82d7..ad80afa441de1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala @@ -21,7 +21,6 @@ import java.time.{Instant, LocalDate} import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{Column, Dataset, Row} -import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -43,10 +42,6 @@ import org.apache.spark.sql.types._ object CSVBenchmark extends SqlBasedBenchmark { import spark.implicits._ - private def toNoop(ds: Dataset[_]): Unit = { - ds.write.format("noop").mode(Overwrite).save() - } - private def quotedValuesBenchmark(rowsNum: Int, numIters: Int): Unit = { val benchmark = new Benchmark(s"Parsing quoted values", rowsNum, output = output) @@ -62,7 +57,7 @@ object CSVBenchmark extends SqlBasedBenchmark { val ds = spark.read.option("header", true).schema(schema).csv(path.getAbsolutePath) benchmark.addCase(s"One quoted string", numIters) { _ => - toNoop(ds) + ds.noop() } benchmark.run() @@ -87,14 +82,14 @@ object CSVBenchmark extends SqlBasedBenchmark { val ds = spark.read.schema(schema).csv(path.getAbsolutePath) benchmark.addCase(s"Select $colsNum columns", numIters) { _ => - toNoop(ds.select("*")) + ds.select("*").noop() } val cols100 = columnNames.take(100).map(Column(_)) benchmark.addCase(s"Select 100 columns", numIters) { _ => - toNoop(ds.select(cols100: _*)) + ds.select(cols100: _*).noop() } benchmark.addCase(s"Select one column", numIters) { _ => - toNoop(ds.select($"col1")) + ds.select($"col1").noop() } benchmark.addCase(s"count()", numIters) { _ => ds.count() @@ -104,7 +99,7 @@ object CSVBenchmark extends SqlBasedBenchmark { (1 until colsNum).map(i => StructField(s"col$i", IntegerType))) val dsErr1 = spark.read.schema(schemaErr1).csv(path.getAbsolutePath) benchmark.addCase(s"Select 100 columns, one bad input field", numIters) { _ => - toNoop(dsErr1.select(cols100: _*)) + dsErr1.select(cols100: _*).noop() } val badRecColName = "badRecord" @@ -113,7 +108,7 @@ object CSVBenchmark extends SqlBasedBenchmark { .option("columnNameOfCorruptRecord", badRecColName) .csv(path.getAbsolutePath) benchmark.addCase(s"Select 100 columns, corrupt record field", numIters) { _ => - toNoop(dsErr2.select((Column(badRecColName) +: cols100): _*)) + dsErr2.select((Column(badRecColName) +: cols100): _*).noop() } benchmark.run() @@ -170,11 +165,11 @@ object CSVBenchmark extends SqlBasedBenchmark { val writeBench = new Benchmark("Write dates and timestamps", rowsNum, output = output) writeBench.addCase(s"Create a dataset of timestamps", numIters) { _ => - toNoop(timestamps) + timestamps.noop() } writeBench.addCase("to_csv(timestamp)", numIters) { _ => - toNoop(timestamps.select(to_csv(struct($"timestamp")))) + timestamps.select(to_csv(struct($"timestamp"))).noop() } writeBench.addCase("write timestamps to files", numIters) { _ => @@ -182,11 +177,11 @@ object CSVBenchmark extends SqlBasedBenchmark { } writeBench.addCase("Create a dataset of dates", numIters) { _ => - toNoop(dates) + dates.noop() } writeBench.addCase("to_csv(date)", numIters) { _ => - toNoop(dates.select(to_csv(struct($"date")))) + dates.select(to_csv(struct($"date"))).noop() } writeBench.addCase("write dates to files", numIters) { _ => @@ -199,7 +194,7 @@ object CSVBenchmark extends SqlBasedBenchmark { val tsSchema = new StructType().add("timestamp", TimestampType) readBench.addCase("read timestamp text from files", numIters) { _ => - toNoop(spark.read.text(timestampDir)) + spark.read.text(timestampDir).noop() } readBench.addCase("read timestamps from files", numIters) { _ => @@ -207,7 +202,7 @@ object CSVBenchmark extends SqlBasedBenchmark { .option("header", true) .schema(tsSchema) .csv(timestampDir) - toNoop(ds) + ds.noop() } readBench.addCase("infer timestamps from files", numIters) { _ => @@ -215,13 +210,13 @@ object CSVBenchmark extends SqlBasedBenchmark { .option("header", true) .option("inferSchema", true) .csv(timestampDir) - toNoop(ds) + ds.noop() } val dateSchema = new StructType().add("date", DateType) readBench.addCase("read date text from files", numIters) { _ => - toNoop(spark.read.text(dateDir)) + spark.read.text(dateDir).noop() } readBench.addCase("read date from files", numIters) { _ => @@ -229,7 +224,7 @@ object CSVBenchmark extends SqlBasedBenchmark { .option("header", true) .schema(dateSchema) .csv(dateDir) - toNoop(ds) + ds.noop() } readBench.addCase("infer date from files", numIters) { _ => @@ -237,7 +232,7 @@ object CSVBenchmark extends SqlBasedBenchmark { .option("header", true) .option("inferSchema", true) .csv(dateDir) - toNoop(ds) + ds.noop() } def timestampStr: Dataset[String] = { @@ -247,7 +242,7 @@ object CSVBenchmark extends SqlBasedBenchmark { } readBench.addCase("timestamp strings", numIters) { _ => - toNoop(timestampStr) + timestampStr.noop() } readBench.addCase("parse timestamps from Dataset[String]", numIters) { _ => @@ -255,7 +250,7 @@ object CSVBenchmark extends SqlBasedBenchmark { .option("header", false) .schema(tsSchema) .csv(timestampStr) - toNoop(ds) + ds.noop() } readBench.addCase("infer timestamps from Dataset[String]", numIters) { _ => @@ -263,7 +258,7 @@ object CSVBenchmark extends SqlBasedBenchmark { .option("header", false) .option("inferSchema", true) .csv(timestampStr) - toNoop(ds) + ds.noop() } def dateStr: Dataset[String] = { @@ -273,7 +268,7 @@ object CSVBenchmark extends SqlBasedBenchmark { } readBench.addCase("date strings", numIters) { _ => - toNoop(dateStr) + dateStr.noop() } readBench.addCase("parse dates from Dataset[String]", numIters) { _ => @@ -281,17 +276,17 @@ object CSVBenchmark extends SqlBasedBenchmark { .option("header", false) .schema(dateSchema) .csv(dateStr) - toNoop(ds) + ds.noop() } readBench.addCase("from_csv(timestamp)", numIters) { _ => val ds = timestampStr.select(from_csv($"timestamp", tsSchema, Map.empty[String, String])) - toNoop(ds) + ds.noop() } readBench.addCase("from_csv(date)", numIters) { _ => val ds = dateStr.select(from_csv($"date", dateSchema, Map.empty[String, String])) - toNoop(ds) + ds.noop() } readBench.run() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala index 58e710a7d66b6..bcecaccc8cc89 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala @@ -21,7 +21,6 @@ import java.time.{Instant, LocalDate} import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{Dataset, Row} -import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -49,10 +48,6 @@ object JsonBenchmark extends SqlBasedBenchmark { // scalastyle:on println } - private def run(ds: Dataset[_]): Unit = { - ds.write.format("noop").mode(Overwrite).save() - } - def schemaInferring(rowsNum: Int, numIters: Int): Unit = { val benchmark = new Benchmark("JSON schema inferring", rowsNum, output = output) @@ -220,11 +215,11 @@ object JsonBenchmark extends SqlBasedBenchmark { benchmark.addCase(s"Select $colsNum columns", numIters) { _ => val ds = in.select("*") - run(ds) + ds.noop() } benchmark.addCase(s"Select 1 column", numIters) { _ => val ds = in.select($"col1") - run(ds) + ds.noop() } benchmark.run() @@ -245,7 +240,7 @@ object JsonBenchmark extends SqlBasedBenchmark { benchmark.addCase("Short column without encoding", numIters) { _ => val ds = spark.read.schema(shortSchema).json(shortColumnPath) - run(ds) + ds.noop() } benchmark.addCase("Short column with UTF-8", numIters) { _ => @@ -253,12 +248,12 @@ object JsonBenchmark extends SqlBasedBenchmark { .option("encoding", "UTF-8") .schema(shortSchema) .json(shortColumnPath) - run(ds) + ds.noop() } benchmark.addCase("Wide column without encoding", numIters) { _ => val ds = spark.read.schema(wideSchema).json(wideColumnPath) - run(ds) + ds.noop() } benchmark.addCase("Wide column with UTF-8", numIters) { _ => @@ -266,7 +261,7 @@ object JsonBenchmark extends SqlBasedBenchmark { .option("encoding", "UTF-8") .schema(wideSchema) .json(wideColumnPath) - run(ds) + ds.noop() } benchmark.run() @@ -281,23 +276,23 @@ object JsonBenchmark extends SqlBasedBenchmark { val in = spark.range(0, rows, 1, 1).map(_ => """{"a":1}""") benchmark.addCase("Text read", iters) { _ => - run(in) + in.noop() } benchmark.addCase("from_json", iters) { _ => val schema = new StructType().add("a", IntegerType) val from_json_ds = in.select(from_json('value, schema)) - run(from_json_ds) + from_json_ds.noop() } benchmark.addCase("json_tuple", iters) { _ => val json_tuple_ds = in.select(json_tuple($"value", "a")) - run(json_tuple_ds) + json_tuple_ds.noop() } benchmark.addCase("get_json_object", iters) { _ => val get_json_object_ds = in.select(get_json_object($"value", "$.a")) - run(get_json_object_ds) + get_json_object_ds.noop() } benchmark.run() @@ -311,7 +306,7 @@ object JsonBenchmark extends SqlBasedBenchmark { val in = spark.range(0, rows, 1, 1).map(_ => """{"a":1}""") benchmark.addCase("Text read", iters) { _ => - run(in) + in.noop() } benchmark.addCase("schema inferring", iters) { _ => @@ -323,7 +318,7 @@ object JsonBenchmark extends SqlBasedBenchmark { val ds = spark.read .schema(schema) .json(in) - run(ds) + ds.noop() } benchmark.run() @@ -344,7 +339,7 @@ object JsonBenchmark extends SqlBasedBenchmark { val ds = spark.read .format("text") .load(path.getAbsolutePath) - run(ds) + ds.noop() } benchmark.addCase("Schema inferring", iters) { _ => @@ -361,7 +356,7 @@ object JsonBenchmark extends SqlBasedBenchmark { .schema(schema) .option("multiLine", false) .json(path.getAbsolutePath) - run(ds) + ds.noop() } benchmark.addCase("Parsing with UTF-8", iters) { _ => @@ -371,7 +366,7 @@ object JsonBenchmark extends SqlBasedBenchmark { .option("charset", "UTF-8") .json(path.getAbsolutePath) - run(ds) + ds.noop() } benchmark.run() @@ -398,11 +393,11 @@ object JsonBenchmark extends SqlBasedBenchmark { val writeBench = new Benchmark("Write dates and timestamps", rowsNum, output = output) writeBench.addCase(s"Create a dataset of timestamps", numIters) { _ => - run(timestamps) + timestamps.noop() } writeBench.addCase("to_json(timestamp)", numIters) { _ => - run(timestamps.select(to_json(struct($"timestamp")))) + timestamps.select(to_json(struct($"timestamp"))).noop() } writeBench.addCase("write timestamps to files", numIters) { _ => @@ -410,11 +405,11 @@ object JsonBenchmark extends SqlBasedBenchmark { } writeBench.addCase("Create a dataset of dates", numIters) { _ => - run(dates) + dates.noop() } writeBench.addCase("to_json(date)", numIters) { _ => - run(dates.select(to_json(struct($"date")))) + dates.select(to_json(struct($"date"))).noop() } writeBench.addCase("write dates to files", numIters) { _ => @@ -427,25 +422,25 @@ object JsonBenchmark extends SqlBasedBenchmark { val tsSchema = new StructType().add("timestamp", TimestampType) readBench.addCase("read timestamp text from files", numIters) { _ => - run(spark.read.text(timestampDir)) + spark.read.text(timestampDir).noop() } readBench.addCase("read timestamps from files", numIters) { _ => - run(spark.read.schema(tsSchema).json(timestampDir)) + spark.read.schema(tsSchema).json(timestampDir).noop() } readBench.addCase("infer timestamps from files", numIters) { _ => - run(spark.read.json(timestampDir)) + spark.read.json(timestampDir).noop() } val dateSchema = new StructType().add("date", DateType) readBench.addCase("read date text from files", numIters) { _ => - run(spark.read.text(dateDir)) + spark.read.text(dateDir).noop() } readBench.addCase("read date from files", numIters) { _ => - run(spark.read.schema(dateSchema).json(dateDir)) + spark.read.schema(dateSchema).json(dateDir).noop() } def timestampStr: Dataset[String] = { @@ -455,15 +450,15 @@ object JsonBenchmark extends SqlBasedBenchmark { } readBench.addCase("timestamp strings", numIters) { _ => - run(timestampStr) + timestampStr.noop() } readBench.addCase("parse timestamps from Dataset[String]", numIters) { _ => - run(spark.read.schema(tsSchema).json(timestampStr)) + spark.read.schema(tsSchema).json(timestampStr).noop() } readBench.addCase("infer timestamps from Dataset[String]", numIters) { _ => - run(spark.read.json(timestampStr)) + spark.read.json(timestampStr).noop() } def dateStr: Dataset[String] = { @@ -473,7 +468,7 @@ object JsonBenchmark extends SqlBasedBenchmark { } readBench.addCase("date strings", numIters) { _ => - run(dateStr) + dateStr.noop() } readBench.addCase("parse dates from Dataset[String]", numIters) { _ => @@ -481,17 +476,17 @@ object JsonBenchmark extends SqlBasedBenchmark { .option("header", false) .schema(dateSchema) .json(dateStr) - run(ds) + ds.noop() } readBench.addCase("from_json(timestamp)", numIters) { _ => val ds = timestampStr.select(from_json($"timestamp", tsSchema, Map.empty[String, String])) - run(ds) + ds.noop() } readBench.addCase("from_json(date)", numIters) { _ => val ds = dateStr.select(from_json($"date", dateSchema, Map.empty[String, String])) - run(ds) + ds.noop() } readBench.run() diff --git a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-jdk11-results.txt b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..4a8058766319f --- /dev/null +++ b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-jdk11-results.txt @@ -0,0 +1,45 @@ +================================================================================================ +Hive UDAF vs Spark AF +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +hive udaf vs spark af: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +hive udaf w/o group by 6492 7169 388 0.0 99066.1 1.0X +spark af w/o group by 58 88 24 1.1 890.2 111.3X +hive udaf w/ group by 4864 4888 33 0.0 74221.0 1.3X +spark af w/ group by w/o fallback 60 67 7 1.1 912.9 108.5X +spark af w/ group by w/ fallback 154 164 27 0.4 2348.2 42.2X + + +================================================================================================ +ObjectHashAggregateExec vs SortAggregateExec - typed_count +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort agg w/ group by 51728 51728 0 2.0 493.3 1.0X +object agg w/ group by w/o fallback 10174 10218 34 10.3 97.0 5.1X +object agg w/ group by w/ fallback 29341 29537 277 3.6 279.8 1.8X +sort agg w/o group by 7541 7577 28 13.9 71.9 6.9X +object agg w/o group by w/o fallback 5574 5620 38 18.8 53.2 9.3X + + +================================================================================================ +ObjectHashAggregateExec vs SortAggregateExec - percentile_approx +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +sort agg w/ group by 900 925 14 2.3 429.0 1.0X +object agg w/ group by w/o fallback 597 633 14 3.5 284.6 1.5X +object agg w/ group by w/ fallback 905 923 10 2.3 431.6 1.0X +sort agg w/o group by 611 631 10 3.4 291.4 1.5X +object agg w/o group by w/o fallback 559 576 11 3.8 266.5 1.6X + + diff --git a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt index 0c394a340333a..8c58a5a5fdf0b 100644 --- a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt +++ b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt @@ -2,44 +2,44 @@ Hive UDAF vs Spark AF ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hive udaf vs spark af: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hive udaf w/o group by 6741 6759 22 0.0 102864.5 1.0X -spark af w/o group by 56 66 9 1.2 851.6 120.8X -hive udaf w/ group by 4610 4642 25 0.0 70350.3 1.5X -spark af w/ group by w/o fallback 60 67 8 1.1 916.7 112.2X -spark af w/ group by w/ fallback 135 144 9 0.5 2065.6 49.8X +hive udaf w/o group by 7014 7206 120 0.0 107031.0 1.0X +spark af w/o group by 47 59 11 1.4 716.9 149.3X +hive udaf w/ group by 4811 4831 28 0.0 73409.1 1.5X +spark af w/ group by w/o fallback 50 56 7 1.3 762.9 140.3X +spark af w/ group by w/ fallback 126 130 8 0.5 1916.6 55.8X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - typed_count ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort agg w/ group by 41568 41894 461 2.5 396.4 1.0X -object agg w/ group by w/o fallback 10314 10494 149 10.2 98.4 4.0X -object agg w/ group by w/ fallback 26720 26951 326 3.9 254.8 1.6X -sort agg w/o group by 6638 6681 38 15.8 63.3 6.3X -object agg w/o group by w/o fallback 5665 5706 30 18.5 54.0 7.3X +sort agg w/ group by 42969 43306 476 2.4 409.8 1.0X +object agg w/ group by w/o fallback 9744 9844 145 10.8 92.9 4.4X +object agg w/ group by w/ fallback 26814 26960 206 3.9 255.7 1.6X +sort agg w/o group by 6278 6330 57 16.7 59.9 6.8X +object agg w/o group by w/o fallback 5433 5478 60 19.3 51.8 7.9X ================================================================================================ ObjectHashAggregateExec vs SortAggregateExec - percentile_approx ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz object agg v.s. sort agg: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -sort agg w/ group by 794 862 33 2.6 378.8 1.0X -object agg w/ group by w/o fallback 605 622 10 3.5 288.5 1.3X -object agg w/ group by w/ fallback 840 860 15 2.5 400.5 0.9X -sort agg w/o group by 555 570 12 3.8 264.6 1.4X -object agg w/o group by w/o fallback 544 562 12 3.9 259.6 1.5X +sort agg w/ group by 756 773 9 2.8 360.3 1.0X +object agg w/ group by w/o fallback 548 560 7 3.8 261.3 1.4X +object agg w/ group by w/ fallback 759 773 7 2.8 362.0 1.0X +sort agg w/o group by 471 483 13 4.4 224.8 1.6X +object agg w/o group by w/o fallback 471 482 12 4.5 224.7 1.6X diff --git a/sql/hive/benchmarks/OrcReadBenchmark-jdk11-results.txt b/sql/hive/benchmarks/OrcReadBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..d516d3369ad05 --- /dev/null +++ b/sql/hive/benchmarks/OrcReadBenchmark-jdk11-results.txt @@ -0,0 +1,156 @@ +================================================================================================ +SQL Single Numeric Column Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1750 1872 173 9.0 111.2 1.0X +Native ORC Vectorized 433 499 68 36.3 27.5 4.0X +Hive built-in ORC 2540 2575 49 6.2 161.5 0.7X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1979 2001 31 7.9 125.8 1.0X +Native ORC Vectorized 261 303 42 60.3 16.6 7.6X +Hive built-in ORC 2559 2583 34 6.1 162.7 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2094 2158 91 7.5 133.2 1.0X +Native ORC Vectorized 309 361 41 50.8 19.7 6.8X +Hive built-in ORC 2649 2744 135 5.9 168.4 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2256 2271 22 7.0 143.4 1.0X +Native ORC Vectorized 511 518 11 30.8 32.5 4.4X +Hive built-in ORC 2867 2880 19 5.5 182.3 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2270 2325 78 6.9 144.3 1.0X +Native ORC Vectorized 502 508 5 31.3 31.9 4.5X +Hive built-in ORC 2862 2880 24 5.5 182.0 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 2376 2426 71 6.6 151.0 1.0X +Native ORC Vectorized 609 616 8 25.8 38.7 3.9X +Hive built-in ORC 2979 2991 17 5.3 189.4 0.8X + + +================================================================================================ +Int and String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 4112 4232 170 2.6 392.1 1.0X +Native ORC Vectorized 2199 2223 35 4.8 209.7 1.9X +Hive built-in ORC 5150 5238 123 2.0 491.2 0.8X + + +================================================================================================ +Partitioned Table Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Data column - Native ORC MR 2398 2435 53 6.6 152.4 1.0X +Data column - Native ORC Vectorized 458 482 26 34.3 29.1 5.2X +Data column - Hive built-in ORC 3126 3171 64 5.0 198.8 0.8X +Partition column - Native ORC MR 1639 1680 58 9.6 104.2 1.5X +Partition column - Native ORC Vectorized 105 119 11 149.6 6.7 22.8X +Partition column - Hive built-in ORC 2223 2229 8 7.1 141.4 1.1X +Both columns - Native ORC MR 2588 2608 28 6.1 164.5 0.9X +Both columns - Native ORC Vectorized 489 522 49 32.2 31.1 4.9X +Both columns - Hive built-in ORC 3258 3292 48 4.8 207.1 0.7X + + +================================================================================================ +Repeated String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1991 2028 52 5.3 189.9 1.0X +Native ORC Vectorized 392 398 8 26.7 37.4 5.1X +Hive built-in ORC 2810 2816 8 3.7 268.0 0.7X + + +================================================================================================ +String with Nulls Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 3638 3647 13 2.9 346.9 1.0X +Native ORC Vectorized 1171 1181 14 9.0 111.7 3.1X +Hive built-in ORC 4847 4871 34 2.2 462.2 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 3280 3283 5 3.2 312.8 1.0X +Native ORC Vectorized 1199 1206 10 8.7 114.4 2.7X +Hive built-in ORC 4263 4273 14 2.5 406.5 0.8X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 1935 1950 21 5.4 184.6 1.0X +Native ORC Vectorized 451 459 10 23.2 43.1 4.3X +Hive built-in ORC 2542 2552 14 4.1 242.4 0.8X + + +================================================================================================ +Single Column Scan From Wide Columns +================================================================================================ + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 270 292 23 3.9 257.2 1.0X +Native ORC Vectorized 143 155 12 7.3 136.2 1.9X +Hive built-in ORC 1593 1627 48 0.7 1519.1 0.2X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 369 386 17 2.8 351.5 1.0X +Native ORC Vectorized 218 231 15 4.8 208.3 1.7X +Hive built-in ORC 3092 3101 12 0.3 2949.1 0.1X + +OpenJDK 64-Bit Server VM 11.0.5+10-post-Ubuntu-0ubuntu1.118.04 on Linux 4.15.0-1044-aws +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC MR 498 531 33 2.1 475.0 1.0X +Native ORC Vectorized 360 376 18 2.9 342.9 1.4X +Hive built-in ORC 4786 4786 1 0.2 4564.1 0.1X + + diff --git a/sql/hive/benchmarks/OrcReadBenchmark-results.txt b/sql/hive/benchmarks/OrcReadBenchmark-results.txt index c47cf27bf617a..c7d6c976192b2 100644 --- a/sql/hive/benchmarks/OrcReadBenchmark-results.txt +++ b/sql/hive/benchmarks/OrcReadBenchmark-results.txt @@ -2,155 +2,155 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 1843 1958 162 8.5 117.2 1.0X -Native ORC Vectorized 321 355 31 48.9 20.4 5.7X -Hive built-in ORC 2143 2175 44 7.3 136.3 0.9X +Native ORC MR 1844 1851 10 8.5 117.2 1.0X +Native ORC Vectorized 284 312 36 55.5 18.0 6.5X +Hive built-in ORC 2380 2380 1 6.6 151.3 0.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 1987 2020 47 7.9 126.3 1.0X -Native ORC Vectorized 276 299 25 57.0 17.6 7.2X -Hive built-in ORC 2350 2357 10 6.7 149.4 0.8X +Native ORC MR 1999 2031 45 7.9 127.1 1.0X +Native ORC Vectorized 252 264 15 62.5 16.0 7.9X +Hive built-in ORC 2483 2509 37 6.3 157.9 0.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 2092 2115 32 7.5 133.0 1.0X -Native ORC Vectorized 360 373 18 43.6 22.9 5.8X -Hive built-in ORC 2550 2557 9 6.2 162.2 0.8X +Native ORC MR 2134 2135 2 7.4 135.7 1.0X +Native ORC Vectorized 329 351 34 47.8 20.9 6.5X +Hive built-in ORC 2672 2716 61 5.9 169.9 0.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 2173 2188 21 7.2 138.2 1.0X -Native ORC Vectorized 435 448 14 36.2 27.7 5.0X -Hive built-in ORC 2683 2690 10 5.9 170.6 0.8X +Native ORC MR 2172 2247 105 7.2 138.1 1.0X +Native ORC Vectorized 407 427 23 38.7 25.9 5.3X +Hive built-in ORC 2806 2822 22 5.6 178.4 0.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 2233 2323 127 7.0 142.0 1.0X -Native ORC Vectorized 475 483 13 33.1 30.2 4.7X -Hive built-in ORC 2605 2610 6 6.0 165.7 0.9X +Native ORC MR 2187 2200 19 7.2 139.0 1.0X +Native ORC Vectorized 451 457 5 34.9 28.7 4.8X +Hive built-in ORC 2886 2938 73 5.4 183.5 0.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 2367 2384 24 6.6 150.5 1.0X -Native ORC Vectorized 600 641 69 26.2 38.1 3.9X -Hive built-in ORC 2860 2877 24 5.5 181.9 0.8X +Native ORC MR 2313 2319 9 6.8 147.1 1.0X +Native ORC Vectorized 554 562 7 28.4 35.2 4.2X +Hive built-in ORC 2927 2933 8 5.4 186.1 0.8X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 4253 4330 108 2.5 405.6 1.0X -Native ORC Vectorized 2295 2301 8 4.6 218.9 1.9X -Hive built-in ORC 5364 5465 144 2.0 511.5 0.8X +Native ORC MR 4162 4294 186 2.5 397.0 1.0X +Native ORC Vectorized 2236 2258 32 4.7 213.2 1.9X +Hive built-in ORC 5054 5135 114 2.1 482.0 0.8X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Data column - Native ORC MR 2443 2448 6 6.4 155.3 1.0X -Data column - Native ORC Vectorized 446 473 44 35.3 28.3 5.5X -Data column - Hive built-in ORC 2868 2877 12 5.5 182.4 0.9X -Partition column - Native ORC MR 1623 1656 47 9.7 103.2 1.5X -Partition column - Native ORC Vectorized 112 121 14 140.8 7.1 21.9X -Partition column - Hive built-in ORC 1846 1850 5 8.5 117.4 1.3X -Both columns - Native ORC MR 2610 2635 36 6.0 165.9 0.9X -Both columns - Native ORC Vectorized 492 508 19 32.0 31.3 5.0X -Both columns - Hive built-in ORC 2969 2973 4 5.3 188.8 0.8X +Data column - Native ORC MR 2436 2447 16 6.5 154.8 1.0X +Data column - Native ORC Vectorized 421 443 35 37.4 26.8 5.8X +Data column - Hive built-in ORC 3007 3026 27 5.2 191.2 0.8X +Partition column - Native ORC MR 1603 1630 39 9.8 101.9 1.5X +Partition column - Native ORC Vectorized 84 96 15 186.7 5.4 28.9X +Partition column - Hive built-in ORC 2174 2187 18 7.2 138.2 1.1X +Both columns - Native ORC MR 2609 2645 51 6.0 165.9 0.9X +Both columns - Native ORC Vectorized 460 470 9 34.2 29.3 5.3X +Both columns - Hive built-in ORC 3094 3099 8 5.1 196.7 0.8X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 2056 2064 11 5.1 196.1 1.0X -Native ORC Vectorized 415 421 7 25.3 39.6 5.0X -Hive built-in ORC 2710 2722 17 3.9 258.4 0.8X +Native ORC MR 2036 2046 13 5.1 194.2 1.0X +Native ORC Vectorized 366 386 18 28.6 34.9 5.6X +Hive built-in ORC 2683 2686 4 3.9 255.9 0.8X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (0.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 3655 3674 27 2.9 348.6 1.0X -Native ORC Vectorized 1166 1167 1 9.0 111.2 3.1X -Hive built-in ORC 5268 5305 52 2.0 502.4 0.7X +Native ORC MR 3614 3643 40 2.9 344.7 1.0X +Native ORC Vectorized 1072 1087 22 9.8 102.2 3.4X +Hive built-in ORC 4625 4636 15 2.3 441.1 0.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (50.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 3447 3467 27 3.0 328.8 1.0X -Native ORC Vectorized 1222 1223 1 8.6 116.6 2.8X -Hive built-in ORC 3947 3959 18 2.7 376.4 0.9X +Native ORC MR 3347 3376 42 3.1 319.2 1.0X +Native ORC Vectorized 1220 1225 7 8.6 116.3 2.7X +Hive built-in ORC 4168 4184 23 2.5 397.5 0.8X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (95.0%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 1912 1917 6 5.5 182.4 1.0X -Native ORC Vectorized 477 484 5 22.0 45.5 4.0X -Hive built-in ORC 2374 2386 17 4.4 226.4 0.8X +Native ORC MR 1851 1862 16 5.7 176.5 1.0X +Native ORC Vectorized 466 471 7 22.5 44.4 4.0X +Hive built-in ORC 2523 2529 8 4.2 240.6 0.7X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 290 350 102 3.6 276.1 1.0X -Native ORC Vectorized 155 166 15 6.7 148.2 1.9X -Hive built-in ORC 520 531 8 2.0 495.8 0.6X +Native ORC MR 250 264 15 4.2 238.1 1.0X +Native ORC Vectorized 121 138 24 8.7 115.5 2.1X +Hive built-in ORC 1761 1792 43 0.6 1679.3 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 200 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 365 406 73 2.9 347.9 1.0X -Native ORC Vectorized 232 246 20 4.5 221.6 1.6X -Hive built-in ORC 794 864 62 1.3 757.6 0.5X +Native ORC MR 319 341 17 3.3 304.5 1.0X +Native ORC Vectorized 188 222 50 5.6 178.8 1.7X +Hive built-in ORC 3492 3508 24 0.3 3329.8 0.1X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_232-8u232-b09-0ubuntu1~18.04.1-b09 on Linux 4.15.0-1044-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 300 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Native ORC MR 501 544 40 2.1 477.6 1.0X -Native ORC Vectorized 365 386 33 2.9 348.0 1.4X -Hive built-in ORC 1153 1153 0 0.9 1099.8 0.4X +Native ORC MR 443 456 12 2.4 422.9 1.0X +Native ORC Vectorized 306 321 23 3.4 292.0 1.4X +Hive built-in ORC 5295 5312 24 0.2 5049.9 0.1X diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala index c475c7b21ab95..3f806ad24ca10 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala @@ -70,13 +70,13 @@ object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { benchmark.addCase("hive udaf w/o group by") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { - sql("SELECT hive_percentile_approx(id, 0.5) FROM t").collect() + sql("SELECT hive_percentile_approx(id, 0.5) FROM t").noop() } } benchmark.addCase("spark af w/o group by") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { - sql("SELECT percentile_approx(id, 0.5) FROM t").collect() + sql("SELECT percentile_approx(id, 0.5) FROM t").noop() } } @@ -84,14 +84,14 @@ object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { sql( s"SELECT hive_percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)" - ).collect() + ).noop() } } benchmark.addCase("spark af w/ group by w/o fallback") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { sql(s"SELECT percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)") - .collect() + .noop() } } @@ -100,7 +100,7 @@ object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { SQLConf.USE_OBJECT_HASH_AGG.key -> "true", SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "2") { sql(s"SELECT percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)") - .collect() + .noop() } } @@ -125,13 +125,13 @@ object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { benchmark.addCase("sort agg w/ group by") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { - df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).noop() } } benchmark.addCase("object agg w/ group by w/o fallback") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { - df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).noop() } } @@ -139,19 +139,19 @@ object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.USE_OBJECT_HASH_AGG.key -> "true", SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "2") { - df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).noop() } } benchmark.addCase("sort agg w/o group by") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { - df.select(typed_count($"id")).collect() + df.select(typed_count($"id")).noop() } } benchmark.addCase("object agg w/o group by w/o fallback") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { - df.select(typed_count($"id")).collect() + df.select(typed_count($"id")).noop() } } @@ -173,13 +173,13 @@ object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { benchmark.addCase("sort agg w/ group by") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { - df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).noop() } } benchmark.addCase("object agg w/ group by w/o fallback") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { - df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).noop() } } @@ -187,19 +187,19 @@ object ObjectHashAggregateExecBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.USE_OBJECT_HASH_AGG.key -> "true", SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "2") { - df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).noop() } } benchmark.addCase("sort agg w/o group by") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { - df.select(percentile_approx($"id", 0.5)).collect() + df.select(percentile_approx($"id", 0.5)).noop() } } benchmark.addCase("object agg w/o group by w/o fallback") { _ => withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { - df.select(percentile_approx($"id", 0.5)).collect() + df.select(percentile_approx($"id", 0.5)).noop() } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index f28b9be60d3cb..a26412c5163ec 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -93,16 +93,16 @@ object OrcReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(id) FROM nativeOrcTable").noop() } } benchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(id) FROM nativeOrcTable").noop() } benchmark.addCase("Hive built-in ORC") { _ => - spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() + spark.sql("SELECT sum(id) FROM hiveOrcTable").noop() } benchmark.run() @@ -124,16 +124,16 @@ object OrcReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").noop() } } benchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").noop() } benchmark.addCase("Hive built-in ORC") { _ => - spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect() + spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").noop() } benchmark.run() @@ -153,44 +153,44 @@ object OrcReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("Data column - Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(id) FROM nativeOrcTable").noop() } } benchmark.addCase("Data column - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(id) FROM nativeOrcTable").noop() } benchmark.addCase("Data column - Hive built-in ORC") { _ => - spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() + spark.sql("SELECT sum(id) FROM hiveOrcTable").noop() } benchmark.addCase("Partition column - Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(p) FROM nativeOrcTable").noop() } } benchmark.addCase("Partition column - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(p) FROM nativeOrcTable").noop() } benchmark.addCase("Partition column - Hive built-in ORC") { _ => - spark.sql("SELECT sum(p) FROM hiveOrcTable").collect() + spark.sql("SELECT sum(p) FROM hiveOrcTable").noop() } benchmark.addCase("Both columns - Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").noop() } } benchmark.addCase("Both columns - Native ORC Vectorized") { _ => - spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").noop() } benchmark.addCase("Both columns - Hive built-in ORC") { _ => - spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect() + spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").noop() } benchmark.run() @@ -209,16 +209,16 @@ object OrcReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").noop() } } benchmark.addCase("Native ORC Vectorized") { _ => - spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() + spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").noop() } benchmark.addCase("Hive built-in ORC") { _ => - spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect() + spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").noop() } benchmark.run() @@ -244,18 +244,18 @@ object OrcReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").noop() } } benchmark.addCase("Native ORC Vectorized") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").noop() } benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").noop() } benchmark.run() @@ -278,16 +278,16 @@ object OrcReadBenchmark extends SqlBasedBenchmark { benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { - spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").noop() } } benchmark.addCase("Native ORC Vectorized") { _ => - spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").noop() } benchmark.addCase("Hive built-in ORC") { _ => - spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect() + spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").noop() } benchmark.run() From 361583d1f5bb7bc2eb87395cf52f2182352e25cb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 12 Jan 2020 23:45:31 +0000 Subject: [PATCH 1083/1387] [SPARK-30409][TEST][FOLLOWUP][HOTFIX] Remove dangling JSONBenchmark-jdk11-results.txt ### What changes were proposed in this pull request? This PR removes a dangling test result, `JSONBenchmark-jdk11-results.txt`. This causes a case-sensitive issue on Mac. ``` $ git clone https://gitbox.apache.org/repos/asf/spark.git spark-gitbox Cloning into 'spark-gitbox'... remote: Counting objects: 671717, done. remote: Compressing objects: 100% (258021/258021), done. remote: Total 671717 (delta 329181), reused 560390 (delta 228097) Receiving objects: 100% (671717/671717), 149.69 MiB | 950.00 KiB/s, done. Resolving deltas: 100% (329181/329181), done. Updating files: 100% (16090/16090), done. warning: the following paths have collided (e.g. case-sensitive paths on a case-insensitive filesystem) and only one from the same colliding group is in the working tree: 'sql/core/benchmarks/JSONBenchmark-jdk11-results.txt' 'sql/core/benchmarks/JsonBenchmark-jdk11-results.txt' ``` ### Why are the changes needed? Previously, since the file name didn't match with `object JSONBenchmark`, it made a confusion when we ran the benchmark. So, 4e0e4e51c4 renamed `JSONBenchmark` to `JsonBenchmark`. However, at the same time frame, https://github.com/apache/spark/pull/26003 regenerated this file. Recently, https://github.com/apache/spark/pull/27078 regenerates the results with the correct file name, `JsonBenchmark-jdk11-results.txt`. So, we can remove the old one. ### Does this PR introduce any user-facing change? No. This is a test result. ### How was this patch tested? Manually check the following correctly generated files in the master. And, check this PR removes the dangling one. - https://github.com/apache/spark/blob/master/sql/core/benchmarks/JsonBenchmark-results.txt - https://github.com/apache/spark/blob/master/sql/core/benchmarks/JsonBenchmark-jdk11-results.txt Closes #27180 from dongjoon-hyun/SPARK-REMOVE. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../JSONBenchmark-jdk11-results.txt | 112 ------------------ 1 file changed, 112 deletions(-) delete mode 100644 sql/core/benchmarks/JSONBenchmark-jdk11-results.txt diff --git a/sql/core/benchmarks/JSONBenchmark-jdk11-results.txt b/sql/core/benchmarks/JSONBenchmark-jdk11-results.txt deleted file mode 100644 index 56a87d41e3d47..0000000000000 --- a/sql/core/benchmarks/JSONBenchmark-jdk11-results.txt +++ /dev/null @@ -1,112 +0,0 @@ -================================================================================================ -Benchmark for performance of JSON parsing -================================================================================================ - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -No encoding 70863 71147 417 1.4 708.6 1.0X -UTF-8 is set 133520 133658 131 0.7 1335.2 0.5X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -No encoding 44462 44655 313 2.2 444.6 1.0X -UTF-8 is set 89342 89386 53 1.1 893.4 0.5X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -No encoding 58617 59071 541 0.2 5861.7 1.0X -UTF-8 is set 102650 102948 493 0.1 10265.0 0.6X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -No encoding 168197 168256 61 0.0 336393.0 1.0X -UTF-8 is set 191539 191724 186 0.0 383078.2 0.9X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Select 10 columns 18983 18998 22 0.5 1898.3 1.0X -Select 1 column 23544 23575 32 0.4 2354.4 0.8X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -creation of JSON parser per line: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Short column without encoding 9336 9402 66 1.1 933.6 1.0X -Short column with UTF-8 13310 13870 489 0.8 1331.0 0.7X -Wide column without encoding 112821 113012 202 0.1 11282.1 0.1X -Wide column with UTF-8 156408 156910 776 0.1 15640.8 0.1X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Text read 743 744 1 13.5 74.3 1.0X -from_json 22379 22566 279 0.4 2237.9 0.0X -json_tuple 32201 32271 89 0.3 3220.1 0.0X -get_json_object 21973 22405 375 0.5 2197.3 0.0X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Text read 3771 3780 8 13.3 75.4 1.0X -schema inferring 34808 35213 506 1.4 696.2 0.1X -parsing 36917 38889 1757 1.4 738.3 0.1X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Text read 11928 11955 38 4.2 238.6 1.0X -Schema inferring 44716 44921 220 1.1 894.3 0.3X -Parsing without charset 35749 35838 77 1.4 715.0 0.3X -Parsing with UTF-8 50521 50530 12 1.0 1010.4 0.2X - -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Create a dataset of timestamps 2473 2487 18 4.0 247.3 1.0X -to_json(timestamp) 16100 16284 236 0.6 1610.0 0.2X -write timestamps to files 13226 13852 676 0.8 1322.6 0.2X -Create a dataset of dates 2661 2680 16 3.8 266.1 0.9X -to_json(date) 10696 10730 55 0.9 1069.6 0.2X -write dates to files 8543 8926 384 1.2 854.3 0.3X - -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -read timestamp text from files 2822 2829 8 3.5 282.2 1.0X -read timestamps from files 34884 34900 23 0.3 3488.4 0.1X -infer timestamps from files 70310 71551 1079 0.1 7031.0 0.0X -read date text from files 2649 2655 6 3.8 264.9 1.1X -read date from files 22875 22904 33 0.4 2287.5 0.1X -timestamp strings 4025 4059 31 2.5 402.5 0.7X -parse timestamps from Dataset[String] 43040 43157 115 0.2 4304.0 0.1X -infer timestamps from Dataset[String] 81762 81927 144 0.1 8176.2 0.0X -date strings 4899 4913 21 2.0 489.9 0.6X -parse dates from Dataset[String] 33530 33560 27 0.3 3353.0 0.1X -from_json(timestamp) 60762 60877 142 0.2 6076.2 0.0X -from_json(date) 52482 52620 150 0.2 5248.2 0.1X - - From 150d49372f6dfe051b8049e4fe0356ee51cb547c Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 13 Jan 2020 10:47:51 +0900 Subject: [PATCH 1084/1387] [SPARK-28752][BUILD][DOCS][FOLLOW-UP] Render examples imported from Jekyll properly via Rouge ### What changes were proposed in this pull request? This PR proposes to use Pygment compatible format by Rouge. As of https://github.com/apache/spark/pull/26521, we use Rouge instead of Pygment wrapper in Ruby. Rouge claims Pygment compatibility; and we should output as Pygment does. ```ruby Rouge::Formatters::HTMLPygments.new(formatter) ``` wraps codes with `
      ...` properly.
      
      ### Why are the changes needed?
      
      To keep the documentation pretty and not broken.
      
      ### Does this PR introduce any user-facing change?
      
      Theoretically, no.
      
      This is rather a regression fix in documentation (that happens only by https://github.com/apache/spark/pull/26521 in master). See the malformed doc in preview - https://spark.apache.org/docs/3.0.0-preview2/sql-pyspark-pandas-with-arrow.html
      
      ### How was this patch tested?
      
      Manually built the doc.
      
      **Before:**
      ![Screen Shot 2020-01-13 at 10 21 28 AM](https://user-images.githubusercontent.com/6477701/72229159-ba766a80-35ef-11ea-9a5d-9583448e7c1c.png)
      
      **After:**
      
      ![Screen Shot 2020-01-13 at 10 26 33 AM](https://user-images.githubusercontent.com/6477701/72229157-b34f5c80-35ef-11ea-8b3a-492e8aa0f82a.png)
      
      Closes #27182 from HyukjinKwon/SPARK-28752-followup.
      
      Authored-by: HyukjinKwon 
      Signed-off-by: HyukjinKwon 
      ---
       docs/_plugins/include_example.rb | 4 ++--
       1 file changed, 2 insertions(+), 2 deletions(-)
      
      diff --git a/docs/_plugins/include_example.rb b/docs/_plugins/include_example.rb
      index 8a3af36e92305..6b4b1c652a81b 100644
      --- a/docs/_plugins/include_example.rb
      +++ b/docs/_plugins/include_example.rb
      @@ -54,9 +54,9 @@ def render(context)
               puts(e.backtrace)
               exit 1
             end
      -      code = select_lines(code)
      +      code = select_lines(code).strip
       
      -      formatter = Rouge::Formatters::HTML.new
      +      formatter = Rouge::Formatters::HTMLPygments.new(Rouge::Formatters::HTML.new)
             lexer = Rouge::Lexer.find(@lang)
             rendered_code = formatter.format(lexer.lex(code))
       
      
      From 308ae287a989f38daf22c72fbb7543a55744f43e Mon Sep 17 00:00:00 2001
      From: zhengruifeng 
      Date: Mon, 13 Jan 2020 13:48:36 +0800
      Subject: [PATCH 1085/1387] [SPARK-30457][ML] Use PeriodicRDDCheckpointer
       instead of NodeIdCache
      
      ### What changes were proposed in this pull request?
      1, del `NodeIdCache`, and use `PeriodicRDDCheckpointer` instead;
      2, reuse broadcasted `Splits` in the whole training;
      
      ### Why are the changes needed?
      1, The functionality of `NodeIdCache` and `PeriodicRDDCheckpointer` are highly similar, and the update process of nodeIds is simple; One goal of "Generalize PeriodicGraphCheckpointer for RDDs" in SPARK-5561 is to use checkpointer in RandomForest;
      2, only need to broadcast `Splits` once;
      
      ### Does this PR introduce any user-facing change?
      No
      
      ### How was this patch tested?
      Existing testsuites
      
      Closes #27145 from zhengruifeng/del_NodeIdCache.
      
      Authored-by: zhengruifeng 
      Signed-off-by: zhengruifeng 
      ---
       .../ml/tree/impl/GradientBoostedTrees.scala   |  48 +++--
       .../spark/ml/tree/impl/NodeIdCache.scala      | 199 ------------------
       .../spark/ml/tree/impl/RandomForest.scala     | 160 +++++++++-----
       .../spark/mllib/tree/RandomForest.scala       |   1 -
       .../ml/tree/impl/RandomForestSuite.scala      |   8 +-
       5 files changed, 137 insertions(+), 279 deletions(-)
       delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala
      
      diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala
      index 23f2554ae6991..18068e9483f2b 100644
      --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala
      +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala
      @@ -317,9 +317,6 @@ private[spark] object GradientBoostedTrees extends Logging {
           // Note: this is checkpointing the unweighted training error
           val predErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)](
             treeStrategy.getCheckpointInterval, sc)
      -    // Note: this is checkpointing the unweighted validation error
      -    val validatePredErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)](
      -      treeStrategy.getCheckpointInterval, sc)
       
           timer.stop("init")
       
      @@ -346,11 +343,6 @@ private[spark] object GradientBoostedTrees extends Logging {
             retaggedInput, splits, metadata)
             .persist(StorageLevel.MEMORY_AND_DISK)
             .setName("binned tree points")
      -    val validationTreePoints = if (validate) {
      -      TreePoint.convertToTreeRDD(
      -        validationInput.retag(classOf[Instance]), splits, metadata)
      -        .persist(StorageLevel.MEMORY_AND_DISK)
      -    } else sc.emptyRDD[TreePoint]
       
           val firstCounts = BaggedPoint
             .convertToBaggedRDD(treePoints, treeStrategy.subsamplingRate, numSubsamples = 1,
      @@ -369,7 +361,7 @@ private[spark] object GradientBoostedTrees extends Logging {
           }
       
           val firstTreeModel = RandomForest.runBagged(baggedInput = firstBagged,
      -      metadata = metadata, splits = splits, strategy = treeStrategy, numTrees = 1,
      +      metadata = metadata, bcSplits = bcSplits, strategy = treeStrategy, numTrees = 1,
             featureSubsetStrategy = featureSubsetStrategy, seed = seed, instr = instr,
             parentUID = None)
             .head.asInstanceOf[DecisionTreeRegressionModel]
      @@ -388,14 +380,24 @@ private[spark] object GradientBoostedTrees extends Logging {
           // Note: A model of type regression is used since we require raw prediction
           timer.stop("building tree 0")
       
      -    var validatePredError = computeInitialPredictionAndError(
      -      validationTreePoints, firstTreeWeight, firstTreeModel, loss, bcSplits)
      -    if (validate) validatePredErrorCheckpointer.update(validatePredError)
      -    var bestValidateError = if (validate) {
      -      computeWeightedError(validationTreePoints, validatePredError)
      -    } else {
      -      0.0
      +    var validationTreePoints: RDD[TreePoint] = null
      +    var validatePredError: RDD[(Double, Double)] = null
      +    var validatePredErrorCheckpointer: PeriodicRDDCheckpointer[(Double, Double)] = null
      +    var bestValidateError = 0.0
      +    if (validate) {
      +      timer.start("init validation")
      +      validationTreePoints = TreePoint.convertToTreeRDD(
      +        validationInput.retag(classOf[Instance]), splits, metadata)
      +        .persist(StorageLevel.MEMORY_AND_DISK)
      +      validatePredError = computeInitialPredictionAndError(
      +        validationTreePoints, firstTreeWeight, firstTreeModel, loss, bcSplits)
      +      validatePredErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)](
      +        treeStrategy.getCheckpointInterval, sc)
      +      validatePredErrorCheckpointer.update(validatePredError)
      +      bestValidateError = computeWeightedError(validationTreePoints, validatePredError)
      +      timer.stop("init validation")
           }
      +
           var bestM = 1
       
           var m = 1
      @@ -428,7 +430,7 @@ private[spark] object GradientBoostedTrees extends Logging {
               }
       
             val model = RandomForest.runBagged(baggedInput = bagged,
      -        metadata = metadata, splits = splits, strategy = treeStrategy,
      +        metadata = metadata, bcSplits = bcSplits, strategy = treeStrategy,
               numTrees = 1, featureSubsetStrategy = featureSubsetStrategy,
               seed = seed + m, instr = None, parentUID = None)
               .head.asInstanceOf[DecisionTreeRegressionModel]
      @@ -476,13 +478,15 @@ private[spark] object GradientBoostedTrees extends Logging {
           logInfo("Internal timing for DecisionTree:")
           logInfo(s"$timer")
       
      +    bcSplits.destroy()
      +    treePoints.unpersist()
           predErrorCheckpointer.unpersistDataSet()
           predErrorCheckpointer.deleteAllCheckpoints()
      -    validatePredErrorCheckpointer.unpersistDataSet()
      -    validatePredErrorCheckpointer.deleteAllCheckpoints()
      -    treePoints.unpersist()
      -    if (validate) validationTreePoints.unpersist()
      -    bcSplits.destroy()
      +    if (validate) {
      +      validationTreePoints.unpersist()
      +      validatePredErrorCheckpointer.unpersistDataSet()
      +      validatePredErrorCheckpointer.deleteAllCheckpoints()
      +    }
       
           if (validate) {
             (baseLearners.slice(0, bestM), baseLearnerWeights.slice(0, bestM))
      diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala
      deleted file mode 100644
      index c270e6c0bd5e8..0000000000000
      --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala
      +++ /dev/null
      @@ -1,199 +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.ml.tree.impl
      -
      -import java.io.IOException
      -
      -import scala.collection.mutable
      -
      -import org.apache.hadoop.fs.Path
      -
      -import org.apache.spark.internal.Logging
      -import org.apache.spark.ml.tree.{LearningNode, Split}
      -import org.apache.spark.rdd.RDD
      -import org.apache.spark.storage.StorageLevel
      -
      -
      -/**
      - * This is used by the node id cache to find the child id that a data point would belong to.
      - * @param split Split information.
      - * @param nodeIndex The current node index of a data point that this will update.
      - */
      -private[tree] case class NodeIndexUpdater(split: Split, nodeIndex: Int) {
      -
      -  /**
      -   * Determine a child node index based on the feature value and the split.
      -   * @param binnedFeature Binned feature value.
      -   * @param splits Split information to convert the bin indices to approximate feature values.
      -   * @return Child node index to update to.
      -   */
      -  def updateNodeIndex(binnedFeature: Int, splits: Array[Split]): Int = {
      -    if (split.shouldGoLeft(binnedFeature, splits)) {
      -      LearningNode.leftChildIndex(nodeIndex)
      -    } else {
      -      LearningNode.rightChildIndex(nodeIndex)
      -    }
      -  }
      -}
      -
      -/**
      - * Each TreePoint belongs to a particular node per tree.
      - * Each row in the nodeIdsForInstances RDD is an array over trees of the node index
      - * in each tree. Initially, values should all be 1 for root node.
      - * The nodeIdsForInstances RDD needs to be updated at each iteration.
      - * @param nodeIdsForInstances The initial values in the cache
      - *                           (should be an Array of all 1's (meaning the root nodes)).
      - * @param checkpointInterval The checkpointing interval
      - *                           (how often should the cache be checkpointed.).
      - */
      -private[spark] class NodeIdCache(
      -  var nodeIdsForInstances: RDD[Array[Int]],
      -  val checkpointInterval: Int) extends Logging {
      -
      -  // Keep a reference to a previous node Ids for instances.
      -  // Because we will keep on re-persisting updated node Ids,
      -  // we want to unpersist the previous RDD.
      -  private var prevNodeIdsForInstances: RDD[Array[Int]] = null
      -
      -  // To keep track of the past checkpointed RDDs.
      -  private val checkpointQueue = mutable.Queue[RDD[Array[Int]]]()
      -  private var rddUpdateCount = 0
      -
      -  // Indicates whether we can checkpoint
      -  private val canCheckpoint = nodeIdsForInstances.sparkContext.getCheckpointDir.nonEmpty
      -
      -  // Hadoop Configuration for deleting checkpoints as needed
      -  private val hadoopConf = nodeIdsForInstances.sparkContext.hadoopConfiguration
      -
      -  /**
      -   * Update the node index values in the cache.
      -   * This updates the RDD and its lineage.
      -   * TODO: Passing bin information to executors seems unnecessary and costly.
      -   * @param data The RDD of training rows.
      -   * @param nodeIdUpdaters A map of node index updaters.
      -   *                       The key is the indices of nodes that we want to update.
      -   * @param splits  Split information needed to find child node indices.
      -   */
      -  def updateNodeIndices(
      -      data: RDD[BaggedPoint[TreePoint]],
      -      nodeIdUpdaters: Array[mutable.Map[Int, NodeIndexUpdater]],
      -      splits: Array[Array[Split]]): Unit = {
      -    if (prevNodeIdsForInstances != null) {
      -      // Unpersist the previous one if one exists.
      -      prevNodeIdsForInstances.unpersist()
      -    }
      -
      -    prevNodeIdsForInstances = nodeIdsForInstances
      -    nodeIdsForInstances = data.zip(nodeIdsForInstances).map { case (point, ids) =>
      -      var treeId = 0
      -      while (treeId < nodeIdUpdaters.length) {
      -        val nodeIdUpdater = nodeIdUpdaters(treeId).getOrElse(ids(treeId), null)
      -        if (nodeIdUpdater != null) {
      -          val featureIndex = nodeIdUpdater.split.featureIndex
      -          val newNodeIndex = nodeIdUpdater.updateNodeIndex(
      -            binnedFeature = point.datum.binnedFeatures(featureIndex),
      -            splits = splits(featureIndex))
      -          ids(treeId) = newNodeIndex
      -        }
      -        treeId += 1
      -      }
      -      ids
      -    }
      -
      -    // Keep on persisting new ones.
      -    nodeIdsForInstances.persist(StorageLevel.MEMORY_AND_DISK)
      -    rddUpdateCount += 1
      -
      -    // Handle checkpointing if the directory is not None.
      -    if (canCheckpoint && checkpointInterval != -1 && (rddUpdateCount % checkpointInterval) == 0) {
      -      // Let's see if we can delete previous checkpoints.
      -      var canDelete = true
      -      while (checkpointQueue.size > 1 && canDelete) {
      -        // We can delete the oldest checkpoint iff
      -        // the next checkpoint actually exists in the file system.
      -        if (checkpointQueue(1).getCheckpointFile.isDefined) {
      -          val old = checkpointQueue.dequeue()
      -          // Since the old checkpoint is not deleted by Spark, we'll manually delete it here.
      -          try {
      -            val path = new Path(old.getCheckpointFile.get)
      -            val fs = path.getFileSystem(hadoopConf)
      -            fs.delete(path, true)
      -          } catch {
      -            case e: IOException =>
      -              logError("Decision Tree learning using cacheNodeIds failed to remove checkpoint" +
      -                s" file: ${old.getCheckpointFile.get}")
      -          }
      -        } else {
      -          canDelete = false
      -        }
      -      }
      -
      -      nodeIdsForInstances.checkpoint()
      -      checkpointQueue.enqueue(nodeIdsForInstances)
      -    }
      -  }
      -
      -  /**
      -   * Call this after training is finished to delete any remaining checkpoints.
      -   */
      -  def deleteAllCheckpoints(): Unit = {
      -    while (checkpointQueue.nonEmpty) {
      -      val old = checkpointQueue.dequeue()
      -      if (old.getCheckpointFile.isDefined) {
      -        try {
      -          val path = new Path(old.getCheckpointFile.get)
      -          val fs = path.getFileSystem(hadoopConf)
      -          fs.delete(path, true)
      -        } catch {
      -          case e: IOException =>
      -            logError("Decision Tree learning using cacheNodeIds failed to remove checkpoint" +
      -              s" file: ${old.getCheckpointFile.get}")
      -        }
      -      }
      -    }
      -    if (nodeIdsForInstances != null) {
      -      // Unpersist current one if one exists.
      -      nodeIdsForInstances.unpersist()
      -    }
      -    if (prevNodeIdsForInstances != null) {
      -      // Unpersist the previous one if one exists.
      -      prevNodeIdsForInstances.unpersist()
      -    }
      -  }
      -}
      -
      -private[spark] object NodeIdCache {
      -  /**
      -   * Initialize the node Id cache with initial node Id values.
      -   * @param data The RDD of training rows.
      -   * @param numTrees The number of trees that we want to create cache for.
      -   * @param checkpointInterval The checkpointing interval
      -   *                           (how often should the cache be checkpointed.).
      -   * @param initVal The initial values in the cache.
      -   * @return A node Id cache containing an RDD of initial root node Indices.
      -   */
      -  def init(
      -      data: RDD[BaggedPoint[TreePoint]],
      -      numTrees: Int,
      -      checkpointInterval: Int,
      -      initVal: Int = 1): NodeIdCache = {
      -    new NodeIdCache(
      -      data.map(_ => Array.fill[Int](numTrees)(initVal)),
      -      checkpointInterval)
      -  }
      -}
      diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala
      index d5c9a1e986a9e..a91a7b8f8e748 100644
      --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala
      +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala
      @@ -17,11 +17,10 @@
       
       package org.apache.spark.ml.tree.impl
       
      -import java.io.IOException
      -
       import scala.collection.mutable
       import scala.util.Random
       
      +import org.apache.spark.broadcast.Broadcast
       import org.apache.spark.internal.Logging
       import org.apache.spark.ml.classification.DecisionTreeClassificationModel
       import org.apache.spark.ml.feature.Instance
      @@ -34,6 +33,7 @@ import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => O
       import org.apache.spark.mllib.tree.impurity.ImpurityCalculator
       import org.apache.spark.mllib.tree.model.ImpurityStats
       import org.apache.spark.rdd.RDD
      +import org.apache.spark.rdd.util.PeriodicRDDCheckpointer
       import org.apache.spark.storage.StorageLevel
       import org.apache.spark.util.collection.OpenHashMap
       import org.apache.spark.util.random.{SamplingUtils, XORShiftRandom}
      @@ -111,7 +111,7 @@ private[spark] object RandomForest extends Logging with Serializable {
         def runBagged(
             baggedInput: RDD[BaggedPoint[TreePoint]],
             metadata: DecisionTreeMetadata,
      -      splits: Array[Array[Split]],
      +      bcSplits: Broadcast[Array[Array[Split]]],
             strategy: OldStrategy,
             numTrees: Int,
             featureSubsetStrategy: String,
      @@ -122,6 +122,8 @@ private[spark] object RandomForest extends Logging with Serializable {
           val timer = new TimeTracker()
           timer.start("total")
       
      +    val sc = baggedInput.sparkContext
      +
           instr match {
             case Some(instrumentation) =>
               instrumentation.logNumFeatures(metadata.numFeatures)
      @@ -154,16 +156,15 @@ private[spark] object RandomForest extends Logging with Serializable {
            * in lower levels).
            */
       
      -    // Create an RDD of node Id cache.
      -    // At first, all the rows belong to the root nodes (node Id == 1).
      -    val nodeIdCache = if (strategy.useNodeIdCache) {
      -      Some(NodeIdCache.init(
      -        data = baggedInput,
      -        numTrees = numTrees,
      -        checkpointInterval = strategy.checkpointInterval,
      -        initVal = 1))
      -    } else {
      -      None
      +    var nodeIds: RDD[Array[Int]] = null
      +    var nodeIdCheckpointer: PeriodicRDDCheckpointer[Array[Int]] = null
      +    if (strategy.useNodeIdCache) {
      +      // Create an RDD of node Id cache.
      +      // At first, all the rows belong to the root nodes (node Id == 1).
      +      nodeIds = baggedInput.map { _ => Array.fill(numTrees)(1) }
      +      nodeIdCheckpointer = new PeriodicRDDCheckpointer[Array[Int]](
      +        strategy.getCheckpointInterval, sc)
      +      nodeIdCheckpointer.update(nodeIds)
           }
       
           /*
      @@ -202,8 +203,14 @@ private[spark] object RandomForest extends Logging with Serializable {
       
             // Choose node splits, and enqueue new nodes as needed.
             timer.start("findBestSplits")
      -      RandomForest.findBestSplits(baggedInput, metadata, topNodesForGroup, nodesForGroup,
      -        treeToNodeToIndexInfo, splits, nodeStack, timer, nodeIdCache)
      +      val bestSplit = RandomForest.findBestSplits(baggedInput, metadata, topNodesForGroup,
      +        nodesForGroup, treeToNodeToIndexInfo, bcSplits, nodeStack, timer, nodeIds,
      +        outputBestSplits = strategy.useNodeIdCache)
      +      if (strategy.useNodeIdCache) {
      +        nodeIds = updateNodeIds(baggedInput, nodeIds, bcSplits, bestSplit)
      +        nodeIdCheckpointer.update(nodeIds)
      +      }
      +
             timer.stop("findBestSplits")
           }
       
      @@ -212,14 +219,10 @@ private[spark] object RandomForest extends Logging with Serializable {
           logInfo("Internal timing for DecisionTree:")
           logInfo(s"$timer")
       
      -    // Delete any remaining checkpoints used for node Id cache.
      -    if (nodeIdCache.nonEmpty) {
      -      try {
      -        nodeIdCache.get.deleteAllCheckpoints()
      -      } catch {
      -        case e: IOException =>
      -          logWarning(s"delete all checkpoints failed. Error reason: ${e.getMessage}")
      -      }
      +    if (strategy.useNodeIdCache) {
      +      // Delete any remaining checkpoints used for node Id cache.
      +      nodeIdCheckpointer.unpersistDataSet()
      +      nodeIdCheckpointer.deleteAllCheckpoints()
           }
       
           val numFeatures = metadata.numFeatures
      @@ -289,18 +292,55 @@ private[spark] object RandomForest extends Logging with Serializable {
       
           val withReplacement = numTrees > 1
       
      +    val bcSplits = input.sparkContext.broadcast(splits)
           val baggedInput = BaggedPoint
             .convertToBaggedRDD(treeInput, strategy.subsamplingRate, numTrees, withReplacement,
               (tp: TreePoint) => tp.weight, seed = seed)
             .persist(StorageLevel.MEMORY_AND_DISK)
      +      .setName("bagged tree points")
       
      -    val trees = runBagged(baggedInput = baggedInput, metadata = metadata, splits = splits,
      +    val trees = runBagged(baggedInput = baggedInput, metadata = metadata, bcSplits = bcSplits,
             strategy = strategy, numTrees = numTrees, featureSubsetStrategy = featureSubsetStrategy,
             seed = seed, instr = instr, prune = prune, parentUID = parentUID)
      +
           baggedInput.unpersist()
      +    bcSplits.destroy()
      +
           trees
         }
       
      +  /**
      +   * Update node indices by newly found splits.
      +   */
      +  private def updateNodeIds(
      +      input: RDD[BaggedPoint[TreePoint]],
      +      nodeIds: RDD[Array[Int]],
      +      bcSplits: Broadcast[Array[Array[Split]]],
      +      bestSplits: Array[Map[Int, Split]]): RDD[Array[Int]] = {
      +    require(nodeIds != null && bestSplits != null)
      +    input.zip(nodeIds).map { case (point, ids) =>
      +      var treeId = 0
      +      while (treeId < bestSplits.length) {
      +        val bestSplitsInTree = bestSplits(treeId)
      +        if (bestSplitsInTree != null) {
      +          val nodeId = ids(treeId)
      +          bestSplitsInTree.get(nodeId).foreach { bestSplit =>
      +            val featureId = bestSplit.featureIndex
      +            val bin = point.datum.binnedFeatures(featureId)
      +            val newNodeId = if (bestSplit.shouldGoLeft(bin, bcSplits.value(featureId))) {
      +              LearningNode.leftChildIndex(nodeId)
      +            } else {
      +              LearningNode.rightChildIndex(nodeId)
      +            }
      +            ids(treeId) = newNodeId
      +          }
      +        }
      +        treeId += 1
      +      }
      +      ids
      +    }
      +  }
      +
         /**
          * Helper for binSeqOp, for data which can contain a mix of ordered and unordered features.
          *
      @@ -414,14 +454,13 @@ private[spark] object RandomForest extends Logging with Serializable {
          * @param treeToNodeToIndexInfo Mapping: treeIndex --> nodeIndex --> nodeIndexInfo,
          *                              where nodeIndexInfo stores the index in the group and the
          *                              feature subsets (if using feature subsets).
      -   * @param splits possible splits for all features, indexed (numFeatures)(numSplits)
      +   * @param bcSplits possible splits for all features, indexed (numFeatures)(numSplits)
          * @param nodeStack  Queue of nodes to split, with values (treeIndex, node).
          *                   Updated with new non-leaf nodes which are created.
      -   * @param nodeIdCache Node Id cache containing an RDD of Array[Int] where
      -   *                    each value in the array is the data point's node Id
      -   *                    for a corresponding tree. This is used to prevent the need
      -   *                    to pass the entire tree to the executors during
      -   *                    the node stat aggregation phase.
      +   * @param nodeIds an RDD of Array[Int] where each value in the array is the data
      +   *                point's node Id for a corresponding tree. This is used to prevent
      +   *                the need to pass the entire tree to the executors during the node
      +   *                stat aggregation phase.
          */
         private[tree] def findBestSplits(
             input: RDD[BaggedPoint[TreePoint]],
      @@ -429,10 +468,11 @@ private[spark] object RandomForest extends Logging with Serializable {
             topNodesForGroup: Map[Int, LearningNode],
             nodesForGroup: Map[Int, Array[LearningNode]],
             treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]],
      -      splits: Array[Array[Split]],
      +      bcSplits: Broadcast[Array[Array[Split]]],
             nodeStack: mutable.ListBuffer[(Int, LearningNode)],
             timer: TimeTracker = new TimeTracker,
      -      nodeIdCache: Option[NodeIdCache] = None): Unit = {
      +      nodeIds: RDD[Array[Int]] = null,
      +      outputBestSplits: Boolean = false): Array[Map[Int, Split]] = {
       
           /*
            * The high-level descriptions of the best split optimizations are noted here.
      @@ -456,6 +496,8 @@ private[spark] object RandomForest extends Logging with Serializable {
            * drastically reduce the communication overhead.
            */
       
      +    val useNodeIdCache = nodeIds != null
      +
           // numNodes:  Number of nodes in this group
           val numNodes = nodesForGroup.values.map(_.length).sum
           logDebug(s"numNodes = $numNodes")
      @@ -464,7 +506,7 @@ private[spark] object RandomForest extends Logging with Serializable {
           logDebug(s"isMulticlass = ${metadata.isMulticlass}")
           logDebug(s"isMulticlassWithCategoricalFeatures = " +
             s"${metadata.isMulticlassWithCategoricalFeatures}")
      -    logDebug(s"using nodeIdCache = ${nodeIdCache.nonEmpty}")
      +    logDebug(s"using nodeIdCache = $useNodeIdCache")
       
           /*
            * Performs a sequential aggregation over a partition for a particular tree and node.
      @@ -482,7 +524,8 @@ private[spark] object RandomForest extends Logging with Serializable {
               treeIndex: Int,
               nodeInfo: NodeIndexInfo,
               agg: Array[DTStatsAggregator],
      -        baggedPoint: BaggedPoint[TreePoint]): Unit = {
      +        baggedPoint: BaggedPoint[TreePoint],
      +        splits: Array[Array[Split]]): Unit = {
             if (nodeInfo != null) {
               val aggNodeIndex = nodeInfo.nodeIndexInGroup
               val featuresForNode = nodeInfo.featureSubset
      @@ -512,11 +555,13 @@ private[spark] object RandomForest extends Logging with Serializable {
            */
           def binSeqOp(
               agg: Array[DTStatsAggregator],
      -        baggedPoint: BaggedPoint[TreePoint]): Array[DTStatsAggregator] = {
      +        baggedPoint: BaggedPoint[TreePoint],
      +        splits: Array[Array[Split]]): Array[DTStatsAggregator] = {
             treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) =>
               val nodeIndex =
                 topNodesForGroup(treeIndex).predictImpl(baggedPoint.datum.binnedFeatures, splits)
      -        nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null), agg, baggedPoint)
      +        nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null),
      +          agg, baggedPoint, splits)
             }
             agg
           }
      @@ -526,14 +571,15 @@ private[spark] object RandomForest extends Logging with Serializable {
            */
           def binSeqOpWithNodeIdCache(
               agg: Array[DTStatsAggregator],
      -        dataPoint: (BaggedPoint[TreePoint], Array[Int])): Array[DTStatsAggregator] = {
      +        dataPoint: (BaggedPoint[TreePoint], Array[Int]),
      +        splits: Array[Array[Split]]): Array[DTStatsAggregator] = {
             treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) =>
               val baggedPoint = dataPoint._1
               val nodeIdCache = dataPoint._2
               val nodeIndex = nodeIdCache(treeIndex)
      -        nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null), agg, baggedPoint)
      +        nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null),
      +          agg, baggedPoint, splits)
             }
      -
             agg
           }
       
      @@ -577,8 +623,9 @@ private[spark] object RandomForest extends Logging with Serializable {
           val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo)
           val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures)
       
      -    val partitionAggregates: RDD[(Int, DTStatsAggregator)] = if (nodeIdCache.nonEmpty) {
      -      input.zip(nodeIdCache.get.nodeIdsForInstances).mapPartitions { points =>
      +    val partitionAggregates = if (useNodeIdCache) {
      +
      +      input.zip(nodeIds).mapPartitions { points =>
               // Construct a nodeStatsAggregators array to hold node aggregate stats,
               // each node will have a nodeStatsAggregator
               val nodeStatsAggregators = Array.tabulate(numNodes) { nodeIndex =>
      @@ -589,7 +636,7 @@ private[spark] object RandomForest extends Logging with Serializable {
               }
       
               // iterator all instances in current partition and update aggregate stats
      -        points.foreach(binSeqOpWithNodeIdCache(nodeStatsAggregators, _))
      +        points.foreach(binSeqOpWithNodeIdCache(nodeStatsAggregators, _, bcSplits.value))
       
               // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs,
               // which can be combined with other partition using `reduceByKey`
      @@ -607,7 +654,7 @@ private[spark] object RandomForest extends Logging with Serializable {
               }
       
               // iterator all instances in current partition and update aggregate stats
      -        points.foreach(binSeqOp(nodeStatsAggregators, _))
      +        points.foreach(binSeqOp(nodeStatsAggregators, _, bcSplits.value))
       
               // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs,
               // which can be combined with other partition using `reduceByKey`
      @@ -623,19 +670,19 @@ private[spark] object RandomForest extends Logging with Serializable {
       
               // find best split for each node
               val (split: Split, stats: ImpurityStats) =
      -          binsToBestSplit(aggStats, splits, featuresForNode, nodes(nodeIndex))
      +          binsToBestSplit(aggStats, bcSplits.value, featuresForNode, nodes(nodeIndex))
               (nodeIndex, (split, stats))
           }.collectAsMap()
           nodeToFeaturesBc.destroy()
       
           timer.stop("chooseSplits")
       
      -    val nodeIdUpdaters = if (nodeIdCache.nonEmpty) {
      -      Array.fill[mutable.Map[Int, NodeIndexUpdater]](
      -        metadata.numTrees)(mutable.Map[Int, NodeIndexUpdater]())
      +    val bestSplits = if (outputBestSplits) {
      +      Array.ofDim[mutable.Map[Int, Split]](metadata.numTrees)
           } else {
             null
           }
      +
           // Iterate over all nodes in this group.
           nodesForGroup.foreach { case (treeIndex, nodesForTree) =>
             nodesForTree.foreach { node =>
      @@ -663,11 +710,13 @@ private[spark] object RandomForest extends Logging with Serializable {
                 node.rightChild = Some(LearningNode(LearningNode.rightChildIndex(nodeIndex),
                   rightChildIsLeaf, ImpurityStats.getEmptyImpurityStats(stats.rightImpurityCalculator)))
       
      -          if (nodeIdCache.nonEmpty) {
      -            val nodeIndexUpdater = NodeIndexUpdater(
      -              split = split,
      -              nodeIndex = nodeIndex)
      -            nodeIdUpdaters(treeIndex).put(nodeIndex, nodeIndexUpdater)
      +          if (outputBestSplits) {
      +            val bestSplitsInTree = bestSplits(treeIndex)
      +            if (bestSplitsInTree == null) {
      +              bestSplits(treeIndex) = mutable.Map[Int, Split](nodeIndex -> split)
      +            } else {
      +              bestSplitsInTree.update(nodeIndex, split)
      +            }
                 }
       
                 // enqueue left child and right child if they are not leaves
      @@ -686,9 +735,10 @@ private[spark] object RandomForest extends Logging with Serializable {
             }
           }
       
      -    if (nodeIdCache.nonEmpty) {
      -      // Update the cache if needed.
      -      nodeIdCache.get.updateNodeIndices(input, nodeIdUpdaters, splits)
      +    if (outputBestSplits) {
      +      bestSplits.map { m => if (m == null) null else m.toMap }
      +    } else {
      +      null
           }
         }
       
      diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala
      index 94224be80752a..ecbb21498e68f 100644
      --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala
      +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala
      @@ -23,7 +23,6 @@ import scala.util.Try
       import org.apache.spark.annotation.Since
       import org.apache.spark.api.java.JavaRDD
       import org.apache.spark.internal.Logging
      -import org.apache.spark.ml.feature.Instance
       import org.apache.spark.ml.tree.{DecisionTreeModel => NewDTModel, TreeEnsembleParams => NewRFParams}
       import org.apache.spark.ml.tree.impl.{RandomForest => NewRandomForest}
       import org.apache.spark.mllib.regression.LabeledPoint
      diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala
      index ae5e979983b4f..2a83d0aaf9699 100644
      --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala
      +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala
      @@ -336,6 +336,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext {
             numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3))
           val metadata = DecisionTreeMetadata.buildMetadata(input, strategy)
           val splits = RandomForest.findSplits(input, metadata, seed = 42)
      +    val bcSplits = input.sparkContext.broadcast(splits)
       
           val treeInput = TreePoint.convertToTreeRDD(input, splits, metadata)
           val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, withReplacement = false)
      @@ -350,7 +351,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext {
           ))
           val nodeStack = new mutable.ListBuffer[(Int, LearningNode)]
           RandomForest.findBestSplits(baggedInput, metadata, Map(0 -> topNode),
      -      nodesForGroup, treeToNodeToIndexInfo, splits, nodeStack)
      +      nodesForGroup, treeToNodeToIndexInfo, bcSplits, nodeStack)
      +    bcSplits.destroy()
       
           // don't enqueue leaf nodes into node queue
           assert(nodeStack.isEmpty)
      @@ -378,6 +380,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext {
             numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3))
           val metadata = DecisionTreeMetadata.buildMetadata(input, strategy)
           val splits = RandomForest.findSplits(input, metadata, seed = 42)
      +    val bcSplits = input.sparkContext.broadcast(splits)
       
           val treeInput = TreePoint.convertToTreeRDD(input, splits, metadata)
           val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, withReplacement = false)
      @@ -392,7 +395,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext {
           ))
           val nodeStack = new mutable.ListBuffer[(Int, LearningNode)]
           RandomForest.findBestSplits(baggedInput, metadata, Map(0 -> topNode),
      -      nodesForGroup, treeToNodeToIndexInfo, splits, nodeStack)
      +      nodesForGroup, treeToNodeToIndexInfo, bcSplits, nodeStack)
      +    bcSplits.destroy()
       
           // don't enqueue a node into node queue if its impurity is 0.0
           assert(nodeStack.isEmpty)
      
      From 8ce7962931680c204e84dd75783b1c943ea9c525 Mon Sep 17 00:00:00 2001
      From: ulysses 
      Date: Mon, 13 Jan 2020 15:12:19 +0900
      Subject: [PATCH 1086/1387] [SPARK-30245][SQL] Add cache for Like and RLike
       when pattern is not static
      
      ### What changes were proposed in this pull request?
      
      Add cache for Like and RLike when pattern is not static
      
      ### Why are the changes needed?
      
      When pattern is not static, we should avoid compile pattern every time if some pattern is same.
      Here is perf numbers, include 3 test groups and use `range` to make it easy.
      ```
      // ---------------------
      // 10,000 rows and 10 partitions
      val df1 = spark.range(0, 10000, 1, 10).withColumnRenamed("id", "id1")
      val df2 = spark.range(0, 10000, 1, 10).withColumnRenamed("id", "id2")
      
      val start = System.currentTimeMillis
      df1.join(df2).where("id2 like id1").count()
      // before  16939
      // after    6352
      println(System.currentTimeMillis - start)
      
      // ---------------------
      // 10,000 rows and 100 partitions
      val df1 = spark.range(0, 10000, 1, 100).withColumnRenamed("id", "id1")
      val df2 = spark.range(0, 10000, 1, 100).withColumnRenamed("id", "id2")
      
      val start = System.currentTimeMillis
      df1.join(df2).where("id2 like id1").count()
      // before  11070
      // after    4680
      println(System.currentTimeMillis - start)
      
      // ---------------------
      // 20,000 rows and 10 partitions
      val df1 = spark.range(0, 20000, 1, 10).withColumnRenamed("id", "id1")
      val df2 = spark.range(0, 20000, 1, 10).withColumnRenamed("id", "id2")
      
      val start = System.currentTimeMillis
      df1.join(df2).where("id2 like id1").count()
      // before 66962
      // after  29934
      println(System.currentTimeMillis - start)
      ```
      
      ### Does this PR introduce any user-facing change?
      
      No.
      
      ### How was this patch tested?
      
      Closes #26875 from ulysses-you/SPARK-30245.
      
      Authored-by: ulysses 
      Signed-off-by: HyukjinKwon 
      ---
       .../expressions/regexpExpressions.scala       | 20 ++++++++++++++-----
       1 file changed, 15 insertions(+), 5 deletions(-)
      
      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 6ff6dc729795d..c6fb6c95e5d31 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
      @@ -152,8 +152,6 @@ case class Like(left: Expression, right: Expression, escapeChar: Char = '\\')
               """)
             }
           } else {
      -      val pattern = ctx.freshName("pattern")
      -      val rightStr = ctx.freshName("rightStr")
             // We need double escape to avoid org.codehaus.commons.compiler.CompileException.
             // '\\' will cause exception 'Single quote must be backslash-escaped in character literal'.
             // '\"' will cause exception 'Line break in literal not allowed'.
      @@ -162,10 +160,17 @@ case class Like(left: Expression, right: Expression, escapeChar: Char = '\\')
             } else {
               escapeChar
             }
      +      val rightStr = ctx.freshName("rightStr")
      +      val pattern = ctx.addMutableState(patternClass, "pattern")
      +      val lastRightStr = ctx.addMutableState(classOf[String].getName, "lastRightStr")
      +
             nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
               s"""
                 String $rightStr = $eval2.toString();
      -          $patternClass $pattern = $patternClass.compile($escapeFunc($rightStr, '$newEscapeChar'));
      +          if (!$rightStr.equals($lastRightStr)) {
      +            $pattern = $patternClass.compile($escapeFunc($rightStr, '$newEscapeChar'));
      +            $lastRightStr = $rightStr;
      +          }
                 ${ev.value} = $pattern.matcher($eval1.toString()).matches();
               """
             })
      @@ -240,11 +245,16 @@ case class RLike(left: Expression, right: Expression) extends StringRegexExpress
             }
           } else {
             val rightStr = ctx.freshName("rightStr")
      -      val pattern = ctx.freshName("pattern")
      +      val pattern = ctx.addMutableState(patternClass, "pattern")
      +      val lastRightStr = ctx.addMutableState(classOf[String].getName, "lastRightStr")
      +
             nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
               s"""
                 String $rightStr = $eval2.toString();
      -          $patternClass $pattern = $patternClass.compile($rightStr);
      +          if (!$rightStr.equals($lastRightStr)) {
      +            $pattern = $patternClass.compile($rightStr);
      +            $lastRightStr = $rightStr;
      +          }
                 ${ev.value} = $pattern.matcher($eval1.toString()).find(0);
               """
             })
      
      From 28fc0437ce6d2f6fbcd83be38aafb8a491c1a67d Mon Sep 17 00:00:00 2001
      From: Dongjoon Hyun 
      Date: Sun, 12 Jan 2020 23:03:34 -0800
      Subject: [PATCH 1087/1387] [SPARK-28152][SQL][FOLLOWUP] Add a legacy conf for
       old MsSqlServerDialect numeric mapping
      
      ### What changes were proposed in this pull request?
      
      This is a follow-up for https://github.com/apache/spark/pull/25248 .
      
      ### Why are the changes needed?
      
      The new behavior cannot access the existing table which is created by old behavior.
      This PR provides a way to avoid new behavior for the existing users.
      
      ### Does this PR introduce any user-facing change?
      
      Yes. This will fix the broken behavior on the existing tables.
      
      ### How was this patch tested?
      
      Pass the Jenkins and manually run JDBC integration test.
      ```
      build/mvn install -DskipTests
      build/mvn -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12 test
      ```
      
      Closes #27184 from dongjoon-hyun/SPARK-28152-CONF.
      
      Authored-by: Dongjoon Hyun 
      Signed-off-by: Dongjoon Hyun 
      ---
       .../jdbc/MsSqlServerIntegrationSuite.scala    | 83 ++++++++++++-------
       .../apache/spark/sql/internal/SQLConf.scala   | 10 +++
       .../spark/sql/jdbc/MsSqlServerDialect.scala   | 16 ++--
       .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 32 +++++--
       4 files changed, 100 insertions(+), 41 deletions(-)
      
      diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala
      index efd7ca74c796b..5738307095933 100644
      --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala
      +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala
      @@ -21,6 +21,7 @@ import java.math.BigDecimal
       import java.sql.{Connection, Date, Timestamp}
       import java.util.Properties
       
      +import org.apache.spark.sql.internal.SQLConf
       import org.apache.spark.tags.DockerTest
       
       @DockerTest
      @@ -112,36 +113,58 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite {
         }
       
         test("Numeric types") {
      -    val df = spark.read.jdbc(jdbcUrl, "numbers", new Properties)
      -    val rows = df.collect()
      -    assert(rows.length == 1)
      -    val row = rows(0)
      -    val types = row.toSeq.map(x => x.getClass.toString)
      -    assert(types.length == 12)
      -    assert(types(0).equals("class java.lang.Boolean"))
      -    assert(types(1).equals("class java.lang.Integer"))
      -    assert(types(2).equals("class java.lang.Short"))
      -    assert(types(3).equals("class java.lang.Integer"))
      -    assert(types(4).equals("class java.lang.Long"))
      -    assert(types(5).equals("class java.lang.Double"))
      -    assert(types(6).equals("class java.lang.Float"))
      -    assert(types(7).equals("class java.lang.Float"))
      -    assert(types(8).equals("class java.math.BigDecimal"))
      -    assert(types(9).equals("class java.math.BigDecimal"))
      -    assert(types(10).equals("class java.math.BigDecimal"))
      -    assert(types(11).equals("class java.math.BigDecimal"))
      -    assert(row.getBoolean(0) == false)
      -    assert(row.getInt(1) == 255)
      -    assert(row.getShort(2) == 32767)
      -    assert(row.getInt(3) == 2147483647)
      -    assert(row.getLong(4) == 9223372036854775807L)
      -    assert(row.getDouble(5) == 1.2345678901234512E14) // float = float(53) has 15-digits precision
      -    assert(row.getFloat(6) == 1.23456788103168E14)   // float(24) has 7-digits precision
      -    assert(row.getFloat(7) == 1.23456788103168E14)   // real = float(24)
      -    assert(row.getAs[BigDecimal](8).equals(new BigDecimal("123.00")))
      -    assert(row.getAs[BigDecimal](9).equals(new BigDecimal("12345.12000")))
      -    assert(row.getAs[BigDecimal](10).equals(new BigDecimal("922337203685477.5800")))
      -    assert(row.getAs[BigDecimal](11).equals(new BigDecimal("214748.3647")))
      +    Seq(true, false).foreach { flag =>
      +      withSQLConf(SQLConf.LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED.key -> s"$flag") {
      +        val df = spark.read.jdbc(jdbcUrl, "numbers", new Properties)
      +        val rows = df.collect()
      +        assert(rows.length == 1)
      +        val row = rows(0)
      +        val types = row.toSeq.map(x => x.getClass.toString)
      +        assert(types.length == 12)
      +        assert(types(0).equals("class java.lang.Boolean"))
      +        assert(types(1).equals("class java.lang.Integer"))
      +        if (flag) {
      +          assert(types(2).equals("class java.lang.Integer"))
      +        } else {
      +          assert(types(2).equals("class java.lang.Short"))
      +        }
      +        assert(types(3).equals("class java.lang.Integer"))
      +        assert(types(4).equals("class java.lang.Long"))
      +        assert(types(5).equals("class java.lang.Double"))
      +        if (flag) {
      +          assert(types(6).equals("class java.lang.Double"))
      +          assert(types(7).equals("class java.lang.Double"))
      +        } else {
      +          assert(types(6).equals("class java.lang.Float"))
      +          assert(types(7).equals("class java.lang.Float"))
      +        }
      +        assert(types(8).equals("class java.math.BigDecimal"))
      +        assert(types(9).equals("class java.math.BigDecimal"))
      +        assert(types(10).equals("class java.math.BigDecimal"))
      +        assert(types(11).equals("class java.math.BigDecimal"))
      +        assert(row.getBoolean(0) == false)
      +        assert(row.getInt(1) == 255)
      +        if (flag) {
      +          assert(row.getInt(2) == 32767)
      +        } else {
      +          assert(row.getShort(2) == 32767)
      +        }
      +        assert(row.getInt(3) == 2147483647)
      +        assert(row.getLong(4) == 9223372036854775807L)
      +        assert(row.getDouble(5) == 1.2345678901234512E14) // float(53) has 15-digits precision
      +        if (flag) {
      +          assert(row.getDouble(6) == 1.23456788103168E14) // float(24) has 7-digits precision
      +          assert(row.getDouble(7) == 1.23456788103168E14) // real = float(24)
      +        } else {
      +          assert(row.getFloat(6) == 1.23456788103168E14)  // float(24) has 7-digits precision
      +          assert(row.getFloat(7) == 1.23456788103168E14)  // real = float(24)
      +        }
      +        assert(row.getAs[BigDecimal](8).equals(new BigDecimal("123.00")))
      +        assert(row.getAs[BigDecimal](9).equals(new BigDecimal("12345.12000")))
      +        assert(row.getAs[BigDecimal](10).equals(new BigDecimal("922337203685477.5800")))
      +        assert(row.getAs[BigDecimal](11).equals(new BigDecimal("214748.3647")))
      +      }
      +    }
         }
       
         test("Date types") {
      diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
      index 41a429303efb4..06fd7a79c4c93 100644
      --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
      +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
      @@ -2161,6 +2161,13 @@ object SQLConf {
           .booleanConf
           .createWithDefault(true)
       
      +  val LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED =
      +    buildConf("spark.sql.legacy.mssqlserver.numericMapping.enabled")
      +      .internal()
      +      .doc("When true, use legacy MySqlServer SMALLINT and REAL type mapping.")
      +      .booleanConf
      +      .createWithDefault(false)
      +
         /**
          * Holds information about keys that have been deprecated.
          *
      @@ -2417,6 +2424,9 @@ class SQLConf extends Serializable with Logging {
       
         def addDirectoryRecursiveEnabled: Boolean = getConf(LEGACY_ADD_DIRECTORY_USING_RECURSIVE)
       
      +  def legacyMsSqlServerNumericMappingEnabled: Boolean =
      +    getConf(LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED)
      +
         /**
          * Returns the [[Resolver]] for the current configuration, which can be used to determine if two
          * identifiers are equal.
      diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
      index 2511067abc3fd..72284b5996201 100644
      --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
      +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
      @@ -19,6 +19,7 @@ package org.apache.spark.sql.jdbc
       
       import java.util.Locale
       
      +import org.apache.spark.sql.internal.SQLConf
       import org.apache.spark.sql.types._
       
       
      @@ -33,10 +34,14 @@ private object MsSqlServerDialect extends JdbcDialect {
             // String is recommend by Microsoft SQL Server for datetimeoffset types in non-MS clients
             Option(StringType)
           } else {
      -      sqlType match {
      -        case java.sql.Types.SMALLINT => Some(ShortType)
      -        case java.sql.Types.REAL => Some(FloatType)
      -        case _ => None
      +      if (SQLConf.get.legacyMsSqlServerNumericMappingEnabled) {
      +        None
      +      } else {
      +        sqlType match {
      +          case java.sql.Types.SMALLINT => Some(ShortType)
      +          case java.sql.Types.REAL => Some(FloatType)
      +          case _ => None
      +        }
             }
           }
         }
      @@ -46,7 +51,8 @@ private object MsSqlServerDialect extends JdbcDialect {
           case StringType => Some(JdbcType("NVARCHAR(MAX)", java.sql.Types.NVARCHAR))
           case BooleanType => Some(JdbcType("BIT", java.sql.Types.BIT))
           case BinaryType => Some(JdbcType("VARBINARY(MAX)", java.sql.Types.VARBINARY))
      -    case ShortType => Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT))
      +    case ShortType if !SQLConf.get.legacyMsSqlServerNumericMappingEnabled =>
      +      Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT))
           case _ => None
         }
       
      diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
      index 66ddc6ee83d06..9cba95f7d7df2 100644
      --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
      +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
      @@ -884,17 +884,37 @@ class JDBCSuite extends QueryTest
             "BIT")
           assert(msSqlServerDialect.getJDBCType(BinaryType).map(_.databaseTypeDefinition).get ==
             "VARBINARY(MAX)")
      -    assert(msSqlServerDialect.getJDBCType(ShortType).map(_.databaseTypeDefinition).get ==
      -      "SMALLINT")
      +    Seq(true, false).foreach { flag =>
      +      withSQLConf(SQLConf.LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED.key -> s"$flag") {
      +        if (SQLConf.get.legacyMsSqlServerNumericMappingEnabled) {
      +          assert(msSqlServerDialect.getJDBCType(ShortType).map(_.databaseTypeDefinition).isEmpty)
      +        } else {
      +          assert(msSqlServerDialect.getJDBCType(ShortType).map(_.databaseTypeDefinition).get ==
      +            "SMALLINT")
      +        }
      +      }
      +    }
         }
       
         test("SPARK-28152 MsSqlServerDialect catalyst type mapping") {
           val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver")
           val metadata = new MetadataBuilder().putLong("scale", 1)
      -    assert(msSqlServerDialect.getCatalystType(java.sql.Types.SMALLINT, "SMALLINT", 1,
      -      metadata).get == ShortType)
      -    assert(msSqlServerDialect.getCatalystType(java.sql.Types.REAL, "REAL", 1,
      -      metadata).get == FloatType)
      +
      +    Seq(true, false).foreach { flag =>
      +      withSQLConf(SQLConf.LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED.key -> s"$flag") {
      +        if (SQLConf.get.legacyMsSqlServerNumericMappingEnabled) {
      +          assert(msSqlServerDialect.getCatalystType(java.sql.Types.SMALLINT, "SMALLINT", 1,
      +            metadata).isEmpty)
      +          assert(msSqlServerDialect.getCatalystType(java.sql.Types.REAL, "REAL", 1,
      +            metadata).isEmpty)
      +        } else {
      +          assert(msSqlServerDialect.getCatalystType(java.sql.Types.SMALLINT, "SMALLINT", 1,
      +            metadata).get == ShortType)
      +          assert(msSqlServerDialect.getCatalystType(java.sql.Types.REAL, "REAL", 1,
      +            metadata).get == FloatType)
      +        }
      +      }
      +    }
         }
       
         test("table exists query by jdbc dialect") {
      
      From eefcc7d762a627bf19cab7041a1a82f88862e7e1 Mon Sep 17 00:00:00 2001
      From: "Jungtaek Lim (HeartSaVioR)" 
      Date: Sun, 12 Jan 2020 23:19:37 -0800
      Subject: [PATCH 1088/1387] [SPARK-21869][SS][DOCS][FOLLOWUP] Document Kafka
       producer pool configuration
      
      ### What changes were proposed in this pull request?
      
      This patch documents the configuration for the Kafka producer pool, newly revised via SPARK-21869 (#26845)
      
      ### Why are the changes needed?
      
      The explanation of new Kafka producer pool configuration is missing, whereas the doc has Kafka
       consumer pool configuration.
      
      ### Does this PR introduce any user-facing change?
      
      Yes. This is a documentation change.
      
      ![Screen Shot 2020-01-12 at 11 16 19 PM](https://user-images.githubusercontent.com/9700541/72238148-c8959e00-3591-11ea-87fc-a8918792017e.png)
      
      ### How was this patch tested?
      
      N/A
      
      Closes #27146 from HeartSaVioR/SPARK-21869-FOLLOWUP.
      
      Authored-by: Jungtaek Lim (HeartSaVioR) 
      Signed-off-by: Dongjoon Hyun 
      ---
       .../structured-streaming-kafka-integration.md | 32 +++++++++++++++++--
       1 file changed, 30 insertions(+), 2 deletions(-)
      
      diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md
      index e939e83449b2e..0820b38a863c2 100644
      --- a/docs/structured-streaming-kafka-integration.md
      +++ b/docs/structured-streaming-kafka-integration.md
      @@ -539,7 +539,7 @@ The following properties are available to configure the consumer pool:
       
      - + @@ -580,7 +580,7 @@ The following properties are available to configure the fetched data pool: - +
      spark.kafka.consumer.cache.evictorThreadRunInterval The interval of time between runs of the idle evictor thread for consumer pool. When non-positive, no idle evictor thread will be run.1m (1 minutes)1m (1 minute)
      spark.kafka.consumer.cache.jmx.enable
      spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval The interval of time between runs of the idle evictor thread for fetched data pool. When non-positive, no idle evictor thread will be run.1m (1 minutes)1m (1 minute)
      @@ -802,6 +802,34 @@ df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") \ +### Producer Caching + +Given Kafka producer instance is designed to be thread-safe, Spark initializes a Kafka producer instance and co-use across tasks for same caching key. + +The caching key is built up from the following information: + +* Kafka producer configuration + +This includes configuration for authorization, which Spark will automatically include when delegation token is being used. Even we take authorization into account, you can expect same Kafka producer instance will be used among same Kafka producer configuration. +It will use different Kafka producer when delegation token is renewed; Kafka producer instance for old delegation token will be evicted according to the cache policy. + +The following properties are available to configure the producer pool: + + + + + + + + + + + + + +
      Property NameDefaultMeaning
      spark.kafka.producer.cache.timeoutThe minimum amount of time a producer may sit idle in the pool before it is eligible for eviction by the evictor.10m (10 minutes)
      spark.kafka.producer.cache.evictorThreadRunIntervalThe interval of time between runs of the idle evictor thread for producer pool. When non-positive, no idle evictor thread will be run.1m (1 minute)
      + +Idle eviction thread periodically removes producers which are not used longer than given timeout. Note that the producer is shared and used concurrently, so the last used timestamp is determined by the moment the producer instance is returned and reference count is 0. ## Kafka Specific Configurations From 0823aec4630e70323e66bea243871aaab761d9ca Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 13 Jan 2020 18:47:15 +0900 Subject: [PATCH 1089/1387] [SPARK-30480][PYTHON][TESTS] Increases the memory limit being tested in 'WorkerMemoryTest.test_memory_limit' ### What changes were proposed in this pull request? This PR proposes to increase the memory in `WorkerMemoryTest.test_memory_limit` in order to make the test pass with PyPy. The test is currently failed only in PyPy as below in some PRs unexpectedly: ``` Current mem limits: 18446744073709551615 of max 18446744073709551615 Setting mem limits to 1048576 of max 1048576 RPython traceback: File "pypy_module_pypyjit_interp_jit.c", line 289, in portal_5 File "pypy_interpreter_pyopcode.c", line 3468, in handle_bytecode__AccessDirect_None File "pypy_interpreter_pyopcode.c", line 5558, in dispatch_bytecode__AccessDirect_None out of memory: couldn't allocate the next arena ERROR ``` It seems related to how PyPy allocates the memory and GC works PyPy-specifically. There seems nothing wrong in this configuration implementation itself in PySpark side. I roughly tested in higher PyPy versions on Ubuntu (PyPy v7.3.0) and this test seems passing fine so I suspect this might be an issue in old PyPy behaviours. The change only increases the limit so it would not affect actual memory allocations. It just needs to test if the limit is properly set in worker sides. For clarification, the memory is unlimited in the machine if not set. ### Why are the changes needed? To make the tests pass and unblock other PRs. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually and Jenkins should test it out. Closes #27186 from HyukjinKwon/SPARK-30480. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/tests/test_worker.py | 6 +++--- python/run-tests.py | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/python/pyspark/tests/test_worker.py b/python/pyspark/tests/test_worker.py index ccbe21f3a6f38..9d7deb23da604 100644 --- a/python/pyspark/tests/test_worker.py +++ b/python/pyspark/tests/test_worker.py @@ -183,7 +183,7 @@ def test_reuse_worker_of_parallelize_xrange(self): class WorkerMemoryTest(PySparkTestCase): def test_memory_limit(self): - self.sc._conf.set("spark.executor.pyspark.memory", "1m") + self.sc._conf.set("spark.executor.pyspark.memory", "2g") rdd = self.sc.parallelize(xrange(1), 1) def getrlimit(): @@ -194,8 +194,8 @@ def getrlimit(): self.assertTrue(len(actual) == 1) self.assertTrue(len(actual[0]) == 2) [(soft_limit, hard_limit)] = actual - self.assertEqual(soft_limit, 1024 * 1024) - self.assertEqual(hard_limit, 1024 * 1024) + self.assertEqual(soft_limit, 2 * 1024 * 1024 * 1024) + self.assertEqual(hard_limit, 2 * 1024 * 1024 * 1024) if __name__ == "__main__": diff --git a/python/run-tests.py b/python/run-tests.py index b52eca06a3f6a..b677a5134ec93 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -267,8 +267,8 @@ def main(): python_implementation = subprocess_check_output( [python_exec, "-c", "import platform; print(platform.python_implementation())"], universal_newlines=True).strip() - LOGGER.debug("%s python_implementation is %s", python_exec, python_implementation) - LOGGER.debug("%s version is: %s", python_exec, subprocess_check_output( + LOGGER.info("%s python_implementation is %s", python_exec, python_implementation) + LOGGER.info("%s version is: %s", python_exec, subprocess_check_output( [python_exec, "--version"], stderr=subprocess.STDOUT, universal_newlines=True).strip()) if should_test_modules: for module in modules_to_test: From 6502c66025718bf45e0e2ee12398b7b92da41a0c Mon Sep 17 00:00:00 2001 From: zero323 Date: Mon, 13 Jan 2020 19:03:32 +0800 Subject: [PATCH 1090/1387] [SPARK-30493][PYTHON][ML] Remove OneVsRestModel setClassifier, setLabelCol and setWeightCol methods ### What changes were proposed in this pull request? Removal of `OneVsRestModel.setClassifier`, `OneVsRestModel.setLabelCol` and `OneVsRestModel.setWeightCol` methods. ### Why are the changes needed? Aforementioned methods shouldn't by included by [SPARK-29093](https://issues.apache.org/jira/browse/SPARK-29093), as they're not present in Scala `OneVsRestModel` and have no practical application. ### Does this PR introduce any user-facing change? Not beyond scope of SPARK-29093]. ### How was this patch tested? Existing tests. CC huaxingao zhengruifeng Closes #27181 from zero323/SPARK-30493. Authored-by: zero323 Signed-off-by: zhengruifeng --- python/pyspark/ml/classification.py | 22 ++-------------------- 1 file changed, 2 insertions(+), 20 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index af295ac008450..bfed88fdcb5d5 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -2637,13 +2637,6 @@ class OneVsRestModel(Model, _OneVsRestParams, JavaMLReadable, JavaMLWritable): .. versionadded:: 2.0.0 """ - @since("2.0.0") - def setClassifier(self, value): - """ - Sets the value of :py:attr:`classifier`. - """ - return self._set(classifier=value) - def setFeaturesCol(self, value): """ Sets the value of :py:attr:`featuresCol`. @@ -2662,18 +2655,6 @@ def setRawPredictionCol(self, value): """ return self._set(rawPredictionCol=value) - def setLabelCol(self, value): - """ - Sets the value of :py:attr:`labelCol`. - """ - return self._set(labelCol=value) - - def setWeightCol(self, value): - """ - Sets the value of :py:attr:`weightCol`. - """ - return self._set(weightCol=value) - def __init__(self, models): super(OneVsRestModel, self).__init__() self.models = models @@ -2773,8 +2754,9 @@ def _from_java(cls, java_stage): predictionCol = java_stage.getPredictionCol() classifier = JavaParams._from_java(java_stage.getClassifier()) models = [JavaParams._from_java(model) for model in java_stage.models()] - py_stage = cls(models=models).setPredictionCol(predictionCol).setLabelCol(labelCol)\ + py_stage = cls(models=models).setPredictionCol(predictionCol)\ .setFeaturesCol(featuresCol) + py_stage._set(labelCol=labelCol) py_stage._set(classifier=classifier) py_stage._resetUid(java_stage.uid()) return py_stage From d6e28f29228aa3ff3ba40b00bd79e1844fcc0d71 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 13 Jan 2020 08:22:20 -0600 Subject: [PATCH 1091/1387] [SPARK-30377][ML] Make Regressors extend abstract class Regressor ### What changes were proposed in this pull request? Make Regressors extend abstract class Regressor: ```AFTSurvivalRegression extends Estimator => extends Regressor``` ```DecisionTreeRegressor extends Predictor => extends Regressor``` ```FMRegressor extends Predictor => extends Regressor``` ```GBTRegressor extends Predictor => extends Regressor``` ```RandomForestRegressor extends Predictor => extends Regressor``` We will not make ```IsotonicRegression``` extend ```Regressor``` because it is tricky to handle both DoubleType and VectorType. ### Why are the changes needed? Make class hierarchy consistent for all Regressors ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing tests Closes #27168 from huaxingao/spark-30377. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../ml/regression/AFTSurvivalRegression.scala | 43 +++++----------- .../ml/regression/DecisionTreeRegressor.scala | 4 +- .../spark/ml/regression/FMRegressor.scala | 4 +- .../spark/ml/regression/GBTRegressor.scala | 4 +- .../ml/regression/RandomForestRegressor.scala | 4 +- project/MimaExcludes.scala | 10 +++- python/pyspark/ml/regression.py | 49 ++----------------- 7 files changed, 31 insertions(+), 87 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 811287ff5db35..9c27ea8816495 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ @@ -44,9 +44,8 @@ import org.apache.spark.storage.StorageLevel /** * Params for accelerated failure time (AFT) regression. */ -private[regression] trait AFTSurvivalRegressionParams extends Params - with HasFeaturesCol with HasLabelCol with HasPredictionCol with HasMaxIter - with HasTol with HasFitIntercept with HasAggregationDepth with Logging { +private[regression] trait AFTSurvivalRegressionParams extends PredictorParams + with HasMaxIter with HasTol with HasFitIntercept with HasAggregationDepth with Logging { /** * Param for censor column name. @@ -126,28 +125,16 @@ private[regression] trait AFTSurvivalRegressionParams extends Params */ @Since("1.6.0") class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: String) - extends Estimator[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams - with DefaultParamsWritable with Logging { + extends Regressor[Vector, AFTSurvivalRegression, AFTSurvivalRegressionModel] + with AFTSurvivalRegressionParams with DefaultParamsWritable with Logging { @Since("1.6.0") def this() = this(Identifiable.randomUID("aftSurvReg")) - /** @group setParam */ - @Since("1.6.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("1.6.0") - def setLabelCol(value: String): this.type = set(labelCol, value) - /** @group setParam */ @Since("1.6.0") def setCensorCol(value: String): this.type = set(censorCol, value) - /** @group setParam */ - @Since("1.6.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) - /** @group setParam */ @Since("1.6.0") def setQuantileProbabilities(value: Array[Double]): this.type = set(quantileProbabilities, value) @@ -207,9 +194,8 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S } } - @Since("2.0.0") - override def fit(dataset: Dataset[_]): AFTSurvivalRegressionModel = instrumented { instr => - transformSchema(dataset.schema, logging = true) + @Since("3.0.0") + override def train(dataset: Dataset[_]): AFTSurvivalRegressionModel = instrumented { instr => val instances = extractAFTPoints(dataset) val handlePersistence = dataset.storageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) @@ -281,7 +267,7 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S val coefficients = Vectors.dense(rawCoefficients) val intercept = parameters(1) val scale = math.exp(parameters(0)) - copyValues(new AFTSurvivalRegressionModel(uid, coefficients, intercept, scale).setParent(this)) + new AFTSurvivalRegressionModel(uid, coefficients, intercept, scale) } @Since("1.6.0") @@ -309,18 +295,11 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("2.0.0") val coefficients: Vector, @Since("1.6.0") val intercept: Double, @Since("1.6.0") val scale: Double) - extends Model[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams with MLWritable { + extends RegressionModel[Vector, AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams + with MLWritable { @Since("3.0.0") - lazy val numFeatures: Int = coefficients.size - - /** @group setParam */ - @Since("1.6.0") - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - - /** @group setParam */ - @Since("1.6.0") - def setPredictionCol(value: String): this.type = set(predictionCol, value) + override def numFeatures: Int = coefficients.size /** @group setParam */ @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 20be26e4b011f..e69d082a3cc68 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.types.StructType */ @Since("1.4.0") class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Predictor[Vector, DecisionTreeRegressor, DecisionTreeRegressionModel] + extends Regressor[Vector, DecisionTreeRegressor, DecisionTreeRegressionModel] with DecisionTreeRegressorParams with DefaultParamsWritable { @Since("1.4.0") @@ -159,7 +159,7 @@ class DecisionTreeRegressionModel private[ml] ( override val uid: String, override val rootNode: Node, override val numFeatures: Int) - extends PredictionModel[Vector, DecisionTreeRegressionModel] + extends RegressionModel[Vector, DecisionTreeRegressionModel] with DecisionTreeModel with DecisionTreeRegressorParams with MLWritable with Serializable { /** @group setParam */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala index 5e7d8aa2813a8..06212fdd041b8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala @@ -291,7 +291,7 @@ private[regression] trait FMRegressorParams extends FactorizationMachinesParams @Since("3.0.0") class FMRegressor @Since("3.0.0") ( @Since("3.0.0") override val uid: String) - extends Predictor[Vector, FMRegressor, FMRegressionModel] + extends Regressor[Vector, FMRegressor, FMRegressionModel] with FactorizationMachines with FMRegressorParams with DefaultParamsWritable with Logging { @Since("3.0.0") @@ -454,7 +454,7 @@ class FMRegressionModel private[regression] ( @Since("3.0.0") val intercept: Double, @Since("3.0.0") val linear: Vector, @Since("3.0.0") val factors: Matrix) - extends PredictionModel[Vector, FMRegressionModel] + extends RegressionModel[Vector, FMRegressionModel] with FMRegressorParams with MLWritable { @Since("3.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index f79302d2f68b4..29991f59e37cf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -56,7 +56,7 @@ import org.apache.spark.sql.types.StructType */ @Since("1.4.0") class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Predictor[Vector, GBTRegressor, GBTRegressionModel] + extends Regressor[Vector, GBTRegressor, GBTRegressionModel] with GBTRegressorParams with DefaultParamsWritable with Logging { @Since("1.4.0") @@ -227,7 +227,7 @@ class GBTRegressionModel private[ml]( private val _trees: Array[DecisionTreeRegressionModel], private val _treeWeights: Array[Double], override val numFeatures: Int) - extends PredictionModel[Vector, GBTRegressionModel] + extends RegressionModel[Vector, GBTRegressionModel] with GBTRegressorParams with TreeEnsembleModel[DecisionTreeRegressionModel] with MLWritable with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index fa4dbbb47079f..e66109ef5777b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.types.StructType */ @Since("1.4.0") class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Predictor[Vector, RandomForestRegressor, RandomForestRegressionModel] + extends Regressor[Vector, RandomForestRegressor, RandomForestRegressionModel] with RandomForestRegressorParams with DefaultParamsWritable { @Since("1.4.0") @@ -170,7 +170,7 @@ class RandomForestRegressionModel private[ml] ( override val uid: String, private val _trees: Array[DecisionTreeRegressionModel], override val numFeatures: Int) - extends PredictionModel[Vector, RandomForestRegressionModel] + extends RegressionModel[Vector, RandomForestRegressionModel] with RandomForestRegressorParams with TreeEnsembleModel[DecisionTreeRegressionModel] with MLWritable with Serializable { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 8fe99cc65215b..b6ec869c425e6 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -469,7 +469,15 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.appendS3AndSparkHadoopConfigurations"), // [SPARK-29348] Add observable metrics. - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryProgress.this") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryProgress.this"), + + // [SPARK-30377][ML] Make AFTSurvivalRegression extend Regressor + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.fit"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.setFeaturesCol"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.setPredictionCol"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.setFeaturesCol"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.setLabelCol"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.setPredictionCol") ) // Exclude rules for 2.4.x diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 8ec277a8dbb2b..70c131795d9f3 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1515,8 +1515,7 @@ def evaluateEachIteration(self, dataset, loss): return self._call_java("evaluateEachIteration", dataset, loss) -class _AFTSurvivalRegressionParams(HasFeaturesCol, HasLabelCol, HasPredictionCol, - HasMaxIter, HasTol, HasFitIntercept, +class _AFTSurvivalRegressionParams(_JavaPredictorParams, HasMaxIter, HasTol, HasFitIntercept, HasAggregationDepth): """ Params for :py:class:`AFTSurvivalRegression` and :py:class:`AFTSurvivalRegressionModel`. @@ -1563,7 +1562,7 @@ def getQuantilesCol(self): @inherit_doc -class AFTSurvivalRegression(JavaEstimator, _AFTSurvivalRegressionParams, +class AFTSurvivalRegression(JavaPredictor, _AFTSurvivalRegressionParams, JavaMLWritable, JavaMLReadable): """ Accelerated Failure Time (AFT) Model Survival Regression @@ -1682,27 +1681,6 @@ def setMaxIter(self, value): """ return self._set(maxIter=value) - @since("1.6.0") - def setFeaturesCol(self, value): - """ - Sets the value of :py:attr:`featuresCol`. - """ - return self._set(featuresCol=value) - - @since("1.6.0") - def setPredictionCol(self, value): - """ - Sets the value of :py:attr:`predictionCol`. - """ - return self._set(predictionCol=value) - - @since("1.6.0") - def setLabelCol(self, value): - """ - Sets the value of :py:attr:`labelCol`. - """ - return self._set(labelCol=value) - @since("1.6.0") def setTol(self, value): """ @@ -1725,7 +1703,7 @@ def setAggregationDepth(self, value): return self._set(aggregationDepth=value) -class AFTSurvivalRegressionModel(JavaModel, _AFTSurvivalRegressionParams, +class AFTSurvivalRegressionModel(JavaPredictionModel, _AFTSurvivalRegressionParams, JavaMLWritable, JavaMLReadable): """ Model fitted by :class:`AFTSurvivalRegression`. @@ -1733,20 +1711,6 @@ class AFTSurvivalRegressionModel(JavaModel, _AFTSurvivalRegressionParams, .. versionadded:: 1.6.0 """ - @since("3.0.0") - def setFeaturesCol(self, value): - """ - Sets the value of :py:attr:`featuresCol`. - """ - return self._set(featuresCol=value) - - @since("3.0.0") - def setPredictionCol(self, value): - """ - Sets the value of :py:attr:`predictionCol`. - """ - return self._set(predictionCol=value) - @since("3.0.0") def setQuantileProbabilities(self, value): """ @@ -1792,13 +1756,6 @@ def predictQuantiles(self, features): """ return self._call_java("predictQuantiles", features) - @since("2.0.0") - def predict(self, features): - """ - Predicted value - """ - return self._call_java("predict", features) - class _GeneralizedLinearRegressionParams(_JavaPredictorParams, HasFitIntercept, HasMaxIter, HasTol, HasRegParam, HasWeightCol, HasSolver, From f77dcfc55af968cdfb29ac798e21229fe7f6c063 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 13 Jan 2020 08:24:49 -0600 Subject: [PATCH 1092/1387] [SPARK-30351][ML][PYSPARK] BisectingKMeans support instance weighting ### What changes were proposed in this pull request? add weight support in BisectingKMeans ### Why are the changes needed? BisectingKMeans should support instance weighting ### Does this PR introduce any user-facing change? Yes. BisectingKMeans.setWeight ### How was this patch tested? Unit test Closes #27035 from huaxingao/spark_30351. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../spark/ml/clustering/BisectingKMeans.scala | 40 ++++-- .../mllib/clustering/BisectingKMeans.scala | 48 +++++-- .../mllib/clustering/DistanceMeasure.scala | 36 ++--- .../spark/mllib/clustering/KMeans.scala | 28 ++-- .../ml/clustering/BisectingKMeansSuite.scala | 128 ++++++++++++++++++ python/pyspark/ml/clustering.py | 31 +++-- 6 files changed, 241 insertions(+), 70 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 79760d69489c6..6c7112b80569f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -28,10 +28,12 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.clustering.{BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.functions.udf -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} import org.apache.spark.storage.StorageLevel @@ -39,7 +41,8 @@ import org.apache.spark.storage.StorageLevel * Common params for BisectingKMeans and BisectingKMeansModel */ private[clustering] trait BisectingKMeansParams extends Params with HasMaxIter - with HasFeaturesCol with HasSeed with HasPredictionCol with HasDistanceMeasure { + with HasFeaturesCol with HasSeed with HasPredictionCol with HasDistanceMeasure + with HasWeightCol { /** * The desired number of leaf clusters. Must be > 1. Default: 4. @@ -261,20 +264,39 @@ class BisectingKMeans @Since("2.0.0") ( @Since("2.4.0") def setDistanceMeasure(value: String): this.type = set(distanceMeasure, value) + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + @Since("2.0.0") override def fit(dataset: Dataset[_]): BisectingKMeansModel = instrumented { instr => transformSchema(dataset.schema, logging = true) val handlePersistence = dataset.storageLevel == StorageLevel.NONE - val rdd = DatasetUtils.columnToOldVector(dataset, getFeaturesCol) + val w = if (isDefined(weightCol) && $(weightCol).nonEmpty) { + col($(weightCol)).cast(DoubleType) + } else { + lit(1.0) + } + + val instances: RDD[(OldVector, Double)] = dataset + .select(DatasetUtils.columnToVector(dataset, getFeaturesCol), w).rdd.map { + case Row(point: Vector, weight: Double) => (OldVectors.fromML(point), weight) + } if (handlePersistence) { - rdd.persist(StorageLevel.MEMORY_AND_DISK) + instances.persist(StorageLevel.MEMORY_AND_DISK) } instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, featuresCol, predictionCol, k, maxIter, seed, - minDivisibleClusterSize, distanceMeasure) + minDivisibleClusterSize, distanceMeasure, weightCol) val bkm = new MLlibBisectingKMeans() .setK($(k)) @@ -282,10 +304,10 @@ class BisectingKMeans @Since("2.0.0") ( .setMinDivisibleClusterSize($(minDivisibleClusterSize)) .setSeed($(seed)) .setDistanceMeasure($(distanceMeasure)) - val parentModel = bkm.run(rdd, Some(instr)) + val parentModel = bkm.runWithWeight(instances, Some(instr)) val model = copyValues(new BisectingKMeansModel(uid, parentModel).setParent(this)) if (handlePersistence) { - rdd.unpersist() + instances.unpersist() } val summary = new BisectingKMeansSummary( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index b4a31d72f94b9..7c12697be95c8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -27,6 +27,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.ml.util.Instrumentation import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.BLAS.axpy import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -152,24 +153,34 @@ class BisectingKMeans private ( this } - private[spark] def run( input: RDD[Vector], instr: Option[Instrumentation]): BisectingKMeansModel = { - if (input.getStorageLevel == StorageLevel.NONE) { - logWarning(s"The input RDD ${input.id} is not directly cached, which may hurt performance if" - + " its parent RDDs are also not cached.") + val instances: RDD[(Vector, Double)] = input.map { + case (point) => (point, 1.0) } - val d = input.map(_.size).first() + runWithWeight(instances, None) + } + + private[spark] def runWithWeight( + input: RDD[(Vector, Double)], + instr: Option[Instrumentation]): BisectingKMeansModel = { + val d = input.map(_._1.size).first logInfo(s"Feature dimension: $d.") val dMeasure: DistanceMeasure = DistanceMeasure.decodeFromString(this.distanceMeasure) // Compute and cache vector norms for fast distance computation. - val norms = input.map(v => Vectors.norm(v, 2.0)).persist(StorageLevel.MEMORY_AND_DISK) - val vectors = input.zip(norms).map { case (x, norm) => new VectorWithNorm(x, norm) } + val norms = input.map(d => Vectors.norm(d._1, 2.0)) + val vectors = input.zip(norms).map { + case ((x, weight), norm) => new VectorWithNorm(x, norm, weight) + } + if (input.getStorageLevel == StorageLevel.NONE) { + vectors.persist(StorageLevel.MEMORY_AND_DISK) + } var assignments = vectors.map(v => (ROOT_INDEX, v)) var activeClusters = summarize(d, assignments, dMeasure) instr.foreach(_.logNumExamples(activeClusters.values.map(_.size).sum)) + instr.foreach(_.logSumOfWeights(activeClusters.values.map(_.weightSum).sum)) val rootSummary = activeClusters(ROOT_INDEX) val n = rootSummary.size logInfo(s"Number of points: $n.") @@ -239,7 +250,7 @@ class BisectingKMeans private ( if (indices != null) { indices.unpersist() } - norms.unpersist() + vectors.unpersist() val clusters = activeClusters ++ inactiveClusters val root = buildTree(clusters, dMeasure) val totalCost = root.leafNodes.map(_.cost).sum @@ -312,14 +323,16 @@ private object BisectingKMeans extends Serializable { private class ClusterSummaryAggregator(val d: Int, val distanceMeasure: DistanceMeasure) extends Serializable { private var n: Long = 0L + private var weightSum: Double = 0.0 private val sum: Vector = Vectors.zeros(d) private var sumSq: Double = 0.0 /** Adds a point. */ def add(v: VectorWithNorm): this.type = { n += 1L + weightSum += v.weight // TODO: use a numerically stable approach to estimate cost - sumSq += v.norm * v.norm + sumSq += v.norm * v.norm * v.weight distanceMeasure.updateClusterSum(v, sum) this } @@ -327,16 +340,18 @@ private object BisectingKMeans extends Serializable { /** Merges another aggregator. */ def merge(other: ClusterSummaryAggregator): this.type = { n += other.n + weightSum += other.weightSum sumSq += other.sumSq - distanceMeasure.updateClusterSum(new VectorWithNorm(other.sum), sum) + axpy(1.0, other.sum, sum) this } /** Returns the summary. */ def summary: ClusterSummary = { - val center = distanceMeasure.centroid(sum.copy, n) - val cost = distanceMeasure.clusterCost(center, new VectorWithNorm(sum), n, sumSq) - ClusterSummary(n, center, cost) + val center = distanceMeasure.centroid(sum.copy, weightSum) + val cost = distanceMeasure.clusterCost(center, new VectorWithNorm(sum), weightSum, + sumSq) + ClusterSummary(n, weightSum, center, cost) } } @@ -437,10 +452,15 @@ private object BisectingKMeans extends Serializable { * Summary of a cluster. * * @param size the number of points within this cluster + * @param weightSum the weightSum within this cluster * @param center the center of the points within this cluster * @param cost the sum of squared distances to the center */ - private case class ClusterSummary(size: Long, center: VectorWithNorm, cost: Double) + private case class ClusterSummary( + size: Long, + weightSum: Double, + center: VectorWithNorm, + cost: Double) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala index 4d89d1fb9aa92..e83dd3723be15 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/DistanceMeasure.scala @@ -76,7 +76,7 @@ private[spark] abstract class DistanceMeasure extends Serializable { def clusterCost( centroid: VectorWithNorm, pointsSum: VectorWithNorm, - numberOfPoints: Long, + weightSum: Double, pointsSquaredNorm: Double): Double /** @@ -84,20 +84,8 @@ private[spark] abstract class DistanceMeasure extends Serializable { * @param point a `VectorWithNorm` to be added to `sum` of a cluster * @param sum the `sum` for a cluster to be updated */ - def updateClusterSum(point: VectorWithNorm, sum: Vector, weight: Double = 1.0): Unit = { - axpy(weight, point.vector, sum) - } - - /** - * Returns a centroid for a cluster given its `sum` vector and its `count` of points. - * - * @param sum the `sum` for a cluster - * @param count the number of points in the cluster - * @return the centroid of the cluster - */ - def centroid(sum: Vector, count: Long): VectorWithNorm = { - scal(1.0 / count, sum) - new VectorWithNorm(sum) + def updateClusterSum(point: VectorWithNorm, sum: Vector): Unit = { + axpy(point.weight, point.vector, sum) } /** @@ -217,9 +205,9 @@ private[spark] class EuclideanDistanceMeasure extends DistanceMeasure { override def clusterCost( centroid: VectorWithNorm, pointsSum: VectorWithNorm, - numberOfPoints: Long, + weightSum: Double, pointsSquaredNorm: Double): Double = { - math.max(pointsSquaredNorm - numberOfPoints * centroid.norm * centroid.norm, 0.0) + math.max(pointsSquaredNorm - weightSum * centroid.norm * centroid.norm, 0.0) } /** @@ -261,20 +249,20 @@ private[spark] class CosineDistanceMeasure extends DistanceMeasure { * @param point a `VectorWithNorm` to be added to `sum` of a cluster * @param sum the `sum` for a cluster to be updated */ - override def updateClusterSum(point: VectorWithNorm, sum: Vector, weight: Double = 1.0): Unit = { + override def updateClusterSum(point: VectorWithNorm, sum: Vector): Unit = { assert(point.norm > 0, "Cosine distance is not defined for zero-length vectors.") - axpy(weight / point.norm, point.vector, sum) + axpy(point.weight / point.norm, point.vector, sum) } /** * Returns a centroid for a cluster given its `sum` vector and its `count` of points. * * @param sum the `sum` for a cluster - * @param count the number of points in the cluster + * @param weightSum the sum of weight in the cluster * @return the centroid of the cluster */ - override def centroid(sum: Vector, count: Long): VectorWithNorm = { - scal(1.0 / count, sum) + override def centroid(sum: Vector, weightSum: Double): VectorWithNorm = { + scal(1.0 / weightSum, sum) val norm = Vectors.norm(sum, 2) scal(1.0 / norm, sum) new VectorWithNorm(sum, 1) @@ -286,10 +274,10 @@ private[spark] class CosineDistanceMeasure extends DistanceMeasure { override def clusterCost( centroid: VectorWithNorm, pointsSum: VectorWithNorm, - numberOfPoints: Long, + weightSum: Double, pointsSquaredNorm: Double): Double = { val costVector = pointsSum.vector.copy - math.max(numberOfPoints - dot(centroid.vector, costVector) / centroid.norm, 0.0) + math.max(weightSum - dot(centroid.vector, costVector) / centroid.norm, 0.0) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index b24dc23985591..a3cf7f96478af 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -225,7 +225,7 @@ class KMeans private ( } val zippedData = data.zip(norms).map { case ((v, w), norm) => - (new VectorWithNorm(v, norm), w) + new VectorWithNorm(v, norm, w) } if (data.getStorageLevel == StorageLevel.NONE) { @@ -241,7 +241,7 @@ class KMeans private ( * Implementation of K-Means algorithm. */ private def runAlgorithmWithWeight( - data: RDD[(VectorWithNorm, Double)], + data: RDD[VectorWithNorm], instr: Option[Instrumentation]): KMeansModel = { val sc = data.sparkContext @@ -250,16 +250,14 @@ class KMeans private ( val distanceMeasureInstance = DistanceMeasure.decodeFromString(this.distanceMeasure) - val dataVectorWithNorm = data.map(d => d._1) - val centers = initialModel match { case Some(kMeansCenters) => kMeansCenters.clusterCenters.map(new VectorWithNorm(_)) case None => if (initializationMode == KMeans.RANDOM) { - initRandom(dataVectorWithNorm) + initRandom(data) } else { - initKMeansParallel(dataVectorWithNorm, distanceMeasureInstance) + initKMeansParallel(data, distanceMeasureInstance) } } val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 @@ -279,7 +277,7 @@ class KMeans private ( val bcCenters = sc.broadcast(centers) // Find the new centers - val collected = data.mapPartitions { pointsAndWeights => + val collected = data.mapPartitions { points => val thisCenters = bcCenters.value val dims = thisCenters.head.vector.size @@ -290,11 +288,11 @@ class KMeans private ( // sample1 * weight1/clusterWeightSum + sample2 * weight2/clusterWeightSum + ... val clusterWeightSum = Array.ofDim[Double](thisCenters.length) - pointsAndWeights.foreach { case (point, weight) => + points.foreach { point => val (bestCenter, cost) = distanceMeasureInstance.findClosest(thisCenters, point) - costAccum.add(cost * weight) - distanceMeasureInstance.updateClusterSum(point, sums(bestCenter), weight) - clusterWeightSum(bestCenter) += weight + costAccum.add(cost * point.weight) + distanceMeasureInstance.updateClusterSum(point, sums(bestCenter)) + clusterWeightSum(bestCenter) += point.weight } clusterWeightSum.indices.filter(clusterWeightSum(_) > 0) @@ -511,13 +509,15 @@ object KMeans { /** * A vector with its norm for fast distance computation. */ -private[clustering] class VectorWithNorm(val vector: Vector, val norm: Double) - extends Serializable { +private[clustering] class VectorWithNorm( + val vector: Vector, + val norm: Double, + val weight: Double = 1.0) extends Serializable { def this(vector: Vector) = this(vector, Vectors.norm(vector, 2.0)) def this(array: Array[Double]) = this(Vectors.dense(array)) /** Converts the vector to a dense vector. */ - def toDense: VectorWithNorm = new VectorWithNorm(Vectors.dense(vector.toArray), norm) + def toDense: VectorWithNorm = new VectorWithNorm(Vectors.dense(vector.toArray), norm, weight) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index 9984451b08ce1..fc756d4c710d3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -189,6 +189,134 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { model.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) } + test("Comparing with and without weightCol with cosine distance") { + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( + Vectors.dense(1.0, 1.0), + Vectors.dense(10.0, 10.0), + Vectors.dense(1.0, 0.5), + Vectors.dense(10.0, 4.4), + Vectors.dense(-1.0, 1.0), + Vectors.dense(-100.0, 90.0) + )).map(v => TestRow(v))) + + val model1 = new BisectingKMeans() + .setK(3) + .setDistanceMeasure(DistanceMeasure.COSINE) + .setSeed(1) + .fit(df1) + val predictionDf1 = model1.transform(df1) + checkNominalOnDF(predictionDf1, "prediction", model1.getK) + + assert(predictionDf1.select("prediction").distinct().count() == 3) + val predictionsMap1 = predictionDf1.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap1(Vectors.dense(1.0, 1.0)) == + predictionsMap1(Vectors.dense(10.0, 10.0))) + assert(predictionsMap1(Vectors.dense(1.0, 0.5)) == + predictionsMap1(Vectors.dense(10.0, 4.4))) + assert(predictionsMap1(Vectors.dense(-1.0, 1.0)) == + predictionsMap1(Vectors.dense(-100.0, 90.0))) + + model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + (Vectors.dense(1.0, 1.0), 2.0), (Vectors.dense(10.0, 10.0), 2.0), + (Vectors.dense(1.0, 0.5), 2.0), (Vectors.dense(10.0, 4.4), 2.0), + (Vectors.dense(-1.0, 1.0), 2.0), (Vectors.dense(-100.0, 90.0), 2.0)))) + .toDF("features", "weightCol") + + val model2 = new BisectingKMeans() + .setK(3) + .setDistanceMeasure(DistanceMeasure.COSINE) + .setSeed(1) + .setWeightCol("weightCol") + .fit(df2) + val predictionDf2 = model2.transform(df2) + checkNominalOnDF(predictionDf2, "prediction", model2.getK) + + assert(predictionDf2.select("prediction").distinct().count() == 3) + val predictionsMap2 = predictionDf2.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap2(Vectors.dense(1.0, 1.0)) == + predictionsMap2(Vectors.dense(10.0, 10.0))) + assert(predictionsMap2(Vectors.dense(1.0, 0.5)) == + predictionsMap2(Vectors.dense(10.0, 4.4))) + assert(predictionsMap2(Vectors.dense(-1.0, 1.0)) == + predictionsMap2(Vectors.dense(-100.0, 90.0))) + + model2.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + assert(model1.clusterCenters === model2.clusterCenters) + } + + test("Comparing with and without weightCol") { + val df1 = spark.createDataFrame(spark.sparkContext.parallelize(Seq( + Vectors.dense(1.0, 1.0), + Vectors.dense(10.0, 10.0), + Vectors.dense(10.0, 10.0), + Vectors.dense(1.0, 0.5), + Vectors.dense(1.0, 0.5), + Vectors.dense(10.0, 4.4), + Vectors.dense(10.0, 4.4), + Vectors.dense(10.0, 4.4), + Vectors.dense(-1.0, 1.0), + Vectors.dense(-1.0, 1.0), + Vectors.dense(-1.0, 1.0), + Vectors.dense(-100.0, 90.0), + Vectors.dense(-100.0, 90.0), + Vectors.dense(-100.0, 90.0), + Vectors.dense(-100.0, 90.0) + )).map(v => TestRow(v))) + + val model1 = new BisectingKMeans() + .setK(3) + .setSeed(1) + .fit(df1) + val predictionDf1 = model1.transform(df1) + checkNominalOnDF(predictionDf1, "prediction", model1.getK) + + assert(predictionDf1.select("prediction").distinct().count() == 3) + val predictionsMap1 = predictionDf1.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap1(Vectors.dense(1.0, 1.0)) == + predictionsMap1(Vectors.dense(1.0, 0.5))) + assert(predictionsMap1(Vectors.dense(1.0, 1.0)) == + predictionsMap1(Vectors.dense(-1.0, 1.0))) + assert(predictionsMap1(Vectors.dense(10.0, 10.0)) == + predictionsMap1(Vectors.dense(10.0, 4.4))) + + model1.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + val df2 = spark.createDataFrame(spark.sparkContext.parallelize(Array( + (Vectors.dense(1.0, 1.0), 1.0), (Vectors.dense(10.0, 10.0), 2.0), + (Vectors.dense(1.0, 0.5), 2.0), (Vectors.dense(10.0, 4.4), 3.0), + (Vectors.dense(-1.0, 1.0), 3.0), (Vectors.dense(-100.0, 90.0), 4.0)))) + .toDF("features", "weightCol") + + val model2 = new BisectingKMeans() + .setK(3) + .setSeed(1) + .setWeightCol("weightCol") + .fit(df2) + val predictionDf2 = model2.transform(df2) + checkNominalOnDF(predictionDf2, "prediction", model2.getK) + + assert(predictionDf2.select("prediction").distinct().count() == 3) + val predictionsMap2 = predictionDf2.collect().map(row => + row.getAs[Vector]("features") -> row.getAs[Int]("prediction")).toMap + assert(predictionsMap2(Vectors.dense(1.0, 1.0)) == + predictionsMap2(Vectors.dense(1.0, 0.5))) + assert(predictionsMap2(Vectors.dense(1.0, 1.0)) == + predictionsMap2(Vectors.dense(-1.0, 1.0))) + assert(predictionsMap2(Vectors.dense(10.0, 10.0)) == + predictionsMap2(Vectors.dense(10.0, 4.4))) + + model2.clusterCenters.forall(Vectors.norm(_, 2) == 1.0) + + assert(model1.clusterCenters(0) === model2.clusterCenters(0)) + assert(model1.clusterCenters(1) === model2.clusterCenters(1)) + assert(model1.clusterCenters(2) ~== model2.clusterCenters(2) absTol 1e-6) + } + test("BisectingKMeans with Array input") { def trainAndComputeCost(dataset: DataFrame): Double = { val model = new BisectingKMeans().setK(k).setMaxIter(1).setSeed(1).fit(dataset) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index d9b6b886dc511..e236ff0423c91 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -733,7 +733,7 @@ def setWeightCol(self, value): @inherit_doc class _BisectingKMeansParams(HasMaxIter, HasFeaturesCol, HasSeed, HasPredictionCol, - HasDistanceMeasure): + HasDistanceMeasure, HasWeightCol): """ Params for :py:class:`BisectingKMeans` and :py:class:`BisectingKMeansModel`. @@ -838,9 +838,9 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav clusters, larger clusters get higher priority. >>> from pyspark.ml.linalg import Vectors - >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), - ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] - >>> df = spark.createDataFrame(data, ["features"]) + >>> data = [(Vectors.dense([0.0, 0.0]), 2.0), (Vectors.dense([1.0, 1.0]), 2.0), + ... (Vectors.dense([9.0, 8.0]), 2.0), (Vectors.dense([8.0, 9.0]), 2.0)] + >>> df = spark.createDataFrame(data, ["features", "weighCol"]) >>> bkm = BisectingKMeans(k=2, minDivisibleClusterSize=1.0) >>> bkm.setMaxIter(10) BisectingKMeans... @@ -849,6 +849,8 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav >>> bkm.clear(bkm.maxIter) >>> bkm.setSeed(1) BisectingKMeans... + >>> bkm.setWeightCol("weighCol") + BisectingKMeans... >>> bkm.getSeed() 1 >>> bkm.clear(bkm.seed) @@ -872,7 +874,7 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav >>> summary.clusterSizes [2, 2] >>> summary.trainingCost - 2.000... + 4.000... >>> transformed = model.transform(df).select("features", "newPrediction") >>> rows = transformed.collect() >>> rows[0].newPrediction == rows[1].newPrediction @@ -901,10 +903,12 @@ class BisectingKMeans(JavaEstimator, _BisectingKMeansParams, JavaMLWritable, Jav @keyword_only def __init__(self, featuresCol="features", predictionCol="prediction", maxIter=20, - seed=None, k=4, minDivisibleClusterSize=1.0, distanceMeasure="euclidean"): + seed=None, k=4, minDivisibleClusterSize=1.0, distanceMeasure="euclidean", + weightCol=None): """ __init__(self, featuresCol="features", predictionCol="prediction", maxIter=20, \ - seed=None, k=4, minDivisibleClusterSize=1.0, distanceMeasure="euclidean") + seed=None, k=4, minDivisibleClusterSize=1.0, distanceMeasure="euclidean", \ + weightCol=None) """ super(BisectingKMeans, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.BisectingKMeans", @@ -916,10 +920,12 @@ def __init__(self, featuresCol="features", predictionCol="prediction", maxIter=2 @keyword_only @since("2.0.0") def setParams(self, featuresCol="features", predictionCol="prediction", maxIter=20, - seed=None, k=4, minDivisibleClusterSize=1.0, distanceMeasure="euclidean"): + seed=None, k=4, minDivisibleClusterSize=1.0, distanceMeasure="euclidean", + weightCol=None): """ setParams(self, featuresCol="features", predictionCol="prediction", maxIter=20, \ - seed=None, k=4, minDivisibleClusterSize=1.0, distanceMeasure="euclidean") + seed=None, k=4, minDivisibleClusterSize=1.0, distanceMeasure="euclidean", \ + weightCol=None) Sets params for BisectingKMeans. """ kwargs = self._input_kwargs @@ -974,6 +980,13 @@ def setSeed(self, value): """ return self._set(seed=value) + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + def _create_model(self, java_model): return BisectingKMeansModel(java_model) From b389b8c5f0650a7e63098f18437fcaa29998732a Mon Sep 17 00:00:00 2001 From: jiake Date: Mon, 13 Jan 2020 22:55:19 +0800 Subject: [PATCH 1093/1387] [SPARK-30188][SQL] Resolve the failed unit tests when enable AQE ### What changes were proposed in this pull request? Fix all the failed tests when enable AQE. ### Why are the changes needed? Run more tests with AQE to catch bugs, and make it easier to enable AQE by default in the future. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing unit tests Closes #26813 from JkSelf/enableAQEDefault. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../fulltests/test_mllib_recommendation.R | 3 +- .../spark/ml/recommendation/ALSSuite.scala | 7 +- python/pyspark/sql/column.py | 3 +- python/pyspark/sql/dataframe.py | 12 +-- .../sql/tests/test_pandas_udf_grouped_agg.py | 26 +++--- python/pyspark/sql/udf.py | 3 +- python/pyspark/sql/window.py | 16 ++-- .../spark/sql/execution/CacheManager.scala | 21 +++-- .../adaptive/AdaptiveSparkPlanExec.scala | 6 +- .../adaptive/AdaptiveSparkPlanHelper.scala | 12 ++- .../adaptive/DemoteBroadcastHashJoin.scala | 3 +- .../adaptive/InsertAdaptiveSparkPlan.scala | 21 ++++- .../adaptive/OptimizeLocalShuffleReader.scala | 7 +- .../resources/sql-tests/inputs/explain.sql | 3 + .../apache/spark/sql/CachedTableSuite.scala | 18 ++-- .../spark/sql/ConfigBehaviorSuite.scala | 6 +- .../spark/sql/DataFrameAggregateSuite.scala | 19 +++-- .../apache/spark/sql/DataFrameJoinSuite.scala | 7 +- .../org/apache/spark/sql/DataFrameSuite.scala | 27 +++--- .../sql/DataFrameWindowFunctionsSuite.scala | 7 +- .../apache/spark/sql/DatasetCacheSuite.scala | 9 +- .../org/apache/spark/sql/DatasetSuite.scala | 7 +- .../sql/DynamicPartitionPruningSuite.scala | 10 ++- .../org/apache/spark/sql/ExplainSuite.scala | 12 +++ .../spark/sql/FileBasedDataSourceSuite.scala | 13 +-- .../org/apache/spark/sql/JoinHintSuite.scala | 13 +-- .../org/apache/spark/sql/JoinSuite.scala | 11 +-- .../org/apache/spark/sql/SQLQuerySuite.scala | 7 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 5 +- .../sql/SparkSessionExtensionSuite.scala | 3 + .../org/apache/spark/sql/SubquerySuite.scala | 11 ++- .../sql/connector/DataSourceV2Suite.scala | 13 +-- .../execution/BroadcastExchangeSuite.scala | 13 +-- .../DeprecatedWholeStageCodegenSuite.scala | 25 ++++-- .../LogicalPlanTagInSparkPlanSuite.scala | 14 ++++ .../spark/sql/execution/PlannerSuite.scala | 82 +++++++++++-------- .../WholeStageCodegenSparkSubmitSuite.scala | 1 + .../execution/WholeStageCodegenSuite.scala | 13 +++ .../datasources/SchemaPruningSuite.scala | 6 +- .../orc/OrcV2SchemaPruningSuite.scala | 5 +- .../parquet/ParquetSchemaPruningSuite.scala | 5 +- .../sql/execution/debug/DebuggingSuite.scala | 15 ++++ .../execution/joins/BroadcastJoinSuite.scala | 12 +-- .../execution/metric/SQLMetricsSuite.scala | 13 +++ .../python/BatchEvalPythonExecSuite.scala | 7 +- .../ui/SQLAppStatusListenerSuite.scala | 10 ++- .../internal/ExecutorSideSQLConfSuite.scala | 4 +- .../sql/util/DataFrameCallbackSuite.scala | 8 +- .../sql/hive/execution/HiveExplainSuite.scala | 28 ++++--- .../sql/hive/execution/HiveUDAFSuite.scala | 10 ++- .../execution/ObjectHashAggregateSuite.scala | 10 ++- .../sql/hive/execution/SQLMetricsSuite.scala | 14 ++++ 52 files changed, 430 insertions(+), 206 deletions(-) diff --git a/R/pkg/tests/fulltests/test_mllib_recommendation.R b/R/pkg/tests/fulltests/test_mllib_recommendation.R index d50de4123aeb0..73f6cfd67cee9 100644 --- a/R/pkg/tests/fulltests/test_mllib_recommendation.R +++ b/R/pkg/tests/fulltests/test_mllib_recommendation.R @@ -31,7 +31,8 @@ test_that("spark.als", { stats <- summary(model) expect_equal(stats$rank, 10) test <- createDataFrame(list(list(0, 2), list(1, 0), list(2, 0)), c("user", "item")) - predictions <- collect(predict(model, test)) + result <- predict(model, test) + predictions <- collect(arrange(result, desc(result$item), result$user)) expect_equal(predictions$prediction, c(0.6324540, 3.6218479, -0.4568263), tolerance = 1e-4) diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index b7bb127adb94a..a4d1d453ca5c1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -661,11 +661,12 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { (ex, act) => ex.userFactors.first().getSeq[Float](1) === act.userFactors.first().getSeq[Float](1) } { (ex, act, df, enc) => + // With AQE on/off, the order of result may be different. Here sortby the result. val expected = ex.transform(df).selectExpr("prediction") - .first().getFloat(0) + .sort("prediction").first().getFloat(0) testTransformerByGlobalCheckFunc(df, act, "prediction") { case rows: Seq[Row] => - expected ~== rows.head.getFloat(0) absTol 1e-6 + expected ~== rows.sortBy(_.getFloat(0)).head.getFloat(0) absTol 1e-6 }(enc) } } @@ -696,7 +697,7 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { val model = als.fit(df) def testTransformIdExceedsIntRange[A : Encoder](dataFrame: DataFrame): Unit = { val e1 = intercept[SparkException] { - model.transform(dataFrame).first + model.transform(dataFrame).collect() } TestUtils.assertExceptionMsg(e1, msg) val e2 = intercept[StreamingQueryException] { diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index b472a4221cd0c..59d1408e26ad5 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -669,8 +669,9 @@ def over(self, window): >>> window = Window.partitionBy("name").orderBy("age") \ .rowsBetween(Window.unboundedPreceding, Window.currentRow) >>> from pyspark.sql.functions import rank, min + >>> from pyspark.sql.functions import desc >>> df.withColumn("rank", rank().over(window)) \ - .withColumn("min", min('age').over(window)).show() + .withColumn("min", min('age').over(window)).sort(desc("age")).show() +---+-----+----+---+ |age| name|rank|min| +---+-----+----+---+ diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 84fee0816d824..669de26f21dfc 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1016,7 +1016,8 @@ def alias(self, alias): >>> df_as1 = df.alias("df_as1") >>> df_as2 = df.alias("df_as2") >>> joined_df = df_as1.join(df_as2, col("df_as1.name") == col("df_as2.name"), 'inner') - >>> joined_df.select("df_as1.name", "df_as2.name", "df_as2.age").collect() + >>> joined_df.select("df_as1.name", "df_as2.name", "df_as2.age") \ + .sort(desc("df_as1.name")).collect() [Row(name=u'Bob', name=u'Bob', age=5), Row(name=u'Alice', name=u'Alice', age=2)] """ assert isinstance(alias, basestring), "alias should be a string" @@ -1057,11 +1058,12 @@ def join(self, other, on=None, how=None): ``anti``, ``leftanti`` and ``left_anti``. The following performs a full outer join between ``df1`` and ``df2``. + >>> from pyspark.sql.functions import desc + >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height) \ + .sort(desc("name")).collect() + [Row(name=u'Bob', height=85), Row(name=u'Alice', height=None), Row(name=None, height=80)] - >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() - [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] - - >>> df.join(df2, 'name', 'outer').select('name', 'height').collect() + >>> df.join(df2, 'name', 'outer').select('name', 'height').sort(desc("name")).collect() [Row(name=u'Tom', height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] >>> cond = [df.name == df3.name, df.age == df3.age] diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py index 6d460df66da28..974ad560daebf 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py @@ -319,16 +319,18 @@ def test_complex_groupby(self): expected4 = df.groupby(plus_one(df.id)).agg(sum(df.v)) # groupby one scalar pandas UDF - result5 = df.groupby(plus_two(df.id)).agg(sum_udf(df.v)) - expected5 = df.groupby(plus_two(df.id)).agg(sum(df.v)) + result5 = df.groupby(plus_two(df.id)).agg(sum_udf(df.v)).sort('sum(v)') + expected5 = df.groupby(plus_two(df.id)).agg(sum(df.v)).sort('sum(v)') # groupby one expression and one python UDF result6 = df.groupby(df.v % 2, plus_one(df.id)).agg(sum_udf(df.v)) expected6 = df.groupby(df.v % 2, plus_one(df.id)).agg(sum(df.v)) # groupby one expression and one scalar pandas UDF - result7 = df.groupby(df.v % 2, plus_two(df.id)).agg(sum_udf(df.v)).sort('sum(v)') - expected7 = df.groupby(df.v % 2, plus_two(df.id)).agg(sum(df.v)).sort('sum(v)') + result7 = (df.groupby(df.v % 2, plus_two(df.id)) + .agg(sum_udf(df.v)).sort(['sum(v)', 'plus_two(id)'])) + expected7 = (df.groupby(df.v % 2, plus_two(df.id)) + .agg(sum(df.v)).sort(['sum(v)', 'plus_two(id)'])) assert_frame_equal(expected1.toPandas(), result1.toPandas()) assert_frame_equal(expected2.toPandas(), result2.toPandas()) @@ -354,8 +356,8 @@ def test_complex_expressions(self): sum_udf(col('v2')) + 5, plus_one(sum_udf(col('v1'))), sum_udf(plus_one(col('v2')))) - .sort('id') - .toPandas()) + .sort(['id', '(v % 2)']) + .toPandas().sort_index(by=['id', '(v % 2)'])) expected1 = (df.withColumn('v1', df.v + 1) .withColumn('v2', df.v + 2) @@ -365,8 +367,8 @@ def test_complex_expressions(self): sum(col('v2')) + 5, plus_one(sum(col('v1'))), sum(plus_one(col('v2')))) - .sort('id') - .toPandas()) + .sort(['id', '(v % 2)']) + .toPandas().sort_index(by=['id', '(v % 2)'])) # Test complex expressions with sql expression, scala pandas UDF and # group aggregate pandas UDF @@ -378,8 +380,8 @@ def test_complex_expressions(self): sum_udf(col('v2')) + 5, plus_two(sum_udf(col('v1'))), sum_udf(plus_two(col('v2')))) - .sort('id') - .toPandas()) + .sort(['id', '(v % 2)']) + .toPandas().sort_index(by=['id', '(v % 2)'])) expected2 = (df.withColumn('v1', df.v + 1) .withColumn('v2', df.v + 2) @@ -389,8 +391,8 @@ def test_complex_expressions(self): sum(col('v2')) + 5, plus_two(sum(col('v1'))), sum(plus_two(col('v2')))) - .sort('id') - .toPandas()) + .sort(['id', '(v % 2)']) + .toPandas().sort_index(by=['id', '(v % 2)'])) # Test sequential groupby aggregate result3 = (df.groupby('id') diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 7c6c6e108a3da..8e809b3556256 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -430,7 +430,8 @@ def registerJavaUDAF(self, name, javaClassName): >>> spark.udf.registerJavaUDAF("javaUDAF", "test.org.apache.spark.sql.MyDoubleAvg") >>> df = spark.createDataFrame([(1, "a"),(2, "b"), (3, "a")],["id", "name"]) >>> df.createOrReplaceTempView("df") - >>> spark.sql("SELECT name, javaUDAF(id) as avg from df group by name").collect() + >>> spark.sql("SELECT name, javaUDAF(id) as avg from df group by name order by name desc") \ + .collect() [Row(name=u'b', avg=102.0), Row(name=u'a', avg=102.0)] """ diff --git a/python/pyspark/sql/window.py b/python/pyspark/sql/window.py index 67c594c539d52..82f74346ba928 100644 --- a/python/pyspark/sql/window.py +++ b/python/pyspark/sql/window.py @@ -109,16 +109,16 @@ def rowsBetween(start, end): >>> tup = [(1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")] >>> df = sqlContext.createDataFrame(tup, ["id", "category"]) >>> window = Window.partitionBy("category").orderBy("id").rowsBetween(Window.currentRow, 1) - >>> df.withColumn("sum", func.sum("id").over(window)).show() + >>> df.withColumn("sum", func.sum("id").over(window)).sort("id", "category", "sum").show() +---+--------+---+ | id|category|sum| +---+--------+---+ - | 1| b| 3| - | 2| b| 5| - | 3| b| 3| | 1| a| 2| | 1| a| 3| + | 1| b| 3| | 2| a| 2| + | 2| b| 5| + | 3| b| 3| +---+--------+---+ :param start: boundary start, inclusive. @@ -168,16 +168,16 @@ def rangeBetween(start, end): >>> tup = [(1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")] >>> df = sqlContext.createDataFrame(tup, ["id", "category"]) >>> window = Window.partitionBy("category").orderBy("id").rangeBetween(Window.currentRow, 1) - >>> df.withColumn("sum", func.sum("id").over(window)).show() + >>> df.withColumn("sum", func.sum("id").over(window)).sort("id", "category").show() +---+--------+---+ | id|category|sum| +---+--------+---+ - | 1| b| 3| - | 2| b| 5| - | 3| b| 3| | 1| a| 4| | 1| a| 4| + | 1| b| 3| | 2| a| 2| + | 2| b| 5| + | 3| b| 3| +---+--------+---+ :param start: boundary start, inclusive. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 10dc74dd8a8ff..75e11abaa3161 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -80,12 +81,20 @@ class CacheManager extends Logging { } else { val sparkSession = query.sparkSession val qe = sparkSession.sessionState.executePlan(planToCache) - val inMemoryRelation = InMemoryRelation( - sparkSession.sessionState.conf.useCompression, - sparkSession.sessionState.conf.columnBatchSize, storageLevel, - qe.executedPlan, - tableName, - optimizedPlan = qe.optimizedPlan) + val originalValue = sparkSession.sessionState.conf.getConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED) + val inMemoryRelation = try { + // Avoiding changing the output partitioning, here disable AQE. + sparkSession.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, false) + InMemoryRelation( + sparkSession.sessionState.conf.useCompression, + sparkSession.sessionState.conf.columnBatchSize, storageLevel, + qe.executedPlan, + tableName, + optimizedPlan = qe.optimizedPlan) + } finally { + sparkSession.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, originalValue) + } + this.synchronized { if (lookupCachedData(planToCache).nonEmpty) { logWarning("Data has already been cached.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index f5591072f696f..7ff018989a2cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -176,7 +176,8 @@ case class AdaptiveSparkPlanExec( stage.resultOption = Some(res) case StageFailure(stage, ex) => errors.append( - new SparkException(s"Failed to materialize query stage: ${stage.treeString}", ex)) + new SparkException(s"Failed to materialize query stage: ${stage.treeString}." + + s" and the cause is ${ex.getMessage}", ex)) } // In case of errors, we cancel all running stages and throw exception. @@ -506,7 +507,8 @@ case class AdaptiveSparkPlanExec( } } finally { val ex = new SparkException( - "Adaptive execution failed due to stage materialization failures.", errors.head) + "Adaptive execution failed due to stage materialization failures." + + s" and the cause is ${errors.head.getMessage}", errors.head) errors.tail.foreach(ex.addSuppressed) cancelErrors.foreach(ex.addSuppressed) throw ex diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala index 0ec8710e4db43..61ae6cb14ccd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanHelper.scala @@ -60,7 +60,7 @@ trait AdaptiveSparkPlanHelper { * node in this tree in a preorder traversal. * @param f the function to be applied. */ - def map[A](p: SparkPlan)(f: SparkPlan => A): Seq[A] = { + def mapPlans[A](p: SparkPlan)(f: SparkPlan => A): Seq[A] = { val ret = new collection.mutable.ArrayBuffer[A]() foreach(p)(ret += f(_)) ret @@ -127,4 +127,12 @@ trait AdaptiveSparkPlanHelper { case s: QueryStageExec => Seq(s.plan) case _ => p.children } -} + + /** + * Strip the executePlan of AdaptiveSparkPlanExec leaf node. + */ + def stripAQEPlan(p: SparkPlan): SparkPlan = p match { + case a: AdaptiveSparkPlanExec => a.executedPlan + case other => other + } + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala index 09efcb712b5ae..e5642991c59a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/DemoteBroadcastHashJoin.scala @@ -29,7 +29,8 @@ import org.apache.spark.sql.internal.SQLConf case class DemoteBroadcastHashJoin(conf: SQLConf) extends Rule[LogicalPlan] { private def shouldDemote(plan: LogicalPlan): Boolean = plan match { - case LogicalQueryStage(_, stage: ShuffleQueryStageExec) if stage.resultOption.isDefined => + case LogicalQueryStage(_, stage: ShuffleQueryStageExec) if stage.resultOption.isDefined + && stage.resultOption.get != null => val mapOutputStatistics = stage.resultOption.get.asInstanceOf[MapOutputStatistics] val partitionCnt = mapOutputStatistics.bytesByPartitionId.length val nonZeroCnt = mapOutputStatistics.bytesByPartitionId.count(_ > 0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 8aefaf5af09bf..04696209ce10e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -20,12 +20,14 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, DynamicPruningSubquery, ListQuery, Literal} +import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, DynamicPruningSubquery, ListQuery, Literal, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ExecutedCommandExec +import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.internal.SQLConf /** @@ -39,11 +41,26 @@ case class InsertAdaptiveSparkPlan( private val conf = adaptiveExecutionContext.session.sessionState.conf + def containShuffle(plan: SparkPlan): Boolean = { + plan.find { + case _: Exchange => true + case s: SparkPlan => !s.requiredChildDistribution.forall(_ == UnspecifiedDistribution) + }.isDefined + } + + def containSubQuery(plan: SparkPlan): Boolean = { + plan.find(_.expressions.exists(_.find { + case _: SubqueryExpression => true + case _ => false + }.isDefined)).isDefined + } + override def apply(plan: SparkPlan): SparkPlan = applyInternal(plan, false) private def applyInternal(plan: SparkPlan, isSubquery: Boolean): SparkPlan = plan match { case _: ExecutedCommandExec => plan - case _ if conf.adaptiveExecutionEnabled && supportAdaptive(plan) => + case _ if conf.adaptiveExecutionEnabled && supportAdaptive(plan) + && (isSubquery || containShuffle(plan) || containSubQuery(plan)) => try { // Plan sub-queries recursively and pass in the shared stage cache for exchange reuse. Fall // back to non-adaptive mode if adaptive execution is supported in any of the sub-queries. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index 0659a89d2f808..e95441e28aafe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -136,9 +136,10 @@ object OptimizeLocalShuffleReader { } } - def canUseLocalShuffleReader(plan: SparkPlan): Boolean = { - plan.isInstanceOf[ShuffleQueryStageExec] || - plan.isInstanceOf[CoalescedShuffleReaderExec] + def canUseLocalShuffleReader(plan: SparkPlan): Boolean = plan match { + case s: ShuffleQueryStageExec => s.shuffle.canChangeNumPartitions + case CoalescedShuffleReaderExec(s: ShuffleQueryStageExec, _) => s.shuffle.canChangeNumPartitions + case _ => false } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain.sql b/sql/core/src/test/resources/sql-tests/inputs/explain.sql index 773c123992f71..d5253e3daddb0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/explain.sql @@ -1,3 +1,6 @@ +--SET spark.sql.codegen.wholeStage = true +--SET spark.sql.adaptive.enabled = false + -- Test tables CREATE table explain_temp1 (key int, val int) USING PARQUET; CREATE table explain_temp2 (key int, val int) USING PARQUET; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 85619beee0c99..cd2c681dd7e0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Join, JoinStrategyHint, SHUFFLE_HASH} import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.execution.{ExecSubqueryExpression, RDDScanExec, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.functions._ @@ -42,7 +43,9 @@ import org.apache.spark.util.{AccumulatorContext, Utils} private case class BigData(s: String) -class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSession { +class CachedTableSuite extends QueryTest with SQLTestUtils + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits._ setupTestData() @@ -96,7 +99,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi } private def getNumInMemoryTablesRecursively(plan: SparkPlan): Int = { - plan.collect { + collect(plan) { case inMemoryTable @ InMemoryTableScanExec(_, _, relation) => getNumInMemoryTablesRecursively(relation.cachedPlan) + getNumInMemoryTablesInSubquery(inMemoryTable) + 1 @@ -475,7 +478,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi */ private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = { assert( - df.queryExecution.executedPlan.collect { case e: ShuffleExchangeExec => e }.size == expected) + collect(df.queryExecution.executedPlan) { case e: ShuffleExchangeExec => e }.size == expected) } test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { @@ -526,7 +529,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") verifyNumExchanges(query, 1) - assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) checkAnswer( query, testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) @@ -543,7 +546,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") verifyNumExchanges(query, 1) - assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) checkAnswer( query, testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) @@ -559,7 +562,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") verifyNumExchanges(query, 1) - assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 12) + assert(stripAQEPlan(query.queryExecution.executedPlan). + outputPartitioning.numPartitions === 12) checkAnswer( query, testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) @@ -614,7 +618,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSessi val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a and t1.value = t2.b") verifyNumExchanges(query, 1) - assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) checkAnswer( query, df1.join(df2, $"key" === $"a" && $"value" === $"b").select($"key", $"value", $"a", $"b")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala index 0e090c6772d41..982681f18bd98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala @@ -51,7 +51,11 @@ class ConfigBehaviorSuite extends QueryTest with SharedSparkSession { dist) } - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> numPartitions.toString) { + // When enable AQE, the post partition number is changed. + // And the ChiSquareTest result is also need updated. So disable AQE. + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> numPartitions.toString, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The default chi-sq value should be low assert(computeChiSquareTest() < 10) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 49e259ff0242f..8ce87742a71ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -22,6 +22,7 @@ import scala.util.Random import org.scalatest.Matchers.the import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.expressions.Window @@ -34,7 +35,9 @@ import org.apache.spark.unsafe.types.CalendarInterval case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) -class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { +class DataFrameAggregateSuite extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits._ val absTol = 1e-8 @@ -612,7 +615,9 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { Seq((true, true), (true, false), (false, true), (false, false))) { withSQLConf( (SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, wholeStage.toString), - (SQLConf.USE_OBJECT_HASH_AGG.key, useObjectHashAgg.toString)) { + (SQLConf.USE_OBJECT_HASH_AGG.key, useObjectHashAgg.toString), + (SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false")) { + // When enable AQE, the WholeStageCodegenExec is added during QueryStageExec. val df = Seq(("1", 1), ("1", 2), ("2", 3), ("2", 4)).toDF("x", "y") @@ -678,17 +683,17 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { .groupBy("a").agg(collect_list("f").as("g")) val aggPlan = objHashAggDF.queryExecution.executedPlan - val sortAggPlans = aggPlan.collect { + val sortAggPlans = collect(aggPlan) { case sortAgg: SortAggregateExec => sortAgg } assert(sortAggPlans.isEmpty) - val objHashAggPlans = aggPlan.collect { + val objHashAggPlans = collect(aggPlan) { case objHashAgg: ObjectHashAggregateExec => objHashAgg } assert(objHashAggPlans.nonEmpty) - val exchangePlans = aggPlan.collect { + val exchangePlans = collect(aggPlan) { case shuffle: ShuffleExchangeExec => shuffle } assert(exchangePlans.length == 1) @@ -958,13 +963,13 @@ class DataFrameAggregateSuite extends QueryTest with SharedSparkSession { val df1 = Seq((1, "1 day"), (2, "2 day"), (3, "3 day"), (3, null)).toDF("a", "b") val df2 = df1.select(avg($"b" cast CalendarIntervalType)) checkAnswer(df2, Row(new CalendarInterval(0, 2, 0)) :: Nil) - assert(df2.queryExecution.executedPlan.find(_.isInstanceOf[HashAggregateExec]).isDefined) + assert(find(df2.queryExecution.executedPlan)(_.isInstanceOf[HashAggregateExec]).isDefined) val df3 = df1.groupBy($"a").agg(avg($"b" cast CalendarIntervalType)) checkAnswer(df3, Row(1, new CalendarInterval(0, 1, 0)) :: Row(2, new CalendarInterval(0, 2, 0)) :: Row(3, new CalendarInterval(0, 3, 0)) :: Nil) - assert(df3.queryExecution.executedPlan.find(_.isInstanceOf[HashAggregateExec]).isDefined) + assert(find(df3.queryExecution.executedPlan)(_.isInstanceOf[HashAggregateExec]).isDefined) } test("Dataset agg functions support calendar intervals") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index ddc06603e45aa..c7545bcad8962 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -19,13 +19,16 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan, Project} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -class DataFrameJoinSuite extends QueryTest with SharedSparkSession { +class DataFrameJoinSuite extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits._ test("join - join using") { @@ -150,7 +153,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSparkSession { spark.range(10e10.toLong) .join(spark.range(10e10.toLong).hint("broadcast"), "id") .queryExecution.executedPlan - assert(plan2.collect { case p: BroadcastHashJoinExec => p }.size == 1) + assert(collect(plan2) { case p: BroadcastHashJoinExec => p }.size == 1) } test("join - outer join conversion") { 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 8a9b923e284f3..763f92230cdc3 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 @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.Uuid import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} import org.apache.spark.sql.execution.{FilterExec, QueryExecution, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.functions._ @@ -45,7 +46,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom -class DataFrameSuite extends QueryTest with SharedSparkSession { +class DataFrameSuite extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits._ test("analysis error should be eagerly reported") { @@ -109,8 +112,10 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { test("Star Expansion - CreateStruct and CreateArray") { val structDf = testData2.select("a", "b").as("record") // CreateStruct and CreateArray in aggregateExpressions - assert(structDf.groupBy($"a").agg(min(struct($"record.*"))).first() == Row(3, Row(3, 1))) - assert(structDf.groupBy($"a").agg(min(array($"record.*"))).first() == Row(3, Seq(3, 1))) + assert(structDf.groupBy($"a").agg(min(struct($"record.*"))). + sort("a").first() == Row(1, Row(1, 1))) + assert(structDf.groupBy($"a").agg(min(array($"record.*"))). + sort("a").first() == Row(1, Seq(1, 1))) // CreateStruct and CreateArray in project list (unresolved alias) assert(structDf.select(struct($"record.*")).first() == Row(Row(1, 1))) @@ -1694,19 +1699,21 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { val plan = join.queryExecution.executedPlan checkAnswer(join, df) assert( - join.queryExecution.executedPlan.collect { case e: ShuffleExchangeExec => true }.size === 1) + collect(join.queryExecution.executedPlan) { + case e: ShuffleExchangeExec => true }.size === 1) assert( - join.queryExecution.executedPlan.collect { case e: ReusedExchangeExec => true }.size === 1) + collect(join.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size === 1) val broadcasted = broadcast(join) val join2 = join.join(broadcasted, "id").join(broadcasted, "id") checkAnswer(join2, df) assert( - join2.queryExecution.executedPlan.collect { case e: ShuffleExchangeExec => true }.size == 1) + collect(join2.queryExecution.executedPlan) { + case e: ShuffleExchangeExec => true }.size == 1) assert( - join2.queryExecution.executedPlan - .collect { case e: BroadcastExchangeExec => true }.size === 1) + collect(join2.queryExecution.executedPlan) { + case e: BroadcastExchangeExec => true }.size === 1) assert( - join2.queryExecution.executedPlan.collect { case e: ReusedExchangeExec => true }.size == 4) + collect(join2.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size == 4) } } @@ -2243,7 +2250,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { checkAnswer(df3.sort("value"), Row(7) :: Row(9) :: Nil) // Assert that no extra shuffle introduced by cogroup. - val exchanges = df3.queryExecution.executedPlan.collect { + val exchanges = collect(df3.queryExecution.executedPlan) { case h: ShuffleExchangeExec => h } assert(exchanges.size == 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 2e37879ea1658..d398657ec0b6e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.Matchers.the import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.optimizer.TransposeWindow +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ @@ -31,7 +32,9 @@ import org.apache.spark.sql.types._ /** * Window function testing for DataFrame API. */ -class DataFrameWindowFunctionsSuite extends QueryTest with SharedSparkSession { +class DataFrameWindowFunctionsSuite extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper{ import testImplicits._ @@ -716,7 +719,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSparkSession { .select($"sno", $"pno", $"qty", col("sum_qty_2"), sum("qty").over(w1).alias("sum_qty_1")) val expectedNumExchanges = if (transposeWindowEnabled) 1 else 2 - val actualNumExchanges = select.queryExecution.executedPlan.collect { + val actualNumExchanges = stripAQEPlan(select.queryExecution.executedPlan).collect { case e: Exchange => e }.length assert(actualNumExchanges == expectedNumExchanges) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index dc6df13514976..5c144dad23c30 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -20,13 +20,17 @@ package org.apache.spark.sql import org.scalatest.concurrent.TimeLimits import org.scalatest.time.SpanSugar._ +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.storage.StorageLevel -class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimits { +class DatasetCacheSuite extends QueryTest + with SharedSparkSession + with TimeLimits + with AdaptiveSparkPlanHelper { import testImplicits._ /** @@ -36,7 +40,8 @@ class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimit val plan = df.queryExecution.withCachedData assert(plan.isInstanceOf[InMemoryRelation]) val internalPlan = plan.asInstanceOf[InMemoryRelation].cacheBuilder.cachedPlan - assert(internalPlan.find(_.isInstanceOf[InMemoryTableScanExec]).size == numOfCachesDependedUpon) + assert(find(internalPlan)(_.isInstanceOf[InMemoryTableScanExec]).size + == numOfCachesDependedUpon) } test("get storage level") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index e963f40ffcec2..233d67898f909 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SQLExecution} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.expressions.UserDefinedFunction @@ -51,7 +52,9 @@ object TestForTypeAlias { def seqOfTupleTypeAlias: SeqOfTwoInt = Seq((1, 1), (2, 2)) } -class DatasetSuite extends QueryTest with SharedSparkSession { +class DatasetSuite extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits._ private implicit val ordering = Ordering.by((c: ClassData) => c.a -> c.b) @@ -1880,7 +1883,7 @@ class DatasetSuite extends QueryTest with SharedSparkSession { checkDataset(df3, DoubleData(1, "onetwo")) // Assert that no extra shuffle introduced by cogroup. - val exchanges = df3.queryExecution.executedPlan.collect { + val exchanges = collect(df3.queryExecution.executedPlan) { case h: ShuffleExchangeExec => h } assert(exchanges.size == 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index a54528f376d1b..e1f9bcc4e008d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.plans.ExistenceJoin import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec} import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} @@ -35,7 +36,8 @@ import org.apache.spark.sql.test.SharedSparkSession class DynamicPartitionPruningSuite extends QueryTest with SharedSparkSession - with GivenWhenThen { + with GivenWhenThen + with AdaptiveSparkPlanHelper { val tableFormat: String = "parquet" @@ -320,7 +322,7 @@ class DynamicPartitionPruningSuite def getFactScan(plan: SparkPlan): SparkPlan = { val scanOption = - plan.find { + find(plan) { case s: FileSourceScanExec => s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) case _ => false @@ -1204,7 +1206,9 @@ class DynamicPartitionPruningSuite test("join key with multiple references on the filtering plan") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // when enable AQE, the reusedExchange is inserted when executed. withTable("fact", "dim") { spark.range(100).select( $"id", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index f396f254168d2..d9f4d6d5132ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -26,6 +26,18 @@ import org.apache.spark.sql.types.StructType class ExplainSuite extends QueryTest with SharedSparkSession { import testImplicits._ + var originalValue: String = _ + protected override def beforeAll(): Unit = { + super.beforeAll() + originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + } + + protected override def afterAll(): Unit = { + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) + super.afterAll() + } + private def getNormalizedExplain(df: DataFrame, mode: ExplainMode): String = { val output = new java.io.ByteArrayOutputStream() Console.withOut(output) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index a66ba0958bc14..972683512db88 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.TestingUDT.{IntervalUDT, NullData, NullUDT} import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.FilePartition import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2ScanRelation, FileScan} import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable @@ -41,7 +42,9 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { +class FileBasedDataSourceSuite extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits._ override def beforeAll(): Unit = { @@ -716,21 +719,21 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession { val df2FromFile = spark.read.orc(workDirPath + "/data2") val joinedDF = df1FromFile.join(df2FromFile, Seq("count")) if (compressionFactor == 0.5) { - val bJoinExec = joinedDF.queryExecution.executedPlan.collect { + val bJoinExec = collect(joinedDF.queryExecution.executedPlan) { case bJoin: BroadcastHashJoinExec => bJoin } assert(bJoinExec.nonEmpty) - val smJoinExec = joinedDF.queryExecution.executedPlan.collect { + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { case smJoin: SortMergeJoinExec => smJoin } assert(smJoinExec.isEmpty) } else { // compressionFactor is 1.0 - val bJoinExec = joinedDF.queryExecution.executedPlan.collect { + val bJoinExec = collect(joinedDF.queryExecution.executedPlan) { case bJoin: BroadcastHashJoinExec => bJoin } assert(bJoinExec.isEmpty) - val smJoinExec = joinedDF.queryExecution.executedPlan.collect { + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { case smJoin: SortMergeJoinExec => smJoin } assert(smJoinExec.nonEmpty) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index e405864584d07..f766688f2a2da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -26,11 +26,12 @@ import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -class JoinHintSuite extends PlanTest with SharedSparkSession { +class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkPlanHelper { import testImplicits._ lazy val df = spark.range(10) @@ -352,7 +353,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { private def assertBroadcastHashJoin(df: DataFrame, buildSide: BuildSide): Unit = { val executedPlan = df.queryExecution.executedPlan - val broadcastHashJoins = executedPlan.collect { + val broadcastHashJoins = collect(executedPlan) { case b: BroadcastHashJoinExec => b } assert(broadcastHashJoins.size == 1) @@ -361,7 +362,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { private def assertBroadcastNLJoin(df: DataFrame, buildSide: BuildSide): Unit = { val executedPlan = df.queryExecution.executedPlan - val broadcastNLJoins = executedPlan.collect { + val broadcastNLJoins = collect(executedPlan) { case b: BroadcastNestedLoopJoinExec => b } assert(broadcastNLJoins.size == 1) @@ -370,7 +371,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { private def assertShuffleHashJoin(df: DataFrame, buildSide: BuildSide): Unit = { val executedPlan = df.queryExecution.executedPlan - val shuffleHashJoins = executedPlan.collect { + val shuffleHashJoins = collect(executedPlan) { case s: ShuffledHashJoinExec => s } assert(shuffleHashJoins.size == 1) @@ -379,7 +380,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { private def assertShuffleMergeJoin(df: DataFrame): Unit = { val executedPlan = df.queryExecution.executedPlan - val shuffleMergeJoins = executedPlan.collect { + val shuffleMergeJoins = collect(executedPlan) { case s: SortMergeJoinExec => s } assert(shuffleMergeJoins.size == 1) @@ -387,7 +388,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession { private def assertShuffleReplicateNLJoin(df: DataFrame): Unit = { val executedPlan = df.queryExecution.executedPlan - val shuffleReplicateNLJoins = executedPlan.collect { + val shuffleReplicateNLJoins = collect(executedPlan) { case c: CartesianProductExec => c } assert(shuffleReplicateNLJoins.size == 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 068ea05ead351..f45bd950040ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -30,13 +30,14 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrder} import org.apache.spark.sql.catalyst.plans.logical.Filter import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType -class JoinSuite extends QueryTest with SharedSparkSession { +class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { import testImplicits._ private def attachCleanupResourceChecker(plan: SparkPlan): Unit = { @@ -842,7 +843,7 @@ class JoinSuite extends QueryTest with SharedSparkSession { case j: SortMergeJoinExec => j } val executed = df.queryExecution.executedPlan - val executedJoins = executed.collect { + val executedJoins = collect(executed) { case j: SortMergeJoinExec => j } // This only applies to the above tested queries, in which a child SortMergeJoin always @@ -1026,12 +1027,12 @@ class JoinSuite extends QueryTest with SharedSparkSession { val right = Seq((1, 2), (3, 4)).toDF("c", "d") val df = left.join(right, pythonTestUDF(left("a")) === pythonTestUDF(right.col("c"))) - val joinNode = df.queryExecution.executedPlan.find(_.isInstanceOf[BroadcastHashJoinExec]) + val joinNode = find(df.queryExecution.executedPlan)(_.isInstanceOf[BroadcastHashJoinExec]) assert(joinNode.isDefined) // There are two PythonUDFs which use attribute from left and right of join, individually. // So two PythonUDFs should be evaluated before the join operator, at left and right side. - val pythonEvals = joinNode.get.collect { + val pythonEvals = collect(joinNode.get) { case p: BatchEvalPythonExec => p } assert(pythonEvals.size == 2) @@ -1055,7 +1056,7 @@ class JoinSuite extends QueryTest with SharedSparkSession { assert(filterInAnalysis.isDefined) // Filter predicate was pushdown as join condition. So there is no Filter exec operator. - val filterExec = df.queryExecution.executedPlan.find(_.isInstanceOf[FilterExec]) + val filterExec = find(df.queryExecution.executedPlan)(_.isInstanceOf[FilterExec]) assert(filterExec.isEmpty) checkAnswer(df, Row(1, 2, 1, 2) :: Nil) 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 ce46296dabc5a..e472ceac7c1a6 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 @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.FunctionsCommand @@ -44,7 +45,7 @@ import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval -class SQLQuerySuite extends QueryTest with SharedSparkSession { +class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { import testImplicits._ setupTestData() @@ -191,7 +192,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { val actual = unindentAndTrim( hiveResultString(df.queryExecution.executedPlan).mkString("\n")) val expected = unindentAndTrim(output) - assert(actual === expected) + assert(actual.sorted === expected.sorted) case _ => }) } @@ -3278,7 +3279,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { |on leftside.a = rightside.a """.stripMargin) - val inMemoryTableScan = queryDf.queryExecution.executedPlan.collect { + val inMemoryTableScan = collect(queryDf.queryExecution.executedPlan) { case i: InMemoryTableScanExec => i } assert(inMemoryTableScan.size == 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 9169b3819f0a4..03d0aa999f5d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -419,9 +419,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { s"Schema did not match for query #$i\n${expected.sql}: $output") { output.schema } - assertResult(expected.output, s"Result did not match for query #$i\n${expected.sql}") { - output.output - } + assertResult(expected.output.sorted, s"Result did not match" + + s" for query #$i\n${expected.sql}") { output.output.sorted } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 2a4c15233fe39..8c204bc47c6c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.COLUMN_BATCH_SIZE import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS import org.apache.spark.sql.types.{DataType, Decimal, IntegerType, LongType, Metadata, StructType} @@ -150,6 +151,8 @@ class SparkSessionExtensionSuite extends SparkFunSuite { MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) } withSession(extensions) { session => + // The ApplyColumnarRulesAndInsertTransitions rule is not applied when enable AQE + session.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, false) assert(session.sessionState.columnarRules.contains( MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) import session.sqlContext.implicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 2f0142f3a6c2d..ff8f94c68c5ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -22,11 +22,12 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort} import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, FileSourceScanExec, InputAdapter, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.FileScanRDD import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -class SubquerySuite extends QueryTest with SharedSparkSession { +class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { import testImplicits._ setupTestData() @@ -1293,7 +1294,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession { sql("create temporary view t1(a int) using parquet") sql("create temporary view t2(b int) using parquet") val plan = sql("select * from t2 where b > (select max(a) from t1)") - val subqueries = plan.queryExecution.executedPlan.collect { + val subqueries = stripAQEPlan(plan.queryExecution.executedPlan).collect { case p => p.subqueries }.flatten assert(subqueries.length == 1) @@ -1308,7 +1309,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession { val df = sql("SELECT * FROM a WHERE p <= (SELECT MIN(id) FROM b)") checkAnswer(df, Seq(Row(0, 0), Row(2, 0))) // need to execute the query before we can examine fs.inputRDDs() - assert(df.queryExecution.executedPlan match { + assert(stripAQEPlan(df.queryExecution.executedPlan) match { case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _)))) => partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && @@ -1358,7 +1359,9 @@ class SubquerySuite extends QueryTest with SharedSparkSession { test("SPARK-27279: Reuse Subquery") { Seq(true, false).foreach { reuse => - withSQLConf(SQLConf.SUBQUERY_REUSE_ENABLED.key -> reuse.toString) { + withSQLConf(SQLConf.SUBQUERY_REUSE_ENABLED.key -> reuse.toString, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // when enable AQE, the reusedExchange is inserted when executed. val df = sql( """ |SELECT (SELECT avg(key) FROM testData) + (SELECT avg(key) FROM testData) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 55c71c7d02d2b..85ff86ef3fc5b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapabil import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, DataSourceV2ScanRelation} import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector @@ -42,7 +43,7 @@ import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.vectorized.ColumnarBatch -class DataSourceV2Suite extends QueryTest with SharedSparkSession { +class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { import testImplicits._ private def getBatch(query: DataFrame): AdvancedBatch = { @@ -164,25 +165,25 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { val groupByColA = df.groupBy('i).agg(sum('j)) checkAnswer(groupByColA, Seq(Row(1, 8), Row(2, 6), Row(3, 6), Row(4, 4))) - assert(groupByColA.queryExecution.executedPlan.collectFirst { + assert(collectFirst(groupByColA.queryExecution.executedPlan) { case e: ShuffleExchangeExec => e }.isEmpty) val groupByColAB = df.groupBy('i, 'j).agg(count("*")) checkAnswer(groupByColAB, Seq(Row(1, 4, 2), Row(2, 6, 1), Row(3, 6, 1), Row(4, 2, 2))) - assert(groupByColAB.queryExecution.executedPlan.collectFirst { + assert(collectFirst(groupByColAB.queryExecution.executedPlan) { case e: ShuffleExchangeExec => e }.isEmpty) val groupByColB = df.groupBy('j).agg(sum('i)) checkAnswer(groupByColB, Seq(Row(2, 8), Row(4, 2), Row(6, 5))) - assert(groupByColB.queryExecution.executedPlan.collectFirst { + assert(collectFirst(groupByColB.queryExecution.executedPlan) { case e: ShuffleExchangeExec => e }.isDefined) val groupByAPlusB = df.groupBy('i + 'j).agg(count("*")) checkAnswer(groupByAPlusB, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1))) - assert(groupByAPlusB.queryExecution.executedPlan.collectFirst { + assert(collectFirst(groupByAPlusB.queryExecution.executedPlan) { case e: ShuffleExchangeExec => e }.isDefined) } @@ -385,7 +386,7 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession { val t2 = spark.read.format(classOf[SimpleDataSourceV2].getName).load() Seq(2, 3).toDF("a").createTempView("t1") val df = t2.where("i < (select max(a) from t1)").select('i) - val subqueries = df.queryExecution.executedPlan.collect { + val subqueries = stripAQEPlan(df.queryExecution.executedPlan).collect { case p => p.subqueries }.flatten assert(subqueries.length == 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala index 43e29c2d50786..7d6306b65ff47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala @@ -21,13 +21,16 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} import org.apache.spark.SparkException import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec import org.apache.spark.sql.execution.joins.HashedRelation import org.apache.spark.sql.functions.broadcast import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -class BroadcastExchangeSuite extends SparkPlanTest with SharedSparkSession { +class BroadcastExchangeSuite extends SparkPlanTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits._ @@ -53,8 +56,8 @@ class BroadcastExchangeSuite extends SparkPlanTest with SharedSparkSession { }).where("id = value") // get the exchange physical plan - val hashExchange = df.queryExecution.executedPlan - .collect { case p: BroadcastExchangeExec => p }.head + val hashExchange = collect( + df.queryExecution.executedPlan) { case p: BroadcastExchangeExec => p }.head // materialize the future and wait for the job being scheduled hashExchange.prepare() @@ -84,8 +87,8 @@ class BroadcastExchangeSuite extends SparkPlanTest with SharedSparkSession { withSQLConf(SQLConf.BROADCAST_TIMEOUT.key -> "-1") { val df = spark.range(1).toDF() val joinDF = df.join(broadcast(df), "id") - val broadcastExchangeExec = joinDF.queryExecution.executedPlan - .collect { case p: BroadcastExchangeExec => p } + val broadcastExchangeExec = collect( + joinDF.queryExecution.executedPlan) { case p: BroadcastExchangeExec => p } assert(broadcastExchangeExec.size == 1, "one and only BroadcastExchangeExec") assert(joinDF.collect().length == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala index c198978f5888d..1e90754ad7721 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala @@ -18,23 +18,30 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.expressions.scalalang.typed +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @deprecated("This test suite will be removed.", "3.0.0") -class DeprecatedWholeStageCodegenSuite extends QueryTest with SharedSparkSession { +class DeprecatedWholeStageCodegenSuite extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { test("simple typed UDAF should be included in WholeStageCodegen") { - import testImplicits._ + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // With enable AQE, the WholeStageCodegenExec rule is applied when running QueryStageExec. + import testImplicits._ - val ds = Seq(("a", 10), ("b", 1), ("b", 2), ("c", 1)).toDS() - .groupByKey(_._1).agg(typed.sum(_._2)) + val ds = Seq(("a", 10), ("b", 1), ("b", 2), ("c", 1)).toDS() + .groupByKey(_._1).agg(typed.sum(_._2)) - val plan = ds.queryExecution.executedPlan - assert(plan.find(p => - p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) - assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) + val plan = ds.queryExecution.executedPlan + assert(find(plan)(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) + assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala index 44af422b90837..311f84c07a955 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala @@ -29,9 +29,23 @@ import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.execution.window.WindowExec +import org.apache.spark.sql.internal.SQLConf class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite { + var originalValue: String = _ + // when enable AQE, the 'AdaptiveSparkPlanExec' node does not have a logical plan link + override def beforeAll(): Unit = { + super.beforeAll() + originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + } + + override def afterAll(): Unit = { + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) + super.afterAll() + } + override protected def checkGeneratedCode( plan: SparkPlan, checkMethodCodeSize: Boolean = true): Unit = { super.checkGeneratedCode(plan, checkMethodCodeSize) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 017e548809413..563c42901ecaa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range, Repartition, Sort, Union} import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ReuseExchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} @@ -32,7 +33,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class PlannerSuite extends SharedSparkSession { +class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { import testImplicits._ setupTestData() @@ -254,29 +255,31 @@ class PlannerSuite extends SharedSparkSession { // Disable broadcast join withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { { - val numExchanges = sql( + val plan = sql( """ |SELECT * |FROM | normal JOIN small ON (normal.key = small.key) | JOIN tiny ON (small.key = tiny.key) """.stripMargin - ).queryExecution.executedPlan.collect { + ).queryExecution.executedPlan + val numExchanges = collect(plan) { case exchange: ShuffleExchangeExec => exchange }.length assert(numExchanges === 5) } { - // This second query joins on different keys: - val numExchanges = sql( + val plan = sql( """ |SELECT * |FROM | normal JOIN small ON (normal.key = small.key) | JOIN tiny ON (normal.key = tiny.key) """.stripMargin - ).queryExecution.executedPlan.collect { + ).queryExecution.executedPlan + // This second query joins on different keys: + val numExchanges = collect(plan) { case exchange: ShuffleExchangeExec => exchange }.length assert(numExchanges === 5) @@ -438,10 +441,12 @@ class PlannerSuite extends SharedSparkSession { case _ => false }.isEmpty, "RoundRobinPartitioning should be changed to RangePartitioning") - - val query = testData.select('key, 'value).repartition(2).sort('key.asc) - assert(query.rdd.getNumPartitions == 2) - assert(query.rdd.collectPartitions()(0).map(_.get(0)).toSeq == (1 to 50)) + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // when enable AQE, the post partiiton number is changed. + val query = testData.select('key, 'value).repartition(2).sort('key.asc) + assert(query.rdd.getNumPartitions == 2) + assert(query.rdd.collectPartitions()(0).map(_.get(0)).toSeq == (1 to 50)) + } } test("SPARK-30036: Remove unnecessary HashPartitioning " + @@ -461,10 +466,12 @@ class PlannerSuite extends SharedSparkSession { case _ => false }.isEmpty, "HashPartitioning should be changed to RangePartitioning") - - val query = testData.select('key, 'value).repartition(5, 'key).sort('key.asc) - assert(query.rdd.getNumPartitions == 5) - assert(query.rdd.collectPartitions()(0).map(_.get(0)).toSeq == (1 to 20)) + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // when enable AQE, the post partiiton number is changed. + val query = testData.select('key, 'value).repartition(5, 'key).sort('key.asc) + assert(query.rdd.getNumPartitions == 5) + assert(query.rdd.collectPartitions()(0).map(_.get(0)).toSeq == (1 to 20)) + } } test("EnsureRequirements does not eliminate Exchange with different partitioning") { @@ -808,7 +815,7 @@ class PlannerSuite extends SharedSparkSession { def checkReusedExchangeOutputPartitioningRewrite( df: DataFrame, expectedPartitioningClass: Class[_]): Unit = { - val reusedExchange = df.queryExecution.executedPlan.collect { + val reusedExchange = collect(df.queryExecution.executedPlan) { case r: ReusedExchangeExec => r } checkOutputPartitioningRewrite(reusedExchange, expectedPartitioningClass) @@ -817,31 +824,34 @@ class PlannerSuite extends SharedSparkSession { def checkInMemoryTableScanOutputPartitioningRewrite( df: DataFrame, expectedPartitioningClass: Class[_]): Unit = { - val inMemoryScan = df.queryExecution.executedPlan.collect { + val inMemoryScan = collect(df.queryExecution.executedPlan) { case m: InMemoryTableScanExec => m } checkOutputPartitioningRewrite(inMemoryScan, expectedPartitioningClass) } + // when enable AQE, the reusedExchange is inserted when executed. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // ReusedExchange is HashPartitioning + val df1 = Seq(1 -> "a").toDF("i", "j").repartition($"i") + val df2 = Seq(1 -> "a").toDF("i", "j").repartition($"i") + checkReusedExchangeOutputPartitioningRewrite(df1.union(df2), classOf[HashPartitioning]) + + // ReusedExchange is RangePartitioning + val df3 = Seq(1 -> "a").toDF("i", "j").orderBy($"i") + val df4 = Seq(1 -> "a").toDF("i", "j").orderBy($"i") + checkReusedExchangeOutputPartitioningRewrite(df3.union(df4), classOf[RangePartitioning]) + + // InMemoryTableScan is HashPartitioning + Seq(1 -> "a").toDF("i", "j").repartition($"i").persist() + checkInMemoryTableScanOutputPartitioningRewrite( + Seq(1 -> "a").toDF("i", "j").repartition($"i"), classOf[HashPartitioning]) - // ReusedExchange is HashPartitioning - val df1 = Seq(1 -> "a").toDF("i", "j").repartition($"i") - val df2 = Seq(1 -> "a").toDF("i", "j").repartition($"i") - checkReusedExchangeOutputPartitioningRewrite(df1.union(df2), classOf[HashPartitioning]) - - // ReusedExchange is RangePartitioning - val df3 = Seq(1 -> "a").toDF("i", "j").orderBy($"i") - val df4 = Seq(1 -> "a").toDF("i", "j").orderBy($"i") - checkReusedExchangeOutputPartitioningRewrite(df3.union(df4), classOf[RangePartitioning]) - - // InMemoryTableScan is HashPartitioning - Seq(1 -> "a").toDF("i", "j").repartition($"i").persist() - checkInMemoryTableScanOutputPartitioningRewrite( - Seq(1 -> "a").toDF("i", "j").repartition($"i"), classOf[HashPartitioning]) - - // InMemoryTableScan is RangePartitioning - spark.range(1, 100, 1, 10).toDF().persist() - checkInMemoryTableScanOutputPartitioningRewrite( - spark.range(1, 100, 1, 10).toDF(), classOf[RangePartitioning]) + // InMemoryTableScan is RangePartitioning + spark.range(1, 100, 1, 10).toDF().persist() + checkInMemoryTableScanOutputPartitioningRewrite( + spark.range(1, 100, 1, 10).toDF(), classOf[RangePartitioning]) + } // InMemoryTableScan is PartitioningCollection withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { @@ -913,7 +923,7 @@ class PlannerSuite extends SharedSparkSession { |FROM range(5) |""".stripMargin) - val Seq(subquery) = df.queryExecution.executedPlan.subqueriesAll + val Seq(subquery) = stripAQEPlan(df.queryExecution.executedPlan).subqueriesAll subquery.foreach { node => node.expressions.foreach { expression => expression.foreach { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala index f985386eee292..f6814d8ff8a3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala @@ -48,6 +48,7 @@ class WholeStageCodegenSparkSubmitSuite extends SparkFunSuite "--conf", "spark.master.rest.enabled=false", "--conf", "spark.driver.extraJavaOptions=-XX:-UseCompressedOops", "--conf", "spark.executor.extraJavaOptions=-XX:+UseCompressedOops", + "--conf", "spark.sql.adaptive.enabled=false", unusedJar.toString) SparkSubmitSuite.runSparkSubmit(argsForSparkSubmit, "../..") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 572932fc2750b..06a016fac5300 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -32,6 +32,19 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession { import testImplicits._ + var originalValue: String = _ + // With on AQE, the WholeStageCodegenExec is added when running QueryStageExec. + override def beforeAll(): Unit = { + super.beforeAll() + originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + } + + override def afterAll(): Unit = { + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) + super.afterAll() + } + test("range/filter should be combined") { val df = spark.range(10).filter("id = 1").selectExpr("id + 1") val plan = df.queryExecution.executedPlan diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 328a10704109c..a3d4905e82cee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.SchemaPruningTest import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -34,7 +35,8 @@ abstract class SchemaPruningSuite extends QueryTest with FileBasedDataSourceTest with SchemaPruningTest - with SharedSparkSession { + with SharedSparkSession + with AdaptiveSparkPlanHelper { case class FullName(first: String, middle: String, last: String) case class Company(name: String, address: String) case class Employer(id: Int, company: Company) @@ -468,7 +470,7 @@ abstract class SchemaPruningSuite protected def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { val fileSourceScanSchemata = - df.queryExecution.executedPlan.collect { + collect(df.queryExecution.executedPlan) { case scan: FileSourceScanExec => scan.requiredSchema } assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala index 80cfbd6a02676..6c9bd32913178 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala @@ -19,12 +19,13 @@ package org.apache.spark.sql.execution.datasources.orc import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.SchemaPruningSuite import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf -class OrcV2SchemaPruningSuite extends SchemaPruningSuite { +class OrcV2SchemaPruningSuite extends SchemaPruningSuite with AdaptiveSparkPlanHelper { override protected val dataSourceName: String = "orc" override protected val vectorizedReaderEnabledKey: String = SQLConf.ORC_VECTORIZED_READER_ENABLED.key @@ -36,7 +37,7 @@ class OrcV2SchemaPruningSuite extends SchemaPruningSuite { override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { val fileSourceScanSchemata = - df.queryExecution.executedPlan.collect { + collect(df.queryExecution.executedPlan) { case BatchScanExec(_, scan: OrcScan) => scan.readDataSchema } assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 309507d4ddd84..c64e95078e916 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.spark.SparkConf import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.SchemaPruningSuite import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf -abstract class ParquetSchemaPruningSuite extends SchemaPruningSuite { +abstract class ParquetSchemaPruningSuite extends SchemaPruningSuite with AdaptiveSparkPlanHelper { override protected val dataSourceName: String = "parquet" override protected val vectorizedReaderEnabledKey: String = SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key @@ -48,7 +49,7 @@ class ParquetV2SchemaPruningSuite extends ParquetSchemaPruningSuite { override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { val fileSourceScanSchemata = - df.queryExecution.executedPlan.collect { + collect(df.queryExecution.executedPlan) { case scan: BatchScanExec => scan.scan.asInstanceOf[ParquetScan].readDataSchema } assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index 9a48c1ea0f318..3c187a2ed0ff4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -25,12 +25,27 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.execution.{CodegenSupport, LeafExecNode, WholeStageCodegenExec} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.TestData import org.apache.spark.sql.types.StructType class DebuggingSuite extends SharedSparkSession { + + var originalValue: String = _ + // With on AQE, the WholeStageCodegenExec is added when running QueryStageExec. + override def beforeAll(): Unit = { + super.beforeAll() + originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + } + + override def afterAll(): Unit = { + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) + super.afterAll() + } + test("DataFrame.debug()") { testData.debug() } 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 91cb919479bfa..5ce758e1e4eb8 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 @@ -24,6 +24,7 @@ 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.catalyst.plans.logical.BROADCAST import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.functions._ @@ -38,7 +39,7 @@ import org.apache.spark.sql.types.{LongType, ShortType} * unsafe map in [[org.apache.spark.sql.execution.joins.UnsafeHashedRelation]] is not triggered * without serializing the hashed relation, which does not happen in local mode. */ -class BroadcastJoinSuite extends QueryTest with SQLTestUtils { +class BroadcastJoinSuite extends QueryTest with SQLTestUtils with AdaptiveSparkPlanHelper { import testImplicits._ protected var spark: SparkSession = null @@ -122,7 +123,7 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { val df2 = Seq((1, "1"), (2, "2")).toDF("key", "value") df2.cache() val df3 = df1.join(broadcast(df2), Seq("key"), "inner") - val numBroadCastHashJoin = df3.queryExecution.executedPlan.collect { + val numBroadCastHashJoin = collect(df3.queryExecution.executedPlan) { case b: BroadcastHashJoinExec => b }.size assert(numBroadCastHashJoin === 1) @@ -140,13 +141,13 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { broadcast(df2).cache() val df3 = df1.join(df2, Seq("key"), "inner") - val numCachedPlan = df3.queryExecution.executedPlan.collect { + val numCachedPlan = collect(df3.queryExecution.executedPlan) { case i: InMemoryTableScanExec => i }.size // df2 should be cached. assert(numCachedPlan === 1) - val numBroadCastHashJoin = df3.queryExecution.executedPlan.collect { + val numBroadCastHashJoin = collect(df3.queryExecution.executedPlan) { case b: BroadcastHashJoinExec => b }.size // df2 should not be broadcasted. @@ -272,7 +273,6 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { } test("Shouldn't change broadcast join buildSide if user clearly specified") { - withTempView("t1", "t2") { Seq((1, "4"), (2, "2")).toDF("key", "value").createTempView("t1") Seq((1, "1"), (2, "12.3"), (2, "123")).toDF("key", "value").createTempView("t2") @@ -378,7 +378,7 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { private val bl = BroadcastNestedLoopJoinExec.toString private def assertJoinBuildSide(sqlStr: String, joinMethod: String, buildSide: BuildSide): Any = { - val executedPlan = sql(sqlStr).queryExecution.executedPlan + val executedPlan = stripAQEPlan(sql(sqlStr).queryExecution.executedPlan) executedPlan match { case b: BroadcastNestedLoopJoinExec => assert(b.getClass.getSimpleName === joinMethod) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 206bd78c01a87..7d09577075d5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -36,6 +36,19 @@ import org.apache.spark.util.{AccumulatorContext, JsonProtocol} class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils { import testImplicits._ + var originalValue: String = _ + // With AQE on/off, the metric info is different. + override def beforeAll(): Unit = { + super.beforeAll() + originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + } + + override def afterAll(): Unit = { + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) + super.afterAll() + } + /** * Generates a `DataFrame` by filling randomly generated bytes for hash collision. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala index d26989b00a651..5fe3d6a71167e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala @@ -24,10 +24,13 @@ import org.apache.spark.api.python.{PythonEvalType, PythonFunction} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, In} import org.apache.spark.sql.execution.{FilterExec, InputAdapter, SparkPlanTest, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{BooleanType, DoubleType} -class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSparkSession { +class BatchEvalPythonExecSuite extends SparkPlanTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits.newProductEncoder import testImplicits.localSeqToDatasetHolder @@ -95,7 +98,7 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSparkSession { val df = Seq(("Hello", 4)).toDF("a", "b") val df2 = Seq(("Hello", 4)).toDF("c", "d") val joinDF = df.crossJoin(df2).where("dummyPythonUDF(a, c) == dummyPythonUDF(d, c)") - val qualifiedPlanNodes = joinDF.queryExecution.executedPlan.collect { + val qualifiedPlanNodes = collect(joinDF.queryExecution.executedPlan) { case b: BatchEvalPythonExec => b } assert(qualifiedPlanNodes.size == 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 4113c2c5d296d..55b551d0af078 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.functions.count +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.UI_RETAINED_EXECUTIONS import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.status.ElementTrackingStore @@ -620,9 +621,12 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils } test("SPARK-29894 test Codegen Stage Id in SparkPlanInfo") { - val df = createTestDataFrame.select(count("*")) - val sparkPlanInfo = SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan) - assert(sparkPlanInfo.nodeName === "WholeStageCodegen (2)") + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // with AQE on, the WholeStageCodegen rule is applied when running QueryStageExec. + val df = createTestDataFrame.select(count("*")) + val sparkPlanInfo = SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan) + assert(sparkPlanInfo.nodeName === "WholeStageCodegen (2)") + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index 42213b9a81882..776cdb107084d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -98,7 +98,9 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { test("SPARK-22219: refactor to control to generate comment") { Seq(true, false).foreach { flag => - withSQLConf(StaticSQLConf.CODEGEN_COMMENTS.key -> flag.toString) { + withSQLConf(StaticSQLConf.CODEGEN_COMMENTS.key -> flag.toString, + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // with AQE on, the WholeStageCodegen rule is applied when running QueryStageExec. val res = codegenStringSeq(spark.range(10).groupBy(col("id") * 2).count() .queryExecution.executedPlan) assert(res.length == 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index f4ab232af28b5..6881812286b24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -24,11 +24,15 @@ import org.apache.spark.sql.{functions, AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoStatement, LogicalPlan, Project} import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { +class DataFrameCallbackSuite extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { import testImplicits._ import functions._ @@ -95,7 +99,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSparkSession { override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { - val metric = qe.executedPlan match { + val metric = stripAQEPlan(qe.executedPlan) match { case w: WholeStageCodegenExec => w.child.longMetric("numOutputRows") case other => other.longMetric("numOutputRows") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 921b46edc0a20..f9a4e2cd210e3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -134,19 +134,21 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("explain output of physical plan should contain proper codegen stage ID") { - checkKeywordsExist(sql( - """ - |EXPLAIN SELECT t1.id AS a, t2.id AS b FROM - |(SELECT * FROM range(3)) t1 JOIN - |(SELECT * FROM range(10)) t2 ON t1.id == t2.id % 3 - """.stripMargin), - "== Physical Plan ==", - "*(2) Project ", - "+- *(2) BroadcastHashJoin ", - " :- BroadcastExchange ", - " : +- *(1) Range ", - " +- *(2) Range " - ) + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + checkKeywordsExist(sql( + """ + |EXPLAIN SELECT t1.id AS a, t2.id AS b FROM + |(SELECT * FROM range(3)) t1 JOIN + |(SELECT * FROM range(10)) t2 ON t1.id == t2.id % 3 + """.stripMargin), + "== Physical Plan ==", + "*(2) Project ", + "+- *(2) BroadcastHashJoin ", + " :- BroadcastExchange ", + " : +- *(1) Range ", + " +- *(2) Range " + ) + } } test("EXPLAIN CODEGEN command") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala index b0d615c1acee9..9e33a8ee4cc5c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -29,12 +29,14 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo import test.org.apache.spark.sql.MyDoubleAvg import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils -class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { +class HiveUDAFSuite extends QueryTest + with TestHiveSingleton with SQLTestUtils with AdaptiveSparkPlanHelper { import testImplicits._ protected override def beforeAll(): Unit = { @@ -63,7 +65,7 @@ class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("built-in Hive UDAF") { val df = sql("SELECT key % 2, hive_max(key) FROM t GROUP BY key % 2") - val aggs = df.queryExecution.executedPlan.collect { + val aggs = collect(df.queryExecution.executedPlan) { case agg: ObjectHashAggregateExec => agg } @@ -80,7 +82,7 @@ class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("customized Hive UDAF") { val df = sql("SELECT key % 2, mock(value) FROM t GROUP BY key % 2") - val aggs = df.queryExecution.executedPlan.collect { + val aggs = collect(df.queryExecution.executedPlan) { case agg: ObjectHashAggregateExec => agg } @@ -99,7 +101,7 @@ class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { spark.range(100).createTempView("v") val df = sql("SELECT id % 2, mock2(id) FROM v GROUP BY id % 2") - val aggs = df.queryExecution.executedPlan.collect { + val aggs = collect(df.queryExecution.executedPlan) { case agg: ObjectHashAggregateExec => agg } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala index 930f801467497..327e4104d59a8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedFunction import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -38,7 +39,8 @@ class ObjectHashAggregateSuite extends QueryTest with SQLTestUtils with TestHiveSingleton - with ExpressionEvalHelper { + with ExpressionEvalHelper + with AdaptiveSparkPlanHelper { import testImplicits._ @@ -394,19 +396,19 @@ class ObjectHashAggregateSuite } private def containsSortAggregateExec(df: DataFrame): Boolean = { - df.queryExecution.executedPlan.collectFirst { + collectFirst(df.queryExecution.executedPlan) { case _: SortAggregateExec => () }.nonEmpty } private def containsObjectHashAggregateExec(df: DataFrame): Boolean = { - df.queryExecution.executedPlan.collectFirst { + collectFirst(df.queryExecution.executedPlan) { case _: ObjectHashAggregateExec => () }.nonEmpty } private def containsHashAggregateExec(df: DataFrame): Boolean = { - df.queryExecution.executedPlan.collectFirst { + collectFirst(df.queryExecution.executedPlan) { case _: HashAggregateExec => () }.nonEmpty } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index 022cb7177339d..16668f93bd4e7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -19,9 +19,23 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.execution.metric.SQLMetricsTestUtils import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf class SQLMetricsSuite extends SQLMetricsTestUtils with TestHiveSingleton { + var originalValue: String = _ + // With AQE on/off, the metric info is different. + override def beforeAll(): Unit = { + super.beforeAll() + originalValue = spark.conf.get(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key) + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + } + + override def afterAll(): Unit = { + spark.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, originalValue) + super.afterAll() + } + test("writing data out metrics: hive") { testMetricsNonDynamicPartition("hive", "t1") } From 81e1a2188a093255f466666f04a2492d357a670e Mon Sep 17 00:00:00 2001 From: iRakson Date: Tue, 14 Jan 2020 09:31:09 +0900 Subject: [PATCH 1094/1387] [SPARK-30234][SQL][DOCS][FOLOWUP] Update Documentation for ADD FILE and LIST FILE ### What changes were proposed in this pull request? Updated the doc for ADD FILE and LIST FILE ### Why are the changes needed? Due to changes made in #26863 , it is necessary to update ADD FILE and LIST FILE doc. ### Does this PR introduce any user-facing change? Yeah. Document updated. ### How was this patch tested? Manually Closes #27188 from iRakson/SPARK-30234_FOLLOWUP. Authored-by: iRakson Signed-off-by: Takeshi Yamamuro --- docs/sql-ref-syntax-aux-resource-mgmt-add-file.md | 9 +++++---- docs/sql-ref-syntax-aux-resource-mgmt-list-file.md | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md index 409834afd8f54..7e485cbafe709 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-file.md @@ -20,17 +20,17 @@ license: | --- ### Description -`ADD FILE` adds a file to the list of resources. The added file can be listed using [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html). +`ADD FILE` can be used to add a single file as well as a directory to the list of resources. The added resource can be listed using [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html). ### Syntax {% highlight sql %} -ADD FILE file_name +ADD FILE resource_name {% endhighlight %} ### Parameters
      -
      file_name
      -
      The name of the file to be added.
      +
      resource_name
      +
      The name of the file or directory to be added.
      ### Examples @@ -39,6 +39,7 @@ ADD FILE /tmp/test; ADD FILE "/path/to/file/abc.txt"; ADD FILE '/another/test.txt'; ADD FILE "/path with space/abc.txt"; +ADD FILE "/path/to/some/directory"; {% endhighlight %} ### Related Statements diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md index 916e092403f2b..c42bf7ae8dd41 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md @@ -20,7 +20,7 @@ license: | --- ### Description -`LIST FILE` lists the files added by [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html). +`LIST FILE` lists the resources added by [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html). ### Syntax {% highlight sql %} From 6646b3e13e46b220a33b5798ef266d8a14f3c85b Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 14 Jan 2020 10:40:35 +0900 Subject: [PATCH 1095/1387] Revert "[SPARK-28670][SQL] create function should thrown Exception if the resource is not found" This reverts commit 16e5e79877d9bad73f3f96688efe08c9a052340f. --- docs/sql-migration-guide.md | 2 -- .../spark/sql/execution/command/functions.scala | 13 +------------ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 14 -------------- 3 files changed, 1 insertion(+), 28 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 2140316d2ad0d..120b115721803 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -23,8 +23,6 @@ license: | {:toc} ## Upgrading from Spark SQL 2.4 to 3.0 - - Since Spark 3.0, the permanent function created using resource throws `AnalysisException` if the resource does not exists. - - Since Spark 3.0, when inserting a value into a table column with a different data type, the type coercion is performed as per ANSI SQL standard. Certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean` are disallowed. A runtime exception will be thrown if the value is out-of-range for the data type of the column. In Spark version 2.4 and earlier, type conversions during table insertion are allowed as long as they are valid `Cast`. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of byte type, the result is 1. The behavior is controlled by the option `spark.sql.storeAssignmentPolicy`, with a default value as "ANSI". Setting the option as "Legacy" restores the previous behavior. - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 13c9a06f78fdd..6fdc7f4a58195 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -19,13 +19,11 @@ package org.apache.spark.sql.execution.command import java.util.Locale -import org.apache.hadoop.fs.Path - import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchFunctionException} import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, FunctionResource} -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, ExpressionInfo} import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.types.{StringType, StructField, StructType} @@ -76,15 +74,6 @@ case class CreateFunctionCommand( } override def run(sparkSession: SparkSession): Seq[Row] = { - // Checks if the given resources exist - val hadoopConf = sparkSession.sparkContext.hadoopConfiguration - val nonExistentResources = resources.filter { r => - val path = new Path(r.uri) - !path.getFileSystem(hadoopConf).exists(path) - } - if (nonExistentResources.nonEmpty) { - throw new AnalysisException(s"Resources not found: ${nonExistentResources.mkString(",")}") - } val catalog = sparkSession.sessionState.catalog val func = CatalogFunction(FunctionIdentifier(functionName, databaseName), className, resources) if (isTemp) { 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 e472ceac7c1a6..9e629844eb496 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 @@ -3372,20 +3372,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } - test("SPARK-28670: create function should throw AnalysisException if UDF class not found") { - Seq(true, false).foreach { isTemporary => - val exp = intercept[AnalysisException] { - sql( - s""" - |CREATE ${if (isTemporary) "TEMPORARY" else ""} FUNCTION udtf_test - |AS 'org.apache.spark.sql.hive.execution.UDFTest' - |USING JAR '/var/invalid/invalid.jar' - """.stripMargin) - } - assert(exp.getMessage.contains("Resources not found")) - } - } - test("SPARK-30447: fix constant propagation inside NOT") { withTempView("t") { Seq[Integer](1, null).toDF("c").createOrReplaceTempView("t") From 1846b0261b84ce1bca079bc59fb4518bff910c18 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 14 Jan 2020 11:06:48 +0900 Subject: [PATCH 1096/1387] [SPARK-30500][SPARK-30501][SQL] Remove SQL configs deprecated in Spark 2.1 and 2.3 ### What changes were proposed in this pull request? In the PR, I propose to remove already deprecated SQL configs: - `spark.sql.variable.substitute.depth` deprecated in Spark 2.1 - `spark.sql.parquet.int64AsTimestampMillis` deprecated in Spark 2.3 Also I moved `removedSQLConfigs` closer to `deprecatedSQLConfigs`. This will allow to have references to other config entries. ### Why are the changes needed? To improve code maintainability. ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? By existing test suites `ParquetQuerySuite` and `SQLConfSuite`. Closes #27169 from MaxGekk/remove-deprecated-conf-2.4. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../apache/spark/sql/internal/SQLConf.scala | 100 +++++++----------- .../parquet/ParquetQuerySuite.scala | 57 ---------- .../spark/sql/internal/SQLConfSuite.scala | 10 +- 3 files changed, 39 insertions(+), 128 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 06fd7a79c4c93..c2798d36d7769 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -171,36 +171,6 @@ object SQLConf { } } - /** - * Holds information about keys that have been removed. - * - * @param key The removed config key. - * @param version Version of Spark where key was removed. - * @param defaultValue The default config value. It can be used to notice - * users that they set non-default value to an already removed config. - * @param comment Additional info regarding to the removed config. - */ - case class RemovedConfig(key: String, version: String, defaultValue: String, comment: String) - - /** - * The map contains info about removed SQL configs. Keys are SQL config names, - * map values contain extra information like the version in which the config was removed, - * config's default value and a comment. - */ - val removedSQLConfigs: Map[String, RemovedConfig] = { - val configs = Seq( - RemovedConfig("spark.sql.fromJsonForceNullableSchema", "3.0.0", "true", - "It was removed to prevent errors like SPARK-23173 for non-default value."), - RemovedConfig( - "spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation", "3.0.0", "false", - "It was removed to prevent loosing of users data for non-default value."), - RemovedConfig("spark.sql.legacy.compareDateTimestampInTimestamp", "3.0.0", "true", - "It was removed to prevent errors like SPARK-23549 for non-default value.") - ) - - Map(configs.map { cfg => cfg.key -> cfg } : _*) - } - val ANALYZER_MAX_ITERATIONS = buildConf("spark.sql.analyzer.maxIterations") .internal() .doc("The max number of iterations the analyzer runs.") @@ -555,14 +525,6 @@ object SQLConf { .checkValues(ParquetOutputTimestampType.values.map(_.toString)) .createWithDefault(ParquetOutputTimestampType.TIMESTAMP_MICROS.toString) - val PARQUET_INT64_AS_TIMESTAMP_MILLIS = buildConf("spark.sql.parquet.int64AsTimestampMillis") - .doc(s"(Deprecated since Spark 2.3, please set ${PARQUET_OUTPUT_TIMESTAMP_TYPE.key}.) " + - "When true, timestamp values will be stored as INT64 with TIMESTAMP_MILLIS as the " + - "extended type. In this mode, the microsecond portion of the timestamp value will be " + - "truncated.") - .booleanConf - .createWithDefault(false) - val PARQUET_COMPRESSION = buildConf("spark.sql.parquet.compression.codec") .doc("Sets the compression codec used when writing Parquet files. If either `compression` or " + "`parquet.compression` is specified in the table-specific options/properties, the " + @@ -1171,13 +1133,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val VARIABLE_SUBSTITUTE_DEPTH = - buildConf("spark.sql.variable.substitute.depth") - .internal() - .doc("Deprecated: The maximum replacements the substitution engine will do.") - .intConf - .createWithDefault(40) - val ENABLE_TWOLEVEL_AGG_MAP = buildConf("spark.sql.codegen.aggregate.map.twolevel.enabled") .internal() @@ -2186,13 +2141,9 @@ object SQLConf { */ val deprecatedSQLConfigs: Map[String, DeprecatedConfig] = { val configs = Seq( - DeprecatedConfig(VARIABLE_SUBSTITUTE_DEPTH.key, "2.1", - "The SQL config is not used by Spark anymore."), DeprecatedConfig(PANDAS_RESPECT_SESSION_LOCAL_TIMEZONE.key, "2.3", "Behavior for `false` config value is considered as a bug, and " + - "it will be prohibited in the future releases."), - DeprecatedConfig(PARQUET_INT64_AS_TIMESTAMP_MILLIS.key, "2.3", - s"Use '${PARQUET_OUTPUT_TIMESTAMP_TYPE.key}' instead of it."), + "it will be prohibited in the future releases."), DeprecatedConfig( PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key, "2.4", "The config allows to switch to the behaviour before Spark 2.4 " + @@ -2207,6 +2158,41 @@ object SQLConf { Map(configs.map { cfg => cfg.key -> cfg } : _*) } + + /** + * Holds information about keys that have been removed. + * + * @param key The removed config key. + * @param version Version of Spark where key was removed. + * @param defaultValue The default config value. It can be used to notice + * users that they set non-default value to an already removed config. + * @param comment Additional info regarding to the removed config. + */ + case class RemovedConfig(key: String, version: String, defaultValue: String, comment: String) + + /** + * The map contains info about removed SQL configs. Keys are SQL config names, + * map values contain extra information like the version in which the config was removed, + * config's default value and a comment. + */ + val removedSQLConfigs: Map[String, RemovedConfig] = { + val configs = Seq( + RemovedConfig("spark.sql.fromJsonForceNullableSchema", "3.0.0", "true", + "It was removed to prevent errors like SPARK-23173 for non-default value."), + RemovedConfig( + "spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation", "3.0.0", "false", + "It was removed to prevent loosing of users data for non-default value."), + RemovedConfig("spark.sql.legacy.compareDateTimestampInTimestamp", "3.0.0", "true", + "It was removed to prevent errors like SPARK-23549 for non-default value."), + RemovedConfig("spark.sql.variable.substitute.depth", "3.0.0", "40", + "It was deprecated since Spark 2.1, and not used in Spark 2.4."), + RemovedConfig("spark.sql.parquet.int64AsTimestampMillis", "3.0.0", "false", + "The config was deprecated since Spark 2.3." + + s"Use '${PARQUET_OUTPUT_TIMESTAMP_TYPE.key}' instead of it.") + ) + + Map(configs.map { cfg => cfg.key -> cfg } : _*) + } } /** @@ -2470,18 +2456,8 @@ class SQLConf extends Serializable with Logging { def isParquetINT96TimestampConversion: Boolean = getConf(PARQUET_INT96_TIMESTAMP_CONVERSION) - def isParquetINT64AsTimestampMillis: Boolean = getConf(PARQUET_INT64_AS_TIMESTAMP_MILLIS) - def parquetOutputTimestampType: ParquetOutputTimestampType.Value = { - val isOutputTimestampTypeSet = settings.containsKey(PARQUET_OUTPUT_TIMESTAMP_TYPE.key) - if (!isOutputTimestampTypeSet && isParquetINT64AsTimestampMillis) { - // If PARQUET_OUTPUT_TIMESTAMP_TYPE is not set and PARQUET_INT64_AS_TIMESTAMP_MILLIS is set, - // respect PARQUET_INT64_AS_TIMESTAMP_MILLIS and use TIMESTAMP_MILLIS. Otherwise, - // PARQUET_OUTPUT_TIMESTAMP_TYPE has higher priority. - ParquetOutputTimestampType.TIMESTAMP_MILLIS - } else { - ParquetOutputTimestampType.withName(getConf(PARQUET_OUTPUT_TIMESTAMP_TYPE)) - } + ParquetOutputTimestampType.withName(getConf(PARQUET_OUTPUT_TIMESTAMP_TYPE)) } def writeLegacyParquetFormat: Boolean = getConf(PARQUET_WRITE_LEGACY_FORMAT) @@ -2539,8 +2515,6 @@ class SQLConf extends Serializable with Logging { def variableSubstituteEnabled: Boolean = getConf(VARIABLE_SUBSTITUTE_ENABLED) - def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH) - def warehousePath: String = new Path(getConf(StaticSQLConf.WAREHOUSE_PATH)).toString def hiveThriftServerSingleSession: Boolean = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 61e93a7acd3f4..917aaba2669ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -141,24 +141,6 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS } } - test("SPARK-10634 timestamp written and read as INT64 - TIMESTAMP_MILLIS") { - val data = (1 to 10).map(i => Row(i, new java.sql.Timestamp(i))) - val schema = StructType(List(StructField("d", IntegerType, false), - StructField("time", TimestampType, false)).toArray) - withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "true") { - withTempPath { file => - val df = spark.createDataFrame(sparkContext.parallelize(data), schema) - df.write.parquet(file.getCanonicalPath) - ("true" :: "false" :: Nil).foreach { vectorized => - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { - val df2 = spark.read.parquet(file.getCanonicalPath) - checkAnswer(df2, df.collect().toSeq) - } - } - } - } - } - test("SPARK-10634 timestamp written and read as INT64 - truncation") { withTable("ts") { sql("create table ts (c1 int, c2 timestamp) using parquet") @@ -172,45 +154,6 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS .toDS().select('_1, $"_2".cast("timestamp")) checkAnswer(sql("select * from ts"), expected) } - - // The microsecond portion is truncated when written as TIMESTAMP_MILLIS. - withTable("ts") { - withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "true") { - sql("create table ts (c1 int, c2 timestamp) using parquet") - sql("insert into ts values (1, timestamp'2016-01-01 10:11:12.123456')") - sql("insert into ts values (2, null)") - sql("insert into ts values (3, timestamp'1965-01-01 10:11:12.125456')") - sql("insert into ts values (4, timestamp'1965-01-01 10:11:12.125')") - sql("insert into ts values (5, timestamp'1965-01-01 10:11:12.1')") - sql("insert into ts values (6, timestamp'1965-01-01 10:11:12.123456789')") - sql("insert into ts values (7, timestamp'0001-01-01 00:00:00.000000')") - val expected = Seq( - (1, "2016-01-01 10:11:12.123"), - (2, null), - (3, "1965-01-01 10:11:12.125"), - (4, "1965-01-01 10:11:12.125"), - (5, "1965-01-01 10:11:12.1"), - (6, "1965-01-01 10:11:12.123"), - (7, "0001-01-01 00:00:00.000")) - .toDS().select('_1, $"_2".cast("timestamp")) - checkAnswer(sql("select * from ts"), expected) - - // Read timestamps that were encoded as TIMESTAMP_MILLIS annotated as INT64 - // with PARQUET_INT64_AS_TIMESTAMP_MILLIS set to false. - withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "false") { - val expected = Seq( - (1, "2016-01-01 10:11:12.123"), - (2, null), - (3, "1965-01-01 10:11:12.125"), - (4, "1965-01-01 10:11:12.125"), - (5, "1965-01-01 10:11:12.1"), - (6, "1965-01-01 10:11:12.123"), - (7, "0001-01-01 00:00:00.000")) - .toDS().select('_1, $"_2".cast("timestamp")) - checkAnswer(sql("select * from ts"), expected) - } - } - } } test("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 23a70f912a5d5..29cd3d42f52df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -263,12 +263,6 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { assert(spark.sessionState.conf.parquetOutputTimestampType == SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS) - // PARQUET_INT64_AS_TIMESTAMP_MILLIS should be respected. - spark.sessionState.conf.setConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS, true) - assert(spark.sessionState.conf.parquetOutputTimestampType == - SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS) - - // PARQUET_OUTPUT_TIMESTAMP_TYPE has higher priority over PARQUET_INT64_AS_TIMESTAMP_MILLIS spark.sessionState.conf.setConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE, "timestamp_micros") assert(spark.sessionState.conf.parquetOutputTimestampType == SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS) @@ -350,13 +344,13 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { e.getRenderedMessage.contains(config))) } - val config1 = "spark.sql.hive.verifyPartitionPath" + val config1 = SQLConf.HIVE_VERIFY_PARTITION_PATH.key withLogAppender(logAppender) { spark.conf.set(config1, true) } check(config1) - val config2 = "spark.sql.execution.pandas.respectSessionTimeZone" + val config2 = SQLConf.ARROW_EXECUTION_ENABLED.key withLogAppender(logAppender) { spark.conf.unset(config2) } From 88fc8dbc09c5d24ae89413ab1e1fbabdf1fd8028 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 14 Jan 2020 16:03:10 +0900 Subject: [PATCH 1097/1387] [SPARK-30482][SQL][CORE][TESTS] Add sub-class of `AppenderSkeleton` reusable in tests ### What changes were proposed in this pull request? In the PR, I propose to define a sub-class of `AppenderSkeleton` in `SparkFunSuite` and reuse it from other tests. The class stores incoming `LoggingEvent` in an array which is available to tests for future analysis of logged events. ### Why are the changes needed? This eliminates code duplication in tests. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites - `CSVSuite`, `OptimizerLoggingSuite`, `JoinHintSuite`, `CodeGenerationSuite` and `SQLConfSuite`. Closes #27166 from MaxGekk/dedup-appender-skeleton. Authored-by: Maxim Gekk Signed-off-by: Takeshi Yamamuro --- .../org/apache/spark/SparkFunSuite.scala | 21 +++++++-- .../catalyst/analysis/ResolveHintsSuite.scala | 15 +------ .../expressions/CodeGenerationSuite.scala | 21 ++------- .../optimizer/OptimizerLoggingSuite.scala | 43 +++++-------------- .../org/apache/spark/sql/JoinHintSuite.scala | 15 +------ .../execution/datasources/csv/CSVSuite.scala | 17 ++------ .../spark/sql/internal/SQLConfSuite.scala | 12 +----- 7 files changed, 41 insertions(+), 103 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 9dd113262653b..bb64ad8cb19f6 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -20,15 +20,17 @@ package org.apache.spark // scalastyle:off import java.io.File -import scala.annotation.tailrec +import org.apache.log4j.spi.LoggingEvent -import org.apache.log4j.{Appender, Level, Logger} +import scala.annotation.tailrec +import org.apache.log4j.{Appender, AppenderSkeleton, Level, Logger} import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, Outcome} - import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.util.{AccumulatorContext, Utils} +import scala.collection.mutable.ArrayBuffer + /** * Base abstract class for all unit tests in Spark for handling common functionality. * @@ -186,4 +188,17 @@ abstract class SparkFunSuite } } } + + class LogAppender(maxEvents: Int = 100) extends AppenderSkeleton { + val loggingEvents = new ArrayBuffer[LoggingEvent]() + + override def append(loggingEvent: LoggingEvent): Unit = { + if (loggingEvents.size >= maxEvents) { + throw new IllegalStateException(s"Number of logging event reached the limit: $maxEvents") + } + loggingEvents.append(loggingEvent) + } + override def close(): Unit = {} + override def requiresLayout(): Boolean = false + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index 49ab34d2ea3a0..4fda65e201e71 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -17,10 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import scala.collection.mutable.ArrayBuffer - -import org.apache.log4j.{AppenderSkeleton, Level} -import org.apache.log4j.spi.LoggingEvent +import org.apache.log4j.Level import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -33,14 +30,6 @@ import org.apache.spark.sql.types.IntegerType class ResolveHintsSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.analysis.TestRelations._ - class MockAppender extends AppenderSkeleton { - val loggingEvents = new ArrayBuffer[LoggingEvent]() - - override def append(loggingEvent: LoggingEvent): Unit = loggingEvents.append(loggingEvent) - override def close(): Unit = {} - override def requiresLayout(): Boolean = false - } - test("invalid hints should be ignored") { checkAnalysis( UnresolvedHint("some_random_hint_that_does_not_exist", Seq("TaBlE"), table("TaBlE")), @@ -234,7 +223,7 @@ class ResolveHintsSuite extends AnalysisTest { } test("log warnings for invalid hints") { - val logAppender = new MockAppender() + val logAppender = new LogAppender withLogAppender(logAppender) { checkAnalysis( UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE")), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 28d2607e6e43e..32cbb858d86f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -19,9 +19,6 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import org.apache.log4j.AppenderSkeleton -import org.apache.log4j.spi.LoggingEvent - import org.apache.spark.SparkFunSuite import org.apache.spark.metrics.source.CodegenMetrics import org.apache.spark.sql.Row @@ -522,20 +519,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-25113: should log when there exists generated methods above HugeMethodLimit") { - class MockAppender extends AppenderSkeleton { - var seenMessage = false - - override def append(loggingEvent: LoggingEvent): Unit = { - if (loggingEvent.getRenderedMessage().contains("Generated method too long")) { - seenMessage = true - } - } - - override def close(): Unit = {} - override def requiresLayout(): Boolean = false - } - - val appender = new MockAppender() + val appender = new LogAppender withLogAppender(appender, loggerName = Some(classOf[CodeGenerator[_, _]].getName)) { val x = 42 val expr = HugeCodeIntExpression(x) @@ -543,7 +527,8 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val actual = proj(null) assert(actual.getInt(0) == x) } - assert(appender.seenMessage) + assert(appender.loggingEvents + .exists(_.getRenderedMessage().contains("Generated method too long"))) } test("SPARK-28916: subexrepssion elimination can cause 64kb code limit on UnsafeProjection") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala index 7a432d269abe6..927adc1551a88 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala @@ -17,10 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import scala.collection.mutable.ArrayBuffer - -import org.apache.log4j.{Appender, AppenderSkeleton, Level, Logger} -import org.apache.log4j.spi.LoggingEvent +import org.apache.log4j.Level import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -39,34 +36,8 @@ class OptimizerLoggingSuite extends PlanTest { ColumnPruning) :: Nil } - class MockAppender extends AppenderSkeleton { - val loggingEvents = new ArrayBuffer[LoggingEvent]() - - override def append(loggingEvent: LoggingEvent): Unit = { - if (loggingEvent.getRenderedMessage().contains("Applying Rule") || - loggingEvent.getRenderedMessage().contains("Result of Batch") || - loggingEvent.getRenderedMessage().contains("has no effect")) { - loggingEvents.append(loggingEvent) - } - } - - override def close(): Unit = {} - override def requiresLayout(): Boolean = false - } - - private def withLogLevelAndAppender(level: Level, appender: Appender)(f: => Unit): Unit = { - val logger = Logger.getLogger(Optimize.getClass.getName.dropRight(1)) - val restoreLevel = logger.getLevel - logger.setLevel(level) - logger.addAppender(appender) - try f finally { - logger.setLevel(restoreLevel) - logger.removeAppender(appender) - } - } - private def verifyLog(expectedLevel: Level, expectedRulesOrBatches: Seq[String]): Unit = { - val logAppender = new MockAppender() + val logAppender = new LogAppender withLogAppender(logAppender, loggerName = Some(Optimize.getClass.getName.dropRight(1)), level = Some(Level.TRACE)) { val input = LocalRelation('a.int, 'b.string, 'c.double) @@ -74,10 +45,16 @@ class OptimizerLoggingSuite extends PlanTest { val expected = input.where('a > 1).select('a).analyze comparePlans(Optimize.execute(query), expected) } - val logMessages = logAppender.loggingEvents.map(_.getRenderedMessage) + val events = logAppender.loggingEvents.filter { + case event => Seq( + "Applying Rule", + "Result of Batch", + "has no effect").exists(event.getRenderedMessage().contains) + } + val logMessages = events.map(_.getRenderedMessage) assert(expectedRulesOrBatches.forall (ruleOrBatch => logMessages.exists(_.contains(ruleOrBatch)))) - assert(logAppender.loggingEvents.forall(_.getLevel == expectedLevel)) + assert(events.forall(_.getLevel == expectedLevel)) } test("test log level") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index f766688f2a2da..5767d76f2c14f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -17,10 +17,7 @@ package org.apache.spark.sql -import scala.collection.mutable.ArrayBuffer - -import org.apache.log4j.{AppenderSkeleton, Level} -import org.apache.log4j.spi.LoggingEvent +import org.apache.log4j.Level import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint import org.apache.spark.sql.catalyst.plans.PlanTest @@ -39,14 +36,6 @@ class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkP lazy val df2 = df.selectExpr("id as b1", "id as b2") lazy val df3 = df.selectExpr("id as c1", "id as c2") - class MockAppender extends AppenderSkeleton { - val loggingEvents = new ArrayBuffer[LoggingEvent]() - - override def append(loggingEvent: LoggingEvent): Unit = loggingEvents.append(loggingEvent) - override def close(): Unit = {} - override def requiresLayout(): Boolean = false - } - def msgNoHintRelationFound(relation: String, hint: String): String = s"Count not find relation '$relation' specified in hint '$hint'." @@ -60,7 +49,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkP df: => DataFrame, expectedHints: Seq[JoinHint], warnings: Seq[String]): Unit = { - val logAppender = new MockAppender() + val logAppender = new LogAppender withLogAppender(logAppender) { verifyJoinHint(df, expectedHints) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index f6adc7acb2772..ae9aaf15aae9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -32,8 +32,6 @@ import com.univocity.parsers.common.TextParsingException import org.apache.commons.lang3.time.FastDateFormat import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec -import org.apache.log4j.{AppenderSkeleton, LogManager} -import org.apache.log4j.spi.LoggingEvent import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} @@ -1763,24 +1761,17 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } test("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") { - class TestAppender extends AppenderSkeleton { - var events = new java.util.ArrayList[LoggingEvent] - override def close(): Unit = {} - override def requiresLayout: Boolean = false - protected def append(event: LoggingEvent): Unit = events.add(event) - } - - val testAppender1 = new TestAppender + val testAppender1 = new LogAppender withLogAppender(testAppender1) { val ds = Seq("columnA,columnB", "1.0,1000.0").toDS() val ischema = new StructType().add("columnB", DoubleType).add("columnA", DoubleType) spark.read.schema(ischema).option("header", true).option("enforceSchema", true).csv(ds) } - assert(testAppender1.events.asScala + assert(testAppender1.loggingEvents .exists(msg => msg.getRenderedMessage.contains("CSV header does not conform to the schema"))) - val testAppender2 = new TestAppender + val testAppender2 = new LogAppender withLogAppender(testAppender2) { withTempPath { path => val oschema = new StructType().add("f1", DoubleType).add("f2", DoubleType) @@ -1795,7 +1786,7 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { .collect() } } - assert(testAppender2.events.asScala + assert(testAppender2.loggingEvents .exists(msg => msg.getRenderedMessage.contains("CSV header does not conform to the schema"))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 29cd3d42f52df..db01db76f8699 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.internal -import scala.collection.mutable.ArrayBuffer import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path -import org.apache.log4j.{AppenderSkeleton, Level} -import org.apache.log4j.spi.LoggingEvent +import org.apache.log4j.Level import org.apache.spark.sql._ import org.apache.spark.sql.internal.StaticSQLConf._ @@ -331,13 +329,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } test("log deprecation warnings") { - val logAppender = new AppenderSkeleton { - val loggingEvents = new ArrayBuffer[LoggingEvent]() - - override def append(loggingEvent: LoggingEvent): Unit = loggingEvents.append(loggingEvent) - override def close(): Unit = {} - override def requiresLayout(): Boolean = false - } + val logAppender = new LogAppender def check(config: String): Unit = { assert(logAppender.loggingEvents.exists( e => e.getLevel == Level.WARN && From e0efd213eb6ff61ce2b0f54f55201899fa4171a6 Mon Sep 17 00:00:00 2001 From: root1 Date: Tue, 14 Jan 2020 17:03:10 +0800 Subject: [PATCH 1098/1387] [SPARK-30292][SQL] Throw Exception when invalid string is cast to numeric type in ANSI mode ### What changes were proposed in this pull request? If spark.sql.ansi.enabled is set, throw exception when cast to any numeric type do not follow the ANSI SQL standards. ### Why are the changes needed? ANSI SQL standards do not allow invalid strings to get casted into numeric types and throw exception for that. Currently spark sql gives NULL in such cases. Before: `select cast('str' as decimal) => NULL` After : `select cast('str' as decimal) => invalid input syntax for type numeric: str` These results are after setting `spark.sql.ansi.enabled=true` ### Does this PR introduce any user-facing change? Yes. Now when ansi mode is on users will get arithmetic exception for invalid strings. ### How was this patch tested? Unit Tests Added. Closes #26933 from iRakson/castDecimalANSI. Lead-authored-by: root1 Co-authored-by: iRakson Signed-off-by: Wenchen Fan --- .../apache/spark/unsafe/types/UTF8String.java | 46 ++++ .../spark/sql/catalyst/expressions/Cast.scala | 59 ++++- .../sql/catalyst/expressions/CastSuite.scala | 221 ++++++++++++------ .../results/postgreSQL/float4.sql.out | 20 +- .../results/postgreSQL/float8.sql.out | 20 +- .../sql-tests/results/postgreSQL/text.sql.out | 10 +- .../results/postgreSQL/window_part2.sql.out | 11 +- .../results/postgreSQL/window_part4.sql.out | 9 +- 8 files changed, 278 insertions(+), 118 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 3754a1a0374a8..c5384669eb922 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -1294,6 +1294,52 @@ public boolean toByte(IntWrapper intWrapper) { return false; } + /** + * Parses UTF8String(trimmed if needed) to long. This method is used when ANSI is enabled. + * + * @return If string contains valid numeric value then it returns the long value otherwise a + * NumberFormatException is thrown. + */ + public long toLongExact() { + LongWrapper result = new LongWrapper(); + if (toLong(result)) { + return result.value; + } + throw new NumberFormatException("invalid input syntax for type numeric: " + this); + } + + /** + * Parses UTF8String(trimmed if needed) to int. This method is used when ANSI is enabled. + * + * @return If string contains valid numeric value then it returns the int value otherwise a + * NumberFormatException is thrown. + */ + public int toIntExact() { + IntWrapper result = new IntWrapper(); + if (toInt(result)) { + return result.value; + } + throw new NumberFormatException("invalid input syntax for type numeric: " + this); + } + + public short toShortExact() { + int value = this.toIntExact(); + short result = (short) value; + if (result == value) { + return result; + } + throw new NumberFormatException("invalid input syntax for type numeric: " + this); + } + + public byte toByteExact() { + int value = this.toIntExact(); + byte result = (byte) value; + if (result == value) { + return result; + } + throw new NumberFormatException("invalid input syntax for type numeric: " + this); + } + @Override public String toString() { return new String(getBytes(), StandardCharsets.UTF_8); 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 fa27a48419dbb..4fd74a4e4658b 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 @@ -482,6 +482,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // LongConverter private[this] def castToLong(from: DataType): Any => Any = from match { + case StringType if ansiEnabled => + buildCast[UTF8String](_, _.toLongExact()) case StringType => val result = new LongWrapper() buildCast[UTF8String](_, s => if (s.toLong(result)) result.value else null) @@ -499,6 +501,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // IntConverter private[this] def castToInt(from: DataType): Any => Any = from match { + case StringType if ansiEnabled => + buildCast[UTF8String](_, _.toIntExact()) case StringType => val result = new IntWrapper() buildCast[UTF8String](_, s => if (s.toInt(result)) result.value else null) @@ -518,6 +522,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // ShortConverter private[this] def castToShort(from: DataType): Any => Any = from match { + case StringType if ansiEnabled => + buildCast[UTF8String](_, _.toShortExact()) case StringType => val result = new IntWrapper() buildCast[UTF8String](_, s => if (s.toShort(result)) { @@ -559,6 +565,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // ByteConverter private[this] def castToByte(from: DataType): Any => Any = from match { + case StringType if ansiEnabled => + buildCast[UTF8String](_, _.toByteExact()) case StringType => val result = new IntWrapper() buildCast[UTF8String](_, s => if (s.toByte(result)) { @@ -636,7 +644,12 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // Please refer to https://github.com/apache/spark/pull/26640 changePrecision(Decimal(new JavaBigDecimal(s.toString.trim)), target) } catch { - case _: NumberFormatException => null + case _: NumberFormatException => + if (ansiEnabled) { + throw new NumberFormatException(s"invalid input syntax for type numeric: $s") + } else { + null + } }) case BooleanType => buildCast[Boolean](_, b => toPrecision(if (b) Decimal.ONE else Decimal.ZERO, target)) @@ -664,7 +677,12 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val doubleStr = s.toString try doubleStr.toDouble catch { case _: NumberFormatException => - Cast.processFloatingPointSpecialLiterals(doubleStr, false) + val d = Cast.processFloatingPointSpecialLiterals(doubleStr, false) + if(ansiEnabled && d == null) { + throw new NumberFormatException(s"invalid input syntax for type numeric: $s") + } else { + d + } } }) case BooleanType => @@ -684,7 +702,12 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit val floatStr = s.toString try floatStr.toFloat catch { case _: NumberFormatException => - Cast.processFloatingPointSpecialLiterals(floatStr, true) + val f = Cast.processFloatingPointSpecialLiterals(floatStr, true) + if (ansiEnabled && f == null) { + throw new NumberFormatException(s"invalid input syntax for type numeric: $s") + } else { + f + } } }) case BooleanType => @@ -1128,12 +1151,17 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit from match { case StringType => (c, evPrim, evNull) => + val handleException = if (ansiEnabled) { + s"""throw new NumberFormatException("invalid input syntax for type numeric: $c");""" + } else { + s"$evNull =true;" + } code""" try { Decimal $tmp = Decimal.apply(new java.math.BigDecimal($c.toString().trim())); ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast)} } catch (java.lang.NumberFormatException e) { - $evNull = true; + $handleException } """ case BooleanType => @@ -1355,6 +1383,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } private[this] def castToByteCode(from: DataType, ctx: CodegenContext): CastFunction = from match { + case StringType if ansiEnabled => + (c, evPrim, evNull) => code"$evPrim = $c.toByteExact();" case StringType => val wrapper = ctx.freshVariable("intWrapper", classOf[UTF8String.IntWrapper]) (c, evPrim, evNull) => @@ -1386,6 +1416,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def castToShortCode( from: DataType, ctx: CodegenContext): CastFunction = from match { + case StringType if ansiEnabled => + (c, evPrim, evNull) => code"$evPrim = $c.toShortExact();" case StringType => val wrapper = ctx.freshVariable("intWrapper", classOf[UTF8String.IntWrapper]) (c, evPrim, evNull) => @@ -1415,6 +1447,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } private[this] def castToIntCode(from: DataType, ctx: CodegenContext): CastFunction = from match { + case StringType if ansiEnabled => + (c, evPrim, evNull) => code"$evPrim = $c.toIntExact();" case StringType => val wrapper = ctx.freshVariable("intWrapper", classOf[UTF8String.IntWrapper]) (c, evPrim, evNull) => @@ -1443,9 +1477,10 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } private[this] def castToLongCode(from: DataType, ctx: CodegenContext): CastFunction = from match { + case StringType if ansiEnabled => + (c, evPrim, evNull) => code"$evPrim = $c.toLongExact();" case StringType => val wrapper = ctx.freshVariable("longWrapper", classOf[UTF8String.LongWrapper]) - (c, evPrim, evNull) => code""" UTF8String.LongWrapper $wrapper = new UTF8String.LongWrapper(); @@ -1476,6 +1511,11 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case StringType => val floatStr = ctx.freshVariable("floatStr", StringType) (c, evPrim, evNull) => + val handleNull = if (ansiEnabled) { + s"""throw new NumberFormatException("invalid input syntax for type numeric: $c");""" + } else { + s"$evNull = true;" + } code""" final String $floatStr = $c.toString(); try { @@ -1483,7 +1523,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } catch (java.lang.NumberFormatException e) { final Float f = (Float) Cast.processFloatingPointSpecialLiterals($floatStr, true); if (f == null) { - $evNull = true; + $handleNull } else { $evPrim = f.floatValue(); } @@ -1507,6 +1547,11 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit case StringType => val doubleStr = ctx.freshVariable("doubleStr", StringType) (c, evPrim, evNull) => + val handleNull = if (ansiEnabled) { + s"""throw new NumberFormatException("invalid input syntax for type numeric: $c");""" + } else { + s"$evNull = true;" + } code""" final String $doubleStr = $c.toString(); try { @@ -1514,7 +1559,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } catch (java.lang.NumberFormatException e) { final Double d = (Double) Cast.processFloatingPointSpecialLiterals($doubleStr, false); if (d == null) { - $evNull = true; + $handleNull } else { $evPrim = d.doubleValue(); } 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 2d8f22c34ade7..7418870d9c731 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 @@ -284,7 +284,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { val gmtId = Option("GMT") checkEvaluation(cast("abdef", StringType), "abdef") - checkEvaluation(cast("abdef", DecimalType.USER_DEFAULT), null) checkEvaluation(cast("abdef", TimestampType, gmtId), null) checkEvaluation(cast("12.65", DecimalType.SYSTEM_DEFAULT), Decimal(12.65)) @@ -324,7 +323,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast("23", DecimalType.USER_DEFAULT), Decimal(23)) checkEvaluation(cast("23", ByteType), 23.toByte) checkEvaluation(cast("23", ShortType), 23.toShort) - checkEvaluation(cast("2012-12-11", DoubleType), null) checkEvaluation(cast(123, IntegerType), 123) checkEvaluation(cast(Literal.create(null, IntegerType), ShortType), null) @@ -410,15 +408,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkNullCast(ArrayType(StringType), ArrayType(IntegerType)) - { - val ret = cast(array, ArrayType(IntegerType, containsNull = true)) - assert(ret.resolved) - checkEvaluation(ret, Seq(123, null, null, null)) - } - { - val ret = cast(array, ArrayType(IntegerType, containsNull = false)) - assert(ret.resolved === false) - } { val ret = cast(array, ArrayType(BooleanType, containsNull = true)) assert(ret.resolved) @@ -429,15 +418,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { assert(ret.resolved === false) } - { - val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = true)) - assert(ret.resolved) - checkEvaluation(ret, Seq(123, null, null)) - } - { - val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = false)) - assert(ret.resolved === false) - } { val ret = cast(array_notNull, ArrayType(BooleanType, containsNull = true)) assert(ret.resolved) @@ -464,15 +444,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkNullCast(MapType(StringType, IntegerType), MapType(StringType, StringType)) - { - val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = true)) - assert(ret.resolved) - checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null, "d" -> null)) - } - { - val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = false)) - assert(ret.resolved === false) - } { val ret = cast(map, MapType(StringType, BooleanType, valueContainsNull = true)) assert(ret.resolved) @@ -486,16 +457,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { val ret = cast(map, MapType(IntegerType, StringType, valueContainsNull = true)) assert(ret.resolved === false) } - - { - val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = true)) - assert(ret.resolved) - checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null)) - } - { - val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = false)) - assert(ret.resolved === false) - } { val ret = cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = true)) assert(ret.resolved) @@ -546,23 +507,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { StructField("b", StringType, nullable = false), StructField("c", StringType, nullable = false)))) - { - val ret = cast(struct, StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", IntegerType, nullable = true), - StructField("c", IntegerType, nullable = true), - StructField("d", IntegerType, nullable = true)))) - assert(ret.resolved) - checkEvaluation(ret, InternalRow(123, null, null, null)) - } - { - val ret = cast(struct, StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", IntegerType, nullable = true), - StructField("c", IntegerType, nullable = false), - StructField("d", IntegerType, nullable = true)))) - assert(ret.resolved === false) - } { val ret = cast(struct, StructType(Seq( StructField("a", BooleanType, nullable = true), @@ -581,21 +525,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { assert(ret.resolved === false) } - { - val ret = cast(struct_notNull, StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", IntegerType, nullable = true), - StructField("c", IntegerType, nullable = true)))) - assert(ret.resolved) - checkEvaluation(ret, InternalRow(123, null, null)) - } - { - val ret = cast(struct_notNull, StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", IntegerType, nullable = true), - StructField("c", IntegerType, nullable = false)))) - assert(ret.resolved === false) - } { val ret = cast(struct_notNull, StructType(Seq( StructField("a", BooleanType, nullable = true), @@ -921,11 +850,6 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { Seq("nan", "nAn", " nan ").foreach { value => checkEvaluation(cast(value, DoubleType), Double.NaN) } - - // Invalid literals when casted to double and float results in null. - Seq(DoubleType, FloatType).foreach { dataType => - checkEvaluation(cast("badvalue", dataType), null) - } } private def testIntMaxAndMin(dt: DataType): Unit = { @@ -1054,7 +978,6 @@ class CastSuite extends CastSuiteBase { } } - test("cast from int") { checkCast(0, false) checkCast(1, true) @@ -1214,6 +1137,125 @@ class CastSuite extends CastSuiteBase { val set = CollectSet(Literal(1)) assert(Cast.canCast(set.dataType, ArrayType(StringType, false))) } + + test("Cast should output null for invalid strings when ANSI is not enabled.") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + checkEvaluation(cast("abdef", DecimalType.USER_DEFAULT), null) + checkEvaluation(cast("2012-12-11", DoubleType), null) + + // cast to array + val array = Literal.create(Seq("123", "true", "f", null), + ArrayType(StringType, containsNull = true)) + val array_notNull = Literal.create(Seq("123", "true", "f"), + ArrayType(StringType, containsNull = false)) + + { + val ret = cast(array, ArrayType(IntegerType, containsNull = true)) + assert(ret.resolved) + checkEvaluation(ret, Seq(123, null, null, null)) + } + { + val ret = cast(array, ArrayType(IntegerType, containsNull = false)) + assert(ret.resolved === false) + } + { + val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = true)) + assert(ret.resolved) + checkEvaluation(ret, Seq(123, null, null)) + } + { + val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = false)) + assert(ret.resolved === false) + } + + // cast from map + val map = Literal.create( + Map("a" -> "123", "b" -> "true", "c" -> "f", "d" -> null), + MapType(StringType, StringType, valueContainsNull = true)) + val map_notNull = Literal.create( + Map("a" -> "123", "b" -> "true", "c" -> "f"), + MapType(StringType, StringType, valueContainsNull = false)) + + { + val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = true)) + assert(ret.resolved) + checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null, "d" -> null)) + } + { + val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = false)) + assert(ret.resolved === false) + } + { + val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = true)) + assert(ret.resolved) + checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null)) + } + { + val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = false)) + assert(ret.resolved === false) + } + + // cast from struct + val struct = Literal.create( + InternalRow( + UTF8String.fromString("123"), + UTF8String.fromString("true"), + UTF8String.fromString("f"), + null), + StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = true), + StructField("c", StringType, nullable = true), + StructField("d", StringType, nullable = true)))) + val struct_notNull = Literal.create( + InternalRow( + UTF8String.fromString("123"), + UTF8String.fromString("true"), + UTF8String.fromString("f")), + StructType(Seq( + StructField("a", StringType, nullable = false), + StructField("b", StringType, nullable = false), + StructField("c", StringType, nullable = false)))) + + { + val ret = cast(struct, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = true), + StructField("d", IntegerType, nullable = true)))) + assert(ret.resolved) + checkEvaluation(ret, InternalRow(123, null, null, null)) + } + { + val ret = cast(struct, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false), + StructField("d", IntegerType, nullable = true)))) + assert(ret.resolved === false) + } + { + val ret = cast(struct_notNull, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = true)))) + assert(ret.resolved) + checkEvaluation(ret, InternalRow(123, null, null)) + } + { + val ret = cast(struct_notNull, StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", IntegerType, nullable = true), + StructField("c", IntegerType, nullable = false)))) + assert(ret.resolved === false) + } + + // Invalid literals when casted to double and float results in null. + Seq(DoubleType, FloatType).foreach { dataType => + checkEvaluation(cast("badvalue", dataType), null) + } + } + } } /** @@ -1229,4 +1271,29 @@ class AnsiCastSuite extends CastSuiteBase { case _ => AnsiCast(Literal(v), targetType, timeZoneId) } } + + test("cast from invalid string to numeric should throw NumberFormatException") { + // cast to IntegerType + Seq(IntegerType, ShortType, ByteType, LongType).foreach { dataType => + val array = Literal.create(Seq("123", "true", "f", null), + ArrayType(StringType, containsNull = true)) + checkExceptionInExpression[NumberFormatException]( + cast(array, ArrayType(dataType, containsNull = true)), "invalid input") + checkExceptionInExpression[NumberFormatException]( + cast("string", dataType), "invalid input") + checkExceptionInExpression[NumberFormatException]( + cast("123-string", dataType), "invalid input") + checkExceptionInExpression[NumberFormatException]( + cast("2020-07-19", dataType), "invalid input") + } + + Seq(DoubleType, FloatType, DecimalType.USER_DEFAULT).foreach { dataType => + checkExceptionInExpression[NumberFormatException]( + cast("string", dataType), "invalid input") + checkExceptionInExpression[NumberFormatException]( + cast("123.000.00", dataType), "invalid input") + checkExceptionInExpression[NumberFormatException]( + cast("abc.com", dataType), "invalid input") + } + } } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out index 64608a349b610..7ca4c7eb8aa30 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out @@ -93,25 +93,28 @@ struct -- !query 11 SELECT float('N A N') -- !query 11 schema -struct +struct<> -- !query 11 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: N A N -- !query 12 SELECT float('NaN x') -- !query 12 schema -struct +struct<> -- !query 12 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: NaN x -- !query 13 SELECT float(' INFINITY x') -- !query 13 schema -struct +struct<> -- !query 13 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: INFINITY x -- !query 14 @@ -141,9 +144,10 @@ NaN -- !query 17 SELECT float(decimal('nan')) -- !query 17 schema -struct +struct<> -- !query 17 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: nan -- !query 18 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out index d38e36e956985..9d170d2fd8898 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out @@ -125,25 +125,28 @@ struct -- !query 15 SELECT double('N A N') -- !query 15 schema -struct +struct<> -- !query 15 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: N A N -- !query 16 SELECT double('NaN x') -- !query 16 schema -struct +struct<> -- !query 16 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: NaN x -- !query 17 SELECT double(' INFINITY x') -- !query 17 schema -struct +struct<> -- !query 17 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: INFINITY x -- !query 18 @@ -173,9 +176,10 @@ NaN -- !query 21 SELECT double(decimal('nan')) -- !query 21 schema -struct +struct<> -- !query 21 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: nan -- !query 22 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 311b0eb5a5844..6faa5b6924460 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -62,17 +62,19 @@ struct -- !query 7 select string('four: ') || 2+2 -- !query 7 schema -struct<(CAST(concat(CAST(four: AS STRING), CAST(2 AS STRING)) AS DOUBLE) + CAST(2 AS DOUBLE)):double> +struct<> -- !query 7 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: four: 2 -- !query 8 select 'four: ' || 2+2 -- !query 8 schema -struct<(CAST(concat(four: , CAST(2 AS STRING)) AS DOUBLE) + CAST(2 AS DOUBLE)):double> +struct<> -- !query 8 output -NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: four: 2 -- !query 9 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out index 0d2c78847b97c..9183eb659237e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -452,15 +452,10 @@ from numerics window w as (order by f_numeric range between 1.1 preceding and 'NaN' following) -- !query 28 schema -struct +struct<> -- !query 28 output -1 -3 NULL NULL -2 -1 NULL NULL -3 0 NULL NULL -4 1 NULL NULL -5 1 NULL NULL -6 2 NULL NULL -7 100 NULL NULL +java.lang.NumberFormatException +invalid input syntax for type numeric: NaN -- !query 29 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index 4721ceb03a966..333e28f04a4d9 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -498,10 +498,7 @@ SELECT a, b, SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b) -- !query 38 schema -struct +struct<> -- !query 38 output -1 1 1 -2 2 3 -3 NULL 2 -4 3 3 -5 4 7 +org.apache.spark.sql.AnalysisException +failed to evaluate expression CAST('nan' AS INT): invalid input syntax for type numeric: nan; line 3 pos 6 From 446275621642c6ec0a2801c3cf24666321e68a46 Mon Sep 17 00:00:00 2001 From: yu Date: Tue, 14 Jan 2020 17:17:13 +0800 Subject: [PATCH 1099/1387] [SPARK-30325][CORE] markPartitionCompleted cause task status inconsistent ### **What changes were proposed in this pull request?** Fix task status inconsistent in `executorLost` which caused by `markPartitionCompleted` ### **Why are the changes needed?** The inconsistent will cause app hung up. The bugs occurs in the corer case as follows: 1. The stage occurs during stage retry, scheduler will resubmit a new stage with unfinished tasks. 2. Those unfinished tasks in origin stage finished and the same task on the new retry stage hasn't finished, it will mark the task partition on the current retry stage as succesuful in TSM `successful` array variable. 3. The executor crashed when it is running tasks which have succeeded by origin stage, it cause TSM run `executorLost` to rescheduler the task on the executor, and it will change the partition's running status in `copiesRunning` twice to -1. 4. 'dequeueTaskFromList' will use `copiesRunning` equal 0 as reschedule basis when rescheduler tasks, and now it is -1, can't to reschedule, and the app will hung forever. ### **Does this PR introduce any user-facing change?** No ### **How was this patch tested?** Closes #26975 from seayoun/fix_stageRetry_executorCrash_cause_problems. Authored-by: yu Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 11a1dc154ac8a..3b620ec69a9ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -945,7 +945,10 @@ private[spark] class TaskSetManager( && !isZombie) { for ((tid, info) <- taskInfos if info.executorId == execId) { val index = taskInfos(tid).index - if (successful(index) && !killedByOtherAttempt.contains(tid)) { + // We may have a running task whose partition has been marked as successful, + // this partition has another task completed in another stage attempt. + // We treat it as a running task and will call handleFailedTask later. + if (successful(index) && !info.running && !killedByOtherAttempt.contains(tid)) { successful(index) = false copiesRunning(index) -= 1 tasksSuccessful -= 1 From 2688faeea560dc41599352380d2d3cad361a1ab2 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 14 Jan 2020 17:24:17 +0800 Subject: [PATCH 1100/1387] [SPARK-30498][ML][PYSPARK] Fix some ml parity issues between python and scala ### What changes were proposed in this pull request? There are some parity issues between python and scala ### Why are the changes needed? keep parity between python and scala ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? existing tests Closes #27196 from huaxingao/spark-30498. Authored-by: Huaxin Gao Signed-off-by: zhengruifeng --- python/pyspark/ml/classification.py | 8 ----- python/pyspark/ml/clustering.py | 4 --- python/pyspark/ml/feature.py | 42 ++++++++---------------- python/pyspark/ml/regression.py | 1 - python/pyspark/ml/tuning.py | 50 +++-------------------------- 5 files changed, 18 insertions(+), 87 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index bfed88fdcb5d5..58044556b307d 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1311,7 +1311,6 @@ def setCheckpointInterval(self, value): """ return self._set(checkpointInterval=value) - @since("1.4.0") def setSeed(self, value): """ Sets the value of :py:attr:`seed`. @@ -2163,13 +2162,6 @@ def getBlockSize(self): """ return self.getOrDefault(self.blockSize) - @since("2.0.0") - def getStepSize(self): - """ - Gets the value of stepSize or its default value. - """ - return self.getOrDefault(self.stepSize) - @since("2.0.0") def getInitialWeights(self): """ diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index e236ff0423c91..7465cef7d64d4 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -1155,10 +1155,6 @@ def setSeed(self, value): def setTopicDistributionCol(self, value): """ Sets the value of :py:attr:`topicDistributionCol`. - - >>> algo = LDA().setTopicDistributionCol("topicDistributionCol") - >>> algo.getTopicDistributionCol() - 'topicDistributionCol' """ return self._set(topicDistributionCol=value) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 839386b3a4c16..dc6cc7d674179 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -449,20 +449,6 @@ class BucketedRandomProjectionLSHModel(_LSHModel, _BucketedRandomProjectionLSHPa .. versionadded:: 2.2.0 """ - @since("3.0.0") - def setInputCol(self, value): - """ - Sets the value of :py:attr:`inputCol`. - """ - return self._set(inputCol=value) - - @since("3.0.0") - def setOutputCol(self, value): - """ - Sets the value of :py:attr:`outputCol`. - """ - return self._set(outputCol=value) - @inherit_doc class Bucketizer(JavaTransformer, HasInputCol, HasOutputCol, HasInputCols, HasOutputCols, @@ -873,20 +859,6 @@ def setOutputCol(self, value): """ return self._set(outputCol=value) - @since("3.0.0") - def setMinTF(self, value): - """ - Sets the value of :py:attr:`minTF`. - """ - return self._set(minTF=value) - - @since("3.0.0") - def setBinary(self, value): - """ - Sets the value of :py:attr:`binary`. - """ - return self._set(binary=value) - @classmethod @since("2.4.0") def from_vocabulary(cls, vocabulary, inputCol, outputCol=None, minTF=None, binary=None): @@ -1714,6 +1686,20 @@ def setOutputCols(self, value): """ return self._set(outputCols=value) + @since("3.0.0") + def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ + return self._set(inputCol=value) + + @since("3.0.0") + def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ + return self._set(outputCol=value) + @property @since("2.2.0") def surrogateDF(self): diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 70c131795d9f3..d5a8dd5e8457a 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -930,7 +930,6 @@ def setCheckpointInterval(self, value): """ return self._set(checkpointInterval=value) - @since("1.4.0") def setSeed(self, value): """ Sets the value of :py:attr:`seed`. diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 5eb8ae44d3d66..cb4542c5d025f 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -455,27 +455,6 @@ def __init__(self, bestModel, avgMetrics=[], subModels=None): #: sub model list from cross validation self.subModels = subModels - @since("2.0.0") - def setEstimator(self, value): - """ - Sets the value of :py:attr:`estimator`. - """ - return self._set(estimator=value) - - @since("2.0.0") - def setEstimatorParamMaps(self, value): - """ - Sets the value of :py:attr:`estimatorParamMaps`. - """ - return self._set(estimatorParamMaps=value) - - @since("2.0.0") - def setEvaluator(self, value): - """ - Sets the value of :py:attr:`evaluator`. - """ - return self._set(evaluator=value) - def _transform(self, dataset): return self.bestModel.transform(dataset) @@ -520,8 +499,8 @@ def _from_java(cls, java_stage): avgMetrics = _java2py(sc, java_stage.avgMetrics()) estimator, epms, evaluator = super(CrossValidatorModel, cls)._from_java_impl(java_stage) - py_stage = cls(bestModel=bestModel, avgMetrics=avgMetrics).setEstimator(estimator) - py_stage = py_stage.setEstimatorParamMaps(epms).setEvaluator(evaluator) + py_stage = cls(bestModel=bestModel, avgMetrics=avgMetrics)._set(estimator=estimator) + py_stage = py_stage._set(estimatorParamMaps=epms)._set(evaluator=evaluator) if java_stage.hasSubModels(): py_stage.subModels = [[JavaParams._from_java(sub_model) @@ -807,27 +786,6 @@ def __init__(self, bestModel, validationMetrics=[], subModels=None): #: sub models from train validation split self.subModels = subModels - @since("2.0.0") - def setEstimator(self, value): - """ - Sets the value of :py:attr:`estimator`. - """ - return self._set(estimator=value) - - @since("2.0.0") - def setEstimatorParamMaps(self, value): - """ - Sets the value of :py:attr:`estimatorParamMaps`. - """ - return self._set(estimatorParamMaps=value) - - @since("2.0.0") - def setEvaluator(self, value): - """ - Sets the value of :py:attr:`evaluator`. - """ - return self._set(evaluator=value) - def _transform(self, dataset): return self.bestModel.transform(dataset) @@ -877,8 +835,8 @@ def _from_java(cls, java_stage): cls)._from_java_impl(java_stage) # Create a new instance of this stage. py_stage = cls(bestModel=bestModel, - validationMetrics=validationMetrics).setEstimator(estimator) - py_stage = py_stage.setEstimatorParamMaps(epms).setEvaluator(evaluator) + validationMetrics=validationMetrics)._set(estimator=estimator) + py_stage = py_stage._set(estimatorParamMaps=epms)._set(evaluator=evaluator) if java_stage.hasSubModels(): py_stage.subModels = [JavaParams._from_java(sub_model) From a2aa966ef64bc06f65a646777568427d360605e9 Mon Sep 17 00:00:00 2001 From: jiake Date: Tue, 14 Jan 2020 20:31:44 +0800 Subject: [PATCH 1101/1387] [SPARK-29544][SQL] optimize skewed partition based on data size ### What changes were proposed in this pull request? Skew Join is common and can severely downgrade performance of queries, especially those with joins. This PR aim to optimization the skew join based on the runtime Map output statistics by adding "OptimizeSkewedPartitions" rule. And The details design doc is [here](https://docs.google.com/document/d/1NkXN-ck8jUOS0COz3f8LUW5xzF8j9HFjoZXWGGX2HAg/edit). Currently we can support "Inner, Cross, LeftSemi, LeftAnti, LeftOuter, RightOuter" join type. ### Why are the changes needed? To optimize the skewed partition in runtime based on AQE ### Does this PR introduce any user-facing change? No ### How was this patch tested? UT Closes #26434 from JkSelf/skewedPartitionBasedSize. Lead-authored-by: jiake Co-authored-by: Wenchen Fan Co-authored-by: JiaKe Signed-off-by: Wenchen Fan --- .../org/apache/spark/MapOutputTracker.scala | 62 ++-- .../apache/spark/shuffle/ShuffleManager.scala | 10 +- .../shuffle/sort/SortShuffleManager.scala | 9 +- .../apache/spark/sql/internal/SQLConf.scala | 30 ++ .../spark/sql/execution/ShuffledRowRDD.scala | 23 +- .../adaptive/AdaptiveSparkPlanExec.scala | 4 + .../adaptive/LocalShuffledRowRDD.scala | 6 +- .../adaptive/OptimizeLocalShuffleReader.scala | 2 +- .../adaptive/OptimizeSkewedJoin.scala | 293 ++++++++++++++++++ .../adaptive/ReduceNumShufflePartitions.scala | 125 +++++--- .../adaptive/SkewedShuffledRowRDD.scala | 78 +++++ .../exchange/ShuffleExchangeExec.scala | 15 +- .../ReduceNumShufflePartitionsSuite.scala | 5 +- .../adaptive/AdaptiveQueryExecSuite.scala | 147 +++++++++ 14 files changed, 703 insertions(+), 106 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d2169246fe7e8..f229061a6d0f6 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -343,15 +343,18 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging /** * Called from executors to get the server URIs and output sizes for each shuffle block that * needs to be read from a given range of map output partitions (startPartition is included but - * endPartition is excluded from the range) and is produced by a specific mapper. + * endPartition is excluded from the range) and is produced by + * a range of mappers (startMapIndex, endMapIndex, startMapIndex is included and + * the endMapIndex is excluded). * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. */ - def getMapSizesByMapIndex( + def getMapSizesByRange( shuffleId: Int, - mapIndex: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] @@ -688,20 +691,25 @@ private[spark] class MapOutputTrackerMaster( } /** - * Return the location where the Mapper ran. The locations each includes both a host and an + * Return the locations where the Mappers ran. The locations each includes both a host and an * executor id on that host. * * @param dep shuffle dependency object - * @param mapId the map id + * @param startMapIndex the start map index + * @param endMapIndex the end map index * @return a sequence of locations where task runs. */ - def getMapLocation(dep: ShuffleDependency[_, _, _], mapId: Int): Seq[String] = + def getMapLocation( + dep: ShuffleDependency[_, _, _], + startMapIndex: Int, + endMapIndex: Int): Seq[String] = { val shuffleStatus = shuffleStatuses.get(dep.shuffleId).orNull if (shuffleStatus != null) { shuffleStatus.withMapStatuses { statuses => - if (mapId >= 0 && mapId < statuses.length) { - Seq(statuses(mapId).location.host) + if (startMapIndex < endMapIndex && (startMapIndex >= 0 && endMapIndex < statuses.length)) { + val statusesPicked = statuses.slice(startMapIndex, endMapIndex).filter(_ != null) + statusesPicked.map(_.location.host).toSeq } else { Nil } @@ -737,29 +745,26 @@ private[spark] class MapOutputTrackerMaster( case Some (shuffleStatus) => shuffleStatus.withMapStatuses { statuses => MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses) + shuffleId, startPartition, endPartition, statuses, 0, shuffleStatus.mapStatuses.length) } case None => Iterator.empty } } - override def getMapSizesByMapIndex( + override def getMapSizesByRange( shuffleId: Int, - mapIndex: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mapIndex $mapIndex" + + logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex-$endMapIndex" + s"partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { - case Some (shuffleStatus) => + case Some(shuffleStatus) => shuffleStatus.withMapStatuses { statuses => MapOutputTracker.convertMapStatuses( - shuffleId, - startPartition, - endPartition, - statuses, - Some(mapIndex)) + shuffleId, startPartition, endPartition, statuses, startMapIndex, endMapIndex) } case None => Iterator.empty @@ -802,7 +807,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr val statuses = getStatuses(shuffleId, conf) try { MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses) + shuffleId, startPartition, endPartition, statuses, 0, statuses.length) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -811,17 +816,18 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } - override def getMapSizesByMapIndex( + override def getMapSizesByRange( shuffleId: Int, - mapIndex: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - logDebug(s"Fetching outputs for shuffle $shuffleId, mapIndex $mapIndex" + + logDebug(s"Fetching outputs for shuffle $shuffleId, mappers $startMapIndex-$endMapIndex" + s"partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId, conf) try { - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, - statuses, Some(mapIndex)) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses, startMapIndex, endMapIndex) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -980,7 +986,8 @@ private[spark] object MapOutputTracker extends Logging { * @param startPartition Start of map output partition ID range (included in range) * @param endPartition End of map output partition ID range (excluded from range) * @param statuses List of map statuses, indexed by map partition index. - * @param mapIndex When specified, only shuffle blocks from this mapper will be processed. + * @param startMapIndex Start Map index. + * @param endMapIndex End Map index. * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. @@ -990,11 +997,12 @@ private[spark] object MapOutputTracker extends Logging { startPartition: Int, endPartition: Int, statuses: Array[MapStatus], - mapIndex : Option[Int] = None): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + startMapIndex : Int, + endMapIndex: Int): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] val iter = statuses.iterator.zipWithIndex - for ((status, mapIndex) <- mapIndex.map(index => iter.filter(_._2 == index)).getOrElse(iter)) { + for ((status, mapIndex) <- iter.slice(startMapIndex, endMapIndex)) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 01aa43eb9763d..057b0d6e0b0a7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -55,12 +55,14 @@ private[spark] trait ShuffleManager { metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] /** - * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) - * that are produced by one specific mapper. Called on executors by reduce tasks. + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive) to + * read from map output (startMapIndex to endMapIndex - 1, inclusive). + * Called on executors by reduce tasks. */ - def getReaderForOneMapper[K, C]( + def getReaderForRange[K, C]( handle: ShuffleHandle, - mapIndex: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int, context: TaskContext, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 3cd04de0f741c..aefcb59b8bb87 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -131,15 +131,16 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) } - override def getReaderForOneMapper[K, C]( + override def getReaderForRange[K, C]( handle: ShuffleHandle, - mapIndex: Int, + startMapIndex: Int, + endMapIndex: Int, startPartition: Int, endPartition: Int, context: TaskContext, metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { - val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByMapIndex( - handle.shuffleId, mapIndex, startPartition, endPartition) + val blocksByAddress = SparkEnv.get.mapOutputTracker.getMapSizesByRange( + handle.shuffleId, startMapIndex, endMapIndex, startPartition, endPartition) new BlockStoreShuffleReader( handle.asInstanceOf[BaseShuffleHandle[K, _, C]], blocksByAddress, context, metrics, shouldBatchFetch = canUseBatchFetch(startPartition, endPartition, context)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c2798d36d7769..6d45d30a787dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -422,6 +422,36 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED = + buildConf("spark.sql.adaptive.optimizeSkewedJoin.enabled") + .doc("When true and adaptive execution is enabled, a skewed join is automatically handled at " + + "runtime.") + .booleanConf + .createWithDefault(true) + + val ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD = + buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionSizeThreshold") + .doc("Configures the minimum size in bytes for a partition that is considered as a skewed " + + "partition in adaptive skewed join.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(64 * 1024 * 1024) + + val ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR = + buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionFactor") + .doc("A partition is considered as a skewed partition if its size is larger than" + + " this factor multiple the median partition size and also larger than " + + s" ${ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key}") + .intConf + .createWithDefault(10) + + val ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS = + buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionMaxSplits") + .doc("Configures the maximum number of task to handle a skewed partition in adaptive skewed" + + "join.") + .intConf + .checkValue( _ >= 1, "The split size at least be 1") + .createWithDefault(5) + val NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN = buildConf("spark.sql.adaptive.nonEmptyPartitionRatioForBroadcastJoin") .doc("The relation with a non-empty partition ratio lower than this config will not be " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index 4c19f95796d04..efa493923ccc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -116,7 +116,7 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A class ShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], metrics: Map[String, SQLMetric], - specifiedPartitionStartIndices: Option[Array[Int]] = None) + specifiedPartitionIndices: Option[Array[(Int, Int)]] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { if (SQLConf.get.fetchShuffleBlocksInBatchEnabled) { @@ -126,8 +126,8 @@ class ShuffledRowRDD( private[this] val numPreShufflePartitions = dependency.partitioner.numPartitions - private[this] val partitionStartIndices: Array[Int] = specifiedPartitionStartIndices match { - case Some(indices) => indices + private[this] val partitionStartIndices: Array[Int] = specifiedPartitionIndices match { + case Some(indices) => indices.map(_._1) case None => // When specifiedPartitionStartIndices is not defined, every post-shuffle partition // corresponds to a pre-shuffle partition. @@ -142,16 +142,15 @@ class ShuffledRowRDD( override val partitioner: Option[Partitioner] = Some(part) override def getPartitions: Array[Partition] = { - assert(partitionStartIndices.length == part.numPartitions) - Array.tabulate[Partition](partitionStartIndices.length) { i => - val startIndex = partitionStartIndices(i) - val endIndex = - if (i < partitionStartIndices.length - 1) { - partitionStartIndices(i + 1) - } else { - numPreShufflePartitions + specifiedPartitionIndices match { + case Some(indices) => + Array.tabulate[Partition](indices.length) { i => + new ShuffledRowRDDPartition(i, indices(i)._1, indices(i)._2) + } + case None => + Array.tabulate[Partition](numPreShufflePartitions) { i => + new ShuffledRowRDDPartition(i, i, i + 1) } - new ShuffledRowRDDPartition(i, startIndex, endIndex) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 7ff018989a2cc..fb43c8f9941ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -87,6 +87,10 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReuseAdaptiveSubquery(conf, context.subqueryCache), + // Here the 'OptimizeSkewedPartitions' rule should be executed + // before 'ReduceNumShufflePartitions', as the skewed partition handled + // in 'OptimizeSkewedPartitions' rule, should be omitted in 'ReduceNumShufflePartitions'. + OptimizeSkewedJoin(conf), ReduceNumShufflePartitions(conf), // The rule of 'OptimizeLocalShuffleReader' need to make use of the 'partitionStartIndices' // in 'ReduceNumShufflePartitions' rule. So it must be after 'ReduceNumShufflePartitions' rule. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala index 6385ea67c49fe..19b78f5e36c9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LocalShuffledRowRDD.scala @@ -82,7 +82,7 @@ class LocalShuffledRowRDD( override def getPreferredLocations(partition: Partition): Seq[String] = { val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - tracker.getMapLocation(dependency, partition.index) + tracker.getMapLocation(dependency, partition.index, partition.index + 1) } override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { @@ -92,9 +92,11 @@ class LocalShuffledRowRDD( // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, // as well as the `tempMetrics` for basic shuffle metrics. val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) - val reader = SparkEnv.get.shuffleManager.getReaderForOneMapper( + + val reader = SparkEnv.get.shuffleManager.getReaderForRange( dependency.shuffleHandle, mapIndex, + mapIndex + 1, localRowPartition.startPartition, localRowPartition.endPartition, context, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index e95441e28aafe..a8d8f358ab660 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -71,7 +71,7 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { plan match { case c @ CoalescedShuffleReaderExec(s: ShuffleQueryStageExec, _) => LocalShuffleReaderExec( - s, getPartitionStartIndices(s, Some(c.partitionStartIndices.length))) + s, getPartitionStartIndices(s, Some(c.partitionIndices.length))) case s: ShuffleQueryStageExec => LocalShuffleReaderExec(s, getPartitionStartIndices(s, None)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala new file mode 100644 index 0000000000000..75d4184a2c14e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -0,0 +1,293 @@ +/* + * 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.adaptive + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkEnv} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.internal.SQLConf + +case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { + + private val supportedJoinTypes = + Inner :: Cross :: LeftSemi :: LeftAnti :: LeftOuter :: RightOuter :: Nil + + /** + * A partition is considered as a skewed partition if its size is larger than the median + * partition size * spark.sql.adaptive.skewedPartitionFactor and also larger than + * spark.sql.adaptive.skewedPartitionSizeThreshold. + */ + private def isSkewed( + stats: MapOutputStatistics, + partitionId: Int, + medianSize: Long): Boolean = { + val size = stats.bytesByPartitionId(partitionId) + size > medianSize * conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR) && + size > conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD) + } + + private def medianSize(stats: MapOutputStatistics): Long = { + val numPartitions = stats.bytesByPartitionId.length + val bytes = stats.bytesByPartitionId.sorted + if (bytes(numPartitions / 2) > 0) bytes(numPartitions / 2) else 1 + } + + /** + * Get the map size of the specific reduce shuffle Id. + */ + private def getMapSizesForReduceId(shuffleId: Int, partitionId: Int): Array[Long] = { + val mapOutputTracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + mapOutputTracker.shuffleStatuses(shuffleId).mapStatuses.map{_.getSizeForBlock(partitionId)} + } + + /** + * Split the skewed partition based on the map size and the max split number. + */ + private def getMapStartIndices(stage: ShuffleQueryStageExec, partitionId: Int): Array[Int] = { + val shuffleId = stage.shuffle.shuffleDependency.shuffleHandle.shuffleId + val mapPartitionSizes = getMapSizesForReduceId(shuffleId, partitionId) + val maxSplits = math.min(conf.getConf( + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_MAX_SPLITS), mapPartitionSizes.length) + val avgPartitionSize = mapPartitionSizes.sum / maxSplits + val advisoryPartitionSize = math.max(avgPartitionSize, + conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD)) + val partitionIndices = mapPartitionSizes.indices + val partitionStartIndices = ArrayBuffer[Int]() + var postMapPartitionSize = mapPartitionSizes(0) + partitionStartIndices += 0 + partitionIndices.drop(1).foreach { nextPartitionIndex => + val nextMapPartitionSize = mapPartitionSizes(nextPartitionIndex) + if (postMapPartitionSize + nextMapPartitionSize > advisoryPartitionSize) { + partitionStartIndices += nextPartitionIndex + postMapPartitionSize = nextMapPartitionSize + } else { + postMapPartitionSize += nextMapPartitionSize + } + } + + if (partitionStartIndices.size > maxSplits) { + partitionStartIndices.take(maxSplits).toArray + } else partitionStartIndices.toArray + } + + private def getStatistics(stage: ShuffleQueryStageExec): MapOutputStatistics = { + assert(stage.resultOption.isDefined, "ShuffleQueryStageExec should" + + " already be ready when executing OptimizeSkewedPartitions rule") + stage.resultOption.get.asInstanceOf[MapOutputStatistics] + } + + private def supportSplitOnLeftPartition(joinType: JoinType) = { + joinType == Inner || joinType == Cross || joinType == LeftSemi || + joinType == LeftAnti || joinType == LeftOuter + } + + private def supportSplitOnRightPartition(joinType: JoinType) = { + joinType == Inner || joinType == Cross || joinType == RightOuter + } + + private def getNumMappers(stage: ShuffleQueryStageExec): Int = { + stage.shuffle.shuffleDependency.rdd.partitions.length + } + + def handleSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { + case smj @ SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, + s1 @ SortExec(_, _, left: ShuffleQueryStageExec, _), + s2 @ SortExec(_, _, right: ShuffleQueryStageExec, _)) + if supportedJoinTypes.contains(joinType) => + val leftStats = getStatistics(left) + val rightStats = getStatistics(right) + val numPartitions = leftStats.bytesByPartitionId.length + + val leftMedSize = medianSize(leftStats) + val rightMedSize = medianSize(rightStats) + val leftSizeInfo = s"median size: $leftMedSize, max size: ${leftStats.bytesByPartitionId.max}" + val rightSizeInfo = s"median size: $rightMedSize," + + s" max size: ${rightStats.bytesByPartitionId.max}" + logDebug( + s""" + |Try to optimize skewed join. + |Left side partition size: $leftSizeInfo + |Right side partition size: $rightSizeInfo + """.stripMargin) + + val skewedPartitions = mutable.HashSet[Int]() + val subJoins = mutable.ArrayBuffer[SparkPlan]() + for (partitionId <- 0 until numPartitions) { + val isLeftSkew = isSkewed(leftStats, partitionId, leftMedSize) + val isRightSkew = isSkewed(rightStats, partitionId, rightMedSize) + val leftMapIdStartIndices = if (isLeftSkew && supportSplitOnLeftPartition(joinType)) { + getMapStartIndices(left, partitionId) + } else { + Array(0) + } + val rightMapIdStartIndices = if (isRightSkew && supportSplitOnRightPartition(joinType)) { + getMapStartIndices(right, partitionId) + } else { + Array(0) + } + + if (leftMapIdStartIndices.length > 1 || rightMapIdStartIndices.length > 1) { + skewedPartitions += partitionId + for (i <- 0 until leftMapIdStartIndices.length; + j <- 0 until rightMapIdStartIndices.length) { + val leftEndMapId = if (i == leftMapIdStartIndices.length - 1) { + getNumMappers(left) + } else { + leftMapIdStartIndices(i + 1) + } + val rightEndMapId = if (j == rightMapIdStartIndices.length - 1) { + getNumMappers(right) + } else { + rightMapIdStartIndices(j + 1) + } + // TODO: we may can optimize the sort merge join to broad cast join after + // obtaining the raw data size of per partition, + val leftSkewedReader = SkewedPartitionReaderExec( + left, partitionId, leftMapIdStartIndices(i), leftEndMapId) + val rightSkewedReader = SkewedPartitionReaderExec(right, partitionId, + rightMapIdStartIndices(j), rightEndMapId) + subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, + s1.copy(child = leftSkewedReader), s2.copy(child = rightSkewedReader)) + } + } + } + logDebug(s"number of skewed partitions is ${skewedPartitions.size}") + if (skewedPartitions.nonEmpty) { + val optimizedSmj = smj.transformDown { + case sort @ SortExec(_, _, shuffleStage: ShuffleQueryStageExec, _) => + sort.copy(child = PartialShuffleReaderExec(shuffleStage, skewedPartitions.toSet)) + } + subJoins += optimizedSmj + UnionExec(subJoins) + } else { + smj + } + } + + override def apply(plan: SparkPlan): SparkPlan = { + if (!conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED)) { + return plan + } + + def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { + case _: LocalShuffleReaderExec => Nil + case _: CoalescedShuffleReaderExec => Nil + case stage: ShuffleQueryStageExec => Seq(stage) + case _ => plan.children.flatMap(collectShuffleStages) + } + + val shuffleStages = collectShuffleStages(plan) + + if (shuffleStages.length == 2) { + // Currently we only support handling skewed join for 2 table join. + handleSkewJoin(plan) + } else { + plan + + } + } +} + +/** + * A wrapper of shuffle query stage, which submits one reduce task to read a single + * shuffle partition 'partitionIndex' produced by the mappers in range [startMapIndex, endMapIndex). + * This is used to increase the parallelism when reading skewed partitions. + * + * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange + * node during canonicalization. + * @param partitionIndex The pre shuffle partition index. + * @param startMapIndex The start map index. + * @param endMapIndex The end map index. + */ +case class SkewedPartitionReaderExec( + child: QueryStageExec, + partitionIndex: Int, + startMapIndex: Int, + endMapIndex: Int) extends LeafExecNode { + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = { + UnknownPartitioning(1) + } + private var cachedSkewedShuffleRDD: SkewedShuffledRowRDD = null + + override def doExecute(): RDD[InternalRow] = { + if (cachedSkewedShuffleRDD == null) { + cachedSkewedShuffleRDD = child match { + case stage: ShuffleQueryStageExec => + stage.shuffle.createSkewedShuffleRDD(partitionIndex, startMapIndex, endMapIndex) + case _ => + throw new IllegalStateException("operating on canonicalization plan") + } + } + cachedSkewedShuffleRDD + } +} + +/** + * A wrapper of shuffle query stage, which skips some partitions when reading the shuffle blocks. + * + * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange node during + * canonicalization. + * @param excludedPartitions The partitions to skip when reading. + */ +case class PartialShuffleReaderExec( + child: QueryStageExec, + excludedPartitions: Set[Int]) extends UnaryExecNode { + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = { + UnknownPartitioning(1) + } + + private def shuffleExchange(): ShuffleExchangeExec = child match { + case stage: ShuffleQueryStageExec => stage.shuffle + case _ => + throw new IllegalStateException("operating on canonicalization plan") + } + + private def getPartitionIndexRanges(): Array[(Int, Int)] = { + val length = shuffleExchange().shuffleDependency.partitioner.numPartitions + (0 until length).filterNot(excludedPartitions.contains).map(i => (i, i + 1)).toArray + } + + private var cachedShuffleRDD: RDD[InternalRow] = null + + override def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = if (excludedPartitions.isEmpty) { + child.execute() + } else { + shuffleExchange().createShuffledRDD(Some(getPartitionIndexRanges())) + } + } + cachedShuffleRDD + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 3767e6cae46e5..2c50b638b4d45 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.MapOutputStatistics import org.apache.spark.rdd.RDD @@ -54,20 +54,28 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { if (!conf.reducePostShufflePartitionsEnabled) { return plan } - if (!plan.collectLeaves().forall(_.isInstanceOf[QueryStageExec])) { + // 'SkewedShufflePartitionReader' is added by us, so it's safe to ignore it when changing + // number of reducers. + val leafNodes = plan.collectLeaves().filter(!_.isInstanceOf[SkewedPartitionReaderExec]) + if (!leafNodes.forall(_.isInstanceOf[QueryStageExec])) { // If not all leaf nodes are query stages, it's not safe to reduce the number of // shuffle partitions, because we may break the assumption that all children of a spark plan // have same number of output partitions. return plan } - def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { + def collectShuffles(plan: SparkPlan): Seq[SparkPlan] = plan match { case _: LocalShuffleReaderExec => Nil + case p: PartialShuffleReaderExec => Seq(p) case stage: ShuffleQueryStageExec => Seq(stage) - case _ => plan.children.flatMap(collectShuffleStages) + case _ => plan.children.flatMap(collectShuffles) } - val shuffleStages = collectShuffleStages(plan) + val shuffles = collectShuffles(plan) + val shuffleStages = shuffles.map { + case PartialShuffleReaderExec(s: ShuffleQueryStageExec, _) => s + case s: ShuffleQueryStageExec => s + } // ShuffleExchanges introduced by repartition do not support changing the number of partitions. // We change the number of partitions in the stage only if all the ShuffleExchanges support it. if (!shuffleStages.forall(_.shuffle.canChangeNumPartitions)) { @@ -86,15 +94,32 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // partition) and a result of a SortMergeJoin (multiple partitions). val distinctNumPreShufflePartitions = validMetrics.map(stats => stats.bytesByPartitionId.length).distinct - if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { - val partitionStartIndices = estimatePartitionStartIndices(validMetrics.toArray) + val distinctExcludedPartitions = shuffles.map { + case PartialShuffleReaderExec(_, excludedPartitions) => excludedPartitions + case _: ShuffleQueryStageExec => Set.empty[Int] + }.distinct + if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1 + && distinctExcludedPartitions.length == 1) { + val excludedPartitions = distinctExcludedPartitions.head + val partitionIndices = estimatePartitionStartAndEndIndices( + validMetrics.toArray, excludedPartitions) // This transformation adds new nodes, so we must use `transformUp` here. - plan.transformUp { - // even for shuffle exchange whose input RDD has 0 partition, we should still update its - // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same - // number of output partitions. - case stage: ShuffleQueryStageExec => - CoalescedShuffleReaderExec(stage, partitionStartIndices) + // Even for shuffle exchange whose input RDD has 0 partition, we should still update its + // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same + // number of output partitions. + val visitedStages = HashSet.empty[Int] + plan.transformDown { + // Replace `PartialShuffleReaderExec` with `CoalescedShuffleReaderExec`, which keeps the + // "excludedPartition" requirement and also merges some partitions. + case PartialShuffleReaderExec(stage: ShuffleQueryStageExec, _) => + visitedStages.add(stage.id) + CoalescedShuffleReaderExec(stage, partitionIndices) + + // We are doing `transformDown`, so the `ShuffleQueryStageExec` may already be optimized + // and wrapped by `CoalescedShuffleReaderExec`. + case stage: ShuffleQueryStageExec if !visitedStages.contains(stage.id) => + visitedStages.add(stage.id) + CoalescedShuffleReaderExec(stage, partitionIndices) } } else { plan @@ -103,13 +128,15 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { } /** - * Estimates partition start indices for post-shuffle partitions based on - * mapOutputStatistics provided by all pre-shuffle stages. + * Estimates partition start and end indices for post-shuffle partitions based on + * mapOutputStatistics provided by all pre-shuffle stages and skip the omittedPartitions + * already handled in skewed partition optimization. */ // visible for testing. - private[sql] def estimatePartitionStartIndices( - mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = { - val minNumPostShufflePartitions = conf.minNumPostShufflePartitions + private[sql] def estimatePartitionStartAndEndIndices( + mapOutputStatistics: Array[MapOutputStatistics], + excludedPartitions: Set[Int] = Set.empty): Array[(Int, Int)] = { + val minNumPostShufflePartitions = conf.minNumPostShufflePartitions - excludedPartitions.size val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize // If minNumPostShufflePartitions is defined, it is possible that we need to use a // value less than advisoryTargetPostShuffleInputSize as the target input size of @@ -141,39 +168,35 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { distinctNumPreShufflePartitions.length == 1, "There should be only one distinct value of the number pre-shuffle partitions " + "among registered Exchange operator.") - val numPreShufflePartitions = distinctNumPreShufflePartitions.head val partitionStartIndices = ArrayBuffer[Int]() - // The first element of partitionStartIndices is always 0. - partitionStartIndices += 0 - - var postShuffleInputSize = 0L - - var i = 0 - while (i < numPreShufflePartitions) { - // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages. - // Then, we add the total size to postShuffleInputSize. - var nextShuffleInputSize = 0L - var j = 0 - while (j < mapOutputStatistics.length) { - nextShuffleInputSize += mapOutputStatistics(j).bytesByPartitionId(i) - j += 1 - } - - // If including the nextShuffleInputSize would exceed the target partition size, then start a - // new partition. - if (i > 0 && postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize) { - partitionStartIndices += i - // reset postShuffleInputSize. - postShuffleInputSize = nextShuffleInputSize - } else { - postShuffleInputSize += nextShuffleInputSize - } - - i += 1 + val partitionEndIndices = ArrayBuffer[Int]() + val numPartitions = distinctNumPreShufflePartitions.head + val includedPartitions = (0 until numPartitions).filter(!excludedPartitions.contains(_)) + val firstStartIndex = includedPartitions(0) + partitionStartIndices += firstStartIndex + var postShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(firstStartIndex)).sum + var i = firstStartIndex + includedPartitions.drop(1).foreach { nextPartitionIndex => + val nextShuffleInputSize = + mapOutputStatistics.map(_.bytesByPartitionId(nextPartitionIndex)).sum + // If nextPartitionIndices is skewed and omitted, or including + // the nextShuffleInputSize would exceed the target partition size, + // then start a new partition. + if (nextPartitionIndex != i + 1 || + (postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize)) { + partitionEndIndices += i + 1 + partitionStartIndices += nextPartitionIndex + // reset postShuffleInputSize. + postShuffleInputSize = nextShuffleInputSize + i = nextPartitionIndex + } else { + postShuffleInputSize += nextShuffleInputSize + i += 1 + } } - - partitionStartIndices.toArray + partitionEndIndices += i + 1 + partitionStartIndices.zip(partitionEndIndices).toArray } } @@ -186,12 +209,12 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { */ case class CoalescedShuffleReaderExec( child: SparkPlan, - partitionStartIndices: Array[Int]) extends UnaryExecNode { + partitionIndices: Array[(Int, Int)]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = { - UnknownPartitioning(partitionStartIndices.length) + UnknownPartitioning(partitionIndices.length) } private var cachedShuffleRDD: ShuffledRowRDD = null @@ -200,7 +223,7 @@ case class CoalescedShuffleReaderExec( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.shuffle.createShuffledRDD(Some(partitionStartIndices)) + stage.shuffle.createShuffledRDD(Some(partitionIndices)) case _ => throw new IllegalStateException("operating on canonicalization plan") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala new file mode 100644 index 0000000000000..52f793b24aa17 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala @@ -0,0 +1,78 @@ +/* + * 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.adaptive + +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} + +/** + * The [[Partition]] used by [[SkewedShuffledRowRDD]]. + */ +class SkewedShuffledRowRDDPartition(override val index: Int) extends Partition + +/** + * This is a specialized version of [[org.apache.spark.sql.execution.ShuffledRowRDD]]. This is used + * in Spark SQL adaptive execution to solve data skew issues. This RDD includes rearranged + * partitions from mappers. + * + * This RDD takes a [[ShuffleDependency]] (`dependency`), a partitionIndex + * and the range of startMapIndex to endMapIndex. + */ +class SkewedShuffledRowRDD( + var dependency: ShuffleDependency[Int, InternalRow, InternalRow], + partitionIndex: Int, + startMapIndex: Int, + endMapIndex: Int, + metrics: Map[String, SQLMetric]) + extends RDD[InternalRow](dependency.rdd.context, Nil) { + + override def getDependencies: Seq[Dependency[_]] = List(dependency) + + override def getPartitions: Array[Partition] = { + Array(new SkewedShuffledRowRDDPartition(0)) + } + + override def getPreferredLocations(partition: Partition): Seq[String] = { + val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + tracker.getMapLocation(dependency, startMapIndex, endMapIndex) + } + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() + // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, + // as well as the `tempMetrics` for basic shuffle metrics. + val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) + + val reader = SparkEnv.get.shuffleManager.getReaderForRange( + dependency.shuffleHandle, + startMapIndex, + endMapIndex, + partitionIndex, + partitionIndex + 1, + context, + sqlMetricsReporter) + reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) + } + + override def clearDependencies() { + super.clearDependencies() + dependency = null + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 4281f01e2756a..ffcd6c7783354 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Uns import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.LocalShuffledRowRDD +import org.apache.spark.sql.execution.adaptive.{LocalShuffledRowRDD, SkewedShuffledRowRDD} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -90,8 +90,9 @@ case class ShuffleExchangeExec( writeMetrics) } - def createShuffledRDD(partitionStartIndices: Option[Array[Int]]): ShuffledRowRDD = { - new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) + def createShuffledRDD( + partitionRanges: Option[Array[(Int, Int)]]): ShuffledRowRDD = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionRanges) } def createLocalShuffleRDD( @@ -99,6 +100,14 @@ case class ShuffleExchangeExec( new LocalShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndicesPerMapper) } + def createSkewedShuffleRDD( + partitionIndex: Int, + startMapIndex: Int, + endMapIndex: Int): SkewedShuffledRowRDD = { + new SkewedShuffledRowRDD(shuffleDependency, + partitionIndex, startMapIndex, endMapIndex, readMetrics) + } + /** * Caches the created ShuffleRowRDD so we can reuse that. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 70d5a8a615ad3..04b4d4f29f850 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -61,7 +61,7 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA new MapOutputStatistics(index, bytesByPartitionId) } val estimatedPartitionStartIndices = - rule.estimatePartitionStartIndices(mapOutputStatistics) + rule.estimatePartitionStartAndEndIndices(mapOutputStatistics).map(_._1) assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } @@ -133,7 +133,8 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA Array( new MapOutputStatistics(0, bytesByPartitionId1), new MapOutputStatistics(1, bytesByPartitionId2)) - intercept[AssertionError](rule.estimatePartitionStartIndices(mapOutputStatistics)) + intercept[AssertionError](rule.estimatePartitionStartAndEndIndices( + mapOutputStatistics)) } { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index fb24eaf2a4bf7..c2daae071afc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -579,6 +579,153 @@ class AdaptiveQueryExecSuite } } + test("SPARK-29544: adaptive skew join with different join types") { + Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", + SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled, + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "700") { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 2 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + // skewed inner join optimization + val (innerPlan, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 join skewData2 ON key1 = key2") + val innerSmj = findTopLevelSortMergeJoin(innerPlan) + assert(innerSmj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // the partition 0 in both left and right side are all skewed. + // And divide into 5 splits both in left and right (the max splits number). + // So there are 5 x 5 smjs for partition 0. + // Partition 4 in left side is skewed and is divided into 5 splits. + // The right side of partition 4 is not skewed. + // So there are 5 smjs for partition 4. + // So total (25 + 5 + 1) smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val innerSmjAfter = findTopLevelSortMergeJoin(innerAdaptivePlan) + assert(innerSmjAfter.size == 31) + + // skewed left outer join optimization + val (leftPlan, leftAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") + val leftSmj = findTopLevelSortMergeJoin(leftPlan) + assert(leftSmj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // The partition 0 in both left and right are all skewed. + // The partition 4 in left side is skewed. + // But for left outer join, we don't split the right partition even skewed. + // So the partition 0 in left side is divided into 5 splits(the max split number). + // the partition 4 in left side is divided into 5 splits(the max split number). + // So total (5 + 5 + 1) smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val leftSmjAfter = findTopLevelSortMergeJoin(leftAdaptivePlan) + assert(leftSmjAfter.size == 11) + + // skewed right outer join optimization + val (rightPlan, rightAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") + val rightSmj = findTopLevelSortMergeJoin(rightPlan) + assert(rightSmj.size == 1) + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // The partition 0 in both left and right side are all skewed. + // And the partition 4 in left side is skewed. + // But for right outer join, we don't split the left partition even skewed. + // And divide right side into 5 splits(the max split number) + // So total 6 smjs. + // Union + // +- SortMergeJoin + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- CoalescedShuffleReader + // +- ShuffleQueryStage + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // . + // . + // . + // +- SortMergeJoin + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + // +- Sort + // +- SkewedShuffleReader + // +- ShuffleQueryStage + + val rightSmjAfter = findTopLevelSortMergeJoin(rightAdaptivePlan) + assert(rightSmjAfter.size == 6) + } + } + } + } + test("SPARK-30291: AQE should catch the exceptions when doing materialize") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { From 176b69642e217883be47951654b37c86d7d82eaf Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Tue, 14 Jan 2020 22:07:13 +0800 Subject: [PATCH 1102/1387] [SPARK-30423][SQL] Deprecate UserDefinedAggregateFunction ### What changes were proposed in this pull request? * Annotate UserDefinedAggregateFunction as deprecated by SPARK-27296 * Update user doc examples to reflect new ability to register typed Aggregator[IN, BUF, OUT] as an untyped aggregating UDF ### Why are the changes needed? UserDefinedAggregateFunction is being deprecated ### Does this PR introduce any user-facing change? Changes are to user documentation, and deprecation annotations. ### How was this patch tested? Testing was via package build to verify doc generation, deprecation warnings, and successful example compilation. Closes #27193 from erikerlandson/spark-30423. Authored-by: Erik Erlandson Signed-off-by: Wenchen Fan --- docs/sql-getting-started.md | 23 ++-- .../JavaUserDefinedUntypedAggregation.java | 116 +++++++++--------- .../sql/UserDefinedUntypedAggregation.scala | 62 ++++------ .../apache/spark/sql/UDFRegistration.scala | 4 + .../apache/spark/sql/expressions/udaf.scala | 4 + 5 files changed, 105 insertions(+), 104 deletions(-) diff --git a/docs/sql-getting-started.md b/docs/sql-getting-started.md index 0ded2654719c1..f637697a29ad7 100644 --- a/docs/sql-getting-started.md +++ b/docs/sql-getting-started.md @@ -358,30 +358,29 @@ While those functions are designed for DataFrames, Spark SQL also has type-safe [Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. Moreover, users are not limited to the predefined aggregate functions and can create their own. -### Untyped User-Defined Aggregate Functions -Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) -abstract class to implement a custom untyped aggregate function. For example, a user-defined average -can look like: +### Type-Safe User-Defined Aggregate Functions + +User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. +For example, a type-safe user-defined average can look like:
      -{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} +{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%}
      -{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} +{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%}
      -### Type-Safe User-Defined Aggregate Functions - -User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. -For example, a type-safe user-defined average can look like: +### Untyped User-Defined Aggregate Functions +Typed aggregations, as described above, may also be registered as untyped aggregating UDFs for use with DataFrames. +For example, a user-defined average for untyped DataFrames can look like:
      -{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} +{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%}
      -{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%} +{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%}
      diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java index 6da60a1fc6b88..d300018845add 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java @@ -17,81 +17,85 @@ package org.apache.spark.examples.sql; // $example on:untyped_custom_aggregation$ -import java.util.ArrayList; -import java.util.List; +import java.io.Serializable; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.expressions.MutableAggregationBuffer; -import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.expressions.Aggregator; +import org.apache.spark.sql.functions; // $example off:untyped_custom_aggregation$ public class JavaUserDefinedUntypedAggregation { // $example on:untyped_custom_aggregation$ - public static class MyAverage extends UserDefinedAggregateFunction { + public static class Average implements Serializable { + private long sum; + private long count; - private StructType inputSchema; - private StructType bufferSchema; + // Constructors, getters, setters... + // $example off:typed_custom_aggregation$ + public Average() { + } + + public Average(long sum, long count) { + this.sum = sum; + this.count = count; + } - public MyAverage() { - List inputFields = new ArrayList<>(); - inputFields.add(DataTypes.createStructField("inputColumn", DataTypes.LongType, true)); - inputSchema = DataTypes.createStructType(inputFields); + public long getSum() { + return sum; + } - List bufferFields = new ArrayList<>(); - bufferFields.add(DataTypes.createStructField("sum", DataTypes.LongType, true)); - bufferFields.add(DataTypes.createStructField("count", DataTypes.LongType, true)); - bufferSchema = DataTypes.createStructType(bufferFields); + public void setSum(long sum) { + this.sum = sum; } - // Data types of input arguments of this aggregate function - public StructType inputSchema() { - return inputSchema; + + public long getCount() { + return count; } - // Data types of values in the aggregation buffer - public StructType bufferSchema() { - return bufferSchema; + + public void setCount(long count) { + this.count = count; } - // The data type of the returned value - public DataType dataType() { - return DataTypes.DoubleType; + // $example on:typed_custom_aggregation$ + } + + public static class MyAverage extends Aggregator { + // A zero value for this aggregation. Should satisfy the property that any b + zero = b + public Average zero() { + return new Average(0L, 0L); } - // Whether this function always returns the same output on the identical input - public boolean deterministic() { - return true; + // Combine two values to produce a new value. For performance, the function may modify `buffer` + // and return it instead of constructing a new object + public Average reduce(Average buffer, Long data) { + long newSum = buffer.getSum() + data; + long newCount = buffer.getCount() + 1; + buffer.setSum(newSum); + buffer.setCount(newCount); + return buffer; } - // Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to - // standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides - // the opportunity to update its values. Note that arrays and maps inside the buffer are still - // immutable. - public void initialize(MutableAggregationBuffer buffer) { - buffer.update(0, 0L); - buffer.update(1, 0L); + // Merge two intermediate values + public Average merge(Average b1, Average b2) { + long mergedSum = b1.getSum() + b2.getSum(); + long mergedCount = b1.getCount() + b2.getCount(); + b1.setSum(mergedSum); + b1.setCount(mergedCount); + return b1; } - // Updates the given aggregation buffer `buffer` with new input data from `input` - public void update(MutableAggregationBuffer buffer, Row input) { - if (!input.isNullAt(0)) { - long updatedSum = buffer.getLong(0) + input.getLong(0); - long updatedCount = buffer.getLong(1) + 1; - buffer.update(0, updatedSum); - buffer.update(1, updatedCount); - } + // Transform the output of the reduction + public Double finish(Average reduction) { + return ((double) reduction.getSum()) / reduction.getCount(); } - // Merges two aggregation buffers and stores the updated buffer values back to `buffer1` - public void merge(MutableAggregationBuffer buffer1, Row buffer2) { - long mergedSum = buffer1.getLong(0) + buffer2.getLong(0); - long mergedCount = buffer1.getLong(1) + buffer2.getLong(1); - buffer1.update(0, mergedSum); - buffer1.update(1, mergedCount); + // Specifies the Encoder for the intermediate value type + public Encoder bufferEncoder() { + return Encoders.bean(Average.class); } - // Calculates the final result - public Double evaluate(Row buffer) { - return ((double) buffer.getLong(0)) / buffer.getLong(1); + // Specifies the Encoder for the final output value type + public Encoder outputEncoder() { + return Encoders.DOUBLE(); } } // $example off:untyped_custom_aggregation$ @@ -104,7 +108,7 @@ public static void main(String[] args) { // $example on:untyped_custom_aggregation$ // Register the function to access it - spark.udf().register("myAverage", new MyAverage()); + spark.udf().register("myAverage", functions.udaf(new MyAverage(), Encoders.LONG())); Dataset df = spark.read().json("examples/src/main/resources/employees.json"); df.createOrReplaceTempView("employees"); diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala index 3656a84c571db..c7ab88806b7a5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala @@ -17,48 +17,38 @@ package org.apache.spark.examples.sql // $example on:untyped_custom_aggregation$ -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.expressions.MutableAggregationBuffer -import org.apache.spark.sql.expressions.UserDefinedAggregateFunction -import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Encoder, Encoders, SparkSession} +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.functions // $example off:untyped_custom_aggregation$ object UserDefinedUntypedAggregation { // $example on:untyped_custom_aggregation$ - object MyAverage extends UserDefinedAggregateFunction { - // Data types of input arguments of this aggregate function - def inputSchema: StructType = StructType(StructField("inputColumn", LongType) :: Nil) - // Data types of values in the aggregation buffer - def bufferSchema: StructType = { - StructType(StructField("sum", LongType) :: StructField("count", LongType) :: Nil) - } - // The data type of the returned value - def dataType: DataType = DoubleType - // Whether this function always returns the same output on the identical input - def deterministic: Boolean = true - // Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to - // standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides - // the opportunity to update its values. Note that arrays and maps inside the buffer are still - // immutable. - def initialize(buffer: MutableAggregationBuffer): Unit = { - buffer(0) = 0L - buffer(1) = 0L - } - // Updates the given aggregation buffer `buffer` with new input data from `input` - def update(buffer: MutableAggregationBuffer, input: Row): Unit = { - if (!input.isNullAt(0)) { - buffer(0) = buffer.getLong(0) + input.getLong(0) - buffer(1) = buffer.getLong(1) + 1 - } + case class Average(var sum: Long, var count: Long) + + object MyAverage extends Aggregator[Long, Average, Double] { + // A zero value for this aggregation. Should satisfy the property that any b + zero = b + def zero: Average = Average(0L, 0L) + // Combine two values to produce a new value. For performance, the function may modify `buffer` + // and return it instead of constructing a new object + def reduce(buffer: Average, data: Long): Average = { + buffer.sum += data + buffer.count += 1 + buffer } - // Merges two aggregation buffers and stores the updated buffer values back to `buffer1` - def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = { - buffer1(0) = buffer1.getLong(0) + buffer2.getLong(0) - buffer1(1) = buffer1.getLong(1) + buffer2.getLong(1) + // Merge two intermediate values + def merge(b1: Average, b2: Average): Average = { + b1.sum += b2.sum + b1.count += b2.count + b1 } - // Calculates the final result - def evaluate(buffer: Row): Double = buffer.getLong(0).toDouble / buffer.getLong(1) + // Transform the output of the reduction + def finish(reduction: Average): Double = reduction.sum.toDouble / reduction.count + // Specifies the Encoder for the intermediate value type + def bufferEncoder: Encoder[Average] = Encoders.product + // Specifies the Encoder for the final output value type + def outputEncoder: Encoder[Double] = Encoders.scalaDouble } // $example off:untyped_custom_aggregation$ @@ -70,7 +60,7 @@ object UserDefinedUntypedAggregation { // $example on:untyped_custom_aggregation$ // Register the function to access it - spark.udf.register("myAverage", MyAverage) + spark.udf.register("myAverage", functions.udaf(MyAverage)) val df = spark.read.json("examples/src/main/resources/employees.json") df.createOrReplaceTempView("employees") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index a4ff09596ad8e..0f08e10c00d22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -73,7 +73,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @return the registered UDAF. * * @since 1.5.0 + * @deprecated this method and the use of UserDefinedAggregateFunction are deprecated. + * Aggregator[IN, BUF, OUT] should now be registered as a UDF via the functions.udaf(agg) method. */ + @deprecated("Aggregator[IN, BUF, OUT] should now be registered as a UDF" + + " via the functions.udaf(agg) method.", "3.0.0") def register(name: String, udaf: UserDefinedAggregateFunction): UserDefinedAggregateFunction = { def builder(children: Seq[Expression]) = ScalaUDAF(children, udaf) functionRegistry.createOrReplaceTempFunction(name, builder) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index 4e8cb3a6ddd66..8407b1419af62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -27,8 +27,12 @@ import org.apache.spark.sql.types._ * The base class for implementing user-defined aggregate functions (UDAF). * * @since 1.5.0 + * @deprecated UserDefinedAggregateFunction is deprecated. + * Aggregator[IN, BUF, OUT] should now be registered as a UDF via the functions.udaf(agg) method. */ @Stable +@deprecated("Aggregator[IN, BUF, OUT] should now be registered as a UDF" + + " via the functions.udaf(agg) method.", "3.0.0") abstract class UserDefinedAggregateFunction extends Serializable { /** From 93200115d7650ced5bc56372fda96dde47bb9ceb Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Tue, 14 Jan 2020 08:25:51 -0600 Subject: [PATCH 1103/1387] [SPARK-9478][ML][PYSPARK] Add sample weights to Random Forest ### What changes were proposed in this pull request? 1, change `convertToBaggedRDDSamplingWithReplacement` to attach instance weights 2, make RF supports weights ### Why are the changes needed? `weightCol` is already exposed, while RF has not support weights. ### Does this PR introduce any user-facing change? Yes, new setters ### How was this patch tested? added testsuites Closes #27097 from zhengruifeng/rf_support_weight. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../RandomForestClassifier.scala | 26 +++++++++---- .../ml/regression/RandomForestRegressor.scala | 23 +++++++++-- .../spark/ml/tree/impl/BaggedPoint.scala | 6 ++- .../RandomForestClassifierSuite.scala | 35 +++++++++++++++++ .../RandomForestRegressorSuite.scala | 39 +++++++++++++++++++ .../spark/ml/tree/impl/BaggedPointSuite.scala | 3 +- python/pyspark/ml/classification.py | 24 ++++++++++-- python/pyspark/ml/regression.py | 28 +++++++++++-- 8 files changed, 161 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 7d36402a59372..de52740e7d981 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -21,7 +21,6 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ @@ -33,7 +32,6 @@ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} -import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.StructType @@ -69,6 +67,10 @@ class RandomForestClassifier @Since("1.4.0") ( @Since("1.4.0") def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + /** @group setParam */ + @Since("3.0.0") + def setMinWeightFractionPerNode(value: Double): this.type = set(minWeightFractionPerNode, value) + /** @group setParam */ @Since("1.4.0") def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) @@ -118,6 +120,16 @@ class RandomForestClassifier @Since("1.4.0") ( def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * By default the weightCol is not set, so all instances have weight 1.0. + * + * @group setParam + */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + override protected def train( dataset: Dataset[_]): RandomForestClassificationModel = instrumented { instr => instr.logPipelineStage(this) @@ -132,14 +144,14 @@ class RandomForestClassifier @Since("1.4.0") ( s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - val instances: RDD[Instance] = extractLabeledPoints(dataset, numClasses).map(_.toInstance) + val instances = extractInstances(dataset, numClasses) val strategy = super.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, getOldImpurity) - instr.logParams(this, labelCol, featuresCol, predictionCol, probabilityCol, rawPredictionCol, - leafCol, impurity, numTrees, featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, - minInfoGain, minInstancesPerNode, seed, subsamplingRate, thresholds, cacheNodeIds, - checkpointInterval) + instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, probabilityCol, + rawPredictionCol, leafCol, impurity, numTrees, featureSubsetStrategy, maxDepth, maxBins, + maxMemoryInMB, minInfoGain, minInstancesPerNode, minWeightFractionPerNode, seed, + subsamplingRate, thresholds, cacheNodeIds, checkpointInterval) val trees = RandomForest .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index e66109ef5777b..9d11f95816fb5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -64,6 +64,10 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S @Since("1.4.0") def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + /** @group setParam */ + @Since("3.0.0") + def setMinWeightFractionPerNode(value: Double): this.type = set(minWeightFractionPerNode, value) + /** @group setParam */ @Since("1.4.0") def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) @@ -113,20 +117,31 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * By default the weightCol is not set, so all instances have weight 1.0. + * + * @group setParam + */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + override protected def train( dataset: Dataset[_]): RandomForestRegressionModel = instrumented { instr => val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val instances = extractLabeledPoints(dataset).map(_.toInstance) + val instances = extractInstances(dataset) val strategy = super.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, getOldImpurity) instr.logPipelineStage(this) instr.logDataset(instances) - instr.logParams(this, labelCol, featuresCol, predictionCol, leafCol, impurity, numTrees, - featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, - minInstancesPerNode, seed, subsamplingRate, cacheNodeIds, checkpointInterval) + instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, leafCol, impurity, + numTrees, featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, + minInstancesPerNode, minWeightFractionPerNode, seed, subsamplingRate, cacheNodeIds, + checkpointInterval) val trees = RandomForest .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala index aac3dbf6c5a64..0ec7c6f03a11f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala @@ -65,7 +65,8 @@ private[spark] object BaggedPoint { seed: Long = Utils.random.nextLong()): RDD[BaggedPoint[Datum]] = { // TODO: implement weighted bootstrapping if (withReplacement) { - convertToBaggedRDDSamplingWithReplacement(input, subsamplingRate, numSubsamples, seed) + convertToBaggedRDDSamplingWithReplacement(input, subsamplingRate, numSubsamples, + extractSampleWeight, seed) } else { if (numSubsamples == 1 && subsamplingRate == 1.0) { convertToBaggedRDDWithoutSampling(input, extractSampleWeight) @@ -104,6 +105,7 @@ private[spark] object BaggedPoint { input: RDD[Datum], subsample: Double, numSubsamples: Int, + extractSampleWeight: (Datum => Double), seed: Long): RDD[BaggedPoint[Datum]] = { input.mapPartitionsWithIndex { (partitionIndex, instances) => // Use random seed = seed + partitionIndex + 1 to make generation reproducible. @@ -116,7 +118,7 @@ private[spark] object BaggedPoint { subsampleCounts(subsampleIndex) = poisson.sample() subsampleIndex += 1 } - new BaggedPoint(instance, subsampleCounts) + new BaggedPoint(instance, subsampleCounts, extractSampleWeight(instance)) } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index bf0f521cd28d2..b21ab81438d21 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LinearSVCSuite.generateSVMInput import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite @@ -41,6 +42,8 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { private var orderedLabeledPoints50_1000: RDD[LabeledPoint] = _ private var orderedLabeledPoints5_20: RDD[LabeledPoint] = _ + private var binaryDataset: DataFrame = _ + private val seed = 42 override def beforeAll(): Unit = { super.beforeAll() @@ -50,6 +53,7 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { orderedLabeledPoints5_20 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 5, 20)) .map(_.asML) + binaryDataset = generateSVMInput(0.01, Array[Double](-1.5, 1.0), 1000, seed).toDF() } ///////////////////////////////////////////////////////////////////////////// @@ -259,6 +263,37 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { }) } + test("training with sample weights") { + val df = binaryDataset + val numClasses = 2 + // (numTrees, maxDepth, subsamplingRate, fractionInTol) + val testParams = Seq( + (20, 5, 1.0, 0.96), + (20, 10, 1.0, 0.96), + (20, 10, 0.95, 0.96) + ) + + for ((numTrees, maxDepth, subsamplingRate, tol) <- testParams) { + val estimator = new RandomForestClassifier() + .setNumTrees(numTrees) + .setMaxDepth(maxDepth) + .setSubsamplingRate(subsamplingRate) + .setSeed(seed) + .setMinWeightFractionPerNode(0.049) + + MLTestingUtils.testArbitrarilyScaledWeights[RandomForestClassificationModel, + RandomForestClassifier](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, _ == _, tol)) + MLTestingUtils.testOutliersWithSmallWeights[RandomForestClassificationModel, + RandomForestClassifier](df.as[LabeledPoint], estimator, + numClasses, MLTestingUtils.modelPredictionEquals(df, _ == _, tol), + outlierRatio = 2) + MLTestingUtils.testOversamplingVsWeighting[RandomForestClassificationModel, + RandomForestClassifier](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, _ == _, tol), seed) + } + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index f3b0f0470e579..fff5cdd4ec50c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -22,9 +22,11 @@ import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.util.LinearDataGenerator import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row} @@ -37,12 +39,18 @@ class RandomForestRegressorSuite extends MLTest with DefaultReadWriteTest{ import testImplicits._ private var orderedLabeledPoints50_1000: RDD[LabeledPoint] = _ + private var linearRegressionData: DataFrame = _ + private val seed = 42 override def beforeAll(): Unit = { super.beforeAll() orderedLabeledPoints50_1000 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) .map(_.asML)) + + linearRegressionData = sc.parallelize(LinearDataGenerator.generateLinearInput( + intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), + xVariance = Array(0.7, 1.2), nPoints = 1000, seed, eps = 0.5), 2).map(_.asML).toDF() } ///////////////////////////////////////////////////////////////////////////// @@ -158,6 +166,37 @@ class RandomForestRegressorSuite extends MLTest with DefaultReadWriteTest{ }) } + test("training with sample weights") { + val df = linearRegressionData + val numClasses = 0 + // (numTrees, maxDepth, subsamplingRate, fractionInTol) + val testParams = Seq( + (50, 5, 1.0, 0.75), + (50, 10, 1.0, 0.75), + (50, 10, 0.95, 0.78) + ) + + for ((numTrees, maxDepth, subsamplingRate, tol) <- testParams) { + val estimator = new RandomForestRegressor() + .setNumTrees(numTrees) + .setMaxDepth(maxDepth) + .setSubsamplingRate(subsamplingRate) + .setSeed(seed) + .setMinWeightFractionPerNode(0.05) + + MLTestingUtils.testArbitrarilyScaledWeights[RandomForestRegressionModel, + RandomForestRegressor](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.2, tol)) + MLTestingUtils.testOutliersWithSmallWeights[RandomForestRegressionModel, + RandomForestRegressor](df.as[LabeledPoint], estimator, + numClasses, MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.2, tol), + outlierRatio = 2) + MLTestingUtils.testOversamplingVsWeighting[RandomForestRegressionModel, + RandomForestRegressor](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.2, tol), seed) + } + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala index 1f7a1ba816247..2a95faef98b63 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala @@ -54,8 +54,7 @@ class BaggedPointSuite extends SparkFunSuite with MLlibTestSparkContext { baggedRDD.map(_.subsampleCounts.map(_.toDouble)).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) - // should ignore weight function for now - assert(baggedRDD.collect().forall(_.sampleWeight === 1.0)) + assert(baggedRDD.collect().forall(_.sampleWeight === 2.0)) } } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 58044556b307d..d3b7fdd15b452 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1386,6 +1386,8 @@ class RandomForestClassifier(JavaProbabilisticClassifier, _RandomForestClassifie >>> td = si_model.transform(df) >>> rf = RandomForestClassifier(numTrees=3, maxDepth=2, labelCol="indexed", seed=42, ... leafCol="leafId") + >>> rf.getMinWeightFractionPerNode() + 0.0 >>> model = rf.fit(td) >>> model.getLabelCol() 'indexed' @@ -1440,14 +1442,14 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", numTrees=20, featureSubsetStrategy="auto", seed=None, subsamplingRate=1.0, - leafCol="", minWeightFractionPerNode=0.0): + leafCol="", minWeightFractionPerNode=0.0, weightCol=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", \ numTrees=20, featureSubsetStrategy="auto", seed=None, subsamplingRate=1.0, \ - leafCol="", minWeightFractionPerNode=0.0) + leafCol="", minWeightFractionPerNode=0.0, weightCol=None) """ super(RandomForestClassifier, self).__init__() self._java_obj = self._new_java_obj( @@ -1466,14 +1468,14 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, impurity="gini", numTrees=20, featureSubsetStrategy="auto", subsamplingRate=1.0, - leafCol="", minWeightFractionPerNode=0.0): + leafCol="", minWeightFractionPerNode=0.0, weightCol=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, \ impurity="gini", numTrees=20, featureSubsetStrategy="auto", subsamplingRate=1.0, \ - leafCol="", minWeightFractionPerNode=0.0) + leafCol="", minWeightFractionPerNode=0.0, weightCol=None) Sets params for linear classification. """ kwargs = self._input_kwargs @@ -1558,6 +1560,20 @@ def setCheckpointInterval(self, value): """ return self._set(checkpointInterval=value) + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + @since("3.0.0") + def setMinWeightFractionPerNode(self, value): + """ + Sets the value of :py:attr:`minWeightFractionPerNode`. + """ + return self._set(minWeightFractionPerNode=value) + class RandomForestClassificationModel(_TreeEnsembleModel, JavaProbabilisticClassificationModel, _RandomForestClassifierParams, JavaMLWritable, diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index d5a8dd5e8457a..7902eeefc3a98 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1012,6 +1012,8 @@ class RandomForestRegressor(JavaPredictor, _RandomForestRegressorParams, JavaMLW ... (1.0, Vectors.dense(1.0)), ... (0.0, Vectors.sparse(1, [], []))], ["label", "features"]) >>> rf = RandomForestRegressor(numTrees=2, maxDepth=2) + >>> rf.getMinWeightFractionPerNode() + 0.0 >>> rf.setSeed(42) RandomForestRegressor... >>> model = rf.fit(df) @@ -1061,13 +1063,15 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, - featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0): + featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, + weightCol=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, \ - featureSubsetStrategy="auto", leafCol=", minWeightFractionPerNode=0.0") + featureSubsetStrategy="auto", leafCol=", minWeightFractionPerNode=0.0", \ + weightCol=None) """ super(RandomForestRegressor, self).__init__() self._java_obj = self._new_java_obj( @@ -1085,13 +1089,15 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, - featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0): + featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, + weightCol=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, \ - featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0) + featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, \ + weightCol=None) Sets params for linear regression. """ kwargs = self._input_kwargs @@ -1176,6 +1182,20 @@ def setSeed(self, value): """ return self._set(seed=value) + @since("3.0.0") + def setWeightCol(self, value): + """ + Sets the value of :py:attr:`weightCol`. + """ + return self._set(weightCol=value) + + @since("3.0.0") + def setMinWeightFractionPerNode(self, value): + """ + Sets the value of :py:attr:`minWeightFractionPerNode`. + """ + return self._set(minWeightFractionPerNode=value) + class RandomForestRegressionModel(_TreeEnsembleModel, _RandomForestRegressorParams, JavaMLWritable, JavaMLReadable): From 0c6bd3bd0b95d17bc1eebb503269eda43df90394 Mon Sep 17 00:00:00 2001 From: Ajith Date: Tue, 14 Jan 2020 10:05:47 -0800 Subject: [PATCH 1104/1387] [SPARK-27142][SQL] Provide REST API for SQL information ### What changes were proposed in this pull request? Currently for Monitoring Spark application SQL information is not available from REST but only via UI. REST provides only applications,jobs,stages,environment. This Jira is targeted to provide a REST API so that SQL level information can be found A single SQL query can result into multiple jobs. So for end user who is using STS or spark-sql, the intended highest level of probe is the SQL which he has executed. This information can be seen from SQL tab. Attaching a sample. ![image](https://user-images.githubusercontent.com/22072336/54298729-5524a800-45df-11e9-8e4d-b99a8b882031.png) But same information he cannot access using the REST API exposed by spark and he always have to rely on jobs API which may be difficult. So i intend to expose the information seen in SQL tab in UI via REST API Mainly: Id : Long - execution id of the sql status : String - possible values COMPLETED/RUNNING/FAILED description : String - executed SQL string planDescription : String - Plan representation metrics : Seq[Metrics] - `Metrics` contain `metricName: String, metricValue: String` submissionTime : String - formatted `Date` time of SQL submission duration : Long - total run time in milliseconds runningJobIds : Seq[Int] - sequence of running job ids failedJobIds : Seq[Int] - sequence of failed job ids successJobIds : Seq[Int] - sequence of success job ids * To fetch sql executions: /sql?details=boolean&offset=integer&length=integer * To fetch single execution: /sql/{executionID}?details=boolean | parameter | type | remarks | | ------------- |:-------------:| -----| | details | boolean | Optional. Set true to get plan description and metrics information, defaults to false | | offset | integer | Optional. offset to fetch the executions, defaults to 0 | | length | integer | Optional. total number of executions to be fetched, defaults to 20 | ### Why are the changes needed? To support users query SQL information via REST API ### Does this PR introduce any user-facing change? Yes. It provides a new monitoring URL for SQL ### How was this patch tested? Tested manually ![image](https://user-images.githubusercontent.com/22072336/54282168-6d85ca00-45c1-11e9-8935-7586ccf0efff.png) ![image](https://user-images.githubusercontent.com/22072336/54282191-7b3b4f80-45c1-11e9-941c-f0ec37026192.png) Closes #24076 from ajithme/restapi. Lead-authored-by: Ajith Co-authored-by: Gengliang Wang Signed-off-by: Marcelo Vanzin --- .../sql/execution/ui/SQLAppStatusStore.scala | 4 + .../api/v1/sql/ApiSqlRootResource.scala | 29 +++++ .../spark/status/api/v1/sql/SqlResource.scala | 101 ++++++++++++++++++ .../apache/spark/status/api/v1/sql/api.scala | 33 ++++++ 4 files changed, 167 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/ApiSqlRootResource.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/SqlResource.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/api.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala index c6e7f3978469d..a90f37a80d525 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusStore.scala @@ -42,6 +42,10 @@ class SQLAppStatusStore( store.view(classOf[SQLExecutionUIData]).asScala.toSeq } + def executionsList(offset: Int, length: Int): Seq[SQLExecutionUIData] = { + store.view(classOf[SQLExecutionUIData]).skip(offset).max(length).asScala.toSeq + } + def execution(executionId: Long): Option[SQLExecutionUIData] = { try { Some(store.read(classOf[SQLExecutionUIData], executionId)) diff --git a/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/ApiSqlRootResource.scala b/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/ApiSqlRootResource.scala new file mode 100644 index 0000000000000..5fc7123c9097b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/ApiSqlRootResource.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1.sql + +import javax.ws.rs.Path + +import org.apache.spark.status.api.v1.ApiRequestContext + +@Path("/v1") +private[v1] class ApiSqlRootResource extends ApiRequestContext { + + @Path("applications/{appId}/sql") + def sqlList(): Class[SqlResource] = classOf[SqlResource] +} diff --git a/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/SqlResource.scala b/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/SqlResource.scala new file mode 100644 index 0000000000000..346e07f2bef15 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/SqlResource.scala @@ -0,0 +1,101 @@ +/* + * 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.status.api.v1.sql + +import java.util.Date +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.sql.execution.ui.{SQLAppStatusStore, SQLExecutionUIData, SQLPlanMetric} +import org.apache.spark.status.api.v1.{BaseAppResource, NotFoundException} + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class SqlResource extends BaseAppResource { + + @GET + def sqlList( + @DefaultValue("false") @QueryParam("details") details: Boolean, + @DefaultValue("0") @QueryParam("offset") offset: Int, + @DefaultValue("20") @QueryParam("length") length: Int): Seq[ExecutionData] = { + withUI { ui => + val sqlStore = new SQLAppStatusStore(ui.store.store) + sqlStore.executionsList(offset, length).map(prepareExecutionData(_, details)) + } + } + + @GET + @Path("{executionId:\\d+}") + def sql( + @PathParam("executionId") execId: Long, + @DefaultValue("false") @QueryParam("details") details: Boolean): ExecutionData = { + withUI { ui => + val sqlStore = new SQLAppStatusStore(ui.store.store) + sqlStore + .execution(execId) + .map(prepareExecutionData(_, details)) + .getOrElse(throw new NotFoundException("unknown id: " + execId)) + } + } + + private def printableMetrics( + metrics: Seq[SQLPlanMetric], + metricValues: Map[Long, String]): Seq[Metrics] = { + metrics.map(metric => + Metrics(metric.name, metricValues.get(metric.accumulatorId).getOrElse(""))) + } + + private def prepareExecutionData(exec: SQLExecutionUIData, details: Boolean): ExecutionData = { + var running = Seq[Int]() + var completed = Seq[Int]() + var failed = Seq[Int]() + + exec.jobs.foreach { + case (id, JobExecutionStatus.RUNNING) => + running = running :+ id + case (id, JobExecutionStatus.SUCCEEDED) => + completed = completed :+ id + case (id, JobExecutionStatus.FAILED) => + failed = failed :+ id + case _ => + } + + val status = if (exec.jobs.size == completed.size) { + "COMPLETED" + } else if (failed.nonEmpty) { + "FAILED" + } else { + "RUNNING" + } + + val duration = exec.completionTime.getOrElse(new Date()).getTime - exec.submissionTime + val planDetails = if (details) exec.physicalPlanDescription else "" + val metrics = if (details) printableMetrics(exec.metrics, exec.metricValues) else Seq.empty + new ExecutionData( + exec.executionId, + status, + exec.description, + planDetails, + metrics, + new Date(exec.submissionTime), + duration, + running, + completed, + failed) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/api.scala b/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/api.scala new file mode 100644 index 0000000000000..7ace66ffb06e1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/status/api/v1/sql/api.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1.sql + +import java.util.Date + +class ExecutionData private[spark] ( + val id: Long, + val status: String, + val description: String, + val planDescription: String, + val metrics: Seq[Metrics], + val submissionTime: Date, + val duration: Long, + val runningJobIds: Seq[Int], + val successJobIds: Seq[Int], + val failedJobIds: Seq[Int]) + +case class Metrics private[spark] (metricName: String, metricValue: String) From db7262a00fc558551e943201467424ac74ded929 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 14 Jan 2020 11:48:07 -0800 Subject: [PATCH 1105/1387] [SPARK-30509][SQL] Fix deprecation log warning in Avro schema inferring ### What changes were proposed in this pull request? In the PR, I propose to check the `ignoreExtensionKey` option in the case insensitive map of `AvroOption`. ### Why are the changes needed? The map `options` passed to `AvroUtils.inferSchema` contains all keys in the lower cases in fact. Actually, the map is converted from a `CaseInsensitiveStringMap`. Consequently, the check https://github.com/apache/spark/blob/3663dbe541826949cecf5e1ea205fe35c163d147/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala#L45 always return `false`, and the deprecation log warning is never printed. ### Does this PR introduce any user-facing change? Yes, after the changes the log warning is printed once. ### How was this patch tested? Added new test to `AvroSuite` which checks existence of log warning. Closes #27200 from MaxGekk/avro-fix-ignoreExtension-contains. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/avro/AvroUtils.scala | 9 +++---- .../org/apache/spark/sql/avro/AvroSuite.scala | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala index b978b7974b92d..70dcd58a600fc 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.avro.AvroOptions.ignoreExtensionKey import org.apache.spark.sql.execution.datasources.OutputWriterFactory import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -42,12 +43,12 @@ object AvroUtils extends Logging { options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { val conf = spark.sessionState.newHadoopConf() - if (options.contains("ignoreExtension")) { - logWarning(s"Option ${AvroOptions.ignoreExtensionKey} is deprecated. Please use the " + - "general data source option pathGlobFilter for filtering file names.") - } val parsedOptions = new AvroOptions(options, conf) + if (parsedOptions.parameters.contains(ignoreExtensionKey)) { + logWarning(s"Option $ignoreExtensionKey is deprecated. Please use the " + + "general data source option pathGlobFilter for filtering file names.") + } // User can specify an optional avro json schema. val avroSchema = parsedOptions.schema .map(new Schema.Parser().parse) diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 3f2744014c199..d00065c828417 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1497,6 +1497,30 @@ abstract class AvroSuite extends QueryTest with SharedSparkSession { |} """.stripMargin) } + + test("log a warning of ignoreExtension deprecation") { + val logAppender = new LogAppender + withTempPath { dir => + Seq(("a", 1, 2), ("b", 1, 2), ("c", 2, 1), ("d", 2, 1)) + .toDF("value", "p1", "p2") + .repartition(2) + .write + .format("avro") + .save(dir.getCanonicalPath) + withLogAppender(logAppender) { + spark + .read + .format("avro") + .option(AvroOptions.ignoreExtensionKey, false) + .load(dir.getCanonicalPath) + .count() + } + val deprecatedEvents = logAppender.loggingEvents + .filter(_.getRenderedMessage.contains( + s"Option ${AvroOptions.ignoreExtensionKey} is deprecated")) + assert(deprecatedEvents.size === 1) + } + } } class AvroV1Suite extends AvroSuite { From 0f3d744c3f19750ab03eeae3606e122dcffae5df Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 14 Jan 2020 11:49:50 -0800 Subject: [PATCH 1106/1387] [MINOR][TESTS] Remove unsupported `header` option in AvroSuite ### What changes were proposed in this pull request? In the PR, I propose to remove the `header` option in the `Avro source v2: support partition pruning` test. ### Why are the changes needed? The option is not supported by Avro, and may misleading readers. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By `AvroSuite`. Closes #27203 from MaxGekk/avro-suite-remove-header-option. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index d00065c828417..b349ac57892db 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1545,12 +1545,10 @@ class AvroV2Suite extends AvroSuite { .write .format("avro") .partitionBy("p1", "p2") - .option("header", true) .save(dir.getCanonicalPath) val df = spark .read .format("avro") - .option("header", true) .load(dir.getCanonicalPath) .where("p1 = 1 and p2 = 2 and value != \"a\"") From 3668291e6bc1e6ef20f4545689551cffef0ed023 Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 15 Jan 2020 12:29:23 +0800 Subject: [PATCH 1107/1387] [SPARK-30452][ML][PYSPARK][FOLLOWUP] Change IsotonicRegressionModel.numFeatures to property ### What changes were proposed in this pull request? Change `IsotonicRegressionModel.numFeatures` from plain method to property. ### Why are the changes needed? Consistency. Right now we use `numFeatures` in two other places in `pyspark.ml` https://github.com/apache/spark/blob/0f3d744c3f19750ab03eeae3606e122dcffae5df/python/pyspark/ml/feature.py#L4289-L4291 https://github.com/apache/spark/blob/0f3d744c3f19750ab03eeae3606e122dcffae5df/python/pyspark/ml/wrapper.py#L437-L439 and one in `pyspark,mllib` https://github.com/apache/spark/blob/0f3d744c3f19750ab03eeae3606e122dcffae5df/python/pyspark/mllib/classification.py#L177-L179 each time as a property. Additionally all similar values in `ml` are exposed as properties, for example https://github.com/apache/spark/blob/0f3d744c3f19750ab03eeae3606e122dcffae5df/python/pyspark/ml/regression.py#L451-L453 ### Does this PR introduce any user-facing change? Yes, but current API hasn't been released yet. ### How was this patch tested? Existing doctests. Closes #27206 from zero323/SPARK-30452-FOLLOWUP. Authored-by: zero323 Signed-off-by: zhengruifeng --- python/pyspark/ml/regression.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 7902eeefc3a98..6f01d54368353 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -593,7 +593,7 @@ class IsotonicRegression(JavaEstimator, _IsotonicRegressionParams, HasWeightCol, >>> model = ir.fit(df) >>> model.setFeaturesCol("features") IsotonicRegressionModel... - >>> model.numFeatures() + >>> model.numFeatures 1 >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"]) >>> model.transform(test0).head().prediction @@ -731,6 +731,7 @@ def predictions(self): """ return self._call_java("predictions") + @property @since("3.0.0") def numFeatures(self): """ From 51d29175ab406e8ceac514af76f1d97d1f332ffa Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 15 Jan 2020 16:41:26 +0900 Subject: [PATCH 1108/1387] [SPARK-30505][DOCS] Deprecate Avro option `ignoreExtension` in sql-data-sources-avro.md ### What changes were proposed in this pull request? Updated `docs/sql-data-sources-avro.md`, and added a few sentences about already deprecated in code Avro option `ignoreExtension`. Screen Shot 2020-01-15 at 10 24 14 Closes #27174 ### Why are the changes needed? To make users doc consistent to the code where `ignoreExtension` has been already deprecated, see https://github.com/apache/spark/blob/3663dbe541826949cecf5e1ea205fe35c163d147/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala#L46-L47 ### Does this PR introduce any user-facing change? No ### How was this patch tested? by building docs Closes #27194 from MaxGekk/avro-doc-deprecation-ignoreExtension. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- docs/sql-data-sources-avro.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md index dc534abce28e9..b0076878e02da 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/sql-data-sources-avro.md @@ -230,7 +230,7 @@ Data source options of Avro can be set via:
      ignoreExtension trueThe option controls ignoring of files without .avro extensions in read.
      If the option is enabled, all files (with and without .avro extension) are loaded.
      The option controls ignoring of files without .avro extensions in read.
      If the option is enabled, all files (with and without .avro extension) are loaded.
      The option has been deprecated, and it will be removed in the future releases. Please use the general data source option pathGlobFilter for filtering file names.
      read
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      + Property(case sensitive) + + Database Reserved + + Table Reserved + + Remarks +
      + provider + + no + + yes + + For tables, please use the USING clause to specify it. Once set, it can't be changed. +
      + location + + yes + + yes + + For databases and tables, please use the LOCATION clause to specify it. +
      + ownerName + + yes + + no + + For databases, please use the ALTER DATABASE ... SET OWNER syntax to modify it. +
      + ownerType + + yes + + no + + For databases, please use the ALTER DATABASE ... SET OWNER syntax to modify it. +
      - Since Spark 3.0, `ADD FILE` can be used to add file directories as well. Earlier only single files can be added using this command. To restore the behaviour of earlier versions, set `spark.sql.legacy.addDirectory.recursive` to false. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index b73ba01389067..b79dcd0c1ecad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -139,7 +139,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), - writeOptions = c.options.filterKeys(_ != "path"), + writeOptions = c.options, ignoreIfExists = c.ifNotExists) case RefreshTableStatement(NonSessionCatalogAndTable(catalog, tbl)) => @@ -165,7 +165,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), - writeOptions = c.options.filterKeys(_ != "path"), + writeOptions = c.options, orCreate = c.orCreate) case DropTableStatement(NonSessionCatalogAndTable(catalog, tbl), ifExists, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index e326e48da6581..5ddeff50e47b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit -import org.apache.spark.sql.connector.catalog.SupportsNamespaces +import org.apache.spark.sql.connector.catalog.{SupportsNamespaces, TableCatalog} import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.internal.SQLConf @@ -2399,7 +2399,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging /** * Type to keep track of table clauses: - * (partitioning, bucketSpec, options, locationSpec, properties, comment). + * (partitioning, bucketSpec, properties, options, location, comment). */ type TableClauses = (Seq[Transform], Option[BucketSpec], Map[String, String], Map[String, String], Option[String], Option[String]) @@ -2527,22 +2527,20 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging properties: Map[String, String], ctx: ParserRuleContext): Map[String, String] = withOrigin(ctx) { import SupportsNamespaces._ - if (!conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED)) { - properties.foreach { - case (PROP_LOCATION, _) => - throw new ParseException(s"$PROP_LOCATION is a reserved namespace property, please use" + - s" the LOCATION clause to specify it.", ctx) - case (PROP_COMMENT, _) => - throw new ParseException(s"$PROP_COMMENT is a reserved namespace property, please use" + - s" the COMMENT clause to specify it.", ctx) - case (ownership, _) if ownership == PROP_OWNER_NAME || ownership == PROP_OWNER_TYPE => + val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) + properties.filter { + case (PROP_LOCATION, _) if !legacyOn => + throw new ParseException(s"$PROP_LOCATION is a reserved namespace property, please use" + + s" the LOCATION clause to specify it.", ctx) + case (PROP_LOCATION, _) => false + case (ownership, _) if ownership == PROP_OWNER_NAME || ownership == PROP_OWNER_TYPE => + if (legacyOn) { + false + } else { throw new ParseException(s"$ownership is a reserved namespace property , please use" + " ALTER NAMESPACE ... SET OWNER ... to specify it.", ctx) - case _ => - } - properties - } else { - properties -- RESERVED_PROPERTIES.asScala + } + case _ => true } } @@ -2669,6 +2667,41 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx.EXTENDED != null) } + def cleanTableProperties( + ctx: ParserRuleContext, properties: Map[String, String]): Map[String, String] = { + import TableCatalog._ + val legacyOn = conf.getConf(SQLConf.LEGACY_PROPERTY_NON_RESERVED) + properties.filter { + case (PROP_PROVIDER, _) if !legacyOn => + throw new ParseException(s"$PROP_PROVIDER is a reserved table property, please use" + + s" the USING clause to specify it.", ctx) + case (PROP_PROVIDER, _) => false + case (PROP_LOCATION, _) if !legacyOn => + throw new ParseException(s"$PROP_LOCATION is a reserved table property, please use" + + s" the LOCATION clause to specify it.", ctx) + case (PROP_LOCATION, _) => false + case _ => true + } + } + + def cleanTableOptions( + ctx: ParserRuleContext, + options: Map[String, String], + location: Option[String]): (Map[String, String], Option[String]) = { + var path = location + val filtered = cleanTableProperties(ctx, options).filter { + case (k, v) if k.equalsIgnoreCase("path") && path.nonEmpty => + throw new ParseException(s"Duplicated table paths found: '${path.get}' and '$v'. LOCATION" + + s" and the case insensitive key 'path' in OPTIONS are all used to indicate the custom" + + s" table path, you can only specify one of them.", ctx) + case (k, v) if k.equalsIgnoreCase("path") => + path = Some(v) + false + case _ => true + } + (filtered, path) + } + override def visitCreateTableClauses(ctx: CreateTableClausesContext): TableClauses = { checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) @@ -2681,10 +2714,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Option(ctx.partitioning).map(visitTransformList).getOrElse(Nil) val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) + val cleanedProperties = cleanTableProperties(ctx, properties) val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) val location = visitLocationSpecList(ctx.locationSpec()) + val (cleanedOptions, newLocation) = cleanTableOptions(ctx, options, location) val comment = visitCommentSpecList(ctx.commentSpec()) - (partitioning, bucketSpec, properties, options, location, comment) + (partitioning, bucketSpec, cleanedProperties, cleanedOptions, newLocation, comment) } /** @@ -3010,18 +3045,19 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * * For example: * {{{ - * ALTER TABLE table SET TBLPROPERTIES ('comment' = new_comment); - * ALTER VIEW view SET TBLPROPERTIES ('comment' = new_comment); + * ALTER TABLE table SET TBLPROPERTIES ('table_property' = 'property_value'); + * ALTER VIEW view SET TBLPROPERTIES ('table_property' = 'property_value'); * }}} */ override def visitSetTableProperties( ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { val identifier = visitMultipartIdentifier(ctx.multipartIdentifier) val properties = visitPropertyKeyValues(ctx.tablePropertyList) + val cleanedTableProperties = cleanTableProperties(ctx, properties) if (ctx.VIEW != null) { - AlterViewSetPropertiesStatement(identifier, properties) + AlterViewSetPropertiesStatement(identifier, cleanedTableProperties) } else { - AlterTableSetPropertiesStatement(identifier, properties) + AlterTableSetPropertiesStatement(identifier, cleanedTableProperties) } } @@ -3038,11 +3074,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { val identifier = visitMultipartIdentifier(ctx.multipartIdentifier) val properties = visitPropertyKeys(ctx.tablePropertyList) + val cleanedProperties = cleanTableProperties(ctx, properties.map(_ -> "").toMap).keys.toSeq + val ifExists = ctx.EXISTS != null if (ctx.VIEW != null) { - AlterViewUnsetPropertiesStatement(identifier, properties, ifExists) + AlterViewUnsetPropertiesStatement(identifier, cleanedProperties, ifExists) } else { - AlterTableUnsetPropertiesStatement(identifier, properties, ifExists) + AlterTableUnsetPropertiesStatement(identifier, cleanedProperties, ifExists) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index b28e3e6bc27d4..f8fc4bc7ce23f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -21,9 +21,7 @@ import java.util import java.util.Collections import scala.collection.JavaConverters._ -import scala.collection.mutable -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.plans.logical.AlterTable import org.apache.spark.sql.connector.catalog.TableChange._ @@ -271,40 +269,11 @@ private[sql] object CatalogV2Util { location: Option[String], comment: Option[String], provider: String): Map[String, String] = { - if (options.contains("path") && location.isDefined) { - throw new AnalysisException( - "LOCATION and 'path' in OPTIONS are both used to indicate the custom table path, " + - "you can only specify one of them.") - } - - if ((options.contains(TableCatalog.PROP_COMMENT) - || properties.contains(TableCatalog.PROP_COMMENT)) && comment.isDefined) { - throw new AnalysisException( - s"COMMENT and option/property '${TableCatalog.PROP_COMMENT}' " + - s"are both used to set the table comment, you can only specify one of them.") - } - - if (options.contains(TableCatalog.PROP_PROVIDER) - || properties.contains(TableCatalog.PROP_PROVIDER)) { - throw new AnalysisException( - "USING and option/property 'provider' are both used to set the provider implementation, " + - "you can only specify one of them.") - } - - val filteredOptions = options.filterKeys(_ != "path") - - // create table properties from TBLPROPERTIES and OPTIONS clauses - val tableProperties = new mutable.HashMap[String, String]() - tableProperties ++= properties - tableProperties ++= filteredOptions - - // convert USING, LOCATION, and COMMENT clauses to table properties - tableProperties += (TableCatalog.PROP_PROVIDER -> provider) - comment.map(text => tableProperties += (TableCatalog.PROP_COMMENT -> text)) - location.orElse(options.get("path")).map( - loc => tableProperties += (TableCatalog.PROP_LOCATION -> loc)) - - tableProperties.toMap + properties ++ + options ++ + Map(TableCatalog.PROP_PROVIDER -> provider) ++ + comment.map(TableCatalog.PROP_COMMENT -> _) ++ + location.map(TableCatalog.PROP_LOCATION -> _) } def createAlterTable( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index f0c87812daff8..106fdc4333960 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} @@ -271,7 +271,7 @@ class ResolveSessionCatalog( c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), - writeOptions = c.options.filterKeys(_ != "path"), + writeOptions = c.options, ignoreIfExists = c.ifNotExists) } @@ -307,7 +307,7 @@ class ResolveSessionCatalog( c.partitioning ++ c.bucketSpec.map(_.asTransform), c.asSelect, convertTableProperties(c.properties, c.options, c.location, c.comment, c.provider), - writeOptions = c.options.filterKeys(_ != "path"), + writeOptions = c.options, orCreate = c.orCreate) } @@ -553,16 +553,11 @@ class ResolveSessionCatalog( location: Option[String], comment: Option[String], ifNotExists: Boolean): CatalogTable = { + val storage = CatalogStorageFormat.empty.copy( + locationUri = location.map(CatalogUtils.stringToURI), + properties = options) - val storage = DataSource.buildStorageFormatFromOptions(options) - if (location.isDefined && storage.locationUri.isDefined) { - throw new AnalysisException( - "LOCATION and 'path' in OPTIONS are both used to indicate the custom table path, " + - "you can only specify one of them.") - } - val customLocation = storage.locationUri.orElse(location.map(CatalogUtils.stringToURI)) - - val tableType = if (customLocation.isDefined) { + val tableType = if (location.isDefined) { CatalogTableType.EXTERNAL } else { CatalogTableType.MANAGED @@ -571,7 +566,7 @@ class ResolveSessionCatalog( CatalogTable( identifier = table, tableType = tableType, - storage = storage.copy(locationUri = customLocation), + storage = storage, schema = schema, provider = Some(provider), partitionColumnNames = partitioning.asPartitionColumns, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index dcc5aec83e713..d75b39610a754 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter @@ -277,7 +278,7 @@ case class AlterTableSetPropertiesCommand( // direct property. val newTable = table.copy( properties = table.properties ++ properties, - comment = properties.get("comment").orElse(table.comment)) + comment = properties.get(TableCatalog.PROP_COMMENT).orElse(table.comment)) catalog.alterTable(newTable) Seq.empty[Row] } @@ -306,14 +307,14 @@ case class AlterTableUnsetPropertiesCommand( DDLUtils.verifyAlterTableType(catalog, table, isView) if (!ifExists) { propKeys.foreach { k => - if (!table.properties.contains(k) && k != "comment") { + if (!table.properties.contains(k) && k != TableCatalog.PROP_COMMENT) { throw new AnalysisException( s"Attempted to unset non-existent property '$k' in table '${table.identifier}'") } } } // If comment is in the table property, we reset it to None - val tableComment = if (propKeys.contains("comment")) None else table.comment + val tableComment = if (propKeys.contains(TableCatalog.PROP_COMMENT)) None else table.comment val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) } val newTable = table.copy(properties = newProperties, comment = tableComment) catalog.alterTable(newTable) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 5a7774abf0926..4ee8a6803ea5e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -874,8 +874,9 @@ class DataSourceV2SQLSuite } test("CreateNameSpace: reserved properties") { + import SupportsNamespaces._ withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { key => + RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => val exception = intercept[ParseException] { sql(s"CREATE NAMESPACE testcat.reservedTest WITH DBPROPERTIES('$key'='dummyVal')") } @@ -883,7 +884,7 @@ class DataSourceV2SQLSuite } } withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { key => + RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => withNamespace("testcat.reservedTest") { sql(s"CREATE NAMESPACE testcat.reservedTest WITH DBPROPERTIES('$key'='foo')") assert(sql("DESC NAMESPACE EXTENDED testcat.reservedTest") @@ -899,6 +900,87 @@ class DataSourceV2SQLSuite } } + test("create/replace/alter table - reserved properties") { + import TableCatalog._ + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { + RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => + Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => + Seq("CREATE", "REPLACE").foreach { action => + val e = intercept[ParseException] { + sql(s"$action TABLE testcat.reservedTest (key int) USING foo $clause ('$key'='bar')") + } + assert(e.getMessage.contains(s"$key is a reserved table property")) + } + } + + val e1 = intercept[ParseException] { + sql(s"ALTER TABLE testcat.reservedTest SET TBLPROPERTIES ('$key'='bar')") + } + assert(e1.getMessage.contains(s"$key is a reserved table property")) + + val e2 = intercept[ParseException] { + sql(s"ALTER TABLE testcat.reservedTest UNSET TBLPROPERTIES ('$key')") + } + assert(e2.getMessage.contains(s"$key is a reserved table property")) + } + } + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { + RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => + Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => + withTable("testcat.reservedTest") { + Seq("CREATE", "REPLACE").foreach { action => + sql(s"$action TABLE testcat.reservedTest (key int) USING foo $clause ('$key'='bar')") + val tableCatalog = catalog("testcat").asTableCatalog + val identifier = Identifier.of(Array(), "reservedTest") + val originValue = tableCatalog.loadTable(identifier).properties().get(key) + assert(originValue != "bar", "reserved properties should not have side effects") + sql(s"ALTER TABLE testcat.reservedTest SET TBLPROPERTIES ('$key'='newValue')") + assert(tableCatalog.loadTable(identifier).properties().get(key) == originValue, + "reserved properties should not have side effects") + sql(s"ALTER TABLE testcat.reservedTest UNSET TBLPROPERTIES ('$key')") + assert(tableCatalog.loadTable(identifier).properties().get(key) == originValue, + "reserved properties should not have side effects") + } + } + } + } + } + } + + test("create/replace - path property") { + Seq("true", "false").foreach { conf => + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, conf)) { + withTable("testcat.reservedTest") { + Seq("CREATE", "REPLACE").foreach { action => + val e1 = intercept[ParseException] { + sql(s"$action TABLE testcat.reservedTest USING foo LOCATION 'foo' OPTIONS" + + s" ('path'='bar')") + } + assert(e1.getMessage.contains(s"Duplicated table paths found: 'foo' and 'bar'")) + + val e2 = intercept[ParseException] { + sql(s"$action TABLE testcat.reservedTest USING foo OPTIONS" + + s" ('path'='foo', 'PaTh'='bar')") + } + assert(e2.getMessage.contains(s"Duplicated table paths found: 'foo' and 'bar'")) + + sql(s"$action TABLE testcat.reservedTest USING foo LOCATION 'foo' TBLPROPERTIES" + + s" ('path'='bar', 'Path'='noop')") + val tableCatalog = catalog("testcat").asTableCatalog + val identifier = Identifier.of(Array(), "reservedTest") + assert(tableCatalog.loadTable(identifier).properties() + .get(TableCatalog.PROP_LOCATION) == "foo", + "path as a table property should not have side effects") + assert(tableCatalog.loadTable(identifier).properties().get("path") == "bar", + "path as a table property should not have side effects") + assert(tableCatalog.loadTable(identifier).properties().get("Path") == "noop", + "path as a table property should not have side effects") + } + } + } + } + } + test("DropNamespace: basic tests") { // Session catalog is used. sql("CREATE NAMESPACE ns") @@ -1001,8 +1083,9 @@ class DataSourceV2SQLSuite } test("AlterNamespaceSetProperties: reserved properties") { + import SupportsNamespaces._ withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { key => + RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => withNamespace("testcat.reservedTest") { sql("CREATE NAMESPACE testcat.reservedTest") val exception = intercept[ParseException] { @@ -1013,7 +1096,7 @@ class DataSourceV2SQLSuite } } withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { key => + RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => withNamespace("testcat.reservedTest") { sql(s"CREATE NAMESPACE testcat.reservedTest") sql(s"ALTER NAMESPACE testcat.reservedTest SET PROPERTIES ('$key'='foo')") @@ -1854,8 +1937,8 @@ class DataSourceV2SQLSuite val expected = Seq( Row("owner", owner), - Row("provider", provider), - Row("status", status)) + Row("status", status), + Row("provider", provider)) assert(properties.schema === schema) assert(expected === properties.collect()) From 6e5b4bf113b220f20acbc0b861bde93cfeef4c9a Mon Sep 17 00:00:00 2001 From: jiake Date: Thu, 16 Jan 2020 22:52:00 +0800 Subject: [PATCH 1128/1387] [SPARK-30524][SQL] Disable OptimizeSkewedJoin rule when introducing additional shuffle ### What changes were proposed in this pull request? `OptimizeSkewedJoin `rule change the `outputPartitioning `after inserting `PartialShuffleReaderExec `or `SkewedPartitionReaderExec`. So it may need to introduce additional to ensure the right result. This PR disable `OptimizeSkewedJoin ` rule when introducing additional shuffle. ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add new ut Closes #27226 from JkSelf/followup-skewedoptimization. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../adaptive/AdaptiveSparkPlanExec.scala | 4 +- .../adaptive/OptimizeSkewedJoin.scala | 107 ++++++++++++++---- .../execution/joins/SortMergeJoinExec.scala | 17 ++- .../adaptive/AdaptiveQueryExecSuite.scala | 26 +++++ 4 files changed, 125 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index fb43c8f9941ea..56ea12918448c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -87,9 +87,9 @@ case class AdaptiveSparkPlanExec( // optimizations should be stage-independent. @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( ReuseAdaptiveSubquery(conf, context.subqueryCache), - // Here the 'OptimizeSkewedPartitions' rule should be executed + // Here the 'OptimizeSkewedJoin' rule should be executed // before 'ReduceNumShufflePartitions', as the skewed partition handled - // in 'OptimizeSkewedPartitions' rule, should be omitted in 'ReduceNumShufflePartitions'. + // in 'OptimizeSkewedJoin' rule, should be omitted in 'ReduceNumShufflePartitions'. OptimizeSkewedJoin(conf), ReduceNumShufflePartitions(conf), // The rule of 'OptimizeLocalShuffleReader' need to make use of the 'partitionStartIndices' diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index 75d4184a2c14e..ad828006e3315 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -28,12 +28,14 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.internal.SQLConf case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { + private val ensureRequirements = EnsureRequirements(conf) + private val supportedJoinTypes = Inner :: Cross :: LeftSemi :: LeftAnti :: LeftOuter :: RightOuter :: Nil @@ -54,7 +56,11 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { private def medianSize(stats: MapOutputStatistics): Long = { val numPartitions = stats.bytesByPartitionId.length val bytes = stats.bytesByPartitionId.sorted - if (bytes(numPartitions / 2) > 0) bytes(numPartitions / 2) else 1 + numPartitions match { + case _ if (numPartitions % 2 == 0) => + math.max((bytes(numPartitions / 2) + bytes(numPartitions / 2 - 1)) / 2, 1) + case _ => math.max(bytes(numPartitions / 2), 1) + } } /** @@ -76,18 +82,19 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { val avgPartitionSize = mapPartitionSizes.sum / maxSplits val advisoryPartitionSize = math.max(avgPartitionSize, conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD)) - val partitionIndices = mapPartitionSizes.indices val partitionStartIndices = ArrayBuffer[Int]() - var postMapPartitionSize = mapPartitionSizes(0) partitionStartIndices += 0 - partitionIndices.drop(1).foreach { nextPartitionIndex => - val nextMapPartitionSize = mapPartitionSizes(nextPartitionIndex) - if (postMapPartitionSize + nextMapPartitionSize > advisoryPartitionSize) { - partitionStartIndices += nextPartitionIndex + var i = 0 + var postMapPartitionSize = 0L + while (i < mapPartitionSizes.length) { + val nextMapPartitionSize = mapPartitionSizes(i) + if (i > 0 && postMapPartitionSize + nextMapPartitionSize > advisoryPartitionSize) { + partitionStartIndices += i postMapPartitionSize = nextMapPartitionSize } else { postMapPartitionSize += nextMapPartitionSize } + i += 1 } if (partitionStartIndices.size > maxSplits) { @@ -114,25 +121,56 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { stage.shuffle.shuffleDependency.rdd.partitions.length } - def handleSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { - case smj @ SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, - s1 @ SortExec(_, _, left: ShuffleQueryStageExec, _), - s2 @ SortExec(_, _, right: ShuffleQueryStageExec, _)) - if supportedJoinTypes.contains(joinType) => + private def getShuffleQueryStage(plan : SparkPlan): Option[ShuffleQueryStageExec] = + plan match { + case stage: ShuffleQueryStageExec => Some(stage) + case SortExec(_, _, s: ShuffleQueryStageExec, _) => + Some(s) + case _ => None + } + + private def reOptimizeChild( + skewedReader: SkewedPartitionReaderExec, + child: SparkPlan): SparkPlan = child match { + case sort @ SortExec(_, _, s: ShuffleQueryStageExec, _) => + sort.copy(child = skewedReader) + case _: ShuffleQueryStageExec => skewedReader + } + + private def getSizeInfo(medianSize: Long, maxSize: Long): String = { + s"median size: $medianSize, max size: ${maxSize}" + } + + /* + * This method aim to optimize the skewed join with the following steps: + * 1. Check whether the shuffle partition is skewed based on the median size + * and the skewed partition threshold in origin smj. + * 2. Assuming partition0 is skewed in left side, and it has 5 mappers (Map0, Map1...Map4). + * And we will split the 5 Mappers into 3 mapper ranges [(Map0, Map1), (Map2, Map3), (Map4)] + * based on the map size and the max split number. + * 3. Create the 3 smjs with separately reading the above mapper ranges and then join with + * the Partition0 in right side. + * 4. Finally union the above 3 split smjs and the origin smj. + */ + def optimizeSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { + case smj @ SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, leftPlan, rightPlan) + if (getShuffleQueryStage(leftPlan).nonEmpty && getShuffleQueryStage(rightPlan).nonEmpty) && + supportedJoinTypes.contains(joinType) => + val left = getShuffleQueryStage(leftPlan).get + val right = getShuffleQueryStage(rightPlan).get val leftStats = getStatistics(left) val rightStats = getStatistics(right) val numPartitions = leftStats.bytesByPartitionId.length val leftMedSize = medianSize(leftStats) val rightMedSize = medianSize(rightStats) - val leftSizeInfo = s"median size: $leftMedSize, max size: ${leftStats.bytesByPartitionId.max}" - val rightSizeInfo = s"median size: $rightMedSize," + - s" max size: ${rightStats.bytesByPartitionId.max}" logDebug( s""" |Try to optimize skewed join. - |Left side partition size: $leftSizeInfo - |Right side partition size: $rightSizeInfo + |Left side partition size: + |${getSizeInfo(leftMedSize, leftStats.bytesByPartitionId.max)} + |Right side partition size: + |${getSizeInfo(rightMedSize, rightStats.bytesByPartitionId.max)} """.stripMargin) val skewedPartitions = mutable.HashSet[Int]() @@ -171,16 +209,19 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { left, partitionId, leftMapIdStartIndices(i), leftEndMapId) val rightSkewedReader = SkewedPartitionReaderExec(right, partitionId, rightMapIdStartIndices(j), rightEndMapId) + val skewedLeft = reOptimizeChild(leftSkewedReader, leftPlan) + val skewedRight = reOptimizeChild(rightSkewedReader, rightPlan) subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, - s1.copy(child = leftSkewedReader), s2.copy(child = rightSkewedReader)) + skewedLeft, skewedRight) } } } logDebug(s"number of skewed partitions is ${skewedPartitions.size}") if (skewedPartitions.nonEmpty) { - val optimizedSmj = smj.transformDown { - case sort @ SortExec(_, _, shuffleStage: ShuffleQueryStageExec, _) => - sort.copy(child = PartialShuffleReaderExec(shuffleStage, skewedPartitions.toSet)) + val optimizedSmj = smj.transformUp { + case shuffleStage: ShuffleQueryStageExec if shuffleStage.id == left.id || + shuffleStage.id == right.id => + PartialShuffleReaderExec(shuffleStage, skewedPartitions.toSet) } subJoins += optimizedSmj UnionExec(subJoins) @@ -204,11 +245,27 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { val shuffleStages = collectShuffleStages(plan) if (shuffleStages.length == 2) { - // Currently we only support handling skewed join for 2 table join. - handleSkewJoin(plan) + // When multi table join, there will be too many complex combination to consider. + // Currently we only handle 2 table join like following two use cases. + // SMJ SMJ + // Sort Shuffle + // Shuffle or Shuffle + // Sort + // Shuffle + val optimizePlan = optimizeSkewJoin(plan) + val numShuffles = ensureRequirements.apply(optimizePlan).collect { + case e: ShuffleExchangeExec => e + }.length + + if (numShuffles > 0) { + logDebug("OptimizeSkewedJoin rule is not applied due" + + " to additional shuffles will be introduced.") + plan + } else { + optimizePlan + } } else { plan - } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index f327e84563da9..985e1db2736fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.{PartialShuffleReaderExec, SkewedPartitionReaderExec} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.util.collection.BitSet @@ -95,8 +96,20 @@ case class SortMergeJoinExec( s"${getClass.getSimpleName} should not take $x as the JoinType") } - override def requiredChildDistribution: Seq[Distribution] = - HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + private def containSkewedReader(plan: SparkPlan): Boolean = plan match { + case s: SkewedPartitionReaderExec => true + case p: PartialShuffleReaderExec => true + case s: SortExec => containSkewedReader(s.child) + case _ => false + } + + override def requiredChildDistribution: Seq[Distribution] = { + if (containSkewedReader(left)) { + UnspecifiedDistribution :: UnspecifiedDistribution :: Nil + } else { + HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil + } + } override def outputOrdering: Seq[SortOrder] = joinType match { // For inner join, orders of both sides keys should be kept. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index c2daae071afc5..78a1183664749 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -579,6 +579,32 @@ class AdaptiveQueryExecSuite } } + test("SPARK-30524: Do not optimize skew join if introduce additional shuffle") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "700") { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 2 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + val (innerPlan, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT key1 FROM skewData1 join skewData2 ON key1 = key2 group by key1") + val innerSmj = findTopLevelSortMergeJoin(innerPlan) + assert(innerSmj.size == 1) + // Additional shuffle introduced, so disable the "OptimizeSkewedJoin" optimization + val innerSmjAfter = findTopLevelSortMergeJoin(innerAdaptivePlan) + assert(innerSmjAfter.size == 1) + } + } + } + test("SPARK-29544: adaptive skew join with different join types") { Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => withSQLConf( From dca838058ffd0e2c01591fd9ab0f192de446d606 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 16 Jan 2020 13:37:11 -0800 Subject: [PATCH 1129/1387] [SPARK-29950][K8S] Blacklist deleted executors in K8S with dynamic allocation The issue here is that when Spark is downscaling the application and deletes a few pod requests that aren't needed anymore, it may actually race with the K8S scheduler, who may be bringing up those executors. So they may have enough time to connect back to the driver, register, to just be deleted soon after. This wastes resources and causes misleading entries in the driver log. The change (ab)uses the blacklisting mechanism to consider the deleted excess pods as blacklisted, so that if they try to connect back, the driver will deny it. It also changes the executor registration slightly, since even with the above change there were misleading logs. That was because the executor registration message was an RPC that always succeeded (bar network issues), so the executor would always try to send an unregistration message to the driver, which would then log several messages about not knowing anything about the executor. The change makes the registration RPC succeed or fail directly, instead of using the separate failure message that would lead to this issue. Note the last change required some changes in a standalone test suite related to dynamic allocation, since it relied on the driver not throwing exceptions when a duplicate executor registration happened. Tested with existing unit tests, and with live cluster with dyn alloc on. Closes #26586 from vanzin/SPARK-29950. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../CoarseGrainedExecutorBackend.scala | 14 ++-- .../cluster/CoarseGrainedClusterMessage.scala | 7 -- .../CoarseGrainedSchedulerBackend.scala | 19 ++++-- .../StandaloneDynamicAllocationSuite.scala | 65 ++++++++++++------- .../CoarseGrainedSchedulerBackendSuite.scala | 1 + .../cluster/k8s/ExecutorPodsAllocator.scala | 18 +++++ .../KubernetesClusterSchedulerBackend.scala | 4 ++ ...erministicExecutorPodsSnapshotsStore.scala | 9 +++ .../k8s/ExecutorPodsAllocatorSuite.scala | 11 ++++ 9 files changed, 105 insertions(+), 43 deletions(-) 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 b1837c9c0c9ea..1fe901a83af33 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -54,6 +54,8 @@ private[spark] class CoarseGrainedExecutorBackend( resourcesFileOpt: Option[String]) extends IsolatedRpcEndpoint with ExecutorBackend with Logging { + import CoarseGrainedExecutorBackend._ + private implicit val formats = DefaultFormats private[this] val stopping = new AtomicBoolean(false) @@ -80,9 +82,8 @@ private[spark] class CoarseGrainedExecutorBackend( ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls, extractAttributes, resources)) }(ThreadUtils.sameThread).onComplete { - // This is a very fast action so we can use "ThreadUtils.sameThread" - case Success(msg) => - // Always receive `true`. Just ignore it + case Success(_) => + self.send(RegisteredExecutor) case Failure(e) => exitExecutor(1, s"Cannot register with driver: $driverUrl", e, notifyDriver = false) }(ThreadUtils.sameThread) @@ -133,9 +134,6 @@ private[spark] class CoarseGrainedExecutorBackend( exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) } - case RegisterExecutorFailed(message) => - exitExecutor(1, "Slave registration failed: " + message) - case LaunchTask(data) => if (executor == null) { exitExecutor(1, "Received LaunchTask command but executor was null") @@ -226,6 +224,10 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend extends Logging { + // Message used internally to start the executor when the driver successfully accepted the + // registration request. + case object RegisteredExecutor + case class Arguments( driverUrl: String, executorId: String, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 9ce23683245eb..57317e7f6af00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -48,13 +48,6 @@ private[spark] object CoarseGrainedClusterMessages { case class KillExecutorsOnHost(host: String) extends CoarseGrainedClusterMessage - sealed trait RegisterExecutorResponse - - case object RegisteredExecutor extends CoarseGrainedClusterMessage with RegisterExecutorResponse - - case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage - with RegisterExecutorResponse - case class UpdateDelegationTokens(tokens: Array[Byte]) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 7c7d8c2eac9aa..031b9afaa17aa 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 @@ -207,15 +207,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls, attributes, resources) => if (executorDataMap.contains(executorId)) { - executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) - context.reply(true) - } else if (scheduler.nodeBlacklist.contains(hostname)) { + context.sendFailure(new IllegalStateException(s"Duplicate executor ID: $executorId")) + } else if (scheduler.nodeBlacklist.contains(hostname) || + isBlacklisted(executorId, hostname)) { // If the cluster manager gives us an executor on a blacklisted node (because it // already started allocating those resources before we informed it of our blacklist, // or if it ignored our blacklist), then we reject that executor immediately. logInfo(s"Rejecting $executorId as it has been blacklisted.") - executorRef.send(RegisterExecutorFailed(s"Executor is blacklisted: $executorId")) - context.reply(true) + context.sendFailure(new IllegalStateException(s"Executor is blacklisted: $executorId")) } else { // If the executor's rpc env is not listening for incoming connections, `hostPort` // will be null, and the client connection should be used to contact the executor. @@ -250,7 +249,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") } } - executorRef.send(RegisteredExecutor) // Note: some tests expect the reply to come after we put the executor in the map context.reply(true) listenerBus.post( @@ -776,6 +774,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp protected def currentDelegationTokens: Array[Byte] = delegationTokens.get() + /** + * Checks whether the executor is blacklisted. This is called when the executor tries to + * register with the scheduler, and will deny registration if this method returns true. + * + * This is in addition to the blacklist kept by the task scheduler, so custom implementations + * don't need to check there. + */ + protected def isBlacklisted(executorId: String, hostname: String): Boolean = false + // SPARK-27112: We need to ensure that there is ordering of lock acquisition // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in order to fix // the deadlock issue exposed in SPARK-27112 diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index dd790b8dbb853..e316da738440e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.{BeforeAndAfterAll, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ -import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.ApplicationInfo import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.Worker @@ -34,7 +34,7 @@ import org.apache.spark.internal.config import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster._ -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{LaunchedExecutor, RegisterExecutor, RegisterExecutorFailed} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{LaunchedExecutor, RegisterExecutor} /** * End-to-end tests for dynamic allocation in standalone mode. @@ -482,12 +482,16 @@ class StandaloneDynamicAllocationSuite assert(apps.head.getExecutorLimit === Int.MaxValue) } val beforeList = getApplications().head.executors.keys.toSet - assert(killExecutorsOnHost(sc, "localhost").equals(true)) - syncExecutors(sc) - val afterList = getApplications().head.executors.keys.toSet + + sc.schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => + b.killExecutorsOnHost("localhost") + case _ => fail("expected coarse grained scheduler") + } eventually(timeout(10.seconds), interval(100.millis)) { + val afterList = getApplications().head.executors.keys.toSet assert(beforeList.intersect(afterList).size == 0) } } @@ -514,10 +518,11 @@ class StandaloneDynamicAllocationSuite val scheduler = new CoarseGrainedSchedulerBackend(taskScheduler, rpcEnv) try { scheduler.start() - scheduler.driverEndpoint.ask[Boolean](message) - eventually(timeout(10.seconds), interval(100.millis)) { - verify(endpointRef).send(RegisterExecutorFailed(any())) + val e = intercept[SparkException] { + scheduler.driverEndpoint.askSync[Boolean](message) } + assert(e.getCause().isInstanceOf[IllegalStateException]) + assert(scheduler.getExecutorIds().isEmpty) } finally { scheduler.stop() } @@ -536,6 +541,11 @@ class StandaloneDynamicAllocationSuite .setMaster(masterRpcEnv.address.toSparkURL) .setAppName("test") .set(config.EXECUTOR_MEMORY.key, "256m") + // Because we're faking executor launches in the Worker, set the config so that the driver + // will not timeout anything related to executors. + .set(config.Network.NETWORK_TIMEOUT.key, "2h") + .set(config.EXECUTOR_HEARTBEAT_INTERVAL.key, "1h") + .set(config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT.key, "1h") } /** Make a master to which our application will send executor requests. */ @@ -549,8 +559,7 @@ class StandaloneDynamicAllocationSuite private def makeWorkers(cores: Int, memory: Int): Seq[Worker] = { (0 until numWorkers).map { i => val rpcEnv = workerRpcEnvs(i) - val worker = new Worker(rpcEnv, 0, cores, memory, Array(masterRpcEnv.address), - Worker.ENDPOINT_NAME, null, conf, securityManager) + val worker = new TestWorker(rpcEnv, cores, memory) rpcEnv.setupEndpoint(Worker.ENDPOINT_NAME, worker) worker } @@ -588,16 +597,6 @@ class StandaloneDynamicAllocationSuite } } - /** Kill the executors on a given host. */ - private def killExecutorsOnHost(sc: SparkContext, host: String): Boolean = { - syncExecutors(sc) - sc.schedulerBackend match { - case b: CoarseGrainedSchedulerBackend => - b.killExecutorsOnHost(host) - case _ => fail("expected coarse grained scheduler") - } - } - /** * Return a list of executor IDs belonging to this application. * @@ -620,9 +619,8 @@ class StandaloneDynamicAllocationSuite * we submit a request to kill them. This must be called before each kill request. */ private def syncExecutors(sc: SparkContext): Unit = { - val driverExecutors = sc.env.blockManager.master.getStorageStatus - .map(_.blockManagerId.executorId) - .filter { _ != SparkContext.DRIVER_IDENTIFIER} + val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] + val driverExecutors = backend.getExecutorIds() val masterExecutors = getExecutorIds(sc) val missingExecutors = masterExecutors.toSet.diff(driverExecutors.toSet).toSeq.sorted missingExecutors.foreach { id => @@ -632,10 +630,29 @@ class StandaloneDynamicAllocationSuite when(endpointRef.address).thenReturn(mockAddress) val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty, Map.empty, Map.empty) - val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askSync[Boolean](message) backend.driverEndpoint.send(LaunchedExecutor(id)) } } + /** + * Worker implementation that does not actually launch any executors, but reports them as + * running so the Master keeps track of them. This requires that `syncExecutors` be used + * to make sure the Master instance and the SparkContext under test agree about what + * executors are running. + */ + private class TestWorker(rpcEnv: RpcEnv, cores: Int, memory: Int) + extends Worker( + rpcEnv, 0, cores, memory, Array(masterRpcEnv.address), Worker.ENDPOINT_NAME, + null, conf, securityManager) { + + override def receive: PartialFunction[Any, Unit] = testReceive.orElse(super.receive) + + private def testReceive: PartialFunction[Any, Unit] = synchronized { + case LaunchExecutor(_, appId, execId, _, _, _, _) => + self.send(ExecutorStateChanged(appId, execId, ExecutorState.RUNNING, None, None)) + } + + } + } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index f916f634f97cb..29160a3e0fd53 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -189,6 +189,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo val conf = new SparkConf() .set(EXECUTOR_CORES, 1) .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test + .set(EXECUTOR_INSTANCES, 0) // avoid errors about duplicate executor registrations .setMaster( "coarseclustermanager[org.apache.spark.scheduler.TestCoarseGrainedSchedulerBackend]") .setAppName("test") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 2201bf91d3905..b394f35b15111 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -72,6 +72,11 @@ private[spark] class ExecutorPodsAllocator( private var lastSnapshot = ExecutorPodsSnapshot(Nil) + // Executors that have been deleted by this allocator but not yet detected as deleted in + // a snapshot from the API server. This is used to deny registration from these executors + // if they happen to come up before the deletion takes effect. + @volatile private var deletedExecutorIds = Set.empty[Long] + def start(applicationId: String): Unit = { snapshotsStore.addSubscriber(podAllocationDelay) { onNewSnapshots(applicationId, _) @@ -85,6 +90,8 @@ private[spark] class ExecutorPodsAllocator( } } + def isDeleted(executorId: String): Boolean = deletedExecutorIds.contains(executorId.toLong) + private def onNewSnapshots( applicationId: String, snapshots: Seq[ExecutorPodsSnapshot]): Unit = synchronized { @@ -141,10 +148,17 @@ private[spark] class ExecutorPodsAllocator( } .map { case (id, _) => id } + // Make a local, non-volatile copy of the reference since it's used multiple times. This + // is the only method that modifies the list, so this is safe. + var _deletedExecutorIds = deletedExecutorIds + if (snapshots.nonEmpty) { logDebug(s"Pod allocation status: $currentRunningCount running, " + s"${currentPendingExecutors.size} pending, " + s"${newlyCreatedExecutors.size} unacknowledged.") + + val existingExecs = lastSnapshot.executorPods.keySet + _deletedExecutorIds = _deletedExecutorIds.filter(existingExecs.contains) } val currentTotalExpectedExecutors = totalExpectedExecutors.get @@ -169,6 +183,8 @@ private[spark] class ExecutorPodsAllocator( if (toDelete.nonEmpty) { logInfo(s"Deleting ${toDelete.size} excess pod requests (${toDelete.mkString(",")}).") + _deletedExecutorIds = _deletedExecutorIds ++ toDelete + Utils.tryLogNonFatalError { kubernetesClient .pods() @@ -209,6 +225,8 @@ private[spark] class ExecutorPodsAllocator( } } + deletedExecutorIds = _deletedExecutorIds + // Update the flag that helps the setTotalExpectedExecutors() callback avoid triggering this // update method when not needed. hasPendingPods.set(knownPendingCount + newlyCreatedExecutors.size > 0) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index e221a926daca8..105841ac834b3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -181,6 +181,10 @@ private[spark] class KubernetesClusterSchedulerBackend( Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration, driverEndpoint)) } + override protected def isBlacklisted(executorId: String, hostname: String): Boolean = { + podAllocator.isDeleted(executorId) + } + private class KubernetesDriverEndpoint extends DriverEndpoint { override def onDisconnected(rpcAddress: RpcAddress): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala index 1b6dfe5443377..9ac7e0222054a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/DeterministicExecutorPodsSnapshotsStore.scala @@ -48,4 +48,13 @@ class DeterministicExecutorPodsSnapshotsStore extends ExecutorPodsSnapshotsStore currentSnapshot = ExecutorPodsSnapshot(newSnapshot) snapshotsBuffer += currentSnapshot } + + def removeDeletedExecutors(): Unit = { + val nonDeleted = currentSnapshot.executorPods.filter { + case (_, PodDeleted(_)) => false + case _ => true + } + currentSnapshot = ExecutorPodsSnapshot(nonDeleted) + snapshotsBuffer += currentSnapshot + } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 4475d5db6f03a..a0abded3823bb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -189,6 +189,17 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { verify(podOperations, times(4)).create(any()) verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "3", "4") verify(podOperations).delete() + assert(podsAllocatorUnderTest.isDeleted("3")) + assert(podsAllocatorUnderTest.isDeleted("4")) + + // Update the snapshot to not contain the deleted executors, make sure the + // allocator cleans up internal state. + snapshotsStore.updatePod(deletedExecutor(3)) + snapshotsStore.updatePod(deletedExecutor(4)) + snapshotsStore.removeDeletedExecutors() + snapshotsStore.notifySubscribers() + assert(!podsAllocatorUnderTest.isDeleted("3")) + assert(!podsAllocatorUnderTest.isDeleted("4")) } private def executorPodAnswer(): Answer[SparkPod] = From 384899944b25cb0abf5e71f9cc2610fecad4e8f5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 16 Jan 2020 16:00:58 -0800 Subject: [PATCH 1130/1387] [SPARK-30534][INFRA] Use mvn in `dev/scalastyle` ### What changes were proposed in this pull request? This PR aims to use `mvn` instead of `sbt` in `dev/scalastyle` to recover GitHub Action. ### Why are the changes needed? As of now, Apache Spark sbt build is broken by the Maven Central repository policy. https://stackoverflow.com/questions/59764749/requests-to-http-repo1-maven-org-maven2-return-a-501-https-required-status-an > Effective January 15, 2020, The Central Maven Repository no longer supports insecure > communication over plain HTTP and requires that all requests to the repository are > encrypted over HTTPS. We can reproduce this locally by the following. ``` $ rm -rf ~/.m2/repository/org/apache/apache/18/ $ build/sbt clean ``` And, in GitHub Action, `lint-scala` is the only one which is using `sbt`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? First of all, GitHub Action should be recovered. Also, manually, do the following. **Without Scalastyle violation** ``` $ dev/scalastyle OpenJDK 64-Bit Server VM warning: ignoring option MaxPermSize=384m; support was removed in 8.0 Using `mvn` from path: /usr/local/bin/mvn Scalastyle checks passed. ``` **With Scalastyle violation** ``` $ dev/scalastyle OpenJDK 64-Bit Server VM warning: ignoring option MaxPermSize=384m; support was removed in 8.0 Using `mvn` from path: /usr/local/bin/mvn Scalastyle checks failed at following occurrences: error file=/Users/dongjoon/PRS/SPARK-HTTP-501/core/src/main/scala/org/apache/spark/SparkConf.scala message=There should be no empty line separating imports in the same group. line=22 column=0 error file=/Users/dongjoon/PRS/SPARK-HTTP-501/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala message=There should be no empty line separating imports in the same group. line=22 column=0 ``` Closes #27242 from dongjoon-hyun/SPARK-30534. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/scalastyle | 14 +++----------- pom.xml | 2 +- 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/dev/scalastyle b/dev/scalastyle index 212ef900eb9b4..b555ce08b789e 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,18 +17,10 @@ # limitations under the License. # -SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"} +SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" -# NOTE: echo "q" is needed because SBT prompts the user for input on encountering a build file -# with failure (either resolution or compilation); the "q" makes SBT quit. -ERRORS=$(echo -e "q\n" \ - | build/sbt \ - ${SPARK_PROFILES} \ - -Pdocker-integration-tests \ - -Pkubernetes-integration-tests \ - scalastyle test:scalastyle \ - | awk '{if($1~/error/)print}' \ -) +SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive -Pdocker-integration-tests -Pkubernetes-integration-tests"} +ERRORS=$($SCRIPT_DIR/../build/mvn $SPARK_PROFILES scalastyle:check | grep "^error file") if test ! -z "$ERRORS"; then echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" diff --git a/pom.xml b/pom.xml index e72fcd90a38d8..58c05357ef417 100644 --- a/pom.xml +++ b/pom.xml @@ -2760,7 +2760,7 @@ false true - false + true false ${basedir}/src/main/scala ${basedir}/src/test/scala From 92dd7c9d2a6d0eae9502bfa44b632187071626fd Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 17 Jan 2020 10:04:41 +0800 Subject: [PATCH 1131/1387] [MINOR][ML] Change DecisionTreeClassifier to FMClassifier in OneVsRest setWeightCol test ### What changes were proposed in this pull request? Change ```DecisionTreeClassifier``` to ```FMClassifier``` in ```OneVsRest``` setWeightCol test ### Why are the changes needed? In ```OneVsRest```, if the classifier doesn't support instance weight, ```OneVsRest``` weightCol will be ignored, so unit test has tested one classifier(```LogisticRegression```) that support instance weight, and one classifier (```DecisionTreeClassifier```) that doesn't support instance weight. Since ```DecisionTreeClassifier``` now supports instance weight, we need to change it to the classifier that doesn't have weight support. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing test Closes #27204 from huaxingao/spark-ovr-minor. Authored-by: Huaxin Gao Signed-off-by: zhengruifeng --- .../org/apache/spark/ml/classification/OneVsRestSuite.scala | 2 +- python/pyspark/ml/tests/test_algorithms.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 024a3870d8bca..a4b1a5f18141a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -206,7 +206,7 @@ class OneVsRestSuite extends MLTest with DefaultReadWriteTest { val ova = new OneVsRest().setWeightCol("weight").setClassifier(new LogisticRegression()) assert(ova.fit(dataset2) !== null) // classifier doesn't inherit hasWeightCol - val ova2 = new OneVsRest().setWeightCol("weight").setClassifier(new DecisionTreeClassifier()) + val ova2 = new OneVsRest().setWeightCol("weight").setClassifier(new FMClassifier()) assert(ova2.fit(dataset2) !== null) } diff --git a/python/pyspark/ml/tests/test_algorithms.py b/python/pyspark/ml/tests/test_algorithms.py index 1864a66487a28..2faf2d98f0271 100644 --- a/python/pyspark/ml/tests/test_algorithms.py +++ b/python/pyspark/ml/tests/test_algorithms.py @@ -21,7 +21,7 @@ import numpy as np -from pyspark.ml.classification import DecisionTreeClassifier, LogisticRegression, \ +from pyspark.ml.classification import FMClassifier, LogisticRegression, \ MultilayerPerceptronClassifier, OneVsRest from pyspark.ml.clustering import DistributedLDAModel, KMeans, LocalLDAModel, LDA, LDAModel from pyspark.ml.fpm import FPGrowth @@ -140,7 +140,7 @@ def test_support_for_weightCol(self): ovr = OneVsRest(classifier=lr, weightCol="weight") self.assertIsNotNone(ovr.fit(df)) # classifier doesn't inherit hasWeightCol - dt = DecisionTreeClassifier() + dt = FMClassifier() ovr2 = OneVsRest(classifier=dt, weightCol="weight") self.assertIsNotNone(ovr2.fit(df)) From 1a9de8c31fe6a91c7fd6c9196e6882ef9779420b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 17 Jan 2020 11:44:49 +0900 Subject: [PATCH 1132/1387] [SPARK-30499][SQL] Remove SQL config spark.sql.execution.pandas.respectSessionTimeZone ### What changes were proposed in this pull request? In the PR, I propose to remove the SQL config `spark.sql.execution.pandas.respectSessionTimeZone` which has been deprecated since Spark 2.3. ### Why are the changes needed? To improve code maintainability. ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? by running python tests, https://spark.apache.org/docs/latest/building-spark.html#pyspark-tests-with-maven-or-sbt Closes #27218 from MaxGekk/remove-respectSessionTimeZone. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- python/pyspark/sql/pandas/conversion.py | 10 ++------- python/pyspark/sql/tests/test_arrow.py | 22 +++++++------------ .../sql/tests/test_pandas_udf_scalar.py | 11 ++++------ .../apache/spark/sql/internal/SQLConf.scala | 19 ++++------------ .../apache/spark/sql/util/ArrowUtils.scala | 6 +---- 5 files changed, 19 insertions(+), 49 deletions(-) diff --git a/python/pyspark/sql/pandas/conversion.py b/python/pyspark/sql/pandas/conversion.py index 42374100da28b..f7680ccdc8431 100644 --- a/python/pyspark/sql/pandas/conversion.py +++ b/python/pyspark/sql/pandas/conversion.py @@ -64,10 +64,7 @@ def toPandas(self): import numpy as np import pandas as pd - if self.sql_ctx._conf.pandasRespectSessionTimeZone(): - timezone = self.sql_ctx._conf.sessionLocalTimeZone() - else: - timezone = None + timezone = self.sql_ctx._conf.sessionLocalTimeZone() if self.sql_ctx._conf.arrowPySparkEnabled(): use_arrow = True @@ -231,10 +228,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr from pyspark.sql.pandas.utils import require_minimum_pandas_version require_minimum_pandas_version() - if self._wrapped._conf.pandasRespectSessionTimeZone(): - timezone = self._wrapped._conf.sessionLocalTimeZone() - else: - timezone = None + timezone = self._wrapped._conf.sessionLocalTimeZone() # If no schema supplied by user then get the names of columns only if schema is None: diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index f0930125e310f..98f44dfd29da5 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -163,16 +163,13 @@ def test_toPandas_arrow_toggle(self): def test_toPandas_respect_session_timezone(self): df = self.spark.createDataFrame(self.data, schema=self.schema) - timezone = "America/New_York" - with self.sql_conf({ - "spark.sql.execution.pandas.respectSessionTimeZone": False, - "spark.sql.session.timeZone": timezone}): + timezone = "America/Los_Angeles" + with self.sql_conf({"spark.sql.session.timeZone": timezone}): pdf_la, pdf_arrow_la = self._toPandas_arrow_toggle(df) assert_frame_equal(pdf_arrow_la, pdf_la) - with self.sql_conf({ - "spark.sql.execution.pandas.respectSessionTimeZone": True, - "spark.sql.session.timeZone": timezone}): + timezone = "America/New_York" + with self.sql_conf({"spark.sql.session.timeZone": timezone}): pdf_ny, pdf_arrow_ny = self._toPandas_arrow_toggle(df) assert_frame_equal(pdf_arrow_ny, pdf_ny) @@ -234,18 +231,15 @@ def test_createDataFrame_toggle(self): def test_createDataFrame_respect_session_timezone(self): from datetime import timedelta pdf = self.create_pandas_data_frame() - timezone = "America/New_York" - with self.sql_conf({ - "spark.sql.execution.pandas.respectSessionTimeZone": False, - "spark.sql.session.timeZone": timezone}): + timezone = "America/Los_Angeles" + with self.sql_conf({"spark.sql.session.timeZone": timezone}): df_no_arrow_la, df_arrow_la = self._createDataFrame_toggle(pdf, schema=self.schema) result_la = df_no_arrow_la.collect() result_arrow_la = df_arrow_la.collect() self.assertEqual(result_la, result_arrow_la) - with self.sql_conf({ - "spark.sql.execution.pandas.respectSessionTimeZone": True, - "spark.sql.session.timeZone": timezone}): + timezone = "America/New_York" + with self.sql_conf({"spark.sql.session.timeZone": timezone}): df_no_arrow_ny, df_arrow_ny = self._createDataFrame_toggle(pdf, schema=self.schema) result_ny = df_no_arrow_ny.collect() result_arrow_ny = df_arrow_ny.collect() diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index d0eeaa5c62182..89e687ab71295 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -745,10 +745,8 @@ def iter_internal_value(it): for internal_value, udf_type in [(scalar_internal_value, PandasUDFType.SCALAR), (iter_internal_value, PandasUDFType.SCALAR_ITER)]: f_timestamp_copy = pandas_udf(lambda ts: ts, TimestampType(), udf_type) - timezone = "America/New_York" - with self.sql_conf({ - "spark.sql.execution.pandas.respectSessionTimeZone": False, - "spark.sql.session.timeZone": timezone}): + timezone = "America/Los_Angeles" + with self.sql_conf({"spark.sql.session.timeZone": timezone}): df_la = df.withColumn("tscopy", f_timestamp_copy(col("timestamp"))) \ .withColumn("internal_value", internal_value(col("timestamp"))) result_la = df_la.select(col("idx"), col("internal_value")).collect() @@ -757,9 +755,8 @@ def iter_internal_value(it): result_la_corrected = \ df_la.select(col("idx"), col("tscopy"), col("internal_value") + diff).collect() - with self.sql_conf({ - "spark.sql.execution.pandas.respectSessionTimeZone": True, - "spark.sql.session.timeZone": timezone}): + timezone = "America/New_York" + with self.sql_conf({"spark.sql.session.timeZone": timezone}): df_ny = df.withColumn("tscopy", f_timestamp_copy(col("timestamp"))) \ .withColumn("internal_value", internal_value(col("timestamp"))) result_ny = df_ny.select(col("idx"), col("tscopy"), col("internal_value")).collect() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d63f4daf00e26..98a65515b6dfc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1609,15 +1609,6 @@ object SQLConf { "it might degrade performance. See SPARK-27870.") .fallbackConf(BUFFER_SIZE) - val PANDAS_RESPECT_SESSION_LOCAL_TIMEZONE = - buildConf("spark.sql.execution.pandas.respectSessionTimeZone") - .internal() - .doc("When true, make Pandas DataFrame with timestamp type respecting session local " + - "timezone when converting to/from Pandas DataFrame. This configuration will be " + - "deprecated in the future releases.") - .booleanConf - .createWithDefault(true) - val PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME = buildConf("spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName") .internal() @@ -2176,9 +2167,6 @@ object SQLConf { */ val deprecatedSQLConfigs: Map[String, DeprecatedConfig] = { val configs = Seq( - DeprecatedConfig(PANDAS_RESPECT_SESSION_LOCAL_TIMEZONE.key, "2.3", - "Behavior for `false` config value is considered as a bug, and " + - "it will be prohibited in the future releases."), DeprecatedConfig( PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key, "2.4", "The config allows to switch to the behaviour before Spark 2.4 " + @@ -2223,7 +2211,10 @@ object SQLConf { "It was deprecated since Spark 2.1, and not used in Spark 2.4."), RemovedConfig("spark.sql.parquet.int64AsTimestampMillis", "3.0.0", "false", "The config was deprecated since Spark 2.3." + - s"Use '${PARQUET_OUTPUT_TIMESTAMP_TYPE.key}' instead of it.") + s"Use '${PARQUET_OUTPUT_TIMESTAMP_TYPE.key}' instead of it."), + RemovedConfig("spark.sql.execution.pandas.respectSessionTimeZone", "3.0.0", "true", + "The non-default behavior is considered as a bug, see SPARK-22395. " + + "The config was deprecated since Spark 2.3.") ) Map(configs.map { cfg => cfg.key -> cfg } : _*) @@ -2634,8 +2625,6 @@ class SQLConf extends Serializable with Logging { def pandasUDFBufferSize: Int = getConf(PANDAS_UDF_BUFFER_SIZE) - def pandasRespectSessionTimeZone: Boolean = getConf(PANDAS_RESPECT_SESSION_LOCAL_TIMEZONE) - def pandasGroupedMapAssignColumnsByName: Boolean = getConf(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index 317eb635ba412..2da0d1a51cb29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -142,11 +142,7 @@ object ArrowUtils { /** Return Map with conf settings to be used in ArrowPythonRunner */ def getPythonRunnerConfMap(conf: SQLConf): Map[String, String] = { - val timeZoneConf = if (conf.pandasRespectSessionTimeZone) { - Seq(SQLConf.SESSION_LOCAL_TIMEZONE.key -> conf.sessionLocalTimeZone) - } else { - Nil - } + val timeZoneConf = Seq(SQLConf.SESSION_LOCAL_TIMEZONE.key -> conf.sessionLocalTimeZone) val pandasColsByName = Seq(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key -> conf.pandasGroupedMapAssignColumnsByName.toString) val arrowSafeTypeCheck = Seq(SQLConf.PANDAS_ARROW_SAFE_TYPE_CONVERSION.key -> From 4398dfa709598226517474afbf47cd9e3e384826 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 16 Jan 2020 19:27:30 -0800 Subject: [PATCH 1133/1387] [SPARK-29188][PYTHON][FOLLOW-UP] Explicitly disable Arrow execution for the test of toPandas empty types ### What changes were proposed in this pull request? This PR proposes to explicitly disable Arrow execution for the test of toPandas empty types. If `spark.sql.execution.arrow.pyspark.enabled` is enabled by default, this test alone fails as below: ``` ====================================================================== ERROR [0.205s]: test_to_pandas_from_empty_dataframe (pyspark.sql.tests.test_dataframe.DataFrameTests) ---------------------------------------------------------------------- Traceback (most recent call last): File "/.../pyspark/sql/tests/test_dataframe.py", line 568, in test_to_pandas_from_empty_dataframe self.assertTrue(np.all(dtypes_when_empty_df == dtypes_when_nonempty_df)) AssertionError: False is not true ---------------------------------------------------------------------- ``` it should be best to explicitly disable for the test that only works when it's disabled. ### Why are the changes needed? To make the test independent of default values of configuration. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested and Jenkins should test. Closes #27247 from HyukjinKwon/SPARK-29188-followup. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_dataframe.py | 33 +++++++++++----------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 433a26d42ae98..bffdd3e59c2d4 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -549,22 +549,23 @@ def test_to_pandas_avoid_astype(self): @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_empty_dataframe(self): - # SPARK-29188 test that toPandas() on an empty dataframe has the correct dtypes - import numpy as np - sql = """ - SELECT CAST(1 AS TINYINT) AS tinyint, - CAST(1 AS SMALLINT) AS smallint, - CAST(1 AS INT) AS int, - CAST(1 AS BIGINT) AS bigint, - CAST(0 AS FLOAT) AS float, - CAST(0 AS DOUBLE) AS double, - CAST(1 AS BOOLEAN) AS boolean, - CAST('foo' AS STRING) AS string, - CAST('2019-01-01' AS TIMESTAMP) AS timestamp - """ - dtypes_when_nonempty_df = self.spark.sql(sql).toPandas().dtypes - dtypes_when_empty_df = self.spark.sql(sql).filter("False").toPandas().dtypes - self.assertTrue(np.all(dtypes_when_empty_df == dtypes_when_nonempty_df)) + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): + # SPARK-29188 test that toPandas() on an empty dataframe has the correct dtypes + import numpy as np + sql = """ + SELECT CAST(1 AS TINYINT) AS tinyint, + CAST(1 AS SMALLINT) AS smallint, + CAST(1 AS INT) AS int, + CAST(1 AS BIGINT) AS bigint, + CAST(0 AS FLOAT) AS float, + CAST(0 AS DOUBLE) AS double, + CAST(1 AS BOOLEAN) AS boolean, + CAST('foo' AS STRING) AS string, + CAST('2019-01-01' AS TIMESTAMP) AS timestamp + """ + dtypes_when_nonempty_df = self.spark.sql(sql).toPandas().dtypes + dtypes_when_empty_df = self.spark.sql(sql).filter("False").toPandas().dtypes + self.assertTrue(np.all(dtypes_when_empty_df == dtypes_when_nonempty_df)) @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_null_dataframe(self): From 0bd7a3dfab41336dba2788a3d1fa3cf5b9f410d3 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 17 Jan 2020 12:40:51 +0800 Subject: [PATCH 1134/1387] [SPARK-29572][SQL] add v1 read fallback API in DS v2 ### What changes were proposed in this pull request? Add a `V1Scan` interface, so that data source v1 implementations can migrate to DS v2 much easier. ### Why are the changes needed? It's a lot of work to migrate v1 sources to DS v2. The new API added here can allow v1 sources to go through v2 code paths without implementing all the Batch, Stream, PartitionReaderFactory, ... stuff. We already have a v1 write fallback API after https://github.com/apache/spark/pull/25348 ### Does this PR introduce any user-facing change? no ### How was this patch tested? new test suite Closes #26231 from cloud-fan/v1-read-fallback. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/connector/InMemoryTableCatalog.scala | 4 +- .../spark/sql/connector/read/V1Scan.java | 43 ++++ .../sql/connector/write/V1WriteBuilder.java} | 19 +- .../spark/sql/execution/SparkPlanner.scala | 5 +- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../datasources/DataSourceStrategy.scala | 26 ++- .../datasources/v2/DataSourceV2Strategy.scala | 59 ++++-- .../datasources/v2/PushDownUtils.scala | 10 +- .../v2/V2ScanRelationPushDown.scala | 21 +- .../streaming/IncrementalExecution.scala | 2 +- .../internal/BaseSessionStateBuilder.scala | 2 +- .../sql/connector/V1ReadFallbackSuite.scala | 199 ++++++++++++++++++ .../sql/hive/HiveSessionStateBuilder.scala | 2 +- 13 files changed, 340 insertions(+), 54 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/connector/read/V1Scan.java rename sql/core/src/main/{scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala => java/org/apache/spark/sql/connector/write/V1WriteBuilder.java} (73%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala index 57c83ec68a649..6824efd9880a9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTableCatalog.scala @@ -34,8 +34,8 @@ class BasicInMemoryTableCatalog extends TableCatalog { protected val namespaces: util.Map[List[String], Map[String, String]] = new ConcurrentHashMap[List[String], Map[String, String]]() - protected val tables: util.Map[Identifier, InMemoryTable] = - new ConcurrentHashMap[Identifier, InMemoryTable]() + protected val tables: util.Map[Identifier, Table] = + new ConcurrentHashMap[Identifier, Table]() private val invalidatedTables: util.Set[Identifier] = ConcurrentHashMap.newKeySet() diff --git a/sql/core/src/main/java/org/apache/spark/sql/connector/read/V1Scan.java b/sql/core/src/main/java/org/apache/spark/sql/connector/read/V1Scan.java new file mode 100644 index 0000000000000..c9d7cb1bf80a3 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/connector/read/V1Scan.java @@ -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. + */ + +package org.apache.spark.sql.connector.read; + +import org.apache.spark.annotation.Unstable; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.sources.BaseRelation; +import org.apache.spark.sql.sources.TableScan; + +/** + * A trait that should be implemented by V1 DataSources that would like to leverage the DataSource + * V2 read code paths. + * + * This interface is designed to provide Spark DataSources time to migrate to DataSource V2 and + * will be removed in a future Spark release. + * + * @since 3.0.0 + */ +@Unstable +public interface V1Scan extends Scan { + + /** + * Create an `BaseRelation` with `TableScan` that can scan data from DataSource v1 to RDD[Row]. + * + * @since 3.0.0 + */ + T toV1TableScan(SQLContext context); +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala b/sql/core/src/main/java/org/apache/spark/sql/connector/write/V1WriteBuilder.java similarity index 73% rename from sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala rename to sql/core/src/main/java/org/apache/spark/sql/connector/write/V1WriteBuilder.java index e738ad1ede446..89b567b5231ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/connector/write/V1WriteBuilder.scala +++ b/sql/core/src/main/java/org/apache/spark/sql/connector/write/V1WriteBuilder.java @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.connector.write +package org.apache.spark.sql.connector.write; -import org.apache.spark.annotation.{Experimental, Unstable} -import org.apache.spark.sql.connector.write.streaming.StreamingWrite -import org.apache.spark.sql.sources.InsertableRelation +import org.apache.spark.annotation.Unstable; +import org.apache.spark.sql.sources.InsertableRelation; /** * A trait that should be implemented by V1 DataSources that would like to leverage the DataSource @@ -32,10 +31,8 @@ * * @since 3.0.0 */ -@Experimental @Unstable -trait V1WriteBuilder extends WriteBuilder { - +public interface V1WriteBuilder extends WriteBuilder { /** * Creates an InsertableRelation that allows appending a DataFrame to a * a destination (using data source-specific parameters). The insert method will only be @@ -44,11 +41,5 @@ * * @since 3.0.0 */ - def buildForV1Write(): InsertableRelation - - // These methods cannot be implemented by a V1WriteBuilder. The super class will throw - // an Unsupported OperationException - override final def buildForBatch(): BatchWrite = super.buildForBatch() - - override final def buildForStreaming(): StreamingWrite = super.buildForStreaming() + InsertableRelation buildForV1Write(); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala index dc7fb7741e7a7..895eeedd86b8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution -import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -27,7 +26,7 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy import org.apache.spark.sql.internal.SQLConf class SparkPlanner( - val sparkContext: SparkContext, + val session: SparkSession, val conf: SQLConf, val experimentalMethods: ExperimentalMethods) extends SparkStrategies { @@ -39,7 +38,7 @@ class SparkPlanner( extraPlanningStrategies ++ ( LogicalQueryStageStrategy :: PythonEvals :: - DataSourceV2Strategy :: + new DataSourceV2Strategy(session) :: FileSourceStrategy :: DataSourceStrategy(conf) :: SpecialLimits :: 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 418401ac4e5cc..00ad4e0fe0c11 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 @@ -570,7 +570,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } - protected lazy val singleRowRdd = sparkContext.parallelize(Seq(InternalRow()), 1) + protected lazy val singleRowRdd = session.sparkContext.parallelize(Seq(InternalRow()), 1) object InMemoryScans extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index d44cb11e28762..e3a0a0a6c34e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -409,14 +409,7 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with relation: LogicalRelation, output: Seq[Attribute], rdd: RDD[Row]): RDD[InternalRow] = { - if (relation.relation.needConversion) { - val converters = RowEncoder(StructType.fromAttributes(output)) - rdd.mapPartitions { iterator => - iterator.map(converters.toRow) - } - } else { - rdd.asInstanceOf[RDD[InternalRow]] - } + DataSourceStrategy.toCatalystRDD(relation.relation, output, rdd) } /** @@ -624,4 +617,21 @@ object DataSourceStrategy { (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, handledFilters) } + + /** + * Convert RDD of Row into RDD of InternalRow with objects in catalyst types + */ + private[sql] def toCatalystRDD( + relation: BaseRelation, + output: Seq[Attribute], + rdd: RDD[Row]): RDD[InternalRow] = { + if (relation.needConversion) { + val converters = RowEncoder(StructType.fromAttributes(output)) + rdd.mapPartitions { iterator => + iterator.map(converters.toRow) + } + } else { + rdd.asInstanceOf[RDD[InternalRow]] + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index b452b66e03813..568ffba4854cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -19,40 +19,69 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ -import org.apache.spark.sql.{AnalysisException, Strategy} +import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy} import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable} -import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, TableChange} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} -import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, ProjectExec, RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} +import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.util.CaseInsensitiveStringMap -object DataSourceV2Strategy extends Strategy with PredicateHelper { +class DataSourceV2Strategy(session: SparkSession) extends Strategy with PredicateHelper { import DataSourceV2Implicits._ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + private def withProjectAndFilter( + project: Seq[NamedExpression], + filters: Seq[Expression], + scan: LeafExecNode, + needsUnsafeConversion: Boolean): SparkPlan = { + val filterCondition = filters.reduceLeftOption(And) + val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan) + + if (withFilter.output != project || needsUnsafeConversion) { + ProjectExec(project, withFilter) + } else { + withFilter + } + } + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case PhysicalOperation(project, filters, + relation @ DataSourceV2ScanRelation(_, V1ScanWrapper(scan, translated, pushed), output)) => + val v1Relation = scan.toV1TableScan[BaseRelation with TableScan](session.sqlContext) + if (v1Relation.schema != scan.readSchema()) { + throw new IllegalArgumentException( + "The fallback v1 relation reports inconsistent schema:\n" + + "Schema of v2 scan: " + scan.readSchema() + "\n" + + "Schema of v1 relation: " + v1Relation.schema) + } + val rdd = v1Relation.buildScan() + val unsafeRowRDD = DataSourceStrategy.toCatalystRDD(v1Relation, output, rdd) + val originalOutputNames = relation.table.schema().map(_.name) + val requiredColumnsIndex = output.map(_.name).map(originalOutputNames.indexOf) + val dsScan = RowDataSourceScanExec( + output, + requiredColumnsIndex, + translated.toSet, + pushed.toSet, + unsafeRowRDD, + v1Relation, + tableIdentifier = None) + withProjectAndFilter(project, filters, dsScan, needsUnsafeConversion = false) :: Nil + case PhysicalOperation(project, filters, relation: DataSourceV2ScanRelation) => // projection and filters were already pushed down in the optimizer. // this uses PhysicalOperation to get the projection and ensure that if the batch scan does // not support columnar, a projection is added to convert the rows to UnsafeRow. val batchExec = BatchScanExec(relation.output, relation.scan) - - val filterCondition = filters.reduceLeftOption(And) - val withFilter = filterCondition.map(FilterExec(_, batchExec)).getOrElse(batchExec) - - val withProjection = if (withFilter.output != project || !batchExec.supportsColumnar) { - ProjectExec(project, withFilter) - } else { - withFilter - } - - withProjection :: Nil + withProjectAndFilter(project, filters, batchExec, !batchExec.supportsColumnar) :: Nil case r: StreamingDataSourceV2Relation if r.startOffset.isDefined && r.endOffset.isDefined => val microBatchStream = r.stream.asInstanceOf[MicroBatchStream] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala index 09a8a7ebb6ddc..33338b06565c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala @@ -34,7 +34,7 @@ object PushDownUtils extends PredicateHelper { */ def pushFilters( scanBuilder: ScanBuilder, - filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + filters: Seq[Expression]): (Seq[sources.Filter], Seq[Expression]) = { scanBuilder match { case r: SupportsPushDownFilters => // A map from translated data source leaf node filters to original catalyst filter @@ -62,11 +62,7 @@ object PushDownUtils extends PredicateHelper { val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) } - // The filters which are marked as pushed to this data source - val pushedFilters = r.pushedFilters().map { filter => - DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) - } - (pushedFilters, untranslatableExprs ++ postScanFilters) + (r.pushedFilters(), untranslatableExprs ++ postScanFilters) case _ => (Nil, filters) } @@ -75,7 +71,7 @@ object PushDownUtils extends PredicateHelper { /** * Applies column pruning to the data source, w.r.t. the references of the given expressions. * - * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), + * @return the `Scan` instance (since column pruning is the last step of operator pushdown), * and new output attributes after column pruning. */ def pruneColumns( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index 92acd3ba8d902..59089fa6b77e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -21,7 +21,10 @@ import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpressi import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.read.{Scan, V1Scan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType object V2ScanRelationPushDown extends Rule[LogicalPlan] { import DataSourceV2Implicits._ @@ -54,7 +57,14 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] { |Output: ${output.mkString(", ")} """.stripMargin) - val scanRelation = DataSourceV2ScanRelation(relation.table, scan, output) + val wrappedScan = scan match { + case v1: V1Scan => + val translated = filters.flatMap(DataSourceStrategy.translateFilter) + V1ScanWrapper(v1, translated, pushedFilters) + case _ => scan + } + + val scanRelation = DataSourceV2ScanRelation(relation.table, wrappedScan, output) val projectionOverSchema = ProjectionOverSchema(output.toStructType) val projectionFunc = (expr: Expression) => expr transformDown { @@ -77,3 +87,12 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] { withProjection } } + +// A wrapper for v1 scan to carry the translated filters and the handled ones. This is required by +// the physical v1 scan node. +case class V1ScanWrapper( + v1Scan: V1Scan, + translatedFilters: Seq[sources.Filter], + handledFilters: Seq[sources.Filter]) extends Scan { + override def readSchema(): StructType = v1Scan.readSchema() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index b8e18b89b54bc..bf80a0b1c167a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -50,7 +50,7 @@ class IncrementalExecution( // Modified planner with stateful operations. override val planner: SparkPlanner = new SparkPlanner( - sparkSession.sparkContext, + sparkSession, sparkSession.sessionState.conf, sparkSession.sessionState.experimentalMethods) { override def strategies: Seq[Strategy] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 91c693ab34c8e..eb658e2d8850e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -250,7 +250,7 @@ abstract class BaseSessionStateBuilder( * Note: this depends on the `conf` and `experimentalMethods` fields. */ protected def planner: SparkPlanner = { - new SparkPlanner(session.sparkContext, conf, experimentalMethods) { + new SparkPlanner(session, conf, experimentalMethods) { override def extraPlanningStrategies: Seq[Strategy] = super.extraPlanningStrategies ++ customPlanningStrategies } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala new file mode 100644 index 0000000000000..8e2c63417b377 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala @@ -0,0 +1,199 @@ +/* + * 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.connector + +import java.util + +import scala.collection.JavaConverters._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, QueryTest, Row, SparkSession, SQLContext} +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns, V1Scan} +import org.apache.spark.sql.execution.RowDataSourceScanExec +import org.apache.spark.sql.sources.{BaseRelation, Filter, GreaterThan, TableScan} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +abstract class V1ReadFallbackSuite extends QueryTest with SharedSparkSession { + protected def baseTableScan(): DataFrame + + test("full scan") { + val df = baseTableScan() + val v1Scan = df.queryExecution.executedPlan.collect { + case s: RowDataSourceScanExec => s + } + assert(v1Scan.length == 1) + checkAnswer(df, Seq(Row(1, 10), Row(2, 20), Row(3, 30))) + } + + test("column pruning") { + val df = baseTableScan().select("i") + val v1Scan = df.queryExecution.executedPlan.collect { + case s: RowDataSourceScanExec => s + } + assert(v1Scan.length == 1) + assert(v1Scan.head.output.map(_.name) == Seq("i")) + checkAnswer(df, Seq(Row(1), Row(2), Row(3))) + } + + test("filter push down") { + val df = baseTableScan().filter("i > 1 and j < 30") + val v1Scan = df.queryExecution.executedPlan.collect { + case s: RowDataSourceScanExec => s + } + assert(v1Scan.length == 1) + // `j < 30` can't be pushed. + assert(v1Scan.head.handledFilters.size == 1) + checkAnswer(df, Seq(Row(2, 20))) + } + + test("filter push down + column pruning") { + val df = baseTableScan().filter("i > 1").select("i") + val v1Scan = df.queryExecution.executedPlan.collect { + case s: RowDataSourceScanExec => s + } + assert(v1Scan.length == 1) + assert(v1Scan.head.output.map(_.name) == Seq("i")) + assert(v1Scan.head.handledFilters.size == 1) + checkAnswer(df, Seq(Row(2), Row(3))) + } +} + +class V1ReadFallbackWithDataFrameReaderSuite extends V1ReadFallbackSuite { + override protected def baseTableScan(): DataFrame = { + spark.read.format(classOf[V1ReadFallbackTableProvider].getName).load() + } +} + +class V1ReadFallbackWithCatalogSuite extends V1ReadFallbackSuite { + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set("spark.sql.catalog.read_fallback", classOf[V1ReadFallbackCatalog].getName) + sql("CREATE TABLE read_fallback.tbl(i int, j int) USING foo") + } + + override def afterAll(): Unit = { + spark.conf.unset("spark.sql.catalog.read_fallback") + super.afterAll() + } + + override protected def baseTableScan(): DataFrame = { + spark.table("read_fallback.tbl") + } +} + +class V1ReadFallbackCatalog extends BasicInMemoryTableCatalog { + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + // To simplify the test implementation, only support fixed schema. + if (schema != V1ReadFallbackCatalog.schema || partitions.nonEmpty) { + throw new UnsupportedOperationException + } + val table = new TableWithV1ReadFallback(ident.toString) + tables.put(ident, table) + table + } +} + +object V1ReadFallbackCatalog { + val schema = new StructType().add("i", "int").add("j", "int") +} + +class V1ReadFallbackTableProvider extends TableProvider { + override def getTable(options: CaseInsensitiveStringMap): Table = { + new TableWithV1ReadFallback("v1-read-fallback") + } +} + +class TableWithV1ReadFallback(override val name: String) extends Table with SupportsRead { + + override def schema(): StructType = V1ReadFallbackCatalog.schema + + override def capabilities(): util.Set[TableCapability] = { + Set(TableCapability.BATCH_READ).asJava + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new V1ReadFallbackScanBuilder + } + + private class V1ReadFallbackScanBuilder extends ScanBuilder + with SupportsPushDownRequiredColumns with SupportsPushDownFilters { + + private var requiredSchema: StructType = schema() + override def pruneColumns(requiredSchema: StructType): Unit = { + this.requiredSchema = requiredSchema + } + + private var filters: Array[Filter] = Array.empty + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + val (supported, unsupported) = filters.partition { + case GreaterThan("i", _: Int) => true + case _ => false + } + this.filters = supported + unsupported + } + override def pushedFilters(): Array[Filter] = filters + + override def build(): Scan = new V1ReadFallbackScan(requiredSchema, filters) + } + + private class V1ReadFallbackScan( + requiredSchema: StructType, + filters: Array[Filter]) extends V1Scan { + override def readSchema(): StructType = requiredSchema + + override def toV1TableScan[T <: BaseRelation with TableScan](context: SQLContext): T = { + new V1TableScan(context, requiredSchema, filters).asInstanceOf[T] + } + } +} + +class V1TableScan( + context: SQLContext, + requiredSchema: StructType, + filters: Array[Filter]) extends BaseRelation with TableScan { + override def sqlContext: SQLContext = context + override def schema: StructType = requiredSchema + override def buildScan(): RDD[Row] = { + val lowerBound = if (filters.isEmpty) { + 0 + } else { + filters.collect { case GreaterThan("i", v: Int) => v }.max + } + val data = Seq(Row(1, 10), Row(2, 20), Row(3, 30)).filter(_.getInt(0) > lowerBound) + val result = if (requiredSchema.length == 2) { + data + } else if (requiredSchema.map(_.name) == Seq("i")) { + data.map(row => Row(row.getInt(0))) + } else if (requiredSchema.map(_.name) == Seq("j")) { + data.map(row => Row(row.getInt(1))) + } else { + throw new UnsupportedOperationException + } + + SparkSession.active.sparkContext.makeRDD(result) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 3df77fec20993..de21a13e6edb8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -97,7 +97,7 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session * Planner that takes into account Hive-specific strategies. */ override protected def planner: SparkPlanner = { - new SparkPlanner(session.sparkContext, conf, experimentalMethods) with HiveStrategies { + new SparkPlanner(session, conf, experimentalMethods) with HiveStrategies { override val sparkSession: SparkSession = session override def extraPlanningStrategies: Seq[Strategy] = From 1881caa95e336c660d2da89e5f67fe5edddfb73d Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 17 Jan 2020 15:00:18 +0900 Subject: [PATCH 1135/1387] [SPARK-29188][PYTHON][FOLLOW-UP] Explicitly disable Arrow execution for all test of toPandas empty types ### What changes were proposed in this pull request? Another followup of https://github.com/apache/spark/commit/4398dfa709598226517474afbf47cd9e3e384826 I missed two more tests added: ``` ====================================================================== ERROR [0.133s]: test_to_pandas_from_mixed_dataframe (pyspark.sql.tests.test_dataframe.DataFrameTests) ---------------------------------------------------------------------- Traceback (most recent call last): File "/home/jenkins/python/pyspark/sql/tests/test_dataframe.py", line 617, in test_to_pandas_from_mixed_dataframe self.assertTrue(np.all(pdf_with_only_nulls.dtypes == pdf_with_some_nulls.dtypes)) AssertionError: False is not true ====================================================================== ERROR [0.061s]: test_to_pandas_from_null_dataframe (pyspark.sql.tests.test_dataframe.DataFrameTests) ---------------------------------------------------------------------- Traceback (most recent call last): File "/home/jenkins/python/pyspark/sql/tests/test_dataframe.py", line 588, in test_to_pandas_from_null_dataframe self.assertEqual(types[0], np.float64) AssertionError: dtype('O') != ---------------------------------------------------------------------- ``` ### Why are the changes needed? To make the test independent of default values of configuration. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested and Jenkins should test. Closes #27250 from HyukjinKwon/SPARK-29188-followup2. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/tests/test_dataframe.py | 86 +++++++++++----------- 1 file changed, 44 insertions(+), 42 deletions(-) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index bffdd3e59c2d4..d738449799bda 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -569,51 +569,53 @@ def test_to_pandas_from_empty_dataframe(self): @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_null_dataframe(self): - # SPARK-29188 test that toPandas() on a dataframe with only nulls has correct dtypes - import numpy as np - sql = """ - SELECT CAST(NULL AS TINYINT) AS tinyint, - CAST(NULL AS SMALLINT) AS smallint, - CAST(NULL AS INT) AS int, - CAST(NULL AS BIGINT) AS bigint, - CAST(NULL AS FLOAT) AS float, - CAST(NULL AS DOUBLE) AS double, - CAST(NULL AS BOOLEAN) AS boolean, - CAST(NULL AS STRING) AS string, - CAST(NULL AS TIMESTAMP) AS timestamp - """ - pdf = self.spark.sql(sql).toPandas() - types = pdf.dtypes - self.assertEqual(types[0], np.float64) - self.assertEqual(types[1], np.float64) - self.assertEqual(types[2], np.float64) - self.assertEqual(types[3], np.float64) - self.assertEqual(types[4], np.float32) - self.assertEqual(types[5], np.float64) - self.assertEqual(types[6], np.object) - self.assertEqual(types[7], np.object) - self.assertTrue(np.can_cast(np.datetime64, types[8])) + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): + # SPARK-29188 test that toPandas() on a dataframe with only nulls has correct dtypes + import numpy as np + sql = """ + SELECT CAST(NULL AS TINYINT) AS tinyint, + CAST(NULL AS SMALLINT) AS smallint, + CAST(NULL AS INT) AS int, + CAST(NULL AS BIGINT) AS bigint, + CAST(NULL AS FLOAT) AS float, + CAST(NULL AS DOUBLE) AS double, + CAST(NULL AS BOOLEAN) AS boolean, + CAST(NULL AS STRING) AS string, + CAST(NULL AS TIMESTAMP) AS timestamp + """ + pdf = self.spark.sql(sql).toPandas() + types = pdf.dtypes + self.assertEqual(types[0], np.float64) + self.assertEqual(types[1], np.float64) + self.assertEqual(types[2], np.float64) + self.assertEqual(types[3], np.float64) + self.assertEqual(types[4], np.float32) + self.assertEqual(types[5], np.float64) + self.assertEqual(types[6], np.object) + self.assertEqual(types[7], np.object) + self.assertTrue(np.can_cast(np.datetime64, types[8])) @unittest.skipIf(not have_pandas, pandas_requirement_message) def test_to_pandas_from_mixed_dataframe(self): - # SPARK-29188 test that toPandas() on a dataframe with some nulls has correct dtypes - import numpy as np - sql = """ - SELECT CAST(col1 AS TINYINT) AS tinyint, - CAST(col2 AS SMALLINT) AS smallint, - CAST(col3 AS INT) AS int, - CAST(col4 AS BIGINT) AS bigint, - CAST(col5 AS FLOAT) AS float, - CAST(col6 AS DOUBLE) AS double, - CAST(col7 AS BOOLEAN) AS boolean, - CAST(col8 AS STRING) AS string, - CAST(col9 AS TIMESTAMP) AS timestamp - FROM VALUES (1, 1, 1, 1, 1, 1, 1, 1, 1), - (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL) - """ - pdf_with_some_nulls = self.spark.sql(sql).toPandas() - pdf_with_only_nulls = self.spark.sql(sql).filter('tinyint is null').toPandas() - self.assertTrue(np.all(pdf_with_only_nulls.dtypes == pdf_with_some_nulls.dtypes)) + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): + # SPARK-29188 test that toPandas() on a dataframe with some nulls has correct dtypes + import numpy as np + sql = """ + SELECT CAST(col1 AS TINYINT) AS tinyint, + CAST(col2 AS SMALLINT) AS smallint, + CAST(col3 AS INT) AS int, + CAST(col4 AS BIGINT) AS bigint, + CAST(col5 AS FLOAT) AS float, + CAST(col6 AS DOUBLE) AS double, + CAST(col7 AS BOOLEAN) AS boolean, + CAST(col8 AS STRING) AS string, + CAST(col9 AS TIMESTAMP) AS timestamp + FROM VALUES (1, 1, 1, 1, 1, 1, 1, 1, 1), + (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL) + """ + pdf_with_some_nulls = self.spark.sql(sql).toPandas() + pdf_with_only_nulls = self.spark.sql(sql).filter('tinyint is null').toPandas() + self.assertTrue(np.all(pdf_with_only_nulls.dtypes == pdf_with_some_nulls.dtypes)) def test_create_dataframe_from_array_of_long(self): import array From 64fe192fef9bd2fd3c0fca536e569a8d5dfa78f8 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 17 Jan 2020 16:51:44 +0800 Subject: [PATCH 1136/1387] [SPARK-30282][SQL] Migrate SHOW TBLPROPERTIES to new framework ### What changes were proposed in this pull request? Use the new framework to resolve the SHOW TBLPROPERTIES command. This PR along with #27243 should update all the existing V2 commands with `UnresolvedV2Relation`. ### Why are the changes needed? This is a part of effort to make the relation lookup behavior consistent: [SPARK-2990](https://issues.apache.org/jira/browse/SPARK-29900). ### Does this PR introduce any user-facing change? Yes `SHOW TBLPROPERTIES temp_view` now fails with `AnalysisException` will be thrown with a message `temp_view is a temp view not table`. Previously, it was returning empty row. ### How was this patch tested? Existing tests Closes #26921 from imback82/consistnet_v2command. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 ++ .../sql/catalyst/analysis/Analyzer.scala | 5 ---- .../catalyst/analysis/ResolveCatalogs.scala | 5 ---- .../sql/catalyst/parser/AstBuilder.scala | 4 +-- .../catalyst/plans/logical/statements.scala | 7 ------ .../catalyst/plans/logical/v2Commands.scala | 6 +++-- .../sql/catalyst/parser/DDLParserSuite.scala | 4 +-- .../analysis/ResolveSessionCatalog.scala | 6 ++--- .../spark/sql/execution/command/tables.scala | 25 +++++++------------ .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../sql/hive/execution/HiveCommandSuite.scala | 12 +++++---- 11 files changed, 29 insertions(+), 49 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index bb086ff657e41..04ec327b7cd75 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -338,6 +338,8 @@ license: | - Since Spark 3.0, `ADD FILE` can be used to add file directories as well. Earlier only single files can be added using this command. To restore the behaviour of earlier versions, set `spark.sql.legacy.addDirectory.recursive` to false. + - Since Spark 3.0, `SHOW TBLPROPERTIES` on a temporary view will cause `AnalysisException`. In Spark version 2.4 and earlier, it returned an empty result. + ## Upgrading from Spark SQL 2.4 to 2.4.1 - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6e0e257038fa0..5317af494d8e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -820,11 +820,6 @@ class Analyzer( .map(rel => alter.copy(table = rel)) .getOrElse(alter) - case show @ ShowTableProperties(u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.tableName) - .map(rel => show.copy(table = rel)) - .getOrElse(show) - case u: UnresolvedV2Relation => CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index b79dcd0c1ecad..88a3c0a73a10b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -191,11 +191,6 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case ShowCurrentNamespaceStatement() => ShowCurrentNamespace(catalogManager) - - case ShowTablePropertiesStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), propertyKey) => - val r = UnresolvedV2Relation(nameParts, catalog.asTableCatalog, tbl.asIdentifier) - ShowTableProperties(r, propertyKey) } object NonSessionCatalogAndTable { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 5ddeff50e47b4..f744787c9082a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3503,8 +3503,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitShowTblProperties( ctx: ShowTblPropertiesContext): LogicalPlan = withOrigin(ctx) { - ShowTablePropertiesStatement( - visitMultipartIdentifier(ctx.table), + ShowTableProperties( + UnresolvedTable(visitMultipartIdentifier(ctx.table)), Option(ctx.key).map(visitTablePropertyKey)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 1e097899602a3..44f7b4143926d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -438,13 +438,6 @@ case class ShowColumnsStatement( */ case class ShowCurrentNamespaceStatement() extends ParsedStatement -/** - * A SHOW TBLPROPERTIES statement, as parsed from SQL - */ -case class ShowTablePropertiesStatement( - tableName: Seq[String], - propertyKey: Option[String]) extends ParsedStatement - /** * A DESCRIBE FUNCTION statement, as parsed from SQL */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index e98b2cf7abfcc..e1e7eac4cc085 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -474,8 +474,10 @@ case class ShowCurrentNamespace(catalogManager: CatalogManager) extends Command * The logical plan of the SHOW TBLPROPERTIES command that works for v2 catalogs. */ case class ShowTableProperties( - table: NamedRelation, - propertyKey: Option[String]) extends Command{ + table: LogicalPlan, + propertyKey: Option[String]) extends Command { + override def children: Seq[LogicalPlan] = table :: Nil + override val output: Seq[Attribute] = Seq( AttributeReference("key", StringType, nullable = false)(), AttributeReference("value", StringType, nullable = false)()) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 3a4c08235731e..47387fa18411e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1912,11 +1912,11 @@ class DDLParserSuite extends AnalysisTest { test("SHOW TBLPROPERTIES table") { comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c"), - ShowTablePropertiesStatement(Seq("a", "b", "c"), None)) + ShowTableProperties(UnresolvedTable(Seq("a", "b", "c")), None)) comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c('propKey1')"), - ShowTablePropertiesStatement(Seq("a", "b", "c"), Some("propKey1"))) + ShowTableProperties(UnresolvedTable(Seq("a", "b", "c")), Some("propKey1"))) } test("DESCRIBE FUNCTION") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 106fdc4333960..8b0d339dbb864 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -485,10 +485,8 @@ class ResolveSessionCatalog( replace, viewType) - case ShowTablePropertiesStatement(SessionCatalogAndTable(_, tbl), propertyKey) => - ShowTablePropertiesCommand( - tbl.asTableIdentifier, - propertyKey) + case ShowTableProperties(r: ResolvedTable, propertyKey) if isSessionCatalog(r.catalog) => + ShowTablePropertiesCommand(r.identifier.asTableIdentifier, propertyKey) case DescribeFunctionStatement(CatalogAndIdentifier(catalog, ident), extended) => val functionIdent = 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 447d00c11e7cf..a92fbdf25975b 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 @@ -900,22 +900,15 @@ case class ShowTablePropertiesCommand(table: TableIdentifier, propertyKey: Optio } override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog - - if (catalog.isTemporaryTable(table)) { - Seq.empty[Row] - } else { - val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(table) - - propertyKey match { - case Some(p) => - val propValue = catalogTable - .properties - .getOrElse(p, s"Table ${catalogTable.qualifiedName} does not have property: $p") - Seq(Row(propValue)) - case None => - catalogTable.properties.map(p => Row(p._1, p._2)).toSeq - } + val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(table) + propertyKey match { + case Some(p) => + val propValue = catalogTable + .properties + .getOrElse(p, s"Table ${catalogTable.qualifiedName} does not have property: $p") + Seq(Row(propValue)) + case None => + catalogTable.properties.map(p => Row(p._1, p._2)).toSeq } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 568ffba4854cc..d848e34648667 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -278,7 +278,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case r: ShowCurrentNamespace => ShowCurrentNamespaceExec(r.output, r.catalogManager) :: Nil - case r @ ShowTableProperties(DataSourceV2Relation(table, _, _), propertyKey) => + case r @ ShowTableProperties(ResolvedTable(_, _, table), propertyKey) => ShowTablePropertiesExec(r.output, table, propertyKey) :: Nil case AlterNamespaceSetOwner(ResolvedNamespace(catalog, namespace), name, typ) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 07cff1e7d99f0..dbbf2b29fe8b7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -130,10 +130,10 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } test("show tblproperties for datasource table - errors") { - val message1 = intercept[NoSuchTableException] { + val message = intercept[AnalysisException] { sql("SHOW TBLPROPERTIES badtable") }.getMessage - assert(message1.contains("Table or view 'badtable' not found in database 'default'")) + assert(message.contains("Table not found: badtable")) // When key is not found, a row containing the error is returned. checkAnswer( @@ -147,7 +147,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('`prop2Key`')"), Row("prop2Val")) } - test("show tblproperties for spark temporary table - empty row") { + test("show tblproperties for spark temporary table - AnalysisException is thrown") { withTempView("parquet_temp") { sql( """ @@ -155,8 +155,10 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto |USING org.apache.spark.sql.parquet.DefaultSource """.stripMargin) - // An empty sequence of row is returned for session temporary table. - checkAnswer(sql("SHOW TBLPROPERTIES parquet_temp"), Nil) + val message = intercept[AnalysisException] { + sql("SHOW TBLPROPERTIES parquet_temp") + }.getMessage + assert(message.contains("parquet_temp is a temp view not table")) } } From 6dbfa2bb9c5215aab97ec3f86b3325a11a7ff4d1 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 17 Jan 2020 08:15:25 -0600 Subject: [PATCH 1137/1387] [SPARK-29306][CORE] Stage Level Sched: Executors need to track what ResourceProfile they are created with ### What changes were proposed in this pull request? This is the second PR for the Stage Level Scheduling. This is adding in the necessary executor side changes: 1) executors to know what ResourceProfile they should be using 2) handle parsing the resource profile settings - these are not in the global configs 3) then reporting back to the driver what resource profile it was started with. This PR adds all the piping for YARN to pass the information all the way to executors, but it just uses the default ResourceProfile (which is the global applicatino level configs). At a high level these changes include: 1) adding a new --resourceProfileId option to the CoarseGrainedExecutorBackend 2) Add the ResourceProfile settings to new internal confs that gets passed into the Executor 3) Executor changes that use the resource profile id passed in to read the corresponding ResourceProfile confs and then parse those requests and discover resources as necessary 4) Executor registers to Driver with the Resource profile id so that the ExecutorMonitor can track how many executor with each profile are running 5) YARN side changes to show that passing the resource profile id and confs actually works. Just uses the DefaultResourceProfile for now. I also removed a check from the CoarseGrainedExecutorBackend that used to check to make sure there were task requirements before parsing any custom resource executor requests. With the resource profiles this becomes much more expensive because we would then have to pass the task requests to each executor and the check was just a short cut and not really needed. It was much cleaner just to remove it. Note there were some changes to the ResourceProfile, ExecutorResourceRequests, and TaskResourceRequests in this PR as well because I discovered some issues with things not being immutable. That api now look like: val rpBuilder = new ResourceProfileBuilder() val ereq = new ExecutorResourceRequests() val treq = new TaskResourceRequests() ereq.cores(2).memory("6g").memoryOverhead("2g").pysparkMemory("2g").resource("gpu", 2, "/home/tgraves/getGpus") treq.cpus(2).resource("gpu", 2) val resourceProfile = rpBuilder.require(ereq).require(treq).build This makes is so that ResourceProfile is immutable and Spark can use it directly without worrying about the user changing it. ### Why are the changes needed? These changes are needed for the executor to report which ResourceProfile they are using so that ultimately the dynamic allocation manager can use that information to know how many with a profile are running and how many more it needs to request. Its also needed to get the resource profile confs to the executor so that it can run the appropriate discovery script if needed. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit tests and manually on YARN. Closes #26682 from tgravescs/SPARK-29306. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../CoarseGrainedExecutorBackend.scala | 56 +++--- .../resource/ExecutorResourceRequest.scala | 39 +++-- .../resource/ExecutorResourceRequests.scala | 37 ++-- .../spark/resource/ResourceProfile.scala | 160 +++++++++++------- .../resource/ResourceProfileBuilder.scala | 84 +++++++++ .../apache/spark/resource/ResourceUtils.scala | 113 +++++++++++-- .../spark/resource/TaskResourceRequest.scala | 20 ++- .../spark/resource/TaskResourceRequests.scala | 25 ++- .../cluster/CoarseGrainedClusterMessage.scala | 10 +- .../CoarseGrainedSchedulerBackend.scala | 21 ++- .../scheduler/cluster/ExecutorData.scala | 7 +- .../scheduler/cluster/ExecutorInfo.scala | 32 +++- .../scheduler/dynalloc/ExecutorMonitor.scala | 60 ++++++- .../resource/JavaResourceProfileSuite.java | 2 +- .../ExecutorAllocationManagerSuite.scala | 6 +- .../apache/spark/HeartbeatReceiverSuite.scala | 5 +- .../org/apache/spark/LocalSparkContext.scala | 3 + .../StandaloneDynamicAllocationSuite.scala | 5 +- .../CoarseGrainedExecutorBackendSuite.scala | 130 +++++++++----- .../spark/resource/ResourceProfileSuite.scala | 114 +++++++------ .../spark/resource/ResourceUtilsSuite.scala | 37 +++- .../CoarseGrainedSchedulerBackendSuite.scala | 24 ++- .../dynalloc/ExecutorMonitorSuite.scala | 57 ++++--- project/MimaExcludes.scala | 8 + ...osCoarseGrainedSchedulerBackendSuite.scala | 3 +- .../spark/deploy/yarn/ApplicationMaster.scala | 4 +- .../spark/deploy/yarn/ExecutorRunnable.scala | 11 +- .../spark/deploy/yarn/YarnAllocator.scala | 4 +- .../YarnCoarseGrainedExecutorBackend.scala | 13 +- 29 files changed, 755 insertions(+), 335 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala 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 1fe901a83af33..f56e7c6d78c48 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -35,6 +35,8 @@ import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceProfile +import org.apache.spark.resource.ResourceProfile._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} @@ -51,7 +53,8 @@ private[spark] class CoarseGrainedExecutorBackend( cores: Int, userClassPath: Seq[URL], env: SparkEnv, - resourcesFileOpt: Option[String]) + resourcesFileOpt: Option[String], + resourceProfile: ResourceProfile) extends IsolatedRpcEndpoint with ExecutorBackend with Logging { import CoarseGrainedExecutorBackend._ @@ -80,7 +83,7 @@ private[spark] class CoarseGrainedExecutorBackend( // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls, - extractAttributes, resources)) + extractAttributes, resources, resourceProfile.id)) }(ThreadUtils.sameThread).onComplete { case Success(_) => self.send(RegisteredExecutor) @@ -91,24 +94,13 @@ private[spark] class CoarseGrainedExecutorBackend( // visible for testing def parseOrFindResources(resourcesFileOpt: Option[String]): Map[String, ResourceInformation] = { - // only parse the resources if a task requires them - val resourceInfo = if (parseResourceRequirements(env.conf, SPARK_TASK_PREFIX).nonEmpty) { - val resources = getOrDiscoverAllResources(env.conf, SPARK_EXECUTOR_PREFIX, resourcesFileOpt) - if (resources.isEmpty) { - throw new SparkException("User specified resources per task via: " + - s"$SPARK_TASK_PREFIX, but can't find any resources available on the executor.") - } else { - logResourceInfo(SPARK_EXECUTOR_PREFIX, resources) - } - resources - } else { - if (resourcesFileOpt.nonEmpty) { - logWarning("A resources file was specified but the application is not configured " + - s"to use any resources, see the configs with prefix: ${SPARK_TASK_PREFIX}") - } - Map.empty[String, ResourceInformation] - } - resourceInfo + logDebug(s"Resource profile id is: ${resourceProfile.id}") + val resources = getOrDiscoverAllResourcesForResourceProfile( + resourcesFileOpt, + SPARK_EXECUTOR_PREFIX, + resourceProfile) + logResourceInfo(SPARK_EXECUTOR_PREFIX, resources) + resources } def extractLogUrls: Map[String, String] = { @@ -237,14 +229,15 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { appId: String, workerUrl: Option[String], userClassPath: mutable.ListBuffer[URL], - resourcesFileOpt: Option[String]) + resourcesFileOpt: Option[String], + resourceProfileId: Int) def main(args: Array[String]): Unit = { - val createFn: (RpcEnv, Arguments, SparkEnv) => - CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => + val createFn: (RpcEnv, Arguments, SparkEnv, ResourceProfile) => + CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env, resourceProfile) => new CoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, - arguments.resourcesFileOpt) + arguments.resourcesFileOpt, resourceProfile) } run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn) System.exit(0) @@ -252,7 +245,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { def run( arguments: Arguments, - backendCreateFn: (RpcEnv, Arguments, SparkEnv) => CoarseGrainedExecutorBackend): Unit = { + backendCreateFn: (RpcEnv, Arguments, SparkEnv, ResourceProfile) => + CoarseGrainedExecutorBackend): Unit = { Utils.initDaemon(log) @@ -284,7 +278,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } } - val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig) + val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig(arguments.resourceProfileId)) val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", arguments.appId)) fetcher.shutdown() @@ -307,7 +301,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.bindAddress, arguments.hostname, arguments.cores, cfg.ioEncryptionKey, isLocal = false) - env.rpcEnv.setupEndpoint("Executor", backendCreateFn(env.rpcEnv, arguments, env)) + env.rpcEnv.setupEndpoint("Executor", + backendCreateFn(env.rpcEnv, arguments, env, cfg.resourceProfile)) arguments.workerUrl.foreach { url => env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } @@ -325,6 +320,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { var appId: String = null var workerUrl: Option[String] = None val userClassPath = new mutable.ListBuffer[URL]() + var resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID var argv = args.toList while (!argv.isEmpty) { @@ -357,6 +353,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case ("--user-class-path") :: value :: tail => userClassPath += new URL(value) argv = tail + case ("--resourceProfileId") :: value :: tail => + resourceProfileId = value.toInt + argv = tail case Nil => case tail => // scalastyle:off println @@ -380,7 +379,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } Arguments(driverUrl, executorId, bindAddress, hostname, cores, appId, workerUrl, - userClassPath, resourcesFileOpt) + userClassPath, resourcesFileOpt, resourceProfileId) } private def printUsageAndExit(classNameForEntry: String): Unit = { @@ -399,6 +398,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | --app-id | --worker-url | --user-class-path + | --resourceProfileId |""".stripMargin) // scalastyle:on println System.exit(1) diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala index 88ceaad69b672..9a920914ed674 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala @@ -17,10 +17,6 @@ package org.apache.spark.resource -import scala.collection.mutable - -import org.apache.spark.resource.ResourceUtils.RESOURCE_DOT - /** * An Executor resource request. This is used in conjunction with the ResourceProfile to * programmatically specify the resources needed for an RDD that will be applied at the @@ -28,16 +24,13 @@ import org.apache.spark.resource.ResourceUtils.RESOURCE_DOT * * This is used to specify what the resource requirements are for an Executor and how * Spark can find out specific details about those resources. Not all the parameters are - * required for every resource type. The resources names supported - * correspond to the regular Spark configs with the prefix removed. For instance overhead - * memory in this api is memoryOverhead, which is spark.executor.memoryOverhead with - * spark.executor removed. Resources like GPUs are resource.gpu - * (spark configs spark.executor.resource.gpu.*). The amount, discoveryScript, and vendor - * parameters for resources are all the same parameters a user would specify through the + * required for every resource type. Resources like GPUs are supported and have same limitations + * as using the global spark configs spark.executor.resource.gpu.*. The amount, discoveryScript, + * and vendor parameters for resources are all the same parameters a user would specify through the * configs: spark.executor.resource.{resourceName}.{amount, discoveryScript, vendor}. * * For instance, a user wants to allocate an Executor with GPU resources on YARN. The user has - * to specify the resource name (resource.gpu), the amount or number of GPUs per Executor, + * to specify the resource name (gpu), the amount or number of GPUs per Executor, * the discovery script would be specified so that when the Executor starts up it can * discovery what GPU addresses are available for it to use because YARN doesn't tell * Spark that, then vendor would not be used because its specific for Kubernetes. @@ -63,15 +56,21 @@ private[spark] class ExecutorResourceRequest( val discoveryScript: String = "", val vendor: String = "") extends Serializable { - // A list of allowed Spark internal resources. Custom resources (spark.executor.resource.*) - // like GPUs/FPGAs are also allowed, see the check below. - private val allowedExecutorResources = mutable.HashSet[String]( - ResourceProfile.MEMORY, - ResourceProfile.OVERHEAD_MEM, - ResourceProfile.PYSPARK_MEM, - ResourceProfile.CORES) + override def equals(obj: Any): Boolean = { + obj match { + case that: ExecutorResourceRequest => + that.getClass == this.getClass && + that.resourceName == resourceName && that.amount == amount && + that.discoveryScript == discoveryScript && that.vendor == vendor + case _ => + false + } + } + + override def hashCode(): Int = + Seq(resourceName, amount, discoveryScript, vendor).hashCode() - if (!allowedExecutorResources.contains(resourceName) && !resourceName.startsWith(RESOURCE_DOT)) { - throw new IllegalArgumentException(s"Executor resource not allowed: $resourceName") + override def toString(): String = { + s"name: $resourceName, amount: $amount, script: $discoveryScript, vendor: $vendor" } } diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index 6ffcc0c296468..d345674d6635c 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -17,7 +17,9 @@ package org.apache.spark.resource -import scala.collection.mutable +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ import org.apache.spark.network.util.JavaUtils import org.apache.spark.resource.ResourceProfile._ @@ -32,9 +34,9 @@ import org.apache.spark.resource.ResourceProfile._ */ private[spark] class ExecutorResourceRequests() extends Serializable { - private val _executorResources = new mutable.HashMap[String, ExecutorResourceRequest]() + private val _executorResources = new ConcurrentHashMap[String, ExecutorResourceRequest]() - def requests: Map[String, ExecutorResourceRequest] = _executorResources.toMap + def requests: Map[String, ExecutorResourceRequest] = _executorResources.asScala.toMap /** * Specify heap memory. The value specified will be converted to MiB. @@ -44,8 +46,8 @@ private[spark] class ExecutorResourceRequests() extends Serializable { */ def memory(amount: String): this.type = { val amountMiB = JavaUtils.byteStringAsMb(amount) - val rr = new ExecutorResourceRequest(MEMORY, amountMiB) - _executorResources(MEMORY) = rr + val req = new ExecutorResourceRequest(MEMORY, amountMiB) + _executorResources.put(MEMORY, req) this } @@ -57,8 +59,8 @@ private[spark] class ExecutorResourceRequests() extends Serializable { */ def memoryOverhead(amount: String): this.type = { val amountMiB = JavaUtils.byteStringAsMb(amount) - val rr = new ExecutorResourceRequest(OVERHEAD_MEM, amountMiB) - _executorResources(OVERHEAD_MEM) = rr + val req = new ExecutorResourceRequest(OVERHEAD_MEM, amountMiB) + _executorResources.put(OVERHEAD_MEM, req) this } @@ -70,8 +72,8 @@ private[spark] class ExecutorResourceRequests() extends Serializable { */ def pysparkMemory(amount: String): this.type = { val amountMiB = JavaUtils.byteStringAsMb(amount) - val rr = new ExecutorResourceRequest(PYSPARK_MEM, amountMiB) - _executorResources(PYSPARK_MEM) = rr + val req = new ExecutorResourceRequest(PYSPARK_MEM, amountMiB) + _executorResources.put(PYSPARK_MEM, req) this } @@ -81,15 +83,17 @@ private[spark] class ExecutorResourceRequests() extends Serializable { * @param amount Number of cores to allocate per Executor. */ def cores(amount: Int): this.type = { - val t = new ExecutorResourceRequest(CORES, amount) - _executorResources(CORES) = t + val req = new ExecutorResourceRequest(CORES, amount) + _executorResources.put(CORES, req) this } /** * Amount of a particular custom resource(GPU, FPGA, etc) to use. The resource names supported * correspond to the regular Spark configs with the prefix removed. For instance, resources - * like GPUs are resource.gpu (spark configs spark.executor.resource.gpu.*) + * like GPUs are gpu (spark configs spark.executor.resource.gpu.*). If you pass in a resource + * that the cluster manager doesn't support the result is undefined, it may error or may just + * be ignored. * * @param resourceName Name of the resource. * @param amount amount of that resource per executor to use. @@ -106,13 +110,8 @@ private[spark] class ExecutorResourceRequests() extends Serializable { vendor: String = ""): this.type = { // a bit weird but for Java api use empty string as meaning None because empty // string is otherwise invalid for those paramters anyway - val eReq = new ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor) - _executorResources(resourceName) = eReq - this - } - - def addRequest(ereq: ExecutorResourceRequest): this.type = { - _executorResources(ereq.resourceName) = ereq + val req = new ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor) + _executorResources.put(resourceName, req) this } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 876a655b13ad6..eb713a27be603 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -18,130 +18,164 @@ package org.apache.spark.resource import java.util.{Map => JMap} -import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} +import java.util.concurrent.atomic.AtomicInteger +import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ -import scala.collection.mutable import org.apache.spark.SparkConf import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.resource.ResourceUtils.RESOURCE_PREFIX +import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY /** * Resource profile to associate with an RDD. A ResourceProfile allows the user to * specify executor and task requirements for an RDD that will get applied during a * stage. This allows the user to change the resource requirements between stages. - * - * This class is private now for initial development, once we have the feature in place - * this will become public. + * This is meant to be immutable so user can't change it after building. */ @Evolving -private[spark] class ResourceProfile() extends Serializable { +class ResourceProfile( + val executorResources: Map[String, ExecutorResourceRequest], + val taskResources: Map[String, TaskResourceRequest]) extends Serializable with Logging { - private val _id = ResourceProfile.getNextProfileId - private val _taskResources = new mutable.HashMap[String, TaskResourceRequest]() - private val _executorResources = new mutable.HashMap[String, ExecutorResourceRequest]() + // _id is only a var for testing purposes + private var _id = ResourceProfile.getNextProfileId def id: Int = _id - def taskResources: Map[String, TaskResourceRequest] = _taskResources.toMap - def executorResources: Map[String, ExecutorResourceRequest] = _executorResources.toMap /** * (Java-specific) gets a Java Map of resources to TaskResourceRequest */ - def taskResourcesJMap: JMap[String, TaskResourceRequest] = _taskResources.asJava + def taskResourcesJMap: JMap[String, TaskResourceRequest] = taskResources.asJava /** * (Java-specific) gets a Java Map of resources to ExecutorResourceRequest */ - def executorResourcesJMap: JMap[String, ExecutorResourceRequest] = _executorResources.asJava + def executorResourcesJMap: JMap[String, ExecutorResourceRequest] = { + executorResources.asJava + } + + // Note that some cluster managers don't set the executor cores explicitly so + // be sure to check the Option as required + private[spark] def getExecutorCores: Option[Int] = { + executorResources.get(ResourceProfile.CORES).map(_.amount.toInt) + } - def reset(): Unit = { - _taskResources.clear() - _executorResources.clear() + private[spark] def getTaskCpus: Option[Int] = { + taskResources.get(ResourceProfile.CPUS).map(_.amount.toInt) } - def require(requests: ExecutorResourceRequests): this.type = { - _executorResources ++= requests.requests - this + // testing only + private[spark] def setToDefaultProfile(): Unit = { + _id = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID } - def require(requests: TaskResourceRequests): this.type = { - _taskResources ++= requests.requests - this + override def equals(obj: Any): Boolean = { + obj match { + case that: ResourceProfile => + that.getClass == this.getClass && that.id == _id && + that.taskResources == taskResources && that.executorResources == executorResources + case _ => + false + } } + override def hashCode(): Int = Seq(taskResources, executorResources).hashCode() + override def toString(): String = { - s"Profile: id = ${_id}, executor resources: ${_executorResources}, " + - s"task resources: ${_taskResources}" + s"Profile: id = ${_id}, executor resources: ${executorResources.mkString(",")}, " + + s"task resources: ${taskResources.mkString(",")}" } } -private[spark] object ResourceProfile extends Logging { - val UNKNOWN_RESOURCE_PROFILE_ID = -1 - val DEFAULT_RESOURCE_PROFILE_ID = 0 - +object ResourceProfile extends Logging { + // task resources val CPUS = "cpus" + // Executor resources val CORES = "cores" val MEMORY = "memory" val OVERHEAD_MEM = "memoryOverhead" val PYSPARK_MEM = "pyspark.memory" + // all supported spark executor resources (minus the custom resources like GPUs/FPGAs) + val allSupportedExecutorResources = Seq(CORES, MEMORY, OVERHEAD_MEM, PYSPARK_MEM) + + val UNKNOWN_RESOURCE_PROFILE_ID = -1 + val DEFAULT_RESOURCE_PROFILE_ID = 0 + private lazy val nextProfileId = new AtomicInteger(0) + private val DEFAULT_PROFILE_LOCK = new Object() // The default resource profile uses the application level configs. - // Create the default profile immediately to get ID 0, its initialized later when fetched. - private val defaultProfileRef: AtomicReference[ResourceProfile] = - new AtomicReference[ResourceProfile](new ResourceProfile()) - - assert(defaultProfileRef.get().id == DEFAULT_RESOURCE_PROFILE_ID, - s"Default Profile must have the default profile id: $DEFAULT_RESOURCE_PROFILE_ID") - - def getNextProfileId: Int = nextProfileId.getAndIncrement() - - def getOrCreateDefaultProfile(conf: SparkConf): ResourceProfile = { - val defaultProf = defaultProfileRef.get() - // check to see if the default profile was initialized yet - if (defaultProf.executorResources == Map.empty) { - synchronized { - val prof = defaultProfileRef.get() - if (prof.executorResources == Map.empty) { - addDefaultTaskResources(prof, conf) - addDefaultExecutorResources(prof, conf) - } - prof + // var so that it can be reset for testing purposes. + @GuardedBy("DEFAULT_PROFILE_LOCK") + private var defaultProfile: Option[ResourceProfile] = None + + private[spark] def getNextProfileId: Int = nextProfileId.getAndIncrement() + + private[spark] def getOrCreateDefaultProfile(conf: SparkConf): ResourceProfile = { + DEFAULT_PROFILE_LOCK.synchronized { + defaultProfile match { + case Some(prof) => prof + case None => + val taskResources = getDefaultTaskResources(conf) + val executorResources = getDefaultExecutorResources(conf) + val defProf = new ResourceProfile(executorResources, taskResources) + defProf.setToDefaultProfile + defaultProfile = Some(defProf) + logInfo("Default ResourceProfile created, executor resources: " + + s"${defProf.executorResources}, task resources: " + + s"${defProf.taskResources}") + defProf } - } else { - defaultProf } } - private def addDefaultTaskResources(rprof: ResourceProfile, conf: SparkConf): Unit = { + private def getDefaultTaskResources(conf: SparkConf): Map[String, TaskResourceRequest] = { val cpusPerTask = conf.get(CPUS_PER_TASK) val treqs = new TaskResourceRequests().cpus(cpusPerTask) - val taskReq = ResourceUtils.parseResourceRequirements(conf, SPARK_TASK_PREFIX) - taskReq.foreach { req => - val name = s"${RESOURCE_PREFIX}.${req.resourceName}" - treqs.resource(name, req.amount) - } - rprof.require(treqs) + ResourceUtils.addTaskResourceRequests(conf, treqs) + treqs.requests } - private def addDefaultExecutorResources(rprof: ResourceProfile, conf: SparkConf): Unit = { + private def getDefaultExecutorResources(conf: SparkConf): Map[String, ExecutorResourceRequest] = { val ereqs = new ExecutorResourceRequests() ereqs.cores(conf.get(EXECUTOR_CORES)) ereqs.memory(conf.get(EXECUTOR_MEMORY).toString) + conf.get(EXECUTOR_MEMORY_OVERHEAD).map(mem => ereqs.memoryOverhead(mem.toString)) + conf.get(PYSPARK_EXECUTOR_MEMORY).map(mem => ereqs.pysparkMemory(mem.toString)) val execReq = ResourceUtils.parseAllResourceRequests(conf, SPARK_EXECUTOR_PREFIX) execReq.foreach { req => - val name = s"${RESOURCE_PREFIX}.${req.id.resourceName}" + val name = req.id.resourceName ereqs.resource(name, req.amount, req.discoveryScript.getOrElse(""), req.vendor.getOrElse("")) } - rprof.require(ereqs) + ereqs.requests } - // for testing purposes - def resetDefaultProfile(conf: SparkConf): Unit = getOrCreateDefaultProfile(conf).reset() + // for testing only + private[spark] def reInitDefaultProfile(conf: SparkConf): Unit = { + clearDefaultProfile + // force recreate it after clearing + getOrCreateDefaultProfile(conf) + } + + // for testing only + private[spark] def clearDefaultProfile: Unit = { + DEFAULT_PROFILE_LOCK.synchronized { + defaultProfile = None + } + } + + private[spark] def getCustomTaskResources( + rp: ResourceProfile): Map[String, TaskResourceRequest] = { + rp.taskResources.filterKeys(k => !k.equals(ResourceProfile.CPUS)) + } + + private[spark] def getCustomExecutorResources( + rp: ResourceProfile): Map[String, ExecutorResourceRequest] = { + rp.executorResources.filterKeys(k => !ResourceProfile.allSupportedExecutorResources.contains(k)) + } } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala new file mode 100644 index 0000000000000..0d55c176eeb65 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.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.resource + +import java.util.{Map => JMap} +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Evolving + +/** + * Resource profile builder to build a Resource profile to associate with an RDD. + * A ResourceProfile allows the user to specify executor and task requirements for an RDD + * that will get applied during a stage. This allows the user to change the resource + * requirements between stages. + */ +@Evolving +class ResourceProfileBuilder() { + + private val _taskResources = new ConcurrentHashMap[String, TaskResourceRequest]() + private val _executorResources = new ConcurrentHashMap[String, ExecutorResourceRequest]() + + def taskResources: Map[String, TaskResourceRequest] = _taskResources.asScala.toMap + def executorResources: Map[String, ExecutorResourceRequest] = _executorResources.asScala.toMap + + /** + * (Java-specific) gets a Java Map of resources to TaskResourceRequest + */ + def taskResourcesJMap: JMap[String, TaskResourceRequest] = _taskResources.asScala.asJava + + /** + * (Java-specific) gets a Java Map of resources to ExecutorResourceRequest + */ + def executorResourcesJMap: JMap[String, ExecutorResourceRequest] = { + _executorResources.asScala.asJava + } + + def require(requests: ExecutorResourceRequests): this.type = { + _executorResources.putAll(requests.requests.asJava) + this + } + + def require(requests: TaskResourceRequests): this.type = { + _taskResources.putAll(requests.requests.asJava) + this + } + + def clearExecutorResourceRequests(): this.type = { + _executorResources.clear() + this + } + + def clearTaskResourceRequests(): this.type = { + _taskResources.clear() + this + } + + override def toString(): String = { + "Profile executor resources: " + + s"${_executorResources.asScala.map(pair => s"${pair._1}=${pair._2.toString()}")}, " + + s"task resources: ${_taskResources.asScala.map(pair => s"${pair._1}=${pair._2.toString()}")}" + } + + def build: ResourceProfile = { + new ResourceProfile(executorResources, taskResources) + } +} + diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index fdd5c9a84ce12..190b0cdc8887a 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -111,7 +111,7 @@ private[spark] object ResourceUtils extends Logging { } def listResourceIds(sparkConf: SparkConf, componentName: String): Seq[ResourceID] = { - sparkConf.getAllWithPrefix(s"$componentName.$RESOURCE_DOT").map { case (key, _) => + sparkConf.getAllWithPrefix(s"$componentName.$RESOURCE_PREFIX.").map { case (key, _) => key.substring(0, key.indexOf('.')) }.toSet.toSeq.map(name => ResourceID(componentName, name)) } @@ -124,6 +124,35 @@ private[spark] object ResourceUtils extends Logging { .filter(_.amount > 0) } + // Used to take a fraction amount from a task resource requirement and split into a real + // integer amount and the number of parts expected. For instance, if the amount is 0.5, + // the we get (1, 2) back out. + // Returns tuple of (amount, numParts) + def calculateAmountAndPartsForFraction(amount: Double): (Int, Int) = { + val parts = if (amount <= 0.5) { + Math.floor(1.0 / amount).toInt + } else if (amount % 1 != 0) { + throw new SparkException( + s"The resource amount ${amount} must be either <= 0.5, or a whole number.") + } else { + 1 + } + (Math.ceil(amount).toInt, parts) + } + + // Add any task resource requests from the spark conf to the TaskResourceRequests passed in + def addTaskResourceRequests( + sparkConf: SparkConf, + treqs: TaskResourceRequests): Unit = { + listResourceIds(sparkConf, SPARK_TASK_PREFIX).map { resourceId => + val settings = sparkConf.getAllWithPrefix(resourceId.confPrefix).toMap + val amountDouble = settings.getOrElse(AMOUNT, + throw new SparkException(s"You must specify an amount for ${resourceId.resourceName}") + ).toDouble + treqs.resource(resourceId.resourceName, amountDouble) + } + } + def parseResourceRequirements(sparkConf: SparkConf, componentName: String) : Seq[ResourceRequirement] = { val resourceIds = listResourceIds(sparkConf, componentName) @@ -136,15 +165,7 @@ private[spark] object ResourceUtils extends Logging { } rnamesAndAmounts.filter { case (_, amount) => amount > 0 }.map { case (rName, amountDouble) => val (amount, parts) = if (componentName.equalsIgnoreCase(SPARK_TASK_PREFIX)) { - val parts = if (amountDouble <= 0.5) { - Math.floor(1.0 / amountDouble).toInt - } else if (amountDouble % 1 != 0) { - throw new SparkException( - s"The resource amount ${amountDouble} must be either <= 0.5, or a whole number.") - } else { - 1 - } - (Math.ceil(amountDouble).toInt, parts) + calculateAmountAndPartsForFraction(amountDouble) } else if (amountDouble % 1 != 0) { throw new SparkException( s"Only tasks support fractional resources, please check your $componentName settings") @@ -181,12 +202,18 @@ private[spark] object ResourceUtils extends Logging { } } + def parseAllocated( + resourcesFileOpt: Option[String], + componentName: String): Seq[ResourceAllocation] = { + resourcesFileOpt.toSeq.flatMap(parseAllocatedFromJsonFile) + .filter(_.id.componentName == componentName) + } + private def parseAllocatedOrDiscoverResources( sparkConf: SparkConf, componentName: String, resourcesFileOpt: Option[String]): Seq[ResourceAllocation] = { - val allocated = resourcesFileOpt.toSeq.flatMap(parseAllocatedFromJsonFile) - .filter(_.id.componentName == componentName) + val allocated = parseAllocated(resourcesFileOpt, componentName) val otherResourceIds = listResourceIds(sparkConf, componentName).diff(allocated.map(_.id)) val otherResources = otherResourceIds.flatMap { id => val request = parseResourceRequest(sparkConf, id) @@ -215,9 +242,24 @@ private[spark] object ResourceUtils extends Logging { requests.foreach(r => assertResourceAllocationMeetsRequest(allocated(r.id), r)) } + private def assertAllResourceAllocationsMatchResourceProfile( + allocations: Map[String, ResourceInformation], + execReqs: Map[String, ExecutorResourceRequest]): Unit = { + execReqs.foreach { case (rName, req) => + require(allocations.contains(rName) && allocations(rName).addresses.size >= req.amount, + s"Resource: ${rName}, with addresses: " + + s"${allocations(rName).addresses.mkString(",")} " + + s"is less than what the user requested: ${req.amount})") + } + } + /** * Gets all allocated resource information for the input component from input resources file and - * discover the remaining via discovery scripts. + * the application level Spark configs. It first looks to see if resource were explicitly + * specified in the resources file (this would include specified address assignments and it only + * specified in certain cluster managers) and then it looks at the Spark configs to get any + * others not specified in the file. The resources not explicitly set in the file require a + * discovery script for it to run to get the addresses of the resource. * It also verifies the resource allocation meets required amount for each resource. * @return a map from resource name to resource info */ @@ -232,6 +274,37 @@ private[spark] object ResourceUtils extends Logging { resourceInfoMap } + /** + * This function is similar to getOrDiscoverallResources, except for it uses the ResourceProfile + * information instead of the application level configs. + * + * It first looks to see if resource were explicitly specified in the resources file + * (this would include specified address assignments and it only specified in certain + * cluster managers) and then it looks at the ResourceProfile to get + * any others not specified in the file. The resources not explicitly set in the file require a + * discovery script for it to run to get the addresses of the resource. + * It also verifies the resource allocation meets required amount for each resource. + * + * @return a map from resource name to resource info + */ + def getOrDiscoverAllResourcesForResourceProfile( + resourcesFileOpt: Option[String], + componentName: String, + resourceProfile: ResourceProfile): Map[String, ResourceInformation] = { + val fileAllocated = parseAllocated(resourcesFileOpt, componentName) + val fileAllocResMap = fileAllocated.map(a => (a.id.resourceName, a.toResourceInformation)).toMap + // only want to look at the ResourceProfile for resources not in the resources file + val execReq = ResourceProfile.getCustomExecutorResources(resourceProfile) + val filteredExecreq = execReq.filterNot { case (rname, _) => fileAllocResMap.contains(rname) } + val rpAllocations = filteredExecreq.map { case (rName, execRequest) => + val addrs = discoverResource(rName, Option(execRequest.discoveryScript)).addresses + (rName, new ResourceInformation(rName, addrs)) + } + val allAllocations = fileAllocResMap ++ rpAllocations + assertAllResourceAllocationsMatchResourceProfile(allAllocations, execReq) + allAllocations + } + def logResourceInfo(componentName: String, resources: Map[String, ResourceInformation]) : Unit = { logInfo("==============================================================") @@ -240,9 +313,9 @@ private[spark] object ResourceUtils extends Logging { } // visible for test - private[spark] def discoverResource(resourceRequest: ResourceRequest): ResourceInformation = { - val resourceName = resourceRequest.id.resourceName - val script = resourceRequest.discoveryScript + private[spark] def discoverResource( + resourceName: String, + script: Option[String]): ResourceInformation = { val result = if (script.nonEmpty) { val scriptFile = new File(script.get) // check that script exists and try to execute @@ -264,10 +337,16 @@ private[spark] object ResourceUtils extends Logging { result } + // visible for test + private[spark] def discoverResource(resourceRequest: ResourceRequest): ResourceInformation = { + val resourceName = resourceRequest.id.resourceName + val script = resourceRequest.discoveryScript + discoverResource(resourceName, script) + } + // known types of resources final val GPU: String = "gpu" final val FPGA: String = "fpga" final val RESOURCE_PREFIX: String = "resource" - final val RESOURCE_DOT: String = s"$RESOURCE_PREFIX." } diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala index 22eda52c423a1..bffb0a2f523b1 100644 --- a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala +++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala @@ -17,10 +17,6 @@ package org.apache.spark.resource -import scala.collection.mutable - -import org.apache.spark.resource.ResourceUtils.RESOURCE_DOT - /** * A task resource request. This is used in conjuntion with the ResourceProfile to * programmatically specify the resources needed for an RDD that will be applied at the @@ -37,7 +33,19 @@ private[spark] class TaskResourceRequest(val resourceName: String, val amount: D assert(amount <= 0.5 || amount % 1 == 0, s"The resource amount ${amount} must be either <= 0.5, or a whole number.") - if (!resourceName.equals(ResourceProfile.CPUS) && !resourceName.startsWith(RESOURCE_DOT)) { - throw new IllegalArgumentException(s"Task resource not allowed: $resourceName") + override def equals(obj: Any): Boolean = { + obj match { + case that: TaskResourceRequest => + that.getClass == this.getClass && + that.resourceName == resourceName && that.amount == amount + case _ => + false + } + } + + override def hashCode(): Int = Seq(resourceName, amount).hashCode() + + override def toString(): String = { + s"name: $resourceName, amount: $amount" } } diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala index 21cbc5d805ac0..9624b51dd158e 100644 --- a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala @@ -17,10 +17,11 @@ package org.apache.spark.resource -import scala.collection.mutable +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ import org.apache.spark.resource.ResourceProfile._ -import org.apache.spark.resource.ResourceUtils._ /** * A set of task resource requests. This is used in conjuntion with the ResourceProfile to @@ -32,9 +33,9 @@ import org.apache.spark.resource.ResourceUtils._ */ private[spark] class TaskResourceRequests() extends Serializable { - private val _taskResources = new mutable.HashMap[String, TaskResourceRequest]() + private val _taskResources = new ConcurrentHashMap[String, TaskResourceRequest]() - def requests: Map[String, TaskResourceRequest] = _taskResources.toMap + def requests: Map[String, TaskResourceRequest] = _taskResources.asScala.toMap /** * Specify number of cpus per Task. @@ -42,15 +43,13 @@ private[spark] class TaskResourceRequests() extends Serializable { * @param amount Number of cpus to allocate per Task. */ def cpus(amount: Int): this.type = { - val t = new TaskResourceRequest(CPUS, amount) - _taskResources(CPUS) = t + val treq = new TaskResourceRequest(CPUS, amount) + _taskResources.put(CPUS, treq) this } /** - * Amount of a particular custom resource(GPU, FPGA, etc) to use. The resource names supported - * correspond to the regular Spark configs with the prefix removed. For instance, resources - * like GPUs are resource.gpu (spark configs spark.task.resource.gpu.*) + * Amount of a particular custom resource(GPU, FPGA, etc) to use. * * @param resourceName Name of the resource. * @param amount Amount requesting as a Double to support fractional resource requests. @@ -58,14 +57,14 @@ private[spark] class TaskResourceRequests() extends Serializable { * lets you configure X number of tasks to run on a single resource, * ie amount equals 0.5 translates into 2 tasks per resource address. */ - def resource(rName: String, amount: Double): this.type = { - val t = new TaskResourceRequest(rName, amount) - _taskResources(rName) = t + def resource(resourceName: String, amount: Double): this.type = { + val treq = new TaskResourceRequest(resourceName, amount) + _taskResources.put(resourceName, treq) this } def addRequest(treq: TaskResourceRequest): this.type = { - _taskResources(treq.resourceName) = treq + _taskResources.put(treq.resourceName, treq) this } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 57317e7f6af00..283390814a6c0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState -import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.ExecutorLossReason import org.apache.spark.util.SerializableBuffer @@ -29,12 +29,13 @@ private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { - case object RetrieveSparkAppConfig extends CoarseGrainedClusterMessage + case class RetrieveSparkAppConfig(resourceProfileId: Int) extends CoarseGrainedClusterMessage case class SparkAppConfig( sparkProperties: Seq[(String, String)], ioEncryptionKey: Option[Array[Byte]], - hadoopDelegationCreds: Option[Array[Byte]]) + hadoopDelegationCreds: Option[Array[Byte]], + resourceProfile: ResourceProfile) extends CoarseGrainedClusterMessage case object RetrieveLastAllocatedExecutorId extends CoarseGrainedClusterMessage @@ -59,7 +60,8 @@ private[spark] object CoarseGrainedClusterMessages { cores: Int, logUrls: Map[String, String], attributes: Map[String, String], - resources: Map[String, ResourceInformation]) + resources: Map[String, ResourceInformation], + resourceProfileId: Int) extends CoarseGrainedClusterMessage case class LaunchedExecutor(executorId: String) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 031b9afaa17aa..55f4005ef1b45 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 @@ -33,7 +33,7 @@ import org.apache.spark.executor.ExecutorLogUrlHandler import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network._ -import org.apache.spark.resource.ResourceRequirement +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -205,7 +205,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls, - attributes, resources) => + attributes, resources, resourceProfileId) => if (executorDataMap.contains(executorId)) { context.sendFailure(new IllegalStateException(s"Duplicate executor ID: $executorId")) } else if (scheduler.nodeBlacklist.contains(hostname) || @@ -236,7 +236,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } val data = new ExecutorData(executorRef, executorAddress, hostname, 0, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, - resourcesInfo) + resourcesInfo, resourceProfileId) // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { @@ -270,11 +270,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp removeWorker(workerId, host, message) context.reply(true) - case RetrieveSparkAppConfig => + case RetrieveSparkAppConfig(resourceProfileId) => + // note this will be updated in later prs to get the ResourceProfile from a + // ResourceProfileManager that matches the resource profile id + // for now just use default profile + val rp = ResourceProfile.getOrCreateDefaultProfile(conf) val reply = SparkAppConfig( sparkProperties, SparkEnv.get.securityManager.getIOEncryptionKey(), - Option(delegationTokens.get())) + Option(delegationTokens.get()), + rp) context.reply(reply) } @@ -570,6 +575,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorDataMap.get(executorId).map(_.resourcesInfo).getOrElse(Map.empty) } + // this function is for testing only + def getExecutorResourceProfileId(executorId: String): Int = synchronized { + val res = executorDataMap.get(executorId) + res.map(_.resourceProfileId).getOrElse(ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) + } + /** * Request an additional number of executors from the cluster manager. * @return whether the request is acknowledged. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 17907d88e50c8..062146174f6a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -29,6 +29,7 @@ import org.apache.spark.scheduler.ExecutorResourceInfo * @param freeCores The current number of cores available for work on the executor * @param totalCores The total number of cores available to the executor * @param resourcesInfo The information of the currently available resources on the executor + * @param resourceProfileId The id of the ResourceProfile being used by this executor */ private[cluster] class ExecutorData( val executorEndpoint: RpcEndpointRef, @@ -38,5 +39,7 @@ private[cluster] class ExecutorData( override val totalCores: Int, override val logUrlMap: Map[String, String], override val attributes: Map[String, String], - override val resourcesInfo: Map[String, ExecutorResourceInfo] -) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes, resourcesInfo) + override val resourcesInfo: Map[String, ExecutorResourceInfo], + override val resourceProfileId: Int +) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes, + resourcesInfo, resourceProfileId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index 5a4ad6e00eb43..a97b08941ba78 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.annotation.DeveloperApi import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID /** * :: DeveloperApi :: @@ -25,14 +26,15 @@ import org.apache.spark.resource.ResourceInformation */ @DeveloperApi class ExecutorInfo( - val executorHost: String, - val totalCores: Int, - val logUrlMap: Map[String, String], - val attributes: Map[String, String], - val resourcesInfo: Map[String, ResourceInformation]) { + val executorHost: String, + val totalCores: Int, + val logUrlMap: Map[String, String], + val attributes: Map[String, String], + val resourcesInfo: Map[String, ResourceInformation], + val resourceProfileId: Int) { def this(executorHost: String, totalCores: Int, logUrlMap: Map[String, String]) = { - this(executorHost, totalCores, logUrlMap, Map.empty, Map.empty) + this(executorHost, totalCores, logUrlMap, Map.empty, Map.empty, DEFAULT_RESOURCE_PROFILE_ID) } def this( @@ -40,7 +42,17 @@ class ExecutorInfo( totalCores: Int, logUrlMap: Map[String, String], attributes: Map[String, String]) = { - this(executorHost, totalCores, logUrlMap, attributes, Map.empty) + this(executorHost, totalCores, logUrlMap, attributes, Map.empty, DEFAULT_RESOURCE_PROFILE_ID) + } + + def this( + executorHost: String, + totalCores: Int, + logUrlMap: Map[String, String], + attributes: Map[String, String], + resourcesInfo: Map[String, ResourceInformation]) = { + this(executorHost, totalCores, logUrlMap, attributes, resourcesInfo, + DEFAULT_RESOURCE_PROFILE_ID) } def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] @@ -52,12 +64,14 @@ class ExecutorInfo( totalCores == that.totalCores && logUrlMap == that.logUrlMap && attributes == that.attributes && - resourcesInfo == that.resourcesInfo + resourcesInfo == that.resourcesInfo && + resourceProfileId == that.resourceProfileId case _ => false } override def hashCode(): Int = { - val state = Seq(executorHost, totalCores, logUrlMap, attributes, resourcesInfo) + val state = Seq(executorHost, totalCores, logUrlMap, attributes, resourcesInfo, + resourceProfileId) state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index 3dfd1eac8cc9c..a24f1902faa31 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -26,6 +26,7 @@ import scala.collection.mutable import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ import org.apache.spark.storage.RDDBlockId import org.apache.spark.util.Clock @@ -52,6 +53,7 @@ private[spark] class ExecutorMonitor( conf.get(DYN_ALLOCATION_SHUFFLE_TRACKING) private val executors = new ConcurrentHashMap[String, Tracker]() + private val execResourceProfileCount = new ConcurrentHashMap[Int, Int]() // The following fields are an optimization to avoid having to scan all executors on every EAM // schedule interval to find out which ones are timed out. They keep track of when the next @@ -92,6 +94,7 @@ private[spark] class ExecutorMonitor( def reset(): Unit = { executors.clear() + execResourceProfileCount.clear() nextTimeout.set(Long.MaxValue) timedOutExecs = Nil } @@ -148,8 +151,25 @@ private[spark] class ExecutorMonitor( def executorCount: Int = executors.size() + def executorCountWithResourceProfile(id: Int): Int = { + execResourceProfileCount.getOrDefault(id, 0) + } + + def getResourceProfileId(executorId: String): Int = { + val execTrackingInfo = executors.get(executorId) + if (execTrackingInfo != null) { + execTrackingInfo.resourceProfileId + } else { + UNKNOWN_RESOURCE_PROFILE_ID + } + } + def pendingRemovalCount: Int = executors.asScala.count { case (_, exec) => exec.pendingRemoval } + def pendingRemovalCountPerResourceProfileId(id: Int): Int = { + executors.asScala.filter { case (k, v) => v.resourceProfileId == id && v.pendingRemoval }.size + } + override def onJobStart(event: SparkListenerJobStart): Unit = { if (!shuffleTrackingEnabled) { return @@ -261,7 +281,7 @@ private[spark] class ExecutorMonitor( val executorId = event.taskInfo.executorId // Guard against a late arriving task start event (SPARK-26927). if (client.isExecutorActive(executorId)) { - val exec = ensureExecutorIsTracked(executorId) + val exec = ensureExecutorIsTracked(executorId, UNKNOWN_RESOURCE_PROFILE_ID) exec.updateRunningTasks(1) } } @@ -290,15 +310,21 @@ private[spark] class ExecutorMonitor( } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { - val exec = ensureExecutorIsTracked(event.executorId) + val exec = ensureExecutorIsTracked(event.executorId, event.executorInfo.resourceProfileId) exec.updateRunningTasks(0) logInfo(s"New executor ${event.executorId} has registered (new total is ${executors.size()})") } + private def decrementExecResourceProfileCount(rpId: Int): Unit = { + val count = execResourceProfileCount.getOrDefault(rpId, 0) + execResourceProfileCount.replace(rpId, count, count - 1) + execResourceProfileCount.remove(rpId, 0) + } + override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { val removed = executors.remove(event.executorId) if (removed != null) { - logInfo(s"Executor ${event.executorId} removed (new total is ${executors.size()})") + decrementExecResourceProfileCount(removed.resourceProfileId) if (!removed.pendingRemoval) { nextTimeout.set(Long.MinValue) } @@ -309,8 +335,8 @@ private[spark] class ExecutorMonitor( if (!event.blockUpdatedInfo.blockId.isInstanceOf[RDDBlockId]) { return } - - val exec = ensureExecutorIsTracked(event.blockUpdatedInfo.blockManagerId.executorId) + val exec = ensureExecutorIsTracked(event.blockUpdatedInfo.blockManagerId.executorId, + UNKNOWN_RESOURCE_PROFILE_ID) val storageLevel = event.blockUpdatedInfo.storageLevel val blockId = event.blockUpdatedInfo.blockId.asInstanceOf[RDDBlockId] @@ -392,8 +418,26 @@ private[spark] class ExecutorMonitor( * which the `SparkListenerTaskStart` event is posted before the `SparkListenerBlockManagerAdded` * event, which is possible because these events are posted in different threads. (see SPARK-4951) */ - private def ensureExecutorIsTracked(id: String): Tracker = { - executors.computeIfAbsent(id, _ => new Tracker()) + private def ensureExecutorIsTracked(id: String, resourceProfileId: Int): Tracker = { + val numExecsWithRpId = execResourceProfileCount.computeIfAbsent(resourceProfileId, _ => 0) + val execTracker = executors.computeIfAbsent(id, _ => { + val newcount = numExecsWithRpId + 1 + execResourceProfileCount.put(resourceProfileId, newcount) + logDebug(s"Executor added with ResourceProfile id: $resourceProfileId " + + s"count is now $newcount") + new Tracker(resourceProfileId) + }) + // if we had added executor before without knowing the resource profile id, fix it up + if (execTracker.resourceProfileId == UNKNOWN_RESOURCE_PROFILE_ID && + resourceProfileId != UNKNOWN_RESOURCE_PROFILE_ID) { + logDebug(s"Executor: $id, resource profile id was unknown, setting " + + s"it to $resourceProfileId") + execTracker.resourceProfileId = resourceProfileId + // fix up the counts for each resource profile id + execResourceProfileCount.put(resourceProfileId, numExecsWithRpId + 1) + decrementExecResourceProfileCount(UNKNOWN_RESOURCE_PROFILE_ID) + } + execTracker } private def updateNextTimeout(newValue: Long): Unit = { @@ -413,7 +457,7 @@ private[spark] class ExecutorMonitor( } } - private class Tracker { + private class Tracker(var resourceProfileId: Int) { @volatile var timeoutAt: Long = Long.MaxValue // Tracks whether this executor is thought to be timed out. It's used to detect when the list diff --git a/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java b/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java index 077120724c291..bb413c00fb972 100644 --- a/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java +++ b/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java @@ -35,7 +35,7 @@ public void testResourceProfileAccessFromJava() throws Exception { ExecutorResourceRequests execReqFpga = new ExecutorResourceRequests().resource(FPGAResource, 3, "myfpgascript", "nvidia"); - ResourceProfile rprof = new ResourceProfile(); + ResourceProfileBuilder rprof = new ResourceProfileBuilder(); rprof.require(execReqGpu); rprof.require(execReqFpga); TaskResourceRequests taskReq1 = new TaskResourceRequests().resource(GpuResource, 1); diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 6ae1f197cf744..99f3e3b2e478d 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{Clock, ManualClock, SystemClock} @@ -1018,8 +1019,11 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { manager } + private val execInfo = new ExecutorInfo("host1", 1, Map.empty, + Map.empty, Map.empty, DEFAULT_RESOURCE_PROFILE_ID) + private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { - post(SparkListenerExecutorAdded(0L, id, null)) + post(SparkListenerExecutorAdded(0L, id, execInfo)) } private def onExecutorRemoved(manager: ExecutorAllocationManager, id: String): Unit = { diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index b468e6fa50344..ff0f2f9134ed3 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -30,6 +30,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.DYN_ALLOCATION_TESTING +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -177,10 +178,10 @@ class HeartbeatReceiverSuite val dummyExecutorEndpointRef2 = rpcEnv.setupEndpoint("fake-executor-2", dummyExecutorEndpoint2) fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty, Map.empty, - Map.empty)) + Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty, Map.empty, - Map.empty)) + Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) addExecutorAndVerify(executorId2) diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index d050ee2c45e7a..1fe12e116d96e 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -22,6 +22,8 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.BeforeAndAfterEach import org.scalatest.Suite +import org.apache.spark.resource.ResourceProfile + /** Manages a local `sc` `SparkContext` variable, correctly stopping it after each test. */ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite => @@ -42,6 +44,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self def resetSparkContext(): Unit = { LocalSparkContext.stop(sc) + ResourceProfile.clearDefaultProfile sc = null } diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index e316da738440e..f8b99302c4ad5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.deploy.master.ApplicationInfo import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.Worker import org.apache.spark.internal.config +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster._ @@ -505,7 +506,7 @@ class StandaloneDynamicAllocationSuite val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty, - Map.empty, Map.empty) + Map.empty, Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val taskScheduler = mock(classOf[TaskSchedulerImpl]) when(taskScheduler.nodeBlacklist()).thenReturn(Set("blacklisted-host")) @@ -629,7 +630,7 @@ class StandaloneDynamicAllocationSuite val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty, Map.empty, - Map.empty) + Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) backend.driverEndpoint.askSync[Boolean](message) backend.driverEndpoint.send(LaunchedExecutor(id)) } diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index 7e96039ca924f..a996fc4a0b648 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.executor +import java.io.File import java.net.URL import java.nio.ByteBuffer import java.util.Properties @@ -33,7 +34,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.TestUtils._ -import org.apache.spark.resource.{ResourceAllocation, ResourceInformation} +import org.apache.spark.resource._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEnv @@ -49,13 +50,13 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("parsing no resources") { val conf = new SparkConf - conf.set(TASK_GPU_ID.amountConf, "2") + val resourceProfile = ResourceProfile.getOrCreateDefaultProfile(conf) val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", - 4, Seq.empty[URL], env, None) + 4, Seq.empty[URL], env, None, resourceProfile) withTempDir { tmpDir => val testResourceArgs: JObject = ("" -> "") val ja = JArray(List(testResourceArgs)) @@ -72,12 +73,11 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("parsing one resource") { val conf = new SparkConf conf.set(EXECUTOR_GPU_ID.amountConf, "2") - conf.set(TASK_GPU_ID.amountConf, "2") val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", - 4, Seq.empty[URL], env, None) + 4, Seq.empty[URL], env, None, ResourceProfile.getOrCreateDefaultProfile(conf)) withTempDir { tmpDir => val ra = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) val ja = Extraction.decompose(Seq(ra)) @@ -91,18 +91,27 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite } } + test("parsing multiple resources resource profile") { + val rpBuilder = new ResourceProfileBuilder + val ereqs = new ExecutorResourceRequests().resource(GPU, 2) + ereqs.resource(FPGA, 3) + val rp = rpBuilder.require(ereqs).build + testParsingMultipleResources(new SparkConf, rp) + } + test("parsing multiple resources") { val conf = new SparkConf conf.set(EXECUTOR_GPU_ID.amountConf, "2") - conf.set(TASK_GPU_ID.amountConf, "2") conf.set(EXECUTOR_FPGA_ID.amountConf, "3") - conf.set(TASK_FPGA_ID.amountConf, "3") + testParsingMultipleResources(conf, ResourceProfile.getOrCreateDefaultProfile(conf)) + } + def testParsingMultipleResources(conf: SparkConf, resourceProfile: ResourceProfile) { val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", "host1", - 4, Seq.empty[URL], env, None) + 4, Seq.empty[URL], env, None, resourceProfile) withTempDir { tmpDir => val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) @@ -125,12 +134,11 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("error checking parsing resources and executor and task configs") { val conf = new SparkConf conf.set(EXECUTOR_GPU_ID.amountConf, "2") - conf.set(TASK_GPU_ID.amountConf, "2") val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", - 4, Seq.empty[URL], env, None) + 4, Seq.empty[URL], env, None, ResourceProfile.getOrCreateDefaultProfile(conf)) // not enough gpu's on the executor withTempDir { tmpDir => @@ -156,20 +164,33 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() - assert(error.contains("User is expecting to use resource: gpu, but didn't specify a " + - "discovery script!")) + assert(error.contains("Resource script: to discover gpu doesn't exist!")) } } + test("executor resource found less than required resource profile") { + val rpBuilder = new ResourceProfileBuilder + val ereqs = new ExecutorResourceRequests().resource(GPU, 4) + val treqs = new TaskResourceRequests().resource(GPU, 1) + val rp = rpBuilder.require(ereqs).require(treqs).build + testExecutorResourceFoundLessThanRequired(new SparkConf, rp) + } + test("executor resource found less than required") { - val conf = new SparkConf + val conf = new SparkConf() conf.set(EXECUTOR_GPU_ID.amountConf, "4") conf.set(TASK_GPU_ID.amountConf, "1") + testExecutorResourceFoundLessThanRequired(conf, ResourceProfile.getOrCreateDefaultProfile(conf)) + } + + private def testExecutorResourceFoundLessThanRequired( + conf: SparkConf, + resourceProfile: ResourceProfile) = { val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", - 4, Seq.empty[URL], env, None) + 4, Seq.empty[URL], env, None, resourceProfile) // executor resources < required withTempDir { tmpDir => @@ -189,7 +210,6 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite test("use resource discovery") { val conf = new SparkConf conf.set(EXECUTOR_FPGA_ID.amountConf, "3") - conf.set(TASK_FPGA_ID.amountConf, "3") assume(!(Utils.isWindows)) withTempDir { dir => val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", @@ -201,7 +221,7 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite // we don't really use this, just need it to get at the parser function val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", - 4, Seq.empty[URL], env, None) + 4, Seq.empty[URL], env, None, ResourceProfile.getOrCreateDefaultProfile(conf)) val parsedResources = backend.parseOrFindResources(None) @@ -212,37 +232,56 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite } } + test("use resource discovery and allocated file option with resource profile") { + assume(!(Utils.isWindows)) + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", + """{"name": "fpga","addresses":["f1", "f2", "f3"]}""") + val rpBuilder = new ResourceProfileBuilder + val ereqs = new ExecutorResourceRequests().resource(FPGA, 3, scriptPath) + ereqs.resource(GPU, 2) + val rp = rpBuilder.require(ereqs).build + allocatedFileAndConfigsResourceDiscoveryTestFpga(dir, new SparkConf, rp) + } + } + test("use resource discovery and allocated file option") { - val conf = new SparkConf - conf.set(EXECUTOR_FPGA_ID.amountConf, "3") - conf.set(TASK_FPGA_ID.amountConf, "3") assume(!(Utils.isWindows)) withTempDir { dir => val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", """{"name": "fpga","addresses":["f1", "f2", "f3"]}""") + val conf = new SparkConf + conf.set(EXECUTOR_FPGA_ID.amountConf, "3") conf.set(EXECUTOR_FPGA_ID.discoveryScriptConf, scriptPath) - - val serializer = new JavaSerializer(conf) - val env = createMockEnv(conf, serializer) - - // we don't really use this, just need it to get at the parser function - val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", - 4, Seq.empty[URL], env, None) - val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) - val ja = Extraction.decompose(Seq(gpuArgs)) - val f1 = createTempJsonFile(dir, "resources", ja) - val parsedResources = backend.parseOrFindResources(Some(f1)) - - assert(parsedResources.size === 2) - assert(parsedResources.get(GPU).nonEmpty) - assert(parsedResources.get(GPU).get.name === GPU) - assert(parsedResources.get(GPU).get.addresses.sameElements(Array("0", "1"))) - assert(parsedResources.get(FPGA).nonEmpty) - assert(parsedResources.get(FPGA).get.name === FPGA) - assert(parsedResources.get(FPGA).get.addresses.sameElements(Array("f1", "f2", "f3"))) + conf.set(EXECUTOR_GPU_ID.amountConf, "2") + val rp = ResourceProfile.getOrCreateDefaultProfile(conf) + allocatedFileAndConfigsResourceDiscoveryTestFpga(dir, conf, rp) } } + private def allocatedFileAndConfigsResourceDiscoveryTestFpga( + dir: File, + conf: SparkConf, + resourceProfile: ResourceProfile) = { + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", "host1", + 4, Seq.empty[URL], env, None, resourceProfile) + val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) + val ja = Extraction.decompose(Seq(gpuArgs)) + val f1 = createTempJsonFile(dir, "resources", ja) + val parsedResources = backend.parseOrFindResources(Some(f1)) + + assert(parsedResources.size === 2) + assert(parsedResources.get(GPU).nonEmpty) + assert(parsedResources.get(GPU).get.name === GPU) + assert(parsedResources.get(GPU).get.addresses.sameElements(Array("0", "1"))) + assert(parsedResources.get(FPGA).nonEmpty) + assert(parsedResources.get(FPGA).get.name === FPGA) + assert(parsedResources.get(FPGA).get.addresses.sameElements(Array("f1", "f2", "f3"))) + } test("track allocated resources by taskId") { val conf = new SparkConf @@ -253,15 +292,16 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite try { val rpcEnv = RpcEnv.create("1", "localhost", 0, conf, securityMgr) val env = createMockEnv(conf, serializer, Some(rpcEnv)) - backend = new CoarseGrainedExecutorBackend(env.rpcEnv, rpcEnv.address.hostPort, "1", - "host1", "host1", 4, Seq.empty[URL], env, None) + backend = new CoarseGrainedExecutorBackend(env.rpcEnv, rpcEnv.address.hostPort, "1", + "host1", "host1", 4, Seq.empty[URL], env, None, + resourceProfile = ResourceProfile.getOrCreateDefaultProfile(conf)) assert(backend.taskResources.isEmpty) val taskId = 1000000 // We don't really verify the data, just pass it around. val data = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) - val taskDescription = new TaskDescription(taskId, 2, "1", "TASK 1000000", 19, 1, - mutable.Map.empty, mutable.Map.empty, new Properties, + val taskDescription = new TaskDescription(taskId, 2, "1", "TASK 1000000", + 19, 1, mutable.Map.empty, mutable.Map.empty, new Properties, Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), data) val serializedTaskDescription = TaskDescription.encode(taskDescription) backend.executor = mock[Executor] @@ -271,13 +311,15 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite backend.self.send(LaunchTask(new SerializableBuffer(serializedTaskDescription))) eventually(timeout(10.seconds)) { assert(backend.taskResources.size == 1) - assert(backend.taskResources(taskId)(GPU).addresses sameElements Array("0", "1")) + val resources = backend.taskResources(taskId) + assert(resources(GPU).addresses sameElements Array("0", "1")) } // Update the status of a running task shall not affect `taskResources` map. backend.statusUpdate(taskId, TaskState.RUNNING, data) assert(backend.taskResources.size == 1) - assert(backend.taskResources(taskId)(GPU).addresses sameElements Array("0", "1")) + val resources = backend.taskResources(taskId) + assert(resources(GPU).addresses sameElements Array("0", "1")) // Update the status of a finished task shall remove the entry from `taskResources` map. backend.statusUpdate(taskId, TaskState.FINISHED, data) diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index a087f18b3fc6d..c0637eeeacaba 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -18,72 +18,97 @@ package org.apache.spark.resource import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD, SPARK_EXECUTOR_PREFIX} +import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY class ResourceProfileSuite extends SparkFunSuite { override def afterEach() { try { - ResourceProfile.resetDefaultProfile(new SparkConf) + ResourceProfile.clearDefaultProfile } finally { super.afterEach() } } - test("Default ResourceProfile") { val rprof = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) assert(rprof.executorResources.size === 2, "Executor resources should contain cores and memory by default") assert(rprof.executorResources(ResourceProfile.CORES).amount === 1, - s"Executor resources should have 1 core") + "Executor resources should have 1 core") + assert(rprof.getExecutorCores.get === 1, + "Executor resources should have 1 core") assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 1024, - s"Executor resources should have 1024 memory") + "Executor resources should have 1024 memory") + assert(rprof.executorResources.get(ResourceProfile.PYSPARK_MEM) == None, + "pyspark memory empty if not specified") + assert(rprof.executorResources.get(ResourceProfile.OVERHEAD_MEM) == None, + "overhead memory empty if not specified") assert(rprof.taskResources.size === 1, "Task resources should just contain cpus by default") assert(rprof.taskResources(ResourceProfile.CPUS).amount === 1, - s"Task resources should have 1 cpu") + "Task resources should have 1 cpu") + assert(rprof.getTaskCpus.get === 1, + "Task resources should have 1 cpu") } test("Default ResourceProfile with app level resources specified") { val conf = new SparkConf + conf.set(PYSPARK_EXECUTOR_MEMORY.key, "2g") + conf.set(EXECUTOR_MEMORY_OVERHEAD.key, "1g") + conf.set(EXECUTOR_MEMORY.key, "4g") + conf.set(EXECUTOR_CORES.key, "4") conf.set("spark.task.resource.gpu.amount", "1") conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.amount", "1") conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.discoveryScript", "nameOfScript") val rprof = ResourceProfile.getOrCreateDefaultProfile(conf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val execResources = rprof.executorResources - assert(execResources.size === 3, + assert(execResources.size === 5, "Executor resources should contain cores, memory, and gpu " + execResources) + assert(execResources.contains("gpu"), "Executor resources should have gpu") + assert(rprof.executorResources(ResourceProfile.CORES).amount === 4, + "Executor resources should have 4 core") + assert(rprof.getExecutorCores.get === 4, "Executor resources should have 4 core") + assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, + "Executor resources should have 1024 memory") + assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount == 2048, + "pyspark memory empty if not specified") + assert(rprof.executorResources(ResourceProfile.OVERHEAD_MEM).amount == 1024, + "overhead memory empty if not specified") assert(rprof.taskResources.size === 2, "Task resources should just contain cpus and gpu") - assert(execResources.contains("resource.gpu"), "Executor resources should have gpu") - assert(rprof.taskResources.contains("resource.gpu"), "Task resources should have gpu") + assert(rprof.taskResources.contains("gpu"), "Task resources should have gpu") } - test("Create ResourceProfile") { - val rprof = new ResourceProfile() - assert(rprof.id > ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) - assert(rprof.executorResources === Map.empty) - assert(rprof.taskResources === Map.empty) + test("test default profile task gpus fractional") { + val sparkConf = new SparkConf() + .set("spark.executor.resource.gpu.amount", "2") + .set("spark.task.resource.gpu.amount", "0.33") + val immrprof = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + assert(immrprof.taskResources.get("gpu").get.amount == 0.33) + } - val taskReq = new TaskResourceRequests().resource("resource.gpu", 1) - val eReq = new ExecutorResourceRequests().resource("resource.gpu", 2, "myscript", "nvidia") + test("Create ResourceProfile") { + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val eReq = new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") rprof.require(taskReq).require(eReq) assert(rprof.executorResources.size === 1) - assert(rprof.executorResources.contains("resource.gpu"), + assert(rprof.executorResources.contains("gpu"), "Executor resources should have gpu") - assert(rprof.executorResources.get("resource.gpu").get.vendor === "nvidia", + assert(rprof.executorResources.get("gpu").get.vendor === "nvidia", "gpu vendor should be nvidia") - assert(rprof.executorResources.get("resource.gpu").get.discoveryScript === "myscript", + assert(rprof.executorResources.get("gpu").get.discoveryScript === "myscript", "discoveryScript should be myscript") - assert(rprof.executorResources.get("resource.gpu").get.amount === 2, + assert(rprof.executorResources.get("gpu").get.amount === 2, "gpu amount should be 2") assert(rprof.taskResources.size === 1, "Should have 1 task resource") - assert(rprof.taskResources.contains("resource.gpu"), "Task resources should have gpu") - assert(rprof.taskResources.get("resource.gpu").get.amount === 1, + assert(rprof.taskResources.contains("gpu"), "Task resources should have gpu") + assert(rprof.taskResources.get("gpu").get.amount === 1, "Task resources should have 1 gpu") val ereqs = new ExecutorResourceRequests() @@ -97,70 +122,59 @@ class ResourceProfileSuite extends SparkFunSuite { assert(rprof.executorResources.size === 5) assert(rprof.executorResources(ResourceProfile.CORES).amount === 2, - s"Executor resources should have 2 cores") + "Executor resources should have 2 cores") assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, - s"Executor resources should have 4096 memory") + "Executor resources should have 4096 memory") assert(rprof.executorResources(ResourceProfile.OVERHEAD_MEM).amount === 2048, - s"Executor resources should have 2048 overhead memory") + "Executor resources should have 2048 overhead memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount === 1024, - s"Executor resources should have 1024 pyspark memory") + "Executor resources should have 1024 pyspark memory") assert(rprof.taskResources.size === 2) assert(rprof.taskResources("cpus").amount === 1, "Task resources should have cpu") - - val error = intercept[IllegalArgumentException] { - rprof.require(new ExecutorResourceRequests().resource("bogusResource", 1)) - }.getMessage() - assert(error.contains("Executor resource not allowed")) - - val taskError = intercept[IllegalArgumentException] { - rprof.require(new TaskResourceRequests().resource("bogusTaskResource", 1)) - }.getMessage() - assert(taskError.contains("Task resource not allowed")) } test("Test ExecutorResourceRequests memory helpers") { - val rprof = new ResourceProfile() + val rprof = new ResourceProfileBuilder() val ereqs = new ExecutorResourceRequests() ereqs.memory("4g") ereqs.memoryOverhead("2000m").pysparkMemory("512000k") rprof.require(ereqs) assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, - s"Executor resources should have 4096 memory") + "Executor resources should have 4096 memory") assert(rprof.executorResources(ResourceProfile.OVERHEAD_MEM).amount === 2000, - s"Executor resources should have 2000 overhead memory") + "Executor resources should have 2000 overhead memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount === 500, - s"Executor resources should have 512 pyspark memory") + "Executor resources should have 512 pyspark memory") } test("Test TaskResourceRequest fractional") { - val rprof = new ResourceProfile() - val treqs = new TaskResourceRequests().resource("resource.gpu", 0.33) + val rprof = new ResourceProfileBuilder() + val treqs = new TaskResourceRequests().resource("gpu", 0.33) rprof.require(treqs) assert(rprof.taskResources.size === 1, "Should have 1 task resource") - assert(rprof.taskResources.contains("resource.gpu"), "Task resources should have gpu") - assert(rprof.taskResources.get("resource.gpu").get.amount === 0.33, + assert(rprof.taskResources.contains("gpu"), "Task resources should have gpu") + assert(rprof.taskResources.get("gpu").get.amount === 0.33, "Task resources should have 0.33 gpu") - val fpgaReqs = new TaskResourceRequests().resource("resource.fpga", 4.0) + val fpgaReqs = new TaskResourceRequests().resource("fpga", 4.0) rprof.require(fpgaReqs) assert(rprof.taskResources.size === 2, "Should have 2 task resource") - assert(rprof.taskResources.contains("resource.fpga"), "Task resources should have gpu") - assert(rprof.taskResources.get("resource.fpga").get.amount === 4.0, + assert(rprof.taskResources.contains("fpga"), "Task resources should have gpu") + assert(rprof.taskResources.get("fpga").get.amount === 4.0, "Task resources should have 4.0 gpu") var taskError = intercept[AssertionError] { - rprof.require(new TaskResourceRequests().resource("resource.gpu", 1.5)) + rprof.require(new TaskResourceRequests().resource("gpu", 1.5)) }.getMessage() assert(taskError.contains("The resource amount 1.5 must be either <= 0.5, or a whole number.")) taskError = intercept[AssertionError] { - rprof.require(new TaskResourceRequests().resource("resource.gpu", 0.7)) + rprof.require(new TaskResourceRequests().resource("gpu", 0.7)) }.getMessage() assert(taskError.contains("The resource amount 0.7 must be either <= 0.5, or a whole number.")) } } - diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala index b6d46d20d70d0..b809469fd7163 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -128,7 +128,8 @@ class ResourceUtilsSuite extends SparkFunSuite assert(resourcesFromFileOnly(FPGA) === expectedFpgaInfo) val gpuDiscovery = createTempScriptWithExpectedOutput( - dir, "gpuDiscoveryScript", """{"name": "gpu", "addresses": ["0", "1"]}""") + dir, "gpuDiscoveryScript", + """{"name": "gpu", "addresses": ["0", "1"]}""") conf.set(EXECUTOR_GPU_ID.amountConf, "2") conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, gpuDiscovery) val resourcesFromBoth = getOrDiscoverAllResources( @@ -139,6 +140,38 @@ class ResourceUtilsSuite extends SparkFunSuite } } + test("get from resources file and discover resource profile remaining") { + val conf = new SparkConf + val rpId = 1 + assume(!(Utils.isWindows)) + withTempDir { dir => + implicit val formats = DefaultFormats + val fpgaAddrs = Seq("f1", "f2", "f3") + val fpgaAllocation = ResourceAllocation(EXECUTOR_FPGA_ID, fpgaAddrs) + val resourcesFile = createTempJsonFile( + dir, "resources", Extraction.decompose(Seq(fpgaAllocation))) + val resourcesFromFileOnly = getOrDiscoverAllResourcesForResourceProfile( + Some(resourcesFile), + SPARK_EXECUTOR_PREFIX, + ResourceProfile.getOrCreateDefaultProfile(conf)) + val expectedFpgaInfo = new ResourceInformation(FPGA, fpgaAddrs.toArray) + assert(resourcesFromFileOnly(FPGA) === expectedFpgaInfo) + + val gpuDiscovery = createTempScriptWithExpectedOutput( + dir, "gpuDiscoveryScript", + """{"name": "gpu", "addresses": ["0", "1"]}""") + val rpBuilder = new ResourceProfileBuilder() + val ereqs = new ExecutorResourceRequests().resource(GPU, 2, gpuDiscovery) + val treqs = new TaskResourceRequests().resource(GPU, 1) + val rp = rpBuilder.require(ereqs).require(treqs).build + val resourcesFromBoth = getOrDiscoverAllResourcesForResourceProfile( + Some(resourcesFile), SPARK_EXECUTOR_PREFIX, rp) + val expectedGpuInfo = new ResourceInformation(GPU, Array("0", "1")) + assert(resourcesFromBoth(FPGA) === expectedFpgaInfo) + assert(resourcesFromBoth(GPU) === expectedGpuInfo) + } + } + test("list resource ids") { val conf = new SparkConf conf.set(DRIVER_GPU_ID.amountConf, "2") @@ -148,7 +181,7 @@ class ResourceUtilsSuite extends SparkFunSuite conf.set(DRIVER_FPGA_ID.amountConf, "2") val resourcesMap = listResourceIds(conf, SPARK_DRIVER_PREFIX) - .map{ rId => (rId.resourceName, 1)}.toMap + .map { rId => (rId.resourceName, 1) }.toMap assert(resourcesMap.size === 2, "should only have GPU for resource") assert(resourcesMap.get(GPU).nonEmpty, "should have GPU") assert(resourcesMap.get(FPGA).nonEmpty, "should have FPGA") diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 29160a3e0fd53..c063301673598 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.rdd.RDD -import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} @@ -173,11 +173,14 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo sc.addSparkListener(listener) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, + Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, + Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, + Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) assert(executorAddedCount === 3) @@ -214,20 +217,25 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo sc.addSparkListener(listener) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) backend.driverEndpoint.askSync[Boolean]( - RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources, + 5)) val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) val bytebuffer = java.nio.ByteBuffer.allocate(frameSize - 100) val buffer = new SerializableBuffer(bytebuffer) var execResources = backend.getExecutorAvailableResources("1") - assert(execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")) + var exec3ResourceProfileId = backend.getExecutorResourceProfileId("3") + assert(exec3ResourceProfileId === 5) + val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index 1397cb7b39004..615389ae5c2d4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -27,7 +27,9 @@ import org.mockito.Mockito.{doAnswer, mock, when} import org.apache.spark._ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceProfile.{DEFAULT_RESOURCE_PROFILE_ID, UNKNOWN_RESOURCE_PROFILE_ID} import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ import org.apache.spark.util.ManualClock @@ -47,6 +49,9 @@ class ExecutorMonitorSuite extends SparkFunSuite { private var client: ExecutorAllocationClient = _ private var clock: ManualClock = _ + private val execInfo = new ExecutorInfo("host1", 1, Map.empty, + Map.empty, Map.empty, DEFAULT_RESOURCE_PROFILE_ID) + // List of known executors. Allows easily mocking which executors are alive without // having to use mockito APIs directly in each test. private val knownExecs = mutable.HashSet[String]() @@ -64,10 +69,12 @@ class ExecutorMonitorSuite extends SparkFunSuite { test("basic executor timeout") { knownExecs += "1" - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) assert(monitor.executorCount === 1) assert(monitor.isExecutorIdle("1")) assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + assert(monitor.executorCountWithResourceProfile(DEFAULT_RESOURCE_PROFILE_ID) === 1) + assert(monitor.getResourceProfileId("1") === DEFAULT_RESOURCE_PROFILE_ID) } test("SPARK-4951, SPARK-26927: handle out of order task start events") { @@ -75,26 +82,38 @@ class ExecutorMonitorSuite extends SparkFunSuite { monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("1", 1))) assert(monitor.executorCount === 1) + assert(monitor.executorCountWithResourceProfile(UNKNOWN_RESOURCE_PROFILE_ID) === 1) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) assert(monitor.executorCount === 1) + assert(monitor.executorCountWithResourceProfile(UNKNOWN_RESOURCE_PROFILE_ID) === 0) + assert(monitor.executorCountWithResourceProfile(DEFAULT_RESOURCE_PROFILE_ID) === 1) + assert(monitor.getResourceProfileId("1") === DEFAULT_RESOURCE_PROFILE_ID) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", execInfo)) assert(monitor.executorCount === 2) + assert(monitor.executorCountWithResourceProfile(DEFAULT_RESOURCE_PROFILE_ID) === 2) + assert(monitor.getResourceProfileId("2") === DEFAULT_RESOURCE_PROFILE_ID) monitor.onExecutorRemoved(SparkListenerExecutorRemoved(clock.getTimeMillis(), "2", null)) assert(monitor.executorCount === 1) + assert(monitor.executorCountWithResourceProfile(DEFAULT_RESOURCE_PROFILE_ID) === 1) knownExecs -= "2" monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("2", 2))) assert(monitor.executorCount === 1) + assert(monitor.executorCountWithResourceProfile(DEFAULT_RESOURCE_PROFILE_ID) === 1) + + monitor.onExecutorRemoved(SparkListenerExecutorRemoved(clock.getTimeMillis(), "1", null)) + assert(monitor.executorCount === 0) + assert(monitor.executorCountWithResourceProfile(DEFAULT_RESOURCE_PROFILE_ID) === 0) } test("track tasks running on executor") { knownExecs += "1" - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("1", 1))) assert(!monitor.isExecutorIdle("1")) @@ -117,7 +136,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { test("use appropriate time out depending on whether blocks are stored") { knownExecs += "1" - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) assert(monitor.isExecutorIdle("1")) assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) @@ -139,7 +158,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { } test("keeps track of stored blocks for each rdd and split") { - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) monitor.onBlockUpdated(rddUpdate(1, 0, "1")) assert(monitor.timedOutExecutors(idleDeadline).isEmpty) @@ -173,19 +192,19 @@ class ExecutorMonitorSuite extends SparkFunSuite { knownExecs ++= Set("1", "2", "3") // start exec 1 at 0s (should idle time out at 60s) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) assert(monitor.isExecutorIdle("1")) // start exec 2 at 30s, store a block (should idle time out at 150s) clock.setTime(TimeUnit.SECONDS.toMillis(30)) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", execInfo)) monitor.onBlockUpdated(rddUpdate(1, 0, "2")) assert(monitor.isExecutorIdle("2")) assert(!monitor.timedOutExecutors(idleDeadline).contains("2")) // start exec 3 at 60s (should idle timeout at 120s, exec 1 should time out) clock.setTime(TimeUnit.SECONDS.toMillis(60)) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", execInfo)) assert(monitor.timedOutExecutors(clock.nanoTime()) === Seq("1")) // store block on exec 3 (should now idle time out at 180s) @@ -205,7 +224,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { test("SPARK-27677: don't track blocks stored on disk when using shuffle service") { // First make sure that blocks on disk are counted when no shuffle service is available. - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.DISK_ONLY)) assert(monitor.timedOutExecutors(idleDeadline).isEmpty) assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) @@ -213,7 +232,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { conf.set(SHUFFLE_SERVICE_ENABLED, true).set(SHUFFLE_SERVICE_FETCH_RDD_ENABLED, true) monitor = new ExecutorMonitor(conf, client, null, clock) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.MEMORY_ONLY)) monitor.onBlockUpdated(rddUpdate(1, 1, "1", level = StorageLevel.MEMORY_ONLY)) assert(monitor.timedOutExecutors(idleDeadline).isEmpty) @@ -236,9 +255,9 @@ class ExecutorMonitorSuite extends SparkFunSuite { test("track executors pending for removal") { knownExecs ++= Set("1", "2", "3") - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", execInfo)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", execInfo)) clock.setTime(idleDeadline) assert(monitor.timedOutExecutors().toSet === Set("1", "2", "3")) assert(monitor.pendingRemovalCount === 0) @@ -286,7 +305,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage1, stage2))) monitor.onJobStart(SparkListenerJobStart(2, clock.getTimeMillis(), Seq(stage3, stage4))) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) // First a failed task, to make sure it does not count. @@ -342,7 +361,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { throw new IllegalStateException("No event should be sent.") } } - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) monitor.shuffleCleaned(0) } @@ -351,8 +370,8 @@ class ExecutorMonitorSuite extends SparkFunSuite { conf.set(DYN_ALLOCATION_SHUFFLE_TRACKING, true).set(SHUFFLE_SERVICE_ENABLED, false) monitor = new ExecutorMonitor(conf, client, bus, clock) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", execInfo)) // Two separate jobs with separate shuffles. The first job will only run tasks on // executor 1, the second on executor 2. Ensures that jobs finishing don't affect @@ -401,7 +420,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { val stage = stageInfo(1, shuffleId = 0) monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage))) clock.advance(1000L) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), new ExecutorMetrics, null)) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b6ec869c425e6..f1bbe0b10e228 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,14 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( + // [SPARK-29306] Add support for Stage level scheduling for executors + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages#RetrieveSparkAppConfig.productElement"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages#RetrieveSparkAppConfig.productArity"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages#RetrieveSparkAppConfig.canEqual"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages#RetrieveSparkAppConfig.productIterator"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages#RetrieveSparkAppConfig.productPrefix"), + ProblemFilters.exclude[FinalMethodProblem]("org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages#RetrieveSparkAppConfig.toString"), + // [SPARK-29399][core] Remove old ExecutorPlugin interface. ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ExecutorPlugin"), diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 8f6ae5904f394..1876861700fc0 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkCon import org.apache.spark.deploy.mesos.{config => mesosConfig} import org.apache.spark.internal.config._ import org.apache.spark.network.shuffle.mesos.MesosExternalBlockStoreClient +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor} @@ -716,7 +717,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty, - Map.empty, Map.empty) + Map.empty, Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) backend.driverEndpoint.askSync[Boolean](message) } 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 2e9576e3355d1..1e8f4084ef9c7 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 @@ -48,6 +48,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Streaming.STREAMING_DYN_ALLOCATION_MAX_EXECUTORS import org.apache.spark.internal.config.UI._ import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -455,7 +456,8 @@ private[spark] class ApplicationMaster( val executorMemory = _sparkConf.get(EXECUTOR_MEMORY).toInt val executorCores = _sparkConf.get(EXECUTOR_CORES) val dummyRunner = new ExecutorRunnable(None, yarnConf, _sparkConf, driverUrl, "", - "", executorMemory, executorCores, appId, securityMgr, localResources) + "", executorMemory, executorCores, appId, securityMgr, localResources, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) dummyRunner.launchContextDebugInfo() } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 7046ad74056fc..d9262bbac6586 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -40,7 +40,8 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils -import org.apache.spark.util.{Utils, YarnContainerInfoHelper} +import org.apache.spark.resource.ResourceProfile +import org.apache.spark.util.Utils private[yarn] class ExecutorRunnable( container: Option[Container], @@ -53,7 +54,8 @@ private[yarn] class ExecutorRunnable( executorCores: Int, appId: String, securityMgr: SecurityManager, - localResources: Map[String, LocalResource]) extends Logging { + localResources: Map[String, LocalResource], + resourceProfileId: Int) extends Logging { var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ @@ -72,7 +74,7 @@ private[yarn] class ExecutorRunnable( s""" |=============================================================================== - |YARN executor launch context: + |Default YARN executor launch context: | env: |${Utils.redact(sparkConf, env.toSeq).map { case (k, v) => s" $k -> $v\n" }.mkString} | command: @@ -207,7 +209,8 @@ private[yarn] class ExecutorRunnable( "--executor-id", executorId, "--hostname", hostname, "--cores", executorCores.toString, - "--app-id", appId) ++ + "--app-id", appId, + "--resourceProfileId", resourceProfileId.toString) ++ userClassPath ++ Seq( s"1>${ApplicationConstants.LOG_DIR_EXPANSION_VAR}/stdout", diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index f68be33e057bc..09414cbbe50a4 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -38,6 +38,7 @@ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor @@ -565,7 +566,8 @@ private[yarn] class YarnAllocator( executorCores, appAttemptId.getApplicationId.toString, securityMgr, - localResources + localResources, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID // use until fully supported ).run() updateInternalState() } catch { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala index d46424e5e99ad..669e39fb7c1c7 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala @@ -22,6 +22,7 @@ import java.net.URL import org.apache.spark.SparkEnv import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.YarnContainerInfoHelper @@ -39,7 +40,8 @@ private[spark] class YarnCoarseGrainedExecutorBackend( cores: Int, userClassPath: Seq[URL], env: SparkEnv, - resourcesFile: Option[String]) + resourcesFile: Option[String], + resourceProfile: ResourceProfile) extends CoarseGrainedExecutorBackend( rpcEnv, driverUrl, @@ -49,7 +51,8 @@ private[spark] class YarnCoarseGrainedExecutorBackend( cores, userClassPath, env, - resourcesFile) with Logging { + resourcesFile, + resourceProfile) with Logging { private lazy val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(env.conf) @@ -67,11 +70,11 @@ private[spark] class YarnCoarseGrainedExecutorBackend( private[spark] object YarnCoarseGrainedExecutorBackend extends Logging { def main(args: Array[String]): Unit = { - val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv) => - CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => + val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv, ResourceProfile) => + CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env, resourceProfile) => new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, - arguments.resourcesFileOpt) + arguments.resourcesFileOpt, resourceProfile) } val backendArgs = CoarseGrainedExecutorBackend.parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")) From f5f05d549efd8f9a81376bfc31474292be7aaa8a Mon Sep 17 00:00:00 2001 From: git Date: Fri, 17 Jan 2020 09:46:29 -0600 Subject: [PATCH 1138/1387] [SPARK-30310][CORE] Resolve missing match case in SparkUncaughtExceptionHandler and added tests ### What changes were proposed in this pull request? 1) Added missing match case to SparkUncaughtExceptionHandler, so that it would not halt the process when the exception doesn't match any of the match case statements. 2) Added log message before halting process. During debugging it wasn't obvious why the Worker process would DEAD (until we set SPARK_NO_DAEMONIZE=1) due to the shell-scripts puts the process into background and essentially absorbs the exit code. 3) Added SparkUncaughtExceptionHandlerSuite. Basically we create a Spark exception-throwing application with SparkUncaughtExceptionHandler and then check its exit code. ### Why are the changes needed? SPARK-30310, because the process would halt unexpectedly. ### How was this patch tested? All unit tests (mvn test) were ran and OK. Closes #26955 from tinhto-000/uncaught_exception_fix. Authored-by: git Signed-off-by: Sean Owen --- .../util/SparkUncaughtExceptionHandler.scala | 19 ++- .../SparkUncaughtExceptionHandlerSuite.scala | 108 ++++++++++++++++++ 2 files changed, 125 insertions(+), 2 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala index 2550634681453..e77128755363d 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala @@ -48,11 +48,26 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException: System.exit(SparkExitCode.OOM) case _ if exitOnUncaughtException => System.exit(SparkExitCode.UNCAUGHT_EXCEPTION) + case _ => + // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false } } } catch { - case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM) - case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE) + case oom: OutOfMemoryError => + try { + logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom) + } catch { + // absorb any exception/error since we're halting the process + case _: Throwable => + } + Runtime.getRuntime.halt(SparkExitCode.OOM) + case t: Throwable => + try { + logError(s"Another uncaught exception in thread $thread, process halted.", t) + } catch { + case _: Throwable => + } + Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE) } } diff --git a/core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala b/core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala new file mode 100644 index 0000000000000..90741a6bde7f0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala @@ -0,0 +1,108 @@ +/* + * 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.util + +import java.io.File + +import scala.util.Try + +import org.apache.spark.SparkFunSuite + +class SparkUncaughtExceptionHandlerSuite extends SparkFunSuite { + + private val sparkHome = + sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) + + Seq( + (ThrowableTypes.RuntimeException, true, SparkExitCode.UNCAUGHT_EXCEPTION), + (ThrowableTypes.RuntimeException, false, 0), + (ThrowableTypes.OutOfMemoryError, true, SparkExitCode.OOM), + (ThrowableTypes.OutOfMemoryError, false, SparkExitCode.OOM), + (ThrowableTypes.SparkFatalRuntimeException, true, SparkExitCode.UNCAUGHT_EXCEPTION), + (ThrowableTypes.SparkFatalRuntimeException, false, 0), + (ThrowableTypes.SparkFatalOutOfMemoryError, true, SparkExitCode.OOM), + (ThrowableTypes.SparkFatalOutOfMemoryError, false, SparkExitCode.OOM) + ).foreach { + case (throwable: ThrowableTypes.ThrowableTypesVal, + exitOnUncaughtException: Boolean, expectedExitCode) => + test(s"SPARK-30310: Test uncaught $throwable, " + + s"exitOnUncaughtException = $exitOnUncaughtException") { + + // creates a ThrowableThrower process via spark-class and verify the exit code + val process = Utils.executeCommand( + Seq(s"$sparkHome/bin/spark-class", + ThrowableThrower.getClass.getCanonicalName.dropRight(1), // drops the "$" at the end + throwable.name, + exitOnUncaughtException.toString), + new File(sparkHome), + Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome) + ) + assert(process.waitFor == expectedExitCode) + } + } +} + +// enumeration object for the Throwable types that SparkUncaughtExceptionHandler handles +object ThrowableTypes extends Enumeration { + + sealed case class ThrowableTypesVal(name: String, t: Throwable) extends Val(name) + + val RuntimeException = ThrowableTypesVal("RuntimeException", new RuntimeException) + val OutOfMemoryError = ThrowableTypesVal("OutOfMemoryError", new OutOfMemoryError) + val SparkFatalRuntimeException = ThrowableTypesVal("SparkFatalException(RuntimeException)", + new SparkFatalException(new RuntimeException)) + val SparkFatalOutOfMemoryError = ThrowableTypesVal("SparkFatalException(OutOfMemoryError)", + new SparkFatalException(new OutOfMemoryError)) + + // returns the actual Throwable by its name + def getThrowableByName(name: String): Throwable = { + super.withName(name).asInstanceOf[ThrowableTypesVal].t + } +} + +// Invoked by spark-class for throwing a Throwable +object ThrowableThrower { + + // a thread that uses SparkUncaughtExceptionHandler and throws a Throwable by name + class ThrowerThread(name: String, exitOnUncaughtException: Boolean) extends Thread { + override def run() { + Thread.setDefaultUncaughtExceptionHandler( + new SparkUncaughtExceptionHandler(exitOnUncaughtException)) + throw ThrowableTypes.getThrowableByName(name) + } + } + + // main() requires 2 args: + // - args(0): name of the Throwable defined in ThrowableTypes + // - args(1): exitOnUncaughtException (true/false) + // + // it exits with the exit code dictated by either: + // - SparkUncaughtExceptionHandler (SparkExitCode) + // - main() (0, or -1 when number of args is wrong) + def main(args: Array[String]): Unit = { + if (args.length == 2) { + val t = new ThrowerThread(args(0), + Try(args(1).toBoolean).getOrElse(false)) + t.start() + t.join() + System.exit(0) + } else { + System.exit(-1) + } + } +} From fd308ade52672840ca4d2afdb655e9b97cb12b28 Mon Sep 17 00:00:00 2001 From: Luca Canali Date: Sat, 18 Jan 2020 01:00:45 +0800 Subject: [PATCH 1139/1387] [SPARK-30041][SQL][WEBUI] Add Codegen Stage Id to Stage DAG visualization in Web UI ### What changes were proposed in this pull request? SPARK-29894 provides information on the Codegen Stage Id in WEBUI for SQL Plan graphs. Similarly, this proposes to add Codegen Stage Id in the DAG visualization for Stage execution. DAGs for Stage execution are available in the WEBUI under the Jobs and Stages tabs. ### Why are the changes needed? This is proposed as an aid for drill-down analysis of complex SQL statement execution, as it is not always easy to match parts of the SQL Plan graph with the corresponding Stage DAG execution graph. Adding Codegen Stage Id for WholeStageCodegen operations makes this task easier. ### Does this PR introduce any user-facing change? Stage DAG visualization in the WEBUI will show codegen stage id for WholeStageCodegen operations, as in the example snippet from the WEBUI, Jobs tab (the query used in the example is TPCDS 2.4 q14a): ![](https://issues.apache.org/jira/secure/attachment/12987461/Snippet_StagesDags_with_CodegenId%20_annotated.png) ### How was this patch tested? Manually tested, see also example snippet. Closes #26675 from LucaCanali/addCodegenStageIdtoStageGraph. Authored-by: Luca Canali Signed-off-by: Wenchen Fan --- docs/img/AllStagesPageDetail4.png | Bin 26211 -> 18098 bytes docs/img/AllStagesPageDetail5.png | Bin 45720 -> 33047 bytes docs/img/JobPageDetail2.png | Bin 25120 -> 24088 bytes docs/web-ui.md | 5 ++++- .../spark/sql/execution/SparkPlanInfo.scala | 8 +------- .../sql/execution/WholeStageCodegenExec.scala | 3 +++ .../sql/execution/debug/DebuggingSuite.scala | 2 +- 7 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/img/AllStagesPageDetail4.png b/docs/img/AllStagesPageDetail4.png index 7880f950345d3212def5d669a2ecdceef789ce97..2f038b3d6196b8af7b124ad3bab7a3e4ac2383d8 100644 GIT binary patch literal 18098 zcmcG#V{|58@GctLw(W_X2`08}^POa3cq+F#pcm2lYn(F5oVzQet2= zAfmtj6eufE1yL}t`Z&aQ6X<_BJV08<1q=*j;J*WW#Hq|23`|l=MnY8G!|);p)mnc! z3;0lop$v$UZ_KTZh)yeW%xi`x5K&Y3(5w;v4Q06)X>a+~tP%r{5Q-?dh$b6#qLKlqeeZsDHv2EnE;;cw}%gktiCJN_~7H zF@S`P?_I)|*M8w5^L`i@vw0hdYhb={>(-wt&7foBEVJ`g~Dl1~VIZ zXRVEfo3$VeCaqq=Q}g+{ZYd^kLu9TViR{Q1;7r*S^Q4X(BfsOZvDRh<)$0T#Ts8Mb*i}in6NTk>Ru?h_U#DB_L0j+_^`qAd~JW+ZdZmY4V&zA zDAYZT!sBJkVG-Thh(;$-dKdidfMolOc;A9oT&u^7w{Z*=rybdSw7Pxc8l#B*vQ}pJ z1Hh^6A+6YnJerr`eW0{~yR;9!dS>7qzI|hRP&4QwpJ2O<<2dg@*Uk)^#pRf3dP74} z+();5Vt+~heVi-F2gwH6TI$g zR2~6`w^Dp3uXfz@%z^q8H7H_tjiqG%cCK93Z5dgD$j~pVzY$Km+cVE;hjRq8t*?Ny zgtyzG8+brNx69u(P}ET(XTZl#^z^&u^)*xcBg3nZ&xzTQ_o*{C&cdd;1|-z}KiSI; z%=BeAq(VBmIYH>C_;v5|^d2Hj{84PLJ_1Pk*=9%Cr2m`)_|G}DIYBkuV*%Vu-!I{^ z=20ff+sm=no@p)rbfyQtXqbm@<#4qT^6@^4DGyW&+vrOIttf9Bg9Rwryga8WiZzBIIzMEWxC?n&`es%* zNlDd?y~)}X{idUr(?EQJ*A+0*mkD{eihUV!m8D8rB?kcJ@ozeotV(?=f6d~zY*Ahh zBSin?EAbZ6eO~9?Lehd~X@sA2SKQY44wryk87)n9K{Y+*1V>D~YU2mnm;ME3($5Y- zUH-*82VP@k25k0iTfCND2x4PRs+Dgn7$R z`|;@Yb(^KeW@gK0y+ZE8 zXMP*wK7mUUC-YaR&2>?^qZzy1-v>SRVn*r{BD?#bF|kfP;2pq9v^Ed5-r`^;Mad~L z4;gIcQg?#k@#k-F&b*1E|JW1H#I{@C3G+)a6t0SkZ4au;Pm8(|?K0)&t=LPx`aTkG8f%7HRXE#1f$sukwk@-Til2_A3 zX@fyC`_6~AVr+kQQOvKwu7g=V1jHzgcp-|~`XYRGUrIL8Q$?n}AjbvHl3jot@q%?f z^3tN%$LBu4Cyrb35-UGH^Jt0gSH1TsVL-fo7(<@7gYC^J9TJH{*b(t5k{ zXmn0$GHJnYy(uiFFTzvX)NFBE&@Gl*YqWj#*-zsvF;HCLcG~KE;|UHQ9UbQ zQ1azlG~pfKho1xnY6@NU$Ex7Lh&_6Xot_Zp&rc?WxPd?K(PunLm*)W`v`t3(Ubr;A zYf??#*MH)iV=^Z7sTEQ!eJn5d zHHXGNDZE^Vh+11(h}Fcz_R{N2Mwvj*7oGZdQ5^QTD^HDBdN)#L`SCNK8z>`A z()3$y7M$S~UeA){=ZiUHZkRl$u+-y`T^^H?z$`w)Vq&*BdLfwg+BF=^H+~ApG6w6YtZ(Q5@!c@%EJa-9G*ce0gw) zOVMK*58Gm?G5e936n+J!vJ-O#Q}6n`qK&^3-BsoWPRXk4gzv8Jx~vqA|2$$TCFI<`sw&4Ju?%QmR{57Tt7u@%mdILtRyyfdJh+K_aHG0t$T z*jtpzgJDoc(!~s@YNNL#hd2;r>%i&*9Uc7e0q48^xY2JRO^VY`m*2$e~yV_#8u9*i)12A6WAeW{mZ@{T- z?`8%Dk#7y6q5-2N83&29=ycZc&I(93ygh7b<5z!Y&|M~ZIn?Do{-_^Xt^8e2?!sCd zI5(_kYTl8_ZQ}c6nS+cNap)qc3_kfpTRr2WuJWp}%f}+i2|UuX zE(F#GJ4}FqP$ScYTMnKuOm(JmKRiBNc7m(S37-h{rHdy-J0uVO%7U=##w=(Q18a*g z%-$cjNn{Z!Mu7rqAD>2d8$tX$7t>ef1qha>gXLSDi)yC1|Ak+Fs^44$3sM+!Vd;Xg zF@W1{gINBxdyRjmZZv5_f_ldkmm5sKYNgiDm3sBuIy<*O?=iM^qYdpkjnuc&^!#zZ zg4U$-#QOa)rM21n$Id76%%eUI4UOXW_2t6mj^6segQ)ksC?vp}(cOg~bo%qQGwH^S z8+k($S3wsei~UJ4u6z5w57)#FF|fj;eG5{zjiD=GID(f&*#j?hu#f{SL6uu$!(-H* zpkkLvam|Ykq!--H;rIw{#jUKM<;SzVyylM>i?X%Mz~$eQ8D3ltbm7tnV8~e}d{-qS z#wQ)G*hf2qXZv$(Q4E;D+M(3G2wo3ozPm?@p)qUvuKZk2;p~ijw0Cv+RY`hDZ2Ts1 zg$JmLVoxY9JI}rv$`eqF5tPniY)7FZ`#R8hzHpuwZP`m8N9l^#8rkn>9FG+A4OnD@ zdv(9`S{+G3)@qkl2*tF!Y+IW8BBFG7{eC06InaLu-wua@(WuYWcA|ulHm^5Z$}tiN zct)2S4M7}i!9Bj(K9PpDgcZ@7K_cmpTJ@V*^*UL=mDnj^atiT40O3gLqVzc8TRW%t zdEe~`7s~a7XTTgT+~T3n-n>6ZzB0pFxr((Wuofn^s~bt=Ylk0S*K-5HB5_Te?Kij? z2>jt_l2C3t#|OfrUH17K(3Zx)DFQRiupvMtX9dmC+MSH0k?|qXF?Wa zX%h=i^MWrMuYI;_ z2P>DR_}%-;`rTFb?@dIvFznyju`B8)j>F55y|CAI>^n|3F6$54)n^B5u1=_!;wKZy z_3N3Lg&mw%qsnEd9f zfjNf~VXkgG<4yltM7+hd5?dvNQ&)>FYIr4E&_@4t2b)&Q%eU!HuIiv|S+TChqGg9r zuj;%Js}J#)F^8bp>6&&*)@9ER*FwRiY?3P-e!E&&wyQ}u)S<%vb|EJGtRGZeeBx4$p=sR zN`NNM>t#fRvq+c}e>vShxWFmj<%$ye%3gP-a?dWG%8J^%Bb!(MVvnCHY!L7=QB<|* zy|W|Z;$8xpB7EJ(P`=B5w^oLXP`E8h8d@LLots2tzWjOafxY!UrE$$2`-V~1^@9}h zYG8`)=famZd*^36>znsq#H(-QOd|W~cI&(MVQ2Z}1|0e6Pn^PpQ+d%->}nvd4-o0? z3GE2V6d?+*CZWtVojvw}Vda^OH*D_XC(>Cp{GC_gDeP2bJGcLa9Z6Yp5l;Bc=zLef zr0gX(6G49?IJb|EkM66%`;5dl&AvIoyu+*d29fEso`*YQbdlqxr>cE1rp~i#O8QH2cUTn{JXY^Caj=oXlw*p$)TeObS z3{yQ1Cn!ZC3mi5Ms!X?*?{DZOY^;izwHSV2-&Xlr0gvWm%G4@u&>JMM<19*pjao zvo0SaDU6mjuT(FeGH)5$2(JcMf01|BpkT}adwSZTE=9>q$Mr49xFodvR$mr>U8GMZ5r7@ZvkP7r;L3%}y(04c+4Kv7wZFfPr`SJ!!()l% zgfHi&RPx=&ot9jJc(yVj*=iqT3#WH1 zJJ?o_=<%Bc8~Po)eC}zESqy_7y(027p~j$9&nNxoO$BaoER8KzTz~|(zR9Nd_rEDb zHDNyNu#~JmsO_M$YIQ{7xb>)4F1jhq<$tz$m7SwIhvCZKA?Nyr4eE> z+HUUz-4abAK}rKPbNoQ0OHBJN-e2x&h8r(2PThm#7?p6V?kz=G13h}K&V_qm*?l5z zl=W^%$lJSBn-{sLru8K%r<1#5vc9`txj>EPxHe;0ayL_$C)XSl^j>CG)wkH6_FlN^ zo{&P{_7~i`MNIX!_aVNN9k9oI*SAq||J9nq=Hc0g5ik+u67TEA z+C(Y@PKoKF4CrAP$OnAzm{N}9{LpsJ85fM`nU2y5xoOT)n|TOg!IOjQ@ZS}x-!-|q zWEnMVKBEeG<<)@>z`;-)FT?(maCD%|&4+SUS>qnR*d;NUx7e0q{(qm;aQpf&z8Y z8&wag-6MU!Is=E98c) z{^fc4wtf(!{7#k)O8pHx!V7AXr?-+Z%(BbchJ`Qp-K&a+%y#u!A_Avkutgy(VXJuM zQtLC(-81sdKyjR;@wMwZUB#M%4+Z&L1W-9PVn)jpO4F-U*rPi<&+sgi(t zZ$SJVm+mv#@&ER}SEMtl+v=j5eP%N^((zCXIGU}X)K|eecy$AF0cOTrQGIt7YZtsH zFAbU05Px;qw8}2VlJ@V0vr_lg?HoC*)lB{kvitbux8}3cFA^mFHmZG27cn$BG*rB?LB0qHL#|wImiEkU zH#AnGU>u?`f1~e6tn|BQ>wMgIEPFzrW($0O9cyZnJQQ6Vs0QpEnB29cR=`-#-&2wAVHwbS#1+l;Nk z<; zfBNx3PIJN)JQ7?{9kxq5qF2BjTA=ZUi=3&lquuedvlCz3cAa8@XLiN5%6}>^i`_Yq z@5`~?*l7k%g!c5kDl`<7*E`X1wd*z4f^?7P#5-%ePv>%LVeJ|ji@-9N67u@(VAbOPfe_kS1OAO|1Q$63-1&`wOJGw= zM?C>X+-11Ce^RJ4{D{Jadp)em}f*p13dn8!Ep08SD)>?JryW(B9>+aNipQEzQQj&)002r7OXYTF;cDU1hCQ3yt@-&h~Zcp&(9*9O;GsZ??(>!ih z?kwMxisHT-y`ouGEhtVh%Ae*wbnPShn~zRIiFyQGvD=5RH0xN&q3Yu;ZB@6!==@0! zOvN2?0cgB6vQ=-P%3#5lOw1Y*1hnyMjj?Qu>Jx_y$7aKo3(rBCiP%)?E5(ut^K1}U zkUWu}_*y^>*WyHD31uHrYyT013tn?vsBk#CoNkuB0C7u?O;6y`PCWCo?A`UhSLNhT zJp5XSkv|MeCaP~KUq#AFvod~vhd;Skv7^$%CX~v|KO7Bj@uUVP?ijV(CDc>L3H%41 z>r4i3OC09c4lR|C6MIssz5FAysvIsWgYTolx_Bhp%0kt@(9HKbty@jYgI-HfN3F=y zOb$pV?*9$z(sz4a){K*NzH;pM@*ouQZ{VZywZy>lX+V- z|7DztC+mnVDbU*2;s8D}#^>qyH$#B_@|%Z--!6sh%T|!+rzhID)*L|Hpv5(|K~;;y z%H%Juc8)I7=H-Z~KG>muyFB#DYqhQi;@EE>eKgU!6!4!0lJ z+xlfd*Vf5^RSNyK5KV6a)tt8-JcQO=nX7HS^;tOb)d#%heviiwg94#A<#cRS^)L>J z!_yJE5A-5Gi(U;{tGvPLq{T0;K-Ci% z)qz&`^=m^z++P>DrifWh-IpmqF+xPe4NYtZF^D=mMOl|ROw8m7OwEX&Zxm0gA5!?| zW?!1V6xNUmf%At~Tm7WH*qH?Z$XW~$I%l46XgehNX$d=haQj^8TWt|c%aw$lAgook z807lEYV17Z%(*V(SYH5wD;JG1aUdek-K!EEp~ z1sn@h!*7y@rAKQgPqVusAU@Mh(sO_64r8~y669~^j&36A_O%ZQ=Ih%asA4>PP#kja zL7ToyKKyuqGR4R}+>)g~LiIa&#D^dG<@1xPec`(g34KAL873QJO8;jS68V&vn}}aQ zzvjb?P?m+8a8Q46PB1y>p_=VC4Z6qzA2i)Y_1N$6Nt${JU)^M>g}{^}Gy2gP1S!!{ z;Dm$+r*{5y7kP-iR~TH?ViVEM!=zEWDIKe8?q)1q_DYY?(w`~-6M{TV`n0rY?YLtz zwVJ6FanPi&6bT5>h>kKS4)5($xKvAow<|BGE@H@<{uHZPN@&FlOc%b;u;O!TK0|6= zIj!hc^I7iF@Kd3{8F_7cS}xsWMED!6S~7QS>`!t(J0=hr>Or>q_S zZ+dGe!l!H~&;by6kaZNamqeJCl_-GK>e{~rqfSNX`jDF8+m;eSld+%(xg(vhqDXNE zVekFS;iNp3UI=+!zBX3~*`0|Q?kUe~=xYBVFC$+ZqOVQ1Qq5Z@Sbt=sg-YN%fkDMY z4&ZvIZwIgDmklR@uf$^DD~EuPr6s2@EUgN+kAC-!AJs+~D&&$@=46sNw*!2$u~oj9 z&F{eyo~7&ypjynm%~S--UN%^S*oWbV&urIlrM&_VcdGD3iNDJSk>KLu`X^Y%q^;2@ zwC<=0yFH0NUD?0?#O)9b*4a%M&?LZP7;sza)T~KoDyU7axFdEc6obp|>=*dPa@K(e-#H8Z#ZK5pqC10054YzlRR&D&pgWPLmo zClE_p=V(oCrk(}E1QSHrDTXD7)eOlEjbfRKaT8nh6(Q}nFgX&nMtsVyO+tDT>u%_V zwCE72<1-|IIpkaJ~ z%N_q#JVs+8+|267s(}pJpg}>?uIxQhvV~njXjOgc?hyefSe(YZF#@-9^JJWQnqf0(ddvW=Ttjncf)Q+zY%1I*m$ zHF>jvnp)7!xzsExb-B@4J$51(Pbe?<*6h87~`rl&&0v^S-w;b<#>Zg*BZ({2Y z@TL|%yKzV_8UGR>8AIe^{Y}V!(m2@viBgk@f`f$#n=8YXOXH10`sJvf)toGqJ3>MA z<9j~zKdlvc`w6si9W*kZWNwA^WsI&D)ZOXhh^3lj>hB_#m!)A39ufhr=?|(dy2L*x z@{gjHRU)pS4=-GC)mX7PQgeLw7EvvGYUPjByJ^E9Oqdbe)d8Tvf^@DxECWt$JwA?d z%p1|bqj5NLBMiq9ltk94!Nh}baOKdXW~j~+Z#^_54)+4mDJlxuzxYe#5fb4n=vbRlrs-xoiSsf3D=Vy zT2FV_93IZppCac;lk{k_Dtl^TWTZNwyeC}R&4KtA8!yK~)fv>OXIOx3HQMhR%b|{b z)KYQR=33&-&VIGNqv5J_XE^R_==t2QLtEXc`RKAqQ9s?vJRV?&6`0T+rYGsPMmgS?1s99B2DM?pKtyC@{Ie#a$5t?HOx}D5#H<+4vdj(&W}OKJSSFy4eo7ue2id zNE$VMrR4yx7VG+?5BK9CNpQn#m5+AHx&2u8JHCfNX!6ae4j$Nj ze&`OE5(E)ghoQ*b<6P;3x@!8;sDNuqjP+Z}wBD)|?P_|WuZ%OyG-T z!)dpw?)_C?L#j7>$MxZ(<9a7dm#OivM%jycgzyK=ZWG!VlY(dsMj*pAQAcq7?vH+r z96B*txReRSX*F6V*h$fn23g!NtsHI7#eCf!VpT6cj)VVlNqf=z`_xN5>DQ$*fU@4kKZz+qXA02L6R{F{qWBH&6I< zW!6Xn&mWZ2hbgIfAGuSDA-NY{oJow}|Cr!;(?iYVyI*QzvDoBAcOO>G?71fFR8%Iv zz^JvHeR}s`nNeyIJ#_I?XUF%0CSyDJlYY0)eVFZ2EF(IDg#;wYe@Lc6SNCU*s0!L! zdGe+fCg{{h{a75+a7sa6uQt|bm^}bp6b6geqC;x>Q+ifRvKw|TMny>1%j?c5YbY{e z=>}xIqY{8z@EAfW#?yLCzGQ*fi@UC|5J?NScPqam7i!0|V(5Hs-i6)#x9biUWrSbO z(Xx=L>zrs);wO%@ec$YXy{|*k+7lBthT_=+$&-XCAsGl))=%+=Eo=Ra*=_>YZI(?D zl=VM#Zs6p+D4_geV@kXK$SUA_z0UUO4yGKekV{kE*J`cr4hOz9I+Qnx3vwlYn7Sc&_~xF*6K*k-ki$F=V@?uxOJ$amq3KIrNb)pCZROlyU# z?Q~JXLlX=q_eP*UyGYzMWTz}r%Gsh_Fn`qZ3NoY>(qm}Q`ihl9yHn?D2*B;`KXMl8 zQ=$|qOvx{u1ipCFfNPHz!eFq!n>Vk%LQy^QIBp!TJXCv*UTIY^&j`$(UJaASk1y16 z3EFkFK1{w zoVy~W6t1ssDEJv!dVtr`Kk&0v+V}cqEG_Y3tPBDpyt%e{jh{41$kMV7=)dVS6DlRg zpS^nIm6hF&YerwHOtoPtc%nl_BkomF-;#-|2&~#$(8BzQsrk@tYR%eHa&!d==Lwczm=cS1p58@9t`W`V4|2DCvcGpgDjLA9x0k&6;L>_LwvG(=P z>+UeeA@4M`0x?ZIu@-2>Y&TXCa=--_H?3;l4KkCvQ6KhtPo#%$C1KI)T7y^!N=gIf z?y#|@t{>w@`9z*xmGGY0EF5&$GP2GxGx`rYzScJfM%pE8rOF3<_t7bg6TK&y+WziT zR}>>SoS5kp@K$?GJmCb3dZwS)Yl#xbHmfF(AMueAi({0ZP99*A^6Tg&5neIMzemcN zNl7T=_t06IF2isdpHFj*ft(jV(S__s;Lfy~)tLG(Wob>BDN#MUd_9aB8f*(D+mi4= zDjy8}DleaY>nrhuBupN8S&{|ps)f~g6+o{G66e-J4SyX)GlR4+8i%;v>!XZ5{L0NJ z8eaY3Wjp0QY|KI?X!KXDrY?sn#380PmL?|)JY`hiovR?1l+du~@9P@#;=|(;*~}Cx zV4QhGJ@!t@9XH3sd7BI!Ofjm~El{phj***rF2wJWfScNR>w}!S1mVCBO>BLHfqMa?VHD& z*?6dCh5IEJ8DWDO$gK?;Kig}T)RTb~3dTrMH^i=cSL^&>vG@)wZ`lzp>viY?r-hdU z2jt8jB=Ts1H{ML89w(Y9m=omV5l>bQ_SJ~rf-!Jk>uda6Vh9mI4@DF^hN2i4&9~e9 zLAN6v-$`^RNznn$?vH;T!u=eL&s)Z^9^13TK|W8CzWs-h_~S|<1tg%i03_M-}W zDK~N74&C{}#-H0+I$Zluxu-_PaZ6f4FOOo0$&N=~=D$*7K|9X>B~(HGe=0>zElkdq z4J$-B?$}@0j+**1E8Hq`GMuLy7;8)1m>&<>@vNfi`az`bOSz%c1z4$#KE3+U8P7{X z!kXC3_BQRAeAN|POeL9D1^1Km2``oPR^!6*-b(pRl9844sr!mheUu$4XM}WgdHl*_ zr1#-tE?pL(y;g0H!20^;u;R-oss_%gWNw(sR=#i=Z;b|lr=I7a zrt?Z5h$ne}{bho>YEZ9{xDM1M;3Ts=CR%bnP65L!W8efef}5ABhvwb89ew%5{X!P0 zxThQF5$hWI!0nf7`=k4M86*4|!|>fuWKgd9;rF0l{$51VE~V#3bkDb>p=D;f>$GNp zx$-}A(qp(-Qp&&Hu?2ak{!0!RIbt@ z-c1j65l6VbFJ@iun}4U{V7O~ugsF(Ox7BScF=y}=$no=39wQGnsts`xSzcct5g4Ko zucA|CGg8Ww4Wc~C&T+4kr1>ed&p7j^!F>ET>-C~#znb->)&6F}14Voey|l-atK|vS zwbO0hY=RD7Rd%SK@s>b%yiO2(AxeooQeV{hSfE#E=_xosh2jyUpjC1SsPb_`0Y65v1BPC84hK|p zxbh0>S35{9;x(TV6e#I0yXWEZg~TrR|Dx!Eja0@!QPwdXKHNeZfZthq`B`&rAAh z^z3Q_5mCkkGg){v*ct}D9>bJ@FC|(nB%_}OKyF84)LB_Tk98u8FB^$cc`Fh6EK1mk zUNbF@^6pOX59e{p3GQD&iaOx$?5A>6$FWB*H1FpOVn-M;AjzF(yk6@+Wjd<MgggOgBu~cpc5EOM1p}#@_TcvJC_xgWS36FgH z%5oe#3+GFlSo39jg6U|rT5=)(ZSn=4XJ?b&%Q81Ga{d_l6mtg|In^J_H%u0!YH66v z_*EEjJrjflbEZeKcc-SFKEY(U#eTVI<7`sUI}FUITH(Q(7)C#uY!u@D;gNUuGCf>l zVP9s4ouS>77;!PS=pOmo6XDY2McdAk`&h%u$eslpZ4w6}cw=UJq}xPqYQZ@9?_ zF!EYum7fG{^3-_cU*EtTYx{daFk8 zaP*!Y7U!F5+W0z@G8qrsFh@8XG_y;@sGWU>>_;cR0YfhQ4h{|SzIM#}wYotEc$tXZoSf7aHJz0$2?rD_3(G#k%{c#b z@|dZ9Lnfp|xJo43+Y49jw`C{y;(s~;h+0>F40UuP^zaE`eIvK04i3ljm-s!W8VY8` zi+Nc+J{Ga_S|lqsFc2G_TCVN6=QkXoO&Cp*`)x;7ObtoZB=Ux8T>~F)9(w82&w)nl zsF0+>DLrv+$@VNSb$>C%WJ6yrdV!WgG2koaLo&-C5phnS^WS z^_yOo9WwTM%oLH*K3V5NdN>W1CG#RiQq@g*tdle7`fFpp!e?~E&5L0 z)cB58Z9*l^ABK}haLqI>kL2Vbl-@SqR7iqV-BL?WfSK9PtKUSUM5{)ApjWk6nbRLG z)+LmNH%qYmnVAu1H2A^wyFUsu78fRnWDO$}gD3lU-QHHS*;GkMaopyzAucWy{M4Ko zkVy-x0&BtD6pN}i3$Y_iUvY+#ZG(4zIFqgWhqZnb%{3p{n(!k=Xvq$RgIA@3HAk?FUDl$Q^P4P&8FE=`yWQ{jMYAKZpA?rO_DK4!p4D^sif#v?(4zHYGGuU8@pq{ap z;|AUF_l=@~51`PFQ#xu#DC7&%_uhVdu-g=433d|G~O0-N%qbk1>>1PU1`+j8^?F_#KJtL;i&N|)T2hjnkm z#BMQlhSRL`11S1t+5%Vx;qQ?`j*S*TrbfRDu$TXEA(`vs z$(09qMs&Dqg^d(NODqp_x@kL19E$bBiS7^VD;AFy@pq%gUkl^jL6J)ETw#hpD8Kwa zVZJ?{-2WQT1{wZwcmMOgGpyo+g($1VT_0LQdP8|TN@vD$?{Yf=3tqm~TkndrO9Cgf z*6qhyRX4hYtSiWE*l;JNk!_qLJ0=SER@Qg)i(L?XxV8?2AnMh26*t9$`|QbnG_m3S zj~HstkhFnpp52S{FF!fE=&`oGuj-?Sk%@{IbhF9*hxd`vB9h^EEZ*bqJKDz=c*z~H zWt^srd_>Bi*1!9RauXacH+~HGy#j8st{e&sXAcE3g-B`T#y1DTg*i7z=3S^O_S=JTP_#tjg=#QJw~_b}lwI#(%>i zX5DsWMrX7moJ_sg=MtBv4uiz_ayoufb&8%gYFE`1Exv8RckN?JxpoO_*B~I)!WX2v^uZ@2mg< z$=Xx<-OHo|V*UW4BcTp(B$tX|TK~*M2s~CWJ2o7tLqAdw^XRS%mk1JVAINt(o9p)Fse#Zk*Y#%Tq(9HJEfGRI<-yXiX9@}z^(WI zTAX#p`JOoJBe8~hq$Oye;3VrOT7_-O(B>t56AhAQ8q=2&g{X_aVDH3vS19Xc!lnyIW={?Q-g@+l zht@$@EZLQ8VZ@C=mCdnh;7=OlPTYebmAy)WQXLpXz-i8r)PCK(QZUeM&{E!UVO`??M)nsuwe^|78_s`My>`1r1GV{K_hTFan!!8#hZ%oX zG$NfdNS545hx<)hT)f^e9Wf|nr64V;zxTmp!EP`{roORC_x%WN7P%9iFCD;Q4sH9u zEWBm9U~y8W^A)Jrlkjja*)w+-d1dGnl%U2~PpZ7)#MF$OT-e%DCsEa`$}TOkdPY^G zO(&ZFBu3R@N3}@GWl9cX?4l`5HxEZ}shH93{;KLhn_E6i20QAPq{l+Pgx8I0ygY7b zm@?0+5a!aFMt}~3*Rs%ys53(Wg}^oN`jVqLFszAxRUC{+XbT<#=v7eVG&Q@?gl!aw9>zFnR#ZtF* zj{*#~5v#NJW}wO$^)HaUELIiV*k_UI>!7-(7R={JX77=9H!X5HLQ1h-w(x?t`d1`Qf4+K+jrxeUp8E z-6@#%yoH21My5M#e)!|SDWuw3;tG-0rxDVLa)Xl$x)EXE)^5~IEppV|_eTw~b{j&92EVUvVCRTTp{iHm3{5y*ERk3qg}N<(;3y-h{mCs zpgpmFlTM&fWu23y#==SM(uQ3#!K_+= zxF}hfrlr-!yR&F7DTdDDo3Sg84uRF^-n~MYVaF=-b0bh~diZD8%7%EEK~e2|jH|6m zv9OvV(dl*kLntxq62!?Sd&o7OVR3|upglZWlV$U}>2nT+KE5T2q4xG88|NYgyq`l0 zQP1z_N@KhHP$*cwzUE!}wG0G$WR=Y|DMj3zY`b>;+;-d9Q%QQa#8hzcD|}+B#2ysIc^&pMglyH zx#m?9^|<1Jp}<%@>*(#x0XP8liLc|-!$lVO^zZ|38Bq@|YR0S1ITD%F9TdY?MJmK& zVJ!^}YpsoSHe`Qs8OBb+L_H4VeADYq-+!?1U8$w_ z>R$BKHIQ21SuOzPue!eJh`bW&%Kw#vZdDZpwVvO(MYk1%)<&I@8^Pj%Uwi-hPLBQF zOLz)0g^ajxXmDEK2qizu9bBWf{DV=g>Cy-Z+ZDeO9Bm+ns7L;H9vt!gXP?7@ZEz%o zET3!KRK20Gx95*@s${%e3e?5D_RJ22Sx96*7$2r=;v0`aNw0*ns{ab2I^ z!t%i&y(Q%@+wuE^JyH6iem{jh zf3AEhV0~f*D182-f&1eqk~b)#Emek!$zz;6^GE_3EA`RKvQ=m2x1(CyE29@A+FtNJ zV?C|6zbOrYrlV`eG-2j&?I?lLf)I*DQZ70eS!W*_c>*durb@;OJH!It|wVH}c z)~?zA#=vIf?0|q%-lw0%=sPblusPZK4LCRUXZl2;c+HI$uKd@!^X!v{&4H4?f0(PZ zH|;rf`ibPezo&{Et`w{4h#aY%;k5C(=;GK+u|FdIH3Ado>hZm=?PcQYzuuV7^rx(|EJBDy z(1SpdI)slPf{#d6)To3fWHSRY9fSX#41 zV5XQ*@bid;Th{(wvOS|cVD5LLz*Cp`taq7}WZc-T9a<5<9lpw0Gg#Z?g*$9K zQRyX9j%)2N?b89KjXM`ERq%YDG@}E!_;6heSCGut376LXSbO){GM>39_gl6uJ97Q3 z&7qc;mv222IsbZr`pWzSL-Q_;g66jG#f5MF#QmN1V&=*F+4rWXz7X8`tSz-HSLI-@ z__bMzRi;rt`X<`t{sV4GIrrhyN2mhHT9mdKI;Vst0E{zj7XSbN literal 26211 zcmcG#Wl$YY^eqZWfZ*;L+}%C69-QC~2X}V}uEE{);C66#C%8*+ch|@7Uv z7_ zqQ0ivW`M+W#!Y?=RQ(EK4icnQGD~Kml9qm~s`mQ!=!?Ch`Tp>(BVajcBrHsxZ5m>M zZik{^w*r0Ov;CcJl=h0}DpeX0O~_IA>M`1>$20&aG45=c%`HNYwW48Rtj^A{O|xBE zc9h6u!mrL_=dduUws4xz#hf|&E4c;05%mqL_8TlIhN_YXE~in$Nuh?+w8z?z(BU(M zBh4;tK%6loruiRTp-1A^?8>B^%**&*huoV12SEw{D&1Tqr_2lEA&X3i9^jUY-4mUQ z!G6N6mTV*uWXEdvfQzikz5f`Y!Pdwh`Q@ZQRS`kt_MB-h(|~vFT(5}??Hhs;WZ`tP zq;^Ooq=4x!rqwU)d~Ok>S$`{Y-&Q{ zX5nm3A}KAe_$vSj8v=p^LR##ns{6`mmWz(+^2^Ysn?EF`A})e3)F~qh>Nj|~nzlv6 zwzf9Dg*C8?jhOf;L|YBBus8}7CcUtse@mB;8G&%Ei(lJ|z{R4Ke)T|$)4}eINA^Lc zQ#JWmU6e2h%J**{s2KUM-@XJ0lLX2Cm;Um@`o9FqcSH2gM1b)3jQyqss^ zN+QBY5)cxCl&^nr4vpqKEF+WBY@;|qgG&G|*q=8XD)$Gr2xG;CU^^oo{T=4!qyLRE zcGd00(c|su0}>VNI&>W!h(!(B4`*+2e2=i>k&y~lE_X`Vp5)YNS=c5bb*GII9jfGi zOU%>QQiiL)e_^+qWGxu`?yYY=6U6u<9aug0@du3(72P|mtMTk$dmiOnvV&Bw5?V7H zR0LcE}^!TcJ;ZJ%f*X=kQSKB#oklHgdKX!PKcJiU`9KQdz28sQYJ(;6T!9jpu zg3fYfl1h*>;*TukX)0Ih0s~CVN%sDm7dercRFMp~TIs}+6X;_(9231Erc-`}**g?4 z*O_1GBT4@^a%B;%#h2)X$m*qs+{^g6f#$Zy>f-TqIug&nLTYPf(l{rxE9ZEYm=*Yu zS2`t^SM{5WaqqdF?`f&xPuG8t zOv|7k!zk`$zwU<|8OE6vS)-c>`?acLZf;za%8rqTvrr@eo|Y{vcKaE$LjWHRNpfdFa`RlBUvn{)DCZW-6?*R|@_tb)&n{-~ekxUb+VZ3p zc%MAv%a%rbi`My%D1ENJLeoujRlhENrVE_c&U#~|(h{(_(RT6naw6+X5N#tyHy;Xr z4HA6&;dDi4 zXe;jhrDC$aj!T1uxr>FPd5??&(rrh!q?RuWVG|@2OyHME;J@BrdY$G!rsZ69FH5kE z6GwsN;wxg!#LVd|8ARSGUxWnPQ_4U7!R7;GWM2Bjo?>we@kEwuk3#h3=*Oas)+fcI zh?g=Zv-oCP7VXV!*9IdvgYOb}1&!GFW<4%hYY85@{GYq{3-rGIt5>13+=n`0v)>~r zqwSQ-VrP0y<7DvXhDE5Xta+6Y+KT?bdUc_(A?q7$xEy`>C)cyraq5-dv7os3Q)RJY z#V;jr*^T+S5BiDi`L;_CI7<*dcFOb0Vf9a1N3$)aycJ|*JDauX#%OfBL^Hlwldgo> z;1g$w(X+89=~2YB;c<0%7?Eb{-G92;V_3UzPe|G*$1*tC0dup6X=?xSkHClVcj5l= zHs&|>4F9%iKAH>zAx@j6iRs9uu!vX~GECZ4?- zfaw#sTX>y!C1eicFwDuTGpi-nTiPEF#Hjey5*8*AqgB*+aNgSc?ZX_YnQq(ML$iJ;~aK*um5jN{>eeVouxD zso+1%`9?cRbn><1Dj1Rvl+H(*v4LsLeU6S8Ku$)piK-i&=om;c%=@A>i}^M##Awc$ z{3qfdTz}S1*58bb`4*f<<%9o#ag`ABt;j(0P#FzMqW#kp_w#;7lUD*jXB^9)h`;%v z=)e#=lEJMpvphL(=pGigtKNCl=Ul^%QztoMhTXEY#Hyr(Wv6RyOX-O$!l5wh^&g^s zet?{+`z6^E%^WV^-Y)f|wRYd&*OKqBkv}+*#_6qT_D`kD8ErQ4r;Zk%<;S(_UWZW} zVJeN!Un?_P@4tgB^x%@MWENS3l;)L10I;SPQ9Kp=zDbujxU*l1Krn7hrs{+mD|(cI~!J0VLx{NT{yBg?(+>fKv4FeManYXxS*M6OZ2k#D*;Yx zylEwzPw3?pN`Ws?Pv>D&i;asTzfxXG7+gnxo-Pc|)qyo0)~n!HnxhD__A{cH*KrbVCv%Vm z7IKKyl)os*wZ5|Uaku&`gwg=*oZQS&w>LPuYSGAw5_^RGrW~kEos15Xd6)LtrjyRb z!VohDD*O*f0(8ssUX|JZ_*ZKd%B_w0@!Hps65RG4d*)#mVcl zR~lXPtoK5p;_Fbg!RYzzaxriX`P$el)JjLYA>zt*xvS-QCemJn$f8?n^&LvqOh;i# zJ)a#)1=(>7LODSMHhONW>I6PGpI>d%bl}7Ki7YE=IXTOf#Y>|)~n5#S9-yU$DCK zBekj=D{=UoXv)5Z3X z)DZ4RgXQQ$B@w41(@E#Q3`xADvRh%*58f36lqxT1O2p8)tB~s_yQEVV5*0f!ImP*Z z{i@pcjzFbd&BtIciwsV?8k(I<%=P5$yXG|^>{_rJ%atx)^4X<*$f5LA#mEYIos9P& zq^^=}l1jJce$ei%Gab`loxyY>c|R2yJo_7wUN{nB+=g4R))}ql^n9qY3OmiG$$Taw z(RRp|fO{my@T)}~Ap`86rUIi}ZBDLz@O)Ab=k_$?@F-1YYEpy^X2L4q3K+-tOT~ee za|8q3bFNeU_Lw=ow@p@lJ4->Fhob&9`r1b`*5{k)Ua;z6@elCZ5so~e%$?_WCT`G; zyZLMZ`@zC@YAUThui`{NkI8a55^OXhqkn(4G8p(1Tr=%Y$q^{qto7JrNJgbcB7#-^L8neLbRlAOA9MA!?jSpPv&{9>T_CkOl7hzcdT)-8Szy1kG-h*4e-F&>QaRMbpS zI;dgicX{KKKh*g6DzI@rbqp~Vb=a1dfvJ2Bx+n`{Tr+Oobgj6Tn+U+F;xv`4{~|!& zT1(ft6_f}FW*RAnr}{ih^w3~ngh7SQV*%{WfALCQjXj;#s)Zy?(lBUJ$x8HR9`Qw7 z5m9y8o@AJq0lF|-XSbaCsv*jNmJtpkc&_^LnsK|gBmDFg<-wk0ogWrNxxK@I`IrRu z?)cFgI{m#nvEcKtio)%`LTv6C=Xaxq=WqDhIp%+HB%DloY`X9#;s0A0*$tUPNAn6kzDe9MV#+!?pz#-iF` zbWm0@RruS9d#qZ--7(lfn;P8;(xB@_I;G zAQ*~WirTMl3fA4EJMkw)b!YL08!?7U9=9&CuXu`L)oNOIfs{}z%C_u6QH;^Hy}kz7f|d;Y$xp zLlU+846k%nDVT94fh1-FFkd1a@ry2t#d%UG?N}U~Kt}O|M#bo>Cn7m|B=NDCQfs}; zn#`ahb#ZI)B^atqCti>v4F&xn+J#NDNVd-cAsdz3iXT5lZ_iLC-qs+cWd4>W3ZLbV z9${Jh1z*E{$61Mc=?P0wZv<{NQP$} zr{0uIg?BzT_4w^OlOS)RN+>6bsgzgx!pTS=y1wh*0`rU`dY$64u9kZ3KN~?H)ntnh z`g6?{tXu4!OeBTXT;)=wdIY`?49zujZGoGG52c+;t1uzH9GbQQ$eI)8kgB~p@7Xsa zW8*s`rag7TCrDAMuJ0x<;x?l*pPl1ee(YFi?0Em@JpBqwl`oe6X3~ZJ2VW>-;n|nT zizwp|u#GXCNAkQ~jJy3m)#5Z{>4a$7toBw!G1-d&HTASa`0|z>ca_ zj*uz$SN0noqe77IZ&xTbv%F)D;xh6hvd-0`Iz@f?{PaSm5lm^^rekV`%Hbjpy6akg zFAB~-uS*>=>F(RJ45fJcf3*mXl&E77W)`~ck)41aq5PWSUr-HWbG6>5ZoCu7zArVW zpS>{I-}$yWsbA!(`>k&9LwPvZosG`04236b&3AN+6Sk+c_-ke>C%V#$CRysaw_;JH zTs>SK8n^ff(N;3tBlZN##|weT3cb3n0xi(D_#=FYRk49LmO>O`7EKNH#VA-)rpvveUqN3uPL?T)xOOR zW5=0dy+atBx;b!d9i~2aU@mzJQ)e*f;4x*4Nd(g`m*gbmm0Q_ktbR|aEV znyZ@vcpjeYW|Qv2^{@$?J;_YTBIS`qc8h(a0G6Yaf}%olw12Dp8{-&kh$~su&pa6m z0W;<9XI;1!L(QO;t2Ruz1AkDs!meL7F|DFV*ImW}kUWYRkg|}ZM{bHYQ9AEHE^YCpnEk!mF5TM) zCoS>&VA{9|n>J8OK~v;DMgDHJ8{JlShX^cqG;6u>t3q~0-w*=_>A%ktO*USi{s!OjDOItIt`=afu#fvJw2_=O;9tLpKRZ2CokH#D39jk_#8?eylM^ z^b{dAKDuKU*Ce4Ar0ov1&uqp=gmKMkb9!!^*2o6%Sp#8gD1<^%#!T_fK5v${;vF=8HrW9HPMe zfvoDM6{;wOy4rHlaQ}Y18VAYWP-oV-r^O!;&h&pj7m{;|xzW(}c|lIY$S(y4qQ`;E z5waZ~2FdJYWId?X5ad_CCw5>IZF($}+|r&|h!2lGaOQkBd;x%xQga#P2wMGbs=@VQ zah3ru-IEp}%#u)+7;Ia|>#Bd|);2u+hs`Y79@s%C%gpwKTu!Am;}(%BZH=m6@2+~Z z?QOQ#Dy=L=wUXh(0>R7WaUS`VgAt~L&dT&@;G@-;v`Fv8O>YCs&d}nFf-9_w_4>+e z&9J^sN72Mh`KGtm7?!0XV#Zm9dnp8%(*JCgcq`5JSK+bK^w!@X4Y|BJ@^c_FI~Uf8f#89Rt#sY@5R~HxeJu~fotnU>L+CKnlheL z>3n1QFEgAhW6+lIo5h8WiG>XvkO$6IL~cG!PAxYmKD-LP4q<9T`E1n-Zg`=afr0$) zt``dD$owoKaL6W4jW_+$D-(qCFzh9nc`V{W4#jds%15)+*9jU%8`A$eUcOiIb3C(| zTJN6#YkeKuW9NxNd|g&zMPctY__newa?Fw^CzyeGHKAl?zAK?*h5ea52e0_FsxsP4 zk-Xf$HQ$+->s9m06m#@Xk{(wl%|VD$+f5Ad&{?zlOU55r&qJX!$_&)!)(<@mM<-J{ zC(4T7rnI0mTPQL7Guj%R>TQH%sKI=0Zr0MPcUDUENPC+MKshb35@toiA30fVN91qTd;^z7{x&Xrkx3#TkFIWjU`*CEtl32O#OdQM1rz9+B?X4`akUrZ-Gpz*4 z5!7GQ-zEnm6(E@z{$_iwQ)C&K^o{F4nny1skae-uD}Mom$XqWv4lwG9C1BL0iMNf; zfHNWg4K1<)M_g1FX!D6Iu*(Y+h_`y*2$r-oZ#ZJH7gG5T#uwx>y~)Tds%921aS2bb zhgMW!N2b0DG^VL>DGNh}IMUGam^|%IDkp!0Z_MAb(O>dNSu#$Z04|2QW-62!-LXcq z)#$j3y@KL@P+0IZpS#`o-nc*H?WI>$#L9gRQtJYu7S8NUMWa}K5~k224*em)jtsj> zTK%ay5V5u#w}DJ_pYAX@ZZ1<-F!~I{!NN8knP7~1ib!FkZ2I}jL0Vxs^p~M7!1kPG zvSegId(^N*Y%8%AQU@-#Jy41h*wob11pZ1utI#i9XD;w<4>lB-dtJ$=E!(w6s?_Nj z`9{X7Y4~#N@TimxRSVq2G;$WAqWX5jtl2b44}#@u2${EQYQ)prQe4eoMO;-p7EKFl ztF>M!L+7*x&4bQ>0_&Tle;7DHD~M4k9d6y{Gp9ZG&X4OBJJkc^_o-+SPpNiPXR|NoJhv!fnlQ8X2RWFRmM%j;z)R` zQ2e4-h40Ot{x~M*16pFW(?ke)w_8mSs5Xa7$rffi<`HR=YsmNREdaq?(iyYsv>l7k z(}qgVtTYuicT?w>z4!Ai-#B8KGWSIlBoh#Bw%43k6ZjInm|P71@x+6gvRMu8Ot;B7 z9smqxb9}PY&@F2?%;mA~;mByTv~%>oSr6k~okWb7Glr1Oyo{thiR|-4hxAJu{Ll{c z$3APx!~PG5baZa2=7Dk37BbUyV@5tEclq6Y$A+G{+FW_xuX>wm2?cC>%h*TS?IbW|MsK1fJ_+fUbIKV*}-g{aG{L zY1@8H5c9*pA41!Uj+@>B{qGB@+qJ07o(c|#g>OJqS|FaH?FyYe_7SW?_r@2&wp+Rj!f8JOiUX04O=96Off>4kLdkg@ zKUY3BgxXWZLuZ4!Y{Z#Hcc0&hT+4hcB7h0sWfP^9BUJ9D3QH;X(WJ=&V;xlG z^WkJIqk`F3lzC+##8RynTqjIz#b zEc1a#JgIBqTQS&mfH4Ejgko^{TDdU(tLwh-yTtgT6aPdRMxQb$8jIoPSAW|D+WTt< z%e`qE^D7lqg&f5h5NcHM89yoleBi!dmn~z*ge|kGWmGoD^`-1fQRzl0@vPgak?T(2 zW~EP&wYWLh9DWt}&@)zqqn^C79w$SlQUTAWQu1d0eyKT;Tej&}B=Q+;c5!rCrwW8| zx)9Kq70Hy>SM;lzicd$Jv!AhqyLdISRrz*41hs7RA30a46UY|bzxfnnK~ZbB;P5aP zU58=PpX$c4s+&SZY2>8)O=dzB^53O4c#ze5arGCKaum1f`byB!$+U49E?q~{ z%b4V!B{|z^@PKE zU&0i+1&TuKps5l~eoO5gGFG-Ugml}+(E@02GkfOwL&Q)q4L~Tf-F3?DqMv?^(Ne$q>}ZKs1C_aoJcG&r!< ze}g8;@a!#0kSi8HhDgCK#l-~3`hKLZi!xP^Rf%ZcPy|PH$X77ln zqVryjAwGS}bav@lioX0~)#H>cp{P~2d$i*2h^rw!Y+>dHXYBt71H}GMI)QH@OaH%G zg8yrIHZ%SP+t@oh7Y^yie&}~HJ+gz+`w&2?u#08rJP9M4esn1%ii1`7Z3Le6RxCyEQ!V%iJvF!@%8VrrwA5HCUVT7j6 zxw9C$z=h~>OG#WG+W6|C-|)4-2Jh>ti9C302Y;zEKL);v6Eer}zCECXqLus<%sD+k zZXAQx_f*RmZb827FjO}5T?fbIoPD$irpVF%v?^xv*pcqtiK<~d`uT801_F}7Mv|hu zW&|OQQ=c9G1Rsh)Pt&6H~AB-kjbT6*Wb+2UIjsMA7*Sm3= z!+?V7hX@>QY)44%LJwV=;%PDNck)>x`k?Jl1Lx z5po3h>otg$ieq#tBYvN5ZwQ~yyeRNof+PS*eS2VrEE5mYh_hW{z1GJ}DO>4L-y8TQX+yVFFuBC&@Km7R~a3x1PIX9hWt`XE=}W%o*6 zXD|KCdv~ag-fCNPm(aL%WMZv+7zwj{*ah!{R^h%Sud9iYM{I+Q=j`VAj&GD~K_^-N z_ymn)>U>_ON@d{Z%bWV!o*HDVkXFhZSLkj}OpN0m=gsc?*Y&;8BHm#u;^KKB3bmJ( z$dhkR&Xj(BVJ19^M0;D*d?^<{F2ZFOvujg!Io`l$XNpDXYc+4?9tRx_3z0lvb)i9m zEP*|OfhsmmsEI{hN(kYw@HQ)-n6HDk}#Hm0urz?_E!a5Hj0&9A}KM0Y{*%jPlE zn<{~|WVq+LfmiRk->F*2u1<_NvE+3@uBy0P_J;9jlosW?@AplPO>{R`%JsGn>F%8` z?#_hWb{tMl7rRCw&gmmbiJ9DY$6`elXd%+UATA@cv$guZXj25Dy5QWz(c4ts+kVL!sun2u??He=vXn==Ls_P*XUj8z_`?riKob5R)1Y=6) zBIj$KW(X8&qomNvmr)QXFy{(HFe57&=<88GBajp11A!687H;*0&@oXA_M{Zkf^g zH24U@h?}5rWn+>koPKHbnC~upYfPtOsP>$!=EZ^ zVF8^70!`4&7M9~$6oxy*$Y4Z-f^#Ag2pgV6^F*2%I>O3R|Eo}fgXwgl z-NC_Dq=FKe^XNQJUBbb6j=(i&FMp#~)47HXGG1^+5JI`r^%eIo3U~&geq4E+MTuV; zws)!;4qg34Mb#mNq3@X))P?C+;l(A8A?EFkHkeEs75%dp?g2H!0ozrQb zLKjXO`yz{ovy@!ijA~#IFsdq|UMdX;0zTw_hI#jEIF`+=+sDek4Fjg6)k2)8loO6R zm|h5WP?xFq_WWTL%k(%Zq_<`oXL%I_!Y{`RN$6@p1|=t9r0h5xX|O6 z;H#>foWy2f39bs6Xy`1QhF$Vl6P58Op{<&B?w)h_YL@W>ITtUecxejDU!yYD9lr_S zXT3kb(sCna_6c9{M4RGE^4fju`CO6y6yHl9~)Q@Usj4ANb_ zFVDS3lu-PGBwiM4bnP}hu}!hQxPQk_f1g2GVjO>QM>dz$ltq`Lmzg#+H?Gs+8y?Yk z%4#hwov%mKM0%Cctp*^ZH$FmtGh$<=2^`o^9y_Yc5g7DU6N`-Jx5?Fx`M8T zC2PZAYd3-J#+{*N-+y=|Zts&kn_r!^+X8ZA+VyjWB#|8`)Fj@V1u6*^OX!vek!;%r zrhd!%ZmRBn)u)VF>5ZN`4`kuBlM+5h|fsnPY&xnWnpTNJ;!0RV6@BI zy~<}qjzL=}!}bjh%sA>LC#8V^P2=HUuJ(J|)m2Kt_(KWNMCmV_EUvP?NH(h}wMsF< z!l=m{H4D3+kWY%EFO#Ic`JZdm!p0PAz19yMYk4UE&)Bpv6Qr^AS>;dVo2>;OUG1Pc z(1*h7nki+#Ra&*1CPveM#MdcX{n=_NicL6Ddhx;TPK;nFQbyXKutf?BR(sYl&eNU4 z)|jOc*tI2}XtC|^PQtj{0 zyg7!*B7`_eB>Nq{CO1b7lqTgLRPsnfumj!v<=xD+Vk%QDg_eHf6SfhLR z7DVCOO4LeqP&`guLPYV~+fu4!)LI=h9Yc(UtG6rs56e9ihF;$S{e1f=^Loz~c{SV~ z{{2nt*>%@SSo^`J>2t(U48=P=!$5~7K^;w@TzF!g(@A97mtIOQFU1kH-yd>@;nY4* z%m%A5ibCk>@6&YZH;gxlU8FJAlxyg#ude^`dzy^`52JY?xutVW1#D6~h2luZCXxG- zFBa`L+K@gaYmxMdCes}*1)|sae{#?EYfHI*>Xo1dU$m4)ITJZsJGk|YvAS+;4PJ0= zDyV;Kd{nuP!bbLqmCl{PI8=(g>M`E`PEc$F%l&mf8gXiwcNb_Dx|?tIFgaI~?CyF6 z#{xI3(u=f%4NQF*vyVyRnSKY_MbMys7rggRRE4*e(d^tTQTVd#BLGE~ge__on=iuK zvx>Koy1a-~`yfLck@VzyW@BP3&xk0td2I~Rr&tCp`3a{0^gb@k_!M@1L^eLbv>f`f+pisNtM_7LX|q zeY7!b2by%9sf7c*Fv%mAm^Fl1l4PB;9`@Sx#sKhMHJR80(3Ad_+SUnA;bKwv+_lg> z${k?P1tw_EkeM(P1zA~*lYO+%uTxGtjw(a9?Dbu4=q*_LJtCXulHO|YM#iYsqo1>7 z!QLPzXw+gRq#zf+@@e*}PI%mBx(CD+e*s$o*>yvbj(5^kdNLwmc+lv`_H6s3l73%O zCkbx_^3V%ir8{mEq;XyJ0rN00a`$G6qRchIejU6Utc+K#ah400kI3Tg#C?VGu;6f_ zBrG?hFu!A$h{2i4C#$AebtF}JP8J*e*OLn6b2;<#pw#6P02Q+tCOH-g&+ z^X|LG3Q>-y=czQCOvd!dXB(R1OdJXP>azQdZ8>(?2-x*-O0Jy!g}TokR5YutwDQVx z@XMOKRE!ua%35q1+bd1Lm%M#dl%RIV#{eHiS<&S1%{JW_<)SGB@y0Qe;X zw@QZc%1o}AY&wnbKZ1GG{}IanuizG?m-0U`eE^y{wf*IzK$$BjhY$2Ux~;V$fu^10ioOOU%;=g&hSrZO;Mf3ye&K7pVqt zh>z%-UsqGtG|a3nP*3MNoX$4_^P>1|0il@MSDUqAlRPmj#V^kf8%B&MufU+QY0XM1 zjr9T4MN7TcHK3`4-MJ=;l|(~{p-W}cV*8L>jG){L|M{o*$8AA^JgIE1_-*S>=Si47 z{s$&5Gr{7ZQ;R8t8>d_UfDc0CbF&J_h$951b8@SNkC%7G`g*UL4Q>RLAagCn%tgn+ z83ZUmGG21Mkq^0c*2uPv*wmGYgo|nRs-E@ip9X5M*9f?{+?_1F^xb)-oKP{Mg~rpq zB{O@yl0e7c48?!kB!oHNJ-;znxVIE@!Nxx3%IS}5dMl1J!Iu{j=|0T?X%7Ua-}xE82UCd=W{S1( zRyC{N2YGnHx*S{~kN`O`oW2_M8G;A#4iA_qr+|G7_Ex@4+ZWe;3#xrV4-^ z0P0}lsk`+Q$qZg4s8>m>Jnbxp{Ise*BusAJnuxNT{ey?JQhPI&Te&vAn-t1=b$X^5 z&D-~djX_jI4vq8fg{?tW?Nlv!>DM`+B)xb|2wj9!I&-z1xJBI$dY(G0Xfz$J@@2NQ zTg@9;B7W4g@gges+OE%BbuL85LsrG^&@Qq0q%S)I&rl8OL%=YSk1yeT{eyhV0g}HJ zX{#x{rNzyWMU;19w@#~R<8L)`&j*iV&3%`Q-i`3x_AIY_%z9wcVMt`r_ zI(IoHJ>a+!aF^QRtI+E(4a?3hN(RfqV{1`#(8qp!P>0y2M$LqxEb z$8hA)|DOWp@~-y+)UqNtat=2)y91Ndg*9oO^+O>y&VtbSm+{3{Y9(3<<+`9cLsya~ zEv~|fYaXuHNmHq*G#$p3_z=^&o85Q7{?uP^TP>O~ZZ?r#{~&f%11K%FpDIOzEsu|k zi%O+jRK8+vI#ZiDS~u3j#Aun%lNml%pu6hIobs*`p~ow2*_eh4Let_KC7SfKApHi0 z!hq*INT-5j;s!StiEnsXQVUgS4QaQR4N3n@XU}OU`AFD@_6&@k8Op?T&Civ!p6wR&T5*@9^NXTL+WM%2~fOQ z$DKZYyEg|auqMlGmy?1 zS*yYC4|$_n=IMf*Qb#ac$-^#mOdIIHzRVq`Zp>^I-r9i0==>C9m2bu-kVjFkT>^Q% zR)AVavv7o|%$-C}w|fzL%Y9BVwdVz;Ha^QLZ; z^Hp1a741{YfnF7w-9&d0zJ0J5CWKSi57-jLAx{0f!I=j!5Drb6)Hrd*7rZWc-*ZHa zX)mF?hR{QKv3l$(G!QcX9{G)@RO!h2Nf{Q3!LPkUncc3@L|JR4PzvBF^F!(Bv#fcEi#ay#Za*P-(MlwbM){;~UxIO=-O zzb@{&maSkT_molcA($<%Q;KvEaH*k{@b&smrqrKll!+tJCT2N$>ARIDMV*b&Dj3kN z^JDN2L`%i`4TDymgWZAU&&dDwsI=0)t+Rp78gP3@aciT5p0Xl;#BV@v<*N5YZ|H4x z8)f>Zh&%#}BifbxhPzY)QAZkZz@x=lrN9S`U=id)%irdgfdhfg42=POew(%wj8xFJ zRrrS)E`^56o?Zw{EYk$HEHxkGB>vE=pncI9mBpW-B#NihHqSIiV<2ql&itd*IawL{ zH@1xBgx0|B)o*jfZlo#BJt>yXzN4L$m7q_t`6oV7+X=t#FFvbDAmEvJS1{`5ZGexN zJKcZ>99TOGvu#t*?SFlHvp&>|3fN_ZPRN{OcOIVT z8*5@iwKI-bIFicHdJ?pf6MLE1ec(fCxKRI+_RVO=bbNfARma%EOhVb4-oZhsf28=L zU1J_0sinWZnfpJ_c7fwNm0{uD4XzPGKp#_T478nq#}tpyCd)yjxVELA8t)5_>n%4b`H%dRKfXPDVj($i)nT7TU z-Ni;nZmVEO=i$JI}U%v@D{1>2Vr#MaPOYl92-rA}H;bsr5{!G=ad zBOr(-nR9LtZk316nQ7GuY>e~q4`N|GQimt@D(!W$uec`G^0&zBv{m}WP~Qe4 zET80No>k*uPS$9G#0KLoStFYHX)Eq7lGz3qiZ~lr$SI~vK1}`{lj%gxH|Aqt!|!zs z;6_v3R!<}7Ru!MkXwEeyO1(kj=VEZGUFxGZBIE6l-vRYs#_iy~E)Y@4E+dT-tb8vQ z^BZ)2Pk7OPI~h#0+YX<<2kYFnbyF~2Q-a9B9dN!XSyA-IY-RxX>mvBNL>}-1{ahh0 zC;p=S*`+v6MCQQ6UIB2p{rfz{8*Yu;oJQV`e8YU~FW@Mch-s8PB7Nn%z95)}W z7umWUe*&xBhGcZmMre`WPwtcEs5-F9LWizExGRQ}*_kABkB4Z__T1XZ#Fs0S)IP}U zC&>ZryQoyzZ?Wb2-;$@j%~vr4Mhf4pz_xN6{%e%rA*O*QUqLzlt>uZswuUG2b_#DW;N+l=_`Ie(%)~|kTGPm5`(u{ z)CwezizTF(yCqQ_ur!oEbPzoYa`+eRgv8@4f98qM(!*&P()2}YH@V<{i~p{*^gD<5 z-ssL05t%VG7us4tdq-;=2|pYC2D3=D5FVbL0$-)X9};=8+H~h===t!dJYPA3bO~SP zk`<{)*J3o0MpQ`_mP2NI9Gss^qZ*0lfzjUT{3mIMDOZ|aI%{R+@z&-|<$uRUKHSsT zX?k8aYJU)Yu9p?m^^~8~3^Kxk`!iRXV3H_9A+z<=C)qC2R)y<%vbV`w6Wps(No$; z>hASxOQv`Bg4yzC#jn^SYJ%8Nv#@xTKF7%0h1u$uImW5uss57~x#d(!%av!$T)7eYpscpy>evuPMqS@%pS}EH zo(M%=1@>Qovy~+s%|FdmeMi;VvDMWsFQS&>RRT45%t_j2d6MBoweb+oqgSh4YR=F5 z@@RgKTs2U(h`QE2&sfR9JLeZ^tFt$Bw}lCni@qDok%zDFRM%MS@EFx`6noJ?dDv+| zDg~WujxpY>eXHmSGh_@;EJ@*Z-X(fc&AAM5k$5{F!K_$b$<$)d8IgocZ=@X>T?TSf z&^h3#W++th_Xoc)`x*=qnl8%;Ji4^wPaGfYgz1yOLo;o)0FfrGHG0A?18-Ma)9*iB zSYrPynb64xZ_p;{uvQN|cGe%u(f@Bi#^;z$h0I0+C&f`y(}gVTDJvcQPy@RBbfiB- zXZ!XHBazmPpBBg9v7K~41RHt4iF!W{p`z+J@;jXt8p3>sNGuh2qJ7qdmUxeD9PN4V5sFI``1a3b21quw zfhDL!u0pk@gRpB7&4y`cOI*%VoS1=A!s1MB?O0{gF6HX z?hxGF-60U%-Q5No++BhV?k>aN?hL+sTl-h-R;@hjYga#X)vdmL&*^i|~~zHw!M4;e*&epn^MS$#{v!X5LFv{&WkawaIy-7 zs_viG=hBG4gvUCLN^Jr+EJc;e@vf9k2C(fPglbvSc7zkkG4PL^8)Po@Mg z#ud7tgE3AN=L9UNTWppB${8xX`va!MvbqB<>4D__RUFqxl zG&CYHMRohX*?A!XGil7P%wb=(SUm3a!G4bZy{VV*erEUG;wq!B=__VWi*-kM3)!aq zHM$t42g?fJwOE6nRt8p2k##GtTNNH&h#pqz7E0?+vgG9oTER^&-zPHatg|}9nSrlX zXMBD6t*sp16Mx|8a=-t^{GuSF_%Gm_tZ;t-0^F@`yg6z(LYuy@wYhbTD73uBtsicn z5>qTJ06`4%vvul6oa)hcPMxHfKi&%~cedNz`slpG@;i+S939tXcgzdaEx4b=M9--B)wM4j6tFQW#L6D zPRcArKmQd8@qR^2?RAffpZHD?B2YRm1N7-4E@Jedz2R@UCDKw=?@#&GQoHSB&IzBcKT^Np3k(R6}= zQLBQwI^vx60lN^chE1K|^ReZ0bNb~cw3#O?_E&z_$47Ufj`$YoKCTQH-nEtC*{MW# zp&F$vE#))Tz|S2%Z;D?nJ0t1 zE>U?F?DoJA*K=Rwq~W^%16czrxMAh|ep}KRq~7R!!nCy{U`2K`P8nM|o0qNQH!$uy zvrqx^8AW@>5Tq3NCIO0R@sSTkMCm60eDzCADj2NUmj8jmGy?u6jrgf98ui>pUPtHal7`f_ z#{<_ju@Lowprj}z!d!Y|uE|Jpz4a271U)|qb&dga(cnj@$IYvwa&3&9*=C_^W{XyK zF$w>{&I%6w30pq;2Y}})ayGn*lS%i`=j$``7hsOwEnC>^qb2H58qO>HfnV<03u9p$ zZz{7P6PQDJ=?QAUf#j8ongg!Cag!c36&=+Wj9SIw=t>0Vs*+`sOZeXFP!MTFHjuI+wM(lD1Tg5lo+?SB7NjR6#Xd{qXFIz-v0A%qHXI^46!HIU%ik7~vK*m8NWzHpXOG&pA`JLDnP{hpl_K zOKIT=l68f*Jg8D!NiahErI~rhUu0%4c3`_F-_5VbpKcchkzX7s(_EF8y!8b$;4>0B zt-m;Z&q1DG3_KwxT4!>|l<^L7MX=iiW%ni47^SuAGn5ayb{EOQqi&REm%r@bIUzas z3=fChJ_m$0GkqP^bYpFfBWB>S`>|RG%evNHwO)36&HjSs?qVd6Mc@HhXhXoBXcq#- z4QE5p>j_ArYeSLEZ;MW3@D~v{sd+P(AD9m_N-&Dm-$GcA*YPI^1-I57B9aVR8NOR* zSF|d|_S!bh`zg7KJz6by%EjrDpR|`U8r}RnplCc$YapYy%)4Z;w;inB^{rQFy*aJ| zcJcw%4Bg%iT!>YO7Q8?Iu&FsKR;#edWl4!*_JP^;(pk+mRho1mnJcXf))1;g+^iYj9D_mpbeV8p42!>}%?p<1`n+R*W6*o=Dqq>&0Q#{%gpv&%xGiU{(U5 zPDqI3OPDJy8=fxjhH_ziLe*&ktz*+s7Po{IRhng>mT? zBJ>l%Dr8+?55hCgU$mSpXnbfJ5|P=Q1NdAU7vOq3zn~QNP3N%q_o%Bvcm+u zwJF6_{q>U-woTbTWW9v8Kv`7%r6l!V#+W2r(I&uC!$absAAU%o z7}U-%x=Q%Nb}qF@t)T;6%OCt9GX)m_ju+~1j3ixEz_Hn!~c^qnk0Q=#QPwg5-Z42{A&Tt#FWRov@ zGwpAUw_|EhpDAV+3K17#5cyipGKV-d-~{ zBhR+nj4$K0WaOVLr)U*ujJK0z4ev6c=B@QbF8n&bbS1iHeOAF0VMo4*Sy1 zxS%Icsycp6Y_G0w0jB6h!s7^P_wojVf2VDF3uc;00cOrLGQ3j8u^zANs!`>*8F0#7 zJQMj#>J|bg(gYUC+9=f(2}q3=kSEwY&rLxBWVQApH;NBouj1@ zMNi&RR%D+A9RY*PTil~eYgvD0HZHoelPj$YKi|@WBY17TwhV|o&5!cnqALncilBgM zI8r`>Z!A{w7_H{TFO-_ND|Prv0wypRxBS|}@Q*Lp2Q#(b zJwWAHeoxZq*1X*}?+dCK!4Pv}tPYw#H3%3Dbbzo3b<*5C^GNv|&k{-M%aDY2rMbKf z7?Xt=l|-3Oj+0K;-ePhNvbnh|=q>#osbq;0A2au+uV~5Fu&`wRLq$GngJgOainopL zjixW-8g;83jQjzW_Urm)3A|k2Td-zSf&O)^NPT%=ud&wu>xDwfq4JaBj?EBY0^L}*AC06at)641p`rR4I zwW3&I_1k|$OIn~<1R{lWr}|?fsXNwTzC?0w^i+!tk0^BAU##Dcg){5qug>CK#x+fz z+SNtiZJEx~gXXqs*^2ke)T1M~R1!Ocu41Vp7(2^!5-il;H`H+4`Ve0M#viJA(dS*B zXREr3lr6u5)(sIlu{K|>sTopyMa!YTlI9!3&|kF|;J zYSnVK4itCxA;gE^QUP?*%(##(h$Mv5FL`&{Fb<#FbHKV1)&*JYVZ@vTT z4+*Y1b|L}r_MGu?54OiPy6y6w*aH&c$BR1lw{&E zYAqsinWC1viz_uNhOKnN3FoKHh;O~lY+-^Y6yLExGxfmxTP;nGEd_O4JbhkbFKn6R z7S^Qb%K4 z3v+XH+<%2RB(>nvsx`&VR8h{^XqI)W5*AqQB$R{CTxeZPkXtlyk?hPwsEH|DGufj~ ziK)LA;s4cC`2$btnO{ETFi#gemxL?FHD00h?_5CR$Hvl`8&q5t%M7~xTZa9)5IF9+ zC=K0cthFJ;s0b)HWyPa0%6f3(vwx8G_b z2t5Kr=BXZBwPYNX5UbXmtkc0f-i@BD*>>DsL2~SE2gup=Rx2diYDcNTD#M-CDu&u$L*rL#D~&Fx z+4QoMN=d2g_ylk0P_@}Ue3kFS(}- zGny_IV|OBtEp<-=#zaM-p;?bkDwhc*4EW5Ge3(zfcL+55lf4ZB$n?9FHpI#u#b|FON-Ez)N8Ld6SQKrxBCD$#okd$ z;@aYPfeu&Qlkd(%2!TwW=HowEs|~}^`*07blq4#}S#d=G122Br%67HQZ2xn8aSV35EmC0^)0^r41&|)XSg0KH~Q8y6POX?8m7aM zDg6BWn^ZkmGip?*cxp95D2c~I@-RE0-ruK1*uyZi()z2o(n=J$` z2u-mHq0$Aj%*iLMk+vCP*6C)yKLK;a>ZbIzyuXL;_eH=azVV9Bfr5|Txq;>Rz%e}l zW`gC&6v4`!kP=)jxwRfFEppVk*lhby)sRaX?3$=;DV~zLnR|<~39efU*>b7JMZY7j z;_>*Y#%UAutl=NN#d#V?)GLk^Z6qV{?{&iJ>+(E`Xu3szj;Tkk>L=>py$Mmbz4%??pf*7Afx@*p-ZT8~{~s4P4=OC%bwJ@Ac%3SactduKNv1}RlNAF(^Tank&`cA$@eovfkS*p$w0WQ+ax!SG4)E2piYz~q zGs0S$6GPRMZ}YT;v#3|dfWxc`uM!%i7PYDP8;7*>)hsRg*7 z!&o#eiRsj*#9eHt^Lnc8U2dG)QAr(WjPcb+A8U%gu|C;*Fe~i=n#y|O}7)+AoRZG?~Bs)$@9}f9= z_dw+RzI^4oH@W`n*oO1gkZEZz@sQ~5;a_rFd1p($W2s~XE4f<74VF{p$Bt?-H~C@1 z(zfEHTbnP++RqI{uEfE0n<2D|DYU)!#@mCF;B6!8-?t1KzK;weDWKh1pBss#o$)c= z?H!fD_S?ah;JXu7LgHf}qrte@7>$z;uhY*?s3yKwaHY189>C)z;yt|x{-&*nC(IDX zrZqC(0Yl_drMy}V8lY)(6M%%a)ihypn@nST(~;J0i>^x=8e7628k;+xnkc;k{HhhI zd~HynKIT2;o8-5eTSR_Q;+VpZ0%SFL5Q%;=47jVOl(AK7{a)V`odeCL%l`V8XR_1h zvDYu>W-alRL)Y|&L|=65Ip3@AAs*g#m#^OWr_O8~8fQ8qV};8R+IR;mo&|70xcuIC zJ>Li7mV>~1FWlQn=A!(^V!tAqVpp4>yUnxV zw?qpMdn z_(?L9bZw=-915U(fa~!~s(xcr2Rkf$5w;*&q|XJ6QdCX3HaRcX1c$oT4*hP5Sh&+^ z96D~^)yuaz!rBTzPSwf`9+p${bv^QWgs;UXgf51@0S|Wdo$0eM6}}YR!oyL#va052 zKQR48A@Z9Ry0Kc)*va%v-Z)Pv5A>bv>6(}F`1O^kjLJqls=F#jx-^g>#@P4i;B#45 zp4~$fk@>-XsENta>-Ha|^)~6IzY{gb`eLhigtzQ-+tx7n=vf*6=YIVK`p)v}MynSH)mSm#)r|W ze1)w$pN>Jz-?C;##HWB+09V;8^9?7|Bl0UKD1X{jqv@8#NI#|b>V~)}Y(xX=5NHrt zeKIkwH{0G0$YJE**T~2h2TMZDLj^leGEpYFjs~>iBusy7M4$JL4I{hR3F3%tKwW6- z%ZlZ{p`Oj!*R#h8XRSHd9tO5aFo}g>6IvQg?8o6zS(qB= z>4{IRvVxLfW4wuDVMr(|qqYDqiatDQKnao8zjymt}TQy%ac@GRUrvld!D1b35EGgBB%RI+NO&psD;5Ej?+4 z*>V2kDw>|XL9$?i+n{2P2VV~W$>@HY86{-w8>^Pc_Y9*RsD|IEfXR{1z#jime5)xg za(a#@VIl<+@!yT!kEbfmI}tU$^`Pg(k~Jtk{;DOm9~rdlulJcPvESOZh*U`~RbWVq$3ekE9`BuJ z;~*v){!5Q%mxs4TB{#M?CRH~UEUd8u4`}94!{B6T9HhIoZwHsEB<9P|{b|fKk3o<8 zl|T;NalCec@v?I-8nBDOe&H>f(%-%-yyf6bvg27m=8E?A+eH9xb8j|LygseT6l zdeAvH!aF%4GLK9bgVI|Sm>!`=T!_A#1PMW-pMMo)Q$5P&M+k?7WhK0vE&eIjTJwVJR@Mdje z=C`XkN{MbAzB~9=U4w|_>mzG_6H_>zx*@4?mEfst;ONgf{5lAm3E?fda{%s~o&+YF zJ%gJROc&orKA`g;_(BV)tip<981dvSX=PouC-ucs(+^f`@f-%fc_J`U=3e7yM~qIy z*Inu)i0!vSqs2tLbG_tYtvTDeZ3rSrSD8z`XssB?2#XD-QsZI8(%ta^+rM3(fG!S0 z<%IDxewOW$5u>SE@+=;gd}W}Vv-JihJo0H4SFg)ERYVJ9jFMB9%mbA9nyc&gru8MF z>s2XSZqm;^xBFTqC38cl#&ouNl0W#|md8YEjrFZf!sO6UM%heC-Y`>qRu#{Z*)G6( zQwXl{^`#-W%^|Ghk0uGfLnCVuqP)BIza(i#N;)AOd+%(MJZ76eZ=f$3kyV9ea&AV1 zwIn#MII{hk0!w#0=XEFPw-Q@;nB&>0e)V5s+bj{T1b9o5i}M$&?YFv8^_|Pj!z@FN zU;DpExUW7XCcvF{?>^xEXGTEMzkXg1b19tJ@@H|M%WSGRmGk^)J_rGAdai-7uw=uC z^o-dKRHbrlzT2Q%6sK}o40f^OTuMhxe)%QE_c2l_tTq*a6Mx6PgbL~?b(N0CbT#Y(1vm|3=vhfS9P2F~{Mf}8&r{WDw7=k`z<$`X zu#QH{B)_M!1=wCjD>3^1)8(M=glg__`tVUQb6GH*L9wBv>H-ZWX{4d#07$&ka!L&Oar!zD#sfQ4XAQn^KNk!fP`%bSVRK+k9;FO}Vm0Xl&bK`%4Ar zgMM5SO0Sr@u+#)sV{&R;$=}WgVphcXU)r$~%2Ni4b1@xx((R3)0;eNiW{>u7dv4g@ zrMV9K$gS|i2_KQ@FI}K`w&H!~kJ^$dCIESI%#yu=hRCvT<20}6=R?qFw=zx6Ys-f+ zeM(UQ3&!a+nW<1W?t;=mO_a}|w=$&*De7PbD9>~fa(4R3#AB2R&+TJ(msmrS=^>9w za)0#^kRscXYqH!8i3l|(7SAcr$KW#&zZbD;6b@=O;Oj>E@=z3^U$suLO>}t(i@Y)u zw$t@DL0ws(l8^k;u6)mkHsNGpnGwpUUa1T@Y3NoNm`=U7d$t1 zkU3{FrP}VUO7rKR*2iMwdg@S2%-&SbQX*bb>1kQ=)rW?GYlv7rmR)?}H6Ap1ZGZnH z(qq@gupq|i941zaMvRBan(+R{B~}p~fCSLh?h;)s;;JlGE1W4j{S{K+u24yq?UP{y zlcP|^*IH;b22wZz4iHP_cT^fKj781gf#|K^((!njXGG(A#ujHpMKgD!TM?t515(-N z<37Cui}tE#ed{RWG(y^~d0aWs9-@&in1>=~ZvJ_BGz0@GPnU;-A}@ctodiV)-(x*m z&;28*qox(jS(9JvAs5vfFP2JpiOiS@LEJGuf1>kh62Gbt0knIu->-4MuP@t&l-&e0 zP@l2vT~`sM9I!ToaIZQ4JGFTeIh+YoNyfttD*iJgPDG7p# z6|rguk=iB1oqm7&oO7RZ-;Z<7A9>`FPd=ab=X&q!eZ8Jn`~y=17A9UM8X6iF!~1&X zG&Hnw)KBI)2I`g3(KBq+uQLJW2D&sA16K*uAN0?306H`@RVmEJPtH<*pZC3Q8$d(D z+WGf$rpM#TDeBamXn68C3-A8;~bq=l*hSpTQcu+_$udeXu zoQLNH1`JI&kAk|$u67C`f>LZ6WV07|u)7<05QsT$Z!9k<-7H;OQ(ZY;&#?%4>UWoF zq_A5InVW`2$p0MGI&IJ;6xBwQaOxQvnzVaDbkzS7w5j#gyBvFghUSeFEe{RNMO%7m zI|x<(|9GIP@fT$PPtb^1GQmd?{>(YY=W8x3)6&FHM3meLE?fynzVKNIdur*g8a#?Gf?`D(=RX4SX2>EZd+ zheA7qjy9>PYjG~9Ae&~Dzcmwc3bmRSGeg_vcT8e#CbyDNIo{J^TOY;FJ*8Gvj91Vt zr(jF>_7XoTCoH1lLbLBX1T78CIqV&sg_loz)l#X3ez|bRr}n>Ijn7wc)|FP>r&@o( zBiVQ57+b8-qq7rk`*PYVXZH`*%1G~$NKGzn zG1PXB0lwGi$M~PV^jKQTatf}j6K-I94m(ilb!_7jcl+ue@ey&~J-XOxv3^do68h44 z`a4jY+DY1#6Jq6q2_v(iFA>%@sCT3h!-X2H5NzARN_0To`KNY8lzLTVF|d<;`iPa8 zuSHwRKkvG&<6e`c4%&SrFxC|z&QPoM3MbdCszV^Sk~Y2&XZ;yT7Jam zMav}BP^U)OmT?dB9^uUztuCAdh}9aeBZL*xC8q*O*QsNsz{V2wo$J7Gdcm0ETaTj_ z>%M-I?j|MP)|=8o5(Dl4$SECsZg0cwHe1>sj{V8U)fquGfwCUc)o|rFj4tqj=3V zqPKgx-=wT2&OsgZ4;VaC=V{^Hc=-|Ni_I%`*UQIS4lB@N7PDMk00ZBk8j=ybxkV2#58gl7o`#;w=y}ier){D zU})6c9Aj;;loU?ge56qld4B5 zR$7Du=pZ16o?q4->6_e}+=l~LwCD#1m{;p@cB5DJI5Elm^btwZK=iWm)az zOB}GnFBRJ^cV#gBtuaLVM;6o&G7nQw?mO|%SI=#V~ z@hbDgI?g6-woV2UMNxo9(#4qX!@BL5LV^=a!BNX4 zreDCO`9=5Y_!%F8@Z;+YNN`)`2(afv5A+s$KycNI3{mH3~; zOIa(#%iu zqj;$p`q3_L|3=z5P;F_%IHMoz)^lT}f(!)=(NoqGS-gG&Kvz@*HtoZ~7(b?n~S(V2|ozva0%P1@*v1RGvAkeP~^u~;o@ z&OrYk7mSg9lMBoE=^gx# z#=p+bR_^+ilT13?7td;2?VGFqq!@s#ewy3w?rSw7N*BKR%iVUqv?9yIJ@qv^%s$<7 zQmv%vqA)jJXR>rl%GlWFop3$B?FGgZkhTG5RyA<7P?&CxX|&a>Y-oHh!y3IfiFNo($CMjaCvx{t#u+=URqM zBo=!s-CPOvICTT4RR5mD&}@4{TRv(76wzEU4(jD%ll3K&>P}?0@#C?S7Yfs2UGuNV zE%wRfv%+^*#?dipxaelxJAt~3gCTioVH|-Qel*h`ZQMMH=6`N#t*$4c`SWJ@vR+s6 zi+3_K!Or>~*ES77wi2h*Z;R3yj%h0|`>FZ4-6x_vHcWhAe=I1F|1IHRe)?>lm(bcMO95)r{dfqEwC zEEfbaVMrrfW5(=8o%lIK1&hn9P*>HBscrQJa)8iIU&{P8#ZMLxy_|C;_CmF)*YK{r zL92p>`xjL0?InBQUY+)3%K@!om4*Id4BmRDJb9U|}m_==7F zFmjY8tP>QzTPDGFq_#?W-;qGk)DCe|DOieyGTUhu{1Fe2SZ(6~B0lYh&}`rKZbVHy z5hfu5&bOIGGoEPdhMd{{br6l+KJ&vs2z$53w5RBu5QB!LpS4MHo79QWEMjtS(^XLX(I1B_HM_YTQZq3Kf`5{Fs}YT z4~p7}h{N*n3X8?OV$!>tua$Z`6Zi^$NE;0k%ilE{DcjOS2q128XcLO zI8&uCvN5t!T1}kr#`*4;X={(Y7P$ z!)LalrZOD;BFIrqzf|N_1rU$y08c%j26_8Hwy8$PO~j>0k3aGbWc1m~g;T z%?USl`!>UDSG5MGwHb?Za9fcBC#M4`_jlb^FaNOKRQPz#b0sZEvM~6>9CxvI$^771 zdA}m5X`yP`yD7c8g3$yy+W5yCthxv-ev(q;8VYY>h*Boil)0ot3toLrAp)d?iAS|6TY#@ZNo__oQ=F=iCn$?Uk5#1;@>Eptl>p zDb@jsx9pK;E3W8c>_r0*4Skye&%*Ocw_?j`V58<=nM)IqkvNe4*<)57E18V533~PI z+3()JG)cGjTy7Ngc%|e%BOzNuc~~E$;@T48t~1ZN{E4Tg$l7ixrh{@fVzFQZ)o6az zGlQFqA4l)<#0EYa#ERnTYNj!(Ji~#Ph6BI;lJ;1f>mT{5M`y{=RXf@@1@pcv9IK5) z=_gfQB4ns!>+y<bjtO9{p9!pat zm<>M9+bKw3rpeLn`5D4DC3#0UErUyUu1ASZUj`mw0gv3zR#+>J|El8jMIrYlChR_z zC;KE~9*3)5v`$wpZgh94Q6GNbQ=cGyE%j#oWJ{~$s~=IXQZ-{6K6|E@c?J<&OY(8z zY)I>p;18p@M?jmPqjH8YkV_k^a&)Lf=bdY6Y^iTRQGjpQ9cPGdTUomjzS7;;X^FRX z#en7g^ON7u7EES8Q`pyv&a%rsOjo;^7AvjMrxQHwWaSPf zD<+G2j5BRX-%fCqZ`!_A1BNrs6L2PB$f=fU!{r(H=e&AFulg7?rR9UH9JjD&dEMt6 zQWSZBn_(LM!yfyL(Zz&cbP`yGrPD= z$_Pvh@F(!;nINN#eOzwU$D@yfmHrrCe)M_#h}zx_jS zZY-(MK`ywjVjPtmd7pCx{{S5=&)CceVV~9a#zHmxhk!TSM{8j-)qr}}lIQ!X+e?qI zKc9}d$gPxqPQuTOey}t%q!(&3wxBvDE3*`BgoEldz?rjs>;|7K1~aP3|30w1fju3G ztyaXw!o?%T-vQ@}NYO8--to;+CRng@Zj-x_p}2Iou}iNb8*Dqq&vJfERsiFJm- z8cmK}XK6Fj?*(h%CtcJIGw=Z=ZE~{)@)Lr?p%K?guHXI&|ZO~=7FIS*lg_mYvUG`)aGDIlDn9~gnAfVuFZSKUNJE{vXiKm z+p8$L!1NAjfpstv!Bp<0ZyjlsBdpE2 zH}Vd!UE%xJC_Kd>uPs*WQBUeV z2lKN!>NiwtMrOesW#XR&PYk4UJ{G63&tA5A^A1@f3@~7gy})Er`R>E90|QAycEw~y z^Osfr^skYDw5HgtC%g6IlS>CTHcP+Qyip0hOyBzwcH(`TUwUzS(4^`fjcEORzxQSr0a}`dw_<({GR9qz#hPe6atU>6dj_Ou1bWKBh}B&yyJtW^ zhK%OQWi?R&=KSim5s60w5FD69-B`cJ zHJmNGR>Rf{&Erdz?$q|1x!y*1UZ(D?!gYQ+37(jVDr5hraus?8jS6f<)hYCB>fa_b z@AtRKx;UM@_B%9Ft8@PL$85c}^N>-|a3RG(zjF5HAj+b_#$I+$K0VG^G|2BN$9I@h zvt-l7kWBRum*3e42*ex|pWnnpB&22C&Rt4Jwy6u>8A#)A=u5T-WvBx@=H@ga?$NzA zjWzu_Tq}}q?m13qzYgYA`MdgfF6=BS3E4fWkDYgL%^Zw*BeSzTe%Mvx9h{|5vs>vq z>~I6Orqm`m-F5r7e3}%Vxi9K!`tid2Sr1qRP`bRk!R(ZH+&HSmxu)B_8xLjV@~Qm@mQDGzhcIQA zt%QCI84ld8Un&FcK7X#KZBeU0T)BQE$1Op5Wb!YDpjyA--~DW^UkJD1b7bj-n(oKW zx6bUvmO6}olW~mje-`EXg#9>u*$9b$ekKWzU zj{-~syT7T_@Ec%$`}c2TY^Joi8)Iw7z-lv981Q4AjLJlURK5an=@RgiD+P1iI~bxD zt_XNd$BXbzNJGZ0e`$UEZOk(bnHG1&S$fYz##tKNoe?Qf2)uCvZ?9N|Q+sP!AYYLy zI}`Yu)FpQ+e{;Pie#jaS!9Mz-0WX^E2SFGcSb(Wg`ydF0#RONM7tdha1kpgFjpZC_5cO3_=cy|p4_J&+2PIp6s zyY03!o_@YT3ln%cbf1$p*5vTU z|DL8|@`n81Pr37vC8!VcDvF4bQ0{)iuE}gkd@|eU zo0X3ID1*M&em^%hTiI)EK6?DiQI0v4O`5INVd!5EXYgN+{@x~(J#B}{crKEDwWp{GU_8m8m-O!MzCdW07Bqkc}-B0^>N zt%UAYKXTn&AuKLl9;m(Zt=LZmOnJ5P=NB8Xqfb#PWWzbBd}JB zQO-#1q@5g(dQA;=9=&?4FpayIj_k=+C_j^kB>C$LEnGwi*_@v2BZwWRTUEuxjyK5^ z0hEx&aA57$Rccf@Mo#Ddm_+q&VTuRr&K@$s(P!SM4$Jo%iqOKY(cLo*c@x+*O$}`N z^%klXfxo04?;Wl)KH#-Wb`P^#RhUKyUqU7FN|}d2QDm{d0Z>0wVH5K}GWu&a;-OHp zrekG=pf%F%czB_{w;JeOK%x3 zm3<{WEzY@`Oefd=i6?00&H?v_`9wq4N_ef(lTZ^)r$TLg>LcUC)%}aoh3l(7w=Mb} zN}G}2J(}{34(Y4fvj?pntMc6Pe}S-wG%!n0KMH?c3t`XNc^#i4FCq$AUs*-eA7^Q0 zT)VE1_BkQUHmZo*Ou8&Br2DD(nk;SyIB(Y_Fmkrq-n>tD1%U1NBQCS>DgxZ><0!cg zI6azezb2L<01Z1mBu!tor}Xu`p06GU z)j!+DLo$ZSqE7@tL%$#TALf4!4tP+(@Z+i6p=X+H=Is)G$L78lP_~8Y&O$#VX zjvct1vvx6TFRpPzOVnSqL+bpE-Hwg}7L_gZlakZQXwq_l1IwFuOO^yQ*GFeL&s2o1 z?tjxI+$hf(?kpBBez|3-v|nw%KNgUIl(JKyxYsG~Bs;1mF*+*bE(h^Qf1|}&8@5#3 zf{Dx{5LL!liiRmb_8+I!7-G|Yugy-~C{E^wmUZlDO+2 z>j|aLzm8BXJbd6Fqp2*)C7PMk*FwrgVrJuQ=Q=bl~)4Wnxm#!buK(`u=935ELr*lEru9^Xko<%KZ?RZ zCa+imC3->u^NA>XgZX+6;h13_<^D32n+*TW zoED+wA%@;TZ4gpXBZ6ljaWhN4&7Cb4Us3}h%`-EN3AC$&wN&-RvTwBg`4H6&{~rtL zM;XCkdl5AtyodMj_VUT-L6nri^wssc(QTWC&r}8D@W_kkt+TnW;XOwFM2${i%$DH{z|jl)gx^MGe11T3;` zWM?36{HSFmAK4P3nNQuZrCTiaix|PxQrOx91UW3?4uG-UM+<BL=VSa@z1RP#BFzvn=|3fMYJ)6 z?PEe*EZ$oE9_T`buup6^C{*oZF=^czmC5D$J#zN7Kn3&$CB(7OH1BtZ$B(Q<=5epA z{6&ko_n?8?M`Av0L!5WC@JmB2)baF4tddd2F1fwgaRz?dUS?#rZ8f=XYqr%%06fXy z+owz3-c!Rb#>-vM-*z3|;;7bNc31;M)ixL+8B$JOwY0Xj<|DzhFiqF*{Xc|B^`i1A z08OQ8eE*t$ogah+QU&3B{ig}hCvS$Pdl zO#Zb`=Nrn?Y}NP{iyi>1Es4-9lR2tb67pHLO%jIY**~7ZNeDI(E(Ji1esm^5#5zfF z`VK1gNQNkRurfj4wQp~H)Q}ttI;wh!fZ4p$aW;*Xcm{IgDFg>R+#h7w9{a9XjnZ04 zx)flIqUBiCm=$9R#57ETzHy%%Pxf_4b{C~QB0IxHQwbj3V=v^|J~B>%*1ERpX^h84 zLhBT`oTm#%IX}T!oX{^yZ`U|HafzIb(UH5K9N0T2I#u7<7wK89lYa9to#OQpjMjTm zsmiq}`+WdzKlD$p>_U{=naAxR;yZOn^wj_>jtkzcr4a{4U`=Kj{Md%;kuEvcd{n?; zziqTyy68v?t3*8R!H`c}aHYS!=HN}p@&Le8XvzJiY6Dowr!*#T`dx|-g{lE5mgtLS z(-^QTDiW9)gpmZDXJ~DepKS{KMh!_O zWOjkZ;_ITBil$GI@#r2S}`2I)NoKE+% z`nihGzoRa?QNjXW;tuNKUsM3Z5M8a~-X0|={q!eC)x_7AY0RZFV@bKmt}|w>>y^<- z{gaJSFcsLIR+XRMe5hrs2W{Mo>1yc-&(R0ci1y`Y-7_KV9jbpMh1G&5&sv8fmhW`% zO&}*HM7dJJ*Bzf$ge#v2L4JLVealdhn|(62KV5C znw_{j#XGmLi%6{VwU^>%e7-f0Bs`Jx?8i;t$I48b+BuIekt{xRi_8WVg~33QBN`7) zvGwR7`igkC5;+$`?pS8jx!Kb$_|N$7Sr=QdQt?<`X7c8n3$+P?-x4We3SZ0J+mx6l zx5`GI^{Mc0ubvRm=Dv7#f>xo5HNa*69*ZU|i7&W4Yr^Y;;`yk1^<@`H?h+60KBc>u z&+zP<#4gxB$u-+Fi~O@_cm04YUZ^3USH`0-}0q^O;&ajkC2#x|NBC7$Av zc*$_MK%}YiaII2Uq*HqkLQv0RW|5+>ZEB+u?+*_Ky!w{Ffe?fr7Nt@2r=eLPmQEv{~=SE?{pCFQt_yr0^2Z763MXuK# z-5^^cOoDb!F5Ehygj5OINoGFDf7c9*UJacH3a{lEv@Rm!+g)Nk<<_fpT*fAad=NmL zIiRG;GVccWLvf`V0C2kO(n9I9lekTNe&S`5o0tnx=~c5Yrs`&lu~Fp(2X>OgSY0i= zrVGHP9w$9)u#g5laopWzp7a&3^CFht&z4@_uzsO6W$YJaPw3d!ge z9|`gcxO=qCZhsuwvY{0?)uw!N!RkKsAQ9$8yH$CI;#O~8PsJbe}yc5ye>8!kzPG|`G}v`w)Z(TVd`DQ{!NeO>N<1L$M>;` z^LuxH9zrw#5k~=kSEq#9uTbkS4aI<#z~uh$kd~$oWoZG^pDQL>9N0(EK-Y}VuK?9` z+Oc`tKrx+4X_eryTRca<@_fl(+FC^Mj+fD8QV}-K4~l=)or;#Zwe9_(PQj=#YZAPy z|1qN4t7W!nVw#X1bwY^>@41e1>ajK^@BV?8HFQmzRa`Hawsya`XZNwKdZ$r|Sp3eW zT8U#|_`JM^tOcvf#Jka11=3x08Lx!$+O8ZicdM>VXsJGJW; z5@I_I&;`z<5P1M2?W>`~{3rR5xZ%uSieW{&T2&{&W6ERdO#9|~FXe=vc71E7l9Gcw z4>}T&9vVX@CzZeuO-8rQ=(?X~fA%cafQZIfr^x(P#feTf(dm*9H`p|!s^=bt?A?Cc zn0VOFS#r+WVz%0(%c1;F2v=xBwmVk^TAb*WgLcP_KdVTyjs4m^V^_k@Ner(yyjPH7 zGIsL7PMW;R1ZJ~5)ZG2<8So=BGNpbApPUqYD2Kb$T8FK0V&^C3DeLG|<2>N~=PSq_ zKm+Z{%9QgTC~hy`L|=e=&=LtH_z}~l1ef@A5gdR0@AE*%;2?FA_fPQK#zB&r+rgW9 z>7q(-`I{qER@m@#>OvkM^zPwm4oV)trfM^wdD=RKt<5{FmmQEJwP6B0JK$YS?)5JD zexPyZVNB3++j(kxJ=spU$2a!k;D<=vjo9TZ@nV8r%?B;7G4kGez1WyBT#38Suk&PD zx@XKM?fuR==qqZzC-SxFo8!X)c{v~5evD4W@$Y|ihNDRM=!IoYDDb^{Z3wvF`V&sK zfH7L|snMT$QXW<;$2&bCp#1xoWPfE{?;g)gboU~w87Zl+bDTBfN!(OF?VYH}3`W_! zG=x|;NC>U!PuYr?P36eH3azA!cZ38mnRu}0ml{rOakBg{!Zrgrv`-!uIGHFp7J}T2beW<$h`O!)ivJH+gg{!F|XFk9M5ob3F%-85gn zY%q9hI1svAQElF=;Dwk0+4k-{9bPR4!;v>Qr=vSgTzTvR{C>ir57ZT;ukm}?g;(wc z9j2sc5SN9FsA@kAjX1}DJ;fA=)N1f+&uWM3t6ckk2SPa=@REale6q*J^GnPF_=8Cq zWp3@Jh=_6K_q*%V3*QO|!I_vs1db=Tt8;yvr=x`q+uG%jb$4`udXTff2o&c%xyK z!?paTJ3Dvw(>;*ABV}4ENSch6HAz?WOK!(`q-x7oB!O>=^MT#pn{j=cX>Uun{3(Yk zEh>iG4!*I873yYd?MFS{tTFyJ2hk^0iJH+liG0cf7{lSPm1KeI$}&xTm@`IHnfu_0mJ|mFkze1fv3u6g(O6sD7LWhpFg}MN?g}o`tN1k$F6EnU-x#FGT8t! zadBB?8W`(NXJMP&6wjYR}KXjx8a_nd#H7xcgYm;lO7-fTurW8jd{E5(cG7 zJ8 zFsr72z)?*}bR|=$(_k0!m=>1RAGWh)n2wZmcClCMOo#kL%W5Se5@kf0O^_xR;~c=h z{l6 zhd@kS*p)cp-#rilTz3lJbZaVDqI;X7jeK2;D?8fACbphmbY6U%7JvQezJo3aJNpmm z5fkYaRb{VqrR^g(rcQ?0oQ1$F8-ca4sS- zGTCBYUfu>p$HMHAu>oaKmdz}mnIGuz>T_>$4*Q7slogTJZj7XlB8@=IwKT3Y5DDCw zxX!KG_>K+@gXN@@1-X$8%kCEFae%a+rpO^dW7=oRv5+t|95|m{jDl7Mn_|x>oxC+} zkjR;d%SYGe@soN0IIgTdgDj_}9|52ti2S-<9>kQj#c-XfDjPG`=?>FV0t4pvVhj_L z5^VYe8XD~tIGPUs(h*2Hg@s1vAKkwXZq{dz_EF7T<}Fbrp68bQ;b_L0on1Sa$4o$v zbIY{97}3w@)Dgn_S^cTJ1j~JBuTJjRAmSu7b0?krz6V2p8y+9N?w!d&6jJuCGMu^UDkTTz7OHBb)+DMUQlohhzf@?Tc!M7of5BOOa z+F6D5U?`Hldy7s0FWcNll;63ld^0 zFcloUP8FtkS6-F$wl#P=)Q9J&wydH&3;Gz~(WiuKMimAr!wLXg@d28t-q-VyCqbWC zrP!h>La@7^wiK7u>I|QTG(T-W;20c$-e`aiUNb)jG6ETiWAW0JpDVM<%<(>eSKE*6 ztKZfJNXNpW>L=~T=$HC%PwRh(VV4@Nka_jI2FSulH z>Xtdzr`za4JS;EY(Bsz^H`~=KR!OshsEHj<2JOSL-0J~!isdsu4A%}n0l3Q1%a_tc z?W~m;2I8sPz3r@o)g~FI?vZK3Jd6k1m6l=-yfW!jMAtO5uQlRnay zWvet%)ebNgN|yiT00<{xEItK)|5UF0&8ShdG&l_MH2N>s@s1tl+uL56K|-E+k=BDHj~a6f#BufJ$HuP`9jmu(v9aaV1MHR*fNzi zBBjK9Dh8jY->!F+7ncCr$qL`ViR9#Qk9aAbnSD2u#JB22RfRv=Tms_jflWn8t9M7c z4*F!2)o$iI4DMAhR~sy<4ZyoUjy^G~hnHp2*6@In64yE32LZ_MLiu@4ASO*~%#q5K zot|w{sGIUXqnPcC-<`D8$58kBU^Hl`VGLmsW5!N>3o&apJwv?-GzDByaO`NyRnV=d8c!_J&_HsuzpA46{-1buNW(EfH2T3PNcH+dc%6I<$1b4?cO(eJyo z{DN{2`7fPGbLTD9@FwbzA*jU9t)VI#q%Cd2&4$X%CJf8A z-q=9ergd+Irc{ZieLHS&!v>^DVp7B1l%I0-ChML((hmIwW-B{**uJ6SGaUFTfg?rm z+-hd@?=c`76`Mm*J=M3nkAiV4KfZ*3n;t)K+7EVT(QKQn)JSQ9X9tFazL&a)%6Qje zrEFnvyOQUF&gN*@b1Pxy+GfQ-F+gsArs9m%-4J~G$^OC8a?SEirGtlDQbyUgi&?P* zD`%EU}%-sESN&-QOw_xO-6Um0ODhSIQ=PX_y^t7uBU zA4rQWlOX%&a$1aIOJ-wn)<&5S*PF8s1vBd0ytemEh%2?`{i8Vz-uN)Yg)eLY4GXT# zcl;nmtvlOMY4&xUUSD2O8K{vsssf9k^aXv{42$L?l0g(hN3GmHeN1Ni#P=0>8}_%B z-ODG;@Ot6lE=^$nwRG5ybH47;K(#5)ZL@u8K}Sto!-M}i6f@X zO*JY0kj)ra@_3s)N5_H#z3b`(Gh8di(CR&&J^;&u& z)Wh6$UpWhsDso>sc4r?gTC_L_K3$gXC~m%Xd+zEGNBu2KnSQK7E*DspnmP91ad>9N z_FUQht%_~`fyF(UqAuUlSDN2cLXAIbI)M%$@nxHdbbDu9Ok`ht;QNz`B&$-fuya-x zNCzDpoITn$`5|O%a5eWLMeaabY`VRib`+m1?{zm~Moe?XX55|Oi)B5$qnVd|c2H_V z8X3Y^*7v_+IY#gRW#iFwn3hlUU`~`IX$&Kiul;rv9k|K+F9gJ-7tSvBt9HtaT4#6; zGJSR_(<81yj{6$mwPsecQy~~+AY|dG)M;pK!BBS1D<K6I)BUWdzKbViZl8SVcz^F)v;+?-phr?tnvGv(~DwmnH#LN}j0h`&^g-O@~tv@0^qX<*PF ztxwrx6=@ui7*1>!n2^y(_nTICVx1}F(Sw^Nl+X>}#N3(>iZ<3ih4q;%uO;1EI4ygo;JoDXZp$YKskk-GNXu z`2O}A(}jRTXDaXU56j{F?<@yuOw3XH@qf@95h~S?7qS-xxn&>q z1@5_n726?e;FW_S*NSgvL@#N;2B#y3YqJa{>|;*?{HjNW3}^(dU4cy8>EU%U(WQ@E8`Ex*DkR#ZigEH2@JVQ;0i9uVBlN(-+|+ z&Eh&0!e;$&Mg6NI)i5}YP+cM48knMYP>(eP!ZdpCEe9z7!%x&jVGZY(M*Vk9;ifZr zk)sK~2dU@W-a?Tlk$O(cx<$C8COx=5E0zB0ynj}8Ve)Vuz<6pZwWF0@t%*42HCik7P&U4urxH(~hPi{ZkIG($rFrzB6*K+6A z36qbkt@H+@$5B~y7O{87^+~juGSK`_1AuAMVpW4j)j@vD{165HTx3G%KbLExf+YM7 zKHLPoOyc4`$a>ZwlVvQ(%Yi#{MV1 z9u}7S9(aw@?X4TL+!Hi$~%O1gm7jV z_kK?Ii7xJVO_4bw(@ei#ih(lP$rLCwdX#8us@_yuP{Ur6BWz?}ke%ZfSdB=|V-F*z zH@0!ymv2prn|>~b6(reh^WW}BLxQKrnN8pV50M?0dgJ!X!pEt>Pr6nG#{tF9Q&Gf= zsKNuPSj%SCy_9j@Z4O6H$laff&l>zf1v}oLH@b7UBn|b4IZjkkx4@Kt@DD?{sSqFp z$sb|B7LT5I;;qkfJ=O0VD77p){I>v^M4H(tIN|=OBNP{SAN#XAfSP{%e=F65D_-06 zk5C$xU%0rRS(}h^=epc2QMvxYZ@R;Ke>2`_j12yg6Lk(mNns9@iKdz;V@eATjnTZa zD8?}|IsVRD?f+mijD)bv-U5Q@_1FE}U1Hc(qU=23E$G6wr$f{{w;(l|w`P52fPjB; z=gxyF`1sN#d%6}g_ZzQj5?pD;-?C|O;ar9%ldOQvl7B^Hr-qMsx(Dk2h|`3>>2xCh zeMU@1w^sY~wbNa1hi;Ywc+Sz-;7{eY?T}M6C1O0cInIC<)t&6N`QvuX;euP3~&X-XLsg z1rcRG97O>nc`$r=VcP(n%u2M>nkfXA$Cupc!$@7D<=H(SeN-InetP0_O6hn*^kpIO5zD<(1 zN_|wqgaqxaTy=}ODecuuEPFN?fbHeU9U;#&r8uOf9Vo_$j40ELR6$i7sc_EHW&YaE@qc*!DARA?vsj_5v5aXVSDpo+H5Xo`fN+V z9TtTXYH9i9{KHfOvr>}LsBhk)!t1jX%4udu^p<={v~MngPNKIxAeXl9XX$uw*9%1Dl$WVP$_61FOKyuWGOvl8&Gc~R{;&wC|_?f^X z&P@%E#vA)M2w8ChWu)=Z{nN(UDRq5ivxpni)X#LNGO4ot*>e`524gh7ErD*!ypCYS>(Kt0%VdZ^d9ls8>Vw$^F)CysORx>FPw*h%=VOpaD{2kT1)6 zf}!gfr;Je!6g!#_UMpJdc);%8*e|Sa<+vg;aLy6@-kM?cJyS`JNcBgRM|X%>`?*1K zR`q>|*so8Vc2v_|27sO;2F7j42v4f@IWUIXeT63jfWO(Vt>YhSmXa!OFx#`dH8TI? z{_CWUp~TIX`~+w$s))Ml!kxHCTCHaYTWIF;cXJ%nWZ*15$Ocz;i+qYOOK)N)q{d0+C|1?J>MxC*-2w%DmCj&k?a-~e}^bE9~O|HSyT{YeI>N#XtfN#RQ}R~eKw>kht${5esW`r@e=&^53XT~mhTV%bNq*oXbcluh^j?EZq2H3KCs1t243;UQjv-a(E5@A88Wi9b{rEd z=WTx$5h~#1)sW^8g7gZ<+xyJqs|&;~zqeHK-Se27T*7d}H%R$u6eYFJx*!NO>(prN zxnyOg2W48${`$^B2gQP_pfp%zKM(I#4V%9F!Y(!5#@8dJ#t&gb{Q#8GxBTQ|z1Mt!Q5KE@9FPG*c<7Hl!|;J`eX46HUhhS<5&7Kr-9 z_SOrM>U!;|>_ds4$r}dNSl3%Mvuf5Nh_Bl-^$ZfeX*}fb3_K)LWr+c}T zO-(e-R5$?}rOhl_9_DwpfyH-nF%?#hVexl#{1B#7XnHyFr=mP=k|V$6gIQ|`H4*t) zZtH2sX%`ZUw#x5~Hw31?lh!7EHrs9bDVm@+S18lgCs|k*qpZ`H%w_kQzme)1a&d{?)>CDD?c>v<2pa~oRbz>LkK{5XS>FvFvn(Dgs(a;41 z6cnThqEw|f=^zSH0th0~5khZD?^RSpnsh>yE>ffgNFXAe(2*J-2!tXfNQY4F#`inl zJ@5CObMH9g{Orw zaetrdl6o1h$m|pxb2{pMr7OqaPobvUptmY*QSab95=sfMZ5g}iHFIs=P&8~we~nmB z^@mDd_gb}Qzb0@axXL&7vg}c)rqJr{`=}EcswP6Q=L?@$9WbSno9cc4Lo z4LbU{OJ9nnAfKVxE&vZy@@m1gU(RAn5OBWV@0;B`^WtARN)h05Sqwt$xpw6aba)0~IxKN@J{WZIn? zG2r$B{g*#%NPaBjy<ED$l_ETAQ(#%WfUR>mzdI{TVPoXQE$0mKcTF=t35CvxxYd2E2+ zI^7W__|@B!RAcGWo8&h$o4i#a>UnZvGLduRnfNIf+fS=AfE2)ljE20h+S*F_nQ2 zPlCj%7x&T}HLt{7c2emzyhWL>bhB!0lIwbC5TD_Jvb{MgI&BO?vy;VOgBq7S#0{NE zg$CBLdYDGX{Pfh^hlbKW#^0&N0OQ@4mc?UMxkY3qRk=K&OB3tQ>$y%}ZH7^qgn@OB z1DWr9)vjCcDlW8zA)1G1ejF$N3l~`=1@aND5cKl{P>oZ&F&#lOGI($)>S)|xg_w76`X^>JpvDQ-bkKDnoxp=UWcz_eq`WPbLE zSqy0^xbNtLroF^qrM}BjYB1+jAA_Q3ct0VXt zX`@uc(7$SwBvSRsB2Sh}A!7!~Q>(1nWl3zO*qQA$274Dm3h+o$p&-~+eI&CG=|*8$ zbNgl0WV`s2B>$0jfdX3FW40CcC8fqKJGtJvwIt~!RUARD%O-c9d5L{yf_g#%P7dkhyDk6BquBS*W z(#0j7)3E5igRL3}R_ZTpp3oxNLSXhAPqr~Z9LVEU8yPSt-02d8;+-OC!(%GDUEUVG z^5!RnW~Wk&VvyoYvR%m{?W{PM*;&cD?~$SJ^78D6#wCk5%D;Y>H=^4AZC{P}j{NB= zIisO(5JygKGU;i=So0x4Z@e-xUdfiK)?RVs`=Wh!Q7R}6C@5bEy}bP@|5P$EWa)Hl zd_&g3elVq=G*GGn*v3JkRr3KdG+iX2ngmz7a*Bq>h6DE9b$2G27f*PlsAP!}%P{M{ zV58vujP;{Q?UF#HmcQ;u(Oyw7X+@ZRV8Y+mhO(ImQ6Yln+QwGxgt&RnC$vhu90J+k z1ihzxfxZEW7Rp*dp38p?K-g{x(^%cz9K!~XxTB>F3VZ8Yh{dxL$Wyj8nG%ZvD>g)0 zP}>L=L%VFwsn7F z(rj_f&lQbk0`->3n$6oxGE|A*OSEHAG8N5tT+ajEO0i|bMy$9@Ec!c{owjtPd_Vhh z>#z`ukUV4$pB=1QBcb~t%k7Hudm=BlA9+~;_MOtMyuRbaKe~<7L$3MP8rXPozHy%O z*+b>>WhF#jE3trJx6D!jLGRbtDeRY@wkVR33h@>cm9E^W^nq5+;a>}*Y4tEZ*12V# zLBLikGx5oCv^zoHy)>UA+Zh=d^i~r*-}wL~;-wpXau$3PdRaJv_V{fA*WqdQ`BCg* znfaMXp{atg%hA3AEWf%5d6*^#+hvhCIU``a@AH_$Zfv|Ve!1S85U9vsELa|fDT5aJ z`xWWmI`)bf<<1E4ub=Y2>(?`iJ3XB_H_Mvbt}zKZ!iwxw-MwUUOs??G_jFkX zyEks00$rN97hL}R93a)j_y?AHW3F`&HhT%-n@va7aI^g-U{IxcFO!076&UDhC zoiWGRUIHTQsDW1x9*4w%nQKQ1yPwkap5T!_uPq8`)QN>FHjz4b6MV5CQ=L4 z9JX5M)tw?U2O@?t-;&gaMW%^GJY*uK2NuLvK1B?D5De09KCvxbLVV{s*KS9axoU{4lQV9Sfad~mQ zjWNDSbpZf`+`1ct8Lh&Ya<2WgGz)&Xgr%m;Mxko0tlab?c;!Mww$%Mi%*2+l;!B7x z?5RrRgOd#JipPOedT*JoEeEl3RJ>e}l?|!_Qd6YoTxu740%19e@|E!Iyo>98S%h65 z6iH9V-ez}T!>Y+Gb((iie(YRRv;q27U|L@9E-#aEBnrzZ?8%bEG5)*giciet<1*8Y zS0#*$33o-NYipLjrftvpNnRCmkB}X#)S+H=GrSu+E#Jq_6Pi5g*69}ihj76%#X}?L za>cb58}f)`iW`2mArd&lS5oS+6VP({vYLW##5+N{gl2ubiAM64&^~#@E6(^FRgU z3ozN-Ae46uZ)q&)PuMfBBvwgqagQu==?T@z;!V;m@}YbTJ(=1?CDBi6z0)2vJgyZs z)I8dzC~=+@fYUTEuBHZ#%vr1@tP+Z3lV0U2wsVQIb^5kIa^1TT^Q@?7#*Xkqo!pFc z%v;!(_;AtIP_;wZG%f>vT6%n72u}YjpJ+l#rkDEd z;QEe@`2E*um7z?%9tmS!6V{oIlb5&G=jcGG4VA&?BpXyYguDwCdFV`?z z+NDi4OnUX`bPD*%1+QiFnO%a$r^41i?sJ)`be3S|7A96$3{@;iG8xj`4&!0$5=0Zt zWNy{03Wb=@_>`pcy1fhzsx`<=#T~a*S6g?N*`OIW0I7ng-ln=~LJRgr0Ng3Jdx1ba zSt?)uL~0or&VtZ~($tXWKe>5Xd`L`vK?a3bB0=xQ->VV7RZ#y%d}`9{+l^p1x^3TDFS+!6V2nPLM0rELm_<8RQG-2Ua{EVw0P_pJ^8?TwtON~gYJ1FKbuza(GdcTk0&kCWxkTbxy z%f@M#r~cyYCalv7HNCw+TYJe29Ab-PaoSIJq9v*GqOa}axX_+R#3gjX518r-Zxy@0 zop_04?4WSpGLDaLn&utv^>h6y}ZoC#)} zgRT^alN|-f%eb&b1)8>RU zg0I7~ryw`%sCRMdsFHpDuGtwZSJUykUXC$#+ z6-3^a@FQXkkjQ*@^9rodugcYXMc=!l=;RlFM|%K;fVx&N4xAuKCau3bAfNdhXQ$!5 zxdky#1Xv0{cjO8Q0H4Q^q1Pw!(8aaxzk6$}9uai0f;CoP6q7aA5}jni? z*^dSJx1!uxHe8c#+<5AGKuyzqg(3d2IdLOAs8j>f@7Qu`w-bECqn-EDh8Q(iZ-0<&wHWwq% zG}nwA1kOYP!Vuy|Kq*J_^Z_|x9e;AEh{YzDpj`4H6<@_mfSZse`}sf4n# zqFrXBacixPHsgx$Z9z~a%ofAyWYm4aDteRQz$q0$8mYN+I)bV(m*)CmQ{pG$8T4@m zm*Jp~Kz~i3K@sVcU%UlQBVh_REGz9>C~2t$4oHnPS&2mwUh1jz%*7eq7j%i^J10l( zAMur9a#4-*RU$B;w3 zx5JWJSCx5kkgKWcaY)D5cSy1R>dBQ>>5`9%FGurYnn3R(fRVDx-muKEZ;}5fFEn9k zccn|SiiUbsE~LV&e4B`xRZ%!)x&Jk((Piqu!RuwCse-e1Ge4=zpZ)T#(XQB=HAqzA z&}&AX&%q6J9rf3PMp1;wk&4L*5-y*$fb~`aaN|@Od&LrJ-P`UyMRkSt`+MCtDhYVV zfRoC|kW{(X@k@qNxN>^fQ`Le^2d5m52Ytd9BrCY1bVt0oV|<|m?}GQee)f2&;9$79 zbWHqtW)nO{I(BbyT*9}-sP0LL_`BcIdtTHlx+!7ikU~k=d0Gt zEV_^%hv^S>Mp)~l9zrwM8=rNDwG~e_Y&r68(is*+1hMCIAF@V?K8EyTFq;N$S9;s- z4w+n|wg7|?ls-r#6p#wf;zrfvTew0#0Xv4sC7L%Y*Zu0UA7ZIQ`md1plZSaO&;ciPS-Jw53$+6qV5Y!N%;>;RfYoqx1{~ewlC?+cLE=4y78s6d zb2IbXRF-a(7GK&nMpx6fbb|$P{$sN~$~mj#ic)i-J}WRcU-m5T_Duy}eMiER2ilZ6 zxN`RE>pG5XRmnd?eY^}gCN7l^42A{03P1d+X_@I)6f2=z;LmQnC2DgdC+U(xmYtdD zlZ|Ye!XC+L3Y@TpuU%qInG2dcofH_M>-$HQJNJ|#@O-Xy6#?S(WogAHeZB3$Mw5l&GW>dNm`kya(&g#U!SaN`^N)n6mDEOKEIx1P_4e7lE8LTjs zkmhn_jX~`by^`{q>Ld1x&wA@8|I4>znLA{?-jV!tHd2(sHGn0hN%c`jdMPG1oJV&1 zr#0vCd$oHV!o8i5Bd>gMue%;5l$m*GeK-^5VRS(X0GZQn@&Z@R(IJop)QDlRgJ@*3 z`N`UOJSuq2JAXTDpuS-xriJTFX8B%4Hysb4zW`*(1fNRwYzMlNRX0<7y{EawvzuD_ zNUo5`ssgIjISl_R_1OP&`q%$`9rFKCVcYUwdSb4+XuUZ3O=$}m^A(ozuY?lh+-pL@ z!0FjWj^{D$mTv2LUcNw6hg5J6r$~TDL0fO|F_O`|5!?@b?u;SJkqvZHhjjk;5E#h! zXmKHSaWr`5&iQ^GThBfq#HoRn&~S7tMO5x_Zws!mrIf9mD|wxxZSe9S9(q8n)wiS} zpz%XF%U%Ed^{)gU&K(>bH4y#suKqrc3G?ojZnHx1p~5iG<5wMv|rlMGWnz4%`Ms zC<81&JKPhTL;9NiqVfQ9b%G^q z`#HhXzgG{Ssw`~&o5yXsIZvMD3kUB~}4s$4AB~{(uC# zxFnDXWKOG8PWDO^aKdscbe!WcKWxlPYI%-IGn~TO>*OZG8+v$kIl*pQIne%=tj^^3 zUl7`b@KyG*l1a~nz^ZS|hs=tC7rMPAzev+=+L=is+~Hl(l8@`dT8YU^nerV99Je1E zXjC>UQP2M3N7_EglTED+jvl)qrdBrD&y)G1FsAPMc8H_ys)^nZ4;i;>9jdwUsiR2m zJ?GElqjgUoo3;YY5@+d-`=Yfak6^EGJxu7!JMSd&9*0jeJ5;H;EW{`64OeXk9h`ch z-eAkRx!9^>wE9$d_fLwIjBNUo?d=^R!!Gn2?OhVqUk&{uCynWQ$yJGl#9+ZC7|ig3 z*=@=t{d#*)Q>Y>-Pw^jxxw|}Q>w{{ruDE<{&ItRhD=aJk9dAa*hKq8bAAFZf$S~u} zM*YH4LDeWRhxajw!-nbh`QF$3UKKM8(EIwOx$nSG^_tw2^CAg(5hS z+?kdnK)?3&LWrWp^(lj-;~ATtKk53QzxpGh{tc1S&0X}qgTtm(0*rIsMXMlfJj=q5lp zj>4>tVLEldT3c{m@dvf2HFpYnYMiS$frY2~hQa4kx5~-3XPIFrF(ameuYQJ8XAdSF z9YBe+p_g^+YTY3&J6^s^s^|jFCyrNhBIb@2Ljw<-VC?J@;|qf~S5;D|D|oqH84P^p z*u@mby|T&V$E|yh%oMbIz;D(WI@`aI$wTxvRJjFFdu#dDn&*OqGLxNw+0xMe*(Jnp zyq0*WehCdvHtKD*n=1gn;RnjF6_g#crynR@MahM@TnybM-K4fyt*+elW5c*jBZ_+Sg3?b7^CdrGhfr^8bw<%o{Jf`j>h(uA?!Hee>A%n# zS`dahdGGirsWna!Q*|Lu*LOdpPr(;Mv1QviW$~$k(Nzy_c45bynwpGd4!Mq7?j?nz za44qdAJSpHEZK?F11*KTJKNGXi)JNe8w7`x7~X+aWd2_d>BLYyR*r%7C{!>45^|Ym z8#FN$yriVo6m{uQQSGBBG^AxDV&z*YtYQG>okfB>9DY)9j;{bZl*ok{d#&&Eba59w z^g^(uAmQbew->zr#@M>eOYj$MD&(~a+GH&61*4M!dxg7D$b6nXd6yC|GYQB(26*ErF_T3`}y1QquUv9gQEVSpV$IcC%P3RuzfY>leNUL%0w>bK<1+5 ztZ{zZuAS6Ws5$;|?t5tL+OyOH+C;(vz;l9K!D#2IAD?U6%GokYy>F}foj0K`m7mx0qV*VFd^)X|HQMZP zp;g6#?99}XoGbUn^4{U{zVRgYsh5;2VTk3$UdK_edh18c9LbF{yK$O(40LuLqi=7c zj}Z@KrM~akl(-u6(YP|YvaOmscHUJtXST==G2Wmt&(SV*P6QuU%HTub(lXDnL}@ph z8UEyM%hMBzU2Ifv!_5*c3Ha-SQD$4--H@ieGF|K&$I9ED@S#eXjnbo-NDG0QrXMz0 zIa^<0dCp|9O{AjI3eJjo%Dj-z`f7}?O!T$ml|w_K2Z{v60@thAiYM|=A*x`^PSBdhyKZ9;^dL$#{S{?KtGsEt`sn4xZeq1!?5hT2X} z^OoJj=I^upz!JPP*hPdOf%WN#7IWw=B0g@Y1)tq169Xbb^)k+H*I6|-(BjUkk}_C} zagi0K+f5v8mU7>}Z&fwPdoTps_&UBM0;4`pDsG}66>7Yf!ma{^>(1K35>(89 zWbw}qmb(p{ij9N^(!ZvHXlvacfYmPYKXLH_>+$8|is!S(h-S`NR?(Iv#)=WAMtP1* zPQsAnj*~2T<_rFGYu*cZArJatLSu#faWW+CIMbBt~L}H33{42E$-Rd60p0%W0ifM2WS&Q6)hyyphmAlo={t$b& zke)Vq2e8fonbwcF_Sqvy#Q4>z@oxu}J48f~DNk7$_=xxK*W-*+X7hOFeHr!brd+G+ zZ8)T$GdTL`B<402_W07K-j>{|jL8mw1eNkAOr`*#BZ2a@oGU<0vHZp-&pEe-h!HeM3!7AkYEhz#i>EQYrGT%HtDf3q00qx&VDFJ=J!|og=26YcO*LpeFojzJ?bT zoBrVa8?FJMBF85vx2Ld}UU5Rk7~GxY;&NUnym^UkotmRuIqk0k9((#l8Q95pF(}Qu zGw`BDJZ%Dc^6d{r*^l7wVe|jc3F+T6K|Z)57d>4h97zCD{B8eFN+jS$0x*WTKp@4T z|NCMekoWmyNCVP$>W}`(#B5lw8h$aKDoF!SmO(joHHc9+0~Ce_xUWr)RE)O?kP_UC zLj5IHZwxSQDO6Af{I<3K_}Ke)ssnEri!t)ioBoOfg4?R~QW1N<)#BNjMs9(TktlXL z7pnIw1f<7Nw~+x_`)8F(TtS`@9}w%L07*?s8epXYv}(`gPC);dtV>FHI*f4bzS~nR zCzp>uqm~+^A`I$6KWHDgod2%R5FukxJeoTC?!Mh+9&E6AsEvtd90$&DtkV`K{&%b8 zBcvXH|0zOFc8e5}a6t#~?Mz&Ns{BF20Kf}7uKtUq3Z(Vy2LC>b|EC?R{{27v7ZX)* z0Hli-N^Kzz(7#W#vH0^n_;#NHKBl+v%-uyGWLAuw3y@Zm9HRWY^}r*6IN$TvT?$C_ zEkV}zs8#A6lHOS2oCDTVto!!txl`O%B=UiZiB~wlAd|EwSFj7|5>FC7n$IV2o7I8k zJiG+aT|W~N2*w+|1@cM}z-OkPah@Ku;#sEPuT>}OhU|qe$d?@512$Cwd3j!-##YaA zpKF^tg*whYPs4yn=e;7ANFbz!OC>xtY9Lar?C9!+CDuxMa?0OD($06v4d_IVoFoC5 zVo?*N&97Q3yP*Ma~V6TYG`jx8>y&*(~`D|k?B(dKC1msI{nqI75=g5 zOk$AYfIT2tFGw;$&de^9S{qv$s?#OSG#f9?!ZyTVnqFk`3>g@O3)JYi zC>r&#CG^}awDW=$`Y~c{kzB@^Fyi_ecATXaq8l!oc0Tlk4Df(=CIR6! zXpNHgw!3H5T+}A+JrlVXj8S+QA9`@X8Q#>!d@qPRllrH@ODO6;8@#j$|BJ!PCBHnj zM*xio1j-i&mpAAUy>G9&b2X#}mFxYeP>Tgu2){KU>wz+|&;GgO{c|N$BA!v)tsF}2 zTcJPu(R2G@&a>6XXb#tn8QkpP!G(uhoE9KR#7zUuv3`ST%je5?)0Gc>07o4EkWTw0 zN#Oka7DEH4DWhAr zZm+A0mc-pfejxP; znD^h{Nm8Eq!13#E8>nhD;FE>zr4Eb>So$zt^OQg7v}l;~?uAoQPg+xfw6E>3GELMy zpWX7d4a_O)#pz_-FJB7eeY6WoB>!VSKjAUj5XAaK)e|%Hu5u*R)-ZN8A-m`IZ8j9Z zrRmDKr3~KW_%w+ski_s+FY(vl0QK-2j^v;ZQRR&))Xiu{&a z<{w3H`JS1>dUnvTo)*}_!~ucgS73p6p&FL;yHmgFyB%)5@I600psP@ROReVb4`-2b zd&t)TuE7z7SOPg_9IU(609BA;x>Wf6b7?7${A?BKJsO#Gv<4Wb*{RK6twL@Xq<|-@NfL2wfcv9d2dtT(Tmyjr3)7 z{;px@@@4uGrke?r{a&J>W@FOVBFBC08kUDKz?Mn|1PTrBU&_m_2}tm8J61 zxY~P}>yLMjF@%#?1A+*p2@O%d zrago;6QIfCYN+agJw!(*{r7T3En%#e-gAt6A2_K$QLrLzY6;N(0Q1MY0shz@l+Pe| z*-Cbv52Rh;hHpJf~UYNadwZ21ElZm;|sY{>Ed6KrU{k7}_I z?9yQVLcqfaAQ3rykf~_T@*JlWz)z&0!D0Nr1P$*g;_7`%oWw)BGIxLm5Zp&qCVD@m z=u)Xmr3tH|e$Nst@Td3EZ|kim9?1g=5p#ES=&T<=NzTrm>TBGmzC`>iH&DL><6J0P z-)y!?-cdNrgh+$yCjw?XYr{}#J_YQ=gCeWR@ zm__SsZ2K_?S=-8B`oCzEoT_6-^A~^I`7z8BJ&GFMXV^ZbB?)^X+GG)*zZ*-aLjf%A z+)==Dp%0BsV{=YPepvP>M8e<;&9w>d0WV6zut2%#(cp3?JDkE7(Vz&zZyrW<-OuRm=||jdt#K;TrsnaR`&WM-{}AhHGD@gahlG+7SIzV-kZ=M5WR==PBCZNDo{+L8jhoMB0Vl26 z>;J$Coj(pyH%A5uX<+BJ@l~Ysc}x7Fo!kYj?v?gE2t-V02`t)sQ^RAB=AoZDe*eq0 zgVR0~@nd&`LZmaR;;Z(5v4XojoiQ)-cBS8U>l%v}l%OSpAq=9hYA zbjvdRW#mAE+hnAk6Xo+=R0t?Fgqo09}d$>h~_>fOvV)d2_av z2KU=>*lB~>TbSSiY`msf51@Gi-~V`jH0cgh$=}}fuwd3nJwV$~HErA-;zR- zcU=2*P!6Do=sGyxwDU;x(RvgB=vXk4#fwSfN;{d~xp~g{lOD)eo9l%YI-x1MJ=XW= ze;K0@=pQ3NX0gF8$fTH((P6A!1i>{TV6k4|1=Hjfk_dj=B8v>LXh>DeE0t%N99zCV z-yM2OQX>$p>+6H^!QYuS!l$0^))MTuyc#{z3*Nis8eOH9ebp+PQqQ*2$B%tP)G>_U zLy-JLY$xf1Gxm>tZrYwzlbnc(*)8JS<>4=`Nqh&1Rx;lTm>*V+G^gWIFD*SlYd9~F zQ?U8VJhtANa1_eh86g!rCZ5-<<>Mvt{QQg8jps_`UU92XNCvR1|KNz)UQ`nN6P@$i zY%$9UbA4>LWcbIQ#^3N6l7CpC=Cz?ot&(={p&nFDgjw8GpB?nOgS983?9``fic)3R z;~_N-$7^s^Lk{e@SJp3$iHiuB@;}xJXDqB- z+i@Y;uh?QUaqY2|X?`L}DzAwh|2#3yBz1qJ4;Mxia$*#vCC)ZU8s8gi?g=&9NJ1im zEaLDvN&6FcZK*mzT%Db($2RD^-mWrvkGF@Zt)82)sJTM#+ag)kAN~-}>(34)NzbI> zmMBbATa%Y0R>#^KOw9v7m2my8s%2Ry=pkm$q*xG0dvMsT;)-C0H{&6V#vkdBWAjta zX1Upy?5e0Z(s0dwP9SWv{{@byboYOZBT6%Qqe z^&9%pq|+P^x4V*{2U?iid7dm>pbN8#b!I>chlE*t;RImB=-T|24qdu@pPHrHudo#) z#r`_!#4>=^0@}J!OJmLzO?}-gdqq^8Frt`#cWN5KDZScgL`0X9AIJ`+pOq%85m|$) zCUbRqs;}pb5qkzBZc&-7XKFw5Va*Ms3ZRiu5GqeJ?X0er31rar$;{p0Y(~C7=H03@X0PS0S3Q1 zTo)JHO!#|O5x)cpVuQ~Pm+3;XfUUtzsOy@nEe-u-@Q26$5{>4)YPA~P4uyiYbCzSX zR6!JP0dR5%=(lFe{(y?YJ-HjcIkKZNz$O<9x<>8IPcXC9%^PnDw6ibfq?*VZ<@cDF z(k#|IB(>Gt^xJxF67)N_D}Llvnmm^cG&y9pZDSm;Cpx|h7V@Wt68@?;d*Qzm0$?ka zgtIS%(?oh8^)5FDp_d>mgpA+v21p5pf2Jw(r~*_I#dq}25IzO|1U&z>gYEyewf!z_ zXs=>*immiCuw!>YlZ%YX_piqv^5wmK_u&kNvB*SNGQe?Pd2W$n6rLsVUQ_IwbSDbQ>C|8RuAjIcmt1nrng0z}9w zg|EUoDBBag*+q+O!rD|WCnX3Xek}n|s*jlb;2*d6zrewE?aYq};0pDVLvluspAbeN zuMcgc*J1oXN}gDBxGFmQdzQ@YxX=5Jgj4b561c8qI2pX4eJD?{!`pAbOlqXgTw4Bc zA_#an2G8xRJ3@q)yT1MUEn?U6P!d*8LHoxdnKbM6e?VUUjfnsN8AE_0;eWg4)ECdm Z+WXoC#-7F+0z(CqY)8Grtpo6@QHo%A!BU%Qj;z{ei zK+0gpqGE{kG;D8A87}k;XsAJ|2_U|KIykr}I>#ViNx{$N}X&4r}QhMA&Yd79hw0 z3GJZp@cuJkCRMNPT=Tq5epk##kKa5C9ZTUrWgEZL0%0g6CJgfV&z0R#6!#^8winlM z0s)~V{^tYgUcl%2C4_XAkP(60gTg_iCwjMzsQ(gSIg6+}3)$J&nAkdl2sxSNZ_whFL}kiZBpSEkBM3Q4ord!~o{&E|xy%>qa8^OAtsR{Us1Y zF8rqzTIov=9Z2yd_;+hAF30;+R{ctkVFEI$Ae6%yJr;jemk6?{CfPY3D7Z*A`OV%} z$7YveO7~`gjZ^uBD?RCCz+`uhv$-zDy3T$G(m%Z@sQoKm`RB4z{ZcAaTMmm7@=bbr z;_TIkpBcV>aQ$VGOz>iw5ywLJRe}#vF~`E@8T3o83>%rZ-5tuPwb4K-trGM`loE_4 z2&Lel?C^UXY-s*mMk?7n}UZnPaQlurd5+j7=Q>^Z5Rx5dvbKt6;ZlhVb`l zh6oD9wiKwB6VB8VGx}e2QVt##QdttVHqR$~E&ws=qc^{^9*caEIB*mHnULMBL~bM8 zv$OV|-k={E&D&guJXd`$D^?uP>am>$zJj{XBvLTTcUf-Bs7nF#dwf^aMH%YjcN2)t zag!1*v86x19wB*Cfes=GV%DAaY|m>8Z|MLQ$E!Z2;KXV79A#?1K-zvSK$MDsy4-tw zyO`!&UKpJ~pRjX9rVk5E#@w5--yxI2Qbbi4Pw+IgPcS^|-$0)%TNRljw$Vkar)*!e zfPQ4)p; zf@f08>LZo9!5A0Nw-T~V2_qU8P?{_-QI!2Nxf>CWvPiwlo4|AcwpBmCM~TB*8GM|i za#IEf7N!^_%J8~;7qVx)7K<|WtUtpmS!vS!dhGXn(iTLm-5d?>v}(e}b8 z&``tlWWPs^@5PX6AoN;YVTs9-XSHKQzblQXGX$^y_;hsK)cvRPpB1zbbStuD14vo% z8LEEK=Ed?b1@gQ>@B`kLZ1{llljdt=c}DHOk; z^Q@4Y-o#F7T)#McwC8sS_iR#3IJS05tbdG-+o8D_shfl|G24tIk7)C&SM+ca^S86GK z375Y|wxg!U-#o*Y!GyM87fEG{8wxAa+j>H|_(NpENP@3DXi*wuGW7k~aCQFGq|j3+ zae#~#uHbJS9t_j2t0tE3wZaMwdBZ|>uI2#NE(4<~k(`@LiU;mU4IK1Q{KoS^CXu{! zVV^UoB|$migQ|U}SRyP^sK2D|9XlFLj_0sx635(>x$tzHEs5g?oP8CX57MVEr2AZBCA6+HFrh^o@P$LMrO&M3=Ia!5V)voy3F>eo)?TMru{6BN5q=$)_#6Qty0$(O zXsSHS{#Sfad1KA51+n3h1Bq?PS(zyV90XBar;VoBnnzgmSD5w_XCp7I1Zz5EPI^fo z6w3chtYW+-GY(%8gn*-JF=7Ghx@|_Y-64JH=*IMysui$ruPxqQIw30x5xU#-(Qtw* zej2~M3O568j$W16RrR|^G&5f4IweWub1(F#La5j?ePI_G=-fINz7c3qzJ9!-OBBH- zqyCtL)6Lx=h8xzqBTsX>>{H?qc35SY4<0~YYbzazBC#F8gr?FRPsNg=knc7=6%SV<5Cer##HcyEFrLgD z=`C0|{^8<#Q9s3A=8-+N1m$atOOy5z=aPa+A)7jCSJ#w_097dIV&)i^Hgml(Ne=MS ze=m}vQ0AwC&7GVxD%Kj)`0`|IzbW1{G?_!q)@l>yr;?*qZ+(b-cE&MKM#8+KHhF}| zSGW>rx-LoE@yIREy>&Arzf~ze!1_fJLlO)E$Z>q`5L;m_egVHX!oCIeA;MBk9vPtuKWLmGL7^9_44u_Wpo==5tV^(XknG7!cY zlp59`*EhNeyk4k%Q>$awO{0V;7#!=XejJ(Fk{S;uF^AX2ew+}OE2J7YaSqg+NL}M_ zhj^7yB}d5_4b+5e z1_I@t^G<6=c(nuUd5>7RddLcfD) z=;2&j|)1XU&K{#Ua*JM*!h~jV4e!aX$fw}X!*Tu6@29zQi$t-T^lcq{g&0zY} zE<7)ap0(g*WHo>zBz*5Pc;{f_Mn0B{ur)B}t$U^gg(56PYU%Y`wVCh8O%2CVgUzR3h!VgGS7s}ne}7;{qbI6m z)Pyv5H-g&*JpJj;=}w35U{W=0rO}>`vyHBk#l`rF6NcYm*4X!b+f9ucI9<#7LRxmR;cbShzM23Qq7m>`j9^Kj1YFMy~kY651G zEb-E+&2;sA-#9S0USALoc!C12IHybFFYFtIQ8t@syE>Oblt1m;iHIE>7!7N|k&bj3?P|9zcm{m?+cvo0{EAzD_j@ykjWwn2Z~kbb^q=e4E#Y-Ztp47Wj5Bih z6Mhvv@+Tj1t43~tBLlc94>Z5bpvxSX$ETz@)QLx&ZU01l=Jmg&#(@<531Izszl7>` zR|1f9Lb-N+59vA8qUO%dMPSbRSi8INsuGh+GS|?tmA7B`zwHWYXKuEkTmYP70}yu^ z*NsA^Ngp1T)&4Jb2mcOY{vF~p>kRL8Nald9pbs}mXegt?$Pa+9Y1wN2YLuGPE-TrgdhW(DNr>Eq$$BLc+STi>Q^IuP`ODR63d)a#kq@e zN)Rs#<@J*oOQ5vjs=3sJz8RL1Kc(eMlNfCB6Ib);8Og)pah~|941YL<;-Tl4GGD9O z&q;;Ghj^Fe&G-;$he*JmDi5>QSkQkahbaIu#?A;A-rJG6tS4tNe~#1&IxPxy+2fVT zSrs$Ns2?iBEv7s4hmwrJa4RZqk6?4Mnp!5SP#}D5OHRxQg}o#T%c&y;;Mjupf)vqB zF4PIT&F37l^K%;qsNQ4Fkaug- z^WGPt$n`VCMktHaXqu>Xv9qyLY-VyqIxCicm^jYd$rMbl(kbTzCxe5@kSemvFN1Sw zdZiA_wYGfCTkx<`t}mm{-A59cm-iAFr23=MSLfvE5QK>5qls9?wFp)9!7Em#k)Inj zn*v>VA!ImYXu3k_du#f@_%TzQlZp=mi0ec~^n={Be%9iXrTzx&-Ew57eOaOO(-*<%xlHDHjVqp4?RPyZ2*{8SXN=M-Ba7#r){i+$h6)^Z zGQ-_^(%#aNf5zb>{R8=SbM_B`6Xal*O&qD!H@-;UlqxCSoU7lUNxm_>e?+g9PSl4+ zD-&${;dI9P;Am}>f2hhCKIcFP&+~fkup}+JXWlF@-1?N{Ap_8uWTmw|TFywfjDNmu z&fNJh2i8f^1iEF8I=$0Ts=?I@)oI))*?3MRi&rL00`j0zR3(eVZc=4< z-Mo^{s614cNRWypD7mIRk#Ra6SRz0Q3$Iubl=S&BS+XU%bri3P>F*|PW53^&MoA1ZAb~_y zlR`~*bFk?yrfNa)S5MzNK~uc<%C}j+t()@;Ly{gC(*?2PuBg^YCOnz0f8j$QT`_Z7CXg{z&iI5Ne!7WMRN-T6C4Y zoHd{4O!+-@s@T;CjPuLdoniV-;|Lhq66QjXEZ(Mvf>4vVJ@M=T7D zEP02X^QK_ztaIIBiejyAX9lp-jTdol_l_rD;+hJ*0f!RT= z`be8>C#A|>f5foIGssav89}i1v9IiuXFj}C@UpSSo7MF`lt}ArB3C1MUPdCBErw<* zA})l>p8*dl;)MT<_QzB6lpHSpO*-+fDpnB1%5-Yl49c6FnK-ADO<^~MZ!tZIUEnjH zRdPMQ#O}u0IXihuY14R&vW&bYFasj&TgDLK8pU(5O=>+J;nQz$W-1LZ0bT23p&5{W ztv|ktGAdZJSa^LbM{u~H*NK^+C{BIWu2rCsI&2G*yPT0%*To;ax<=z6ixndcu@DqU z#{@N;eJq_TpUDOSwS;rPX3$c78GM%|mc-m%9#{GWos;n1>>qu!?@3V}ubh{OblJTQ?!LLUs;pcsY75 zEUX4~wrLB=;#*Zcy+*7!{<%i>k<@PbZod05#m{C|wfkBE{heoKhrsMp$J`oD;*HUr zyvvMzQ-7twA9jg8AW{*MRMs|%y=B$WKE2%8{)HdCl6&uoi!J7xm>@Jz9?Ay2uSMo; zWWXWfad?LxtaOAY#W>J|$qSWN=lI$ZdrS7bPqYr{x1qpw_`xLu57Z*ervvOa{;C(n zCNm{uQ|_Vm(;j!#+um2~BD8O>Kr-7|E@_<}agr)E_@HvGmfaF`lOlpD+Vhs+VklugfFJqGCBp zI#3l(!C7i<+k{<;VA4!Tvu-2?*LHQ@QyARpaOI?98tn8txpO-qb4_weu-NJ~u}Rz~ zy?05a2*igq2q%lI+c#)4no))CU>a|ta;)sXH_=7IKig%BAs~5u14!2<4JL;NIdd=| zooU>+>kNb??&H}@x*ZIUM#rQGG}=%$_Y9B$@P`{B5U3O4k)1~7=;hf}4@4QrVoXlc z804nJgi#TNF4ZLCwfW|&EZifN%V1q({7b9{wMbM`=5Owbc{Pt+YKUECR?N2w#abei z%Q83#$F(aQj!-A>N4NSHXIba+N@fhV@A@y&??*1leuw6`l)uO}KQDPNc_C@uv)zs( zWe%yiOKNFQ4BJ08-k)!&g+2JwoBBs4oDbKE)GhJGvc-fdU+Ywjz@)f7`6Ed`+u@lg zSyqTAkvp>DTgpEdYQeO+z}eTy!RAs&jhT9?#| zTdk3U?+sS`aiYf0E8>zFikYss7g9!u3u9uhmhEV)zHI@|N4S1_(c2ua|ET&Ait^7T zk-Qrak#H-~862Gbv=0w7+eigf^mc^AQd?nd^FSK|l#M@78c;dlmqOww)Pi7c4J zv72y`X7C!$M}Xg%d;QALVk|I(`g&MgnqZ-ERd1!7$$a!vZXr?rc)kJB>$L-_ti3a0&K|SKlvu%Tg7)Q!d7q zpEjzDj}2SQ#HlSEh|Y#&jJ%MJFM8_T4PYnEP;2cD*;Pp~jO3{?%N`2Wh990tD6zL_i5nOG&l6fy57@J)#>UxS^GmZN?XaiZw(v!n{0j#hAOB$dxzp-X z>+l<_$ip~P(lEwjn7(Ov{8QnPJOAz$!yXgp!KDDCH5(gwxXYS>v?ER1qxvA(0wc@b z7IwMJcmUHSEqf|mQp5EwrQp?*HuhWou6s)0uNm7N>#MZvIussAUdNI8$sQ?3>R9Q@ zZo3whvx(F_axw6oSL;5Pj#(-BMC6RnsYFdE$M}M3{K*?fX7gO6jdROk+g}O?R3&L5 zG>gyVV0J^k3g;p#OCn0&3`CJHbNHhe8upC9pT&bLJn>V)`O@w!)*It@p=*y&lFV+g zf4TNttXKf7x!8DM1dbG!+0dDm8=q*~xk!5X2R0oKL8NMWe`W*6K39n~-Pe>O>A5_Y zfy$&0gb8JMcv^aC$Ah_ct%R6ZY0XsWsNtu~C+y~R)tJnP^Csg7TSvKTHo@hL3_mkF zF8Q1G?w>V}>d?sUH<1Q0yOTmp`ba0T;ij>s>md~7wQLB5sJ(tp$)zD81CqkyUV1#J zvsA^;TAVD7EA3rR=Eh&$F&Z|R1s*6OXtwgaZ1BuFFQ#%`#6ssP-iAK1RHEX@9f+WB zU2}u#348xyXGKboHmYcyHl6bSLSh8nLXW&&=svs*X{!qYvqC?NM+SQd<}zF@z-O_< zBRr5NpE%1j80tapg(nzj@N!G<+L8id_xD7IGQ%ysyFB7d1dhlVTmgi{>9Ka!4J7Z9 zy3*G!o&QugHSTi%wZXqsKkA*xR?%U_q9FbL;A$OC_`>lbEHr4Ar5|x|)C?cq zwwHqxbAU9WniFW_aBV-7nW}gC@p`=Yx9uyI9$yR9Rg02U!f+be!NXmy4t$pL8`Amo z$vRzXhal!$5VpUClC{~GjB(~^!2i?tbc+gq1y9133%)?dscgV>zQ5Gu*K?GiaR=`- z*lL3sSM^t#_;&$^4zEwR4&V6BoY^f@^CQx#n|a9A^AD7UvUQ3f%a$7lWYT7;sR^6D zllQfiwo@D4=TfGqsk@focUFap5zJk^^5-bhL?^G6j$^0BbMl;1o^S4shGl;1pyCs2 z!%f78hS+~Wqnhm3|MK6$503G~{;~f`LLLJljvMid{+QlyONYiWMxhLNrzFcqF4ccl z@Sc)5D2@zE@TnQ_JTRT97AlbDl165EC$JKViA%MOqi`te1rTRQY&nc8q(~uixU&aV zW5WaoJ7B`o8S42GpSC_L@DL?sgg@@Aav#p>X0hNn;GT?^z`f%2xS5kViB@$}u)^FfUhIE-GOHMp z={m5`wtbo{P906<*?u2%&b}CvLvc2Q1Y}5dC8*zruOf9VlTTh~$OZ4R+d4mmAc&7o z53eyNYM^u#Qb$Gxuweqsgm93eZy|?Za9^WaoDk;2dZpSP-ly1A)OLkMlbbX^ z!GrUB7=s^uS@l7*-?MVOgz!#*vs(mTdzgDbZX`(sT{`bXPjHmU;$-)Vb0*HSgP9rgK=>Lf6s#vLFA&0we*Te6D_U>MbS1Pgvc) zj{`JH(hLO(919(l{6NIc)ZZ-y?>bAZp>JcB+hS_}%o4PCE9t6+W= zI9PP|cT{PstM1;fVaj@^2;hzJL}7nzbt5s?LgXcQQq^?X`WHs|B!*~38ST`)J1|gu zy-NoAHc(5ycMXIWa4$$KnA{%q1H>;HBJe@^YJK&v4>`VhxB-n?ywthnfVOgFIXoKN zHFQ_Y)a)%=BfqPf5-rI7v%Co0%v#%{B;u6C0&w2h~rl2=P` zC>OFj%#2w3e`TxoO_bTG*lsGBCpCJ+G7B^w91H`rL*pd(%F#-b^KnTGbc<3itLfB; z`>0J6W-g|d*o_v84~a|^76o;F=BdYd#i@c>h+v1@Fk6nE)GbL*#&!&M2*5}ws;oj% zD!VBBjh`E(jK}?vPCmQrpqf|FCC!wOxM%b?s6XOl;G9&(7mPE$djHss*^1U)fH@no1LA{iZx6 z8)W9(apgf^m~2B%-5RNEf5E+A82|rimG!^#sHN=wrK7j^!Usl6xCnqQ1zlMf*H9z{ z3M1jZvH)(>!$k3916!EV^?h|^!Bj8-GV|kt>ZA0p zvZF`-zCbq9yhGRZV74wAxSkn%Jlz2$Wfad*c1@FDc3lL0m6W#(oZ4X35rIst4FsCQM=VA@eX|rIA4PNMZlqBZWv7B1Z^{WwU7ZOli6j>NoMRDr8tFZlt*#vpEjKQ zMZY9y$xHu6+~6eYrIQ2IW7v`W?D+nnbN&tE{JRGGzp^|3->v^5dl0*6Ubx4-)86P0 z1}gnk;uvY^V77hZa!b3(5Ta@67H<)%zs5^IxqYI1-T0cgy8nr5KRR0GngL*4B`2~gARK?U9koQ|K)*twX8=u8IpKz;Nj6l6 zumLLgQqiN!V1)$&8|dPBoKsL0kQUc3;jyv{X4)pm6&iPQ7T-)|tg(G^;mRAfU{nm^ zahtf@Z_uV}*+QZ01d7q2OrlWJa!R<4ww|sp9jj@2=0hwoAOvY(3v~IR5Yuu@*pJn_ zjv9QWNY&5!c?Rv|1PspQ&#qJcsn8PGCfJA-du!{x>TH)OS_$5}UZ5>kh z;pe`iCzlL>VM#Y$6jE07+!l7g>X*2t@eIyHG(iHu;L^n+9jcgUbpM1_QdZ;ypVY8z zKk{BIB{U$Sqym7~PB8w5aG(@}RR4b%WP3(N%3MZP7l+4=-PvS~_sS#B>=eOqEq3hl zUJ2;=@VMdDGOx8vOXJ=*)icY*JD;>YHFo{NLqIIM$ItNKM}wI^nbg?V*j|qZpxFnc z4&HjIDVJqBeqktjk@^naVuuKq!vQ^XwI&w|mG@-SrMzkJObD43fHPP+5q-Do!tP4QJc4(C9wA1-JDy?T|U|~*RK!tDr zB#5o;DA^?dR*Lssc++v(Y_d0AQnl>AS91LC;S4bi#|o@R>mfy+e|GDQis8hjbKJf` z7i-xF>xUI+K3UHyGWVWQ7=L)6UI8zvZKQf#{Xkmp@QkqyEzt9|u>mI>&4G)&1d!nx zw=;C{$9B5pZPC?B*Tz+|{fKEsv+kcaLYlB7-9mRDy4Ddm$-q%yvE)pVpk%QG z#8wW>mW!}2AK65e{GCIGj9(tzbYrpT)0XvDEq$5KiHcCDGhZV< z;ZBagAaWKqZH@w%aO2Y>R7dU@**7KEvM;;mX~#DNW$?=Eirbm>O6EE!;uf~gFjY%c z!w^a1w4CZ^#a!B%@cK!I!iz}v|$;fnd#G4s#Ua3E+1FJd49N30hs z&;SoV1F+p4jh7UxBhT{*V6#v2Xhz;$;4DUUo=8cvMPMDBsp-PvwzmAzjfSQacH>7s zqB?UVx9FPrRda940u$_+y5(D>iYVt4d=;7E^9yzMKyMz z6icj%SQe~kY4 + Maven Repository + https://repo.maven.apache.org/maven2 + + true + + + false + + From a2081ae4e1e309114243d833197c5e07ef51c1a7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 19 Jan 2020 18:39:38 -0800 Subject: [PATCH 1157/1387] [SPARK-29290][CORE] Update to chill 0.9.5 ### What changes were proposed in this pull request? Update Twitter Chill to 0.9.5. ### Why are the changes needed? Primarily, Scala 2.13 support for later. Other changes from 0.9.3 are apparently just minor fixes and improvements: https://github.com/twitter/chill/releases ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests Closes #27227 from srowen/SPARK-29290. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 4 ++-- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 4 ++-- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 4 ++-- pom.xml | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index af702e8283826..c0039a2cc4b15 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -29,8 +29,8 @@ bonecp/0.8.0.RELEASE//bonecp-0.8.0.RELEASE.jar breeze-macros_2.12/1.0//breeze-macros_2.12-1.0.jar breeze_2.12/1.0//breeze_2.12-1.0.jar cats-kernel_2.12/2.0.0-M4//cats-kernel_2.12-2.0.0-M4.jar -chill-java/0.9.3//chill-java-0.9.3.jar -chill_2.12/0.9.3//chill_2.12-0.9.3.jar +chill-java/0.9.5//chill-java-0.9.5.jar +chill_2.12/0.9.5//chill_2.12-0.9.5.jar commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar commons-cli/1.2//commons-cli-1.2.jar commons-codec/1.10//commons-codec-1.10.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 9bfd8014de487..f3e7cf6a4f8e1 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -27,8 +27,8 @@ bonecp/0.8.0.RELEASE//bonecp-0.8.0.RELEASE.jar breeze-macros_2.12/1.0//breeze-macros_2.12-1.0.jar breeze_2.12/1.0//breeze_2.12-1.0.jar cats-kernel_2.12/2.0.0-M4//cats-kernel_2.12-2.0.0-M4.jar -chill-java/0.9.3//chill-java-0.9.3.jar -chill_2.12/0.9.3//chill_2.12-0.9.3.jar +chill-java/0.9.5//chill-java-0.9.5.jar +chill_2.12/0.9.5//chill_2.12-0.9.5.jar commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar commons-cli/1.2//commons-cli-1.2.jar commons-codec/1.10//commons-codec-1.10.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index da2020bbb7171..cffb29836ce6e 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -24,8 +24,8 @@ bonecp/0.8.0.RELEASE//bonecp-0.8.0.RELEASE.jar breeze-macros_2.12/1.0//breeze-macros_2.12-1.0.jar breeze_2.12/1.0//breeze_2.12-1.0.jar cats-kernel_2.12/2.0.0-M4//cats-kernel_2.12-2.0.0-M4.jar -chill-java/0.9.3//chill-java-0.9.3.jar -chill_2.12/0.9.3//chill_2.12-0.9.3.jar +chill-java/0.9.5//chill-java-0.9.5.jar +chill_2.12/0.9.5//chill_2.12-0.9.5.jar commons-beanutils/1.9.4//commons-beanutils-1.9.4.jar commons-cli/1.2//commons-cli-1.2.jar commons-codec/1.10//commons-codec-1.10.jar diff --git a/pom.xml b/pom.xml index 33a219f764465..66297b0d10ffc 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,7 @@ 1.6.0 9.4.18.v20190429 3.1.0 - 0.9.3 + 0.9.5 2.4.0 2.0.8 4.1.1 From 775fae4640ba8c7e7d54f423fd89553206193b4d Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 19 Jan 2020 19:05:30 -0800 Subject: [PATCH 1158/1387] [SPARK-30486][BUILD] Bump lz4-java version to 1.7.1 ### What changes were proposed in this pull request? This pr intends to upgrade lz4-java from 1.7.0 to 1.7.1. ### Why are the changes needed? This release includes a bug fix for older macOS. You can see the link below for the changes; https://github.com/lz4/lz4-java/blob/master/CHANGES.md#171 ### Does this PR introduce any user-facing change? ### How was this patch tested? Existing tests. Closes #27271 from maropu/SPARK-30486. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 2 +- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index c0039a2cc4b15..c38379097157c 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -145,7 +145,7 @@ libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar log4j/1.2.17//log4j-1.2.17.jar logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar -lz4-java/1.7.0//lz4-java-1.7.0.jar +lz4-java/1.7.1//lz4-java-1.7.1.jar machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar mesos/1.4.0/shaded-protobuf/mesos-1.4.0-shaded-protobuf.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index f3e7cf6a4f8e1..4af1c0728e47f 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -161,7 +161,7 @@ libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar log4j/1.2.17//log4j-1.2.17.jar logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar -lz4-java/1.7.0//lz4-java-1.7.0.jar +lz4-java/1.7.1//lz4-java-1.7.1.jar machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar mesos/1.4.0/shaded-protobuf/mesos-1.4.0-shaded-protobuf.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index cffb29836ce6e..5aa0f0d4c5a32 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -173,7 +173,7 @@ libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar log4j/1.2.17//log4j-1.2.17.jar logging-interceptor/3.12.6//logging-interceptor-3.12.6.jar -lz4-java/1.7.0//lz4-java-1.7.0.jar +lz4-java/1.7.1//lz4-java-1.7.1.jar machinist_2.12/0.6.8//machinist_2.12-0.6.8.jar macro-compat_2.12/1.1.1//macro-compat_2.12-1.1.1.jar mesos/1.4.0/shaded-protobuf/mesos-1.4.0-shaded-protobuf.jar diff --git a/pom.xml b/pom.xml index 66297b0d10ffc..ef8fb709bbb59 100644 --- a/pom.xml +++ b/pom.xml @@ -639,7 +639,7 @@ org.lz4 lz4-java - 1.7.0 + 1.7.1 com.github.luben From d50f8df929e7d2a0f81f91bbcfa20cb340d86f62 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 19 Jan 2020 19:12:19 -0800 Subject: [PATCH 1159/1387] [SPARK-30413][SQL] Avoid WrappedArray roundtrip in GenericArrayData constructor, plus related optimization in ParquetMapConverter ### What changes were proposed in this pull request? This PR implements a tiny performance optimization for a `GenericArrayData` constructor, avoiding an unnecessary roundtrip through `WrappedArray` when the provided value is already an array of objects. It also fixes a related performance problem in `ParquetRowConverter`. ### Why are the changes needed? `GenericArrayData` has a `this(seqOrArray: Any)` constructor, which was originally added in #13138 for use in `RowEncoder` (where we may not know concrete types until runtime) but is also called (perhaps unintentionally) in a few other code paths. In this constructor's existing implementation, a call to `new WrappedArray(Array[Object](""))` is dispatched to the `this(seqOrArray: Any)` constructor, where we then call `this(array.toSeq)`: this wraps the provided array into a `WrappedArray`, which is subsequently unwrapped in a `this(seq.toArray)` call. For an interactive example, see https://scastie.scala-lang.org/7jOHydbNTaGSU677FWA8nA This PR changes the `this(seqOrArray: Any)` constructor so that it calls the primary `this(array: Array[Any])` constructor, allowing us to save a `.toSeq.toArray` call; this comes at the cost of one additional `case` in the `match` statement (but I believe this has a negligible performance impact relative to the other savings). As code cleanup, I also reverted the JVM 1.7 workaround from #14271. I also fixed a related performance problem in `ParquetRowConverter`: previously, this code called `ArrayBasedMapData.apply` which, in turn, called the `this(Any)` constructor for `GenericArrayData`: this PR's micro-benchmarks show that this is _significantly_ slower than calling the `this(Array[Any])` constructor (and I also observed time spent here during other Parquet scan benchmarking work). To fix this performance problem, I replaced the call to the `ArrayBasedMapData.apply` method with direct calls to the `ArrayBasedMapData` and `GenericArrayData` constructors. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? I tested this by running code in a debugger and by running microbenchmarks (which I've added to a new `GenericArrayDataBenchmark` in this PR): - With JDK8 benchmarks: this PR's changes more than double the performance of calls to the `this(Any)` constructor. Even after improvements, however, calls to the `this(Array[Any])` constructor are still ~60x faster than calls to `this(Any)` when passing a non-primitive array (thereby motivating this patch's other change in `ParquetRowConverter`). - With JDK11 benchmarks: the changes more-or-less completely eliminate the performance penalty associated with the `this(Any)` constructor. Closes #27088 from JoshRosen/joshrosen/GenericArrayData-optimization. Authored-by: Josh Rosen Signed-off-by: Dongjoon Hyun --- ...enericArrayDataBenchmark-jdk11-results.txt | 10 ++ .../GenericArrayDataBenchmark-results.txt | 10 ++ .../sql/catalyst/util/GenericArrayData.scala | 16 +--- .../util/GenericArrayDataBenchmark.scala | 93 +++++++++++++++++++ .../parquet/ParquetRowConverter.scala | 5 +- 5 files changed, 122 insertions(+), 12 deletions(-) create mode 100644 sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk11-results.txt create mode 100644 sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/GenericArrayDataBenchmark.scala diff --git a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk11-results.txt b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk11-results.txt new file mode 100644 index 0000000000000..6a5a9b781e894 --- /dev/null +++ b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-jdk11-results.txt @@ -0,0 +1,10 @@ +OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.14.6 +Intel(R) Core(TM) i5-8210Y CPU @ 1.60GHz +constructor: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +arrayOfAny 6 7 1 1770.9 0.6 1.0X +arrayOfAnyAsObject 6 7 2 1709.3 0.6 1.0X +arrayOfAnyAsSeq 5 6 2 2195.5 0.5 1.2X +arrayOfInt 452 469 13 22.1 45.2 0.0X +arrayOfIntAsObject 678 690 11 14.7 67.8 0.0X + diff --git a/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt new file mode 100644 index 0000000000000..02971749662f5 --- /dev/null +++ b/sql/catalyst/benchmarks/GenericArrayDataBenchmark-results.txt @@ -0,0 +1,10 @@ +Java HotSpot(TM) 64-Bit Server VM 1.8.0_191-b12 on Mac OS X 10.14.6 +Intel(R) Core(TM) i5-8210Y CPU @ 1.60GHz +constructor: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +arrayOfAny 7 8 2 1471.6 0.7 1.0X +arrayOfAnyAsObject 197 207 9 50.7 19.7 0.0X +arrayOfAnyAsSeq 25 27 2 398.0 2.5 0.3X +arrayOfInt 613 630 15 16.3 61.3 0.0X +arrayOfIntAsObject 866 872 8 11.5 86.6 0.0X + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala index 83ad08d8e1758..1f88a700847de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GenericArrayData.scala @@ -23,16 +23,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.{DataType, Decimal} import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -private object GenericArrayData { - - // SPARK-16634: Workaround for JVM bug present in some 1.7 versions. - def anyToSeq(seqOrArray: Any): Seq[Any] = seqOrArray match { - case seq: Seq[Any] => seq - case array: Array[_] => array.toSeq - } - -} - class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(seq: Seq[Any]) = this(seq.toArray) @@ -47,7 +37,11 @@ class GenericArrayData(val array: Array[Any]) extends ArrayData { def this(primitiveArray: Array[Byte]) = this(primitiveArray.toSeq) def this(primitiveArray: Array[Boolean]) = this(primitiveArray.toSeq) - def this(seqOrArray: Any) = this(GenericArrayData.anyToSeq(seqOrArray)) + def this(seqOrArray: Any) = this(seqOrArray match { + case seq: Seq[Any] => seq.toArray + case array: Array[Any] => array // array of objects, so no need to convert + case array: Array[_] => array.toSeq.toArray[Any] // array of primitives, so box them + }) override def copy(): ArrayData = { val newValues = new Array[Any](array.length) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/GenericArrayDataBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/GenericArrayDataBenchmark.scala new file mode 100644 index 0000000000000..3ad045f29c07d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/GenericArrayDataBenchmark.scala @@ -0,0 +1,93 @@ +/* + * 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.util + +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} + +/** + * Benchmark for [[GenericArrayData]]. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "catalyst/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "catalyst/test:runMain " + * Results will be written to "benchmarks/GenericArrayDataBenchmark-results.txt". + * }}} + */ +object GenericArrayDataBenchmark extends BenchmarkBase { + + // Benchmarks of GenericArrayData's constructors (see SPARK-30413): + def constructorBenchmark(): Unit = { + val valuesPerIteration: Long = 1000 * 1000 * 10 + val arraySize = 10 + val benchmark = new Benchmark("constructor", valuesPerIteration, output = output) + + benchmark.addCase("arrayOfAny") { _ => + val arr: Array[Any] = new Array[Any](arraySize) + var n = 0 + while (n < valuesPerIteration) { + new GenericArrayData(arr) + n += 1 + } + } + + benchmark.addCase("arrayOfAnyAsObject") { _ => + val arr: Object = new Array[Any](arraySize) + var n = 0 + while (n < valuesPerIteration) { + new GenericArrayData(arr) + n += 1 + } + } + + benchmark.addCase("arrayOfAnyAsSeq") { _ => + val arr: Seq[Any] = new Array[Any](arraySize) + var n = 0 + while (n < valuesPerIteration) { + new GenericArrayData(arr) + n += 1 + } + } + + benchmark.addCase("arrayOfInt") { _ => + val arr: Array[Int] = new Array[Int](arraySize) + var n = 0 + while (n < valuesPerIteration) { + new GenericArrayData(arr) + n += 1 + } + } + + benchmark.addCase("arrayOfIntAsObject") { _ => + val arr: Object = new Array[Int](arraySize) + var n = 0 + while (n < valuesPerIteration) { + new GenericArrayData(arr) + n += 1 + } + } + + benchmark.run() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + constructorBenchmark() + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index c4a2df6642519..98ac2ecd2955c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -591,7 +591,10 @@ private[parquet] class ParquetRowConverter( // The parquet map may contains null or duplicated map keys. When it happens, the behavior is // undefined. // TODO (SPARK-26174): disallow it with a config. - updater.set(ArrayBasedMapData(currentKeys.toArray, currentValues.toArray)) + updater.set( + new ArrayBasedMapData( + new GenericArrayData(currentKeys.toArray), + new GenericArrayData(currentValues.toArray))) } override def start(): Unit = { From 4806cc5bd19c714501187a7c3db48c278c79a1bf Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 20 Jan 2020 12:17:37 +0800 Subject: [PATCH 1160/1387] [SPARK-30547][SQL] Add unstable annotation to the CalendarInterval class MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? `CalendarInterval` is maintained as a private class but might be used in a public way by users e.g. ```scala scala> spark.udf.register("getIntervalMonth", (_:org.apache.spark.unsafe.types.CalendarInterval).months) scala> sql("select interval 2 month 1 day a").selectExpr("getIntervalMonth(a)").show +-------------------+ |getIntervalMonth(a)| +-------------------+ | 2| +-------------------+ ``` And it exists since 1.5.0, now we go to the 3.x era,may be it's time to make it public ### Why are the changes needed? make the interval more future-proofing ### Does this PR introduce any user-facing change? doc change ### How was this patch tested? add ut. Closes #27258 from yaooqinn/SPARK-30547. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../spark/unsafe/types/CalendarInterval.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index bbfccc3ff2a21..83d02ddba48a6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -17,6 +17,8 @@ package org.apache.spark.unsafe.types; +import org.apache.spark.annotation.Unstable; + import java.io.Serializable; import java.math.BigDecimal; import java.time.Duration; @@ -27,8 +29,21 @@ import static org.apache.spark.sql.catalyst.util.DateTimeConstants.*; /** - * The internal representation of interval type. + * The class representing calendar intervals. The calendar interval is stored internally in + * three components: + *
        + *
      • an integer value representing the number of `months` in this interval,
      • + *
      • an integer value representing the number of `days` in this interval,
      • + *
      • a long value representing the number of `microseconds` in this interval.
      • + *
      + * + * The `months` and `days` are not units of time with a constant length (unlike hours, seconds), so + * they are two separated fields from microseconds. One month may be equal to 28, 29, 30 or 31 days + * and one day may be equal to 23, 24 or 25 hours (daylight saving). + * + * @since 1.5.0 */ +@Unstable public final class CalendarInterval implements Serializable { public final int months; public final int days; From 00039cc482643a4f0a511813970b4727005da1b3 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 20 Jan 2020 13:59:22 +0900 Subject: [PATCH 1161/1387] [SPARK-30554][SQL] Return `Iterable` from `FailureSafeParser.rawParser` ### What changes were proposed in this pull request? Changed signature of `rawParser` passed to `FailureSafeParser`. I propose to change return type from `Seq` to `Iterable`. I took `Iterable` to easier port the changes on Scala collections 2.13. Also, I replaced `Seq` by `Option` in CSV datasource - `UnivocityParser`, and in JSON parser exception one place in the case when specified schema is `StructType`, and JSON input is an array. ### Why are the changes needed? `Seq` is unnecessary requirement for return type from rawParser which may not have multiple rows per input like CSV datasource. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites `JsonSuite`, `UnivocityParserSuite`, `JsonFunctionsSuite`, `JsonExpressionsSuite`, `CsvSuite`, and `CsvFunctionsSuite`. Closes #27264 from MaxGekk/failuresafe-parser-seq. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../sql/catalyst/csv/UnivocityParser.scala | 16 +++++----- .../sql/catalyst/json/JacksonParser.scala | 30 +++++++++---------- .../sql/catalyst/util/FailureSafeParser.scala | 2 +- .../catalyst/csv/UnivocityParserSuite.scala | 18 +++++------ 4 files changed, 33 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 32ff2c90bfa28..22fd69b2ce0da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -78,11 +78,11 @@ class UnivocityParser( new CsvParser(parserSetting) } - // Pre-allocated Seq to avoid the overhead of the seq builder. - private val requiredRow = Seq(new GenericInternalRow(requiredSchema.length)) + // Pre-allocated Some to avoid the overhead of building Some per each-row. + private val requiredRow = Some(new GenericInternalRow(requiredSchema.length)) // Pre-allocated empty sequence returned when the parsed row cannot pass filters. - // We preallocate it avoid unnecessary invokes of the seq builder. - private val noRows = Seq.empty[InternalRow] + // We preallocate it avoid unnecessary allocations. + private val noRows = None private val timestampFormatter = TimestampFormatter( options.timestampFormat, @@ -206,7 +206,7 @@ class UnivocityParser( private val doParse = if (options.columnPruning && requiredSchema.isEmpty) { // If `columnPruning` enabled and partition attributes scanned only, // `schema` gets empty. - (_: String) => Seq(InternalRow.empty) + (_: String) => Some(InternalRow.empty) } else { // parse if the columnPruning is disabled or requiredSchema is nonEmpty (input: String) => convert(tokenizer.parseLine(input)) @@ -216,7 +216,7 @@ class UnivocityParser( * Parses a single CSV string and turns it into either one resulting row or no row (if the * the record is malformed). */ - def parse(input: String): Seq[InternalRow] = doParse(input) + def parse(input: String): Option[InternalRow] = doParse(input) private val getToken = if (options.columnPruning) { (tokens: Array[String], index: Int) => tokens(index) @@ -224,7 +224,7 @@ class UnivocityParser( (tokens: Array[String], index: Int) => tokens(tokenIndexArr(index)) } - private def convert(tokens: Array[String]): Seq[InternalRow] = { + private def convert(tokens: Array[String]): Option[InternalRow] = { if (tokens == null) { throw BadRecordException( () => getCurrentInput, @@ -251,7 +251,7 @@ class UnivocityParser( // 1. Convert the tokens that correspond to the required schema. // 2. Apply the pushdown filters to `requiredRow`. var i = 0 - val row = requiredRow.head + val row = requiredRow.get var skipRow = false while (i < requiredSchema.length) { try { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index ead26665bd6ea..4824b0c860cb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -68,7 +68,7 @@ class JacksonParser( * to a value according to a desired schema. This is a wrapper for the method * `makeConverter()` to handle a row wrapped with an array. */ - private def makeRootConverter(dt: DataType): JsonParser => Seq[InternalRow] = { + private def makeRootConverter(dt: DataType): JsonParser => Iterable[InternalRow] = { dt match { case st: StructType => makeStructRootConverter(st) case mt: MapType => makeMapRootConverter(mt) @@ -76,11 +76,11 @@ class JacksonParser( } } - private def makeStructRootConverter(st: StructType): JsonParser => Seq[InternalRow] = { + private def makeStructRootConverter(st: StructType): JsonParser => Iterable[InternalRow] = { val elementConverter = makeConverter(st) val fieldConverters = st.map(_.dataType).map(makeConverter).toArray - (parser: JsonParser) => parseJsonToken[Seq[InternalRow]](parser, st) { - case START_OBJECT => convertObject(parser, st, fieldConverters) :: Nil + (parser: JsonParser) => parseJsonToken[Iterable[InternalRow]](parser, st) { + case START_OBJECT => Some(convertObject(parser, st, fieldConverters)) // SPARK-3308: support reading top level JSON arrays and take every element // in such an array as a row // @@ -99,26 +99,26 @@ class JacksonParser( // Here, as we support reading top level JSON arrays and take every element // in such an array as a row, this case is possible. if (array.numElements() == 0) { - Nil + Array.empty[InternalRow] } else { - array.toArray[InternalRow](schema).toSeq + array.toArray[InternalRow](schema) } case START_ARRAY => throw new RuntimeException("Parsing JSON arrays as structs is forbidden.") } } - private def makeMapRootConverter(mt: MapType): JsonParser => Seq[InternalRow] = { + private def makeMapRootConverter(mt: MapType): JsonParser => Iterable[InternalRow] = { val fieldConverter = makeConverter(mt.valueType) - (parser: JsonParser) => parseJsonToken[Seq[InternalRow]](parser, mt) { - case START_OBJECT => Seq(InternalRow(convertMap(parser, fieldConverter))) + (parser: JsonParser) => parseJsonToken[Iterable[InternalRow]](parser, mt) { + case START_OBJECT => Some(InternalRow(convertMap(parser, fieldConverter))) } } - private def makeArrayRootConverter(at: ArrayType): JsonParser => Seq[InternalRow] = { + private def makeArrayRootConverter(at: ArrayType): JsonParser => Iterable[InternalRow] = { val elemConverter = makeConverter(at.elementType) - (parser: JsonParser) => parseJsonToken[Seq[InternalRow]](parser, at) { - case START_ARRAY => Seq(InternalRow(convertArray(parser, elemConverter))) + (parser: JsonParser) => parseJsonToken[Iterable[InternalRow]](parser, at) { + case START_ARRAY => Some(InternalRow(convertArray(parser, elemConverter))) case START_OBJECT if at.elementType.isInstanceOf[StructType] => // This handles the case when an input JSON object is a structure but // the specified schema is an array of structures. In that case, the input JSON is @@ -140,7 +140,7 @@ class JacksonParser( // val st = at.elementType.asInstanceOf[StructType] val fieldConverters = st.map(_.dataType).map(makeConverter).toArray - Seq(InternalRow(new GenericArrayData(Seq(convertObject(parser, st, fieldConverters))))) + Some(InternalRow(new GenericArrayData(Seq(convertObject(parser, st, fieldConverters))))) } } @@ -395,13 +395,13 @@ class JacksonParser( def parse[T]( record: T, createParser: (JsonFactory, T) => JsonParser, - recordLiteral: T => UTF8String): Seq[InternalRow] = { + recordLiteral: T => UTF8String): Iterable[InternalRow] = { try { Utils.tryWithResource(createParser(factory, record)) { parser => // a null first token is equivalent to testing for input.trim.isEmpty // but it works on any token stream and not just strings parser.nextToken() match { - case null => Nil + case null => None case _ => rootConverter.apply(parser) match { case null => throw new RuntimeException("Root converter returned null") case rows => rows diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala index 361c8b29db33d..bc5e9be324bb2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String class FailureSafeParser[IN]( - rawParser: IN => Seq[InternalRow], + rawParser: IN => Iterable[InternalRow], mode: ParseMode, schema: StructType, columnNameOfCorruptRecord: String) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index bd4b2529f8b92..77a2ca7e4a828 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -276,7 +276,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { dataSchema: StructType = StructType.fromDDL("i INTEGER, s STRING"), requiredSchema: StructType = StructType.fromDDL("i INTEGER"), filters: Seq[Filter], - expected: Seq[InternalRow]): Unit = { + expected: Option[InternalRow]): Unit = { Seq(false, true).foreach { columnPruning => val options = new CSVOptions(Map.empty[String, String], columnPruning, "GMT") val parser = new UnivocityParser(dataSchema, requiredSchema, options, filters) @@ -285,26 +285,26 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { } } - check(filters = Seq(), expected = Seq(InternalRow(1))) - check(filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow(1))) - check(filters = Seq(EqualTo("i", 2)), expected = Seq()) + check(filters = Seq(), expected = Some(InternalRow(1))) + check(filters = Seq(EqualTo("i", 1)), expected = Some(InternalRow(1))) + check(filters = Seq(EqualTo("i", 2)), expected = None) check( requiredSchema = StructType.fromDDL("s STRING"), filters = Seq(StringStartsWith("s", "b")), - expected = Seq()) + expected = None) check( requiredSchema = StructType.fromDDL("i INTEGER, s STRING"), filters = Seq(StringStartsWith("s", "a")), - expected = Seq(InternalRow(1, UTF8String.fromString("a")))) + expected = Some(InternalRow(1, UTF8String.fromString("a")))) check( input = "1,a,3.14", dataSchema = StructType.fromDDL("i INTEGER, s STRING, d DOUBLE"), requiredSchema = StructType.fromDDL("i INTEGER, d DOUBLE"), filters = Seq(EqualTo("d", 3.14)), - expected = Seq(InternalRow(1, 3.14))) + expected = Some(InternalRow(1, 3.14))) val errMsg = intercept[IllegalArgumentException] { - check(filters = Seq(EqualTo("invalid attr", 1)), expected = Seq()) + check(filters = Seq(EqualTo("invalid attr", 1)), expected = None) }.getMessage assert(errMsg.contains("invalid attr does not exist")) @@ -313,7 +313,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { dataSchema = new StructType(), requiredSchema = new StructType(), filters = Seq(EqualTo("i", 1)), - expected = Seq(InternalRow.empty)) + expected = Some(InternalRow.empty)) }.getMessage assert(errMsg2.contains("i does not exist")) } From ab048990e0bdffe16de6a6b9211f63933976651e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 20 Jan 2020 15:22:23 +0900 Subject: [PATCH 1162/1387] [SPARK-30558][SQL] Avoid rebuilding `AvroOptions` per each partition ### What changes were proposed in this pull request? In the PR, I propose move out creation of `AvroOption` from `AvroPartitionReaderFactory.buildReader`, and create it earlier in `AvroScan.createReaderFactory`. ### Why are the changes needed? - To avoid building `AvroOptions` from a map of Avro options and Hadoop conf per each partition. - If an instance of `AvroOptions` is built only once at the driver side, we could output warnings while parsing Avro options and don't worry about noisiness of the warnings. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By `AvroSuite` Closes #27272 from MaxGekk/avro-options-once-for-read. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- .../spark/sql/v2/avro/AvroPartitionReaderFactory.scala | 5 ++--- .../main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala | 4 +++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala index 0397d15aed924..8230dbaf8ea6c 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala @@ -46,7 +46,7 @@ import org.apache.spark.util.SerializableConfiguration * @param dataSchema Schema of AVRO files. * @param readDataSchema Required data schema of AVRO files. * @param partitionSchema Schema of partitions. - * @param options Options for parsing AVRO files. + * @param parsedOptions Options for parsing AVRO files. */ case class AvroPartitionReaderFactory( sqlConf: SQLConf, @@ -54,11 +54,10 @@ case class AvroPartitionReaderFactory( dataSchema: StructType, readDataSchema: StructType, partitionSchema: StructType, - options: Map[String, String]) extends FilePartitionReaderFactory with Logging { + parsedOptions: AvroOptions) extends FilePartitionReaderFactory with Logging { override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { val conf = broadcastedConf.value.value - val parsedOptions = new AvroOptions(options, conf) val userProvidedSchema = parsedOptions.schema.map(new Schema.Parser().parse) if (parsedOptions.ignoreExtension || partitionedFile.filePath.endsWith(".avro")) { diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala index bb840e69d99a3..d5a29124a276e 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.avro.AvroOptions import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex @@ -45,10 +46,11 @@ case class AvroScan( val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) val broadcastedConf = sparkSession.sparkContext.broadcast( new SerializableConfiguration(hadoopConf)) + val parsedOptions = new AvroOptions(caseSensitiveMap, hadoopConf) // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. AvroPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, caseSensitiveMap) + dataSchema, readDataSchema, readPartitionSchema, parsedOptions) } override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = From b5cb9abdd5ee286cc2b0a06cb5f3eac812922a31 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 20 Jan 2020 21:33:44 +0800 Subject: [PATCH 1163/1387] [SPARK-30535][SQL] Migrate ALTER TABLE commands to the new framework ### What changes were proposed in this pull request? Use the new framework to resolve the ALTER TABLE commands. This PR also refactors ALTER TABLE logical plans such that they extend a base class `AlterTable`. Each plan now implements `def changes: Seq[TableChange]` for any table change operations. Additionally, `UnresolvedV2Relation` and its usage is completely removed. ### Why are the changes needed? This is a part of effort to make the relation lookup behavior consistent: [SPARK-29900](https://issues.apache.org/jira/browse/SPARK-29900). ### Does this PR introduce any user-facing change? No ### How was this patch tested? Updated existing tests Closes #27243 from imback82/v2commands_newframework. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 25 +-- .../sql/catalyst/analysis/CheckAnalysis.scala | 42 +++-- .../catalyst/analysis/ResolveCatalogs.scala | 67 +------ .../sql/catalyst/analysis/unresolved.scala | 23 --- .../catalyst/analysis/v2ResolutionPlans.scala | 14 +- .../sql/catalyst/parser/AstBuilder.scala | 50 ++--- .../catalyst/plans/logical/statements.scala | 56 ------ .../catalyst/plans/logical/v2Commands.scala | 138 +++++++++++--- .../sql/connector/catalog/CatalogV2Util.scala | 14 +- .../sql/catalyst/parser/DDLParserSuite.scala | 90 ++++----- .../analysis/ResolveSessionCatalog.scala | 178 +++++------------- .../spark/sql/execution/command/tables.scala | 8 - .../datasources/v2/DataSourceV2Strategy.scala | 14 +- .../sql-tests/results/change-column.sql.out | 4 +- .../sql/connector/DataSourceV2SQLSuite.scala | 2 +- .../spark/sql/execution/SQLViewSuite.scala | 8 +- .../sql/execution/command/DDLSuite.scala | 5 +- .../command/PlanResolutionSuite.scala | 50 +++-- .../sql/hive/execution/HiveCommandSuite.scala | 2 +- 19 files changed, 326 insertions(+), 464 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5317af494d8e4..89cd8e9648b3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -755,12 +755,14 @@ class Analyzer( .map(view => i.copy(table = view)) .getOrElse(i) case u @ UnresolvedTable(ident) => - lookupTempView(ident).foreach { _ => - u.failAnalysis(s"${ident.quoted} is a temp view not table.") - } - u + lookupTempView(ident) + .map(_ => UnresolvedTableWithViewExists( + ResolvedView(ident.asIdentifier, isTempView = true))) + .getOrElse(u) case u @ UnresolvedTableOrView(ident) => - lookupTempView(ident).map(_ => ResolvedView(ident.asIdentifier)).getOrElse(u) + lookupTempView(ident) + .map(_ => ResolvedView(ident.asIdentifier, isTempView = true)) + .getOrElse(u) } def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = { @@ -814,14 +816,6 @@ class Analyzer( lookupV2Relation(u.multipartIdentifier) .map(v2Relation => i.copy(table = v2Relation)) .getOrElse(i) - - case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => - CatalogV2Util.loadRelation(u.catalog, u.tableName) - .map(rel => alter.copy(table = rel)) - .getOrElse(alter) - - case u: UnresolvedV2Relation => - CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) } /** @@ -888,8 +882,7 @@ class Analyzer( case u @ UnresolvedTable(identifier) => lookupTableOrView(identifier).map { - case v: ResolvedView => - u.failAnalysis(s"${v.identifier.quoted} is a view not table.") + case v: ResolvedView => UnresolvedTableWithViewExists(v) case table => table }.getOrElse(u) @@ -902,7 +895,7 @@ class Analyzer( case SessionCatalogAndIdentifier(catalog, ident) => CatalogV2Util.loadTable(catalog, ident).map { case v1Table: V1Table if v1Table.v1Table.tableType == CatalogTableType.VIEW => - ResolvedView(ident) + ResolvedView(ident, isTempView = false) case table => ResolvedTable(catalog.asTableCatalog, ident, table) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 1d44c84f3123f..65a38aacecd43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -25,7 +25,8 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnType} +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnPosition, UpdateColumnType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -86,6 +87,20 @@ trait CheckAnalysis extends PredicateHelper { } def checkAnalysis(plan: LogicalPlan): Unit = { + // Analysis that needs to be performed top down can be added here. + plan.foreach { + case p if p.analyzed => // Skip already analyzed sub-plans + + case alter: AlterTable => + alter.table match { + case u @ UnresolvedTableWithViewExists(view) if !view.isTempView => + u.failAnalysis("Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead") + case _ => + } + + case _ => // Analysis successful! + } + // We transform up and order the rules so as to catch the first possible failure instead // of the result of cascading resolution failures. plan.foreachUp { @@ -104,23 +119,13 @@ trait CheckAnalysis extends PredicateHelper { case u: UnresolvedRelation => u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") + case u: UnresolvedTableWithViewExists => + val viewKind = if (u.view.isTempView) { "temp view" } else { "view" } + u.failAnalysis(s"${u.view.identifier.quoted} is a $viewKind not a table.") + case InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) => failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") - case u: UnresolvedV2Relation if isView(u.originalNameParts) => - u.failAnalysis( - s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") - - case u: UnresolvedV2Relation => - u.failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") - - case AlterTable(_, _, u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => - u.failAnalysis( - s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") - - case AlterTable(_, _, u: UnresolvedV2Relation, _) => - failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") - case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. // If the arguments of the higher-order functions are resolved but the type check fails, @@ -419,8 +424,9 @@ trait CheckAnalysis extends PredicateHelper { case _ => } - case alter: AlterTable if alter.childrenResolved => - val table = alter.table + case alter: AlterTable + if alter.childrenResolved && alter.table.isInstanceOf[ResolvedTable] => + val table = alter.table.asInstanceOf[ResolvedTable].table def findField(operation: String, fieldName: Array[String]): StructField = { // include collections because structs nested in maps and arrays may be altered val field = table.schema.findNestedField(fieldName, includeCollections = true) @@ -469,6 +475,8 @@ trait CheckAnalysis extends PredicateHelper { throw new AnalysisException( s"Cannot change nullable column to non-nullable: $fieldName") } + case update: UpdateColumnPosition => + findField("update", update.fieldNames) case rename: RenameColumn => findField("rename", rename.fieldNames) case update: UpdateColumnComment => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 88a3c0a73a10b..a44877fc1b4dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog} /** * Resolves catalogs from the multi-part identifiers in SQL statements, and convert the statements @@ -32,71 +32,6 @@ class ResolveCatalogs(val catalogManager: CatalogManager) import org.apache.spark.sql.connector.catalog.CatalogV2Util._ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case AlterTableAddColumnsStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => - val changes = cols.map { col => - TableChange.addColumn( - col.name.toArray, - col.dataType, - col.nullable, - col.comment.orNull, - col.position.orNull) - } - createAlterTable(nameParts, catalog, tbl, changes) - - case a @ AlterTableAlterColumnStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => - val colName = a.column.toArray - val typeChange = a.dataType.map { newDataType => - TableChange.updateColumnType(colName, newDataType) - } - val nullabilityChange = a.nullable.map { nullable => - TableChange.updateColumnNullability(colName, nullable) - } - val commentChange = a.comment.map { newComment => - TableChange.updateColumnComment(colName, newComment) - } - val positionChange = a.position.map { newPosition => - TableChange.updateColumnPosition(colName, newPosition) - } - createAlterTable( - nameParts, - catalog, - tbl, - typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange) - - case AlterTableRenameColumnStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), col, newName) => - val changes = Seq(TableChange.renameColumn(col.toArray, newName)) - createAlterTable(nameParts, catalog, tbl, changes) - - case AlterTableDropColumnsStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => - val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) - createAlterTable(nameParts, catalog, tbl, changes) - - case AlterTableSetPropertiesStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), props) => - val changes = props.map { case (key, value) => - TableChange.setProperty(key, value) - }.toSeq - createAlterTable(nameParts, catalog, tbl, changes) - - // TODO: v2 `UNSET TBLPROPERTIES` should respect the ifExists flag. - case AlterTableUnsetPropertiesStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), keys, _) => - val changes = keys.map(key => TableChange.removeProperty(key)) - createAlterTable(nameParts, catalog, tbl, changes) - - case AlterTableSetLocationStatement( - nameParts @ NonSessionCatalogAndTable(catalog, tbl), partitionSpec, newLoc) => - if (partitionSpec.nonEmpty) { - throw new AnalysisException( - "ALTER TABLE SET LOCATION does not support partition for v2 tables.") - } - val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, newLoc)) - createAlterTable(nameParts, catalog, tbl, changes) - case AlterViewSetPropertiesStatement( NonSessionCatalogAndTable(catalog, tbl), props) => throw new AnalysisException( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 608f39c2d86fd..c7d977a3d4a82 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.parser.ParserUtils import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.quoteIdentifier -import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.types.{DataType, Metadata, StructType} /** @@ -60,28 +59,6 @@ object UnresolvedRelation { UnresolvedRelation(tableIdentifier.database.toSeq :+ tableIdentifier.table) } -/** - * A variant of [[UnresolvedRelation]] which can only be resolved to a v2 relation - * (`DataSourceV2Relation`), not v1 relation or temp view. - * - * @param originalNameParts the original table identifier name parts before catalog is resolved. - * @param catalog The catalog which the table should be looked up from. - * @param tableName The name of the table to look up. - */ -case class UnresolvedV2Relation( - originalNameParts: Seq[String], - catalog: TableCatalog, - tableName: Identifier) - extends LeafNode with NamedRelation { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - - override def name: String = originalNameParts.quoted - - override def output: Seq[Attribute] = Nil - - override lazy val resolved = false -} - /** * An inline table that has not been resolved yet. Once resolved, it is turned by the analyzer into * a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]]. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 239f987e97a76..c6300b0bb079d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces, Table, TableCatalog} /** @@ -41,6 +41,16 @@ case class UnresolvedTable(multipartIdentifier: Seq[String]) extends LeafNode { override def output: Seq[Attribute] = Nil } +/** + * Holds the resolved view. It is used in a scenario where table is expected but the identifier + * is resolved to a (temp) view. + */ +case class UnresolvedTableWithViewExists(view: ResolvedView) extends LeafNode { + override lazy val resolved: Boolean = false + + override def output: Seq[Attribute] = Nil +} + /** * Holds the name of a table or view that has yet to be looked up in a catalog. It will * be resolved to [[ResolvedTable]] or [[ResolvedView]] during analysis. @@ -71,6 +81,6 @@ case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: T */ // TODO: create a generic representation for temp view, v1 view and v2 view, after we add view // support to v2 catalog. For now we only need the identifier to fallback to v1 command. -case class ResolvedView(identifier: Identifier) extends LeafNode { +case class ResolvedView(identifier: Identifier, isTempView: Boolean) extends LeafNode { override def output: Seq[Attribute] = Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f744787c9082a..d09c53ed919cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2907,7 +2907,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableAddColumnsStatement]] command. + * Parse a [[AlterTableAddColumns]] command. * * For example: * {{{ @@ -2916,14 +2916,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * }}} */ override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) { - AlterTableAddColumnsStatement( - visitMultipartIdentifier(ctx.multipartIdentifier), + AlterTableAddColumns( + UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)), ctx.columns.qualifiedColTypeWithPosition.asScala.map(typedVisit[QualifiedColType]) ) } /** - * Parse a [[AlterTableRenameColumnStatement]] command. + * Parse a [[AlterTableRenameColumn]] command. * * For example: * {{{ @@ -2932,14 +2932,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitRenameTableColumn( ctx: RenameTableColumnContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenameColumnStatement( - visitMultipartIdentifier(ctx.table), + AlterTableRenameColumn( + UnresolvedTable(visitMultipartIdentifier(ctx.table)), ctx.from.parts.asScala.map(_.getText), ctx.to.getText) } /** - * Parse a [[AlterTableAlterColumnStatement]] command. + * Parse a [[AlterTableAlterColumn]] command. * * For example: * {{{ @@ -2956,8 +2956,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging s"ALTER TABLE table $verb COLUMN requires a TYPE or a COMMENT or a FIRST/AFTER", ctx) } - AlterTableAlterColumnStatement( - visitMultipartIdentifier(ctx.table), + AlterTableAlterColumn( + UnresolvedTable(visitMultipartIdentifier(ctx.table)), typedVisit[Seq[String]](ctx.column), dataType = Option(ctx.dataType).map(typedVisit[DataType]), nullable = None, @@ -2966,7 +2966,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableAlterColumnStatement]] command to change column nullability. + * Parse a [[AlterTableAlterColumn]] command to change column nullability. * * For example: * {{{ @@ -2980,8 +2980,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case SqlBaseParser.SET => false case SqlBaseParser.DROP => true } - AlterTableAlterColumnStatement( - visitMultipartIdentifier(ctx.table), + AlterTableAlterColumn( + UnresolvedTable(visitMultipartIdentifier(ctx.table)), typedVisit[Seq[String]](ctx.column), dataType = None, nullable = Some(nullable), @@ -2991,7 +2991,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableAlterColumnStatement]] command. This is Hive SQL syntax. + * Parse a [[AlterTableAlterColumn]] command. This is Hive SQL syntax. * * For example: * {{{ @@ -3014,8 +3014,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "please run ALTER COLUMN ... SET/DROP NOT NULL instead.") } - AlterTableAlterColumnStatement( - typedVisit[Seq[String]](ctx.table), + AlterTableAlterColumn( + UnresolvedTable(typedVisit[Seq[String]](ctx.table)), columnNameParts, dataType = Option(ctx.colType().dataType()).map(typedVisit[DataType]), nullable = None, @@ -3024,7 +3024,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableDropColumnsStatement]] command. + * Parse a [[AlterTableDropColumns]] command. * * For example: * {{{ @@ -3035,13 +3035,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitDropTableColumns( ctx: DropTableColumnsContext): LogicalPlan = withOrigin(ctx) { val columnsToDrop = ctx.columns.multipartIdentifier.asScala.map(typedVisit[Seq[String]]) - AlterTableDropColumnsStatement( - visitMultipartIdentifier(ctx.multipartIdentifier), + AlterTableDropColumns( + UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)), columnsToDrop) } /** - * Parse [[AlterViewSetPropertiesStatement]] or [[AlterTableSetPropertiesStatement]] commands. + * Parse [[AlterViewSetPropertiesStatement]] or [[AlterTableSetProperties]] commands. * * For example: * {{{ @@ -3057,12 +3057,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (ctx.VIEW != null) { AlterViewSetPropertiesStatement(identifier, cleanedTableProperties) } else { - AlterTableSetPropertiesStatement(identifier, cleanedTableProperties) + AlterTableSetProperties(UnresolvedTable(identifier), cleanedTableProperties) } } /** - * Parse [[AlterViewUnsetPropertiesStatement]] or [[AlterTableUnsetPropertiesStatement]] commands. + * Parse [[AlterViewUnsetPropertiesStatement]] or [[AlterTableUnsetProperties]] commands. * * For example: * {{{ @@ -3080,12 +3080,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (ctx.VIEW != null) { AlterViewUnsetPropertiesStatement(identifier, cleanedProperties, ifExists) } else { - AlterTableUnsetPropertiesStatement(identifier, cleanedProperties, ifExists) + AlterTableUnsetProperties(UnresolvedTable(identifier), cleanedProperties, ifExists) } } /** - * Create an [[AlterTableSetLocationStatement]] command. + * Create an [[AlterTableSetLocation]] command. * * For example: * {{{ @@ -3093,8 +3093,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * }}} */ override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetLocationStatement( - visitMultipartIdentifier(ctx.multipartIdentifier), + AlterTableSetLocation( + UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)), Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), visitLocationSpec(ctx.locationSpec)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 44f7b4143926d..2083a00cae0da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -149,62 +149,6 @@ case class QualifiedColType( comment: Option[String], position: Option[ColumnPosition]) -/** - * ALTER TABLE ... ADD COLUMNS command, as parsed from SQL. - */ -case class AlterTableAddColumnsStatement( - tableName: Seq[String], - columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement - -/** - * ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL. - */ -case class AlterTableAlterColumnStatement( - tableName: Seq[String], - column: Seq[String], - dataType: Option[DataType], - nullable: Option[Boolean], - comment: Option[String], - position: Option[ColumnPosition]) extends ParsedStatement - -/** - * ALTER TABLE ... RENAME COLUMN command, as parsed from SQL. - */ -case class AlterTableRenameColumnStatement( - tableName: Seq[String], - column: Seq[String], - newName: String) extends ParsedStatement - -/** - * ALTER TABLE ... DROP COLUMNS command, as parsed from SQL. - */ -case class AlterTableDropColumnsStatement( - tableName: Seq[String], - columnsToDrop: Seq[Seq[String]]) extends ParsedStatement - -/** - * ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterTableSetPropertiesStatement( - tableName: Seq[String], - properties: Map[String, String]) extends ParsedStatement - -/** - * ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL. - */ -case class AlterTableUnsetPropertiesStatement( - tableName: Seq[String], - propertyKeys: Seq[String], - ifExists: Boolean) extends ParsedStatement - -/** - * ALTER TABLE ... SET LOCATION command, as parsed from SQL. - */ -case class AlterTableSetLocationStatement( - tableName: Seq[String], - partitionSpec: Option[TablePartitionSpec], - location: String) extends ParsedStatement - /** * ALTER TABLE ... RECOVER PARTITIONS command, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index e1e7eac4cc085..d62869e912989 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Unevaluable} import org.apache.spark.sql.catalyst.plans.DescribeTableSchema import org.apache.spark.sql.connector.catalog._ -import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} +import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.{DataType, MetadataBuilder, StringType, StructType} @@ -394,37 +394,125 @@ case class DropTable( ifExists: Boolean) extends Command /** - * The logical plan of the ALTER TABLE command that works for v2 tables. + * The base class for ALTER TABLE commands that work for v2 tables. */ -case class AlterTable( - catalog: TableCatalog, - ident: Identifier, - table: NamedRelation, - changes: Seq[TableChange]) extends Command { - - override lazy val resolved: Boolean = table.resolved && { - changes.forall { - case add: AddColumn => - add.fieldNames match { - case Array(_) => - // a top-level field can always be added - true - case _ => - // the parent field must exist - table.schema.findNestedField(add.fieldNames.init, includeCollections = true).isDefined - } +abstract class AlterTable extends Command { + def table: LogicalPlan - case colChange: ColumnChange => - // the column that will be changed must exist - table.schema.findNestedField(colChange.fieldNames, includeCollections = true).isDefined + def changes: Seq[TableChange] + + override def children: Seq[LogicalPlan] = Seq(table) + + override lazy val resolved: Boolean = table.resolved +} - case _ => - // property changes require no resolution checks - true +/** + * The logical plan of the ALTER TABLE ... ADD COLUMNS command that works for v2 tables. + */ +case class AlterTableAddColumns( + table: LogicalPlan, + columnsToAdd: Seq[QualifiedColType]) extends AlterTable { + override lazy val changes: Seq[TableChange] = { + columnsToAdd.map { col => + TableChange.addColumn( + col.name.toArray, + col.dataType, + col.nullable, + col.comment.orNull, + col.position.orNull) } } } +/** + * The logical plan of the ALTER TABLE ... CHANGE COLUMN command that works for v2 tables. + */ +case class AlterTableAlterColumn( + table: LogicalPlan, + column: Seq[String], + dataType: Option[DataType], + nullable: Option[Boolean], + comment: Option[String], + position: Option[ColumnPosition]) extends AlterTable { + override lazy val changes: Seq[TableChange] = { + val colName = column.toArray + val typeChange = dataType.map { newDataType => + TableChange.updateColumnType(colName, newDataType) + } + val nullabilityChange = nullable.map { nullable => + TableChange.updateColumnNullability(colName, nullable) + } + val commentChange = comment.map { newComment => + TableChange.updateColumnComment(colName, newComment) + } + val positionChange = position.map { newPosition => + TableChange.updateColumnPosition(colName, newPosition) + } + typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange + } +} + +/** + * The logical plan of the ALTER TABLE ... RENAME COLUMN command that works for v2 tables. + */ +case class AlterTableRenameColumn( + table: LogicalPlan, + column: Seq[String], + newName: String) extends AlterTable { + override lazy val changes: Seq[TableChange] = { + Seq(TableChange.renameColumn(column.toArray, newName)) + } +} + +/** + * The logical plan of the ALTER TABLE ... DROP COLUMNS command that works for v2 tables. + */ +case class AlterTableDropColumns( + table: LogicalPlan, + columnsToDrop: Seq[Seq[String]]) extends AlterTable { + override lazy val changes: Seq[TableChange] = { + columnsToDrop.map(col => TableChange.deleteColumn(col.toArray)) + } +} + +/** + * The logical plan of the ALTER TABLE ... SET TBLPROPERTIES command that works for v2 tables. + */ +case class AlterTableSetProperties( + table: LogicalPlan, + properties: Map[String, String]) extends AlterTable { + override lazy val changes: Seq[TableChange] = { + properties.map { case (key, value) => + TableChange.setProperty(key, value) + }.toSeq + } +} + +/** + * The logical plan of the ALTER TABLE ... UNSET TBLPROPERTIES command that works for v2 tables. + */ +// TODO: v2 `UNSET TBLPROPERTIES` should respect the ifExists flag. +case class AlterTableUnsetProperties( + table: LogicalPlan, + propertyKeys: Seq[String], + ifExists: Boolean) extends AlterTable { + override lazy val changes: Seq[TableChange] = { + propertyKeys.map(key => TableChange.removeProperty(key)) + } +} + +/** + * The logical plan of the ALTER TABLE ... SET LOCATION command that works for v2 tables. + */ +case class AlterTableSetLocation( + table: LogicalPlan, + partitionSpec: Option[TablePartitionSpec], + location: String) extends AlterTable { + override lazy val changes: Seq[TableChange] = { + Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, location)) + } +} + /** * The logical plan of the ALTER TABLE RENAME command that works for v2 tables. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index f8fc4bc7ce23f..3ee22548ca3be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -22,8 +22,7 @@ import java.util.Collections import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} -import org.apache.spark.sql.catalyst.plans.logical.AlterTable +import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException} import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} @@ -276,17 +275,6 @@ private[sql] object CatalogV2Util { location.map(TableCatalog.PROP_LOCATION -> _) } - def createAlterTable( - originalNameParts: Seq[String], - catalog: CatalogPlugin, - tableName: Seq[String], - changes: Seq[TableChange]): AlterTable = { - val tableCatalog = catalog.asTableCatalog - val ident = tableName.asIdentifier - val unresolved = UnresolvedV2Relation(originalNameParts, tableCatalog, ident) - AlterTable(tableCatalog, ident, unresolved, changes) - } - def getTableProviderCatalog( provider: SupportsCatalogOptions, catalogManager: CatalogManager, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 47387fa18411e..5e45b57187f18 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -476,22 +476,22 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql1_table), - AlterTableSetPropertiesStatement( - Seq("table_name"), Map("test" -> "test", "comment" -> "new_comment"))) + AlterTableSetProperties( + UnresolvedTable(Seq("table_name")), Map("test" -> "test", "comment" -> "new_comment"))) comparePlans( parsePlan(sql2_table), - AlterTableUnsetPropertiesStatement( - Seq("table_name"), Seq("comment", "test"), ifExists = false)) + AlterTableUnsetProperties( + UnresolvedTable(Seq("table_name")), Seq("comment", "test"), ifExists = false)) comparePlans( parsePlan(sql3_table), - AlterTableUnsetPropertiesStatement( - Seq("table_name"), Seq("comment", "test"), ifExists = true)) + AlterTableUnsetProperties( + UnresolvedTable(Seq("table_name")), Seq("comment", "test"), ifExists = true)) } test("alter table: add column") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, None) ))) } @@ -499,7 +499,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add multiple columns") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS x int, y string"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, None), QualifiedColType(Seq("y"), StringType, true, None, None) ))) @@ -508,7 +508,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with COLUMNS") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS x int"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, None) ))) } @@ -516,7 +516,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with COLUMNS (...)") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS (x int)"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, None) ))) } @@ -524,7 +524,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with COLUMNS (...) and COMMENT") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS (x int COMMENT 'doc')"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, true, Some("doc"), None) ))) } @@ -532,7 +532,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add non-nullable column") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int NOT NULL"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, false, None, None) ))) } @@ -540,7 +540,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with COMMENT") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int COMMENT 'doc'"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, true, Some("doc"), None) ))) } @@ -548,13 +548,13 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with position") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int FIRST"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, Some(first())) ))) comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int AFTER y"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, Some(after("y"))) ))) } @@ -562,7 +562,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with nested column name") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc'"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x", "y", "z"), IntegerType, true, Some("doc"), None) ))) } @@ -570,7 +570,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add multiple columns with nested column name") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc', a.b string FIRST"), - AlterTableAddColumnsStatement(Seq("table_name"), Seq( + AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( QualifiedColType(Seq("x", "y", "z"), IntegerType, true, Some("doc"), None), QualifiedColType(Seq("a", "b"), StringType, true, None, Some(first())) ))) @@ -579,12 +579,12 @@ class DDLParserSuite extends AnalysisTest { test("alter table: set location") { comparePlans( parsePlan("ALTER TABLE a.b.c SET LOCATION 'new location'"), - AlterTableSetLocationStatement(Seq("a", "b", "c"), None, "new location")) + AlterTableSetLocation(UnresolvedTable(Seq("a", "b", "c")), None, "new location")) comparePlans( parsePlan("ALTER TABLE a.b.c PARTITION(ds='2017-06-10') SET LOCATION 'new location'"), - AlterTableSetLocationStatement( - Seq("a", "b", "c"), + AlterTableSetLocation( + UnresolvedTable(Seq("a", "b", "c")), Some(Map("ds" -> "2017-06-10")), "new location")) } @@ -592,8 +592,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: rename column") { comparePlans( parsePlan("ALTER TABLE table_name RENAME COLUMN a.b.c TO d"), - AlterTableRenameColumnStatement( - Seq("table_name"), + AlterTableRenameColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), "d")) } @@ -601,8 +601,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: update column type using ALTER") { comparePlans( parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint"), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), Some(LongType), None, @@ -613,8 +613,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: update column type") { comparePlans( parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c TYPE bigint"), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), Some(LongType), None, @@ -625,8 +625,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: update column comment") { comparePlans( parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c COMMENT 'new comment'"), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), None, None, @@ -637,8 +637,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: update column position") { comparePlans( parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c FIRST"), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), None, None, @@ -650,8 +650,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c " + "TYPE bigint COMMENT 'new comment' AFTER d"), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), Some(LongType), None, @@ -662,8 +662,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: SET/DROP NOT NULL") { comparePlans( parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c SET NOT NULL"), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), None, Some(false), @@ -672,8 +672,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c DROP NOT NULL"), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), None, Some(true), @@ -684,7 +684,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: drop column") { comparePlans( parsePlan("ALTER TABLE table_name DROP COLUMN a.b.c"), - AlterTableDropColumnsStatement(Seq("table_name"), Seq(Seq("a", "b", "c")))) + AlterTableDropColumns(UnresolvedTable(Seq("table_name")), Seq(Seq("a", "b", "c")))) } test("alter table: drop multiple columns") { @@ -692,8 +692,8 @@ class DDLParserSuite extends AnalysisTest { Seq(sql, sql.replace("COLUMN", "COLUMNS")).foreach { drop => comparePlans( parsePlan(drop), - AlterTableDropColumnsStatement( - Seq("table_name"), + AlterTableDropColumns( + UnresolvedTable(Seq("table_name")), Seq(Seq("x"), Seq("y"), Seq("a", "b", "c")))) } } @@ -705,8 +705,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql1), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), Some(IntegerType), None, @@ -715,8 +715,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql2), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), Some(IntegerType), None, @@ -725,8 +725,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql3), - AlterTableAlterColumnStatement( - Seq("table_name"), + AlterTableAlterColumn( + UnresolvedTable(Seq("table_name")), Seq("a", "b", "c"), Some(IntegerType), None, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 8b0d339dbb864..0aaf9d7e2e1ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} @@ -47,141 +47,63 @@ class ResolveSessionCatalog( import org.apache.spark.sql.connector.catalog.CatalogV2Util._ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case AlterTableAddColumnsStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => - loadTable(catalog, tbl.asIdentifier).collect { - case v1Table: V1Table => - cols.foreach { c => - assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand") - if (!c.nullable) { - throw new AnalysisException( - "ADD COLUMN with v1 tables cannot specify NOT NULL.") - } - } - AlterTableAddColumnsCommand(tbl.asTableIdentifier, cols.map(convertToStructField)) - }.getOrElse { - val changes = cols.map { col => - TableChange.addColumn( - col.name.toArray, - col.dataType, - col.nullable, - col.comment.orNull, - col.position.orNull) + case AlterTableAddColumns(ResolvedTable(_, ident, _: V1Table), cols) => + cols.foreach { c => + assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand") + if (!c.nullable) { + throw new AnalysisException( + "ADD COLUMN with v1 tables cannot specify NOT NULL.") } - createAlterTable(nameParts, catalog, tbl, changes) } + AlterTableAddColumnsCommand(ident.asTableIdentifier, cols.map(convertToStructField)) - case a @ AlterTableAlterColumnStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => - loadTable(catalog, tbl.asIdentifier).collect { - case v1Table: V1Table => - if (a.column.length > 1) { - throw new AnalysisException( - "ALTER COLUMN with qualified column is only supported with v2 tables.") - } - if (a.dataType.isEmpty) { - throw new AnalysisException( - "ALTER COLUMN with v1 tables must specify new data type.") - } - if (a.nullable.isDefined) { - throw new AnalysisException( - "ALTER COLUMN with v1 tables cannot specify NOT NULL.") - } - if (a.position.isDefined) { - throw new AnalysisException("" + - "ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.") - } - val builder = new MetadataBuilder - // Add comment to metadata - a.comment.map(c => builder.putString("comment", c)) - // Add Hive type string to metadata. - val cleanedDataType = HiveStringType.replaceCharType(a.dataType.get) - if (a.dataType.get != cleanedDataType) { - builder.putString(HIVE_TYPE_STRING, a.dataType.get.catalogString) - } - val newColumn = StructField( - a.column(0), - cleanedDataType, - nullable = true, - builder.build()) - AlterTableChangeColumnCommand(tbl.asTableIdentifier, a.column(0), newColumn) - }.getOrElse { - val colName = a.column.toArray - val typeChange = a.dataType.map { newDataType => - TableChange.updateColumnType(colName, newDataType) - } - val nullabilityChange = a.nullable.map { nullable => - TableChange.updateColumnNullability(colName, nullable) - } - val commentChange = a.comment.map { newComment => - TableChange.updateColumnComment(colName, newComment) - } - val positionChange = a.position.map { newPosition => - TableChange.updateColumnPosition(colName, newPosition) - } - createAlterTable( - nameParts, - catalog, - tbl, - typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange) + case a @ AlterTableAlterColumn(ResolvedTable(_, ident, _: V1Table), _, _, _, _, _) => + if (a.column.length > 1) { + throw new AnalysisException( + "ALTER COLUMN with qualified column is only supported with v2 tables.") } - - case AlterTableRenameColumnStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), col, newName) => - loadTable(catalog, tbl.asIdentifier).collect { - case v1Table: V1Table => - throw new AnalysisException("RENAME COLUMN is only supported with v2 tables.") - }.getOrElse { - val changes = Seq(TableChange.renameColumn(col.toArray, newName)) - createAlterTable(nameParts, catalog, tbl, changes) + if (a.dataType.isEmpty) { + throw new AnalysisException( + "ALTER COLUMN with v1 tables must specify new data type.") } - - case AlterTableDropColumnsStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => - loadTable(catalog, tbl.asIdentifier).collect { - case v1Table: V1Table => - throw new AnalysisException("DROP COLUMN is only supported with v2 tables.") - }.getOrElse { - val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) - createAlterTable(nameParts, catalog, tbl, changes) + if (a.nullable.isDefined) { + throw new AnalysisException( + "ALTER COLUMN with v1 tables cannot specify NOT NULL.") } - - case AlterTableSetPropertiesStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), props) => - loadTable(catalog, tbl.asIdentifier).collect { - case v1Table: V1Table => - AlterTableSetPropertiesCommand(tbl.asTableIdentifier, props, isView = false) - }.getOrElse { - val changes = props.map { case (key, value) => - TableChange.setProperty(key, value) - }.toSeq - createAlterTable(nameParts, catalog, tbl, changes) + if (a.position.isDefined) { + throw new AnalysisException("" + + "ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.") } - - case AlterTableUnsetPropertiesStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), keys, ifExists) => - loadTable(catalog, tbl.asIdentifier).collect { - case v1Table: V1Table => - AlterTableUnsetPropertiesCommand( - tbl.asTableIdentifier, keys, ifExists, isView = false) - }.getOrElse { - val changes = keys.map(key => TableChange.removeProperty(key)) - createAlterTable(nameParts, catalog, tbl, changes) + val builder = new MetadataBuilder + // Add comment to metadata + a.comment.map(c => builder.putString("comment", c)) + // Add Hive type string to metadata. + val cleanedDataType = HiveStringType.replaceCharType(a.dataType.get) + if (a.dataType.get != cleanedDataType) { + builder.putString(HIVE_TYPE_STRING, a.dataType.get.catalogString) } + val newColumn = StructField( + a.column(0), + cleanedDataType, + nullable = true, + builder.build()) + AlterTableChangeColumnCommand(ident.asTableIdentifier, a.column(0), newColumn) - case AlterTableSetLocationStatement( - nameParts @ SessionCatalogAndTable(catalog, tbl), partitionSpec, newLoc) => - loadTable(catalog, tbl.asIdentifier).collect { - case v1Table: V1Table => - AlterTableSetLocationCommand(tbl.asTableIdentifier, partitionSpec, newLoc) - }.getOrElse { - if (partitionSpec.nonEmpty) { - throw new AnalysisException( - "ALTER TABLE SET LOCATION does not support partition for v2 tables.") - } - val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, newLoc)) - createAlterTable(nameParts, catalog, tbl, changes) - } + case AlterTableRenameColumn(ResolvedTable(_, _, _: V1Table), _, _) => + throw new AnalysisException("RENAME COLUMN is only supported with v2 tables.") + + case AlterTableDropColumns(ResolvedTable(_, _, _: V1Table), _) => + throw new AnalysisException("DROP COLUMN is only supported with v2 tables.") + + case AlterTableSetProperties(ResolvedTable(_, ident, _: V1Table), props) => + AlterTableSetPropertiesCommand(ident.asTableIdentifier, props, isView = false) + + case AlterTableUnsetProperties(ResolvedTable(_, ident, _: V1Table), keys, ifExists) => + AlterTableUnsetPropertiesCommand(ident.asTableIdentifier, keys, ifExists, isView = false) + + case AlterTableSetLocation( + ResolvedTable(_, ident, _: V1Table), partitionSpec, newLoc) => + AlterTableSetLocationCommand(ident.asTableIdentifier, partitionSpec, newLoc) // ALTER VIEW should always use v1 command if the resolved catalog is session catalog. case AlterViewSetPropertiesStatement(SessionCatalogAndTable(_, tbl), props) => @@ -218,7 +140,7 @@ class ResolveSessionCatalog( DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. - case DescribeRelation(ResolvedView(ident), partitionSpec, isExtended) => + case DescribeRelation(ResolvedView(ident, _), partitionSpec, isExtended) => DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) case DescribeColumnStatement( 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 a92fbdf25975b..ab47d640f4705 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 @@ -257,14 +257,6 @@ case class AlterTableAddColumnsCommand( table: TableIdentifier): CatalogTable = { val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) - if (catalogTable.tableType == CatalogTableType.VIEW) { - throw new AnalysisException( - s""" - |ALTER ADD COLUMNS does not support views. - |You must drop and re-create the views for adding the new columns. Views: $table - """.stripMargin) - } - if (DDLUtils.isDatasourceTable(catalogTable)) { DataSource.lookupDataSource(catalogTable.provider.get, conf). getConstructor().newInstance() match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index d848e34648667..7169a437359a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -235,8 +235,18 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil - case AlterTable(catalog, ident, _, changes) => - AlterTableExec(catalog, ident, changes) :: Nil + case a @ AlterTableSetLocation(r: ResolvedTable, partitionSpec, _) => + if (partitionSpec.nonEmpty) { + throw new AnalysisException( + "ALTER TABLE SET LOCATION does not support partition for v2 tables.") + } + AlterTableExec(r.catalog, r.identifier, a.changes) :: Nil + + case a: AlterTable => + a.table match { + case r: ResolvedTable => AlterTableExec(r.catalog, r.identifier, a.changes) :: Nil + case _ => Nil + } case RenameTable(catalog, oldIdent, newIdent) => RenameTableExec(catalog, oldIdent, newIdent) :: Nil diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 82326346b361c..bb5b4ae84d3b7 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -195,7 +195,7 @@ ALTER TABLE temp_view CHANGE a TYPE INT COMMENT 'this is column a' struct<> -- !query 20 output org.apache.spark.sql.AnalysisException -Invalid command: 'temp_view' is a view not a table.; line 1 pos 0 +temp_view is a temp view not a table.; line 1 pos 0 -- !query 21 @@ -212,7 +212,7 @@ ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT COMMENT 'this is colu struct<> -- !query 22 output org.apache.spark.sql.AnalysisException -Invalid command: 'global_temp.global_temp_view' is a view not a table.; line 1 pos 0 +global_temp.global_temp_view is a temp view not a table.; line 1 pos 0 -- !query 23 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 4ee8a6803ea5e..4c5b1d95b12da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2194,7 +2194,7 @@ class DataSourceV2SQLSuite withTempView("v") { sql("create global temp view v as select 1") val e = intercept[AnalysisException](sql("COMMENT ON TABLE global_temp.v IS NULL")) - assert(e.getMessage.contains("global_temp.v is a temp view not table.")) + assert(e.getMessage.contains("global_temp.v is a temp view not a table.")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 9a393f19ce9bb..4ed2506b35a80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -145,13 +145,13 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { // For v2 ALTER TABLE statements, we have better error message saying view is not supported. assertAnalysisError( s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'", - s"'$viewName' is a view not a table") + s"$viewName is a temp view not a table") - // For the following v2 ALERT TABLE statements, unsupported operations are checked first - // before resolving the relations. + // For the following v2 ALERT TABLE statements, relations are first resolved before + // unsupported operations are checked. assertAnalysisError( s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'", - "ALTER TABLE SET LOCATION does not support partition for v2 tables") + s"$viewName is a temp view not a table") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 11c6487e25e90..64c85c19ba489 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2779,7 +2779,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)") } - assert(e.message.contains("'tmp_v' is a view not a table")) + assert(e.message.contains("tmp_v is a temp view not a table")) } } @@ -2789,7 +2789,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)") } - assert(e.message.contains("ALTER ADD COLUMNS does not support views")) + assert(e.message.contains( + "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 0901c66cccceb..abc20049735cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -26,7 +26,7 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolvedTable, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolvedTable, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedTable} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{EqualTo, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -131,9 +131,9 @@ class PlanResolutionSuite extends AnalysisTest { val rules = Seq( CTESubstitution, analyzer.ResolveRelations, + analyzer.ResolveTables, new ResolveCatalogs(catalogManager), - new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), - analyzer.ResolveTables) + new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v"))) rules.foldLeft(parsePlan(query)) { case (plan, rule) => rule.apply(plan) } @@ -711,24 +711,24 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed3, expected3) } else { parsed1 match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq( + case a: AlterTable => + assert(a.changes == Seq( TableChange.setProperty("test", "test"), TableChange.setProperty("comment", "new_comment"))) case _ => fail("expect AlterTable") } parsed2 match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq( + case a: AlterTable => + assert(a.changes == Seq( TableChange.removeProperty("comment"), TableChange.removeProperty("test"))) case _ => fail("expect AlterTable") } parsed3 match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq( + case a: AlterTable => + assert(a.changes == Seq( TableChange.removeProperty("comment"), TableChange.removeProperty("test"))) case _ => fail("expect AlterTable") @@ -741,15 +741,9 @@ class PlanResolutionSuite extends AnalysisTest { val parsed4 = parseAndResolve(sql4) val parsed5 = parseAndResolve(sql5) - // For non-existing tables, we convert it to v2 command with `UnresolvedV2Table` - parsed4 match { - case AlterTable(_, _, _: UnresolvedV2Relation, _) => // OK - case _ => fail("Expect AlterTable, but got:\n" + parsed4.treeString) - } - parsed5 match { - case AlterTable(_, _, _: UnresolvedV2Relation, _) => // OK - case _ => fail("Expect AlterTable, but got:\n" + parsed5.treeString) - } + // For non-existing tables, we expect `UnresolvedTable` in the resolved plan. + assert(parsed4.collect{ case u: UnresolvedTable => u }.length == 1) + assert(parsed5.collect{ case u: UnresolvedTable => u }.length == 1) } test("support for other types in TBLPROPERTIES") { @@ -770,8 +764,8 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed, expected) } else { parsed match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq( + case a: AlterTable => + assert(a.changes == Seq( TableChange.setProperty("a", "1"), TableChange.setProperty("b", "0.1"), TableChange.setProperty("c", "true"))) @@ -794,8 +788,8 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed, expected) } else { parsed match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq(TableChange.setProperty("location", "new location"))) + case a: AlterTable => + assert(a.changes == Seq(TableChange.setProperty("location", "new location"))) case _ => fail("Expect AlterTable, but got:\n" + parsed.treeString) } } @@ -1036,23 +1030,23 @@ class PlanResolutionSuite extends AnalysisTest { val parsed3 = parseAndResolve(sql3) parsed1 match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq( + case a: AlterTable => + assert(a.changes == Seq( TableChange.updateColumnType(Array("i"), LongType))) case _ => fail("expect AlterTable") } parsed2 match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq( + case a: AlterTable => + assert(a.changes == Seq( TableChange.updateColumnType(Array("i"), LongType), TableChange.updateColumnComment(Array("i"), "new comment"))) case _ => fail("expect AlterTable") } parsed3 match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq( + case a: AlterTable => + assert(a.changes == Seq( TableChange.updateColumnComment(Array("i"), "new comment"))) case _ => fail("expect AlterTable") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index dbbf2b29fe8b7..9b12ac1d79e73 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -158,7 +158,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val message = intercept[AnalysisException] { sql("SHOW TBLPROPERTIES parquet_temp") }.getMessage - assert(message.contains("parquet_temp is a temp view not table")) + assert(message.contains("parquet_temp is a temp view not a table")) } } From f5b345cf3d665c273b68282bcc78f9148fe0ce7f Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Mon, 20 Jan 2020 21:42:33 +0800 Subject: [PATCH 1164/1387] [SPARK-30578][SQL][TEST] Explicitly set conf to use DSv2 for orc in OrcFilterSuite ### What changes were proposed in this pull request? Explicitly set conf to let orc use DSv2 in `OrcFilterSuite` in both v1.2 and v2.3. ### Why are the changes needed? Tests should not rely on default conf when they're going to test something intentionally, which can be fail when conf changes. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass Jenkins. Closes #27285 from Ngone51/fix-orcfilter-test. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../sql/execution/datasources/orc/OrcFilterSuite.scala | 7 +++++++ .../sql/execution/datasources/orc/OrcFilterSuite.scala | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 47632f40bc5ed..ee5162bced8ac 100644 --- a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -25,12 +25,14 @@ import scala.collection.JavaConverters._ import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} +import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -43,6 +45,11 @@ import org.apache.spark.sql.types._ */ class OrcFilterSuite extends OrcTest with SharedSparkSession { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + protected def checkFilterPredicate( df: DataFrame, predicate: Predicate, diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index f88fec7ed4d65..1baa69e82bb18 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} +import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -32,6 +33,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} import org.apache.spark.sql.execution.datasources.v2.orc.{OrcScan, OrcTable} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -44,6 +46,11 @@ import org.apache.spark.sql.types._ */ class OrcFilterSuite extends OrcTest with SharedSparkSession { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + protected def checkFilterPredicate( df: DataFrame, predicate: Predicate, From fd695335934f464b6af135e20de5f1a79aeb4850 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 21 Jan 2020 10:19:07 +0900 Subject: [PATCH 1165/1387] [SPARK-30482][CORE][SQL][TESTS][FOLLOW-UP] Output caller info in log appenders while reaching the limit ### What changes were proposed in this pull request? In the PR, I propose to output additional msg from the tests where a log appender is added. The message is printed as a part of `IllegalStateException` in the case of reaching the limit of maximum number of logged events. ### Why are the changes needed? If a log appender is not removed from the log4j appenders list. the caller message could help to investigate the problem and find the test which doesn't remove the log appender. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the modified test suites `AvroSuite`, `CSVSuite`, `ResolveHintsSuite` and etc. Closes #27296 from MaxGekk/assign-name-to-log-appender. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- core/src/test/scala/org/apache/spark/SparkFunSuite.scala | 6 ++++-- .../test/scala/org/apache/spark/sql/avro/AvroSuite.scala | 2 +- .../spark/sql/catalyst/analysis/ResolveHintsSuite.scala | 2 +- .../sql/catalyst/expressions/CodeGenerationSuite.scala | 2 +- .../sql/catalyst/optimizer/OptimizerLoggingSuite.scala | 2 +- .../src/test/scala/org/apache/spark/sql/JoinHintSuite.scala | 2 +- .../spark/sql/execution/datasources/csv/CSVSuite.scala | 4 ++-- .../scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 2 +- 8 files changed, 12 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index bb64ad8cb19f6..096806142ef13 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -189,12 +189,14 @@ abstract class SparkFunSuite } } - class LogAppender(maxEvents: Int = 100) extends AppenderSkeleton { + class LogAppender(msg: String = "", maxEvents: Int = 100) extends AppenderSkeleton { val loggingEvents = new ArrayBuffer[LoggingEvent]() override def append(loggingEvent: LoggingEvent): Unit = { if (loggingEvents.size >= maxEvents) { - throw new IllegalStateException(s"Number of logging event reached the limit: $maxEvents") + val loggingInfo = if (msg == "") "." else s" while logging $msg." + throw new IllegalStateException( + s"Number of events reached the limit of $maxEvents$loggingInfo") } loggingEvents.append(loggingEvent) } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index b349ac57892db..5d2f78deb4f05 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1499,7 +1499,7 @@ abstract class AvroSuite extends QueryTest with SharedSparkSession { } test("log a warning of ignoreExtension deprecation") { - val logAppender = new LogAppender + val logAppender = new LogAppender("deprecated Avro option 'ignoreExtension'") withTempPath { dir => Seq(("a", 1, 2), ("b", 1, 2), ("c", 2, 1), ("d", 2, 1)) .toDF("value", "p1", "p2") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala index 4fda65e201e71..5e66c038738a4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveHintsSuite.scala @@ -223,7 +223,7 @@ class ResolveHintsSuite extends AnalysisTest { } test("log warnings for invalid hints") { - val logAppender = new LogAppender + val logAppender = new LogAppender("invalid hints") withLogAppender(logAppender) { checkAnalysis( UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE")), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 32cbb858d86f0..f7b39b679a357 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -519,7 +519,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-25113: should log when there exists generated methods above HugeMethodLimit") { - val appender = new LogAppender + val appender = new LogAppender("huge method limit") withLogAppender(appender, loggerName = Some(classOf[CodeGenerator[_, _]].getName)) { val x = 42 val expr = HugeCodeIntExpression(x) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala index 927adc1551a88..d3b0a0e7d9347 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala @@ -37,7 +37,7 @@ class OptimizerLoggingSuite extends PlanTest { } private def verifyLog(expectedLevel: Level, expectedRulesOrBatches: Seq[String]): Unit = { - val logAppender = new LogAppender + val logAppender = new LogAppender("optimizer rules") withLogAppender(logAppender, loggerName = Some(Optimize.getClass.getName.dropRight(1)), level = Some(Level.TRACE)) { val input = LocalRelation('a.int, 'b.string, 'c.double) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index 5767d76f2c14f..f68c416941266 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -49,7 +49,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkP df: => DataFrame, expectedHints: Seq[JoinHint], warnings: Seq[String]): Unit = { - val logAppender = new LogAppender + val logAppender = new LogAppender("join hints") withLogAppender(logAppender) { verifyJoinHint(df, expectedHints) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index d88ec62822b50..95b3c9d70738b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -1761,7 +1761,7 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } test("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") { - val testAppender1 = new LogAppender + val testAppender1 = new LogAppender("CSV header matches to schema") withLogAppender(testAppender1) { val ds = Seq("columnA,columnB", "1.0,1000.0").toDS() val ischema = new StructType().add("columnB", DoubleType).add("columnA", DoubleType) @@ -1771,7 +1771,7 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { assert(testAppender1.loggingEvents .exists(msg => msg.getRenderedMessage.contains("CSV header does not conform to the schema"))) - val testAppender2 = new LogAppender + val testAppender2 = new LogAppender("CSV header matches to schema w/ enforceSchema") withLogAppender(testAppender2) { withTempPath { path => val oschema = new StructType().add("f1", DoubleType).add("f2", DoubleType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index db01db76f8699..61be3672f3ebe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -329,7 +329,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } test("log deprecation warnings") { - val logAppender = new LogAppender + val logAppender = new LogAppender("deprecated SQL configs") def check(config: String): Unit = { assert(logAppender.loggingEvents.exists( e => e.getLevel == Level.WARN && From 14bc2a21621de520a657fbde6a019d81f184cee3 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 21 Jan 2020 10:20:01 +0900 Subject: [PATCH 1166/1387] [SPARK-30530][SQL][FOLLOW-UP] Remove unnecessary codes and fix comments accordingly in UnivocityParser ### What changes were proposed in this pull request? This PR proposes to clean up `UnivocityParser`. ### Why are the changes needed? It will slightly improve the performance since we don't do unnecessary computation for Array concatenations/creation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually ran the existing tests. Closes #27287 from HyukjinKwon/SPARK-30530-followup. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../sql/catalyst/csv/UnivocityParser.scala | 50 ++++++++----------- 1 file changed, 21 insertions(+), 29 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 22fd69b2ce0da..61b1b437da04c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -67,11 +67,10 @@ class UnivocityParser( // their positions in the data schema. private val parsedSchema = if (options.columnPruning) requiredSchema else dataSchema - val tokenizer = { + val tokenizer: CsvParser = { val parserSetting = options.asParserSettings // When to-be-parsed schema is shorter than the to-be-read data schema, we let Univocity CSV // parser select a sequence of fields for reading by their positions. - // if (options.columnPruning && requiredSchema.length < dataSchema.length) { if (parsedSchema.length < dataSchema.length) { parserSetting.selectIndexes(tokenIndexArr: _*) } @@ -203,20 +202,21 @@ class UnivocityParser( } } - private val doParse = if (options.columnPruning && requiredSchema.isEmpty) { - // If `columnPruning` enabled and partition attributes scanned only, - // `schema` gets empty. - (_: String) => Some(InternalRow.empty) - } else { - // parse if the columnPruning is disabled or requiredSchema is nonEmpty - (input: String) => convert(tokenizer.parseLine(input)) - } - /** * Parses a single CSV string and turns it into either one resulting row or no row (if the * the record is malformed). */ - def parse(input: String): Option[InternalRow] = doParse(input) + val parse: String => Option[InternalRow] = { + // This is intentionally a val to create a function once and reuse. + if (options.columnPruning && requiredSchema.isEmpty) { + // If `columnPruning` enabled and partition attributes scanned only, + // `schema` gets empty. + (_: String) => Some(InternalRow.empty) + } else { + // parse if the columnPruning is disabled or requiredSchema is nonEmpty + (input: String) => convert(tokenizer.parseLine(input)) + } + } private val getToken = if (options.columnPruning) { (tokens: Array[String], index: Int) => tokens(index) @@ -232,20 +232,13 @@ class UnivocityParser( new RuntimeException("Malformed CSV record")) } - var checkedTokens = tokens - var badRecordException: Option[Throwable] = None - - if (tokens.length != parsedSchema.length) { + var badRecordException: Option[Throwable] = if (tokens.length != parsedSchema.length) { // If the number of tokens doesn't match the schema, we should treat it as a malformed record. - // However, we still have chance to parse some of the tokens, by adding extra null tokens in - // the tail if the number is smaller, or by dropping extra tokens if the number is larger. - checkedTokens = if (parsedSchema.length > tokens.length) { - tokens ++ new Array[String](parsedSchema.length - tokens.length) - } else { - tokens.take(parsedSchema.length) - } - badRecordException = Some(new RuntimeException("Malformed CSV record")) - } + // However, we still have chance to parse some of the tokens. It continues to parses the + // tokens normally and sets null when `ArrayIndexOutOfBoundsException` occurs for missing + // tokens. + Some(new RuntimeException("Malformed CSV record")) + } else None // When the length of the returned tokens is identical to the length of the parsed schema, // we just need to: // 1. Convert the tokens that correspond to the required schema. @@ -255,15 +248,14 @@ class UnivocityParser( var skipRow = false while (i < requiredSchema.length) { try { - if (!skipRow) { + if (skipRow) { + row.setNullAt(i) + } else { row(i) = valueConverters(i).apply(getToken(tokens, i)) if (csvFilters.skipRow(row, i)) { skipRow = true } } - if (skipRow) { - row.setNullAt(i) - } } catch { case NonFatal(e) => badRecordException = badRecordException.orElse(Some(e)) From 24efa43826298c3bf0a01945670efb70c22207d9 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 21 Jan 2020 10:37:49 +0800 Subject: [PATCH 1167/1387] [SPARK-30019][SQL] Add the owner property to v2 table ### What changes were proposed in this pull request? Add `owner` property to v2 table, it is reversed by `TableCatalog`, indicates the table's owner. ### Why are the changes needed? enhance ownership management of catalog API ### Does this PR introduce any user-facing change? yes, add 1 reserved property - `owner` , and it is not allowed to use in OPTIONS/TBLPROPERTIES anymore, only if legacy on ### How was this patch tested? add uts Closes #27249 from yaooqinn/SPARK-30019. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 18 +++++- .../sql/connector/catalog/TableCatalog.java | 8 ++- .../sql/catalyst/parser/AstBuilder.scala | 4 ++ .../sql/connector/catalog/CatalogV2Util.scala | 5 ++ .../datasources/v2/DataSourceV2Strategy.scala | 26 +++++---- .../datasources/v2/V2SessionCatalog.scala | 4 +- .../spark/sql/DataFrameWriterV2Suite.scala | 27 +++++---- .../spark/sql/connector/AlterTableTests.scala | 12 ++-- .../sql/connector/DataSourceV2SQLSuite.scala | 56 ++++++++++--------- .../spark/sql/hive/HiveExternalCatalog.scala | 8 ++- .../sql/hive/execution/HiveDDLSuite.scala | 14 ++++- 11 files changed, 121 insertions(+), 61 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 7e0def60e531a..2ddbdb9d88d71 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -317,7 +317,7 @@ license: | no - For databases, please use the ALTER DATABASE ... SET OWNER syntax to modify it. + For databases, please use the ALTER DATABASE ... SET OWNER syntax to modify it @@ -331,7 +331,21 @@ license: | no - For databases, please use the ALTER DATABASE ... SET OWNER syntax to modify it. + For databases, please use the ALTER DATABASE ... SET OWNER syntax to modify it + + + + + owner + + + no + + + yes + + + For tables, it is determined by the user who runs spark and create the table. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index 32c6f8f2cde16..591e1c631be13 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -56,10 +56,16 @@ public interface TableCatalog extends CatalogPlugin { */ String PROP_PROVIDER = "provider"; + /** + * A property to specify the owner of the table. + */ + String PROP_OWNER = "owner"; + /** * The list of reserved table properties. */ - List RESERVED_PROPERTIES = Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_PROVIDER); + List RESERVED_PROPERTIES = + Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_PROVIDER, PROP_OWNER); /** * List the tables in a namespace from the catalog. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d09c53ed919cb..2050ec3399303 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2680,6 +2680,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging throw new ParseException(s"$PROP_LOCATION is a reserved table property, please use" + s" the LOCATION clause to specify it.", ctx) case (PROP_LOCATION, _) => false + case (PROP_OWNER, _) if !legacyOn => + throw new ParseException(s"$PROP_OWNER is a reserved table property, it will be" + + s" set to the current user by default.", ctx) + case (PROP_OWNER, _) => false case _ => true } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 3ee22548ca3be..a4c7b4c3a2894 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils private[sql] object CatalogV2Util { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -275,6 +276,10 @@ private[sql] object CatalogV2Util { location.map(TableCatalog.PROP_LOCATION -> _) } + def withDefaultOwnership(properties: Map[String, String]): Map[String, String] = { + properties ++ Map(TableCatalog.PROP_OWNER -> Utils.getCurrentUserName()) + } + def getTableProviderCatalog( provider: SupportsCatalogOptions, catalogManager: CatalogManager, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 7169a437359a7..c6d8a12a94549 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable} import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, TableChange} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, ProjectExec, RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy @@ -114,31 +114,37 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil case CreateV2Table(catalog, ident, schema, parts, props, ifNotExists) => - CreateTableExec(catalog, ident, schema, parts, props, ifNotExists) :: Nil + val propsWithOwner = CatalogV2Util.withDefaultOwnership(props) + CreateTableExec(catalog, ident, schema, parts, propsWithOwner, ifNotExists) :: Nil case CreateTableAsSelect(catalog, ident, parts, query, props, options, ifNotExists) => + val propsWithOwner = CatalogV2Util.withDefaultOwnership(props) val writeOptions = new CaseInsensitiveStringMap(options.asJava) catalog match { case staging: StagingTableCatalog => - AtomicCreateTableAsSelectExec( - staging, ident, parts, query, planLater(query), props, writeOptions, ifNotExists) :: Nil + AtomicCreateTableAsSelectExec(staging, ident, parts, query, planLater(query), + propsWithOwner, writeOptions, ifNotExists) :: Nil case _ => - CreateTableAsSelectExec( - catalog, ident, parts, query, planLater(query), props, writeOptions, ifNotExists) :: Nil + CreateTableAsSelectExec(catalog, ident, parts, query, planLater(query), + propsWithOwner, writeOptions, ifNotExists) :: Nil } case RefreshTable(catalog, ident) => RefreshTableExec(catalog, ident) :: Nil case ReplaceTable(catalog, ident, schema, parts, props, orCreate) => + val propsWithOwner = CatalogV2Util.withDefaultOwnership(props) catalog match { case staging: StagingTableCatalog => - AtomicReplaceTableExec(staging, ident, schema, parts, props, orCreate = orCreate) :: Nil + AtomicReplaceTableExec( + staging, ident, schema, parts, propsWithOwner, orCreate = orCreate) :: Nil case _ => - ReplaceTableExec(catalog, ident, schema, parts, props, orCreate = orCreate) :: Nil + ReplaceTableExec( + catalog, ident, schema, parts, propsWithOwner, orCreate = orCreate) :: Nil } case ReplaceTableAsSelect(catalog, ident, parts, query, props, options, orCreate) => + val propsWithOwner = CatalogV2Util.withDefaultOwnership(props) val writeOptions = new CaseInsensitiveStringMap(options.asJava) catalog match { case staging: StagingTableCatalog => @@ -148,7 +154,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat parts, query, planLater(query), - props, + propsWithOwner, writeOptions, orCreate = orCreate) :: Nil case _ => @@ -158,7 +164,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat parts, query, planLater(query), - props, + propsWithOwner, writeOptions, orCreate = orCreate) :: Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index ddb2926eb6c9a..8eea1cf9c06e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -125,10 +125,12 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) val properties = CatalogV2Util.applyPropertiesChanges(catalogTable.properties, changes) val schema = CatalogV2Util.applySchemaChanges(catalogTable.schema, changes) val comment = properties.get(TableCatalog.PROP_COMMENT) + val owner = properties.getOrElse(TableCatalog.PROP_OWNER, catalogTable.owner) try { catalog.alterTable( - catalogTable.copy(properties = properties, schema = schema, comment = comment)) + catalogTable + .copy(properties = properties, schema = schema, owner = owner, comment = comment)) } catch { case _: NoSuchTableException => throw new NoSuchTableException(ident) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index ce0a5f21fd7ec..4e6381aea3c31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} +import org.apache.spark.util.Utils class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with BeforeAndAfter { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -37,6 +38,8 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.sessionState.catalogManager.catalog(name).asTableCatalog } + private val defaultOwnership = Map(TableCatalog.PROP_OWNER -> Utils.getCurrentUserName()) + before { spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName) @@ -234,7 +237,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(table.name === "testcat.table_name") assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) assert(table.partitioning.isEmpty) - assert(table.properties.isEmpty) + assert(table.properties == defaultOwnership.asJava) } test("Create: with using") { @@ -249,7 +252,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(table.name === "testcat.table_name") assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) assert(table.partitioning.isEmpty) - assert(table.properties === Map("provider" -> "foo").asJava) + assert(table.properties === (Map("provider" -> "foo") ++ defaultOwnership).asJava) } test("Create: with property") { @@ -264,7 +267,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(table.name === "testcat.table_name") assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) assert(table.partitioning.isEmpty) - assert(table.properties === Map("prop" -> "value").asJava) + assert(table.properties === (Map("prop" -> "value") ++ defaultOwnership).asJava) } test("Create: identity partitioned table") { @@ -279,7 +282,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(table.name === "testcat.table_name") assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) - assert(table.properties.isEmpty) + assert(table.properties == defaultOwnership.asJava) } test("Create: partitioned by years(ts)") { @@ -368,7 +371,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(table.name === "testcat.table_name") assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) - assert(table.properties === Map("provider" -> "foo").asJava) + assert(table.properties === (Map("provider" -> "foo") ++ defaultOwnership).asJava) } test("Replace: basic behavior") { @@ -386,7 +389,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(table.name === "testcat.table_name") assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) - assert(table.properties === Map("provider" -> "foo").asJava) + assert(table.properties === (Map("provider" -> "foo") ++ defaultOwnership).asJava) spark.table("source2") .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) @@ -405,7 +408,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo .add("data", StringType) .add("even_or_odd", StringType)) assert(replaced.partitioning.isEmpty) - assert(replaced.properties.isEmpty) + assert(replaced.properties === defaultOwnership.asJava) } test("Replace: partitioned table") { @@ -422,7 +425,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(table.name === "testcat.table_name") assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) assert(table.partitioning.isEmpty) - assert(table.properties === Map("provider" -> "foo").asJava) + assert(table.properties === (Map("provider" -> "foo") ++ defaultOwnership).asJava) spark.table("source2") .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) @@ -441,7 +444,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo .add("data", StringType) .add("even_or_odd", StringType)) assert(replaced.partitioning === Seq(IdentityTransform(FieldReference("id")))) - assert(replaced.properties.isEmpty) + assert(replaced.properties === defaultOwnership.asJava) } test("Replace: fail if table does not exist") { @@ -465,7 +468,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(replaced.name === "testcat.table_name") assert(replaced.schema === new StructType().add("id", LongType).add("data", StringType)) assert(replaced.partitioning.isEmpty) - assert(replaced.properties.isEmpty) + assert(replaced.properties === defaultOwnership.asJava) } test("CreateOrReplace: table exists") { @@ -483,7 +486,7 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(table.name === "testcat.table_name") assert(table.schema === new StructType().add("id", LongType).add("data", StringType)) assert(table.partitioning === Seq(IdentityTransform(FieldReference("id")))) - assert(table.properties === Map("provider" -> "foo").asJava) + assert(table.properties === (Map("provider" -> "foo") ++ defaultOwnership).asJava) spark.table("source2") .withColumn("even_or_odd", when(($"id" % 2) === 0, "even").otherwise("odd")) @@ -502,6 +505,6 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo .add("data", StringType) .add("even_or_odd", StringType)) assert(replaced.partitioning.isEmpty) - assert(replaced.properties.isEmpty) + assert(replaced.properties === defaultOwnership.asJava) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index d304d5b2ca6a2..2fc5020c39ade 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -945,7 +946,7 @@ trait AlterTableTests extends SharedSparkSession { assert(table.name === fullTableName(t)) assert(table.properties === - Map("provider" -> v2Format, "location" -> "s3://bucket/path").asJava) + withDefaultOwnership(Map("provider" -> v2Format, "location" -> "s3://bucket/path")).asJava) } } @@ -971,7 +972,8 @@ trait AlterTableTests extends SharedSparkSession { val table = getTableMetadata(t) assert(table.name === fullTableName(t)) - assert(table.properties === Map("provider" -> v2Format, "test" -> "34").asJava) + assert(table.properties === + withDefaultOwnership(Map("provider" -> v2Format, "test" -> "34")).asJava) } } @@ -983,15 +985,15 @@ trait AlterTableTests extends SharedSparkSession { val table = getTableMetadata(t) assert(table.name === fullTableName(t)) - assert(table.properties === Map("provider" -> v2Format, "test" -> "34").asJava) + assert(table.properties === + withDefaultOwnership(Map("provider" -> v2Format, "test" -> "34")).asJava) sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('test')") val updated = getTableMetadata(t) assert(updated.name === fullTableName(t)) - assert(updated.properties === Map("provider" -> v2Format).asJava) + assert(updated.properties === withDefaultOwnership(Map("provider" -> v2Format)).asJava) } } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 4c5b1d95b12da..e65030f715204 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableExceptio import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME +import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.sources.SimpleScanSource @@ -41,6 +42,7 @@ class DataSourceV2SQLSuite private val v2Source = classOf[FakeV2Provider].getName override protected val v2Format = v2Source override protected val catalogAndNamespace = "testcat.ns1.ns2." + private val defaultUser: String = Utils.getCurrentUserName() private def catalog(name: String): CatalogPlugin = { spark.sessionState.catalogManager.catalog(name) @@ -94,7 +96,7 @@ class DataSourceV2SQLSuite assert(table.name == "testcat.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType() .add("id", LongType, nullable = false) .add("data", StringType)) @@ -160,6 +162,7 @@ class DataSourceV2SQLSuite Array("Comment", "this is a test table", ""), Array("Location", "/tmp/testcat/table_name", ""), Array("Provider", "foo", ""), + Array(TableCatalog.PROP_OWNER.capitalize, defaultUser, ""), Array("Table Properties", "[bar=baz]", ""))) } @@ -172,7 +175,7 @@ class DataSourceV2SQLSuite assert(table.name == "default.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> v2Source).asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> v2Source)).asJava) assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) @@ -187,7 +190,7 @@ class DataSourceV2SQLSuite val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) // run a second create query that should fail @@ -201,7 +204,7 @@ class DataSourceV2SQLSuite val table2 = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table2.name == "testcat.table_name") assert(table2.partitioning.isEmpty) - assert(table2.properties == Map("provider" -> "foo").asJava) + assert(table2.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table2.schema == new StructType().add("id", LongType).add("data", StringType)) // check that the table is still empty @@ -218,7 +221,7 @@ class DataSourceV2SQLSuite assert(table.name == "testcat.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) spark.sql("CREATE TABLE IF NOT EXISTS testcat.table_name (id bigint, data string) USING bar") @@ -227,7 +230,7 @@ class DataSourceV2SQLSuite val table2 = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table2.name == "testcat.table_name") assert(table2.partitioning.isEmpty) - assert(table2.properties == Map("provider" -> "foo").asJava) + assert(table2.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table2.schema == new StructType().add("id", LongType).add("data", StringType)) // check that the table is still empty @@ -244,7 +247,7 @@ class DataSourceV2SQLSuite assert(table.name == "testcat.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) // check that the table is empty @@ -266,7 +269,7 @@ class DataSourceV2SQLSuite assert(table.name == identifier) assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType() .add("id", LongType) .add("data", StringType)) @@ -293,7 +296,7 @@ class DataSourceV2SQLSuite assert(replacedTable != originalTable, "Table should have been replaced.") assert(replacedTable.name == identifier) assert(replacedTable.partitioning.isEmpty) - assert(replacedTable.properties == Map("provider" -> "foo").asJava) + assert(replacedTable.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(replacedTable.schema == new StructType().add("id", LongType)) val rdd = spark.sparkContext.parallelize(replacedTable.asInstanceOf[InMemoryTable].rows) @@ -431,7 +434,7 @@ class DataSourceV2SQLSuite assert(table.name == "default.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> v2Source).asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> v2Source)).asJava) assert(table.schema == new StructType() .add("id", LongType) .add("data", StringType)) @@ -448,7 +451,7 @@ class DataSourceV2SQLSuite val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table.name == "testcat.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType() .add("id", LongType) .add("data", StringType)) @@ -468,7 +471,7 @@ class DataSourceV2SQLSuite val table2 = testCatalog.loadTable(Identifier.of(Array(), "table_name")) assert(table2.name == "testcat.table_name") assert(table2.partitioning.isEmpty) - assert(table2.properties == Map("provider" -> "foo").asJava) + assert(table2.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table2.schema == new StructType() .add("id", LongType) .add("data", StringType)) @@ -486,7 +489,7 @@ class DataSourceV2SQLSuite assert(table.name == "testcat.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType() .add("id", LongType) .add("data", StringType)) @@ -517,7 +520,7 @@ class DataSourceV2SQLSuite assert(table.name == "testcat.table_name") assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType() .add("id", LongType) .add("data", StringType)) @@ -557,7 +560,7 @@ class DataSourceV2SQLSuite assert(table.name == identifier) assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.properties == withDefaultOwnership(Map("provider" -> "foo")).asJava) assert(table.schema == new StructType().add("i", "int")) val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) @@ -1059,7 +1062,7 @@ class DataSourceV2SQLSuite Row("Namespace Name", "ns2"), Row("Description", "test namespace"), Row("Location", "/tmp/ns_test"), - Row("Owner Name", Utils.getCurrentUserName()), + Row("Owner Name", defaultUser), Row("Owner Type", "USER") )) } @@ -1075,7 +1078,7 @@ class DataSourceV2SQLSuite Row("Namespace Name", "ns2"), Row("Description", "test namespace"), Row("Location", "/tmp/ns_test"), - Row("Owner Name", Utils.getCurrentUserName()), + Row("Owner Name", defaultUser), Row("Owner Type", "USER"), Row("Properties", "((a,b),(b,a),(c,c))") )) @@ -1123,7 +1126,7 @@ class DataSourceV2SQLSuite Row("Namespace Name", "ns2"), Row("Description", "test namespace"), Row("Location", "/tmp/ns_test_2"), - Row("Owner Name", Utils.getCurrentUserName()), + Row("Owner Name", defaultUser), Row("Owner Type", "USER") )) } @@ -1923,22 +1926,23 @@ class DataSourceV2SQLSuite test("SHOW TBLPROPERTIES: v2 table") { val t = "testcat.ns1.ns2.tbl" withTable(t) { - val owner = "andrew" + val user = "andrew" val status = "new" val provider = "foo" spark.sql(s"CREATE TABLE $t (id bigint, data string) USING $provider " + - s"TBLPROPERTIES ('owner'='$owner', 'status'='$status')") + s"TBLPROPERTIES ('user'='$user', 'status'='$status')") - val properties = sql(s"SHOW TBLPROPERTIES $t") + val properties = sql(s"SHOW TBLPROPERTIES $t").orderBy("key") val schema = new StructType() .add("key", StringType, nullable = false) .add("value", StringType, nullable = false) val expected = Seq( - Row("owner", owner), + Row(TableCatalog.PROP_OWNER, defaultUser), + Row("provider", provider), Row("status", status), - Row("provider", provider)) + Row("user", user)) assert(properties.schema === schema) assert(expected === properties.collect()) @@ -1948,11 +1952,11 @@ class DataSourceV2SQLSuite test("SHOW TBLPROPERTIES(key): v2 table") { val t = "testcat.ns1.ns2.tbl" withTable(t) { - val owner = "andrew" + val user = "andrew" val status = "new" val provider = "foo" spark.sql(s"CREATE TABLE $t (id bigint, data string) USING $provider " + - s"TBLPROPERTIES ('owner'='$owner', 'status'='$status')") + s"TBLPROPERTIES ('user'='$user', 'status'='$status')") val properties = sql(s"SHOW TBLPROPERTIES $t ('status')") @@ -1967,7 +1971,7 @@ class DataSourceV2SQLSuite withTable(t) { val nonExistingKey = "nonExistingKey" spark.sql(s"CREATE TABLE $t (id bigint, data string) USING foo " + - s"TBLPROPERTIES ('owner'='andrew', 'status'='new')") + s"TBLPROPERTIES ('user'='andrew', 'status'='new')") val properties = sql(s"SHOW TBLPROPERTIES $t ('$nonExistingKey')") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 03874d005a6e6..ca292f65efeee 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -40,8 +40,8 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.ColumnStat import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.{PartitioningUtils, SourceOptions} import org.apache.spark.sql.hive.client.HiveClient @@ -635,12 +635,16 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat k.startsWith(CREATED_SPARK_VERSION) } val newTableProps = propsFromOldTable ++ tableDefinition.properties + partitionProviderProp + + // // Add old table's owner if we need to restore + val owner = Option(tableDefinition.owner).filter(_.nonEmpty).getOrElse(oldTableDef.owner) val newDef = tableDefinition.copy( storage = newStorage, schema = oldTableDef.schema, partitionColumnNames = oldTableDef.partitionColumnNames, bucketSpec = oldTableDef.bucketSpec, - properties = newTableProps) + properties = newTableProps, + owner = owner) client.alterTable(newDef) } 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 b3f7fc4d0557e..59eadb844837e 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.connector.catalog.{CatalogManager, TableCatalog} import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} import org.apache.spark.sql.functions._ @@ -418,13 +418,23 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA catalog.reset() } } + + test("Table Ownership") { + val catalog = spark.sessionState.catalog + try { + sql(s"CREATE TABLE spark_30019(k int)") + assert(sql(s"DESCRIBE TABLE EXTENDED spark_30019").where("col_name='Owner'") + .collect().head.getString(1) === Utils.getCurrentUserName()) + } finally { + catalog.reset() + } + } } class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { import testImplicits._ val hiveFormats = Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO") - private val reversedProperties = Seq("ownerName", "ownerType") override def afterEach(): Unit = { try { From 0388b7a3ecf25344b808b029df97cccccb698165 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 21 Jan 2020 11:14:26 +0800 Subject: [PATCH 1168/1387] [SPARK-30568][SQL] Invalidate interval type as a field table schema ### What changes were proposed in this pull request? After this commit https://github.com/apache/spark/commit/d67b98ea016e9b714bef68feaac108edd08159c9, we are able to create table or alter table with interval column types if the external catalog accepts which is varying the interval type's purpose for internal usage. With https://github.com/apache/spark/commit/d67b98ea016e9b714bef68feaac108edd08159c9 's original purpose it should only work from cast logic. Instead of adding type checker for the interval type from commands to commands to work among catalogs, It much simpler to treat interval as an invalid data type but can be identified by cast only. ### Why are the changes needed? enhance interval internal usage purpose. ### Does this PR introduce any user-facing change? NO, Additionally, this PR restores user behavior when using interval type to create/alter table schema, e.g. for hive catalog for 2.4, ```java Caused by: org.apache.spark.sql.catalyst.parser.ParseException: DataType calendarinterval is not supported.(line 1, pos 0) ``` for master after https://github.com/apache/spark/commit/d67b98ea016e9b714bef68feaac108edd08159c9 ```java Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.IllegalArgumentException: Error: type expected at the position 0 of 'interval' but 'interval' is found. at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:862) ``` now with this pr, we restore the type checker in spark side. ### How was this patch tested? add more ut Closes #27277 from yaooqinn/SPARK-30568. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/CheckAnalysis.scala | 31 ++++++++++++------- .../spark/sql/catalyst/util/TypeUtils.scala | 15 +++++++++ .../spark/sql/connector/AlterTableTests.scala | 24 +++++++++++++- .../DataSourceV2DataFrameSuite.scala | 25 ++++++++++++++- .../sql/connector/DataSourceV2SQLSuite.scala | 22 +++++++++++++ 5 files changed, 104 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 65a38aacecd43..624c25d95c704 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnPosition, UpdateColumnType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -384,6 +384,11 @@ trait CheckAnalysis extends PredicateHelper { failAnalysis(s"Invalid partitioning: ${badReferences.mkString(", ")}") } + create.tableSchema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) + + case write: V2WriteCommand if write.resolved => + write.query.schema.foreach(f => TypeUtils.failWithIntervalType(f.dataType)) + // If the view output doesn't have the same number of columns neither with the child // output, nor with the query column names, throw an AnalysisException. // If the view's child output can't up cast to the view output, @@ -443,23 +448,27 @@ trait CheckAnalysis extends PredicateHelper { if (parent.nonEmpty) { findField("add to", parent) } + TypeUtils.failWithIntervalType(add.dataType()) case update: UpdateColumnType => val field = findField("update", update.fieldNames) val fieldName = update.fieldNames.quoted update.newDataType match { case _: StructType => - throw new AnalysisException( - s"Cannot update ${table.name} field $fieldName type: " + - s"update a struct by adding, deleting, or updating its fields") + alter.failAnalysis(s"Cannot update ${table.name} field $fieldName type: " + + s"update a struct by updating its fields") case _: MapType => - throw new AnalysisException( - s"Cannot update ${table.name} field $fieldName type: " + - s"update a map by updating $fieldName.key or $fieldName.value") + alter.failAnalysis(s"Cannot update ${table.name} field $fieldName type: " + + s"update a map by updating $fieldName.key or $fieldName.value") case _: ArrayType => - throw new AnalysisException( - s"Cannot update ${table.name} field $fieldName type: " + - s"update the element by updating $fieldName.element") - case _: AtomicType => + alter.failAnalysis(s"Cannot update ${table.name} field $fieldName type: " + + s"update the element by updating $fieldName.element") + case u: UserDefinedType[_] => + alter.failAnalysis(s"Cannot update ${table.name} field $fieldName type: " + + s"update a UserDefinedType[${u.sql}] by updating its fields") + case _: CalendarIntervalType => + alter.failAnalysis(s"Cannot update ${table.name} field $fieldName to " + + s"interval type") + case _ => // update is okay } if (!Cast.canUpCast(field.dataType, update.newDataType)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala index 9680ea3cd2067..e8266dd401362 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.util +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.RowOrdering import org.apache.spark.sql.types._ @@ -98,4 +99,18 @@ object TypeUtils { case _: AtomicType => true case _ => false } + + def failWithIntervalType(dataType: DataType): Unit = { + dataType match { + case CalendarIntervalType => + throw new AnalysisException("Cannot use interval type in the table schema.") + case ArrayType(et, _) => failWithIntervalType(et) + case MapType(kt, vt, _) => + failWithIntervalType(kt) + failWithIntervalType(vt) + case s: StructType => s.foreach(f => failWithIntervalType(f.dataType)) + case u: UserDefinedType[_] => failWithIntervalType(u.sqlType) + case _ => + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 2fc5020c39ade..ee7f205b3fa52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -117,6 +117,19 @@ trait AlterTableTests extends SharedSparkSession { } } + test("AlterTable: add column with interval type") { + val t = s"${catalogAndNamespace}table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING $v2Format") + val e1 = + intercept[AnalysisException](sql(s"ALTER TABLE $t ADD COLUMN data interval")) + assert(e1.getMessage.contains("Cannot use interval type in the table schema.")) + val e2 = + intercept[AnalysisException](sql(s"ALTER TABLE $t ADD COLUMN point.z interval")) + assert(e2.getMessage.contains("Cannot use interval type in the table schema.")) + } + } + test("AlterTable: add column with position") { val t = s"${catalogAndNamespace}table_name" withTable(t) { @@ -311,6 +324,15 @@ trait AlterTableTests extends SharedSparkSession { } } + test("AlterTable: update column type to interval") { + val t = s"${catalogAndNamespace}table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING $v2Format") + val e = intercept[AnalysisException](sql(s"ALTER TABLE $t ALTER COLUMN id TYPE interval")) + assert(e.getMessage.contains("id to interval type")) + } + } + test("AlterTable: SET/DROP NOT NULL") { val t = s"${catalogAndNamespace}table_name" withTable(t) { @@ -359,7 +381,7 @@ trait AlterTableTests extends SharedSparkSession { } assert(exc.getMessage.contains("point")) - assert(exc.getMessage.contains("update a struct by adding, deleting, or updating its fields")) + assert(exc.getMessage.contains("update a struct by updating its fields")) val table = getTableMetadata(t) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala index 72d4629a1a320..0a6897b829994 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala @@ -17,14 +17,19 @@ package org.apache.spark.sql.connector -import org.apache.spark.sql.{DataFrame, Row, SaveMode} +import java.util.Collections + +import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan} +import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.QueryExecutionListener class DataSourceV2DataFrameSuite extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = false) { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import testImplicits._ before { @@ -163,4 +168,22 @@ class DataSourceV2DataFrameSuite spark.listenerManager.unregister(listener) } } + + test("Cannot write data with intervals to v2") { + withTable("testcat.table_name") { + val testCatalog = spark.sessionState.catalogManager.catalog("testcat").asTableCatalog + testCatalog.createTable( + Identifier.of(Array(), "table_name"), + new StructType().add("i", "interval"), + Array.empty, Collections.emptyMap[String, String]) + val df = sql("select interval 1 day as i") + val v2Writer = df.writeTo("testcat.table_name") + val e1 = intercept[AnalysisException](v2Writer.append()) + assert(e1.getMessage.contains(s"Cannot use interval type in the table schema.")) + val e2 = intercept[AnalysisException](v2Writer.overwrite(df("i"))) + assert(e2.getMessage.contains(s"Cannot use interval type in the table schema.")) + val e3 = intercept[AnalysisException](v2Writer.overwritePartitions()) + assert(e3.getMessage.contains(s"Cannot use interval type in the table schema.")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index e65030f715204..4dd70fa9626d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -255,6 +255,28 @@ class DataSourceV2SQLSuite checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) } + test("CreateTable/RepalceTable: invalid schema if has interval type") { + Seq("CREATE", "REPLACE").foreach { action => + val e1 = intercept[AnalysisException]( + sql(s"$action TABLE table_name (id int, value interval) USING $v2Format")) + assert(e1.getMessage.contains(s"Cannot use interval type in the table schema.")) + val e2 = intercept[AnalysisException]( + sql(s"$action TABLE table_name (id array) USING $v2Format")) + assert(e2.getMessage.contains(s"Cannot use interval type in the table schema.")) + } + } + + test("CTAS/RTAS: invalid schema if has interval type") { + Seq("CREATE", "REPLACE").foreach { action => + val e1 = intercept[AnalysisException]( + sql(s"$action TABLE table_name USING $v2Format as select interval 1 day")) + assert(e1.getMessage.contains(s"Cannot use interval type in the table schema.")) + val e2 = intercept[AnalysisException]( + sql(s"$action TABLE table_name USING $v2Format as select array(interval 1 day)")) + assert(e2.getMessage.contains(s"Cannot use interval type in the table schema.")) + } + } + test("CreateTableAsSelect: use v2 plan because catalog is set") { val basicCatalog = catalog("testcat").asTableCatalog val atomicCatalog = catalog("testcat_atomic").asTableCatalog From 94284c8ecc3a4ced5d4197aab1471478e93098fa Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 21 Jan 2020 11:38:05 +0800 Subject: [PATCH 1169/1387] [SPARK-30587][SQL][TESTS] Add test suites for CSV and JSON v1 ### What changes were proposed in this pull request? In the PR, I propose to make `JsonSuite` and `CSVSuite` abstract classes, and add sub-classes that check JSON/CSV datasource v1 and v2. ### Why are the changes needed? To improve test coverage and test JSON/CSV v1 which is still supported, and can be enabled by users. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running new test suites `JsonV1Suite` and `CSVv1Suite`. Closes #27294 from MaxGekk/csv-json-v1-test-suites. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../execution/datasources/csv/CSVSuite.scala | 18 ++++++++++++++++-- .../execution/datasources/json/JsonSuite.scala | 18 ++++++++++++++++-- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 95b3c9d70738b..97dfbbdb7fd2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -33,14 +33,14 @@ import org.apache.commons.lang3.time.FastDateFormat import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec -import org.apache.spark.{SparkException, TestUtils} +import org.apache.spark.{SparkConf, SparkException, TestUtils} import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ -class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { +abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { import testImplicits._ private val carsFile = "test-data/cars.csv" @@ -2295,3 +2295,17 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } } } + +class CSVv1Suite extends CSVSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "csv") +} + +class CSVv2Suite extends CSVSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index e3e0195f08e3f..1c679394ce2ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec -import org.apache.spark.{SparkException, TestUtils} +import org.apache.spark.{SparkConf, SparkException, TestUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{functions => F, _} import org.apache.spark.sql.catalyst.json._ @@ -45,7 +45,7 @@ class TestFileFilter extends PathFilter { override def accept(path: Path): Boolean = path.getParent.getName != "p=2" } -class JsonSuite extends QueryTest with SharedSparkSession with TestJsonData { +abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJsonData { import testImplicits._ test("Type promotion") { @@ -2527,3 +2527,17 @@ class JsonSuite extends QueryTest with SharedSparkSession with TestJsonData { } } } + +class JsonV1Suite extends JsonSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "json") +} + +class JsonV2Suite extends JsonSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} From 2d59ca464eb12d4de03bfa383a3efcfe0bc0441d Mon Sep 17 00:00:00 2001 From: Guy Khazma Date: Mon, 20 Jan 2020 20:20:37 -0800 Subject: [PATCH 1170/1387] [SPARK-30475][SQL] File source V2: Push data filters for file listing ### What changes were proposed in this pull request? Follow up on [SPARK-30428](https://github.com/apache/spark/pull/27112) which added support for partition pruning in File source V2. This PR implements the necessary changes in order to pass the `dataFilters` to the `listFiles`. This enables having `FileIndex` implementations which use the `dataFilters` for further pruning the file listing (see the discussion [here](https://github.com/apache/spark/pull/27112#discussion_r364757217)). ### Why are the changes needed? Datasources such as `csv` and `json` do not implement the `SupportsPushDownFilters` trait. In order to support data skipping uniformly for all file based data sources, one can override the `listFiles` method in a `FileIndex` implementation, which consults external metadata and prunes the list of files. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Modifying the unit tests for v2 file sources to verify the `dataFilters` are passed Closes #27157 from guykhazma/PushdataFiltersInFileListing. Authored-by: Guy Khazma Signed-off-by: Gengliang Wang --- .../apache/spark/sql/v2/avro/AvroScan.scala | 8 +++-- .../org/apache/spark/sql/avro/AvroSuite.scala | 29 +++++++++++++++ .../PruneFileSourcePartitions.scala | 22 +++++++----- .../execution/datasources/v2/FileScan.scala | 16 ++++++--- .../datasources/v2/csv/CSVScan.scala | 8 +++-- .../datasources/v2/json/JsonScan.scala | 8 +++-- .../datasources/v2/orc/OrcScan.scala | 8 +++-- .../datasources/v2/parquet/ParquetScan.scala | 8 +++-- .../datasources/v2/text/TextScan.scala | 8 +++-- .../spark/sql/FileBasedDataSourceSuite.scala | 36 +++++++++++++++++++ 10 files changed, 120 insertions(+), 31 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala index d5a29124a276e..fe7315c739296 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala @@ -37,7 +37,8 @@ case class AvroScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, - partitionFilters: Seq[Expression] = Seq.empty) extends FileScan { + partitionFilters: Seq[Expression] = Seq.empty, + dataFilters: Seq[Expression] = Seq.empty) extends FileScan { override def isSplitable(path: Path): Boolean = true override def createReaderFactory(): PartitionReaderFactory = { @@ -53,8 +54,9 @@ case class AvroScan( dataSchema, readDataSchema, readPartitionSchema, parsedOptions) } - override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters) + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) override def equals(obj: Any): Boolean = obj match { case a: AvroScan => super.equals(a) && dataSchema == a.dataSchema && options == a.options diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 5d2f78deb4f05..360160c9c9398 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1566,6 +1566,7 @@ class AvroV2Suite extends AvroSuite { } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) + assert(fileScan.get.dataFilters.nonEmpty) assert(fileScan.get.planInputPartitions().forall { partition => partition.asInstanceOf[FilePartition].files.forall { file => file.filePath.contains("p1=1") && file.filePath.contains("p2=2") @@ -1575,6 +1576,34 @@ class AvroV2Suite extends AvroSuite { } } + test("Avro source v2: support passing data filters to FileScan without partitionFilters") { + withTempPath { dir => + Seq(("a", 1, 2), ("b", 1, 2), ("c", 2, 1)) + .toDF("value", "p1", "p2") + .write + .format("avro") + .save(dir.getCanonicalPath) + val df = spark + .read + .format("avro") + .load(dir.getCanonicalPath) + .where("value = 'a'") + + val filterCondition = df.queryExecution.optimizedPlan.collectFirst { + case f: Filter => f.condition + } + assert(filterCondition.isDefined) + + val fileScan = df.queryExecution.executedPlan collectFirst { + case BatchScanExec(_, f: AvroScan) => f + } + assert(fileScan.nonEmpty) + assert(fileScan.get.partitionFilters.isEmpty) + assert(fileScan.get.dataFilters.nonEmpty) + checkAnswer(df, Row("a", 1, 2)) + } + } + private def getBatchScanExec(plan: SparkPlan): BatchScanExec = { plan.find(_.isInstanceOf[BatchScanExec]).get.asInstanceOf[BatchScanExec] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 7fd154ccac445..59c55c161bc89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -28,20 +28,22 @@ import org.apache.spark.sql.types.StructType private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { - private def getPartitionKeyFilters( + private def getPartitionKeyFiltersAndDataFilters( sparkSession: SparkSession, relation: LeafNode, partitionSchema: StructType, filters: Seq[Expression], - output: Seq[AttributeReference]): ExpressionSet = { + output: Seq[AttributeReference]): (ExpressionSet, Seq[Expression]) = { val normalizedFilters = DataSourceStrategy.normalizeExprs( filters.filter(f => f.deterministic && !SubqueryExpression.hasSubquery(f)), output) val partitionColumns = relation.resolve(partitionSchema, sparkSession.sessionState.analyzer.resolver) val partitionSet = AttributeSet(partitionColumns) - ExpressionSet(normalizedFilters.filter { f => + val (partitionFilters, dataFilters) = normalizedFilters.partition(f => f.references.subsetOf(partitionSet) - }) + ) + + (ExpressionSet(partitionFilters), dataFilters) } private def rebuildPhysicalOperation( @@ -72,7 +74,7 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { _, _)) if filters.nonEmpty && fsRelation.partitionSchemaOption.isDefined => - val partitionKeyFilters = getPartitionKeyFilters( + val (partitionKeyFilters, _) = getPartitionKeyFiltersAndDataFilters( fsRelation.sparkSession, logicalRelation, partitionSchema, filters, logicalRelation.output) if (partitionKeyFilters.nonEmpty) { val prunedFileIndex = catalogFileIndex.filterPartitions(partitionKeyFilters.toSeq) @@ -92,11 +94,13 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { case op @ PhysicalOperation(projects, filters, v2Relation @ DataSourceV2ScanRelation(_, scan: FileScan, output)) if filters.nonEmpty && scan.readDataSchema.nonEmpty => - val partitionKeyFilters = getPartitionKeyFilters(scan.sparkSession, - v2Relation, scan.readPartitionSchema, filters, output) - if (partitionKeyFilters.nonEmpty) { + val (partitionKeyFilters, dataFilters) = + getPartitionKeyFiltersAndDataFilters(scan.sparkSession, v2Relation, + scan.readPartitionSchema, filters, output) + // The dataFilters are pushed down only once + if (partitionKeyFilters.nonEmpty || (dataFilters.nonEmpty && scan.dataFilters.isEmpty)) { val prunedV2Relation = - v2Relation.copy(scan = scan.withPartitionFilters(partitionKeyFilters.toSeq)) + v2Relation.copy(scan = scan.withFilters(partitionKeyFilters.toSeq, dataFilters)) // The pushed down partition filters don't need to be reevaluated. val afterScanFilters = ExpressionSet(filters) -- partitionKeyFilters.filter(_.references.nonEmpty) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index a22e1ccfe4515..6e05aa56f4f72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -61,9 +61,15 @@ trait FileScan extends Scan with Batch with SupportsReportStatistics with Loggin def partitionFilters: Seq[Expression] /** - * Create a new `FileScan` instance from the current one with different `partitionFilters`. + * Returns the data filters that can be use for file listing */ - def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan + def dataFilters: Seq[Expression] + + /** + * Create a new `FileScan` instance from the current one + * with different `partitionFilters` and `dataFilters` + */ + def withFilters(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan /** * If a file with `path` is unsplittable, return the unsplittable reason, @@ -79,7 +85,8 @@ trait FileScan extends Scan with Batch with SupportsReportStatistics with Loggin override def equals(obj: Any): Boolean = obj match { case f: FileScan => fileIndex == f.fileIndex && readSchema == f.readSchema - ExpressionSet(partitionFilters) == ExpressionSet(f.partitionFilters) + ExpressionSet(partitionFilters) == ExpressionSet(f.partitionFilters) && + ExpressionSet(dataFilters) == ExpressionSet(f.dataFilters) case _ => false } @@ -92,6 +99,7 @@ trait FileScan extends Scan with Batch with SupportsReportStatistics with Loggin val metadata: Map[String, String] = Map( "ReadSchema" -> readDataSchema.catalogString, "PartitionFilters" -> seqToString(partitionFilters), + "DataFilters" -> seqToString(dataFilters), "Location" -> locationDesc) val metadataStr = metadata.toSeq.sorted.map { case (key, value) => @@ -103,7 +111,7 @@ trait FileScan extends Scan with Batch with SupportsReportStatistics with Loggin } protected def partitions: Seq[FilePartition] = { - val selectedPartitions = fileIndex.listFiles(partitionFilters, Seq.empty) + val selectedPartitions = fileIndex.listFiles(partitionFilters, dataFilters) val maxSplitBytes = FilePartition.maxSplitBytes(sparkSession, selectedPartitions) val partitionAttributes = fileIndex.partitionSchema.toAttributes val attributeMap = partitionAttributes.map(a => normalizeName(a.name) -> a).toMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index 690d66908e613..4f510322815ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -40,7 +40,8 @@ case class CSVScan( readPartitionSchema: StructType, options: CaseInsensitiveStringMap, pushedFilters: Array[Filter], - partitionFilters: Seq[Expression] = Seq.empty) + partitionFilters: Seq[Expression] = Seq.empty, + dataFilters: Seq[Expression] = Seq.empty) extends TextBasedFileScan(sparkSession, options) { private lazy val parsedOptions: CSVOptions = new CSVOptions( @@ -91,8 +92,9 @@ case class CSVScan( dataSchema, readDataSchema, readPartitionSchema, parsedOptions, pushedFilters) } - override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters) + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) override def equals(obj: Any): Boolean = obj match { case c: CSVScan => super.equals(c) && dataSchema == c.dataSchema && options == c.options && diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala index 153b402476c40..75231625676ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala @@ -39,7 +39,8 @@ case class JsonScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, - partitionFilters: Seq[Expression] = Seq.empty) + partitionFilters: Seq[Expression] = Seq.empty, + dataFilters: Seq[Expression] = Seq.empty) extends TextBasedFileScan(sparkSession, options) { private val parsedOptions = new JSONOptionsInRead( @@ -88,8 +89,9 @@ case class JsonScan( dataSchema, readDataSchema, readPartitionSchema, parsedOptions) } - override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters) + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) override def equals(obj: Any): Boolean = obj match { case j: JsonScan => super.equals(j) && dataSchema == j.dataSchema && options == j.options diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index f0595cb6d09c3..62894fa7a2538 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -38,7 +38,8 @@ case class OrcScan( readPartitionSchema: StructType, options: CaseInsensitiveStringMap, pushedFilters: Array[Filter], - partitionFilters: Seq[Expression] = Seq.empty) extends FileScan { + partitionFilters: Seq[Expression] = Seq.empty, + dataFilters: Seq[Expression] = Seq.empty) extends FileScan { override def isSplitable(path: Path): Boolean = true override def createReaderFactory(): PartitionReaderFactory = { @@ -64,6 +65,7 @@ case class OrcScan( super.description() + ", PushedFilters: " + seqToString(pushedFilters) } - override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters) + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala index 44179e2e42a4c..bb315262a8211 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala @@ -41,7 +41,8 @@ case class ParquetScan( readPartitionSchema: StructType, pushedFilters: Array[Filter], options: CaseInsensitiveStringMap, - partitionFilters: Seq[Expression] = Seq.empty) extends FileScan { + partitionFilters: Seq[Expression] = Seq.empty, + dataFilters: Seq[Expression] = Seq.empty) extends FileScan { override def isSplitable(path: Path): Boolean = true override def createReaderFactory(): PartitionReaderFactory = { @@ -92,6 +93,7 @@ case class ParquetScan( super.description() + ", PushedFilters: " + seqToString(pushedFilters) } - override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters) + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala index cf6595e5c126c..e75de2c4a4079 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala @@ -36,7 +36,8 @@ case class TextScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, - partitionFilters: Seq[Expression] = Seq.empty) + partitionFilters: Seq[Expression] = Seq.empty, + dataFilters: Seq[Expression] = Seq.empty) extends TextBasedFileScan(sparkSession, options) { private val optionsAsScala = options.asScala.toMap @@ -70,8 +71,9 @@ case class TextScan( readPartitionSchema, textOptions) } - override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters) + override def withFilters( + partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) override def equals(obj: Any): Boolean = obj match { case t: TextScan => super.equals(t) && options == t.options diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 972683512db88..c870958128483 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -775,6 +775,7 @@ class FileBasedDataSourceSuite extends QueryTest } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) + assert(fileScan.get.dataFilters.nonEmpty) assert(fileScan.get.planInputPartitions().forall { partition => partition.asInstanceOf[FilePartition].files.forall { file => file.filePath.contains("p1=1") && file.filePath.contains("p2=2") @@ -786,6 +787,41 @@ class FileBasedDataSourceSuite extends QueryTest } } + test("File source v2: support passing data filters to FileScan without partitionFilters") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + allFileBasedDataSources.foreach { format => + withTempPath { dir => + Seq(("a", 1, 2), ("b", 1, 2), ("c", 2, 1)) + .toDF("value", "p1", "p2") + .write + .format(format) + .partitionBy("p1", "p2") + .option("header", true) + .save(dir.getCanonicalPath) + val df = spark + .read + .format(format) + .option("header", true) + .load(dir.getCanonicalPath) + .where("value = 'a'") + + val filterCondition = df.queryExecution.optimizedPlan.collectFirst { + case f: Filter => f.condition + } + assert(filterCondition.isDefined) + + val fileScan = df.queryExecution.executedPlan collectFirst { + case BatchScanExec(_, f: FileScan) => f + } + assert(fileScan.nonEmpty) + assert(fileScan.get.partitionFilters.isEmpty) + assert(fileScan.get.dataFilters.nonEmpty) + checkAnswer(df, Row("a", 1, 2)) + } + } + } + } + test("File table location should include both values of option `path` and `paths`") { withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { withTempPaths(3) { paths => From 78df532556950a066d8efa9f6dba6d8ebdc5ea50 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 21 Jan 2020 14:23:55 +0800 Subject: [PATCH 1171/1387] [SPARK-30433][SQL][FOLLOW-UP] Optimize collect conflict plans ### What changes were proposed in this pull request? For LogicalPlan(e.g. `MultiInstanceRelation`, `Project`, `Aggregate`, etc) whose output doesn't inherit directly from its children, we could just stop collect on it. Because we could always replace all the lower conflict attributes with the new attributes from the new plan. Otherwise, we should recursively collect conflict plans, like `Generate`, `Window`. ### Why are the changes needed? Performance improvement. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass existed tests. Closes #27263 from Ngone51/spark_30433_followup. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 33 +++++++++++++------ 1 file changed, 23 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 89cd8e9648b3d..a96e04f043ac4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1055,34 +1055,43 @@ class Analyzer( logDebug(s"Conflicting attributes ${conflictingAttributes.mkString(",")} " + s"between $left and $right") - val conflictPlans = right.collect { + /** + * For LogicalPlan likes MultiInstanceRelation, Project, Aggregate, etc, whose output doesn't + * inherit directly from its children, we could just stop collect on it. Because we could + * always replace all the lower conflict attributes with the new attributes from the new + * plan. Theoretically, we should do recursively collect for Generate and Window but we leave + * it to the next batch to reduce possible overhead because this should be a corner case. + */ + def collectConflictPlans(plan: LogicalPlan): Seq[(LogicalPlan, LogicalPlan)] = plan match { // Handle base relations that might appear more than once. case oldVersion: MultiInstanceRelation if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => val newVersion = oldVersion.newInstance() - (oldVersion, newVersion) + Seq((oldVersion, newVersion)) case oldVersion: SerializeFromObject if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - (oldVersion, oldVersion.copy(serializer = oldVersion.serializer.map(_.newInstance()))) + Seq((oldVersion, oldVersion.copy( + serializer = oldVersion.serializer.map(_.newInstance())))) // Handle projects that create conflicting aliases. case oldVersion @ Project(projectList, _) if findAliases(projectList).intersect(conflictingAttributes).nonEmpty => - (oldVersion, oldVersion.copy(projectList = newAliases(projectList))) + Seq((oldVersion, oldVersion.copy(projectList = newAliases(projectList)))) case oldVersion @ Aggregate(_, aggregateExpressions, _) if findAliases(aggregateExpressions).intersect(conflictingAttributes).nonEmpty => - (oldVersion, oldVersion.copy(aggregateExpressions = newAliases(aggregateExpressions))) + Seq((oldVersion, oldVersion.copy( + aggregateExpressions = newAliases(aggregateExpressions)))) case oldVersion @ FlatMapGroupsInPandas(_, _, output, _) if oldVersion.outputSet.intersect(conflictingAttributes).nonEmpty => - (oldVersion, oldVersion.copy(output = output.map(_.newInstance()))) + Seq((oldVersion, oldVersion.copy(output = output.map(_.newInstance())))) case oldVersion: Generate if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => val newOutput = oldVersion.generatorOutput.map(_.newInstance()) - (oldVersion, oldVersion.copy(generatorOutput = newOutput)) + Seq((oldVersion, oldVersion.copy(generatorOutput = newOutput))) case oldVersion: Expand if oldVersion.producedAttributes.intersect(conflictingAttributes).nonEmpty => @@ -1094,14 +1103,18 @@ class Analyzer( attr } } - (oldVersion, oldVersion.copy(output = newOutput)) + Seq((oldVersion, oldVersion.copy(output = newOutput))) case oldVersion @ Window(windowExpressions, _, _, child) if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) - .nonEmpty => - (oldVersion, oldVersion.copy(windowExpressions = newAliases(windowExpressions))) + .nonEmpty => + Seq((oldVersion, oldVersion.copy(windowExpressions = newAliases(windowExpressions)))) + + case _ => plan.children.flatMap(collectConflictPlans) } + val conflictPlans = collectConflictPlans(right) + /* * Note that it's possible `conflictPlans` can be empty which implies that there * is a logical plan node that produces new references that this rule cannot handle. From 595cdb09a42744ce66a33c6b6eb5bad40d15753e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 21 Jan 2020 14:45:50 +0800 Subject: [PATCH 1172/1387] [SPARK-30571][CORE] fix splitting shuffle fetch requests ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/26930 to fix a bug. When we create shuffle fetch requests, we first collect blocks until they reach the max size. Then we try to merge the blocks (the batch shuffle fetch feature) and split the merged blocks to several groups, to make sure each group doesn't reach the max numBlocks. For the last group, if it's smaller than the max numBlocks, put it back to the input list and deal with it again later. The last step has a problem: 1. if we put a merged block back to the input list and merge it again, it fails. 2. when putting back some blocks, we should update `numBlocksToFetch` This PR fixes these 2 problems. ### Why are the changes needed? bug fix ### Does this PR introduce any user-facing change? no ### How was this patch tested? new test Closes #27280 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../storage/ShuffleBlockFetcherIterator.scala | 116 +++++++++++------- .../ShuffleBlockFetcherIteratorSuite.scala | 34 +++-- 2 files changed, 97 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index b9b06dc566ea1..97e0600da4a99 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -332,37 +332,47 @@ final class ShuffleBlockFetcherIterator( val iterator = blockInfos.iterator var curRequestSize = 0L var curBlocks = new ArrayBuffer[FetchBlockInfo] + + def createFetchRequest(blocks: Seq[FetchBlockInfo]): Unit = { + collectedRemoteRequests += FetchRequest(address, blocks) + logDebug(s"Creating fetch request of $curRequestSize at $address " + + s"with ${blocks.size} blocks") + } + + def createFetchRequests(): Unit = { + val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) + curBlocks = new ArrayBuffer[FetchBlockInfo] + if (mergedBlocks.length <= maxBlocksInFlightPerAddress) { + createFetchRequest(mergedBlocks) + } else { + mergedBlocks.grouped(maxBlocksInFlightPerAddress).foreach { blocks => + if (blocks.length == maxBlocksInFlightPerAddress) { + createFetchRequest(blocks) + } else { + // The last group does not exceed `maxBlocksInFlightPerAddress`. Put it back + // to `curBlocks`. + curBlocks = blocks + numBlocksToFetch -= blocks.size + } + } + } + curRequestSize = curBlocks.map(_.size).sum + } + while (iterator.hasNext) { val (blockId, size, mapIndex) = iterator.next() assertPositiveBlockSize(blockId, size) curBlocks += FetchBlockInfo(blockId, size, mapIndex) curRequestSize += size // For batch fetch, the actual block in flight should count for merged block. - val exceedsMaxBlocksInFlightPerAddress = !doBatchFetch && - curBlocks.size >= maxBlocksInFlightPerAddress - if (curRequestSize >= targetRemoteRequestSize || exceedsMaxBlocksInFlightPerAddress) { - // Add this FetchRequest - val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) - .grouped(maxBlocksInFlightPerAddress) - curBlocks = new ArrayBuffer[FetchBlockInfo] - mergedBlocks.foreach { mergedBlock => - if (mergedBlock.size == maxBlocksInFlightPerAddress) { - collectedRemoteRequests += new FetchRequest(address, mergedBlock) - logDebug(s"Creating fetch request of $curRequestSize at $address " - + s"with ${mergedBlock.size} blocks") - } else { - // The last group does not exceed `maxBlocksInFlightPerAddress`. Put it back - // to `curBlocks`. - curBlocks = mergedBlock - } - } - curRequestSize = 0 + val mayExceedsMaxBlocks = !doBatchFetch && curBlocks.size >= maxBlocksInFlightPerAddress + if (curRequestSize >= targetRemoteRequestSize || mayExceedsMaxBlocks) { + createFetchRequests() } } // Add in the final request if (curBlocks.nonEmpty) { - val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks) - collectedRemoteRequests += new FetchRequest(address, mergedBlocks) + createFetchRequests() } } @@ -380,35 +390,57 @@ final class ShuffleBlockFetcherIterator( private[this] def mergeContinuousShuffleBlockIdsIfNeeded( blocks: ArrayBuffer[FetchBlockInfo]): ArrayBuffer[FetchBlockInfo] = { - - def mergeFetchBlockInfo(toBeMerged: ArrayBuffer[FetchBlockInfo]): FetchBlockInfo = { - val startBlockId = toBeMerged.head.blockId.asInstanceOf[ShuffleBlockId] - FetchBlockInfo( - ShuffleBlockBatchId( - startBlockId.shuffleId, - startBlockId.mapId, - startBlockId.reduceId, - toBeMerged.last.blockId.asInstanceOf[ShuffleBlockId].reduceId + 1), - toBeMerged.map(_.size).sum, - toBeMerged.head.mapIndex) - } - val result = if (doBatchFetch) { var curBlocks = new ArrayBuffer[FetchBlockInfo] val mergedBlockInfo = new ArrayBuffer[FetchBlockInfo] - val iter = blocks.iterator + def mergeFetchBlockInfo(toBeMerged: ArrayBuffer[FetchBlockInfo]): FetchBlockInfo = { + val startBlockId = toBeMerged.head.blockId.asInstanceOf[ShuffleBlockId] + + // The last merged block may comes from the input, and we can merge more blocks + // into it, if the map id is the same. + def shouldMergeIntoPreviousBatchBlockId = + mergedBlockInfo.last.blockId.asInstanceOf[ShuffleBlockBatchId].mapId == startBlockId.mapId + + val startReduceId = if (mergedBlockInfo.nonEmpty && shouldMergeIntoPreviousBatchBlockId) { + // Remove the previous batch block id as we will add a new one to replace it. + mergedBlockInfo.remove(mergedBlockInfo.length - 1).blockId + .asInstanceOf[ShuffleBlockBatchId].startReduceId + } else { + startBlockId.reduceId + } + + FetchBlockInfo( + ShuffleBlockBatchId( + startBlockId.shuffleId, + startBlockId.mapId, + startReduceId, + toBeMerged.last.blockId.asInstanceOf[ShuffleBlockId].reduceId + 1), + toBeMerged.map(_.size).sum, + toBeMerged.head.mapIndex) + } + + val iter = blocks.iterator while (iter.hasNext) { val info = iter.next() - val curBlockId = info.blockId.asInstanceOf[ShuffleBlockId] - if (curBlocks.isEmpty) { - curBlocks += info + // It's possible that the input block id is already a batch ID. For example, we merge some + // blocks, and then make fetch requests with the merged blocks according to "max blocks per + // request". The last fetch request may be too small, and we give up and put the remaining + // merged blocks back to the input list. + if (info.blockId.isInstanceOf[ShuffleBlockBatchId]) { + mergedBlockInfo += info } else { - if (curBlockId.mapId != curBlocks.head.blockId.asInstanceOf[ShuffleBlockId].mapId) { - mergedBlockInfo += mergeFetchBlockInfo(curBlocks) - curBlocks.clear() + if (curBlocks.isEmpty) { + curBlocks += info + } else { + val curBlockId = info.blockId.asInstanceOf[ShuffleBlockId] + val currentMapId = curBlocks.head.blockId.asInstanceOf[ShuffleBlockId].mapId + if (curBlockId.mapId != currentMapId) { + mergedBlockInfo += mergeFetchBlockInfo(curBlocks) + curBlocks.clear() + } + curBlocks += info } - curBlocks += info } } if (curBlocks.nonEmpty) { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 311aa22d607ed..45f47c7c49bca 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -341,7 +341,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT assert(blockManager.hostLocalDirManager.get.getCachedHostLocalDirs().size === 1) } - test("fetch continuous blocks in batch respects maxBlocksInFlightPerAddress") { + test("fetch continuous blocks in batch respects maxSize and maxBlocks") { val blockManager = mock(classOf[BlockManager]) val localBmId = BlockManagerId("test-client", "test-local-host", 1) doReturn(localBmId).when(blockManager).blockManagerId @@ -352,9 +352,15 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT ShuffleBlockId(0, 3, 0), ShuffleBlockId(0, 3, 1), ShuffleBlockId(0, 3, 2), - ShuffleBlockId(0, 3, 3)) + ShuffleBlockId(0, 4, 0), + ShuffleBlockId(0, 4, 1), + ShuffleBlockId(0, 5, 0), + ShuffleBlockId(0, 5, 1), + ShuffleBlockId(0, 5, 2)) val mergedRemoteBlocks = Map[BlockId, ManagedBuffer]( - ShuffleBlockBatchId(0, 3, 0, 4) -> createMockManagedBuffer()) + ShuffleBlockBatchId(0, 3, 0, 3) -> createMockManagedBuffer(), + ShuffleBlockBatchId(0, 4, 0, 2) -> createMockManagedBuffer(), + ShuffleBlockBatchId(0, 5, 0, 3) -> createMockManagedBuffer()) val transfer = createMockTransfer(mergedRemoteBlocks) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( @@ -369,21 +375,27 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT blockManager, blocksByAddress, (_, in) => in, - 48 * 1024 * 1024, + 35, Int.MaxValue, - 1, + 2, Int.MaxValue, true, false, metrics, true) - assert(iterator.hasNext) - val (blockId, inputStream) = iterator.next() - verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any(), any()) - // Make sure we release buffers when a wrapped input stream is closed. - val mockBuf = mergedRemoteBlocks(blockId) - verifyBufferRelease(mockBuf, inputStream) + var numResults = 0 + while (iterator.hasNext) { + val (blockId, inputStream) = iterator.next() + // Make sure we release buffers when a wrapped input stream is closed. + val mockBuf = mergedRemoteBlocks(blockId) + verifyBufferRelease(mockBuf, inputStream) + numResults += 1 + } + // The first 2 batch block ids are in the same fetch request as they don't exceed the max size + // and max blocks, so 2 requests in total. + verify(transfer, times(2)).fetchBlocks(any(), any(), any(), any(), any(), any()) + assert(numResults == 3) } test("release current unexhausted buffer in case the task completes early") { From a94a4fcf903fdbb20c3916e0834c4b3979b2d8e9 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 20 Jan 2020 23:08:24 -0800 Subject: [PATCH 1173/1387] [MINOR][DOCS] Fix Jenkins build image and link in README.md ### What changes were proposed in this pull request? Jenkins link in README.md is currently broken: ![Screen Shot 2020-01-21 at 3 11 10 PM](https://user-images.githubusercontent.com/6477701/72779777-678c5b00-3c60-11ea-8523-9d82abc0493e.png) Seems new jobs are configured to test Hive 1.2 and 2.3 profiles. The link pointed out `spark-master-test-maven-hadoop-2.7` before. Now it become two. ``` spark-master-test-maven-hadoop-2.7 -> spark-master-test-maven-hadoop-2.7-hive-2.3 spark-master-test-maven-hadoop-2.7-hive-1.2 ``` Since the PR builder uses Hive 2.3 by default, this PR fixes the link to point out `spark-master-test-maven-hadoop-2.7-hive-2.3` ### Why are the changes needed? To fix the image and broken link. ### Does this PR introduce any user-facing change? No. Dev only change. ### How was this patch tested? Manually clicking. Closes #27301 from HyukjinKwon/minor-link. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index a632293646ab5..d7931263b0fc7 100644 --- a/README.md +++ b/README.md @@ -9,7 +9,7 @@ and Structured Streaming for stream processing. -[![Jenkins Build](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/badge/icon)](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7) +[![Jenkins Build](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7-hive-2.3/badge/icon)](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7-hive-2.3) [![AppVeyor Build](https://img.shields.io/appveyor/ci/ApacheSoftwareFoundation/spark/master.svg?style=plastic&logo=appveyor)](https://ci.appveyor.com/project/ApacheSoftwareFoundation/spark) [![PySpark Coverage](https://img.shields.io/badge/dynamic/xml.svg?label=pyspark%20coverage&url=https%3A%2F%2Fspark-test.github.io%2Fpyspark-coverage-site&query=%2Fhtml%2Fbody%2Fdiv%5B1%5D%2Fdiv%2Fh1%2Fspan&colorB=brightgreen&style=plastic)](https://spark-test.github.io/pyspark-coverage-site) From e170422f74650b71c1eba408ac62020c31755e2b Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 21 Jan 2020 18:23:03 +0900 Subject: [PATCH 1174/1387] Revert "[SPARK-30534][INFRA] Use mvn in `dev/scalastyle`" This reverts commit 384899944b25cb0abf5e71f9cc2610fecad4e8f5. --- dev/scalastyle | 14 +++++++++++--- pom.xml | 2 +- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/dev/scalastyle b/dev/scalastyle index b555ce08b789e..212ef900eb9b4 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,10 +17,18 @@ # limitations under the License. # -SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" +SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"} -SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive -Pdocker-integration-tests -Pkubernetes-integration-tests"} -ERRORS=$($SCRIPT_DIR/../build/mvn $SPARK_PROFILES scalastyle:check | grep "^error file") +# NOTE: echo "q" is needed because SBT prompts the user for input on encountering a build file +# with failure (either resolution or compilation); the "q" makes SBT quit. +ERRORS=$(echo -e "q\n" \ + | build/sbt \ + ${SPARK_PROFILES} \ + -Pdocker-integration-tests \ + -Pkubernetes-integration-tests \ + scalastyle test:scalastyle \ + | awk '{if($1~/error/)print}' \ +) if test ! -z "$ERRORS"; then echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" diff --git a/pom.xml b/pom.xml index ef8fb709bbb59..d2b2b0205060f 100644 --- a/pom.xml +++ b/pom.xml @@ -2776,7 +2776,7 @@ false true - true + false false ${basedir}/src/main/scala ${basedir}/src/test/scala From 730388b3693953795087cdc1eec31005f9eab98c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 21 Jan 2020 20:35:47 +0800 Subject: [PATCH 1175/1387] [SPARK-30547][SQL][FOLLOWUP] Update since anotation for CalendarInterval class ### What changes were proposed in this pull request? Mark `CalendarInterval` class with `since 3.0.0`. ### Why are the changes needed? https://www.oracle.com/technetwork/java/javase/documentation/index-137868.html#since This class is the first time going to the public, the annotation is the first time to add, and we don't want people to get confused and try to use it 2.4.x. ### Does this PR introduce any user-facing change? no ### How was this patch tested? no Closes #27299 from yaooqinn/SPARK-30547-F. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../java/org/apache/spark/unsafe/types/CalendarInterval.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 83d02ddba48a6..884db6fe27c4c 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -41,7 +41,7 @@ * they are two separated fields from microseconds. One month may be equal to 28, 29, 30 or 31 days * and one day may be equal to 23, 24 or 25 hours (daylight saving). * - * @since 1.5.0 + * @since 3.0.0 */ @Unstable public final class CalendarInterval implements Serializable { From af705421db172471d57d92fc7feee3179169a68e Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 21 Jan 2020 20:51:10 +0800 Subject: [PATCH 1176/1387] [SPARK-30593][SQL] Revert interval ISO/ANSI SQL Standard output since we decide not to follow ANSI and no round trip ### What changes were proposed in this pull request? This revert https://github.com/apache/spark/pull/26418, file a new ticket under https://issues.apache.org/jira/browse/SPARK-30546 for better tracking interval behavior ### Why are the changes needed? Revert interval ISO/ANSI SQL Standard output since we decide not to follow ANSI and there is no round trip ### Does this PR introduce any user-facing change? no, not released yet ### How was this patch tested? existing uts Closes #27304 from yaooqinn/SPARK-30593. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../unsafe/types/CalendarIntervalSuite.java | 30 +++++++ .../spark/sql/catalyst/expressions/Cast.scala | 20 +---- .../sql/catalyst/expressions/literals.scala | 2 +- .../sql/catalyst/util/IntervalUtils.scala | 79 ------------------- .../apache/spark/sql/internal/SQLConf.scala | 19 ----- .../catalyst/util/IntervalUtilsSuite.scala | 64 --------------- .../spark/sql/execution/HiveResult.scala | 9 +-- .../inputs/interval-display-iso_8601.sql | 3 - .../inputs/interval-display-sql_standard.sql | 3 - .../sql-tests/inputs/interval-display.sql | 14 ---- .../sql-tests/inputs/postgreSQL/interval.sql | 10 +-- .../results/interval-display-iso_8601.sql.out | 21 ----- .../interval-display-sql_standard.sql.out | 21 ----- .../results/interval-display.sql.out | 21 ----- .../results/postgreSQL/interval.sql.out | 29 +------ .../apache/spark/sql/DateFunctionsSuite.scala | 8 +- 16 files changed, 44 insertions(+), 309 deletions(-) delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/interval-display.sql delete mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/interval-display.sql.out diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 01bf7eb2438ad..6397f26c02f3a 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -46,6 +46,36 @@ public void equalsTest() { assertEquals(i1, i6); } + @Test + public void toStringTest() { + CalendarInterval i; + + i = new CalendarInterval(0, 0, 0); + assertEquals("0 seconds", i.toString()); + + i = new CalendarInterval(34, 0, 0); + assertEquals("2 years 10 months", i.toString()); + + i = new CalendarInterval(-34, 0, 0); + assertEquals("-2 years -10 months", i.toString()); + + i = new CalendarInterval(0, 31, 0); + assertEquals("31 days", i.toString()); + + i = new CalendarInterval(0, -31, 0); + assertEquals("-31 days", i.toString()); + + i = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); + assertEquals("3 hours 13 minutes 0.000123 seconds", i.toString()); + + i = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123); + assertEquals("-3 hours -13 minutes -0.000123 seconds", i.toString()); + + i = new CalendarInterval(34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123); + assertEquals("2 years 10 months 31 days 3 hours 13 minutes 0.000123 seconds", + i.toString()); + } + @Test public void periodAndDurationTest() { CalendarInterval interval = new CalendarInterval(120, -40, 123456); 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 4fd74a4e4658b..05b4fbef2b697 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 @@ -30,9 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ -import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.IntervalStyle._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -283,14 +281,8 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { - case CalendarIntervalType => SQLConf.get.intervalOutputStyle match { - case SQL_STANDARD => - buildCast[CalendarInterval](_, i => UTF8String.fromString(toSqlStandardString(i))) - case ISO_8601 => - buildCast[CalendarInterval](_, i => UTF8String.fromString(toIso8601String(i))) - case MULTI_UNITS => - buildCast[CalendarInterval](_, i => UTF8String.fromString(toMultiUnitsString(i))) - } + case CalendarIntervalType => + buildCast[CalendarInterval](_, i => UTF8String.fromString(i.toString)) case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes) case DateType => buildCast[Int](_, d => UTF8String.fromString(dateFormatter.format(d))) case TimestampType => buildCast[Long](_, @@ -1021,13 +1013,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString( org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($tf, $c));""" case CalendarIntervalType => - val iu = IntervalUtils.getClass.getCanonicalName.stripSuffix("$") - val funcName = SQLConf.get.intervalOutputStyle match { - case SQL_STANDARD => "toSqlStandardString" - case ISO_8601 => "toIso8601String" - case MULTI_UNITS => "toMultiUnitsString" - } - (c, evPrim, _) => code"""$evPrim = UTF8String.fromString($iu.$funcName($c));""" + (c, evPrim, _) => code"""$evPrim = UTF8String.fromString($c.toString());""" case ArrayType(et, _) => (c, evPrim, evNull) => { val buffer = ctx.freshVariable("buffer", classOf[UTF8StringBuilder]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 48b8c9c0fbf8b..371a29ecf83ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -409,7 +409,7 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) s"TIMESTAMP '${formatter.format(v)}'" case (i: CalendarInterval, CalendarIntervalType) => - s"INTERVAL '${IntervalUtils.toMultiUnitsString(i)}'" + s"INTERVAL '${i.toString}'" case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'" case _ => value.toString } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 8763f24b05edc..7692299a46ef5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -497,85 +497,6 @@ object IntervalUtils { fromDoubles(interval.months / num, interval.days / num, interval.microseconds / num) } - // `toString` implementation in CalendarInterval is the multi-units format currently. - def toMultiUnitsString(interval: CalendarInterval): String = interval.toString - - def toSqlStandardString(interval: CalendarInterval): String = { - val yearMonthPart = if (interval.months < 0) { - val ma = math.abs(interval.months) - "-" + ma / 12 + "-" + ma % 12 - } else if (interval.months > 0) { - "+" + interval.months / 12 + "-" + interval.months % 12 - } else { - "" - } - - val dayPart = if (interval.days < 0) { - interval.days.toString - } else if (interval.days > 0) { - "+" + interval.days - } else { - "" - } - - val timePart = if (interval.microseconds != 0) { - val sign = if (interval.microseconds > 0) "+" else "-" - val sb = new StringBuilder(sign) - var rest = math.abs(interval.microseconds) - sb.append(rest / MICROS_PER_HOUR) - sb.append(':') - rest %= MICROS_PER_HOUR - val minutes = rest / MICROS_PER_MINUTE; - if (minutes < 10) { - sb.append(0) - } - sb.append(minutes) - sb.append(':') - rest %= MICROS_PER_MINUTE - val bd = BigDecimal.valueOf(rest, 6) - if (bd.compareTo(new BigDecimal(10)) < 0) { - sb.append(0) - } - val s = bd.stripTrailingZeros().toPlainString - sb.append(s) - sb.toString() - } else { - "" - } - - val intervalList = Seq(yearMonthPart, dayPart, timePart).filter(_.nonEmpty) - if (intervalList.nonEmpty) intervalList.mkString(" ") else "0" - } - - def toIso8601String(interval: CalendarInterval): String = { - val sb = new StringBuilder("P") - - val year = interval.months / 12 - if (year != 0) sb.append(year + "Y") - val month = interval.months % 12 - if (month != 0) sb.append(month + "M") - - if (interval.days != 0) sb.append(interval.days + "D") - - if (interval.microseconds != 0) { - sb.append('T') - var rest = interval.microseconds - val hour = rest / MICROS_PER_HOUR - if (hour != 0) sb.append(hour + "H") - rest %= MICROS_PER_HOUR - val minute = rest / MICROS_PER_MINUTE - if (minute != 0) sb.append(minute + "M") - rest %= MICROS_PER_MINUTE - if (rest != 0) { - val bd = BigDecimal.valueOf(rest, 6) - sb.append(bd.stripTrailingZeros().toPlainString + "S") - } - } else if (interval.days == 0 && interval.months == 0) { - sb.append("T0S") - } - sb.toString() - } - private object ParseState extends Enumeration { type ParseState = Value diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 279c79ff14080..45ca022c36f72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1799,23 +1799,6 @@ object SQLConf { .checkValues(StoreAssignmentPolicy.values.map(_.toString)) .createWithDefault(StoreAssignmentPolicy.ANSI.toString) - object IntervalStyle extends Enumeration { - type IntervalStyle = Value - val SQL_STANDARD, ISO_8601, MULTI_UNITS = Value - } - - val INTERVAL_STYLE = buildConf("spark.sql.intervalOutputStyle") - .doc("When converting interval values to strings (i.e. for display), this config decides the" + - " interval string format. The value SQL_STANDARD will produce output matching SQL standard" + - " interval literals (i.e. '+3-2 +10 -00:00:01'). The value ISO_8601 will produce output" + - " matching the ISO 8601 standard (i.e. 'P3Y2M10DT-1S'). The value MULTI_UNITS (which is the" + - " default) will produce output in form of value unit pairs, (i.e. '3 year 2 months 10 days" + - " -1 seconds'") - .stringConf - .transform(_.toUpperCase(Locale.ROOT)) - .checkValues(IntervalStyle.values.map(_.toString)) - .createWithDefault(IntervalStyle.MULTI_UNITS.toString) - val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + @@ -2667,8 +2650,6 @@ class SQLConf extends Serializable with Logging { def storeAssignmentPolicy: StoreAssignmentPolicy.Value = StoreAssignmentPolicy.withName(getConf(STORE_ASSIGNMENT_POLICY)) - def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE)) - def ansiEnabled: Boolean = getConf(ANSI_ENABLED) def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala index a0b4387331e5d..514804cbda16c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala @@ -304,70 +304,6 @@ class IntervalUtilsSuite extends SparkFunSuite with SQLHelper { } } - test("to ansi sql standard string") { - val i1 = new CalendarInterval(0, 0, 0) - assert(IntervalUtils.toSqlStandardString(i1) === "0") - val i2 = new CalendarInterval(34, 0, 0) - assert(IntervalUtils.toSqlStandardString(i2) === "+2-10") - val i3 = new CalendarInterval(-34, 0, 0) - assert(IntervalUtils.toSqlStandardString(i3) === "-2-10") - val i4 = new CalendarInterval(0, 31, 0) - assert(IntervalUtils.toSqlStandardString(i4) === "+31") - val i5 = new CalendarInterval(0, -31, 0) - assert(IntervalUtils.toSqlStandardString(i5) === "-31") - val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) - assert(IntervalUtils.toSqlStandardString(i6) === "+3:13:00.000123") - val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) - assert(IntervalUtils.toSqlStandardString(i7) === "-3:13:00.000123") - val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) - assert(IntervalUtils.toSqlStandardString(i8) === "-2-10 +31 +3:13:00.000123") - val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) - assert(IntervalUtils.toSqlStandardString(i9) === "-3000:00:00") - } - - test("to iso 8601 string") { - val i1 = new CalendarInterval(0, 0, 0) - assert(IntervalUtils.toIso8601String(i1) === "PT0S") - val i2 = new CalendarInterval(34, 0, 0) - assert(IntervalUtils.toIso8601String(i2) === "P2Y10M") - val i3 = new CalendarInterval(-34, 0, 0) - assert(IntervalUtils.toIso8601String(i3) === "P-2Y-10M") - val i4 = new CalendarInterval(0, 31, 0) - assert(IntervalUtils.toIso8601String(i4) === "P31D") - val i5 = new CalendarInterval(0, -31, 0) - assert(IntervalUtils.toIso8601String(i5) === "P-31D") - val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) - assert(IntervalUtils.toIso8601String(i6) === "PT3H13M0.000123S") - val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) - assert(IntervalUtils.toIso8601String(i7) === "PT-3H-13M-0.000123S") - val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) - assert(IntervalUtils.toIso8601String(i8) === "P-2Y-10M31DT3H13M0.000123S") - val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) - assert(IntervalUtils.toIso8601String(i9) === "PT-3000H") - } - - test("to multi units string") { - val i1 = new CalendarInterval(0, 0, 0) - assert(IntervalUtils.toMultiUnitsString(i1) === "0 seconds") - val i2 = new CalendarInterval(34, 0, 0) - assert(IntervalUtils.toMultiUnitsString(i2) === "2 years 10 months") - val i3 = new CalendarInterval(-34, 0, 0) - assert(IntervalUtils.toMultiUnitsString(i3) === "-2 years -10 months") - val i4 = new CalendarInterval(0, 31, 0) - assert(IntervalUtils.toMultiUnitsString(i4) === "31 days") - val i5 = new CalendarInterval(0, -31, 0) - assert(IntervalUtils.toMultiUnitsString(i5) === "-31 days") - val i6 = new CalendarInterval(0, 0, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) - assert(IntervalUtils.toMultiUnitsString(i6) === "3 hours 13 minutes 0.000123 seconds") - val i7 = new CalendarInterval(0, 0, -3 * MICROS_PER_HOUR - 13 * MICROS_PER_MINUTE - 123) - assert(IntervalUtils.toMultiUnitsString(i7) === "-3 hours -13 minutes -0.000123 seconds") - val i8 = new CalendarInterval(-34, 31, 3 * MICROS_PER_HOUR + 13 * MICROS_PER_MINUTE + 123) - assert(IntervalUtils.toMultiUnitsString(i8) === - "-2 years -10 months 31 days 3 hours 13 minutes 0.000123 seconds") - val i9 = new CalendarInterval(0, 0, -3000 * MICROS_PER_HOUR) - assert(IntervalUtils.toMultiUnitsString(i9) === "-3000 hours") - } - test("from day-time string") { def check(input: String, from: IntervalUnit, to: IntervalUnit, expected: String): Unit = { withClue(s"from = $from, to = $to") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index c92b10cc03645..bbe47a63f4d61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -22,10 +22,8 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} -import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.execution.command.{DescribeCommandBase, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.IntervalStyle._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -75,12 +73,7 @@ object HiveResult { case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString case (n, _: NumericType) => n.toString case (s: String, StringType) => if (nested) "\"" + s + "\"" else s - case (interval: CalendarInterval, CalendarIntervalType) => - SQLConf.get.intervalOutputStyle match { - case SQL_STANDARD => toSqlStandardString(interval) - case ISO_8601 => toIso8601String(interval) - case MULTI_UNITS => toMultiUnitsString(interval) - } + case (interval: CalendarInterval, CalendarIntervalType) => interval.toString case (seq: Seq[_], ArrayType(typ, _)) => seq.map(v => (v, typ)).map(e => toHiveString(e, true)).mkString("[", ",", "]") case (m: Map[_, _], MapType(kType, vType, _)) => diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql deleted file mode 100644 index 3b63c715a6aa1..0000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/interval-display-iso_8601.sql +++ /dev/null @@ -1,3 +0,0 @@ --- tests for interval output style with iso_8601 format ---SET spark.sql.intervalOutputStyle = ISO_8601 ---IMPORT interval-display.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql deleted file mode 100644 index d96865b160bb6..0000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/interval-display-sql_standard.sql +++ /dev/null @@ -1,3 +0,0 @@ --- tests for interval output style with sql standard format ---SET spark.sql.intervalOutputStyle = SQL_STANDARD ---IMPORT interval-display.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql b/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql deleted file mode 100644 index ae19f1b6374ba..0000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/interval-display.sql +++ /dev/null @@ -1,14 +0,0 @@ --- tests for interval output style - -SELECT - cast(null as interval), -- null - interval '0 day', -- 0 - interval '1 year', -- year only - interval '1 month', -- month only - interval '1 year 2 month', -- year month only - interval '1 day -1 hours', - interval '-1 day -1 hours', - interval '-1 day 1 hours', - interval '-1 days +1 hours', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql index 7edcae59c49ff..eb8cc34419519 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/interval.sql @@ -272,12 +272,10 @@ SELECT interval '1 2:03:04' minute to second; -- test output of couple non-standard interval values in the sql style -- [SPARK-29406] Interval output styles -- SET IntervalStyle TO sql_standard; -set spark.sql.intervalOutputStyle=SQL_STANDARD; -SELECT interval '1 day -1 hours', - interval '-1 days +1 hours', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; -set spark.sql.intervalOutputStyle=MULTI_UNITS; +-- SELECT interval '1 day -1 hours', +-- interval '-1 days +1 hours', +-- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', +-- - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; -- test outputting iso8601 intervals -- [SPARK-29406] Interval output styles diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out deleted file mode 100644 index 2da098f17e168..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-iso_8601.sql.out +++ /dev/null @@ -1,21 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- Number of queries: 1 - - --- !query 0 -SELECT - cast(null as interval), -- null - interval '0 day', -- 0 - interval '1 year', -- year only - interval '1 month', -- month only - interval '1 year 2 month', -- year month only - interval '1 day -1 hours', - interval '-1 day -1 hours', - interval '-1 day 1 hours', - interval '-1 days +1 hours', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' --- !query 0 schema -struct --- !query 0 output -NULL PT0S P1Y P1M P1Y2M P1DT-1H P-1DT-1H P-1DT1H P-1DT1H P1Y2M-3DT4H5M6.789S P-1Y-2M3DT-4H-5M-6.789S diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out deleted file mode 100644 index 7ef2f9c51218e..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/interval-display-sql_standard.sql.out +++ /dev/null @@ -1,21 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- Number of queries: 1 - - --- !query 0 -SELECT - cast(null as interval), -- null - interval '0 day', -- 0 - interval '1 year', -- year only - interval '1 month', -- month only - interval '1 year 2 month', -- year month only - interval '1 day -1 hours', - interval '-1 day -1 hours', - interval '-1 day 1 hours', - interval '-1 days +1 hours', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' --- !query 0 schema -struct --- !query 0 output -NULL 0 +1-0 +0-1 +1-2 +1 -1:00:00 -1 -1:00:00 -1 +1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 diff --git a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out b/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out deleted file mode 100644 index a292fe3c83028..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/interval-display.sql.out +++ /dev/null @@ -1,21 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- Number of queries: 1 - - --- !query 0 -SELECT - cast(null as interval), -- null - interval '0 day', -- 0 - interval '1 year', -- year only - interval '1 month', -- month only - interval '1 year 2 month', -- year month only - interval '1 day -1 hours', - interval '-1 day -1 hours', - interval '-1 day 1 hours', - interval '-1 days +1 hours', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' --- !query 0 schema -struct --- !query 0 output -NULL 0 seconds 1 years 1 months 1 years 2 months 1 days -1 hours -1 days -1 hours -1 days 1 hours -1 days 1 hours 1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds -1 years -2 months 3 days -4 hours -5 minutes -6.789 seconds diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index 6a6aed1d9bf68..cb906c53c84e7 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 27 +-- Number of queries: 24 -- !query 0 @@ -252,30 +252,3 @@ requirement failed: Interval string must match day-time format of '^(?[+|- == SQL == SELECT interval '1 2:03:04' minute to second ----------------^^^ - - --- !query 24 -set spark.sql.intervalOutputStyle=SQL_STANDARD --- !query 24 schema -struct --- !query 24 output -spark.sql.intervalOutputStyle SQL_STANDARD - - --- !query 25 -SELECT interval '1 day -1 hours', - interval '-1 days +1 hours', - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', - - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds' --- !query 25 schema -struct --- !query 25 output -+1 -1:00:00 -1 +1:00:00 +1-2 -3 +4:05:06.789 -1-2 +3 -4:05:06.789 - - --- !query 26 -set spark.sql.intervalOutputStyle=MULTI_UNITS --- !query 26 schema -struct --- !query 26 output -spark.sql.intervalOutputStyle MULTI_UNITS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 7b6b935496678..72d323e88406e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -293,10 +293,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val i = new CalendarInterval(2, 2, 2000000L) val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( - df.selectExpr(s"d + INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), + df.selectExpr(s"d + INTERVAL'${i.toString}'"), Seq(Row(Date.valueOf("2015-10-02")), Row(Date.valueOf("2016-03-02")))) checkAnswer( - df.selectExpr(s"t + INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), + df.selectExpr(s"t + INTERVAL'${i.toString}'"), Seq(Row(Timestamp.valueOf("2015-10-03 00:00:01")), Row(Timestamp.valueOf("2016-03-02 00:00:02")))) } @@ -309,10 +309,10 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val i = new CalendarInterval(2, 2, 2000000L) val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( - df.selectExpr(s"d - INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), + df.selectExpr(s"d - INTERVAL'${i.toString}'"), Seq(Row(Date.valueOf("2015-07-27")), Row(Date.valueOf("2015-12-26")))) checkAnswer( - df.selectExpr(s"t - INTERVAL'${IntervalUtils.toMultiUnitsString(i)}'"), + df.selectExpr(s"t - INTERVAL'${i.toString}'"), Seq(Row(Timestamp.valueOf("2015-07-29 23:59:59")), Row(Timestamp.valueOf("2015-12-27 00:00:00")))) } From ff39c9271ca04951b045c5d9fca2128a82d50b46 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Tue, 21 Jan 2020 21:09:48 +0800 Subject: [PATCH 1177/1387] [SPARK-30252][SQL] Disallow negative scale of Decimal ### What changes were proposed in this pull request? This PR propose to disallow negative `scale` of `Decimal` in Spark. And this PR brings two behavior changes: 1) for literals like `1.23E4BD` or `1.23E4`(with `spark.sql.legacy.exponentLiteralAsDecimal.enabled`=true, see [SPARK-29956](https://issues.apache.org/jira/browse/SPARK-29956)), we set its `(precision, scale)` to (5, 0) rather than (3, -2); 2) add negative `scale` check inside the decimal method if it exposes to set `scale` explicitly. If check fails, `AnalysisException` throws. And user could still use `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled` to restore the previous behavior. ### Why are the changes needed? According to SQL standard, > 4.4.2 Characteristics of numbers An exact numeric type has a precision P and a scale S. P is a positive integer that determines the number of significant digits in a particular radix R, where R is either 2 or 10. S is a non-negative integer. scale of Decimal should always be non-negative. And other mainstream databases, like Presto, PostgreSQL, also don't allow negative scale. Presto: ``` presto:default> create table t (i decimal(2, -1)); Query 20191213_081238_00017_i448h failed: line 1:30: mismatched input '-'. Expecting: , create table t (i decimal(2, -1)) ``` PostgrelSQL: ``` postgres=# create table t(i decimal(2, -1)); ERROR: NUMERIC scale -1 must be between 0 and precision 2 LINE 1: create table t(i decimal(2, -1)); ^ ``` And, actually, Spark itself already doesn't allow to create table with negative decimal types using SQL: ``` scala> spark.sql("create table t(i decimal(2, -1))"); org.apache.spark.sql.catalyst.parser.ParseException: no viable alternative at input 'create table t(i decimal(2, -'(line 1, pos 28) == SQL == create table t(i decimal(2, -1)) ----------------------------^^^ at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263) at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130) at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48) at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76) at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605) ... 35 elided ``` However, it is still possible to create such table or `DatFrame` using Spark SQL programming API: ``` scala> val tb = CatalogTable( TableIdentifier("test", None), CatalogTableType.MANAGED, CatalogStorageFormat.empty, StructType(StructField("i", DecimalType(2, -1) ) :: Nil)) ``` ``` scala> spark.sql("SELECT 1.23E4BD") res2: org.apache.spark.sql.DataFrame = [1.23E+4: decimal(3,-2)] ``` while, these two different behavior could make user confused. On the other side, even if user creates such table or `DataFrame` with negative scale decimal type, it can't write data out if using format, like `parquet` or `orc`. Because these formats have their own check for negative scale and fail on it. ``` scala> spark.sql("SELECT 1.23E4BD").write.saveAsTable("parquet") 19/12/13 17:37:04 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.lang.IllegalArgumentException: Invalid DECIMAL scale: -2 at org.apache.parquet.Preconditions.checkArgument(Preconditions.java:53) at org.apache.parquet.schema.Types$BasePrimitiveBuilder.decimalMetadata(Types.java:495) at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:403) at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:309) at org.apache.parquet.schema.Types$Builder.named(Types.java:290) at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:428) at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:334) at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.$anonfun$convert$2(ParquetSchemaConverter.scala:326) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238) at scala.collection.Iterator.foreach(Iterator.scala:941) at scala.collection.Iterator.foreach$(Iterator.scala:941) at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) at scala.collection.IterableLike.foreach(IterableLike.scala:74) at scala.collection.IterableLike.foreach$(IterableLike.scala:73) at org.apache.spark.sql.types.StructType.foreach(StructType.scala:99) at scala.collection.TraversableLike.map(TraversableLike.scala:238) at scala.collection.TraversableLike.map$(TraversableLike.scala:231) at org.apache.spark.sql.types.StructType.map(StructType.scala:99) at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convert(ParquetSchemaConverter.scala:326) at org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.init(ParquetWriteSupport.scala:97) at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:388) at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349) at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.(ParquetOutputWriter.scala:37) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:150) at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:124) at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.(FileFormatDataWriter.scala:109) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:264) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:127) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:441) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:444) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) ``` So, I think it would be better to disallow negative scale totally and make behaviors above be consistent. ### Does this PR introduce any user-facing change? Yes, if `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=false`, user couldn't create Decimal value with negative scale anymore. ### How was this patch tested? Added new tests in `ExpressionParserSuite` and `DecimalSuite`; Updated `SQLQueryTestSuite`. Closes #26881 from Ngone51/nonnegative-scale. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 ++ python/pyspark/sql/tests/test_types.py | 12 ++++--- python/pyspark/sql/types.py | 2 -- .../sql/catalyst/expressions/literals.scala | 7 ++-- .../apache/spark/sql/internal/SQLConf.scala | 12 +++++++ .../org/apache/spark/sql/types/Decimal.scala | 13 +++++++- .../apache/spark/sql/types/DecimalType.scala | 24 ++++++++++---- .../analysis/DecimalPrecisionSuite.scala | 15 +++++---- .../catalyst/csv/CSVInferSchemaSuite.scala | 11 ++++--- .../sql/catalyst/expressions/CastSuite.scala | 31 +++++++++-------- .../parser/ExpressionParserSuite.scala | 10 ++++++ .../apache/spark/sql/types/DecimalSuite.scala | 33 ++++++++++++++----- .../ansi/decimalArithmeticOperations.sql.out | 2 +- .../decimalArithmeticOperations.sql.out | 16 ++++----- .../subquery/in-subquery/in-group-by.sql.out | 2 +- .../subquery/in-subquery/in-limit.sql.out | 6 ++-- .../subquery/in-subquery/in-order-by.sql.out | 14 ++++---- .../in-subquery/in-set-operations.sql.out | 2 +- .../subquery/in-subquery/simple-in.sql.out | 4 +-- .../scalar-subquery-predicate.sql.out | 4 +-- .../apache/spark/sql/MathFunctionsSuite.scala | 29 ++++++++-------- .../datasources/json/JsonSuite.scala | 20 +++++------ .../datasources/json/TestJsonData.scala | 4 +-- .../ThriftServerQueryTestSuite.scala | 2 +- 24 files changed, 178 insertions(+), 99 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 2ddbdb9d88d71..eec1fb260b396 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -257,6 +257,8 @@ license: | - Since Spark 3.0, the unary arithmetic operator plus(`+`) only accepts string, numeric and interval type values as inputs. Besides, `+` with a integral string representation will be coerced to double value, e.g. `+'1'` results `1.0`. In Spark version 2.4 and earlier, this operator is ignored. There is no type checking for it, thus, all type values with a `+` prefix are valid, e.g. `+ array(1, 2)` is valid and results `[1, 2]`. Besides, there is no type coercion for it at all, e.g. in Spark 2.4, the result of `+'1'` is string `1`. - Since Spark 3.0, day-time interval strings are converted to intervals with respect to the `from` and `to` bounds. If an input string does not match to the pattern defined by specified bounds, the `ParseException` exception is thrown. For example, `interval '2 10:20' hour to minute` raises the exception because the expected format is `[+|-]h[h]:[m]m`. In Spark version 2.4, the `from` bound was not taken into account, and the `to` bound was used to truncate the resulted interval. For instance, the day-time interval string from the showed example is converted to `interval 10 hours 20 minutes`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.fromDayTimeString.enabled` to `true`. + + - Since Spark 3.0, negative scale of decimal is not allowed by default, e.g. data type of literal like `1E10BD` is `DecimalType(11, 0)`. In Spark version 2.4 and earlier, it was `DecimalType(2, -9)`. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled` to `true`. - Since Spark 3.0, the `date_add` and `date_sub` functions only accepts int, smallint, tinyint as the 2nd argument, fractional and string types are not valid anymore, e.g. `date_add(cast('1964-05-23' as date), '12.34')` will cause `AnalysisException`. In Spark version 2.4 and earlier, if the 2nd argument is fractional or string value, it will be coerced to int value, and the result will be a date value of `1964-06-04`. diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index b41c4381984c1..37ce62b7f7159 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -204,10 +204,14 @@ def test_create_dataframe_from_dict_respects_schema(self): self.assertEqual(df.columns, ['b']) def test_negative_decimal(self): - df = self.spark.createDataFrame([(1, ), (11, )], ["value"]) - ret = df.select(col("value").cast(DecimalType(1, -1))).collect() - actual = list(map(lambda r: int(r.value), ret)) - self.assertEqual(actual, [0, 10]) + try: + self.spark.sql("set spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=true") + df = self.spark.createDataFrame([(1, ), (11, )], ["value"]) + ret = df.select(col("value").cast(DecimalType(1, -1))).collect() + actual = list(map(lambda r: int(r.value), ret)) + self.assertEqual(actual, [0, 10]) + finally: + self.spark.sql("set spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=false") def test_create_dataframe_from_objects(self): data = [MyObject(1, "1"), MyObject(2, "2")] diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 112406aa5a6e5..8afff77b723a8 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -867,8 +867,6 @@ def _parse_datatype_json_string(json_string): >>> complex_maptype = MapType(complex_structtype, ... complex_arraytype, False) >>> check_datatype(complex_maptype) - >>> # Decimal with negative scale. - >>> check_datatype(DecimalType(1,-1)) """ return _parse_datatype_json_value(json.loads(json_string)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 371a29ecf83ab..213a58a3244e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -63,9 +63,12 @@ object Literal { case s: String => Literal(UTF8String.fromString(s), StringType) case c: Char => Literal(UTF8String.fromString(c.toString), StringType) case b: Boolean => Literal(b, BooleanType) - case d: BigDecimal => Literal(Decimal(d), DecimalType.fromBigDecimal(d)) + case d: BigDecimal => + val decimal = Decimal(d) + Literal(decimal, DecimalType.fromDecimal(decimal)) case d: JavaBigDecimal => - Literal(Decimal(d), DecimalType(Math.max(d.precision, d.scale), d.scale())) + val decimal = Decimal(d) + Literal(decimal, DecimalType.fromDecimal(decimal)) case d: Decimal => Literal(d, DecimalType(Math.max(d.precision, d.scale), d.scale)) case i: Instant => Literal(instantToMicros(i), TimestampType) case t: Timestamp => Literal(DateTimeUtils.fromJavaTimestamp(t), TimestampType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 45ca022c36f72..6fde8ce9f7f32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1953,6 +1953,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED = + buildConf("spark.sql.legacy.allowNegativeScaleOfDecimal.enabled") + .internal() + .doc("When set to true, negative scale of Decimal type is allowed. For example, " + + "the type of number 1E10BD under legacy mode is DecimalType(2, -9), but is " + + "Decimal(11, 0) in non legacy mode.") + .booleanConf + .createWithDefault(false) + val LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED = buildConf("spark.sql.legacy.createHiveTableByDefault.enabled") .internal() @@ -2681,6 +2690,9 @@ class SQLConf extends Serializable with Logging { def exponentLiteralAsDecimalEnabled: Boolean = getConf(SQLConf.LEGACY_EXPONENT_LITERAL_AS_DECIMAL_ENABLED) + def allowNegativeScaleOfDecimalEnabled: Boolean = + getConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED) + def createHiveTableByDefaultEnabled: Boolean = getConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 533cddb556a9b..9ce64b09f7870 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -23,6 +23,7 @@ import java.math.{BigInteger, MathContext, RoundingMode} import scala.util.Try import org.apache.spark.annotation.Unstable +import org.apache.spark.sql.internal.SQLConf /** * A mutable implementation of BigDecimal that can hold a Long if values are small enough. @@ -89,6 +90,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { * and return it, or return null if it cannot be set due to overflow. */ def setOrNull(unscaled: Long, precision: Int, scale: Int): Decimal = { + DecimalType.checkNegativeScale(scale) if (unscaled <= -POW_10(MAX_LONG_DIGITS) || unscaled >= POW_10(MAX_LONG_DIGITS)) { // We can't represent this compactly as a long without risking overflow if (precision < 19) { @@ -113,6 +115,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { * Set this Decimal to the given BigDecimal value, with a given precision and scale. */ def set(decimal: BigDecimal, precision: Int, scale: Int): Decimal = { + DecimalType.checkNegativeScale(scale) this.decimalVal = decimal.setScale(scale, ROUND_HALF_UP) if (decimalVal.precision > precision) { throw new ArithmeticException( @@ -136,10 +139,16 @@ final class Decimal extends Ordered[Decimal] with Serializable { // result. For example, the precision of 0.01 equals to 1 based on the definition, but // the scale is 2. The expected precision should be 2. this._precision = decimal.scale + this._scale = decimal.scale + } else if (decimal.scale < 0 && !SQLConf.get.allowNegativeScaleOfDecimalEnabled) { + this._precision = decimal.precision - decimal.scale + this._scale = 0 + // set scale to 0 to correct unscaled value + this.decimalVal = decimal.setScale(0) } else { this._precision = decimal.precision + this._scale = decimal.scale } - this._scale = decimal.scale this } @@ -375,6 +384,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { if (precision == this.precision && scale == this.scale) { return true } + DecimalType.checkNegativeScale(scale) // First, update our longVal if we can, or transfer over to using a BigDecimal if (decimalVal.eq(null)) { if (scale < _scale) { @@ -583,6 +593,7 @@ object Decimal { * Creates a decimal from unscaled, precision and scale without checking the bounds. */ def createUnsafe(unscaled: Long, precision: Int, scale: Int): Decimal = { + DecimalType.checkNegativeScale(scale) val dec = new Decimal() dec.longVal = unscaled dec._precision = precision diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 26657cb6c5c27..05069e24e2329 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -24,6 +24,7 @@ import scala.reflect.runtime.universe.typeTag import org.apache.spark.annotation.Stable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.internal.SQLConf /** * The data type representing `java.math.BigDecimal` values. @@ -41,6 +42,8 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} @Stable case class DecimalType(precision: Int, scale: Int) extends FractionalType { + DecimalType.checkNegativeScale(scale) + if (scale > precision) { throw new AnalysisException( s"Decimal scale ($scale) cannot be greater than precision ($precision).") @@ -141,20 +144,26 @@ object DecimalType extends AbstractDataType { } private[sql] def fromLiteral(literal: Literal): DecimalType = literal.value match { - case v: Short => fromBigDecimal(BigDecimal(v)) - case v: Int => fromBigDecimal(BigDecimal(v)) - case v: Long => fromBigDecimal(BigDecimal(v)) + case v: Short => fromDecimal(Decimal(BigDecimal(v))) + case v: Int => fromDecimal(Decimal(BigDecimal(v))) + case v: Long => fromDecimal(Decimal(BigDecimal(v))) case _ => forType(literal.dataType) } - private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { - DecimalType(Math.max(d.precision, d.scale), d.scale) - } + private[sql] def fromDecimal(d: Decimal): DecimalType = DecimalType(d.precision, d.scale) private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + private[sql] def checkNegativeScale(scale: Int): Unit = { + if (scale < 0 && !SQLConf.get.allowNegativeScaleOfDecimalEnabled) { + throw new AnalysisException(s"Negative scale is not allowed: $scale. " + + s"You can use spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=true " + + s"to enable legacy mode to allow it.") + } + } + /** * Scale adjustment implementation is based on Hive's one, which is itself inspired to * SQLServer's one. In particular, when a result precision is greater than @@ -164,7 +173,8 @@ object DecimalType extends AbstractDataType { * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. */ private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { - // Assumption: + // Assumptions: + checkNegativeScale(scale) assert(precision >= scale) if (precision <= MAX_PRECISION) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index 79fc38c4d30ea..c3e18c7f9557f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLite import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project, Union} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -273,12 +274,14 @@ class DecimalPrecisionSuite extends AnalysisTest with BeforeAndAfter { } test("SPARK-24468: operations on decimals with negative scale") { - val a = AttributeReference("a", DecimalType(3, -10))() - val b = AttributeReference("b", DecimalType(1, -1))() - val c = AttributeReference("c", DecimalType(35, 1))() - checkType(Multiply(a, b), DecimalType(5, -11)) - checkType(Multiply(a, c), DecimalType(38, -9)) - checkType(Multiply(b, c), DecimalType(37, 0)) + withSQLConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key -> "true") { + val a = AttributeReference("a", DecimalType(3, -10))() + val b = AttributeReference("b", DecimalType(1, -1))() + val c = AttributeReference("c", DecimalType(35, 1))() + checkType(Multiply(a, b), DecimalType(5, -11)) + checkType(Multiply(a, c), DecimalType(38, -9)) + checkType(Multiply(b, c), DecimalType(37, 0)) + } } /** strength reduction for integer/decimal comparisons */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala index 909d0a12b2626..ee73da33a1a90 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala @@ -22,6 +22,7 @@ import java.util.Locale import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { @@ -147,13 +148,15 @@ class CSVInferSchemaSuite extends SparkFunSuite with SQLHelper { val options = new CSVOptions(Map.empty[String, String], false, "GMT") val inferSchema = new CSVInferSchema(options) - // 9.03E+12 is Decimal(3, -10) and 1.19E+11 is Decimal(3, -9). - assert(inferSchema.inferField(DecimalType(3, -10), "1.19E11") == - DecimalType(4, -9)) + withSQLConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key -> "true") { + // 9.03E+12 is Decimal(3, -10) and 1.19E+11 is Decimal(3, -9). + assert(inferSchema.inferField(DecimalType(3, -10), "1.19E11") == + DecimalType(4, -9)) + } // BigDecimal("12345678901234567890.01234567890123456789") is precision 40 and scale 20. val value = "12345678901234567890.01234567890123456789" - assert(inferSchema.inferField(DecimalType(3, -10), value) == DoubleType) + assert(inferSchema.inferField(DecimalType(3, 0), value) == DoubleType) // Seq(s"${Long.MaxValue}1", "2015-12-01 00:00:00") should be StringType assert(inferSchema.inferField(NullType, s"${Long.MaxValue}1") == DecimalType(20, 0)) 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 7418870d9c731..ad66873c02518 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 @@ -1048,15 +1048,9 @@ class CastSuite extends CastSuiteBase { assert(cast(Decimal(9.95), DecimalType(2, 1)).nullable) assert(cast(Decimal(9.95), DecimalType(3, 1)).nullable === false) - assert(cast(Decimal("1003"), DecimalType(3, -1)).nullable) - assert(cast(Decimal("1003"), DecimalType(4, -1)).nullable === false) - assert(cast(Decimal("995"), DecimalType(2, -1)).nullable) - assert(cast(Decimal("995"), DecimalType(3, -1)).nullable === false) - assert(cast(true, DecimalType.SYSTEM_DEFAULT).nullable === false) assert(cast(true, DecimalType(1, 1)).nullable) - checkEvaluation(cast(10.03, DecimalType.SYSTEM_DEFAULT), Decimal(10.03)) checkEvaluation(cast(10.03, DecimalType(4, 2)), Decimal(10.03)) checkEvaluation(cast(10.03, DecimalType(3, 1)), Decimal(10.0)) @@ -1095,17 +1089,9 @@ class CastSuite extends CastSuiteBase { checkEvaluation(cast(Decimal("1003"), DecimalType.SYSTEM_DEFAULT), Decimal(1003)) checkEvaluation(cast(Decimal("1003"), DecimalType(4, 0)), Decimal(1003)) - checkEvaluation(cast(Decimal("1003"), DecimalType(3, -1)), Decimal(1000)) - checkEvaluation(cast(Decimal("1003"), DecimalType(2, -2)), Decimal(1000)) - checkEvaluation(cast(Decimal("1003"), DecimalType(1, -2)), null) - checkEvaluation(cast(Decimal("1003"), DecimalType(2, -1)), null) checkEvaluation(cast(Decimal("1003"), DecimalType(3, 0)), null) checkEvaluation(cast(Decimal("995"), DecimalType(3, 0)), Decimal(995)) - checkEvaluation(cast(Decimal("995"), DecimalType(3, -1)), Decimal(1000)) - checkEvaluation(cast(Decimal("995"), DecimalType(2, -2)), Decimal(1000)) - checkEvaluation(cast(Decimal("995"), DecimalType(2, -1)), null) - checkEvaluation(cast(Decimal("995"), DecimalType(1, -2)), null) checkEvaluation(cast(Double.NaN, DecimalType.SYSTEM_DEFAULT), null) checkEvaluation(cast(1.0 / 0.0, DecimalType.SYSTEM_DEFAULT), null) @@ -1119,6 +1105,23 @@ class CastSuite extends CastSuiteBase { checkEvaluation(cast(true, DecimalType(2, 1)), Decimal(1)) checkEvaluation(cast(true, DecimalType(1, 1)), null) + + withSQLConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key -> "true") { + assert(cast(Decimal("1003"), DecimalType(3, -1)).nullable) + assert(cast(Decimal("1003"), DecimalType(4, -1)).nullable === false) + assert(cast(Decimal("995"), DecimalType(2, -1)).nullable) + assert(cast(Decimal("995"), DecimalType(3, -1)).nullable === false) + + checkEvaluation(cast(Decimal("1003"), DecimalType(3, -1)), Decimal(1000)) + checkEvaluation(cast(Decimal("1003"), DecimalType(2, -2)), Decimal(1000)) + checkEvaluation(cast(Decimal("1003"), DecimalType(1, -2)), null) + checkEvaluation(cast(Decimal("1003"), DecimalType(2, -1)), null) + + checkEvaluation(cast(Decimal("995"), DecimalType(3, -1)), Decimal(1000)) + checkEvaluation(cast(Decimal("995"), DecimalType(2, -2)), Decimal(1000)) + checkEvaluation(cast(Decimal("995"), DecimalType(2, -1)), null) + checkEvaluation(cast(Decimal("995"), DecimalType(1, -2)), null) + } } test("SPARK-28470: Cast should honor nullOnOverflow property") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index fd4288dac6ec0..85efc6accf01f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -524,6 +524,16 @@ class ExpressionParserSuite extends AnalysisTest { intercept("1.20E-38BD", "decimal can only support precision up to 38") } + test("SPARK-30252: Decimal should set zero scale rather than negative scale by default") { + assertEqual("123.0BD", Literal(Decimal(BigDecimal("123.0")), DecimalType(4, 1))) + assertEqual("123BD", Literal(Decimal(BigDecimal("123")), DecimalType(3, 0))) + assertEqual("123E10BD", Literal(Decimal(BigDecimal("123E10")), DecimalType(13, 0))) + assertEqual("123E+10BD", Literal(Decimal(BigDecimal("123E+10")), DecimalType(13, 0))) + assertEqual("123E-10BD", Literal(Decimal(BigDecimal("123E-10")), DecimalType(10, 10))) + assertEqual("1.23E10BD", Literal(Decimal(BigDecimal("1.23E10")), DecimalType(11, 0))) + assertEqual("-1.23E10BD", Literal(Decimal(BigDecimal("-1.23E10")), DecimalType(11, 0))) + } + test("SPARK-29956: scientific decimal should be parsed as Decimal in legacy mode") { def testDecimal(value: String, parser: ParserInterface): Unit = { assertEqual(value, Literal(BigDecimal(value).underlying), parser) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index 0c01d32421274..7d0346fc0145e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql.types import org.scalatest.PrivateMethodTester import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.Decimal._ -class DecimalSuite extends SparkFunSuite with PrivateMethodTester { +class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper { /** Check that a Decimal has the given string representation, precision and scale */ private def checkDecimal(d: Decimal, string: String, precision: Int, scale: Int): Unit = { assert(d.toString === string) @@ -63,13 +66,27 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { intercept[ArithmeticException](Decimal(1e17.toLong, 17, 0)) } - test("creating decimals with negative scale") { - checkDecimal(Decimal(BigDecimal("98765"), 5, -3), "9.9E+4", 5, -3) - checkDecimal(Decimal(BigDecimal("314.159"), 6, -2), "3E+2", 6, -2) - checkDecimal(Decimal(BigDecimal(1.579e12), 4, -9), "1.579E+12", 4, -9) - checkDecimal(Decimal(BigDecimal(1.579e12), 4, -10), "1.58E+12", 4, -10) - checkDecimal(Decimal(103050709L, 9, -10), "1.03050709E+18", 9, -10) - checkDecimal(Decimal(1e8.toLong, 10, -10), "1.00000000E+18", 10, -10) + test("creating decimals with negative scale under legacy mode") { + withSQLConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key -> "true") { + checkDecimal(Decimal(BigDecimal("98765"), 5, -3), "9.9E+4", 5, -3) + checkDecimal(Decimal(BigDecimal("314.159"), 6, -2), "3E+2", 6, -2) + checkDecimal(Decimal(BigDecimal(1.579e12), 4, -9), "1.579E+12", 4, -9) + checkDecimal(Decimal(BigDecimal(1.579e12), 4, -10), "1.58E+12", 4, -10) + checkDecimal(Decimal(103050709L, 9, -10), "1.03050709E+18", 9, -10) + checkDecimal(Decimal(1e8.toLong, 10, -10), "1.00000000E+18", 10, -10) + } + } + + test("SPARK-30252: Negative scale is not allowed by default") { + def checkNegativeScaleDecimal(d: => Decimal): Unit = { + intercept[AnalysisException](d) + .getMessage + .contains("Negative scale is not allowed under ansi mode") + } + checkNegativeScaleDecimal(Decimal(BigDecimal("98765"), 5, -3)) + checkNegativeScaleDecimal(Decimal(BigDecimal("98765").underlying(), 5, -3)) + checkNegativeScaleDecimal(Decimal(98765L, 5, -3)) + checkNegativeScaleDecimal(Decimal.createUnsafe(98765L, 5, -3)) } test("double and long values") { diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index eada0f5697bee..510f2339ba2a9 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -94,7 +94,7 @@ select 12345678901234567890.0 * 12345678901234567890.0 struct<> -- !query 10 output java.lang.ArithmeticException -Decimal(expanded,1.5241578753238836750190519987501905210E+38,38,-1}) cannot be represented as Decimal(38, 2). +Decimal(expanded,152415787532388367501905199875019052100,39,0}) cannot be represented as Decimal(38, 2). -- !query 11 diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index bf23833c8e3e4..324a3ae8efd11 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -124,7 +124,7 @@ struct<(2.35E10 * CAST(1.0 AS DOUBLE)):double> -- !query 14 select (5e36BD + 0.1) + 5e36BD -- !query 14 schema -struct<(CAST((CAST(5E+36 AS DECIMAL(38,1)) + CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) + CAST(5E+36 AS DECIMAL(38,1))):decimal(38,1)> +struct<(CAST((CAST(5000000000000000000000000000000000000 AS DECIMAL(38,1)) + CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) + CAST(5000000000000000000000000000000000000 AS DECIMAL(38,1))):decimal(38,1)> -- !query 14 output NULL @@ -132,7 +132,7 @@ NULL -- !query 15 select (-4e36BD - 0.1) - 7e36BD -- !query 15 schema -struct<(CAST((CAST(-4E+36 AS DECIMAL(38,1)) - CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) - CAST(7E+36 AS DECIMAL(38,1))):decimal(38,1)> +struct<(CAST((CAST(-4000000000000000000000000000000000000 AS DECIMAL(38,1)) - CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) - CAST(7000000000000000000000000000000000000 AS DECIMAL(38,1))):decimal(38,1)> -- !query 15 output NULL @@ -148,7 +148,7 @@ NULL -- !query 17 select 1e35BD / 0.1 -- !query 17 schema -struct<(CAST(1E+35 AS DECIMAL(37,1)) / CAST(0.1 AS DECIMAL(37,1))):decimal(38,6)> +struct<(CAST(100000000000000000000000000000000000 AS DECIMAL(37,1)) / CAST(0.1 AS DECIMAL(37,1))):decimal(38,6)> -- !query 17 output NULL @@ -156,7 +156,7 @@ NULL -- !query 18 select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD -- !query 18 schema -struct<(CAST(1.2345678901234567890E+30 AS DECIMAL(25,-6)) * CAST(1.2345678901234567890E+25 AS DECIMAL(25,-6))):decimal(38,-17)> +struct<(CAST(1234567890123456789000000000000 AS DECIMAL(31,0)) * CAST(12345678901234567890000000 AS DECIMAL(31,0))):decimal(38,0)> -- !query 18 output NULL @@ -266,7 +266,7 @@ struct<(2.35E10 * CAST(1.0 AS DOUBLE)):double> -- !query 31 select (5e36BD + 0.1) + 5e36BD -- !query 31 schema -struct<(CAST((CAST(5E+36 AS DECIMAL(38,1)) + CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) + CAST(5E+36 AS DECIMAL(38,1))):decimal(38,1)> +struct<(CAST((CAST(5000000000000000000000000000000000000 AS DECIMAL(38,1)) + CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) + CAST(5000000000000000000000000000000000000 AS DECIMAL(38,1))):decimal(38,1)> -- !query 31 output NULL @@ -274,7 +274,7 @@ NULL -- !query 32 select (-4e36BD - 0.1) - 7e36BD -- !query 32 schema -struct<(CAST((CAST(-4E+36 AS DECIMAL(38,1)) - CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) - CAST(7E+36 AS DECIMAL(38,1))):decimal(38,1)> +struct<(CAST((CAST(-4000000000000000000000000000000000000 AS DECIMAL(38,1)) - CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) - CAST(7000000000000000000000000000000000000 AS DECIMAL(38,1))):decimal(38,1)> -- !query 32 output NULL @@ -290,7 +290,7 @@ NULL -- !query 34 select 1e35BD / 0.1 -- !query 34 schema -struct<(CAST(1E+35 AS DECIMAL(37,1)) / CAST(0.1 AS DECIMAL(37,1))):decimal(38,3)> +struct<(CAST(100000000000000000000000000000000000 AS DECIMAL(37,1)) / CAST(0.1 AS DECIMAL(37,1))):decimal(38,3)> -- !query 34 output NULL @@ -298,7 +298,7 @@ NULL -- !query 35 select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD -- !query 35 schema -struct<(CAST(1.2345678901234567890E+30 AS DECIMAL(25,-6)) * CAST(1.2345678901234567890E+25 AS DECIMAL(25,-6))):decimal(38,-17)> +struct<(CAST(1234567890123456789000000000000 AS DECIMAL(31,0)) * CAST(12345678901234567890000000 AS DECIMAL(31,0))):decimal(38,0)> -- !query 35 output NULL diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out index 6067d95490621..64605c995b443 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out @@ -172,7 +172,7 @@ WHERE t1b IN (SELECT Max(t2b) FROM t2 GROUP BY t2a) -- !query 9 schema -struct +struct -- !query 9 output t1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 t1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out index 42f611a4c829f..0975cb97b19fa 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out @@ -74,7 +74,7 @@ WHERE t1a IN (SELECT t2a WHERE t1d = t2d) LIMIT 2 -- !query 3 schema -struct +struct -- !query 3 output val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 @@ -89,7 +89,7 @@ WHERE t1c IN (SELECT t2c LIMIT 2) LIMIT 4 -- !query 4 schema -struct +struct -- !query 4 output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 @@ -122,7 +122,7 @@ WHERE t1b NOT IN (SELECT t2b WHERE t2b > 6 LIMIT 2) -- !query 6 schema -struct +struct -- !query 6 output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out index bff386cbf1715..aa36be9040369 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out @@ -73,7 +73,7 @@ WHERE t1a IN (SELECT t2a FROM t2) ORDER BY t1a -- !query 3 schema -struct +struct -- !query 3 output val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 @@ -130,7 +130,7 @@ WHERE t1b IN (SELECT t2c FROM t2 ORDER BY t2d) -- !query 7 schema -struct +struct -- !query 7 output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 @@ -145,7 +145,7 @@ WHERE t1b IN (SELECT Min(t2b) ORDER BY Min(t2b)) ORDER BY t1c DESC nulls first -- !query 8 schema -struct +struct -- !query 8 output val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 @@ -184,7 +184,7 @@ WHERE t1a NOT IN (SELECT t2a FROM t2) ORDER BY t1a -- !query 10 schema -struct +struct -- !query 10 output val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 @@ -226,7 +226,7 @@ WHERE t1a NOT IN (SELECT t2a ORDER BY t2b DESC nulls last) ORDER BY t1c DESC nulls last -- !query 12 schema -struct +struct -- !query 12 output val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL @@ -242,7 +242,7 @@ WHERE t1b IN (SELECT Min(t2b) GROUP BY t2a ORDER BY t2a DESC) -- !query 13 schema -struct +struct -- !query 13 output val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 @@ -276,7 +276,7 @@ WHERE t1b NOT IN (SELECT Min(t2b) GROUP BY t2a ORDER BY t2a) -- !query 15 schema -struct +struct -- !query 15 output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out index c4160c19ce342..53ddaa36948ff 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out @@ -498,7 +498,7 @@ FROM (SELECT * FROM t3 WHERE t4.t2a = t3a)) -- !query 13 schema -struct +struct -- !query 13 output val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out index a07d9975fa7df..f717382835ab4 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out @@ -72,7 +72,7 @@ FROM t1 WHERE t1a IN (SELECT t2a FROM t2) -- !query 3 schema -struct +struct -- !query 3 output t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 t1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 @@ -88,7 +88,7 @@ WHERE t1b IN (SELECT t2b FROM t2 WHERE t1a = t2a) -- !query 4 schema -struct +struct -- !query 4 output t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 7af20fb6dba0b..399660d195066 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -174,7 +174,7 @@ FROM t1 WHERE t1c + 5 = (SELECT max(t2e) FROM t2) -- !query 11 schema -struct +struct -- !query 11 output val1a 16 2000 val1a 16 2000 @@ -217,7 +217,7 @@ WHERE t2a = t1a AND max_t1g = (SELECT max(t1g) FROM t1) -- !query 14 schema -struct +struct -- !query 14 output val1b 2600 val1c 2600 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala index 567bcdd1878a8..bd86c2ec075b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets import org.apache.spark.sql.functions._ import org.apache.spark.sql.functions.{log => logarithm} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession private object MathFunctionsTestData { @@ -218,19 +219,21 @@ class MathFunctionsSuite extends QueryTest with SharedSparkSession { Seq(Row(5, 0, 0), Row(55, 60, 100), Row(555, 560, 600)) ) - val pi = "3.1415" - checkAnswer( - sql(s"SELECT round($pi, -3), round($pi, -2), round($pi, -1), " + - s"round($pi, 0), round($pi, 1), round($pi, 2), round($pi, 3)"), - Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), - BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) - ) - checkAnswer( - sql(s"SELECT bround($pi, -3), bround($pi, -2), bround($pi, -1), " + - s"bround($pi, 0), bround($pi, 1), bround($pi, 2), bround($pi, 3)"), - Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), - BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) - ) + withSQLConf(SQLConf.LEGACY_ALLOW_NEGATIVE_SCALE_OF_DECIMAL_ENABLED.key -> "true") { + val pi = "3.1415" + checkAnswer( + sql(s"SELECT round($pi, -3), round($pi, -2), round($pi, -1), " + + s"round($pi, 0), round($pi, 1), round($pi, 2), round($pi, 3)"), + Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), + BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) + ) + checkAnswer( + sql(s"SELECT bround($pi, -3), bround($pi, -2), bround($pi, -1), " + + s"bround($pi, 0), bround($pi, 1), bround($pi, 2), bround($pi, 3)"), + Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), + BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) + ) + } val bdPi: BigDecimal = BigDecimal(31415925L, 7) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 1c679394ce2ec..d0e2e001034fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -92,7 +92,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson checkTypePromotion( Decimal(longNumber), enforceCorrectType(longNumber, DecimalType.SYSTEM_DEFAULT)) - val doubleNumber: Double = 1.7976931348623157E308d + val doubleNumber: Double = 1.7976931348623157d checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber * 1000L)), @@ -284,7 +284,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson sql("select * from jsonTable"), Row(new java.math.BigDecimal("92233720368547758070"), true, - 1.7976931348623157E308, + 1.7976931348623157, 10, 21474836470L, null, @@ -624,7 +624,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson sql("select * from jsonTable"), Row(new java.math.BigDecimal("92233720368547758070"), true, - 1.7976931348623157E308, + 1.7976931348623157, 10, 21474836470L, null, @@ -656,7 +656,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson sql("select * from jsonTable"), Row("92233720368547758070", "true", - "1.7976931348623157E308", + "1.7976931348623157", "10", "21474836470", null, @@ -768,7 +768,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson val expectedSchema = StructType( StructField("bigInteger", DecimalType(20, 0), true) :: StructField("boolean", BooleanType, true) :: - StructField("double", DecimalType(17, -292), true) :: + StructField("double", DecimalType(17, 16), true) :: StructField("integer", LongType, true) :: StructField("long", LongType, true) :: StructField("null", StringType, true) :: @@ -782,7 +782,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson sql("select * from jsonTable"), Row(BigDecimal("92233720368547758070"), true, - BigDecimal("1.7976931348623157E308"), + BigDecimal("1.7976931348623157"), 10, 21474836470L, null, @@ -875,7 +875,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson sql("select * from jsonTableSQL"), Row(new java.math.BigDecimal("92233720368547758070"), true, - 1.7976931348623157E308, + 1.7976931348623157, 10, 21474836470L, null, @@ -908,7 +908,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson sql("select * from jsonTable1"), Row(new java.math.BigDecimal("92233720368547758070"), true, - 1.7976931348623157E308, + 1.7976931348623157, 10, 21474836470L, null, @@ -925,7 +925,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson sql("select * from jsonTable2"), Row(new java.math.BigDecimal("92233720368547758070"), true, - 1.7976931348623157E308, + 1.7976931348623157, 10, 21474836470L, null, @@ -1274,7 +1274,7 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson sql("select * from primitiveTable"), Row(new java.math.BigDecimal("92233720368547758070"), true, - 1.7976931348623157E308, + 1.7976931348623157, 10, 21474836470L, "this is a simple string.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala index 17503330bfd5c..5c35ee03fb271 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala @@ -28,7 +28,7 @@ private[json] trait TestJsonData { "integer":10, "long":21474836470, "bigInteger":92233720368547758070, - "double":1.7976931348623157E308, + "double":1.7976931348623157, "boolean":true, "null":null }""" :: Nil))(Encoders.STRING) @@ -87,7 +87,7 @@ private[json] trait TestJsonData { "arrayOfInteger":[1, 2147483647, -2147483648], "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], - "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], + "arrayOfDouble":[1.2, 1.7976931348623157, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index aa41fae2d8975..48fedff2150a7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -336,7 +336,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ case t: Timestamp => HiveResult.toHiveString((t, TimestampType)) case d: java.math.BigDecimal => - HiveResult.toHiveString((d, DecimalType.fromBigDecimal(d))) + HiveResult.toHiveString((d, DecimalType.fromDecimal(Decimal(d)))) case bin: Array[Byte] => HiveResult.toHiveString((bin, BinaryType)) case other => From cfb1706eaa59848cfda88a2f3a6fd90262f55b3e Mon Sep 17 00:00:00 2001 From: fuwhu Date: Tue, 21 Jan 2020 21:26:30 +0800 Subject: [PATCH 1178/1387] [SPARK-15616][SQL] Add optimizer rule PruneHiveTablePartitions ### What changes were proposed in this pull request? Add optimizer rule PruneHiveTablePartitions pruning hive table partitions based on filters on partition columns. Doing so, the total size of pruned partitions may be small enough for broadcast join in JoinSelection strategy. ### Why are the changes needed? In JoinSelection strategy, spark use the "plan.stats.sizeInBytes" to decide whether the plan is suitable for broadcast join. Currently, "plan.stats.sizeInBytes" does not take "pruned partitions" into account, so it may miss some broadcast join and take sort-merge join instead, which will definitely impact join performance. This PR aim at taking "pruned partitions" into account for hive table in "plan.stats.sizeInBytes" and then improve performance by using broadcast join if possible. ### Does this PR introduce any user-facing change? no ### How was this patch tested? Added unit tests. This is based on #25919, credits should go to lianhuiwang and advancedxy. Closes #26805 from fuwhu/SPARK-15616. Authored-by: fuwhu Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/interface.scala | 4 +- .../PruneFileSourcePartitions.scala | 2 +- .../benchmark/TPCDSQueryBenchmark.scala | 2 +- .../sql/hive/HiveSessionStateBuilder.scala | 18 ++- .../hive/execution/HiveTableScanExec.scala | 8 +- .../execution/PruneHiveTablePartitions.scala | 110 ++++++++++++++++++ .../PruneHiveTablePartitionsSuite.scala | 57 +++++++++ 7 files changed, 193 insertions(+), 8 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index d51690367bf35..81561c53f83c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -651,7 +651,9 @@ case class HiveTableRelation( tableMeta: CatalogTable, dataCols: Seq[AttributeReference], partitionCols: Seq[AttributeReference], - tableStats: Option[Statistics] = None) extends LeafNode with MultiInstanceRelation { + tableStats: Option[Statistics] = None, + @transient prunedPartitions: Option[Seq[CatalogTablePartition]] = None) + extends LeafNode with MultiInstanceRelation { assert(tableMeta.identifier.database.isDefined) assert(tableMeta.partitionSchema.sameType(partitionCols.toStructType)) assert(tableMeta.dataSchema.sameType(dataCols.toStructType)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 59c55c161bc89..1ea19c187e51a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, FileScan, FileTable} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, FileScan} import org.apache.spark.sql.types.StructType private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala index be02447db2e55..c93d27f02c686 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -84,7 +84,7 @@ object TPCDSQueryBenchmark extends SqlBasedBenchmark { queryRelations.add(alias.identifier) case LogicalRelation(_, _, Some(catalogTable), _) => queryRelations.add(catalogTable.identifier.table) - case HiveTableRelation(tableMeta, _, _, _) => + case HiveTableRelation(tableMeta, _, _, _, _) => queryRelations.add(tableMeta.identifier.table) case _ => } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index de21a13e6edb8..b117c582a3e6e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -21,13 +21,15 @@ import org.apache.spark.annotation.Unstable import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, ResolveSessionCatalog} import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener +import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.SparkPlanner +import org.apache.spark.sql.execution.{SparkOptimizer, SparkPlanner} import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.v2.TableCapabilityCheck import org.apache.spark.sql.hive.client.HiveClient +import org.apache.spark.sql.hive.execution.PruneHiveTablePartitions import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState} /** @@ -93,6 +95,20 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session customCheckRules } + /** + * Logical query plan optimizer that takes into account Hive. + */ + override protected def optimizer: Optimizer = { + new SparkOptimizer(catalogManager, catalog, experimentalMethods) { + override def postHocOptimizationBatches: Seq[Batch] = Seq( + Batch("Prune Hive Table Partitions", Once, new PruneHiveTablePartitions(session)) + ) + + override def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = + super.extendedOperatorOptimizationRules ++ customOperatorOptimizationRules + } + } + /** * Planner that takes into account Hive-specific strategies. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index 5b00e2ebafa43..9f0f05023bc24 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -166,14 +166,14 @@ case class HiveTableScanExec( @transient lazy val rawPartitions = { val prunedPartitions = if (sparkSession.sessionState.conf.metastorePartitionPruning && - partitionPruningPred.size > 0) { + partitionPruningPred.nonEmpty) { // Retrieve the original attributes based on expression ID so that capitalization matches. val normalizedFilters = partitionPruningPred.map(_.transform { case a: AttributeReference => originalAttributes(a) }) - sparkSession.sessionState.catalog.listPartitionsByFilter( - relation.tableMeta.identifier, - normalizedFilters) + relation.prunedPartitions.getOrElse( + sparkSession.sessionState.catalog + .listPartitionsByFilter(relation.tableMeta.identifier, normalizedFilters)) } else { sparkSession.sessionState.catalog.listPartitions(relation.tableMeta.identifier) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala new file mode 100644 index 0000000000000..a0349f627d107 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala @@ -0,0 +1,110 @@ +/* + * 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.hive.execution + +import org.apache.hadoop.hive.common.StatsSetupConst + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.CastSupport +import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTable, CatalogTablePartition, ExternalCatalogUtils, HiveTableRelation} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, ExpressionSet, SubqueryExpression} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.internal.SQLConf + +/** + * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source. + */ +private[sql] class PruneHiveTablePartitions(session: SparkSession) + extends Rule[LogicalPlan] with CastSupport { + + override val conf: SQLConf = session.sessionState.conf + + /** + * Extract the partition filters from the filters on the table. + */ + private def getPartitionKeyFilters( + filters: Seq[Expression], + relation: HiveTableRelation): ExpressionSet = { + val normalizedFilters = DataSourceStrategy.normalizeExprs( + filters.filter(f => f.deterministic && !SubqueryExpression.hasSubquery(f)), relation.output) + val partitionColumnSet = AttributeSet(relation.partitionCols) + ExpressionSet(normalizedFilters.filter { f => + !f.references.isEmpty && f.references.subsetOf(partitionColumnSet) + }) + } + + /** + * Prune the hive table using filters on the partitions of the table. + */ + private def prunePartitions( + relation: HiveTableRelation, + partitionFilters: ExpressionSet): Seq[CatalogTablePartition] = { + if (conf.metastorePartitionPruning) { + session.sessionState.catalog.listPartitionsByFilter( + relation.tableMeta.identifier, partitionFilters.toSeq) + } else { + ExternalCatalogUtils.prunePartitionsByFilter(relation.tableMeta, + session.sessionState.catalog.listPartitions(relation.tableMeta.identifier), + partitionFilters.toSeq, conf.sessionLocalTimeZone) + } + } + + /** + * Update the statistics of the table. + */ + private def updateTableMeta( + tableMeta: CatalogTable, + prunedPartitions: Seq[CatalogTablePartition]): CatalogTable = { + val sizeOfPartitions = prunedPartitions.map { partition => + val rawDataSize = partition.parameters.get(StatsSetupConst.RAW_DATA_SIZE).map(_.toLong) + val totalSize = partition.parameters.get(StatsSetupConst.TOTAL_SIZE).map(_.toLong) + if (rawDataSize.isDefined && rawDataSize.get > 0) { + rawDataSize.get + } else if (totalSize.isDefined && totalSize.get > 0L) { + totalSize.get + } else { + 0L + } + } + if (sizeOfPartitions.forall(_ > 0)) { + val sizeInBytes = sizeOfPartitions.sum + tableMeta.copy(stats = Some(CatalogStatistics(sizeInBytes = BigInt(sizeInBytes)))) + } else { + tableMeta + } + } + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case op @ PhysicalOperation(projections, filters, relation: HiveTableRelation) + if filters.nonEmpty && relation.isPartitioned && relation.prunedPartitions.isEmpty => + val partitionKeyFilters = getPartitionKeyFilters(filters, relation) + if (partitionKeyFilters.nonEmpty) { + val newPartitions = prunePartitions(relation, partitionKeyFilters) + val newTableMeta = updateTableMeta(relation.tableMeta, newPartitions) + val newRelation = relation.copy( + tableMeta = newTableMeta, prunedPartitions = Some(newPartitions)) + // Keep partition filters so that they are visible in physical planning + Project(projections, Filter(filters.reduceLeft(And), newRelation)) + } else { + op + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala new file mode 100644 index 0000000000000..e41709841a736 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitionsSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils + +class PruneHiveTablePartitionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("PruneHiveTablePartitions", Once, + EliminateSubqueryAliases, new PruneHiveTablePartitions(spark)) :: Nil + } + + test("SPARK-15616 statistics pruned after going throuhg PruneHiveTablePartitions") { + withTable("test", "temp") { + sql( + s""" + |CREATE TABLE test(i int) + |PARTITIONED BY (p int) + |STORED AS textfile""".stripMargin) + spark.range(0, 1000, 1).selectExpr("id as col") + .createOrReplaceTempView("temp") + + for (part <- Seq(1, 2, 3, 4)) { + sql( + s""" + |INSERT OVERWRITE TABLE test PARTITION (p='$part') + |select col from temp""".stripMargin) + } + val analyzed1 = sql("select i from test where p > 0").queryExecution.analyzed + val analyzed2 = sql("select i from test where p = 1").queryExecution.analyzed + assert(Optimize.execute(analyzed1).stats.sizeInBytes / 4 === + Optimize.execute(analyzed2).stats.sizeInBytes) + } + } +} From a131031f95dbd426516b88e1dac38965351eb501 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 21 Jan 2020 14:27:55 -0800 Subject: [PATCH 1179/1387] [SPARK-30599][CORE][TESTS] Increase the maximum number of log events in LogAppender ### What changes were proposed in this pull request? Increased the limit for log events that could be stored in `SparkFunSuite.LogAppender` from 100 to 1000. ### Why are the changes needed? Sometimes (see traces in SPARK-30599) additional info is logged via log4j, and appended to `LogAppender`. For example, unusual log entries are: ``` [36] Removed broadcast_214_piece0 on 192.168.1.66:52354 in memory (size: 5.7 KiB, free: 2003.8 MiB) [37] Removed broadcast_204_piece0 on 192.168.1.66:52354 in memory (size: 5.7 KiB, free: 2003.9 MiB) [38] Removed broadcast_200_piece0 on 192.168.1.66:52354 in memory (size: 3.7 KiB, free: 2003.9 MiB) [39] Removed broadcast_207_piece0 on 192.168.1.66:52354 in memory (size: 24.2 KiB, free: 2003.9 MiB) [40] Removed broadcast_208_piece0 on 192.168.1.66:52354 in memory (size: 24.2 KiB, free: 2003.9 MiB) ``` and a test which uses `LogAppender` can fail with the exception: ``` java.lang.IllegalStateException: Number of events reached the limit of 100 while logging CSV header matches to schema w/ enforceSchema. ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? By re-running `"SPARK-23786: warning should be printed if CSV header doesn't conform to schema"` in a loop. Closes #27312 from MaxGekk/log-appender-filter. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- core/src/test/scala/org/apache/spark/SparkFunSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 096806142ef13..cf4400e080e37 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -189,7 +189,7 @@ abstract class SparkFunSuite } } - class LogAppender(msg: String = "", maxEvents: Int = 100) extends AppenderSkeleton { + class LogAppender(msg: String = "", maxEvents: Int = 1000) extends AppenderSkeleton { val loggingEvents = new ArrayBuffer[LoggingEvent]() override def append(loggingEvent: LoggingEvent): Unit = { From 3c4e61918fc8266368bd33ea0612144de77571e6 Mon Sep 17 00:00:00 2001 From: bettermouse Date: Tue, 21 Jan 2020 21:37:21 -0800 Subject: [PATCH 1180/1387] [SPARK-30553][DOCS] fix structured-streaming java example error # What changes were proposed in this pull request? Fix structured-streaming java example error. ```java Dataset windowedCounts = words .withWatermark("timestamp", "10 minutes") .groupBy( functions.window(words.col("timestamp"), "10 minutes", "5 minutes"), words.col("word")) .count(); ``` It does not clean up old state.May cause OOM > Before the fix ```scala == Physical Plan == WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWriter48e331f0 +- *(4) HashAggregate(keys=[window#13, word#4], functions=[count(1)], output=[window#13, word#4, count#12L]) +- StateStoreSave [window#13, word#4], state info [ checkpoint = file:/C:/Users/chenhao/AppData/Local/Temp/temporary-91124080-0e20-41c0-9150-91735bdc22c0/state, runId = 5c425536-a3ae-4385-8167-5fa529e6760d, opId = 0, ver = 6, numPartitions = 1], Update, 1579530890886, 2 +- *(3) HashAggregate(keys=[window#13, word#4], functions=[merge_count(1)], output=[window#13, word#4, count#23L]) +- StateStoreRestore [window#13, word#4], state info [ checkpoint = file:/C:/Users/chenhao/AppData/Local/Temp/temporary-91124080-0e20-41c0-9150-91735bdc22c0/state, runId = 5c425536-a3ae-4385-8167-5fa529e6760d, opId = 0, ver = 6, numPartitions = 1], 2 +- *(2) HashAggregate(keys=[window#13, word#4], functions=[merge_count(1)], output=[window#13, word#4, count#23L]) +- Exchange hashpartitioning(window#13, word#4, 1) +- *(1) HashAggregate(keys=[window#13, word#4], functions=[partial_count(1)], output=[window#13, word#4, count#23L]) +- *(1) Project [window#13, word#4] +- *(1) Filter (((isnotnull(timestamp#5) && isnotnull(window#13)) && (timestamp#5 >= window#13.start)) && (timestamp#5 < window#13.end)) +- *(1) Expand [List(named_struct(start, precisetimestampconversion(((((CASE WHEN (cast(CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) as double) = (cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) THEN (CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) + 1) ELSE CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) END + 0) - 2) * 300000000) + 0), LongType, TimestampType), end, precisetimestampconversion(((((CASE WHEN (cast(CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) as double) = (cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) THEN (CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) + 1) ELSE CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) END + 0) - 2) * 300000000) + 600000000), LongType, TimestampType)), word#4, timestamp#5-T600000ms), List(named_struct(start, precisetimestampconversion(((((CASE WHEN (cast(CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) as double) = (cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) THEN (CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) + 1) ELSE CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) END + 1) - 2) * 300000000) + 0), LongType, TimestampType), end, precisetimestampconversion(((((CASE WHEN (cast(CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) as double) = (cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) THEN (CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) + 1) ELSE CEIL((cast((precisetimestampconversion(timestamp#5, TimestampType, LongType) - 0) as double) / 3.0E8)) END + 1) - 2) * 300000000) + 600000000), LongType, TimestampType)), word#4, timestamp#5-T600000ms)], [window#13, word#4, timestamp#5-T600000ms] +- EventTimeWatermark timestamp#5: timestamp, interval 10 minutes +- LocalTableScan , [word#4, timestamp#5] ``` > After the fix ```scala == Physical Plan == WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWriter1df12a96 +- *(4) HashAggregate(keys=[window#13-T600000ms, word#4], functions=[count(1)], output=[window#8-T600000ms, word#4, count#12L]) +- StateStoreSave [window#13-T600000ms, word#4], state info [ checkpoint = file:/C:/Users/chenhao/AppData/Local/Temp/temporary-95ac74cc-aca6-42eb-827d-7586aa69bcd3/state, runId = 91fa311d-d47e-4726-9d0a-f21ef268d9d0, opId = 0, ver = 4, numPartitions = 1], Update, 1579529975342, 2 +- *(3) HashAggregate(keys=[window#13-T600000ms, word#4], functions=[merge_count(1)], output=[window#13-T600000ms, word#4, count#23L]) +- StateStoreRestore [window#13-T600000ms, word#4], state info [ checkpoint = file:/C:/Users/chenhao/AppData/Local/Temp/temporary-95ac74cc-aca6-42eb-827d-7586aa69bcd3/state, runId = 91fa311d-d47e-4726-9d0a-f21ef268d9d0, opId = 0, ver = 4, numPartitions = 1], 2 +- *(2) HashAggregate(keys=[window#13-T600000ms, word#4], functions=[merge_count(1)], output=[window#13-T600000ms, word#4, count#23L]) +- Exchange hashpartitioning(window#13-T600000ms, word#4, 1) +- *(1) HashAggregate(keys=[window#13-T600000ms, word#4], functions=[partial_count(1)], output=[window#13-T600000ms, word#4, count#23L]) +- *(1) Project [window#13-T600000ms, word#4] +- *(1) Filter (((isnotnull(timestamp#5-T600000ms) && isnotnull(window#13-T600000ms)) && (timestamp#5-T600000ms >= window#13-T600000ms.start)) && (timestamp#5-T600000ms < window#13-T600000ms.end)) +- *(1) Expand [List(named_struct(start, precisetimestampconversion(((((CASE WHEN (cast(CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) as double) = (cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) THEN (CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) + 1) ELSE CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) END + 0) - 2) * 300000000) + 0), LongType, TimestampType), end, precisetimestampconversion(((((CASE WHEN (cast(CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) as double) = (cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) THEN (CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) + 1) ELSE CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) END + 0) - 2) * 300000000) + 600000000), LongType, TimestampType)), word#4, timestamp#5-T600000ms), List(named_struct(start, precisetimestampconversion(((((CASE WHEN (cast(CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) as double) = (cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) THEN (CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) + 1) ELSE CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) END + 1) - 2) * 300000000) + 0), LongType, TimestampType), end, precisetimestampconversion(((((CASE WHEN (cast(CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) as double) = (cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) THEN (CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) + 1) ELSE CEIL((cast((precisetimestampconversion(timestamp#5-T600000ms, TimestampType, LongType) - 0) as double) / 3.0E8)) END + 1) - 2) * 300000000) + 600000000), LongType, TimestampType)), word#4, timestamp#5-T600000ms)], [window#13-T600000ms, word#4, timestamp#5-T600000ms] +- EventTimeWatermark timestamp#5: timestamp, interval 10 minutes +- LocalTableScan , [word#4, timestamp#5] ``` ### Why are the changes needed? If we write the code according to the documentation.It does not clean up old state.May cause OOM ### Does this PR introduce any user-facing change? No ### How was this patch tested? ```java SparkSession spark = SparkSession.builder().appName("test").master("local[*]") .config("spark.sql.shuffle.partitions", 1) .getOrCreate(); Dataset lines = spark.readStream().format("socket") .option("host", "skynet") .option("includeTimestamp", true) .option("port", 8888).load(); Dataset words = lines.toDF("word", "timestamp"); Dataset windowedCounts = words .withWatermark("timestamp", "10 minutes") .groupBy( window(col("timestamp"), "10 minutes", "5 minutes"), col("word")) .count(); StreamingQuery start = windowedCounts.writeStream() .outputMode("update") .format("console").start(); start.awaitTermination(); ``` We can write an example like this.And input some date 1. see the matrics `stateOnCurrentVersionSizeBytes` in log.Is it increasing all the time? 2. see the Physical Plan.Whether it contains things like `HashAggregate(keys=[window#11-T10000ms, value#39]` 3. We can debug in `storeManager.remove(store, keyRow)`.Whether it will remove the old state. Closes #27268 from bettermouse/spark-30553. Authored-by: bettermouse Signed-off-by: Dongjoon Hyun --- docs/structured-streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 429d45688938a..727bda18a14fb 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -982,8 +982,8 @@ Dataset words = ... // streaming DataFrame of schema { timestamp: Timestamp Dataset windowedCounts = words .withWatermark("timestamp", "10 minutes") .groupBy( - functions.window(words.col("timestamp"), "10 minutes", "5 minutes"), - words.col("word")) + window(col("timestamp"), "10 minutes", "5 minutes"), + col("word")) .count(); {% endhighlight %} From ab0890bdb18dcd0441f6082afbe4c84219611e87 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 22 Jan 2020 15:32:58 +0900 Subject: [PATCH 1181/1387] [SPARK-28264][PYTHON][SQL] Support type hints in pandas UDF and rename/move inconsistent pandas UDF types ### What changes were proposed in this pull request? This PR proposes to redesign pandas UDFs as described in [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ```python from pyspark.sql.functions import pandas_udf import pandas as pd pandas_udf("long") def plug_one(s: pd.Series) -> pd.Series: return s + 1 spark.range(10).select(plug_one("id")).show() ``` ``` +------------+ |plug_one(id)| +------------+ | 1| | 2| | 3| | 4| | 5| | 6| | 7| | 8| | 9| | 10| +------------+ ``` Note that, this PR address one of the future improvements described [here](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit#heading=h.h3ncjpk6ujqu), "A couple of less-intuitive pandas UDF types" (by zero323) together. In short, - Adds new way with type hints as an alternative and experimental way. ```python pandas_udf(schema='...') def func(c1: Series, c2: Series) -> DataFrame: pass ``` - Replace and/or add an alias for three types below from UDF, and make them as separate standalone APIs. So, `pandas_udf` is now consistent with regular `udf`s and other expressions. `df.mapInPandas(udf)` -replace-> `df.mapInPandas(f, schema)` `df.groupby.apply(udf)` -alias-> `df.groupby.applyInPandas(f, schema)` `df.groupby.cogroup.apply(udf)` -replace-> `df.groupby.cogroup.applyInPandas(f, schema)` *`df.groupby.apply` was added from 2.3 while the other were added in the master only. - No deprecation for the existing ways for now. ```python pandas_udf(schema='...', functionType=PandasUDFType.SCALAR) def func(c1, c2): pass ``` If users are happy with this, I plan to deprecate the existing way and declare using type hints is not experimental anymore. One design goal in this PR was that, avoid touching the internal (since we didn't deprecate the old ways for now), but supports type hints with a minimised changes only at the interface. - Once we deprecate or remove the old ways, I think it requires another refactoring for the internal in the future. At the very least, we should rename internal pandas evaluation types. - If users find this experimental type hints isn't quite helpful, we should simply revert the changes at the interface level. ### Why are the changes needed? In order to address old design issues. Please see [the proposal](https://docs.google.com/document/d/1-kV0FS_LF2zvaRh_GhkV32Uqksm_Sq8SvnBBmRyxm30/edit?usp=sharing). ### Does this PR introduce any user-facing change? For behaviour changes, No. It adds new ways to use pandas UDFs by using type hints. See below. **SCALAR**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> Series: pass # DataFrame represents a struct column ``` **SCALAR_ITER**: ```python pandas_udf(schema='...') def func(iter: Iterator[Tuple[Series, DataFrame, ...]]) -> Iterator[Series]: pass # Same as SCALAR but wrapped by Iterator ``` **GROUPED_AGG**: ```python pandas_udf(schema='...') def func(c1: Series, c2: DataFrame) -> int: pass # DataFrame represents a struct column ``` **GROUPED_MAP**: This was added in Spark 2.3 as of SPARK-20396. As described above, it keeps the existing behaviour. Additionally, we now have a new alias `groupby.applyInPandas` for `groupby.apply`. See the example below: ```python def func(pdf): return pdf df.groupby("...").applyInPandas(func, schema=df.schema) ``` **MAP_ITER**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-28198; and this PR replaces the usages. See the example below: ```python def func(iter): for df in iter: yield df df.mapInPandas(func, df.schema) ``` **COGROUPED_MAP**: this is not a pandas UDF anymore This was added in Spark 3.0 as of SPARK-27463; and this PR replaces the usages. See the example below: ```python def asof_join(left, right): return pd.merge_asof(left, right, on="...", by="...") df1.groupby("...").cogroup(df2.groupby("...")).applyInPandas(asof_join, schema="...") ``` ### How was this patch tested? Unittests added and tested against Python 2.7, 3.6 and 3.7. Closes #27165 from HyukjinKwon/revisit-pandas. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/sparktestsupport/modules.py | 8 +- python/pyspark/sql/pandas/functions.py | 169 +++++------ python/pyspark/sql/pandas/group_ops.py | 111 +++++-- python/pyspark/sql/pandas/map_ops.py | 37 ++- python/pyspark/sql/pandas/typehints.py | 141 +++++++++ ...ed_map.py => test_pandas_cogrouped_map.py} | 69 ++--- ...uped_map.py => test_pandas_grouped_map.py} | 11 +- ..._pandas_udf_iter.py => test_pandas_map.py} | 22 +- python/pyspark/sql/tests/test_pandas_udf.py | 8 +- .../sql/tests/test_pandas_udf_scalar.py | 8 +- .../sql/tests/test_pandas_udf_typehints.py | 273 ++++++++++++++++++ python/pyspark/sql/tests/test_udf.py | 2 +- python/pyspark/sql/udf.py | 68 ++--- 13 files changed, 644 insertions(+), 283 deletions(-) create mode 100644 python/pyspark/sql/pandas/typehints.py rename python/pyspark/sql/tests/{test_pandas_udf_cogrouped_map.py => test_pandas_cogrouped_map.py} (74%) rename python/pyspark/sql/tests/{test_pandas_udf_grouped_map.py => test_pandas_grouped_map.py} (98%) rename python/pyspark/sql/tests/{test_pandas_udf_iter.py => test_pandas_map.py} (82%) create mode 100644 python/pyspark/sql/tests/test_pandas_udf_typehints.py diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index c4b62d971c1fe..40f2ca288d694 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -368,6 +368,7 @@ def __hash__(self): "pyspark.sql.pandas.group_ops", "pyspark.sql.pandas.types", "pyspark.sql.pandas.serializers", + "pyspark.sql.pandas.typehints", "pyspark.sql.pandas.utils", # unittests "pyspark.sql.tests.test_arrow", @@ -379,12 +380,13 @@ def __hash__(self): "pyspark.sql.tests.test_datasources", "pyspark.sql.tests.test_functions", "pyspark.sql.tests.test_group", + "pyspark.sql.tests.test_pandas_cogrouped_map", + "pyspark.sql.tests.test_pandas_grouped_map", + "pyspark.sql.tests.test_pandas_map", "pyspark.sql.tests.test_pandas_udf", - "pyspark.sql.tests.test_pandas_udf_cogrouped_map", "pyspark.sql.tests.test_pandas_udf_grouped_agg", - "pyspark.sql.tests.test_pandas_udf_grouped_map", - "pyspark.sql.tests.test_pandas_udf_iter", "pyspark.sql.tests.test_pandas_udf_scalar", + "pyspark.sql.tests.test_pandas_udf_typehints", "pyspark.sql.tests.test_pandas_udf_window", "pyspark.sql.tests.test_readwriter", "pyspark.sql.tests.test_serde", diff --git a/python/pyspark/sql/pandas/functions.py b/python/pyspark/sql/pandas/functions.py index 26241dbe68188..30602789a33a9 100644 --- a/python/pyspark/sql/pandas/functions.py +++ b/python/pyspark/sql/pandas/functions.py @@ -17,11 +17,15 @@ import functools import sys +import warnings from pyspark import since from pyspark.rdd import PythonEvalType +from pyspark.sql.pandas.typehints import infer_eval_type +from pyspark.sql.pandas.utils import require_minimum_pandas_version, require_minimum_pyarrow_version from pyspark.sql.types import DataType from pyspark.sql.udf import _create_udf +from pyspark.util import _get_argspec class PandasUDFType(object): @@ -33,12 +37,8 @@ class PandasUDFType(object): GROUPED_MAP = PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF - COGROUPED_MAP = PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF - GROUPED_AGG = PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF - MAP_ITER = PythonEvalType.SQL_MAP_PANDAS_ITER_UDF - @since(2.3) def pandas_udf(f=None, returnType=None, functionType=None): @@ -51,6 +51,10 @@ def pandas_udf(f=None, returnType=None, functionType=None): :param functionType: an enum value in :class:`pyspark.sql.functions.PandasUDFType`. Default: SCALAR. + .. seealso:: :meth:`pyspark.sql.DataFrame.mapInPandas` + .. seealso:: :meth:`pyspark.sql.GroupedData.applyInPandas` + .. seealso:: :meth:`pyspark.sql.PandasCogroupedOps.applyInPandas` + The function type of the UDF can be one of the following: 1. SCALAR @@ -337,97 +341,6 @@ def pandas_udf(f=None, returnType=None, functionType=None): .. seealso:: :meth:`pyspark.sql.GroupedData.agg` and :class:`pyspark.sql.Window` - 5. MAP_ITER - - A map iterator Pandas UDFs are used to transform data with an iterator of batches. - It can be used with :meth:`pyspark.sql.DataFrame.mapInPandas`. - - It can return the output of arbitrary length in contrast to the scalar Pandas UDF. - It maps an iterator of batches in the current :class:`DataFrame` using a Pandas user-defined - function and returns the result as a :class:`DataFrame`. - - The user-defined function should take an iterator of `pandas.DataFrame`\\s and return another - iterator of `pandas.DataFrame`\\s. All columns are passed together as an - iterator of `pandas.DataFrame`\\s to the user-defined function and the returned iterator of - `pandas.DataFrame`\\s are combined as a :class:`DataFrame`. - - >>> df = spark.createDataFrame([(1, 21), (2, 30)], - ... ("id", "age")) # doctest: +SKIP - >>> @pandas_udf(df.schema, PandasUDFType.MAP_ITER) # doctest: +SKIP - ... def filter_func(batch_iter): - ... for pdf in batch_iter: - ... yield pdf[pdf.id == 1] - >>> df.mapInPandas(filter_func).show() # doctest: +SKIP - +---+---+ - | id|age| - +---+---+ - | 1| 21| - +---+---+ - - 6. COGROUPED_MAP - - A cogrouped map UDF defines transformation: (`pandas.DataFrame`, `pandas.DataFrame`) -> - `pandas.DataFrame`. The `returnType` should be a :class:`StructType` describing the schema - of the returned `pandas.DataFrame`. The column labels of the returned `pandas.DataFrame` - must either match the field names in the defined `returnType` schema if specified as strings, - or match the field data types by position if not strings, e.g. integer indices. The length - of the returned `pandas.DataFrame` can be arbitrary. - - CoGrouped map UDFs are used with :meth:`pyspark.sql.CoGroupedData.apply`. - - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType - >>> df1 = spark.createDataFrame( - ... [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], - ... ("time", "id", "v1")) - >>> df2 = spark.createDataFrame( - ... [(20000101, 1, "x"), (20000101, 2, "y")], - ... ("time", "id", "v2")) - >>> @pandas_udf("time int, id int, v1 double, v2 string", - ... PandasUDFType.COGROUPED_MAP) # doctest: +SKIP - ... def asof_join(l, r): - ... return pd.merge_asof(l, r, on="time", by="id") - >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() # doctest: +SKIP - +---------+---+---+---+ - | time| id| v1| v2| - +---------+---+---+---+ - | 20000101| 1|1.0| x| - | 20000102| 1|3.0| x| - | 20000101| 2|2.0| y| - | 20000102| 2|4.0| y| - +---------+---+---+---+ - - Alternatively, the user can define a function that takes three arguments. In this case, - the grouping key(s) will be passed as the first argument and the data will be passed as the - second and third arguments. The grouping key(s) will be passed as a tuple of numpy data - types, e.g., `numpy.int32` and `numpy.float64`. The data will still be passed in as two - `pandas.DataFrame` containing all columns from the original Spark DataFrames. - >>> @pandas_udf("time int, id int, v1 double, v2 string", - ... PandasUDFType.COGROUPED_MAP) # doctest: +SKIP - ... def asof_join(k, l, r): - ... if k == (1,): - ... return pd.merge_asof(l, r, on="time", by="id") - ... else: - ... return pd.DataFrame(columns=['time', 'id', 'v1', 'v2']) - >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() # doctest: +SKIP - +---------+---+---+---+ - | time| id| v1| v2| - +---------+---+---+---+ - | 20000101| 1|1.0| x| - | 20000102| 1|3.0| x| - +---------+---+---+---+ - - .. note:: The user-defined functions are considered deterministic by default. Due to - optimization, duplicate invocations may be eliminated or the function may even be invoked - more times than it is present in the query. If your function is not deterministic, call - `asNondeterministic` on the user defined function. E.g.: - - >>> @pandas_udf('double', PandasUDFType.SCALAR) # doctest: +SKIP - ... def random(v): - ... import numpy as np - ... import pandas as pd - ... return pd.Series(np.random.randn(len(v)) - >>> random = random.asNondeterministic() # doctest: +SKIP - .. note:: The user-defined functions do not support conditional expressions or short circuiting in boolean expressions and it ends up with being executed all internally. If the functions can fail on special rows, the workaround is to incorporate the condition into the functions. @@ -472,6 +385,8 @@ def pandas_udf(f=None, returnType=None, functionType=None): # Note: Python 3.7.3, Pandas 0.24.2 and PyArrow 0.13.0 are used. # Note: Timezone is KST. # Note: 'X' means it throws an exception during the conversion. + require_minimum_pandas_version() + require_minimum_pyarrow_version() # decorator @pandas_udf(returnType, functionType) is_decorator = f is None or isinstance(f, (str, DataType)) @@ -490,31 +405,81 @@ def pandas_udf(f=None, returnType=None, functionType=None): eval_type = returnType else: # @pandas_udf(dataType) or @pandas_udf(returnType=dataType) - eval_type = PythonEvalType.SQL_SCALAR_PANDAS_UDF + eval_type = None else: return_type = returnType if functionType is not None: eval_type = functionType else: - eval_type = PythonEvalType.SQL_SCALAR_PANDAS_UDF + eval_type = None if return_type is None: - raise ValueError("Invalid returnType: returnType can not be None") + raise ValueError("Invalid return type: returnType can not be None") if eval_type not in [PythonEvalType.SQL_SCALAR_PANDAS_UDF, PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, - PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF]: - raise ValueError("Invalid functionType: " + PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, + None]: # None means it should infer the type from type hints. + + raise ValueError("Invalid function type: " "functionType must be one the values from PandasUDFType") if is_decorator: - return functools.partial(_create_udf, returnType=return_type, evalType=eval_type) + return functools.partial(_create_pandas_udf, returnType=return_type, evalType=eval_type) else: - return _create_udf(f=f, returnType=return_type, evalType=eval_type) + return _create_pandas_udf(f=f, returnType=return_type, evalType=eval_type) + + +def _create_pandas_udf(f, returnType, evalType): + argspec = _get_argspec(f) + + # pandas UDF by type hints. + if sys.version_info >= (3, 6): + from inspect import signature + + if evalType in [PythonEvalType.SQL_SCALAR_PANDAS_UDF, + PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, + PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF]: + warnings.warn( + "In Python 3.6+ and Spark 3.0+, it is preferred to specify type hints for " + "pandas UDF instead of specifying pandas UDF type which will be deprecated " + "in the future releases. See SPARK-28264 for more details.", UserWarning) + elif len(argspec.annotations) > 0: + evalType = infer_eval_type(signature(f)) + assert evalType is not None + + if evalType is None: + # Set default is scalar UDF. + evalType = PythonEvalType.SQL_SCALAR_PANDAS_UDF + + if (evalType == PythonEvalType.SQL_SCALAR_PANDAS_UDF or + evalType == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF) and \ + len(argspec.args) == 0 and \ + argspec.varargs is None: + raise ValueError( + "Invalid function: 0-arg pandas_udfs are not supported. " + "Instead, create a 1-arg pandas_udf and ignore the arg in your function." + ) + + if evalType == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF \ + and len(argspec.args) not in (1, 2): + raise ValueError( + "Invalid function: pandas_udf with function type GROUPED_MAP or " + "the function in groupby.applyInPandas " + "must take either one argument (data) or two arguments (key, data).") + + if evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF \ + and len(argspec.args) not in (2, 3): + raise ValueError( + "Invalid function: the function in cogroup.applyInPandas " + "must take either two arguments (left, right) " + "or three arguments (key, left, right).") + + return _create_udf(f, returnType, evalType) def _test(): diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index 00f01d2105894..3152271ba9df8 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -15,6 +15,7 @@ # limitations under the License. # import sys +import warnings from pyspark import since from pyspark.rdd import PythonEvalType @@ -28,12 +29,61 @@ class PandasGroupedOpsMixin(object): can use this class. """ + @since(2.3) def apply(self, udf): + """ + It is an alias of :meth:`pyspark.sql.GroupedData.applyInPandas`; however, it takes a + :meth:`pyspark.sql.functions.pandas_udf` whereas + :meth:`pyspark.sql.GroupedData.applyInPandas` takes a Python native function. + + .. note:: It is preferred to use :meth:`pyspark.sql.GroupedData.applyInPandas` over this + API. This API will be deprecated in the future releases. + + :param udf: a grouped map user-defined function returned by + :func:`pyspark.sql.functions.pandas_udf`. + + >>> from pyspark.sql.functions import pandas_udf, PandasUDFType + >>> df = spark.createDataFrame( + ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], + ... ("id", "v")) + >>> @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) # doctest: +SKIP + ... def normalize(pdf): + ... v = pdf.v + ... return pdf.assign(v=(v - v.mean()) / v.std()) + >>> df.groupby("id").apply(normalize).show() # doctest: +SKIP + +---+-------------------+ + | id| v| + +---+-------------------+ + | 1|-0.7071067811865475| + | 1| 0.7071067811865475| + | 2|-0.8320502943378437| + | 2|-0.2773500981126146| + | 2| 1.1094003924504583| + +---+-------------------+ + + .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` + + """ + # Columns are special because hasattr always return True + if isinstance(udf, Column) or not hasattr(udf, 'func') \ + or udf.evalType != PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF: + raise ValueError("Invalid udf: the udf argument must be a pandas_udf of type " + "GROUPED_MAP.") + + warnings.warn( + "It is preferred to use 'applyInPandas' over this " + "API. This API will be deprecated in the future releases. See SPARK-28264 for " + "more details.", UserWarning) + + return self.applyInPandas(udf.func, schema=udf.returnType) + + @since(3.0) + def applyInPandas(self, func, schema): """ Maps each group of the current :class:`DataFrame` using a pandas udf and returns the result as a `DataFrame`. - The user-defined function should take a `pandas.DataFrame` and return another + The function should take a `pandas.DataFrame` and return another `pandas.DataFrame`. For each group, all columns are passed together as a `pandas.DataFrame` to the user-function and the returned `pandas.DataFrame` are combined as a :class:`DataFrame`. @@ -45,18 +95,22 @@ def apply(self, udf): into memory, so the user should be aware of the potential OOM risk if data is skewed and certain groups are too large to fit in memory. - :param udf: a grouped map user-defined function returned by - :func:`pyspark.sql.functions.pandas_udf`. + :param func: a Python native function that takes a `pandas.DataFrame`, and outputs a + `pandas.DataFrame`. + :param schema: the return type of the `func` in PySpark. The value can be either a + :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. + + .. note:: Experimental >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> df = spark.createDataFrame( ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ... ("id", "v")) - >>> @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) # doctest: +SKIP - ... def normalize(pdf): + >>> def normalize(pdf): ... v = pdf.v ... return pdf.assign(v=(v - v.mean()) / v.std()) - >>> df.groupby("id").apply(normalize).show() # doctest: +SKIP + >>> df.groupby("id").applyInPandas(normalize, schema="id long, v double").show() + ... # doctest: +SKIP +---+-------------------+ | id| v| +---+-------------------+ @@ -71,14 +125,12 @@ def apply(self, udf): """ from pyspark.sql import GroupedData + from pyspark.sql.functions import pandas_udf, PandasUDFType assert isinstance(self, GroupedData) - # Columns are special because hasattr always return True - if isinstance(udf, Column) or not hasattr(udf, 'func') \ - or udf.evalType != PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF: - raise ValueError("Invalid udf: the udf argument must be a pandas_udf of type " - "GROUPED_MAP.") + udf = pandas_udf( + func, returnType=schema, functionType=PandasUDFType.GROUPED_MAP) df = self._df udf_column = udf(*[df[col] for col in df.columns]) jdf = self._jgd.flatMapGroupsInPandas(udf_column._jc.expr()) @@ -114,12 +166,12 @@ def __init__(self, gd1, gd2): self.sql_ctx = gd1.sql_ctx @since(3.0) - def apply(self, udf): + def applyInPandas(self, func, schema): """ - Applies a function to each cogroup using a pandas udf and returns the result + Applies a function to each cogroup using pandas and returns the result as a `DataFrame`. - The user-defined function should take two `pandas.DataFrame` and return another + The function should take two `pandas.DataFrame`\\s and return another `pandas.DataFrame`. For each side of the cogroup, all columns are passed together as a `pandas.DataFrame` to the user-function and the returned `pandas.DataFrame` are combined as a :class:`DataFrame`. @@ -133,8 +185,11 @@ def apply(self, udf): .. note:: Experimental - :param udf: a cogrouped map user-defined function returned by - :func:`pyspark.sql.functions.pandas_udf`. + :param func: a Python native function that takes two `pandas.DataFrame`\\s, and + outputs a `pandas.DataFrame`, or that takes one tuple (grouping keys) and two + pandas ``DataFrame``s, and outputs a pandas ``DataFrame``. + :param schema: the return type of the `func` in PySpark. The value can be either a + :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> df1 = spark.createDataFrame( @@ -143,11 +198,11 @@ def apply(self, udf): >>> df2 = spark.createDataFrame( ... [(20000101, 1, "x"), (20000101, 2, "y")], ... ("time", "id", "v2")) - >>> @pandas_udf("time int, id int, v1 double, v2 string", - ... PandasUDFType.COGROUPED_MAP) # doctest: +SKIP - ... def asof_join(l, r): + >>> def asof_join(l, r): ... return pd.merge_asof(l, r, on="time", by="id") - >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() # doctest: +SKIP + >>> df1.groupby("id").cogroup(df2.groupby("id")).applyInPandas( + ... asof_join, schema="time int, id int, v1 double, v2 string" + ... ).show() # doctest: +SKIP +--------+---+---+---+ | time| id| v1| v2| +--------+---+---+---+ @@ -163,14 +218,13 @@ def apply(self, udf): types, e.g., `numpy.int32` and `numpy.float64`. The data will still be passed in as two `pandas.DataFrame` containing all columns from the original Spark DataFrames. - >>> @pandas_udf("time int, id int, v1 double, v2 string", - ... PandasUDFType.COGROUPED_MAP) # doctest: +SKIP - ... def asof_join(k, l, r): + >>> def asof_join(k, l, r): ... if k == (1,): ... return pd.merge_asof(l, r, on="time", by="id") ... else: ... return pd.DataFrame(columns=['time', 'id', 'v1', 'v2']) - >>> df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() # doctest: +SKIP + >>> df1.groupby("id").cogroup(df2.groupby("id")).applyInPandas( + ... asof_join, "time int, id int, v1 double, v2 string").show() # doctest: +SKIP +--------+---+---+---+ | time| id| v1| v2| +--------+---+---+---+ @@ -181,11 +235,10 @@ def apply(self, udf): .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` """ - # Columns are special because hasattr always return True - if isinstance(udf, Column) or not hasattr(udf, 'func') \ - or udf.evalType != PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: - raise ValueError("Invalid udf: the udf argument must be a pandas_udf of type " - "COGROUPED_MAP.") + from pyspark.sql.pandas.functions import pandas_udf + + udf = pandas_udf( + func, returnType=schema, functionType=PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF) all_cols = self._extract_cols(self._gd1) + self._extract_cols(self._gd2) udf_column = udf(*all_cols) jdf = self._gd1._jgd.flatMapCoGroupsInPandas(self._gd2._jgd, udf_column._jc.expr()) diff --git a/python/pyspark/sql/pandas/map_ops.py b/python/pyspark/sql/pandas/map_ops.py index 6466d60ea74e2..75cacd797f9dd 100644 --- a/python/pyspark/sql/pandas/map_ops.py +++ b/python/pyspark/sql/pandas/map_ops.py @@ -27,29 +27,30 @@ class PandasMapOpsMixin(object): """ @since(3.0) - def mapInPandas(self, udf): + def mapInPandas(self, func, schema): """ - Maps an iterator of batches in the current :class:`DataFrame` using a Pandas user-defined - function and returns the result as a :class:`DataFrame`. + Maps an iterator of batches in the current :class:`DataFrame` using a Python native + function that takes and outputs a pandas DataFrame, and returns the result as a + :class:`DataFrame`. - The user-defined function should take an iterator of `pandas.DataFrame`\\s and return + The function should take an iterator of `pandas.DataFrame`\\s and return another iterator of `pandas.DataFrame`\\s. All columns are passed - together as an iterator of `pandas.DataFrame`\\s to the user-defined function and the + together as an iterator of `pandas.DataFrame`\\s to the function and the returned iterator of `pandas.DataFrame`\\s are combined as a :class:`DataFrame`. Each `pandas.DataFrame` size can be controlled by `spark.sql.execution.arrow.maxRecordsPerBatch`. - Its schema must match the returnType of the Pandas user-defined function. - :param udf: A function object returned by :meth:`pyspark.sql.functions.pandas_udf` + :param func: a Python native function that takes an iterator of `pandas.DataFrame`\\s, and + outputs an iterator of `pandas.DataFrame`\\s. + :param schema: the return type of the `func` in PySpark. The value can be either a + :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. >>> from pyspark.sql.functions import pandas_udf, PandasUDFType - >>> df = spark.createDataFrame([(1, 21), (2, 30)], - ... ("id", "age")) # doctest: +SKIP - >>> @pandas_udf(df.schema, PandasUDFType.MAP_ITER) # doctest: +SKIP - ... def filter_func(batch_iter): + >>> df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age")) + >>> def filter_func(batch_iter): ... for pdf in batch_iter: ... yield pdf[pdf.id == 1] - >>> df.mapInPandas(filter_func).show() # doctest: +SKIP + >>> df.mapInPandas(filter_func, df.schema).show() # doctest: +SKIP +---+---+ | id|age| +---+---+ @@ -58,17 +59,15 @@ def mapInPandas(self, udf): .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` + .. note:: Experimental """ - from pyspark.sql import Column, DataFrame + from pyspark.sql import DataFrame + from pyspark.sql.pandas.functions import pandas_udf assert isinstance(self, DataFrame) - # Columns are special because hasattr always return True - if isinstance(udf, Column) or not hasattr(udf, 'func') \ - or udf.evalType != PythonEvalType.SQL_MAP_PANDAS_ITER_UDF: - raise ValueError("Invalid udf: the udf argument must be a pandas_udf of type " - "MAP_ITER.") - + udf = pandas_udf( + func, returnType=schema, functionType=PythonEvalType.SQL_MAP_PANDAS_ITER_UDF) udf_column = udf(*[self[col] for col in self.columns]) jdf = self._jdf.mapInPandas(udf_column._jc.expr()) return DataFrame(jdf, self.sql_ctx) diff --git a/python/pyspark/sql/pandas/typehints.py b/python/pyspark/sql/pandas/typehints.py new file mode 100644 index 0000000000000..b0323ba1697df --- /dev/null +++ b/python/pyspark/sql/pandas/typehints.py @@ -0,0 +1,141 @@ +# +# 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 pyspark.sql.pandas.utils import require_minimum_pandas_version + + +def infer_eval_type(sig): + """ + Infers the evaluation type in :class:`pyspark.rdd.PythonEvalType` from + :class:`inspect.Signature` instance. + """ + from pyspark.sql.pandas.functions import PandasUDFType + + require_minimum_pandas_version() + + import pandas as pd + + annotations = {} + for param in sig.parameters.values(): + if param.annotation is not param.empty: + annotations[param.name] = param.annotation + + # Check if all arguments have type hints + parameters_sig = [annotations[parameter] for parameter + in sig.parameters if parameter in annotations] + if len(parameters_sig) != len(sig.parameters): + raise ValueError( + "Type hints for all parameters should be specified; however, got %s" % sig) + + # Check if the return has a type hint + return_annotation = sig.return_annotation + if sig.empty is return_annotation: + raise ValueError( + "Type hint for the return type should be specified; however, got %s" % sig) + + # Series, Frame or Union[DataFrame, Series], ... -> Series or Frame + is_series_or_frame = ( + all(a == pd.Series or # Series + a == pd.DataFrame or # DataFrame + check_union_annotation( # Union[DataFrame, Series] + a, + parameter_check_func=lambda na: na == pd.Series or na == pd.DataFrame) + for a in parameters_sig) and + (return_annotation == pd.Series or return_annotation == pd.DataFrame)) + + # Iterator[Tuple[Series, Frame or Union[DataFrame, Series], ...] -> Iterator[Series or Frame] + is_iterator_tuple_series_or_frame = ( + len(parameters_sig) == 1 and + check_iterator_annotation( # Iterator + parameters_sig[0], + parameter_check_func=lambda a: check_tuple_annotation( # Tuple + a, + parameter_check_func=lambda ta: ( + ta == Ellipsis or # ... + ta == pd.Series or # Series + ta == pd.DataFrame or # DataFrame + check_union_annotation( # Union[DataFrame, Series] + ta, + parameter_check_func=lambda na: ( + na == pd.Series or na == pd.DataFrame))))) and + check_iterator_annotation( + return_annotation, + parameter_check_func=lambda a: a == pd.DataFrame or a == pd.Series)) + + # Iterator[Series, Frame or Union[DataFrame, Series]] -> Iterator[Series or Frame] + is_iterator_series_or_frame = ( + len(parameters_sig) == 1 and + check_iterator_annotation( + parameters_sig[0], + parameter_check_func=lambda a: ( + a == pd.Series or # Series + a == pd.DataFrame or # DataFrame + check_union_annotation( # Union[DataFrame, Series] + a, + parameter_check_func=lambda ua: ua == pd.Series or ua == pd.DataFrame))) and + check_iterator_annotation( + return_annotation, + parameter_check_func=lambda a: a == pd.DataFrame or a == pd.Series)) + + # Series, Frame or Union[DataFrame, Series], ... -> Any + is_series_or_frame_agg = ( + all(a == pd.Series or # Series + a == pd.DataFrame or # DataFrame + check_union_annotation( # Union[DataFrame, Series] + a, + parameter_check_func=lambda ua: ua == pd.Series or ua == pd.DataFrame) + for a in parameters_sig) and ( + # It's tricky to whitelist which types pd.Series constructor can take. + # Simply blacklist common types used here for now (which becomes object + # types Spark can't recognize). + return_annotation != pd.Series and + return_annotation != pd.DataFrame and + not check_iterator_annotation(return_annotation) and + not check_tuple_annotation(return_annotation) + )) + + if is_series_or_frame: + return PandasUDFType.SCALAR + elif is_iterator_tuple_series_or_frame or is_iterator_series_or_frame: + return PandasUDFType.SCALAR_ITER + elif is_series_or_frame_agg: + return PandasUDFType.GROUPED_AGG + else: + raise NotImplementedError("Unsupported signature: %s." % sig) + + +def check_tuple_annotation(annotation, parameter_check_func=None): + # Python 3.6 has `__name__`. Python 3.7 and 3.8 have `_name`. + # Check if the name is Tuple first. After that, check the generic types. + name = getattr(annotation, "_name", getattr(annotation, "__name__", None)) + return name == "Tuple" and ( + parameter_check_func is None or all(map(parameter_check_func, annotation.__args__))) + + +def check_iterator_annotation(annotation, parameter_check_func=None): + name = getattr(annotation, "_name", getattr(annotation, "__name__", None)) + return name == "Iterator" and ( + parameter_check_func is None or all(map(parameter_check_func, annotation.__args__))) + + +def check_union_annotation(annotation, parameter_check_func=None): + import typing + + # Note that we cannot rely on '__origin__' in other type hints as it has changed from version + # to version. For example, it's abc.Iterator in Python 3.7 but typing.Iterator in Python 3.6. + origin = getattr(annotation, "__origin__", None) + return origin == typing.Union and ( + parameter_check_func is None or all(map(parameter_check_func, annotation.__args__))) diff --git a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py similarity index 74% rename from python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py rename to python/pyspark/sql/tests/test_pandas_cogrouped_map.py index bc2265fc5fe19..3ed9d2ac62fd3 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_cogrouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_cogrouped_map.py @@ -40,7 +40,7 @@ @unittest.skipIf( not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message) -class CoGroupedMapPandasUDFTests(ReusedSQLTestCase): +class CogroupedMapInPandasTests(ReusedSQLTestCase): @property def data1(self): @@ -94,13 +94,12 @@ def test_complex_group_by(self): .createDataFrame(right) \ .groupby(col('id') % 2 == 0) - @pandas_udf('k long, v long, v2 long', PandasUDFType.COGROUPED_MAP) def merge_pandas(l, r): return pd.merge(l[['k', 'v']], r[['k', 'v2']], on=['k']) result = left_gdf \ .cogroup(right_gdf) \ - .apply(merge_pandas) \ + .applyInPandas(merge_pandas, 'k long, v long, v2 long') \ .sort(['k']) \ .toPandas() @@ -116,12 +115,11 @@ def test_empty_group_by(self): left = self.data1 right = self.data2 - @pandas_udf('id long, k int, v int, v2 int', PandasUDFType.COGROUPED_MAP) def merge_pandas(l, r): return pd.merge(l, r, on=['id', 'k']) result = left.groupby().cogroup(right.groupby())\ - .apply(merge_pandas) \ + .applyInPandas(merge_pandas, 'id long, k int, v int, v2 int') \ .sort(['id', 'k']) \ .toPandas() @@ -139,10 +137,9 @@ def test_mixed_scalar_udfs_followed_by_cogrouby_apply(self): df = df.withColumn('v2', udf(lambda x: x + 1, 'int')(df['v1'])) \ .withColumn('v3', pandas_udf(lambda x: x + 2, 'int')(df['v1'])) - result = df.groupby().cogroup(df.groupby())\ - .apply(pandas_udf(lambda x, y: pd.DataFrame([(x.sum().sum(), y.sum().sum())]), - 'sum1 int, sum2 int', - PandasUDFType.COGROUPED_MAP)).collect() + result = df.groupby().cogroup(df.groupby()) \ + .applyInPandas(lambda x, y: pd.DataFrame([(x.sum().sum(), y.sum().sum())]), + 'sum1 int, sum2 int').collect() self.assertEquals(result[0]['sum1'], 165) self.assertEquals(result[0]['sum2'], 165) @@ -163,14 +160,13 @@ def test_with_key_right_group_empty(self): def test_with_key_complex(self): - @pandas_udf('id long, k int, v int, key boolean', PandasUDFType.COGROUPED_MAP) def left_assign_key(key, l, _): return l.assign(key=key[0]) result = self.data1 \ .groupby(col('id') % 2 == 0)\ .cogroup(self.data2.groupby(col('id') % 2 == 0)) \ - .apply(left_assign_key) \ + .applyInPandas(left_assign_key, 'id long, k int, v int, key boolean') \ .sort(['id', 'k']) \ .toPandas() @@ -180,59 +176,33 @@ def left_assign_key(key, l, _): assert_frame_equal(expected, result, check_column_type=_check_column_type) def test_wrong_return_type(self): + # Test that we get a sensible exception invalid values passed to apply + left = self.data1 + right = self.data2 with QuietTest(self.sc): with self.assertRaisesRegexp( NotImplementedError, - 'Invalid returnType.*cogrouped map Pandas UDF.*MapType'): - pandas_udf( - lambda l, r: l, - 'id long, v map', - PandasUDFType.COGROUPED_MAP) + 'Invalid return type.*MapType'): + left.groupby('id').cogroup(right.groupby('id')).applyInPandas( + lambda l, r: l, 'id long, v map') def test_wrong_args(self): - # Test that we get a sensible exception invalid values passed to apply left = self.data1 right = self.data2 - with QuietTest(self.sc): - # Function rather than a udf - with self.assertRaisesRegexp(ValueError, 'Invalid udf'): - left.groupby('id').cogroup(right.groupby('id')).apply(lambda l, r: l) - - # Udf missing return type - with self.assertRaisesRegexp(ValueError, 'Invalid udf'): - left.groupby('id').cogroup(right.groupby('id'))\ - .apply(udf(lambda l, r: l, DoubleType())) - - # Pass in expression rather than udf - with self.assertRaisesRegexp(ValueError, 'Invalid udf'): - left.groupby('id').cogroup(right.groupby('id')).apply(left.v + 1) - - # Zero arg function - with self.assertRaisesRegexp(ValueError, 'Invalid function'): - left.groupby('id').cogroup(right.groupby('id'))\ - .apply(pandas_udf(lambda: 1, StructType([StructField("d", DoubleType())]))) - - # Udf without PandasUDFType - with self.assertRaisesRegexp(ValueError, 'Invalid udf'): - left.groupby('id').cogroup(right.groupby('id'))\ - .apply(pandas_udf(lambda x, y: x, DoubleType())) - - # Udf with incorrect PandasUDFType - with self.assertRaisesRegexp(ValueError, 'Invalid udf.*COGROUPED_MAP'): - left.groupby('id').cogroup(right.groupby('id'))\ - .apply(pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) + with self.assertRaisesRegexp(ValueError, 'Invalid function'): + left.groupby('id').cogroup(right.groupby('id')) \ + .applyInPandas(lambda: 1, StructType([StructField("d", DoubleType())])) @staticmethod def _test_with_key(left, right, isLeft): - @pandas_udf('id long, k int, v int, key long', PandasUDFType.COGROUPED_MAP) def right_assign_key(key, l, r): return l.assign(key=key[0]) if isLeft else r.assign(key=key[0]) result = left \ .groupby('id') \ .cogroup(right.groupby('id')) \ - .apply(right_assign_key) \ + .applyInPandas(right_assign_key, 'id long, k int, v int, key long') \ .toPandas() expected = left.toPandas() if isLeft else right.toPandas() @@ -243,14 +213,13 @@ def right_assign_key(key, l, r): @staticmethod def _test_merge(left, right, output_schema='id long, k int, v int, v2 int'): - @pandas_udf(output_schema, PandasUDFType.COGROUPED_MAP) def merge_pandas(l, r): return pd.merge(l, r, on=['id', 'k']) result = left \ .groupby('id') \ .cogroup(right.groupby('id')) \ - .apply(merge_pandas)\ + .applyInPandas(merge_pandas, output_schema)\ .sort(['id', 'k']) \ .toPandas() @@ -265,7 +234,7 @@ def merge_pandas(l, r): if __name__ == "__main__": - from pyspark.sql.tests.test_pandas_udf_cogrouped_map import * + from pyspark.sql.tests.test_pandas_cogrouped_map import * try: import xmlrunner diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py b/python/pyspark/sql/tests/test_pandas_grouped_map.py similarity index 98% rename from python/pyspark/sql/tests/test_pandas_udf_grouped_map.py rename to python/pyspark/sql/tests/test_pandas_grouped_map.py index e9a970b581f13..51dd07fd7d70c 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_grouped_map.py @@ -46,7 +46,7 @@ @unittest.skipIf( not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message) -class GroupedMapPandasUDFTests(ReusedSQLTestCase): +class GroupedMapInPandasTests(ReusedSQLTestCase): @property def data(self): @@ -250,7 +250,7 @@ def test_wrong_return_type(self): with QuietTest(self.sc): with self.assertRaisesRegexp( NotImplementedError, - 'Invalid returnType.*grouped map Pandas UDF.*MapType'): + 'Invalid return type.*grouped map Pandas UDF.*MapType'): pandas_udf( lambda pdf: pdf, 'id long, v map', @@ -278,7 +278,7 @@ def test_wrong_args(self): pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) def test_unsupported_types(self): - common_err_msg = 'Invalid returnType.*grouped map Pandas UDF.*' + common_err_msg = 'Invalid return type.*grouped map Pandas UDF.*' unsupported_types = [ StructField('map', MapType(StringType(), IntegerType())), StructField('arr_ts', ArrayType(TimestampType())), @@ -533,13 +533,12 @@ def test_grouped_over_window(self): df = self.spark.createDataFrame(data, ['id', 'group', 'ts', 'result']) df = df.select(col('id'), col('group'), col('ts').cast('timestamp'), col('result')) - @pandas_udf(df.schema, PandasUDFType.GROUPED_MAP) def f(pdf): # Assign each result element the ids of the windowed group pdf['result'] = [pdf['id']] * len(pdf) return pdf - result = df.groupby('group', window('ts', '5 days')).apply(f)\ + result = df.groupby('group', window('ts', '5 days')).applyInPandas(f, df.schema)\ .select('id', 'result').collect() for r in result: self.assertListEqual(expected[r[0]], r[1]) @@ -590,7 +589,7 @@ def f(key, pdf): if __name__ == "__main__": - from pyspark.sql.tests.test_pandas_udf_grouped_map import * + from pyspark.sql.tests.test_pandas_grouped_map import * try: import xmlrunner diff --git a/python/pyspark/sql/tests/test_pandas_udf_iter.py b/python/pyspark/sql/tests/test_pandas_map.py similarity index 82% rename from python/pyspark/sql/tests/test_pandas_udf_iter.py rename to python/pyspark/sql/tests/test_pandas_map.py index 2a5709e0407ff..f1956a2523e48 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_iter.py +++ b/python/pyspark/sql/tests/test_pandas_map.py @@ -33,7 +33,7 @@ @unittest.skipIf( not have_pandas or not have_pyarrow, pandas_requirement_message or pyarrow_requirement_message) -class ScalarPandasIterUDFTests(ReusedSQLTestCase): +class MapInPandasTests(ReusedSQLTestCase): @classmethod def setUpClass(cls): @@ -57,7 +57,6 @@ def tearDownClass(cls): ReusedSQLTestCase.tearDownClass() def test_map_partitions_in_pandas(self): - @pandas_udf('id long', PandasUDFType.MAP_ITER) def func(iterator): for pdf in iterator: assert isinstance(pdf, pd.DataFrame) @@ -65,7 +64,7 @@ def func(iterator): yield pdf df = self.spark.range(10) - actual = df.mapInPandas(func).collect() + actual = df.mapInPandas(func, 'id long').collect() expected = df.collect() self.assertEquals(actual, expected) @@ -73,45 +72,40 @@ def test_multiple_columns(self): data = [(1, "foo"), (2, None), (3, "bar"), (4, "bar")] df = self.spark.createDataFrame(data, "a int, b string") - @pandas_udf(df.schema, PandasUDFType.MAP_ITER) def func(iterator): for pdf in iterator: assert isinstance(pdf, pd.DataFrame) assert [d.name for d in list(pdf.dtypes)] == ['int32', 'object'] yield pdf - actual = df.mapInPandas(func).collect() + actual = df.mapInPandas(func, df.schema).collect() expected = df.collect() self.assertEquals(actual, expected) def test_different_output_length(self): - @pandas_udf('a long', PandasUDFType.MAP_ITER) def func(iterator): for _ in iterator: yield pd.DataFrame({'a': list(range(100))}) df = self.spark.range(10) - actual = df.repartition(1).mapInPandas(func).collect() + actual = df.repartition(1).mapInPandas(func, 'a long').collect() self.assertEquals(set((r.a for r in actual)), set(range(100))) def test_empty_iterator(self): - @pandas_udf('a int, b string', PandasUDFType.MAP_ITER) def empty_iter(_): return iter([]) self.assertEqual( - self.spark.range(10).mapInPandas(empty_iter).count(), 0) + self.spark.range(10).mapInPandas(empty_iter, 'a int, b string').count(), 0) def test_empty_rows(self): - @pandas_udf('a int', PandasUDFType.MAP_ITER) def empty_rows(_): return iter([pd.DataFrame({'a': []})]) self.assertEqual( - self.spark.range(10).mapInPandas(empty_rows).count(), 0) + self.spark.range(10).mapInPandas(empty_rows, 'a int').count(), 0) def test_chain_map_partitions_in_pandas(self): - @pandas_udf('id long', PandasUDFType.MAP_ITER) def func(iterator): for pdf in iterator: assert isinstance(pdf, pd.DataFrame) @@ -119,13 +113,13 @@ def func(iterator): yield pdf df = self.spark.range(10) - actual = df.mapInPandas(func).mapInPandas(func).collect() + actual = df.mapInPandas(func, 'id long').mapInPandas(func, 'id long').collect() expected = df.collect() self.assertEquals(actual, expected) if __name__ == "__main__": - from pyspark.sql.tests.test_pandas_udf_iter import * + from pyspark.sql.tests.test_pandas_map import * try: import xmlrunner diff --git a/python/pyspark/sql/tests/test_pandas_udf.py b/python/pyspark/sql/tests/test_pandas_udf.py index e16e7b2b2e92e..94801a16ce7cd 100644 --- a/python/pyspark/sql/tests/test_pandas_udf.py +++ b/python/pyspark/sql/tests/test_pandas_udf.py @@ -116,11 +116,11 @@ def test_udf_wrong_arg(self): @pandas_udf('blah') def foo(x): return x - with self.assertRaisesRegexp(ValueError, 'Invalid returnType.*None'): + with self.assertRaisesRegexp(ValueError, 'Invalid return type.*None'): @pandas_udf(functionType=PandasUDFType.SCALAR) def foo(x): return x - with self.assertRaisesRegexp(ValueError, 'Invalid functionType'): + with self.assertRaisesRegexp(ValueError, 'Invalid function'): @pandas_udf('double', 100) def foo(x): return x @@ -132,11 +132,11 @@ def foo(x): def zero_with_type(): return 1 - with self.assertRaisesRegexp(TypeError, 'Invalid returnType'): + with self.assertRaisesRegexp(TypeError, 'Invalid return type'): @pandas_udf(returnType=PandasUDFType.GROUPED_MAP) def foo(df): return df - with self.assertRaisesRegexp(TypeError, 'Invalid returnType'): + with self.assertRaisesRegexp(TypeError, 'Invalid return type'): @pandas_udf(returnType='double', functionType=PandasUDFType.GROUPED_MAP) def foo(df): return df diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index 89e687ab71295..8e7e85ffe6b73 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -381,7 +381,7 @@ def test_vectorized_udf_nested_struct(self): with QuietTest(self.sc): with self.assertRaisesRegexp( Exception, - 'Invalid returnType with scalar Pandas UDFs'): + 'Invalid return type with scalar Pandas UDFs'): pandas_udf(lambda x: x, returnType=nested_type, functionType=udf_type) def test_vectorized_udf_complex(self): @@ -509,7 +509,7 @@ def test_vectorized_udf_wrong_return_type(self): for udf_type in [PandasUDFType.SCALAR, PandasUDFType.SCALAR_ITER]: with self.assertRaisesRegexp( NotImplementedError, - 'Invalid returnType.*scalar Pandas UDF.*MapType'): + 'Invalid return type.*scalar Pandas UDF.*MapType'): pandas_udf(lambda x: x, MapType(LongType(), LongType()), udf_type) def test_vectorized_udf_return_scalar(self): @@ -582,11 +582,11 @@ def test_vectorized_udf_unsupported_types(self): for udf_type in [PandasUDFType.SCALAR, PandasUDFType.SCALAR_ITER]: with self.assertRaisesRegexp( NotImplementedError, - 'Invalid returnType.*scalar Pandas UDF.*MapType'): + 'Invalid return type.*scalar Pandas UDF.*MapType'): pandas_udf(lambda x: x, MapType(StringType(), IntegerType()), udf_type) with self.assertRaisesRegexp( NotImplementedError, - 'Invalid returnType.*scalar Pandas UDF.*ArrayType.StructType'): + 'Invalid return type.*scalar Pandas UDF.*ArrayType.StructType'): pandas_udf(lambda x: x, ArrayType(StructType([StructField('a', IntegerType())])), udf_type) diff --git a/python/pyspark/sql/tests/test_pandas_udf_typehints.py b/python/pyspark/sql/tests/test_pandas_udf_typehints.py new file mode 100644 index 0000000000000..7c83c78f108d9 --- /dev/null +++ b/python/pyspark/sql/tests/test_pandas_udf_typehints.py @@ -0,0 +1,273 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +import sys +import unittest +import inspect + +from pyspark.sql.functions import mean, lit +from pyspark.testing.sqlutils import ReusedSQLTestCase, \ + have_pandas, have_pyarrow, pandas_requirement_message, \ + pyarrow_requirement_message +from pyspark.sql.pandas.typehints import infer_eval_type +from pyspark.sql.pandas.functions import pandas_udf, PandasUDFType + +if have_pandas: + import pandas as pd + from pandas.util.testing import assert_frame_equal + +python_requirement_message = "pandas UDF with type hints are supported with Python 3.6+." + + +@unittest.skipIf( + not have_pandas or not have_pyarrow or sys.version_info[:2] < (3, 6), + pandas_requirement_message or pyarrow_requirement_message or python_requirement_message) +class PandasUDFTypeHintsTests(ReusedSQLTestCase): + # Note that, we should remove `exec` once we drop Python 2 in this class. + + def setUp(self): + self.local = {'pd': pd} + + def test_type_annotation_scalar(self): + exec( + "def func(col: pd.Series) -> pd.Series: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + + exec( + "def func(col: pd.DataFrame, col1: pd.Series) -> pd.DataFrame: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + + exec( + "def func(col: pd.DataFrame, *args: pd.Series) -> pd.Series: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + + exec( + "def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame) -> pd.Series:\n" + " pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + + exec( + "def func(col: pd.Series, *, col2: pd.DataFrame) -> pd.DataFrame:\n" + " pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + + exec( + "from typing import Union\n" + "def func(col: Union[pd.Series, pd.DataFrame], *, col2: pd.DataFrame) -> pd.Series:\n" + " pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR) + + def test_type_annotation_scalar_iter(self): + exec( + "from typing import Iterator\n" + "def func(iter: Iterator[pd.Series]) -> Iterator[pd.Series]: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR_ITER) + + exec( + "from typing import Iterator, Tuple\n" + "def func(iter: Iterator[Tuple[pd.DataFrame, pd.Series]]) -> Iterator[pd.DataFrame]:\n" + " pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR_ITER) + + exec( + "from typing import Iterator, Tuple\n" + "def func(iter: Iterator[Tuple[pd.DataFrame, ...]]) -> Iterator[pd.Series]: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR_ITER) + + exec( + "from typing import Iterator, Tuple, Union\n" + "def func(iter: Iterator[Tuple[Union[pd.DataFrame, pd.Series], ...]])" + " -> Iterator[pd.Series]: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.SCALAR_ITER) + + def test_type_annotation_group_agg(self): + exec( + "def func(col: pd.Series) -> str: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + + exec( + "def func(col: pd.DataFrame, col1: pd.Series) -> int: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + + exec( + "from pyspark.sql import Row\n" + "def func(col: pd.DataFrame, *args: pd.Series) -> Row: pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + + exec( + "def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame) -> str:\n" + " pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + + exec( + "def func(col: pd.Series, *, col2: pd.DataFrame) -> float:\n" + " pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + + exec( + "from typing import Union\n" + "def func(col: Union[pd.Series, pd.DataFrame], *, col2: pd.DataFrame) -> float:\n" + " pass", + self.local) + self.assertEqual( + infer_eval_type(inspect.signature(self.local['func'])), PandasUDFType.GROUPED_AGG) + + def test_type_annotation_negative(self): + exec( + "def func(col: str) -> pd.Series: pass", + self.local) + self.assertRaisesRegex( + NotImplementedError, + "Unsupported signature.*str", + infer_eval_type, inspect.signature(self.local['func'])) + + exec( + "def func(col: pd.DataFrame, col1: int) -> pd.DataFrame: pass", + self.local) + self.assertRaisesRegex( + NotImplementedError, + "Unsupported signature.*int", + infer_eval_type, inspect.signature(self.local['func'])) + + exec( + "from typing import Union\n" + "def func(col: Union[pd.DataFrame, str], col1: int) -> pd.DataFrame: pass", + self.local) + self.assertRaisesRegex( + NotImplementedError, + "Unsupported signature.*str", + infer_eval_type, inspect.signature(self.local['func'])) + + exec( + "from typing import Tuple\n" + "def func(col: pd.Series) -> Tuple[pd.DataFrame]: pass", + self.local) + self.assertRaisesRegex( + NotImplementedError, + "Unsupported signature.*Tuple", + infer_eval_type, inspect.signature(self.local['func'])) + + exec( + "def func(col, *args: pd.Series) -> pd.Series: pass", + self.local) + self.assertRaisesRegex( + ValueError, + "should be specified.*Series", + infer_eval_type, inspect.signature(self.local['func'])) + + exec( + "def func(col: pd.Series, *args: pd.Series, **kwargs: pd.DataFrame):\n" + " pass", + self.local) + self.assertRaisesRegex( + ValueError, + "should be specified.*Series", + infer_eval_type, inspect.signature(self.local['func'])) + + exec( + "def func(col: pd.Series, *, col2) -> pd.DataFrame:\n" + " pass", + self.local) + self.assertRaisesRegex( + ValueError, + "should be specified.*Series", + infer_eval_type, inspect.signature(self.local['func'])) + + def test_scalar_udf_type_hint(self): + df = self.spark.range(10).selectExpr("id", "id as v") + + exec( + "import typing\n" + "def plus_one(v: typing.Union[pd.Series, pd.DataFrame]) -> pd.Series:\n" + " return v + 1", + self.local) + + plus_one = pandas_udf("long")(self.local["plus_one"]) + + actual = df.select(plus_one(df.v).alias("plus_one")) + expected = df.selectExpr("(v + 1) as plus_one") + assert_frame_equal(expected.toPandas(), actual.toPandas()) + + def test_scalar_iter_udf_type_hint(self): + df = self.spark.range(10).selectExpr("id", "id as v") + + exec( + "import typing\n" + "def plus_one(itr: typing.Iterator[pd.Series]) -> typing.Iterator[pd.Series]:\n" + " for s in itr:\n" + " yield s + 1", + self.local) + + plus_one = pandas_udf("long")(self.local["plus_one"]) + + actual = df.select(plus_one(df.v).alias("plus_one")) + expected = df.selectExpr("(v + 1) as plus_one") + assert_frame_equal(expected.toPandas(), actual.toPandas()) + + def test_group_agg_udf_type_hint(self): + df = self.spark.range(10).selectExpr("id", "id as v") + exec( + "import numpy as np\n" + "def weighted_mean(v: pd.Series, w: pd.Series) -> float:\n" + " return np.average(v, weights=w)", + self.local) + + weighted_mean = pandas_udf("double")(self.local["weighted_mean"]) + + actual = df.groupby('id').agg(weighted_mean(df.v, lit(1.0))).sort('id') + expected = df.groupby('id').agg(mean(df.v).alias('weighted_mean(v, 1.0)')).sort('id') + assert_frame_equal(expected.toPandas(), actual.toPandas()) + + +if __name__ == "__main__": + from pyspark.sql.tests.test_pandas_udf_typehints import * + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index 3b9f12f575462..061d3f5e1f7ac 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -96,7 +96,7 @@ def test_udf_registration_return_type_none(self): def test_udf_registration_return_type_not_none(self): with QuietTest(self.sc): - with self.assertRaisesRegexp(TypeError, "Invalid returnType"): + with self.assertRaisesRegexp(TypeError, "Invalid return type"): self.spark.catalog.registerFunction( "f", UserDefinedFunction(lambda x, y: len(x) + y, StringType()), StringType()) diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 8e809b3556256..433c5fc845c59 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -25,7 +25,6 @@ from pyspark.sql.column import Column, _to_java_column, _to_seq from pyspark.sql.types import StringType, DataType, StructType, _parse_datatype_string from pyspark.sql.pandas.types import to_arrow_type -from pyspark.util import _get_argspec __all__ = ["UDFRegistration"] @@ -38,41 +37,6 @@ def _wrap_function(sc, func, returnType): def _create_udf(f, returnType, evalType): - - if evalType in (PythonEvalType.SQL_SCALAR_PANDAS_UDF, - PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF, - PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, - PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF, - PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, - PythonEvalType.SQL_MAP_PANDAS_ITER_UDF): - - from pyspark.sql.pandas.utils import require_minimum_pyarrow_version - require_minimum_pyarrow_version() - - argspec = _get_argspec(f) - - if (evalType == PythonEvalType.SQL_SCALAR_PANDAS_UDF or - evalType == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF) and \ - len(argspec.args) == 0 and \ - argspec.varargs is None: - raise ValueError( - "Invalid function: 0-arg pandas_udfs are not supported. " - "Instead, create a 1-arg pandas_udf and ignore the arg in your function." - ) - - if evalType == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF \ - and len(argspec.args) not in (1, 2): - raise ValueError( - "Invalid function: pandas_udfs with function type GROUPED_MAP " - "must take either one argument (data) or two arguments (key, data).") - - if evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF \ - and len(argspec.args) not in (2, 3): - raise ValueError( - "Invalid function: pandas_udfs with function type COGROUPED_MAP " - "must take either two arguments (left, right) " - "or three arguments (key, left, right).") - # Set the name of the UserDefinedFunction object to be the name of function f udf_obj = UserDefinedFunction( f, returnType=returnType, name=None, evalType=evalType, deterministic=True) @@ -101,12 +65,12 @@ def __init__(self, func, if not isinstance(returnType, (DataType, str)): raise TypeError( - "Invalid returnType: returnType should be DataType or str " + "Invalid return type: returnType should be DataType or str " "but is {}".format(returnType)) if not isinstance(evalType, int): raise TypeError( - "Invalid evalType: evalType should be an int but is {}".format(evalType)) + "Invalid evaluation type: evalType should be an int but is {}".format(evalType)) self.func = func self._returnType = returnType @@ -135,7 +99,7 @@ def returnType(self): to_arrow_type(self._returnType_placeholder) except TypeError: raise NotImplementedError( - "Invalid returnType with scalar Pandas UDFs: %s is " + "Invalid return type with scalar Pandas UDFs: %s is " "not supported" % str(self._returnType_placeholder)) elif self.evalType == PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF: if isinstance(self._returnType_placeholder, StructType): @@ -143,33 +107,35 @@ def returnType(self): to_arrow_type(self._returnType_placeholder) except TypeError: raise NotImplementedError( - "Invalid returnType with grouped map Pandas UDFs: " - "%s is not supported" % str(self._returnType_placeholder)) + "Invalid return type with grouped map Pandas UDFs or " + "at groupby.applyInPandas: %s is not supported" % str( + self._returnType_placeholder)) else: - raise TypeError("Invalid returnType for grouped map Pandas " - "UDFs: returnType must be a StructType.") + raise TypeError("Invalid return type for grouped map Pandas " + "UDFs or at groupby.applyInPandas: return type must be a " + "StructType.") elif self.evalType == PythonEvalType.SQL_MAP_PANDAS_ITER_UDF: if isinstance(self._returnType_placeholder, StructType): try: to_arrow_type(self._returnType_placeholder) except TypeError: raise NotImplementedError( - "Invalid returnType with map iterator Pandas UDFs: " + "Invalid return type in mapInPandas: " "%s is not supported" % str(self._returnType_placeholder)) else: - raise TypeError("Invalid returnType for map iterator Pandas " - "UDFs: returnType must be a StructType.") + raise TypeError("Invalid return type in mapInPandas: " + "return type must be a StructType.") elif self.evalType == PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF: if isinstance(self._returnType_placeholder, StructType): try: to_arrow_type(self._returnType_placeholder) except TypeError: raise NotImplementedError( - "Invalid returnType with cogrouped map Pandas UDFs: " + "Invalid return type in cogroup.applyInPandas: " "%s is not supported" % str(self._returnType_placeholder)) else: - raise TypeError("Invalid returnType for cogrouped map Pandas " - "UDFs: returnType must be a StructType.") + raise TypeError("Invalid return type in cogroup.applyInPandas: " + "return type must be a StructType.") elif self.evalType == PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF: try: # StructType is not yet allowed as a return type, explicitly check here to fail fast @@ -178,7 +144,7 @@ def returnType(self): to_arrow_type(self._returnType_placeholder) except TypeError: raise NotImplementedError( - "Invalid returnType with grouped aggregate Pandas UDFs: " + "Invalid return type with grouped aggregate Pandas UDFs: " "%s is not supported" % str(self._returnType_placeholder)) return self._returnType_placeholder @@ -358,7 +324,7 @@ def register(self, name, f, returnType=None): if hasattr(f, 'asNondeterministic'): if returnType is not None: raise TypeError( - "Invalid returnType: data type can not be specified when f is" + "Invalid return type: data type can not be specified when f is" "a user-defined function, but got %s." % returnType) if f.evalType not in [PythonEvalType.SQL_BATCHED_UDF, PythonEvalType.SQL_SCALAR_PANDAS_UDF, From f2d71f5838f4a611e3ebd60f49ee80521d772524 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 22 Jan 2020 16:00:05 +0800 Subject: [PATCH 1182/1387] [SPARK-30591][SQL] Remove the nonstandard SET OWNER syntax for namespaces ### What changes were proposed in this pull request? This pr removes the nonstandard `SET OWNER` syntax for namespaces and changes the owner reserved properties from `ownerName` and `ownerType` to `owner`. ### Why are the changes needed? the `SET OWNER` syntax for namespaces is hive-specific and non-sql standard, we need a more future-proofing design before we implement user-facing changes for SQL security issues ### Does this PR introduce any user-facing change? no, just revert an unpublic syntax ### How was this patch tested? modified uts Closes #27300 from yaooqinn/SPARK-30591. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 1 - docs/sql-migration-guide.md | 32 +---------- .../spark/sql/catalyst/parser/SqlBase.g4 | 5 -- .../connector/catalog/SupportsNamespaces.java | 10 +--- .../sql/catalyst/parser/AstBuilder.scala | 32 ++--------- .../catalyst/plans/logical/v2Commands.scala | 10 ---- .../catalog/ExternalCatalogSuite.scala | 7 ++- .../catalog/SessionCatalogSuite.scala | 7 ++- .../sql/catalyst/parser/DDLParserSuite.scala | 13 ----- .../spark/sql/execution/command/ddl.scala | 5 +- .../datasources/v2/CreateNamespaceExec.scala | 2 +- .../datasources/v2/DataSourceV2Strategy.scala | 5 -- .../v2/DescribeNamespaceExec.scala | 14 ++--- .../sql/connector/DataSourceV2SQLSuite.scala | 49 ++++++----------- .../sql/execution/command/DDLSuite.scala | 10 ++-- .../sql/hive/client/HiveClientImpl.scala | 18 +++---- .../spark/sql/hive/client/HiveShim.scala | 28 ---------- .../spark/sql/hive/client/VersionsSuite.scala | 10 ++-- .../sql/hive/execution/HiveDDLSuite.scala | 53 +++++-------------- 19 files changed, 66 insertions(+), 245 deletions(-) diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 4d8ae710fa73b..b18855366bb2b 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -197,7 +197,6 @@ Below is a list of all the keywords in Spark SQL. OVERLAPSreservednon-reservedreserved OVERLAYnon-reservednon-reservednon-reserved OVERWRITEnon-reservednon-reservednon-reserved - OWNERnon-reservednon-reservednon-reserved PARTITIONnon-reservednon-reservedreserved PARTITIONEDnon-reservednon-reservednon-reserved PARTITIONSnon-reservednon-reservednon-reserved diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index eec1fb260b396..73d2d599aff49 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -308,46 +308,18 @@ license: | For databases and tables, please use the LOCATION clause to specify it. - - - ownerName - - - yes - - - no - - - For databases, please use the ALTER DATABASE ... SET OWNER syntax to modify it - - - - - ownerType - - - yes - - - no - - - For databases, please use the ALTER DATABASE ... SET OWNER syntax to modify it - - owner - no + yes yes - For tables, it is determined by the user who runs spark and create the table. + For databases and tables, it is determined by the user who runs spark and create the table. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 645d0d709e063..358943fc484ca 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -103,8 +103,6 @@ statement SET (DBPROPERTIES | PROPERTIES) tablePropertyList #setNamespaceProperties | ALTER namespace multipartIdentifier SET locationSpec #setNamespaceLocation - | ALTER namespace multipartIdentifier - SET OWNER ownerType=(USER | ROLE | GROUP) identifier #setNamespaceOwner | DROP namespace (IF EXISTS)? multipartIdentifier (RESTRICT | CASCADE)? #dropNamespace | SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)? @@ -1100,7 +1098,6 @@ ansiNonReserved | OVER | OVERLAY | OVERWRITE - | OWNER | PARTITION | PARTITIONED | PARTITIONS @@ -1353,7 +1350,6 @@ nonReserved | OVERLAPS | OVERLAY | OVERWRITE - | OWNER | PARTITION | PARTITIONED | PARTITIONS @@ -1612,7 +1608,6 @@ OVER: 'OVER'; OVERLAPS: 'OVERLAPS'; OVERLAY: 'OVERLAY'; OVERWRITE: 'OVERWRITE'; -OWNER: 'OWNER'; PARTITION: 'PARTITION'; PARTITIONED: 'PARTITIONED'; PARTITIONS: 'PARTITIONS'; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java index d58fe611b2704..757d303e82255 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java @@ -56,12 +56,7 @@ public interface SupportsNamespaces extends CatalogPlugin { /** * A property to specify the owner of the namespace. */ - String PROP_OWNER_NAME = "ownerName"; - - /** - * A property to specify the type of the namespace's owner. - */ - String PROP_OWNER_TYPE = "ownerType"; + String PROP_OWNER = "owner"; /** * The list of reserved namespace properties, which can not be removed or changed directly by @@ -72,8 +67,7 @@ public interface SupportsNamespaces extends CatalogPlugin { * * They need specific syntax to modify */ - List RESERVED_PROPERTIES = - Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_OWNER_NAME, PROP_OWNER_TYPE); + List RESERVED_PROPERTIES = Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_OWNER); /** * Return a default namespace for the catalog. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 2050ec3399303..a8a96f0f6803a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2533,13 +2533,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging throw new ParseException(s"$PROP_LOCATION is a reserved namespace property, please use" + s" the LOCATION clause to specify it.", ctx) case (PROP_LOCATION, _) => false - case (ownership, _) if ownership == PROP_OWNER_NAME || ownership == PROP_OWNER_TYPE => - if (legacyOn) { - false - } else { - throw new ParseException(s"$ownership is a reserved namespace property , please use" + - " ALTER NAMESPACE ... SET OWNER ... to specify it.", ctx) - } + case (PROP_OWNER, _) if !legacyOn => + throw new ParseException(s"$PROP_OWNER is a reserved namespace property, it will be" + + s" set to the current user.", ctx) + case (PROP_OWNER, _) => false case _ => true } } @@ -2682,7 +2679,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case (PROP_LOCATION, _) => false case (PROP_OWNER, _) if !legacyOn => throw new ParseException(s"$PROP_OWNER is a reserved table property, it will be" + - s" set to the current user by default.", ctx) + s" set to the current user", ctx) case (PROP_OWNER, _) => false case _ => true } @@ -3607,23 +3604,4 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier) CommentOnTable(UnresolvedTable(nameParts), comment) } - - /** - * Create an [[AlterNamespaceSetOwner]] logical plan. - * - * For example: - * {{{ - * ALTER (DATABASE|SCHEMA|NAMESPACE) namespace SET OWNER (USER|ROLE|GROUP) identityName; - * }}} - */ - override def visitSetNamespaceOwner(ctx: SetNamespaceOwnerContext): LogicalPlan = { - withOrigin(ctx) { - val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier) - AlterNamespaceSetOwner( - UnresolvedNamespace(nameParts), - ctx.identifier.getText, - ctx.ownerType.getText) - } - } - } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index d62869e912989..3e3c81c22b61d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -292,16 +292,6 @@ case class AlterNamespaceSetLocation( override def children: Seq[LogicalPlan] = Seq(namespace) } -/** - * ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET OWNER command, as parsed from SQL. - */ -case class AlterNamespaceSetOwner( - child: LogicalPlan, - ownerName: String, - ownerType: String) extends Command { - override def children: Seq[LogicalPlan] = child :: Nil -} - /** * The logical plan of the SHOW NAMESPACES command that works for v2 catalogs. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 6b869990c2f54..55712d0da518d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.analysis.{FunctionAlreadyExistsException, N import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -144,9 +144,8 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac // Note: alter properties here because Hive does not support altering other fields catalog.alterDatabase(db1.copy(properties = Map("k" -> "v3", "good" -> "true"))) val newDb1 = catalog.getDatabase("db1") - val reversedProperties = Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE) - assert((db1.properties -- reversedProperties).isEmpty) - assert((newDb1.properties -- reversedProperties).size == 2) + assert((db1.properties -- Seq(PROP_OWNER)).isEmpty) + assert((newDb1.properties -- Seq(PROP_OWNER)).size == 2) assert(newDb1.properties.get("k") == Some("v3")) assert(newDb1.properties.get("good") == Some("true")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 4ccbc0aefc3ea..0d9e2f61e812a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View} import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -209,9 +209,8 @@ abstract class SessionCatalogSuite extends AnalysisTest { // Note: alter properties here because Hive does not support altering other fields catalog.alterDatabase(db1.copy(properties = Map("k" -> "v3", "good" -> "true"))) val newDb1 = catalog.getDatabaseMetadata("db1") - val reversedProperties = Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE) - assert((db1.properties -- reversedProperties).isEmpty) - assert((newDb1.properties -- reversedProperties).size == 2) + assert((db1.properties -- Seq(PROP_OWNER)).isEmpty) + assert((newDb1.properties -- Seq(PROP_OWNER)).size == 2) assert(newDb1.properties.get("k") == Some("v3")) assert(newDb1.properties.get("good") == Some("true")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 5e45b57187f18..0bcfccdd8b90a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1343,19 +1343,6 @@ class DDLParserSuite extends AnalysisTest { UnresolvedNamespace(Seq("a", "b", "c")), "/home/user/db")) } - test("set namespace owner") { - comparePlans( - parsePlan("ALTER DATABASE a.b.c SET OWNER USER user1"), - AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "user1", "USER")) - - comparePlans( - parsePlan("ALTER DATABASE a.b.c SET OWNER ROLE role1"), - AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "role1", "ROLE")) - comparePlans( - parsePlan("ALTER DATABASE a.b.c SET OWNER GROUP group1"), - AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "group1", "GROUP")) - } - test("show databases: basic") { comparePlans( parsePlan("SHOW DATABASES"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index d75b39610a754..9682778d40383 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -178,10 +178,9 @@ case class DescribeDatabaseCommand( val allDbProperties = dbMetadata.properties val result = Row("Database Name", dbMetadata.name) :: - Row("Description", dbMetadata.description) :: + Row("Comment", dbMetadata.description) :: Row("Location", CatalogUtils.URIToString(dbMetadata.locationUri)):: - Row("Owner Name", allDbProperties.getOrElse(PROP_OWNER_NAME, "")) :: - Row("Owner Type", allDbProperties.getOrElse(PROP_OWNER_TYPE, "")) :: Nil + Row("Owner", allDbProperties.getOrElse(PROP_OWNER, "")) :: Nil if (extended) { val properties = allDbProperties -- RESERVED_PROPERTIES.asScala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala index 5af592cfd5f5a..d5b81d13a7cc4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala @@ -42,7 +42,7 @@ case class CreateNamespaceExec( if (!catalog.namespaceExists(ns)) { try { val ownership = - Map(PROP_OWNER_NAME -> Utils.getCurrentUserName(), PROP_OWNER_TYPE -> "USER") + Map(PROP_OWNER -> Utils.getCurrentUserName()) catalog.createNamespace(ns, (properties ++ ownership).asJava) } catch { case _: NamespaceAlreadyExistsException if ifNotExists => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index c6d8a12a94549..7c8fd4e105ca7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -297,11 +297,6 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case r @ ShowTableProperties(ResolvedTable(_, _, table), propertyKey) => ShowTablePropertiesExec(r.output, table, propertyKey) :: Nil - case AlterNamespaceSetOwner(ResolvedNamespace(catalog, namespace), name, typ) => - val properties = - Map(SupportsNamespaces.PROP_OWNER_NAME -> name, SupportsNamespaces.PROP_OWNER_TYPE -> typ) - AlterNamespaceSetPropertiesExec(catalog, namespace, properties) :: Nil - case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala index 979d740efa8ff..9a0481ef01b63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala @@ -43,17 +43,9 @@ case class DescribeNamespaceExec( val metadata = catalog.loadNamespaceMetadata(ns) rows += toCatalystRow("Namespace Name", ns.last) - Option(metadata.get(PROP_COMMENT)).foreach { - rows += toCatalystRow("Description", _) - } - Option(metadata.get(PROP_LOCATION)).foreach { - rows += toCatalystRow("Location", _) - } - Option(metadata.get(PROP_OWNER_NAME)).foreach { - rows += toCatalystRow("Owner Name", _) - } - Option(metadata.get(PROP_OWNER_TYPE)).foreach { - rows += toCatalystRow("Owner Type", _) + + SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { p => + rows ++= Option(metadata.get(p)).map(toCatalystRow(p.capitalize, _)) } if (isExtended) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 4dd70fa9626d3..75c9bb7be05f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1082,11 +1082,10 @@ class DataSourceV2SQLSuite val description = descriptionDf.collect() assert(description === Seq( Row("Namespace Name", "ns2"), - Row("Description", "test namespace"), - Row("Location", "/tmp/ns_test"), - Row("Owner Name", defaultUser), - Row("Owner Type", "USER") - )) + Row(SupportsNamespaces.PROP_COMMENT.capitalize, "test namespace"), + Row(SupportsNamespaces.PROP_LOCATION.capitalize, "/tmp/ns_test"), + Row(SupportsNamespaces.PROP_OWNER.capitalize, defaultUser)) + ) } } @@ -1098,12 +1097,11 @@ class DataSourceV2SQLSuite val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") assert(descriptionDf.collect() === Seq( Row("Namespace Name", "ns2"), - Row("Description", "test namespace"), - Row("Location", "/tmp/ns_test"), - Row("Owner Name", defaultUser), - Row("Owner Type", "USER"), - Row("Properties", "((a,b),(b,a),(c,c))") - )) + Row(SupportsNamespaces.PROP_COMMENT.capitalize, "test namespace"), + Row(SupportsNamespaces.PROP_LOCATION.capitalize, "/tmp/ns_test"), + Row(SupportsNamespaces.PROP_OWNER.capitalize, defaultUser), + Row("Properties", "((a,b),(b,a),(c,c))")) + ) } } @@ -1146,27 +1144,10 @@ class DataSourceV2SQLSuite val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") assert(descriptionDf.collect() === Seq( Row("Namespace Name", "ns2"), - Row("Description", "test namespace"), - Row("Location", "/tmp/ns_test_2"), - Row("Owner Name", defaultUser), - Row("Owner Type", "USER") - )) - } - } - - test("AlterNamespaceSetOwner using v2 catalog") { - withNamespace("testcat.ns1.ns2") { - sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + - "'test namespace' LOCATION '/tmp/ns_test_3'") - sql("ALTER NAMESPACE testcat.ns1.ns2 SET OWNER ROLE adminRole") - val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") - assert(descriptionDf.collect() === Seq( - Row("Namespace Name", "ns2"), - Row("Description", "test namespace"), - Row("Location", "/tmp/ns_test_3"), - Row("Owner Name", "adminRole"), - Row("Owner Type", "ROLE") - )) + Row(SupportsNamespaces.PROP_COMMENT.capitalize, "test namespace"), + Row(SupportsNamespaces.PROP_LOCATION.capitalize, "/tmp/ns_test_2"), + Row(SupportsNamespaces.PROP_OWNER.capitalize, defaultUser)) + ) } } @@ -2190,7 +2171,7 @@ class DataSourceV2SQLSuite Option(comment).map("'" + _ + "'").getOrElse("NULL")) val expectedComment = Option(comment).getOrElse("") assert(sql(s"DESC NAMESPACE extended $namespace").toDF("k", "v") - .where("k='Description'") + .where(s"k='${SupportsNamespaces.PROP_COMMENT.capitalize}'") .head().getString(1) === expectedComment) } @@ -2228,7 +2209,7 @@ class DataSourceV2SQLSuite sql(s"COMMENT ON TABLE $tableName IS " + Option(comment).map("'" + _ + "'").getOrElse("NULL")) val expectedComment = Option(comment).getOrElse("") assert(sql(s"DESC extended $tableName").toDF("k", "v", "c") - .where("k='Comment'") + .where(s"k='${TableCatalog.PROP_COMMENT.capitalize}'") .head().getString(1) === expectedComment) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 64c85c19ba489..e3fb535ab4cdd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchDatabaseException, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -228,7 +228,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { abstract class DDLSuite extends QueryTest with SQLTestUtils { - protected val reversedProperties = Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE) + protected val reversedProperties = Seq(PROP_OWNER) protected def isUsingHiveMetastore: Boolean = { spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive" @@ -795,7 +795,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") .where("key not like 'Owner%'"), // filter for consistency with in-memory catalog Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: + Row("Comment", "") :: Row("Location", CatalogUtils.URIToString(location)) :: Row("Properties", "") :: Nil) @@ -805,7 +805,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") .where("key not like 'Owner%'"), // filter for consistency with in-memory catalog Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: + Row("Comment", "") :: Row("Location", CatalogUtils.URIToString(location)) :: Row("Properties", "((a,a), (b,b), (c,c))") :: Nil) @@ -815,7 +815,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") .where("key not like 'Owner%'"), // filter for consistency with in-memory catalog Row("Database Name", dbNameWithoutBackTicks) :: - Row("Description", "") :: + Row("Comment", "") :: Row("Location", CatalogUtils.URIToString(location)) :: Row("Properties", "((a,a), (b,b), (c,c), (d,d))") :: Nil) 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 f196e94a83f97..b5c5f0e9381bc 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 @@ -355,7 +355,7 @@ private[hive] class HiveClientImpl( override def createDatabase( database: CatalogDatabase, ignoreIfExists: Boolean): Unit = withHiveState { - val hiveDb = toHiveDatabase(database, true) + val hiveDb = toHiveDatabase(database, Some(userName)) client.createDatabase(hiveDb, ignoreIfExists) } @@ -374,32 +374,28 @@ private[hive] class HiveClientImpl( s"Hive ${version.fullVersion} does not support altering database location") } } - val hiveDb = toHiveDatabase(database, false) + val hiveDb = toHiveDatabase(database) client.alterDatabase(database.name, hiveDb) } - private def toHiveDatabase(database: CatalogDatabase, isCreate: Boolean): HiveDatabase = { + private def toHiveDatabase( + database: CatalogDatabase, userName: Option[String] = None): HiveDatabase = { val props = database.properties val hiveDb = new HiveDatabase( database.name, database.description, CatalogUtils.URIToString(database.locationUri), - (props -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE)).asJava) - props.get(PROP_OWNER_NAME).orElse(if (isCreate) Some(userName) else None).foreach { ownerName => + (props -- Seq(PROP_OWNER)).asJava) + props.get(PROP_OWNER).orElse(userName).foreach { ownerName => shim.setDatabaseOwnerName(hiveDb, ownerName) } - props.get(PROP_OWNER_TYPE).orElse(if (isCreate) Some(PrincipalType.USER.name) else None) - .foreach { ownerType => - shim.setDatabaseOwnerType(hiveDb, ownerType) - } hiveDb } override def getDatabase(dbName: String): CatalogDatabase = withHiveState { Option(client.getDatabase(dbName)).map { d => val paras = Option(d.getParameters).map(_.asScala.toMap).getOrElse(Map()) ++ - Map(PROP_OWNER_NAME -> shim.getDatabaseOwnerName(d), - PROP_OWNER_TYPE -> shim.getDatabaseOwnerType(d)) + Map(PROP_OWNER -> shim.getDatabaseOwnerName(d)) CatalogDatabase( name = d.getName, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index d8078a534a874..50ce536a160c8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -157,10 +157,6 @@ private[client] sealed abstract class Shim { def setDatabaseOwnerName(db: Database, owner: String): Unit - def getDatabaseOwnerType(db: Database): String - - def setDatabaseOwnerType(db: Database, ownerType: String): Unit - protected def findStaticMethod(klass: Class[_], name: String, args: Class[_]*): Method = { val method = findMethod(klass, name, args: _*) require(Modifier.isStatic(method.getModifiers()), @@ -467,10 +463,6 @@ private[client] class Shim_v0_12 extends Shim with Logging { override def getDatabaseOwnerName(db: Database): String = "" override def setDatabaseOwnerName(db: Database, owner: String): Unit = {} - - override def getDatabaseOwnerType(db: Database): String = "" - - override def setDatabaseOwnerType(db: Database, ownerType: String): Unit = {} } private[client] class Shim_v0_13 extends Shim_v0_12 { @@ -519,17 +511,6 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { "setOwnerName", classOf[String]) - private lazy val getDatabaseOwnerTypeMethod = - findMethod( - classOf[Database], - "getOwnerType") - - private lazy val setDatabaseOwnerTypeMethod = - findMethod( - classOf[Database], - "setOwnerType", - classOf[PrincipalType]) - override def setCurrentSessionState(state: SessionState): Unit = setCurrentSessionStateMethod.invoke(null, state) @@ -855,15 +836,6 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { override def setDatabaseOwnerName(db: Database, owner: String): Unit = { setDatabaseOwnerNameMethod.invoke(db, owner) } - - override def getDatabaseOwnerType(db: Database): String = { - Option(getDatabaseOwnerTypeMethod.invoke(db)) - .map(_.asInstanceOf[PrincipalType].name()).getOrElse("") - } - - override def setDatabaseOwnerType(db: Database, ownerType: String): Unit = { - setDatabaseOwnerTypeMethod.invoke(db, PrincipalType.valueOf(ownerType)) - } } private[client] class Shim_v0_14 extends Shim_v0_13 { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index e07978ae180e4..4760af7aa46ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -177,25 +177,25 @@ class VersionsSuite extends SparkFunSuite with Logging { val ownerName = "SPARK_29425" val db1 = "SPARK_29425_1" val db2 = "SPARK_29425_2" - val ownerProps = Map("ownerName" -> ownerName) + val ownerProps = Map("owner" -> ownerName) // create database with owner val dbWithOwner = CatalogDatabase(db1, "desc", Utils.createTempDir().toURI, ownerProps) client.createDatabase(dbWithOwner, ignoreIfExists = true) val getDbWithOwner = client.getDatabase(db1) - assert(getDbWithOwner.properties("ownerName") === ownerName) + assert(getDbWithOwner.properties("owner") === ownerName) // alter database without owner client.alterDatabase(getDbWithOwner.copy(properties = Map())) - assert(client.getDatabase(db1).properties("ownerName") === "") + assert(client.getDatabase(db1).properties("owner") === "") // create database without owner val dbWithoutOwner = CatalogDatabase(db2, "desc", Utils.createTempDir().toURI, Map()) client.createDatabase(dbWithoutOwner, ignoreIfExists = true) val getDbWithoutOwner = client.getDatabase(db2) - assert(getDbWithoutOwner.properties("ownerName") === currentUser) + assert(getDbWithoutOwner.properties("owner") === currentUser) // alter database with owner client.alterDatabase(getDbWithoutOwner.copy(properties = ownerProps)) - assert(client.getDatabase(db2).properties("ownerName") === ownerName) + assert(client.getDatabase(db2).properties("owner") === ownerName) } } 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 59eadb844837e..ba48cfd4142f6 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 @@ -31,8 +31,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.connector.catalog.{CatalogManager, TableCatalog} -import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE} +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.HiveExternalCatalog @@ -375,45 +375,18 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA } } - private def checkOwner(db: String, expectedOwnerName: String, expectedOwnerType: String): Unit = { - val df = sql(s"DESCRIBE DATABASE EXTENDED $db") - val owner = df.where("database_description_item='Owner Name'") - .collect().head.getString(1) - val typ = df.where("database_description_item='Owner Type'") - .collect().head.getString(1) - assert(owner === expectedOwnerName) - assert(typ === expectedOwnerType) - } - test("Database Ownership") { val catalog = spark.sessionState.catalog try { - val db1 = "spark_29425_1" - val db2 = "spark_29425_2" - val owner = "spark_29425" - val currentUser = Utils.getCurrentUserName() - - sql(s"CREATE DATABASE $db1") - checkOwner(db1, currentUser, "USER") - sql(s"ALTER DATABASE $db1 SET DBPROPERTIES ('a'='a')") - checkOwner(db1, currentUser, "USER") - val e = intercept[ParseException](sql(s"ALTER DATABASE $db1 SET DBPROPERTIES ('a'='a'," - + s"'ownerName'='$owner','ownerType'='XXX')")) - assert(e.getMessage.contains("ownerName")) - sql(s"ALTER DATABASE $db1 SET OWNER ROLE $owner") - checkOwner(db1, owner, "ROLE") - - val e2 = intercept[ParseException]( - sql(s"CREATE DATABASE $db2 WITH DBPROPERTIES('ownerName'='$owner')")) - assert(e2.getMessage.contains("ownerName")) - sql(s"CREATE DATABASE $db2") - checkOwner(db2, currentUser, "USER") - sql(s"ALTER DATABASE $db2 SET OWNER GROUP $owner") - checkOwner(db2, owner, "GROUP") - sql(s"ALTER DATABASE $db2 SET OWNER GROUP `$owner`") - checkOwner(db2, owner, "GROUP") - sql(s"ALTER DATABASE $db2 SET OWNER GROUP OWNER") - checkOwner(db2, "OWNER", "GROUP") + val db = "spark_29425_1" + sql(s"CREATE DATABASE $db") + assert(sql(s"DESCRIBE DATABASE EXTENDED $db") + .where("database_description_item='Owner'") + .collect().head.getString(1) === Utils.getCurrentUserName()) + sql(s"ALTER DATABASE $db SET DBPROPERTIES('abc'='xyz')") + assert(sql(s"DESCRIBE DATABASE EXTENDED $db") + .where("database_description_item='Owner'") + .collect().head.getString(1) === Utils.getCurrentUserName()) } finally { catalog.reset() } @@ -1169,7 +1142,7 @@ class HiveDDLSuite sql(s"CREATE DATABASE $dbName Location '${tmpDir.toURI.getPath.stripSuffix("/")}'") val db1 = catalog.getDatabaseMetadata(dbName) val dbPath = new URI(tmpDir.toURI.toString.stripSuffix("/")) - assert(db1.copy(properties = db1.properties -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE)) === + assert(db1.copy(properties = db1.properties -- Seq(PROP_OWNER)) === CatalogDatabase(dbName, "", dbPath, Map.empty)) sql("USE db1") @@ -1208,7 +1181,7 @@ class HiveDDLSuite val expectedDBLocation = s"file:${dbPath.toUri.getPath.stripSuffix("/")}/$dbName.db" val expectedDBUri = CatalogUtils.stringToURI(expectedDBLocation) val db1 = catalog.getDatabaseMetadata(dbName) - assert(db1.copy(properties = db1.properties -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE)) == + assert(db1.copy(properties = db1.properties -- Seq(PROP_OWNER)) == CatalogDatabase( dbName, "", From b8cb52a8a7c247df0b8647917f79cbd929a72c3d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 22 Jan 2020 21:31:11 +0800 Subject: [PATCH 1183/1387] [SPARK-30555][SQL] MERGE INTO insert action should only access columns from source table ### What changes were proposed in this pull request? when resolving the `Assignment` of insert action in MERGE INTO, only resolve with the source table, to avoid ambiguous attribute failure if there is a same-name column in the target table. ### Why are the changes needed? The insert action is used when NOT MATCHED, so it can't access the row from the target table anyway. ### Does this PR introduce any user-facing change? on ### How was this patch tested? new tests Closes #27265 from cloud-fan/merge. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 27 +- .../command/PlanResolutionSuite.scala | 380 ++++++++++-------- 2 files changed, 229 insertions(+), 178 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index a96e04f043ac4..7e9f85b64e4a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1332,13 +1332,21 @@ class Analyzer( DeleteAction(resolvedDeleteCondition) case UpdateAction(updateCondition, assignments) => val resolvedUpdateCondition = updateCondition.map(resolveExpressionTopDown(_, m)) - UpdateAction(resolvedUpdateCondition, resolveAssignments(assignments, m)) + // The update value can access columns from both target and source tables. + UpdateAction( + resolvedUpdateCondition, + resolveAssignments(assignments, m, resolveValuesWithSourceOnly = false)) case o => o } val newNotMatchedActions = m.notMatchedActions.map { case InsertAction(insertCondition, assignments) => - val resolvedInsertCondition = insertCondition.map(resolveExpressionTopDown(_, m)) - InsertAction(resolvedInsertCondition, resolveAssignments(assignments, m)) + // The insert action is used when not matched, so its condition and value can only + // access columns from the source table. + val resolvedInsertCondition = + insertCondition.map(resolveExpressionTopDown(_, Project(Nil, m.sourceTable))) + InsertAction( + resolvedInsertCondition, + resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true)) case o => o } val resolvedMergeCondition = resolveExpressionTopDown(m.mergeCondition, m) @@ -1353,7 +1361,8 @@ class Analyzer( def resolveAssignments( assignments: Seq[Assignment], - mergeInto: MergeIntoTable): Seq[Assignment] = { + mergeInto: MergeIntoTable, + resolveValuesWithSourceOnly: Boolean): Seq[Assignment] = { if (assignments.isEmpty) { val expandedColumns = mergeInto.targetTable.output val expandedValues = mergeInto.sourceTable.output @@ -1361,12 +1370,18 @@ class Analyzer( } else { assignments.map { assign => val resolvedKey = assign.key match { - case c if !c.resolved => resolveExpressionTopDown(c, mergeInto.targetTable) + case c if !c.resolved => + resolveExpressionTopDown(c, Project(Nil, mergeInto.targetTable)) case o => o } val resolvedValue = assign.value match { // The update values may contain target and/or source references. - case c if !c.resolved => resolveExpressionTopDown(c, mergeInto) + case c if !c.resolved => + if (resolveValuesWithSourceOnly) { + resolveExpressionTopDown(c, Project(Nil, mergeInto.sourceTable)) + } else { + resolveExpressionTopDown(c, mergeInto) + } case o => o } Assignment(resolvedKey, resolvedValue) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index abc20049735cb..8c73b366fa857 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -26,9 +26,9 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolvedTable, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedTable} +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions.{EqualTo, InSubquery, IntegerLiteral, ListQuery, StringLiteral} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.InMemoryTableProvider @@ -45,13 +45,13 @@ class PlanResolutionSuite extends AnalysisTest { private val table: Table = { val t = mock(classOf[Table]) - when(t.schema()).thenReturn(new StructType().add("i", "int")) + when(t.schema()).thenReturn(new StructType().add("i", "int").add("s", "string")) t } private val v1Table: V1Table = { val t = mock(classOf[CatalogTable]) - when(t.schema).thenReturn(new StructType().add("i", "int")) + when(t.schema).thenReturn(new StructType().add("i", "int").add("s", "string")) when(t.tableType).thenReturn(CatalogTableType.MANAGED) V1Table(t) } @@ -128,12 +128,17 @@ class PlanResolutionSuite extends AnalysisTest { catalogManagerWithoutDefault } val analyzer = new Analyzer(catalogManager, conf) + // TODO: run the analyzer directly. val rules = Seq( CTESubstitution, analyzer.ResolveRelations, analyzer.ResolveTables, new ResolveCatalogs(catalogManager), - new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v"))) + new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), + analyzer.ResolveTables, + analyzer.ResolveReferences, + analyzer.ResolveSubqueryColumnAliases, + analyzer.ResolveReferences) rules.foldLeft(parsePlan(query)) { case (plan, rule) => rule.apply(plan) } @@ -1055,6 +1060,49 @@ class PlanResolutionSuite extends AnalysisTest { } test("MERGE INTO TABLE") { + def checkResolution( + target: LogicalPlan, + source: LogicalPlan, + mergeCondition: Expression, + deleteCondAttr: Option[AttributeReference], + updateCondAttr: Option[AttributeReference], + insertCondAttr: Option[AttributeReference], + updateAssigns: Seq[Assignment], + insertAssigns: Seq[Assignment], + starInUpdate: Boolean = false): Unit = { + val ti = target.output.find(_.name == "i").get.asInstanceOf[AttributeReference] + val ts = target.output.find(_.name == "s").get.asInstanceOf[AttributeReference] + val si = source.output.find(_.name == "i").get.asInstanceOf[AttributeReference] + val ss = source.output.find(_.name == "s").get.asInstanceOf[AttributeReference] + + mergeCondition match { + case EqualTo(l: AttributeReference, r: AttributeReference) => + assert(l.sameRef(ti) && r.sameRef(si)) + case other => fail("unexpected merge condition " + other) + } + + deleteCondAttr.foreach(a => assert(a.sameRef(ts))) + updateCondAttr.foreach(a => assert(a.sameRef(ts))) + insertCondAttr.foreach(a => assert(a.sameRef(ss))) + + if (starInUpdate) { + assert(updateAssigns.size == 2) + assert(updateAssigns(0).key.asInstanceOf[AttributeReference].sameRef(ti)) + assert(updateAssigns(0).value.asInstanceOf[AttributeReference].sameRef(si)) + assert(updateAssigns(1).key.asInstanceOf[AttributeReference].sameRef(ts)) + assert(updateAssigns(1).value.asInstanceOf[AttributeReference].sameRef(ss)) + } else { + assert(updateAssigns.size == 1) + assert(updateAssigns.head.key.asInstanceOf[AttributeReference].sameRef(ts)) + assert(updateAssigns.head.value.asInstanceOf[AttributeReference].sameRef(ss)) + } + assert(insertAssigns.size == 2) + assert(insertAssigns(0).key.asInstanceOf[AttributeReference].sameRef(ti)) + assert(insertAssigns(0).value.asInstanceOf[AttributeReference].sameRef(si)) + assert(insertAssigns(1).key.asInstanceOf[AttributeReference].sameRef(ts)) + assert(insertAssigns(1).value.asInstanceOf[AttributeReference].sameRef(ss)) + } + Seq(("v2Table", "v2Table1"), ("testcat.tab", "testcat.tab1")).foreach { case(target, source) => // basic @@ -1065,9 +1113,25 @@ class PlanResolutionSuite extends AnalysisTest { |ON target.i = source.i |WHEN MATCHED AND (target.s='delete') THEN DELETE |WHEN MATCHED AND (target.s='update') THEN UPDATE SET target.s = source.s - |WHEN NOT MATCHED AND (target.s='insert') + |WHEN NOT MATCHED AND (source.s='insert') | THEN INSERT (target.i, target.s) values (source.i, source.s) """.stripMargin + parseAndResolve(sql1) match { + case MergeIntoTable( + SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("source", None), source: DataSourceV2Relation), + mergeCondition, + Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), + UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), + updateAssigns)), + Seq(InsertAction(Some(EqualTo(il: AttributeReference, StringLiteral("insert"))), + insertAssigns))) => + checkResolution(target, source, mergeCondition, Some(dl), Some(ul), Some(il), + updateAssigns, insertAssigns) + + case other => fail("Expect MergeIntoTable, but got:\n" + other.treeString) + } + // star val sql2 = s""" @@ -1076,8 +1140,24 @@ class PlanResolutionSuite extends AnalysisTest { |ON target.i = source.i |WHEN MATCHED AND (target.s='delete') THEN DELETE |WHEN MATCHED AND (target.s='update') THEN UPDATE SET * - |WHEN NOT MATCHED AND (target.s='insert') THEN INSERT * + |WHEN NOT MATCHED AND (source.s='insert') THEN INSERT * """.stripMargin + parseAndResolve(sql2) match { + case MergeIntoTable( + SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("source", None), source: DataSourceV2Relation), + mergeCondition, + Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), + UpdateAction(Some(EqualTo(ul: AttributeReference, + StringLiteral("update"))), updateAssigns)), + Seq(InsertAction(Some(EqualTo(il: AttributeReference, StringLiteral("insert"))), + insertAssigns))) => + checkResolution(target, source, mergeCondition, Some(dl), Some(ul), Some(il), + updateAssigns, insertAssigns, starInUpdate = true) + + case other => fail("Expect MergeIntoTable, but got:\n" + other.treeString) + } + // no additional conditions val sql3 = s""" @@ -1088,6 +1168,19 @@ class PlanResolutionSuite extends AnalysisTest { |WHEN MATCHED THEN UPDATE SET target.s = source.s |WHEN NOT MATCHED THEN INSERT (target.i, target.s) values (source.i, source.s) """.stripMargin + parseAndResolve(sql3) match { + case MergeIntoTable( + SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("source", None), source: DataSourceV2Relation), + mergeCondition, + Seq(DeleteAction(None), UpdateAction(None, updateAssigns)), + Seq(InsertAction(None, insertAssigns))) => + checkResolution(target, source, mergeCondition, None, None, None, + updateAssigns, insertAssigns) + + case other => fail("Expect MergeIntoTable, but got:\n" + other.treeString) + } + // using subquery val sql4 = s""" @@ -1096,9 +1189,25 @@ class PlanResolutionSuite extends AnalysisTest { |ON target.i = source.i |WHEN MATCHED AND (target.s='delete') THEN DELETE |WHEN MATCHED AND (target.s='update') THEN UPDATE SET target.s = source.s - |WHEN NOT MATCHED AND (target.s='insert') + |WHEN NOT MATCHED AND (source.s='insert') | THEN INSERT (target.i, target.s) values (source.i, source.s) """.stripMargin + parseAndResolve(sql4) match { + case MergeIntoTable( + SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("source", None), source: Project), + mergeCondition, + Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), + UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), + updateAssigns)), + Seq(InsertAction(Some(EqualTo(il: AttributeReference, StringLiteral("insert"))), + insertAssigns))) => + checkResolution(target, source, mergeCondition, Some(dl), Some(ul), Some(il), + updateAssigns, insertAssigns) + + case other => fail("Expect MergeIntoTable, but got:\n" + other.treeString) + } + // cte val sql5 = s""" @@ -1109,142 +1218,24 @@ class PlanResolutionSuite extends AnalysisTest { |ON target.i = source.i |WHEN MATCHED AND (target.s='delete') THEN DELETE |WHEN MATCHED AND (target.s='update') THEN UPDATE SET target.s = source.s - |WHEN NOT MATCHED AND (target.s='insert') + |WHEN NOT MATCHED AND (source.s='insert') |THEN INSERT (target.i, target.s) values (source.i, source.s) """.stripMargin - - val parsed1 = parseAndResolve(sql1) - val parsed2 = parseAndResolve(sql2) - val parsed3 = parseAndResolve(sql3) - val parsed4 = parseAndResolve(sql4) - val parsed5 = parseAndResolve(sql5) - - parsed1 match { + parseAndResolve(sql5) match { case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), _: DataSourceV2Relation), - EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), - Seq(DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), - UpdateAction(Some(EqualTo(ul: UnresolvedAttribute, StringLiteral("update"))), + SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("source", None), source: Project), + mergeCondition, + Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), + UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), updateAssigns)), - Seq(InsertAction(Some(EqualTo(il: UnresolvedAttribute, StringLiteral("insert"))), + Seq(InsertAction(Some(EqualTo(il: AttributeReference, StringLiteral("insert"))), insertAssigns))) => - assert(l.name == "target.i" && r.name == "source.i") - assert(dl.name == "target.s") - assert(ul.name == "target.s") - assert(il.name == "target.s") - assert(updateAssigns.size == 1) - assert(updateAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.s") - assert(updateAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.s") - assert(insertAssigns.size == 2) - assert(insertAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.i") - assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed1.treeString) - } + assert(source.output.map(_.name) == Seq("i", "s")) + checkResolution(target, source, mergeCondition, Some(dl), Some(ul), Some(il), + updateAssigns, insertAssigns) - parsed2 match { - case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), _: DataSourceV2Relation), - EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), - Seq(DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), - UpdateAction(Some(EqualTo(ul: UnresolvedAttribute, - StringLiteral("update"))), Seq())), - Seq(InsertAction(Some(EqualTo(il: UnresolvedAttribute, StringLiteral("insert"))), - Seq()))) => - assert(l.name == "target.i" && r.name == "source.i") - assert(dl.name == "target.s") - assert(ul.name == "target.s") - assert(il.name == "target.s") - - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed2.treeString) - } - - parsed3 match { - case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), _: DataSourceV2Relation), - EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), - Seq(DeleteAction(None), UpdateAction(None, updateAssigns)), - Seq(InsertAction(None, insertAssigns))) => - assert(l.name == "target.i" && r.name == "source.i") - assert(updateAssigns.size == 1) - assert(updateAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.s") - assert(updateAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.s") - assert(insertAssigns.size == 2) - assert(insertAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.i") - assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed3.treeString) - } - - parsed4 match { - case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), _: Project), - EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), - Seq(DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), - UpdateAction(Some(EqualTo(ul: UnresolvedAttribute, StringLiteral("update"))), - updateAssigns)), - Seq(InsertAction(Some(EqualTo(il: UnresolvedAttribute, StringLiteral("insert"))), - insertAssigns))) => - assert(l.name == "target.i" && r.name == "source.i") - assert(dl.name == "target.s") - assert(ul.name == "target.s") - assert(il.name == "target.s") - assert(updateAssigns.size == 1) - assert(updateAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.s") - assert(updateAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.s") - assert(insertAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.i") - assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed4.treeString) - } - - parsed5 match { - case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), - UnresolvedSubqueryColumnAliases(outputColumnNames, - Project(projects, _: DataSourceV2Relation))), - EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), - Seq(DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), - UpdateAction(Some(EqualTo(ul: UnresolvedAttribute, StringLiteral("update"))), - updateAssigns)), - Seq(InsertAction(Some(EqualTo(il: UnresolvedAttribute, StringLiteral("insert"))), - insertAssigns))) => - assert(outputColumnNames.size == 2 && - outputColumnNames.head == "i" && - outputColumnNames.last == "s") - assert(projects.size == 1 && projects.head.isInstanceOf[UnresolvedStar]) - assert(l.name == "target.i" && r.name == "source.i") - assert(dl.name == "target.s") - assert(ul.name == "target.s") - assert(il.name == "target.s") - assert(updateAssigns.size == 1) - assert(updateAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.s") - assert(updateAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.s") - assert(insertAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.i") - assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") - - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed5.treeString) + case other => fail("Expect MergeIntoTable, but got:\n" + other.treeString) } } @@ -1255,49 +1246,94 @@ class PlanResolutionSuite extends AnalysisTest { val target = pair._1 val source = pair._2 - val sql = + val sql1 = s""" |MERGE INTO $target |USING $source - |ON $target.i = $source.i - |WHEN MATCHED AND ($target.s='delete') THEN DELETE - |WHEN MATCHED AND ($target.s='update') THEN UPDATE SET $target.s = $source.s - |WHEN NOT MATCHED AND ($target.s='insert') - |THEN INSERT ($target.i, $target.s) values ($source.i, $source.s) + |ON 1 = 1 + |WHEN MATCHED THEN DELETE + |WHEN MATCHED THEN UPDATE SET s = 1 + |WHEN NOT MATCHED AND (s = 'a') THEN INSERT (i) values (i) """.stripMargin - val parsed = parseAndResolve(sql) - - parsed match { + parseAndResolve(sql1) match { case MergeIntoTable( - _: DataSourceV2Relation, - _: DataSourceV2Relation, - EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), - Seq(DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), - UpdateAction(Some(EqualTo(ul: UnresolvedAttribute, StringLiteral("update"))), - updateAssigns)), - Seq(InsertAction(Some(EqualTo(il: UnresolvedAttribute, StringLiteral("insert"))), - insertAssigns))) => - assert(l.name == s"$target.i" && r.name == s"$source.i") - assert(dl.name == s"$target.s") - assert(ul.name == s"$target.s") - assert(il.name == s"$target.s") + target: DataSourceV2Relation, + source: DataSourceV2Relation, + _, + Seq(DeleteAction(None), UpdateAction(None, updateAssigns)), + Seq(InsertAction( + Some(EqualTo(il: AttributeReference, StringLiteral("a"))), + insertAssigns))) => + val ti = target.output.find(_.name == "i").get + val ts = target.output.find(_.name == "s").get + val si = source.output.find(_.name == "i").get + val ss = source.output.find(_.name == "s").get + + // INSERT condition is resolved with source table only, so column `s` is not ambiguous. + assert(il.sameRef(ss)) assert(updateAssigns.size == 1) - assert(updateAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == s"$target.s") - assert(updateAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - updateAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == s"$source.s") - assert(insertAssigns.size == 2) - assert(insertAssigns.head.key.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == s"$target.i") - assert(insertAssigns.head.value.isInstanceOf[UnresolvedAttribute] && - insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == s"$source.i") - - case _ => fail("Expect MergeIntoTable, but got:\n" + parsed.treeString) + // UPDATE key is resolved with target table only, so column `s` is not ambiguous. + assert(updateAssigns.head.key.asInstanceOf[AttributeReference].sameRef(ts)) + assert(insertAssigns.size == 1) + // INSERT key is resolved with target table only, so column `i` is not ambiguous. + assert(insertAssigns.head.key.asInstanceOf[AttributeReference].sameRef(ti)) + // INSERT value is resolved with source table only, so column `i` is not ambiguous. + assert(insertAssigns.head.value.asInstanceOf[AttributeReference].sameRef(si)) + + case p => fail("Expect MergeIntoTable, but got:\n" + p.treeString) } + + val sql2 = + s""" + |MERGE INTO $target + |USING $source + |ON i = 1 + |WHEN MATCHED THEN DELETE + """.stripMargin + // merge condition is resolved with both target and source tables, and we can't + // resolve column `i` as it's ambiguous. + val e2 = intercept[AnalysisException](parseAndResolve(sql2)) + assert(e2.message.contains("Reference 'i' is ambiguous")) + + val sql3 = + s""" + |MERGE INTO $target + |USING $source + |ON 1 = 1 + |WHEN MATCHED AND (s='delete') THEN DELETE + """.stripMargin + // delete condition is resolved with both target and source tables, and we can't + // resolve column `s` as it's ambiguous. + val e3 = intercept[AnalysisException](parseAndResolve(sql3)) + assert(e3.message.contains("Reference 's' is ambiguous")) + + val sql4 = + s""" + |MERGE INTO $target + |USING $source + |ON 1 = 1 + |WHEN MATCHED AND (s = 'a') THEN UPDATE SET i = 1 + """.stripMargin + // update condition is resolved with both target and source tables, and we can't + // resolve column `s` as it's ambiguous. + val e4 = intercept[AnalysisException](parseAndResolve(sql4)) + assert(e4.message.contains("Reference 's' is ambiguous")) + + val sql5 = + s""" + |MERGE INTO $target + |USING $source + |ON 1 = 1 + |WHEN MATCHED THEN UPDATE SET s = s + """.stripMargin + // update value is resolved with both target and source tables, and we can't + // resolve column `s` as it's ambiguous. + val e5 = intercept[AnalysisException](parseAndResolve(sql5)) + assert(e5.message.contains("Reference 's' is ambiguous")) } - val sql = + val sql6 = s""" |MERGE INTO non_exist_target |USING non_exist_source @@ -1306,7 +1342,7 @@ class PlanResolutionSuite extends AnalysisTest { |WHEN MATCHED THEN UPDATE SET * |WHEN NOT MATCHED THEN INSERT * """.stripMargin - val parsed = parseAndResolve(sql) + val parsed = parseAndResolve(sql6) parsed match { case u: MergeIntoTable => assert(u.targetTable.isInstanceOf[UnresolvedRelation]) From 1c46bd9e6072cdd1656288e949374440a5c9907c Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Wed, 22 Jan 2020 08:24:11 -0600 Subject: [PATCH 1184/1387] [SPARK-30503][ML] OnlineLDAOptimizer does not handle persistance correctly ### What changes were proposed in this pull request? unpersist graph outside checkpointer, like what Pregel does ### Why are the changes needed? Shown in [SPARK-30503](https://issues.apache.org/jira/browse/SPARK-30503), intermediate edges are not unpersisted ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing testsuites and manual test Closes #27261 from zhengruifeng/lda_checkpointer. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../org/apache/spark/mllib/clustering/LDAOptimizer.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index 5eea69022562b..dc90f6c3e3885 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -142,7 +142,7 @@ final class EMLDAOptimizer extends LDAOptimizer { // For each document, create an edge (Document -> Term) for each unique term in the document. val edges: RDD[Edge[TokenCount]] = docs.flatMap { case (docID: Long, termCounts: Vector) => // Add edges for terms with non-zero counts. - termCounts.asBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => + termCounts.nonZeroIterator.map { case (term, cnt) => Edge(docID, term2index(term), cnt) } } @@ -211,11 +211,14 @@ final class EMLDAOptimizer extends LDAOptimizer { val docTopicDistributions: VertexRDD[TopicCounts] = graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) .mapValues(_._2) + val prevGraph = graph // Update the vertex descriptors with the new counts. val newGraph = Graph(docTopicDistributions, graph.edges) graph = newGraph graphCheckpointer.update(newGraph) globalTopicTotals = computeGlobalTopicTotals() + prevGraph.unpersistVertices() + prevGraph.edges.unpersist() this } From 8097b7eaf3e66a956b19207cbecca3776c233767 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Wed, 22 Jan 2020 08:41:31 -0600 Subject: [PATCH 1185/1387] [SPARK-30573][DOC] Document WHERE Clause of SELECT statement in SQL Reference ### What changes were proposed in this pull request? Document WHERE Clause of SELECT statement in SQL Reference Guide. I ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After** Screen Shot 2020-01-19 at 5 03 49 PM Screen Shot 2020-01-19 at 5 04 07 PM Screen Shot 2020-01-19 at 5 04 23 PM ### How was this patch tested? Tested using jykyll build --serve Closes #27282 from dilipbiswal/sql-ref-select-where. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/sql-ref-syntax-qry-select-where.md | 113 ++++++++++++++++++++++++ 1 file changed, 113 insertions(+) create mode 100644 docs/sql-ref-syntax-qry-select-where.md diff --git a/docs/sql-ref-syntax-qry-select-where.md b/docs/sql-ref-syntax-qry-select-where.md new file mode 100644 index 0000000000000..2c8d449172eed --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-where.md @@ -0,0 +1,113 @@ +--- +layout: global +title: SELECT +displayTitle: WHERE clause +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- +The WHERE clause is used to limit the results of the FROM +clause of query or subquery based on the specified condition. + +### Syntax +{% highlight sql %} +WHERE boolean_expression +{% endhighlight %} + +### Parameters +
      +
      boolean_expression
      +
      + Specifies any expression that evaluates to a result type boolean. Two or + more expressions may be combined together using the logical + operators ( AND, OR ). +
      +
      + +### Examples +{% highlight sql %} +CREATE TABLE person (id INT, name STRING, age INT); +INSERT INTO person VALUES (100, 'John', 30), + (200, 'Mary', NULL), + (300, 'Mike', 80), + (400, 'Dan', 50); + +-- Comparison operator in `WHERE` clause. +SELECT * FROM person WHERE id > 200 ORDER BY id; + +---+----+---+ + |id |name|age| + +---+----+---+ + |300|Mike|80 | + |400|Dan |50 | + +---+----+---+ + +-- Comparison and logical operators in `WHERE` clause. +SELECT * FROM person WHERE id = 200 OR id = 300 ORDER BY id; + +---+----+----+ + |id |name|age | + +---+----+----+ + |200|Mary|null| + |300|Mike|80 | + +---+----+----+ + +-- ISNULL expression in `WHERE` clause. +SELECT * FROM person WHERE id > 300 OR age IS NULL ORDER BY id; + +---+----+----+ + |id |name|age | + +---+----+----+ + |200|Mary|null| + |400|Dan |50 | + +---+----+----+ + +-- Function expression in `WHERE` clause. +SELECT * FROM person WHERE length(name) > 3 ORDER BY id; + +---+----+----+ + |id |name|age | + +---+----+----+ + |100|John|30 | + |200|Mary|null| + |300|Mike|80 | + +---+----+----+ + +-- `BETWEEN` expression in `WHERE` clause. +SELECT * FROM person WHERE id BETWEEN 200 AND 300 ORDER BY id; + +---+----+----+ + | id|name| age| + +---+----+----+ + |200|Mary|null| + |300|Mike| 80| + +---+----+----+ + +-- Scalar Subquery in `WHERE` clause. +SELECT * FROM person WHERE age > (SELECT avg(age) FROM person); + +---+----+---+ + |id |name|age| + +---+----+---+ + |300|Mike|80 | + +---+----+---+ + +-- Correlated column reference in `WHERE` clause of subquery. +SELECT * FROM person AS parent +WHERE EXISTS ( + SELECT 1 FROM person AS child + WHERE parent.id = child.id AND child.age IS NULL + ); + +---+----+----+ + |id |name|age | + +---+----+----+ + |200|Mary|null| + +---+----+----+ + +{% endhighlight %} From a6030eff3059033155cff4ed9dac6958d186426c Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Wed, 22 Jan 2020 08:45:03 -0600 Subject: [PATCH 1186/1387] [SPARK-30575][DOC] Document HAVING Clause of SELECT statement in SQL Reference ### What changes were proposed in this pull request? Document HAVING clause of SELECT statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2020-01-19 at 6 03 52 PM Screen Shot 2020-01-19 at 6 04 11 PM Screen Shot 2020-01-19 at 6 04 28 PM ### How was this patch tested? Tested using jykyll build --serve Closes #27284 from dilipbiswal/sql-ref-select-having. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/sql-ref-syntax-qry-select-having.md | 99 +++++++++++++++++++++++- 1 file changed, 98 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-qry-select-having.md b/docs/sql-ref-syntax-qry-select-having.md index ca92eb0d4daf0..718c22fd4b114 100644 --- a/docs/sql-ref-syntax-qry-select-having.md +++ b/docs/sql-ref-syntax-qry-select-having.md @@ -18,5 +18,102 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- +The HAVING clause is used to filter the results produced by +GROUP BY based on the specified condition. It is often used +in the conjunction with a [GROUP BY](sql-ref-syntax-qry-select-groupby.html) +clause. -**This page is under construction** +### Syntax +{% highlight sql %} +HAVING boolean_expression +{% endhighlight %} + +### Parameters +
      +
      boolean_expression
      +
      + Specifies any expression that evaluates to a result type boolean. Two or + more expressions may be combined together using the logical + operators ( AND, OR ).

      + + Note
      + The expressions specified in the HAVING clause can only refer to: +
        +
      1. Constants
      2. +
      3. Expressions that appear in GROUP BY
      4. +
      5. Aggregate functions
      6. +
      +
      +
      + +### Examples +{% highlight sql %} +CREATE TABLE dealer (id INT, city STRING, car_model STRING, quantity INT); +INSERT INTO dealer VALUES (100, 'Fremont', 'Honda Civic', 10), + (100, 'Fremont', 'Honda Accord', 15), + (100, 'Fremont', 'Honda CRV', 7), + (200, 'Dublin', 'Honda Civic', 20), + (200, 'Dublin', 'Honda Accord', 10), + (200, 'Dublin', 'Honda CRV', 3), + (300, 'San Jose', 'Honda Civic', 5), + (300, 'San Jose', 'Honda Accord', 8); + +-- `HAVING` clause referring to column in `GROUP BY`. +SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING city = 'Fremont'; + + +-------+---+ + |city |sum| + +-------+---+ + |Fremont|32 | + +-------+---+ + +-- `HAVING` clause referring to aggregate function. +SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING sum(quantity) > 15; + + +-------+---+ + | city|sum| + +-------+---+ + | Dublin| 33| + |Fremont| 32| + +-------+---+ + +-- `HAVING` clause referring to aggregate function by its alias. +SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING sum > 15; + + +-------+---+ + | city|sum| + +-------+---+ + | Dublin| 33| + |Fremont| 32| + +-------+---+ + +-- `HAVING` clause referring to a different aggregate function than what is present in +-- `SELECT` list. +SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING max(quantity) > 15; + + +------+---+ + |city |sum| + +------+---+ + |Dublin|33 | + +------+---+ + +-- `HAVING` clause referring to constant expression. +SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING 1 > 0 ORDER BY city; + + +--------+---+ + | city|sum| + +--------+---+ + | Dublin| 33| + | Fremont| 32| + |San Jose| 13| + +--------+---+ + +-- `HAVING` clause without a `GROUP BY` clause. +SELECT sum(quantity) AS sum FROM dealer HAVING sum(quantity) > 10; + +---+ + |sum| + +---+ + | 78| + +---+ + +{% endhighlight %} From 8f7f4d57958b9896bfacce81a80e1285f9461340 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Wed, 22 Jan 2020 08:59:34 -0600 Subject: [PATCH 1187/1387] [SPARK-30583][DOC] Document LIMIT Clause of SELECT statement in SQL Reference ### What changes were proposed in this pull request? Document LIMIT clause of SELECT statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2020-01-20 at 1 37 28 AM Screen Shot 2020-01-20 at 1 37 43 AM ### How was this patch tested? Tested using jykyll build --serve Closes #27290 from dilipbiswal/sql-ref-select-limit. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/sql-ref-syntax-qry-select-limit.md | 72 +++++++++++++++++++++++-- 1 file changed, 69 insertions(+), 3 deletions(-) diff --git a/docs/sql-ref-syntax-qry-select-limit.md b/docs/sql-ref-syntax-qry-select-limit.md index d7fac3bb98929..609bfb98a097a 100644 --- a/docs/sql-ref-syntax-qry-select-limit.md +++ b/docs/sql-ref-syntax-qry-select-limit.md @@ -1,7 +1,7 @@ --- layout: global -title: LIMIT operator -displayTitle: LIMIT operator +title: LIMIT Clause +displayTitle: LIMIT Clause license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -18,5 +18,71 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- +The LIMIT clause is used to constrain the number of rows returned by the SELECT statement. +In general, this clause is used in conjuction with ORDER BY to ensure that the results are deterministic. -**This page is under construction** +### Syntax +{% highlight sql %} +LIMIT { ALL | integer_expression } +{% endhighlight %} + +### Parameters +
      +
      ALL
      +
      + If specified, the query returns all the rows. In other words, no limit is applied if this + option is specified. +
      +
      integer_expression
      +
      + Specifies an expression that returns an integer. +
      +
      + +### Examples +{% highlight sql %} +CREATE TABLE person (name STRING, age INT); +INSERT INTO person VALUES ('Zen Hui', 25), + ('Anil B', 18), + ('Shone S', 16), + ('Mike A', 25), + ('John A', 18), + ('Jack N', 16); + +-- Select the first two rows. +SELECT name, age FROM person ORDER BY name LIMIT 2; + + +------+---+ + |name |age| + +------+---+ + |Anil B|18 | + |Jack N|16 | + +------+---+ + +-- Specifying ALL option on LIMIT returns all the rows. +SELECT name, age FROM person ORDER BY name LIMIT ALL; + + +-------+---+ + |name |age| + +-------+---+ + |Anil B |18 | + |Jack N |16 | + |John A |18 | + |Mike A |25 | + |Shone S|16 | + |Zen Hui|25 | + +-------+---+ + +-- A function expression as an input to limit. +SELECT name, age FROM person ORDER BY name LIMIT length('SPARK') + + +-------+---+ + | name|age| + +-------+---+ + | Anil B| 18| + | Jack N| 16| + | John A| 18| + | Mike A| 25| + |Shone S| 16| + +-------+---+ +{% endhighlight %} From 8e280cebf25e47bf40df224461a76fc4c84cc997 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 23 Jan 2020 00:41:46 +0800 Subject: [PATCH 1188/1387] [SPARK-30592][SQL] Interval support for csv and json funtions ### What changes were proposed in this pull request? In this PR, I'd propose to fully support interval for the CSV and JSON functions. On one hand, CSV and JSON records consist of string values, in the cast logic, we can cast string from/to interval now, so we can make those functions support intervals easily. Before this change we can only use this as a workaround. ```sql SELECT cast(from_csv('1, 1 day', 'a INT, b string').b as interval) struct 1 days ``` On the other hand, we ban reading or writing intervals from CSV and JSON files. To directly read and write with external json/csv storage, you still need explicit cast, e.g. ```scala spark.read.schema("a string").json("a.json").selectExpr("cast(a as interval)").show +------+ | a| +------+ |1 days| +------+ ``` ### Why are the changes needed? for interval's future-proofing purpose ### Does this PR introduce any user-facing change? yes, the `to_json`/`from_json` function can deal with intervals now. e.g. for `from_json` there is no such use case because we do not support `a interval` for `to_json`, we can use interval values now #### before ```sql SELECT to_json(map('a', interval 25 month 100 day 130 minute)); Error in query: cannot resolve 'to_json(map('a', INTERVAL '2 years 1 months 100 days 2 hours 10 minutes'))' due to data type mismatch: Unable to convert column a of type interval to JSON.; line 1 pos 7; 'Project [unresolvedalias(to_json(map(a, 2 years 1 months 100 days 2 hours 10 minutes), Some(Asia/Shanghai)), None)] +- OneRowRelation ``` #### after ```sql SELECT to_json(map('a', interval 25 month 100 day 130 minute)) {"a":"2 years 1 months 100 days 2 hours 10 minutes"} ``` ### How was this patch tested? add ut Closes #27317 from yaooqinn/SPARK-30592. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/sql/Row.scala | 8 +++-- .../sql/catalyst/csv/UnivocityParser.scala | 5 +++ .../expressions/jsonExpressions.scala | 4 +-- .../sql/catalyst/json/JacksonGenerator.scala | 4 +++ .../sql/catalyst/json/JacksonParser.scala | 8 ++++- .../sql/catalyst/json/JacksonUtils.scala | 3 +- .../sql/types/CalendarIntervalType.scala | 2 +- .../expressions/CsvExpressionsSuite.scala | 12 ++++++- .../expressions/JsonExpressionsSuite.scala | 31 ++++++----------- .../resources/sql-tests/inputs/interval.sql | 6 ++++ .../sql-tests/results/ansi/interval.sql.out | 34 ++++++++++++++++++- .../sql-tests/results/interval.sql.out | 34 ++++++++++++++++++- .../apache/spark/sql/JsonFunctionsSuite.scala | 33 +++++++----------- 13 files changed, 130 insertions(+), 54 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 3f9d07520e05f..9a7e077b658df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} import java.time.{Instant, LocalDate} -import java.util.{Base64, TimeZone} +import java.util.Base64 import scala.collection.JavaConverters._ import scala.collection.mutable @@ -29,12 +29,13 @@ import org.json4s._ import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.{Private, Stable, Unstable} +import org.apache.spark.annotation.{Stable, Unstable} import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType, Decimal, MapType, StringType, StructType, UserDefinedType} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval /** * @since 1.3.0 @@ -572,6 +573,7 @@ trait Row extends Serializable { JString(timestampFormatter.format(DateTimeUtils.instantToMicros(i))) case (t: Timestamp, _) => JString(timestampFormatter.format(DateTimeUtils.fromJavaTimestamp(t))) + case (i: CalendarInterval, _) => JString(i.toString) case (a: Array[_], ArrayType(elementType, _)) => iteratorToJsonArray(a.iterator, elementType) case (s: Seq[_], ArrayType(elementType, _)) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 61b1b437da04c..5510953804025 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -180,6 +180,11 @@ class UnivocityParser( case _: StringType => (d: String) => nullSafeDatum(d, name, nullable, options)(UTF8String.fromString) + case CalendarIntervalType => (d: String) => + nullSafeDatum(d, name, nullable, options) { datum => + IntervalUtils.safeStringToInterval(UTF8String.fromString(datum)) + } + case udt: UserDefinedType[_] => makeConverter(name, udt.sqlType, nullable) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 43b01eb0d4e1c..61afdb6c9492f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -694,10 +694,8 @@ case class StructsToJson( TypeCheckResult.TypeCheckFailure(e.getMessage) } case map: MapType => - // TODO: let `JacksonUtils.verifySchema` verify a `MapType` try { - val st = StructType(StructField("a", map) :: Nil) - JacksonUtils.verifySchema(st) + JacksonUtils.verifyType(prettyName, map) TypeCheckResult.TypeCheckSuccess } catch { case e: UnsupportedOperationException => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index aaf2ecf7923ce..9c63593ea1752 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -133,6 +133,10 @@ private[sql] class JacksonGenerator( val dateString = dateFormatter.format(row.getInt(ordinal)) gen.writeString(dateString) + case CalendarIntervalType => + (row: SpecializedGetters, ordinal: Int) => + gen.writeString(row.getInterval(ordinal).toString) + case BinaryType => (row: SpecializedGetters, ordinal: Int) => gen.writeBinary(row.getBinary(ordinal)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 4824b0c860cb4..c44025ca8bcfd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.Utils /** @@ -252,6 +252,12 @@ class JacksonParser( Decimal(bigDecimal, dt.precision, dt.scale) } + case CalendarIntervalType => (parser: JsonParser) => + parseJsonToken[CalendarInterval](parser, dataType) { + case VALUE_STRING => + IntervalUtils.safeStringToInterval(UTF8String.fromString(parser.getText)) + } + case st: StructType => val fieldConverters = st.map(_.dataType).map(makeConverter).toArray (parser: JsonParser) => parseJsonToken[InternalRow](parser, dataType) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala index 2d89c7066d080..386c7e3f7541c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala @@ -34,8 +34,7 @@ object JacksonUtils { def verifyType(name: String, dataType: DataType): Unit = { dataType match { - case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | - DoubleType | StringType | TimestampType | DateType | BinaryType | _: DecimalType => + case NullType | _: AtomicType | CalendarIntervalType => case st: StructType => st.foreach(field => verifyType(field.name, field.dataType)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala index 5889f1ce4e1df..35ad864db0e7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala @@ -37,7 +37,7 @@ class CalendarIntervalType private() extends DataType { override def defaultSize: Int = 16 - override def simpleString: String = "interval" + override def typeName: String = "interval" private[spark] override def asNullable: CalendarIntervalType = this } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala index 98c93a4946f4f..e623910e2efe1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with PlanTestBase { val badCsv = "\u0000\u0000\u0000A\u0001AAA" @@ -237,4 +237,14 @@ class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with P timeZoneId = gmtId), expectedErrMsg = "The field for corrupt records must be string type and nullable") } + + test("from/to csv with intervals") { + val schema = new StructType().add("a", "interval") + checkEvaluation( + StructsToCsv(Map.empty, Literal.create(create_row(new CalendarInterval(1, 2, 3)), schema)), + "1 months 2 days 0.000003 seconds") + checkEvaluation( + CsvToStructs(schema, Map.empty, Literal.create("1 day")), + InternalRow(new CalendarInterval(0, 1, 0))) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index d5cc1d4f0fdde..3693531f47610 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -25,12 +25,10 @@ import org.scalatest.exceptions.TestFailedException import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with PlanTestBase { val json = @@ -680,25 +678,18 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with output) } - test("to_json: verify MapType's value type instead of key type") { - // Keys in map are treated as strings when converting to JSON. The type doesn't matter at all. - val mapType1 = MapType(CalendarIntervalType, IntegerType) - val schema1 = StructType(StructField("a", mapType1) :: Nil) - val struct1 = Literal.create(null, schema1) + test("from/to json - interval support") { + val schema = StructType(StructField("i", CalendarIntervalType) :: Nil) checkEvaluation( - StructsToJson(Map.empty, struct1, gmtId), - null - ) + JsonToStructs(schema, Map.empty, Literal.create("""{"i":"1 year 1 day"}""", StringType)), + InternalRow(new CalendarInterval(12, 1, 0))) - // The value type must be valid for converting to JSON. - val mapType2 = MapType(IntegerType, CalendarIntervalType) - val schema2 = StructType(StructField("a", mapType2) :: Nil) - val struct2 = Literal.create(null, schema2) - StructsToJson(Map.empty, struct2, gmtId).checkInputDataTypes() match { - case TypeCheckResult.TypeCheckFailure(msg) => - assert(msg.contains("Unable to convert column a of type interval to JSON")) - case _ => fail("from_json should not work on interval map value type.") - } + Seq(MapType(CalendarIntervalType, IntegerType), MapType(IntegerType, CalendarIntervalType)) + .foreach { dt => + val schema = StructField("a", dt) :: Nil + val struct = Literal.create(null, StructType(schema)) + assert(StructsToJson(Map.empty, struct).checkInputDataTypes().isSuccess) + } } test("from_json missing fields") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index 5de5656cbec56..fb6c485f619ae 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -220,3 +220,9 @@ select a - b from values (interval '-2147483648 months', interval '2147483647 mo select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); + +-- interval support for csv and json functions +SELECT from_csv('1, 1 day', 'a INT, b interval'); +SELECT to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)); +SELECT from_json('{"a":"1 days"}', 'a interval'); +SELECT to_json(map('a', interval 25 month 100 day 130 minute)); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 2893b6a01d1eb..64107f98ede6c 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 97 +-- Number of queries: 101 -- !query 0 @@ -985,3 +985,35 @@ struct<> -- !query 96 output java.lang.ArithmeticException integer overflow + + +-- !query 97 +SELECT from_csv('1, 1 day', 'a INT, b interval') +-- !query 97 schema +struct> +-- !query 97 output +{"a":1,"b":1 days} + + +-- !query 98 +SELECT to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)) +-- !query 98 schema +struct +-- !query 98 output +2 years 8 months,1 hours 10 minutes + + +-- !query 99 +SELECT from_json('{"a":"1 days"}', 'a interval') +-- !query 99 schema +struct> +-- !query 99 output +{"a":1 days} + + +-- !query 100 +SELECT to_json(map('a', interval 25 month 100 day 130 minute)) +-- !query 100 schema +struct +-- !query 100 output +{"a":"2 years 1 months 100 days 2 hours 10 minutes"} diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 633ee6fccc98f..d494399cedddc 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 97 +-- Number of queries: 101 -- !query 0 @@ -966,3 +966,35 @@ struct<> -- !query 96 output java.lang.ArithmeticException integer overflow + + +-- !query 97 +SELECT from_csv('1, 1 day', 'a INT, b interval') +-- !query 97 schema +struct> +-- !query 97 output +{"a":1,"b":1 days} + + +-- !query 98 +SELECT to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)) +-- !query 98 schema +struct +-- !query 98 output +2 years 8 months,1 hours 10 minutes + + +-- !query 99 +SELECT from_json('{"a":"1 days"}', 'a interval') +-- !query 99 schema +struct> +-- !query 99 output +{"a":1 days} + + +-- !query 100 +SELECT to_json(map('a', interval 25 month 100 day 130 minute)) +-- !query 100 schema +struct +-- !query 100 output +{"a":"2 years 1 months 100 days 2 hours 10 minutes"} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 6f55676f318d5..fd1e9e309558e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -222,33 +222,24 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { Row("""{"_1":"26/08/2015 18:00"}""") :: Nil) } - test("to_json - key types of map don't matter") { - // interval type is invalid for converting to JSON. However, the keys of a map are treated - // as strings, so its type doesn't matter. - val df = Seq(Tuple1(Tuple1("-3 month 7 hours"))).toDF("a") - .select(struct(map($"a._1".cast(CalendarIntervalType), lit("a")).as("col1")).as("c")) + test("to_json - interval support") { + val baseDf = Seq(Tuple1(Tuple1("-3 month 7 hours"))).toDF("a") + val df = baseDf.select(struct($"a._1".cast(CalendarIntervalType).as("a")).as("c")) checkAnswer( df.select(to_json($"c")), - Row("""{"col1":{"-3 months 7 hours":"a"}}""") :: Nil) - } + Row("""{"a":"-3 months 7 hours"}""") :: Nil) - test("to_json unsupported type") { - val baseDf = Seq(Tuple1(Tuple1("-3 month 7 hours"))).toDF("a") - val df = baseDf.select(struct($"a._1".cast(CalendarIntervalType).as("a")).as("c")) - val e = intercept[AnalysisException]{ - // Unsupported type throws an exception - df.select(to_json($"c")).collect() - } - assert(e.getMessage.contains( - "Unable to convert column a of type interval to JSON.")) + val df1 = baseDf + .select(struct(map($"a._1".cast(CalendarIntervalType), lit("a")).as("col1")).as("c")) + checkAnswer( + df1.select(to_json($"c")), + Row("""{"col1":{"-3 months 7 hours":"a"}}""") :: Nil) - // interval type is invalid for converting to JSON. We can't use it as value type of a map. val df2 = baseDf .select(struct(map(lit("a"), $"a._1".cast(CalendarIntervalType)).as("col1")).as("c")) - val e2 = intercept[AnalysisException] { - df2.select(to_json($"c")).collect() - } - assert(e2.getMessage.contains("Unable to convert column col1 of type interval to JSON")) + checkAnswer( + df2.select(to_json($"c")), + Row("""{"col1":{"a":"-3 months 7 hours"}}""") :: Nil) } test("roundtrip in to_json and from_json - struct") { From 6dfaa0783f7779972752cac48fabbb321811f3c0 Mon Sep 17 00:00:00 2001 From: jiake Date: Wed, 22 Jan 2020 09:02:34 -0800 Subject: [PATCH 1189/1387] [SPARK-30549][SQL] Fix the subquery shown issue in UI When enable AQE ### What changes were proposed in this pull request? After [PR#25316](https://github.com/apache/spark/pull/25316) fixed the dead lock issue in [PR#25308](https://github.com/apache/spark/pull/25308), the subquery metrics can not be shown in UI as following screenshot. ![image](https://user-images.githubusercontent.com/11972570/72891385-160ec980-3d4f-11ea-91fc-ccaad890f7dc.png) This PR fix the subquery UI shown issue by adding `SparkListenerSQLAdaptiveSQLMetricUpdates` event to update the suquery sql metric. After with this PR, the suquery UI can show correctly as following screenshot: ![image](https://user-images.githubusercontent.com/11972570/72893610-66d4f100-3d54-11ea-93c9-f444b2f31952.png) ### Why are the changes needed? Showing the subquery metric in UI when enable AQE ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing UT Closes #27260 from JkSelf/fixSubqueryUI. Authored-by: jiake Signed-off-by: Xiao Li --- .../adaptive/AdaptiveSparkPlanExec.scala | 51 +++++++++++++++---- .../execution/ui/SQLAppStatusListener.scala | 9 ++++ .../spark/sql/execution/ui/SQLListener.scala | 6 +++ 3 files changed, 55 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 56ea12918448c..3f20b59361988 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -37,7 +37,8 @@ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._ import org.apache.spark.sql.execution.exchange._ -import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.ui.{SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLAdaptiveSQLMetricUpdates, SQLPlanMetric} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils @@ -132,16 +133,27 @@ case class AdaptiveSparkPlanExec( executedPlan.resetMetrics() } + private def collectSQLMetrics(plan: SparkPlan): Seq[SQLMetric] = { + val metrics = new mutable.ArrayBuffer[SQLMetric]() + plan.foreach { + case p: ShuffleQueryStageExec if (p.resultOption.isEmpty) => + collectSQLMetrics(p.plan).foreach(metrics += _) + case p: BroadcastQueryStageExec if (p.resultOption.isEmpty) => + collectSQLMetrics(p.plan).foreach(metrics += _) + case p: SparkPlan => + p.metrics.foreach { case metric => + metrics += metric._2 + } + } + metrics + } + private def getFinalPhysicalPlan(): SparkPlan = lock.synchronized { if (!isFinalPlan) { // Subqueries do not have their own execution IDs and therefore rely on the main query to // update UI. - val executionId = if (isSubquery) { - None - } else { - Option(context.session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)) - .map(_.toLong) - } + val executionId = Option(context.session.sparkContext.getLocalProperty( + SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) var currentLogicalPlan = currentPhysicalPlan.logicalLink.get var result = createQueryStages(currentPhysicalPlan) val events = new LinkedBlockingQueue[StageMaterializationEvent]() @@ -484,10 +496,27 @@ case class AdaptiveSparkPlanExec( * Notify the listeners of the physical plan change. */ private def onUpdatePlan(executionId: Long): Unit = { - context.session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( - executionId, - SQLExecution.getQueryExecution(executionId).toString, - SparkPlanInfo.fromSparkPlan(this))) + if (isSubquery) { + // When executing subqueries, we can't update the query plan in the UI as the + // UI doesn't support partial update yet. However, the subquery may have been + // optimized into a different plan and we must let the UI know the SQL metrics + // of the new plan nodes, so that it can track the valid accumulator updates later + // and display SQL metrics correctly. + onUpdateSQLMetrics(collectSQLMetrics(currentPhysicalPlan), executionId) + } else { + context.session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( + executionId, + SQLExecution.getQueryExecution(executionId).toString, + SparkPlanInfo.fromSparkPlan(this))) + } + } + + private def onUpdateSQLMetrics(sqlMetrics: Seq[SQLMetric], executionId: Long): Unit = { + val sqlPlanMetrics = sqlMetrics.map { case sqlMetric => + SQLPlanMetric(sqlMetric.name.get, sqlMetric.id, sqlMetric.metricType) + } + context.session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveSQLMetricUpdates( + executionId.toLong, sqlPlanMetrics)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index d5bb36e8a0c95..1454cc05ed4da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -344,6 +344,14 @@ class SQLAppStatusListener( update(exec) } + private def onAdaptiveSQLMetricUpdate(event: SparkListenerSQLAdaptiveSQLMetricUpdates): Unit = { + val SparkListenerSQLAdaptiveSQLMetricUpdates(executionId, sqlPlanMetrics) = event + + val exec = getOrCreateExecution(executionId) + exec.metrics = exec.metrics ++ sqlPlanMetrics + update(exec) + } + private def onExecutionEnd(event: SparkListenerSQLExecutionEnd): Unit = { val SparkListenerSQLExecutionEnd(executionId, time) = event Option(liveExecutions.get(executionId)).foreach { exec => @@ -383,6 +391,7 @@ class SQLAppStatusListener( override def onOtherEvent(event: SparkListenerEvent): Unit = event match { case e: SparkListenerSQLExecutionStart => onExecutionStart(e) case e: SparkListenerSQLAdaptiveExecutionUpdate => onAdaptiveExecutionUpdate(e) + case e: SparkListenerSQLAdaptiveSQLMetricUpdates => onAdaptiveSQLMetricUpdate(e) case e: SparkListenerSQLExecutionEnd => onExecutionEnd(e) case e: SparkListenerDriverAccumUpdates => onDriverAccumUpdates(e) case _ => // Ignore diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 81cbc7f54c7eb..6a6a71c46f213 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -34,6 +34,12 @@ case class SparkListenerSQLAdaptiveExecutionUpdate( sparkPlanInfo: SparkPlanInfo) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerSQLAdaptiveSQLMetricUpdates( + executionId: Long, + sqlPlanMetrics: Seq[SQLPlanMetric]) + extends SparkListenerEvent + @DeveloperApi case class SparkListenerSQLExecutionStart( executionId: Long, From 84f11548e428abc28617218e4405a159d44c0eac Mon Sep 17 00:00:00 2001 From: Udbhav30 Date: Wed, 22 Jan 2020 14:20:28 -0800 Subject: [PATCH 1190/1387] [SPARK-30604][CORE] Fix a log message by including hostLocalBlockBytes to total bytes ### What changes were proposed in this pull request? Add HostLocalBlock size in log total bytes ### Why are the changes needed? total size in log is wrong as hostlocal block size is missed ### Does this PR introduce any user-facing change? no ### How was this patch tested? Manually checking the log Closes #27320 from Udbhav30/bug. Authored-by: Udbhav30 Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/storage/ShuffleBlockFetcherIterator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 97e0600da4a99..cd4c86006af5a 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -317,7 +317,7 @@ final class ShuffleBlockFetcherIterator( collectFetchRequests(address, blockInfos, collectedRemoteRequests) } } - val totalBytes = localBlockBytes + remoteBlockBytes + val totalBytes = localBlockBytes + remoteBlockBytes + hostLocalBlockBytes logInfo(s"Getting $numBlocksToFetch (${Utils.bytesToString(totalBytes)}) non-empty blocks " + s"including ${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local and " + s"${hostLocalBlocks.size} (${Utils.bytesToString(hostLocalBlockBytes)}) " + From 4ca31b470f47f5cefd603778852e828420a89456 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 22 Jan 2020 15:40:24 -0800 Subject: [PATCH 1191/1387] [SPARK-30606][SQL] Fix the `like` function with 2 parameters ### What changes were proposed in this pull request? In the PR, I propose to add additional constructor in the `Like` expression. The constructor can be used on applying the `like` function with 2 parameters. ### Why are the changes needed? `FunctionRegistry` cannot find a constructor if the `like` function is applied to 2 parameters. ### Does this PR introduce any user-facing change? Yes, before: ```sql spark-sql> SELECT like('Spark', '_park'); Invalid arguments for function like; line 1 pos 7 org.apache.spark.sql.AnalysisException: Invalid arguments for function like; line 1 pos 7 at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$.$anonfun$expression$7(FunctionRegistry.scala:618) at scala.Option.getOrElse(Option.scala:189) at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$.$anonfun$expression$4(FunctionRegistry.scala:602) at org.apache.spark.sql.catalyst.analysis.SimpleFunctionRegistry.lookupFunction(FunctionRegistry.scala:121) at org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupFunction(SessionCatalog.scala:1412) ``` After: ```sql spark-sql> SELECT like('Spark', '_park'); true ``` ### How was this patch tested? By running `check outputs of expression examples` from `SQLQuerySuite`. Closes #27323 from MaxGekk/fix-like-func. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/regexpExpressions.scala | 6 +++++- .../spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala | 4 ++-- sql/core/src/main/scala/org/apache/spark/sql/Column.scala | 2 +- 3 files changed, 8 insertions(+), 4 deletions(-) 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 c6fb6c95e5d31..2354087768615 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 @@ -95,6 +95,8 @@ abstract class StringRegexExpression extends BinaryExpression """, examples = """ Examples: + > SELECT _FUNC_('Spark', '_park'); + true > SET spark.sql.parser.escapedStringLiterals=true; spark.sql.parser.escapedStringLiterals true > SELECT '%SystemDrive%\Users\John' _FUNC_ '\%SystemDrive\%\\Users%'; @@ -111,9 +113,11 @@ abstract class StringRegexExpression extends BinaryExpression """, since = "1.0.0") // scalastyle:on line.contains.tab -case class Like(left: Expression, right: Expression, escapeChar: Char = '\\') +case class Like(left: Expression, right: Expression, escapeChar: Char) extends StringRegexExpression { + def this(left: Expression, right: Expression) = this(left, right, '\\') + override def escape(v: String): String = StringUtils.escapeLikeRegex(v, escapeChar) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 641c89873dcc4..23ab6b2df3e64 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -196,8 +196,8 @@ class ConstantFoldingSuite extends PlanTest { EqualTo(Literal.create(null, IntegerType), 1) as 'c11, EqualTo(1, Literal.create(null, IntegerType)) as 'c12, - Like(Literal.create(null, StringType), "abc") as 'c13, - Like("abc", Literal.create(null, StringType)) as 'c14, + new Like(Literal.create(null, StringType), "abc") as 'c13, + new Like("abc", Literal.create(null, StringType)) as 'c14, Upper(Literal.create(null, StringType)) as 'c15, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index ed10843b08596..8bd5835fd931b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -841,7 +841,7 @@ class Column(val expr: Expression) extends Logging { * @group expr_ops * @since 1.3.0 */ - def like(literal: String): Column = withExpr { Like(expr, lit(literal).expr) } + def like(literal: String): Column = withExpr { new Like(expr, lit(literal).expr) } /** * SQL RLIKE expression (LIKE with Regex). Returns a boolean column based on a regex From 2e74dba3d065d40a4e487c9135ec3c4c99d3d50a Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Wed, 22 Jan 2020 18:30:42 -0600 Subject: [PATCH 1192/1387] [SPARK-30574][DOC] Document GROUP BY Clause of SELECT statement in SQL Reference ### What changes were proposed in this pull request? Document GROUP BY clause of SELECT statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2020-01-19 at 5 11 12 PM Screen Shot 2020-01-19 at 5 11 32 PM Screen Shot 2020-01-19 at 5 11 49 PM Screen Shot 2020-01-19 at 5 12 05 PM Screen Shot 2020-01-19 at 5 12 31 PM Screen Shot 2020-01-19 at 5 26 38 PM ### How was this patch tested? Tested using jykyll build --serve Closes #27283 from dilipbiswal/sql-ref-select-groupby. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/sql-ref-syntax-qry-select-groupby.md | 209 +++++++++++++++++++++- 1 file changed, 206 insertions(+), 3 deletions(-) diff --git a/docs/sql-ref-syntax-qry-select-groupby.md b/docs/sql-ref-syntax-qry-select-groupby.md index 8ba7d155f3716..2d13c349561a8 100644 --- a/docs/sql-ref-syntax-qry-select-groupby.md +++ b/docs/sql-ref-syntax-qry-select-groupby.md @@ -1,7 +1,7 @@ --- layout: global -title: GROUPBY Clause -displayTitle: GROUPBY Clause +title: GROUP BY Clause +displayTitle: GROUP BY Clause license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -18,5 +18,208 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- +The GROUP BY clause is used to group the rows based on a set of specified grouping expressions and compute aggregations on +the group of rows based on one or more specified aggregate functions. Spark also supports advanced aggregations to do multiple +aggregations for the same input record set via `GROUPING SETS`, `CUBE`, `ROLLUP` clauses. -**This page is under construction** +### Syntax +{% highlight sql %} +GROUP BY [ GROUPING SETS grouping_sets ] group_expression [ , group_expression [ , ... ] ] + [ ( WITH ROLLUP | WITH CUBE | GROUPING SETS grouping_sets ) ) ] +{% endhighlight %} + +### Parameters +
      +
      GROUPING SETS
      +
      + Groups the rows for each subset of the expressions specified in the grouping sets. For example, + GROUP BY GROUPING SETS (warehouse, product) is semantically equivalent + to union of results of GROUP BY warehouse and GROUP BY product. This clause + is shorthand for a UNION ALL where each leg of the UNION ALL + operator performs aggregation of subset of the columns specified in the GROUPING SETS clause. +
      +
      grouping_sets
      +
      + Specifies one of more groupings based on which the GROUP BY clause performs aggregations. A grouping + set is specified by a list of comma-separated expressions in parentheses.

      + Syntax: + + (() | (expression [ , ...])) + +
      +
      grouping_expression
      +
      + Specifies the critieria based on which the rows are grouped together. The grouping of rows is performed based on + result values of the grouping expressions. A grouping expression may be a column alias, a column position + or an expression. +
      +
      ROLLUP
      +
      + Specifies multiple levels of aggregations in a single statement. This clause is used to compute aggregations + based on multiple grouping sets. ROLLUP is shorthand for GROUPING SETS. For example, + GROUP BY warehouse, product WITH ROLLUP is equivalent to GROUP BY warehouse, product GROUPING SETS + ((warehouse, product), (warehouse), ()). + The N elements of a ROLLUP specification results in N+1 GROUPING SETS. +
      +
      CUBE
      +
      + CUBE clause is used to perform aggregations based on combination of grouping columns specified in the + GROUP BY clause. For example, GROUP BY warehouse, product WITH CUBE is equivalent + to GROUP BY warehouse, product GROUPING SETS ((warehouse, product), (warehouse), (product), ()). + The N elements of a CUBE specification results in 2^N GROUPING SETS. +
      +
      + +### Examples +{% highlight sql %} +CREATE TABLE dealer (id INT, city STRING, car_model STRING, quantity INT); +INSERT INTO dealer VALUES (100, 'Fremont', 'Honda Civic', 10), + (100, 'Fremont', 'Honda Accord', 15), + (100, 'Fremont', 'Honda CRV', 7), + (200, 'Dublin', 'Honda Civic', 20), + (200, 'Dublin', 'Honda Accord', 10), + (200, 'Dublin', 'Honda CRV', 3), + (300, 'San Jose', 'Honda Civic', 5), + (300, 'San Jose', 'Honda Accord', 8); + +-- Sum of quantity per dealership. Group by `id`. +SELECT id, sum(quantity) FROM dealer GROUP BY id ORDER BY id; + + +---+-------------+ + |id |sum(quantity)| + +---+-------------+ + |100|32 | + |200|33 | + |300|13 | + +---+-------------+ + +-- Use column position in GROUP by clause. +SELECT id, sum(quantity) FROM dealer GROUP BY 1 ORDER BY 1; + + +---+-------------+ + |id |sum(quantity)| + +---+-------------+ + |100|32 | + |200|33 | + |300|13 | + +---+-------------+ + +-- Multiple aggregations. +-- 1. Sum of quantity per dealership. +-- 2. Max quantity per dealership. +SELECT id, sum(quantity) AS sum, max(quantity) AS max FROM dealer GROUP BY id ORDER BY id; + + +---+---+---+ + |id |sum|max| + +---+---+---+ + |100|32 |15 | + |200|33 |20 | + |300|13 |8 | + +---+---+---+ + +-- Aggregations using multiple sets of grouping columns in a single statement. +-- Following performs aggregations based on four sets of grouping columns. +-- 1. city, car_model +-- 2. city +-- 3. car_model +-- 4. Empty grouping set. Returns quantities for all city and car models. +SELECT city, car_model, sum(quantity) AS sum FROM dealer + GROUP BY GROUPING SETS ((city, car_model), (city), (car_model), ()) + ORDER BY city; + + +--------+------------+---+ + |city |car_model |sum| + +--------+------------+---+ + |null |null |78 | + |null |Honda Accord|33 | + |null |Honda CRV |10 | + |null |Honda Civic |35 | + |Dublin |null |33 | + |Dublin |Honda Accord|10 | + |Dublin |Honda CRV |3 | + |Dublin |Honda Civic |20 | + |Fremont |null |32 | + |Fremont |Honda Accord|15 | + |Fremont |Honda CRV |7 | + |Fremont |Honda Civic |10 | + |San Jose|null |13 | + |San Jose|Honda Accord|8 | + |San Jose|Honda Civic |5 | + +--------+------------+---+ + +-- Alternate syntax for `GROUPING SETS` in which both `GROUP BY` and `GROUPING SETS` +-- specifications are present. +SELECT city, car_model, sum(quantity) AS sum FROM dealer + GROUP BY city, car_model GROUPING SETS ((city, car_model), (city), (car_model), ()) + ORDER BY city, car_model; + + +--------+------------+---+ + |city |car_model |sum| + +--------+------------+---+ + |null |null |78 | + |null |Honda Accord|33 | + |null |Honda CRV |10 | + |null |Honda Civic |35 | + |Dublin |null |33 | + |Dublin |Honda Accord|10 | + |Dublin |Honda CRV |3 | + |Dublin |Honda Civic |20 | + |Fremont |null |32 | + |Fremont |Honda Accord|15 | + |Fremont |Honda CRV |7 | + |Fremont |Honda Civic |10 | + |San Jose|null |13 | + |San Jose|Honda Accord|8 | + |San Jose|Honda Civic |5 | + +--------+------------+---+ + +-- Group by processing with `ROLLUP` clause. +-- Equivalent GROUP BY GROUPING SETS ((city, car_model), (city), ()) +SELECT city, car_model, sum(quantity) AS sum FROM dealer + GROUP BY city, car_model WITH ROLLUP + ORDER BY city, car_model; + + +--------+------------+---+ + |city |car_model |sum| + +--------+------------+---+ + |null |null |78 | + |Dublin |null |33 | + |Dublin |Honda Accord|10 | + |Dublin |Honda CRV |3 | + |Dublin |Honda Civic |20 | + |Fremont |null |32 | + |Fremont |Honda Accord|15 | + |Fremont |Honda CRV |7 | + |Fremont |Honda Civic |10 | + |San Jose|null |13 | + |San Jose|Honda Accord|8 | + |San Jose|Honda Civic |5 | + +--------+------------+---+ + +-- Group by processing with `CUBE` clause. +-- Equivalent GROUP BY GROUPING SETS ((city, car_model), (city), (car_model), ()) +SELECT city, car_model, sum(quantity) AS sum FROM dealer + GROUP BY city, car_model WITH CUBE + ORDER BY city, car_model; + + +--------+------------+---+ + |city |car_model |sum| + +--------+------------+---+ + |null |null |78 | + |null |Honda Accord|33 | + |null |Honda CRV |10 | + |null |Honda Civic |35 | + |Dublin |null |33 | + |Dublin |Honda Accord|10 | + |Dublin |Honda CRV |3 | + |Dublin |Honda Civic |20 | + |Fremont |null |32 | + |Fremont |Honda Accord|15 | + |Fremont |Honda CRV |7 | + |Fremont |Honda Civic |10 | + |San Jose|null |13 | + |San Jose|Honda Accord|8 | + |San Jose|Honda Civic |5 | + +--------+------------+---+ + +{% endhighlight %} From 38f4e599b385a54bd5a8866585927ded7caf3939 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Wed, 22 Jan 2020 18:46:28 -0600 Subject: [PATCH 1193/1387] [SPARK-28801][DOC] Document SELECT statement in SQL Reference (Main page) ### What changes were proposed in this pull request? Document SELECT statement in SQL Reference Guide. In this PR includes the main entry page for SELECT. I will open follow-up PRs for different clauses. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2020-01-19 at 11 20 41 PM Screen Shot 2020-01-19 at 11 21 55 PM Screen Shot 2020-01-19 at 11 22 16 PM ### How was this patch tested? Tested using jykyll build --serve Closes #27216 from dilipbiswal/sql_ref_select_hook. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/sql-ref-syntax-qry-select.md | 119 +++++++++++++++++++++++++++++- 1 file changed, 115 insertions(+), 4 deletions(-) diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md index 41972ef070831..05feda5f9a5dd 100644 --- a/docs/sql-ref-syntax-qry-select.md +++ b/docs/sql-ref-syntax-qry-select.md @@ -18,8 +18,119 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- +Spark supports a `SELECT` statement and conforms to the ANSI SQL standard. Queries are +used to retrieve result sets from one or more tables. The following section +describes the overall query syntax and the sub-sections cover different constructs +of a query along with examples. -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +### Syntax +{% highlight sql %} +[ WITH with_query [ , ... ] ] +SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } + FROM { from_item [ , ...] } + [ WHERE boolean_expression ] + [ GROUP BY expression [ , ...] ] + [ HAVING boolean_expression ] + [ ORDER BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] + [ SORT BY { expression [ ASC | DESC ] [ NULLS { FIRST | LAST } ] [ , ...] } ] + [ CLUSTER BY { expression [ , ...] } ] + [ DISTRIBUTE BY { expression [, ...] } ] + { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select ] + [ WINDOW { named_window [ , WINDOW named_window, ... ] } ] + [ LIMIT { ALL | expression } ] +{% endhighlight %} + +### Parameters +
      +
      with_query
      +
      + Specifies the common table expressions (CTEs) before the main SELECT query block. + These table expressions are allowed to be referenced later in the main query. This is useful to abstract + out repeated subquery blocks in the main query and improves readability of the query. +
      +
      hints
      +
      + Hints can be specified to help spark optimizer make better planning decisions. Currently spark supports hints + that influence selection of join strategies and repartitioning of the data. +
      +
      ALL
      +
      + Select all matching rows from the relation and is enabled by default. +
      +
      DISTINCT
      +
      + Select all matching rows from the relation after removing duplicates in results. +
      +
      named_expression
      +
      + An expression with an assigned name. In general, it denotes a column expression.

      + Syntax: + + expression [AS] [alias] + +
      +
      from_item
      +
      + Specifies a source of input for the query. It can be one of the following: +
        +
      1. Table relation
      2. +
      3. Join relation
      4. +
      5. Table valued function
      6. +
      7. Inlined table
      8. +
      9. Subquery
      10. +
      +
      +
      WHERE
      +
      + Filters the result of the FROM clause based on the supplied predicates. +
      +
      GROUP BY
      +
      + Specifies the expressions that are used to group the rows. This is used in conjunction with aggregate functions + (MIN, MAX, COUNT, SUM, AVG) to group rows based on the grouping expressions. +
      +
      HAVING
      +
      + Specifies the predicates by which the rows produced by GROUP BY are filtered. The HAVING clause is used to + filter rows after the grouping is performed. +
      +
      ORDER BY
      +
      + Specifies an ordering of the rows of the complete result set of the query. The output rows are ordered + across the partitions. This parameter is mutually exclusive with SORT BY, + CLUSTER BY and DISTRIBUTE BY and can not be specified together. +
      +
      SORT BY
      +
      + Specifies an ordering by which the rows are ordered within each partition. This parameter is mutually + exclusive with ORDER BY and CLUSTER BY and can not be specified together. +
      +
      CLUSTER BY
      +
      + Specifies a set of expressions that is used to repartition and sort the rows. Using this clause has + the same effect of using DISTRIBUTE BY and SORT BY together. +
      +
      DISTRIBUTE BY
      +
      + Specifies a set of expressions by which the result rows are repartitioned. This parameter is mutually + exclusive with ORDER BY and CLUSTER BY and can not be specified together. +
      +
      LIMIT
      +
      + Specifies the maximum number of rows that can be returned by a statement or subquery. This clause + is mostly used in the conjunction with ORDER BY to produce a deterministic result. +
      +
      boolean_expression
      +
      + Specifies an expression with a return type of boolean. +
      +
      expression
      +
      + Specifies a combination of one or more values, operators, and SQL functions that evaluates to a value. +
      +
      named_window
      +
      + Specifies aliases for one or more source window specifications. The source window specifications can + be referenced in the widow definitions in the query. +
      +
      From eccae13a5faf93a524754e4cfcf71cbe8f3ad4e6 Mon Sep 17 00:00:00 2001 From: Enrico Minack Date: Wed, 22 Jan 2020 19:51:08 -0600 Subject: [PATCH 1194/1387] [SPARK-30531][WEB UI] Do not render plan viz when it exists already ### What changes were proposed in this pull request? When you save a Spark UI SQL query page to disk and then display the html file with your browser, the query plan will be rendered a second time. This change avoids rendering the plan visualization when it exists already. This is master: ![grafik](https://user-images.githubusercontent.com/44700269/72543429-fcb8d980-3885-11ea-82aa-c0b3638847e5.png) And with the fix: ![grafik](https://user-images.githubusercontent.com/44700269/72543641-57523580-3886-11ea-8cdf-5fb0cdffa983.png) ### Why are the changes needed? The duplicate query plan is unexpected and redundant. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually tested. Testing this in a reproducible way requires a running browser or HTML rendering engine that executes the JavaScript. Closes #27238 from EnricoMi/branch-sql-ui-duplicate-plan. Authored-by: Enrico Minack Signed-off-by: Sean Owen --- .../org/apache/spark/sql/execution/ui/static/spark-sql-viz.js | 4 ++++ .../org/apache/spark/sql/execution/ui/ExecutionPage.scala | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js index 2329fd262ddfb..754711bd5ad85 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js @@ -20,6 +20,10 @@ var PlanVizConstants = { svgMarginY: 16 }; +function shouldRenderPlanViz() { + return planVizContainer().selectAll("svg").empty(); +} + function renderPlanViz() { var svg = planVizContainer().append("svg"); var metadata = d3.select("#plan-viz-metadata"); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala index 875086cda258d..91360e0e50314 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala @@ -116,7 +116,7 @@ class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging {metadata}

    {planVisualizationResources(request)} - + } From bbab2bb961a8d0c8fd57a2899c94b00c747e8444 Mon Sep 17 00:00:00 2001 From: Ajith Date: Wed, 22 Jan 2020 18:21:11 -0800 Subject: [PATCH 1195/1387] [SPARK-30556][SQL] Copy sparkContext.localproperties to child thread inSubqueryExec.executionContext ### What changes were proposed in this pull request? In `org.apache.spark.sql.execution.SubqueryExec#relationFuture` make a copy of `org.apache.spark.SparkContext#localProperties` and pass it to the sub-execution thread in `org.apache.spark.sql.execution.SubqueryExec#executionContext` ### Why are the changes needed? Local properties set via sparkContext are not available as TaskContext properties when executing jobs and threadpools have idle threads which are reused Explanation: When `SubqueryExec`, the relationFuture is evaluated via a separate thread. The threads inherit the `localProperties` from `sparkContext` as they are the child threads. These threads are created in the `executionContext` (thread pools). Each Thread pool has a default keepAliveSeconds of 60 seconds for idle threads. Scenarios where the thread pool has threads which are idle and reused for a subsequent new query, the thread local properties will not be inherited from spark context (thread properties are inherited only on thread creation) hence end up having old or no properties set. This will cause taskset properties to be missing when properties are transferred by child thread via `sparkContext.runJob/submitJob` ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added UT Closes #27267 from ajithme/subquerylocalprop. Authored-by: Ajith Signed-off-by: Dongjoon Hyun --- .../spark/sql/internal/StaticSQLConf.scala | 10 +++++- .../spark/sql/execution/SQLExecution.scala | 18 ++++++++++ .../execution/basicPhysicalOperators.scala | 12 ++++--- .../internal/ExecutorSideSQLConfSuite.scala | 36 +++++++++++++++++-- 4 files changed, 69 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index d2f27da239016..66ac9ddb21aaa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -145,9 +145,17 @@ object StaticSQLConf { "cause longer waiting for other broadcasting. Also, increasing parallelism may " + "cause memory problem.") .intConf - .checkValue(thres => thres > 0 && thres <= 128, "The threshold must be in [0,128].") + .checkValue(thres => thres > 0 && thres <= 128, "The threshold must be in (0,128].") .createWithDefault(128) + val SUBQUERY_MAX_THREAD_THRESHOLD = + buildStaticConf("spark.sql.subquery.maxThreadThreshold") + .internal() + .doc("The maximum degree of parallelism to execute the subquery.") + .intConf + .checkValue(thres => thres > 0 && thres <= 128, "The threshold must be in (0,128].") + .createWithDefault(16) + val SQL_EVENT_TRUNCATE_LENGTH = buildStaticConf("spark.sql.event.truncate.length") .doc("Threshold of SQL length beyond which it will be truncated before adding to " + "event. Defaults to no truncation. If set to 0, callsite will be logged instead.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 6046805ae95d4..995d94ef5eac7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong +import scala.concurrent.{ExecutionContext, Future} + import org.apache.spark.SparkContext import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.sql.SparkSession @@ -164,4 +166,20 @@ object SQLExecution { } } } + + /** + * Wrap passed function to ensure necessary thread-local variables like + * SparkContext local properties are forwarded to execution thread + */ + def withThreadLocalCaptured[T]( + sparkSession: SparkSession, exec: ExecutionContext)(body: => T): Future[T] = { + val activeSession = sparkSession + val sc = sparkSession.sparkContext + val localProps = Utils.cloneProperties(sc.getLocalProperties) + Future { + SparkSession.setActiveSession(activeSession) + sc.setLocalProperties(localProps) + body + }(exec) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index e128d59dca6ba..f3f756425a153 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -31,8 +31,9 @@ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.{LongType, StructType} -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler} /** Physical plan for Project. */ @@ -749,7 +750,9 @@ case class SubqueryExec(name: String, child: SparkPlan) private lazy val relationFuture: Future[Array[InternalRow]] = { // relationFuture is used in "doExecute". Therefore we can get the execution id correctly here. val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - Future { + SQLExecution.withThreadLocalCaptured[Array[InternalRow]]( + sqlContext.sparkSession, + SubqueryExec.executionContext) { // This will run in another thread. Set the execution id so that we can connect these jobs // with the correct execution. SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { @@ -764,7 +767,7 @@ case class SubqueryExec(name: String, child: SparkPlan) SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) rows } - }(SubqueryExec.executionContext) + } } protected override def doCanonicalize(): SparkPlan = { @@ -788,7 +791,8 @@ case class SubqueryExec(name: String, child: SparkPlan) object SubqueryExec { private[execution] val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("subquery", 16)) + ThreadUtils.newDaemonCachedThreadPool("subquery", + SQLConf.get.getConf(StaticSQLConf.SUBQUERY_MAX_THREAD_THRESHOLD))) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index 776cdb107084d..0cc658c499615 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.internal import org.scalatest.Assertions._ -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite, TaskContext} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LocalRelation @@ -125,6 +125,38 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { val e = intercept[SparkException](dummyQueryExecution1.toRdd.collect()) assert(e.getCause.isInstanceOf[NoSuchElementException]) } + + test("SPARK-30556 propagate local properties to subquery execution thread") { + withSQLConf(StaticSQLConf.SUBQUERY_MAX_THREAD_THRESHOLD.key -> "1") { + withTempView("l", "m", "n") { + Seq(true).toDF().createOrReplaceTempView("l") + val confKey = "spark.sql.y" + + def createDataframe(confKey: String, confValue: String): Dataset[Boolean] = { + Seq(true) + .toDF() + .mapPartitions { _ => + TaskContext.get.getLocalProperty(confKey) == confValue match { + case true => Iterator(true) + case false => Iterator.empty + } + } + } + + // set local configuration and assert + val confValue1 = "e" + createDataframe(confKey, confValue1).createOrReplaceTempView("m") + spark.sparkContext.setLocalProperty(confKey, confValue1) + assert(sql("SELECT * FROM l WHERE EXISTS (SELECT * FROM m)").collect.size == 1) + + // change the conf value and assert again + val confValue2 = "f" + createDataframe(confKey, confValue2).createOrReplaceTempView("n") + spark.sparkContext.setLocalProperty(confKey, confValue2) + assert(sql("SELECT * FROM l WHERE EXISTS (SELECT * FROM n)").collect().size == 1) + } + } + } } case class SQLConfAssertPlan(confToCheck: Seq[(String, String)]) extends LeafExecNode { From d2bca8ff70e6c82e915f633bb9f2f8a4582f7026 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 22 Jan 2020 19:20:25 -0800 Subject: [PATCH 1196/1387] [SPARK-30609] Allow default merge command resolution to be bypassed by DSv2 tables ### What changes were proposed in this pull request? Skip resolving the merge expressions if the target is a DSv2 table with ACCEPT_ANY_SCHEMA capability. ### Why are the changes needed? Some DSv2 sources may want to customize the merge resolution logic. For example, a table that can accept any schema (TableCapability.ACCEPT_ANY_SCHEMA) may want to allow certain merge queries that are blocked (that is, throws AnalysisError) by the default resolution logic. So there should be a way to completely bypass the merge resolution logic in the Analyzer. ### Does this PR introduce any user-facing change? No, since merge itself is an unreleased feature ### How was this patch tested? added unit test to specifically test the skipping. Closes #27326 from tdas/SPARK-30609. Authored-by: Tathagata Das Signed-off-by: Tathagata Das --- .../sql/catalyst/analysis/Analyzer.scala | 62 +++++++++++-------- .../command/PlanResolutionSuite.scala | 52 +++++++++++++++- 2 files changed, 86 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 7e9f85b64e4a9..503dab1782172 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1326,33 +1326,43 @@ class Analyzer( case m @ MergeIntoTable(targetTable, sourceTable, _, _, _) if !m.resolved && targetTable.resolved && sourceTable.resolved => - val newMatchedActions = m.matchedActions.map { - case DeleteAction(deleteCondition) => - val resolvedDeleteCondition = deleteCondition.map(resolveExpressionTopDown(_, m)) - DeleteAction(resolvedDeleteCondition) - case UpdateAction(updateCondition, assignments) => - val resolvedUpdateCondition = updateCondition.map(resolveExpressionTopDown(_, m)) - // The update value can access columns from both target and source tables. - UpdateAction( - resolvedUpdateCondition, - resolveAssignments(assignments, m, resolveValuesWithSourceOnly = false)) - case o => o - } - val newNotMatchedActions = m.notMatchedActions.map { - case InsertAction(insertCondition, assignments) => - // The insert action is used when not matched, so its condition and value can only - // access columns from the source table. - val resolvedInsertCondition = - insertCondition.map(resolveExpressionTopDown(_, Project(Nil, m.sourceTable))) - InsertAction( - resolvedInsertCondition, - resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true)) - case o => o + + EliminateSubqueryAliases(targetTable) match { + case r: NamedRelation if r.skipSchemaResolution => + // Do not resolve the expression if the target table accepts any schema. + // This allows data sources to customize their own resolution logic using + // custom resolution rules. + m + + case _ => + val newMatchedActions = m.matchedActions.map { + case DeleteAction(deleteCondition) => + val resolvedDeleteCondition = deleteCondition.map(resolveExpressionTopDown(_, m)) + DeleteAction(resolvedDeleteCondition) + case UpdateAction(updateCondition, assignments) => + val resolvedUpdateCondition = updateCondition.map(resolveExpressionTopDown(_, m)) + // The update value can access columns from both target and source tables. + UpdateAction( + resolvedUpdateCondition, + resolveAssignments(assignments, m, resolveValuesWithSourceOnly = false)) + case o => o + } + val newNotMatchedActions = m.notMatchedActions.map { + case InsertAction(insertCondition, assignments) => + // The insert action is used when not matched, so its condition and value can only + // access columns from the source table. + val resolvedInsertCondition = + insertCondition.map(resolveExpressionTopDown(_, Project(Nil, m.sourceTable))) + InsertAction( + resolvedInsertCondition, + resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true)) + case o => o + } + val resolvedMergeCondition = resolveExpressionTopDown(m.mergeCondition, m) + m.copy(mergeCondition = resolvedMergeCondition, + matchedActions = newMatchedActions, + notMatchedActions = newNotMatchedActions) } - val resolvedMergeCondition = resolveExpressionTopDown(m.mergeCondition, m) - m.copy(mergeCondition = resolvedMergeCondition, - matchedActions = newMatchedActions, - notMatchedActions = newNotMatchedActions) case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString(SQLConf.get.maxToStringFields)}") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 8c73b366fa857..35b2003e952e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.command import java.net.URI -import java.util.Locale +import java.util.{Collections, Locale} import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, when} @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, E import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.InMemoryTableProvider -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCapability, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf @@ -49,6 +49,13 @@ class PlanResolutionSuite extends AnalysisTest { t } + private val tableWithAcceptAnySchemaCapability: Table = { + val t = mock(classOf[Table]) + when(t.schema()).thenReturn(new StructType().add("i", "int")) + when(t.capabilities()).thenReturn(Collections.singleton(TableCapability.ACCEPT_ANY_SCHEMA)) + t + } + private val v1Table: V1Table = { val t = mock(classOf[CatalogTable]) when(t.schema).thenReturn(new StructType().add("i", "int").add("s", "string")) @@ -77,6 +84,7 @@ class PlanResolutionSuite extends AnalysisTest { case "v1Table1" => v1Table case "v2Table" => table case "v2Table1" => table + case "v2TableWithAcceptAnySchemaCapability" => tableWithAcceptAnySchemaCapability case name => throw new NoSuchTableException(name) } }) @@ -1351,5 +1359,45 @@ class PlanResolutionSuite extends AnalysisTest { } } + test("MERGE INTO TABLE - skip resolution on v2 tables that accept any schema") { + val sql = + s""" + |MERGE INTO v2TableWithAcceptAnySchemaCapability AS target + |USING v2Table AS source + |ON target.i = source.i + |WHEN MATCHED AND (target.s='delete') THEN DELETE + |WHEN MATCHED AND (target.s='update') THEN UPDATE SET target.s = source.s + |WHEN NOT MATCHED AND (target.s='insert') + | THEN INSERT (target.i, target.s) values (source.i, source.s) + """.stripMargin + + parseAndResolve(sql) match { + case MergeIntoTable( + SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("source", None), _: DataSourceV2Relation), + EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), + Seq( + DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), + UpdateAction( + Some(EqualTo(ul: UnresolvedAttribute, StringLiteral("update"))), + updateAssigns)), + Seq( + InsertAction( + Some(EqualTo(il: UnresolvedAttribute, StringLiteral("insert"))), + insertAssigns))) => + assert(l.name == "target.i" && r.name == "source.i") + assert(dl.name == "target.s") + assert(ul.name == "target.s") + assert(il.name == "target.s") + assert(updateAssigns.size == 1) + assert(updateAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.s") + assert(updateAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.s") + assert(insertAssigns.size == 2) + assert(insertAssigns.head.key.asInstanceOf[UnresolvedAttribute].name == "target.i") + assert(insertAssigns.head.value.asInstanceOf[UnresolvedAttribute].name == "source.i") + + case l => fail("Expected unresolved MergeIntoTable, but got:\n" + l.treeString) + } + } // TODO: add tests for more commands. } From db528e4fe1907b6bbb1a2e4132427b5c1345710d Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 22 Jan 2020 22:43:46 -0800 Subject: [PATCH 1197/1387] [SPARK-30535][SQL] Revert "[] Migrate ALTER TABLE commands to the new framework ### What changes were proposed in this pull request? This reverts commit b5cb9abdd5ee286cc2b0a06cb5f3eac812922a31. ### Why are the changes needed? The merged commit (#27243) was too risky for several reasons: 1. It doesn't fix a bug 2. It makes the resolution of the table that's going to be altered a child. We had avoided this on purpose as having an arbitrary rule change the child of AlterTable seemed risky. This change alone is a big -1 for me for this change. 3. While the code may look cleaner, I think this approach makes certain things harder, e.g. differentiating between the Hive based Alter table CHANGE COLUMN and ALTER COLUMN syntax. Resolving and normalizing columns for ALTER COLUMN also becomes a bit harder, as we now have to check every single AlterTable command instead of just a single ALTER TABLE ALTER COLUMN statement ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing unit tests This closes #27315 Closes #27327 from brkyvz/revAlter. Authored-by: Burak Yavuz Signed-off-by: Xiao Li --- .../sql/catalyst/analysis/Analyzer.scala | 25 ++- .../sql/catalyst/analysis/CheckAnalysis.scala | 41 ++-- .../catalyst/analysis/ResolveCatalogs.scala | 67 ++++++- .../sql/catalyst/analysis/unresolved.scala | 23 +++ .../catalyst/analysis/v2ResolutionPlans.scala | 14 +- .../sql/catalyst/parser/AstBuilder.scala | 50 ++--- .../catalyst/plans/logical/statements.scala | 56 ++++++ .../catalyst/plans/logical/v2Commands.scala | 138 +++----------- .../sql/connector/catalog/CatalogV2Util.scala | 14 +- .../sql/catalyst/parser/DDLParserSuite.scala | 90 ++++----- .../analysis/ResolveSessionCatalog.scala | 178 +++++++++++++----- .../spark/sql/execution/command/tables.scala | 8 + .../datasources/v2/DataSourceV2Strategy.scala | 14 +- .../sql-tests/results/change-column.sql.out | 4 +- .../sql/connector/DataSourceV2SQLSuite.scala | 2 +- .../spark/sql/execution/SQLViewSuite.scala | 8 +- .../sql/execution/command/DDLSuite.scala | 5 +- .../command/PlanResolutionSuite.scala | 47 ++--- .../sql/hive/execution/HiveCommandSuite.scala | 2 +- 19 files changed, 462 insertions(+), 324 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 503dab1782172..36e558b0dc571 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -755,14 +755,12 @@ class Analyzer( .map(view => i.copy(table = view)) .getOrElse(i) case u @ UnresolvedTable(ident) => - lookupTempView(ident) - .map(_ => UnresolvedTableWithViewExists( - ResolvedView(ident.asIdentifier, isTempView = true))) - .getOrElse(u) + lookupTempView(ident).foreach { _ => + u.failAnalysis(s"${ident.quoted} is a temp view not table.") + } + u case u @ UnresolvedTableOrView(ident) => - lookupTempView(ident) - .map(_ => ResolvedView(ident.asIdentifier, isTempView = true)) - .getOrElse(u) + lookupTempView(ident).map(_ => ResolvedView(ident.asIdentifier)).getOrElse(u) } def lookupTempView(identifier: Seq[String]): Option[LogicalPlan] = { @@ -816,6 +814,14 @@ class Analyzer( lookupV2Relation(u.multipartIdentifier) .map(v2Relation => i.copy(table = v2Relation)) .getOrElse(i) + + case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => + CatalogV2Util.loadRelation(u.catalog, u.tableName) + .map(rel => alter.copy(table = rel)) + .getOrElse(alter) + + case u: UnresolvedV2Relation => + CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) } /** @@ -882,7 +888,8 @@ class Analyzer( case u @ UnresolvedTable(identifier) => lookupTableOrView(identifier).map { - case v: ResolvedView => UnresolvedTableWithViewExists(v) + case v: ResolvedView => + u.failAnalysis(s"${v.identifier.quoted} is a view not table.") case table => table }.getOrElse(u) @@ -895,7 +902,7 @@ class Analyzer( case SessionCatalogAndIdentifier(catalog, ident) => CatalogV2Util.loadTable(catalog, ident).map { case v1Table: V1Table if v1Table.v1Table.tableType == CatalogTableType.VIEW => - ResolvedView(ident, isTempView = false) + ResolvedView(ident) case table => ResolvedTable(catalog.asTableCatalog, ident, table) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 624c25d95c704..d6fc1dc6ddc3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.TypeUtils -import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnPosition, UpdateColumnType} +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -87,20 +87,6 @@ trait CheckAnalysis extends PredicateHelper { } def checkAnalysis(plan: LogicalPlan): Unit = { - // Analysis that needs to be performed top down can be added here. - plan.foreach { - case p if p.analyzed => // Skip already analyzed sub-plans - - case alter: AlterTable => - alter.table match { - case u @ UnresolvedTableWithViewExists(view) if !view.isTempView => - u.failAnalysis("Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead") - case _ => - } - - case _ => // Analysis successful! - } - // We transform up and order the rules so as to catch the first possible failure instead // of the result of cascading resolution failures. plan.foreachUp { @@ -119,13 +105,23 @@ trait CheckAnalysis extends PredicateHelper { case u: UnresolvedRelation => u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") - case u: UnresolvedTableWithViewExists => - val viewKind = if (u.view.isTempView) { "temp view" } else { "view" } - u.failAnalysis(s"${u.view.identifier.quoted} is a $viewKind not a table.") - case InsertIntoStatement(u: UnresolvedRelation, _, _, _, _) => failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") + case u: UnresolvedV2Relation if isView(u.originalNameParts) => + u.failAnalysis( + s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") + + case u: UnresolvedV2Relation => + u.failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + + case AlterTable(_, _, u: UnresolvedV2Relation, _) if isView(u.originalNameParts) => + u.failAnalysis( + s"Invalid command: '${u.originalNameParts.quoted}' is a view not a table.") + + case AlterTable(_, _, u: UnresolvedV2Relation, _) => + failAnalysis(s"Table not found: ${u.originalNameParts.quoted}") + case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. // If the arguments of the higher-order functions are resolved but the type check fails, @@ -429,9 +425,8 @@ trait CheckAnalysis extends PredicateHelper { case _ => } - case alter: AlterTable - if alter.childrenResolved && alter.table.isInstanceOf[ResolvedTable] => - val table = alter.table.asInstanceOf[ResolvedTable].table + case alter: AlterTable if alter.childrenResolved => + val table = alter.table def findField(operation: String, fieldName: Array[String]): StructField = { // include collections because structs nested in maps and arrays may be altered val field = table.schema.findNestedField(fieldName, includeCollections = true) @@ -484,8 +479,6 @@ trait CheckAnalysis extends PredicateHelper { throw new AnalysisException( s"Cannot change nullable column to non-nullable: $fieldName") } - case update: UpdateColumnPosition => - findField("update", update.fieldNames) case rename: RenameColumn => findField("rename", rename.fieldNames) case update: UpdateColumnComment => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index a44877fc1b4dd..88a3c0a73a10b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange} /** * Resolves catalogs from the multi-part identifiers in SQL statements, and convert the statements @@ -32,6 +32,71 @@ class ResolveCatalogs(val catalogManager: CatalogManager) import org.apache.spark.sql.connector.catalog.CatalogV2Util._ override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case AlterTableAddColumnsStatement( + nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => + val changes = cols.map { col => + TableChange.addColumn( + col.name.toArray, + col.dataType, + col.nullable, + col.comment.orNull, + col.position.orNull) + } + createAlterTable(nameParts, catalog, tbl, changes) + + case a @ AlterTableAlterColumnStatement( + nameParts @ NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => + val colName = a.column.toArray + val typeChange = a.dataType.map { newDataType => + TableChange.updateColumnType(colName, newDataType) + } + val nullabilityChange = a.nullable.map { nullable => + TableChange.updateColumnNullability(colName, nullable) + } + val commentChange = a.comment.map { newComment => + TableChange.updateColumnComment(colName, newComment) + } + val positionChange = a.position.map { newPosition => + TableChange.updateColumnPosition(colName, newPosition) + } + createAlterTable( + nameParts, + catalog, + tbl, + typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange) + + case AlterTableRenameColumnStatement( + nameParts @ NonSessionCatalogAndTable(catalog, tbl), col, newName) => + val changes = Seq(TableChange.renameColumn(col.toArray, newName)) + createAlterTable(nameParts, catalog, tbl, changes) + + case AlterTableDropColumnsStatement( + nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => + val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) + createAlterTable(nameParts, catalog, tbl, changes) + + case AlterTableSetPropertiesStatement( + nameParts @ NonSessionCatalogAndTable(catalog, tbl), props) => + val changes = props.map { case (key, value) => + TableChange.setProperty(key, value) + }.toSeq + createAlterTable(nameParts, catalog, tbl, changes) + + // TODO: v2 `UNSET TBLPROPERTIES` should respect the ifExists flag. + case AlterTableUnsetPropertiesStatement( + nameParts @ NonSessionCatalogAndTable(catalog, tbl), keys, _) => + val changes = keys.map(key => TableChange.removeProperty(key)) + createAlterTable(nameParts, catalog, tbl, changes) + + case AlterTableSetLocationStatement( + nameParts @ NonSessionCatalogAndTable(catalog, tbl), partitionSpec, newLoc) => + if (partitionSpec.nonEmpty) { + throw new AnalysisException( + "ALTER TABLE SET LOCATION does not support partition for v2 tables.") + } + val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, newLoc)) + createAlterTable(nameParts, catalog, tbl, changes) + case AlterViewSetPropertiesStatement( NonSessionCatalogAndTable(catalog, tbl), props) => throw new AnalysisException( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index c7d977a3d4a82..608f39c2d86fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.parser.ParserUtils import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.types.{DataType, Metadata, StructType} /** @@ -59,6 +60,28 @@ object UnresolvedRelation { UnresolvedRelation(tableIdentifier.database.toSeq :+ tableIdentifier.table) } +/** + * A variant of [[UnresolvedRelation]] which can only be resolved to a v2 relation + * (`DataSourceV2Relation`), not v1 relation or temp view. + * + * @param originalNameParts the original table identifier name parts before catalog is resolved. + * @param catalog The catalog which the table should be looked up from. + * @param tableName The name of the table to look up. + */ +case class UnresolvedV2Relation( + originalNameParts: Seq[String], + catalog: TableCatalog, + tableName: Identifier) + extends LeafNode with NamedRelation { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override def name: String = originalNameParts.quoted + + override def output: Seq[Attribute] = Nil + + override lazy val resolved = false +} + /** * An inline table that has not been resolved yet. Once resolved, it is turned by the analyzer into * a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]]. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index c6300b0bb079d..239f987e97a76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces, Table, TableCatalog} /** @@ -41,16 +41,6 @@ case class UnresolvedTable(multipartIdentifier: Seq[String]) extends LeafNode { override def output: Seq[Attribute] = Nil } -/** - * Holds the resolved view. It is used in a scenario where table is expected but the identifier - * is resolved to a (temp) view. - */ -case class UnresolvedTableWithViewExists(view: ResolvedView) extends LeafNode { - override lazy val resolved: Boolean = false - - override def output: Seq[Attribute] = Nil -} - /** * Holds the name of a table or view that has yet to be looked up in a catalog. It will * be resolved to [[ResolvedTable]] or [[ResolvedView]] during analysis. @@ -81,6 +71,6 @@ case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: T */ // TODO: create a generic representation for temp view, v1 view and v2 view, after we add view // support to v2 catalog. For now we only need the identifier to fallback to v1 command. -case class ResolvedView(identifier: Identifier, isTempView: Boolean) extends LeafNode { +case class ResolvedView(identifier: Identifier) extends LeafNode { override def output: Seq[Attribute] = Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index a8a96f0f6803a..e1dca4e945397 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2908,7 +2908,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableAddColumns]] command. + * Parse a [[AlterTableAddColumnsStatement]] command. * * For example: * {{{ @@ -2917,14 +2917,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * }}} */ override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) { - AlterTableAddColumns( - UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)), + AlterTableAddColumnsStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), ctx.columns.qualifiedColTypeWithPosition.asScala.map(typedVisit[QualifiedColType]) ) } /** - * Parse a [[AlterTableRenameColumn]] command. + * Parse a [[AlterTableRenameColumnStatement]] command. * * For example: * {{{ @@ -2933,14 +2933,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitRenameTableColumn( ctx: RenameTableColumnContext): LogicalPlan = withOrigin(ctx) { - AlterTableRenameColumn( - UnresolvedTable(visitMultipartIdentifier(ctx.table)), + AlterTableRenameColumnStatement( + visitMultipartIdentifier(ctx.table), ctx.from.parts.asScala.map(_.getText), ctx.to.getText) } /** - * Parse a [[AlterTableAlterColumn]] command. + * Parse a [[AlterTableAlterColumnStatement]] command. * * For example: * {{{ @@ -2957,8 +2957,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging s"ALTER TABLE table $verb COLUMN requires a TYPE or a COMMENT or a FIRST/AFTER", ctx) } - AlterTableAlterColumn( - UnresolvedTable(visitMultipartIdentifier(ctx.table)), + AlterTableAlterColumnStatement( + visitMultipartIdentifier(ctx.table), typedVisit[Seq[String]](ctx.column), dataType = Option(ctx.dataType).map(typedVisit[DataType]), nullable = None, @@ -2967,7 +2967,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableAlterColumn]] command to change column nullability. + * Parse a [[AlterTableAlterColumnStatement]] command to change column nullability. * * For example: * {{{ @@ -2981,8 +2981,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case SqlBaseParser.SET => false case SqlBaseParser.DROP => true } - AlterTableAlterColumn( - UnresolvedTable(visitMultipartIdentifier(ctx.table)), + AlterTableAlterColumnStatement( + visitMultipartIdentifier(ctx.table), typedVisit[Seq[String]](ctx.column), dataType = None, nullable = Some(nullable), @@ -2992,7 +2992,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableAlterColumn]] command. This is Hive SQL syntax. + * Parse a [[AlterTableAlterColumnStatement]] command. This is Hive SQL syntax. * * For example: * {{{ @@ -3015,8 +3015,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "please run ALTER COLUMN ... SET/DROP NOT NULL instead.") } - AlterTableAlterColumn( - UnresolvedTable(typedVisit[Seq[String]](ctx.table)), + AlterTableAlterColumnStatement( + typedVisit[Seq[String]](ctx.table), columnNameParts, dataType = Option(ctx.colType().dataType()).map(typedVisit[DataType]), nullable = None, @@ -3025,7 +3025,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableDropColumns]] command. + * Parse a [[AlterTableDropColumnsStatement]] command. * * For example: * {{{ @@ -3036,13 +3036,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitDropTableColumns( ctx: DropTableColumnsContext): LogicalPlan = withOrigin(ctx) { val columnsToDrop = ctx.columns.multipartIdentifier.asScala.map(typedVisit[Seq[String]]) - AlterTableDropColumns( - UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)), + AlterTableDropColumnsStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), columnsToDrop) } /** - * Parse [[AlterViewSetPropertiesStatement]] or [[AlterTableSetProperties]] commands. + * Parse [[AlterViewSetPropertiesStatement]] or [[AlterTableSetPropertiesStatement]] commands. * * For example: * {{{ @@ -3058,12 +3058,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (ctx.VIEW != null) { AlterViewSetPropertiesStatement(identifier, cleanedTableProperties) } else { - AlterTableSetProperties(UnresolvedTable(identifier), cleanedTableProperties) + AlterTableSetPropertiesStatement(identifier, cleanedTableProperties) } } /** - * Parse [[AlterViewUnsetPropertiesStatement]] or [[AlterTableUnsetProperties]] commands. + * Parse [[AlterViewUnsetPropertiesStatement]] or [[AlterTableUnsetPropertiesStatement]] commands. * * For example: * {{{ @@ -3081,12 +3081,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging if (ctx.VIEW != null) { AlterViewUnsetPropertiesStatement(identifier, cleanedProperties, ifExists) } else { - AlterTableUnsetProperties(UnresolvedTable(identifier), cleanedProperties, ifExists) + AlterTableUnsetPropertiesStatement(identifier, cleanedProperties, ifExists) } } /** - * Create an [[AlterTableSetLocation]] command. + * Create an [[AlterTableSetLocationStatement]] command. * * For example: * {{{ @@ -3094,8 +3094,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * }}} */ override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetLocation( - UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier)), + AlterTableSetLocationStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), visitLocationSpec(ctx.locationSpec)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 2083a00cae0da..44f7b4143926d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -149,6 +149,62 @@ case class QualifiedColType( comment: Option[String], position: Option[ColumnPosition]) +/** + * ALTER TABLE ... ADD COLUMNS command, as parsed from SQL. + */ +case class AlterTableAddColumnsStatement( + tableName: Seq[String], + columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement + +/** + * ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL. + */ +case class AlterTableAlterColumnStatement( + tableName: Seq[String], + column: Seq[String], + dataType: Option[DataType], + nullable: Option[Boolean], + comment: Option[String], + position: Option[ColumnPosition]) extends ParsedStatement + +/** + * ALTER TABLE ... RENAME COLUMN command, as parsed from SQL. + */ +case class AlterTableRenameColumnStatement( + tableName: Seq[String], + column: Seq[String], + newName: String) extends ParsedStatement + +/** + * ALTER TABLE ... DROP COLUMNS command, as parsed from SQL. + */ +case class AlterTableDropColumnsStatement( + tableName: Seq[String], + columnsToDrop: Seq[Seq[String]]) extends ParsedStatement + +/** + * ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterTableSetPropertiesStatement( + tableName: Seq[String], + properties: Map[String, String]) extends ParsedStatement + +/** + * ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterTableUnsetPropertiesStatement( + tableName: Seq[String], + propertyKeys: Seq[String], + ifExists: Boolean) extends ParsedStatement + +/** + * ALTER TABLE ... SET LOCATION command, as parsed from SQL. + */ +case class AlterTableSetLocationStatement( + tableName: Seq[String], + partitionSpec: Option[TablePartitionSpec], + location: String) extends ParsedStatement + /** * ALTER TABLE ... RECOVER PARTITIONS command, as parsed from SQL. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 3e3c81c22b61d..c04e56355a68f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Unevaluable} import org.apache.spark.sql.catalyst.plans.DescribeTableSchema import org.apache.spark.sql.connector.catalog._ -import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, ColumnChange} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.{DataType, MetadataBuilder, StringType, StructType} @@ -384,125 +384,37 @@ case class DropTable( ifExists: Boolean) extends Command /** - * The base class for ALTER TABLE commands that work for v2 tables. + * The logical plan of the ALTER TABLE command that works for v2 tables. */ -abstract class AlterTable extends Command { - def table: LogicalPlan - - def changes: Seq[TableChange] - - override def children: Seq[LogicalPlan] = Seq(table) +case class AlterTable( + catalog: TableCatalog, + ident: Identifier, + table: NamedRelation, + changes: Seq[TableChange]) extends Command { + + override lazy val resolved: Boolean = table.resolved && { + changes.forall { + case add: AddColumn => + add.fieldNames match { + case Array(_) => + // a top-level field can always be added + true + case _ => + // the parent field must exist + table.schema.findNestedField(add.fieldNames.init, includeCollections = true).isDefined + } - override lazy val resolved: Boolean = table.resolved -} + case colChange: ColumnChange => + // the column that will be changed must exist + table.schema.findNestedField(colChange.fieldNames, includeCollections = true).isDefined -/** - * The logical plan of the ALTER TABLE ... ADD COLUMNS command that works for v2 tables. - */ -case class AlterTableAddColumns( - table: LogicalPlan, - columnsToAdd: Seq[QualifiedColType]) extends AlterTable { - override lazy val changes: Seq[TableChange] = { - columnsToAdd.map { col => - TableChange.addColumn( - col.name.toArray, - col.dataType, - col.nullable, - col.comment.orNull, - col.position.orNull) + case _ => + // property changes require no resolution checks + true } } } -/** - * The logical plan of the ALTER TABLE ... CHANGE COLUMN command that works for v2 tables. - */ -case class AlterTableAlterColumn( - table: LogicalPlan, - column: Seq[String], - dataType: Option[DataType], - nullable: Option[Boolean], - comment: Option[String], - position: Option[ColumnPosition]) extends AlterTable { - override lazy val changes: Seq[TableChange] = { - val colName = column.toArray - val typeChange = dataType.map { newDataType => - TableChange.updateColumnType(colName, newDataType) - } - val nullabilityChange = nullable.map { nullable => - TableChange.updateColumnNullability(colName, nullable) - } - val commentChange = comment.map { newComment => - TableChange.updateColumnComment(colName, newComment) - } - val positionChange = position.map { newPosition => - TableChange.updateColumnPosition(colName, newPosition) - } - typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange - } -} - -/** - * The logical plan of the ALTER TABLE ... RENAME COLUMN command that works for v2 tables. - */ -case class AlterTableRenameColumn( - table: LogicalPlan, - column: Seq[String], - newName: String) extends AlterTable { - override lazy val changes: Seq[TableChange] = { - Seq(TableChange.renameColumn(column.toArray, newName)) - } -} - -/** - * The logical plan of the ALTER TABLE ... DROP COLUMNS command that works for v2 tables. - */ -case class AlterTableDropColumns( - table: LogicalPlan, - columnsToDrop: Seq[Seq[String]]) extends AlterTable { - override lazy val changes: Seq[TableChange] = { - columnsToDrop.map(col => TableChange.deleteColumn(col.toArray)) - } -} - -/** - * The logical plan of the ALTER TABLE ... SET TBLPROPERTIES command that works for v2 tables. - */ -case class AlterTableSetProperties( - table: LogicalPlan, - properties: Map[String, String]) extends AlterTable { - override lazy val changes: Seq[TableChange] = { - properties.map { case (key, value) => - TableChange.setProperty(key, value) - }.toSeq - } -} - -/** - * The logical plan of the ALTER TABLE ... UNSET TBLPROPERTIES command that works for v2 tables. - */ -// TODO: v2 `UNSET TBLPROPERTIES` should respect the ifExists flag. -case class AlterTableUnsetProperties( - table: LogicalPlan, - propertyKeys: Seq[String], - ifExists: Boolean) extends AlterTable { - override lazy val changes: Seq[TableChange] = { - propertyKeys.map(key => TableChange.removeProperty(key)) - } -} - -/** - * The logical plan of the ALTER TABLE ... SET LOCATION command that works for v2 tables. - */ -case class AlterTableSetLocation( - table: LogicalPlan, - partitionSpec: Option[TablePartitionSpec], - location: String) extends AlterTable { - override lazy val changes: Seq[TableChange] = { - Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, location)) - } -} - /** * The logical plan of the ALTER TABLE RENAME command that works for v2 tables. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index a4c7b4c3a2894..67726c7343524 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -22,7 +22,8 @@ import java.util.Collections import scala.collection.JavaConverters._ -import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException} +import org.apache.spark.sql.catalyst.analysis.{NamedRelation, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.plans.logical.AlterTable import org.apache.spark.sql.connector.catalog.TableChange._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} @@ -280,6 +281,17 @@ private[sql] object CatalogV2Util { properties ++ Map(TableCatalog.PROP_OWNER -> Utils.getCurrentUserName()) } + def createAlterTable( + originalNameParts: Seq[String], + catalog: CatalogPlugin, + tableName: Seq[String], + changes: Seq[TableChange]): AlterTable = { + val tableCatalog = catalog.asTableCatalog + val ident = tableName.asIdentifier + val unresolved = UnresolvedV2Relation(originalNameParts, tableCatalog, ident) + AlterTable(tableCatalog, ident, unresolved, changes) + } + def getTableProviderCatalog( provider: SupportsCatalogOptions, catalogManager: CatalogManager, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 0bcfccdd8b90a..56d52571d1cc3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -476,22 +476,22 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql1_table), - AlterTableSetProperties( - UnresolvedTable(Seq("table_name")), Map("test" -> "test", "comment" -> "new_comment"))) + AlterTableSetPropertiesStatement( + Seq("table_name"), Map("test" -> "test", "comment" -> "new_comment"))) comparePlans( parsePlan(sql2_table), - AlterTableUnsetProperties( - UnresolvedTable(Seq("table_name")), Seq("comment", "test"), ifExists = false)) + AlterTableUnsetPropertiesStatement( + Seq("table_name"), Seq("comment", "test"), ifExists = false)) comparePlans( parsePlan(sql3_table), - AlterTableUnsetProperties( - UnresolvedTable(Seq("table_name")), Seq("comment", "test"), ifExists = true)) + AlterTableUnsetPropertiesStatement( + Seq("table_name"), Seq("comment", "test"), ifExists = true)) } test("alter table: add column") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, None) ))) } @@ -499,7 +499,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add multiple columns") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS x int, y string"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, None), QualifiedColType(Seq("y"), StringType, true, None, None) ))) @@ -508,7 +508,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with COLUMNS") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS x int"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, None) ))) } @@ -516,7 +516,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with COLUMNS (...)") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS (x int)"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, None) ))) } @@ -524,7 +524,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with COLUMNS (...) and COMMENT") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMNS (x int COMMENT 'doc')"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, true, Some("doc"), None) ))) } @@ -532,7 +532,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add non-nullable column") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int NOT NULL"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, false, None, None) ))) } @@ -540,7 +540,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with COMMENT") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int COMMENT 'doc'"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, true, Some("doc"), None) ))) } @@ -548,13 +548,13 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with position") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int FIRST"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, Some(first())) ))) comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int AFTER y"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x"), IntegerType, true, None, Some(after("y"))) ))) } @@ -562,7 +562,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add column with nested column name") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc'"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x", "y", "z"), IntegerType, true, Some("doc"), None) ))) } @@ -570,7 +570,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: add multiple columns with nested column name") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc', a.b string FIRST"), - AlterTableAddColumns(UnresolvedTable(Seq("table_name")), Seq( + AlterTableAddColumnsStatement(Seq("table_name"), Seq( QualifiedColType(Seq("x", "y", "z"), IntegerType, true, Some("doc"), None), QualifiedColType(Seq("a", "b"), StringType, true, None, Some(first())) ))) @@ -579,12 +579,12 @@ class DDLParserSuite extends AnalysisTest { test("alter table: set location") { comparePlans( parsePlan("ALTER TABLE a.b.c SET LOCATION 'new location'"), - AlterTableSetLocation(UnresolvedTable(Seq("a", "b", "c")), None, "new location")) + AlterTableSetLocationStatement(Seq("a", "b", "c"), None, "new location")) comparePlans( parsePlan("ALTER TABLE a.b.c PARTITION(ds='2017-06-10') SET LOCATION 'new location'"), - AlterTableSetLocation( - UnresolvedTable(Seq("a", "b", "c")), + AlterTableSetLocationStatement( + Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10")), "new location")) } @@ -592,8 +592,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: rename column") { comparePlans( parsePlan("ALTER TABLE table_name RENAME COLUMN a.b.c TO d"), - AlterTableRenameColumn( - UnresolvedTable(Seq("table_name")), + AlterTableRenameColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), "d")) } @@ -601,8 +601,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: update column type using ALTER") { comparePlans( parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint"), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), Some(LongType), None, @@ -613,8 +613,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: update column type") { comparePlans( parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c TYPE bigint"), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), Some(LongType), None, @@ -625,8 +625,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: update column comment") { comparePlans( parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c COMMENT 'new comment'"), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), None, None, @@ -637,8 +637,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: update column position") { comparePlans( parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c FIRST"), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), None, None, @@ -650,8 +650,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c " + "TYPE bigint COMMENT 'new comment' AFTER d"), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), Some(LongType), None, @@ -662,8 +662,8 @@ class DDLParserSuite extends AnalysisTest { test("alter table: SET/DROP NOT NULL") { comparePlans( parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c SET NOT NULL"), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), None, Some(false), @@ -672,8 +672,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c DROP NOT NULL"), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), None, Some(true), @@ -684,7 +684,7 @@ class DDLParserSuite extends AnalysisTest { test("alter table: drop column") { comparePlans( parsePlan("ALTER TABLE table_name DROP COLUMN a.b.c"), - AlterTableDropColumns(UnresolvedTable(Seq("table_name")), Seq(Seq("a", "b", "c")))) + AlterTableDropColumnsStatement(Seq("table_name"), Seq(Seq("a", "b", "c")))) } test("alter table: drop multiple columns") { @@ -692,8 +692,8 @@ class DDLParserSuite extends AnalysisTest { Seq(sql, sql.replace("COLUMN", "COLUMNS")).foreach { drop => comparePlans( parsePlan(drop), - AlterTableDropColumns( - UnresolvedTable(Seq("table_name")), + AlterTableDropColumnsStatement( + Seq("table_name"), Seq(Seq("x"), Seq("y"), Seq("a", "b", "c")))) } } @@ -705,8 +705,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql1), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), Some(IntegerType), None, @@ -715,8 +715,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql2), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), Some(IntegerType), None, @@ -725,8 +725,8 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan(sql3), - AlterTableAlterColumn( - UnresolvedTable(Seq("table_name")), + AlterTableAlterColumnStatement( + Seq("table_name"), Seq("a", "b", "c"), Some(IntegerType), None, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 0aaf9d7e2e1ac..8b0d339dbb864 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} @@ -47,63 +47,141 @@ class ResolveSessionCatalog( import org.apache.spark.sql.connector.catalog.CatalogV2Util._ override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case AlterTableAddColumns(ResolvedTable(_, ident, _: V1Table), cols) => - cols.foreach { c => - assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand") - if (!c.nullable) { - throw new AnalysisException( - "ADD COLUMN with v1 tables cannot specify NOT NULL.") + case AlterTableAddColumnsStatement( + nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => + loadTable(catalog, tbl.asIdentifier).collect { + case v1Table: V1Table => + cols.foreach { c => + assertTopLevelColumn(c.name, "AlterTableAddColumnsCommand") + if (!c.nullable) { + throw new AnalysisException( + "ADD COLUMN with v1 tables cannot specify NOT NULL.") + } + } + AlterTableAddColumnsCommand(tbl.asTableIdentifier, cols.map(convertToStructField)) + }.getOrElse { + val changes = cols.map { col => + TableChange.addColumn( + col.name.toArray, + col.dataType, + col.nullable, + col.comment.orNull, + col.position.orNull) } + createAlterTable(nameParts, catalog, tbl, changes) } - AlterTableAddColumnsCommand(ident.asTableIdentifier, cols.map(convertToStructField)) - case a @ AlterTableAlterColumn(ResolvedTable(_, ident, _: V1Table), _, _, _, _, _) => - if (a.column.length > 1) { - throw new AnalysisException( - "ALTER COLUMN with qualified column is only supported with v2 tables.") - } - if (a.dataType.isEmpty) { - throw new AnalysisException( - "ALTER COLUMN with v1 tables must specify new data type.") - } - if (a.nullable.isDefined) { - throw new AnalysisException( - "ALTER COLUMN with v1 tables cannot specify NOT NULL.") - } - if (a.position.isDefined) { - throw new AnalysisException("" + - "ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.") - } - val builder = new MetadataBuilder - // Add comment to metadata - a.comment.map(c => builder.putString("comment", c)) - // Add Hive type string to metadata. - val cleanedDataType = HiveStringType.replaceCharType(a.dataType.get) - if (a.dataType.get != cleanedDataType) { - builder.putString(HIVE_TYPE_STRING, a.dataType.get.catalogString) + case a @ AlterTableAlterColumnStatement( + nameParts @ SessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => + loadTable(catalog, tbl.asIdentifier).collect { + case v1Table: V1Table => + if (a.column.length > 1) { + throw new AnalysisException( + "ALTER COLUMN with qualified column is only supported with v2 tables.") + } + if (a.dataType.isEmpty) { + throw new AnalysisException( + "ALTER COLUMN with v1 tables must specify new data type.") + } + if (a.nullable.isDefined) { + throw new AnalysisException( + "ALTER COLUMN with v1 tables cannot specify NOT NULL.") + } + if (a.position.isDefined) { + throw new AnalysisException("" + + "ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.") + } + val builder = new MetadataBuilder + // Add comment to metadata + a.comment.map(c => builder.putString("comment", c)) + // Add Hive type string to metadata. + val cleanedDataType = HiveStringType.replaceCharType(a.dataType.get) + if (a.dataType.get != cleanedDataType) { + builder.putString(HIVE_TYPE_STRING, a.dataType.get.catalogString) + } + val newColumn = StructField( + a.column(0), + cleanedDataType, + nullable = true, + builder.build()) + AlterTableChangeColumnCommand(tbl.asTableIdentifier, a.column(0), newColumn) + }.getOrElse { + val colName = a.column.toArray + val typeChange = a.dataType.map { newDataType => + TableChange.updateColumnType(colName, newDataType) + } + val nullabilityChange = a.nullable.map { nullable => + TableChange.updateColumnNullability(colName, nullable) + } + val commentChange = a.comment.map { newComment => + TableChange.updateColumnComment(colName, newComment) + } + val positionChange = a.position.map { newPosition => + TableChange.updateColumnPosition(colName, newPosition) + } + createAlterTable( + nameParts, + catalog, + tbl, + typeChange.toSeq ++ nullabilityChange ++ commentChange ++ positionChange) } - val newColumn = StructField( - a.column(0), - cleanedDataType, - nullable = true, - builder.build()) - AlterTableChangeColumnCommand(ident.asTableIdentifier, a.column(0), newColumn) - case AlterTableRenameColumn(ResolvedTable(_, _, _: V1Table), _, _) => - throw new AnalysisException("RENAME COLUMN is only supported with v2 tables.") + case AlterTableRenameColumnStatement( + nameParts @ SessionCatalogAndTable(catalog, tbl), col, newName) => + loadTable(catalog, tbl.asIdentifier).collect { + case v1Table: V1Table => + throw new AnalysisException("RENAME COLUMN is only supported with v2 tables.") + }.getOrElse { + val changes = Seq(TableChange.renameColumn(col.toArray, newName)) + createAlterTable(nameParts, catalog, tbl, changes) + } - case AlterTableDropColumns(ResolvedTable(_, _, _: V1Table), _) => - throw new AnalysisException("DROP COLUMN is only supported with v2 tables.") + case AlterTableDropColumnsStatement( + nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => + loadTable(catalog, tbl.asIdentifier).collect { + case v1Table: V1Table => + throw new AnalysisException("DROP COLUMN is only supported with v2 tables.") + }.getOrElse { + val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) + createAlterTable(nameParts, catalog, tbl, changes) + } - case AlterTableSetProperties(ResolvedTable(_, ident, _: V1Table), props) => - AlterTableSetPropertiesCommand(ident.asTableIdentifier, props, isView = false) + case AlterTableSetPropertiesStatement( + nameParts @ SessionCatalogAndTable(catalog, tbl), props) => + loadTable(catalog, tbl.asIdentifier).collect { + case v1Table: V1Table => + AlterTableSetPropertiesCommand(tbl.asTableIdentifier, props, isView = false) + }.getOrElse { + val changes = props.map { case (key, value) => + TableChange.setProperty(key, value) + }.toSeq + createAlterTable(nameParts, catalog, tbl, changes) + } - case AlterTableUnsetProperties(ResolvedTable(_, ident, _: V1Table), keys, ifExists) => - AlterTableUnsetPropertiesCommand(ident.asTableIdentifier, keys, ifExists, isView = false) + case AlterTableUnsetPropertiesStatement( + nameParts @ SessionCatalogAndTable(catalog, tbl), keys, ifExists) => + loadTable(catalog, tbl.asIdentifier).collect { + case v1Table: V1Table => + AlterTableUnsetPropertiesCommand( + tbl.asTableIdentifier, keys, ifExists, isView = false) + }.getOrElse { + val changes = keys.map(key => TableChange.removeProperty(key)) + createAlterTable(nameParts, catalog, tbl, changes) + } - case AlterTableSetLocation( - ResolvedTable(_, ident, _: V1Table), partitionSpec, newLoc) => - AlterTableSetLocationCommand(ident.asTableIdentifier, partitionSpec, newLoc) + case AlterTableSetLocationStatement( + nameParts @ SessionCatalogAndTable(catalog, tbl), partitionSpec, newLoc) => + loadTable(catalog, tbl.asIdentifier).collect { + case v1Table: V1Table => + AlterTableSetLocationCommand(tbl.asTableIdentifier, partitionSpec, newLoc) + }.getOrElse { + if (partitionSpec.nonEmpty) { + throw new AnalysisException( + "ALTER TABLE SET LOCATION does not support partition for v2 tables.") + } + val changes = Seq(TableChange.setProperty(TableCatalog.PROP_LOCATION, newLoc)) + createAlterTable(nameParts, catalog, tbl, changes) + } // ALTER VIEW should always use v1 command if the resolved catalog is session catalog. case AlterViewSetPropertiesStatement(SessionCatalogAndTable(_, tbl), props) => @@ -140,7 +218,7 @@ class ResolveSessionCatalog( DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. - case DescribeRelation(ResolvedView(ident, _), partitionSpec, isExtended) => + case DescribeRelation(ResolvedView(ident), partitionSpec, isExtended) => DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) case DescribeColumnStatement( 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 ab47d640f4705..a92fbdf25975b 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 @@ -257,6 +257,14 @@ case class AlterTableAddColumnsCommand( table: TableIdentifier): CatalogTable = { val catalogTable = catalog.getTempViewOrPermanentTableMetadata(table) + if (catalogTable.tableType == CatalogTableType.VIEW) { + throw new AnalysisException( + s""" + |ALTER ADD COLUMNS does not support views. + |You must drop and re-create the views for adding the new columns. Views: $table + """.stripMargin) + } + if (DDLUtils.isDatasourceTable(catalogTable)) { DataSource.lookupDataSource(catalogTable.provider.get, conf). getConstructor().newInstance() match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 7c8fd4e105ca7..448a4354ddd66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -241,18 +241,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil - case a @ AlterTableSetLocation(r: ResolvedTable, partitionSpec, _) => - if (partitionSpec.nonEmpty) { - throw new AnalysisException( - "ALTER TABLE SET LOCATION does not support partition for v2 tables.") - } - AlterTableExec(r.catalog, r.identifier, a.changes) :: Nil - - case a: AlterTable => - a.table match { - case r: ResolvedTable => AlterTableExec(r.catalog, r.identifier, a.changes) :: Nil - case _ => Nil - } + case AlterTable(catalog, ident, _, changes) => + AlterTableExec(catalog, ident, changes) :: Nil case RenameTable(catalog, oldIdent, newIdent) => RenameTableExec(catalog, oldIdent, newIdent) :: Nil diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index bb5b4ae84d3b7..82326346b361c 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -195,7 +195,7 @@ ALTER TABLE temp_view CHANGE a TYPE INT COMMENT 'this is column a' struct<> -- !query 20 output org.apache.spark.sql.AnalysisException -temp_view is a temp view not a table.; line 1 pos 0 +Invalid command: 'temp_view' is a view not a table.; line 1 pos 0 -- !query 21 @@ -212,7 +212,7 @@ ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT COMMENT 'this is colu struct<> -- !query 22 output org.apache.spark.sql.AnalysisException -global_temp.global_temp_view is a temp view not a table.; line 1 pos 0 +Invalid command: 'global_temp.global_temp_view' is a view not a table.; line 1 pos 0 -- !query 23 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 75c9bb7be05f4..c19352a2267df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -2201,7 +2201,7 @@ class DataSourceV2SQLSuite withTempView("v") { sql("create global temp view v as select 1") val e = intercept[AnalysisException](sql("COMMENT ON TABLE global_temp.v IS NULL")) - assert(e.getMessage.contains("global_temp.v is a temp view not a table.")) + assert(e.getMessage.contains("global_temp.v is a temp view not table.")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 4ed2506b35a80..9a393f19ce9bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -145,13 +145,13 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { // For v2 ALTER TABLE statements, we have better error message saying view is not supported. assertAnalysisError( s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'", - s"$viewName is a temp view not a table") + s"'$viewName' is a view not a table") - // For the following v2 ALERT TABLE statements, relations are first resolved before - // unsupported operations are checked. + // For the following v2 ALERT TABLE statements, unsupported operations are checked first + // before resolving the relations. assertAnalysisError( s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'", - s"$viewName is a temp view not a table") + "ALTER TABLE SET LOCATION does not support partition for v2 tables") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index e3fb535ab4cdd..1a9fe46bd6a91 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2779,7 +2779,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)") } - assert(e.message.contains("tmp_v is a temp view not a table")) + assert(e.message.contains("'tmp_v' is a view not a table")) } } @@ -2789,8 +2789,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val e = intercept[AnalysisException] { sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)") } - assert(e.message.contains( - "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + assert(e.message.contains("ALTER ADD COLUMNS does not support views")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 35b2003e952e1..8f17ce7f32c82 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -26,7 +26,7 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolvedTable, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -140,7 +140,6 @@ class PlanResolutionSuite extends AnalysisTest { val rules = Seq( CTESubstitution, analyzer.ResolveRelations, - analyzer.ResolveTables, new ResolveCatalogs(catalogManager), new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), analyzer.ResolveTables, @@ -724,24 +723,24 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed3, expected3) } else { parsed1 match { - case a: AlterTable => - assert(a.changes == Seq( + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( TableChange.setProperty("test", "test"), TableChange.setProperty("comment", "new_comment"))) case _ => fail("expect AlterTable") } parsed2 match { - case a: AlterTable => - assert(a.changes == Seq( + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( TableChange.removeProperty("comment"), TableChange.removeProperty("test"))) case _ => fail("expect AlterTable") } parsed3 match { - case a: AlterTable => - assert(a.changes == Seq( + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( TableChange.removeProperty("comment"), TableChange.removeProperty("test"))) case _ => fail("expect AlterTable") @@ -754,9 +753,15 @@ class PlanResolutionSuite extends AnalysisTest { val parsed4 = parseAndResolve(sql4) val parsed5 = parseAndResolve(sql5) - // For non-existing tables, we expect `UnresolvedTable` in the resolved plan. - assert(parsed4.collect{ case u: UnresolvedTable => u }.length == 1) - assert(parsed5.collect{ case u: UnresolvedTable => u }.length == 1) + // For non-existing tables, we convert it to v2 command with `UnresolvedV2Table` + parsed4 match { + case AlterTable(_, _, _: UnresolvedV2Relation, _) => // OK + case _ => fail("Expect AlterTable, but got:\n" + parsed4.treeString) + } + parsed5 match { + case AlterTable(_, _, _: UnresolvedV2Relation, _) => // OK + case _ => fail("Expect AlterTable, but got:\n" + parsed5.treeString) + } } test("support for other types in TBLPROPERTIES") { @@ -777,8 +782,8 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed, expected) } else { parsed match { - case a: AlterTable => - assert(a.changes == Seq( + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( TableChange.setProperty("a", "1"), TableChange.setProperty("b", "0.1"), TableChange.setProperty("c", "true"))) @@ -801,8 +806,8 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed, expected) } else { parsed match { - case a: AlterTable => - assert(a.changes == Seq(TableChange.setProperty("location", "new location"))) + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq(TableChange.setProperty("location", "new location"))) case _ => fail("Expect AlterTable, but got:\n" + parsed.treeString) } } @@ -1043,23 +1048,23 @@ class PlanResolutionSuite extends AnalysisTest { val parsed3 = parseAndResolve(sql3) parsed1 match { - case a: AlterTable => - assert(a.changes == Seq( + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( TableChange.updateColumnType(Array("i"), LongType))) case _ => fail("expect AlterTable") } parsed2 match { - case a: AlterTable => - assert(a.changes == Seq( + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( TableChange.updateColumnType(Array("i"), LongType), TableChange.updateColumnComment(Array("i"), "new comment"))) case _ => fail("expect AlterTable") } parsed3 match { - case a: AlterTable => - assert(a.changes == Seq( + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes == Seq( TableChange.updateColumnComment(Array("i"), "new comment"))) case _ => fail("expect AlterTable") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 9b12ac1d79e73..dbbf2b29fe8b7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -158,7 +158,7 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto val message = intercept[AnalysisException] { sql("SHOW TBLPROPERTIES parquet_temp") }.getMessage - assert(message.contains("parquet_temp is a temp view not a table")) + assert(message.contains("parquet_temp is a temp view not table")) } } From cd9ccdc0aca588699e39b45e708bd87f6622031c Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 23 Jan 2020 16:00:21 +0900 Subject: [PATCH 1198/1387] [SPARK-30601][BUILD] Add a Google Maven Central as a primary repository ### What changes were proposed in this pull request? This PR proposes to address four things. Three issues and fixes were a bit mixed so this PR sorts it out. See also http://apache-spark-developers-list.1001551.n3.nabble.com/Adding-Maven-Central-mirror-from-Google-to-the-build-td28728.html for the discussion in the mailing list. 1. Add the Google Maven Central mirror (GCS) as a primary repository. This will not only help development more stable but also in order to make Github Actions build (where it is always required to download jars) stable. In case of Jenkins PR builder, it wouldn't be affected too much as it uses the pre-downloaded jars under `.m2`. - Google Maven Central seems stable for heavy workload but not synced very quickly (e.g., new release is missing) - Maven Central (default) seems less stable but synced quickly. We already added this GCS mirror as a default additional remote repository at SPARK-29175. So I don't see an issue to add it as a repo. https://github.com/apache/spark/blob/abf759a91e01497586b8bb6b7a314dd28fd6cff1/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L2111-L2118 2. Currently, we have the hard-corded repository in [`sbt-pom-reader`](https://github.com/JoshRosen/sbt-pom-reader/blob/v1.0.0-spark/src/main/scala/com/typesafe/sbt/pom/MavenPomResolver.scala#L32) and this seems overwriting Maven's existing resolver by the same ID `central` with `http://` when initially the pom file is ported into SBT instance. This uses `http://` which latently Maven Central disallowed (see https://github.com/apache/spark/pull/27242) My speculation is that we just need to be able to load plugin and let it convert POM to SBT instance with another fallback repo. After that, it _seems_ using `central` with `https` properly. See also https://github.com/apache/spark/pull/27307#issuecomment-576720395. I double checked that we use `https` properly from the SBT build as well: ``` [debug] downloading https://repo1.maven.org/maven2/com/etsy/sbt-checkstyle-plugin_2.10_0.13/3.1.1/sbt-checkstyle-plugin-3.1.1.pom ... [debug] public: downloading https://repo1.maven.org/maven2/com/etsy/sbt-checkstyle-plugin_2.10_0.13/3.1.1/sbt-checkstyle-plugin-3.1.1.pom [debug] public: downloading https://repo1.maven.org/maven2/com/etsy/sbt-checkstyle-plugin_2.10_0.13/3.1.1/sbt-checkstyle-plugin-3.1.1.pom.sha1 ``` This was fixed by adding the same repo (https://github.com/apache/spark/pull/27281), `central_without_mirror`, which is a bit awkward. Instead, this PR adds GCS as a main repo, and community Maven central as a fallback repo. So, presumably the community Maven central repo is used when the plugin is loaded as a fallback. 3. While I am here, I fix another issue. Github Action at https://github.com/apache/spark/pull/27279 is being failed. The reason seems to be scalafmt 1.0.3 is in Maven central but not in GCS. ``` org.apache.maven.plugin.PluginResolutionException: Plugin org.antipathy:mvn-scalafmt_2.12:1.0.3 or one of its dependencies could not be resolved: Could not find artifact org.antipathy:mvn-scalafmt_2.12:jar:1.0.3 in google-maven-central (https://maven-central.storage-download.googleapis.com/repos/central/data/) at org.apache.maven.plugin.internal.DefaultPluginDependenciesResolver.resolve (DefaultPluginDependenciesResolver.java:131) ``` `mvn-scalafmt` exists in Maven central: ```bash $ curl https://repo.maven.apache.org/maven2/org/antipathy/mvn-scalafmt_2.12/1.0.3/mvn-scalafmt_2.12-1.0.3.pom ``` ```xml 4.0.0 ... ``` whereas not in GCS mirror: ```bash $ curl https://maven-central.storage-download.googleapis.com/repos/central/data/org/antipathy/mvn-scalafmt_2.12/1.0.3/mvn-scalafmt_2.12-1.0.3.pom ``` ```xml NoSuchKeyThe specified key does not exist.
    No such object: maven-central/repos/central/data/org/antipathy/mvn-scalafmt_2.12/1.0.3/mvn-scalafmt_2.12-1.0.3.pom
    % ``` In this PR, simply make both repos accessible by adding to `pluginRepositories`. 4. Remove the workarounds in Github Actions to switch mirrors because now we have same repos in the same order (Google Maven Central first, and Maven Central second) ### Why are the changes needed? To make the build and Github Action more stable. ### Does this PR introduce any user-facing change? No, dev only change. ### How was this patch tested? I roughly checked local and PR against my fork (https://github.com/HyukjinKwon/spark/pull/2 and https://github.com/HyukjinKwon/spark/pull/3). Closes #27307 from HyukjinKwon/SPARK-30572. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .github/workflows/master.yml | 8 -------- pom.xml | 32 ++++++++++++++++++++++++-------- project/SparkBuild.scala | 3 +++ 3 files changed, 27 insertions(+), 16 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e4d2d8470cc34..d53119ad75599 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -66,14 +66,6 @@ jobs: export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN" export MAVEN_CLI_OPTS="--no-transfer-progress" mkdir -p ~/.m2 - # `Maven Central` is too flaky in terms of downloading artifacts in `GitHub Action` environment. - # `Google Maven Central Mirror` is too slow in terms of sycing upstream. To get the best combination, - # 1) we set `Google Maven Central` as a mirror of `central` in `GitHub Action` environment only. - # 2) we duplicates `Maven Central` in pom.xml with ID `central_without_mirror`. - # In other words, in GitHub Action environment, `central` is mirrored by `Google Maven Central` first. - # If `Google Maven Central` doesn't provide the artifact due to its slowness, `central_without_mirror` will be used. - # Note that we aim to achieve the above while keeping the existing behavior of non-`GitHub Action` environment unchanged. - echo "google-maven-centralGCS Maven Central mirrorhttps://maven-central.storage-download.googleapis.com/repos/central/data/central" > ~/.m2/settings.xml ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -P${{ matrix.hive }} -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install rm -rf ~/.m2/repository/org/apache/spark diff --git a/pom.xml b/pom.xml index d2b2b0205060f..4e6c571ee052c 100644 --- a/pom.xml +++ b/pom.xml @@ -246,10 +246,13 @@ - central - - Maven Repository - https://repo.maven.apache.org/maven2 + gcs-maven-central-mirror + + GCS Maven Central mirror + https://maven-central.storage-download.googleapis.com/repos/central/data/ true @@ -258,12 +261,10 @@ - central_without_mirror + central Maven Repository https://repo.maven.apache.org/maven2 @@ -275,6 +276,21 @@ + + gcs-maven-central-mirror + + GCS Maven Central mirror + https://maven-central.storage-download.googleapis.com/repos/central/data/ + + true + + + false + + central https://repo.maven.apache.org/maven2 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9385505ad8ac5..87e584bd28b68 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -224,6 +224,9 @@ object SparkBuild extends PomBuild { // Override SBT's default resolvers: resolvers := Seq( + // Google Mirror of Maven Central, placed first so that it's used instead of flaky Maven Central. + // See https://storage-download.googleapis.com/maven-central/index.html for more info. + "gcs-maven-central-mirror" at "https://maven-central.storage-download.googleapis.com/repos/central/data/", DefaultMavenRepository, Resolver.mavenLocal, Resolver.file("local", file(Path.userHome.absolutePath + "/.ivy2/local"))(Resolver.ivyStylePatterns) From 2330a5682d376c21b73dbdf5ea10e253941e8cc8 Mon Sep 17 00:00:00 2001 From: zero323 Date: Thu, 23 Jan 2020 16:16:47 +0900 Subject: [PATCH 1199/1387] [SPARK-30607][SQL][PYSPARK][SPARKR] Add overlay wrappers for SparkR and PySpark ### What changes were proposed in this pull request? This PR adds: - `pyspark.sql.functions.overlay` function to PySpark - `overlay` function to SparkR ### Why are the changes needed? Feature parity. At the moment R and Python users can access this function only using SQL or `expr` / `selectExpr`. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? New unit tests. Closes #27325 from zero323/SPARK-30607. Authored-by: zero323 Signed-off-by: HyukjinKwon --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 39 ++++++++++++++++++++-- R/pkg/R/generics.R | 4 +++ R/pkg/tests/fulltests/test_sparkSQL.R | 2 ++ python/pyspark/sql/functions.py | 34 +++++++++++++++++++ python/pyspark/sql/tests/test_functions.py | 27 +++++++++++++++ 6 files changed, 105 insertions(+), 2 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index f9d9494ca6fa1..7ed2e36d59531 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -335,6 +335,7 @@ exportMethods("%<=>%", "ntile", "otherwise", "over", + "overlay", "percent_rank", "pmod", "posexplode", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index e340b0604e983..af3b18620e0d7 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -136,6 +136,14 @@ NULL #' format to. See 'Details'. #' } #' @param y Column to compute on. +#' @param pos In \itemize{ +#' \item \code{locate}: a start position of search. +#' \item \code{overlay}: a start postiton for replacement. +#' } +#' @param len In \itemize{ +#' \item \code{lpad} the maximum length of each output result. +#' \item \code{overlay} a number of bytes to replace. +#' } #' @param ... additional Columns. #' @name column_string_functions #' @rdname column_string_functions @@ -1319,6 +1327,35 @@ setMethod("negate", column(jc) }) +#' @details +#' \code{overlay}: Overlay the specified portion of \code{x} with \code{replace}, +#' starting from byte position \code{pos} of \code{src} and proceeding for +#' \code{len} bytes. +#' +#' @param replace a Column with replacement. +#' +#' @rdname column_string_functions +#' @aliases overlay overlay,Column-method,numericOrColumn-method +#' @note overlay since 3.0.0 +setMethod("overlay", + signature(x = "Column", replace = "Column", pos = "numericOrColumn"), + function(x, replace, pos, len = -1) { + if (is.numeric(pos)) { + pos <- lit(as.integer(pos)) + } + + if (is.numeric(len)) { + len <- lit(as.integer(len)) + } + + jc <- callJStatic( + "org.apache.spark.sql.functions", "overlay", + x@jc, replace@jc, pos@jc, len@jc + ) + + column(jc) + }) + #' @details #' \code{quarter}: Extracts the quarter as an integer from a given date/timestamp/string. #' @@ -2819,7 +2856,6 @@ setMethod("window", signature(x = "Column"), #' #' @param substr a character string to be matched. #' @param str a Column where matches are sought for each entry. -#' @param pos start position of search. #' @rdname column_string_functions #' @aliases locate locate,character,Column-method #' @note locate since 1.5.0 @@ -2834,7 +2870,6 @@ setMethod("locate", signature(substr = "character", str = "Column"), #' @details #' \code{lpad}: Left-padded with pad to a length of len. #' -#' @param len maximum length of each output result. #' @param pad a character string to be padded with. #' @rdname column_string_functions #' @aliases lpad lpad,Column,numeric,character-method diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index f849dd172247c..4134d5cecc888 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1149,6 +1149,10 @@ setGeneric("ntile", function(x) { standardGeneric("ntile") }) #' @name NULL setGeneric("n_distinct", function(x, ...) { standardGeneric("n_distinct") }) +#' @rdname column_string_functions +#' @name NULL +setGeneric("overlay", function(x, replace, pos, ...) { standardGeneric("overlay") }) + #' @rdname column_window_functions #' @name NULL setGeneric("percent_rank", function(x = "missing") { standardGeneric("percent_rank") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index cb47353d600db..d435a8b6d7c4a 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1405,6 +1405,8 @@ test_that("column functions", { trunc(c, "month") + trunc(c, "mon") + trunc(c, "mm") c24 <- date_trunc("hour", c) + date_trunc("minute", c) + date_trunc("week", c) + date_trunc("quarter", c) + current_date() + current_timestamp() + c25 <- overlay(c1, c2, c3, c3) + overlay(c1, c2, c3) + overlay(c1, c2, 1) + + overlay(c1, c2, 3, 4) # Test if base::is.nan() is exposed expect_equal(is.nan(c("a", "b")), c(FALSE, FALSE)) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 176729eb518e3..7f4cda832972a 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1600,6 +1600,40 @@ def instr(str, substr): return Column(sc._jvm.functions.instr(_to_java_column(str), substr)) +@since(3.0) +def overlay(src, replace, pos, len=-1): + """ + Overlay the specified portion of `src` with `replace`, + starting from byte position `pos` of `src` and proceeding for `len` bytes. + + >>> df = spark.createDataFrame([("SPARK_SQL", "CORE")], ("x", "y")) + >>> df.select(overlay("x", "y", 7).alias("overlayed")).show() + +----------+ + | overlayed| + +----------+ + |SPARK_CORE| + +----------+ + """ + if not isinstance(pos, (int, str, Column)): + raise TypeError( + "pos should be an integer or a Column / column name, got {}".format(type(pos))) + if len is not None and not isinstance(len, (int, str, Column)): + raise TypeError( + "len should be an integer or a Column / column name, got {}".format(type(len))) + + pos = _create_column_from_literal(pos) if isinstance(pos, int) else _to_java_column(pos) + len = _create_column_from_literal(len) if isinstance(len, int) else _to_java_column(len) + + sc = SparkContext._active_spark_context + + return Column(sc._jvm.functions.overlay( + _to_java_column(src), + _to_java_column(replace), + pos, + len + )) + + @since(1.5) @ignore_unicode_prefix def substring(str, pos, len): diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 36e1e8a660f00..fa9ee57ff5f90 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -310,6 +310,33 @@ def test_input_file_name_udf(self): file_name = df.collect()[0].file self.assertTrue("python/test_support/hello/hello.txt" in file_name) + def test_overlay(self): + from pyspark.sql.functions import col, lit, overlay + from itertools import chain + import re + + actual = list(chain.from_iterable([ + re.findall("(overlay\\(.*\\))", str(x)) for x in [ + overlay(col("foo"), col("bar"), 1), + overlay("x", "y", 3), + overlay(col("x"), col("y"), 1, 3), + overlay("x", "y", 2, 5), + overlay("x", "y", lit(11)), + overlay("x", "y", lit(2), lit(5)), + ] + ])) + + expected = [ + "overlay(foo, bar, 1, -1)", + "overlay(x, y, 3, -1)", + "overlay(x, y, 1, 3)", + "overlay(x, y, 2, 5)", + "overlay(x, y, 11, -1)", + "overlay(x, y, 2, 5)", + ] + + self.assertListEqual(actual, expected) + if __name__ == "__main__": import unittest From f35f3520964b526a9e98dd56943ecaafcbb91be9 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 23 Jan 2020 16:44:13 +0800 Subject: [PATCH 1200/1387] [SPARK-30543][ML][PYSPARK][R] RandomForest add Param bootstrap to control sampling method ### What changes were proposed in this pull request? add a param `bootstrap` to control whether bootstrap samples are used. ### Why are the changes needed? Current RF with numTrees=1 will directly build a tree using the orignial dataset, while with numTrees>1 it will use bootstrap samples to build trees. This design is for training a DecisionTreeModel by the impl of RandomForest, however, it is somewhat strange. In Scikit-Learn, there is a param [bootstrap](https://scikit-learn.org/stable/modules/generated/sklearn.ensemble.RandomForestClassifier.html#sklearn.ensemble.RandomForestClassifier) to control whether bootstrap samples are used. ### Does this PR introduce any user-facing change? Yes, new param is added ### How was this patch tested? existing testsuites Closes #27254 from zhengruifeng/add_bootstrap. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- R/pkg/R/mllib_tree.R | 9 ++++++--- R/pkg/tests/fulltests/test_mllib_tree.R | 2 +- .../DecisionTreeClassifier.scala | 1 + .../RandomForestClassifier.scala | 7 ++++++- .../r/RandomForestClassificationWrapper.scala | 4 +++- .../ml/r/RandomForestRegressionWrapper.scala | 4 +++- .../ml/regression/AFTSurvivalRegression.scala | 2 +- .../ml/regression/DecisionTreeRegressor.scala | 2 +- .../spark/ml/regression/FMRegressor.scala | 2 +- .../spark/ml/regression/GBTRegressor.scala | 1 - .../ml/regression/RandomForestRegressor.scala | 8 ++++++-- .../spark/ml/tree/impl/BaggedPoint.scala | 18 ++++++++++------- .../ml/tree/impl/GradientBoostedTrees.scala | 5 +++-- .../spark/ml/tree/impl/RandomForest.scala | 4 +--- .../org/apache/spark/ml/tree/treeParams.scala | 14 +++++++++++++ .../spark/mllib/tree/RandomForest.scala | 8 +++++++- .../mllib/tree/configuration/Strategy.scala | 3 ++- .../RandomForestClassifierSuite.scala | 5 ++++- .../RandomForestRegressorSuite.scala | 2 ++ python/pyspark/ml/classification.py | 20 ++++++++++++++----- python/pyspark/ml/regression.py | 20 ++++++++++++++----- python/pyspark/ml/tree.py | 12 ++++++++++- 22 files changed, 114 insertions(+), 39 deletions(-) diff --git a/R/pkg/R/mllib_tree.R b/R/pkg/R/mllib_tree.R index ff16b436217dc..f6aa48f5fa04a 100644 --- a/R/pkg/R/mllib_tree.R +++ b/R/pkg/R/mllib_tree.R @@ -393,6 +393,7 @@ setMethod("write.ml", signature(object = "GBTClassificationModel", path = "chara #' "error" (throw an error), "keep" (put invalid data in #' a special additional bucket, at index numLabels). Default #' is "error". +#' @param bootstrap Whether bootstrap samples are used when building trees. #' @param ... additional arguments passed to the method. #' @aliases spark.randomForest,SparkDataFrame,formula-method #' @return \code{spark.randomForest} returns a fitted Random Forest model. @@ -428,7 +429,8 @@ setMethod("spark.randomForest", signature(data = "SparkDataFrame", formula = "fo featureSubsetStrategy = "auto", seed = NULL, subsamplingRate = 1.0, minInstancesPerNode = 1, minInfoGain = 0.0, checkpointInterval = 10, maxMemoryInMB = 256, cacheNodeIds = FALSE, - handleInvalid = c("error", "keep", "skip")) { + handleInvalid = c("error", "keep", "skip"), + bootstrap = TRUE) { type <- match.arg(type) formula <- paste(deparse(formula), collapse = "") if (!is.null(seed)) { @@ -445,7 +447,8 @@ setMethod("spark.randomForest", signature(data = "SparkDataFrame", formula = "fo as.numeric(minInfoGain), as.integer(checkpointInterval), as.character(featureSubsetStrategy), seed, as.numeric(subsamplingRate), - as.integer(maxMemoryInMB), as.logical(cacheNodeIds)) + as.integer(maxMemoryInMB), as.logical(cacheNodeIds), + as.logical(bootstrap)) new("RandomForestRegressionModel", jobj = jobj) }, classification = { @@ -460,7 +463,7 @@ setMethod("spark.randomForest", signature(data = "SparkDataFrame", formula = "fo as.character(featureSubsetStrategy), seed, as.numeric(subsamplingRate), as.integer(maxMemoryInMB), as.logical(cacheNodeIds), - handleInvalid) + handleInvalid, as.logical(bootstrap)) new("RandomForestClassificationModel", jobj = jobj) } ) diff --git a/R/pkg/tests/fulltests/test_mllib_tree.R b/R/pkg/tests/fulltests/test_mllib_tree.R index ad68700c7ff4e..ee5043a744bba 100644 --- a/R/pkg/tests/fulltests/test_mllib_tree.R +++ b/R/pkg/tests/fulltests/test_mllib_tree.R @@ -130,7 +130,7 @@ test_that("spark.randomForest", { # regression data <- suppressWarnings(createDataFrame(longley)) model <- spark.randomForest(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, - numTrees = 1, seed = 1) + numTrees = 1, seed = 1, bootstrap = FALSE) predictions <- collect(predict(model, data)) expect_equal(predictions$prediction, c(60.323, 61.122, 60.171, 61.187, diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 99ed12d560303..530943c910d7d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -125,6 +125,7 @@ class DecisionTreeClassifier @Since("1.4.0") ( validateNumClasses(numClasses) val instances = extractInstances(dataset, numClasses) val strategy = getOldStrategy(categoricalFeatures, numClasses) + require(!strategy.bootstrap, "DecisionTreeClassifier does not need bootstrap sampling") instr.logNumClasses(numClasses) instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, probabilityCol, leafCol, maxDepth, maxBins, minInstancesPerNode, minInfoGain, diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index de52740e7d981..9ed81bf893450 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -115,6 +115,10 @@ class RandomForestClassifier @Since("1.4.0") ( @Since("1.4.0") def setNumTrees(value: Int): this.type = set(numTrees, value) + /** @group setParam */ + @Since("3.0.0") + def setBootstrap(value: Boolean): this.type = set(bootstrap, value) + /** @group setParam */ @Since("1.4.0") def setFeatureSubsetStrategy(value: String): this.type = @@ -147,11 +151,12 @@ class RandomForestClassifier @Since("1.4.0") ( val instances = extractInstances(dataset, numClasses) val strategy = super.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, getOldImpurity) + strategy.bootstrap = $(bootstrap) instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, probabilityCol, rawPredictionCol, leafCol, impurity, numTrees, featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, minInstancesPerNode, minWeightFractionPerNode, seed, - subsamplingRate, thresholds, cacheNodeIds, checkpointInterval) + subsamplingRate, thresholds, cacheNodeIds, checkpointInterval, bootstrap) val trees = RandomForest .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala index 132345fb9a6d9..64c2d04ac7462 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala @@ -79,7 +79,8 @@ private[r] object RandomForestClassifierWrapper extends MLReadable[RandomForestC subsamplingRate: Double, maxMemoryInMB: Int, cacheNodeIds: Boolean, - handleInvalid: String): RandomForestClassifierWrapper = { + handleInvalid: String, + bootstrap: Boolean): RandomForestClassifierWrapper = { val rFormula = new RFormula() .setFormula(formula) @@ -107,6 +108,7 @@ private[r] object RandomForestClassifierWrapper extends MLReadable[RandomForestC .setFeaturesCol(rFormula.getFeaturesCol) .setLabelCol(rFormula.getLabelCol) .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + .setBootstrap(bootstrap) if (seed != null && seed.length > 0) rfc.setSeed(seed.toLong) val idxToStr = new IndexToString() diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala index 038bd79c7022b..60ecc24ae2745 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala @@ -69,7 +69,8 @@ private[r] object RandomForestRegressorWrapper extends MLReadable[RandomForestRe seed: String, subsamplingRate: Double, maxMemoryInMB: Int, - cacheNodeIds: Boolean): RandomForestRegressorWrapper = { + cacheNodeIds: Boolean, + bootstrap: Boolean): RandomForestRegressorWrapper = { val rFormula = new RFormula() .setFormula(formula) @@ -96,6 +97,7 @@ private[r] object RandomForestRegressorWrapper extends MLReadable[RandomForestRe .setMaxMemoryInMB(maxMemoryInMB) .setCacheNodeIds(cacheNodeIds) .setFeaturesCol(rFormula.getFeaturesCol) + .setBootstrap(bootstrap) if (seed != null && seed.length > 0) rfr.setSeed(seed.toLong) val pipeline = new Pipeline() diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 9c27ea8816495..8c95d25f3b5b8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index e69d082a3cc68..3cb69e7fe8212 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -22,7 +22,6 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ @@ -117,6 +116,7 @@ class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) val instances = extractInstances(dataset) val strategy = getOldStrategy(categoricalFeatures) + require(!strategy.bootstrap, "DecisionTreeRegressor does not need bootstrap sampling") instr.logPipelineStage(this) instr.logDataset(instances) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala index 06212fdd041b8..f7810eb17cf59 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/FMRegressor.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.linalg._ import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.param._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 29991f59e37cf..f348dd2b72e73 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -23,7 +23,6 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 9d11f95816fb5..bb74c562f25b7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -21,7 +21,6 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.ml.{PredictionModel, Predictor} import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ @@ -112,6 +111,10 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S @Since("1.4.0") def setNumTrees(value: Int): this.type = set(numTrees, value) + /** @group setParam */ + @Since("3.0.0") + def setBootstrap(value: Boolean): this.type = set(bootstrap, value) + /** @group setParam */ @Since("1.4.0") def setFeatureSubsetStrategy(value: String): this.type = @@ -135,13 +138,14 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S val instances = extractInstances(dataset) val strategy = super.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, getOldImpurity) + strategy.bootstrap = $(bootstrap) instr.logPipelineStage(this) instr.logDataset(instances) instr.logParams(this, labelCol, featuresCol, weightCol, predictionCol, leafCol, impurity, numTrees, featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, minInstancesPerNode, minWeightFractionPerNode, seed, subsamplingRate, cacheNodeIds, - checkpointInterval) + checkpointInterval, bootstrap) val trees = RandomForest .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala index 0ec7c6f03a11f..5b8620c5eee9f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala @@ -67,13 +67,11 @@ private[spark] object BaggedPoint { if (withReplacement) { convertToBaggedRDDSamplingWithReplacement(input, subsamplingRate, numSubsamples, extractSampleWeight, seed) + } else if (subsamplingRate == 1.0) { + convertToBaggedRDDWithoutSampling(input, numSubsamples, extractSampleWeight) } else { - if (numSubsamples == 1 && subsamplingRate == 1.0) { - convertToBaggedRDDWithoutSampling(input, extractSampleWeight) - } else { - convertToBaggedRDDSamplingWithoutReplacement(input, subsamplingRate, numSubsamples, - extractSampleWeight, seed) - } + convertToBaggedRDDSamplingWithoutReplacement(input, subsamplingRate, numSubsamples, + extractSampleWeight, seed) } } @@ -125,7 +123,13 @@ private[spark] object BaggedPoint { private def convertToBaggedRDDWithoutSampling[Datum] ( input: RDD[Datum], + numSubsamples: Int, extractSampleWeight: (Datum => Double)): RDD[BaggedPoint[Datum]] = { - input.map(datum => new BaggedPoint(datum, Array(1), extractSampleWeight(datum))) + input.mapPartitions { instances => + val subsampleCounts = Array.fill(numSubsamples)(1) + instances.map { instance => + new BaggedPoint(instance, subsampleCounts, extractSampleWeight(instance)) + } + } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index 51aad3f08c96a..19ea7e4b393b4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -311,6 +311,7 @@ private[spark] object GradientBoostedTrees extends Logging { val validationTol = boostingStrategy.validationTol treeStrategy.algo = OldAlgo.Regression treeStrategy.impurity = OldVariance + require(!treeStrategy.bootstrap, "GradientBoostedTrees does not need bootstrap sampling") treeStrategy.assertValid() // Prepare periodic checkpointers @@ -346,7 +347,7 @@ private[spark] object GradientBoostedTrees extends Logging { val firstCounts = BaggedPoint .convertToBaggedRDD(treePoints, treeStrategy.subsamplingRate, numSubsamples = 1, - withReplacement = false, (tp: TreePoint) => tp.weight, seed = seed) + treeStrategy.bootstrap, (tp: TreePoint) => tp.weight, seed = seed) .map { bagged => require(bagged.subsampleCounts.length == 1) require(bagged.sampleWeight == bagged.datum.weight) @@ -411,7 +412,7 @@ private[spark] object GradientBoostedTrees extends Logging { // (label: Double, count: Int) val labelWithCounts = BaggedPoint .convertToBaggedRDD(treePoints, treeStrategy.subsamplingRate, numSubsamples = 1, - withReplacement = false, (tp: TreePoint) => tp.weight, seed = seed + m) + treeStrategy.bootstrap, (tp: TreePoint) => tp.weight, seed = seed + m) .zip(predError) .map { case (bagged, (pred, _)) => require(bagged.subsampleCounts.length == 1) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index b3848c1695a99..e0382c694b038 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -290,11 +290,9 @@ private[spark] object RandomForest extends Logging with Serializable { // Cache input RDD for speedup during multiple passes. val treeInput = TreePoint.convertToTreeRDD(retaggedInput, splits, metadata) - val withReplacement = numTrees > 1 - val bcSplits = input.sparkContext.broadcast(splits) val baggedInput = BaggedPoint - .convertToBaggedRDD(treeInput, strategy.subsamplingRate, numTrees, withReplacement, + .convertToBaggedRDD(treeInput, strategy.subsamplingRate, numTrees, strategy.bootstrap, (tp: TreePoint) => tp.weight, seed = seed) .persist(StorageLevel.MEMORY_AND_DISK) .setName("bagged tree points") diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 48675dc773c3e..26a639eea2992 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -449,6 +449,20 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { /** @group getParam */ final def getNumTrees: Int = $(numTrees) + + /** + * Whether bootstrap samples are used when building trees. + * @group expertParam + */ + @Since("3.0.0") + final val bootstrap: BooleanParam = new BooleanParam(this, "bootstrap", + "Whether bootstrap samples are used when building trees.") + + setDefault(bootstrap -> true) + + /** @group getParam */ + @Since("3.0.0") + final def getBootstrap: Boolean = $(bootstrap) } private[ml] trait RandomForestClassifierParams diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index ecbb21498e68f..4f0c51e293319 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -91,8 +91,14 @@ private class RandomForest ( * @return RandomForestModel that can be used for prediction. */ def run(input: RDD[LabeledPoint]): RandomForestModel = { + val treeStrategy = strategy.copy + if (numTrees == 1) { + treeStrategy.bootstrap = false + } else { + treeStrategy.bootstrap = true + } val trees: Array[NewDTModel] = - NewRandomForest.run(input, strategy, numTrees, featureSubsetStrategy, seed.toLong) + NewRandomForest.run(input, treeStrategy, numTrees, featureSubsetStrategy, seed.toLong) new RandomForestModel(strategy.algo, trees.map(_.toOld)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index d9dcb80013400..09e3e22030546 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -81,7 +81,8 @@ class Strategy @Since("1.3.0") ( @Since("1.2.0") @BeanProperty var subsamplingRate: Double = 1, @Since("1.2.0") @BeanProperty var useNodeIdCache: Boolean = false, @Since("1.2.0") @BeanProperty var checkpointInterval: Int = 10, - @Since("3.0.0") @BeanProperty var minWeightFractionPerNode: Double = 0.0) extends Serializable { + @Since("3.0.0") @BeanProperty var minWeightFractionPerNode: Double = 0.0, + @BeanProperty private[spark] var bootstrap: Boolean = false) extends Serializable { /** */ diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index b21ab81438d21..e30e93ad4628c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -82,12 +82,14 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { test("Binary classification with continuous features:" + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val rf = new RandomForestClassifier() + .setBootstrap(false) binaryClassificationTestWithContinuousFeatures(rf) } test("Binary classification with continuous features and node Id cache:" + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val rf = new RandomForestClassifier() + .setBootstrap(false) .setCacheNodeIds(true) binaryClassificationTestWithContinuousFeatures(rf) } @@ -333,6 +335,7 @@ private object RandomForestClassifierSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, rf.getOldImpurity) + oldStrategy.bootstrap = rf.getBootstrap val oldModel = OldRandomForest.trainClassifier( data.map(OldLabeledPoint.fromML), oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) @@ -344,7 +347,7 @@ private object RandomForestClassifierSuite extends SparkFunSuite { numClasses) TreeTests.checkEqual(oldModelAsNew, newModel) assert(newModel.hasParent) - assert(!newModel.trees.head.asInstanceOf[DecisionTreeClassificationModel].hasParent) + assert(!newModel.trees.head.hasParent) assert(newModel.numClasses === numClasses) assert(newModel.numFeatures === numFeatures) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index fff5cdd4ec50c..31dc6d379e76c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -72,12 +72,14 @@ class RandomForestRegressorSuite extends MLTest with DefaultReadWriteTest{ test("Regression with continuous features:" + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val rf = new RandomForestRegressor() + .setBootstrap(false) regressionTestWithContinuousFeatures(rf) } test("Regression with continuous features and node Id cache :" + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val rf = new RandomForestRegressor() + .setBootstrap(false) .setCacheNodeIds(true) regressionTestWithContinuousFeatures(rf) } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d587782db88b0..5ab8e606bda03 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1395,6 +1395,8 @@ class RandomForestClassifier(JavaProbabilisticClassifier, _RandomForestClassifie RandomForestClassificationModel... >>> model.setRawPredictionCol("newRawPrediction") RandomForestClassificationModel... + >>> model.getBootstrap() + True >>> model.getRawPredictionCol() 'newRawPrediction' >>> model.featureImportances @@ -1442,14 +1444,14 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", numTrees=20, featureSubsetStrategy="auto", seed=None, subsamplingRate=1.0, - leafCol="", minWeightFractionPerNode=0.0, weightCol=None): + leafCol="", minWeightFractionPerNode=0.0, weightCol=None, bootstrap=True): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", \ numTrees=20, featureSubsetStrategy="auto", seed=None, subsamplingRate=1.0, \ - leafCol="", minWeightFractionPerNode=0.0, weightCol=None) + leafCol="", minWeightFractionPerNode=0.0, weightCol=None, bootstrap=True) """ super(RandomForestClassifier, self).__init__() self._java_obj = self._new_java_obj( @@ -1457,7 +1459,8 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", numTrees=20, featureSubsetStrategy="auto", - subsamplingRate=1.0, leafCol="", minWeightFractionPerNode=0.0) + subsamplingRate=1.0, leafCol="", minWeightFractionPerNode=0.0, + bootstrap=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1468,14 +1471,14 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, impurity="gini", numTrees=20, featureSubsetStrategy="auto", subsamplingRate=1.0, - leafCol="", minWeightFractionPerNode=0.0, weightCol=None): + leafCol="", minWeightFractionPerNode=0.0, weightCol=None, bootstrap=True): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, \ impurity="gini", numTrees=20, featureSubsetStrategy="auto", subsamplingRate=1.0, \ - leafCol="", minWeightFractionPerNode=0.0, weightCol=None) + leafCol="", minWeightFractionPerNode=0.0, weightCol=None, bootstrap=True) Sets params for linear classification. """ kwargs = self._input_kwargs @@ -1534,6 +1537,13 @@ def setNumTrees(self, value): """ return self._set(numTrees=value) + @since("3.0.0") + def setBootstrap(self, value): + """ + Sets the value of :py:attr:`bootstrap`. + """ + return self._set(bootstrap=value) + @since("1.4.0") def setSubsamplingRate(self, value): """ diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index fd41c12ca3351..481271f523151 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1039,6 +1039,8 @@ class RandomForestRegressor(JavaRegressor, _RandomForestRegressorParams, JavaMLW >>> rf.setSeed(42) RandomForestRegressor... >>> model = rf.fit(df) + >>> model.getBootstrap() + True >>> model.getSeed() 42 >>> model.setLeafCol("leafId") @@ -1086,14 +1088,14 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, - weightCol=None): + weightCol=None, bootstrap=True): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, \ featureSubsetStrategy="auto", leafCol=", minWeightFractionPerNode=0.0", \ - weightCol=None) + weightCol=None, bootstrap=True) """ super(RandomForestRegressor, self).__init__() self._java_obj = self._new_java_obj( @@ -1101,7 +1103,8 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, numTrees=20, - featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0) + featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, + bootstrap=True) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -1112,14 +1115,14 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, - weightCol=None): + weightCol=None, bootstrap=True): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, \ featureSubsetStrategy="auto", leafCol="", minWeightFractionPerNode=0.0, \ - weightCol=None) + weightCol=None, bootstrap=True) Sets params for linear regression. """ kwargs = self._input_kwargs @@ -1178,6 +1181,13 @@ def setNumTrees(self, value): """ return self._set(numTrees=value) + @since("3.0.0") + def setBootstrap(self, value): + """ + Sets the value of :py:attr:`bootstrap`. + """ + return self._set(bootstrap=value) + @since("1.4.0") def setSubsamplingRate(self, value): """ diff --git a/python/pyspark/ml/tree.py b/python/pyspark/ml/tree.py index d97a950c9276e..a13b27ec8a79c 100644 --- a/python/pyspark/ml/tree.py +++ b/python/pyspark/ml/tree.py @@ -254,6 +254,9 @@ class _RandomForestParams(_TreeEnsembleParams): numTrees = Param(Params._dummy(), "numTrees", "Number of trees to train (>= 1).", typeConverter=TypeConverters.toInt) + bootstrap = Param(Params._dummy(), "bootstrap", "Whether bootstrap samples are used " + "when building trees.", typeConverter=TypeConverters.toBoolean) + def __init__(self): super(_RandomForestParams, self).__init__() @@ -264,6 +267,13 @@ def getNumTrees(self): """ return self.getOrDefault(self.numTrees) + @since("3.0.0") + def getBootstrap(self): + """ + Gets the value of bootstrap or its default value. + """ + return self.getOrDefault(self.bootstrap) + class _GBTParams(_TreeEnsembleParams, HasMaxIter, HasStepSize, HasValidationIndicatorCol): """ @@ -313,7 +323,7 @@ def getImpurity(self): return self.getOrDefault(self.impurity) -class _TreeClassifierParams(object): +class _TreeClassifierParams(Params): """ Private class to track supported impurity measures. From d0bf4474212e8eef1c6e124eca0772931fccd1fd Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 23 Jan 2020 17:51:16 +0900 Subject: [PATCH 1201/1387] [SPARK-30575][DOCS][FOLLOWUP] Fix typos in documents ### What changes were proposed in this pull request? Fix a few super nit problems ### Why are the changes needed? To make doc look better ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? Tested using jykyll build --serve Closes #27332 from huaxingao/spark-30575-followup. Authored-by: Huaxin Gao Signed-off-by: Takeshi Yamamuro --- docs/sql-ref-syntax-qry-select-groupby.md | 2 +- docs/sql-ref-syntax-qry-select-having.md | 4 ++-- docs/sql-ref-syntax-qry-select-where.md | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/sql-ref-syntax-qry-select-groupby.md b/docs/sql-ref-syntax-qry-select-groupby.md index 2d13c349561a8..e47ca0bf3ceb2 100644 --- a/docs/sql-ref-syntax-qry-select-groupby.md +++ b/docs/sql-ref-syntax-qry-select-groupby.md @@ -44,7 +44,7 @@ GROUP BY [ GROUPING SETS grouping_sets ] group_expression [ , group_expression [ set is specified by a list of comma-separated expressions in parentheses.

    Syntax: - (() | (expression [ , ...])) + ( () | ( expression [ , ... ] ) )
    grouping_expression
    diff --git a/docs/sql-ref-syntax-qry-select-having.md b/docs/sql-ref-syntax-qry-select-having.md index 718c22fd4b114..428d8556e73e0 100644 --- a/docs/sql-ref-syntax-qry-select-having.md +++ b/docs/sql-ref-syntax-qry-select-having.md @@ -20,7 +20,7 @@ license: | --- The HAVING clause is used to filter the results produced by GROUP BY based on the specified condition. It is often used -in the conjunction with a [GROUP BY](sql-ref-syntax-qry-select-groupby.html) +in conjunction with a [GROUP BY](sql-ref-syntax-qry-select-groupby.html) clause. ### Syntax @@ -49,7 +49,7 @@ HAVING boolean_expression ### Examples {% highlight sql %} CREATE TABLE dealer (id INT, city STRING, car_model STRING, quantity INT); -INSERT INTO dealer VALUES (100, 'Fremont', 'Honda Civic', 10), +INSERT INTO dealer VALUES (100, 'Fremont', 'Honda Civic', 10), (100, 'Fremont', 'Honda Accord', 15), (100, 'Fremont', 'Honda CRV', 7), (200, 'Dublin', 'Honda Civic', 20), diff --git a/docs/sql-ref-syntax-qry-select-where.md b/docs/sql-ref-syntax-qry-select-where.md index 2c8d449172eed..09fab64bbe7bb 100644 --- a/docs/sql-ref-syntax-qry-select-where.md +++ b/docs/sql-ref-syntax-qry-select-where.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- The WHERE clause is used to limit the results of the FROM -clause of query or subquery based on the specified condition. +clause of a query or a subquery based on the specified condition. ### Syntax {% highlight sql %} @@ -62,7 +62,7 @@ SELECT * FROM person WHERE id = 200 OR id = 300 ORDER BY id; |300|Mike|80 | +---+----+----+ --- ISNULL expression in `WHERE` clause. +-- IS NULL expression in `WHERE` clause. SELECT * FROM person WHERE id > 300 OR age IS NULL ORDER BY id; +---+----+----+ |id |name|age | From ffd435b57f55c84718c83a74847ae37836e8e012 Mon Sep 17 00:00:00 2001 From: Xiao Li Date: Thu, 23 Jan 2020 21:41:56 +0800 Subject: [PATCH 1202/1387] [SPARK-27871][SQL][FOLLOW-UP] Remove the conf spark.sql.optimizer.reassignLambdaVariableID.enabled ### What changes were proposed in this pull request? This PR is to remove the conf ### Why are the changes needed? This rule can be excluded using spark.sql.optimizer.excludedRules without an extra conf ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? N/A Closes #27334 from gatorsmile/spark27871. Authored-by: Xiao Li Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/optimizer/objects.scala | 2 -- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 7 ------- .../org/apache/spark/sql/DatasetOptimizationSuite.scala | 9 --------- 3 files changed, 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala index ad93ef347a974..0fa29e87e5f39 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala @@ -235,8 +235,6 @@ object ObjectSerializerPruning extends Rule[LogicalPlan] { */ object ReassignLambdaVariableID extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { - if (!SQLConf.get.getConf(SQLConf.OPTIMIZER_REASSIGN_LAMBDA_VARIABLE_ID)) return plan - // The original LambdaVariable IDs are all positive. To avoid conflicts, the new IDs are all // negative and starts from -1. var newId = 0L diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6fde8ce9f7f32..9515861a01e40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -234,13 +234,6 @@ object SQLConf { .stringConf .createOptional - val OPTIMIZER_REASSIGN_LAMBDA_VARIABLE_ID = - buildConf("spark.sql.optimizer.reassignLambdaVariableID.enabled") - .doc("When true, Spark optimizer reassigns per-query unique IDs to LambdaVariable, so that " + - "it's more likely to hit codegen cache.") - .booleanConf - .createWithDefault(true) - val DYNAMIC_PARTITION_PRUNING_ENABLED = buildConf("spark.sql.optimizer.dynamicPartitionPruning.enabled") .doc("When true, we will generate predicate for partition column when it's used as join key") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala index 892122b94b977..0ac99905f35f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala @@ -181,15 +181,6 @@ class DatasetOptimizationSuite extends QueryTest with SharedSparkSession { // codegen cache should work for Datasets of same type. val count3 = getCodegenCount() assert(count3 == count2) - - withSQLConf(SQLConf.OPTIMIZER_REASSIGN_LAMBDA_VARIABLE_ID.key -> "false") { - // trigger codegen for another Dataset of same type - createDataset().collect() - // with the rule disabled, codegen happens again for encoder serializer and encoder - // deserializer - val count4 = getCodegenCount() - assert(count4 == (count3 + 2)) - } } withClue("array type") { From dbed4c72f322c4dc61f1a4192d3665d2500e38f4 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 23 Jan 2020 21:56:54 +0800 Subject: [PATCH 1203/1387] [SPARK-30605][SQL] move defaultNamespace from SupportsNamespace to CatalogPlugin ### What changes were proposed in this pull request? Move the `defaultNamespace` method from the interface `SupportsNamespace` to `CatalogPlugin`. ### Why are the changes needed? While I'm implementing JDBC V2, I realize that the default namespace is very an important information. Even if you don't want to implement namespace manipulation functionalities like CREATE/DROP/ALTER namespace, you still need to report the default namespace. The default namespace is not about functionality but a matter of correctness. If you don't know the default namespace of a catalog, it's wrong to assume it's `[]`. You may get table not found exception if you do so. I think it's more reasonable to put the `defaultNamespace` method in the base class `CatalogPlugin`. It returns `[]` by default so won't bother implementation if they really don't have namespace concept. ### Does this PR introduce any user-facing change? yes, but for an unreleased API. ### How was this patch tested? existing tests Closes #27319 from cloud-fan/ns. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/connector/catalog/CatalogPlugin.java | 14 +++++++++++ .../catalog/DelegatingCatalogExtension.java | 10 ++++---- .../connector/catalog/SupportsNamespaces.java | 14 ----------- .../connector/catalog/CatalogManager.scala | 7 +----- .../catalog/CatalogManagerSuite.scala | 25 ++----------------- 5 files changed, 22 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java index 615aab3ef3259..34fdc5dd4d0fb 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/CatalogPlugin.java @@ -58,4 +58,18 @@ public interface CatalogPlugin { * called to pass the catalog's name. */ String name(); + + /** + * Return a default namespace for the catalog. + *

    + * When this catalog is set as the current catalog, the namespace returned by this method will be + * set as the current namespace. + *

    + * The namespace returned by this method is not required to exist. + * + * @return a multi-part namespace + */ + default String[] defaultNamespace() { + return new String[0]; + } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java index b93acdc777e9c..cf0eef96d5a9c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/DelegatingCatalogExtension.java @@ -51,6 +51,11 @@ public String name() { @Override public final void initialize(String name, CaseInsensitiveStringMap options) {} + @Override + public String[] defaultNamespace() { + return delegate.defaultNamespace(); + } + @Override public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException { return asTableCatalog().listTables(namespace); @@ -99,11 +104,6 @@ public void renameTable( asTableCatalog().renameTable(oldIdent, newIdent); } - @Override - public String[] defaultNamespace() { - return asNamespaceCatalog().defaultNamespace(); - } - @Override public String[][] listNamespaces() throws NoSuchNamespaceException { return asNamespaceCatalog().listNamespaces(); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java index 757d303e82255..cfe98b8b5cf49 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java @@ -69,20 +69,6 @@ public interface SupportsNamespaces extends CatalogPlugin { */ List RESERVED_PROPERTIES = Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_OWNER); - /** - * Return a default namespace for the catalog. - *

    - * When this catalog is set as the current catalog, the namespace returned by this method will be - * set as the current namespace. - *

    - * The namespace returned by this method is not required to exist. - * - * @return a multi-part namespace - */ - default String[] defaultNamespace() { - return new String[0]; - } - /** * List top-level namespaces from the catalog. *

    diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index bbb7a182c6a6f..a81d8f79d6fcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -93,11 +93,6 @@ class CatalogManager( }.getOrElse(defaultSessionCatalog) } - private def getDefaultNamespace(c: CatalogPlugin) = c match { - case c: SupportsNamespaces => c.defaultNamespace() - case _ => Array.empty[String] - } - private var _currentNamespace: Option[Array[String]] = None def currentNamespace: Array[String] = synchronized { @@ -105,7 +100,7 @@ class CatalogManager( if (currentCatalog.name() == SESSION_CATALOG_NAME) { Array(v1SessionCatalog.getCurrentDatabase) } else { - getDefaultNamespace(currentCatalog) + currentCatalog.defaultNamespace() } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala index fc7b2e720b7b7..17d326019f86b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogManagerSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.connector.catalog import java.net.URI -import java.util import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, FakeV2SessionCatalog, NoSuchNamespaceException} @@ -112,31 +111,11 @@ class CatalogManagerSuite extends SparkFunSuite { } } -class DummyCatalog extends SupportsNamespaces { - override def defaultNamespace(): Array[String] = Array("a", "b") - - override def listNamespaces(): Array[Array[String]] = { - throw new UnsupportedOperationException - } - override def listNamespaces(namespace: Array[String]): Array[Array[String]] = { - throw new UnsupportedOperationException - } - override def loadNamespaceMetadata(namespace: Array[String]): util.Map[String, String] = { - throw new UnsupportedOperationException - } - override def createNamespace( - namespace: Array[String], metadata: util.Map[String, String]): Unit = { - throw new UnsupportedOperationException - } - override def alterNamespace(namespace: Array[String], changes: NamespaceChange*): Unit = { - throw new UnsupportedOperationException - } - override def dropNamespace(namespace: Array[String]): Boolean = { - throw new UnsupportedOperationException - } +class DummyCatalog extends CatalogPlugin { override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { _name = name } private var _name: String = null override def name(): String = _name + override def defaultNamespace(): Array[String] = Array("a", "b") } From 705fc6ad9328c1092299b83071b6ec3b1d6f9c4d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 23 Jan 2020 23:01:25 +0800 Subject: [PATCH 1204/1387] [SPARK-30188][SQL][TESTS][FOLLOW-UP] Remove `sorted` in asserts of comparing two strings ### What changes were proposed in this pull request? In the PR, I propose to remove sorting in the asserts of checking output of: - expression examples, - SQL tests in `SQLQueryTestSuite`. ### Why are the changes needed? * Sorted `actual` and `expected` make assert output unusable. Instead of `"[true]" did not equal "[false]"`, it looks like `"[ertu]" did not equal "[aefls]"`. * Output of expression examples should be always the same except nondeterministic expressions listed in the `ignoreSet` of the `check outputs of expression examples` test. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `SQLQuerySuite` via `./build/sbt "sql/test:testOnly org.apache.spark.sql.SQLQuerySuite"`. Closes #27324 from MaxGekk/remove-sorting-in-examples-tests. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) 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 7af7b678f8c30..2d97eb592b616 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 @@ -193,7 +193,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark val actual = unindentAndTrim( hiveResultString(df.queryExecution.executedPlan).mkString("\n")) val expected = unindentAndTrim(output) - assert(actual.sorted === expected.sorted) + assert(actual === expected) case _ => }) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 75a630c200cce..350b43dc94dae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -419,8 +419,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { s"Schema did not match for query #$i\n${expected.sql}: $output") { output.schema } - assertResult(expected.output.sorted, s"Result did not match" + - s" for query #$i\n${expected.sql}") { output.output.sorted } + assertResult(expected.output, s"Result did not match" + + s" for query #$i\n${expected.sql}") { output.output } } } } From 3d7359ad4202067b26a199657b6a3e1f38be0e4d Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 23 Jan 2020 08:35:32 -0800 Subject: [PATCH 1205/1387] [SPARK-29175][SQL][FOLLOW-UP] Rename the config name to spark.sql.maven.additionalRemoteRepositories ### What changes were proposed in this pull request? Rename the config added in #25849 to `spark.sql.maven.additionalRemoteRepositories`. ### Why are the changes needed? Follow the advice in [SPARK-29175](https://issues.apache.org/jira/browse/SPARK-29175?focusedCommentId=17021586&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17021586), the new name is more clear. ### Does this PR introduce any user-facing change? Yes, the config name changed. ### How was this patch tested? Existing test. Closes #27339 from xuanyuanking/SPARK-29175. Authored-by: Yuanjian Li Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9515861a01e40..4471c4d4d1b38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2094,7 +2094,7 @@ object SQLConf { .createWithDefault(true) val ADDITIONAL_REMOTE_REPOSITORIES = - buildConf("spark.sql.additionalRemoteRepositories") + buildConf("spark.sql.maven.additionalRemoteRepositories") .doc("A comma-delimited string config of the optional additional remote Maven mirror " + "repositories. This is only used for downloading Hive jars in IsolatedClientLoader " + "if the default Maven Central repo is unreachable.") From 3c8b3609a123ed1ffd11b46f37b7fdd5b780bba3 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 24 Jan 2020 01:15:57 +0800 Subject: [PATCH 1206/1387] [SPARK-30620][SQL] avoid unnecessary serialization in AggregateExpression ### What changes were proposed in this pull request? Expressions are very likely to be serialized and sent to executors, we should avoid unnecessary serialization overhead as much as we can. This PR fixes `AggregateExpression`. ### Why are the changes needed? small improvement ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #27342 from cloud-fan/fix. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/aggregate/interfaces.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index 24731c4b8577c..222ad6fab19e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -96,6 +96,7 @@ case class AggregateExpression( extends Expression with Unevaluable { + @transient lazy val resultAttribute: Attribute = if (aggregateFunction.resolved) { AttributeReference( aggregateFunction.toString, @@ -108,7 +109,7 @@ case class AggregateExpression( UnresolvedAttribute(aggregateFunction.toString) } - lazy val filterAttributes: AttributeSet = filter.map(_.references).getOrElse(AttributeSet.empty) + def filterAttributes: AttributeSet = filter.map(_.references).getOrElse(AttributeSet.empty) // We compute the same thing regardless of our final result. override lazy val canonicalized: Expression = { From afe70b3b5321439318a456c7d19b7074171a286a Mon Sep 17 00:00:00 2001 From: Pavithra Ramachandran Date: Thu, 23 Jan 2020 11:29:13 -0600 Subject: [PATCH 1207/1387] [SPARK-28794][SQL][DOC] Documentation for Create table Command ### What changes were proposed in this pull request? Document CREATE TABLE statement in SQL Reference Guide. ### Why are the changes needed? Adding documentation for SQL reference. ### Does this PR introduce any user-facing change? yes Before: There was no documentation for this. ### How was this patch tested? Used jekyll build and serve to verify. Closes #26759 from PavithraRamachandran/create_doc. Authored-by: Pavithra Ramachandran Signed-off-by: Sean Owen --- ...-ref-syntax-ddl-create-table-datasource.md | 115 +++++++++++++++++ ...-ref-syntax-ddl-create-table-hiveformat.md | 122 ++++++++++++++++++ docs/sql-ref-syntax-ddl-create-table-like.md | 97 ++++++++++++++ docs/sql-ref-syntax-ddl-create-table.md | 12 +- 4 files changed, 345 insertions(+), 1 deletion(-) create mode 100644 docs/sql-ref-syntax-ddl-create-table-datasource.md create mode 100644 docs/sql-ref-syntax-ddl-create-table-hiveformat.md create mode 100644 docs/sql-ref-syntax-ddl-create-table-like.md diff --git a/docs/sql-ref-syntax-ddl-create-table-datasource.md b/docs/sql-ref-syntax-ddl-create-table-datasource.md new file mode 100644 index 0000000000000..9b15c0865b38a --- /dev/null +++ b/docs/sql-ref-syntax-ddl-create-table-datasource.md @@ -0,0 +1,115 @@ +--- +layout: global +title: CREATE DATASOURCE TABLE +displayTitle: CREATE DATASOURCE TABLE +license: | + 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. +--- + +### Description + +The `CREATE TABLE` statement defines a new table using a Data Source. + +### Syntax +{% highlight sql %} +CREATE TABLE [ IF NOT EXISTS ] table_identifier + [ ( col_name1 col_type1 [ COMMENT col_comment1 ], ... ) ] + USING data_source + [ OPTIONS ( key1=val1, key2=val2, ... ) ] + [ PARTITIONED BY ( col_name1, col_name2, ... ) ] + [ CLUSTERED BY ( col_name3, col_name4, ... ) + [ SORTED BY ( col_name [ ASC | DESC ], ... ) ] + INTO num_buckets BUCKETS ] + [ LOCATION path ] + [ COMMENT table_comment ] + [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] + [ AS select_statement ] +{% endhighlight %} + +### Parameters + +

    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    +
    +
    +
    USING data_source
    +
    Data Source is the input format used to create the table. Data source can be CSV, TXT, ORC, JDBC, PARQUET, etc.
    +
    + +
    +
    PARTITIONED BY
    +
    Partitions are created on the table, based on the columns specified.
    +
    + +
    +
    CLUSTERED BY
    +
    + Partitions created on the table will be bucketed into fixed buckets based on the column specified for bucketing.

    + NOTE:Bucketing is an optimization technique that uses buckets (and bucketing columns) to determine data partitioning and avoid data shuffle.
    +
    SORTED BY
    +
    Determines the order in which the data is stored in buckets. Default is Ascending order.
    + +
    + +
    +
    LOCATION
    +
    Path to the directory where table data is stored, which could be a path on distributed storage like HDFS, etc.
    +
    + +
    +
    COMMENT
    +
    Table comments are added.
    +
    + +
    +
    TBLPROPERTIES
    +
    Table properties that have to be set are specified, such as `created.by.user`, `owner`, etc. +
    +
    + +
    +
    AS select_statement
    +
    The table is populated using the data from the select statement.
    +
    + +### Examples +{% highlight sql %} + +--Using data source +CREATE TABLE Student (Id INT,name STRING ,age INT) USING CSV; + +--Using data from another table +CREATE TABLE StudentInfo + AS SELECT * FROM Student; + +--Partitioned and bucketed +CREATE TABLE Student (Id INT,name STRING ,age INT) + USING CSV + PARTITIONED BY (age) + CLUSTERED BY (Id) INTO 4 buckets; + +{% endhighlight %} + +### Related Statements +* [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) +* [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) diff --git a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md new file mode 100644 index 0000000000000..78b7d0581e985 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md @@ -0,0 +1,122 @@ +--- +layout: global +title: CREATE HIVEFORMAT TABLE +displayTitle: CREATE HIVEFORMAT TABLE +license: | + 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. +--- +### Description + +The `CREATE TABLE` statement defines a new table using Hive format. + +### Syntax +{% highlight sql %} +CREATE [ EXTERNAL ] TABLE [ IF NOT EXISTS ] table_identifier + [ ( col_name1[:] col_type1 [ COMMENT col_comment1 ], ... ) ] + [ COMMENT table_comment ] + [ PARTITIONED BY ( col_name2[:] col_type2 [ COMMENT col_comment2 ], ... ) + | ( col_name1, col_name2, ... ) ] + [ ROW FORMAT row_format ] + [ STORED AS file_format ] + [ LOCATION path ] + [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] + [ AS select_statement ] + +{% endhighlight %} + +### Parameters + +
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: + + [ database_name. ] table_name + +
    +
    + +
    +
    EXTERNAL
    +
    Table is defined using the path provided as LOCATION, does not use default location for this table.
    +
    + +
    +
    PARTITIONED BY
    +
    Partitions are created on the table, based on the columns specified.
    +
    + +
    +
    ROW FORMAT
    +
    SERDE is used to specify a custom SerDe or the DELIMITED clause in order to use the native SerDe.
    +
    + +
    +
    STORED AS
    +
    File format for table storage, could be TEXTFILE, ORC, PARQUET,etc.
    +
    + +
    +
    LOCATION
    +
    Path to the directory where table data is stored, Path to the directory where table data is stored, which could be a path on distributed storage like HDFS, etc.
    +
    + +
    +
    COMMENT
    +
    Table comments are added.
    +
    + +
    +
    TBLPROPERTIES
    +
    + Table properties that have to be set are specified, such as `created.by.user`, `owner`, etc. +
    +
    + +
    +
    AS select_statement
    +
    The table is populated using the data from the select statement.
    +
    + + +### Examples +{% highlight sql %} + +--Using Comment and loading data from another table into the created table +CREATE TABLE StudentInfo + COMMENT 'Table is created using existing data' + AS SELECT * FROM Student; + +--Partitioned table +CREATE TABLE Student (Id INT,name STRING) + PARTITIONED BY (age INT) + TBLPROPERTIES ('owner'='xxxx'); + +CREATE TABLE Student (Id INT,name STRING,age INT) + PARTITIONED BY (name,age); + +--Using Row Format and file format +CREATE TABLE Student (Id INT,name STRING) + ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + STORED AS TEXTFILE; + +{% endhighlight %} + + +### Related Statements +* [CREATE TABLE USING DATASOURCE](sql-ref-syntax-ddl-create-table-datasource.html) +* [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) diff --git a/docs/sql-ref-syntax-ddl-create-table-like.md b/docs/sql-ref-syntax-ddl-create-table-like.md new file mode 100644 index 0000000000000..b7d7bdd1d9805 --- /dev/null +++ b/docs/sql-ref-syntax-ddl-create-table-like.md @@ -0,0 +1,97 @@ +--- +layout: global +title: CREATE TABLE LIKE +displayTitle: CREATE TABLE LIKE +license: | + 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. +--- +### Description + +The `CREATE TABLE` statement defines a new table using the definition/metadata of an existing table or view. + +### Syntax +{% highlight sql %} +CREATE TABLE [IF NOT EXISTS] table_identifier LIKE source_table_identifier +USING data_source +[ ROW FORMAT row_format ] +[ STORED AS file_format ] +[ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] +[ LOCATION path ] +{% endhighlight %} + +### Parameters +
    +
    table_identifier
    +
    + Specifies a table name, which may be optionally qualified with a database name.

    + Syntax: [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] + + [ database_name. ] table_name + +
    +
    + +
    +
    USING data_source
    +
    Data Source is the input format used to create the table. Data source can be CSV, TXT, ORC, JDBC, PARQUET, etc.
    +
    + +
    +
    ROW FORMAT
    +
    SERDE is used to specify a custom SerDe or the DELIMITED clause in order to use the native SerDe.
    +
    + +
    +
    STORED AS
    +
    File format for table storage, could be TEXTFILE, ORC, PARQUET,etc.
    +
    + +
    +
    TBLPROPERTIES
    +
    Table properties that have to be set are specified, such as `created.by.user`, `owner`, etc. +
    +
    + +
    +
    LOCATION
    +
    Path to the directory where table data is stored,Path to the directory where table data is stored, which could be a path on distributed storage like HDFS, etc. Location to create an external table.
    +
    + + +### Examples +{% highlight sql %} + +--Create table using an exsisting table +CREATE TABLE Student_Dupli like Student; + +--Create table like using a data source +CREATE TABLE Student_Dupli like Student USING CSV; + +--Table is created as external table at the location specified +CREATE TABLE Student_Dupli like Student location '/root1/home'; + +--Create table like using a rowformat +CREATE TABLE Student_Dupli like Student + ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + STORED AS TEXTFILE + TBLPROPERTIES ('owner'='xxxx'); + +{% endhighlight %} + +### Related Statements +* [CREATE TABLE USING DATASOURCE](sql-ref-syntax-ddl-create-table-datasource.html) +* [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) + diff --git a/docs/sql-ref-syntax-ddl-create-table.md b/docs/sql-ref-syntax-ddl-create-table.md index 4ce8ef697c2c2..c0e58a41cf5cc 100644 --- a/docs/sql-ref-syntax-ddl-create-table.md +++ b/docs/sql-ref-syntax-ddl-create-table.md @@ -19,4 +19,14 @@ license: | limitations under the License. --- -**This page is under construction** +### Description +`CREATE TABLE` statement is used to define a table in an exsisting database. + +The CREATE statements: +* [CREATE TABLE USING DATASOURCE](sql-ref-syntax-ddl-create-table-datasource.html) +* [CREATE TABLE USING HIVE FORMAT](sql-ref-syntax-ddl-create-table-hiveformat.html) +* [CREATE TABLE LIKE](sql-ref-syntax-ddl-create-table-like.html) + +### Related Statements +- [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) +- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) From 843224ebd473508cd52e362a55d0e17492257c2a Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Thu, 23 Jan 2020 12:44:43 -0800 Subject: [PATCH 1208/1387] [SPARK-30570][BUILD] Update scalafmt plugin to 1.0.3 with onlyChangedFiles feature ### What changes were proposed in this pull request? Update the scalafmt plugin to 1.0.3 and use its new onlyChangedFiles feature rather than --diff ### Why are the changes needed? Older versions of the plugin either didn't work with scala 2.13, or got rid of the --diff argument and didn't allow for formatting only changed files ### Does this PR introduce any user-facing change? The /dev/scalafmt script no longer passes through arbitrary args, instead using the arg to select scala version. The issue here is the plugin name literally contains the scala version, and doesn't appear to have a shorter way to refer to it. If srowen or someone else with better maven-fu has an idea I'm all ears. ### How was this patch tested? Manually, e.g. edited a file and ran dev/scalafmt or dev/scalafmt 2.13 Closes #27279 from koeninger/SPARK-30570. Authored-by: cody koeninger Signed-off-by: Dongjoon Hyun --- dev/scalafmt | 5 ++--- pom.xml | 44 ++++++++++++++++++++++---------------------- 2 files changed, 24 insertions(+), 25 deletions(-) diff --git a/dev/scalafmt b/dev/scalafmt index 0541a4058467b..3f69bc98f51c7 100755 --- a/dev/scalafmt +++ b/dev/scalafmt @@ -17,7 +17,6 @@ # limitations under the License. # -# by default, format only files that differ from git master -params="${@:---diff}" +VERSION="${@:-2.12}" +./build/mvn -Pscala-$VERSION mvn-scalafmt_$VERSION:format -Dscalafmt.skip=false -./build/mvn -Pscala-2.12 mvn-scalafmt_2.12:format -Dscalafmt.skip=false -Dscalafmt.parameters="$params" \ No newline at end of file diff --git a/pom.xml b/pom.xml index 4e6c571ee052c..91191f9dabe97 100644 --- a/pom.xml +++ b/pom.xml @@ -162,7 +162,7 @@ 3.2.2 2.12.10 2.12 - --diff --test + --test true 1.9.13 @@ -2895,6 +2895,27 @@
    + + org.antipathy + mvn-scalafmt_${scala.binary.version} + 1.0.3 + + ${scalafmt.parameters} + ${scalafmt.skip} + ${scalafmt.skip} + ${scalafmt.skip} + dev/.scalafmt.conf + true + + + + validate + + format + + + + - - org.antipathy - mvn-scalafmt_2.12 - 0.12_1.5.1 - - ${scalafmt.parameters} - ${scalafmt.skip} - ${scalafmt.skip} - ${scalafmt.skip} - dev/.scalafmt.conf - - - - validate - - format - - - - From 976946a910d877c22213df8fe4508969f6472aa0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 23 Jan 2020 13:02:10 -0800 Subject: [PATCH 1209/1387] [SPARK-29947][SQL][FOLLOWUP] Fix table lookup cache ### What changes were proposed in this pull request? Fix a bug in https://github.com/apache/spark/pull/26589 , to make this feature work. ### Why are the changes needed? This feature doesn't work actually. ### Does this PR introduce any user-facing change? no ### How was this patch tested? new test Closes #27341 from cloud-fan/cache. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 8 +- .../analysis/TableLookupCacheSuite.scala | 99 +++++++++++++++++++ 2 files changed, 103 insertions(+), 4 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 36e558b0dc571..45547bff8a9d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -917,14 +917,14 @@ class Analyzer( private def lookupRelation(identifier: Seq[String]): Option[LogicalPlan] = { expandRelationName(identifier) match { case SessionCatalogAndIdentifier(catalog, ident) => - CatalogV2Util.loadTable(catalog, ident).map { + def loaded = CatalogV2Util.loadTable(catalog, ident).map { case v1Table: V1Table => - val key = catalog.name +: ident.namespace :+ ident.name - AnalysisContext.get.relationCache.getOrElseUpdate( - key, v1SessionCatalog.getRelation(v1Table.v1Table)) + v1SessionCatalog.getRelation(v1Table.v1Table) case table => DataSourceV2Relation.create(table) } + val key = catalog.name +: ident.namespace :+ ident.name + Option(AnalysisContext.get.relationCache.getOrElseUpdate(key, loaded.orNull)) case _ => None } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala new file mode 100644 index 0000000000000..eed962cd0f69d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import java.io.File + +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.scalatest.Matchers + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, V1Table} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +class TableLookupCacheSuite extends AnalysisTest with Matchers { + private def getAnalyzer(externalCatalog: ExternalCatalog, databasePath: File): Analyzer = { + val conf = new SQLConf() + val v1Catalog = new SessionCatalog(externalCatalog, FunctionRegistry.builtin, conf) + v1Catalog.createDatabase( + CatalogDatabase("default", "", databasePath.toURI, Map.empty), + ignoreIfExists = false) + v1Catalog.createTable( + CatalogTable( + TableIdentifier("t1", Some("default")), + CatalogTableType.MANAGED, + CatalogStorageFormat.empty, + StructType(Seq(StructField("a", IntegerType)))), + ignoreIfExists = false) + val v2Catalog = new InMemoryTableCatalog { + override def loadTable(ident: Identifier): Table = { + V1Table(externalCatalog.getTable("default", ident.name)) + } + override def name: String = CatalogManager.SESSION_CATALOG_NAME + } + val catalogManager = mock(classOf[CatalogManager]) + when(catalogManager.catalog(any())).thenAnswer((invocation: InvocationOnMock) => { + invocation.getArgument[String](0) match { + case CatalogManager.SESSION_CATALOG_NAME => v2Catalog + case name => + throw new CatalogNotFoundException(s"No such catalog: $name") + } + }) + when(catalogManager.v1SessionCatalog).thenReturn(v1Catalog) + when(catalogManager.currentCatalog).thenReturn(v2Catalog) + when(catalogManager.currentNamespace).thenReturn(Array("default")) + + new Analyzer(catalogManager, conf) + } + + test("table lookups to external catalog are cached") { + withTempDir { tempDir => + val inMemoryCatalog = new InMemoryCatalog + val catalog = spy(inMemoryCatalog) + val analyzer = getAnalyzer(catalog, tempDir) + reset(catalog) + analyzer.execute(table("t1").join(table("t1")).join(table("t1"))) + verify(catalog, times(1)).getTable("default", "t1") + } + } + + test("table lookups via nested views are cached") { + withTempDir { tempDir => + val inMemoryCatalog = new InMemoryCatalog + val catalog = spy(inMemoryCatalog) + val analyzer = getAnalyzer(catalog, tempDir) + val viewDef = CatalogTable( + TableIdentifier("view", Some("default")), + CatalogTableType.VIEW, + CatalogStorageFormat.empty, + StructType(Seq(StructField("a", IntegerType, nullable = true))), + viewText = Some("select * from t1") + ) + catalog.createTable(viewDef, ignoreIfExists = false) + reset(catalog) + analyzer.execute(table("t1").join(table("view")).join(table("view"))) + verify(catalog, times(1)).getTable("default", "t1") + } + } +} From 3228d723a4637d188a3918c22e2ad9eb17eb00ac Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 23 Jan 2020 13:13:25 -0800 Subject: [PATCH 1210/1387] [SPARK-30603][SQL] Move RESERVED_PROPERTIES from SupportsNamespaces and TableCatalog to CatalogV2Util ### What changes were proposed in this pull request? In this PR, I propose to move the `RESERVED_PROPERTIES `s from `SupportsNamespaces` and `TableCatalog` to `CatalogV2Util`, which can keep `RESERVED_PROPERTIES ` safe for interval usages only. ### Why are the changes needed? the `RESERVED_PROPERTIES` should not be changed by subclasses ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing uts Closes #27318 from yaooqinn/SPARK-30603. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../connector/catalog/SupportsNamespaces.java | 19 ++---------- .../sql/connector/catalog/TableCatalog.java | 16 +++------- .../sql/connector/catalog/CatalogV2Util.scala | 29 +++++++++++++++++++ .../analysis/ResolveSessionCatalog.scala | 6 ++-- .../spark/sql/execution/command/ddl.scala | 5 ++-- .../v2/DescribeNamespaceExec.scala | 7 ++--- .../datasources/v2/DescribeTableExec.scala | 6 ++-- .../datasources/v2/V2SessionCatalog.scala | 2 +- .../sql/connector/DataSourceV2SQLSuite.scala | 12 ++++---- .../v2/V2SessionCatalogSuite.scala | 6 ++-- 10 files changed, 56 insertions(+), 52 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java index cfe98b8b5cf49..190f1a14d7129 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java @@ -21,8 +21,6 @@ import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; -import java.util.Arrays; -import java.util.List; import java.util.Map; /** @@ -42,33 +40,22 @@ public interface SupportsNamespaces extends CatalogPlugin { /** - * A property to specify the location of the namespace. If the namespace + * A reserved property to specify the location of the namespace. If the namespace * needs to store files, it should be under this location. */ String PROP_LOCATION = "location"; /** - * A property to specify the description of the namespace. The description + * A reserved property to specify the description of the namespace. The description * will be returned in the result of "DESCRIBE NAMESPACE" command. */ String PROP_COMMENT = "comment"; /** - * A property to specify the owner of the namespace. + * A reserved property to specify the owner of the namespace. */ String PROP_OWNER = "owner"; - /** - * The list of reserved namespace properties, which can not be removed or changed directly by - * the syntax: - * {{ - * ALTER NAMESPACE ... SET PROPERTIES ... - * }} - * - * They need specific syntax to modify - */ - List RESERVED_PROPERTIES = Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_OWNER); - /** * List top-level namespaces from the catalog. *

    diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index 591e1c631be13..a69b23bf84d0c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -24,8 +24,6 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; import org.apache.spark.sql.types.StructType; -import java.util.Arrays; -import java.util.List; import java.util.Map; /** @@ -41,32 +39,26 @@ public interface TableCatalog extends CatalogPlugin { /** - * A property to specify the location of the table. The files of the table + * A reserved property to specify the location of the table. The files of the table * should be under this location. */ String PROP_LOCATION = "location"; /** - * A property to specify the description of the table. + * A reserved property to specify the description of the table. */ String PROP_COMMENT = "comment"; /** - * A property to specify the provider of the table. + * A reserved property to specify the provider of the table. */ String PROP_PROVIDER = "provider"; /** - * A property to specify the owner of the table. + * A reserved property to specify the owner of the table. */ String PROP_OWNER = "owner"; - /** - * The list of reserved table properties. - */ - List RESERVED_PROPERTIES = - Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_PROVIDER, PROP_OWNER); - /** * List the tables in a namespace from the catalog. *

    diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 67726c7343524..83c15af27ccf0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -33,6 +33,35 @@ import org.apache.spark.util.Utils private[sql] object CatalogV2Util { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + /** + * The list of reserved table properties, which can not be removed or changed directly by + * the syntax: + * {{ + * ALTER TABLE ... SET TBLPROPERTIES ... + * }} + * + * They need specific syntax to modify + */ + val TABLE_RESERVED_PROPERTIES = + Seq(TableCatalog.PROP_COMMENT, + TableCatalog.PROP_LOCATION, + TableCatalog.PROP_PROVIDER, + TableCatalog.PROP_OWNER) + + /** + * The list of reserved namespace properties, which can not be removed or changed directly by + * the syntax: + * {{ + * ALTER NAMESPACE ... SET PROPERTIES ... + * }} + * + * They need specific syntax to modify + */ + val NAMESPACE_RESERVED_PROPERTIES = + Seq(SupportsNamespaces.PROP_COMMENT, + SupportsNamespaces.PROP_LOCATION, + SupportsNamespaces.PROP_OWNER) + /** * Apply properties changes to a map and return the result. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 8b0d339dbb864..191029ab7b432 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -17,14 +17,12 @@ package org.apache.spark.sql.catalyst.analysis -import scala.collection.JavaConverters._ - import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, CatalogV2Util, Identifier, LookupCatalog, SupportsNamespaces, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} @@ -326,7 +324,7 @@ class ResolveSessionCatalog( val comment = c.properties.get(SupportsNamespaces.PROP_COMMENT) val location = c.properties.get(SupportsNamespaces.PROP_LOCATION) - val newProperties = c.properties -- SupportsNamespaces.RESERVED_PROPERTIES.asScala + val newProperties = c.properties -- CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES CreateDatabaseCommand(ns.head, c.ifNotExists, location, comment, newProperties) case d @ DropNamespace(SessionCatalogAndNamespace(_, ns), _, _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 9682778d40383..9c0053c6ae0ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -21,7 +21,6 @@ import java.util.Locale import java.util.concurrent.TimeUnit._ import scala.collection.{GenMap, GenSeq} -import scala.collection.JavaConverters._ import scala.collection.parallel.ForkJoinTaskSupport import scala.collection.parallel.immutable.ParVector import scala.util.control.NonFatal @@ -38,8 +37,8 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog} import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ -import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter @@ -183,7 +182,7 @@ case class DescribeDatabaseCommand( Row("Owner", allDbProperties.getOrElse(PROP_OWNER, "")) :: Nil if (extended) { - val properties = allDbProperties -- RESERVED_PROPERTIES.asScala + val properties = allDbProperties -- CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES val propertiesStr = if (properties.isEmpty) { "" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala index 9a0481ef01b63..64b98fb83b8fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} -import org.apache.spark.sql.connector.catalog.SupportsNamespaces +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces} import org.apache.spark.sql.types.StructType /** @@ -35,7 +35,6 @@ case class DescribeNamespaceExec( namespace: Seq[String], isExtended: Boolean) extends V2CommandExec { private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind() - import SupportsNamespaces._ override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() @@ -44,12 +43,12 @@ case class DescribeNamespaceExec( rows += toCatalystRow("Namespace Name", ns.last) - SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { p => + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.foreach { p => rows ++= Option(metadata.get(p)).map(toCatalystRow(p.capitalize, _)) } if (isExtended) { - val properties = metadata.asScala -- RESERVED_PROPERTIES.asScala + val properties = metadata.asScala -- CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES if (properties.nonEmpty) { rows += toCatalystRow("Properties", properties.toSeq.mkString("(", ",", ")")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 2815b0ac131f0..9c280206c548e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} -import org.apache.spark.sql.connector.catalog.{Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Table, TableCatalog} import org.apache.spark.sql.types.StructType case class DescribeTableExec( @@ -49,14 +49,14 @@ case class DescribeTableExec( rows += toCatalystRow("# Detailed Table Information", "", "") rows += toCatalystRow("Name", table.name(), "") - TableCatalog.RESERVED_PROPERTIES.asScala.toList.foreach(propKey => { + CatalogV2Util.TABLE_RESERVED_PROPERTIES.foreach(propKey => { if (table.properties.containsKey(propKey)) { rows += toCatalystRow(propKey.capitalize, table.properties.get(propKey), "") } }) val properties = table.properties.asScala.toList - .filter(kv => !TableCatalog.RESERVED_PROPERTIES.contains(kv._1)) + .filter(kv => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(kv._1)) .sortBy(_._1).map { case (key, value) => key + "=" + value }.mkString("[", ",", "]") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 8eea1cf9c06e4..cef9b5f675889 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -232,7 +232,7 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf) // validate that this catalog's reserved properties are not removed changes.foreach { case remove: RemoveProperty - if SupportsNamespaces.RESERVED_PROPERTIES.contains(remove.property) => + if CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.contains(remove.property) => throw new UnsupportedOperationException( s"Cannot remove reserved property: ${remove.property}") case _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index c19352a2267df..04e5a8dfd78ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -901,7 +901,7 @@ class DataSourceV2SQLSuite test("CreateNameSpace: reserved properties") { import SupportsNamespaces._ withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => val exception = intercept[ParseException] { sql(s"CREATE NAMESPACE testcat.reservedTest WITH DBPROPERTIES('$key'='dummyVal')") } @@ -909,7 +909,7 @@ class DataSourceV2SQLSuite } } withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => withNamespace("testcat.reservedTest") { sql(s"CREATE NAMESPACE testcat.reservedTest WITH DBPROPERTIES('$key'='foo')") assert(sql("DESC NAMESPACE EXTENDED testcat.reservedTest") @@ -928,7 +928,7 @@ class DataSourceV2SQLSuite test("create/replace/alter table - reserved properties") { import TableCatalog._ withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => + CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => Seq("CREATE", "REPLACE").foreach { action => val e = intercept[ParseException] { @@ -950,7 +950,7 @@ class DataSourceV2SQLSuite } } withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => + CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => withTable("testcat.reservedTest") { Seq("CREATE", "REPLACE").foreach { action => @@ -1108,7 +1108,7 @@ class DataSourceV2SQLSuite test("AlterNamespaceSetProperties: reserved properties") { import SupportsNamespaces._ withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => withNamespace("testcat.reservedTest") { sql("CREATE NAMESPACE testcat.reservedTest") val exception = intercept[ParseException] { @@ -1119,7 +1119,7 @@ class DataSourceV2SQLSuite } } withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - RESERVED_PROPERTIES.asScala.filterNot(_ == PROP_COMMENT).foreach { key => + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => withNamespace("testcat.reservedTest") { sql(s"CREATE NAMESPACE testcat.reservedTest") sql(s"ALTER NAMESPACE testcat.reservedTest SET PROPERTIES ('$key'='foo')") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index 9466ed92819c9..c399a011f9073 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.connector.catalog.{Identifier, NamespaceChange, SupportsNamespaces, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, NamespaceChange, SupportsNamespaces, TableChange} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -742,7 +742,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { actual: scala.collection.Map[String, String]): Unit = { // remove location and comment that are automatically added by HMS unless they are expected val toRemove = - SupportsNamespaces.RESERVED_PROPERTIES.asScala.filter(expected.contains) + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filter(expected.contains) assert(expected -- toRemove === actual) } @@ -1000,7 +1000,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { catalog.createNamespace(testNs, emptyProps) - SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { p => + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.foreach { p => val exc = intercept[UnsupportedOperationException] { catalog.alterNamespace(testNs, NamespaceChange.removeProperty(p)) } From 4847f7380d7559f693c6604f7e1e4b4a17d0dfed Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 24 Jan 2020 08:23:07 +0900 Subject: [PATCH 1211/1387] [SPARK-30298][SQL] Respect aliases in output partitioning of projects and aggregates ### What changes were proposed in this pull request? Currently, in the following scenario, bucket join is not utilized: ```scala val df = (0 until 20).map(i => (i, i)).toDF("i", "j").as("df") df.write.format("parquet").bucketBy(8, "i").saveAsTable("t") sql("CREATE VIEW v AS SELECT * FROM t") sql("SELECT * FROM t a JOIN v b ON a.i = b.i").explain ``` ``` == Physical Plan == *(4) SortMergeJoin [i#13], [i#15], Inner :- *(1) Sort [i#13 ASC NULLS FIRST], false, 0 : +- *(1) Project [i#13, j#14] : +- *(1) Filter isnotnull(i#13) : +- *(1) ColumnarToRow : +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct, SelectedBucketsCount: 8 out of 8 +- *(3) Sort [i#15 ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(i#15, 8), true, [id=#64] <----- Exchange node introduced +- *(2) Project [i#13 AS i#15, j#14 AS j#16] +- *(2) Filter isnotnull(i#13) +- *(2) ColumnarToRow +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:..., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct, SelectedBucketsCount: 8 out of 8 ``` Notice that `Exchange` is present. This is because `Project` introduces aliases and `outputPartitioning` and `requiredChildDistribution` do not consider aliases while considering bucket join in `EnsureRequirements`. This PR addresses to allow this scenario. ### Why are the changes needed? This allows bucket join to be utilized in the above example. ### Does this PR introduce any user-facing change? Yes, now with the fix, the `explain` out is as follows: ``` == Physical Plan == *(3) SortMergeJoin [i#13], [i#15], Inner :- *(1) Sort [i#13 ASC NULLS FIRST], false, 0 : +- *(1) Project [i#13, j#14] : +- *(1) Filter isnotnull(i#13) : +- *(1) ColumnarToRow : +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:.., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct, SelectedBucketsCount: 8 out of 8 +- *(2) Sort [i#15 ASC NULLS FIRST], false, 0 +- *(2) Project [i#13 AS i#15, j#14 AS j#16] +- *(2) Filter isnotnull(i#13) +- *(2) ColumnarToRow +- FileScan parquet default.t[i#13,j#14] Batched: true, DataFilters: [isnotnull(i#13)], Format: Parquet, Location: InMemoryFileIndex[file:.., PartitionFilters: [], PushedFilters: [IsNotNull(i)], ReadSchema: struct, SelectedBucketsCount: 8 out of 8 ``` Note that the `Exchange` is no longer present. ### How was this patch tested? Closes #26943 from imback82/bucket_alias. Authored-by: Terry Kim Signed-off-by: Takeshi Yamamuro --- .../AliasAwareOutputPartitioning.scala | 55 ++++++++++++ .../aggregate/HashAggregateExec.scala | 4 +- .../aggregate/ObjectHashAggregateExec.scala | 4 +- .../aggregate/SortAggregateExec.scala | 6 +- .../execution/basicPhysicalOperators.scala | 5 +- .../spark/sql/execution/PlannerSuite.scala | 88 +++++++++++++++++++ .../spark/sql/sources/BucketedReadSuite.scala | 14 +++ 7 files changed, 166 insertions(+), 10 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputPartitioning.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputPartitioning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputPartitioning.scala new file mode 100644 index 0000000000000..2c7faea019322 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/AliasAwareOutputPartitioning.scala @@ -0,0 +1,55 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} + +/** + * A trait that handles aliases in the `outputExpressions` to produce `outputPartitioning` + * that satisfies output distribution requirements. + */ +trait AliasAwareOutputPartitioning extends UnaryExecNode { + protected def outputExpressions: Seq[NamedExpression] + + final override def outputPartitioning: Partitioning = { + if (hasAlias) { + child.outputPartitioning match { + case h: HashPartitioning => h.copy(expressions = replaceAliases(h.expressions)) + case other => other + } + } else { + child.outputPartitioning + } + } + + private def hasAlias: Boolean = outputExpressions.collectFirst { case _: Alias => }.isDefined + + private def replaceAliases(exprs: Seq[Expression]): Seq[Expression] = { + exprs.map { + case a: AttributeReference => replaceAlias(a).getOrElse(a) + case other => other + } + } + + private def replaceAlias(attr: AttributeReference): Option[Attribute] = { + outputExpressions.collectFirst { + case a @ Alias(child: AttributeReference, _) if child.semanticEquals(attr) => + a.toAttribute + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 7f19d2754673d..f73e214a6b41f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -53,7 +53,7 @@ case class HashAggregateExec( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode with BlockingOperatorWithCodegen { + extends UnaryExecNode with BlockingOperatorWithCodegen with AliasAwareOutputPartitioning { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) @@ -75,7 +75,7 @@ case class HashAggregateExec( override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) - override def outputPartitioning: Partitioning = child.outputPartitioning + override protected def outputExpressions: Seq[NamedExpression] = resultExpressions override def producedAttributes: AttributeSet = AttributeSet(aggregateAttributes) ++ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index 953622afebf89..4376f6b6edd57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -67,7 +67,7 @@ case class ObjectHashAggregateExec( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode { + extends UnaryExecNode with AliasAwareOutputPartitioning { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) @@ -97,7 +97,7 @@ case class ObjectHashAggregateExec( } } - override def outputPartitioning: Partitioning = child.outputPartitioning + override protected def outputExpressions: Seq[NamedExpression] = resultExpressions protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { val numOutputRows = longMetric("numOutputRows") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 0ddf95771d5b2..b6e684e62ea5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{AliasAwareOutputPartitioning, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -38,7 +38,7 @@ case class SortAggregateExec( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode { + extends UnaryExecNode with AliasAwareOutputPartitioning { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) @@ -66,7 +66,7 @@ case class SortAggregateExec( groupingExpressions.map(SortOrder(_, Ascending)) :: Nil } - override def outputPartitioning: Partitioning = child.outputPartitioning + override protected def outputExpressions: Seq[NamedExpression] = resultExpressions override def outputOrdering: Seq[SortOrder] = { groupingExpressions.map(SortOrder(_, Ascending)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index f3f756425a153..c35c48496e1c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -38,7 +38,7 @@ import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler} /** Physical plan for Project. */ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode with CodegenSupport { + extends UnaryExecNode with CodegenSupport with AliasAwareOutputPartitioning { override def output: Seq[Attribute] = projectList.map(_.toAttribute) @@ -81,7 +81,7 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override def outputPartitioning: Partitioning = child.outputPartitioning + override protected def outputExpressions: Seq[NamedExpression] = projectList override def verboseStringWithOperatorId(): String = { s""" @@ -92,7 +92,6 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) } } - /** Physical plan for Filter. */ case class FilterExec(condition: Expression, child: SparkPlan) extends UnaryExecNode with CodegenSupport with PredicateHelper { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index a10db54855c8a..94ce3559bb44b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range, Repartition, Sort, Union} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ReuseExchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} @@ -937,6 +938,93 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { } } } + + test("aliases in the project should not introduce extra shuffle") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTempView("df1", "df2") { + spark.range(10).selectExpr("id AS key", "0").repartition($"key").createTempView("df1") + spark.range(20).selectExpr("id AS key", "0").repartition($"key").createTempView("df2") + val planned = sql( + """ + |SELECT * FROM + | (SELECT key AS k from df1) t1 + |INNER JOIN + | (SELECT key AS k from df2) t2 + |ON t1.k = t2.k + """.stripMargin).queryExecution.executedPlan + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + assert(exchanges.size == 2) + } + } + } + + test("aliases to expressions should not be replaced") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTempView("df1", "df2") { + spark.range(10).selectExpr("id AS key", "0").repartition($"key").createTempView("df1") + spark.range(20).selectExpr("id AS key", "0").repartition($"key").createTempView("df2") + val planned = sql( + """ + |SELECT * FROM + | (SELECT key + 1 AS k1 from df1) t1 + |INNER JOIN + | (SELECT key + 1 AS k2 from df2) t2 + |ON t1.k1 = t2.k2 + |""".stripMargin).queryExecution.executedPlan + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + + // Make sure aliases to an expression (key + 1) are not replaced. + Seq("k1", "k2").foreach { alias => + assert(exchanges.exists(_.outputPartitioning match { + case HashPartitioning(Seq(a: AttributeReference), _) => a.name == alias + case _ => false + })) + } + } + } + } + + test("aliases in the aggregate expressions should not introduce extra shuffle") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val t1 = spark.range(10).selectExpr("floor(id/4) as k1") + val t2 = spark.range(20).selectExpr("floor(id/4) as k2") + + val agg1 = t1.groupBy("k1").agg(count(lit("1")).as("cnt1")) + val agg2 = t2.groupBy("k2").agg(count(lit("1")).as("cnt2")).withColumnRenamed("k2", "k3") + + val planned = agg1.join(agg2, $"k1" === $"k3").queryExecution.executedPlan + + assert(planned.collect { case h: HashAggregateExec => h }.nonEmpty) + + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + assert(exchanges.size == 2) + } + } + + test("aliases in the object hash/sort aggregate expressions should not introduce extra shuffle") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + Seq(true, false).foreach { useObjectHashAgg => + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> useObjectHashAgg.toString) { + val t1 = spark.range(10).selectExpr("floor(id/4) as k1") + val t2 = spark.range(20).selectExpr("floor(id/4) as k2") + + val agg1 = t1.groupBy("k1").agg(collect_list("k1")) + val agg2 = t2.groupBy("k2").agg(collect_list("k2")).withColumnRenamed("k2", "k3") + + val planned = agg1.join(agg2, $"k1" === $"k3").queryExecution.executedPlan + + if (useObjectHashAgg) { + assert(planned.collect { case o: ObjectHashAggregateExec => o }.nonEmpty) + } else { + assert(planned.collect { case s: SortAggregateExec => s }.nonEmpty) + } + + val exchanges = planned.collect { case s: ShuffleExchangeExec => s } + assert(exchanges.size == 2) + } + } + } + } } // Used for unit-testing EnsureRequirements diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index a585f215ad681..c7266c886128c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -604,6 +604,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { } } + test("bucket join should work with SubqueryAlias plan") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { + withTable("t") { + withView("v") { + spark.range(20).selectExpr("id as i").write.bucketBy(8, "i").saveAsTable("t") + sql("CREATE VIEW v AS SELECT * FROM t").collect() + + val plan = sql("SELECT * FROM t a JOIN v b ON a.i = b.i").queryExecution.executedPlan + assert(plan.collect { case exchange: ShuffleExchangeExec => exchange }.isEmpty) + } + } + } + } + test("avoid shuffle when grouping keys are a super-set of bucket keys") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") From 3f76bd40025181841de70a11e576d0ee948de5c0 Mon Sep 17 00:00:00 2001 From: Xiao Li Date: Thu, 23 Jan 2020 15:34:54 -0800 Subject: [PATCH 1212/1387] [SPARK-27083][SQL][FOLLOW-UP] Rename spark.sql.subquery.reuse to spark.sql.execution.subquery.reuse.enabled ### What changes were proposed in this pull request? This PR is to rename spark.sql.subquery.reuse to spark.sql.execution.subquery.reuse.enabled ### Why are the changes needed? Make it consistent and clear. ### Does this PR introduce any user-facing change? N/A. This is a [new conf added in Spark 3.0](https://github.com/apache/spark/pull/23998) ### How was this patch tested? N/A Closes #27346 from gatorsmile/spark27083. Authored-by: Xiao Li Signed-off-by: Xiao Li --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4471c4d4d1b38..fe52d2ff11cd0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1053,7 +1053,7 @@ object SQLConf { .booleanConf .createWithDefault(true) - val SUBQUERY_REUSE_ENABLED = buildConf("spark.sql.subquery.reuse") + val SUBQUERY_REUSE_ENABLED = buildConf("spark.sql.execution.subquery.reuse.enabled") .internal() .doc("When true, the planner will try to find out duplicated subqueries and re-use them.") .booleanConf From ddf83159a8c61fa12237a60124f7a7aa4e3a53c1 Mon Sep 17 00:00:00 2001 From: Xiao Li Date: Thu, 23 Jan 2020 16:23:16 -0800 Subject: [PATCH 1213/1387] [SPARK-28962][SQL][FOLLOW-UP] Add the parameter description for the Scala function API filter ### What changes were proposed in this pull request? This PR is a follow-up PR https://github.com/apache/spark/pull/25666 for adding the description and example for the Scala function API `filter`. ### Why are the changes needed? It is hard to tell which parameter is the index column. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #27336 from gatorsmile/spark28962. Authored-by: Xiao Li Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/functions.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index fde6d3e576e47..b12d0371bdcdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3455,6 +3455,13 @@ object functions { /** * Returns an array of elements for which a predicate holds in a given array. + * {{{ + * df.select(filter(col("s"), x => x % 2 === 0)) + * df.selectExpr("filter(col, x -> x % 2 == 0)") + * }}} + * + * @param column: the input array column + * @param f: col => predicate, the boolean predicate to filter the input column * * @group collection_funcs * @since 3.0.0 @@ -3465,6 +3472,14 @@ object functions { /** * Returns an array of elements for which a predicate holds in a given array. + * {{{ + * df.select(filter(col("s"), (x, i) => i % 2 === 0)) + * df.selectExpr("filter(col, (x, i) -> i % 2 == 0)") + * }}} + * + * @param column: the input array column + * @param f: (col, index) => predicate, the boolean predicate to filter the input column + * given the index. Indices start at 0. * * @group collection_funcs * @since 3.0.0 From 53fd83a8c5f2311a52ed3645c13eab72a3d1cb94 Mon Sep 17 00:00:00 2001 From: Deepyaman Datta Date: Fri, 24 Jan 2020 13:10:09 +0900 Subject: [PATCH 1214/1387] [MINOR][DOCS] Fix src/dest type documentation for `to_timestamp` ### What changes were proposed in this pull request? Minor documentation fix ### Why are the changes needed? ### Does this PR introduce any user-facing change? ### How was this patch tested? Manually; consider adding tests? Closes #27295 from deepyaman/patch-2. Authored-by: Deepyaman Datta Signed-off-by: HyukjinKwon --- python/pyspark/sql/functions.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 7f4cda832972a..8b14dd915d541 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1135,12 +1135,11 @@ def months_between(date1, date2, roundOff=True): @since(2.2) def to_date(col, format=None): - """Converts a :class:`Column` of :class:`pyspark.sql.types.StringType` or - :class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType` + """Converts a :class:`Column` into :class:`pyspark.sql.types.DateType` using the optionally specified format. Specify formats according to `DateTimeFormatter `_. # noqa By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format - is omitted (equivalent to ``col.cast("date")``). + is omitted. Equivalent to ``col.cast("date")``. >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_date(df.t).alias('date')).collect() @@ -1160,12 +1159,11 @@ def to_date(col, format=None): @since(2.2) def to_timestamp(col, format=None): - """Converts a :class:`Column` of :class:`pyspark.sql.types.StringType` or - :class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType` + """Converts a :class:`Column` into :class:`pyspark.sql.types.TimestampType` using the optionally specified format. Specify formats according to `DateTimeFormatter `_. # noqa By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format - is omitted (equivalent to ``col.cast("timestamp")``). + is omitted. Equivalent to ``col.cast("timestamp")``. >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_timestamp(df.t).alias('dt')).collect() From ed44926117d81aa5fa8bd823d401efd235260872 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 23 Jan 2020 21:42:43 -0800 Subject: [PATCH 1215/1387] [SPARK-30627][SQL] Disable all the V2 file sources by default ### What changes were proposed in this pull request? Disable all the V2 file sources in Spark 3.0 by default. ### Why are the changes needed? There are still some missing parts in the file source V2 framework: 1. It doesn't support reporting file scan metrics such as "numOutputRows"/"numFiles"/"fileSize" like `FileSourceScanExec`. This requires another patch in the data source V2 framework. Tracked by [SPARK-30362](https://issues.apache.org/jira/browse/SPARK-30362) 2. It doesn't support partition pruning with subqueries(including dynamic partition pruning) for now. Tracked by [SPARK-30628](https://issues.apache.org/jira/browse/SPARK-30628) As we are going to code freeze on Jan 31st, this PR proposes to disable all the V2 file sources in Spark 3.0 by default. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #27348 from gengliangwang/disableFileSourceV2. Authored-by: Gengliang Wang Signed-off-by: Dongjoon Hyun --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- .../spark/sql/connector/FileDataSourceV2FallBackSuite.scala | 3 +++ .../execution/datasources/orc/OrcPartitionDiscoverySuite.scala | 2 ++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index fe52d2ff11cd0..4792022ce14b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1728,7 +1728,7 @@ object SQLConf { "implementation class names for which Data Source V2 code path is disabled. These data " + "sources will fallback to Data Source V1 code path.") .stringConf - .createWithDefault("kafka") + .createWithDefault("avro,csv,json,kafka,orc,parquet,text") val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers") .doc("A comma-separated list of fully qualified data source register class names for which" + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala index 21938f301dafb..b0da2eb697f36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.connector import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} @@ -86,6 +87,8 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { private val dummyReadOnlyFileSourceV2 = classOf[DummyReadOnlyFileDataSourceV2].getName private val dummyWriteOnlyFileSourceV2 = classOf[DummyWriteOnlyFileDataSourceV2].getName + override protected def sparkConf: SparkConf = super.sparkConf.set(SQLConf.USE_V1_SOURCE_LIST, "") + test("Fall back to v1 when writing to file with read only FileDataSourceV2") { val df = spark.range(10).toDF() withTempPath { file => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index 5d21ee698f4e6..ea839b8e1ef10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -169,6 +169,8 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { } class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSparkSession { + override protected def sparkConf: SparkConf = super.sparkConf.set(SQLConf.USE_V1_SOURCE_LIST, "") + test("read partitioned table - partition key included in orc file") { withTempDir { base => for { From d1a673a1bb6f0a3d2816ce7a2c4e6737b52b4c81 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 24 Jan 2020 11:49:24 -0800 Subject: [PATCH 1216/1387] [SPARK-29924][DOCS] Document Apache Arrow JDK11 requirement ### What changes were proposed in this pull request? This adds a note for additional setting for Apache Arrow library for Java 11. ### Why are the changes needed? Since Apache Arrow 0.14.0, an additional setting is required for Java 9+. - https://issues.apache.org/jira/browse/ARROW-3191 It's explicitly documented at Apache Arrow 0.15.0. - https://issues.apache.org/jira/browse/ARROW-6206 However, there is no plan to handle that inside Apache Arrow side. - https://issues.apache.org/jira/browse/ARROW-7223 In short, we need to document this for the users who is using Arrow-related feature on JDK11. For dev environment, we handle this via [SPARK-29923](https://github.com/apache/spark/pull/26552) . ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? Generated document and see the pages. ![doc](https://user-images.githubusercontent.com/9700541/73096611-0f409d80-3e9a-11ea-804b-c6b5ec7bd78d.png) Closes #27356 from dongjoon-hyun/SPARK-JDK11-ARROW-DOC. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/index.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/index.md b/docs/index.md index d86471d56d5d9..f6ec595231f9e 100644 --- a/docs/index.md +++ b/docs/index.md @@ -52,6 +52,8 @@ For the Scala API, Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version ({{site.SCALA_BINARY_VERSION}}.x). +For Java 11, `-Dio.netty.tryReflectionSetAccessible=true` is required additionally for Apache Arrow library. This prevents `java.lang.UnsupportedOperationException: sun.misc.Unsafe or java.nio.DirectByteBuffer.(long, int) not available` when Apache Arrow uses Netty internally. + # Running the Examples and Shell Spark comes with several sample programs. Scala, Java, Python and R examples are in the From f86a1b9590c2fc661232e2f7fd5859daf118729e Mon Sep 17 00:00:00 2001 From: Jiaxin Shan Date: Fri, 24 Jan 2020 12:00:30 -0800 Subject: [PATCH 1217/1387] [SPARK-30626][K8S] Add SPARK_APPLICATION_ID into driver pod env ### What changes were proposed in this pull request? Add SPARK_APPLICATION_ID environment when spark configure driver pod. ### Why are the changes needed? Currently, driver doesn't have this in environments and it's no convenient to retrieve spark id. The use case is we want to look up spark application id and create application folder and redirect driver logs to application folder. ### Does this PR introduce any user-facing change? no ### How was this patch tested? unit tested. I also build new distribution and container image to kick off a job in Kubernetes and I do see SPARK_APPLICATION_ID added there. . Closes #27347 from Jeffwan/SPARK-30626. Authored-by: Jiaxin Shan Signed-off-by: Dongjoon Hyun --- .../spark/deploy/k8s/features/BasicDriverFeatureStep.scala | 4 +++- .../deploy/k8s/features/BasicDriverFeatureStepSuite.scala | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 1944ba9766cff..e2d70d7692ef3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -70,7 +70,9 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB override def configurePod(pod: SparkPod): SparkPod = { - val driverCustomEnvs = conf.environment.toSeq + val driverCustomEnvs = (Seq( + (ENV_APPLICATION_ID, conf.appId) + ) ++ conf.environment) .map { env => new EnvVarBuilder() .withName(env._1) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 370672162ef0d..710f28a859347 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -93,6 +93,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { assert(envs(v) === v) } assert(envs(ENV_SPARK_USER) === Utils.getCurrentUserName()) + assert(envs(ENV_APPLICATION_ID) === kubernetesConf.appId) assert(configuredPod.pod.getSpec().getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) From 2f8e4d0d6e56188fa24528741a514ce1f04d2bf2 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 24 Jan 2020 12:12:46 -0800 Subject: [PATCH 1218/1387] [SPARK-30630][ML] Remove numTrees in GBT in 3.0.0 ### What changes were proposed in this pull request? Remove ```numTrees``` in GBT in 3.0.0. ### Why are the changes needed? Currently, GBT has ``` /** * Number of trees in ensemble */ Since("2.0.0") val getNumTrees: Int = trees.length ``` and ``` /** Number of trees in ensemble */ val numTrees: Int = trees.length ``` I think we should remove one of them. We deprecated it in 2.4.5 via https://github.com/apache/spark/pull/27352. ### Does this PR introduce any user-facing change? Yes, remove ```numTrees``` in GBT in 3.0.0 ### How was this patch tested? existing tests Closes #27330 from huaxingao/spark-numTrees. Authored-by: Huaxin Gao Signed-off-by: Dongjoon Hyun --- .../spark/ml/classification/GBTClassifier.scala | 7 ++----- .../apache/spark/ml/regression/GBTRegressor.scala | 7 ++----- .../ml/classification/GBTClassifierSuite.scala | 13 +++++++------ .../spark/ml/regression/GBTRegressorSuite.scala | 10 +++++----- project/MimaExcludes.scala | 4 ++++ 5 files changed, 20 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 4044c08789219..46810bccc8e69 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -342,9 +342,6 @@ class GBTClassificationModel private[ml]( } } - /** Number of trees in ensemble */ - val numTrees: Int = trees.length - @Since("1.4.0") override def copy(extra: ParamMap): GBTClassificationModel = { copyValues(new GBTClassificationModel(uid, _trees, _treeWeights, numFeatures, numClasses), @@ -353,7 +350,7 @@ class GBTClassificationModel private[ml]( @Since("1.4.0") override def toString: String = { - s"GBTClassificationModel: uid = $uid, numTrees=$numTrees, numClasses=$numClasses, " + + s"GBTClassificationModel: uid = $uid, numTrees=$getNumTrees, numClasses=$numClasses, " + s"numFeatures=$numFeatures" } @@ -374,7 +371,7 @@ class GBTClassificationModel private[ml]( /** Raw prediction for the positive class. */ private def margin(features: Vector): Double = { val treePredictions = _trees.map(_.rootNode.predictImpl(features).prediction) - blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) + blas.ddot(getNumTrees, treePredictions, 1, _treeWeights, 1) } /** (private[ml]) Convert to a model in the old API */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index f348dd2b72e73..78d5ddaa2758b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -299,12 +299,9 @@ class GBTRegressionModel private[ml]( // TODO: When we add a generic Boosting class, handle transform there? SPARK-7129 // Classifies by thresholding sum of weighted tree predictions val treePredictions = _trees.map(_.rootNode.predictImpl(features).prediction) - blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) + blas.ddot(getNumTrees, treePredictions, 1, _treeWeights, 1) } - /** Number of trees in ensemble */ - val numTrees: Int = trees.length - @Since("1.4.0") override def copy(extra: ParamMap): GBTRegressionModel = { copyValues(new GBTRegressionModel(uid, _trees, _treeWeights, numFeatures), @@ -313,7 +310,7 @@ class GBTRegressionModel private[ml]( @Since("1.4.0") override def toString: String = { - s"GBTRegressionModel: uid=$uid, numTrees=$numTrees, numFeatures=$numFeatures" + s"GBTRegressionModel: uid=$uid, numTrees=$getNumTrees, numFeatures=$numFeatures" } /** diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index abeb4b5331acf..a2208edcb8394 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -179,7 +179,8 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { assert(raw.size === 2) // check that raw prediction is tree predictions dot tree weights val treePredictions = gbtModel.trees.map(_.rootNode.predictImpl(features).prediction) - val prediction = blas.ddot(gbtModel.numTrees, treePredictions, 1, gbtModel.treeWeights, 1) + val prediction = blas.ddot(gbtModel.getNumTrees, treePredictions, 1, + gbtModel.treeWeights, 1) assert(raw ~== Vectors.dense(-prediction, prediction) relTol eps) // Compare rawPrediction with probability @@ -436,9 +437,9 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { gbt.setValidationIndicatorCol(validationIndicatorCol) val modelWithValidation = gbt.fit(trainDF.union(validationDF)) - assert(modelWithoutValidation.numTrees === numIter) + assert(modelWithoutValidation.getNumTrees === numIter) // early stop - assert(modelWithValidation.numTrees < numIter) + assert(modelWithValidation.getNumTrees < numIter) val (errorWithoutValidation, errorWithValidation) = { val remappedRdd = validationData.map { @@ -457,10 +458,10 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { modelWithoutValidation.treeWeights, modelWithoutValidation.getOldLossType, OldAlgo.Classification) assert(evaluationArray.length === numIter) - assert(evaluationArray(modelWithValidation.numTrees) > - evaluationArray(modelWithValidation.numTrees - 1)) + assert(evaluationArray(modelWithValidation.getNumTrees) > + evaluationArray(modelWithValidation.getNumTrees - 1)) var i = 1 - while (i < modelWithValidation.numTrees) { + while (i < modelWithValidation.getNumTrees) { assert(evaluationArray(i) <= evaluationArray(i - 1)) i += 1 } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index 35c0fc9b02b10..04b0d4b8470f3 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -274,9 +274,9 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { gbt.setValidationIndicatorCol(validationIndicatorCol) val modelWithValidation = gbt.fit(trainDF.union(validationDF)) - assert(modelWithoutValidation.numTrees === numIter) + assert(modelWithoutValidation.getNumTrees === numIter) // early stop - assert(modelWithValidation.numTrees < numIter) + assert(modelWithValidation.getNumTrees < numIter) val errorWithoutValidation = GradientBoostedTrees.computeWeightedError( validationData.map(_.toInstance), @@ -294,10 +294,10 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { modelWithoutValidation.treeWeights, modelWithoutValidation.getOldLossType, OldAlgo.Regression) assert(evaluationArray.length === numIter) - assert(evaluationArray(modelWithValidation.numTrees) > - evaluationArray(modelWithValidation.numTrees - 1)) + assert(evaluationArray(modelWithValidation.getNumTrees) > + evaluationArray(modelWithValidation.getNumTrees - 1)) var i = 1 - while (i < modelWithValidation.numTrees) { + while (i < modelWithValidation.getNumTrees) { assert(evaluationArray(i) <= evaluationArray(i - 1)) i += 1 } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index f1bbe0b10e228..68e9313805e19 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -344,6 +344,10 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.layers"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel.this"), + // [SPARK-30630][ML] Remove numTrees in GBT + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.classification.GBTClassificationModel.numTrees"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.regression.GBTRegressionModel.numTrees"), + // Data Source V2 API changes (problem: Problem) => problem match { case MissingClassProblem(cls) => From a0e63b61e7c5d55ae2a9213b95ab1e87ac7c203c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 24 Jan 2020 22:17:28 -0800 Subject: [PATCH 1219/1387] [SPARK-29721][SQL] Prune unnecessary nested fields from Generate without Project ### What changes were proposed in this pull request? This patch proposes to prune unnecessary nested fields from Generate which has no Project on top of it. ### Why are the changes needed? In Optimizer, we can prune nested columns from Project(projectList, Generate). However, unnecessary columns could still possibly be read in Generate, if no Project on top of it. We should prune it too. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test. Closes #26978 from viirya/SPARK-29721. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../optimizer/NestedColumnAliasing.scala | 47 +++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 43 +++++++---------- .../datasources/SchemaPruningSuite.scala | 32 +++++++++++++ 3 files changed, 97 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index 43a6006f9b5c0..ea85014a37bd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -155,6 +155,53 @@ object NestedColumnAliasing { case MapType(keyType, valueType, _) => totalFieldNum(keyType) + totalFieldNum(valueType) case _ => 1 // UDT and others } +} + +/** + * This prunes unnessary nested columns from `Generate` and optional `Project` on top + * of it. + */ +object GeneratorNestedColumnAliasing { + def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { + // Either `nestedPruningOnExpressions` or `nestedSchemaPruningEnabled` is enabled, we + // need to prune nested columns through Project and under Generate. The difference is + // when `nestedSchemaPruningEnabled` is on, nested columns will be pruned further at + // file format readers if it is supported. + case Project(projectList, g: Generate) if (SQLConf.get.nestedPruningOnExpressions || + SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) => + // On top on `Generate`, a `Project` that might have nested column accessors. + // We try to get alias maps for both project list and generator's children expressions. + NestedColumnAliasing.getAliasSubMap(projectList ++ g.generator.children).map { + case (nestedFieldToAlias, attrToAliases) => + val newChild = pruneGenerate(g, nestedFieldToAlias, attrToAliases) + Project(NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias), newChild) + } + + case g: Generate if SQLConf.get.nestedSchemaPruningEnabled && + canPruneGenerator(g.generator) => + NestedColumnAliasing.getAliasSubMap(g.generator.children).map { + case (nestedFieldToAlias, attrToAliases) => + pruneGenerate(g, nestedFieldToAlias, attrToAliases) + } + + case _ => + None + } + + private def pruneGenerate( + g: Generate, + nestedFieldToAlias: Map[ExtractValue, Alias], + attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = { + val newGenerator = g.generator.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f) => + nestedFieldToAlias(f).toAttribute + }.asInstanceOf[Generator] + + // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. + val newGenerate = g.copy(generator = newGenerator) + + NestedColumnAliasing.replaceChildrenWithAliases(newGenerate, attrToAliases) + } /** * This is a while-list for pruning nested fields at `Generator`. 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 935d62015afa1..0fdf6b022d885 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 @@ -598,31 +598,24 @@ object ColumnPruning extends Rule[LogicalPlan] { s.copy(child = prunedChild(child, s.references)) // prune unrequired references - case p @ Project(_, g: Generate) if p.references != g.outputSet => - val requiredAttrs = p.references -- g.producedAttributes ++ g.generator.references - val newChild = prunedChild(g.child, requiredAttrs) - val unrequired = g.generator.references -- p.references - val unrequiredIndices = newChild.output.zipWithIndex.filter(t => unrequired.contains(t._1)) - .map(_._2) - p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices)) - - // prune unrequired nested fields - case p @ Project(projectList, g: Generate) if SQLConf.get.nestedPruningOnExpressions && - NestedColumnAliasing.canPruneGenerator(g.generator) => - NestedColumnAliasing.getAliasSubMap(projectList ++ g.generator.children).map { - case (nestedFieldToAlias, attrToAliases) => - val newGenerator = g.generator.transform { - case f: ExtractValue if nestedFieldToAlias.contains(f) => - nestedFieldToAlias(f).toAttribute - }.asInstanceOf[Generator] - - // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. - val newGenerate = g.copy(generator = newGenerator) - - val newChild = NestedColumnAliasing.replaceChildrenWithAliases(newGenerate, attrToAliases) - - Project(NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias), newChild) - }.getOrElse(p) + case p @ Project(_, g: Generate) => + val currP = if (p.references != g.outputSet) { + val requiredAttrs = p.references -- g.producedAttributes ++ g.generator.references + val newChild = prunedChild(g.child, requiredAttrs) + val unrequired = g.generator.references -- p.references + val unrequiredIndices = newChild.output.zipWithIndex.filter(t => unrequired.contains(t._1)) + .map(_._2) + p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices)) + } else { + p + } + // If we can prune nested column on Project + Generate, do it now. + // Otherwise by transforming down to Generate, it could be pruned individually, + // and causes nested column on top Project unable to resolve. + GeneratorNestedColumnAliasing.unapply(currP).getOrElse(currP) + + // prune unrequired nested fields from `Generate`. + case GeneratorNestedColumnAliasing(p) => p // Eliminate unneeded attributes from right side of a Left Existence Join. case j @ Join(_, right, LeftExistence(_), _, _) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index a3d4905e82cee..5977e867f788a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -301,6 +301,38 @@ abstract class SchemaPruningSuite checkAnswer(query, Row("Y.", 1) :: Row("X.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } + testSchemaPruning("select explode of nested field of array of struct") { + // Config combinations + val configs = Seq((true, true), (true, false), (false, true), (false, false)) + + configs.foreach { case (nestedPruning, nestedPruningOnExpr) => + withSQLConf( + SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> nestedPruning.toString, + SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> nestedPruningOnExpr.toString) { + val query1 = spark.table("contacts") + .select(explode(col("friends.first"))) + if (nestedPruning) { + // If `NESTED_SCHEMA_PRUNING_ENABLED` is enabled, + // even disabling `NESTED_PRUNING_ON_EXPRESSIONS`, + // nested schema is still pruned at scan node. + checkScan(query1, "struct>>") + } else { + checkScan(query1, "struct>>") + } + checkAnswer(query1, Row("Susan") :: Nil) + + val query2 = spark.table("contacts") + .select(explode(col("friends.first")), col("friends.middle")) + if (nestedPruning) { + checkScan(query2, "struct>>") + } else { + checkScan(query2, "struct>>") + } + checkAnswer(query2, Row("Susan", Array("Z.")) :: Nil) + } + } + } + protected def testSchemaPruning(testName: String)(testThunk: => Unit): Unit = { test(s"Spark vectorized reader - without partition data column - $testName") { withSQLConf(vectorizedReaderEnabledKey -> "true") { From 862959747ec3eb1f90d23ec91c1c6419468c9ea9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 25 Jan 2020 15:41:55 -0800 Subject: [PATCH 1220/1387] [SPARK-30639][BUILD] Upgrade Jersey to 2.30 ### What changes were proposed in this pull request? For better JDK11 support, this PR aims to upgrade **Jersey** and **javassist** to `2.30` and `3.35.0-GA` respectively. ### Why are the changes needed? **Jersey**: This will bring the following `Jersey` updates. - https://eclipse-ee4j.github.io/jersey.github.io/release-notes/2.30.html - https://github.com/eclipse-ee4j/jersey/issues/4245 (Java 11 java.desktop module dependency) **javassist**: This is a transitive dependency from 3.20.0-CR2 to 3.25.0-GA. - `javassist` officially supports JDK11 from [3.24.0-GA release note](https://github.com/jboss-javassist/javassist/blob/master/Readme.html#L308). ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with both JDK8 and JDK11. Closes #27357 from dongjoon-hyun/SPARK-30639. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 16 ++++++++-------- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 16 ++++++++-------- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 16 ++++++++-------- pom.xml | 2 +- 4 files changed, 25 insertions(+), 25 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index c38379097157c..e75d2cd800b34 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -106,7 +106,7 @@ jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar janino/3.0.15//janino-3.0.15.jar -javassist/3.22.0-CR2//javassist-3.22.0-CR2.jar +javassist/3.25.0-GA//javassist-3.25.0-GA.jar javax.inject/1//javax.inject-1.jar javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar javolution/5.5.1//javolution-5.5.1.jar @@ -114,13 +114,13 @@ jaxb-api/2.2.2//jaxb-api-2.2.2.jar jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar jcl-over-slf4j/1.7.16//jcl-over-slf4j-1.7.16.jar jdo-api/3.0.1//jdo-api-3.0.1.jar -jersey-client/2.29.1//jersey-client-2.29.1.jar -jersey-common/2.29.1//jersey-common-2.29.1.jar -jersey-container-servlet-core/2.29.1//jersey-container-servlet-core-2.29.1.jar -jersey-container-servlet/2.29.1//jersey-container-servlet-2.29.1.jar -jersey-hk2/2.29.1//jersey-hk2-2.29.1.jar -jersey-media-jaxb/2.29.1//jersey-media-jaxb-2.29.1.jar -jersey-server/2.29.1//jersey-server-2.29.1.jar +jersey-client/2.30//jersey-client-2.30.jar +jersey-common/2.30//jersey-common-2.30.jar +jersey-container-servlet-core/2.30//jersey-container-servlet-core-2.30.jar +jersey-container-servlet/2.30//jersey-container-servlet-2.30.jar +jersey-hk2/2.30//jersey-hk2-2.30.jar +jersey-media-jaxb/2.30//jersey-media-jaxb-2.30.jar +jersey-server/2.30//jersey-server-2.30.jar jetty-sslengine/6.1.26//jetty-sslengine-6.1.26.jar jetty-util/6.1.26//jetty-util-6.1.26.jar jetty/6.1.26//jetty-6.1.26.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 4af1c0728e47f..f27eb64bd121e 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -120,7 +120,7 @@ jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar janino/3.0.15//janino-3.0.15.jar -javassist/3.22.0-CR2//javassist-3.22.0-CR2.jar +javassist/3.25.0-GA//javassist-3.25.0-GA.jar javax.inject/1//javax.inject-1.jar javax.jdo/3.2.0-m3//javax.jdo-3.2.0-m3.jar javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar @@ -129,13 +129,13 @@ jaxb-api/2.2.2//jaxb-api-2.2.2.jar jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar jcl-over-slf4j/1.7.16//jcl-over-slf4j-1.7.16.jar jdo-api/3.0.1//jdo-api-3.0.1.jar -jersey-client/2.29.1//jersey-client-2.29.1.jar -jersey-common/2.29.1//jersey-common-2.29.1.jar -jersey-container-servlet-core/2.29.1//jersey-container-servlet-core-2.29.1.jar -jersey-container-servlet/2.29.1//jersey-container-servlet-2.29.1.jar -jersey-hk2/2.29.1//jersey-hk2-2.29.1.jar -jersey-media-jaxb/2.29.1//jersey-media-jaxb-2.29.1.jar -jersey-server/2.29.1//jersey-server-2.29.1.jar +jersey-client/2.30//jersey-client-2.30.jar +jersey-common/2.30//jersey-common-2.30.jar +jersey-container-servlet-core/2.30//jersey-container-servlet-core-2.30.jar +jersey-container-servlet/2.30//jersey-container-servlet-2.30.jar +jersey-hk2/2.30//jersey-hk2-2.30.jar +jersey-media-jaxb/2.30//jersey-media-jaxb-2.30.jar +jersey-server/2.30//jersey-server-2.30.jar jetty-sslengine/6.1.26//jetty-sslengine-6.1.26.jar jetty-util/6.1.26//jetty-util-6.1.26.jar jetty/6.1.26//jetty-6.1.26.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 5aa0f0d4c5a32..b1d98fe0f62c9 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -119,7 +119,7 @@ jakarta.validation-api/2.0.2//jakarta.validation-api-2.0.2.jar jakarta.ws.rs-api/2.1.6//jakarta.ws.rs-api-2.1.6.jar jakarta.xml.bind-api/2.3.2//jakarta.xml.bind-api-2.3.2.jar janino/3.0.15//janino-3.0.15.jar -javassist/3.22.0-CR2//javassist-3.22.0-CR2.jar +javassist/3.25.0-GA//javassist-3.25.0-GA.jar javax.inject/1//javax.inject-1.jar javax.jdo/3.2.0-m3//javax.jdo-3.2.0-m3.jar javax.servlet-api/3.1.0//javax.servlet-api-3.1.0.jar @@ -129,13 +129,13 @@ jaxb-runtime/2.3.2//jaxb-runtime-2.3.2.jar jcip-annotations/1.0-1//jcip-annotations-1.0-1.jar jcl-over-slf4j/1.7.16//jcl-over-slf4j-1.7.16.jar jdo-api/3.0.1//jdo-api-3.0.1.jar -jersey-client/2.29.1//jersey-client-2.29.1.jar -jersey-common/2.29.1//jersey-common-2.29.1.jar -jersey-container-servlet-core/2.29.1//jersey-container-servlet-core-2.29.1.jar -jersey-container-servlet/2.29.1//jersey-container-servlet-2.29.1.jar -jersey-hk2/2.29.1//jersey-hk2-2.29.1.jar -jersey-media-jaxb/2.29.1//jersey-media-jaxb-2.29.1.jar -jersey-server/2.29.1//jersey-server-2.29.1.jar +jersey-client/2.30//jersey-client-2.30.jar +jersey-common/2.30//jersey-common-2.30.jar +jersey-container-servlet-core/2.30//jersey-container-servlet-core-2.30.jar +jersey-container-servlet/2.30//jersey-container-servlet-2.30.jar +jersey-hk2/2.30//jersey-hk2-2.30.jar +jersey-media-jaxb/2.30//jersey-media-jaxb-2.30.jar +jersey-server/2.30//jersey-server-2.30.jar jline/2.14.6//jline-2.14.6.jar joda-time/2.10.5//joda-time-2.10.5.jar jodd-core/3.5.2//jodd-core-3.5.2.jar diff --git a/pom.xml b/pom.xml index 91191f9dabe97..6a74b8b2e8e28 100644 --- a/pom.xml +++ b/pom.xml @@ -179,7 +179,7 @@ 2.6.2 4.1.17 3.0.15 - 2.29.1 + 2.30 2.10.5 3.5.2 3.0.0 From d5b92b24c41b047c64a4d89cc4061ebf534f0995 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Sun, 26 Jan 2020 09:11:33 +0900 Subject: [PATCH 1221/1387] [SPARK-30579][DOC] Document ORDER BY Clause of SELECT statement in SQL Reference ### What changes were proposed in this pull request? Document ORDER BY clause of SELECT statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2020-01-19 at 11 50 57 PM Screen Shot 2020-01-19 at 11 51 14 PM Screen Shot 2020-01-19 at 11 51 33 PM ### How was this patch tested? Tested using jykyll build --serve Closes #27288 from dilipbiswal/sql-ref-select-orderby. Authored-by: Dilip Biswal Signed-off-by: Takeshi Yamamuro --- docs/sql-ref-syntax-qry-select-orderby.md | 123 +++++++++++++++++++++- 1 file changed, 122 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-qry-select-orderby.md b/docs/sql-ref-syntax-qry-select-orderby.md index 1f7c031e4aa88..1a5d2d404e2c0 100644 --- a/docs/sql-ref-syntax-qry-select-orderby.md +++ b/docs/sql-ref-syntax-qry-select-orderby.md @@ -18,5 +18,126 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- +The ORDER BY clause is used to return the result rows in a sorted manner +in the user specified order. Unlike the SORT BY clause, this clause guarantees +a total order in the output. -**This page is under construction** +### Syntax +{% highlight sql %} +ORDER BY { expression [ sort_direction | nulls_sort_oder ] [ , ... ] } +{% endhighlight %} + +### Parameters +

    +
    ORDER BY
    +
    + Specifies a comma-separated list of expressions along with optional parameters sort_direction + and nulls_sort_order which are used to sort the rows. +
    +
    sort_direction
    +
    + Optionally specifies whether to sort the rows in ascending or descending + order. The valid values for the sort direction are ASC for ascending + and DESC for descending. If sort direction is not explicitly specified, then by default + rows are sorted ascending.

    + Syntax: + + [ ASC | DESC ] + +
    +
    nulls_sort_order
    +
    + Optionally specifies whether NULL values are returned before/after non-NULL values, based on the + sort direction. In Spark, NULL values are considered to be lower than any non-NULL values by default. + Therefore the ordering of NULL values depend on the sort direction. If null_sort_order is + not specified, then NULLs sort first if sort order is ASC and NULLS sort last if + sort order is DESC.

    +
      +
    1. If NULLS FIRST (the default) is specified, then NULL values are returned first + regardless of the sort order.
    2. +
    3. If NULLS LAST is specified, then NULL values are returned last regardless of + the sort order.
    4. +

    + Syntax: + + [ NULLS { FIRST | LAST } ] + +
    +
    + +### Examples +{% highlight sql %} +CREATE TABLE person (id INT, name STRING, age INT); +INSERT INTO person VALUES + (100, 'John', 30), + (200, 'Mary', NULL), + (300, 'Mike', 80), + (400, 'Jerry', NULL), + (500, 'Dan', 50); + +-- Sort rows by age. By default rows are sorted in ascending manner. +SELECT name, age FROM person ORDER BY age; + + +-----+----+ + |name |age | + +-----+----+ + |Jerry|null| + |Mary |null| + |John |30 | + |Dan |50 | + |Mike |80 | + +-----+----+ + +-- Sort rows in ascending manner keeping null values to be last. +SELECT name, age FROM person ORDER BY age NULLS LAST; + + +-----+----+ + |name |age | + +-----+----+ + |John |30 | + |Dan |50 | + |Mike |80 | + |Mary |null| + |Jerry|null| + +-----+----+ + +-- Sort rows by age in descending manner. +SELECT name, age FROM person ORDER BY age DESC; + + +-----+----+ + |name |age | + +-----+----+ + |Mike |80 | + |Dan |50 | + |John |30 | + |Jerry|null| + |Mary |null| + +-----+----+ + +-- Sort rows in ascending manner keeping null values to be first. +SELECT name, age FROM person ORDER BY age DESC NULLS FIRST; + + +-----+----+ + |name |age | + +-----+----+ + |Jerry|null| + |Mary |null| + |Mike |80 | + |Dan |50 | + |John |30 | + +-----+----+ + +-- Sort rows based on more than one column with each column having different +-- sort direction. +SELECT * FROM person ORDER BY name ASC, age DESC; + + +---+-----+----+ + |id |name |age | + +---+-----+----+ + |500|Dan |50 | + |400|Jerry|null| + |100|John |30 | + |200|Mary |null| + |300|Mike |80 | + +---+-----+----+ +{% endhighlight %} From 40b1f4d87e0f24e4e7e2fd6fe37cc5398ae778f8 Mon Sep 17 00:00:00 2001 From: zero323 Date: Sun, 26 Jan 2020 12:59:53 +0900 Subject: [PATCH 1222/1387] [SPARK-30645][SPARKR][TESTS][WINDOWS] Move Unicode test data to external file MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Reference data for "collect() support Unicode characters" has been moved to an external file, to make test OS and locale independent. ### Why are the changes needed? As-is, embedded data is not properly encoded on Windows: ``` library(SparkR) SparkR::sparkR.session() Sys.info() # sysname release version # "Windows" "Server x64" "build 17763" # nodename machine login # "WIN-5BLT6Q610KH" "x86-64" "Administrator" # user effective_user # "Administrator" "Administrator" Sys.getlocale() # [1] "LC_COLLATE=English_United States.1252;LC_CTYPE=English_United States.1252;LC_MONETARY=English_United States.1252;LC_NUMERIC=C;LC_TIME=English_United States.1252" lines <- c("{\"name\":\"안녕하세요\"}", "{\"name\":\"您好\", \"age\":30}", "{\"name\":\"こんにちは\", \"age\":19}", "{\"name\":\"Xin chào\"}") system(paste0("cat ", jsonPath)) # {"name":""} # {"name":"", "age":30} # {"name":"", "age":19} # {"name":"Xin chào"} # [1] 0 jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPath) df <- read.df(jsonPath, "json") printSchema(df) # root # |-- _corrupt_record: string (nullable = true) # |-- age: long (nullable = true) # |-- name: string (nullable = true) head(df) # _corrupt_record age name # 1 NA # 2 30 # 3 19 # 4 {"name":"Xin cho"} NA ``` This can be reproduced outside tests (Windows Server 2019, English locale), and causes failures, when `testthat` is updated to 2.x (https://github.com/apache/spark/pull/27359). Somehow problem is not picked-up when test is executed on `testthat` 1.0.2. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Running modified test, manual testing. ### Note Alternative seems to be to used bytes, but it hasn't been properly tested. ``` test_that("collect() support Unicode characters", { lines <- markUtf8(c( '{"name": "안녕하세요"}', '{"name": "您好", "age": 30}', '{"name": "こんにちは", "age": 19}', '{"name": "Xin ch\xc3\xa0o"}' )) jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPath, useBytes = TRUE) expected <- regmatches(lines, regexec('(?<="name": ").*?(?=")', lines, perl = TRUE)) df <- read.df(jsonPath, "json") rdf <- collect(df) expect_true(is.data.frame(rdf)) rdf$name <- markUtf8(rdf$name) expect_equal(rdf$name[1], expected[[1]]) expect_equal(rdf$name[2], expected[[2]]) expect_equal(rdf$name[3], expected[[3]]) expect_equal(rdf$name[4], expected[[4]]) df1 <- createDataFrame(rdf) expect_equal( collect( where(df1, df1$name == expected[[2]]) )$name, expected[[2]] ) }) ``` Closes #27362 from zero323/SPARK-30645. Authored-by: zero323 Signed-off-by: HyukjinKwon --- .../tests/fulltests/data/test_utils_utf.json | 4 +++ R/pkg/tests/fulltests/test_sparkSQL.R | 29 ++++++++++++------- 2 files changed, 22 insertions(+), 11 deletions(-) create mode 100644 R/pkg/tests/fulltests/data/test_utils_utf.json diff --git a/R/pkg/tests/fulltests/data/test_utils_utf.json b/R/pkg/tests/fulltests/data/test_utils_utf.json new file mode 100644 index 0000000000000..b78352ee52ef1 --- /dev/null +++ b/R/pkg/tests/fulltests/data/test_utils_utf.json @@ -0,0 +1,4 @@ +{"name": "안녕하세요"} +{"name": "您好", "age": 30} +{"name": "こんにちは", "age": 19} +{"name": "Xin chào"} diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index d435a8b6d7c4a..4fcc2baa05467 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -848,24 +848,31 @@ test_that("collect() and take() on a DataFrame return the same number of rows an }) test_that("collect() support Unicode characters", { - lines <- c("{\"name\":\"안녕하세요\"}", - "{\"name\":\"您好\", \"age\":30}", - "{\"name\":\"こんにちは\", \"age\":19}", - "{\"name\":\"Xin chào\"}") + jsonPath <- file.path( + Sys.getenv("SPARK_HOME"), + "R", "pkg", "tests", "fulltests", "data", + "test_utils_utf.json" + ) + + lines <- readLines(jsonPath, encoding = "UTF-8") - jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") - writeLines(lines, jsonPath) + expected <- regmatches(lines, gregexpr('(?<="name": ").*?(?=")', lines, perl = TRUE)) df <- read.df(jsonPath, "json") rdf <- collect(df) expect_true(is.data.frame(rdf)) - expect_equal(rdf$name[1], markUtf8("안녕하세요")) - expect_equal(rdf$name[2], markUtf8("您好")) - expect_equal(rdf$name[3], markUtf8("こんにちは")) - expect_equal(rdf$name[4], markUtf8("Xin chào")) + expect_equal(rdf$name[1], expected[[1]]) + expect_equal(rdf$name[2], expected[[2]]) + expect_equal(rdf$name[3], expected[[3]]) + expect_equal(rdf$name[4], expected[[4]]) df1 <- createDataFrame(rdf) - expect_equal(collect(where(df1, df1$name == markUtf8("您好")))$name, markUtf8("您好")) + expect_equal( + collect( + where(df1, df1$name == expected[[2]]) + )$name, + expected[[2]] + ) }) test_that("multiple pipeline transformations result in an RDD with the correct values", { From c3f17fa03952e4d299de2ca64cb83fe7e13218c7 Mon Sep 17 00:00:00 2001 From: zero323 Date: Sat, 25 Jan 2020 21:16:22 -0800 Subject: [PATCH 1223/1387] [SPARK-29777][FOLLOW-UP][SPARKR] Remove no longer valid test for recursive calls ### What changes were proposed in this pull request? Disabling test for cleaning closure of recursive function. ### Why are the changes needed? As of https://github.com/apache/spark/commit/9514b822a70d77a6298ece48e6c053200360302c this test is no longer valid, and recursive calls, even simple ones: ```lead f <- function(x) { if(x > 0) { f(x - 1) } else { x } } ``` lead to ``` Error: node stack overflow ``` This is issue is silenced when tested with `testthat` 1.x (reason unknown), but cause failures when using `testthat` 2.x (issue can be reproduced outside test context). Problem is known and tracked by [SPARK-30629](https://issues.apache.org/jira/browse/SPARK-30629) Therefore, keeping this test active doesn't make sense, as it will lead to continuous test failures, when `testthat` is updated (https://github.com/apache/spark/pull/27359 / SPARK-23435). ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. CC falaki Closes #27363 from zero323/SPARK-29777-FOLLOWUP. Authored-by: zero323 Signed-off-by: Dongjoon Hyun --- R/pkg/tests/fulltests/test_utils.R | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/R/pkg/tests/fulltests/test_utils.R b/R/pkg/tests/fulltests/test_utils.R index c4fcbecee18e9..c3fb9046fcda4 100644 --- a/R/pkg/tests/fulltests/test_utils.R +++ b/R/pkg/tests/fulltests/test_utils.R @@ -89,7 +89,10 @@ test_that("cleanClosure on R functions", { lapply(x, g) + 1 # Test for capturing function call "g"'s closure as a argument of lapply. l$field[1, 1] <- 3 # Test for access operators `$`. res <- defUse + l$field[1, ] # Test for def-use chain of "defUse", and "" symbol. - f(res) # Test for recursive calls. + # Enable once SPARK-30629 is fixed + # nolint start + # f(res) # Test for recursive calls. + # nolint end } newF <- cleanClosure(f) env <- environment(newF) @@ -101,7 +104,10 @@ test_that("cleanClosure on R functions", { # nolint end expect_true("g" %in% ls(env)) expect_true("l" %in% ls(env)) - expect_true("f" %in% ls(env)) + # Enable once SPARK-30629 is fixed + # nolint start + # expect_true("f" %in% ls(env)) + # nolint end expect_equal(get("l", envir = env, inherits = FALSE), l) # "y" should be in the environment of g. newG <- get("g", envir = env, inherits = FALSE) From d69ed9afdf2bd8d03aaf835292b92692ec8189e9 Mon Sep 17 00:00:00 2001 From: Xiao Li Date: Sat, 25 Jan 2020 21:34:12 -0800 Subject: [PATCH 1224/1387] Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp" This reverts commit 1d20d13149140f53df307f47420740f45b4fa5f6. Closes #27351 from gatorsmile/revertSPARK25496. Authored-by: Xiao Li Signed-off-by: Dongjoon Hyun --- R/pkg/R/functions.R | 2 - R/pkg/tests/fulltests/test_sparkSQL.R | 18 +-- python/pyspark/sql/functions.py | 10 -- .../expressions/datetimeExpressions.scala | 21 +--- .../apache/spark/sql/internal/SQLConf.scala | 8 -- .../expressions/CodeGenerationSuite.scala | 59 ++++----- .../expressions/DateExpressionsSuite.scala | 63 ++++------ .../org/apache/spark/sql/functions.scala | 4 - .../apache/spark/sql/DateFunctionsSuite.scala | 68 +++++++++++ .../sql/DeprecatedDateFunctionsSuite.scala | 113 ------------------ .../org/apache/spark/sql/SQLQuerySuite.scala | 40 +++---- .../benchmark/DateTimeBenchmark.scala | 9 +- .../streaming/StreamingAggregationSuite.scala | 11 +- 13 files changed, 150 insertions(+), 276 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDateFunctionsSuite.scala diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index af3b18620e0d7..48f69d5769620 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2496,7 +2496,6 @@ setMethod("schema_of_csv", signature(x = "characterOrColumn"), #' @note from_utc_timestamp since 1.5.0 setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { - .Deprecated(msg = "from_utc_timestamp is deprecated. See SPARK-25496.") jc <- callJStatic("org.apache.spark.sql.functions", "from_utc_timestamp", y@jc, x) column(jc) }) @@ -2555,7 +2554,6 @@ setMethod("next_day", signature(y = "Column", x = "character"), #' @note to_utc_timestamp since 1.5.0 setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { - .Deprecated(msg = "to_utc_timestamp is deprecated. See SPARK-25496.") jc <- callJStatic("org.apache.spark.sql.functions", "to_utc_timestamp", y@jc, x) column(jc) }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 4fcc2baa05467..cbe73d09dd221 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1914,20 +1914,10 @@ test_that("date functions on a DataFrame", { df2 <- createDataFrame(l2) expect_equal(collect(select(df2, minute(df2$b)))[, 1], c(34, 24)) expect_equal(collect(select(df2, second(df2$b)))[, 1], c(0, 34)) - conf <- callJMethod(sparkSession, "conf") - isUtcTimestampFuncEnabled <- callJMethod(conf, "get", "spark.sql.legacy.utcTimestampFunc.enabled") - callJMethod(conf, "set", "spark.sql.legacy.utcTimestampFunc.enabled", "true") - tryCatch({ - # Both from_utc_timestamp and to_utc_timestamp are deprecated as of SPARK-25496 - expect_equal(suppressWarnings(collect(select(df2, from_utc_timestamp(df2$b, "JST"))))[, 1], - c(as.POSIXct("2012-12-13 21:34:00 UTC"), as.POSIXct("2014-12-15 10:24:34 UTC"))) - expect_equal(suppressWarnings(collect(select(df2, to_utc_timestamp(df2$b, "JST"))))[, 1], - c(as.POSIXct("2012-12-13 03:34:00 UTC"), as.POSIXct("2014-12-14 16:24:34 UTC"))) - }, - finally = { - # Reverting the conf back - callJMethod(conf, "set", "spark.sql.legacy.utcTimestampFunc.enabled", isUtcTimestampFuncEnabled) - }) + expect_equal(collect(select(df2, from_utc_timestamp(df2$b, "JST")))[, 1], + c(as.POSIXct("2012-12-13 21:34:00 UTC"), as.POSIXct("2014-12-15 10:24:34 UTC"))) + expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], + c(as.POSIXct("2012-12-13 03:34:00 UTC"), as.POSIXct("2014-12-14 16:24:34 UTC"))) expect_gt(collect(select(df2, unix_timestamp()))[1, 1], 0) expect_gt(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 8b14dd915d541..e80d556cc89e3 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1311,10 +1311,7 @@ def from_utc_timestamp(timestamp, tz): [Row(local_time=datetime.datetime(1997, 2, 28, 2, 30))] >>> df.select(from_utc_timestamp(df.ts, df.tz).alias('local_time')).collect() [Row(local_time=datetime.datetime(1997, 2, 28, 19, 30))] - - .. note:: Deprecated in 3.0. See SPARK-25496 """ - warnings.warn("Deprecated in 3.0. See SPARK-25496", DeprecationWarning) sc = SparkContext._active_spark_context if isinstance(tz, Column): tz = _to_java_column(tz) @@ -1348,10 +1345,7 @@ def to_utc_timestamp(timestamp, tz): [Row(utc_time=datetime.datetime(1997, 2, 28, 18, 30))] >>> df.select(to_utc_timestamp(df.ts, df.tz).alias('utc_time')).collect() [Row(utc_time=datetime.datetime(1997, 2, 28, 1, 30))] - - .. note:: Deprecated in 3.0. See SPARK-25496 """ - warnings.warn("Deprecated in 3.0. See SPARK-25496", DeprecationWarning) sc = SparkContext._active_spark_context if isinstance(tz, Column): tz = _to_java_column(tz) @@ -2955,13 +2949,9 @@ def _test(): globs['sc'] = sc globs['spark'] = spark globs['df'] = spark.createDataFrame([Row(name='Alice', age=2), Row(name='Bob', age=5)]) - - spark.conf.set("spark.sql.legacy.utcTimestampFunc.enabled", "true") (failure_count, test_count) = doctest.testmod( pyspark.sql.functions, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) - spark.conf.unset("spark.sql.legacy.utcTimestampFunc.enabled") - spark.stop() if failure_count: sys.exit(-1) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index a9186ba514c42..28f1d34267224 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -1153,19 +1152,11 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); 2016-08-31 09:00:00 """, - since = "1.5.0", - deprecated = """ - Deprecated since 3.0.0. See SPARK-25496. - """) + since = "1.5.0") // scalastyle:on line.size.limit case class FromUTCTimestamp(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { - if (!SQLConf.get.utcTimestampFuncEnabled) { - throw new AnalysisException(s"The $prettyName function has been disabled since Spark 3.0." + - s"Set ${SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key} to true to enable this function.") - } - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) override def dataType: DataType = TimestampType override def prettyName: String = "from_utc_timestamp" @@ -1367,19 +1358,11 @@ case class MonthsBetween( > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); 2016-08-30 15:00:00 """, - since = "1.5.0", - deprecated = """ - Deprecated since 3.0.0. See SPARK-25496. - """) + since = "1.5.0") // scalastyle:on line.size.limit case class ToUTCTimestamp(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { - if (!SQLConf.get.utcTimestampFuncEnabled) { - throw new AnalysisException(s"The $prettyName function has been disabled since Spark 3.0. " + - s"Set ${SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key} to true to enable this function.") - } - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) override def dataType: DataType = TimestampType override def prettyName: String = "to_utc_timestamp" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4792022ce14b6..df1a11cc13481 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2038,12 +2038,6 @@ object SQLConf { .booleanConf .createWithDefault(false) - val UTC_TIMESTAMP_FUNC_ENABLED = buildConf("spark.sql.legacy.utcTimestampFunc.enabled") - .doc("The configuration property enables the to_utc_timestamp() " + - "and from_utc_timestamp() functions.") - .booleanConf - .createWithDefault(false) - val SOURCES_BINARY_FILE_MAX_LENGTH = buildConf("spark.sql.sources.binaryFile.maxLength") .doc("The max length of a file that can be read by the binary file data source. " + "Spark will fail fast and not attempt to read the file if its length exceeds this value. " + @@ -2423,8 +2417,6 @@ class SQLConf extends Serializable with Logging { def datetimeJava8ApiEnabled: Boolean = getConf(DATETIME_JAVA8API_ENABLED) - def utcTimestampFuncEnabled: Boolean = getConf(UTC_TIMESTAMP_FUNC_ENABLED) - def addDirectoryRecursiveEnabled: Boolean = getConf(LEGACY_ADD_DIRECTORY_USING_RECURSIVE) def legacyMsSqlServerNumericMappingEnabled: Boolean = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index f7b39b679a357..b7628a373d0bd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils} -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ThreadUtils @@ -187,42 +186,36 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-17702: split wide constructor into blocks due to JVM code size limit") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - val length = 5000 - val expressions = Seq.fill(length) { - ToUTCTimestamp( - Literal.create(Timestamp.valueOf("2015-07-24 00:00:00"), TimestampType), - Literal.create("PST", StringType)) - } - val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) - val expected = Seq.fill(length)( - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-07-24 07:00:00"))) - - if (actual != expected) { - fail( - s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") - } + val length = 5000 + val expressions = Seq.fill(length) { + ToUTCTimestamp( + Literal.create(Timestamp.valueOf("2015-07-24 00:00:00"), TimestampType), + Literal.create("PST", StringType)) + } + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) + val expected = Seq.fill(length)( + DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-07-24 07:00:00"))) + + if (actual != expected) { + fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } test("SPARK-22226: group splitted expressions into one method per nested class") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - val length = 10000 - val expressions = Seq.fill(length) { - ToUTCTimestamp( - Literal.create(Timestamp.valueOf("2017-10-10 00:00:00"), TimestampType), - Literal.create("PST", StringType)) - } - val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) - val expected = Seq.fill(length)( - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2017-10-10 07:00:00"))) - - if (actual != expected) { - fail( - s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") - } + val length = 10000 + val expressions = Seq.fill(length) { + ToUTCTimestamp( + Literal.create(Timestamp.valueOf("2017-10-10 00:00:00"), TimestampType), + Literal.create("PST", StringType)) + } + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) + val expected = Seq.fill(length)( + DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2017-10-10 07:00:00"))) + + if (actual != expected) { + fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 7d015aa478df0..274d0beebd300 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -25,14 +25,12 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -894,29 +892,21 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { NonFoldableLiteral.create(tz, StringType)), if (expected != null) Timestamp.valueOf(expected) else null) } - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00") - test("2015-01-24 00:00:00", "PST", "2015-01-24 08:00:00") - test(null, "UTC", null) - test("2015-07-24 00:00:00", null, null) - test(null, null, null) - } - val msg = intercept[AnalysisException] { - test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00") - }.getMessage - assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) + test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00") + test("2015-01-24 00:00:00", "PST", "2015-01-24 08:00:00") + test(null, "UTC", null) + test("2015-07-24 00:00:00", null, null) + test(null, null, null) } test("to_utc_timestamp - invalid time zone id") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz => - val msg = intercept[java.time.DateTimeException] { - GenerateUnsafeProjection.generate( - ToUTCTimestamp( - Literal(Timestamp.valueOf("2015-07-24 00:00:00")), Literal(invalidTz)) :: Nil) - }.getMessage - assert(msg.contains(invalidTz)) - } + Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz => + val msg = intercept[java.time.DateTimeException] { + GenerateUnsafeProjection.generate( + ToUTCTimestamp( + Literal(Timestamp.valueOf("2015-07-24 00:00:00")), Literal(invalidTz)) :: Nil) + }.getMessage + assert(msg.contains(invalidTz)) } } @@ -933,28 +923,19 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { NonFoldableLiteral.create(tz, StringType)), if (expected != null) Timestamp.valueOf(expected) else null) } - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00") - test("2015-01-24 00:00:00", "PST", "2015-01-23 16:00:00") - test(null, "UTC", null) - test("2015-07-24 00:00:00", null, null) - test(null, null, null) - } - val msg = intercept[AnalysisException] { - test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00") - }.getMessage - assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) + test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00") + test("2015-01-24 00:00:00", "PST", "2015-01-23 16:00:00") + test(null, "UTC", null) + test("2015-07-24 00:00:00", null, null) + test(null, null, null) } test("from_utc_timestamp - invalid time zone id") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz => - val msg = intercept[java.time.DateTimeException] { - GenerateUnsafeProjection.generate( - FromUTCTimestamp(Literal(0), Literal(invalidTz)) :: Nil) - }.getMessage - assert(msg.contains(invalidTz)) - } + Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz => + val msg = intercept[java.time.DateTimeException] { + GenerateUnsafeProjection.generate(FromUTCTimestamp(Literal(0), Literal(invalidTz)) :: Nil) + }.getMessage + assert(msg.contains(invalidTz)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index b12d0371bdcdf..e986a564ae0fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3053,7 +3053,6 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - @deprecated("This function is deprecated and will be removed in future versions.", "3.0.0") def from_utc_timestamp(ts: Column, tz: String): Column = withExpr { FromUTCTimestamp(ts.expr, Literal(tz)) } @@ -3065,7 +3064,6 @@ object functions { * @group datetime_funcs * @since 2.4.0 */ - @deprecated("This function is deprecated and will be removed in future versions.", "3.0.0") def from_utc_timestamp(ts: Column, tz: Column): Column = withExpr { FromUTCTimestamp(ts.expr, tz.expr) } @@ -3084,7 +3082,6 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - @deprecated("This function is deprecated and will be removed in future versions.", "3.0.0") def to_utc_timestamp(ts: Column, tz: String): Column = withExpr { ToUTCTimestamp(ts.expr, Literal(tz)) } @@ -3096,7 +3093,6 @@ object functions { * @group datetime_funcs * @since 2.4.0 */ - @deprecated("This function is deprecated and will be removed in future versions.", "3.0.0") def to_utc_timestamp(ts: Column, tz: Column): Column = withExpr { ToUTCTimestamp(ts.expr, tz.expr) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 72d323e88406e..d7d8c2c52d12b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -713,6 +713,40 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { } } + test("from_utc_timestamp with literal zone") { + val df = Seq( + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") + ).toDF("a", "b") + checkAnswer( + df.select(from_utc_timestamp(col("a"), "PST")), + Seq( + Row(Timestamp.valueOf("2015-07-23 17:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + checkAnswer( + df.select(from_utc_timestamp(col("b"), "PST")), + Seq( + Row(Timestamp.valueOf("2015-07-23 17:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + } + + test("from_utc_timestamp with column zone") { + val df = Seq( + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "CET"), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "PST") + ).toDF("a", "b", "c") + checkAnswer( + df.select(from_utc_timestamp(col("a"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 02:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + checkAnswer( + df.select(from_utc_timestamp(col("b"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 02:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + } + test("handling null field by date_part") { val input = Seq(Date.valueOf("2019-09-20")).toDF("d") Seq("date_part(null, d)", "date_part(null, date'2019-09-20')").foreach { expr => @@ -721,4 +755,38 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df, Row(null)) } } + + test("to_utc_timestamp with literal zone") { + val df = Seq( + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") + ).toDF("a", "b") + checkAnswer( + df.select(to_utc_timestamp(col("a"), "PST")), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-25 07:00:00")))) + checkAnswer( + df.select(to_utc_timestamp(col("b"), "PST")), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-25 07:00:00")))) + } + + test("to_utc_timestamp with column zone") { + val df = Seq( + (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "PST"), + (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "CET") + ).toDF("a", "b", "c") + checkAnswer( + df.select(to_utc_timestamp(col("a"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-24 22:00:00")))) + checkAnswer( + df.select(to_utc_timestamp(col("b"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-24 22:00:00")))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDateFunctionsSuite.scala deleted file mode 100644 index bef83ee199cc3..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/DeprecatedDateFunctionsSuite.scala +++ /dev/null @@ -1,113 +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.sql - -import java.sql.Timestamp - -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession - -@deprecated("This test suite will be removed.", "3.0.0") -class DeprecatedDateFunctionsSuite extends QueryTest with SharedSparkSession { - import testImplicits._ - - test("from_utc_timestamp with literal zone") { - val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") - ).toDF("a", "b") - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - checkAnswer( - df.select(from_utc_timestamp(col("a"), "PST")), - Seq( - Row(Timestamp.valueOf("2015-07-23 17:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - checkAnswer( - df.select(from_utc_timestamp(col("b"), "PST")), - Seq( - Row(Timestamp.valueOf("2015-07-23 17:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - } - val msg = intercept[AnalysisException] { - df.select(from_utc_timestamp(col("a"), "PST")).collect() - }.getMessage - assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) - } - - test("from_utc_timestamp with column zone") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "CET"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "PST") - ).toDF("a", "b", "c") - checkAnswer( - df.select(from_utc_timestamp(col("a"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 02:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - checkAnswer( - df.select(from_utc_timestamp(col("b"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 02:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - } - } - - test("to_utc_timestamp with literal zone") { - val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00") - ).toDF("a", "b") - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - checkAnswer( - df.select(to_utc_timestamp(col("a"), "PST")), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-25 07:00:00")))) - checkAnswer( - df.select(to_utc_timestamp(col("b"), "PST")), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-25 07:00:00")))) - } - val msg = intercept[AnalysisException] { - df.select(to_utc_timestamp(col("a"), "PST")).collect() - }.getMessage - assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) - } - - test("to_utc_timestamp with column zone") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - val df = Seq( - (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00", "PST"), - (Timestamp.valueOf("2015-07-25 00:00:00"), "2015-07-25 00:00:00", "CET") - ).toDF("a", "b", "c") - checkAnswer( - df.select(to_utc_timestamp(col("a"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-24 22:00:00")))) - checkAnswer( - df.select(to_utc_timestamp(col("b"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-24 22:00:00")))) - } - } -} 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 2d97eb592b616..a6dae9a269740 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 @@ -176,27 +176,25 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark // The example calls methods that return unstable results. "org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection") - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - val parFuncs = new ParVector(spark.sessionState.functionRegistry.listFunction().toVector) - parFuncs.foreach { funcId => - // Examples can change settings. We clone the session to prevent tests clashing. - val clonedSpark = spark.cloneSession() - val info = clonedSpark.sessionState.catalog.lookupFunctionInfo(funcId) - val className = info.getClassName - if (!ignoreSet.contains(className)) { - withClue(s"Function '${info.getName}', Expression class '$className'") { - val example = info.getExamples - checkExampleSyntax(example) - example.split(" > ").toList.foreach(_ match { - case exampleRe(sql, output) => - val df = clonedSpark.sql(sql) - val actual = unindentAndTrim( - hiveResultString(df.queryExecution.executedPlan).mkString("\n")) - val expected = unindentAndTrim(output) - assert(actual === expected) - case _ => - }) - } + val parFuncs = new ParVector(spark.sessionState.functionRegistry.listFunction().toVector) + parFuncs.foreach { funcId => + // Examples can change settings. We clone the session to prevent tests clashing. + val clonedSpark = spark.cloneSession() + val info = clonedSpark.sessionState.catalog.lookupFunctionInfo(funcId) + val className = info.getClassName + if (!ignoreSet.contains(className)) { + withClue(s"Function '${info.getName}', Expression class '$className'") { + val example = info.getExamples + checkExampleSyntax(example) + example.split(" > ").toList.foreach(_ match { + case exampleRe(sql, output) => + val df = clonedSpark.sql(sql) + val actual = unindentAndTrim( + hiveResultString(df.queryExecution.executedPlan).mkString("\n")) + val expected = unindentAndTrim(output) + assert(actual === expected) + case _ => + }) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala index 92dadccde207c..086583fdafe6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.benchmark import java.sql.Timestamp import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.internal.SQLConf /** * Synthetic benchmark for date and timestamp functions. @@ -91,11 +90,9 @@ object DateTimeBenchmark extends SqlBasedBenchmark { run(N, "from_unixtime", "from_unixtime(id, 'yyyy-MM-dd HH:mm:ss.SSSSSS')") } runBenchmark("Convert timestamps") { - withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { - val timestampExpr = "cast(id as timestamp)" - run(N, "from_utc_timestamp", s"from_utc_timestamp($timestampExpr, 'CET')") - run(N, "to_utc_timestamp", s"to_utc_timestamp($timestampExpr, 'CET')") - } + val timestampExpr = "cast(id as timestamp)" + run(N, "from_utc_timestamp", s"from_utc_timestamp($timestampExpr, 'CET')") + run(N, "to_utc_timestamp", s"to_utc_timestamp($timestampExpr, 'CET')") } runBenchmark("Intervals") { val (start, end) = ("cast(id as timestamp)", "cast((id+8640000) as timestamp)") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 22caa48686c7f..741355381222d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.streaming import java.io.File -import java.util.Locale +import java.util.{Locale, TimeZone} import scala.collection.mutable @@ -397,15 +397,16 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { testWithAllStateVersions("prune results by current_date, complete mode") { import testImplicits._ val clock = new StreamManualClock + val tz = TimeZone.getDefault.getID val inputData = MemoryStream[Long] val aggregated = inputData.toDF() - .select(($"value" * SECONDS_PER_DAY).cast("timestamp").as("value")) + .select(to_utc_timestamp(from_unixtime('value * SECONDS_PER_DAY), tz)) + .toDF("value") .groupBy($"value") .agg(count("*")) - .where($"value".cast("date") >= date_sub(current_timestamp().cast("date"), 10)) - .select( - ($"value".cast("long") / SECONDS_PER_DAY).cast("long"), $"count(1)") + .where($"value".cast("date") >= date_sub(current_date(), 10)) + .select(($"value".cast("long") / SECONDS_PER_DAY).cast("long"), $"count(1)") testStream(aggregated, Complete)( StartStream(Trigger.ProcessingTime("10 day"), triggerClock = clock), // advance clock to 10 days, should retain all keys From 48f647882a852725e9c8ddeccdd9aa138350b2e9 Mon Sep 17 00:00:00 2001 From: Xiao Li Date: Sat, 25 Jan 2020 23:17:36 -0800 Subject: [PATCH 1225/1387] [SPARK-30644][SQL][TEST] Remove query index from the golden files of SQLQueryTestSuite ### What changes were proposed in this pull request? This PR is to remove query index from the golden files of SQLQueryTestSuite ### Why are the changes needed? Because the SQLQueryTestSuite's golden files have the query index for each query, removal of any query statement [except the last one] will generate many unneeded difference. This will make code review harder. The number of changed lines is misleading. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #27361 from gatorsmile/removeIndexNum. Authored-by: Xiao Li Signed-off-by: Dongjoon Hyun --- .../ansi/decimalArithmeticOperations.sql.out | 96 +- .../ansi/higher-order-functions.sql.out | 174 +- .../sql-tests/results/ansi/interval.sql.out | 606 +- .../sql-tests/results/ansi/literals.sql.out | 300 +- .../resources/sql-tests/results/array.sql.out | 72 +- .../sql-tests/results/bitwise.sql.out | 162 +- .../resources/sql-tests/results/cast.sql.out | 276 +- .../sql-tests/results/change-column.sql.out | 150 +- .../results/columnresolution-negative.sql.out | 168 +- .../results/columnresolution-views.sql.out | 102 +- .../results/columnresolution.sql.out | 330 +- .../sql-tests/results/comparator.sql.out | 60 +- .../resources/sql-tests/results/count.sql.out | 30 +- .../sql-tests/results/cross-join.sql.out | 78 +- .../sql-tests/results/csv-functions.sql.out | 102 +- .../sql-tests/results/cte-legacy.sql.out | 102 +- .../resources/sql-tests/results/cte.sql.out | 162 +- .../sql-tests/results/date_part.sql.out | 660 +- .../sql-tests/results/datetime.sql.out | 282 +- .../decimalArithmeticOperations.sql.out | 240 +- .../describe-part-after-analyze.sql.out | 90 +- .../sql-tests/results/describe-query.sql.out | 114 +- .../describe-table-after-alter-table.sql.out | 72 +- .../results/describe-table-column.sql.out | 168 +- .../sql-tests/results/describe.sql.out | 246 +- .../sql-tests/results/except-all.sql.out | 162 +- .../sql-tests/results/except.sql.out | 54 +- .../sql-tests/results/explain.sql.out | 108 +- .../sql-tests/results/extract.sql.out | 384 +- .../sql-tests/results/group-analytics.sql.out | 174 +- .../sql-tests/results/group-by-filter.sql.out | 222 +- .../results/group-by-ordinal.sql.out | 120 +- .../sql-tests/results/group-by.sql.out | 336 +- .../sql-tests/results/grouping_set.sql.out | 108 +- .../sql-tests/results/having.sql.out | 30 +- .../results/higher-order-functions.sql.out | 174 +- .../sql-tests/results/inline-table.sql.out | 102 +- .../sql-tests/results/inner-join.sql.out | 42 +- .../sql-tests/results/intersect-all.sql.out | 132 +- .../sql-tests/results/interval.sql.out | 606 +- .../results/join-empty-relation.sql.out | 144 +- .../sql-tests/results/json-functions.sql.out | 252 +- .../resources/sql-tests/results/limit.sql.out | 84 +- .../sql-tests/results/literals.sql.out | 300 +- .../sql-tests/results/misc-functions.sql.out | 42 +- .../sql-tests/results/natural-join.sql.out | 36 +- .../sql-tests/results/null-handling.sql.out | 168 +- .../results/null-propagation.sql.out | 24 +- .../sql-tests/results/operator-div.sql.out | 108 +- .../sql-tests/results/operators.sql.out | 294 +- .../results/order-by-nulls-ordering.sql.out | 102 +- .../results/order-by-ordinal.sql.out | 72 +- .../sql-tests/results/outer-join.sql.out | 36 +- .../resources/sql-tests/results/pivot.sql.out | 192 +- .../postgreSQL/aggregates_part1.sql.out | 264 +- .../postgreSQL/aggregates_part2.sql.out | 174 +- .../postgreSQL/aggregates_part3.sql.out | 24 +- .../results/postgreSQL/boolean.sql.out | 552 +- .../sql-tests/results/postgreSQL/case.sql.out | 210 +- .../results/postgreSQL/comments.sql.out | 78 +- .../results/postgreSQL/create_view.sql.out | 1170 +-- .../sql-tests/results/postgreSQL/date.sql.out | 588 +- .../results/postgreSQL/float4.sql.out | 258 +- .../results/postgreSQL/float8.sql.out | 570 +- .../results/postgreSQL/groupingsets.sql.out | 324 +- .../results/postgreSQL/insert.sql.out | 54 +- .../sql-tests/results/postgreSQL/int2.sql.out | 210 +- .../sql-tests/results/postgreSQL/int4.sql.out | 318 +- .../sql-tests/results/postgreSQL/int8.sql.out | 510 +- .../results/postgreSQL/interval.sql.out | 144 +- .../sql-tests/results/postgreSQL/join.sql.out | 1086 +-- .../results/postgreSQL/limit.sql.out | 42 +- .../results/postgreSQL/numeric.sql.out | 3462 ++++----- .../results/postgreSQL/select.sql.out | 222 +- .../postgreSQL/select_distinct.sql.out | 114 +- .../results/postgreSQL/select_having.sql.out | 132 +- .../postgreSQL/select_implicit.sql.out | 228 +- .../results/postgreSQL/strings.sql.out | 732 +- .../sql-tests/results/postgreSQL/text.sql.out | 252 +- .../results/postgreSQL/timestamp.sql.out | 192 +- .../results/postgreSQL/union.sql.out | 432 +- .../results/postgreSQL/window_part1.sql.out | 258 +- .../results/postgreSQL/window_part2.sql.out | 186 +- .../results/postgreSQL/window_part3.sql.out | 174 +- .../results/postgreSQL/window_part4.sql.out | 234 +- .../sql-tests/results/postgreSQL/with.sql.out | 306 +- .../sql-tests/results/pred-pushdown.sql.out | 24 +- .../results/predicate-functions.sql.out | 222 +- .../results/query_regex_column.sql.out | 204 +- .../sql-tests/results/random.sql.out | 60 +- .../results/show-create-table.sql.out | 228 +- .../sql-tests/results/show-tables.sql.out | 156 +- .../sql-tests/results/show_columns.sql.out | 150 +- .../sql-compatibility-functions.sql.out | 84 +- .../results/string-functions.sql.out | 198 +- .../sql-tests/results/struct.sql.out | 54 +- .../exists-subquery/exists-aggregate.sql.out | 66 +- .../exists-subquery/exists-basic.sql.out | 78 +- .../exists-subquery/exists-cte.sql.out | 48 +- .../exists-subquery/exists-having.sql.out | 48 +- .../exists-joins-and-set-ops.sql.out | 102 +- .../exists-orderby-limit.sql.out | 72 +- .../exists-within-and-or.sql.out | 48 +- .../subquery/in-subquery/in-basic.sql.out | 42 +- .../subquery/in-subquery/in-group-by.sql.out | 114 +- .../subquery/in-subquery/in-having.sql.out | 72 +- .../subquery/in-subquery/in-joins.sql.out | 204 +- .../subquery/in-subquery/in-limit.sql.out | 48 +- .../in-subquery/in-multiple-columns.sql.out | 48 +- .../subquery/in-subquery/in-order-by.sql.out | 108 +- .../in-subquery/in-set-operations.sql.out | 96 +- .../subquery/in-subquery/in-with-cte.sql.out | 78 +- .../in-subquery/not-in-group-by.sql.out | 48 +- .../subquery/in-subquery/not-in-joins.sql.out | 54 +- ...in-unit-tests-multi-column-literal.sql.out | 24 +- .../not-in-unit-tests-multi-column.sql.out | 54 +- ...n-unit-tests-single-column-literal.sql.out | 30 +- .../not-in-unit-tests-single-column.sql.out | 60 +- .../subquery/in-subquery/simple-in.sql.out | 84 +- .../invalid-correlation.sql.out | 48 +- .../subq-input-typecheck.sql.out | 60 +- .../scalar-subquery-predicate.sql.out | 162 +- .../scalar-subquery-select.sql.out | 66 +- .../results/subquery/subquery-in-from.sql.out | 36 +- .../sql-tests/results/table-aliases.sql.out | 66 +- .../results/table-valued-functions.sql.out | 48 +- .../results/tablesample-negative.sql.out | 36 +- .../typeCoercion/native/arrayJoin.sql.out | 66 +- .../native/binaryComparison.sql.out | 1590 ++-- .../native/booleanEquality.sql.out | 582 +- .../native/caseWhenCoercion.sql.out | 870 +-- .../typeCoercion/native/concat.sql.out | 84 +- .../native/dateTimeOperations.sql.out | 240 +- .../native/decimalPrecision.sql.out | 6870 ++++++++--------- .../typeCoercion/native/division.sql.out | 870 +-- .../results/typeCoercion/native/elt.sql.out | 36 +- .../typeCoercion/native/ifCoercion.sql.out | 870 +-- .../native/implicitTypeCasts.sql.out | 264 +- .../typeCoercion/native/inConversion.sql.out | 1734 ++--- .../typeCoercion/native/mapZipWith.sql.out | 96 +- .../typeCoercion/native/mapconcat.sql.out | 48 +- .../native/promoteStrings.sql.out | 1896 ++--- .../native/stringCastAndExpressions.sql.out | 192 +- .../native/widenSetOperationTypes.sql.out | 870 +-- .../native/windowFrameCoercion.sql.out | 150 +- .../resources/sql-tests/results/udaf.sql.out | 48 +- .../postgreSQL/udf-aggregates_part1.sql.out | 258 +- .../postgreSQL/udf-aggregates_part2.sql.out | 150 +- .../postgreSQL/udf-aggregates_part3.sql.out | 12 +- .../results/udf/postgreSQL/udf-case.sql.out | 210 +- .../results/udf/postgreSQL/udf-join.sql.out | 1110 +-- .../udf/postgreSQL/udf-select_having.sql.out | 132 +- .../postgreSQL/udf-select_implicit.sql.out | 228 +- .../sql-tests/results/udf/udf-count.sql.out | 30 +- .../results/udf/udf-cross-join.sql.out | 78 +- .../results/udf/udf-except-all.sql.out | 162 +- .../sql-tests/results/udf/udf-except.sql.out | 54 +- .../results/udf/udf-group-analytics.sql.out | 174 +- .../results/udf/udf-group-by.sql.out | 312 +- .../sql-tests/results/udf/udf-having.sql.out | 30 +- .../results/udf/udf-inline-table.sql.out | 102 +- .../results/udf/udf-inner-join.sql.out | 42 +- .../results/udf/udf-intersect-all.sql.out | 132 +- .../udf/udf-join-empty-relation.sql.out | 144 +- .../results/udf/udf-natural-join.sql.out | 36 +- .../results/udf/udf-outer-join.sql.out | 48 +- .../sql-tests/results/udf/udf-pivot.sql.out | 180 +- .../results/udf/udf-special-values.sql.out | 36 +- .../sql-tests/results/udf/udf-udaf.sql.out | 48 +- .../sql-tests/results/udf/udf-union.sql.out | 96 +- .../sql-tests/results/udf/udf-window.sql.out | 138 +- .../resources/sql-tests/results/union.sql.out | 96 +- .../sql-tests/results/window.sql.out | 138 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 20 +- .../ThriftServerQueryTestSuite.scala | 2 +- 175 files changed, 24026 insertions(+), 24026 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out index 510f2339ba2a9..ce53e1c2863e0 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out @@ -2,137 +2,137 @@ -- Number of queries: 16 --- !query 0 +-- !query create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query select id, a*10, b/10 from decimals_test order by id --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 1000.000000000000000 99.900000000000000000 2 123451.230000000000000 1234.512300000000000000 3 1.234567891011000 123.410000000000000000 4 1234567891234567890.000000000000000 0.112345678912345679 --- !query 3 +-- !query select 10.3 * 3.0 --- !query 3 schema +-- !query schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> --- !query 3 output +-- !query output 30.90 --- !query 4 +-- !query select 10.3000 * 3.0 --- !query 4 schema +-- !query schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> --- !query 4 output +-- !query output 30.90000 --- !query 5 +-- !query select 10.30000 * 30.0 --- !query 5 schema +-- !query schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> --- !query 5 output +-- !query output 309.000000 --- !query 6 +-- !query select 10.300000000000000000 * 3.000000000000000000 --- !query 6 schema +-- !query schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> --- !query 6 output +-- !query output 30.9000000000000000000000000000000000 --- !query 7 +-- !query select 10.300000000000000000 * 3.0000000000000000000 --- !query 7 schema +-- !query schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> --- !query 7 output +-- !query output 30.9000000000000000000000000000000000 --- !query 8 +-- !query select (5e36BD + 0.1) + 5e36BD --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output java.lang.ArithmeticException Decimal(expanded,10000000000000000000000000000000000000.1,39,1}) cannot be represented as Decimal(38, 1). --- !query 9 +-- !query select (-4e36BD - 0.1) - 7e36BD --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output java.lang.ArithmeticException Decimal(expanded,-11000000000000000000000000000000000000.1,39,1}) cannot be represented as Decimal(38, 1). --- !query 10 +-- !query select 12345678901234567890.0 * 12345678901234567890.0 --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output java.lang.ArithmeticException Decimal(expanded,152415787532388367501905199875019052100,39,0}) cannot be represented as Decimal(38, 2). --- !query 11 +-- !query select 1e35BD / 0.1 --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output java.lang.ArithmeticException Decimal(expanded,1000000000000000000000000000000000000,37,0}) cannot be represented as Decimal(38, 6). --- !query 12 +-- !query select 123456789123456789.1234567890 * 1.123456789123456789 --- !query 12 schema +-- !query schema struct<(CAST(123456789123456789.1234567890 AS DECIMAL(36,18)) * CAST(1.123456789123456789 AS DECIMAL(36,18))):decimal(38,18)> --- !query 12 output +-- !query output 138698367904130467.654320988515622621 --- !query 13 +-- !query select 123456789123456789.1234567890 * 1.123456789123456789 --- !query 13 schema +-- !query schema struct<(CAST(123456789123456789.1234567890 AS DECIMAL(36,18)) * CAST(1.123456789123456789 AS DECIMAL(36,18))):decimal(38,18)> --- !query 13 output +-- !query output 138698367904130467.654320988515622621 --- !query 14 +-- !query select 12345678912345.123456789123 / 0.000000012345678 --- !query 14 schema +-- !query schema struct<(CAST(12345678912345.123456789123 AS DECIMAL(29,15)) / CAST(1.2345678E-8 AS DECIMAL(29,15))):decimal(38,9)> --- !query 14 output +-- !query output 1000000073899961059796.725866332 --- !query 15 +-- !query drop table decimals_test --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out index e7dae6595a890..7bef1bad4507e 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out @@ -2,265 +2,265 @@ -- Number of queries: 29 --- !query 0 +-- !query create or replace temporary view nested as values (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), (3, array(12), array(array(17))) as t(x, ys, zs) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select upper(x -> x) as v --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output org.apache.spark.sql.AnalysisException A lambda function should only be used in a higher order function. However, its class is org.apache.spark.sql.catalyst.expressions.Upper, which is not a higher order function.; line 1 pos 7 --- !query 2 +-- !query select transform(zs, z -> z) as v from nested --- !query 2 schema +-- !query schema struct>> --- !query 2 output +-- !query output [[12,99],[123,42],[1]] [[17]] [[6,96,65],[-1,-2]] --- !query 3 +-- !query select transform(ys, y -> y * y) as v from nested --- !query 3 schema +-- !query schema struct> --- !query 3 output +-- !query output [1024,9409] [144] [5929,5776] --- !query 4 +-- !query select transform(ys, (y, i) -> y + i) as v from nested --- !query 4 schema +-- !query schema struct> --- !query 4 output +-- !query output [12] [32,98] [77,-75] --- !query 5 +-- !query select transform(zs, z -> concat(ys, z)) as v from nested --- !query 5 schema +-- !query schema struct>> --- !query 5 output +-- !query output [[12,17]] [[32,97,12,99],[32,97,123,42],[32,97,1]] [[77,-76,6,96,65],[77,-76,-1,-2]] --- !query 6 +-- !query select transform(ys, 0) as v from nested --- !query 6 schema +-- !query schema struct> --- !query 6 output +-- !query output [0,0] [0,0] [0] --- !query 7 +-- !query select transform(cast(null as array), x -> x + 1) as v --- !query 7 schema +-- !query schema struct> --- !query 7 output +-- !query output NULL --- !query 8 +-- !query select filter(ys, y -> y > 30) as v from nested --- !query 8 schema +-- !query schema struct> --- !query 8 output +-- !query output [32,97] [77] [] --- !query 9 +-- !query select filter(cast(null as array), y -> true) as v --- !query 9 schema +-- !query schema struct> --- !query 9 output +-- !query output NULL --- !query 10 +-- !query select transform(zs, z -> filter(z, zz -> zz > 50)) as v from nested --- !query 10 schema +-- !query schema struct>> --- !query 10 output +-- !query output [[96,65],[]] [[99],[123],[]] [[]] --- !query 11 +-- !query select aggregate(ys, 0, (y, a) -> y + a + x) as v from nested --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 131 15 5 --- !query 12 +-- !query select aggregate(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 0.5 12.0 64.5 --- !query 13 +-- !query select transform(zs, z -> aggregate(z, 1, (acc, val) -> acc * val * size(z))) as v from nested --- !query 13 schema +-- !query schema struct> --- !query 13 output +-- !query output [1010880,8] [17] [4752,20664,1] --- !query 14 +-- !query select aggregate(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output NULL --- !query 15 +-- !query select exists(ys, y -> y > 30) as v from nested --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output false true true --- !query 16 +-- !query select exists(cast(null as array), y -> y > 30) as v --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output NULL --- !query 17 +-- !query select zip_with(ys, zs, (a, b) -> a + size(b)) as v from nested --- !query 17 schema +-- !query schema struct> --- !query 17 output +-- !query output [13] [34,99,null] [80,-74] --- !query 18 +-- !query select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)) as v --- !query 18 schema +-- !query schema struct> --- !query 18 output +-- !query output ["ad","be","cf"] --- !query 19 +-- !query select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v --- !query 19 schema +-- !query schema struct> --- !query 19 output +-- !query output ["a",null,"f"] --- !query 20 +-- !query create or replace temporary view nested as values (1, map(1, 1, 2, 2, 3, 3)), (2, map(4, 4, 5, 5, 6, 6)) as t(x, ys) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query select transform_keys(ys, (k, v) -> k) as v from nested --- !query 21 schema +-- !query schema struct> --- !query 21 output +-- !query output {1:1,2:2,3:3} {4:4,5:5,6:6} --- !query 22 +-- !query select transform_keys(ys, (k, v) -> k + 1) as v from nested --- !query 22 schema +-- !query schema struct> --- !query 22 output +-- !query output {2:1,3:2,4:3} {5:4,6:5,7:6} --- !query 23 +-- !query select transform_keys(ys, (k, v) -> k + v) as v from nested --- !query 23 schema +-- !query schema struct> --- !query 23 output +-- !query output {10:5,12:6,8:4} {2:1,4:2,6:3} --- !query 24 +-- !query select transform_values(ys, (k, v) -> v) as v from nested --- !query 24 schema +-- !query schema struct> --- !query 24 output +-- !query output {1:1,2:2,3:3} {4:4,5:5,6:6} --- !query 25 +-- !query select transform_values(ys, (k, v) -> v + 1) as v from nested --- !query 25 schema +-- !query schema struct> --- !query 25 output +-- !query output {1:2,2:3,3:4} {4:5,5:6,6:7} --- !query 26 +-- !query select transform_values(ys, (k, v) -> k + v) as v from nested --- !query 26 schema +-- !query schema struct> --- !query 26 output +-- !query output {1:2,2:4,3:6} {4:8,5:10,6:12} --- !query 27 +-- !query select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'all'(line 1, pos 21) @@ -270,11 +270,11 @@ select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) ---------------------^^^ --- !query 28 +-- !query select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'all'(line 1, pos 22) diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 64107f98ede6c..ab6130da869c4 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -2,308 +2,308 @@ -- Number of queries: 101 --- !query 0 +-- !query select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 30 hours 33 minutes 36.003006 seconds --- !query 1 +-- !query select interval 4 month 2 weeks 3 microseconds * 1.5 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 6 months 21 days 0.000005 seconds --- !query 2 +-- !query select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 16 hours --- !query 3 +-- !query select interval '2 seconds' / 0 --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output java.lang.ArithmeticException divide by zero --- !query 4 +-- !query select interval '2 seconds' / null --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output NULL --- !query 5 +-- !query select interval '2 seconds' * null --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output NULL --- !query 6 +-- !query select null * interval '2 seconds' --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL --- !query 7 +-- !query select -interval '-1 month 1 day -1 second' --- !query 7 schema +-- !query schema struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> --- !query 7 output +-- !query output 1 months -1 days 1 seconds --- !query 8 +-- !query select -interval -1 month 1 day -1 second --- !query 8 schema +-- !query schema struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> --- !query 8 output +-- !query output 1 months -1 days 1 seconds --- !query 9 +-- !query select +interval '-1 month 1 day -1 second' --- !query 9 schema +-- !query schema struct<(+ INTERVAL '-1 months 1 days -1 seconds'):interval> --- !query 9 output +-- !query output -1 months 1 days -1 seconds --- !query 10 +-- !query select +interval -1 month 1 day -1 second --- !query 10 schema +-- !query schema struct<(+ INTERVAL '-1 months 1 days -1 seconds'):interval> --- !query 10 output +-- !query output -1 months 1 days -1 seconds --- !query 11 +-- !query select make_interval(1) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 years --- !query 12 +-- !query select make_interval(1, 2) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 years 2 months --- !query 13 +-- !query select make_interval(1, 2, 3) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 years 2 months 21 days --- !query 14 +-- !query select make_interval(1, 2, 3, 4) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 years 2 months 25 days --- !query 15 +-- !query select make_interval(1, 2, 3, 4, 5) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 years 2 months 25 days 5 hours --- !query 16 +-- !query select make_interval(1, 2, 3, 4, 5, 6) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1 years 2 months 25 days 5 hours 6 minutes --- !query 17 +-- !query select make_interval(1, 2, 3, 4, 5, 6, 7.008009) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds --- !query 18 +-- !query select cast('1 second' as interval) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 seconds --- !query 19 +-- !query select cast('+1 second' as interval) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1 seconds --- !query 20 +-- !query select cast('-1 second' as interval) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output -1 seconds --- !query 21 +-- !query select cast('+ 1 second' as interval) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 seconds --- !query 22 +-- !query select cast('- 1 second' as interval) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output -1 seconds --- !query 23 +-- !query select cast('- -1 second' as interval) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output NULL --- !query 24 +-- !query select cast('- +1 second' as interval) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output NULL --- !query 25 +-- !query select interval 13.123456789 seconds, interval -13.123456789 second --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 13.123456 seconds -13.123456 seconds --- !query 26 +-- !query select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds --- !query 27 +-- !query select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds --- !query 28 +-- !query select interval '0 0:0:0.1' day to second --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 0.1 seconds --- !query 29 +-- !query select interval '10-9' year to month --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 10 years 9 months --- !query 30 +-- !query select interval '20 15' day to hour --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 20 days 15 hours --- !query 31 +-- !query select interval '20 15:40' day to minute --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 20 days 15 hours 40 minutes --- !query 32 +-- !query select interval '20 15:40:32.99899999' day to second --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 20 days 15 hours 40 minutes 32.998999 seconds --- !query 33 +-- !query select interval '15:40' hour to minute --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 15 hours 40 minutes --- !query 34 +-- !query select interval '15:40:32.99899999' hour to second --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 15 hours 40 minutes 32.998999 seconds --- !query 35 +-- !query select interval '40:32.99899999' minute to second --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 40 minutes 32.998999 seconds --- !query 36 +-- !query select interval '40:32' minute to second --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 40 minutes 32 seconds --- !query 37 +-- !query select interval 30 day day --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 23) @@ -313,11 +313,11 @@ select interval 30 day day -----------------------^^^ --- !query 38 +-- !query select interval '20 15:40:32.99899999' day to hour --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) @@ -327,11 +327,11 @@ select interval '20 15:40:32.99899999' day to hour ----------------^^^ --- !query 39 +-- !query select interval '20 15:40:32.99899999' day to minute --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) @@ -341,11 +341,11 @@ select interval '20 15:40:32.99899999' day to minute ----------------^^^ --- !query 40 +-- !query select interval '15:40:32.99899999' hour to minute --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 15:40:32.99899999(line 1, pos 16) @@ -355,11 +355,11 @@ select interval '15:40:32.99899999' hour to minute ----------------^^^ --- !query 41 +-- !query select interval '15:40.99899999' hour to second --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40.99899999(line 1, pos 16) @@ -369,11 +369,11 @@ select interval '15:40.99899999' hour to second ----------------^^^ --- !query 42 +-- !query select interval '15:40' hour to second --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40(line 1, pos 16) @@ -383,11 +383,11 @@ select interval '15:40' hour to second ----------------^^^ --- !query 43 +-- !query select interval '20 40:32.99899999' minute to second --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 20 40:32.99899999(line 1, pos 16) @@ -397,11 +397,11 @@ select interval '20 40:32.99899999' minute to second ----------------^^^ --- !query 44 +-- !query select interval 10 nanoseconds --- !query 44 schema +-- !query schema struct<> --- !query 44 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'(line 1, pos 16) @@ -411,35 +411,35 @@ select interval 10 nanoseconds ----------------^^^ --- !query 45 +-- !query select map(1, interval 1 day, 2, interval 3 week) --- !query 45 schema +-- !query schema struct> --- !query 45 output +-- !query output {1:1 days,2:21 days} --- !query 46 +-- !query select interval 'interval 3 year 1 hour' --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output 3 years 1 hours --- !query 47 +-- !query select interval '3 year 1 hour' --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 3 years 1 hours --- !query 48 +-- !query select interval --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException at least one time unit should be given for interval literal(line 1, pos 7) @@ -449,11 +449,11 @@ select interval -------^^^ --- !query 49 +-- !query select interval 1 fake_unit --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Error parsing ' 1 fake_unit' to interval, invalid unit 'fake_unit'(line 1, pos 16) @@ -463,11 +463,11 @@ select interval 1 fake_unit ----------------^^^ --- !query 50 +-- !query select interval 1 year to month --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 16) @@ -477,11 +477,11 @@ select interval 1 year to month ----------------^^^ --- !query 51 +-- !query select interval '1' year to second --- !query 51 schema +-- !query schema struct<> --- !query 51 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 16) @@ -491,11 +491,11 @@ select interval '1' year to second ----------------^^^ --- !query 52 +-- !query select interval '10-9' year to month '2-1' year to month --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -505,11 +505,11 @@ select interval '10-9' year to month '2-1' year to month -------------------------------------^^^ --- !query 53 +-- !query select interval '10-9' year to month '12:11:10' hour to second --- !query 53 schema +-- !query schema struct<> --- !query 53 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -519,11 +519,11 @@ select interval '10-9' year to month '12:11:10' hour to second -------------------------------------^^^ --- !query 54 +-- !query select interval '1 15:11' day to minute '12:11:10' hour to second --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) @@ -533,11 +533,11 @@ select interval '1 15:11' day to minute '12:11:10' hour to second ----------------------------------------^^^ --- !query 55 +-- !query select interval 1 year '2-1' year to month --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -547,11 +547,11 @@ select interval 1 year '2-1' year to month -----------------------^^^ --- !query 56 +-- !query select interval 1 year '12:11:10' hour to second --- !query 56 schema +-- !query schema struct<> --- !query 56 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -561,11 +561,11 @@ select interval 1 year '12:11:10' hour to second -----------------------^^^ --- !query 57 +-- !query select interval '10-9' year to month '1' year --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -575,11 +575,11 @@ select interval '10-9' year to month '1' year -------------------------------------^^^ --- !query 58 +-- !query select interval '12:11:10' hour to second '1' year --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) @@ -589,11 +589,11 @@ select interval '12:11:10' hour to second '1' year ------------------------------------------^^^ --- !query 59 +-- !query select interval (-30) day --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 22) @@ -603,11 +603,11 @@ select interval (-30) day ----------------------^^^ --- !query 60 +-- !query select interval (a + 1) day --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 24) @@ -617,11 +617,11 @@ select interval (a + 1) day ------------------------^^^ --- !query 61 +-- !query select interval 30 day day day --- !query 61 schema +-- !query schema struct<> --- !query 61 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'day'(line 1, pos 23) @@ -631,192 +631,192 @@ select interval 30 day day day -----------------------^^^ --- !query 62 +-- !query select sum(cast(null as interval)) --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output NULL --- !query 63 +-- !query select sum(cast(v as interval)) from VALUES ('1 seconds') t(v) where 1=0 --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output NULL --- !query 64 +-- !query select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 3 seconds --- !query 65 +-- !query select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output 1 seconds --- !query 66 +-- !query select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output -3 seconds --- !query 67 +-- !query select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output -7 days 2 seconds --- !query 68 +-- !query select i, sum(cast(v as interval)) from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) group by i --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 1 -2 days 2 2 seconds 3 NULL --- !query 69 +-- !query select sum(cast(v as interval)) as sv from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) having sv is not null --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output -2 days 2 seconds --- !query 70 +-- !query SELECT i, sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM VALUES(1, '1 seconds'), (1, '2 seconds'), (2, NULL), (2, NULL) t(i,v) --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output 1 2 seconds 1 3 seconds 2 NULL 2 NULL --- !query 71 +-- !query select avg(cast(v as interval)) from VALUES (null) t(v) --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output NULL --- !query 72 +-- !query select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0 --- !query 72 schema +-- !query schema struct --- !query 72 output +-- !query output NULL --- !query 73 +-- !query select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output 1.5 seconds --- !query 74 +-- !query select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output 0.5 seconds --- !query 75 +-- !query select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 75 schema +-- !query schema struct --- !query 75 output +-- !query output -1.5 seconds --- !query 76 +-- !query select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 76 schema +-- !query schema struct --- !query 76 output +-- !query output -3 days -11 hours -59 minutes -59 seconds --- !query 77 +-- !query select i, avg(cast(v as interval)) from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) group by i --- !query 77 schema +-- !query schema struct --- !query 77 output +-- !query output 1 -1 days 2 2 seconds 3 NULL --- !query 78 +-- !query select avg(cast(v as interval)) as sv from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) having sv is not null --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output -15 hours -59 minutes -59.333333 seconds --- !query 79 +-- !query SELECT i, avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v) --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output 1 1.5 seconds 1 2 seconds 2 NULL 2 NULL --- !query 80 +-- !query create temporary view interval_arithmetic as select CAST(dateval AS date), CAST(tsval AS timestamp) from values ('2012-01-01', '2012-01-01') as interval_arithmetic(dateval, tsval) --- !query 80 schema +-- !query schema struct<> --- !query 80 output +-- !query output --- !query 81 +-- !query select dateval, dateval - interval '2-2' year to month, @@ -826,13 +826,13 @@ select - interval '2-2' year to month + dateval, interval '2-2' year to month + dateval from interval_arithmetic --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 --- !query 82 +-- !query select tsval, tsval - interval '2-2' year to month, @@ -842,24 +842,24 @@ select - interval '2-2' year to month + tsval, interval '2-2' year to month + tsval from interval_arithmetic --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 --- !query 83 +-- !query select interval '2-2' year to month + interval '3-3' year to month, interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic --- !query 83 schema +-- !query schema struct<(INTERVAL '2 years 2 months' + INTERVAL '3 years 3 months'):interval,(INTERVAL '2 years 2 months' - INTERVAL '3 years 3 months'):interval> --- !query 83 output +-- !query output 5 years 5 months -1 years -1 months --- !query 84 +-- !query select dateval, dateval - interval '99 11:22:33.123456789' day to second, @@ -869,13 +869,13 @@ select -interval '99 11:22:33.123456789' day to second + dateval, interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 --- !query 85 +-- !query select tsval, tsval - interval '99 11:22:33.123456789' day to second, @@ -885,135 +885,135 @@ select -interval '99 11:22:33.123456789' day to second + tsval, interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 --- !query 86 +-- !query select interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic --- !query 86 schema +-- !query schema struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> --- !query 86 output +-- !query output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds --- !query 87 +-- !query select interval '\t interval 1 day' --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 1 days --- !query 88 +-- !query select interval 'interval \t 1\tday' --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output 1 days --- !query 89 +-- !query select interval 'interval\t1\tday' --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output 1 days --- !query 90 +-- !query select interval '1\t' day --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 1 days --- !query 91 +-- !query select interval '1 ' day --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output 1 days --- !query 92 +-- !query select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 92 schema +-- !query schema struct<> --- !query 92 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 93 +-- !query select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 94 +-- !query select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 95 +-- !query select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 95 schema +-- !query schema struct<> --- !query 95 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 96 +-- !query select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 96 schema +-- !query schema struct<> --- !query 96 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 97 +-- !query SELECT from_csv('1, 1 day', 'a INT, b interval') --- !query 97 schema +-- !query schema struct> --- !query 97 output +-- !query output {"a":1,"b":1 days} --- !query 98 +-- !query SELECT to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)) --- !query 98 schema +-- !query schema struct --- !query 98 output +-- !query output 2 years 8 months,1 hours 10 minutes --- !query 99 +-- !query SELECT from_json('{"a":"1 days"}', 'a interval') --- !query 99 schema +-- !query schema struct> --- !query 99 output +-- !query output {"a":1 days} --- !query 100 +-- !query SELECT to_json(map('a', interval 25 month 100 day 130 minute)) --- !query 100 schema +-- !query schema struct --- !query 100 output +-- !query output {"a":"2 years 1 months 100 days 2 hours 10 minutes"} diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out index 382294010914f..f6720f6c5faa4 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out @@ -2,43 +2,43 @@ -- Number of queries: 50 --- !query 0 +-- !query select null, Null, nUll --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output NULL NULL NULL --- !query 1 +-- !query select true, tRue, false, fALse --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output true true false false --- !query 2 +-- !query select 1Y --- !query 2 schema +-- !query schema struct<1:tinyint> --- !query 2 output +-- !query output 1 --- !query 3 +-- !query select 127Y, -128Y --- !query 3 schema +-- !query schema struct<127:tinyint,-128:tinyint> --- !query 3 output +-- !query output 127 -128 --- !query 4 +-- !query select 128Y --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7) @@ -48,27 +48,27 @@ select 128Y -------^^^ --- !query 5 +-- !query select 1S --- !query 5 schema +-- !query schema struct<1:smallint> --- !query 5 output +-- !query output 1 --- !query 6 +-- !query select 32767S, -32768S --- !query 6 schema +-- !query schema struct<32767:smallint,-32768:smallint> --- !query 6 output +-- !query output 32767 -32768 --- !query 7 +-- !query select 32768S --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7) @@ -78,27 +78,27 @@ select 32768S -------^^^ --- !query 8 +-- !query select 1L, 2147483648L --- !query 8 schema +-- !query schema struct<1:bigint,2147483648:bigint> --- !query 8 output +-- !query output 1 2147483648 --- !query 9 +-- !query select 9223372036854775807L, -9223372036854775808L --- !query 9 schema +-- !query schema struct<9223372036854775807:bigint,-9223372036854775808:bigint> --- !query 9 output +-- !query output 9223372036854775807 -9223372036854775808 --- !query 10 +-- !query select 9223372036854775808L --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7) @@ -108,43 +108,43 @@ select 9223372036854775808L -------^^^ --- !query 11 +-- !query select 1, -1 --- !query 11 schema +-- !query schema struct<1:int,-1:int> --- !query 11 output +-- !query output 1 -1 --- !query 12 +-- !query select 2147483647, -2147483648 --- !query 12 schema +-- !query schema struct<2147483647:int,-2147483648:int> --- !query 12 output +-- !query output 2147483647 -2147483648 --- !query 13 +-- !query select 9223372036854775807, -9223372036854775808 --- !query 13 schema +-- !query schema struct<9223372036854775807:bigint,-9223372036854775808:bigint> --- !query 13 output +-- !query output 9223372036854775807 -9223372036854775808 --- !query 14 +-- !query select 9223372036854775808, -9223372036854775809 --- !query 14 schema +-- !query schema struct<9223372036854775808:decimal(19,0),-9223372036854775809:decimal(19,0)> --- !query 14 output +-- !query output 9223372036854775808 -9223372036854775809 --- !query 15 +-- !query select 1234567890123456789012345678901234567890 --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38 @@ -152,11 +152,11 @@ decimal can only support precision up to 38 select 1234567890123456789012345678901234567890 --- !query 16 +-- !query select 1234567890123456789012345678901234567890.0 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38 @@ -164,27 +164,27 @@ decimal can only support precision up to 38 select 1234567890123456789012345678901234567890.0 --- !query 17 +-- !query select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 --- !query 17 schema +-- !query schema struct<1.0:double,1.2:double,1.0E10:double,150000.0:double,0.1:double,0.1:double,10000.0:double,90.0:double,90.0:double,90.0:double,90.0:double> --- !query 17 output +-- !query output 1.0 1.2 1.0E10 150000.0 0.1 0.1 10000.0 90.0 90.0 90.0 90.0 --- !query 18 +-- !query select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 --- !query 18 schema +-- !query schema struct<-1.0:double,-1.2:double,-1.0E10:double,-150000.0:double,-0.1:double,-0.1:double,-10000.0:double> --- !query 18 output +-- !query output -1.0 -1.2 -1.0E10 -150000.0 -0.1 -0.1 -10000.0 --- !query 19 +-- !query select .e3 --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'select .'(line 1, pos 7) @@ -194,11 +194,11 @@ select .e3 -------^^^ --- !query 20 +-- !query select 1E309, -1E309 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Numeric literal 1E309 does not fit in range [-1.7976931348623157E+308, 1.7976931348623157E+308] for type double(line 1, pos 7) @@ -208,93 +208,93 @@ select 1E309, -1E309 -------^^^ --- !query 21 +-- !query select 0.3, -0.8, .5, -.18, 0.1111, .1111 --- !query 21 schema +-- !query schema struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> --- !query 21 output +-- !query output 0.3 -0.8 0.5 -0.18 0.1111 0.1111 --- !query 22 +-- !query select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d --- !query 22 schema +-- !query schema struct<1.2345678901234568E48:double,1.2345678901234568E48:double> --- !query 22 output +-- !query output 1.2345678901234568E48 1.2345678901234568E48 --- !query 23 +-- !query select "Hello Peter!", 'hello lee!' --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output Hello Peter! hello lee! --- !query 24 +-- !query select 'hello' 'world', 'hello' " " 'lee' --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output helloworld hello lee --- !query 25 +-- !query select "hello 'peter'" --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output hello 'peter' --- !query 26 +-- !query select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output pattern% no-pattern\% pattern\% pattern\\% --- !query 27 +-- !query select '\'', '"', '\n', '\r', '\t', 'Z' --- !query 27 schema +-- !query schema struct<':string,":string, :string, :string, :string,Z:string> --- !query 27 output +-- !query output ' " Z --- !query 28 +-- !query select '\110\145\154\154\157\041' --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output Hello! --- !query 29 +-- !query select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output World :) --- !query 30 +-- !query select dAte '2016-03-12' --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 2016-03-12 --- !query 31 +-- !query select date 'mar 11 2016' --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: mar 11 2016(line 1, pos 7) @@ -304,19 +304,19 @@ select date 'mar 11 2016' -------^^^ --- !query 32 +-- !query select tImEstAmp '2016-03-11 20:54:00.000' --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 2016-03-11 20:54:00 --- !query 33 +-- !query select timestamp '2016-33-11 20:54:00.000' --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the TIMESTAMP value: 2016-33-11 20:54:00.000(line 1, pos 7) @@ -326,11 +326,11 @@ select timestamp '2016-33-11 20:54:00.000' -------^^^ --- !query 34 +-- !query select GEO '(10,-6)' --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'GEO' are currently not supported.(line 1, pos 7) @@ -340,19 +340,19 @@ select GEO '(10,-6)' -------^^^ --- !query 35 +-- !query select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD --- !query 35 schema +-- !query schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> --- !query 35 output +-- !query output 90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 --- !query 36 +-- !query select 1.20E-38BD --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38(line 1, pos 7) @@ -362,19 +362,19 @@ select 1.20E-38BD -------^^^ --- !query 37 +-- !query select x'2379ACFe' --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output #y�� --- !query 38 +-- !query select X'XuZ' --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException contains illegal character for hexBinary: 0XuZ(line 1, pos 7) @@ -384,98 +384,98 @@ select X'XuZ' -------^^^ --- !query 39 +-- !query SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 --- !query 39 schema +-- !query schema struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E8:double,3.14E-8:double,-3.14E8:double,-3.14E-8:double,3.14E8:double,3.14E8:double,3.14E-8:double> --- !query 39 output +-- !query output 3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 --- !query 40 +-- !query select +date '1999-01-01' --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 --- !query 41 +-- !query select +timestamp '1999-01-01' --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 --- !query 42 +-- !query select +interval '1 day' --- !query 42 schema +-- !query schema struct<(+ INTERVAL '1 days'):interval> --- !query 42 output +-- !query output 1 days --- !query 43 +-- !query select +map(1, 2) --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ map(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'map(1, 2)' is of map type.; line 1 pos 7 --- !query 44 +-- !query select +array(1,2) --- !query 44 schema +-- !query schema struct<> --- !query 44 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ array(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'array(1, 2)' is of array type.; line 1 pos 7 --- !query 45 +-- !query select +named_struct('a', 1, 'b', 'spark') --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ named_struct('a', 1, 'b', 'spark'))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'named_struct('a', 1, 'b', 'spark')' is of struct type.; line 1 pos 7 --- !query 46 +-- !query select +X'1' --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7 --- !query 47 +-- !query select -date '1999-01-01' --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 --- !query 48 +-- !query select -timestamp '1999-01-01' --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 --- !query 49 +-- !query select -x'2379ACFe' --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 31987de5da9bd..2c2b1a7856304 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -2,45 +2,45 @@ -- Number of queries: 12 --- !query 0 +-- !query create temporary view data as select * from values ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) as data(a, b, c) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select * from data --- !query 1 schema +-- !query schema struct,c:array>> --- !query 1 output +-- !query output one [11,12,13] [[111,112,113],[121,122,123]] two [21,22,23] [[211,212,213],[221,222,223]] --- !query 2 +-- !query select a, b[0], b[0] + b[1] from data --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output one 11 23 two 21 43 --- !query 3 +-- !query select a, c[0][0] + c[0][0 + 1] from data --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output one 223 two 423 --- !query 4 +-- !query create temporary view primitive_arrays as select * from values ( array(true), array(2Y, 1Y), @@ -64,21 +64,21 @@ create temporary view primitive_arrays as select * from values ( date_array, timestamp_array ) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query select * from primitive_arrays --- !query 5 schema +-- !query schema struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,date_array:array,timestamp_array:array> --- !query 5 output +-- !query output [true] [2,1] [2,1] [2,1] [2,1] [9223372036854775809,9223372036854775808] [2.0,1.0] [2.0,1.0] [2016-03-14,2016-03-13] [2016-11-15 20:54:00,2016-11-12 20:54:00] --- !query 6 +-- !query select array_contains(boolean_array, true), array_contains(boolean_array, false), array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), @@ -91,22 +91,22 @@ select array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') from primitive_arrays --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output true false true false true false true false true false true false true false true false true false true false --- !query 7 +-- !query select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output false false true true --- !query 8 +-- !query select sort_array(boolean_array), sort_array(tinyint_array), @@ -119,31 +119,31 @@ select sort_array(date_array), sort_array(timestamp_array) from primitive_arrays --- !query 8 schema +-- !query schema struct,sort_array(tinyint_array, true):array,sort_array(smallint_array, true):array,sort_array(int_array, true):array,sort_array(bigint_array, true):array,sort_array(decimal_array, true):array,sort_array(double_array, true):array,sort_array(float_array, true):array,sort_array(date_array, true):array,sort_array(timestamp_array, true):array> --- !query 8 output +-- !query output [true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00,2016-11-15 20:54:00] --- !query 9 +-- !query select sort_array(array('b', 'd'), '1') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7 --- !query 10 +-- !query select sort_array(array('b', 'd'), cast(NULL as boolean)) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'sort_array(array('b', 'd'), CAST(NULL AS BOOLEAN))' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7 --- !query 11 +-- !query select size(boolean_array), size(tinyint_array), @@ -156,7 +156,7 @@ select size(date_array), size(timestamp_array) from primitive_arrays --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 2 2 2 2 2 2 2 2 2 diff --git a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out index 42c22a317eb46..552b027df1bc0 100644 --- a/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out @@ -2,196 +2,196 @@ -- Number of queries: 27 --- !query 0 +-- !query select bit_count(null) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output NULL --- !query 1 +-- !query select bit_count(true) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 --- !query 2 +-- !query select bit_count(false) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 0 --- !query 3 +-- !query select bit_count(cast(1 as tinyint)) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 --- !query 4 +-- !query select bit_count(cast(2 as tinyint)) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 --- !query 5 +-- !query select bit_count(cast(3 as tinyint)) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 2 --- !query 6 +-- !query select bit_count(1S) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 --- !query 7 +-- !query select bit_count(2S) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 --- !query 8 +-- !query select bit_count(3S) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 2 --- !query 9 +-- !query select bit_count(1) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 --- !query 10 +-- !query select bit_count(2) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 --- !query 11 +-- !query select bit_count(3) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 2 --- !query 12 +-- !query select bit_count(1L) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 --- !query 13 +-- !query select bit_count(2L) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 --- !query 14 +-- !query select bit_count(3L) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 2 --- !query 15 +-- !query select bit_count(-1L) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 64 --- !query 16 +-- !query select bit_count(9223372036854775807L) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 63 --- !query 17 +-- !query select bit_count(-9223372036854775808L) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 --- !query 18 +-- !query select bit_count("bit count") --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'bit_count('bit count')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''bit count'' is of string type.; line 1 pos 7 --- !query 19 +-- !query select bit_count('a') --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'bit_count('a')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''a'' is of string type.; line 1 pos 7 --- !query 20 +-- !query CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES (1, 1, 1, 1L), (2, 3, 4, null), (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query SELECT BIT_XOR(b3) AS n1 FROM bitwise_test where 1 = 0 --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL --- !query 22 +-- !query SELECT BIT_XOR(b4) AS n1 FROM bitwise_test where b4 is null --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL --- !query 23 +-- !query SELECT BIT_XOR(cast(b1 as tinyint)) AS a4, BIT_XOR(cast(b2 as smallint)) AS b5, @@ -199,35 +199,35 @@ SELECT BIT_XOR(b4) AS d2, BIT_XOR(distinct b4) AS e2 FROM bitwise_test --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 4 5 2 2 2 --- !query 24 +-- !query SELECT bit_xor(b3) FROM bitwise_test GROUP BY b1 & 1 --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 4 6 --- !query 25 +-- !query SELECT b1, bit_xor(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1 1 2 3 --- !query 26 +-- !query SELECT b1, b2, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 1 1 2 3 3 7 7 7 diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 343f9748c5ba8..35b4c0e79720b 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -2,269 +2,269 @@ -- Number of queries: 46 --- !query 0 +-- !query SELECT CAST('1.23' AS int) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1 --- !query 1 +-- !query SELECT CAST('1.23' AS long) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 --- !query 2 +-- !query SELECT CAST('-4.56' AS int) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output -4 --- !query 3 +-- !query SELECT CAST('-4.56' AS long) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output -4 --- !query 4 +-- !query SELECT CAST('abc' AS int) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output NULL --- !query 5 +-- !query SELECT CAST('abc' AS long) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output NULL --- !query 6 +-- !query SELECT CAST('1234567890123' AS int) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL --- !query 7 +-- !query SELECT CAST('12345678901234567890123' AS long) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL --- !query 8 +-- !query SELECT CAST('' AS int) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NULL --- !query 9 +-- !query SELECT CAST('' AS long) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output NULL --- !query 10 +-- !query SELECT CAST(NULL AS int) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output NULL --- !query 11 +-- !query SELECT CAST(NULL AS long) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output NULL --- !query 12 +-- !query SELECT CAST('123.a' AS int) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output NULL --- !query 13 +-- !query SELECT CAST('123.a' AS long) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output NULL --- !query 14 +-- !query SELECT CAST('-2147483648' AS int) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output -2147483648 --- !query 15 +-- !query SELECT CAST('-2147483649' AS int) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output NULL --- !query 16 +-- !query SELECT CAST('2147483647' AS int) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2147483647 --- !query 17 +-- !query SELECT CAST('2147483648' AS int) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output NULL --- !query 18 +-- !query SELECT CAST('-9223372036854775808' AS long) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output -9223372036854775808 --- !query 19 +-- !query SELECT CAST('-9223372036854775809' AS long) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output NULL --- !query 20 +-- !query SELECT CAST('9223372036854775807' AS long) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 9223372036854775807 --- !query 21 +-- !query SELECT CAST('9223372036854775808' AS long) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL --- !query 22 +-- !query SELECT HEX(CAST('abc' AS binary)) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 616263 --- !query 23 +-- !query SELECT HEX(CAST(CAST(123 AS byte) AS binary)) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 7B --- !query 24 +-- !query SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 85 --- !query 25 +-- !query SELECT HEX(CAST(123S AS binary)) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 007B --- !query 26 +-- !query SELECT HEX(CAST(-123S AS binary)) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output FF85 --- !query 27 +-- !query SELECT HEX(CAST(123 AS binary)) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 0000007B --- !query 28 +-- !query SELECT HEX(CAST(-123 AS binary)) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output FFFFFF85 --- !query 29 +-- !query SELECT HEX(CAST(123L AS binary)) --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 000000000000007B --- !query 30 +-- !query SELECT HEX(CAST(-123L AS binary)) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output FFFFFFFFFFFFFF85 --- !query 31 +-- !query DESC FUNCTION boolean --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output Class: org.apache.spark.sql.catalyst.expressions.Cast Function: boolean Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. --- !query 32 +-- !query DESC FUNCTION EXTENDED boolean --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output Class: org.apache.spark.sql.catalyst.expressions.Cast Extended Usage: No example/argument for boolean. @@ -273,108 +273,108 @@ Function: boolean Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. --- !query 33 +-- !query SELECT CAST('interval 3 month 1 hour' AS interval) --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 3 months 1 hours --- !query 34 +-- !query SELECT CAST(interval 3 month 1 hour AS string) --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 3 months 1 hours --- !query 35 +-- !query select cast(' 1' as tinyint) --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 1 --- !query 36 +-- !query select cast(' 1\t' as tinyint) --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 1 --- !query 37 +-- !query select cast(' 1' as smallint) --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 1 --- !query 38 +-- !query select cast(' 1' as INT) --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 1 --- !query 39 +-- !query select cast(' 1' as bigint) --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 1 --- !query 40 +-- !query select cast(' 1' as float) --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 1.0 --- !query 41 +-- !query select cast(' 1 ' as DOUBLE) --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 1.0 --- !query 42 +-- !query select cast('1.0 ' as DEC) --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 1 --- !query 43 +-- !query select cast('\t\t true \n\r ' as boolean) --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output true --- !query 44 +-- !query select cast('\t\n false \t\r' as boolean) --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output false --- !query 45 +-- !query select cast('\t\n xyz \t\r' as boolean) --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output NULL diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 82326346b361c..5bb00e028c4b7 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -2,29 +2,29 @@ -- Number of queries: 25 --- !query 0 +-- !query CREATE TABLE test_change(a INT, b STRING, c INT) using parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query DESC test_change --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output a int b string c int --- !query 2 +-- !query ALTER TABLE test_change CHANGE a --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Operation not allowed: ALTER TABLE table CHANGE COLUMN requires a TYPE or a COMMENT or a FIRST/AFTER(line 1, pos 0) @@ -34,198 +34,198 @@ ALTER TABLE test_change CHANGE a ^^^ --- !query 3 +-- !query DESC test_change --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output a int b string c int --- !query 4 +-- !query ALTER TABLE test_change RENAME COLUMN a TO a1 --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException RENAME COLUMN is only supported with v2 tables.; --- !query 5 +-- !query DESC test_change --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output a int b string c int --- !query 6 +-- !query ALTER TABLE test_change CHANGE a TYPE STRING --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'a' with type 'StringType'; --- !query 7 +-- !query DESC test_change --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output a int b string c int --- !query 8 +-- !query ALTER TABLE test_change CHANGE a TYPE INT AFTER b --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.; --- !query 9 +-- !query ALTER TABLE test_change CHANGE b TYPE STRING FIRST --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.; --- !query 10 +-- !query DESC test_change --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output a int b string c int --- !query 11 +-- !query ALTER TABLE test_change CHANGE a TYPE INT COMMENT 'this is column a' --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query ALTER TABLE test_change CHANGE b TYPE STRING COMMENT '#*02?`' --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query ALTER TABLE test_change CHANGE c TYPE INT COMMENT '' --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query DESC test_change --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output a int this is column a b string #*02?` c int --- !query 15 +-- !query ALTER TABLE test_change CHANGE a TYPE INT COMMENT 'this is column a' --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query DESC test_change --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output a int this is column a b string #*02?` c int --- !query 17 +-- !query ALTER TABLE test_change CHANGE invalid_col TYPE INT --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException Can't find column `invalid_col` given table data columns [`a`, `b`, `c`]; --- !query 18 +-- !query DESC test_change --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output a int this is column a b string #*02?` c int --- !query 19 +-- !query CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query ALTER TABLE temp_view CHANGE a TYPE INT COMMENT 'this is column a' --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.AnalysisException Invalid command: 'temp_view' is a view not a table.; line 1 pos 0 --- !query 21 +-- !query CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT COMMENT 'this is column a' --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException Invalid command: 'global_temp.global_temp_view' is a view not a table.; line 1 pos 0 --- !query 23 +-- !query DROP TABLE test_change --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query DROP VIEW global_temp.global_temp_view --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out index 9fc97f0c39149..f34b75a379aae 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out @@ -2,239 +2,239 @@ -- Number of queries: 28 --- !query 0 +-- !query CREATE DATABASE mydb1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query USE mydb1 --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE DATABASE mydb2 --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query USE mydb2 --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TABLE t1 USING parquet AS SELECT 20 AS i1 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SET spark.sql.crossJoin.enabled = true --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output spark.sql.crossJoin.enabled true --- !query 7 +-- !query USE mydb1 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT i1 FROM t1, mydb1.t1 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'i1' is ambiguous, could be: mydb1.t1.i1, mydb1.t1.i1.; line 1 pos 7 --- !query 9 +-- !query SELECT t1.i1 FROM t1, mydb1.t1 --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException Reference 't1.i1' is ambiguous, could be: mydb1.t1.i1, mydb1.t1.i1.; line 1 pos 7 --- !query 10 +-- !query SELECT mydb1.t1.i1 FROM t1, mydb1.t1 --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'mydb1.t1.i1' is ambiguous, could be: mydb1.t1.i1, mydb1.t1.i1.; line 1 pos 7 --- !query 11 +-- !query SELECT i1 FROM t1, mydb2.t1 --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'i1' is ambiguous, could be: mydb1.t1.i1, mydb2.t1.i1.; line 1 pos 7 --- !query 12 +-- !query SELECT t1.i1 FROM t1, mydb2.t1 --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException Reference 't1.i1' is ambiguous, could be: mydb1.t1.i1, mydb2.t1.i1.; line 1 pos 7 --- !query 13 +-- !query USE mydb2 --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query SELECT i1 FROM t1, mydb1.t1 --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'i1' is ambiguous, could be: mydb2.t1.i1, mydb1.t1.i1.; line 1 pos 7 --- !query 15 +-- !query SELECT t1.i1 FROM t1, mydb1.t1 --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException Reference 't1.i1' is ambiguous, could be: mydb2.t1.i1, mydb1.t1.i1.; line 1 pos 7 --- !query 16 +-- !query SELECT i1 FROM t1, mydb2.t1 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'i1' is ambiguous, could be: mydb2.t1.i1, mydb2.t1.i1.; line 1 pos 7 --- !query 17 +-- !query SELECT t1.i1 FROM t1, mydb2.t1 --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException Reference 't1.i1' is ambiguous, could be: mydb2.t1.i1, mydb2.t1.i1.; line 1 pos 7 --- !query 18 +-- !query SELECT db1.t1.i1 FROM t1, mydb2.t1 --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`db1.t1.i1`' given input columns: [mydb2.t1.i1, mydb2.t1.i1]; line 1 pos 7 --- !query 19 +-- !query SET spark.sql.crossJoin.enabled = false --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output spark.sql.crossJoin.enabled false --- !query 20 +-- !query USE mydb1 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query SELECT mydb1.t1 FROM t1 --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`mydb1.t1`' given input columns: [mydb1.t1.i1]; line 1 pos 7 --- !query 22 +-- !query SELECT t1.x.y.* FROM t1 --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 't1.x.y.*' given input columns 'i1'; --- !query 23 +-- !query SELECT t1 FROM mydb1.t1 --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`t1`' given input columns: [mydb1.t1.i1]; line 1 pos 7 --- !query 24 +-- !query USE mydb2 --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query SELECT mydb1.t1.i1 FROM t1 --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`mydb1.t1.i1`' given input columns: [mydb2.t1.i1]; line 1 pos 7 --- !query 26 +-- !query DROP DATABASE mydb1 CASCADE --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query DROP DATABASE mydb2 CASCADE --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out index 3d8fb661afe55..16ff4f51bd5f9 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out @@ -2,137 +2,137 @@ -- Number of queries: 17 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW view1 AS SELECT 2 AS i1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT view1.* FROM view1 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 2 --- !query 2 +-- !query SELECT * FROM view1 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 2 --- !query 3 +-- !query SELECT view1.i1 FROM view1 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 2 --- !query 4 +-- !query SELECT i1 FROM view1 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 2 --- !query 5 +-- !query SELECT a.i1 FROM view1 AS a --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 2 --- !query 6 +-- !query SELECT i1 FROM view1 AS a --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 2 --- !query 7 +-- !query DROP VIEW view1 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query CREATE OR REPLACE GLOBAL TEMPORARY VIEW view1 as SELECT 1 as i1 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query SELECT * FROM global_temp.view1 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 --- !query 10 +-- !query SELECT global_temp.view1.* FROM global_temp.view1 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 --- !query 11 +-- !query SELECT i1 FROM global_temp.view1 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 --- !query 12 +-- !query SELECT global_temp.view1.i1 FROM global_temp.view1 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 --- !query 13 +-- !query SELECT view1.i1 FROM global_temp.view1 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 --- !query 14 +-- !query SELECT a.i1 FROM global_temp.view1 AS a --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 --- !query 15 +-- !query SELECT i1 FROM global_temp.view1 AS a --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 --- !query 16 +-- !query DROP VIEW global_temp.view1 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out index 73e3fdc08232c..dcfd48b687b17 100644 --- a/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out @@ -2,442 +2,442 @@ -- Number of queries: 55 --- !query 0 +-- !query CREATE DATABASE mydb1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query USE mydb1 --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE DATABASE mydb2 --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query USE mydb2 --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TABLE t1 USING parquet AS SELECT 20 AS i1 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query USE mydb1 --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query SELECT i1 FROM t1 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 --- !query 8 +-- !query SELECT i1 FROM mydb1.t1 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 --- !query 9 +-- !query SELECT t1.i1 FROM t1 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 --- !query 10 +-- !query SELECT t1.i1 FROM mydb1.t1 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 --- !query 11 +-- !query SELECT mydb1.t1.i1 FROM t1 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 --- !query 12 +-- !query SELECT mydb1.t1.i1 FROM mydb1.t1 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 --- !query 13 +-- !query USE mydb2 --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query SELECT i1 FROM t1 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 20 --- !query 15 +-- !query SELECT i1 FROM mydb1.t1 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 --- !query 16 +-- !query SELECT t1.i1 FROM t1 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 20 --- !query 17 +-- !query SELECT t1.i1 FROM mydb1.t1 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 --- !query 18 +-- !query SELECT mydb1.t1.i1 FROM mydb1.t1 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 --- !query 19 +-- !query USE mydb1 --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query SELECT t1.* FROM t1 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1 --- !query 21 +-- !query SELECT mydb1.t1.* FROM mydb1.t1 --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 --- !query 22 +-- !query SELECT t1.* FROM mydb1.t1 --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 1 --- !query 23 +-- !query USE mydb2 --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query SELECT t1.* FROM t1 --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 20 --- !query 25 +-- !query SELECT mydb1.t1.* FROM mydb1.t1 --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1 --- !query 26 +-- !query SELECT t1.* FROM mydb1.t1 --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 --- !query 27 +-- !query SELECT a.* FROM mydb1.t1 AS a --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 1 --- !query 28 +-- !query USE mydb1 --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output --- !query 29 +-- !query CREATE TABLE t3 USING parquet AS SELECT * FROM VALUES (4,1), (3,1) AS t3(c1, c2) --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output --- !query 30 +-- !query CREATE TABLE t4 USING parquet AS SELECT * FROM VALUES (4,1), (2,1) AS t4(c2, c3) --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output --- !query 31 +-- !query SELECT * FROM t3 WHERE c1 IN (SELECT c2 FROM t4 WHERE t4.c3 = t3.c2) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 4 1 --- !query 32 +-- !query SELECT * FROM mydb1.t3 WHERE c1 IN (SELECT mydb1.t4.c2 FROM mydb1.t4 WHERE mydb1.t4.c3 = mydb1.t3.c2) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 4 1 --- !query 33 +-- !query SET spark.sql.crossJoin.enabled = true --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output spark.sql.crossJoin.enabled true --- !query 34 +-- !query SELECT mydb1.t1.i1 FROM t1, mydb2.t1 --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 1 --- !query 35 +-- !query SELECT mydb1.t1.i1 FROM mydb1.t1, mydb2.t1 --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 1 --- !query 36 +-- !query USE mydb2 --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output --- !query 37 +-- !query SELECT mydb1.t1.i1 FROM t1, mydb1.t1 --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 1 --- !query 38 +-- !query SET spark.sql.crossJoin.enabled = false --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output spark.sql.crossJoin.enabled false --- !query 39 +-- !query USE mydb1 --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output --- !query 40 +-- !query CREATE TABLE t5(i1 INT, t5 STRUCT) USING parquet --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output --- !query 41 +-- !query INSERT INTO t5 VALUES(1, (2, 3)) --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output --- !query 42 +-- !query SELECT t5.i1 FROM t5 --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 1 --- !query 43 +-- !query SELECT t5.t5.i1 FROM t5 --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 2 --- !query 44 +-- !query SELECT t5.t5.i1 FROM mydb1.t5 --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 2 --- !query 45 +-- !query SELECT t5.i1 FROM mydb1.t5 --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 1 --- !query 46 +-- !query SELECT t5.* FROM mydb1.t5 --- !query 46 schema +-- !query schema struct> --- !query 46 output +-- !query output 1 {"i1":2,"i2":3} --- !query 47 +-- !query SELECT t5.t5.* FROM mydb1.t5 --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 2 3 --- !query 48 +-- !query SELECT mydb1.t5.t5.i1 FROM mydb1.t5 --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output 2 --- !query 49 +-- !query SELECT mydb1.t5.t5.i2 FROM mydb1.t5 --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output 3 --- !query 50 +-- !query SELECT mydb1.t5.* FROM mydb1.t5 --- !query 50 schema +-- !query schema struct> --- !query 50 output +-- !query output 1 {"i1":2,"i2":3} --- !query 51 +-- !query SELECT mydb1.t5.* FROM t5 --- !query 51 schema +-- !query schema struct> --- !query 51 output +-- !query output 1 {"i1":2,"i2":3} --- !query 52 +-- !query USE default --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output --- !query 53 +-- !query DROP DATABASE mydb1 CASCADE --- !query 53 schema +-- !query schema struct<> --- !query 53 output +-- !query output --- !query 54 +-- !query DROP DATABASE mydb2 CASCADE --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/comparator.sql.out b/sql/core/src/test/resources/sql-tests/results/comparator.sql.out index a182da2a79637..721b56cc1da2f 100644 --- a/sql/core/src/test/resources/sql-tests/results/comparator.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/comparator.sql.out @@ -2,81 +2,81 @@ -- Number of queries: 10 --- !query 0 +-- !query select x'00' < x'0f' --- !query 0 schema +-- !query schema struct<(X'00' < X'0F'):boolean> --- !query 0 output +-- !query output true --- !query 1 +-- !query select x'00' < x'ff' --- !query 1 schema +-- !query schema struct<(X'00' < X'FF'):boolean> --- !query 1 output +-- !query output true --- !query 2 +-- !query select '1 ' = 1Y --- !query 2 schema +-- !query schema struct<(CAST(1 AS TINYINT) = 1):boolean> --- !query 2 output +-- !query output true --- !query 3 +-- !query select '\t1 ' = 1Y --- !query 3 schema +-- !query schema struct<(CAST( 1 AS TINYINT) = 1):boolean> --- !query 3 output +-- !query output true --- !query 4 +-- !query select '1 ' = 1S --- !query 4 schema +-- !query schema struct<(CAST(1 AS SMALLINT) = 1):boolean> --- !query 4 output +-- !query output true --- !query 5 +-- !query select '1 ' = 1 --- !query 5 schema +-- !query schema struct<(CAST(1 AS INT) = 1):boolean> --- !query 5 output +-- !query output true --- !query 6 +-- !query select ' 1' = 1L --- !query 6 schema +-- !query schema struct<(CAST( 1 AS BIGINT) = 1):boolean> --- !query 6 output +-- !query output true --- !query 7 +-- !query select ' 1' = cast(1.0 as float) --- !query 7 schema +-- !query schema struct<(CAST( 1 AS FLOAT) = CAST(1.0 AS FLOAT)):boolean> --- !query 7 output +-- !query output true --- !query 8 +-- !query select ' 1.0 ' = 1.0D --- !query 8 schema +-- !query schema struct<(CAST( 1.0 AS DOUBLE) = 1.0):boolean> --- !query 8 output +-- !query output true --- !query 9 +-- !query select ' 1.0 ' = 1.0BD --- !query 9 schema +-- !query schema struct<(CAST( 1.0 AS DOUBLE) = CAST(1.0 AS DOUBLE)):boolean> --- !query 9 output +-- !query output true diff --git a/sql/core/src/test/resources/sql-tests/results/count.sql.out b/sql/core/src/test/resources/sql-tests/results/count.sql.out index b8a86d4c44594..68a5114bb5859 100644 --- a/sql/core/src/test/resources/sql-tests/results/count.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/count.sql.out @@ -2,27 +2,27 @@ -- Number of queries: 5 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) AS testData(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) FROM testData --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 7 7 0 5 5 4 7 --- !query 2 +-- !query SELECT count(DISTINCT 1), count(DISTINCT null), @@ -31,25 +31,25 @@ SELECT count(DISTINCT (a + b)), count(DISTINCT (a, b)) FROM testData --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 0 2 2 2 6 --- !query 3 +-- !query SELECT count(a, b), count(b, a), count(testData.*) FROM testData --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 4 4 4 --- !query 4 +-- !query SELECT count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*) FROM testData --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 3 3 3 3 diff --git a/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out b/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out index 3833c42bdfecf..ce2305cb7ec90 100644 --- a/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out @@ -2,35 +2,35 @@ -- Number of queries: 13 --- !query 0 +-- !query create temporary view nt1 as select * from values ("one", 1), ("two", 2), ("three", 3) as nt1(k, v1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view nt2 as select * from values ("one", 1), ("two", 22), ("one", 5) as nt2(k, v2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM nt1 cross join nt2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output one 1 one 1 one 1 one 5 one 1 two 22 @@ -42,82 +42,82 @@ two 2 one 5 two 2 two 22 --- !query 3 +-- !query SELECT * FROM nt1 cross join nt2 where nt1.k = nt2.k --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output one 1 one 1 one 1 one 5 two 2 two 22 --- !query 4 +-- !query SELECT * FROM nt1 cross join nt2 on (nt1.k = nt2.k) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output one 1 one 1 one 1 one 5 two 2 two 22 --- !query 5 +-- !query SELECT * FROM nt1 cross join nt2 where nt1.v1 = 1 and nt2.v2 = 22 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output one 1 two 22 --- !query 6 +-- !query SELECT a.key, b.key FROM (SELECT k key FROM nt1 WHERE v1 < 2) a CROSS JOIN (SELECT k key FROM nt2 WHERE v2 = 22) b --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output one two --- !query 7 +-- !query create temporary view A(a, va) as select * from nt1 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query create temporary view B(b, vb) as select * from nt1 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query create temporary view C(c, vc) as select * from nt1 --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query create temporary view D(d, vd) as select * from nt1 --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query select * from ((A join B on (a = b)) cross join C) join D on (a = d) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output one 1 one 1 one 1 one 1 one 1 one 1 three 3 one 1 one 1 one 1 two 2 one 1 @@ -129,11 +129,11 @@ two 2 two 2 three 3 two 2 two 2 two 2 two 2 two 2 --- !query 12 +-- !query SELECT * FROM nt1 CROSS JOIN nt2 ON (nt1.k > nt2.k) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output three 3 one 1 three 3 one 5 two 2 one 1 diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out index 6cae948f1b4b7..8495bef9122ef 100644 --- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -2,36 +2,36 @@ -- Number of queries: 17 --- !query 0 +-- !query select from_csv('1, 3.14', 'a INT, f FLOAT') --- !query 0 schema +-- !query schema struct> --- !query 0 output +-- !query output {"a":1,"f":3.14} --- !query 1 +-- !query select from_csv('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) --- !query 1 schema +-- !query schema struct> --- !query 1 output +-- !query output {"time":2015-08-26 00:00:00} --- !query 2 +-- !query select from_csv('1', 1) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output org.apache.spark.sql.AnalysisException Schema should be specified in DDL format as a string literal or output of the schema_of_csv function instead of 1;; line 1 pos 7 --- !query 3 +-- !query select from_csv('1', 'a InvalidType') --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException DataType invalidtype is not supported.(line 1, pos 2) @@ -42,112 +42,112 @@ a InvalidType ; line 1 pos 7 --- !query 4 +-- !query select from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE')) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException Must use a map() function for options;; line 1 pos 7 --- !query 5 +-- !query select from_csv('1', 'a INT', map('mode', 1)) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException A type of keys and values in map() must be string, but got map;; line 1 pos 7 --- !query 6 +-- !query select from_csv() --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7 --- !query 7 +-- !query select from_csv('1,abc', schema_of_csv('1,abc')) --- !query 7 schema +-- !query schema struct> --- !query 7 output +-- !query output {"_c0":1,"_c1":"abc"} --- !query 8 +-- !query select schema_of_csv('1|abc', map('delimiter', '|')) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output struct<_c0:int,_c1:string> --- !query 9 +-- !query select schema_of_csv(null) --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'schema_of_csv(NULL)' due to data type mismatch: The input csv should be a string literal and not null; however, got NULL.; line 1 pos 7 --- !query 10 +-- !query CREATE TEMPORARY VIEW csvTable(csvField, a) AS SELECT * FROM VALUES ('1,abc', 'a') --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT schema_of_csv(csvField) FROM csvTable --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'schema_of_csv(csvtable.`csvField`)' due to data type mismatch: The input csv should be a string literal and not null; however, got csvtable.`csvField`.; line 1 pos 7 --- !query 12 +-- !query DROP VIEW IF EXISTS csvTable --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query select to_csv(named_struct('a', 1, 'b', 2)) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1,2 --- !query 14 +-- !query select to_csv(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 26/08/2015 --- !query 15 +-- !query select to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException Must use a map() function for options;; line 1 pos 7 --- !query 16 +-- !query select to_csv(named_struct('a', 1, 'b', 2), map('mode', 1)) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.AnalysisException A type of keys and values in map() must be string, but got map;; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out index 5193e2536c0cc..a9709c4a79793 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out @@ -2,65 +2,65 @@ -- Number of queries: 17 --- !query 0 +-- !query create temporary view t as select * from values 0, 1, 2 as t(id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values 0, 1 as t(id) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SET spark.sql.legacy.ctePrecedence.enabled=true --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output spark.sql.legacy.ctePrecedence.enabled true --- !query 3 +-- !query WITH t as ( WITH t2 AS (SELECT 1) SELECT * FROM t2 ) SELECT * FROM t --- !query 3 schema +-- !query schema struct<1:int> --- !query 3 output +-- !query output 1 --- !query 4 +-- !query SELECT max(c) FROM ( WITH t(c) AS (SELECT 1) SELECT * FROM t ) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 --- !query 5 +-- !query SELECT ( WITH t AS (SELECT 1) SELECT * FROM t ) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 --- !query 6 +-- !query WITH t AS (SELECT 1), t2 AS ( @@ -68,13 +68,13 @@ WITH SELECT * FROM t ) SELECT * FROM t2 --- !query 6 schema +-- !query schema struct<1:int> --- !query 6 output +-- !query output 1 --- !query 7 +-- !query WITH t(c) AS (SELECT 1), t2 AS ( @@ -86,13 +86,13 @@ WITH ) ) SELECT * FROM t2 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 --- !query 8 +-- !query WITH t AS (SELECT 1), t2 AS ( @@ -104,25 +104,25 @@ WITH SELECT * FROM t2 ) SELECT * FROM t2 --- !query 8 schema +-- !query schema struct<2:int> --- !query 8 output +-- !query output 2 --- !query 9 +-- !query WITH t(c) AS (SELECT 1) SELECT max(c) FROM ( WITH t(c) AS (SELECT 2) SELECT * FROM t ) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 2 --- !query 10 +-- !query WITH t(c) AS (SELECT 1) SELECT sum(c) FROM ( SELECT max(c) AS c FROM ( @@ -130,13 +130,13 @@ SELECT sum(c) FROM ( SELECT * FROM t ) ) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 2 --- !query 11 +-- !query WITH t(c) AS (SELECT 1) SELECT sum(c) FROM ( WITH t(c) AS (SELECT 2) @@ -145,25 +145,25 @@ SELECT sum(c) FROM ( SELECT * FROM t ) ) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 3 --- !query 12 +-- !query WITH t AS (SELECT 1) SELECT ( WITH t AS (SELECT 2) SELECT * FROM t ) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 --- !query 13 +-- !query WITH t AS (SELECT 1) SELECT ( SELECT ( @@ -171,13 +171,13 @@ SELECT ( SELECT * FROM t ) ) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 --- !query 14 +-- !query WITH t AS (SELECT 1) SELECT ( WITH t AS (SELECT 2) @@ -186,23 +186,23 @@ SELECT ( SELECT * FROM t ) ) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 --- !query 15 +-- !query DROP VIEW IF EXISTS t --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query DROP VIEW IF EXISTS t2 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index b7dd76c725209..2d87781193c25 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -2,72 +2,72 @@ -- Number of queries: 27 --- !query 0 +-- !query create temporary view t as select * from values 0, 1, 2 as t(id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values 0, 1 as t(id) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query WITH s AS (SELECT 1 FROM s) SELECT * FROM s --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output org.apache.spark.sql.AnalysisException Table or view not found: s; line 1 pos 25 --- !query 3 +-- !query WITH r AS (SELECT (SELECT * FROM r)) SELECT * FROM r --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException Table or view not found: r; line 1 pos 33 --- !query 4 +-- !query WITH t AS (SELECT 1 FROM t) SELECT * FROM t --- !query 4 schema +-- !query schema struct<1:int> --- !query 4 output +-- !query output 1 1 1 --- !query 5 +-- !query WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException Table or view not found: s2; line 1 pos 26 --- !query 6 +-- !query WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 0 2 0 2 1 2 1 2 --- !query 7 +-- !query WITH CTE1 AS ( SELECT b.id AS id FROM T2 a @@ -77,9 +77,9 @@ SELECT t1.id AS c1, t2.id AS c2 FROM CTE1 t1 CROSS JOIN CTE1 t2 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 0 0 0 0 0 0 @@ -98,39 +98,39 @@ struct 1 1 --- !query 8 +-- !query WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE x = 1 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 --- !query 9 +-- !query WITH t(x, y) AS (SELECT 1, 2) SELECT * FROM t WHERE x = 1 AND y = 2 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 2 --- !query 10 +-- !query WITH t(x, x) AS (SELECT 1, 2) SELECT * FROM t --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 2 --- !query 11 +-- !query WITH t() AS (SELECT 1) SELECT * FROM t --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'WITH t()'(line 1, pos 7) @@ -141,14 +141,14 @@ WITH t() AS (SELECT 1) SELECT * FROM t --- !query 12 +-- !query WITH t(x) AS (SELECT 1), t(x) AS (SELECT 2) SELECT * FROM t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException CTE definition can't have duplicate names: 't'.(line 1, pos 0) @@ -161,41 +161,41 @@ WITH SELECT * FROM t --- !query 13 +-- !query WITH t as ( WITH t2 AS (SELECT 1) SELECT * FROM t2 ) SELECT * FROM t --- !query 13 schema +-- !query schema struct<1:int> --- !query 13 output +-- !query output 1 --- !query 14 +-- !query SELECT max(c) FROM ( WITH t(c) AS (SELECT 1) SELECT * FROM t ) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 --- !query 15 +-- !query SELECT ( WITH t AS (SELECT 1) SELECT * FROM t ) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 --- !query 16 +-- !query WITH t AS (SELECT 1), t2 AS ( @@ -203,13 +203,13 @@ WITH SELECT * FROM t ) SELECT * FROM t2 --- !query 16 schema +-- !query schema struct<2:int> --- !query 16 output +-- !query output 2 --- !query 17 +-- !query WITH t(c) AS (SELECT 1), t2 AS ( @@ -221,13 +221,13 @@ WITH ) ) SELECT * FROM t2 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 2 --- !query 18 +-- !query WITH t AS (SELECT 1), t2 AS ( @@ -239,25 +239,25 @@ WITH SELECT * FROM t2 ) SELECT * FROM t2 --- !query 18 schema +-- !query schema struct<3:int> --- !query 18 output +-- !query output 3 --- !query 19 +-- !query WITH t(c) AS (SELECT 1) SELECT max(c) FROM ( WITH t(c) AS (SELECT 2) SELECT * FROM t ) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 2 --- !query 20 +-- !query WITH t(c) AS (SELECT 1) SELECT sum(c) FROM ( SELECT max(c) AS c FROM ( @@ -265,13 +265,13 @@ SELECT sum(c) FROM ( SELECT * FROM t ) ) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 2 --- !query 21 +-- !query WITH t(c) AS (SELECT 1) SELECT sum(c) FROM ( WITH t(c) AS (SELECT 2) @@ -280,25 +280,25 @@ SELECT sum(c) FROM ( SELECT * FROM t ) ) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 3 --- !query 22 +-- !query WITH t AS (SELECT 1) SELECT ( WITH t AS (SELECT 2) SELECT * FROM t ) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 2 --- !query 23 +-- !query WITH t AS (SELECT 1) SELECT ( SELECT ( @@ -306,13 +306,13 @@ SELECT ( SELECT * FROM t ) ) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 2 --- !query 24 +-- !query WITH t AS (SELECT 1) SELECT ( WITH t AS (SELECT 2) @@ -321,23 +321,23 @@ SELECT ( SELECT * FROM t ) ) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 3 --- !query 25 +-- !query DROP VIEW IF EXISTS t --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query DROP VIEW IF EXISTS t2 --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 30bae9cd48a9e..b4cceedffd98b 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -2,885 +2,885 @@ -- Number of queries: 110 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select date_part('millennium', c) from t --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 3 --- !query 2 +-- !query select date_part('millennia', c) from t --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 3 --- !query 3 +-- !query select date_part('mil', c) from t --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 3 --- !query 4 +-- !query select date_part('mils', c) from t --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 3 --- !query 5 +-- !query select date_part('century', c) from t --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 21 --- !query 6 +-- !query select date_part('centuries', c) from t --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 21 --- !query 7 +-- !query select date_part('c', c) from t --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 21 --- !query 8 +-- !query select date_part('cent', c) from t --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 21 --- !query 9 +-- !query select date_part('decade', c) from t --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 201 --- !query 10 +-- !query select date_part('decades', c) from t --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 201 --- !query 11 +-- !query select date_part('dec', c) from t --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 201 --- !query 12 +-- !query select date_part('decs', c) from t --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 201 --- !query 13 +-- !query select date_part('year', c) from t --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 2011 --- !query 14 +-- !query select date_part('y', c) from t --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 2011 --- !query 15 +-- !query select date_part('years', c) from t --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 2011 --- !query 16 +-- !query select date_part('yr', c) from t --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2011 --- !query 17 +-- !query select date_part('yrs', c) from t --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 2011 --- !query 18 +-- !query select date_part('quarter', c) from t --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 2 --- !query 19 +-- !query select date_part('qtr', c) from t --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 2 --- !query 20 +-- !query select date_part('month', c) from t --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 5 --- !query 21 +-- !query select date_part('mon', c) from t --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 5 --- !query 22 +-- !query select date_part('mons', c) from t --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 5 --- !query 23 +-- !query select date_part('months', c) from t --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 5 --- !query 24 +-- !query select date_part('week', c) from t --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 18 --- !query 25 +-- !query select date_part('w', c) from t --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 18 --- !query 26 +-- !query select date_part('weeks', c) from t --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 18 --- !query 27 +-- !query select date_part('day', c) from t --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 6 --- !query 28 +-- !query select date_part('d', c) from t --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 6 --- !query 29 +-- !query select date_part('days', c) from t --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 6 --- !query 30 +-- !query select date_part('dayofweek', c) from t --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 6 --- !query 31 +-- !query select date_part('dow', c) from t --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 5 --- !query 32 +-- !query select date_part('isodow', c) from t --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 5 --- !query 33 +-- !query select date_part('doy', c) from t --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 126 --- !query 34 +-- !query select date_part('hour', c) from t --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 7 --- !query 35 +-- !query select date_part('h', c) from t --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 7 --- !query 36 +-- !query select date_part('hours', c) from t --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 7 --- !query 37 +-- !query select date_part('hr', c) from t --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 7 --- !query 38 +-- !query select date_part('hrs', c) from t --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 7 --- !query 39 +-- !query select date_part('minute', c) from t --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 8 --- !query 40 +-- !query select date_part('m', c) from t --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 8 --- !query 41 +-- !query select date_part('min', c) from t --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 8 --- !query 42 +-- !query select date_part('mins', c) from t --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 8 --- !query 43 +-- !query select date_part('minutes', c) from t --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 8 --- !query 44 +-- !query select date_part('second', c) from t --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 9.123456 --- !query 45 +-- !query select date_part('s', c) from t --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 9.123456 --- !query 46 +-- !query select date_part('sec', c) from t --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output 9.123456 --- !query 47 +-- !query select date_part('seconds', c) from t --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 9.123456 --- !query 48 +-- !query select date_part('secs', c) from t --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output 9.123456 --- !query 49 +-- !query select date_part('not_supported', c) from t --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output org.apache.spark.sql.AnalysisException Literals of type 'not_supported' are currently not supported for the string type.;; line 1 pos 7 --- !query 50 +-- !query select date_part(c, c) from t --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output org.apache.spark.sql.AnalysisException The field parameter needs to be a foldable string value.;; line 1 pos 7 --- !query 51 +-- !query select date_part(null, c) from t --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output NULL --- !query 52 +-- !query CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output --- !query 53 +-- !query select date_part('millennium', c) from t2 --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 1 --- !query 54 +-- !query select date_part('millennia', c) from t2 --- !query 54 schema +-- !query schema struct --- !query 54 output +-- !query output 1 --- !query 55 +-- !query select date_part('mil', c) from t2 --- !query 55 schema +-- !query schema struct --- !query 55 output +-- !query output 1 --- !query 56 +-- !query select date_part('mils', c) from t2 --- !query 56 schema +-- !query schema struct --- !query 56 output +-- !query output 1 --- !query 57 +-- !query select date_part('century', c) from t2 --- !query 57 schema +-- !query schema struct --- !query 57 output +-- !query output 10 --- !query 58 +-- !query select date_part('centuries', c) from t2 --- !query 58 schema +-- !query schema struct --- !query 58 output +-- !query output 10 --- !query 59 +-- !query select date_part('c', c) from t2 --- !query 59 schema +-- !query schema struct --- !query 59 output +-- !query output 10 --- !query 60 +-- !query select date_part('cent', c) from t2 --- !query 60 schema +-- !query schema struct --- !query 60 output +-- !query output 10 --- !query 61 +-- !query select date_part('decade', c) from t2 --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output 101 --- !query 62 +-- !query select date_part('decades', c) from t2 --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output 101 --- !query 63 +-- !query select date_part('dec', c) from t2 --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output 101 --- !query 64 +-- !query select date_part('decs', c) from t2 --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 101 --- !query 65 +-- !query select date_part('year', c) from t2 --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output 1010 --- !query 66 +-- !query select date_part('y', c) from t2 --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output 1010 --- !query 67 +-- !query select date_part('years', c) from t2 --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output 1010 --- !query 68 +-- !query select date_part('yr', c) from t2 --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 1010 --- !query 69 +-- !query select date_part('yrs', c) from t2 --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output 1010 --- !query 70 +-- !query select date_part('quarter', c) from t2 --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output 4 --- !query 71 +-- !query select date_part('qtr', c) from t2 --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output 4 --- !query 72 +-- !query select date_part('month', c) from t2 --- !query 72 schema +-- !query schema struct --- !query 72 output +-- !query output 9 --- !query 73 +-- !query select date_part('mon', c) from t2 --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output 9 --- !query 74 +-- !query select date_part('mons', c) from t2 --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output 9 --- !query 75 +-- !query select date_part('months', c) from t2 --- !query 75 schema +-- !query schema struct --- !query 75 output +-- !query output 9 --- !query 76 +-- !query select date_part('day', c) from t2 --- !query 76 schema +-- !query schema struct --- !query 76 output +-- !query output 8 --- !query 77 +-- !query select date_part('d', c) from t2 --- !query 77 schema +-- !query schema struct --- !query 77 output +-- !query output 8 --- !query 78 +-- !query select date_part('days', c) from t2 --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output 8 --- !query 79 +-- !query select date_part('hour', c) from t2 --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output 7 --- !query 80 +-- !query select date_part('h', c) from t2 --- !query 80 schema +-- !query schema struct --- !query 80 output +-- !query output 7 --- !query 81 +-- !query select date_part('hours', c) from t2 --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output 7 --- !query 82 +-- !query select date_part('hr', c) from t2 --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output 7 --- !query 83 +-- !query select date_part('hrs', c) from t2 --- !query 83 schema +-- !query schema struct --- !query 83 output +-- !query output 7 --- !query 84 +-- !query select date_part('minute', c) from t2 --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output 6 --- !query 85 +-- !query select date_part('m', c) from t2 --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output 6 --- !query 86 +-- !query select date_part('min', c) from t2 --- !query 86 schema +-- !query schema struct --- !query 86 output +-- !query output 6 --- !query 87 +-- !query select date_part('mins', c) from t2 --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 6 --- !query 88 +-- !query select date_part('minutes', c) from t2 --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output 6 --- !query 89 +-- !query select date_part('second', c) from t2 --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output 5.004003 --- !query 90 +-- !query select date_part('s', c) from t2 --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 5.004003 --- !query 91 +-- !query select date_part('sec', c) from t2 --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output 5.004003 --- !query 92 +-- !query select date_part('seconds', c) from t2 --- !query 92 schema +-- !query schema struct --- !query 92 output +-- !query output 5.004003 --- !query 93 +-- !query select date_part('secs', c) from t2 --- !query 93 schema +-- !query schema struct --- !query 93 output +-- !query output 5.004003 --- !query 94 +-- !query select date_part('milliseconds', c) from t2 --- !query 94 schema +-- !query schema struct --- !query 94 output +-- !query output 5004.003 --- !query 95 +-- !query select date_part('msec', c) from t2 --- !query 95 schema +-- !query schema struct --- !query 95 output +-- !query output 5004.003 --- !query 96 +-- !query select date_part('msecs', c) from t2 --- !query 96 schema +-- !query schema struct --- !query 96 output +-- !query output 5004.003 --- !query 97 +-- !query select date_part('millisecon', c) from t2 --- !query 97 schema +-- !query schema struct --- !query 97 output +-- !query output 5004.003 --- !query 98 +-- !query select date_part('mseconds', c) from t2 --- !query 98 schema +-- !query schema struct --- !query 98 output +-- !query output 5004.003 --- !query 99 +-- !query select date_part('ms', c) from t2 --- !query 99 schema +-- !query schema struct --- !query 99 output +-- !query output 5004.003 --- !query 100 +-- !query select date_part('microseconds', c) from t2 --- !query 100 schema +-- !query schema struct --- !query 100 output +-- !query output 5004003 --- !query 101 +-- !query select date_part('usec', c) from t2 --- !query 101 schema +-- !query schema struct --- !query 101 output +-- !query output 5004003 --- !query 102 +-- !query select date_part('usecs', c) from t2 --- !query 102 schema +-- !query schema struct --- !query 102 output +-- !query output 5004003 --- !query 103 +-- !query select date_part('useconds', c) from t2 --- !query 103 schema +-- !query schema struct --- !query 103 output +-- !query output 5004003 --- !query 104 +-- !query select date_part('microsecon', c) from t2 --- !query 104 schema +-- !query schema struct --- !query 104 output +-- !query output 5004003 --- !query 105 +-- !query select date_part('us', c) from t2 --- !query 105 schema +-- !query schema struct --- !query 105 output +-- !query output 5004003 --- !query 106 +-- !query select date_part('epoch', c) from t2 --- !query 106 schema +-- !query schema struct --- !query 106 output +-- !query output 31897220765.004003 --- !query 107 +-- !query select date_part('not_supported', c) from t2 --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output org.apache.spark.sql.AnalysisException Literals of type 'not_supported' are currently not supported for the interval type.;; line 1 pos 7 --- !query 108 +-- !query select date_part(c, c) from t2 --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output org.apache.spark.sql.AnalysisException The field parameter needs to be a foldable string value.;; line 1 pos 7 --- !query 109 +-- !query select date_part(null, c) from t2 --- !query 109 schema +-- !query schema struct --- !query 109 output +-- !query output NULL diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 09584550e3f4c..a7b098d79a706 100755 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -2,392 +2,392 @@ -- Number of queries: 47 --- !query 0 +-- !query select current_date = current_date(), current_timestamp = current_timestamp() --- !query 0 schema +-- !query schema struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean> --- !query 0 output +-- !query output true true --- !query 1 +-- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output NULL 2016-12-31 2016-12-31 --- !query 2 +-- !query select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 --- !query 3 +-- !query select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15') --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 7 5 7 NULL 6 --- !query 4 +-- !query create temporary view ttf1 as select * from values (1, 2), (2, 3) as ttf1(current_date, current_timestamp) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query select current_date, current_timestamp from ttf1 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 2 2 3 --- !query 6 +-- !query create temporary view ttf2 as select * from values (1, 2), (2, 3) as ttf2(a, b) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 --- !query 7 schema +-- !query schema struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> --- !query 7 output +-- !query output true true 1 2 true true 2 3 --- !query 8 +-- !query select a, b from ttf2 order by a, current_date --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 2 2 3 --- !query 9 +-- !query select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), weekday('1582-10-15 13:10:15') --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 5 3 5 NULL 4 --- !query 10 +-- !query select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01') --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1500 1 1 --- !query 11 +-- !query select date '2019-01-01\t' --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 2019-01-01 --- !query 12 +-- !query select timestamp '2019-01-01\t' --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 2019-01-01 00:00:00 --- !query 13 +-- !query select timestamp'2011-11-11 11:11:11' + interval '2' day --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 2011-11-13 11:11:11 --- !query 14 +-- !query select timestamp'2011-11-11 11:11:11' - interval '2' day --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 2011-11-09 11:11:11 --- !query 15 +-- !query select date'2011-11-11 11:11:11' + interval '2' second --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 2011-11-11 --- !query 16 +-- !query select date'2011-11-11 11:11:11' - interval '2' second --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2011-11-10 --- !query 17 +-- !query select '2011-11-11' - interval '2' day --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 2011-11-09 00:00:00 --- !query 18 +-- !query select '2011-11-11 11:11:11' - interval '2' second --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 2011-11-11 11:11:09 --- !query 19 +-- !query select '1' - interval '2' second --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output NULL --- !query 20 +-- !query select 1 - interval '2' second --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '1 - INTERVAL '2 seconds'' due to data type mismatch: argument 1 requires timestamp type, however, '1' is of int type.; line 1 pos 7 --- !query 21 +-- !query select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 2078 hours 48 minutes 47.654322 seconds --- !query 22 +-- !query select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output -2078 hours -48 minutes -47.654322 seconds --- !query 23 +-- !query select timestamp'2019-10-06 10:11:12.345678' - null --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output NULL --- !query 24 +-- !query select null - timestamp'2019-10-06 10:11:12.345678' --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output NULL --- !query 25 +-- !query select date_add('2011-11-11', 1Y) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 2011-11-12 --- !query 26 +-- !query select date_add('2011-11-11', 1S) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 2011-11-12 --- !query 27 +-- !query select date_add('2011-11-11', 1) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 2011-11-12 --- !query 28 +-- !query select date_add('2011-11-11', 1L) --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7 --- !query 29 +-- !query select date_add('2011-11-11', 1.0) --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7 --- !query 30 +-- !query select date_add('2011-11-11', 1E1) --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 --- !query 31 +-- !query select date_add('2011-11-11', '1') --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2011-11-11' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 --- !query 32 +-- !query select date_add(date'2011-11-11', 1) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 2011-11-12 --- !query 33 +-- !query select date_add(timestamp'2011-11-11', 1) --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 2011-11-12 --- !query 34 +-- !query select date_sub(date'2011-11-11', 1) --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 2011-11-10 --- !query 35 +-- !query select date_sub(timestamp'2011-11-11', 1) --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 2011-11-10 --- !query 36 +-- !query select date_sub(null, 1) --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output NULL --- !query 37 +-- !query select date_sub(date'2011-11-11', null) --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output NULL --- !query 38 +-- !query select date'2011-11-11' + 1E1 --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(DATE '2011-11-11', 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7 --- !query 39 +-- !query select null + date '2001-09-28' --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output NULL --- !query 40 +-- !query select date '2001-09-28' + 7Y --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 2001-10-05 --- !query 41 +-- !query select 7S + date '2001-09-28' --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 2001-10-05 --- !query 42 +-- !query select date '2001-10-01' - 7 --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 2001-09-24 --- !query 43 +-- !query select date '2001-09-28' + null --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output NULL --- !query 44 +-- !query select date '2001-09-28' - null --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output NULL --- !query 45 +-- !query select null - date '2019-10-06' --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output NULL --- !query 46 +-- !query select date '2001-10-01' - date '2001-09-28' --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output 3 days diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index 324a3ae8efd11..72e46ef493a5d 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -2,334 +2,334 @@ -- Number of queries: 40 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1.0 as a, 0.0 as b --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select a / b from t --- !query 1 schema +-- !query schema struct<(CAST(a AS DECIMAL(2,1)) / CAST(b AS DECIMAL(2,1))):decimal(8,6)> --- !query 1 output +-- !query output NULL --- !query 2 +-- !query select a % b from t --- !query 2 schema +-- !query schema struct<(CAST(a AS DECIMAL(2,1)) % CAST(b AS DECIMAL(2,1))):decimal(1,1)> --- !query 2 output +-- !query output NULL --- !query 3 +-- !query select pmod(a, b) from t --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output NULL --- !query 4 +-- !query create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query select id, a+b, a-b, a*b, a/b from decimals_test order by id --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 1099.00000000000000000 -899.00000000000000000 99900.000000 0.100100 2 24690.24600000000000000 0.00000000000000000 152402061.885129 1.000000 3 1234.22345678910110000 -1233.97654321089890000 152.358023 0.000100 4 123456789123456790.12345678912345679 123456789123456787.87654321087654321 138698367904130467.515623 109890109097814272.043109 --- !query 7 +-- !query select id, a*10, b/10 from decimals_test order by id --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 1000.000000000000000 99.900000000000000000 2 123451.230000000000000 1234.512300000000000000 3 1.234567891011000 123.410000000000000000 4 1234567891234567890.000000000000000 0.112345678912345679 --- !query 8 +-- !query select 10.3 * 3.0 --- !query 8 schema +-- !query schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> --- !query 8 output +-- !query output 30.90 --- !query 9 +-- !query select 10.3000 * 3.0 --- !query 9 schema +-- !query schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> --- !query 9 output +-- !query output 30.90000 --- !query 10 +-- !query select 10.30000 * 30.0 --- !query 10 schema +-- !query schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> --- !query 10 output +-- !query output 309.000000 --- !query 11 +-- !query select 10.300000000000000000 * 3.000000000000000000 --- !query 11 schema +-- !query schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,34)> --- !query 11 output +-- !query output 30.9000000000000000000000000000000000 --- !query 12 +-- !query select 10.300000000000000000 * 3.0000000000000000000 --- !query 12 schema +-- !query schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,34)> --- !query 12 output +-- !query output 30.9000000000000000000000000000000000 --- !query 13 +-- !query select 2.35E10 * 1.0 --- !query 13 schema +-- !query schema struct<(2.35E10 * CAST(1.0 AS DOUBLE)):double> --- !query 13 output +-- !query output 2.35E10 --- !query 14 +-- !query select (5e36BD + 0.1) + 5e36BD --- !query 14 schema +-- !query schema struct<(CAST((CAST(5000000000000000000000000000000000000 AS DECIMAL(38,1)) + CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) + CAST(5000000000000000000000000000000000000 AS DECIMAL(38,1))):decimal(38,1)> --- !query 14 output +-- !query output NULL --- !query 15 +-- !query select (-4e36BD - 0.1) - 7e36BD --- !query 15 schema +-- !query schema struct<(CAST((CAST(-4000000000000000000000000000000000000 AS DECIMAL(38,1)) - CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) - CAST(7000000000000000000000000000000000000 AS DECIMAL(38,1))):decimal(38,1)> --- !query 15 output +-- !query output NULL --- !query 16 +-- !query select 12345678901234567890.0 * 12345678901234567890.0 --- !query 16 schema +-- !query schema struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)> --- !query 16 output +-- !query output NULL --- !query 17 +-- !query select 1e35BD / 0.1 --- !query 17 schema +-- !query schema struct<(CAST(100000000000000000000000000000000000 AS DECIMAL(37,1)) / CAST(0.1 AS DECIMAL(37,1))):decimal(38,6)> --- !query 17 output +-- !query output NULL --- !query 18 +-- !query select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD --- !query 18 schema +-- !query schema struct<(CAST(1234567890123456789000000000000 AS DECIMAL(31,0)) * CAST(12345678901234567890000000 AS DECIMAL(31,0))):decimal(38,0)> --- !query 18 output +-- !query output NULL --- !query 19 +-- !query select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 --- !query 19 schema +-- !query schema struct<(CAST(12345678912345678912345678912.1234567 AS DECIMAL(38,6)) + CAST(9999999999999999999999999999999.12345 AS DECIMAL(38,6))):decimal(38,6)> --- !query 19 output +-- !query output 10012345678912345678912345678911.246907 --- !query 20 +-- !query select 123456789123456789.1234567890 * 1.123456789123456789 --- !query 20 schema +-- !query schema struct<(CAST(123456789123456789.1234567890 AS DECIMAL(36,18)) * CAST(1.123456789123456789 AS DECIMAL(36,18))):decimal(38,18)> --- !query 20 output +-- !query output 138698367904130467.654320988515622621 --- !query 21 +-- !query select 12345678912345.123456789123 / 0.000000012345678 --- !query 21 schema +-- !query schema struct<(CAST(12345678912345.123456789123 AS DECIMAL(29,15)) / CAST(1.2345678E-8 AS DECIMAL(29,15))):decimal(38,9)> --- !query 21 output +-- !query output 1000000073899961059796.725866332 --- !query 22 +-- !query set spark.sql.decimalOperations.allowPrecisionLoss=false --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output spark.sql.decimalOperations.allowPrecisionLoss false --- !query 23 +-- !query select id, a+b, a-b, a*b, a/b from decimals_test order by id --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 1 1099.000000000000000000 -899.000000000000000000 NULL 0.100100100100100100 2 24690.246000000000000000 0.000000000000000000 NULL 1.000000000000000000 3 1234.223456789101100000 -1233.976543210898900000 NULL 0.000100037913541123 4 123456789123456790.123456789123456789 123456789123456787.876543210876543211 NULL 109890109097814272.043109406191131436 --- !query 24 +-- !query select id, a*10, b/10 from decimals_test order by id --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1 1000.000000000000000000 99.9000000000000000000 2 123451.230000000000000000 1234.5123000000000000000 3 1.234567891011000000 123.4100000000000000000 4 1234567891234567890.000000000000000000 0.1123456789123456789 --- !query 25 +-- !query select 10.3 * 3.0 --- !query 25 schema +-- !query schema struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> --- !query 25 output +-- !query output 30.90 --- !query 26 +-- !query select 10.3000 * 3.0 --- !query 26 schema +-- !query schema struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> --- !query 26 output +-- !query output 30.90000 --- !query 27 +-- !query select 10.30000 * 30.0 --- !query 27 schema +-- !query schema struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> --- !query 27 output +-- !query output 309.000000 --- !query 28 +-- !query select 10.300000000000000000 * 3.000000000000000000 --- !query 28 schema +-- !query schema struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,36)> --- !query 28 output +-- !query output 30.900000000000000000000000000000000000 --- !query 29 +-- !query select 10.300000000000000000 * 3.0000000000000000000 --- !query 29 schema +-- !query schema struct<(CAST(10.300000000000000000 AS DECIMAL(21,19)) * CAST(3.0000000000000000000 AS DECIMAL(21,19))):decimal(38,37)> --- !query 29 output +-- !query output NULL --- !query 30 +-- !query select 2.35E10 * 1.0 --- !query 30 schema +-- !query schema struct<(2.35E10 * CAST(1.0 AS DOUBLE)):double> --- !query 30 output +-- !query output 2.35E10 --- !query 31 +-- !query select (5e36BD + 0.1) + 5e36BD --- !query 31 schema +-- !query schema struct<(CAST((CAST(5000000000000000000000000000000000000 AS DECIMAL(38,1)) + CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) + CAST(5000000000000000000000000000000000000 AS DECIMAL(38,1))):decimal(38,1)> --- !query 31 output +-- !query output NULL --- !query 32 +-- !query select (-4e36BD - 0.1) - 7e36BD --- !query 32 schema +-- !query schema struct<(CAST((CAST(-4000000000000000000000000000000000000 AS DECIMAL(38,1)) - CAST(0.1 AS DECIMAL(38,1))) AS DECIMAL(38,1)) - CAST(7000000000000000000000000000000000000 AS DECIMAL(38,1))):decimal(38,1)> --- !query 32 output +-- !query output NULL --- !query 33 +-- !query select 12345678901234567890.0 * 12345678901234567890.0 --- !query 33 schema +-- !query schema struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)> --- !query 33 output +-- !query output NULL --- !query 34 +-- !query select 1e35BD / 0.1 --- !query 34 schema +-- !query schema struct<(CAST(100000000000000000000000000000000000 AS DECIMAL(37,1)) / CAST(0.1 AS DECIMAL(37,1))):decimal(38,3)> --- !query 34 output +-- !query output NULL --- !query 35 +-- !query select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD --- !query 35 schema +-- !query schema struct<(CAST(1234567890123456789000000000000 AS DECIMAL(31,0)) * CAST(12345678901234567890000000 AS DECIMAL(31,0))):decimal(38,0)> --- !query 35 output +-- !query output NULL --- !query 36 +-- !query select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 --- !query 36 schema +-- !query schema struct<(CAST(12345678912345678912345678912.1234567 AS DECIMAL(38,7)) + CAST(9999999999999999999999999999999.12345 AS DECIMAL(38,7))):decimal(38,7)> --- !query 36 output +-- !query output NULL --- !query 37 +-- !query select 123456789123456789.1234567890 * 1.123456789123456789 --- !query 37 schema +-- !query schema struct<(CAST(123456789123456789.1234567890 AS DECIMAL(36,18)) * CAST(1.123456789123456789 AS DECIMAL(36,18))):decimal(38,28)> --- !query 37 output +-- !query output NULL --- !query 38 +-- !query select 12345678912345.123456789123 / 0.000000012345678 --- !query 38 schema +-- !query schema struct<(CAST(12345678912345.123456789123 AS DECIMAL(29,15)) / CAST(1.2345678E-8 AS DECIMAL(29,15))):decimal(38,18)> --- !query 38 output +-- !query output NULL --- !query 39 +-- !query drop table decimals_test --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out index e12889f9bff83..24927c34c57b4 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out @@ -2,47 +2,47 @@ -- Number of queries: 15 --- !query 0 +-- !query CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet PARTITIONED BY (ds, hr) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=10) VALUES ('k1', 100), ('k2', 200), ('k3', 300) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=11) VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO TABLE t PARTITION (ds='2017-09-01', hr=5) VALUES ('k1', 102), ('k2', 202) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output key string value string ds string @@ -64,19 +64,19 @@ Last Access [not included in comparison] Location [not included in comparison]/{warehouse_dir}/t --- !query 5 +-- !query ANALYZE TABLE t PARTITION (ds='2017-08-01', hr=10) COMPUTE STATISTICS --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output key string value string ds string @@ -99,19 +99,19 @@ Partition Statistics [not included in comparison] bytes, 3 rows Location [not included in comparison]/{warehouse_dir}/t --- !query 7 +-- !query ANALYZE TABLE t PARTITION (ds='2017-08-01') COMPUTE STATISTICS --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output key string value string ds string @@ -134,11 +134,11 @@ Partition Statistics [not included in comparison] bytes, 3 rows Location [not included in comparison]/{warehouse_dir}/t --- !query 9 +-- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output key string value string ds string @@ -161,19 +161,19 @@ Partition Statistics [not included in comparison] bytes, 4 rows Location [not included in comparison]/{warehouse_dir}/t --- !query 10 +-- !query ANALYZE TABLE t PARTITION (ds, hr) COMPUTE STATISTICS --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output key string value string ds string @@ -196,11 +196,11 @@ Partition Statistics [not included in comparison] bytes, 3 rows Location [not included in comparison]/{warehouse_dir}/t --- !query 12 +-- !query DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output key string value string ds string @@ -223,11 +223,11 @@ Partition Statistics [not included in comparison] bytes, 4 rows Location [not included in comparison]/{warehouse_dir}/t --- !query 13 +-- !query DESC EXTENDED t PARTITION (ds='2017-09-01', hr=5) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output key string value string ds string @@ -250,9 +250,9 @@ Partition Statistics [not included in comparison] bytes, 2 rows Location [not included in comparison]/{warehouse_dir}/t --- !query 14 +-- !query DROP TABLE t --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out index e41534681dc91..6b16aba268f50 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out @@ -2,114 +2,114 @@ -- Number of queries: 19 --- !query 0 +-- !query CREATE table desc_temp1 (key int COMMENT 'column_comment', val string) USING PARQUET --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE table desc_temp2 (key int, val string) USING PARQUET --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query DESC SELECT key, key + 1 as plusone FROM desc_temp1 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output key int column_comment plusone int --- !query 3 +-- !query DESC QUERY SELECT * FROM desc_temp2 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output key int val string --- !query 4 +-- !query DESC SELECT key, COUNT(*) as count FROM desc_temp1 group by key --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output key int column_comment count bigint --- !query 5 +-- !query DESC SELECT 10.00D as col1 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output col1 double --- !query 6 +-- !query DESC QUERY SELECT key FROM desc_temp1 UNION ALL select CAST(1 AS DOUBLE) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output key double --- !query 7 +-- !query DESC QUERY VALUES(1.00D, 'hello') as tab1(col1, col2) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output col1 double col2 string --- !query 8 +-- !query DESC QUERY FROM desc_temp1 a SELECT * --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output key int column_comment val string --- !query 9 +-- !query DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output col1 string --- !query 10 +-- !query DESCRIBE QUERY WITH s AS (SELECT * from desc_temp1) SELECT * FROM s --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output key int column_comment val string --- !query 11 +-- !query DESCRIBE SELECT * FROM (FROM desc_temp2 select * select *) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output key int val string --- !query 12 +-- !query DESCRIBE INSERT INTO desc_temp1 values (1, 'val1') --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input 'desc_temp1' expecting {, '.'}(line 1, pos 21) @@ -119,11 +119,11 @@ DESCRIBE INSERT INTO desc_temp1 values (1, 'val1') ---------------------^^^ --- !query 13 +-- !query DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2 --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input 'desc_temp1' expecting {, '.'}(line 1, pos 21) @@ -133,14 +133,14 @@ DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2 ---------------------^^^ --- !query 14 +-- !query DESCRIBE FROM desc_temp1 a insert into desc_temp1 select * insert into desc_temp2 select * --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input 'insert' expecting {'MAP', 'REDUCE', 'SELECT'}(line 3, pos 5) @@ -153,21 +153,21 @@ DESCRIBE insert into desc_temp2 select * --- !query 15 +-- !query EXPLAIN DESC QUERY SELECT * FROM desc_temp2 WHERE key > 0 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output == Physical Plan == Execute DescribeQueryCommand +- DescribeQueryCommand SELECT * FROM desc_temp2 WHERE key > 0 --- !query 16 +-- !query EXPLAIN EXTENDED DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output == Parsed Logical Plan == DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s @@ -183,17 +183,17 @@ Execute DescribeQueryCommand +- DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s --- !query 17 +-- !query DROP TABLE desc_temp1 --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output --- !query 18 +-- !query DROP TABLE desc_temp2 --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out index ffd6fb31620d3..3029fa8e83077 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out @@ -2,19 +2,19 @@ -- Number of queries: 12 --- !query 0 +-- !query CREATE TABLE table_with_comment (a STRING, b INT, c STRING, d STRING) USING parquet COMMENT 'added' --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query DESC FORMATTED table_with_comment --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output a string b int c string @@ -32,19 +32,19 @@ Comment added Location [not included in comparison]/{warehouse_dir}/table_with_comment --- !query 2 +-- !query ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment", "type"= "parquet") --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query DESC FORMATTED table_with_comment --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output a string b int c string @@ -63,27 +63,27 @@ Table Properties [type=parquet] Location [not included in comparison]/{warehouse_dir}/table_with_comment --- !query 4 +-- !query DROP TABLE table_with_comment --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TABLE table_comment (a STRING, b INT) USING parquet --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query DESC FORMATTED table_comment --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output a string b int @@ -98,19 +98,19 @@ Provider parquet Location [not included in comparison]/{warehouse_dir}/table_comment --- !query 7 +-- !query ALTER TABLE table_comment SET TBLPROPERTIES(comment = "added comment") --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query DESC formatted table_comment --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output a string b int @@ -126,19 +126,19 @@ Comment added comment Location [not included in comparison]/{warehouse_dir}/table_comment --- !query 9 +-- !query ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query DESC FORMATTED table_comment --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output a string b int @@ -153,9 +153,9 @@ Provider parquet Location [not included in comparison]/{warehouse_dir}/table_comment --- !query 11 +-- !query DROP TABLE table_comment --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out index a5471c0f1c8ec..ae9240ec588da 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out @@ -2,29 +2,29 @@ -- Number of queries: 28 --- !query 0 +-- !query CREATE TEMPORARY VIEW desc_col_temp_view (key int COMMENT 'column_comment') USING PARQUET --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query DESC desc_col_temp_view key --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output col_name key data_type int comment column_comment --- !query 2 +-- !query DESC EXTENDED desc_col_temp_view key --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output col_name key data_type int comment column_comment @@ -37,11 +37,11 @@ max_col_len NULL histogram NULL --- !query 3 +-- !query DESC FORMATTED desc_col_temp_view key --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output col_name key data_type int comment column_comment @@ -54,11 +54,11 @@ max_col_len NULL histogram NULL --- !query 4 +-- !query DESC FORMATTED desc_col_temp_view desc_col_temp_view.key --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output col_name key data_type int comment column_comment @@ -71,46 +71,46 @@ max_col_len NULL histogram NULL --- !query 5 +-- !query DESC desc_col_temp_view key1 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException Column key1 does not exist; --- !query 6 +-- !query CREATE TABLE desc_col_table (key int COMMENT 'column_comment') USING PARQUET --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query DESC desc_col_table key --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output col_name key data_type int comment column_comment --- !query 9 +-- !query DESC EXTENDED desc_col_table key --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output col_name key data_type int comment column_comment @@ -123,11 +123,11 @@ max_col_len 4 histogram NULL --- !query 10 +-- !query DESC FORMATTED desc_col_table key --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output col_name key data_type int comment column_comment @@ -140,19 +140,19 @@ max_col_len 4 histogram NULL --- !query 11 +-- !query CREATE TABLE desc_complex_col_table (`a.b` int, col struct) USING PARQUET --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query DESC FORMATTED desc_complex_col_table `a.b` --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output col_name a.b data_type int comment NULL @@ -165,11 +165,11 @@ max_col_len NULL histogram NULL --- !query 13 +-- !query DESC FORMATTED desc_complex_col_table col --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output col_name col data_type struct comment NULL @@ -182,52 +182,52 @@ max_col_len NULL histogram NULL --- !query 14 +-- !query DESC FORMATTED desc_complex_col_table col.x --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException DESC TABLE COLUMN command does not support nested data types: col.x; --- !query 15 +-- !query SET spark.sql.statistics.histogram.enabled=true --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output spark.sql.statistics.histogram.enabled true --- !query 16 +-- !query SET spark.sql.statistics.histogram.numBins=2 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output spark.sql.statistics.histogram.numBins 2 --- !query 17 +-- !query INSERT INTO desc_col_table values 1, 2, 3, 4 --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output --- !query 18 +-- !query ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query DESC EXTENDED desc_col_table key --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output col_name key data_type int comment column_comment @@ -242,59 +242,59 @@ bin_0 lower_bound: 1.0, upper_bound: 2.0, distinct_count: 2 bin_1 lower_bound: 2.0, upper_bound: 4.0, distinct_count: 2 --- !query 20 +-- !query DROP VIEW desc_col_temp_view --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query DROP TABLE desc_col_table --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query DROP TABLE desc_complex_col_table --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output --- !query 23 +-- !query CREATE TABLE customer(CName STRING) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query INSERT INTO customer VALUES('Maria') --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query ANALYZE TABLE customer COMPUTE STATISTICS FOR COLUMNS cname --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query DESC EXTENDED customer cname --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output col_name cname data_type string comment NULL @@ -307,9 +307,9 @@ max_col_len 5 histogram NULL --- !query 27 +-- !query DROP TABLE customer --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 1d83717f60b8d..697e006544acf 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -2,68 +2,68 @@ -- Number of queries: 41 --- !query 0 +-- !query CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet OPTIONS (a '1', b '2') PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS COMMENT 'table_comment' TBLPROPERTIES (t 'test') --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW temp_Data_Source_View USING org.apache.spark.sql.sources.DDLScanSource OPTIONS ( From '1', To '10', Table 'test1') --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE VIEW v AS SELECT * FROM t --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query ALTER TABLE t SET TBLPROPERTIES (e = '3') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query ALTER TABLE t ADD PARTITION (c='Us', d=1) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query DESCRIBE t --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output a string b int c string @@ -74,11 +74,11 @@ c string d string --- !query 7 +-- !query DESC default.t --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output a string b int c string @@ -89,11 +89,11 @@ c string d string --- !query 8 +-- !query DESC TABLE t --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output a string b int c string @@ -104,11 +104,11 @@ c string d string --- !query 9 +-- !query DESC FORMATTED t --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output a string b int c string @@ -136,11 +136,11 @@ Storage Properties [a=1, b=2] Partition Provider Catalog --- !query 10 +-- !query DESC EXTENDED t --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output a string b int c string @@ -168,19 +168,19 @@ Storage Properties [a=1, b=2] Partition Provider Catalog --- !query 11 +-- !query ALTER TABLE t UNSET TBLPROPERTIES (e) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query DESC EXTENDED t --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output a string b int c string @@ -208,19 +208,19 @@ Storage Properties [a=1, b=2] Partition Provider Catalog --- !query 13 +-- !query ALTER TABLE t UNSET TBLPROPERTIES (comment) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query DESC EXTENDED t --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output a string b int c string @@ -247,11 +247,11 @@ Storage Properties [a=1, b=2] Partition Provider Catalog --- !query 15 +-- !query DESC t PARTITION (c='Us', d=1) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output a string b int c string @@ -262,11 +262,11 @@ c string d string --- !query 16 +-- !query DESC EXTENDED t PARTITION (c='Us', d=1) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output a string b int c string @@ -293,11 +293,11 @@ Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] --- !query 17 +-- !query DESC FORMATTED t PARTITION (c='Us', d=1) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output a string b int c string @@ -324,31 +324,31 @@ Location [not included in comparison]/{warehouse_dir}/t Storage Properties [a=1, b=2] --- !query 18 +-- !query DESC t PARTITION (c='Us', d=2) --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException Partition not found in table 't' database 'default': c -> Us d -> 2; --- !query 19 +-- !query DESC t PARTITION (c='Us') --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`default`.`t`'; --- !query 20 +-- !query DESC t PARTITION (c='Us', d) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException PARTITION specification is incomplete: `d`(line 1, pos 0) @@ -358,55 +358,55 @@ DESC t PARTITION (c='Us', d) ^^^ --- !query 21 +-- !query DESC temp_v --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output a string b int c string d string --- !query 22 +-- !query DESC TABLE temp_v --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output a string b int c string d string --- !query 23 +-- !query DESC FORMATTED temp_v --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output a string b int c string d string --- !query 24 +-- !query DESC EXTENDED temp_v --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output a string b int c string d string --- !query 25 +-- !query DESC temp_Data_Source_View --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output intType int test comment test1 stringType string dateType date @@ -425,42 +425,42 @@ arrayType array structType struct --- !query 26 +-- !query DESC temp_v PARTITION (c='Us', d=1) --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output org.apache.spark.sql.AnalysisException DESC PARTITION is not allowed on a temporary view: temp_v; --- !query 27 +-- !query DESC v --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output a string b int c string d string --- !query 28 +-- !query DESC TABLE v --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output a string b int c string d string --- !query 29 +-- !query DESC FORMATTED v --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output a string b int c string @@ -480,11 +480,11 @@ View Query Output Columns [a, b, c, d] Table Properties [view.query.out.col.3=d, view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=4, view.query.out.col.1=b, view.catalogAndNamespace.part.0=spark_catalog, view.query.out.col.2=c, view.catalogAndNamespace.part.1=default] --- !query 30 +-- !query DESC EXTENDED v --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output a string b int c string @@ -504,40 +504,40 @@ View Query Output Columns [a, b, c, d] Table Properties [view.query.out.col.3=d, view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=4, view.query.out.col.1=b, view.catalogAndNamespace.part.0=spark_catalog, view.query.out.col.2=c, view.catalogAndNamespace.part.1=default] --- !query 31 +-- !query DESC v PARTITION (c='Us', d=1) --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output org.apache.spark.sql.AnalysisException DESC PARTITION is not allowed on a view: v; --- !query 32 +-- !query EXPLAIN DESC t --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output == Physical Plan == Execute DescribeTableCommand +- DescribeTableCommand `t`, false --- !query 33 +-- !query EXPLAIN DESC EXTENDED t --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output == Physical Plan == Execute DescribeTableCommand +- DescribeTableCommand `t`, true --- !query 34 +-- !query EXPLAIN EXTENDED DESC t --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output == Parsed Logical Plan == 'DescribeRelation false +- 'UnresolvedTableOrView [t] @@ -554,53 +554,53 @@ Execute DescribeTableCommand +- DescribeTableCommand `t`, false --- !query 35 +-- !query EXPLAIN DESCRIBE t b --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output == Physical Plan == Execute DescribeColumnCommand +- DescribeColumnCommand `t`, [b], false --- !query 36 +-- !query EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output == Physical Plan == Execute DescribeTableCommand +- DescribeTableCommand `t`, Map(c -> Us, d -> 2), false --- !query 37 +-- !query DROP TABLE t --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output --- !query 38 +-- !query DROP VIEW temp_v --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output --- !query 39 +-- !query DROP VIEW temp_Data_Source_View --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output --- !query 40 +-- !query DROP VIEW v --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/except-all.sql.out b/sql/core/src/test/resources/sql-tests/results/except-all.sql.out index 01091a2f751ce..601ff8f024214 100644 --- a/sql/core/src/test/resources/sql-tests/results/except-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/except-all.sql.out @@ -2,25 +2,25 @@ -- Number of queries: 27 --- !query 0 +-- !query CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES (1, 2), (1, 2), @@ -28,13 +28,13 @@ CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES (2, 3), (2, 2) AS tab3(k, v) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES (1, 2), (2, 3), @@ -42,45 +42,45 @@ CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES (2, 2), (2, 20) AS tab4(k, v) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query SELECT * FROM tab1 EXCEPT ALL SELECT * FROM tab2 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 0 2 2 NULL --- !query 5 +-- !query SELECT * FROM tab1 MINUS ALL SELECT * FROM tab2 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 0 2 2 NULL --- !query 6 +-- !query SELECT * FROM tab1 EXCEPT ALL SELECT * FROM tab2 WHERE c1 IS NOT NULL --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 0 2 2 @@ -88,23 +88,23 @@ NULL NULL --- !query 7 +-- !query SELECT * FROM tab1 WHERE c1 > 5 EXCEPT ALL SELECT * FROM tab2 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT * FROM tab1 EXCEPT ALL SELECT * FROM tab2 WHERE c1 > 6 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 0 1 2 @@ -116,13 +116,13 @@ NULL NULL --- !query 9 +-- !query SELECT * FROM tab1 EXCEPT ALL SELECT CAST(1 AS BIGINT) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 0 2 2 @@ -133,65 +133,65 @@ NULL NULL --- !query 10 +-- !query SELECT * FROM tab1 EXCEPT ALL SELECT array(1) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException ExceptAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; --- !query 11 +-- !query SELECT * FROM tab3 EXCEPT ALL SELECT * FROM tab4 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 2 1 3 --- !query 12 +-- !query SELECT * FROM tab4 EXCEPT ALL SELECT * FROM tab3 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 2 2 2 20 --- !query 13 +-- !query SELECT * FROM tab4 EXCEPT ALL SELECT * FROM tab3 INTERSECT DISTINCT SELECT * FROM tab4 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 2 2 2 20 --- !query 14 +-- !query SELECT * FROM tab4 EXCEPT ALL SELECT * FROM tab3 EXCEPT DISTINCT SELECT * FROM tab4 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output --- !query 15 +-- !query SELECT * FROM tab3 EXCEPT ALL SELECT * FROM tab4 @@ -199,24 +199,24 @@ UNION ALL SELECT * FROM tab3 EXCEPT DISTINCT SELECT * FROM tab4 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 3 --- !query 16 +-- !query SELECT k FROM tab3 EXCEPT ALL SELECT k, v FROM tab4 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.AnalysisException ExceptAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; --- !query 17 +-- !query SELECT * FROM tab3 EXCEPT ALL SELECT * FROM tab4 @@ -224,13 +224,13 @@ UNION SELECT * FROM tab3 EXCEPT DISTINCT SELECT * FROM tab4 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 3 --- !query 18 +-- !query SELECT * FROM tab3 MINUS ALL SELECT * FROM tab4 @@ -238,13 +238,13 @@ UNION SELECT * FROM tab3 MINUS DISTINCT SELECT * FROM tab4 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 3 --- !query 19 +-- !query SELECT * FROM tab3 EXCEPT ALL SELECT * FROM tab4 @@ -252,13 +252,13 @@ EXCEPT DISTINCT SELECT * FROM tab3 EXCEPT DISTINCT SELECT * FROM tab4 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output --- !query 20 +-- !query SELECT * FROM (SELECT tab3.k, tab4.v @@ -272,13 +272,13 @@ FROM (SELECT tab3.k, FROM tab3 JOIN tab4 ON tab3.k = tab4.k) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output --- !query 21 +-- !query SELECT * FROM (SELECT tab3.k, tab4.v @@ -292,9 +292,9 @@ FROM (SELECT tab4.v AS k, FROM tab3 JOIN tab4 ON tab3.k = tab4.k) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 2 1 2 1 2 @@ -304,43 +304,43 @@ struct 2 3 --- !query 22 +-- !query SELECT v FROM tab3 GROUP BY v EXCEPT ALL SELECT k FROM tab4 GROUP BY k --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 3 --- !query 23 +-- !query DROP VIEW IF EXISTS tab1 --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query DROP VIEW IF EXISTS tab2 --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query DROP VIEW IF EXISTS tab3 --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query DROP VIEW IF EXISTS tab4 --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/except.sql.out b/sql/core/src/test/resources/sql-tests/results/except.sql.out index c9b712d4d2949..62d695219d01d 100644 --- a/sql/core/src/test/resources/sql-tests/results/except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/except.sql.out @@ -2,20 +2,20 @@ -- Number of queries: 9 --- !query 0 +-- !query create temporary view t1 as select * from values ("one", 1), ("two", 2), ("three", 3), ("one", NULL) as t1(k, v) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("one", 1), ("two", 22), @@ -23,71 +23,71 @@ create temporary view t2 as select * from values ("one", NULL), (NULL, 5) as t2(k, v) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM t1 EXCEPT SELECT * FROM t2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output three 3 two 2 --- !query 3 +-- !query SELECT * FROM t1 EXCEPT SELECT * FROM t1 where v <> 1 and v <> 2 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output one 1 one NULL two 2 --- !query 4 +-- !query SELECT * FROM t1 where v <> 1 and v <> 22 EXCEPT SELECT * FROM t1 where v <> 2 and v >= 3 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output two 2 --- !query 5 +-- !query SELECT t1.* FROM t1, t2 where t1.k = t2.k EXCEPT SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != 'one' --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output one 1 one NULL --- !query 6 +-- !query SELECT * FROM t2 where v >= 1 and v <> 22 EXCEPT SELECT * FROM t1 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL 5 one 5 --- !query 7 +-- !query SELECT (SELECT min(k) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 MINUS SELECT (SELECT min(k) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = 'one' --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL two --- !query 8 +-- !query SELECT t1.k FROM t1 WHERE t1.v <= (SELECT max(t2.v) @@ -99,7 +99,7 @@ FROM t1 WHERE t1.v >= (SELECT min(t2.v) FROM t2 WHERE t2.k = t1.k) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output two diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index 85c938773efec..756c14f28a657 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -2,48 +2,48 @@ -- Number of queries: 18 --- !query 0 +-- !query CREATE table explain_temp1 (key int, val int) USING PARQUET --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE table explain_temp2 (key int, val int) USING PARQUET --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE table explain_temp3 (key int, val int) USING PARQUET --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SET spark.sql.codegen.wholeStage = true --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output spark.sql.codegen.wholeStage true --- !query 4 +-- !query EXPLAIN FORMATTED SELECT key, max(val) FROM explain_temp1 WHERE key > 0 GROUP BY key ORDER BY key --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output == Physical Plan == * Sort (9) +- Exchange (8) @@ -90,16 +90,16 @@ Input: [key#x, max(val)#x] Input: [key#x, max(val)#x] --- !query 5 +-- !query EXPLAIN FORMATTED SELECT key, max(val) FROM explain_temp1 WHERE key > 0 GROUP BY key HAVING max(val) > 0 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output == Physical Plan == * Project (9) +- * Filter (8) @@ -148,14 +148,14 @@ Output : [key#x, max(val)#x] Input : [key#x, max(val)#x, max(val#x)#x] --- !query 6 +-- !query EXPLAIN FORMATTED SELECT key, val FROM explain_temp1 WHERE key > 0 UNION SELECT key, val FROM explain_temp1 WHERE key > 0 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output == Physical Plan == * HashAggregate (12) +- Exchange (11) @@ -219,15 +219,15 @@ Input: [key#x, val#x] Input: [key#x, val#x] --- !query 7 +-- !query EXPLAIN FORMATTED SELECT * FROM explain_temp1 a, explain_temp2 b WHERE a.key = b.key --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output == Physical Plan == * BroadcastHashJoin Inner BuildRight (10) :- * Project (4) @@ -286,15 +286,15 @@ Right keys: List(key#x) Join condition: None --- !query 8 +-- !query EXPLAIN FORMATTED SELECT * FROM explain_temp1 a LEFT OUTER JOIN explain_temp2 b ON a.key = b.key --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output == Physical Plan == * BroadcastHashJoin LeftOuter BuildRight (8) :- * ColumnarToRow (2) @@ -342,7 +342,7 @@ Right keys: List(key#x) Join condition: None --- !query 9 +-- !query EXPLAIN FORMATTED SELECT * FROM explain_temp1 @@ -353,9 +353,9 @@ EXPLAIN FORMATTED WHERE val > 0) AND val = 2) AND val > 3 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output == Physical Plan == * Project (4) +- * Filter (3) @@ -458,7 +458,7 @@ Input: [max#x] Input: [max#x] --- !query 10 +-- !query EXPLAIN FORMATTED SELECT * FROM explain_temp1 @@ -469,9 +469,9 @@ EXPLAIN FORMATTED key = (SELECT max(key) FROM explain_temp3 WHERE val > 0) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output == Physical Plan == * Filter (3) +- * ColumnarToRow (2) @@ -568,13 +568,13 @@ Input: [max#x] Input: [max#x] --- !query 11 +-- !query EXPLAIN FORMATTED SELECT (SELECT Avg(key) FROM explain_temp1) + (SELECT Avg(key) FROM explain_temp1) FROM explain_temp1 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output == Physical Plan == * Project (3) +- * ColumnarToRow (2) @@ -625,7 +625,7 @@ Input: [sum#x, count#xL] Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#x, [id=#x] --- !query 12 +-- !query EXPLAIN FORMATTED WITH cte1 AS ( SELECT * @@ -633,9 +633,9 @@ EXPLAIN FORMATTED WHERE key > 10 ) SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output == Physical Plan == * BroadcastHashJoin Inner BuildRight (10) :- * Project (4) @@ -694,7 +694,7 @@ Right keys: List(key#x) Join condition: None --- !query 13 +-- !query EXPLAIN FORMATTED WITH cte1 AS ( SELECT key, max(val) @@ -703,9 +703,9 @@ EXPLAIN FORMATTED GROUP BY key ) SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output == Physical Plan == * BroadcastHashJoin Inner BuildRight (11) :- * HashAggregate (7) @@ -762,13 +762,13 @@ Right keys: List(key#x) Join condition: None --- !query 14 +-- !query EXPLAIN FORMATTED CREATE VIEW explain_view AS SELECT key, val FROM explain_temp1 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output == Physical Plan == Execute CreateViewCommand (1) +- CreateViewCommand (2) @@ -786,25 +786,25 @@ Output: [] (4) Project --- !query 15 +-- !query DROP TABLE explain_temp1 --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query DROP TABLE explain_temp2 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output --- !query 17 +-- !query DROP TABLE explain_temp3 --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/extract.sql.out b/sql/core/src/test/resources/sql-tests/results/extract.sql.out index 1424cc45fbb42..583459f9037b8 100644 --- a/sql/core/src/test/resources/sql-tests/results/extract.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/extract.sql.out @@ -2,515 +2,515 @@ -- Number of queries: 64 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select extract(millennium from c) from t --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 3 --- !query 2 +-- !query select extract(millennia from c) from t --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 3 --- !query 3 +-- !query select extract(mil from c) from t --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 3 --- !query 4 +-- !query select extract(mils from c) from t --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 3 --- !query 5 +-- !query select extract(century from c) from t --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 21 --- !query 6 +-- !query select extract(centuries from c) from t --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 21 --- !query 7 +-- !query select extract(c from c) from t --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 21 --- !query 8 +-- !query select extract(cent from c) from t --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 21 --- !query 9 +-- !query select extract(decade from c) from t --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 201 --- !query 10 +-- !query select extract(decades from c) from t --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 201 --- !query 11 +-- !query select extract(dec from c) from t --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 201 --- !query 12 +-- !query select extract(decs from c) from t --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 201 --- !query 13 +-- !query select extract(year from c) from t --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 2011 --- !query 14 +-- !query select extract(y from c) from t --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 2011 --- !query 15 +-- !query select extract(years from c) from t --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 2011 --- !query 16 +-- !query select extract(yr from c) from t --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2011 --- !query 17 +-- !query select extract(yrs from c) from t --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 2011 --- !query 18 +-- !query select extract(isoyear from c) from t --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 2011 --- !query 19 +-- !query select extract(quarter from c) from t --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 2 --- !query 20 +-- !query select extract(qtr from c) from t --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 2 --- !query 21 +-- !query select extract(month from c) from t --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 5 --- !query 22 +-- !query select extract(mon from c) from t --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 5 --- !query 23 +-- !query select extract(mons from c) from t --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 5 --- !query 24 +-- !query select extract(months from c) from t --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 5 --- !query 25 +-- !query select extract(week from c) from t --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 18 --- !query 26 +-- !query select extract(w from c) from t --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 18 --- !query 27 +-- !query select extract(weeks from c) from t --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 18 --- !query 28 +-- !query select extract(day from c) from t --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 6 --- !query 29 +-- !query select extract(d from c) from t --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 6 --- !query 30 +-- !query select extract(days from c) from t --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 6 --- !query 31 +-- !query select extract(dayofweek from c) from t --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 6 --- !query 32 +-- !query select extract(dow from c) from t --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 5 --- !query 33 +-- !query select extract(isodow from c) from t --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 5 --- !query 34 +-- !query select extract(doy from c) from t --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 126 --- !query 35 +-- !query select extract(hour from c) from t --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 7 --- !query 36 +-- !query select extract(h from c) from t --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 7 --- !query 37 +-- !query select extract(hours from c) from t --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 7 --- !query 38 +-- !query select extract(hr from c) from t --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 7 --- !query 39 +-- !query select extract(hrs from c) from t --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 7 --- !query 40 +-- !query select extract(minute from c) from t --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 8 --- !query 41 +-- !query select extract(m from c) from t --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 8 --- !query 42 +-- !query select extract(min from c) from t --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 8 --- !query 43 +-- !query select extract(mins from c) from t --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 8 --- !query 44 +-- !query select extract(minutes from c) from t --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 8 --- !query 45 +-- !query select extract(second from c) from t --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 9.123456 --- !query 46 +-- !query select extract(s from c) from t --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output 9.123456 --- !query 47 +-- !query select extract(sec from c) from t --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 9.123456 --- !query 48 +-- !query select extract(seconds from c) from t --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output 9.123456 --- !query 49 +-- !query select extract(secs from c) from t --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output 9.123456 --- !query 50 +-- !query select extract(milliseconds from c) from t --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output 9123.456 --- !query 51 +-- !query select extract(msec from c) from t --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output 9123.456 --- !query 52 +-- !query select extract(msecs from c) from t --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output 9123.456 --- !query 53 +-- !query select extract(millisecon from c) from t --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 9123.456 --- !query 54 +-- !query select extract(mseconds from c) from t --- !query 54 schema +-- !query schema struct --- !query 54 output +-- !query output 9123.456 --- !query 55 +-- !query select extract(ms from c) from t --- !query 55 schema +-- !query schema struct --- !query 55 output +-- !query output 9123.456 --- !query 56 +-- !query select extract(microseconds from c) from t --- !query 56 schema +-- !query schema struct --- !query 56 output +-- !query output 9123456 --- !query 57 +-- !query select extract(usec from c) from t --- !query 57 schema +-- !query schema struct --- !query 57 output +-- !query output 9123456 --- !query 58 +-- !query select extract(usecs from c) from t --- !query 58 schema +-- !query schema struct --- !query 58 output +-- !query output 9123456 --- !query 59 +-- !query select extract(useconds from c) from t --- !query 59 schema +-- !query schema struct --- !query 59 output +-- !query output 9123456 --- !query 60 +-- !query select extract(microsecon from c) from t --- !query 60 schema +-- !query schema struct --- !query 60 output +-- !query output 9123456 --- !query 61 +-- !query select extract(us from c) from t --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output 9123456 --- !query 62 +-- !query select extract(epoch from c) from t --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output 1304665689.123456 --- !query 63 +-- !query select extract(not_supported from c) from t --- !query 63 schema +-- !query schema struct<> --- !query 63 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'not_supported' are currently not supported.(line 1, pos 7) diff --git a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out index 3439a05727f95..4584b823a6e70 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out @@ -2,21 +2,21 @@ -- Number of queries: 29 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) AS testData(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE --- !query 1 schema +-- !query schema struct<(a + b):int,b:int,sum((a - b)):bigint> --- !query 1 output +-- !query output 2 1 0 2 NULL 0 3 1 1 @@ -32,11 +32,11 @@ NULL 2 0 NULL NULL 3 --- !query 2 +-- !query SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 1 1 1 2 2 1 NULL 3 @@ -51,11 +51,11 @@ NULL 2 6 NULL NULL 9 --- !query 3 +-- !query SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP --- !query 3 schema +-- !query schema struct<(a + b):int,b:int,sum((a - b)):bigint> --- !query 3 output +-- !query output 2 1 0 2 NULL 0 3 1 1 @@ -69,11 +69,11 @@ struct<(a + b):int,b:int,sum((a - b)):bigint> NULL NULL 3 --- !query 4 +-- !query SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 1 1 1 2 2 1 NULL 3 @@ -86,21 +86,21 @@ struct NULL NULL 9 --- !query 5 +-- !query CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES ("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) AS courseSales(course, year, earnings) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL NULL 113000 Java NULL 50000 Java 2012 20000 @@ -110,11 +110,11 @@ dotNET 2012 15000 dotNET 2013 48000 --- !query 7 +-- !query SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL NULL 113000 NULL 2012 35000 NULL 2013 78000 @@ -126,41 +126,41 @@ dotNET 2012 15000 dotNET 2013 48000 --- !query 8 +-- !query SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output Java NULL 50000 NULL 2012 35000 NULL 2013 78000 dotNET NULL 63000 --- !query 9 +-- !query SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output Java NULL 50000 dotNET NULL 63000 --- !query 10 +-- !query SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output NULL 2012 35000 NULL 2013 78000 --- !query 11 +-- !query SELECT course, SUM(earnings) AS sum FROM courseSales GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output NULL 113000 Java 20000 Java 30000 @@ -171,12 +171,12 @@ dotNET 48000 dotNET 63000 --- !query 12 +-- !query SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output NULL 113000 3 Java 20000 0 Java 30000 0 @@ -187,12 +187,12 @@ dotNET 48000 0 dotNET 63000 1 --- !query 13 +-- !query SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output Java 2012 0 0 0 Java 2013 0 0 0 Java NULL 0 1 1 @@ -204,29 +204,29 @@ dotNET 2013 0 0 0 dotNET NULL 0 1 1 --- !query 14 +-- !query SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException grouping() can only be used with GroupingSets/Cube/Rollup; --- !query 15 +-- !query SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 16 +-- !query SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output Java 2012 0 Java 2013 0 dotNET 2012 0 @@ -238,40 +238,40 @@ NULL 2013 2 NULL NULL 3 --- !query 17 +-- !query SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, year --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output NULL NULL Java NULL dotNET NULL --- !query 18 +-- !query SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0 --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.AnalysisException grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 19 +-- !query SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 20 +-- !query SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output Java NULL NULL 2012 NULL 2013 @@ -279,12 +279,12 @@ NULL NULL dotNET NULL --- !query 21 +-- !query SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) ORDER BY GROUPING(course), GROUPING(year), course, year --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output Java 2012 0 0 Java 2013 0 0 dotNET 2012 0 0 @@ -296,12 +296,12 @@ NULL 2013 1 0 NULL NULL 1 1 --- !query 22 +-- !query SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) ORDER BY GROUPING(course), GROUPING(year), course, year --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output Java 2012 0 Java 2013 0 dotNET 2012 0 @@ -313,29 +313,29 @@ NULL 2013 2 NULL NULL 3 --- !query 23 +-- !query SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(course) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 24 +-- !query SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID(course) --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 25 +-- !query SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output Java 2012 Java 2013 dotNET 2012 @@ -347,11 +347,11 @@ NULL 2013 NULL NULL --- !query 26 +-- !query SELECT a + b AS k1, b AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 2 1 0 2 NULL 0 3 1 1 @@ -367,11 +367,11 @@ NULL 2 0 NULL NULL 3 --- !query 27 +-- !query SELECT a + b AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 2 1 0 2 NULL 0 3 1 1 @@ -385,10 +385,10 @@ struct NULL NULL 3 --- !query 28 +-- !query SELECT a + b, b AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) --- !query 28 schema +-- !query schema struct<(a + b):int,k:int,sum((a - b)):bigint> --- !query 28 output +-- !query output NULL 1 3 NULL 2 0 diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out index fbb66878f891f..a032678e90fe8 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -2,17 +2,17 @@ -- Number of queries: 37 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) AS testData(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), @@ -24,13 +24,13 @@ CREATE OR REPLACE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (700, "emp 7", date "2010-01-01", 400.00D, 100), (800, "emp 8", date "2016-01-01", 150.00D, 70) AS EMP(id, emp_name, hiredate, salary, dept_id) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE OR REPLACE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (10, "dept 1", "CA"), (20, "dept 2", "NY"), @@ -39,97 +39,97 @@ CREATE OR REPLACE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (50, "dept 5 - unassigned", "NJ"), (70, "dept 7", "FL") AS DEPT(dept_id, dept_name, state) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) FILTER (WHERE (testdata.`a` >= 2)) AS `count(b) FILTER (WHERE (a >= 2))`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; --- !query 4 +-- !query SELECT COUNT(a) FILTER (WHERE a = 1), COUNT(b) FILTER (WHERE a > 1) FROM testData --- !query 4 schema +-- !query schema struct 1)):bigint> --- !query 4 output +-- !query output 2 4 --- !query 5 +-- !query SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 2 --- !query 6 +-- !query SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 2 --- !query 7 +-- !query SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 2 --- !query 8 +-- !query SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 2 --- !query 9 +-- !query SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a --- !query 9 schema +-- !query schema struct= 2)):bigint> --- !query 9 output +-- !query output 1 0 2 2 3 2 NULL 0 --- !query 10 +-- !query SELECT a, COUNT(b) FILTER (WHERE a != 2) FROM testData GROUP BY b --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 11 +-- !query SELECT COUNT(a) FILTER (WHERE a >= 0), COUNT(b) FILTER (WHERE a >= 3) FROM testData GROUP BY a --- !query 11 schema +-- !query schema struct= 0)):bigint,count(b) FILTER (WHERE (a >= 3)):bigint> --- !query 11 output +-- !query output 0 0 2 0 2 0 3 2 --- !query 12 +-- !query SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp GROUP BY dept_id --- !query 12 schema +-- !query schema struct DATE '2003-01-01')):double> --- !query 12 output +-- !query output 10 200.0 100 400.0 20 NULL @@ -138,11 +138,11 @@ struct DATE '2003-01-01')):dou NULL NULL --- !query 13 +-- !query SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_date("2003-01-01")) FROM emp GROUP BY dept_id --- !query 13 schema +-- !query schema struct to_date('2003-01-01'))):double> --- !query 13 output +-- !query output 10 200.0 100 400.0 20 NULL @@ -151,11 +151,11 @@ struct to_date('2003-01-01'))) NULL NULL --- !query 14 +-- !query SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_timestamp("2003-01-01 00:00:00")) FROM emp GROUP BY dept_id --- !query 14 schema +-- !query schema struct to_timestamp('2003-01-01 00:00:00'))):double> --- !query 14 output +-- !query output 10 200.0 100 400.0 20 NULL @@ -164,11 +164,11 @@ struct to_t NULL NULL --- !query 15 +-- !query SELECT dept_id, SUM(salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2003-01-01") FROM emp GROUP BY dept_id --- !query 15 schema +-- !query schema struct 2003-01-01)):double> --- !query 15 output +-- !query output 10 200.0 100 400.0 20 NULL @@ -177,43 +177,43 @@ struct --- !query 16 output +-- !query output foo 6 --- !query 17 +-- !query SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= date "2003-01-01") FROM emp GROUP BY 1 --- !query 17 schema +-- !query schema struct= DATE '2003-01-01')):double> --- !query 17 output +-- !query output foo 1350.0 --- !query 18 +-- !query SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_date("2003-01-01")) FROM emp GROUP BY 1 --- !query 18 schema +-- !query schema struct= to_date('2003-01-01'))):double> --- !query 18 output +-- !query output foo 1350.0 --- !query 19 +-- !query SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_timestamp("2003-01-01")) FROM emp GROUP BY 1 --- !query 19 schema +-- !query schema struct= to_timestamp('2003-01-01'))):double> --- !query 19 output +-- !query output foo 1350.0 --- !query 20 +-- !query select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id --- !query 20 schema +-- !query schema struct 200)):double> --- !query 20 output +-- !query output 10 2 2 400.0 NULL 100 2 2 800.0 800.0 20 1 1 300.0 300.0 @@ -222,11 +222,11 @@ struct 500) from emp group by dept_id --- !query 21 schema +-- !query schema struct 500)):double> --- !query 21 output +-- !query output 10 2 2 400.0 NULL 100 2 2 800.0 800.0 20 1 1 300.0 NULL @@ -235,11 +235,11 @@ struct 200) from emp group by dept_id --- !query 22 schema +-- !query schema struct 200)):double> --- !query 22 output +-- !query output 10 2 2 400.0 NULL 100 2 2 NULL 800.0 20 1 1 300.0 300.0 @@ -248,11 +248,11 @@ struct 500) from emp group by dept_id --- !query 23 schema +-- !query schema struct 500)):double> --- !query 23 output +-- !query output 10 2 2 400.0 NULL 100 2 2 NULL 800.0 20 1 1 300.0 NULL @@ -261,27 +261,27 @@ struct= 0) FROM testData WHERE a = 0 GROUP BY 1 --- !query 24 schema +-- !query schema struct= 0)):bigint> --- !query 24 output +-- !query output --- !query 25 +-- !query SELECT 'foo', MAX(STRUCT(a)) FILTER (WHERE b >= 1) FROM testData WHERE a = 0 GROUP BY 1 --- !query 25 schema +-- !query schema struct= 1)):struct> --- !query 25 output +-- !query output --- !query 26 +-- !query SELECT a + b, COUNT(b) FILTER (WHERE b >= 2) FROM testData GROUP BY a + b --- !query 26 schema +-- !query schema struct<(a + b):int,count(b) FILTER (WHERE (b >= 2)):bigint> --- !query 26 output +-- !query output 2 0 3 1 4 1 @@ -289,46 +289,46 @@ struct<(a + b):int,count(b) FILTER (WHERE (b >= 2)):bigint> NULL 0 --- !query 27 +-- !query SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1 --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output org.apache.spark.sql.AnalysisException expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 28 +-- !query SELECT a + 1 + 1, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY a + 1 --- !query 28 schema +-- !query schema struct<((a + 1) + 1):int,count(b) FILTER (WHERE (b > 0)):bigint> --- !query 28 output +-- !query output 3 2 4 2 5 2 NULL 1 --- !query 29 +-- !query SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k --- !query 29 schema +-- !query schema struct 0)):bigint> --- !query 29 output +-- !query output 1 2 2 2 3 2 NULL 1 --- !query 30 +-- !query SELECT emp.dept_id, avg(salary), avg(salary) FILTER (WHERE id > (SELECT 200)) FROM emp GROUP BY dept_id --- !query 30 schema +-- !query schema struct scalarsubquery())):double> --- !query 30 output +-- !query output 10 133.33333333333334 NULL 100 400.0 400.0 20 300.0 300.0 @@ -337,15 +337,15 @@ struct scalarsubq NULL 400.0 400.0 --- !query 31 +-- !query SELECT emp.dept_id, avg(salary), avg(salary) FILTER (WHERE emp.dept_id = (SELECT dept_id FROM dept LIMIT 1)) FROM emp GROUP BY dept_id --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 10 133.33333333333334 133.33333333333334 100 400.0 NULL 20 300.0 NULL @@ -354,7 +354,7 @@ struct --- !query 32 output +-- !query output org.apache.spark.sql.AnalysisException IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE exists#x [dept_id#x]) AS avg(salary) FILTER (WHERE exists(dept_id))#x] : +- Project [state#x] @@ -380,7 +380,7 @@ IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few comman ; --- !query 33 +-- !query SELECT emp.dept_id, Sum(salary), Sum(salary) FILTER (WHERE NOT EXISTS (SELECT state @@ -388,9 +388,9 @@ SELECT emp.dept_id, WHERE dept.dept_id = emp.dept_id)) FROM emp GROUP BY dept_id --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT exists#x [dept_id#x]) AS sum(salary) FILTER (WHERE (NOT exists(dept_id)))#x] : +- Project [state#x] @@ -406,16 +406,16 @@ IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few comman ; --- !query 34 +-- !query SELECT emp.dept_id, avg(salary), avg(salary) FILTER (WHERE emp.dept_id IN (SELECT DISTINCT dept_id FROM dept)) FROM emp GROUP BY dept_id --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE dept_id#x IN (list#x [])) AS avg(salary) FILTER (WHERE (dept_id IN (listquery())))#x] : +- Distinct @@ -431,16 +431,16 @@ IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few comman ; --- !query 35 +-- !query SELECT emp.dept_id, Sum(salary), Sum(salary) FILTER (WHERE emp.dept_id NOT IN (SELECT DISTINCT dept_id FROM dept)) FROM emp GROUP BY dept_id --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT dept_id#x IN (list#x [])) AS sum(salary) FILTER (WHERE (NOT (dept_id IN (listquery()))))#x] : +- Distinct @@ -456,9 +456,9 @@ IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few comman ; --- !query 36 +-- !query SELECT t1.b FROM (SELECT COUNT(b) FILTER (WHERE a >= 2) AS b FROM testData) t1 --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 4 diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out index 09e2c632f6386..bf9f606a2224e 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 20 --- !query 0 +-- !query create temporary view data as select * from values (1, 1), (1, 2), @@ -11,55 +11,55 @@ create temporary view data as select * from values (3, 1), (3, 2) as data(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select a, sum(b) from data group by 1 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 3 2 3 3 3 --- !query 2 +-- !query select 1, 2, sum(b) from data group by 1, 2 --- !query 2 schema +-- !query schema struct<1:int,2:int,sum(b):bigint> --- !query 2 output +-- !query output 1 2 9 --- !query 3 +-- !query select a, 1, sum(b) from data group by a, 1 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 1 3 2 1 3 3 1 3 --- !query 4 +-- !query select a, 1, sum(b) from data group by 1, 2 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 1 3 2 1 3 3 1 3 --- !query 5 +-- !query select a, b + 2, count(2) from data group by a, 2 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 3 1 1 4 1 2 3 1 @@ -68,11 +68,11 @@ struct 3 4 1 --- !query 6 +-- !query select a as aa, b + 2 as bb, count(2) from data group by 1, 2 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 3 1 1 4 1 2 3 1 @@ -81,66 +81,66 @@ struct 3 4 1 --- !query 7 +-- !query select sum(b) from data group by 1 + 0 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 9 --- !query 8 +-- !query select a, b from data group by -1 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException GROUP BY position -1 is not in select list (valid range is [1, 2]); line 1 pos 31 --- !query 9 +-- !query select a, b from data group by 0 --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException GROUP BY position 0 is not in select list (valid range is [1, 2]); line 1 pos 31 --- !query 10 +-- !query select a, b from data group by 3 --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 31 --- !query 11 +-- !query select a, b, sum(b) from data group by 3 --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException aggregate functions are not allowed in GROUP BY, but found sum(CAST(data.`b` AS BIGINT)); --- !query 12 +-- !query select a, b, sum(b) + 2 from data group by 3 --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException aggregate functions are not allowed in GROUP BY, but found (sum(CAST(data.`b` AS BIGINT)) + CAST(2 AS BIGINT)); --- !query 13 +-- !query select a, rand(0), sum(b) from (select /*+ REPARTITION(1) */ a, b from data) group by a, 2 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 0.5234194256885571 2 1 0.7604953758285915 1 2 0.0953472826424725 1 @@ -149,52 +149,52 @@ struct 3 0.7141011170991605 1 --- !query 14 +-- !query select * from data group by a, b, 1 --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException Star (*) is not allowed in select list when GROUP BY ordinal position is used; --- !query 15 +-- !query select a, count(a) from (select 1 as a) tmp group by 1 order by 1 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 1 --- !query 16 +-- !query select count(a), a from (select 1 as a) tmp group by 2 having a > 0 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1 1 --- !query 17 +-- !query select a, a AS k, count(b) from data group by k, 1 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 1 2 2 2 2 3 3 2 --- !query 18 +-- !query set spark.sql.groupByOrdinal=false --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output spark.sql.groupByOrdinal false --- !query 19 +-- !query select sum(b) from data group by -1 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 9 diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 62a1666497084..7bfdd0ad53a95 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -2,101 +2,101 @@ -- Number of queries: 56 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) AS testData(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT a, COUNT(b) FROM testData --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) AS `count(b)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; --- !query 2 +-- !query SELECT COUNT(a), COUNT(b) FROM testData --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 7 7 --- !query 3 +-- !query SELECT a, COUNT(b) FROM testData GROUP BY a --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 2 2 3 2 NULL 1 --- !query 4 +-- !query SELECT a, COUNT(b) FROM testData GROUP BY b --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 5 +-- !query SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 0 1 2 2 2 2 3 2 --- !query 6 +-- !query SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output foo 7 --- !query 7 +-- !query SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output --- !query 9 +-- !query SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 --- !query 9 schema +-- !query schema struct> --- !query 9 output +-- !query output --- !query 10 +-- !query SELECT a + b, COUNT(b) FROM testData GROUP BY a + b --- !query 10 schema +-- !query schema struct<(a + b):int,count(b):bigint> --- !query 10 output +-- !query output 2 1 3 2 4 2 @@ -104,132 +104,132 @@ struct<(a + b):int,count(b):bigint> NULL 1 --- !query 11 +-- !query SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 12 +-- !query SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 --- !query 12 schema +-- !query schema struct<((a + 1) + 1):int,count(b):bigint> --- !query 12 output +-- !query output 3 2 4 2 5 2 NULL 1 --- !query 13 +-- !query SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM testData --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 --- !query 14 +-- !query SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 1 --- !query 15 +-- !query SELECT a AS k, COUNT(b) FROM testData GROUP BY k --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 2 2 2 3 2 NULL 1 --- !query 16 +-- !query SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2 2 3 2 --- !query 17 +-- !query SELECT COUNT(b) AS k FROM testData GROUP BY k --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException aggregate functions are not allowed in GROUP BY, but found count(testdata.`b`); --- !query 18 +-- !query CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES (1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException expression 'testdatahassamenamewithalias.`k`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 20 +-- !query set spark.sql.groupByAliases=false --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output spark.sql.groupByAliases false --- !query 21 +-- !query SELECT a AS k, COUNT(b) FROM testData GROUP BY k --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47 --- !query 22 +-- !query SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output --- !query 23 +-- !query SELECT COUNT(1) FROM testData WHERE false --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 0 --- !query 24 +-- !query SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t --- !query 24 schema +-- !query schema struct<1:int> --- !query 24 output +-- !query output 1 --- !query 25 +-- !query SELECT 1 from ( SELECT 1 AS z, MIN(a.x) @@ -237,88 +237,88 @@ SELECT 1 from ( WHERE false ) b where b.z != b.z --- !query 25 schema +-- !query schema struct<1:int> --- !query 25 output +-- !query output --- !query 26 +-- !query SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1.0 1.0 3 --- !query 27 +-- !query SELECT 1 FROM range(10) HAVING true --- !query 27 schema +-- !query schema struct<1:int> --- !query 27 output +-- !query output 1 --- !query 28 +-- !query SELECT 1 FROM range(10) HAVING MAX(id) > 0 --- !query 28 schema +-- !query schema struct<1:int> --- !query 28 output +-- !query output 1 --- !query 29 +-- !query SELECT id FROM range(10) HAVING id > 0 --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get.; --- !query 30 +-- !query CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (1, true), (1, false), (2, true), (3, false), (3, null), (4, null), (4, null), (5, null), (5, true), (5, false) AS test_agg(k, v) --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output --- !query 31 +-- !query SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output NULL NULL NULL NULL NULL --- !query 32 +-- !query SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output NULL NULL NULL NULL NULL --- !query 33 +-- !query SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output false true true false true --- !query 34 +-- !query SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 1 false true true false true 2 true true true true true 3 false false false false false @@ -326,25 +326,25 @@ struct --- !query 35 output +-- !query output 1 false 3 false 5 false --- !query 36 +-- !query SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 4 NULL --- !query 37 +-- !query SELECT k, Every(v) AS every FROM test_agg @@ -353,13 +353,13 @@ WHERE k = 2 FROM test_agg WHERE k = 1) GROUP BY k --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 2 true --- !query 38 +-- !query SELECT k, Every(v) AS every FROM test_agg @@ -368,71 +368,71 @@ WHERE k = 2 FROM test_agg WHERE k = 1) GROUP BY k --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output --- !query 39 +-- !query SELECT every(1) --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'every(1)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 --- !query 40 +-- !query SELECT some(1S) --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'some(1S)' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 --- !query 41 +-- !query SELECT any(1L) --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'any(1L)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 --- !query 42 +-- !query SELECT every("true") --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7 --- !query 43 +-- !query SELECT bool_and(1.0) --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'bool_and(1.0BD)' due to data type mismatch: Input to function 'bool_and' should have been boolean, but it's [decimal(2,1)].; line 1 pos 7 --- !query 44 +-- !query SELECT bool_or(1.0D) --- !query 44 schema +-- !query schema struct<> --- !query 44 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'bool_or(1.0D)' due to data type mismatch: Input to function 'bool_or' should have been boolean, but it's [double].; line 1 pos 7 --- !query 45 +-- !query SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 1 false false 1 true false 2 true true @@ -445,11 +445,11 @@ struct --- !query 46 output +-- !query output 1 false false 1 true true 2 true true @@ -462,11 +462,11 @@ struct --- !query 47 output +-- !query output 1 false false 1 true true 2 true true @@ -479,11 +479,11 @@ struct --- !query 48 output +-- !query output 1 false false 1 true false 2 true true @@ -496,11 +496,11 @@ struct --- !query 49 output +-- !query output 1 false false 1 true true 2 true true @@ -513,37 +513,37 @@ struct 1L --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output 10 --- !query 51 +-- !query SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output 1 true 2 true 5 true --- !query 52 +-- !query SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output 10 --- !query 53 +-- !query SELECT count(*) FROM test_agg WHERE count(*) > 1L --- !query 53 schema +-- !query schema struct<> --- !query 53 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -551,11 +551,11 @@ Expression in where clause: [(count(1) > 1L)] Invalid expressions: [count(1)]; --- !query 54 +-- !query SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -563,11 +563,11 @@ Expression in where clause: [((count(1) + 1L) > 1L)] Invalid expressions: [count(1)]; --- !query 55 +-- !query SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. diff --git a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out index 9b81a55c7de6f..8eeabb34b4fab 100644 --- a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out @@ -2,114 +2,114 @@ -- Number of queries: 18 --- !query 0 +-- !query CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES ("1", "2", "3", 1), ("4", "5", "6", 1), ("7", "8", "9", 1) as grouping(a, b, c, d) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS (()) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output NULL NULL NULL 3 --- !query 2 +-- !query SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a)) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 NULL NULL 1 4 NULL NULL 1 7 NULL NULL 1 --- !query 3 +-- !query SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c)) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output NULL NULL 3 1 NULL NULL 6 1 NULL NULL 9 1 --- !query 4 +-- !query SELECT c1, sum(c2) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output x 10 y 20 --- !query 5 +-- !query SELECT c1, sum(c2), grouping(c1) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output x 10 0 y 20 0 --- !query 6 +-- !query SELECT c1, c2, Sum(c3), grouping__id FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3) GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) ) HAVING GROUPING__ID > 1 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL a 10 2 NULL b 20 2 --- !query 7 +-- !query SELECT grouping(c1) FROM (VALUES ('x', 'a', 10), ('y', 'b', 20)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1,c2) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 0 0 1 1 --- !query 8 +-- !query SELECT -c1 AS c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS ((c1), (c1, c2)) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output -1 -1 -3 -3 --- !query 9 +-- !query SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b)) --- !query 9 schema +-- !query schema struct<(a + b):int,b:int,sum(c):bigint> --- !query 9 output +-- !query output 2 NULL 1 4 NULL 2 NULL 1 1 NULL 2 2 --- !query 10 +-- !query SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b + a), (b)) --- !query 10 schema +-- !query schema struct<(a + b):int,b:int,sum(c):bigint> --- !query 10 output +-- !query output 2 NULL 1 2 NULL 1 4 NULL 2 @@ -118,24 +118,24 @@ NULL 1 1 NULL 2 2 --- !query 11 +-- !query SELECT c1 AS col1, c2 AS col2 FROM (VALUES (1, 2), (3, 2)) t(c1, c2) GROUP BY GROUPING SETS ( ( c1 ), ( c1, c2 ) ) HAVING col2 IS NOT NULL ORDER BY -col1 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 3 2 1 2 --- !query 12 +-- !query SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException extraneous input 'ROLLUP' expecting (line 1, pos 53) @@ -145,11 +145,11 @@ SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP -----------------------------------------------------^^^ --- !query 13 +-- !query SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException extraneous input 'CUBE' expecting (line 1, pos 53) @@ -159,20 +159,20 @@ SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE -----------------------------------------------------^^^ --- !query 14 +-- !query SELECT c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS (()) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException expression '`c1`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 15 +-- !query SELECT k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 1 1.0 1 1 1.0 1 NULL 1.0 @@ -181,11 +181,11 @@ struct 2 NULL 2.0 --- !query 16 +-- !query SELECT grouping__id, k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 0 1 1 1.0 0 1 1 1.0 0 2 2 2.0 @@ -194,11 +194,11 @@ struct 1 2 NULL 2.0 --- !query 17 +-- !query SELECT grouping(k1), k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 0 1 1 1.0 0 1 1 1.0 0 1 NULL 1.0 diff --git a/sql/core/src/test/resources/sql-tests/results/having.sql.out b/sql/core/src/test/resources/sql-tests/results/having.sql.out index d87ee5221647f..5bd185d7b815d 100644 --- a/sql/core/src/test/resources/sql-tests/results/having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/having.sql.out @@ -2,48 +2,48 @@ -- Number of queries: 5 --- !query 0 +-- !query create temporary view hav as select * from values ("one", 1), ("two", 2), ("three", 3), ("one", 5) as hav(k, v) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output one 6 three 3 --- !query 2 +-- !query SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 --- !query 3 +-- !query SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 --- !query 4 +-- !query SELECT a + b FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > 1 --- !query 4 schema +-- !query schema struct<(a + CAST(b AS BIGINT)):bigint> --- !query 4 output +-- !query output 3 7 diff --git a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out index ca1d747874c5d..d35d0d5d944bb 100644 --- a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out @@ -2,271 +2,271 @@ -- Number of queries: 29 --- !query 0 +-- !query create or replace temporary view nested as values (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), (3, array(12), array(array(17))) as t(x, ys, zs) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select upper(x -> x) as v --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output org.apache.spark.sql.AnalysisException A lambda function should only be used in a higher order function. However, its class is org.apache.spark.sql.catalyst.expressions.Upper, which is not a higher order function.; line 1 pos 7 --- !query 2 +-- !query select transform(zs, z -> z) as v from nested --- !query 2 schema +-- !query schema struct>> --- !query 2 output +-- !query output [[12,99],[123,42],[1]] [[17]] [[6,96,65],[-1,-2]] --- !query 3 +-- !query select transform(ys, y -> y * y) as v from nested --- !query 3 schema +-- !query schema struct> --- !query 3 output +-- !query output [1024,9409] [144] [5929,5776] --- !query 4 +-- !query select transform(ys, (y, i) -> y + i) as v from nested --- !query 4 schema +-- !query schema struct> --- !query 4 output +-- !query output [12] [32,98] [77,-75] --- !query 5 +-- !query select transform(zs, z -> concat(ys, z)) as v from nested --- !query 5 schema +-- !query schema struct>> --- !query 5 output +-- !query output [[12,17]] [[32,97,12,99],[32,97,123,42],[32,97,1]] [[77,-76,6,96,65],[77,-76,-1,-2]] --- !query 6 +-- !query select transform(ys, 0) as v from nested --- !query 6 schema +-- !query schema struct> --- !query 6 output +-- !query output [0,0] [0,0] [0] --- !query 7 +-- !query select transform(cast(null as array), x -> x + 1) as v --- !query 7 schema +-- !query schema struct> --- !query 7 output +-- !query output NULL --- !query 8 +-- !query select filter(ys, y -> y > 30) as v from nested --- !query 8 schema +-- !query schema struct> --- !query 8 output +-- !query output [32,97] [77] [] --- !query 9 +-- !query select filter(cast(null as array), y -> true) as v --- !query 9 schema +-- !query schema struct> --- !query 9 output +-- !query output NULL --- !query 10 +-- !query select transform(zs, z -> filter(z, zz -> zz > 50)) as v from nested --- !query 10 schema +-- !query schema struct>> --- !query 10 output +-- !query output [[96,65],[]] [[99],[123],[]] [[]] --- !query 11 +-- !query select aggregate(ys, 0, (y, a) -> y + a + x) as v from nested --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 131 15 5 --- !query 12 +-- !query select aggregate(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 0.5 12.0 64.5 --- !query 13 +-- !query select transform(zs, z -> aggregate(z, 1, (acc, val) -> acc * val * size(z))) as v from nested --- !query 13 schema +-- !query schema struct> --- !query 13 output +-- !query output [1010880,8] [17] [4752,20664,1] --- !query 14 +-- !query select aggregate(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output NULL --- !query 15 +-- !query select exists(ys, y -> y > 30) as v from nested --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output false true true --- !query 16 +-- !query select exists(cast(null as array), y -> y > 30) as v --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output NULL --- !query 17 +-- !query select zip_with(ys, zs, (a, b) -> a + size(b)) as v from nested --- !query 17 schema +-- !query schema struct> --- !query 17 output +-- !query output [13] [34,99,null] [80,-74] --- !query 18 +-- !query select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)) as v --- !query 18 schema +-- !query schema struct> --- !query 18 output +-- !query output ["ad","be","cf"] --- !query 19 +-- !query select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v --- !query 19 schema +-- !query schema struct> --- !query 19 output +-- !query output ["a",null,"f"] --- !query 20 +-- !query create or replace temporary view nested as values (1, map(1, 1, 2, 2, 3, 3)), (2, map(4, 4, 5, 5, 6, 6)) as t(x, ys) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query select transform_keys(ys, (k, v) -> k) as v from nested --- !query 21 schema +-- !query schema struct> --- !query 21 output +-- !query output {1:1,2:2,3:3} {4:4,5:5,6:6} --- !query 22 +-- !query select transform_keys(ys, (k, v) -> k + 1) as v from nested --- !query 22 schema +-- !query schema struct> --- !query 22 output +-- !query output {2:1,3:2,4:3} {5:4,6:5,7:6} --- !query 23 +-- !query select transform_keys(ys, (k, v) -> k + v) as v from nested --- !query 23 schema +-- !query schema struct> --- !query 23 output +-- !query output {10:5,12:6,8:4} {2:1,4:2,6:3} --- !query 24 +-- !query select transform_values(ys, (k, v) -> v) as v from nested --- !query 24 schema +-- !query schema struct> --- !query 24 output +-- !query output {1:1,2:2,3:3} {4:4,5:5,6:6} --- !query 25 +-- !query select transform_values(ys, (k, v) -> v + 1) as v from nested --- !query 25 schema +-- !query schema struct> --- !query 25 output +-- !query output {1:2,2:3,3:4} {4:5,5:6,6:7} --- !query 26 +-- !query select transform_values(ys, (k, v) -> k + v) as v from nested --- !query 26 schema +-- !query schema struct> --- !query 26 output +-- !query output {1:2,2:4,3:6} {4:8,5:10,6:12} --- !query 27 +-- !query select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) --- !query 27 schema +-- !query schema struct> --- !query 27 output +-- !query output [1024,9409] --- !query 28 +-- !query select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) --- !query 28 schema +-- !query schema struct> --- !query 28 output +-- !query output [32,98] diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out index bf5a560ec0722..9943b93c431df 100644 --- a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -2,152 +2,152 @@ -- Number of queries: 17 --- !query 0 +-- !query select * from values ("one", 1) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output one 1 --- !query 1 +-- !query select * from values ("one", 1) as data --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output one 1 --- !query 2 +-- !query select * from values ("one", 1) as data(a, b) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output one 1 --- !query 3 +-- !query select * from values 1, 2, 3 as data(a) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 3 --- !query 4 +-- !query select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output one 1 three NULL two 2 --- !query 5 +-- !query select * from values ("one", null), ("two", null) as data(a, b) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output one NULL two NULL --- !query 6 +-- !query select * from values ("one", 1), ("two", 2L) as data(a, b) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output one 1 two 2 --- !query 7 +-- !query select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output one 1 two 4 --- !query 8 +-- !query select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) --- !query 8 schema +-- !query schema struct> --- !query 8 output +-- !query output one [0,1] two [2,3] --- !query 9 +-- !query select * from values ("one", 2.0), ("two", 3.0D) as data(a, b) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output one 2.0 two 3.0 --- !query 10 +-- !query select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot evaluate expression rand(5) in inline table definition; line 1 pos 29 --- !query 11 +-- !query select * from values ("one", 2.0), ("two") as data(a, b) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException expected 2 columns but found 1 columns in row 1; line 1 pos 14 --- !query 12 +-- !query select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException incompatible types found in column b for inline table; line 1 pos 14 --- !query 13 +-- !query select * from values ("one"), ("two") as data(a, b) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException expected 2 columns but found 1 columns in row 0; line 1 pos 14 --- !query 14 +-- !query select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException Undefined function: 'random_not_exist_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 29 --- !query 15 +-- !query select * from values ("one", count(1)), ("two", 2) as data(a, b) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException cannot evaluate expression count(1) in inline table definition; line 1 pos 29 --- !query 16 +-- !query select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b) --- !query 16 schema +-- !query schema struct> --- !query 16 output +-- !query output 1991-12-06 00:00:00 [1991-12-06 01:00:00,1991-12-06 12:00:00] diff --git a/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out b/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out index 8d56ebe9fd3b4..7c3774c8bd7fb 100644 --- a/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out @@ -2,65 +2,65 @@ -- Number of queries: 7 --- !query 0 +-- !query CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE TEMPORARY VIEW ta AS SELECT a, 'a' AS tag FROM t1 UNION ALL SELECT a, 'b' AS tag FROM t2 --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TEMPORARY VIEW tb AS SELECT a, 'a' AS tag FROM t3 UNION ALL SELECT a, 'b' AS tag FROM t4 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 a 1 a 1 b diff --git a/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out index 63dd56ce468bc..4762082dc3be2 100644 --- a/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 22 --- !query 0 +-- !query CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES (1, 2), (1, 2), @@ -12,13 +12,13 @@ CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES (null, null), (null, null) AS tab1(k, v) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES (1, 2), (1, 2), @@ -27,19 +27,19 @@ CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES (null, null), (null, null) AS tab2(k, v) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM tab1 INTERSECT ALL SELECT * FROM tab2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 2 1 2 2 3 @@ -47,80 +47,80 @@ NULL NULL NULL NULL --- !query 3 +-- !query SELECT * FROM tab1 INTERSECT ALL SELECT * FROM tab1 WHERE k = 1 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 1 2 1 3 1 3 --- !query 4 +-- !query SELECT * FROM tab1 WHERE k > 2 INTERSECT ALL SELECT * FROM tab2 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT * FROM tab1 INTERSECT ALL SELECT * FROM tab2 WHERE k > 3 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT * FROM tab1 INTERSECT ALL SELECT CAST(1 AS BIGINT), CAST(2 AS BIGINT) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 2 --- !query 7 +-- !query SELECT * FROM tab1 INTERSECT ALL SELECT array(1), 2 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.AnalysisException IntersectAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; --- !query 8 +-- !query SELECT k FROM tab1 INTERSECT ALL SELECT k, v FROM tab2 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException IntersectAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; --- !query 9 +-- !query SELECT * FROM tab2 INTERSECT ALL SELECT * FROM tab1 INTERSECT ALL SELECT * FROM tab2 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 2 1 2 2 3 @@ -128,7 +128,7 @@ NULL NULL NULL NULL --- !query 10 +-- !query SELECT * FROM tab1 EXCEPT SELECT * FROM tab2 @@ -136,9 +136,9 @@ UNION ALL SELECT * FROM tab1 INTERSECT ALL SELECT * FROM tab2 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 2 1 2 1 3 @@ -147,7 +147,7 @@ NULL NULL NULL NULL --- !query 11 +-- !query SELECT * FROM tab1 EXCEPT SELECT * FROM tab2 @@ -155,13 +155,13 @@ EXCEPT SELECT * FROM tab1 INTERSECT ALL SELECT * FROM tab2 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 3 --- !query 12 +-- !query ( ( ( @@ -175,13 +175,13 @@ struct INTERSECT ALL SELECT * FROM tab2 ) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output --- !query 13 +-- !query SELECT * FROM (SELECT tab1.k, tab2.v @@ -195,9 +195,9 @@ FROM (SELECT tab1.k, FROM tab1 JOIN tab2 ON tab1.k = tab2.k) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 2 1 2 1 2 @@ -209,7 +209,7 @@ struct 2 3 --- !query 14 +-- !query SELECT * FROM (SELECT tab1.k, tab2.v @@ -223,33 +223,33 @@ FROM (SELECT tab2.v AS k, FROM tab1 JOIN tab2 ON tab1.k = tab2.k) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output --- !query 15 +-- !query SELECT v FROM tab1 GROUP BY v INTERSECT ALL SELECT k FROM tab2 GROUP BY k --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 2 3 NULL --- !query 16 +-- !query SET spark.sql.legacy.setopsPrecedence.enabled= true --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output spark.sql.legacy.setopsPrecedence.enabled true --- !query 17 +-- !query SELECT * FROM tab1 EXCEPT SELECT * FROM tab2 @@ -257,9 +257,9 @@ UNION ALL SELECT * FROM tab1 INTERSECT ALL SELECT * FROM tab2 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 2 1 2 2 3 @@ -267,7 +267,7 @@ NULL NULL NULL NULL --- !query 18 +-- !query SELECT * FROM tab1 EXCEPT SELECT * FROM tab2 @@ -275,33 +275,33 @@ UNION ALL SELECT * FROM tab1 INTERSECT SELECT * FROM tab2 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 2 2 3 NULL NULL --- !query 19 +-- !query SET spark.sql.legacy.setopsPrecedence.enabled = false --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output spark.sql.legacy.setopsPrecedence.enabled false --- !query 20 +-- !query DROP VIEW IF EXISTS tab1 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query DROP VIEW IF EXISTS tab2 --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index d494399cedddc..8f523a35f3c19 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -2,316 +2,316 @@ -- Number of queries: 101 --- !query 0 +-- !query select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 30 hours 33 minutes 36.003006 seconds --- !query 1 +-- !query select interval 4 month 2 weeks 3 microseconds * 1.5 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 6 months 21 days 0.000005 seconds --- !query 2 +-- !query select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 16 hours --- !query 3 +-- !query select interval '2 seconds' / 0 --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output java.lang.ArithmeticException divide by zero --- !query 4 +-- !query select interval '2 seconds' / null --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output NULL --- !query 5 +-- !query select interval '2 seconds' * null --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output NULL --- !query 6 +-- !query select null * interval '2 seconds' --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL --- !query 7 +-- !query select -interval '-1 month 1 day -1 second' --- !query 7 schema +-- !query schema struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> --- !query 7 output +-- !query output 1 months -1 days 1 seconds --- !query 8 +-- !query select -interval -1 month 1 day -1 second --- !query 8 schema +-- !query schema struct<(- INTERVAL '-1 months 1 days -1 seconds'):interval> --- !query 8 output +-- !query output 1 months -1 days 1 seconds --- !query 9 +-- !query select +interval '-1 month 1 day -1 second' --- !query 9 schema +-- !query schema struct<(+ INTERVAL '-1 months 1 days -1 seconds'):interval> --- !query 9 output +-- !query output -1 months 1 days -1 seconds --- !query 10 +-- !query select +interval -1 month 1 day -1 second --- !query 10 schema +-- !query schema struct<(+ INTERVAL '-1 months 1 days -1 seconds'):interval> --- !query 10 output +-- !query output -1 months 1 days -1 seconds --- !query 11 +-- !query select make_interval(1) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 years --- !query 12 +-- !query select make_interval(1, 2) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 years 2 months --- !query 13 +-- !query select make_interval(1, 2, 3) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 years 2 months 21 days --- !query 14 +-- !query select make_interval(1, 2, 3, 4) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 years 2 months 25 days --- !query 15 +-- !query select make_interval(1, 2, 3, 4, 5) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 years 2 months 25 days 5 hours --- !query 16 +-- !query select make_interval(1, 2, 3, 4, 5, 6) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1 years 2 months 25 days 5 hours 6 minutes --- !query 17 +-- !query select make_interval(1, 2, 3, 4, 5, 6, 7.008009) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds --- !query 18 +-- !query select cast('1 second' as interval) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 seconds --- !query 19 +-- !query select cast('+1 second' as interval) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1 seconds --- !query 20 +-- !query select cast('-1 second' as interval) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output -1 seconds --- !query 21 +-- !query select cast('+ 1 second' as interval) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 seconds --- !query 22 +-- !query select cast('- 1 second' as interval) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output -1 seconds --- !query 23 +-- !query select cast('- -1 second' as interval) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output NULL --- !query 24 +-- !query select cast('- +1 second' as interval) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output NULL --- !query 25 +-- !query select interval 13.123456789 seconds, interval -13.123456789 second --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 13.123456 seconds -13.123456 seconds --- !query 26 +-- !query select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds --- !query 27 +-- !query select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 32 years 1 months -100 days 41 hours 24 minutes 59.889987 seconds --- !query 28 +-- !query select interval '0 0:0:0.1' day to second --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 0.1 seconds --- !query 29 +-- !query select interval '10-9' year to month --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 10 years 9 months --- !query 30 +-- !query select interval '20 15' day to hour --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 20 days 15 hours --- !query 31 +-- !query select interval '20 15:40' day to minute --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 20 days 15 hours 40 minutes --- !query 32 +-- !query select interval '20 15:40:32.99899999' day to second --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 20 days 15 hours 40 minutes 32.998999 seconds --- !query 33 +-- !query select interval '15:40' hour to minute --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 15 hours 40 minutes --- !query 34 +-- !query select interval '15:40:32.99899999' hour to second --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 15 hours 40 minutes 32.998999 seconds --- !query 35 +-- !query select interval '40:32.99899999' minute to second --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 40 minutes 32.998999 seconds --- !query 36 +-- !query select interval '40:32' minute to second --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 40 minutes 32 seconds --- !query 37 +-- !query select interval 30 day day --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 30 days --- !query 38 +-- !query select interval '20 15:40:32.99899999' day to hour --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) @@ -321,11 +321,11 @@ select interval '20 15:40:32.99899999' day to hour ----------------^^^ --- !query 39 +-- !query select interval '20 15:40:32.99899999' day to minute --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) @@ -335,11 +335,11 @@ select interval '20 15:40:32.99899999' day to minute ----------------^^^ --- !query 40 +-- !query select interval '15:40:32.99899999' hour to minute --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 15:40:32.99899999(line 1, pos 16) @@ -349,11 +349,11 @@ select interval '15:40:32.99899999' hour to minute ----------------^^^ --- !query 41 +-- !query select interval '15:40.99899999' hour to second --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40.99899999(line 1, pos 16) @@ -363,11 +363,11 @@ select interval '15:40.99899999' hour to second ----------------^^^ --- !query 42 +-- !query select interval '15:40' hour to second --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40(line 1, pos 16) @@ -377,11 +377,11 @@ select interval '15:40' hour to second ----------------^^^ --- !query 43 +-- !query select interval '20 40:32.99899999' minute to second --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 20 40:32.99899999(line 1, pos 16) @@ -391,11 +391,11 @@ select interval '20 40:32.99899999' minute to second ----------------^^^ --- !query 44 +-- !query select interval 10 nanoseconds --- !query 44 schema +-- !query schema struct<> --- !query 44 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'(line 1, pos 16) @@ -405,35 +405,35 @@ select interval 10 nanoseconds ----------------^^^ --- !query 45 +-- !query select map(1, interval 1 day, 2, interval 3 week) --- !query 45 schema +-- !query schema struct> --- !query 45 output +-- !query output {1:1 days,2:21 days} --- !query 46 +-- !query select interval 'interval 3 year 1 hour' --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output 3 years 1 hours --- !query 47 +-- !query select interval '3 year 1 hour' --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 3 years 1 hours --- !query 48 +-- !query select interval --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException at least one time unit should be given for interval literal(line 1, pos 7) @@ -443,11 +443,11 @@ select interval -------^^^ --- !query 49 +-- !query select interval 1 fake_unit --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Error parsing ' 1 fake_unit' to interval, invalid unit 'fake_unit'(line 1, pos 16) @@ -457,11 +457,11 @@ select interval 1 fake_unit ----------------^^^ --- !query 50 +-- !query select interval 1 year to month --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException The value of from-to unit must be a string(line 1, pos 16) @@ -471,11 +471,11 @@ select interval 1 year to month ----------------^^^ --- !query 51 +-- !query select interval '1' year to second --- !query 51 schema +-- !query schema struct<> --- !query 51 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Intervals FROM year TO second are not supported.(line 1, pos 16) @@ -485,11 +485,11 @@ select interval '1' year to second ----------------^^^ --- !query 52 +-- !query select interval '10-9' year to month '2-1' year to month --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -499,11 +499,11 @@ select interval '10-9' year to month '2-1' year to month -------------------------------------^^^ --- !query 53 +-- !query select interval '10-9' year to month '12:11:10' hour to second --- !query 53 schema +-- !query schema struct<> --- !query 53 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -513,11 +513,11 @@ select interval '10-9' year to month '12:11:10' hour to second -------------------------------------^^^ --- !query 54 +-- !query select interval '1 15:11' day to minute '12:11:10' hour to second --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 40) @@ -527,11 +527,11 @@ select interval '1 15:11' day to minute '12:11:10' hour to second ----------------------------------------^^^ --- !query 55 +-- !query select interval 1 year '2-1' year to month --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -541,11 +541,11 @@ select interval 1 year '2-1' year to month -----------------------^^^ --- !query 56 +-- !query select interval 1 year '12:11:10' hour to second --- !query 56 schema +-- !query schema struct<> --- !query 56 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 23) @@ -555,11 +555,11 @@ select interval 1 year '12:11:10' hour to second -----------------------^^^ --- !query 57 +-- !query select interval '10-9' year to month '1' year --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 37) @@ -569,11 +569,11 @@ select interval '10-9' year to month '1' year -------------------------------------^^^ --- !query 58 +-- !query select interval '12:11:10' hour to second '1' year --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Can only have a single from-to unit in the interval literal syntax(line 1, pos 42) @@ -583,29 +583,29 @@ select interval '12:11:10' hour to second '1' year ------------------------------------------^^^ --- !query 59 +-- !query select interval (-30) day --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException Undefined function: 'interval'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 --- !query 60 +-- !query select interval (a + 1) day --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException Undefined function: 'interval'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 --- !query 61 +-- !query select interval 30 day day day --- !query 61 schema +-- !query schema struct<> --- !query 61 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException extraneous input 'day' expecting (line 1, pos 27) @@ -615,192 +615,192 @@ select interval 30 day day day ---------------------------^^^ --- !query 62 +-- !query select sum(cast(null as interval)) --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output NULL --- !query 63 +-- !query select sum(cast(v as interval)) from VALUES ('1 seconds') t(v) where 1=0 --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output NULL --- !query 64 +-- !query select sum(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 3 seconds --- !query 65 +-- !query select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output 1 seconds --- !query 66 +-- !query select sum(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output -3 seconds --- !query 67 +-- !query select sum(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output -7 days 2 seconds --- !query 68 +-- !query select i, sum(cast(v as interval)) from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) group by i --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 1 -2 days 2 2 seconds 3 NULL --- !query 69 +-- !query select sum(cast(v as interval)) as sv from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) having sv is not null --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output -2 days 2 seconds --- !query 70 +-- !query SELECT i, sum(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM VALUES(1, '1 seconds'), (1, '2 seconds'), (2, NULL), (2, NULL) t(i,v) --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output 1 2 seconds 1 3 seconds 2 NULL 2 NULL --- !query 71 +-- !query select avg(cast(v as interval)) from VALUES (null) t(v) --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output NULL --- !query 72 +-- !query select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) where 1=0 --- !query 72 schema +-- !query schema struct --- !query 72 output +-- !query output NULL --- !query 73 +-- !query select avg(cast(v as interval)) from VALUES ('1 seconds'), ('2 seconds'), (null) t(v) --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output 1.5 seconds --- !query 74 +-- !query select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('2 seconds'), (null) t(v) --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output 0.5 seconds --- !query 75 +-- !query select avg(cast(v as interval)) from VALUES ('-1 seconds'), ('-2 seconds'), (null) t(v) --- !query 75 schema +-- !query schema struct --- !query 75 output +-- !query output -1.5 seconds --- !query 76 +-- !query select avg(cast(v as interval)) from VALUES ('-1 weeks'), ('2 seconds'), (null) t(v) --- !query 76 schema +-- !query schema struct --- !query 76 output +-- !query output -3 days -11 hours -59 minutes -59 seconds --- !query 77 +-- !query select i, avg(cast(v as interval)) from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) group by i --- !query 77 schema +-- !query schema struct --- !query 77 output +-- !query output 1 -1 days 2 2 seconds 3 NULL --- !query 78 +-- !query select avg(cast(v as interval)) as sv from VALUES (1, '-1 weeks'), (2, '2 seconds'), (3, null), (1, '5 days') t(i, v) having sv is not null --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output -15 hours -59 minutes -59.333333 seconds --- !query 79 +-- !query SELECT i, avg(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM VALUES (1,'1 seconds'), (1,'2 seconds'), (2,NULL), (2,NULL) t(i,v) --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output 1 1.5 seconds 1 2 seconds 2 NULL 2 NULL --- !query 80 +-- !query create temporary view interval_arithmetic as select CAST(dateval AS date), CAST(tsval AS timestamp) from values ('2012-01-01', '2012-01-01') as interval_arithmetic(dateval, tsval) --- !query 80 schema +-- !query schema struct<> --- !query 80 output +-- !query output --- !query 81 +-- !query select dateval, dateval - interval '2-2' year to month, @@ -810,13 +810,13 @@ select - interval '2-2' year to month + dateval, interval '2-2' year to month + dateval from interval_arithmetic --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output 2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 --- !query 82 +-- !query select tsval, tsval - interval '2-2' year to month, @@ -826,24 +826,24 @@ select - interval '2-2' year to month + tsval, interval '2-2' year to month + tsval from interval_arithmetic --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output 2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 --- !query 83 +-- !query select interval '2-2' year to month + interval '3-3' year to month, interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic --- !query 83 schema +-- !query schema struct<(INTERVAL '2 years 2 months' + INTERVAL '3 years 3 months'):interval,(INTERVAL '2 years 2 months' - INTERVAL '3 years 3 months'):interval> --- !query 83 output +-- !query output 5 years 5 months -1 years -1 months --- !query 84 +-- !query select dateval, dateval - interval '99 11:22:33.123456789' day to second, @@ -853,13 +853,13 @@ select -interval '99 11:22:33.123456789' day to second + dateval, interval '99 11:22:33.123456789' day to second + dateval from interval_arithmetic --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output 2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 --- !query 85 +-- !query select tsval, tsval - interval '99 11:22:33.123456789' day to second, @@ -869,132 +869,132 @@ select -interval '99 11:22:33.123456789' day to second + tsval, interval '99 11:22:33.123456789' day to second + tsval from interval_arithmetic --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output 2012-01-01 00:00:00 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 2012-04-09 11:22:33.123456 2011-09-23 12:37:26.876544 2011-09-23 12:37:26.876544 2012-04-09 11:22:33.123456 --- !query 86 +-- !query select interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic --- !query 86 schema +-- !query schema struct<(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' + INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval,(INTERVAL '99 days 11 hours 22 minutes 33.123456 seconds' - INTERVAL '10 days 9 hours 8 minutes 7.123456 seconds'):interval> --- !query 86 output +-- !query output 109 days 20 hours 30 minutes 40.246912 seconds 89 days 2 hours 14 minutes 26 seconds --- !query 87 +-- !query select interval '\t interval 1 day' --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 1 days --- !query 88 +-- !query select interval 'interval \t 1\tday' --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output 1 days --- !query 89 +-- !query select interval 'interval\t1\tday' --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output 1 days --- !query 90 +-- !query select interval '1\t' day --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 1 days --- !query 91 +-- !query select interval '1 ' day --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output 1 days --- !query 92 +-- !query select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 92 schema +-- !query schema struct<(- a):interval> --- !query 92 output +-- !query output -178956970 years -8 months --- !query 93 +-- !query select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 93 schema +-- !query schema struct<(a - b):interval> --- !query 93 output +-- !query output 1 months --- !query 94 +-- !query select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 94 schema +-- !query schema struct<(b + INTERVAL '1 months'):interval> --- !query 94 output +-- !query output -178956970 years -8 months --- !query 95 +-- !query select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 95 schema +-- !query schema struct<> --- !query 95 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 96 +-- !query select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) --- !query 96 schema +-- !query schema struct<> --- !query 96 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 97 +-- !query SELECT from_csv('1, 1 day', 'a INT, b interval') --- !query 97 schema +-- !query schema struct> --- !query 97 output +-- !query output {"a":1,"b":1 days} --- !query 98 +-- !query SELECT to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)) --- !query 98 schema +-- !query schema struct --- !query 98 output +-- !query output 2 years 8 months,1 hours 10 minutes --- !query 99 +-- !query SELECT from_json('{"a":"1 days"}', 'a interval') --- !query 99 schema +-- !query schema struct> --- !query 99 output +-- !query output {"a":1 days} --- !query 100 +-- !query SELECT to_json(map('a', interval 25 month 100 day 130 minute)) --- !query 100 schema +-- !query schema struct --- !query 100 output +-- !query output {"a":"2 years 1 months 100 days 2 hours 10 minutes"} diff --git a/sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out b/sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out index 857073a827f24..6b7edfb2356f4 100644 --- a/sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out @@ -2,193 +2,193 @@ -- Number of queries: 24 --- !query 0 +-- !query CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM t1 INNER JOIN empty_table --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output --- !query 4 +-- !query SELECT * FROM t1 CROSS JOIN empty_table --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT * FROM t1 LEFT OUTER JOIN empty_table --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 NULL --- !query 6 +-- !query SELECT * FROM t1 RIGHT OUTER JOIN empty_table --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output --- !query 7 +-- !query SELECT * FROM t1 FULL OUTER JOIN empty_table --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 NULL --- !query 8 +-- !query SELECT * FROM t1 LEFT SEMI JOIN empty_table --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output --- !query 9 +-- !query SELECT * FROM t1 LEFT ANTI JOIN empty_table --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 --- !query 10 +-- !query SELECT * FROM empty_table INNER JOIN t1 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT * FROM empty_table CROSS JOIN t1 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output --- !query 12 +-- !query SELECT * FROM empty_table LEFT OUTER JOIN t1 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output --- !query 13 +-- !query SELECT * FROM empty_table RIGHT OUTER JOIN t1 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output NULL 1 --- !query 14 +-- !query SELECT * FROM empty_table FULL OUTER JOIN t1 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output NULL 1 --- !query 15 +-- !query SELECT * FROM empty_table LEFT SEMI JOIN t1 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output --- !query 16 +-- !query SELECT * FROM empty_table LEFT ANTI JOIN t1 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output --- !query 17 +-- !query SELECT * FROM empty_table INNER JOIN empty_table --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output --- !query 18 +-- !query SELECT * FROM empty_table CROSS JOIN empty_table --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output --- !query 19 +-- !query SELECT * FROM empty_table LEFT OUTER JOIN empty_table --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output --- !query 20 +-- !query SELECT * FROM empty_table RIGHT OUTER JOIN empty_table --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output --- !query 21 +-- !query SELECT * FROM empty_table FULL OUTER JOIN empty_table --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output --- !query 22 +-- !query SELECT * FROM empty_table LEFT SEMI JOIN empty_table --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output --- !query 23 +-- !query SELECT * FROM empty_table LEFT ANTI JOIN empty_table --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index 1f6e5e78ca226..21a3531caf732 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -2,127 +2,127 @@ -- Number of queries: 42 --- !query 0 +-- !query select to_json(named_struct('a', 1, 'b', 2)) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output {"a":1,"b":2} --- !query 1 +-- !query select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output {"time":"26/08/2015"} --- !query 2 +-- !query select to_json(array(named_struct('a', 1, 'b', 2))) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output [{"a":1,"b":2}] --- !query 3 +-- !query select to_json(map(named_struct('a', 1, 'b', 2), named_struct('a', 1, 'b', 2))) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output {"[1,2]":{"a":1,"b":2}} --- !query 4 +-- !query select to_json(map('a', named_struct('a', 1, 'b', 2))) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output {"a":{"a":1,"b":2}} --- !query 5 +-- !query select to_json(map('a', 1)) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output {"a":1} --- !query 6 +-- !query select to_json(array(map('a',1))) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output [{"a":1}] --- !query 7 +-- !query select to_json(array(map('a',1), map('b',2))) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output [{"a":1},{"b":2}] --- !query 8 +-- !query select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException Must use a map() function for options;; line 1 pos 7 --- !query 9 +-- !query select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)) --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException A type of keys and values in map() must be string, but got map;; line 1 pos 7 --- !query 10 +-- !query select to_json() --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7 --- !query 11 +-- !query select from_json('{"a":1}', 'a INT') --- !query 11 schema +-- !query schema struct> --- !query 11 output +-- !query output {"a":1} --- !query 12 +-- !query select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) --- !query 12 schema +-- !query schema struct> --- !query 12 output +-- !query output {"time":2015-08-26 00:00:00} --- !query 13 +-- !query select from_json('{"a":1}', 1) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException Schema should be specified in DDL format as a string literal or output of the schema_of_json function instead of 1;; line 1 pos 7 --- !query 14 +-- !query select from_json('{"a":1}', 'a InvalidType') --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException DataType invalidtype is not supported.(line 1, pos 2) @@ -133,222 +133,222 @@ a InvalidType ; line 1 pos 7 --- !query 15 +-- !query select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE')) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException Must use a map() function for options;; line 1 pos 7 --- !query 16 +-- !query select from_json('{"a":1}', 'a INT', map('mode', 1)) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.AnalysisException A type of keys and values in map() must be string, but got map;; line 1 pos 7 --- !query 17 +-- !query select from_json() --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7 --- !query 18 +-- !query SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a') --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output NULL 2 NULL 1 --- !query 19 +-- !query CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query SELECT json_tuple(jsonField, 'b', CAST(NULL AS STRING), a) FROM jsonTable --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 2 NULL 1 --- !query 21 +-- !query DROP VIEW IF EXISTS jsonTable --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query select from_json('{"a":1, "b":2}', 'map') --- !query 22 schema +-- !query schema struct> --- !query 22 output +-- !query output {"a":1,"b":2} --- !query 23 +-- !query select from_json('{"a":1, "b":"2"}', 'struct') --- !query 23 schema +-- !query schema struct> --- !query 23 output +-- !query output {"a":1,"b":"2"} --- !query 24 +-- !query select schema_of_json('{"c1":0, "c2":[1]}') --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output struct> --- !query 25 +-- !query select from_json('{"c1":[1, 2, 3]}', schema_of_json('{"c1":[0]}')) --- !query 25 schema +-- !query schema struct>> --- !query 25 output +-- !query output {"c1":[1,2,3]} --- !query 26 +-- !query select from_json('[1, 2, 3]', 'array') --- !query 26 schema +-- !query schema struct> --- !query 26 output +-- !query output [1,2,3] --- !query 27 +-- !query select from_json('[1, "2", 3]', 'array') --- !query 27 schema +-- !query schema struct> --- !query 27 output +-- !query output NULL --- !query 28 +-- !query select from_json('[1, 2, null]', 'array') --- !query 28 schema +-- !query schema struct> --- !query 28 output +-- !query output [1,2,null] --- !query 29 +-- !query select from_json('[{"a": 1}, {"a":2}]', 'array>') --- !query 29 schema +-- !query schema struct>> --- !query 29 output +-- !query output [{"a":1},{"a":2}] --- !query 30 +-- !query select from_json('{"a": 1}', 'array>') --- !query 30 schema +-- !query schema struct>> --- !query 30 output +-- !query output [{"a":1}] --- !query 31 +-- !query select from_json('[null, {"a":2}]', 'array>') --- !query 31 schema +-- !query schema struct>> --- !query 31 output +-- !query output [null,{"a":2}] --- !query 32 +-- !query select from_json('[{"a": 1}, {"b":2}]', 'array>') --- !query 32 schema +-- !query schema struct>> --- !query 32 output +-- !query output [{"a":1},{"b":2}] --- !query 33 +-- !query select from_json('[{"a": 1}, 2]', 'array>') --- !query 33 schema +-- !query schema struct>> --- !query 33 output +-- !query output NULL --- !query 34 +-- !query select to_json(array('1', '2', '3')) --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output ["1","2","3"] --- !query 35 +-- !query select to_json(array(array(1, 2, 3), array(4))) --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output [[1,2,3],[4]] --- !query 36 +-- !query select schema_of_json('{"c1":1}', map('primitivesAsString', 'true')) --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output struct --- !query 37 +-- !query select schema_of_json('{"c1":01, "c2":0.1}', map('allowNumericLeadingZeros', 'true', 'prefersDecimal', 'true')) --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output struct --- !query 38 +-- !query select schema_of_json(null) --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'schema_of_json(NULL)' due to data type mismatch: The input json should be a string literal and not null; however, got NULL.; line 1 pos 7 --- !query 39 +-- !query CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output --- !query 40 +-- !query SELECT schema_of_json(jsonField) FROM jsonTable --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'schema_of_json(jsontable.`jsonField`)' due to data type mismatch: The input json should be a string literal and not null; however, got jsontable.`jsonField`.; line 1 pos 7 --- !query 41 +-- !query DROP VIEW IF EXISTS jsonTable --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/limit.sql.out index 02fe1de84f753..281326e22a97a 100644 --- a/sql/core/src/test/resources/sql-tests/results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/limit.sql.out @@ -2,125 +2,125 @@ -- Number of queries: 14 --- !query 0 +-- !query SELECT * FROM testdata LIMIT 2 --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1 1 2 2 --- !query 1 +-- !query SELECT * FROM arraydata LIMIT 2 --- !query 1 schema +-- !query schema struct,nestedarraycol:array>> --- !query 1 output +-- !query output [1,2,3] [[1,2,3]] [2,3,4] [[2,3,4]] --- !query 2 +-- !query SELECT * FROM mapdata LIMIT 2 --- !query 2 schema +-- !query schema struct> --- !query 2 output +-- !query output {1:"a1",2:"b1",3:"c1",4:"d1",5:"e1"} {1:"a2",2:"b2",3:"c2",4:"d2"} --- !query 3 +-- !query SELECT * FROM testdata LIMIT 2 + 1 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 1 2 2 3 3 --- !query 4 +-- !query SELECT * FROM testdata LIMIT CAST(1 AS int) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 1 --- !query 5 +-- !query SELECT * FROM testdata LIMIT -1 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException The limit expression must be equal to or greater than 0, but got -1; --- !query 6 +-- !query SELECT * FROM testData TABLESAMPLE (-1 ROWS) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException The limit expression must be equal to or greater than 0, but got -1; --- !query 7 +-- !query SELECT * FROM testdata LIMIT CAST(1 AS INT) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 1 --- !query 8 +-- !query SELECT * FROM testdata LIMIT CAST(NULL AS INT) --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException The evaluated limit expression must not be null, but got CAST(NULL AS INT); --- !query 9 +-- !query SELECT * FROM testdata LIMIT key > 3 --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException The limit expression must evaluate to a constant value, but got (testdata.`key` > 3); --- !query 10 +-- !query SELECT * FROM testdata LIMIT true --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException The limit expression must be integer type, but got boolean; --- !query 11 +-- !query SELECT * FROM testdata LIMIT 'a' --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException The limit expression must be integer type, but got string; --- !query 12 +-- !query SELECT * FROM (SELECT * FROM range(10) LIMIT 5) WHERE id > 3 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 4 --- !query 13 +-- !query SELECT * FROM testdata WHERE key < 3 LIMIT ALL --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 1 2 2 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 382294010914f..f6720f6c5faa4 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -2,43 +2,43 @@ -- Number of queries: 50 --- !query 0 +-- !query select null, Null, nUll --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output NULL NULL NULL --- !query 1 +-- !query select true, tRue, false, fALse --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output true true false false --- !query 2 +-- !query select 1Y --- !query 2 schema +-- !query schema struct<1:tinyint> --- !query 2 output +-- !query output 1 --- !query 3 +-- !query select 127Y, -128Y --- !query 3 schema +-- !query schema struct<127:tinyint,-128:tinyint> --- !query 3 output +-- !query output 127 -128 --- !query 4 +-- !query select 128Y --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7) @@ -48,27 +48,27 @@ select 128Y -------^^^ --- !query 5 +-- !query select 1S --- !query 5 schema +-- !query schema struct<1:smallint> --- !query 5 output +-- !query output 1 --- !query 6 +-- !query select 32767S, -32768S --- !query 6 schema +-- !query schema struct<32767:smallint,-32768:smallint> --- !query 6 output +-- !query output 32767 -32768 --- !query 7 +-- !query select 32768S --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7) @@ -78,27 +78,27 @@ select 32768S -------^^^ --- !query 8 +-- !query select 1L, 2147483648L --- !query 8 schema +-- !query schema struct<1:bigint,2147483648:bigint> --- !query 8 output +-- !query output 1 2147483648 --- !query 9 +-- !query select 9223372036854775807L, -9223372036854775808L --- !query 9 schema +-- !query schema struct<9223372036854775807:bigint,-9223372036854775808:bigint> --- !query 9 output +-- !query output 9223372036854775807 -9223372036854775808 --- !query 10 +-- !query select 9223372036854775808L --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7) @@ -108,43 +108,43 @@ select 9223372036854775808L -------^^^ --- !query 11 +-- !query select 1, -1 --- !query 11 schema +-- !query schema struct<1:int,-1:int> --- !query 11 output +-- !query output 1 -1 --- !query 12 +-- !query select 2147483647, -2147483648 --- !query 12 schema +-- !query schema struct<2147483647:int,-2147483648:int> --- !query 12 output +-- !query output 2147483647 -2147483648 --- !query 13 +-- !query select 9223372036854775807, -9223372036854775808 --- !query 13 schema +-- !query schema struct<9223372036854775807:bigint,-9223372036854775808:bigint> --- !query 13 output +-- !query output 9223372036854775807 -9223372036854775808 --- !query 14 +-- !query select 9223372036854775808, -9223372036854775809 --- !query 14 schema +-- !query schema struct<9223372036854775808:decimal(19,0),-9223372036854775809:decimal(19,0)> --- !query 14 output +-- !query output 9223372036854775808 -9223372036854775809 --- !query 15 +-- !query select 1234567890123456789012345678901234567890 --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38 @@ -152,11 +152,11 @@ decimal can only support precision up to 38 select 1234567890123456789012345678901234567890 --- !query 16 +-- !query select 1234567890123456789012345678901234567890.0 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38 @@ -164,27 +164,27 @@ decimal can only support precision up to 38 select 1234567890123456789012345678901234567890.0 --- !query 17 +-- !query select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 --- !query 17 schema +-- !query schema struct<1.0:double,1.2:double,1.0E10:double,150000.0:double,0.1:double,0.1:double,10000.0:double,90.0:double,90.0:double,90.0:double,90.0:double> --- !query 17 output +-- !query output 1.0 1.2 1.0E10 150000.0 0.1 0.1 10000.0 90.0 90.0 90.0 90.0 --- !query 18 +-- !query select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 --- !query 18 schema +-- !query schema struct<-1.0:double,-1.2:double,-1.0E10:double,-150000.0:double,-0.1:double,-0.1:double,-10000.0:double> --- !query 18 output +-- !query output -1.0 -1.2 -1.0E10 -150000.0 -0.1 -0.1 -10000.0 --- !query 19 +-- !query select .e3 --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'select .'(line 1, pos 7) @@ -194,11 +194,11 @@ select .e3 -------^^^ --- !query 20 +-- !query select 1E309, -1E309 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Numeric literal 1E309 does not fit in range [-1.7976931348623157E+308, 1.7976931348623157E+308] for type double(line 1, pos 7) @@ -208,93 +208,93 @@ select 1E309, -1E309 -------^^^ --- !query 21 +-- !query select 0.3, -0.8, .5, -.18, 0.1111, .1111 --- !query 21 schema +-- !query schema struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> --- !query 21 output +-- !query output 0.3 -0.8 0.5 -0.18 0.1111 0.1111 --- !query 22 +-- !query select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d --- !query 22 schema +-- !query schema struct<1.2345678901234568E48:double,1.2345678901234568E48:double> --- !query 22 output +-- !query output 1.2345678901234568E48 1.2345678901234568E48 --- !query 23 +-- !query select "Hello Peter!", 'hello lee!' --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output Hello Peter! hello lee! --- !query 24 +-- !query select 'hello' 'world', 'hello' " " 'lee' --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output helloworld hello lee --- !query 25 +-- !query select "hello 'peter'" --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output hello 'peter' --- !query 26 +-- !query select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output pattern% no-pattern\% pattern\% pattern\\% --- !query 27 +-- !query select '\'', '"', '\n', '\r', '\t', 'Z' --- !query 27 schema +-- !query schema struct<':string,":string, :string, :string, :string,Z:string> --- !query 27 output +-- !query output ' " Z --- !query 28 +-- !query select '\110\145\154\154\157\041' --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output Hello! --- !query 29 +-- !query select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output World :) --- !query 30 +-- !query select dAte '2016-03-12' --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 2016-03-12 --- !query 31 +-- !query select date 'mar 11 2016' --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: mar 11 2016(line 1, pos 7) @@ -304,19 +304,19 @@ select date 'mar 11 2016' -------^^^ --- !query 32 +-- !query select tImEstAmp '2016-03-11 20:54:00.000' --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 2016-03-11 20:54:00 --- !query 33 +-- !query select timestamp '2016-33-11 20:54:00.000' --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the TIMESTAMP value: 2016-33-11 20:54:00.000(line 1, pos 7) @@ -326,11 +326,11 @@ select timestamp '2016-33-11 20:54:00.000' -------^^^ --- !query 34 +-- !query select GEO '(10,-6)' --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Literals of type 'GEO' are currently not supported.(line 1, pos 7) @@ -340,19 +340,19 @@ select GEO '(10,-6)' -------^^^ --- !query 35 +-- !query select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD --- !query 35 schema +-- !query schema struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> --- !query 35 output +-- !query output 90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 --- !query 36 +-- !query select 1.20E-38BD --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38(line 1, pos 7) @@ -362,19 +362,19 @@ select 1.20E-38BD -------^^^ --- !query 37 +-- !query select x'2379ACFe' --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output #y�� --- !query 38 +-- !query select X'XuZ' --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException contains illegal character for hexBinary: 0XuZ(line 1, pos 7) @@ -384,98 +384,98 @@ select X'XuZ' -------^^^ --- !query 39 +-- !query SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 --- !query 39 schema +-- !query schema struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E8:double,3.14E-8:double,-3.14E8:double,-3.14E-8:double,3.14E8:double,3.14E8:double,3.14E-8:double> --- !query 39 output +-- !query output 3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 --- !query 40 +-- !query select +date '1999-01-01' --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 --- !query 41 +-- !query select +timestamp '1999-01-01' --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 --- !query 42 +-- !query select +interval '1 day' --- !query 42 schema +-- !query schema struct<(+ INTERVAL '1 days'):interval> --- !query 42 output +-- !query output 1 days --- !query 43 +-- !query select +map(1, 2) --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ map(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'map(1, 2)' is of map type.; line 1 pos 7 --- !query 44 +-- !query select +array(1,2) --- !query 44 schema +-- !query schema struct<> --- !query 44 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ array(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'array(1, 2)' is of array type.; line 1 pos 7 --- !query 45 +-- !query select +named_struct('a', 1, 'b', 'spark') --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ named_struct('a', 1, 'b', 'spark'))' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'named_struct('a', 1, 'b', 'spark')' is of struct type.; line 1 pos 7 --- !query 46 +-- !query select +X'1' --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(+ X'01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7 --- !query 47 +-- !query select -date '1999-01-01' --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7 --- !query 48 +-- !query select -timestamp '1999-01-01' --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7 --- !query 49 +-- !query select -x'2379ACFe' --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out index cd0818a5189b5..bd8ffb82ee129 100644 --- a/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/misc-functions.sql.out @@ -2,57 +2,57 @@ -- Number of queries: 7 --- !query 0 +-- !query select typeof(null) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output null --- !query 1 +-- !query select typeof(true) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output boolean --- !query 2 +-- !query select typeof(1Y), typeof(1S), typeof(1), typeof(1L) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output tinyint smallint int bigint --- !query 3 +-- !query select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output float double decimal(2,1) --- !query 4 +-- !query select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days') --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output date timestamp interval --- !query 5 +-- !query select typeof(x'ABCD'), typeof('SPARK') --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output binary string --- !query 6 +-- !query select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output array map struct diff --git a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out index 43f2f9af61d9b..13f319700df3f 100644 --- a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out @@ -2,63 +2,63 @@ -- Number of queries: 6 --- !query 0 +-- !query create temporary view nt1 as select * from values ("one", 1), ("two", 2), ("three", 3) as nt1(k, v1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view nt2 as select * from values ("one", 1), ("two", 22), ("one", 5) as nt2(k, v2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM nt1 natural join nt2 where k = "one" --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output one 1 1 one 1 5 --- !query 3 +-- !query SELECT * FROM nt1 natural left join nt2 order by v1, v2 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output one 1 1 one 1 5 two 2 22 three 3 NULL --- !query 4 +-- !query SELECT * FROM nt1 natural right join nt2 order by v1, v2 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output one 1 1 one 1 5 two 2 22 --- !query 5 +-- !query SELECT count(*) FROM nt1 natural full outer join nt2 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 4 diff --git a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out index 5005dfeb6cd14..5e7eec56743b1 100644 --- a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out @@ -2,75 +2,75 @@ -- Number of queries: 28 --- !query 0 +-- !query create table t1(a int, b int, c int) using parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query insert into t1 values(1,0,0) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query insert into t1 values(2,0,1) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query insert into t1 values(3,1,0) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query insert into t1 values(4,1,1) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query insert into t1 values(5,null,0) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query insert into t1 values(6,null,1) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query insert into t1 values(7,null,null) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query select a, b+c from t1 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 0 2 1 3 1 @@ -80,11 +80,11 @@ struct 7 NULL --- !query 9 +-- !query select a+10, b*0 from t1 --- !query 9 schema +-- !query schema struct<(a + 10):int,(b * 0):int> --- !query 9 output +-- !query output 11 0 12 0 13 0 @@ -94,31 +94,31 @@ struct<(a + 10):int,(b * 0):int> 17 NULL --- !query 10 +-- !query select distinct b from t1 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 0 1 NULL --- !query 11 +-- !query select b from t1 union select b from t1 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 0 1 NULL --- !query 12 +-- !query select a+20, case b when c then 1 else 0 end from t1 --- !query 12 schema +-- !query schema struct<(a + 20):int,CASE WHEN (b = c) THEN 1 ELSE 0 END:int> --- !query 12 output +-- !query output 21 1 22 0 23 0 @@ -128,11 +128,11 @@ struct<(a + 20):int,CASE WHEN (b = c) THEN 1 ELSE 0 END:int> 27 0 --- !query 13 +-- !query select a+30, case c when b then 1 else 0 end from t1 --- !query 13 schema +-- !query schema struct<(a + 30):int,CASE WHEN (c = b) THEN 1 ELSE 0 END:int> --- !query 13 output +-- !query output 31 1 32 0 33 0 @@ -142,11 +142,11 @@ struct<(a + 30):int,CASE WHEN (c = b) THEN 1 ELSE 0 END:int> 37 0 --- !query 14 +-- !query select a+40, case when b<>0 then 1 else 0 end from t1 --- !query 14 schema +-- !query schema struct<(a + 40):int,CASE WHEN (NOT (b = 0)) THEN 1 ELSE 0 END:int> --- !query 14 output +-- !query output 41 0 42 0 43 1 @@ -156,11 +156,11 @@ struct<(a + 40):int,CASE WHEN (NOT (b = 0)) THEN 1 ELSE 0 END:int> 47 0 --- !query 15 +-- !query select a+50, case when not b<>0 then 1 else 0 end from t1 --- !query 15 schema +-- !query schema struct<(a + 50):int,CASE WHEN (NOT (NOT (b = 0))) THEN 1 ELSE 0 END:int> --- !query 15 output +-- !query output 51 1 52 1 53 0 @@ -170,11 +170,11 @@ struct<(a + 50):int,CASE WHEN (NOT (NOT (b = 0))) THEN 1 ELSE 0 END:int> 57 0 --- !query 16 +-- !query select a+60, case when b<>0 and c<>0 then 1 else 0 end from t1 --- !query 16 schema +-- !query schema struct<(a + 60):int,CASE WHEN ((NOT (b = 0)) AND (NOT (c = 0))) THEN 1 ELSE 0 END:int> --- !query 16 output +-- !query output 61 0 62 0 63 0 @@ -184,11 +184,11 @@ struct<(a + 60):int,CASE WHEN ((NOT (b = 0)) AND (NOT (c = 0))) THEN 1 ELSE 0 EN 67 0 --- !query 17 +-- !query select a+70, case when not (b<>0 and c<>0) then 1 else 0 end from t1 --- !query 17 schema +-- !query schema struct<(a + 70):int,CASE WHEN (NOT ((NOT (b = 0)) AND (NOT (c = 0)))) THEN 1 ELSE 0 END:int> --- !query 17 output +-- !query output 71 1 72 1 73 1 @@ -198,11 +198,11 @@ struct<(a + 70):int,CASE WHEN (NOT ((NOT (b = 0)) AND (NOT (c = 0)))) THEN 1 ELS 77 0 --- !query 18 +-- !query select a+80, case when b<>0 or c<>0 then 1 else 0 end from t1 --- !query 18 schema +-- !query schema struct<(a + 80):int,CASE WHEN ((NOT (b = 0)) OR (NOT (c = 0))) THEN 1 ELSE 0 END:int> --- !query 18 output +-- !query output 81 0 82 1 83 1 @@ -212,11 +212,11 @@ struct<(a + 80):int,CASE WHEN ((NOT (b = 0)) OR (NOT (c = 0))) THEN 1 ELSE 0 END 87 0 --- !query 19 +-- !query select a+90, case when not (b<>0 or c<>0) then 1 else 0 end from t1 --- !query 19 schema +-- !query schema struct<(a + 90):int,CASE WHEN (NOT ((NOT (b = 0)) OR (NOT (c = 0)))) THEN 1 ELSE 0 END:int> --- !query 19 output +-- !query output 91 1 92 0 93 0 @@ -226,41 +226,41 @@ struct<(a + 90):int,CASE WHEN (NOT ((NOT (b = 0)) OR (NOT (c = 0)))) THEN 1 ELSE 97 0 --- !query 20 +-- !query select count(*), count(b), sum(b), avg(b), min(b), max(b) from t1 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 7 4 2 0.5 0 1 --- !query 21 +-- !query select a+100 from t1 where b<10 --- !query 21 schema +-- !query schema struct<(a + 100):int> --- !query 21 output +-- !query output 101 102 103 104 --- !query 22 +-- !query select a+110 from t1 where not b>10 --- !query 22 schema +-- !query schema struct<(a + 110):int> --- !query 22 output +-- !query output 111 112 113 114 --- !query 23 +-- !query select a+120 from t1 where b<10 OR c=1 --- !query 23 schema +-- !query schema struct<(a + 120):int> --- !query 23 output +-- !query output 121 122 123 @@ -268,38 +268,38 @@ struct<(a + 120):int> 126 --- !query 24 +-- !query select a+130 from t1 where b<10 AND c=1 --- !query 24 schema +-- !query schema struct<(a + 130):int> --- !query 24 output +-- !query output 132 134 --- !query 25 +-- !query select a+140 from t1 where not (b<10 AND c=1) --- !query 25 schema +-- !query schema struct<(a + 140):int> --- !query 25 output +-- !query output 141 143 145 --- !query 26 +-- !query select a+150 from t1 where not (c=1 AND b<10) --- !query 26 schema +-- !query schema struct<(a + 150):int> --- !query 26 output +-- !query output 151 153 155 --- !query 27 +-- !query drop table t1 --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out b/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out index ed3a651aa6614..76a41f9170388 100644 --- a/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out @@ -2,37 +2,37 @@ -- Number of queries: 4 --- !query 0 +-- !query SELECT COUNT(NULL) FROM VALUES 1, 2, 3 --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 0 --- !query 1 +-- !query SELECT COUNT(1 + NULL) FROM VALUES 1, 2, 3 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 0 --- !query 2 +-- !query SELECT COUNT(NULL) OVER () FROM VALUES 1, 2, 3 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 0 0 0 --- !query 3 +-- !query SELECT COUNT(1 + NULL) OVER () FROM VALUES 1, 2, 3 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 0 0 0 diff --git a/sql/core/src/test/resources/sql-tests/results/operator-div.sql.out b/sql/core/src/test/resources/sql-tests/results/operator-div.sql.out index 75736bee669b0..3f933f4c0e449 100644 --- a/sql/core/src/test/resources/sql-tests/results/operator-div.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operator-div.sql.out @@ -2,145 +2,145 @@ -- Number of queries: 18 --- !query 0 +-- !query set spark.sql.legacy.integralDivide.returnBigint=true --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output spark.sql.legacy.integralDivide.returnBigint true --- !query 1 +-- !query select 5 div 2 --- !query 1 schema +-- !query schema struct<(5 div 2):bigint> --- !query 1 output +-- !query output 2 --- !query 2 +-- !query select 5 div 0 --- !query 2 schema +-- !query schema struct<(5 div 0):bigint> --- !query 2 output +-- !query output NULL --- !query 3 +-- !query select 5 div null --- !query 3 schema +-- !query schema struct<(5 div CAST(NULL AS INT)):bigint> --- !query 3 output +-- !query output NULL --- !query 4 +-- !query select null div 5 --- !query 4 schema +-- !query schema struct<(CAST(NULL AS INT) div 5):bigint> --- !query 4 output +-- !query output NULL --- !query 5 +-- !query select cast(51 as decimal(10, 0)) div cast(2 as decimal(2, 0)) --- !query 5 schema +-- !query schema struct<(CAST(CAST(51 AS DECIMAL(10,0)) AS DECIMAL(10,0)) div CAST(CAST(2 AS DECIMAL(2,0)) AS DECIMAL(10,0))):bigint> --- !query 5 output +-- !query output 25 --- !query 6 +-- !query select cast(5 as decimal(1, 0)) div cast(0 as decimal(2, 0)) --- !query 6 schema +-- !query schema struct<(CAST(CAST(5 AS DECIMAL(1,0)) AS DECIMAL(2,0)) div CAST(CAST(0 AS DECIMAL(2,0)) AS DECIMAL(2,0))):bigint> --- !query 6 output +-- !query output NULL --- !query 7 +-- !query select cast(5 as decimal(1, 0)) div cast(null as decimal(2, 0)) --- !query 7 schema +-- !query schema struct<(CAST(CAST(5 AS DECIMAL(1,0)) AS DECIMAL(2,0)) div CAST(CAST(NULL AS DECIMAL(2,0)) AS DECIMAL(2,0))):bigint> --- !query 7 output +-- !query output NULL --- !query 8 +-- !query select cast(null as decimal(1, 0)) div cast(5 as decimal(2, 0)) --- !query 8 schema +-- !query schema struct<(CAST(CAST(NULL AS DECIMAL(1,0)) AS DECIMAL(2,0)) div CAST(CAST(5 AS DECIMAL(2,0)) AS DECIMAL(2,0))):bigint> --- !query 8 output +-- !query output NULL --- !query 9 +-- !query set spark.sql.legacy.integralDivide.returnBigint=false --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output spark.sql.legacy.integralDivide.returnBigint false --- !query 10 +-- !query select 5 div 2 --- !query 10 schema +-- !query schema struct<(5 div 2):int> --- !query 10 output +-- !query output 2 --- !query 11 +-- !query select 5 div 0 --- !query 11 schema +-- !query schema struct<(5 div 0):int> --- !query 11 output +-- !query output NULL --- !query 12 +-- !query select 5 div null --- !query 12 schema +-- !query schema struct<(5 div CAST(NULL AS INT)):int> --- !query 12 output +-- !query output NULL --- !query 13 +-- !query select null div 5 --- !query 13 schema +-- !query schema struct<(CAST(NULL AS INT) div 5):int> --- !query 13 output +-- !query output NULL --- !query 14 +-- !query select cast(51 as decimal(10, 0)) div cast(2 as decimal(2, 0)) --- !query 14 schema +-- !query schema struct<(CAST(CAST(51 AS DECIMAL(10,0)) AS DECIMAL(10,0)) div CAST(CAST(2 AS DECIMAL(2,0)) AS DECIMAL(10,0))):decimal(10,0)> --- !query 14 output +-- !query output 25 --- !query 15 +-- !query select cast(5 as decimal(1, 0)) div cast(0 as decimal(2, 0)) --- !query 15 schema +-- !query schema struct<(CAST(CAST(5 AS DECIMAL(1,0)) AS DECIMAL(2,0)) div CAST(CAST(0 AS DECIMAL(2,0)) AS DECIMAL(2,0))):decimal(1,0)> --- !query 15 output +-- !query output NULL --- !query 16 +-- !query select cast(5 as decimal(1, 0)) div cast(null as decimal(2, 0)) --- !query 16 schema +-- !query schema struct<(CAST(CAST(5 AS DECIMAL(1,0)) AS DECIMAL(2,0)) div CAST(CAST(NULL AS DECIMAL(2,0)) AS DECIMAL(2,0))):decimal(1,0)> --- !query 16 output +-- !query output NULL --- !query 17 +-- !query select cast(null as decimal(1, 0)) div cast(5 as decimal(2, 0)) --- !query 17 schema +-- !query schema struct<(CAST(CAST(NULL AS DECIMAL(1,0)) AS DECIMAL(2,0)) div CAST(CAST(5 AS DECIMAL(2,0)) AS DECIMAL(2,0))):decimal(1,0)> --- !query 17 output +-- !query output NULL diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index 89dbf53c3c2ef..548281014afd7 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -2,393 +2,393 @@ -- Number of queries: 49 --- !query 0 +-- !query select -100 --- !query 0 schema +-- !query schema struct<-100:int> --- !query 0 output +-- !query output -100 --- !query 1 +-- !query select +230 --- !query 1 schema +-- !query schema struct<(+ 230):int> --- !query 1 output +-- !query output 230 --- !query 2 +-- !query select -5.2 --- !query 2 schema +-- !query schema struct<-5.2:decimal(2,1)> --- !query 2 output +-- !query output -5.2 --- !query 3 +-- !query select +6.8e0 --- !query 3 schema +-- !query schema struct<(+ 6.8):double> --- !query 3 output +-- !query output 6.8 --- !query 4 +-- !query select -key, +key from testdata where key = 2 --- !query 4 schema +-- !query schema struct<(- key):int,(+ key):int> --- !query 4 output +-- !query output -2 2 --- !query 5 +-- !query select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1 --- !query 5 schema +-- !query schema struct<(- (key + 1)):int,((- key) + 1):int,(+ (key + 5)):int> --- !query 5 output +-- !query output -2 0 6 --- !query 6 +-- !query select -max(key), +max(key) from testdata --- !query 6 schema +-- !query schema struct<(- max(key)):int,(+ max(key)):int> --- !query 6 output +-- !query output -100 100 --- !query 7 +-- !query select - (-10) --- !query 7 schema +-- !query schema struct<(- -10):int> --- !query 7 output +-- !query output 10 --- !query 8 +-- !query select + (-key) from testdata where key = 32 --- !query 8 schema +-- !query schema struct<(+ (- key)):int> --- !query 8 output +-- !query output -32 --- !query 9 +-- !query select - (+max(key)) from testdata --- !query 9 schema +-- !query schema struct<(- (+ max(key))):int> --- !query 9 output +-- !query output -100 --- !query 10 +-- !query select - - 3 --- !query 10 schema +-- !query schema struct<(- -3):int> --- !query 10 output +-- !query output 3 --- !query 11 +-- !query select - + 20 --- !query 11 schema +-- !query schema struct<(- (+ 20)):int> --- !query 11 output +-- !query output -20 --- !query 12 +-- !query select + + 100 --- !query 12 schema +-- !query schema struct<(+ (+ 100)):int> --- !query 12 output +-- !query output 100 --- !query 13 +-- !query select - - max(key) from testdata --- !query 13 schema +-- !query schema struct<(- (- max(key))):int> --- !query 13 output +-- !query output 100 --- !query 14 +-- !query select + - key from testdata where key = 33 --- !query 14 schema +-- !query schema struct<(+ (- key)):int> --- !query 14 output +-- !query output -33 --- !query 15 +-- !query select 5 / 2 --- !query 15 schema +-- !query schema struct<(CAST(5 AS DOUBLE) / CAST(2 AS DOUBLE)):double> --- !query 15 output +-- !query output 2.5 --- !query 16 +-- !query select 5 / 0 --- !query 16 schema +-- !query schema struct<(CAST(5 AS DOUBLE) / CAST(0 AS DOUBLE)):double> --- !query 16 output +-- !query output NULL --- !query 17 +-- !query select 5 / null --- !query 17 schema +-- !query schema struct<(CAST(5 AS DOUBLE) / CAST(NULL AS DOUBLE)):double> --- !query 17 output +-- !query output NULL --- !query 18 +-- !query select null / 5 --- !query 18 schema +-- !query schema struct<(CAST(NULL AS DOUBLE) / CAST(5 AS DOUBLE)):double> --- !query 18 output +-- !query output NULL --- !query 19 +-- !query select 1 + 2 --- !query 19 schema +-- !query schema struct<(1 + 2):int> --- !query 19 output +-- !query output 3 --- !query 20 +-- !query select 1 - 2 --- !query 20 schema +-- !query schema struct<(1 - 2):int> --- !query 20 output +-- !query output -1 --- !query 21 +-- !query select 2 * 5 --- !query 21 schema +-- !query schema struct<(2 * 5):int> --- !query 21 output +-- !query output 10 --- !query 22 +-- !query select 5 % 3 --- !query 22 schema +-- !query schema struct<(5 % 3):int> --- !query 22 output +-- !query output 2 --- !query 23 +-- !query select pmod(-7, 3) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 2 --- !query 24 +-- !query select cot(1) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 0.6420926159343306 --- !query 25 +-- !query select cot(null) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output NULL --- !query 26 +-- !query select cot(0) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output Infinity --- !query 27 +-- !query select cot(-1) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output -0.6420926159343306 --- !query 28 +-- !query select ceiling(0) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 0 --- !query 29 +-- !query select ceiling(1) --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 1 --- !query 30 +-- !query select ceil(1234567890123456) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1234567890123456 --- !query 31 +-- !query select ceiling(1234567890123456) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 1234567890123456 --- !query 32 +-- !query select ceil(0.01) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 1 --- !query 33 +-- !query select ceiling(-0.10) --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 0 --- !query 34 +-- !query select floor(0) --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 0 --- !query 35 +-- !query select floor(1) --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 1 --- !query 36 +-- !query select floor(1234567890123456) --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 1234567890123456 --- !query 37 +-- !query select floor(0.01) --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 0 --- !query 38 +-- !query select floor(-0.10) --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output -1 --- !query 39 +-- !query select 1 > 0.00001 --- !query 39 schema +-- !query schema struct<(CAST(1 AS BIGINT) > 0):boolean> --- !query 39 output +-- !query output true --- !query 40 +-- !query select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null) --- !query 40 schema +-- !query schema struct<(7 % 2):int,(7 % 0):int,(0 % 2):int,(7 % CAST(NULL AS INT)):int,(CAST(NULL AS INT) % 2):int,(CAST(NULL AS DOUBLE) % CAST(NULL AS DOUBLE)):double> --- !query 40 output +-- !query output 1 NULL 0 NULL NULL NULL --- !query 41 +-- !query select BIT_LENGTH('abc') --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 24 --- !query 42 +-- !query select CHAR_LENGTH('abc') --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 3 --- !query 43 +-- !query select CHARACTER_LENGTH('abc') --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 3 --- !query 44 +-- !query select OCTET_LENGTH('abc') --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 3 --- !query 45 +-- !query select abs(-3.13), abs('-2.19') --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 3.13 2.19 --- !query 46 +-- !query select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11) --- !query 46 schema +-- !query schema struct<(+ CAST(-1.11 AS DOUBLE)):double,(+ -1.11):decimal(3,2),(- CAST(-1.11 AS DOUBLE)):double,(- -1.11):decimal(3,2)> --- !query 46 output +-- !query output -1.11 -1.11 1.11 1.11 --- !query 47 +-- !query select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null) --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 1 0 NULL NULL NULL NULL --- !query 48 +-- !query select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)) --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out index e1289dd8c8bde..67d271790eef0 100644 --- a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out @@ -2,32 +2,32 @@ -- Number of queries: 17 --- !query 0 +-- !query create table spark_10747(col1 int, col2 int, col3 int) using parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8), (6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query select col1, col2, col3, sum(col2) over (partition by col1 order by col3 desc nulls last, col2 rows between 2 preceding and 2 following ) as sum_col2 from spark_10747 where col1 = 6 order by sum_col2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 6 9 10 28 6 13 NULL 34 6 10 NULL 41 @@ -39,15 +39,15 @@ struct 6 7 4 58 --- !query 3 +-- !query select col1, col2, col3, sum(col2) over (partition by col1 order by col3 desc nulls first, col2 rows between 2 preceding and 2 following ) as sum_col2 from spark_10747 where col1 = 6 order by sum_col2 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 6 10 NULL 32 6 11 4 33 6 13 NULL 44 @@ -59,15 +59,15 @@ struct 6 7 8 58 --- !query 4 +-- !query select col1, col2, col3, sum(col2) over (partition by col1 order by col3 asc nulls last, col2 rows between 2 preceding and 2 following ) as sum_col2 from spark_10747 where col1 = 6 order by sum_col2 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 6 7 4 25 6 13 NULL 35 6 11 4 40 @@ -79,15 +79,15 @@ struct 6 9 10 61 --- !query 5 +-- !query select col1, col2, col3, sum(col2) over (partition by col1 order by col3 asc nulls first, col2 rows between 2 preceding and 2 following ) as sum_col2 from spark_10747 where col1 = 6 order by sum_col2 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 6 10 NULL 30 6 12 10 36 6 13 NULL 41 @@ -99,11 +99,11 @@ struct 6 15 8 58 --- !query 6 +-- !query SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 6 10 NULL 6 13 NULL 6 7 4 @@ -115,11 +115,11 @@ struct 6 12 10 --- !query 7 +-- !query SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 6 7 4 6 11 4 6 7 8 @@ -131,11 +131,11 @@ struct 6 13 NULL --- !query 8 +-- !query SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 6 10 NULL 6 13 NULL 6 9 10 @@ -147,11 +147,11 @@ struct 6 11 4 --- !query 9 +-- !query SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 6 9 10 6 12 10 6 7 8 @@ -163,15 +163,15 @@ struct 6 13 NULL --- !query 10 +-- !query drop table spark_10747 --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query create table spark_10747_mix( col1 string, col2 int, @@ -179,13 +179,13 @@ col3 double, col4 decimal(10,2), col5 decimal(20,1)) using parquet --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query INSERT INTO spark_10747_mix VALUES ('b', 2, 1.0, 1.00, 10.0), ('d', 3, 2.0, 3.00, 0.0), @@ -195,17 +195,17 @@ INSERT INTO spark_10747_mix VALUES ('d', 3, null, 4.00, 1.0), ('a', 1, 1.0, 1.00, null), ('c', 3, 2.0, 2.00, null) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query select * from spark_10747_mix order by col1 nulls last, col5 nulls last --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output a 1 1.0 1.00 NULL b 2 1.0 1.00 10.0 c 3 2.0 2.00 15.1 @@ -216,11 +216,11 @@ d 3 NULL 4.00 1.0 NULL 3 0.0 3.00 1.0 --- !query 14 +-- !query select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output NULL 3 0.0 3.00 1.0 d 3 0.0 3.00 1.0 d 3 NULL 4.00 1.0 @@ -231,11 +231,11 @@ b 2 1.0 1.00 10.0 a 1 1.0 1.00 NULL --- !query 15 +-- !query select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output c 3 2.0 2.00 NULL a 1 1.0 1.00 NULL c 3 2.0 2.00 15.1 @@ -246,9 +246,9 @@ d 3 NULL 4.00 1.0 d 3 2.0 3.00 0.0 --- !query 16 +-- !query drop table spark_10747_mix --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out index cc47cc67c87c8..44c811a7439c0 100644 --- a/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 12 --- !query 0 +-- !query create temporary view data as select * from values (1, 1), (1, 2), @@ -11,17 +11,17 @@ create temporary view data as select * from values (3, 1), (3, 2) as data(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select * from data order by 1 desc --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 3 1 3 2 2 1 @@ -30,11 +30,11 @@ struct 1 2 --- !query 2 +-- !query select * from data order by 1 desc, b desc --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 3 2 3 1 2 2 @@ -43,11 +43,11 @@ struct 1 1 --- !query 3 +-- !query select * from data order by 1 desc, 2 desc --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 3 2 3 1 2 2 @@ -56,11 +56,11 @@ struct 1 1 --- !query 4 +-- !query select * from data order by 1 + 0 desc, b desc --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 2 2 2 3 2 @@ -69,38 +69,38 @@ struct 3 1 --- !query 5 +-- !query select * from data order by 0 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException ORDER BY position 0 is not in select list (valid range is [1, 2]); line 1 pos 28 --- !query 6 +-- !query select * from data order by -1 --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException ORDER BY position -1 is not in select list (valid range is [1, 2]); line 1 pos 28 --- !query 7 +-- !query select * from data order by 3 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.AnalysisException ORDER BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 28 --- !query 8 +-- !query select * from data sort by 1 desc --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 1 1 2 2 1 @@ -109,19 +109,19 @@ struct 3 2 --- !query 9 +-- !query set spark.sql.orderByOrdinal=false --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output spark.sql.orderByOrdinal false --- !query 10 +-- !query select * from data order by 0 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 1 1 2 2 1 @@ -130,11 +130,11 @@ struct 3 2 --- !query 11 +-- !query select * from data sort by 0 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 1 1 2 2 1 diff --git a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out index b39fdb0e58720..703ce231c53ff 100644 --- a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out @@ -2,27 +2,27 @@ -- Number of queries: 6 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (-234), (145), (367), (975), (298) as t1(int_col1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) as t2(int_col0, int_col1) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT (SUM(COALESCE(t1.int_col1, t2.int_col0))), ((COALESCE(t1.int_col1, t2.int_col0)) * 2) @@ -33,32 +33,32 @@ GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)), COALESCE(t1.int_col1, t2.int_col0) HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0))) > ((COALESCE(t1.int_col1, t2.int_col0)) * 2) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output -367 -734 -507 -1014 -769 -1538 -800 -1600 --- !query 3 +-- !query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT * FROM ( SELECT @@ -66,7 +66,7 @@ SELECT FROM t1 LEFT JOIN t2 ON false ) t where (t.int_col) is not null --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 97 diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index 9a8f783da4369..ac4e71e244bc0 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 32 --- !query 0 +-- !query create temporary view courseSales as select * from values ("dotNET", 2012, 10000), ("Java", 2012, 20000), @@ -10,35 +10,35 @@ create temporary view courseSales as select * from values ("dotNET", 2013, 48000), ("Java", 2013, 30000) as courseSales(course, year, earnings) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view years as select * from values (2012, 1), (2013, 2) as years(y, s) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view yearsWithComplexTypes as select * from values (2012, array(1, 1), map('1', 1), struct(1, 'a')), (2013, array(2, 2), map('2', 2), struct(2, 'b')) as yearsWithComplexTypes(y, a, m, s) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -46,27 +46,27 @@ PIVOT ( sum(earnings) FOR course IN ('dotNET', 'Java') ) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 2012 15000 20000 2013 48000 30000 --- !query 4 +-- !query SELECT * FROM courseSales PIVOT ( sum(earnings) FOR year IN (2012, 2013) ) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output Java 20000 30000 dotNET 15000 48000 --- !query 5 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -74,14 +74,14 @@ PIVOT ( sum(earnings), avg(earnings) FOR course IN ('dotNET', 'Java') ) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 2012 15000 7500.0 20000 20000.0 2013 48000 48000.0 30000 30000.0 --- !query 6 +-- !query SELECT * FROM ( SELECT course, earnings FROM courseSales ) @@ -89,13 +89,13 @@ PIVOT ( sum(earnings) FOR course IN ('dotNET', 'Java') ) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 63000 50000 --- !query 7 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -103,13 +103,13 @@ PIVOT ( sum(earnings), min(year) FOR course IN ('dotNET', 'Java') ) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 63000 2012 50000 2012 --- !query 8 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -119,16 +119,16 @@ PIVOT ( sum(earnings) FOR s IN (1, 2) ) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output Java 2012 20000 NULL Java 2013 NULL 30000 dotNET 2012 15000 NULL dotNET 2013 NULL 48000 --- !query 9 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -138,14 +138,14 @@ PIVOT ( sum(earnings), min(s) FOR course IN ('dotNET', 'Java') ) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 2012 15000 1 20000 1 2013 48000 2 30000 2 --- !query 10 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -155,14 +155,14 @@ PIVOT ( sum(earnings * s) FOR course IN ('dotNET', 'Java') ) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 2012 15000 20000 2013 96000 60000 --- !query 11 +-- !query SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( SELECT year y, course c, earnings e FROM courseSales ) @@ -170,14 +170,14 @@ PIVOT ( sum(e) s, avg(e) a FOR y IN (2012, 2013) ) --- !query 11 schema +-- !query schema struct<2012_s:bigint,2013_s:bigint,2012_a:double,2013_a:double,c:string> --- !query 11 output +-- !query output 15000 48000 7500.0 48000.0 dotNET 20000 30000 20000.0 30000.0 Java --- !query 12 +-- !query SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( SELECT year y, course c, earnings e FROM courseSales ) @@ -185,27 +185,27 @@ PIVOT ( sum(e) s, avg(e) a FOR y IN (2012 as firstYear, 2013 secondYear) ) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 15000 48000 7500.0 48000.0 dotNET 20000 30000 20000.0 30000.0 Java --- !query 13 +-- !query SELECT * FROM courseSales PIVOT ( abs(earnings) FOR year IN (2012, 2013) ) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate expression required for pivot, but 'coursesales.`earnings`' did not appear in any aggregate function.; --- !query 14 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -213,14 +213,14 @@ PIVOT ( sum(earnings), year FOR course IN ('dotNET', 'Java') ) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate expression required for pivot, but '__auto_generated_subquery_name.`year`' did not appear in any aggregate function.; --- !query 15 +-- !query SELECT * FROM ( SELECT course, earnings FROM courseSales ) @@ -228,14 +228,14 @@ PIVOT ( sum(earnings) FOR year IN (2012, 2013) ) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`year`' given input columns: [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 --- !query 16 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -243,14 +243,14 @@ PIVOT ( ceil(sum(earnings)), avg(earnings) + 1 as a1 FOR course IN ('dotNET', 'Java') ) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2012 15000 7501.0 20000 20001.0 2013 48000 48001.0 30000 30001.0 --- !query 17 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -258,14 +258,14 @@ PIVOT ( sum(avg(earnings)) FOR course IN ('dotNET', 'Java') ) --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query.; --- !query 18 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -275,14 +275,14 @@ PIVOT ( sum(earnings) FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) ) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 15000 NULL 2 NULL 30000 --- !query 19 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -292,14 +292,14 @@ PIVOT ( sum(earnings) FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) ) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 2012 NULL 20000 2013 48000 NULL --- !query 20 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -309,40 +309,40 @@ PIVOT ( sum(earnings) FOR (course, year) IN ('dotNET', 'Java') ) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.AnalysisException Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct; --- !query 21 +-- !query SELECT * FROM courseSales PIVOT ( sum(earnings) FOR year IN (s, 2013) ) --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`s`' given input columns: [coursesales.course, coursesales.earnings, coursesales.year]; line 4 pos 15 --- !query 22 +-- !query SELECT * FROM courseSales PIVOT ( sum(earnings) FOR year IN (course, 2013) ) --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException Literal expressions required for pivot values, found 'course#x'; --- !query 23 +-- !query SELECT * FROM ( SELECT course, year, a FROM courseSales @@ -352,14 +352,14 @@ PIVOT ( min(a) FOR course IN ('dotNET', 'Java') ) --- !query 23 schema +-- !query schema struct,Java:array> --- !query 23 output +-- !query output 2012 [1,1] [1,1] 2013 [2,2] [2,2] --- !query 24 +-- !query SELECT * FROM ( SELECT course, year, y, a FROM courseSales @@ -369,14 +369,14 @@ PIVOT ( max(a) FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) ) --- !query 24 schema +-- !query schema struct,[2013, Java]:array> --- !query 24 output +-- !query output 2012 [1,1] NULL 2013 NULL [2,2] --- !query 25 +-- !query SELECT * FROM ( SELECT earnings, year, a FROM courseSales @@ -386,14 +386,14 @@ PIVOT ( sum(earnings) FOR a IN (array(1, 1), array(2, 2)) ) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 2012 35000 NULL 2013 NULL 78000 --- !query 26 +-- !query SELECT * FROM ( SELECT course, earnings, year, a FROM courseSales @@ -403,14 +403,14 @@ PIVOT ( sum(earnings) FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) ) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 2012 15000 NULL 2013 NULL 30000 --- !query 27 +-- !query SELECT * FROM ( SELECT earnings, year, s FROM courseSales @@ -420,14 +420,14 @@ PIVOT ( sum(earnings) FOR s IN ((1, 'a'), (2, 'b')) ) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 2012 35000 NULL 2013 NULL 78000 --- !query 28 +-- !query SELECT * FROM ( SELECT course, earnings, year, s FROM courseSales @@ -437,14 +437,14 @@ PIVOT ( sum(earnings) FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) ) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 2012 15000 NULL 2013 NULL 30000 --- !query 29 +-- !query SELECT * FROM ( SELECT earnings, year, m FROM courseSales @@ -454,14 +454,14 @@ PIVOT ( sum(earnings) FOR m IN (map('1', 1), map('2', 2)) ) --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.AnalysisException Invalid pivot column 'm#x'. Pivot columns must be comparable.; --- !query 30 +-- !query SELECT * FROM ( SELECT course, earnings, year, m FROM courseSales @@ -471,14 +471,14 @@ PIVOT ( sum(earnings) FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) ) --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output org.apache.spark.sql.AnalysisException Invalid pivot column 'named_struct(course, course#x, m, m#x)'. Pivot columns must be comparable.; --- !query 31 +-- !query SELECT * FROM ( SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w FROM courseSales @@ -487,7 +487,7 @@ PIVOT ( sum(Earnings) FOR Course IN ('dotNET', 'Java') ) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output a z b y c x d w 63000 50000 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out index 29bafb42f579e..5efb58c7fc1b0 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -2,330 +2,330 @@ -- Number of queries: 44 --- !query 0 +-- !query SELECT avg(four) AS avg_1 FROM onek --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1.5 --- !query 1 +-- !query SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 32.666666666666664 --- !query 2 +-- !query select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 107.943 --- !query 3 +-- !query SELECT sum(four) AS sum_1500 FROM onek --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1500 --- !query 4 +-- !query SELECT sum(a) AS sum_198 FROM aggtest --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 198 --- !query 5 +-- !query SELECT sum(b) AS avg_431_773 FROM aggtest --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 431.77260909229517 --- !query 6 +-- !query SELECT max(four) AS max_3 FROM onek --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 3 --- !query 7 +-- !query SELECT max(a) AS max_100 FROM aggtest --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 100 --- !query 8 +-- !query SELECT max(aggtest.b) AS max_324_78 FROM aggtest --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 324.78 --- !query 9 +-- !query SELECT stddev_pop(b) FROM aggtest --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 131.10703231895047 --- !query 10 +-- !query SELECT stddev_samp(b) FROM aggtest --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 151.38936080399804 --- !query 11 +-- !query SELECT var_pop(b) FROM aggtest --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 17189.053923482323 --- !query 12 +-- !query SELECT var_samp(b) FROM aggtest --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 22918.738564643096 --- !query 13 +-- !query SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 131.18117242958306 --- !query 14 +-- !query SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 151.47497042966097 --- !query 15 +-- !query SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 17208.5 --- !query 16 +-- !query SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 22944.666666666668 --- !query 17 +-- !query SELECT var_pop(1.0), var_samp(2.0) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 0.0 NaN --- !query 18 +-- !query SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0))) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 0.0 NaN --- !query 19 +-- !query select sum(CAST(null AS int)) from range(1,4) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output NULL --- !query 20 +-- !query select sum(CAST(null AS long)) from range(1,4) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output NULL --- !query 21 +-- !query select sum(CAST(null AS Decimal(38,0))) from range(1,4) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL --- !query 22 +-- !query select sum(CAST(null AS DOUBLE)) from range(1,4) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL --- !query 23 +-- !query select avg(CAST(null AS int)) from range(1,4) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output NULL --- !query 24 +-- !query select avg(CAST(null AS long)) from range(1,4) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output NULL --- !query 25 +-- !query select avg(CAST(null AS Decimal(38,0))) from range(1,4) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output NULL --- !query 26 +-- !query select avg(CAST(null AS DOUBLE)) from range(1,4) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output NULL --- !query 27 +-- !query select sum(CAST('NaN' AS DOUBLE)) from range(1,4) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output NaN --- !query 28 +-- !query select avg(CAST('NaN' AS DOUBLE)) from range(1,4) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output NaN --- !query 29 +-- !query SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('infinity' AS DOUBLE))) v(x) --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output Infinity NaN --- !query 30 +-- !query SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES ('infinity'), ('1')) v(x) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output Infinity NaN --- !query 31 +-- !query SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES ('infinity'), ('infinity')) v(x) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output Infinity NaN --- !query 32 +-- !query SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES ('-infinity'), ('infinity')) v(x) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output NaN NaN --- !query 33 +-- !query SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 1.00000005E8 2.5 --- !query 34 +-- !query SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) FROM (VALUES (7000000000005), (7000000000007)) v(x) --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 7.000000000006E12 1.0 --- !query 35 +-- !query SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 653.6289553875104 871.5052738500139 --- !query 36 +-- !query SELECT corr(b, a) FROM aggtest --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 0.1396345165178734 --- !query 37 +-- !query SELECT count(four) AS cnt_1000 FROM onek --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 1000 --- !query 38 +-- !query SELECT count(DISTINCT four) AS cnt_4 FROM onek --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 4 --- !query 39 +-- !query select ten, count(*), sum(four) from onek group by ten order by ten --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 0 100 100 1 100 200 2 100 100 @@ -338,12 +338,12 @@ struct 9 100 200 --- !query 40 +-- !query select ten, count(four), sum(DISTINCT four) from onek group by ten order by ten --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 0 100 2 1 100 4 2 100 2 @@ -356,13 +356,13 @@ struct 9 100 4 --- !query 41 +-- !query select ten, sum(distinct four) from onek a group by ten having exists (select 1 from onek b where sum(distinct a.four) = b.four) --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 0 2 2 2 4 2 @@ -370,14 +370,14 @@ struct 8 2 --- !query 42 +-- !query select ten, sum(distinct four) from onek a group by ten having exists (select 1 from onek b where sum(distinct a.four + b.four) = b.four) --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -385,12 +385,12 @@ Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))]; --- !query 43 +-- !query select (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) from tenk1 o --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 63 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out index fdca51ff1325b..6633bf5d114ed 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part2.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 29 --- !query 0 +-- !query create temporary view int4_tbl as select * from values (0), (123456), @@ -10,40 +10,40 @@ create temporary view int4_tbl as select * from values (2147483647), (-2147483647) as int4_tbl(f1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES (1, 1, 1, 1L), (3, 3, 3, null), (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output NULL NULL --- !query 3 +-- !query SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output NULL NULL --- !query 4 +-- !query SELECT BIT_AND(cast(b1 as tinyint)) AS a1, BIT_AND(cast(b2 as smallint)) AS b1, @@ -54,52 +54,52 @@ SELECT BIT_OR(b3) AS g7, BIT_OR(b4) AS h3 FROM bitwise_test --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 1 1 1 7 7 7 3 --- !query 5 +-- !query SELECT b1 , bit_and(b2), bit_or(b4) FROM bitwise_test GROUP BY b1 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 1 1 3 3 NULL 7 7 3 --- !query 6 +-- !query SELECT b1, bit_and(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 1 3 3 --- !query 7 +-- !query SELECT b1, b2, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 1 1 3 3 3 7 7 7 --- !query 8 +-- !query SELECT b1, b2, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 1 1 3 3 3 7 7 7 --- !query 9 +-- !query SELECT (NULL AND NULL) IS NULL AS `t`, (TRUE AND NULL) IS NULL AS `t`, @@ -110,13 +110,13 @@ SELECT NOT (TRUE AND FALSE) AS `t`, NOT (FALSE AND TRUE) AS `t`, NOT (FALSE AND FALSE) AS `t` --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output true true false true false true true true true --- !query 10 +-- !query SELECT (NULL OR NULL) IS NULL AS `t`, (TRUE OR NULL) IS NULL AS `t`, @@ -127,32 +127,32 @@ SELECT (TRUE OR FALSE) AS `t`, (FALSE OR TRUE) AS `t`, NOT (FALSE OR FALSE) AS `t` --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output true false true false true true true true true --- !query 11 +-- !query CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES (TRUE, null, FALSE, null), (FALSE, TRUE, null, null), (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output NULL NULL --- !query 13 +-- !query SELECT BOOL_AND(b1) AS f1, BOOL_AND(b2) AS t2, @@ -161,13 +161,13 @@ SELECT BOOL_AND(NOT b2) AS f5, BOOL_AND(NOT b3) AS t6 FROM bool_test --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output false true false NULL false true --- !query 14 +-- !query SELECT EVERY(b1) AS f1, EVERY(b2) AS t2, @@ -176,13 +176,13 @@ SELECT EVERY(NOT b2) AS f5, EVERY(NOT b3) AS t6 FROM bool_test --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output false true false NULL false true --- !query 15 +-- !query SELECT BOOL_OR(b1) AS t1, BOOL_OR(b2) AS t2, @@ -191,113 +191,113 @@ SELECT BOOL_OR(NOT b2) AS f5, BOOL_OR(NOT b3) AS t6 FROM bool_test --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output true true false NULL false true --- !query 16 +-- !query select min(unique1) from tenk1 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 0 --- !query 17 +-- !query select max(unique1) from tenk1 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 9999 --- !query 18 +-- !query select max(unique1) from tenk1 where unique1 < 42 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 41 --- !query 19 +-- !query select max(unique1) from tenk1 where unique1 > 42 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 9999 --- !query 20 +-- !query select max(unique1) from tenk1 where unique1 > 42000 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output NULL --- !query 21 +-- !query select max(tenthous) from tenk1 where thousand = 33 --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 9033 --- !query 22 +-- !query select min(tenthous) from tenk1 where thousand = 33 --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 33 --- !query 23 +-- !query select distinct max(unique2) from tenk1 --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 9999 --- !query 24 +-- !query select max(unique2) from tenk1 order by 1 --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 9999 --- !query 25 +-- !query select max(unique2) from tenk1 order by max(unique2) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 9999 --- !query 26 +-- !query select max(unique2) from tenk1 order by max(unique2)+1 --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 9999 --- !query 27 +-- !query select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 9999 3 9999 2 9999 1 --- !query 28 +-- !query select max(100) from tenk1 --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 100 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out index d2ab138efcdae..69f96b02782e3 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out @@ -2,37 +2,37 @@ -- Number of queries: 4 --- !query 0 +-- !query select max(min(unique1)) from tenk1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output org.apache.spark.sql.AnalysisException It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query.; --- !query 1 +-- !query select min(unique1) filter (where unique1 > 100) from tenk1 --- !query 1 schema +-- !query schema struct 100)):int> --- !query 1 output +-- !query output 101 --- !query 2 +-- !query select sum(1/ten) filter (where ten > 0) from tenk1 --- !query 2 schema +-- !query schema struct 0)):double> --- !query 2 output +-- !query output 2828.9682539682954 --- !query 3 +-- !query select (select count(*) from (values (1)) t0(inner_c)) from (values (2),(3)) t1(outer_c) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 1 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out index 39f239d7dbf2e..0347e0dc7853b 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out @@ -2,475 +2,475 @@ -- Number of queries: 92 --- !query 0 +-- !query SELECT 1 AS one --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1 --- !query 1 +-- !query SELECT true AS true --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output true --- !query 2 +-- !query SELECT false AS `false` --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output false --- !query 3 +-- !query SELECT boolean('t') AS true --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output true --- !query 4 +-- !query SELECT boolean(' f ') AS `false` --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output false --- !query 5 +-- !query SELECT boolean('true') AS true --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output true --- !query 6 +-- !query SELECT boolean('test') AS error --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL --- !query 7 +-- !query SELECT boolean('false') AS `false` --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output false --- !query 8 +-- !query SELECT boolean('foo') AS error --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NULL --- !query 9 +-- !query SELECT boolean('y') AS true --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output true --- !query 10 +-- !query SELECT boolean('yes') AS true --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output true --- !query 11 +-- !query SELECT boolean('yeah') AS error --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output NULL --- !query 12 +-- !query SELECT boolean('n') AS `false` --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output false --- !query 13 +-- !query SELECT boolean('no') AS `false` --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output false --- !query 14 +-- !query SELECT boolean('nay') AS error --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output NULL --- !query 15 +-- !query SELECT boolean('on') AS true --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output NULL --- !query 16 +-- !query SELECT boolean('off') AS `false` --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output NULL --- !query 17 +-- !query SELECT boolean('of') AS `false` --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output NULL --- !query 18 +-- !query SELECT boolean('o') AS error --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output NULL --- !query 19 +-- !query SELECT boolean('on_') AS error --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output NULL --- !query 20 +-- !query SELECT boolean('off_') AS error --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output NULL --- !query 21 +-- !query SELECT boolean('1') AS true --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output true --- !query 22 +-- !query SELECT boolean('11') AS error --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL --- !query 23 +-- !query SELECT boolean('0') AS `false` --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output false --- !query 24 +-- !query SELECT boolean('000') AS error --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output NULL --- !query 25 +-- !query SELECT boolean('') AS error --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output NULL --- !query 26 +-- !query SELECT boolean('t') or boolean('f') AS true --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output true --- !query 27 +-- !query SELECT boolean('t') and boolean('f') AS `false` --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output false --- !query 28 +-- !query SELECT not boolean('f') AS true --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output true --- !query 29 +-- !query SELECT boolean('t') = boolean('f') AS `false` --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output false --- !query 30 +-- !query SELECT boolean('t') <> boolean('f') AS true --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output true --- !query 31 +-- !query SELECT boolean('t') > boolean('f') AS true --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output true --- !query 32 +-- !query SELECT boolean('t') >= boolean('f') AS true --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output true --- !query 33 +-- !query SELECT boolean('f') < boolean('t') AS true --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output true --- !query 34 +-- !query SELECT boolean('f') <= boolean('t') AS true --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output true --- !query 35 +-- !query SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false` --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output true false --- !query 36 +-- !query SELECT boolean(string(' true ')) AS true, boolean(string(' FALSE')) AS `false` --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output true false --- !query 37 +-- !query SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false` --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output true false --- !query 38 +-- !query SELECT boolean(string(' tru e ')) AS invalid --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output NULL --- !query 39 +-- !query SELECT boolean(string('')) AS invalid --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output NULL --- !query 40 +-- !query CREATE TABLE BOOLTBL1 (f1 boolean) USING parquet --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output --- !query 41 +-- !query INSERT INTO BOOLTBL1 VALUES (cast('t' as boolean)) --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output --- !query 42 +-- !query INSERT INTO BOOLTBL1 VALUES (cast('True' as boolean)) --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output --- !query 43 +-- !query INSERT INTO BOOLTBL1 VALUES (cast('true' as boolean)) --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output --- !query 44 +-- !query SELECT '' AS t_3, BOOLTBL1.* FROM BOOLTBL1 --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output true true true --- !query 45 +-- !query SELECT '' AS t_3, BOOLTBL1.* FROM BOOLTBL1 WHERE f1 = boolean('true') --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output true true true --- !query 46 +-- !query SELECT '' AS t_3, BOOLTBL1.* FROM BOOLTBL1 WHERE f1 <> boolean('false') --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output true true true --- !query 47 +-- !query SELECT '' AS zero, BOOLTBL1.* FROM BOOLTBL1 WHERE booleq(boolean('false'), f1) --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output --- !query 48 +-- !query INSERT INTO BOOLTBL1 VALUES (boolean('f')) --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output --- !query 49 +-- !query SELECT '' AS f_1, BOOLTBL1.* FROM BOOLTBL1 WHERE f1 = boolean('false') --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output false --- !query 50 +-- !query CREATE TABLE BOOLTBL2 (f1 boolean) USING parquet --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output --- !query 51 +-- !query INSERT INTO BOOLTBL2 VALUES (boolean('f')) --- !query 51 schema +-- !query schema struct<> --- !query 51 output +-- !query output --- !query 52 +-- !query INSERT INTO BOOLTBL2 VALUES (boolean('false')) --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output --- !query 53 +-- !query INSERT INTO BOOLTBL2 VALUES (boolean('False')) --- !query 53 schema +-- !query schema struct<> --- !query 53 output +-- !query output --- !query 54 +-- !query INSERT INTO BOOLTBL2 VALUES (boolean('FALSE')) --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output --- !query 55 +-- !query INSERT INTO BOOLTBL2 VALUES (boolean('XXX')) --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output --- !query 56 +-- !query SELECT '' AS f_4, BOOLTBL2.* FROM BOOLTBL2 --- !query 56 schema +-- !query schema struct --- !query 56 output +-- !query output NULL false false @@ -478,13 +478,13 @@ struct false --- !query 57 +-- !query SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* FROM BOOLTBL1, BOOLTBL2 WHERE BOOLTBL2.f1 <> BOOLTBL1.f1 --- !query 57 schema +-- !query schema struct --- !query 57 output +-- !query output true false true false true false @@ -499,13 +499,13 @@ struct true false --- !query 58 +-- !query SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* FROM BOOLTBL1, BOOLTBL2 WHERE boolne(BOOLTBL2.f1,BOOLTBL1.f1) --- !query 58 schema +-- !query schema struct --- !query 58 output +-- !query output true false true false true false @@ -520,27 +520,27 @@ struct true false --- !query 59 +-- !query SELECT '' AS ff_4, BOOLTBL1.*, BOOLTBL2.* FROM BOOLTBL1, BOOLTBL2 WHERE BOOLTBL2.f1 = BOOLTBL1.f1 and BOOLTBL1.f1 = boolean('false') --- !query 59 schema +-- !query schema struct --- !query 59 output +-- !query output false false false false false false false false --- !query 60 +-- !query SELECT '' AS tf_12_ff_4, BOOLTBL1.*, BOOLTBL2.* FROM BOOLTBL1, BOOLTBL2 WHERE BOOLTBL2.f1 = BOOLTBL1.f1 or BOOLTBL1.f1 = boolean('true') ORDER BY BOOLTBL1.f1, BOOLTBL2.f1 --- !query 60 schema +-- !query schema struct --- !query 60 output +-- !query output false false false false false false @@ -562,90 +562,90 @@ struct true false --- !query 61 +-- !query SELECT '' AS True, f1 FROM BOOLTBL1 WHERE f1 IS TRUE --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output true true true --- !query 62 +-- !query SELECT '' AS `Not False`, f1 FROM BOOLTBL1 WHERE f1 IS NOT FALSE --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output true true true --- !query 63 +-- !query SELECT '' AS `False`, f1 FROM BOOLTBL1 WHERE f1 IS FALSE --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output false --- !query 64 +-- !query SELECT '' AS `Not True`, f1 FROM BOOLTBL1 WHERE f1 IS NOT TRUE --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output false --- !query 65 +-- !query SELECT '' AS `True`, f1 FROM BOOLTBL2 WHERE f1 IS TRUE --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output --- !query 66 +-- !query SELECT '' AS `Not False`, f1 FROM BOOLTBL2 WHERE f1 IS NOT FALSE --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output NULL --- !query 67 +-- !query SELECT '' AS `False`, f1 FROM BOOLTBL2 WHERE f1 IS FALSE --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output false false false false --- !query 68 +-- !query SELECT '' AS `Not True`, f1 FROM BOOLTBL2 WHERE f1 IS NOT TRUE --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output NULL false false @@ -653,39 +653,39 @@ struct false --- !query 69 +-- !query CREATE TABLE BOOLTBL3 (d string, b boolean, o int) USING parquet --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output --- !query 70 +-- !query INSERT INTO BOOLTBL3 VALUES ('true', true, 1) --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output --- !query 71 +-- !query INSERT INTO BOOLTBL3 VALUES ('false', false, 2) --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output --- !query 72 +-- !query INSERT INTO BOOLTBL3 VALUES ('null', null, 3) --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output --- !query 73 +-- !query SELECT d, b IS TRUE AS istrue, @@ -695,153 +695,153 @@ SELECT b IS UNKNOWN AS isunknown, b IS NOT UNKNOWN AS isnotunknown FROM booltbl3 ORDER BY o --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output true true false false true false true false false true true false false true null false true false true true false --- !query 74 +-- !query CREATE TABLE booltbl4(isfalse boolean, istrue boolean, isnul boolean) USING parquet --- !query 74 schema +-- !query schema struct<> --- !query 74 output +-- !query output --- !query 75 +-- !query INSERT INTO booltbl4 VALUES (false, true, null) --- !query 75 schema +-- !query schema struct<> --- !query 75 output +-- !query output --- !query 76 +-- !query SELECT istrue AND isnul AND istrue FROM booltbl4 --- !query 76 schema +-- !query schema struct<((istrue AND isnul) AND istrue):boolean> --- !query 76 output +-- !query output NULL --- !query 77 +-- !query SELECT istrue AND istrue AND isnul FROM booltbl4 --- !query 77 schema +-- !query schema struct<((istrue AND istrue) AND isnul):boolean> --- !query 77 output +-- !query output NULL --- !query 78 +-- !query SELECT isnul AND istrue AND istrue FROM booltbl4 --- !query 78 schema +-- !query schema struct<((isnul AND istrue) AND istrue):boolean> --- !query 78 output +-- !query output NULL --- !query 79 +-- !query SELECT isfalse AND isnul AND istrue FROM booltbl4 --- !query 79 schema +-- !query schema struct<((isfalse AND isnul) AND istrue):boolean> --- !query 79 output +-- !query output false --- !query 80 +-- !query SELECT istrue AND isfalse AND isnul FROM booltbl4 --- !query 80 schema +-- !query schema struct<((istrue AND isfalse) AND isnul):boolean> --- !query 80 output +-- !query output false --- !query 81 +-- !query SELECT isnul AND istrue AND isfalse FROM booltbl4 --- !query 81 schema +-- !query schema struct<((isnul AND istrue) AND isfalse):boolean> --- !query 81 output +-- !query output false --- !query 82 +-- !query SELECT isfalse OR isnul OR isfalse FROM booltbl4 --- !query 82 schema +-- !query schema struct<((isfalse OR isnul) OR isfalse):boolean> --- !query 82 output +-- !query output NULL --- !query 83 +-- !query SELECT isfalse OR isfalse OR isnul FROM booltbl4 --- !query 83 schema +-- !query schema struct<((isfalse OR isfalse) OR isnul):boolean> --- !query 83 output +-- !query output NULL --- !query 84 +-- !query SELECT isnul OR isfalse OR isfalse FROM booltbl4 --- !query 84 schema +-- !query schema struct<((isnul OR isfalse) OR isfalse):boolean> --- !query 84 output +-- !query output NULL --- !query 85 +-- !query SELECT isfalse OR isnul OR istrue FROM booltbl4 --- !query 85 schema +-- !query schema struct<((isfalse OR isnul) OR istrue):boolean> --- !query 85 output +-- !query output true --- !query 86 +-- !query SELECT istrue OR isfalse OR isnul FROM booltbl4 --- !query 86 schema +-- !query schema struct<((istrue OR isfalse) OR isnul):boolean> --- !query 86 output +-- !query output true --- !query 87 +-- !query SELECT isnul OR istrue OR isfalse FROM booltbl4 --- !query 87 schema +-- !query schema struct<((isnul OR istrue) OR isfalse):boolean> --- !query 87 output +-- !query output true --- !query 88 +-- !query DROP TABLE BOOLTBL1 --- !query 88 schema +-- !query schema struct<> --- !query 88 output +-- !query output --- !query 89 +-- !query DROP TABLE BOOLTBL2 --- !query 89 schema +-- !query schema struct<> --- !query 89 output +-- !query output --- !query 90 +-- !query DROP TABLE BOOLTBL3 --- !query 90 schema +-- !query schema struct<> --- !query 90 output +-- !query output --- !query 91 +-- !query DROP TABLE BOOLTBL4 --- !query 91 schema +-- !query schema struct<> --- !query 91 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out index a3410684e85dd..1b002c3f48ae2 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out @@ -2,243 +2,243 @@ -- Number of queries: 35 --- !query 0 +-- !query CREATE TABLE CASE_TBL ( i integer, f double ) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TABLE CASE2_TBL ( i integer, j integer ) USING parquet --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO CASE_TBL VALUES (1, 10.1) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO CASE_TBL VALUES (2, 20.2) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO CASE_TBL VALUES (3, -30.3) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO CASE_TBL VALUES (4, NULL) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO CASE2_TBL VALUES (1, -1) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO CASE2_TBL VALUES (2, -2) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO CASE2_TBL VALUES (3, -3) --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO CASE2_TBL VALUES (2, -4) --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO CASE2_TBL VALUES (1, NULL) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query INSERT INTO CASE2_TBL VALUES (NULL, -6) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query SELECT '3' AS `One`, CASE WHEN 1 < 2 THEN 3 END AS `Simple WHEN` --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 3 3 --- !query 13 +-- !query SELECT '' AS `One`, CASE WHEN 1 > 2 THEN 3 END AS `Simple default` --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output NULL --- !query 14 +-- !query SELECT '3' AS `One`, CASE WHEN 1 < 2 THEN 3 ELSE 4 END AS `Simple ELSE` --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 3 3 --- !query 15 +-- !query SELECT '4' AS `One`, CASE WHEN 1 > 2 THEN 3 ELSE 4 END AS `ELSE default` --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 4 4 --- !query 16 +-- !query SELECT '6' AS `One`, CASE WHEN 1 > 2 THEN 3 WHEN 4 < 5 THEN 6 ELSE 7 END AS `Two WHEN with default` --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 6 6 --- !query 17 +-- !query SELECT '7' AS `None`, CASE WHEN rand() < 0 THEN 1 END AS `NULL on no matches` --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 7 NULL --- !query 18 +-- !query SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1.0 --- !query 19 +-- !query SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1.0 --- !query 20 +-- !query SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl --- !query 20 schema +-- !query schema struct 100) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) ELSE CAST(0 AS DOUBLE) END:double> --- !query 20 output +-- !query output 0.0 0.0 0.0 0.0 --- !query 21 +-- !query SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 --- !query 22 +-- !query SELECT '' AS `Five`, CASE WHEN i >= 3 THEN i END AS `>= 3 or Null` FROM CASE_TBL --- !query 22 schema +-- !query schema struct= 3 or Null:int> --- !query 22 output +-- !query output 3 4 NULL NULL --- !query 23 +-- !query SELECT '' AS `Five`, CASE WHEN i >= 3 THEN (i + i) ELSE i END AS `Simplest Math` FROM CASE_TBL --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 1 2 6 8 --- !query 24 +-- !query SELECT '' AS `Five`, i AS `Value`, CASE WHEN (i < 0) THEN 'small' WHEN (i = 0) THEN 'zero' @@ -247,16 +247,16 @@ SELECT '' AS `Five`, i AS `Value`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1 one 2 two 3 big 4 big --- !query 25 +-- !query SELECT '' AS `Five`, CASE WHEN ((i < 0) or (i < 0)) THEN 'small' WHEN ((i = 0) or (i = 0)) THEN 'zero' @@ -265,37 +265,37 @@ SELECT '' AS `Five`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output big big one two --- !query 26 +-- !query SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4 --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 4 NULL --- !query 27 +-- !query SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2 --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output --- !query 28 +-- !query SELECT COALESCE(a.f, b.i, b.j) FROM CASE_TBL a, CASE2_TBL b --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output -30.3 -30.3 -30.3 @@ -322,24 +322,24 @@ struct 3.0 --- !query 29 +-- !query SELECT * FROM CASE_TBL a, CASE2_TBL b WHERE COALESCE(a.f, b.i, b.j) = 2 --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 4 NULL 2 -2 4 NULL 2 -4 --- !query 30 +-- !query SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, NULLIF(b.i, 4) AS `NULLIF(b.i,4)` FROM CASE_TBL a, CASE2_TBL b --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1 2 1 2 1 3 @@ -366,18 +366,18 @@ struct NULL 3 --- !query 31 +-- !query SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b WHERE COALESCE(f,b.i) = 2 --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 4 NULL 2 -2 4 NULL 2 -4 --- !query 32 +-- !query SELECT CASE (CASE vol('bar') WHEN 'foo' THEN 'it was foo!' @@ -387,23 +387,23 @@ SELECT CASE WHEN 'it was foo!' THEN 'foo recognized' WHEN 'it was bar!' THEN 'bar recognized' ELSE 'unrecognized' END --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output bar recognized --- !query 33 +-- !query DROP TABLE CASE_TBL --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output --- !query 34 +-- !query DROP TABLE CASE2_TBL --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out index bb44931a5ca8a..4ea49013a62d1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out @@ -2,44 +2,44 @@ -- Number of queries: 13 --- !query 0 +-- !query SELECT 'trailing' AS first --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output trailing --- !query 1 +-- !query SELECT /* embedded single line */ 'embedded' AS `second` --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output embedded --- !query 2 +-- !query SELECT /* both embedded and trailing single line */ 'both' AS third --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output both --- !query 3 +-- !query SELECT 'before multi-line' AS fourth --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output before multi-line --- !query 4 +-- !query /* This is an example of SQL which should not execute: * select 'multi-line' --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) @@ -50,12 +50,12 @@ mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR' * select 'multi-line' --- !query 5 +-- !query */ SELECT 'after multi-line' AS fifth --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) @@ -66,12 +66,12 @@ extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR SELECT 'after multi-line' AS fifth --- !query 6 +-- !query /* SELECT 'trailing' as x1 --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) @@ -82,14 +82,14 @@ mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR' SELECT 'trailing' as x1 --- !query 7 +-- !query */ /* This block comment surrounds a query which itself has a block comment... SELECT /* embedded single line */ 'embedded' AS x2 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) @@ -102,16 +102,16 @@ mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR SELECT /* embedded single line */ 'embedded' AS x2 --- !query 8 +-- !query */ SELECT -- continued after the following block comments... /* Deeply nested comment. This includes a single apostrophe to make sure we aren't decoding this part as a string. SELECT 'deep nest' AS n1 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) @@ -126,12 +126,12 @@ SELECT -- continued after the following block comments... SELECT 'deep nest' AS n1 --- !query 9 +-- !query /* Second level of nesting... SELECT 'deeper nest' as n2 --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) @@ -142,12 +142,12 @@ mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR' SELECT 'deeper nest' as n2 --- !query 10 +-- !query /* Third level of nesting... SELECT 'deepest nest' as n3 --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) @@ -158,16 +158,16 @@ mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR' SELECT 'deepest nest' as n3 --- !query 11 +-- !query */ Hoo boy. Still two deep... */ Now just one deep... */ 'deeply nested example' AS sixth --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) @@ -182,11 +182,11 @@ Now just one deep... 'deeply nested example' AS sixth --- !query 12 +-- !query /* and this is the end of the file */ --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input '' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 37) diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out index 7983a254054c5..436b33ce43980 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out @@ -2,69 +2,69 @@ -- Number of queries: 195 --- !query 0 +-- !query CREATE TABLE emp ( name string, age int, salary int, manager string ) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE VIEW toyemp AS SELECT name, age, /* location ,*/ 12*salary AS annualsal FROM emp --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query DROP VIEW toyemp --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query DROP TABLE emp --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE VIEW key_dependent_view AS SELECT * FROM view_base_table GROUP BY key --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException expression 'default.view_base_table.`data`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 6 +-- !query CREATE VIEW key_dependent_view_no_cols AS SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0 --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'FROM'(line 2, pos 10) @@ -75,174 +75,174 @@ CREATE VIEW key_dependent_view_no_cols AS ----------^^^ --- !query 7 +-- !query CREATE TABLE viewtest_tbl (a int, b int) using parquet --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO viewtest_tbl VALUES (5, 10), (10, 15), (15, 20), (20, 25) --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query CREATE OR REPLACE VIEW viewtest AS SELECT * FROM viewtest_tbl --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query CREATE OR REPLACE VIEW viewtest AS SELECT * FROM viewtest_tbl WHERE a > 10 --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT * FROM viewtest --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 15 20 20 25 --- !query 12 +-- !query CREATE OR REPLACE VIEW viewtest AS SELECT a, b FROM viewtest_tbl WHERE a > 5 ORDER BY b DESC --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query SELECT * FROM viewtest --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 20 25 15 20 10 15 --- !query 14 +-- !query CREATE OR REPLACE VIEW viewtest AS SELECT a FROM viewtest_tbl WHERE a <> 20 --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query CREATE OR REPLACE VIEW viewtest AS SELECT 1, * FROM viewtest_tbl --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query CREATE OR REPLACE VIEW viewtest AS SELECT a, decimal(b) FROM viewtest_tbl --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output --- !query 17 +-- !query CREATE OR REPLACE VIEW viewtest AS SELECT a, b, 0 AS c FROM viewtest_tbl --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output --- !query 18 +-- !query DROP VIEW viewtest --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query DROP TABLE viewtest_tbl --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query CREATE SCHEMA temp_view_test --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query CREATE TABLE temp_view_test.base_table (a int, id int) using parquet --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query CREATE TABLE temp_view_test.base_table2 (a int, id int) using parquet --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output --- !query 23 +-- !query USE temp_view_test --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query CREATE TEMPORARY VIEW temp_table AS SELECT * FROM VALUES (1, 1) as temp_table(a, id) --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query CREATE VIEW v1 AS SELECT * FROM base_table --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query DESC TABLE EXTENDED v1 --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output a int id int @@ -260,45 +260,45 @@ View Query Output Columns [a, id] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=2, view.query.out.col.1=id, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=temp_view_test] --- !query 27 +-- !query CREATE VIEW v1_temp AS SELECT * FROM temp_table --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `v1_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW; --- !query 28 +-- !query CREATE TEMP VIEW v2_temp AS SELECT * FROM base_table --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output --- !query 29 +-- !query DESC TABLE EXTENDED v2_temp --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output a int id int --- !query 30 +-- !query CREATE VIEW temp_view_test.v2 AS SELECT * FROM base_table --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output --- !query 31 +-- !query DESC TABLE EXTENDED temp_view_test.v2 --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output a int id int @@ -316,31 +316,31 @@ View Query Output Columns [a, id] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=2, view.query.out.col.1=id, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=temp_view_test] --- !query 32 +-- !query CREATE VIEW temp_view_test.v3_temp AS SELECT * FROM temp_table --- !query 32 schema +-- !query schema struct<> --- !query 32 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `temp_view_test`.`v3_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW; --- !query 33 +-- !query CREATE VIEW v3 AS SELECT t1.a AS t1_a, t2.a AS t2_a FROM base_table t1, base_table2 t2 WHERE t1.id = t2.id --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output --- !query 34 +-- !query DESC TABLE EXTENDED v3 --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output t1_a int t2_a int @@ -362,43 +362,43 @@ View Query Output Columns [t1_a, t2_a] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=t1_a, view.query.out.numCols=2, view.query.out.col.1=t2_a, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=temp_view_test] --- !query 35 +-- !query CREATE VIEW v4_temp AS SELECT t1.a AS t1_a, t2.a AS t2_a FROM base_table t1, temp_table t2 WHERE t1.id = t2.id --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `v4_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW; --- !query 36 +-- !query CREATE VIEW v5_temp AS SELECT t1.a AS t1_a, t2.a AS t2_a, t3.a AS t3_a FROM base_table t1, base_table2 t2, temp_table t3 WHERE t1.id = t2.id and t2.id = t3.id --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `v5_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW; --- !query 37 +-- !query CREATE VIEW v4 AS SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2) --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output --- !query 38 +-- !query DESC TABLE EXTENDED v4 --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output a int id int @@ -416,19 +416,19 @@ View Query Output Columns [a, id] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=2, view.query.out.col.1=id, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=temp_view_test] --- !query 39 +-- !query CREATE VIEW v5 AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2 --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output --- !query 40 +-- !query DESC TABLE EXTENDED v5 --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output id int a int @@ -446,19 +446,19 @@ View Query Output Columns [id, a] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=id, view.query.out.numCols=2, view.query.out.col.1=a, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=temp_view_test] --- !query 41 +-- !query CREATE VIEW v6 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2) --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output --- !query 42 +-- !query DESC TABLE EXTENDED v6 --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output a int id int @@ -476,19 +476,19 @@ View Query Output Columns [a, id] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=2, view.query.out.col.1=id, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=temp_view_test] --- !query 43 +-- !query CREATE VIEW v7 AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2) --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output --- !query 44 +-- !query DESC TABLE EXTENDED v7 --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output a int id int @@ -506,19 +506,19 @@ View Query Output Columns [a, id] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=2, view.query.out.col.1=id, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=temp_view_test] --- !query 45 +-- !query CREATE VIEW v8 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1) --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output --- !query 46 +-- !query DESC TABLE EXTENDED v8 --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output a int id int @@ -536,123 +536,123 @@ View Query Output Columns [a, id] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=2, view.query.out.col.1=id, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=temp_view_test] --- !query 47 +-- !query CREATE VIEW v6_temp AS SELECT * FROM base_table WHERE id IN (SELECT id FROM temp_table) --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `v6_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW; --- !query 48 +-- !query CREATE VIEW v7_temp AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM temp_table) t2 --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `v7_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW; --- !query 49 +-- !query CREATE VIEW v8_temp AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM temp_table) --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `v8_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW; --- !query 50 +-- !query CREATE VIEW v9_temp AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM temp_table) --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `v9_temp` by referencing a temporary view temp_table. Please create a temp view instead by CREATE TEMP VIEW; --- !query 51 +-- !query CREATE VIEW v10_temp AS SELECT * FROM v7_temp --- !query 51 schema +-- !query schema struct<> --- !query 51 output +-- !query output org.apache.spark.sql.AnalysisException Table or view not found: v7_temp; line 1 pos 38 --- !query 52 +-- !query CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2 --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output org.apache.spark.sql.AnalysisException Table or view not found: v10_temp; line 1 pos 63 --- !query 53 +-- !query CREATE VIEW v12_temp AS SELECT true FROM v11_temp --- !query 53 schema +-- !query schema struct<> --- !query 53 output +-- !query output org.apache.spark.sql.AnalysisException Table or view not found: v11_temp; line 1 pos 41 --- !query 54 +-- !query CREATE SCHEMA testviewschm2 --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output --- !query 55 +-- !query USE testviewschm2 --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output --- !query 56 +-- !query CREATE TABLE t1 (num int, name string) using parquet --- !query 56 schema +-- !query schema struct<> --- !query 56 output +-- !query output --- !query 57 +-- !query CREATE TABLE t2 (num2 int, value string) using parquet --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output --- !query 58 +-- !query CREATE TEMP VIEW tt AS SELECT * FROM VALUES (1, 'a') AS tt(num2, value) --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output --- !query 59 +-- !query CREATE VIEW nontemp1 AS SELECT * FROM t1 CROSS JOIN t2 --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output --- !query 60 +-- !query DESC TABLE EXTENDED nontemp1 --- !query 60 schema +-- !query schema struct --- !query 60 output +-- !query output num int name string num2 int @@ -672,28 +672,28 @@ View Query Output Columns [num, name, num2, value] Table Properties [view.query.out.col.3=value, view.catalogAndNamespace.numParts=2, view.query.out.col.0=num, view.query.out.numCols=4, view.query.out.col.1=name, view.catalogAndNamespace.part.0=spark_catalog, view.query.out.col.2=num2, view.catalogAndNamespace.part.1=testviewschm2] --- !query 61 +-- !query CREATE VIEW temporal1 AS SELECT * FROM t1 CROSS JOIN tt --- !query 61 schema +-- !query schema struct<> --- !query 61 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `temporal1` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW; --- !query 62 +-- !query CREATE VIEW nontemp2 AS SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2 --- !query 62 schema +-- !query schema struct<> --- !query 62 output +-- !query output --- !query 63 +-- !query DESC TABLE EXTENDED nontemp2 --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output num int name string num2 int @@ -713,28 +713,28 @@ View Query Output Columns [num, name, num2, value] Table Properties [view.query.out.col.3=value, view.catalogAndNamespace.numParts=2, view.query.out.col.0=num, view.query.out.numCols=4, view.query.out.col.1=name, view.catalogAndNamespace.part.0=spark_catalog, view.query.out.col.2=num2, view.catalogAndNamespace.part.1=testviewschm2] --- !query 64 +-- !query CREATE VIEW temporal2 AS SELECT * FROM t1 INNER JOIN tt ON t1.num = tt.num2 --- !query 64 schema +-- !query schema struct<> --- !query 64 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `temporal2` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW; --- !query 65 +-- !query CREATE VIEW nontemp3 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 --- !query 65 schema +-- !query schema struct<> --- !query 65 output +-- !query output --- !query 66 +-- !query DESC TABLE EXTENDED nontemp3 --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output num int name string num2 int @@ -754,28 +754,28 @@ View Query Output Columns [num, name, num2, value] Table Properties [view.query.out.col.3=value, view.catalogAndNamespace.numParts=2, view.query.out.col.0=num, view.query.out.numCols=4, view.query.out.col.1=name, view.catalogAndNamespace.part.0=spark_catalog, view.query.out.col.2=num2, view.catalogAndNamespace.part.1=testviewschm2] --- !query 67 +-- !query CREATE VIEW temporal3 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 --- !query 67 schema +-- !query schema struct<> --- !query 67 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `temporal3` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW; --- !query 68 +-- !query CREATE VIEW nontemp4 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx' --- !query 68 schema +-- !query schema struct<> --- !query 68 output +-- !query output --- !query 69 +-- !query DESC TABLE EXTENDED nontemp4 --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output num int name string num2 int @@ -795,87 +795,87 @@ View Query Output Columns [num, name, num2, value] Table Properties [view.query.out.col.3=value, view.catalogAndNamespace.numParts=2, view.query.out.col.0=num, view.query.out.numCols=4, view.query.out.col.1=name, view.catalogAndNamespace.part.0=spark_catalog, view.query.out.col.2=num2, view.catalogAndNamespace.part.1=testviewschm2] --- !query 70 +-- !query CREATE VIEW temporal4 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 AND tt.value = 'xxx' --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `temporal4` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW; --- !query 71 +-- !query CREATE VIEW temporal5 AS SELECT * FROM t1 WHERE num IN (SELECT num FROM t1 WHERE EXISTS (SELECT 1 FROM tt)) --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output org.apache.spark.sql.AnalysisException Not allowed to create a permanent view `temporal5` by referencing a temporary view tt. Please create a temp view instead by CREATE TEMP VIEW; --- !query 72 +-- !query CREATE TABLE tbl1 ( a int, b int) using parquet --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output --- !query 73 +-- !query CREATE TABLE tbl2 (c int, d int) using parquet --- !query 73 schema +-- !query schema struct<> --- !query 73 output +-- !query output --- !query 74 +-- !query CREATE TABLE tbl3 (e int, f int) using parquet --- !query 74 schema +-- !query schema struct<> --- !query 74 output +-- !query output --- !query 75 +-- !query CREATE TABLE tbl4 (g int, h int) using parquet --- !query 75 schema +-- !query schema struct<> --- !query 75 output +-- !query output --- !query 76 +-- !query CREATE TABLE tmptbl (i int, j int) using parquet --- !query 76 schema +-- !query schema struct<> --- !query 76 output +-- !query output --- !query 77 +-- !query INSERT INTO tmptbl VALUES (1, 1) --- !query 77 schema +-- !query schema struct<> --- !query 77 output +-- !query output --- !query 78 +-- !query CREATE VIEW pubview AS SELECT * FROM tbl1 WHERE tbl1.a BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) --- !query 78 schema +-- !query schema struct<> --- !query 78 output +-- !query output --- !query 79 +-- !query DESC TABLE EXTENDED pubview --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output a int b int @@ -897,22 +897,22 @@ View Query Output Columns [a, b] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=2, view.query.out.col.1=b, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=testviewschm2] --- !query 80 +-- !query CREATE VIEW mytempview AS SELECT * FROM tbl1 WHERE tbl1.a BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j) --- !query 80 schema +-- !query schema struct<> --- !query 80 output +-- !query output --- !query 81 +-- !query DESC TABLE EXTENDED mytempview --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output a int b int @@ -936,403 +936,403 @@ View Query Output Columns [a, b] Table Properties [view.catalogAndNamespace.numParts=2, view.query.out.col.0=a, view.query.out.numCols=2, view.query.out.col.1=b, view.catalogAndNamespace.part.0=spark_catalog, view.catalogAndNamespace.part.1=testviewschm2] --- !query 82 +-- !query CREATE VIEW tt1 AS SELECT * FROM ( VALUES ('abc', '0123456789', 42, 'abcd'), ('0123456789', 'abc', 42.12, 'abc') ) vv(a,b,c,d) --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output --- !query 83 +-- !query SELECT * FROM tt1 --- !query 83 schema +-- !query schema struct --- !query 83 output +-- !query output 0123456789 abc 42.12 abc abc 0123456789 42.00 abcd --- !query 84 +-- !query SELECT string(a) FROM tt1 --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output 0123456789 abc --- !query 85 +-- !query DROP VIEW tt1 --- !query 85 schema +-- !query schema struct<> --- !query 85 output +-- !query output --- !query 86 +-- !query CREATE TABLE tt1 (f1 int, f2 int, f3 string) using parquet --- !query 86 schema +-- !query schema struct<> --- !query 86 output +-- !query output --- !query 87 +-- !query CREATE TABLE tx1 (x1 int, x2 int, x3 string) using parquet --- !query 87 schema +-- !query schema struct<> --- !query 87 output +-- !query output --- !query 88 +-- !query CREATE TABLE temp_view_test.tt1 (y1 int, f2 int, f3 string) using parquet --- !query 88 schema +-- !query schema struct<> --- !query 88 output +-- !query output --- !query 89 +-- !query CREATE VIEW aliased_view_1 AS select * from tt1 where exists (select 1 from tx1 where tt1.f1 = tx1.x1) --- !query 89 schema +-- !query schema struct<> --- !query 89 output +-- !query output --- !query 90 +-- !query CREATE VIEW aliased_view_2 AS select * from tt1 a1 where exists (select 1 from tx1 where a1.f1 = tx1.x1) --- !query 90 schema +-- !query schema struct<> --- !query 90 output +-- !query output --- !query 91 +-- !query CREATE VIEW aliased_view_3 AS select * from tt1 where exists (select 1 from tx1 a2 where tt1.f1 = a2.x1) --- !query 91 schema +-- !query schema struct<> --- !query 91 output +-- !query output --- !query 92 +-- !query CREATE VIEW aliased_view_4 AS select * from temp_view_test.tt1 where exists (select 1 from tt1 where temp_view_test.tt1.y1 = tt1.f1) --- !query 92 schema +-- !query schema struct<> --- !query 92 output +-- !query output --- !query 93 +-- !query DESC TABLE aliased_view_1 --- !query 93 schema +-- !query schema struct --- !query 93 output +-- !query output f1 int f2 int f3 string --- !query 94 +-- !query DESC TABLE aliased_view_2 --- !query 94 schema +-- !query schema struct --- !query 94 output +-- !query output f1 int f2 int f3 string --- !query 95 +-- !query DESC TABLE aliased_view_3 --- !query 95 schema +-- !query schema struct --- !query 95 output +-- !query output f1 int f2 int f3 string --- !query 96 +-- !query DESC TABLE aliased_view_4 --- !query 96 schema +-- !query schema struct --- !query 96 output +-- !query output y1 int f2 int f3 string --- !query 97 +-- !query ALTER TABLE tx1 RENAME TO a1 --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output --- !query 98 +-- !query DESC TABLE aliased_view_1 --- !query 98 schema +-- !query schema struct --- !query 98 output +-- !query output f1 int f2 int f3 string --- !query 99 +-- !query DESC TABLE aliased_view_2 --- !query 99 schema +-- !query schema struct --- !query 99 output +-- !query output f1 int f2 int f3 string --- !query 100 +-- !query DESC TABLE aliased_view_3 --- !query 100 schema +-- !query schema struct --- !query 100 output +-- !query output f1 int f2 int f3 string --- !query 101 +-- !query DESC TABLE aliased_view_4 --- !query 101 schema +-- !query schema struct --- !query 101 output +-- !query output y1 int f2 int f3 string --- !query 102 +-- !query ALTER TABLE tt1 RENAME TO a2 --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output --- !query 103 +-- !query DESC TABLE aliased_view_1 --- !query 103 schema +-- !query schema struct --- !query 103 output +-- !query output f1 int f2 int f3 string --- !query 104 +-- !query DESC TABLE aliased_view_2 --- !query 104 schema +-- !query schema struct --- !query 104 output +-- !query output f1 int f2 int f3 string --- !query 105 +-- !query DESC TABLE aliased_view_3 --- !query 105 schema +-- !query schema struct --- !query 105 output +-- !query output f1 int f2 int f3 string --- !query 106 +-- !query DESC TABLE aliased_view_4 --- !query 106 schema +-- !query schema struct --- !query 106 output +-- !query output y1 int f2 int f3 string --- !query 107 +-- !query ALTER TABLE a1 RENAME TO tt1 --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output --- !query 108 +-- !query DESC TABLE aliased_view_1 --- !query 108 schema +-- !query schema struct --- !query 108 output +-- !query output f1 int f2 int f3 string --- !query 109 +-- !query DESC TABLE aliased_view_2 --- !query 109 schema +-- !query schema struct --- !query 109 output +-- !query output f1 int f2 int f3 string --- !query 110 +-- !query DESC TABLE aliased_view_3 --- !query 110 schema +-- !query schema struct --- !query 110 output +-- !query output f1 int f2 int f3 string --- !query 111 +-- !query DESC TABLE aliased_view_4 --- !query 111 schema +-- !query schema struct --- !query 111 output +-- !query output y1 int f2 int f3 string --- !query 112 +-- !query ALTER TABLE a2 RENAME TO tx1 --- !query 112 schema +-- !query schema struct<> --- !query 112 output +-- !query output --- !query 113 +-- !query create view view_of_joins as select * from (select * from (tbl1 cross join tbl2) same) ss, (tbl3 cross join tbl4) same --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output --- !query 114 +-- !query create table tt2 (a int, b int, c int) using parquet --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output --- !query 115 +-- !query create table tt3 (ax bigint, b short, c decimal) using parquet --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output --- !query 116 +-- !query create table tt4 (ay int, b int, q int) using parquet --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output --- !query 117 +-- !query create view v1 as select * from tt2 natural join tt3 --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output --- !query 118 +-- !query create view v1a as select * from (tt2 natural join tt3) j --- !query 118 schema +-- !query schema struct<> --- !query 118 output +-- !query output --- !query 119 +-- !query create view v2 as select * from tt2 join tt3 using (b,c) join tt4 using (b) --- !query 119 schema +-- !query schema struct<> --- !query 119 output +-- !query output --- !query 120 +-- !query create view v2a as select * from (tt2 join tt3 using (b,c) join tt4 using (b)) j --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output --- !query 121 +-- !query create view v3 as select * from tt2 join tt3 using (b,c) full join tt4 using (b) --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output --- !query 122 +-- !query DESC TABLE v1 --- !query 122 schema +-- !query schema struct --- !query 122 output +-- !query output b int c int a int ax bigint --- !query 123 +-- !query DESC TABLE v1a --- !query 123 schema +-- !query schema struct --- !query 123 output +-- !query output b int c int a int ax bigint --- !query 124 +-- !query DESC TABLE v2 --- !query 124 schema +-- !query schema struct --- !query 124 output +-- !query output b int c int a int @@ -1341,11 +1341,11 @@ ay int q int --- !query 125 +-- !query DESC TABLE v2a --- !query 125 schema +-- !query schema struct --- !query 125 output +-- !query output b int c int a int @@ -1354,11 +1354,11 @@ ay int q int --- !query 126 +-- !query DESC TABLE v3 --- !query 126 schema +-- !query schema struct --- !query 126 output +-- !query output b int c int a int @@ -1367,49 +1367,49 @@ ay int q int --- !query 127 +-- !query alter table tt2 add column d int --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output --- !query 128 +-- !query alter table tt2 add column e int --- !query 128 schema +-- !query schema struct<> --- !query 128 output +-- !query output --- !query 129 +-- !query DESC TABLE v1 --- !query 129 schema +-- !query schema struct --- !query 129 output +-- !query output b int c int a int ax bigint --- !query 130 +-- !query DESC TABLE v1a --- !query 130 schema +-- !query schema struct --- !query 130 output +-- !query output b int c int a int ax bigint --- !query 131 +-- !query DESC TABLE v2 --- !query 131 schema +-- !query schema struct --- !query 131 output +-- !query output b int c int a int @@ -1418,11 +1418,11 @@ ay int q int --- !query 132 +-- !query DESC TABLE v2a --- !query 132 schema +-- !query schema struct --- !query 132 output +-- !query output b int c int a int @@ -1431,11 +1431,11 @@ ay int q int --- !query 133 +-- !query DESC TABLE v3 --- !query 133 schema +-- !query schema struct --- !query 133 output +-- !query output b int c int a int @@ -1444,65 +1444,65 @@ ay int q int --- !query 134 +-- !query drop table tt3 --- !query 134 schema +-- !query schema struct<> --- !query 134 output +-- !query output --- !query 135 +-- !query create table tt3 (ax bigint, b short, d decimal) using parquet --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output --- !query 136 +-- !query alter table tt3 add column c int --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output --- !query 137 +-- !query alter table tt3 add column e int --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output --- !query 138 +-- !query DESC TABLE v1 --- !query 138 schema +-- !query schema struct --- !query 138 output +-- !query output b int c int a int ax bigint --- !query 139 +-- !query DESC TABLE v1a --- !query 139 schema +-- !query schema struct --- !query 139 output +-- !query output b int c int a int ax bigint --- !query 140 +-- !query DESC TABLE v2 --- !query 140 schema +-- !query schema struct --- !query 140 output +-- !query output b int c int a int @@ -1511,11 +1511,11 @@ ay int q int --- !query 141 +-- !query DESC TABLE v2a --- !query 141 schema +-- !query schema struct --- !query 141 output +-- !query output b int c int a int @@ -1524,11 +1524,11 @@ ay int q int --- !query 142 +-- !query DESC TABLE v3 --- !query 142 schema +-- !query schema struct --- !query 142 output +-- !query output b int c int a int @@ -1537,111 +1537,111 @@ ay int q int --- !query 143 +-- !query create table tt5 (a int, b int) using parquet --- !query 143 schema +-- !query schema struct<> --- !query 143 output +-- !query output --- !query 144 +-- !query create table tt6 (c int, d int) using parquet --- !query 144 schema +-- !query schema struct<> --- !query 144 output +-- !query output --- !query 145 +-- !query create view vv1 as select * from (tt5 cross join tt6) j(aa,bb,cc,dd) --- !query 145 schema +-- !query schema struct<> --- !query 145 output +-- !query output --- !query 146 +-- !query DESC TABLE vv1 --- !query 146 schema +-- !query schema struct --- !query 146 output +-- !query output aa int bb int cc int dd int --- !query 147 +-- !query alter table tt5 add column c int --- !query 147 schema +-- !query schema struct<> --- !query 147 output +-- !query output --- !query 148 +-- !query DESC TABLE vv1 --- !query 148 schema +-- !query schema struct --- !query 148 output +-- !query output aa int bb int cc int dd int --- !query 149 +-- !query alter table tt5 add column cc int --- !query 149 schema +-- !query schema struct<> --- !query 149 output +-- !query output --- !query 150 +-- !query DESC TABLE vv1 --- !query 150 schema +-- !query schema struct --- !query 150 output +-- !query output aa int bb int cc int dd int --- !query 151 +-- !query create table tt7 (x int, /* xx int, */ y int) using parquet --- !query 151 schema +-- !query schema struct<> --- !query 151 output +-- !query output --- !query 152 +-- !query create table tt8 (x int, z int) using parquet --- !query 152 schema +-- !query schema struct<> --- !query 152 output +-- !query output --- !query 153 +-- !query create view vv2 as select * from (values(1,2,3,4,5)) v(a,b,c,d,e) union all select * from tt7 full join tt8 using (x), tt8 tt8x --- !query 153 schema +-- !query schema struct<> --- !query 153 output +-- !query output --- !query 154 +-- !query DESC TABLE vv2 --- !query 154 schema +-- !query schema struct --- !query 154 output +-- !query output a int b int c int @@ -1649,24 +1649,24 @@ d int e int --- !query 155 +-- !query create view vv3 as select * from (values(1,2,3,4,5,6)) v(a,b,c,x,e,f) union all select * from tt7 full join tt8 using (x), tt7 tt7x full join tt8 tt8x using (x) --- !query 155 schema +-- !query schema struct<> --- !query 155 output +-- !query output --- !query 156 +-- !query DESC TABLE vv3 --- !query 156 schema +-- !query schema struct --- !query 156 output +-- !query output a int b int c int @@ -1675,24 +1675,24 @@ e int f int --- !query 157 +-- !query create view vv4 as select * from (values(1,2,3,4,5,6,7)) v(a,b,c,x,e,f,g) union all select * from tt7 full join tt8 using (x), tt7 tt7x full join tt8 tt8x using (x) full join tt8 tt8y using (x) --- !query 157 schema +-- !query schema struct<> --- !query 157 output +-- !query output --- !query 158 +-- !query DESC TABLE vv4 --- !query 158 schema +-- !query schema struct --- !query 158 output +-- !query output a int b int c int @@ -1702,35 +1702,35 @@ f int g int --- !query 159 +-- !query alter table tt7 add column zz int --- !query 159 schema +-- !query schema struct<> --- !query 159 output +-- !query output --- !query 160 +-- !query alter table tt7 add column z int --- !query 160 schema +-- !query schema struct<> --- !query 160 output +-- !query output --- !query 161 +-- !query alter table tt8 add column z2 int --- !query 161 schema +-- !query schema struct<> --- !query 161 output +-- !query output --- !query 162 +-- !query DESC TABLE vv2 --- !query 162 schema +-- !query schema struct --- !query 162 output +-- !query output a int b int c int @@ -1738,11 +1738,11 @@ d int e int --- !query 163 +-- !query DESC TABLE vv3 --- !query 163 schema +-- !query schema struct --- !query 163 output +-- !query output a int b int c int @@ -1751,11 +1751,11 @@ e int f int --- !query 164 +-- !query DESC TABLE vv4 --- !query 164 schema +-- !query schema struct --- !query 164 output +-- !query output a int b int c int @@ -1765,38 +1765,38 @@ f int g int --- !query 165 +-- !query create table tt7a (x date, /* xx int, */ y int) using parquet --- !query 165 schema +-- !query schema struct<> --- !query 165 output +-- !query output --- !query 166 +-- !query create table tt8a (x timestamp, z int) using parquet --- !query 166 schema +-- !query schema struct<> --- !query 166 output +-- !query output --- !query 167 +-- !query create view vv2a as select * from (values(now(),2,3,now(),5)) v(a,b,c,d,e) union all select * from tt7a left join tt8a using (x), tt8a tt8ax --- !query 167 schema +-- !query schema struct<> --- !query 167 output +-- !query output --- !query 168 +-- !query DESC TABLE vv4 --- !query 168 schema +-- !query schema struct --- !query 168 output +-- !query output a int b int c int @@ -1806,11 +1806,11 @@ f int g int --- !query 169 +-- !query DESC TABLE vv2a --- !query 169 schema +-- !query schema struct --- !query 169 output +-- !query output a timestamp b int c int @@ -1818,155 +1818,155 @@ d timestamp e int --- !query 170 +-- !query create table tt9 (x int, xx int, y int) using parquet --- !query 170 schema +-- !query schema struct<> --- !query 170 output +-- !query output --- !query 171 +-- !query create table tt10 (x int, z int) using parquet --- !query 171 schema +-- !query schema struct<> --- !query 171 output +-- !query output --- !query 172 +-- !query create view vv5 as select x,y,z from tt9 join tt10 using(x) --- !query 172 schema +-- !query schema struct<> --- !query 172 output +-- !query output --- !query 173 +-- !query DESC TABLE vv5 --- !query 173 schema +-- !query schema struct --- !query 173 output +-- !query output x int y int z int --- !query 174 +-- !query DESC TABLE vv5 --- !query 174 schema +-- !query schema struct --- !query 174 output +-- !query output x int y int z int --- !query 175 +-- !query create table tt11 (x int, y int) using parquet --- !query 175 schema +-- !query schema struct<> --- !query 175 output +-- !query output --- !query 176 +-- !query create table tt12 (x int, z int) using parquet --- !query 176 schema +-- !query schema struct<> --- !query 176 output +-- !query output --- !query 177 +-- !query create table tt13 (z int, q int) using parquet --- !query 177 schema +-- !query schema struct<> --- !query 177 output +-- !query output --- !query 178 +-- !query create view vv6 as select x,y,z,q from (tt11 join tt12 using(x)) join tt13 using(z) --- !query 178 schema +-- !query schema struct<> --- !query 178 output +-- !query output --- !query 179 +-- !query DESC TABLE vv6 --- !query 179 schema +-- !query schema struct --- !query 179 output +-- !query output x int y int z int q int --- !query 180 +-- !query alter table tt11 add column z int --- !query 180 schema +-- !query schema struct<> --- !query 180 output +-- !query output --- !query 181 +-- !query DESC TABLE vv6 --- !query 181 schema +-- !query schema struct --- !query 181 output +-- !query output x int y int z int q int --- !query 182 +-- !query CREATE TABLE int8_tbl (q1 int, q2 int) USING parquet --- !query 182 schema +-- !query schema struct<> --- !query 182 output +-- !query output --- !query 183 +-- !query create view tt18v as select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxy union all select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxz --- !query 183 schema +-- !query schema struct<> --- !query 183 output +-- !query output --- !query 184 +-- !query DESC TABLE tt18v --- !query 184 schema +-- !query schema struct --- !query 184 output +-- !query output q1 int q2 int --- !query 185 +-- !query create view tt21v as select * from tt5 natural inner join tt6 --- !query 185 schema +-- !query schema struct<> --- !query 185 output +-- !query output --- !query 186 +-- !query DESC TABLE tt21v --- !query 186 schema +-- !query schema struct --- !query 186 output +-- !query output c int a int b int @@ -1974,20 +1974,20 @@ cc int d int --- !query 187 +-- !query create view tt22v as select * from tt5 natural left join tt6 --- !query 187 schema +-- !query schema struct<> --- !query 187 output +-- !query output --- !query 188 +-- !query DESC TABLE tt22v --- !query 188 schema +-- !query schema struct --- !query 188 output +-- !query output c int a int b int @@ -1995,53 +1995,53 @@ cc int d int --- !query 189 +-- !query create view tt23v (col_a, col_b) as select q1 as other_name1, q2 as other_name2 from int8_tbl union select 42, 43 --- !query 189 schema +-- !query schema struct<> --- !query 189 output +-- !query output --- !query 190 +-- !query DESC TABLE tt23v --- !query 190 schema +-- !query schema struct --- !query 190 output +-- !query output col_a int col_b int --- !query 191 +-- !query DROP SCHEMA temp_view_test CASCADE --- !query 191 schema +-- !query schema struct<> --- !query 191 output +-- !query output --- !query 192 +-- !query DROP SCHEMA testviewschm2 CASCADE --- !query 192 schema +-- !query schema struct<> --- !query 192 output +-- !query output --- !query 193 +-- !query DROP VIEW temp_table --- !query 193 schema +-- !query schema struct<> --- !query 193 output +-- !query output --- !query 194 +-- !query DROP VIEW tt --- !query 194 schema +-- !query schema struct<> --- !query 194 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index 700476e91eaac..fd5dc42632176 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -2,139 +2,139 @@ -- Number of queries: 98 --- !query 0 +-- !query CREATE TABLE DATE_TBL (f1 date) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO DATE_TBL VALUES (date('1957-04-09')) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO DATE_TBL VALUES (date('1957-06-13')) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO DATE_TBL VALUES (date('1996-02-28')) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO DATE_TBL VALUES (date('1996-02-29')) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO DATE_TBL VALUES (date('1996-03-01')) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO DATE_TBL VALUES (date('1996-03-02')) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO DATE_TBL VALUES (date('1997-02-28')) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO DATE_TBL VALUES (date('1997-03-01')) --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO DATE_TBL VALUES (date('1997-03-02')) --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO DATE_TBL VALUES (date('2000-04-01')) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query INSERT INTO DATE_TBL VALUES (date('2000-04-02')) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query INSERT INTO DATE_TBL VALUES (date('2000-04-03')) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query INSERT INTO DATE_TBL VALUES (date('2038-04-08')) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query INSERT INTO DATE_TBL VALUES (date('2039-04-09')) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query INSERT INTO DATE_TBL VALUES (date('2040-04-10')) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query SELECT f1 AS `Fifteen` FROM DATE_TBL --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1957-04-09 1957-06-13 1996-02-28 @@ -152,11 +152,11 @@ struct 2040-04-10 --- !query 17 +-- !query SELECT f1 AS `Nine` FROM DATE_TBL WHERE f1 < '2000-01-01' --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1957-04-09 1957-06-13 1996-02-28 @@ -168,38 +168,38 @@ struct 1997-03-02 --- !query 18 +-- !query SELECT f1 AS `Three` FROM DATE_TBL WHERE f1 BETWEEN '2000-01-01' AND '2001-01-01' --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 2000-04-01 2000-04-02 2000-04-03 --- !query 19 +-- !query SELECT date '1999-01-08' --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1999-01-08 --- !query 20 +-- !query SELECT date '1999-01-18' --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1999-01-18 --- !query 21 +-- !query SELECT date '1999 Jan 08' --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) @@ -209,11 +209,11 @@ SELECT date '1999 Jan 08' -------^^^ --- !query 22 +-- !query SELECT date '1999 08 Jan' --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) @@ -223,27 +223,27 @@ SELECT date '1999 08 Jan' -------^^^ --- !query 23 +-- !query SELECT date '1999-01-08' --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 1999-01-08 --- !query 24 +-- !query SELECT date '1999-08-01' --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1999-08-01 --- !query 25 +-- !query SELECT date '1999 01 08' --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 01 08(line 1, pos 7) @@ -253,11 +253,11 @@ SELECT date '1999 01 08' -------^^^ --- !query 26 +-- !query SELECT date '1999 08 01' --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 08 01(line 1, pos 7) @@ -267,19 +267,19 @@ SELECT date '1999 08 01' -------^^^ --- !query 27 +-- !query SELECT date '1999-01-08' --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 1999-01-08 --- !query 28 +-- !query SELECT date '1999 Jan 08' --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) @@ -289,11 +289,11 @@ SELECT date '1999 Jan 08' -------^^^ --- !query 29 +-- !query SELECT date '1999 08 Jan' --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) @@ -303,27 +303,27 @@ SELECT date '1999 08 Jan' -------^^^ --- !query 30 +-- !query SELECT date '1999-01-08' --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1999-01-08 --- !query 31 +-- !query SELECT date '1999-08-01' --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 1999-08-01 --- !query 32 +-- !query SELECT date '1999 01 08' --- !query 32 schema +-- !query schema struct<> --- !query 32 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 01 08(line 1, pos 7) @@ -333,11 +333,11 @@ SELECT date '1999 01 08' -------^^^ --- !query 33 +-- !query SELECT date '1999 08 01' --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 08 01(line 1, pos 7) @@ -347,27 +347,27 @@ SELECT date '1999 08 01' -------^^^ --- !query 34 +-- !query SELECT date '1999-01-08' --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 1999-01-08 --- !query 35 +-- !query SELECT date '1999-01-18' --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 1999-01-18 --- !query 36 +-- !query SELECT date '1999 Jan 08' --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) @@ -377,11 +377,11 @@ SELECT date '1999 Jan 08' -------^^^ --- !query 37 +-- !query SELECT date '1999 08 Jan' --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) @@ -391,27 +391,27 @@ SELECT date '1999 08 Jan' -------^^^ --- !query 38 +-- !query SELECT date '1999-01-08' --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 1999-01-08 --- !query 39 +-- !query SELECT date '1999-08-01' --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 1999-08-01 --- !query 40 +-- !query SELECT date '1999 01 08' --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 01 08(line 1, pos 7) @@ -421,11 +421,11 @@ SELECT date '1999 01 08' -------^^^ --- !query 41 +-- !query SELECT date '1999 08 01' --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 1999 08 01(line 1, pos 7) @@ -435,27 +435,27 @@ SELECT date '1999 08 01' -------^^^ --- !query 42 +-- !query SELECT date '4714-11-24 BC' --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 4714-11-24 --- !query 43 +-- !query SELECT date '4714-11-23 BC' --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 4714-11-23 --- !query 44 +-- !query SELECT date '5874897-12-31' --- !query 44 schema +-- !query schema struct<> --- !query 44 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 5874897-12-31(line 1, pos 7) @@ -465,11 +465,11 @@ SELECT date '5874897-12-31' -------^^^ --- !query 45 +-- !query SELECT date '5874898-01-01' --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Cannot parse the DATE value: 5874898-01-01(line 1, pos 7) @@ -479,11 +479,11 @@ SELECT date '5874898-01-01' -------^^^ --- !query 46 +-- !query SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output -2 years -10 months -2 years -10 months -1 days -2 years -9 months -30 days @@ -501,11 +501,11 @@ struct 40 years 3 months 9 days --- !query 47 +-- !query SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output -12 years -6 months -18 days -12 years -8 months -22 days 26 years 1 months 27 days @@ -523,401 +523,401 @@ struct 70 years 3 months 9 days --- !query 48 +-- !query SELECT date 'yesterday' - date 'today' AS `One day` --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output -1 days --- !query 49 +-- !query SELECT date 'today' - date 'tomorrow' AS `One day` --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output -1 days --- !query 50 +-- !query SELECT date 'yesterday' - date 'tomorrow' AS `Two days` --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output -2 days --- !query 51 +-- !query SELECT date 'tomorrow' - date 'today' AS `One day` --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output 1 days --- !query 52 +-- !query SELECT date 'today' - date 'yesterday' AS `One day` --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output 1 days --- !query 53 +-- !query SELECT date 'tomorrow' - date 'yesterday' AS `Two days` --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 2 days --- !query 54 +-- !query SELECT EXTRACT(EPOCH FROM DATE '1970-01-01') --- !query 54 schema +-- !query schema struct --- !query 54 output +-- !query output 0.000000 --- !query 55 +-- !query SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01') --- !query 55 schema +-- !query schema struct --- !query 55 output +-- !query output 0.000000 --- !query 56 +-- !query SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')) --- !query 56 schema +-- !query schema struct --- !query 56 output +-- !query output -2 --- !query 57 +-- !query SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')) --- !query 57 schema +-- !query schema struct --- !query 57 output +-- !query output -1 --- !query 58 +-- !query SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) --- !query 58 schema +-- !query schema struct --- !query 58 output +-- !query output -1 --- !query 59 +-- !query SELECT EXTRACT(CENTURY FROM DATE '0001-01-01') --- !query 59 schema +-- !query schema struct --- !query 59 output +-- !query output 1 --- !query 60 +-- !query SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD') --- !query 60 schema +-- !query schema struct --- !query 60 output +-- !query output 1 --- !query 61 +-- !query SELECT EXTRACT(CENTURY FROM DATE '1900-12-31') --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output 19 --- !query 62 +-- !query SELECT EXTRACT(CENTURY FROM DATE '1901-01-01') --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output 20 --- !query 63 +-- !query SELECT EXTRACT(CENTURY FROM DATE '2000-12-31') --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output 20 --- !query 64 +-- !query SELECT EXTRACT(CENTURY FROM DATE '2001-01-01') --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 21 --- !query 65 +-- !query SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output true --- !query 66 +-- !query SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')) --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output -1 --- !query 67 +-- !query SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD') --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output 1 --- !query 68 +-- !query SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31') --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 1 --- !query 69 +-- !query SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01') --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output 2 --- !query 70 +-- !query SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31') --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output 2 --- !query 71 +-- !query SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01') --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output 3 --- !query 72 +-- !query SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE) --- !query 72 schema +-- !query schema struct --- !query 72 output +-- !query output 3 --- !query 73 +-- !query SELECT EXTRACT(DECADE FROM DATE '1994-12-25') --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output 199 --- !query 74 +-- !query SELECT EXTRACT(DECADE FROM DATE '0010-01-01') --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output 1 --- !query 75 +-- !query SELECT EXTRACT(DECADE FROM DATE '0009-12-31') --- !query 75 schema +-- !query schema struct --- !query 75 output +-- !query output 0 --- !query 76 +-- !query SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')) --- !query 76 schema +-- !query schema struct --- !query 76 output +-- !query output 0 --- !query 77 +-- !query SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) --- !query 77 schema +-- !query schema struct --- !query 77 output +-- !query output -1 --- !query 78 +-- !query SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')) --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output -1 --- !query 79 +-- !query SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')) --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output -2 --- !query 80 +-- !query SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True --- !query 80 schema +-- !query schema struct --- !query 80 output +-- !query output true --- !query 81 +-- !query SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000') --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output 20 --- !query 82 +-- !query SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output 1001-01-01 00:07:02 --- !query 83 +-- !query SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') --- !query 83 schema +-- !query schema struct --- !query 83 output +-- !query output 1001-01-01 00:07:02 --- !query 84 +-- !query SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000') --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output 1901-01-01 00:00:00 --- !query 85 +-- !query SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20') --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output 1901-01-01 00:00:00 --- !query 86 +-- !query SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10') --- !query 86 schema +-- !query schema struct --- !query 86 output +-- !query output 2001-01-01 00:00:00 --- !query 87 +-- !query SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 0001-01-01 00:07:02 --- !query 88 +-- !query SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output -0099-01-01 00:07:02 --- !query 89 +-- !query SELECT DATE_TRUNC('DECADE', DATE '1993-12-25') --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output 1990-01-01 00:00:00 --- !query 90 +-- !query SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 0000-01-01 00:07:02 --- !query 91 +-- !query SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output -0010-01-01 00:07:02 --- !query 92 +-- !query select make_date(2013, 7, 15) --- !query 92 schema +-- !query schema struct --- !query 92 output +-- !query output 2013-07-15 --- !query 93 +-- !query select make_date(-44, 3, 15) --- !query 93 schema +-- !query schema struct --- !query 93 output +-- !query output -0044-03-15 --- !query 94 +-- !query select make_date(2013, 2, 30) --- !query 94 schema +-- !query schema struct --- !query 94 output +-- !query output NULL --- !query 95 +-- !query select make_date(2013, 13, 1) --- !query 95 schema +-- !query schema struct --- !query 95 output +-- !query output NULL --- !query 96 +-- !query select make_date(2013, 11, -1) --- !query 96 schema +-- !query schema struct --- !query 96 output +-- !query output NULL --- !query 97 +-- !query DROP TABLE DATE_TBL --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out index 7ca4c7eb8aa30..ba913789d5623 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out @@ -2,159 +2,159 @@ -- Number of queries: 43 --- !query 0 +-- !query CREATE TABLE FLOAT4_TBL (f1 float) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT float('NaN') --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NaN --- !query 7 +-- !query SELECT float('nan') --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NaN --- !query 8 +-- !query SELECT float(' NAN ') --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NaN --- !query 9 +-- !query SELECT float('infinity') --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output Infinity --- !query 10 +-- !query SELECT float(' -INFINiTY ') --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output -Infinity --- !query 11 +-- !query SELECT float('N A N') --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: N A N --- !query 12 +-- !query SELECT float('NaN x') --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: NaN x --- !query 13 +-- !query SELECT float(' INFINITY x') --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: INFINITY x --- !query 14 +-- !query SELECT float('Infinity') + 100.0 --- !query 14 schema +-- !query schema struct<(CAST(CAST(Infinity AS FLOAT) AS DOUBLE) + CAST(100.0 AS DOUBLE)):double> --- !query 14 output +-- !query output Infinity --- !query 15 +-- !query SELECT float('Infinity') / float('Infinity') --- !query 15 schema +-- !query schema struct<(CAST(CAST(Infinity AS FLOAT) AS DOUBLE) / CAST(CAST(Infinity AS FLOAT) AS DOUBLE)):double> --- !query 15 output +-- !query output NaN --- !query 16 +-- !query SELECT float('nan') / float('nan') --- !query 16 schema +-- !query schema struct<(CAST(CAST(nan AS FLOAT) AS DOUBLE) / CAST(CAST(nan AS FLOAT) AS DOUBLE)):double> --- !query 16 output +-- !query output NaN --- !query 17 +-- !query SELECT float(decimal('nan')) --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: nan --- !query 18 +-- !query SELECT '' AS five, * FROM FLOAT4_TBL --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output -34.84 0.0 1.2345679E-20 @@ -162,116 +162,116 @@ struct 1004.3 --- !query 19 +-- !query SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <> '1004.3' --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output -34.84 0.0 1.2345679E-20 1.2345679E20 --- !query 20 +-- !query SELECT '' AS one, f.* FROM FLOAT4_TBL f WHERE f.f1 = '1004.3' --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1004.3 --- !query 21 +-- !query SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE '1004.3' > f.f1 --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output -34.84 0.0 1.2345679E-20 --- !query 22 +-- !query SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE f.f1 < '1004.3' --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output -34.84 0.0 1.2345679E-20 --- !query 23 +-- !query SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE '1004.3' >= f.f1 --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output -34.84 0.0 1.2345679E-20 1004.3 --- !query 24 +-- !query SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <= '1004.3' --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output -34.84 0.0 1.2345679E-20 1004.3 --- !query 25 +-- !query SELECT '' AS three, f.f1, f.f1 * '-10' AS x FROM FLOAT4_TBL f WHERE f.f1 > '0.0' --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1.2345679E-20 -1.2345678720289608E-19 1.2345679E20 -1.2345678955701443E21 1004.3 -10042.999877929688 --- !query 26 +-- !query SELECT '' AS three, f.f1, f.f1 + '-10' AS x FROM FLOAT4_TBL f WHERE f.f1 > '0.0' --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1.2345679E-20 -10.0 1.2345679E20 1.2345678955701443E20 1004.3 994.2999877929688 --- !query 27 +-- !query SELECT '' AS three, f.f1, f.f1 / '-10' AS x FROM FLOAT4_TBL f WHERE f.f1 > '0.0' --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 1.2345679E-20 -1.2345678720289608E-21 1.2345679E20 -1.2345678955701443E19 1004.3 -100.42999877929688 --- !query 28 +-- !query SELECT '' AS three, f.f1, f.f1 - '-10' AS x FROM FLOAT4_TBL f WHERE f.f1 > '0.0' --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 1.2345679E-20 10.0 1.2345679E20 1.2345678955701443E20 1004.3 1014.2999877929688 --- !query 29 +-- !query SELECT '' AS five, * FROM FLOAT4_TBL --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output -34.84 0.0 1.2345679E-20 @@ -279,107 +279,107 @@ struct 1004.3 --- !query 30 +-- !query SELECT smallint(float('32767.4')) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 32767 --- !query 31 +-- !query SELECT smallint(float('32767.6')) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 32767 --- !query 32 +-- !query SELECT smallint(float('-32768.4')) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output -32768 --- !query 33 +-- !query SELECT smallint(float('-32768.6')) --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output -32768 --- !query 34 +-- !query SELECT int(float('2147483520')) --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 2147483520 --- !query 35 +-- !query SELECT int(float('2147483647')) --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 2147483647 --- !query 36 +-- !query SELECT int(float('-2147483648.5')) --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output -2147483648 --- !query 37 +-- !query SELECT int(float('-2147483900')) --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output java.lang.ArithmeticException Casting -2.1474839E9 to int causes overflow --- !query 38 +-- !query SELECT bigint(float('9223369837831520256')) --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 9223369837831520256 --- !query 39 +-- !query SELECT bigint(float('9223372036854775807')) --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 9223372036854775807 --- !query 40 +-- !query SELECT bigint(float('-9223372036854775808.5')) --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output -9223372036854775808 --- !query 41 +-- !query SELECT bigint(float('-9223380000000000000')) --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output java.lang.ArithmeticException Casting -9.22338E18 to int causes overflow --- !query 42 +-- !query DROP TABLE FLOAT4_TBL --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out index 9d170d2fd8898..4cdb6958a230a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float8.sql.out @@ -2,191 +2,191 @@ -- Number of queries: 95 --- !query 0 +-- !query CREATE TABLE FLOAT8_TBL(f1 double) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 ')) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO FLOAT8_TBL VALUES (double('1004.30 ')) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO FLOAT8_TBL VALUES (double(' -34.84')) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200')) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200')) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT double('10e400') --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output Infinity --- !query 7 +-- !query SELECT double('-10e400') --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output -Infinity --- !query 8 +-- !query SELECT double('10e-400') --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 0.0 --- !query 9 +-- !query SELECT double('-10e-400') --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output -0.0 --- !query 10 +-- !query SELECT double('NaN') --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output NaN --- !query 11 +-- !query SELECT double('nan') --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output NaN --- !query 12 +-- !query SELECT double(' NAN ') --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output NaN --- !query 13 +-- !query SELECT double('infinity') --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output Infinity --- !query 14 +-- !query SELECT double(' -INFINiTY ') --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output -Infinity --- !query 15 +-- !query SELECT double('N A N') --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: N A N --- !query 16 +-- !query SELECT double('NaN x') --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: NaN x --- !query 17 +-- !query SELECT double(' INFINITY x') --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: INFINITY x --- !query 18 +-- !query SELECT double('Infinity') + 100.0 --- !query 18 schema +-- !query schema struct<(CAST(Infinity AS DOUBLE) + CAST(100.0 AS DOUBLE)):double> --- !query 18 output +-- !query output Infinity --- !query 19 +-- !query SELECT double('Infinity') / double('Infinity') --- !query 19 schema +-- !query schema struct<(CAST(Infinity AS DOUBLE) / CAST(Infinity AS DOUBLE)):double> --- !query 19 output +-- !query output NaN --- !query 20 +-- !query SELECT double('NaN') / double('NaN') --- !query 20 schema +-- !query schema struct<(CAST(NaN AS DOUBLE) / CAST(NaN AS DOUBLE)):double> --- !query 20 output +-- !query output NaN --- !query 21 +-- !query SELECT double(decimal('nan')) --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: nan --- !query 22 +-- !query SELECT '' AS five, * FROM FLOAT8_TBL --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output -34.84 0.0 1.2345678901234E-200 @@ -194,121 +194,121 @@ struct 1004.3 --- !query 23 +-- !query SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3' --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output -34.84 0.0 1.2345678901234E-200 1.2345678901234E200 --- !query 24 +-- !query SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3' --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1004.3 --- !query 25 +-- !query SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1 --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output -34.84 0.0 1.2345678901234E-200 --- !query 26 +-- !query SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3' --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output -34.84 0.0 1.2345678901234E-200 --- !query 27 +-- !query SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1 --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output -34.84 0.0 1.2345678901234E-200 1004.3 --- !query 28 +-- !query SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3' --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output -34.84 0.0 1.2345678901234E-200 1004.3 --- !query 29 +-- !query SELECT '' AS three, f.f1, f.f1 * '-10' AS x FROM FLOAT8_TBL f WHERE f.f1 > '0.0' --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 1.2345678901234E-200 -1.2345678901234E-199 1.2345678901234E200 -1.2345678901234E201 1004.3 -10043.0 --- !query 30 +-- !query SELECT '' AS three, f.f1, f.f1 + '-10' AS x FROM FLOAT8_TBL f WHERE f.f1 > '0.0' --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1.2345678901234E-200 -10.0 1.2345678901234E200 1.2345678901234E200 1004.3 994.3 --- !query 31 +-- !query SELECT '' AS three, f.f1, f.f1 / '-10' AS x FROM FLOAT8_TBL f WHERE f.f1 > '0.0' --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 1.2345678901234E-200 -1.2345678901234E-201 1.2345678901234E200 -1.2345678901234E199 1004.3 -100.42999999999999 --- !query 32 +-- !query SELECT '' AS three, f.f1, f.f1 - '-10' AS x FROM FLOAT8_TBL f WHERE f.f1 > '0.0' --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 1.2345678901234E-200 10.0 1.2345678901234E200 1.2345678901234E200 1004.3 1014.3 --- !query 33 +-- !query SELECT '' AS five, f.f1, round(f.f1) AS round_f1 FROM FLOAT8_TBL f --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output -34.84 -35.0 0.0 0.0 1.2345678901234E-200 0.0 @@ -316,11 +316,11 @@ struct 1004.3 1004.0 --- !query 34 +-- !query select ceil(f1) as ceil_f1 from float8_tbl f --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output -34 0 1 @@ -328,11 +328,11 @@ struct 9223372036854775807 --- !query 35 +-- !query select ceiling(f1) as ceiling_f1 from float8_tbl f --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output -34 0 1 @@ -340,11 +340,11 @@ struct 9223372036854775807 --- !query 36 +-- !query select floor(f1) as floor_f1 from float8_tbl f --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output -35 0 0 @@ -352,11 +352,11 @@ struct 9223372036854775807 --- !query 37 +-- !query select sign(f1) as sign_f1 from float8_tbl f --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output -1.0 0.0 1.0 @@ -364,87 +364,87 @@ struct 1.0 --- !query 38 +-- !query SELECT sqrt(double('64')) AS eight --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 8.0 --- !query 39 +-- !query SELECT power(double('144'), double('0.5')) --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 12.0 --- !query 40 +-- !query SELECT power(double('NaN'), double('0.5')) --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output NaN --- !query 41 +-- !query SELECT power(double('144'), double('NaN')) --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output NaN --- !query 42 +-- !query SELECT power(double('NaN'), double('NaN')) --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output NaN --- !query 43 +-- !query SELECT power(double('-1'), double('NaN')) --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output NaN --- !query 44 +-- !query SELECT power(double('1'), double('NaN')) --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output NaN --- !query 45 +-- !query SELECT power(double('NaN'), double('0')) --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 1.0 --- !query 46 +-- !query SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 FROM FLOAT8_TBL f WHERE f.f1 > '0.0' --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output 1.2345678901234E-200 1.2345678901233948E-200 1.2345678901234E200 1.234567890123379E200 1004.3 1004.3000000000004 --- !query 47 +-- !query SELECT '' AS five, * FROM FLOAT8_TBL --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output -34.84 0.0 1.2345678901234E-200 @@ -452,22 +452,22 @@ struct 1004.3 --- !query 48 +-- !query CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as SELECT CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 FROM FLOAT8_TBL --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output --- !query 49 +-- !query SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output -1.0042999999999999E203 -1.2345678901234 -3.484E201 @@ -475,11 +475,11 @@ struct 0.0 --- !query 50 +-- !query SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output -1.2345678901234E-200 -1.2345678901234E200 -1004.3 @@ -487,251 +487,251 @@ struct 0.0 --- !query 51 +-- !query SELECT sinh(double('1')) --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output 1.1752011936438014 --- !query 52 +-- !query SELECT cosh(double('1')) --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output 1.543080634815244 --- !query 53 +-- !query SELECT tanh(double('1')) --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 0.7615941559557649 --- !query 54 +-- !query SELECT asinh(double('1')) --- !query 54 schema +-- !query schema struct --- !query 54 output +-- !query output 0.8813735870195429 --- !query 55 +-- !query SELECT acosh(double('2')) --- !query 55 schema +-- !query schema struct --- !query 55 output +-- !query output 1.3169578969248166 --- !query 56 +-- !query SELECT atanh(double('0.5')) --- !query 56 schema +-- !query schema struct --- !query 56 output +-- !query output 0.5493061443340548 --- !query 57 +-- !query SELECT sinh(double('Infinity')) --- !query 57 schema +-- !query schema struct --- !query 57 output +-- !query output Infinity --- !query 58 +-- !query SELECT sinh(double('-Infinity')) --- !query 58 schema +-- !query schema struct --- !query 58 output +-- !query output -Infinity --- !query 59 +-- !query SELECT sinh(double('NaN')) --- !query 59 schema +-- !query schema struct --- !query 59 output +-- !query output NaN --- !query 60 +-- !query SELECT cosh(double('Infinity')) --- !query 60 schema +-- !query schema struct --- !query 60 output +-- !query output Infinity --- !query 61 +-- !query SELECT cosh(double('-Infinity')) --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output Infinity --- !query 62 +-- !query SELECT cosh(double('NaN')) --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output NaN --- !query 63 +-- !query SELECT tanh(double('Infinity')) --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output 1.0 --- !query 64 +-- !query SELECT tanh(double('-Infinity')) --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output -1.0 --- !query 65 +-- !query SELECT tanh(double('NaN')) --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output NaN --- !query 66 +-- !query SELECT asinh(double('Infinity')) --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output Infinity --- !query 67 +-- !query SELECT asinh(double('-Infinity')) --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output -Infinity --- !query 68 +-- !query SELECT asinh(double('NaN')) --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output NaN --- !query 69 +-- !query SELECT acosh(double('Infinity')) --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output Infinity --- !query 70 +-- !query SELECT acosh(double('-Infinity')) --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output NaN --- !query 71 +-- !query SELECT acosh(double('NaN')) --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output NaN --- !query 72 +-- !query SELECT atanh(double('Infinity')) --- !query 72 schema +-- !query schema struct --- !query 72 output +-- !query output NaN --- !query 73 +-- !query SELECT atanh(double('-Infinity')) --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output NaN --- !query 74 +-- !query SELECT atanh(double('NaN')) --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output NaN --- !query 75 +-- !query TRUNCATE TABLE FLOAT8_TBL --- !query 75 schema +-- !query schema struct<> --- !query 75 output +-- !query output --- !query 76 +-- !query INSERT INTO FLOAT8_TBL VALUES (double('0.0')) --- !query 76 schema +-- !query schema struct<> --- !query 76 output +-- !query output --- !query 77 +-- !query INSERT INTO FLOAT8_TBL VALUES (double('-34.84')) --- !query 77 schema +-- !query schema struct<> --- !query 77 output +-- !query output --- !query 78 +-- !query INSERT INTO FLOAT8_TBL VALUES (double('-1004.30')) --- !query 78 schema +-- !query schema struct<> --- !query 78 output +-- !query output --- !query 79 +-- !query INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200')) --- !query 79 schema +-- !query schema struct<> --- !query 79 output +-- !query output --- !query 80 +-- !query INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200')) --- !query 80 schema +-- !query schema struct<> --- !query 80 output +-- !query output --- !query 81 +-- !query SELECT '' AS five, * FROM FLOAT8_TBL --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output -1.2345678901234E-200 -1.2345678901234E200 -1004.3 @@ -739,106 +739,106 @@ struct 0.0 --- !query 82 +-- !query SELECT smallint(double('32767.4')) --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output 32767 --- !query 83 +-- !query SELECT smallint(double('32767.6')) --- !query 83 schema +-- !query schema struct --- !query 83 output +-- !query output 32767 --- !query 84 +-- !query SELECT smallint(double('-32768.4')) --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output -32768 --- !query 85 +-- !query SELECT smallint(double('-32768.6')) --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output -32768 --- !query 86 +-- !query SELECT int(double('2147483647.4')) --- !query 86 schema +-- !query schema struct --- !query 86 output +-- !query output 2147483647 --- !query 87 +-- !query SELECT int(double('2147483647.6')) --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 2147483647 --- !query 88 +-- !query SELECT int(double('-2147483648.4')) --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output -2147483648 --- !query 89 +-- !query SELECT int(double('-2147483648.6')) --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output -2147483648 --- !query 90 +-- !query SELECT bigint(double('9223372036854773760')) --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 9223372036854773760 --- !query 91 +-- !query SELECT bigint(double('9223372036854775807')) --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output 9223372036854775807 --- !query 92 +-- !query SELECT bigint(double('-9223372036854775808.5')) --- !query 92 schema +-- !query schema struct --- !query 92 output +-- !query output -9223372036854775808 --- !query 93 +-- !query SELECT bigint(double('-9223372036854780000')) --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output java.lang.ArithmeticException Casting -9.22337203685478E18 to long causes overflow --- !query 94 +-- !query DROP TABLE FLOAT8_TBL --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out index 6e77ab207c94a..24fd9dcbfc826 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/groupingsets.sql.out @@ -2,28 +2,28 @@ -- Number of queries: 54 --- !query 0 +-- !query create temp view gstest1(a,b,v) as values (1,1,10),(1,1,11),(1,2,12),(1,2,13),(1,3,14), (2,3,15), (3,3,16),(3,4,17), (4,1,18),(4,1,19) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create table gstest2 (a integer, b integer, c integer, d integer, e integer, f integer, g integer, h integer) using parquet --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query insert into gstest2 values (1, 1, 1, 1, 1, 1, 1, 1), (1, 1, 1, 1, 1, 1, 1, 2), @@ -34,65 +34,65 @@ insert into gstest2 values (1, 1, 2, 2, 2, 2, 2, 2), (1, 2, 2, 2, 2, 2, 2, 2), (2, 2, 2, 2, 2, 2, 2, 2) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query create table gstest3 (a integer, b integer, c integer, d integer) using parquet --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query insert into gstest3 values (1, 1, 1, 1), (2, 2, 2, 2) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query create table gstest4(id integer, v integer, unhashable_col /* bit(4) */ byte, unsortable_col /* xid */ integer) using parquet --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query insert into gstest4 values (1,1,tinyint('0'),1), (2,2,tinyint('1'),1), (3,4,tinyint('2'),2), (4,8,tinyint('3'),2), (5,16,tinyint('0'),2), (6,32,tinyint('1'),2), (7,64,tinyint('2'),1), (8,128,tinyint('3'),1) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query create table gstest_empty (a integer, b integer, v integer) using parquet --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by rollup (a,b) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 1 0 0 21 2 11 1 2 0 0 25 2 13 1 3 0 0 14 1 14 @@ -107,12 +107,12 @@ struct --- !query 9 output +-- !query output NULL NULL 1 1 145 10 19 1 NULL 0 1 60 5 14 1 1 0 0 21 2 11 @@ -127,12 +127,12 @@ NULL NULL 1 1 145 10 19 4 1 0 0 37 2 19 --- !query 10 +-- !query select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by rollup (a,b) order by b desc, a --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 3 4 0 0 17 1 17 1 3 0 0 14 1 14 2 3 0 0 15 1 15 @@ -147,12 +147,12 @@ NULL NULL 1 1 145 10 19 4 NULL 0 1 37 2 19 --- !query 11 +-- !query select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by rollup (a,b) order by coalesce(a,0)+coalesce(b,0) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output NULL NULL 1 1 145 10 19 1 NULL 0 1 60 5 14 1 1 0 0 21 2 11 @@ -167,12 +167,12 @@ NULL NULL 1 1 145 10 19 3 4 0 0 17 1 17 --- !query 12 +-- !query select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum from gstest2 group by rollup (a,b) order by rsum, a, b --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output NULL NULL 12 12 1 NULL 10 22 1 1 8 30 @@ -181,54 +181,54 @@ NULL NULL 12 12 2 2 2 36 --- !query 13 +-- !query select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output --- !query 14 +-- !query select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),()) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output --- !query 15 +-- !query select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output --- !query 16 +-- !query select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output --- !query 17 +-- !query select t1.a, t2.b, sum(t1.v), count(*) from gstest_empty t1, gstest_empty t2 group by grouping sets ((t1.a,t2.b),()) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output --- !query 18 +-- !query select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) from gstest1 t1, gstest2 t2 group by grouping sets ((t1.a, t2.b), ()) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 1 0 0 420 1 1 2 0 0 120 2 2 1 0 0 105 1 @@ -240,50 +240,50 @@ struct --- !query 19 output +-- !query output 1 1 0 0 420 1 1 2 0 0 60 1 2 2 0 0 15 2 NULL NULL 1 1 495 2 --- !query 20 +-- !query select a, b, grouping(a), grouping(b), sum(t1.v), max(t2.c) from gstest1 t1 join gstest2 t2 using (a,b) group by grouping sets ((a, b), ()) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1 1 0 0 147 2 1 2 0 0 25 2 NULL NULL 1 1 172 2 --- !query 21 +-- !query select four, x from (select four, ten, 'foo' as x from tenk1) as t group by grouping sets (four, x) having x = 'foo' --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL foo --- !query 22 +-- !query select four, x || 'x' from (select four, ten, 'foo' as x from tenk1) as t group by grouping sets (four, x) order by four --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL foox 0 NULL 1 NULL @@ -291,38 +291,38 @@ NULL foox 3 NULL --- !query 23 +-- !query select (x+y)*1, sum(z) from (select 1 as x, 2 as y, 3 as z) s group by grouping sets (x+y, x) --- !query 23 schema +-- !query schema struct<((x + y) * 1):int,sum(z):bigint> --- !query 23 output +-- !query output 3 3 NULL 3 --- !query 24 +-- !query CREATE TEMP VIEW int8_tbl AS SELECT * FROM VALUES (123L, 456L), (123L, 4567890123456789L), (4567890123456789L, 123L), (4567890123456789L, 4567890123456789L), (4567890123456789L, -4567890123456789L) as int8_tbl(q1, q2) --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query select x, not x as not_x, q2 from (select *, q1 = 1 as x from int8_tbl i1) as t group by grouping sets(x, q2) order by x, q2 --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output NULL NULL -4567890123456789 NULL NULL 123 NULL NULL 456 @@ -330,21 +330,21 @@ NULL NULL 4567890123456789 false true NULL --- !query 26 +-- !query DROP VIEW int8_tbl --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query select ten, sum(distinct four) from onek a group by grouping sets((ten,four),(ten)) having exists (select 1 from onek b where sum(distinct a.four) = b.four) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 0 0 0 2 0 2 @@ -372,32 +372,32 @@ struct 9 3 --- !query 28 +-- !query select a,count(*) from gstest2 group by rollup(a) order by a --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output NULL 9 1 8 2 1 --- !query 29 +-- !query select a,count(*) from gstest2 group by rollup(a) having a is distinct from 1 order by a --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output NULL 9 2 1 --- !query 30 +-- !query select ten, grouping(ten) from onek group by grouping sets(ten) having grouping(ten) >= 0 order by 2,1 --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 0 0 1 0 2 0 @@ -410,44 +410,44 @@ struct 9 0 --- !query 31 +-- !query select ten, grouping(ten) from onek group by grouping sets(ten, four) having grouping(ten) > 0 order by 2,1 --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output NULL 1 NULL 1 NULL 1 NULL 1 --- !query 32 +-- !query select ten, grouping(ten) from onek group by rollup(ten) having grouping(ten) > 0 order by 2,1 --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output NULL 1 --- !query 33 +-- !query select ten, grouping(ten) from onek group by cube(ten) having grouping(ten) > 0 order by 2,1 --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output NULL 1 --- !query 34 +-- !query select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col) --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 1 1 1 @@ -463,12 +463,12 @@ struct 8 --- !query 35 +-- !query select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by grouping sets ((a),(b)) order by 3,4,1,2 /* 3,1,2 */ --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 1 NULL 0 1 60 5 14 2 NULL 0 1 15 1 15 3 NULL 0 1 33 2 17 @@ -479,12 +479,12 @@ NULL 3 1 0 45 3 16 NULL 4 1 0 17 1 17 --- !query 36 +-- !query select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by cube(a,b) order by 3,4,1,2 /* 3,1,2 */ --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 1 1 0 0 21 2 11 1 2 0 0 25 2 13 1 3 0 0 14 1 14 @@ -503,28 +503,28 @@ NULL 4 1 0 17 1 17 NULL NULL 1 1 145 10 19 --- !query 37 +-- !query select unsortable_col, count(*) from gstest4 group by grouping sets ((unsortable_col),(unsortable_col)) order by string(unsortable_col) --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 1 4 1 4 2 4 2 4 --- !query 38 +-- !query select unhashable_col, unsortable_col, grouping(unhashable_col), grouping(unsortable_col), count(*), sum(v) from gstest4 group by grouping sets ((unhashable_col),(unsortable_col)) order by 3, 4, 6 /* 3, 5 */ --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 0 NULL 0 1 2 17 1 NULL 0 1 2 34 2 NULL 0 1 2 68 @@ -533,15 +533,15 @@ NULL 2 1 0 4 60 NULL 1 1 0 4 195 --- !query 39 +-- !query select unhashable_col, unsortable_col, grouping(unhashable_col), grouping(unsortable_col), count(*), sum(v) from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col)) order by 3, 4, 6 /* 3,5 */ --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 0 NULL 0 1 1 1 1 NULL 0 1 1 2 2 NULL 0 1 1 4 @@ -560,44 +560,44 @@ NULL 1 1 0 1 64 NULL 1 1 0 1 128 --- !query 40 +-- !query select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a) --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output --- !query 41 +-- !query select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),()) --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output --- !query 42 +-- !query select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()) --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output --- !query 43 +-- !query select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()) --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output --- !query 44 +-- !query select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,4,7 /* 3,6 */ --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 1 1 0 0 21 2 11 1 2 0 0 25 2 13 1 3 0 0 14 1 14 @@ -621,12 +621,12 @@ NULL NULL 1 1 37 2 19 NULL NULL 1 1 37 2 19 --- !query 45 +-- !query select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum from gstest2 group by cube (a,b) order by rsum, a, b --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output NULL NULL 12 12 NULL 1 8 20 NULL 2 4 24 @@ -637,11 +637,11 @@ NULL 2 4 24 2 2 2 48 --- !query 46 +-- !query SELECT a, b, count(*), max(a), max(b) FROM gstest3 GROUP BY GROUPING SETS(a, b,()) ORDER BY a, b --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output NULL NULL 2 2 2 NULL 1 1 1 1 NULL 2 1 2 2 @@ -649,67 +649,67 @@ NULL 2 1 2 2 2 NULL 1 2 2 --- !query 47 +-- !query select v||'a', case grouping(v||'a') when 1 then 1 else 0 end, count(*) from values (1, 'a'), (1, 'b') u(i,v) group by rollup(i, v||'a') order by 1,3 --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output NULL 1 2 NULL 1 2 aa 0 1 ba 0 1 --- !query 48 +-- !query select v||'a', case when grouping(v||'a') = 1 then 1 else 0 end, count(*) from values (1, 'a'), (1, 'b') u(i,v) group by rollup(i, v||'a') order by 1,3 --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output NULL 1 2 NULL 1 2 aa 0 1 ba 0 1 --- !query 49 +-- !query DROP VIEW gstest1 --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output --- !query 50 +-- !query DROP TABLE gstest2 --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output --- !query 51 +-- !query DROP TABLE gstest3 --- !query 51 schema +-- !query schema struct<> --- !query 51 output +-- !query output --- !query 52 +-- !query DROP TABLE gstest4 --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output --- !query 53 +-- !query DROP TABLE gstest_empty --- !query 53 schema +-- !query schema struct<> --- !query 53 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/insert.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/insert.sql.out index e83879f631c0b..1046d0ec86bbd 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/insert.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/insert.sql.out @@ -2,70 +2,70 @@ -- Number of queries: 9 --- !query 0 +-- !query create table inserttest (col1 int, col2 int /* NOT NULL */, col3 string /* default 'testing' */) using parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query insert into inserttest values (NULL, 3, 'testing') --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query insert into inserttest values (NULL, 5, 'testing') --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query insert into inserttest values (NULL, 5, 'test') --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query insert into inserttest values (NULL, 7, 'testing') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query select * from inserttest --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output NULL 3 testing NULL 5 test NULL 5 testing NULL 7 testing --- !query 6 +-- !query insert into inserttest values(30, 50, repeat('x', 10000)) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query select col1, col2, char_length(col3) from inserttest --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 30 50 10000 NULL 3 7 NULL 5 4 @@ -73,9 +73,9 @@ NULL 5 7 NULL 7 7 --- !query 8 +-- !query drop table inserttest --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out index d6e9f7ec79d68..02e373f2d2b2b 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -2,59 +2,59 @@ -- Number of queries: 35 --- !query 0 +-- !query CREATE TABLE INT2_TBL(f1 smallint) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO INT2_TBL VALUES (smallint(trim('0 '))) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 '))) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234'))) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO INT2_TBL VALUES (smallint('32767')) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO INT2_TBL VALUES (smallint('-32767')) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT '' AS five, * FROM INT2_TBL --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output -1234 -32767 0 @@ -62,154 +62,154 @@ struct 32767 --- !query 7 +-- !query SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> smallint('0') --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output -1234 -32767 1234 32767 --- !query 8 +-- !query SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> int('0') --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output -1234 -32767 1234 32767 --- !query 9 +-- !query SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = smallint('0') --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 0 --- !query 10 +-- !query SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = int('0') --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 0 --- !query 11 +-- !query SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < smallint('0') --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output -1234 -32767 --- !query 12 +-- !query SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < int('0') --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output -1234 -32767 --- !query 13 +-- !query SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= smallint('0') --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output -1234 -32767 0 --- !query 14 +-- !query SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= int('0') --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output -1234 -32767 0 --- !query 15 +-- !query SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > smallint('0') --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1234 32767 --- !query 16 +-- !query SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > int('0') --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1234 32767 --- !query 17 +-- !query SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= smallint('0') --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 0 1234 32767 --- !query 18 +-- !query SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= int('0') --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 0 1234 32767 --- !query 19 +-- !query SELECT '' AS one, i.* FROM INT2_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 32767 --- !query 20 +-- !query SELECT '' AS three, i.* FROM INT2_TBL i WHERE (i.f1 % int('2')) = smallint('0') --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output -1234 0 1234 --- !query 21 +-- !query SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i WHERE abs(f1) < 16384 --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output -1234 -2468 0 0 1234 2468 --- !query 22 +-- !query SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT2_TBL i --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output -1234 -2468 -32767 -65534 0 0 @@ -217,23 +217,23 @@ struct 32767 65534 --- !query 23 +-- !query SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i WHERE f1 < 32766 --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output -1234 -1232 -32767 -32765 0 2 1234 1236 --- !query 24 +-- !query SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT2_TBL i --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output -1234 -1232 -32767 -32765 0 2 @@ -241,23 +241,23 @@ struct 32767 32769 --- !query 25 +-- !query SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i WHERE f1 > -32767 --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output -1234 -1236 0 -2 1234 1232 32767 32765 --- !query 26 +-- !query SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output -1234 -1236 -32767 -32769 0 -2 @@ -265,11 +265,11 @@ struct 32767 32765 --- !query 27 +-- !query SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output -1234 -617.0 -32767 -16383.5 0 0.0 @@ -277,11 +277,11 @@ struct 32767 16383.5 --- !query 28 +-- !query SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output -1234 -617.0 -32767 -16383.5 0 0.0 @@ -289,31 +289,31 @@ struct 32767 16383.5 --- !query 29 +-- !query SELECT string(shiftleft(smallint(-1), 15)) --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output -32768 --- !query 30 +-- !query SELECT string(smallint(shiftleft(smallint(-1), 15))+1) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output -32767 --- !query 31 +-- !query SELECT smallint(-32768) % smallint(-1) --- !query 31 schema +-- !query schema struct<(CAST(-32768 AS SMALLINT) % CAST(-1 AS SMALLINT)):smallint> --- !query 31 output +-- !query output 0 --- !query 32 +-- !query SELECT x, smallint(x) AS int2_value FROM (VALUES float(-2.5), float(-1.5), @@ -322,9 +322,9 @@ FROM (VALUES float(-2.5), float(0.5), float(1.5), float(2.5)) t(x) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output -0.5 0 -1.5 -1 -2.5 -2 @@ -334,7 +334,7 @@ struct 2.5 2 --- !query 33 +-- !query SELECT x, smallint(x) AS int2_value FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), @@ -343,9 +343,9 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(0.5 as decimal(38, 18)), cast(1.5 as decimal(38, 18)), cast(2.5 as decimal(38, 18))) t(x) --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output -0.500000000000000000 0 -1.500000000000000000 -1 -2.500000000000000000 -2 @@ -355,9 +355,9 @@ struct 2.500000000000000000 2 --- !query 34 +-- !query DROP TABLE INT2_TBL --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index e1cb96d09ec75..3d80c5d595d53 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -2,59 +2,59 @@ -- Number of queries: 53 --- !query 0 +-- !query CREATE TABLE INT4_TBL(f1 int) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO INT4_TBL VALUES (int(trim(' 0 '))) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO INT4_TBL VALUES (int(trim('123456 '))) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO INT4_TBL VALUES (int(trim(' -123456'))) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO INT4_TBL VALUES (int('2147483647')) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO INT4_TBL VALUES (int('-2147483647')) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT '' AS five, * FROM INT4_TBL --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output -123456 -2147483647 0 @@ -62,267 +62,267 @@ struct 2147483647 --- !query 7 +-- !query SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0') --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output -123456 -2147483647 123456 2147483647 --- !query 8 +-- !query SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0') --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output -123456 -2147483647 123456 2147483647 --- !query 9 +-- !query SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0') --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 0 --- !query 10 +-- !query SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0') --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 0 --- !query 11 +-- !query SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0') --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output -123456 -2147483647 --- !query 12 +-- !query SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0') --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output -123456 -2147483647 --- !query 13 +-- !query SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0') --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output -123456 -2147483647 0 --- !query 14 +-- !query SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0') --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output -123456 -2147483647 0 --- !query 15 +-- !query SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0') --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 123456 2147483647 --- !query 16 +-- !query SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0') --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 123456 2147483647 --- !query 17 +-- !query SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0') --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 0 123456 2147483647 --- !query 18 +-- !query SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0') --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 0 123456 2147483647 --- !query 19 +-- !query SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 2147483647 --- !query 20 +-- !query SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0') --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output -123456 0 123456 --- !query 21 +-- !query SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 22 +-- !query SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i WHERE abs(f1) < 1073741824 --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output -123456 -246912 0 0 123456 246912 --- !query 23 +-- !query SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 24 +-- !query SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i WHERE abs(f1) < 1073741824 --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output -123456 -246912 0 0 123456 246912 --- !query 25 +-- !query SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 26 +-- !query SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i WHERE f1 < 2147483646 --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output -123456 -123454 -2147483647 -2147483645 0 2 123456 123458 --- !query 27 +-- !query SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 28 +-- !query SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i WHERE f1 < 2147483646 --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output -123456 -123454 -2147483647 -2147483645 0 2 123456 123458 --- !query 29 +-- !query SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 30 +-- !query SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i WHERE f1 > -2147483647 --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output -123456 -123458 0 -2 123456 123454 2147483647 2147483645 --- !query 31 +-- !query SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output java.lang.ArithmeticException integer overflow --- !query 32 +-- !query SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i WHERE f1 > -2147483647 --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output -123456 -123458 0 -2 123456 123454 2147483647 2147483645 --- !query 33 +-- !query SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output -123456 -61728.0 -2147483647 -1.0737418235E9 0 0.0 @@ -330,11 +330,11 @@ struct 2147483647 1.0737418235E9 --- !query 34 +-- !query SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output -123456 -61728.0 -2147483647 -1.0737418235E9 0 0.0 @@ -342,127 +342,127 @@ struct 2147483647 1.0737418235E9 --- !query 35 +-- !query SELECT -2+3 AS one --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 1 --- !query 36 +-- !query SELECT 4-2 AS two --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 2 --- !query 37 +-- !query SELECT 2- -1 AS three --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 3 --- !query 38 +-- !query SELECT 2 - -2 AS four --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 4 --- !query 39 +-- !query SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output true --- !query 40 +-- !query SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output true --- !query 41 +-- !query SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output true --- !query 42 +-- !query SELECT int('1000') < int('999') AS `false` --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output false --- !query 43 +-- !query SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 10 --- !query 44 +-- !query SELECT 2 + 2 / 2 AS three --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 3.0 --- !query 45 +-- !query SELECT (2 + 2) / 2 AS two --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 2.0 --- !query 46 +-- !query SELECT string(shiftleft(int(-1), 31)) --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output -2147483648 --- !query 47 +-- !query SELECT string(int(shiftleft(int(-1), 31))+1) --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output -2147483647 --- !query 48 +-- !query SELECT int(-2147483648) % int(-1) --- !query 48 schema +-- !query schema struct<(CAST(-2147483648 AS INT) % CAST(-1 AS INT)):int> --- !query 48 output +-- !query output 0 --- !query 49 +-- !query SELECT int(-2147483648) % smallint(-1) --- !query 49 schema +-- !query schema struct<(CAST(-2147483648 AS INT) % CAST(CAST(-1 AS SMALLINT) AS INT)):int> --- !query 49 output +-- !query output 0 --- !query 50 +-- !query SELECT x, int(x) AS int4_value FROM (VALUES double(-2.5), double(-1.5), @@ -471,9 +471,9 @@ FROM (VALUES double(-2.5), double(0.5), double(1.5), double(2.5)) t(x) --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output -0.5 0 -1.5 -1 -2.5 -2 @@ -483,7 +483,7 @@ struct 2.5 2 --- !query 51 +-- !query SELECT x, int(x) AS int4_value FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), @@ -492,9 +492,9 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(0.5 as decimal(38, 18)), cast(1.5 as decimal(38, 18)), cast(2.5 as decimal(38, 18))) t(x) --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output -0.500000000000000000 0 -1.500000000000000000 -1 -2.500000000000000000 -2 @@ -504,9 +504,9 @@ struct 2.500000000000000000 2 --- !query 52 +-- !query DROP TABLE INT4_TBL --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index 8e2700caa3345..18b0c821ae70f 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -2,59 +2,59 @@ -- Number of queries: 85 --- !query 0 +-- !query CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT * FROM INT8_TBL --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 123 456 123 4567890123456789 4567890123456789 -4567890123456789 @@ -62,48 +62,48 @@ struct 4567890123456789 4567890123456789 --- !query 7 +-- !query SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 123 4567890123456789 4567890123456789 4567890123456789 --- !query 8 +-- !query SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 123 456 4567890123456789 -4567890123456789 4567890123456789 123 --- !query 9 +-- !query SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 123 456 4567890123456789 -4567890123456789 4567890123456789 123 --- !query 10 +-- !query SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 123 456 123 4567890123456789 4567890123456789 -4567890123456789 @@ -111,114 +111,114 @@ struct 4567890123456789 4567890123456789 --- !query 12 +-- !query SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 123 4567890123456789 4567890123456789 4567890123456789 --- !query 13 +-- !query SELECT * FROM INT8_TBL WHERE q2 = 456 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 123 456 --- !query 14 +-- !query SELECT * FROM INT8_TBL WHERE q2 <> 456 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 123 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 --- !query 15 +-- !query SELECT * FROM INT8_TBL WHERE q2 < 456 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 4567890123456789 -4567890123456789 4567890123456789 123 --- !query 16 +-- !query SELECT * FROM INT8_TBL WHERE q2 > 456 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 123 4567890123456789 4567890123456789 4567890123456789 --- !query 17 +-- !query SELECT * FROM INT8_TBL WHERE q2 <= 456 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 123 456 4567890123456789 -4567890123456789 4567890123456789 123 --- !query 18 +-- !query SELECT * FROM INT8_TBL WHERE q2 >= 456 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 123 456 123 4567890123456789 4567890123456789 4567890123456789 --- !query 19 +-- !query SELECT * FROM INT8_TBL WHERE 123 = q1 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 123 456 123 4567890123456789 --- !query 20 +-- !query SELECT * FROM INT8_TBL WHERE 123 <> q1 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 4567890123456789 -4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 --- !query 21 +-- !query SELECT * FROM INT8_TBL WHERE 123 < q1 --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 4567890123456789 -4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 --- !query 22 +-- !query SELECT * FROM INT8_TBL WHERE 123 > q1 --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output --- !query 23 +-- !query SELECT * FROM INT8_TBL WHERE 123 <= q1 --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 123 456 123 4567890123456789 4567890123456789 -4567890123456789 @@ -226,114 +226,114 @@ struct 4567890123456789 4567890123456789 --- !query 24 +-- !query SELECT * FROM INT8_TBL WHERE 123 >= q1 --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 123 456 123 4567890123456789 --- !query 25 +-- !query SELECT * FROM INT8_TBL WHERE q2 = smallint('456') --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 123 456 --- !query 26 +-- !query SELECT * FROM INT8_TBL WHERE q2 <> smallint('456') --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 123 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 --- !query 27 +-- !query SELECT * FROM INT8_TBL WHERE q2 < smallint('456') --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 4567890123456789 -4567890123456789 4567890123456789 123 --- !query 28 +-- !query SELECT * FROM INT8_TBL WHERE q2 > smallint('456') --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 123 4567890123456789 4567890123456789 4567890123456789 --- !query 29 +-- !query SELECT * FROM INT8_TBL WHERE q2 <= smallint('456') --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 123 456 4567890123456789 -4567890123456789 4567890123456789 123 --- !query 30 +-- !query SELECT * FROM INT8_TBL WHERE q2 >= smallint('456') --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 123 456 123 4567890123456789 4567890123456789 4567890123456789 --- !query 31 +-- !query SELECT * FROM INT8_TBL WHERE smallint('123') = q1 --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 123 456 123 4567890123456789 --- !query 32 +-- !query SELECT * FROM INT8_TBL WHERE smallint('123') <> q1 --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 4567890123456789 -4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 --- !query 33 +-- !query SELECT * FROM INT8_TBL WHERE smallint('123') < q1 --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 4567890123456789 -4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 --- !query 34 +-- !query SELECT * FROM INT8_TBL WHERE smallint('123') > q1 --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output --- !query 35 +-- !query SELECT * FROM INT8_TBL WHERE smallint('123') <= q1 --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 123 456 123 4567890123456789 4567890123456789 -4567890123456789 @@ -341,20 +341,20 @@ struct 4567890123456789 4567890123456789 --- !query 36 +-- !query SELECT * FROM INT8_TBL WHERE smallint('123') >= q1 --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 123 456 123 4567890123456789 --- !query 37 +-- !query SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 123 -123 123 -123 4567890123456789 -4567890123456789 @@ -362,11 +362,11 @@ struct 4567890123456789 -4567890123456789 --- !query 38 +-- !query SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 123 456 579 123 4567890123456789 4567890123456912 4567890123456789 -4567890123456789 0 @@ -374,11 +374,11 @@ struct 4567890123456789 4567890123456789 9135780246913578 --- !query 39 +-- !query SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 123 456 -333 123 4567890123456789 -4567890123456666 4567890123456789 -4567890123456789 9135780246913578 @@ -386,31 +386,31 @@ struct 4567890123456789 4567890123456789 0 --- !query 40 +-- !query SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output java.lang.ArithmeticException long overflow --- !query 41 +-- !query SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL WHERE q1 < 1000 or (q2 > 0 and q2 < 1000) --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 123 456 56088 123 4567890123456789 561850485185185047 4567890123456789 123 561850485185185047 --- !query 42 +-- !query SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 123 456 0.26973684210526316 123 123 4567890123456789 2.6927092525360204E-14 123 4567890123456789 -4567890123456789 -1.0 0 @@ -418,11 +418,11 @@ struct 4567890123456789 4567890123456789 1.0 0 --- !query 43 +-- !query SELECT '' AS five, q1, double(q1) FROM INT8_TBL --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 123 123.0 123 123.0 4567890123456789 4.567890123456789E15 @@ -430,11 +430,11 @@ struct 4567890123456789 4.567890123456789E15 --- !query 44 +-- !query SELECT '' AS five, q2, double(q2) FROM INT8_TBL --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output -4567890123456789 -4.567890123456789E15 123 123.0 456 456.0 @@ -442,11 +442,11 @@ struct 4567890123456789 4.567890123456789E15 --- !query 45 +-- !query SELECT 37 + q1 AS plus4 FROM INT8_TBL --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 160 160 4567890123456826 @@ -454,11 +454,11 @@ struct 4567890123456826 --- !query 46 +-- !query SELECT 37 - q1 AS minus4 FROM INT8_TBL --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output -4567890123456752 -4567890123456752 -4567890123456752 @@ -466,11 +466,11 @@ struct -86 --- !query 47 +-- !query SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 246 246 9135780246913578 @@ -478,11 +478,11 @@ struct 9135780246913578 --- !query 48 +-- !query SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output 246 246 9135780246913578 @@ -490,11 +490,11 @@ struct 9135780246913578 --- !query 49 +-- !query SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL --- !query 49 schema +-- !query schema struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:double> --- !query 49 output +-- !query output 165 81 5166 2.9285714285714284 165 81 5166 2.9285714285714284 4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 @@ -502,11 +502,11 @@ struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:double> 4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 --- !query 50 +-- !query SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL --- !query 50 schema +-- !query schema struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:double> --- !query 50 output +-- !query output 369 123 30258 2.0 369 123 30258 2.0 4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 @@ -514,11 +514,11 @@ struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:double> 4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 --- !query 51 +-- !query SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL --- !query 51 schema +-- !query schema struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:double> --- !query 51 output +-- !query output 165 81 5166 2.9285714285714284 165 81 5166 2.9285714285714284 4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 @@ -526,11 +526,11 @@ struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:double> 4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 --- !query 52 +-- !query SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL --- !query 52 schema +-- !query schema struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:double> --- !query 52 output +-- !query output 369 123 30258 2.0 369 123 30258 2.0 4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 @@ -538,11 +538,11 @@ struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:double> 4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 --- !query 53 +-- !query SELECT q2, abs(q2) FROM INT8_TBL --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output -4567890123456789 4567890123456789 123 123 456 456 @@ -550,93 +550,93 @@ struct 4567890123456789 4567890123456789 --- !query 54 +-- !query SELECT min(q1), min(q2) FROM INT8_TBL --- !query 54 schema +-- !query schema struct --- !query 54 output +-- !query output 123 -4567890123456789 --- !query 55 +-- !query SELECT max(q1), max(q2) FROM INT8_TBL --- !query 55 schema +-- !query schema struct --- !query 55 output +-- !query output 4567890123456789 4567890123456789 --- !query 56 +-- !query select bigint('9223372036854775800') / bigint('0') --- !query 56 schema +-- !query schema struct<(CAST(CAST(9223372036854775800 AS BIGINT) AS DOUBLE) / CAST(CAST(0 AS BIGINT) AS DOUBLE)):double> --- !query 56 output +-- !query output NULL --- !query 57 +-- !query select bigint('-9223372036854775808') / smallint('0') --- !query 57 schema +-- !query schema struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(0 AS SMALLINT) AS DOUBLE)):double> --- !query 57 output +-- !query output NULL --- !query 58 +-- !query select smallint('100') / bigint('0') --- !query 58 schema +-- !query schema struct<(CAST(CAST(100 AS SMALLINT) AS DOUBLE) / CAST(CAST(0 AS BIGINT) AS DOUBLE)):double> --- !query 58 output +-- !query output NULL --- !query 59 +-- !query SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456 --- !query 59 schema +-- !query schema struct --- !query 59 output +-- !query output 123 --- !query 60 +-- !query SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output java.lang.ArithmeticException Casting 4567890123456789 to int causes overflow --- !query 61 +-- !query SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456 --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output 123 --- !query 62 +-- !query SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 --- !query 62 schema +-- !query schema struct<> --- !query 62 output +-- !query output java.lang.ArithmeticException Casting 4567890123456789 to short causes overflow --- !query 63 +-- !query SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint) --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output 42 -37 --- !query 64 +-- !query SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 123.0 4.567890123456789E15 123.0 456.0 4.5678899E15 -4.567890123456789E15 @@ -644,28 +644,28 @@ struct 4.5678899E15 4.567890123456789E15 --- !query 65 +-- !query SELECT CAST(float('36854775807.0') AS bigint) --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output 36854775808 --- !query 66 +-- !query SELECT CAST(double('922337203685477580700.0') AS bigint) --- !query 66 schema +-- !query schema struct<> --- !query 66 output +-- !query output java.lang.ArithmeticException Casting 9.223372036854776E20 to long causes overflow --- !query 67 +-- !query SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output 123 456 72 507 -124 123 4567890123456789 17 4567890123456895 -124 4567890123456789 -4567890123456789 1 -1 -4567890123456790 @@ -673,11 +673,11 @@ struct 4567890123456789 4567890123456789 4567890123456789 4567890123456789 -4567890123456790 --- !query 68 +-- !query SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')) --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 4567890123456789 4567890123456790 4567890123456791 @@ -690,20 +690,20 @@ struct 4567890123456798 --- !query 69 +-- !query SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0) --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output java.lang.IllegalArgumentException requirement failed: step (0) cannot be 0 --- !query 70 +-- !query SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2) --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output 4567890123456789 4567890123456791 4567890123456793 @@ -711,99 +711,99 @@ struct 4567890123456797 --- !query 71 +-- !query SELECT string(shiftleft(bigint(-1), 63)) --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output -9223372036854775808 --- !query 72 +-- !query SELECT string(int(shiftleft(bigint(-1), 63))+1) --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output java.lang.ArithmeticException Casting -9223372036854775808 to int causes overflow --- !query 73 +-- !query SELECT bigint((-9223372036854775808)) * bigint((-1)) --- !query 73 schema +-- !query schema struct<> --- !query 73 output +-- !query output java.lang.ArithmeticException long overflow --- !query 74 +-- !query SELECT bigint((-9223372036854775808)) / bigint((-1)) --- !query 74 schema +-- !query schema struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS BIGINT) AS DOUBLE)):double> --- !query 74 output +-- !query output 9.223372036854776E18 --- !query 75 +-- !query SELECT bigint((-9223372036854775808)) % bigint((-1)) --- !query 75 schema +-- !query schema struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(-1 AS BIGINT)):bigint> --- !query 75 output +-- !query output 0 --- !query 76 +-- !query SELECT bigint((-9223372036854775808)) * int((-1)) --- !query 76 schema +-- !query schema struct<> --- !query 76 output +-- !query output java.lang.ArithmeticException long overflow --- !query 77 +-- !query SELECT bigint((-9223372036854775808)) / int((-1)) --- !query 77 schema +-- !query schema struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS INT) AS DOUBLE)):double> --- !query 77 output +-- !query output 9.223372036854776E18 --- !query 78 +-- !query SELECT bigint((-9223372036854775808)) % int((-1)) --- !query 78 schema +-- !query schema struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(CAST(-1 AS INT) AS BIGINT)):bigint> --- !query 78 output +-- !query output 0 --- !query 79 +-- !query SELECT bigint((-9223372036854775808)) * smallint((-1)) --- !query 79 schema +-- !query schema struct<> --- !query 79 output +-- !query output java.lang.ArithmeticException long overflow --- !query 80 +-- !query SELECT bigint((-9223372036854775808)) / smallint((-1)) --- !query 80 schema +-- !query schema struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS SMALLINT) AS DOUBLE)):double> --- !query 80 output +-- !query output 9.223372036854776E18 --- !query 81 +-- !query SELECT bigint((-9223372036854775808)) % smallint((-1)) --- !query 81 schema +-- !query schema struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> --- !query 81 output +-- !query output 0 --- !query 82 +-- !query SELECT x, bigint(x) AS int8_value FROM (VALUES (double(-2.5)), (double(-1.5)), @@ -812,9 +812,9 @@ FROM (VALUES (double(-2.5)), (double(0.5)), (double(1.5)), (double(2.5))) t(x) --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output -0.5 0 -1.5 -1 -2.5 -2 @@ -824,7 +824,7 @@ struct 2.5 2 --- !query 83 +-- !query SELECT x, bigint(x) AS int8_value FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), @@ -833,9 +833,9 @@ FROM (VALUES cast(-2.5 as decimal(38, 18)), cast(0.5 as decimal(38, 18)), cast(1.5 as decimal(38, 18)), cast(2.5 as decimal(38, 18))) t(x) --- !query 83 schema +-- !query schema struct --- !query 83 output +-- !query output -0.500000000000000000 0 -1.500000000000000000 -1 -2.500000000000000000 -2 @@ -845,9 +845,9 @@ struct 2.500000000000000000 2 --- !query 84 +-- !query DROP TABLE INT8_TBL --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index cb906c53c84e7..4bd846d3ff923 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -2,107 +2,107 @@ -- Number of queries: 24 --- !query 0 +-- !query SELECT interval '999' second --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 16 minutes 39 seconds --- !query 1 +-- !query SELECT interval '999' minute --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 16 hours 39 minutes --- !query 2 +-- !query SELECT interval '999' hour --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 999 hours --- !query 3 +-- !query SELECT interval '999' day --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 999 days --- !query 4 +-- !query SELECT interval '999' month --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 83 years 3 months --- !query 5 +-- !query SELECT interval '1' year --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 years --- !query 6 +-- !query SELECT interval '2' month --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 2 months --- !query 7 +-- !query SELECT interval '3' day --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 3 days --- !query 8 +-- !query SELECT interval '4' hour --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 4 hours --- !query 9 +-- !query SELECT interval '5' minute --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 5 minutes --- !query 10 +-- !query SELECT interval '6' second --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 6 seconds --- !query 11 +-- !query SELECT interval '1-2' year to month --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 years 2 months --- !query 12 +-- !query SELECT interval '1 2:03' day to hour --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 1 2:03(line 1, pos 16) @@ -112,11 +112,11 @@ SELECT interval '1 2:03' day to hour ----------------^^^ --- !query 13 +-- !query SELECT interval '1 2:03:04' day to hour --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 1 2:03:04(line 1, pos 16) @@ -126,19 +126,19 @@ SELECT interval '1 2:03:04' day to hour ----------------^^^ --- !query 14 +-- !query SELECT interval '1 2:03' day to minute --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 days 2 hours 3 minutes --- !query 15 +-- !query SELECT interval '1 2:03:04' day to minute --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 1 2:03:04(line 1, pos 16) @@ -148,11 +148,11 @@ SELECT interval '1 2:03:04' day to minute ----------------^^^ --- !query 16 +-- !query SELECT interval '1 2:03' day to second --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) @@ -162,19 +162,19 @@ SELECT interval '1 2:03' day to second ----------------^^^ --- !query 17 +-- !query SELECT interval '1 2:03:04' day to second --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 days 2 hours 3 minutes 4 seconds --- !query 18 +-- !query SELECT interval '1 2:03' hour to minute --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 1 2:03(line 1, pos 16) @@ -184,11 +184,11 @@ SELECT interval '1 2:03' hour to minute ----------------^^^ --- !query 19 +-- !query SELECT interval '1 2:03:04' hour to minute --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 1 2:03:04(line 1, pos 16) @@ -198,11 +198,11 @@ SELECT interval '1 2:03:04' hour to minute ----------------^^^ --- !query 20 +-- !query SELECT interval '1 2:03' hour to second --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) @@ -212,11 +212,11 @@ SELECT interval '1 2:03' hour to second ----------------^^^ --- !query 21 +-- !query SELECT interval '1 2:03:04' hour to second --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03:04(line 1, pos 16) @@ -226,11 +226,11 @@ SELECT interval '1 2:03:04' hour to second ----------------^^^ --- !query 22 +-- !query SELECT interval '1 2:03' minute to second --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) @@ -240,11 +240,11 @@ SELECT interval '1 2:03' minute to second ----------------^^^ --- !query 23 +-- !query SELECT interval '1 2:03:04' minute to second --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03:04(line 1, pos 16) diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out index f823d4e1459d4..5332dfff9f101 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out @@ -2,17 +2,17 @@ -- Number of queries: 181 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) AS v(f1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (VALUES (123, 456), @@ -21,230 +21,230 @@ CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (4567890123456789, 4567890123456789), (4567890123456789, -4567890123456789)) AS v(q1, q2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM (VALUES (0.0), (1004.30), (-34.84), (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) AS v(f1) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM (VALUES ('doh!'), ('hi de ho neighbor')) AS v(f1) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TABLE J1_TBL ( i integer, j integer, t string ) USING parquet --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query CREATE TABLE J2_TBL ( i integer, k integer ) USING parquet --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO J1_TBL VALUES (1, 4, 'one') --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO J1_TBL VALUES (2, 3, 'two') --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO J1_TBL VALUES (3, 2, 'three') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO J1_TBL VALUES (4, 1, 'four') --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query INSERT INTO J1_TBL VALUES (5, 0, 'five') --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query INSERT INTO J1_TBL VALUES (6, 6, 'six') --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query INSERT INTO J1_TBL VALUES (7, 7, 'seven') --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query INSERT INTO J1_TBL VALUES (8, 8, 'eight') --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query INSERT INTO J1_TBL VALUES (0, NULL, 'zero') --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query INSERT INTO J1_TBL VALUES (NULL, NULL, 'null') --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output --- !query 17 +-- !query INSERT INTO J1_TBL VALUES (NULL, 0, 'zero') --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output --- !query 18 +-- !query INSERT INTO J2_TBL VALUES (1, -1) --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query INSERT INTO J2_TBL VALUES (2, 2) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query INSERT INTO J2_TBL VALUES (3, -3) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query INSERT INTO J2_TBL VALUES (2, 4) --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query INSERT INTO J2_TBL VALUES (5, -5) --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output --- !query 23 +-- !query INSERT INTO J2_TBL VALUES (5, -5) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query INSERT INTO J2_TBL VALUES (0, NULL) --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query INSERT INTO J2_TBL VALUES (NULL, NULL) --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query INSERT INTO J2_TBL VALUES (NULL, 0) --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL AS tx --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 0 NULL zero 1 4 one 2 3 two @@ -258,12 +258,12 @@ struct NULL NULL null --- !query 28 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL tx --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 0 NULL zero 1 4 one 2 3 two @@ -277,12 +277,12 @@ struct NULL NULL null --- !query 29 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL AS t1 (a, b, c) --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 0 NULL zero 1 4 one 2 3 two @@ -296,12 +296,12 @@ struct NULL NULL null --- !query 30 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL t1 (a, b, c) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 0 NULL zero 1 4 one 2 3 two @@ -315,12 +315,12 @@ struct NULL NULL null --- !query 31 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 0 NULL zero 0 NULL 0 NULL zero 1 -1 0 NULL zero 2 2 @@ -422,12 +422,12 @@ struct NULL NULL null NULL NULL --- !query 32 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL CROSS JOIN J2_TBL --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 0 NULL zero 0 NULL 0 NULL zero 1 -1 0 NULL zero 2 2 @@ -529,22 +529,22 @@ struct NULL NULL null NULL NULL --- !query 33 +-- !query SELECT '' AS `xxx`, i, k, t FROM J1_TBL CROSS JOIN J2_TBL --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'i' is ambiguous, could be: default.j1_tbl.i, default.j2_tbl.i.; line 1 pos 20 --- !query 34 +-- !query SELECT '' AS `xxx`, t1.i, k, t FROM J1_TBL t1 CROSS JOIN J2_TBL t2 --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 0 -1 zero 0 -3 zero 0 -5 zero @@ -646,13 +646,13 @@ struct NULL NULL zero --- !query 35 +-- !query SELECT '' AS `xxx`, ii, tt, kk FROM (J1_TBL CROSS JOIN J2_TBL) AS tx (ii, jj, tt, ii2, kk) --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 0 zero -1 0 zero -3 0 zero -5 @@ -754,12 +754,12 @@ struct NULL zero NULL --- !query 36 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 0 NULL zero 0 NULL 0 NULL 0 NULL zero 0 NULL 1 -1 0 NULL zero 0 NULL 2 2 @@ -1653,12 +1653,12 @@ struct NULL NULL null NULL NULL NULL NULL --- !query 37 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL INNER JOIN J2_TBL USING (i) --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1668,12 +1668,12 @@ struct 5 0 five -5 --- !query 38 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL JOIN J2_TBL USING (i) --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1683,13 +1683,13 @@ struct 5 0 five -5 --- !query 39 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) ORDER BY a, d --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1699,12 +1699,12 @@ struct 5 0 five -5 --- !query 40 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL NATURAL JOIN J2_TBL --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1714,12 +1714,12 @@ struct 5 0 five -5 --- !query 41 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d) --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1729,23 +1729,23 @@ struct 5 0 five -5 --- !query 42 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a) --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 0 NULL zero NULL 2 3 two 2 4 1 four 2 --- !query 43 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.i) --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 0 NULL zero 0 NULL 1 4 one 1 -1 2 3 two 2 2 @@ -1755,23 +1755,23 @@ struct 5 0 five 5 -5 --- !query 44 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.k) --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 0 NULL zero NULL 0 2 3 two 2 2 4 1 four 2 4 --- !query 45 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i <= J2_TBL.k) --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 0 NULL zero 2 2 0 NULL zero 2 4 0 NULL zero NULL 0 @@ -1783,13 +1783,13 @@ struct 4 1 four 2 4 --- !query 46 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) ORDER BY i, k, t --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output NULL NULL null NULL NULL 0 zero NULL 0 NULL zero NULL @@ -1805,13 +1805,13 @@ struct 8 8 eight NULL --- !query 47 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL LEFT JOIN J2_TBL USING (i) ORDER BY i, k, t --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output NULL NULL null NULL NULL 0 zero NULL 0 NULL zero NULL @@ -1827,12 +1827,12 @@ struct 8 8 eight NULL --- !query 48 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i) --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1844,12 +1844,12 @@ struct NULL NULL NULL NULL --- !query 49 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL RIGHT JOIN J2_TBL USING (i) --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1861,13 +1861,13 @@ struct NULL NULL NULL NULL --- !query 50 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) ORDER BY i, k, t --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output NULL NULL NULL NULL NULL NULL null NULL NULL 0 zero NULL @@ -1885,13 +1885,13 @@ struct 8 8 eight NULL --- !query 51 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL FULL JOIN J2_TBL USING (i) ORDER BY i, k, t --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output NULL NULL NULL NULL NULL NULL null NULL NULL 0 zero NULL @@ -1909,226 +1909,226 @@ struct 8 8 eight NULL --- !query 52 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (k = 1) --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output --- !query 53 +-- !query SELECT '' AS `xxx`, * FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (i = 1) --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 1 4 one -1 --- !query 54 +-- !query CREATE TABLE t1 (name STRING, n INTEGER) USING parquet --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output --- !query 55 +-- !query CREATE TABLE t2 (name STRING, n INTEGER) USING parquet --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output --- !query 56 +-- !query CREATE TABLE t3 (name STRING, n INTEGER) USING parquet --- !query 56 schema +-- !query schema struct<> --- !query 56 output +-- !query output --- !query 57 +-- !query INSERT INTO t1 VALUES ( 'bb', 11 ) --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output --- !query 58 +-- !query INSERT INTO t2 VALUES ( 'bb', 12 ) --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output --- !query 59 +-- !query INSERT INTO t2 VALUES ( 'cc', 22 ) --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output --- !query 60 +-- !query INSERT INTO t2 VALUES ( 'ee', 42 ) --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output --- !query 61 +-- !query INSERT INTO t3 VALUES ( 'bb', 13 ) --- !query 61 schema +-- !query schema struct<> --- !query 61 output +-- !query output --- !query 62 +-- !query INSERT INTO t3 VALUES ( 'cc', 23 ) --- !query 62 schema +-- !query schema struct<> --- !query 62 output +-- !query output --- !query 63 +-- !query INSERT INTO t3 VALUES ( 'dd', 33 ) --- !query 63 schema +-- !query schema struct<> --- !query 63 output +-- !query output --- !query 64 +-- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output bb 11 12 13 cc NULL 22 23 dd NULL NULL 33 ee NULL 42 NULL --- !query 65 +-- !query SELECT * FROM (SELECT * FROM t2) as s2 INNER JOIN (SELECT * FROM t3) s3 USING (name) --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output bb 12 13 cc 22 23 --- !query 66 +-- !query SELECT * FROM (SELECT * FROM t2) as s2 LEFT JOIN (SELECT * FROM t3) s3 USING (name) --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output bb 12 13 cc 22 23 ee 42 NULL --- !query 67 +-- !query SELECT * FROM (SELECT * FROM t2) as s2 FULL JOIN (SELECT * FROM t3) s3 USING (name) --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output bb 12 13 cc 22 23 dd NULL 33 ee 42 NULL --- !query 68 +-- !query SELECT * FROM (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL INNER JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output bb 12 2 13 3 cc 22 2 23 3 --- !query 69 +-- !query SELECT * FROM (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL LEFT JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output bb 12 2 13 3 cc 22 2 23 3 ee 42 2 NULL NULL --- !query 70 +-- !query SELECT * FROM (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL FULL JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output bb 12 2 13 3 cc 22 2 23 3 dd NULL NULL 33 3 ee 42 2 NULL NULL --- !query 71 +-- !query SELECT * FROM (SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 NATURAL INNER JOIN (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL INNER JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output bb 11 1 12 2 13 3 --- !query 72 +-- !query SELECT * FROM (SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 NATURAL FULL JOIN (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL FULL JOIN (SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 --- !query 72 schema +-- !query schema struct --- !query 72 output +-- !query output bb 11 1 12 2 13 3 cc NULL NULL 22 2 23 3 dd NULL NULL NULL NULL 33 3 ee NULL NULL 42 2 NULL NULL --- !query 73 +-- !query SELECT * FROM (SELECT name, n as s1_n FROM t1) as s1 NATURAL FULL JOIN @@ -2137,16 +2137,16 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output bb 11 12 13 cc NULL 22 23 dd NULL NULL 33 ee NULL 42 NULL --- !query 74 +-- !query SELECT * FROM (SELECT name, n as s1_n FROM t1) as s1 NATURAL FULL JOIN @@ -2155,55 +2155,55 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT name, n as s3_n FROM t3) as s3 ) ss2 --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output bb 11 12 2 13 cc NULL 22 2 23 dd NULL NULL NULL 33 ee NULL 42 2 NULL --- !query 75 +-- !query SELECT * FROM (SELECT name, n as s1_n FROM t1) as s1 FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (s1_n = s2_n) --- !query 75 schema +-- !query schema struct --- !query 75 output +-- !query output NULL NULL bb 2 NULL NULL cc 2 NULL NULL ee 2 bb 11 NULL NULL --- !query 76 +-- !query create or replace temporary view x as select * from (values (1,11), (2,22), (3,null), (4,44), (5,null)) as v(x1, x2) --- !query 76 schema +-- !query schema struct<> --- !query 76 output +-- !query output --- !query 77 +-- !query create or replace temporary view y as select * from (values (1,111), (2,222), (3,333), (4,null)) as v(y1, y2) --- !query 77 schema +-- !query schema struct<> --- !query 77 output +-- !query output --- !query 78 +-- !query select * from x --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output 1 11 2 22 3 NULL @@ -2211,22 +2211,22 @@ struct 5 NULL --- !query 79 +-- !query select * from y --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output 1 111 2 222 3 333 4 NULL --- !query 80 +-- !query select * from x left join y on (x1 = y1 and x2 is not null) --- !query 80 schema +-- !query schema struct --- !query 80 output +-- !query output 1 11 1 111 2 22 2 222 3 NULL NULL NULL @@ -2234,11 +2234,11 @@ struct 5 NULL NULL NULL --- !query 81 +-- !query select * from x left join y on (x1 = y1 and y2 is not null) --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output 1 11 1 111 2 22 2 222 3 NULL 3 333 @@ -2246,12 +2246,12 @@ struct 5 NULL NULL NULL --- !query 82 +-- !query select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) on (x1 = xx1) --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 3 NULL @@ -2259,12 +2259,12 @@ struct 5 NULL NULL NULL 5 NULL --- !query 83 +-- !query select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) on (x1 = xx1 and x2 is not null) --- !query 83 schema +-- !query schema struct --- !query 83 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 NULL NULL @@ -2272,12 +2272,12 @@ struct 5 NULL NULL NULL NULL NULL --- !query 84 +-- !query select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) on (x1 = xx1 and y2 is not null) --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 3 NULL @@ -2285,12 +2285,12 @@ struct 5 NULL NULL NULL NULL NULL --- !query 85 +-- !query select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) on (x1 = xx1 and xx2 is not null) --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 NULL NULL @@ -2298,78 +2298,78 @@ struct 5 NULL NULL NULL NULL NULL --- !query 86 +-- !query select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) on (x1 = xx1) where (x2 is not null) --- !query 86 schema +-- !query schema struct --- !query 86 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 4 44 4 NULL 4 44 --- !query 87 +-- !query select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) on (x1 = xx1) where (y2 is not null) --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 3 NULL --- !query 88 +-- !query select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) on (x1 = xx1) where (xx2 is not null) --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 4 44 4 NULL 4 44 --- !query 89 +-- !query select count(*) from tenk1 a where unique1 in (select unique1 from tenk1 b join tenk1 c using (unique1) where b.unique2 = 42) --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output 1 --- !query 90 +-- !query select count(*) from tenk1 x where x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and x.unique1 = 0 and x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1) --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 1 --- !query 91 +-- !query select count(*) from tenk1 x where x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and x.unique1 = 0 and x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1) --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output 1 --- !query 92 +-- !query select * from int8_tbl i1 left join (int8_tbl i2 join (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2 order by 1, 2 --- !query 92 schema +-- !query schema struct --- !query 92 output +-- !query output 123 456 123 456 123 123 4567890123456789 123 4567890123456789 123 4567890123456789 -4567890123456789 NULL NULL NULL @@ -2377,7 +2377,7 @@ struct 4567890123456789 4567890123456789 123 4567890123456789 123 --- !query 93 +-- !query select count(*) from (select t3.tenthous as x1, coalesce(t1.stringu1, t2.stringu1) as x2 @@ -2387,32 +2387,32 @@ from tenk1 t4, tenk1 t5 where t4.thousand = t5.unique1 and ss.x1 = t4.tenthous and ss.x2 = t5.stringu1 --- !query 93 schema +-- !query schema struct --- !query 93 output +-- !query output 1000 --- !query 94 +-- !query select a.f1, b.f1, t.thousand, t.tenthous from tenk1 t, (select sum(f1)+1 as f1 from int4_tbl i4a) a, (select sum(f1) as f1 from int4_tbl i4b) b where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous --- !query 94 schema +-- !query schema struct --- !query 94 output +-- !query output --- !query 95 +-- !query select * from j1_tbl full join (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k --- !query 95 schema +-- !query schema struct --- !query 95 output +-- !query output 0 NULL zero NULL NULL 1 4 one NULL NULL 2 3 two 2 2 @@ -2434,128 +2434,128 @@ NULL NULL NULL NULL NULL NULL NULL null NULL NULL --- !query 96 +-- !query select count(*) from (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x left join (select * from tenk1 y order by y.unique2) y on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2 --- !query 96 schema +-- !query schema struct --- !query 96 output +-- !query output 10000 --- !query 97 +-- !query DROP TABLE t1 --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output --- !query 98 +-- !query DROP TABLE t2 --- !query 98 schema +-- !query schema struct<> --- !query 98 output +-- !query output --- !query 99 +-- !query DROP TABLE t3 --- !query 99 schema +-- !query schema struct<> --- !query 99 output +-- !query output --- !query 100 +-- !query DROP TABLE J1_TBL --- !query 100 schema +-- !query schema struct<> --- !query 100 output +-- !query output --- !query 101 +-- !query DROP TABLE J2_TBL --- !query 101 schema +-- !query schema struct<> --- !query 101 output +-- !query output --- !query 102 +-- !query create or replace temporary view tt1 as select * from (values (1, 11), (2, NULL)) as v(tt1_id, joincol) --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output --- !query 103 +-- !query create or replace temporary view tt2 as select * from (values (21, 11), (22, 11)) as v(tt2_id, joincol) --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output --- !query 104 +-- !query select tt1.*, tt2.* from tt1 left join tt2 on tt1.joincol = tt2.joincol --- !query 104 schema +-- !query schema struct --- !query 104 output +-- !query output 1 11 21 11 1 11 22 11 2 NULL NULL NULL --- !query 105 +-- !query select tt1.*, tt2.* from tt2 right join tt1 on tt1.joincol = tt2.joincol --- !query 105 schema +-- !query schema struct --- !query 105 output +-- !query output 1 11 21 11 1 11 22 11 2 NULL NULL NULL --- !query 106 +-- !query select count(*) from tenk1 a, tenk1 b where a.hundred = b.thousand and (b.fivethous % 10) < 10 --- !query 106 schema +-- !query schema struct --- !query 106 output +-- !query output 100000 --- !query 107 +-- !query create or replace temporary view tt3 as select * from (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) as v(f1, f2) --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output --- !query 108 +-- !query create or replace temporary view tt4 as select * from (values (0), (1), (9999)) as v(f1) --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output --- !query 109 +-- !query SELECT a.f1 FROM tt4 a LEFT JOIN ( @@ -2564,242 +2564,242 @@ LEFT JOIN ( WHERE c.f1 IS NULL ) AS d ON (a.f1 = d.f1) WHERE d.f1 IS NULL --- !query 109 schema +-- !query schema struct --- !query 109 output +-- !query output 0 1 9999 --- !query 110 +-- !query create or replace temporary view tt5 as select * from (values (1, 10), (1, 11)) as v(f1, f2) --- !query 110 schema +-- !query schema struct<> --- !query 110 output +-- !query output --- !query 111 +-- !query create or replace temporary view tt6 as select * from (values (1, 9), (1, 2), (2, 9)) as v(f1, f2) --- !query 111 schema +-- !query schema struct<> --- !query 111 output +-- !query output --- !query 112 +-- !query select * from tt5,tt6 where tt5.f1 = tt6.f1 and tt5.f1 = tt5.f2 - tt6.f2 --- !query 112 schema +-- !query schema struct --- !query 112 output +-- !query output 1 10 1 9 --- !query 113 +-- !query create or replace temporary view xx as select * from (values (1), (2), (3)) as v(pkxx) --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output --- !query 114 +-- !query create or replace temporary view yy as select * from (values (101, 1), (201, 2), (301, NULL)) as v(pkyy, pkxx) --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output --- !query 115 +-- !query select yy.pkyy as yy_pkyy, yy.pkxx as yy_pkxx, yya.pkyy as yya_pkyy, xxa.pkxx as xxa_pkxx, xxb.pkxx as xxb_pkxx from yy left join (SELECT * FROM yy where pkyy = 101) as yya ON yy.pkyy = yya.pkyy left join xx xxa on yya.pkxx = xxa.pkxx left join xx xxb on coalesce (xxa.pkxx, 1) = xxb.pkxx --- !query 115 schema +-- !query schema struct --- !query 115 output +-- !query output 101 1 101 1 1 201 2 NULL NULL 1 301 NULL NULL NULL 1 --- !query 116 +-- !query create or replace temporary view zt1 as select * from (values (53)) as v(f1) --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output --- !query 117 +-- !query create or replace temporary view zt2 as select * from (values (53)) as v(f2) --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output --- !query 118 +-- !query create or replace temporary view zt3(f3 int) using parquet --- !query 118 schema +-- !query schema struct<> --- !query 118 output +-- !query output --- !query 119 +-- !query select * from zt2 left join zt3 on (f2 = f3) left join zt1 on (f3 = f1) where f2 = 53 --- !query 119 schema +-- !query schema struct --- !query 119 output +-- !query output 53 NULL NULL --- !query 120 +-- !query create temp view zv1 as select *,'dummy' AS junk from zt1 --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output --- !query 121 +-- !query select * from zt2 left join zt3 on (f2 = f3) left join zv1 on (f3 = f1) where f2 = 53 --- !query 121 schema +-- !query schema struct --- !query 121 output +-- !query output 53 NULL NULL NULL --- !query 122 +-- !query select a.unique2, a.ten, b.tenthous, b.unique2, b.hundred from tenk1 a left join tenk1 b on a.unique2 = b.tenthous where a.unique1 = 42 and ((b.unique2 is null and a.ten = 2) or b.hundred = 3) --- !query 122 schema +-- !query schema struct --- !query 122 output +-- !query output --- !query 123 +-- !query create or replace temporary view a (i integer) using parquet --- !query 123 schema +-- !query schema struct<> --- !query 123 output +-- !query output --- !query 124 +-- !query create or replace temporary view b (x integer, y integer) using parquet --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output --- !query 125 +-- !query select * from a left join b on i = x and i = y and x = i --- !query 125 schema +-- !query schema struct --- !query 125 output +-- !query output --- !query 126 +-- !query select t1.q2, count(t2.*) from int8_tbl t1 left join int8_tbl t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 126 schema +-- !query schema struct --- !query 126 output +-- !query output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 127 +-- !query select t1.q2, count(t2.*) from int8_tbl t1 left join (select * from int8_tbl) t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 127 schema +-- !query schema struct --- !query 127 output +-- !query output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 128 +-- !query select t1.q2, count(t2.*) from int8_tbl t1 left join (select q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 on (t1.q2 = t2.q1) group by t1.q2 order by 1 --- !query 128 schema +-- !query schema struct --- !query 128 output +-- !query output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 129 +-- !query create or replace temporary view a as select * from (values ('p'), ('q')) as v(code) --- !query 129 schema +-- !query schema struct<> --- !query 129 output +-- !query output --- !query 130 +-- !query create or replace temporary view b as select * from (values ('p', 1), ('p', 2)) as v(a, num) --- !query 130 schema +-- !query schema struct<> --- !query 130 output +-- !query output --- !query 131 +-- !query create or replace temporary view c as select * from (values ('A', 'p'), ('B', 'q'), ('C', null)) as v(name, a) --- !query 131 schema +-- !query schema struct<> --- !query 131 output +-- !query output --- !query 132 +-- !query select c.name, ss.code, ss.b_cnt, ss.const from c left join (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const @@ -2809,15 +2809,15 @@ from c left join ) as ss on (c.a = ss.code) order by c.name --- !query 132 schema +-- !query schema struct --- !query 132 output +-- !query output A p 2 -1 B q 0 -1 C NULL NULL NULL --- !query 133 +-- !query SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN @@ -2833,13 +2833,13 @@ LEFT JOIN ON sub4.key5 = sub3.key3 ) sub2 ON sub1.key1 = sub2.key3 --- !query 133 schema +-- !query schema struct --- !query 133 output +-- !query output 1 1 1 1 --- !query 134 +-- !query SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN @@ -2855,13 +2855,13 @@ LEFT JOIN ON sub4.key5 = sub3.key3 ) sub2 ON sub1.key1 = sub2.key3 --- !query 134 schema +-- !query schema struct --- !query 134 output +-- !query output 1 1 1 1 --- !query 135 +-- !query SELECT qq, unique1 FROM ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 @@ -2869,45 +2869,45 @@ SELECT qq, unique1 ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 USING (qq) INNER JOIN tenk1 c ON qq = unique2 --- !query 135 schema +-- !query schema struct --- !query 135 output +-- !query output 123 4596 123 4596 456 7318 --- !query 136 +-- !query create or replace temporary view nt1 as select * from (values(1,true,true), (2,true,false), (3,false,false)) as v(id, a1, a2) --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output --- !query 137 +-- !query create or replace temporary view nt2 as select * from (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) as v(id, nt1_id, b1, b2) --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output --- !query 138 +-- !query create or replace temporary view nt3 as select * from (values(1,1,true), (2,2,false), (3,3,true)) as v(id, nt2_id, c1) --- !query 138 schema +-- !query schema struct<> --- !query 138 output +-- !query output --- !query 139 +-- !query select nt3.id from nt3 as nt3 left join @@ -2919,17 +2919,17 @@ from nt3 as nt3 ) as ss2 on ss2.id = nt3.nt2_id where nt3.id = 1 and ss2.b3 --- !query 139 schema +-- !query schema struct --- !query 139 output +-- !query output 1 --- !query 140 +-- !query select * from int4_tbl a full join int4_tbl b on true --- !query 140 schema +-- !query schema struct --- !query 140 output +-- !query output -123456 -123456 -123456 -2147483647 -123456 0 @@ -2957,11 +2957,11 @@ struct 2147483647 2147483647 --- !query 141 +-- !query select * from int4_tbl a full join int4_tbl b on false --- !query 141 schema +-- !query schema struct --- !query 141 output +-- !query output -123456 NULL -2147483647 NULL 0 NULL @@ -2974,27 +2974,27 @@ NULL 123456 NULL 2147483647 --- !query 142 +-- !query select count(*) from tenk1 a join tenk1 b on a.unique1 = b.unique2 left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand join int4_tbl on b.thousand = f1 --- !query 142 schema +-- !query schema struct --- !query 142 output +-- !query output 10 --- !query 143 +-- !query select b.unique1 from tenk1 a join tenk1 b on a.unique1 = b.unique2 left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand join int4_tbl i1 on b.thousand = f1 right join int4_tbl i2 on i2.f1 = b.tenthous order by 1 --- !query 143 schema +-- !query schema struct --- !query 143 output +-- !query output NULL NULL NULL @@ -3002,7 +3002,7 @@ NULL 0 --- !query 144 +-- !query select * from ( select unique1, q1, coalesce(unique1, -1) + q1 as fault @@ -3010,43 +3010,43 @@ select * from ) ss where fault = 122 order by fault --- !query 144 schema +-- !query schema struct --- !query 144 output +-- !query output NULL 123 122 --- !query 145 +-- !query select q1, unique2, thousand, hundred from int8_tbl a left join tenk1 b on q1 = unique2 where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123) --- !query 145 schema +-- !query schema struct --- !query 145 output +-- !query output --- !query 146 +-- !query select f1, unique2, case when unique2 is null then f1 else 0 end from int4_tbl a left join tenk1 b on f1 = unique2 where (case when unique2 is null then f1 else 0 end) = 0 --- !query 146 schema +-- !query schema struct --- !query 146 output +-- !query output 0 0 0 --- !query 147 +-- !query select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44 --- !query 147 schema +-- !query schema struct --- !query 147 output +-- !query output --- !query 148 +-- !query select * from text_tbl t1 inner join int8_tbl i8 @@ -3055,32 +3055,32 @@ select * from on t1.f1 = 'doh!' left join int4_tbl i4 on i8.q1 = i4.f1 --- !query 148 schema +-- !query schema struct --- !query 148 output +-- !query output doh! 123 456 doh! NULL doh! 123 456 hi de ho neighbor NULL --- !query 149 +-- !query select * from (select 1 as id) as xx left join (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) on (xx.id = coalesce(yy.id)) --- !query 149 schema +-- !query schema struct --- !query 149 output +-- !query output 1 1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx 1 --- !query 150 +-- !query select a.q2, b.q1 from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) where coalesce(b.q1, 1) > 0 --- !query 150 schema +-- !query schema struct --- !query 150 output +-- !query output -4567890123456789 NULL 123 123 123 123 @@ -3093,124 +3093,124 @@ struct 4567890123456789 4567890123456789 --- !query 151 +-- !query create or replace temporary view parent as select * from (values (1, 10), (2, 20), (3, 30)) as v(k, pd) --- !query 151 schema +-- !query schema struct<> --- !query 151 output +-- !query output --- !query 152 +-- !query create or replace temporary view child as select * from (values (1, 100), (4, 400)) as v(k, cd) --- !query 152 schema +-- !query schema struct<> --- !query 152 output +-- !query output --- !query 153 +-- !query select p.* from parent p left join child c on (p.k = c.k) --- !query 153 schema +-- !query schema struct --- !query 153 output +-- !query output 1 10 2 20 3 30 --- !query 154 +-- !query select p.*, linked from parent p left join (select c.*, true as linked from child c) as ss on (p.k = ss.k) --- !query 154 schema +-- !query schema struct --- !query 154 output +-- !query output 1 10 true 2 20 NULL 3 30 NULL --- !query 155 +-- !query select p.* from parent p left join child c on (p.k = c.k) where p.k = 1 and p.k = 2 --- !query 155 schema +-- !query schema struct --- !query 155 output +-- !query output --- !query 156 +-- !query select p.* from (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k where p.k = 1 and p.k = 2 --- !query 156 schema +-- !query schema struct --- !query 156 output +-- !query output --- !query 157 +-- !query create or replace temporary view a as select * from (values (0), (1)) as v(id) --- !query 157 schema +-- !query schema struct<> --- !query 157 output +-- !query output --- !query 158 +-- !query create or replace temporary view b as select * from (values (0, 0), (1, NULL)) as v(id, a_id) --- !query 158 schema +-- !query schema struct<> --- !query 158 output +-- !query output --- !query 159 +-- !query SELECT * FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) --- !query 159 schema +-- !query schema struct --- !query 159 output +-- !query output 1 NULL NULL --- !query 160 +-- !query SELECT b.* FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) --- !query 160 schema +-- !query schema struct --- !query 160 output +-- !query output 1 NULL --- !query 161 +-- !query create or replace temporary view innertab as select * from (values (123L, 42L)) as v(id, dat1) --- !query 161 schema +-- !query schema struct<> --- !query 161 output +-- !query output --- !query 162 +-- !query SELECT * FROM (SELECT 1 AS x) ss1 LEFT JOIN (SELECT q1, q2, COALESCE(dat1, q1) AS y FROM int8_tbl LEFT JOIN innertab ON q2 = id) ss2 ON true --- !query 162 schema +-- !query schema struct --- !query 162 output +-- !query output 1 123 456 123 1 123 4567890123456789 123 1 4567890123456789 -4567890123456789 4567890123456789 @@ -3218,163 +3218,163 @@ struct 1 4567890123456789 4567890123456789 4567890123456789 --- !query 163 +-- !query select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = f1 --- !query 163 schema +-- !query schema struct<> --- !query 163 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 63 --- !query 164 +-- !query select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1 --- !query 164 schema +-- !query schema struct<> --- !query 164 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63 --- !query 165 +-- !query select * from int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1 --- !query 165 schema +-- !query schema struct --- !query 165 output +-- !query output --- !query 166 +-- !query select t1.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 166 schema +-- !query schema struct<> --- !query 166 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 167 +-- !query select t2.uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 167 schema +-- !query schema struct<> --- !query 167 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 168 +-- !query select uunique1 from tenk1 t1 join tenk2 t2 on t1.two = t2.two --- !query 168 schema +-- !query schema struct<> --- !query 168 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 7 --- !query 169 +-- !query select f1,g from int4_tbl a, (select f1 as g) ss --- !query 169 schema +-- !query schema struct<> --- !query 169 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`f1`' given input columns: []; line 1 pos 37 --- !query 170 +-- !query select f1,g from int4_tbl a, (select a.f1 as g) ss --- !query 170 schema +-- !query schema struct<> --- !query 170 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`a.f1`' given input columns: []; line 1 pos 37 --- !query 171 +-- !query select f1,g from int4_tbl a cross join (select f1 as g) ss --- !query 171 schema +-- !query schema struct<> --- !query 171 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`f1`' given input columns: []; line 1 pos 47 --- !query 172 +-- !query select f1,g from int4_tbl a cross join (select a.f1 as g) ss --- !query 172 schema +-- !query schema struct<> --- !query 172 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`a.f1`' given input columns: []; line 1 pos 47 --- !query 173 +-- !query CREATE TABLE j1 (id1 int, id2 int) USING parquet --- !query 173 schema +-- !query schema struct<> --- !query 173 output +-- !query output --- !query 174 +-- !query CREATE TABLE j2 (id1 int, id2 int) USING parquet --- !query 174 schema +-- !query schema struct<> --- !query 174 output +-- !query output --- !query 175 +-- !query INSERT INTO j1 values(1,1),(1,2) --- !query 175 schema +-- !query schema struct<> --- !query 175 output +-- !query output --- !query 176 +-- !query INSERT INTO j2 values(1,1) --- !query 176 schema +-- !query schema struct<> --- !query 176 output +-- !query output --- !query 177 +-- !query INSERT INTO j2 values(1,2) --- !query 177 schema +-- !query schema struct<> --- !query 177 output +-- !query output --- !query 178 +-- !query select * from j1 inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1 --- !query 178 schema +-- !query schema struct --- !query 178 output +-- !query output 1 1 1 1 1 2 1 2 --- !query 179 +-- !query drop table j1 --- !query 179 schema +-- !query schema struct<> --- !query 179 output +-- !query output --- !query 180 +-- !query drop table j2 --- !query 180 schema +-- !query schema struct<> --- !query 180 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out index 5a06b32f56289..2c8bc31dbc6ca 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out @@ -2,24 +2,24 @@ -- Number of queries: 7 --- !query 0 +-- !query SELECT '' AS two, unique1, unique2, stringu1 FROM onek WHERE unique1 > 50 ORDER BY unique1 LIMIT 2 --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 51 76 ZBAAAA 52 985 ACAAAA --- !query 1 +-- !query SELECT '' AS five, unique1, unique2, stringu1 FROM onek WHERE unique1 > 60 ORDER BY unique1 LIMIT 5 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 61 560 JCAAAA 62 633 KCAAAA 63 296 LCAAAA @@ -27,18 +27,18 @@ struct 65 64 NCAAAA --- !query 2 +-- !query SELECT '' AS two, unique1, unique2, stringu1 FROM onek WHERE unique1 > 60 AND unique1 < 63 ORDER BY unique1 LIMIT 5 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 61 560 JCAAAA 62 633 KCAAAA --- !query 3 +-- !query CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (VALUES (123, 456), @@ -47,35 +47,35 @@ CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (4567890123456789, 4567890123456789), (4567890123456789, -4567890123456789)) AS v(q1, q2) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException The limit expression must evaluate to a constant value, but got CASE WHEN (`_nondeterministic` < CAST(0.5BD AS DOUBLE)) THEN CAST(NULL AS BIGINT) END; --- !query 5 +-- !query DROP VIEW INT8_TBL --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query select sum(tenthous) as s1, sum(tenthous) + random()*0 as s2 from tenk1 group by thousand order by thousand limit 3 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 45000 45000.0 45010 45010.0 45020 45020.0 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out index 74babe2580a58..bdb605e406b8a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out @@ -2,3587 +2,3587 @@ -- Number of queries: 577 --- !query 0 +-- !query CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO num_exp_add VALUES (0,0,0) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query INSERT INTO num_exp_sub VALUES (0,0,0) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query INSERT INTO num_exp_mul VALUES (0,0,0) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query INSERT INTO num_exp_div VALUES (0,0,double('NaN')) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query INSERT INTO num_exp_add VALUES (0,1,0) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query INSERT INTO num_exp_sub VALUES (0,1,0) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query INSERT INTO num_exp_mul VALUES (0,1,0) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output --- !query 17 +-- !query INSERT INTO num_exp_div VALUES (0,1,double('NaN')) --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output --- !query 18 +-- !query INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047) --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query INSERT INTO num_exp_mul VALUES (0,2,0) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query INSERT INTO num_exp_div VALUES (0,2,0) --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query INSERT INTO num_exp_add VALUES (0,3,4.31) --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output --- !query 23 +-- !query INSERT INTO num_exp_sub VALUES (0,3,-4.31) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query INSERT INTO num_exp_mul VALUES (0,3,0) --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query INSERT INTO num_exp_div VALUES (0,3,0) --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query INSERT INTO num_exp_add VALUES (0,4,7799461.4119) --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119) --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output --- !query 28 +-- !query INSERT INTO num_exp_mul VALUES (0,4,0) --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output --- !query 29 +-- !query INSERT INTO num_exp_div VALUES (0,4,0) --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output --- !query 30 +-- !query INSERT INTO num_exp_add VALUES (0,5,16397.038491) --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output --- !query 31 +-- !query INSERT INTO num_exp_sub VALUES (0,5,-16397.038491) --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output --- !query 32 +-- !query INSERT INTO num_exp_mul VALUES (0,5,0) --- !query 32 schema +-- !query schema struct<> --- !query 32 output +-- !query output --- !query 33 +-- !query INSERT INTO num_exp_div VALUES (0,5,0) --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output --- !query 34 +-- !query INSERT INTO num_exp_add VALUES (0,6,93901.57763026) --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output --- !query 35 +-- !query INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026) --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output --- !query 36 +-- !query INSERT INTO num_exp_mul VALUES (0,6,0) --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output --- !query 37 +-- !query INSERT INTO num_exp_div VALUES (0,6,0) --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output --- !query 38 +-- !query INSERT INTO num_exp_add VALUES (0,7,-83028485) --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output --- !query 39 +-- !query INSERT INTO num_exp_sub VALUES (0,7,83028485) --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output --- !query 40 +-- !query INSERT INTO num_exp_mul VALUES (0,7,0) --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output --- !query 41 +-- !query INSERT INTO num_exp_div VALUES (0,7,0) --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output --- !query 42 +-- !query INSERT INTO num_exp_add VALUES (0,8,74881) --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output --- !query 43 +-- !query INSERT INTO num_exp_sub VALUES (0,8,-74881) --- !query 43 schema +-- !query schema struct<> --- !query 43 output +-- !query output --- !query 44 +-- !query INSERT INTO num_exp_mul VALUES (0,8,0) --- !query 44 schema +-- !query schema struct<> --- !query 44 output +-- !query output --- !query 45 +-- !query INSERT INTO num_exp_div VALUES (0,8,0) --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output --- !query 46 +-- !query INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420) --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output --- !query 47 +-- !query INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420) --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output --- !query 48 +-- !query INSERT INTO num_exp_mul VALUES (0,9,0) --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output --- !query 49 +-- !query INSERT INTO num_exp_div VALUES (0,9,0) --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output --- !query 50 +-- !query INSERT INTO num_exp_add VALUES (1,0,0) --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output --- !query 51 +-- !query INSERT INTO num_exp_sub VALUES (1,0,0) --- !query 51 schema +-- !query schema struct<> --- !query 51 output +-- !query output --- !query 52 +-- !query INSERT INTO num_exp_mul VALUES (1,0,0) --- !query 52 schema +-- !query schema struct<> --- !query 52 output +-- !query output --- !query 53 +-- !query INSERT INTO num_exp_div VALUES (1,0,double('NaN')) --- !query 53 schema +-- !query schema struct<> --- !query 53 output +-- !query output --- !query 54 +-- !query INSERT INTO num_exp_add VALUES (1,1,0) --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output --- !query 55 +-- !query INSERT INTO num_exp_sub VALUES (1,1,0) --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output --- !query 56 +-- !query INSERT INTO num_exp_mul VALUES (1,1,0) --- !query 56 schema +-- !query schema struct<> --- !query 56 output +-- !query output --- !query 57 +-- !query INSERT INTO num_exp_div VALUES (1,1,double('NaN')) --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output --- !query 58 +-- !query INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047) --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output --- !query 59 +-- !query INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047) --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output --- !query 60 +-- !query INSERT INTO num_exp_mul VALUES (1,2,0) --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output --- !query 61 +-- !query INSERT INTO num_exp_div VALUES (1,2,0) --- !query 61 schema +-- !query schema struct<> --- !query 61 output +-- !query output --- !query 62 +-- !query INSERT INTO num_exp_add VALUES (1,3,4.31) --- !query 62 schema +-- !query schema struct<> --- !query 62 output +-- !query output --- !query 63 +-- !query INSERT INTO num_exp_sub VALUES (1,3,-4.31) --- !query 63 schema +-- !query schema struct<> --- !query 63 output +-- !query output --- !query 64 +-- !query INSERT INTO num_exp_mul VALUES (1,3,0) --- !query 64 schema +-- !query schema struct<> --- !query 64 output +-- !query output --- !query 65 +-- !query INSERT INTO num_exp_div VALUES (1,3,0) --- !query 65 schema +-- !query schema struct<> --- !query 65 output +-- !query output --- !query 66 +-- !query INSERT INTO num_exp_add VALUES (1,4,7799461.4119) --- !query 66 schema +-- !query schema struct<> --- !query 66 output +-- !query output --- !query 67 +-- !query INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119) --- !query 67 schema +-- !query schema struct<> --- !query 67 output +-- !query output --- !query 68 +-- !query INSERT INTO num_exp_mul VALUES (1,4,0) --- !query 68 schema +-- !query schema struct<> --- !query 68 output +-- !query output --- !query 69 +-- !query INSERT INTO num_exp_div VALUES (1,4,0) --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output --- !query 70 +-- !query INSERT INTO num_exp_add VALUES (1,5,16397.038491) --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output --- !query 71 +-- !query INSERT INTO num_exp_sub VALUES (1,5,-16397.038491) --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output --- !query 72 +-- !query INSERT INTO num_exp_mul VALUES (1,5,0) --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output --- !query 73 +-- !query INSERT INTO num_exp_div VALUES (1,5,0) --- !query 73 schema +-- !query schema struct<> --- !query 73 output +-- !query output --- !query 74 +-- !query INSERT INTO num_exp_add VALUES (1,6,93901.57763026) --- !query 74 schema +-- !query schema struct<> --- !query 74 output +-- !query output --- !query 75 +-- !query INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026) --- !query 75 schema +-- !query schema struct<> --- !query 75 output +-- !query output --- !query 76 +-- !query INSERT INTO num_exp_mul VALUES (1,6,0) --- !query 76 schema +-- !query schema struct<> --- !query 76 output +-- !query output --- !query 77 +-- !query INSERT INTO num_exp_div VALUES (1,6,0) --- !query 77 schema +-- !query schema struct<> --- !query 77 output +-- !query output --- !query 78 +-- !query INSERT INTO num_exp_add VALUES (1,7,-83028485) --- !query 78 schema +-- !query schema struct<> --- !query 78 output +-- !query output --- !query 79 +-- !query INSERT INTO num_exp_sub VALUES (1,7,83028485) --- !query 79 schema +-- !query schema struct<> --- !query 79 output +-- !query output --- !query 80 +-- !query INSERT INTO num_exp_mul VALUES (1,7,0) --- !query 80 schema +-- !query schema struct<> --- !query 80 output +-- !query output --- !query 81 +-- !query INSERT INTO num_exp_div VALUES (1,7,0) --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output --- !query 82 +-- !query INSERT INTO num_exp_add VALUES (1,8,74881) --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output --- !query 83 +-- !query INSERT INTO num_exp_sub VALUES (1,8,-74881) --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output --- !query 84 +-- !query INSERT INTO num_exp_mul VALUES (1,8,0) --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output --- !query 85 +-- !query INSERT INTO num_exp_div VALUES (1,8,0) --- !query 85 schema +-- !query schema struct<> --- !query 85 output +-- !query output --- !query 86 +-- !query INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420) --- !query 86 schema +-- !query schema struct<> --- !query 86 output +-- !query output --- !query 87 +-- !query INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420) --- !query 87 schema +-- !query schema struct<> --- !query 87 output +-- !query output --- !query 88 +-- !query INSERT INTO num_exp_mul VALUES (1,9,0) --- !query 88 schema +-- !query schema struct<> --- !query 88 output +-- !query output --- !query 89 +-- !query INSERT INTO num_exp_div VALUES (1,9,0) --- !query 89 schema +-- !query schema struct<> --- !query 89 output +-- !query output --- !query 90 +-- !query INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047) --- !query 90 schema +-- !query schema struct<> --- !query 90 output +-- !query output --- !query 91 +-- !query INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047) --- !query 91 schema +-- !query schema struct<> --- !query 91 output +-- !query output --- !query 92 +-- !query INSERT INTO num_exp_mul VALUES (2,0,0) --- !query 92 schema +-- !query schema struct<> --- !query 92 output +-- !query output --- !query 93 +-- !query INSERT INTO num_exp_div VALUES (2,0,double('NaN')) --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output --- !query 94 +-- !query INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047) --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output --- !query 95 +-- !query INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047) --- !query 95 schema +-- !query schema struct<> --- !query 95 output +-- !query output --- !query 96 +-- !query INSERT INTO num_exp_mul VALUES (2,1,0) --- !query 96 schema +-- !query schema struct<> --- !query 96 output +-- !query output --- !query 97 +-- !query INSERT INTO num_exp_div VALUES (2,1,double('NaN')) --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output --- !query 98 +-- !query INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094) --- !query 98 schema +-- !query schema struct<> --- !query 98 output +-- !query output --- !query 99 +-- !query INSERT INTO num_exp_sub VALUES (2,2,0) --- !query 99 schema +-- !query schema struct<> --- !query 99 output +-- !query output --- !query 100 +-- !query INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209) --- !query 100 schema +-- !query schema struct<> --- !query 100 output +-- !query output --- !query 101 +-- !query INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000) --- !query 101 schema +-- !query schema struct<> --- !query 101 output +-- !query output --- !query 102 +-- !query INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047) --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output --- !query 103 +-- !query INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047) --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output --- !query 104 +-- !query INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257) --- !query 104 schema +-- !query schema struct<> --- !query 104 output +-- !query output --- !query 105 +-- !query INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266) --- !query 105 schema +-- !query schema struct<> --- !query 105 output +-- !query output --- !query 106 +-- !query INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047) --- !query 106 schema +-- !query schema struct<> --- !query 106 output +-- !query output --- !query 107 +-- !query INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047) --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output --- !query 108 +-- !query INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593) --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output --- !query 109 +-- !query INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685) --- !query 109 schema +-- !query schema struct<> --- !query 109 output +-- !query output --- !query 110 +-- !query INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047) --- !query 110 schema +-- !query schema struct<> --- !query 110 output +-- !query output --- !query 111 +-- !query INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047) --- !query 111 schema +-- !query schema struct<> --- !query 111 output +-- !query output --- !query 112 +-- !query INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077) --- !query 112 schema +-- !query schema struct<> --- !query 112 output +-- !query output --- !query 113 +-- !query INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429) --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output --- !query 114 +-- !query INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787) --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output --- !query 115 +-- !query INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307) --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output --- !query 116 +-- !query INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222) --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output --- !query 117 +-- !query INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940) --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output --- !query 118 +-- !query INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047) --- !query 118 schema +-- !query schema struct<> --- !query 118 output +-- !query output --- !query 119 +-- !query INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953) --- !query 119 schema +-- !query schema struct<> --- !query 119 output +-- !query output --- !query 120 +-- !query INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795) --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output --- !query 121 +-- !query INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518) --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output --- !query 122 +-- !query INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047) --- !query 122 schema +-- !query schema struct<> --- !query 122 output +-- !query output --- !query 123 +-- !query INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047) --- !query 123 schema +-- !query schema struct<> --- !query 123 output +-- !query output --- !query 124 +-- !query INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407) --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output --- !query 125 +-- !query INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476) --- !query 125 schema +-- !query schema struct<> --- !query 125 output +-- !query output --- !query 126 +-- !query INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467) --- !query 126 schema +-- !query schema struct<> --- !query 126 output +-- !query output --- !query 127 +-- !query INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627) --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output --- !query 128 +-- !query INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740) --- !query 128 schema +-- !query schema struct<> --- !query 128 output +-- !query output --- !query 129 +-- !query INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811) --- !query 129 schema +-- !query schema struct<> --- !query 129 output +-- !query output --- !query 130 +-- !query INSERT INTO num_exp_add VALUES (3,0,4.31) --- !query 130 schema +-- !query schema struct<> --- !query 130 output +-- !query output --- !query 131 +-- !query INSERT INTO num_exp_sub VALUES (3,0,4.31) --- !query 131 schema +-- !query schema struct<> --- !query 131 output +-- !query output --- !query 132 +-- !query INSERT INTO num_exp_mul VALUES (3,0,0) --- !query 132 schema +-- !query schema struct<> --- !query 132 output +-- !query output --- !query 133 +-- !query INSERT INTO num_exp_div VALUES (3,0,double('NaN')) --- !query 133 schema +-- !query schema struct<> --- !query 133 output +-- !query output --- !query 134 +-- !query INSERT INTO num_exp_add VALUES (3,1,4.31) --- !query 134 schema +-- !query schema struct<> --- !query 134 output +-- !query output --- !query 135 +-- !query INSERT INTO num_exp_sub VALUES (3,1,4.31) --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output --- !query 136 +-- !query INSERT INTO num_exp_mul VALUES (3,1,0) --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output --- !query 137 +-- !query INSERT INTO num_exp_div VALUES (3,1,double('NaN')) --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output --- !query 138 +-- !query INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047) --- !query 138 schema +-- !query schema struct<> --- !query 138 output +-- !query output --- !query 139 +-- !query INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047) --- !query 139 schema +-- !query schema struct<> --- !query 139 output +-- !query output --- !query 140 +-- !query INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257) --- !query 140 schema +-- !query schema struct<> --- !query 140 output +-- !query output --- !query 141 +-- !query INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352) --- !query 141 schema +-- !query schema struct<> --- !query 141 output +-- !query output --- !query 142 +-- !query INSERT INTO num_exp_add VALUES (3,3,8.62) --- !query 142 schema +-- !query schema struct<> --- !query 142 output +-- !query output --- !query 143 +-- !query INSERT INTO num_exp_sub VALUES (3,3,0) --- !query 143 schema +-- !query schema struct<> --- !query 143 output +-- !query output --- !query 144 +-- !query INSERT INTO num_exp_mul VALUES (3,3,18.5761) --- !query 144 schema +-- !query schema struct<> --- !query 144 output +-- !query output --- !query 145 +-- !query INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000) --- !query 145 schema +-- !query schema struct<> --- !query 145 output +-- !query output --- !query 146 +-- !query INSERT INTO num_exp_add VALUES (3,4,7799465.7219) --- !query 146 schema +-- !query schema struct<> --- !query 146 output +-- !query output --- !query 147 +-- !query INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019) --- !query 147 schema +-- !query schema struct<> --- !query 147 output +-- !query output --- !query 148 +-- !query INSERT INTO num_exp_mul VALUES (3,4,33615678.685289) --- !query 148 schema +-- !query schema struct<> --- !query 148 output +-- !query output --- !query 149 +-- !query INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552) --- !query 149 schema +-- !query schema struct<> --- !query 149 output +-- !query output --- !query 150 +-- !query INSERT INTO num_exp_add VALUES (3,5,16401.348491) --- !query 150 schema +-- !query schema struct<> --- !query 150 output +-- !query output --- !query 151 +-- !query INSERT INTO num_exp_sub VALUES (3,5,-16392.728491) --- !query 151 schema +-- !query schema struct<> --- !query 151 output +-- !query output --- !query 152 +-- !query INSERT INTO num_exp_mul VALUES (3,5,70671.23589621) --- !query 152 schema +-- !query schema struct<> --- !query 152 output +-- !query output --- !query 153 +-- !query INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504) --- !query 153 schema +-- !query schema struct<> --- !query 153 output +-- !query output --- !query 154 +-- !query INSERT INTO num_exp_add VALUES (3,6,93905.88763026) --- !query 154 schema +-- !query schema struct<> --- !query 154 output +-- !query output --- !query 155 +-- !query INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026) --- !query 155 schema +-- !query schema struct<> --- !query 155 output +-- !query output --- !query 156 +-- !query INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206) --- !query 156 schema +-- !query schema struct<> --- !query 156 output +-- !query output --- !query 157 +-- !query INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595) --- !query 157 schema +-- !query schema struct<> --- !query 157 output +-- !query output --- !query 158 +-- !query INSERT INTO num_exp_add VALUES (3,7,-83028480.69) --- !query 158 schema +-- !query schema struct<> --- !query 158 output +-- !query output --- !query 159 +-- !query INSERT INTO num_exp_sub VALUES (3,7,83028489.31) --- !query 159 schema +-- !query schema struct<> --- !query 159 output +-- !query output --- !query 160 +-- !query INSERT INTO num_exp_mul VALUES (3,7,-357852770.35) --- !query 160 schema +-- !query schema struct<> --- !query 160 output +-- !query output --- !query 161 +-- !query INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240) --- !query 161 schema +-- !query schema struct<> --- !query 161 output +-- !query output --- !query 162 +-- !query INSERT INTO num_exp_add VALUES (3,8,74885.31) --- !query 162 schema +-- !query schema struct<> --- !query 162 output +-- !query output --- !query 163 +-- !query INSERT INTO num_exp_sub VALUES (3,8,-74876.69) --- !query 163 schema +-- !query schema struct<> --- !query 163 output +-- !query output --- !query 164 +-- !query INSERT INTO num_exp_mul VALUES (3,8,322737.11) --- !query 164 schema +-- !query schema struct<> --- !query 164 output +-- !query output --- !query 165 +-- !query INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553) --- !query 165 schema +-- !query schema struct<> --- !query 165 output +-- !query output --- !query 166 +-- !query INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420) --- !query 166 schema +-- !query schema struct<> --- !query 166 output +-- !query output --- !query 167 +-- !query INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420) --- !query 167 schema +-- !query schema struct<> --- !query 167 output +-- !query output --- !query 168 +-- !query INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020) --- !query 168 schema +-- !query schema struct<> --- !query 168 output +-- !query output --- !query 169 +-- !query INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854) --- !query 169 schema +-- !query schema struct<> --- !query 169 output +-- !query output --- !query 170 +-- !query INSERT INTO num_exp_add VALUES (4,0,7799461.4119) --- !query 170 schema +-- !query schema struct<> --- !query 170 output +-- !query output --- !query 171 +-- !query INSERT INTO num_exp_sub VALUES (4,0,7799461.4119) --- !query 171 schema +-- !query schema struct<> --- !query 171 output +-- !query output --- !query 172 +-- !query INSERT INTO num_exp_mul VALUES (4,0,0) --- !query 172 schema +-- !query schema struct<> --- !query 172 output +-- !query output --- !query 173 +-- !query INSERT INTO num_exp_div VALUES (4,0,double('NaN')) --- !query 173 schema +-- !query schema struct<> --- !query 173 output +-- !query output --- !query 174 +-- !query INSERT INTO num_exp_add VALUES (4,1,7799461.4119) --- !query 174 schema +-- !query schema struct<> --- !query 174 output +-- !query output --- !query 175 +-- !query INSERT INTO num_exp_sub VALUES (4,1,7799461.4119) --- !query 175 schema +-- !query schema struct<> --- !query 175 output +-- !query output --- !query 176 +-- !query INSERT INTO num_exp_mul VALUES (4,1,0) --- !query 176 schema +-- !query schema struct<> --- !query 176 output +-- !query output --- !query 177 +-- !query INSERT INTO num_exp_div VALUES (4,1,double('NaN')) --- !query 177 schema +-- !query schema struct<> --- !query 177 output +-- !query output --- !query 178 +-- !query INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047) --- !query 178 schema +-- !query schema struct<> --- !query 178 output +-- !query output --- !query 179 +-- !query INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047) --- !query 179 schema +-- !query schema struct<> --- !query 179 output +-- !query output --- !query 180 +-- !query INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593) --- !query 180 schema +-- !query schema struct<> --- !query 180 output +-- !query output --- !query 181 +-- !query INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385) --- !query 181 schema +-- !query schema struct<> --- !query 181 output +-- !query output --- !query 182 +-- !query INSERT INTO num_exp_add VALUES (4,3,7799465.7219) --- !query 182 schema +-- !query schema struct<> --- !query 182 output +-- !query output --- !query 183 +-- !query INSERT INTO num_exp_sub VALUES (4,3,7799457.1019) --- !query 183 schema +-- !query schema struct<> --- !query 183 output +-- !query output --- !query 184 +-- !query INSERT INTO num_exp_mul VALUES (4,3,33615678.685289) --- !query 184 schema +-- !query schema struct<> --- !query 184 output +-- !query output --- !query 185 +-- !query INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883) --- !query 185 schema +-- !query schema struct<> --- !query 185 output +-- !query output --- !query 186 +-- !query INSERT INTO num_exp_add VALUES (4,4,15598922.8238) --- !query 186 schema +-- !query schema struct<> --- !query 186 output +-- !query output --- !query 187 +-- !query INSERT INTO num_exp_sub VALUES (4,4,0) --- !query 187 schema +-- !query schema struct<> --- !query 187 output +-- !query output --- !query 188 +-- !query INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161) --- !query 188 schema +-- !query schema struct<> --- !query 188 output +-- !query output --- !query 189 +-- !query INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000) --- !query 189 schema +-- !query schema struct<> --- !query 189 output +-- !query output --- !query 190 +-- !query INSERT INTO num_exp_add VALUES (4,5,7815858.450391) --- !query 190 schema +-- !query schema struct<> --- !query 190 output +-- !query output --- !query 191 +-- !query INSERT INTO num_exp_sub VALUES (4,5,7783064.373409) --- !query 191 schema +-- !query schema struct<> --- !query 191 output +-- !query output --- !query 192 +-- !query INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429) --- !query 192 schema +-- !query schema struct<> --- !query 192 output +-- !query output --- !query 193 +-- !query INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061) --- !query 193 schema +-- !query schema struct<> --- !query 193 output +-- !query output --- !query 194 +-- !query INSERT INTO num_exp_add VALUES (4,6,7893362.98953026) --- !query 194 schema +-- !query schema struct<> --- !query 194 output +-- !query output --- !query 195 +-- !query INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974) --- !query 195 schema +-- !query schema struct<> --- !query 195 output +-- !query output --- !query 196 +-- !query INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094) --- !query 196 schema +-- !query schema struct<> --- !query 196 output +-- !query output --- !query 197 +-- !query INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606) --- !query 197 schema +-- !query schema struct<> --- !query 197 output +-- !query output --- !query 198 +-- !query INSERT INTO num_exp_add VALUES (4,7,-75229023.5881) --- !query 198 schema +-- !query schema struct<> --- !query 198 output +-- !query output --- !query 199 +-- !query INSERT INTO num_exp_sub VALUES (4,7,90827946.4119) --- !query 199 schema +-- !query schema struct<> --- !query 199 output +-- !query output --- !query 200 +-- !query INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715) --- !query 200 schema +-- !query schema struct<> --- !query 200 output +-- !query output --- !query 201 +-- !query INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637) --- !query 201 schema +-- !query schema struct<> --- !query 201 output +-- !query output --- !query 202 +-- !query INSERT INTO num_exp_add VALUES (4,8,7874342.4119) --- !query 202 schema +-- !query schema struct<> --- !query 202 output +-- !query output --- !query 203 +-- !query INSERT INTO num_exp_sub VALUES (4,8,7724580.4119) --- !query 203 schema +-- !query schema struct<> --- !query 203 output +-- !query output --- !query 204 +-- !query INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839) --- !query 204 schema +-- !query schema struct<> --- !query 204 output +-- !query output --- !query 205 +-- !query INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143) --- !query 205 schema +-- !query schema struct<> --- !query 205 output +-- !query output --- !query 206 +-- !query INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420) --- !query 206 schema +-- !query schema struct<> --- !query 206 output +-- !query output --- !query 207 +-- !query INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420) --- !query 207 schema +-- !query schema struct<> --- !query 207 output +-- !query output --- !query 208 +-- !query INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980) --- !query 208 schema +-- !query schema struct<> --- !query 208 output +-- !query output --- !query 209 +-- !query INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409) --- !query 209 schema +-- !query schema struct<> --- !query 209 output +-- !query output --- !query 210 +-- !query INSERT INTO num_exp_add VALUES (5,0,16397.038491) --- !query 210 schema +-- !query schema struct<> --- !query 210 output +-- !query output --- !query 211 +-- !query INSERT INTO num_exp_sub VALUES (5,0,16397.038491) --- !query 211 schema +-- !query schema struct<> --- !query 211 output +-- !query output --- !query 212 +-- !query INSERT INTO num_exp_mul VALUES (5,0,0) --- !query 212 schema +-- !query schema struct<> --- !query 212 output +-- !query output --- !query 213 +-- !query INSERT INTO num_exp_div VALUES (5,0,double('NaN')) --- !query 213 schema +-- !query schema struct<> --- !query 213 output +-- !query output --- !query 214 +-- !query INSERT INTO num_exp_add VALUES (5,1,16397.038491) --- !query 214 schema +-- !query schema struct<> --- !query 214 output +-- !query output --- !query 215 +-- !query INSERT INTO num_exp_sub VALUES (5,1,16397.038491) --- !query 215 schema +-- !query schema struct<> --- !query 215 output +-- !query output --- !query 216 +-- !query INSERT INTO num_exp_mul VALUES (5,1,0) --- !query 216 schema +-- !query schema struct<> --- !query 216 output +-- !query output --- !query 217 +-- !query INSERT INTO num_exp_div VALUES (5,1,double('NaN')) --- !query 217 schema +-- !query schema struct<> --- !query 217 output +-- !query output --- !query 218 +-- !query INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047) --- !query 218 schema +-- !query schema struct<> --- !query 218 output +-- !query output --- !query 219 +-- !query INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047) --- !query 219 schema +-- !query schema struct<> --- !query 219 output +-- !query output --- !query 220 +-- !query INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077) --- !query 220 schema +-- !query schema struct<> --- !query 220 output +-- !query output --- !query 221 +-- !query INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446) --- !query 221 schema +-- !query schema struct<> --- !query 221 output +-- !query output --- !query 222 +-- !query INSERT INTO num_exp_add VALUES (5,3,16401.348491) --- !query 222 schema +-- !query schema struct<> --- !query 222 output +-- !query output --- !query 223 +-- !query INSERT INTO num_exp_sub VALUES (5,3,16392.728491) --- !query 223 schema +-- !query schema struct<> --- !query 223 output +-- !query output --- !query 224 +-- !query INSERT INTO num_exp_mul VALUES (5,3,70671.23589621) --- !query 224 schema +-- !query schema struct<> --- !query 224 output +-- !query output --- !query 225 +-- !query INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584) --- !query 225 schema +-- !query schema struct<> --- !query 225 output +-- !query output --- !query 226 +-- !query INSERT INTO num_exp_add VALUES (5,4,7815858.450391) --- !query 226 schema +-- !query schema struct<> --- !query 226 output +-- !query output --- !query 227 +-- !query INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409) --- !query 227 schema +-- !query schema struct<> --- !query 227 output +-- !query output --- !query 228 +-- !query INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429) --- !query 228 schema +-- !query schema struct<> --- !query 228 output +-- !query output --- !query 229 +-- !query INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192) --- !query 229 schema +-- !query schema struct<> --- !query 229 output +-- !query output --- !query 230 +-- !query INSERT INTO num_exp_add VALUES (5,5,32794.076982) --- !query 230 schema +-- !query schema struct<> --- !query 230 output +-- !query output --- !query 231 +-- !query INSERT INTO num_exp_sub VALUES (5,5,0) --- !query 231 schema +-- !query schema struct<> --- !query 231 output +-- !query output --- !query 232 +-- !query INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081) --- !query 232 schema +-- !query schema struct<> --- !query 232 output +-- !query output --- !query 233 +-- !query INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000) --- !query 233 schema +-- !query schema struct<> --- !query 233 output +-- !query output --- !query 234 +-- !query INSERT INTO num_exp_add VALUES (5,6,110298.61612126) --- !query 234 schema +-- !query schema struct<> --- !query 234 output +-- !query output --- !query 235 +-- !query INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926) --- !query 235 schema +-- !query schema struct<> --- !query 235 output +-- !query output --- !query 236 +-- !query INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766) --- !query 236 schema +-- !query schema struct<> --- !query 236 output +-- !query output --- !query 237 +-- !query INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689) --- !query 237 schema +-- !query schema struct<> --- !query 237 output +-- !query output --- !query 238 +-- !query INSERT INTO num_exp_add VALUES (5,7,-83012087.961509) --- !query 238 schema +-- !query schema struct<> --- !query 238 output +-- !query output --- !query 239 +-- !query INSERT INTO num_exp_sub VALUES (5,7,83044882.038491) --- !query 239 schema +-- !query schema struct<> --- !query 239 output +-- !query output --- !query 240 +-- !query INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135) --- !query 240 schema +-- !query schema struct<> --- !query 240 output +-- !query output --- !query 241 +-- !query INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710) --- !query 241 schema +-- !query schema struct<> --- !query 241 output +-- !query output --- !query 242 +-- !query INSERT INTO num_exp_add VALUES (5,8,91278.038491) --- !query 242 schema +-- !query schema struct<> --- !query 242 output +-- !query output --- !query 243 +-- !query INSERT INTO num_exp_sub VALUES (5,8,-58483.961509) --- !query 243 schema +-- !query schema struct<> --- !query 243 output +-- !query output --- !query 244 +-- !query INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571) --- !query 244 schema +-- !query schema struct<> --- !query 244 output +-- !query output --- !query 245 +-- !query INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228) --- !query 245 schema +-- !query schema struct<> --- !query 245 output +-- !query output --- !query 246 +-- !query INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420) --- !query 246 schema +-- !query schema struct<> --- !query 246 output +-- !query output --- !query 247 +-- !query INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420) --- !query 247 schema +-- !query schema struct<> --- !query 247 output +-- !query output --- !query 248 +-- !query INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220) --- !query 248 schema +-- !query schema struct<> --- !query 248 output +-- !query output --- !query 249 +-- !query INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427) --- !query 249 schema +-- !query schema struct<> --- !query 249 output +-- !query output --- !query 250 +-- !query INSERT INTO num_exp_add VALUES (6,0,93901.57763026) --- !query 250 schema +-- !query schema struct<> --- !query 250 output +-- !query output --- !query 251 +-- !query INSERT INTO num_exp_sub VALUES (6,0,93901.57763026) --- !query 251 schema +-- !query schema struct<> --- !query 251 output +-- !query output --- !query 252 +-- !query INSERT INTO num_exp_mul VALUES (6,0,0) --- !query 252 schema +-- !query schema struct<> --- !query 252 output +-- !query output --- !query 253 +-- !query INSERT INTO num_exp_div VALUES (6,0,double('NaN')) --- !query 253 schema +-- !query schema struct<> --- !query 253 output +-- !query output --- !query 254 +-- !query INSERT INTO num_exp_add VALUES (6,1,93901.57763026) --- !query 254 schema +-- !query schema struct<> --- !query 254 output +-- !query output --- !query 255 +-- !query INSERT INTO num_exp_sub VALUES (6,1,93901.57763026) --- !query 255 schema +-- !query schema struct<> --- !query 255 output +-- !query output --- !query 256 +-- !query INSERT INTO num_exp_mul VALUES (6,1,0) --- !query 256 schema +-- !query schema struct<> --- !query 256 output +-- !query output --- !query 257 +-- !query INSERT INTO num_exp_div VALUES (6,1,double('NaN')) --- !query 257 schema +-- !query schema struct<> --- !query 257 output +-- !query output --- !query 258 +-- !query INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787) --- !query 258 schema +-- !query schema struct<> --- !query 258 output +-- !query output --- !query 259 +-- !query INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307) --- !query 259 schema +-- !query schema struct<> --- !query 259 output +-- !query output --- !query 260 +-- !query INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222) --- !query 260 schema +-- !query schema struct<> --- !query 260 output +-- !query output --- !query 261 +-- !query INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823) --- !query 261 schema +-- !query schema struct<> --- !query 261 output +-- !query output --- !query 262 +-- !query INSERT INTO num_exp_add VALUES (6,3,93905.88763026) --- !query 262 schema +-- !query schema struct<> --- !query 262 output +-- !query output --- !query 263 +-- !query INSERT INTO num_exp_sub VALUES (6,3,93897.26763026) --- !query 263 schema +-- !query schema struct<> --- !query 263 output +-- !query output --- !query 264 +-- !query INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206) --- !query 264 schema +-- !query schema struct<> --- !query 264 output +-- !query output --- !query 265 +-- !query INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907) --- !query 265 schema +-- !query schema struct<> --- !query 265 output +-- !query output --- !query 266 +-- !query INSERT INTO num_exp_add VALUES (6,4,7893362.98953026) --- !query 266 schema +-- !query schema struct<> --- !query 266 output +-- !query output --- !query 267 +-- !query INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974) --- !query 267 schema +-- !query schema struct<> --- !query 267 output +-- !query output --- !query 268 +-- !query INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094) --- !query 268 schema +-- !query schema struct<> --- !query 268 output +-- !query output --- !query 269 +-- !query INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469) --- !query 269 schema +-- !query schema struct<> --- !query 269 output +-- !query output --- !query 270 +-- !query INSERT INTO num_exp_add VALUES (6,5,110298.61612126) --- !query 270 schema +-- !query schema struct<> --- !query 270 output +-- !query output --- !query 271 +-- !query INSERT INTO num_exp_sub VALUES (6,5,77504.53913926) --- !query 271 schema +-- !query schema struct<> --- !query 271 output +-- !query output --- !query 272 +-- !query INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766) --- !query 272 schema +-- !query schema struct<> --- !query 272 output +-- !query output --- !query 273 +-- !query INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679) --- !query 273 schema +-- !query schema struct<> --- !query 273 output +-- !query output --- !query 274 +-- !query INSERT INTO num_exp_add VALUES (6,6,187803.15526052) --- !query 274 schema +-- !query schema struct<> --- !query 274 output +-- !query output --- !query 275 +-- !query INSERT INTO num_exp_sub VALUES (6,6,0) --- !query 275 schema +-- !query schema struct<> --- !query 275 output +-- !query output --- !query 276 +-- !query INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676) --- !query 276 schema +-- !query schema struct<> --- !query 276 output +-- !query output --- !query 277 +-- !query INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000) --- !query 277 schema +-- !query schema struct<> --- !query 277 output +-- !query output --- !query 278 +-- !query INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974) --- !query 278 schema +-- !query schema struct<> --- !query 278 output +-- !query output --- !query 279 +-- !query INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026) --- !query 279 schema +-- !query schema struct<> --- !query 279 output +-- !query output --- !query 280 +-- !query INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610) --- !query 280 schema +-- !query schema struct<> --- !query 280 output +-- !query output --- !query 281 +-- !query INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980) --- !query 281 schema +-- !query schema struct<> --- !query 281 output +-- !query output --- !query 282 +-- !query INSERT INTO num_exp_add VALUES (6,8,168782.57763026) --- !query 282 schema +-- !query schema struct<> --- !query 282 output +-- !query output --- !query 283 +-- !query INSERT INTO num_exp_sub VALUES (6,8,19020.57763026) --- !query 283 schema +-- !query schema struct<> --- !query 283 output +-- !query output --- !query 284 +-- !query INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906) --- !query 284 schema +-- !query schema struct<> --- !query 284 output +-- !query output --- !query 285 +-- !query INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184) --- !query 285 schema +-- !query schema struct<> --- !query 285 output +-- !query output --- !query 286 +-- !query INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160) --- !query 286 schema +-- !query schema struct<> --- !query 286 output +-- !query output --- !query 287 +-- !query INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680) --- !query 287 schema +-- !query schema struct<> --- !query 287 output +-- !query output --- !query 288 +-- !query INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920) --- !query 288 schema +-- !query schema struct<> --- !query 288 output +-- !query output --- !query 289 +-- !query INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789) --- !query 289 schema +-- !query schema struct<> --- !query 289 output +-- !query output --- !query 290 +-- !query INSERT INTO num_exp_add VALUES (7,0,-83028485) --- !query 290 schema +-- !query schema struct<> --- !query 290 output +-- !query output --- !query 291 +-- !query INSERT INTO num_exp_sub VALUES (7,0,-83028485) --- !query 291 schema +-- !query schema struct<> --- !query 291 output +-- !query output --- !query 292 +-- !query INSERT INTO num_exp_mul VALUES (7,0,0) --- !query 292 schema +-- !query schema struct<> --- !query 292 output +-- !query output --- !query 293 +-- !query INSERT INTO num_exp_div VALUES (7,0,double('NaN')) --- !query 293 schema +-- !query schema struct<> --- !query 293 output +-- !query output --- !query 294 +-- !query INSERT INTO num_exp_add VALUES (7,1,-83028485) --- !query 294 schema +-- !query schema struct<> --- !query 294 output +-- !query output --- !query 295 +-- !query INSERT INTO num_exp_sub VALUES (7,1,-83028485) --- !query 295 schema +-- !query schema struct<> --- !query 295 output +-- !query output --- !query 296 +-- !query INSERT INTO num_exp_mul VALUES (7,1,0) --- !query 296 schema +-- !query schema struct<> --- !query 296 output +-- !query output --- !query 297 +-- !query INSERT INTO num_exp_div VALUES (7,1,double('NaN')) --- !query 297 schema +-- !query schema struct<> --- !query 297 output +-- !query output --- !query 298 +-- !query INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047) --- !query 298 schema +-- !query schema struct<> --- !query 298 output +-- !query output --- !query 299 +-- !query INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953) --- !query 299 schema +-- !query schema struct<> --- !query 299 output +-- !query output --- !query 300 +-- !query INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795) --- !query 300 schema +-- !query schema struct<> --- !query 300 output +-- !query output --- !query 301 +-- !query INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700) --- !query 301 schema +-- !query schema struct<> --- !query 301 output +-- !query output --- !query 302 +-- !query INSERT INTO num_exp_add VALUES (7,3,-83028480.69) --- !query 302 schema +-- !query schema struct<> --- !query 302 output +-- !query output --- !query 303 +-- !query INSERT INTO num_exp_sub VALUES (7,3,-83028489.31) --- !query 303 schema +-- !query schema struct<> --- !query 303 output +-- !query output --- !query 304 +-- !query INSERT INTO num_exp_mul VALUES (7,3,-357852770.35) --- !query 304 schema +-- !query schema struct<> --- !query 304 output +-- !query output --- !query 305 +-- !query INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974) --- !query 305 schema +-- !query schema struct<> --- !query 305 output +-- !query output --- !query 306 +-- !query INSERT INTO num_exp_add VALUES (7,4,-75229023.5881) --- !query 306 schema +-- !query schema struct<> --- !query 306 output +-- !query output --- !query 307 +-- !query INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119) --- !query 307 schema +-- !query schema struct<> --- !query 307 output +-- !query output --- !query 308 +-- !query INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715) --- !query 308 schema +-- !query schema struct<> --- !query 308 output +-- !query output --- !query 309 +-- !query INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686) --- !query 309 schema +-- !query schema struct<> --- !query 309 output +-- !query output --- !query 310 +-- !query INSERT INTO num_exp_add VALUES (7,5,-83012087.961509) --- !query 310 schema +-- !query schema struct<> --- !query 310 output +-- !query output --- !query 311 +-- !query INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491) --- !query 311 schema +-- !query schema struct<> --- !query 311 output +-- !query output --- !query 312 +-- !query INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135) --- !query 312 schema +-- !query schema struct<> --- !query 312 output +-- !query output --- !query 313 +-- !query INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574) --- !query 313 schema +-- !query schema struct<> --- !query 313 output +-- !query output --- !query 314 +-- !query INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974) --- !query 314 schema +-- !query schema struct<> --- !query 314 output +-- !query output --- !query 315 +-- !query INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026) --- !query 315 schema +-- !query schema struct<> --- !query 315 output +-- !query output --- !query 316 +-- !query INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610) --- !query 316 schema +-- !query schema struct<> --- !query 316 output +-- !query output --- !query 317 +-- !query INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294) --- !query 317 schema +-- !query schema struct<> --- !query 317 output +-- !query output --- !query 318 +-- !query INSERT INTO num_exp_add VALUES (7,7,-166056970) --- !query 318 schema +-- !query schema struct<> --- !query 318 output +-- !query output --- !query 319 +-- !query INSERT INTO num_exp_sub VALUES (7,7,0) --- !query 319 schema +-- !query schema struct<> --- !query 319 output +-- !query output --- !query 320 +-- !query INSERT INTO num_exp_mul VALUES (7,7,6893729321395225) --- !query 320 schema +-- !query schema struct<> --- !query 320 output +-- !query output --- !query 321 +-- !query INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000) --- !query 321 schema +-- !query schema struct<> --- !query 321 output +-- !query output --- !query 322 +-- !query INSERT INTO num_exp_add VALUES (7,8,-82953604) --- !query 322 schema +-- !query schema struct<> --- !query 322 output +-- !query output --- !query 323 +-- !query INSERT INTO num_exp_sub VALUES (7,8,-83103366) --- !query 323 schema +-- !query schema struct<> --- !query 323 output +-- !query output --- !query 324 +-- !query INSERT INTO num_exp_mul VALUES (7,8,-6217255985285) --- !query 324 schema +-- !query schema struct<> --- !query 324 output +-- !query output --- !query 325 +-- !query INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118) --- !query 325 schema +-- !query schema struct<> --- !query 325 output +-- !query output --- !query 326 +-- !query INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420) --- !query 326 schema +-- !query schema struct<> --- !query 326 output +-- !query output --- !query 327 +-- !query INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580) --- !query 327 schema +-- !query schema struct<> --- !query 327 output +-- !query output --- !query 328 +-- !query INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700) --- !query 328 schema +-- !query schema struct<> --- !query 328 output +-- !query output --- !query 329 +-- !query INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382) --- !query 329 schema +-- !query schema struct<> --- !query 329 output +-- !query output --- !query 330 +-- !query INSERT INTO num_exp_add VALUES (8,0,74881) --- !query 330 schema +-- !query schema struct<> --- !query 330 output +-- !query output --- !query 331 +-- !query INSERT INTO num_exp_sub VALUES (8,0,74881) --- !query 331 schema +-- !query schema struct<> --- !query 331 output +-- !query output --- !query 332 +-- !query INSERT INTO num_exp_mul VALUES (8,0,0) --- !query 332 schema +-- !query schema struct<> --- !query 332 output +-- !query output --- !query 333 +-- !query INSERT INTO num_exp_div VALUES (8,0,double('NaN')) --- !query 333 schema +-- !query schema struct<> --- !query 333 output +-- !query output --- !query 334 +-- !query INSERT INTO num_exp_add VALUES (8,1,74881) --- !query 334 schema +-- !query schema struct<> --- !query 334 output +-- !query output --- !query 335 +-- !query INSERT INTO num_exp_sub VALUES (8,1,74881) --- !query 335 schema +-- !query schema struct<> --- !query 335 output +-- !query output --- !query 336 +-- !query INSERT INTO num_exp_mul VALUES (8,1,0) --- !query 336 schema +-- !query schema struct<> --- !query 336 output +-- !query output --- !query 337 +-- !query INSERT INTO num_exp_div VALUES (8,1,double('NaN')) --- !query 337 schema +-- !query schema struct<> --- !query 337 output +-- !query output --- !query 338 +-- !query INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047) --- !query 338 schema +-- !query schema struct<> --- !query 338 output +-- !query output --- !query 339 +-- !query INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047) --- !query 339 schema +-- !query schema struct<> --- !query 339 output +-- !query output --- !query 340 +-- !query INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407) --- !query 340 schema +-- !query schema struct<> --- !query 340 output +-- !query output --- !query 341 +-- !query INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615) --- !query 341 schema +-- !query schema struct<> --- !query 341 output +-- !query output --- !query 342 +-- !query INSERT INTO num_exp_add VALUES (8,3,74885.31) --- !query 342 schema +-- !query schema struct<> --- !query 342 output +-- !query output --- !query 343 +-- !query INSERT INTO num_exp_sub VALUES (8,3,74876.69) --- !query 343 schema +-- !query schema struct<> --- !query 343 output +-- !query output --- !query 344 +-- !query INSERT INTO num_exp_mul VALUES (8,3,322737.11) --- !query 344 schema +-- !query schema struct<> --- !query 344 output +-- !query output --- !query 345 +-- !query INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410) --- !query 345 schema +-- !query schema struct<> --- !query 345 output +-- !query output --- !query 346 +-- !query INSERT INTO num_exp_add VALUES (8,4,7874342.4119) --- !query 346 schema +-- !query schema struct<> --- !query 346 output +-- !query output --- !query 347 +-- !query INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119) --- !query 347 schema +-- !query schema struct<> --- !query 347 output +-- !query output --- !query 348 +-- !query INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839) --- !query 348 schema +-- !query schema struct<> --- !query 348 output +-- !query output --- !query 349 +-- !query INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956) --- !query 349 schema +-- !query schema struct<> --- !query 349 output +-- !query output --- !query 350 +-- !query INSERT INTO num_exp_add VALUES (8,5,91278.038491) --- !query 350 schema +-- !query schema struct<> --- !query 350 output +-- !query output --- !query 351 +-- !query INSERT INTO num_exp_sub VALUES (8,5,58483.961509) --- !query 351 schema +-- !query schema struct<> --- !query 351 output +-- !query output --- !query 352 +-- !query INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571) --- !query 352 schema +-- !query schema struct<> --- !query 352 output +-- !query output --- !query 353 +-- !query INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456) --- !query 353 schema +-- !query schema struct<> --- !query 353 output +-- !query output --- !query 354 +-- !query INSERT INTO num_exp_add VALUES (8,6,168782.57763026) --- !query 354 schema +-- !query schema struct<> --- !query 354 output +-- !query output --- !query 355 +-- !query INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026) --- !query 355 schema +-- !query schema struct<> --- !query 355 output +-- !query output --- !query 356 +-- !query INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906) --- !query 356 schema +-- !query schema struct<> --- !query 356 output +-- !query output --- !query 357 +-- !query INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424) --- !query 357 schema +-- !query schema struct<> --- !query 357 output +-- !query output --- !query 358 +-- !query INSERT INTO num_exp_add VALUES (8,7,-82953604) --- !query 358 schema +-- !query schema struct<> --- !query 358 output +-- !query output --- !query 359 +-- !query INSERT INTO num_exp_sub VALUES (8,7,83103366) --- !query 359 schema +-- !query schema struct<> --- !query 359 output +-- !query output --- !query 360 +-- !query INSERT INTO num_exp_mul VALUES (8,7,-6217255985285) --- !query 360 schema +-- !query schema struct<> --- !query 360 output +-- !query output --- !query 361 +-- !query INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172) --- !query 361 schema +-- !query schema struct<> --- !query 361 output +-- !query output --- !query 362 +-- !query INSERT INTO num_exp_add VALUES (8,8,149762) --- !query 362 schema +-- !query schema struct<> --- !query 362 output +-- !query output --- !query 363 +-- !query INSERT INTO num_exp_sub VALUES (8,8,0) --- !query 363 schema +-- !query schema struct<> --- !query 363 output +-- !query output --- !query 364 +-- !query INSERT INTO num_exp_mul VALUES (8,8,5607164161) --- !query 364 schema +-- !query schema struct<> --- !query 364 output +-- !query output --- !query 365 +-- !query INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000) --- !query 365 schema +-- !query schema struct<> --- !query 365 output +-- !query output --- !query 366 +-- !query INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420) --- !query 366 schema +-- !query schema struct<> --- !query 366 output +-- !query output --- !query 367 +-- !query INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420) --- !query 367 schema +-- !query schema struct<> --- !query 367 output +-- !query output --- !query 368 +-- !query INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020) --- !query 368 schema +-- !query schema struct<> --- !query 368 output +-- !query output --- !query 369 +-- !query INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735) --- !query 369 schema +-- !query schema struct<> --- !query 369 output +-- !query output --- !query 370 +-- !query INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420) --- !query 370 schema +-- !query schema struct<> --- !query 370 output +-- !query output --- !query 371 +-- !query INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420) --- !query 371 schema +-- !query schema struct<> --- !query 371 output +-- !query output --- !query 372 +-- !query INSERT INTO num_exp_mul VALUES (9,0,0) --- !query 372 schema +-- !query schema struct<> --- !query 372 output +-- !query output --- !query 373 +-- !query INSERT INTO num_exp_div VALUES (9,0,double('NaN')) --- !query 373 schema +-- !query schema struct<> --- !query 373 output +-- !query output --- !query 374 +-- !query INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420) --- !query 374 schema +-- !query schema struct<> --- !query 374 output +-- !query output --- !query 375 +-- !query INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420) --- !query 375 schema +-- !query schema struct<> --- !query 375 output +-- !query output --- !query 376 +-- !query INSERT INTO num_exp_mul VALUES (9,1,0) --- !query 376 schema +-- !query schema struct<> --- !query 376 output +-- !query output --- !query 377 +-- !query INSERT INTO num_exp_div VALUES (9,1,double('NaN')) --- !query 377 schema +-- !query schema struct<> --- !query 377 output +-- !query output --- !query 378 +-- !query INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467) --- !query 378 schema +-- !query schema struct<> --- !query 378 output +-- !query output --- !query 379 +-- !query INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627) --- !query 379 schema +-- !query schema struct<> --- !query 379 output +-- !query output --- !query 380 +-- !query INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740) --- !query 380 schema +-- !query schema struct<> --- !query 380 output +-- !query output --- !query 381 +-- !query INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526) --- !query 381 schema +-- !query schema struct<> --- !query 381 output +-- !query output --- !query 382 +-- !query INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420) --- !query 382 schema +-- !query schema struct<> --- !query 382 output +-- !query output --- !query 383 +-- !query INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420) --- !query 383 schema +-- !query schema struct<> --- !query 383 output +-- !query output --- !query 384 +-- !query INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020) --- !query 384 schema +-- !query schema struct<> --- !query 384 output +-- !query output --- !query 385 +-- !query INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677) --- !query 385 schema +-- !query schema struct<> --- !query 385 output +-- !query output --- !query 386 +-- !query INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420) --- !query 386 schema +-- !query schema struct<> --- !query 386 output +-- !query output --- !query 387 +-- !query INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420) --- !query 387 schema +-- !query schema struct<> --- !query 387 output +-- !query output --- !query 388 +-- !query INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980) --- !query 388 schema +-- !query schema struct<> --- !query 388 output +-- !query output --- !query 389 +-- !query INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484) --- !query 389 schema +-- !query schema struct<> --- !query 389 output +-- !query output --- !query 390 +-- !query INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420) --- !query 390 schema +-- !query schema struct<> --- !query 390 output +-- !query output --- !query 391 +-- !query INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420) --- !query 391 schema +-- !query schema struct<> --- !query 391 output +-- !query output --- !query 392 +-- !query INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220) --- !query 392 schema +-- !query schema struct<> --- !query 392 output +-- !query output --- !query 393 +-- !query INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807) --- !query 393 schema +-- !query schema struct<> --- !query 393 output +-- !query output --- !query 394 +-- !query INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160) --- !query 394 schema +-- !query schema struct<> --- !query 394 output +-- !query output --- !query 395 +-- !query INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680) --- !query 395 schema +-- !query schema struct<> --- !query 395 output +-- !query output --- !query 396 +-- !query INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920) --- !query 396 schema +-- !query schema struct<> --- !query 396 output +-- !query output --- !query 397 +-- !query INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280) --- !query 397 schema +-- !query schema struct<> --- !query 397 output +-- !query output --- !query 398 +-- !query INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420) --- !query 398 schema +-- !query schema struct<> --- !query 398 output +-- !query output --- !query 399 +-- !query INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580) --- !query 399 schema +-- !query schema struct<> --- !query 399 output +-- !query output --- !query 400 +-- !query INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700) --- !query 400 schema +-- !query schema struct<> --- !query 400 output +-- !query output --- !query 401 +-- !query INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689) --- !query 401 schema +-- !query schema struct<> --- !query 401 output +-- !query output --- !query 402 +-- !query INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420) --- !query 402 schema +-- !query schema struct<> --- !query 402 output +-- !query output --- !query 403 +-- !query INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420) --- !query 403 schema +-- !query schema struct<> --- !query 403 output +-- !query output --- !query 404 +-- !query INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020) --- !query 404 schema +-- !query schema struct<> --- !query 404 output +-- !query output --- !query 405 +-- !query INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748) --- !query 405 schema +-- !query schema struct<> --- !query 405 output +-- !query output --- !query 406 +-- !query INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840) --- !query 406 schema +-- !query schema struct<> --- !query 406 output +-- !query output --- !query 407 +-- !query INSERT INTO num_exp_sub VALUES (9,9,0) --- !query 407 schema +-- !query schema struct<> --- !query 407 output +-- !query output --- !query 408 +-- !query INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400) --- !query 408 schema +-- !query schema struct<> --- !query 408 output +-- !query output --- !query 409 +-- !query INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000) --- !query 409 schema +-- !query schema struct<> --- !query 409 output +-- !query output --- !query 410 +-- !query INSERT INTO num_exp_sqrt VALUES (0,0) --- !query 410 schema +-- !query schema struct<> --- !query 410 output +-- !query output --- !query 411 +-- !query INSERT INTO num_exp_sqrt VALUES (1,0) --- !query 411 schema +-- !query schema struct<> --- !query 411 output +-- !query output --- !query 412 +-- !query INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505) --- !query 412 schema +-- !query schema struct<> --- !query 412 output +-- !query output --- !query 413 +-- !query INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396) --- !query 413 schema +-- !query schema struct<> --- !query 413 output +-- !query output --- !query 414 +-- !query INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923) --- !query 414 schema +-- !query schema struct<> --- !query 414 output +-- !query output --- !query 415 +-- !query INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473) --- !query 415 schema +-- !query schema struct<> --- !query 415 output +-- !query output --- !query 416 +-- !query INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406) --- !query 416 schema +-- !query schema struct<> --- !query 416 output +-- !query output --- !query 417 +-- !query INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230) --- !query 417 schema +-- !query schema struct<> --- !query 417 output +-- !query output --- !query 418 +-- !query INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542) --- !query 418 schema +-- !query schema struct<> --- !query 418 output +-- !query output --- !query 419 +-- !query INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766) --- !query 419 schema +-- !query schema struct<> --- !query 419 output +-- !query output --- !query 420 +-- !query INSERT INTO num_exp_ln VALUES (0,double('NaN')) --- !query 420 schema +-- !query schema struct<> --- !query 420 output +-- !query output --- !query 421 +-- !query INSERT INTO num_exp_ln VALUES (1,double('NaN')) --- !query 421 schema +-- !query schema struct<> --- !query 421 output +-- !query output --- !query 422 +-- !query INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514) --- !query 422 schema +-- !query schema struct<> --- !query 422 output +-- !query output --- !query 423 +-- !query INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971) --- !query 423 schema +-- !query schema struct<> --- !query 423 output +-- !query output --- !query 424 +-- !query INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464) --- !query 424 schema +-- !query schema struct<> --- !query 424 output +-- !query output --- !query 425 +-- !query INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038) --- !query 425 schema +-- !query schema struct<> --- !query 425 output +-- !query output --- !query 426 +-- !query INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127) --- !query 426 schema +-- !query schema struct<> --- !query 426 output +-- !query output --- !query 427 +-- !query INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991) --- !query 427 schema +-- !query schema struct<> --- !query 427 output +-- !query output --- !query 428 +-- !query INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668) --- !query 428 schema +-- !query schema struct<> --- !query 428 output +-- !query output --- !query 429 +-- !query INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962) --- !query 429 schema +-- !query schema struct<> --- !query 429 output +-- !query output --- !query 430 +-- !query INSERT INTO num_exp_log10 VALUES (0,double('NaN')) --- !query 430 schema +-- !query schema struct<> --- !query 430 output +-- !query output --- !query 431 +-- !query INSERT INTO num_exp_log10 VALUES (1,double('NaN')) --- !query 431 schema +-- !query schema struct<> --- !query 431 output +-- !query output --- !query 432 +-- !query INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459) --- !query 432 schema +-- !query schema struct<> --- !query 432 output +-- !query output --- !query 433 +-- !query INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075) --- !query 433 schema +-- !query schema struct<> --- !query 433 output +-- !query output --- !query 434 +-- !query INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345) --- !query 434 schema +-- !query schema struct<> --- !query 434 output +-- !query output --- !query 435 +-- !query INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626) --- !query 435 schema +-- !query schema struct<> --- !query 435 output +-- !query output --- !query 436 +-- !query INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671) --- !query 436 schema +-- !query schema struct<> --- !query 436 output +-- !query output --- !query 437 +-- !query INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914) --- !query 437 schema +-- !query schema struct<> --- !query 437 output +-- !query output --- !query 438 +-- !query INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138) --- !query 438 schema +-- !query schema struct<> --- !query 438 output +-- !query output --- !query 439 +-- !query INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059) --- !query 439 schema +-- !query schema struct<> --- !query 439 output +-- !query output --- !query 440 +-- !query INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')) --- !query 440 schema +-- !query schema struct<> --- !query 440 output +-- !query output --- !query 441 +-- !query INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')) --- !query 441 schema +-- !query schema struct<> --- !query 441 output +-- !query output --- !query 442 +-- !query INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184) --- !query 442 schema +-- !query schema struct<> --- !query 442 output +-- !query output --- !query 443 +-- !query INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393) --- !query 443 schema +-- !query schema struct<> --- !query 443 output +-- !query output --- !query 444 +-- !query INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636) --- !query 444 schema +-- !query schema struct<> --- !query 444 output +-- !query output --- !query 445 +-- !query INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098) --- !query 445 schema +-- !query schema struct<> --- !query 445 output +-- !query output --- !query 446 +-- !query INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067) --- !query 446 schema +-- !query schema struct<> --- !query 446 output +-- !query output --- !query 447 +-- !query INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) --- !query 447 schema +-- !query schema struct<> --- !query 447 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException decimal can only support precision up to 38 @@ -3590,107 +3590,107 @@ decimal can only support precision up to 38 INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) --- !query 448 +-- !query INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952) --- !query 448 schema +-- !query schema struct<> --- !query 448 output +-- !query output --- !query 449 +-- !query INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457) --- !query 449 schema +-- !query schema struct<> --- !query 449 output +-- !query output --- !query 450 +-- !query INSERT INTO num_data VALUES (0, 0) --- !query 450 schema +-- !query schema struct<> --- !query 450 output +-- !query output --- !query 451 +-- !query INSERT INTO num_data VALUES (1, 0) --- !query 451 schema +-- !query schema struct<> --- !query 451 output +-- !query output --- !query 452 +-- !query INSERT INTO num_data VALUES (2, -34338492.215397047) --- !query 452 schema +-- !query schema struct<> --- !query 452 output +-- !query output --- !query 453 +-- !query INSERT INTO num_data VALUES (3, 4.31) --- !query 453 schema +-- !query schema struct<> --- !query 453 output +-- !query output --- !query 454 +-- !query INSERT INTO num_data VALUES (4, 7799461.4119) --- !query 454 schema +-- !query schema struct<> --- !query 454 output +-- !query output --- !query 455 +-- !query INSERT INTO num_data VALUES (5, 16397.038491) --- !query 455 schema +-- !query schema struct<> --- !query 455 output +-- !query output --- !query 456 +-- !query INSERT INTO num_data VALUES (6, 93901.57763026) --- !query 456 schema +-- !query schema struct<> --- !query 456 output +-- !query output --- !query 457 +-- !query INSERT INTO num_data VALUES (7, -83028485) --- !query 457 schema +-- !query schema struct<> --- !query 457 output +-- !query output --- !query 458 +-- !query INSERT INTO num_data VALUES (8, 74881) --- !query 458 schema +-- !query schema struct<> --- !query 458 output +-- !query output --- !query 459 +-- !query INSERT INTO num_data VALUES (9, -24926804.045047420) --- !query 459 schema +-- !query schema struct<> --- !query 459 output +-- !query output --- !query 460 +-- !query SELECT * FROM num_data --- !query 460 schema +-- !query schema struct --- !query 460 output +-- !query output 0 0.0000000000 1 0.0000000000 2 -34338492.2153970470 @@ -3703,172 +3703,172 @@ struct 9 -24926804.0450474200 --- !query 461 +-- !query TRUNCATE TABLE num_result --- !query 461 schema +-- !query schema struct<> --- !query 461 output +-- !query output --- !query 462 +-- !query INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val FROM num_data t1, num_data t2 --- !query 462 schema +-- !query schema struct<> --- !query 462 output +-- !query output --- !query 463 +-- !query SELECT t1.id1, t1.id2, t1.result, t2.expected FROM num_result t1, num_exp_add t2 WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 AND t1.result != t2.expected --- !query 463 schema +-- !query schema struct --- !query 463 output +-- !query output --- !query 464 +-- !query TRUNCATE TABLE num_result --- !query 464 schema +-- !query schema struct<> --- !query 464 output +-- !query output --- !query 465 +-- !query INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) FROM num_data t1, num_data t2 --- !query 465 schema +-- !query schema struct<> --- !query 465 output +-- !query output --- !query 466 +-- !query SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected FROM num_result t1, num_exp_add t2 WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 AND t1.result != round(t2.expected, 10) --- !query 466 schema +-- !query schema struct --- !query 466 output +-- !query output --- !query 467 +-- !query TRUNCATE TABLE num_result --- !query 467 schema +-- !query schema struct<> --- !query 467 output +-- !query output --- !query 468 +-- !query INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val FROM num_data t1, num_data t2 --- !query 468 schema +-- !query schema struct<> --- !query 468 output +-- !query output --- !query 469 +-- !query SELECT t1.id1, t1.id2, t1.result, t2.expected FROM num_result t1, num_exp_sub t2 WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 AND t1.result != t2.expected --- !query 469 schema +-- !query schema struct --- !query 469 output +-- !query output --- !query 470 +-- !query TRUNCATE TABLE num_result --- !query 470 schema +-- !query schema struct<> --- !query 470 output +-- !query output --- !query 471 +-- !query INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) FROM num_data t1, num_data t2 --- !query 471 schema +-- !query schema struct<> --- !query 471 output +-- !query output --- !query 472 +-- !query SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) FROM num_result t1, num_exp_sub t2 WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 AND t1.result != round(t2.expected, 40) --- !query 472 schema +-- !query schema struct --- !query 472 output +-- !query output --- !query 473 +-- !query TRUNCATE TABLE num_result --- !query 473 schema +-- !query schema struct<> --- !query 473 output +-- !query output --- !query 474 +-- !query INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val FROM num_data t1, num_data t2 --- !query 474 schema +-- !query schema struct<> --- !query 474 output +-- !query output org.apache.spark.sql.AnalysisException `default`.`num_result` requires that the data to be inserted have the same number of columns as the target table: target table has 3 column(s) but the inserted data has 5 column(s), including 0 partition column(s) having constant value(s).; --- !query 475 +-- !query SELECT t1.id1, t1.id2, t1.result, t2.expected FROM num_result t1, num_exp_mul t2 WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 AND t1.result != t2.expected --- !query 475 schema +-- !query schema struct --- !query 475 output +-- !query output --- !query 476 +-- !query TRUNCATE TABLE num_result --- !query 476 schema +-- !query schema struct<> --- !query 476 output +-- !query output --- !query 477 +-- !query INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) FROM num_data t1, num_data t2 --- !query 477 schema +-- !query schema struct<> --- !query 477 output +-- !query output --- !query 478 +-- !query SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected FROM num_result t1, num_exp_mul t2 WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 AND t1.result != round(t2.expected, 30) --- !query 478 schema +-- !query schema struct --- !query 478 output +-- !query output 2 2 1179132047626883.5968620000 1179132047626883.5968621359 2 3 -147998901.4483610000 -147998901.4483612726 2 4 -267821744976817.8111140000 -267821744976817.8111137107 @@ -3916,32 +3916,32 @@ struct 9 9 621345559900192.4201210000 621345559900192.4201206300 --- !query 479 +-- !query TRUNCATE TABLE num_result --- !query 479 schema +-- !query schema struct<> --- !query 479 output +-- !query output --- !query 480 +-- !query INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val FROM num_data t1, num_data t2 WHERE t2.val != '0.0' --- !query 480 schema +-- !query schema struct<> --- !query 480 output +-- !query output --- !query 481 +-- !query SELECT t1.id1, t1.id2, t1.result, t2.expected FROM num_result t1, num_exp_div t2 WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 AND t1.result != t2.expected --- !query 481 schema +-- !query schema struct --- !query 481 output +-- !query output 2 3 -7967167.5673780000 -7967167.5673775051 2 4 -4.4026750000 -4.4026748005 2 5 -2094.1886690000 -2094.1886691456 @@ -4000,32 +4000,32 @@ struct 9 8 -332.8855660000 -332.8855656982 --- !query 482 +-- !query TRUNCATE TABLE num_result --- !query 482 schema +-- !query schema struct<> --- !query 482 output +-- !query output --- !query 483 +-- !query INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) FROM num_data t1, num_data t2 WHERE t2.val != '0.0' --- !query 483 schema +-- !query schema struct<> --- !query 483 output +-- !query output --- !query 484 +-- !query SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected FROM num_result t1, num_exp_div t2 WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 AND t1.result != round(t2.expected, 80) --- !query 484 schema +-- !query schema struct --- !query 484 output +-- !query output 2 3 -7967167.5673780000 -7967167.5673775051 2 4 -4.4026750000 -4.4026748005 2 5 -2094.1886690000 -2094.1886691456 @@ -4084,118 +4084,118 @@ struct 9 8 -332.8855660000 -332.8855656982 --- !query 485 +-- !query TRUNCATE TABLE num_result --- !query 485 schema +-- !query schema struct<> --- !query 485 output +-- !query output --- !query 486 +-- !query INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) FROM num_data --- !query 486 schema +-- !query schema struct<> --- !query 486 output +-- !query output --- !query 487 +-- !query SELECT t1.id1, t1.result, t2.expected FROM num_result t1, num_exp_sqrt t2 WHERE t1.id1 = t2.id AND t1.result != t2.expected --- !query 487 schema +-- !query schema struct --- !query 487 output +-- !query output --- !query 488 +-- !query TRUNCATE TABLE num_result --- !query 488 schema +-- !query schema struct<> --- !query 488 output +-- !query output --- !query 489 +-- !query INSERT INTO num_result SELECT id, 0, LN(ABS(val)) FROM num_data WHERE val != '0.0' --- !query 489 schema +-- !query schema struct<> --- !query 489 output +-- !query output --- !query 490 +-- !query SELECT t1.id1, t1.result, t2.expected FROM num_result t1, num_exp_ln t2 WHERE t1.id1 = t2.id AND t1.result != t2.expected --- !query 490 schema +-- !query schema struct --- !query 490 output +-- !query output --- !query 491 +-- !query TRUNCATE TABLE num_result --- !query 491 schema +-- !query schema struct<> --- !query 491 output +-- !query output --- !query 492 +-- !query INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) FROM num_data WHERE val != '0.0' --- !query 492 schema +-- !query schema struct<> --- !query 492 output +-- !query output --- !query 493 +-- !query SELECT t1.id1, t1.result, t2.expected FROM num_result t1, num_exp_log10 t2 WHERE t1.id1 = t2.id AND t1.result != t2.expected --- !query 493 schema +-- !query schema struct --- !query 493 output +-- !query output --- !query 494 +-- !query TRUNCATE TABLE num_result --- !query 494 schema +-- !query schema struct<> --- !query 494 output +-- !query output --- !query 495 +-- !query INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) FROM num_data WHERE val != '0.0' --- !query 495 schema +-- !query schema struct<> --- !query 495 output +-- !query output --- !query 496 +-- !query SELECT t1.id1, t1.result, t2.expected FROM num_result t1, num_exp_power_10_ln t2 WHERE t1.id1 = t2.id AND t1.result != t2.expected --- !query 496 schema +-- !query schema struct --- !query 496 output +-- !query output 2 224790267919917440.0000000000 224790267919917955.1326161858 4 7405685069595001.0000000000 7405685069594999.0773399947 5 5068226527.3212630000 5068226527.3212726541 @@ -4204,75 +4204,75 @@ struct 9 107511333880051872.0000000000 107511333880052007.0414112467 --- !query 497 +-- !query SELECT AVG(val) FROM num_data --- !query 497 schema +-- !query schema struct --- !query 497 output +-- !query output -13430913.59224232070000 --- !query 498 +-- !query CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet --- !query 498 schema +-- !query schema struct<> --- !query 498 output +-- !query output --- !query 499 +-- !query INSERT INTO fract_only VALUES (1, 0.0) --- !query 499 schema +-- !query schema struct<> --- !query 499 output +-- !query output --- !query 500 +-- !query INSERT INTO fract_only VALUES (2, 0.1) --- !query 500 schema +-- !query schema struct<> --- !query 500 output +-- !query output --- !query 501 +-- !query INSERT INTO fract_only VALUES (4, -0.9999) --- !query 501 schema +-- !query schema struct<> --- !query 501 output +-- !query output --- !query 502 +-- !query INSERT INTO fract_only VALUES (5, 0.99994) --- !query 502 schema +-- !query schema struct<> --- !query 502 output +-- !query output --- !query 503 +-- !query INSERT INTO fract_only VALUES (7, 0.00001) --- !query 503 schema +-- !query schema struct<> --- !query 503 output +-- !query output --- !query 504 +-- !query INSERT INTO fract_only VALUES (8, 0.00017) --- !query 504 schema +-- !query schema struct<> --- !query 504 output +-- !query output --- !query 505 +-- !query SELECT * FROM fract_only --- !query 505 schema +-- !query schema struct --- !query 505 output +-- !query output 1 0.0000 2 0.1000 4 -0.9999 @@ -4281,131 +4281,131 @@ struct 8 0.0002 --- !query 506 +-- !query DROP TABLE fract_only --- !query 506 schema +-- !query schema struct<> --- !query 506 output +-- !query output --- !query 507 +-- !query SELECT decimal(double('NaN')) --- !query 507 schema +-- !query schema struct --- !query 507 output +-- !query output NULL --- !query 508 +-- !query SELECT decimal(double('Infinity')) --- !query 508 schema +-- !query schema struct --- !query 508 output +-- !query output NULL --- !query 509 +-- !query SELECT decimal(double('-Infinity')) --- !query 509 schema +-- !query schema struct --- !query 509 output +-- !query output NULL --- !query 510 +-- !query SELECT decimal(float('NaN')) --- !query 510 schema +-- !query schema struct --- !query 510 output +-- !query output NULL --- !query 511 +-- !query SELECT decimal(float('Infinity')) --- !query 511 schema +-- !query schema struct --- !query 511 output +-- !query output NULL --- !query 512 +-- !query SELECT decimal(float('-Infinity')) --- !query 512 schema +-- !query schema struct --- !query 512 output +-- !query output NULL --- !query 513 +-- !query CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet --- !query 513 schema +-- !query schema struct<> --- !query 513 output +-- !query output --- !query 514 +-- !query INSERT INTO ceil_floor_round VALUES (-5.5) --- !query 514 schema +-- !query schema struct<> --- !query 514 output +-- !query output --- !query 515 +-- !query INSERT INTO ceil_floor_round VALUES (-5.499999) --- !query 515 schema +-- !query schema struct<> --- !query 515 output +-- !query output --- !query 516 +-- !query INSERT INTO ceil_floor_round VALUES (9.5) --- !query 516 schema +-- !query schema struct<> --- !query 516 output +-- !query output --- !query 517 +-- !query INSERT INTO ceil_floor_round VALUES (9.4999999) --- !query 517 schema +-- !query schema struct<> --- !query 517 output +-- !query output --- !query 518 +-- !query INSERT INTO ceil_floor_round VALUES (0.0) --- !query 518 schema +-- !query schema struct<> --- !query 518 output +-- !query output --- !query 519 +-- !query INSERT INTO ceil_floor_round VALUES (0.0000001) --- !query 519 schema +-- !query schema struct<> --- !query 519 output +-- !query output --- !query 520 +-- !query INSERT INTO ceil_floor_round VALUES (-0.000001) --- !query 520 schema +-- !query schema struct<> --- !query 520 output +-- !query output --- !query 521 +-- !query SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round --- !query 521 schema +-- !query schema struct --- !query 521 output +-- !query output -0.000001000000000000 0 0 -1 0 -5.499999000000000000 -5 -5 -6 -5 -5.500000000000000000 -5 -5 -6 -6 @@ -4415,67 +4415,67 @@ struct --- !query 522 output +-- !query output --- !query 523 +-- !query CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet --- !query 523 schema +-- !query schema struct<> --- !query 523 output +-- !query output --- !query 524 +-- !query INSERT INTO num_input_test VALUES (double(trim(' 123'))) --- !query 524 schema +-- !query schema struct<> --- !query 524 output +-- !query output --- !query 525 +-- !query INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))) --- !query 525 schema +-- !query schema struct<> --- !query 525 output +-- !query output --- !query 526 +-- !query INSERT INTO num_input_test VALUES (double(trim(' -93853'))) --- !query 526 schema +-- !query schema struct<> --- !query 526 output +-- !query output --- !query 527 +-- !query INSERT INTO num_input_test VALUES (555.50) --- !query 527 schema +-- !query schema struct<> --- !query 527 output +-- !query output --- !query 528 +-- !query INSERT INTO num_input_test VALUES (-555.50) --- !query 528 schema +-- !query schema struct<> --- !query 528 output +-- !query output --- !query 529 +-- !query SELECT * FROM num_input_test --- !query 529 schema +-- !query schema struct --- !query 529 output +-- !query output -555.500000000000000000 -93853.000000000000000000 123.000000000000000000 @@ -4483,385 +4483,385 @@ struct 555.500000000000000000 --- !query 530 +-- !query select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 --- !query 530 schema +-- !query schema struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) / CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,6)> --- !query 530 output +-- !query output 1.000000 --- !query 531 +-- !query select div(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) --- !query 531 schema +-- !query schema struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) div CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,0)> --- !query 531 output +-- !query output 0 --- !query 532 +-- !query select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) --- !query 532 schema +-- !query schema struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(22,0)> --- !query 532 output +-- !query output 999999999999999999999 --- !query 533 +-- !query select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) --- !query 533 schema +-- !query schema struct<(CAST(CAST(-9999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) div CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,0)> --- !query 533 output +-- !query output -9 --- !query 534 +-- !query select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) --- !query 534 schema +-- !query schema struct<(CAST(CAST(-9999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(22,0)> --- !query 534 output +-- !query output -999999999999999999999 --- !query 535 +-- !query select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)*1000000000000000000000 + mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) --- !query 535 schema +-- !query schema struct<(CAST((CAST((CAST(CAST(-9999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) div CAST(1000000000000000000000 AS DECIMAL(38,0))) AS DECIMAL(38,0)) * CAST(1000000000000000000000 AS DECIMAL(38,0))) AS DECIMAL(38,0)) + CAST((CAST(CAST(-9999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))) AS DECIMAL(38,0))):decimal(38,0)> --- !query 535 output +-- !query output -9999999999999999999999 --- !query 536 +-- !query select mod (70.0,70) --- !query 536 schema +-- !query schema struct<(CAST(70.0 AS DECIMAL(3,1)) % CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(3,1)> --- !query 536 output +-- !query output 0.0 --- !query 537 +-- !query select div (70.0,70) --- !query 537 schema +-- !query schema struct<(CAST(70.0 AS DECIMAL(3,1)) div CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(2,0)> --- !query 537 output +-- !query output 1 --- !query 538 +-- !query select 70.0 / 70 --- !query 538 schema +-- !query schema struct<(CAST(70.0 AS DECIMAL(3,1)) / CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(8,6)> --- !query 538 output +-- !query output 1.000000 --- !query 539 +-- !query select 12345678901234567890 % 123 --- !query 539 schema +-- !query schema struct<(CAST(12345678901234567890 AS DECIMAL(20,0)) % CAST(CAST(123 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(3,0)> --- !query 539 output +-- !query output 78 --- !query 540 +-- !query select exp(0.0) --- !query 540 schema +-- !query schema struct --- !query 540 output +-- !query output 1.0 --- !query 541 +-- !query select exp(1.0) --- !query 541 schema +-- !query schema struct --- !query 541 output +-- !query output 2.7182818284590455 --- !query 542 +-- !query select exp(32.999) --- !query 542 schema +-- !query schema struct --- !query 542 output +-- !query output 2.1442904349215556E14 --- !query 543 +-- !query select exp(-32.999) --- !query 543 schema +-- !query schema struct --- !query 543 output +-- !query output 4.663547361468238E-15 --- !query 544 +-- !query select exp(123.456) --- !query 544 schema +-- !query schema struct --- !query 544 output +-- !query output 4.132944352778106E53 --- !query 545 +-- !query select exp(-123.456) --- !query 545 schema +-- !query schema struct --- !query 545 output +-- !query output 2.4195825412645934E-54 --- !query 546 +-- !query select exp(1234.5678) --- !query 546 schema +-- !query schema struct --- !query 546 output +-- !query output Infinity --- !query 547 +-- !query select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))) --- !query 547 schema +-- !query schema struct --- !query 547 output +-- !query output 0 1 2 3 --- !query 548 +-- !query select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))) --- !query 548 schema +-- !query schema struct --- !query 548 output +-- !query output 0 1 2 3 --- !query 549 +-- !query select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))) --- !query 549 schema +-- !query schema struct --- !query 549 output +-- !query output 0 2 4 --- !query 550 +-- !query select ln(1.2345678e-28) --- !query 550 schema +-- !query schema struct --- !query 550 output +-- !query output -64.26166165451762 --- !query 551 +-- !query select ln(0.0456789) --- !query 551 schema +-- !query schema struct --- !query 551 output +-- !query output -3.0861187944847437 --- !query 552 +-- !query select ln(0.99949452) --- !query 552 schema +-- !query schema struct --- !query 552 output +-- !query output -5.056077980832118E-4 --- !query 553 +-- !query select ln(1.00049687395) --- !query 553 schema +-- !query schema struct --- !query 553 output +-- !query output 4.967505490136803E-4 --- !query 554 +-- !query select ln(1234.567890123456789) --- !query 554 schema +-- !query schema struct --- !query 554 output +-- !query output 7.11847630129779 --- !query 555 +-- !query select ln(5.80397490724e5) --- !query 555 schema +-- !query schema struct --- !query 555 output +-- !query output 13.271468476626518 --- !query 556 +-- !query select ln(9.342536355e34) --- !query 556 schema +-- !query schema struct --- !query 556 output +-- !query output 80.52247093552418 --- !query 557 +-- !query select log(3.4634998359873254962349856073435545) --- !query 557 schema +-- !query schema struct --- !query 557 output +-- !query output 1.2422795911259166 --- !query 558 +-- !query select log(9.999999999999999999) --- !query 558 schema +-- !query schema struct --- !query 558 output +-- !query output 2.302585092994046 --- !query 559 +-- !query select log(10.00000000000000000) --- !query 559 schema +-- !query schema struct --- !query 559 output +-- !query output 2.302585092994046 --- !query 560 +-- !query select log(10.00000000000000001) --- !query 560 schema +-- !query schema struct --- !query 560 output +-- !query output 2.302585092994046 --- !query 561 +-- !query select log(590489.45235237) --- !query 561 schema +-- !query schema struct --- !query 561 output +-- !query output 13.288707052228641 --- !query 562 +-- !query select log(0.99923, 4.58934e34) --- !query 562 schema +-- !query schema struct --- !query 562 output +-- !query output -103611.55579543479 --- !query 563 +-- !query select log(1.000016, 8.452010e18) --- !query 563 schema +-- !query schema struct --- !query 563 output +-- !query output 2723830.287707013 --- !query 564 +-- !query SELECT SUM(decimal(9999)) FROM range(1, 100001) --- !query 564 schema +-- !query schema struct --- !query 564 output +-- !query output 999900000 --- !query 565 +-- !query SELECT SUM(decimal(-9999)) FROM range(1, 100001) --- !query 565 schema +-- !query schema struct --- !query 565 output +-- !query output -999900000 --- !query 566 +-- !query DROP TABLE num_data --- !query 566 schema +-- !query schema struct<> --- !query 566 output +-- !query output --- !query 567 +-- !query DROP TABLE num_exp_add --- !query 567 schema +-- !query schema struct<> --- !query 567 output +-- !query output --- !query 568 +-- !query DROP TABLE num_exp_sub --- !query 568 schema +-- !query schema struct<> --- !query 568 output +-- !query output --- !query 569 +-- !query DROP TABLE num_exp_div --- !query 569 schema +-- !query schema struct<> --- !query 569 output +-- !query output --- !query 570 +-- !query DROP TABLE num_exp_mul --- !query 570 schema +-- !query schema struct<> --- !query 570 output +-- !query output --- !query 571 +-- !query DROP TABLE num_exp_sqrt --- !query 571 schema +-- !query schema struct<> --- !query 571 output +-- !query output --- !query 572 +-- !query DROP TABLE num_exp_ln --- !query 572 schema +-- !query schema struct<> --- !query 572 output +-- !query output --- !query 573 +-- !query DROP TABLE num_exp_log10 --- !query 573 schema +-- !query schema struct<> --- !query 573 output +-- !query output --- !query 574 +-- !query DROP TABLE num_exp_power_10_ln --- !query 574 schema +-- !query schema struct<> --- !query 574 output +-- !query output --- !query 575 +-- !query DROP TABLE num_result --- !query 575 schema +-- !query schema struct<> --- !query 575 output +-- !query output --- !query 576 +-- !query DROP TABLE num_input_test --- !query 576 schema +-- !query schema struct<> --- !query 576 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out index a106e33c2a595..1e59036b979b4 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select.sql.out @@ -2,15 +2,15 @@ -- Number of queries: 37 --- !query 0 +-- !query create or replace temporary view onek2 as select * from onek --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create or replace temporary view INT8_TBL as select * from values (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), @@ -18,19 +18,19 @@ create or replace temporary view INT8_TBL as select * from values (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) as INT8_TBL(q1, q2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM onek WHERE onek.unique1 < 10 ORDER BY onek.unique1 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx 1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx 2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx @@ -43,13 +43,13 @@ struct --- !query 3 output +-- !query output 19 TAAAAA 18 SAAAAA 17 RAAAAA @@ -72,13 +72,13 @@ struct 0 AAAAAA --- !query 4 +-- !query SELECT onek.unique1, onek.stringu1 FROM onek WHERE onek.unique1 > 980 ORDER BY stringu1 ASC --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 988 AMAAAA 989 BMAAAA 990 CMAAAA @@ -100,13 +100,13 @@ struct 987 ZLAAAA --- !query 5 +-- !query SELECT onek.unique1, onek.string4 FROM onek WHERE onek.unique1 > 980 ORDER BY string4 ASC, unique1 DESC --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 999 AAAAxx 995 AAAAxx 983 AAAAxx @@ -128,13 +128,13 @@ struct 984 VVVVxx --- !query 6 +-- !query SELECT onek.unique1, onek.string4 FROM onek WHERE onek.unique1 > 980 ORDER BY string4 DESC, unique1 ASC --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 984 VVVVxx 989 VVVVxx 992 VVVVxx @@ -156,13 +156,13 @@ struct 999 AAAAxx --- !query 7 +-- !query SELECT onek.unique1, onek.string4 FROM onek WHERE onek.unique1 < 20 ORDER BY unique1 DESC, string4 ASC --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 19 OOOOxx 18 VVVVxx 17 HHHHxx @@ -185,13 +185,13 @@ struct 0 OOOOxx --- !query 8 +-- !query SELECT onek.unique1, onek.string4 FROM onek WHERE onek.unique1 < 20 ORDER BY unique1 ASC, string4 DESC --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 0 OOOOxx 1 OOOOxx 2 OOOOxx @@ -214,11 +214,11 @@ struct 19 OOOOxx --- !query 9 +-- !query SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx 1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx 2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx @@ -231,13 +231,13 @@ struct --- !query 10 output +-- !query output 19 TAAAAA 18 SAAAAA 17 RAAAAA @@ -260,12 +260,12 @@ struct 0 AAAAAA --- !query 11 +-- !query SELECT onek2.unique1, onek2.stringu1 FROM onek2 WHERE onek2.unique1 > 980 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 981 TLAAAA 982 ULAAAA 983 VLAAAA @@ -287,69 +287,69 @@ struct 999 LMAAAA --- !query 12 +-- !query CREATE TABLE tmp USING parquet AS SELECT two, stringu1, ten, string4 FROM onek --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query select foo.* from (select 1) as foo --- !query 13 schema +-- !query schema struct<1:int> --- !query 13 output +-- !query output 1 --- !query 14 +-- !query select foo.* from (select null) as foo --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output NULL --- !query 15 +-- !query select foo.* from (select 'xyzzy',1,null) as foo --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output xyzzy 1 NULL --- !query 16 +-- !query select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) WHERE onek.unique1 = v.i and onek.stringu1 = v.j --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 147 0 1 3 7 7 7 47 147 147 147 14 15 RFAAAA AAAAAA AAAAxx 147 RFAAAA 931 1 1 3 1 11 1 31 131 431 931 2 3 VJAAAA BAAAAA HHHHxx 931 VJAAAA --- !query 17 +-- !query VALUES (1,2), (3,4+4), (7,77.7) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 2.0 3 8.0 7 77.7 --- !query 18 +-- !query VALUES (1,2), (3,4+4), (7,77.7) UNION ALL SELECT 2+2, 57 UNION ALL TABLE int8_tbl --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 2.0 123 456.0 123 4567890123456789.0 @@ -361,20 +361,20 @@ struct 7 77.7 --- !query 19 +-- !query CREATE OR REPLACE TEMPORARY VIEW foo AS SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query SELECT * FROM foo ORDER BY f1 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output NULL NULL 1 @@ -384,11 +384,11 @@ NULL 42 --- !query 21 +-- !query SELECT * FROM foo ORDER BY f1 ASC --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL NULL 1 @@ -398,11 +398,11 @@ NULL 42 --- !query 22 +-- !query SELECT * FROM foo ORDER BY f1 NULLS FIRST --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL NULL 1 @@ -412,11 +412,11 @@ NULL 42 --- !query 23 +-- !query SELECT * FROM foo ORDER BY f1 DESC --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 42 10 7 @@ -426,11 +426,11 @@ NULL NULL --- !query 24 +-- !query SELECT * FROM foo ORDER BY f1 DESC NULLS LAST --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 42 10 7 @@ -440,103 +440,103 @@ NULL NULL --- !query 25 +-- !query select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx --- !query 26 +-- !query select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 11 --- !query 27 +-- !query select * from onek2 where unique2 = 11 and stringu1 < 'B' --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx --- !query 28 +-- !query select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 11 --- !query 29 +-- !query select unique2 from onek2 where unique2 = 11 and stringu1 < 'C' --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 11 --- !query 30 +-- !query select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 11 --- !query 31 +-- !query select unique1, unique2 from onek2 where (unique2 = 11 or unique1 = 0) and stringu1 < 'B' --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 0 998 494 11 --- !query 32 +-- !query select unique1, unique2 from onek2 where (unique2 = 11 and stringu1 < 'B') or unique1 = 0 --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 0 998 494 11 --- !query 33 +-- !query SELECT 1 AS x ORDER BY x --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 1 --- !query 34 +-- !query select * from (values (2),(null),(1)) v(k) where k = k order by k --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 1 2 --- !query 35 +-- !query select * from (values (2),(null),(1)) v(k) where k = k --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 1 2 --- !query 36 +-- !query drop table tmp --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_distinct.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_distinct.sql.out index 38eae1739f553..53003e70f289a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_distinct.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_distinct.sql.out @@ -2,30 +2,30 @@ -- Number of queries: 19 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW tmp AS SELECT two, stringu1, ten, string4 FROM onek --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT DISTINCT two FROM tmp ORDER BY 1 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 0 1 --- !query 2 +-- !query SELECT DISTINCT ten FROM tmp ORDER BY 1 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 0 1 2 @@ -38,24 +38,24 @@ struct 9 --- !query 3 +-- !query SELECT DISTINCT string4 FROM tmp ORDER BY 1 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output AAAAxx HHHHxx OOOOxx VVVVxx --- !query 4 +-- !query SELECT DISTINCT two, string4, ten FROM tmp ORDER BY two ASC, string4 ASC, ten ASC --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 0 AAAAxx 0 0 AAAAxx 2 0 AAAAxx 4 @@ -98,128 +98,128 @@ struct 1 VVVVxx 9 --- !query 5 +-- !query SELECT count(*) FROM (SELECT DISTINCT two, four, two FROM tenk1) ss --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 4 --- !query 6 +-- !query CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM (VALUES (1), (2), (3), (NULL)) AS v(f1) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 true 2 false 3 true NULL true --- !query 8 +-- !query SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 true 2 true 3 true NULL false --- !query 9 +-- !query SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 false 2 false 3 false NULL false --- !query 10 +-- !query SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 true 2 true 3 true NULL false --- !query 11 +-- !query SELECT 1 IS DISTINCT FROM 2 as `yes` --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output true --- !query 12 +-- !query SELECT 2 IS DISTINCT FROM 2 as `no` --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output false --- !query 13 +-- !query SELECT 2 IS DISTINCT FROM null as `yes` --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output true --- !query 14 +-- !query SELECT null IS DISTINCT FROM null as `no` --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output false --- !query 15 +-- !query SELECT 1 IS NOT DISTINCT FROM 2 as `no` --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output false --- !query 16 +-- !query SELECT 2 IS NOT DISTINCT FROM 2 as `yes` --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output true --- !query 17 +-- !query SELECT 2 IS NOT DISTINCT FROM null as `no` --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output false --- !query 18 +-- !query SELECT null IS NOT DISTINCT FROM null as `yes` --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output true diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out index 02536ebd8ebea..cbf4cfa58cdb9 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out @@ -2,186 +2,186 @@ -- Number of queries: 22 --- !query 0 +-- !query CREATE TABLE test_having (a int, b int, c string, d string) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT b, c FROM test_having GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 XXXX 3 bbbb --- !query 12 +-- !query SELECT b, c FROM test_having GROUP BY b, c HAVING b = 3 ORDER BY b, c --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 3 BBBB 3 bbbb --- !query 13 +-- !query SELECT c, max(a) FROM test_having GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) ORDER BY c --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output XXXX 0 bbbb 5 --- !query 14 +-- !query SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output --- !query 15 +-- !query SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 0 9 --- !query 16 +-- !query SELECT a FROM test_having HAVING min(a) < max(a) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and 'default.test_having.`a`' is not an aggregate function. Wrap '(min(default.test_having.`a`) AS `min(a#x)`, max(default.test_having.`a`) AS `max(a#x)`)' in windowing function(s) or wrap 'default.test_having.`a`' in first() (or first_value) if you don't care which value you get.; --- !query 17 +-- !query SELECT 1 AS one FROM test_having HAVING a > 1 --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`a`' given input columns: [one]; line 1 pos 40 --- !query 18 +-- !query SELECT 1 AS one FROM test_having HAVING 1 > 2 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output --- !query 19 +-- !query SELECT 1 AS one FROM test_having HAVING 1 < 2 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1 --- !query 20 +-- !query SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1 --- !query 21 +-- !query DROP TABLE test_having --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out index e9ba62801d6a2..4ecfabccdf414 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out @@ -2,99 +2,99 @@ -- Number of queries: 38 --- !query 0 +-- !query CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output ABAB 2 BBBB 2 CCCC 2 @@ -103,11 +103,11 @@ bbbb 1 cccc 2 --- !query 12 +-- !query SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 2 2 2 @@ -116,43 +116,43 @@ struct 2 --- !query 13 +-- !query SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`b`' given input columns: [count(1)]; line 1 pos 61 --- !query 14 +-- !query SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 2 3 4 --- !query 15 +-- !query SELECT test_missing_target.b, count(*) FROM test_missing_target GROUP BY b ORDER BY b --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 1 2 2 3 3 4 4 --- !query 16 +-- !query SELECT c FROM test_missing_target ORDER BY a --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output XXXX ABAB ABAB @@ -165,30 +165,30 @@ CCCC CCCC --- !query 17 +-- !query SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 4 3 2 1 --- !query 18 +-- !query SELECT count(*) FROM test_missing_target ORDER BY 1 desc --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 10 --- !query 19 +-- !query SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output ABAB 2 BBBB 2 CCCC 2 @@ -197,32 +197,32 @@ bbbb 1 cccc 2 --- !query 20 +-- !query SELECT c, count(*) FROM test_missing_target GROUP BY 3 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.AnalysisException GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 53 --- !query 21 +-- !query SELECT count(*) FROM test_missing_target x, test_missing_target y WHERE x.a = y.a GROUP BY b ORDER BY b --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 --- !query 22 +-- !query SELECT a, a FROM test_missing_target ORDER BY a --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 0 0 1 1 2 2 @@ -235,12 +235,12 @@ struct 9 9 --- !query 23 +-- !query SELECT a/2, a/2 FROM test_missing_target ORDER BY a/2 --- !query 23 schema +-- !query schema struct<(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double,(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double> --- !query 23 output +-- !query output 0.0 0.0 0.5 0.5 1.0 1.0 @@ -253,12 +253,12 @@ struct<(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double,(CAST(a AS DOUBLE) / CAST( 4.5 4.5 --- !query 24 +-- !query SELECT a/2, a/2 FROM test_missing_target GROUP BY a/2 ORDER BY a/2 --- !query 24 schema +-- !query schema struct<(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double,(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double> --- !query 24 output +-- !query output 0.0 0.0 0.5 0.5 1.0 1.0 @@ -271,93 +271,93 @@ struct<(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double,(CAST(a AS DOUBLE) / CAST( 4.5 4.5 --- !query 25 +-- !query SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y WHERE x.a = y.a GROUP BY x.b ORDER BY x.b --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1 1 2 2 3 3 4 4 --- !query 26 +-- !query SELECT count(*) FROM test_missing_target x, test_missing_target y WHERE x.a = y.a GROUP BY x.b ORDER BY x.b --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 2 3 4 --- !query 27 +-- !query SELECT a%2, count(b) FROM test_missing_target GROUP BY test_missing_target.a%2 ORDER BY test_missing_target.a%2 --- !query 27 schema +-- !query schema struct<(a % 2):int,count(b):bigint> --- !query 27 output +-- !query output 0 5 1 5 --- !query 28 +-- !query SELECT count(c) FROM test_missing_target GROUP BY lower(test_missing_target.c) ORDER BY lower(test_missing_target.c) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 2 3 4 1 --- !query 29 +-- !query SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`b`' given input columns: [count(a)]; line 1 pos 61 --- !query 30 +-- !query SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2 --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1 2 3 4 --- !query 31 +-- !query SELECT lower(test_missing_target.c), count(c) FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output abab 2 bbbb 3 cccc 4 xxxx 1 --- !query 32 +-- !query SELECT a FROM test_missing_target ORDER BY upper(d) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 0 1 2 @@ -370,56 +370,56 @@ struct 9 --- !query 33 +-- !query SELECT count(b) FROM test_missing_target GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 4 3 2 1 --- !query 34 +-- !query SELECT count(x.a) FROM test_missing_target x, test_missing_target y WHERE x.a = y.a GROUP BY b/2 ORDER BY b/2 --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 --- !query 35 +-- !query SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y WHERE x.a = y.a GROUP BY x.b/2 ORDER BY x.b/2 --- !query 35 schema +-- !query schema struct<(CAST(b AS DOUBLE) / CAST(2 AS DOUBLE)):double,count(b):bigint> --- !query 35 output +-- !query output 0.5 1 1.0 2 1.5 3 2.0 4 --- !query 36 +-- !query SELECT count(b) FROM test_missing_target x, test_missing_target y WHERE x.a = y.a GROUP BY x.b/2 --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 13 --- !query 37 +-- !query DROP TABLE test_missing_target --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out index 4619f0ded7d92..c30eea8ab689d 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/strings.sql.out @@ -2,997 +2,997 @@ -- Number of queries: 122 --- !query 0 +-- !query SELECT 'first line' ' - next line' ' - third line' AS `Three lines to one` --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output first line - next line - third line --- !query 1 +-- !query SELECT 'first line' ' - next line' /* this comment is not allowed here */ ' - third line' AS `Illegal comment within continuation` --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output first line - next line - third line --- !query 2 +-- !query SELECT binary('\\xDeAdBeEf') --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output \xDeAdBeEf --- !query 3 +-- !query SELECT binary('\\x De Ad Be Ef ') --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output \x De Ad Be Ef --- !query 4 +-- !query SELECT binary('\\xDe00BeEf') --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output \xDe00BeEf --- !query 5 +-- !query SELECT binary('DeAdBeEf') --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output DeAdBeEf --- !query 6 +-- !query SELECT binary('De\\000dBeEf') --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output De\000dBeEf --- !query 7 +-- !query SELECT binary('De\\123dBeEf') --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output De\123dBeEf --- !query 8 +-- !query SELECT TRIM(BOTH FROM ' bunch o blanks ') = 'bunch o blanks' AS `bunch o blanks` --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output true --- !query 9 +-- !query SELECT TRIM(LEADING FROM ' bunch o blanks ') = 'bunch o blanks ' AS `bunch o blanks ` --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output true --- !query 10 +-- !query SELECT TRIM(TRAILING FROM ' bunch o blanks ') = ' bunch o blanks' AS ` bunch o blanks` --- !query 10 schema +-- !query schema struct< bunch o blanks:boolean> --- !query 10 output +-- !query output true --- !query 11 +-- !query SELECT TRIM(BOTH 'x' FROM 'xxxxxsome Xsxxxxx') = 'some Xs' AS `some Xs` --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output true --- !query 12 +-- !query SELECT SUBSTRING('1234567890' FROM 3) = '34567890' AS `34567890` --- !query 12 schema +-- !query schema struct<34567890:boolean> --- !query 12 output +-- !query output true --- !query 13 +-- !query SELECT SUBSTRING('1234567890' FROM 4 FOR 3) = '456' AS `456` --- !query 13 schema +-- !query schema struct<456:boolean> --- !query 13 output +-- !query output true --- !query 14 +-- !query SELECT POSITION('4' IN '1234567890') = '4' AS `4` --- !query 14 schema +-- !query schema struct<4:boolean> --- !query 14 output +-- !query output true --- !query 15 +-- !query SELECT POSITION('5' IN '1234567890') = '5' AS `5` --- !query 15 schema +-- !query schema struct<5:boolean> --- !query 15 output +-- !query output true --- !query 16 +-- !query SELECT OVERLAY('abcdef' PLACING '45' FROM 4) AS `abc45f` --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output abc45f --- !query 17 +-- !query SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5) AS `yabadaba` --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output yabadaba --- !query 18 +-- !query SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0) AS `yabadabadoo` --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output yabadabadoo --- !query 19 +-- !query SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4) AS `bubba` --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output bubba --- !query 20 +-- !query SELECT 'hawkeye' LIKE 'h%' AS `true` --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output true --- !query 21 +-- !query SELECT 'hawkeye' NOT LIKE 'h%' AS `false` --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output false --- !query 22 +-- !query SELECT 'hawkeye' LIKE 'H%' AS `false` --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output false --- !query 23 +-- !query SELECT 'hawkeye' NOT LIKE 'H%' AS `true` --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output true --- !query 24 +-- !query SELECT 'hawkeye' LIKE 'indio%' AS `false` --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output false --- !query 25 +-- !query SELECT 'hawkeye' NOT LIKE 'indio%' AS `true` --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output true --- !query 26 +-- !query SELECT 'hawkeye' LIKE 'h%eye' AS `true` --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output true --- !query 27 +-- !query SELECT 'hawkeye' NOT LIKE 'h%eye' AS `false` --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output false --- !query 28 +-- !query SELECT 'indio' LIKE '_ndio' AS `true` --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output true --- !query 29 +-- !query SELECT 'indio' NOT LIKE '_ndio' AS `false` --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output false --- !query 30 +-- !query SELECT 'indio' LIKE 'in__o' AS `true` --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output true --- !query 31 +-- !query SELECT 'indio' NOT LIKE 'in__o' AS `false` --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output false --- !query 32 +-- !query SELECT 'indio' LIKE 'in_o' AS `false` --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output false --- !query 33 +-- !query SELECT 'indio' NOT LIKE 'in_o' AS `true` --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output true --- !query 34 +-- !query SELECT 'hawkeye' LIKE 'h%' ESCAPE '#' AS `true` --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output true --- !query 35 +-- !query SELECT 'hawkeye' NOT LIKE 'h%' ESCAPE '#' AS `false` --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output false --- !query 36 +-- !query SELECT 'indio' LIKE 'ind_o' ESCAPE '$' AS `true` --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output true --- !query 37 +-- !query SELECT 'indio' NOT LIKE 'ind_o' ESCAPE '$' AS `false` --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output false --- !query 38 +-- !query SELECT 'h%' LIKE 'h#%' ESCAPE '#' AS `true` --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output true --- !query 39 +-- !query SELECT 'h%' NOT LIKE 'h#%' ESCAPE '#' AS `false` --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output false --- !query 40 +-- !query SELECT 'h%wkeye' LIKE 'h#%' ESCAPE '#' AS `false` --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output false --- !query 41 +-- !query SELECT 'h%wkeye' NOT LIKE 'h#%' ESCAPE '#' AS `true` --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output true --- !query 42 +-- !query SELECT 'h%wkeye' LIKE 'h#%%' ESCAPE '#' AS `true` --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output true --- !query 43 +-- !query SELECT 'h%wkeye' NOT LIKE 'h#%%' ESCAPE '#' AS `false` --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output false --- !query 44 +-- !query SELECT 'h%awkeye' LIKE 'h#%a%k%e' ESCAPE '#' AS `true` --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output true --- !query 45 +-- !query SELECT 'h%awkeye' NOT LIKE 'h#%a%k%e' ESCAPE '#' AS `false` --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output false --- !query 46 +-- !query SELECT 'indio' LIKE '_ndio' ESCAPE '$' AS `true` --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output true --- !query 47 +-- !query SELECT 'indio' NOT LIKE '_ndio' ESCAPE '$' AS `false` --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output false --- !query 48 +-- !query SELECT 'i_dio' LIKE 'i$_d_o' ESCAPE '$' AS `true` --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output true --- !query 49 +-- !query SELECT 'i_dio' NOT LIKE 'i$_d_o' ESCAPE '$' AS `false` --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output false --- !query 50 +-- !query SELECT 'i_dio' LIKE 'i$_nd_o' ESCAPE '$' AS `false` --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output false --- !query 51 +-- !query SELECT 'i_dio' NOT LIKE 'i$_nd_o' ESCAPE '$' AS `true` --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output true --- !query 52 +-- !query SELECT 'i_dio' LIKE 'i$_d%o' ESCAPE '$' AS `true` --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output true --- !query 53 +-- !query SELECT 'i_dio' NOT LIKE 'i$_d%o' ESCAPE '$' AS `false` --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output false --- !query 54 +-- !query SELECT 'maca' LIKE 'm%aca' ESCAPE '%' AS `true` --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output org.apache.spark.sql.AnalysisException the pattern 'm%aca' is invalid, the escape character is not allowed to precede 'a'; --- !query 55 +-- !query SELECT 'maca' NOT LIKE 'm%aca' ESCAPE '%' AS `false` --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output org.apache.spark.sql.AnalysisException the pattern 'm%aca' is invalid, the escape character is not allowed to precede 'a'; --- !query 56 +-- !query SELECT 'ma%a' LIKE 'm%a%%a' ESCAPE '%' AS `true` --- !query 56 schema +-- !query schema struct<> --- !query 56 output +-- !query output org.apache.spark.sql.AnalysisException the pattern 'm%a%%a' is invalid, the escape character is not allowed to precede 'a'; --- !query 57 +-- !query SELECT 'ma%a' NOT LIKE 'm%a%%a' ESCAPE '%' AS `false` --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.AnalysisException the pattern 'm%a%%a' is invalid, the escape character is not allowed to precede 'a'; --- !query 58 +-- !query SELECT 'bear' LIKE 'b_ear' ESCAPE '_' AS `true` --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.AnalysisException the pattern 'b_ear' is invalid, the escape character is not allowed to precede 'e'; --- !query 59 +-- !query SELECT 'bear' NOT LIKE 'b_ear' ESCAPE '_' AS `false` --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException the pattern 'b_ear' is invalid, the escape character is not allowed to precede 'e'; --- !query 60 +-- !query SELECT 'be_r' LIKE 'b_e__r' ESCAPE '_' AS `true` --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException the pattern 'b_e__r' is invalid, the escape character is not allowed to precede 'e'; --- !query 61 +-- !query SELECT 'be_r' NOT LIKE 'b_e__r' ESCAPE '_' AS `false` --- !query 61 schema +-- !query schema struct<> --- !query 61 output +-- !query output org.apache.spark.sql.AnalysisException the pattern 'b_e__r' is invalid, the escape character is not allowed to precede 'e'; --- !query 62 +-- !query SELECT 'be_r' LIKE '__e__r' ESCAPE '_' AS `false` --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output false --- !query 63 +-- !query SELECT 'be_r' NOT LIKE '__e__r' ESCAPE '_' AS `true` --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output true --- !query 64 +-- !query SELECT 'foo' LIKE '_%' as t, 'f' LIKE '_%' as t, '' LIKE '_%' as f --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output true true false --- !query 65 +-- !query SELECT 'foo' LIKE '%_' as t, 'f' LIKE '%_' as t, '' LIKE '%_' as f --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output true true false --- !query 66 +-- !query SELECT 'foo' LIKE '__%' as t, 'foo' LIKE '___%' as t, 'foo' LIKE '____%' as f --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output true true false --- !query 67 +-- !query SELECT 'foo' LIKE '%__' as t, 'foo' LIKE '%___' as t, 'foo' LIKE '%____' as f --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output true true false --- !query 68 +-- !query SELECT 'jack' LIKE '%____%' AS t --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output true --- !query 69 +-- !query SELECT 'unknown' || ' and unknown' AS `Concat unknown types` --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output unknown and unknown --- !query 70 +-- !query SELECT string('text') || ' and unknown' AS `Concat text to unknown type` --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output text and unknown --- !query 71 +-- !query CREATE TABLE toasttest(f1 string) USING parquet --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output --- !query 72 +-- !query insert into toasttest values(repeat('1234567890',10000)) --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output --- !query 73 +-- !query insert into toasttest values(repeat('1234567890',10000)) --- !query 73 schema +-- !query schema struct<> --- !query 73 output +-- !query output --- !query 74 +-- !query insert into toasttest values(repeat('1234567890',10000)) --- !query 74 schema +-- !query schema struct<> --- !query 74 output +-- !query output --- !query 75 +-- !query insert into toasttest values(repeat('1234567890',10000)) --- !query 75 schema +-- !query schema struct<> --- !query 75 output +-- !query output --- !query 76 +-- !query SELECT substr(f1, 99995) from toasttest --- !query 76 schema +-- !query schema struct --- !query 76 output +-- !query output 567890 567890 567890 567890 --- !query 77 +-- !query SELECT substr(f1, 99995, 10) from toasttest --- !query 77 schema +-- !query schema struct --- !query 77 output +-- !query output 567890 567890 567890 567890 --- !query 78 +-- !query SELECT length('abcdef') AS `length_6` --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output 6 --- !query 79 +-- !query SELECT position('cd', 'abcdef') AS `pos_3` --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output 3 --- !query 80 +-- !query SELECT position('xy', 'abcdef') AS `pos_0` --- !query 80 schema +-- !query schema struct --- !query 80 output +-- !query output 0 --- !query 81 +-- !query SELECT replace('abcdef', 'de', '45') AS `abc45f` --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output abc45f --- !query 82 +-- !query SELECT replace('yabadabadoo', 'ba', '123') AS `ya123da123doo` --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output ya123da123doo --- !query 83 +-- !query SELECT replace('yabadoo', 'bad', '') AS `yaoo` --- !query 83 schema +-- !query schema struct --- !query 83 output +-- !query output yaoo --- !query 84 +-- !query select hex(256*256*256 - 1) AS `ffffff` --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output FFFFFF --- !query 85 +-- !query select hex(bigint(bigint(bigint(bigint(256)*256)*256)*256) - 1) AS `ffffffff` --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output FFFFFFFF --- !query 86 +-- !query select md5('') = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` --- !query 86 schema +-- !query schema struct --- !query 86 output +-- !query output true --- !query 87 +-- !query select md5('a') = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output true --- !query 88 +-- !query select md5('abc') = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output true --- !query 89 +-- !query select md5('message digest') = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output true --- !query 90 +-- !query select md5('abcdefghijklmnopqrstuvwxyz') = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output true --- !query 91 +-- !query select md5('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789') = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output true --- !query 92 +-- !query select md5('12345678901234567890123456789012345678901234567890123456789012345678901234567890') = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` --- !query 92 schema +-- !query schema struct --- !query 92 output +-- !query output true --- !query 93 +-- !query select md5(binary('')) = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` --- !query 93 schema +-- !query schema struct --- !query 93 output +-- !query output true --- !query 94 +-- !query select md5(binary('a')) = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` --- !query 94 schema +-- !query schema struct --- !query 94 output +-- !query output true --- !query 95 +-- !query select md5(binary('abc')) = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` --- !query 95 schema +-- !query schema struct --- !query 95 output +-- !query output true --- !query 96 +-- !query select md5(binary('message digest')) = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` --- !query 96 schema +-- !query schema struct --- !query 96 output +-- !query output true --- !query 97 +-- !query select md5(binary('abcdefghijklmnopqrstuvwxyz')) = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` --- !query 97 schema +-- !query schema struct --- !query 97 output +-- !query output true --- !query 98 +-- !query select md5(binary('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789')) = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` --- !query 98 schema +-- !query schema struct --- !query 98 output +-- !query output true --- !query 99 +-- !query select md5(binary('12345678901234567890123456789012345678901234567890123456789012345678901234567890')) = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` --- !query 99 schema +-- !query schema struct --- !query 99 output +-- !query output true --- !query 100 +-- !query SELECT initcap('hi THOMAS') --- !query 100 schema +-- !query schema struct --- !query 100 output +-- !query output Hi Thomas --- !query 101 +-- !query SELECT lpad('hi', 5, 'xy') --- !query 101 schema +-- !query schema struct --- !query 101 output +-- !query output xyxhi --- !query 102 +-- !query SELECT lpad('hi', 5) --- !query 102 schema +-- !query schema struct --- !query 102 output +-- !query output hi --- !query 103 +-- !query SELECT lpad('hi', -5, 'xy') --- !query 103 schema +-- !query schema struct --- !query 103 output +-- !query output --- !query 104 +-- !query SELECT lpad('hello', 2) --- !query 104 schema +-- !query schema struct --- !query 104 output +-- !query output he --- !query 105 +-- !query SELECT lpad('hi', 5, '') --- !query 105 schema +-- !query schema struct --- !query 105 output +-- !query output hi --- !query 106 +-- !query SELECT rpad('hi', 5, 'xy') --- !query 106 schema +-- !query schema struct --- !query 106 output +-- !query output hixyx --- !query 107 +-- !query SELECT rpad('hi', 5) --- !query 107 schema +-- !query schema struct --- !query 107 output +-- !query output hi --- !query 108 +-- !query SELECT rpad('hi', -5, 'xy') --- !query 108 schema +-- !query schema struct --- !query 108 output +-- !query output --- !query 109 +-- !query SELECT rpad('hello', 2) --- !query 109 schema +-- !query schema struct --- !query 109 output +-- !query output he --- !query 110 +-- !query SELECT rpad('hi', 5, '') --- !query 110 schema +-- !query schema struct --- !query 110 output +-- !query output hi --- !query 111 +-- !query SELECT ltrim('zzzytrim', 'xyz') --- !query 111 schema +-- !query schema struct --- !query 111 output +-- !query output trim --- !query 112 +-- !query SELECT translate('', '14', 'ax') --- !query 112 schema +-- !query schema struct --- !query 112 output +-- !query output --- !query 113 +-- !query SELECT translate('12345', '14', 'ax') --- !query 113 schema +-- !query schema struct --- !query 113 output +-- !query output a23x5 --- !query 114 +-- !query SELECT ascii('x') --- !query 114 schema +-- !query schema struct --- !query 114 output +-- !query output 120 --- !query 115 +-- !query SELECT ascii('') --- !query 115 schema +-- !query schema struct --- !query 115 output +-- !query output 0 --- !query 116 +-- !query SELECT chr(65) --- !query 116 schema +-- !query schema struct --- !query 116 output +-- !query output A --- !query 117 +-- !query SELECT chr(0) --- !query 117 schema +-- !query schema struct --- !query 117 output +-- !query output --- !query 118 +-- !query SELECT repeat('Pg', 4) --- !query 118 schema +-- !query schema struct --- !query 118 output +-- !query output PgPgPgPg --- !query 119 +-- !query SELECT repeat('Pg', -4) --- !query 119 schema +-- !query schema struct --- !query 119 output +-- !query output --- !query 120 +-- !query SELECT trim(binary('\\000') from binary('\\000Tom\\000')) --- !query 120 schema +-- !query schema struct --- !query 120 output +-- !query output Tom --- !query 121 +-- !query DROP TABLE toasttest --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index 6faa5b6924460..ccca1ba8cd8b4 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -2,161 +2,161 @@ -- Number of queries: 42 --- !query 0 +-- !query SELECT string('this is a text string') = string('this is a text string') AS true --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output true --- !query 1 +-- !query SELECT string('this is a text string') = string('this is a text strin') AS `false` --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output false --- !query 2 +-- !query CREATE TABLE TEXT_TBL (f1 string) USING parquet --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO TEXT_TBL VALUES ('doh!') --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO TEXT_TBL VALUES ('hi de ho neighbor') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT '' AS two, * FROM TEXT_TBL --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output doh! hi de ho neighbor --- !query 6 +-- !query select length(42) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 2 --- !query 7 +-- !query select string('four: ') || 2+2 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: four: 2 --- !query 8 +-- !query select 'four: ' || 2+2 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: four: 2 --- !query 9 +-- !query select 3 || 4.0 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 34.0 --- !query 10 +-- !query /* * various string functions */ select concat('one') --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output one --- !query 11 +-- !query select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 123hellotruefalse2010-03-09 --- !query 12 +-- !query select concat_ws('#','one') --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output one --- !query 13 +-- !query select concat_ws('#',1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1#x#x#hello#true#false#x-03-09 --- !query 14 +-- !query select concat_ws(',',10,20,null,30) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 10,20,30 --- !query 15 +-- !query select concat_ws('',10,20,null,30) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 102030 --- !query 16 +-- !query select concat_ws(NULL,10,20,null,30) is null --- !query 16 schema +-- !query schema struct<(concat_ws(CAST(NULL AS STRING), CAST(10 AS STRING), CAST(20 AS STRING), NULL, CAST(30 AS STRING)) IS NULL):boolean> --- !query 16 output +-- !query output true --- !query 17 +-- !query select reverse('abcde') --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output edcba --- !query 18 +-- !query select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output -5 -4 -3 @@ -170,192 +170,192 @@ struct 5 ahoj ahoj --- !query 19 +-- !query /* * format */ select format_string(NULL) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output NULL --- !query 20 +-- !query select format_string('Hello') --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output Hello --- !query 21 +-- !query select format_string('Hello %s', 'World') --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output Hello World --- !query 22 +-- !query select format_string('Hello %%') --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output Hello % --- !query 23 +-- !query select format_string('Hello %%%%') --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output Hello %% --- !query 24 +-- !query select format_string('Hello %s %s', 'World') --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 25 +-- !query select format_string('Hello %s') --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 26 +-- !query select format_string('Hello %x', 20) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output Hello 14 --- !query 27 +-- !query select format_string('%1$s %3$s', 1, 2, 3) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 1 3 --- !query 28 +-- !query select format_string('%1$s %12$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 1 12 --- !query 29 +-- !query select format_string('%1$s %4$s', 1, 2, 3) --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output java.util.MissingFormatArgumentException Format specifier '%4$s' --- !query 30 +-- !query select format_string('%1$s %13$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output java.util.MissingFormatArgumentException Format specifier '%13$s' --- !query 31 +-- !query select format_string('%0$s', 'Hello') --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output Hello --- !query 32 +-- !query select format_string('Hello %s %1$s %s', 'World', 'Hello again') --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output Hello World World Hello again --- !query 33 +-- !query select format_string('Hello %s %s, %2$s %2$s', 'World', 'Hello again') --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output Hello World Hello again, Hello again Hello again --- !query 34 +-- !query select format_string('>>%10s<<', 'Hello') --- !query 34 schema +-- !query schema struct>%10s<<, Hello):string> --- !query 34 output +-- !query output >> Hello<< --- !query 35 +-- !query select format_string('>>%10s<<', NULL) --- !query 35 schema +-- !query schema struct>%10s<<, NULL):string> --- !query 35 output +-- !query output >> null<< --- !query 36 +-- !query select format_string('>>%10s<<', '') --- !query 36 schema +-- !query schema struct>%10s<<, ):string> --- !query 36 output +-- !query output >> << --- !query 37 +-- !query select format_string('>>%-10s<<', '') --- !query 37 schema +-- !query schema struct>%-10s<<, ):string> --- !query 37 output +-- !query output >> << --- !query 38 +-- !query select format_string('>>%-10s<<', 'Hello') --- !query 38 schema +-- !query schema struct>%-10s<<, Hello):string> --- !query 38 output +-- !query output >>Hello << --- !query 39 +-- !query select format_string('>>%-10s<<', NULL) --- !query 39 schema +-- !query schema struct>%-10s<<, NULL):string> --- !query 39 output +-- !query output >>null << --- !query 40 +-- !query select format_string('>>%1$10s<<', 'Hello') --- !query 40 schema +-- !query schema struct>%1$10s<<, Hello):string> --- !query 40 output +-- !query output >> Hello<< --- !query 41 +-- !query DROP TABLE TEXT_TBL --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out index 460b6d1fdd02f..75ea3f3c42932 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/timestamp.sql.out @@ -2,147 +2,147 @@ -- Number of queries: 32 --- !query 0 +-- !query CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('now')) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('today')) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('yesterday')) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow')) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow EST')) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('tomorrow Zulu')) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today' --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 --- !query 9 +-- !query SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow' --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 3 --- !query 10 +-- !query SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday' --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 --- !query 11 +-- !query TRUNCATE TABLE TIMESTAMP_TBL --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('epoch')) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02')) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05')) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08')) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20')) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output --- !query 17 +-- !query SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL --- !query 17 schema +-- !query schema struct<64:string,d1:timestamp> --- !query 17 output +-- !query output 1969-12-31 16:00:00 1997-01-02 00:00:00 1997-01-02 03:04:05 @@ -150,75 +150,75 @@ struct<64:string,d1:timestamp> 2001-09-22 18:19:20 --- !query 18 +-- !query SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL WHERE d1 > timestamp '1997-01-02' --- !query 18 schema +-- !query schema struct<48:string,d1:timestamp> --- !query 18 output +-- !query output 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 19 +-- !query SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL WHERE d1 < timestamp '1997-01-02' --- !query 19 schema +-- !query schema struct<15:string,d1:timestamp> --- !query 19 output +-- !query output 1969-12-31 16:00:00 --- !query 20 +-- !query SELECT '' AS one, d1 FROM TIMESTAMP_TBL WHERE d1 = timestamp '1997-01-02' --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1997-01-02 00:00:00 --- !query 21 +-- !query SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL WHERE d1 != timestamp '1997-01-02' --- !query 21 schema +-- !query schema struct<63:string,d1:timestamp> --- !query 21 output +-- !query output 1969-12-31 16:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 22 +-- !query SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL WHERE d1 <= timestamp '1997-01-02' --- !query 22 schema +-- !query schema struct<16:string,d1:timestamp> --- !query 22 output +-- !query output 1969-12-31 16:00:00 1997-01-02 00:00:00 --- !query 23 +-- !query SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL WHERE d1 >= timestamp '1997-01-02' --- !query 23 schema +-- !query schema struct<49:string,d1:timestamp> --- !query 23 output +-- !query output 1997-01-02 00:00:00 1997-01-02 03:04:05 1997-02-10 17:32:01 2001-09-22 18:19:20 --- !query 24 +-- !query SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' --- !query 24 schema +-- !query schema struct<54:string,diff:interval> --- !query 24 output +-- !query output -236720 hours 0 seconds 3 hours 4 minutes 5 seconds @@ -226,22 +226,22 @@ struct<54:string,diff:interval> 953 hours 32 minutes 1 seconds --- !query 25 +-- !query SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 2004-02-23 00:00:00 --- !query 26 +-- !query SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff FROM TIMESTAMP_TBL WHERE d1 BETWEEN timestamp '1902-01-01' AND timestamp '2038-01-01' --- !query 26 schema +-- !query schema struct<54:string,diff:interval> --- !query 26 output +-- !query output -236720 hours 0 seconds 3 hours 4 minutes 5 seconds @@ -249,15 +249,15 @@ struct<54:string,diff:interval> 953 hours 32 minutes 1 seconds --- !query 27 +-- !query SELECT '' AS `54`, d1 as `timestamp`, date_part( 'year', d1) AS `year`, date_part( 'month', d1) AS `month`, date_part( 'day', d1) AS `day`, date_part( 'hour', d1) AS `hour`, date_part( 'minute', d1) AS `minute`, date_part( 'second', d1) AS `second` FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' --- !query 27 schema +-- !query schema struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:decimal(8,6)> --- !query 27 output +-- !query output 1969-12-31 16:00:00 1969 12 31 16 0 0.000000 1997-01-02 00:00:00 1997 1 2 0 0 0.000000 1997-01-02 03:04:05 1997 1 2 3 4 5.000000 @@ -265,14 +265,14 @@ struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute: 2001-09-22 18:19:20 2001 9 22 18 19 20.000000 --- !query 28 +-- !query SELECT '' AS `54`, d1 as `timestamp`, date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, date_part( 'usec', d1) AS usec FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' --- !query 28 schema +-- !query schema struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> --- !query 28 output +-- !query output 1969-12-31 16:00:00 4 0.000 0 1997-01-02 00:00:00 1 0.000 0 1997-01-02 03:04:05 1 5000.000 5000000 @@ -280,14 +280,14 @@ struct<54:string,timestamp:timestamp,quarter:int,msec:decimal(8,3),usec:int> 2001-09-22 18:19:20 3 20000.000 20000000 --- !query 29 +-- !query SELECT '' AS `54`, d1 as `timestamp`, date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, date_part( 'dow', d1) AS dow FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' --- !query 29 schema +-- !query schema struct<54:string,timestamp:timestamp,isoyear:int,week:int,dow:int> --- !query 29 output +-- !query output 1969-12-31 16:00:00 1970 1 3 1997-01-02 00:00:00 1997 1 4 1997-01-02 03:04:05 1997 1 4 @@ -295,17 +295,17 @@ struct<54:string,timestamp:timestamp,isoyear:int,week:int,dow:int> 2001-09-22 18:19:20 2001 38 6 --- !query 30 +-- !query SELECT make_timestamp(2014,12,28,6,30,45.887) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 2014-12-28 06:30:45.887 --- !query 31 +-- !query DROP TABLE TIMESTAMP_TBL --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out index 703463e6d6b5a..2fe53055cf656 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out @@ -2,17 +2,17 @@ -- Number of queries: 72 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) AS v(f1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (VALUES (123, 456), @@ -21,186 +21,186 @@ CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (4567890123456789, 4567890123456789), (4567890123456789, -4567890123456789)) AS v(q1, q2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM (VALUES (0.0), (-34.84), (-1004.30), (CAST('-1.2345678901234e+200' AS DOUBLE)), (CAST('-1.2345678901234e-200' AS DOUBLE))) AS v(f1) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT 1 AS two UNION SELECT 2 ORDER BY 1 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 --- !query 4 +-- !query SELECT 1 AS one UNION SELECT 1 ORDER BY 1 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 --- !query 5 +-- !query SELECT 1 AS two UNION ALL SELECT 2 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 2 --- !query 6 +-- !query SELECT 1 AS two UNION ALL SELECT 1 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 1 --- !query 7 +-- !query SELECT 1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 2 3 --- !query 8 +-- !query SELECT 1 AS two UNION SELECT 2 UNION SELECT 2 ORDER BY 1 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 2 --- !query 9 +-- !query SELECT 1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 2 2 --- !query 10 +-- !query SELECT 1.1 AS two UNION SELECT 2.2 ORDER BY 1 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1.1 2.2 --- !query 11 +-- !query SELECT 1.1 AS two UNION SELECT 2 ORDER BY 1 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1.1 2.0 --- !query 12 +-- !query SELECT 1 AS two UNION SELECT 2.2 ORDER BY 1 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1.0 2.2 --- !query 13 +-- !query SELECT 1 AS one UNION SELECT double(1.0) ORDER BY 1 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1.0 --- !query 14 +-- !query SELECT 1.1 AS two UNION ALL SELECT 2 ORDER BY 1 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1.1 2.0 --- !query 15 +-- !query SELECT double(1.0) AS two UNION ALL SELECT 1 ORDER BY 1 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1.0 1.0 --- !query 16 +-- !query SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1.1 2.0 3.0 --- !query 17 +-- !query SELECT double(1.1) AS two UNION SELECT 2 UNION SELECT double(2.0) ORDER BY 1 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1.1 2.0 --- !query 18 +-- !query SELECT 1.1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1.1 2.0 2.0 --- !query 19 +-- !query SELECT 1.1 AS two UNION (SELECT 2 UNION ALL SELECT 2) ORDER BY 1 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1.1 2.0 --- !query 20 +-- !query SELECT f1 AS five FROM FLOAT8_TBL UNION SELECT f1 FROM FLOAT8_TBL ORDER BY 1 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output -1.2345678901234E200 -1004.3 -34.84 @@ -208,13 +208,13 @@ struct 0.0 --- !query 21 +-- !query SELECT f1 AS ten FROM FLOAT8_TBL UNION ALL SELECT f1 FROM FLOAT8_TBL --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output -1.2345678901234E-200 -1.2345678901234E-200 -1.2345678901234E200 @@ -227,14 +227,14 @@ struct 0.0 --- !query 22 +-- !query SELECT f1 AS nine FROM FLOAT8_TBL UNION SELECT f1 FROM INT4_TBL ORDER BY 1 --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output -1.2345678901234E200 -2.147483647E9 -123456.0 @@ -246,13 +246,13 @@ struct 2.147483647E9 --- !query 23 +-- !query SELECT f1 AS ten FROM FLOAT8_TBL UNION ALL SELECT f1 FROM INT4_TBL --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output -1.2345678901234E-200 -1.2345678901234E200 -1004.3 @@ -265,16 +265,16 @@ struct 2.147483647E9 --- !query 24 +-- !query SELECT f1 AS five FROM FLOAT8_TBL WHERE f1 BETWEEN -1e6 AND 1e6 UNION SELECT f1 FROM INT4_TBL WHERE f1 BETWEEN 0 AND 1000000 ORDER BY 1 --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output -1004.3 -34.84 -1.2345678901234E-200 @@ -282,170 +282,170 @@ struct 123456.0 --- !query 25 +-- !query SELECT q2 FROM int8_tbl INTERSECT SELECT q1 FROM int8_tbl ORDER BY 1 --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 123 4567890123456789 --- !query 26 +-- !query SELECT q2 FROM int8_tbl INTERSECT ALL SELECT q1 FROM int8_tbl ORDER BY 1 --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 123 4567890123456789 4567890123456789 --- !query 27 +-- !query SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output -4567890123456789 456 --- !query 28 +-- !query SELECT q2 FROM int8_tbl EXCEPT ALL SELECT q1 FROM int8_tbl ORDER BY 1 --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output -4567890123456789 456 --- !query 29 +-- !query SELECT q2 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q1 FROM int8_tbl ORDER BY 1 --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output -4567890123456789 456 4567890123456789 --- !query 30 +-- !query SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY 1 --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output --- !query 31 +-- !query SELECT q1 FROM int8_tbl EXCEPT ALL SELECT q2 FROM int8_tbl ORDER BY 1 --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 123 4567890123456789 --- !query 32 +-- !query SELECT q1 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q2 FROM int8_tbl ORDER BY 1 --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 123 4567890123456789 4567890123456789 --- !query 33 +-- !query (SELECT 1,2,3 UNION SELECT 4,5,6) INTERSECT SELECT 4,5,6 --- !query 33 schema +-- !query schema struct<1:int,2:int,3:int> --- !query 33 output +-- !query output 4 5 6 --- !query 34 +-- !query (SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) INTERSECT SELECT 4,5,6 --- !query 34 schema +-- !query schema struct<1:int,2:int,3:int> --- !query 34 output +-- !query output 4 5 6 --- !query 35 +-- !query (SELECT 1,2,3 UNION SELECT 4,5,6) EXCEPT SELECT 4,5,6 --- !query 35 schema +-- !query schema struct<1:int,2:int,3:int> --- !query 35 output +-- !query output 1 2 3 --- !query 36 +-- !query (SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) EXCEPT SELECT 4,5,6 --- !query 36 schema +-- !query schema struct<1:int,2:int,3:int> --- !query 36 output +-- !query output 1 2 3 --- !query 37 +-- !query select count(*) from ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 5000 --- !query 38 +-- !query select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10 --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 10 --- !query 39 +-- !query select count(*) from ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 5000 --- !query 40 +-- !query select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10 --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 10 --- !query 41 +-- !query SELECT f1 FROM float8_tbl INTERSECT SELECT f1 FROM int4_tbl ORDER BY 1 --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 0.0 --- !query 42 +-- !query SELECT f1 FROM float8_tbl EXCEPT SELECT f1 FROM int4_tbl ORDER BY 1 --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output -1.2345678901234E200 -1004.3 -34.84 -1.2345678901234E-200 --- !query 43 +-- !query SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl ORDER BY 1 --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output -4567890123456789 123 123 @@ -455,20 +455,20 @@ struct 4567890123456789 --- !query 44 +-- !query SELECT q1 FROM int8_tbl INTERSECT (((SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl))) ORDER BY 1 --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 123 4567890123456789 --- !query 45 +-- !query (((SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl ORDER BY 1))) UNION ALL SELECT q2 FROM int8_tbl --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 123 4567890123456789 456 @@ -478,20 +478,20 @@ struct -4567890123456789 --- !query 46 +-- !query SELECT q1 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output -4567890123456789 456 --- !query 47 +-- !query SELECT q1 FROM int8_tbl UNION ALL (((SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1))) --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 123 123 4567890123456789 @@ -501,48 +501,48 @@ struct 456 --- !query 48 +-- !query (((SELECT q1 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl))) EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output -4567890123456789 456 --- !query 49 +-- !query SELECT q1,q2 FROM int8_tbl EXCEPT SELECT q2,q1 FROM int8_tbl ORDER BY q2,q1 --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output 4567890123456789 -4567890123456789 123 456 --- !query 50 +-- !query SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1 --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`q2`' given input columns: [int8_tbl.q1]; line 1 pos 64 --- !query 51 +-- !query SELECT q1 FROM int8_tbl EXCEPT (((SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1))) ORDER BY 1 --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output 123 4567890123456789 --- !query 52 +-- !query (((((select * from int8_tbl))))) --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output 123 456 123 4567890123456789 4567890123456789 -4567890123456789 @@ -550,22 +550,22 @@ struct 4567890123456789 4567890123456789 --- !query 53 +-- !query select * from range(1,5) union select * from range(1,3) --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 1 2 3 4 --- !query 54 +-- !query select * from range(1,6) union all select * from range(1,4) --- !query 54 schema +-- !query schema struct --- !query 54 output +-- !query output 1 1 2 @@ -576,49 +576,49 @@ struct 5 --- !query 55 +-- !query select * from range(1,6) intersect select * from range(1,4) --- !query 55 schema +-- !query schema struct --- !query 55 output +-- !query output 1 2 3 --- !query 56 +-- !query select * from range(1,6) intersect all select * from range(1,4) --- !query 56 schema +-- !query schema struct --- !query 56 output +-- !query output 1 2 3 --- !query 57 +-- !query select * from range(1,6) except select * from range(1,4) --- !query 57 schema +-- !query schema struct --- !query 57 output +-- !query output 4 5 --- !query 58 +-- !query select * from range(1,6) except all select * from range(1,4) --- !query 58 schema +-- !query schema struct --- !query 58 output +-- !query output 4 5 --- !query 59 +-- !query select * from range(1,6) union select * from range(1,4) --- !query 59 schema +-- !query schema struct --- !query 59 output +-- !query output 1 2 3 @@ -626,11 +626,11 @@ struct 5 --- !query 60 +-- !query select * from range(1,6) union all select * from range(1,4) --- !query 60 schema +-- !query schema struct --- !query 60 output +-- !query output 1 1 2 @@ -641,128 +641,128 @@ struct 5 --- !query 61 +-- !query select * from range(1,6) intersect select * from range(1,4) --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output 1 2 3 --- !query 62 +-- !query select * from range(1,6) intersect all select * from range(1,4) --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output 1 2 3 --- !query 63 +-- !query select * from range(1,6) except select * from range(1,4) --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output 4 5 --- !query 64 +-- !query select * from range(1,6) except all select * from range(1,4) --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 4 5 --- !query 65 +-- !query SELECT cast('3.4' as decimal(38, 18)) UNION SELECT 'foo' --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output 3.400000000000000000 foo --- !query 66 +-- !query SELECT * FROM (SELECT 1 AS t, 2 AS x UNION SELECT 2 AS t, 4 AS x) ss WHERE x < 4 ORDER BY x --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output 1 2 --- !query 67 +-- !query SELECT * FROM (SELECT 1 AS t, id as x from range(1,11) UNION SELECT 2 AS t, 4 AS x) ss WHERE x < 4 ORDER BY x --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output 1 1 1 2 1 3 --- !query 68 +-- !query SELECT * FROM (SELECT 1 AS t, int((random()*3)) AS x UNION SELECT 2 AS t, 4 AS x) ss WHERE x > 3 ORDER BY x --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 2 4 --- !query 69 +-- !query select distinct q1 from (select distinct * from int8_tbl i81 union all select distinct * from int8_tbl i82) ss where q2 = q2 --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output 123 4567890123456789 --- !query 70 +-- !query select distinct q1 from (select distinct * from int8_tbl i81 union all select distinct * from int8_tbl i82) ss where -q1 = q2 --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output 4567890123456789 --- !query 71 +-- !query select * from (select *, 0 as x from int8_tbl a union all select *, 1 as x from int8_tbl b) ss where (x = 0) or (q1 >= q2 and q1 <= q2) --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output 123 456 0 123 4567890123456789 0 4567890123456789 -4567890123456789 0 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out index 61b86b556e1a8..2b1de87a6be5e 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out @@ -2,20 +2,20 @@ -- Number of queries: 43 --- !query 0 +-- !query CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 GROUP BY four, ten ORDER BY four, ten --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 0 0 0 0.0 0 2 0 2.0 0 4 0 4.0 @@ -38,11 +38,11 @@ struct --- !query 2 output +-- !query output 10 10 10 @@ -55,11 +55,11 @@ struct --- !query 3 output +-- !query output 10 10 10 @@ -72,19 +72,19 @@ struct --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 0 0 0 0 0 0 0 4 0 @@ -97,11 +97,11 @@ struct 5 1 1 --- !query 6 +-- !query SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 10 2 @@ -114,11 +114,11 @@ struct --- !query 7 output +-- !query output 1 0 0 1 0 0 1 0 2 @@ -131,11 +131,11 @@ struct 4 9 1 --- !query 8 +-- !query SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 0 0 1 0 0 1 0 2 @@ -148,11 +148,11 @@ struct --- !query 9 output +-- !query output 0.0 0 0 0.0 0 0 0.0 0 2 @@ -165,11 +165,11 @@ struct --- !query 10 output +-- !query output 0.5 1 1 0.5 1 1 0.5 1 3 @@ -182,11 +182,11 @@ struct --- !query 11 output +-- !query output 1 0 0 1 0 0 1 0 2 @@ -199,11 +199,11 @@ struct --- !query 12 output +-- !query output 0 0 0 0 4 0 1 1 1 @@ -216,11 +216,11 @@ NULL 1 1 NULL 1 3 --- !query 13 +-- !query SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 0 0 0 1 1 1 3 1 3 @@ -233,11 +233,11 @@ NULL 4 0 NULL 9 1 --- !query 14 +-- !query SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 0 0 0 14 1 1 18 7 1 @@ -250,11 +250,11 @@ NULL 4 0 NULL 9 1 --- !query 15 +-- !query SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output -1 0 2 -1 3 3 -1 4 0 @@ -267,11 +267,11 @@ struct --- !query 16 output +-- !query output 0 0 0 0 0 0 0 0 2 @@ -284,11 +284,11 @@ struct --- !query 17 output +-- !query output 0 4 0 1 1 1 1 1 1 @@ -301,13 +301,13 @@ struct --- !query 18 output +-- !query output 4 0 0 4 0 0 4 4 0 @@ -320,12 +320,12 @@ struct --- !query 19 output +-- !query output 0 0 45000 45000 1 1 46000 46000 2 0 47000 92000 @@ -338,11 +338,11 @@ struct 9 1 54000 250000 --- !query 20 +-- !query SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 2 3 2 3 4 1 @@ -351,13 +351,13 @@ struct --- !query 21 output +-- !query output 136 22 22 @@ -370,7 +370,7 @@ struct 92 --- !query 22 +-- !query SELECT * FROM( SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, @@ -378,17 +378,17 @@ SELECT * FROM( sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum FROM tenk1 )sub WHERE total <> fourcount + twosum --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output --- !query 23 +-- !query SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 0.0 0.0 0.0 @@ -401,12 +401,12 @@ struct --- !query 24 output +-- !query output 0 0 45000 45000 1 1 46000 46000 2 0 47000 92000 @@ -419,15 +419,15 @@ struct 9 1 54000 250000 --- !query 25 +-- !query SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output --- !query 26 +-- !query create temporary view int4_tbl as select * from values (0), (123456), @@ -435,29 +435,29 @@ create temporary view int4_tbl as select * from values (2147483647), (-2147483647) as int4_tbl(f1) --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42 --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 0 --- !query 28 +-- !query select ten, sum(unique1) + sum(unique2) as res, rank() over (order by sum(unique1) + sum(unique2)) as rank from tenk1 group by ten order by ten --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 0 9976146 4 1 10114187 9 2 10059554 8 @@ -470,14 +470,14 @@ struct 9 10040184 7 --- !query 29 +-- !query SELECT four, ten, sum(ten) over (partition by four order by ten), last(ten) over (partition by four order by ten) FROM (select distinct ten, four from tenk1) ss --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 0 0 0 0 0 2 2 2 0 4 6 4 @@ -500,14 +500,14 @@ struct --- !query 30 output +-- !query output 0 0 0 0 0 2 2 2 0 4 6 4 @@ -530,14 +530,14 @@ struct --- !query 31 output +-- !query output 0 0 20 8 0 2 20 8 0 4 20 8 @@ -560,13 +560,13 @@ struct --- !query 32 output +-- !query output 10 3 3 10 7 3 18 2 2 @@ -579,13 +579,13 @@ struct --- !query 33 output +-- !query output 0 0 0 10 3 3 15 5 1 @@ -598,13 +598,13 @@ struct --- !query 34 output +-- !query output 10 0 0 13 2 2 15 7 3 @@ -617,13 +617,13 @@ struct --- !query 35 output +-- !query output 10 0 0 13 3 3 15 8 0 @@ -636,13 +636,13 @@ struct --- !query 36 output +-- !query output 0 7 3 10 5 1 15 8 0 @@ -655,13 +655,13 @@ struct --- !query 37 output +-- !query output 13 1 1 22 6 2 30 9 1 @@ -674,21 +674,21 @@ struct --- !query 38 output +-- !query output --- !query 39 +-- !query SELECT * FROM v_window --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 1 3 10 19 2 6 @@ -701,25 +701,25 @@ struct 9 27 --- !query 40 +-- !query DROP VIEW v_window --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output --- !query 41 +-- !query DROP VIEW tenk2 --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output --- !query 42 +-- !query DROP VIEW int4_tbl --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out index 9183eb659237e..0015740a0638e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -2,26 +2,26 @@ -- Number of queries: 31 --- !query 0 +-- !query CREATE TABLE empsalary ( depname string, empno integer, salary int, enroll_date date ) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), unique1, four FROM tenk1 WHERE unique1 < 10 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 12 1 1 12 5 1 12 9 1 @@ -34,13 +34,13 @@ NULL 4 0 NULL 8 0 --- !query 2 +-- !query SELECT sum(unique1) over (order by four desc range between 2 preceding and 1 preceding), unique1, four FROM tenk1 WHERE unique1 < 10 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 10 2 2 10 6 2 18 1 1 @@ -53,13 +53,13 @@ NULL 3 3 NULL 7 3 --- !query 3 +-- !query SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following), unique1, four FROM tenk1 WHERE unique1 < 10 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 10 3 3 10 7 3 12 4 0 @@ -72,18 +72,18 @@ struct --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException Window Frame specifiedwindowframe(RangeFrame, -1000, 1000) must match the required frame specifiedwindowframe(RowFrame, -1, -1); --- !query 5 +-- !query select ss.id, ss.y, first(ss.y) over w, last(ss.y) over w @@ -93,9 +93,9 @@ from union all select null, 43) ss window w as (order by ss.id asc nulls first range between 2 preceding and 2 following) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 1 1 3 2 2 1 4 3 3 1 5 @@ -105,7 +105,7 @@ NULL 42 42 43 NULL 43 42 43 --- !query 6 +-- !query select ss.id, ss.y, first(ss.y) over w, last(ss.y) over w @@ -115,9 +115,9 @@ from union all select null, 43) ss window w as (order by ss.id asc nulls last range between 2 preceding and 2 following) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 1 1 3 2 2 1 4 3 3 1 5 @@ -127,7 +127,7 @@ NULL 42 42 43 NULL 43 42 43 --- !query 7 +-- !query select ss.id, ss.y, first(ss.y) over w, last(ss.y) over w @@ -137,9 +137,9 @@ from union all select null, 43) ss window w as (order by ss.id desc nulls first range between 2 preceding and 2 following) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 1 3 1 2 2 4 1 3 3 5 1 @@ -149,7 +149,7 @@ NULL 42 42 43 NULL 43 42 43 --- !query 8 +-- !query select ss.id, ss.y, first(ss.y) over w, last(ss.y) over w @@ -159,9 +159,9 @@ from union all select null, 43) ss window w as (order by ss.id desc nulls last range between 2 preceding and 2 following) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 1 3 1 2 2 4 1 3 3 5 1 @@ -171,80 +171,80 @@ NULL 42 42 43 NULL 43 42 43 --- !query 9 +-- !query select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) from range(32764, 32767) x --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 32764 32766 32765 32766 32766 32766 --- !query 10 +-- !query select x.id, last(x.id) over (order by x.id desc range between current row and 2147450885 following) from range(-32766, -32765) x --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output -32766 -32766 --- !query 11 +-- !query select x.id, last(x.id) over (order by x.id range between current row and 4 following) from range(2147483644, 2147483647) x --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 2147483644 2147483646 2147483645 2147483646 2147483646 2147483646 --- !query 12 +-- !query select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) from range(-2147483646, -2147483645) x --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output -2147483646 -2147483646 --- !query 13 +-- !query select x.id, last(x.id) over (order by x.id range between current row and 4 following) from range(9223372036854775804, 9223372036854775807) x --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output java.lang.ArithmeticException long overflow --- !query 14 +-- !query select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) from range(-9223372036854775806, -9223372036854775805) x --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output java.lang.ArithmeticException long overflow --- !query 15 +-- !query create table numerics ( id int, f_float4 float, f_float8 float, f_numeric int ) using parquet --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query insert into numerics values (1, -3, -3, -3), (2, -1, -1, -1), @@ -253,20 +253,20 @@ insert into numerics values (5, 1.12, 1.12, 1.12), (6, 2, 2, 2), (7, 100, 100, 100) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output --- !query 17 +-- !query select id, f_float4, first(id) over w, last(id) over w from numerics window w as (order by f_float4 range between 1 preceding and 1 following) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 -3.0 1 1 2 -1.0 2 3 3 0.0 2 3 @@ -276,14 +276,14 @@ struct --- !query 18 output +-- !query output 1 -3.0 1 1 2 -1.0 2 3 3 0.0 2 4 @@ -293,14 +293,14 @@ struct --- !query 19 output +-- !query output 1 -3.0 1 7 2 -1.0 1 7 3 0.0 1 7 @@ -310,14 +310,14 @@ struct --- !query 20 output +-- !query output 1 -3.0 1 7 2 -1.0 2 7 3 0.0 2 7 @@ -327,14 +327,14 @@ struct --- !query 21 output +-- !query output 1 -3.0 1 1 2 -1.0 2 3 3 0.0 2 3 @@ -344,14 +344,14 @@ struct --- !query 22 output +-- !query output 1 -3.0 1 1 2 -1.0 2 3 3 0.0 2 4 @@ -361,14 +361,14 @@ struct --- !query 23 output +-- !query output 1 -3.0 1 7 2 -1.0 1 7 3 0.0 1 7 @@ -378,14 +378,14 @@ struct --- !query 24 output +-- !query output 1 -3.0 1 7 2 -1.0 2 7 3 0.0 2 7 @@ -395,14 +395,14 @@ struct --- !query 25 output +-- !query output 1 -3 1 1 2 -1 2 3 3 0 2 5 @@ -412,14 +412,14 @@ struct --- !query 26 output +-- !query output 1 -3 1 1 2 -1 2 3 3 0 2 5 @@ -429,14 +429,14 @@ struct --- !query 27 output +-- !query output 1 -3 1 1 2 -1 2 3 3 0 2 5 @@ -446,29 +446,29 @@ struct --- !query 28 output +-- !query output java.lang.NumberFormatException invalid input syntax for type numeric: NaN --- !query 29 +-- !query drop table empsalary --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output --- !query 30 +-- !query drop table numerics --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out index 80e4385ebd84b..5a52358fe1c53 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out @@ -2,28 +2,28 @@ -- Number of queries: 29 --- !query 0 +-- !query CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TABLE empsalary ( depname string, empno integer, salary int, enroll_date date ) USING parquet --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO empsalary VALUES ('develop', 10, 5200, date '2007-08-01'), ('sales', 1, 5000, date '2006-10-01'), @@ -35,13 +35,13 @@ INSERT INTO empsalary VALUES ('sales', 3, 4800, date '2007-08-01'), ('develop', 8, 6000, date '2006-10-01'), ('develop', 11, 5200, date '2007-08-15') --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query create table datetimes ( id int, f_time timestamp, @@ -50,13 +50,13 @@ create table datetimes ( f_timestamptz timestamp, f_timestamp timestamp ) using parquet --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query insert into datetimes values (1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'), (2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), @@ -68,22 +68,22 @@ insert into datetimes values (8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'), (9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'), (10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query WITH cte (x) AS ( SELECT * FROM range(1, 36, 2) ) SELECT x, (sum(x) over w) FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 4 11 33 13 39 @@ -104,16 +104,16 @@ struct --- !query 6 output +-- !query output 1 1 11 11 13 13 @@ -134,7 +134,7 @@ struct --- !query 7 output +-- !query output 1 2 1 3 1 7 @@ -173,7 +173,7 @@ struct --- !query 8 output +-- !query output 1 3 1 3 1 3 @@ -212,96 +212,96 @@ struct --- !query 9 output +-- !query output --- !query 10 +-- !query create table t1 (f1 int, f2 int) using parquet --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query insert into t1 values (1,1),(1,2),(2,2) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query select f1, sum(f1) over (partition by f1 range between 1 preceding and 1 following) from t1 where f1 = f2 --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY default.t1.`f1` RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 24 --- !query 13 +-- !query select f1, sum(f1) over (partition by f1 order by f2 range between 1 preceding and 1 following) from t1 where f1 = f2 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 1 2 2 --- !query 14 +-- !query select f1, sum(f1) over (partition by f1, f1 order by f2 range between 2 preceding and 1 preceding) from t1 where f1 = f2 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 NULL 2 NULL --- !query 15 +-- !query select f1, sum(f1) over (partition by f1, f2 order by f2 range between 1 following and 2 following) from t1 where f1 = f2 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 NULL 2 NULL --- !query 16 +-- !query SELECT rank() OVER (ORDER BY length('abc')) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1 --- !query 17 +-- !query SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10 --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException It is not allowed to use window functions inside WHERE and HAVING clauses; --- !query 18 +-- !query SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10 --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.AnalysisException The query operator `Join` contains one or more unsupported @@ -309,11 +309,11 @@ expression types Aggregate, Window or Generate. Invalid expressions: [row_number() OVER (ORDER BY default.empsalary.`salary` ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]; --- !query 19 +-- !query SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1 --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException The query operator `Aggregate` contains one or more unsupported @@ -321,11 +321,11 @@ expression types Aggregate, Window or Generate. Invalid expressions: [RANK() OVER (ORDER BY 1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]; --- !query 20 +-- !query SELECT * FROM rank() OVER (ORDER BY random()) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'ORDER'(line 1, pos 27) @@ -335,29 +335,29 @@ SELECT * FROM rank() OVER (ORDER BY random()) ---------------------------^^^ --- !query 21 +-- !query SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10 --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException It is not allowed to use window functions inside WHERE and HAVING clauses; --- !query 22 +-- !query SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random()) --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException It is not allowed to use window functions inside WHERE and HAVING clauses; --- !query 23 +-- !query select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1 --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'ORDER'(line 1, pos 39) @@ -367,43 +367,43 @@ select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1 ---------------------------------------^^^ --- !query 24 +-- !query SELECT range(1, 100) OVER () FROM empsalary --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException Undefined function: 'range'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 --- !query 25 +-- !query SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1 --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'ntile(0)' due to data type mismatch: Buckets expression must be positive, but got: 0; line 1 pos 7 --- !query 26 +-- !query DROP TABLE empsalary --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query DROP TABLE datetimes --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output --- !query 28 +-- !query DROP TABLE t1 --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out index 333e28f04a4d9..4dd4712345a89 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part4.sql.out @@ -2,143 +2,143 @@ -- Number of queries: 39 --- !query 0 +-- !query SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1 1.5 2 2.0 3 NULL 4 NULL --- !query 1 +-- !query SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 1.5 2 2.0 3 NULL 4 NULL --- !query 2 +-- !query SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 1.5 2 2.0 3 NULL 4 NULL --- !query 3 +-- !query SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1.5),(2,2.5),(3,NULL),(4,NULL)) t(i,v) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2.00000 2 2.50000 3 NULL 4 NULL --- !query 4 +-- !query SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 3 2 2 3 NULL 4 NULL --- !query 5 +-- !query SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 3 2 2 3 NULL 4 NULL --- !query 6 +-- !query SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 3 2 2 3 NULL 4 NULL --- !query 7 +-- !query SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1.1),(2,2.2),(3,NULL),(4,NULL)) t(i,v) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 3.3 2 2.2 3 NULL 4 NULL --- !query 8 +-- !query SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1.01),(2,2),(3,3)) v(i,n) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 3.00 5.00 6.01 --- !query 9 +-- !query SELECT i,COUNT(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 2 2 1 3 0 4 0 --- !query 10 +-- !query SELECT i,COUNT(*) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 4 2 3 3 2 4 1 --- !query 11 +-- !query SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 0.0 11266.666666666666 13868.750000000002 @@ -146,12 +146,12 @@ struct --- !query 12 output +-- !query output 0.0 11266.666666666666 13868.750000000002 @@ -159,12 +159,12 @@ struct --- !query 13 output +-- !query output 0.0 11266.666666666666 13868.750000000002 @@ -172,12 +172,12 @@ struct --- !query 14 output +-- !query output 0.0 11266.666666666666 13868.750000000002 @@ -185,12 +185,12 @@ struct --- !query 15 output +-- !query output 16900.0 18491.666666666668 27129.999999999996 @@ -198,12 +198,12 @@ struct --- !query 16 output +-- !query output 16900.0 18491.666666666668 27129.999999999996 @@ -211,12 +211,12 @@ struct --- !query 17 output +-- !query output 16900.0 18491.666666666668 27129.999999999996 @@ -224,12 +224,12 @@ struct --- !query 18 output +-- !query output 16900.0 18491.666666666668 27129.999999999996 @@ -237,12 +237,12 @@ struct --- !query 19 output +-- !query output 16900.0 18491.666666666668 27129.999999999996 @@ -250,12 +250,12 @@ struct --- !query 20 output +-- !query output 16900.0 18491.666666666668 27129.999999999996 @@ -263,12 +263,12 @@ struct --- !query 21 output +-- !query output 16900.0 18491.666666666668 27129.999999999996 @@ -276,12 +276,12 @@ struct --- !query 22 output +-- !query output 16900.0 18491.666666666668 27129.999999999996 @@ -289,12 +289,12 @@ struct --- !query 23 output +-- !query output 0.0 106.14455552060438 117.76565713313879 @@ -303,12 +303,12 @@ struct --- !query 24 output +-- !query output 0.0 106.14455552060438 117.76565713313879 @@ -317,12 +317,12 @@ struct --- !query 25 output +-- !query output 0.0 106.14455552060438 117.76565713313879 @@ -331,12 +331,12 @@ struct --- !query 26 output +-- !query output 0.0 106.14455552060438 117.76565713313879 @@ -345,12 +345,12 @@ struct --- !query 27 output +-- !query output 130.0 135.9840676942217 164.7118696390761 @@ -359,12 +359,12 @@ struct --- !query 28 output +-- !query output 130.0 135.9840676942217 164.7118696390761 @@ -373,12 +373,12 @@ struct --- !query 29 output +-- !query output 130.0 135.9840676942217 164.7118696390761 @@ -387,12 +387,12 @@ struct --- !query 30 output +-- !query output 130.0 135.9840676942217 164.7118696390761 @@ -401,12 +401,12 @@ struct --- !query 31 output +-- !query output 130.0 135.9840676942217 164.7118696390761 @@ -415,12 +415,12 @@ struct --- !query 32 output +-- !query output 130.0 135.9840676942217 164.7118696390761 @@ -429,12 +429,12 @@ struct --- !query 33 output +-- !query output 130.0 135.9840676942217 164.7118696390761 @@ -443,12 +443,12 @@ struct --- !query 34 output +-- !query output 130.0 135.9840676942217 164.7118696390761 @@ -457,48 +457,48 @@ struct --- !query 35 output +-- !query output 1 1 2 2 3 NULL 4 NULL --- !query 36 +-- !query SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 1 3 2 2 3 NULL 4 NULL --- !query 37 +-- !query SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) FROM (VALUES(1,1),(2,2),(3,3),(4,4)) t(i,v) --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 1 3 2 6 3 9 4 7 --- !query 38 +-- !query SELECT a, b, SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b) --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.AnalysisException failed to evaluate expression CAST('nan' AS INT): invalid input syntax for type numeric: nan; line 3 pos 6 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out index 91b0ff20b6ab0..badafc9e659e2 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/with.sql.out @@ -2,134 +2,134 @@ -- Number of queries: 51 --- !query 0 +-- !query WITH q1(x,y) AS (SELECT 1,2) SELECT * FROM q1, q1 AS q2 --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1 2 1 2 --- !query 1 +-- !query SELECT count(*) FROM ( WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) SELECT * FROM q1 UNION SELECT * FROM q1 ) ss --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 10 --- !query 2 +-- !query CREATE TABLE department ( id INTEGER, -- department ID parent_department INTEGER, -- upper department ID name string -- department name ) USING parquet --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO department VALUES (0, NULL, 'ROOT') --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO department VALUES (1, 0, 'A') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO department VALUES (2, 1, 'B') --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO department VALUES (3, 2, 'C') --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO department VALUES (4, 2, 'D') --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO department VALUES (5, 0, 'E') --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO department VALUES (6, 4, 'F') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO department VALUES (7, 5, 'G') --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query CREATE TABLE tree( id INTEGER, parent_id INTEGER ) USING parquet --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query INSERT INTO tree VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query create table graph( f int, t int, label string ) USING parquet --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query insert into graph values (1, 2, 'arc 1 -> 2'), (1, 3, 'arc 1 -> 3'), @@ -137,61 +137,61 @@ insert into graph values (1, 4, 'arc 1 -> 4'), (4, 5, 'arc 4 -> 5'), (5, 1, 'arc 5 -> 1') --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query CREATE TABLE y (a INTEGER) USING parquet --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output --- !query 17 +-- !query DROP TABLE y --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output --- !query 18 +-- !query CREATE TABLE y (a INTEGER) USING parquet --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 42 --- !query 21 +-- !query WITH outermost(x) AS ( SELECT 1 UNION (WITH innermost as (SELECT 2) @@ -199,15 +199,15 @@ WITH outermost(x) AS ( UNION SELECT 3) ) SELECT * FROM outermost ORDER BY 1 --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 2 3 --- !query 22 +-- !query WITH outermost(x) AS ( SELECT 1 UNION (WITH innermost as (SELECT 2) @@ -215,26 +215,26 @@ WITH outermost(x) AS ( UNION SELECT * FROM innermost) ) SELECT * FROM outermost ORDER BY 1 --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException Table or view not found: outermost; line 4 pos 23 --- !query 23 +-- !query CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i || ' v' AS string) v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query SELECT * FROM withz ORDER BY k --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1 1 v 4 4 v 7 7 v @@ -243,111 +243,111 @@ struct 16 16 v --- !query 25 +-- !query DROP TABLE withz --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query TRUNCATE TABLE y --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)) --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output --- !query 28 +-- !query CREATE TABLE yy (a INTEGER) USING parquet --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output --- !query 29 +-- !query SELECT * FROM y --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 1 2 3 --- !query 30 +-- !query SELECT * FROM yy --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output --- !query 31 +-- !query SELECT * FROM y --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 1 2 3 --- !query 32 +-- !query SELECT * FROM yy --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output --- !query 33 +-- !query CREATE TABLE parent ( id int, val string ) USING parquet --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output --- !query 34 +-- !query INSERT INTO parent VALUES ( 1, 'p1' ) --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output --- !query 35 +-- !query SELECT * FROM parent --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 1 p1 --- !query 36 +-- !query SELECT * FROM parent --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 1 p1 --- !query 37 +-- !query create table foo (with baz) --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'with'(line 1, pos 18) @@ -357,11 +357,11 @@ create table foo (with baz) ------------------^^^ --- !query 38 +-- !query create table foo (with ordinality) --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException no viable alternative at input 'with'(line 1, pos 18) @@ -371,98 +371,98 @@ create table foo (with ordinality) ------------------^^^ --- !query 39 +-- !query with ordinality as (select 1 as x) select * from ordinality --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 1 --- !query 40 +-- !query WITH test AS (SELECT 42) INSERT INTO test VALUES (1) --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.AnalysisException Table not found: test; --- !query 41 +-- !query create table test (i int) USING parquet --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output --- !query 42 +-- !query with test as (select 42) insert into test select * from test --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output --- !query 43 +-- !query select * from test --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 42 --- !query 44 +-- !query drop table test --- !query 44 schema +-- !query schema struct<> --- !query 44 output +-- !query output --- !query 45 +-- !query DROP TABLE department --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output --- !query 46 +-- !query DROP TABLE tree --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output --- !query 47 +-- !query DROP TABLE graph --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output --- !query 48 +-- !query DROP TABLE y --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output --- !query 49 +-- !query DROP TABLE yy --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output --- !query 50 +-- !query DROP TABLE parent --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/pred-pushdown.sql.out b/sql/core/src/test/resources/sql-tests/results/pred-pushdown.sql.out index 1b8ddbe4c7211..a64b8d3f6632d 100644 --- a/sql/core/src/test/resources/sql-tests/results/pred-pushdown.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pred-pushdown.sql.out @@ -2,39 +2,39 @@ -- Number of queries: 4 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW tbl_a AS VALUES (1, 1), (2, 1), (3, 6) AS T(c1, c2) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW tbl_b AS VALUES 1 AS T(c1) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM tbl_a LEFT ANTI JOIN tbl_b ON ((tbl_a.c1 = tbl_a.c2) IS NULL OR tbl_a.c1 = tbl_a.c2) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 2 1 3 6 --- !query 3 +-- !query SELECT l.c1, l.c2 FROM tbl_a l WHERE EXISTS (SELECT 1 FROM tbl_b r WHERE l.c1 = l.c2) OR l.c2 < 2 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 1 2 1 diff --git a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out index d38cab8fa7862..08cc6fa993e0b 100644 --- a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out @@ -2,297 +2,297 @@ -- Number of queries: 37 --- !query 0 +-- !query select 1 = 1 --- !query 0 schema +-- !query schema struct<(1 = 1):boolean> --- !query 0 output +-- !query output true --- !query 1 +-- !query select 1 = '1' --- !query 1 schema +-- !query schema struct<(1 = CAST(1 AS INT)):boolean> --- !query 1 output +-- !query output true --- !query 2 +-- !query select 1.0 = '1' --- !query 2 schema +-- !query schema struct<(CAST(1.0 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 2 output +-- !query output true --- !query 3 +-- !query select 1.5 = '1.51' --- !query 3 schema +-- !query schema struct<(CAST(1.5 AS DOUBLE) = CAST(1.51 AS DOUBLE)):boolean> --- !query 3 output +-- !query output false --- !query 4 +-- !query select 1 > '1' --- !query 4 schema +-- !query schema struct<(1 > CAST(1 AS INT)):boolean> --- !query 4 output +-- !query output false --- !query 5 +-- !query select 2 > '1.0' --- !query 5 schema +-- !query schema struct<(2 > CAST(1.0 AS INT)):boolean> --- !query 5 output +-- !query output true --- !query 6 +-- !query select 2 > '2.0' --- !query 6 schema +-- !query schema struct<(2 > CAST(2.0 AS INT)):boolean> --- !query 6 output +-- !query output false --- !query 7 +-- !query select 2 > '2.2' --- !query 7 schema +-- !query schema struct<(2 > CAST(2.2 AS INT)):boolean> --- !query 7 output +-- !query output false --- !query 8 +-- !query select '1.5' > 0.5 --- !query 8 schema +-- !query schema struct<(CAST(1.5 AS DOUBLE) > CAST(0.5 AS DOUBLE)):boolean> --- !query 8 output +-- !query output true --- !query 9 +-- !query select to_date('2009-07-30 04:17:52') > to_date('2009-07-30 04:17:52') --- !query 9 schema +-- !query schema struct<(to_date('2009-07-30 04:17:52') > to_date('2009-07-30 04:17:52')):boolean> --- !query 9 output +-- !query output false --- !query 10 +-- !query select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52' --- !query 10 schema +-- !query schema struct<(to_date('2009-07-30 04:17:52') > CAST(2009-07-30 04:17:52 AS DATE)):boolean> --- !query 10 output +-- !query output false --- !query 11 +-- !query select 1 >= '1' --- !query 11 schema +-- !query schema struct<(1 >= CAST(1 AS INT)):boolean> --- !query 11 output +-- !query output true --- !query 12 +-- !query select 2 >= '1.0' --- !query 12 schema +-- !query schema struct<(2 >= CAST(1.0 AS INT)):boolean> --- !query 12 output +-- !query output true --- !query 13 +-- !query select 2 >= '2.0' --- !query 13 schema +-- !query schema struct<(2 >= CAST(2.0 AS INT)):boolean> --- !query 13 output +-- !query output true --- !query 14 +-- !query select 2.0 >= '2.2' --- !query 14 schema +-- !query schema struct<(CAST(2.0 AS DOUBLE) >= CAST(2.2 AS DOUBLE)):boolean> --- !query 14 output +-- !query output false --- !query 15 +-- !query select '1.5' >= 0.5 --- !query 15 schema +-- !query schema struct<(CAST(1.5 AS DOUBLE) >= CAST(0.5 AS DOUBLE)):boolean> --- !query 15 output +-- !query output true --- !query 16 +-- !query select to_date('2009-07-30 04:17:52') >= to_date('2009-07-30 04:17:52') --- !query 16 schema +-- !query schema struct<(to_date('2009-07-30 04:17:52') >= to_date('2009-07-30 04:17:52')):boolean> --- !query 16 output +-- !query output true --- !query 17 +-- !query select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52' --- !query 17 schema +-- !query schema struct<(to_date('2009-07-30 04:17:52') >= CAST(2009-07-30 04:17:52 AS DATE)):boolean> --- !query 17 output +-- !query output true --- !query 18 +-- !query select 1 < '1' --- !query 18 schema +-- !query schema struct<(1 < CAST(1 AS INT)):boolean> --- !query 18 output +-- !query output false --- !query 19 +-- !query select 2 < '1.0' --- !query 19 schema +-- !query schema struct<(2 < CAST(1.0 AS INT)):boolean> --- !query 19 output +-- !query output false --- !query 20 +-- !query select 2 < '2.0' --- !query 20 schema +-- !query schema struct<(2 < CAST(2.0 AS INT)):boolean> --- !query 20 output +-- !query output false --- !query 21 +-- !query select 2.0 < '2.2' --- !query 21 schema +-- !query schema struct<(CAST(2.0 AS DOUBLE) < CAST(2.2 AS DOUBLE)):boolean> --- !query 21 output +-- !query output true --- !query 22 +-- !query select 0.5 < '1.5' --- !query 22 schema +-- !query schema struct<(CAST(0.5 AS DOUBLE) < CAST(1.5 AS DOUBLE)):boolean> --- !query 22 output +-- !query output true --- !query 23 +-- !query select to_date('2009-07-30 04:17:52') < to_date('2009-07-30 04:17:52') --- !query 23 schema +-- !query schema struct<(to_date('2009-07-30 04:17:52') < to_date('2009-07-30 04:17:52')):boolean> --- !query 23 output +-- !query output false --- !query 24 +-- !query select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52' --- !query 24 schema +-- !query schema struct<(to_date('2009-07-30 04:17:52') < CAST(2009-07-30 04:17:52 AS DATE)):boolean> --- !query 24 output +-- !query output false --- !query 25 +-- !query select 1 <= '1' --- !query 25 schema +-- !query schema struct<(1 <= CAST(1 AS INT)):boolean> --- !query 25 output +-- !query output true --- !query 26 +-- !query select 2 <= '1.0' --- !query 26 schema +-- !query schema struct<(2 <= CAST(1.0 AS INT)):boolean> --- !query 26 output +-- !query output false --- !query 27 +-- !query select 2 <= '2.0' --- !query 27 schema +-- !query schema struct<(2 <= CAST(2.0 AS INT)):boolean> --- !query 27 output +-- !query output true --- !query 28 +-- !query select 2.0 <= '2.2' --- !query 28 schema +-- !query schema struct<(CAST(2.0 AS DOUBLE) <= CAST(2.2 AS DOUBLE)):boolean> --- !query 28 output +-- !query output true --- !query 29 +-- !query select 0.5 <= '1.5' --- !query 29 schema +-- !query schema struct<(CAST(0.5 AS DOUBLE) <= CAST(1.5 AS DOUBLE)):boolean> --- !query 29 output +-- !query output true --- !query 30 +-- !query select to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52') --- !query 30 schema +-- !query schema struct<(to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52')):boolean> --- !query 30 output +-- !query output true --- !query 31 +-- !query select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52' --- !query 31 schema +-- !query schema struct<(to_date('2009-07-30 04:17:52') <= CAST(2009-07-30 04:17:52 AS DATE)):boolean> --- !query 31 output +-- !query output true --- !query 32 +-- !query select to_date('2017-03-01') = to_timestamp('2017-03-01 00:00:00') --- !query 32 schema +-- !query schema struct<(CAST(to_date('2017-03-01') AS TIMESTAMP) = to_timestamp('2017-03-01 00:00:00')):boolean> --- !query 32 output +-- !query output true --- !query 33 +-- !query select to_timestamp('2017-03-01 00:00:01') > to_date('2017-03-01') --- !query 33 schema +-- !query schema struct<(to_timestamp('2017-03-01 00:00:01') > CAST(to_date('2017-03-01') AS TIMESTAMP)):boolean> --- !query 33 output +-- !query output true --- !query 34 +-- !query select to_timestamp('2017-03-01 00:00:01') >= to_date('2017-03-01') --- !query 34 schema +-- !query schema struct<(to_timestamp('2017-03-01 00:00:01') >= CAST(to_date('2017-03-01') AS TIMESTAMP)):boolean> --- !query 34 output +-- !query output true --- !query 35 +-- !query select to_date('2017-03-01') < to_timestamp('2017-03-01 00:00:01') --- !query 35 schema +-- !query schema struct<(CAST(to_date('2017-03-01') AS TIMESTAMP) < to_timestamp('2017-03-01 00:00:01')):boolean> --- !query 35 output +-- !query output true --- !query 36 +-- !query select to_date('2017-03-01') <= to_timestamp('2017-03-01 00:00:01') --- !query 36 schema +-- !query schema struct<(CAST(to_date('2017-03-01') AS TIMESTAMP) <= to_timestamp('2017-03-01 00:00:01')):boolean> --- !query 36 output +-- !query output true diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out index 2dade86f35df9..2e93ee286fd47 100644 --- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -2,312 +2,312 @@ -- Number of queries: 34 --- !query 0 +-- !query set spark.sql.parser.quotedRegexColumnNames=false --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output spark.sql.parser.quotedRegexColumnNames false --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") AS testData(key, value1, value2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES (1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) AS testData2(A, B, c, d) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT `(a)?+.+` FROM testData2 WHERE a = 1 --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`(a)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 --- !query 4 +-- !query SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 't.`(a)?+.+`' given input columns: [t.A, t.B, t.c, t.d]; line 1 pos 7 --- !query 5 +-- !query SELECT `(a|b)` FROM testData2 WHERE a = 2 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`(a|b)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 --- !query 6 +-- !query SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 --- !query 7 +-- !query SELECT SUM(`(a|b)?+.+`) FROM testData2 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 --- !query 8 +-- !query SELECT SUM(`(a)`) FROM testData2 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`(a)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 --- !query 9 +-- !query set spark.sql.parser.quotedRegexColumnNames=true --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output spark.sql.parser.quotedRegexColumnNames true --- !query 10 +-- !query SELECT `(a)?+.+` FROM testData2 WHERE a = 1 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 1 2 2 1 2 --- !query 11 +-- !query SELECT `(A)?+.+` FROM testData2 WHERE a = 1 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 1 2 2 1 2 --- !query 12 +-- !query SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 1 2 2 1 2 --- !query 13 +-- !query SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 1 2 2 1 2 --- !query 14 +-- !query SELECT `(a|B)` FROM testData2 WHERE a = 2 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 2 1 2 2 --- !query 15 +-- !query SELECT `(A|b)` FROM testData2 WHERE a = 2 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 2 1 2 2 --- !query 16 +-- !query SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2 3 2 3 --- !query 17 +-- !query SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 2 3 2 3 --- !query 18 +-- !query SELECT `(e|f)` FROM testData2 --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query SELECT t.`(e|f)` FROM testData2 t --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1 11 1 1 1 2 1 11 2 1 1 2 2 22 1 2 2 3 2 22 2 2 2 3 --- !query 21 +-- !query SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 11 1 1 1 2 1 11 2 1 1 2 2 22 1 2 2 3 2 22 2 2 2 3 --- !query 22 +-- !query set spark.sql.caseSensitive=true --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output spark.sql.caseSensitive true --- !query 23 +-- !query CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES (0, 1), (1, 2), (2, 3), (3, 4) AS testdata3(a, b) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query SELECT `(A)?+.+` FROM testdata3 --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 0 1 1 2 2 3 3 4 --- !query 25 +-- !query SELECT `(a)?+.+` FROM testdata3 --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1 2 3 4 --- !query 26 +-- !query SELECT `(A)?+.+` FROM testdata3 WHERE a > 1 --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 2 3 3 4 --- !query 27 +-- !query SELECT `(a)?+.+` FROM testdata3 where `a` > 1 --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 3 4 --- !query 28 +-- !query SELECT SUM(`a`) FROM testdata3 --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 6 --- !query 29 +-- !query SELECT SUM(`(a)`) FROM testdata3 --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 6 --- !query 30 +-- !query SELECT SUM(`(a)?+.+`) FROM testdata3 --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 10 --- !query 31 +-- !query SELECT SUM(a) FROM testdata3 GROUP BY `a` --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 0 1 2 3 --- !query 32 +-- !query SELECT SUM(a) FROM testdata3 GROUP BY `(a)` --- !query 32 schema +-- !query schema struct<> --- !query 32 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`(a)`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38 --- !query 33 +-- !query SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`(a)?+.+`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38 diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out index acd0609aabb16..9d00a82b76780 100644 --- a/sql/core/src/test/resources/sql-tests/results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out @@ -2,83 +2,83 @@ -- Number of queries: 10 --- !query 0 +-- !query SELECT rand(0) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 0.7604953758285915 --- !query 1 +-- !query SELECT rand(cast(3 / 7 AS int)) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 0.7604953758285915 --- !query 2 +-- !query SELECT rand(NULL) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 0.7604953758285915 --- !query 3 +-- !query SELECT rand(cast(NULL AS int)) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 0.7604953758285915 --- !query 4 +-- !query SELECT rand(1.0) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'rand(1.0BD)' due to data type mismatch: argument 1 requires (int or bigint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7 --- !query 5 +-- !query SELECT randn(0L) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1.6034991609278433 --- !query 6 +-- !query SELECT randn(cast(3 / 7 AS long)) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1.6034991609278433 --- !query 7 +-- !query SELECT randn(NULL) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1.6034991609278433 --- !query 8 +-- !query SELECT randn(cast(NULL AS long)) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1.6034991609278433 --- !query 9 +-- !query SELECT rand('1') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'rand('1')' due to data type mismatch: argument 1 requires (int or bigint) type, however, ''1'' is of string type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out index bd12922c79fa4..5771f218e3b57 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out @@ -2,19 +2,19 @@ -- Number of queries: 38 --- !query 0 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SHOW CREATE TABLE tbl --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output CREATE TABLE `tbl` ( `a` INT, `b` STRING, @@ -22,28 +22,28 @@ CREATE TABLE `tbl` ( USING parquet --- !query 2 +-- !query DROP TABLE tbl --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet OPTIONS ('a' 1) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query SHOW CREATE TABLE tbl --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output CREATE TABLE `tbl` ( `a` INT, `b` STRING, @@ -53,28 +53,28 @@ OPTIONS ( `a` '1') --- !query 5 +-- !query DROP TABLE tbl --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet OPTIONS ('path' '/path/to/table') --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query SHOW CREATE TABLE tbl --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output CREATE TABLE `tbl` ( `a` INT, `b` STRING, @@ -83,28 +83,28 @@ USING parquet LOCATION 'file:/path/to/table' --- !query 8 +-- !query DROP TABLE tbl --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet LOCATION '/path/to/table' --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query SHOW CREATE TABLE tbl --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output CREATE TABLE `tbl` ( `a` INT, `b` STRING, @@ -113,28 +113,28 @@ USING parquet LOCATION 'file:/path/to/table' --- !query 11 +-- !query DROP TABLE tbl --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet PARTITIONED BY (a) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query SHOW CREATE TABLE tbl --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output CREATE TABLE `tbl` ( `b` STRING, `c` INT, @@ -143,28 +143,28 @@ USING parquet PARTITIONED BY (a) --- !query 14 +-- !query DROP TABLE tbl --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query SHOW CREATE TABLE tbl --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output CREATE TABLE `tbl` ( `a` INT, `b` STRING, @@ -175,28 +175,28 @@ SORTED BY (b) INTO 2 BUCKETS --- !query 17 +-- !query DROP TABLE tbl --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output --- !query 18 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet COMMENT 'This is a comment' --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query SHOW CREATE TABLE tbl --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output CREATE TABLE `tbl` ( `a` INT, `b` STRING, @@ -205,28 +205,28 @@ USING parquet COMMENT 'This is a comment' --- !query 20 +-- !query DROP TABLE tbl --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet TBLPROPERTIES ('a' = '1') --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query SHOW CREATE TABLE tbl --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output CREATE TABLE `tbl` ( `a` INT, `b` STRING, @@ -236,27 +236,27 @@ TBLPROPERTIES ( 'a' = '1') --- !query 23 +-- !query DROP TABLE tbl --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query CREATE TABLE tbl (a REAL, b NUMERIC, c NUMERIC(10), d NUMERIC(10,1)) USING parquet --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query SHOW CREATE TABLE tbl --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output CREATE TABLE `tbl` ( `a` FLOAT, `b` DECIMAL(10,0), @@ -265,65 +265,65 @@ CREATE TABLE `tbl` ( USING parquet --- !query 26 +-- !query DROP TABLE tbl --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output --- !query 28 +-- !query CREATE VIEW view_SPARK_30302 (aaa, bbb) AS SELECT a, b FROM tbl --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output --- !query 29 +-- !query SHOW CREATE TABLE view_SPARK_30302 --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output CREATE VIEW `view_SPARK_30302`( `aaa`, `bbb`) AS SELECT a, b FROM tbl --- !query 30 +-- !query DROP VIEW view_SPARK_30302 --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output --- !query 31 +-- !query CREATE VIEW view_SPARK_30302 (aaa COMMENT 'comment with \'quoted text\' for aaa', bbb) COMMENT 'This is a comment with \'quoted text\' for view' AS SELECT a, b FROM tbl --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output --- !query 32 +-- !query SHOW CREATE TABLE view_SPARK_30302 --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output CREATE VIEW `view_SPARK_30302`( `aaa` COMMENT 'comment with \'quoted text\' for aaa', `bbb`) @@ -331,29 +331,29 @@ COMMENT 'This is a comment with \'quoted text\' for view' AS SELECT a, b FROM tbl --- !query 33 +-- !query DROP VIEW view_SPARK_30302 --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output --- !query 34 +-- !query CREATE VIEW view_SPARK_30302 (aaa, bbb) TBLPROPERTIES ('a' = '1', 'b' = '2') AS SELECT a, b FROM tbl --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output --- !query 35 +-- !query SHOW CREATE TABLE view_SPARK_30302 --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output CREATE VIEW `view_SPARK_30302`( `aaa`, `bbb`) @@ -363,17 +363,17 @@ TBLPROPERTIES ( AS SELECT a, b FROM tbl --- !query 36 +-- !query DROP VIEW view_SPARK_30302 --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output --- !query 37 +-- !query DROP TABLE tbl --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index ee05e2413799a..501e185b07f7a 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -2,67 +2,67 @@ -- Number of queries: 26 --- !query 0 +-- !query CREATE DATABASE showdb --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query USE showdb --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TABLE show_t1(a String, b Int, c String, d String) USING parquet PARTITIONED BY (c, d) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE TABLE show_t2(b String, d Int) USING parquet --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TEMPORARY VIEW show_t3(e int) USING parquet --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query CREATE GLOBAL TEMP VIEW show_t4 AS SELECT 1 as col1 --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query SHOW TABLES --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output aggtest arraydata mapdata @@ -74,11 +74,11 @@ tenk1 testdata --- !query 8 +-- !query SHOW TABLES IN showdb --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output aggtest arraydata mapdata @@ -90,40 +90,40 @@ tenk1 testdata --- !query 9 +-- !query SHOW TABLES 'show_t*' --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output show_t1 show_t2 show_t3 --- !query 10 +-- !query SHOW TABLES LIKE 'show_t1*|show_t2*' --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output show_t1 show_t2 --- !query 11 +-- !query SHOW TABLES IN showdb 'show_t*' --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output show_t1 show_t2 show_t3 --- !query 12 +-- !query SHOW TABLE EXTENDED LIKE 'show_t*' --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output show_t3 true Table: show_t3 Created Time [not included in comparison] Last Access [not included in comparison] @@ -163,11 +163,11 @@ Schema: root |-- d: integer (nullable = true) --- !query 13 +-- !query SHOW TABLE EXTENDED --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input '' expecting {'FROM', 'IN', 'LIKE'}(line 1, pos 19) @@ -177,22 +177,22 @@ SHOW TABLE EXTENDED -------------------^^^ --- !query 14 +-- !query SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output showdb show_t1 false Partition Values: [c=Us, d=1] Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t1/c=Us/d=1 Created Time [not included in comparison] Last Access [not included in comparison] --- !query 15 +-- !query SHOW TABLE EXTENDED PARTITION(c='Us', d=1) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException mismatched input 'PARTITION' expecting {'FROM', 'IN', 'LIKE'}(line 1, pos 20) @@ -202,87 +202,87 @@ SHOW TABLE EXTENDED PARTITION(c='Us', d=1) --------------------^^^ --- !query 16 +-- !query SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.catalyst.analysis.NoSuchTableException Table or view 'show_t*' not found in database 'showdb'; --- !query 17 +-- !query SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us') --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`showdb`.`show_t1`'; --- !query 18 +-- !query SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(a='Us', d=1) --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.AnalysisException Partition spec is invalid. The spec (a, d) must match the partition spec (c, d) defined in table '`showdb`.`show_t1`'; --- !query 19 +-- !query SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException Partition not found in table 'show_t1' database 'showdb': c -> Ch d -> 1; --- !query 20 +-- !query DROP TABLE show_t1 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query DROP TABLE show_t2 --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query DROP VIEW show_t3 --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output --- !query 23 +-- !query DROP VIEW global_temp.show_t4 --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query USE default --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query DROP DATABASE showdb --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out index 71d6e120e8943..4f5db7f6c6b2f 100644 --- a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -2,216 +2,216 @@ -- Number of queries: 25 --- !query 0 +-- !query CREATE DATABASE showdb --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query USE showdb --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TABLE showcolumn1 (col1 int, `col 2` int) USING json --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE TABLE showcolumn2 (price int, qty int, year int, month int) USING parquet partitioned by (year, month) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING json --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5` --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SHOW COLUMNS IN showcolumn1 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output col 2 col1 --- !query 7 +-- !query SHOW COLUMNS IN showdb.showcolumn1 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output col 2 col1 --- !query 8 +-- !query SHOW COLUMNS IN showcolumn1 FROM showdb --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output col 2 col1 --- !query 9 +-- !query SHOW COLUMNS IN showcolumn2 IN showdb --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output month price qty year --- !query 10 +-- !query SHOW COLUMNS IN badtable FROM showdb --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.catalyst.analysis.NoSuchTableException Table or view 'badtable' not found in database 'showdb'; --- !query 11 +-- !query SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output col 2 col1 --- !query 12 +-- !query SHOW COLUMNS IN showdb.showcolumn1 FROM baddb --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException SHOW COLUMNS with conflicting databases: 'baddb' != 'showdb'; --- !query 13 +-- !query SHOW COLUMNS IN showcolumn3 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output col 4 col3 --- !query 14 +-- !query SHOW COLUMNS IN showdb.showcolumn3 --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.catalyst.analysis.NoSuchTableException Table or view 'showcolumn3' not found in database 'showdb'; --- !query 15 +-- !query SHOW COLUMNS IN showcolumn3 FROM showdb --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.catalyst.analysis.NoSuchTableException Table or view 'showcolumn3' not found in database 'showdb'; --- !query 16 +-- !query SHOW COLUMNS IN showcolumn4 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.catalyst.analysis.NoSuchTableException Table or view 'showcolumn4' not found in database 'showdb'; --- !query 17 +-- !query SHOW COLUMNS IN global_temp.showcolumn4 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output col 5 col1 --- !query 18 +-- !query SHOW COLUMNS IN showcolumn4 FROM global_temp --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output col 5 col1 --- !query 19 +-- !query DROP TABLE showcolumn1 --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query DROP TABLE showColumn2 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query DROP VIEW showcolumn3 --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query DROP VIEW global_temp.showcolumn4 --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output --- !query 23 +-- !query use default --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query DROP DATABASE showdb --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out index 69a8e958000db..6f1bbd03bc223 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -2,114 +2,114 @@ -- Number of queries: 14 --- !query 0 +-- !query SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output x y NULL --- !query 1 +-- !query SELECT nullif('x', 'x'), nullif('x', 'y') --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output NULL x --- !query 2 +-- !query SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output x y NULL --- !query 3 +-- !query SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output y x NULL --- !query 4 +-- !query SELECT ifnull(1, 2.1d), ifnull(null, 2.1d) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1.0 2.1 --- !query 5 +-- !query SELECT nullif(1, 2.1d), nullif(1, 1.0d) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 NULL --- !query 6 +-- !query SELECT nvl(1, 2.1d), nvl(null, 2.1d) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1.0 2.1 --- !query 7 +-- !query SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 2.1 1.0 --- !query 8 +-- !query SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output true 1 1 1 1 --- !query 9 +-- !query SELECT float(1), double(1), decimal(1) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1.0 1.0 1 --- !query 10 +-- !query SELECT date("2014-04-04"), timestamp(date("2014-04-04")) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 2014-04-04 2014-04-04 00:00:00 --- !query 11 +-- !query SELECT string(1, 2) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException Function string accepts only one argument; line 1 pos 7 --- !query 12 +-- !query CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value") --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output gamma 1 diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 87c3e04017643..33d1b25aee483 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -2,267 +2,267 @@ -- Number of queries: 33 --- !query 0 +-- !query select concat_ws() --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output org.apache.spark.sql.AnalysisException requirement failed: concat_ws requires at least one argument.; line 1 pos 7 --- !query 1 +-- !query select format_string() --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output org.apache.spark.sql.AnalysisException requirement failed: format_string() should take at least 1 argument; line 1 pos 7 --- !query 2 +-- !query select 'a' || 'b' || 'c' --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output abc --- !query 3 +-- !query select replace('abc', 'b', '123') --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output a123c --- !query 4 +-- !query select replace('abc', 'b') --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output ac --- !query 5 +-- !query select length(uuid()), (uuid() <> uuid()) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 36 true --- !query 6 +-- !query select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 4 NULL NULL --- !query 7 +-- !query select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output ab abcd ab NULL --- !query 8 +-- !query select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a') --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NULL NULL --- !query 9 +-- !query select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output cd abcd cd NULL --- !query 10 +-- !query select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a') --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output NULL NULL --- !query 11 +-- !query SELECT split('aa1cc2ee3', '[1-9]+') --- !query 11 schema +-- !query schema struct> --- !query 11 output +-- !query output ["aa","cc","ee",""] --- !query 12 +-- !query SELECT split('aa1cc2ee3', '[1-9]+', 2) --- !query 12 schema +-- !query schema struct> --- !query 12 output +-- !query output ["aa","cc2ee3"] --- !query 13 +-- !query SELECT substr('Spark SQL', 5) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output k SQL --- !query 14 +-- !query SELECT substr('Spark SQL', -3) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output SQL --- !query 15 +-- !query SELECT substr('Spark SQL', 5, 1) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output k --- !query 16 +-- !query SELECT substr('Spark SQL' from 5) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output k SQL --- !query 17 +-- !query SELECT substr('Spark SQL' from -3) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output SQL --- !query 18 +-- !query SELECT substr('Spark SQL' from 5 for 1) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output k --- !query 19 +-- !query SELECT substring('Spark SQL', 5) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output k SQL --- !query 20 +-- !query SELECT substring('Spark SQL', -3) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output SQL --- !query 21 +-- !query SELECT substring('Spark SQL', 5, 1) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output k --- !query 22 +-- !query SELECT substring('Spark SQL' from 5) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output k SQL --- !query 23 +-- !query SELECT substring('Spark SQL' from -3) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output SQL --- !query 24 +-- !query SELECT substring('Spark SQL' from 5 for 1) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output k --- !query 25 +-- !query SELECT trim('yxTomxx', 'xyz'), trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx') --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output Tom Tom Tom --- !query 26 +-- !query SELECT trim('xxxbarxxx', 'x'), trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx') --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output bar bar bar --- !query 27 +-- !query SELECT ltrim('zzzytest', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytest') --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output test test --- !query 28 +-- !query SELECT ltrim('zzzytestxyz', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytestxyz') --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output testxyz testxyz --- !query 29 +-- !query SELECT ltrim('xyxXxyLAST WORD', 'xy'), trim(LEADING 'xy' FROM 'xyxXxyLAST WORD') --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output XxyLAST WORD XxyLAST WORD --- !query 30 +-- !query SELECT rtrim('testxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'testxxzx') --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output test test --- !query 31 +-- !query SELECT rtrim('xyztestxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'xyztestxxzx') --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output xyztest xyztest --- !query 32 +-- !query SELECT rtrim('TURNERyxXxy', 'xy'), trim(TRAILING 'xy' FROM 'TURNERyxXxy') --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output TURNERyxX TURNERyxX diff --git a/sql/core/src/test/resources/sql-tests/results/struct.sql.out b/sql/core/src/test/resources/sql-tests/results/struct.sql.out index 1da33bc736f0b..f294c5213d319 100644 --- a/sql/core/src/test/resources/sql-tests/results/struct.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/struct.sql.out @@ -2,89 +2,89 @@ -- Number of queries: 9 --- !query 0 +-- !query CREATE TEMPORARY VIEW tbl_x AS VALUES (1, NAMED_STRUCT('C', 'gamma', 'D', 'delta')), (2, NAMED_STRUCT('C', 'epsilon', 'D', 'eta')), (3, NAMED_STRUCT('C', 'theta', 'D', 'iota')) AS T(ID, ST) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT STRUCT('alpha', 'beta') ST --- !query 1 schema +-- !query schema struct> --- !query 1 output +-- !query output {"col1":"alpha","col2":"beta"} --- !query 2 +-- !query SELECT STRUCT('alpha' AS A, 'beta' AS B) ST --- !query 2 schema +-- !query schema struct> --- !query 2 output +-- !query output {"A":"alpha","B":"beta"} --- !query 3 +-- !query SELECT ID, STRUCT(ST.*) NST FROM tbl_x --- !query 3 schema +-- !query schema struct> --- !query 3 output +-- !query output 1 {"C":"gamma","D":"delta"} 2 {"C":"epsilon","D":"eta"} 3 {"C":"theta","D":"iota"} --- !query 4 +-- !query SELECT ID, STRUCT(ST.*,CAST(ID AS STRING) AS E) NST FROM tbl_x --- !query 4 schema +-- !query schema struct> --- !query 4 output +-- !query output 1 {"C":"gamma","D":"delta","E":"1"} 2 {"C":"epsilon","D":"eta","E":"2"} 3 {"C":"theta","D":"iota","E":"3"} --- !query 5 +-- !query SELECT ID, STRUCT(CAST(ID AS STRING) AS AA, ST.*) NST FROM tbl_x --- !query 5 schema +-- !query schema struct> --- !query 5 output +-- !query output 1 {"AA":"1","C":"gamma","D":"delta"} 2 {"AA":"2","C":"epsilon","D":"eta"} 3 {"AA":"3","C":"theta","D":"iota"} --- !query 6 +-- !query SELECT ID, STRUCT(ST.*).C NST FROM tbl_x --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 gamma 2 epsilon 3 theta --- !query 7 +-- !query SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 delta 2 eta 3 iota --- !query 8 +-- !query SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 delta 2 eta 3 iota diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out index 97f494cc05063..9f11b46d4088b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 11 --- !query 0 +-- !query CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), @@ -14,13 +14,13 @@ CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (700, "emp 7", date "2010-01-01", 400.00D, 100), (800, "emp 8", date "2016-01-01", 150.00D, 70) AS EMP(id, emp_name, hiredate, salary, dept_id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (10, "dept 1", "CA"), (20, "dept 2", "NY"), @@ -29,13 +29,13 @@ CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (50, "dept 5 - unassigned", "NJ"), (70, "dept 7", "FL") AS DEPT(dept_id, dept_name, state) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 1", 10.00D), ("emp 1", 20.00D), @@ -46,13 +46,13 @@ CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 5", 1000.00D), ("emp 6 - no dept", 500.00D) AS BONUS(emp_name, bonus_amt) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT emp.dept_id, avg(salary), sum(salary) @@ -61,25 +61,25 @@ WHERE EXISTS (SELECT state FROM dept WHERE dept.dept_id = emp.dept_id) GROUP BY dept_id --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 10 133.33333333333334 400.0 20 300.0 300.0 30 400.0 400.0 70 150.0 150.0 --- !query 4 +-- !query SELECT emp_name FROM emp WHERE EXISTS (SELECT max(dept.dept_id) a FROM dept WHERE dept.dept_id = emp.dept_id GROUP BY dept.dept_id) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output emp 1 emp 1 emp 2 @@ -88,20 +88,20 @@ emp 4 emp 8 --- !query 5 +-- !query SELECT count(*) FROM emp WHERE EXISTS (SELECT max(dept.dept_id) a FROM dept WHERE dept.dept_id = emp.dept_id GROUP BY dept.dept_id) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 6 --- !query 6 +-- !query SELECT * FROM bonus WHERE EXISTS (SELECT 1 @@ -111,9 +111,9 @@ WHERE EXISTS (SELECT 1 FROM dept WHERE emp.dept_id = dept.dept_id GROUP BY dept.dept_id)) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 @@ -122,7 +122,7 @@ emp 3 300.0 emp 4 100.0 --- !query 7 +-- !query SELECT emp.dept_id, Avg(salary), Sum(salary) @@ -131,42 +131,42 @@ WHERE NOT EXISTS (SELECT state FROM dept WHERE dept.dept_id = emp.dept_id) GROUP BY dept_id --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 100 400.0 800.0 NULL 400.0 400.0 --- !query 8 +-- !query SELECT emp_name FROM emp WHERE NOT EXISTS (SELECT max(dept.dept_id) a FROM dept WHERE dept.dept_id = emp.dept_id GROUP BY dept.dept_id) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output emp 5 emp 6 - no dept emp 7 --- !query 9 +-- !query SELECT count(*) FROM emp WHERE NOT EXISTS (SELECT max(dept.dept_id) a FROM dept WHERE dept.dept_id = emp.dept_id GROUP BY dept.dept_id) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 3 --- !query 10 +-- !query SELECT * FROM bonus WHERE NOT EXISTS (SELECT 1 @@ -176,8 +176,8 @@ WHERE NOT EXISTS (SELECT 1 FROM dept WHERE emp.dept_id = dept.dept_id GROUP BY dept.dept_id)) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output emp 5 1000.0 emp 6 - no dept 500.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out index 900e4d573bef1..a54fb47fe34f8 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 13 --- !query 0 +-- !query CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), @@ -14,13 +14,13 @@ CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (700, "emp 7", date "2010-01-01", 400.00D, 100), (800, "emp 8", date "2016-01-01", 150.00D, 70) AS EMP(id, emp_name, hiredate, salary, dept_id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (10, "dept 1", "CA"), (20, "dept 2", "NY"), @@ -29,13 +29,13 @@ CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (50, "dept 5 - unassigned", "NJ"), (70, "dept 7", "FL") AS DEPT(dept_id, dept_name, state) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 1", 10.00D), ("emp 1", 20.00D), @@ -46,22 +46,22 @@ CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 5", 1000.00D), ("emp 6 - no dept", 500.00D) AS BONUS(emp_name, bonus_amt) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT 1 FROM dept WHERE dept.dept_id > 10 AND dept.dept_id < 30) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -73,15 +73,15 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 4 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT dept.dept_name FROM dept WHERE emp.dept_id = dept.dept_id) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -90,16 +90,16 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 5 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT dept.dept_name FROM dept WHERE emp.dept_id = dept.dept_id OR emp.dept_id IS NULL) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -109,92 +109,92 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 6 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT dept.dept_name FROM dept WHERE emp.dept_id = dept.dept_id) AND emp.id > 200 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 300 emp 3 2002-01-01 300.0 20 400 emp 4 2005-01-01 400.0 30 800 emp 8 2016-01-01 150.0 70 --- !query 7 +-- !query SELECT emp.emp_name FROM emp WHERE EXISTS (SELECT dept.state FROM dept WHERE emp.dept_id = dept.dept_id) AND emp.id > 200 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output emp 3 emp 4 emp 8 --- !query 8 +-- !query SELECT * FROM dept WHERE NOT EXISTS (SELECT emp_name FROM emp WHERE emp.dept_id = dept.dept_id) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 40 dept 4 - unassigned OR 50 dept 5 - unassigned NJ --- !query 9 +-- !query SELECT * FROM dept WHERE NOT EXISTS (SELECT emp_name FROM emp WHERE emp.dept_id = dept.dept_id OR state = 'NJ') --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 40 dept 4 - unassigned OR --- !query 10 +-- !query SELECT * FROM bonus WHERE NOT EXISTS (SELECT * FROM emp WHERE emp.emp_name = emp_name AND bonus_amt > emp.salary) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 emp 4 100.0 --- !query 11 +-- !query SELECT emp.* FROM emp WHERE NOT EXISTS (SELECT NULL FROM bonus WHERE bonus.emp_name = emp.emp_name) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 700 emp 7 2010-01-01 400.0 100 800 emp 8 2016-01-01 150.0 70 --- !query 12 +-- !query SELECT * FROM bonus WHERE EXISTS (SELECT emp_name @@ -203,9 +203,9 @@ WHERE EXISTS (SELECT emp_name AND EXISTS (SELECT state FROM dept WHERE dept.dept_id = emp.dept_id)) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out index c6c1c04e1c73d..3c8a19998a786 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 8 --- !query 0 +-- !query CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), @@ -14,13 +14,13 @@ CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (700, "emp 7", date "2010-01-01", 400.00D, 100), (800, "emp 8", date "2016-01-01", 150.00D, 70) AS EMP(id, emp_name, hiredate, salary, dept_id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (10, "dept 1", "CA"), (20, "dept 2", "NY"), @@ -29,13 +29,13 @@ CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (50, "dept 5 - unassigned", "NJ"), (70, "dept 7", "FL") AS DEPT(dept_id, dept_name, state) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 1", 10.00D), ("emp 1", 20.00D), @@ -46,13 +46,13 @@ CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 5", 1000.00D), ("emp 6 - no dept", 500.00D) AS BONUS(emp_name, bonus_amt) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query WITH bonus_cte AS (SELECT * FROM bonus @@ -73,16 +73,16 @@ WHERE a.bonus_amt > 30 AND EXISTS (SELECT 1 FROM bonus_cte b WHERE a.emp_name = b.emp_name) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output emp 2 100.0 emp 2 300.0 emp 3 300.0 emp 4 100.0 --- !query 4 +-- !query WITH emp_cte AS (SELECT * FROM emp @@ -99,16 +99,16 @@ WHERE EXISTS (SELECT * JOIN dept_cte b ON a.dept_id = b.dept_id WHERE bonus.emp_name = a.emp_name) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 emp 2 300.0 --- !query 5 +-- !query WITH emp_cte AS (SELECT * FROM emp @@ -130,9 +130,9 @@ WHERE e.dept_id = d.dept_id LEFT JOIN dept_cte b ON a.dept_id = b.dept_id WHERE e.emp_name = a.emp_name) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 @@ -140,7 +140,7 @@ emp 2 300.0 emp 3 300.0 --- !query 6 +-- !query WITH empdept AS (SELECT id, salary, @@ -159,9 +159,9 @@ WHERE EXISTS (SELECT dept_id, GROUP BY dept_id HAVING count(*) > 1) GROUP BY emp_name --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output emp 1 30.0 emp 2 400.0 emp 3 300.0 @@ -170,7 +170,7 @@ emp 5 1000.0 emp 6 - no dept 500.0 --- !query 7 +-- !query WITH empdept AS (SELECT id, salary, @@ -189,9 +189,9 @@ WHERE NOT EXISTS (SELECT dept_id, GROUP BY dept_id HAVING count(*) < 1) GROUP BY emp_name --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output emp 1 30.0 emp 2 400.0 emp 3 300.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-having.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-having.sql.out index de90f5e260e1b..aa4d2ab7e4133 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-having.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 8 --- !query 0 +-- !query CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), @@ -14,13 +14,13 @@ CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (700, "emp 7", date "2010-01-01", 400.00D, 100), (800, "emp 8", date "2016-01-01", 150.00D, 70) AS EMP(id, emp_name, hiredate, salary, dept_id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (10, "dept 1", "CA"), (20, "dept 2", "NY"), @@ -29,13 +29,13 @@ CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (50, "dept 5 - unassigned", "NJ"), (70, "dept 7", "FL") AS DEPT(dept_id, dept_name, state) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 1", 10.00D), ("emp 1", 20.00D), @@ -46,22 +46,22 @@ CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 5", 1000.00D), ("emp 6 - no dept", 500.00D) AS BONUS(emp_name, bonus_amt) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT dept_id, count(*) FROM emp GROUP BY dept_id HAVING EXISTS (SELECT 1 FROM bonus WHERE bonus_amt < min(emp.salary)) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 10 3 100 2 20 1 @@ -70,7 +70,7 @@ struct NULL 1 --- !query 4 +-- !query SELECT * FROM dept WHERE EXISTS (SELECT dept_id, @@ -80,9 +80,9 @@ WHERE EXISTS (SELECT dept_id, HAVING EXISTS (SELECT 1 FROM bonus WHERE bonus_amt < Min(emp.salary))) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 10 dept 1 CA 20 dept 2 NY 30 dept 3 TX @@ -91,7 +91,7 @@ struct 70 dept 7 FL --- !query 5 +-- !query SELECT dept_id, Max(salary) FROM emp gp @@ -103,9 +103,9 @@ WHERE EXISTS (SELECT dept_id, FROM bonus WHERE bonus_amt < Min(p.salary))) GROUP BY gp.dept_id --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 10 200.0 100 400.0 20 300.0 @@ -114,7 +114,7 @@ struct NULL 400.0 --- !query 6 +-- !query SELECT * FROM dept WHERE EXISTS (SELECT dept_id, @@ -124,9 +124,9 @@ WHERE EXISTS (SELECT dept_id, HAVING EXISTS (SELECT 1 FROM bonus WHERE bonus_amt > Min(emp.salary))) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 10 dept 1 CA 20 dept 2 NY 30 dept 3 TX @@ -135,7 +135,7 @@ struct 70 dept 7 FL --- !query 7 +-- !query SELECT * FROM dept WHERE EXISTS (SELECT dept_id, @@ -147,7 +147,7 @@ WHERE EXISTS (SELECT dept_id, FROM bonus WHERE ( bonus_amt > min(emp.salary) AND count(emp.dept_id) > 1 ))) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 10 dept 1 CA diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out index c488cba01d4d0..1a5294930422a 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 17 --- !query 0 +-- !query CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), @@ -14,13 +14,13 @@ CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (700, "emp 7", date "2010-01-01", 400.00D, 100), (800, "emp 8", date "2016-01-01", 150.00D, 70) AS EMP(id, emp_name, hiredate, salary, dept_id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (10, "dept 1", "CA"), (20, "dept 2", "NY"), @@ -29,13 +29,13 @@ CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (50, "dept 5 - unassigned", "NJ"), (70, "dept 7", "FL") AS DEPT(dept_id, dept_name, state) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 1", 10.00D), ("emp 1", 20.00D), @@ -46,13 +46,13 @@ CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 5", 1000.00D), ("emp 6 - no dept", 500.00D) AS BONUS(emp_name, bonus_amt) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM emp, dept @@ -60,9 +60,9 @@ WHERE emp.dept_id = dept.dept_id AND EXISTS (SELECT * FROM bonus WHERE bonus.emp_name = emp.emp_name) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 100 emp 1 2005-01-01 100.0 10 10 dept 1 CA 100 emp 1 2005-01-01 100.0 10 10 dept 1 CA 200 emp 2 2003-01-01 200.0 10 10 dept 1 CA @@ -70,7 +70,7 @@ struct --- !query 4 output +-- !query output 100 emp 1 2005-01-01 100.0 10 10 dept 1 CA 100 emp 1 2005-01-01 100.0 10 10 dept 1 CA 200 emp 2 2003-01-01 200.0 10 10 dept 1 CA @@ -88,7 +88,7 @@ struct --- !query 5 output +-- !query output 100 emp 1 2005-01-01 100.0 10 10 dept 1 CA 100 emp 1 2005-01-01 100.0 10 10 dept 1 CA 200 emp 2 2003-01-01 200.0 10 10 dept 1 CA @@ -108,7 +108,7 @@ struct --- !query 6 output +-- !query output 800 emp 8 2016-01-01 150.0 70 70 dept 7 FL --- !query 7 +-- !query SELECT * FROM bonus WHERE EXISTS (SELECT * @@ -130,9 +130,9 @@ WHERE EXISTS (SELECT * JOIN dept ON dept.dept_id = emp.dept_id WHERE bonus.emp_name = emp.emp_name) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 @@ -141,7 +141,7 @@ emp 3 300.0 emp 4 100.0 --- !query 8 +-- !query SELECT * FROM bonus WHERE EXISTS (SELECT * @@ -149,9 +149,9 @@ WHERE EXISTS (SELECT * RIGHT JOIN dept ON dept.dept_id = emp.dept_id WHERE bonus.emp_name = emp.emp_name) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 @@ -160,7 +160,7 @@ emp 3 300.0 emp 4 100.0 --- !query 9 +-- !query SELECT * FROM bonus WHERE EXISTS (SELECT dept.dept_id, @@ -174,9 +174,9 @@ WHERE EXISTS (SELECT dept.dept_id, GROUP BY dept.dept_id, emp.emp_name ORDER BY emp.emp_name) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 @@ -185,7 +185,7 @@ emp 3 300.0 emp 4 100.0 --- !query 10 +-- !query SELECT emp_name, Sum(bonus_amt) FROM bonus @@ -199,13 +199,13 @@ WHERE EXISTS (SELECT emp_name, HAVING Count(*) > 1 ORDER BY emp_name) GROUP BY emp_name --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output emp 1 30.0 --- !query 11 +-- !query SELECT emp_name, Sum(bonus_amt) FROM bonus @@ -219,9 +219,9 @@ WHERE NOT EXISTS (SELECT emp_name, HAVING Count(*) > 1 ORDER BY emp_name) GROUP BY emp_name --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output emp 2 400.0 emp 3 300.0 emp 4 100.0 @@ -229,7 +229,7 @@ emp 5 1000.0 emp 6 - no dept 500.0 --- !query 12 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT * @@ -240,9 +240,9 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id >= 30 AND dept_id <= 50) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -254,7 +254,7 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 13 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT * @@ -265,13 +265,13 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id >= 30 AND dept_id <= 50) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output --- !query 14 +-- !query SELECT * FROM emp WHERE NOT EXISTS (SELECT * @@ -282,9 +282,9 @@ WHERE NOT EXISTS (SELECT * FROM dept WHERE dept_id >= 30 AND dept_id <= 50) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -296,7 +296,7 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 15 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT * @@ -316,9 +316,9 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id >= 30 AND dept_id <= 50) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -330,7 +330,7 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 16 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT * @@ -350,9 +350,9 @@ WHERE EXISTS (SELECT * FROM dept WHERE dept_id >= 30 AND dept_id <= 50) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 300 emp 3 2002-01-01 300.0 20 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out index ee13ff2c4f38d..ebd4da6ccbd5d 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 12 --- !query 0 +-- !query CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), @@ -14,13 +14,13 @@ CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (700, "emp 7", date "2010-01-01", 400.00D, 100), (800, "emp 8", date "2016-01-01", 150.00D, 70) AS EMP(id, emp_name, hiredate, salary, dept_id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (10, "dept 1", "CA"), (20, "dept 2", "NY"), @@ -29,13 +29,13 @@ CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (50, "dept 5 - unassigned", "NJ"), (70, "dept 7", "FL") AS DEPT(dept_id, dept_name, state) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 1", 10.00D), ("emp 1", 20.00D), @@ -46,13 +46,13 @@ CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 5", 1000.00D), ("emp 6 - no dept", 500.00D) AS BONUS(emp_name, bonus_amt) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT dept.dept_id @@ -60,9 +60,9 @@ WHERE EXISTS (SELECT dept.dept_id WHERE emp.dept_id = dept.dept_id ORDER BY state) ORDER BY hiredate --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 300 emp 3 2002-01-01 300.0 20 200 emp 2 2003-01-01 200.0 10 100 emp 1 2005-01-01 100.0 10 @@ -71,7 +71,7 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 4 +-- !query SELECT id, hiredate FROM emp @@ -80,9 +80,9 @@ WHERE EXISTS (SELECT dept.dept_id WHERE emp.dept_id = dept.dept_id ORDER BY state) ORDER BY hiredate DESC --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 800 2016-01-01 100 2005-01-01 100 2005-01-01 @@ -91,7 +91,7 @@ struct 300 2002-01-01 --- !query 5 +-- !query SELECT * FROM emp WHERE NOT EXISTS (SELECT dept.dept_id @@ -99,15 +99,15 @@ WHERE NOT EXISTS (SELECT dept.dept_id WHERE emp.dept_id = dept.dept_id ORDER BY state) ORDER BY hiredate --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 500 emp 5 2001-01-01 400.0 NULL 600 emp 6 - no dept 2001-01-01 400.0 100 700 emp 7 2010-01-01 400.0 100 --- !query 6 +-- !query SELECT emp_name FROM emp WHERE NOT EXISTS (SELECT max(dept.dept_id) a @@ -115,15 +115,15 @@ WHERE NOT EXISTS (SELECT max(dept.dept_id) a WHERE dept.dept_id = emp.dept_id GROUP BY state ORDER BY state) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output emp 5 emp 6 - no dept emp 7 --- !query 7 +-- !query SELECT count(*) FROM emp WHERE NOT EXISTS (SELECT max(dept.dept_id) a @@ -131,22 +131,22 @@ WHERE NOT EXISTS (SELECT max(dept.dept_id) a WHERE dept.dept_id = emp.dept_id GROUP BY dept_id ORDER BY dept_id) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 3 --- !query 8 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT dept.dept_name FROM dept WHERE dept.dept_id > 10 LIMIT 1) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -158,16 +158,16 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 9 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT max(dept.dept_id) FROM dept GROUP BY state LIMIT 1) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -179,16 +179,16 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 10 +-- !query SELECT * FROM emp WHERE NOT EXISTS (SELECT dept.dept_name FROM dept WHERE dept.dept_id > 100 LIMIT 1) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -200,7 +200,7 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 11 +-- !query SELECT * FROM emp WHERE NOT EXISTS (SELECT max(dept.dept_id) @@ -208,9 +208,9 @@ WHERE NOT EXISTS (SELECT max(dept.dept_id) WHERE dept.dept_id > 100 GROUP BY state LIMIT 1) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out index 865e4ed14e4ab..6a17c2fc86d40 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 8 --- !query 0 +-- !query CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (100, "emp 1", date "2005-01-01", 100.00D, 10), (100, "emp 1", date "2005-01-01", 100.00D, 10), @@ -14,13 +14,13 @@ CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES (700, "emp 7", date "2010-01-01", 400.00D, 100), (800, "emp 8", date "2016-01-01", 150.00D, 70) AS EMP(id, emp_name, hiredate, salary, dept_id) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (10, "dept 1", "CA"), (20, "dept 2", "NY"), @@ -29,13 +29,13 @@ CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES (50, "dept 5 - unassigned", "NJ"), (70, "dept 7", "FL") AS DEPT(dept_id, dept_name, state) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 1", 10.00D), ("emp 1", 20.00D), @@ -46,22 +46,22 @@ CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES ("emp 5", 1000.00D), ("emp 6 - no dept", 500.00D) AS BONUS(emp_name, bonus_amt) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT emp.emp_name FROM emp WHERE EXISTS (SELECT dept.state FROM dept WHERE emp.dept_id = dept.dept_id) OR emp.id > 200 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output emp 1 emp 1 emp 2 @@ -73,16 +73,16 @@ emp 7 emp 8 --- !query 4 +-- !query SELECT * FROM emp WHERE EXISTS (SELECT dept.dept_name FROM dept WHERE emp.dept_id = dept.dept_id) OR emp.dept_id IS NULL --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 100 emp 1 2005-01-01 100.0 10 100 emp 1 2005-01-01 100.0 10 200 emp 2 2003-01-01 200.0 10 @@ -92,7 +92,7 @@ struct 800 emp 8 2016-01-01 150.0 70 --- !query 5 +-- !query SELECT emp.emp_name FROM emp WHERE EXISTS (SELECT dept.state @@ -103,14 +103,14 @@ WHERE EXISTS (SELECT dept.state FROM dept WHERE emp.dept_id = dept.dept_id AND dept.dept_id = 30) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output emp 3 emp 4 --- !query 6 +-- !query SELECT * FROM bonus WHERE ( NOT EXISTS (SELECT * @@ -121,9 +121,9 @@ WHERE ( NOT EXISTS (SELECT * FROM emp WHERE emp.emp_name = emp_name OR bonus_amt < emp.salary) ) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 @@ -134,7 +134,7 @@ emp 5 1000.0 emp 6 - no dept 500.0 --- !query 7 +-- !query SELECT * FROM bonus WHERE NOT EXISTS ( SELECT * @@ -147,9 +147,9 @@ emp_name IN SELECT emp_name FROM emp WHERE bonus_amt < emp.salary) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output emp 1 10.0 emp 1 20.0 emp 2 100.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out index 686fe4975379b..a33f78abf27f9 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-basic.sql.out @@ -2,44 +2,44 @@ -- Number of queries: 7 --- !query 0 +-- !query create temporary view tab_a as select * from values (1, 1) as tab_a(a1, b1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view tab_b as select * from values (1, 1) as tab_b(a2, b2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view struct_tab as select struct(col1 as a, col2 as b) as record from values (1, 1), (1, 2), (2, 1), (2, 2) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query select 1 from tab_a where (a1, b1) not in (select a2, b2 from tab_b) --- !query 3 schema +-- !query schema struct<1:int> --- !query 3 output +-- !query output --- !query 4 +-- !query select 1 from tab_a where (a1, b1) not in (select (a2, b2) from tab_b) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(named_struct('a1', tab_a.`a1`, 'b1', tab_a.`b1`) IN (listquery()))' due to data type mismatch: The number of columns in the left hand side of an IN subquery does not match the @@ -52,19 +52,19 @@ Right side columns: [`named_struct(a2, a2, b2, b2)`].; --- !query 5 +-- !query select count(*) from struct_tab where record in (select (a2 as a, b2 as b) from tab_b) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 --- !query 6 +-- !query select count(*) from struct_tab where record not in (select (a2 as a, b2 as b) from tab_b) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 3 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out index 64605c995b443..f378664014fdb 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 19 --- !query 0 +-- !query create temporary view t1 as select * from values ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,28 +60,28 @@ create temporary view t3 as select * from values ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT t1a, Avg(t1b) FROM t1 WHERE t1a IN (SELECT t2a FROM t2) GROUP BY t1a --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output t1b 8.0 t1c 8.0 t1e 10.0 --- !query 4 +-- !query SELECT t1a, Max(t1b) FROM t1 @@ -90,13 +90,13 @@ WHERE t1b IN (SELECT t2b WHERE t1a = t2a) GROUP BY t1a, t1d --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output t1b 8 --- !query 5 +-- !query SELECT t1a, t1b FROM t1 @@ -105,14 +105,14 @@ WHERE t1c IN (SELECT t2c WHERE t1a = t2a) GROUP BY t1a, t1b --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output t1b 8 t1c 8 --- !query 6 +-- !query SELECT t1a, Sum(DISTINCT( t1b )) FROM t1 @@ -124,14 +124,14 @@ WHERE t1c IN (SELECT t2c WHERE t1a = t3a) GROUP BY t1a, t1c --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output t1b 8 t1c 8 --- !query 7 +-- !query SELECT t1a, Sum(DISTINCT( t1b )) FROM t1 @@ -143,13 +143,13 @@ WHERE t1c IN (SELECT t2c WHERE t1a = t3a) GROUP BY t1a, t1c --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output t1b 8 --- !query 8 +-- !query SELECT t1a, Count(DISTINCT( t1b )) FROM t1 @@ -159,21 +159,21 @@ WHERE t1c IN (SELECT t2c GROUP BY t1a, t1c HAVING t1a = "t1b" --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output t1b 1 --- !query 9 +-- !query SELECT * FROM t1 WHERE t1b IN (SELECT Max(t2b) FROM t2 GROUP BY t2a) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output t1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 t1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 @@ -184,7 +184,7 @@ t1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 t1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 --- !query 10 +-- !query SELECT * FROM (SELECT t2a, t2b @@ -194,13 +194,13 @@ FROM (SELECT t2a, WHERE t1b = t2b) GROUP BY t2a, t2b) t2 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output t1b 8 --- !query 11 +-- !query SELECT Count(DISTINCT( * )) FROM t1 WHERE t1b IN (SELECT Min(t2b) @@ -208,13 +208,13 @@ WHERE t1b IN (SELECT Min(t2b) WHERE t1a = t2a AND t1c = t2c GROUP BY t2a) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 --- !query 12 +-- !query SELECT t1a, t1b FROM t1 @@ -224,14 +224,14 @@ WHERE t1c IN (SELECT Max(t2c) GROUP BY t2a, t2c HAVING t2c > 8) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output t1b 8 t1c 8 --- !query 13 +-- !query SELECT t1a, t1b FROM t1 @@ -242,9 +242,9 @@ WHERE t1c IN (SELECT t2c WHERE t3a = t2a GROUP BY t3b) GROUP BY t2c) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output t1a 16 t1a 16 t1b 8 @@ -253,7 +253,7 @@ t1d NULL t1d NULL --- !query 14 +-- !query SELECT t1a, Min(t1b) FROM t1 @@ -262,14 +262,14 @@ WHERE t1c IN (SELECT Min(t2c) WHERE t2b = t1b GROUP BY t2a) GROUP BY t1a --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output t1b 8 t1c 8 --- !query 15 +-- !query SELECT t1a, Min(t1b) FROM t1 @@ -282,16 +282,16 @@ WHERE t1c IN (SELECT Min(t2c) GROUP BY t2c) GROUP BY t1a, t1d --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output t1b 8 t1c 8 t1d NULL t1d NULL --- !query 16 +-- !query SELECT t1a, Min(t1b) FROM t1 @@ -304,14 +304,14 @@ WHERE t1c IN (SELECT Min(t2c) WHERE t1c = t3c GROUP BY t3d) GROUP BY t1a --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output t1b 8 t1c 8 --- !query 17 +-- !query SELECT t1a, Min(t1b) FROM t1 @@ -324,16 +324,16 @@ WHERE t1c IN (SELECT Min(t2c) WHERE t1c = t3c GROUP BY t3d) GROUP BY t1a --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output t1a 16 t1b 8 t1c 8 t1d NULL --- !query 18 +-- !query SELECT t1a, Min(t1b) FROM t1 @@ -349,9 +349,9 @@ WHERE t1c IN (SELECT Min(t2c) HAVING t3d = t1d) GROUP BY t1a HAVING Min(t1b) IS NOT NULL --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output t1a 16 t1b 8 t1c 8 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-having.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-having.sql.out index b90ebf57e739b..09b6adbe62b36 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-having.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 12 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,13 +60,13 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT t1a, t1b, t1h @@ -75,16 +75,16 @@ WHERE t1b IN (SELECT t2b FROM t2 GROUP BY t2b HAVING t2b < 10) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output val1a 6 2014-04-04 01:00:00 val1a 6 2014-04-04 01:02:00.001 val1b 8 2014-05-04 01:01:00 val1c 8 2014-05-04 01:02:00.001 --- !query 4 +-- !query SELECT t1a, t1b, t1c @@ -94,13 +94,13 @@ WHERE t1b IN (SELECT Min(t2b) WHERE t1a = t2a GROUP BY t2b HAVING t2b > 1) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output val1b 8 16 --- !query 5 +-- !query SELECT t1a, t1b, t1c FROM t1 WHERE t1b IN (SELECT t2b @@ -108,13 +108,13 @@ WHERE t1b IN (SELECT t2b WHERE t1c < t2c) GROUP BY t1a, t1b, t1c HAVING t1b < 10 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output val1a 6 8 --- !query 6 +-- !query SELECT t1a, t1b, t1c FROM t1 WHERE t1b IN (SELECT t2b @@ -122,14 +122,14 @@ WHERE t1b IN (SELECT t2b WHERE t1c = t2c) GROUP BY t1a, t1b, t1c HAVING COUNT (DISTINCT t1b) < 10 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output val1b 8 16 val1c 8 16 --- !query 7 +-- !query SELECT Count(DISTINCT( t1a )), t1b FROM t1 @@ -140,13 +140,13 @@ WHERE t1c IN (SELECT t2c HAVING t2c > 10) GROUP BY t1b HAVING t1b >= 8 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 2 8 --- !query 8 +-- !query SELECT t1a, Max(t1b) FROM t1 @@ -158,13 +158,13 @@ HAVING t1a IN (SELECT t2a FROM t3 WHERE t2c = t3c) ) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output val1b 8 --- !query 9 +-- !query SELECT t1a, t1c, Min(t1d) @@ -175,16 +175,16 @@ WHERE t1a NOT IN (SELECT t2a HAVING t2a > 'val2a') GROUP BY t1a, t1c HAVING Min(t1d) > t1c --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output val1a 8 10 val1b 16 19 val1c 16 19 val1d 16 19 --- !query 10 +-- !query SELECT t1a, t1b FROM t1 @@ -195,13 +195,13 @@ WHERE t1d NOT IN (SELECT t2d HAVING t2c > 8) GROUP BY t1a, t1b HAVING t1b < 10 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output val1a 6 --- !query 11 +-- !query SELECT t1a, Max(t1b) FROM t1 @@ -210,8 +210,8 @@ GROUP BY t1a HAVING t1a NOT IN (SELECT t2a FROM t2 WHERE t2b > 3) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output val1a 16 val1d 10 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out index 9a11ef13cdf1f..615b67f629e55 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 34 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,43 +60,43 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query create temporary view s1 as select * from values (1), (3), (5), (7), (9) as s1(id) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query create temporary view s2 as select * from values (1), (3), (4), (6), (9) as s2(id) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query create temporary view s3 as select * from values (3), (4), (6), (9) as s3(id) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT t1a, t1b, t1c, t3a, t3b, t3c FROM t1 natural JOIN t3 WHERE t1a IN (SELECT t2a @@ -107,14 +107,14 @@ WHERE t1a IN (SELECT t2a ORDER BY t1a, t1b, t1c DESC nulls first --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output val1b 8 16 val1b 8 16 val1b 8 16 val1b 8 16 --- !query 7 +-- !query SELECT Count(DISTINCT(t1a)), t1b, t3a, @@ -133,9 +133,9 @@ GROUP BY t1a, t3b, t3c ORDER BY t1a DESC, t3b DESC, t3c ASC --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 10 val3b 8 NULL 1 10 val1b 8 16 1 10 val3a 6 12 @@ -143,7 +143,7 @@ struct 1 8 val3a 6 12 --- !query 8 +-- !query SELECT Count(DISTINCT(t1a)) FROM t1 natural right JOIN t3 WHERE t1a IN @@ -159,13 +159,13 @@ AND t1d IN AND t1a = t3a GROUP BY t1a ORDER BY t1a --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 --- !query 9 +-- !query SELECT t1a, t1b, t1c, @@ -181,9 +181,9 @@ where t1a IN AND t1b != t3b AND t1a = 'val1b' ORDER BY t1a --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output val1b 8 16 val3a 6 12 val1b 8 16 val3a 6 12 val1b 8 16 val1b 10 12 @@ -192,7 +192,7 @@ val1b 8 16 val3c 17 16 val1b 8 16 val3c 17 16 --- !query 10 +-- !query SELECT Count(DISTINCT(t1a)), t1b FROM t1 RIGHT JOIN t3 @@ -211,13 +211,13 @@ GROUP BY t1a, t1b HAVING t1b > 8 ORDER BY t1a --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 10 --- !query 11 +-- !query SELECT Count(DISTINCT(t1a)) FROM t1 LEFT OUTER JOIN t3 @@ -229,15 +229,15 @@ WHERE t1a IN WHERE t1h < t2h ) GROUP BY t1a ORDER BY t1a --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 1 1 --- !query 12 +-- !query SELECT Count(DISTINCT(t1a)), t1b FROM t1 INNER JOIN t2 @@ -254,14 +254,14 @@ OR t1a IN WHERE t2h < t1h) GROUP BY t1b HAVING t1b > 6 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 10 1 8 --- !query 13 +-- !query SELECT Count(DISTINCT(t1a)), t1b FROM t1 @@ -279,13 +279,13 @@ AND t1h IN where t2b = t3b) GROUP BY t1b HAVING t1b > 8 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 10 --- !query 14 +-- !query SELECT Count(DISTINCT(t1a)), t1b FROM t1 @@ -310,13 +310,13 @@ AND t1b IN GROUP BY t1b HAVING t1b > 8 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 10 --- !query 15 +-- !query SELECT Count(DISTINCT(t1a)), t1b FROM t1 @@ -344,13 +344,13 @@ AND t1b IN AND t1a = t2a GROUP BY t1b ORDER BY t1b DESC --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 8 --- !query 16 +-- !query SELECT t1a, t1b, t1c, @@ -375,79 +375,79 @@ and t1a = t2a Group By t1a, t1b, t1c, t2a, t2b, t2c HAVING t2c IS NOT NULL ORDER By t2b DESC nulls last --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output val1b 8 16 1 10 12 val1b 8 16 1 8 16 val1b 8 16 1 NULL 16 --- !query 17 +-- !query SELECT s1.id FROM s1 JOIN s2 ON s1.id = s2.id AND s1.id IN (SELECT 9) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 9 --- !query 18 +-- !query SELECT s1.id FROM s1 JOIN s2 ON s1.id = s2.id AND s1.id NOT IN (SELECT 9) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 3 --- !query 19 +-- !query SELECT s1.id FROM s1 JOIN s2 ON s1.id = s2.id AND s1.id IN (SELECT id FROM s3) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 3 9 --- !query 20 +-- !query SELECT s1.id AS id2 FROM s1 LEFT SEMI JOIN s2 ON s1.id = s2.id AND s1.id IN (SELECT id FROM s3) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 3 9 --- !query 21 +-- !query SELECT s1.id as id2 FROM s1 LEFT ANTI JOIN s2 ON s1.id = s2.id AND s1.id IN (SELECT id FROM s3) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 5 7 --- !query 22 +-- !query SELECT s1.id, s2.id as id2 FROM s1 LEFT OUTER JOIN s2 ON s1.id = s2.id AND s1.id IN (SELECT id FROM s3) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 1 NULL 3 3 5 NULL @@ -455,14 +455,14 @@ struct 9 9 --- !query 23 +-- !query SELECT s1.id, s2.id as id2 FROM s1 RIGHT OUTER JOIN s2 ON s1.id = s2.id AND s1.id IN (SELECT id FROM s3) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 3 3 9 9 NULL 1 @@ -470,14 +470,14 @@ NULL 4 NULL 6 --- !query 24 +-- !query SELECT s1.id, s2.id AS id2 FROM s1 FULL OUTER JOIN s2 ON s1.id = s2.id AND s1.id IN (SELECT id FROM s3) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1 NULL 3 3 5 NULL @@ -488,49 +488,49 @@ NULL 4 NULL 6 --- !query 25 +-- !query SELECT s1.id FROM s1 JOIN s2 ON s1.id = s2.id AND s1.id NOT IN (SELECT id FROM s3) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1 --- !query 26 +-- !query SELECT s1.id AS id2 FROM s1 LEFT SEMI JOIN s2 ON s1.id = s2.id AND s1.id NOT IN (SELECT id FROM s3) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 --- !query 27 +-- !query SELECT s1.id AS id2 FROM s1 LEFT ANTI JOIN s2 ON s1.id = s2.id AND s1.id NOT IN (SELECT id FROM s3) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 3 5 7 9 --- !query 28 +-- !query SELECT s1.id, s2.id AS id2 FROM s1 LEFT OUTER JOIN s2 ON s1.id = s2.id AND s1.id NOT IN (SELECT id FROM s3) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 1 1 3 NULL 5 NULL @@ -538,14 +538,14 @@ struct 9 NULL --- !query 29 +-- !query SELECT s1.id, s2.id AS id2 FROM s1 RIGHT OUTER JOIN s2 ON s1.id = s2.id AND s1.id NOT IN (SELECT id FROM s3) --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 1 1 NULL 3 NULL 4 @@ -553,14 +553,14 @@ NULL 6 NULL 9 --- !query 30 +-- !query SELECT s1.id, s2.id AS id2 FROM s1 FULL OUTER JOIN s2 ON s1.id = s2.id AND s1.id NOT IN (SELECT id FROM s3) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1 1 3 NULL 5 NULL @@ -572,25 +572,25 @@ NULL 6 NULL 9 --- !query 31 +-- !query DROP VIEW s1 --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output --- !query 32 +-- !query DROP VIEW s2 --- !query 32 schema +-- !query schema struct<> --- !query 32 output +-- !query output --- !query 33 +-- !query DROP VIEW s3 --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out index 0975cb97b19fa..1c335445114c7 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 8 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,27 +60,27 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM t1 WHERE t1a IN (SELECT t2a FROM t2 WHERE t1d = t2d) LIMIT 2 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 --- !query 4 +-- !query SELECT * FROM t1 WHERE t1c IN (SELECT t2c @@ -88,16 +88,16 @@ WHERE t1c IN (SELECT t2c WHERE t2b >= 8 LIMIT 2) LIMIT 4 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 --- !query 5 +-- !query SELECT Count(DISTINCT( t1a )), t1b FROM t1 @@ -108,29 +108,29 @@ WHERE t1d IN (SELECT t2d GROUP BY t1b ORDER BY t1b DESC NULLS FIRST LIMIT 1 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 NULL --- !query 6 +-- !query SELECT * FROM t1 WHERE t1b NOT IN (SELECT t2b FROM t2 WHERE t2b > 6 LIMIT 2) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 --- !query 7 +-- !query SELECT Count(DISTINCT( t1a )), t1b FROM t1 @@ -141,7 +141,7 @@ WHERE t1d NOT IN (SELECT t2d GROUP BY t1b ORDER BY t1b NULLS last LIMIT 1 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 6 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out index 7a96c4bc5a30b..c6e13715bd9fa 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 8 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,13 +60,13 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT t1a, t1b, t1h @@ -77,16 +77,16 @@ WHERE ( t1a, t1h ) NOT IN (SELECT t2a, WHERE t2a = t1a ORDER BY t2a) AND t1a = 'val1a' --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output val1a 16 2014-06-04 01:02:00.001 val1a 16 2014-07-04 01:01:00 val1a 6 2014-04-04 01:00:00 val1a 6 2014-04-04 01:02:00.001 --- !query 4 +-- !query SELECT t1a, t1b, t1d @@ -97,14 +97,14 @@ WHERE ( t1b, t1d ) IN (SELECT t2b, WHERE t2i IN (SELECT t3i FROM t3 WHERE t2b > t3b)) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output val1e 10 19 val1e 10 19 --- !query 5 +-- !query SELECT t1a, t1b, t1d @@ -116,16 +116,16 @@ WHERE ( t1b, t1d ) NOT IN (SELECT t2b, FROM t3 WHERE t2b > t3b)) AND t1a = 'val1a' --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output val1a 16 10 val1a 16 21 val1a 6 10 val1a 6 10 --- !query 6 +-- !query SELECT t2a FROM (SELECT t2a FROM t2 @@ -144,13 +144,13 @@ FROM (SELECT t2a WHERE ( t2a, t2b ) IN (SELECT t3a, t3b FROM t3)) AS t4 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output val1b --- !query 7 +-- !query WITH cte1 AS ( SELECT t1a, @@ -169,9 +169,9 @@ FROM ( FROM cte1 JOIN cte1 cte2 on cte1.t1b = cte2.t1b) s --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output val1b 8 val1b 8 val1b 8 val1c 8 val1c 8 val1b 8 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out index aa36be9040369..96b418c54bf5b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 18 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,21 +60,21 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM t1 WHERE t1a IN (SELECT t2a FROM t2) ORDER BY t1a --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 @@ -82,20 +82,20 @@ val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 --- !query 4 +-- !query SELECT t1a FROM t1 WHERE t1b IN (SELECT t2b FROM t2 WHERE t1a = t2a) ORDER BY t1b DESC --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output val1b --- !query 5 +-- !query SELECT t1a, t1b FROM t1 @@ -103,40 +103,40 @@ WHERE t1c IN (SELECT t2c FROM t2 WHERE t1a = t2a) ORDER BY 2 DESC nulls last --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output val1b 8 val1c 8 --- !query 6 +-- !query SELECT Count(DISTINCT( t1a )) FROM t1 WHERE t1b IN (SELECT t2b FROM t2 WHERE t1a = t2a) ORDER BY Count(DISTINCT( t1a )) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 --- !query 7 +-- !query SELECT * FROM t1 WHERE t1b IN (SELECT t2c FROM t2 ORDER BY t2d) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 --- !query 8 +-- !query SELECT * FROM t1 WHERE t1b IN (SELECT Min(t2b) @@ -144,9 +144,9 @@ WHERE t1b IN (SELECT Min(t2b) WHERE t1b = t2b ORDER BY Min(t2b)) ORDER BY t1c DESC nulls first --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 @@ -157,7 +157,7 @@ val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 --- !query 9 +-- !query SELECT t1a, t1b, t1h @@ -170,22 +170,22 @@ WHERE t1c IN (SELECT t2c FROM t2 WHERE t1h > t2h) ORDER BY t1h DESC nulls last --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output val1c 8 2014-05-04 01:02:00.001 val1b 8 2014-05-04 01:01:00 --- !query 10 +-- !query SELECT * FROM t1 WHERE t1a NOT IN (SELECT t2a FROM t2) ORDER BY t1a --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 @@ -195,7 +195,7 @@ val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 --- !query 11 +-- !query SELECT t1a, t1b FROM t1 @@ -203,9 +203,9 @@ WHERE t1a NOT IN (SELECT t2a FROM t2 WHERE t1a = t2a) ORDER BY t1b DESC nulls last --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output val1a 16 val1a 16 val1d 10 @@ -215,7 +215,7 @@ val1d NULL val1d NULL --- !query 12 +-- !query SELECT * FROM t1 WHERE t1a NOT IN (SELECT t2a @@ -225,32 +225,32 @@ WHERE t1a NOT IN (SELECT t2a FROM t2 ORDER BY t2b DESC nulls last) ORDER BY t1c DESC nulls last --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 --- !query 13 +-- !query SELECT * FROM t1 WHERE t1b IN (SELECT Min(t2b) FROM t2 GROUP BY t2a ORDER BY t2a DESC) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 --- !query 14 +-- !query SELECT t1a, Count(DISTINCT( t1b )) FROM t1 @@ -262,22 +262,22 @@ WHERE t1b IN (SELECT Min(t2b) GROUP BY t1a, t1h ORDER BY t1a --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output val1b 1 --- !query 15 +-- !query SELECT * FROM t1 WHERE t1b NOT IN (SELECT Min(t2b) FROM t2 GROUP BY t2a ORDER BY t2a) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 @@ -286,7 +286,7 @@ val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 --- !query 16 +-- !query SELECT t1a, Sum(DISTINCT( t1b )) FROM t1 @@ -296,16 +296,16 @@ WHERE t1b NOT IN (SELECT Min(t2b) GROUP BY t2c ORDER BY t2c DESC nulls last) GROUP BY t1a --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output val1a 22 val1c 8 val1d 10 val1e 10 --- !query 17 +-- !query SELECT Count(DISTINCT( t1a )), t1b FROM t1 @@ -317,9 +317,9 @@ WHERE t1h NOT IN (SELECT t2h GROUP BY t1a, t1b ORDER BY t1b DESC nulls last --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 16 1 10 1 10 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out index 53ddaa36948ff..783f4031a452b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 16 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,13 +60,13 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT t2a, t2b, t2c, @@ -84,16 +84,16 @@ FROM (SELECT * WHERE t2i IS NOT NULL AND 2 * t2b = t2c ORDER BY t2c DESC nulls first --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output val1b 8 16 2015-05-04 01:01:00 2015-05-04 val1b 8 16 2014-07-04 01:01:00 2014-07-04 val1b 8 16 2014-06-04 01:02:00 2014-06-04 val1b 8 16 2014-07-04 01:02:00 2014-07-04 --- !query 4 +-- !query SELECT t2a, t2b, t2d, @@ -115,15 +115,15 @@ GROUP BY t2a, t2d, t2i ORDER BY t2d DESC --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output val1b 8 119 1 2015-05-04 val1b 8 19 1 2014-07-04 val1b 8 19 1 2014-05-04 --- !query 5 +-- !query SELECT t2a, t2b, t2c, @@ -163,9 +163,9 @@ WHERE t1a IN (SELECT t3a FROM t3 WHERE t3d = t1d) GROUP BY t1a, t1b, t1c --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output val1b 10 12 19 val1b 8 16 119 val1b 8 16 19 @@ -174,7 +174,7 @@ val1b NULL 16 319 val1c 12 16 219 --- !query 6 +-- !query SELECT DISTINCT( t2a ), t2b, Count(t2c), @@ -209,16 +209,16 @@ GROUP BY t2a, t2h, t2i HAVING t2b IS NOT NULL --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output val1b 8 1 119 2015-05-04 01:01:00 2015-05-04 val1b 8 1 19 2014-07-04 01:01:00 2014-07-04 val1c 12 1 19 2014-08-04 01:01:00 2014-08-05 val1c 12 1 219 2016-05-04 01:01:00 2016-05-04 --- !query 7 +-- !query SELECT t2a, t2b, Count(t2c), @@ -265,9 +265,9 @@ FROM t2 WHERE t2d IN (SELECT min(t1d) FROM t1 WHERE t2c = t1c) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output val1b 8 1 119 2015-05-04 01:01:00 2015-05-04 val1b 8 1 19 2014-07-04 01:01:00 2014-07-04 val1b 8 16 19 2014-07-04 01:01:00 2014-07-04 @@ -275,7 +275,7 @@ val1b NULL 16 19 2014-05-04 01:01:00 NULL val1c 12 16 19 2014-08-04 01:01:00 2014-08-05 --- !query 8 +-- !query SELECT t2a, t2b, t2c, @@ -312,16 +312,16 @@ FROM t2 WHERE t2c IN (SELECT Max(t1c) FROM t1 WHERE t1d = t2d) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output val1b 8 16 119 val1b 8 16 19 val1b NULL 16 19 val1c 12 16 19 --- !query 9 +-- !query SELECT DISTINCT(t1a), t1b, t1c, @@ -354,9 +354,9 @@ WHERE t1a IN (SELECT t3a GROUP BY t1a, t1b, t1c, t1d HAVING t1c IS NOT NULL AND t1b IS NOT NULL ORDER BY t1c DESC, t1a DESC --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output val1c 8 16 19 val1b 8 16 19 val1a 16 12 21 @@ -364,7 +364,7 @@ val1a 16 12 10 val1a 6 8 10 --- !query 10 +-- !query SELECT t1a, t1b, t1c @@ -378,9 +378,9 @@ WHERE t1b IN (SELECT t2b FROM t1 WHERE t1b > 6) AS t3 WHERE t2b = t1b) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output val1b 8 16 val1c 8 16 val1d 10 NULL @@ -389,7 +389,7 @@ val1e 10 NULL val1e 10 NULL --- !query 11 +-- !query SELECT t1a, t1b, t1c @@ -401,9 +401,9 @@ WHERE t1h IN (SELECT t2h SELECT t3h FROM t3) AS t3) ORDER BY t1b DESC NULLs first, t1c DESC NULLs last --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output val1d NULL 16 val1a 16 12 val1e 10 NULL @@ -412,7 +412,7 @@ val1e 10 NULL val1b 8 16 --- !query 12 +-- !query SELECT t1a, t1b, t1c @@ -446,16 +446,16 @@ WHERE t1b IN WHERE t1b > 6) AS t4 WHERE t2b = t1b) ORDER BY t1c DESC NULLS last, t1a DESC --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output val1c 8 16 val1b 8 16 val1e 10 NULL val1d 10 NULL --- !query 13 +-- !query SELECT * FROM (SELECT * FROM (SELECT * @@ -497,13 +497,13 @@ FROM (SELECT * WHERE t4.t2b IN (SELECT Min(t3b) FROM t3 WHERE t4.t2a = t3a)) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 --- !query 14 +-- !query SELECT t2a, t2b, t2c, @@ -530,14 +530,14 @@ WHERE t3.t2a NOT IN (SELECT t1a FROM t2) AND t2c IS NOT NULL ORDER BY t2a --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output val2a 6 12 2014-04-04 val2a 6 12 2014-04-04 --- !query 15 +-- !query SELECT Count(DISTINCT(t1a)), t1b, t1c, @@ -581,9 +581,9 @@ HAVING t1b NOT IN SELECT t3b FROM t3) ORDER BY t1c DESC NULLS LAST, t1i --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 8 16 2014-05-04 1 8 16 2014-05-05 1 16 12 2014-06-04 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out index 7d3943e3764c5..b9cc68a339746 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 13 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,13 +60,13 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query WITH cte1 AS (SELECT t1a, t1b @@ -81,16 +81,16 @@ FROM t1 WHERE t1b IN (SELECT cte1.t1b FROM cte1 WHERE cte1.t1b > 0) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output val1a 16 12 10 2014-07-04 01:01:00 val1a 16 12 21 2014-06-04 01:02:00.001 val1a 6 8 10 2014-04-04 01:00:00 val1a 6 8 10 2014-04-04 01:02:00.001 --- !query 4 +-- !query WITH cte1 AS ( SELECT t1a, @@ -118,16 +118,16 @@ WHERE t1b IN FROM cte1 ) GROUP BY t1a, t1b, t1c HAVING t1c IS NOT NULL --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 16 12 1 6 8 1 8 16 1 8 16 --- !query 5 +-- !query WITH cte1 AS ( SELECT t1a, @@ -155,16 +155,16 @@ WHERE t1c IN ON cte1.t1b < cte5.t1b LEFT OUTER JOIN cte1 cte6 ON cte1.t1d > cte6.t1d) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output val1b 8 16 2014-05-04 01:01:00 val1c 8 16 2014-05-04 01:02:00.001 val1d NULL 16 2014-06-04 01:01:00 val1d NULL 16 2014-07-04 01:02:00.001 --- !query 6 +-- !query WITH cte1 AS (SELECT t1a, t1b @@ -186,13 +186,13 @@ FROM (SELECT * ON cte1.t1a = cte3.t1a INNER JOIN cte1 cte4 ON cte1.t1b = cte4.t1b) s --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output val1b 8 val1b 8 val1b 8 val1b 8 --- !query 7 +-- !query WITH cte1 AS ( SELECT t1a, @@ -217,13 +217,13 @@ WHERE t1b IN SELECT t1b FROM t1) GROUP BY t1b --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 2 8 --- !query 8 +-- !query WITH cte1 AS ( SELECT t1a, @@ -244,13 +244,13 @@ FROM ( RIGHT OUTER JOIN cte1 cte3 ON cte1.t1b = cte3.t1b LEFT OUTER JOIN cte1 cte4 ON cte1.t1c = cte4.t1c ) s --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output val1b 8 16 val1b 8 16 val1b 8 16 val1b 8 16 --- !query 9 +-- !query WITH cte1 AS (SELECT t1a, t1b @@ -266,13 +266,13 @@ FROM (SELECT cte1.t1a, RIGHT OUTER JOIN cte1 cte2 ON cte1.t1a = cte2.t1a) s GROUP BY s.t1b --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 2 8 --- !query 10 +-- !query WITH cte1 AS ( SELECT t1a, @@ -295,13 +295,13 @@ WHERE s.t1b IN FROM t1 INNER JOIN cte1 ON t1.t1a = cte1.t1a) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 8 --- !query 11 +-- !query WITH cte1 AS (SELECT t1a, t1b @@ -316,9 +316,9 @@ WHERE t1b NOT IN (SELECT cte1.t1b FROM cte1 WHERE cte1.t1b < 0) AND t1c > 10 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output val1a 16 12 2014-06-04 01:02:00.001 val1a 16 12 2014-07-04 01:01:00 val1b 8 16 2014-05-04 01:01:00 @@ -327,7 +327,7 @@ val1d NULL 16 2014-06-04 01:01:00 val1d NULL 16 2014-07-04 01:02:00.001 --- !query 12 +-- !query WITH cte1 AS ( SELECT t1a, @@ -357,8 +357,8 @@ WHERE t1b NOT IN JOIN cte1 cte4 ON cte1.t1c = cte4.t1c) AND t1c IS NOT NULL ORDER BY t1c DESC --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output val1b 8 16 19 2014-05-04 01:01:00 val1c 8 16 19 2014-05-04 01:02:00.001 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out index 6b86a9f6a0d00..720db9e8bdb15 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 8 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,27 +60,27 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT t1a, Avg(t1b) FROM t1 WHERE t1a NOT IN (SELECT t2a FROM t2) GROUP BY t1a --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output val1a 11.0 val1d 10.0 --- !query 4 +-- !query SELECT t1a, Sum(DISTINCT( t1b )) FROM t1 @@ -88,15 +88,15 @@ WHERE t1d NOT IN (SELECT t2d FROM t2 WHERE t1h < t2h) GROUP BY t1a --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output val1a 22 val1d 10 val1e 10 --- !query 5 +-- !query SELECT Count(*) FROM (SELECT * FROM t2 @@ -107,13 +107,13 @@ WHERE t2b NOT IN (SELECT Min(t2b) FROM t2 WHERE t2b = t2b GROUP BY t2c) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 4 --- !query 6 +-- !query SELECT t1a, max(t1b) FROM t1 @@ -122,16 +122,16 @@ WHERE t1c NOT IN (SELECT Max(t2b) WHERE t1a = t2a GROUP BY t2a) GROUP BY t1a --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output val1a 16 val1b 8 val1c 8 val1d 10 --- !query 7 +-- !query SELECT t1a, t1b FROM t1 @@ -141,9 +141,9 @@ WHERE t1c IN (SELECT t2b FROM t3 WHERE t3a = t2a GROUP BY t3b) order by t2a) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output val1a 16 val1a 16 val1a 6 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out index 498378bfa5c37..4872e3c953ff6 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 9 --- !query 0 +-- !query create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,13 +60,13 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT t1a, t1b, t1c, @@ -78,9 +78,9 @@ FROM t1 WHERE t1a NOT IN (SELECT t2a FROM t2) AND t1b = t3b --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output val1a 6 8 val3a 6 12 val1a 6 8 val3a 6 12 val1a 6 8 val3a 6 12 @@ -89,7 +89,7 @@ val1d 10 NULL val1b 10 12 val1d 10 NULL val1b 10 12 --- !query 4 +-- !query SELECT t1a, t1b, t1c, @@ -113,15 +113,15 @@ AND t1d = t2d GROUP BY t1a, t1b, t1c, t3a, t3b, t3c HAVING count(distinct(t3a)) >= 1 ORDER BY t1a, t3b --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output val1c 8 16 1 6 12 val1c 8 16 1 10 12 val1c 8 16 1 17 16 --- !query 5 +-- !query SELECT t1a, t1b, t1c, @@ -141,9 +141,9 @@ AND t1d NOT IN FROM t2 RIGHT JOIN t1 on t2e = t1e WHERE t1a = t2a) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output val1a 16 12 10 2014-07-04 01:01:00 val1a 16 12 21 2014-06-04 01:02:00.001 val1a 6 8 10 2014-04-04 01:00:00 @@ -153,7 +153,7 @@ val1d NULL 16 22 2014-06-04 01:01:00 val1e 10 NULL 25 2014-08-04 01:01:00 --- !query 6 +-- !query SELECT Count(DISTINCT( t1a )), t1b, t1c, @@ -170,9 +170,9 @@ HAVING t1d NOT IN (SELECT t2d FROM t2 WHERE t1d = t2d) ORDER BY t1b DESC, t1d ASC --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 16 12 10 1 16 12 21 1 10 NULL 12 @@ -180,7 +180,7 @@ struct 1 NULL 16 22 --- !query 7 +-- !query SELECT COUNT(DISTINCT(t1a)), t1b, t1c, @@ -195,13 +195,13 @@ GROUP BY t1b, t1c, t1d HAVING t1b < sum(t1c) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 6 8 10 --- !query 8 +-- !query SELECT COUNT(DISTINCT(t1a)), t1b, t1c, @@ -223,7 +223,7 @@ GROUP BY t1b, t1c, t1d HAVING t1b < sum(t1c) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 6 8 10 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out index 3045cff1de19b..bc9e6f842557e 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out @@ -2,47 +2,47 @@ -- Number of queries: 4 --- !query 0 +-- !query CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, null), (null, 1.0), (2, 3.0), (4, 5.0) AS m(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT * FROM m WHERE b = 1.0 -- Matches (null, 1.0) AND (a, b) NOT IN ((2, 3.0)) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output NULL 1.0 --- !query 2 +-- !query SELECT * FROM m WHERE b = 3.0 -- Matches (2, 3.0) AND (a, b) NOT IN ((2, 3.0)) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM m WHERE b = 5.0 -- Matches (4, 5.0) AND (a, b) NOT IN ((2, 3.0)) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 4 5.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out index 5ccea3846cd36..54d6da8d0da83 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out @@ -2,119 +2,119 @@ -- Number of queries: 9 --- !query 0 +-- !query CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, null), (null, 1.0), (2, 3.0), (4, 5.0) AS m(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW s AS SELECT * FROM VALUES (null, null), (0, 1.0), (2, 3.0), (4, null) AS s(c, d) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM m WHERE (a, b) NOT IN (SELECT * FROM s WHERE d > 5.0) -- Matches no rows --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 2 3.0 4 5.0 NULL 1.0 NULL NULL --- !query 3 +-- !query SELECT * FROM m WHERE (a, b) NOT IN (SELECT * FROM s WHERE c IS NULL AND d IS NULL) -- Matches only (null, null) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output --- !query 4 +-- !query SELECT * FROM m WHERE a IS NULL AND b IS NULL -- Matches only (null, null) AND (a, b) NOT IN (SELECT * FROM s WHERE c IS NOT NULL) -- Matches (0, 1.0), (2, 3.0), (4, null) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT * FROM m WHERE b = 1.0 -- Matches (null, 1.0) AND (a, b) NOT IN (SELECT * FROM s WHERE c IS NOT NULL) -- Matches (0, 1.0), (2, 3.0), (4, null) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT * FROM m WHERE b = 1.0 -- Matches (null, 1.0) AND (a, b) NOT IN (SELECT * FROM s WHERE c = 2) -- Matches (2, 3.0) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL 1.0 --- !query 7 +-- !query SELECT * FROM m WHERE b = 3.0 -- Matches (2, 3.0) AND (a, b) NOT IN (SELECT * FROM s WHERE c = 2) -- Matches (2, 3.0) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT * FROM m WHERE b = 5.0 -- Matches (4, 5.0) AND (a, b) NOT IN (SELECT * FROM s WHERE c = 2) -- Matches (2, 3.0) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 4 5.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out index 9190e9c1e7a4b..0fc9cf289155d 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out @@ -2,56 +2,56 @@ -- Number of queries: 5 --- !query 0 +-- !query CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, 1.0), (2, 3.0), (4, 5.0) AS m(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT * FROM m WHERE a NOT IN (null) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM m WHERE b = 1.0 -- Only matches (null, 1.0) AND a NOT IN (2) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) AND a NOT IN (2) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output --- !query 4 +-- !query SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) AND a NOT IN (6) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 2 3.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out index d7473e1ada151..ef40fd462f883 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out @@ -2,130 +2,130 @@ -- Number of queries: 10 --- !query 0 +-- !query CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES (null, 1.0), (2, 3.0), (4, 5.0) AS m(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW s AS SELECT * FROM VALUES (null, 1.0), (2, 3.0), (6, 7.0) AS s(c, d) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM m WHERE a NOT IN (SELECT c FROM s WHERE d > 10.0) -- (empty subquery) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 2 3.0 4 5.0 NULL 1.0 --- !query 3 +-- !query SELECT * FROM m WHERE a NOT IN (SELECT c FROM s WHERE d = 1.0) -- Only matches (null, 1.0) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output --- !query 4 +-- !query SELECT * FROM m WHERE b = 1.0 -- Only matches (null, 1.0) AND a NOT IN (SELECT c FROM s WHERE d = 3.0) -- Matches (2, 3.0) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) AND a NOT IN (SELECT c FROM s WHERE d = 3.0) -- Matches (2, 3.0) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) AND a NOT IN (SELECT c FROM s WHERE d = 7.0) -- Matches (6, 7.0) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 2 3.0 --- !query 7 +-- !query SELECT * FROM m WHERE a NOT IN (SELECT c FROM s WHERE d = b + 10) -- Matches no row --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 2 3.0 4 5.0 NULL 1.0 --- !query 8 +-- !query SELECT * FROM m WHERE b = 1.0 -- Only matches (null, 1.0) AND a NOT IN (SELECT c FROM s WHERE d = b + 10) -- Matches no row --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NULL 1.0 --- !query 9 +-- !query SELECT * FROM m WHERE b = 3.0 -- Only matches (2, 3.0) AND a NOT IN (SELECT c FROM s WHERE d = b + 10) -- Matches no row --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 2 3.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out index f717382835ab4..0661e1c9e4d96 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 14 --- !query 0 +-- !query create temporary view t1 as select * from values ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,20 +60,20 @@ create temporary view t3 as select * from values ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM t1 WHERE t1a IN (SELECT t2a FROM t2) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 t1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 t1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 @@ -81,35 +81,35 @@ t1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 t1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 --- !query 4 +-- !query SELECT * FROM t1 WHERE t1b IN (SELECT t2b FROM t2 WHERE t1a = t2a) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 --- !query 5 +-- !query SELECT t1a, t1b FROM t1 WHERE t1c IN (SELECT t2b FROM t2 WHERE t1a != t2a) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output t1a 16 t1a 16 t1a 6 t1a 6 --- !query 6 +-- !query SELECT t1a, t1b FROM t1 @@ -117,14 +117,14 @@ WHERE t1c IN (SELECT t2b FROM t2 WHERE t1a = t2a OR t1b > t2b) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output t1a 16 t1a 16 --- !query 7 +-- !query SELECT t1a, t1b FROM t1 @@ -133,14 +133,14 @@ WHERE t1c IN (SELECT t2b WHERE t2i IN (SELECT t3i FROM t3 WHERE t2c = t3c)) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output t1a 6 t1a 6 --- !query 8 +-- !query SELECT t1a, t1b FROM t1 @@ -150,23 +150,23 @@ WHERE t1c IN (SELECT t2b FROM t3 WHERE t2c = t3c AND t2b IS NOT NULL)) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output t1a 6 t1a 6 --- !query 9 +-- !query SELECT DISTINCT( t1a ), t1b, t1h FROM t1 WHERE t1a NOT IN (SELECT t2a FROM t2) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output t1a 16 2014-06-04 01:02:00.001 t1a 16 2014-07-04 01:01:00 t1a 6 2014-04-04 01:00:00 @@ -176,49 +176,49 @@ t1d NULL 2014-06-04 01:01:00 t1d NULL 2014-07-04 01:02:00.001 --- !query 10 +-- !query create temporary view a as select * from values (1, 1), (2, 1), (null, 1), (1, 3), (null, 3), (1, null), (null, 2) as a(a1, a2) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query create temporary view b as select * from values (1, 1, 2), (null, 3, 2), (1, null, 2), (1, 2, null) as b(b1, b2, b3) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query SELECT a1, a2 FROM a WHERE a1 NOT IN (SELECT b.b1 FROM b WHERE a.a2 = b.b2) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 NULL 2 1 --- !query 13 +-- !query SELECT a1, a2 FROM a WHERE a1 NOT IN (SELECT b.b1 FROM b WHERE a.a2 = b.b2 AND b.b3 > 1) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 NULL 2 1 NULL 2 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out index 7b47a6139f60a..1599634ff9efb 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -2,37 +2,37 @@ -- Number of queries: 8 --- !query 0 +-- !query CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1, 2, 3) AS t1(t1a, t1b, t1c) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1, 0, 1) AS t2(t2a, t2b, t2c) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (3, 1, 2) AS t3(t3a, t3b, t3c) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT t1a, t2b FROM t1, t2 WHERE t1b = t2c @@ -42,14 +42,14 @@ AND t2b = (SELECT max(avg) WHERE t2a = t1.t1b ) ) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and 't2.`t2b`' is not an aggregate function. Wrap '(avg(CAST(t2.`t2b` AS BIGINT)) AS `avg`)' in windowing function(s) or wrap 't2.`t2b`' in first() (or first_value) if you don't care which value you get.; --- !query 4 +-- !query SELECT * FROM t1 WHERE t1a IN (SELECT min(t2a) @@ -59,14 +59,14 @@ WHERE t1a IN (SELECT min(t2a) FROM t3 GROUP BY t3b HAVING t3b > t2b )) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException Resolved attribute(s) t2b#x missing from min(t2a)#x,t2c#x in operator !Filter t2c#x IN (list#x [t2b#x]).; --- !query 5 +-- !query SELECT t1a FROM t1 GROUP BY 1 @@ -74,14 +74,14 @@ HAVING EXISTS (SELECT t2a FROM t2 GROUP BY 1 HAVING t2a < min(t1a + t2a)) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException Found an aggregate expression in a correlated predicate that has both outer and local references, which is not supported yet. Aggregate expression: min((t1.`t1a` + t2.`t2a`)), Outer references: t1.`t1a`, Local references: t2.`t2a`.; --- !query 6 +-- !query SELECT t1a FROM t1 WHERE t1a IN (SELECT t2a @@ -90,23 +90,23 @@ WHERE t1a IN (SELECT t2a FROM t3 GROUP BY 1 HAVING min(t2a + t3a) > 1)) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException Found an aggregate expression in a correlated predicate that has both outer and local references, which is not supported yet. Aggregate expression: min((t2.`t2a` + t3.`t3a`)), Outer references: t2.`t2a`, Local references: t3.`t3a`.; --- !query 7 +-- !query SELECT t1a FROM t1 WHERE t1a IN (SELECT t2a FROM t2 WHERE EXISTS (SELECT min(t2a) FROM t3)) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.AnalysisException Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses: Aggregate [min(outer(t2a#x)) AS min(outer())#x] diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out index ea7585716de83..776598127075b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -2,57 +2,57 @@ -- Number of queries: 10 --- !query 0 +-- !query CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1, 2, 3) AS t1(t1a, t1b, t1c) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1, 0, 1) AS t2(t2a, t2b, t2c) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (3, 1, 2) AS t3(t3a, t3b, t3c) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (CAST(1 AS DOUBLE), CAST(2 AS STRING), CAST(3 AS STRING)) AS t1(t4a, t4b, t4c) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE TEMPORARY VIEW t5 AS SELECT * FROM VALUES (CAST('2011-01-01 01:01:01' AS TIMESTAMP), CAST(2 AS STRING), CAST(3 AS BIGINT)) AS t1(t5a, t5b, t5c) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT ( SELECT max(t2b), min(t2b) FROM t2 @@ -60,14 +60,14 @@ SELECT GROUP BY t2.t2b ) FROM t1 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException Scalar subquery must return only one column, but got 2; --- !query 6 +-- !query SELECT ( SELECT max(t2b), min(t2b) FROM t2 @@ -75,22 +75,22 @@ SELECT GROUP BY t2.t2b ) FROM t1 --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException Scalar subquery must return only one column, but got 2; --- !query 7 +-- !query SELECT * FROM t1 WHERE t1a IN (SELECT t2a, t2b FROM t2 WHERE t1a = t2a) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(t1.`t1a` IN (listquery(t1.`t1a`)))' due to data type mismatch: The number of columns in the left hand side of an IN subquery does not match the @@ -103,15 +103,15 @@ Right side columns: [t2.`t2a`, t2.`t2b`].; --- !query 8 +-- !query SELECT * FROM T1 WHERE (t1a, t1b) IN (SELECT t2a FROM t2 WHERE t1a = t2a) --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(named_struct('t1a', t1.`t1a`, 't1b', t1.`t1b`) IN (listquery(t1.`t1a`)))' due to data type mismatch: The number of columns in the left hand side of an IN subquery does not match the @@ -124,16 +124,16 @@ Right side columns: [t2.`t2a`].; --- !query 9 +-- !query SELECT * FROM t4 WHERE (t4a, t4b, t4c) IN (SELECT t5a, t5b, t5c FROM t5) --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(named_struct('t4a', t4.`t4a`, 't4b', t4.`t4b`, 't4c', t4.`t4c`) IN (listquery()))' due to data type mismatch: The data type of one or more elements in the left hand side of an IN subquery diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 399660d195066..b7eef929864fc 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -2,36 +2,36 @@ -- Number of queries: 27 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW p AS VALUES (1, 1) AS T(pk, pv) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW c AS VALUES (1, 1) AS T(ck, cv) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT pk, cv FROM p, c WHERE p.pk = c.ck AND c.cv = (SELECT avg(c1.cv) FROM c c1 WHERE c1.ck = p.pk) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 1 --- !query 3 +-- !query SELECT pk, cv FROM p, c WHERE p.pk = c.ck @@ -40,13 +40,13 @@ AND c.cv = (SELECT max(avg) FROM c c1 WHERE c1.ck = p.pk GROUP BY c1.cv)) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 1 --- !query 4 +-- !query create temporary view t1 as select * from values ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -61,13 +61,13 @@ create temporary view t1 as select * from values ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query create temporary view t2 as select * from values ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -83,13 +83,13 @@ create temporary view t2 as select * from values ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query create temporary view t3 as select * from values ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -104,41 +104,41 @@ create temporary view t3 as select * from values ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query SELECT t1a, t1b FROM t1 WHERE t1c = (SELECT max(t2c) FROM t2) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output val1b 8 val1c 8 val1d NULL val1d NULL --- !query 8 +-- !query SELECT t1a, t1d, t1f FROM t1 WHERE t1c = (SELECT max(t2c) FROM t2) AND t1b > (SELECT min(t3b) FROM t3) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output val1b 19 25.0 val1c 19 25.0 --- !query 9 +-- !query SELECT t1a, t1h FROM t1 WHERE t1c = (SELECT max(t2c) @@ -146,69 +146,69 @@ WHERE t1c = (SELECT max(t2c) OR t1b = (SELECT min(t3b) FROM t3 WHERE t3b > 10) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output val1b 2014-05-04 01:01:00 val1c 2014-05-04 01:02:00.001 val1d 2014-06-04 01:01:00 val1d 2014-07-04 01:02:00.001 --- !query 10 +-- !query SELECT t1a, t1b, t2d FROM t1 LEFT JOIN t2 ON t1a = t2a WHERE t1b = (SELECT min(t3b) FROM t3) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output val1a 6 NULL val1a 6 NULL --- !query 11 +-- !query SELECT t1a, t1b, t1g FROM t1 WHERE t1c + 5 = (SELECT max(t2e) FROM t2) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output val1a 16 2000 val1a 16 2000 --- !query 12 +-- !query SELECT t1a, t1h FROM t1 WHERE date(t1h) = (SELECT min(t2i) FROM t2) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output val1a 2014-04-04 00:00:00 val1a 2014-04-04 01:02:00.001 --- !query 13 +-- !query SELECT t2d, t1a FROM t1, t2 WHERE t1b = t2b AND t2c + 1 = (SELECT max(t2c) + 1 FROM t2, t1 WHERE t2b = t1b) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 119 val1b 119 val1c 19 val1b 19 val1c --- !query 14 +-- !query SELECT DISTINCT t2a, max_t1g FROM t2, (SELECT max(t1g) max_t1g, t1a FROM t1 @@ -216,15 +216,15 @@ FROM t2, (SELECT max(t1g) max_t1g, t1a WHERE t2a = t1a AND max_t1g = (SELECT max(t1g) FROM t1) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output val1b 2600 val1c 2600 val1e 2600 --- !query 15 +-- !query SELECT t3b, t3c FROM t3 WHERE (SELECT max(t3c) @@ -234,40 +234,40 @@ WHERE (SELECT max(t3c) FROM t3 WHERE t3c > 0) AND (t3b is null or t3c is null) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 8 NULL 8 NULL NULL 16 NULL 16 --- !query 16 +-- !query SELECT t1a FROM t1 WHERE t1a < (SELECT max(t2a) FROM t2 WHERE t2c = t1c GROUP BY t2c) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output val1a val1a val1b --- !query 17 +-- !query SELECT t1a, t1c FROM t1 WHERE (SELECT max(t2a) FROM t2 WHERE t2c = t1c GROUP BY t2c) IS NULL --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output val1a 8 val1a 8 val1d NULL @@ -276,7 +276,7 @@ val1e NULL val1e NULL --- !query 18 +-- !query SELECT t1a FROM t1 WHERE t1a = (SELECT max(t2a) @@ -285,14 +285,14 @@ WHERE t1a = (SELECT max(t2a) GROUP BY t2c HAVING count(*) >= 0) OR t1i > '2014-12-31' --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output val1c val1d --- !query 19 +-- !query SELECT t1a FROM t1 WHERE t1a = (SELECT max(t2a) @@ -301,14 +301,14 @@ WHERE t1a = (SELECT max(t2a) GROUP BY t2c HAVING count(*) >= 1) OR t1i > '2014-12-31' --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output val1c val1d --- !query 20 +-- !query SELECT count(t1a) FROM t1 RIGHT JOIN t2 ON t1d = t2d @@ -316,13 +316,13 @@ WHERE t1a < (SELECT max(t2a) FROM t2 WHERE t2c = t1c GROUP BY t2c) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 7 --- !query 21 +-- !query SELECT t1a FROM t1 WHERE t1b <= (SELECT max(t2b) @@ -333,14 +333,14 @@ AND t1b >= (SELECT min(t2b) FROM t2 WHERE t2c = t1c GROUP BY t2c) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output val1b val1c --- !query 22 +-- !query SELECT t1a FROM t1 WHERE t1a <= (SELECT max(t2a) @@ -354,14 +354,14 @@ WHERE t1a >= (SELECT min(t2a) FROM t2 WHERE t2c = t1c GROUP BY t2c) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output val1b val1c --- !query 23 +-- !query SELECT t1a FROM t1 WHERE t1a <= (SELECT max(t2a) @@ -375,9 +375,9 @@ WHERE t1a >= (SELECT min(t2a) FROM t2 WHERE t2c = t1c GROUP BY t2c) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output val1a val1a val1b @@ -388,7 +388,7 @@ val1d val1d --- !query 24 +-- !query SELECT t1a FROM t1 WHERE t1a <= (SELECT max(t2a) @@ -402,16 +402,16 @@ WHERE t1a >= (SELECT min(t2a) FROM t2 WHERE t2c = t1c GROUP BY t2c) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output val1a val1b val1c val1d --- !query 25 +-- !query SELECT t1a FROM t1 WHERE t1a <= (SELECT max(t2a) @@ -425,13 +425,13 @@ WHERE t1a >= (SELECT min(t2a) FROM t2 WHERE t2c = t1c GROUP BY t2c) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output val1a --- !query 26 +-- !query SELECT t1a FROM t1 GROUP BY t1a, t1c @@ -439,8 +439,8 @@ HAVING max(t1b) <= (SELECT max(t2b) FROM t2 WHERE t2c = t1c GROUP BY t2c) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output val1b val1c diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 807bb47221885..184b8daf9d28e 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 11 --- !query 0 +-- !query create temporary view t1 as select * from values ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -17,13 +17,13 @@ create temporary view t1 as select * from values ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -39,13 +39,13 @@ create temporary view t2 as select * from values ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view t3 as select * from values ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -60,24 +60,24 @@ create temporary view t3 as select * from values ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT (SELECT min(t3d) FROM t3) min_t3d, (SELECT max(t2h) FROM t2) max_t2h FROM t1 WHERE t1a = 'val1c' --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 10 2017-05-04 01:01:00 --- !query 4 +-- !query SELECT t1a, count(*) FROM t1 WHERE t1c IN (SELECT (SELECT min(t3c) FROM t3) @@ -85,13 +85,13 @@ WHERE t1c IN (SELECT (SELECT min(t3c) FROM t3) GROUP BY t2g HAVING count(*) > 1) GROUP BY t1a --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output val1a 2 --- !query 5 +-- !query SELECT (SELECT min(t3d) FROM t3) min_t3d, null FROM t1 @@ -101,14 +101,14 @@ SELECT null, (SELECT max(t2h) FROM t2) max_t2h FROM t1 WHERE t1a = 'val1c' --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 10 NULL NULL 2017-05-04 01:01:00 --- !query 6 +-- !query SELECT (SELECT min(t3c) FROM t3) min_t3d FROM t1 WHERE t1a = 'val1a' @@ -116,13 +116,13 @@ INTERSECT SELECT (SELECT min(t2c) FROM t2) min_t2d FROM t1 WHERE t1a = 'val1d' --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 12 --- !query 7 +-- !query SELECT q1.t1a, q2.t2a, q1.min_t3d, q2.avg_t3d FROM (SELECT t1a, (SELECT min(t3d) FROM t3) min_t3d FROM t1 @@ -133,9 +133,9 @@ FROM (SELECT t1a, (SELECT min(t3d) FROM t3) min_t3d WHERE t2a IN ('val1c', 'val2a')) q2 ON q1.t1a = q2.t2a AND q1.min_t3d < q2.avg_t3d --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL val2a NULL 200.83333333333334 val1c val1c 10 200.83333333333334 val1c val1c 10 200.83333333333334 @@ -144,18 +144,18 @@ val1e NULL 10 NULL val1e NULL 10 NULL --- !query 8 +-- !query SELECT (SELECT min(t3d) FROM t3 WHERE t3.t3a = t1.t1a) min_t3d, (SELECT max(t2h) FROM t2 WHERE t2.t2a = t1.t1a) max_t2h FROM t1 WHERE t1a = 'val1b' --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 19 2017-05-04 01:01:00 --- !query 9 +-- !query SELECT (SELECT min(t3d) FROM t3 WHERE t3a = t1a) min_t3d FROM t1 WHERE t1a = 'val1b' @@ -163,13 +163,13 @@ MINUS SELECT (SELECT min(t3d) FROM t3) abs_min_t3d FROM t1 WHERE t1a = 'val1b' --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 19 --- !query 10 +-- !query SELECT t1a, t1b FROM t1 WHERE NOT EXISTS (SELECT (SELECT max(t2b) @@ -182,9 +182,9 @@ WHERE NOT EXISTS (SELECT (SELECT max(t2b) ON t2a = t1a WHERE t2c = t3c) AND t3a = t1a) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output val1a 16 val1a 16 val1a 6 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/subquery-in-from.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/subquery-in-from.sql.out index 50370df349168..11a51dca25341 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/subquery-in-from.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/subquery-in-from.sql.out @@ -2,49 +2,49 @@ -- Number of queries: 6 --- !query 0 +-- !query SELECT * FROM (SELECT * FROM testData) AS t WHERE key = 1 --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1 1 --- !query 1 +-- !query FROM (SELECT * FROM testData WHERE key = 1) AS t SELECT * --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 1 --- !query 2 +-- !query SELECT * FROM (SELECT * FROM testData) t WHERE key = 1 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 1 --- !query 3 +-- !query FROM (SELECT * FROM testData WHERE key = 1) t SELECT * --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 1 --- !query 4 +-- !query SELECT * FROM (SELECT * FROM testData) WHERE key = 1 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 1 --- !query 5 +-- !query FROM (SELECT * FROM testData WHERE key = 1) SELECT * --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 1 diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out index 653dc1ca357f8..25967a3968f23 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -2,96 +2,96 @@ -- Number of queries: 11 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1) AS testData(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT * FROM testData AS t(col1, col2) WHERE col1 = 1 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 1 1 2 --- !query 2 +-- !query SELECT * FROM testData AS t(col1, col2) WHERE col1 = 2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 2 1 --- !query 3 +-- !query SELECT col1 AS k, SUM(col2) FROM testData AS t(col1, col2) GROUP BY k --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 3 2 1 --- !query 4 +-- !query SELECT * FROM testData AS t(col1, col2, col3) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException Number of column aliases does not match number of columns. Number of column aliases: 3; number of columns: 2.; line 1 pos 14 --- !query 5 +-- !query SELECT * FROM testData AS t(col1) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException Number of column aliases does not match number of columns. Number of column aliases: 1; number of columns: 2.; line 1 pos 14 --- !query 6 +-- !query SELECT a AS col1, b AS col2 FROM testData AS t(c, d) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`a`' given input columns: [c, d]; line 1 pos 7 --- !query 7 +-- !query SELECT * FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 1 --- !query 8 +-- !query CREATE OR REPLACE TEMPORARY VIEW src1 AS SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1) --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2) --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 a 1 8.5 2 b 2 1.0 3 c 3 3.2 diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out index fdbea0ee90720..16d483df62fd5 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -2,20 +2,20 @@ -- Number of queries: 8 --- !query 0 +-- !query select * from dummy(3) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output org.apache.spark.sql.AnalysisException could not resolve `dummy` to a table-valued function; line 1 pos 14 --- !query 1 +-- !query select * from range(6 + cos(3)) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 0 1 2 @@ -23,11 +23,11 @@ struct 4 --- !query 2 +-- !query select * from range(5, 10) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 5 6 7 @@ -35,11 +35,11 @@ struct 9 --- !query 3 +-- !query select * from range(0, 10, 2) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 0 2 4 @@ -47,11 +47,11 @@ struct 8 --- !query 4 +-- !query select * from range(0, 10, 1, 200) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 0 1 2 @@ -64,11 +64,11 @@ struct 9 --- !query 5 +-- !query select * from range(1, 1, 1, 1, 1) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException error: table-valued function range with alternatives: (end: long) @@ -78,11 +78,11 @@ error: table-valued function range with alternatives: cannot be applied to: (integer, integer, integer, integer, integer); line 1 pos 14 --- !query 6 +-- !query select * from range(1, null) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException error: table-valued function range with alternatives: (end: long) @@ -92,10 +92,10 @@ error: table-valued function range with alternatives: cannot be applied to: (integer, null); line 1 pos 14 --- !query 7 +-- !query select * from RaNgE(2) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 0 1 diff --git a/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out index 35f3931736b83..0188cdd0f8e71 100644 --- a/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out @@ -2,35 +2,35 @@ -- Number of queries: 6 --- !query 0 +-- !query CREATE DATABASE mydb1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query USE mydb1 --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Sampling fraction (-0.01) must be on interval [0, 1](line 1, pos 24) @@ -40,11 +40,11 @@ SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT) ------------------------^^^ --- !query 4 +-- !query SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Sampling fraction (1.01) must be on interval [0, 1](line 1, pos 24) @@ -54,9 +54,9 @@ SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT) ------------------------^^^ --- !query 5 +-- !query DROP DATABASE mydb1 CASCADE --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out index 4e4f592a137fb..b49e6b5f21b16 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out @@ -2,89 +2,89 @@ -- Number of queries: 11 --- !query 0 +-- !query SELECT array_join(array(true, false), ', ') --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output true, false --- !query 1 +-- !query SELECT array_join(array(2Y, 1Y), ', ') --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 2, 1 --- !query 2 +-- !query SELECT array_join(array(2S, 1S), ', ') --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 2, 1 --- !query 3 +-- !query SELECT array_join(array(2, 1), ', ') --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 2, 1 --- !query 4 +-- !query SELECT array_join(array(2L, 1L), ', ') --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 2, 1 --- !query 5 +-- !query SELECT array_join(array(9223372036854775809, 9223372036854775808), ', ') --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 9223372036854775809, 9223372036854775808 --- !query 6 +-- !query SELECT array_join(array(2.0D, 1.0D), ', ') --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 2.0, 1.0 --- !query 7 +-- !query SELECT array_join(array(float(2.0), float(1.0)), ', ') --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 2.0, 1.0 --- !query 8 +-- !query SELECT array_join(array(date '2016-03-14', date '2016-03-13'), ', ') --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 2016-03-14, 2016-03-13 --- !query 9 +-- !query SELECT array_join(array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), ', ') --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 2016-11-15 20:54:00, 2016-11-12 20:54:00 --- !query 10 +-- !query SELECT array_join(array('a', 'b'), ', ') --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output a, b diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out index 55caab8528fa9..0e1a3d0bc4d9e 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out @@ -2,2121 +2,2121 @@ -- Number of queries: 265 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT cast(1 as binary) = '1' FROM t --- !query 1 schema +-- !query schema struct<(CAST(1 AS BINARY) = CAST(1 AS BINARY)):boolean> --- !query 1 output +-- !query output false --- !query 2 +-- !query SELECT cast(1 as binary) > '2' FROM t --- !query 2 schema +-- !query schema struct<(CAST(1 AS BINARY) > CAST(2 AS BINARY)):boolean> --- !query 2 output +-- !query output false --- !query 3 +-- !query SELECT cast(1 as binary) >= '2' FROM t --- !query 3 schema +-- !query schema struct<(CAST(1 AS BINARY) >= CAST(2 AS BINARY)):boolean> --- !query 3 output +-- !query output false --- !query 4 +-- !query SELECT cast(1 as binary) < '2' FROM t --- !query 4 schema +-- !query schema struct<(CAST(1 AS BINARY) < CAST(2 AS BINARY)):boolean> --- !query 4 output +-- !query output true --- !query 5 +-- !query SELECT cast(1 as binary) <= '2' FROM t --- !query 5 schema +-- !query schema struct<(CAST(1 AS BINARY) <= CAST(2 AS BINARY)):boolean> --- !query 5 output +-- !query output true --- !query 6 +-- !query SELECT cast(1 as binary) <> '2' FROM t --- !query 6 schema +-- !query schema struct<(NOT (CAST(1 AS BINARY) = CAST(2 AS BINARY))):boolean> --- !query 6 output +-- !query output true --- !query 7 +-- !query SELECT cast(1 as binary) = cast(null as string) FROM t --- !query 7 schema +-- !query schema struct<(CAST(1 AS BINARY) = CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 7 output +-- !query output NULL --- !query 8 +-- !query SELECT cast(1 as binary) > cast(null as string) FROM t --- !query 8 schema +-- !query schema struct<(CAST(1 AS BINARY) > CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 8 output +-- !query output NULL --- !query 9 +-- !query SELECT cast(1 as binary) >= cast(null as string) FROM t --- !query 9 schema +-- !query schema struct<(CAST(1 AS BINARY) >= CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 9 output +-- !query output NULL --- !query 10 +-- !query SELECT cast(1 as binary) < cast(null as string) FROM t --- !query 10 schema +-- !query schema struct<(CAST(1 AS BINARY) < CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 10 output +-- !query output NULL --- !query 11 +-- !query SELECT cast(1 as binary) <= cast(null as string) FROM t --- !query 11 schema +-- !query schema struct<(CAST(1 AS BINARY) <= CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 11 output +-- !query output NULL --- !query 12 +-- !query SELECT cast(1 as binary) <> cast(null as string) FROM t --- !query 12 schema +-- !query schema struct<(NOT (CAST(1 AS BINARY) = CAST(CAST(NULL AS STRING) AS BINARY))):boolean> --- !query 12 output +-- !query output NULL --- !query 13 +-- !query SELECT '1' = cast(1 as binary) FROM t --- !query 13 schema +-- !query schema struct<(CAST(1 AS BINARY) = CAST(1 AS BINARY)):boolean> --- !query 13 output +-- !query output false --- !query 14 +-- !query SELECT '2' > cast(1 as binary) FROM t --- !query 14 schema +-- !query schema struct<(CAST(2 AS BINARY) > CAST(1 AS BINARY)):boolean> --- !query 14 output +-- !query output true --- !query 15 +-- !query SELECT '2' >= cast(1 as binary) FROM t --- !query 15 schema +-- !query schema struct<(CAST(2 AS BINARY) >= CAST(1 AS BINARY)):boolean> --- !query 15 output +-- !query output true --- !query 16 +-- !query SELECT '2' < cast(1 as binary) FROM t --- !query 16 schema +-- !query schema struct<(CAST(2 AS BINARY) < CAST(1 AS BINARY)):boolean> --- !query 16 output +-- !query output false --- !query 17 +-- !query SELECT '2' <= cast(1 as binary) FROM t --- !query 17 schema +-- !query schema struct<(CAST(2 AS BINARY) <= CAST(1 AS BINARY)):boolean> --- !query 17 output +-- !query output false --- !query 18 +-- !query SELECT '2' <> cast(1 as binary) FROM t --- !query 18 schema +-- !query schema struct<(NOT (CAST(2 AS BINARY) = CAST(1 AS BINARY))):boolean> --- !query 18 output +-- !query output true --- !query 19 +-- !query SELECT cast(null as string) = cast(1 as binary) FROM t --- !query 19 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) = CAST(1 AS BINARY)):boolean> --- !query 19 output +-- !query output NULL --- !query 20 +-- !query SELECT cast(null as string) > cast(1 as binary) FROM t --- !query 20 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) > CAST(1 AS BINARY)):boolean> --- !query 20 output +-- !query output NULL --- !query 21 +-- !query SELECT cast(null as string) >= cast(1 as binary) FROM t --- !query 21 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) >= CAST(1 AS BINARY)):boolean> --- !query 21 output +-- !query output NULL --- !query 22 +-- !query SELECT cast(null as string) < cast(1 as binary) FROM t --- !query 22 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) < CAST(1 AS BINARY)):boolean> --- !query 22 output +-- !query output NULL --- !query 23 +-- !query SELECT cast(null as string) <= cast(1 as binary) FROM t --- !query 23 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) <= CAST(1 AS BINARY)):boolean> --- !query 23 output +-- !query output NULL --- !query 24 +-- !query SELECT cast(null as string) <> cast(1 as binary) FROM t --- !query 24 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS BINARY) = CAST(1 AS BINARY))):boolean> --- !query 24 output +-- !query output NULL --- !query 25 +-- !query SELECT cast(1 as tinyint) = '1' FROM t --- !query 25 schema +-- !query schema struct<(CAST(1 AS TINYINT) = CAST(1 AS TINYINT)):boolean> --- !query 25 output +-- !query output true --- !query 26 +-- !query SELECT cast(1 as tinyint) > '2' FROM t --- !query 26 schema +-- !query schema struct<(CAST(1 AS TINYINT) > CAST(2 AS TINYINT)):boolean> --- !query 26 output +-- !query output false --- !query 27 +-- !query SELECT cast(1 as tinyint) >= '2' FROM t --- !query 27 schema +-- !query schema struct<(CAST(1 AS TINYINT) >= CAST(2 AS TINYINT)):boolean> --- !query 27 output +-- !query output false --- !query 28 +-- !query SELECT cast(1 as tinyint) < '2' FROM t --- !query 28 schema +-- !query schema struct<(CAST(1 AS TINYINT) < CAST(2 AS TINYINT)):boolean> --- !query 28 output +-- !query output true --- !query 29 +-- !query SELECT cast(1 as tinyint) <= '2' FROM t --- !query 29 schema +-- !query schema struct<(CAST(1 AS TINYINT) <= CAST(2 AS TINYINT)):boolean> --- !query 29 output +-- !query output true --- !query 30 +-- !query SELECT cast(1 as tinyint) <> '2' FROM t --- !query 30 schema +-- !query schema struct<(NOT (CAST(1 AS TINYINT) = CAST(2 AS TINYINT))):boolean> --- !query 30 output +-- !query output true --- !query 31 +-- !query SELECT cast(1 as tinyint) = cast(null as string) FROM t --- !query 31 schema +-- !query schema struct<(CAST(1 AS TINYINT) = CAST(CAST(NULL AS STRING) AS TINYINT)):boolean> --- !query 31 output +-- !query output NULL --- !query 32 +-- !query SELECT cast(1 as tinyint) > cast(null as string) FROM t --- !query 32 schema +-- !query schema struct<(CAST(1 AS TINYINT) > CAST(CAST(NULL AS STRING) AS TINYINT)):boolean> --- !query 32 output +-- !query output NULL --- !query 33 +-- !query SELECT cast(1 as tinyint) >= cast(null as string) FROM t --- !query 33 schema +-- !query schema struct<(CAST(1 AS TINYINT) >= CAST(CAST(NULL AS STRING) AS TINYINT)):boolean> --- !query 33 output +-- !query output NULL --- !query 34 +-- !query SELECT cast(1 as tinyint) < cast(null as string) FROM t --- !query 34 schema +-- !query schema struct<(CAST(1 AS TINYINT) < CAST(CAST(NULL AS STRING) AS TINYINT)):boolean> --- !query 34 output +-- !query output NULL --- !query 35 +-- !query SELECT cast(1 as tinyint) <= cast(null as string) FROM t --- !query 35 schema +-- !query schema struct<(CAST(1 AS TINYINT) <= CAST(CAST(NULL AS STRING) AS TINYINT)):boolean> --- !query 35 output +-- !query output NULL --- !query 36 +-- !query SELECT cast(1 as tinyint) <> cast(null as string) FROM t --- !query 36 schema +-- !query schema struct<(NOT (CAST(1 AS TINYINT) = CAST(CAST(NULL AS STRING) AS TINYINT))):boolean> --- !query 36 output +-- !query output NULL --- !query 37 +-- !query SELECT '1' = cast(1 as tinyint) FROM t --- !query 37 schema +-- !query schema struct<(CAST(1 AS TINYINT) = CAST(1 AS TINYINT)):boolean> --- !query 37 output +-- !query output true --- !query 38 +-- !query SELECT '2' > cast(1 as tinyint) FROM t --- !query 38 schema +-- !query schema struct<(CAST(2 AS TINYINT) > CAST(1 AS TINYINT)):boolean> --- !query 38 output +-- !query output true --- !query 39 +-- !query SELECT '2' >= cast(1 as tinyint) FROM t --- !query 39 schema +-- !query schema struct<(CAST(2 AS TINYINT) >= CAST(1 AS TINYINT)):boolean> --- !query 39 output +-- !query output true --- !query 40 +-- !query SELECT '2' < cast(1 as tinyint) FROM t --- !query 40 schema +-- !query schema struct<(CAST(2 AS TINYINT) < CAST(1 AS TINYINT)):boolean> --- !query 40 output +-- !query output false --- !query 41 +-- !query SELECT '2' <= cast(1 as tinyint) FROM t --- !query 41 schema +-- !query schema struct<(CAST(2 AS TINYINT) <= CAST(1 AS TINYINT)):boolean> --- !query 41 output +-- !query output false --- !query 42 +-- !query SELECT '2' <> cast(1 as tinyint) FROM t --- !query 42 schema +-- !query schema struct<(NOT (CAST(2 AS TINYINT) = CAST(1 AS TINYINT))):boolean> --- !query 42 output +-- !query output true --- !query 43 +-- !query SELECT cast(null as string) = cast(1 as tinyint) FROM t --- !query 43 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TINYINT) = CAST(1 AS TINYINT)):boolean> --- !query 43 output +-- !query output NULL --- !query 44 +-- !query SELECT cast(null as string) > cast(1 as tinyint) FROM t --- !query 44 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TINYINT) > CAST(1 AS TINYINT)):boolean> --- !query 44 output +-- !query output NULL --- !query 45 +-- !query SELECT cast(null as string) >= cast(1 as tinyint) FROM t --- !query 45 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TINYINT) >= CAST(1 AS TINYINT)):boolean> --- !query 45 output +-- !query output NULL --- !query 46 +-- !query SELECT cast(null as string) < cast(1 as tinyint) FROM t --- !query 46 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TINYINT) < CAST(1 AS TINYINT)):boolean> --- !query 46 output +-- !query output NULL --- !query 47 +-- !query SELECT cast(null as string) <= cast(1 as tinyint) FROM t --- !query 47 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TINYINT) <= CAST(1 AS TINYINT)):boolean> --- !query 47 output +-- !query output NULL --- !query 48 +-- !query SELECT cast(null as string) <> cast(1 as tinyint) FROM t --- !query 48 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS TINYINT) = CAST(1 AS TINYINT))):boolean> --- !query 48 output +-- !query output NULL --- !query 49 +-- !query SELECT cast(1 as smallint) = '1' FROM t --- !query 49 schema +-- !query schema struct<(CAST(1 AS SMALLINT) = CAST(1 AS SMALLINT)):boolean> --- !query 49 output +-- !query output true --- !query 50 +-- !query SELECT cast(1 as smallint) > '2' FROM t --- !query 50 schema +-- !query schema struct<(CAST(1 AS SMALLINT) > CAST(2 AS SMALLINT)):boolean> --- !query 50 output +-- !query output false --- !query 51 +-- !query SELECT cast(1 as smallint) >= '2' FROM t --- !query 51 schema +-- !query schema struct<(CAST(1 AS SMALLINT) >= CAST(2 AS SMALLINT)):boolean> --- !query 51 output +-- !query output false --- !query 52 +-- !query SELECT cast(1 as smallint) < '2' FROM t --- !query 52 schema +-- !query schema struct<(CAST(1 AS SMALLINT) < CAST(2 AS SMALLINT)):boolean> --- !query 52 output +-- !query output true --- !query 53 +-- !query SELECT cast(1 as smallint) <= '2' FROM t --- !query 53 schema +-- !query schema struct<(CAST(1 AS SMALLINT) <= CAST(2 AS SMALLINT)):boolean> --- !query 53 output +-- !query output true --- !query 54 +-- !query SELECT cast(1 as smallint) <> '2' FROM t --- !query 54 schema +-- !query schema struct<(NOT (CAST(1 AS SMALLINT) = CAST(2 AS SMALLINT))):boolean> --- !query 54 output +-- !query output true --- !query 55 +-- !query SELECT cast(1 as smallint) = cast(null as string) FROM t --- !query 55 schema +-- !query schema struct<(CAST(1 AS SMALLINT) = CAST(CAST(NULL AS STRING) AS SMALLINT)):boolean> --- !query 55 output +-- !query output NULL --- !query 56 +-- !query SELECT cast(1 as smallint) > cast(null as string) FROM t --- !query 56 schema +-- !query schema struct<(CAST(1 AS SMALLINT) > CAST(CAST(NULL AS STRING) AS SMALLINT)):boolean> --- !query 56 output +-- !query output NULL --- !query 57 +-- !query SELECT cast(1 as smallint) >= cast(null as string) FROM t --- !query 57 schema +-- !query schema struct<(CAST(1 AS SMALLINT) >= CAST(CAST(NULL AS STRING) AS SMALLINT)):boolean> --- !query 57 output +-- !query output NULL --- !query 58 +-- !query SELECT cast(1 as smallint) < cast(null as string) FROM t --- !query 58 schema +-- !query schema struct<(CAST(1 AS SMALLINT) < CAST(CAST(NULL AS STRING) AS SMALLINT)):boolean> --- !query 58 output +-- !query output NULL --- !query 59 +-- !query SELECT cast(1 as smallint) <= cast(null as string) FROM t --- !query 59 schema +-- !query schema struct<(CAST(1 AS SMALLINT) <= CAST(CAST(NULL AS STRING) AS SMALLINT)):boolean> --- !query 59 output +-- !query output NULL --- !query 60 +-- !query SELECT cast(1 as smallint) <> cast(null as string) FROM t --- !query 60 schema +-- !query schema struct<(NOT (CAST(1 AS SMALLINT) = CAST(CAST(NULL AS STRING) AS SMALLINT))):boolean> --- !query 60 output +-- !query output NULL --- !query 61 +-- !query SELECT '1' = cast(1 as smallint) FROM t --- !query 61 schema +-- !query schema struct<(CAST(1 AS SMALLINT) = CAST(1 AS SMALLINT)):boolean> --- !query 61 output +-- !query output true --- !query 62 +-- !query SELECT '2' > cast(1 as smallint) FROM t --- !query 62 schema +-- !query schema struct<(CAST(2 AS SMALLINT) > CAST(1 AS SMALLINT)):boolean> --- !query 62 output +-- !query output true --- !query 63 +-- !query SELECT '2' >= cast(1 as smallint) FROM t --- !query 63 schema +-- !query schema struct<(CAST(2 AS SMALLINT) >= CAST(1 AS SMALLINT)):boolean> --- !query 63 output +-- !query output true --- !query 64 +-- !query SELECT '2' < cast(1 as smallint) FROM t --- !query 64 schema +-- !query schema struct<(CAST(2 AS SMALLINT) < CAST(1 AS SMALLINT)):boolean> --- !query 64 output +-- !query output false --- !query 65 +-- !query SELECT '2' <= cast(1 as smallint) FROM t --- !query 65 schema +-- !query schema struct<(CAST(2 AS SMALLINT) <= CAST(1 AS SMALLINT)):boolean> --- !query 65 output +-- !query output false --- !query 66 +-- !query SELECT '2' <> cast(1 as smallint) FROM t --- !query 66 schema +-- !query schema struct<(NOT (CAST(2 AS SMALLINT) = CAST(1 AS SMALLINT))):boolean> --- !query 66 output +-- !query output true --- !query 67 +-- !query SELECT cast(null as string) = cast(1 as smallint) FROM t --- !query 67 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS SMALLINT) = CAST(1 AS SMALLINT)):boolean> --- !query 67 output +-- !query output NULL --- !query 68 +-- !query SELECT cast(null as string) > cast(1 as smallint) FROM t --- !query 68 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS SMALLINT) > CAST(1 AS SMALLINT)):boolean> --- !query 68 output +-- !query output NULL --- !query 69 +-- !query SELECT cast(null as string) >= cast(1 as smallint) FROM t --- !query 69 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS SMALLINT) >= CAST(1 AS SMALLINT)):boolean> --- !query 69 output +-- !query output NULL --- !query 70 +-- !query SELECT cast(null as string) < cast(1 as smallint) FROM t --- !query 70 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS SMALLINT) < CAST(1 AS SMALLINT)):boolean> --- !query 70 output +-- !query output NULL --- !query 71 +-- !query SELECT cast(null as string) <= cast(1 as smallint) FROM t --- !query 71 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS SMALLINT) <= CAST(1 AS SMALLINT)):boolean> --- !query 71 output +-- !query output NULL --- !query 72 +-- !query SELECT cast(null as string) <> cast(1 as smallint) FROM t --- !query 72 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS SMALLINT) = CAST(1 AS SMALLINT))):boolean> --- !query 72 output +-- !query output NULL --- !query 73 +-- !query SELECT cast(1 as int) = '1' FROM t --- !query 73 schema +-- !query schema struct<(CAST(1 AS INT) = CAST(1 AS INT)):boolean> --- !query 73 output +-- !query output true --- !query 74 +-- !query SELECT cast(1 as int) > '2' FROM t --- !query 74 schema +-- !query schema struct<(CAST(1 AS INT) > CAST(2 AS INT)):boolean> --- !query 74 output +-- !query output false --- !query 75 +-- !query SELECT cast(1 as int) >= '2' FROM t --- !query 75 schema +-- !query schema struct<(CAST(1 AS INT) >= CAST(2 AS INT)):boolean> --- !query 75 output +-- !query output false --- !query 76 +-- !query SELECT cast(1 as int) < '2' FROM t --- !query 76 schema +-- !query schema struct<(CAST(1 AS INT) < CAST(2 AS INT)):boolean> --- !query 76 output +-- !query output true --- !query 77 +-- !query SELECT cast(1 as int) <= '2' FROM t --- !query 77 schema +-- !query schema struct<(CAST(1 AS INT) <= CAST(2 AS INT)):boolean> --- !query 77 output +-- !query output true --- !query 78 +-- !query SELECT cast(1 as int) <> '2' FROM t --- !query 78 schema +-- !query schema struct<(NOT (CAST(1 AS INT) = CAST(2 AS INT))):boolean> --- !query 78 output +-- !query output true --- !query 79 +-- !query SELECT cast(1 as int) = cast(null as string) FROM t --- !query 79 schema +-- !query schema struct<(CAST(1 AS INT) = CAST(CAST(NULL AS STRING) AS INT)):boolean> --- !query 79 output +-- !query output NULL --- !query 80 +-- !query SELECT cast(1 as int) > cast(null as string) FROM t --- !query 80 schema +-- !query schema struct<(CAST(1 AS INT) > CAST(CAST(NULL AS STRING) AS INT)):boolean> --- !query 80 output +-- !query output NULL --- !query 81 +-- !query SELECT cast(1 as int) >= cast(null as string) FROM t --- !query 81 schema +-- !query schema struct<(CAST(1 AS INT) >= CAST(CAST(NULL AS STRING) AS INT)):boolean> --- !query 81 output +-- !query output NULL --- !query 82 +-- !query SELECT cast(1 as int) < cast(null as string) FROM t --- !query 82 schema +-- !query schema struct<(CAST(1 AS INT) < CAST(CAST(NULL AS STRING) AS INT)):boolean> --- !query 82 output +-- !query output NULL --- !query 83 +-- !query SELECT cast(1 as int) <= cast(null as string) FROM t --- !query 83 schema +-- !query schema struct<(CAST(1 AS INT) <= CAST(CAST(NULL AS STRING) AS INT)):boolean> --- !query 83 output +-- !query output NULL --- !query 84 +-- !query SELECT cast(1 as int) <> cast(null as string) FROM t --- !query 84 schema +-- !query schema struct<(NOT (CAST(1 AS INT) = CAST(CAST(NULL AS STRING) AS INT))):boolean> --- !query 84 output +-- !query output NULL --- !query 85 +-- !query SELECT '1' = cast(1 as int) FROM t --- !query 85 schema +-- !query schema struct<(CAST(1 AS INT) = CAST(1 AS INT)):boolean> --- !query 85 output +-- !query output true --- !query 86 +-- !query SELECT '2' > cast(1 as int) FROM t --- !query 86 schema +-- !query schema struct<(CAST(2 AS INT) > CAST(1 AS INT)):boolean> --- !query 86 output +-- !query output true --- !query 87 +-- !query SELECT '2' >= cast(1 as int) FROM t --- !query 87 schema +-- !query schema struct<(CAST(2 AS INT) >= CAST(1 AS INT)):boolean> --- !query 87 output +-- !query output true --- !query 88 +-- !query SELECT '2' < cast(1 as int) FROM t --- !query 88 schema +-- !query schema struct<(CAST(2 AS INT) < CAST(1 AS INT)):boolean> --- !query 88 output +-- !query output false --- !query 89 +-- !query SELECT '2' <> cast(1 as int) FROM t --- !query 89 schema +-- !query schema struct<(NOT (CAST(2 AS INT) = CAST(1 AS INT))):boolean> --- !query 89 output +-- !query output true --- !query 90 +-- !query SELECT '2' <= cast(1 as int) FROM t --- !query 90 schema +-- !query schema struct<(CAST(2 AS INT) <= CAST(1 AS INT)):boolean> --- !query 90 output +-- !query output false --- !query 91 +-- !query SELECT cast(null as string) = cast(1 as int) FROM t --- !query 91 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS INT) = CAST(1 AS INT)):boolean> --- !query 91 output +-- !query output NULL --- !query 92 +-- !query SELECT cast(null as string) > cast(1 as int) FROM t --- !query 92 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS INT) > CAST(1 AS INT)):boolean> --- !query 92 output +-- !query output NULL --- !query 93 +-- !query SELECT cast(null as string) >= cast(1 as int) FROM t --- !query 93 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS INT) >= CAST(1 AS INT)):boolean> --- !query 93 output +-- !query output NULL --- !query 94 +-- !query SELECT cast(null as string) < cast(1 as int) FROM t --- !query 94 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS INT) < CAST(1 AS INT)):boolean> --- !query 94 output +-- !query output NULL --- !query 95 +-- !query SELECT cast(null as string) <> cast(1 as int) FROM t --- !query 95 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS INT) = CAST(1 AS INT))):boolean> --- !query 95 output +-- !query output NULL --- !query 96 +-- !query SELECT cast(null as string) <= cast(1 as int) FROM t --- !query 96 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS INT) <= CAST(1 AS INT)):boolean> --- !query 96 output +-- !query output NULL --- !query 97 +-- !query SELECT cast(1 as bigint) = '1' FROM t --- !query 97 schema +-- !query schema struct<(CAST(1 AS BIGINT) = CAST(1 AS BIGINT)):boolean> --- !query 97 output +-- !query output true --- !query 98 +-- !query SELECT cast(1 as bigint) > '2' FROM t --- !query 98 schema +-- !query schema struct<(CAST(1 AS BIGINT) > CAST(2 AS BIGINT)):boolean> --- !query 98 output +-- !query output false --- !query 99 +-- !query SELECT cast(1 as bigint) >= '2' FROM t --- !query 99 schema +-- !query schema struct<(CAST(1 AS BIGINT) >= CAST(2 AS BIGINT)):boolean> --- !query 99 output +-- !query output false --- !query 100 +-- !query SELECT cast(1 as bigint) < '2' FROM t --- !query 100 schema +-- !query schema struct<(CAST(1 AS BIGINT) < CAST(2 AS BIGINT)):boolean> --- !query 100 output +-- !query output true --- !query 101 +-- !query SELECT cast(1 as bigint) <= '2' FROM t --- !query 101 schema +-- !query schema struct<(CAST(1 AS BIGINT) <= CAST(2 AS BIGINT)):boolean> --- !query 101 output +-- !query output true --- !query 102 +-- !query SELECT cast(1 as bigint) <> '2' FROM t --- !query 102 schema +-- !query schema struct<(NOT (CAST(1 AS BIGINT) = CAST(2 AS BIGINT))):boolean> --- !query 102 output +-- !query output true --- !query 103 +-- !query SELECT cast(1 as bigint) = cast(null as string) FROM t --- !query 103 schema +-- !query schema struct<(CAST(1 AS BIGINT) = CAST(CAST(NULL AS STRING) AS BIGINT)):boolean> --- !query 103 output +-- !query output NULL --- !query 104 +-- !query SELECT cast(1 as bigint) > cast(null as string) FROM t --- !query 104 schema +-- !query schema struct<(CAST(1 AS BIGINT) > CAST(CAST(NULL AS STRING) AS BIGINT)):boolean> --- !query 104 output +-- !query output NULL --- !query 105 +-- !query SELECT cast(1 as bigint) >= cast(null as string) FROM t --- !query 105 schema +-- !query schema struct<(CAST(1 AS BIGINT) >= CAST(CAST(NULL AS STRING) AS BIGINT)):boolean> --- !query 105 output +-- !query output NULL --- !query 106 +-- !query SELECT cast(1 as bigint) < cast(null as string) FROM t --- !query 106 schema +-- !query schema struct<(CAST(1 AS BIGINT) < CAST(CAST(NULL AS STRING) AS BIGINT)):boolean> --- !query 106 output +-- !query output NULL --- !query 107 +-- !query SELECT cast(1 as bigint) <= cast(null as string) FROM t --- !query 107 schema +-- !query schema struct<(CAST(1 AS BIGINT) <= CAST(CAST(NULL AS STRING) AS BIGINT)):boolean> --- !query 107 output +-- !query output NULL --- !query 108 +-- !query SELECT cast(1 as bigint) <> cast(null as string) FROM t --- !query 108 schema +-- !query schema struct<(NOT (CAST(1 AS BIGINT) = CAST(CAST(NULL AS STRING) AS BIGINT))):boolean> --- !query 108 output +-- !query output NULL --- !query 109 +-- !query SELECT '1' = cast(1 as bigint) FROM t --- !query 109 schema +-- !query schema struct<(CAST(1 AS BIGINT) = CAST(1 AS BIGINT)):boolean> --- !query 109 output +-- !query output true --- !query 110 +-- !query SELECT '2' > cast(1 as bigint) FROM t --- !query 110 schema +-- !query schema struct<(CAST(2 AS BIGINT) > CAST(1 AS BIGINT)):boolean> --- !query 110 output +-- !query output true --- !query 111 +-- !query SELECT '2' >= cast(1 as bigint) FROM t --- !query 111 schema +-- !query schema struct<(CAST(2 AS BIGINT) >= CAST(1 AS BIGINT)):boolean> --- !query 111 output +-- !query output true --- !query 112 +-- !query SELECT '2' < cast(1 as bigint) FROM t --- !query 112 schema +-- !query schema struct<(CAST(2 AS BIGINT) < CAST(1 AS BIGINT)):boolean> --- !query 112 output +-- !query output false --- !query 113 +-- !query SELECT '2' <= cast(1 as bigint) FROM t --- !query 113 schema +-- !query schema struct<(CAST(2 AS BIGINT) <= CAST(1 AS BIGINT)):boolean> --- !query 113 output +-- !query output false --- !query 114 +-- !query SELECT '2' <> cast(1 as bigint) FROM t --- !query 114 schema +-- !query schema struct<(NOT (CAST(2 AS BIGINT) = CAST(1 AS BIGINT))):boolean> --- !query 114 output +-- !query output true --- !query 115 +-- !query SELECT cast(null as string) = cast(1 as bigint) FROM t --- !query 115 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BIGINT) = CAST(1 AS BIGINT)):boolean> --- !query 115 output +-- !query output NULL --- !query 116 +-- !query SELECT cast(null as string) > cast(1 as bigint) FROM t --- !query 116 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BIGINT) > CAST(1 AS BIGINT)):boolean> --- !query 116 output +-- !query output NULL --- !query 117 +-- !query SELECT cast(null as string) >= cast(1 as bigint) FROM t --- !query 117 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BIGINT) >= CAST(1 AS BIGINT)):boolean> --- !query 117 output +-- !query output NULL --- !query 118 +-- !query SELECT cast(null as string) < cast(1 as bigint) FROM t --- !query 118 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BIGINT) < CAST(1 AS BIGINT)):boolean> --- !query 118 output +-- !query output NULL --- !query 119 +-- !query SELECT cast(null as string) <= cast(1 as bigint) FROM t --- !query 119 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BIGINT) <= CAST(1 AS BIGINT)):boolean> --- !query 119 output +-- !query output NULL --- !query 120 +-- !query SELECT cast(null as string) <> cast(1 as bigint) FROM t --- !query 120 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS BIGINT) = CAST(1 AS BIGINT))):boolean> --- !query 120 output +-- !query output NULL --- !query 121 +-- !query SELECT cast(1 as decimal(10, 0)) = '1' FROM t --- !query 121 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 121 output +-- !query output true --- !query 122 +-- !query SELECT cast(1 as decimal(10, 0)) > '2' FROM t --- !query 122 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(2 AS DOUBLE)):boolean> --- !query 122 output +-- !query output false --- !query 123 +-- !query SELECT cast(1 as decimal(10, 0)) >= '2' FROM t --- !query 123 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(2 AS DOUBLE)):boolean> --- !query 123 output +-- !query output false --- !query 124 +-- !query SELECT cast(1 as decimal(10, 0)) < '2' FROM t --- !query 124 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(2 AS DOUBLE)):boolean> --- !query 124 output +-- !query output true --- !query 125 +-- !query SELECT cast(1 as decimal(10, 0)) <> '2' FROM t --- !query 125 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(2 AS DOUBLE))):boolean> --- !query 125 output +-- !query output true --- !query 126 +-- !query SELECT cast(1 as decimal(10, 0)) <= '2' FROM t --- !query 126 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(2 AS DOUBLE)):boolean> --- !query 126 output +-- !query output true --- !query 127 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(null as string) FROM t --- !query 127 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 127 output +-- !query output NULL --- !query 128 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(null as string) FROM t --- !query 128 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 128 output +-- !query output NULL --- !query 129 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(null as string) FROM t --- !query 129 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 129 output +-- !query output NULL --- !query 130 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(null as string) FROM t --- !query 130 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 130 output +-- !query output NULL --- !query 131 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(null as string) FROM t --- !query 131 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(NULL AS STRING) AS DOUBLE))):boolean> --- !query 131 output +-- !query output NULL --- !query 132 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(null as string) FROM t --- !query 132 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 132 output +-- !query output NULL --- !query 133 +-- !query SELECT '1' = cast(1 as decimal(10, 0)) FROM t --- !query 133 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 133 output +-- !query output true --- !query 134 +-- !query SELECT '2' > cast(1 as decimal(10, 0)) FROM t --- !query 134 schema +-- !query schema struct<(CAST(2 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 134 output +-- !query output true --- !query 135 +-- !query SELECT '2' >= cast(1 as decimal(10, 0)) FROM t --- !query 135 schema +-- !query schema struct<(CAST(2 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 135 output +-- !query output true --- !query 136 +-- !query SELECT '2' < cast(1 as decimal(10, 0)) FROM t --- !query 136 schema +-- !query schema struct<(CAST(2 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 136 output +-- !query output false --- !query 137 +-- !query SELECT '2' <= cast(1 as decimal(10, 0)) FROM t --- !query 137 schema +-- !query schema struct<(CAST(2 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 137 output +-- !query output false --- !query 138 +-- !query SELECT '2' <> cast(1 as decimal(10, 0)) FROM t --- !query 138 schema +-- !query schema struct<(NOT (CAST(2 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 138 output +-- !query output true --- !query 139 +-- !query SELECT cast(null as string) = cast(1 as decimal(10, 0)) FROM t --- !query 139 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 139 output +-- !query output NULL --- !query 140 +-- !query SELECT cast(null as string) > cast(1 as decimal(10, 0)) FROM t --- !query 140 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 140 output +-- !query output NULL --- !query 141 +-- !query SELECT cast(null as string) >= cast(1 as decimal(10, 0)) FROM t --- !query 141 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 141 output +-- !query output NULL --- !query 142 +-- !query SELECT cast(null as string) < cast(1 as decimal(10, 0)) FROM t --- !query 142 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 142 output +-- !query output NULL --- !query 143 +-- !query SELECT cast(null as string) <= cast(1 as decimal(10, 0)) FROM t --- !query 143 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 143 output +-- !query output NULL --- !query 144 +-- !query SELECT cast(null as string) <> cast(1 as decimal(10, 0)) FROM t --- !query 144 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 144 output +-- !query output NULL --- !query 145 +-- !query SELECT cast(1 as double) = '1' FROM t --- !query 145 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 145 output +-- !query output true --- !query 146 +-- !query SELECT cast(1 as double) > '2' FROM t --- !query 146 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(2 AS DOUBLE)):boolean> --- !query 146 output +-- !query output false --- !query 147 +-- !query SELECT cast(1 as double) >= '2' FROM t --- !query 147 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(2 AS DOUBLE)):boolean> --- !query 147 output +-- !query output false --- !query 148 +-- !query SELECT cast(1 as double) < '2' FROM t --- !query 148 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(2 AS DOUBLE)):boolean> --- !query 148 output +-- !query output true --- !query 149 +-- !query SELECT cast(1 as double) <= '2' FROM t --- !query 149 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(2 AS DOUBLE)):boolean> --- !query 149 output +-- !query output true --- !query 150 +-- !query SELECT cast(1 as double) <> '2' FROM t --- !query 150 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(2 AS DOUBLE))):boolean> --- !query 150 output +-- !query output true --- !query 151 +-- !query SELECT cast(1 as double) = cast(null as string) FROM t --- !query 151 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 151 output +-- !query output NULL --- !query 152 +-- !query SELECT cast(1 as double) > cast(null as string) FROM t --- !query 152 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 152 output +-- !query output NULL --- !query 153 +-- !query SELECT cast(1 as double) >= cast(null as string) FROM t --- !query 153 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 153 output +-- !query output NULL --- !query 154 +-- !query SELECT cast(1 as double) < cast(null as string) FROM t --- !query 154 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 154 output +-- !query output NULL --- !query 155 +-- !query SELECT cast(1 as double) <= cast(null as string) FROM t --- !query 155 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(CAST(NULL AS STRING) AS DOUBLE)):boolean> --- !query 155 output +-- !query output NULL --- !query 156 +-- !query SELECT cast(1 as double) <> cast(null as string) FROM t --- !query 156 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(NULL AS STRING) AS DOUBLE))):boolean> --- !query 156 output +-- !query output NULL --- !query 157 +-- !query SELECT '1' = cast(1 as double) FROM t --- !query 157 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 157 output +-- !query output true --- !query 158 +-- !query SELECT '2' > cast(1 as double) FROM t --- !query 158 schema +-- !query schema struct<(CAST(2 AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 158 output +-- !query output true --- !query 159 +-- !query SELECT '2' >= cast(1 as double) FROM t --- !query 159 schema +-- !query schema struct<(CAST(2 AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 159 output +-- !query output true --- !query 160 +-- !query SELECT '2' < cast(1 as double) FROM t --- !query 160 schema +-- !query schema struct<(CAST(2 AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 160 output +-- !query output false --- !query 161 +-- !query SELECT '2' <= cast(1 as double) FROM t --- !query 161 schema +-- !query schema struct<(CAST(2 AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 161 output +-- !query output false --- !query 162 +-- !query SELECT '2' <> cast(1 as double) FROM t --- !query 162 schema +-- !query schema struct<(NOT (CAST(2 AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 162 output +-- !query output true --- !query 163 +-- !query SELECT cast(null as string) = cast(1 as double) FROM t --- !query 163 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 163 output +-- !query output NULL --- !query 164 +-- !query SELECT cast(null as string) > cast(1 as double) FROM t --- !query 164 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 164 output +-- !query output NULL --- !query 165 +-- !query SELECT cast(null as string) >= cast(1 as double) FROM t --- !query 165 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 165 output +-- !query output NULL --- !query 166 +-- !query SELECT cast(null as string) < cast(1 as double) FROM t --- !query 166 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 166 output +-- !query output NULL --- !query 167 +-- !query SELECT cast(null as string) <= cast(1 as double) FROM t --- !query 167 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 167 output +-- !query output NULL --- !query 168 +-- !query SELECT cast(null as string) <> cast(1 as double) FROM t --- !query 168 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 168 output +-- !query output NULL --- !query 169 +-- !query SELECT cast(1 as float) = '1' FROM t --- !query 169 schema +-- !query schema struct<(CAST(1 AS FLOAT) = CAST(1 AS FLOAT)):boolean> --- !query 169 output +-- !query output true --- !query 170 +-- !query SELECT cast(1 as float) > '2' FROM t --- !query 170 schema +-- !query schema struct<(CAST(1 AS FLOAT) > CAST(2 AS FLOAT)):boolean> --- !query 170 output +-- !query output false --- !query 171 +-- !query SELECT cast(1 as float) >= '2' FROM t --- !query 171 schema +-- !query schema struct<(CAST(1 AS FLOAT) >= CAST(2 AS FLOAT)):boolean> --- !query 171 output +-- !query output false --- !query 172 +-- !query SELECT cast(1 as float) < '2' FROM t --- !query 172 schema +-- !query schema struct<(CAST(1 AS FLOAT) < CAST(2 AS FLOAT)):boolean> --- !query 172 output +-- !query output true --- !query 173 +-- !query SELECT cast(1 as float) <= '2' FROM t --- !query 173 schema +-- !query schema struct<(CAST(1 AS FLOAT) <= CAST(2 AS FLOAT)):boolean> --- !query 173 output +-- !query output true --- !query 174 +-- !query SELECT cast(1 as float) <> '2' FROM t --- !query 174 schema +-- !query schema struct<(NOT (CAST(1 AS FLOAT) = CAST(2 AS FLOAT))):boolean> --- !query 174 output +-- !query output true --- !query 175 +-- !query SELECT cast(1 as float) = cast(null as string) FROM t --- !query 175 schema +-- !query schema struct<(CAST(1 AS FLOAT) = CAST(CAST(NULL AS STRING) AS FLOAT)):boolean> --- !query 175 output +-- !query output NULL --- !query 176 +-- !query SELECT cast(1 as float) > cast(null as string) FROM t --- !query 176 schema +-- !query schema struct<(CAST(1 AS FLOAT) > CAST(CAST(NULL AS STRING) AS FLOAT)):boolean> --- !query 176 output +-- !query output NULL --- !query 177 +-- !query SELECT cast(1 as float) >= cast(null as string) FROM t --- !query 177 schema +-- !query schema struct<(CAST(1 AS FLOAT) >= CAST(CAST(NULL AS STRING) AS FLOAT)):boolean> --- !query 177 output +-- !query output NULL --- !query 178 +-- !query SELECT cast(1 as float) < cast(null as string) FROM t --- !query 178 schema +-- !query schema struct<(CAST(1 AS FLOAT) < CAST(CAST(NULL AS STRING) AS FLOAT)):boolean> --- !query 178 output +-- !query output NULL --- !query 179 +-- !query SELECT cast(1 as float) <= cast(null as string) FROM t --- !query 179 schema +-- !query schema struct<(CAST(1 AS FLOAT) <= CAST(CAST(NULL AS STRING) AS FLOAT)):boolean> --- !query 179 output +-- !query output NULL --- !query 180 +-- !query SELECT cast(1 as float) <> cast(null as string) FROM t --- !query 180 schema +-- !query schema struct<(NOT (CAST(1 AS FLOAT) = CAST(CAST(NULL AS STRING) AS FLOAT))):boolean> --- !query 180 output +-- !query output NULL --- !query 181 +-- !query SELECT '1' = cast(1 as float) FROM t --- !query 181 schema +-- !query schema struct<(CAST(1 AS FLOAT) = CAST(1 AS FLOAT)):boolean> --- !query 181 output +-- !query output true --- !query 182 +-- !query SELECT '2' > cast(1 as float) FROM t --- !query 182 schema +-- !query schema struct<(CAST(2 AS FLOAT) > CAST(1 AS FLOAT)):boolean> --- !query 182 output +-- !query output true --- !query 183 +-- !query SELECT '2' >= cast(1 as float) FROM t --- !query 183 schema +-- !query schema struct<(CAST(2 AS FLOAT) >= CAST(1 AS FLOAT)):boolean> --- !query 183 output +-- !query output true --- !query 184 +-- !query SELECT '2' < cast(1 as float) FROM t --- !query 184 schema +-- !query schema struct<(CAST(2 AS FLOAT) < CAST(1 AS FLOAT)):boolean> --- !query 184 output +-- !query output false --- !query 185 +-- !query SELECT '2' <= cast(1 as float) FROM t --- !query 185 schema +-- !query schema struct<(CAST(2 AS FLOAT) <= CAST(1 AS FLOAT)):boolean> --- !query 185 output +-- !query output false --- !query 186 +-- !query SELECT '2' <> cast(1 as float) FROM t --- !query 186 schema +-- !query schema struct<(NOT (CAST(2 AS FLOAT) = CAST(1 AS FLOAT))):boolean> --- !query 186 output +-- !query output true --- !query 187 +-- !query SELECT cast(null as string) = cast(1 as float) FROM t --- !query 187 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS FLOAT) = CAST(1 AS FLOAT)):boolean> --- !query 187 output +-- !query output NULL --- !query 188 +-- !query SELECT cast(null as string) > cast(1 as float) FROM t --- !query 188 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS FLOAT) > CAST(1 AS FLOAT)):boolean> --- !query 188 output +-- !query output NULL --- !query 189 +-- !query SELECT cast(null as string) >= cast(1 as float) FROM t --- !query 189 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS FLOAT) >= CAST(1 AS FLOAT)):boolean> --- !query 189 output +-- !query output NULL --- !query 190 +-- !query SELECT cast(null as string) < cast(1 as float) FROM t --- !query 190 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS FLOAT) < CAST(1 AS FLOAT)):boolean> --- !query 190 output +-- !query output NULL --- !query 191 +-- !query SELECT cast(null as string) <= cast(1 as float) FROM t --- !query 191 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS FLOAT) <= CAST(1 AS FLOAT)):boolean> --- !query 191 output +-- !query output NULL --- !query 192 +-- !query SELECT cast(null as string) <> cast(1 as float) FROM t --- !query 192 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS FLOAT) = CAST(1 AS FLOAT))):boolean> --- !query 192 output +-- !query output NULL --- !query 193 +-- !query SELECT '1996-09-09' = date('1996-09-09') FROM t --- !query 193 schema +-- !query schema struct<(CAST(1996-09-09 AS DATE) = CAST(1996-09-09 AS DATE)):boolean> --- !query 193 output +-- !query output true --- !query 194 +-- !query SELECT '1996-9-10' > date('1996-09-09') FROM t --- !query 194 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) > CAST(1996-09-09 AS DATE)):boolean> --- !query 194 output +-- !query output true --- !query 195 +-- !query SELECT '1996-9-10' >= date('1996-09-09') FROM t --- !query 195 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) >= CAST(1996-09-09 AS DATE)):boolean> --- !query 195 output +-- !query output true --- !query 196 +-- !query SELECT '1996-9-10' < date('1996-09-09') FROM t --- !query 196 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) < CAST(1996-09-09 AS DATE)):boolean> --- !query 196 output +-- !query output false --- !query 197 +-- !query SELECT '1996-9-10' <= date('1996-09-09') FROM t --- !query 197 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) <= CAST(1996-09-09 AS DATE)):boolean> --- !query 197 output +-- !query output false --- !query 198 +-- !query SELECT '1996-9-10' <> date('1996-09-09') FROM t --- !query 198 schema +-- !query schema struct<(NOT (CAST(1996-9-10 AS DATE) = CAST(1996-09-09 AS DATE))):boolean> --- !query 198 output +-- !query output true --- !query 199 +-- !query SELECT cast(null as string) = date('1996-09-09') FROM t --- !query 199 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DATE) = CAST(1996-09-09 AS DATE)):boolean> --- !query 199 output +-- !query output NULL --- !query 200 +-- !query SELECT cast(null as string)> date('1996-09-09') FROM t --- !query 200 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DATE) > CAST(1996-09-09 AS DATE)):boolean> --- !query 200 output +-- !query output NULL --- !query 201 +-- !query SELECT cast(null as string)>= date('1996-09-09') FROM t --- !query 201 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DATE) >= CAST(1996-09-09 AS DATE)):boolean> --- !query 201 output +-- !query output NULL --- !query 202 +-- !query SELECT cast(null as string)< date('1996-09-09') FROM t --- !query 202 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DATE) < CAST(1996-09-09 AS DATE)):boolean> --- !query 202 output +-- !query output NULL --- !query 203 +-- !query SELECT cast(null as string)<= date('1996-09-09') FROM t --- !query 203 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS DATE) <= CAST(1996-09-09 AS DATE)):boolean> --- !query 203 output +-- !query output NULL --- !query 204 +-- !query SELECT cast(null as string)<> date('1996-09-09') FROM t --- !query 204 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS DATE) = CAST(1996-09-09 AS DATE))):boolean> --- !query 204 output +-- !query output NULL --- !query 205 +-- !query SELECT date('1996-09-09') = '1996-09-09' FROM t --- !query 205 schema +-- !query schema struct<(CAST(1996-09-09 AS DATE) = CAST(1996-09-09 AS DATE)):boolean> --- !query 205 output +-- !query output true --- !query 206 +-- !query SELECT date('1996-9-10') > '1996-09-09' FROM t --- !query 206 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) > CAST(1996-09-09 AS DATE)):boolean> --- !query 206 output +-- !query output true --- !query 207 +-- !query SELECT date('1996-9-10') >= '1996-09-09' FROM t --- !query 207 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) >= CAST(1996-09-09 AS DATE)):boolean> --- !query 207 output +-- !query output true --- !query 208 +-- !query SELECT date('1996-9-10') < '1996-09-09' FROM t --- !query 208 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) < CAST(1996-09-09 AS DATE)):boolean> --- !query 208 output +-- !query output false --- !query 209 +-- !query SELECT date('1996-9-10') <= '1996-09-09' FROM t --- !query 209 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) <= CAST(1996-09-09 AS DATE)):boolean> --- !query 209 output +-- !query output false --- !query 210 +-- !query SELECT date('1996-9-10') <> '1996-09-09' FROM t --- !query 210 schema +-- !query schema struct<(NOT (CAST(1996-9-10 AS DATE) = CAST(1996-09-09 AS DATE))):boolean> --- !query 210 output +-- !query output true --- !query 211 +-- !query SELECT date('1996-09-09') = cast(null as string) FROM t --- !query 211 schema +-- !query schema struct<(CAST(1996-09-09 AS DATE) = CAST(CAST(NULL AS STRING) AS DATE)):boolean> --- !query 211 output +-- !query output NULL --- !query 212 +-- !query SELECT date('1996-9-10') > cast(null as string) FROM t --- !query 212 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) > CAST(CAST(NULL AS STRING) AS DATE)):boolean> --- !query 212 output +-- !query output NULL --- !query 213 +-- !query SELECT date('1996-9-10') >= cast(null as string) FROM t --- !query 213 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) >= CAST(CAST(NULL AS STRING) AS DATE)):boolean> --- !query 213 output +-- !query output NULL --- !query 214 +-- !query SELECT date('1996-9-10') < cast(null as string) FROM t --- !query 214 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) < CAST(CAST(NULL AS STRING) AS DATE)):boolean> --- !query 214 output +-- !query output NULL --- !query 215 +-- !query SELECT date('1996-9-10') <= cast(null as string) FROM t --- !query 215 schema +-- !query schema struct<(CAST(1996-9-10 AS DATE) <= CAST(CAST(NULL AS STRING) AS DATE)):boolean> --- !query 215 output +-- !query output NULL --- !query 216 +-- !query SELECT date('1996-9-10') <> cast(null as string) FROM t --- !query 216 schema +-- !query schema struct<(NOT (CAST(1996-9-10 AS DATE) = CAST(CAST(NULL AS STRING) AS DATE))):boolean> --- !query 216 output +-- !query output NULL --- !query 217 +-- !query SELECT '1996-09-09 12:12:12.4' = timestamp('1996-09-09 12:12:12.4') FROM t --- !query 217 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) = CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 217 output +-- !query output true --- !query 218 +-- !query SELECT '1996-09-09 12:12:12.5' > timestamp('1996-09-09 12:12:12.4') FROM t --- !query 218 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) > CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 218 output +-- !query output true --- !query 219 +-- !query SELECT '1996-09-09 12:12:12.5' >= timestamp('1996-09-09 12:12:12.4') FROM t --- !query 219 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) >= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 219 output +-- !query output true --- !query 220 +-- !query SELECT '1996-09-09 12:12:12.5' < timestamp('1996-09-09 12:12:12.4') FROM t --- !query 220 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) < CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 220 output +-- !query output false --- !query 221 +-- !query SELECT '1996-09-09 12:12:12.5' <= timestamp('1996-09-09 12:12:12.4') FROM t --- !query 221 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) <= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 221 output +-- !query output false --- !query 222 +-- !query SELECT '1996-09-09 12:12:12.5' <> timestamp('1996-09-09 12:12:12.4') FROM t --- !query 222 schema +-- !query schema struct<(NOT (CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) = CAST(1996-09-09 12:12:12.4 AS TIMESTAMP))):boolean> --- !query 222 output +-- !query output true --- !query 223 +-- !query SELECT cast(null as string) = timestamp('1996-09-09 12:12:12.4') FROM t --- !query 223 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) = CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 223 output +-- !query output NULL --- !query 224 +-- !query SELECT cast(null as string) > timestamp('1996-09-09 12:12:12.4') FROM t --- !query 224 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) > CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 224 output +-- !query output NULL --- !query 225 +-- !query SELECT cast(null as string) >= timestamp('1996-09-09 12:12:12.4') FROM t --- !query 225 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) >= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 225 output +-- !query output NULL --- !query 226 +-- !query SELECT cast(null as string) < timestamp('1996-09-09 12:12:12.4') FROM t --- !query 226 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) < CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 226 output +-- !query output NULL --- !query 227 +-- !query SELECT cast(null as string) <= timestamp('1996-09-09 12:12:12.4') FROM t --- !query 227 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) <= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 227 output +-- !query output NULL --- !query 228 +-- !query SELECT cast(null as string) <> timestamp('1996-09-09 12:12:12.4') FROM t --- !query 228 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS TIMESTAMP) = CAST(1996-09-09 12:12:12.4 AS TIMESTAMP))):boolean> --- !query 228 output +-- !query output NULL --- !query 229 +-- !query SELECT timestamp('1996-09-09 12:12:12.4' )= '1996-09-09 12:12:12.4' FROM t --- !query 229 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) = CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 229 output +-- !query output true --- !query 230 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )> '1996-09-09 12:12:12.4' FROM t --- !query 230 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) > CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 230 output +-- !query output true --- !query 231 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )>= '1996-09-09 12:12:12.4' FROM t --- !query 231 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) >= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 231 output +-- !query output true --- !query 232 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )< '1996-09-09 12:12:12.4' FROM t --- !query 232 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) < CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 232 output +-- !query output false --- !query 233 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )<= '1996-09-09 12:12:12.4' FROM t --- !query 233 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) <= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> --- !query 233 output +-- !query output false --- !query 234 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )<> '1996-09-09 12:12:12.4' FROM t --- !query 234 schema +-- !query schema struct<(NOT (CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) = CAST(1996-09-09 12:12:12.4 AS TIMESTAMP))):boolean> --- !query 234 output +-- !query output true --- !query 235 +-- !query SELECT timestamp('1996-09-09 12:12:12.4' )= cast(null as string) FROM t --- !query 235 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) = CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> --- !query 235 output +-- !query output NULL --- !query 236 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )> cast(null as string) FROM t --- !query 236 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) > CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> --- !query 236 output +-- !query output NULL --- !query 237 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )>= cast(null as string) FROM t --- !query 237 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) >= CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> --- !query 237 output +-- !query output NULL --- !query 238 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )< cast(null as string) FROM t --- !query 238 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) < CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> --- !query 238 output +-- !query output NULL --- !query 239 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )<= cast(null as string) FROM t --- !query 239 schema +-- !query schema struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) <= CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> --- !query 239 output +-- !query output NULL --- !query 240 +-- !query SELECT timestamp('1996-09-09 12:12:12.5' )<> cast(null as string) FROM t --- !query 240 schema +-- !query schema struct<(NOT (CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) = CAST(CAST(NULL AS STRING) AS TIMESTAMP))):boolean> --- !query 240 output +-- !query output NULL --- !query 241 +-- !query SELECT ' ' = X'0020' FROM t --- !query 241 schema +-- !query schema struct<(CAST( AS BINARY) = X'0020'):boolean> --- !query 241 output +-- !query output false --- !query 242 +-- !query SELECT ' ' > X'001F' FROM t --- !query 242 schema +-- !query schema struct<(CAST( AS BINARY) > X'001F'):boolean> --- !query 242 output +-- !query output true --- !query 243 +-- !query SELECT ' ' >= X'001F' FROM t --- !query 243 schema +-- !query schema struct<(CAST( AS BINARY) >= X'001F'):boolean> --- !query 243 output +-- !query output true --- !query 244 +-- !query SELECT ' ' < X'001F' FROM t --- !query 244 schema +-- !query schema struct<(CAST( AS BINARY) < X'001F'):boolean> --- !query 244 output +-- !query output false --- !query 245 +-- !query SELECT ' ' <= X'001F' FROM t --- !query 245 schema +-- !query schema struct<(CAST( AS BINARY) <= X'001F'):boolean> --- !query 245 output +-- !query output false --- !query 246 +-- !query SELECT ' ' <> X'001F' FROM t --- !query 246 schema +-- !query schema struct<(NOT (CAST( AS BINARY) = X'001F')):boolean> --- !query 246 output +-- !query output true --- !query 247 +-- !query SELECT cast(null as string) = X'0020' FROM t --- !query 247 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) = X'0020'):boolean> --- !query 247 output +-- !query output NULL --- !query 248 +-- !query SELECT cast(null as string) > X'001F' FROM t --- !query 248 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) > X'001F'):boolean> --- !query 248 output +-- !query output NULL --- !query 249 +-- !query SELECT cast(null as string) >= X'001F' FROM t --- !query 249 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) >= X'001F'):boolean> --- !query 249 output +-- !query output NULL --- !query 250 +-- !query SELECT cast(null as string) < X'001F' FROM t --- !query 250 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) < X'001F'):boolean> --- !query 250 output +-- !query output NULL --- !query 251 +-- !query SELECT cast(null as string) <= X'001F' FROM t --- !query 251 schema +-- !query schema struct<(CAST(CAST(NULL AS STRING) AS BINARY) <= X'001F'):boolean> --- !query 251 output +-- !query output NULL --- !query 252 +-- !query SELECT cast(null as string) <> X'001F' FROM t --- !query 252 schema +-- !query schema struct<(NOT (CAST(CAST(NULL AS STRING) AS BINARY) = X'001F')):boolean> --- !query 252 output +-- !query output NULL --- !query 253 +-- !query SELECT X'0020' = ' ' FROM t --- !query 253 schema +-- !query schema struct<(X'0020' = CAST( AS BINARY)):boolean> --- !query 253 output +-- !query output false --- !query 254 +-- !query SELECT X'001F' > ' ' FROM t --- !query 254 schema +-- !query schema struct<(X'001F' > CAST( AS BINARY)):boolean> --- !query 254 output +-- !query output false --- !query 255 +-- !query SELECT X'001F' >= ' ' FROM t --- !query 255 schema +-- !query schema struct<(X'001F' >= CAST( AS BINARY)):boolean> --- !query 255 output +-- !query output false --- !query 256 +-- !query SELECT X'001F' < ' ' FROM t --- !query 256 schema +-- !query schema struct<(X'001F' < CAST( AS BINARY)):boolean> --- !query 256 output +-- !query output true --- !query 257 +-- !query SELECT X'001F' <= ' ' FROM t --- !query 257 schema +-- !query schema struct<(X'001F' <= CAST( AS BINARY)):boolean> --- !query 257 output +-- !query output true --- !query 258 +-- !query SELECT X'001F' <> ' ' FROM t --- !query 258 schema +-- !query schema struct<(NOT (X'001F' = CAST( AS BINARY))):boolean> --- !query 258 output +-- !query output true --- !query 259 +-- !query SELECT X'0020' = cast(null as string) FROM t --- !query 259 schema +-- !query schema struct<(X'0020' = CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 259 output +-- !query output NULL --- !query 260 +-- !query SELECT X'001F' > cast(null as string) FROM t --- !query 260 schema +-- !query schema struct<(X'001F' > CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 260 output +-- !query output NULL --- !query 261 +-- !query SELECT X'001F' >= cast(null as string) FROM t --- !query 261 schema +-- !query schema struct<(X'001F' >= CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 261 output +-- !query output NULL --- !query 262 +-- !query SELECT X'001F' < cast(null as string) FROM t --- !query 262 schema +-- !query schema struct<(X'001F' < CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 262 output +-- !query output NULL --- !query 263 +-- !query SELECT X'001F' <= cast(null as string) FROM t --- !query 263 schema +-- !query schema struct<(X'001F' <= CAST(CAST(NULL AS STRING) AS BINARY)):boolean> --- !query 263 output +-- !query output NULL --- !query 264 +-- !query SELECT X'001F' <> cast(null as string) FROM t --- !query 264 schema +-- !query schema struct<(NOT (X'001F' = CAST(CAST(NULL AS STRING) AS BINARY))):boolean> --- !query 264 output +-- !query output NULL diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out index 46775d79ff4a2..dc068e70d66db 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out @@ -2,801 +2,801 @@ -- Number of queries: 97 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT true = cast(1 as tinyint) FROM t --- !query 1 schema +-- !query schema struct<(CAST(true AS TINYINT) = CAST(1 AS TINYINT)):boolean> --- !query 1 output +-- !query output true --- !query 2 +-- !query SELECT true = cast(1 as smallint) FROM t --- !query 2 schema +-- !query schema struct<(CAST(true AS SMALLINT) = CAST(1 AS SMALLINT)):boolean> --- !query 2 output +-- !query output true --- !query 3 +-- !query SELECT true = cast(1 as int) FROM t --- !query 3 schema +-- !query schema struct<(CAST(true AS INT) = CAST(1 AS INT)):boolean> --- !query 3 output +-- !query output true --- !query 4 +-- !query SELECT true = cast(1 as bigint) FROM t --- !query 4 schema +-- !query schema struct<(CAST(true AS BIGINT) = CAST(1 AS BIGINT)):boolean> --- !query 4 output +-- !query output true --- !query 5 +-- !query SELECT true = cast(1 as float) FROM t --- !query 5 schema +-- !query schema struct<(CAST(true AS FLOAT) = CAST(1 AS FLOAT)):boolean> --- !query 5 output +-- !query output true --- !query 6 +-- !query SELECT true = cast(1 as double) FROM t --- !query 6 schema +-- !query schema struct<(CAST(true AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 6 output +-- !query output true --- !query 7 +-- !query SELECT true = cast(1 as decimal(10, 0)) FROM t --- !query 7 schema +-- !query schema struct<(CAST(true AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> --- !query 7 output +-- !query output true --- !query 8 +-- !query SELECT true = cast(1 as string) FROM t --- !query 8 schema +-- !query schema struct<(true = CAST(CAST(1 AS STRING) AS BOOLEAN)):boolean> --- !query 8 output +-- !query output true --- !query 9 +-- !query SELECT true = cast('1' as binary) FROM t --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(true = CAST('1' AS BINARY))' (boolean and binary).; line 1 pos 7 --- !query 10 +-- !query SELECT true = cast(1 as boolean) FROM t --- !query 10 schema +-- !query schema struct<(true = CAST(1 AS BOOLEAN)):boolean> --- !query 10 output +-- !query output true --- !query 11 +-- !query SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(true = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7 --- !query 12 +-- !query SELECT true = cast('2017-12-11 09:30:00' as date) FROM t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(true = CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7 --- !query 13 +-- !query SELECT true <=> cast(1 as tinyint) FROM t --- !query 13 schema +-- !query schema struct<(CAST(true AS TINYINT) <=> CAST(1 AS TINYINT)):boolean> --- !query 13 output +-- !query output true --- !query 14 +-- !query SELECT true <=> cast(1 as smallint) FROM t --- !query 14 schema +-- !query schema struct<(CAST(true AS SMALLINT) <=> CAST(1 AS SMALLINT)):boolean> --- !query 14 output +-- !query output true --- !query 15 +-- !query SELECT true <=> cast(1 as int) FROM t --- !query 15 schema +-- !query schema struct<(CAST(true AS INT) <=> CAST(1 AS INT)):boolean> --- !query 15 output +-- !query output true --- !query 16 +-- !query SELECT true <=> cast(1 as bigint) FROM t --- !query 16 schema +-- !query schema struct<(CAST(true AS BIGINT) <=> CAST(1 AS BIGINT)):boolean> --- !query 16 output +-- !query output true --- !query 17 +-- !query SELECT true <=> cast(1 as float) FROM t --- !query 17 schema +-- !query schema struct<(CAST(true AS FLOAT) <=> CAST(1 AS FLOAT)):boolean> --- !query 17 output +-- !query output true --- !query 18 +-- !query SELECT true <=> cast(1 as double) FROM t --- !query 18 schema +-- !query schema struct<(CAST(true AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> --- !query 18 output +-- !query output true --- !query 19 +-- !query SELECT true <=> cast(1 as decimal(10, 0)) FROM t --- !query 19 schema +-- !query schema struct<(CAST(true AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> --- !query 19 output +-- !query output true --- !query 20 +-- !query SELECT true <=> cast(1 as string) FROM t --- !query 20 schema +-- !query schema struct<(true <=> CAST(CAST(1 AS STRING) AS BOOLEAN)):boolean> --- !query 20 output +-- !query output true --- !query 21 +-- !query SELECT true <=> cast('1' as binary) FROM t --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(true <=> CAST('1' AS BINARY))' (boolean and binary).; line 1 pos 7 --- !query 22 +-- !query SELECT true <=> cast(1 as boolean) FROM t --- !query 22 schema +-- !query schema struct<(true <=> CAST(1 AS BOOLEAN)):boolean> --- !query 22 output +-- !query output true --- !query 23 +-- !query SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(true <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7 --- !query 24 +-- !query SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(true <=> CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7 --- !query 25 +-- !query SELECT cast(1 as tinyint) = true FROM t --- !query 25 schema +-- !query schema struct<(CAST(1 AS TINYINT) = CAST(true AS TINYINT)):boolean> --- !query 25 output +-- !query output true --- !query 26 +-- !query SELECT cast(1 as smallint) = true FROM t --- !query 26 schema +-- !query schema struct<(CAST(1 AS SMALLINT) = CAST(true AS SMALLINT)):boolean> --- !query 26 output +-- !query output true --- !query 27 +-- !query SELECT cast(1 as int) = true FROM t --- !query 27 schema +-- !query schema struct<(CAST(1 AS INT) = CAST(true AS INT)):boolean> --- !query 27 output +-- !query output true --- !query 28 +-- !query SELECT cast(1 as bigint) = true FROM t --- !query 28 schema +-- !query schema struct<(CAST(1 AS BIGINT) = CAST(true AS BIGINT)):boolean> --- !query 28 output +-- !query output true --- !query 29 +-- !query SELECT cast(1 as float) = true FROM t --- !query 29 schema +-- !query schema struct<(CAST(1 AS FLOAT) = CAST(true AS FLOAT)):boolean> --- !query 29 output +-- !query output true --- !query 30 +-- !query SELECT cast(1 as double) = true FROM t --- !query 30 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(true AS DOUBLE)):boolean> --- !query 30 output +-- !query output true --- !query 31 +-- !query SELECT cast(1 as decimal(10, 0)) = true FROM t --- !query 31 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) = CAST(true AS DECIMAL(10,0))):boolean> --- !query 31 output +-- !query output true --- !query 32 +-- !query SELECT cast(1 as string) = true FROM t --- !query 32 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS BOOLEAN) = true):boolean> --- !query 32 output +-- !query output true --- !query 33 +-- !query SELECT cast('1' as binary) = true FROM t --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = true)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = true)' (binary and boolean).; line 1 pos 7 --- !query 34 +-- !query SELECT cast(1 as boolean) = true FROM t --- !query 34 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) = true):boolean> --- !query 34 output +-- !query output true --- !query 35 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = true)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = true)' (timestamp and boolean).; line 1 pos 7 --- !query 36 +-- !query SELECT cast('2017-12-11 09:30:00' as date) = true FROM t --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = true)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = true)' (date and boolean).; line 1 pos 7 --- !query 37 +-- !query SELECT cast(1 as tinyint) <=> true FROM t --- !query 37 schema +-- !query schema struct<(CAST(1 AS TINYINT) <=> CAST(true AS TINYINT)):boolean> --- !query 37 output +-- !query output true --- !query 38 +-- !query SELECT cast(1 as smallint) <=> true FROM t --- !query 38 schema +-- !query schema struct<(CAST(1 AS SMALLINT) <=> CAST(true AS SMALLINT)):boolean> --- !query 38 output +-- !query output true --- !query 39 +-- !query SELECT cast(1 as int) <=> true FROM t --- !query 39 schema +-- !query schema struct<(CAST(1 AS INT) <=> CAST(true AS INT)):boolean> --- !query 39 output +-- !query output true --- !query 40 +-- !query SELECT cast(1 as bigint) <=> true FROM t --- !query 40 schema +-- !query schema struct<(CAST(1 AS BIGINT) <=> CAST(true AS BIGINT)):boolean> --- !query 40 output +-- !query output true --- !query 41 +-- !query SELECT cast(1 as float) <=> true FROM t --- !query 41 schema +-- !query schema struct<(CAST(1 AS FLOAT) <=> CAST(true AS FLOAT)):boolean> --- !query 41 output +-- !query output true --- !query 42 +-- !query SELECT cast(1 as double) <=> true FROM t --- !query 42 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <=> CAST(true AS DOUBLE)):boolean> --- !query 42 output +-- !query output true --- !query 43 +-- !query SELECT cast(1 as decimal(10, 0)) <=> true FROM t --- !query 43 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(true AS DECIMAL(10,0))):boolean> --- !query 43 output +-- !query output true --- !query 44 +-- !query SELECT cast(1 as string) <=> true FROM t --- !query 44 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS BOOLEAN) <=> true):boolean> --- !query 44 output +-- !query output true --- !query 45 +-- !query SELECT cast('1' as binary) <=> true FROM t --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <=> true)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> true)' (binary and boolean).; line 1 pos 7 --- !query 46 +-- !query SELECT cast(1 as boolean) <=> true FROM t --- !query 46 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) <=> true):boolean> --- !query 46 output +-- !query output true --- !query 47 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> true)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> true)' (timestamp and boolean).; line 1 pos 7 --- !query 48 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> true)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> true)' (date and boolean).; line 1 pos 7 --- !query 49 +-- !query SELECT false = cast(0 as tinyint) FROM t --- !query 49 schema +-- !query schema struct<(CAST(false AS TINYINT) = CAST(0 AS TINYINT)):boolean> --- !query 49 output +-- !query output true --- !query 50 +-- !query SELECT false = cast(0 as smallint) FROM t --- !query 50 schema +-- !query schema struct<(CAST(false AS SMALLINT) = CAST(0 AS SMALLINT)):boolean> --- !query 50 output +-- !query output true --- !query 51 +-- !query SELECT false = cast(0 as int) FROM t --- !query 51 schema +-- !query schema struct<(CAST(false AS INT) = CAST(0 AS INT)):boolean> --- !query 51 output +-- !query output true --- !query 52 +-- !query SELECT false = cast(0 as bigint) FROM t --- !query 52 schema +-- !query schema struct<(CAST(false AS BIGINT) = CAST(0 AS BIGINT)):boolean> --- !query 52 output +-- !query output true --- !query 53 +-- !query SELECT false = cast(0 as float) FROM t --- !query 53 schema +-- !query schema struct<(CAST(false AS FLOAT) = CAST(0 AS FLOAT)):boolean> --- !query 53 output +-- !query output true --- !query 54 +-- !query SELECT false = cast(0 as double) FROM t --- !query 54 schema +-- !query schema struct<(CAST(false AS DOUBLE) = CAST(0 AS DOUBLE)):boolean> --- !query 54 output +-- !query output true --- !query 55 +-- !query SELECT false = cast(0 as decimal(10, 0)) FROM t --- !query 55 schema +-- !query schema struct<(CAST(false AS DECIMAL(10,0)) = CAST(0 AS DECIMAL(10,0))):boolean> --- !query 55 output +-- !query output true --- !query 56 +-- !query SELECT false = cast(0 as string) FROM t --- !query 56 schema +-- !query schema struct<(false = CAST(CAST(0 AS STRING) AS BOOLEAN)):boolean> --- !query 56 output +-- !query output true --- !query 57 +-- !query SELECT false = cast('0' as binary) FROM t --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(false = CAST('0' AS BINARY))' due to data type mismatch: differing types in '(false = CAST('0' AS BINARY))' (boolean and binary).; line 1 pos 7 --- !query 58 +-- !query SELECT false = cast(0 as boolean) FROM t --- !query 58 schema +-- !query schema struct<(false = CAST(0 AS BOOLEAN)):boolean> --- !query 58 output +-- !query output true --- !query 59 +-- !query SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(false = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(false = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7 --- !query 60 +-- !query SELECT false = cast('2017-12-11 09:30:00' as date) FROM t --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(false = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(false = CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7 --- !query 61 +-- !query SELECT false <=> cast(0 as tinyint) FROM t --- !query 61 schema +-- !query schema struct<(CAST(false AS TINYINT) <=> CAST(0 AS TINYINT)):boolean> --- !query 61 output +-- !query output true --- !query 62 +-- !query SELECT false <=> cast(0 as smallint) FROM t --- !query 62 schema +-- !query schema struct<(CAST(false AS SMALLINT) <=> CAST(0 AS SMALLINT)):boolean> --- !query 62 output +-- !query output true --- !query 63 +-- !query SELECT false <=> cast(0 as int) FROM t --- !query 63 schema +-- !query schema struct<(CAST(false AS INT) <=> CAST(0 AS INT)):boolean> --- !query 63 output +-- !query output true --- !query 64 +-- !query SELECT false <=> cast(0 as bigint) FROM t --- !query 64 schema +-- !query schema struct<(CAST(false AS BIGINT) <=> CAST(0 AS BIGINT)):boolean> --- !query 64 output +-- !query output true --- !query 65 +-- !query SELECT false <=> cast(0 as float) FROM t --- !query 65 schema +-- !query schema struct<(CAST(false AS FLOAT) <=> CAST(0 AS FLOAT)):boolean> --- !query 65 output +-- !query output true --- !query 66 +-- !query SELECT false <=> cast(0 as double) FROM t --- !query 66 schema +-- !query schema struct<(CAST(false AS DOUBLE) <=> CAST(0 AS DOUBLE)):boolean> --- !query 66 output +-- !query output true --- !query 67 +-- !query SELECT false <=> cast(0 as decimal(10, 0)) FROM t --- !query 67 schema +-- !query schema struct<(CAST(false AS DECIMAL(10,0)) <=> CAST(0 AS DECIMAL(10,0))):boolean> --- !query 67 output +-- !query output true --- !query 68 +-- !query SELECT false <=> cast(0 as string) FROM t --- !query 68 schema +-- !query schema struct<(false <=> CAST(CAST(0 AS STRING) AS BOOLEAN)):boolean> --- !query 68 output +-- !query output true --- !query 69 +-- !query SELECT false <=> cast('0' as binary) FROM t --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(false <=> CAST('0' AS BINARY))' due to data type mismatch: differing types in '(false <=> CAST('0' AS BINARY))' (boolean and binary).; line 1 pos 7 --- !query 70 +-- !query SELECT false <=> cast(0 as boolean) FROM t --- !query 70 schema +-- !query schema struct<(false <=> CAST(0 AS BOOLEAN)):boolean> --- !query 70 output +-- !query output true --- !query 71 +-- !query SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(false <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(false <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7 --- !query 72 +-- !query SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(false <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(false <=> CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7 --- !query 73 +-- !query SELECT cast(0 as tinyint) = false FROM t --- !query 73 schema +-- !query schema struct<(CAST(0 AS TINYINT) = CAST(false AS TINYINT)):boolean> --- !query 73 output +-- !query output true --- !query 74 +-- !query SELECT cast(0 as smallint) = false FROM t --- !query 74 schema +-- !query schema struct<(CAST(0 AS SMALLINT) = CAST(false AS SMALLINT)):boolean> --- !query 74 output +-- !query output true --- !query 75 +-- !query SELECT cast(0 as int) = false FROM t --- !query 75 schema +-- !query schema struct<(CAST(0 AS INT) = CAST(false AS INT)):boolean> --- !query 75 output +-- !query output true --- !query 76 +-- !query SELECT cast(0 as bigint) = false FROM t --- !query 76 schema +-- !query schema struct<(CAST(0 AS BIGINT) = CAST(false AS BIGINT)):boolean> --- !query 76 output +-- !query output true --- !query 77 +-- !query SELECT cast(0 as float) = false FROM t --- !query 77 schema +-- !query schema struct<(CAST(0 AS FLOAT) = CAST(false AS FLOAT)):boolean> --- !query 77 output +-- !query output true --- !query 78 +-- !query SELECT cast(0 as double) = false FROM t --- !query 78 schema +-- !query schema struct<(CAST(0 AS DOUBLE) = CAST(false AS DOUBLE)):boolean> --- !query 78 output +-- !query output true --- !query 79 +-- !query SELECT cast(0 as decimal(10, 0)) = false FROM t --- !query 79 schema +-- !query schema struct<(CAST(0 AS DECIMAL(10,0)) = CAST(false AS DECIMAL(10,0))):boolean> --- !query 79 output +-- !query output true --- !query 80 +-- !query SELECT cast(0 as string) = false FROM t --- !query 80 schema +-- !query schema struct<(CAST(CAST(0 AS STRING) AS BOOLEAN) = false):boolean> --- !query 80 output +-- !query output true --- !query 81 +-- !query SELECT cast('0' as binary) = false FROM t --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('0' AS BINARY) = false)' due to data type mismatch: differing types in '(CAST('0' AS BINARY) = false)' (binary and boolean).; line 1 pos 7 --- !query 82 +-- !query SELECT cast(0 as boolean) = false FROM t --- !query 82 schema +-- !query schema struct<(CAST(0 AS BOOLEAN) = false):boolean> --- !query 82 output +-- !query output true --- !query 83 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = false)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = false)' (timestamp and boolean).; line 1 pos 7 --- !query 84 +-- !query SELECT cast('2017-12-11 09:30:00' as date) = false FROM t --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = false)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = false)' (date and boolean).; line 1 pos 7 --- !query 85 +-- !query SELECT cast(0 as tinyint) <=> false FROM t --- !query 85 schema +-- !query schema struct<(CAST(0 AS TINYINT) <=> CAST(false AS TINYINT)):boolean> --- !query 85 output +-- !query output true --- !query 86 +-- !query SELECT cast(0 as smallint) <=> false FROM t --- !query 86 schema +-- !query schema struct<(CAST(0 AS SMALLINT) <=> CAST(false AS SMALLINT)):boolean> --- !query 86 output +-- !query output true --- !query 87 +-- !query SELECT cast(0 as int) <=> false FROM t --- !query 87 schema +-- !query schema struct<(CAST(0 AS INT) <=> CAST(false AS INT)):boolean> --- !query 87 output +-- !query output true --- !query 88 +-- !query SELECT cast(0 as bigint) <=> false FROM t --- !query 88 schema +-- !query schema struct<(CAST(0 AS BIGINT) <=> CAST(false AS BIGINT)):boolean> --- !query 88 output +-- !query output true --- !query 89 +-- !query SELECT cast(0 as float) <=> false FROM t --- !query 89 schema +-- !query schema struct<(CAST(0 AS FLOAT) <=> CAST(false AS FLOAT)):boolean> --- !query 89 output +-- !query output true --- !query 90 +-- !query SELECT cast(0 as double) <=> false FROM t --- !query 90 schema +-- !query schema struct<(CAST(0 AS DOUBLE) <=> CAST(false AS DOUBLE)):boolean> --- !query 90 output +-- !query output true --- !query 91 +-- !query SELECT cast(0 as decimal(10, 0)) <=> false FROM t --- !query 91 schema +-- !query schema struct<(CAST(0 AS DECIMAL(10,0)) <=> CAST(false AS DECIMAL(10,0))):boolean> --- !query 91 output +-- !query output true --- !query 92 +-- !query SELECT cast(0 as string) <=> false FROM t --- !query 92 schema +-- !query schema struct<(CAST(CAST(0 AS STRING) AS BOOLEAN) <=> false):boolean> --- !query 92 output +-- !query output true --- !query 93 +-- !query SELECT cast('0' as binary) <=> false FROM t --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('0' AS BINARY) <=> false)' due to data type mismatch: differing types in '(CAST('0' AS BINARY) <=> false)' (binary and boolean).; line 1 pos 7 --- !query 94 +-- !query SELECT cast(0 as boolean) <=> false FROM t --- !query 94 schema +-- !query schema struct<(CAST(0 AS BOOLEAN) <=> false):boolean> --- !query 94 output +-- !query output true --- !query 95 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t --- !query 95 schema +-- !query schema struct<> --- !query 95 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> false)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> false)' (timestamp and boolean).; line 1 pos 7 --- !query 96 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t --- !query 96 schema +-- !query schema struct<> --- !query 96 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> false)' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> false)' (date and boolean).; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out index 1e1cbc3304141..18d97c2f1b42a 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out @@ -2,1231 +2,1231 @@ -- Number of queries: 145 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as tinyint) END FROM t --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 --- !query 2 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as smallint) END FROM t --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 --- !query 3 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as int) END FROM t --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 --- !query 4 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as bigint) END FROM t --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 --- !query 5 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as float) END FROM t --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1.0 --- !query 6 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as double) END FROM t --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1.0 --- !query 7 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 --- !query 8 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as string) END FROM t --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 --- !query 9 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END FROM t --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE binary END; line 1 pos 7 --- !query 10 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END FROM t --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE boolean END; line 1 pos 7 --- !query 11 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE timestamp END; line 1 pos 7 --- !query 12 +-- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE date END; line 1 pos 7 --- !query 13 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as tinyint) END FROM t --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 --- !query 14 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as smallint) END FROM t --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 --- !query 15 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as int) END FROM t --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 --- !query 16 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as bigint) END FROM t --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1 --- !query 17 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as float) END FROM t --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1.0 --- !query 18 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as double) END FROM t --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1.0 --- !query 19 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1 --- !query 20 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as string) END FROM t --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1 --- !query 21 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END FROM t --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE binary END; line 1 pos 7 --- !query 22 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END FROM t --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE boolean END; line 1 pos 7 --- !query 23 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE timestamp END; line 1 pos 7 --- !query 24 +-- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE date END; line 1 pos 7 --- !query 25 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as tinyint) END FROM t --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1 --- !query 26 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as smallint) END FROM t --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 --- !query 27 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as int) END FROM t --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 1 --- !query 28 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as bigint) END FROM t --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 1 --- !query 29 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as float) END FROM t --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 1.0 --- !query 30 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as double) END FROM t --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1.0 --- !query 31 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 1 --- !query 32 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as string) END FROM t --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 1 --- !query 33 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2' as binary) END FROM t --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE binary END; line 1 pos 7 --- !query 34 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END FROM t --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE boolean END; line 1 pos 7 --- !query 35 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE timestamp END; line 1 pos 7 --- !query 36 +-- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE date END; line 1 pos 7 --- !query 37 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as tinyint) END FROM t --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 1 --- !query 38 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as smallint) END FROM t --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 1 --- !query 39 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as int) END FROM t --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 1 --- !query 40 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as bigint) END FROM t --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 1 --- !query 41 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as float) END FROM t --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 1.0 --- !query 42 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as double) END FROM t --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 1.0 --- !query 43 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 1 --- !query 44 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as string) END FROM t --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 1 --- !query 45 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2' as binary) END FROM t --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE binary END; line 1 pos 7 --- !query 46 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END FROM t --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE boolean END; line 1 pos 7 --- !query 47 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE timestamp END; line 1 pos 7 --- !query 48 +-- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE date END; line 1 pos 7 --- !query 49 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as tinyint) END FROM t --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output 1.0 --- !query 50 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as smallint) END FROM t --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output 1.0 --- !query 51 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as int) END FROM t --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output 1.0 --- !query 52 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as bigint) END FROM t --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output 1.0 --- !query 53 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as float) END FROM t --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 1.0 --- !query 54 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as double) END FROM t --- !query 54 schema +-- !query schema struct --- !query 54 output +-- !query output 1.0 --- !query 55 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 55 schema +-- !query schema struct --- !query 55 output +-- !query output 1.0 --- !query 56 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as string) END FROM t --- !query 56 schema +-- !query schema struct --- !query 56 output +-- !query output 1.0 --- !query 57 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2' as binary) END FROM t --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE binary END; line 1 pos 7 --- !query 58 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END FROM t --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE boolean END; line 1 pos 7 --- !query 59 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE timestamp END; line 1 pos 7 --- !query 60 +-- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE date END; line 1 pos 7 --- !query 61 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as tinyint) END FROM t --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output 1.0 --- !query 62 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as smallint) END FROM t --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output 1.0 --- !query 63 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as int) END FROM t --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output 1.0 --- !query 64 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as bigint) END FROM t --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 1.0 --- !query 65 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as float) END FROM t --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output 1.0 --- !query 66 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as double) END FROM t --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output 1.0 --- !query 67 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output 1.0 --- !query 68 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as string) END FROM t --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 1.0 --- !query 69 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2' as binary) END FROM t --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE binary END; line 1 pos 7 --- !query 70 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END FROM t --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE boolean END; line 1 pos 7 --- !query 71 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE timestamp END; line 1 pos 7 --- !query 72 +-- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE date END; line 1 pos 7 --- !query 73 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as tinyint) END FROM t --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output 1 --- !query 74 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as smallint) END FROM t --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output 1 --- !query 75 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as int) END FROM t --- !query 75 schema +-- !query schema struct --- !query 75 output +-- !query output 1 --- !query 76 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as bigint) END FROM t --- !query 76 schema +-- !query schema struct --- !query 76 output +-- !query output 1 --- !query 77 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as float) END FROM t --- !query 77 schema +-- !query schema struct --- !query 77 output +-- !query output 1.0 --- !query 78 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as double) END FROM t --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output 1.0 --- !query 79 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output 1 --- !query 80 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as string) END FROM t --- !query 80 schema +-- !query schema struct --- !query 80 output +-- !query output 1 --- !query 81 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2' as binary) END FROM t --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE binary END; line 1 pos 7 --- !query 82 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END FROM t --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE boolean END; line 1 pos 7 --- !query 83 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE timestamp END; line 1 pos 7 --- !query 84 +-- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE date END; line 1 pos 7 --- !query 85 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as tinyint) END FROM t --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output 1 --- !query 86 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as smallint) END FROM t --- !query 86 schema +-- !query schema struct --- !query 86 output +-- !query output 1 --- !query 87 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as int) END FROM t --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 1 --- !query 88 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as bigint) END FROM t --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output 1 --- !query 89 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as float) END FROM t --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output 1 --- !query 90 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as double) END FROM t --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 1 --- !query 91 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output 1 --- !query 92 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as string) END FROM t --- !query 92 schema +-- !query schema struct --- !query 92 output +-- !query output 1 --- !query 93 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN string ELSE binary END; line 1 pos 7 --- !query 94 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN string ELSE boolean END; line 1 pos 7 --- !query 95 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 95 schema +-- !query schema struct --- !query 95 output +-- !query output 1 --- !query 96 +-- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 96 schema +-- !query schema struct --- !query 96 output +-- !query output 1 --- !query 97 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END FROM t --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE tinyint END; line 1 pos 7 --- !query 98 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END FROM t --- !query 98 schema +-- !query schema struct<> --- !query 98 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE smallint END; line 1 pos 7 --- !query 99 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END FROM t --- !query 99 schema +-- !query schema struct<> --- !query 99 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE int END; line 1 pos 7 --- !query 100 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END FROM t --- !query 100 schema +-- !query schema struct<> --- !query 100 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE bigint END; line 1 pos 7 --- !query 101 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END FROM t --- !query 101 schema +-- !query schema struct<> --- !query 101 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE float END; line 1 pos 7 --- !query 102 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END FROM t --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE double END; line 1 pos 7 --- !query 103 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE decimal(10,0) END; line 1 pos 7 --- !query 104 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t --- !query 104 schema +-- !query schema struct<> --- !query 104 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS STRING) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE string END; line 1 pos 7 --- !query 105 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2' as binary) END FROM t --- !query 105 schema +-- !query schema struct --- !query 105 output +-- !query output 1 --- !query 106 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as boolean) END FROM t --- !query 106 schema +-- !query schema struct<> --- !query 106 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE boolean END; line 1 pos 7 --- !query 107 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE timestamp END; line 1 pos 7 --- !query 108 +-- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE date END; line 1 pos 7 --- !query 109 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END FROM t --- !query 109 schema +-- !query schema struct<> --- !query 109 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE tinyint END; line 1 pos 7 --- !query 110 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END FROM t --- !query 110 schema +-- !query schema struct<> --- !query 110 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE smallint END; line 1 pos 7 --- !query 111 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END FROM t --- !query 111 schema +-- !query schema struct<> --- !query 111 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE int END; line 1 pos 7 --- !query 112 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END FROM t --- !query 112 schema +-- !query schema struct<> --- !query 112 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE bigint END; line 1 pos 7 --- !query 113 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END FROM t --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE float END; line 1 pos 7 --- !query 114 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END FROM t --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE double END; line 1 pos 7 --- !query 115 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE decimal(10,0) END; line 1 pos 7 --- !query 116 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS STRING) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE string END; line 1 pos 7 --- !query 117 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END FROM t --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE binary END; line 1 pos 7 --- !query 118 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as boolean) END FROM t --- !query 118 schema +-- !query schema struct --- !query 118 output +-- !query output true --- !query 119 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 119 schema +-- !query schema struct<> --- !query 119 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE timestamp END; line 1 pos 7 --- !query 120 +-- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE date END; line 1 pos 7 --- !query 121 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as tinyint) END FROM t --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE tinyint END; line 1 pos 7 --- !query 122 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as smallint) END FROM t --- !query 122 schema +-- !query schema struct<> --- !query 122 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE smallint END; line 1 pos 7 --- !query 123 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as int) END FROM t --- !query 123 schema +-- !query schema struct<> --- !query 123 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE int END; line 1 pos 7 --- !query 124 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as bigint) END FROM t --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE bigint END; line 1 pos 7 --- !query 125 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as float) END FROM t --- !query 125 schema +-- !query schema struct<> --- !query 125 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE float END; line 1 pos 7 --- !query 126 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as double) END FROM t --- !query 126 schema +-- !query schema struct<> --- !query 126 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE double END; line 1 pos 7 --- !query 127 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE decimal(10,0) END; line 1 pos 7 --- !query 128 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as string) END FROM t --- !query 128 schema +-- !query schema struct --- !query 128 output +-- !query output 2017-12-12 09:30:00 --- !query 129 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2' as binary) END FROM t --- !query 129 schema +-- !query schema struct<> --- !query 129 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE binary END; line 1 pos 7 --- !query 130 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as boolean) END FROM t --- !query 130 schema +-- !query schema struct<> --- !query 130 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE boolean END; line 1 pos 7 --- !query 131 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 131 schema +-- !query schema struct --- !query 131 output +-- !query output 2017-12-12 09:30:00 --- !query 132 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 132 schema +-- !query schema struct --- !query 132 output +-- !query output 2017-12-12 09:30:00 --- !query 133 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as tinyint) END FROM t --- !query 133 schema +-- !query schema struct<> --- !query 133 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE tinyint END; line 1 pos 7 --- !query 134 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as smallint) END FROM t --- !query 134 schema +-- !query schema struct<> --- !query 134 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE smallint END; line 1 pos 7 --- !query 135 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as int) END FROM t --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE int END; line 1 pos 7 --- !query 136 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bigint) END FROM t --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE bigint END; line 1 pos 7 --- !query 137 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as float) END FROM t --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE float END; line 1 pos 7 --- !query 138 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as double) END FROM t --- !query 138 schema +-- !query schema struct<> --- !query 138 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE double END; line 1 pos 7 --- !query 139 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as decimal(10, 0)) END FROM t --- !query 139 schema +-- !query schema struct<> --- !query 139 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE decimal(10,0) END; line 1 pos 7 --- !query 140 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as string) END FROM t --- !query 140 schema +-- !query schema struct --- !query 140 output +-- !query output 2017-12-12 --- !query 141 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2' as binary) END FROM t --- !query 141 schema +-- !query schema struct<> --- !query 141 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE binary END; line 1 pos 7 --- !query 142 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as boolean) END FROM t --- !query 142 schema +-- !query schema struct<> --- !query 142 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE boolean END; line 1 pos 7 --- !query 143 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query 143 schema +-- !query schema struct --- !query 143 output +-- !query output 2017-12-12 00:00:00 --- !query 144 +-- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query 144 schema +-- !query schema struct --- !query 144 output +-- !query output 2017-12-12 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out index d7ebc3c77ed5d..bd157c474d249 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 14 --- !query 0 +-- !query SELECT (col1 || col2 || col3) col FROM ( SELECT @@ -11,9 +11,9 @@ FROM ( encode(string(id + 2), 'utf-8') col3 FROM range(10) ) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 012 123 234 @@ -26,7 +26,7 @@ struct 91011 --- !query 1 +-- !query SELECT ((col1 || col2) || (col3 || col4) || col5) col FROM ( SELECT @@ -37,9 +37,9 @@ FROM ( CAST(id AS DOUBLE) col5 FROM range(10) ) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output prefix_0120.0 prefix_1231.0 prefix_2342.0 @@ -52,7 +52,7 @@ prefix_89108.0 prefix_910119.0 --- !query 2 +-- !query SELECT ((col1 || col2) || (col3 || col4)) col FROM ( SELECT @@ -62,9 +62,9 @@ FROM ( encode(string(id + 3), 'utf-8') col4 FROM range(10) ) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 0123 1234 2345 @@ -77,15 +77,15 @@ struct 9101112 --- !query 3 +-- !query set spark.sql.function.concatBinaryAsString=true --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output spark.sql.function.concatBinaryAsString true --- !query 4 +-- !query SELECT (col1 || col2) col FROM ( SELECT @@ -93,9 +93,9 @@ FROM ( encode(string(id + 1), 'utf-8') col2 FROM range(10) ) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 01 12 23 @@ -108,7 +108,7 @@ struct 910 --- !query 5 +-- !query SELECT (col1 || col2 || col3 || col4) col FROM ( SELECT @@ -118,9 +118,9 @@ FROM ( encode(string(id + 3), 'utf-8') col4 FROM range(10) ) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 0123 1234 2345 @@ -133,7 +133,7 @@ struct 9101112 --- !query 6 +-- !query SELECT ((col1 || col2) || (col3 || col4)) col FROM ( SELECT @@ -143,9 +143,9 @@ FROM ( encode(string(id + 3), 'utf-8') col4 FROM range(10) ) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 0123 1234 2345 @@ -158,15 +158,15 @@ struct 9101112 --- !query 7 +-- !query set spark.sql.function.concatBinaryAsString=false --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output spark.sql.function.concatBinaryAsString false --- !query 8 +-- !query SELECT (col1 || col2) col FROM ( SELECT @@ -174,9 +174,9 @@ FROM ( encode(string(id + 1), 'utf-8') col2 FROM range(10) ) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 01 12 23 @@ -189,7 +189,7 @@ struct 910 --- !query 9 +-- !query SELECT (col1 || col2 || col3 || col4) col FROM ( SELECT @@ -199,9 +199,9 @@ FROM ( encode(string(id + 3), 'utf-8') col4 FROM range(10) ) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 0123 1234 2345 @@ -214,7 +214,7 @@ struct 9101112 --- !query 10 +-- !query SELECT ((col1 || col2) || (col3 || col4)) col FROM ( SELECT @@ -224,9 +224,9 @@ FROM ( encode(string(id + 3), 'utf-8') col4 FROM range(10) ) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 0123 1234 2345 @@ -239,7 +239,7 @@ struct 9101112 --- !query 11 +-- !query CREATE TEMPORARY VIEW various_arrays AS SELECT * FROM VALUES ( array(true, false), array(true), array(2Y, 1Y), array(3Y, 4Y), @@ -272,13 +272,13 @@ CREATE TEMPORARY VIEW various_arrays AS SELECT * FROM VALUES ( struct_array1, struct_array2, map_array1, map_array2 ) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query SELECT (boolean_array1 || boolean_array2) boolean_array, (tinyint_array1 || tinyint_array2) tinyint_array, @@ -295,13 +295,13 @@ SELECT (struct_array1 || struct_array2) struct_array, (map_array1 || map_array2) map_array FROM various_arrays --- !query 12 schema +-- !query schema struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,data_array:array,timestamp_array:array,string_array:array,array_array:array>,struct_array:array>,map_array:array>> --- !query 12 output +-- !query output [true,false,true] [2,1,3,4] [2,1,3,4] [2,1,3,4] [2,1,3,4] [9223372036854775809,9223372036854775808,9223372036854775808,9223372036854775809] [2.0,1.0,3.0,4.0] [2.0,1.0,3.0,4.0] [2016-03-14,2016-03-13,2016-03-12,2016-03-11] [2016-11-15 20:54:00,2016-11-12 20:54:00,2016-11-11 20:54:00] ["a","b","c","d"] [["a","b"],["c","d"],["e"],["f"]] [{"col1":"a","col2":1},{"col1":"b","col2":2},{"col1":"c","col2":3},{"col1":"d","col2":4}] [{"a":1},{"b":2},{"c":3},{"d":4}] --- !query 13 +-- !query SELECT (tinyint_array1 || smallint_array2) ts_array, (smallint_array1 || int_array2) si_array, @@ -313,7 +313,7 @@ SELECT (timestamp_array1 || string_array2) tst_array, (string_array1 || int_array2) sti_array FROM various_arrays --- !query 13 schema +-- !query schema struct,si_array:array,ib_array:array,bd_array:array,dd_array:array,df_array:array,std_array:array,tst_array:array,sti_array:array> --- !query 13 output +-- !query output [2,1,3,4] [2,1,3,4] [2,1,3,4] [2,1,9223372036854775808,9223372036854775809] [9.223372036854776E18,9.223372036854776E18,3.0,4.0] [2.0,1.0,3.0,4.0] ["a","b","2016-03-12","2016-03-11"] ["2016-11-15 20:54:00","2016-11-12 20:54:00","c","d"] ["a","b","3","4"] diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out index 07bdfe560a425..d5c27ade8e152 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out @@ -2,348 +2,348 @@ -- Number of queries: 40 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select cast(1 as tinyint) + interval 2 day --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 --- !query 2 +-- !query select cast(1 as smallint) + interval 2 day --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 --- !query 3 +-- !query select cast(1 as int) + interval 2 day --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 --- !query 4 +-- !query select cast(1 as bigint) + interval 2 day --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 --- !query 5 +-- !query select cast(1 as float) + interval 2 day --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 --- !query 6 +-- !query select cast(1 as double) + interval 2 day --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 --- !query 7 +-- !query select cast(1 as decimal(10, 0)) + interval 2 day --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 8 +-- !query select cast('2017-12-11' as string) + interval 2 day --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 2017-12-13 00:00:00 --- !query 9 +-- !query select cast('2017-12-11 09:30:00' as string) + interval 2 day --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 2017-12-13 09:30:00 --- !query 10 +-- !query select cast('1' as binary) + interval 2 day --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 --- !query 11 +-- !query select cast(1 as boolean) + interval 2 day --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 --- !query 12 +-- !query select cast('2017-12-11 09:30:00.0' as timestamp) + interval 2 day --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 2017-12-13 09:30:00 --- !query 13 +-- !query select cast('2017-12-11 09:30:00' as date) + interval 2 day --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 2017-12-13 --- !query 14 +-- !query select interval 2 day + cast(1 as tinyint) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS TINYINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 --- !query 15 +-- !query select interval 2 day + cast(1 as smallint) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS SMALLINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 --- !query 16 +-- !query select interval 2 day + cast(1 as int) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS INT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 --- !query 17 +-- !query select interval 2 day + cast(1 as bigint) --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS BIGINT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 --- !query 18 +-- !query select interval 2 day + cast(1 as float) --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS FLOAT) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 --- !query 19 +-- !query select interval 2 day + cast(1 as double) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS DOUBLE) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 --- !query 20 +-- !query select interval 2 day + cast(1 as decimal(10, 0)) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS DECIMAL(10,0)) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 21 +-- !query select interval 2 day + cast('2017-12-11' as string) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 2017-12-13 00:00:00 --- !query 22 +-- !query select interval 2 day + cast('2017-12-11 09:30:00' as string) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 2017-12-13 09:30:00 --- !query 23 +-- !query select interval 2 day + cast('1' as binary) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST('1' AS BINARY) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 --- !query 24 +-- !query select interval 2 day + cast(1 as boolean) --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS BOOLEAN) + INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 --- !query 25 +-- !query select interval 2 day + cast('2017-12-11 09:30:00.0' as timestamp) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 2017-12-13 09:30:00 --- !query 26 +-- !query select interval 2 day + cast('2017-12-11 09:30:00' as date) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 2017-12-13 --- !query 27 +-- !query select cast(1 as tinyint) - interval 2 day --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS TINYINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS TINYINT)' is of tinyint type.; line 1 pos 7 --- !query 28 +-- !query select cast(1 as smallint) - interval 2 day --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS SMALLINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS SMALLINT)' is of smallint type.; line 1 pos 7 --- !query 29 +-- !query select cast(1 as int) - interval 2 day --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS INT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS INT)' is of int type.; line 1 pos 7 --- !query 30 +-- !query select cast(1 as bigint) - interval 2 day --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS BIGINT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BIGINT)' is of bigint type.; line 1 pos 7 --- !query 31 +-- !query select cast(1 as float) - interval 2 day --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS FLOAT) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS FLOAT)' is of float type.; line 1 pos 7 --- !query 32 +-- !query select cast(1 as double) - interval 2 day --- !query 32 schema +-- !query schema struct<> --- !query 32 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS DOUBLE) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DOUBLE)' is of double type.; line 1 pos 7 --- !query 33 +-- !query select cast(1 as decimal(10, 0)) - interval 2 day --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS DECIMAL(10,0)) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 34 +-- !query select cast('2017-12-11' as string) - interval 2 day --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 2017-12-09 00:00:00 --- !query 35 +-- !query select cast('2017-12-11 09:30:00' as string) - interval 2 day --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 2017-12-09 09:30:00 --- !query 36 +-- !query select cast('1' as binary) - interval 2 day --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST('1' AS BINARY) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST('1' AS BINARY)' is of binary type.; line 1 pos 7 --- !query 37 +-- !query select cast(1 as boolean) - interval 2 day --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'CAST(1 AS BOOLEAN) - INTERVAL '2 days'' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS BOOLEAN)' is of boolean type.; line 1 pos 7 --- !query 38 +-- !query select cast('2017-12-11 09:30:00.0' as timestamp) - interval 2 day --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 2017-12-09 09:30:00 --- !query 39 +-- !query select cast('2017-12-11 09:30:00' as date) - interval 2 day --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 2017-12-09 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index ec6f3d7696b03..33bd3850732f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -2,9513 +2,9513 @@ -- Number of queries: 1145 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT cast(1 as tinyint) + cast(1 as decimal(3, 0)) FROM t --- !query 1 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) + CAST(1 AS DECIMAL(3,0))):decimal(4,0)> --- !query 1 output +-- !query output 2 --- !query 2 +-- !query SELECT cast(1 as tinyint) + cast(1 as decimal(5, 0)) FROM t --- !query 2 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0)) + CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)> --- !query 2 output +-- !query output 2 --- !query 3 +-- !query SELECT cast(1 as tinyint) + cast(1 as decimal(10, 0)) FROM t --- !query 3 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 3 output +-- !query output 2 --- !query 4 +-- !query SELECT cast(1 as tinyint) + cast(1 as decimal(20, 0)) FROM t --- !query 4 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 4 output +-- !query output 2 --- !query 5 +-- !query SELECT cast(1 as smallint) + cast(1 as decimal(3, 0)) FROM t --- !query 5 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0)) + CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)> --- !query 5 output +-- !query output 2 --- !query 6 +-- !query SELECT cast(1 as smallint) + cast(1 as decimal(5, 0)) FROM t --- !query 6 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) + CAST(1 AS DECIMAL(5,0))):decimal(6,0)> --- !query 6 output +-- !query output 2 --- !query 7 +-- !query SELECT cast(1 as smallint) + cast(1 as decimal(10, 0)) FROM t --- !query 7 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 7 output +-- !query output 2 --- !query 8 +-- !query SELECT cast(1 as smallint) + cast(1 as decimal(20, 0)) FROM t --- !query 8 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 8 output +-- !query output 2 --- !query 9 +-- !query SELECT cast(1 as int) + cast(1 as decimal(3, 0)) FROM t --- !query 9 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 9 output +-- !query output 2 --- !query 10 +-- !query SELECT cast(1 as int) + cast(1 as decimal(5, 0)) FROM t --- !query 10 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 10 output +-- !query output 2 --- !query 11 +-- !query SELECT cast(1 as int) + cast(1 as decimal(10, 0)) FROM t --- !query 11 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> --- !query 11 output +-- !query output 2 --- !query 12 +-- !query SELECT cast(1 as int) + cast(1 as decimal(20, 0)) FROM t --- !query 12 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 12 output +-- !query output 2 --- !query 13 +-- !query SELECT cast(1 as bigint) + cast(1 as decimal(3, 0)) FROM t --- !query 13 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 13 output +-- !query output 2 --- !query 14 +-- !query SELECT cast(1 as bigint) + cast(1 as decimal(5, 0)) FROM t --- !query 14 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 14 output +-- !query output 2 --- !query 15 +-- !query SELECT cast(1 as bigint) + cast(1 as decimal(10, 0)) FROM t --- !query 15 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 15 output +-- !query output 2 --- !query 16 +-- !query SELECT cast(1 as bigint) + cast(1 as decimal(20, 0)) FROM t --- !query 16 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) + CAST(1 AS DECIMAL(20,0))):decimal(21,0)> --- !query 16 output +-- !query output 2 --- !query 17 +-- !query SELECT cast(1 as float) + cast(1 as decimal(3, 0)) FROM t --- !query 17 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 17 output +-- !query output 2.0 --- !query 18 +-- !query SELECT cast(1 as float) + cast(1 as decimal(5, 0)) FROM t --- !query 18 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 18 output +-- !query output 2.0 --- !query 19 +-- !query SELECT cast(1 as float) + cast(1 as decimal(10, 0)) FROM t --- !query 19 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 19 output +-- !query output 2.0 --- !query 20 +-- !query SELECT cast(1 as float) + cast(1 as decimal(20, 0)) FROM t --- !query 20 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 20 output +-- !query output 2.0 --- !query 21 +-- !query SELECT cast(1 as double) + cast(1 as decimal(3, 0)) FROM t --- !query 21 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 21 output +-- !query output 2.0 --- !query 22 +-- !query SELECT cast(1 as double) + cast(1 as decimal(5, 0)) FROM t --- !query 22 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 22 output +-- !query output 2.0 --- !query 23 +-- !query SELECT cast(1 as double) + cast(1 as decimal(10, 0)) FROM t --- !query 23 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 23 output +-- !query output 2.0 --- !query 24 +-- !query SELECT cast(1 as double) + cast(1 as decimal(20, 0)) FROM t --- !query 24 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 24 output +-- !query output 2.0 --- !query 25 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(3, 0)) FROM t --- !query 25 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 25 output +-- !query output 2 --- !query 26 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(5, 0)) FROM t --- !query 26 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 26 output +-- !query output 2 --- !query 27 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t --- !query 27 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> --- !query 27 output +-- !query output 2 --- !query 28 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(20, 0)) FROM t --- !query 28 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 28 output +-- !query output 2 --- !query 29 +-- !query SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 30 +-- !query SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 31 +-- !query SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t --- !query 31 schema +-- !query schema struct<> --- !query 31 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 32 +-- !query SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t --- !query 32 schema +-- !query schema struct<> --- !query 32 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 33 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 34 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 35 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 36 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 37 +-- !query SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 --- !query 38 +-- !query SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t --- !query 38 schema +-- !query schema struct<> --- !query 38 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 --- !query 39 +-- !query SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 40 +-- !query SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 --- !query 41 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as tinyint) FROM t --- !query 41 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) + CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(4,0)> --- !query 41 output +-- !query output 2 --- !query 42 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as tinyint) FROM t --- !query 42 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0)) + CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)> --- !query 42 output +-- !query output 2 --- !query 43 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as tinyint) FROM t --- !query 43 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 43 output +-- !query output 2 --- !query 44 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as tinyint) FROM t --- !query 44 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 44 output +-- !query output 2 --- !query 45 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as smallint) FROM t --- !query 45 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0)) + CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)> --- !query 45 output +-- !query output 2 --- !query 46 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as smallint) FROM t --- !query 46 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) + CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(6,0)> --- !query 46 output +-- !query output 2 --- !query 47 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as smallint) FROM t --- !query 47 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) + CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 47 output +-- !query output 2 --- !query 48 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as smallint) FROM t --- !query 48 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 48 output +-- !query output 2 --- !query 49 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as int) FROM t --- !query 49 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0)) + CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 49 output +-- !query output 2 --- !query 50 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as int) FROM t --- !query 50 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) + CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 50 output +-- !query output 2 --- !query 51 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as int) FROM t --- !query 51 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) + CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(11,0)> --- !query 51 output +-- !query output 2 --- !query 52 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as int) FROM t --- !query 52 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 52 output +-- !query output 2 --- !query 53 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as bigint) FROM t --- !query 53 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 53 output +-- !query output 2 --- !query 54 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as bigint) FROM t --- !query 54 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 54 output +-- !query output 2 --- !query 55 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as bigint) FROM t --- !query 55 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0)) + CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 55 output +-- !query output 2 --- !query 56 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as bigint) FROM t --- !query 56 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) + CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(21,0)> --- !query 56 output +-- !query output 2 --- !query 57 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as float) FROM t --- !query 57 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 57 output +-- !query output 2.0 --- !query 58 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as float) FROM t --- !query 58 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 58 output +-- !query output 2.0 --- !query 59 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as float) FROM t --- !query 59 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 59 output +-- !query output 2.0 --- !query 60 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as float) FROM t --- !query 60 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 60 output +-- !query output 2.0 --- !query 61 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as double) FROM t --- !query 61 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 61 output +-- !query output 2.0 --- !query 62 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as double) FROM t --- !query 62 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 62 output +-- !query output 2.0 --- !query 63 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as double) FROM t --- !query 63 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 63 output +-- !query output 2.0 --- !query 64 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as double) FROM t --- !query 64 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 64 output +-- !query output 2.0 --- !query 65 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as decimal(10, 0)) FROM t --- !query 65 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 65 output +-- !query output 2 --- !query 66 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as decimal(10, 0)) FROM t --- !query 66 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 66 output +-- !query output 2 --- !query 67 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t --- !query 67 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> --- !query 67 output +-- !query output 2 --- !query 68 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as decimal(10, 0)) FROM t --- !query 68 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) + CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 68 output +-- !query output 2 --- !query 69 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as string) FROM t --- !query 69 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) + CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 69 output +-- !query output 2.0 --- !query 70 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as string) FROM t --- !query 70 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) + CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 70 output +-- !query output 2.0 --- !query 71 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as string) FROM t --- !query 71 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) + CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 71 output +-- !query output 2.0 --- !query 72 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as string) FROM t --- !query 72 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) + CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 72 output +-- !query output 2.0 --- !query 73 +-- !query SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t --- !query 73 schema +-- !query schema struct<> --- !query 73 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 74 +-- !query SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t --- !query 74 schema +-- !query schema struct<> --- !query 74 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 75 +-- !query SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t --- !query 75 schema +-- !query schema struct<> --- !query 75 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 76 +-- !query SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t --- !query 76 schema +-- !query schema struct<> --- !query 76 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 77 +-- !query SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t --- !query 77 schema +-- !query schema struct<> --- !query 77 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 78 +-- !query SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t --- !query 78 schema +-- !query schema struct<> --- !query 78 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 79 +-- !query SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t --- !query 79 schema +-- !query schema struct<> --- !query 79 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 80 +-- !query SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t --- !query 80 schema +-- !query schema struct<> --- !query 80 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 81 +-- !query SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 82 +-- !query SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 83 +-- !query SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 84 +-- !query SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 85 +-- !query SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t --- !query 85 schema +-- !query schema struct<> --- !query 85 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 --- !query 86 +-- !query SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t --- !query 86 schema +-- !query schema struct<> --- !query 86 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 --- !query 87 +-- !query SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t --- !query 87 schema +-- !query schema struct<> --- !query 87 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 88 +-- !query SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t --- !query 88 schema +-- !query schema struct<> --- !query 88 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 --- !query 89 +-- !query SELECT cast(1 as tinyint) - cast(1 as decimal(3, 0)) FROM t --- !query 89 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) - CAST(1 AS DECIMAL(3,0))):decimal(4,0)> --- !query 89 output +-- !query output 0 --- !query 90 +-- !query SELECT cast(1 as tinyint) - cast(1 as decimal(5, 0)) FROM t --- !query 90 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0)) - CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)> --- !query 90 output +-- !query output 0 --- !query 91 +-- !query SELECT cast(1 as tinyint) - cast(1 as decimal(10, 0)) FROM t --- !query 91 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 91 output +-- !query output 0 --- !query 92 +-- !query SELECT cast(1 as tinyint) - cast(1 as decimal(20, 0)) FROM t --- !query 92 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 92 output +-- !query output 0 --- !query 93 +-- !query SELECT cast(1 as smallint) - cast(1 as decimal(3, 0)) FROM t --- !query 93 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0)) - CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)> --- !query 93 output +-- !query output 0 --- !query 94 +-- !query SELECT cast(1 as smallint) - cast(1 as decimal(5, 0)) FROM t --- !query 94 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) - CAST(1 AS DECIMAL(5,0))):decimal(6,0)> --- !query 94 output +-- !query output 0 --- !query 95 +-- !query SELECT cast(1 as smallint) - cast(1 as decimal(10, 0)) FROM t --- !query 95 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 95 output +-- !query output 0 --- !query 96 +-- !query SELECT cast(1 as smallint) - cast(1 as decimal(20, 0)) FROM t --- !query 96 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 96 output +-- !query output 0 --- !query 97 +-- !query SELECT cast(1 as int) - cast(1 as decimal(3, 0)) FROM t --- !query 97 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 97 output +-- !query output 0 --- !query 98 +-- !query SELECT cast(1 as int) - cast(1 as decimal(5, 0)) FROM t --- !query 98 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 98 output +-- !query output 0 --- !query 99 +-- !query SELECT cast(1 as int) - cast(1 as decimal(10, 0)) FROM t --- !query 99 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> --- !query 99 output +-- !query output 0 --- !query 100 +-- !query SELECT cast(1 as int) - cast(1 as decimal(20, 0)) FROM t --- !query 100 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 100 output +-- !query output 0 --- !query 101 +-- !query SELECT cast(1 as bigint) - cast(1 as decimal(3, 0)) FROM t --- !query 101 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 101 output +-- !query output 0 --- !query 102 +-- !query SELECT cast(1 as bigint) - cast(1 as decimal(5, 0)) FROM t --- !query 102 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 102 output +-- !query output 0 --- !query 103 +-- !query SELECT cast(1 as bigint) - cast(1 as decimal(10, 0)) FROM t --- !query 103 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 103 output +-- !query output 0 --- !query 104 +-- !query SELECT cast(1 as bigint) - cast(1 as decimal(20, 0)) FROM t --- !query 104 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) - CAST(1 AS DECIMAL(20,0))):decimal(21,0)> --- !query 104 output +-- !query output 0 --- !query 105 +-- !query SELECT cast(1 as float) - cast(1 as decimal(3, 0)) FROM t --- !query 105 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 105 output +-- !query output 0.0 --- !query 106 +-- !query SELECT cast(1 as float) - cast(1 as decimal(5, 0)) FROM t --- !query 106 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 106 output +-- !query output 0.0 --- !query 107 +-- !query SELECT cast(1 as float) - cast(1 as decimal(10, 0)) FROM t --- !query 107 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 107 output +-- !query output 0.0 --- !query 108 +-- !query SELECT cast(1 as float) - cast(1 as decimal(20, 0)) FROM t --- !query 108 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 108 output +-- !query output 0.0 --- !query 109 +-- !query SELECT cast(1 as double) - cast(1 as decimal(3, 0)) FROM t --- !query 109 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 109 output +-- !query output 0.0 --- !query 110 +-- !query SELECT cast(1 as double) - cast(1 as decimal(5, 0)) FROM t --- !query 110 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 110 output +-- !query output 0.0 --- !query 111 +-- !query SELECT cast(1 as double) - cast(1 as decimal(10, 0)) FROM t --- !query 111 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 111 output +-- !query output 0.0 --- !query 112 +-- !query SELECT cast(1 as double) - cast(1 as decimal(20, 0)) FROM t --- !query 112 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 112 output +-- !query output 0.0 --- !query 113 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(3, 0)) FROM t --- !query 113 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 113 output +-- !query output 0 --- !query 114 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(5, 0)) FROM t --- !query 114 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 114 output +-- !query output 0 --- !query 115 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t --- !query 115 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> --- !query 115 output +-- !query output 0 --- !query 116 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(20, 0)) FROM t --- !query 116 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 116 output +-- !query output 0 --- !query 117 +-- !query SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 118 +-- !query SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t --- !query 118 schema +-- !query schema struct<> --- !query 118 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 119 +-- !query SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t --- !query 119 schema +-- !query schema struct<> --- !query 119 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 120 +-- !query SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 121 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FROM t --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 --- !query 122 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FROM t --- !query 122 schema +-- !query schema struct<> --- !query 122 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 --- !query 123 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FROM t --- !query 123 schema +-- !query schema struct<> --- !query 123 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 124 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FROM t --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires timestamp type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 --- !query 125 +-- !query SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t --- !query 125 schema +-- !query schema struct<> --- !query 125 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 --- !query 126 +-- !query SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t --- !query 126 schema +-- !query schema struct<> --- !query 126 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 --- !query 127 +-- !query SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 128 +-- !query SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t --- !query 128 schema +-- !query schema struct<> --- !query 128 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 --- !query 129 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as tinyint) FROM t --- !query 129 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) - CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(4,0)> --- !query 129 output +-- !query output 0 --- !query 130 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as tinyint) FROM t --- !query 130 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(6,0)) - CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(6,0))):decimal(6,0)> --- !query 130 output +-- !query output 0 --- !query 131 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as tinyint) FROM t --- !query 131 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 131 output +-- !query output 0 --- !query 132 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as tinyint) FROM t --- !query 132 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 132 output +-- !query output 0 --- !query 133 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as smallint) FROM t --- !query 133 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(6,0)) - CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(6,0))):decimal(6,0)> --- !query 133 output +-- !query output 0 --- !query 134 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as smallint) FROM t --- !query 134 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) - CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(6,0)> --- !query 134 output +-- !query output 0 --- !query 135 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as smallint) FROM t --- !query 135 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0)) - CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 135 output +-- !query output 0 --- !query 136 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as smallint) FROM t --- !query 136 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 136 output +-- !query output 0 --- !query 137 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as int) FROM t --- !query 137 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0)) - CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 137 output +-- !query output 0 --- !query 138 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as int) FROM t --- !query 138 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) - CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 138 output +-- !query output 0 --- !query 139 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as int) FROM t --- !query 139 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) - CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(11,0)> --- !query 139 output +-- !query output 0 --- !query 140 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as int) FROM t --- !query 140 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 140 output +-- !query output 0 --- !query 141 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as bigint) FROM t --- !query 141 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 141 output +-- !query output 0 --- !query 142 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as bigint) FROM t --- !query 142 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 142 output +-- !query output 0 --- !query 143 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as bigint) FROM t --- !query 143 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0)) - CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 143 output +-- !query output 0 --- !query 144 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as bigint) FROM t --- !query 144 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) - CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(21,0)> --- !query 144 output +-- !query output 0 --- !query 145 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as float) FROM t --- !query 145 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 145 output +-- !query output 0.0 --- !query 146 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as float) FROM t --- !query 146 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 146 output +-- !query output 0.0 --- !query 147 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as float) FROM t --- !query 147 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 147 output +-- !query output 0.0 --- !query 148 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as float) FROM t --- !query 148 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 148 output +-- !query output 0.0 --- !query 149 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as double) FROM t --- !query 149 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 149 output +-- !query output 0.0 --- !query 150 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as double) FROM t --- !query 150 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 150 output +-- !query output 0.0 --- !query 151 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as double) FROM t --- !query 151 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 151 output +-- !query output 0.0 --- !query 152 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as double) FROM t --- !query 152 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 152 output +-- !query output 0.0 --- !query 153 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as decimal(10, 0)) FROM t --- !query 153 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 153 output +-- !query output 0 --- !query 154 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as decimal(10, 0)) FROM t --- !query 154 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(11,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(11,0))):decimal(11,0)> --- !query 154 output +-- !query output 0 --- !query 155 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t --- !query 155 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> --- !query 155 output +-- !query output 0 --- !query 156 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as decimal(10, 0)) FROM t --- !query 156 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(21,0)) - CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(21,0))):decimal(21,0)> --- !query 156 output +-- !query output 0 --- !query 157 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as string) FROM t --- !query 157 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) - CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 157 output +-- !query output 0.0 --- !query 158 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as string) FROM t --- !query 158 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) - CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 158 output +-- !query output 0.0 --- !query 159 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as string) FROM t --- !query 159 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) - CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 159 output +-- !query output 0.0 --- !query 160 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as string) FROM t --- !query 160 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) - CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 160 output +-- !query output 0.0 --- !query 161 +-- !query SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t --- !query 161 schema +-- !query schema struct<> --- !query 161 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 162 +-- !query SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t --- !query 162 schema +-- !query schema struct<> --- !query 162 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 163 +-- !query SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t --- !query 163 schema +-- !query schema struct<> --- !query 163 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 164 +-- !query SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t --- !query 164 schema +-- !query schema struct<> --- !query 164 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 165 +-- !query SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t --- !query 165 schema +-- !query schema struct<> --- !query 165 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 166 +-- !query SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t --- !query 166 schema +-- !query schema struct<> --- !query 166 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 167 +-- !query SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t --- !query 167 schema +-- !query schema struct<> --- !query 167 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 168 +-- !query SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t --- !query 168 schema +-- !query schema struct<> --- !query 168 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 169 +-- !query SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 169 schema +-- !query schema struct<> --- !query 169 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 --- !query 170 +-- !query SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 170 schema +-- !query schema struct<> --- !query 170 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 --- !query 171 +-- !query SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 171 schema +-- !query schema struct<> --- !query 171 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 172 +-- !query SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 172 schema +-- !query schema struct<> --- !query 172 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 --- !query 173 +-- !query SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t --- !query 173 schema +-- !query schema struct<> --- !query 173 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtractdates(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(3,0))' is of decimal(3,0) type.; line 1 pos 7 --- !query 174 +-- !query SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t --- !query 174 schema +-- !query schema struct<> --- !query 174 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtractdates(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(5,0))' is of decimal(5,0) type.; line 1 pos 7 --- !query 175 +-- !query SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t --- !query 175 schema +-- !query schema struct<> --- !query 175 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtractdates(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(10,0))' is of decimal(10,0) type.; line 1 pos 7 --- !query 176 +-- !query SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t --- !query 176 schema +-- !query schema struct<> --- !query 176 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtractdates(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: argument 1 requires date type, however, 'CAST(1 AS DECIMAL(20,0))' is of decimal(20,0) type.; line 1 pos 7 --- !query 177 +-- !query SELECT cast(1 as tinyint) * cast(1 as decimal(3, 0)) FROM t --- !query 177 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) * CAST(1 AS DECIMAL(3,0))):decimal(7,0)> --- !query 177 output +-- !query output 1 --- !query 178 +-- !query SELECT cast(1 as tinyint) * cast(1 as decimal(5, 0)) FROM t --- !query 178 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) * CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,0)> --- !query 178 output +-- !query output 1 --- !query 179 +-- !query SELECT cast(1 as tinyint) * cast(1 as decimal(10, 0)) FROM t --- !query 179 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,0)> --- !query 179 output +-- !query output 1 --- !query 180 +-- !query SELECT cast(1 as tinyint) * cast(1 as decimal(20, 0)) FROM t --- !query 180 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,0)> --- !query 180 output +-- !query output 1 --- !query 181 +-- !query SELECT cast(1 as smallint) * cast(1 as decimal(3, 0)) FROM t --- !query 181 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) * CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(9,0)> --- !query 181 output +-- !query output 1 --- !query 182 +-- !query SELECT cast(1 as smallint) * cast(1 as decimal(5, 0)) FROM t --- !query 182 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) * CAST(1 AS DECIMAL(5,0))):decimal(11,0)> --- !query 182 output +-- !query output 1 --- !query 183 +-- !query SELECT cast(1 as smallint) * cast(1 as decimal(10, 0)) FROM t --- !query 183 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,0)> --- !query 183 output +-- !query output 1 --- !query 184 +-- !query SELECT cast(1 as smallint) * cast(1 as decimal(20, 0)) FROM t --- !query 184 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,0)> --- !query 184 output +-- !query output 1 --- !query 185 +-- !query SELECT cast(1 as int) * cast(1 as decimal(3, 0)) FROM t --- !query 185 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(14,0)> --- !query 185 output +-- !query output 1 --- !query 186 +-- !query SELECT cast(1 as int) * cast(1 as decimal(5, 0)) FROM t --- !query 186 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,0)> --- !query 186 output +-- !query output 1 --- !query 187 +-- !query SELECT cast(1 as int) * cast(1 as decimal(10, 0)) FROM t --- !query 187 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> --- !query 187 output +-- !query output 1 --- !query 188 +-- !query SELECT cast(1 as int) * cast(1 as decimal(20, 0)) FROM t --- !query 188 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,0)> --- !query 188 output +-- !query output 1 --- !query 189 +-- !query SELECT cast(1 as bigint) * cast(1 as decimal(3, 0)) FROM t --- !query 189 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(24,0)> --- !query 189 output +-- !query output 1 --- !query 190 +-- !query SELECT cast(1 as bigint) * cast(1 as decimal(5, 0)) FROM t --- !query 190 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,0)> --- !query 190 output +-- !query output 1 --- !query 191 +-- !query SELECT cast(1 as bigint) * cast(1 as decimal(10, 0)) FROM t --- !query 191 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,0)> --- !query 191 output +-- !query output 1 --- !query 192 +-- !query SELECT cast(1 as bigint) * cast(1 as decimal(20, 0)) FROM t --- !query 192 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) * CAST(1 AS DECIMAL(20,0))):decimal(38,0)> --- !query 192 output +-- !query output 1 --- !query 193 +-- !query SELECT cast(1 as float) * cast(1 as decimal(3, 0)) FROM t --- !query 193 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 193 output +-- !query output 1.0 --- !query 194 +-- !query SELECT cast(1 as float) * cast(1 as decimal(5, 0)) FROM t --- !query 194 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 194 output +-- !query output 1.0 --- !query 195 +-- !query SELECT cast(1 as float) * cast(1 as decimal(10, 0)) FROM t --- !query 195 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 195 output +-- !query output 1.0 --- !query 196 +-- !query SELECT cast(1 as float) * cast(1 as decimal(20, 0)) FROM t --- !query 196 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 196 output +-- !query output 1.0 --- !query 197 +-- !query SELECT cast(1 as double) * cast(1 as decimal(3, 0)) FROM t --- !query 197 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 197 output +-- !query output 1.0 --- !query 198 +-- !query SELECT cast(1 as double) * cast(1 as decimal(5, 0)) FROM t --- !query 198 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 198 output +-- !query output 1.0 --- !query 199 +-- !query SELECT cast(1 as double) * cast(1 as decimal(10, 0)) FROM t --- !query 199 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 199 output +-- !query output 1.0 --- !query 200 +-- !query SELECT cast(1 as double) * cast(1 as decimal(20, 0)) FROM t --- !query 200 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 200 output +-- !query output 1.0 --- !query 201 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(3, 0)) FROM t --- !query 201 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(14,0)> --- !query 201 output +-- !query output 1 --- !query 202 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(5, 0)) FROM t --- !query 202 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,0)> --- !query 202 output +-- !query output 1 --- !query 203 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t --- !query 203 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> --- !query 203 output +-- !query output 1 --- !query 204 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(20, 0)) FROM t --- !query 204 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,0)> --- !query 204 output +-- !query output 1 --- !query 205 +-- !query SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t --- !query 205 schema +-- !query schema struct<> --- !query 205 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 206 +-- !query SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t --- !query 206 schema +-- !query schema struct<> --- !query 206 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 207 +-- !query SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t --- !query 207 schema +-- !query schema struct<> --- !query 207 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 208 +-- !query SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t --- !query 208 schema +-- !query schema struct<> --- !query 208 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 209 +-- !query SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t --- !query 209 schema +-- !query schema struct<> --- !query 209 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 210 +-- !query SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t --- !query 210 schema +-- !query schema struct<> --- !query 210 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 211 +-- !query SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t --- !query 211 schema +-- !query schema struct<> --- !query 211 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 212 +-- !query SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t --- !query 212 schema +-- !query schema struct<> --- !query 212 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00.0' AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 213 +-- !query SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t --- !query 213 schema +-- !query schema struct<> --- !query 213 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 214 +-- !query SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t --- !query 214 schema +-- !query schema struct<> --- !query 214 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 215 +-- !query SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t --- !query 215 schema +-- !query schema struct<> --- !query 215 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 216 +-- !query SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t --- !query 216 schema +-- !query schema struct<> --- !query 216 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017*12*11 09:30:00' AS DATE) * CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 217 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as tinyint) FROM t --- !query 217 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) * CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(7,0)> --- !query 217 output +-- !query output 1 --- !query 218 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as tinyint) FROM t --- !query 218 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) * CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(9,0)> --- !query 218 output +-- !query output 1 --- !query 219 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as tinyint) FROM t --- !query 219 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(14,0)> --- !query 219 output +-- !query output 1 --- !query 220 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as tinyint) FROM t --- !query 220 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(24,0)> --- !query 220 output +-- !query output 1 --- !query 221 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as smallint) FROM t --- !query 221 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) * CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,0)> --- !query 221 output +-- !query output 1 --- !query 222 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as smallint) FROM t --- !query 222 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) * CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(11,0)> --- !query 222 output +-- !query output 1 --- !query 223 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as smallint) FROM t --- !query 223 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) * CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,0)> --- !query 223 output +-- !query output 1 --- !query 224 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as smallint) FROM t --- !query 224 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,0)> --- !query 224 output +-- !query output 1 --- !query 225 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as int) FROM t --- !query 225 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) * CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,0)> --- !query 225 output +-- !query output 1 --- !query 226 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as int) FROM t --- !query 226 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) * CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,0)> --- !query 226 output +-- !query output 1 --- !query 227 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as int) FROM t --- !query 227 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) * CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,0)> --- !query 227 output +-- !query output 1 --- !query 228 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as int) FROM t --- !query 228 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,0)> --- !query 228 output +-- !query output 1 --- !query 229 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as bigint) FROM t --- !query 229 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,0)> --- !query 229 output +-- !query output 1 --- !query 230 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as bigint) FROM t --- !query 230 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,0)> --- !query 230 output +-- !query output 1 --- !query 231 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as bigint) FROM t --- !query 231 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) * CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,0)> --- !query 231 output +-- !query output 1 --- !query 232 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as bigint) FROM t --- !query 232 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) * CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(38,0)> --- !query 232 output +-- !query output 1 --- !query 233 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as float) FROM t --- !query 233 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 233 output +-- !query output 1.0 --- !query 234 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as float) FROM t --- !query 234 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 234 output +-- !query output 1.0 --- !query 235 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as float) FROM t --- !query 235 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 235 output +-- !query output 1.0 --- !query 236 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as float) FROM t --- !query 236 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 236 output +-- !query output 1.0 --- !query 237 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as double) FROM t --- !query 237 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 237 output +-- !query output 1.0 --- !query 238 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as double) FROM t --- !query 238 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 238 output +-- !query output 1.0 --- !query 239 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as double) FROM t --- !query 239 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 239 output +-- !query output 1.0 --- !query 240 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as double) FROM t --- !query 240 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 240 output +-- !query output 1.0 --- !query 241 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as decimal(10, 0)) FROM t --- !query 241 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,0)> --- !query 241 output +-- !query output 1 --- !query 242 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as decimal(10, 0)) FROM t --- !query 242 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,0)> --- !query 242 output +-- !query output 1 --- !query 243 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t --- !query 243 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> --- !query 243 output +-- !query output 1 --- !query 244 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as decimal(10, 0)) FROM t --- !query 244 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) * CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,0)> --- !query 244 output +-- !query output 1 --- !query 245 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as string) FROM t --- !query 245 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) * CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 245 output +-- !query output 1.0 --- !query 246 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as string) FROM t --- !query 246 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) * CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 246 output +-- !query output 1.0 --- !query 247 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as string) FROM t --- !query 247 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) * CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 247 output +-- !query output 1.0 --- !query 248 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as string) FROM t --- !query 248 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) * CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 248 output +-- !query output 1.0 --- !query 249 +-- !query SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t --- !query 249 schema +-- !query schema struct<> --- !query 249 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) * CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 250 +-- !query SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t --- !query 250 schema +-- !query schema struct<> --- !query 250 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) * CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 251 +-- !query SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t --- !query 251 schema +-- !query schema struct<> --- !query 251 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) * CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 252 +-- !query SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t --- !query 252 schema +-- !query schema struct<> --- !query 252 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) * CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 253 +-- !query SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t --- !query 253 schema +-- !query schema struct<> --- !query 253 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 254 +-- !query SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t --- !query 254 schema +-- !query schema struct<> --- !query 254 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 255 +-- !query SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t --- !query 255 schema +-- !query schema struct<> --- !query 255 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 256 +-- !query SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t --- !query 256 schema +-- !query schema struct<> --- !query 256 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 257 +-- !query SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t --- !query 257 schema +-- !query schema struct<> --- !query 257 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 258 +-- !query SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t --- !query 258 schema +-- !query schema struct<> --- !query 258 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 259 +-- !query SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t --- !query 259 schema +-- !query schema struct<> --- !query 259 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 260 +-- !query SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t --- !query 260 schema +-- !query schema struct<> --- !query 260 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) * CAST('2017*12*11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 261 +-- !query SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t --- !query 261 schema +-- !query schema struct<> --- !query 261 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) * CAST('2017*12*11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) * CAST('2017*12*11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 262 +-- !query SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t --- !query 262 schema +-- !query schema struct<> --- !query 262 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) * CAST('2017*12*11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) * CAST('2017*12*11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 263 +-- !query SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t --- !query 263 schema +-- !query schema struct<> --- !query 263 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) * CAST('2017*12*11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) * CAST('2017*12*11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 264 +-- !query SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t --- !query 264 schema +-- !query schema struct<> --- !query 264 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) * CAST('2017*12*11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) * CAST('2017*12*11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 265 +-- !query SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t --- !query 265 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) / CAST(1 AS DECIMAL(3,0))):decimal(9,6)> --- !query 265 output +-- !query output 1.000000 --- !query 266 +-- !query SELECT cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t --- !query 266 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> --- !query 266 output +-- !query output 1.000000 --- !query 267 +-- !query SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t --- !query 267 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> --- !query 267 output +-- !query output 1.00000000000 --- !query 268 +-- !query SELECT cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t --- !query 268 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> --- !query 268 output +-- !query output 1.000000000000000000000 --- !query 269 +-- !query SELECT cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t --- !query 269 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> --- !query 269 output +-- !query output 1.000000 --- !query 270 +-- !query SELECT cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t --- !query 270 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) / CAST(1 AS DECIMAL(5,0))):decimal(11,6)> --- !query 270 output +-- !query output 1.000000 --- !query 271 +-- !query SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t --- !query 271 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> --- !query 271 output +-- !query output 1.00000000000 --- !query 272 +-- !query SELECT cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t --- !query 272 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> --- !query 272 output +-- !query output 1.000000000000000000000 --- !query 273 +-- !query SELECT cast(1 as int) / cast(1 as decimal(3, 0)) FROM t --- !query 273 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> --- !query 273 output +-- !query output 1.000000 --- !query 274 +-- !query SELECT cast(1 as int) / cast(1 as decimal(5, 0)) FROM t --- !query 274 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> --- !query 274 output +-- !query output 1.000000 --- !query 275 +-- !query SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t --- !query 275 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> --- !query 275 output +-- !query output 1.00000000000 --- !query 276 +-- !query SELECT cast(1 as int) / cast(1 as decimal(20, 0)) FROM t --- !query 276 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> --- !query 276 output +-- !query output 1.000000000000000000000 --- !query 277 +-- !query SELECT cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t --- !query 277 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> --- !query 277 output +-- !query output 1.000000 --- !query 278 +-- !query SELECT cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t --- !query 278 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> --- !query 278 output +-- !query output 1.000000 --- !query 279 +-- !query SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t --- !query 279 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> --- !query 279 output +-- !query output 1.00000000000 --- !query 280 +-- !query SELECT cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t --- !query 280 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) / CAST(1 AS DECIMAL(20,0))):decimal(38,18)> --- !query 280 output +-- !query output 1.000000000000000000 --- !query 281 +-- !query SELECT cast(1 as float) / cast(1 as decimal(3, 0)) FROM t --- !query 281 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) AS DOUBLE)):double> --- !query 281 output +-- !query output 1.0 --- !query 282 +-- !query SELECT cast(1 as float) / cast(1 as decimal(5, 0)) FROM t --- !query 282 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) AS DOUBLE)):double> --- !query 282 output +-- !query output 1.0 --- !query 283 +-- !query SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t --- !query 283 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) AS DOUBLE)):double> --- !query 283 output +-- !query output 1.0 --- !query 284 +-- !query SELECT cast(1 as float) / cast(1 as decimal(20, 0)) FROM t --- !query 284 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) AS DOUBLE)):double> --- !query 284 output +-- !query output 1.0 --- !query 285 +-- !query SELECT cast(1 as double) / cast(1 as decimal(3, 0)) FROM t --- !query 285 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 285 output +-- !query output 1.0 --- !query 286 +-- !query SELECT cast(1 as double) / cast(1 as decimal(5, 0)) FROM t --- !query 286 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 286 output +-- !query output 1.0 --- !query 287 +-- !query SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t --- !query 287 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 287 output +-- !query output 1.0 --- !query 288 +-- !query SELECT cast(1 as double) / cast(1 as decimal(20, 0)) FROM t --- !query 288 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 288 output +-- !query output 1.0 --- !query 289 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t --- !query 289 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> --- !query 289 output +-- !query output 1.000000 --- !query 290 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t --- !query 290 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> --- !query 290 output +-- !query output 1.000000 --- !query 291 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t --- !query 291 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> --- !query 291 output +-- !query output 1.00000000000 --- !query 292 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t --- !query 292 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> --- !query 292 output +-- !query output 1.000000000000000000000 --- !query 293 +-- !query SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t --- !query 293 schema +-- !query schema struct<> --- !query 293 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 294 +-- !query SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t --- !query 294 schema +-- !query schema struct<> --- !query 294 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 295 +-- !query SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t --- !query 295 schema +-- !query schema struct<> --- !query 295 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 296 +-- !query SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t --- !query 296 schema +-- !query schema struct<> --- !query 296 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 297 +-- !query SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t --- !query 297 schema +-- !query schema struct<> --- !query 297 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 298 +-- !query SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t --- !query 298 schema +-- !query schema struct<> --- !query 298 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 299 +-- !query SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t --- !query 299 schema +-- !query schema struct<> --- !query 299 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 300 +-- !query SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t --- !query 300 schema +-- !query schema struct<> --- !query 300 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 301 +-- !query SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t --- !query 301 schema +-- !query schema struct<> --- !query 301 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 302 +-- !query SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t --- !query 302 schema +-- !query schema struct<> --- !query 302 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 303 +-- !query SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t --- !query 303 schema +-- !query schema struct<> --- !query 303 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 304 +-- !query SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t --- !query 304 schema +-- !query schema struct<> --- !query 304 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017/12/11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 305 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t --- !query 305 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) / CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(9,6)> --- !query 305 output +-- !query output 1.000000 --- !query 306 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t --- !query 306 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(11,6)> --- !query 306 output +-- !query output 1.000000 --- !query 307 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t --- !query 307 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> --- !query 307 output +-- !query output 1.000000 --- !query 308 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t --- !query 308 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(26,6)> --- !query 308 output +-- !query output 1.000000 --- !query 309 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t --- !query 309 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(9,6)> --- !query 309 output +-- !query output 1.000000 --- !query 310 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t --- !query 310 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) / CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(11,6)> --- !query 310 output +-- !query output 1.000000 --- !query 311 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t --- !query 311 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> --- !query 311 output +-- !query output 1.000000 --- !query 312 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t --- !query 312 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(26,6)> --- !query 312 output +-- !query output 1.000000 --- !query 313 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as int) FROM t --- !query 313 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> --- !query 313 output +-- !query output 1.00000000000 --- !query 314 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as int) FROM t --- !query 314 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> --- !query 314 output +-- !query output 1.00000000000 --- !query 315 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t --- !query 315 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> --- !query 315 output +-- !query output 1.00000000000 --- !query 316 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as int) FROM t --- !query 316 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> --- !query 316 output +-- !query output 1.00000000000 --- !query 317 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t --- !query 317 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(24,21)> --- !query 317 output +-- !query output 1.000000000000000000000 --- !query 318 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t --- !query 318 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(26,21)> --- !query 318 output +-- !query output 1.000000000000000000000 --- !query 319 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t --- !query 319 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> --- !query 319 output +-- !query output 1.000000000000000000000 --- !query 320 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t --- !query 320 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) / CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(38,18)> --- !query 320 output +-- !query output 1.000000000000000000 --- !query 321 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as float) FROM t --- !query 321 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 321 output +-- !query output 1.0 --- !query 322 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as float) FROM t --- !query 322 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 322 output +-- !query output 1.0 --- !query 323 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t --- !query 323 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 323 output +-- !query output 1.0 --- !query 324 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as float) FROM t --- !query 324 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 324 output +-- !query output 1.0 --- !query 325 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as double) FROM t --- !query 325 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 325 output +-- !query output 1.0 --- !query 326 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as double) FROM t --- !query 326 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 326 output +-- !query output 1.0 --- !query 327 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t --- !query 327 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 327 output +-- !query output 1.0 --- !query 328 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as double) FROM t --- !query 328 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 328 output +-- !query output 1.0 --- !query 329 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t --- !query 329 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> --- !query 329 output +-- !query output 1.00000000000 --- !query 330 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t --- !query 330 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> --- !query 330 output +-- !query output 1.00000000000 --- !query 331 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t --- !query 331 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> --- !query 331 output +-- !query output 1.00000000000 --- !query 332 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t --- !query 332 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> --- !query 332 output +-- !query output 1.00000000000 --- !query 333 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as string) FROM t --- !query 333 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 333 output +-- !query output 1.0 --- !query 334 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as string) FROM t --- !query 334 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 334 output +-- !query output 1.0 --- !query 335 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t --- !query 335 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 335 output +-- !query output 1.0 --- !query 336 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as string) FROM t --- !query 336 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 336 output +-- !query output 1.0 --- !query 337 +-- !query SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t --- !query 337 schema +-- !query schema struct<> --- !query 337 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) / CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 338 +-- !query SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t --- !query 338 schema +-- !query schema struct<> --- !query 338 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) / CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 339 +-- !query SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t --- !query 339 schema +-- !query schema struct<> --- !query 339 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 340 +-- !query SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t --- !query 340 schema +-- !query schema struct<> --- !query 340 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) / CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 341 +-- !query SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t --- !query 341 schema +-- !query schema struct<> --- !query 341 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 342 +-- !query SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t --- !query 342 schema +-- !query schema struct<> --- !query 342 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 343 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t --- !query 343 schema +-- !query schema struct<> --- !query 343 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 344 +-- !query SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t --- !query 344 schema +-- !query schema struct<> --- !query 344 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 345 +-- !query SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t --- !query 345 schema +-- !query schema struct<> --- !query 345 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 346 +-- !query SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t --- !query 346 schema +-- !query schema struct<> --- !query 346 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 347 +-- !query SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t --- !query 347 schema +-- !query schema struct<> --- !query 347 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 348 +-- !query SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t --- !query 348 schema +-- !query schema struct<> --- !query 348 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) / CAST('2017/12/11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 349 +-- !query SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t --- !query 349 schema +-- !query schema struct<> --- !query 349 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) / CAST('2017/12/11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) / CAST('2017/12/11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 350 +-- !query SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t --- !query 350 schema +-- !query schema struct<> --- !query 350 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) / CAST('2017/12/11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) / CAST('2017/12/11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 351 +-- !query SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t --- !query 351 schema +-- !query schema struct<> --- !query 351 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('2017/12/11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('2017/12/11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 352 +-- !query SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t --- !query 352 schema +-- !query schema struct<> --- !query 352 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) / CAST('2017/12/11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) / CAST('2017/12/11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 353 +-- !query SELECT cast(1 as tinyint) % cast(1 as decimal(3, 0)) FROM t --- !query 353 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) % CAST(1 AS DECIMAL(3,0))):decimal(3,0)> --- !query 353 output +-- !query output 0 --- !query 354 +-- !query SELECT cast(1 as tinyint) % cast(1 as decimal(5, 0)) FROM t --- !query 354 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) % CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(3,0)> --- !query 354 output +-- !query output 0 --- !query 355 +-- !query SELECT cast(1 as tinyint) % cast(1 as decimal(10, 0)) FROM t --- !query 355 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(3,0)> --- !query 355 output +-- !query output 0 --- !query 356 +-- !query SELECT cast(1 as tinyint) % cast(1 as decimal(20, 0)) FROM t --- !query 356 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(3,0)> --- !query 356 output +-- !query output 0 --- !query 357 +-- !query SELECT cast(1 as smallint) % cast(1 as decimal(3, 0)) FROM t --- !query 357 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) % CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(3,0)> --- !query 357 output +-- !query output 0 --- !query 358 +-- !query SELECT cast(1 as smallint) % cast(1 as decimal(5, 0)) FROM t --- !query 358 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) % CAST(1 AS DECIMAL(5,0))):decimal(5,0)> --- !query 358 output +-- !query output 0 --- !query 359 +-- !query SELECT cast(1 as smallint) % cast(1 as decimal(10, 0)) FROM t --- !query 359 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(5,0)> --- !query 359 output +-- !query output 0 --- !query 360 +-- !query SELECT cast(1 as smallint) % cast(1 as decimal(20, 0)) FROM t --- !query 360 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(5,0)> --- !query 360 output +-- !query output 0 --- !query 361 +-- !query SELECT cast(1 as int) % cast(1 as decimal(3, 0)) FROM t --- !query 361 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(3,0)> --- !query 361 output +-- !query output 0 --- !query 362 +-- !query SELECT cast(1 as int) % cast(1 as decimal(5, 0)) FROM t --- !query 362 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(5,0)> --- !query 362 output +-- !query output 0 --- !query 363 +-- !query SELECT cast(1 as int) % cast(1 as decimal(10, 0)) FROM t --- !query 363 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> --- !query 363 output +-- !query output 0 --- !query 364 +-- !query SELECT cast(1 as int) % cast(1 as decimal(20, 0)) FROM t --- !query 364 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(10,0)> --- !query 364 output +-- !query output 0 --- !query 365 +-- !query SELECT cast(1 as bigint) % cast(1 as decimal(3, 0)) FROM t --- !query 365 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(3,0)> --- !query 365 output +-- !query output 0 --- !query 366 +-- !query SELECT cast(1 as bigint) % cast(1 as decimal(5, 0)) FROM t --- !query 366 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(5,0)> --- !query 366 output +-- !query output 0 --- !query 367 +-- !query SELECT cast(1 as bigint) % cast(1 as decimal(10, 0)) FROM t --- !query 367 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(10,0)> --- !query 367 output +-- !query output 0 --- !query 368 +-- !query SELECT cast(1 as bigint) % cast(1 as decimal(20, 0)) FROM t --- !query 368 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) % CAST(1 AS DECIMAL(20,0))):decimal(20,0)> --- !query 368 output +-- !query output 0 --- !query 369 +-- !query SELECT cast(1 as float) % cast(1 as decimal(3, 0)) FROM t --- !query 369 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 369 output +-- !query output 0.0 --- !query 370 +-- !query SELECT cast(1 as float) % cast(1 as decimal(5, 0)) FROM t --- !query 370 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 370 output +-- !query output 0.0 --- !query 371 +-- !query SELECT cast(1 as float) % cast(1 as decimal(10, 0)) FROM t --- !query 371 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 371 output +-- !query output 0.0 --- !query 372 +-- !query SELECT cast(1 as float) % cast(1 as decimal(20, 0)) FROM t --- !query 372 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 372 output +-- !query output 0.0 --- !query 373 +-- !query SELECT cast(1 as double) % cast(1 as decimal(3, 0)) FROM t --- !query 373 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):double> --- !query 373 output +-- !query output 0.0 --- !query 374 +-- !query SELECT cast(1 as double) % cast(1 as decimal(5, 0)) FROM t --- !query 374 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):double> --- !query 374 output +-- !query output 0.0 --- !query 375 +-- !query SELECT cast(1 as double) % cast(1 as decimal(10, 0)) FROM t --- !query 375 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 375 output +-- !query output 0.0 --- !query 376 +-- !query SELECT cast(1 as double) % cast(1 as decimal(20, 0)) FROM t --- !query 376 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):double> --- !query 376 output +-- !query output 0.0 --- !query 377 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(3, 0)) FROM t --- !query 377 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(3,0)> --- !query 377 output +-- !query output 0 --- !query 378 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(5, 0)) FROM t --- !query 378 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(5,0)> --- !query 378 output +-- !query output 0 --- !query 379 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t --- !query 379 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> --- !query 379 output +-- !query output 0 --- !query 380 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(20, 0)) FROM t --- !query 380 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(10,0)> --- !query 380 output +-- !query output 0 --- !query 381 +-- !query SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t --- !query 381 schema +-- !query schema struct<> --- !query 381 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 382 +-- !query SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t --- !query 382 schema +-- !query schema struct<> --- !query 382 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 383 +-- !query SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t --- !query 383 schema +-- !query schema struct<> --- !query 383 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 384 +-- !query SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t --- !query 384 schema +-- !query schema struct<> --- !query 384 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 385 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t --- !query 385 schema +-- !query schema struct<> --- !query 385 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 386 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t --- !query 386 schema +-- !query schema struct<> --- !query 386 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 387 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t --- !query 387 schema +-- !query schema struct<> --- !query 387 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 388 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t --- !query 388 schema +-- !query schema struct<> --- !query 388 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 389 +-- !query SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t --- !query 389 schema +-- !query schema struct<> --- !query 389 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 390 +-- !query SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t --- !query 390 schema +-- !query schema struct<> --- !query 390 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 391 +-- !query SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t --- !query 391 schema +-- !query schema struct<> --- !query 391 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 392 +-- !query SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t --- !query 392 schema +-- !query schema struct<> --- !query 392 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 393 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as tinyint) FROM t --- !query 393 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) % CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):decimal(3,0)> --- !query 393 output +-- !query output 0 --- !query 394 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as tinyint) FROM t --- !query 394 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) % CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):decimal(3,0)> --- !query 394 output +-- !query output 0 --- !query 395 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as tinyint) FROM t --- !query 395 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(3,0)> --- !query 395 output +-- !query output 0 --- !query 396 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as tinyint) FROM t --- !query 396 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(3,0)> --- !query 396 output +-- !query output 0 --- !query 397 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as smallint) FROM t --- !query 397 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) % CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):decimal(3,0)> --- !query 397 output +-- !query output 0 --- !query 398 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as smallint) FROM t --- !query 398 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) % CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):decimal(5,0)> --- !query 398 output +-- !query output 0 --- !query 399 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as smallint) FROM t --- !query 399 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) % CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(5,0)> --- !query 399 output +-- !query output 0 --- !query 400 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as smallint) FROM t --- !query 400 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):decimal(5,0)> --- !query 400 output +-- !query output 0 --- !query 401 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as int) FROM t --- !query 401 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) % CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(3,0)> --- !query 401 output +-- !query output 0 --- !query 402 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as int) FROM t --- !query 402 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) % CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(5,0)> --- !query 402 output +-- !query output 0 --- !query 403 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as int) FROM t --- !query 403 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) % CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(10,0)> --- !query 403 output +-- !query output 0 --- !query 404 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as int) FROM t --- !query 404 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(10,0)> --- !query 404 output +-- !query output 0 --- !query 405 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as bigint) FROM t --- !query 405 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(3,0)> --- !query 405 output +-- !query output 0 --- !query 406 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as bigint) FROM t --- !query 406 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(5,0)> --- !query 406 output +-- !query output 0 --- !query 407 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as bigint) FROM t --- !query 407 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) % CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(10,0)> --- !query 407 output +-- !query output 0 --- !query 408 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as bigint) FROM t --- !query 408 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) % CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):decimal(20,0)> --- !query 408 output +-- !query output 0 --- !query 409 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as float) FROM t --- !query 409 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 409 output +-- !query output 0.0 --- !query 410 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as float) FROM t --- !query 410 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 410 output +-- !query output 0.0 --- !query 411 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as float) FROM t --- !query 411 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 411 output +-- !query output 0.0 --- !query 412 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as float) FROM t --- !query 412 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 412 output +-- !query output 0.0 --- !query 413 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as double) FROM t --- !query 413 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 413 output +-- !query output 0.0 --- !query 414 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as double) FROM t --- !query 414 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 414 output +-- !query output 0.0 --- !query 415 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as double) FROM t --- !query 415 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 415 output +-- !query output 0.0 --- !query 416 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as double) FROM t --- !query 416 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 416 output +-- !query output 0.0 --- !query 417 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as decimal(10, 0)) FROM t --- !query 417 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(3,0)> --- !query 417 output +-- !query output 0 --- !query 418 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as decimal(10, 0)) FROM t --- !query 418 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(5,0)> --- !query 418 output +-- !query output 0 --- !query 419 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t --- !query 419 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> --- !query 419 output +-- !query output 0 --- !query 420 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as decimal(10, 0)) FROM t --- !query 420 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) % CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(10,0)> --- !query 420 output +-- !query output 0 --- !query 421 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as string) FROM t --- !query 421 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) % CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 421 output +-- !query output 0.0 --- !query 422 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as string) FROM t --- !query 422 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) % CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 422 output +-- !query output 0.0 --- !query 423 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as string) FROM t --- !query 423 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) % CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 423 output +-- !query output 0.0 --- !query 424 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as string) FROM t --- !query 424 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) % CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 424 output +-- !query output 0.0 --- !query 425 +-- !query SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t --- !query 425 schema +-- !query schema struct<> --- !query 425 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) % CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 426 +-- !query SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t --- !query 426 schema +-- !query schema struct<> --- !query 426 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) % CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 427 +-- !query SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t --- !query 427 schema +-- !query schema struct<> --- !query 427 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) % CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 428 +-- !query SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t --- !query 428 schema +-- !query schema struct<> --- !query 428 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) % CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 429 +-- !query SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t --- !query 429 schema +-- !query schema struct<> --- !query 429 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 430 +-- !query SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t --- !query 430 schema +-- !query schema struct<> --- !query 430 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 431 +-- !query SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t --- !query 431 schema +-- !query schema struct<> --- !query 431 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 432 +-- !query SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t --- !query 432 schema +-- !query schema struct<> --- !query 432 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 433 +-- !query SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 433 schema +-- !query schema struct<> --- !query 433 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 434 +-- !query SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 434 schema +-- !query schema struct<> --- !query 434 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 435 +-- !query SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 435 schema +-- !query schema struct<> --- !query 435 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 436 +-- !query SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 436 schema +-- !query schema struct<> --- !query 436 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 437 +-- !query SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t --- !query 437 schema +-- !query schema struct<> --- !query 437 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) % CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 438 +-- !query SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t --- !query 438 schema +-- !query schema struct<> --- !query 438 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) % CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 439 +-- !query SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t --- !query 439 schema +-- !query schema struct<> --- !query 439 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) % CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 440 +-- !query SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t --- !query 440 schema +-- !query schema struct<> --- !query 440 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) % CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 441 +-- !query SELECT pmod(cast(1 as tinyint), cast(1 as decimal(3, 0))) FROM t --- !query 441 schema +-- !query schema struct --- !query 441 output +-- !query output 0 --- !query 442 +-- !query SELECT pmod(cast(1 as tinyint), cast(1 as decimal(5, 0))) FROM t --- !query 442 schema +-- !query schema struct --- !query 442 output +-- !query output 0 --- !query 443 +-- !query SELECT pmod(cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t --- !query 443 schema +-- !query schema struct --- !query 443 output +-- !query output 0 --- !query 444 +-- !query SELECT pmod(cast(1 as tinyint), cast(1 as decimal(20, 0))) FROM t --- !query 444 schema +-- !query schema struct --- !query 444 output +-- !query output 0 --- !query 445 +-- !query SELECT pmod(cast(1 as smallint), cast(1 as decimal(3, 0))) FROM t --- !query 445 schema +-- !query schema struct --- !query 445 output +-- !query output 0 --- !query 446 +-- !query SELECT pmod(cast(1 as smallint), cast(1 as decimal(5, 0))) FROM t --- !query 446 schema +-- !query schema struct --- !query 446 output +-- !query output 0 --- !query 447 +-- !query SELECT pmod(cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t --- !query 447 schema +-- !query schema struct --- !query 447 output +-- !query output 0 --- !query 448 +-- !query SELECT pmod(cast(1 as smallint), cast(1 as decimal(20, 0))) FROM t --- !query 448 schema +-- !query schema struct --- !query 448 output +-- !query output 0 --- !query 449 +-- !query SELECT pmod(cast(1 as int), cast(1 as decimal(3, 0))) FROM t --- !query 449 schema +-- !query schema struct --- !query 449 output +-- !query output 0 --- !query 450 +-- !query SELECT pmod(cast(1 as int), cast(1 as decimal(5, 0))) FROM t --- !query 450 schema +-- !query schema struct --- !query 450 output +-- !query output 0 --- !query 451 +-- !query SELECT pmod(cast(1 as int), cast(1 as decimal(10, 0))) FROM t --- !query 451 schema +-- !query schema struct --- !query 451 output +-- !query output 0 --- !query 452 +-- !query SELECT pmod(cast(1 as int), cast(1 as decimal(20, 0))) FROM t --- !query 452 schema +-- !query schema struct --- !query 452 output +-- !query output 0 --- !query 453 +-- !query SELECT pmod(cast(1 as bigint), cast(1 as decimal(3, 0))) FROM t --- !query 453 schema +-- !query schema struct --- !query 453 output +-- !query output 0 --- !query 454 +-- !query SELECT pmod(cast(1 as bigint), cast(1 as decimal(5, 0))) FROM t --- !query 454 schema +-- !query schema struct --- !query 454 output +-- !query output 0 --- !query 455 +-- !query SELECT pmod(cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t --- !query 455 schema +-- !query schema struct --- !query 455 output +-- !query output 0 --- !query 456 +-- !query SELECT pmod(cast(1 as bigint), cast(1 as decimal(20, 0))) FROM t --- !query 456 schema +-- !query schema struct --- !query 456 output +-- !query output 0 --- !query 457 +-- !query SELECT pmod(cast(1 as float), cast(1 as decimal(3, 0))) FROM t --- !query 457 schema +-- !query schema struct --- !query 457 output +-- !query output 0.0 --- !query 458 +-- !query SELECT pmod(cast(1 as float), cast(1 as decimal(5, 0))) FROM t --- !query 458 schema +-- !query schema struct --- !query 458 output +-- !query output 0.0 --- !query 459 +-- !query SELECT pmod(cast(1 as float), cast(1 as decimal(10, 0))) FROM t --- !query 459 schema +-- !query schema struct --- !query 459 output +-- !query output 0.0 --- !query 460 +-- !query SELECT pmod(cast(1 as float), cast(1 as decimal(20, 0))) FROM t --- !query 460 schema +-- !query schema struct --- !query 460 output +-- !query output 0.0 --- !query 461 +-- !query SELECT pmod(cast(1 as double), cast(1 as decimal(3, 0))) FROM t --- !query 461 schema +-- !query schema struct --- !query 461 output +-- !query output 0.0 --- !query 462 +-- !query SELECT pmod(cast(1 as double), cast(1 as decimal(5, 0))) FROM t --- !query 462 schema +-- !query schema struct --- !query 462 output +-- !query output 0.0 --- !query 463 +-- !query SELECT pmod(cast(1 as double), cast(1 as decimal(10, 0))) FROM t --- !query 463 schema +-- !query schema struct --- !query 463 output +-- !query output 0.0 --- !query 464 +-- !query SELECT pmod(cast(1 as double), cast(1 as decimal(20, 0))) FROM t --- !query 464 schema +-- !query schema struct --- !query 464 output +-- !query output 0.0 --- !query 465 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(3, 0))) FROM t --- !query 465 schema +-- !query schema struct --- !query 465 output +-- !query output 0 --- !query 466 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(5, 0))) FROM t --- !query 466 schema +-- !query schema struct --- !query 466 output +-- !query output 0 --- !query 467 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t --- !query 467 schema +-- !query schema struct --- !query 467 output +-- !query output 0 --- !query 468 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(20, 0))) FROM t --- !query 468 schema +-- !query schema struct --- !query 468 output +-- !query output 0 --- !query 469 +-- !query SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t --- !query 469 schema +-- !query schema struct<> --- !query 469 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 470 +-- !query SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t --- !query 470 schema +-- !query schema struct<> --- !query 470 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 471 +-- !query SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t --- !query 471 schema +-- !query schema struct<> --- !query 471 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 472 +-- !query SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t --- !query 472 schema +-- !query schema struct<> --- !query 472 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 473 +-- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t --- !query 473 schema +-- !query schema struct<> --- !query 473 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 474 +-- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t --- !query 474 schema +-- !query schema struct<> --- !query 474 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 475 +-- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t --- !query 475 schema +-- !query schema struct<> --- !query 475 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 476 +-- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t --- !query 476 schema +-- !query schema struct<> --- !query 476 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 477 +-- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t --- !query 477 schema +-- !query schema struct<> --- !query 477 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 478 +-- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t --- !query 478 schema +-- !query schema struct<> --- !query 478 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 479 +-- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t --- !query 479 schema +-- !query schema struct<> --- !query 479 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 480 +-- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t --- !query 480 schema +-- !query schema struct<> --- !query 480 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 481 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as tinyint)) FROM t --- !query 481 schema +-- !query schema struct --- !query 481 output +-- !query output 0 --- !query 482 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as tinyint)) FROM t --- !query 482 schema +-- !query schema struct --- !query 482 output +-- !query output 0 --- !query 483 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t --- !query 483 schema +-- !query schema struct --- !query 483 output +-- !query output 0 --- !query 484 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as tinyint)) FROM t --- !query 484 schema +-- !query schema struct --- !query 484 output +-- !query output 0 --- !query 485 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as smallint)) FROM t --- !query 485 schema +-- !query schema struct --- !query 485 output +-- !query output 0 --- !query 486 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as smallint)) FROM t --- !query 486 schema +-- !query schema struct --- !query 486 output +-- !query output 0 --- !query 487 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t --- !query 487 schema +-- !query schema struct --- !query 487 output +-- !query output 0 --- !query 488 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as smallint)) FROM t --- !query 488 schema +-- !query schema struct --- !query 488 output +-- !query output 0 --- !query 489 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as int)) FROM t --- !query 489 schema +-- !query schema struct --- !query 489 output +-- !query output 0 --- !query 490 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as int)) FROM t --- !query 490 schema +-- !query schema struct --- !query 490 output +-- !query output 0 --- !query 491 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as int)) FROM t --- !query 491 schema +-- !query schema struct --- !query 491 output +-- !query output 0 --- !query 492 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as int)) FROM t --- !query 492 schema +-- !query schema struct --- !query 492 output +-- !query output 0 --- !query 493 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as bigint)) FROM t --- !query 493 schema +-- !query schema struct --- !query 493 output +-- !query output 0 --- !query 494 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as bigint)) FROM t --- !query 494 schema +-- !query schema struct --- !query 494 output +-- !query output 0 --- !query 495 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t --- !query 495 schema +-- !query schema struct --- !query 495 output +-- !query output 0 --- !query 496 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as bigint)) FROM t --- !query 496 schema +-- !query schema struct --- !query 496 output +-- !query output 0 --- !query 497 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as float)) FROM t --- !query 497 schema +-- !query schema struct --- !query 497 output +-- !query output 0.0 --- !query 498 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as float)) FROM t --- !query 498 schema +-- !query schema struct --- !query 498 output +-- !query output 0.0 --- !query 499 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as float)) FROM t --- !query 499 schema +-- !query schema struct --- !query 499 output +-- !query output 0.0 --- !query 500 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as float)) FROM t --- !query 500 schema +-- !query schema struct --- !query 500 output +-- !query output 0.0 --- !query 501 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as double)) FROM t --- !query 501 schema +-- !query schema struct --- !query 501 output +-- !query output 0.0 --- !query 502 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as double)) FROM t --- !query 502 schema +-- !query schema struct --- !query 502 output +-- !query output 0.0 --- !query 503 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as double)) FROM t --- !query 503 schema +-- !query schema struct --- !query 503 output +-- !query output 0.0 --- !query 504 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as double)) FROM t --- !query 504 schema +-- !query schema struct --- !query 504 output +-- !query output 0.0 --- !query 505 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as decimal(10, 0))) FROM t --- !query 505 schema +-- !query schema struct --- !query 505 output +-- !query output 0 --- !query 506 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as decimal(10, 0))) FROM t --- !query 506 schema +-- !query schema struct --- !query 506 output +-- !query output 0 --- !query 507 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t --- !query 507 schema +-- !query schema struct --- !query 507 output +-- !query output 0 --- !query 508 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as decimal(10, 0))) FROM t --- !query 508 schema +-- !query schema struct --- !query 508 output +-- !query output 0 --- !query 509 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as string)) FROM t --- !query 509 schema +-- !query schema struct --- !query 509 output +-- !query output 0.0 --- !query 510 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as string)) FROM t --- !query 510 schema +-- !query schema struct --- !query 510 output +-- !query output 0.0 --- !query 511 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as string)) FROM t --- !query 511 schema +-- !query schema struct --- !query 511 output +-- !query output 0.0 --- !query 512 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as string)) FROM t --- !query 512 schema +-- !query schema struct --- !query 512 output +-- !query output 0.0 --- !query 513 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t --- !query 513 schema +-- !query schema struct<> --- !query 513 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 514 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t --- !query 514 schema +-- !query schema struct<> --- !query 514 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 515 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t --- !query 515 schema +-- !query schema struct<> --- !query 515 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 516 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t --- !query 516 schema +-- !query schema struct<> --- !query 516 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 517 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t --- !query 517 schema +-- !query schema struct<> --- !query 517 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 518 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t --- !query 518 schema +-- !query schema struct<> --- !query 518 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 519 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t --- !query 519 schema +-- !query schema struct<> --- !query 519 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 520 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t --- !query 520 schema +-- !query schema struct<> --- !query 520 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 521 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 521 schema +-- !query schema struct<> --- !query 521 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 522 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 522 schema +-- !query schema struct<> --- !query 522 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 523 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 523 schema +-- !query schema struct<> --- !query 523 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 524 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 524 schema +-- !query schema struct<> --- !query 524 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 525 +-- !query SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t --- !query 525 schema +-- !query schema struct<> --- !query 525 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(3,0)), CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 526 +-- !query SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t --- !query 526 schema +-- !query schema struct<> --- !query 526 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(5,0)), CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 527 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 527 schema +-- !query schema struct<> --- !query 527 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 528 +-- !query SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 528 schema +-- !query schema struct<> --- !query 528 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS DECIMAL(20,0)), CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 529 +-- !query SELECT cast(1 as tinyint) = cast(1 as decimal(3, 0)) FROM t --- !query 529 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) = CAST(1 AS DECIMAL(3,0))):boolean> --- !query 529 output +-- !query output true --- !query 530 +-- !query SELECT cast(1 as tinyint) = cast(1 as decimal(5, 0)) FROM t --- !query 530 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 530 output +-- !query output true --- !query 531 +-- !query SELECT cast(1 as tinyint) = cast(1 as decimal(10, 0)) FROM t --- !query 531 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 531 output +-- !query output true --- !query 532 +-- !query SELECT cast(1 as tinyint) = cast(1 as decimal(20, 0)) FROM t --- !query 532 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 532 output +-- !query output true --- !query 533 +-- !query SELECT cast(1 as smallint) = cast(1 as decimal(3, 0)) FROM t --- !query 533 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 533 output +-- !query output true --- !query 534 +-- !query SELECT cast(1 as smallint) = cast(1 as decimal(5, 0)) FROM t --- !query 534 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) = CAST(1 AS DECIMAL(5,0))):boolean> --- !query 534 output +-- !query output true --- !query 535 +-- !query SELECT cast(1 as smallint) = cast(1 as decimal(10, 0)) FROM t --- !query 535 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 535 output +-- !query output true --- !query 536 +-- !query SELECT cast(1 as smallint) = cast(1 as decimal(20, 0)) FROM t --- !query 536 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 536 output +-- !query output true --- !query 537 +-- !query SELECT cast(1 as int) = cast(1 as decimal(3, 0)) FROM t --- !query 537 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 537 output +-- !query output true --- !query 538 +-- !query SELECT cast(1 as int) = cast(1 as decimal(5, 0)) FROM t --- !query 538 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 538 output +-- !query output true --- !query 539 +-- !query SELECT cast(1 as int) = cast(1 as decimal(10, 0)) FROM t --- !query 539 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> --- !query 539 output +-- !query output true --- !query 540 +-- !query SELECT cast(1 as int) = cast(1 as decimal(20, 0)) FROM t --- !query 540 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 540 output +-- !query output true --- !query 541 +-- !query SELECT cast(1 as bigint) = cast(1 as decimal(3, 0)) FROM t --- !query 541 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 541 output +-- !query output true --- !query 542 +-- !query SELECT cast(1 as bigint) = cast(1 as decimal(5, 0)) FROM t --- !query 542 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 542 output +-- !query output true --- !query 543 +-- !query SELECT cast(1 as bigint) = cast(1 as decimal(10, 0)) FROM t --- !query 543 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 543 output +-- !query output true --- !query 544 +-- !query SELECT cast(1 as bigint) = cast(1 as decimal(20, 0)) FROM t --- !query 544 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) = CAST(1 AS DECIMAL(20,0))):boolean> --- !query 544 output +-- !query output true --- !query 545 +-- !query SELECT cast(1 as float) = cast(1 as decimal(3, 0)) FROM t --- !query 545 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 545 output +-- !query output true --- !query 546 +-- !query SELECT cast(1 as float) = cast(1 as decimal(5, 0)) FROM t --- !query 546 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 546 output +-- !query output true --- !query 547 +-- !query SELECT cast(1 as float) = cast(1 as decimal(10, 0)) FROM t --- !query 547 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 547 output +-- !query output true --- !query 548 +-- !query SELECT cast(1 as float) = cast(1 as decimal(20, 0)) FROM t --- !query 548 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 548 output +-- !query output true --- !query 549 +-- !query SELECT cast(1 as double) = cast(1 as decimal(3, 0)) FROM t --- !query 549 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 549 output +-- !query output true --- !query 550 +-- !query SELECT cast(1 as double) = cast(1 as decimal(5, 0)) FROM t --- !query 550 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 550 output +-- !query output true --- !query 551 +-- !query SELECT cast(1 as double) = cast(1 as decimal(10, 0)) FROM t --- !query 551 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 551 output +-- !query output true --- !query 552 +-- !query SELECT cast(1 as double) = cast(1 as decimal(20, 0)) FROM t --- !query 552 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 552 output +-- !query output true --- !query 553 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(3, 0)) FROM t --- !query 553 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 553 output +-- !query output true --- !query 554 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(5, 0)) FROM t --- !query 554 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 554 output +-- !query output true --- !query 555 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t --- !query 555 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> --- !query 555 output +-- !query output true --- !query 556 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(20, 0)) FROM t --- !query 556 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 556 output +-- !query output true --- !query 557 +-- !query SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t --- !query 557 schema +-- !query schema struct<> --- !query 557 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 558 +-- !query SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t --- !query 558 schema +-- !query schema struct<> --- !query 558 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 559 +-- !query SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t --- !query 559 schema +-- !query schema struct<> --- !query 559 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 560 +-- !query SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t --- !query 560 schema +-- !query schema struct<> --- !query 560 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 561 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t --- !query 561 schema +-- !query schema struct<> --- !query 561 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 562 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t --- !query 562 schema +-- !query schema struct<> --- !query 562 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 563 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t --- !query 563 schema +-- !query schema struct<> --- !query 563 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 564 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t --- !query 564 schema +-- !query schema struct<> --- !query 564 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 565 +-- !query SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t --- !query 565 schema +-- !query schema struct<> --- !query 565 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 566 +-- !query SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t --- !query 566 schema +-- !query schema struct<> --- !query 566 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 567 +-- !query SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t --- !query 567 schema +-- !query schema struct<> --- !query 567 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 568 +-- !query SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t --- !query 568 schema +-- !query schema struct<> --- !query 568 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 569 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as tinyint) FROM t --- !query 569 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) = CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> --- !query 569 output +-- !query output true --- !query 570 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as tinyint) FROM t --- !query 570 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 570 output +-- !query output true --- !query 571 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as tinyint) FROM t --- !query 571 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 571 output +-- !query output true --- !query 572 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as tinyint) FROM t --- !query 572 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 572 output +-- !query output true --- !query 573 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as smallint) FROM t --- !query 573 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 573 output +-- !query output true --- !query 574 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as smallint) FROM t --- !query 574 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) = CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> --- !query 574 output +-- !query output true --- !query 575 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as smallint) FROM t --- !query 575 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 575 output +-- !query output true --- !query 576 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as smallint) FROM t --- !query 576 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 576 output +-- !query output true --- !query 577 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as int) FROM t --- !query 577 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 577 output +-- !query output true --- !query 578 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as int) FROM t --- !query 578 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 578 output +-- !query output true --- !query 579 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as int) FROM t --- !query 579 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) = CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> --- !query 579 output +-- !query output true --- !query 580 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as int) FROM t --- !query 580 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 580 output +-- !query output true --- !query 581 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as bigint) FROM t --- !query 581 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 581 output +-- !query output true --- !query 582 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as bigint) FROM t --- !query 582 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 582 output +-- !query output true --- !query 583 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as bigint) FROM t --- !query 583 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 583 output +-- !query output true --- !query 584 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as bigint) FROM t --- !query 584 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) = CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> --- !query 584 output +-- !query output true --- !query 585 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as float) FROM t --- !query 585 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 585 output +-- !query output true --- !query 586 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as float) FROM t --- !query 586 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 586 output +-- !query output true --- !query 587 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as float) FROM t --- !query 587 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 587 output +-- !query output true --- !query 588 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as float) FROM t --- !query 588 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 588 output +-- !query output true --- !query 589 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as double) FROM t --- !query 589 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 589 output +-- !query output true --- !query 590 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as double) FROM t --- !query 590 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 590 output +-- !query output true --- !query 591 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as double) FROM t --- !query 591 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 591 output +-- !query output true --- !query 592 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as double) FROM t --- !query 592 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 592 output +-- !query output true --- !query 593 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as decimal(10, 0)) FROM t --- !query 593 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 593 output +-- !query output true --- !query 594 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as decimal(10, 0)) FROM t --- !query 594 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 594 output +-- !query output true --- !query 595 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t --- !query 595 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> --- !query 595 output +-- !query output true --- !query 596 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as decimal(10, 0)) FROM t --- !query 596 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 596 output +-- !query output true --- !query 597 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as string) FROM t --- !query 597 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 597 output +-- !query output true --- !query 598 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as string) FROM t --- !query 598 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 598 output +-- !query output true --- !query 599 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as string) FROM t --- !query 599 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 599 output +-- !query output true --- !query 600 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as string) FROM t --- !query 600 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 600 output +-- !query output true --- !query 601 +-- !query SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t --- !query 601 schema +-- !query schema struct<> --- !query 601 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 602 +-- !query SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t --- !query 602 schema +-- !query schema struct<> --- !query 602 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 603 +-- !query SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t --- !query 603 schema +-- !query schema struct<> --- !query 603 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 604 +-- !query SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t --- !query 604 schema +-- !query schema struct<> --- !query 604 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 605 +-- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as boolean) FROM t --- !query 605 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(3,0))):boolean> --- !query 605 output +-- !query output true --- !query 606 +-- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as boolean) FROM t --- !query 606 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(5,0))):boolean> --- !query 606 output +-- !query output true --- !query 607 +-- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as boolean) FROM t --- !query 607 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(10,0))):boolean> --- !query 607 output +-- !query output true --- !query 608 +-- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as boolean) FROM t --- !query 608 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(20,0))):boolean> --- !query 608 output +-- !query output true --- !query 609 +-- !query SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 609 schema +-- !query schema struct<> --- !query 609 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 610 +-- !query SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 610 schema +-- !query schema struct<> --- !query 610 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 611 +-- !query SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 611 schema +-- !query schema struct<> --- !query 611 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 612 +-- !query SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 612 schema +-- !query schema struct<> --- !query 612 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 613 +-- !query SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t --- !query 613 schema +-- !query schema struct<> --- !query 613 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 614 +-- !query SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t --- !query 614 schema +-- !query schema struct<> --- !query 614 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 615 +-- !query SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t --- !query 615 schema +-- !query schema struct<> --- !query 615 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 616 +-- !query SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t --- !query 616 schema +-- !query schema struct<> --- !query 616 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 617 +-- !query SELECT cast(1 as tinyint) <=> cast(1 as decimal(3, 0)) FROM t --- !query 617 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) <=> CAST(1 AS DECIMAL(3,0))):boolean> --- !query 617 output +-- !query output true --- !query 618 +-- !query SELECT cast(1 as tinyint) <=> cast(1 as decimal(5, 0)) FROM t --- !query 618 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 618 output +-- !query output true --- !query 619 +-- !query SELECT cast(1 as tinyint) <=> cast(1 as decimal(10, 0)) FROM t --- !query 619 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 619 output +-- !query output true --- !query 620 +-- !query SELECT cast(1 as tinyint) <=> cast(1 as decimal(20, 0)) FROM t --- !query 620 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 620 output +-- !query output true --- !query 621 +-- !query SELECT cast(1 as smallint) <=> cast(1 as decimal(3, 0)) FROM t --- !query 621 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 621 output +-- !query output true --- !query 622 +-- !query SELECT cast(1 as smallint) <=> cast(1 as decimal(5, 0)) FROM t --- !query 622 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) <=> CAST(1 AS DECIMAL(5,0))):boolean> --- !query 622 output +-- !query output true --- !query 623 +-- !query SELECT cast(1 as smallint) <=> cast(1 as decimal(10, 0)) FROM t --- !query 623 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 623 output +-- !query output true --- !query 624 +-- !query SELECT cast(1 as smallint) <=> cast(1 as decimal(20, 0)) FROM t --- !query 624 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 624 output +-- !query output true --- !query 625 +-- !query SELECT cast(1 as int) <=> cast(1 as decimal(3, 0)) FROM t --- !query 625 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 625 output +-- !query output true --- !query 626 +-- !query SELECT cast(1 as int) <=> cast(1 as decimal(5, 0)) FROM t --- !query 626 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 626 output +-- !query output true --- !query 627 +-- !query SELECT cast(1 as int) <=> cast(1 as decimal(10, 0)) FROM t --- !query 627 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> --- !query 627 output +-- !query output true --- !query 628 +-- !query SELECT cast(1 as int) <=> cast(1 as decimal(20, 0)) FROM t --- !query 628 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 628 output +-- !query output true --- !query 629 +-- !query SELECT cast(1 as bigint) <=> cast(1 as decimal(3, 0)) FROM t --- !query 629 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 629 output +-- !query output true --- !query 630 +-- !query SELECT cast(1 as bigint) <=> cast(1 as decimal(5, 0)) FROM t --- !query 630 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 630 output +-- !query output true --- !query 631 +-- !query SELECT cast(1 as bigint) <=> cast(1 as decimal(10, 0)) FROM t --- !query 631 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 631 output +-- !query output true --- !query 632 +-- !query SELECT cast(1 as bigint) <=> cast(1 as decimal(20, 0)) FROM t --- !query 632 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) <=> CAST(1 AS DECIMAL(20,0))):boolean> --- !query 632 output +-- !query output true --- !query 633 +-- !query SELECT cast(1 as float) <=> cast(1 as decimal(3, 0)) FROM t --- !query 633 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 633 output +-- !query output true --- !query 634 +-- !query SELECT cast(1 as float) <=> cast(1 as decimal(5, 0)) FROM t --- !query 634 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 634 output +-- !query output true --- !query 635 +-- !query SELECT cast(1 as float) <=> cast(1 as decimal(10, 0)) FROM t --- !query 635 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 635 output +-- !query output true --- !query 636 +-- !query SELECT cast(1 as float) <=> cast(1 as decimal(20, 0)) FROM t --- !query 636 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 636 output +-- !query output true --- !query 637 +-- !query SELECT cast(1 as double) <=> cast(1 as decimal(3, 0)) FROM t --- !query 637 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 637 output +-- !query output true --- !query 638 +-- !query SELECT cast(1 as double) <=> cast(1 as decimal(5, 0)) FROM t --- !query 638 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 638 output +-- !query output true --- !query 639 +-- !query SELECT cast(1 as double) <=> cast(1 as decimal(10, 0)) FROM t --- !query 639 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 639 output +-- !query output true --- !query 640 +-- !query SELECT cast(1 as double) <=> cast(1 as decimal(20, 0)) FROM t --- !query 640 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 640 output +-- !query output true --- !query 641 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(3, 0)) FROM t --- !query 641 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 641 output +-- !query output true --- !query 642 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(5, 0)) FROM t --- !query 642 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 642 output +-- !query output true --- !query 643 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t --- !query 643 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> --- !query 643 output +-- !query output true --- !query 644 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(20, 0)) FROM t --- !query 644 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 644 output +-- !query output true --- !query 645 +-- !query SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t --- !query 645 schema +-- !query schema struct<> --- !query 645 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 646 +-- !query SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t --- !query 646 schema +-- !query schema struct<> --- !query 646 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 647 +-- !query SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t --- !query 647 schema +-- !query schema struct<> --- !query 647 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 648 +-- !query SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t --- !query 648 schema +-- !query schema struct<> --- !query 648 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 649 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t --- !query 649 schema +-- !query schema struct<> --- !query 649 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 650 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t --- !query 650 schema +-- !query schema struct<> --- !query 650 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 651 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t --- !query 651 schema +-- !query schema struct<> --- !query 651 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 652 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t --- !query 652 schema +-- !query schema struct<> --- !query 652 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 653 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t --- !query 653 schema +-- !query schema struct<> --- !query 653 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 654 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t --- !query 654 schema +-- !query schema struct<> --- !query 654 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 655 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t --- !query 655 schema +-- !query schema struct<> --- !query 655 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 656 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t --- !query 656 schema +-- !query schema struct<> --- !query 656 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <=> CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 657 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as tinyint) FROM t --- !query 657 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> --- !query 657 output +-- !query output true --- !query 658 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as tinyint) FROM t --- !query 658 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) <=> CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 658 output +-- !query output true --- !query 659 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as tinyint) FROM t --- !query 659 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 659 output +-- !query output true --- !query 660 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as tinyint) FROM t --- !query 660 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 660 output +-- !query output true --- !query 661 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as smallint) FROM t --- !query 661 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) <=> CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 661 output +-- !query output true --- !query 662 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as smallint) FROM t --- !query 662 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> --- !query 662 output +-- !query output true --- !query 663 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as smallint) FROM t --- !query 663 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <=> CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 663 output +-- !query output true --- !query 664 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as smallint) FROM t --- !query 664 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 664 output +-- !query output true --- !query 665 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as int) FROM t --- !query 665 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) <=> CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 665 output +-- !query output true --- !query 666 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as int) FROM t --- !query 666 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) <=> CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 666 output +-- !query output true --- !query 667 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as int) FROM t --- !query 667 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> --- !query 667 output +-- !query output true --- !query 668 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as int) FROM t --- !query 668 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 668 output +-- !query output true --- !query 669 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as bigint) FROM t --- !query 669 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 669 output +-- !query output true --- !query 670 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as bigint) FROM t --- !query 670 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 670 output +-- !query output true --- !query 671 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as bigint) FROM t --- !query 671 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) <=> CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 671 output +-- !query output true --- !query 672 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as bigint) FROM t --- !query 672 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> --- !query 672 output +-- !query output true --- !query 673 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as float) FROM t --- !query 673 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <=> CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 673 output +-- !query output true --- !query 674 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as float) FROM t --- !query 674 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <=> CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 674 output +-- !query output true --- !query 675 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as float) FROM t --- !query 675 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <=> CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 675 output +-- !query output true --- !query 676 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as float) FROM t --- !query 676 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <=> CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 676 output +-- !query output true --- !query 677 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as double) FROM t --- !query 677 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> --- !query 677 output +-- !query output true --- !query 678 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as double) FROM t --- !query 678 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> --- !query 678 output +-- !query output true --- !query 679 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as double) FROM t --- !query 679 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> --- !query 679 output +-- !query output true --- !query 680 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as double) FROM t --- !query 680 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> --- !query 680 output +-- !query output true --- !query 681 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as decimal(10, 0)) FROM t --- !query 681 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 681 output +-- !query output true --- !query 682 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as decimal(10, 0)) FROM t --- !query 682 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 682 output +-- !query output true --- !query 683 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t --- !query 683 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> --- !query 683 output +-- !query output true --- !query 684 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as decimal(10, 0)) FROM t --- !query 684 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 684 output +-- !query output true --- !query 685 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as string) FROM t --- !query 685 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <=> CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 685 output +-- !query output true --- !query 686 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as string) FROM t --- !query 686 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <=> CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 686 output +-- !query output true --- !query 687 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as string) FROM t --- !query 687 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <=> CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 687 output +-- !query output true --- !query 688 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as string) FROM t --- !query 688 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <=> CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 688 output +-- !query output true --- !query 689 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t --- !query 689 schema +-- !query schema struct<> --- !query 689 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <=> CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 690 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t --- !query 690 schema +-- !query schema struct<> --- !query 690 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <=> CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 691 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t --- !query 691 schema +-- !query schema struct<> --- !query 691 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <=> CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 692 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t --- !query 692 schema +-- !query schema struct<> --- !query 692 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) <=> CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <=> CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 693 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as boolean) FROM t --- !query 693 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(CAST(1 AS BOOLEAN) AS DECIMAL(3,0))):boolean> --- !query 693 output +-- !query output true --- !query 694 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as boolean) FROM t --- !query 694 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(CAST(1 AS BOOLEAN) AS DECIMAL(5,0))):boolean> --- !query 694 output +-- !query output true --- !query 695 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as boolean) FROM t --- !query 695 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(CAST(1 AS BOOLEAN) AS DECIMAL(10,0))):boolean> --- !query 695 output +-- !query output true --- !query 696 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as boolean) FROM t --- !query 696 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(CAST(1 AS BOOLEAN) AS DECIMAL(20,0))):boolean> --- !query 696 output +-- !query output true --- !query 697 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 697 schema +-- !query schema struct<> --- !query 697 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 698 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 698 schema +-- !query schema struct<> --- !query 698 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 699 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 699 schema +-- !query schema struct<> --- !query 699 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 700 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 700 schema +-- !query schema struct<> --- !query 700 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <=> CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 701 +-- !query SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t --- !query 701 schema +-- !query schema struct<> --- !query 701 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 702 +-- !query SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t --- !query 702 schema +-- !query schema struct<> --- !query 702 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 703 +-- !query SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t --- !query 703 schema +-- !query schema struct<> --- !query 703 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 704 +-- !query SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t --- !query 704 schema +-- !query schema struct<> --- !query 704 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <=> CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 705 +-- !query SELECT cast(1 as tinyint) < cast(1 as decimal(3, 0)) FROM t --- !query 705 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) < CAST(1 AS DECIMAL(3,0))):boolean> --- !query 705 output +-- !query output false --- !query 706 +-- !query SELECT cast(1 as tinyint) < cast(1 as decimal(5, 0)) FROM t --- !query 706 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) < CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 706 output +-- !query output false --- !query 707 +-- !query SELECT cast(1 as tinyint) < cast(1 as decimal(10, 0)) FROM t --- !query 707 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 707 output +-- !query output false --- !query 708 +-- !query SELECT cast(1 as tinyint) < cast(1 as decimal(20, 0)) FROM t --- !query 708 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 708 output +-- !query output false --- !query 709 +-- !query SELECT cast(1 as smallint) < cast(1 as decimal(3, 0)) FROM t --- !query 709 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) < CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 709 output +-- !query output false --- !query 710 +-- !query SELECT cast(1 as smallint) < cast(1 as decimal(5, 0)) FROM t --- !query 710 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) < CAST(1 AS DECIMAL(5,0))):boolean> --- !query 710 output +-- !query output false --- !query 711 +-- !query SELECT cast(1 as smallint) < cast(1 as decimal(10, 0)) FROM t --- !query 711 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 711 output +-- !query output false --- !query 712 +-- !query SELECT cast(1 as smallint) < cast(1 as decimal(20, 0)) FROM t --- !query 712 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 712 output +-- !query output false --- !query 713 +-- !query SELECT cast(1 as int) < cast(1 as decimal(3, 0)) FROM t --- !query 713 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 713 output +-- !query output false --- !query 714 +-- !query SELECT cast(1 as int) < cast(1 as decimal(5, 0)) FROM t --- !query 714 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 714 output +-- !query output false --- !query 715 +-- !query SELECT cast(1 as int) < cast(1 as decimal(10, 0)) FROM t --- !query 715 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(10,0))):boolean> --- !query 715 output +-- !query output false --- !query 716 +-- !query SELECT cast(1 as int) < cast(1 as decimal(20, 0)) FROM t --- !query 716 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 716 output +-- !query output false --- !query 717 +-- !query SELECT cast(1 as bigint) < cast(1 as decimal(3, 0)) FROM t --- !query 717 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 717 output +-- !query output false --- !query 718 +-- !query SELECT cast(1 as bigint) < cast(1 as decimal(5, 0)) FROM t --- !query 718 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 718 output +-- !query output false --- !query 719 +-- !query SELECT cast(1 as bigint) < cast(1 as decimal(10, 0)) FROM t --- !query 719 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 719 output +-- !query output false --- !query 720 +-- !query SELECT cast(1 as bigint) < cast(1 as decimal(20, 0)) FROM t --- !query 720 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) < CAST(1 AS DECIMAL(20,0))):boolean> --- !query 720 output +-- !query output false --- !query 721 +-- !query SELECT cast(1 as float) < cast(1 as decimal(3, 0)) FROM t --- !query 721 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 721 output +-- !query output false --- !query 722 +-- !query SELECT cast(1 as float) < cast(1 as decimal(5, 0)) FROM t --- !query 722 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 722 output +-- !query output false --- !query 723 +-- !query SELECT cast(1 as float) < cast(1 as decimal(10, 0)) FROM t --- !query 723 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 723 output +-- !query output false --- !query 724 +-- !query SELECT cast(1 as float) < cast(1 as decimal(20, 0)) FROM t --- !query 724 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) < CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 724 output +-- !query output false --- !query 725 +-- !query SELECT cast(1 as double) < cast(1 as decimal(3, 0)) FROM t --- !query 725 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 725 output +-- !query output false --- !query 726 +-- !query SELECT cast(1 as double) < cast(1 as decimal(5, 0)) FROM t --- !query 726 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 726 output +-- !query output false --- !query 727 +-- !query SELECT cast(1 as double) < cast(1 as decimal(10, 0)) FROM t --- !query 727 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 727 output +-- !query output false --- !query 728 +-- !query SELECT cast(1 as double) < cast(1 as decimal(20, 0)) FROM t --- !query 728 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 728 output +-- !query output false --- !query 729 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(3, 0)) FROM t --- !query 729 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 729 output +-- !query output false --- !query 730 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(5, 0)) FROM t --- !query 730 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 730 output +-- !query output false --- !query 731 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t --- !query 731 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(10,0))):boolean> --- !query 731 output +-- !query output false --- !query 732 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(20, 0)) FROM t --- !query 732 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 732 output +-- !query output false --- !query 733 +-- !query SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t --- !query 733 schema +-- !query schema struct<> --- !query 733 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 734 +-- !query SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t --- !query 734 schema +-- !query schema struct<> --- !query 734 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 735 +-- !query SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t --- !query 735 schema +-- !query schema struct<> --- !query 735 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 736 +-- !query SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t --- !query 736 schema +-- !query schema struct<> --- !query 736 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) < CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 737 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t --- !query 737 schema +-- !query schema struct<> --- !query 737 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 738 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t --- !query 738 schema +-- !query schema struct<> --- !query 738 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 739 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t --- !query 739 schema +-- !query schema struct<> --- !query 739 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 740 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t --- !query 740 schema +-- !query schema struct<> --- !query 740 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 741 +-- !query SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t --- !query 741 schema +-- !query schema struct<> --- !query 741 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 742 +-- !query SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t --- !query 742 schema +-- !query schema struct<> --- !query 742 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 743 +-- !query SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t --- !query 743 schema +-- !query schema struct<> --- !query 743 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 744 +-- !query SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t --- !query 744 schema +-- !query schema struct<> --- !query 744 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) < CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 745 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as tinyint) FROM t --- !query 745 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) < CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> --- !query 745 output +-- !query output false --- !query 746 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as tinyint) FROM t --- !query 746 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) < CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 746 output +-- !query output false --- !query 747 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as tinyint) FROM t --- !query 747 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 747 output +-- !query output false --- !query 748 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as tinyint) FROM t --- !query 748 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 748 output +-- !query output false --- !query 749 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as smallint) FROM t --- !query 749 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) < CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 749 output +-- !query output false --- !query 750 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as smallint) FROM t --- !query 750 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) < CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> --- !query 750 output +-- !query output false --- !query 751 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as smallint) FROM t --- !query 751 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) < CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 751 output +-- !query output false --- !query 752 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as smallint) FROM t --- !query 752 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 752 output +-- !query output false --- !query 753 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as int) FROM t --- !query 753 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) < CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 753 output +-- !query output false --- !query 754 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as int) FROM t --- !query 754 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) < CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 754 output +-- !query output false --- !query 755 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as int) FROM t --- !query 755 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) < CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> --- !query 755 output +-- !query output false --- !query 756 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as int) FROM t --- !query 756 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 756 output +-- !query output false --- !query 757 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as bigint) FROM t --- !query 757 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 757 output +-- !query output false --- !query 758 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as bigint) FROM t --- !query 758 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 758 output +-- !query output false --- !query 759 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as bigint) FROM t --- !query 759 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) < CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 759 output +-- !query output false --- !query 760 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as bigint) FROM t --- !query 760 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) < CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> --- !query 760 output +-- !query output false --- !query 761 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as float) FROM t --- !query 761 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) < CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 761 output +-- !query output false --- !query 762 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as float) FROM t --- !query 762 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) < CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 762 output +-- !query output false --- !query 763 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as float) FROM t --- !query 763 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 763 output +-- !query output false --- !query 764 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as float) FROM t --- !query 764 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) < CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 764 output +-- !query output false --- !query 765 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as double) FROM t --- !query 765 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 765 output +-- !query output false --- !query 766 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as double) FROM t --- !query 766 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 766 output +-- !query output false --- !query 767 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as double) FROM t --- !query 767 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 767 output +-- !query output false --- !query 768 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as double) FROM t --- !query 768 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 768 output +-- !query output false --- !query 769 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as decimal(10, 0)) FROM t --- !query 769 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 769 output +-- !query output false --- !query 770 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as decimal(10, 0)) FROM t --- !query 770 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 770 output +-- !query output false --- !query 771 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t --- !query 771 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(10,0))):boolean> --- !query 771 output +-- !query output false --- !query 772 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as decimal(10, 0)) FROM t --- !query 772 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) < CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 772 output +-- !query output false --- !query 773 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as string) FROM t --- !query 773 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) < CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 773 output +-- !query output false --- !query 774 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as string) FROM t --- !query 774 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) < CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 774 output +-- !query output false --- !query 775 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as string) FROM t --- !query 775 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 775 output +-- !query output false --- !query 776 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as string) FROM t --- !query 776 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) < CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 776 output +-- !query output false --- !query 777 +-- !query SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t --- !query 777 schema +-- !query schema struct<> --- !query 777 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) < CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) < CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 778 +-- !query SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t --- !query 778 schema +-- !query schema struct<> --- !query 778 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) < CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) < CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 779 +-- !query SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t --- !query 779 schema +-- !query schema struct<> --- !query 779 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) < CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) < CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 780 +-- !query SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t --- !query 780 schema +-- !query schema struct<> --- !query 780 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) < CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) < CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 781 +-- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t --- !query 781 schema +-- !query schema struct<> --- !query 781 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 782 +-- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t --- !query 782 schema +-- !query schema struct<> --- !query 782 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 783 +-- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t --- !query 783 schema +-- !query schema struct<> --- !query 783 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 784 +-- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t --- !query 784 schema +-- !query schema struct<> --- !query 784 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 785 +-- !query SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 785 schema +-- !query schema struct<> --- !query 785 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 786 +-- !query SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 786 schema +-- !query schema struct<> --- !query 786 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 787 +-- !query SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 787 schema +-- !query schema struct<> --- !query 787 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 788 +-- !query SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 788 schema +-- !query schema struct<> --- !query 788 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) < CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 789 +-- !query SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t --- !query 789 schema +-- !query schema struct<> --- !query 789 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) < CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) < CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 790 +-- !query SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t --- !query 790 schema +-- !query schema struct<> --- !query 790 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) < CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) < CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 791 +-- !query SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t --- !query 791 schema +-- !query schema struct<> --- !query 791 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) < CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) < CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 792 +-- !query SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t --- !query 792 schema +-- !query schema struct<> --- !query 792 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) < CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) < CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 793 +-- !query SELECT cast(1 as tinyint) <= cast(1 as decimal(3, 0)) FROM t --- !query 793 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) <= CAST(1 AS DECIMAL(3,0))):boolean> --- !query 793 output +-- !query output true --- !query 794 +-- !query SELECT cast(1 as tinyint) <= cast(1 as decimal(5, 0)) FROM t --- !query 794 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 794 output +-- !query output true --- !query 795 +-- !query SELECT cast(1 as tinyint) <= cast(1 as decimal(10, 0)) FROM t --- !query 795 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 795 output +-- !query output true --- !query 796 +-- !query SELECT cast(1 as tinyint) <= cast(1 as decimal(20, 0)) FROM t --- !query 796 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 796 output +-- !query output true --- !query 797 +-- !query SELECT cast(1 as smallint) <= cast(1 as decimal(3, 0)) FROM t --- !query 797 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 797 output +-- !query output true --- !query 798 +-- !query SELECT cast(1 as smallint) <= cast(1 as decimal(5, 0)) FROM t --- !query 798 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) <= CAST(1 AS DECIMAL(5,0))):boolean> --- !query 798 output +-- !query output true --- !query 799 +-- !query SELECT cast(1 as smallint) <= cast(1 as decimal(10, 0)) FROM t --- !query 799 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 799 output +-- !query output true --- !query 800 +-- !query SELECT cast(1 as smallint) <= cast(1 as decimal(20, 0)) FROM t --- !query 800 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 800 output +-- !query output true --- !query 801 +-- !query SELECT cast(1 as int) <= cast(1 as decimal(3, 0)) FROM t --- !query 801 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 801 output +-- !query output true --- !query 802 +-- !query SELECT cast(1 as int) <= cast(1 as decimal(5, 0)) FROM t --- !query 802 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 802 output +-- !query output true --- !query 803 +-- !query SELECT cast(1 as int) <= cast(1 as decimal(10, 0)) FROM t --- !query 803 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> --- !query 803 output +-- !query output true --- !query 804 +-- !query SELECT cast(1 as int) <= cast(1 as decimal(20, 0)) FROM t --- !query 804 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 804 output +-- !query output true --- !query 805 +-- !query SELECT cast(1 as bigint) <= cast(1 as decimal(3, 0)) FROM t --- !query 805 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 805 output +-- !query output true --- !query 806 +-- !query SELECT cast(1 as bigint) <= cast(1 as decimal(5, 0)) FROM t --- !query 806 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 806 output +-- !query output true --- !query 807 +-- !query SELECT cast(1 as bigint) <= cast(1 as decimal(10, 0)) FROM t --- !query 807 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 807 output +-- !query output true --- !query 808 +-- !query SELECT cast(1 as bigint) <= cast(1 as decimal(20, 0)) FROM t --- !query 808 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) <= CAST(1 AS DECIMAL(20,0))):boolean> --- !query 808 output +-- !query output true --- !query 809 +-- !query SELECT cast(1 as float) <= cast(1 as decimal(3, 0)) FROM t --- !query 809 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 809 output +-- !query output true --- !query 810 +-- !query SELECT cast(1 as float) <= cast(1 as decimal(5, 0)) FROM t --- !query 810 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 810 output +-- !query output true --- !query 811 +-- !query SELECT cast(1 as float) <= cast(1 as decimal(10, 0)) FROM t --- !query 811 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 811 output +-- !query output true --- !query 812 +-- !query SELECT cast(1 as float) <= cast(1 as decimal(20, 0)) FROM t --- !query 812 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 812 output +-- !query output true --- !query 813 +-- !query SELECT cast(1 as double) <= cast(1 as decimal(3, 0)) FROM t --- !query 813 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 813 output +-- !query output true --- !query 814 +-- !query SELECT cast(1 as double) <= cast(1 as decimal(5, 0)) FROM t --- !query 814 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 814 output +-- !query output true --- !query 815 +-- !query SELECT cast(1 as double) <= cast(1 as decimal(10, 0)) FROM t --- !query 815 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 815 output +-- !query output true --- !query 816 +-- !query SELECT cast(1 as double) <= cast(1 as decimal(20, 0)) FROM t --- !query 816 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 816 output +-- !query output true --- !query 817 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(3, 0)) FROM t --- !query 817 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 817 output +-- !query output true --- !query 818 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(5, 0)) FROM t --- !query 818 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 818 output +-- !query output true --- !query 819 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t --- !query 819 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> --- !query 819 output +-- !query output true --- !query 820 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(20, 0)) FROM t --- !query 820 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 820 output +-- !query output true --- !query 821 +-- !query SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t --- !query 821 schema +-- !query schema struct<> --- !query 821 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 822 +-- !query SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t --- !query 822 schema +-- !query schema struct<> --- !query 822 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 823 +-- !query SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t --- !query 823 schema +-- !query schema struct<> --- !query 823 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 824 +-- !query SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t --- !query 824 schema +-- !query schema struct<> --- !query 824 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) <= CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 825 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t --- !query 825 schema +-- !query schema struct<> --- !query 825 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 826 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t --- !query 826 schema +-- !query schema struct<> --- !query 826 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 827 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t --- !query 827 schema +-- !query schema struct<> --- !query 827 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 828 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t --- !query 828 schema +-- !query schema struct<> --- !query 828 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 829 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t --- !query 829 schema +-- !query schema struct<> --- !query 829 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 830 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t --- !query 830 schema +-- !query schema struct<> --- !query 830 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 831 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t --- !query 831 schema +-- !query schema struct<> --- !query 831 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 832 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t --- !query 832 schema +-- !query schema struct<> --- !query 832 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) <= CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 833 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as tinyint) FROM t --- !query 833 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> --- !query 833 output +-- !query output true --- !query 834 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as tinyint) FROM t --- !query 834 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) <= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 834 output +-- !query output true --- !query 835 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as tinyint) FROM t --- !query 835 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 835 output +-- !query output true --- !query 836 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as tinyint) FROM t --- !query 836 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 836 output +-- !query output true --- !query 837 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as smallint) FROM t --- !query 837 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) <= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 837 output +-- !query output true --- !query 838 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as smallint) FROM t --- !query 838 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> --- !query 838 output +-- !query output true --- !query 839 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as smallint) FROM t --- !query 839 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) <= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 839 output +-- !query output true --- !query 840 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as smallint) FROM t --- !query 840 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 840 output +-- !query output true --- !query 841 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as int) FROM t --- !query 841 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) <= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 841 output +-- !query output true --- !query 842 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as int) FROM t --- !query 842 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) <= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 842 output +-- !query output true --- !query 843 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as int) FROM t --- !query 843 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> --- !query 843 output +-- !query output true --- !query 844 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as int) FROM t --- !query 844 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 844 output +-- !query output true --- !query 845 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as bigint) FROM t --- !query 845 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 845 output +-- !query output true --- !query 846 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as bigint) FROM t --- !query 846 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 846 output +-- !query output true --- !query 847 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as bigint) FROM t --- !query 847 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) <= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 847 output +-- !query output true --- !query 848 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as bigint) FROM t --- !query 848 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> --- !query 848 output +-- !query output true --- !query 849 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as float) FROM t --- !query 849 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 849 output +-- !query output true --- !query 850 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as float) FROM t --- !query 850 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 850 output +-- !query output true --- !query 851 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as float) FROM t --- !query 851 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 851 output +-- !query output true --- !query 852 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as float) FROM t --- !query 852 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 852 output +-- !query output true --- !query 853 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as double) FROM t --- !query 853 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 853 output +-- !query output true --- !query 854 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as double) FROM t --- !query 854 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 854 output +-- !query output true --- !query 855 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as double) FROM t --- !query 855 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 855 output +-- !query output true --- !query 856 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as double) FROM t --- !query 856 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 856 output +-- !query output true --- !query 857 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as decimal(10, 0)) FROM t --- !query 857 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 857 output +-- !query output true --- !query 858 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as decimal(10, 0)) FROM t --- !query 858 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 858 output +-- !query output true --- !query 859 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t --- !query 859 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> --- !query 859 output +-- !query output true --- !query 860 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as decimal(10, 0)) FROM t --- !query 860 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 860 output +-- !query output true --- !query 861 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as string) FROM t --- !query 861 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) <= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 861 output +-- !query output true --- !query 862 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as string) FROM t --- !query 862 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) <= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 862 output +-- !query output true --- !query 863 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as string) FROM t --- !query 863 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 863 output +-- !query output true --- !query 864 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as string) FROM t --- !query 864 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) <= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 864 output +-- !query output true --- !query 865 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t --- !query 865 schema +-- !query schema struct<> --- !query 865 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) <= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <= CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 866 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t --- !query 866 schema +-- !query schema struct<> --- !query 866 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) <= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <= CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 867 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t --- !query 867 schema +-- !query schema struct<> --- !query 867 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) <= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <= CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 868 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t --- !query 868 schema +-- !query schema struct<> --- !query 868 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) <= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <= CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 869 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t --- !query 869 schema +-- !query schema struct<> --- !query 869 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 870 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t --- !query 870 schema +-- !query schema struct<> --- !query 870 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 871 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t --- !query 871 schema +-- !query schema struct<> --- !query 871 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 872 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t --- !query 872 schema +-- !query schema struct<> --- !query 872 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 873 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 873 schema +-- !query schema struct<> --- !query 873 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 874 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 874 schema +-- !query schema struct<> --- !query 874 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 875 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 875 schema +-- !query schema struct<> --- !query 875 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 876 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 876 schema +-- !query schema struct<> --- !query 876 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 877 +-- !query SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t --- !query 877 schema +-- !query schema struct<> --- !query 877 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 878 +-- !query SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t --- !query 878 schema +-- !query schema struct<> --- !query 878 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 879 +-- !query SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t --- !query 879 schema +-- !query schema struct<> --- !query 879 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 880 +-- !query SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t --- !query 880 schema +-- !query schema struct<> --- !query 880 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) <= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 881 +-- !query SELECT cast(1 as tinyint) > cast(1 as decimal(3, 0)) FROM t --- !query 881 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) > CAST(1 AS DECIMAL(3,0))):boolean> --- !query 881 output +-- !query output false --- !query 882 +-- !query SELECT cast(1 as tinyint) > cast(1 as decimal(5, 0)) FROM t --- !query 882 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) > CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 882 output +-- !query output false --- !query 883 +-- !query SELECT cast(1 as tinyint) > cast(1 as decimal(10, 0)) FROM t --- !query 883 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 883 output +-- !query output false --- !query 884 +-- !query SELECT cast(1 as tinyint) > cast(1 as decimal(20, 0)) FROM t --- !query 884 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 884 output +-- !query output false --- !query 885 +-- !query SELECT cast(1 as smallint) > cast(1 as decimal(3, 0)) FROM t --- !query 885 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) > CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 885 output +-- !query output false --- !query 886 +-- !query SELECT cast(1 as smallint) > cast(1 as decimal(5, 0)) FROM t --- !query 886 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) > CAST(1 AS DECIMAL(5,0))):boolean> --- !query 886 output +-- !query output false --- !query 887 +-- !query SELECT cast(1 as smallint) > cast(1 as decimal(10, 0)) FROM t --- !query 887 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 887 output +-- !query output false --- !query 888 +-- !query SELECT cast(1 as smallint) > cast(1 as decimal(20, 0)) FROM t --- !query 888 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 888 output +-- !query output false --- !query 889 +-- !query SELECT cast(1 as int) > cast(1 as decimal(3, 0)) FROM t --- !query 889 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 889 output +-- !query output false --- !query 890 +-- !query SELECT cast(1 as int) > cast(1 as decimal(5, 0)) FROM t --- !query 890 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 890 output +-- !query output false --- !query 891 +-- !query SELECT cast(1 as int) > cast(1 as decimal(10, 0)) FROM t --- !query 891 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(10,0))):boolean> --- !query 891 output +-- !query output false --- !query 892 +-- !query SELECT cast(1 as int) > cast(1 as decimal(20, 0)) FROM t --- !query 892 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 892 output +-- !query output false --- !query 893 +-- !query SELECT cast(1 as bigint) > cast(1 as decimal(3, 0)) FROM t --- !query 893 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 893 output +-- !query output false --- !query 894 +-- !query SELECT cast(1 as bigint) > cast(1 as decimal(5, 0)) FROM t --- !query 894 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 894 output +-- !query output false --- !query 895 +-- !query SELECT cast(1 as bigint) > cast(1 as decimal(10, 0)) FROM t --- !query 895 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 895 output +-- !query output false --- !query 896 +-- !query SELECT cast(1 as bigint) > cast(1 as decimal(20, 0)) FROM t --- !query 896 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) > CAST(1 AS DECIMAL(20,0))):boolean> --- !query 896 output +-- !query output false --- !query 897 +-- !query SELECT cast(1 as float) > cast(1 as decimal(3, 0)) FROM t --- !query 897 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 897 output +-- !query output false --- !query 898 +-- !query SELECT cast(1 as float) > cast(1 as decimal(5, 0)) FROM t --- !query 898 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 898 output +-- !query output false --- !query 899 +-- !query SELECT cast(1 as float) > cast(1 as decimal(10, 0)) FROM t --- !query 899 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 899 output +-- !query output false --- !query 900 +-- !query SELECT cast(1 as float) > cast(1 as decimal(20, 0)) FROM t --- !query 900 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) > CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 900 output +-- !query output false --- !query 901 +-- !query SELECT cast(1 as double) > cast(1 as decimal(3, 0)) FROM t --- !query 901 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 901 output +-- !query output false --- !query 902 +-- !query SELECT cast(1 as double) > cast(1 as decimal(5, 0)) FROM t --- !query 902 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 902 output +-- !query output false --- !query 903 +-- !query SELECT cast(1 as double) > cast(1 as decimal(10, 0)) FROM t --- !query 903 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 903 output +-- !query output false --- !query 904 +-- !query SELECT cast(1 as double) > cast(1 as decimal(20, 0)) FROM t --- !query 904 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 904 output +-- !query output false --- !query 905 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(3, 0)) FROM t --- !query 905 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 905 output +-- !query output false --- !query 906 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(5, 0)) FROM t --- !query 906 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 906 output +-- !query output false --- !query 907 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t --- !query 907 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(10,0))):boolean> --- !query 907 output +-- !query output false --- !query 908 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(20, 0)) FROM t --- !query 908 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 908 output +-- !query output false --- !query 909 +-- !query SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t --- !query 909 schema +-- !query schema struct<> --- !query 909 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 910 +-- !query SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t --- !query 910 schema +-- !query schema struct<> --- !query 910 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 911 +-- !query SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t --- !query 911 schema +-- !query schema struct<> --- !query 911 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 912 +-- !query SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t --- !query 912 schema +-- !query schema struct<> --- !query 912 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) > CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 913 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t --- !query 913 schema +-- !query schema struct<> --- !query 913 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 914 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t --- !query 914 schema +-- !query schema struct<> --- !query 914 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 915 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t --- !query 915 schema +-- !query schema struct<> --- !query 915 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 916 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t --- !query 916 schema +-- !query schema struct<> --- !query 916 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 917 +-- !query SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t --- !query 917 schema +-- !query schema struct<> --- !query 917 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 918 +-- !query SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t --- !query 918 schema +-- !query schema struct<> --- !query 918 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 919 +-- !query SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t --- !query 919 schema +-- !query schema struct<> --- !query 919 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 920 +-- !query SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t --- !query 920 schema +-- !query schema struct<> --- !query 920 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) > CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 921 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as tinyint) FROM t --- !query 921 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) > CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> --- !query 921 output +-- !query output false --- !query 922 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as tinyint) FROM t --- !query 922 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) > CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 922 output +-- !query output false --- !query 923 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as tinyint) FROM t --- !query 923 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 923 output +-- !query output false --- !query 924 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as tinyint) FROM t --- !query 924 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 924 output +-- !query output false --- !query 925 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as smallint) FROM t --- !query 925 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) > CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 925 output +-- !query output false --- !query 926 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as smallint) FROM t --- !query 926 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) > CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> --- !query 926 output +-- !query output false --- !query 927 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as smallint) FROM t --- !query 927 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) > CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 927 output +-- !query output false --- !query 928 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as smallint) FROM t --- !query 928 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 928 output +-- !query output false --- !query 929 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as int) FROM t --- !query 929 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) > CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 929 output +-- !query output false --- !query 930 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as int) FROM t --- !query 930 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) > CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 930 output +-- !query output false --- !query 931 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as int) FROM t --- !query 931 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) > CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> --- !query 931 output +-- !query output false --- !query 932 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as int) FROM t --- !query 932 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 932 output +-- !query output false --- !query 933 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as bigint) FROM t --- !query 933 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 933 output +-- !query output false --- !query 934 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as bigint) FROM t --- !query 934 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 934 output +-- !query output false --- !query 935 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as bigint) FROM t --- !query 935 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) > CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 935 output +-- !query output false --- !query 936 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as bigint) FROM t --- !query 936 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) > CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> --- !query 936 output +-- !query output false --- !query 937 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as float) FROM t --- !query 937 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) > CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 937 output +-- !query output false --- !query 938 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as float) FROM t --- !query 938 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) > CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 938 output +-- !query output false --- !query 939 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as float) FROM t --- !query 939 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 939 output +-- !query output false --- !query 940 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as float) FROM t --- !query 940 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) > CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 940 output +-- !query output false --- !query 941 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as double) FROM t --- !query 941 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 941 output +-- !query output false --- !query 942 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as double) FROM t --- !query 942 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 942 output +-- !query output false --- !query 943 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as double) FROM t --- !query 943 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 943 output +-- !query output false --- !query 944 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as double) FROM t --- !query 944 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 944 output +-- !query output false --- !query 945 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as decimal(10, 0)) FROM t --- !query 945 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 945 output +-- !query output false --- !query 946 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as decimal(10, 0)) FROM t --- !query 946 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 946 output +-- !query output false --- !query 947 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t --- !query 947 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(10,0))):boolean> --- !query 947 output +-- !query output false --- !query 948 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as decimal(10, 0)) FROM t --- !query 948 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) > CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 948 output +-- !query output false --- !query 949 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as string) FROM t --- !query 949 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) > CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 949 output +-- !query output false --- !query 950 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as string) FROM t --- !query 950 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) > CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 950 output +-- !query output false --- !query 951 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as string) FROM t --- !query 951 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 951 output +-- !query output false --- !query 952 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as string) FROM t --- !query 952 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) > CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 952 output +-- !query output false --- !query 953 +-- !query SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t --- !query 953 schema +-- !query schema struct<> --- !query 953 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) > CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) > CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 954 +-- !query SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t --- !query 954 schema +-- !query schema struct<> --- !query 954 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) > CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) > CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 955 +-- !query SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t --- !query 955 schema +-- !query schema struct<> --- !query 955 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) > CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) > CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 956 +-- !query SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t --- !query 956 schema +-- !query schema struct<> --- !query 956 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) > CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) > CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 957 +-- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t --- !query 957 schema +-- !query schema struct<> --- !query 957 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 958 +-- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t --- !query 958 schema +-- !query schema struct<> --- !query 958 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 959 +-- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t --- !query 959 schema +-- !query schema struct<> --- !query 959 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 960 +-- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t --- !query 960 schema +-- !query schema struct<> --- !query 960 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 961 +-- !query SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 961 schema +-- !query schema struct<> --- !query 961 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 962 +-- !query SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 962 schema +-- !query schema struct<> --- !query 962 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 963 +-- !query SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 963 schema +-- !query schema struct<> --- !query 963 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 964 +-- !query SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 964 schema +-- !query schema struct<> --- !query 964 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) > CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 965 +-- !query SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t --- !query 965 schema +-- !query schema struct<> --- !query 965 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) > CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) > CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 966 +-- !query SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t --- !query 966 schema +-- !query schema struct<> --- !query 966 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) > CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) > CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 967 +-- !query SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t --- !query 967 schema +-- !query schema struct<> --- !query 967 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) > CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) > CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 968 +-- !query SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t --- !query 968 schema +-- !query schema struct<> --- !query 968 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) > CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) > CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 969 +-- !query SELECT cast(1 as tinyint) >= cast(1 as decimal(3, 0)) FROM t --- !query 969 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) >= CAST(1 AS DECIMAL(3,0))):boolean> --- !query 969 output +-- !query output true --- !query 970 +-- !query SELECT cast(1 as tinyint) >= cast(1 as decimal(5, 0)) FROM t --- !query 970 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 970 output +-- !query output true --- !query 971 +-- !query SELECT cast(1 as tinyint) >= cast(1 as decimal(10, 0)) FROM t --- !query 971 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 971 output +-- !query output true --- !query 972 +-- !query SELECT cast(1 as tinyint) >= cast(1 as decimal(20, 0)) FROM t --- !query 972 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 972 output +-- !query output true --- !query 973 +-- !query SELECT cast(1 as smallint) >= cast(1 as decimal(3, 0)) FROM t --- !query 973 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 973 output +-- !query output true --- !query 974 +-- !query SELECT cast(1 as smallint) >= cast(1 as decimal(5, 0)) FROM t --- !query 974 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) >= CAST(1 AS DECIMAL(5,0))):boolean> --- !query 974 output +-- !query output true --- !query 975 +-- !query SELECT cast(1 as smallint) >= cast(1 as decimal(10, 0)) FROM t --- !query 975 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 975 output +-- !query output true --- !query 976 +-- !query SELECT cast(1 as smallint) >= cast(1 as decimal(20, 0)) FROM t --- !query 976 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 976 output +-- !query output true --- !query 977 +-- !query SELECT cast(1 as int) >= cast(1 as decimal(3, 0)) FROM t --- !query 977 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 977 output +-- !query output true --- !query 978 +-- !query SELECT cast(1 as int) >= cast(1 as decimal(5, 0)) FROM t --- !query 978 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 978 output +-- !query output true --- !query 979 +-- !query SELECT cast(1 as int) >= cast(1 as decimal(10, 0)) FROM t --- !query 979 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> --- !query 979 output +-- !query output true --- !query 980 +-- !query SELECT cast(1 as int) >= cast(1 as decimal(20, 0)) FROM t --- !query 980 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 980 output +-- !query output true --- !query 981 +-- !query SELECT cast(1 as bigint) >= cast(1 as decimal(3, 0)) FROM t --- !query 981 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 981 output +-- !query output true --- !query 982 +-- !query SELECT cast(1 as bigint) >= cast(1 as decimal(5, 0)) FROM t --- !query 982 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 982 output +-- !query output true --- !query 983 +-- !query SELECT cast(1 as bigint) >= cast(1 as decimal(10, 0)) FROM t --- !query 983 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 983 output +-- !query output true --- !query 984 +-- !query SELECT cast(1 as bigint) >= cast(1 as decimal(20, 0)) FROM t --- !query 984 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) >= CAST(1 AS DECIMAL(20,0))):boolean> --- !query 984 output +-- !query output true --- !query 985 +-- !query SELECT cast(1 as float) >= cast(1 as decimal(3, 0)) FROM t --- !query 985 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 985 output +-- !query output true --- !query 986 +-- !query SELECT cast(1 as float) >= cast(1 as decimal(5, 0)) FROM t --- !query 986 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 986 output +-- !query output true --- !query 987 +-- !query SELECT cast(1 as float) >= cast(1 as decimal(10, 0)) FROM t --- !query 987 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 987 output +-- !query output true --- !query 988 +-- !query SELECT cast(1 as float) >= cast(1 as decimal(20, 0)) FROM t --- !query 988 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 988 output +-- !query output true --- !query 989 +-- !query SELECT cast(1 as double) >= cast(1 as decimal(3, 0)) FROM t --- !query 989 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE)):boolean> --- !query 989 output +-- !query output true --- !query 990 +-- !query SELECT cast(1 as double) >= cast(1 as decimal(5, 0)) FROM t --- !query 990 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE)):boolean> --- !query 990 output +-- !query output true --- !query 991 +-- !query SELECT cast(1 as double) >= cast(1 as decimal(10, 0)) FROM t --- !query 991 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 991 output +-- !query output true --- !query 992 +-- !query SELECT cast(1 as double) >= cast(1 as decimal(20, 0)) FROM t --- !query 992 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE)):boolean> --- !query 992 output +-- !query output true --- !query 993 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(3, 0)) FROM t --- !query 993 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 993 output +-- !query output true --- !query 994 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(5, 0)) FROM t --- !query 994 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 994 output +-- !query output true --- !query 995 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t --- !query 995 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> --- !query 995 output +-- !query output true --- !query 996 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(20, 0)) FROM t --- !query 996 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 996 output +-- !query output true --- !query 997 +-- !query SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t --- !query 997 schema +-- !query schema struct<> --- !query 997 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 998 +-- !query SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t --- !query 998 schema +-- !query schema struct<> --- !query 998 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 999 +-- !query SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t --- !query 999 schema +-- !query schema struct<> --- !query 999 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 1000 +-- !query SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t --- !query 1000 schema +-- !query schema struct<> --- !query 1000 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) >= CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 1001 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t --- !query 1001 schema +-- !query schema struct<> --- !query 1001 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 1002 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t --- !query 1002 schema +-- !query schema struct<> --- !query 1002 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 1003 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t --- !query 1003 schema +-- !query schema struct<> --- !query 1003 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 1004 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t --- !query 1004 schema +-- !query schema struct<> --- !query 1004 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 1005 +-- !query SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t --- !query 1005 schema +-- !query schema struct<> --- !query 1005 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 1006 +-- !query SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t --- !query 1006 schema +-- !query schema struct<> --- !query 1006 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 1007 +-- !query SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t --- !query 1007 schema +-- !query schema struct<> --- !query 1007 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 1008 +-- !query SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t --- !query 1008 schema +-- !query schema struct<> --- !query 1008 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) >= CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 1009 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as tinyint) FROM t --- !query 1009 schema +-- !query schema struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0))):boolean> --- !query 1009 output +-- !query output true --- !query 1010 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as tinyint) FROM t --- !query 1010 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) >= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0))):boolean> --- !query 1010 output +-- !query output true --- !query 1011 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as tinyint) FROM t --- !query 1011 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):boolean> --- !query 1011 output +-- !query output true --- !query 1012 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as tinyint) FROM t --- !query 1012 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0))):boolean> --- !query 1012 output +-- !query output true --- !query 1013 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as smallint) FROM t --- !query 1013 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) >= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0))):boolean> --- !query 1013 output +-- !query output true --- !query 1014 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as smallint) FROM t --- !query 1014 schema +-- !query schema struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0))):boolean> --- !query 1014 output +-- !query output true --- !query 1015 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as smallint) FROM t --- !query 1015 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) >= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):boolean> --- !query 1015 output +-- !query output true --- !query 1016 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as smallint) FROM t --- !query 1016 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0))):boolean> --- !query 1016 output +-- !query output true --- !query 1017 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as int) FROM t --- !query 1017 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) >= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 1017 output +-- !query output true --- !query 1018 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as int) FROM t --- !query 1018 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) >= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 1018 output +-- !query output true --- !query 1019 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as int) FROM t --- !query 1019 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(CAST(1 AS INT) AS DECIMAL(10,0))):boolean> --- !query 1019 output +-- !query output true --- !query 1020 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as int) FROM t --- !query 1020 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 1020 output +-- !query output true --- !query 1021 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as bigint) FROM t --- !query 1021 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 1021 output +-- !query output true --- !query 1022 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as bigint) FROM t --- !query 1022 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 1022 output +-- !query output true --- !query 1023 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as bigint) FROM t --- !query 1023 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) >= CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):boolean> --- !query 1023 output +-- !query output true --- !query 1024 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as bigint) FROM t --- !query 1024 schema +-- !query schema struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0))):boolean> --- !query 1024 output +-- !query output true --- !query 1025 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as float) FROM t --- !query 1025 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) >= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 1025 output +-- !query output true --- !query 1026 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as float) FROM t --- !query 1026 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) >= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 1026 output +-- !query output true --- !query 1027 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as float) FROM t --- !query 1027 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 1027 output +-- !query output true --- !query 1028 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as float) FROM t --- !query 1028 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) >= CAST(CAST(1 AS FLOAT) AS DOUBLE)):boolean> --- !query 1028 output +-- !query output true --- !query 1029 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as double) FROM t --- !query 1029 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 1029 output +-- !query output true --- !query 1030 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as double) FROM t --- !query 1030 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 1030 output +-- !query output true --- !query 1031 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as double) FROM t --- !query 1031 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 1031 output +-- !query output true --- !query 1032 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as double) FROM t --- !query 1032 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 1032 output +-- !query output true --- !query 1033 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as decimal(10, 0)) FROM t --- !query 1033 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 1033 output +-- !query output true --- !query 1034 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as decimal(10, 0)) FROM t --- !query 1034 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):boolean> --- !query 1034 output +-- !query output true --- !query 1035 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t --- !query 1035 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> --- !query 1035 output +-- !query output true --- !query 1036 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as decimal(10, 0)) FROM t --- !query 1036 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):boolean> --- !query 1036 output +-- !query output true --- !query 1037 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as string) FROM t --- !query 1037 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) >= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 1037 output +-- !query output true --- !query 1038 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as string) FROM t --- !query 1038 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) >= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 1038 output +-- !query output true --- !query 1039 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as string) FROM t --- !query 1039 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 1039 output +-- !query output true --- !query 1040 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as string) FROM t --- !query 1040 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) >= CAST(CAST(1 AS STRING) AS DOUBLE)):boolean> --- !query 1040 output +-- !query output true --- !query 1041 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t --- !query 1041 schema +-- !query schema struct<> --- !query 1041 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) >= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) >= CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 1042 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t --- !query 1042 schema +-- !query schema struct<> --- !query 1042 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) >= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) >= CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 1043 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t --- !query 1043 schema +-- !query schema struct<> --- !query 1043 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) >= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) >= CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 1044 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t --- !query 1044 schema +-- !query schema struct<> --- !query 1044 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) >= CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) >= CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 1045 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t --- !query 1045 schema +-- !query schema struct<> --- !query 1045 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BOOLEAN))' (decimal(3,0) and boolean).; line 1 pos 7 --- !query 1046 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t --- !query 1046 schema +-- !query schema struct<> --- !query 1046 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BOOLEAN))' (decimal(5,0) and boolean).; line 1 pos 7 --- !query 1047 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t --- !query 1047 schema +-- !query schema struct<> --- !query 1047 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 1048 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t --- !query 1048 schema +-- !query schema struct<> --- !query 1048 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BOOLEAN))' (decimal(20,0) and boolean).; line 1 pos 7 --- !query 1049 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 1049 schema +-- !query schema struct<> --- !query 1049 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 1050 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 1050 schema +-- !query schema struct<> --- !query 1050 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 1051 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 1051 schema +-- !query schema struct<> --- !query 1051 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 1052 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 1052 schema +-- !query schema struct<> --- !query 1052 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) >= CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 1053 +-- !query SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t --- !query 1053 schema +-- !query schema struct<> --- !query 1053 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 1054 +-- !query SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t --- !query 1054 schema +-- !query schema struct<> --- !query 1054 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 1055 +-- !query SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t --- !query 1055 schema +-- !query schema struct<> --- !query 1055 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 1056 +-- !query SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t --- !query 1056 schema +-- !query schema struct<> --- !query 1056 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) >= CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 --- !query 1057 +-- !query SELECT cast(1 as tinyint) <> cast(1 as decimal(3, 0)) FROM t --- !query 1057 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) = CAST(1 AS DECIMAL(3,0)))):boolean> --- !query 1057 output +-- !query output false --- !query 1058 +-- !query SELECT cast(1 as tinyint) <> cast(1 as decimal(5, 0)) FROM t --- !query 1058 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)))):boolean> --- !query 1058 output +-- !query output false --- !query 1059 +-- !query SELECT cast(1 as tinyint) <> cast(1 as decimal(10, 0)) FROM t --- !query 1059 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 1059 output +-- !query output false --- !query 1060 +-- !query SELECT cast(1 as tinyint) <> cast(1 as decimal(20, 0)) FROM t --- !query 1060 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> --- !query 1060 output +-- !query output false --- !query 1061 +-- !query SELECT cast(1 as smallint) <> cast(1 as decimal(3, 0)) FROM t --- !query 1061 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)))):boolean> --- !query 1061 output +-- !query output false --- !query 1062 +-- !query SELECT cast(1 as smallint) <> cast(1 as decimal(5, 0)) FROM t --- !query 1062 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) = CAST(1 AS DECIMAL(5,0)))):boolean> --- !query 1062 output +-- !query output false --- !query 1063 +-- !query SELECT cast(1 as smallint) <> cast(1 as decimal(10, 0)) FROM t --- !query 1063 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 1063 output +-- !query output false --- !query 1064 +-- !query SELECT cast(1 as smallint) <> cast(1 as decimal(20, 0)) FROM t --- !query 1064 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> --- !query 1064 output +-- !query output false --- !query 1065 +-- !query SELECT cast(1 as int) <> cast(1 as decimal(3, 0)) FROM t --- !query 1065 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)))):boolean> --- !query 1065 output +-- !query output false --- !query 1066 +-- !query SELECT cast(1 as int) <> cast(1 as decimal(5, 0)) FROM t --- !query 1066 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)))):boolean> --- !query 1066 output +-- !query output false --- !query 1067 +-- !query SELECT cast(1 as int) <> cast(1 as decimal(10, 0)) FROM t --- !query 1067 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS INT) AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> --- !query 1067 output +-- !query output false --- !query 1068 +-- !query SELECT cast(1 as int) <> cast(1 as decimal(20, 0)) FROM t --- !query 1068 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> --- !query 1068 output +-- !query output false --- !query 1069 +-- !query SELECT cast(1 as bigint) <> cast(1 as decimal(3, 0)) FROM t --- !query 1069 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)))):boolean> --- !query 1069 output +-- !query output false --- !query 1070 +-- !query SELECT cast(1 as bigint) <> cast(1 as decimal(5, 0)) FROM t --- !query 1070 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)))):boolean> --- !query 1070 output +-- !query output false --- !query 1071 +-- !query SELECT cast(1 as bigint) <> cast(1 as decimal(10, 0)) FROM t --- !query 1071 schema +-- !query schema struct<(NOT (CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)))):boolean> --- !query 1071 output +-- !query output false --- !query 1072 +-- !query SELECT cast(1 as bigint) <> cast(1 as decimal(20, 0)) FROM t --- !query 1072 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) = CAST(1 AS DECIMAL(20,0)))):boolean> --- !query 1072 output +-- !query output false --- !query 1073 +-- !query SELECT cast(1 as float) <> cast(1 as decimal(3, 0)) FROM t --- !query 1073 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE))):boolean> --- !query 1073 output +-- !query output false --- !query 1074 +-- !query SELECT cast(1 as float) <> cast(1 as decimal(5, 0)) FROM t --- !query 1074 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE))):boolean> --- !query 1074 output +-- !query output false --- !query 1075 +-- !query SELECT cast(1 as float) <> cast(1 as decimal(10, 0)) FROM t --- !query 1075 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 1075 output +-- !query output false --- !query 1076 +-- !query SELECT cast(1 as float) <> cast(1 as decimal(20, 0)) FROM t --- !query 1076 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS FLOAT) AS DOUBLE) = CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE))):boolean> --- !query 1076 output +-- !query output false --- !query 1077 +-- !query SELECT cast(1 as double) <> cast(1 as decimal(3, 0)) FROM t --- !query 1077 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE))):boolean> --- !query 1077 output +-- !query output false --- !query 1078 +-- !query SELECT cast(1 as double) <> cast(1 as decimal(5, 0)) FROM t --- !query 1078 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE))):boolean> --- !query 1078 output +-- !query output false --- !query 1079 +-- !query SELECT cast(1 as double) <> cast(1 as decimal(10, 0)) FROM t --- !query 1079 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 1079 output +-- !query output false --- !query 1080 +-- !query SELECT cast(1 as double) <> cast(1 as decimal(20, 0)) FROM t --- !query 1080 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE))):boolean> --- !query 1080 output +-- !query output false --- !query 1081 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(3, 0)) FROM t --- !query 1081 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)))):boolean> --- !query 1081 output +-- !query output false --- !query 1082 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(5, 0)) FROM t --- !query 1082 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)))):boolean> --- !query 1082 output +-- !query output false --- !query 1083 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t --- !query 1083 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> --- !query 1083 output +-- !query output false --- !query 1084 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(20, 0)) FROM t --- !query 1084 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> --- !query 1084 output +-- !query output false --- !query 1085 +-- !query SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t --- !query 1085 schema +-- !query schema struct<> --- !query 1085 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(3,0)))' (binary and decimal(3,0)).; line 1 pos 7 --- !query 1086 +-- !query SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t --- !query 1086 schema +-- !query schema struct<> --- !query 1086 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(5,0)))' (binary and decimal(5,0)).; line 1 pos 7 --- !query 1087 +-- !query SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t --- !query 1087 schema +-- !query schema struct<> --- !query 1087 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 1088 +-- !query SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t --- !query 1088 schema +-- !query schema struct<> --- !query 1088 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) = CAST(1 AS DECIMAL(20,0)))' (binary and decimal(20,0)).; line 1 pos 7 --- !query 1089 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t --- !query 1089 schema +-- !query schema struct<> --- !query 1089 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))' (timestamp and decimal(3,0)).; line 1 pos 7 --- !query 1090 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t --- !query 1090 schema +-- !query schema struct<> --- !query 1090 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))' (timestamp and decimal(5,0)).; line 1 pos 7 --- !query 1091 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t --- !query 1091 schema +-- !query schema struct<> --- !query 1091 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 1092 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t --- !query 1092 schema +-- !query schema struct<> --- !query 1092 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))' (timestamp and decimal(20,0)).; line 1 pos 7 --- !query 1093 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t --- !query 1093 schema +-- !query schema struct<> --- !query 1093 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(3,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(3,0)))' (date and decimal(3,0)).; line 1 pos 7 --- !query 1094 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t --- !query 1094 schema +-- !query schema struct<> --- !query 1094 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(5,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(5,0)))' (date and decimal(5,0)).; line 1 pos 7 --- !query 1095 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t --- !query 1095 schema +-- !query schema struct<> --- !query 1095 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 1096 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t --- !query 1096 schema +-- !query schema struct<> --- !query 1096 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(20,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) = CAST(1 AS DECIMAL(20,0)))' (date and decimal(20,0)).; line 1 pos 7 --- !query 1097 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as tinyint) FROM t --- !query 1097 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)))):boolean> --- !query 1097 output +-- !query output false --- !query 1098 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as tinyint) FROM t --- !query 1098 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(5,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(5,0)))):boolean> --- !query 1098 output +-- !query output false --- !query 1099 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as tinyint) FROM t --- !query 1099 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)))):boolean> --- !query 1099 output +-- !query output false --- !query 1100 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as tinyint) FROM t --- !query 1100 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(20,0)))):boolean> --- !query 1100 output +-- !query output false --- !query 1101 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as smallint) FROM t --- !query 1101 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(5,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(5,0)))):boolean> --- !query 1101 output +-- !query output false --- !query 1102 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as smallint) FROM t --- !query 1102 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)))):boolean> --- !query 1102 output +-- !query output false --- !query 1103 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as smallint) FROM t --- !query 1103 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)))):boolean> --- !query 1103 output +-- !query output false --- !query 1104 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as smallint) FROM t --- !query 1104 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(20,0)))):boolean> --- !query 1104 output +-- !query output false --- !query 1105 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as int) FROM t --- !query 1105 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 1105 output +-- !query output false --- !query 1106 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as int) FROM t --- !query 1106 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 1106 output +-- !query output false --- !query 1107 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as int) FROM t --- !query 1107 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(CAST(1 AS INT) AS DECIMAL(10,0)))):boolean> --- !query 1107 output +-- !query output false --- !query 1108 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as int) FROM t --- !query 1108 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) AS DECIMAL(20,0)))):boolean> --- !query 1108 output +-- !query output false --- !query 1109 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as bigint) FROM t --- !query 1109 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> --- !query 1109 output +-- !query output false --- !query 1110 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as bigint) FROM t --- !query 1110 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> --- !query 1110 output +-- !query output false --- !query 1111 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as bigint) FROM t --- !query 1111 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) = CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)))):boolean> --- !query 1111 output +-- !query output false --- !query 1112 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as bigint) FROM t --- !query 1112 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)))):boolean> --- !query 1112 output +-- !query output false --- !query 1113 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as float) FROM t --- !query 1113 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> --- !query 1113 output +-- !query output false --- !query 1114 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as float) FROM t --- !query 1114 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> --- !query 1114 output +-- !query output false --- !query 1115 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as float) FROM t --- !query 1115 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> --- !query 1115 output +-- !query output false --- !query 1116 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as float) FROM t --- !query 1116 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> --- !query 1116 output +-- !query output false --- !query 1117 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as double) FROM t --- !query 1117 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 1117 output +-- !query output false --- !query 1118 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as double) FROM t --- !query 1118 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 1118 output +-- !query output false --- !query 1119 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as double) FROM t --- !query 1119 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 1119 output +-- !query output false --- !query 1120 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as double) FROM t --- !query 1120 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 1120 output +-- !query output false --- !query 1121 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as decimal(10, 0)) FROM t --- !query 1121 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 1121 output +-- !query output false --- !query 1122 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as decimal(10, 0)) FROM t --- !query 1122 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DECIMAL(10,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 1122 output +-- !query output false --- !query 1123 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t --- !query 1123 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> --- !query 1123 output +-- !query output false --- !query 1124 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as decimal(10, 0)) FROM t --- !query 1124 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DECIMAL(20,0)) = CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)))):boolean> --- !query 1124 output +-- !query output false --- !query 1125 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as string) FROM t --- !query 1125 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(3,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE))):boolean> --- !query 1125 output +-- !query output false --- !query 1126 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as string) FROM t --- !query 1126 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(5,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE))):boolean> --- !query 1126 output +-- !query output false --- !query 1127 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as string) FROM t --- !query 1127 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE))):boolean> --- !query 1127 output +-- !query output false --- !query 1128 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as string) FROM t --- !query 1128 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(20,0)) AS DOUBLE) = CAST(CAST(1 AS STRING) AS DOUBLE))):boolean> --- !query 1128 output +-- !query output false --- !query 1129 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t --- !query 1129 schema +-- !query schema struct<> --- !query 1129 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('1' AS BINARY))' (decimal(3,0) and binary).; line 1 pos 7 --- !query 1130 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t --- !query 1130 schema +-- !query schema struct<> --- !query 1130 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('1' AS BINARY))' (decimal(5,0) and binary).; line 1 pos 7 --- !query 1131 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t --- !query 1131 schema +-- !query schema struct<> --- !query 1131 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 1132 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t --- !query 1132 schema +-- !query schema struct<> --- !query 1132 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('1' AS BINARY))' (decimal(20,0) and binary).; line 1 pos 7 --- !query 1133 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as boolean) FROM t --- !query 1133 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(3,0)))):boolean> --- !query 1133 output +-- !query output false --- !query 1134 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as boolean) FROM t --- !query 1134 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(5,0)))):boolean> --- !query 1134 output +-- !query output false --- !query 1135 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as boolean) FROM t --- !query 1135 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(10,0)))):boolean> --- !query 1135 output +-- !query output false --- !query 1136 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as boolean) FROM t --- !query 1136 schema +-- !query schema struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(CAST(1 AS BOOLEAN) AS DECIMAL(20,0)))):boolean> --- !query 1136 output +-- !query output false --- !query 1137 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 1137 schema +-- !query schema struct<> --- !query 1137 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(3,0) and timestamp).; line 1 pos 7 --- !query 1138 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 1138 schema +-- !query schema struct<> --- !query 1138 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(5,0) and timestamp).; line 1 pos 7 --- !query 1139 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 1139 schema +-- !query schema struct<> --- !query 1139 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 1140 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 1140 schema +-- !query schema struct<> --- !query 1140 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(20,0) and timestamp).; line 1 pos 7 --- !query 1141 +-- !query SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t --- !query 1141 schema +-- !query schema struct<> --- !query 1141 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(3,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(3,0) and date).; line 1 pos 7 --- !query 1142 +-- !query SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t --- !query 1142 schema +-- !query schema struct<> --- !query 1142 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(5,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(5,0) and date).; line 1 pos 7 --- !query 1143 +-- !query SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t --- !query 1143 schema +-- !query schema struct<> --- !query 1143 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 1144 +-- !query SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t --- !query 1144 schema +-- !query schema struct<> --- !query 1144 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(20,0)) = CAST('2017-12-11 09:30:00' AS DATE))' (decimal(20,0) and date).; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out index 97cdf01763a70..ae933da59f63f 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out @@ -2,1241 +2,1241 @@ -- Number of queries: 145 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT cast(1 as tinyint) / cast(1 as tinyint) FROM t --- !query 1 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 1 output +-- !query output 1.0 --- !query 2 +-- !query SELECT cast(1 as tinyint) / cast(1 as smallint) FROM t --- !query 2 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 2 output +-- !query output 1.0 --- !query 3 +-- !query SELECT cast(1 as tinyint) / cast(1 as int) FROM t --- !query 3 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 3 output +-- !query output 1.0 --- !query 4 +-- !query SELECT cast(1 as tinyint) / cast(1 as bigint) FROM t --- !query 4 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 4 output +-- !query output 1.0 --- !query 5 +-- !query SELECT cast(1 as tinyint) / cast(1 as float) FROM t --- !query 5 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 5 output +-- !query output 1.0 --- !query 6 +-- !query SELECT cast(1 as tinyint) / cast(1 as double) FROM t --- !query 6 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 6 output +-- !query output 1.0 --- !query 7 +-- !query SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t --- !query 7 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(14,11)> --- !query 7 output +-- !query output 1.00000000000 --- !query 8 +-- !query SELECT cast(1 as tinyint) / cast(1 as string) FROM t --- !query 8 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double> --- !query 8 output +-- !query output 1.0 --- !query 9 +-- !query SELECT cast(1 as tinyint) / cast('1' as binary) FROM t --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / CAST('1' AS BINARY))' (tinyint and binary).; line 1 pos 7 --- !query 10 +-- !query SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))' (tinyint and boolean).; line 1 pos 7 --- !query 11 +-- !query SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (tinyint and timestamp).; line 1 pos 7 --- !query 12 +-- !query SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) / CAST('2017-12-11 09:30:00' AS DATE))' (tinyint and date).; line 1 pos 7 --- !query 13 +-- !query SELECT cast(1 as smallint) / cast(1 as tinyint) FROM t --- !query 13 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 13 output +-- !query output 1.0 --- !query 14 +-- !query SELECT cast(1 as smallint) / cast(1 as smallint) FROM t --- !query 14 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 14 output +-- !query output 1.0 --- !query 15 +-- !query SELECT cast(1 as smallint) / cast(1 as int) FROM t --- !query 15 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 15 output +-- !query output 1.0 --- !query 16 +-- !query SELECT cast(1 as smallint) / cast(1 as bigint) FROM t --- !query 16 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 16 output +-- !query output 1.0 --- !query 17 +-- !query SELECT cast(1 as smallint) / cast(1 as float) FROM t --- !query 17 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 17 output +-- !query output 1.0 --- !query 18 +-- !query SELECT cast(1 as smallint) / cast(1 as double) FROM t --- !query 18 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 18 output +-- !query output 1.0 --- !query 19 +-- !query SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t --- !query 19 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0))):decimal(16,11)> --- !query 19 output +-- !query output 1.00000000000 --- !query 20 +-- !query SELECT cast(1 as smallint) / cast(1 as string) FROM t --- !query 20 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double> --- !query 20 output +-- !query output 1.0 --- !query 21 +-- !query SELECT cast(1 as smallint) / cast('1' as binary) FROM t --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) / CAST('1' AS BINARY))' (smallint and binary).; line 1 pos 7 --- !query 22 +-- !query SELECT cast(1 as smallint) / cast(1 as boolean) FROM t --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))' (smallint and boolean).; line 1 pos 7 --- !query 23 +-- !query SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (smallint and timestamp).; line 1 pos 7 --- !query 24 +-- !query SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) / CAST('2017-12-11 09:30:00' AS DATE))' (smallint and date).; line 1 pos 7 --- !query 25 +-- !query SELECT cast(1 as int) / cast(1 as tinyint) FROM t --- !query 25 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 25 output +-- !query output 1.0 --- !query 26 +-- !query SELECT cast(1 as int) / cast(1 as smallint) FROM t --- !query 26 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 26 output +-- !query output 1.0 --- !query 27 +-- !query SELECT cast(1 as int) / cast(1 as int) FROM t --- !query 27 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 27 output +-- !query output 1.0 --- !query 28 +-- !query SELECT cast(1 as int) / cast(1 as bigint) FROM t --- !query 28 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 28 output +-- !query output 1.0 --- !query 29 +-- !query SELECT cast(1 as int) / cast(1 as float) FROM t --- !query 29 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 29 output +-- !query output 1.0 --- !query 30 +-- !query SELECT cast(1 as int) / cast(1 as double) FROM t --- !query 30 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 30 output +-- !query output 1.0 --- !query 31 +-- !query SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t --- !query 31 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> --- !query 31 output +-- !query output 1.00000000000 --- !query 32 +-- !query SELECT cast(1 as int) / cast(1 as string) FROM t --- !query 32 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double> --- !query 32 output +-- !query output 1.0 --- !query 33 +-- !query SELECT cast(1 as int) / cast('1' as binary) FROM t --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS INT) / CAST('1' AS BINARY))' (int and binary).; line 1 pos 7 --- !query 34 +-- !query SELECT cast(1 as int) / cast(1 as boolean) FROM t --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS INT) / CAST(1 AS BOOLEAN))' (int and boolean).; line 1 pos 7 --- !query 35 +-- !query SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS INT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (int and timestamp).; line 1 pos 7 --- !query 36 +-- !query SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS INT) / CAST('2017-12-11 09:30:00' AS DATE))' (int and date).; line 1 pos 7 --- !query 37 +-- !query SELECT cast(1 as bigint) / cast(1 as tinyint) FROM t --- !query 37 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 37 output +-- !query output 1.0 --- !query 38 +-- !query SELECT cast(1 as bigint) / cast(1 as smallint) FROM t --- !query 38 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 38 output +-- !query output 1.0 --- !query 39 +-- !query SELECT cast(1 as bigint) / cast(1 as int) FROM t --- !query 39 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 39 output +-- !query output 1.0 --- !query 40 +-- !query SELECT cast(1 as bigint) / cast(1 as bigint) FROM t --- !query 40 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 40 output +-- !query output 1.0 --- !query 41 +-- !query SELECT cast(1 as bigint) / cast(1 as float) FROM t --- !query 41 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 41 output +-- !query output 1.0 --- !query 42 +-- !query SELECT cast(1 as bigint) / cast(1 as double) FROM t --- !query 42 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 42 output +-- !query output 1.0 --- !query 43 +-- !query SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t --- !query 43 schema +-- !query schema struct<(CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0)) / CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0))):decimal(31,11)> --- !query 43 output +-- !query output 1.00000000000 --- !query 44 +-- !query SELECT cast(1 as bigint) / cast(1 as string) FROM t --- !query 44 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double> --- !query 44 output +-- !query output 1.0 --- !query 45 +-- !query SELECT cast(1 as bigint) / cast('1' as binary) FROM t --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) / CAST('1' AS BINARY))' (bigint and binary).; line 1 pos 7 --- !query 46 +-- !query SELECT cast(1 as bigint) / cast(1 as boolean) FROM t --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))' (bigint and boolean).; line 1 pos 7 --- !query 47 +-- !query SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (bigint and timestamp).; line 1 pos 7 --- !query 48 +-- !query SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) / CAST('2017-12-11 09:30:00' AS DATE))' (bigint and date).; line 1 pos 7 --- !query 49 +-- !query SELECT cast(1 as float) / cast(1 as tinyint) FROM t --- !query 49 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 49 output +-- !query output 1.0 --- !query 50 +-- !query SELECT cast(1 as float) / cast(1 as smallint) FROM t --- !query 50 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 50 output +-- !query output 1.0 --- !query 51 +-- !query SELECT cast(1 as float) / cast(1 as int) FROM t --- !query 51 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 51 output +-- !query output 1.0 --- !query 52 +-- !query SELECT cast(1 as float) / cast(1 as bigint) FROM t --- !query 52 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 52 output +-- !query output 1.0 --- !query 53 +-- !query SELECT cast(1 as float) / cast(1 as float) FROM t --- !query 53 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 53 output +-- !query output 1.0 --- !query 54 +-- !query SELECT cast(1 as float) / cast(1 as double) FROM t --- !query 54 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 54 output +-- !query output 1.0 --- !query 55 +-- !query SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t --- !query 55 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) AS DOUBLE)):double> --- !query 55 output +-- !query output 1.0 --- !query 56 +-- !query SELECT cast(1 as float) / cast(1 as string) FROM t --- !query 56 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(CAST(1 AS STRING) AS DOUBLE) AS DOUBLE)):double> --- !query 56 output +-- !query output 1.0 --- !query 57 +-- !query SELECT cast(1 as float) / cast('1' as binary) FROM t --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) / CAST('1' AS BINARY))' (float and binary).; line 1 pos 7 --- !query 58 +-- !query SELECT cast(1 as float) / cast(1 as boolean) FROM t --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))' (float and boolean).; line 1 pos 7 --- !query 59 +-- !query SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (float and timestamp).; line 1 pos 7 --- !query 60 +-- !query SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) / CAST('2017-12-11 09:30:00' AS DATE))' (float and date).; line 1 pos 7 --- !query 61 +-- !query SELECT cast(1 as double) / cast(1 as tinyint) FROM t --- !query 61 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 61 output +-- !query output 1.0 --- !query 62 +-- !query SELECT cast(1 as double) / cast(1 as smallint) FROM t --- !query 62 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 62 output +-- !query output 1.0 --- !query 63 +-- !query SELECT cast(1 as double) / cast(1 as int) FROM t --- !query 63 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 63 output +-- !query output 1.0 --- !query 64 +-- !query SELECT cast(1 as double) / cast(1 as bigint) FROM t --- !query 64 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 64 output +-- !query output 1.0 --- !query 65 +-- !query SELECT cast(1 as double) / cast(1 as float) FROM t --- !query 65 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 65 output +-- !query output 1.0 --- !query 66 +-- !query SELECT cast(1 as double) / cast(1 as double) FROM t --- !query 66 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 66 output +-- !query output 1.0 --- !query 67 +-- !query SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t --- !query 67 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 67 output +-- !query output 1.0 --- !query 68 +-- !query SELECT cast(1 as double) / cast(1 as string) FROM t --- !query 68 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 68 output +-- !query output 1.0 --- !query 69 +-- !query SELECT cast(1 as double) / cast('1' as binary) FROM t --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) / CAST('1' AS BINARY))' (double and binary).; line 1 pos 7 --- !query 70 +-- !query SELECT cast(1 as double) / cast(1 as boolean) FROM t --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7 --- !query 71 +-- !query SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 --- !query 72 +-- !query SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 --- !query 73 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t --- !query 73 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS TINYINT) AS DECIMAL(3,0)) AS DECIMAL(10,0))):decimal(16,6)> --- !query 73 output +-- !query output 1.000000 --- !query 74 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t --- !query 74 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) / CAST(CAST(CAST(1 AS SMALLINT) AS DECIMAL(5,0)) AS DECIMAL(10,0))):decimal(16,6)> --- !query 74 output +-- !query output 1.000000 --- !query 75 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t --- !query 75 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(CAST(1 AS INT) AS DECIMAL(10,0))):decimal(21,11)> --- !query 75 output +-- !query output 1.00000000000 --- !query 76 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t --- !query 76 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) / CAST(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) AS DECIMAL(20,0))):decimal(31,21)> --- !query 76 output +-- !query output 1.000000000000000000000 --- !query 77 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t --- !query 77 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 77 output +-- !query output 1.0 --- !query 78 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t --- !query 78 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 78 output +-- !query output 1.0 --- !query 79 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t --- !query 79 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> --- !query 79 output +-- !query output 1.00000000000 --- !query 80 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t --- !query 80 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 80 output +-- !query output 1.0 --- !query 81 +-- !query SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('1' AS BINARY))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 82 +-- !query SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 83 +-- !query SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 84 +-- !query SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) / CAST('2017-12-11 09:30:00' AS DATE))' (decimal(10,0) and date).; line 1 pos 7 --- !query 85 +-- !query SELECT cast(1 as string) / cast(1 as tinyint) FROM t --- !query 85 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 85 output +-- !query output 1.0 --- !query 86 +-- !query SELECT cast(1 as string) / cast(1 as smallint) FROM t --- !query 86 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 86 output +-- !query output 1.0 --- !query 87 +-- !query SELECT cast(1 as string) / cast(1 as int) FROM t --- !query 87 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 87 output +-- !query output 1.0 --- !query 88 +-- !query SELECT cast(1 as string) / cast(1 as bigint) FROM t --- !query 88 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 88 output +-- !query output 1.0 --- !query 89 +-- !query SELECT cast(1 as string) / cast(1 as float) FROM t --- !query 89 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 89 output +-- !query output 1.0 --- !query 90 +-- !query SELECT cast(1 as string) / cast(1 as double) FROM t --- !query 90 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 90 output +-- !query output 1.0 --- !query 91 +-- !query SELECT cast(1 as string) / cast(1 as decimal(10, 0)) FROM t --- !query 91 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 91 output +-- !query output 1.0 --- !query 92 +-- !query SELECT cast(1 as string) / cast(1 as string) FROM t --- !query 92 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(CAST(1 AS STRING) AS DOUBLE)):double> --- !query 92 output +-- !query output 1.0 --- !query 93 +-- !query SELECT cast(1 as string) / cast('1' as binary) FROM t --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('1' AS BINARY))' (double and binary).; line 1 pos 7 --- !query 94 +-- !query SELECT cast(1 as string) / cast(1 as boolean) FROM t --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7 --- !query 95 +-- !query SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 95 schema +-- !query schema struct<> --- !query 95 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 --- !query 96 +-- !query SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 96 schema +-- !query schema struct<> --- !query 96 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(CAST(1 AS STRING) AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 --- !query 97 +-- !query SELECT cast('1' as binary) / cast(1 as tinyint) FROM t --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS TINYINT))' (binary and tinyint).; line 1 pos 7 --- !query 98 +-- !query SELECT cast('1' as binary) / cast(1 as smallint) FROM t --- !query 98 schema +-- !query schema struct<> --- !query 98 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS SMALLINT))' (binary and smallint).; line 1 pos 7 --- !query 99 +-- !query SELECT cast('1' as binary) / cast(1 as int) FROM t --- !query 99 schema +-- !query schema struct<> --- !query 99 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS INT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS INT))' (binary and int).; line 1 pos 7 --- !query 100 +-- !query SELECT cast('1' as binary) / cast(1 as bigint) FROM t --- !query 100 schema +-- !query schema struct<> --- !query 100 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS BIGINT))' (binary and bigint).; line 1 pos 7 --- !query 101 +-- !query SELECT cast('1' as binary) / cast(1 as float) FROM t --- !query 101 schema +-- !query schema struct<> --- !query 101 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS FLOAT))' (binary and float).; line 1 pos 7 --- !query 102 +-- !query SELECT cast('1' as binary) / cast(1 as double) FROM t --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DOUBLE))' (binary and double).; line 1 pos 7 --- !query 103 +-- !query SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS DECIMAL(10,0)))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 104 +-- !query SELECT cast('1' as binary) / cast(1 as string) FROM t --- !query 104 schema +-- !query schema struct<> --- !query 104 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(CAST(1 AS STRING) AS DOUBLE))' (binary and double).; line 1 pos 7 --- !query 105 +-- !query SELECT cast('1' as binary) / cast('1' as binary) FROM t --- !query 105 schema +-- !query schema struct<> --- !query 105 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST('1' AS BINARY))' due to data type mismatch: '(CAST('1' AS BINARY) / CAST('1' AS BINARY))' requires (double or decimal) type, not binary; line 1 pos 7 --- !query 106 +-- !query SELECT cast('1' as binary) / cast(1 as boolean) FROM t --- !query 106 schema +-- !query schema struct<> --- !query 106 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST(1 AS BOOLEAN))' (binary and boolean).; line 1 pos 7 --- !query 107 +-- !query SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (binary and timestamp).; line 1 pos 7 --- !query 108 +-- !query SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST('2017-12-11 09:30:00' AS DATE))' (binary and date).; line 1 pos 7 --- !query 109 +-- !query SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t --- !query 109 schema +-- !query schema struct<> --- !query 109 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))' (boolean and tinyint).; line 1 pos 7 --- !query 110 +-- !query SELECT cast(1 as boolean) / cast(1 as smallint) FROM t --- !query 110 schema +-- !query schema struct<> --- !query 110 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))' (boolean and smallint).; line 1 pos 7 --- !query 111 +-- !query SELECT cast(1 as boolean) / cast(1 as int) FROM t --- !query 111 schema +-- !query schema struct<> --- !query 111 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS INT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS INT))' (boolean and int).; line 1 pos 7 --- !query 112 +-- !query SELECT cast(1 as boolean) / cast(1 as bigint) FROM t --- !query 112 schema +-- !query schema struct<> --- !query 112 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))' (boolean and bigint).; line 1 pos 7 --- !query 113 +-- !query SELECT cast(1 as boolean) / cast(1 as float) FROM t --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))' (boolean and float).; line 1 pos 7 --- !query 114 +-- !query SELECT cast(1 as boolean) / cast(1 as double) FROM t --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))' (boolean and double).; line 1 pos 7 --- !query 115 +-- !query SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))' (boolean and decimal(10,0)).; line 1 pos 7 --- !query 116 +-- !query SELECT cast(1 as boolean) / cast(1 as string) FROM t --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST(CAST(1 AS STRING) AS DOUBLE))' (boolean and double).; line 1 pos 7 --- !query 117 +-- !query SELECT cast(1 as boolean) / cast('1' as binary) FROM t --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST('1' AS BINARY))' (boolean and binary).; line 1 pos 7 --- !query 118 +-- !query SELECT cast(1 as boolean) / cast(1 as boolean) FROM t --- !query 118 schema +-- !query schema struct<> --- !query 118 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))' due to data type mismatch: '(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))' requires (double or decimal) type, not boolean; line 1 pos 7 --- !query 119 +-- !query SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 119 schema +-- !query schema struct<> --- !query 119 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (boolean and timestamp).; line 1 pos 7 --- !query 120 +-- !query SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST('2017-12-11 09:30:00' AS DATE))' (boolean and date).; line 1 pos 7 --- !query 121 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS TINYINT))' (timestamp and tinyint).; line 1 pos 7 --- !query 122 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t --- !query 122 schema +-- !query schema struct<> --- !query 122 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS SMALLINT))' (timestamp and smallint).; line 1 pos 7 --- !query 123 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t --- !query 123 schema +-- !query schema struct<> --- !query 123 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS INT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS INT))' (timestamp and int).; line 1 pos 7 --- !query 124 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS BIGINT))' (timestamp and bigint).; line 1 pos 7 --- !query 125 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t --- !query 125 schema +-- !query schema struct<> --- !query 125 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS FLOAT))' (timestamp and float).; line 1 pos 7 --- !query 126 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t --- !query 126 schema +-- !query schema struct<> --- !query 126 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DOUBLE))' (timestamp and double).; line 1 pos 7 --- !query 127 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 128 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t --- !query 128 schema +-- !query schema struct<> --- !query 128 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(CAST(1 AS STRING) AS DOUBLE))' (timestamp and double).; line 1 pos 7 --- !query 129 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t --- !query 129 schema +-- !query schema struct<> --- !query 129 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('1' AS BINARY))' (timestamp and binary).; line 1 pos 7 --- !query 130 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t --- !query 130 schema +-- !query schema struct<> --- !query 130 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST(1 AS BOOLEAN))' (timestamp and boolean).; line 1 pos 7 --- !query 131 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 131 schema +-- !query schema struct<> --- !query 131 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' requires (double or decimal) type, not timestamp; line 1 pos 7 --- !query 132 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 132 schema +-- !query schema struct<> --- !query 132 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('2017-12-11 09:30:00' AS DATE))' (timestamp and date).; line 1 pos 7 --- !query 133 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t --- !query 133 schema +-- !query schema struct<> --- !query 133 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS TINYINT))' (date and tinyint).; line 1 pos 7 --- !query 134 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t --- !query 134 schema +-- !query schema struct<> --- !query 134 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS SMALLINT))' (date and smallint).; line 1 pos 7 --- !query 135 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS INT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS INT))' (date and int).; line 1 pos 7 --- !query 136 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS BIGINT))' (date and bigint).; line 1 pos 7 --- !query 137 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS FLOAT))' (date and float).; line 1 pos 7 --- !query 138 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t --- !query 138 schema +-- !query schema struct<> --- !query 138 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS DOUBLE))' (date and double).; line 1 pos 7 --- !query 139 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t --- !query 139 schema +-- !query schema struct<> --- !query 139 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS DECIMAL(10,0)))' (date and decimal(10,0)).; line 1 pos 7 --- !query 140 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t --- !query 140 schema +-- !query schema struct<> --- !query 140 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(CAST(1 AS STRING) AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(CAST(1 AS STRING) AS DOUBLE))' (date and double).; line 1 pos 7 --- !query 141 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t --- !query 141 schema +-- !query schema struct<> --- !query 141 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('1' AS BINARY))' (date and binary).; line 1 pos 7 --- !query 142 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t --- !query 142 schema +-- !query schema struct<> --- !query 142 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST(1 AS BOOLEAN))' (date and boolean).; line 1 pos 7 --- !query 143 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 143 schema +-- !query schema struct<> --- !query 143 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (date and timestamp).; line 1 pos 7 --- !query 144 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t --- !query 144 schema +-- !query schema struct<> --- !query 144 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('2017-12-11 09:30:00' AS DATE))' requires (double or decimal) type, not date; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/elt.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/elt.sql.out index b62e1b6826045..5e335df904a3d 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/elt.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/elt.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 6 --- !query 0 +-- !query SELECT elt(2, col1, col2, col3, col4, col5) col FROM ( SELECT @@ -13,9 +13,9 @@ FROM ( CAST(id AS DOUBLE) col5 FROM range(10) ) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 0 1 2 @@ -28,7 +28,7 @@ struct 9 --- !query 1 +-- !query SELECT elt(3, col1, col2, col3, col4) col FROM ( SELECT @@ -38,9 +38,9 @@ FROM ( encode(string(id + 3), 'utf-8') col4 FROM range(10) ) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 10 11 2 @@ -53,15 +53,15 @@ struct 9 --- !query 2 +-- !query set spark.sql.function.eltOutputAsString=true --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output spark.sql.function.eltOutputAsString true --- !query 3 +-- !query SELECT elt(1, col1, col2) col FROM ( SELECT @@ -69,9 +69,9 @@ FROM ( encode(string(id + 1), 'utf-8') col2 FROM range(10) ) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 0 1 2 @@ -84,15 +84,15 @@ struct 9 --- !query 4 +-- !query set spark.sql.function.eltOutputAsString=false --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output spark.sql.function.eltOutputAsString false --- !query 5 +-- !query SELECT elt(2, col1, col2) col FROM ( SELECT @@ -100,9 +100,9 @@ FROM ( encode(string(id + 1), 'utf-8') col2 FROM range(10) ) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 10 2 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out index 7097027872707..bb49d296eaada 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out @@ -2,1231 +2,1231 @@ -- Number of queries: 145 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as tinyint)) FROM t --- !query 1 schema +-- !query schema struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS TINYINT))):tinyint> --- !query 1 output +-- !query output 1 --- !query 2 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as smallint)) FROM t --- !query 2 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS TINYINT) AS SMALLINT), CAST(2 AS SMALLINT))):smallint> --- !query 2 output +-- !query output 1 --- !query 3 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as int)) FROM t --- !query 3 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS TINYINT) AS INT), CAST(2 AS INT))):int> --- !query 3 output +-- !query output 1 --- !query 4 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as bigint)) FROM t --- !query 4 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS TINYINT) AS BIGINT), CAST(2 AS BIGINT))):bigint> --- !query 4 output +-- !query output 1 --- !query 5 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as float)) FROM t --- !query 5 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS TINYINT) AS FLOAT), CAST(2 AS FLOAT))):float> --- !query 5 output +-- !query output 1.0 --- !query 6 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as double)) FROM t --- !query 6 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS TINYINT) AS DOUBLE), CAST(2 AS DOUBLE))):double> --- !query 6 output +-- !query output 1.0 --- !query 7 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as decimal(10, 0))) FROM t --- !query 7 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS TINYINT) AS DECIMAL(10,0)), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> --- !query 7 output +-- !query output 1 --- !query 8 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as string)) FROM t --- !query 8 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS TINYINT) AS STRING), CAST(2 AS STRING))):string> --- !query 8 output +-- !query output 1 --- !query 9 +-- !query SELECT IF(true, cast(1 as tinyint), cast('2' as binary)) FROM t --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS TINYINT), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS TINYINT), CAST('2' AS BINARY)))' (tinyint and binary).; line 1 pos 7 --- !query 10 +-- !query SELECT IF(true, cast(1 as tinyint), cast(2 as boolean)) FROM t --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS TINYINT), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS TINYINT), CAST(2 AS BOOLEAN)))' (tinyint and boolean).; line 1 pos 7 --- !query 11 +-- !query SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (tinyint and timestamp).; line 1 pos 7 --- !query 12 +-- !query SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00' AS DATE)))' (tinyint and date).; line 1 pos 7 --- !query 13 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as tinyint)) FROM t --- !query 13 schema +-- !query schema struct<(IF(true, CAST(1 AS SMALLINT), CAST(CAST(2 AS TINYINT) AS SMALLINT))):smallint> --- !query 13 output +-- !query output 1 --- !query 14 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as smallint)) FROM t --- !query 14 schema +-- !query schema struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS SMALLINT))):smallint> --- !query 14 output +-- !query output 1 --- !query 15 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as int)) FROM t --- !query 15 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS SMALLINT) AS INT), CAST(2 AS INT))):int> --- !query 15 output +-- !query output 1 --- !query 16 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as bigint)) FROM t --- !query 16 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS SMALLINT) AS BIGINT), CAST(2 AS BIGINT))):bigint> --- !query 16 output +-- !query output 1 --- !query 17 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as float)) FROM t --- !query 17 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS SMALLINT) AS FLOAT), CAST(2 AS FLOAT))):float> --- !query 17 output +-- !query output 1.0 --- !query 18 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as double)) FROM t --- !query 18 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS SMALLINT) AS DOUBLE), CAST(2 AS DOUBLE))):double> --- !query 18 output +-- !query output 1.0 --- !query 19 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as decimal(10, 0))) FROM t --- !query 19 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS SMALLINT) AS DECIMAL(10,0)), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> --- !query 19 output +-- !query output 1 --- !query 20 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as string)) FROM t --- !query 20 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS SMALLINT) AS STRING), CAST(2 AS STRING))):string> --- !query 20 output +-- !query output 1 --- !query 21 +-- !query SELECT IF(true, cast(1 as smallint), cast('2' as binary)) FROM t --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS SMALLINT), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS SMALLINT), CAST('2' AS BINARY)))' (smallint and binary).; line 1 pos 7 --- !query 22 +-- !query SELECT IF(true, cast(1 as smallint), cast(2 as boolean)) FROM t --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BOOLEAN)))' (smallint and boolean).; line 1 pos 7 --- !query 23 +-- !query SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (smallint and timestamp).; line 1 pos 7 --- !query 24 +-- !query SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00' AS DATE)))' (smallint and date).; line 1 pos 7 --- !query 25 +-- !query SELECT IF(true, cast(1 as int), cast(2 as tinyint)) FROM t --- !query 25 schema +-- !query schema struct<(IF(true, CAST(1 AS INT), CAST(CAST(2 AS TINYINT) AS INT))):int> --- !query 25 output +-- !query output 1 --- !query 26 +-- !query SELECT IF(true, cast(1 as int), cast(2 as smallint)) FROM t --- !query 26 schema +-- !query schema struct<(IF(true, CAST(1 AS INT), CAST(CAST(2 AS SMALLINT) AS INT))):int> --- !query 26 output +-- !query output 1 --- !query 27 +-- !query SELECT IF(true, cast(1 as int), cast(2 as int)) FROM t --- !query 27 schema +-- !query schema struct<(IF(true, CAST(1 AS INT), CAST(2 AS INT))):int> --- !query 27 output +-- !query output 1 --- !query 28 +-- !query SELECT IF(true, cast(1 as int), cast(2 as bigint)) FROM t --- !query 28 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS INT) AS BIGINT), CAST(2 AS BIGINT))):bigint> --- !query 28 output +-- !query output 1 --- !query 29 +-- !query SELECT IF(true, cast(1 as int), cast(2 as float)) FROM t --- !query 29 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS INT) AS FLOAT), CAST(2 AS FLOAT))):float> --- !query 29 output +-- !query output 1.0 --- !query 30 +-- !query SELECT IF(true, cast(1 as int), cast(2 as double)) FROM t --- !query 30 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS INT) AS DOUBLE), CAST(2 AS DOUBLE))):double> --- !query 30 output +-- !query output 1.0 --- !query 31 +-- !query SELECT IF(true, cast(1 as int), cast(2 as decimal(10, 0))) FROM t --- !query 31 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS INT) AS DECIMAL(10,0)), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> --- !query 31 output +-- !query output 1 --- !query 32 +-- !query SELECT IF(true, cast(1 as int), cast(2 as string)) FROM t --- !query 32 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS INT) AS STRING), CAST(2 AS STRING))):string> --- !query 32 output +-- !query output 1 --- !query 33 +-- !query SELECT IF(true, cast(1 as int), cast('2' as binary)) FROM t --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS INT), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS INT), CAST('2' AS BINARY)))' (int and binary).; line 1 pos 7 --- !query 34 +-- !query SELECT IF(true, cast(1 as int), cast(2 as boolean)) FROM t --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS INT), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS INT), CAST(2 AS BOOLEAN)))' (int and boolean).; line 1 pos 7 --- !query 35 +-- !query SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS INT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS INT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (int and timestamp).; line 1 pos 7 --- !query 36 +-- !query SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS INT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS INT), CAST('2017-12-11 09:30:00' AS DATE)))' (int and date).; line 1 pos 7 --- !query 37 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as tinyint)) FROM t --- !query 37 schema +-- !query schema struct<(IF(true, CAST(1 AS BIGINT), CAST(CAST(2 AS TINYINT) AS BIGINT))):bigint> --- !query 37 output +-- !query output 1 --- !query 38 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as smallint)) FROM t --- !query 38 schema +-- !query schema struct<(IF(true, CAST(1 AS BIGINT), CAST(CAST(2 AS SMALLINT) AS BIGINT))):bigint> --- !query 38 output +-- !query output 1 --- !query 39 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as int)) FROM t --- !query 39 schema +-- !query schema struct<(IF(true, CAST(1 AS BIGINT), CAST(CAST(2 AS INT) AS BIGINT))):bigint> --- !query 39 output +-- !query output 1 --- !query 40 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as bigint)) FROM t --- !query 40 schema +-- !query schema struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS BIGINT))):bigint> --- !query 40 output +-- !query output 1 --- !query 41 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as float)) FROM t --- !query 41 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS BIGINT) AS FLOAT), CAST(2 AS FLOAT))):float> --- !query 41 output +-- !query output 1.0 --- !query 42 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as double)) FROM t --- !query 42 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS BIGINT) AS DOUBLE), CAST(2 AS DOUBLE))):double> --- !query 42 output +-- !query output 1.0 --- !query 43 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as decimal(10, 0))) FROM t --- !query 43 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)), CAST(CAST(2 AS DECIMAL(10,0)) AS DECIMAL(20,0)))):decimal(20,0)> --- !query 43 output +-- !query output 1 --- !query 44 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as string)) FROM t --- !query 44 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS BIGINT) AS STRING), CAST(2 AS STRING))):string> --- !query 44 output +-- !query output 1 --- !query 45 +-- !query SELECT IF(true, cast(1 as bigint), cast('2' as binary)) FROM t --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BIGINT), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BIGINT), CAST('2' AS BINARY)))' (bigint and binary).; line 1 pos 7 --- !query 46 +-- !query SELECT IF(true, cast(1 as bigint), cast(2 as boolean)) FROM t --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BIGINT), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BIGINT), CAST(2 AS BOOLEAN)))' (bigint and boolean).; line 1 pos 7 --- !query 47 +-- !query SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (bigint and timestamp).; line 1 pos 7 --- !query 48 +-- !query SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00' AS DATE)))' (bigint and date).; line 1 pos 7 --- !query 49 +-- !query SELECT IF(true, cast(1 as float), cast(2 as tinyint)) FROM t --- !query 49 schema +-- !query schema struct<(IF(true, CAST(1 AS FLOAT), CAST(CAST(2 AS TINYINT) AS FLOAT))):float> --- !query 49 output +-- !query output 1.0 --- !query 50 +-- !query SELECT IF(true, cast(1 as float), cast(2 as smallint)) FROM t --- !query 50 schema +-- !query schema struct<(IF(true, CAST(1 AS FLOAT), CAST(CAST(2 AS SMALLINT) AS FLOAT))):float> --- !query 50 output +-- !query output 1.0 --- !query 51 +-- !query SELECT IF(true, cast(1 as float), cast(2 as int)) FROM t --- !query 51 schema +-- !query schema struct<(IF(true, CAST(1 AS FLOAT), CAST(CAST(2 AS INT) AS FLOAT))):float> --- !query 51 output +-- !query output 1.0 --- !query 52 +-- !query SELECT IF(true, cast(1 as float), cast(2 as bigint)) FROM t --- !query 52 schema +-- !query schema struct<(IF(true, CAST(1 AS FLOAT), CAST(CAST(2 AS BIGINT) AS FLOAT))):float> --- !query 52 output +-- !query output 1.0 --- !query 53 +-- !query SELECT IF(true, cast(1 as float), cast(2 as float)) FROM t --- !query 53 schema +-- !query schema struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS FLOAT))):float> --- !query 53 output +-- !query output 1.0 --- !query 54 +-- !query SELECT IF(true, cast(1 as float), cast(2 as double)) FROM t --- !query 54 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS FLOAT) AS DOUBLE), CAST(2 AS DOUBLE))):double> --- !query 54 output +-- !query output 1.0 --- !query 55 +-- !query SELECT IF(true, cast(1 as float), cast(2 as decimal(10, 0))) FROM t --- !query 55 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS FLOAT) AS DOUBLE), CAST(CAST(2 AS DECIMAL(10,0)) AS DOUBLE))):double> --- !query 55 output +-- !query output 1.0 --- !query 56 +-- !query SELECT IF(true, cast(1 as float), cast(2 as string)) FROM t --- !query 56 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS FLOAT) AS STRING), CAST(2 AS STRING))):string> --- !query 56 output +-- !query output 1.0 --- !query 57 +-- !query SELECT IF(true, cast(1 as float), cast('2' as binary)) FROM t --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS FLOAT), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS FLOAT), CAST('2' AS BINARY)))' (float and binary).; line 1 pos 7 --- !query 58 +-- !query SELECT IF(true, cast(1 as float), cast(2 as boolean)) FROM t --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS FLOAT), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS FLOAT), CAST(2 AS BOOLEAN)))' (float and boolean).; line 1 pos 7 --- !query 59 +-- !query SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (float and timestamp).; line 1 pos 7 --- !query 60 +-- !query SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00' AS DATE)))' (float and date).; line 1 pos 7 --- !query 61 +-- !query SELECT IF(true, cast(1 as double), cast(2 as tinyint)) FROM t --- !query 61 schema +-- !query schema struct<(IF(true, CAST(1 AS DOUBLE), CAST(CAST(2 AS TINYINT) AS DOUBLE))):double> --- !query 61 output +-- !query output 1.0 --- !query 62 +-- !query SELECT IF(true, cast(1 as double), cast(2 as smallint)) FROM t --- !query 62 schema +-- !query schema struct<(IF(true, CAST(1 AS DOUBLE), CAST(CAST(2 AS SMALLINT) AS DOUBLE))):double> --- !query 62 output +-- !query output 1.0 --- !query 63 +-- !query SELECT IF(true, cast(1 as double), cast(2 as int)) FROM t --- !query 63 schema +-- !query schema struct<(IF(true, CAST(1 AS DOUBLE), CAST(CAST(2 AS INT) AS DOUBLE))):double> --- !query 63 output +-- !query output 1.0 --- !query 64 +-- !query SELECT IF(true, cast(1 as double), cast(2 as bigint)) FROM t --- !query 64 schema +-- !query schema struct<(IF(true, CAST(1 AS DOUBLE), CAST(CAST(2 AS BIGINT) AS DOUBLE))):double> --- !query 64 output +-- !query output 1.0 --- !query 65 +-- !query SELECT IF(true, cast(1 as double), cast(2 as float)) FROM t --- !query 65 schema +-- !query schema struct<(IF(true, CAST(1 AS DOUBLE), CAST(CAST(2 AS FLOAT) AS DOUBLE))):double> --- !query 65 output +-- !query output 1.0 --- !query 66 +-- !query SELECT IF(true, cast(1 as double), cast(2 as double)) FROM t --- !query 66 schema +-- !query schema struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS DOUBLE))):double> --- !query 66 output +-- !query output 1.0 --- !query 67 +-- !query SELECT IF(true, cast(1 as double), cast(2 as decimal(10, 0))) FROM t --- !query 67 schema +-- !query schema struct<(IF(true, CAST(1 AS DOUBLE), CAST(CAST(2 AS DECIMAL(10,0)) AS DOUBLE))):double> --- !query 67 output +-- !query output 1.0 --- !query 68 +-- !query SELECT IF(true, cast(1 as double), cast(2 as string)) FROM t --- !query 68 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS DOUBLE) AS STRING), CAST(2 AS STRING))):string> --- !query 68 output +-- !query output 1.0 --- !query 69 +-- !query SELECT IF(true, cast(1 as double), cast('2' as binary)) FROM t --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS DOUBLE), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS DOUBLE), CAST('2' AS BINARY)))' (double and binary).; line 1 pos 7 --- !query 70 +-- !query SELECT IF(true, cast(1 as double), cast(2 as boolean)) FROM t --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BOOLEAN)))' (double and boolean).; line 1 pos 7 --- !query 71 +-- !query SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (double and timestamp).; line 1 pos 7 --- !query 72 +-- !query SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00' AS DATE)))' (double and date).; line 1 pos 7 --- !query 73 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as tinyint)) FROM t --- !query 73 schema +-- !query schema struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(CAST(2 AS TINYINT) AS DECIMAL(10,0)))):decimal(10,0)> --- !query 73 output +-- !query output 1 --- !query 74 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as smallint)) FROM t --- !query 74 schema +-- !query schema struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(CAST(2 AS SMALLINT) AS DECIMAL(10,0)))):decimal(10,0)> --- !query 74 output +-- !query output 1 --- !query 75 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as int)) FROM t --- !query 75 schema +-- !query schema struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(CAST(2 AS INT) AS DECIMAL(10,0)))):decimal(10,0)> --- !query 75 output +-- !query output 1 --- !query 76 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as bigint)) FROM t --- !query 76 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)), CAST(CAST(2 AS BIGINT) AS DECIMAL(20,0)))):decimal(20,0)> --- !query 76 output +-- !query output 1 --- !query 77 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as float)) FROM t --- !query 77 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE), CAST(CAST(2 AS FLOAT) AS DOUBLE))):double> --- !query 77 output +-- !query output 1.0 --- !query 78 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as double)) FROM t --- !query 78 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE), CAST(2 AS DOUBLE))):double> --- !query 78 output +-- !query output 1.0 --- !query 79 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as decimal(10, 0))) FROM t --- !query 79 schema +-- !query schema struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> --- !query 79 output +-- !query output 1 --- !query 80 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as string)) FROM t --- !query 80 schema +-- !query schema struct<(IF(true, CAST(CAST(1 AS DECIMAL(10,0)) AS STRING), CAST(2 AS STRING))):string> --- !query 80 output +-- !query output 1 --- !query 81 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast('2' as binary)) FROM t --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2' AS BINARY)))' (decimal(10,0) and binary).; line 1 pos 7 --- !query 82 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as boolean)) FROM t --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BOOLEAN)))' (decimal(10,0) and boolean).; line 1 pos 7 --- !query 83 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (decimal(10,0) and timestamp).; line 1 pos 7 --- !query 84 +-- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE)))' (decimal(10,0) and date).; line 1 pos 7 --- !query 85 +-- !query SELECT IF(true, cast(1 as string), cast(2 as tinyint)) FROM t --- !query 85 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2 AS TINYINT) AS STRING))):string> --- !query 85 output +-- !query output 1 --- !query 86 +-- !query SELECT IF(true, cast(1 as string), cast(2 as smallint)) FROM t --- !query 86 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2 AS SMALLINT) AS STRING))):string> --- !query 86 output +-- !query output 1 --- !query 87 +-- !query SELECT IF(true, cast(1 as string), cast(2 as int)) FROM t --- !query 87 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2 AS INT) AS STRING))):string> --- !query 87 output +-- !query output 1 --- !query 88 +-- !query SELECT IF(true, cast(1 as string), cast(2 as bigint)) FROM t --- !query 88 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2 AS BIGINT) AS STRING))):string> --- !query 88 output +-- !query output 1 --- !query 89 +-- !query SELECT IF(true, cast(1 as string), cast(2 as float)) FROM t --- !query 89 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2 AS FLOAT) AS STRING))):string> --- !query 89 output +-- !query output 1 --- !query 90 +-- !query SELECT IF(true, cast(1 as string), cast(2 as double)) FROM t --- !query 90 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2 AS DOUBLE) AS STRING))):string> --- !query 90 output +-- !query output 1 --- !query 91 +-- !query SELECT IF(true, cast(1 as string), cast(2 as decimal(10, 0))) FROM t --- !query 91 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2 AS DECIMAL(10,0)) AS STRING))):string> --- !query 91 output +-- !query output 1 --- !query 92 +-- !query SELECT IF(true, cast(1 as string), cast(2 as string)) FROM t --- !query 92 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(2 AS STRING))):string> --- !query 92 output +-- !query output 1 --- !query 93 +-- !query SELECT IF(true, cast(1 as string), cast('2' as binary)) FROM t --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS STRING), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS STRING), CAST('2' AS BINARY)))' (string and binary).; line 1 pos 7 --- !query 94 +-- !query SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN)))' (string and boolean).; line 1 pos 7 --- !query 95 +-- !query SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 95 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING))):string> --- !query 95 output +-- !query output 1 --- !query 96 +-- !query SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 96 schema +-- !query schema struct<(IF(true, CAST(1 AS STRING), CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING))):string> --- !query 96 output +-- !query output 1 --- !query 97 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as tinyint)) FROM t --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS TINYINT)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS TINYINT)))' (binary and tinyint).; line 1 pos 7 --- !query 98 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as smallint)) FROM t --- !query 98 schema +-- !query schema struct<> --- !query 98 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS SMALLINT)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS SMALLINT)))' (binary and smallint).; line 1 pos 7 --- !query 99 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as int)) FROM t --- !query 99 schema +-- !query schema struct<> --- !query 99 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS INT)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS INT)))' (binary and int).; line 1 pos 7 --- !query 100 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as bigint)) FROM t --- !query 100 schema +-- !query schema struct<> --- !query 100 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS BIGINT)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS BIGINT)))' (binary and bigint).; line 1 pos 7 --- !query 101 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as float)) FROM t --- !query 101 schema +-- !query schema struct<> --- !query 101 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS FLOAT)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS FLOAT)))' (binary and float).; line 1 pos 7 --- !query 102 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as double)) FROM t --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS DOUBLE)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS DOUBLE)))' (binary and double).; line 1 pos 7 --- !query 103 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as decimal(10, 0))) FROM t --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS DECIMAL(10,0))))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS DECIMAL(10,0))))' (binary and decimal(10,0)).; line 1 pos 7 --- !query 104 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as string)) FROM t --- !query 104 schema +-- !query schema struct<> --- !query 104 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS STRING)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS STRING)))' (binary and string).; line 1 pos 7 --- !query 105 +-- !query SELECT IF(true, cast('1' as binary), cast('2' as binary)) FROM t --- !query 105 schema +-- !query schema struct<(IF(true, CAST(1 AS BINARY), CAST(2 AS BINARY))):binary> --- !query 105 output +-- !query output 1 --- !query 106 +-- !query SELECT IF(true, cast('1' as binary), cast(2 as boolean)) FROM t --- !query 106 schema +-- !query schema struct<> --- !query 106 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST(2 AS BOOLEAN)))' (binary and boolean).; line 1 pos 7 --- !query 107 +-- !query SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (binary and timestamp).; line 1 pos 7 --- !query 108 +-- !query SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('1' AS BINARY), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST('1' AS BINARY), CAST('2017-12-11 09:30:00' AS DATE)))' (binary and date).; line 1 pos 7 --- !query 109 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as tinyint)) FROM t --- !query 109 schema +-- !query schema struct<> --- !query 109 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS TINYINT)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS TINYINT)))' (boolean and tinyint).; line 1 pos 7 --- !query 110 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as smallint)) FROM t --- !query 110 schema +-- !query schema struct<> --- !query 110 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS SMALLINT)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS SMALLINT)))' (boolean and smallint).; line 1 pos 7 --- !query 111 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as int)) FROM t --- !query 111 schema +-- !query schema struct<> --- !query 111 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS INT)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS INT)))' (boolean and int).; line 1 pos 7 --- !query 112 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as bigint)) FROM t --- !query 112 schema +-- !query schema struct<> --- !query 112 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BIGINT)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BIGINT)))' (boolean and bigint).; line 1 pos 7 --- !query 113 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as float)) FROM t --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS FLOAT)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS FLOAT)))' (boolean and float).; line 1 pos 7 --- !query 114 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as double)) FROM t --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DOUBLE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DOUBLE)))' (boolean and double).; line 1 pos 7 --- !query 115 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as decimal(10, 0))) FROM t --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DECIMAL(10,0))))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DECIMAL(10,0))))' (boolean and decimal(10,0)).; line 1 pos 7 --- !query 116 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as string)) FROM t --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING)))' (boolean and string).; line 1 pos 7 --- !query 117 +-- !query SELECT IF(true, cast(1 as boolean), cast('2' as binary)) FROM t --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST('2' AS BINARY)))' (boolean and binary).; line 1 pos 7 --- !query 118 +-- !query SELECT IF(true, cast(1 as boolean), cast(2 as boolean)) FROM t --- !query 118 schema +-- !query schema struct<(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BOOLEAN))):boolean> --- !query 118 output +-- !query output true --- !query 119 +-- !query SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 119 schema +-- !query schema struct<> --- !query 119 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' (boolean and timestamp).; line 1 pos 7 --- !query 120 +-- !query SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST(1 AS BOOLEAN), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: differing types in '(IF(true, CAST(1 AS BOOLEAN), CAST('2017-12-11 09:30:00' AS DATE)))' (boolean and date).; line 1 pos 7 --- !query 121 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as tinyint)) FROM t --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS TINYINT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS TINYINT)))' (timestamp and tinyint).; line 1 pos 7 --- !query 122 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as smallint)) FROM t --- !query 122 schema +-- !query schema struct<> --- !query 122 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS SMALLINT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS SMALLINT)))' (timestamp and smallint).; line 1 pos 7 --- !query 123 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as int)) FROM t --- !query 123 schema +-- !query schema struct<> --- !query 123 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS INT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS INT)))' (timestamp and int).; line 1 pos 7 --- !query 124 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as bigint)) FROM t --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS BIGINT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS BIGINT)))' (timestamp and bigint).; line 1 pos 7 --- !query 125 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as float)) FROM t --- !query 125 schema +-- !query schema struct<> --- !query 125 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS FLOAT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS FLOAT)))' (timestamp and float).; line 1 pos 7 --- !query 126 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as double)) FROM t --- !query 126 schema +-- !query schema struct<> --- !query 126 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS DOUBLE)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS DOUBLE)))' (timestamp and double).; line 1 pos 7 --- !query 127 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as decimal(10, 0))) FROM t --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS DECIMAL(10,0))))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS DECIMAL(10,0))))' (timestamp and decimal(10,0)).; line 1 pos 7 --- !query 128 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as string)) FROM t --- !query 128 schema +-- !query schema struct<(IF(true, CAST(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) AS STRING), CAST(2 AS STRING))):string> --- !query 128 output +-- !query output 2017-12-12 09:30:00 --- !query 129 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2' as binary)) FROM t --- !query 129 schema +-- !query schema struct<> --- !query 129 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST('2' AS BINARY)))' (timestamp and binary).; line 1 pos 7 --- !query 130 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as boolean)) FROM t --- !query 130 schema +-- !query schema struct<> --- !query 130 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(2 AS BOOLEAN)))' (timestamp and boolean).; line 1 pos 7 --- !query 131 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 131 schema +-- !query schema struct<(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):timestamp> --- !query 131 output +-- !query output 2017-12-12 09:30:00 --- !query 132 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 132 schema +-- !query schema struct<(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(CAST(2017-12-11 09:30:00 AS DATE) AS TIMESTAMP))):timestamp> --- !query 132 output +-- !query output 2017-12-12 09:30:00 --- !query 133 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as tinyint)) FROM t --- !query 133 schema +-- !query schema struct<> --- !query 133 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS TINYINT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS TINYINT)))' (date and tinyint).; line 1 pos 7 --- !query 134 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as smallint)) FROM t --- !query 134 schema +-- !query schema struct<> --- !query 134 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS SMALLINT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS SMALLINT)))' (date and smallint).; line 1 pos 7 --- !query 135 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as int)) FROM t --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS INT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS INT)))' (date and int).; line 1 pos 7 --- !query 136 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as bigint)) FROM t --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS BIGINT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS BIGINT)))' (date and bigint).; line 1 pos 7 --- !query 137 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as float)) FROM t --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS FLOAT)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS FLOAT)))' (date and float).; line 1 pos 7 --- !query 138 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as double)) FROM t --- !query 138 schema +-- !query schema struct<> --- !query 138 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS DOUBLE)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS DOUBLE)))' (date and double).; line 1 pos 7 --- !query 139 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as decimal(10, 0))) FROM t --- !query 139 schema +-- !query schema struct<> --- !query 139 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS DECIMAL(10,0))))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS DECIMAL(10,0))))' (date and decimal(10,0)).; line 1 pos 7 --- !query 140 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as string)) FROM t --- !query 140 schema +-- !query schema struct<(IF(true, CAST(CAST(2017-12-12 09:30:00 AS DATE) AS STRING), CAST(2 AS STRING))):string> --- !query 140 output +-- !query output 2017-12-12 --- !query 141 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2' as binary)) FROM t --- !query 141 schema +-- !query schema struct<> --- !query 141 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST('2' AS BINARY)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST('2' AS BINARY)))' (date and binary).; line 1 pos 7 --- !query 142 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as boolean)) FROM t --- !query 142 schema +-- !query schema struct<> --- !query 142 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS BOOLEAN)))' due to data type mismatch: differing types in '(IF(true, CAST('2017-12-12 09:30:00' AS DATE), CAST(2 AS BOOLEAN)))' (date and boolean).; line 1 pos 7 --- !query 143 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 143 schema +-- !query schema struct<(IF(true, CAST(CAST(2017-12-12 09:30:00 AS DATE) AS TIMESTAMP), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):timestamp> --- !query 143 output +-- !query output 2017-12-12 00:00:00 --- !query 144 +-- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 144 schema +-- !query schema struct<(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2017-12-11 09:30:00 AS DATE))):date> --- !query 144 output +-- !query output 2017-12-12 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out index 44fa48e2697b3..f841adf89612e 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out @@ -2,353 +2,353 @@ -- Number of queries: 44 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT 1 + '2' FROM t --- !query 1 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(2 AS DOUBLE)):double> --- !query 1 output +-- !query output 3.0 --- !query 2 +-- !query SELECT 1 - '2' FROM t --- !query 2 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(2 AS DOUBLE)):double> --- !query 2 output +-- !query output -1.0 --- !query 3 +-- !query SELECT 1 * '2' FROM t --- !query 3 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(2 AS DOUBLE)):double> --- !query 3 output +-- !query output 2.0 --- !query 4 +-- !query SELECT 4 / '2' FROM t --- !query 4 schema +-- !query schema struct<(CAST(4 AS DOUBLE) / CAST(CAST(2 AS DOUBLE) AS DOUBLE)):double> --- !query 4 output +-- !query output 2.0 --- !query 5 +-- !query SELECT 1.1 + '2' FROM t --- !query 5 schema +-- !query schema struct<(CAST(1.1 AS DOUBLE) + CAST(2 AS DOUBLE)):double> --- !query 5 output +-- !query output 3.1 --- !query 6 +-- !query SELECT 1.1 - '2' FROM t --- !query 6 schema +-- !query schema struct<(CAST(1.1 AS DOUBLE) - CAST(2 AS DOUBLE)):double> --- !query 6 output +-- !query output -0.8999999999999999 --- !query 7 +-- !query SELECT 1.1 * '2' FROM t --- !query 7 schema +-- !query schema struct<(CAST(1.1 AS DOUBLE) * CAST(2 AS DOUBLE)):double> --- !query 7 output +-- !query output 2.2 --- !query 8 +-- !query SELECT 4.4 / '2' FROM t --- !query 8 schema +-- !query schema struct<(CAST(4.4 AS DOUBLE) / CAST(2 AS DOUBLE)):double> --- !query 8 output +-- !query output 2.2 --- !query 9 +-- !query SELECT 1.1 + '2.2' FROM t --- !query 9 schema +-- !query schema struct<(CAST(1.1 AS DOUBLE) + CAST(2.2 AS DOUBLE)):double> --- !query 9 output +-- !query output 3.3000000000000003 --- !query 10 +-- !query SELECT 1.1 - '2.2' FROM t --- !query 10 schema +-- !query schema struct<(CAST(1.1 AS DOUBLE) - CAST(2.2 AS DOUBLE)):double> --- !query 10 output +-- !query output -1.1 --- !query 11 +-- !query SELECT 1.1 * '2.2' FROM t --- !query 11 schema +-- !query schema struct<(CAST(1.1 AS DOUBLE) * CAST(2.2 AS DOUBLE)):double> --- !query 11 output +-- !query output 2.4200000000000004 --- !query 12 +-- !query SELECT 4.4 / '2.2' FROM t --- !query 12 schema +-- !query schema struct<(CAST(4.4 AS DOUBLE) / CAST(2.2 AS DOUBLE)):double> --- !query 12 output +-- !query output 2.0 --- !query 13 +-- !query SELECT '$' || cast(1 as smallint) || '$' FROM t --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output $1$ --- !query 14 +-- !query SELECT '$' || 1 || '$' FROM t --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output $1$ --- !query 15 +-- !query SELECT '$' || cast(1 as bigint) || '$' FROM t --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output $1$ --- !query 16 +-- !query SELECT '$' || cast(1.1 as float) || '$' FROM t --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output $1.1$ --- !query 17 +-- !query SELECT '$' || cast(1.1 as double) || '$' FROM t --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output $1.1$ --- !query 18 +-- !query SELECT '$' || 1.1 || '$' FROM t --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output $1.1$ --- !query 19 +-- !query SELECT '$' || cast(1.1 as decimal(8,3)) || '$' FROM t --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output $1.100$ --- !query 20 +-- !query SELECT '$' || 'abcd' || '$' FROM t --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output $abcd$ --- !query 21 +-- !query SELECT '$' || date('1996-09-09') || '$' FROM t --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output $1996-09-09$ --- !query 22 +-- !query SELECT '$' || timestamp('1996-09-09 10:11:12.4' )|| '$' FROM t --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output $1996-09-09 10:11:12.4$ --- !query 23 +-- !query SELECT length(cast(1 as smallint)) FROM t --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 1 --- !query 24 +-- !query SELECT length(cast(1 as int)) FROM t --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1 --- !query 25 +-- !query SELECT length(cast(1 as bigint)) FROM t --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1 --- !query 26 +-- !query SELECT length(cast(1.1 as float)) FROM t --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 3 --- !query 27 +-- !query SELECT length(cast(1.1 as double)) FROM t --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 3 --- !query 28 +-- !query SELECT length(1.1) FROM t --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 3 --- !query 29 +-- !query SELECT length(cast(1.1 as decimal(8,3))) FROM t --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 5 --- !query 30 +-- !query SELECT length('four') FROM t --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 4 --- !query 31 +-- !query SELECT length(date('1996-09-10')) FROM t --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 10 --- !query 32 +-- !query SELECT length(timestamp('1996-09-10 10:11:12.4')) FROM t --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 21 --- !query 33 +-- !query SELECT year( '1996-01-10') FROM t --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 1996 --- !query 34 +-- !query SELECT month( '1996-01-10') FROM t --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 1 --- !query 35 +-- !query SELECT day( '1996-01-10') FROM t --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 10 --- !query 36 +-- !query SELECT hour( '10:11:12') FROM t --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 10 --- !query 37 +-- !query SELECT minute( '10:11:12') FROM t --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 11 --- !query 38 +-- !query SELECT second( '10:11:12') FROM t --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 12 --- !query 39 +-- !query select 1 like '%' FROM t --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output true --- !query 40 +-- !query select date('1996-09-10') like '19%' FROM t --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output true --- !query 41 +-- !query select '1' like 1 FROM t --- !query 41 schema +-- !query schema struct<1 LIKE CAST(1 AS STRING):boolean> --- !query 41 output +-- !query output true --- !query 42 +-- !query select '1 ' like 1 FROM t --- !query 42 schema +-- !query schema struct<1 LIKE CAST(1 AS STRING):boolean> --- !query 42 output +-- !query output false --- !query 43 +-- !query select '1996-09-10' like date('1996-09-10') FROM t --- !query 43 schema +-- !query schema struct<1996-09-10 LIKE CAST(CAST(1996-09-10 AS DATE) AS STRING):boolean> --- !query 43 output +-- !query output true diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out index 875ccc1341ec4..21d0a0e0fef4e 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out @@ -2,2453 +2,2453 @@ -- Number of queries: 289 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint)) FROM t --- !query 1 schema +-- !query schema struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT))):boolean> --- !query 1 output +-- !query output true --- !query 2 +-- !query SELECT cast(1 as tinyint) in (cast(1 as smallint)) FROM t --- !query 2 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS SMALLINT) IN (CAST(CAST(1 AS SMALLINT) AS SMALLINT))):boolean> --- !query 2 output +-- !query output true --- !query 3 +-- !query SELECT cast(1 as tinyint) in (cast(1 as int)) FROM t --- !query 3 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS INT) IN (CAST(CAST(1 AS INT) AS INT))):boolean> --- !query 3 output +-- !query output true --- !query 4 +-- !query SELECT cast(1 as tinyint) in (cast(1 as bigint)) FROM t --- !query 4 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS BIGINT) IN (CAST(CAST(1 AS BIGINT) AS BIGINT))):boolean> --- !query 4 output +-- !query output true --- !query 5 +-- !query SELECT cast(1 as tinyint) in (cast(1 as float)) FROM t --- !query 5 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS FLOAT))):boolean> --- !query 5 output +-- !query output true --- !query 6 +-- !query SELECT cast(1 as tinyint) in (cast(1 as double)) FROM t --- !query 6 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 6 output +-- !query output true --- !query 7 +-- !query SELECT cast(1 as tinyint) in (cast(1 as decimal(10, 0))) FROM t --- !query 7 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 7 output +-- !query output true --- !query 8 +-- !query SELECT cast(1 as tinyint) in (cast(1 as string)) FROM t --- !query 8 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 8 output +-- !query output true --- !query 9 +-- !query SELECT cast(1 as tinyint) in (cast('1' as binary)) FROM t --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: tinyint != binary; line 1 pos 26 --- !query 10 +-- !query SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: tinyint != boolean; line 1 pos 26 --- !query 11 +-- !query SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: tinyint != timestamp; line 1 pos 26 --- !query 12 +-- !query SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: tinyint != date; line 1 pos 26 --- !query 13 +-- !query SELECT cast(1 as smallint) in (cast(1 as tinyint)) FROM t --- !query 13 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS SMALLINT) IN (CAST(CAST(1 AS TINYINT) AS SMALLINT))):boolean> --- !query 13 output +-- !query output true --- !query 14 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint)) FROM t --- !query 14 schema +-- !query schema struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT))):boolean> --- !query 14 output +-- !query output true --- !query 15 +-- !query SELECT cast(1 as smallint) in (cast(1 as int)) FROM t --- !query 15 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS INT) IN (CAST(CAST(1 AS INT) AS INT))):boolean> --- !query 15 output +-- !query output true --- !query 16 +-- !query SELECT cast(1 as smallint) in (cast(1 as bigint)) FROM t --- !query 16 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS BIGINT) IN (CAST(CAST(1 AS BIGINT) AS BIGINT))):boolean> --- !query 16 output +-- !query output true --- !query 17 +-- !query SELECT cast(1 as smallint) in (cast(1 as float)) FROM t --- !query 17 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS FLOAT))):boolean> --- !query 17 output +-- !query output true --- !query 18 +-- !query SELECT cast(1 as smallint) in (cast(1 as double)) FROM t --- !query 18 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 18 output +-- !query output true --- !query 19 +-- !query SELECT cast(1 as smallint) in (cast(1 as decimal(10, 0))) FROM t --- !query 19 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 19 output +-- !query output true --- !query 20 +-- !query SELECT cast(1 as smallint) in (cast(1 as string)) FROM t --- !query 20 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 20 output +-- !query output true --- !query 21 +-- !query SELECT cast(1 as smallint) in (cast('1' as binary)) FROM t --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: smallint != binary; line 1 pos 27 --- !query 22 +-- !query SELECT cast(1 as smallint) in (cast(1 as boolean)) FROM t --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: smallint != boolean; line 1 pos 27 --- !query 23 +-- !query SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: smallint != timestamp; line 1 pos 27 --- !query 24 +-- !query SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: smallint != date; line 1 pos 27 --- !query 25 +-- !query SELECT cast(1 as int) in (cast(1 as tinyint)) FROM t --- !query 25 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS INT) IN (CAST(CAST(1 AS TINYINT) AS INT))):boolean> --- !query 25 output +-- !query output true --- !query 26 +-- !query SELECT cast(1 as int) in (cast(1 as smallint)) FROM t --- !query 26 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS INT) IN (CAST(CAST(1 AS SMALLINT) AS INT))):boolean> --- !query 26 output +-- !query output true --- !query 27 +-- !query SELECT cast(1 as int) in (cast(1 as int)) FROM t --- !query 27 schema +-- !query schema struct<(CAST(1 AS INT) IN (CAST(1 AS INT))):boolean> --- !query 27 output +-- !query output true --- !query 28 +-- !query SELECT cast(1 as int) in (cast(1 as bigint)) FROM t --- !query 28 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS BIGINT) IN (CAST(CAST(1 AS BIGINT) AS BIGINT))):boolean> --- !query 28 output +-- !query output true --- !query 29 +-- !query SELECT cast(1 as int) in (cast(1 as float)) FROM t --- !query 29 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS FLOAT))):boolean> --- !query 29 output +-- !query output true --- !query 30 +-- !query SELECT cast(1 as int) in (cast(1 as double)) FROM t --- !query 30 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 30 output +-- !query output true --- !query 31 +-- !query SELECT cast(1 as int) in (cast(1 as decimal(10, 0))) FROM t --- !query 31 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 31 output +-- !query output true --- !query 32 +-- !query SELECT cast(1 as int) in (cast(1 as string)) FROM t --- !query 32 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 32 output +-- !query output true --- !query 33 +-- !query SELECT cast(1 as int) in (cast('1' as binary)) FROM t --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: int != binary; line 1 pos 22 --- !query 34 +-- !query SELECT cast(1 as int) in (cast(1 as boolean)) FROM t --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: int != boolean; line 1 pos 22 --- !query 35 +-- !query SELECT cast(1 as int) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: int != timestamp; line 1 pos 22 --- !query 36 +-- !query SELECT cast(1 as int) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: int != date; line 1 pos 22 --- !query 37 +-- !query SELECT cast(1 as bigint) in (cast(1 as tinyint)) FROM t --- !query 37 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS BIGINT) IN (CAST(CAST(1 AS TINYINT) AS BIGINT))):boolean> --- !query 37 output +-- !query output true --- !query 38 +-- !query SELECT cast(1 as bigint) in (cast(1 as smallint)) FROM t --- !query 38 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS BIGINT) IN (CAST(CAST(1 AS SMALLINT) AS BIGINT))):boolean> --- !query 38 output +-- !query output true --- !query 39 +-- !query SELECT cast(1 as bigint) in (cast(1 as int)) FROM t --- !query 39 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS BIGINT) IN (CAST(CAST(1 AS INT) AS BIGINT))):boolean> --- !query 39 output +-- !query output true --- !query 40 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint)) FROM t --- !query 40 schema +-- !query schema struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT))):boolean> --- !query 40 output +-- !query output true --- !query 41 +-- !query SELECT cast(1 as bigint) in (cast(1 as float)) FROM t --- !query 41 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS FLOAT))):boolean> --- !query 41 output +-- !query output true --- !query 42 +-- !query SELECT cast(1 as bigint) in (cast(1 as double)) FROM t --- !query 42 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 42 output +-- !query output true --- !query 43 +-- !query SELECT cast(1 as bigint) in (cast(1 as decimal(10, 0))) FROM t --- !query 43 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)))):boolean> --- !query 43 output +-- !query output true --- !query 44 +-- !query SELECT cast(1 as bigint) in (cast(1 as string)) FROM t --- !query 44 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 44 output +-- !query output true --- !query 45 +-- !query SELECT cast(1 as bigint) in (cast('1' as binary)) FROM t --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: bigint != binary; line 1 pos 25 --- !query 46 +-- !query SELECT cast(1 as bigint) in (cast(1 as boolean)) FROM t --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: bigint != boolean; line 1 pos 25 --- !query 47 +-- !query SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: bigint != timestamp; line 1 pos 25 --- !query 48 +-- !query SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: bigint != date; line 1 pos 25 --- !query 49 +-- !query SELECT cast(1 as float) in (cast(1 as tinyint)) FROM t --- !query 49 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS FLOAT) IN (CAST(CAST(1 AS TINYINT) AS FLOAT))):boolean> --- !query 49 output +-- !query output true --- !query 50 +-- !query SELECT cast(1 as float) in (cast(1 as smallint)) FROM t --- !query 50 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS FLOAT) IN (CAST(CAST(1 AS SMALLINT) AS FLOAT))):boolean> --- !query 50 output +-- !query output true --- !query 51 +-- !query SELECT cast(1 as float) in (cast(1 as int)) FROM t --- !query 51 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS FLOAT) IN (CAST(CAST(1 AS INT) AS FLOAT))):boolean> --- !query 51 output +-- !query output true --- !query 52 +-- !query SELECT cast(1 as float) in (cast(1 as bigint)) FROM t --- !query 52 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS FLOAT) IN (CAST(CAST(1 AS BIGINT) AS FLOAT))):boolean> --- !query 52 output +-- !query output true --- !query 53 +-- !query SELECT cast(1 as float) in (cast(1 as float)) FROM t --- !query 53 schema +-- !query schema struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT))):boolean> --- !query 53 output +-- !query output true --- !query 54 +-- !query SELECT cast(1 as float) in (cast(1 as double)) FROM t --- !query 54 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 54 output +-- !query output true --- !query 55 +-- !query SELECT cast(1 as float) in (cast(1 as decimal(10, 0))) FROM t --- !query 55 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 55 output +-- !query output true --- !query 56 +-- !query SELECT cast(1 as float) in (cast(1 as string)) FROM t --- !query 56 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 56 output +-- !query output false --- !query 57 +-- !query SELECT cast(1 as float) in (cast('1' as binary)) FROM t --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: float != binary; line 1 pos 24 --- !query 58 +-- !query SELECT cast(1 as float) in (cast(1 as boolean)) FROM t --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: float != boolean; line 1 pos 24 --- !query 59 +-- !query SELECT cast(1 as float) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: float != timestamp; line 1 pos 24 --- !query 60 +-- !query SELECT cast(1 as float) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: float != date; line 1 pos 24 --- !query 61 +-- !query SELECT cast(1 as double) in (cast(1 as tinyint)) FROM t --- !query 61 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS TINYINT) AS DOUBLE))):boolean> --- !query 61 output +-- !query output true --- !query 62 +-- !query SELECT cast(1 as double) in (cast(1 as smallint)) FROM t --- !query 62 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS SMALLINT) AS DOUBLE))):boolean> --- !query 62 output +-- !query output true --- !query 63 +-- !query SELECT cast(1 as double) in (cast(1 as int)) FROM t --- !query 63 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS INT) AS DOUBLE))):boolean> --- !query 63 output +-- !query output true --- !query 64 +-- !query SELECT cast(1 as double) in (cast(1 as bigint)) FROM t --- !query 64 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS BIGINT) AS DOUBLE))):boolean> --- !query 64 output +-- !query output true --- !query 65 +-- !query SELECT cast(1 as double) in (cast(1 as float)) FROM t --- !query 65 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> --- !query 65 output +-- !query output true --- !query 66 +-- !query SELECT cast(1 as double) in (cast(1 as double)) FROM t --- !query 66 schema +-- !query schema struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE))):boolean> --- !query 66 output +-- !query output true --- !query 67 +-- !query SELECT cast(1 as double) in (cast(1 as decimal(10, 0))) FROM t --- !query 67 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 67 output +-- !query output true --- !query 68 +-- !query SELECT cast(1 as double) in (cast(1 as string)) FROM t --- !query 68 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 68 output +-- !query output false --- !query 69 +-- !query SELECT cast(1 as double) in (cast('1' as binary)) FROM t --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: double != binary; line 1 pos 25 --- !query 70 +-- !query SELECT cast(1 as double) in (cast(1 as boolean)) FROM t --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: double != boolean; line 1 pos 25 --- !query 71 +-- !query SELECT cast(1 as double) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: double != timestamp; line 1 pos 25 --- !query 72 +-- !query SELECT cast(1 as double) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: double != date; line 1 pos 25 --- !query 73 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as tinyint)) FROM t --- !query 73 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS TINYINT) AS DECIMAL(10,0)))):boolean> --- !query 73 output +-- !query output true --- !query 74 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as smallint)) FROM t --- !query 74 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS SMALLINT) AS DECIMAL(10,0)))):boolean> --- !query 74 output +-- !query output true --- !query 75 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as int)) FROM t --- !query 75 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS INT) AS DECIMAL(10,0)))):boolean> --- !query 75 output +-- !query output true --- !query 76 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as bigint)) FROM t --- !query 76 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) IN (CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)))):boolean> --- !query 76 output +-- !query output true --- !query 77 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as float)) FROM t --- !query 77 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) IN (CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> --- !query 77 output +-- !query output true --- !query 78 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as double)) FROM t --- !query 78 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 78 output +-- !query output true --- !query 79 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0))) FROM t --- !query 79 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)))):boolean> --- !query 79 output +-- !query output true --- !query 80 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as string)) FROM t --- !query 80 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 80 output +-- !query output true --- !query 81 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast('1' as binary)) FROM t --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: decimal(10,0) != binary; line 1 pos 33 --- !query 82 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as boolean)) FROM t --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: decimal(10,0) != boolean; line 1 pos 33 --- !query 83 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: decimal(10,0) != timestamp; line 1 pos 33 --- !query 84 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: decimal(10,0) != date; line 1 pos 33 --- !query 85 +-- !query SELECT cast(1 as string) in (cast(1 as tinyint)) FROM t --- !query 85 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS TINYINT) AS STRING))):boolean> --- !query 85 output +-- !query output true --- !query 86 +-- !query SELECT cast(1 as string) in (cast(1 as smallint)) FROM t --- !query 86 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS SMALLINT) AS STRING))):boolean> --- !query 86 output +-- !query output true --- !query 87 +-- !query SELECT cast(1 as string) in (cast(1 as int)) FROM t --- !query 87 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS INT) AS STRING))):boolean> --- !query 87 output +-- !query output true --- !query 88 +-- !query SELECT cast(1 as string) in (cast(1 as bigint)) FROM t --- !query 88 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS BIGINT) AS STRING))):boolean> --- !query 88 output +-- !query output true --- !query 89 +-- !query SELECT cast(1 as string) in (cast(1 as float)) FROM t --- !query 89 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS FLOAT) AS STRING))):boolean> --- !query 89 output +-- !query output false --- !query 90 +-- !query SELECT cast(1 as string) in (cast(1 as double)) FROM t --- !query 90 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS DOUBLE) AS STRING))):boolean> --- !query 90 output +-- !query output false --- !query 91 +-- !query SELECT cast(1 as string) in (cast(1 as decimal(10, 0))) FROM t --- !query 91 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS STRING))):boolean> --- !query 91 output +-- !query output true --- !query 92 +-- !query SELECT cast(1 as string) in (cast(1 as string)) FROM t --- !query 92 schema +-- !query schema struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING))):boolean> --- !query 92 output +-- !query output true --- !query 93 +-- !query SELECT cast(1 as string) in (cast('1' as binary)) FROM t --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS STRING) IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: string != binary; line 1 pos 25 --- !query 94 +-- !query SELECT cast(1 as string) in (cast(1 as boolean)) FROM t --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS STRING) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: string != boolean; line 1 pos 25 --- !query 95 +-- !query SELECT cast(1 as string) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 95 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING))):boolean> --- !query 95 output +-- !query output false --- !query 96 +-- !query SELECT cast(1 as string) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 96 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING))):boolean> --- !query 96 output +-- !query output false --- !query 97 +-- !query SELECT cast('1' as binary) in (cast(1 as tinyint)) FROM t --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS TINYINT)))' due to data type mismatch: Arguments must be same type but were: binary != tinyint; line 1 pos 27 --- !query 98 +-- !query SELECT cast('1' as binary) in (cast(1 as smallint)) FROM t --- !query 98 schema +-- !query schema struct<> --- !query 98 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS SMALLINT)))' due to data type mismatch: Arguments must be same type but were: binary != smallint; line 1 pos 27 --- !query 99 +-- !query SELECT cast('1' as binary) in (cast(1 as int)) FROM t --- !query 99 schema +-- !query schema struct<> --- !query 99 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS INT)))' due to data type mismatch: Arguments must be same type but were: binary != int; line 1 pos 27 --- !query 100 +-- !query SELECT cast('1' as binary) in (cast(1 as bigint)) FROM t --- !query 100 schema +-- !query schema struct<> --- !query 100 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS BIGINT)))' due to data type mismatch: Arguments must be same type but were: binary != bigint; line 1 pos 27 --- !query 101 +-- !query SELECT cast('1' as binary) in (cast(1 as float)) FROM t --- !query 101 schema +-- !query schema struct<> --- !query 101 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS FLOAT)))' due to data type mismatch: Arguments must be same type but were: binary != float; line 1 pos 27 --- !query 102 +-- !query SELECT cast('1' as binary) in (cast(1 as double)) FROM t --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS DOUBLE)))' due to data type mismatch: Arguments must be same type but were: binary != double; line 1 pos 27 --- !query 103 +-- !query SELECT cast('1' as binary) in (cast(1 as decimal(10, 0))) FROM t --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS DECIMAL(10,0))))' due to data type mismatch: Arguments must be same type but were: binary != decimal(10,0); line 1 pos 27 --- !query 104 +-- !query SELECT cast('1' as binary) in (cast(1 as string)) FROM t --- !query 104 schema +-- !query schema struct<> --- !query 104 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS STRING)))' due to data type mismatch: Arguments must be same type but were: binary != string; line 1 pos 27 --- !query 105 +-- !query SELECT cast('1' as binary) in (cast('1' as binary)) FROM t --- !query 105 schema +-- !query schema struct<(CAST(1 AS BINARY) IN (CAST(1 AS BINARY))):boolean> --- !query 105 output +-- !query output true --- !query 106 +-- !query SELECT cast('1' as binary) in (cast(1 as boolean)) FROM t --- !query 106 schema +-- !query schema struct<> --- !query 106 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: binary != boolean; line 1 pos 27 --- !query 107 +-- !query SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: binary != timestamp; line 1 pos 27 --- !query 108 +-- !query SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: binary != date; line 1 pos 27 --- !query 109 +-- !query SELECT true in (cast(1 as tinyint)) FROM t --- !query 109 schema +-- !query schema struct<> --- !query 109 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST(1 AS TINYINT)))' due to data type mismatch: Arguments must be same type but were: boolean != tinyint; line 1 pos 12 --- !query 110 +-- !query SELECT true in (cast(1 as smallint)) FROM t --- !query 110 schema +-- !query schema struct<> --- !query 110 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST(1 AS SMALLINT)))' due to data type mismatch: Arguments must be same type but were: boolean != smallint; line 1 pos 12 --- !query 111 +-- !query SELECT true in (cast(1 as int)) FROM t --- !query 111 schema +-- !query schema struct<> --- !query 111 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST(1 AS INT)))' due to data type mismatch: Arguments must be same type but were: boolean != int; line 1 pos 12 --- !query 112 +-- !query SELECT true in (cast(1 as bigint)) FROM t --- !query 112 schema +-- !query schema struct<> --- !query 112 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST(1 AS BIGINT)))' due to data type mismatch: Arguments must be same type but were: boolean != bigint; line 1 pos 12 --- !query 113 +-- !query SELECT true in (cast(1 as float)) FROM t --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST(1 AS FLOAT)))' due to data type mismatch: Arguments must be same type but were: boolean != float; line 1 pos 12 --- !query 114 +-- !query SELECT true in (cast(1 as double)) FROM t --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST(1 AS DOUBLE)))' due to data type mismatch: Arguments must be same type but were: boolean != double; line 1 pos 12 --- !query 115 +-- !query SELECT true in (cast(1 as decimal(10, 0))) FROM t --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST(1 AS DECIMAL(10,0))))' due to data type mismatch: Arguments must be same type but were: boolean != decimal(10,0); line 1 pos 12 --- !query 116 +-- !query SELECT true in (cast(1 as string)) FROM t --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST(1 AS STRING)))' due to data type mismatch: Arguments must be same type but were: boolean != string; line 1 pos 12 --- !query 117 +-- !query SELECT true in (cast('1' as binary)) FROM t --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: boolean != binary; line 1 pos 12 --- !query 118 +-- !query SELECT true in (cast(1 as boolean)) FROM t --- !query 118 schema +-- !query schema struct<(true IN (CAST(1 AS BOOLEAN))):boolean> --- !query 118 output +-- !query output true --- !query 119 +-- !query SELECT true in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 119 schema +-- !query schema struct<> --- !query 119 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: boolean != timestamp; line 1 pos 12 --- !query 120 +-- !query SELECT true in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(true IN (CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: boolean != date; line 1 pos 12 --- !query 121 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as tinyint)) FROM t --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS TINYINT)))' due to data type mismatch: Arguments must be same type but were: timestamp != tinyint; line 1 pos 50 --- !query 122 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as smallint)) FROM t --- !query 122 schema +-- !query schema struct<> --- !query 122 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS SMALLINT)))' due to data type mismatch: Arguments must be same type but were: timestamp != smallint; line 1 pos 50 --- !query 123 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as int)) FROM t --- !query 123 schema +-- !query schema struct<> --- !query 123 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS INT)))' due to data type mismatch: Arguments must be same type but were: timestamp != int; line 1 pos 50 --- !query 124 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as bigint)) FROM t --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS BIGINT)))' due to data type mismatch: Arguments must be same type but were: timestamp != bigint; line 1 pos 50 --- !query 125 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as float)) FROM t --- !query 125 schema +-- !query schema struct<> --- !query 125 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS FLOAT)))' due to data type mismatch: Arguments must be same type but were: timestamp != float; line 1 pos 50 --- !query 126 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as double)) FROM t --- !query 126 schema +-- !query schema struct<> --- !query 126 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS DOUBLE)))' due to data type mismatch: Arguments must be same type but were: timestamp != double; line 1 pos 50 --- !query 127 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as decimal(10, 0))) FROM t --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS DECIMAL(10,0))))' due to data type mismatch: Arguments must be same type but were: timestamp != decimal(10,0); line 1 pos 50 --- !query 128 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as string)) FROM t --- !query 128 schema +-- !query schema struct<(CAST(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) AS STRING) IN (CAST(CAST(2 AS STRING) AS STRING))):boolean> --- !query 128 output +-- !query output false --- !query 129 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2' as binary)) FROM t --- !query 129 schema +-- !query schema struct<> --- !query 129 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: timestamp != binary; line 1 pos 50 --- !query 130 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as boolean)) FROM t --- !query 130 schema +-- !query schema struct<> --- !query 130 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST(2 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: timestamp != boolean; line 1 pos 50 --- !query 131 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 131 schema +-- !query schema struct<(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> --- !query 131 output +-- !query output false --- !query 132 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 132 schema +-- !query schema struct<(CAST(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) AS TIMESTAMP) IN (CAST(CAST(2017-12-11 09:30:00 AS DATE) AS TIMESTAMP))):boolean> --- !query 132 output +-- !query output false --- !query 133 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as tinyint)) FROM t --- !query 133 schema +-- !query schema struct<> --- !query 133 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS TINYINT)))' due to data type mismatch: Arguments must be same type but were: date != tinyint; line 1 pos 43 --- !query 134 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as smallint)) FROM t --- !query 134 schema +-- !query schema struct<> --- !query 134 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS SMALLINT)))' due to data type mismatch: Arguments must be same type but were: date != smallint; line 1 pos 43 --- !query 135 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as int)) FROM t --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS INT)))' due to data type mismatch: Arguments must be same type but were: date != int; line 1 pos 43 --- !query 136 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as bigint)) FROM t --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS BIGINT)))' due to data type mismatch: Arguments must be same type but were: date != bigint; line 1 pos 43 --- !query 137 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as float)) FROM t --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS FLOAT)))' due to data type mismatch: Arguments must be same type but were: date != float; line 1 pos 43 --- !query 138 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as double)) FROM t --- !query 138 schema +-- !query schema struct<> --- !query 138 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS DOUBLE)))' due to data type mismatch: Arguments must be same type but were: date != double; line 1 pos 43 --- !query 139 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as decimal(10, 0))) FROM t --- !query 139 schema +-- !query schema struct<> --- !query 139 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS DECIMAL(10,0))))' due to data type mismatch: Arguments must be same type but were: date != decimal(10,0); line 1 pos 43 --- !query 140 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as string)) FROM t --- !query 140 schema +-- !query schema struct<(CAST(CAST(2017-12-12 09:30:00 AS DATE) AS STRING) IN (CAST(CAST(2 AS STRING) AS STRING))):boolean> --- !query 140 output +-- !query output false --- !query 141 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2' as binary)) FROM t --- !query 141 schema +-- !query schema struct<> --- !query 141 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: date != binary; line 1 pos 43 --- !query 142 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as boolean)) FROM t --- !query 142 schema +-- !query schema struct<> --- !query 142 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST(2 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: date != boolean; line 1 pos 43 --- !query 143 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 143 schema +-- !query schema struct<(CAST(CAST(2017-12-12 09:30:00 AS DATE) AS TIMESTAMP) IN (CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS TIMESTAMP))):boolean> --- !query 143 output +-- !query output false --- !query 144 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query 144 schema +-- !query schema struct<(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-11 09:30:00 AS DATE))):boolean> --- !query 144 output +-- !query output false --- !query 145 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as tinyint)) FROM t --- !query 145 schema +-- !query schema struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS TINYINT))):boolean> --- !query 145 output +-- !query output true --- !query 146 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as smallint)) FROM t --- !query 146 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS SMALLINT) IN (CAST(CAST(1 AS TINYINT) AS SMALLINT), CAST(CAST(1 AS SMALLINT) AS SMALLINT))):boolean> --- !query 146 output +-- !query output true --- !query 147 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as int)) FROM t --- !query 147 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS INT) IN (CAST(CAST(1 AS TINYINT) AS INT), CAST(CAST(1 AS INT) AS INT))):boolean> --- !query 147 output +-- !query output true --- !query 148 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as bigint)) FROM t --- !query 148 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS BIGINT) IN (CAST(CAST(1 AS TINYINT) AS BIGINT), CAST(CAST(1 AS BIGINT) AS BIGINT))):boolean> --- !query 148 output +-- !query output true --- !query 149 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as float)) FROM t --- !query 149 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS FLOAT) IN (CAST(CAST(1 AS TINYINT) AS FLOAT), CAST(CAST(1 AS FLOAT) AS FLOAT))):boolean> --- !query 149 output +-- !query output true --- !query 150 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as double)) FROM t --- !query 150 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) IN (CAST(CAST(1 AS TINYINT) AS DOUBLE), CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 150 output +-- !query output true --- !query 151 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t --- !query 151 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS TINYINT) AS DECIMAL(10,0)), CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 151 output +-- !query output true --- !query 152 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as string)) FROM t --- !query 152 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS STRING) IN (CAST(CAST(1 AS TINYINT) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 152 output +-- !query output true --- !query 153 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('1' as binary)) FROM t --- !query 153 schema +-- !query schema struct<> --- !query 153 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: tinyint != binary; line 1 pos 26 --- !query 154 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as boolean)) FROM t --- !query 154 schema +-- !query schema struct<> --- !query 154 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: tinyint != boolean; line 1 pos 26 --- !query 155 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 155 schema +-- !query schema struct<> --- !query 155 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: tinyint != timestamp; line 1 pos 26 --- !query 156 +-- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 156 schema +-- !query schema struct<> --- !query 156 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: tinyint != date; line 1 pos 26 --- !query 157 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as tinyint)) FROM t --- !query 157 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS SMALLINT) IN (CAST(CAST(1 AS SMALLINT) AS SMALLINT), CAST(CAST(1 AS TINYINT) AS SMALLINT))):boolean> --- !query 157 output +-- !query output true --- !query 158 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as smallint)) FROM t --- !query 158 schema +-- !query schema struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS SMALLINT))):boolean> --- !query 158 output +-- !query output true --- !query 159 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as int)) FROM t --- !query 159 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS INT) IN (CAST(CAST(1 AS SMALLINT) AS INT), CAST(CAST(1 AS INT) AS INT))):boolean> --- !query 159 output +-- !query output true --- !query 160 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as bigint)) FROM t --- !query 160 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS BIGINT) IN (CAST(CAST(1 AS SMALLINT) AS BIGINT), CAST(CAST(1 AS BIGINT) AS BIGINT))):boolean> --- !query 160 output +-- !query output true --- !query 161 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as float)) FROM t --- !query 161 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS FLOAT) IN (CAST(CAST(1 AS SMALLINT) AS FLOAT), CAST(CAST(1 AS FLOAT) AS FLOAT))):boolean> --- !query 161 output +-- !query output true --- !query 162 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as double)) FROM t --- !query 162 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) IN (CAST(CAST(1 AS SMALLINT) AS DOUBLE), CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 162 output +-- !query output true --- !query 163 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t --- !query 163 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS SMALLINT) AS DECIMAL(10,0)), CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 163 output +-- !query output true --- !query 164 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as string)) FROM t --- !query 164 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS STRING) IN (CAST(CAST(1 AS SMALLINT) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 164 output +-- !query output true --- !query 165 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast('1' as binary)) FROM t --- !query 165 schema +-- !query schema struct<> --- !query 165 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: smallint != binary; line 1 pos 27 --- !query 166 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as boolean)) FROM t --- !query 166 schema +-- !query schema struct<> --- !query 166 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: smallint != boolean; line 1 pos 27 --- !query 167 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 167 schema +-- !query schema struct<> --- !query 167 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: smallint != timestamp; line 1 pos 27 --- !query 168 +-- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 168 schema +-- !query schema struct<> --- !query 168 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: smallint != date; line 1 pos 27 --- !query 169 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as tinyint)) FROM t --- !query 169 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS INT) IN (CAST(CAST(1 AS INT) AS INT), CAST(CAST(1 AS TINYINT) AS INT))):boolean> --- !query 169 output +-- !query output true --- !query 170 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as smallint)) FROM t --- !query 170 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS INT) IN (CAST(CAST(1 AS INT) AS INT), CAST(CAST(1 AS SMALLINT) AS INT))):boolean> --- !query 170 output +-- !query output true --- !query 171 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as int)) FROM t --- !query 171 schema +-- !query schema struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS INT))):boolean> --- !query 171 output +-- !query output true --- !query 172 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as bigint)) FROM t --- !query 172 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS BIGINT) IN (CAST(CAST(1 AS INT) AS BIGINT), CAST(CAST(1 AS BIGINT) AS BIGINT))):boolean> --- !query 172 output +-- !query output true --- !query 173 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as float)) FROM t --- !query 173 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS FLOAT) IN (CAST(CAST(1 AS INT) AS FLOAT), CAST(CAST(1 AS FLOAT) AS FLOAT))):boolean> --- !query 173 output +-- !query output true --- !query 174 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as double)) FROM t --- !query 174 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) IN (CAST(CAST(1 AS INT) AS DOUBLE), CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 174 output +-- !query output true --- !query 175 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as decimal(10, 0))) FROM t --- !query 175 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS INT) AS DECIMAL(10,0)), CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)))):boolean> --- !query 175 output +-- !query output true --- !query 176 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as string)) FROM t --- !query 176 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS STRING) IN (CAST(CAST(1 AS INT) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 176 output +-- !query output true --- !query 177 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast('1' as binary)) FROM t --- !query 177 schema +-- !query schema struct<> --- !query 177 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) IN (CAST(1 AS INT), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: int != binary; line 1 pos 22 --- !query 178 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as boolean)) FROM t --- !query 178 schema +-- !query schema struct<> --- !query 178 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: int != boolean; line 1 pos 22 --- !query 179 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 179 schema +-- !query schema struct<> --- !query 179 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) IN (CAST(1 AS INT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: int != timestamp; line 1 pos 22 --- !query 180 +-- !query SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 180 schema +-- !query schema struct<> --- !query 180 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS INT) IN (CAST(1 AS INT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: int != date; line 1 pos 22 --- !query 181 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as tinyint)) FROM t --- !query 181 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS BIGINT) IN (CAST(CAST(1 AS BIGINT) AS BIGINT), CAST(CAST(1 AS TINYINT) AS BIGINT))):boolean> --- !query 181 output +-- !query output true --- !query 182 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as smallint)) FROM t --- !query 182 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS BIGINT) IN (CAST(CAST(1 AS BIGINT) AS BIGINT), CAST(CAST(1 AS SMALLINT) AS BIGINT))):boolean> --- !query 182 output +-- !query output true --- !query 183 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as int)) FROM t --- !query 183 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS BIGINT) IN (CAST(CAST(1 AS BIGINT) AS BIGINT), CAST(CAST(1 AS INT) AS BIGINT))):boolean> --- !query 183 output +-- !query output true --- !query 184 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as bigint)) FROM t --- !query 184 schema +-- !query schema struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BIGINT))):boolean> --- !query 184 output +-- !query output true --- !query 185 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as float)) FROM t --- !query 185 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS FLOAT) IN (CAST(CAST(1 AS BIGINT) AS FLOAT), CAST(CAST(1 AS FLOAT) AS FLOAT))):boolean> --- !query 185 output +-- !query output true --- !query 186 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as double)) FROM t --- !query 186 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) IN (CAST(CAST(1 AS BIGINT) AS DOUBLE), CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 186 output +-- !query output true --- !query 187 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t --- !query 187 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)) IN (CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)), CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)))):boolean> --- !query 187 output +-- !query output true --- !query 188 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as string)) FROM t --- !query 188 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS STRING) IN (CAST(CAST(1 AS BIGINT) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 188 output +-- !query output true --- !query 189 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast('1' as binary)) FROM t --- !query 189 schema +-- !query schema struct<> --- !query 189 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: bigint != binary; line 1 pos 25 --- !query 190 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as boolean)) FROM t --- !query 190 schema +-- !query schema struct<> --- !query 190 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: bigint != boolean; line 1 pos 25 --- !query 191 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 191 schema +-- !query schema struct<> --- !query 191 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: bigint != timestamp; line 1 pos 25 --- !query 192 +-- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 192 schema +-- !query schema struct<> --- !query 192 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: bigint != date; line 1 pos 25 --- !query 193 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as tinyint)) FROM t --- !query 193 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS FLOAT), CAST(CAST(1 AS TINYINT) AS FLOAT))):boolean> --- !query 193 output +-- !query output true --- !query 194 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as smallint)) FROM t --- !query 194 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS FLOAT), CAST(CAST(1 AS SMALLINT) AS FLOAT))):boolean> --- !query 194 output +-- !query output true --- !query 195 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as int)) FROM t --- !query 195 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS FLOAT), CAST(CAST(1 AS INT) AS FLOAT))):boolean> --- !query 195 output +-- !query output true --- !query 196 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as bigint)) FROM t --- !query 196 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS FLOAT) IN (CAST(CAST(1 AS FLOAT) AS FLOAT), CAST(CAST(1 AS BIGINT) AS FLOAT))):boolean> --- !query 196 output +-- !query output true --- !query 197 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as float)) FROM t --- !query 197 schema +-- !query schema struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS FLOAT))):boolean> --- !query 197 output +-- !query output true --- !query 198 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as double)) FROM t --- !query 198 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) IN (CAST(CAST(1 AS FLOAT) AS DOUBLE), CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 198 output +-- !query output true --- !query 199 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as decimal(10, 0))) FROM t --- !query 199 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) IN (CAST(CAST(1 AS FLOAT) AS DOUBLE), CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 199 output +-- !query output true --- !query 200 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as string)) FROM t --- !query 200 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS STRING) IN (CAST(CAST(1 AS FLOAT) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 200 output +-- !query output true --- !query 201 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast('1' as binary)) FROM t --- !query 201 schema +-- !query schema struct<> --- !query 201 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: float != binary; line 1 pos 24 --- !query 202 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as boolean)) FROM t --- !query 202 schema +-- !query schema struct<> --- !query 202 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: float != boolean; line 1 pos 24 --- !query 203 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 203 schema +-- !query schema struct<> --- !query 203 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: float != timestamp; line 1 pos 24 --- !query 204 +-- !query SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 204 schema +-- !query schema struct<> --- !query 204 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: float != date; line 1 pos 24 --- !query 205 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as tinyint)) FROM t --- !query 205 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE), CAST(CAST(1 AS TINYINT) AS DOUBLE))):boolean> --- !query 205 output +-- !query output true --- !query 206 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as smallint)) FROM t --- !query 206 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE), CAST(CAST(1 AS SMALLINT) AS DOUBLE))):boolean> --- !query 206 output +-- !query output true --- !query 207 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as int)) FROM t --- !query 207 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE), CAST(CAST(1 AS INT) AS DOUBLE))):boolean> --- !query 207 output +-- !query output true --- !query 208 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as bigint)) FROM t --- !query 208 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE), CAST(CAST(1 AS BIGINT) AS DOUBLE))):boolean> --- !query 208 output +-- !query output true --- !query 209 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as float)) FROM t --- !query 209 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE), CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> --- !query 209 output +-- !query output true --- !query 210 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as double)) FROM t --- !query 210 schema +-- !query schema struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS DOUBLE))):boolean> --- !query 210 output +-- !query output true --- !query 211 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as decimal(10, 0))) FROM t --- !query 211 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS DOUBLE) IN (CAST(CAST(1 AS DOUBLE) AS DOUBLE), CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 211 output +-- !query output true --- !query 212 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as string)) FROM t --- !query 212 schema +-- !query schema struct<(CAST(CAST(1 AS DOUBLE) AS STRING) IN (CAST(CAST(1 AS DOUBLE) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 212 output +-- !query output true --- !query 213 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast('1' as binary)) FROM t --- !query 213 schema +-- !query schema struct<> --- !query 213 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: double != binary; line 1 pos 25 --- !query 214 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as boolean)) FROM t --- !query 214 schema +-- !query schema struct<> --- !query 214 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: double != boolean; line 1 pos 25 --- !query 215 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 215 schema +-- !query schema struct<> --- !query 215 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: double != timestamp; line 1 pos 25 --- !query 216 +-- !query SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 216 schema +-- !query schema struct<> --- !query 216 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: double != date; line 1 pos 25 --- !query 217 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t --- !query 217 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)), CAST(CAST(1 AS TINYINT) AS DECIMAL(10,0)))):boolean> --- !query 217 output +-- !query output true --- !query 218 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t --- !query 218 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)), CAST(CAST(1 AS SMALLINT) AS DECIMAL(10,0)))):boolean> --- !query 218 output +-- !query output true --- !query 219 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as int)) FROM t --- !query 219 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(10,0)), CAST(CAST(1 AS INT) AS DECIMAL(10,0)))):boolean> --- !query 219 output +-- !query output true --- !query 220 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t --- !query 220 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DECIMAL(20,0)), CAST(CAST(1 AS BIGINT) AS DECIMAL(20,0)))):boolean> --- !query 220 output +-- !query output true --- !query 221 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as float)) FROM t --- !query 221 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE), CAST(CAST(1 AS FLOAT) AS DOUBLE))):boolean> --- !query 221 output +-- !query output true --- !query 222 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as double)) FROM t --- !query 222 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE), CAST(CAST(1 AS DOUBLE) AS DOUBLE))):boolean> --- !query 222 output +-- !query output true --- !query 223 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t --- !query 223 schema +-- !query schema struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS DECIMAL(10,0)))):boolean> --- !query 223 output +-- !query output true --- !query 224 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as string)) FROM t --- !query 224 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS STRING) IN (CAST(CAST(1 AS DECIMAL(10,0)) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 224 output +-- !query output true --- !query 225 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t --- !query 225 schema +-- !query schema struct<> --- !query 225 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: decimal(10,0) != binary; line 1 pos 33 --- !query 226 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t --- !query 226 schema +-- !query schema struct<> --- !query 226 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: decimal(10,0) != boolean; line 1 pos 33 --- !query 227 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 227 schema +-- !query schema struct<> --- !query 227 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: decimal(10,0) != timestamp; line 1 pos 33 --- !query 228 +-- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 228 schema +-- !query schema struct<> --- !query 228 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: decimal(10,0) != date; line 1 pos 33 --- !query 229 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as tinyint)) FROM t --- !query 229 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(1 AS TINYINT) AS STRING))):boolean> --- !query 229 output +-- !query output true --- !query 230 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as smallint)) FROM t --- !query 230 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(1 AS SMALLINT) AS STRING))):boolean> --- !query 230 output +-- !query output true --- !query 231 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as int)) FROM t --- !query 231 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(1 AS INT) AS STRING))):boolean> --- !query 231 output +-- !query output true --- !query 232 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as bigint)) FROM t --- !query 232 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(1 AS BIGINT) AS STRING))):boolean> --- !query 232 output +-- !query output true --- !query 233 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as float)) FROM t --- !query 233 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(1 AS FLOAT) AS STRING))):boolean> --- !query 233 output +-- !query output true --- !query 234 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as double)) FROM t --- !query 234 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(1 AS DOUBLE) AS STRING))):boolean> --- !query 234 output +-- !query output true --- !query 235 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as decimal(10, 0))) FROM t --- !query 235 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(1 AS DECIMAL(10,0)) AS STRING))):boolean> --- !query 235 output +-- !query output true --- !query 236 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as string)) FROM t --- !query 236 schema +-- !query schema struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS STRING))):boolean> --- !query 236 output +-- !query output true --- !query 237 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast('1' as binary)) FROM t --- !query 237 schema +-- !query schema struct<> --- !query 237 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: string != binary; line 1 pos 25 --- !query 238 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t --- !query 238 schema +-- !query schema struct<> --- !query 238 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: string != boolean; line 1 pos 25 --- !query 239 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 239 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING))):boolean> --- !query 239 output +-- !query output true --- !query 240 +-- !query SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 240 schema +-- !query schema struct<(CAST(CAST(1 AS STRING) AS STRING) IN (CAST(CAST(1 AS STRING) AS STRING), CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING))):boolean> --- !query 240 output +-- !query output true --- !query 241 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as tinyint)) FROM t --- !query 241 schema +-- !query schema struct<> --- !query 241 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS TINYINT)))' due to data type mismatch: Arguments must be same type but were: binary != tinyint; line 1 pos 27 --- !query 242 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as smallint)) FROM t --- !query 242 schema +-- !query schema struct<> --- !query 242 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS SMALLINT)))' due to data type mismatch: Arguments must be same type but were: binary != smallint; line 1 pos 27 --- !query 243 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as int)) FROM t --- !query 243 schema +-- !query schema struct<> --- !query 243 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS INT)))' due to data type mismatch: Arguments must be same type but were: binary != int; line 1 pos 27 --- !query 244 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as bigint)) FROM t --- !query 244 schema +-- !query schema struct<> --- !query 244 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS BIGINT)))' due to data type mismatch: Arguments must be same type but were: binary != bigint; line 1 pos 27 --- !query 245 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as float)) FROM t --- !query 245 schema +-- !query schema struct<> --- !query 245 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS FLOAT)))' due to data type mismatch: Arguments must be same type but were: binary != float; line 1 pos 27 --- !query 246 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as double)) FROM t --- !query 246 schema +-- !query schema struct<> --- !query 246 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS DOUBLE)))' due to data type mismatch: Arguments must be same type but were: binary != double; line 1 pos 27 --- !query 247 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as decimal(10, 0))) FROM t --- !query 247 schema +-- !query schema struct<> --- !query 247 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS DECIMAL(10,0))))' due to data type mismatch: Arguments must be same type but were: binary != decimal(10,0); line 1 pos 27 --- !query 248 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as string)) FROM t --- !query 248 schema +-- !query schema struct<> --- !query 248 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS STRING)))' due to data type mismatch: Arguments must be same type but were: binary != string; line 1 pos 27 --- !query 249 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast('1' as binary)) FROM t --- !query 249 schema +-- !query schema struct<(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS BINARY))):boolean> --- !query 249 output +-- !query output true --- !query 250 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as boolean)) FROM t --- !query 250 schema +-- !query schema struct<> --- !query 250 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: binary != boolean; line 1 pos 27 --- !query 251 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 251 schema +-- !query schema struct<> --- !query 251 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: binary != timestamp; line 1 pos 27 --- !query 252 +-- !query SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 252 schema +-- !query schema struct<> --- !query 252 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) IN (CAST('1' AS BINARY), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: binary != date; line 1 pos 27 --- !query 253 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as tinyint)) FROM t --- !query 253 schema +-- !query schema struct<> --- !query 253 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS TINYINT)))' due to data type mismatch: Arguments must be same type but were: boolean != tinyint; line 1 pos 28 --- !query 254 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as smallint)) FROM t --- !query 254 schema +-- !query schema struct<> --- !query 254 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS SMALLINT)))' due to data type mismatch: Arguments must be same type but were: boolean != smallint; line 1 pos 28 --- !query 255 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as int)) FROM t --- !query 255 schema +-- !query schema struct<> --- !query 255 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS INT)))' due to data type mismatch: Arguments must be same type but were: boolean != int; line 1 pos 28 --- !query 256 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as bigint)) FROM t --- !query 256 schema +-- !query schema struct<> --- !query 256 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS BIGINT)))' due to data type mismatch: Arguments must be same type but were: boolean != bigint; line 1 pos 28 --- !query 257 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as float)) FROM t --- !query 257 schema +-- !query schema struct<> --- !query 257 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS FLOAT)))' due to data type mismatch: Arguments must be same type but were: boolean != float; line 1 pos 28 --- !query 258 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as double)) FROM t --- !query 258 schema +-- !query schema struct<> --- !query 258 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS DOUBLE)))' due to data type mismatch: Arguments must be same type but were: boolean != double; line 1 pos 28 --- !query 259 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as decimal(10, 0))) FROM t --- !query 259 schema +-- !query schema struct<> --- !query 259 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS DECIMAL(10,0))))' due to data type mismatch: Arguments must be same type but were: boolean != decimal(10,0); line 1 pos 28 --- !query 260 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as string)) FROM t --- !query 260 schema +-- !query schema struct<> --- !query 260 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST(1 AS STRING)))' due to data type mismatch: Arguments must be same type but were: boolean != string; line 1 pos 28 --- !query 261 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast('1' as binary)) FROM t --- !query 261 schema +-- !query schema struct<> --- !query 261 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: boolean != binary; line 1 pos 28 --- !query 262 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as boolean)) FROM t --- !query 262 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS BOOLEAN))):boolean> --- !query 262 output +-- !query output true --- !query 263 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 263 schema +-- !query schema struct<> --- !query 263 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP)))' due to data type mismatch: Arguments must be same type but were: boolean != timestamp; line 1 pos 28 --- !query 264 +-- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 264 schema +-- !query schema struct<> --- !query 264 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BOOLEAN) IN (CAST('1' AS BOOLEAN), CAST('2017-12-11 09:30:00' AS DATE)))' due to data type mismatch: Arguments must be same type but were: boolean != date; line 1 pos 28 --- !query 265 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as tinyint)) FROM t --- !query 265 schema +-- !query schema struct<> --- !query 265 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(1 AS TINYINT)))' due to data type mismatch: Arguments must be same type but were: timestamp != tinyint; line 1 pos 50 --- !query 266 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as smallint)) FROM t --- !query 266 schema +-- !query schema struct<> --- !query 266 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(1 AS SMALLINT)))' due to data type mismatch: Arguments must be same type but were: timestamp != smallint; line 1 pos 50 --- !query 267 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as int)) FROM t --- !query 267 schema +-- !query schema struct<> --- !query 267 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(1 AS INT)))' due to data type mismatch: Arguments must be same type but were: timestamp != int; line 1 pos 50 --- !query 268 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as bigint)) FROM t --- !query 268 schema +-- !query schema struct<> --- !query 268 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(1 AS BIGINT)))' due to data type mismatch: Arguments must be same type but were: timestamp != bigint; line 1 pos 50 --- !query 269 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as float)) FROM t --- !query 269 schema +-- !query schema struct<> --- !query 269 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(1 AS FLOAT)))' due to data type mismatch: Arguments must be same type but were: timestamp != float; line 1 pos 50 --- !query 270 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as double)) FROM t --- !query 270 schema +-- !query schema struct<> --- !query 270 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(1 AS DOUBLE)))' due to data type mismatch: Arguments must be same type but were: timestamp != double; line 1 pos 50 --- !query 271 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t --- !query 271 schema +-- !query schema struct<> --- !query 271 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(1 AS DECIMAL(10,0))))' due to data type mismatch: Arguments must be same type but were: timestamp != decimal(10,0); line 1 pos 50 --- !query 272 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as string)) FROM t --- !query 272 schema +-- !query schema struct<(CAST(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) AS STRING) IN (CAST(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 272 output +-- !query output true --- !query 273 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('1' as binary)) FROM t --- !query 273 schema +-- !query schema struct<> --- !query 273 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: timestamp != binary; line 1 pos 50 --- !query 274 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as boolean)) FROM t --- !query 274 schema +-- !query schema struct<> --- !query 274 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) IN (CAST('2017-12-12 09:30:00.0' AS TIMESTAMP), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: timestamp != boolean; line 1 pos 50 --- !query 275 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 275 schema +-- !query schema struct<(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> --- !query 275 output +-- !query output true --- !query 276 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 276 schema +-- !query schema struct<(CAST(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) AS TIMESTAMP) IN (CAST(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) AS TIMESTAMP), CAST(CAST(2017-12-11 09:30:00 AS DATE) AS TIMESTAMP))):boolean> --- !query 276 output +-- !query output true --- !query 277 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as tinyint)) FROM t --- !query 277 schema +-- !query schema struct<> --- !query 277 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS TINYINT)))' due to data type mismatch: Arguments must be same type but were: date != tinyint; line 1 pos 43 --- !query 278 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as smallint)) FROM t --- !query 278 schema +-- !query schema struct<> --- !query 278 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS SMALLINT)))' due to data type mismatch: Arguments must be same type but were: date != smallint; line 1 pos 43 --- !query 279 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as int)) FROM t --- !query 279 schema +-- !query schema struct<> --- !query 279 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS INT)))' due to data type mismatch: Arguments must be same type but were: date != int; line 1 pos 43 --- !query 280 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as bigint)) FROM t --- !query 280 schema +-- !query schema struct<> --- !query 280 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS BIGINT)))' due to data type mismatch: Arguments must be same type but were: date != bigint; line 1 pos 43 --- !query 281 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as float)) FROM t --- !query 281 schema +-- !query schema struct<> --- !query 281 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS FLOAT)))' due to data type mismatch: Arguments must be same type but were: date != float; line 1 pos 43 --- !query 282 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as double)) FROM t --- !query 282 schema +-- !query schema struct<> --- !query 282 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS DOUBLE)))' due to data type mismatch: Arguments must be same type but were: date != double; line 1 pos 43 --- !query 283 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t --- !query 283 schema +-- !query schema struct<> --- !query 283 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS DECIMAL(10,0))))' due to data type mismatch: Arguments must be same type but were: date != decimal(10,0); line 1 pos 43 --- !query 284 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as string)) FROM t --- !query 284 schema +-- !query schema struct<(CAST(CAST(2017-12-12 09:30:00 AS DATE) AS STRING) IN (CAST(CAST(2017-12-12 09:30:00 AS DATE) AS STRING), CAST(CAST(1 AS STRING) AS STRING))):boolean> --- !query 284 output +-- !query output true --- !query 285 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('1' as binary)) FROM t --- !query 285 schema +-- !query schema struct<> --- !query 285 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST('1' AS BINARY)))' due to data type mismatch: Arguments must be same type but were: date != binary; line 1 pos 43 --- !query 286 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as boolean)) FROM t --- !query 286 schema +-- !query schema struct<> --- !query 286 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-12 09:30:00' AS DATE) IN (CAST('2017-12-12 09:30:00' AS DATE), CAST(1 AS BOOLEAN)))' due to data type mismatch: Arguments must be same type but were: date != boolean; line 1 pos 43 --- !query 287 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 287 schema +-- !query schema struct<(CAST(CAST(2017-12-12 09:30:00 AS DATE) AS TIMESTAMP) IN (CAST(CAST(2017-12-12 09:30:00 AS DATE) AS TIMESTAMP), CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS TIMESTAMP))):boolean> --- !query 287 output +-- !query output true --- !query 288 +-- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00' as date)) FROM t --- !query 288 schema +-- !query schema struct<(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(2017-12-11 09:30:00 AS DATE))):boolean> --- !query 288 output +-- !query output true diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out index 2fdaf63cd3bf1..ed7ab5a342c12 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 16 --- !query 0 +-- !query CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( map(true, false), map(2Y, 1Y), @@ -36,144 +36,144 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( array_map1, array_map2, struct_map1, struct_map2 ) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT map_zip_with(tinyint_map, smallint_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 1 schema +-- !query schema struct>> --- !query 1 output +-- !query output {2:{"k":2,"v1":1,"v2":1}} --- !query 2 +-- !query SELECT map_zip_with(smallint_map, int_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 2 schema +-- !query schema struct>> --- !query 2 output +-- !query output {2:{"k":2,"v1":1,"v2":1}} --- !query 3 +-- !query SELECT map_zip_with(int_map, bigint_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 3 schema +-- !query schema struct>> --- !query 3 output +-- !query output {2:{"k":2,"v1":1,"v2":1}} --- !query 4 +-- !query SELECT map_zip_with(double_map, float_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 4 schema +-- !query schema struct>> --- !query 4 output +-- !query output {2.0:{"k":2.0,"v1":1.0,"v2":1.0}} --- !query 5 +-- !query SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7 --- !query 6 +-- !query SELECT map_zip_with(decimal_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 6 schema +-- !query schema struct>> --- !query 6 output +-- !query output {2:{"k":2,"v1":null,"v2":1},922337203685477897945456575809789456:{"k":922337203685477897945456575809789456,"v1":922337203685477897945456575809789456,"v2":null}} --- !query 7 +-- !query SELECT map_zip_with(decimal_map1, double_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 7 schema +-- !query schema struct>> --- !query 7 output +-- !query output {2.0:{"k":2.0,"v1":null,"v2":1.0},9.223372036854779E35:{"k":9.223372036854779E35,"v1":922337203685477897945456575809789456,"v2":null}} --- !query 8 +-- !query SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7 --- !query 9 +-- !query SELECT map_zip_with(decimal_map2, double_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 9 schema +-- !query schema struct>> --- !query 9 output +-- !query output {2.0:{"k":2.0,"v1":null,"v2":1.0},9.223372036854778:{"k":9.223372036854778,"v1":9.22337203685477897945456575809789456,"v2":null}} --- !query 10 +-- !query SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 10 schema +-- !query schema struct>> --- !query 10 output +-- !query output {"2":{"k":"2","v1":"1","v2":1},"true":{"k":"true","v1":"false","v2":null}} --- !query 11 +-- !query SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 11 schema +-- !query schema struct>> --- !query 11 output +-- !query output {"2016-03-14":{"k":"2016-03-14","v1":"2016-03-13","v2":2016-03-13}} --- !query 12 +-- !query SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 12 schema +-- !query schema struct>> --- !query 12 output +-- !query output {"2016-11-15 20:54:00":{"k":"2016-11-15 20:54:00","v1":2016-11-12 20:54:00,"v2":null},"2016-11-15 20:54:00.000":{"k":"2016-11-15 20:54:00.000","v1":null,"v2":"2016-11-12 20:54:00.000"}} --- !query 13 +-- !query SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 13 schema +-- !query schema struct>> --- !query 13 output +-- !query output {"922337203685477897945456575809789456":{"k":"922337203685477897945456575809789456","v1":922337203685477897945456575809789456,"v2":"text"}} --- !query 14 +-- !query SELECT map_zip_with(array_map1, array_map2, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 14 schema +-- !query schema struct,struct,v1:array,v2:array>>> --- !query 14 output +-- !query output {[1,2]:{"k":[1,2],"v1":[1,2],"v2":[1,2]}} --- !query 15 +-- !query SELECT map_zip_with(struct_map1, struct_map2, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps --- !query 15 schema +-- !query schema struct,struct,v1:struct,v2:struct>>> --- !query 15 output +-- !query output {{"col1":1,"col2":2}:{"k":{"col1":1,"col2":2},"v1":{"col1":1,"col2":2},"v2":{"col1":1,"col2":2}}} diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out index ee6d62b483884..fcf1afc72efe9 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 8 --- !query 0 +-- !query CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( map(true, false), map(false, true), map(1Y, 2Y), map(3Y, 4Y), @@ -38,13 +38,13 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( string_int_map1, string_int_map2, int_string_map1, int_string_map2 ) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT map_concat(boolean_map1, boolean_map2) boolean_map, map_concat(tinyint_map1, tinyint_map2) tinyint_map, @@ -62,13 +62,13 @@ SELECT map_concat(string_int_map1, string_int_map2) string_int_map, map_concat(int_string_map1, int_string_map2) int_string_map FROM various_maps --- !query 1 schema +-- !query schema struct,tinyint_map:map,smallint_map:map,int_map:map,bigint_map:map,decimal_map:map,float_map:map,double_map:map,date_map:map,timestamp_map:map,string_map:map,array_map:map,array>,struct_map:map,struct>,string_int_map:map,int_string_map:map> --- !query 1 output +-- !query output {false:true,true:false} {1:2,3:4} {1:2,3:4} {4:6,7:8} {6:7,8:9} {9223372036854775808:9223372036854775809,9223372036854775809:9223372036854775808} {1.0:2.0,3.0:4.0} {1.0:2.0,3.0:4.0} {2016-03-12:2016-03-11,2016-03-14:2016-03-13} {2016-11-11 20:54:00:2016-11-09 20:54:00,2016-11-15 20:54:00:2016-11-12 20:54:00} {"a":"b","c":"d"} {["a","b"]:["c","d"],["e"]:["f"]} {{"col1":"a","col2":1}:{"col1":"b","col2":2},{"col1":"c","col2":3}:{"col1":"d","col2":4}} {"a":1,"c":2} {1:"a",2:"c"} --- !query 2 +-- !query SELECT map_concat(tinyint_map1, smallint_map2) ts_map, map_concat(smallint_map1, int_map2) si_map, @@ -80,62 +80,62 @@ SELECT map_concat(string_map1, int_map2) sti_map, map_concat(int_string_map1, tinyint_map2) istt_map FROM various_maps --- !query 2 schema +-- !query schema struct,si_map:map,ib_map:map,bd_map:map,df_map:map,std_map:map,tst_map:map,sti_map:map,istt_map:map> --- !query 2 output +-- !query output {1:2,3:4} {1:2,7:8} {4:6,8:9} {6:7,9223372036854775808:9223372036854775809} {3.0:4.0,9.223372036854776E18:9.223372036854776E18} {"2016-03-12":"2016-03-11","a":"b"} {"2016-11-15 20:54:00":"2016-11-12 20:54:00","c":"d"} {"7":"8","a":"b"} {1:"a",3:"4"} --- !query 3 +-- !query SELECT map_concat(tinyint_map1, array_map1) tm_map FROM various_maps --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'map_concat(various_maps.`tinyint_map1`, various_maps.`array_map1`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map, map,array>]; line 2 pos 4 --- !query 4 +-- !query SELECT map_concat(boolean_map1, int_map2) bi_map FROM various_maps --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'map_concat(various_maps.`boolean_map1`, various_maps.`int_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map, map]; line 2 pos 4 --- !query 5 +-- !query SELECT map_concat(int_map1, struct_map2) is_map FROM various_maps --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'map_concat(various_maps.`int_map1`, various_maps.`struct_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map, map,struct>]; line 2 pos 4 --- !query 6 +-- !query SELECT map_concat(struct_map1, array_map2) ma_map FROM various_maps --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'map_concat(various_maps.`struct_map1`, various_maps.`array_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map,struct>, map,array>]; line 2 pos 4 --- !query 7 +-- !query SELECT map_concat(int_map1, array_map2) ms_map FROM various_maps --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'map_concat(various_maps.`int_map1`, various_maps.`array_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map, map,array>]; line 2 pos 4 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out index af0e348ae8ef1..31353bdedc69f 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out @@ -2,2576 +2,2576 @@ -- Number of queries: 316 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT '1' + cast(1 as tinyint) FROM t --- !query 1 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 1 output +-- !query output 2.0 --- !query 2 +-- !query SELECT '1' + cast(1 as smallint) FROM t --- !query 2 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 2 output +-- !query output 2.0 --- !query 3 +-- !query SELECT '1' + cast(1 as int) FROM t --- !query 3 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 3 output +-- !query output 2.0 --- !query 4 +-- !query SELECT '1' + cast(1 as bigint) FROM t --- !query 4 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 4 output +-- !query output 2.0 --- !query 5 +-- !query SELECT '1' + cast(1 as float) FROM t --- !query 5 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 5 output +-- !query output 2.0 --- !query 6 +-- !query SELECT '1' + cast(1 as double) FROM t --- !query 6 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 6 output +-- !query output 2.0 --- !query 7 +-- !query SELECT '1' + cast(1 as decimal(10, 0)) FROM t --- !query 7 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 7 output +-- !query output 2.0 --- !query 8 +-- !query SELECT '1' + '1' FROM t --- !query 8 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 8 output +-- !query output 2.0 --- !query 9 +-- !query SELECT '1' + cast('1' as binary) FROM t --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) + CAST('1' AS BINARY))' (double and binary).; line 1 pos 7 --- !query 10 +-- !query SELECT '1' + cast(1 as boolean) FROM t --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) + CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7 --- !query 11 +-- !query SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) + CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 --- !query 12 +-- !query SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 --- !query 13 +-- !query SELECT '1' - cast(1 as tinyint) FROM t --- !query 13 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 13 output +-- !query output 0.0 --- !query 14 +-- !query SELECT '1' - cast(1 as smallint) FROM t --- !query 14 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 14 output +-- !query output 0.0 --- !query 15 +-- !query SELECT '1' - cast(1 as int) FROM t --- !query 15 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 15 output +-- !query output 0.0 --- !query 16 +-- !query SELECT '1' - cast(1 as bigint) FROM t --- !query 16 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 16 output +-- !query output 0.0 --- !query 17 +-- !query SELECT '1' - cast(1 as float) FROM t --- !query 17 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 17 output +-- !query output 0.0 --- !query 18 +-- !query SELECT '1' - cast(1 as double) FROM t --- !query 18 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 18 output +-- !query output 0.0 --- !query 19 +-- !query SELECT '1' - cast(1 as decimal(10, 0)) FROM t --- !query 19 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 19 output +-- !query output 0.0 --- !query 20 +-- !query SELECT '1' - '1' FROM t --- !query 20 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 20 output +-- !query output 0.0 --- !query 21 +-- !query SELECT '1' - cast('1' as binary) FROM t --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) - CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) - CAST('1' AS BINARY))' (double and binary).; line 1 pos 7 --- !query 22 +-- !query SELECT '1' - cast(1 as boolean) FROM t --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) - CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) - CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7 --- !query 23 +-- !query SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps('1', CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: argument 1 requires timestamp type, however, ''1'' is of string type.; line 1 pos 7 --- !query 24 +-- !query SELECT '1' - cast('2017-12-11 09:30:00' as date) FROM t --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output NULL --- !query 25 +-- !query SELECT '1' * cast(1 as tinyint) FROM t --- !query 25 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 25 output +-- !query output 1.0 --- !query 26 +-- !query SELECT '1' * cast(1 as smallint) FROM t --- !query 26 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 26 output +-- !query output 1.0 --- !query 27 +-- !query SELECT '1' * cast(1 as int) FROM t --- !query 27 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 27 output +-- !query output 1.0 --- !query 28 +-- !query SELECT '1' * cast(1 as bigint) FROM t --- !query 28 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 28 output +-- !query output 1.0 --- !query 29 +-- !query SELECT '1' * cast(1 as float) FROM t --- !query 29 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 29 output +-- !query output 1.0 --- !query 30 +-- !query SELECT '1' * cast(1 as double) FROM t --- !query 30 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 30 output +-- !query output 1.0 --- !query 31 +-- !query SELECT '1' * cast(1 as decimal(10, 0)) FROM t --- !query 31 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 31 output +-- !query output 1.0 --- !query 32 +-- !query SELECT '1' * '1' FROM t --- !query 32 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 32 output +-- !query output 1.0 --- !query 33 +-- !query SELECT '1' * cast('1' as binary) FROM t --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) * CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) * CAST('1' AS BINARY))' (double and binary).; line 1 pos 7 --- !query 34 +-- !query SELECT '1' * cast(1 as boolean) FROM t --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) * CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) * CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7 --- !query 35 +-- !query SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) * CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) * CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 --- !query 36 +-- !query SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) * CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) * CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 --- !query 37 +-- !query SELECT '1' / cast(1 as tinyint) FROM t --- !query 37 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 37 output +-- !query output 1.0 --- !query 38 +-- !query SELECT '1' / cast(1 as smallint) FROM t --- !query 38 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 38 output +-- !query output 1.0 --- !query 39 +-- !query SELECT '1' / cast(1 as int) FROM t --- !query 39 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 39 output +-- !query output 1.0 --- !query 40 +-- !query SELECT '1' / cast(1 as bigint) FROM t --- !query 40 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 40 output +-- !query output 1.0 --- !query 41 +-- !query SELECT '1' / cast(1 as float) FROM t --- !query 41 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 41 output +-- !query output 1.0 --- !query 42 +-- !query SELECT '1' / cast(1 as double) FROM t --- !query 42 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 42 output +-- !query output 1.0 --- !query 43 +-- !query SELECT '1' / cast(1 as decimal(10, 0)) FROM t --- !query 43 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 43 output +-- !query output 1.0 --- !query 44 +-- !query SELECT '1' / '1' FROM t --- !query 44 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 44 output +-- !query output 1.0 --- !query 45 +-- !query SELECT '1' / cast('1' as binary) FROM t --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) / CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) / CAST('1' AS BINARY))' (double and binary).; line 1 pos 7 --- !query 46 +-- !query SELECT '1' / cast(1 as boolean) FROM t --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) / CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) / CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7 --- !query 47 +-- !query SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) / CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 --- !query 48 +-- !query SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) / CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 --- !query 49 +-- !query SELECT '1' % cast(1 as tinyint) FROM t --- !query 49 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS TINYINT) AS DOUBLE)):double> --- !query 49 output +-- !query output 0.0 --- !query 50 +-- !query SELECT '1' % cast(1 as smallint) FROM t --- !query 50 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS SMALLINT) AS DOUBLE)):double> --- !query 50 output +-- !query output 0.0 --- !query 51 +-- !query SELECT '1' % cast(1 as int) FROM t --- !query 51 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS INT) AS DOUBLE)):double> --- !query 51 output +-- !query output 0.0 --- !query 52 +-- !query SELECT '1' % cast(1 as bigint) FROM t --- !query 52 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS BIGINT) AS DOUBLE)):double> --- !query 52 output +-- !query output 0.0 --- !query 53 +-- !query SELECT '1' % cast(1 as float) FROM t --- !query 53 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS FLOAT) AS DOUBLE)):double> --- !query 53 output +-- !query output 0.0 --- !query 54 +-- !query SELECT '1' % cast(1 as double) FROM t --- !query 54 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 54 output +-- !query output 0.0 --- !query 55 +-- !query SELECT '1' % cast(1 as decimal(10, 0)) FROM t --- !query 55 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):double> --- !query 55 output +-- !query output 0.0 --- !query 56 +-- !query SELECT '1' % '1' FROM t --- !query 56 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 56 output +-- !query output 0.0 --- !query 57 +-- !query SELECT '1' % cast('1' as binary) FROM t --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) % CAST('1' AS BINARY))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) % CAST('1' AS BINARY))' (double and binary).; line 1 pos 7 --- !query 58 +-- !query SELECT '1' % cast(1 as boolean) FROM t --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) % CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) % CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7 --- !query 59 +-- !query SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) % CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 --- !query 60 +-- !query SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS DOUBLE) % CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) % CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 --- !query 61 +-- !query SELECT pmod('1', cast(1 as tinyint)) FROM t --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output 0.0 --- !query 62 +-- !query SELECT pmod('1', cast(1 as smallint)) FROM t --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output 0.0 --- !query 63 +-- !query SELECT pmod('1', cast(1 as int)) FROM t --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output 0.0 --- !query 64 +-- !query SELECT pmod('1', cast(1 as bigint)) FROM t --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 0.0 --- !query 65 +-- !query SELECT pmod('1', cast(1 as float)) FROM t --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output 0.0 --- !query 66 +-- !query SELECT pmod('1', cast(1 as double)) FROM t --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output 0.0 --- !query 67 +-- !query SELECT pmod('1', cast(1 as decimal(10, 0))) FROM t --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output 0.0 --- !query 68 +-- !query SELECT pmod('1', '1') FROM t --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 0.0 --- !query 69 +-- !query SELECT pmod('1', cast('1' as binary)) FROM t --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS DOUBLE), CAST('1' AS BINARY))' due to data type mismatch: differing types in 'pmod(CAST('1' AS DOUBLE), CAST('1' AS BINARY))' (double and binary).; line 1 pos 7 --- !query 70 +-- !query SELECT pmod('1', cast(1 as boolean)) FROM t --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS DOUBLE), CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in 'pmod(CAST('1' AS DOUBLE), CAST(1 AS BOOLEAN))' (double and boolean).; line 1 pos 7 --- !query 71 +-- !query SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' due to data type mismatch: differing types in 'pmod(CAST('1' AS DOUBLE), CAST('2017-12-11 09:30:00.0' AS TIMESTAMP))' (double and timestamp).; line 1 pos 7 --- !query 72 +-- !query SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS DOUBLE), CAST('2017-12-11 09:30:00' AS DATE))' due to data type mismatch: differing types in 'pmod(CAST('1' AS DOUBLE), CAST('2017-12-11 09:30:00' AS DATE))' (double and date).; line 1 pos 7 --- !query 73 +-- !query SELECT cast(1 as tinyint) + '1' FROM t --- !query 73 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 73 output +-- !query output 2.0 --- !query 74 +-- !query SELECT cast(1 as smallint) + '1' FROM t --- !query 74 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 74 output +-- !query output 2.0 --- !query 75 +-- !query SELECT cast(1 as int) + '1' FROM t --- !query 75 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 75 output +-- !query output 2.0 --- !query 76 +-- !query SELECT cast(1 as bigint) + '1' FROM t --- !query 76 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 76 output +-- !query output 2.0 --- !query 77 +-- !query SELECT cast(1 as float) + '1' FROM t --- !query 77 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 77 output +-- !query output 2.0 --- !query 78 +-- !query SELECT cast(1 as double) + '1' FROM t --- !query 78 schema +-- !query schema struct<(CAST(1 AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 78 output +-- !query output 2.0 --- !query 79 +-- !query SELECT cast(1 as decimal(10, 0)) + '1' FROM t --- !query 79 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) + CAST(1 AS DOUBLE)):double> --- !query 79 output +-- !query output 2.0 --- !query 80 +-- !query SELECT cast('1' as binary) + '1' FROM t --- !query 80 schema +-- !query schema struct<> --- !query 80 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) + CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + CAST('1' AS DOUBLE))' (binary and double).; line 1 pos 7 --- !query 81 +-- !query SELECT cast(1 as boolean) + '1' FROM t --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) + CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + CAST('1' AS DOUBLE))' (boolean and double).; line 1 pos 7 --- !query 82 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) + CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7 --- !query 83 +-- !query SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_add(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 --- !query 84 +-- !query SELECT cast(1 as tinyint) - '1' FROM t --- !query 84 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 84 output +-- !query output 0.0 --- !query 85 +-- !query SELECT cast(1 as smallint) - '1' FROM t --- !query 85 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 85 output +-- !query output 0.0 --- !query 86 +-- !query SELECT cast(1 as int) - '1' FROM t --- !query 86 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 86 output +-- !query output 0.0 --- !query 87 +-- !query SELECT cast(1 as bigint) - '1' FROM t --- !query 87 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 87 output +-- !query output 0.0 --- !query 88 +-- !query SELECT cast(1 as float) - '1' FROM t --- !query 88 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 88 output +-- !query output 0.0 --- !query 89 +-- !query SELECT cast(1 as double) - '1' FROM t --- !query 89 schema +-- !query schema struct<(CAST(1 AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 89 output +-- !query output 0.0 --- !query 90 +-- !query SELECT cast(1 as decimal(10, 0)) - '1' FROM t --- !query 90 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) - CAST(1 AS DOUBLE)):double> --- !query 90 output +-- !query output 0.0 --- !query 91 +-- !query SELECT cast('1' as binary) - '1' FROM t --- !query 91 schema +-- !query schema struct<> --- !query 91 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) - CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - CAST('1' AS DOUBLE))' (binary and double).; line 1 pos 7 --- !query 92 +-- !query SELECT cast(1 as boolean) - '1' FROM t --- !query 92 schema +-- !query schema struct<> --- !query 92 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) - CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - CAST('1' AS DOUBLE))' (boolean and double).; line 1 pos 7 --- !query 93 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'subtracttimestamps(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), '1')' due to data type mismatch: argument 2 requires timestamp type, however, ''1'' is of string type.; line 1 pos 7 --- !query 94 +-- !query SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'date_sub(CAST('2017-12-11 09:30:00' AS DATE), '1')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, ''1'' is of string type.; line 1 pos 7 --- !query 95 +-- !query SELECT cast(1 as tinyint) * '1' FROM t --- !query 95 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 95 output +-- !query output 1.0 --- !query 96 +-- !query SELECT cast(1 as smallint) * '1' FROM t --- !query 96 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 96 output +-- !query output 1.0 --- !query 97 +-- !query SELECT cast(1 as int) * '1' FROM t --- !query 97 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 97 output +-- !query output 1.0 --- !query 98 +-- !query SELECT cast(1 as bigint) * '1' FROM t --- !query 98 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 98 output +-- !query output 1.0 --- !query 99 +-- !query SELECT cast(1 as float) * '1' FROM t --- !query 99 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 99 output +-- !query output 1.0 --- !query 100 +-- !query SELECT cast(1 as double) * '1' FROM t --- !query 100 schema +-- !query schema struct<(CAST(1 AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 100 output +-- !query output 1.0 --- !query 101 +-- !query SELECT cast(1 as decimal(10, 0)) * '1' FROM t --- !query 101 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) * CAST(1 AS DOUBLE)):double> --- !query 101 output +-- !query output 1.0 --- !query 102 +-- !query SELECT cast('1' as binary) * '1' FROM t --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) * CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) * CAST('1' AS DOUBLE))' (binary and double).; line 1 pos 7 --- !query 103 +-- !query SELECT cast(1 as boolean) * '1' FROM t --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) * CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) * CAST('1' AS DOUBLE))' (boolean and double).; line 1 pos 7 --- !query 104 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t --- !query 104 schema +-- !query schema struct<> --- !query 104 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) * CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) * CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7 --- !query 105 +-- !query SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t --- !query 105 schema +-- !query schema struct<> --- !query 105 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) * CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) * CAST('1' AS DOUBLE))' (date and double).; line 1 pos 7 --- !query 106 +-- !query SELECT cast(1 as tinyint) / '1' FROM t --- !query 106 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 106 output +-- !query output 1.0 --- !query 107 +-- !query SELECT cast(1 as smallint) / '1' FROM t --- !query 107 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 107 output +-- !query output 1.0 --- !query 108 +-- !query SELECT cast(1 as int) / '1' FROM t --- !query 108 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 108 output +-- !query output 1.0 --- !query 109 +-- !query SELECT cast(1 as bigint) / '1' FROM t --- !query 109 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 109 output +-- !query output 1.0 --- !query 110 +-- !query SELECT cast(1 as float) / '1' FROM t --- !query 110 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) / CAST(CAST(1 AS DOUBLE) AS DOUBLE)):double> --- !query 110 output +-- !query output 1.0 --- !query 111 +-- !query SELECT cast(1 as double) / '1' FROM t --- !query 111 schema +-- !query schema struct<(CAST(1 AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 111 output +-- !query output 1.0 --- !query 112 +-- !query SELECT cast(1 as decimal(10, 0)) / '1' FROM t --- !query 112 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) / CAST(1 AS DOUBLE)):double> --- !query 112 output +-- !query output 1.0 --- !query 113 +-- !query SELECT cast('1' as binary) / '1' FROM t --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) / CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) / CAST('1' AS DOUBLE))' (binary and double).; line 1 pos 7 --- !query 114 +-- !query SELECT cast(1 as boolean) / '1' FROM t --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) / CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) / CAST('1' AS DOUBLE))' (boolean and double).; line 1 pos 7 --- !query 115 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) / CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7 --- !query 116 +-- !query SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) / CAST('1' AS DOUBLE))' (date and double).; line 1 pos 7 --- !query 117 +-- !query SELECT cast(1 as tinyint) % '1' FROM t --- !query 117 schema +-- !query schema struct<(CAST(CAST(1 AS TINYINT) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 117 output +-- !query output 0.0 --- !query 118 +-- !query SELECT cast(1 as smallint) % '1' FROM t --- !query 118 schema +-- !query schema struct<(CAST(CAST(1 AS SMALLINT) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 118 output +-- !query output 0.0 --- !query 119 +-- !query SELECT cast(1 as int) % '1' FROM t --- !query 119 schema +-- !query schema struct<(CAST(CAST(1 AS INT) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 119 output +-- !query output 0.0 --- !query 120 +-- !query SELECT cast(1 as bigint) % '1' FROM t --- !query 120 schema +-- !query schema struct<(CAST(CAST(1 AS BIGINT) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 120 output +-- !query output 0.0 --- !query 121 +-- !query SELECT cast(1 as float) % '1' FROM t --- !query 121 schema +-- !query schema struct<(CAST(CAST(1 AS FLOAT) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 121 output +-- !query output 0.0 --- !query 122 +-- !query SELECT cast(1 as double) % '1' FROM t --- !query 122 schema +-- !query schema struct<(CAST(1 AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 122 output +-- !query output 0.0 --- !query 123 +-- !query SELECT cast(1 as decimal(10, 0)) % '1' FROM t --- !query 123 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) % CAST(1 AS DOUBLE)):double> --- !query 123 output +-- !query output 0.0 --- !query 124 +-- !query SELECT cast('1' as binary) % '1' FROM t --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('1' AS BINARY) % CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('1' AS BINARY) % CAST('1' AS DOUBLE))' (binary and double).; line 1 pos 7 --- !query 125 +-- !query SELECT cast(1 as boolean) % '1' FROM t --- !query 125 schema +-- !query schema struct<> --- !query 125 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST(1 AS BOOLEAN) % CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) % CAST('1' AS DOUBLE))' (boolean and double).; line 1 pos 7 --- !query 126 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t --- !query 126 schema +-- !query schema struct<> --- !query 126 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) % CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7 --- !query 127 +-- !query SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) % CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(CAST('2017-12-11 09:30:00' AS DATE) % CAST('1' AS DOUBLE))' (date and double).; line 1 pos 7 --- !query 128 +-- !query SELECT pmod(cast(1 as tinyint), '1') FROM t --- !query 128 schema +-- !query schema struct --- !query 128 output +-- !query output 0.0 --- !query 129 +-- !query SELECT pmod(cast(1 as smallint), '1') FROM t --- !query 129 schema +-- !query schema struct --- !query 129 output +-- !query output 0.0 --- !query 130 +-- !query SELECT pmod(cast(1 as int), '1') FROM t --- !query 130 schema +-- !query schema struct --- !query 130 output +-- !query output 0.0 --- !query 131 +-- !query SELECT pmod(cast(1 as bigint), '1') FROM t --- !query 131 schema +-- !query schema struct --- !query 131 output +-- !query output 0.0 --- !query 132 +-- !query SELECT pmod(cast(1 as float), '1') FROM t --- !query 132 schema +-- !query schema struct --- !query 132 output +-- !query output 0.0 --- !query 133 +-- !query SELECT pmod(cast(1 as double), '1') FROM t --- !query 133 schema +-- !query schema struct --- !query 133 output +-- !query output 0.0 --- !query 134 +-- !query SELECT pmod(cast(1 as decimal(10, 0)), '1') FROM t --- !query 134 schema +-- !query schema struct --- !query 134 output +-- !query output 0.0 --- !query 135 +-- !query SELECT pmod(cast('1' as binary), '1') FROM t --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('1' AS BINARY), CAST('1' AS DOUBLE))' due to data type mismatch: differing types in 'pmod(CAST('1' AS BINARY), CAST('1' AS DOUBLE))' (binary and double).; line 1 pos 7 --- !query 136 +-- !query SELECT pmod(cast(1 as boolean), '1') FROM t --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST(1 AS BOOLEAN), CAST('1' AS DOUBLE))' due to data type mismatch: differing types in 'pmod(CAST(1 AS BOOLEAN), CAST('1' AS DOUBLE))' (boolean and double).; line 1 pos 7 --- !query 137 +-- !query SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST('1' AS DOUBLE))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00.0' AS TIMESTAMP), CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7 --- !query 138 +-- !query SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t --- !query 138 schema +-- !query schema struct<> --- !query 138 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST('1' AS DOUBLE))' due to data type mismatch: differing types in 'pmod(CAST('2017-12-11 09:30:00' AS DATE), CAST('1' AS DOUBLE))' (date and double).; line 1 pos 7 --- !query 139 +-- !query SELECT '1' = cast(1 as tinyint) FROM t --- !query 139 schema +-- !query schema struct<(CAST(1 AS TINYINT) = CAST(1 AS TINYINT)):boolean> --- !query 139 output +-- !query output true --- !query 140 +-- !query SELECT '1' = cast(1 as smallint) FROM t --- !query 140 schema +-- !query schema struct<(CAST(1 AS SMALLINT) = CAST(1 AS SMALLINT)):boolean> --- !query 140 output +-- !query output true --- !query 141 +-- !query SELECT '1' = cast(1 as int) FROM t --- !query 141 schema +-- !query schema struct<(CAST(1 AS INT) = CAST(1 AS INT)):boolean> --- !query 141 output +-- !query output true --- !query 142 +-- !query SELECT '1' = cast(1 as bigint) FROM t --- !query 142 schema +-- !query schema struct<(CAST(1 AS BIGINT) = CAST(1 AS BIGINT)):boolean> --- !query 142 output +-- !query output true --- !query 143 +-- !query SELECT '1' = cast(1 as float) FROM t --- !query 143 schema +-- !query schema struct<(CAST(1 AS FLOAT) = CAST(1 AS FLOAT)):boolean> --- !query 143 output +-- !query output true --- !query 144 +-- !query SELECT '1' = cast(1 as double) FROM t --- !query 144 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 144 output +-- !query output true --- !query 145 +-- !query SELECT '1' = cast(1 as decimal(10, 0)) FROM t --- !query 145 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 145 output +-- !query output true --- !query 146 +-- !query SELECT '1' = '1' FROM t --- !query 146 schema +-- !query schema struct<(1 = 1):boolean> --- !query 146 output +-- !query output true --- !query 147 +-- !query SELECT '1' = cast('1' as binary) FROM t --- !query 147 schema +-- !query schema struct<(CAST(1 AS BINARY) = CAST(1 AS BINARY)):boolean> --- !query 147 output +-- !query output true --- !query 148 +-- !query SELECT '1' = cast(1 as boolean) FROM t --- !query 148 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) = CAST(1 AS BOOLEAN)):boolean> --- !query 148 output +-- !query output true --- !query 149 +-- !query SELECT '1' = cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 149 schema +-- !query schema struct<(CAST(1 AS TIMESTAMP) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> --- !query 149 output +-- !query output NULL --- !query 150 +-- !query SELECT '1' = cast('2017-12-11 09:30:00' as date) FROM t --- !query 150 schema +-- !query schema struct<(CAST(1 AS DATE) = CAST(2017-12-11 09:30:00 AS DATE)):boolean> --- !query 150 output +-- !query output NULL --- !query 151 +-- !query SELECT cast(1 as tinyint) = '1' FROM t --- !query 151 schema +-- !query schema struct<(CAST(1 AS TINYINT) = CAST(1 AS TINYINT)):boolean> --- !query 151 output +-- !query output true --- !query 152 +-- !query SELECT cast(1 as smallint) = '1' FROM t --- !query 152 schema +-- !query schema struct<(CAST(1 AS SMALLINT) = CAST(1 AS SMALLINT)):boolean> --- !query 152 output +-- !query output true --- !query 153 +-- !query SELECT cast(1 as int) = '1' FROM t --- !query 153 schema +-- !query schema struct<(CAST(1 AS INT) = CAST(1 AS INT)):boolean> --- !query 153 output +-- !query output true --- !query 154 +-- !query SELECT cast(1 as bigint) = '1' FROM t --- !query 154 schema +-- !query schema struct<(CAST(1 AS BIGINT) = CAST(1 AS BIGINT)):boolean> --- !query 154 output +-- !query output true --- !query 155 +-- !query SELECT cast(1 as float) = '1' FROM t --- !query 155 schema +-- !query schema struct<(CAST(1 AS FLOAT) = CAST(1 AS FLOAT)):boolean> --- !query 155 output +-- !query output true --- !query 156 +-- !query SELECT cast(1 as double) = '1' FROM t --- !query 156 schema +-- !query schema struct<(CAST(1 AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 156 output +-- !query output true --- !query 157 +-- !query SELECT cast(1 as decimal(10, 0)) = '1' FROM t --- !query 157 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(1 AS DOUBLE)):boolean> --- !query 157 output +-- !query output true --- !query 158 +-- !query SELECT cast('1' as binary) = '1' FROM t --- !query 158 schema +-- !query schema struct<(CAST(1 AS BINARY) = CAST(1 AS BINARY)):boolean> --- !query 158 output +-- !query output true --- !query 159 +-- !query SELECT cast(1 as boolean) = '1' FROM t --- !query 159 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) = CAST(1 AS BOOLEAN)):boolean> --- !query 159 output +-- !query output true --- !query 160 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = '1' FROM t --- !query 160 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS TIMESTAMP)):boolean> --- !query 160 output +-- !query output NULL --- !query 161 +-- !query SELECT cast('2017-12-11 09:30:00' as date) = '1' FROM t --- !query 161 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DATE)):boolean> --- !query 161 output +-- !query output NULL --- !query 162 +-- !query SELECT '1' <=> cast(1 as tinyint) FROM t --- !query 162 schema +-- !query schema struct<(CAST(1 AS TINYINT) <=> CAST(1 AS TINYINT)):boolean> --- !query 162 output +-- !query output true --- !query 163 +-- !query SELECT '1' <=> cast(1 as smallint) FROM t --- !query 163 schema +-- !query schema struct<(CAST(1 AS SMALLINT) <=> CAST(1 AS SMALLINT)):boolean> --- !query 163 output +-- !query output true --- !query 164 +-- !query SELECT '1' <=> cast(1 as int) FROM t --- !query 164 schema +-- !query schema struct<(CAST(1 AS INT) <=> CAST(1 AS INT)):boolean> --- !query 164 output +-- !query output true --- !query 165 +-- !query SELECT '1' <=> cast(1 as bigint) FROM t --- !query 165 schema +-- !query schema struct<(CAST(1 AS BIGINT) <=> CAST(1 AS BIGINT)):boolean> --- !query 165 output +-- !query output true --- !query 166 +-- !query SELECT '1' <=> cast(1 as float) FROM t --- !query 166 schema +-- !query schema struct<(CAST(1 AS FLOAT) <=> CAST(1 AS FLOAT)):boolean> --- !query 166 output +-- !query output true --- !query 167 +-- !query SELECT '1' <=> cast(1 as double) FROM t --- !query 167 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> --- !query 167 output +-- !query output true --- !query 168 +-- !query SELECT '1' <=> cast(1 as decimal(10, 0)) FROM t --- !query 168 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <=> CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 168 output +-- !query output true --- !query 169 +-- !query SELECT '1' <=> '1' FROM t --- !query 169 schema +-- !query schema struct<(1 <=> 1):boolean> --- !query 169 output +-- !query output true --- !query 170 +-- !query SELECT '1' <=> cast('1' as binary) FROM t --- !query 170 schema +-- !query schema struct<(CAST(1 AS BINARY) <=> CAST(1 AS BINARY)):boolean> --- !query 170 output +-- !query output true --- !query 171 +-- !query SELECT '1' <=> cast(1 as boolean) FROM t --- !query 171 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) <=> CAST(1 AS BOOLEAN)):boolean> --- !query 171 output +-- !query output true --- !query 172 +-- !query SELECT '1' <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 172 schema +-- !query schema struct<(CAST(1 AS TIMESTAMP) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> --- !query 172 output +-- !query output false --- !query 173 +-- !query SELECT '1' <=> cast('2017-12-11 09:30:00' as date) FROM t --- !query 173 schema +-- !query schema struct<(CAST(1 AS DATE) <=> CAST(2017-12-11 09:30:00 AS DATE)):boolean> --- !query 173 output +-- !query output false --- !query 174 +-- !query SELECT cast(1 as tinyint) <=> '1' FROM t --- !query 174 schema +-- !query schema struct<(CAST(1 AS TINYINT) <=> CAST(1 AS TINYINT)):boolean> --- !query 174 output +-- !query output true --- !query 175 +-- !query SELECT cast(1 as smallint) <=> '1' FROM t --- !query 175 schema +-- !query schema struct<(CAST(1 AS SMALLINT) <=> CAST(1 AS SMALLINT)):boolean> --- !query 175 output +-- !query output true --- !query 176 +-- !query SELECT cast(1 as int) <=> '1' FROM t --- !query 176 schema +-- !query schema struct<(CAST(1 AS INT) <=> CAST(1 AS INT)):boolean> --- !query 176 output +-- !query output true --- !query 177 +-- !query SELECT cast(1 as bigint) <=> '1' FROM t --- !query 177 schema +-- !query schema struct<(CAST(1 AS BIGINT) <=> CAST(1 AS BIGINT)):boolean> --- !query 177 output +-- !query output true --- !query 178 +-- !query SELECT cast(1 as float) <=> '1' FROM t --- !query 178 schema +-- !query schema struct<(CAST(1 AS FLOAT) <=> CAST(1 AS FLOAT)):boolean> --- !query 178 output +-- !query output true --- !query 179 +-- !query SELECT cast(1 as double) <=> '1' FROM t --- !query 179 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> --- !query 179 output +-- !query output true --- !query 180 +-- !query SELECT cast(1 as decimal(10, 0)) <=> '1' FROM t --- !query 180 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <=> CAST(1 AS DOUBLE)):boolean> --- !query 180 output +-- !query output true --- !query 181 +-- !query SELECT cast('1' as binary) <=> '1' FROM t --- !query 181 schema +-- !query schema struct<(CAST(1 AS BINARY) <=> CAST(1 AS BINARY)):boolean> --- !query 181 output +-- !query output true --- !query 182 +-- !query SELECT cast(1 as boolean) <=> '1' FROM t --- !query 182 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) <=> CAST(1 AS BOOLEAN)):boolean> --- !query 182 output +-- !query output true --- !query 183 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> '1' FROM t --- !query 183 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS TIMESTAMP)):boolean> --- !query 183 output +-- !query output false --- !query 184 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <=> '1' FROM t --- !query 184 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DATE)):boolean> --- !query 184 output +-- !query output false --- !query 185 +-- !query SELECT '1' < cast(1 as tinyint) FROM t --- !query 185 schema +-- !query schema struct<(CAST(1 AS TINYINT) < CAST(1 AS TINYINT)):boolean> --- !query 185 output +-- !query output false --- !query 186 +-- !query SELECT '1' < cast(1 as smallint) FROM t --- !query 186 schema +-- !query schema struct<(CAST(1 AS SMALLINT) < CAST(1 AS SMALLINT)):boolean> --- !query 186 output +-- !query output false --- !query 187 +-- !query SELECT '1' < cast(1 as int) FROM t --- !query 187 schema +-- !query schema struct<(CAST(1 AS INT) < CAST(1 AS INT)):boolean> --- !query 187 output +-- !query output false --- !query 188 +-- !query SELECT '1' < cast(1 as bigint) FROM t --- !query 188 schema +-- !query schema struct<(CAST(1 AS BIGINT) < CAST(1 AS BIGINT)):boolean> --- !query 188 output +-- !query output false --- !query 189 +-- !query SELECT '1' < cast(1 as float) FROM t --- !query 189 schema +-- !query schema struct<(CAST(1 AS FLOAT) < CAST(1 AS FLOAT)):boolean> --- !query 189 output +-- !query output false --- !query 190 +-- !query SELECT '1' < cast(1 as double) FROM t --- !query 190 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 190 output +-- !query output false --- !query 191 +-- !query SELECT '1' < cast(1 as decimal(10, 0)) FROM t --- !query 191 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 191 output +-- !query output false --- !query 192 +-- !query SELECT '1' < '1' FROM t --- !query 192 schema +-- !query schema struct<(1 < 1):boolean> --- !query 192 output +-- !query output false --- !query 193 +-- !query SELECT '1' < cast('1' as binary) FROM t --- !query 193 schema +-- !query schema struct<(CAST(1 AS BINARY) < CAST(1 AS BINARY)):boolean> --- !query 193 output +-- !query output false --- !query 194 +-- !query SELECT '1' < cast(1 as boolean) FROM t --- !query 194 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) < CAST(1 AS BOOLEAN)):boolean> --- !query 194 output +-- !query output false --- !query 195 +-- !query SELECT '1' < cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 195 schema +-- !query schema struct<(CAST(1 AS TIMESTAMP) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> --- !query 195 output +-- !query output NULL --- !query 196 +-- !query SELECT '1' < cast('2017-12-11 09:30:00' as date) FROM t --- !query 196 schema +-- !query schema struct<(CAST(1 AS DATE) < CAST(2017-12-11 09:30:00 AS DATE)):boolean> --- !query 196 output +-- !query output NULL --- !query 197 +-- !query SELECT '1' <= cast(1 as tinyint) FROM t --- !query 197 schema +-- !query schema struct<(CAST(1 AS TINYINT) <= CAST(1 AS TINYINT)):boolean> --- !query 197 output +-- !query output true --- !query 198 +-- !query SELECT '1' <= cast(1 as smallint) FROM t --- !query 198 schema +-- !query schema struct<(CAST(1 AS SMALLINT) <= CAST(1 AS SMALLINT)):boolean> --- !query 198 output +-- !query output true --- !query 199 +-- !query SELECT '1' <= cast(1 as int) FROM t --- !query 199 schema +-- !query schema struct<(CAST(1 AS INT) <= CAST(1 AS INT)):boolean> --- !query 199 output +-- !query output true --- !query 200 +-- !query SELECT '1' <= cast(1 as bigint) FROM t --- !query 200 schema +-- !query schema struct<(CAST(1 AS BIGINT) <= CAST(1 AS BIGINT)):boolean> --- !query 200 output +-- !query output true --- !query 201 +-- !query SELECT '1' <= cast(1 as float) FROM t --- !query 201 schema +-- !query schema struct<(CAST(1 AS FLOAT) <= CAST(1 AS FLOAT)):boolean> --- !query 201 output +-- !query output true --- !query 202 +-- !query SELECT '1' <= cast(1 as double) FROM t --- !query 202 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 202 output +-- !query output true --- !query 203 +-- !query SELECT '1' <= cast(1 as decimal(10, 0)) FROM t --- !query 203 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 203 output +-- !query output true --- !query 204 +-- !query SELECT '1' <= '1' FROM t --- !query 204 schema +-- !query schema struct<(1 <= 1):boolean> --- !query 204 output +-- !query output true --- !query 205 +-- !query SELECT '1' <= cast('1' as binary) FROM t --- !query 205 schema +-- !query schema struct<(CAST(1 AS BINARY) <= CAST(1 AS BINARY)):boolean> --- !query 205 output +-- !query output true --- !query 206 +-- !query SELECT '1' <= cast(1 as boolean) FROM t --- !query 206 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) <= CAST(1 AS BOOLEAN)):boolean> --- !query 206 output +-- !query output true --- !query 207 +-- !query SELECT '1' <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 207 schema +-- !query schema struct<(CAST(1 AS TIMESTAMP) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> --- !query 207 output +-- !query output NULL --- !query 208 +-- !query SELECT '1' <= cast('2017-12-11 09:30:00' as date) FROM t --- !query 208 schema +-- !query schema struct<(CAST(1 AS DATE) <= CAST(2017-12-11 09:30:00 AS DATE)):boolean> --- !query 208 output +-- !query output NULL --- !query 209 +-- !query SELECT '1' > cast(1 as tinyint) FROM t --- !query 209 schema +-- !query schema struct<(CAST(1 AS TINYINT) > CAST(1 AS TINYINT)):boolean> --- !query 209 output +-- !query output false --- !query 210 +-- !query SELECT '1' > cast(1 as smallint) FROM t --- !query 210 schema +-- !query schema struct<(CAST(1 AS SMALLINT) > CAST(1 AS SMALLINT)):boolean> --- !query 210 output +-- !query output false --- !query 211 +-- !query SELECT '1' > cast(1 as int) FROM t --- !query 211 schema +-- !query schema struct<(CAST(1 AS INT) > CAST(1 AS INT)):boolean> --- !query 211 output +-- !query output false --- !query 212 +-- !query SELECT '1' > cast(1 as bigint) FROM t --- !query 212 schema +-- !query schema struct<(CAST(1 AS BIGINT) > CAST(1 AS BIGINT)):boolean> --- !query 212 output +-- !query output false --- !query 213 +-- !query SELECT '1' > cast(1 as float) FROM t --- !query 213 schema +-- !query schema struct<(CAST(1 AS FLOAT) > CAST(1 AS FLOAT)):boolean> --- !query 213 output +-- !query output false --- !query 214 +-- !query SELECT '1' > cast(1 as double) FROM t --- !query 214 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 214 output +-- !query output false --- !query 215 +-- !query SELECT '1' > cast(1 as decimal(10, 0)) FROM t --- !query 215 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 215 output +-- !query output false --- !query 216 +-- !query SELECT '1' > '1' FROM t --- !query 216 schema +-- !query schema struct<(1 > 1):boolean> --- !query 216 output +-- !query output false --- !query 217 +-- !query SELECT '1' > cast('1' as binary) FROM t --- !query 217 schema +-- !query schema struct<(CAST(1 AS BINARY) > CAST(1 AS BINARY)):boolean> --- !query 217 output +-- !query output false --- !query 218 +-- !query SELECT '1' > cast(1 as boolean) FROM t --- !query 218 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) > CAST(1 AS BOOLEAN)):boolean> --- !query 218 output +-- !query output false --- !query 219 +-- !query SELECT '1' > cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 219 schema +-- !query schema struct<(CAST(1 AS TIMESTAMP) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> --- !query 219 output +-- !query output NULL --- !query 220 +-- !query SELECT '1' > cast('2017-12-11 09:30:00' as date) FROM t --- !query 220 schema +-- !query schema struct<(CAST(1 AS DATE) > CAST(2017-12-11 09:30:00 AS DATE)):boolean> --- !query 220 output +-- !query output NULL --- !query 221 +-- !query SELECT '1' >= cast(1 as tinyint) FROM t --- !query 221 schema +-- !query schema struct<(CAST(1 AS TINYINT) >= CAST(1 AS TINYINT)):boolean> --- !query 221 output +-- !query output true --- !query 222 +-- !query SELECT '1' >= cast(1 as smallint) FROM t --- !query 222 schema +-- !query schema struct<(CAST(1 AS SMALLINT) >= CAST(1 AS SMALLINT)):boolean> --- !query 222 output +-- !query output true --- !query 223 +-- !query SELECT '1' >= cast(1 as int) FROM t --- !query 223 schema +-- !query schema struct<(CAST(1 AS INT) >= CAST(1 AS INT)):boolean> --- !query 223 output +-- !query output true --- !query 224 +-- !query SELECT '1' >= cast(1 as bigint) FROM t --- !query 224 schema +-- !query schema struct<(CAST(1 AS BIGINT) >= CAST(1 AS BIGINT)):boolean> --- !query 224 output +-- !query output true --- !query 225 +-- !query SELECT '1' >= cast(1 as float) FROM t --- !query 225 schema +-- !query schema struct<(CAST(1 AS FLOAT) >= CAST(1 AS FLOAT)):boolean> --- !query 225 output +-- !query output true --- !query 226 +-- !query SELECT '1' >= cast(1 as double) FROM t --- !query 226 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 226 output +-- !query output true --- !query 227 +-- !query SELECT '1' >= cast(1 as decimal(10, 0)) FROM t --- !query 227 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE)):boolean> --- !query 227 output +-- !query output true --- !query 228 +-- !query SELECT '1' >= '1' FROM t --- !query 228 schema +-- !query schema struct<(1 >= 1):boolean> --- !query 228 output +-- !query output true --- !query 229 +-- !query SELECT '1' >= cast('1' as binary) FROM t --- !query 229 schema +-- !query schema struct<(CAST(1 AS BINARY) >= CAST(1 AS BINARY)):boolean> --- !query 229 output +-- !query output true --- !query 230 +-- !query SELECT '1' >= cast(1 as boolean) FROM t --- !query 230 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) >= CAST(1 AS BOOLEAN)):boolean> --- !query 230 output +-- !query output true --- !query 231 +-- !query SELECT '1' >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 231 schema +-- !query schema struct<(CAST(1 AS TIMESTAMP) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> --- !query 231 output +-- !query output NULL --- !query 232 +-- !query SELECT '1' >= cast('2017-12-11 09:30:00' as date) FROM t --- !query 232 schema +-- !query schema struct<(CAST(1 AS DATE) >= CAST(2017-12-11 09:30:00 AS DATE)):boolean> --- !query 232 output +-- !query output NULL --- !query 233 +-- !query SELECT '1' <> cast(1 as tinyint) FROM t --- !query 233 schema +-- !query schema struct<(NOT (CAST(1 AS TINYINT) = CAST(1 AS TINYINT))):boolean> --- !query 233 output +-- !query output false --- !query 234 +-- !query SELECT '1' <> cast(1 as smallint) FROM t --- !query 234 schema +-- !query schema struct<(NOT (CAST(1 AS SMALLINT) = CAST(1 AS SMALLINT))):boolean> --- !query 234 output +-- !query output false --- !query 235 +-- !query SELECT '1' <> cast(1 as int) FROM t --- !query 235 schema +-- !query schema struct<(NOT (CAST(1 AS INT) = CAST(1 AS INT))):boolean> --- !query 235 output +-- !query output false --- !query 236 +-- !query SELECT '1' <> cast(1 as bigint) FROM t --- !query 236 schema +-- !query schema struct<(NOT (CAST(1 AS BIGINT) = CAST(1 AS BIGINT))):boolean> --- !query 236 output +-- !query output false --- !query 237 +-- !query SELECT '1' <> cast(1 as float) FROM t --- !query 237 schema +-- !query schema struct<(NOT (CAST(1 AS FLOAT) = CAST(1 AS FLOAT))):boolean> --- !query 237 output +-- !query output false --- !query 238 +-- !query SELECT '1' <> cast(1 as double) FROM t --- !query 238 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 238 output +-- !query output false --- !query 239 +-- !query SELECT '1' <> cast(1 as decimal(10, 0)) FROM t --- !query 239 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE))):boolean> --- !query 239 output +-- !query output false --- !query 240 +-- !query SELECT '1' <> '1' FROM t --- !query 240 schema +-- !query schema struct<(NOT (1 = 1)):boolean> --- !query 240 output +-- !query output false --- !query 241 +-- !query SELECT '1' <> cast('1' as binary) FROM t --- !query 241 schema +-- !query schema struct<(NOT (CAST(1 AS BINARY) = CAST(1 AS BINARY))):boolean> --- !query 241 output +-- !query output false --- !query 242 +-- !query SELECT '1' <> cast(1 as boolean) FROM t --- !query 242 schema +-- !query schema struct<(NOT (CAST(1 AS BOOLEAN) = CAST(1 AS BOOLEAN))):boolean> --- !query 242 output +-- !query output false --- !query 243 +-- !query SELECT '1' <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 243 schema +-- !query schema struct<(NOT (CAST(1 AS TIMESTAMP) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> --- !query 243 output +-- !query output NULL --- !query 244 +-- !query SELECT '1' <> cast('2017-12-11 09:30:00' as date) FROM t --- !query 244 schema +-- !query schema struct<(NOT (CAST(1 AS DATE) = CAST(2017-12-11 09:30:00 AS DATE))):boolean> --- !query 244 output +-- !query output NULL --- !query 245 +-- !query SELECT cast(1 as tinyint) < '1' FROM t --- !query 245 schema +-- !query schema struct<(CAST(1 AS TINYINT) < CAST(1 AS TINYINT)):boolean> --- !query 245 output +-- !query output false --- !query 246 +-- !query SELECT cast(1 as smallint) < '1' FROM t --- !query 246 schema +-- !query schema struct<(CAST(1 AS SMALLINT) < CAST(1 AS SMALLINT)):boolean> --- !query 246 output +-- !query output false --- !query 247 +-- !query SELECT cast(1 as int) < '1' FROM t --- !query 247 schema +-- !query schema struct<(CAST(1 AS INT) < CAST(1 AS INT)):boolean> --- !query 247 output +-- !query output false --- !query 248 +-- !query SELECT cast(1 as bigint) < '1' FROM t --- !query 248 schema +-- !query schema struct<(CAST(1 AS BIGINT) < CAST(1 AS BIGINT)):boolean> --- !query 248 output +-- !query output false --- !query 249 +-- !query SELECT cast(1 as float) < '1' FROM t --- !query 249 schema +-- !query schema struct<(CAST(1 AS FLOAT) < CAST(1 AS FLOAT)):boolean> --- !query 249 output +-- !query output false --- !query 250 +-- !query SELECT cast(1 as double) < '1' FROM t --- !query 250 schema +-- !query schema struct<(CAST(1 AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 250 output +-- !query output false --- !query 251 +-- !query SELECT cast(1 as decimal(10, 0)) < '1' FROM t --- !query 251 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) < CAST(1 AS DOUBLE)):boolean> --- !query 251 output +-- !query output false --- !query 252 +-- !query SELECT '1' < '1' FROM t --- !query 252 schema +-- !query schema struct<(1 < 1):boolean> --- !query 252 output +-- !query output false --- !query 253 +-- !query SELECT cast('1' as binary) < '1' FROM t --- !query 253 schema +-- !query schema struct<(CAST(1 AS BINARY) < CAST(1 AS BINARY)):boolean> --- !query 253 output +-- !query output false --- !query 254 +-- !query SELECT cast(1 as boolean) < '1' FROM t --- !query 254 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) < CAST(1 AS BOOLEAN)):boolean> --- !query 254 output +-- !query output false --- !query 255 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < '1' FROM t --- !query 255 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS TIMESTAMP)):boolean> --- !query 255 output +-- !query output NULL --- !query 256 +-- !query SELECT cast('2017-12-11 09:30:00' as date) < '1' FROM t --- !query 256 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DATE)):boolean> --- !query 256 output +-- !query output NULL --- !query 257 +-- !query SELECT cast(1 as tinyint) <= '1' FROM t --- !query 257 schema +-- !query schema struct<(CAST(1 AS TINYINT) <= CAST(1 AS TINYINT)):boolean> --- !query 257 output +-- !query output true --- !query 258 +-- !query SELECT cast(1 as smallint) <= '1' FROM t --- !query 258 schema +-- !query schema struct<(CAST(1 AS SMALLINT) <= CAST(1 AS SMALLINT)):boolean> --- !query 258 output +-- !query output true --- !query 259 +-- !query SELECT cast(1 as int) <= '1' FROM t --- !query 259 schema +-- !query schema struct<(CAST(1 AS INT) <= CAST(1 AS INT)):boolean> --- !query 259 output +-- !query output true --- !query 260 +-- !query SELECT cast(1 as bigint) <= '1' FROM t --- !query 260 schema +-- !query schema struct<(CAST(1 AS BIGINT) <= CAST(1 AS BIGINT)):boolean> --- !query 260 output +-- !query output true --- !query 261 +-- !query SELECT cast(1 as float) <= '1' FROM t --- !query 261 schema +-- !query schema struct<(CAST(1 AS FLOAT) <= CAST(1 AS FLOAT)):boolean> --- !query 261 output +-- !query output true --- !query 262 +-- !query SELECT cast(1 as double) <= '1' FROM t --- !query 262 schema +-- !query schema struct<(CAST(1 AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 262 output +-- !query output true --- !query 263 +-- !query SELECT cast(1 as decimal(10, 0)) <= '1' FROM t --- !query 263 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) <= CAST(1 AS DOUBLE)):boolean> --- !query 263 output +-- !query output true --- !query 264 +-- !query SELECT '1' <= '1' FROM t --- !query 264 schema +-- !query schema struct<(1 <= 1):boolean> --- !query 264 output +-- !query output true --- !query 265 +-- !query SELECT cast('1' as binary) <= '1' FROM t --- !query 265 schema +-- !query schema struct<(CAST(1 AS BINARY) <= CAST(1 AS BINARY)):boolean> --- !query 265 output +-- !query output true --- !query 266 +-- !query SELECT cast(1 as boolean) <= '1' FROM t --- !query 266 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) <= CAST(1 AS BOOLEAN)):boolean> --- !query 266 output +-- !query output true --- !query 267 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= '1' FROM t --- !query 267 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS TIMESTAMP)):boolean> --- !query 267 output +-- !query output NULL --- !query 268 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <= '1' FROM t --- !query 268 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DATE)):boolean> --- !query 268 output +-- !query output NULL --- !query 269 +-- !query SELECT cast(1 as tinyint) > '1' FROM t --- !query 269 schema +-- !query schema struct<(CAST(1 AS TINYINT) > CAST(1 AS TINYINT)):boolean> --- !query 269 output +-- !query output false --- !query 270 +-- !query SELECT cast(1 as smallint) > '1' FROM t --- !query 270 schema +-- !query schema struct<(CAST(1 AS SMALLINT) > CAST(1 AS SMALLINT)):boolean> --- !query 270 output +-- !query output false --- !query 271 +-- !query SELECT cast(1 as int) > '1' FROM t --- !query 271 schema +-- !query schema struct<(CAST(1 AS INT) > CAST(1 AS INT)):boolean> --- !query 271 output +-- !query output false --- !query 272 +-- !query SELECT cast(1 as bigint) > '1' FROM t --- !query 272 schema +-- !query schema struct<(CAST(1 AS BIGINT) > CAST(1 AS BIGINT)):boolean> --- !query 272 output +-- !query output false --- !query 273 +-- !query SELECT cast(1 as float) > '1' FROM t --- !query 273 schema +-- !query schema struct<(CAST(1 AS FLOAT) > CAST(1 AS FLOAT)):boolean> --- !query 273 output +-- !query output false --- !query 274 +-- !query SELECT cast(1 as double) > '1' FROM t --- !query 274 schema +-- !query schema struct<(CAST(1 AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 274 output +-- !query output false --- !query 275 +-- !query SELECT cast(1 as decimal(10, 0)) > '1' FROM t --- !query 275 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) > CAST(1 AS DOUBLE)):boolean> --- !query 275 output +-- !query output false --- !query 276 +-- !query SELECT '1' > '1' FROM t --- !query 276 schema +-- !query schema struct<(1 > 1):boolean> --- !query 276 output +-- !query output false --- !query 277 +-- !query SELECT cast('1' as binary) > '1' FROM t --- !query 277 schema +-- !query schema struct<(CAST(1 AS BINARY) > CAST(1 AS BINARY)):boolean> --- !query 277 output +-- !query output false --- !query 278 +-- !query SELECT cast(1 as boolean) > '1' FROM t --- !query 278 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) > CAST(1 AS BOOLEAN)):boolean> --- !query 278 output +-- !query output false --- !query 279 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > '1' FROM t --- !query 279 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS TIMESTAMP)):boolean> --- !query 279 output +-- !query output NULL --- !query 280 +-- !query SELECT cast('2017-12-11 09:30:00' as date) > '1' FROM t --- !query 280 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DATE)):boolean> --- !query 280 output +-- !query output NULL --- !query 281 +-- !query SELECT cast(1 as tinyint) >= '1' FROM t --- !query 281 schema +-- !query schema struct<(CAST(1 AS TINYINT) >= CAST(1 AS TINYINT)):boolean> --- !query 281 output +-- !query output true --- !query 282 +-- !query SELECT cast(1 as smallint) >= '1' FROM t --- !query 282 schema +-- !query schema struct<(CAST(1 AS SMALLINT) >= CAST(1 AS SMALLINT)):boolean> --- !query 282 output +-- !query output true --- !query 283 +-- !query SELECT cast(1 as int) >= '1' FROM t --- !query 283 schema +-- !query schema struct<(CAST(1 AS INT) >= CAST(1 AS INT)):boolean> --- !query 283 output +-- !query output true --- !query 284 +-- !query SELECT cast(1 as bigint) >= '1' FROM t --- !query 284 schema +-- !query schema struct<(CAST(1 AS BIGINT) >= CAST(1 AS BIGINT)):boolean> --- !query 284 output +-- !query output true --- !query 285 +-- !query SELECT cast(1 as float) >= '1' FROM t --- !query 285 schema +-- !query schema struct<(CAST(1 AS FLOAT) >= CAST(1 AS FLOAT)):boolean> --- !query 285 output +-- !query output true --- !query 286 +-- !query SELECT cast(1 as double) >= '1' FROM t --- !query 286 schema +-- !query schema struct<(CAST(1 AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 286 output +-- !query output true --- !query 287 +-- !query SELECT cast(1 as decimal(10, 0)) >= '1' FROM t --- !query 287 schema +-- !query schema struct<(CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) >= CAST(1 AS DOUBLE)):boolean> --- !query 287 output +-- !query output true --- !query 288 +-- !query SELECT '1' >= '1' FROM t --- !query 288 schema +-- !query schema struct<(1 >= 1):boolean> --- !query 288 output +-- !query output true --- !query 289 +-- !query SELECT cast('1' as binary) >= '1' FROM t --- !query 289 schema +-- !query schema struct<(CAST(1 AS BINARY) >= CAST(1 AS BINARY)):boolean> --- !query 289 output +-- !query output true --- !query 290 +-- !query SELECT cast(1 as boolean) >= '1' FROM t --- !query 290 schema +-- !query schema struct<(CAST(1 AS BOOLEAN) >= CAST(1 AS BOOLEAN)):boolean> --- !query 290 output +-- !query output true --- !query 291 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= '1' FROM t --- !query 291 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS TIMESTAMP)):boolean> --- !query 291 output +-- !query output NULL --- !query 292 +-- !query SELECT cast('2017-12-11 09:30:00' as date) >= '1' FROM t --- !query 292 schema +-- !query schema struct<(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DATE)):boolean> --- !query 292 output +-- !query output NULL --- !query 293 +-- !query SELECT cast(1 as tinyint) <> '1' FROM t --- !query 293 schema +-- !query schema struct<(NOT (CAST(1 AS TINYINT) = CAST(1 AS TINYINT))):boolean> --- !query 293 output +-- !query output false --- !query 294 +-- !query SELECT cast(1 as smallint) <> '1' FROM t --- !query 294 schema +-- !query schema struct<(NOT (CAST(1 AS SMALLINT) = CAST(1 AS SMALLINT))):boolean> --- !query 294 output +-- !query output false --- !query 295 +-- !query SELECT cast(1 as int) <> '1' FROM t --- !query 295 schema +-- !query schema struct<(NOT (CAST(1 AS INT) = CAST(1 AS INT))):boolean> --- !query 295 output +-- !query output false --- !query 296 +-- !query SELECT cast(1 as bigint) <> '1' FROM t --- !query 296 schema +-- !query schema struct<(NOT (CAST(1 AS BIGINT) = CAST(1 AS BIGINT))):boolean> --- !query 296 output +-- !query output false --- !query 297 +-- !query SELECT cast(1 as float) <> '1' FROM t --- !query 297 schema +-- !query schema struct<(NOT (CAST(1 AS FLOAT) = CAST(1 AS FLOAT))):boolean> --- !query 297 output +-- !query output false --- !query 298 +-- !query SELECT cast(1 as double) <> '1' FROM t --- !query 298 schema +-- !query schema struct<(NOT (CAST(1 AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 298 output +-- !query output false --- !query 299 +-- !query SELECT cast(1 as decimal(10, 0)) <> '1' FROM t --- !query 299 schema +-- !query schema struct<(NOT (CAST(CAST(1 AS DECIMAL(10,0)) AS DOUBLE) = CAST(1 AS DOUBLE))):boolean> --- !query 299 output +-- !query output false --- !query 300 +-- !query SELECT '1' <> '1' FROM t --- !query 300 schema +-- !query schema struct<(NOT (1 = 1)):boolean> --- !query 300 output +-- !query output false --- !query 301 +-- !query SELECT cast('1' as binary) <> '1' FROM t --- !query 301 schema +-- !query schema struct<(NOT (CAST(1 AS BINARY) = CAST(1 AS BINARY))):boolean> --- !query 301 output +-- !query output false --- !query 302 +-- !query SELECT cast(1 as boolean) <> '1' FROM t --- !query 302 schema +-- !query schema struct<(NOT (CAST(1 AS BOOLEAN) = CAST(1 AS BOOLEAN))):boolean> --- !query 302 output +-- !query output false --- !query 303 +-- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> '1' FROM t --- !query 303 schema +-- !query schema struct<(NOT (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS TIMESTAMP))):boolean> --- !query 303 output +-- !query output NULL --- !query 304 +-- !query SELECT cast('2017-12-11 09:30:00' as date) <> '1' FROM t --- !query 304 schema +-- !query schema struct<(NOT (CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DATE))):boolean> --- !query 304 output +-- !query output NULL --- !query 305 +-- !query SELECT abs('1') FROM t --- !query 305 schema +-- !query schema struct --- !query 305 output +-- !query output 1.0 --- !query 306 +-- !query SELECT sum('1') FROM t --- !query 306 schema +-- !query schema struct --- !query 306 output +-- !query output 1.0 --- !query 307 +-- !query SELECT avg('1') FROM t --- !query 307 schema +-- !query schema struct --- !query 307 output +-- !query output 1.0 --- !query 308 +-- !query SELECT stddev_pop('1') FROM t --- !query 308 schema +-- !query schema struct --- !query 308 output +-- !query output 0.0 --- !query 309 +-- !query SELECT stddev_samp('1') FROM t --- !query 309 schema +-- !query schema struct --- !query 309 output +-- !query output NaN --- !query 310 +-- !query SELECT - '1' FROM t --- !query 310 schema +-- !query schema struct<(- CAST(1 AS DOUBLE)):double> --- !query 310 output +-- !query output -1.0 --- !query 311 +-- !query SELECT + '1' FROM t --- !query 311 schema +-- !query schema struct<(+ CAST(1 AS DOUBLE)):double> --- !query 311 output +-- !query output 1.0 --- !query 312 +-- !query SELECT var_pop('1') FROM t --- !query 312 schema +-- !query schema struct --- !query 312 output +-- !query output 0.0 --- !query 313 +-- !query SELECT var_samp('1') FROM t --- !query 313 schema +-- !query schema struct --- !query 313 output +-- !query output NaN --- !query 314 +-- !query SELECT skewness('1') FROM t --- !query 314 schema +-- !query schema struct --- !query 314 output +-- !query output NaN --- !query 315 +-- !query SELECT kurtosis('1') FROM t --- !query 315 schema +-- !query schema struct --- !query 315 output +-- !query output NaN diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out index 850cf9171a2fd..7b419c6702586 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -2,260 +2,260 @@ -- Number of queries: 32 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 'aa' as a --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query select cast(a as byte) from t --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output NULL --- !query 2 +-- !query select cast(a as short) from t --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output NULL --- !query 3 +-- !query select cast(a as int) from t --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output NULL --- !query 4 +-- !query select cast(a as long) from t --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output NULL --- !query 5 +-- !query select cast(a as float) from t --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output NULL --- !query 6 +-- !query select cast(a as double) from t --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL --- !query 7 +-- !query select cast(a as decimal) from t --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL --- !query 8 +-- !query select cast(a as boolean) from t --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NULL --- !query 9 +-- !query select cast(a as timestamp) from t --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output NULL --- !query 10 +-- !query select cast(a as date) from t --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output NULL --- !query 11 +-- !query select cast(a as binary) from t --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output aa --- !query 12 +-- !query select cast(a as array) from t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 't.`a`' due to data type mismatch: cannot cast string to array; line 1 pos 7 --- !query 13 +-- !query select cast(a as struct) from t --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 't.`a`' due to data type mismatch: cannot cast string to struct; line 1 pos 7 --- !query 14 +-- !query select cast(a as map) from t --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 't.`a`' due to data type mismatch: cannot cast string to map; line 1 pos 7 --- !query 15 +-- !query select to_timestamp(a) from t --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output NULL --- !query 16 +-- !query select to_timestamp('2018-01-01', a) from t --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output NULL --- !query 17 +-- !query select to_unix_timestamp(a) from t --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output NULL --- !query 18 +-- !query select to_unix_timestamp('2018-01-01', a) from t --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output NULL --- !query 19 +-- !query select unix_timestamp(a) from t --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output NULL --- !query 20 +-- !query select unix_timestamp('2018-01-01', a) from t --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output NULL --- !query 21 +-- !query select from_unixtime(a) from t --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL --- !query 22 +-- !query select from_unixtime('2018-01-01', a) from t --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL --- !query 23 +-- !query select next_day(a, 'MO') from t --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output NULL --- !query 24 +-- !query select next_day('2018-01-01', a) from t --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output NULL --- !query 25 +-- !query select trunc(a, 'MM') from t --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output NULL --- !query 26 +-- !query select trunc('2018-01-01', a) from t --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output NULL --- !query 27 +-- !query select unhex('-123') --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output NULL --- !query 28 +-- !query select sha2(a, a) from t --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output NULL --- !query 29 +-- !query select get_json_object(a, a) from t --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output NULL --- !query 30 +-- !query select json_tuple(a, a) from t --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output NULL --- !query 31 +-- !query select from_json(a, 'a INT') from t --- !query 31 schema +-- !query schema struct> --- !query 31 output +-- !query output {"a":null} diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out index 20a9e47217238..89b1cdb3e353d 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out @@ -2,1304 +2,1304 @@ -- Number of queries: 145 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 2 --- !query 2 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 2 --- !query 3 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as int) FROM t --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 --- !query 4 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 2 --- !query 5 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as float) FROM t --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1.0 2.0 --- !query 6 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as double) FROM t --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1.0 2.0 --- !query 7 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 2 --- !query 8 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as string) FROM t --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 2 --- !query 9 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> tinyint at the first column of the second table; --- !query 10 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> tinyint at the first column of the second table; --- !query 11 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> tinyint at the first column of the second table; --- !query 12 +-- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> tinyint at the first column of the second table; --- !query 13 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 2 --- !query 14 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 2 --- !query 15 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as int) FROM t --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 2 --- !query 16 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1 2 --- !query 17 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as float) FROM t --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1.0 2.0 --- !query 18 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as double) FROM t --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1.0 2.0 --- !query 19 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1 2 --- !query 20 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as string) FROM t --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1 2 --- !query 21 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> smallint at the first column of the second table; --- !query 22 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> smallint at the first column of the second table; --- !query 23 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> smallint at the first column of the second table; --- !query 24 +-- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> smallint at the first column of the second table; --- !query 25 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 1 2 --- !query 26 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 2 --- !query 27 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as int) FROM t --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 1 2 --- !query 28 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 1 2 --- !query 29 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as float) FROM t --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output 1.0 2.0 --- !query 30 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as double) FROM t --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1.0 2.0 --- !query 31 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 1 2 --- !query 32 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as string) FROM t --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 1 2 --- !query 33 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> int at the first column of the second table; --- !query 34 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> int at the first column of the second table; --- !query 35 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 35 schema +-- !query schema struct<> --- !query 35 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> int at the first column of the second table; --- !query 36 +-- !query SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> int at the first column of the second table; --- !query 37 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 1 2 --- !query 38 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 1 2 --- !query 39 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as int) FROM t --- !query 39 schema +-- !query schema struct --- !query 39 output +-- !query output 1 2 --- !query 40 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 1 2 --- !query 41 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as float) FROM t --- !query 41 schema +-- !query schema struct --- !query 41 output +-- !query output 1.0 2.0 --- !query 42 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as double) FROM t --- !query 42 schema +-- !query schema struct --- !query 42 output +-- !query output 1.0 2.0 --- !query 43 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 1 2 --- !query 44 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as string) FROM t --- !query 44 schema +-- !query schema struct --- !query 44 output +-- !query output 1 2 --- !query 45 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 45 schema +-- !query schema struct<> --- !query 45 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> bigint at the first column of the second table; --- !query 46 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 46 schema +-- !query schema struct<> --- !query 46 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> bigint at the first column of the second table; --- !query 47 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 47 schema +-- !query schema struct<> --- !query 47 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> bigint at the first column of the second table; --- !query 48 +-- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 48 schema +-- !query schema struct<> --- !query 48 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> bigint at the first column of the second table; --- !query 49 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 49 schema +-- !query schema struct --- !query 49 output +-- !query output 1.0 2.0 --- !query 50 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 50 schema +-- !query schema struct --- !query 50 output +-- !query output 1.0 2.0 --- !query 51 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as int) FROM t --- !query 51 schema +-- !query schema struct --- !query 51 output +-- !query output 1.0 2.0 --- !query 52 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 52 schema +-- !query schema struct --- !query 52 output +-- !query output 1.0 2.0 --- !query 53 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as float) FROM t --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 1.0 2.0 --- !query 54 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as double) FROM t --- !query 54 schema +-- !query schema struct --- !query 54 output +-- !query output 1.0 2.0 --- !query 55 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 55 schema +-- !query schema struct --- !query 55 output +-- !query output 1.0 2.0 --- !query 56 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as string) FROM t --- !query 56 schema +-- !query schema struct --- !query 56 output +-- !query output 1.0 2 --- !query 57 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> float at the first column of the second table; --- !query 58 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> float at the first column of the second table; --- !query 59 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> float at the first column of the second table; --- !query 60 +-- !query SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> float at the first column of the second table; --- !query 61 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 61 schema +-- !query schema struct --- !query 61 output +-- !query output 1.0 2.0 --- !query 62 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 62 schema +-- !query schema struct --- !query 62 output +-- !query output 1.0 2.0 --- !query 63 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as int) FROM t --- !query 63 schema +-- !query schema struct --- !query 63 output +-- !query output 1.0 2.0 --- !query 64 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output 1.0 2.0 --- !query 65 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as float) FROM t --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output 1.0 2.0 --- !query 66 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as double) FROM t --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output 1.0 2.0 --- !query 67 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output 1.0 2.0 --- !query 68 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as string) FROM t --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output 1.0 2 --- !query 69 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 69 schema +-- !query schema struct<> --- !query 69 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> double at the first column of the second table; --- !query 70 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 70 schema +-- !query schema struct<> --- !query 70 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> double at the first column of the second table; --- !query 71 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 71 schema +-- !query schema struct<> --- !query 71 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> double at the first column of the second table; --- !query 72 +-- !query SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 72 schema +-- !query schema struct<> --- !query 72 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> double at the first column of the second table; --- !query 73 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output 1 2 --- !query 74 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output 1 2 --- !query 75 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as int) FROM t --- !query 75 schema +-- !query schema struct --- !query 75 output +-- !query output 1 2 --- !query 76 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 76 schema +-- !query schema struct --- !query 76 output +-- !query output 1 2 --- !query 77 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as float) FROM t --- !query 77 schema +-- !query schema struct --- !query 77 output +-- !query output 1.0 2.0 --- !query 78 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as double) FROM t --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output 1.0 2.0 --- !query 79 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 79 schema +-- !query schema struct --- !query 79 output +-- !query output 1 2 --- !query 80 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as string) FROM t --- !query 80 schema +-- !query schema struct --- !query 80 output +-- !query output 1 2 --- !query 81 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 81 schema +-- !query schema struct<> --- !query 81 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> decimal(10,0) at the first column of the second table; --- !query 82 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 82 schema +-- !query schema struct<> --- !query 82 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> decimal(10,0) at the first column of the second table; --- !query 83 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 83 schema +-- !query schema struct<> --- !query 83 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> decimal(10,0) at the first column of the second table; --- !query 84 +-- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 84 schema +-- !query schema struct<> --- !query 84 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> decimal(10,0) at the first column of the second table; --- !query 85 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output 1 2 --- !query 86 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 86 schema +-- !query schema struct --- !query 86 output +-- !query output 1 2 --- !query 87 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as int) FROM t --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 1 2 --- !query 88 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output 1 2 --- !query 89 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as float) FROM t --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output 1 2.0 --- !query 90 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as double) FROM t --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 1 2.0 --- !query 91 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output 1 2 --- !query 92 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as string) FROM t --- !query 92 schema +-- !query schema struct --- !query 92 output +-- !query output 1 2 --- !query 93 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 93 schema +-- !query schema struct<> --- !query 93 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> string at the first column of the second table; --- !query 94 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 94 schema +-- !query schema struct<> --- !query 94 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> string at the first column of the second table; --- !query 95 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 95 schema +-- !query schema struct --- !query 95 output +-- !query output 1 2017-12-11 09:30:00 --- !query 96 +-- !query SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 96 schema +-- !query schema struct --- !query 96 output +-- !query output 1 2017-12-11 --- !query 97 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. tinyint <> binary at the first column of the second table; --- !query 98 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 98 schema +-- !query schema struct<> --- !query 98 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. smallint <> binary at the first column of the second table; --- !query 99 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t --- !query 99 schema +-- !query schema struct<> --- !query 99 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. int <> binary at the first column of the second table; --- !query 100 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 100 schema +-- !query schema struct<> --- !query 100 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. bigint <> binary at the first column of the second table; --- !query 101 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t --- !query 101 schema +-- !query schema struct<> --- !query 101 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. float <> binary at the first column of the second table; --- !query 102 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. double <> binary at the first column of the second table; --- !query 103 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. decimal(10,0) <> binary at the first column of the second table; --- !query 104 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t --- !query 104 schema +-- !query schema struct<> --- !query 104 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. string <> binary at the first column of the second table; --- !query 105 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 105 schema +-- !query schema struct --- !query 105 output +-- !query output 1 2 --- !query 106 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 106 schema +-- !query schema struct<> --- !query 106 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> binary at the first column of the second table; --- !query 107 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> binary at the first column of the second table; --- !query 108 +-- !query SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> binary at the first column of the second table; --- !query 109 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 109 schema +-- !query schema struct<> --- !query 109 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. tinyint <> boolean at the first column of the second table; --- !query 110 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 110 schema +-- !query schema struct<> --- !query 110 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. smallint <> boolean at the first column of the second table; --- !query 111 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t --- !query 111 schema +-- !query schema struct<> --- !query 111 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. int <> boolean at the first column of the second table; --- !query 112 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 112 schema +-- !query schema struct<> --- !query 112 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. bigint <> boolean at the first column of the second table; --- !query 113 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t --- !query 113 schema +-- !query schema struct<> --- !query 113 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. float <> boolean at the first column of the second table; --- !query 114 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. double <> boolean at the first column of the second table; --- !query 115 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. decimal(10,0) <> boolean at the first column of the second table; --- !query 116 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t --- !query 116 schema +-- !query schema struct<> --- !query 116 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. string <> boolean at the first column of the second table; --- !query 117 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> boolean at the first column of the second table; --- !query 118 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 118 schema +-- !query schema struct --- !query 118 output +-- !query output true --- !query 119 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 119 schema +-- !query schema struct<> --- !query 119 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. timestamp <> boolean at the first column of the second table; --- !query 120 +-- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. date <> boolean at the first column of the second table; --- !query 121 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. tinyint <> timestamp at the first column of the second table; --- !query 122 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 122 schema +-- !query schema struct<> --- !query 122 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. smallint <> timestamp at the first column of the second table; --- !query 123 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t --- !query 123 schema +-- !query schema struct<> --- !query 123 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. int <> timestamp at the first column of the second table; --- !query 124 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. bigint <> timestamp at the first column of the second table; --- !query 125 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t --- !query 125 schema +-- !query schema struct<> --- !query 125 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. float <> timestamp at the first column of the second table; --- !query 126 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as double) FROM t --- !query 126 schema +-- !query schema struct<> --- !query 126 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. double <> timestamp at the first column of the second table; --- !query 127 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. decimal(10,0) <> timestamp at the first column of the second table; --- !query 128 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t --- !query 128 schema +-- !query schema struct --- !query 128 output +-- !query output 2 2017-12-12 09:30:00 --- !query 129 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 129 schema +-- !query schema struct<> --- !query 129 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> timestamp at the first column of the second table; --- !query 130 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 130 schema +-- !query schema struct<> --- !query 130 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> timestamp at the first column of the second table; --- !query 131 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 131 schema +-- !query schema struct --- !query 131 output +-- !query output 2017-12-11 09:30:00 2017-12-12 09:30:00 --- !query 132 +-- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 132 schema +-- !query schema struct --- !query 132 output +-- !query output 2017-12-11 00:00:00 2017-12-12 09:30:00 --- !query 133 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t --- !query 133 schema +-- !query schema struct<> --- !query 133 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. tinyint <> date at the first column of the second table; --- !query 134 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t --- !query 134 schema +-- !query schema struct<> --- !query 134 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. smallint <> date at the first column of the second table; --- !query 135 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. int <> date at the first column of the second table; --- !query 136 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) FROM t --- !query 136 schema +-- !query schema struct<> --- !query 136 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. bigint <> date at the first column of the second table; --- !query 137 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t --- !query 137 schema +-- !query schema struct<> --- !query 137 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. float <> date at the first column of the second table; --- !query 138 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) FROM t --- !query 138 schema +-- !query schema struct<> --- !query 138 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. double <> date at the first column of the second table; --- !query 139 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t --- !query 139 schema +-- !query schema struct<> --- !query 139 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. decimal(10,0) <> date at the first column of the second table; --- !query 140 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t --- !query 140 schema +-- !query schema struct --- !query 140 output +-- !query output 2 2017-12-12 --- !query 141 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t --- !query 141 schema +-- !query schema struct<> --- !query 141 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. binary <> date at the first column of the second table; --- !query 142 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t --- !query 142 schema +-- !query schema struct<> --- !query 142 output +-- !query output org.apache.spark.sql.AnalysisException Union can only be performed on tables with the compatible column types. boolean <> date at the first column of the second table; --- !query 143 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query 143 schema +-- !query schema struct --- !query 143 output +-- !query output 2017-12-11 09:30:00 2017-12-12 00:00:00 --- !query 144 +-- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query 144 schema +-- !query schema struct --- !query 144 output +-- !query output 2017-12-11 2017-12-12 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out index 5b77bf9f35f25..12af1b7d034da 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out @@ -2,205 +2,205 @@ -- Number of queries: 25 --- !query 0 +-- !query CREATE TEMPORARY VIEW t AS SELECT 1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint)) FROM t --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 --- !query 2 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint)) FROM t --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 --- !query 3 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int)) FROM t --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 --- !query 4 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint)) FROM t --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 --- !query 5 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float)) FROM t --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 --- !query 6 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double)) FROM t --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 --- !query 7 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0))) FROM t --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 --- !query 8 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string)) FROM t --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 --- !query 9 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary)) FROM t --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 --- !query 10 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean)) FROM t --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 --- !query 11 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 --- !query 12 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date)) FROM t --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 1 --- !query 13 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 --- !query 14 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 --- !query 15 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 --- !query 16 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 1 --- !query 17 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 --- !query 18 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 --- !query 19 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0)) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1 --- !query 20 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS STRING) FOLLOWING' due to data type mismatch: The data type of the upper bound 'string' does not match the expected data type '(numeric or interval)'.; line 1 pos 21 --- !query 21 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS BINARY) FOLLOWING' due to data type mismatch: The data type of the upper bound 'binary' does not match the expected data type '(numeric or interval)'.; line 1 pos 21 --- !query 22 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS BOOLEAN) FOLLOWING' due to data type mismatch: The data type of the upper bound 'boolean' does not match the expected data type '(numeric or interval)'.; line 1 pos 21 --- !query 23 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY 1 ORDER BY CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'timestamp' used in the order specification does not match the data type 'int' which is used in the range frame.; line 1 pos 21 --- !query 24 +-- !query SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1 diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out index f4455bb717578..9f4229a11b65d 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -2,69 +2,69 @@ -- Number of queries: 8 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1), (2), (3), (4) as t1(int_col1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg' --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT default.myDoubleAvg(int_col1) as my_avg from t1 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 102.5 --- !query 3 +-- !query SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1 --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 --- !query 4 +-- !query CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT default.udaf1(int_col1) as udaf1 from t1 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 7 --- !query 6 +-- !query DROP FUNCTION myDoubleAvg --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query DROP FUNCTION udaf1 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out index a2f64717d73a1..d65c56774eafd 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -2,321 +2,321 @@ -- Number of queries: 43 --- !query 0 +-- !query SELECT avg(udf(four)) AS avg_1 FROM onek --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1.5 --- !query 1 +-- !query SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 32.666666666666664 --- !query 2 +-- !query select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 107.943 --- !query 3 +-- !query SELECT sum(udf(four)) AS sum_1500 FROM onek --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1500 --- !query 4 +-- !query SELECT udf(sum(a)) AS sum_198 FROM aggtest --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 198 --- !query 5 +-- !query SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 431.77260909229517 --- !query 6 +-- !query SELECT udf(max(four)) AS max_3 FROM onek --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 3 --- !query 7 +-- !query SELECT max(udf(a)) AS max_100 FROM aggtest --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 100 --- !query 8 +-- !query SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 324.78 --- !query 9 +-- !query SELECT stddev_pop(udf(b)) FROM aggtest --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 131.10703231895047 --- !query 10 +-- !query SELECT udf(stddev_samp(b)) FROM aggtest --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 151.38936080399804 --- !query 11 +-- !query SELECT var_pop(udf(b)) FROM aggtest --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 17189.053923482323 --- !query 12 +-- !query SELECT udf(var_samp(b)) FROM aggtest --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 22918.738564643096 --- !query 13 +-- !query SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 131.18117242958306 --- !query 14 +-- !query SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 151.47497042966097 --- !query 15 +-- !query SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 17208.5 --- !query 16 +-- !query SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 22944.666666666668 --- !query 17 +-- !query SELECT udf(var_pop(1.0)), var_samp(udf(2.0)) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 0.0 NaN --- !query 18 +-- !query SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 0.0 NaN --- !query 19 +-- !query select sum(udf(CAST(null AS int))) from range(1,4) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output NULL --- !query 20 +-- !query select sum(udf(CAST(null AS long))) from range(1,4) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output NULL --- !query 21 +-- !query select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL --- !query 22 +-- !query select sum(udf(CAST(null AS DOUBLE))) from range(1,4) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL --- !query 23 +-- !query select avg(udf(CAST(null AS int))) from range(1,4) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output NULL --- !query 24 +-- !query select avg(udf(CAST(null AS long))) from range(1,4) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output NULL --- !query 25 +-- !query select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output NULL --- !query 26 +-- !query select avg(udf(CAST(null AS DOUBLE))) from range(1,4) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output NULL --- !query 27 +-- !query select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output NaN --- !query 28 +-- !query select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output NaN --- !query 29 +-- !query SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) FROM (VALUES ('Infinity'), ('1')) v(x) --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output Infinity NaN --- !query 30 +-- !query SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) FROM (VALUES ('Infinity'), ('Infinity')) v(x) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output Infinity NaN --- !query 31 +-- !query SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) FROM (VALUES ('-Infinity'), ('Infinity')) v(x) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output NaN NaN --- !query 32 +-- !query SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 1.00000005E8 2.5 --- !query 33 +-- !query SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) FROM (VALUES (7000000000005), (7000000000007)) v(x) --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 7.000000000006E12 1.0 --- !query 34 +-- !query SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 653.6289553875104 871.5052738500139 --- !query 35 +-- !query SELECT corr(b, udf(a)) FROM aggtest --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 0.1396345165178734 --- !query 36 +-- !query SELECT count(udf(four)) AS cnt_1000 FROM onek --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 1000 --- !query 37 +-- !query SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 4 --- !query 38 +-- !query select ten, udf(count(*)), sum(udf(four)) from onek group by ten order by ten --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output 0 100 100 1 100 200 2 100 100 @@ -329,12 +329,12 @@ struct --- !query 39 output +-- !query output 0 100 2 1 100 4 2 100 2 @@ -347,13 +347,13 @@ struct --- !query 40 output +-- !query output 0 2 2 2 4 2 @@ -361,14 +361,14 @@ struct --- !query 41 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -376,12 +376,12 @@ Expression in where clause: [(sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT)) Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))]; --- !query 42 +-- !query select (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) from tenk1 o --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 67 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out index 3b54637c449d8..c10fe9b51dd72 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 25 --- !query 0 +-- !query create temporary view int4_tbl as select * from values (0), (123456), @@ -10,40 +10,40 @@ create temporary view int4_tbl as select * from values (2147483647), (-2147483647) as int4_tbl(f1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES (1, 1, 1, 1L), (3, 3, 3, null), (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output NULL NULL --- !query 3 +-- !query SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output NULL NULL --- !query 4 +-- !query SELECT BIT_AND(cast(b1 as tinyint)) AS a1, BIT_AND(cast(b2 as smallint)) AS b1, @@ -54,13 +54,13 @@ SELECT BIT_OR(b3) AS g7, BIT_OR(b4) AS h3 FROM bitwise_test --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 1 1 1 7 7 7 3 --- !query 5 +-- !query SELECT (NULL AND NULL) IS NULL AS `t`, (TRUE AND NULL) IS NULL AS `t`, @@ -71,13 +71,13 @@ SELECT NOT (TRUE AND FALSE) AS `t`, NOT (FALSE AND TRUE) AS `t`, NOT (FALSE AND FALSE) AS `t` --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output true true false true false true true true true --- !query 6 +-- !query SELECT (NULL OR NULL) IS NULL AS `t`, (TRUE OR NULL) IS NULL AS `t`, @@ -88,32 +88,32 @@ SELECT (TRUE OR FALSE) AS `t`, (FALSE OR TRUE) AS `t`, NOT (FALSE OR FALSE) AS `t` --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output true false true false true true true true true --- !query 7 +-- !query CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES (TRUE, null, FALSE, null), (FALSE, TRUE, null, null), (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NULL NULL --- !query 9 +-- !query SELECT BOOL_AND(b1) AS f1, BOOL_AND(b2) AS t2, @@ -122,13 +122,13 @@ SELECT BOOL_AND(NOT b2) AS f5, BOOL_AND(NOT b3) AS t6 FROM bool_test --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output false true false NULL false true --- !query 10 +-- !query SELECT EVERY(b1) AS f1, EVERY(b2) AS t2, @@ -137,13 +137,13 @@ SELECT EVERY(NOT b2) AS f5, EVERY(NOT b3) AS t6 FROM bool_test --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output false true false NULL false true --- !query 11 +-- !query SELECT BOOL_OR(b1) AS t1, BOOL_OR(b2) AS t2, @@ -152,113 +152,113 @@ SELECT BOOL_OR(NOT b2) AS f5, BOOL_OR(NOT b3) AS t6 FROM bool_test --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output true true false NULL false true --- !query 12 +-- !query select min(udf(unique1)) from tenk1 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 0 --- !query 13 +-- !query select udf(max(unique1)) from tenk1 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 9999 --- !query 14 +-- !query select max(unique1) from tenk1 where udf(unique1) < 42 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 41 --- !query 15 +-- !query select max(unique1) from tenk1 where unique1 > udf(42) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 9999 --- !query 16 +-- !query select max(unique1) from tenk1 where udf(unique1) > 42000 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output NULL --- !query 17 +-- !query select max(tenthous) from tenk1 where udf(thousand) = 33 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 9033 --- !query 18 +-- !query select min(tenthous) from tenk1 where udf(thousand) = 33 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 33 --- !query 19 +-- !query select distinct max(udf(unique2)) from tenk1 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 9999 --- !query 20 +-- !query select max(unique2) from tenk1 order by udf(1) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 9999 --- !query 21 +-- !query select max(unique2) from tenk1 order by max(udf(unique2)) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 9999 --- !query 22 +-- !query select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 9999 --- !query 23 +-- !query select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 9999 3 9999 2 9999 1 --- !query 24 +-- !query select udf(max(100)) from tenk1 --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 100 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out index eff33f280cff4..f491d9b9ba3a8 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out @@ -2,21 +2,21 @@ -- Number of queries: 2 --- !query 0 +-- !query select udf(max(min(unique1))) from tenk1 --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output org.apache.spark.sql.AnalysisException It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query.; --- !query 1 +-- !query select udf((select udf(count(*)) from (values (1)) t0(inner_c))) as col from (values (2),(3)) t1(outer_c) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 1 1 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out index 7012b15213662..04c4f54b02a3e 100755 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out @@ -2,243 +2,243 @@ -- Number of queries: 35 --- !query 0 +-- !query CREATE TABLE CASE_TBL ( i integer, f double ) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TABLE CASE2_TBL ( i integer, j integer ) USING parquet --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO CASE_TBL VALUES (1, 10.1) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO CASE_TBL VALUES (2, 20.2) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO CASE_TBL VALUES (3, -30.3) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO CASE_TBL VALUES (4, NULL) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO CASE2_TBL VALUES (1, -1) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO CASE2_TBL VALUES (2, -2) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO CASE2_TBL VALUES (3, -3) --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO CASE2_TBL VALUES (2, -4) --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO CASE2_TBL VALUES (1, NULL) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query INSERT INTO CASE2_TBL VALUES (NULL, -6) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query SELECT '3' AS `One`, CASE WHEN udf(1 < 2) THEN 3 END AS `Simple WHEN` --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 3 3 --- !query 13 +-- !query SELECT '' AS `One`, CASE WHEN 1 > 2 THEN udf(3) END AS `Simple default` --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output NULL --- !query 14 +-- !query SELECT '3' AS `One`, CASE WHEN udf(1) < 2 THEN udf(3) ELSE udf(4) END AS `Simple ELSE` --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 3 3 --- !query 15 +-- !query SELECT udf('4') AS `One`, CASE WHEN 1 > 2 THEN 3 ELSE 4 END AS `ELSE default` --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 4 4 --- !query 16 +-- !query SELECT udf('6') AS `One`, CASE WHEN udf(1 > 2) THEN 3 WHEN udf(4) < 5 THEN 6 ELSE 7 END AS `Two WHEN with default` --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 6 6 --- !query 17 +-- !query SELECT '7' AS `None`, CASE WHEN rand() < udf(0) THEN 1 END AS `NULL on no matches` --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 7 NULL --- !query 18 +-- !query SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1.0 --- !query 19 +-- !query SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1.0 --- !query 20 +-- !query SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl --- !query 20 schema +-- !query schema struct 100) THEN CAST(udf(cast((cast(1 as double) / cast(0 as double)) as string)) AS DOUBLE) ELSE CAST(CAST(udf(cast(0 as string)) AS INT) AS DOUBLE) END:double> --- !query 20 output +-- !query output 0.0 0.0 0.0 0.0 --- !query 21 +-- !query SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 --- !query 22 +-- !query SELECT '' AS `Five`, CASE WHEN i >= 3 THEN i END AS `>= 3 or Null` FROM CASE_TBL --- !query 22 schema +-- !query schema struct= 3 or Null:int> --- !query 22 output +-- !query output 3 4 NULL NULL --- !query 23 +-- !query SELECT '' AS `Five`, CASE WHEN i >= 3 THEN (i + i) ELSE i END AS `Simplest Math` FROM CASE_TBL --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 1 2 6 8 --- !query 24 +-- !query SELECT '' AS `Five`, i AS `Value`, CASE WHEN (i < 0) THEN 'small' WHEN (i = 0) THEN 'zero' @@ -247,16 +247,16 @@ SELECT '' AS `Five`, i AS `Value`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 1 one 2 two 3 big 4 big --- !query 25 +-- !query SELECT '' AS `Five`, CASE WHEN ((i < 0) or (i < 0)) THEN 'small' WHEN ((i = 0) or (i = 0)) THEN 'zero' @@ -265,37 +265,37 @@ SELECT '' AS `Five`, ELSE 'big' END AS `Category` FROM CASE_TBL --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output big big one two --- !query 26 +-- !query SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4 --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 4 NULL --- !query 27 +-- !query SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2 --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output --- !query 28 +-- !query SELECT udf(COALESCE(a.f, b.i, b.j)) FROM CASE_TBL a, CASE2_TBL b --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output -30.3 -30.3 -30.3 @@ -322,24 +322,24 @@ struct --- !query 29 output +-- !query output 4 NULL 2 -2 4 NULL 2 -4 --- !query 30 +-- !query SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, NULLIF(b.i, 4) AS `NULLIF(b.i,4)` FROM CASE_TBL a, CASE2_TBL b --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1 2 1 2 1 3 @@ -366,18 +366,18 @@ struct NULL 3 --- !query 31 +-- !query SELECT '' AS `Two`, * FROM CASE_TBL a, CASE2_TBL b WHERE udf(COALESCE(f,b.i) = 2) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output 4 NULL 2 -2 4 NULL 2 -4 --- !query 32 +-- !query SELECT CASE (CASE vol('bar') WHEN udf('foo') THEN 'it was foo!' @@ -387,23 +387,23 @@ SELECT CASE WHEN udf('it was foo!') THEN 'foo recognized' WHEN 'it was bar!' THEN udf('bar recognized') ELSE 'unrecognized' END AS col --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output bar recognized --- !query 33 +-- !query DROP TABLE CASE_TBL --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output --- !query 34 +-- !query DROP TABLE CASE2_TBL --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out index 6fcff129d7568..f113aee6d3b51 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out @@ -2,17 +2,17 @@ -- Number of queries: 185 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) AS v(f1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (VALUES (123, 456), @@ -21,230 +21,230 @@ CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (4567890123456789, 4567890123456789), (4567890123456789, -4567890123456789)) AS v(q1, q2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM (VALUES (0.0), (1004.30), (-34.84), (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) AS v(f1) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM (VALUES ('doh!'), ('hi de ho neighbor')) AS v(f1) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TABLE J1_TBL ( i integer, j integer, t string ) USING parquet --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query CREATE TABLE J2_TBL ( i integer, k integer ) USING parquet --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO J1_TBL VALUES (1, 4, 'one') --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO J1_TBL VALUES (2, 3, 'two') --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO J1_TBL VALUES (3, 2, 'three') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO J1_TBL VALUES (4, 1, 'four') --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query INSERT INTO J1_TBL VALUES (5, 0, 'five') --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query INSERT INTO J1_TBL VALUES (6, 6, 'six') --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query INSERT INTO J1_TBL VALUES (7, 7, 'seven') --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query INSERT INTO J1_TBL VALUES (8, 8, 'eight') --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query INSERT INTO J1_TBL VALUES (0, NULL, 'zero') --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output --- !query 16 +-- !query INSERT INTO J1_TBL VALUES (NULL, NULL, 'null') --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output --- !query 17 +-- !query INSERT INTO J1_TBL VALUES (NULL, 0, 'zero') --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output --- !query 18 +-- !query INSERT INTO J2_TBL VALUES (1, -1) --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query INSERT INTO J2_TBL VALUES (2, 2) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output --- !query 20 +-- !query INSERT INTO J2_TBL VALUES (3, -3) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query INSERT INTO J2_TBL VALUES (2, 4) --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output --- !query 22 +-- !query INSERT INTO J2_TBL VALUES (5, -5) --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output --- !query 23 +-- !query INSERT INTO J2_TBL VALUES (5, -5) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query INSERT INTO J2_TBL VALUES (0, NULL) --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query INSERT INTO J2_TBL VALUES (NULL, NULL) --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query INSERT INTO J2_TBL VALUES (NULL, 0) --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output --- !query 27 +-- !query SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t) FROM J1_TBL AS tx --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 0 NULL zero 1 4 one 2 3 two @@ -258,12 +258,12 @@ struct --- !query 28 output +-- !query output 0 NULL zero 1 4 one 2 3 two @@ -277,12 +277,12 @@ struct --- !query 29 output +-- !query output 0 NULL zero 1 4 one 2 3 two @@ -296,12 +296,12 @@ struct --- !query 30 output +-- !query output 0 NULL zero 1 4 one 2 3 two @@ -315,12 +315,12 @@ struct --- !query 31 output +-- !query output 0 NULL zero 0 NULL 0 NULL zero 1 -1 0 NULL zero 2 2 @@ -422,12 +422,12 @@ struct --- !query 32 output +-- !query output 0 NULL zero 0 NULL 0 NULL zero 1 -1 0 NULL zero 2 2 @@ -529,22 +529,22 @@ struct NULL NULL null NULL NULL --- !query 33 +-- !query SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t FROM J1_TBL CROSS JOIN J2_TBL --- !query 33 schema +-- !query schema struct<> --- !query 33 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'i' is ambiguous, could be: default.j1_tbl.i, default.j2_tbl.i.; line 1 pos 29 --- !query 34 +-- !query SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t) FROM J1_TBL t1 CROSS JOIN J2_TBL t2 --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 0 -1 zero 0 -3 zero 0 -5 zero @@ -646,13 +646,13 @@ struct --- !query 35 output +-- !query output 0 zero -1 0 zero -3 0 zero -5 @@ -754,12 +754,12 @@ struct --- !query 36 output +-- !query output 0 NULL zero 0 NULL 0 NULL 0 NULL zero 0 NULL 1 -1 0 NULL zero 0 NULL 2 2 @@ -1653,12 +1653,12 @@ struct --- !query 37 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1668,12 +1668,12 @@ struct --- !query 38 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1683,13 +1683,13 @@ struct --- !query 39 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1699,12 +1699,12 @@ struct 5 0 five -5 --- !query 40 +-- !query SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) FROM J1_TBL NATURAL JOIN J2_TBL --- !query 40 schema +-- !query schema struct --- !query 40 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1714,12 +1714,12 @@ struct --- !query 41 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1729,23 +1729,23 @@ struct --- !query 42 output +-- !query output 0 NULL zero NULL 2 3 two 2 4 1 four 2 --- !query 43 +-- !query SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i) --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 0 NULL zero 0 NULL 1 4 one 1 -1 2 3 two 2 2 @@ -1755,23 +1755,23 @@ struct --- !query 44 output +-- !query output 0 NULL zero NULL 0 2 3 two 2 2 4 1 four 2 4 --- !query 45 +-- !query SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k))) --- !query 45 schema +-- !query schema struct --- !query 45 output +-- !query output 0 NULL zero 2 2 0 NULL zero 2 4 0 NULL zero NULL 0 @@ -1783,13 +1783,13 @@ struct --- !query 46 output +-- !query output NULL NULL null NULL NULL 0 zero NULL 0 NULL zero NULL @@ -1805,13 +1805,13 @@ struct --- !query 47 output +-- !query output NULL NULL null NULL NULL 0 zero NULL 0 NULL zero NULL @@ -1827,12 +1827,12 @@ struct --- !query 48 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1844,12 +1844,12 @@ struct --- !query 49 output +-- !query output 0 NULL zero NULL 1 4 one -1 2 3 two 2 @@ -1861,13 +1861,13 @@ struct --- !query 50 output +-- !query output NULL NULL NULL NULL NULL NULL null NULL NULL 0 zero NULL @@ -1885,13 +1885,13 @@ struct --- !query 51 output +-- !query output NULL NULL NULL NULL NULL NULL null NULL NULL 0 zero NULL @@ -1909,226 +1909,226 @@ struct --- !query 52 output +-- !query output --- !query 53 +-- !query SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1)) --- !query 53 schema +-- !query schema struct --- !query 53 output +-- !query output 1 4 one -1 --- !query 54 +-- !query CREATE TABLE t1 (name STRING, n INTEGER) USING parquet --- !query 54 schema +-- !query schema struct<> --- !query 54 output +-- !query output --- !query 55 +-- !query CREATE TABLE t2 (name STRING, n INTEGER) USING parquet --- !query 55 schema +-- !query schema struct<> --- !query 55 output +-- !query output --- !query 56 +-- !query CREATE TABLE t3 (name STRING, n INTEGER) USING parquet --- !query 56 schema +-- !query schema struct<> --- !query 56 output +-- !query output --- !query 57 +-- !query INSERT INTO t1 VALUES ( 'bb', 11 ) --- !query 57 schema +-- !query schema struct<> --- !query 57 output +-- !query output --- !query 58 +-- !query INSERT INTO t2 VALUES ( 'bb', 12 ) --- !query 58 schema +-- !query schema struct<> --- !query 58 output +-- !query output --- !query 59 +-- !query INSERT INTO t2 VALUES ( 'cc', 22 ) --- !query 59 schema +-- !query schema struct<> --- !query 59 output +-- !query output --- !query 60 +-- !query INSERT INTO t2 VALUES ( 'ee', 42 ) --- !query 60 schema +-- !query schema struct<> --- !query 60 output +-- !query output --- !query 61 +-- !query INSERT INTO t3 VALUES ( 'bb', 13 ) --- !query 61 schema +-- !query schema struct<> --- !query 61 output +-- !query output --- !query 62 +-- !query INSERT INTO t3 VALUES ( 'cc', 23 ) --- !query 62 schema +-- !query schema struct<> --- !query 62 output +-- !query output --- !query 63 +-- !query INSERT INTO t3 VALUES ( 'dd', 33 ) --- !query 63 schema +-- !query schema struct<> --- !query 63 output +-- !query output --- !query 64 +-- !query SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) --- !query 64 schema +-- !query schema struct --- !query 64 output +-- !query output bb 11 12 13 cc NULL 22 23 dd NULL NULL 33 ee NULL 42 NULL --- !query 65 +-- !query SELECT * FROM (SELECT udf(name) as name, t2.n FROM t2) as s2 INNER JOIN (SELECT udf(udf(name)) as name, t3.n FROM t3) s3 USING (name) --- !query 65 schema +-- !query schema struct --- !query 65 output +-- !query output bb 12 13 cc 22 23 --- !query 66 +-- !query SELECT * FROM (SELECT udf(udf(name)) as name, t2.n FROM t2) as s2 LEFT JOIN (SELECT udf(name) as name, t3.n FROM t3) s3 USING (name) --- !query 66 schema +-- !query schema struct --- !query 66 output +-- !query output bb 12 13 cc 22 23 ee 42 NULL --- !query 67 +-- !query SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM (SELECT * FROM t2) as s2 FULL JOIN (SELECT * FROM t3) s3 USING (name) --- !query 67 schema +-- !query schema struct --- !query 67 output +-- !query output bb 12 13 cc 22 23 dd NULL 33 ee 42 NULL --- !query 68 +-- !query SELECT * FROM (SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2 NATURAL INNER JOIN (SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3 --- !query 68 schema +-- !query schema struct --- !query 68 output +-- !query output bb 12 2 13 3 cc 22 2 23 3 --- !query 69 +-- !query SELECT * FROM (SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL LEFT JOIN (SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 --- !query 69 schema +-- !query schema struct --- !query 69 output +-- !query output bb 12 2 13 3 cc 22 2 23 3 ee 42 2 NULL NULL --- !query 70 +-- !query SELECT * FROM (SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL FULL JOIN (SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3 --- !query 70 schema +-- !query schema struct --- !query 70 output +-- !query output bb 12 2 13 3 cc 22 2 23 3 dd NULL NULL 33 3 ee 42 2 NULL NULL --- !query 71 +-- !query SELECT * FROM (SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1 NATURAL INNER JOIN (SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 NATURAL INNER JOIN (SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 --- !query 71 schema +-- !query schema struct --- !query 71 output +-- !query output bb 11 1 12 2 13 3 --- !query 72 +-- !query SELECT * FROM (SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1 NATURAL FULL JOIN (SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2 NATURAL FULL JOIN (SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 --- !query 72 schema +-- !query schema struct --- !query 72 output +-- !query output bb 11 1 12 2 13 3 cc NULL NULL 22 2 23 3 dd NULL NULL NULL NULL 33 3 ee NULL NULL 42 2 NULL NULL --- !query 73 +-- !query SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM (SELECT name, udf(udf(n)) as s1_n FROM t1) as s1 NATURAL FULL JOIN @@ -2137,16 +2137,16 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 ) ss2 --- !query 73 schema +-- !query schema struct --- !query 73 output +-- !query output bb 11 12 13 cc NULL 22 23 dd NULL NULL 33 ee NULL 42 NULL --- !query 74 +-- !query SELECT * FROM (SELECT name, n as s1_n FROM t1) as s1 NATURAL FULL JOIN @@ -2155,55 +2155,55 @@ NATURAL FULL JOIN NATURAL FULL JOIN (SELECT name, udf(n) as s3_n FROM t3) as s3 ) ss2 --- !query 74 schema +-- !query schema struct --- !query 74 output +-- !query output bb 11 12 2 13 cc NULL 22 2 23 dd NULL NULL NULL 33 ee NULL 42 2 NULL --- !query 75 +-- !query SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM (SELECT name, udf(n) as s1_n FROM t1) as s1 FULL JOIN (SELECT name, 2 as s2_n FROM t2) as s2 ON (udf(udf(s1_n)) = udf(s2_n)) --- !query 75 schema +-- !query schema struct --- !query 75 output +-- !query output NULL NULL bb 2 NULL NULL cc 2 NULL NULL ee 2 bb 11 NULL NULL --- !query 76 +-- !query create or replace temporary view x as select * from (values (1,11), (2,22), (3,null), (4,44), (5,null)) as v(x1, x2) --- !query 76 schema +-- !query schema struct<> --- !query 76 output +-- !query output --- !query 77 +-- !query create or replace temporary view y as select * from (values (1,111), (2,222), (3,333), (4,null)) as v(y1, y2) --- !query 77 schema +-- !query schema struct<> --- !query 77 output +-- !query output --- !query 78 +-- !query select udf(udf(x1)), udf(x2) from x --- !query 78 schema +-- !query schema struct --- !query 78 output +-- !query output 1 11 2 22 3 NULL @@ -2211,22 +2211,22 @@ struct --- !query 79 output +-- !query output 1 111 2 222 3 333 4 NULL --- !query 80 +-- !query select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null) --- !query 80 schema +-- !query schema struct --- !query 80 output +-- !query output 1 11 1 111 2 22 2 222 3 NULL NULL NULL @@ -2234,11 +2234,11 @@ struct 5 NULL NULL NULL --- !query 81 +-- !query select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null) --- !query 81 schema +-- !query schema struct --- !query 81 output +-- !query output 1 11 1 111 2 22 2 222 3 NULL 3 333 @@ -2246,12 +2246,12 @@ struct 5 NULL NULL NULL --- !query 82 +-- !query select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2) on (udf(udf(x1)) = udf(xx1)) --- !query 82 schema +-- !query schema struct --- !query 82 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 3 NULL @@ -2259,12 +2259,12 @@ struct 5 NULL NULL NULL 5 NULL --- !query 83 +-- !query select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) on (udf(x1) = xx1 and udf(x2) is not null) --- !query 83 schema +-- !query schema struct --- !query 83 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 NULL NULL @@ -2272,12 +2272,12 @@ struct 5 NULL NULL NULL NULL NULL --- !query 84 +-- !query select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2) on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null) --- !query 84 schema +-- !query schema struct --- !query 84 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 3 NULL @@ -2285,12 +2285,12 @@ struct 5 NULL NULL NULL NULL NULL --- !query 85 +-- !query select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2) on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null) --- !query 85 schema +-- !query schema struct --- !query 85 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 NULL NULL @@ -2298,78 +2298,78 @@ struct 5 NULL NULL NULL NULL NULL --- !query 86 +-- !query select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2) on (udf(x1) = udf(xx1)) where (udf(x2) is not null) --- !query 86 schema +-- !query schema struct --- !query 86 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 4 44 4 NULL 4 44 --- !query 87 +-- !query select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) on (udf(x1) = xx1) where (udf(y2) is not null) --- !query 87 schema +-- !query schema struct --- !query 87 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 3 NULL 3 333 3 NULL --- !query 88 +-- !query select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) on (x1 = udf(xx1)) where (xx2 is not null) --- !query 88 schema +-- !query schema struct --- !query 88 output +-- !query output 1 11 1 111 1 11 2 22 2 222 2 22 4 44 4 NULL 4 44 --- !query 89 +-- !query select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in (select udf(unique1) from tenk1 b join tenk1 c using (unique1) where udf(udf(b.unique2)) = udf(42)) --- !query 89 schema +-- !query schema struct --- !query 89 output +-- !query output 1 --- !query 90 +-- !query select udf(count(*)) from tenk1 x where udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and udf(x.unique1) = 0 and udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1))) --- !query 90 schema +-- !query schema struct --- !query 90 output +-- !query output 1 --- !query 91 +-- !query select udf(udf(count(*))) from tenk1 x where udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and udf(x.unique1) = 0 and udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1))) --- !query 91 schema +-- !query schema struct --- !query 91 output +-- !query output 1 --- !query 92 +-- !query select * from int8_tbl i1 left join (int8_tbl i2 join (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2)) order by udf(udf(1)), 2 --- !query 92 schema +-- !query schema struct --- !query 92 output +-- !query output 4567890123456789 -4567890123456789 NULL NULL NULL 4567890123456789 123 NULL NULL NULL 123 456 123 456 123 @@ -2377,7 +2377,7 @@ struct 4567890123456789 4567890123456789 123 4567890123456789 123 --- !query 93 +-- !query select udf(count(*)) from (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2 @@ -2387,32 +2387,32 @@ from tenk1 t4, tenk1 t5 where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1)) --- !query 93 schema +-- !query schema struct --- !query 93 output +-- !query output 1000 --- !query 94 +-- !query select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from tenk1 t, (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a, (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous)) --- !query 94 schema +-- !query schema struct --- !query 94 output +-- !query output --- !query 95 +-- !query select * from j1_tbl full join (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k) --- !query 95 schema +-- !query schema struct --- !query 95 output +-- !query output 0 NULL zero NULL NULL 1 4 one NULL NULL 2 3 two 2 2 @@ -2434,156 +2434,156 @@ NULL NULL NULL NULL NULL NULL NULL null NULL NULL --- !query 96 +-- !query select udf(count(*)) from (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x left join (select * from tenk1 y order by udf(y.unique2)) y on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2 --- !query 96 schema +-- !query schema struct --- !query 96 output +-- !query output 10000 --- !query 97 +-- !query DROP TABLE t1 --- !query 97 schema +-- !query schema struct<> --- !query 97 output +-- !query output --- !query 98 +-- !query DROP TABLE t2 --- !query 98 schema +-- !query schema struct<> --- !query 98 output +-- !query output --- !query 99 +-- !query DROP TABLE t3 --- !query 99 schema +-- !query schema struct<> --- !query 99 output +-- !query output --- !query 100 +-- !query DROP TABLE J1_TBL --- !query 100 schema +-- !query schema struct<> --- !query 100 output +-- !query output --- !query 101 +-- !query DROP TABLE J2_TBL --- !query 101 schema +-- !query schema struct<> --- !query 101 output +-- !query output --- !query 102 +-- !query create or replace temporary view tt1 as select * from (values (1, 11), (2, NULL)) as v(tt1_id, joincol) --- !query 102 schema +-- !query schema struct<> --- !query 102 output +-- !query output --- !query 103 +-- !query create or replace temporary view tt2 as select * from (values (21, 11), (22, 11)) as v(tt2_id, joincol) --- !query 103 schema +-- !query schema struct<> --- !query 103 output +-- !query output --- !query 104 +-- !query select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol) --- !query 104 schema +-- !query schema struct --- !query 104 output +-- !query output 1 11 21 11 1 11 22 11 2 NULL NULL NULL --- !query 105 +-- !query select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol)) --- !query 105 schema +-- !query schema struct --- !query 105 output +-- !query output 1 11 21 11 1 11 22 11 2 NULL NULL NULL --- !query 106 +-- !query select udf(count(*)) from tenk1 a, tenk1 b where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10) --- !query 106 schema +-- !query schema struct --- !query 106 output +-- !query output 100000 --- !query 107 +-- !query DROP TABLE IF EXISTS tt3 --- !query 107 schema +-- !query schema struct<> --- !query 107 output +-- !query output --- !query 108 +-- !query CREATE TABLE tt3(f1 int, f2 string) USING parquet --- !query 108 schema +-- !query schema struct<> --- !query 108 output +-- !query output --- !query 109 +-- !query INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x --- !query 109 schema +-- !query schema struct<> --- !query 109 output +-- !query output --- !query 110 +-- !query DROP TABLE IF EXISTS tt4 --- !query 110 schema +-- !query schema struct<> --- !query 110 output +-- !query output --- !query 111 +-- !query CREATE TABLE tt4(f1 int) USING parquet --- !query 111 schema +-- !query schema struct<> --- !query 111 output +-- !query output --- !query 112 +-- !query INSERT INTO tt4 VALUES (0),(1),(9999) --- !query 112 schema +-- !query schema struct<> --- !query 112 output +-- !query output --- !query 113 +-- !query SELECT udf(udf(a.f1)) as f1 FROM tt4 a LEFT JOIN ( @@ -2592,242 +2592,242 @@ LEFT JOIN ( WHERE udf(c.f1) IS NULL ) AS d ON udf(a.f1) = d.f1 WHERE udf(udf(d.f1)) IS NULL --- !query 113 schema +-- !query schema struct --- !query 113 output +-- !query output 0 1 9999 --- !query 114 +-- !query create or replace temporary view tt5 as select * from (values (1, 10), (1, 11)) as v(f1, f2) --- !query 114 schema +-- !query schema struct<> --- !query 114 output +-- !query output --- !query 115 +-- !query create or replace temporary view tt6 as select * from (values (1, 9), (1, 2), (2, 9)) as v(f1, f2) --- !query 115 schema +-- !query schema struct<> --- !query 115 output +-- !query output --- !query 116 +-- !query select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2)) --- !query 116 schema +-- !query schema struct --- !query 116 output +-- !query output 1 10 1 9 --- !query 117 +-- !query create or replace temporary view xx as select * from (values (1), (2), (3)) as v(pkxx) --- !query 117 schema +-- !query schema struct<> --- !query 117 output +-- !query output --- !query 118 +-- !query create or replace temporary view yy as select * from (values (101, 1), (201, 2), (301, NULL)) as v(pkyy, pkxx) --- !query 118 schema +-- !query schema struct<> --- !query 118 output +-- !query output --- !query 119 +-- !query select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy, udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx from yy left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy) left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx)) left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx) --- !query 119 schema +-- !query schema struct --- !query 119 output +-- !query output 101 1 101 1 1 201 2 NULL NULL 1 301 NULL NULL NULL 1 --- !query 120 +-- !query create or replace temporary view zt1 as select * from (values (53)) as v(f1) --- !query 120 schema +-- !query schema struct<> --- !query 120 output +-- !query output --- !query 121 +-- !query create or replace temporary view zt2 as select * from (values (53)) as v(f2) --- !query 121 schema +-- !query schema struct<> --- !query 121 output +-- !query output --- !query 122 +-- !query create or replace temporary view zt3(f3 int) using parquet --- !query 122 schema +-- !query schema struct<> --- !query 122 output +-- !query output --- !query 123 +-- !query select * from zt2 left join zt3 on (udf(f2) = udf(udf(f3))) left join zt1 on (udf(udf(f3)) = udf(f1)) where udf(f2) = 53 --- !query 123 schema +-- !query schema struct --- !query 123 output +-- !query output 53 NULL NULL --- !query 124 +-- !query create temp view zv1 as select *,'dummy' AS junk from zt1 --- !query 124 schema +-- !query schema struct<> --- !query 124 output +-- !query output --- !query 125 +-- !query select * from zt2 left join zt3 on (f2 = udf(f3)) left join zv1 on (udf(f3) = f1) where udf(udf(f2)) = 53 --- !query 125 schema +-- !query schema struct --- !query 125 output +-- !query output 53 NULL NULL NULL --- !query 126 +-- !query select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred) from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous) where udf(a.unique1) = 42 and ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3))) --- !query 126 schema +-- !query schema struct --- !query 126 output +-- !query output --- !query 127 +-- !query create or replace temporary view a (i integer) using parquet --- !query 127 schema +-- !query schema struct<> --- !query 127 output +-- !query output --- !query 128 +-- !query create or replace temporary view b (x integer, y integer) using parquet --- !query 128 schema +-- !query schema struct<> --- !query 128 output +-- !query output --- !query 129 +-- !query select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i) --- !query 129 schema +-- !query schema struct --- !query 129 output +-- !query output --- !query 130 +-- !query select udf(t1.q2), udf(count(t2.*)) from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1) group by udf(t1.q2) order by 1 --- !query 130 schema +-- !query schema struct --- !query 130 output +-- !query output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 131 +-- !query select udf(udf(t1.q2)), udf(count(t2.*)) from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1)) group by udf(udf(t1.q2)) order by 1 --- !query 131 schema +-- !query schema struct --- !query 131 output +-- !query output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 132 +-- !query select udf(t1.q2) as q2, udf(udf(count(t2.*))) from int8_tbl t1 left join (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 on (udf(t1.q2) = udf(t2.q1)) group by t1.q2 order by 1 --- !query 132 schema +-- !query schema struct --- !query 132 output +-- !query output -4567890123456789 0 123 2 456 0 4567890123456789 6 --- !query 133 +-- !query create or replace temporary view a as select * from (values ('p'), ('q')) as v(code) --- !query 133 schema +-- !query schema struct<> --- !query 133 output +-- !query output --- !query 134 +-- !query create or replace temporary view b as select * from (values ('p', 1), ('p', 2)) as v(a, num) --- !query 134 schema +-- !query schema struct<> --- !query 134 output +-- !query output --- !query 135 +-- !query create or replace temporary view c as select * from (values ('A', 'p'), ('B', 'q'), ('C', null)) as v(name, a) --- !query 135 schema +-- !query schema struct<> --- !query 135 output +-- !query output --- !query 136 +-- !query select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const) from c left join (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const @@ -2837,15 +2837,15 @@ from c left join ) as ss on (udf(udf(c.a)) = udf(ss.code)) order by c.name --- !query 136 schema +-- !query schema struct --- !query 136 output +-- !query output A p 2 -1 B q 0 -1 C NULL NULL NULL --- !query 137 +-- !query SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN @@ -2861,13 +2861,13 @@ LEFT JOIN ON udf(sub4.key5) = sub3.key3 ) sub2 ON udf(udf(sub1.key1)) = udf(udf(sub2.key3)) --- !query 137 schema +-- !query schema struct --- !query 137 output +-- !query output 1 1 1 1 --- !query 138 +-- !query SELECT * FROM ( SELECT 1 as key1 ) sub1 LEFT JOIN @@ -2883,13 +2883,13 @@ LEFT JOIN ON sub4.key5 = sub3.key3 ) sub2 ON sub1.key1 = udf(udf(sub2.key3)) --- !query 138 schema +-- !query schema struct --- !query 138 output +-- !query output 1 1 1 1 --- !query 139 +-- !query SELECT udf(qq), udf(udf(unique1)) FROM ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1 @@ -2897,45 +2897,45 @@ SELECT udf(qq), udf(udf(unique1)) ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2 USING (qq) INNER JOIN tenk1 c ON udf(qq) = udf(unique2) --- !query 139 schema +-- !query schema struct --- !query 139 output +-- !query output 123 4596 123 4596 456 7318 --- !query 140 +-- !query create or replace temporary view nt1 as select * from (values(1,true,true), (2,true,false), (3,false,false)) as v(id, a1, a2) --- !query 140 schema +-- !query schema struct<> --- !query 140 output +-- !query output --- !query 141 +-- !query create or replace temporary view nt2 as select * from (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) as v(id, nt1_id, b1, b2) --- !query 141 schema +-- !query schema struct<> --- !query 141 output +-- !query output --- !query 142 +-- !query create or replace temporary view nt3 as select * from (values(1,1,true), (2,2,false), (3,3,true)) as v(id, nt2_id, c1) --- !query 142 schema +-- !query schema struct<> --- !query 142 output +-- !query output --- !query 143 +-- !query select udf(nt3.id) from nt3 as nt3 left join @@ -2947,17 +2947,17 @@ from nt3 as nt3 ) as ss2 on udf(ss2.id) = nt3.nt2_id where udf(nt3.id) = 1 and udf(ss2.b3) --- !query 143 schema +-- !query schema struct --- !query 143 output +-- !query output 1 --- !query 144 +-- !query select * from int4_tbl a full join int4_tbl b on true --- !query 144 schema +-- !query schema struct --- !query 144 output +-- !query output -123456 -123456 -123456 -2147483647 -123456 0 @@ -2985,11 +2985,11 @@ struct 2147483647 2147483647 --- !query 145 +-- !query select * from int4_tbl a full join int4_tbl b on false --- !query 145 schema +-- !query schema struct --- !query 145 output +-- !query output -123456 NULL -2147483647 NULL 0 NULL @@ -3002,27 +3002,27 @@ NULL 123456 NULL 2147483647 --- !query 146 +-- !query select udf(count(*)) from tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand)) join int4_tbl on udf(b.thousand) = f1 --- !query 146 schema +-- !query schema struct --- !query 146 output +-- !query output 10 --- !query 147 +-- !query select udf(b.unique1) from tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand) join int4_tbl i1 on udf(b.thousand) = udf(udf(f1)) right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous) order by udf(1) --- !query 147 schema +-- !query schema struct --- !query 147 output +-- !query output NULL NULL 0 @@ -3030,7 +3030,7 @@ NULL NULL --- !query 148 +-- !query select * from ( select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault @@ -3038,43 +3038,43 @@ select * from ) ss where udf(fault) = udf(122) order by udf(fault) --- !query 148 schema +-- !query schema struct --- !query 148 output +-- !query output NULL 123 122 --- !query 149 +-- !query select udf(q1), udf(unique2), udf(thousand), udf(hundred) from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2) where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123))) --- !query 149 schema +-- !query schema struct --- !query 149 output +-- !query output --- !query 150 +-- !query select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2)) where (case when udf(unique2) is null then udf(f1) else 0 end) = 0 --- !query 150 schema +-- !query schema struct --- !query 150 output +-- !query output 0 0 0 --- !query 151 +-- !query select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand)) from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand)) where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44) --- !query 151 schema +-- !query schema struct --- !query 151 output +-- !query output --- !query 152 +-- !query select * from text_tbl t1 inner join int8_tbl i8 @@ -3083,32 +3083,32 @@ select * from on udf(t1.f1) = udf(udf('doh!')) left join int4_tbl i4 on udf(udf(i8.q1)) = i4.f1 --- !query 152 schema +-- !query schema struct --- !query 152 output +-- !query output doh! 123 456 doh! NULL doh! 123 456 hi de ho neighbor NULL --- !query 153 +-- !query select * from (select udf(udf(1)) as id) as xx left join (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id))) on (xx.id = udf(udf(coalesce(yy.id)))) --- !query 153 schema +-- !query schema struct --- !query 153 output +-- !query output 1 1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx 1 --- !query 154 +-- !query select udf(a.q2), udf(b.q1) from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1) where udf(udf(coalesce(b.q1, 1)) > 0) --- !query 154 schema +-- !query schema struct --- !query 154 output +-- !query output -4567890123456789 NULL 123 123 123 123 @@ -3121,124 +3121,124 @@ struct --- !query 155 output +-- !query output --- !query 156 +-- !query create or replace temporary view child as select * from (values (1, 100), (4, 400)) as v(k, cd) --- !query 156 schema +-- !query schema struct<> --- !query 156 output +-- !query output --- !query 157 +-- !query select p.* from parent p left join child c on (udf(p.k) = udf(c.k)) --- !query 157 schema +-- !query schema struct --- !query 157 output +-- !query output 1 10 2 20 3 30 --- !query 158 +-- !query select p.*, linked from parent p left join (select c.*, udf(udf(true)) as linked from child c) as ss on (udf(p.k) = udf(udf(ss.k))) --- !query 158 schema +-- !query schema struct --- !query 158 output +-- !query output 1 10 true 2 20 NULL 3 30 NULL --- !query 159 +-- !query select p.* from parent p left join child c on (udf(p.k) = c.k) where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2)) --- !query 159 schema +-- !query schema struct --- !query 159 output +-- !query output --- !query 160 +-- !query select p.* from (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k) where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2)) --- !query 160 schema +-- !query schema struct --- !query 160 output +-- !query output --- !query 161 +-- !query create or replace temporary view a as select * from (values (0), (1)) as v(id) --- !query 161 schema +-- !query schema struct<> --- !query 161 output +-- !query output --- !query 162 +-- !query create or replace temporary view b as select * from (values (0, 0), (1, NULL)) as v(id, a_id) --- !query 162 schema +-- !query schema struct<> --- !query 162 output +-- !query output --- !query 163 +-- !query SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0) --- !query 163 schema +-- !query schema struct --- !query 163 output +-- !query output 1 NULL NULL --- !query 164 +-- !query SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0) --- !query 164 schema +-- !query schema struct --- !query 164 output +-- !query output 1 NULL --- !query 165 +-- !query create or replace temporary view innertab as select * from (values (123L, 42L)) as v(id, dat1) --- !query 165 schema +-- !query schema struct<> --- !query 165 output +-- !query output --- !query 166 +-- !query SELECT * FROM (SELECT udf(1) AS x) ss1 LEFT JOIN (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2 ON true --- !query 166 schema +-- !query schema struct --- !query 166 output +-- !query output 1 123 456 123 1 123 4567890123456789 123 1 4567890123456789 -4567890123456789 4567890123456789 @@ -3246,163 +3246,163 @@ struct --- !query 167 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'f1' is ambiguous, could be: j.f1, j.f1.; line 2 pos 72 --- !query 168 +-- !query select * from int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1) --- !query 168 schema +-- !query schema struct<> --- !query 168 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`y.f1`' given input columns: [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72 --- !query 169 +-- !query select * from int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1)) --- !query 169 schema +-- !query schema struct --- !query 169 output +-- !query output --- !query 170 +-- !query select udf(t1.uunique1) from tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two) --- !query 170 schema +-- !query schema struct<> --- !query 170 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`t1.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11 --- !query 171 +-- !query select udf(udf(t2.uunique1)) from tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two --- !query 171 schema +-- !query schema struct<> --- !query 171 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`t2.uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 15 --- !query 172 +-- !query select udf(uunique1) from tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two) --- !query 172 schema +-- !query schema struct<> --- !query 172 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`uunique1`' given input columns: [t1.even, t2.even, t1.fivethous, t2.fivethous, t1.four, t2.four, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.string4, t2.string4, t1.stringu1, t2.stringu1, t1.stringu2, t2.stringu2, t1.ten, t2.ten, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.twenty, t2.twenty, t1.two, t2.two, t1.twothousand, t2.twothousand, t1.unique1, t2.unique1, t1.unique2, t2.unique2]; line 1 pos 11 --- !query 173 +-- !query select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss --- !query 173 schema +-- !query schema struct<> --- !query 173 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`f1`' given input columns: []; line 1 pos 55 --- !query 174 +-- !query select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss --- !query 174 schema +-- !query schema struct<> --- !query 174 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`a.f1`' given input columns: []; line 1 pos 42 --- !query 175 +-- !query select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss --- !query 175 schema +-- !query schema struct<> --- !query 175 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`f1`' given input columns: []; line 1 pos 61 --- !query 176 +-- !query select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss --- !query 176 schema +-- !query schema struct<> --- !query 176 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`a.f1`' given input columns: []; line 1 pos 60 --- !query 177 +-- !query CREATE TABLE j1 (id1 int, id2 int) USING parquet --- !query 177 schema +-- !query schema struct<> --- !query 177 output +-- !query output --- !query 178 +-- !query CREATE TABLE j2 (id1 int, id2 int) USING parquet --- !query 178 schema +-- !query schema struct<> --- !query 178 output +-- !query output --- !query 179 +-- !query INSERT INTO j1 values(1,1),(1,2) --- !query 179 schema +-- !query schema struct<> --- !query 179 output +-- !query output --- !query 180 +-- !query INSERT INTO j2 values(1,1) --- !query 180 schema +-- !query schema struct<> --- !query 180 output +-- !query output --- !query 181 +-- !query INSERT INTO j2 values(1,2) --- !query 181 schema +-- !query schema struct<> --- !query 181 output +-- !query output --- !query 182 +-- !query select * from j1 inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2) where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1 --- !query 182 schema +-- !query schema struct --- !query 182 output +-- !query output 1 1 1 1 1 2 1 2 --- !query 183 +-- !query drop table j1 --- !query 183 schema +-- !query schema struct<> --- !query 183 output +-- !query output --- !query 184 +-- !query drop table j2 --- !query 184 schema +-- !query schema struct<> --- !query 184 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out index f731d11c6d3da..68113afdfae30 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out @@ -2,186 +2,186 @@ -- Number of queries: 22 --- !query 0 +-- !query CREATE TABLE test_having (a int, b int, c string, d string) USING parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT udf(b), udf(c) FROM test_having GROUP BY b, c HAVING udf(count(*)) = 1 ORDER BY udf(b), udf(c) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 XXXX 3 bbbb --- !query 12 +-- !query SELECT udf(b), udf(c) FROM test_having GROUP BY b, c HAVING udf(b) = 3 ORDER BY udf(b), udf(c) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 3 BBBB 3 bbbb --- !query 13 +-- !query SELECT udf(c), max(udf(a)) FROM test_having GROUP BY c HAVING udf(count(*)) > 2 OR udf(min(a)) = udf(max(a)) ORDER BY c --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output XXXX 0 bbbb 5 --- !query 14 +-- !query SELECT udf(udf(min(udf(a)))), udf(udf(max(udf(a)))) FROM test_having HAVING udf(udf(min(udf(a)))) = udf(udf(max(udf(a)))) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output --- !query 15 +-- !query SELECT udf(min(udf(a))), udf(udf(max(a))) FROM test_having HAVING udf(min(a)) < udf(max(udf(a))) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 0 9 --- !query 16 +-- !query SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a)) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and 'default.test_having.`a`' is not an aggregate function. Wrap '(min(default.test_having.`a`) AS `min(a#x)`, max(default.test_having.`a`) AS `max(a#x)`)' in windowing function(s) or wrap 'default.test_having.`a`' in first() (or first_value) if you don't care which value you get.; --- !query 17 +-- !query SELECT 1 AS one FROM test_having HAVING udf(a) > 1 --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`a`' given input columns: [one]; line 1 pos 44 --- !query 18 +-- !query SELECT 1 AS one FROM test_having HAVING udf(udf(1) > udf(2)) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output --- !query 19 +-- !query SELECT 1 AS one FROM test_having HAVING udf(udf(1) < udf(2)) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 1 --- !query 20 +-- !query SELECT 1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output 1 --- !query 21 +-- !query DROP TABLE test_having --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out index 2540f0260f09c..11cb682ee1494 100755 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out @@ -2,101 +2,101 @@ -- Number of queries: 38 --- !query 0 +-- !query CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY udf(test_missing_target.c) ORDER BY udf(c) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output ABAB 2 BBBB 2 CCCC 2 @@ -105,12 +105,12 @@ bbbb 1 cccc 2 --- !query 12 +-- !query SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(test_missing_target.c) ORDER BY udf(c) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 2 2 2 @@ -119,43 +119,43 @@ struct 2 --- !query 13 +-- !query SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`b`' given input columns: [CAST(udf(cast(count(1) as string)) AS BIGINT)]; line 1 pos 75 --- !query 14 +-- !query SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 2 3 4 --- !query 15 +-- !query SELECT udf(test_missing_target.b), udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 1 2 2 3 3 4 4 --- !query 16 +-- !query SELECT udf(c) FROM test_missing_target ORDER BY udf(a) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output XXXX ABAB ABAB @@ -168,30 +168,30 @@ CCCC CCCC --- !query 17 +-- !query SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) desc --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 4 3 2 1 --- !query 18 +-- !query SELECT udf(count(*)) FROM test_missing_target ORDER BY udf(1) desc --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 10 --- !query 19 +-- !query SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 1 ORDER BY 1 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output ABAB 2 BBBB 2 CCCC 2 @@ -200,32 +200,32 @@ bbbb 1 cccc 2 --- !query 20 +-- !query SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 3 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.AnalysisException GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 63 --- !query 21 +-- !query SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y WHERE udf(x.a) = udf(y.a) GROUP BY udf(b) ORDER BY udf(b) --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 14 --- !query 22 +-- !query SELECT udf(a), udf(a) FROM test_missing_target ORDER BY udf(a) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output 0 0 1 1 2 2 @@ -238,12 +238,12 @@ struct --- !query 23 output +-- !query output 0.0 0.0 0.5 0.5 1.0 1.0 @@ -256,12 +256,12 @@ struct --- !query 24 output +-- !query output 0.0 0.0 0.5 0.5 1.0 1.0 @@ -274,93 +274,93 @@ struct --- !query 25 output +-- !query output 1 1 2 2 3 3 4 4 --- !query 26 +-- !query SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y WHERE udf(x.a) = udf(y.a) GROUP BY udf(x.b) ORDER BY udf(x.b) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1 2 3 4 --- !query 27 +-- !query SELECT udf(a%2), udf(count(udf(b))) FROM test_missing_target GROUP BY udf(test_missing_target.a%2) ORDER BY udf(test_missing_target.a%2) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 0 5 1 5 --- !query 28 +-- !query SELECT udf(count(c)) FROM test_missing_target GROUP BY udf(lower(test_missing_target.c)) ORDER BY udf(lower(test_missing_target.c)) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 2 3 4 1 --- !query 29 +-- !query SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`b`' given input columns: [CAST(udf(cast(count(cast(udf(cast(a as string)) as int)) as string)) AS BIGINT)]; line 1 pos 80 --- !query 30 +-- !query SELECT udf(count(b)) FROM test_missing_target GROUP BY udf(b/2) ORDER BY udf(b/2) --- !query 30 schema +-- !query schema struct --- !query 30 output +-- !query output 1 2 3 4 --- !query 31 +-- !query SELECT udf(lower(test_missing_target.c)), udf(count(udf(c))) FROM test_missing_target GROUP BY udf(lower(c)) ORDER BY udf(lower(c)) --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output abab 2 bbbb 3 cccc 4 xxxx 1 --- !query 32 +-- !query SELECT udf(a) FROM test_missing_target ORDER BY udf(upper(udf(d))) --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output 0 1 2 @@ -373,57 +373,57 @@ struct 9 --- !query 33 +-- !query SELECT udf(count(b)) FROM test_missing_target GROUP BY udf((b + 1) / 2) ORDER BY udf((b + 1) / 2) desc --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output 4 3 2 1 --- !query 34 +-- !query SELECT udf(count(udf(x.a))) FROM test_missing_target x, test_missing_target y WHERE udf(x.a) = udf(y.a) GROUP BY udf(b/2) ORDER BY udf(b/2) --- !query 34 schema +-- !query schema struct<> --- !query 34 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 14 --- !query 35 +-- !query SELECT udf(x.b/2), udf(count(udf(x.b))) FROM test_missing_target x, test_missing_target y WHERE udf(x.a) = udf(y.a) GROUP BY udf(x.b/2) ORDER BY udf(x.b/2) --- !query 35 schema +-- !query schema struct --- !query 35 output +-- !query output 0.5 1 1.0 2 1.5 3 2.0 4 --- !query 36 +-- !query SELECT udf(count(udf(b))) FROM test_missing_target x, test_missing_target y WHERE udf(x.a) = udf(y.a) GROUP BY udf(x.b/2) --- !query 36 schema +-- !query schema struct<> --- !query 36 output +-- !query output org.apache.spark.sql.AnalysisException Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 21 --- !query 37 +-- !query DROP TABLE test_missing_target --- !query 37 schema +-- !query schema struct<> --- !query 37 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out index 3d7c64054a6ac..e66948dcdea34 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out @@ -2,27 +2,27 @@ -- Number of queries: 5 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) AS testData(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) FROM testData --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 7 7 0 5 5 4 7 --- !query 2 +-- !query SELECT udf(count(DISTINCT 1)), udf(count(DISTINCT null)), @@ -31,25 +31,25 @@ SELECT udf(count(DISTINCT (a + b))), udf(count(DISTINCT (a, b))) FROM testData --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 0 2 2 2 6 --- !query 3 +-- !query SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*)) FROM testData --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 4 4 4 --- !query 4 +-- !query SELECT udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*)) FROM testData --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 3 3 3 3 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out index 98d3ad37a8dfa..fdddfc55978b4 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out @@ -2,35 +2,35 @@ -- Number of queries: 13 --- !query 0 +-- !query create temporary view nt1 as select * from values ("one", 1), ("two", 2), ("three", 3) as nt1(k, v1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view nt2 as select * from values ("one", 1), ("two", 22), ("one", 5) as nt2(k, v2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM nt1 cross join nt2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output one 1 one 1 one 1 one 5 one 1 two 22 @@ -42,82 +42,82 @@ two 2 one 5 two 2 two 22 --- !query 3 +-- !query SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output one 1 one 1 one 1 one 5 two 2 two 22 --- !query 4 +-- !query SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output one 1 one 1 one 1 one 5 two 2 two 22 --- !query 5 +-- !query SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22" --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output one 1 two 22 --- !query 6 +-- !query SELECT udf(a.key), udf(b.key) FROM (SELECT udf(k) key FROM nt1 WHERE v1 < 2) a CROSS JOIN (SELECT udf(k) key FROM nt2 WHERE v2 = 22) b --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output one two --- !query 7 +-- !query create temporary view A(a, va) as select * from nt1 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query create temporary view B(b, vb) as select * from nt1 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output --- !query 9 +-- !query create temporary view C(c, vc) as select * from nt1 --- !query 9 schema +-- !query schema struct<> --- !query 9 output +-- !query output --- !query 10 +-- !query create temporary view D(d, vd) as select * from nt1 --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output --- !query 11 +-- !query select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output one 1 one 1 one 1 one 1 one 1 one 1 three 3 one 1 one 1 one 1 two 2 one 1 @@ -129,11 +129,11 @@ two 2 two 2 three 3 two 2 two 2 two 2 two 2 two 2 --- !query 12 +-- !query SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output three 3 one 1 three 3 one 5 two 2 one 1 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out index b7bfad0e538ac..2613120e004df 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out @@ -2,25 +2,25 @@ -- Number of queries: 27 --- !query 0 +-- !query CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES (1, 2), (1, 2), @@ -28,13 +28,13 @@ CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES (2, 3), (2, 2) AS tab3(k, v) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES (1, 2), (2, 3), @@ -42,45 +42,45 @@ CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES (2, 2), (2, 20) AS tab4(k, v) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query SELECT udf(c1) FROM tab1 EXCEPT ALL SELECT udf(c1) FROM tab2 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 0 2 2 NULL --- !query 5 +-- !query SELECT udf(c1) FROM tab1 MINUS ALL SELECT udf(c1) FROM tab2 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 0 2 2 NULL --- !query 6 +-- !query SELECT udf(c1) FROM tab1 EXCEPT ALL SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 0 2 2 @@ -88,23 +88,23 @@ NULL NULL --- !query 7 +-- !query SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 EXCEPT ALL SELECT udf(c1) FROM tab2 --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT udf(c1) FROM tab1 EXCEPT ALL SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 0 1 2 @@ -116,13 +116,13 @@ NULL NULL --- !query 9 +-- !query SELECT udf(c1) FROM tab1 EXCEPT ALL SELECT CAST(udf(1) AS BIGINT) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 0 2 2 @@ -133,65 +133,65 @@ NULL NULL --- !query 10 +-- !query SELECT udf(c1) FROM tab1 EXCEPT ALL SELECT array(1) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException ExceptAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; --- !query 11 +-- !query SELECT udf(k), v FROM tab3 EXCEPT ALL SELECT k, udf(v) FROM tab4 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 2 1 3 --- !query 12 +-- !query SELECT k, udf(v) FROM tab4 EXCEPT ALL SELECT udf(k), v FROM tab3 --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 2 2 2 20 --- !query 13 +-- !query SELECT udf(k), udf(v) FROM tab4 EXCEPT ALL SELECT udf(k), udf(v) FROM tab3 INTERSECT DISTINCT SELECT udf(k), udf(v) FROM tab4 --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 2 2 2 20 --- !query 14 +-- !query SELECT udf(k), v FROM tab4 EXCEPT ALL SELECT k, udf(v) FROM tab3 EXCEPT DISTINCT SELECT udf(k), udf(v) FROM tab4 --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output --- !query 15 +-- !query SELECT k, udf(v) FROM tab3 EXCEPT ALL SELECT udf(k), udf(v) FROM tab4 @@ -199,24 +199,24 @@ UNION ALL SELECT udf(k), v FROM tab3 EXCEPT DISTINCT SELECT k, udf(v) FROM tab4 --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 3 --- !query 16 +-- !query SELECT k FROM tab3 EXCEPT ALL SELECT k, v FROM tab4 --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.AnalysisException ExceptAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; --- !query 17 +-- !query SELECT udf(k), udf(v) FROM tab3 EXCEPT ALL SELECT udf(k), udf(v) FROM tab4 @@ -224,13 +224,13 @@ UNION SELECT udf(k), udf(v) FROM tab3 EXCEPT DISTINCT SELECT udf(k), udf(v) FROM tab4 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 3 --- !query 18 +-- !query SELECT udf(k), udf(v) FROM tab3 MINUS ALL SELECT k, udf(v) FROM tab4 @@ -238,13 +238,13 @@ UNION SELECT udf(k), udf(v) FROM tab3 MINUS DISTINCT SELECT k, udf(v) FROM tab4 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 3 --- !query 19 +-- !query SELECT k, udf(v) FROM tab3 EXCEPT ALL SELECT udf(k), v FROM tab4 @@ -252,13 +252,13 @@ EXCEPT DISTINCT SELECT k, udf(v) FROM tab3 EXCEPT DISTINCT SELECT udf(k), v FROM tab4 --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output --- !query 20 +-- !query SELECT * FROM (SELECT tab3.k, udf(tab4.v) @@ -272,13 +272,13 @@ FROM (SELECT udf(tab3.k), FROM tab3 JOIN tab4 ON tab3.k = udf(tab4.k)) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output --- !query 21 +-- !query SELECT * FROM (SELECT udf(udf(tab3.k)), udf(tab4.v) @@ -292,9 +292,9 @@ FROM (SELECT udf(tab4.v) AS k, FROM tab3 JOIN tab4 ON udf(tab3.k) = udf(tab4.k)) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output 1 2 1 2 1 2 @@ -304,43 +304,43 @@ struct --- !query 22 output +-- !query output 3 --- !query 23 +-- !query DROP VIEW IF EXISTS tab1 --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output --- !query 24 +-- !query DROP VIEW IF EXISTS tab2 --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output --- !query 25 +-- !query DROP VIEW IF EXISTS tab3 --- !query 25 schema +-- !query schema struct<> --- !query 25 output +-- !query output --- !query 26 +-- !query DROP VIEW IF EXISTS tab4 --- !query 26 schema +-- !query schema struct<> --- !query 26 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index 0badaf050e194..054ee00ecc2ae 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -2,20 +2,20 @@ -- Number of queries: 9 --- !query 0 +-- !query create temporary view t1 as select * from values ("one", 1), ("two", 2), ("three", 3), ("one", NULL) as t1(k, v) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view t2 as select * from values ("one", 1), ("two", 22), @@ -23,71 +23,71 @@ create temporary view t2 as select * from values ("one", NULL), (NULL, 5) as t2(k, v) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT udf(k), udf(v) FROM t1 EXCEPT SELECT udf(k), udf(v) FROM t2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output three 3 two 2 --- !query 3 +-- !query SELECT * FROM t1 EXCEPT SELECT * FROM t1 where udf(v) <> 1 and v <> udf(2) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output one 1 one NULL two 2 --- !query 4 +-- !query SELECT * FROM t1 where udf(v) <> 1 and v <> udf(22) EXCEPT SELECT * FROM t1 where udf(v) <> 2 and v >= udf(3) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output two 2 --- !query 5 +-- !query SELECT t1.* FROM t1, t2 where t1.k = t2.k EXCEPT SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != udf('one') --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output one 1 one NULL --- !query 6 +-- !query SELECT * FROM t2 where v >= udf(1) and udf(v) <> 22 EXCEPT SELECT * FROM t1 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL 5 one 5 --- !query 7 +-- !query SELECT (SELECT min(udf(k)) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 MINUS SELECT (SELECT udf(min(k)) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = udf('one') --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL two --- !query 8 +-- !query SELECT t1.k FROM t1 WHERE t1.v <= (SELECT udf(max(udf(t2.v))) @@ -99,7 +99,7 @@ FROM t1 WHERE udf(t1.v) >= (SELECT min(udf(t2.v)) FROM t2 WHERE t2.k = t1.k) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output two diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out index de297ab166965..dc291a7696ea7 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out @@ -2,21 +2,21 @@ -- Number of queries: 29 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) AS testData(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT udf(a + b), b, udf(SUM(a - b)) FROM testData GROUP BY udf(a + b), b WITH CUBE --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output 2 1 0 2 NULL 0 3 1 1 @@ -32,11 +32,11 @@ NULL 2 0 NULL NULL 3 --- !query 2 +-- !query SELECT udf(a), udf(b), SUM(b) FROM testData GROUP BY udf(a), b WITH CUBE --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 1 1 1 2 2 1 NULL 3 @@ -51,11 +51,11 @@ NULL 2 6 NULL NULL 9 --- !query 3 +-- !query SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 2 1 0 2 NULL 0 3 1 1 @@ -69,11 +69,11 @@ struct NULL NULL 3 --- !query 4 +-- !query SELECT udf(a), b, udf(SUM(b)) FROM testData GROUP BY udf(a), b WITH ROLLUP --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 1 1 1 1 2 2 1 NULL 3 @@ -86,21 +86,21 @@ struct --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL NULL 113000 Java NULL 50000 Java 2012 20000 @@ -110,11 +110,11 @@ dotNET 2012 15000 dotNET 2013 48000 --- !query 7 +-- !query SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL NULL 113000 NULL 2012 35000 NULL 2013 78000 @@ -126,41 +126,41 @@ dotNET 2012 15000 dotNET 2013 48000 --- !query 8 +-- !query SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output Java NULL 50000 NULL 2012 35000 NULL 2013 78000 dotNET NULL 63000 --- !query 9 +-- !query SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output Java NULL 50000 dotNET NULL 63000 --- !query 10 +-- !query SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output NULL 2012 35000 NULL 2013 78000 --- !query 11 +-- !query SELECT course, udf(SUM(earnings)) AS sum FROM courseSales GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output NULL 113000 Java 20000 Java 30000 @@ -171,12 +171,12 @@ dotNET 48000 dotNET 63000 --- !query 12 +-- !query SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY udf(course), sum --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output NULL 113000 3 Java 20000 0 Java 30000 0 @@ -187,12 +187,12 @@ dotNET 48000 0 dotNET 63000 1 --- !query 13 +-- !query SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output Java 2012 0 0 0 Java 2013 0 0 0 Java NULL 0 1 1 @@ -204,29 +204,29 @@ dotNET 2013 0 0 0 dotNET NULL 0 1 1 --- !query 14 +-- !query SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, udf(year) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException grouping() can only be used with GroupingSets/Cube/Rollup; --- !query 15 +-- !query SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY udf(course), year --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 16 +-- !query SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output Java 2012 0 Java 2013 0 dotNET 2012 0 @@ -238,40 +238,40 @@ NULL 2013 2 NULL NULL 3 --- !query 17 +-- !query SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf(year) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output NULL NULL Java NULL dotNET NULL --- !query 18 +-- !query SELECT course, udf(year) FROM courseSales GROUP BY udf(course), year HAVING GROUPING(course) > 0 --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output org.apache.spark.sql.AnalysisException grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 19 +-- !query SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 20 +-- !query SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output Java NULL NULL 2012 NULL 2013 @@ -279,12 +279,12 @@ NULL NULL dotNET NULL --- !query 21 +-- !query SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) ORDER BY GROUPING(course), GROUPING(year), course, udf(year) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output Java 2012 0 0 Java 2013 0 0 dotNET 2012 0 0 @@ -296,12 +296,12 @@ NULL 2013 1 0 NULL NULL 1 1 --- !query 22 +-- !query SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) ORDER BY GROUPING(course), GROUPING(year), course, udf(year) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output Java 2012 0 Java 2013 0 dotNET 2012 0 @@ -313,29 +313,29 @@ NULL 2013 2 NULL NULL 3 --- !query 23 +-- !query SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course) --- !query 23 schema +-- !query schema struct<> --- !query 23 output +-- !query output org.apache.spark.sql.AnalysisException grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 24 +-- !query SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course) --- !query 24 schema +-- !query schema struct<> --- !query 24 output +-- !query output org.apache.spark.sql.AnalysisException grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; --- !query 25 +-- !query SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output Java 2012 Java 2013 dotNET 2012 @@ -347,11 +347,11 @@ NULL 2013 NULL NULL --- !query 26 +-- !query SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 2 1 0 2 NULL 0 3 1 1 @@ -367,11 +367,11 @@ NULL 2 0 NULL NULL 3 --- !query 27 +-- !query SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 2 1 0 2 NULL 0 3 1 1 @@ -385,10 +385,10 @@ struct NULL NULL 3 --- !query 28 +-- !query SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output NULL 1 3 NULL 2 0 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index a835740a6a86d..6403406413db9 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -2,101 +2,101 @@ -- Number of queries: 52 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) AS testData(a, b) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT udf(a), udf(COUNT(b)) FROM testData --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(CAST(udf(cast(count(b) as string)) AS BIGINT) AS `CAST(udf(cast(count(b) as string)) AS BIGINT)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; --- !query 2 +-- !query SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 7 7 --- !query 3 +-- !query SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 2 2 3 2 NULL 1 --- !query 4 +-- !query SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output org.apache.spark.sql.AnalysisException expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 5 +-- !query SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 0 1 2 2 2 2 3 2 --- !query 6 +-- !query SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1 --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output foo 7 --- !query 7 +-- !query SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output --- !query 9 +-- !query SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) --- !query 9 schema +-- !query schema struct> --- !query 9 output +-- !query output --- !query 10 +-- !query SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 2 1 3 2 4 2 @@ -104,132 +104,132 @@ struct --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 12 +-- !query SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1) --- !query 12 schema +-- !query schema struct<(CAST(udf(cast((a + 1) as string)) AS INT) + 1):int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint> --- !query 12 output +-- !query output 3 2 4 2 5 2 NULL 1 --- !query 13 +-- !query SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) FROM testData --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output -0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 --- !query 14 +-- !query SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output 1 1 --- !query 15 +-- !query SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 1 2 2 2 3 2 NULL 1 --- !query 16 +-- !query SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2 2 3 2 --- !query 17 +-- !query SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException aggregate functions are not allowed in GROUP BY, but found CAST(udf(cast(count(b) as string)) AS BIGINT); --- !query 18 +-- !query CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES (1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) --- !query 18 schema +-- !query schema struct<> --- !query 18 output +-- !query output --- !query 19 +-- !query SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException expression 'testdatahassamenamewithalias.`k`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; --- !query 20 +-- !query set spark.sql.groupByAliases=false --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output spark.sql.groupByAliases false --- !query 21 +-- !query SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 57 --- !query 22 +-- !query SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output --- !query 23 +-- !query SELECT udf(COUNT(1)) FROM testData WHERE false --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 0 --- !query 24 +-- !query SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t --- !query 24 schema +-- !query schema struct<1:int> --- !query 24 output +-- !query output 1 --- !query 25 +-- !query SELECT 1 from ( SELECT 1 AS z, udf(MIN(a.x)) @@ -237,88 +237,88 @@ SELECT 1 from ( WHERE false ) b where b.z != b.z --- !query 25 schema +-- !query schema struct<1:int> --- !query 25 output +-- !query output --- !query 26 +-- !query SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 1.0 1.0 3 --- !query 27 +-- !query SELECT udf(1) FROM range(10) HAVING true --- !query 27 schema +-- !query schema struct --- !query 27 output +-- !query output 1 --- !query 28 +-- !query SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 --- !query 28 schema +-- !query schema struct --- !query 28 output +-- !query output 1 --- !query 29 +-- !query SELECT udf(id) FROM range(10) HAVING id > 0 --- !query 29 schema +-- !query schema struct<> --- !query 29 output +-- !query output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get.; --- !query 30 +-- !query CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (1, true), (1, false), (2, true), (3, false), (3, null), (4, null), (4, null), (5, null), (5, true), (5, false) AS test_agg(k, v) --- !query 30 schema +-- !query schema struct<> --- !query 30 output +-- !query output --- !query 31 +-- !query SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 --- !query 31 schema +-- !query schema struct --- !query 31 output +-- !query output NULL NULL NULL --- !query 32 +-- !query SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 --- !query 32 schema +-- !query schema struct --- !query 32 output +-- !query output NULL NULL NULL --- !query 33 +-- !query SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 --- !query 33 schema +-- !query schema struct --- !query 33 output +-- !query output false true true --- !query 34 +-- !query SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) --- !query 34 schema +-- !query schema struct --- !query 34 output +-- !query output 1 false true true 2 true true true 3 false false false @@ -326,25 +326,25 @@ struct --- !query 35 output +-- !query output 1 false 3 false 5 false --- !query 36 +-- !query SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL --- !query 36 schema +-- !query schema struct --- !query 36 output +-- !query output 4 NULL --- !query 37 +-- !query SELECT udf(k), udf(Every(v)) AS every FROM test_agg @@ -353,13 +353,13 @@ WHERE k = 2 FROM test_agg WHERE k = 1) GROUP BY udf(k) --- !query 37 schema +-- !query schema struct --- !query 37 output +-- !query output 2 true --- !query 38 +-- !query SELECT udf(udf(k)), Every(v) AS every FROM test_agg @@ -368,53 +368,53 @@ WHERE k = 2 FROM test_agg WHERE k = 1) GROUP BY udf(udf(k)) --- !query 38 schema +-- !query schema struct --- !query 38 output +-- !query output --- !query 39 +-- !query SELECT every(udf(1)) --- !query 39 schema +-- !query schema struct<> --- !query 39 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 --- !query 40 +-- !query SELECT some(udf(1S)) --- !query 40 schema +-- !query schema struct<> --- !query 40 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 --- !query 41 +-- !query SELECT any(udf(1L)) --- !query 41 schema +-- !query schema struct<> --- !query 41 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 --- !query 42 +-- !query SELECT udf(every("true")) --- !query 42 schema +-- !query schema struct<> --- !query 42 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11 --- !query 43 +-- !query SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg --- !query 43 schema +-- !query schema struct --- !query 43 output +-- !query output 1 false false 1 true false 2 true true @@ -427,11 +427,11 @@ struct --- !query 44 output +-- !query output 1 false false 1 true true 2 true true @@ -444,11 +444,11 @@ struct --- !query 45 output +-- !query output 1 false false 1 true true 2 true true @@ -461,37 +461,37 @@ struct 1L --- !query 46 schema +-- !query schema struct --- !query 46 output +-- !query output 10 --- !query 47 +-- !query SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true --- !query 47 schema +-- !query schema struct --- !query 47 output +-- !query output 1 true 2 true 5 true --- !query 48 +-- !query SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L --- !query 48 schema +-- !query schema struct --- !query 48 output +-- !query output 10 --- !query 49 +-- !query SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L --- !query 49 schema +-- !query schema struct<> --- !query 49 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -499,11 +499,11 @@ Expression in where clause: [(count(1) > 1L)] Invalid expressions: [count(1)]; --- !query 50 +-- !query SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L --- !query 50 schema +-- !query schema struct<> --- !query 50 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. @@ -511,11 +511,11 @@ Expression in where clause: [((count(1) + 1L) > 1L)] Invalid expressions: [count(1)]; --- !query 51 +-- !query SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 --- !query 51 schema +-- !query schema struct<> --- !query 51 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate/Window/Generate expressions are not valid in where clause of the query. diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out index 1effcc8470e19..9be27bb77f81a 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out @@ -2,48 +2,48 @@ -- Number of queries: 5 --- !query 0 +-- !query create temporary view hav as select * from values ("one", 1), ("two", 2), ("three", 3), ("one", 5) as hav(k, v) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2 --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output one 6 three 3 --- !query 2 +-- !query SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 --- !query 3 +-- !query SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 --- !query 4 +-- !query SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 3 7 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out index 9203c2b31dc11..d78d347bc9802 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out @@ -2,152 +2,152 @@ -- Number of queries: 17 --- !query 0 +-- !query select udf(col1), udf(col2) from values ("one", 1) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output one 1 --- !query 1 +-- !query select udf(col1), udf(udf(col2)) from values ("one", 1) as data --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output one 1 --- !query 2 +-- !query select udf(a), b from values ("one", 1) as data(a, b) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output one 1 --- !query 3 +-- !query select udf(a) from values 1, 2, 3 as data(a) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 3 --- !query 4 +-- !query select udf(a), b from values ("one", 1), ("two", 2), ("three", null) as data(a, b) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output one 1 three NULL two 2 --- !query 5 +-- !query select udf(a), b from values ("one", null), ("two", null) as data(a, b) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output one NULL two NULL --- !query 6 +-- !query select udf(a), b from values ("one", 1), ("two", 2L) as data(a, b) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output one 1 two 2 --- !query 7 +-- !query select udf(udf(a)), udf(b) from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output one 1 two 4 --- !query 8 +-- !query select udf(a), b from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) --- !query 8 schema +-- !query schema struct> --- !query 8 output +-- !query output one [0,1] two [2,3] --- !query 9 +-- !query select udf(a), b from values ("one", 2.0), ("two", 3.0D) as data(a, b) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output one 2.0 two 3.0 --- !query 10 +-- !query select udf(a), b from values ("one", rand(5)), ("two", 3.0D) as data(a, b) --- !query 10 schema +-- !query schema struct<> --- !query 10 output +-- !query output org.apache.spark.sql.AnalysisException cannot evaluate expression rand(5) in inline table definition; line 1 pos 37 --- !query 11 +-- !query select udf(a), udf(b) from values ("one", 2.0), ("two") as data(a, b) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException expected 2 columns but found 1 columns in row 1; line 1 pos 27 --- !query 12 +-- !query select udf(a), udf(b) from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException incompatible types found in column b for inline table; line 1 pos 27 --- !query 13 +-- !query select udf(a), udf(b) from values ("one"), ("two") as data(a, b) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException expected 2 columns but found 1 columns in row 0; line 1 pos 27 --- !query 14 +-- !query select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException Undefined function: 'random_not_exist_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 42 --- !query 15 +-- !query select udf(a), udf(b) from values ("one", count(1)), ("two", 2) as data(a, b) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException cannot evaluate expression count(1) in inline table definition; line 1 pos 42 --- !query 16 +-- !query select udf(a), b from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b) --- !query 16 schema +-- !query schema struct> --- !query 16 output +-- !query output 1991-12-06 00:00:00 [1991-12-06 01:00:00,1991-12-06 12:00:00] diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out index 120f2d39f73dc..107fe9eb2fe55 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out @@ -2,65 +2,65 @@ -- Number of queries: 7 --- !query 0 +-- !query CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE TEMPORARY VIEW ta AS SELECT udf(a) AS a, udf('a') AS tag FROM t1 UNION ALL SELECT udf(a) AS a, udf('b') AS tag FROM t2 --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query CREATE TEMPORARY VIEW tb AS SELECT udf(a) AS a, udf('a') AS tag FROM t3 UNION ALL SELECT udf(a) AS a, udf('b') AS tag FROM t4 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 a 1 a 1 b diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out index 0cb82be2da228..b3735ae153267 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 22 --- !query 0 +-- !query CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES (1, 2), (1, 2), @@ -12,13 +12,13 @@ CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES (null, null), (null, null) AS tab1(k, v) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES (1, 2), (1, 2), @@ -27,19 +27,19 @@ CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES (null, null), (null, null) AS tab2(k, v) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT udf(k), v FROM tab1 INTERSECT ALL SELECT k, udf(v) FROM tab2 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 2 1 2 2 3 @@ -47,80 +47,80 @@ NULL NULL NULL NULL --- !query 3 +-- !query SELECT k, udf(v) FROM tab1 INTERSECT ALL SELECT udf(k), v FROM tab1 WHERE udf(k) = 1 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 1 2 1 3 1 3 --- !query 4 +-- !query SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) INTERSECT ALL SELECT udf(k), udf(v) FROM tab2 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT udf(k), v FROM tab1 INTERSECT ALL SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output --- !query 6 +-- !query SELECT udf(k), v FROM tab1 INTERSECT ALL SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 1 2 --- !query 7 +-- !query SELECT k, udf(v) FROM tab1 INTERSECT ALL SELECT array(1), udf(2) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output org.apache.spark.sql.AnalysisException IntersectAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; --- !query 8 +-- !query SELECT udf(k) FROM tab1 INTERSECT ALL SELECT udf(k), udf(v) FROM tab2 --- !query 8 schema +-- !query schema struct<> --- !query 8 output +-- !query output org.apache.spark.sql.AnalysisException IntersectAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; --- !query 9 +-- !query SELECT udf(k), v FROM tab2 INTERSECT ALL SELECT k, udf(v) FROM tab1 INTERSECT ALL SELECT udf(k), udf(v) FROM tab2 --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 2 1 2 2 3 @@ -128,7 +128,7 @@ NULL NULL NULL NULL --- !query 10 +-- !query SELECT udf(k), v FROM tab1 EXCEPT SELECT k, udf(v) FROM tab2 @@ -136,9 +136,9 @@ UNION ALL SELECT k, udf(udf(v)) FROM tab1 INTERSECT ALL SELECT udf(k), v FROM tab2 --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 1 2 1 2 1 3 @@ -147,7 +147,7 @@ NULL NULL NULL NULL --- !query 11 +-- !query SELECT udf(k), udf(v) FROM tab1 EXCEPT SELECT udf(k), v FROM tab2 @@ -155,13 +155,13 @@ EXCEPT SELECT k, udf(v) FROM tab1 INTERSECT ALL SELECT udf(k), udf(udf(v)) FROM tab2 --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output 1 3 --- !query 12 +-- !query ( ( ( @@ -175,13 +175,13 @@ struct --- !query 12 output +-- !query output --- !query 13 +-- !query SELECT * FROM (SELECT udf(tab1.k), udf(tab2.v) @@ -195,9 +195,9 @@ FROM (SELECT udf(tab1.k), FROM tab1 JOIN tab2 ON udf(tab1.k) = udf(udf(tab2.k))) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output 1 2 1 2 1 2 @@ -209,7 +209,7 @@ struct --- !query 14 output +-- !query output --- !query 15 +-- !query SELECT udf(v) FROM tab1 GROUP BY v INTERSECT ALL SELECT udf(udf(k)) FROM tab2 GROUP BY k --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output 2 3 NULL --- !query 16 +-- !query SET spark.sql.legacy.setopsPrecedence.enabled= true --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output spark.sql.legacy.setopsPrecedence.enabled true --- !query 17 +-- !query SELECT udf(k), v FROM tab1 EXCEPT SELECT k, udf(v) FROM tab2 @@ -257,9 +257,9 @@ UNION ALL SELECT udf(k), udf(v) FROM tab1 INTERSECT ALL SELECT udf(udf(k)), udf(v) FROM tab2 --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output 1 2 1 2 2 3 @@ -267,7 +267,7 @@ NULL NULL NULL NULL --- !query 18 +-- !query SELECT k, udf(v) FROM tab1 EXCEPT SELECT udf(k), v FROM tab2 @@ -275,33 +275,33 @@ UNION ALL SELECT udf(k), udf(v) FROM tab1 INTERSECT SELECT udf(k), udf(udf(v)) FROM tab2 --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 2 2 3 NULL NULL --- !query 19 +-- !query SET spark.sql.legacy.setopsPrecedence.enabled = false --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output spark.sql.legacy.setopsPrecedence.enabled false --- !query 20 +-- !query DROP VIEW IF EXISTS tab1 --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output --- !query 21 +-- !query DROP VIEW IF EXISTS tab2 --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out index e79d01fb14d60..0802eb9a9f62b 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out @@ -2,193 +2,193 @@ -- Number of queries: 24 --- !query 0 +-- !query CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT udf(t1.a), udf(empty_table.a) FROM t1 INNER JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output --- !query 4 +-- !query SELECT udf(t1.a), udf(udf(empty_table.a)) FROM t1 CROSS JOIN empty_table ON (udf(udf(t1.a)) = udf(empty_table.a)) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT udf(udf(t1.a)), empty_table.a FROM t1 LEFT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 1 NULL --- !query 6 +-- !query SELECT udf(t1.a), udf(empty_table.a) FROM t1 RIGHT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output --- !query 7 +-- !query SELECT udf(t1.a), empty_table.a FROM t1 FULL OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 1 NULL --- !query 8 +-- !query SELECT udf(udf(t1.a)) FROM t1 LEFT SEMI JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output --- !query 9 +-- !query SELECT udf(t1.a) FROM t1 LEFT ANTI JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 1 --- !query 10 +-- !query SELECT udf(empty_table.a), udf(t1.a) FROM empty_table INNER JOIN t1 ON (udf(udf(empty_table.a)) = udf(t1.a)) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output --- !query 11 +-- !query SELECT udf(empty_table.a), udf(udf(t1.a)) FROM empty_table CROSS JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) --- !query 11 schema +-- !query schema struct --- !query 11 output +-- !query output --- !query 12 +-- !query SELECT udf(udf(empty_table.a)), udf(t1.a) FROM empty_table LEFT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output --- !query 13 +-- !query SELECT empty_table.a, udf(t1.a) FROM empty_table RIGHT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) --- !query 13 schema +-- !query schema struct --- !query 13 output +-- !query output NULL 1 --- !query 14 +-- !query SELECT empty_table.a, udf(udf(t1.a)) FROM empty_table FULL OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) --- !query 14 schema +-- !query schema struct --- !query 14 output +-- !query output NULL 1 --- !query 15 +-- !query SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) --- !query 15 schema +-- !query schema struct --- !query 15 output +-- !query output --- !query 16 +-- !query SELECT empty_table.a FROM empty_table LEFT ANTI JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output --- !query 17 +-- !query SELECT udf(empty_table.a) FROM empty_table INNER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))) --- !query 17 schema +-- !query schema struct --- !query 17 output +-- !query output --- !query 18 +-- !query SELECT udf(udf(empty_table.a)) FROM empty_table CROSS JOIN empty_table AS empty_table2 ON (udf(udf(empty_table.a)) = udf(empty_table2.a)) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output --- !query 19 +-- !query SELECT udf(empty_table.a) FROM empty_table LEFT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output --- !query 20 +-- !query SELECT udf(udf(empty_table.a)) FROM empty_table RIGHT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))) --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output --- !query 21 +-- !query SELECT udf(empty_table.a) FROM empty_table FULL OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output --- !query 22 +-- !query SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output --- !query 23 +-- !query SELECT udf(empty_table.a) FROM empty_table LEFT ANTI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out index 950809ddcaf25..a8233a0e398b0 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out @@ -2,63 +2,63 @@ -- Number of queries: 6 --- !query 0 +-- !query create temporary view nt1 as select * from values ("one", 1), ("two", 2), ("three", 3) as nt1(k, v1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view nt2 as select * from values ("one", 1), ("two", 22), ("one", 5) as nt2(k, v2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM nt1 natural join nt2 where udf(k) = "one" --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output one 1 1 one 1 5 --- !query 3 +-- !query SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2 --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output one 1 1 one 1 5 two 2 22 three 3 NULL --- !query 4 +-- !query SELECT * FROM nt1 natural right join nt2 where udf(k) <> udf("") order by v1, v2 --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output one 1 1 one 1 5 two 2 22 --- !query 5 +-- !query SELECT udf(count(*)) FROM nt1 natural full outer join nt2 --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 4 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out index 819f786070882..afebbb0c1da92 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out @@ -2,27 +2,27 @@ -- Number of queries: 8 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (-234), (145), (367), (975), (298) as t1(int_col1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) as t2(int_col0, int_col1) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT (udf(SUM(udf(COALESCE(t1.int_col1, t2.int_col0))))), (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) @@ -33,40 +33,40 @@ GROUP BY udf(GREATEST(COALESCE(udf(t2.int_col1), 109), COALESCE(t1.int_col1, udf COALESCE(t1.int_col1, t2.int_col0) HAVING (udf(SUM(COALESCE(udf(t1.int_col1), udf(t2.int_col0))))) > (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output -367 -734 -507 -1014 -769 -1538 -800 -1600 --- !query 3 +-- !query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output --- !query 4 +-- !query CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query set spark.sql.crossJoin.enabled = true --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output spark.sql.crossJoin.enabled true --- !query 6 +-- !query SELECT * FROM ( SELECT @@ -74,15 +74,15 @@ SELECT FROM t1 LEFT JOIN t2 ON false ) t where (udf(t.int_col)) is not null --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 97 --- !query 7 +-- !query set spark.sql.crossJoin.enabled = false --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output spark.sql.crossJoin.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out index cb9e4d736c9a0..087b4ed9302d8 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 30 --- !query 0 +-- !query create temporary view courseSales as select * from values ("dotNET", 2012, 10000), ("Java", 2012, 20000), @@ -10,35 +10,35 @@ create temporary view courseSales as select * from values ("dotNET", 2013, 48000), ("Java", 2013, 30000) as courseSales(course, year, earnings) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query create temporary view years as select * from values (2012, 1), (2013, 2) as years(y, s) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query create temporary view yearsWithComplexTypes as select * from values (2012, array(1, 1), map('1', 1), struct(1, 'a')), (2013, array(2, 2), map('2', 2), struct(2, 'b')) as yearsWithComplexTypes(y, a, m, s) --- !query 2 schema +-- !query schema struct<> --- !query 2 output +-- !query output --- !query 3 +-- !query SELECT * FROM ( SELECT udf(year), course, earnings FROM courseSales ) @@ -46,27 +46,27 @@ PIVOT ( udf(sum(earnings)) FOR course IN ('dotNET', 'Java') ) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 2012 15000 20000 2013 48000 30000 --- !query 4 +-- !query SELECT * FROM courseSales PIVOT ( udf(sum(earnings)) FOR year IN (2012, 2013) ) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output Java 20000 30000 dotNET 15000 48000 --- !query 5 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -74,14 +74,14 @@ PIVOT ( udf(sum(earnings)), udf(avg(earnings)) FOR course IN ('dotNET', 'Java') ) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 2012 15000 7500.0 20000 20000.0 2013 48000 48000.0 30000 30000.0 --- !query 6 +-- !query SELECT * FROM ( SELECT udf(course) as course, earnings FROM courseSales ) @@ -89,13 +89,13 @@ PIVOT ( udf(sum(earnings)) FOR course IN ('dotNET', 'Java') ) --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output 63000 50000 --- !query 7 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -103,13 +103,13 @@ PIVOT ( udf(sum(udf(earnings))), udf(min(year)) FOR course IN ('dotNET', 'Java') ) --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output 63000 2012 50000 2012 --- !query 8 +-- !query SELECT * FROM ( SELECT course, year, earnings, udf(s) as s FROM courseSales @@ -119,16 +119,16 @@ PIVOT ( udf(sum(earnings)) FOR s IN (1, 2) ) --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output Java 2012 20000 NULL Java 2013 NULL 30000 dotNET 2012 15000 NULL dotNET 2013 NULL 48000 --- !query 9 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -138,14 +138,14 @@ PIVOT ( udf(sum(earnings)), udf(min(s)) FOR course IN ('dotNET', 'Java') ) --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output 2012 15000 1 20000 1 2013 48000 2 30000 2 --- !query 10 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -155,14 +155,14 @@ PIVOT ( udf(sum(earnings * s)) FOR course IN ('dotNET', 'Java') ) --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output 2012 15000 20000 2013 96000 60000 --- !query 11 +-- !query SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( SELECT year y, course c, earnings e FROM courseSales ) @@ -170,14 +170,14 @@ PIVOT ( udf(sum(e)) s, udf(avg(e)) a FOR y IN (2012, 2013) ) --- !query 11 schema +-- !query schema struct<2012_s:bigint,2013_s:bigint,2012_a:double,2013_a:double,c:string> --- !query 11 output +-- !query output 15000 48000 7500.0 48000.0 dotNET 20000 30000 20000.0 30000.0 Java --- !query 12 +-- !query SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( SELECT year y, course c, earnings e FROM courseSales ) @@ -185,27 +185,27 @@ PIVOT ( udf(sum(e)) s, udf(avg(e)) a FOR y IN (2012 as firstYear, 2013 secondYear) ) --- !query 12 schema +-- !query schema struct --- !query 12 output +-- !query output 15000 48000 7500.0 48000.0 dotNET 20000 30000 20000.0 30000.0 Java --- !query 13 +-- !query SELECT * FROM courseSales PIVOT ( udf(abs(earnings)) FOR year IN (2012, 2013) ) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate expression required for pivot, but 'coursesales.`earnings`' did not appear in any aggregate function.; --- !query 14 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -213,14 +213,14 @@ PIVOT ( udf(sum(earnings)), year FOR course IN ('dotNET', 'Java') ) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException Aggregate expression required for pivot, but '__auto_generated_subquery_name.`year`' did not appear in any aggregate function.; --- !query 15 +-- !query SELECT * FROM ( SELECT course, earnings FROM courseSales ) @@ -228,14 +228,14 @@ PIVOT ( udf(sum(earnings)) FOR year IN (2012, 2013) ) --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`year`' given input columns: [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 --- !query 16 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -243,14 +243,14 @@ PIVOT ( udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1 FOR course IN ('dotNET', 'Java') ) --- !query 16 schema +-- !query schema struct --- !query 16 output +-- !query output 2012 15000 7501.0 20000 20001.0 2013 48000 48001.0 30000 30001.0 --- !query 17 +-- !query SELECT * FROM ( SELECT year, course, earnings FROM courseSales ) @@ -258,14 +258,14 @@ PIVOT ( sum(udf(avg(earnings))) FOR course IN ('dotNET', 'Java') ) --- !query 17 schema +-- !query schema struct<> --- !query 17 output +-- !query output org.apache.spark.sql.AnalysisException It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query.; --- !query 18 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -275,14 +275,14 @@ PIVOT ( udf(sum(earnings)) FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) ) --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output 1 15000 NULL 2 NULL 30000 --- !query 19 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -292,14 +292,14 @@ PIVOT ( udf(sum(earnings)) FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) ) --- !query 19 schema +-- !query schema struct --- !query 19 output +-- !query output 2012 NULL 20000 2013 48000 NULL --- !query 20 +-- !query SELECT * FROM ( SELECT course, year, earnings, s FROM courseSales @@ -309,40 +309,40 @@ PIVOT ( udf(sum(earnings)) FOR (course, year) IN ('dotNET', 'Java') ) --- !query 20 schema +-- !query schema struct<> --- !query 20 output +-- !query output org.apache.spark.sql.AnalysisException Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct; --- !query 21 +-- !query SELECT * FROM courseSales PIVOT ( udf(sum(earnings)) FOR year IN (s, 2013) ) --- !query 21 schema +-- !query schema struct<> --- !query 21 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '`s`' given input columns: [coursesales.course, coursesales.earnings, coursesales.year]; line 4 pos 15 --- !query 22 +-- !query SELECT * FROM courseSales PIVOT ( udf(sum(earnings)) FOR year IN (course, 2013) ) --- !query 22 schema +-- !query schema struct<> --- !query 22 output +-- !query output org.apache.spark.sql.AnalysisException Literal expressions required for pivot values, found 'course#x'; --- !query 23 +-- !query SELECT * FROM ( SELECT earnings, year, a FROM courseSales @@ -352,14 +352,14 @@ PIVOT ( udf(sum(earnings)) FOR a IN (array(1, 1), array(2, 2)) ) --- !query 23 schema +-- !query schema struct --- !query 23 output +-- !query output 2012 35000 NULL 2013 NULL 78000 --- !query 24 +-- !query SELECT * FROM ( SELECT course, earnings, udf(year) as year, a FROM courseSales @@ -369,14 +369,14 @@ PIVOT ( udf(sum(earnings)) FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) ) --- !query 24 schema +-- !query schema struct --- !query 24 output +-- !query output 2012 15000 NULL 2013 NULL 30000 --- !query 25 +-- !query SELECT * FROM ( SELECT earnings, year, s FROM courseSales @@ -386,14 +386,14 @@ PIVOT ( udf(sum(earnings)) FOR s IN ((1, 'a'), (2, 'b')) ) --- !query 25 schema +-- !query schema struct --- !query 25 output +-- !query output 2012 35000 NULL 2013 NULL 78000 --- !query 26 +-- !query SELECT * FROM ( SELECT course, earnings, year, s FROM courseSales @@ -403,14 +403,14 @@ PIVOT ( udf(sum(earnings)) FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) ) --- !query 26 schema +-- !query schema struct --- !query 26 output +-- !query output 2012 15000 NULL 2013 NULL 30000 --- !query 27 +-- !query SELECT * FROM ( SELECT earnings, year, m FROM courseSales @@ -420,14 +420,14 @@ PIVOT ( udf(sum(earnings)) FOR m IN (map('1', 1), map('2', 2)) ) --- !query 27 schema +-- !query schema struct<> --- !query 27 output +-- !query output org.apache.spark.sql.AnalysisException Invalid pivot column 'm#x'. Pivot columns must be comparable.; --- !query 28 +-- !query SELECT * FROM ( SELECT course, earnings, year, m FROM courseSales @@ -437,14 +437,14 @@ PIVOT ( udf(sum(earnings)) FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) ) --- !query 28 schema +-- !query schema struct<> --- !query 28 output +-- !query output org.apache.spark.sql.AnalysisException Invalid pivot column 'named_struct(course, course#x, m, m#x)'. Pivot columns must be comparable.; --- !query 29 +-- !query SELECT * FROM ( SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y, udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w @@ -454,7 +454,7 @@ PIVOT ( udf(sum(Earnings)) FOR Course IN ('dotNET', 'Java') ) --- !query 29 schema +-- !query schema struct --- !query 29 output +-- !query output a z b y c x d w 63000 50000 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out index 7b2b5dbe578cc..5e5c79172bb7a 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out @@ -2,61 +2,61 @@ -- Number of queries: 6 --- !query 0 +-- !query SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x) --- !query 0 schema +-- !query schema struct --- !query 0 output +-- !query output 1 2 NULL --- !query 1 +-- !query SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output A B NULL --- !query 2 +-- !query SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 2 NaN --- !query 3 +-- !query SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1 2 Infinity --- !query 4 +-- !query SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output -Infinity 1 2 --- !query 5 +-- !query SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x) --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output 0.00000001 0.00000002 0.00000003 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out index f8e5fe6a62f33..19221947b4a88 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out @@ -2,69 +2,69 @@ -- Number of queries: 8 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1), (2), (3), (4) as t1(int_col1) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg' --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT default.myDoubleAvg(udf(int_col1)) as my_avg, udf(default.myDoubleAvg(udf(int_col1))) as my_avg2, udf(default.myDoubleAvg(int_col1)) as my_avg3 from t1 --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 102.5 102.5 102.5 --- !query 3 +-- !query SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 --- !query 4 +-- !query CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' --- !query 4 schema +-- !query schema struct<> --- !query 4 output +-- !query output --- !query 5 +-- !query SELECT default.udaf1(udf(int_col1)) as udaf1, udf(default.udaf1(udf(int_col1))) as udaf2, udf(default.udaf1(int_col1)) as udaf3 from t1 --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output org.apache.spark.sql.AnalysisException Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 94 --- !query 6 +-- !query DROP FUNCTION myDoubleAvg --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query DROP FUNCTION udaf1 --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out index 835abe3d32775..c06c35d34cd74 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out @@ -2,46 +2,46 @@ -- Number of queries: 16 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT udf(c1) as c1, udf(c2) as c2 FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 UNION ALL SELECT udf(c1) as c1, udf(c2) as c2 FROM t1) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 a 1 a 2 b 2 b --- !query 3 +-- !query SELECT udf(c1) as c1, udf(c2) as c2 FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 UNION ALL SELECT udf(c1) as c1, udf(c2) as c2 FROM t2 UNION ALL SELECT udf(c1) as c1, udf(c2) as c2 FROM t2) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1.0 1 1.0 1 1.0 a @@ -50,45 +50,45 @@ struct 2.0 b --- !query 4 +-- !query SELECT udf(udf(a)) as a FROM (SELECT udf(0) a, udf(0) b UNION ALL SELECT udf(SUM(1)) a, udf(CAST(0 AS BIGINT)) b UNION ALL SELECT udf(0) a, udf(0) b) T --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 0 0 1 --- !query 5 +-- !query CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT udf(1) AS x, udf(col) as col FROM (SELECT udf(col) AS col @@ -97,70 +97,70 @@ FROM (SELECT udf(col) AS col UNION ALL SELECT udf(col) FROM p3) T1) T2 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 1 1 1 --- !query 9 +-- !query SELECT map(1, 2), udf('str') as str UNION ALL SELECT map(1, 2, 3, NULL), udf(1) --- !query 9 schema +-- !query schema struct,str:string> --- !query 9 output +-- !query output {1:2,3:null} 1 {1:2} str --- !query 10 +-- !query SELECT array(1, 2), udf('str') as str UNION ALL SELECT array(1, 2, 3, NULL), udf(1) --- !query 10 schema +-- !query schema struct,str:string> --- !query 10 output +-- !query output [1,2,3,null] 1 [1,2] str --- !query 11 +-- !query DROP VIEW IF EXISTS t1 --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query DROP VIEW IF EXISTS t2 --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query DROP VIEW IF EXISTS p1 --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query DROP VIEW IF EXISTS p2 --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query DROP VIEW IF EXISTS p3 --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index 7fed9e147608b..a915c1bd6c717 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 23 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), (1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), @@ -14,18 +14,18 @@ CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (null, null, null, null, null, null), (3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT udf(val), cate, count(val) OVER(PARTITION BY cate ORDER BY udf(val) ROWS CURRENT ROW) FROM testData ORDER BY cate, udf(val) --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output NULL NULL 0 3 NULL 1 NULL a 0 @@ -37,12 +37,12 @@ NULL a 0 3 b 1 --- !query 2 +-- !query SELECT udf(val), cate, sum(val) OVER(PARTITION BY cate ORDER BY udf(val) ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output NULL NULL 3 3 NULL 3 NULL a 1 @@ -54,22 +54,22 @@ NULL a 1 3 b 6 --- !query 3 +-- !query SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY cate ORDER BY udf(val_long) ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data type mismatch: The data type of the upper bound 'bigint' does not match the expected data type 'int'.; line 1 pos 46 --- !query 4 +-- !query SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val RANGE 1 PRECEDING) FROM testData ORDER BY cate, udf(val) --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output NULL NULL 0 3 NULL 1 NULL a 0 @@ -81,12 +81,12 @@ NULL a 0 3 b 2 --- !query 5 +-- !query SELECT val, udf(cate), sum(val) OVER(PARTITION BY udf(cate) ORDER BY val RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY udf(cate), val --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output NULL NULL NULL 3 NULL 3 NULL a NULL @@ -98,12 +98,12 @@ NULL a NULL 3 b 3 --- !query 6 +-- !query SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY udf(cate) ORDER BY val_long RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL NULL NULL 1 NULL 1 1 a 4 @@ -115,12 +115,12 @@ NULL b NULL 2147483650 b 2147483650 --- !query 7 +-- !query SELECT val_double, udf(cate), sum(val_double) OVER(PARTITION BY udf(cate) ORDER BY val_double RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY udf(cate), val_double --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL NULL NULL 1.0 NULL 1.0 1.0 a 4.5 @@ -132,12 +132,12 @@ NULL NULL NULL 100.001 b 100.001 --- !query 8 +-- !query SELECT val_date, udf(cate), max(val_date) OVER(PARTITION BY udf(cate) ORDER BY val_date RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY udf(cate), val_date --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NULL NULL NULL 2017-08-01 NULL 2017-08-01 2017-08-01 a 2017-08-02 @@ -149,13 +149,13 @@ NULL NULL NULL 2020-12-31 b 2020-12-31 --- !query 9 +-- !query SELECT val_timestamp, udf(cate), avg(val_timestamp) OVER(PARTITION BY udf(cate) ORDER BY val_timestamp RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData ORDER BY udf(cate), val_timestamp --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output NULL NULL NULL 2017-07-31 17:00:00 NULL 1.5015456E9 2017-07-31 17:00:00 a 1.5016970666666667E9 @@ -167,12 +167,12 @@ NULL NULL NULL 2020-12-30 16:00:00 b 1.6093728E9 --- !query 10 +-- !query SELECT val, udf(cate), sum(val) OVER(PARTITION BY cate ORDER BY val DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output NULL NULL NULL 3 NULL 3 NULL a NULL @@ -184,62 +184,62 @@ NULL a NULL 3 b 5 --- !query 11 +-- !query SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not follow the lower bound 'unboundedfollowing$()'.; line 1 pos 38 --- !query 12 +-- !query SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 38 --- !query 13 +-- !query SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val), cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY CAST(udf(cast(val as string)) AS INT) ASC NULLS FIRST, testdata.`cate` ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST,cate#x ASC NULLS FIRST; line 1 pos 38 --- !query 14 +-- !query SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY current_timestamp RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'timestamp' used in the order specification does not match the data type 'int' which is used in the range frame.; line 1 pos 38 --- !query 15 +-- !query SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY udf(cate), val --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING' due to data type mismatch: The lower bound of a window frame must be less than or equal to the upper bound; line 1 pos 38 --- !query 16 +-- !query SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val) RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val(val) --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Frame bound value must be a literal.(line 2, pos 30) @@ -250,7 +250,7 @@ RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cat ------------------------------^^^ --- !query 17 +-- !query SELECT udf(val), cate, max(udf(val)) OVER w AS max, min(udf(val)) OVER w AS min, @@ -285,9 +285,9 @@ kurtosis(udf(val_double)) OVER w AS kurtosis FROM testData WINDOW w AS (PARTITION BY udf(cate) ORDER BY udf(val)) ORDER BY cate, udf(val) --- !query 17 schema +-- !query schema struct,collect_set:array,skewness:double,kurtosis:double> --- !query 17 output +-- !query output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL 3 NULL 3 3 3 1 3 3.0 NaN NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NaN 1 0.0 NaN NaN 0.0 [3] [3] NaN NaN NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NaN NaN @@ -299,11 +299,11 @@ NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0. 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 --- !query 18 +-- !query SELECT udf(val), cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output NULL NULL NULL 3 NULL NULL NULL a NULL @@ -315,20 +315,20 @@ NULL a NULL 3 b NULL --- !query 19 +-- !query SELECT udf(val), cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, udf(val) --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException Window function row_number() requires window to be ordered, please add ORDER BY clause. For example SELECT row_number()(value_expr) OVER (PARTITION BY window_partition ORDER BY window_ordering) from table; --- !query 20 +-- !query SELECT udf(val), cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output NULL NULL 13 1.8571428571428572 3 NULL 13 1.8571428571428572 NULL a 13 1.8571428571428572 @@ -340,7 +340,7 @@ NULL a 13 1.8571428571428572 3 b 13 1.8571428571428572 --- !query 21 +-- !query SELECT udf(val), cate, first_value(false) OVER w AS first_value, first_value(true, true) OVER w AS first_value_ignore_null, @@ -351,9 +351,9 @@ last_value(false, false) OVER w AS last_value_contain_null FROM testData WINDOW w AS () ORDER BY cate, val --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL NULL false true false false true false 3 NULL false true false false true false NULL a false true false false true false @@ -365,14 +365,14 @@ NULL a false true false false true false 3 b false true false false true false --- !query 22 +-- !query SELECT udf(cate), sum(val) OVER (w) FROM testData WHERE val is not null WINDOW w AS (PARTITION BY cate ORDER BY val) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL 3 a 2 a 2 diff --git a/sql/core/src/test/resources/sql-tests/results/union.sql.out b/sql/core/src/test/resources/sql-tests/results/union.sql.out index 95a10f0b0a8c9..44002406836a4 100644 --- a/sql/core/src/test/resources/sql-tests/results/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/union.sql.out @@ -2,46 +2,46 @@ -- Number of queries: 16 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2) --- !query 1 schema +-- !query schema struct<> --- !query 1 output +-- !query output --- !query 2 +-- !query SELECT * FROM (SELECT * FROM t1 UNION ALL SELECT * FROM t1) --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output 1 a 1 a 2 b 2 b --- !query 3 +-- !query SELECT * FROM (SELECT * FROM t1 UNION ALL SELECT * FROM t2 UNION ALL SELECT * FROM t2) --- !query 3 schema +-- !query schema struct --- !query 3 output +-- !query output 1.0 1 1.0 1 1.0 a @@ -50,45 +50,45 @@ struct 2.0 b --- !query 4 +-- !query SELECT a FROM (SELECT 0 a, 0 b UNION ALL SELECT SUM(1) a, CAST(0 AS BIGINT) b UNION ALL SELECT 0 a, 0 b) T --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output 0 0 1 --- !query 5 +-- !query CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col) --- !query 5 schema +-- !query schema struct<> --- !query 5 output +-- !query output --- !query 6 +-- !query CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col) --- !query 6 schema +-- !query schema struct<> --- !query 6 output +-- !query output --- !query 7 +-- !query CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col) --- !query 7 schema +-- !query schema struct<> --- !query 7 output +-- !query output --- !query 8 +-- !query SELECT 1 AS x, col FROM (SELECT col AS col @@ -97,70 +97,70 @@ FROM (SELECT col AS col UNION ALL SELECT col FROM p3) T1) T2 --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output 1 1 1 1 --- !query 9 +-- !query SELECT map(1, 2), 'str' UNION ALL SELECT map(1, 2, 3, NULL), 1 --- !query 9 schema +-- !query schema struct,str:string> --- !query 9 output +-- !query output {1:2,3:null} 1 {1:2} str --- !query 10 +-- !query SELECT array(1, 2), 'str' UNION ALL SELECT array(1, 2, 3, NULL), 1 --- !query 10 schema +-- !query schema struct,str:string> --- !query 10 output +-- !query output [1,2,3,null] 1 [1,2] str --- !query 11 +-- !query DROP VIEW IF EXISTS t1 --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output --- !query 12 +-- !query DROP VIEW IF EXISTS t2 --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output --- !query 13 +-- !query DROP VIEW IF EXISTS p1 --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output --- !query 14 +-- !query DROP VIEW IF EXISTS p2 --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output --- !query 15 +-- !query DROP VIEW IF EXISTS p3 --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index f3b9c252b8330..f795374735f59 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -2,7 +2,7 @@ -- Number of queries: 23 --- !query 0 +-- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), (1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), @@ -14,18 +14,18 @@ CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (null, null, null, null, null, null), (3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate) --- !query 0 schema +-- !query schema struct<> --- !query 0 output +-- !query output --- !query 1 +-- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData ORDER BY cate, val --- !query 1 schema +-- !query schema struct --- !query 1 output +-- !query output NULL NULL 0 3 NULL 1 NULL a 0 @@ -37,12 +37,12 @@ NULL a 0 3 b 1 --- !query 2 +-- !query SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 2 schema +-- !query schema struct --- !query 2 output +-- !query output NULL NULL 3 3 NULL 3 NULL a 1 @@ -54,22 +54,22 @@ NULL a 1 3 b 6 --- !query 3 +-- !query SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long --- !query 3 schema +-- !query schema struct<> --- !query 3 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data type mismatch: The data type of the upper bound 'bigint' does not match the expected data type 'int'.; line 1 pos 41 --- !query 4 +-- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData ORDER BY cate, val --- !query 4 schema +-- !query schema struct --- !query 4 output +-- !query output NULL NULL 0 3 NULL 1 NULL a 0 @@ -81,12 +81,12 @@ NULL a 0 3 b 2 --- !query 5 +-- !query SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 5 schema +-- !query schema struct --- !query 5 output +-- !query output NULL NULL NULL 3 NULL 3 NULL a NULL @@ -98,12 +98,12 @@ NULL a NULL 3 b 3 --- !query 6 +-- !query SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long --- !query 6 schema +-- !query schema struct --- !query 6 output +-- !query output NULL NULL NULL 1 NULL 1 1 a 4 @@ -115,12 +115,12 @@ NULL b NULL 2147483650 b 2147483650 --- !query 7 +-- !query SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double --- !query 7 schema +-- !query schema struct --- !query 7 output +-- !query output NULL NULL NULL 1.0 NULL 1.0 1.0 a 4.5 @@ -132,12 +132,12 @@ NULL NULL NULL 100.001 b 100.001 --- !query 8 +-- !query SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date --- !query 8 schema +-- !query schema struct --- !query 8 output +-- !query output NULL NULL NULL 2017-08-01 NULL 2017-08-01 2017-08-01 a 2017-08-02 @@ -149,13 +149,13 @@ NULL NULL NULL 2020-12-31 b 2020-12-31 --- !query 9 +-- !query SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData ORDER BY cate, val_timestamp --- !query 9 schema +-- !query schema struct --- !query 9 output +-- !query output NULL NULL NULL 2017-07-31 17:00:00 NULL 1.5015456E9 2017-07-31 17:00:00 a 1.5016970666666667E9 @@ -167,12 +167,12 @@ NULL NULL NULL 2020-12-30 16:00:00 b 1.6093728E9 --- !query 10 +-- !query SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 10 schema +-- !query schema struct --- !query 10 output +-- !query output NULL NULL NULL 3 NULL 3 NULL a NULL @@ -184,62 +184,62 @@ NULL a NULL 3 b 5 --- !query 11 +-- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 11 schema +-- !query schema struct<> --- !query 11 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not follow the lower bound 'unboundedfollowing$()'.; line 1 pos 33 --- !query 12 +-- !query SELECT val, cate, count(val) OVER(PARTITION BY cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 12 schema +-- !query schema struct<> --- !query 12 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY testdata.`cate` RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 33 --- !query 13 +-- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 13 schema +-- !query schema struct<> --- !query 13 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY testdata.`cate` ORDER BY testdata.`val` ASC NULLS FIRST, testdata.`cate` ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: val#x ASC NULLS FIRST,cate#x ASC NULLS FIRST; line 1 pos 33 --- !query 14 +-- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 14 schema +-- !query schema struct<> --- !query 14 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY testdata.`cate` ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'timestamp' used in the order specification does not match the data type 'int' which is used in the range frame.; line 1 pos 33 --- !query 15 +-- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val --- !query 15 schema +-- !query schema struct<> --- !query 15 output +-- !query output org.apache.spark.sql.AnalysisException cannot resolve 'RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING' due to data type mismatch: The lower bound of a window frame must be less than or equal to the upper bound; line 1 pos 33 --- !query 16 +-- !query SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val --- !query 16 schema +-- !query schema struct<> --- !query 16 output +-- !query output org.apache.spark.sql.catalyst.parser.ParseException Frame bound value must be a literal.(line 2, pos 30) @@ -250,7 +250,7 @@ RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cat ------------------------------^^^ --- !query 17 +-- !query SELECT val, cate, max(val) OVER w AS max, min(val) OVER w AS min, @@ -285,9 +285,9 @@ kurtosis(val_double) OVER w AS kurtosis FROM testData WINDOW w AS (PARTITION BY cate ORDER BY val) ORDER BY cate, val --- !query 17 schema +-- !query schema struct,collect_set:array,skewness:double,kurtosis:double> --- !query 17 output +-- !query output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL 3 NULL 3 3 3 1 3 3.0 NaN NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NaN 1 0.0 NaN NaN 0.0 [3] [3] NaN NaN NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NaN NaN @@ -299,11 +299,11 @@ NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0. 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 --- !query 18 +-- !query SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val --- !query 18 schema +-- !query schema struct --- !query 18 output +-- !query output NULL NULL NULL 3 NULL NULL NULL a NULL @@ -315,20 +315,20 @@ NULL a NULL 3 b NULL --- !query 19 +-- !query SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val --- !query 19 schema +-- !query schema struct<> --- !query 19 output +-- !query output org.apache.spark.sql.AnalysisException Window function row_number() requires window to be ordered, please add ORDER BY clause. For example SELECT row_number()(value_expr) OVER (PARTITION BY window_partition ORDER BY window_ordering) from table; --- !query 20 +-- !query SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val --- !query 20 schema +-- !query schema struct --- !query 20 output +-- !query output NULL NULL 13 1.8571428571428572 3 NULL 13 1.8571428571428572 NULL a 13 1.8571428571428572 @@ -340,7 +340,7 @@ NULL a 13 1.8571428571428572 3 b 13 1.8571428571428572 --- !query 21 +-- !query SELECT val, cate, first_value(false) OVER w AS first_value, first_value(true, true) OVER w AS first_value_ignore_null, @@ -351,9 +351,9 @@ last_value(false, false) OVER w AS last_value_contain_null FROM testData WINDOW w AS () ORDER BY cate, val --- !query 21 schema +-- !query schema struct --- !query 21 output +-- !query output NULL NULL false true false false true false 3 NULL false true false false true false NULL a false true false false true false @@ -365,14 +365,14 @@ NULL a false true false false true false 3 b false true false false true false --- !query 22 +-- !query SELECT cate, sum(val) OVER (w) FROM testData WHERE val is not null WINDOW w AS (PARTITION BY cate ORDER BY val) --- !query 22 schema +-- !query schema struct --- !query 22 output +-- !query output NULL 3 a 2 a 2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 350b43dc94dae..2e5a9e0b4d45d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -87,16 +87,16 @@ import org.apache.spark.tags.ExtendedSQLTest * {{{ * -- some header information * - * -- !query 0 + * -- !query * select 1, -1 - * -- !query 0 schema + * -- !query schema * struct<...schema...> - * -- !query 0 output + * -- !query output * ... data row 1 ... * ... data row 2 ... * ... * - * -- !query 1 + * -- !query * ... * }}} * @@ -159,13 +159,13 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { /** A single SQL query's output. */ protected case class QueryOutput(sql: String, schema: String, output: String) { - def toString(queryIndex: Int): String = { + override def toString: String = { // We are explicitly not using multi-line string due to stripMargin removing "|" in output. - s"-- !query $queryIndex\n" + + s"-- !query\n" + sql + "\n" + - s"-- !query $queryIndex schema\n" + + s"-- !query schema\n" + schema + "\n" + - s"-- !query $queryIndex output\n" + + s"-- !query output\n" + output } } @@ -360,7 +360,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { val goldenOutput = { s"-- Automatically generated by ${getClass.getSimpleName}\n" + s"-- Number of queries: ${outputs.size}\n\n\n" + - outputs.zipWithIndex.map{case (qr, i) => qr.toString(i)}.mkString("\n\n\n") + "\n" + outputs.zipWithIndex.map{case (qr, i) => qr.toString}.mkString("\n\n\n") + "\n" } val resultFile = new File(testCase.resultFile) val parent = resultFile.getParentFile @@ -391,7 +391,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // Read back the golden file. val expectedOutputs: Seq[QueryOutput] = { val goldenOutput = fileToString(new File(testCase.resultFile)) - val segments = goldenOutput.split("-- !query.+\n") + val segments = goldenOutput.split("-- !query.*\n") // each query has 3 segments, plus the header assert(segments.size == outputs.size * 3 + 1, diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 48fedff2150a7..d9ac9ab441f0c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -106,7 +106,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ // Read back the golden file. val expectedOutputs: Seq[QueryOutput] = { val goldenOutput = fileToString(new File(testCase.resultFile)) - val segments = goldenOutput.split("-- !query.+\n") + val segments = goldenOutput.split("-- !query.*\n") // each query has 3 segments, plus the header assert(segments.size == outputs.size * 3 + 1, From d0800fc8e2e71a79bf0f72c3e4bc608ae34053e7 Mon Sep 17 00:00:00 2001 From: Yuchen Huo Date: Sun, 26 Jan 2020 12:59:24 -0800 Subject: [PATCH 1226/1387] [SPARK-30314] Add identifier and catalog information to DataSourceV2Relation ### What changes were proposed in this pull request? Add identifier and catalog information in DataSourceV2Relation so it would be possible to do richer checks in checkAnalysis step. ### Why are the changes needed? In data source v2, table implementations are all customized so we may not be able to get the resolved identifier from tables them selves. Therefore we encode the table and catalog information in DSV2Relation so no external changes are needed to make sure this information is available. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit tests in the following suites: CatalogManagerSuite.scala CatalogV2UtilSuite.scala SupportsCatalogOptionsSuite.scala PlanResolutionSuite.scala Closes #26957 from yuchenhuo/SPARK-30314. Authored-by: Yuchen Huo Signed-off-by: Burak Yavuz --- .../sql/kafka010/KafkaRelationSuite.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 9 ++- .../sql/connector/catalog/CatalogV2Util.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 21 +++++- .../catalog/CatalogV2UtilSuite.scala | 40 ++++++++++ .../apache/spark/sql/DataFrameReader.scala | 16 ++-- .../apache/spark/sql/DataFrameWriter.scala | 22 +++--- .../apache/spark/sql/DataFrameWriterV2.scala | 10 ++- .../spark/sql/execution/CacheManager.scala | 2 +- .../datasources/FallBackFileSourceV2.scala | 3 +- .../datasources/v2/DataSourceV2Strategy.scala | 8 +- .../spark/sql/DataFrameWriterV2Suite.scala | 43 +++++++++++ .../SupportsCatalogOptionsSuite.scala | 75 ++++++++++++++++++- .../connector/TableCapabilityCheckSuite.scala | 32 +++++--- .../command/PlanResolutionSuite.scala | 55 +++++++++++++- .../ParquetPartitionDiscoverySuite.scala | 2 +- .../sql/streaming/FileStreamSinkSuite.scala | 2 +- 17 files changed, 290 insertions(+), 54 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 063e2e2bc8b77..2c022c11950bc 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -624,7 +624,7 @@ class KafkaRelationSuiteV2 extends KafkaRelationSuiteBase { val topic = newTopic() val df = createDF(topic) assert(df.logicalPlan.collect { - case DataSourceV2Relation(_, _, _) => true + case _: DataSourceV2Relation => true }.nonEmpty) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 45547bff8a9d6..15ebf6971d9dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -817,8 +817,8 @@ class Analyzer( case alter @ AlterTable(_, _, u: UnresolvedV2Relation, _) => CatalogV2Util.loadRelation(u.catalog, u.tableName) - .map(rel => alter.copy(table = rel)) - .getOrElse(alter) + .map(rel => alter.copy(table = rel)) + .getOrElse(alter) case u: UnresolvedV2Relation => CatalogV2Util.loadRelation(u.catalog, u.tableName).getOrElse(u) @@ -831,7 +831,8 @@ class Analyzer( expandRelationName(identifier) match { case NonSessionCatalogAndIdentifier(catalog, ident) => CatalogV2Util.loadTable(catalog, ident) match { - case Some(table) => Some(DataSourceV2Relation.create(table)) + case Some(table) => + Some(DataSourceV2Relation.create(table, Some(catalog), Some(ident))) case None => None } case _ => None @@ -921,7 +922,7 @@ class Analyzer( case v1Table: V1Table => v1SessionCatalog.getRelation(v1Table.v1Table) case table => - DataSourceV2Relation.create(table) + DataSourceV2Relation.create(table, Some(catalog), Some(ident)) } val key = catalog.name +: ident.namespace :+ ident.name Option(AnalysisContext.get.relationCache.getOrElseUpdate(key, loaded.orNull)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index 83c15af27ccf0..0fabe4df6c9a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -286,7 +286,7 @@ private[sql] object CatalogV2Util { } def loadRelation(catalog: CatalogPlugin, ident: Identifier): Option[NamedRelation] = { - loadTable(catalog, ident).map(DataSourceV2Relation.create) + loadTable(catalog, ident).map(DataSourceV2Relation.create(_, Some(catalog), Some(ident))) } def isSessionCatalog(catalog: CatalogPlugin): Boolean = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 87d3419e8115f..45d89498f5ae9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelat import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.connector.catalog.{Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, Table, TableCapability} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, Statistics => V2Statistics, SupportsReportStatistics} import org.apache.spark.sql.connector.read.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.connector.write.WriteBuilder @@ -32,12 +32,17 @@ import org.apache.spark.util.Utils * A logical plan representing a data source v2 table. * * @param table The table that this relation represents. + * @param output the output attributes of this relation. + * @param catalog catalogPlugin for the table. None if no catalog is specified. + * @param identifier the identifier for the table. None if no identifier is defined. * @param options The options for this table operation. It's used to create fresh [[ScanBuilder]] * and [[WriteBuilder]]. */ case class DataSourceV2Relation( table: Table, output: Seq[AttributeReference], + catalog: Option[CatalogPlugin], + identifier: Option[Identifier], options: CaseInsensitiveStringMap) extends LeafNode with MultiInstanceRelation with NamedRelation { @@ -137,12 +142,20 @@ case class StreamingDataSourceV2Relation( } object DataSourceV2Relation { - def create(table: Table, options: CaseInsensitiveStringMap): DataSourceV2Relation = { + def create( + table: Table, + catalog: Option[CatalogPlugin], + identifier: Option[Identifier], + options: CaseInsensitiveStringMap): DataSourceV2Relation = { val output = table.schema().toAttributes - DataSourceV2Relation(table, output, options) + DataSourceV2Relation(table, output, catalog, identifier, options) } - def create(table: Table): DataSourceV2Relation = create(table, CaseInsensitiveStringMap.empty) + def create( + table: Table, + catalog: Option[CatalogPlugin], + identifier: Option[Identifier]): DataSourceV2Relation = + create(table, catalog, identifier, CaseInsensitiveStringMap.empty) /** * This is used to transform data source v2 statistics to logical.Statistics. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala new file mode 100644 index 0000000000000..7a9a7f52ff8fd --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/CatalogV2UtilSuite.scala @@ -0,0 +1,40 @@ +/* + * 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.connector.catalog + +import org.mockito.Mockito.{mock, when} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.types.StructType + +class CatalogV2UtilSuite extends SparkFunSuite { + test("Load relation should encode the identifiers for V2Relations") { + val testCatalog = mock(classOf[TableCatalog]) + val ident = mock(classOf[Identifier]) + val table = mock(classOf[Table]) + when(table.schema()).thenReturn(mock(classOf[StructType])) + when(testCatalog.loadTable(ident)).thenReturn(table) + val r = CatalogV2Util.loadRelation(testCatalog, ident) + assert(r.isDefined) + assert(r.get.isInstanceOf[DataSourceV2Relation]) + val v2Relation = r.get.asInstanceOf[DataSourceV2Relation] + assert(v2Relation.catalog.exists(_ == testCatalog)) + assert(v2Relation.identifier.exists(_ == ident)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 1af4931c553ee..b5d7bbca9064d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -195,6 +195,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } DataSource.lookupDataSourceV2(source, sparkSession.sessionState.conf).map { provider => + val catalogManager = sparkSession.sessionState.catalogManager val sessionOptions = DataSourceV2Utils.extractSessionConfigs( source = provider, conf = sparkSession.sessionState.conf) val pathsOption = if (paths.isEmpty) { @@ -206,7 +207,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val finalOptions = sessionOptions ++ extraOptions.toMap ++ pathsOption val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) - val table = provider match { + val (table, catalog, ident) = provider match { case _: SupportsCatalogOptions if userSpecifiedSchema.nonEmpty => throw new IllegalArgumentException( s"$source does not support user specified schema. Please don't specify the schema.") @@ -214,19 +215,22 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val ident = hasCatalog.extractIdentifier(dsOptions) val catalog = CatalogV2Util.getTableProviderCatalog( hasCatalog, - sparkSession.sessionState.catalogManager, + catalogManager, dsOptions) - catalog.loadTable(ident) + (catalog.loadTable(ident), Some(catalog), Some(ident)) case _ => + // TODO: Non-catalog paths for DSV2 are currently not well defined. userSpecifiedSchema match { - case Some(schema) => provider.getTable(dsOptions, schema) - case _ => provider.getTable(dsOptions) + case Some(schema) => (provider.getTable(dsOptions, schema), None, None) + case _ => (provider.getTable(dsOptions), None, None) } } import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { case _: SupportsRead if table.supports(BATCH_READ) => - Dataset.ofRows(sparkSession, DataSourceV2Relation.create(table, dsOptions)) + Dataset.ofRows( + sparkSession, + DataSourceV2Relation.create(table, catalog, ident, dsOptions)) case _ => loadV1Source(paths: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 998ec9ebdff85..c041d14c8b8df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -258,20 +258,20 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val dsOptions = new CaseInsensitiveStringMap(options.asJava) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + val catalogManager = df.sparkSession.sessionState.catalogManager mode match { case SaveMode.Append | SaveMode.Overwrite => - val table = provider match { + val (table, catalog, ident) = provider match { case supportsExtract: SupportsCatalogOptions => val ident = supportsExtract.extractIdentifier(dsOptions) - val sessionState = df.sparkSession.sessionState val catalog = CatalogV2Util.getTableProviderCatalog( - supportsExtract, sessionState.catalogManager, dsOptions) + supportsExtract, catalogManager, dsOptions) - catalog.loadTable(ident) + (catalog.loadTable(ident), Some(catalog), Some(ident)) case tableProvider: TableProvider => val t = tableProvider.getTable(dsOptions) if (t.supports(BATCH_WRITE)) { - t + (t, None, None) } else { // Streaming also uses the data source V2 API. So it may be that the data source // implements v2, but has no v2 implementation for batch writes. In that case, we @@ -280,7 +280,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } - val relation = DataSourceV2Relation.create(table, dsOptions) + val relation = DataSourceV2Relation.create(table, catalog, ident, dsOptions) checkPartitioningMatchesV2Table(table) if (mode == SaveMode.Append) { runCommand(df.sparkSession, "save") { @@ -299,9 +299,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { provider match { case supportsExtract: SupportsCatalogOptions => val ident = supportsExtract.extractIdentifier(dsOptions) - val sessionState = df.sparkSession.sessionState val catalog = CatalogV2Util.getTableProviderCatalog( - supportsExtract, sessionState.catalogManager, dsOptions) + supportsExtract, catalogManager, dsOptions) val location = Option(dsOptions.get("path")).map(TableCatalog.PROP_LOCATION -> _) @@ -419,7 +418,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case _: V1Table => return insertInto(TableIdentifier(ident.name(), ident.namespace().headOption)) case t => - DataSourceV2Relation.create(t) + DataSourceV2Relation.create(t, Some(catalog), Some(ident)) } val command = mode match { @@ -554,12 +553,13 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } val command = (mode, tableOpt) match { - case (_, Some(table: V1Table)) => + case (_, Some(_: V1Table)) => return saveAsTable(TableIdentifier(ident.name(), ident.namespace().headOption)) case (SaveMode.Append, Some(table)) => checkPartitioningMatchesV2Table(table) - AppendData.byName(DataSourceV2Relation.create(table), df.logicalPlan, extraOptions.toMap) + val v2Relation = DataSourceV2Relation.create(table, Some(catalog), Some(ident)) + AppendData.byName(v2Relation, df.logicalPlan, extraOptions.toMap) case (SaveMode.Overwrite, _) => ReplaceTableAsSelect( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index f0758809bd749..f5dd7613d4103 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -158,7 +158,9 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) def append(): Unit = { val append = loadTable(catalog, identifier) match { case Some(t) => - AppendData.byName(DataSourceV2Relation.create(t), logicalPlan, options.toMap) + AppendData.byName( + DataSourceV2Relation.create(t, Some(catalog), Some(identifier)), + logicalPlan, options.toMap) case _ => throw new NoSuchTableException(identifier) } @@ -181,7 +183,8 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val overwrite = loadTable(catalog, identifier) match { case Some(t) => OverwriteByExpression.byName( - DataSourceV2Relation.create(t), logicalPlan, condition.expr, options.toMap) + DataSourceV2Relation.create(t, Some(catalog), Some(identifier)), + logicalPlan, condition.expr, options.toMap) case _ => throw new NoSuchTableException(identifier) } @@ -207,7 +210,8 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) val dynamicOverwrite = loadTable(catalog, identifier) match { case Some(t) => OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(t), logicalPlan, options.toMap) + DataSourceV2Relation.create(t, Some(catalog), Some(identifier)), + logicalPlan, options.toMap) case _ => throw new NoSuchTableException(identifier) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index 75e11abaa3161..413bd7b29cf45 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -270,7 +270,7 @@ class CacheManager extends Logging { case _ => false } - case DataSourceV2Relation(fileTable: FileTable, _, _) => + case DataSourceV2Relation(fileTable: FileTable, _, _, _, _) => refreshFileIndexIfNecessary(fileTable.fileIndex, fs, qualifiedPath) case _ => false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala index 5f6c3e8f7eeed..28a63c26604ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala @@ -33,7 +33,8 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, File */ class FallBackFileSourceV2(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case i @ InsertIntoStatement(d @ DataSourceV2Relation(table: FileTable, _, _), _, _, _, _) => + case i @ + InsertIntoStatement(d @ DataSourceV2Relation(table: FileTable, _, _, _, _), _, _, _, _) => val v1FileFormat = table.fallbackFileFormat.newInstance() val relation = HadoopFsRelation( table.fileIndex, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 448a4354ddd66..8b4b6fb64658a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -232,11 +232,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case desc @ DescribeNamespace(ResolvedNamespace(catalog, ns), extended) => DescribeNamespaceExec(desc.output, catalog, ns, extended) :: Nil - case desc @ DescribeRelation(ResolvedTable(_, _, table), partitionSpec, isExtended) => + case desc @ DescribeRelation(r: ResolvedTable, partitionSpec, isExtended) => if (partitionSpec.nonEmpty) { throw new AnalysisException("DESCRIBE does not support partition for v2 tables.") } - DescribeTableExec(desc.output, table, isExtended) :: Nil + DescribeTableExec(desc.output, r.table, isExtended) :: Nil case DropTable(catalog, ident, ifExists) => DropTableExec(catalog, ident, ifExists) :: Nil @@ -284,8 +284,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case r: ShowCurrentNamespace => ShowCurrentNamespaceExec(r.output, r.catalogManager) :: Nil - case r @ ShowTableProperties(ResolvedTable(_, _, table), propertyKey) => - ShowTablePropertiesExec(r.output, table, propertyKey) :: Nil + case r @ ShowTableProperties(rt: ResolvedTable, propertyKey) => + ShowTablePropertiesExec(r.output, rt.table, propertyKey) :: Nil case _ => Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index 4e6381aea3c31..d49dc58e93ddb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -22,11 +22,15 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} import org.apache.spark.sql.connector.InMemoryTableCatalog import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.spark.util.Utils class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with BeforeAndAfter { @@ -54,6 +58,45 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo spark.sessionState.conf.clear() } + test("DataFrameWriteV2 encode identifiers correctly") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + var plan: LogicalPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.analyzed + + } + override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} + } + spark.listenerManager.register(listener) + + spark.table("source").writeTo("testcat.table_name").append() + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[AppendData]) + checkV2Identifiers(plan.asInstanceOf[AppendData].table) + + spark.table("source").writeTo("testcat.table_name").overwrite(lit(true)) + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[OverwriteByExpression]) + checkV2Identifiers(plan.asInstanceOf[OverwriteByExpression].table) + + spark.table("source").writeTo("testcat.table_name").overwritePartitions() + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[OverwritePartitionsDynamic]) + checkV2Identifiers(plan.asInstanceOf[OverwritePartitionsDynamic].table) + } + + private def checkV2Identifiers( + plan: LogicalPlan, + identifier: String = "table_name", + catalogPlugin: TableCatalog = catalog("testcat")): Unit = { + assert(plan.isInstanceOf[DataSourceV2Relation]) + val v2 = plan.asInstanceOf[DataSourceV2Relation] + assert(v2.identifier.exists(_.name() == identifier)) + assert(v2.catalog.exists(_ == catalogPlugin)) + } + test("Append: basic append") { spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index 0148bb07ee967..cec48bb368aef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -26,13 +26,16 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression} import org.apache.spark.sql.connector.catalog.{Identifier, SupportsCatalogOptions, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, QueryExecutionListener} class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with BeforeAndAfter { @@ -196,11 +199,79 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with assert(e.getMessage.contains("not support user specified schema")) } + test("DataFrameReader creates v2Relation with identifiers") { + sql(s"create table $catalogName.t1 (id bigint) using $format") + val df = load("t1", Some(catalogName)) + checkV2Identifiers(df.logicalPlan) + } + + test("DataFrameWriter creates v2Relation with identifiers") { + sql(s"create table $catalogName.t1 (id bigint) using $format") + + var plan: LogicalPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.analyzed + } + override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} + } + + spark.listenerManager.register(listener) + + try { + // Test append + save("t1", SaveMode.Append, Some(catalogName)) + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[AppendData]) + val appendRelation = plan.asInstanceOf[AppendData].table + checkV2Identifiers(appendRelation) + + // Test overwrite + save("t1", SaveMode.Overwrite, Some(catalogName)) + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[OverwriteByExpression]) + val overwriteRelation = plan.asInstanceOf[OverwriteByExpression].table + checkV2Identifiers(overwriteRelation) + + // Test insert + spark.range(10).write.format(format).insertInto(s"$catalogName.t1") + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[AppendData]) + val insertRelation = plan.asInstanceOf[AppendData].table + checkV2Identifiers(insertRelation) + + // Test saveAsTable append + spark.range(10).write.format(format).mode(SaveMode.Append).saveAsTable(s"$catalogName.t1") + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[AppendData]) + val saveAsTableRelation = plan.asInstanceOf[AppendData].table + checkV2Identifiers(saveAsTableRelation) + } finally { + spark.listenerManager.unregister(listener) + } + } + + private def checkV2Identifiers( + plan: LogicalPlan, + identifier: String = "t1", + catalogPlugin: TableCatalog = catalog(catalogName)): Unit = { + assert(plan.isInstanceOf[DataSourceV2Relation]) + val v2 = plan.asInstanceOf[DataSourceV2Relation] + assert(v2.identifier.exists(_.name() == identifier)) + assert(v2.catalog.exists(_ == catalogPlugin)) + } + private def load(name: String, catalogOpt: Option[String]): DataFrame = { - val dfr = spark.read.format(format).option("name", "t1") + val dfr = spark.read.format(format).option("name", name) catalogOpt.foreach(cName => dfr.option("catalog", cName)) dfr.load() } + + private def save(name: String, mode: SaveMode, catalogOpt: Option[String]): Unit = { + val df = spark.range(10).write.format(format).option("name", name) + catalogOpt.foreach(cName => df.option("catalog", cName)) + df.mode(mode).save() + } } class CatalogSupportingInMemoryTableProvider diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index ce6d56cf84df1..5196ca65276e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.analysis.{AnalysisSuite, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.connector.catalog.{Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, Table, TableCapability, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, TableCapabilityCheck} @@ -37,6 +37,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { + private val emptyMap = CaseInsensitiveStringMap.empty private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = { StreamingRelationV2( TestTableProvider, @@ -53,9 +54,9 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("batch scan: check missing capabilities") { val e = intercept[AnalysisException] { - TableCapabilityCheck.apply(DataSourceV2Relation.create( - CapabilityTable(), - CaseInsensitiveStringMap.empty)) + TableCapabilityCheck.apply( + DataSourceV2Relation.create(CapabilityTable(), None, None, emptyMap) + ) } assert(e.message.contains("does not support batch scan")) } @@ -88,7 +89,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check missing capabilities") { val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(), CaseInsensitiveStringMap.empty), TestRelation) + DataSourceV2Relation.create(CapabilityTable(), None, None, emptyMap), + TestRelation) val exc = intercept[AnalysisException]{ TableCapabilityCheck.apply(plan) @@ -100,7 +102,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("AppendData: check correct capabilities") { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val plan = AppendData.byName( - DataSourceV2Relation.create(CapabilityTable(write), CaseInsensitiveStringMap.empty), + DataSourceV2Relation.create(CapabilityTable(write), None, None, emptyMap), TestRelation) TableCapabilityCheck.apply(plan) @@ -115,7 +117,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create(table, None, None, emptyMap), + TestRelation, Literal(true)) val exc = intercept[AnalysisException]{ @@ -133,7 +136,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(V1_BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create(table, None, None, emptyMap), + TestRelation, Literal(true)) TableCapabilityCheck.apply(plan) @@ -147,7 +151,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create(table, None, None, emptyMap), + TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) val exc = intercept[AnalysisException]{ @@ -162,7 +167,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { Seq(BATCH_WRITE, V1_BATCH_WRITE).foreach { write => val table = CapabilityTable(write, OVERWRITE_BY_FILTER) val plan = OverwriteByExpression.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + DataSourceV2Relation.create(table, None, None, emptyMap), + TestRelation, EqualTo(AttributeReference("x", LongType)(), Literal(5))) TableCapabilityCheck.apply(plan) @@ -175,7 +181,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { CapabilityTable(OVERWRITE_DYNAMIC)).foreach { table => val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation) + DataSourceV2Relation.create(table, None, None, emptyMap), + TestRelation) val exc = intercept[AnalysisException] { TableCapabilityCheck.apply(plan) @@ -188,7 +195,8 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { test("OverwritePartitionsDynamic: check correct capabilities") { val table = CapabilityTable(BATCH_WRITE, OVERWRITE_DYNAMIC) val plan = OverwritePartitionsDynamic.byName( - DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation) + DataSourceV2Relation.create(table, None, None, emptyMap), + TestRelation) TableCapabilityCheck.apply(plan) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 8f17ce7f32c82..70b9b7ec12ea2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -26,11 +26,11 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolvedTable, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, Analyzer, CTESubstitution, EmptyFunctionRegistry, NoSuchTableException, ResolveCatalogs, ResolvedTable, ResolveInlineTables, ResolveSessionCatalog, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedV2Relation} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, SubqueryAlias, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, ShowTableProperties, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.InMemoryTableProvider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCapability, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.execution.datasources.CreateTable @@ -104,6 +104,8 @@ class PlanResolutionSuite extends AnalysisTest { invocation.getArgument[String](0) match { case "testcat" => testCat + case CatalogManager.SESSION_CATALOG_NAME => + v2SessionCatalog case name => throw new CatalogNotFoundException(s"No such catalog: $name") } @@ -139,6 +141,7 @@ class PlanResolutionSuite extends AnalysisTest { // TODO: run the analyzer directly. val rules = Seq( CTESubstitution, + ResolveInlineTables(conf), analyzer.ResolveRelations, new ResolveCatalogs(catalogManager), new ResolveSessionCatalog(catalogManager, conf, _ == Seq("v")), @@ -1072,6 +1075,54 @@ class PlanResolutionSuite extends AnalysisTest { } } + val DSV2ResolutionTests = { + val v2SessionCatalogTable = s"${CatalogManager.SESSION_CATALOG_NAME}.v2Table" + Seq( + ("ALTER TABLE testcat.tab ALTER COLUMN i TYPE bigint", false), + ("ALTER TABLE tab ALTER COLUMN i TYPE bigint", false), + (s"ALTER TABLE $v2SessionCatalogTable ALTER COLUMN i TYPE bigint", true), + ("INSERT INTO TABLE tab VALUES (1)", false), + ("INSERT INTO TABLE testcat.tab VALUES (1)", false), + (s"INSERT INTO TABLE $v2SessionCatalogTable VALUES (1)", true), + ("DESC TABLE tab", false), + ("DESC TABLE testcat.tab", false), + (s"DESC TABLE $v2SessionCatalogTable", true), + ("SHOW TBLPROPERTIES tab", false), + ("SHOW TBLPROPERTIES testcat.tab", false), + (s"SHOW TBLPROPERTIES $v2SessionCatalogTable", true), + ("SELECT * from tab", false), + ("SELECT * from testcat.tab", false), + (s"SELECT * from ${CatalogManager.SESSION_CATALOG_NAME}.v2Table", true) + ) + } + + DSV2ResolutionTests.foreach { case (sql, isSessionCatlog) => + test(s"Data source V2 relation resolution '$sql'") { + val parsed = parseAndResolve(sql, withDefault = true) + val catlogIdent = if (isSessionCatlog) v2SessionCatalog else testCat + val tableIdent = if (isSessionCatlog) "v2Table" else "tab" + parsed match { + case AlterTable(_, _, r: DataSourceV2Relation, _) => + assert(r.catalog.exists(_ == catlogIdent)) + assert(r.identifier.exists(_.name() == tableIdent)) + case Project(_, r: DataSourceV2Relation) => + assert(r.catalog.exists(_ == catlogIdent)) + assert(r.identifier.exists(_.name() == tableIdent)) + case InsertIntoStatement(r: DataSourceV2Relation, _, _, _, _) => + assert(r.catalog.exists(_ == catlogIdent)) + assert(r.identifier.exists(_.name() == tableIdent)) + case DescribeRelation(r: ResolvedTable, _, _) => + assert(r.catalog == catlogIdent) + assert(r.identifier.name() == tableIdent) + case ShowTableProperties(r: ResolvedTable, _) => + assert(r.catalog == catlogIdent) + assert(r.identifier.name() == tableIdent) + case ShowTablePropertiesCommand(t: TableIdentifier, _) => + assert(t.identifier == tableIdent) + } + } + } + test("MERGE INTO TABLE") { def checkResolution( target: LogicalPlan, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 6497a1ceb5c0e..e63929470ce5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -1281,7 +1281,7 @@ class ParquetV2PartitionDiscoverySuite extends ParquetPartitionDiscoverySuite { (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case DataSourceV2Relation(fileTable: FileTable, _, _) => + case DataSourceV2Relation(fileTable: FileTable, _, _, _, _) => assert(fileTable.fileIndex.partitionSpec() === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a matching DataSourceV2Relation, but got:\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 9bce7f3568e81..877965100f018 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -657,7 +657,7 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has // been inferred val table = df.queryExecution.analyzed.collect { - case DataSourceV2Relation(table: FileTable, _, _) => table + case DataSourceV2Relation(table: FileTable, _, _, _, _) => table } assert(table.size === 1) assert(table.head.fileIndex.isInstanceOf[MetadataLogFileIndex]) From 43d9c7e7e57749ee611e0c97781a71a0645b5e9b Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Sun, 26 Jan 2020 15:21:06 -0800 Subject: [PATCH 1227/1387] [SPARK-30640][PYTHON][SQL] Prevent unnecessary copies of data during Arrow to Pandas conversion ### What changes were proposed in this pull request? Prevent unnecessary copies of data during conversion from Arrow to Pandas. ### Why are the changes needed? During conversion of pyarrow data to Pandas, columns are checked for timestamp types and then modified to correct for local timezone. If the data contains no timestamp types, then unnecessary copies of the data can be made. This is most prevalent when checking columns of a pandas DataFrame where each series is assigned back to the DataFrame, regardless if it had timestamps. See https://www.mail-archive.com/devarrow.apache.org/msg17008.html and ARROW-7596 for discussion. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests Closes #27358 from BryanCutler/pyspark-pandas-timestamp-copy-fix-SPARK-30640. Authored-by: Bryan Cutler Signed-off-by: Bryan Cutler --- python/pyspark/sql/pandas/conversion.py | 8 ++++++-- python/pyspark/sql/pandas/serializers.py | 7 +++++-- python/pyspark/sql/pandas/types.py | 16 ---------------- 3 files changed, 11 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql/pandas/conversion.py b/python/pyspark/sql/pandas/conversion.py index f7680ccdc8431..8548cd222bf10 100644 --- a/python/pyspark/sql/pandas/conversion.py +++ b/python/pyspark/sql/pandas/conversion.py @@ -100,7 +100,7 @@ def toPandas(self): # of PyArrow is found, if 'spark.sql.execution.arrow.pyspark.enabled' is enabled. if use_arrow: try: - from pyspark.sql.pandas.types import _check_dataframe_localize_timestamps + from pyspark.sql.pandas.types import _check_series_localize_timestamps import pyarrow batches = self._collect_as_arrow() if len(batches) > 0: @@ -109,7 +109,11 @@ def toPandas(self): # values, but we should use datetime.date to match the behavior with when # Arrow optimization is disabled. pdf = table.to_pandas(date_as_object=True) - return _check_dataframe_localize_timestamps(pdf, timezone) + for field in self.schema: + if isinstance(field.dataType, TimestampType): + pdf[field.name] = \ + _check_series_localize_timestamps(pdf[field.name], timezone) + return pdf else: return pd.DataFrame.from_records([], columns=self.columns) except Exception as e: diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 4bb5b8fb1729c..6f46e92f5a2a4 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -120,14 +120,17 @@ def __init__(self, timezone, safecheck, assign_cols_by_name): def arrow_to_pandas(self, arrow_column): from pyspark.sql.pandas.types import _check_series_localize_timestamps + import pyarrow # If the given column is a date type column, creates a series of datetime.date directly # instead of creating datetime64[ns] as intermediate data to avoid overflow caused by # datetime64[ns] type handling. s = arrow_column.to_pandas(date_as_object=True) - s = _check_series_localize_timestamps(s, self._timezone) - return s + if pyarrow.types.is_timestamp(arrow_column.type): + return _check_series_localize_timestamps(s, self._timezone) + else: + return s def _create_batch(self, series): """ diff --git a/python/pyspark/sql/pandas/types.py b/python/pyspark/sql/pandas/types.py index 81618bd41f58f..d1edf3f9c47c1 100644 --- a/python/pyspark/sql/pandas/types.py +++ b/python/pyspark/sql/pandas/types.py @@ -165,22 +165,6 @@ def _check_series_localize_timestamps(s, timezone): return s -def _check_dataframe_localize_timestamps(pdf, timezone): - """ - Convert timezone aware timestamps to timezone-naive in the specified timezone or local timezone - - :param pdf: pandas.DataFrame - :param timezone: the timezone to convert. if None then use local timezone - :return pandas.DataFrame where any timezone aware columns have been converted to tz-naive - """ - from pyspark.sql.pandas.utils import require_minimum_pandas_version - require_minimum_pandas_version() - - for column, series in pdf.iteritems(): - pdf[column] = _check_series_localize_timestamps(series, timezone) - return pdf - - def _check_series_convert_timestamps_internal(s, timezone): """ Convert a tz-naive timestamp in the specified timezone or local timezone to UTC normalized for From 7e1b991d12aae42c8c3652d68a852f439c2043b4 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Mon, 27 Jan 2020 08:37:42 -0600 Subject: [PATCH 1228/1387] [SPARK-30581][DOC] Document SORT BY Clause of SELECT statement in SQLReference ### What changes were proposed in this pull request? Document SORT BY clause of SELECT statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2020-01-20 at 1 25 57 AM Screen Shot 2020-01-20 at 1 26 11 AM Screen Shot 2020-01-20 at 1 26 28 AM Screen Shot 2020-01-20 at 1 26 46 AM Screen Shot 2020-01-20 at 1 27 02 AM ### How was this patch tested? Tested using jykyll build --serve Closes #27289 from dilipbiswal/sql-ref-select-sortby. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/sql-ref-syntax-qry-select-sortby.md | 176 +++++++++++++++++++++++ 1 file changed, 176 insertions(+) create mode 100644 docs/sql-ref-syntax-qry-select-sortby.md diff --git a/docs/sql-ref-syntax-qry-select-sortby.md b/docs/sql-ref-syntax-qry-select-sortby.md new file mode 100644 index 0000000000000..ee2e006a79c5a --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-sortby.md @@ -0,0 +1,176 @@ +--- +layout: global +title: SORT BY Clause +displayTitle: SORT BY Clause +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- +The SORT BY clause is used to return the result rows sorted +within each partition in the user specified order. When there is more than one partition +SORT BY may return result that is partially ordered. This is different +than ORDER BY clause which guarantees a total order of the output. + +### Syntax +{% highlight sql %} +SORT BY { expression [ sort_direction | nulls_sort_order ] [ , ... ] } +{% endhighlight %} + +### Parameters +
    +
    SORT BY
    +
    + Specifies a comma-separated list of expressions along with optional parameters sort_direction + and nulls_sort_order which are used to sort the rows within each partition. +
    +
    sort_direction
    +
    + Optionally specifies whether to sort the rows in ascending or descending + order. The valid values for the sort direction are ASC for ascending + and DESC for descending. If sort direction is not explicitly specified, then by default + rows are sorted ascending.

    + Syntax: + + [ ASC | DESC ] + +
    +
    nulls_sort_order
    +
    + Optionally specifies whether NULL values are returned before/after non-NULL values, based on the + sort direction. In Spark, NULL values are considered to be lower than any non-NULL values by default. + Therefore the ordering of NULL values depend on the sort direction. If null_sort_order is + not specified, then NULLs sort first if sort order is ASC and NULLS sort last if + sort order is DESC.

    +
      +
    1. If NULLS FIRST (the default) is specified, then NULL values are returned first + regardless of the sort order.
    2. +
    3. If NULLS LAST is specified, then NULL values are returned last regardless of + the sort order.
    4. +

    + Syntax: + + [ NULLS { FIRST | LAST } ] + +
    +
    + +### Examples +{% highlight sql %} +CREATE TABLE person (zip_code INT, name STRING, age INT); +INSERT INTO person VALUES + (94588, 'Zen Hui', 50), + (94588, 'Dan Li', 18), + (94588, 'Anil K', 27), + (94588, 'John V', NULL), + (94511, 'David K', 42), + (94511, 'Aryan B.', 18), + (94511, 'Lalit B.', NULL); + +-- Use `REPARTITION` hint to partition the data by `zip_code` to +-- examine the `SORT BY` behavior. This is used in rest of the +-- examples. + +-- Sort rows by `name` within each partition in ascending manner +SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person SORT BY name; + + +--------+----+--------+ + |name |age |zip_code| + +--------+----+--------+ + |Anil K |27 |94588 | + |Dan Li |18 |94588 | + |John V |null|94588 | + |Zen Hui |50 |94588 | + |Aryan B.|18 |94511 | + |David K |42 |94511 | + |Lalit B.|null|94511 | + +--------+----+--------+ + +-- Sort rows within each partition using column position. +SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person SORT BY 1; + + +--------+----+--------+ + |name |age |zip_code| + +--------+----+--------+ + |Anil K |27 |94588 | + |Dan Li |18 |94588 | + |John V |null|94588 | + |Zen Hui |50 |94588 | + |Aryan B.|18 |94511 | + |David K |42 |94511 | + |Lalit B.|null|94511 | + +--------+----+--------+ + +-- Sort rows within partition in ascending manner keeping null values to be last. +SELECT /*+ REPARTITION(zip_code) */ age, name, zip_code FROM person SORT BY age NULLS LAST; + + +----+--------+--------+ + |age |name |zip_code| + +----+--------+--------+ + |18 |Dan Li |94588 | + |27 |Anil K |94588 | + |50 |Zen Hui |94588 | + |null|John V |94588 | + |18 |Aryan B.|94511 | + |42 |David K |94511 | + |null|Lalit B.|94511 | + +----+--------+--------+ + +-- Sort rows by age within each partition in descending manner. +SELECT /*+ REPARTITION(zip_code) */ age, name, zip_code FROM person SORT BY age DESC; + + +----+--------+--------+ + |age |name |zip_code| + +----+--------+--------+ + |50 |Zen Hui |94588 | + |27 |Anil K |94588 | + |18 |Dan Li |94588 | + |null|John V |94588 | + |42 |David K |94511 | + |18 |Aryan B.|94511 | + |null|Lalit B.|94511 | + +----+--------+--------+ + +-- Sort rows by age within each partition in ascending manner keeping null values to be first. +SELECT /*+ REPARTITION(zip_code) */ age, name, zip_code FROM person SORT BY age DESC NULLS FIRST; + + +----+--------+--------+ + |age |name |zip_code| + +----+--------+--------+ + |null|John V |94588 | + |50 |Zen Hui |94588 | + |27 |Anil K |94588 | + |18 |Dan Li |94588 | + |null|Lalit B.|94511 | + |42 |David K |94511 | + |18 |Aryan B.|94511 | + +----+--------+--------+ + +-- Sort rows within each partition based on more than one column with each column having +-- different sort direction. +SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person + SORT BY name ASC, age DESC; + + +--------+----+--------+ + |name |age |zip_code| + +--------+----+--------+ + |Anil K |27 |94588 | + |Dan Li |18 |94588 | + |John V |null|94588 | + |Zen Hui |50 |94588 | + |Aryan B.|18 |94511 | + |David K |42 |94511 | + |Lalit B.|null|94511 | + +--------+----+--------+ +{% endhighlight %} From 5781e57127fd2f4dad0c92f00a1491fd03011b81 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Mon, 27 Jan 2020 08:58:45 -0600 Subject: [PATCH 1229/1387] [SPARK-30589][DOC] Document DISTRIBUTE BY Clause of SELECT statement in SQL Reference ### What changes were proposed in this pull request? Document DISTRIBUTE BY clause of SELECT statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2020-01-20 at 3 08 24 PM Screen Shot 2020-01-20 at 3 08 34 PM ### How was this patch tested? Tested using jykyll build --serve Closes #27298 from dilipbiswal/sql-ref-select-distributeby. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- ...sql-ref-syntax-qry-select-distribute-by.md | 84 +++++++++++++++++++ 1 file changed, 84 insertions(+) create mode 100644 docs/sql-ref-syntax-qry-select-distribute-by.md diff --git a/docs/sql-ref-syntax-qry-select-distribute-by.md b/docs/sql-ref-syntax-qry-select-distribute-by.md new file mode 100644 index 0000000000000..a1b3fcbfb5993 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-distribute-by.md @@ -0,0 +1,84 @@ +--- +layout: global +title: DISTRIBUTE BY Clause +displayTitle: DISTRIBUTE BY Clause +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- +The DISTRIBUTE BY clause is used to repartition the data based +on the input expressions. Unlike the `CLUSTER BY` clause, this does not +sort the data within each partition. + +### Syntax +{% highlight sql %} +DISTRIBUTE BY { expression [ , ... ] } +{% endhighlight %} + +### Parameters +
    +
    expression
    +
    + Specifies combination of one or more values, operators and SQL functions that results in a value. +
    +
    + +### Examples +{% highlight sql %} +CREATE TABLE person (name STRING, age INT); +INSERT INTO person VALUES + ('Zen Hui', 25), + ('Anil B', 18), + ('Shone S', 16), + ('Mike A', 25), + ('John A', 18), + ('Jack N', 16); + +-- Reduce the number of shuffle partitions to 2 to illustrate the behavior of `DISTRIBUTE BY`. +-- It's easier to see the clustering and sorting behavior with less number of partitions. +SET spark.sql.shuffle.partitions = 2; + +-- Select the rows with no ordering. Please note that without any sort directive, the result +-- of the query is not deterministic. It's included here to just contrast it with the +-- behavior of `DISTRIBUTE BY`. The query below produces rows where age columns are not +-- clustered together. +SELECT age, name FROM person; + + +---+-------+ + |age|name | + +---+-------+ + |16 |Shone S| + |25 |Zen Hui| + |16 |Jack N | + |25 |Mike A | + |18 |John A | + |18 |Anil B | + +---+-------+ + +-- Produces rows clustered by age. Persons with same age are clustered together. +-- Unlike `CLUSTER BY` clause, the rows are not sorted within a partition. +SELECT age, name FROM person DISTRIBUTE BY age; + + +---+-------+ + |age|name | + +---+-------+ + |25 |Zen Hui| + |25 |Mike A | + |18 |John A | + |18 |Anil B | + |16 |Shone S| + |16 |Jack N | + +---+-------+ +{% endhighlight %} From 8a24cf2bfe409e0e964404c3f217fe9733af5553 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Mon, 27 Jan 2020 08:59:48 -0600 Subject: [PATCH 1230/1387] [SPARK-30588][DOC] Document CLUSTER BY Clause of SELECT statement in SQL Reference ### What changes were proposed in this pull request? Document CLUSTER BY clause of SELECT statement in SQL Reference Guide. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. **Before:** There was no documentation for this. **After.** Screen Shot 2020-01-20 at 2 59 05 PM Screen Shot 2020-01-20 at 2 59 19 PM ### How was this patch tested? Tested using jykyll build --serve Closes #27297 from dilipbiswal/sql-ref-select-clusterby. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/sql-ref-syntax-qry-select-clusterby.md | 88 +++++++++++++++++++++ 1 file changed, 88 insertions(+) create mode 100644 docs/sql-ref-syntax-qry-select-clusterby.md diff --git a/docs/sql-ref-syntax-qry-select-clusterby.md b/docs/sql-ref-syntax-qry-select-clusterby.md new file mode 100644 index 0000000000000..4e59a3e55ae3c --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-clusterby.md @@ -0,0 +1,88 @@ +--- +layout: global +title: CLUSTER BY Clause +displayTitle: CLUSTER BY Clause +license: | + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +--- +The CLUSTER BY clause is used to first repartition the data based +on the input expressions and then sort the data within each partition. This is +semantically equivalent to performing a DISTRIBUTE BY followed by +a SORT BY. This clause only ensures that the resultant rows are +sorted within each partition and does not guarantee a total order of output. + +### Syntax +{% highlight sql %} +CLUSTER BY { expression [ , ... ] } +{% endhighlight %} + +### Parameters +
    +
    expression
    +
    + Specifies combination of one or more values, operators and SQL functions that results in a value. +
    +
    + +### Examples +{% highlight sql %} +CREATE TABLE person (name STRING, age INT); +INSERT INTO person VALUES + ('Zen Hui', 25), + ('Anil B', 18), + ('Shone S', 16), + ('Mike A', 25), + ('John A', 18), + ('Jack N', 16); + +-- Reduce the number of shuffle partitions to 2 to illustrate the behavior of `CLUSTER BY`. +-- It's easier to see the clustering and sorting behavior with less number of partitions. +SET spark.sql.shuffle.partitions = 2; + +-- Select the rows with no ordering. Please note that without any sort directive, the results +-- of the query is not deterministic. It's included here to show the difference in behavior +-- of a query when `CLUSTER BY` is not used vs when it's used. The query below produces rows +-- where age column is not sorted. +SELECT age, name FROM person; + + +---+-------+ + |age|name | + +---+-------+ + |16 |Shone S| + |25 |Zen Hui| + |16 |Jack N | + |25 |Mike A | + |18 |John A | + |18 |Anil B | + +---+-------+ + +-- Produces rows clustered by age. Persons with same age are clustered together. +-- In the query below, persons with age 18 and 25 are in first partition and the +-- persons with age 16 are in the second partition. The rows are sorted based +-- on age within each partition. +SELECT age, name FROM person CLUSTER BY age; + + +---+-------+ + |age|name | + +---+-------+ + |18 |John A | + |18 |Anil B | + |25 |Zen Hui| + |25 |Mike A | + |16 |Shone S| + |16 |Jack N | + +---+-------+ +{% endhighlight %} From 0436b3d3f86ebcbdb5352a395a3cf4f220a5c93c Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 27 Jan 2020 10:20:51 -0800 Subject: [PATCH 1231/1387] [SPARK-30653][INFRA][SQL] EOL character enforcement for java/scala/xml/py/R files ### What changes were proposed in this pull request? This patch converts CR/LF into LF in 3 source files, which most files are only using LF. This patch also add rules to enforce EOL as LF for all java, scala, xml, py, R files. ### Why are the changes needed? The majority of source code files are using LF and only three files are CR/LF. While using IDE would let us don't bother with the difference, it still has a chance to make unnecessary diff if the file is modified with the editor which doesn't handle it automatically. ### Does this PR introduce any user-facing change? No ### How was this patch tested? ``` grep -IUrl --color "^M" . | grep "\.java\|\.scala\|\.xml\|\.py\|\.R" | grep -v "/target/" | grep -v "/build/" | grep -v "/dist/" | grep -v "dependency-reduced-pom.xml" | grep -v ".pyc" ``` (Please note you'll need to type CTRL+V -> CTRL+M in bash shell to get `^M` because it's representing CR/LF, not a combination of `^` and `M`.) Before the patch, the result is: ``` ./sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java ./sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala ``` and after the patch, the result is None. And git shows WARNING message if EOL of any of source files in given types are modified to CR/LF, like below: ``` warning: CRLF will be replaced by LF in sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala. The file will have its original line endings in your working directory. ``` Closes #27365 from HeartSaVioR/MINOR-remove-CRLF-in-source-codes. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .gitattributes | 5 + .../sql/catalyst/optimizer/ComplexTypes.scala | 128 +-- .../optimizer/complexTypesSuite.scala | 910 +++++++++--------- .../execution/columnar/ColumnDictionary.java | 116 +-- 4 files changed, 582 insertions(+), 577 deletions(-) diff --git a/.gitattributes b/.gitattributes index 2b65f6fe3cc80..e2211a2af515e 100644 --- a/.gitattributes +++ b/.gitattributes @@ -1,2 +1,7 @@ *.bat text eol=crlf *.cmd text eol=crlf +*.java text eol=lf +*.scala text eol=lf +*.xml text eol=lf +*.py text eol=lf +*.R text eol=lf diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala index 1743565ccb6c1..28dc8e9d0d5f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ComplexTypes.scala @@ -1,64 +1,64 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.optimizer - -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} -import org.apache.spark.sql.catalyst.rules.Rule - -/** - * Simplify redundant [[CreateNamedStruct]], [[CreateArray]] and [[CreateMap]] expressions. - */ -object SimplifyExtractValueOps extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = plan transform { - // One place where this optimization is invalid is an aggregation where the select - // list expression is a function of a grouping expression: - // - // SELECT struct(a,b).a FROM tbl GROUP BY struct(a,b) - // - // cannot be simplified to SELECT a FROM tbl GROUP BY struct(a,b). So just skip this - // optimization for Aggregates (although this misses some cases where the optimization - // can be made). - case a: Aggregate => a - case p => p.transformExpressionsUp { - // Remove redundant field extraction. - case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) => - createNamedStruct.valExprs(ordinal) - - // Remove redundant array indexing. - case GetArrayStructFields(CreateArray(elems), field, ordinal, _, _) => - // Instead of selecting the field on the entire array, select it from each member - // of the array. Pushing down the operation this way may open other optimizations - // opportunities (i.e. struct(...,x,...).x) - CreateArray(elems.map(GetStructField(_, ordinal, Some(field.name)))) - - // Remove redundant map lookup. - case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) => - // Instead of creating the array and then selecting one row, remove array creation - // altogether. - if (idx >= 0 && idx < elems.size) { - // valid index - elems(idx) - } else { - // out of bounds, mimic the runtime behavior and return null - Literal(null, ga.dataType) - } - case GetMapValue(CreateMap(elems), key) => CaseKeyWhen(key, elems) - } - } -} +/* + * 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.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * Simplify redundant [[CreateNamedStruct]], [[CreateArray]] and [[CreateMap]] expressions. + */ +object SimplifyExtractValueOps extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // One place where this optimization is invalid is an aggregation where the select + // list expression is a function of a grouping expression: + // + // SELECT struct(a,b).a FROM tbl GROUP BY struct(a,b) + // + // cannot be simplified to SELECT a FROM tbl GROUP BY struct(a,b). So just skip this + // optimization for Aggregates (although this misses some cases where the optimization + // can be made). + case a: Aggregate => a + case p => p.transformExpressionsUp { + // Remove redundant field extraction. + case GetStructField(createNamedStruct: CreateNamedStruct, ordinal, _) => + createNamedStruct.valExprs(ordinal) + + // Remove redundant array indexing. + case GetArrayStructFields(CreateArray(elems), field, ordinal, _, _) => + // Instead of selecting the field on the entire array, select it from each member + // of the array. Pushing down the operation this way may open other optimizations + // opportunities (i.e. struct(...,x,...).x) + CreateArray(elems.map(GetStructField(_, ordinal, Some(field.name)))) + + // Remove redundant map lookup. + case ga @ GetArrayItem(CreateArray(elems), IntegerLiteral(idx)) => + // Instead of creating the array and then selecting one row, remove array creation + // altogether. + if (idx >= 0 && idx < elems.size) { + // valid index + elems(idx) + } else { + // out of bounds, mimic the runtime behavior and return null + Literal(null, ga.dataType) + } + case GetMapValue(CreateMap(elems), key) => CaseKeyWhen(key, elems) + } + } +} 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 5452e72b38647..d55746002783a 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 @@ -1,455 +1,455 @@ -/* - * 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._ -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 -import org.apache.spark.sql.catalyst.util.GenericArrayData -import org.apache.spark.sql.types._ - -/** -* SPARK-18601 discusses simplification direct access to complex types creators. -* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be simplified to {{{`x` * `x`}}}. -* sam applies to create_array and create_map -*/ -class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { - - object Optimizer extends RuleExecutor[LogicalPlan] { - val batches = - Batch("collapse projections", FixedPoint(10), - CollapseProject) :: - Batch("Constant Folding", FixedPoint(10), - NullPropagation, - ConstantFolding, - BooleanSimplification, - SimplifyConditionals, - SimplifyBinaryComparison, - SimplifyExtractValueOps) :: Nil - } - - private val idAtt = ('id).long.notNull - private val nullableIdAtt = ('nullable_id).long - - private val relation = LocalRelation(idAtt, nullableIdAtt) - private val testRelation = LocalRelation('a.int, 'b.int, 'c.int, 'd.double, 'e.int) - - private def checkRule(originalQuery: LogicalPlan, correctAnswer: LogicalPlan) = { - val optimized = Optimizer.execute(originalQuery.analyze) - assert(optimized.resolved, "optimized plans must be still resolvable") - comparePlans(optimized, correctAnswer.analyze) - } - - test("explicit get from namedStruct") { - val query = relation - .select( - GetStructField( - CreateNamedStruct(Seq("att", 'id )), - 0, - None) as "outerAtt") - val expected = relation.select('id as "outerAtt") - - checkRule(query, expected) - } - - test("explicit get from named_struct- expression maintains original deduced alias") { - val query = relation - .select(GetStructField(CreateNamedStruct(Seq("att", 'id)), 0, None)) - - val expected = relation - .select('id as "named_struct(att, id).att") - - checkRule(query, expected) - } - - test("collapsed getStructField ontop of namedStruct") { - val query = relation - .select(CreateNamedStruct(Seq("att", 'id)) as "struct1") - .select(GetStructField('struct1, 0, None) as "struct1Att") - val expected = relation.select('id as "struct1Att") - checkRule(query, expected) - } - - test("collapse multiple CreateNamedStruct/GetStructField pairs") { - val query = relation - .select( - CreateNamedStruct(Seq( - "att1", 'id, - "att2", 'id * 'id)) as "struct1") - .select( - GetStructField('struct1, 0, None) as "struct1Att1", - GetStructField('struct1, 1, None) as "struct1Att2") - - val expected = - relation. - select( - 'id as "struct1Att1", - ('id * 'id) as "struct1Att2") - - checkRule(query, expected) - } - - test("collapsed2 - deduced names") { - val query = relation - .select( - CreateNamedStruct(Seq( - "att1", 'id, - "att2", 'id * 'id)) as "struct1") - .select( - GetStructField('struct1, 0, None), - GetStructField('struct1, 1, None)) - - val expected = - relation. - select( - 'id as "struct1.att1", - ('id * 'id) as "struct1.att2") - - checkRule(query, expected) - } - - test("simplified array ops") { - val rel = relation.select( - CreateArray(Seq( - CreateNamedStruct(Seq( - "att1", 'id, - "att2", 'id * 'id)), - CreateNamedStruct(Seq( - "att1", 'id + 1, - "att2", ('id + 1) * ('id + 1)) - )) - ) as "arr" - ) - val query = rel - .select( - GetArrayStructFields('arr, StructField("att1", LongType, false), 0, 1, false) as "a1", - GetArrayItem('arr, 1) as "a2", - GetStructField(GetArrayItem('arr, 1), 0, None) as "a3", - GetArrayItem( - GetArrayStructFields('arr, - StructField("att1", LongType, false), - 0, - 1, - false), - 1) as "a4") - - val expected = relation - .select( - CreateArray(Seq('id, 'id + 1L)) as "a1", - CreateNamedStruct(Seq( - "att1", ('id + 1L), - "att2", (('id + 1L) * ('id + 1L)))) as "a2", - ('id + 1L) as "a3", - ('id + 1L) as "a4") - checkRule(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.inlinedMutableStates.length == 0) - } - - test("SPARK-23208: Test code splitting for create array related methods") { - val inputs = (1 to 2500).map(x => Literal(s"l_$x")) - checkEvaluation(CreateArray(inputs), new GenericArrayData(inputs.map(_.eval()))) - } - - test("simplify map ops") { - val rel = relation - .select( - CreateMap(Seq( - "r1", CreateNamedStruct(Seq("att1", 'id)), - "r2", CreateNamedStruct(Seq("att1", ('id + 1L))))) as "m") - val query = rel - .select( - GetMapValue('m, "r1") as "a1", - GetStructField(GetMapValue('m, "r1"), 0, None) as "a2", - GetMapValue('m, "r32") as "a3", - GetStructField(GetMapValue('m, "r32"), 0, None) as "a4") - - val expected = - relation.select( - CreateNamedStruct(Seq("att1", 'id)) as "a1", - 'id as "a2", - Literal.create( - null, - StructType( - StructField("att1", LongType, nullable = false) :: Nil - ) - ) as "a3", - Literal.create(null, LongType) as "a4") - checkRule(query, expected) - } - - test("simplify map ops, constant lookup, dynamic keys") { - val query = relation.select( - GetMapValue( - CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - Literal(13L), 'id, - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), - 13L) as "a") - - val expected = relation - .select( - CaseWhen(Seq( - (EqualTo(13L, 'id), ('id + 1L)), - (EqualTo(13L, ('id + 1L)), ('id + 2L)), - (EqualTo(13L, ('id + 2L)), ('id + 3L)), - (Literal(true), 'id))) as "a") - checkRule(query, expected) - } - - test("simplify map ops, dynamic lookup, dynamic keys, lookup is equivalent to one of the keys") { - val query = relation - .select( - GetMapValue( - CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), - ('id + 3L)) as "a") - val expected = relation - .select( - CaseWhen(Seq( - (EqualTo('id + 3L, 'id), ('id + 1L)), - (EqualTo('id + 3L, ('id + 1L)), ('id + 2L)), - (EqualTo('id + 3L, ('id + 2L)), ('id + 3L)), - (Literal(true), ('id + 4L)))) as "a") - checkRule(query, expected) - } - - test("simplify map ops, no positive match") { - val rel = relation - .select( - GetMapValue( - CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), - 'id + 30L) as "a") - val expected = relation.select( - CaseWhen(Seq( - (EqualTo('id + 30L, 'id), ('id + 1L)), - (EqualTo('id + 30L, ('id + 1L)), ('id + 2L)), - (EqualTo('id + 30L, ('id + 2L)), ('id + 3L)), - (EqualTo('id + 30L, ('id + 3L)), ('id + 4L)), - (EqualTo('id + 30L, ('id + 4L)), ('id + 5L)))) as "a") - checkRule(rel, expected) - } - - test("simplify map ops, constant lookup, mixed keys, eliminated constants") { - val rel = relation - .select( - GetMapValue( - CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - Literal(14L), 'id, - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), - 13L) as "a") - - val expected = relation - .select( - CaseKeyWhen(13L, - Seq('id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), ('id + 3L), - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))) as "a") - - checkRule(rel, expected) - } - - test("simplify map ops, potential dynamic match with null value + an absolute constant match") { - val rel = relation - .select( - GetMapValue( - CreateMap(Seq( - 'id, ('id + 1L), - ('id + 1L), ('id + 2L), - ('id + 2L), Literal.create(null, LongType), - Literal(2L), 'id, - ('id + 3L), ('id + 4L), - ('id + 4L), ('id + 5L))), - 2L ) as "a") - - val expected = relation - .select( - CaseWhen(Seq( - (EqualTo(2L, 'id), ('id + 1L)), - // these two are possible matches, we can't tell until runtime - (EqualTo(2L, ('id + 1L)), ('id + 2L)), - (EqualTo(2L, 'id + 2L), Literal.create(null, LongType)), - // this is a definite match (two constants), - // but it cannot override a potential match with ('id + 2L), - // which is exactly what [[Coalesce]] would do in this case. - (Literal.TrueLiteral, 'id))) as "a") - checkRule(rel, expected) - } - - test("SPARK-23500: Simplify array ops that are not at the top node") { - val query = LocalRelation('id.long) - .select( - CreateArray(Seq( - CreateNamedStruct(Seq( - "att1", 'id, - "att2", 'id * 'id)), - CreateNamedStruct(Seq( - "att1", 'id + 1, - "att2", ('id + 1) * ('id + 1)) - )) - ) as "arr") - .select( - GetStructField(GetArrayItem('arr, 1), 0, None) as "a1", - GetArrayItem( - GetArrayStructFields('arr, - StructField("att1", LongType, nullable = false), - ordinal = 0, - numFields = 1, - containsNull = false), - ordinal = 1) as "a2") - .orderBy('id.asc) - - val expected = LocalRelation('id.long) - .select( - ('id + 1L) as "a1", - ('id + 1L) as "a2") - .orderBy('id.asc) - checkRule(query, expected) - } - - test("SPARK-23500: Simplify map ops that are not top nodes") { - val query = - LocalRelation('id.long) - .select( - CreateMap(Seq( - "r1", 'id, - "r2", 'id + 1L)) as "m") - .select( - GetMapValue('m, "r1") as "a1", - GetMapValue('m, "r32") as "a2") - .orderBy('id.asc) - .select('a1, 'a2) - - val expected = - LocalRelation('id.long).select( - 'id as "a1", - Literal.create(null, LongType) as "a2") - .orderBy('id.asc) - checkRule(query, expected) - } - - test("SPARK-23500: Simplify complex ops that aren't at the plan root") { - val structRel = relation - .select(GetStructField(CreateNamedStruct(Seq("att1", 'nullable_id)), 0, None) as "foo") - .groupBy($"foo")("1") - val structExpected = relation - .select('nullable_id as "foo") - .groupBy($"foo")("1") - checkRule(structRel, structExpected) - - val arrayRel = relation - .select(GetArrayItem(CreateArray(Seq('nullable_id, 'nullable_id + 1L)), 0) as "a1") - .groupBy($"a1")("1") - val arrayExpected = relation.select('nullable_id as "a1").groupBy($"a1")("1") - checkRule(arrayRel, arrayExpected) - - val mapRel = relation - .select(GetMapValue(CreateMap(Seq("id", 'nullable_id)), "id") as "m1") - .groupBy($"m1")("1") - val mapExpected = relation - .select('nullable_id as "m1") - .groupBy($"m1")("1") - checkRule(mapRel, mapExpected) - } - - test("SPARK-23500: Ensure that aggregation expressions are not simplified") { - // Make sure that aggregation exprs are correctly ignored. Maps can't be used in - // grouping exprs so aren't tested here. - val structAggRel = relation.groupBy( - CreateNamedStruct(Seq("att1", 'nullable_id)))( - GetStructField(CreateNamedStruct(Seq("att1", 'nullable_id)), 0, None)) - checkRule(structAggRel, structAggRel) - - val arrayAggRel = relation.groupBy( - CreateArray(Seq('nullable_id)))(GetArrayItem(CreateArray(Seq('nullable_id)), 0)) - checkRule(arrayAggRel, arrayAggRel) - - // This could be done if we had a more complex rule that checks that - // the CreateMap does not come from key. - val originalQuery = relation - .groupBy('id)( - GetMapValue(CreateMap(Seq('id, 'id + 1L)), 0L) as "a" - ) - checkRule(originalQuery, originalQuery) - } - - test("SPARK-23500: namedStruct and getField in the same Project #1") { - val originalQuery = - testRelation - .select( - namedStruct("col1", 'b, "col2", 'c).as("s1"), 'a, 'b) - .select('s1 getField "col2" as 's1Col2, - namedStruct("col1", 'a, "col2", 'b).as("s2")) - .select('s1Col2, 's2 getField "col2" as 's2Col2) - val correctAnswer = - testRelation - .select('c as 's1Col2, 'b as 's2Col2) - checkRule(originalQuery, correctAnswer) - } - - test("SPARK-23500: namedStruct and getField in the same Project #2") { - val originalQuery = - testRelation - .select( - namedStruct("col1", 'b, "col2", 'c) getField "col2" as 'sCol2, - namedStruct("col1", 'a, "col2", 'c) getField "col1" as 'sCol1) - val correctAnswer = - testRelation - .select('c as 'sCol2, 'a as 'sCol1) - checkRule(originalQuery, correctAnswer) - } - - test("SPARK-24313: support binary type as map keys in GetMapValue") { - val mb0 = Literal.create( - Map(Array[Byte](1, 2) -> "1", Array[Byte](3, 4) -> null, Array[Byte](2, 1) -> "2"), - MapType(BinaryType, StringType)) - val mb1 = Literal.create(Map[Array[Byte], String](), MapType(BinaryType, StringType)) - - checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](1, 2, 3))), null) - - checkEvaluation(GetMapValue(mb1, Literal(Array[Byte](1, 2))), null) - checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](2, 1), BinaryType)), "2") - checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](3, 4))), null) - } -} +/* + * 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._ +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 +import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.types._ + +/** +* SPARK-18601 discusses simplification direct access to complex types creators. +* i.e. {{{create_named_struct(square, `x` * `x`).square}}} can be simplified to {{{`x` * `x`}}}. +* sam applies to create_array and create_map +*/ +class ComplexTypesSuite extends PlanTest with ExpressionEvalHelper { + + object Optimizer extends RuleExecutor[LogicalPlan] { + val batches = + Batch("collapse projections", FixedPoint(10), + CollapseProject) :: + Batch("Constant Folding", FixedPoint(10), + NullPropagation, + ConstantFolding, + BooleanSimplification, + SimplifyConditionals, + SimplifyBinaryComparison, + SimplifyExtractValueOps) :: Nil + } + + private val idAtt = ('id).long.notNull + private val nullableIdAtt = ('nullable_id).long + + private val relation = LocalRelation(idAtt, nullableIdAtt) + private val testRelation = LocalRelation('a.int, 'b.int, 'c.int, 'd.double, 'e.int) + + private def checkRule(originalQuery: LogicalPlan, correctAnswer: LogicalPlan) = { + val optimized = Optimizer.execute(originalQuery.analyze) + assert(optimized.resolved, "optimized plans must be still resolvable") + comparePlans(optimized, correctAnswer.analyze) + } + + test("explicit get from namedStruct") { + val query = relation + .select( + GetStructField( + CreateNamedStruct(Seq("att", 'id )), + 0, + None) as "outerAtt") + val expected = relation.select('id as "outerAtt") + + checkRule(query, expected) + } + + test("explicit get from named_struct- expression maintains original deduced alias") { + val query = relation + .select(GetStructField(CreateNamedStruct(Seq("att", 'id)), 0, None)) + + val expected = relation + .select('id as "named_struct(att, id).att") + + checkRule(query, expected) + } + + test("collapsed getStructField ontop of namedStruct") { + val query = relation + .select(CreateNamedStruct(Seq("att", 'id)) as "struct1") + .select(GetStructField('struct1, 0, None) as "struct1Att") + val expected = relation.select('id as "struct1Att") + checkRule(query, expected) + } + + test("collapse multiple CreateNamedStruct/GetStructField pairs") { + val query = relation + .select( + CreateNamedStruct(Seq( + "att1", 'id, + "att2", 'id * 'id)) as "struct1") + .select( + GetStructField('struct1, 0, None) as "struct1Att1", + GetStructField('struct1, 1, None) as "struct1Att2") + + val expected = + relation. + select( + 'id as "struct1Att1", + ('id * 'id) as "struct1Att2") + + checkRule(query, expected) + } + + test("collapsed2 - deduced names") { + val query = relation + .select( + CreateNamedStruct(Seq( + "att1", 'id, + "att2", 'id * 'id)) as "struct1") + .select( + GetStructField('struct1, 0, None), + GetStructField('struct1, 1, None)) + + val expected = + relation. + select( + 'id as "struct1.att1", + ('id * 'id) as "struct1.att2") + + checkRule(query, expected) + } + + test("simplified array ops") { + val rel = relation.select( + CreateArray(Seq( + CreateNamedStruct(Seq( + "att1", 'id, + "att2", 'id * 'id)), + CreateNamedStruct(Seq( + "att1", 'id + 1, + "att2", ('id + 1) * ('id + 1)) + )) + ) as "arr" + ) + val query = rel + .select( + GetArrayStructFields('arr, StructField("att1", LongType, false), 0, 1, false) as "a1", + GetArrayItem('arr, 1) as "a2", + GetStructField(GetArrayItem('arr, 1), 0, None) as "a3", + GetArrayItem( + GetArrayStructFields('arr, + StructField("att1", LongType, false), + 0, + 1, + false), + 1) as "a4") + + val expected = relation + .select( + CreateArray(Seq('id, 'id + 1L)) as "a1", + CreateNamedStruct(Seq( + "att1", ('id + 1L), + "att2", (('id + 1L) * ('id + 1L)))) as "a2", + ('id + 1L) as "a3", + ('id + 1L) as "a4") + checkRule(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.inlinedMutableStates.length == 0) + } + + test("SPARK-23208: Test code splitting for create array related methods") { + val inputs = (1 to 2500).map(x => Literal(s"l_$x")) + checkEvaluation(CreateArray(inputs), new GenericArrayData(inputs.map(_.eval()))) + } + + test("simplify map ops") { + val rel = relation + .select( + CreateMap(Seq( + "r1", CreateNamedStruct(Seq("att1", 'id)), + "r2", CreateNamedStruct(Seq("att1", ('id + 1L))))) as "m") + val query = rel + .select( + GetMapValue('m, "r1") as "a1", + GetStructField(GetMapValue('m, "r1"), 0, None) as "a2", + GetMapValue('m, "r32") as "a3", + GetStructField(GetMapValue('m, "r32"), 0, None) as "a4") + + val expected = + relation.select( + CreateNamedStruct(Seq("att1", 'id)) as "a1", + 'id as "a2", + Literal.create( + null, + StructType( + StructField("att1", LongType, nullable = false) :: Nil + ) + ) as "a3", + Literal.create(null, LongType) as "a4") + checkRule(query, expected) + } + + test("simplify map ops, constant lookup, dynamic keys") { + val query = relation.select( + GetMapValue( + CreateMap(Seq( + 'id, ('id + 1L), + ('id + 1L), ('id + 2L), + ('id + 2L), ('id + 3L), + Literal(13L), 'id, + ('id + 3L), ('id + 4L), + ('id + 4L), ('id + 5L))), + 13L) as "a") + + val expected = relation + .select( + CaseWhen(Seq( + (EqualTo(13L, 'id), ('id + 1L)), + (EqualTo(13L, ('id + 1L)), ('id + 2L)), + (EqualTo(13L, ('id + 2L)), ('id + 3L)), + (Literal(true), 'id))) as "a") + checkRule(query, expected) + } + + test("simplify map ops, dynamic lookup, dynamic keys, lookup is equivalent to one of the keys") { + val query = relation + .select( + GetMapValue( + CreateMap(Seq( + 'id, ('id + 1L), + ('id + 1L), ('id + 2L), + ('id + 2L), ('id + 3L), + ('id + 3L), ('id + 4L), + ('id + 4L), ('id + 5L))), + ('id + 3L)) as "a") + val expected = relation + .select( + CaseWhen(Seq( + (EqualTo('id + 3L, 'id), ('id + 1L)), + (EqualTo('id + 3L, ('id + 1L)), ('id + 2L)), + (EqualTo('id + 3L, ('id + 2L)), ('id + 3L)), + (Literal(true), ('id + 4L)))) as "a") + checkRule(query, expected) + } + + test("simplify map ops, no positive match") { + val rel = relation + .select( + GetMapValue( + CreateMap(Seq( + 'id, ('id + 1L), + ('id + 1L), ('id + 2L), + ('id + 2L), ('id + 3L), + ('id + 3L), ('id + 4L), + ('id + 4L), ('id + 5L))), + 'id + 30L) as "a") + val expected = relation.select( + CaseWhen(Seq( + (EqualTo('id + 30L, 'id), ('id + 1L)), + (EqualTo('id + 30L, ('id + 1L)), ('id + 2L)), + (EqualTo('id + 30L, ('id + 2L)), ('id + 3L)), + (EqualTo('id + 30L, ('id + 3L)), ('id + 4L)), + (EqualTo('id + 30L, ('id + 4L)), ('id + 5L)))) as "a") + checkRule(rel, expected) + } + + test("simplify map ops, constant lookup, mixed keys, eliminated constants") { + val rel = relation + .select( + GetMapValue( + CreateMap(Seq( + 'id, ('id + 1L), + ('id + 1L), ('id + 2L), + ('id + 2L), ('id + 3L), + Literal(14L), 'id, + ('id + 3L), ('id + 4L), + ('id + 4L), ('id + 5L))), + 13L) as "a") + + val expected = relation + .select( + CaseKeyWhen(13L, + Seq('id, ('id + 1L), + ('id + 1L), ('id + 2L), + ('id + 2L), ('id + 3L), + ('id + 3L), ('id + 4L), + ('id + 4L), ('id + 5L))) as "a") + + checkRule(rel, expected) + } + + test("simplify map ops, potential dynamic match with null value + an absolute constant match") { + val rel = relation + .select( + GetMapValue( + CreateMap(Seq( + 'id, ('id + 1L), + ('id + 1L), ('id + 2L), + ('id + 2L), Literal.create(null, LongType), + Literal(2L), 'id, + ('id + 3L), ('id + 4L), + ('id + 4L), ('id + 5L))), + 2L ) as "a") + + val expected = relation + .select( + CaseWhen(Seq( + (EqualTo(2L, 'id), ('id + 1L)), + // these two are possible matches, we can't tell until runtime + (EqualTo(2L, ('id + 1L)), ('id + 2L)), + (EqualTo(2L, 'id + 2L), Literal.create(null, LongType)), + // this is a definite match (two constants), + // but it cannot override a potential match with ('id + 2L), + // which is exactly what [[Coalesce]] would do in this case. + (Literal.TrueLiteral, 'id))) as "a") + checkRule(rel, expected) + } + + test("SPARK-23500: Simplify array ops that are not at the top node") { + val query = LocalRelation('id.long) + .select( + CreateArray(Seq( + CreateNamedStruct(Seq( + "att1", 'id, + "att2", 'id * 'id)), + CreateNamedStruct(Seq( + "att1", 'id + 1, + "att2", ('id + 1) * ('id + 1)) + )) + ) as "arr") + .select( + GetStructField(GetArrayItem('arr, 1), 0, None) as "a1", + GetArrayItem( + GetArrayStructFields('arr, + StructField("att1", LongType, nullable = false), + ordinal = 0, + numFields = 1, + containsNull = false), + ordinal = 1) as "a2") + .orderBy('id.asc) + + val expected = LocalRelation('id.long) + .select( + ('id + 1L) as "a1", + ('id + 1L) as "a2") + .orderBy('id.asc) + checkRule(query, expected) + } + + test("SPARK-23500: Simplify map ops that are not top nodes") { + val query = + LocalRelation('id.long) + .select( + CreateMap(Seq( + "r1", 'id, + "r2", 'id + 1L)) as "m") + .select( + GetMapValue('m, "r1") as "a1", + GetMapValue('m, "r32") as "a2") + .orderBy('id.asc) + .select('a1, 'a2) + + val expected = + LocalRelation('id.long).select( + 'id as "a1", + Literal.create(null, LongType) as "a2") + .orderBy('id.asc) + checkRule(query, expected) + } + + test("SPARK-23500: Simplify complex ops that aren't at the plan root") { + val structRel = relation + .select(GetStructField(CreateNamedStruct(Seq("att1", 'nullable_id)), 0, None) as "foo") + .groupBy($"foo")("1") + val structExpected = relation + .select('nullable_id as "foo") + .groupBy($"foo")("1") + checkRule(structRel, structExpected) + + val arrayRel = relation + .select(GetArrayItem(CreateArray(Seq('nullable_id, 'nullable_id + 1L)), 0) as "a1") + .groupBy($"a1")("1") + val arrayExpected = relation.select('nullable_id as "a1").groupBy($"a1")("1") + checkRule(arrayRel, arrayExpected) + + val mapRel = relation + .select(GetMapValue(CreateMap(Seq("id", 'nullable_id)), "id") as "m1") + .groupBy($"m1")("1") + val mapExpected = relation + .select('nullable_id as "m1") + .groupBy($"m1")("1") + checkRule(mapRel, mapExpected) + } + + test("SPARK-23500: Ensure that aggregation expressions are not simplified") { + // Make sure that aggregation exprs are correctly ignored. Maps can't be used in + // grouping exprs so aren't tested here. + val structAggRel = relation.groupBy( + CreateNamedStruct(Seq("att1", 'nullable_id)))( + GetStructField(CreateNamedStruct(Seq("att1", 'nullable_id)), 0, None)) + checkRule(structAggRel, structAggRel) + + val arrayAggRel = relation.groupBy( + CreateArray(Seq('nullable_id)))(GetArrayItem(CreateArray(Seq('nullable_id)), 0)) + checkRule(arrayAggRel, arrayAggRel) + + // This could be done if we had a more complex rule that checks that + // the CreateMap does not come from key. + val originalQuery = relation + .groupBy('id)( + GetMapValue(CreateMap(Seq('id, 'id + 1L)), 0L) as "a" + ) + checkRule(originalQuery, originalQuery) + } + + test("SPARK-23500: namedStruct and getField in the same Project #1") { + val originalQuery = + testRelation + .select( + namedStruct("col1", 'b, "col2", 'c).as("s1"), 'a, 'b) + .select('s1 getField "col2" as 's1Col2, + namedStruct("col1", 'a, "col2", 'b).as("s2")) + .select('s1Col2, 's2 getField "col2" as 's2Col2) + val correctAnswer = + testRelation + .select('c as 's1Col2, 'b as 's2Col2) + checkRule(originalQuery, correctAnswer) + } + + test("SPARK-23500: namedStruct and getField in the same Project #2") { + val originalQuery = + testRelation + .select( + namedStruct("col1", 'b, "col2", 'c) getField "col2" as 'sCol2, + namedStruct("col1", 'a, "col2", 'c) getField "col1" as 'sCol1) + val correctAnswer = + testRelation + .select('c as 'sCol2, 'a as 'sCol1) + checkRule(originalQuery, correctAnswer) + } + + test("SPARK-24313: support binary type as map keys in GetMapValue") { + val mb0 = Literal.create( + Map(Array[Byte](1, 2) -> "1", Array[Byte](3, 4) -> null, Array[Byte](2, 1) -> "2"), + MapType(BinaryType, StringType)) + val mb1 = Literal.create(Map[Array[Byte], String](), MapType(BinaryType, StringType)) + + checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](1, 2, 3))), null) + + checkEvaluation(GetMapValue(mb1, Literal(Array[Byte](1, 2))), null) + checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](2, 1), BinaryType)), "2") + checkEvaluation(GetMapValue(mb0, Literal(Array[Byte](3, 4))), null) + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java b/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java index f1785853a94ae..419dda874d3d9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/columnar/ColumnDictionary.java @@ -1,58 +1,58 @@ -/* - * 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.columnar; - -import org.apache.spark.sql.execution.vectorized.Dictionary; - -public final class ColumnDictionary implements Dictionary { - private int[] intDictionary; - private long[] longDictionary; - - public ColumnDictionary(int[] dictionary) { - this.intDictionary = dictionary; - } - - public ColumnDictionary(long[] dictionary) { - this.longDictionary = dictionary; - } - - @Override - public int decodeToInt(int id) { - return intDictionary[id]; - } - - @Override - public long decodeToLong(int id) { - return longDictionary[id]; - } - - @Override - public float decodeToFloat(int id) { - throw new UnsupportedOperationException("Dictionary encoding does not support float"); - } - - @Override - public double decodeToDouble(int id) { - throw new UnsupportedOperationException("Dictionary encoding does not support double"); - } - - @Override - public byte[] decodeToBinary(int id) { - throw new UnsupportedOperationException("Dictionary encoding does not support String"); - } -} +/* + * 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.columnar; + +import org.apache.spark.sql.execution.vectorized.Dictionary; + +public final class ColumnDictionary implements Dictionary { + private int[] intDictionary; + private long[] longDictionary; + + public ColumnDictionary(int[] dictionary) { + this.intDictionary = dictionary; + } + + public ColumnDictionary(long[] dictionary) { + this.longDictionary = dictionary; + } + + @Override + public int decodeToInt(int id) { + return intDictionary[id]; + } + + @Override + public long decodeToLong(int id) { + return longDictionary[id]; + } + + @Override + public float decodeToFloat(int id) { + throw new UnsupportedOperationException("Dictionary encoding does not support float"); + } + + @Override + public double decodeToDouble(int id) { + throw new UnsupportedOperationException("Dictionary encoding does not support double"); + } + + @Override + public byte[] decodeToBinary(int id) { + throw new UnsupportedOperationException("Dictionary encoding does not support String"); + } +} From c5c580ba0d253a04a3df5bbfd5acf6b5d23cdc1c Mon Sep 17 00:00:00 2001 From: Patrick Cording Date: Mon, 27 Jan 2020 10:32:15 -0800 Subject: [PATCH 1232/1387] [SPARK-30633][SQL] Append L to seed when type is LongType ### What changes were proposed in this pull request? Allow for using longs as seed for xxHash. ### Why are the changes needed? Codegen fails when passing a seed to xxHash that is > 2^31. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests pass. Should more be added? Closes #27354 from patrickcording/fix_xxhash_seed_bug. Authored-by: Patrick Cording Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/hash.scala | 3 +- .../expressions/HashExpressionsSuite.scala | 39 +++++++++++++++++++ 2 files changed, 41 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index afaa2fb59c39b..52429a63b306d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -282,6 +282,7 @@ abstract class HashExpression[E] extends Expression { } val hashResultType = CodeGenerator.javaType(dataType) + val typedSeed = if (dataType.sameType(LongType)) s"${seed}L" else s"$seed" val codes = ctx.splitExpressionsWithCurrentInputs( expressions = childrenHash, funcName = "computeHash", @@ -296,7 +297,7 @@ abstract class HashExpression[E] extends Expression { ev.copy(code = code""" - |$hashResultType ${ev.value} = $seed; + |$hashResultType ${ev.value} = $typedSeed; |$codes """.stripMargin) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index 7dc072722352f..e6cf979649c83 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -684,6 +684,33 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(murmur3HashPlan(wideRow).getInt(0) == murmursHashEval) } + test("SPARK-30633: xxHash with different type seeds") { + val literal = Literal.create(42L, LongType) + + val longSeeds = Seq( + Long.MinValue, + Integer.MIN_VALUE.toLong - 1L, + 0L, + Integer.MAX_VALUE.toLong + 1L, + Long.MaxValue + ) + for (seed <- longSeeds) { + checkEvaluation(XxHash64(Seq(literal), seed), XxHash64(Seq(literal), seed).eval()) + } + + val intSeeds = Seq( + Integer.MIN_VALUE, + 0, + Integer.MAX_VALUE + ) + for (seed <- intSeeds) { + checkEvaluation(XxHash64(Seq(literal), seed), XxHash64(Seq(literal), seed).eval()) + } + + checkEvaluation(XxHash64(Seq(literal), 100), XxHash64(Seq(literal), 100L).eval()) + checkEvaluation(XxHash64(Seq(literal), 100L), XxHash64(Seq(literal), 100).eval()) + } + private def testHash(inputSchema: StructType): Unit = { val inputGenerator = RandomDataGenerator.forType(inputSchema, nullable = false).get val encoder = RowEncoder(inputSchema) @@ -700,5 +727,17 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(HiveHash(literals), HiveHash(literals).eval()) } } + + val longSeed = Math.abs(seed).toLong + Integer.MAX_VALUE.toLong + test(s"SPARK-30633: xxHash64 with long seed: ${inputSchema.simpleString}") { + for (_ <- 1 to 10) { + val input = encoder.toRow(inputGenerator.apply().asInstanceOf[Row]).asInstanceOf[UnsafeRow] + val literals = input.toSeq(inputSchema).zip(inputSchema.map(_.dataType)).map { + case (value, dt) => Literal.create(value, dt) + } + // Only test the interpreted version has same result with codegen version. + checkEvaluation(XxHash64(literals, longSeed), XxHash64(literals, longSeed).eval()) + } + } } } From 8aebc80e0e67bcb1aa300b8c8b1a209159237632 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 27 Jan 2020 11:19:32 -0800 Subject: [PATCH 1233/1387] [SPARK-30625][SQL] Support `escape` as third parameter of the `like` function ### What changes were proposed in this pull request? In the PR, I propose to transform the `Like` expression to `TernaryExpression`, and add third parameter `escape`. So, the `like` function will have feature parity with `LIKE ... ESCAPE` syntax supported by https://github.com/apache/spark/commit/187f3c17733f94aa3372caca355ad18ec1198f2f. ### Why are the changes needed? The `like` functions can be called with 2 or 3 parameters, and functionally equivalent to `LIKE` and `LIKE ... ESCAPE` SQL expressions. ### Does this PR introduce any user-facing change? Yes, before `like` fails with the exception: ```sql spark-sql> SELECT like('_Apache Spark_', '__%Spark__', '_'); Error in query: Invalid number of arguments for function like. Expected: 2; Found: 3; line 1 pos 7 ``` After: ```sql spark-sql> SELECT like('_Apache Spark_', '__%Spark__', '_'); true ``` ### How was this patch tested? - Add new example for the `like` function which is checked by `SQLQuerySuite` - Run `RegexpExpressionsSuite` and `ExpressionParserSuite`. Closes #27355 from MaxGekk/like-3-args. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../expressions/regexpExpressions.scala | 91 ++++++++++++------- .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../spark/sql/DataFrameFunctionsSuite.scala | 15 +++ 4 files changed, 78 insertions(+), 34 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index b4a8bafe22dfb..40998080bc4e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -99,7 +99,7 @@ package object dsl { } def like(other: Expression, escapeChar: Char = '\\'): Expression = - Like(expr, other, escapeChar) + Like(expr, other, Literal(escapeChar.toString)) def rlike(other: Expression): Expression = RLike(expr, other) def contains(other: Expression): Expression = Contains(expr, other) def startsWith(other: Expression): Expression = StartsWith(expr, other) 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 2354087768615..c9ddc70bf5bc6 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 @@ -22,6 +22,7 @@ import java.util.regex.{MatchResult, Pattern} import org.apache.commons.text.StringEscapeUtils +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{GenericArrayData, StringUtils} @@ -29,18 +30,20 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -abstract class StringRegexExpression extends BinaryExpression +trait StringRegexExpression extends Expression with ImplicitCastInputTypes with NullIntolerant { + def str: Expression + def pattern: Expression + def escape(v: String): String def matches(regex: Pattern, str: String): Boolean override def dataType: DataType = BooleanType - override def inputTypes: Seq[DataType] = Seq(StringType, StringType) // try cache the pattern for Literal - private lazy val cache: Pattern = right match { - case x @ Literal(value: String, StringType) => compile(value) + private lazy val cache: Pattern = pattern match { + case Literal(value: String, StringType) => compile(value) case _ => null } @@ -51,10 +54,9 @@ abstract class StringRegexExpression extends BinaryExpression Pattern.compile(escape(str)) } - protected def pattern(str: String) = if (cache == null) compile(str) else cache - - protected override def nullSafeEval(input1: Any, input2: Any): Any = { - val regex = pattern(input2.asInstanceOf[UTF8String].toString) + def nullSafeMatch(input1: Any, input2: Any): Any = { + val s = input2.asInstanceOf[UTF8String].toString + val regex = if (cache == null) compile(s) else cache if(regex == null) { null } else { @@ -62,7 +64,7 @@ abstract class StringRegexExpression extends BinaryExpression } } - override def sql: String = s"${left.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}" + override def sql: String = s"${str.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${pattern.sql}" } // scalastyle:off line.contains.tab @@ -107,46 +109,65 @@ abstract class StringRegexExpression extends BinaryExpression true > SELECT '%SystemDrive%/Users/John' _FUNC_ '/%SystemDrive/%//Users%' ESCAPE '/'; true + > SELECT _FUNC_('_Apache Spark_', '__%Spark__', '_'); + true """, note = """ Use RLIKE to match with standard regular expressions. """, since = "1.0.0") // scalastyle:on line.contains.tab -case class Like(left: Expression, right: Expression, escapeChar: Char) - extends StringRegexExpression { +case class Like(str: Expression, pattern: Expression, escape: Expression) + extends TernaryExpression with StringRegexExpression { - def this(left: Expression, right: Expression) = this(left, right, '\\') + def this(str: Expression, pattern: Expression) = this(str, pattern, Literal("\\")) + + override def inputTypes: Seq[DataType] = Seq(StringType, StringType, StringType) + override def children: Seq[Expression] = Seq(str, pattern, escape) + + private lazy val escapeChar: Char = if (escape.foldable) { + escape.eval() match { + case s: UTF8String if s != null && s.numChars() == 1 => s.toString.charAt(0) + case s => throw new AnalysisException( + s"The 'escape' parameter must be a string literal of one char but it is $s.") + } + } else { + throw new AnalysisException("The 'escape' parameter must be a string literal.") + } override def escape(v: String): String = StringUtils.escapeLikeRegex(v, escapeChar) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() override def toString: String = escapeChar match { - case '\\' => s"$left LIKE $right" - case c => s"$left LIKE $right ESCAPE '$c'" + case '\\' => s"$str LIKE $pattern" + case c => s"$str LIKE $pattern ESCAPE '$c'" + } + + protected override def nullSafeEval(input1: Any, input2: Any, input3: Any): Any = { + nullSafeMatch(input1, input2) } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val patternClass = classOf[Pattern].getName val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + ".escapeLikeRegex" - if (right.foldable) { - val rVal = right.eval() - if (rVal != null) { + if (pattern.foldable) { + val patternVal = pattern.eval() + if (patternVal != null) { val regexStr = - StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString())) - val pattern = ctx.addMutableState(patternClass, "patternLike", + StringEscapeUtils.escapeJava(escape(patternVal.asInstanceOf[UTF8String].toString())) + val compiledPattern = ctx.addMutableState(patternClass, "compiledPattern", v => s"""$v = $patternClass.compile("$regexStr");""") // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. - val eval = left.genCode(ctx) + val eval = str.genCode(ctx) ev.copy(code = code""" ${eval.code} boolean ${ev.isNull} = ${eval.isNull}; ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { - ${ev.value} = $pattern.matcher(${eval.value}.toString()).matches(); + ${ev.value} = $compiledPattern.matcher(${eval.value}.toString()).matches(); } """) } else { @@ -164,18 +185,18 @@ case class Like(left: Expression, right: Expression, escapeChar: Char) } else { escapeChar } - val rightStr = ctx.freshName("rightStr") - val pattern = ctx.addMutableState(patternClass, "pattern") - val lastRightStr = ctx.addMutableState(classOf[String].getName, "lastRightStr") + val patternStr = ctx.freshName("patternStr") + val compiledPattern = ctx.addMutableState(patternClass, "compiledPattern") + val lastPatternStr = ctx.addMutableState(classOf[String].getName, "lastPatternStr") - nullSafeCodeGen(ctx, ev, (eval1, eval2) => { + nullSafeCodeGen(ctx, ev, (eval1, eval2, _) => { s""" - String $rightStr = $eval2.toString(); - if (!$rightStr.equals($lastRightStr)) { - $pattern = $patternClass.compile($escapeFunc($rightStr, '$newEscapeChar')); - $lastRightStr = $rightStr; + String $patternStr = $eval2.toString(); + if (!$patternStr.equals($lastPatternStr)) { + $compiledPattern = $patternClass.compile($escapeFunc($patternStr, '$newEscapeChar')); + $lastPatternStr = $patternStr; } - ${ev.value} = $pattern.matcher($eval1.toString()).matches(); + ${ev.value} = $compiledPattern.matcher($eval1.toString()).matches(); """ }) } @@ -214,12 +235,20 @@ case class Like(left: Expression, right: Expression, escapeChar: Char) """, since = "1.0.0") // scalastyle:on line.contains.tab -case class RLike(left: Expression, right: Expression) extends StringRegexExpression { +case class RLike(left: Expression, right: Expression) + extends BinaryExpression with StringRegexExpression { + + override def inputTypes: Seq[DataType] = Seq(StringType, StringType) + + override def str: Expression = left + override def pattern: Expression = right override def escape(v: String): String = v override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) override def toString: String = s"$left RLIKE $right" + protected override def nullSafeEval(input1: Any, input2: Any): Any = nullSafeMatch(input1, input2) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val patternClass = classOf[Pattern].getName diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index e1dca4e945397..967eca77145e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1392,9 +1392,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging throw new ParseException("Invalid escape string." + "Escape string must contains only one character.", ctx) } - str.charAt(0) + str }.getOrElse('\\') - invertIfNotDefined(Like(e, expression(ctx.pattern), escapeChar)) + invertIfNotDefined(Like(e, expression(ctx.pattern), Literal(escapeChar))) case SqlBaseParser.RLIKE => invertIfNotDefined(RLike(e, expression(ctx.pattern))) case SqlBaseParser.NULL if ctx.NOT != null => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 584768eff700b..7fce03658fc16 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -3562,6 +3562,21 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { checkAnswer(df.select("x").filter("exists(i, x -> x % d == 0)"), Seq(Row(1))) } + + test("the like function with the escape parameter") { + val df = Seq(("abc", "a_c", "!")).toDF("str", "pattern", "escape") + checkAnswer(df.selectExpr("like(str, pattern, '@')"), Row(true)) + + val longEscapeError = intercept[AnalysisException] { + df.selectExpr("like(str, pattern, '@%')").collect() + }.getMessage + assert(longEscapeError.contains("The 'escape' parameter must be a string literal of one char")) + + val nonFoldableError = intercept[AnalysisException] { + df.selectExpr("like(str, pattern, escape)").collect() + }.getMessage + assert(nonFoldableError.contains("The 'escape' parameter must be a string literal")) + } } object DataFrameFunctionsSuite { From 96d27274f54062a231ff12fd397a3cc051bc063d Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Tue, 28 Jan 2020 20:55:21 +0800 Subject: [PATCH 1234/1387] [SPARK-30642][ML][PYSPARK] LinearSVC blockify input vectors ### What changes were proposed in this pull request? 1, stack input vectors to blocks (like ALS/MLP); 2, add new param `blockSize`; 3, add a new class `InstanceBlock` 4, standardize the input outside of optimization procedure; ### Why are the changes needed? 1, reduce RAM to persist traing dataset; (save ~40% in test) 2, use Level-2 BLAS routines; (12% ~ 28% faster, without native BLAS) ### Does this PR introduce any user-facing change? a new param `blockSize` ### How was this patch tested? existing and updated testsuites Closes #27360 from zhengruifeng/blockify_svc. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../spark/serializer/KryoSerializer.scala | 1 + .../org/apache/spark/ml/linalg/BLAS.scala | 1 - .../spark/ml/classification/LinearSVC.scala | 51 ++++--- .../apache/spark/ml/feature/Instance.scala | 130 +++++++++++++++++- .../ml/optim/aggregator/HingeAggregator.scala | 121 ++++++++++++++-- .../ml/param/shared/SharedParamsCodeGen.scala | 6 +- .../spark/ml/param/shared/sharedParams.scala | 19 +++ .../ml/classification/LinearSVCSuite.scala | 2 +- .../spark/ml/feature/InstanceSuite.scala | 31 +++++ .../aggregator/HingeAggregatorSuite.scala | 52 +++++-- python/pyspark/ml/classification.py | 23 +++- .../ml/param/_shared_params_code_gen.py | 5 +- python/pyspark/ml/param/shared.py | 18 +++ 13 files changed, 407 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index cdaab599e2a0b..55ac2c410953b 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -502,6 +502,7 @@ private[serializer] object KryoSerializer { "org.apache.spark.ml.attribute.NumericAttribute", "org.apache.spark.ml.feature.Instance", + "org.apache.spark.ml.feature.InstanceBlock", "org.apache.spark.ml.feature.LabeledPoint", "org.apache.spark.ml.feature.OffsetInstance", "org.apache.spark.ml.linalg.DenseMatrix", diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala index e054a15fc9b75..00e5b61dbdc18 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala @@ -682,7 +682,6 @@ private[spark] object BLAS extends Serializable { val xTemp = xValues(k) * alpha while (i < indEnd) { - val rowIndex = Arows(i) yValues(Arows(i)) += Avals(i) * xTemp i += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 905789090d625..6b1cdd8ad3963 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg._ import org.apache.spark.ml.optim.aggregator.HingeAggregator import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} @@ -41,7 +41,7 @@ import org.apache.spark.storage.StorageLevel /** Params for linear SVM Classifier. */ private[classification] trait LinearSVCParams extends ClassifierParams with HasRegParam with HasMaxIter with HasFitIntercept with HasTol with HasStandardization with HasWeightCol - with HasAggregationDepth with HasThreshold { + with HasAggregationDepth with HasThreshold with HasBlockSize { /** * Param for threshold in binary classification prediction. @@ -155,19 +155,26 @@ class LinearSVC @Since("2.2.0") ( def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) setDefault(aggregationDepth -> 2) + /** + * Set block size for stacking input data in matrices. + * Default is 4096. + * + * @group expertSetParam + */ + @Since("3.0.0") + def setBlockSize(value: Int): this.type = set(blockSize, value) + @Since("2.2.0") override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) override protected def train(dataset: Dataset[_]): LinearSVCModel = instrumented { instr => - val handlePersistence = dataset.storageLevel == StorageLevel.NONE - - val instances = extractInstances(dataset) - if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) - instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, - regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth) + regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth, blockSize) + + val sc = dataset.sparkSession.sparkContext + val instances = extractInstances(dataset) val (summarizer, labelSummarizer) = instances.treeAggregate( (Summarizer.createSummarizerBuffer("mean", "std", "count"), new MultiClassSummarizer))( @@ -208,20 +215,33 @@ class LinearSVC @Since("2.2.0") ( throw new SparkException(msg) } - val featuresStd = summarizer.std.toArray - val getFeaturesStd = (j: Int) => featuresStd(j) + val featuresStd = summarizer.std.compressed + val bcFeaturesStd = sc.broadcast(featuresStd) val regParamL2 = $(regParam) - val bcFeaturesStd = instances.context.broadcast(featuresStd) val regularization = if (regParamL2 != 0.0) { val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures Some(new L2Regularization(regParamL2, shouldApply, - if ($(standardization)) None else Some(getFeaturesStd))) + if ($(standardization)) None else Some(featuresStd.apply))) } else { None } - val getAggregatorFunc = new HingeAggregator(bcFeaturesStd, $(fitIntercept))(_) - val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization, + val standardized = instances.map { + case Instance(label, weight, features) => + val featuresStd = bcFeaturesStd.value + val array = Array.ofDim[Double](numFeatures) + features.foreachNonZero { (i, v) => + val std = featuresStd(i) + if (std != 0) array(i) = v / std + } + Instance(label, weight, Vectors.dense(array)) + } + val blocks = InstanceBlock.blokify(standardized, $(blockSize)) + .persist(StorageLevel.MEMORY_AND_DISK) + .setName(s"training dataset (blockSize=${$(blockSize)})") + + val getAggregatorFunc = new HingeAggregator(numFeatures, $(fitIntercept), $(blockSize))(_) + val costFun = new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) def regParamL1Fun = (index: Int) => 0D @@ -238,6 +258,7 @@ class LinearSVC @Since("2.2.0") ( scaledObjectiveHistory += state.adjustedValue } + blocks.unpersist() bcFeaturesStd.destroy() if (state == null) { val msg = s"${optimizer.getClass.getName} failed." @@ -268,8 +289,6 @@ class LinearSVC @Since("2.2.0") ( (Vectors.dense(coefficientArray), intercept, scaledObjectiveHistory.result()) } - if (handlePersistence) instances.unpersist() - copyValues(new LinearSVCModel(uid, coefficientVector, interceptVector)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index 11d0c4689cbba..5476a86eb9d76 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -17,7 +17,10 @@ package org.apache.spark.ml.feature -import org.apache.spark.ml.linalg.Vector +import scala.collection.mutable + +import org.apache.spark.ml.linalg._ +import org.apache.spark.rdd.RDD /** * Class that represents an instance of weighted data point with label and features. @@ -28,6 +31,131 @@ import org.apache.spark.ml.linalg.Vector */ private[spark] case class Instance(label: Double, weight: Double, features: Vector) + +/** + * Class that represents an block of instance. + * If all weights are 1, then an empty array is stored. + */ +private[spark] case class InstanceBlock( + labels: Array[Double], + weights: Array[Double], + matrix: Matrix) { + require(labels.length == matrix.numRows) + require(matrix.isTransposed) + if (weights.nonEmpty) { + require(labels.length == weights.length) + } + + def size: Int = labels.length + + def numFeatures: Int = matrix.numCols + + def instanceIterator: Iterator[Instance] = { + if (weights.nonEmpty) { + labels.iterator.zip(weights.iterator).zip(matrix.rowIter) + .map { case ((label, weight), vec) => Instance(label, weight, vec) } + } else { + labels.iterator.zip(matrix.rowIter) + .map { case (label, vec) => Instance(label, 1.0, vec) } + } + } + + def getLabel(i: Int): Double = labels(i) + + def labelIter: Iterator[Double] = labels.iterator + + @transient lazy val getWeight: Int => Double = { + if (weights.nonEmpty) { + (i: Int) => weights(i) + } else { + (i: Int) => 1.0 + } + } + + def weightIter: Iterator[Double] = { + if (weights.nonEmpty) { + weights.iterator + } else { + Iterator.fill(size)(1.0) + } + } + + // directly get the non-zero iterator of i-th row vector without array copy or slice + @transient lazy val getNonZeroIter: Int => Iterator[(Int, Double)] = { + matrix match { + case dm: DenseMatrix => + (i: Int) => + val start = numFeatures * i + Iterator.tabulate(numFeatures)(j => + (j, dm.values(start + j)) + ).filter(_._2 != 0) + case sm: SparseMatrix => + (i: Int) => + val start = sm.colPtrs(i) + val end = sm.colPtrs(i + 1) + Iterator.tabulate(end - start)(j => + (sm.rowIndices(start + j), sm.values(start + j)) + ).filter(_._2 != 0) + } + } +} + +private[spark] object InstanceBlock { + + def fromInstances(instances: Seq[Instance]): InstanceBlock = { + val labels = instances.map(_.label).toArray + val weights = if (instances.exists(_.weight != 1)) { + instances.map(_.weight).toArray + } else { + Array.emptyDoubleArray + } + val numRows = instances.length + val numCols = instances.head.features.size + val denseSize = Matrices.getDenseSize(numCols, numRows) + val nnz = instances.iterator.map(_.features.numNonzeros).sum + val sparseSize = Matrices.getSparseSize(nnz, numRows + 1) + val matrix = if (denseSize < sparseSize) { + val values = Array.ofDim[Double](numRows * numCols) + var offset = 0 + var j = 0 + while (j < numRows) { + instances(j).features.foreachNonZero { (i, v) => + values(offset + i) = v + } + offset += numCols + j += 1 + } + new DenseMatrix(numRows, numCols, values, true) + } else { + val colIndices = mutable.ArrayBuilder.make[Int] + val values = mutable.ArrayBuilder.make[Double] + val rowPtrs = mutable.ArrayBuilder.make[Int] + var rowPtr = 0 + rowPtrs += 0 + var j = 0 + while (j < numRows) { + var nnz = 0 + instances(j).features.foreachNonZero { (i, v) => + colIndices += i + values += v + nnz += 1 + } + rowPtr += nnz + rowPtrs += rowPtr + j += 1 + } + new SparseMatrix(numRows, numCols, rowPtrs.result(), + colIndices.result(), values.result(), true) + } + InstanceBlock(labels, weights, matrix) + } + + def blokify(instances: RDD[Instance], blockSize: Int): RDD[InstanceBlock] = { + instances.mapPartitions(_.grouped(blockSize).map(InstanceBlock.fromInstances)) + } +} + + /** * Case class that represents an instance of data point with * label, weight, offset and features. diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala index b0906f1b06511..25f7c9ddab42d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg._ /** @@ -32,21 +32,43 @@ import org.apache.spark.ml.linalg._ * * @param bcCoefficients The coefficients corresponding to the features. * @param fitIntercept Whether to fit an intercept term. - * @param bcFeaturesStd The standard deviation values of the features. */ private[ml] class HingeAggregator( - bcFeaturesStd: Broadcast[Array[Double]], - fitIntercept: Boolean)(bcCoefficients: Broadcast[Vector]) - extends DifferentiableLossAggregator[Instance, HingeAggregator] { + numFeatures: Int, + fitIntercept: Boolean, + blockSize: Int = 4096)(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[InstanceBlock, HingeAggregator] { - private val numFeatures: Int = bcFeaturesStd.value.length private val numFeaturesPlusIntercept: Int = if (fitIntercept) numFeatures + 1 else numFeatures + protected override val dim: Int = numFeaturesPlusIntercept @transient private lazy val coefficientsArray = bcCoefficients.value match { case DenseVector(values) => values case _ => throw new IllegalArgumentException(s"coefficients only supports dense vector" + s" but got type ${bcCoefficients.value.getClass}.") } - protected override val dim: Int = numFeaturesPlusIntercept + + @transient private lazy val linear = { + if (fitIntercept) { + new DenseVector(coefficientsArray.take(numFeatures)) + } else { + new DenseVector(coefficientsArray) + } + } + + @transient private lazy val intercept = + if (fitIntercept) coefficientsArray(numFeatures) else 0.0 + + @transient private lazy val linearGradSumVec = { + if (fitIntercept) { + new DenseVector(Array.ofDim[Double](numFeatures)) + } else { + null + } + } + + @transient private lazy val auxiliaryVec = + new DenseVector(Array.ofDim[Double](blockSize)) + /** * Add a new training instance to this HingeAggregator, and update the loss and gradient @@ -62,16 +84,13 @@ private[ml] class HingeAggregator( require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") if (weight == 0.0) return this - val localFeaturesStd = bcFeaturesStd.value val localCoefficients = coefficientsArray val localGradientSumArray = gradientSumArray val dotProduct = { var sum = 0.0 features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - sum += localCoefficients(index) * value / localFeaturesStd(index) - } + sum += localCoefficients(index) * value } if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1) sum @@ -88,9 +107,7 @@ private[ml] class HingeAggregator( if (1.0 > labelScaled * dotProduct) { val gradientScale = -labelScaled * weight features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - localGradientSumArray(index) += value * gradientScale / localFeaturesStd(index) - } + localGradientSumArray(index) += value * gradientScale } if (fitIntercept) { localGradientSumArray(localGradientSumArray.length - 1) += gradientScale @@ -102,4 +119,80 @@ private[ml] class HingeAggregator( this } } + + /** + * Add a new training instance block to this HingeAggregator, and update the loss and gradient + * of the objective function. + * + * @param block The InstanceBlock to be added. + * @return This HingeAggregator object. + */ + def add(block: InstanceBlock): this.type = { + require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + + s"instance. Expecting $numFeatures but got ${block.numFeatures}.") + require(block.weightIter.forall(_ >= 0), + s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") + + if (block.weightIter.forall(_ == 0)) return this + val size = block.size + val localGradientSumArray = gradientSumArray + + // vec here represents dotProducts + val vec = if (size == blockSize) { + auxiliaryVec + } else { + // the last block within one partition may be of size less than blockSize + new DenseVector(Array.ofDim[Double](size)) + } + + if (fitIntercept) { + var i = 0 + while (i < size) { + vec.values(i) = intercept + i += 1 + } + BLAS.gemv(1.0, block.matrix, linear, 1.0, vec) + } else { + BLAS.gemv(1.0, block.matrix, linear, 0.0, vec) + } + + // in-place convert dotProducts to gradient scales + // then, vec represents gradient scales + var i = 0 + while (i < size) { + val weight = block.getWeight(i) + if (weight > 0) { + weightSum += weight + // Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x))) + // Therefore the gradient is -(2y - 1)*x + val label = block.getLabel(i) + val labelScaled = 2 * label - 1.0 + val loss = (1.0 - labelScaled * vec(i)) * weight + if (loss > 0) { + lossSum += loss + val gradScale = -labelScaled * weight + vec.values(i) = gradScale + } else { + vec.values(i) = 0.0 + } + } else { + vec.values(i) = 0.0 + } + i += 1 + } + + // predictions are all correct, no gradient signal + if (vec.values.forall(_ == 0)) return this + + if (fitIntercept) { + BLAS.gemv(1.0, block.matrix.transpose, vec, 0.0, linearGradSumVec) + linearGradSumVec.foreachNonZero { (i, v) => localGradientSumArray(i) += v } + localGradientSumArray(numFeatures) += vec.values.sum + } else { + val gradSumVec = new DenseVector(localGradientSumArray) + BLAS.gemv(1.0, block.matrix.transpose, vec, 1.0, gradSumVec) + } + + this + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index 7ac680ec1183a..eee75e7f5b722 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -104,7 +104,11 @@ private[shared] object SharedParamsCodeGen { isValid = "ParamValidators.inArray(Array(\"euclidean\", \"cosine\"))"), ParamDesc[String]("validationIndicatorCol", "name of the column that indicates whether " + "each row is for training or for validation. False indicates training; true indicates " + - "validation.") + "validation."), + ParamDesc[Int]("blockSize", "block size for stacking input data in matrices. Data is " + + "stacked within partitions. If block size is more than remaining data in a partition " + + "then it is adjusted to the size of this data.", Some("4096"), + isValid = "ParamValidators.gt(0)", isExpertParam = true) ) val code = genSharedParams(params) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 44c993eeafddc..3d1c55a5eb429 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -578,4 +578,23 @@ trait HasValidationIndicatorCol extends Params { /** @group getParam */ final def getValidationIndicatorCol: String = $(validationIndicatorCol) } + +/** + * Trait for shared param blockSize (default: 4096). This trait may be changed or + * removed between minor versions. + */ +@DeveloperApi +trait HasBlockSize extends Params { + + /** + * Param for block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.. + * @group expertParam + */ + final val blockSize: IntParam = new IntParam(this, "blockSize", "block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.", ParamValidators.gt(0)) + + setDefault(blockSize, 4096) + + /** @group expertGetParam */ + final def getBlockSize: Int = $(blockSize) +} // scalastyle:on diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala index c2072cea11859..2b63dc259a14f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -179,7 +179,7 @@ class LinearSVCSuite extends MLTest with DefaultReadWriteTest { test("sparse coefficients in HingeAggregator") { val bcCoefficients = spark.sparkContext.broadcast(Vectors.sparse(2, Array(0), Array(1.0))) val bcFeaturesStd = spark.sparkContext.broadcast(Array(1.0)) - val agg = new HingeAggregator(bcFeaturesStd, true)(bcCoefficients) + val agg = new HingeAggregator(1, true)(bcCoefficients) val thrown = withClue("LinearSVCAggregator cannot handle sparse coefficients") { intercept[IllegalArgumentException] { agg.add(Instance(1.0, 1.0, Vectors.dense(1.0))) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala index 5a74490058398..d780bdf5f5dc8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala @@ -42,5 +42,36 @@ class InstanceSuite extends SparkFunSuite{ val o2 = ser.deserialize[OffsetInstance](ser.serialize(o)) assert(o === o2) } + + val block1 = InstanceBlock.fromInstances(Seq(instance1)) + val block2 = InstanceBlock.fromInstances(Seq(instance1, instance2)) + Seq(block1, block2).foreach { o => + val o2 = ser.deserialize[InstanceBlock](ser.serialize(o)) + assert(o.labels === o2.labels) + assert(o.weights === o2.weights) + assert(o.matrix === o2.matrix) + } + } + + test("InstanceBlock: check correctness") { + val instance1 = Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)) + val instance2 = Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse) + val instances = Seq(instance1, instance2) + + val block = InstanceBlock.fromInstances(instances) + assert(block.size === 2) + assert(block.numFeatures === 2) + block.instanceIterator.zipWithIndex.foreach { + case (instance, i) => + assert(instance.label === instances(i).label) + assert(instance.weight === instances(i).weight) + assert(instance.features.toArray === instances(i).features.toArray) + } + Seq(0, 1).foreach { i => + val nzIter = block.getNonZeroIter(i) + val vec = Vectors.sparse(2, nzIter.toSeq) + assert(vec.toArray === instances(i).features.toArray) + } } + } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala index 61b48ffa10944..c02a0a5e5e7d0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,21 +32,21 @@ class HingeAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { override def beforeAll(): Unit = { super.beforeAll() - instances = Array( + instances = standardize(Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(0.0, 0.3, Vectors.dense(4.0, 0.5)) - ) - instancesConstantFeature = Array( + )) + instancesConstantFeature = standardize(Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), Instance(1.0, 0.3, Vectors.dense(1.0, 0.5)) - ) - instancesConstantFeatureFiltered = Array( + )) + instancesConstantFeatureFiltered = standardize(Array( Instance(0.0, 0.1, Vectors.dense(2.0)), Instance(1.0, 0.5, Vectors.dense(1.0)), Instance(2.0, 0.3, Vectors.dense(0.5)) - ) + )) } /** Get summary statistics for some data and create a new HingeAggregator. */ @@ -54,12 +54,23 @@ class HingeAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { instances: Array[Instance], coefficients: Vector, fitIntercept: Boolean): HingeAggregator = { - val (featuresSummarizer, ySummarizer) = - DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) - val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) - val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) val bcCoefficients = spark.sparkContext.broadcast(coefficients) - new HingeAggregator(bcFeaturesStd, fitIntercept)(bcCoefficients) + new HingeAggregator(instances.head.features.size, fitIntercept)(bcCoefficients) + } + + private def standardize(instances: Array[Instance]): Array[Instance] = { + val (featuresSummarizer, _) = + DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) + val stdArray = featuresSummarizer.variance.toArray.map(math.sqrt) + val numFeatures = stdArray.length + instances.map { case Instance(label, weight, features) => + val standardized = Array.ofDim[Double](numFeatures) + features.foreachNonZero { (i, v) => + val std = stdArray(i) + if (std != 0) standardized(i) = v / std + } + Instance(label, weight, Vectors.dense(standardized).compressed) + } } test("aggregator add method input size") { @@ -160,4 +171,21 @@ class HingeAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { assert(aggConstantFeatureBinary.gradient(1) == aggConstantFeatureBinaryFiltered.gradient(0)) } + test("add instance block") { + val coefArray = Array(1.0, 2.0) + val intercept = 1.0 + + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ Array(intercept)), + fitIntercept = true) + instances.foreach(agg.add) + + val agg2 = getNewAggregator(instances, Vectors.dense(coefArray ++ Array(intercept)), + fitIntercept = true) + val block = InstanceBlock.fromInstances(instances) + agg2.add(block) + + assert(agg.loss ~== agg2.loss relTol 1e-8) + assert(agg.gradient ~== agg2.gradient relTol 1e-8) + } + } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 5ab8e606bda03..89d27fbfa316e 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -165,7 +165,8 @@ def predictProbability(self, value): class _LinearSVCParams(_JavaClassifierParams, HasRegParam, HasMaxIter, HasFitIntercept, HasTol, - HasStandardization, HasWeightCol, HasAggregationDepth, HasThreshold): + HasStandardization, HasWeightCol, HasAggregationDepth, HasThreshold, + HasBlockSize): """ Params for :py:class:`LinearSVC` and :py:class:`LinearSVCModel`. @@ -214,6 +215,8 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable LinearSVCModel... >>> model.getThreshold() 0.5 + >>> model.getBlockSize() + 4096 >>> model.coefficients DenseVector([0.0, -0.2792, -0.1833]) >>> model.intercept @@ -252,18 +255,19 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, - aggregationDepth=2): + aggregationDepth=2, blockSize=4096): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", \ fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, \ - aggregationDepth=2): + aggregationDepth=2, blockSize=4096): """ super(LinearSVC, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LinearSVC", self.uid) self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, - standardization=True, threshold=0.0, aggregationDepth=2) + standardization=True, threshold=0.0, aggregationDepth=2, + blockSize=4096) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -272,12 +276,12 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, - aggregationDepth=2): + aggregationDepth=2, blockSize=4096): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", \ fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, \ - aggregationDepth=2): + aggregationDepth=2, blockSize=4096): Sets params for Linear SVM Classifier. """ kwargs = self._input_kwargs @@ -342,6 +346,13 @@ def setAggregationDepth(self, value): """ return self._set(aggregationDepth=value) + @since("3.0.0") + def setBlockSize(self, value): + """ + Sets the value of :py:attr:`blockSize`. + """ + return self._set(blockSize=value) + class LinearSVCModel(JavaClassificationModel, _LinearSVCParams, JavaMLWritable, JavaMLReadable): """ diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index ded3ca84b30f2..3994625c05f1b 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -164,7 +164,10 @@ def get$Name(self): "'euclidean'", "TypeConverters.toString"), ("validationIndicatorCol", "name of the column that indicates whether each row is for " + "training or for validation. False indicates training; true indicates validation.", - None, "TypeConverters.toString")] + None, "TypeConverters.toString"), + ("blockSize", "block size for stacking input data in matrices. Data is stacked within " + "partitions. If block size is more than remaining data in a partition then it is " + "adjusted to the size of this data.", "4096", "TypeConverters.toInt")] code = [] for name, doc, defaultValueStr, typeConverter in shared: diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 8fc115691f1ab..41ba7b9dc5523 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -580,3 +580,21 @@ def getValidationIndicatorCol(self): Gets the value of validationIndicatorCol or its default value. """ return self.getOrDefault(self.validationIndicatorCol) + + +class HasBlockSize(Params): + """ + Mixin for param blockSize: block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data. + """ + + blockSize = Param(Params._dummy(), "blockSize", "block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.", typeConverter=TypeConverters.toInt) + + def __init__(self): + super(HasBlockSize, self).__init__() + self._setDefault(blockSize=4096) + + def getBlockSize(self): + """ + Gets the value of blockSize or its default value. + """ + return self.getOrDefault(self.blockSize) From 580c2b7e346f08d56fbf8fc7260c57cc23d22ca9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 28 Jan 2020 12:48:16 -0800 Subject: [PATCH 1235/1387] [SPARK-27166][SQL][FOLLOWUP] Refactor to build string once ### What changes were proposed in this pull request? This is a follow-up for https://github.com/apache/spark/pull/24098 to refactor to build string once according to the [review comment](https://github.com/apache/spark/pull/24098#discussion_r369845234) ### Why are the changes needed? Previously, we chose the minimal change way. In this PR, we choose a more robust way than the previous post-step string processing. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? The test case is extended with more cases. Closes #27353 from dongjoon-hyun/SPARK-27166-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/types/ArrayType.scala | 14 +++++++---- .../org/apache/spark/sql/types/DataType.scala | 14 ++++++----- .../org/apache/spark/sql/types/MapType.scala | 18 +++++++++----- .../apache/spark/sql/types/StructField.scala | 12 +++++++--- .../apache/spark/sql/types/StructType.scala | 24 +++++++++---------- .../spark/sql/types/DataTypeSuite.scala | 7 +++--- .../spark/sql/types/StructTypeSuite.scala | 16 +++++++++++++ 7 files changed, 71 insertions(+), 34 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index 7465569868f07..a3a2ccf5ab12c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -23,6 +23,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat /** * Companion object for ArrayType. @@ -66,10 +67,15 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT /** No-arg constructor for kryo. */ protected def this() = this(null, false) - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append( - s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") - DataType.buildFormattedString(elementType, s"$prefix |", builder) + private[sql] def buildFormattedString( + prefix: String, + stringConcat: StringConcat, + maxDepth: Int): Unit = { + if (maxDepth > 0) { + stringConcat.append( + s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") + DataType.buildFormattedString(elementType, s"$prefix |", stringConcat, maxDepth) + } } override private[sql] def jsonValue = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index de062f7efd8d5..8a8cea194bf2c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -35,6 +35,7 @@ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy.{ANSI, STRICT} @@ -224,16 +225,17 @@ object DataType { } protected[types] def buildFormattedString( - dataType: DataType, - prefix: String, - builder: StringBuilder): Unit = { + dataType: DataType, + prefix: String, + stringConcat: StringConcat, + maxDepth: Int): Unit = { dataType match { case array: ArrayType => - array.buildFormattedString(prefix, builder) + array.buildFormattedString(prefix, stringConcat, maxDepth - 1) case struct: StructType => - struct.buildFormattedString(prefix, builder) + struct.buildFormattedString(prefix, stringConcat, maxDepth - 1) case map: MapType => - map.buildFormattedString(prefix, builder) + map.buildFormattedString(prefix, stringConcat, maxDepth - 1) case _ => } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala index 29b9ffc0c3549..2e5c7f731dcc7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -21,6 +21,7 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.apache.spark.annotation.Stable +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat /** * The data type for Maps. Keys in a map are not allowed to have `null` values. @@ -40,12 +41,17 @@ case class MapType( /** No-arg constructor for kryo. */ def this() = this(null, null, false) - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append(s"$prefix-- key: ${keyType.typeName}\n") - DataType.buildFormattedString(keyType, s"$prefix |", builder) - builder.append(s"$prefix-- value: ${valueType.typeName} " + - s"(valueContainsNull = $valueContainsNull)\n") - DataType.buildFormattedString(valueType, s"$prefix |", builder) + private[sql] def buildFormattedString( + prefix: String, + stringConcat: StringConcat, + maxDepth: Int = Int.MaxValue): Unit = { + if (maxDepth > 0) { + stringConcat.append(s"$prefix-- key: ${keyType.typeName}\n") + DataType.buildFormattedString(keyType, s"$prefix |", stringConcat, maxDepth) + stringConcat.append(s"$prefix-- value: ${valueType.typeName} " + + s"(valueContainsNull = $valueContainsNull)\n") + DataType.buildFormattedString(valueType, s"$prefix |", stringConcat, maxDepth) + } } override private[sql] def jsonValue: JValue = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index 6f6b561d67d49..93478af425955 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -22,6 +22,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat /** * A field inside a StructType. @@ -43,9 +44,14 @@ case class StructField( /** No-arg constructor for kryo. */ protected def this() = this(null, null) - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") - DataType.buildFormattedString(dataType, s"$prefix |", builder) + private[sql] def buildFormattedString( + prefix: String, + stringConcat: StringConcat, + maxDepth: Int): Unit = { + if (maxDepth > 0) { + stringConcat.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") + DataType.buildFormattedString(dataType, s"$prefix |", stringConcat, maxDepth) + } } // override the default toString to be compatible with legacy parquet files. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 476d47a2942b2..0dcb1fc7f0006 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -28,6 +28,7 @@ import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, truncatedString, StringUtils} +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.internal.SQLConf /** @@ -361,25 +362,24 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru def treeString: String = treeString(Int.MaxValue) - def treeString(level: Int): String = { - val builder = new StringBuilder - builder.append("root\n") + def treeString(maxDepth: Int): String = { + val stringConcat = new StringUtils.StringConcat() + stringConcat.append("root\n") val prefix = " |" - fields.foreach(field => field.buildFormattedString(prefix, builder)) - - if (level <= 0 || level == Int.MaxValue) { - builder.toString() - } else { - builder.toString().split("\n").filter(_.lastIndexOf("|--") < level * 5 + 1).mkString("\n") - } + val depth = if (maxDepth > 0) maxDepth else Int.MaxValue + fields.foreach(field => field.buildFormattedString(prefix, stringConcat, depth)) + stringConcat.toString() } // scalastyle:off println def printTreeString(): Unit = println(treeString) // scalastyle:on println - private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - fields.foreach(field => field.buildFormattedString(prefix, builder)) + private[sql] def buildFormattedString( + prefix: String, + stringConcat: StringConcat, + maxDepth: Int): Unit = { + fields.foreach(field => field.buildFormattedString(prefix, stringConcat, maxDepth)) } override private[sql] def jsonValue = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index 122a3125ee2c4..b71dc91eed4fa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.JsonParseException import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat class DataTypeSuite extends SparkFunSuite { @@ -462,11 +463,11 @@ class DataTypeSuite extends SparkFunSuite { StructField("c", DataTypes.IntegerType), StructField("d", DataTypes.IntegerType))) - val builder = new StringBuilder + val stringConcat = new StringConcat - MapType(keyType, valueType).buildFormattedString(prefix = "", builder = builder) + MapType(keyType, valueType).buildFormattedString(prefix = "", stringConcat = stringConcat) - val result = builder.toString() + val result = stringConcat.toString() val expected = """-- key: struct | |-- a: integer (nullable = true) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index e1fe983a42c2d..6824a64badc10 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -81,6 +81,22 @@ class StructTypeSuite extends SparkFunSuite { assert(7 == schema.treeString.split("\n").length) assert(7 == schema.treeString(0).split("\n").length) assert(7 == schema.treeString(-1).split("\n").length) + + val schema2 = StructType.fromDDL( + "c1 INT, c2 ARRAY>, c4 STRUCT>>") + assert(4 == schema2.treeString(1).split("\n").length) + assert(7 == schema2.treeString(2).split("\n").length) + assert(9 == schema2.treeString(3).split("\n").length) + assert(10 == schema2.treeString(4).split("\n").length) + assert(10 == schema2.treeString(0).split("\n").length) + + val schema3 = StructType.fromDDL( + "c1 MAP>>, c3 STRUCT>>") + assert(3 == schema3.treeString(1).split("\n").length) + assert(6 == schema3.treeString(2).split("\n").length) + assert(9 == schema3.treeString(3).split("\n").length) + assert(13 == schema3.treeString(4).split("\n").length) + assert(13 == schema3.treeString(0).split("\n").length) } test("interval keyword in schema string") { From a2fe73b83c0e7c61d1c83b236565a71e3d005a71 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 28 Jan 2020 17:16:21 -0800 Subject: [PATCH 1236/1387] [SPARK-30481][CORE] Integrate event log compactor into Spark History Server ### What changes were proposed in this pull request? This patch addresses remaining functionality on event log compaction: integrate compaction into FsHistoryProvider. This patch is next task of SPARK-30479 (#27164), please refer the description of PR #27085 to see overall rationalization of this patch. ### Why are the changes needed? One of major goal of SPARK-28594 is to prevent the event logs to become too huge, and SPARK-29779 achieves the goal. We've got another approach in prior, but the old approach required models in both KVStore and live entities to guarantee compatibility, while they're not designed to do so. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added UT. Closes #27208 from HeartSaVioR/SPARK-30481. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin --- .../history/EventLogFileCompactor.scala | 9 +- .../deploy/history/FsHistoryProvider.scala | 173 +++++++++++++----- .../spark/internal/config/History.scala | 16 ++ .../spark/internal/config/package.scala | 18 -- .../history/EventLogFileCompactorSuite.scala | 49 +++-- .../history/FsHistoryProviderSuite.scala | 126 ++++++++++++- docs/monitoring.md | 21 ++- 7 files changed, 313 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala index 80a0a7067a4e4..8558f765175fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileCompactor.scala @@ -29,7 +29,6 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.spark.SparkConf import org.apache.spark.deploy.history.EventFilter.FilterStatistics import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{EVENT_LOG_COMPACTION_SCORE_THRESHOLD, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} import org.apache.spark.scheduler.ReplayListenerBus import org.apache.spark.util.Utils @@ -49,9 +48,11 @@ import org.apache.spark.util.Utils class EventLogFileCompactor( sparkConf: SparkConf, hadoopConf: Configuration, - fs: FileSystem) extends Logging { - private val maxFilesToRetain: Int = sparkConf.get(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN) - private val compactionThresholdScore: Double = sparkConf.get(EVENT_LOG_COMPACTION_SCORE_THRESHOLD) + fs: FileSystem, + maxFilesToRetain: Int, + compactionThresholdScore: Double) extends Logging { + + require(maxFilesToRetain > 0, "Max event log files to retain should be higher than 0.") /** * Compacts the old event log files into one compact file, and clean old event log files being diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 54c50006c33b7..2c023b00e6549 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -158,6 +158,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) new HistoryServerDiskManager(conf, path, listing, clock) } + private val fileCompactor = new EventLogFileCompactor(conf, hadoopConf, fs, + conf.get(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN), conf.get(EVENT_LOG_COMPACTION_SCORE_THRESHOLD)) + // Used to store the paths, which are being processed. This enable the replay log tasks execute // asynchronously and make sure that checkForLogs would not process a path repeatedly. private val processing = ConcurrentHashMap.newKeySet[String] @@ -475,10 +478,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } if (shouldReloadLog(info, reader)) { - // ignore fastInProgressParsing when the status of application is changed from - // in-progress to completed, which is needed for rolling event log. - if (info.appId.isDefined && (info.isComplete == reader.completed) && - fastInProgressParsing) { + // ignore fastInProgressParsing when rolling event log is enabled on the log path, + // to ensure proceeding compaction even fastInProgressParsing is turned on. + if (info.appId.isDefined && reader.lastIndex.isEmpty && fastInProgressParsing) { // When fast in-progress parsing is on, we don't need to re-parse when the // size changes, but we do need to invalidate any existing UIs. // Also, we need to update the `lastUpdated time` to display the updated time in @@ -518,7 +520,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // to parse it. This will allow the cleaner code to detect the file as stale later on // if it was not possible to parse it. listing.write(LogInfo(reader.rootPath.toString(), newLastScanTime, LogType.EventLogs, - None, None, reader.fileSizeForLastIndex, reader.lastIndex, + None, None, reader.fileSizeForLastIndex, reader.lastIndex, None, reader.completed)) reader.fileSizeForLastIndex > 0 } @@ -532,16 +534,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } updated.foreach { entry => - processing(entry.rootPath) - try { - val task: Runnable = () => mergeApplicationListing(entry, newLastScanTime, true) - replayExecutor.submit(task) - } catch { - // let the iteration over the updated entries break, since an exception on - // replayExecutor.submit (..) indicates the ExecutorService is unable - // to take any more submissions at this time - case e: Exception => - logError(s"Exception while submitting event log for replay", e) + submitLogProcessTask(entry.rootPath) { () => + mergeApplicationListing(entry, newLastScanTime, true) } } @@ -661,27 +655,37 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) reader: EventLogFileReader, scanTime: Long, enableOptimizations: Boolean): Unit = { + val rootPath = reader.rootPath try { + val lastEvaluatedForCompaction: Option[Long] = try { + listing.read(classOf[LogInfo], rootPath.toString).lastEvaluatedForCompaction + } catch { + case _: NoSuchElementException => None + } + pendingReplayTasksCount.incrementAndGet() - doMergeApplicationListing(reader, scanTime, enableOptimizations) + doMergeApplicationListing(reader, scanTime, enableOptimizations, lastEvaluatedForCompaction) if (conf.get(CLEANER_ENABLED)) { - checkAndCleanLog(reader.rootPath.toString) + checkAndCleanLog(rootPath.toString) } } catch { case e: InterruptedException => throw e case e: AccessControlException => // We don't have read permissions on the log file - logWarning(s"Unable to read log ${reader.rootPath}", e) - blacklist(reader.rootPath) + logWarning(s"Unable to read log $rootPath", e) + blacklist(rootPath) // SPARK-28157 We should remove this blacklisted entry from the KVStore // to handle permission-only changes with the same file sizes later. - listing.delete(classOf[LogInfo], reader.rootPath.toString) + listing.delete(classOf[LogInfo], rootPath.toString) case e: Exception => logError("Exception while merging application listings", e) } finally { - endProcessing(reader.rootPath) + endProcessing(rootPath) pendingReplayTasksCount.decrementAndGet() + + // triggering another task for compaction task + submitLogProcessTask(rootPath) { () => compact(reader) } } } @@ -692,7 +696,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) private[history] def doMergeApplicationListing( reader: EventLogFileReader, scanTime: Long, - enableOptimizations: Boolean): Unit = { + enableOptimizations: Boolean, + lastEvaluatedForCompaction: Option[Long]): Unit = { val eventsFilter: ReplayEventsFilter = { eventString => eventString.startsWith(APPL_START_EVENT_PREFIX) || eventString.startsWith(APPL_END_EVENT_PREFIX) || @@ -770,8 +775,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) invalidateUI(app.info.id, app.attempts.head.info.attemptId) addListing(app) listing.write(LogInfo(logPath.toString(), scanTime, LogType.EventLogs, Some(app.info.id), - app.attempts.head.info.attemptId, reader.fileSizeForLastIndex, - reader.lastIndex, reader.completed)) + app.attempts.head.info.attemptId, reader.fileSizeForLastIndex, reader.lastIndex, + lastEvaluatedForCompaction, reader.completed)) // For a finished log, remove the corresponding "in progress" entry from the listing DB if // the file is really gone. @@ -795,7 +800,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // mean the end event is before the configured threshold, so call the method again to // re-parse the whole log. logInfo(s"Reparsing $logPath since end event was not found.") - doMergeApplicationListing(reader, scanTime, enableOptimizations = false) + doMergeApplicationListing(reader, scanTime, enableOptimizations = false, + lastEvaluatedForCompaction) case _ => // If the app hasn't written down its app ID to the logs, still record the entry in the @@ -803,7 +809,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // does not make progress after the configured max log age. listing.write( LogInfo(logPath.toString(), scanTime, LogType.EventLogs, None, None, - reader.fileSizeForLastIndex, reader.lastIndex, reader.completed)) + reader.fileSizeForLastIndex, reader.lastIndex, lastEvaluatedForCompaction, + reader.completed)) + } + } + + private def compact(reader: EventLogFileReader): Unit = { + val rootPath = reader.rootPath + try { + reader.lastIndex match { + case Some(lastIndex) => + try { + val info = listing.read(classOf[LogInfo], reader.rootPath.toString) + if (info.lastEvaluatedForCompaction.isEmpty || + info.lastEvaluatedForCompaction.get < lastIndex) { + // haven't tried compaction for this index, do compaction + fileCompactor.compact(reader.listEventLogFiles) + listing.write(info.copy(lastEvaluatedForCompaction = Some(lastIndex))) + } + } catch { + case _: NoSuchElementException => + // this should exist, but ignoring doesn't hurt much + } + + case None => // This is not applied to single event log file. + } + } finally { + endProcessing(rootPath) } } @@ -962,7 +994,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) case e: NoSuchElementException => // For every new driver log file discovered, create a new entry in listing listing.write(LogInfo(f.getPath().toString(), currentTime, LogType.DriverLogs, None, - None, f.getLen(), None, false)) + None, f.getLen(), None, None, false)) false } if (deleteFile) { @@ -989,9 +1021,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } /** - * Rebuilds the application state store from its event log. + * Rebuilds the application state store from its event log. Exposed for testing. */ - private def rebuildAppStore( + private[spark] def rebuildAppStore( store: KVStore, reader: EventLogFileReader, lastUpdated: Long): Unit = { @@ -1010,8 +1042,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } replayBus.addListener(listener) try { + val eventLogFiles = reader.listEventLogFiles logInfo(s"Parsing ${reader.rootPath} to re-build UI...") - parseAppEventLogs(reader.listEventLogFiles, replayBus, !reader.completed) + parseAppEventLogs(eventLogFiles, replayBus, !reader.completed) trackingStore.close(false) logInfo(s"Finished parsing ${reader.rootPath}") } catch { @@ -1122,30 +1155,59 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // At this point the disk data either does not exist or was deleted because it failed to // load, so the event log needs to be replayed. - val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), - attempt.lastIndex) - val isCompressed = reader.compressionCodec.isDefined - logInfo(s"Leasing disk manager space for app $appId / ${attempt.info.attemptId}...") - val lease = dm.lease(reader.totalSize, isCompressed) - val newStorePath = try { - Utils.tryWithResource(KVUtils.open(lease.tmpPath, metadata)) { store => - rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) + var retried = false + var newStorePath: File = null + while (newStorePath == null) { + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastIndex) + val isCompressed = reader.compressionCodec.isDefined + logInfo(s"Leasing disk manager space for app $appId / ${attempt.info.attemptId}...") + val lease = dm.lease(reader.totalSize, isCompressed) + try { + Utils.tryWithResource(KVUtils.open(lease.tmpPath, metadata)) { store => + rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) + } + newStorePath = lease.commit(appId, attempt.info.attemptId) + } catch { + case _: IOException if !retried => + // compaction may touch the file(s) which app rebuild wants to read + // compaction wouldn't run in short interval, so try again... + logWarning(s"Exception occurred while rebuilding app $appId - trying again...") + lease.rollback() + retried = true + + case e: Exception => + lease.rollback() + throw e } - lease.commit(appId, attempt.info.attemptId) - } catch { - case e: Exception => - lease.rollback() - throw e } KVUtils.open(newStorePath, metadata) } private def createInMemoryStore(attempt: AttemptInfoWrapper): KVStore = { - val store = new InMemoryStore() - val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), - attempt.lastIndex) - rebuildAppStore(store, reader, attempt.info.lastUpdated.getTime()) + var retried = false + var store: KVStore = null + while (store == null) { + try { + val s = new InMemoryStore() + val reader = EventLogFileReader(fs, new Path(logDir, attempt.logPath), + attempt.lastIndex) + rebuildAppStore(s, reader, attempt.info.lastUpdated.getTime()) + store = s + } catch { + case _: IOException if !retried => + // compaction may touch the file(s) which app rebuild wants to read + // compaction wouldn't run in short interval, so try again... + logWarning(s"Exception occurred while rebuilding log path ${attempt.logPath} - " + + "trying again...") + retried = true + + case e: Exception => + throw e + } + } + store } @@ -1175,6 +1237,21 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } deleted } + + /** NOTE: 'task' should ensure it executes 'endProcessing' at the end */ + private def submitLogProcessTask(rootPath: Path)(task: Runnable): Unit = { + try { + processing(rootPath) + replayExecutor.submit(task) + } catch { + // let the iteration over the updated entries break, since an exception on + // replayExecutor.submit (..) indicates the ExecutorService is unable + // to take any more submissions at this time + case e: Exception => + logError(s"Exception while submitting task", e) + endProcessing(rootPath) + } + } } private[history] object FsHistoryProvider { @@ -1218,6 +1295,8 @@ private[history] case class LogInfo( fileSize: Long, @JsonDeserialize(contentAs = classOf[JLong]) lastIndex: Option[Long], + @JsonDeserialize(contentAs = classOf[JLong]) + lastEvaluatedForCompaction: Option[Long], isComplete: Boolean) private[history] class AttemptInfoWrapper( diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index ca9af316dffd0..17fb55d9db860 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -84,6 +84,22 @@ private[spark] object History { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("1m") + private[spark] val EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN = + ConfigBuilder("spark.history.fs.eventLog.rolling.maxFilesToRetain") + .doc("The maximum number of event log files which will be retained as non-compacted. " + + "By default, all event log files will be retained. Please set the configuration " + + s"and ${EVENT_LOG_ROLLING_MAX_FILE_SIZE.key} accordingly if you want to control " + + "the overall size of event log files.") + .intConf + .checkValue(_ > 0, "Max event log files to retain should be higher than 0.") + .createWithDefault(Integer.MAX_VALUE) + + private[spark] val EVENT_LOG_COMPACTION_SCORE_THRESHOLD = + ConfigBuilder("spark.history.fs.eventLog.rolling.compaction.score.threshold") + .internal() + .doubleConf + .createWithDefault(0.7d) + val DRIVER_LOG_CLEANER_ENABLED = ConfigBuilder("spark.history.fs.driverlog.cleaner.enabled") .fallbackConf(CLEANER_ENABLED) 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 110198815c255..40b05cf96d1e3 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 @@ -195,24 +195,6 @@ package object config { "configured to be at least 10 MiB.") .createWithDefaultString("128m") - private[spark] val EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN = - ConfigBuilder("spark.eventLog.rolling.maxFilesToRetain") - // TODO: remove this when integrating compactor with FsHistoryProvider - .internal() - .doc("The maximum number of event log files which will be retained as non-compacted. " + - "By default, all event log files will be retained. Please set the configuration " + - s"and ${EVENT_LOG_ROLLING_MAX_FILE_SIZE.key} accordingly if you want to control " + - "the overall size of event log files.") - .intConf - .checkValue(_ > 0, "Max event log files to retain should be higher than 0.") - .createWithDefault(Integer.MAX_VALUE) - - private[spark] val EVENT_LOG_COMPACTION_SCORE_THRESHOLD = - ConfigBuilder("spark.eventLog.rolling.compaction.score.threshold") - .internal() - .doubleConf - .createWithDefault(0.7d) - private[spark] val EXECUTOR_ID = ConfigBuilder("spark.executor.id").stringConf.createOptional diff --git a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala index 866e610aab980..2a914023ec821 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileCompactorSuite.scala @@ -27,7 +27,6 @@ import org.apache.spark.{SparkConf, SparkFunSuite, Success} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.EventLogTestHelper.writeEventsToRollingWriter import org.apache.spark.executor.ExecutorMetrics -import org.apache.spark.internal.config.{EVENT_LOG_COMPACTION_SCORE_THRESHOLD, EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.status.ListenerEventsTestHelper._ @@ -35,13 +34,16 @@ import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{JsonProtocol, Utils} class EventLogFileCompactorSuite extends SparkFunSuite { - private val sparkConf = testSparkConf() + import EventLogFileCompactorSuite._ + + private val sparkConf = new SparkConf() private val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) test("No event log files") { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, TEST_COMPACTION_SCORE_THRESHOLD) assertNoCompaction(fs, Seq.empty, compactor.compact(Seq.empty), CompactionResultCode.NOT_ENOUGH_FILES) @@ -54,7 +56,8 @@ class EventLogFileCompactorSuite extends SparkFunSuite { val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, (1 to 2).map(_ => testEvent): _*) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, TEST_COMPACTION_SCORE_THRESHOLD) assertNoCompaction(fs, fileStatuses, compactor.compact(fileStatuses), CompactionResultCode.NOT_ENOUGH_FILES) } @@ -66,7 +69,8 @@ class EventLogFileCompactorSuite extends SparkFunSuite { val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, (1 to 5).map(_ => testEvent): _*) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, TEST_COMPACTION_SCORE_THRESHOLD) assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses), expectedNumOfFilesCompacted = 2) } @@ -85,7 +89,8 @@ class EventLogFileCompactorSuite extends SparkFunSuite { assert(fs.rename(fileToCompact, compactedPath)) val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, TEST_COMPACTION_SCORE_THRESHOLD) assertNoCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), CompactionResultCode.NOT_ENOUGH_FILES) } @@ -104,7 +109,8 @@ class EventLogFileCompactorSuite extends SparkFunSuite { assert(fs.rename(fileToCompact, compactedPath)) val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, TEST_COMPACTION_SCORE_THRESHOLD) assertNoCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), CompactionResultCode.NOT_ENOUGH_FILES) } @@ -123,7 +129,8 @@ class EventLogFileCompactorSuite extends SparkFunSuite { assert(fs.rename(fileToCompact, compactedPath)) val newFileStatuses = Seq(fs.getFileStatus(compactedPath)) ++ fileStatuses.drop(1) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, TEST_COMPACTION_SCORE_THRESHOLD) assertCompaction(fs, newFileStatuses, compactor.compact(newFileStatuses), expectedNumOfFilesCompacted = 7) } @@ -145,7 +152,8 @@ class EventLogFileCompactorSuite extends SparkFunSuite { testEvent, testEvent) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, TEST_COMPACTION_SCORE_THRESHOLD) assertCompaction(fs, fileStatuses, compactor.compact(fileStatuses), expectedNumOfFilesCompacted = 2) @@ -166,7 +174,6 @@ class EventLogFileCompactorSuite extends SparkFunSuite { test("Don't compact file if score is lower than threshold") { withTempDir { dir => val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) - val newConf = sparkConf.set(EVENT_LOG_COMPACTION_SCORE_THRESHOLD, 0.7d) // job 1 having 4 tasks val rddsForStage1 = createRddsWithId(1 to 2) @@ -180,7 +187,7 @@ class EventLogFileCompactorSuite extends SparkFunSuite { // here job 1 is finished and job 2 is still live, hence half of total tasks are considered // as live - val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, newConf, hadoopConf, + val fileStatuses = writeEventsToRollingWriter(fs, "app", dir, sparkConf, hadoopConf, Seq(SparkListenerJobStart(1, 0, Seq(stage1)), SparkListenerStageSubmitted(stage1)), tasks, Seq(SparkListenerJobStart(2, 0, Seq(stage2)), SparkListenerStageSubmitted(stage2)), @@ -190,7 +197,8 @@ class EventLogFileCompactorSuite extends SparkFunSuite { testEvent, testEvent) - val compactor = new EventLogFileCompactor(newConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, 0.7d) assertNoCompaction(fs, fileStatuses, compactor.compact(fileStatuses), CompactionResultCode.LOW_SCORE_FOR_COMPACTION) } @@ -260,7 +268,8 @@ class EventLogFileCompactorSuite extends SparkFunSuite { val filters = Seq(new TestEventFilter1, new TestEventFilter2) val logPath = new Path(writer.logPath) - val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs) + val compactor = new EventLogFileCompactor(sparkConf, hadoopConf, fs, + TEST_ROLLING_MAX_FILES_TO_RETAIN, TEST_COMPACTION_SCORE_THRESHOLD) val newPath = compactor.rewrite(filters, Seq(fs.getFileStatus(logPath))) assert(new Path(newPath).getName === logPath.getName + EventLogFileWriter.COMPACTED) @@ -315,12 +324,12 @@ class EventLogFileCompactorSuite extends SparkFunSuite { private def testEvent: Seq[SparkListenerEvent] = Seq(SparkListenerApplicationStart("app", Some("app"), 0, "user", None)) +} - private def testSparkConf(): SparkConf = { - new SparkConf() - .set(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN, 3) - // to simplify the tests, we set the score threshold as 0.0d - // individual test can override the value to verify the functionality - .set(EVENT_LOG_COMPACTION_SCORE_THRESHOLD, 0.0d) - } +object EventLogFileCompactorSuite { + val TEST_ROLLING_MAX_FILES_TO_RETAIN = 3 + + // To simplify the tests, we set the score threshold as 0.0d. + // Individual test can use the other value to verify the functionality. + val TEST_COMPACTION_SCORE_THRESHOLD = 0.0d } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index a96667ffacd26..c2f34fc3a95ed 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -37,7 +37,9 @@ import org.mockito.Mockito.{doThrow, mock, spy, verify, when} import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{SecurityManager, SPARK_VERSION, SparkConf, SparkFunSuite} +import org.apache.spark.{JobExecutionStatus, SecurityManager, SPARK_VERSION, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.EventLogTestHelper._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.DRIVER_LOG_DFS_DIR import org.apache.spark.internal.config.History._ @@ -50,10 +52,10 @@ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.KVUtils.KVStoreScalaSerializer import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils} +import org.apache.spark.util.kvstore.InMemoryStore import org.apache.spark.util.logging.DriverLogger class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { - private var testDir: File = null override def beforeEach(): Unit = { @@ -164,8 +166,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { override private[history] def doMergeApplicationListing( reader: EventLogFileReader, lastSeen: Long, - enableSkipToEnd: Boolean): Unit = { - super.doMergeApplicationListing(reader, lastSeen, enableSkipToEnd) + enableSkipToEnd: Boolean, + lastCompactionIndex: Option[Long]): Unit = { + super.doMergeApplicationListing(reader, lastSeen, enableSkipToEnd, lastCompactionIndex) doMergeApplicationListingCall += 1 } } @@ -1167,7 +1170,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { var fileStatus = new FileStatus(200, false, 0, 0, 0, path) when(mockedFs.getFileStatus(path)).thenReturn(fileStatus) var logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, None, false) + Some("attemptId"), 100, None, None, false) var reader = EventLogFileReader(mockedFs, path) assert(reader.isDefined) assert(mockedProvider.shouldReloadLog(logInfo, reader.get)) @@ -1177,14 +1180,14 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { when(mockedFs.getFileStatus(path)).thenReturn(fileStatus) // DFSInputStream.getFileLength is more than logInfo fileSize logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, None, false) + Some("attemptId"), 100, None, None, false) reader = EventLogFileReader(mockedFs, path) assert(reader.isDefined) assert(mockedProvider.shouldReloadLog(logInfo, reader.get)) // DFSInputStream.getFileLength is equal to logInfo fileSize logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 200, None, false) + Some("attemptId"), 200, None, None, false) reader = EventLogFileReader(mockedFs, path) assert(reader.isDefined) assert(!mockedProvider.shouldReloadLog(logInfo, reader.get)) @@ -1292,11 +1295,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { val serializer = new KVStoreScalaSerializer() val logInfoWithIndexAsNone = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, None, false) + Some("attemptId"), 100, None, None, false) assertSerDe(serializer, logInfoWithIndexAsNone) val logInfoWithIndex = LogInfo("dummy", 0, LogType.EventLogs, Some("appId"), - Some("attemptId"), 100, Some(3), false) + Some("attemptId"), 100, Some(3), None, false) assertSerDe(serializer, logInfoWithIndex) } @@ -1362,6 +1365,111 @@ class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { } } + test("compact event log files") { + def verifyEventLogFiles( + fs: FileSystem, + rootPath: String, + expectedIndexForCompact: Option[Long], + expectedIndicesForNonCompact: Seq[Long]): Unit = { + val reader = EventLogFileReader(fs, new Path(rootPath)).get + var logFiles = reader.listEventLogFiles + + expectedIndexForCompact.foreach { idx => + val headFile = logFiles.head + assert(EventLogFileWriter.isCompacted(headFile.getPath)) + assert(idx == RollingEventLogFilesWriter.getEventLogFileIndex(headFile.getPath.getName)) + logFiles = logFiles.drop(1) + } + + assert(logFiles.size === expectedIndicesForNonCompact.size) + + logFiles.foreach { logFile => + assert(RollingEventLogFilesWriter.isEventLogFile(logFile)) + assert(!EventLogFileWriter.isCompacted(logFile.getPath)) + } + + val indices = logFiles.map { logFile => + RollingEventLogFilesWriter.getEventLogFileIndex(logFile.getPath.getName) + } + assert(expectedIndicesForNonCompact === indices) + } + + withTempDir { dir => + val conf = createTestConf() + conf.set(HISTORY_LOG_DIR, dir.getAbsolutePath) + conf.set(EVENT_LOG_ROLLING_MAX_FILES_TO_RETAIN, 1) + conf.set(EVENT_LOG_COMPACTION_SCORE_THRESHOLD, 0.0d) + val hadoopConf = SparkHadoopUtil.newConfiguration(conf) + val fs = new Path(dir.getAbsolutePath).getFileSystem(hadoopConf) + + val provider = new FsHistoryProvider(conf) + + val writer = new RollingEventLogFilesWriter("app", None, dir.toURI, conf, hadoopConf) + writer.start() + + // writing event log file 1 - don't compact for now + writeEventsToRollingWriter(writer, Seq( + SparkListenerApplicationStart("app", Some("app"), 0, "user", None), + SparkListenerJobStart(1, 0, Seq.empty)), rollFile = false) + + updateAndCheck(provider) { _ => + verifyEventLogFiles(fs, writer.logPath, None, Seq(1)) + val info = provider.listing.read(classOf[LogInfo], writer.logPath) + assert(info.lastEvaluatedForCompaction === Some(1)) + } + + // writing event log file 2 - compact the event log file 1 into 1.compact + writeEventsToRollingWriter(writer, Seq.empty, rollFile = true) + writeEventsToRollingWriter(writer, Seq(SparkListenerUnpersistRDD(1), + SparkListenerJobEnd(1, 1, JobSucceeded)), rollFile = false) + + updateAndCheck(provider) { _ => + verifyEventLogFiles(fs, writer.logPath, Some(1), Seq(2)) + val info = provider.listing.read(classOf[LogInfo], writer.logPath) + assert(info.lastEvaluatedForCompaction === Some(2)) + } + + // writing event log file 3 - compact two files - 1.compact & 2 into one, 2.compact + writeEventsToRollingWriter(writer, Seq.empty, rollFile = true) + writeEventsToRollingWriter(writer, Seq( + SparkListenerExecutorAdded(3, "exec1", new ExecutorInfo("host1", 1, Map.empty)), + SparkListenerJobStart(2, 4, Seq.empty), + SparkListenerJobEnd(2, 5, JobSucceeded)), rollFile = false) + + writer.stop() + + updateAndCheck(provider) { _ => + verifyEventLogFiles(fs, writer.logPath, Some(2), Seq(3)) + + val info = provider.listing.read(classOf[LogInfo], writer.logPath) + assert(info.lastEvaluatedForCompaction === Some(3)) + + val store = new InMemoryStore + val appStore = new AppStatusStore(store) + + val reader = EventLogFileReader(fs, new Path(writer.logPath)).get + provider.rebuildAppStore(store, reader, 0L) + + // replayed store doesn't have any job, as events for job are removed while compacting + intercept[NoSuchElementException] { + appStore.job(1) + } + + // but other events should be available even they were in original files to compact + val appInfo = appStore.applicationInfo() + assert(appInfo.id === "app") + assert(appInfo.name === "app") + + // All events in retained file(s) should be available, including events which would have + // been filtered out if compaction is applied. e.g. finished jobs, removed executors, etc. + val exec1 = appStore.executorSummary("exec1") + assert(exec1.hostPort === "host1") + val job2 = appStore.job(2) + assert(job2.status === JobExecutionStatus.SUCCEEDED) + } + } + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: diff --git a/docs/monitoring.md b/docs/monitoring.md index 090178f5b37eb..31bf1ebdecad3 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -300,7 +300,26 @@ Security options for the Spark History Server are covered more detail in the Even this is set to `true`, this configuration has no effect on a live application, it only affects the history server. - + + spark.history.fs.eventLog.rolling.maxFilesToRetain + Int.MaxValue + + The maximum number of event log files which will be retained as non-compacted. By default, + all event log files will be retained.
    + Please note that compaction will happen in Spark History Server, which means this configuration + should be set to the configuration of Spark History server, and the same value will be applied + across applications which are being loaded in Spark History Server. This also means compaction + and cleanup would require running Spark History Server.
    + Please set the configuration in Spark History Server, and spark.eventLog.rolling.maxFileSize + in each application accordingly if you want to control the overall size of event log files. + The event log files older than these retained files will be compacted into single file and + deleted afterwards.
    + NOTE: Spark History Server may not compact the old event log files if it figures + out not a lot of space would be reduced during compaction. For streaming query + (including Structured Streaming) we normally expect compaction will run, but for + batch query compaction won't run in many cases. + + Note that in all of these UIs, the tables are sortable by clicking their headers, From 298d0a5102e54ddc24f114e83d2b936762722eec Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 29 Jan 2020 10:37:08 +0900 Subject: [PATCH 1237/1387] [SPARK-23435][SPARKR][TESTS] Update testthat to >= 2.0.0 ### What changes were proposed in this pull request? - Update `testthat` to >= 2.0.0 - Replace of `testthat:::run_tests` with `testthat:::test_package_dir` - Add trivial assertions for tests, without any expectations, to avoid skipping. - Update related docs. ### Why are the changes needed? `testthat` version has been frozen by [SPARK-22817](https://issues.apache.org/jira/browse/SPARK-22817) / https://github.com/apache/spark/pull/20003, but 1.0.2 is pretty old, and we shouldn't keep things in this state forever. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? - Existing CI pipeline: - Windows build on AppVeyor, R 3.6.2, testthtat 2.3.1 - Linux build on Jenkins, R 3.1.x, testthat 1.0.2 - Additional builds with thesthat 2.3.1 using [sparkr-build-sandbox](https://github.com/zero323/sparkr-build-sandbox) on c7ed64af9e697b3619779857dd820832176b3be3 R 3.4.4 (image digest ec9032f8cf98) ``` docker pull zero323/sparkr-build-sandbox:3.4.4 docker run zero323/sparkr-build-sandbox:3.4.4 zero323 --branch SPARK-23435 --commit c7ed64af9e697b3619779857dd820832176b3be3 --public-key https://keybase.io/zero323/pgp_keys.asc ``` 3.5.3 (image digest 0b1759ee4d1d) ``` docker pull zero323/sparkr-build-sandbox:3.5.3 docker run zero323/sparkr-build-sandbox:3.5.3 zero323 --branch SPARK-23435 --commit c7ed64af9e697b3619779857dd820832176b3be3 --public-key https://keybase.io/zero323/pgp_keys.asc ``` and 3.6.2 (image digest 6594c8ceb72f) ``` docker pull zero323/sparkr-build-sandbox:3.6.2 docker run zero323/sparkr-build-sandbox:3.6.2 zero323 --branch SPARK-23435 --commit c7ed64af9e697b3619779857dd820832176b3be3 --public-key https://keybase.io/zero323/pgp_keys.asc ```` Corresponding [asciicast](https://asciinema.org/) are available as 10.5281/zenodo.3629431 [![DOI](https://zenodo.org/badge/DOI/10.5281/zenodo.3629431.svg)](https://doi.org/10.5281/zenodo.3629431) (a bit to large to burden asciinema.org, but can run locally via `asciinema play`). ---------------------------- Continued from #27328 Closes #27359 from zero323/SPARK-23435. Authored-by: zero323 Signed-off-by: HyukjinKwon --- R/pkg/tests/fulltests/test_context.R | 4 ++++ R/pkg/tests/fulltests/test_includePackage.R | 2 ++ R/pkg/tests/fulltests/test_sparkSQL.R | 1 + R/pkg/tests/fulltests/test_textFile.R | 1 + R/pkg/tests/run-all.R | 23 +++++++++++++++------ appveyor.yml | 6 +----- docs/README.md | 7 +++---- docs/building-spark.md | 11 +++++----- 8 files changed, 34 insertions(+), 21 deletions(-) diff --git a/R/pkg/tests/fulltests/test_context.R b/R/pkg/tests/fulltests/test_context.R index eb8d2a700e1ea..b9139154bc165 100644 --- a/R/pkg/tests/fulltests/test_context.R +++ b/R/pkg/tests/fulltests/test_context.R @@ -84,6 +84,7 @@ test_that("rdd GC across sparkR.stop", { countRDD(rdd3) countRDD(rdd4) sparkR.session.stop() + expect_true(TRUE) }) test_that("job group functions can be called", { @@ -93,6 +94,7 @@ test_that("job group functions can be called", { clearJobGroup() sparkR.session.stop() + expect_true(TRUE) }) test_that("job description and local properties can be set and got", { @@ -131,6 +133,7 @@ test_that("utility function can be called", { sparkR.sparkContext(master = sparkRTestMaster) setLogLevel("ERROR") sparkR.session.stop() + expect_true(TRUE) }) test_that("getClientModeSparkSubmitOpts() returns spark-submit args from whitelist", { @@ -234,4 +237,5 @@ test_that("SPARK-25234: parallelize should not have integer overflow", { # 47000 * 47000 exceeds integer range parallelize(sc, 1:47000, 47000) sparkR.session.stop() + expect_true(TRUE) }) diff --git a/R/pkg/tests/fulltests/test_includePackage.R b/R/pkg/tests/fulltests/test_includePackage.R index f4ea0d1b5cb27..916361ff4c797 100644 --- a/R/pkg/tests/fulltests/test_includePackage.R +++ b/R/pkg/tests/fulltests/test_includePackage.R @@ -39,6 +39,7 @@ test_that("include inside function", { data <- lapplyPartition(rdd, generateData) actual <- collectRDD(data) } + expect_true(TRUE) }) test_that("use include package", { @@ -55,6 +56,7 @@ test_that("use include package", { data <- lapplyPartition(rdd, generateData) actual <- collectRDD(data) } + expect_true(TRUE) }) sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index cbe73d09dd221..3b3768f7e2715 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1382,6 +1382,7 @@ test_that("column operators", { c5 <- c2 ^ c3 ^ c4 c6 <- c2 %<=>% c3 c7 <- !c6 + expect_true(TRUE) }) test_that("column functions", { diff --git a/R/pkg/tests/fulltests/test_textFile.R b/R/pkg/tests/fulltests/test_textFile.R index be2d2711ff88e..046018c7c2a2d 100644 --- a/R/pkg/tests/fulltests/test_textFile.R +++ b/R/pkg/tests/fulltests/test_textFile.R @@ -75,6 +75,7 @@ test_that("several transformations on RDD created by textFile()", { collectRDD(rdd) unlink(fileName) + expect_true(TRUE) }) test_that("textFile() followed by a saveAsTextFile() returns the same content", { diff --git a/R/pkg/tests/run-all.R b/R/pkg/tests/run-all.R index 1e96418558883..bf02ecdad66ff 100644 --- a/R/pkg/tests/run-all.R +++ b/R/pkg/tests/run-all.R @@ -20,7 +20,6 @@ library(SparkR) # SPARK-25572 if (identical(Sys.getenv("NOT_CRAN"), "true")) { - # Turn all warnings into errors options("warn" = 2) @@ -60,11 +59,23 @@ if (identical(Sys.getenv("NOT_CRAN"), "true")) { if (identical(Sys.getenv("NOT_CRAN"), "true")) { # set random seed for predictable results. mostly for base's sample() in tree and classification set.seed(42) - # for testthat 1.0.2 later, change reporter from "summary" to default_reporter() - testthat:::run_tests("SparkR", - file.path(sparkRDir, "pkg", "tests", "fulltests"), - NULL, - "summary") + + # TODO (SPARK-30663) To be removed once testthat 1.x is removed from all builds + if (grepl("^1\\..*", packageVersion("testthat"))) { + # testthat 1.x + test_runner <- testthat:::run_tests + reporter <- "summary" + + } else { + # testthat >= 2.0.0 + test_runner <- testthat:::test_package_dir + reporter <- testthat::default_reporter() + } + + test_runner("SparkR", + file.path(sparkRDir, "pkg", "tests", "fulltests"), + NULL, + reporter) } SparkR:::uninstallDownloadedSpark() diff --git a/appveyor.yml b/appveyor.yml index 00c688ba18eb6..5d98260265b1a 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -43,11 +43,7 @@ install: - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" - # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. - # As of devtools 2.1.0, it requires testthat higher then 2.1.1 as a dependency. SparkR test requires testthat 1.0.2. - # Therefore, we don't use devtools but installs it directly from the archive including its dependencies. - - cmd: R -e "install.packages(c('crayon', 'praise', 'R6'), repos='https://cloud.r-project.org/')" - - cmd: R -e "install.packages('https://cloud.r-project.org/src/contrib/Archive/testthat/testthat_1.0.2.tar.gz', repos=NULL, type='source')" + - cmd: R -e "install.packages(c('crayon', 'praise', 'R6', 'testthat'), repos='https://cloud.r-project.org/')" - cmd: R -e "packageVersion('knitr'); packageVersion('rmarkdown'); packageVersion('testthat'); packageVersion('e1071'); packageVersion('survival'); packageVersion('arrow')" build_script: diff --git a/docs/README.md b/docs/README.md index ef849d53daf75..2001de6207d10 100644 --- a/docs/README.md +++ b/docs/README.md @@ -6,9 +6,9 @@ license: | 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. @@ -39,9 +39,8 @@ installed. Also install the following libraries: $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs -$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' +$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "testthat", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' -$ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0. diff --git a/docs/building-spark.md b/docs/building-spark.md index 580f982086731..77ab7900dc4a2 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -9,9 +9,9 @@ license: | 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. @@ -70,9 +70,9 @@ This will build Spark distribution along with Python pip and R packages. For mor ## Specifying the Hadoop Version and Enabling YARN -You can specify the exact version of Hadoop to compile against through the `hadoop.version` property. +You can specify the exact version of Hadoop to compile against through the `hadoop.version` property. -You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different +You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different from `hadoop.version`. Example: @@ -238,8 +238,7 @@ The run-tests script also can be limited to a specific Python version or a speci To run the SparkR tests you will need to install the [knitr](https://cran.r-project.org/package=knitr), [rmarkdown](https://cran.r-project.org/package=rmarkdown), [testthat](https://cran.r-project.org/package=testthat), [e1071](https://cran.r-project.org/package=e1071) and [survival](https://cran.r-project.org/package=survival) packages first: - Rscript -e "install.packages(c('knitr', 'rmarkdown', 'devtools', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" - Rscript -e "devtools::install_version('testthat', version = '1.0.2', repos='https://cloud.r-project.org/')" + Rscript -e "install.packages(c('knitr', 'rmarkdown', 'devtools', 'testthat', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" You can run just the SparkR tests using the command: From ec1fb6b4e18ec36452e451021168ae4ae5a12e44 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Wed, 29 Jan 2020 12:23:59 +0900 Subject: [PATCH 1238/1387] [SPARK-30234][SQL][FOLLOWUP] Add `.enabled` in the suffix of the ADD FILE legacy option ### What changes were proposed in this pull request? This pr intends to rename `spark.sql.legacy.addDirectory.recursive` into `spark.sql.legacy.addDirectory.recursive.enabled`. ### Why are the changes needed? For consistent option names. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #27372 from maropu/SPARK-30234-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: HyukjinKwon --- docs/sql-migration-guide.md | 2 +- .../org/apache/spark/sql/internal/SQLConf.scala | 12 +++++++----- .../spark/sql/execution/command/DDLSuite.scala | 4 ++-- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 73d2d599aff49..51e24157cea80 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -324,7 +324,7 @@ license: | - - Since Spark 3.0, `ADD FILE` can be used to add file directories as well. Earlier only single files can be added using this command. To restore the behaviour of earlier versions, set `spark.sql.legacy.addDirectory.recursive` to false. + - Since Spark 3.0, `ADD FILE` can be used to add file directories as well. Earlier only single files can be added using this command. To restore the behaviour of earlier versions, set `spark.sql.legacy.addDirectory.recursive.enabled` to false. - Since Spark 3.0, `SHOW TBLPROPERTIES` will cause `AnalysisException` if the table does not exist. In Spark version 2.4 and earlier, this scenario caused `NoSuchTableException`. Also, `SHOW TBLPROPERTIES` on a temporary view will cause `AnalysisException`. In Spark version 2.4 and earlier, it returned an empty result. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index df1a11cc13481..0e0a8143532d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2116,11 +2116,13 @@ object SQLConf { .booleanConf .createWithDefault(false) - val LEGACY_ADD_DIRECTORY_USING_RECURSIVE = buildConf("spark.sql.legacy.addDirectory.recursive") - .doc("When true, users can add directory by passing path of a directory to ADD FILE " + - "command of SQL. If false, then only a single file can be added.") - .booleanConf - .createWithDefault(true) + val LEGACY_ADD_DIRECTORY_USING_RECURSIVE = + buildConf("spark.sql.legacy.addDirectory.recursive.enabled") + .internal() + .doc("When true, users can add directory by passing path of a directory to ADD FILE " + + "command of SQL. If false, then only a single file can be added.") + .booleanConf + .createWithDefault(true) val LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED = buildConf("spark.sql.legacy.mssqlserver.numericMapping.enabled") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 1a9fe46bd6a91..913cd80a24c6e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2966,14 +2966,14 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("Add a directory when spark.sql.legacy.addDirectory.recursive set to true") { + test("Add a directory when spark.sql.legacy.addDirectory.recursive.enabled set to true") { val directoryToAdd = Utils.createTempDir("/tmp/spark/addDirectory/") val testFile = File.createTempFile("testFile", "1", directoryToAdd) spark.sql(s"ADD FILE $directoryToAdd") assert(new File(SparkFiles.get(s"${directoryToAdd.getName}/${testFile.getName}")).exists()) } - test("Add a directory when spark.sql.legacy.addDirectory.recursive not set to true") { + test("Add a directory when spark.sql.legacy.addDirectory.recursive.enabled not set to true") { withTempDir { testDir => withSQLConf(SQLConf.LEGACY_ADD_DIRECTORY_USING_RECURSIVE.key -> "false") { val msg = intercept[SparkException] { From 3e203c985c0fb7434776b854ecca6fc553e24d58 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Wed, 29 Jan 2020 08:41:40 -0600 Subject: [PATCH 1239/1387] [SPARK-28801][DOC][FOLLOW-UP] Setup links and address other review comments ### What changes were proposed in this pull request? - Sets up links between related sections. - Add "Related sections" for each section. - Change to the left hand side menu to reflect the current status of the doc. - Other minor cleanups. ### Why are the changes needed? Currently Spark lacks documentation on the supported SQL constructs causing confusion among users who sometimes have to look at the code to understand the usage. This is aimed at addressing this issue. ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? Tested using jykyll build --serve Closes #27371 from dilipbiswal/select_finalization. Authored-by: Dilip Biswal Signed-off-by: Sean Owen --- docs/_data/menu-sql.yaml | 32 +++++++------------ docs/sql-ref-syntax-qry-select-clusterby.md | 17 ++++++++-- ...sql-ref-syntax-qry-select-distribute-by.md | 14 ++++++-- docs/sql-ref-syntax-qry-select-groupby.md | 27 +++++++++++----- docs/sql-ref-syntax-qry-select-having.md | 27 +++++++++++----- docs/sql-ref-syntax-qry-select-limit.md | 29 ++++++++++++----- docs/sql-ref-syntax-qry-select-orderby.md | 14 ++++++-- docs/sql-ref-syntax-qry-select-sortby.md | 13 +++++++- docs/sql-ref-syntax-qry-select-where.md | 19 ++++++++--- docs/sql-ref-syntax-qry-select.md | 10 ++++++ docs/sql-ref-syntax-qry.md | 25 +++++++++++---- 11 files changed, 164 insertions(+), 63 deletions(-) diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 0ffe55c3a347f..7673731778e3b 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -123,37 +123,27 @@ - text: SELECT url: sql-ref-syntax-qry-select.html subitems: - - text: DISTINCT Clause - url: sql-ref-syntax-qry-select-distinct.html - - text: Joins - url: sql-ref-syntax-qry-select-join.html - - text: ORDER BY Clause - url: sql-ref-syntax-qry-select-orderby.html + - text: WHERE Clause + url: sql-ref-syntax-qry-select-where.html - text: GROUP BY Clause url: sql-ref-syntax-qry-select-groupby.html - text: HAVING Clause url: sql-ref-syntax-qry-select-having.html + - text: ORDER BY Clause + url: sql-ref-syntax-qry-select-orderby.html + - text: SORT BY Clause + url: sql-ref-syntax-qry-select-sortby.html + - text: CLUSTER BY Clause + url: sql-ref-syntax-qry-select-clusterby.html + - text: DISTRIBUTE BY Clause + url: sql-ref-syntax-qry-select-distribute-by.html - text: LIMIT Clause url: sql-ref-syntax-qry-select-limit.html - - text: Set operations - url: sql-ref-syntax-qry-select-setops.html - text: USE database url: sql-ref-syntax-qry-select-usedb.html - - text: Common Table Expression(CTE) - url: sql-ref-syntax-qry-select-cte.html - - text: Subqueries - url: sql-ref-syntax-qry-select-subqueries.html - - text: Query hints - url: sql-ref-syntax-qry-select-hints.html - - text: SAMPLING - url: sql-ref-syntax-qry-sampling.html - - text: WINDOWING ANALYTIC FUNCTIONS - url: sql-ref-syntax-qry-window.html - - text: AGGREGATION (CUBE/ROLLUP/GROUPING) - url: sql-ref-syntax-qry-aggregation.html - text: EXPLAIN url: sql-ref-syntax-qry-explain.html - - text: Auxilarry Statements + - text: Auxiliary Statements url: sql-ref-syntax-aux.html subitems: - text: Analyze statement diff --git a/docs/sql-ref-syntax-qry-select-clusterby.md b/docs/sql-ref-syntax-qry-select-clusterby.md index 4e59a3e55ae3c..c96c4419212bb 100644 --- a/docs/sql-ref-syntax-qry-select-clusterby.md +++ b/docs/sql-ref-syntax-qry-select-clusterby.md @@ -20,9 +20,10 @@ license: | --- The CLUSTER BY clause is used to first repartition the data based on the input expressions and then sort the data within each partition. This is -semantically equivalent to performing a DISTRIBUTE BY followed by -a SORT BY. This clause only ensures that the resultant rows are -sorted within each partition and does not guarantee a total order of output. +semantically equivalent to performing a +[DISTRIBUTE BY](sql-ref-syntax-qry-select-distribute-by.html) followed by a +[SORT BY](sql-ref-syntax-qry-select-sortby.html). This clause only ensures that the +resultant rows are sorted within each partition and does not guarantee a total order of output. ### Syntax {% highlight sql %} @@ -86,3 +87,13 @@ SELECT age, name FROM person CLUSTER BY age; |16 |Jack N | +---+-------+ {% endhighlight %} + +### Related Clauses +- [SELECT Main](sql-ref-syntax-qry-select.html) +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select-distribute-by.md b/docs/sql-ref-syntax-qry-select-distribute-by.md index a1b3fcbfb5993..e706ccf03917b 100644 --- a/docs/sql-ref-syntax-qry-select-distribute-by.md +++ b/docs/sql-ref-syntax-qry-select-distribute-by.md @@ -19,8 +19,8 @@ license: | limitations under the License. --- The DISTRIBUTE BY clause is used to repartition the data based -on the input expressions. Unlike the `CLUSTER BY` clause, this does not -sort the data within each partition. +on the input expressions. Unlike the [CLUSTER BY](sql-ref-syntax-qry-select-clusterby.html) +clause, this does not sort the data within each partition. ### Syntax {% highlight sql %} @@ -82,3 +82,13 @@ SELECT age, name FROM person DISTRIBUTE BY age; |16 |Jack N | +---+-------+ {% endhighlight %} + +### Related Clauses +- [SELECT Main](sql-ref-syntax-qry-select.html) +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select-groupby.md b/docs/sql-ref-syntax-qry-select-groupby.md index e47ca0bf3ceb2..ab1c5d6eb5ee5 100644 --- a/docs/sql-ref-syntax-qry-select-groupby.md +++ b/docs/sql-ref-syntax-qry-select-groupby.md @@ -73,14 +73,15 @@ GROUP BY [ GROUPING SETS grouping_sets ] group_expression [ , group_expression [ ### Examples {% highlight sql %} CREATE TABLE dealer (id INT, city STRING, car_model STRING, quantity INT); -INSERT INTO dealer VALUES (100, 'Fremont', 'Honda Civic', 10), - (100, 'Fremont', 'Honda Accord', 15), - (100, 'Fremont', 'Honda CRV', 7), - (200, 'Dublin', 'Honda Civic', 20), - (200, 'Dublin', 'Honda Accord', 10), - (200, 'Dublin', 'Honda CRV', 3), - (300, 'San Jose', 'Honda Civic', 5), - (300, 'San Jose', 'Honda Accord', 8); +INSERT INTO dealer VALUES + (100, 'Fremont', 'Honda Civic', 10), + (100, 'Fremont', 'Honda Accord', 15), + (100, 'Fremont', 'Honda CRV', 7), + (200, 'Dublin', 'Honda Civic', 20), + (200, 'Dublin', 'Honda Accord', 10), + (200, 'Dublin', 'Honda CRV', 3), + (300, 'San Jose', 'Honda Civic', 5), + (300, 'San Jose', 'Honda Accord', 8); -- Sum of quantity per dealership. Group by `id`. SELECT id, sum(quantity) FROM dealer GROUP BY id ORDER BY id; @@ -223,3 +224,13 @@ SELECT city, car_model, sum(quantity) AS sum FROM dealer +--------+------------+---+ {% endhighlight %} + +### Related clauses +- [SELECT Main](sql-ref-syntax-qry-select.html) +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-having.md b/docs/sql-ref-syntax-qry-select-having.md index 428d8556e73e0..94d9be649f106 100644 --- a/docs/sql-ref-syntax-qry-select-having.md +++ b/docs/sql-ref-syntax-qry-select-having.md @@ -49,14 +49,15 @@ HAVING boolean_expression ### Examples {% highlight sql %} CREATE TABLE dealer (id INT, city STRING, car_model STRING, quantity INT); -INSERT INTO dealer VALUES (100, 'Fremont', 'Honda Civic', 10), - (100, 'Fremont', 'Honda Accord', 15), - (100, 'Fremont', 'Honda CRV', 7), - (200, 'Dublin', 'Honda Civic', 20), - (200, 'Dublin', 'Honda Accord', 10), - (200, 'Dublin', 'Honda CRV', 3), - (300, 'San Jose', 'Honda Civic', 5), - (300, 'San Jose', 'Honda Accord', 8); +INSERT INTO dealer VALUES + (100, 'Fremont', 'Honda Civic', 10), + (100, 'Fremont', 'Honda Accord', 15), + (100, 'Fremont', 'Honda CRV', 7), + (200, 'Dublin', 'Honda Civic', 20), + (200, 'Dublin', 'Honda Accord', 10), + (200, 'Dublin', 'Honda CRV', 3), + (300, 'San Jose', 'Honda Civic', 5), + (300, 'San Jose', 'Honda Accord', 8); -- `HAVING` clause referring to column in `GROUP BY`. SELECT city, sum(quantity) AS sum FROM dealer GROUP BY city HAVING city = 'Fremont'; @@ -117,3 +118,13 @@ SELECT sum(quantity) AS sum FROM dealer HAVING sum(quantity) > 10; +---+ {% endhighlight %} + +### Related Clauses +- [SELECT Main](sql-ref-syntax-qry-select.html) +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-limit.md b/docs/sql-ref-syntax-qry-select-limit.md index 609bfb98a097a..2b9999cc40783 100644 --- a/docs/sql-ref-syntax-qry-select-limit.md +++ b/docs/sql-ref-syntax-qry-select-limit.md @@ -18,8 +18,10 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- -The LIMIT clause is used to constrain the number of rows returned by the SELECT statement. -In general, this clause is used in conjuction with ORDER BY to ensure that the results are deterministic. +The LIMIT clause is used to constrain the number of rows returned by +the [SELECT](sql-ref-syntax-qry-select.html) statement. In general, this clause +is used in conjuction with [ORDER BY](sql-ref-syntax-qry-select-orderby.html) to +ensure that the results are deterministic. ### Syntax {% highlight sql %} @@ -42,12 +44,13 @@ LIMIT { ALL | integer_expression } ### Examples {% highlight sql %} CREATE TABLE person (name STRING, age INT); -INSERT INTO person VALUES ('Zen Hui', 25), - ('Anil B', 18), - ('Shone S', 16), - ('Mike A', 25), - ('John A', 18), - ('Jack N', 16); +INSERT INTO person VALUES + ('Zen Hui', 25), + ('Anil B', 18), + ('Shone S', 16), + ('Mike A', 25), + ('John A', 18), + ('Jack N', 16); -- Select the first two rows. SELECT name, age FROM person ORDER BY name LIMIT 2; @@ -86,3 +89,13 @@ SELECT name, age FROM person ORDER BY name LIMIT length('SPARK') |Shone S| 16| +-------+---+ {% endhighlight %} + +### Related Clauses +- [SELECT Main](sql-ref-syntax-qry-select.html) +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) diff --git a/docs/sql-ref-syntax-qry-select-orderby.md b/docs/sql-ref-syntax-qry-select-orderby.md index 1a5d2d404e2c0..c4b4ced0b7e73 100644 --- a/docs/sql-ref-syntax-qry-select-orderby.md +++ b/docs/sql-ref-syntax-qry-select-orderby.md @@ -19,8 +19,8 @@ license: | limitations under the License. --- The ORDER BY clause is used to return the result rows in a sorted manner -in the user specified order. Unlike the SORT BY clause, this clause guarantees -a total order in the output. +in the user specified order. Unlike the [SORT BY](sql-ref-syntax-qry-select-sortby.html) +clause, this clause guarantees a total order in the output. ### Syntax {% highlight sql %} @@ -141,3 +141,13 @@ SELECT * FROM person ORDER BY name ASC, age DESC; |300|Mike |80 | +---+-----+----+ {% endhighlight %} + +### Related Clauses +- [SELECT Main](sql-ref-syntax-qry-select.html) +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select-sortby.md b/docs/sql-ref-syntax-qry-select-sortby.md index ee2e006a79c5a..c0a491b78ee83 100644 --- a/docs/sql-ref-syntax-qry-select-sortby.md +++ b/docs/sql-ref-syntax-qry-select-sortby.md @@ -21,7 +21,8 @@ license: | The SORT BY clause is used to return the result rows sorted within each partition in the user specified order. When there is more than one partition SORT BY may return result that is partially ordered. This is different -than ORDER BY clause which guarantees a total order of the output. +than [ORDER BY](sql-ref-syntax-qry-select-orderby.html) clause which guarantees a +total order of the output. ### Syntax {% highlight sql %} @@ -174,3 +175,13 @@ SELECT /*+ REPARTITION(zip_code) */ name, age, zip_code FROM person |Lalit B.|null|94511 | +--------+----+--------+ {% endhighlight %} + +### Related Clauses +- [SELECT Main](sql-ref-syntax-qry-select.html) +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry-select-where.md b/docs/sql-ref-syntax-qry-select-where.md index 09fab64bbe7bb..a493623df480b 100644 --- a/docs/sql-ref-syntax-qry-select-where.md +++ b/docs/sql-ref-syntax-qry-select-where.md @@ -39,10 +39,11 @@ WHERE boolean_expression ### Examples {% highlight sql %} CREATE TABLE person (id INT, name STRING, age INT); -INSERT INTO person VALUES (100, 'John', 30), - (200, 'Mary', NULL), - (300, 'Mike', 80), - (400, 'Dan', 50); +INSERT INTO person VALUES + (100, 'John', 30), + (200, 'Mary', NULL), + (300, 'Mike', 80), + (400, 'Dan', 50); -- Comparison operator in `WHERE` clause. SELECT * FROM person WHERE id > 200 ORDER BY id; @@ -111,3 +112,13 @@ WHERE EXISTS ( +---+----+----+ {% endhighlight %} + +### Related Clauses +- [SELECT Main](sql-ref-syntax-qry-select.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md index 05feda5f9a5dd..00bd719004fb8 100644 --- a/docs/sql-ref-syntax-qry-select.md +++ b/docs/sql-ref-syntax-qry-select.md @@ -134,3 +134,13 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } be referenced in the widow definitions in the query. + +### Related Clauses +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-qry.md b/docs/sql-ref-syntax-qry.md index eb0e73d00e848..cd7c0ffccdc0f 100644 --- a/docs/sql-ref-syntax-qry.md +++ b/docs/sql-ref-syntax-qry.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Data Retrieval +displayTitle: Data Retrieval license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,7 +19,20 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +Spark supports SELECT statement that is used to retrieve rows +from one or more tables according to the specified clauses. The full syntax +and brief description of supported clauses are explained in +[SELECT](sql-ref-syntax-qry-select.html) section. Spark also provides the +ability to generate logical and physical plan for a given query using +[EXPLAIN](sql-ref-syntax-qry-explain.html) statement. + + +- [WHERE Clause](sql-ref-syntax-qry-select-where.html) +- [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +- [HAVING Clause](sql-ref-syntax-qry-select-having.html) +- [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +- [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +- [CLUSTER BY Clause](sql-ref-syntax-qry-select-clusterby.html) +- [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +- [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +- [EXPLAIN Statement](sql-ref-syntax-qry-explain.html) From d0f635e3bc1ad89465be39d00a70ef1591bd6a63 Mon Sep 17 00:00:00 2001 From: Saurabh Chawla Date: Wed, 29 Jan 2020 08:49:45 -0600 Subject: [PATCH 1240/1387] [SPARK-30582][WEBUI] Spark UI is not showing Aggregated Metrics by Executor in stage page ### What changes were proposed in this pull request? There are scenarios where Spark History Server is located behind the VPC. So whenever api calls hit to get the executor Summary(allexecutors). There can be delay in getting the response of executor summary and in mean time "stage-page-template.html" is loaded and the response of executor Summary is not added to the stage-page-template.html. As the result of which Aggregated Metrics by Executor in stage page is showing blank. This scenario can be easily found in the cases when there is some proxy-server which is responsible for sending the request and response to spark History server. This can be reproduced in Knox/In-house proxy servers which are used to send and receive response to Spark History Server. Alternative scenario to test this case, Open the spark UI in developer mode in browser add some breakpoint in stagepage.js, this will add some delay in getting the response and now if we check the spark UI for stage Aggregated Metrics by Executor in stage page is showing blank. So In-order to fix this there is a need to add the change in stagepage.js . There is a need to add the api call to get the html page(stage-page-template.html) first and after that other api calls to get the data that needs to attached in the stagepage (like executor Summary, stageExecutorSummaryInfoKeys exc) ### Why are the changes needed? Since stage page is useful for debugging purpose, This helps in understanding how many task ran on the particular executor and information related to shuffle read and write on that executor. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested. Testing this in a reproducible way requires a running browser or HTML rendering engine that executes the JavaScript.Open the spark UI in developer mode in browser add some breakpoint in stagepage.js, this will add some delay in getting the response and now if we check the spark UI for stage Aggregated Metrics by Executor in stage page is showing blank. Before fix Screenshot 2020-01-20 at 3 21 55 PM After fix Screenshot 2020-01-20 at 3 23 12 PM Closes #27292 from SaurabhChawla100/SPARK-30582. Authored-by: Saurabh Chawla Signed-off-by: Sean Owen --- .../org/apache/spark/ui/static/stagepage.js | 42 +++++++++---------- 1 file changed, 20 insertions(+), 22 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 4b2f007f866cb..ee2b7b353d62e 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -325,6 +325,25 @@ $(document).ready(function () { $('[data-toggle="tooltip"]').tooltip(); var tasksSummary = $("#parent-container"); getStandAloneAppId(function (appId) { + // rendering the UI page + $.get(createTemplateURI(appId, "stagespage"), function(template) { + tasksSummary.append(Mustache.render($(template).filter("#stages-summary-template").html())); + + $("#additionalMetrics").click(function(){ + $("#arrowtoggle1").toggleClass("arrow-open arrow-closed"); + $("#toggle-metrics").toggle(); + if (window.localStorage) { + window.localStorage.setItem("arrowtoggle1class", $("#arrowtoggle1").attr('class')); + } + }); + + $("#aggregatedMetrics").click(function(){ + $("#arrowtoggle2").toggleClass("arrow-open arrow-closed"); + $("#toggle-aggregatedMetrics").toggle(); + if (window.localStorage) { + window.localStorage.setItem("arrowtoggle2class", $("#arrowtoggle2").attr('class')); + } + }); var endPoint = stageEndPoint(appId); var stageAttemptId = getStageAttemptId(); @@ -473,27 +492,6 @@ $(document).ready(function () { var accumulatorTable = responseBody.accumulatorUpdates.filter(accumUpdate => !(accumUpdate.name).toString().includes("internal.")); - // rendering the UI page - var data = {"executors": response}; - $.get(createTemplateURI(appId, "stagespage"), function(template) { - tasksSummary.append(Mustache.render($(template).filter("#stages-summary-template").html(), data)); - - $("#additionalMetrics").click(function(){ - $("#arrowtoggle1").toggleClass("arrow-open arrow-closed"); - $("#toggle-metrics").toggle(); - if (window.localStorage) { - window.localStorage.setItem("arrowtoggle1class", $("#arrowtoggle1").attr('class')); - } - }); - - $("#aggregatedMetrics").click(function(){ - $("#arrowtoggle2").toggleClass("arrow-open arrow-closed"); - $("#toggle-aggregatedMetrics").toggle(); - if (window.localStorage) { - window.localStorage.setItem("arrowtoggle2class", $("#arrowtoggle2").attr('class')); - } - }); - var quantiles = "0,0.25,0.5,0.75,1.0"; $.getJSON(endPoint + "/" + stageAttemptId + "/taskSummary?quantiles=" + quantiles, function(taskMetricsResponse, status, jqXHR) { @@ -965,4 +963,4 @@ $(document).ready(function () { }); }); }); -}); +}); \ No newline at end of file From 6b47ace27d04012bcff47951ea1eea2aa6fb7d60 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 29 Jan 2020 15:02:48 -0600 Subject: [PATCH 1241/1387] [SPARK-30512] Added a dedicated boss event loop group ### What changes were proposed in this pull request? Adding a dedicated boss event loop group to the Netty pipeline in the External Shuffle Service to avoid the delay in channel registration. ``` EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1, conf.getModuleName() + "-boss"); EventLoopGroup workerGroup = NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); bootstrap = new ServerBootstrap() .group(bossGroup, workerGroup) .channel(NettyUtils.getServerChannelClass(ioMode)) .option(ChannelOption.ALLOCATOR, allocator) ``` ### Why are the changes needed? We have been seeing a large number of SASL authentication (RPC requests) timing out with the external shuffle service. ``` java.lang.RuntimeException: java.util.concurrent.TimeoutException: Timeout waiting for task. at org.spark-project.guava.base.Throwables.propagate(Throwables.java:160) at org.apache.spark.network.client.TransportClient.sendRpcSync(TransportClient.java:278) at org.apache.spark.network.sasl.SaslClientBootstrap.doBootstrap(SaslClientBootstrap.java:80) at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:228) at org.apache.spark.network.client.TransportClientFactory.createUnmanagedClient(TransportClientFactory.java:181) at org.apache.spark.network.shuffle.ExternalShuffleClient.registerWithShuffleServer(ExternalShuffleClient.java:141) at org.apache.spark.storage.BlockManager$$anonfun$registerWithExternalShuffleServer$1.apply$mcVI$sp(BlockManager.scala:218) ``` The investigation that we have done is described here: https://github.com/netty/netty/issues/9890 After adding `LoggingHandler` to the netty pipeline, we saw that the registration of the channel was getting delay which is because the worker threads are busy with the existing channels. ### Does this PR introduce any user-facing change? No ### How was this patch tested? We have tested the patch on our clusters and with a stress testing tool. After this change, we didn't see any SASL requests timing out. Existing unit tests pass. Closes #27240 from otterc/SPARK-30512. Authored-by: Chandni Singh Signed-off-by: Thomas Graves --- .../org/apache/spark/network/server/TransportServer.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java index 8396e691e9db1..f0ff9f57e7be5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -100,9 +100,10 @@ public int getPort() { private void init(String hostToBind, int portToBind) { IOMode ioMode = IOMode.valueOf(conf.ioMode()); - EventLoopGroup bossGroup = - NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); - EventLoopGroup workerGroup = bossGroup; + EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1, + conf.getModuleName() + "-boss"); + EventLoopGroup workerGroup = NettyUtils.createEventLoop(ioMode, conf.serverThreads(), + conf.getModuleName() + "-server"); bootstrap = new ServerBootstrap() .group(bossGroup, workerGroup) From 71737861531180bbda9aec8d241b1428fe91cab2 Mon Sep 17 00:00:00 2001 From: uncleGen Date: Wed, 29 Jan 2020 13:43:51 -0800 Subject: [PATCH 1242/1387] [SPARK-29543][SS][UI] Structured Streaming Web UI ### What changes were proposed in this pull request? This PR adds two pages to Web UI for Structured Streaming: - "/streamingquery": Streaming Query Page, providing some aggregate information for running/completed streaming queries. - "/streamingquery/statistics": Streaming Query Statistics Page, providing detailed information for streaming query, including `Input Rate`, `Process Rate`, `Input Rows`, `Batch Duration` and `Operation Duration` ![Screen Shot 2020-01-29 at 1 38 00 PM](https://user-images.githubusercontent.com/1000778/73399837-cd01cc80-429c-11ea-9d4b-1d200a41b8d5.png) ![Screen Shot 2020-01-29 at 1 39 16 PM](https://user-images.githubusercontent.com/1000778/73399838-cd01cc80-429c-11ea-8185-4e56db6866bd.png) ### Why are the changes needed? It helps users to better monitor Structured Streaming query. ### Does this PR introduce any user-facing change? No ### How was this patch tested? - new added and existing UTs - manual test Closes #26201 from uncleGen/SPARK-29543. Lead-authored-by: uncleGen Co-authored-by: Yuanjian Li Co-authored-by: Genmao Yu Signed-off-by: Shixiong Zhu --- .../spark}/ui/static/streaming-page.css | 0 .../apache/spark}/ui/static/streaming-page.js | 0 .../ui/static/structured-streaming-page.js | 171 +++++++++++ .../org/apache/spark/ui/static/webui.js | 2 + .../org/apache/spark/ui/GraphUIData.scala | 169 +++++++++++ .../scala/org/apache/spark/ui/UIUtils.scala | 91 ++++++ .../org/apache/spark/ui/jobs/StagePage.scala | 14 +- .../org/apache/spark/ui/jobs/StageTable.scala | 14 +- project/MimaExcludes.scala | 5 +- .../apache/spark/sql/internal/SQLConf.scala | 16 ++ .../streaming/ProgressReporter.scala | 5 +- .../execution/streaming/StreamExecution.scala | 3 +- .../spark/sql/internal/SharedState.scala | 19 +- .../streaming/StreamingQueryListener.scala | 4 +- .../sql/streaming/StreamingQueryManager.scala | 6 +- .../apache/spark/sql/streaming/progress.scala | 2 + .../sql/streaming/ui/StreamingQueryPage.scala | 147 ++++++++++ .../ui/StreamingQueryStatisticsPage.scala | 271 ++++++++++++++++++ .../ui/StreamingQueryStatusListener.scala | 122 ++++++++ .../sql/streaming/ui/StreamingQueryTab.scala | 39 +++ .../spark/sql/streaming/ui/UIUtils.scala | 60 ++++ .../StreamingQueryListenerSuite.scala | 10 +- ...StreamingQueryStatusAndProgressSuite.scala | 2 + .../sql/streaming/StreamingQuerySuite.scala | 14 +- .../ui/StreamingQueryPageSuite.scala | 125 ++++++++ .../StreamingQueryStatusListenerSuite.scala | 101 +++++++ .../spark/sql/streaming/ui/UIUtilsSuite.scala | 41 +++ .../thriftserver/ui/ThriftServerPage.scala | 16 +- .../spark/streaming/dstream/DStream.scala | 4 +- .../streaming/scheduler/JobScheduler.scala | 4 +- .../spark/streaming/ui/AllBatchesTable.scala | 2 +- .../apache/spark/streaming/ui/BatchPage.scala | 2 +- .../spark/streaming/ui/StreamingPage.scala | 125 +------- .../spark/streaming/ui/StreamingTab.scala | 2 +- .../apache/spark/streaming/ui/UIUtils.scala | 71 +---- .../spark/streaming/DStreamScopeSuite.scala | 6 +- .../spark/streaming/ui/UIUtilsSuite.scala | 12 +- 37 files changed, 1435 insertions(+), 262 deletions(-) rename {streaming/src/main/resources/org/apache/spark/streaming => core/src/main/resources/org/apache/spark}/ui/static/streaming-page.css (100%) rename {streaming/src/main/resources/org/apache/spark/streaming => core/src/main/resources/org/apache/spark}/ui/static/streaming-page.js (100%) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js create mode 100644 core/src/main/scala/org/apache/spark/ui/GraphUIData.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala diff --git a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.css b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.css similarity index 100% rename from streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.css rename to core/src/main/resources/org/apache/spark/ui/static/streaming-page.css diff --git a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js similarity index 100% rename from streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js rename to core/src/main/resources/org/apache/spark/ui/static/streaming-page.js diff --git a/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js b/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js new file mode 100644 index 0000000000000..70250fdbd2d0c --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js @@ -0,0 +1,171 @@ +/* + * 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. + */ + +// pre-define some colors for legends. +var colorPool = ["#F8C471", "#F39C12", "#B9770E", "#73C6B6", "#16A085", "#117A65", "#B2BABB", "#7F8C8D", "#616A6B"]; + +function drawAreaStack(id, labels, values, minX, maxX, minY, maxY) { + d3.select(d3.select(id).node().parentNode) + .style("padding", "8px 0 8px 8px") + .style("border-right", "0px solid white"); + + // Setup svg using Bostock's margin convention + var margin = {top: 20, right: 40, bottom: 30, left: maxMarginLeftForTimeline}; + var width = 850 - margin.left - margin.right; + var height = 300 - margin.top - margin.bottom; + + var svg = d3.select(id) + .append("svg") + .attr("width", width + margin.left + margin.right) + .attr("height", height + margin.top + margin.bottom) + .append("g") + .attr("transform", "translate(" + margin.left + "," + margin.top + ")"); + + var data = values; + + var parse = d3.time.format("%H:%M:%S.%L").parse; + + // Transpose the data into layers + var dataset = d3.layout.stack()(labels.map(function(fruit) { + return data.map(function(d) { + return {_x: d.x, x: parse(d.x), y: +d[fruit]}; + }); + })); + + + // Set x, y and colors + var x = d3.scale.ordinal() + .domain(dataset[0].map(function(d) { return d.x; })) + .rangeRoundBands([10, width-10], 0.02); + + var y = d3.scale.linear() + .domain([0, d3.max(dataset, function(d) { return d3.max(d, function(d) { return d.y0 + d.y; }); })]) + .range([height, 0]); + + var colors = colorPool.slice(0, labels.length) + + // Define and draw axes + var yAxis = d3.svg.axis() + .scale(y) + .orient("left") + .ticks(7) + .tickFormat( function(d) { return d } ); + + var xAxis = d3.svg.axis() + .scale(x) + .orient("bottom") + .tickFormat(d3.time.format("%H:%M:%S.%L")); + + // Only show the first and last time in the graph + var xline = [] + xline.push(x.domain()[0]) + xline.push(x.domain()[x.domain().length - 1]) + xAxis.tickValues(xline); + + svg.append("g") + .attr("class", "y axis") + .call(yAxis) + .append("text") + .attr("transform", "translate(0," + unitLabelYOffset + ")") + .text("ms"); + + svg.append("g") + .attr("class", "x axis") + .attr("transform", "translate(0," + height + ")") + .call(xAxis); + + // Create groups for each series, rects for each segment + var groups = svg.selectAll("g.cost") + .data(dataset) + .enter().append("g") + .attr("class", "cost") + .style("fill", function(d, i) { return colors[i]; }); + + var rect = groups.selectAll("rect") + .data(function(d) { return d; }) + .enter() + .append("rect") + .attr("x", function(d) { return x(d.x); }) + .attr("y", function(d) { return y(d.y0 + d.y); }) + .attr("height", function(d) { return y(d.y0) - y(d.y0 + d.y); }) + .attr("width", x.rangeBand()) + .on('mouseover', function(d) { + var tip = ''; + var idx = 0; + var _values = timeToValues[d._x] + _values.forEach(function (k) { + tip += labels[idx] + ': ' + k + ' '; + idx += 1; + }); + tip += " at " + d._x + showBootstrapTooltip(d3.select(this).node(), tip); + }) + .on('mouseout', function() { + hideBootstrapTooltip(d3.select(this).node()); + }) + .on("mousemove", function(d) { + var xPosition = d3.mouse(this)[0] - 15; + var yPosition = d3.mouse(this)[1] - 25; + tooltip.attr("transform", "translate(" + xPosition + "," + yPosition + ")"); + tooltip.select("text").text(d.y); + }); + + + // Draw legend + var legend = svg.selectAll(".legend") + .data(colors) + .enter().append("g") + .attr("class", "legend") + .attr("transform", function(d, i) { return "translate(30," + i * 19 + ")"; }); + + legend.append("rect") + .attr("x", width - 20) + .attr("width", 18) + .attr("height", 18) + .style("fill", function(d, i) {return colors.slice().reverse()[i];}) + .on('mouseover', function(d, i) { + var len = labels.length + showBootstrapTooltip(d3.select(this).node(), labels[len - 1 - i]); + }) + .on('mouseout', function() { + hideBootstrapTooltip(d3.select(this).node()); + }) + .on("mousemove", function(d) { + var xPosition = d3.mouse(this)[0] - 15; + var yPosition = d3.mouse(this)[1] - 25; + tooltip.attr("transform", "translate(" + xPosition + "," + yPosition + ")"); + tooltip.select("text").text(d.y); + }); + + // Prep the tooltip bits, initial display is hidden + var tooltip = svg.append("g") + .attr("class", "tooltip") + .style("display", "none"); + + tooltip.append("rect") + .attr("width", 30) + .attr("height", 20) + .attr("fill", "white") + .style("opacity", 0.5); + + tooltip.append("text") + .attr("x", 15) + .attr("dy", "1.2em") + .style("text-anchor", "middle") + .attr("font-size", "12px") + .attr("font-weight", "bold"); +} diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.js b/core/src/main/resources/org/apache/spark/ui/static/webui.js index fac464e1353cd..0ba461f02317f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.js +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.js @@ -90,6 +90,8 @@ $(function() { collapseTablePageLoad('collapse-aggregated-sessionstat','aggregated-sessionstat'); collapseTablePageLoad('collapse-aggregated-sqlstat','aggregated-sqlstat'); collapseTablePageLoad('collapse-aggregated-sqlsessionstat','aggregated-sqlsessionstat'); + collapseTablePageLoad('collapse-aggregated-activeQueries','aggregated-activeQueries'); + collapseTablePageLoad('collapse-aggregated-completedQueries','aggregated-completedQueries'); }); $(function() { diff --git a/core/src/main/scala/org/apache/spark/ui/GraphUIData.scala b/core/src/main/scala/org/apache/spark/ui/GraphUIData.scala new file mode 100644 index 0000000000000..87ff677514461 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/GraphUIData.scala @@ -0,0 +1,169 @@ +/* + * 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.ui + +import java.{util => ju} +import java.lang.{Long => JLong} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.xml.{Node, Unparsed} + +/** + * A helper class to generate JavaScript and HTML for both timeline and histogram graphs. + * + * @param timelineDivId the timeline `id` used in the html `div` tag + * @param histogramDivId the timeline `id` used in the html `div` tag + * @param data the data for the graph + * @param minX the min value of X axis + * @param maxX the max value of X axis + * @param minY the min value of Y axis + * @param maxY the max value of Y axis + * @param unitY the unit of Y axis + * @param batchInterval if `batchInterval` is not None, we will draw a line for `batchInterval` in + * the graph + */ +private[spark] class GraphUIData( + timelineDivId: String, + histogramDivId: String, + data: Seq[(Long, Double)], + minX: Long, + maxX: Long, + minY: Double, + maxY: Double, + unitY: String, + batchInterval: Option[Double] = None) { + + private var dataJavaScriptName: String = _ + + def generateDataJs(jsCollector: JsCollector): Unit = { + val jsForData = data.map { case (x, y) => + s"""{"x": $x, "y": $y}""" + }.mkString("[", ",", "]") + dataJavaScriptName = jsCollector.nextVariableName + jsCollector.addPreparedStatement(s"var $dataJavaScriptName = $jsForData;") + } + + def generateTimelineHtml(jsCollector: JsCollector): Seq[Node] = { + jsCollector.addPreparedStatement(s"registerTimeline($minY, $maxY);") + if (batchInterval.isDefined) { + jsCollector.addStatement( + "drawTimeline(" + + s"'#$timelineDivId', $dataJavaScriptName, $minX, $maxX, $minY, $maxY, '$unitY'," + + s" ${batchInterval.get}" + + ");") + } else { + jsCollector.addStatement( + s"drawTimeline('#$timelineDivId', $dataJavaScriptName, $minX, $maxX, $minY, $maxY," + + s" '$unitY');") + } +
    + } + + def generateHistogramHtml(jsCollector: JsCollector): Seq[Node] = { + val histogramData = s"$dataJavaScriptName.map(function(d) { return d.y; })" + jsCollector.addPreparedStatement(s"registerHistogram($histogramData, $minY, $maxY);") + if (batchInterval.isDefined) { + jsCollector.addStatement( + "drawHistogram(" + + s"'#$histogramDivId', $histogramData, $minY, $maxY, '$unitY', ${batchInterval.get}" + + ");") + } else { + jsCollector.addStatement( + s"drawHistogram('#$histogramDivId', $histogramData, $minY, $maxY, '$unitY');") + } +
    + } + + def generateAreaStackHtmlWithData( + jsCollector: JsCollector, + values: Array[(Long, ju.Map[String, JLong])]): Seq[Node] = { + val operationLabels = values.flatMap(_._2.keySet().asScala).toSet + val durationDataPadding = UIUtils.durationDataPadding(values) + val jsForData = durationDataPadding.map { case (x, y) => + val s = y.toSeq.sortBy(_._1).map(e => s""""${e._1}": "${e._2}"""").mkString(",") + s"""{x: "${UIUtils.formatBatchTime(x, 1, showYYYYMMSS = false)}", $s}""" + }.mkString("[", ",", "]") + val jsForLabels = operationLabels.toSeq.sorted.mkString("[\"", "\",\"", "\"]") + + val (maxX, minX, maxY, minY) = if (values != null && values.length > 0) { + val xValues = values.map(_._1.toLong) + val yValues = values.map(_._2.asScala.toSeq.map(_._2.toLong).sum) + (xValues.max, xValues.min, yValues.max, yValues.min) + } else { + (0L, 0L, 0L, 0L) + } + + dataJavaScriptName = jsCollector.nextVariableName + jsCollector.addPreparedStatement(s"var $dataJavaScriptName = $jsForData;") + val labels = jsCollector.nextVariableName + jsCollector.addPreparedStatement(s"var $labels = $jsForLabels;") + jsCollector.addStatement( + s"drawAreaStack('#$timelineDivId', $labels, $dataJavaScriptName, $minX, $maxX, $minY, $maxY)") +
    + } +} + +/** + * A helper class that allows the user to add JavaScript statements which will be executed when the + * DOM has finished loading. + */ +private[spark] class JsCollector { + + private var variableId = 0 + + /** + * Return the next unused JavaScript variable name + */ + def nextVariableName: String = { + variableId += 1 + "v" + variableId + } + + /** + * JavaScript statements that will execute before `statements` + */ + private val preparedStatements = ArrayBuffer[String]() + + /** + * JavaScript statements that will execute after `preparedStatements` + */ + private val statements = ArrayBuffer[String]() + + def addPreparedStatement(js: String): Unit = { + preparedStatements += js + } + + def addStatement(js: String): Unit = { + statements += js + } + + /** + * Generate a html snippet that will execute all scripts when the DOM has finished loading. + */ + def toHtml: Seq[Node] = { + val js = + s""" + |$$(document).ready(function() { + | ${preparedStatements.mkString("\n")} + | ${statements.mkString("\n")} + |});""".stripMargin + + + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 143303df0d10e..94c45215b5ff2 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.ui +import java.{util => ju} +import java.lang.{Long => JLong} import java.net.URLDecoder import java.nio.charset.StandardCharsets.UTF_8 import java.text.SimpleDateFormat @@ -24,6 +26,7 @@ import java.util.{Date, Locale, TimeZone} import javax.servlet.http.HttpServletRequest import javax.ws.rs.core.{MediaType, Response} +import scala.collection.JavaConverters._ import scala.util.control.NonFatal import scala.xml._ import scala.xml.transform.{RewriteRule, RuleTransformer} @@ -119,6 +122,59 @@ private[spark] object UIUtils extends Logging { } } + // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. + private val batchTimeFormat = new ThreadLocal[SimpleDateFormat]() { + override def initialValue(): SimpleDateFormat = + new SimpleDateFormat("yyyy/MM/dd HH:mm:ss", Locale.US) + } + + private val batchTimeFormatWithMilliseconds = new ThreadLocal[SimpleDateFormat]() { + override def initialValue(): SimpleDateFormat = + new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS", Locale.US) + } + + /** + * If `batchInterval` is less than 1 second, format `batchTime` with milliseconds. Otherwise, + * format `batchTime` without milliseconds. + * + * @param batchTime the batch time to be formatted + * @param batchInterval the batch interval + * @param showYYYYMMSS if showing the `yyyy/MM/dd` part. If it's false, the return value wll be + * only `HH:mm:ss` or `HH:mm:ss.SSS` depending on `batchInterval` + * @param timezone only for test + */ + def formatBatchTime( + batchTime: Long, + batchInterval: Long, + showYYYYMMSS: Boolean = true, + timezone: TimeZone = null): String = { + val oldTimezones = + (batchTimeFormat.get.getTimeZone, batchTimeFormatWithMilliseconds.get.getTimeZone) + if (timezone != null) { + batchTimeFormat.get.setTimeZone(timezone) + batchTimeFormatWithMilliseconds.get.setTimeZone(timezone) + } + try { + val formattedBatchTime = + if (batchInterval < 1000) { + batchTimeFormatWithMilliseconds.get.format(batchTime) + } else { + // If batchInterval >= 1 second, don't show milliseconds + batchTimeFormat.get.format(batchTime) + } + if (showYYYYMMSS) { + formattedBatchTime + } else { + formattedBatchTime.substring(formattedBatchTime.indexOf(' ') + 1) + } + } finally { + if (timezone != null) { + batchTimeFormat.get.setTimeZone(oldTimezones._1) + batchTimeFormatWithMilliseconds.get.setTimeZone(oldTimezones._2) + } + } + } + /** Generate a human-readable string representing a number (e.g. 100 K) */ def formatNumber(records: Double): String = { val trillion = 1e12 @@ -572,4 +628,39 @@ private[spark] object UIUtils extends Logging { def buildErrorResponse(status: Response.Status, msg: String): Response = { Response.status(status).entity(msg).`type`(MediaType.TEXT_PLAIN).build() } + + /** + * There may be different duration labels in each batch. So we need to + * mark those missing duration label as '0d' to avoid UI rending error. + */ + def durationDataPadding( + values: Array[(Long, ju.Map[String, JLong])]): Array[(Long, Map[String, Double])] = { + val operationLabels = values.flatMap(_._2.keySet().asScala).toSet + values.map { case (xValue, yValue) => + val dataPadding = operationLabels.map { opLabel => + if (yValue.containsKey(opLabel)) { + (opLabel, yValue.get(opLabel).toDouble) + } else { + (opLabel, 0d) + } + } + (xValue, dataPadding.toMap) + } + } + + def detailsUINode(isMultiline: Boolean, message: String): Seq[Node] = { + if (isMultiline) { + // scalastyle:off + + +details + ++ + + // scalastyle:on + } else { + Seq.empty[Node] + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 4dc5349cb8ad3..ccaa70b9daae0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -721,19 +721,7 @@ private[ui] class TaskPagedTable( } else { error }) - val details = if (isMultiline) { - // scalastyle:off - - +details - ++ - - // scalastyle:on - } else { - "" - } + val details = UIUtils.detailsUINode(isMultiline, error) {errorSummary}{details} } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index ac431c90e14f7..a7d38e9b04b70 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -309,19 +309,7 @@ private[ui] class StagePagedTable( } else { failureReason }) - val details = if (isMultiline) { - // scalastyle:off - - +details - ++ - - // scalastyle:on - } else { - "" - } + val details = UIUtils.detailsUINode(isMultiline, failureReason) {failureReasonSummary}{details} } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 68e9313805e19..65ffa228eddec 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -489,7 +489,10 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegressionModel.setPredictionCol"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.setFeaturesCol"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.setLabelCol"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.setPredictionCol") + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.AFTSurvivalRegression.setPredictionCol"), + + // [SPARK-29543][SS][UI] Init structured streaming ui + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStartedEvent.this") ) // Exclude rules for 2.4.x diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 0e0a8143532d5..e13d65bf81821 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1150,6 +1150,18 @@ object SQLConf { .booleanConf .createWithDefault(true) + val STREAMING_UI_ENABLED = + buildConf("spark.sql.streaming.ui.enabled") + .doc("Whether to run the structured streaming UI for the Spark application.") + .booleanConf + .createWithDefault(true) + + val STREAMING_UI_INACTIVE_QUERY_RETENTION = + buildConf("spark.sql.streaming.ui.numInactiveQueries") + .doc("The number of inactive queries to retain for structured streaming ui.") + .intConf + .createWithDefault(100) + val VARIABLE_SUBSTITUTE_ENABLED = buildConf("spark.sql.variable.substitute") .doc("This enables substitution using syntax like ${var} ${system:var} and ${env:var}.") @@ -2262,6 +2274,10 @@ class SQLConf extends Serializable with Logging { def isUnsupportedOperationCheckEnabled: Boolean = getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED) + def isStreamingUIEnabled: Boolean = getConf(STREAMING_UI_ENABLED) + + def streamingUIInactiveQueryRetention: Int = getConf(STREAMING_UI_INACTIVE_QUERY_RETENTION) + def streamingFileCommitProtocolClass: String = getConf(STREAMING_FILE_COMMIT_PROTOCOL_CLASS) def fileSinkLogDeletion: Boolean = getConf(FILE_SINK_LOG_DELETION) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 71bcd53435850..f20291e11fd70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -148,8 +148,8 @@ trait ProgressReporter extends Logging { currentTriggerEndTimestamp = triggerClock.getTimeMillis() val executionStats = extractExecutionStats(hasNewData) - val processingTimeSec = Math.max(1L, - currentTriggerEndTimestamp - currentTriggerStartTimestamp).toDouble / MILLIS_PER_SECOND + val processingTimeMills = currentTriggerEndTimestamp - currentTriggerStartTimestamp + val processingTimeSec = Math.max(1L, processingTimeMills).toDouble / MILLIS_PER_SECOND val inputTimeSec = if (lastTriggerStartTimestamp >= 0) { (currentTriggerStartTimestamp - lastTriggerStartTimestamp).toDouble / MILLIS_PER_SECOND @@ -181,6 +181,7 @@ trait ProgressReporter extends Logging { name = name, timestamp = formatTimestamp(currentTriggerStartTimestamp), batchId = currentBatchId, + batchDuration = processingTimeMills, durationMs = new java.util.HashMap(currentDurationsMs.toMap.mapValues(long2Long).asJava), eventTime = new java.util.HashMap(executionStats.eventTimeStats.asJava), stateOperators = executionStats.stateOperators.toArray, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 6dff5c6f26ee7..ed908a8bad483 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -307,7 +307,8 @@ abstract class StreamExecution( } // `postEvent` does not throw non fatal exception. - postEvent(new QueryStartedEvent(id, runId, name)) + val submissionTime = triggerClock.getTimeMillis() + postEvent(new QueryStartedEvent(id, runId, name, submissionTime)) // Unblock starting thread startLatch.countDown() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index de3805e105802..fefd72dcf1752 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.internal import java.net.URL -import java.util.{Locale, UUID} +import java.util.UUID import java.util.concurrent.ConcurrentHashMap import javax.annotation.concurrent.GuardedBy @@ -36,6 +36,8 @@ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.streaming.StreamingQueryListener +import org.apache.spark.sql.streaming.ui.{StreamingQueryStatusListener, StreamingQueryTab} import org.apache.spark.status.ElementTrackingStore import org.apache.spark.util.Utils @@ -138,6 +140,21 @@ private[sql] class SharedState( statusStore } + /** + * A [[StreamingQueryListener]] for structured streaming ui, it contains all streaming query ui + * data to show. + */ + lazy val streamingQueryStatusListener: Option[StreamingQueryStatusListener] = { + val sqlConf = SQLConf.get + if (sqlConf.isStreamingUIEnabled) { + val statusListener = new StreamingQueryStatusListener(sqlConf) + sparkContext.ui.foreach(new StreamingQueryTab(statusListener, _)) + Some(statusListener) + } else { + None + } + } + /** * A catalog that interacts with external systems. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index cc81cf6cfafec..dd842cd1a3e99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -82,13 +82,15 @@ object StreamingQueryListener { * @param id A unique query id that persists across restarts. See `StreamingQuery.id()`. * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`. * @param name User-specified name of the query, null if not specified. + * @param submissionTime The timestamp to start a query. * @since 2.1.0 */ @Evolving class QueryStartedEvent private[sql]( val id: UUID, val runId: UUID, - val name: String) extends Event + val name: String, + val submissionTime: Long) extends Event /** * Event representing any progress updates in a query. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index 810f4a182fca6..4d0d8ffd959c6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} @@ -37,7 +38,7 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS -import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} +import org.apache.spark.util.{Clock, SystemClock, Utils} /** * A class to manage all the [[StreamingQuery]] active in a `SparkSession`. @@ -68,6 +69,9 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo logInfo(s"Registered listener ${listener.getClass.getName}") }) } + sparkSession.sharedState.streamingQueryStatusListener.foreach { listener => + addListener(listener) + } } catch { case e: Exception => throw new SparkException("Exception when registering StreamingQueryListener", e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index a9681dbd0c676..13b506b60a126 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -85,6 +85,7 @@ class StateOperatorProgress private[sql]( * case of retries after a failure a given batchId my be executed more than once. * Similarly, when there is no data to be processed, the batchId will not be * incremented. + * @param batchDuration The process duration of each batch. * @param durationMs The amount of time taken to perform various operations in milliseconds. * @param eventTime Statistics of event time seen in this batch. It may contain the following keys: * {{{ @@ -105,6 +106,7 @@ class StreamingQueryProgress private[sql]( val name: String, val timestamp: String, val batchId: Long, + val batchDuration: Long, val durationMs: ju.Map[String, JLong], val eventTime: ju.Map[String, String], val stateOperators: Array[StateOperatorProgress], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala new file mode 100644 index 0000000000000..650f64fe1688c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala @@ -0,0 +1,147 @@ +/* + * 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.streaming.ui + +import java.text.SimpleDateFormat +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.commons.lang3.StringEscapeUtils + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone +import org.apache.spark.sql.streaming.ui.UIUtils._ +import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} + +private[ui] class StreamingQueryPage(parent: StreamingQueryTab) + extends WebUIPage("") with Logging { + val df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") + df.setTimeZone(getTimeZone("UTC")) + + override def render(request: HttpServletRequest): Seq[Node] = { + val content = generateStreamingQueryTable(request) + SparkUIUtils.headerSparkPage(request, "Streaming Query", content, parent) + } + + def generateDataRow(request: HttpServletRequest, queryActive: Boolean) + (query: StreamingQueryUIData): Seq[Node] = { + + def details(detail: Any): Seq[Node] = { + if (queryActive) { + return Seq.empty[Node] + } + val detailString = detail.asInstanceOf[String] + val isMultiline = detailString.indexOf('\n') >= 0 + val summary = StringEscapeUtils.escapeHtml4( + if (isMultiline) detailString.substring(0, detailString.indexOf('\n')) else detailString + ) + val details = SparkUIUtils.detailsUINode(isMultiline, detailString) + {summary}{details} + } + + val statisticsLink = "%s/%s/statistics?id=%s" + .format(SparkUIUtils.prependBaseUri(request, parent.basePath), parent.prefix, query.runId) + + val name = UIUtils.getQueryName(query) + val status = UIUtils.getQueryStatus(query) + val duration = if (queryActive) { + SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - query.submissionTime) + } else { + withNoProgress(query, { + val endTimeMs = query.lastProgress.timestamp + SparkUIUtils.formatDurationVerbose(df.parse(endTimeMs).getTime - query.submissionTime) + }, "-") + } + + + {name} + {status} + {query.id} + {query.runId} + {SparkUIUtils.formatDate(query.submissionTime)} + {duration} + {withNoProgress(query, { + (query.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / + query.recentProgress.length).formatted("%.2f") }, "NaN")} + + {withNoProgress(query, { + (query.recentProgress.map(p => withNumberInvalid(p.processedRowsPerSecond)).sum / + query.recentProgress.length).formatted("%.2f") }, "NaN")} + + {withNoProgress(query, { query.lastProgress.batchId }, "NaN")} + {details(query.exception.getOrElse("-"))} + + } + + private def generateStreamingQueryTable(request: HttpServletRequest): Seq[Node] = { + val (activeQueries, inactiveQueries) = parent.statusListener.allQueryStatus + .partition(_.isActive) + val activeQueryTables = if (activeQueries.nonEmpty) { + val headerRow = Seq( + "Name", "Status", "Id", "Run ID", "Submitted Time", "Duration", "Avg Input /sec", + "Avg Process /sec", "Lastest Batch") + + Some(SparkUIUtils.listingTable(headerRow, generateDataRow(request, queryActive = true), + activeQueries, true, None, Seq(null), false)) + } else { + None + } + + val inactiveQueryTables = if (inactiveQueries.nonEmpty) { + val headerRow = Seq( + "Name", "Status", "Id", "Run ID", "Submitted Time", "Duration", "Avg Input /sec", + "Avg Process /sec", "Lastest Batch", "Error") + + Some(SparkUIUtils.listingTable(headerRow, generateDataRow(request, queryActive = false), + inactiveQueries, true, None, Seq(null), false)) + } else { + None + } + + // scalastyle:off + val content = + +
    + + Active Streaming Queries ({activeQueries.length}) +
    +
    ++ +
    +
      + {activeQueryTables.getOrElse(Seq.empty[Node])} +
    +
    ++ + +
    + + Completed Streaming Queries ({inactiveQueries.length}) +
    +
    ++ +
    +
      + {inactiveQueryTables.getOrElse(Seq.empty[Node])} +
    +
    + // scalastyle:on + + content + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala new file mode 100644 index 0000000000000..56672ce328bff --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatisticsPage.scala @@ -0,0 +1,271 @@ +/* + * 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.streaming.ui + +import java.{util => ju} +import java.lang.{Long => JLong} +import java.text.SimpleDateFormat +import java.util.UUID +import javax.servlet.http.HttpServletRequest + +import scala.xml.{Node, Unparsed} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.util.DateTimeUtils.getTimeZone +import org.apache.spark.sql.streaming.ui.UIUtils._ +import org.apache.spark.ui.{GraphUIData, JsCollector, UIUtils => SparkUIUtils, WebUIPage} + +private[ui] class StreamingQueryStatisticsPage(parent: StreamingQueryTab) + extends WebUIPage("statistics") with Logging { + val df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") + df.setTimeZone(getTimeZone("UTC")) + + def generateLoadResources(request: HttpServletRequest): Seq[Node] = { + // scalastyle:off + + + + + // scalastyle:on + } + + override def render(request: HttpServletRequest): Seq[Node] = { + val parameterId = request.getParameter("id") + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + + val query = parent.statusListener.allQueryStatus.find { case q => + q.runId.equals(UUID.fromString(parameterId)) + }.getOrElse(throw new IllegalArgumentException(s"Failed to find streaming query $parameterId")) + + val resources = generateLoadResources(request) + val basicInfo = generateBasicInfo(query) + val content = + resources ++ + basicInfo ++ + generateStatTable(query) + SparkUIUtils.headerSparkPage(request, "Streaming Query Statistics", content, parent) + } + + def generateTimeMap(times: Seq[Long]): Seq[Node] = { + val js = "var timeFormat = {};\n" + times.map { time => + val formattedTime = SparkUIUtils.formatBatchTime(time, 1, showYYYYMMSS = false) + s"timeFormat[$time] = '$formattedTime';" + }.mkString("\n") + + + } + + def generateVar(values: Array[(Long, ju.Map[String, JLong])]): Seq[Node] = { + val durationDataPadding = SparkUIUtils.durationDataPadding(values) + val js = "var timeToValues = {};\n" + durationDataPadding.map { case (x, y) => + val s = y.toSeq.sortBy(_._1).map(e => s""""${e._2}"""").mkString("[", ",", "]") + s"""timeToValues["${SparkUIUtils.formatBatchTime(x, 1, showYYYYMMSS = false)}"] = $s;""" + }.mkString("\n") + + + } + + def generateBasicInfo(query: StreamingQueryUIData): Seq[Node] = { + val duration = if (query.isActive) { + SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - query.submissionTime) + } else { + withNoProgress(query, { + val end = query.lastProgress.timestamp + val start = query.recentProgress.head.timestamp + SparkUIUtils.formatDurationVerbose( + df.parse(end).getTime - df.parse(start).getTime) + }, "-") + } + + val name = UIUtils.getQueryName(query) + val numBatches = withNoProgress(query, { query.lastProgress.batchId + 1L }, 0) +
    Running batches for + + {duration} + + since + + {SparkUIUtils.formatDate(query.submissionTime)} + + ({numBatches} completed batches) +
    +
    +
    Name: {name}
    +
    Id: {query.id}
    +
    RunId: {query.runId}
    +
    + } + + def generateStatTable(query: StreamingQueryUIData): Seq[Node] = { + val batchTimes = withNoProgress(query, + query.recentProgress.map(p => df.parse(p.timestamp).getTime), Array.empty[Long]) + val minBatchTime = + withNoProgress(query, df.parse(query.recentProgress.head.timestamp).getTime, 0L) + val maxBatchTime = + withNoProgress(query, df.parse(query.lastProgress.timestamp).getTime, 0L) + val maxRecordRate = + withNoProgress(query, query.recentProgress.map(_.inputRowsPerSecond).max, 0L) + val minRecordRate = 0L + val maxProcessRate = + withNoProgress(query, query.recentProgress.map(_.processedRowsPerSecond).max, 0L) + + val minProcessRate = 0L + val maxRows = withNoProgress(query, query.recentProgress.map(_.numInputRows).max, 0L) + val minRows = 0L + val maxBatchDuration = withNoProgress(query, query.recentProgress.map(_.batchDuration).max, 0L) + val minBatchDuration = 0L + + val inputRateData = withNoProgress(query, + query.recentProgress.map(p => (df.parse(p.timestamp).getTime, + withNumberInvalid { p.inputRowsPerSecond })), Array.empty[(Long, Double)]) + val processRateData = withNoProgress(query, + query.recentProgress.map(p => (df.parse(p.timestamp).getTime, + withNumberInvalid { p.processedRowsPerSecond })), Array.empty[(Long, Double)]) + val inputRowsData = withNoProgress(query, + query.recentProgress.map(p => (df.parse(p.timestamp).getTime, + withNumberInvalid { p.numInputRows })), Array.empty[(Long, Double)]) + val batchDurations = withNoProgress(query, + query.recentProgress.map(p => (df.parse(p.timestamp).getTime, + withNumberInvalid { p.batchDuration })), Array.empty[(Long, Double)]) + val operationDurationData = withNoProgress(query, query.recentProgress.map { p => + val durationMs = p.durationMs + // remove "triggerExecution" as it count the other operation duration. + durationMs.remove("triggerExecution") + (df.parse(p.timestamp).getTime, durationMs)}, Array.empty[(Long, ju.Map[String, JLong])]) + + val jsCollector = new JsCollector + val graphUIDataForInputRate = + new GraphUIData( + "input-rate-timeline", + "input-rate-histogram", + inputRateData, + minBatchTime, + maxBatchTime, + minRecordRate, + maxRecordRate, + "records/sec") + graphUIDataForInputRate.generateDataJs(jsCollector) + + val graphUIDataForProcessRate = + new GraphUIData( + "process-rate-timeline", + "process-rate-histogram", + processRateData, + minBatchTime, + maxBatchTime, + minProcessRate, + maxProcessRate, + "records/sec") + graphUIDataForProcessRate.generateDataJs(jsCollector) + + val graphUIDataForInputRows = + new GraphUIData( + "input-rows-timeline", + "input-rows-histogram", + inputRowsData, + minBatchTime, + maxBatchTime, + minRows, + maxRows, + "records") + graphUIDataForInputRows.generateDataJs(jsCollector) + + val graphUIDataForBatchDuration = + new GraphUIData( + "batch-duration-timeline", + "batch-duration-histogram", + batchDurations, + minBatchTime, + maxBatchTime, + minBatchDuration, + maxBatchDuration, + "ms") + graphUIDataForBatchDuration.generateDataJs(jsCollector) + + val graphUIDataForDuration = + new GraphUIData( + "duration-area-stack", + "", + Seq.empty[(Long, Double)], + 0L, + 0L, + 0L, + 0L, + "ms") + + val table = + // scalastyle:off + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    TimelinesHistograms
    +
    +
    Input Rate {SparkUIUtils.tooltip("The aggregate (across all sources) rate of data arriving.", "right")}
    +
    +
    {graphUIDataForInputRate.generateTimelineHtml(jsCollector)}{graphUIDataForInputRate.generateHistogramHtml(jsCollector)}
    +
    +
    Process Rate {SparkUIUtils.tooltip("The aggregate (across all sources) rate at which Spark is processing data.", "right")}
    +
    +
    {graphUIDataForProcessRate.generateTimelineHtml(jsCollector)}{graphUIDataForProcessRate.generateHistogramHtml(jsCollector)}
    +
    +
    Input Rows {SparkUIUtils.tooltip("The aggregate (across all sources) number of records processed in a trigger.", "right")}
    +
    +
    {graphUIDataForInputRows.generateTimelineHtml(jsCollector)}{graphUIDataForInputRows.generateHistogramHtml(jsCollector)}
    +
    +
    Batch Duration {SparkUIUtils.tooltip("The process duration of each batch.", "right")}
    +
    +
    {graphUIDataForBatchDuration.generateTimelineHtml(jsCollector)}{graphUIDataForBatchDuration.generateHistogramHtml(jsCollector)}
    +
    +
    Operation Duration {SparkUIUtils.tooltip("The amount of time taken to perform various operations in milliseconds.", "right")}
    +
    +
    {graphUIDataForDuration.generateAreaStackHtmlWithData(jsCollector, operationDurationData)}
    + // scalastyle:on + + generateVar(operationDurationData) ++ generateTimeMap(batchTimes) ++ table ++ jsCollector.toHtml + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala new file mode 100644 index 0000000000000..db085dbe87ec4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala @@ -0,0 +1,122 @@ +/* + * 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.streaming.ui + +import java.text.SimpleDateFormat +import java.util.UUID +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress} + +/** + * A customized StreamingQueryListener used in structured streaming UI, which contains all + * UI data for both active and inactive query. + * TODO: Add support for history server. + */ +private[sql] class StreamingQueryStatusListener(sqlConf: SQLConf) extends StreamingQueryListener { + + private val timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 + timestampFormat.setTimeZone(DateTimeUtils.getTimeZone("UTC")) + + /** + * We use runId as the key here instead of id in active query status map, + * because the runId is unique for every started query, even it its a restart. + */ + private[ui] val activeQueryStatus = new ConcurrentHashMap[UUID, StreamingQueryUIData]() + private[ui] val inactiveQueryStatus = new mutable.Queue[StreamingQueryUIData]() + + private val streamingProgressRetention = sqlConf.streamingProgressRetention + private val inactiveQueryStatusRetention = sqlConf.streamingUIInactiveQueryRetention + + override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = { + activeQueryStatus.putIfAbsent(event.runId, + new StreamingQueryUIData(event.name, event.id, event.runId, event.submissionTime)) + } + + override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = { + val batchTimestamp = timestampFormat.parse(event.progress.timestamp).getTime + val queryStatus = activeQueryStatus.getOrDefault( + event.progress.runId, + new StreamingQueryUIData(event.progress.name, event.progress.id, event.progress.runId, + batchTimestamp)) + queryStatus.updateProcess(event.progress, streamingProgressRetention) + } + + override def onQueryTerminated( + event: StreamingQueryListener.QueryTerminatedEvent): Unit = synchronized { + val queryStatus = activeQueryStatus.remove(event.runId) + if (queryStatus != null) { + queryStatus.queryTerminated(event) + inactiveQueryStatus += queryStatus + while (inactiveQueryStatus.length >= inactiveQueryStatusRetention) { + inactiveQueryStatus.dequeue() + } + } + } + + def allQueryStatus: Seq[StreamingQueryUIData] = synchronized { + activeQueryStatus.values().asScala.toSeq ++ inactiveQueryStatus + } +} + +/** + * This class contains all message related to UI display, each instance corresponds to a single + * [[org.apache.spark.sql.streaming.StreamingQuery]]. + */ +private[ui] class StreamingQueryUIData( + val name: String, + val id: UUID, + val runId: UUID, + val submissionTime: Long) { + + /** Holds the most recent query progress updates. */ + private val progressBuffer = new mutable.Queue[StreamingQueryProgress]() + + private var _isActive = true + private var _exception: Option[String] = None + + def isActive: Boolean = synchronized { _isActive } + + def exception: Option[String] = synchronized { _exception } + + def queryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = synchronized { + _isActive = false + _exception = event.exception + } + + def updateProcess( + newProgress: StreamingQueryProgress, retentionNum: Int): Unit = progressBuffer.synchronized { + progressBuffer += newProgress + while (progressBuffer.length >= retentionNum) { + progressBuffer.dequeue() + } + } + + def recentProgress: Array[StreamingQueryProgress] = progressBuffer.synchronized { + progressBuffer.toArray + } + + def lastProgress: StreamingQueryProgress = progressBuffer.synchronized { + progressBuffer.lastOption.orNull + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala new file mode 100644 index 0000000000000..f909cfd97514e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala @@ -0,0 +1,39 @@ +/* + * 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.streaming.ui + +import org.apache.spark.internal.Logging +import org.apache.spark.ui.{SparkUI, SparkUITab} + +private[sql] class StreamingQueryTab( + val statusListener: StreamingQueryStatusListener, + sparkUI: SparkUI) extends SparkUITab(sparkUI, "StreamingQuery") with Logging { + + override val name = "Structured Streaming" + + val parent = sparkUI + + attachPage(new StreamingQueryPage(this)) + attachPage(new StreamingQueryStatisticsPage(this)) + parent.attachTab(this) + + parent.addStaticHandler(StreamingQueryTab.STATIC_RESOURCE_DIR, "/static/sql") +} + +object StreamingQueryTab { + private val STATIC_RESOURCE_DIR = "org/apache/spark/sql/execution/ui/static" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala new file mode 100644 index 0000000000000..57b9dec81f28a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/UIUtils.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming.ui + +private[ui] object UIUtils { + + /** + * Check whether `number` is valid, if not return 0.0d + */ + def withNumberInvalid(number: => Double): Double = { + if (number.isNaN || number.isInfinite) { + 0.0d + } else { + number + } + } + + /** + * Execute a block of code when there is already one completed batch in streaming query, + * otherwise return `default` value. + */ + def withNoProgress[T](query: StreamingQueryUIData, body: => T, default: T): T = { + if (query.lastProgress != null) { + body + } else { + default + } + } + + def getQueryName(query: StreamingQueryUIData): String = { + if (query.name == null || query.name.isEmpty) { + "" + } else { + query.name + } + } + + def getQueryStatus(query: StreamingQueryUIData): String = { + if (query.isActive) { + "RUNNING" + } else { + query.exception.map(_ => "FAILED").getOrElse("FINISHED") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 2f66dd3255b11..9d0f829ac9684 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -23,7 +23,6 @@ import scala.collection.mutable import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter -import org.scalatest.PrivateMethodTester._ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.concurrent.Waiters.Waiter @@ -34,6 +33,7 @@ import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQueryListener._ +import org.apache.spark.sql.streaming.ui.StreamingQueryStatusListener import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.util.JsonProtocol @@ -47,7 +47,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { after { spark.streams.active.foreach(_.stop()) assert(spark.streams.active.isEmpty) - assert(spark.streams.listListeners().isEmpty) + // Skip check default `StreamingQueryStatusListener` which is for streaming UI. + assert(spark.streams.listListeners() + .filterNot(_.isInstanceOf[StreamingQueryStatusListener]).isEmpty) // Make sure we don't leak any events to the next test spark.sparkContext.listenerBus.waitUntilEmpty() } @@ -252,8 +254,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(newEvent.name === event.name) } - testSerialization(new QueryStartedEvent(UUID.randomUUID, UUID.randomUUID, "name")) - testSerialization(new QueryStartedEvent(UUID.randomUUID, UUID.randomUUID, null)) + testSerialization(new QueryStartedEvent(UUID.randomUUID, UUID.randomUUID, "name", 1L)) + testSerialization(new QueryStartedEvent(UUID.randomUUID, UUID.randomUUID, null, 1L)) } test("QueryProgressEvent serialization") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index b6a6be2bb0312..6f00b528cb8bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -312,6 +312,7 @@ object StreamingQueryStatusAndProgressSuite { name = "myName", timestamp = "2016-12-05T20:54:20.827Z", batchId = 2L, + batchDuration = 0L, durationMs = new java.util.HashMap(Map("total" -> 0L).mapValues(long2Long).asJava), eventTime = new java.util.HashMap(Map( "max" -> "2016-12-05T20:54:20.827Z", @@ -346,6 +347,7 @@ object StreamingQueryStatusAndProgressSuite { name = null, // should not be present in the json timestamp = "2016-12-05T20:54:20.827Z", batchId = 2L, + batchDuration = 0L, durationMs = new java.util.HashMap(Map("total" -> 0L).mapValues(long2Long).asJava), // empty maps should be handled correctly eventTime = new java.util.HashMap(Map.empty[String, String].asJava), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 4121f499bd69c..77f5c856ff0f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -466,7 +466,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi val streamingTriggerDF = spark.createDataset(1 to 10).toDF val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF).toDF("value") - val progress = getFirstProgress(streamingInputDF.join(streamingInputDF, "value")) + val progress = getStreamingQuery(streamingInputDF.join(streamingInputDF, "value")) + .recentProgress.head assert(progress.numInputRows === 20) // data is read multiple times in self-joins assert(progress.sources.size === 1) assert(progress.sources(0).numInputRows === 20) @@ -479,7 +480,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi // Trigger input has 10 rows, static input has 2 rows, // therefore after the first trigger, the calculated input rows should be 10 - val progress = getFirstProgress(streamingInputDF.join(staticInputDF, "value")) + val progress = getStreamingQuery(streamingInputDF.join(staticInputDF, "value")) + .recentProgress.head assert(progress.numInputRows === 10) assert(progress.sources.size === 1) assert(progress.sources(0).numInputRows === 10) @@ -492,7 +494,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF) // After the first trigger, the calculated input rows should be 10 - val progress = getFirstProgress(streamingInputDF) + val progress = getStreamingQuery(streamingInputDF).recentProgress.head assert(progress.numInputRows === 10) assert(progress.sources.size === 1) assert(progress.sources(0).numInputRows === 10) @@ -1120,12 +1122,12 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi StreamingExecutionRelation(source, spark) } - /** Returns the query progress at the end of the first trigger of streaming DF */ - private def getFirstProgress(streamingDF: DataFrame): StreamingQueryProgress = { + /** Returns the query at the end of the first trigger of streaming DF */ + private def getStreamingQuery(streamingDF: DataFrame): StreamingQuery = { try { val q = streamingDF.writeStream.format("memory").queryName("test").start() q.processAllAvailable() - q.recentProgress.head + q } finally { spark.streams.active.map(_.stop()) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala new file mode 100644 index 0000000000000..de43e470e8e13 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming.ui + +import java.util.{Locale, UUID} +import javax.servlet.http.HttpServletRequest + +import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.scalatest.BeforeAndAfter +import scala.xml.Node + +import org.apache.spark.sql.streaming.StreamingQueryProgress +import org.apache.spark.sql.test.SharedSparkSession + +class StreamingQueryPageSuite extends SharedSparkSession with BeforeAndAfter { + + test("correctly display streaming query page") { + val id = UUID.randomUUID() + val request = mock(classOf[HttpServletRequest]) + val tab = mock(classOf[StreamingQueryTab], RETURNS_SMART_NULLS) + val statusListener = mock(classOf[StreamingQueryStatusListener], RETURNS_SMART_NULLS) + when(tab.appName).thenReturn("testing") + when(tab.headerTabs).thenReturn(Seq.empty) + when(tab.statusListener).thenReturn(statusListener) + + val streamQuery = createStreamQueryUIData(id) + when(statusListener.allQueryStatus).thenReturn(Seq(streamQuery)) + var html = renderStreamingQueryPage(request, tab) + .toString().toLowerCase(Locale.ROOT) + assert(html.contains("active streaming queries (1)")) + assert(html.contains("completed streaming queries (0)")) + + when(streamQuery.isActive).thenReturn(false) + when(streamQuery.exception).thenReturn(None) + html = renderStreamingQueryPage(request, tab) + .toString().toLowerCase(Locale.ROOT) + assert(html.contains("active streaming queries (0)")) + assert(html.contains("completed streaming queries (1)")) + assert(html.contains("finished")) + + when(streamQuery.isActive).thenReturn(false) + when(streamQuery.exception).thenReturn(Option("exception in query")) + html = renderStreamingQueryPage(request, tab) + .toString().toLowerCase(Locale.ROOT) + assert(html.contains("active streaming queries (0)")) + assert(html.contains("completed streaming queries (1)")) + assert(html.contains("failed")) + assert(html.contains("exception in query")) + } + + test("correctly display streaming query statistics page") { + val id = UUID.randomUUID() + val request = mock(classOf[HttpServletRequest]) + val tab = mock(classOf[StreamingQueryTab], RETURNS_SMART_NULLS) + val statusListener = mock(classOf[StreamingQueryStatusListener], RETURNS_SMART_NULLS) + when(request.getParameter("id")).thenReturn(id.toString) + when(tab.appName).thenReturn("testing") + when(tab.headerTabs).thenReturn(Seq.empty) + when(tab.statusListener).thenReturn(statusListener) + + val streamQuery = createStreamQueryUIData(id) + when(statusListener.allQueryStatus).thenReturn(Seq(streamQuery)) + val html = renderStreamingQueryStatisticsPage(request, tab) + .toString().toLowerCase(Locale.ROOT) + + assert(html.contains("name: query<")) + assert(html.contains("""{"x": 1001898000100, "y": 10.0}""")) + assert(html.contains("""{"x": 1001898000100, "y": 12.0}""")) + assert(html.contains("(3 completed batches)")) + } + + private def createStreamQueryUIData(id: UUID): StreamingQueryUIData = { + val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) + when(progress.timestamp).thenReturn("2001-10-01T01:00:00.100Z") + when(progress.inputRowsPerSecond).thenReturn(10.0) + when(progress.processedRowsPerSecond).thenReturn(12.0) + when(progress.batchId).thenReturn(2) + when(progress.prettyJson).thenReturn("""{"a":1}""") + + val streamQuery = mock(classOf[StreamingQueryUIData], RETURNS_SMART_NULLS) + when(streamQuery.isActive).thenReturn(true) + when(streamQuery.name).thenReturn("query") + when(streamQuery.id).thenReturn(id) + when(streamQuery.runId).thenReturn(id) + when(streamQuery.submissionTime).thenReturn(1L) + when(streamQuery.lastProgress).thenReturn(progress) + when(streamQuery.recentProgress).thenReturn(Array(progress)) + when(streamQuery.exception).thenReturn(None) + + streamQuery + } + + /** + * Render a stage page started with the given conf and return the HTML. + * This also runs a dummy execution page to populate the page with useful content. + */ + private def renderStreamingQueryPage( + request: HttpServletRequest, + tab: StreamingQueryTab): Seq[Node] = { + val page = new StreamingQueryPage(tab) + page.render(request) + } + + private def renderStreamingQueryStatisticsPage( + request: HttpServletRequest, + tab: StreamingQueryTab): Seq[Node] = { + val page = new StreamingQueryStatisticsPage(tab) + page.render(request) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala new file mode 100644 index 0000000000000..bd74ed340b408 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala @@ -0,0 +1,101 @@ +/* + * 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.streaming.ui + +import java.util.UUID + +import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} + +import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress, StreamTest} +import org.apache.spark.sql.streaming + +class StreamingQueryStatusListenerSuite extends StreamTest { + + test("onQueryStarted, onQueryProgress, onQueryTerminated") { + val listener = new StreamingQueryStatusListener(spark.sqlContext.conf) + + // hanlde query started event + val id = UUID.randomUUID() + val runId = UUID.randomUUID() + val startEvent = new StreamingQueryListener.QueryStartedEvent(id, runId, "test", 1L) + listener.onQueryStarted(startEvent) + + // result checking + assert(listener.activeQueryStatus.size() == 1) + assert(listener.activeQueryStatus.get(runId).name == "test") + + // handle query progress event + val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) + when(progress.id).thenReturn(id) + when(progress.runId).thenReturn(runId) + when(progress.timestamp).thenReturn("2001-10-01T01:00:00.100Z") + when(progress.inputRowsPerSecond).thenReturn(10.0) + when(progress.processedRowsPerSecond).thenReturn(12.0) + when(progress.batchId).thenReturn(2) + when(progress.prettyJson).thenReturn("""{"a":1}""") + val processEvent = new streaming.StreamingQueryListener.QueryProgressEvent(progress) + listener.onQueryProgress(processEvent) + + // result checking + val activeQuery = listener.activeQueryStatus.get(runId) + assert(activeQuery.isActive) + assert(activeQuery.recentProgress.length == 1) + assert(activeQuery.lastProgress.id == id) + assert(activeQuery.lastProgress.runId == runId) + assert(activeQuery.lastProgress.timestamp == "2001-10-01T01:00:00.100Z") + assert(activeQuery.lastProgress.inputRowsPerSecond == 10.0) + assert(activeQuery.lastProgress.processedRowsPerSecond == 12.0) + assert(activeQuery.lastProgress.batchId == 2) + assert(activeQuery.lastProgress.prettyJson == """{"a":1}""") + + // handle terminate event + val terminateEvent = new StreamingQueryListener.QueryTerminatedEvent(id, runId, None) + listener.onQueryTerminated(terminateEvent) + + assert(!listener.inactiveQueryStatus.head.isActive) + assert(listener.inactiveQueryStatus.head.runId == runId) + assert(listener.inactiveQueryStatus.head.id == id) + } + + test("same query start multiple times") { + val listener = new StreamingQueryStatusListener(spark.sqlContext.conf) + + // handle first time start + val id = UUID.randomUUID() + val runId0 = UUID.randomUUID() + val startEvent0 = new StreamingQueryListener.QueryStartedEvent(id, runId0, "test", 1L) + listener.onQueryStarted(startEvent0) + + // handle terminate event + val terminateEvent0 = new StreamingQueryListener.QueryTerminatedEvent(id, runId0, None) + listener.onQueryTerminated(terminateEvent0) + + // handle second time start + val runId1 = UUID.randomUUID() + val startEvent1 = new StreamingQueryListener.QueryStartedEvent(id, runId1, "test", 1L) + listener.onQueryStarted(startEvent1) + + // result checking + assert(listener.activeQueryStatus.size() == 1) + assert(listener.inactiveQueryStatus.length == 1) + assert(listener.activeQueryStatus.containsKey(runId1)) + assert(listener.activeQueryStatus.get(runId1).id == id) + assert(listener.inactiveQueryStatus.head.runId == runId0) + assert(listener.inactiveQueryStatus.head.id == id) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala new file mode 100644 index 0000000000000..46f2eadc05835 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming.ui + +import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.scalatest.Matchers + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.streaming.StreamingQueryProgress + +class UIUtilsSuite extends SparkFunSuite with Matchers { + test("streaming query started with no batch completed") { + val query = mock(classOf[StreamingQueryUIData], RETURNS_SMART_NULLS) + when(query.lastProgress).thenReturn(null) + + assert(0 == UIUtils.withNoProgress(query, 1, 0)) + } + + test("streaming query started with at least one batch completed") { + val query = mock(classOf[StreamingQueryUIData], RETURNS_SMART_NULLS) + val progress = mock(classOf[StreamingQueryProgress], RETURNS_SMART_NULLS) + when(query.lastProgress).thenReturn(progress) + + assert(1 == UIUtils.withNoProgress(query, 1, 0)) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index adfda0c56585f..890a668275b81 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -375,21 +375,7 @@ private[ui] class SqlStatsPagedTable( } else { errorMessage }) - val details = if (isMultiline) { - // scalastyle:off - - + details - ++ - - // scalastyle:on - } else { - "" - } + val details = detailsUINode(isMultiline, errorMessage) {errorSummary}{details} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index e360b4a86d9d3..6c981b293ac76 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -33,7 +33,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext.rddToFileName import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.streaming.ui.UIUtils +import org.apache.spark.ui.{UIUtils => SparkUIUtils} import org.apache.spark.util.{CallSite, Utils} /** @@ -138,7 +138,7 @@ abstract class DStream[T: ClassTag] ( */ private def makeScope(time: Time): Option[RDDOperationScope] = { baseScope.map { bsJson => - val formattedBatchTime = UIUtils.formatBatchTime( + val formattedBatchTime = SparkUIUtils.formatBatchTime( time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) val bs = RDDOperationScope.fromJson(bsJson) val baseName = bs.name // e.g. countByWindow, "kafka stream [0]" diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 5d543c5fe1051..7eea57cc083ed 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -28,7 +28,7 @@ import org.apache.spark.internal.io.SparkHadoopWriterUtils import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.api.python.PythonDStream -import org.apache.spark.streaming.ui.UIUtils +import org.apache.spark.ui.{UIUtils => SparkUIUtils} import org.apache.spark.util.{EventLoop, ThreadUtils, Utils} @@ -230,7 +230,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { val oldProps = ssc.sparkContext.getLocalProperties try { ssc.sparkContext.setLocalProperties(Utils.cloneProperties(ssc.savedProperties.get())) - val formattedTime = UIUtils.formatBatchTime( + val formattedTime = SparkUIUtils.formatBatchTime( job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" val batchLinkText = s"[output operation ${job.outputOpId}, batch time ${formattedTime}]" diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala index f1070e9029cb5..b5a0e92e69c04 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala @@ -51,7 +51,7 @@ private[ui] abstract class BatchTableBase(tableId: String, batchInterval: Long) protected def baseRow(batch: BatchUIData): Seq[Node] = { val batchTime = batch.batchTime.milliseconds - val formattedBatchTime = UIUtils.formatBatchTime(batchTime, batchInterval) + val formattedBatchTime = SparkUIUtils.formatBatchTime(batchTime, batchInterval) val numRecords = batch.numRecords val schedulingDelay = batch.schedulingDelay val formattedSchedulingDelay = schedulingDelay.map(SparkUIUtils.formatDuration).getOrElse("-") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index 2c85d26738757..04cd063a28713 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -325,7 +325,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { throw new IllegalArgumentException(s"Missing id parameter") } val formattedBatchTime = - UIUtils.formatBatchTime(batchTime.milliseconds, streamingListener.batchDuration) + SparkUIUtils.formatBatchTime(batchTime.milliseconds, streamingListener.batchDuration) val batchUIData = streamingListener.getBatchUIData(batchTime).getOrElse { throw new IllegalArgumentException(s"Batch $formattedBatchTime does not exist") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 31ebb4ce53c17..d47287b6077f8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -20,78 +20,10 @@ package org.apache.spark.streaming.ui import java.util.concurrent.TimeUnit import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.ArrayBuffer import scala.xml.{Node, Unparsed} import org.apache.spark.internal.Logging -import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} - -/** - * A helper class to generate JavaScript and HTML for both timeline and histogram graphs. - * - * @param timelineDivId the timeline `id` used in the html `div` tag - * @param histogramDivId the timeline `id` used in the html `div` tag - * @param data the data for the graph - * @param minX the min value of X axis - * @param maxX the max value of X axis - * @param minY the min value of Y axis - * @param maxY the max value of Y axis - * @param unitY the unit of Y axis - * @param batchInterval if `batchInterval` is not None, we will draw a line for `batchInterval` in - * the graph - */ -private[ui] class GraphUIData( - timelineDivId: String, - histogramDivId: String, - data: Seq[(Long, Double)], - minX: Long, - maxX: Long, - minY: Double, - maxY: Double, - unitY: String, - batchInterval: Option[Double] = None) { - - private var dataJavaScriptName: String = _ - - def generateDataJs(jsCollector: JsCollector): Unit = { - val jsForData = data.map { case (x, y) => - s"""{"x": $x, "y": $y}""" - }.mkString("[", ",", "]") - dataJavaScriptName = jsCollector.nextVariableName - jsCollector.addPreparedStatement(s"var $dataJavaScriptName = $jsForData;") - } - - def generateTimelineHtml(jsCollector: JsCollector): Seq[Node] = { - jsCollector.addPreparedStatement(s"registerTimeline($minY, $maxY);") - if (batchInterval.isDefined) { - jsCollector.addStatement( - "drawTimeline(" + - s"'#$timelineDivId', $dataJavaScriptName, $minX, $maxX, $minY, $maxY, '$unitY'," + - s" ${batchInterval.get}" + - ");") - } else { - jsCollector.addStatement( - s"drawTimeline('#$timelineDivId', $dataJavaScriptName, $minX, $maxX, $minY, $maxY," + - s" '$unitY');") - } -
    - } - - def generateHistogramHtml(jsCollector: JsCollector): Seq[Node] = { - val histogramData = s"$dataJavaScriptName.map(function(d) { return d.y; })" - jsCollector.addPreparedStatement(s"registerHistogram($histogramData, $minY, $maxY);") - if (batchInterval.isDefined) { - jsCollector.addStatement( - "drawHistogram(" + - s"'#$histogramDivId', $histogramData, $minY, $maxY, '$unitY', ${batchInterval.get}" + - ");") - } else { - jsCollector.addStatement( - s"drawHistogram('#$histogramDivId', $histogramData, $minY, $maxY, '$unitY');") - } -
    - } -} +import org.apache.spark.ui.{GraphUIData, JsCollector, UIUtils => SparkUIUtils, WebUIPage} /** * A helper class for "scheduling delay", "processing time" and "total delay" to generate data that @@ -164,8 +96,8 @@ private[ui] class StreamingPage(parent: StreamingTab) private def generateLoadResources(request: HttpServletRequest): Seq[Node] = { // scalastyle:off - - + + // scalastyle:on } @@ -201,7 +133,7 @@ private[ui] class StreamingPage(parent: StreamingTab) private def generateTimeMap(times: Seq[Long]): Seq[Node] = { val js = "var timeFormat = {};\n" + times.map { time => val formattedTime = - UIUtils.formatBatchTime(time, listener.batchDuration, showYYYYMMSS = false) + SparkUIUtils.formatBatchTime(time, listener.batchDuration, showYYYYMMSS = false) s"timeFormat[$time] = '$formattedTime';" }.mkString("\n") @@ -544,52 +476,3 @@ private[ui] object StreamingPage { } -/** - * A helper class that allows the user to add JavaScript statements which will be executed when the - * DOM has finished loading. - */ -private[ui] class JsCollector { - - private var variableId = 0 - - /** - * Return the next unused JavaScript variable name - */ - def nextVariableName: String = { - variableId += 1 - "v" + variableId - } - - /** - * JavaScript statements that will execute before `statements` - */ - private val preparedStatements = ArrayBuffer[String]() - - /** - * JavaScript statements that will execute after `preparedStatements` - */ - private val statements = ArrayBuffer[String]() - - def addPreparedStatement(js: String): Unit = { - preparedStatements += js - } - - def addStatement(js: String): Unit = { - statements += js - } - - /** - * Generate a html snippet that will execute all scripts when the DOM has finished loading. - */ - def toHtml: Seq[Node] = { - val js = - s""" - |$$(document).ready(function() { - | ${preparedStatements.mkString("\n")} - | ${statements.mkString("\n")} - |});""".stripMargin - - - } -} - diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 3ecc448e46c27..d616b47117f1c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -28,7 +28,7 @@ import org.apache.spark.ui.{SparkUI, SparkUITab} private[spark] class StreamingTab(val ssc: StreamingContext, sparkUI: SparkUI) extends SparkUITab(sparkUI, "streaming") with Logging { - private val STATIC_RESOURCE_DIR = "org/apache/spark/streaming/ui/static" + private val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" val parent = sparkUI val listener = ssc.progressListener diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala index c21912ab2816c..dc1af0a940ec7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala @@ -17,14 +17,14 @@ package org.apache.spark.streaming.ui -import java.text.SimpleDateFormat -import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit import scala.xml.Node import org.apache.commons.text.StringEscapeUtils +import org.apache.spark.ui.{ UIUtils => SparkUIUtils } + private[streaming] object UIUtils { /** @@ -78,59 +78,6 @@ private[streaming] object UIUtils { case TimeUnit.DAYS => milliseconds / 1000.0 / 60.0 / 60.0 / 24.0 } - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. - private val batchTimeFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = - new SimpleDateFormat("yyyy/MM/dd HH:mm:ss", Locale.US) - } - - private val batchTimeFormatWithMilliseconds = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = - new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS", Locale.US) - } - - /** - * If `batchInterval` is less than 1 second, format `batchTime` with milliseconds. Otherwise, - * format `batchTime` without milliseconds. - * - * @param batchTime the batch time to be formatted - * @param batchInterval the batch interval - * @param showYYYYMMSS if showing the `yyyy/MM/dd` part. If it's false, the return value wll be - * only `HH:mm:ss` or `HH:mm:ss.SSS` depending on `batchInterval` - * @param timezone only for test - */ - def formatBatchTime( - batchTime: Long, - batchInterval: Long, - showYYYYMMSS: Boolean = true, - timezone: TimeZone = null): String = { - val oldTimezones = - (batchTimeFormat.get.getTimeZone, batchTimeFormatWithMilliseconds.get.getTimeZone) - if (timezone != null) { - batchTimeFormat.get.setTimeZone(timezone) - batchTimeFormatWithMilliseconds.get.setTimeZone(timezone) - } - try { - val formattedBatchTime = - if (batchInterval < 1000) { - batchTimeFormatWithMilliseconds.get.format(batchTime) - } else { - // If batchInterval >= 1 second, don't show milliseconds - batchTimeFormat.get.format(batchTime) - } - if (showYYYYMMSS) { - formattedBatchTime - } else { - formattedBatchTime.substring(formattedBatchTime.indexOf(' ') + 1) - } - } finally { - if (timezone != null) { - batchTimeFormat.get.setTimeZone(oldTimezones._1) - batchTimeFormatWithMilliseconds.get.setTimeZone(oldTimezones._2) - } - } - } - def createOutputOperationFailureForUI(failure: String): String = { if (failure.startsWith("org.apache.spark.Spark")) { // SparkException or SparkDriverExecutionException @@ -164,19 +111,7 @@ private[streaming] object UIUtils { } else { failureReason } - val details = if (isMultiline) { - // scalastyle:off - - +details - ++ - - // scalastyle:on - } else { - "" - } + val details = SparkUIUtils.detailsUINode(isMultiline, failureDetails) if (rowspan == 1) { {failureReasonSummary}{details} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala index 1bb41164d362e..36036fcd44b04 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala @@ -19,12 +19,10 @@ package org.apache.spark.streaming import scala.collection.mutable.ArrayBuffer -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} - import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.streaming.dstream.DStream -import org.apache.spark.streaming.ui.UIUtils +import org.apache.spark.ui.{UIUtils => SparkUIUtils} import org.apache.spark.util.ManualClock /** @@ -214,7 +212,7 @@ class DStreamScopeSuite rddScope: RDDOperationScope, batchTime: Long): Unit = { val (baseScopeId, baseScopeName) = (baseScope.id, baseScope.name) - val formattedBatchTime = UIUtils.formatBatchTime( + val formattedBatchTime = SparkUIUtils.formatBatchTime( batchTime, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) assert(rddScope.id === s"${baseScopeId}_$batchTime") assert(rddScope.name.replaceAll("\\n", " ") === s"$baseScopeName @ $formattedBatchTime") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala index d3ca2b58f36c2..576083723f8bd 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/UIUtilsSuite.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import org.scalatest.Matchers import org.apache.spark.SparkFunSuite +import org.apache.spark.ui.{UIUtils => SparkUIUtils} class UIUtilsSuite extends SparkFunSuite with Matchers{ @@ -70,10 +71,13 @@ class UIUtilsSuite extends SparkFunSuite with Matchers{ test("formatBatchTime") { val tzForTest = TimeZone.getTimeZone("America/Los_Angeles") val batchTime = 1431637480452L // Thu May 14 14:04:40 PDT 2015 - assert("2015/05/14 14:04:40" === UIUtils.formatBatchTime(batchTime, 1000, timezone = tzForTest)) + assert("2015/05/14 14:04:40" === + SparkUIUtils.formatBatchTime(batchTime, 1000, timezone = tzForTest)) assert("2015/05/14 14:04:40.452" === - UIUtils.formatBatchTime(batchTime, 999, timezone = tzForTest)) - assert("14:04:40" === UIUtils.formatBatchTime(batchTime, 1000, false, timezone = tzForTest)) - assert("14:04:40.452" === UIUtils.formatBatchTime(batchTime, 999, false, timezone = tzForTest)) + SparkUIUtils.formatBatchTime(batchTime, 999, timezone = tzForTest)) + assert("14:04:40" === + SparkUIUtils.formatBatchTime(batchTime, 1000, false, timezone = tzForTest)) + assert("14:04:40.452" === + SparkUIUtils.formatBatchTime(batchTime, 999, false, timezone = tzForTest)) } } From e5c7f8908276e4ef20828c746c50fed30f26911c Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 29 Jan 2020 15:37:11 -0800 Subject: [PATCH 1243/1387] [SPARK-30529][CORE] Improve error messages when Executor dies before registering with driver MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … ### What changes were proposed in this pull request? If the resource discovery goes bad, like it doesn't return enough GPUs, currently it just throws an exception. This is hard for users to see because you have to go find the executor logs and its not reported back to the driver. On yarn if you exit explicitly then the driver logs show the error thrown and its much more useful. On yarn with the explicit exit with non-zero it also goes against failed executor launch attempts and the application will eventually exit. so if its fundamentally a bad configuration or bad discovery script it won't just hang forever. I also tested on k8s and standalone and the behaviors there don't change, the executor cleanly exit with an error message in the logs. The standalone ui makes it easy to see failed executors. ### Why are the changes needed? better user experience. ### Does this PR introduce any user-facing change? no api changes ### How was this patch tested? ran unit tests and manually tested on yarn, standalone, and k8s. Closes #27385 from tgravescs/SPARK-30529. Authored-by: Thomas Graves Signed-off-by: Dongjoon Hyun --- .../spark/executor/CoarseGrainedExecutorBackend.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) 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 f56e7c6d78c48..511c63ad3d551 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -78,7 +78,13 @@ private[spark] class CoarseGrainedExecutorBackend( override def onStart(): Unit = { logInfo("Connecting to driver: " + driverUrl) - val resources = parseOrFindResources(resourcesFileOpt) + var resources = Map.empty[String, ResourceInformation] + try { + resources = parseOrFindResources(resourcesFileOpt) + } catch { + case NonFatal(e) => + exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) + } rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) From b1f81f0072b06865ea1fdfe4c510474e64d39cfe Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 29 Jan 2020 18:42:18 -0600 Subject: [PATCH 1244/1387] [MINOR][SQL][DOCS] Fix typos in scaladoc strings of higher order functions ### What changes were proposed in this pull request? Fix following typos: - tranformation -> transformation - the boolean -> the Boolean - signle -> single ### Why are the changes needed? ### Does this PR introduce any user-facing change? No ### How was this patch tested? Scala linter. Closes #27382 from zero323/functions-typos. Authored-by: zero323 Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/sql/functions.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index e986a564ae0fc..da26c5a2f4625 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3408,7 +3408,7 @@ object functions { } /** - * Returns an array of elements after applying a tranformation to each element + * Returns an array of elements after applying a transformation to each element * in the input array. * * @group collection_funcs @@ -3419,7 +3419,7 @@ object functions { } /** - * Returns an array of elements after applying a tranformation to each element + * Returns an array of elements after applying a transformation to each element * in the input array. * * @group collection_funcs @@ -3457,7 +3457,7 @@ object functions { * }}} * * @param column: the input array column - * @param f: col => predicate, the boolean predicate to filter the input column + * @param f: col => predicate, the Boolean predicate to filter the input column * * @group collection_funcs * @since 3.0.0 @@ -3474,7 +3474,7 @@ object functions { * }}} * * @param column: the input array column - * @param f: (col, index) => predicate, the boolean predicate to filter the input column + * @param f: (col, index) => predicate, the Boolean predicate to filter the input column * given the index. Indices start at 0. * * @group collection_funcs @@ -3516,7 +3516,7 @@ object functions { aggregate(expr, zero, merge, c => c) /** - * Merge two given arrays, element-wise, into a signle array using a function. + * Merge two given arrays, element-wise, into a single array using a function. * If one array is shorter, nulls are appended at the end to match the length of the longer * array, before applying the function. * From c228810edcc2bc631029aa12a435aea701d061c2 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 30 Jan 2020 13:04:53 +0900 Subject: [PATCH 1245/1387] [SPARK-30672][BUILD] Add numpy to API docs readme ### What changes were proposed in this pull request? This PR adds `numpy` to the list of things that need to be installed in order to build the API docs. It doesn't add a new dependency; it just documents an existing dependency. ### Why are the changes needed? You cannot build the PySpark API docs without numpy installed. Otherwise you get this series of errors: ``` $ SKIP_SCALADOC=1 SKIP_RDOC=1 SKIP_SQLDOC=1 jekyll serve Configuration file: .../spark/docs/_config.yml Moving to python/docs directory and building sphinx. sphinx-build -b html -d _build/doctrees . _build/html Running Sphinx v2.3.1 loading pickled environment... done building [mo]: targets for 0 po files that are out of date building [html]: targets for 0 source files that are out of date updating environment: 0 added, 2 changed, 0 removed reading sources... [100%] pyspark.mllib WARNING: autodoc: failed to import module 'ml' from module 'pyspark'; the following exception was raised: No module named 'numpy' WARNING: autodoc: failed to import module 'ml.param' from module 'pyspark'; the following exception was raised: No module named 'numpy' ... ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually, by building the API docs with and without numpy. Closes #27390 from nchammas/SPARK-30672-numpy-pyspark-docs. Authored-by: Nicholas Chammas Signed-off-by: HyukjinKwon --- docs/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/README.md b/docs/README.md index 2001de6207d10..5197fc55f6b5a 100644 --- a/docs/README.md +++ b/docs/README.md @@ -38,7 +38,7 @@ installed. Also install the following libraries: ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs -$ sudo pip install sphinx pypandoc mkdocs +$ sudo pip install sphinx pypandoc mkdocs numpy $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "testthat", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' ``` From 246c398d591a7d6987d596a00db179202478f294 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 29 Jan 2020 20:55:29 -0800 Subject: [PATCH 1246/1387] [SPARK-30435][DOC] Update doc of Supported Hive Features ### What changes were proposed in this pull request? add supported hive features ### Why are the changes needed? update doc ### Does this PR introduce any user-facing change? Before change UI info: ![image](https://user-images.githubusercontent.com/46485123/72592726-29302c80-393e-11ea-8f4d-76432d4cb658.png) After this pr: ![image](https://user-images.githubusercontent.com/46485123/72593569-42d27380-3940-11ea-91c7-f2998d476364.png) ![image](https://user-images.githubusercontent.com/46485123/72962218-afd98380-3dee-11ea-82a1-0bf533ebfd9f.png) ### How was this patch tested? For PR about Spark Doc Web UI, we need to show UI format before and after pr. We can build our local web server about spark docs with reference `$SPARK_PROJECT/docs/README.md` You should install python and ruby in your env and also install plugin like below ```sh $ sudo gem install jekyll jekyll-redirect-from rouge # Following is needed only for generating API docs $ sudo pip install sphinx pypandoc mkdocs $ sudo Rscript -e 'install.packages(c("knitr", "devtools", "rmarkdown"), repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' $ sudo Rscript -e 'devtools::install_version("testthat", version = "1.0.2", repos="https://cloud.r-project.org/")' ``` Then we call `jekyll serve --watch` after build we see below message ``` ~/Documents/project/AngersZhu/spark/sql Moving back into docs dir. Making directory api/sql cp -r ../sql/site/. api/sql Source: /Users/angerszhu/Documents/project/AngersZhu/spark/docs Destination: /Users/angerszhu/Documents/project/AngersZhu/spark/docs/_site Incremental build: disabled. Enable with --incremental Generating... done in 24.717 seconds. Auto-regeneration: enabled for '/Users/angerszhu/Documents/project/AngersZhu/spark/docs' Server address: http://127.0.0.1:4000 Server running... press ctrl-c to stop. ``` Visit http://127.0.0.1:4000 to get your newest change in doc web. Closes #27106 from AngersZhuuuu/SPARK-30435. Authored-by: angerszhu Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 27 ++++++++++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 51e24157cea80..18e7df7aca5aa 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -919,6 +919,7 @@ Spark SQL supports the vast majority of Hive features, such as: * `SELECT` * `GROUP BY` * `ORDER BY` + * `DISTRIBUTE BY` * `CLUSTER BY` * `SORT BY` * All Hive operators, including: @@ -936,10 +937,34 @@ Spark SQL supports the vast majority of Hive features, such as: * `JOIN` * `{LEFT|RIGHT|FULL} OUTER JOIN` * `LEFT SEMI JOIN` + * `LEFT ANTI JOIN` * `CROSS JOIN` * Unions * Sub-queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` + * Sub-queries in the FROM Clause + + ```SELECT col FROM (SELECT a + b AS col FROM t1) t2``` + * Sub-queries in WHERE Clause + * Correlated or non-correlated IN and NOT IN statement in WHERE Clause + + ``` + SELECT col FROM t1 WHERE col IN (SELECT a FROM t2 WHERE t1.a = t2.a) + SELECT col FROM t1 WHERE col IN (SELECT a FROM t2) + ``` + * Correlated or non-correlated EXISTS and NOT EXISTS statement in WHERE Clause + + ``` + SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t1.a = t2.a AND t2.a > 10) + SELECT col FROM t1 WHERE EXISTS (SELECT t2.a FROM t2 WHERE t2.a > 10) + ``` + * Non-correlated IN and NOT IN statement in JOIN Condition + + ```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND t1.a IN (SELECT a FROM t3)``` + + * Non-correlated EXISTS and NOT EXISTS statement in JOIN Condition + + ```SELECT t1.col FROM t1 JOIN t2 ON t1.a = t2.a AND EXISTS (SELECT * FROM t3 WHERE t3.a > 10)``` + * Sampling * Explain * Partitioned tables including dynamic partition insertion From bda0669110fcfce5279221d2b9c2986a3cc7d15a Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 30 Jan 2020 16:40:38 +0900 Subject: [PATCH 1247/1387] [SPARK-30665][DOCS][BUILD][PYTHON] Eliminate pypandoc dependency ### What changes were proposed in this pull request? This PR removes any dependencies on pypandoc. It also makes related tweaks to the docs README to clarify the dependency on pandoc (not pypandoc). ### Why are the changes needed? We are using pypandoc to convert the Spark README from Markdown to ReST for PyPI. PyPI now natively supports Markdown, so we don't need pypandoc anymore. The dependency on pypandoc also sometimes causes issues when installing Python packages that depend on PySpark, as described in #18981. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually: ```sh python -m venv venv source venv/bin/activate pip install -U pip cd python/ python setup.py sdist pip install dist/pyspark-3.0.0.dev0.tar.gz pyspark --version ``` I also built the PySpark and R API docs with `jekyll` and reviewed them locally. It would be good if a maintainer could also test this by creating a PySpark distribution and uploading it to [Test PyPI](https://test.pypi.org) to confirm the README looks as it should. Closes #27376 from nchammas/SPARK-30665-pypandoc. Authored-by: Nicholas Chammas Signed-off-by: HyukjinKwon --- dev/create-release/spark-rm/Dockerfile | 4 ++-- dev/requirements.txt | 1 - dev/run-pip-tests | 2 +- docs/README.md | 22 ++++++++++++++++++---- python/setup.py | 13 +++---------- 5 files changed, 24 insertions(+), 18 deletions(-) diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 12195f5561076..3ba8e97929613 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -20,7 +20,7 @@ # Includes: # * Java 8 # * Ivy -# * Python/PyPandoc (2.7.15/3.6.7) +# * Python (2.7.15/3.6.7) # * R-base/R-base-dev (3.6.1) # * Ruby 2.3 build utilities @@ -34,7 +34,7 @@ ENV DEBCONF_NONINTERACTIVE_SEEN true ARG APT_INSTALL="apt-get install --no-install-recommends -y" ARG BASE_PIP_PKGS="setuptools wheel" -ARG PIP_PKGS="pyopenssl pypandoc numpy sphinx" +ARG PIP_PKGS="pyopenssl numpy sphinx" # Install extra needed repos and refresh. # - CRAN repo diff --git a/dev/requirements.txt b/dev/requirements.txt index 3fdd3425ffcc2..baea9213dbc97 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -2,5 +2,4 @@ flake8==3.5.0 jira==1.0.3 PyGithub==1.26.0 Unidecode==0.04.19 -pypandoc==1.3.3 sphinx diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 1294a9096fb95..470f21e69d46a 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -85,7 +85,7 @@ for python in "${PYTHON_EXECS[@]}"; do fi # Upgrade pip & friends if using virtual env if [ ! -n "$USE_CONDA" ]; then - pip install --upgrade pip pypandoc wheel numpy + pip install --upgrade pip wheel numpy fi echo "Creating pip installable source dist" diff --git a/docs/README.md b/docs/README.md index 5197fc55f6b5a..22039871cf63d 100644 --- a/docs/README.md +++ b/docs/README.md @@ -37,16 +37,30 @@ installed. Also install the following libraries: ```sh $ sudo gem install jekyll jekyll-redirect-from rouge -# Following is needed only for generating API docs -$ sudo pip install sphinx pypandoc mkdocs numpy -$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "testthat", "rmarkdown"), repos="https://cloud.r-project.org/")' -$ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' ``` Note: If you are on a system with both Ruby 1.9 and Ruby 2.0 you may need to replace gem with gem2.0. +### R Documentation + +If you'd like to generate R documentation, you'll need to [install Pandoc](https://pandoc.org/installing.html) +and install these libraries: + +```sh +$ sudo Rscript -e 'install.packages(c("knitr", "devtools", "testthat", "rmarkdown"), repos="https://cloud.r-project.org/")' +$ sudo Rscript -e 'devtools::install_version("roxygen2", version = "5.0.1", repos="https://cloud.r-project.org/")' +``` + Note: Other versions of roxygen2 might work in SparkR documentation generation but `RoxygenNote` field in `$SPARK_HOME/R/pkg/DESCRIPTION` is 5.0.1, which is updated if the version is mismatched. +### API Documentation + +To generate API docs for any language, you'll need to install these libraries: + +```sh +$ sudo pip install sphinx mkdocs numpy +``` + ## Generating the Documentation HTML We include the Spark documentation as part of the source (as opposed to using a hosted wiki, such as diff --git a/python/setup.py b/python/setup.py index 965927a5694ba..40b49aaeeb27c 100755 --- a/python/setup.py +++ b/python/setup.py @@ -153,21 +153,15 @@ def _supports_symlinks(): # will search for SPARK_HOME with Python. scripts.append("pyspark/find_spark_home.py") - # Parse the README markdown file into rst for PyPI - long_description = "!!!!! missing pandoc do not upload to PyPI !!!!" - try: - import pypandoc - long_description = pypandoc.convert('README.md', 'rst') - except ImportError: - print("Could not import pypandoc - required to package PySpark", file=sys.stderr) - except OSError: - print("Could not convert - pandoc is not installed", file=sys.stderr) + with open('README.md') as f: + long_description = f.read() setup( name='pyspark', version=VERSION, description='Apache Spark Python API', long_description=long_description, + long_description_content_type="text/markdown", author='Spark Developers', author_email='dev@spark.apache.org', url='https://github.com/apache/spark/tree/master/python', @@ -213,7 +207,6 @@ def _supports_symlinks(): scripts=scripts, license='http://www.apache.org/licenses/LICENSE-2.0', install_requires=['py4j==0.10.8.1'], - setup_requires=['pypandoc'], extras_require={ 'ml': ['numpy>=1.7'], 'mllib': ['numpy>=1.7'], From 561e9b968821ca3e501aa5cba7ba5ceaa45796ea Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 30 Jan 2020 03:17:29 -0800 Subject: [PATCH 1248/1387] [SPARK-30674][INFRA] Use python3 in dev/lint-python ### What changes were proposed in this pull request? This PR aims to use `python3` instead of `python` in `dev/lint-python`. ### Why are the changes needed? Currently, `dev/lint-python` fails at Python 2. And, Python 2 is EOL since January 1st 2020. ``` $ python -V Python 2.7.17 $ dev/lint-python starting python compilation test... Python compilation failed with the following errors: Compiling ./python/setup.py ... File "./python/setup.py", line 27 file=sys.stderr) ^ SyntaxError: invalid syntax ``` ### Does this PR introduce any user-facing change? No. This is a dev environment. ### How was this patch tested? Jenkins is running this with Python 3 already. The following is a manual test. ``` $ python -V Python 3.8.0 $ dev/lint-python starting python compilation test... python compilation succeeded. ``` Closes #27394 from dongjoon-hyun/SPARK-30674. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/lint-python | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dev/lint-python b/dev/lint-python index 47f4b4cc12d6c..e9ed83dec5fce 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -36,9 +36,9 @@ function compile_python_test { exit 1; fi - # compileall: https://docs.python.org/2/library/compileall.html + # compileall: https://docs.python.org/3/library/compileall.html echo "starting python compilation test..." - COMPILE_REPORT=$( (python -B -mcompileall -q -l -x "[/\\\\][.]git" $1) 2>&1) + COMPILE_REPORT=$( (python3 -B -mcompileall -q -l -x "[/\\\\][.]git" $1) 2>&1) COMPILE_STATUS=$? if [ $COMPILE_STATUS -ne 0 ]; then @@ -70,7 +70,7 @@ function pycodestyle_test { RUN_LOCAL_PYCODESTYLE="False" if hash "$PYCODESTYLE_BUILD" 2> /dev/null; then VERSION=$( $PYCODESTYLE_BUILD --version 2> /dev/null) - EXPECTED_PYCODESTYLE=$( (python -c 'from distutils.version import LooseVersion; + EXPECTED_PYCODESTYLE=$( (python3 -c 'from distutils.version import LooseVersion; print(LooseVersion("""'${VERSION[0]}'""") >= LooseVersion("""'$MINIMUM_PYCODESTYLE'"""))')\ 2> /dev/null) @@ -96,7 +96,7 @@ function pycodestyle_test { fi echo "starting pycodestyle test..." - PYCODESTYLE_REPORT=$( (python "$PYCODESTYLE_SCRIPT_PATH" --config=dev/tox.ini $1) 2>&1) + PYCODESTYLE_REPORT=$( (python3 "$PYCODESTYLE_SCRIPT_PATH" --config=dev/tox.ini $1) 2>&1) PYCODESTYLE_STATUS=$? else # we have the right version installed, so run locally @@ -130,7 +130,7 @@ function flake8_test { FLAKE8_VERSION="$($FLAKE8_BUILD --version 2> /dev/null)" VERSION=($FLAKE8_VERSION) - EXPECTED_FLAKE8=$( (python -c 'from distutils.version import LooseVersion; + EXPECTED_FLAKE8=$( (python3 -c 'from distutils.version import LooseVersion; print(LooseVersion("""'${VERSION[0]}'""") >= LooseVersion("""'$MINIMUM_FLAKE8'"""))') \ 2> /dev/null) @@ -175,7 +175,7 @@ function pydocstyle_test { fi PYDOCSTYLE_VERSION="$($PYDOCSTYLEBUILD --version 2> /dev/null)" - EXPECTED_PYDOCSTYLE=$(python -c 'from distutils.version import LooseVersion; \ + EXPECTED_PYDOCSTYLE=$(python3 -c 'from distutils.version import LooseVersion; \ print(LooseVersion("""'$PYDOCSTYLE_VERSION'""") >= LooseVersion("""'$MINIMUM_PYDOCSTYLE'"""))' \ 2> /dev/null) From 073ce125436a8ad55470057718a14c92b6b5b939 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Thu, 30 Jan 2020 10:52:07 -0600 Subject: [PATCH 1249/1387] [SPARK-30659][ML][PYSPARK] LogisticRegression blockify input vectors ### What changes were proposed in this pull request? 1, use blocks instead of vectors 2, use Level-2 BLAS for binary, use Level-3 BLAS for multinomial ### Why are the changes needed? 1, less RAM to persist training data; (save ~40%) 2, faster than existing impl; (40% ~ 92%) ### Does this PR introduce any user-facing change? add a new expert param `blockSize` ### How was this patch tested? updated testsuites Closes #27374 from zhengruifeng/blockify_lor. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../spark/ml/classification/LinearSVC.scala | 4 +- .../classification/LogisticRegression.scala | 58 ++-- .../ml/optim/aggregator/HingeAggregator.scala | 31 +-- .../optim/aggregator/LogisticAggregator.scala | 252 ++++++++++++++++-- .../ml/param/shared/SharedParamsCodeGen.scala | 4 +- .../spark/ml/param/shared/sharedParams.scala | 12 +- .../classification/LogisticRegression.scala | 4 +- .../LogisticRegressionSuite.scala | 4 +- .../aggregator/LogisticAggregatorSuite.scala | 56 +++- python/pyspark/ml/classification.py | 34 ++- .../ml/param/_shared_params_code_gen.py | 2 +- python/pyspark/ml/param/shared.py | 2 +- 12 files changed, 350 insertions(+), 113 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 6b1cdd8ad3963..f16648d2abee6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -157,7 +157,7 @@ class LinearSVC @Since("2.2.0") ( /** * Set block size for stacking input data in matrices. - * Default is 4096. + * Default is 1024. * * @group expertSetParam */ @@ -240,7 +240,7 @@ class LinearSVC @Since("2.2.0") ( .persist(StorageLevel.MEMORY_AND_DISK) .setName(s"training dataset (blockSize=${$(blockSize)})") - val getAggregatorFunc = new HingeAggregator(numFeatures, $(fitIntercept), $(blockSize))(_) + val getAggregatorFunc = new HingeAggregator(numFeatures, $(fitIntercept))(_) val costFun = new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 50c14d086957f..9b5b36257a584 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg._ import org.apache.spark.ml.optim.aggregator.LogisticAggregator import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} @@ -50,7 +50,8 @@ import org.apache.spark.util.VersionUtils */ private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasFitIntercept with HasTol - with HasStandardization with HasWeightCol with HasThreshold with HasAggregationDepth { + with HasStandardization with HasWeightCol with HasThreshold with HasAggregationDepth + with HasBlockSize { import org.apache.spark.ml.classification.LogisticRegression.supportedFamilyNames @@ -430,6 +431,15 @@ class LogisticRegression @Since("1.2.0") ( @Since("2.2.0") def setUpperBoundsOnIntercepts(value: Vector): this.type = set(upperBoundsOnIntercepts, value) + /** + * Set block size for stacking input data in matrices. + * Default is 1024. + * + * @group expertSetParam + */ + @Since("3.0.0") + def setBlockSize(value: Int): this.type = set(blockSize, value) + private def assertBoundConstrainedOptimizationParamsValid( numCoefficientSets: Int, numFeatures: Int): Unit = { @@ -482,24 +492,17 @@ class LogisticRegression @Since("1.2.0") ( this } - override protected[spark] def train(dataset: Dataset[_]): LogisticRegressionModel = { - val handlePersistence = dataset.storageLevel == StorageLevel.NONE - train(dataset, handlePersistence) - } - - protected[spark] def train( - dataset: Dataset[_], - handlePersistence: Boolean): LogisticRegressionModel = instrumented { instr => - val instances = extractInstances(dataset) - - if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) - + override protected[spark] def train( + dataset: Dataset[_]): LogisticRegressionModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, probabilityCol, regParam, elasticNetParam, standardization, threshold, maxIter, tol, fitIntercept) + val sc = dataset.sparkSession.sparkContext + val instances = extractInstances(dataset) + val (summarizer, labelSummarizer) = instances.treeAggregate( (Summarizer.createSummarizerBuffer("mean", "std", "count"), new MultiClassSummarizer))( seqOp = (c: (SummarizerBuffer, MultiClassSummarizer), instance: Instance) => @@ -582,8 +585,9 @@ class LogisticRegression @Since("1.2.0") ( s"dangerous ground, so the algorithm may not converge.") } - val featuresMean = summarizer.mean.toArray - val featuresStd = summarizer.std.toArray + val featuresMean = summarizer.mean.compressed + val featuresStd = summarizer.std.compressed + val bcFeaturesStd = sc.broadcast(featuresStd) if (!$(fitIntercept) && (0 until numFeatures).exists { i => featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { @@ -595,8 +599,7 @@ class LogisticRegression @Since("1.2.0") ( val regParamL1 = $(elasticNetParam) * $(regParam) val regParamL2 = (1.0 - $(elasticNetParam)) * $(regParam) - val bcFeaturesStd = instances.context.broadcast(featuresStd) - val getAggregatorFunc = new LogisticAggregator(bcFeaturesStd, numClasses, $(fitIntercept), + val getAggregatorFunc = new LogisticAggregator(numFeatures, numClasses, $(fitIntercept), multinomial = isMultinomial)(_) val getFeaturesStd = (j: Int) => if (j >= 0 && j < numCoefficientSets * numFeatures) { featuresStd(j / numCoefficientSets) @@ -612,7 +615,21 @@ class LogisticRegression @Since("1.2.0") ( None } - val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization, + val standardized = instances.map { + case Instance(label, weight, features) => + val featuresStd = bcFeaturesStd.value + val array = Array.ofDim[Double](numFeatures) + features.foreachNonZero { (i, v) => + val std = featuresStd(i) + if (std != 0) array(i) = v / std + } + Instance(label, weight, Vectors.dense(array)) + } + val blocks = InstanceBlock.blokify(standardized, $(blockSize)) + .persist(StorageLevel.MEMORY_AND_DISK) + .setName(s"training dataset (blockSize=${$(blockSize)})") + + val costFun = new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) val numCoeffsPlusIntercepts = numFeaturesPlusIntercept * numCoefficientSets @@ -806,6 +823,7 @@ class LogisticRegression @Since("1.2.0") ( state = states.next() arrayBuilder += state.adjustedValue } + blocks.unpersist() bcFeaturesStd.destroy() if (state == null) { @@ -875,8 +893,6 @@ class LogisticRegression @Since("1.2.0") ( } } - if (handlePersistence) instances.unpersist() - val model = copyValues(new LogisticRegressionModel(uid, coefficientMatrix, interceptVector, numClasses, isMultinomial)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala index 25f7c9ddab42d..292187b3e146e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala @@ -35,8 +35,7 @@ import org.apache.spark.ml.linalg._ */ private[ml] class HingeAggregator( numFeatures: Int, - fitIntercept: Boolean, - blockSize: Int = 4096)(bcCoefficients: Broadcast[Vector]) + fitIntercept: Boolean)(bcCoefficients: Broadcast[Vector]) extends DifferentiableLossAggregator[InstanceBlock, HingeAggregator] { private val numFeaturesPlusIntercept: Int = if (fitIntercept) numFeatures + 1 else numFeatures @@ -55,20 +54,6 @@ private[ml] class HingeAggregator( } } - @transient private lazy val intercept = - if (fitIntercept) coefficientsArray(numFeatures) else 0.0 - - @transient private lazy val linearGradSumVec = { - if (fitIntercept) { - new DenseVector(Array.ofDim[Double](numFeatures)) - } else { - null - } - } - - @transient private lazy val auxiliaryVec = - new DenseVector(Array.ofDim[Double](blockSize)) - /** * Add a new training instance to this HingeAggregator, and update the loss and gradient @@ -138,19 +123,14 @@ private[ml] class HingeAggregator( val localGradientSumArray = gradientSumArray // vec here represents dotProducts - val vec = if (size == blockSize) { - auxiliaryVec + val vec = if (fitIntercept && coefficientsArray.last != 0) { + val intercept = coefficientsArray.last + new DenseVector(Array.fill(size)(intercept)) } else { - // the last block within one partition may be of size less than blockSize new DenseVector(Array.ofDim[Double](size)) } if (fitIntercept) { - var i = 0 - while (i < size) { - vec.values(i) = intercept - i += 1 - } BLAS.gemv(1.0, block.matrix, linear, 1.0, vec) } else { BLAS.gemv(1.0, block.matrix, linear, 0.0, vec) @@ -185,6 +165,9 @@ private[ml] class HingeAggregator( if (vec.values.forall(_ == 0)) return this if (fitIntercept) { + // localGradientSumArray is of size numFeatures+1, so can not + // be directly used as the output of BLAS.gemv + val linearGradSumVec = new DenseVector(Array.ofDim[Double](numFeatures)) BLAS.gemv(1.0, block.matrix.transpose, vec, 0.0, linearGradSumVec) linearGradSumVec.foreachNonZero { (i, v) => localGradientSumArray(i) += v } localGradientSumArray(numFeatures) += vec.values.sum diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala index f2b3566f8f09e..76d21995a2c50 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.linalg.{DenseVector, Vector} +import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.linalg._ import org.apache.spark.mllib.util.MLUtils /** @@ -171,7 +171,6 @@ import org.apache.spark.mllib.util.MLUtils * * * @param bcCoefficients The broadcast coefficients corresponding to the features. - * @param bcFeaturesStd The broadcast standard deviation values of the features. * @param numClasses the number of possible outcomes for k classes classification problem in * Multinomial Logistic Regression. * @param fitIntercept Whether to fit an intercept term. @@ -183,13 +182,12 @@ import org.apache.spark.mllib.util.MLUtils * since this form is optimal for the matrix operations used for prediction. */ private[ml] class LogisticAggregator( - bcFeaturesStd: Broadcast[Array[Double]], + numFeatures: Int, numClasses: Int, fitIntercept: Boolean, multinomial: Boolean)(bcCoefficients: Broadcast[Vector]) - extends DifferentiableLossAggregator[Instance, LogisticAggregator] with Logging { + extends DifferentiableLossAggregator[InstanceBlock, LogisticAggregator] with Logging { - private val numFeatures = bcFeaturesStd.value.length private val numFeaturesPlusIntercept = if (fitIntercept) numFeatures + 1 else numFeatures private val coefficientSize = bcCoefficients.value.size protected override val dim: Int = coefficientSize @@ -209,6 +207,31 @@ private[ml] class LogisticAggregator( s"got type ${bcCoefficients.value.getClass}.)") } + @transient private lazy val binaryLinear = { + if (!multinomial) { + if (fitIntercept) { + new DenseVector(coefficientsArray.take(numFeatures)) + } else { + new DenseVector(coefficientsArray) + } + } else { + null + } + } + + @transient private lazy val multinomialLinear = { + if (multinomial) { + if (fitIntercept) { + new DenseMatrix(numClasses, numFeatures, coefficientsArray.take(numClasses * numFeatures)) + } else { + new DenseMatrix(numClasses, numFeatures, coefficientsArray) + } + } else { + null + } + } + + if (multinomial && numClasses <= 2) { logInfo(s"Multinomial logistic regression for binary classification yields separate " + s"coefficients for positive and negative classes. When no regularization is applied, the" + @@ -219,15 +242,12 @@ private[ml] class LogisticAggregator( /** Update gradient and loss using binary loss function. */ private def binaryUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = { - val localFeaturesStd = bcFeaturesStd.value val localCoefficients = coefficientsArray val localGradientArray = gradientSumArray val margin = - { var sum = 0.0 features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - sum += localCoefficients(index) * value / localFeaturesStd(index) - } + sum += localCoefficients(index) * value } if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1) sum @@ -236,9 +256,7 @@ private[ml] class LogisticAggregator( val multiplier = weight * (1.0 / (1.0 + math.exp(margin)) - label) features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - localGradientArray(index) += multiplier * value / localFeaturesStd(index) - } + localGradientArray(index) += multiplier * value } if (fitIntercept) { @@ -253,6 +271,61 @@ private[ml] class LogisticAggregator( } } + /** Update gradient and loss using binary loss function. */ + private def binaryUpdateInPlace(block: InstanceBlock): Unit = { + val size = block.size + val localGradientSumArray = gradientSumArray + + // vec here represents margins or negative dotProducts + val vec = if (fitIntercept && coefficientsArray.last != 0) { + val intercept = coefficientsArray.last + new DenseVector(Array.fill(size)(intercept)) + } else { + new DenseVector(Array.ofDim[Double](size)) + } + + if (fitIntercept) { + BLAS.gemv(-1.0, block.matrix, binaryLinear, -1.0, vec) + } else { + BLAS.gemv(-1.0, block.matrix, binaryLinear, 0.0, vec) + } + + // in-place convert margins to multiplier + // then, vec represents multiplier + var i = 0 + while (i < size) { + val weight = block.getWeight(i) + if (weight > 0) { + weightSum += weight + val label = block.getLabel(i) + val margin = vec(i) + if (label > 0) { + // The following is equivalent to log(1 + exp(margin)) but more numerically stable. + lossSum += weight * MLUtils.log1pExp(margin) + } else { + lossSum += weight * (MLUtils.log1pExp(margin) - margin) + } + val multiplier = weight * (1.0 / (1.0 + math.exp(margin)) - label) + vec.values(i) = multiplier + } else { + vec.values(i) = 0.0 + } + i += 1 + } + + if (fitIntercept) { + // localGradientSumArray is of size numFeatures+1, so can not + // be directly used as the output of BLAS.gemv + val linearGradSumVec = new DenseVector(Array.ofDim[Double](numFeatures)) + BLAS.gemv(1.0, block.matrix.transpose, vec, 0.0, linearGradSumVec) + linearGradSumVec.foreachNonZero { (i, v) => localGradientSumArray(i) += v } + localGradientSumArray(numFeatures) += vec.values.sum + } else { + val gradSumVec = new DenseVector(localGradientSumArray) + BLAS.gemv(1.0, block.matrix.transpose, vec, 1.0, gradSumVec) + } + } + /** Update gradient and loss using multinomial (softmax) loss function. */ private def multinomialUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = { // TODO: use level 2 BLAS operations @@ -260,7 +333,6 @@ private[ml] class LogisticAggregator( Note: this can still be used when numClasses = 2 for binary logistic regression without pivoting. */ - val localFeaturesStd = bcFeaturesStd.value val localCoefficients = coefficientsArray val localGradientArray = gradientSumArray @@ -270,13 +342,10 @@ private[ml] class LogisticAggregator( val margins = new Array[Double](numClasses) features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - val stdValue = value / localFeaturesStd(index) - var j = 0 - while (j < numClasses) { - margins(j) += localCoefficients(index * numClasses + j) * stdValue - j += 1 - } + var j = 0 + while (j < numClasses) { + margins(j) += localCoefficients(index * numClasses + j) * value + j += 1 } } var i = 0 @@ -314,13 +383,10 @@ private[ml] class LogisticAggregator( multipliers(i) = multipliers(i) / sum - (if (label == i) 1.0 else 0.0) } features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - val stdValue = value / localFeaturesStd(index) - var j = 0 - while (j < numClasses) { - localGradientArray(index * numClasses + j) += weight * multipliers(j) * stdValue - j += 1 - } + var j = 0 + while (j < numClasses) { + localGradientArray(index * numClasses + j) += weight * multipliers(j) * value + j += 1 } } if (fitIntercept) { @@ -339,6 +405,112 @@ private[ml] class LogisticAggregator( lossSum += weight * loss } + /** Update gradient and loss using multinomial (softmax) loss function. */ + private def multinomialUpdateInPlace(block: InstanceBlock): Unit = { + val size = block.size + val localGradientSumArray = gradientSumArray + + // mat here represents margins, shape: S X C + val mat = new DenseMatrix(size, numClasses, Array.ofDim[Double](size * numClasses)) + + if (fitIntercept) { + val intercept = coefficientsArray.takeRight(numClasses) + var i = 0 + while (i < size) { + var j = 0 + while (j < numClasses) { + mat.update(i, j, intercept(j)) + j += 1 + } + i += 1 + } + BLAS.gemm(1.0, block.matrix, multinomialLinear.transpose, 1.0, mat) + } else { + BLAS.gemm(1.0, block.matrix, multinomialLinear.transpose, 0.0, mat) + } + + // in-place convert margins to multipliers + // then, mat represents multipliers + var i = 0 + val tmp = Array.ofDim[Double](numClasses) + while (i < size) { + val weight = block.getWeight(i) + if (weight > 0) { + weightSum += weight + val label = block.getLabel(i) + + var maxMargin = Double.NegativeInfinity + var j = 0 + while (j < numClasses) { + tmp(j) = mat(i, j) + maxMargin = math.max(maxMargin, tmp(j)) + j += 1 + } + + // marginOfLabel is margins(label) in the formula + val marginOfLabel = tmp(label.toInt) + + var sum = 0.0 + j = 0 + while (j < numClasses) { + if (maxMargin > 0) tmp(j) -= maxMargin + val exp = math.exp(tmp(j)) + sum += exp + tmp(j) = exp + j += 1 + } + + j = 0 + while (j < numClasses) { + val multiplier = weight * (tmp(j) / sum - (if (label == j) 1.0 else 0.0)) + mat.update(i, j, multiplier) + j += 1 + } + + if (maxMargin > 0) { + lossSum += weight * (math.log(sum) - marginOfLabel + maxMargin) + } else { + lossSum += weight * (math.log(sum) - marginOfLabel) + } + } else { + var j = 0 + while (j < numClasses) { + mat.update(i, j, 0.0) + j += 1 + } + } + i += 1 + } + + // block.matrix: S X F, unknown type + // mat (multipliers): S X C, dense + // gradSumMat(gradientSumArray): C X FPI (numFeaturesPlusIntercept), dense + block.matrix match { + case dm: DenseMatrix if !fitIntercept => + // If fitIntercept==false, gradientSumArray += mat.T X matrix + // GEMM requires block.matrix is dense + val gradSumMat = new DenseMatrix(numClasses, numFeatures, localGradientSumArray) + BLAS.gemm(1.0, mat.transpose, dm, 1.0, gradSumMat) + + case _ => + // Otherwise, use linearGradSumMat (F X C) as a temp matrix: + // linearGradSumMat = matrix.T X mat + val linearGradSumMat = new DenseMatrix(numFeatures, numClasses, + Array.ofDim[Double](numFeatures * numClasses)) + BLAS.gemm(1.0, block.matrix.transpose, mat, 0.0, linearGradSumMat) + linearGradSumMat.foreachActive { (i, j, v) => + if (v != 0) localGradientSumArray(i * numClasses + j) += v + } + + if (fitIntercept) { + val start = numClasses * numFeatures + mat.foreachActive { (i, j, v) => + if (v != 0) localGradientSumArray(start + j) += v + } + } + } + } + /** * Add a new training instance to this LogisticAggregator, and update the loss and gradient * of the objective function. @@ -363,4 +535,28 @@ private[ml] class LogisticAggregator( this } } + + /** + * Add a new training instance block to this LogisticAggregator, and update the loss and gradient + * of the objective function. + * + * @param block The instance block of data point to be added. + * @return This LogisticAggregator object. + */ + def add(block: InstanceBlock): this.type = { + require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + + s"instance. Expecting $numFeatures but got ${block.numFeatures}.") + require(block.weightIter.forall(_ >= 0), + s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") + + if (block.weightIter.forall(_ == 0)) return this + + if (multinomial) { + multinomialUpdateInPlace(block) + } else { + binaryUpdateInPlace(block) + } + + this + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index eee75e7f5b722..3d1fab8692af7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -104,10 +104,10 @@ private[shared] object SharedParamsCodeGen { isValid = "ParamValidators.inArray(Array(\"euclidean\", \"cosine\"))"), ParamDesc[String]("validationIndicatorCol", "name of the column that indicates whether " + "each row is for training or for validation. False indicates training; true indicates " + - "validation."), + "validation"), ParamDesc[Int]("blockSize", "block size for stacking input data in matrices. Data is " + "stacked within partitions. If block size is more than remaining data in a partition " + - "then it is adjusted to the size of this data.", Some("4096"), + "then it is adjusted to the size of this data", Some("1024"), isValid = "ParamValidators.gt(0)", isExpertParam = true) ) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 3d1c55a5eb429..7fe8ccd973a72 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -570,29 +570,29 @@ trait HasDistanceMeasure extends Params { trait HasValidationIndicatorCol extends Params { /** - * Param for name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation.. + * Param for name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation. * @group param */ - final val validationIndicatorCol: Param[String] = new Param[String](this, "validationIndicatorCol", "name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation.") + final val validationIndicatorCol: Param[String] = new Param[String](this, "validationIndicatorCol", "name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation") /** @group getParam */ final def getValidationIndicatorCol: String = $(validationIndicatorCol) } /** - * Trait for shared param blockSize (default: 4096). This trait may be changed or + * Trait for shared param blockSize (default: 1024). This trait may be changed or * removed between minor versions. */ @DeveloperApi trait HasBlockSize extends Params { /** - * Param for block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.. + * Param for block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data. * @group expertParam */ - final val blockSize: IntParam = new IntParam(this, "blockSize", "block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.", ParamValidators.gt(0)) + final val blockSize: IntParam = new IntParam(this, "blockSize", "block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data", ParamValidators.gt(0)) - setDefault(blockSize, 4096) + setDefault(blockSize, 1024) /** @group expertGetParam */ final def getBlockSize: Int = $(blockSize) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 21eb17dfaacb3..f88f3fce61b33 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -339,10 +339,8 @@ class LogisticRegressionWithLBFGS // Convert our input into a DataFrame val spark = SparkSession.builder().sparkContext(input.context).getOrCreate() val df = spark.createDataFrame(input.map(_.asML)) - // Determine if we should cache the DF - val handlePersistence = input.getStorageLevel == StorageLevel.NONE // Train our model - val mlLogisticRegressionModel = lr.train(df, handlePersistence) + val mlLogisticRegressionModel = lr.train(df) // convert the model val weights = Vectors.dense(mlLogisticRegressionModel.coefficients.toArray) createModel(weights, mlLogisticRegressionModel.intercept) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 6d31e6efc7e1c..9e359ba098bfb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -542,7 +542,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { test("sparse coefficients in LogisticAggregator") { val bcCoefficientsBinary = spark.sparkContext.broadcast(Vectors.sparse(2, Array(0), Array(1.0))) val bcFeaturesStd = spark.sparkContext.broadcast(Array(1.0)) - val binaryAgg = new LogisticAggregator(bcFeaturesStd, 2, + val binaryAgg = new LogisticAggregator(1, 2, fitIntercept = true, multinomial = false)(bcCoefficientsBinary) val thrownBinary = withClue("binary logistic aggregator cannot handle sparse coefficients") { intercept[IllegalArgumentException] { @@ -552,7 +552,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(thrownBinary.getMessage.contains("coefficients only supports dense")) val bcCoefficientsMulti = spark.sparkContext.broadcast(Vectors.sparse(6, Array(0), Array(1.0))) - val multinomialAgg = new LogisticAggregator(bcFeaturesStd, 3, + val multinomialAgg = new LogisticAggregator(1, 3, fitIntercept = true, multinomial = true)(bcCoefficientsMulti) val thrown = withClue("multinomial logistic aggregator cannot handle sparse coefficients") { intercept[IllegalArgumentException] { diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala index e699adcc14c03..83718076dde7b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg.{BLAS, Matrices, Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,21 +32,21 @@ class LogisticAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { override def beforeAll(): Unit = { super.beforeAll() - instances = Array( + instances = standardize(Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) - ) - instancesConstantFeature = Array( + )) + instancesConstantFeature = standardize(Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) - ) - instancesConstantFeatureFiltered = Array( + )) + instancesConstantFeatureFiltered = standardize(Array( Instance(0.0, 0.1, Vectors.dense(2.0)), Instance(1.0, 0.5, Vectors.dense(1.0)), Instance(2.0, 0.3, Vectors.dense(0.5)) - ) + )) } /** Get summary statistics for some data and create a new LogisticAggregator. */ @@ -55,13 +55,27 @@ class LogisticAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { coefficients: Vector, fitIntercept: Boolean, isMultinomial: Boolean): LogisticAggregator = { - val (featuresSummarizer, ySummarizer) = + val (_, ySummarizer) = DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) val numClasses = ySummarizer.histogram.length - val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) - val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) + val numFeatures = instances.head.features.size val bcCoefficients = spark.sparkContext.broadcast(coefficients) - new LogisticAggregator(bcFeaturesStd, numClasses, fitIntercept, isMultinomial)(bcCoefficients) + new LogisticAggregator(numFeatures, numClasses, fitIntercept, isMultinomial)(bcCoefficients) + } + + private def standardize(instances: Array[Instance]): Array[Instance] = { + val (featuresSummarizer, _) = + DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) + val stdArray = featuresSummarizer.variance.toArray.map(math.sqrt) + val numFeatures = stdArray.length + instances.map { case Instance(label, weight, features) => + val standardized = Array.ofDim[Double](numFeatures) + features.foreachNonZero { (i, v) => + val std = stdArray(i) + if (std != 0) standardized(i) = v / std + } + Instance(label, weight, Vectors.dense(standardized).compressed) + } } test("aggregator add method input size") { @@ -277,4 +291,24 @@ class LogisticAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { validateGradient(aggConstantFeatureBinary.gradient, aggConstantFeatureBinaryFiltered.gradient, 1) } + + test("add instance block") { + val binaryInstances = instances.map { instance => + if (instance.label <= 1.0) instance else Instance(0.0, instance.weight, instance.features) + } + val coefArray = Array(1.0, 2.0) + val intercept = 1.0 + + val agg = getNewAggregator(binaryInstances, Vectors.dense(coefArray ++ Array(intercept)), + fitIntercept = true, isMultinomial = false) + binaryInstances.foreach(agg.add) + + val agg2 = getNewAggregator(binaryInstances, Vectors.dense(coefArray ++ Array(intercept)), + fitIntercept = true, isMultinomial = false) + val block = InstanceBlock.fromInstances(binaryInstances) + agg2.add(block) + + assert(agg.loss ~== agg2.loss relTol 1e-8) + assert(agg.gradient ~== agg2.gradient relTol 1e-8) + } } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 89d27fbfa316e..bb9cd034808fc 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -216,7 +216,7 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable >>> model.getThreshold() 0.5 >>> model.getBlockSize() - 4096 + 1024 >>> model.coefficients DenseVector([0.0, -0.2792, -0.1833]) >>> model.intercept @@ -255,19 +255,19 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, - aggregationDepth=2, blockSize=4096): + aggregationDepth=2, blockSize=1024): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", \ fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, \ - aggregationDepth=2, blockSize=4096): + aggregationDepth=2, blockSize=1024): """ super(LinearSVC, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LinearSVC", self.uid) self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, standardization=True, threshold=0.0, aggregationDepth=2, - blockSize=4096) + blockSize=1024) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -276,12 +276,12 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, - aggregationDepth=2, blockSize=4096): + aggregationDepth=2, blockSize=1024): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", \ fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, \ - aggregationDepth=2, blockSize=4096): + aggregationDepth=2, blockSize=1024): Sets params for Linear SVM Classifier. """ kwargs = self._input_kwargs @@ -388,7 +388,7 @@ def intercept(self): class _LogisticRegressionParams(_JavaProbabilisticClassifierParams, HasRegParam, HasElasticNetParam, HasMaxIter, HasFitIntercept, HasTol, HasStandardization, HasWeightCol, HasAggregationDepth, - HasThreshold): + HasThreshold, HasBlockSize): """ Params for :py:class:`LogisticRegression` and :py:class:`LogisticRegressionModel`. @@ -570,6 +570,8 @@ class LogisticRegression(JavaProbabilisticClassifier, _LogisticRegressionParams, 10 >>> blor.clear(blor.maxIter) >>> blorModel = blor.fit(bdf) + >>> blorModel.getBlockSize() + 1024 >>> blorModel.setFeaturesCol("features") LogisticRegressionModel... >>> blorModel.setProbabilityCol("newProbability") @@ -638,7 +640,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred rawPredictionCol="rawPrediction", standardization=True, weightCol=None, aggregationDepth=2, family="auto", lowerBoundsOnCoefficients=None, upperBoundsOnCoefficients=None, - lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None): + lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None, blockSize=1024): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ @@ -647,13 +649,14 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred rawPredictionCol="rawPrediction", standardization=True, weightCol=None, \ aggregationDepth=2, family="auto", \ lowerBoundsOnCoefficients=None, upperBoundsOnCoefficients=None, \ - lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None): + lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None, blockSize=1024): If the threshold and thresholds Params are both set, they must be equivalent. """ super(LogisticRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LogisticRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto") + self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto", + blockSize=1024) kwargs = self._input_kwargs self.setParams(**kwargs) self._checkThresholdConsistency() @@ -666,7 +669,7 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre rawPredictionCol="rawPrediction", standardization=True, weightCol=None, aggregationDepth=2, family="auto", lowerBoundsOnCoefficients=None, upperBoundsOnCoefficients=None, - lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None): + lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None, blockSize=1024): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, \ @@ -674,7 +677,7 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre rawPredictionCol="rawPrediction", standardization=True, weightCol=None, \ aggregationDepth=2, family="auto", \ lowerBoundsOnCoefficients=None, upperBoundsOnCoefficients=None, \ - lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None): + lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None, blockSize=1024): Sets params for logistic regression. If the threshold and thresholds Params are both set, they must be equivalent. """ @@ -769,6 +772,13 @@ def setAggregationDepth(self, value): """ return self._set(aggregationDepth=value) + @since("3.0.0") + def setBlockSize(self, value): + """ + Sets the value of :py:attr:`blockSize`. + """ + return self._set(blockSize=value) + class LogisticRegressionModel(JavaProbabilisticClassificationModel, _LogisticRegressionParams, JavaMLWritable, JavaMLReadable, HasTrainingSummary): diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 3994625c05f1b..fb4d55d57a2df 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -167,7 +167,7 @@ def get$Name(self): None, "TypeConverters.toString"), ("blockSize", "block size for stacking input data in matrices. Data is stacked within " "partitions. If block size is more than remaining data in a partition then it is " - "adjusted to the size of this data.", "4096", "TypeConverters.toInt")] + "adjusted to the size of this data.", "1024", "TypeConverters.toInt")] code = [] for name, doc, defaultValueStr, typeConverter in shared: diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 41ba7b9dc5523..456463580878c 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -591,7 +591,7 @@ class HasBlockSize(Params): def __init__(self): super(HasBlockSize, self).__init__() - self._setDefault(blockSize=4096) + self._setDefault(blockSize=1024) def getBlockSize(self): """ From a291433ed316932618583544ee6d0f1b2f829b80 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 30 Jan 2020 09:05:14 -0800 Subject: [PATCH 1250/1387] [SPARK-30678][MLLIB][TESTS] Eliminate warnings from deprecated BisectingKMeansModel.computeCost ### What changes were proposed in this pull request? In the PR, I propose to replace deprecated method `computeCost` of `BisectingKMeansModel` by `summary.trainingCost`. ### Why are the changes needed? The changes eliminate deprecation warnings: ``` BisectingKMeansSuite.scala:108: method computeCost in class BisectingKMeansModel is deprecated (since 3.0.0): This method is deprecated and will be removed in future versions. Use ClusteringEvaluator instead. You can also get the cost on the training dataset in the summary. [warn] assert(model.computeCost(dataset) < 0.1) BisectingKMeansSuite.scala:135: method computeCost in class BisectingKMeansModel is deprecated (since 3.0.0): This method is deprecated and will be removed in future versions. Use ClusteringEvaluator instead. You can also get the cost on the training dataset in the summary. [warn] assert(model.computeCost(dataset) == summary.trainingCost) BisectingKMeansSuite.scala:323: method computeCost in class BisectingKMeansModel is deprecated (since 3.0.0): This method is deprecated and will be removed in future versions. Use ClusteringEvaluator instead. You can also get the cost on the training dataset in the summary. [warn] model.computeCost(dataset) ``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running `BisectingKMeansSuite` via: ``` ./build/sbt "test:testOnly *BisectingKMeansSuite" ``` Closes #27401 from MaxGekk/kmeans-computeCost-warning. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../apache/spark/ml/clustering/BisectingKMeansSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index fc756d4c710d3..debd0dd65d0c8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -105,7 +105,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { val bkm = new BisectingKMeans().setK(k).setPredictionCol(predictionColName).setSeed(1) val model = bkm.fit(dataset) assert(model.clusterCenters.length === k) - assert(model.computeCost(dataset) < 0.1) + assert(model.summary.trainingCost < 0.1) assert(model.hasParent) testTransformerByGlobalCheckFunc[Tuple1[Vector]](dataset.toDF(), model, @@ -132,7 +132,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { assert(clusterSizes.forall(_ >= 0)) assert(summary.numIter == 20) assert(summary.trainingCost < 0.1) - assert(model.computeCost(dataset) == summary.trainingCost) + assert(model.summary.trainingCost == summary.trainingCost) model.setSummary(None) assert(!model.hasSummary) @@ -320,7 +320,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { test("BisectingKMeans with Array input") { def trainAndComputeCost(dataset: DataFrame): Double = { val model = new BisectingKMeans().setK(k).setMaxIter(1).setSeed(1).fit(dataset) - model.computeCost(dataset) + model.summary.trainingCost } val (newDataset, newDatasetD, newDatasetF) = MLTestingUtils.generateArrayFeatureDataset(dataset) From b0db6231fddefbd418da3309bc76f54bc7875549 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Thu, 30 Jan 2020 09:41:32 -0800 Subject: [PATCH 1251/1387] [SPARK-29020][FOLLOWUP][SQL] Update description of array_sort function ### What changes were proposed in this pull request? This PR is a follow-up of #25728. #25728 introduces additional arguments to determine sort order. Thus, this function does not sort only in ascending order. However, the description was not updated. This PR updates the description to follow the latest feature. ### Why are the changes needed? ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests since this PR just updates description text. Closes #27404 from kiszk/SPARK-29020-followup. Authored-by: Kazuaki Ishizaki Signed-off-by: Dongjoon Hyun --- .../catalyst/expressions/higherOrderFunctions.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index adeda0981fe8e..f8142d6b993cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -291,11 +291,11 @@ case class ArrayTransform( */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = """_FUNC_(expr, func) - Sorts the input array in ascending order. The elements of the - input array must be orderable. Null elements will be placed at the end of the returned - array. Since 3.0.0 this function also sorts and returns the array based on the given - comparator function. The comparator will take two arguments - representing two elements of the array. + usage = """_FUNC_(expr, func) - Sorts the input array. If func is omitted, sort + in ascending order. The elements of the input array must be orderable. Null elements + will be placed at the end of the returned array. Since 3.0.0 this function also sorts + and returns the array based on the given comparator function. The comparator will + take two arguments representing two elements of the array. It returns -1, 0, or 1 as the first element is less than, equal to, or greater than the second element. If the comparator function returns other values (including null), the function will fail and raise an error. From 7503e76af0a2cc1887ea89a1be2a5d3f3bb7d351 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 30 Jan 2020 10:27:35 -0800 Subject: [PATCH 1252/1387] [SPARK-30622][SQL] commands should return dummy statistics ### What changes were proposed in this pull request? override `Command.stats` to return a dummy statistics (Long.Max). ### Why are the changes needed? Commands are eagerly executed. They will be converted to LocalRelation after the DataFrame is created. That said, the statistics of a command is useless. We should avoid unnecessary statistics calculation of command's children. ### Does this PR introduce any user-facing change? no ### How was this patch tested? new test Closes #27344 from cloud-fan/command. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/plans/logical/Command.scala | 4 ++++ .../sql/catalyst/plans/logical/Statistics.scala | 3 +++ .../statsEstimation/BasicStatsEstimationSuite.scala | 13 +++++++++++++ 3 files changed, 20 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala index ec5766e1f67f2..732c8ce2b5d98 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala @@ -27,4 +27,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute trait Command extends LogicalPlan { override def output: Seq[Attribute] = Seq.empty override def children: Seq[LogicalPlan] = Seq.empty + // Commands are eagerly executed. They will be converted to LocalRelation after the DataFrame + // is created. That said, the statistics of a command is useless. Here we just return a dummy + // statistics to avoid unnecessary statistics calculation of command's children. + override def stats: Statistics = Statistics.DUMMY } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index c008d776e4794..49f89bed154bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -35,6 +35,9 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils +object Statistics { + val DUMMY = Statistics(Long.MaxValue) +} /** * Estimates of various statistics. The default estimation logic simply lazily multiplies the diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 16a5c2d3001a7..dfe790dca54d8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.catalyst.statsEstimation +import org.mockito.Mockito.mock + +import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Literal} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.connector.catalog.SupportsNamespaces import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType @@ -115,6 +119,15 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { plan, expectedStatsCboOn = expectedCboStats, expectedStatsCboOff = expectedDefaultStats) } + test("command should report a dummy stats") { + val plan = CommentOnNamespace( + ResolvedNamespace(mock(classOf[SupportsNamespaces]), Array("ns")), "comment") + checkStats( + plan, + expectedStatsCboOn = Statistics.DUMMY, + expectedStatsCboOff = Statistics.DUMMY) + } + /** Check estimated stats when cbo is turned on/off. */ private def checkStats( plan: LogicalPlan, From e5f572af067adf3d7e9ba4adbaa8e7efa3385527 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 30 Jan 2020 10:34:59 -0800 Subject: [PATCH 1253/1387] [SPARK-30680][SQL] ResolvedNamespace does not require a namespace catalog ### What changes were proposed in this pull request? Update `ResolvedNamespace` to accept catalog as `CatalogPlugin` not `SupportsNamespaces`. This is extracted from https://github.com/apache/spark/pull/27345 ### Why are the changes needed? not all commands that need to resolve namespaces require a namespace catalog. For example, `SHOW TABLE` is implemented by `TableCatalog.listTables`, and is nothing to do with `SupportsNamespace`. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #27403 from cloud-fan/ns. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 7 +++---- .../sql/catalyst/analysis/v2ResolutionPlans.scala | 4 ++-- .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 2 +- .../datasources/v2/DataSourceV2Strategy.scala | 10 +++++----- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 15ebf6971d9dd..53d73d503b097 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -731,12 +731,11 @@ class Analyzer( extends Rule[LogicalPlan] with LookupCatalog { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case s @ ShowTables(UnresolvedNamespace(Seq()), _) => - s.copy(namespace = - ResolvedNamespace(currentCatalog.asNamespaceCatalog, catalogManager.currentNamespace)) + s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) case UnresolvedNamespace(Seq()) => - ResolvedNamespace(currentCatalog.asNamespaceCatalog, Seq.empty[String]) + ResolvedNamespace(currentCatalog, Seq.empty[String]) case UnresolvedNamespace(CatalogAndNamespace(catalog, ns)) => - ResolvedNamespace(catalog.asNamespaceCatalog, ns) + ResolvedNamespace(catalog, ns) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 239f987e97a76..f3d40c6d36cc3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} -import org.apache.spark.sql.connector.catalog.{Identifier, SupportsNamespaces, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, SupportsNamespaces, Table, TableCatalog} /** * Holds the name of a namespace that has yet to be looked up in a catalog. It will be resolved to @@ -53,7 +53,7 @@ case class UnresolvedTableOrView(multipartIdentifier: Seq[String]) extends LeafN /** * A plan containing resolved namespace. */ -case class ResolvedNamespace(catalog: SupportsNamespaces, namespace: Seq[String]) +case class ResolvedNamespace(catalog: CatalogPlugin, namespace: Seq[String]) extends LeafNode { override def output: Seq[Attribute] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 191029ab7b432..12439fd94c679 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -580,7 +580,7 @@ class ResolveSessionCatalog( } object SessionCatalogAndNamespace { - def unapply(resolved: ResolvedNamespace): Option[(SupportsNamespaces, Seq[String])] = + def unapply(resolved: ResolvedNamespace): Option[(CatalogPlugin, Seq[String])] = if (isSessionCatalog(resolved.catalog)) { Some(resolved.catalog -> resolved.namespace) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 8b4b6fb64658a..8f4e2d256c714 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -230,7 +230,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat } case desc @ DescribeNamespace(ResolvedNamespace(catalog, ns), extended) => - DescribeNamespaceExec(desc.output, catalog, ns, extended) :: Nil + DescribeNamespaceExec(desc.output, catalog.asNamespaceCatalog, ns, extended) :: Nil case desc @ DescribeRelation(r: ResolvedTable, partitionSpec, isExtended) => if (partitionSpec.nonEmpty) { @@ -248,17 +248,17 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat RenameTableExec(catalog, oldIdent, newIdent) :: Nil case AlterNamespaceSetProperties(ResolvedNamespace(catalog, ns), properties) => - AlterNamespaceSetPropertiesExec(catalog, ns, properties) :: Nil + AlterNamespaceSetPropertiesExec(catalog.asNamespaceCatalog, ns, properties) :: Nil case AlterNamespaceSetLocation(ResolvedNamespace(catalog, ns), location) => AlterNamespaceSetPropertiesExec( - catalog, + catalog.asNamespaceCatalog, ns, Map(SupportsNamespaces.PROP_LOCATION -> location)) :: Nil case CommentOnNamespace(ResolvedNamespace(catalog, ns), comment) => AlterNamespaceSetPropertiesExec( - catalog, + catalog.asNamespaceCatalog, ns, Map(SupportsNamespaces.PROP_COMMENT -> comment)) :: Nil @@ -273,7 +273,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat DropNamespaceExec(catalog, ns, ifExists, cascade) :: Nil case r @ ShowNamespaces(ResolvedNamespace(catalog, ns), pattern) => - ShowNamespacesExec(r.output, catalog, ns, pattern) :: Nil + ShowNamespacesExec(r.output, catalog.asNamespaceCatalog, ns, pattern) :: Nil case r @ ShowTables(ResolvedNamespace(catalog, ns), pattern) => ShowTablesExec(r.output, catalog.asTableCatalog, ns, pattern) :: Nil From f59685acaa3e9c227f14fe4d8f9e94a1ac664b05 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 30 Jan 2020 13:13:10 -0600 Subject: [PATCH 1254/1387] [SPARK-30662][ML][PYSPARK] ALS/MLP extend HasBlockSize ### What changes were proposed in this pull request? Make ALS/MLP extend ```HasBlockSize``` ### Why are the changes needed? Currently, MLP has its own ```blockSize``` param, we should make MLP extend ```HasBlockSize``` since ```HasBlockSize``` was added in ```sharedParams.scala``` recently. ALS doesn't have ```blockSize``` param now, we can make it extend ```HasBlockSize```, so user can specify the ```blockSize```. ### Does this PR introduce any user-facing change? Yes ```ALS.setBlockSize``` and ```ALS.getBlockSize``` ```ALSModel.setBlockSize``` and ```ALSModel.getBlockSize``` ### How was this patch tested? Manually tested. Also added doctest. Closes #27389 from huaxingao/spark-30662. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../MultilayerPerceptronClassifier.scala | 22 +-------- .../apache/spark/ml/recommendation/ALS.scala | 46 ++++++++++++++----- python/pyspark/ml/classification.py | 22 ++++----- python/pyspark/ml/recommendation.py | 29 +++++++++--- 4 files changed, 66 insertions(+), 53 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index c7a8237849b5b..6e8f92b9b1e64 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.VersionUtils.majorMinorVersion /** Params for Multilayer Perceptron. */ private[classification] trait MultilayerPerceptronParams extends ProbabilisticClassifierParams - with HasSeed with HasMaxIter with HasTol with HasStepSize with HasSolver { + with HasSeed with HasMaxIter with HasTol with HasStepSize with HasSolver with HasBlockSize { import MultilayerPerceptronClassifier._ @@ -54,26 +54,6 @@ private[classification] trait MultilayerPerceptronParams extends ProbabilisticCl @Since("1.5.0") final def getLayers: Array[Int] = $(layers) - /** - * Block size for stacking input data in matrices to speed up the computation. - * Data is stacked within partitions. If block size is more than remaining data in - * a partition then it is adjusted to the size of this data. - * Recommended size is between 10 and 1000. - * Default: 128 - * - * @group expertParam - */ - @Since("1.5.0") - final val blockSize: IntParam = new IntParam(this, "blockSize", - "Block size for stacking input data in matrices. Data is stacked within partitions." + - " If block size is more than remaining data in a partition then " + - "it is adjusted to the size of this data. Recommended size is between 10 and 1000", - ParamValidators.gt(0)) - - /** @group expertGetParam */ - @Since("1.5.0") - final def getBlockSize: Int = $(blockSize) - /** * The solver algorithm for optimization. * Supported options: "gd" (minibatch gradient descent) or "l-bfgs". diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 2fb9a276be887..002146f89e79a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -54,7 +54,8 @@ import org.apache.spark.util.random.XORShiftRandom /** * Common params for ALS and ALSModel. */ -private[recommendation] trait ALSModelParams extends Params with HasPredictionCol { +private[recommendation] trait ALSModelParams extends Params with HasPredictionCol + with HasBlockSize { /** * Param for the column name for user ids. Ids must be integers. Other * numeric types are supported for this column, but will be cast to integers as long as they @@ -125,6 +126,8 @@ private[recommendation] trait ALSModelParams extends Params with HasPredictionCo /** @group expertGetParam */ def getColdStartStrategy: String = $(coldStartStrategy).toLowerCase(Locale.ROOT) + + setDefault(blockSize -> 4096) } /** @@ -288,6 +291,15 @@ class ALSModel private[ml] ( @Since("2.2.0") def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) + /** + * Set block size for stacking input data in matrices. + * Default is 4096. + * + * @group expertSetParam + */ + @Since("3.0.0") + def setBlockSize(value: Int): this.type = set(blockSize, value) + private val predict = udf { (featuresA: Seq[Float], featuresB: Seq[Float]) => if (featuresA != null && featuresB != null) { var dotProduct = 0.0f @@ -351,7 +363,7 @@ class ALSModel private[ml] ( */ @Since("2.2.0") def recommendForAllUsers(numItems: Int): DataFrame = { - recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems) + recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) } /** @@ -366,7 +378,7 @@ class ALSModel private[ml] ( @Since("2.3.0") def recommendForUserSubset(dataset: Dataset[_], numItems: Int): DataFrame = { val srcFactorSubset = getSourceFactorSubset(dataset, userFactors, $(userCol)) - recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems) + recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) } /** @@ -377,7 +389,7 @@ class ALSModel private[ml] ( */ @Since("2.2.0") def recommendForAllItems(numUsers: Int): DataFrame = { - recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers) + recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) } /** @@ -392,7 +404,7 @@ class ALSModel private[ml] ( @Since("2.3.0") def recommendForItemSubset(dataset: Dataset[_], numUsers: Int): DataFrame = { val srcFactorSubset = getSourceFactorSubset(dataset, itemFactors, $(itemCol)) - recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers) + recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) } /** @@ -441,11 +453,12 @@ class ALSModel private[ml] ( dstFactors: DataFrame, srcOutputColumn: String, dstOutputColumn: String, - num: Int): DataFrame = { + num: Int, + blockSize: Int): DataFrame = { import srcFactors.sparkSession.implicits._ - val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])]) - val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])]) + val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])], blockSize) + val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])], blockSize) val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked) .as[(Seq[(Int, Array[Float])], Seq[(Int, Array[Float])])] .flatMap { case (srcIter, dstIter) => @@ -483,11 +496,10 @@ class ALSModel private[ml] ( /** * Blockifies factors to improve the efficiency of cross join - * TODO: SPARK-20443 - expose blockSize as a param? */ private def blockify( factors: Dataset[(Int, Array[Float])], - blockSize: Int = 4096): Dataset[Seq[(Int, Array[Float])]] = { + blockSize: Int): Dataset[Seq[(Int, Array[Float])]] = { import factors.sparkSession.implicits._ factors.mapPartitions(_.grouped(blockSize)) } @@ -654,6 +666,15 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] @Since("2.2.0") def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) + /** + * Set block size for stacking input data in matrices. + * Default is 4096. + * + * @group expertSetParam + */ + @Since("3.0.0") + def setBlockSize(value: Int): this.type = set(blockSize, value) + /** * Sets both numUserBlocks and numItemBlocks to the specific value. * @@ -683,7 +704,7 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] instr.logDataset(dataset) instr.logParams(this, rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, userCol, itemCol, ratingCol, predictionCol, maxIter, regParam, nonnegative, checkpointInterval, - seed, intermediateStorageLevel, finalStorageLevel) + seed, intermediateStorageLevel, finalStorageLevel, blockSize) val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), @@ -694,7 +715,8 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] checkpointInterval = $(checkpointInterval), seed = $(seed)) val userDF = userFactors.toDF("id", "features") val itemDF = itemFactors.toDF("id", "features") - val model = new ALSModel(uid, $(rank), userDF, itemDF).setParent(this) + val model = new ALSModel(uid, $(rank), userDF, itemDF).setBlockSize($(blockSize)) + .setParent(this) copyValues(model) } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index bb9cd034808fc..5ee42318afd45 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -2174,7 +2174,7 @@ def sigma(self): class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, HasMaxIter, - HasTol, HasStepSize, HasSolver): + HasTol, HasStepSize, HasSolver, HasBlockSize): """ Params for :py:class:`MultilayerPerceptronClassifier`. @@ -2185,11 +2185,6 @@ class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, H "E.g., Array(780, 100, 10) means 780 inputs, one hidden layer with 100 " + "neurons and output layer of 10 neurons.", typeConverter=TypeConverters.toListInt) - blockSize = Param(Params._dummy(), "blockSize", "Block size for stacking input data in " + - "matrices. Data is stacked within partitions. If block size is more than " + - "remaining data in a partition then it is adjusted to the size of this " + - "data. Recommended size is between 10 and 1000, default is 128.", - typeConverter=TypeConverters.toInt) solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: l-bfgs, gd.", typeConverter=TypeConverters.toString) initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", @@ -2202,13 +2197,6 @@ def getLayers(self): """ return self.getOrDefault(self.layers) - @since("1.6.0") - def getBlockSize(self): - """ - Gets the value of blockSize or its default value. - """ - return self.getOrDefault(self.blockSize) - @since("2.0.0") def getInitialWeights(self): """ @@ -2232,11 +2220,17 @@ class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, _MultilayerPer ... (1.0, Vectors.dense([0.0, 1.0])), ... (1.0, Vectors.dense([1.0, 0.0])), ... (0.0, Vectors.dense([1.0, 1.0]))], ["label", "features"]) - >>> mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], blockSize=1, seed=123) + >>> mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], seed=123) >>> mlp.setMaxIter(100) MultilayerPerceptronClassifier... >>> mlp.getMaxIter() 100 + >>> mlp.getBlockSize() + 128 + >>> mlp.setBlockSize(1) + MultilayerPerceptronClassifier... + >>> mlp.getBlockSize() + 1 >>> model = mlp.fit(df) >>> model.setFeaturesCol("features") MultilayerPerceptronClassificationModel... diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index ee276962c898b..fe571e25c05f5 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -28,7 +28,7 @@ @inherit_doc -class _ALSModelParams(HasPredictionCol): +class _ALSModelParams(HasPredictionCol, HasBlockSize): """ Params for :py:class:`ALS` and :py:class:`ALSModel`. @@ -223,6 +223,8 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): 0.1 >>> als.clear(als.regParam) >>> model = als.fit(df) + >>> model.getBlockSize() + 4096 >>> model.getUserCol() 'user' >>> model.setUserCol("user") @@ -282,13 +284,13 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan"): + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096): """ __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, \ implicitPrefs=false, alpha=1.0, userCol="user", itemCol="item", seed=None, \ ratingCol="rating", nonnegative=false, checkpointInterval=10, \ intermediateStorageLevel="MEMORY_AND_DISK", \ - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", lockSize=4096) """ super(ALS, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.recommendation.ALS", self.uid) @@ -296,7 +298,8 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", + blockSize=4096) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -306,13 +309,13 @@ def setParams(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItem implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan"): + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096): """ setParams(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, \ implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, \ ratingCol="rating", nonnegative=False, checkpointInterval=10, \ intermediateStorageLevel="MEMORY_AND_DISK", \ - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096) Sets params for ALS. """ kwargs = self._input_kwargs @@ -443,6 +446,13 @@ def setSeed(self, value): """ return self._set(seed=value) + @since("3.0.0") + def setBlockSize(self, value): + """ + Sets the value of :py:attr:`blockSize`. + """ + return self._set(blockSize=value) + class ALSModel(JavaModel, _ALSModelParams, JavaMLWritable, JavaMLReadable): """ @@ -479,6 +489,13 @@ def setPredictionCol(self, value): """ return self._set(predictionCol=value) + @since("3.0.0") + def setBlockSize(self, value): + """ + Sets the value of :py:attr:`blockSize`. + """ + return self._set(blockSize=value) + @property @since("1.4.0") def rank(self): From 5916c7d0d0a68266ede1b7e54c3498a18142cf8f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 30 Jan 2020 13:23:58 -0800 Subject: [PATCH 1255/1387] [SPARK-30673][SQL][TESTS] Test cases in HiveShowCreateTableSuite should create Hive table ### What changes were proposed in this pull request? This patch makes the test cases in HiveShowCreateTableSuite create Hive table instead of data source table. ### Why are the changes needed? Because SparkSQL now creates data source table if no provider is specified in SQL command, some test cases in HiveShowCreateTableSuite don't create Hive table, but data source table. It is confusing and not good for the purpose of this test suite. ### Does this PR introduce any user-facing change? No, only test case. ### How was this patch tested? Unit test. Closes #27393 from viirya/SPARK-30673. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- .../sql/hive/HiveShowCreateTableSuite.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala index a9cd2831f5abe..a551ab704b62a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala @@ -19,9 +19,25 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.{AnalysisException, ShowCreateTableSuite} import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSingleton { + private var origCreateHiveTableConfig = false + + protected override def beforeAll(): Unit = { + super.beforeAll() + origCreateHiveTableConfig = + SQLConf.get.getConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED) + SQLConf.get.setConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED, true) + } + + protected override def afterAll(): Unit = { + SQLConf.get.setConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT_ENABLED, + origCreateHiveTableConfig) + super.afterAll() + } + test("simple hive table") { withTable("t1") { sql( From f56ba37d8bf618f2bef23d808e0fc5704261b139 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 30 Jan 2020 18:14:50 -0800 Subject: [PATCH 1256/1387] [SPARK-30656][SS] Support the "minPartitions" option in Kafka batch source and streaming source v1 ### What changes were proposed in this pull request? - Add `minPartitions` support for Kafka Streaming V1 source. - Add `minPartitions` support for Kafka batch V1 and V2 source. - There is lots of refactoring (moving codes to KafkaOffsetReader) to reuse codes. ### Why are the changes needed? Right now, the "minPartitions" option only works in Kafka streaming source v2. It would be great that we can support it in batch and streaming source v1 (v1 is the fallback mode when a user hits a regression in v2) as well. ### Does this PR introduce any user-facing change? Yep. The `minPartitions` options is supported in Kafka batch and streaming queries for both data source V1 and V2. ### How was this patch tested? New unit tests are added to test "minPartitions". Closes #27388 from zsxwing/kafka-min-partitions. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu --- .../structured-streaming-kafka-integration.md | 2 +- .../spark/sql/kafka010/KafkaBatch.scala | 32 +--- .../sql/kafka010/KafkaMicroBatchStream.scala | 75 +-------- .../kafka010/KafkaOffsetRangeCalculator.scala | 20 ++- .../sql/kafka010/KafkaOffsetReader.scala | 156 ++++++++++++++++++ .../spark/sql/kafka010/KafkaRelation.scala | 32 +--- .../spark/sql/kafka010/KafkaSource.scala | 64 +------ .../spark/sql/kafka010/KafkaSourceRDD.scala | 21 +-- .../kafka010/KafkaMicroBatchSourceSuite.scala | 29 ++++ .../KafkaOffsetRangeCalculatorSuite.scala | 107 +++++++----- .../sql/kafka010/KafkaOffsetReaderSuite.scala | 139 ++++++++++++++++ .../sql/kafka010/KafkaRelationSuite.scala | 22 +++ 12 files changed, 448 insertions(+), 251 deletions(-) create mode 100644 external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderSuite.scala diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 0820b38a863c2..a1eeee54987db 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -469,7 +469,7 @@ The following configurations are optional: minPartitions int none - streaming + streaming and batch Desired minimum number of partitions to read from Kafka. By default, Spark has a 1-1 mapping of topicPartitions to Spark partitions consuming from Kafka. If you set this option to a value greater than your topicPartitions, Spark will divvy up large diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala index 3006770f306c0..9ad083f1cfde5 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala @@ -57,36 +57,12 @@ private[kafka010] class KafkaBatch( driverGroupIdPrefix = s"$uniqueGroupId-driver") // Leverage the KafkaReader to obtain the relevant partition offsets - val (fromPartitionOffsets, untilPartitionOffsets) = { - try { - (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets, isStartingOffsets = true), - kafkaOffsetReader.fetchPartitionOffsets(endingOffsets, isStartingOffsets = false)) - } finally { - kafkaOffsetReader.close() - } + val offsetRanges: Seq[KafkaOffsetRange] = try { + kafkaOffsetReader.getOffsetRangesFromUnresolvedOffsets(startingOffsets, endingOffsets) + } finally { + kafkaOffsetReader.close() } - // Obtain topicPartitions in both from and until partition offset, ignoring - // topic partitions that were added and/or deleted between the two above calls. - if (fromPartitionOffsets.keySet != untilPartitionOffsets.keySet) { - implicit val topicOrdering: Ordering[TopicPartition] = Ordering.by(t => t.topic()) - val fromTopics = fromPartitionOffsets.keySet.toList.sorted.mkString(",") - val untilTopics = untilPartitionOffsets.keySet.toList.sorted.mkString(",") - throw new IllegalStateException("different topic partitions " + - s"for starting offsets topics[${fromTopics}] and " + - s"ending offsets topics[${untilTopics}]") - } - - // Calculate offset ranges - val offsetRanges = untilPartitionOffsets.keySet.map { tp => - val fromOffset = fromPartitionOffsets.getOrElse(tp, - // This should not happen since topicPartitions contains all partitions not in - // fromPartitionOffsets - throw new IllegalStateException(s"$tp doesn't have a from offset")) - val untilOffset = untilPartitionOffsets(tp) - KafkaOffsetRange(tp, fromOffset, untilOffset, None) - }.toArray - val executorKafkaParams = KafkaSourceProvider.kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId) offsetRanges.map { range => diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index 01f6ba4445162..844c963619855 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -66,8 +66,6 @@ private[kafka010] class KafkaMicroBatchStream( private val includeHeaders = options.getBoolean(INCLUDE_HEADERS, false) - private val rangeCalculator = KafkaOffsetRangeCalculator(options) - private var endPartitionOffsets: KafkaSourceOffset = _ /** @@ -94,57 +92,11 @@ private[kafka010] class KafkaMicroBatchStream( val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets val endPartitionOffsets = end.asInstanceOf[KafkaSourceOffset].partitionToOffsets - // Find the new partitions, and get their earliest offsets - val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet) - val newPartitionInitialOffsets = kafkaOffsetReader.fetchEarliestOffsets(newPartitions.toSeq) - if (newPartitionInitialOffsets.keySet != newPartitions) { - // We cannot get from offsets for some partitions. It means they got deleted. - val deletedPartitions = newPartitions.diff(newPartitionInitialOffsets.keySet) - reportDataLoss( - s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed") - } - logInfo(s"Partitions added: $newPartitionInitialOffsets") - newPartitionInitialOffsets.filter(_._2 != 0).foreach { case (p, o) => - reportDataLoss( - s"Added partition $p starts from $o instead of 0. Some data may have been missed") - } - - // Find deleted partitions, and report data loss if required - val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet) - if (deletedPartitions.nonEmpty) { - val message = - if (kafkaOffsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { - s"$deletedPartitions are gone. ${CUSTOM_GROUP_ID_ERROR_MESSAGE}" - } else { - s"$deletedPartitions are gone. Some data may have been missed." - } - reportDataLoss(message) - } - - // Use the end partitions to calculate offset ranges to ignore partitions that have - // been deleted - val topicPartitions = endPartitionOffsets.keySet.filter { tp => - // Ignore partitions that we don't know the from offsets. - newPartitionInitialOffsets.contains(tp) || startPartitionOffsets.contains(tp) - }.toSeq - logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) - - val fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets - val untilOffsets = endPartitionOffsets - untilOffsets.foreach { case (tp, untilOffset) => - fromOffsets.get(tp).foreach { fromOffset => - if (untilOffset < fromOffset) { - reportDataLoss(s"Partition $tp's offset was changed from " + - s"$fromOffset to $untilOffset, some data may have been missed") - } - } - } - - // Calculate offset ranges - val offsetRanges = rangeCalculator.getRanges( - fromOffsets = fromOffsets, - untilOffsets = untilOffsets, - executorLocations = getSortedExecutorList()) + val offsetRanges = kafkaOffsetReader.getOffsetRangesFromResolvedOffsets( + startPartitionOffsets, + endPartitionOffsets, + reportDataLoss + ) // Generate factories based on the offset ranges offsetRanges.map { range => @@ -242,23 +194,6 @@ private[kafka010] class KafkaMicroBatchStream( } } - private def getSortedExecutorList(): Array[String] = { - - def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { - if (a.host == b.host) { - a.executorId > b.executorId - } else { - a.host > b.host - } - } - - val bm = SparkEnv.get.blockManager - bm.master.getPeers(bm.blockManagerId).toArray - .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) - .sortWith(compare) - .map(_.toString) - } - /** * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`. * Otherwise, just log a warning. diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala index ead45423d673f..f7183f7add14b 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala @@ -41,14 +41,9 @@ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int * Empty ranges (`KafkaOffsetRange.size <= 0`) will be dropped. */ def getRanges( - fromOffsets: PartitionOffsetMap, - untilOffsets: PartitionOffsetMap, + ranges: Seq[KafkaOffsetRange], executorLocations: Seq[String] = Seq.empty): Seq[KafkaOffsetRange] = { - val partitionsToRead = untilOffsets.keySet.intersect(fromOffsets.keySet) - - val offsetRanges = partitionsToRead.toSeq.map { tp => - KafkaOffsetRange(tp, fromOffsets(tp), untilOffsets(tp), preferredLoc = None) - }.filter(_.size > 0) + val offsetRanges = ranges.filter(_.size > 0) // If minPartitions not set or there are enough partitions to satisfy minPartitions if (minPartitions.isEmpty || offsetRanges.size > minPartitions.get) { @@ -106,6 +101,13 @@ private[kafka010] case class KafkaOffsetRange( topicPartition: TopicPartition, fromOffset: Long, untilOffset: Long, - preferredLoc: Option[String]) { - lazy val size: Long = untilOffset - fromOffset + preferredLoc: Option[String] = None) { + def topic: String = topicPartition.topic + def partition: Int = topicPartition.partition + /** + * The estimated size of messages in the range. It may be different than the real number of + * messages due to log compaction or transaction metadata. It should not be used to provide + * answers directly. + */ + def size: Long = untilOffset - fromOffset } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 0179f4dd822f1..216e74a85c2ae 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -29,7 +29,9 @@ import scala.util.control.NonFatal import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetAndTimestamp} import org.apache.kafka.common.TopicPartition +import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} @@ -91,9 +93,27 @@ private[kafka010] class KafkaOffsetReader( private[kafka010] val maxOffsetFetchAttempts = readerOptions.getOrElse(KafkaSourceProvider.FETCH_OFFSET_NUM_RETRY, "3").toInt + /** + * Number of partitions to read from Kafka. If this value is greater than the number of Kafka + * topicPartitions, we will split up the read tasks of the skewed partitions to multiple Spark + * tasks. The number of Spark tasks will be *approximately* `numPartitions`. It can be less or + * more depending on rounding errors or Kafka partitions that didn't receive any new data. + */ + private val minPartitions = + readerOptions.get(KafkaSourceProvider.MIN_PARTITIONS_OPTION_KEY).map(_.toInt) + + private val rangeCalculator = new KafkaOffsetRangeCalculator(minPartitions) + private[kafka010] val offsetFetchAttemptIntervalMs = readerOptions.getOrElse(KafkaSourceProvider.FETCH_OFFSET_RETRY_INTERVAL_MS, "1000").toLong + /** + * Whether we should divide Kafka TopicPartitions with a lot of data into smaller Spark tasks. + */ + private def shouldDivvyUpLargePartitions(numTopicPartitions: Int): Boolean = { + minPartitions.map(_ > numTopicPartitions).getOrElse(false) + } + private def nextGroupId(): String = { groupId = driverGroupIdPrefix + "-" + nextId nextId += 1 @@ -372,6 +392,142 @@ private[kafka010] class KafkaOffsetReader( } } + /** + * Return the offset ranges for a Kafka batch query. If `minPartitions` is set, this method may + * split partitions to respect it. Since offsets can be early and late binding which are evaluated + * on the executors, in order to divvy up the partitions we need to perform some substitutions. We + * don't want to send exact offsets to the executors, because data may age out before we can + * consume the data. This method makes some approximate splitting, and replaces the special offset + * values in the final output. + */ + def getOffsetRangesFromUnresolvedOffsets( + startingOffsets: KafkaOffsetRangeLimit, + endingOffsets: KafkaOffsetRangeLimit): Seq[KafkaOffsetRange] = { + val fromPartitionOffsets = fetchPartitionOffsets(startingOffsets, isStartingOffsets = true) + val untilPartitionOffsets = fetchPartitionOffsets(endingOffsets, isStartingOffsets = false) + + // Obtain topicPartitions in both from and until partition offset, ignoring + // topic partitions that were added and/or deleted between the two above calls. + if (fromPartitionOffsets.keySet != untilPartitionOffsets.keySet) { + implicit val topicOrdering: Ordering[TopicPartition] = Ordering.by(t => t.topic()) + val fromTopics = fromPartitionOffsets.keySet.toList.sorted.mkString(",") + val untilTopics = untilPartitionOffsets.keySet.toList.sorted.mkString(",") + throw new IllegalStateException("different topic partitions " + + s"for starting offsets topics[${fromTopics}] and " + + s"ending offsets topics[${untilTopics}]") + } + + // Calculate offset ranges + val offsetRangesBase = untilPartitionOffsets.keySet.map { tp => + val fromOffset = fromPartitionOffsets.get(tp).getOrElse { + // This should not happen since topicPartitions contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset") + } + val untilOffset = untilPartitionOffsets(tp) + KafkaOffsetRange(tp, fromOffset, untilOffset, None) + }.toSeq + + if (shouldDivvyUpLargePartitions(offsetRangesBase.size)) { + val fromOffsetsMap = + offsetRangesBase.map(range => (range.topicPartition, range.fromOffset)).toMap + val untilOffsetsMap = + offsetRangesBase.map(range => (range.topicPartition, range.untilOffset)).toMap + + // No need to report data loss here + val resolvedFromOffsets = fetchSpecificOffsets(fromOffsetsMap, _ => ()).partitionToOffsets + val resolvedUntilOffsets = fetchSpecificOffsets(untilOffsetsMap, _ => ()).partitionToOffsets + val ranges = offsetRangesBase.map(_.topicPartition).map { tp => + KafkaOffsetRange(tp, resolvedFromOffsets(tp), resolvedUntilOffsets(tp), preferredLoc = None) + } + val divvied = rangeCalculator.getRanges(ranges).groupBy(_.topicPartition) + divvied.flatMap { case (tp, splitOffsetRanges) => + if (splitOffsetRanges.length == 1) { + Seq(KafkaOffsetRange(tp, fromOffsetsMap(tp), untilOffsetsMap(tp), None)) + } else { + // the list can't be empty + val first = splitOffsetRanges.head.copy(fromOffset = fromOffsetsMap(tp)) + val end = splitOffsetRanges.last.copy(untilOffset = untilOffsetsMap(tp)) + Seq(first) ++ splitOffsetRanges.drop(1).dropRight(1) :+ end + } + }.toArray.toSeq + } else { + offsetRangesBase + } + } + + private def getSortedExecutorList(): Array[String] = { + def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { + if (a.host == b.host) { + a.executorId > b.executorId + } else { + a.host > b.host + } + } + + val bm = SparkEnv.get.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compare) + .map(_.toString) + } + + /** + * Return the offset ranges for a Kafka streaming batch. If `minPartitions` is set, this method + * may split partitions to respect it. If any data lost issue is detected, `reportDataLoss` will + * be called. + */ + def getOffsetRangesFromResolvedOffsets( + fromPartitionOffsets: PartitionOffsetMap, + untilPartitionOffsets: PartitionOffsetMap, + reportDataLoss: String => Unit): Seq[KafkaOffsetRange] = { + // Find the new partitions, and get their earliest offsets + val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) + val newPartitionInitialOffsets = fetchEarliestOffsets(newPartitions.toSeq) + if (newPartitionInitialOffsets.keySet != newPartitions) { + // We cannot get from offsets for some partitions. It means they got deleted. + val deletedPartitions = newPartitions.diff(newPartitionInitialOffsets.keySet) + reportDataLoss( + s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed") + } + logInfo(s"Partitions added: $newPartitionInitialOffsets") + newPartitionInitialOffsets.filter(_._2 != 0).foreach { case (p, o) => + reportDataLoss( + s"Added partition $p starts from $o instead of 0. Some data may have been missed") + } + + val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) + if (deletedPartitions.nonEmpty) { + val message = if (driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + } else { + s"$deletedPartitions are gone. Some data may have been missed." + } + reportDataLoss(message) + } + + // Use the until partitions to calculate offset ranges to ignore partitions that have + // been deleted + val topicPartitions = untilPartitionOffsets.keySet.filter { tp => + // Ignore partitions that we don't know the from offsets. + newPartitionInitialOffsets.contains(tp) || fromPartitionOffsets.contains(tp) + }.toSeq + logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) + + val fromOffsets = fromPartitionOffsets ++ newPartitionInitialOffsets + val untilOffsets = untilPartitionOffsets + val ranges = topicPartitions.map { tp => + val fromOffset = fromOffsets(tp) + val untilOffset = untilOffsets(tp) + if (untilOffset < fromOffset) { + reportDataLoss(s"Partition $tp's offset was changed from " + + s"$fromOffset to $untilOffset, some data may have been missed") + } + KafkaOffsetRange(tp, fromOffset, untilOffset, preferredLoc = None) + } + rangeCalculator.getRanges(ranges, getSortedExecutorList) + } + private def partitionsAssignedToConsumer( body: ju.Set[TopicPartition] => Map[TopicPartition, Long], fetchingEarliestOffset: Boolean = false) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index 61479c992039b..413a0c4de8bea 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -66,36 +66,12 @@ private[kafka010] class KafkaRelation( driverGroupIdPrefix = s"$uniqueGroupId-driver") // Leverage the KafkaReader to obtain the relevant partition offsets - val (fromPartitionOffsets, untilPartitionOffsets) = { - try { - (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets, isStartingOffsets = true), - kafkaOffsetReader.fetchPartitionOffsets(endingOffsets, isStartingOffsets = false)) - } finally { - kafkaOffsetReader.close() - } + val offsetRanges: Seq[KafkaOffsetRange] = try { + kafkaOffsetReader.getOffsetRangesFromUnresolvedOffsets(startingOffsets, endingOffsets) + } finally { + kafkaOffsetReader.close() } - // Obtain topicPartitions in both from and until partition offset, ignoring - // topic partitions that were added and/or deleted between the two above calls. - if (fromPartitionOffsets.keySet != untilPartitionOffsets.keySet) { - implicit val topicOrdering: Ordering[TopicPartition] = Ordering.by(t => t.topic()) - val fromTopics = fromPartitionOffsets.keySet.toList.sorted.mkString(",") - val untilTopics = untilPartitionOffsets.keySet.toList.sorted.mkString(",") - throw new IllegalStateException("different topic partitions " + - s"for starting offsets topics[${fromTopics}] and " + - s"ending offsets topics[${untilTopics}]") - } - - // Calculate offset ranges - val offsetRanges = untilPartitionOffsets.keySet.map { tp => - val fromOffset = fromPartitionOffsets.getOrElse(tp, - // This should not happen since topicPartitions contains all partitions not in - // fromPartitionOffsets - throw new IllegalStateException(s"$tp doesn't have a from offset")) - val untilOffset = untilPartitionOffsets(tp) - KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, None) - }.toArray - logInfo("GetBatch generating RDD of offset range: " + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index e1392b6215d3a..f0b3bf178dc39 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -210,66 +210,10 @@ private[kafka010] class KafkaSource( initialPartitionOffsets } - // Find the new partitions, and get their earliest offsets - val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) - val newPartitionOffsets = kafkaReader.fetchEarliestOffsets(newPartitions.toSeq) - if (newPartitionOffsets.keySet != newPartitions) { - // We cannot get from offsets for some partitions. It means they got deleted. - val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) - reportDataLoss( - s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed") - } - logInfo(s"Partitions added: $newPartitionOffsets") - newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) => - reportDataLoss( - s"Added partition $p starts from $o instead of 0. Some data may have been missed") - } - - val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) - if (deletedPartitions.nonEmpty) { - val message = if (kafkaReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { - s"$deletedPartitions are gone. ${CUSTOM_GROUP_ID_ERROR_MESSAGE}" - } else { - s"$deletedPartitions are gone. Some data may have been missed." - } - reportDataLoss(message) - } - - // Use the until partitions to calculate offset ranges to ignore partitions that have - // been deleted - val topicPartitions = untilPartitionOffsets.keySet.filter { tp => - // Ignore partitions that we don't know the from offsets. - newPartitionOffsets.contains(tp) || fromPartitionOffsets.contains(tp) - }.toSeq - logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) - - val sortedExecutors = getSortedExecutorList(sc) - val numExecutors = sortedExecutors.length - logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) - - // Calculate offset ranges - val offsetRanges = topicPartitions.map { tp => - val fromOffset = fromPartitionOffsets.getOrElse(tp, newPartitionOffsets.getOrElse(tp, { - // This should not happen since newPartitionOffsets contains all partitions not in - // fromPartitionOffsets - throw new IllegalStateException(s"$tp doesn't have a from offset") - })) - val untilOffset = untilPartitionOffsets(tp) - val preferredLoc = if (numExecutors > 0) { - // This allows cached KafkaConsumers in the executors to be re-used to read the same - // partition in every batch. - Some(sortedExecutors(Math.floorMod(tp.hashCode, numExecutors))) - } else None - KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) - }.filter { range => - if (range.untilOffset < range.fromOffset) { - reportDataLoss(s"Partition ${range.topicPartition}'s offset was changed from " + - s"${range.fromOffset} to ${range.untilOffset}, some data may have been missed") - false - } else { - true - } - }.toArray + val offsetRanges = kafkaReader.getOffsetRangesFromResolvedOffsets( + fromPartitionOffsets, + untilPartitionOffsets, + reportDataLoss) // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. val rdd = if (includeHeaders) { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala index f1f3871fc7db6..5475864500941 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.common.TopicPartition import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD @@ -28,21 +27,9 @@ import org.apache.spark.sql.kafka010.consumer.KafkaDataConsumer import org.apache.spark.storage.StorageLevel import org.apache.spark.util.NextIterator -/** Offset range that one partition of the KafkaSourceRDD has to read */ -private[kafka010] case class KafkaSourceRDDOffsetRange( - topicPartition: TopicPartition, - fromOffset: Long, - untilOffset: Long, - preferredLoc: Option[String]) { - def topic: String = topicPartition.topic - def partition: Int = topicPartition.partition - def size: Long = untilOffset - fromOffset -} - - /** Partition of the KafkaSourceRDD */ private[kafka010] case class KafkaSourceRDDPartition( - index: Int, offsetRange: KafkaSourceRDDOffsetRange) extends Partition + index: Int, offsetRange: KafkaOffsetRange) extends Partition /** @@ -58,7 +45,7 @@ private[kafka010] case class KafkaSourceRDDPartition( private[kafka010] class KafkaSourceRDD( sc: SparkContext, executorKafkaParams: ju.Map[String, Object], - offsetRanges: Seq[KafkaSourceRDDOffsetRange], + offsetRanges: Seq[KafkaOffsetRange], pollTimeoutMs: Long, failOnDataLoss: Boolean) extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { @@ -130,7 +117,7 @@ private[kafka010] class KafkaSourceRDD( } } - private def resolveRange(consumer: KafkaDataConsumer, range: KafkaSourceRDDOffsetRange) = { + private def resolveRange(consumer: KafkaDataConsumer, range: KafkaOffsetRange) = { if (range.fromOffset < 0 || range.untilOffset < 0) { // Late bind the offset range val availableOffsetRange = consumer.getAvailableOffsetRange() @@ -148,7 +135,7 @@ private[kafka010] class KafkaSourceRDD( } else { range.untilOffset } - KafkaSourceRDDOffsetRange(range.topicPartition, + KafkaOffsetRange(range.topicPartition, fromOffset, untilOffset, range.preferredLoc) } else { range diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 3ee59e57a6edf..468b21c1c7c8c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -1063,6 +1063,35 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { test("SPARK-27494: read kafka record containing null key/values.") { testNullableKeyValue(Trigger.ProcessingTime(100)) } + + test("SPARK-30656: minPartitions") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) + + val ds = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .option("minPartitions", "6") + .load() + .select($"value".as[String]) + val q = ds.writeStream.foreachBatch { (batch: Dataset[String], _: Long) => + val partitions = batch.rdd.collectPartitions() + assert(partitions.length >= 6) + assert(partitions.flatten.toSet === (0 to 20).map(_.toString).toSet): Unit + }.start() + try { + q.processAllAvailable() + } finally { + q.stop() + } + } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala index 2374a817422fa..5d010cd553521 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala @@ -34,31 +34,16 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { } } - test("with no minPartition: N TopicPartitions to N offset ranges") { val calc = KafkaOffsetRangeCalculator(CaseInsensitiveStringMap.empty()) assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1), - untilOffsets = Map(tp1 -> 2)) == - Seq(KafkaOffsetRange(tp1, 1, 2, None))) - - assert( - calc.getRanges( - fromOffsets = Map(tp1 -> 1), - untilOffsets = Map(tp1 -> 2, tp2 -> 1), Seq.empty) == + Seq(KafkaOffsetRange(tp1, 1, 2))) == Seq(KafkaOffsetRange(tp1, 1, 2, None))) assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1, tp2 -> 1), - untilOffsets = Map(tp1 -> 2)) == - Seq(KafkaOffsetRange(tp1, 1, 2, None))) - - assert( - calc.getRanges( - fromOffsets = Map(tp1 -> 1, tp2 -> 1), - untilOffsets = Map(tp1 -> 2), + Seq(KafkaOffsetRange(tp1, 1, 2)), executorLocations = Seq("location")) == Seq(KafkaOffsetRange(tp1, 1, 2, Some("location")))) } @@ -67,16 +52,19 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { val calc = KafkaOffsetRangeCalculator(CaseInsensitiveStringMap.empty()) assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1, tp2 -> 1), - untilOffsets = Map(tp1 -> 2, tp2 -> 1)) == + Seq( + KafkaOffsetRange(tp1, 1, 2), + KafkaOffsetRange(tp2, 1, 1))) === Seq(KafkaOffsetRange(tp1, 1, 2, None))) } testWithMinPartitions("N TopicPartitions to N offset ranges", 3) { calc => assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1, tp2 -> 1, tp3 -> 1), - untilOffsets = Map(tp1 -> 2, tp2 -> 2, tp3 -> 2)) == + Seq( + KafkaOffsetRange(tp1, 1, 2), + KafkaOffsetRange(tp2, 1, 2), + KafkaOffsetRange(tp3, 1, 2))) === Seq( KafkaOffsetRange(tp1, 1, 2, None), KafkaOffsetRange(tp2, 1, 2, None), @@ -86,18 +74,16 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { testWithMinPartitions("1 TopicPartition to N offset ranges", 4) { calc => assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1), - untilOffsets = Map(tp1 -> 5)) == - Seq( - KafkaOffsetRange(tp1, 1, 2, None), - KafkaOffsetRange(tp1, 2, 3, None), - KafkaOffsetRange(tp1, 3, 4, None), - KafkaOffsetRange(tp1, 4, 5, None))) + Seq(KafkaOffsetRange(tp1, 1, 5))) == + Seq( + KafkaOffsetRange(tp1, 1, 2, None), + KafkaOffsetRange(tp1, 2, 3, None), + KafkaOffsetRange(tp1, 3, 4, None), + KafkaOffsetRange(tp1, 4, 5, None))) assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1), - untilOffsets = Map(tp1 -> 5), + Seq(KafkaOffsetRange(tp1, 1, 5)), executorLocations = Seq("location")) == Seq( KafkaOffsetRange(tp1, 1, 2, None), @@ -109,8 +95,9 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { testWithMinPartitions("N skewed TopicPartitions to M offset ranges", 3) { calc => assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1, tp2 -> 1), - untilOffsets = Map(tp1 -> 5, tp2 -> 21)) == + Seq( + KafkaOffsetRange(tp1, 1, 5), + KafkaOffsetRange(tp2, 1, 21))) === Seq( KafkaOffsetRange(tp1, 1, 5, None), KafkaOffsetRange(tp2, 1, 7, None), @@ -118,11 +105,51 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { KafkaOffsetRange(tp2, 14, 21, None))) } + testWithMinPartitions("SPARK-30656: ignore empty ranges and split the rest", 4) { calc => + assert( + calc.getRanges( + Seq( + KafkaOffsetRange(tp1, 1, 1), + KafkaOffsetRange(tp2, 1, 21))) === + Seq( + KafkaOffsetRange(tp2, 1, 6, None), + KafkaOffsetRange(tp2, 6, 11, None), + KafkaOffsetRange(tp2, 11, 16, None), + KafkaOffsetRange(tp2, 16, 21, None))) + } + + testWithMinPartitions( + "SPARK-30656: N very skewed TopicPartitions to M offset ranges", + 3) { calc => + assert( + calc.getRanges( + Seq( + KafkaOffsetRange(tp1, 1, 2), + KafkaOffsetRange(tp2, 1, 1001))) === + Seq( + KafkaOffsetRange(tp1, 1, 2, None), + KafkaOffsetRange(tp2, 1, 334, None), + KafkaOffsetRange(tp2, 334, 667, None), + KafkaOffsetRange(tp2, 667, 1001, None))) + } + + testWithMinPartitions( + "SPARK-30656: minPartitions less than the length of topic partitions", + 1) { calc => + assert( + calc.getRanges( + Seq( + KafkaOffsetRange(tp1, 1, 5), + KafkaOffsetRange(tp2, 1, 21))) === + Seq( + KafkaOffsetRange(tp1, 1, 5, None), + KafkaOffsetRange(tp2, 1, 21, None))) + } + testWithMinPartitions("range inexact multiple of minPartitions", 3) { calc => assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1), - untilOffsets = Map(tp1 -> 11)) == + Seq(KafkaOffsetRange(tp1, 1, 11))) == Seq( KafkaOffsetRange(tp1, 1, 4, None), KafkaOffsetRange(tp1, 4, 7, None), @@ -132,8 +159,10 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { testWithMinPartitions("empty ranges ignored", 3) { calc => assert( calc.getRanges( - fromOffsets = Map(tp1 -> 1, tp2 -> 1, tp3 -> 1), - untilOffsets = Map(tp1 -> 5, tp2 -> 21, tp3 -> 1)) == + Seq( + KafkaOffsetRange(tp1, 1, 5), + KafkaOffsetRange(tp2, 1, 21), + KafkaOffsetRange(tp3, 1, 1))) == Seq( KafkaOffsetRange(tp1, 1, 5, None), KafkaOffsetRange(tp2, 1, 7, None), @@ -144,8 +173,10 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { testWithMinPartitions("SPARK-28489: never drop offsets", 6) { calc => assert( calc.getRanges( - fromOffsets = Map(tp1 -> 0, tp2 -> 0, tp3 -> 0), - untilOffsets = Map(tp1 -> 10, tp2 -> 10, tp3 -> 1)) == + Seq( + KafkaOffsetRange(tp1, 0, 10), + KafkaOffsetRange(tp2, 0, 10), + KafkaOffsetRange(tp3, 0, 1))) == Seq( KafkaOffsetRange(tp1, 0, 3, None), KafkaOffsetRange(tp1, 3, 6, None), diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderSuite.scala new file mode 100644 index 0000000000000..ad22a56d9157f --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderSuite.scala @@ -0,0 +1,139 @@ +/* + * 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.kafka010 + +import java.util.UUID +import java.util.concurrent.atomic.AtomicInteger + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.kafka010.KafkaOffsetRangeLimit.{EARLIEST, LATEST} +import org.apache.spark.sql.test.SharedSparkSession + +class KafkaOffsetReaderSuite extends QueryTest with SharedSparkSession with KafkaTest { + + protected var testUtils: KafkaTestUtils = _ + + private val topicId = new AtomicInteger(0) + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils + testUtils.setup() + } + + override def afterAll(): Unit = { + try { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + } + } finally { + super.afterAll() + } + } + + private def createKafkaReader(topic: String, minPartitions: Option[Int]): KafkaOffsetReader = { + new KafkaOffsetReader( + SubscribeStrategy(Seq(topic)), + org.apache.spark.sql.kafka010.KafkaSourceProvider.kafkaParamsForDriver( + Map( + "bootstrap.servers" -> + testUtils.brokerAddress + )), + CaseInsensitiveMap( + minPartitions.map(m => Map("minPartitions" -> m.toString)).getOrElse(Map.empty)), + UUID.randomUUID().toString + ) + } + + test("SPARK-30656: getOffsetRangesFromUnresolvedOffsets - using specific offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, (0 until 10).map(_.toString).toArray, Some(0)) + val tp = new TopicPartition(topic, 0) + val reader = createKafkaReader(topic, minPartitions = Some(3)) + val startingOffsets = SpecificOffsetRangeLimit(Map(tp -> 1)) + val endingOffsets = SpecificOffsetRangeLimit(Map(tp -> 4)) + val offsetRanges = reader.getOffsetRangesFromUnresolvedOffsets(startingOffsets, endingOffsets) + assert(offsetRanges === Seq( + KafkaOffsetRange(tp, 1, 2, None), + KafkaOffsetRange(tp, 2, 3, None), + KafkaOffsetRange(tp, 3, 4, None))) + } + + test("SPARK-30656: getOffsetRangesFromUnresolvedOffsets - using special offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, (0 until 4).map(_.toString).toArray, Some(0)) + val tp = new TopicPartition(topic, 0) + val reader = createKafkaReader(topic, minPartitions = Some(3)) + val startingOffsets = EarliestOffsetRangeLimit + val endingOffsets = LatestOffsetRangeLimit + val offsetRanges = reader.getOffsetRangesFromUnresolvedOffsets(startingOffsets, endingOffsets) + assert(offsetRanges === Seq( + KafkaOffsetRange(tp, EARLIEST, 1, None), + KafkaOffsetRange(tp, 1, 2, None), + KafkaOffsetRange(tp, 2, LATEST, None))) + } + + test("SPARK-30656: getOffsetRangesFromUnresolvedOffsets - multiple topic partitions") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 2) + testUtils.sendMessages(topic, (0 until 100).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (0 until 4).map(_.toString).toArray, Some(1)) + val tp1 = new TopicPartition(topic, 0) + val tp2 = new TopicPartition(topic, 1) + val reader = createKafkaReader(topic, minPartitions = Some(3)) + + val startingOffsets = SpecificOffsetRangeLimit(Map(tp1 -> EARLIEST, tp2 -> EARLIEST)) + val endingOffsets = SpecificOffsetRangeLimit(Map(tp1 -> LATEST, tp2 -> 3)) + val offsetRanges = reader.getOffsetRangesFromUnresolvedOffsets(startingOffsets, endingOffsets) + assert(offsetRanges === Seq( + KafkaOffsetRange(tp2, EARLIEST, 3, None), + KafkaOffsetRange(tp1, EARLIEST, 33, None), + KafkaOffsetRange(tp1, 33, 66, None), + KafkaOffsetRange(tp1, 66, LATEST, None))) + } + + test("SPARK-30656: getOffsetRangesFromResolvedOffsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 2) + testUtils.sendMessages(topic, (0 until 100).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (0 until 4).map(_.toString).toArray, Some(1)) + val tp1 = new TopicPartition(topic, 0) + val tp2 = new TopicPartition(topic, 1) + val reader = createKafkaReader(topic, minPartitions = Some(3)) + + val fromPartitionOffsets = Map(tp1 -> 0L, tp2 -> 0L) + val untilPartitionOffsets = Map(tp1 -> 100L, tp2 -> 3L) + val offsetRanges = reader.getOffsetRangesFromResolvedOffsets( + fromPartitionOffsets, + untilPartitionOffsets, + _ => {}) + assert(offsetRanges === Seq( + KafkaOffsetRange(tp1, 0, 33, None), + KafkaOffsetRange(tp1, 33, 66, None), + KafkaOffsetRange(tp1, 66, 100, None), + KafkaOffsetRange(tp2, 0, 3, None))) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 2c022c11950bc..32d056140a0d7 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -597,6 +597,28 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession checkAnswer(df, (1 to 15).map(_.toString).toDF) } } + + test("SPARK-30656: minPartitions") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) + + // Implicit offset values, should default to earliest and latest + val df = createDF(topic, Map("minPartitions" -> "6")) + val rdd = df.rdd + val partitions = rdd.collectPartitions() + assert(partitions.length >= 6) + assert(partitions.flatMap(_.map(_.getString(0))).toSet === (0 to 20).map(_.toString).toSet) + + // Because of late binding, reused `rdd` and `df` should see the new data. + testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray) + assert(rdd.collectPartitions().flatMap(_.map(_.getString(0))).toSet + === (0 to 30).map(_.toString).toSet) + assert(df.rdd.collectPartitions().flatMap(_.map(_.getString(0))).toSet + === (0 to 30).map(_.toString).toSet) + } } class KafkaRelationSuiteV1 extends KafkaRelationSuiteBase { From cbb714f67e96dfd9678c60586af48febfba878ca Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 30 Jan 2020 20:21:43 -0800 Subject: [PATCH 1257/1387] [SPARK-29438][SS] Use partition ID of StateStoreAwareZipPartitionsRDD for determining partition ID of state store in stream-stream join ### What changes were proposed in this pull request? Credit to uncleGen for discovering the problem and providing simple reproducer as UT. New UT in this patch is borrowed from #26156 and I'm retaining a commit from #26156 (except unnecessary part on this path) to properly give a credit. This patch fixes the issue that partition ID could be mis-assigned when the query contains UNION and stream-stream join is placed on the right side. We assume the range of partition IDs as `(0 ~ number of shuffle partitions - 1)` for stateful operators, but when we use stream-stream join on the right side of UNION, the range of partition ID of task goes to `(number of partitions in left side, number of partitions in left side + number of shuffle partitions - 1)`, which `number of partitions in left side` can be changed in some cases (new UT points out the one of the cases). The root reason of bug is that stream-stream join picks the partition ID from TaskContext, which wouldn't be same as partition ID from source if union is being used. Hopefully we can pick the right partition ID from source in StateStoreAwareZipPartitionsRDD - this patch leverages that partition ID. ### Why are the changes needed? This patch will fix the broken of assumption of partition range on stateful operator, as well as fix the issue reported in JIRA issue SPARK-29438. ### Does this PR introduce any user-facing change? Yes, if their query is using UNION and stream-stream join is placed on the right side. They may encounter the problem to read state from checkpoint and may need to discard checkpoint to continue. ### How was this patch tested? Added UT which fails on current master branch, and passes with this patch. Closes #26162 from HeartSaVioR/SPARK-29438. Lead-authored-by: Jungtaek Lim (HeartSaVioR) Co-authored-by: uncleGen Signed-off-by: Tathagata Das --- .../StreamingSymmetricHashJoinExec.scala | 11 ++-- .../StreamingSymmetricHashJoinHelper.scala | 36 ++++++++--- .../sql/execution/streaming/memory.scala | 32 ++++++++-- .../state/SymmetricHashJoinStateManager.scala | 5 +- .../SymmetricHashJoinStateManagerSuite.scala | 2 +- .../spark/sql/streaming/StreamTest.scala | 6 +- .../sql/streaming/StreamingJoinSuite.scala | 61 ++++++++++++++++++- 7 files changed, 129 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index 3c45f22815db8..198e17db419a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -215,6 +215,7 @@ case class StreamingSymmetricHashJoinExec( } private def processPartitions( + partitionId: Int, leftInputIter: Iterator[InternalRow], rightInputIter: Iterator[InternalRow]): Iterator[InternalRow] = { if (stateInfo.isEmpty) { @@ -238,10 +239,10 @@ case class StreamingSymmetricHashJoinExec( Predicate.create(condition.bothSides.getOrElse(Literal(true)), inputSchema).eval _ val leftSideJoiner = new OneSideHashJoiner( LeftSide, left.output, leftKeys, leftInputIter, - condition.leftSideOnly, postJoinFilter, stateWatermarkPredicates.left) + condition.leftSideOnly, postJoinFilter, stateWatermarkPredicates.left, partitionId) val rightSideJoiner = new OneSideHashJoiner( RightSide, right.output, rightKeys, rightInputIter, - condition.rightSideOnly, postJoinFilter, stateWatermarkPredicates.right) + condition.rightSideOnly, postJoinFilter, stateWatermarkPredicates.right, partitionId) // Join one side input using the other side's buffered/state rows. Here is how it is done. // @@ -406,6 +407,7 @@ case class StreamingSymmetricHashJoinExec( * @param stateWatermarkPredicate The state watermark predicate. See * [[StreamingSymmetricHashJoinExec]] for further description of * state watermarks. + * @param partitionId A partition ID of source RDD. */ private class OneSideHashJoiner( joinSide: JoinSide, @@ -414,7 +416,8 @@ case class StreamingSymmetricHashJoinExec( inputIter: Iterator[InternalRow], preJoinFilterExpr: Option[Expression], postJoinFilter: (InternalRow) => Boolean, - stateWatermarkPredicate: Option[JoinStateWatermarkPredicate]) { + stateWatermarkPredicate: Option[JoinStateWatermarkPredicate], + partitionId: Int) { // Filter the joined rows based on the given condition. val preJoinFilter = @@ -422,7 +425,7 @@ case class StreamingSymmetricHashJoinExec( private val joinStateManager = new SymmetricHashJoinStateManager( joinSide, inputAttributes, joinKeys, stateInfo, storeConf, hadoopConfBcast.value.value, - stateFormatVersion) + partitionId, stateFormatVersion) private[this] val keyGenerator = UnsafeProjection.create(joinKeys, inputAttributes) private[this] val stateKeyWatermarkPredicateFunc = stateWatermarkPredicate match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala index 2d4c3c10e6445..cdd3a854c9a90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.execution.streaming import scala.reflect.ClassTag -import scala.util.control.NonFatal -import org.apache.spark.{Partition, SparkContext} +import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.internal.Logging -import org.apache.spark.rdd.{RDD, ZippedPartitionsRDD2} +import org.apache.spark.rdd.{RDD, ZippedPartitionsBaseRDD, ZippedPartitionsPartition, ZippedPartitionsRDD2} import org.apache.spark.sql.catalyst.analysis.StreamingJoinHelper import org.apache.spark.sql.catalyst.expressions.{Add, And, Attribute, AttributeReference, AttributeSet, BoundReference, Cast, CheckOverflow, Expression, ExpressionSet, GreaterThan, GreaterThanOrEqual, LessThan, LessThanOrEqual, Literal, Multiply, NamedExpression, PreciseTimestampConversion, PredicateHelper, Subtract, TimeAdd, TimeSub, UnaryMinus} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark._ @@ -203,17 +202,18 @@ object StreamingSymmetricHashJoinHelper extends Logging { /** * A custom RDD that allows partitions to be "zipped" together, while ensuring the tasks' * preferred location is based on which executors have the required join state stores already - * loaded. This is class is a modified version of [[ZippedPartitionsRDD2]]. + * loaded. This class is a variant of [[ZippedPartitionsRDD2]] which only changes signature + * of `f`. */ class StateStoreAwareZipPartitionsRDD[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, - f: (Iterator[A], Iterator[B]) => Iterator[V], - rdd1: RDD[A], - rdd2: RDD[B], + var f: (Int, Iterator[A], Iterator[B]) => Iterator[V], + var rdd1: RDD[A], + var rdd2: RDD[B], stateInfo: StatefulOperatorStateInfo, stateStoreNames: Seq[String], @transient private val storeCoordinator: Option[StateStoreCoordinatorRef]) - extends ZippedPartitionsRDD2[A, B, V](sc, f, rdd1, rdd2) { + extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2)) { /** * Set the preferred location of each partition using the executor that has the related @@ -225,6 +225,24 @@ object StreamingSymmetricHashJoinHelper extends Logging { storeCoordinator.flatMap(_.getLocation(stateStoreProviderId)) }.distinct } + + override def compute(s: Partition, context: TaskContext): Iterator[V] = { + val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions + if (partitions(0).index != partitions(1).index) { + throw new IllegalStateException(s"Partition ID should be same in both side: " + + s"left ${partitions(0).index} , right ${partitions(1).index}") + } + + val partitionId = partitions(0).index + f(partitionId, rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context)) + } + + override def clearDependencies(): Unit = { + super.clearDependencies() + rdd1 = null + rdd2 = null + f = null + } } implicit class StateStoreAwareZipPartitionsHelper[T: ClassTag](dataRDD: RDD[T]) { @@ -239,7 +257,7 @@ object StreamingSymmetricHashJoinHelper extends Logging { stateInfo: StatefulOperatorStateInfo, storeNames: Seq[String], storeCoordinator: StateStoreCoordinatorRef - )(f: (Iterator[T], Iterator[U]) => Iterator[V]): RDD[V] = { + )(f: (Int, Iterator[T], Iterator[U]) => Iterator[V]): RDD[V] = { new StateStoreAwareZipPartitionsRDD( dataRDD.sparkContext, f, dataRDD, dataRDD2, stateInfo, storeNames, Some(storeCoordinator)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 911a526428cf4..34ed0daca6ee0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -44,6 +44,9 @@ object MemoryStream { def apply[A : Encoder](implicit sqlContext: SQLContext): MemoryStream[A] = new MemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext) + + def apply[A : Encoder](numPartitions: Int)(implicit sqlContext: SQLContext): MemoryStream[A] = + new MemoryStream[A](memoryStreamId.getAndIncrement(), sqlContext, Some(numPartitions)) } /** @@ -136,9 +139,14 @@ class MemoryStreamScanBuilder(stream: MemoryStreamBase[_]) extends ScanBuilder w * A [[Source]] that produces value stored in memory as they are added by the user. This [[Source]] * is intended for use in unit tests as it can only replay data when the object is still * available. + * + * If numPartitions is provided, the rows will be redistributed to the given number of partitions. */ -case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) - extends MemoryStreamBase[A](sqlContext) with MicroBatchStream with Logging { +case class MemoryStream[A : Encoder]( + id: Int, + sqlContext: SQLContext, + numPartitions: Option[Int] = None) + extends MemoryStreamBase[A](sqlContext) with MicroBatchStream with Logging { protected val output = logicalPlan.output @@ -206,9 +214,23 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) logDebug(generateDebugString(newBlocks.flatten, startOrdinal, endOrdinal)) - newBlocks.map { block => - new MemoryStreamInputPartition(block) - }.toArray + numPartitions match { + case Some(numParts) => + // When the number of partition is provided, we redistribute the rows into + // the given number of partition, via round-robin manner. + val inputRows = newBlocks.flatten.toArray + (0 until numParts).map { newPartIdx => + val records = inputRows.zipWithIndex.filter { case (_, idx) => + idx % numParts == newPartIdx + }.map(_._1) + new MemoryStreamInputPartition(records) + }.toArray + + case _ => + newBlocks.map { block => + new MemoryStreamInputPartition(block) + }.toArray + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala index c10713734dcc6..1a0a43c083879 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala @@ -44,6 +44,7 @@ import org.apache.spark.util.NextIterator * @param stateInfo Information about how to retrieve the correct version of state * @param storeConf Configuration for the state store. * @param hadoopConf Hadoop configuration for reading state data from storage + * @param partitionId A partition ID of source RDD. * @param stateFormatVersion The version of format for state. * * Internally, the key -> multiple values is stored in two [[StateStore]]s. @@ -72,8 +73,8 @@ class SymmetricHashJoinStateManager( stateInfo: Option[StatefulOperatorStateInfo], storeConf: StateStoreConf, hadoopConf: Configuration, + partitionId: Int, stateFormatVersion: Int) extends Logging { - import SymmetricHashJoinStateManager._ /* @@ -356,7 +357,7 @@ class SymmetricHashJoinStateManager( /** Get the StateStore with the given schema */ protected def getStateStore(keySchema: StructType, valueSchema: StructType): StateStore = { val storeProviderId = StateStoreProviderId( - stateInfo.get, TaskContext.getPartitionId(), getStateStoreName(joinSide, stateStoreType)) + stateInfo.get, partitionId, getStateStoreName(joinSide, stateStoreType)) val store = StateStore.get( storeProviderId, keySchema, valueSchema, None, stateInfo.get.storeVersion, storeConf, hadoopConf) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala index b40f8df22b586..ce1eabeb932fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala @@ -170,7 +170,7 @@ class SymmetricHashJoinStateManagerSuite extends StreamTest with BeforeAndAfter val stateInfo = StatefulOperatorStateInfo(file.getAbsolutePath, UUID.randomUUID, 0, 0, 5) val manager = new SymmetricHashJoinStateManager( LeftSide, inputValueAttribs, joinKeyExprs, Some(stateInfo), storeConf, new Configuration, - stateFormatVersion) + partitionId = 0, stateFormatVersion) try { f(manager) } finally { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index e54a5370d351d..6d5ad873eedea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -112,7 +112,11 @@ trait StreamTest extends QueryTest with SharedSparkSession with TimeLimits with object MultiAddData { def apply[A] (source1: MemoryStream[A], data1: A*)(source2: MemoryStream[A], data2: A*): StreamAction = { - val actions = Seq(AddDataMemory(source1, data1), AddDataMemory(source2, data2)) + apply((source1, data1), (source2, data2)) + } + + def apply[A](inputs: (MemoryStream[A], Seq[A])*): StreamAction = { + val actions = inputs.map { case (source, data) => AddDataMemory(source, data) } StreamProgressLockedActions(actions, desc = actions.mkString("[ ", " | ", " ]")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index ae6a4ecb7a6da..3f218c9cb7fd9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -25,6 +25,7 @@ import scala.util.Random import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfter +import org.apache.spark.SparkContext import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.StreamingJoinHelper @@ -376,7 +377,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val rdd1 = spark.sparkContext.makeRDD(1 to 10, numPartitions) val rdd2 = spark.sparkContext.makeRDD((1 to 10).map(_.toString), numPartitions) val rdd = rdd1.stateStoreAwareZipPartitions(rdd2, stateInfo, storeNames, coordinatorRef) { - (left, right) => left.zip(right) + (_, left, right) => left.zip(right) } require(rdd.partitions.length === numPartitions) for (partIndex <- 0 until numPartitions) { @@ -933,5 +934,61 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with assert(e.getMessage.toLowerCase(Locale.ROOT) .contains("the query is using stream-stream outer join with state format version 1")) } -} + test("SPARK-29438: ensure UNION doesn't lead stream-stream join to use shifted partition IDs") { + def constructUnionDf(desiredPartitionsForInput1: Int) + : (MemoryStream[Int], MemoryStream[Int], MemoryStream[Int], DataFrame) = { + val input1 = MemoryStream[Int](desiredPartitionsForInput1) + val df1 = input1.toDF + .select( + 'value as "key", + 'value as "leftValue", + 'value as "rightValue") + val (input2, df2) = setupStream("left", 2) + val (input3, df3) = setupStream("right", 3) + + val joined = df2 + .join(df3, + df2("key") === df3("key") && df2("leftTime") === df3("rightTime"), + "inner") + .select(df2("key"), 'leftValue, 'rightValue) + + (input1, input2, input3, df1.union(joined)) + } + + withTempDir { tempDir => + val (input1, input2, input3, unionDf) = constructUnionDf(2) + + testStream(unionDf)( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + MultiAddData( + (input1, Seq(11, 12, 13)), + (input2, Seq(11, 12, 13, 14, 15)), + (input3, Seq(13, 14, 15, 16, 17))), + CheckNewAnswer(Row(11, 11, 11), Row(12, 12, 12), Row(13, 13, 13), Row(13, 26, 39), + Row(14, 28, 42), Row(15, 30, 45)), + StopStream + ) + + // We're restoring the query with different number of partitions in left side of UNION, + // which leads right side of union to have mismatched partition IDs if it relies on + // TaskContext.partitionId(). SPARK-29438 fixes this issue to not rely on it. + + val (newInput1, newInput2, newInput3, newUnionDf) = constructUnionDf(3) + + newInput1.addData(11, 12, 13) + newInput2.addData(11, 12, 13, 14, 15) + newInput3.addData(13, 14, 15, 16, 17) + + testStream(newUnionDf)( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + MultiAddData( + (newInput1, Seq(21, 22, 23)), + (newInput2, Seq(21, 22, 23, 24, 25)), + (newInput3, Seq(23, 24, 25, 26, 27))), + CheckNewAnswer(Row(21, 21, 21), Row(22, 22, 22), Row(23, 23, 23), Row(23, 46, 69), + Row(24, 48, 72), Row(25, 50, 75)) + ) + } + } +} From ca3a64bffb8e709a3c0e3849cdc4cec6a29e29bf Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 30 Jan 2020 21:04:08 -0800 Subject: [PATCH 1258/1387] [SPARK-30481][CORE][FOLLOWUP] Execute log compaction only when merge application listing is successful ### What changes were proposed in this pull request? This PR fixes a couple of minor issues on SPARK-30481: * SHS runs "compaction" regardless of the result of "merge application listing". If "merge application listing" fails, most likely the application log will have some issue and "compaction" won't work properly then. We can just skip trying compaction when "merge application listing" fails. * When "compaction" throws exception we don't handle it. It's expected to swallow exception, but we don't even log the exception for now. It should be logged properly. ### Why are the changes needed? Described in above section. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing UTs. Closes #27408 from HeartSaVioR/SPARK-30481-FOLLOWUP-MINOR-FIXES. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../spark/deploy/history/FsHistoryProvider.scala | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 2c023b00e6549..99d3eceb1121a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -656,6 +656,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) scanTime: Long, enableOptimizations: Boolean): Unit = { val rootPath = reader.rootPath + var succeeded = false try { val lastEvaluatedForCompaction: Option[Long] = try { listing.read(classOf[LogInfo], rootPath.toString).lastEvaluatedForCompaction @@ -668,6 +669,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) if (conf.get(CLEANER_ENABLED)) { checkAndCleanLog(rootPath.toString) } + + succeeded = true } catch { case e: InterruptedException => throw e @@ -684,8 +687,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) endProcessing(rootPath) pendingReplayTasksCount.decrementAndGet() - // triggering another task for compaction task - submitLogProcessTask(rootPath) { () => compact(reader) } + // triggering another task for compaction task only if it succeeds + if (succeeded) { + submitLogProcessTask(rootPath) { () => compact(reader) } + } } } @@ -834,6 +839,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) case None => // This is not applied to single event log file. } + } catch { + case e: InterruptedException => + throw e + case e: AccessControlException => + logWarning(s"Insufficient permission while compacting log for $rootPath", e) + case e: Exception => + logError(s"Exception while compacting log for $rootPath", e) } finally { endProcessing(rootPath) } From 9f42be25eba462cca8148ce636d6d3d20123d8fb Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 31 Jan 2020 13:37:43 +0800 Subject: [PATCH 1259/1387] [SPARK-29665][SQL] refine the TableProvider interface ### What changes were proposed in this pull request? Instead of having several overloads of `getTable` method in `TableProvider`, it's better to have 2 methods explicitly: `inferSchema` and `inferPartitioning`. With a single `getTable` method that takes everything: schema, partitioning and properties. This PR also adds a `supportsExternalMetadata` method in `TableProvider`, to indicate if the source support external table metadata. If this flag is false: 1. spark.read.schema... is disallowed and fails 2. when we support creating v2 tables in session catalog, spark only keeps table properties in the catalog. ### Why are the changes needed? API improvement. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests Closes #26868 from cloud-fan/provider2. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/kafka010/KafkaSourceProvider.scala | 3 +- .../sql/connector/catalog/TableProvider.java | 57 ++++++++++++++----- .../catalog/CatalogV2Implicits.scala | 5 +- .../connector/SimpleTableProvider.scala | 50 ++++++++++++++++ .../apache/spark/sql/DataFrameReader.scala | 6 +- .../apache/spark/sql/DataFrameWriter.scala | 23 ++++++-- .../datasources/noop/NoopDataSource.scala | 5 +- .../datasources/v2/DataSourceV2Utils.scala | 28 ++++++++- .../datasources/v2/FileDataSourceV2.scala | 42 +++++++++++++- .../execution/datasources/v2/FileTable.scala | 2 +- .../sql/execution/streaming/console.scala | 5 +- .../sql/execution/streaming/memory.scala | 5 +- .../sources/RateStreamProvider.scala | 5 +- .../sources/TextSocketSourceProvider.scala | 5 +- .../sql/streaming/DataStreamReader.scala | 10 ++-- .../sql/streaming/DataStreamWriter.scala | 10 +++- .../connector/JavaAdvancedDataSourceV2.java | 6 +- .../connector/JavaColumnarDataSourceV2.java | 4 +- .../JavaPartitionAwareDataSource.java | 4 +- .../JavaReportStatisticsDataSource.java | 4 +- .../JavaSchemaRequiredDataSource.java | 21 +++++-- .../sql/connector/JavaSimpleBatchTable.java | 3 +- .../sql/connector/JavaSimpleDataSourceV2.java | 4 +- .../sql/connector/JavaSimpleScanBuilder.java | 3 +- ...SourceV2DataFrameSessionCatalogSuite.scala | 8 +-- .../sql/connector/DataSourceV2SQLSuite.scala | 3 +- .../sql/connector/DataSourceV2Suite.scala | 48 ++++++++++++---- .../connector/SimpleWritableDataSource.scala | 5 +- .../SupportsCatalogOptionsSuite.scala | 4 +- .../connector/TableCapabilityCheckSuite.scala | 8 +-- .../sql/connector/V1ReadFallbackSuite.scala | 3 +- .../sql/connector/V1WriteFallbackSuite.scala | 5 +- .../command/PlanResolutionSuite.scala | 4 +- .../sources/TextSocketStreamSuite.scala | 3 +- .../sources/StreamingDataSourceV2Suite.scala | 19 ++++--- .../streaming/util/BlockOnStopSource.scala | 5 +- 36 files changed, 308 insertions(+), 117 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SimpleTableProvider.scala diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 4ffa70f9f31dd..a5e5d01152db8 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBat import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.{Sink, Source} +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType @@ -51,7 +52,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with StreamSinkProvider with RelationProvider with CreatableRelationProvider - with TableProvider + with SimpleTableProvider with Logging { import KafkaSourceProvider._ diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java index e9fd87d0e2d40..732c5352a15ac 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableProvider.java @@ -17,7 +17,10 @@ package org.apache.spark.sql.connector.catalog; +import java.util.Map; + import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -36,26 +39,50 @@ public interface TableProvider { /** - * Return a {@link Table} instance to do read/write with user-specified options. + * Infer the schema of the table identified by the given options. + * + * @param options an immutable case-insensitive string-to-string map that can identify a table, + * e.g. file path, Kafka topic name, etc. + */ + StructType inferSchema(CaseInsensitiveStringMap options); + + /** + * Infer the partitioning of the table identified by the given options. + *

    + * By default this method returns empty partitioning, please override it if this source support + * partitioning. + * + * @param options an immutable case-insensitive string-to-string map that can identify a table, + * e.g. file path, Kafka topic name, etc. + */ + default Transform[] inferPartitioning(CaseInsensitiveStringMap options) { + return new Transform[0]; + } + + /** + * Return a {@link Table} instance with the specified table schema, partitioning and properties + * to do read/write. The returned table should report the same schema and partitioning with the + * specified ones, or Spark may fail the operation. * - * @param options the user-specified options that can identify a table, e.g. file path, Kafka - * topic name, etc. It's an immutable case-insensitive string-to-string map. + * @param schema The specified table schema. + * @param partitioning The specified table partitioning. + * @param properties The specified table properties. It's case preserving (contains exactly what + * users specified) and implementations are free to use it case sensitively or + * insensitively. It should be able to identify a table, e.g. file path, Kafka + * topic name, etc. */ - Table getTable(CaseInsensitiveStringMap options); + Table getTable(StructType schema, Transform[] partitioning, Map properties); /** - * Return a {@link Table} instance to do read/write with user-specified schema and options. + * Returns true if the source has the ability of accepting external table metadata when getting + * tables. The external table metadata includes user-specified schema from + * `DataFrameReader`/`DataStreamReader` and schema/partitioning stored in Spark catalog. *

    - * By default this method throws {@link UnsupportedOperationException}, implementations should - * override this method to handle user-specified schema. - *

    - * @param options the user-specified options that can identify a table, e.g. file path, Kafka - * topic name, etc. It's an immutable case-insensitive string-to-string map. - * @param schema the user-specified schema. - * @throws UnsupportedOperationException + * By default this method returns false, which means the schema and partitioning passed to + * `getTable` are from the infer methods. Please override it if this source has expensive + * schema/partitioning inference and wants external table metadata to avoid inference. */ - default Table getTable(CaseInsensitiveStringMap options, StructType schema) { - throw new UnsupportedOperationException( - this.getClass().getSimpleName() + " source does not support user-specified schema"); + default boolean supportsExternalMetadata() { + return false; } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala index 16aec23521f9f..3478af8783af6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala @@ -21,7 +21,6 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.connector.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} -import org.apache.spark.sql.types.StructType /** * Conversion helpers for working with v2 [[CatalogPlugin]]. @@ -29,9 +28,9 @@ import org.apache.spark.sql.types.StructType private[sql] object CatalogV2Implicits { import LogicalExpressions._ - implicit class PartitionTypeHelper(partitionType: StructType) { + implicit class PartitionTypeHelper(colNames: Seq[String]) { def asTransforms: Array[Transform] = { - partitionType.names.map(col => identity(reference(Seq(col)))).toArray + colNames.map(col => identity(reference(Seq(col)))).toArray } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SimpleTableProvider.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SimpleTableProvider.scala new file mode 100644 index 0000000000000..7bfe1df1117ac --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/connector/SimpleTableProvider.scala @@ -0,0 +1,50 @@ +/* + * 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.internal.connector + +import java.util + +import org.apache.spark.sql.connector.catalog.{Table, TableProvider} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +// A simple version of `TableProvider` which doesn't support specified table schema/partitioning +// and treats table properties case-insensitively. This is private and only used in builtin sources. +trait SimpleTableProvider extends TableProvider { + + def getTable(options: CaseInsensitiveStringMap): Table + + private[this] var loadedTable: Table = _ + private def getOrLoadTable(options: CaseInsensitiveStringMap): Table = { + if (loadedTable == null) loadedTable = getTable(options) + loadedTable + } + + override def inferSchema(options: CaseInsensitiveStringMap): StructType = { + getOrLoadTable(options).schema() + } + + override def getTable( + schema: StructType, + partitioning: Array[Transform], + properties: util.Map[String, String]): Table = { + assert(partitioning.isEmpty) + getOrLoadTable(new CaseInsensitiveStringMap(properties)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b5d7bbca9064d..6cce7203127f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -220,10 +220,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { (catalog.loadTable(ident), Some(catalog), Some(ident)) case _ => // TODO: Non-catalog paths for DSV2 are currently not well defined. - userSpecifiedSchema match { - case Some(schema) => (provider.getTable(dsOptions, schema), None, None) - case _ => (provider.getTable(dsOptions), None, None) - } + val tbl = DataSourceV2Utils.getTableFromProvider(provider, dsOptions, userSpecifiedSchema) + (tbl, None, None) } import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index c041d14c8b8df..4557219abeb15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -257,6 +257,21 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val options = sessionOptions ++ extraOptions val dsOptions = new CaseInsensitiveStringMap(options.asJava) + def getTable: Table = { + // For file source, it's expensive to infer schema/partition at each write. Here we pass + // the schema of input query and the user-specified partitioning to `getTable`. If the + // query schema is not compatible with the existing data, the write can still success but + // following reads would fail. + if (provider.isInstanceOf[FileDataSourceV2]) { + provider.getTable( + df.schema.asNullable, + partitioningAsV2.toArray, + dsOptions.asCaseSensitiveMap()) + } else { + DataSourceV2Utils.getTableFromProvider(provider, dsOptions, userSpecifiedSchema = None) + } + } + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ val catalogManager = df.sparkSession.sessionState.catalogManager mode match { @@ -268,8 +283,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { supportsExtract, catalogManager, dsOptions) (catalog.loadTable(ident), Some(catalog), Some(ident)) - case tableProvider: TableProvider => - val t = tableProvider.getTable(dsOptions) + case _: TableProvider => + val t = getTable if (t.supports(BATCH_WRITE)) { (t, None, None) } else { @@ -314,8 +329,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { extraOptions.toMap, ignoreIfExists = createMode == SaveMode.Ignore) } - case tableProvider: TableProvider => - if (tableProvider.getTable(dsOptions).supports(BATCH_WRITE)) { + case _: TableProvider => + if (getTable.supports(BATCH_WRITE)) { throw new AnalysisException(s"TableProvider implementation $source cannot be " + s"written with $createMode mode, please use Append or Overwrite " + "modes instead.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala index b6149ce7290b7..4fad0a2484cde 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -22,9 +22,10 @@ import java.util import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, LogicalWriteInfo, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite} +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -33,7 +34,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap * This is no-op datasource. It does not do anything besides consuming its input. * This can be useful for benchmarking or to cache data without any additional overhead. */ -class NoopDataSource extends TableProvider with DataSourceRegister { +class NoopDataSource extends SimpleTableProvider with DataSourceRegister { override def shortName(): String = "noop" override def getTable(options: CaseInsensitiveStringMap): Table = NoopTable } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala index 52294ae2cb851..b50b8295463eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala @@ -20,8 +20,10 @@ package org.apache.spark.sql.execution.datasources.v2 import java.util.regex.Pattern import org.apache.spark.internal.Logging -import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, TableProvider} +import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, Table, TableProvider} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap private[sql] object DataSourceV2Utils extends Logging { @@ -57,4 +59,28 @@ private[sql] object DataSourceV2Utils extends Logging { case _ => Map.empty } } + + def getTableFromProvider( + provider: TableProvider, + options: CaseInsensitiveStringMap, + userSpecifiedSchema: Option[StructType]): Table = { + userSpecifiedSchema match { + case Some(schema) => + if (provider.supportsExternalMetadata()) { + provider.getTable( + schema, + provider.inferPartitioning(options), + options.asCaseSensitiveMap()) + } else { + throw new UnsupportedOperationException( + s"${provider.getClass.getSimpleName} source does not support user-specified schema.") + } + + case None => + provider.getTable( + provider.inferSchema(options), + provider.inferPartitioning(options), + options.asCaseSensitiveMap()) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index e0091293d1669..30a964d7e643f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -16,13 +16,17 @@ */ package org.apache.spark.sql.execution.datasources.v2 +import java.util + import com.fasterxml.jackson.databind.ObjectMapper import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.catalog.TableProvider +import org.apache.spark.sql.connector.catalog.{Table, TableProvider} +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils @@ -59,4 +63,40 @@ trait FileDataSourceV2 extends TableProvider with DataSourceRegister { val fs = hdfsPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toString } + + // TODO: To reduce code diff of SPARK-29665, we create stub implementations for file source v2, so + // that we don't need to touch all the file source v2 classes. We should remove the stub + // implementation and directly implement the TableProvider APIs. + protected def getTable(options: CaseInsensitiveStringMap): Table + protected def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + throw new UnsupportedOperationException("user-specified schema") + } + + override def supportsExternalMetadata(): Boolean = true + + private var t: Table = null + + override def inferSchema(options: CaseInsensitiveStringMap): StructType = { + if (t == null) t = getTable(options) + t.schema() + } + + // TODO: implement a light-weight partition inference which only looks at the path of one leaf + // file and return partition column names. For now the partition inference happens in + // `getTable`, because we don't know the user-specified schema here. + override def inferPartitioning(options: CaseInsensitiveStringMap): Array[Transform] = { + Array.empty + } + + override def getTable( + schema: StructType, + partitioning: Array[Transform], + properties: util.Map[String, String]): Table = { + // If the table is already loaded during schema inference, return it directly. + if (t != null) { + t + } else { + getTable(new CaseInsensitiveStringMap(properties), schema) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala index 5329e09916bd6..59dc3ae56bf25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -102,7 +102,7 @@ abstract class FileTable( StructType(fields) } - override def partitioning: Array[Transform] = fileIndex.partitionSchema.asTransforms + override def partitioning: Array[Transform] = fileIndex.partitionSchema.names.toSeq.asTransforms override def properties: util.Map[String, String] = options.asCaseSensitiveMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala index 63e40891942ae..e471e6c601d16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -22,10 +22,11 @@ import java.util import scala.collection.JavaConverters._ import org.apache.spark.sql._ -import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.write.{LogicalWriteInfo, SupportsTruncate, WriteBuilder} import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.streaming.sources.ConsoleWrite +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -35,7 +36,7 @@ case class ConsoleRelation(override val sqlContext: SQLContext, data: DataFrame) override def schema: StructType = data.schema } -class ConsoleSinkProvider extends TableProvider +class ConsoleSinkProvider extends SimpleTableProvider with DataSourceRegister with CreatableRelationProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 34ed0daca6ee0..ea39c549bd072 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -31,10 +31,11 @@ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -97,7 +98,7 @@ abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends Spa // This class is used to indicate the memory stream data source. We don't actually use it, as // memory stream is for test only and we never look it up by name. -object MemoryStreamTableProvider extends TableProvider { +object MemoryStreamTableProvider extends SimpleTableProvider { override def getTable(options: CaseInsensitiveStringMap): Table = { throw new IllegalStateException("MemoryStreamTableProvider should not be used.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala index 3f7b0377f1eab..a093bf54b2107 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala @@ -23,10 +23,11 @@ import scala.collection.JavaConverters._ import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousStream +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -45,7 +46,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap * generated rows. The source will try its best to reach `rowsPerSecond`, but the query may * be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed. */ -class RateStreamProvider extends TableProvider with DataSourceRegister { +class RateStreamProvider extends SimpleTableProvider with DataSourceRegister { import RateStreamProvider._ override def getTable(options: CaseInsensitiveStringMap): Table = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala index fae3cb765c0c9..a4dcb2049eb87 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala @@ -26,15 +26,16 @@ import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging import org.apache.spark.sql._ -import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousStream +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class TextSocketSourceProvider extends TableProvider with DataSourceRegister with Logging { +class TextSocketSourceProvider extends SimpleTableProvider with DataSourceRegister with Logging { private def checkParameters(params: CaseInsensitiveStringMap): Unit = { logWarning("The socket source should not be used for production applications! " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index cfe6192e7d5c5..0eb4776988d9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsRead, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2} import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.types.StructType @@ -173,15 +173,13 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo case _ => None } ds match { - case provider: TableProvider => + // file source v2 does not support streaming yet. + case provider: TableProvider if !provider.isInstanceOf[FileDataSourceV2] => val sessionOptions = DataSourceV2Utils.extractSessionConfigs( source = provider, conf = sparkSession.sessionState.conf) val options = sessionOptions ++ extraOptions val dsOptions = new CaseInsensitiveStringMap(options.asJava) - val table = userSpecifiedSchema match { - case Some(schema) => provider.getTable(dsOptions, schema) - case _ => provider.getTable(dsOptions) - } + val table = DataSourceV2Utils.getTableFromProvider(provider, dsOptions, userSpecifiedSchema) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ table match { case _: SupportsRead if table.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 62a1add8b6d94..1c21a30dd5bd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.connector.catalog.{SupportsWrite, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDataSourceV2} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -308,7 +308,9 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } else { val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",") - val useV1Source = disabledSources.contains(cls.getCanonicalName) + val useV1Source = disabledSources.contains(cls.getCanonicalName) || + // file source v2 does not support streaming yet. + classOf[FileDataSourceV2].isAssignableFrom(cls) val sink = if (classOf[TableProvider].isAssignableFrom(cls) && !useV1Source) { val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] @@ -316,8 +318,10 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { source = provider, conf = df.sparkSession.sessionState.conf) val options = sessionOptions ++ extraOptions val dsOptions = new CaseInsensitiveStringMap(options.asJava) + val table = DataSourceV2Utils.getTableFromProvider( + provider, dsOptions, userSpecifiedSchema = None) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ - provider.getTable(dsOptions) match { + table match { case table: SupportsWrite if table.supports(STREAMING_WRITE) => table case _ => createV1Sink() diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2.java index 9386ab51d64f0..1a55d198361ee 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaAdvancedDataSourceV2.java @@ -22,15 +22,15 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.connector.TestingV2Source; import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableProvider; import org.apache.spark.sql.connector.read.*; import org.apache.spark.sql.sources.Filter; import org.apache.spark.sql.sources.GreaterThan; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class JavaAdvancedDataSourceV2 implements TableProvider { +public class JavaAdvancedDataSourceV2 implements TestingV2Source { @Override public Table getTable(CaseInsensitiveStringMap options) { @@ -45,7 +45,7 @@ public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { static class AdvancedScanBuilder implements ScanBuilder, Scan, SupportsPushDownFilters, SupportsPushDownRequiredColumns { - private StructType requiredSchema = new StructType().add("i", "int").add("j", "int"); + private StructType requiredSchema = TestingV2Source.schema(); private Filter[] filters = new Filter[0]; @Override diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaColumnarDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaColumnarDataSourceV2.java index 76da45e182b3c..2f10c84c999f9 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaColumnarDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaColumnarDataSourceV2.java @@ -20,8 +20,8 @@ import java.io.IOException; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.TestingV2Source; import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableProvider; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.PartitionReader; import org.apache.spark.sql.connector.read.PartitionReaderFactory; @@ -33,7 +33,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch; -public class JavaColumnarDataSourceV2 implements TableProvider { +public class JavaColumnarDataSourceV2 implements TestingV2Source { class MyScanBuilder extends JavaSimpleScanBuilder { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaPartitionAwareDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaPartitionAwareDataSource.java index fbbc457b2945d..9c1db7a379602 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaPartitionAwareDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaPartitionAwareDataSource.java @@ -22,17 +22,17 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.connector.TestingV2Source; import org.apache.spark.sql.connector.expressions.Expressions; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableProvider; import org.apache.spark.sql.connector.read.*; import org.apache.spark.sql.connector.read.partitioning.ClusteredDistribution; import org.apache.spark.sql.connector.read.partitioning.Distribution; import org.apache.spark.sql.connector.read.partitioning.Partitioning; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class JavaPartitionAwareDataSource implements TableProvider { +public class JavaPartitionAwareDataSource implements TestingV2Source { class MyScanBuilder extends JavaSimpleScanBuilder implements SupportsReportPartitioning { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java index 49438fe668d56..9a787c3d2d92c 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaReportStatisticsDataSource.java @@ -19,15 +19,15 @@ import java.util.OptionalLong; +import org.apache.spark.sql.connector.TestingV2Source; import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableProvider; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsReportStatistics; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class JavaReportStatisticsDataSource implements TableProvider { +public class JavaReportStatisticsDataSource implements TestingV2Source { class MyScanBuilder extends JavaSimpleScanBuilder implements SupportsReportStatistics { @Override public Statistics estimateStatistics() { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSchemaRequiredDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSchemaRequiredDataSource.java index 2181887ae54e2..5f73567ade025 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSchemaRequiredDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSchemaRequiredDataSource.java @@ -17,8 +17,11 @@ package test.org.apache.spark.sql.connector; +import java.util.Map; + import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; @@ -46,7 +49,18 @@ public InputPartition[] planInputPartitions() { } @Override - public Table getTable(CaseInsensitiveStringMap options, StructType schema) { + public boolean supportsExternalMetadata() { + return true; + } + + @Override + public StructType inferSchema(CaseInsensitiveStringMap options) { + throw new IllegalArgumentException("requires a user-supplied schema"); + } + + @Override + public Table getTable( + StructType schema, Transform[] partitioning, Map properties) { return new JavaSimpleBatchTable() { @Override @@ -60,9 +74,4 @@ public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { } }; } - - @Override - public Table getTable(CaseInsensitiveStringMap options) { - throw new IllegalArgumentException("requires a user-supplied schema"); - } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java index 97b00477e1764..71cf97b56fe54 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleBatchTable.java @@ -21,6 +21,7 @@ import java.util.HashSet; import java.util.Set; +import org.apache.spark.sql.connector.TestingV2Source; import org.apache.spark.sql.connector.catalog.SupportsRead; import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCapability; @@ -34,7 +35,7 @@ abstract class JavaSimpleBatchTable implements Table, SupportsRead { @Override public StructType schema() { - return new StructType().add("i", "int").add("j", "int"); + return TestingV2Source.schema(); } @Override diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleDataSourceV2.java index 8b6d71b986ff7..8852249d8a01f 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleDataSourceV2.java @@ -17,13 +17,13 @@ package test.org.apache.spark.sql.connector; +import org.apache.spark.sql.connector.TestingV2Source; import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableProvider; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class JavaSimpleDataSourceV2 implements TableProvider { +public class JavaSimpleDataSourceV2 implements TestingV2Source { class MyScanBuilder extends JavaSimpleScanBuilder { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java index 7cbba00420928..bdd9dd3ea0ce0 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaSimpleScanBuilder.java @@ -17,6 +17,7 @@ package test.org.apache.spark.sql.connector; +import org.apache.spark.sql.connector.TestingV2Source; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.PartitionReaderFactory; import org.apache.spark.sql.connector.read.Scan; @@ -37,7 +38,7 @@ public Batch toBatch() { @Override public StructType readSchema() { - return new StructType().add("i", "int").add("j", "int"); + return TestingV2Source.schema(); } @Override diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index 08627e681f9e6..4c67888cbdc48 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -92,12 +92,6 @@ class DataSourceV2DataFrameSessionCatalogSuite } } -class InMemoryTableProvider extends TableProvider { - override def getTable(options: CaseInsensitiveStringMap): Table = { - throw new UnsupportedOperationException("D'oh!") - } -} - class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable] { override def newTable( name: String, @@ -140,7 +134,7 @@ private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2Sessio spark.sessionState.catalogManager.catalog(name) } - protected val v2Format: String = classOf[InMemoryTableProvider].getName + protected val v2Format: String = classOf[FakeV2Provider].getName protected val catalogClassName: String = classOf[InMemoryTableSessionCatalog].getName diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 04e5a8dfd78ba..2c8349a0e6a75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAM import org.apache.spark.sql.connector.catalog.CatalogV2Util.withDefaultOwnership import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.SimpleScanSource import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -2230,7 +2231,7 @@ class DataSourceV2SQLSuite /** Used as a V2 DataSource for V2SessionCatalog DDL */ -class FakeV2Provider extends TableProvider { +class FakeV2Provider extends SimpleTableProvider { override def getTable(options: CaseInsensitiveStringMap): Table = { throw new UnsupportedOperationException("Unnecessary for DDL tests") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 85ff86ef3fc5b..2d8761f872da7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.connector.read.partitioning.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -418,7 +419,7 @@ object SimpleReaderFactory extends PartitionReaderFactory { abstract class SimpleBatchTable extends Table with SupportsRead { - override def schema(): StructType = new StructType().add("i", "int").add("j", "int") + override def schema(): StructType = TestingV2Source.schema override def name(): String = this.getClass.toString @@ -432,12 +433,31 @@ abstract class SimpleScanBuilder extends ScanBuilder override def toBatch: Batch = this - override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int") + override def readSchema(): StructType = TestingV2Source.schema override def createReaderFactory(): PartitionReaderFactory = SimpleReaderFactory } -class SimpleSinglePartitionSource extends TableProvider { +trait TestingV2Source extends TableProvider { + override def inferSchema(options: CaseInsensitiveStringMap): StructType = { + TestingV2Source.schema + } + + override def getTable( + schema: StructType, + partitioning: Array[Transform], + properties: util.Map[String, String]): Table = { + getTable(new CaseInsensitiveStringMap(properties)) + } + + def getTable(options: CaseInsensitiveStringMap): Table +} + +object TestingV2Source { + val schema = new StructType().add("i", "int").add("j", "int") +} + +class SimpleSinglePartitionSource extends TestingV2Source { class MyScanBuilder extends SimpleScanBuilder { override def planInputPartitions(): Array[InputPartition] = { @@ -452,9 +472,10 @@ class SimpleSinglePartitionSource extends TableProvider { } } + // This class is used by pyspark tests. If this class is modified/moved, make sure pyspark // tests still pass. -class SimpleDataSourceV2 extends TableProvider { +class SimpleDataSourceV2 extends TestingV2Source { class MyScanBuilder extends SimpleScanBuilder { override def planInputPartitions(): Array[InputPartition] = { @@ -469,7 +490,7 @@ class SimpleDataSourceV2 extends TableProvider { } } -class AdvancedDataSourceV2 extends TableProvider { +class AdvancedDataSourceV2 extends TestingV2Source { override def getTable(options: CaseInsensitiveStringMap): Table = new SimpleBatchTable { override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { @@ -481,7 +502,7 @@ class AdvancedDataSourceV2 extends TableProvider { class AdvancedScanBuilder extends ScanBuilder with Scan with SupportsPushDownFilters with SupportsPushDownRequiredColumns { - var requiredSchema = new StructType().add("i", "int").add("j", "int") + var requiredSchema = TestingV2Source.schema var filters = Array.empty[Filter] override def pruneColumns(requiredSchema: StructType): Unit = { @@ -567,11 +588,16 @@ class SchemaRequiredDataSource extends TableProvider { override def readSchema(): StructType = schema } - override def getTable(options: CaseInsensitiveStringMap): Table = { + override def supportsExternalMetadata(): Boolean = true + + override def inferSchema(options: CaseInsensitiveStringMap): StructType = { throw new IllegalArgumentException("requires a user-supplied schema") } - override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + override def getTable( + schema: StructType, + partitioning: Array[Transform], + properties: util.Map[String, String]): Table = { val userGivenSchema = schema new SimpleBatchTable { override def schema(): StructType = userGivenSchema @@ -583,7 +609,7 @@ class SchemaRequiredDataSource extends TableProvider { } } -class ColumnarDataSourceV2 extends TableProvider { +class ColumnarDataSourceV2 extends TestingV2Source { class MyScanBuilder extends SimpleScanBuilder { @@ -648,7 +674,7 @@ object ColumnarReaderFactory extends PartitionReaderFactory { } } -class PartitionAwareDataSource extends TableProvider { +class PartitionAwareDataSource extends TestingV2Source { class MyScanBuilder extends SimpleScanBuilder with SupportsReportPartitioning{ @@ -716,7 +742,7 @@ class SimpleWriteOnlyDataSource extends SimpleWritableDataSource { } } -class ReportStatisticsDataSource extends TableProvider { +class ReportStatisticsDataSource extends SimpleWritableDataSource { class MyScanBuilder extends SimpleScanBuilder with SupportsReportStatistics { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala index 0070076459f19..f9306ba28e7f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SimpleWritableDataSource.scala @@ -27,10 +27,11 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{SessionConfigSupport, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory, ScanBuilder} import org.apache.spark.sql.connector.write._ +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -40,7 +41,7 @@ import org.apache.spark.util.SerializableConfiguration * Each task writes data to `target/_temporary/uniqueId/$jobId-$partitionId-$attemptNumber`. * Each job moves files from `target/_temporary/uniqueId/` to `target`. */ -class SimpleWritableDataSource extends TableProvider with SessionConfigSupport { +class SimpleWritableDataSource extends SimpleTableProvider with SessionConfigSupport { private val tableSchema = new StructType().add("i", "long").add("j", "long") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala index cec48bb368aef..7bff955b18360 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.connector -import java.util - import scala.language.implicitConversions import scala.util.Try @@ -275,7 +273,7 @@ class SupportsCatalogOptionsSuite extends QueryTest with SharedSparkSession with } class CatalogSupportingInMemoryTableProvider - extends InMemoryTableProvider + extends FakeV2Provider with SupportsCatalogOptions { override def extractIdentifier(options: CaseInsensitiveStringMap): Identifier = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index 5196ca65276e4..23e4c293cbc28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -40,7 +40,7 @@ class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { private val emptyMap = CaseInsensitiveStringMap.empty private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = { StreamingRelationV2( - TestTableProvider, + new FakeV2Provider, "fake", table, CaseInsensitiveStringMap.empty(), @@ -211,12 +211,6 @@ private case object TestRelation extends LeafNode with NamedRelation { override def output: Seq[AttributeReference] = TableCapabilityCheckSuite.schema.toAttributes } -private object TestTableProvider extends TableProvider { - override def getTable(options: CaseInsensitiveStringMap): Table = { - throw new UnsupportedOperationException - } -} - private case class CapabilityTable(_capabilities: TableCapability*) extends Table { override def name(): String = "capability_test_table" override def schema(): StructType = TableCapabilityCheckSuite.schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala index 8e2c63417b377..74f2ca14234d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.connector.catalog.{Identifier, SupportsRead, Table, import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns, V1Scan} import org.apache.spark.sql.execution.RowDataSourceScanExec +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.{BaseRelation, Filter, GreaterThan, TableScan} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType @@ -120,7 +121,7 @@ object V1ReadFallbackCatalog { val schema = new StructType().add("i", "int").add("j", "int") } -class V1ReadFallbackTableProvider extends TableProvider { +class V1ReadFallbackTableProvider extends SimpleTableProvider { override def getTable(options: CaseInsensitiveStringMap): Table = { new TableWithV1ReadFallback("v1-read-fallback") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index a36e8dbdec506..10ed2048dbf61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -25,10 +25,11 @@ import scala.collection.mutable import org.scalatest.BeforeAndAfter import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession, SQLContext} -import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder} import org.apache.spark.sql.execution.datasources.DataSourceUtils +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.StructType @@ -173,7 +174,7 @@ private object InMemoryV1Provider { } class InMemoryV1Provider - extends TableProvider + extends SimpleTableProvider with DataSourceRegister with CreatableRelationProvider { override def getTable(options: CaseInsensitiveStringMap): Table = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 70b9b7ec12ea2..30b7e93a4beb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, StringLiteral} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, ShowTableProperties, SubqueryAlias, UpdateAction, UpdateTable} -import org.apache.spark.sql.connector.InMemoryTableProvider +import org.apache.spark.sql.connector.FakeV2Provider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCapability, TableCatalog, TableChange, V1Table} import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation @@ -41,7 +41,7 @@ import org.apache.spark.sql.types.{CharType, DoubleType, HIVE_TYPE_STRING, Integ class PlanResolutionSuite extends AnalysisTest { import CatalystSqlParser._ - private val v2Format = classOf[InMemoryTableProvider].getName + private val v2Format = classOf[FakeV2Provider].getName private val table: Table = { val t = mock(classOf[Table]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala index 0f80e2d431bb1..5c66fc52592b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala @@ -194,13 +194,12 @@ class TextSocketStreamSuite extends StreamTest with SharedSparkSession { } test("user-specified schema given") { - val provider = new TextSocketSourceProvider val userSpecifiedSchema = StructType( StructField("name", StringType) :: StructField("area", StringType) :: Nil) val params = Map("host" -> "localhost", "port" -> "1234") val exception = intercept[UnsupportedOperationException] { - provider.getTable(new CaseInsensitiveStringMap(params.asJava), userSpecifiedSchema) + spark.readStream.schema(userSpecifiedSchema).format("socket").options(params).load() } assert(exception.getMessage.contains( "TextSocketSourceProvider source does not support user-specified schema")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index 13bc811a8fe9b..05cf324f8d490 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactor import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RateStreamOffset, Sink, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery, StreamTest, Trigger} import org.apache.spark.sql.types.StructType @@ -93,7 +94,7 @@ trait FakeStreamingWriteTable extends Table with SupportsWrite { class FakeReadMicroBatchOnly extends DataSourceRegister - with TableProvider + with SimpleTableProvider with SessionConfigSupport { override def shortName(): String = "fake-read-microbatch-only" @@ -116,7 +117,7 @@ class FakeReadMicroBatchOnly class FakeReadContinuousOnly extends DataSourceRegister - with TableProvider + with SimpleTableProvider with SessionConfigSupport { override def shortName(): String = "fake-read-continuous-only" @@ -137,7 +138,7 @@ class FakeReadContinuousOnly } } -class FakeReadBothModes extends DataSourceRegister with TableProvider { +class FakeReadBothModes extends DataSourceRegister with SimpleTableProvider { override def shortName(): String = "fake-read-microbatch-continuous" override def getTable(options: CaseInsensitiveStringMap): Table = { @@ -154,7 +155,7 @@ class FakeReadBothModes extends DataSourceRegister with TableProvider { } } -class FakeReadNeitherMode extends DataSourceRegister with TableProvider { +class FakeReadNeitherMode extends DataSourceRegister with SimpleTableProvider { override def shortName(): String = "fake-read-neither-mode" override def getTable(options: CaseInsensitiveStringMap): Table = { @@ -168,7 +169,7 @@ class FakeReadNeitherMode extends DataSourceRegister with TableProvider { class FakeWriteOnly extends DataSourceRegister - with TableProvider + with SimpleTableProvider with SessionConfigSupport { override def shortName(): String = "fake-write-microbatch-continuous" @@ -183,7 +184,7 @@ class FakeWriteOnly } } -class FakeNoWrite extends DataSourceRegister with TableProvider { +class FakeNoWrite extends DataSourceRegister with SimpleTableProvider { override def shortName(): String = "fake-write-neither-mode" override def getTable(options: CaseInsensitiveStringMap): Table = { new Table { @@ -201,7 +202,7 @@ class FakeSink extends Sink { } class FakeWriteSupportProviderV1Fallback extends DataSourceRegister - with TableProvider with StreamSinkProvider { + with SimpleTableProvider with StreamSinkProvider { override def createSink( sqlContext: SQLContext, @@ -378,10 +379,10 @@ class StreamingDataSourceV2Suite extends StreamTest { for ((read, write, trigger) <- cases) { testQuietly(s"stream with read format $read, write format $write, trigger $trigger") { val sourceTable = DataSource.lookupDataSource(read, spark.sqlContext.conf).getConstructor() - .newInstance().asInstanceOf[TableProvider].getTable(CaseInsensitiveStringMap.empty()) + .newInstance().asInstanceOf[SimpleTableProvider].getTable(CaseInsensitiveStringMap.empty()) val sinkTable = DataSource.lookupDataSource(write, spark.sqlContext.conf).getConstructor() - .newInstance().asInstanceOf[TableProvider].getTable(CaseInsensitiveStringMap.empty()) + .newInstance().asInstanceOf[SimpleTableProvider].getTable(CaseInsensitiveStringMap.empty()) import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ trigger match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala index f25758c520691..c594a8523d15e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala @@ -25,11 +25,12 @@ import scala.collection.JavaConverters._ import org.apache.zookeeper.KeeperException.UnimplementedException import org.apache.spark.sql.{DataFrame, Row, SparkSession, SQLContext} -import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability, TableProvider} +import org.apache.spark.sql.connector.catalog.{SupportsRead, Table, TableCapability} import org.apache.spark.sql.connector.catalog.TableCapability.CONTINUOUS_READ import org.apache.spark.sql.connector.read.{streaming, InputPartition, Scan, ScanBuilder} import org.apache.spark.sql.connector.read.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, PartitionOffset} import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, Source} +import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -54,7 +55,7 @@ object BlockOnStopSourceProvider { } } -class BlockOnStopSourceProvider extends StreamSourceProvider with TableProvider { +class BlockOnStopSourceProvider extends StreamSourceProvider with SimpleTableProvider { override def getTable(options: CaseInsensitiveStringMap): Table = { new BlockOnStopSourceTable(BlockOnStopSourceProvider._latch) } From 5f3ec6250f5df1d48f711450ae480f1f9d5d25ce Mon Sep 17 00:00:00 2001 From: sandeep katta Date: Fri, 31 Jan 2020 14:01:32 +0800 Subject: [PATCH 1260/1387] [SPARK-30362][CORE] Update InputMetrics in DataSourceRDD ### What changes were proposed in this pull request? Incase of DS v2 InputMetrics are not updated **Before Fix** ![inputMetrics](https://user-images.githubusercontent.com/35216143/71501010-c216df00-288d-11ea-8522-fdd50b13eae1.png) **After Fix** we can see that `Input Size / Records` is updated in the UI ![image](https://user-images.githubusercontent.com/35216143/71501000-b88d7700-288d-11ea-92fe-a727b2b79908.png) ### Why are the changes needed? InputMetrics like bytesread and recordread should be updated ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added UT and also verified manually Closes #27021 from sandeep-katta/dsv2inputmetrics. Authored-by: sandeep katta Signed-off-by: Wenchen Fan --- .../datasources/v2/DataSourceRDD.scala | 99 ++++++++++++++----- .../DataSourceScanExecRedactionSuite.scala | 32 ++++++ 2 files changed, 109 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala index 71560f60f531c..63403b9577237 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala @@ -17,10 +17,15 @@ package org.apache.spark.sql.execution.datasources.v2 +import scala.language.existentials + import org.apache.spark._ +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.vectorized.ColumnarBatch class DataSourceRDDPartition(val index: Int, val inputPartition: InputPartition) extends Partition with Serializable @@ -47,31 +52,16 @@ class DataSourceRDD( override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val inputPartition = castPartition(split).inputPartition - val reader: PartitionReader[_] = if (columnarReads) { - partitionReaderFactory.createColumnarReader(inputPartition) + val (iter, reader) = if (columnarReads) { + val batchReader = partitionReaderFactory.createColumnarReader(inputPartition) + val iter = new MetricsBatchIterator(new PartitionIterator[ColumnarBatch](batchReader)) + (iter, batchReader) } else { - partitionReaderFactory.createReader(inputPartition) + val rowReader = partitionReaderFactory.createReader(inputPartition) + val iter = new MetricsRowIterator(new PartitionIterator[InternalRow](rowReader)) + (iter, rowReader) } - context.addTaskCompletionListener[Unit](_ => reader.close()) - val iter = new Iterator[Any] { - private[this] var valuePrepared = false - - override def hasNext: Boolean = { - if (!valuePrepared) { - valuePrepared = reader.next() - } - valuePrepared - } - - override def next(): Any = { - if (!hasNext) { - throw new java.util.NoSuchElementException("End of stream") - } - valuePrepared = false - reader.get() - } - } // TODO: SPARK-25083 remove the type erasure hack in data source scan new InterruptibleIterator(context, iter.asInstanceOf[Iterator[InternalRow]]) } @@ -80,3 +70,68 @@ class DataSourceRDD( castPartition(split).inputPartition.preferredLocations() } } + +private class PartitionIterator[T](reader: PartitionReader[T]) extends Iterator[T] { + private[this] var valuePrepared = false + + override def hasNext: Boolean = { + if (!valuePrepared) { + valuePrepared = reader.next() + } + valuePrepared + } + + override def next(): T = { + if (!hasNext) { + throw new java.util.NoSuchElementException("End of stream") + } + valuePrepared = false + reader.get() + } +} + +private class MetricsHandler extends Logging with Serializable { + private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics + private val startingBytesRead = inputMetrics.bytesRead + private val getBytesRead = SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + + def updateMetrics(numRows: Int, force: Boolean = false): Unit = { + inputMetrics.incRecordsRead(numRows) + val shouldUpdateBytesRead = + inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0 + if (shouldUpdateBytesRead || force) { + inputMetrics.setBytesRead(startingBytesRead + getBytesRead()) + } + } +} + +private abstract class MetricsIterator[I](iter: Iterator[I]) extends Iterator[I] { + protected val metricsHandler = new MetricsHandler + + override def hasNext: Boolean = { + if (iter.hasNext) { + true + } else { + metricsHandler.updateMetrics(0, force = true) + false + } + } +} + +private class MetricsRowIterator( + iter: Iterator[InternalRow]) extends MetricsIterator[InternalRow](iter) { + override def next(): InternalRow = { + val item = iter.next + metricsHandler.updateMetrics(1) + item + } +} + +private class MetricsBatchIterator( + iter: Iterator[ColumnarBatch]) extends MetricsIterator[ColumnarBatch](iter) { + override def next(): ColumnarBatch = { + val batch: ColumnarBatch = iter.next + metricsHandler.updateMetrics(batch.numRows) + batch + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala index 073aed8206ed7..f1411b263c77b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala @@ -16,9 +16,12 @@ */ package org.apache.spark.sql.execution +import scala.collection.mutable + import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{DataFrame, QueryTest} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan @@ -167,4 +170,33 @@ class DataSourceV2ScanExecRedactionSuite extends DataSourceScanRedactionTest { } } } + + test("SPARK-30362: test input metrics for DSV2") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + Seq("json", "orc", "parquet").foreach { format => + withTempPath { path => + val dir = path.getCanonicalPath + spark.range(0, 10).write.format(format).save(dir) + val df = spark.read.format(format).load(dir) + val bytesReads = new mutable.ArrayBuffer[Long]() + val recordsRead = new mutable.ArrayBuffer[Long]() + val bytesReadListener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + recordsRead += taskEnd.taskMetrics.inputMetrics.recordsRead + } + } + sparkContext.addSparkListener(bytesReadListener) + try { + df.collect() + sparkContext.listenerBus.waitUntilEmpty() + assert(bytesReads.sum > 0) + assert(recordsRead.sum == 10) + } finally { + sparkContext.removeSparkListener(bytesReadListener) + } + } + } + } + } } From 1cd19ad92da960f18a6673bc3ce670ce633050e5 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 30 Jan 2020 22:02:48 -0800 Subject: [PATCH 1261/1387] [SPARK-30669][SS] Introduce AdmissionControl APIs for StructuredStreaming ### What changes were proposed in this pull request? We propose to add a new interface `SupportsAdmissionControl` and `ReadLimit`. A ReadLimit defines how much data should be read in the next micro-batch. `SupportsAdmissionControl` specifies that a source can rate limit its ingest into the system. The source can tell the system what the user specified as a read limit, and the system can enforce this limit within each micro-batch or impose its own limit if the Trigger is Trigger.Once() for example. We then use this interface in FileStreamSource, KafkaSource, and KafkaMicroBatchStream. ### Why are the changes needed? Sources currently have no information around execution semantics such as whether the stream is being executed in Trigger.Once() mode. This interface will pass this information into the sources as part of planning. With a trigger like Trigger.Once(), the semantics are to process all the data available to the datasource in a single micro-batch. However, this semantic can be broken when data source options such as `maxOffsetsPerTrigger` (in the Kafka source) rate limit the amount of data read for that micro-batch without this interface. ### Does this PR introduce any user-facing change? DataSource developers can extend this interface for their streaming sources to add admission control into their system and correctly support Trigger.Once(). ### How was this patch tested? Existing tests, as this API is mostly internal Closes #27380 from brkyvz/rateLimit. Lead-authored-by: Burak Yavuz Co-authored-by: Burak Yavuz Signed-off-by: Burak Yavuz --- .../sql/kafka010/KafkaMicroBatchStream.scala | 25 ++++++--- .../spark/sql/kafka010/KafkaSource.scala | 29 +++++++--- .../kafka010/KafkaMicroBatchSourceSuite.scala | 22 ++++++++ .../read/streaming/ReadAllAvailable.java | 41 ++++++++++++++ .../connector/read/streaming/ReadLimit.java} | 25 ++++++--- .../read/streaming/ReadMaxFiles.java | 55 ++++++++++++++++++ .../connector/read/streaming/ReadMaxRows.java | 55 ++++++++++++++++++ .../streaming/SupportsAdmissionControl.java | 56 +++++++++++++++++++ .../streaming/FileStreamSource.scala | 25 +++++++-- .../streaming/MicroBatchExecution.scala | 49 +++++++++++----- .../execution/streaming/StreamExecution.scala | 6 +- .../continuous/ContinuousExecution.scala | 4 +- .../sql/streaming/FileStreamSourceSuite.scala | 56 +++++++++++++++++++ 13 files changed, 398 insertions(+), 50 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadAllAvailable.java rename sql/{core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala => catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadLimit.java} (53%) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxFiles.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxRows.java create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsAdmissionControl.java diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index 844c963619855..6599e7e0fe707 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -27,8 +27,7 @@ import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} -import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} -import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchStream +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset, ReadAllAvailable, ReadLimit, ReadMaxRows, SupportsAdmissionControl} import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.UninterruptibleThread @@ -55,7 +54,7 @@ private[kafka010] class KafkaMicroBatchStream( options: CaseInsensitiveStringMap, metadataPath: String, startingOffsets: KafkaOffsetRangeLimit, - failOnDataLoss: Boolean) extends RateControlMicroBatchStream with Logging { + failOnDataLoss: Boolean) extends SupportsAdmissionControl with MicroBatchStream with Logging { private[kafka010] val pollTimeoutMs = options.getLong( KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, @@ -77,13 +76,23 @@ private[kafka010] class KafkaMicroBatchStream( KafkaSourceOffset(getOrCreateInitialPartitionOffsets()) } - override def latestOffset(start: Offset): Offset = { + override def getDefaultReadLimit: ReadLimit = { + maxOffsetsPerTrigger.map(ReadLimit.maxRows).getOrElse(super.getDefaultReadLimit) + } + + override def latestOffset(): Offset = { + throw new UnsupportedOperationException( + "latestOffset(Offset, ReadLimit) should be called instead of this method") + } + + override def latestOffset(start: Offset, readLimit: ReadLimit): Offset = { val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets)) - endPartitionOffsets = KafkaSourceOffset(maxOffsetsPerTrigger.map { maxOffsets => - rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets) - }.getOrElse { - latestPartitionOffsets + endPartitionOffsets = KafkaSourceOffset(readLimit match { + case rows: ReadMaxRows => + rateLimit(rows.maxRows(), startPartitionOffsets, latestPartitionOffsets) + case _: ReadAllAvailable => + latestPartitionOffsets }) endPartitionOffsets } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index f0b3bf178dc39..57879c7ca31cf 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -32,6 +32,8 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.read.streaming +import org.apache.spark.sql.connector.read.streaming.{ReadAllAvailable, ReadLimit, ReadMaxRows, SupportsAdmissionControl} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.kafka010.KafkaSource._ import org.apache.spark.sql.kafka010.KafkaSourceProvider._ @@ -79,7 +81,7 @@ private[kafka010] class KafkaSource( metadataPath: String, startingOffsets: KafkaOffsetRangeLimit, failOnDataLoss: Boolean) - extends Source with Logging { + extends SupportsAdmissionControl with Source with Logging { private val sc = sqlContext.sparkContext @@ -114,6 +116,10 @@ private[kafka010] class KafkaSource( }.partitionToOffsets } + override def getDefaultReadLimit: ReadLimit = { + maxOffsetsPerTrigger.map(ReadLimit.maxRows).getOrElse(super.getDefaultReadLimit) + } + private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None private val converter = new KafkaRecordToRowConverter() @@ -122,23 +128,30 @@ private[kafka010] class KafkaSource( /** Returns the maximum available offset for this source. */ override def getOffset: Option[Offset] = { + throw new UnsupportedOperationException( + "latestOffset(Offset, ReadLimit) should be called instead of this method") + } + + override def latestOffset(startOffset: streaming.Offset, limit: ReadLimit): streaming.Offset = { // Make sure initialPartitionOffsets is initialized initialPartitionOffsets val latest = kafkaReader.fetchLatestOffsets( currentPartitionOffsets.orElse(Some(initialPartitionOffsets))) - val offsets = maxOffsetsPerTrigger match { - case None => + val offsets = limit match { + case rows: ReadMaxRows => + if (currentPartitionOffsets.isEmpty) { + rateLimit(rows.maxRows(), initialPartitionOffsets, latest) + } else { + rateLimit(rows.maxRows(), currentPartitionOffsets.get, latest) + } + case _: ReadAllAvailable => latest - case Some(limit) if currentPartitionOffsets.isEmpty => - rateLimit(limit, initialPartitionOffsets, latest) - case Some(limit) => - rateLimit(limit, currentPartitionOffsets.get, latest) } currentPartitionOffsets = Some(offsets) logDebug(s"GetOffset: ${offsets.toSeq.map(_.toString).sorted}") - Some(KafkaSourceOffset(offsets)) + KafkaSourceOffset(offsets) } /** Proportionally distribute limit number of offsets among topicpartitions */ diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 468b21c1c7c8c..a4601b91af0d6 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -297,6 +297,28 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { 13, 126, 127, 128, 129, 130, 131, 132, 133, 134 ) ) + + // When Trigger.Once() is used, the read limit should be ignored + val allData = Seq(1) ++ (10 to 20) ++ (100 to 200) + withTempDir { dir => + testStream(mapped)( + StartStream(Trigger.Once(), checkpointLocation = dir.getCanonicalPath), + AssertOnQuery { q => + q.processAllAvailable() + true + }, + CheckAnswer(allData: _*), + StopStream, + + AddKafkaData(Set(topic), 1000 to 1010: _*), + StartStream(Trigger.Once(), checkpointLocation = dir.getCanonicalPath), + AssertOnQuery { q => + q.processAllAvailable() + true + }, + CheckAnswer((allData ++ 1000.to(1010)): _*) + ) + } } test("input row metrics") { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadAllAvailable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadAllAvailable.java new file mode 100644 index 0000000000000..5a946ad14b3a2 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadAllAvailable.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.read.streaming; + +import org.apache.spark.annotation.Evolving; + +/** + * Represents a {@link ReadLimit} where the {@link MicroBatchStream} must scan all the data + * available at the streaming source. This is meant to be a hard specification as being able + * to return all available data is necessary for Trigger.Once() to work correctly. + * If a source is unable to scan all available data, then it must throw an error. + * + * @see SupportsAdmissionControl#latestOffset(Offset, ReadLimit) + * @since 3.0.0 + */ +@Evolving +public final class ReadAllAvailable implements ReadLimit { + static final ReadAllAvailable INSTANCE = new ReadAllAvailable(); + + private ReadAllAvailable() {} + + @Override + public String toString() { + return "All Available"; + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadLimit.java similarity index 53% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala rename to sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadLimit.java index fb46f76682688..121ed1ad116f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadLimit.java @@ -15,17 +15,24 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming.sources +package org.apache.spark.sql.connector.read.streaming; -import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset} +import org.apache.spark.annotation.Evolving; -// A special `MicroBatchStream` that can get latestOffset with a start offset. -trait RateControlMicroBatchStream extends MicroBatchStream { +/** + * Interface representing limits on how much to read from a {@link MicroBatchStream} when it + * implements {@link SupportsAdmissionControl}. There are several child interfaces representing + * various kinds of limits. + * + * @see SupportsAdmissionControl#latestOffset(Offset, ReadLimit) + * @see ReadAllAvailable + * @see ReadMaxRows + */ +@Evolving +public interface ReadLimit { + static ReadLimit maxRows(long rows) { return new ReadMaxRows(rows); } - override def latestOffset(): Offset = { - throw new IllegalAccessException( - "latestOffset should not be called for RateControlMicroBatchReadSupport") - } + static ReadLimit maxFiles(int files) { return new ReadMaxFiles(files); } - def latestOffset(start: Offset): Offset + static ReadLimit allAvailable() { return ReadAllAvailable.INSTANCE; } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxFiles.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxFiles.java new file mode 100644 index 0000000000000..441a6c8e77a6f --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxFiles.java @@ -0,0 +1,55 @@ +/* + * 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.connector.read.streaming; + +import org.apache.spark.annotation.Evolving; + +/** + * Represents a {@link ReadLimit} where the {@link MicroBatchStream} should scan approximately the + * given maximum number of files. + * + * @see SupportsAdmissionControl#latestOffset(Offset, ReadLimit) + * @since 3.0.0 + */ +@Evolving +public class ReadMaxFiles implements ReadLimit { + private int files; + + ReadMaxFiles(int maxFiles) { + this.files = maxFiles; + } + + /** Approximate maximum rows to scan. */ + public int maxFiles() { return this.files; } + + @Override + public String toString() { + return "MaxFiles: " + maxFiles(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReadMaxFiles other = (ReadMaxFiles) o; + return other.maxFiles() == maxFiles(); + } + + @Override + public int hashCode() { return files; } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxRows.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxRows.java new file mode 100644 index 0000000000000..65a68c543ff71 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadMaxRows.java @@ -0,0 +1,55 @@ +/* + * 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.connector.read.streaming; + +import org.apache.spark.annotation.Evolving; + +/** + * Represents a {@link ReadLimit} where the {@link MicroBatchStream} should scan approximately the + * given maximum number of rows. + * + * @see SupportsAdmissionControl#latestOffset(Offset, ReadLimit) + * @since 3.0.0 + */ +@Evolving +public final class ReadMaxRows implements ReadLimit { + private long rows; + + ReadMaxRows(long rows) { + this.rows = rows; + } + + /** Approximate maximum rows to scan. */ + public long maxRows() { return this.rows; } + + @Override + public String toString() { + return "MaxRows: " + maxRows(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReadMaxRows other = (ReadMaxRows) o; + return other.maxRows() == maxRows(); + } + + @Override + public int hashCode() { return Long.hashCode(this.rows); } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsAdmissionControl.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsAdmissionControl.java new file mode 100644 index 0000000000000..027763ce6fcdf --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsAdmissionControl.java @@ -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.connector.read.streaming; + +import org.apache.spark.annotation.Evolving; + +/** + * A mix-in interface for {@link SparkDataStream} streaming sources to signal that they can control + * the rate of data ingested into the system. These rate limits can come implicitly from the + * contract of triggers, e.g. Trigger.Once() requires that a micro-batch process all data + * available to the system at the start of the micro-batch. Alternatively, sources can decide to + * limit ingest through data source options. + * + * Through this interface, a MicroBatchStream should be able to return the next offset that it will + * process until given a {@link ReadLimit}. + * + * @since 3.0.0 + */ +@Evolving +public interface SupportsAdmissionControl extends SparkDataStream { + + /** + * Returns the read limits potentially passed to the data source through options when creating + * the data source. + */ + default ReadLimit getDefaultReadLimit() { return ReadLimit.allAvailable(); } + + /** + * Returns the most recent offset available given a read limit. The start offset can be used + * to figure out how much new data should be read given the limit. Users should implement this + * method instead of latestOffset for a MicroBatchStream or getOffset for Source. + * + * When this method is called on a `Source`, the source can return `null` if there is no + * data to process. In addition, for the very first micro-batch, the `startOffset` will be + * null as well. + * + * When this method is called on a MicroBatchStream, the `startOffset` will be `initialOffset` + * for the very first micro-batch. The source can return `null` if there is no data to process. + */ + Offset latestOffset(Offset startOffset, ReadLimit limit); +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 36f70024c57f8..e8ce8e1487093 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -30,6 +30,8 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.connector.read.streaming +import org.apache.spark.sql.connector.read.streaming.{ReadAllAvailable, ReadLimit, ReadMaxFiles, SupportsAdmissionControl} import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -45,7 +47,7 @@ class FileStreamSource( override val schema: StructType, partitionColumns: Seq[String], metadataPath: String, - options: Map[String, String]) extends Source with Logging { + options: Map[String, String]) extends SupportsAdmissionControl with Source with Logging { import FileStreamSource._ @@ -115,15 +117,17 @@ class FileStreamSource( * `synchronized` on this method is for solving race conditions in tests. In the normal usage, * there is no race here, so the cost of `synchronized` should be rare. */ - private def fetchMaxOffset(): FileStreamSourceOffset = synchronized { + private def fetchMaxOffset(limit: ReadLimit): FileStreamSourceOffset = synchronized { // All the new files found - ignore aged files and files that we have seen. val newFiles = fetchAllFiles().filter { case (path, timestamp) => seenFiles.isNewFile(path, timestamp) } // Obey user's setting to limit the number of files in this batch trigger. - val batchFiles = - if (maxFilesPerBatch.nonEmpty) newFiles.take(maxFilesPerBatch.get) else newFiles + val batchFiles = limit match { + case files: ReadMaxFiles => newFiles.take(files.maxFiles()) + case _: ReadAllAvailable => newFiles + } batchFiles.foreach { file => seenFiles.add(file._1, file._2) @@ -150,6 +154,10 @@ class FileStreamSource( FileStreamSourceOffset(metadataLogCurrentOffset) } + override def getDefaultReadLimit: ReadLimit = { + maxFilesPerBatch.map(ReadLimit.maxFiles).getOrElse(super.getDefaultReadLimit) + } + /** * For test only. Run `func` with the internal lock to make sure when `func` is running, * the current offset won't be changed and no new batch will be emitted. @@ -269,7 +277,14 @@ class FileStreamSource( files } - override def getOffset: Option[Offset] = Some(fetchMaxOffset()).filterNot(_.logOffset == -1) + override def getOffset: Option[Offset] = { + throw new UnsupportedOperationException( + "latestOffset(Offset, ReadLimit) should be called instead of this method") + } + + override def latestOffset(startOffset: streaming.Offset, limit: ReadLimit): streaming.Offset = { + Some(fetchMaxOffset(limit)).filterNot(_.logOffset == -1).orNull + } override def toString: String = s"FileStreamSource[$qualifiedBasePath]" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 872c36764104f..83bc347e23ed4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -25,10 +25,10 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentBatch import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, Project} import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream} +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset => OffsetV2, ReadLimit, SparkDataStream, SupportsAdmissionControl} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamWriterCommitProgress, WriteToDataSourceV2Exec} -import org.apache.spark.sql.execution.streaming.sources.{RateControlMicroBatchStream, WriteToMicroBatchDataSource} +import org.apache.spark.sql.execution.streaming.sources.WriteToMicroBatchDataSource import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, Trigger} import org.apache.spark.util.Clock @@ -79,7 +79,7 @@ class MicroBatchExecution( import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ val _logicalPlan = analyzedPlan.transform { - case streamingRelation@StreamingRelation(dataSourceV1, sourceName, output) => + case streamingRelation @ StreamingRelation(dataSourceV1, sourceName, output) => toExecutionRelationMap.getOrElseUpdate(streamingRelation, { // Materialize source to avoid creating it in every batch val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" @@ -122,7 +122,18 @@ class MicroBatchExecution( // v2 source case r: StreamingDataSourceV2Relation => r.stream } - uniqueSources = sources.distinct + uniqueSources = sources.distinct.map { + case source: SupportsAdmissionControl => + val limit = source.getDefaultReadLimit + if (trigger == OneTimeTrigger && limit != ReadLimit.allAvailable()) { + logWarning(s"The read limit $limit for $source is ignored when Trigger.Once() is used.") + source -> ReadLimit.allAvailable() + } else { + source -> limit + } + case other => + other -> ReadLimit.allAvailable() + }.toMap // TODO (SPARK-27484): we should add the writing node before the plan is analyzed. sink match { @@ -354,25 +365,33 @@ class MicroBatchExecution( // Generate a map from each unique source to the next available offset. val latestOffsets: Map[SparkDataStream, Option[OffsetV2]] = uniqueSources.map { - case s: Source => + case (s: SupportsAdmissionControl, limit) => updateStatusMessage(s"Getting offsets from $s") - reportTimeTaken("getOffset") { - (s, s.getOffset) + reportTimeTaken("latestOffset") { + val startOffsetOpt = availableOffsets.get(s) + val startOffset = s match { + case _: Source => + startOffsetOpt.orNull + case v2: MicroBatchStream => + startOffsetOpt.map(offset => v2.deserializeOffset(offset.json)) + .getOrElse(v2.initialOffset()) + } + (s, Option(s.latestOffset(startOffset, limit))) } - case s: RateControlMicroBatchStream => + case (s: Source, _) => updateStatusMessage(s"Getting offsets from $s") - reportTimeTaken("latestOffset") { - val startOffset = availableOffsets - .get(s).map(off => s.deserializeOffset(off.json)) - .getOrElse(s.initialOffset()) - (s, Option(s.latestOffset(startOffset))) + reportTimeTaken("getOffset") { + (s, s.getOffset) } - case s: MicroBatchStream => + case (s: MicroBatchStream, _) => updateStatusMessage(s"Getting offsets from $s") reportTimeTaken("latestOffset") { (s, Option(s.latestOffset())) } - }.toMap + case (s, _) => + // for some reason, the compiler is unhappy and thinks the match is not exhaustive + throw new IllegalStateException(s"Unexpected source: $s") + } availableOffsets ++= latestOffsets.filter { case (_, o) => o.nonEmpty }.mapValues(_.get) // Update the query metadata diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ed908a8bad483..8b3534bc0837a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} -import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, SparkDataStream} +import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLimit, SparkDataStream} import org.apache.spark.sql.connector.write.{LogicalWriteInfoImpl, SupportsTruncate} import org.apache.spark.sql.connector.write.streaming.StreamingWrite import org.apache.spark.sql.execution.QueryExecution @@ -206,7 +206,7 @@ abstract class StreamExecution( /** * A list of unique sources in the query plan. This will be set when generating logical plan. */ - @volatile protected var uniqueSources: Seq[SparkDataStream] = Seq.empty + @volatile protected var uniqueSources: Map[SparkDataStream, ReadLimit] = Map.empty /** Defines the internal state of execution */ protected val state = new AtomicReference[State](INITIALIZING) @@ -425,7 +425,7 @@ abstract class StreamExecution( /** Stops all streaming sources safely. */ protected def stopSources(): Unit = { - uniqueSources.foreach { source => + uniqueSources.foreach { case (source, _) => try { source.stop() } catch { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 481552a2e4a0e..a9b724a73a18e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, TableCapability} -import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, Offset => OffsetV2, PartitionOffset} +import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, Offset => OffsetV2, PartitionOffset, ReadLimit} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming.{StreamingRelationV2, _} @@ -84,7 +84,7 @@ class ContinuousExecution( sources = _logicalPlan.collect { case r: StreamingDataSourceV2Relation => r.stream.asInstanceOf[ContinuousStream] } - uniqueSources = sources.distinct + uniqueSources = sources.distinct.map(s => s -> ReadLimit.allAvailable()).toMap // TODO (SPARK-27484): we should add the writing node before the plan is analyzed. WriteToContinuousDataSource( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 632e007fc9444..fa320333143ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -1174,6 +1174,62 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } + test("SPARK-30669: maxFilesPerTrigger - ignored when using Trigger.Once") { + withTempDirs { (src, target) => + val checkpoint = new File(target, "chk").getCanonicalPath + val targetDir = new File(target, "data").getCanonicalPath + var lastFileModTime: Option[Long] = None + + /** Create a text file with a single data item */ + def createFile(data: Int): File = { + val file = stringToFile(new File(src, s"$data.txt"), data.toString) + if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get + 1000) + lastFileModTime = Some(file.lastModified) + file + } + + createFile(1) + createFile(2) + createFile(3) + + // Set up a query to read text files one at a time + val df = spark + .readStream + .option("maxFilesPerTrigger", 1) + .text(src.getCanonicalPath) + + def startQuery(): StreamingQuery = { + df.writeStream + .format("parquet") + .trigger(Trigger.Once) + .option("checkpointLocation", checkpoint) + .start(targetDir) + } + val q = startQuery() + + try { + assert(q.awaitTermination(streamingTimeout.toMillis)) + assert(q.recentProgress.count(_.numInputRows != 0) == 1) // only one trigger was run + checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 3).map(_.toString).toDF) + } finally { + q.stop() + } + + createFile(4) + createFile(5) + + // run a second batch + val q2 = startQuery() + try { + assert(q2.awaitTermination(streamingTimeout.toMillis)) + assert(q2.recentProgress.count(_.numInputRows != 0) == 1) // only one trigger was run + checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 5).map(_.toString).toDF) + } finally { + q2.stop() + } + } + } + test("explain") { withTempDirs { case (src, tmp) => src.mkdirs() From 05be81d69ede7cc57ec0693087401f687131b2d4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 30 Jan 2020 22:51:51 -0800 Subject: [PATCH 1262/1387] [SPARK-30192][SQL][FOLLOWUP] Rename SINGLETON to INSTANCE ### What changes were proposed in this pull request? This PR renames a variable `SINGLETON` to `INSTANCE`. ### Why are the changes needed? This is a minor change for consistency with the other parts. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the existing tests. Closes #27409 from dongjoon-hyun/SPARK-30192. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/connector/catalog/TableChange.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java index 58a592c9d530f..5ce020912f213 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java @@ -298,7 +298,7 @@ public int hashCode() { interface ColumnPosition { static ColumnPosition first() { - return First.SINGLETON; + return First.INSTANCE; } static ColumnPosition after(String column) { @@ -312,7 +312,7 @@ static ColumnPosition after(String column) { * be the first one within the struct. */ final class First implements ColumnPosition { - private static final First SINGLETON = new First(); + private static final First INSTANCE = new First(); private First() {} From a5c7090ffafbfb4a8f312251b9ce8f25f71497cb Mon Sep 17 00:00:00 2001 From: herman Date: Fri, 31 Jan 2020 16:14:07 +0900 Subject: [PATCH 1263/1387] [SPARK-30671][SQL] emptyDataFrame should use a LocalRelation ### What changes were proposed in this pull request? This PR makes `SparkSession.emptyDataFrame` use an empty local relation instead of an empty RDD. This allows to optimizer to recognize this as an empty relation, and creates the opportunity to do some more aggressive optimizations. ### Why are the changes needed? It allows us to optimize empty dataframes better. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added a test case to `DataFrameSuite`. Closes #27400 from hvanhovell/SPARK-30671. Authored-by: herman Signed-off-by: HyukjinKwon --- .../scala/org/apache/spark/sql/SparkSession.scala | 4 +--- .../scala/org/apache/spark/sql/DataFrameSuite.scala | 13 ++++++++++++- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index bd2bc1c0ad5d7..0ce514fb0f83c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -274,9 +274,7 @@ class SparkSession private( * @since 2.0.0 */ @transient - lazy val emptyDataFrame: DataFrame = { - createDataFrame(sparkContext.emptyRDD[Row].setName("empty"), StructType(Nil)) - } + lazy val emptyDataFrame: DataFrame = Dataset.ofRows(self, LocalRelation()) /** * Creates a new [[Dataset]] of type T containing zero elements. 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 763f92230cdc3..d2d58a83ded5d 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.Uuid import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation -import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, OneRowRelation, Union} import org.apache.spark.sql.execution.{FilterExec, QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.HashAggregateExec @@ -2287,6 +2287,17 @@ class DataFrameSuite extends QueryTest } assert(err.getMessage.contains("cannot resolve '`d`'")) } + + test("emptyDataFrame should be foldable") { + val emptyDf = spark.emptyDataFrame.withColumn("id", lit(1L)) + val joined = spark.range(10).join(emptyDf, "id") + joined.queryExecution.optimizedPlan match { + case LocalRelation(Seq(id), Nil, _) => + assert(id.name == "id") + case _ => + fail("emptyDataFrame should be foldable") + } + } } case class GroupByKey(a: Int, b: Int) From 6fac411076d50a32bbf268a117ff483ae2d2cf3a Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 30 Jan 2020 23:36:39 -0800 Subject: [PATCH 1264/1387] [SPARK-29093][ML][PYSPARK][FOLLOW-UP] Remove duplicate setter ### What changes were proposed in this pull request? remove duplicate setter in ```BucketedRandomProjectionLSH``` ### Why are the changes needed? Remove the duplicate ```setInputCol/setOutputCol``` in ```BucketedRandomProjectionLSH``` because these two setter are already in super class ```LSH``` ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually checked. Closes #27397 from huaxingao/spark-29093. Authored-by: Huaxin Gao Signed-off-by: Dongjoon Hyun --- python/pyspark/ml/feature.py | 12 ------------ python/pyspark/ml/regression.py | 2 +- 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index f9206e957796a..4c25bb495fdfb 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -415,18 +415,6 @@ def setBucketLength(self, value): """ return self._set(bucketLength=value) - def setInputCol(self, value): - """ - Sets the value of :py:attr:`inputCol`. - """ - return self._set(inputCol=value) - - def setOutputCol(self, value): - """ - Sets the value of :py:attr:`outputCol`. - """ - return self._set(outputCol=value) - def setSeed(self, value): """ Sets the value of :py:attr:`seed`. diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 481271f523151..a4c97827d55de 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -2067,7 +2067,7 @@ def setTol(self, value): """ return self._set(tol=value) - @since("2.2.0") + @since("2.0.0") def setFitIntercept(self, value): """ Sets the value of :py:attr:`fitIntercept`. From 290a528bff7bcb449714c1c6f1885bd0f804358d Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 31 Jan 2020 16:41:10 +0800 Subject: [PATCH 1265/1387] [SPARK-30615][SQL] Introduce Analyzer rule for V2 AlterTable column change resolution ### What changes were proposed in this pull request? Adds an Analyzer rule to normalize the column names used in V2 AlterTable table changes. We need to handle all ColumnChange operations. We add an extra match statement for future proofing new changes that may be added. This prevents downstream consumers (e.g. catalogs) to deal about case sensitivity or check that columns exist, etc. We also fix the behavior for ALTER TABLE CHANGE COLUMN (Hive style syntax) for adding comments to complex data types. Currently, the data type needs to be provided as part of the Hive style syntax. This assumes that the data type as changed when it may have not and the user only wants to add a comment, which fails in CheckAnalysis. ### Why are the changes needed? Currently we do not handle case sensitivity correctly for ALTER TABLE ALTER COLUMN operations. ### Does this PR introduce any user-facing change? No, fixes a bug. ### How was this patch tested? Introduced v2CommandsCaseSensitivitySuite and added a test around HiveStyle Change columns to PlanResolutionSuite Closes #27350 from brkyvz/normalizeAlter. Authored-by: Burak Yavuz Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 156 ++++++++++++ .../sql/catalyst/analysis/CheckAnalysis.scala | 60 ++++- .../apache/spark/sql/types/StructType.scala | 85 ++++--- .../sql/catalyst/analysis/AnalysisTest.scala | 9 +- ...eateTablePartitioningValidationSuite.scala | 4 +- .../spark/sql/connector/AlterTableTests.scala | 71 +++++- .../V2CommandsCaseSensitivitySuite.scala | 227 ++++++++++++++++++ .../command/PlanResolutionSuite.scala | 26 +- 8 files changed, 583 insertions(+), 55 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 53d73d503b097..3fd5039a4f116 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, After, ColumnChange, ColumnPosition, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnPosition, UpdateColumnType} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf @@ -240,6 +241,7 @@ class Analyzer( TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*), + Batch("Normalize Alter Table", Once, ResolveAlterTableChanges), Batch("Remove Unresolved Hints", Once, new ResolveHints.RemoveAllHints(conf)), Batch("Nondeterministic", Once, @@ -3001,6 +3003,160 @@ class Analyzer( } } } + + /** Rule to mostly resolve, normalize and rewrite column names based on case sensitivity. */ + object ResolveAlterTableChanges extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case a @ AlterTable(_, _, t: NamedRelation, changes) if t.resolved => + val schema = t.schema + val normalizedChanges = changes.flatMap { + case add: AddColumn => + val parent = add.fieldNames().init + if (parent.nonEmpty) { + // Adding a nested field, need to normalize the parent column and position + val target = schema.findNestedField(parent, includeCollections = true, conf.resolver) + if (target.isEmpty) { + // Leave unresolved. Throws error in CheckAnalysis + Some(add) + } else { + val (normalizedName, sf) = target.get + sf.dataType match { + case struct: StructType => + val pos = findColumnPosition(add.position(), parent.quoted, struct) + Some(TableChange.addColumn( + (normalizedName ++ Seq(sf.name, add.fieldNames().last)).toArray, + add.dataType(), + add.isNullable, + add.comment, + pos)) + + case other => + Some(add) + } + } + } else { + // Adding to the root. Just need to normalize position + val pos = findColumnPosition(add.position(), "root", schema) + Some(TableChange.addColumn( + add.fieldNames(), + add.dataType(), + add.isNullable, + add.comment, + pos)) + } + + case typeChange: UpdateColumnType => + // Hive style syntax provides the column type, even if it may not have changed + val fieldOpt = schema.findNestedField( + typeChange.fieldNames(), includeCollections = true, conf.resolver) + + if (fieldOpt.isEmpty) { + // We couldn't resolve the field. Leave it to CheckAnalysis + Some(typeChange) + } else { + val (fieldNames, field) = fieldOpt.get + if (field.dataType == typeChange.newDataType()) { + // The user didn't want the field to change, so remove this change + None + } else { + Some(TableChange.updateColumnType( + (fieldNames :+ field.name).toArray, typeChange.newDataType())) + } + } + case n: UpdateColumnNullability => + // Need to resolve column + resolveFieldNames( + schema, + n.fieldNames(), + TableChange.updateColumnNullability(_, n.nullable())).orElse(Some(n)) + + case position: UpdateColumnPosition => + position.position() match { + case after: After => + // Need to resolve column as well as position reference + val fieldOpt = schema.findNestedField( + position.fieldNames(), includeCollections = true, conf.resolver) + + if (fieldOpt.isEmpty) { + Some(position) + } else { + val (normalizedPath, field) = fieldOpt.get + val targetCol = schema.findNestedField( + normalizedPath :+ after.column(), includeCollections = true, conf.resolver) + if (targetCol.isEmpty) { + // Leave unchanged to CheckAnalysis + Some(position) + } else { + Some(TableChange.updateColumnPosition( + (normalizedPath :+ field.name).toArray, + ColumnPosition.after(targetCol.get._2.name))) + } + } + case _ => + // Need to resolve column + resolveFieldNames( + schema, + position.fieldNames(), + TableChange.updateColumnPosition(_, position.position())).orElse(Some(position)) + } + + case comment: UpdateColumnComment => + resolveFieldNames( + schema, + comment.fieldNames(), + TableChange.updateColumnComment(_, comment.newComment())).orElse(Some(comment)) + + case rename: RenameColumn => + resolveFieldNames( + schema, + rename.fieldNames(), + TableChange.renameColumn(_, rename.newName())).orElse(Some(rename)) + + case delete: DeleteColumn => + resolveFieldNames(schema, delete.fieldNames(), TableChange.deleteColumn) + .orElse(Some(delete)) + + case column: ColumnChange => + // This is informational for future developers + throw new UnsupportedOperationException( + "Please add an implementation for a column change here") + case other => Some(other) + } + + a.copy(changes = normalizedChanges) + } + + /** + * Returns the table change if the field can be resolved, returns None if the column is not + * found. An error will be thrown in CheckAnalysis for columns that can't be resolved. + */ + private def resolveFieldNames( + schema: StructType, + fieldNames: Array[String], + copy: Array[String] => TableChange): Option[TableChange] = { + val fieldOpt = schema.findNestedField( + fieldNames, includeCollections = true, conf.resolver) + fieldOpt.map { case (path, field) => copy((path :+ field.name).toArray) } + } + + private def findColumnPosition( + position: ColumnPosition, + field: String, + struct: StructType): ColumnPosition = { + position match { + case null => null + case after: After => + struct.fieldNames.find(n => conf.resolver(n, after.column())) match { + case Some(colName) => + ColumnPosition.after(colName) + case None => + throw new AnalysisException("Couldn't find the reference column for " + + s"$after at $field") + } + case other => other + } + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index d6fc1dc6ddc3d..4ec737fd9b70d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.TypeUtils -import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnType} +import org.apache.spark.sql.connector.catalog.TableChange.{AddColumn, After, ColumnPosition, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnNullability, UpdateColumnPosition, UpdateColumnType} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -425,24 +425,56 @@ trait CheckAnalysis extends PredicateHelper { case _ => } - case alter: AlterTable if alter.childrenResolved => + case alter: AlterTable if alter.table.resolved => val table = alter.table def findField(operation: String, fieldName: Array[String]): StructField = { // include collections because structs nested in maps and arrays may be altered val field = table.schema.findNestedField(fieldName, includeCollections = true) if (field.isEmpty) { - throw new AnalysisException( - s"Cannot $operation missing field in ${table.name} schema: ${fieldName.quoted}") + alter.failAnalysis( + s"Cannot $operation missing field ${fieldName.quoted} in ${table.name} schema: " + + table.schema.treeString) + } + field.get._2 + } + def positionArgumentExists(position: ColumnPosition, struct: StructType): Unit = { + position match { + case after: After => + if (!struct.fieldNames.contains(after.column())) { + alter.failAnalysis(s"Couldn't resolve positional argument $position amongst " + + s"${struct.fieldNames.mkString("[", ", ", "]")}") + } + case _ => + } + } + def findParentStruct(operation: String, fieldNames: Array[String]): StructType = { + val parent = fieldNames.init + val field = if (parent.nonEmpty) { + findField(operation, parent).dataType + } else { + table.schema + } + field match { + case s: StructType => s + case o => alter.failAnalysis(s"Cannot $operation ${fieldNames.quoted}, because " + + s"its parent is not a StructType. Found $o") + } + } + def checkColumnNotExists( + operation: String, + fieldNames: Array[String], + struct: StructType): Unit = { + if (struct.findNestedField(fieldNames, includeCollections = true).isDefined) { + alter.failAnalysis(s"Cannot $operation column, because ${fieldNames.quoted} " + + s"already exists in ${struct.treeString}") } - field.get } alter.changes.foreach { case add: AddColumn => - val parent = add.fieldNames.init - if (parent.nonEmpty) { - findField("add to", parent) - } + checkColumnNotExists("add", add.fieldNames(), table.schema) + val parent = findParentStruct("add", add.fieldNames()) + positionArgumentExists(add.position(), parent) TypeUtils.failWithIntervalType(add.dataType()) case update: UpdateColumnType => val field = findField("update", update.fieldNames) @@ -467,7 +499,7 @@ trait CheckAnalysis extends PredicateHelper { // update is okay } if (!Cast.canUpCast(field.dataType, update.newDataType)) { - throw new AnalysisException( + alter.failAnalysis( s"Cannot update ${table.name} field $fieldName: " + s"${field.dataType.simpleString} cannot be cast to " + s"${update.newDataType.simpleString}") @@ -476,11 +508,17 @@ trait CheckAnalysis extends PredicateHelper { val field = findField("update", update.fieldNames) val fieldName = update.fieldNames.quoted if (!update.nullable && field.nullable) { - throw new AnalysisException( + alter.failAnalysis( s"Cannot change nullable column to non-nullable: $fieldName") } + case updatePos: UpdateColumnPosition => + findField("update", updatePos.fieldNames) + val parent = findParentStruct("update", updatePos.fieldNames()) + positionArgumentExists(updatePos.position(), parent) case rename: RenameColumn => findField("rename", rename.fieldNames) + checkColumnNotExists( + "rename", rename.fieldNames().init :+ rename.newName(), table.schema) case update: UpdateColumnComment => findField("update", update.fieldNames) case delete: DeleteColumn => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 0dcb1fc7f0006..bd2c1d5c26299 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -25,6 +25,8 @@ import org.json4s.JsonDSL._ import org.apache.spark.SparkException import org.apache.spark.annotation.Stable +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, truncatedString, StringUtils} @@ -309,52 +311,75 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru } /** - * Returns a field in this struct and its child structs. + * Returns the normalized path to a field and the field in this struct and its child structs. * * If includeCollections is true, this will return fields that are nested in maps and arrays. */ private[sql] def findNestedField( fieldNames: Seq[String], - includeCollections: Boolean = false): Option[StructField] = { - fieldNames.headOption.flatMap(nameToField.get) match { - case Some(field) => - (fieldNames.tail, field.dataType, includeCollections) match { - case (Seq(), _, _) => - Some(field) + includeCollections: Boolean = false, + resolver: Resolver = _ == _): Option[(Seq[String], StructField)] = { + def prettyFieldName(nameParts: Seq[String]): String = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + nameParts.quoted + } + + def findField( + struct: StructType, + searchPath: Seq[String], + normalizedPath: Seq[String]): Option[(Seq[String], StructField)] = { + searchPath.headOption.flatMap { searchName => + val found = struct.fields.filter(f => resolver(searchName, f.name)) + if (found.length > 1) { + val names = found.map(f => prettyFieldName(normalizedPath :+ f.name)) + .mkString("[", ", ", " ]") + throw new AnalysisException( + s"Ambiguous field name: ${prettyFieldName(normalizedPath :+ searchName)}. Found " + + s"multiple columns that can match: $names") + } else if (found.isEmpty) { + None + } else { + val field = found.head + (searchPath.tail, field.dataType, includeCollections) match { + case (Seq(), _, _) => + Some(normalizedPath -> field) - case (names, struct: StructType, _) => - struct.findNestedField(names, includeCollections) + case (names, struct: StructType, _) => + findField(struct, names, normalizedPath :+ field.name) - case (_, _, false) => - None // types nested in maps and arrays are not used + case (_, _, false) => + None // types nested in maps and arrays are not used - case (Seq("key"), MapType(keyType, _, _), true) => - // return the key type as a struct field to include nullability - Some(StructField("key", keyType, nullable = false)) + case (Seq("key"), MapType(keyType, _, _), true) => + // return the key type as a struct field to include nullability + Some((normalizedPath :+ field.name) -> StructField("key", keyType, nullable = false)) - case (Seq("key", names @ _*), MapType(struct: StructType, _, _), true) => - struct.findNestedField(names, includeCollections) + case (Seq("key", names @ _*), MapType(struct: StructType, _, _), true) => + findField(struct, names, normalizedPath ++ Seq(field.name, "key")) - case (Seq("value"), MapType(_, valueType, isNullable), true) => - // return the value type as a struct field to include nullability - Some(StructField("value", valueType, nullable = isNullable)) + case (Seq("value"), MapType(_, valueType, isNullable), true) => + // return the value type as a struct field to include nullability + Some((normalizedPath :+ field.name) -> + StructField("value", valueType, nullable = isNullable)) - case (Seq("value", names @ _*), MapType(_, struct: StructType, _), true) => - struct.findNestedField(names, includeCollections) + case (Seq("value", names @ _*), MapType(_, struct: StructType, _), true) => + findField(struct, names, normalizedPath ++ Seq(field.name, "value")) - case (Seq("element"), ArrayType(elementType, isNullable), true) => - // return the element type as a struct field to include nullability - Some(StructField("element", elementType, nullable = isNullable)) + case (Seq("element"), ArrayType(elementType, isNullable), true) => + // return the element type as a struct field to include nullability + Some((normalizedPath :+ field.name) -> + StructField("element", elementType, nullable = isNullable)) - case (Seq("element", names @ _*), ArrayType(struct: StructType, _), true) => - struct.findNestedField(names, includeCollections) + case (Seq("element", names @ _*), ArrayType(struct: StructType, _), true) => + findField(struct, names, normalizedPath ++ Seq(field.name, "element")) - case _ => - None + case _ => + None + } } - case _ => - None + } } + findField(this, fieldNames, Nil) } protected[sql] def toAttributes: Seq[AttributeReference] = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 7d196f8b8edd2..3f8d409992381 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -26,12 +26,15 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, InMemoryCatalog, import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf trait AnalysisTest extends PlanTest { - protected val caseSensitiveAnalyzer = makeAnalyzer(caseSensitive = true) - protected val caseInsensitiveAnalyzer = makeAnalyzer(caseSensitive = false) + protected lazy val caseSensitiveAnalyzer = makeAnalyzer(caseSensitive = true) + protected lazy val caseInsensitiveAnalyzer = makeAnalyzer(caseSensitive = false) + + protected def extendedAnalysisRules: Seq[Rule[LogicalPlan]] = Nil private def makeAnalyzer(caseSensitive: Boolean): Analyzer = { val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> caseSensitive) @@ -43,7 +46,7 @@ trait AnalysisTest extends PlanTest { catalog.createTempView("TaBlE2", TestRelations.testRelation2, overrideIfExists = true) catalog.createTempView("TaBlE3", TestRelations.testRelation3, overrideIfExists = true) new Analyzer(catalog, conf) { - override val extendedResolutionRules = EliminateSubqueryAliases :: Nil + override val extendedResolutionRules = EliminateSubqueryAliases +: extendedAnalysisRules } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala index 9dd43ea70eb4b..f433229595e9e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala @@ -133,7 +133,7 @@ class CreateTablePartitioningValidationSuite extends AnalysisTest { } } -private object CreateTablePartitioningValidationSuite { +private[sql] object CreateTablePartitioningValidationSuite { val catalog: TableCatalog = { val cat = new InMemoryTableCatalog() cat.initialize("test", CaseInsensitiveStringMap.empty()) @@ -146,7 +146,7 @@ private object CreateTablePartitioningValidationSuite { .add("point", new StructType().add("x", DoubleType).add("y", DoubleType)) } -private case object TestRelation2 extends LeafNode with NamedRelation { +private[sql] case object TestRelation2 extends LeafNode with NamedRelation { override def name: String = "source_relation" override def output: Seq[AttributeReference] = CreateTablePartitioningValidationSuite.schema.toAttributes diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index ee7f205b3fa52..3cdac59c20fc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -146,9 +146,9 @@ trait AlterTableTests extends SharedSparkSession { .add("point", new StructType().add("x", IntegerType)) .add("b", StringType)) - val e1 = intercept[SparkException]( + val e1 = intercept[AnalysisException]( sql(s"ALTER TABLE $t ADD COLUMN c string AFTER non_exist")) - assert(e1.getMessage().contains("AFTER column not found")) + assert(e1.getMessage().contains("Couldn't find the reference column")) sql(s"ALTER TABLE $t ADD COLUMN point.y int FIRST") assert(getTableMetadata(t).schema == new StructType() @@ -167,9 +167,9 @@ trait AlterTableTests extends SharedSparkSession { .add("z", IntegerType)) .add("b", StringType)) - val e2 = intercept[SparkException]( + val e2 = intercept[AnalysisException]( sql(s"ALTER TABLE $t ADD COLUMN point.x2 int AFTER non_exist")) - assert(e2.getMessage().contains("AFTER column not found")) + assert(e2.getMessage().contains("Couldn't find the reference column")) } } @@ -312,6 +312,30 @@ trait AlterTableTests extends SharedSparkSession { } } + test("AlterTable: add column - new column should not exist") { + val t = s"${catalogAndNamespace}table_name" + withTable(t) { + sql( + s"""CREATE TABLE $t ( + |id int, + |point struct, + |arr array>, + |mk map, string>, + |mv map> + |) + |USING $v2Format""".stripMargin) + + Seq("id", "point.x", "arr.element.x", "mk.key.x", "mv.value.x").foreach { field => + + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ADD COLUMNS $field double") + } + assert(e.getMessage.contains("add")) + assert(e.getMessage.contains(s"$field already exists")) + } + } + } + test("AlterTable: update column type int -> long") { val t = s"${catalogAndNamespace}table_name" withTable(t) { @@ -595,9 +619,9 @@ trait AlterTableTests extends SharedSparkSession { .add("z", IntegerType)) .add("b", IntegerType)) - val e1 = intercept[SparkException]( + val e1 = intercept[AnalysisException]( sql(s"ALTER TABLE $t ALTER COLUMN b AFTER non_exist")) - assert(e1.getMessage.contains("AFTER column not found")) + assert(e1.getMessage.contains("Couldn't resolve positional argument")) sql(s"ALTER TABLE $t ALTER COLUMN point.y FIRST") assert(getTableMetadata(t).schema == new StructType() @@ -617,9 +641,9 @@ trait AlterTableTests extends SharedSparkSession { .add("y", IntegerType)) .add("b", IntegerType)) - val e2 = intercept[SparkException]( + val e2 = intercept[AnalysisException]( sql(s"ALTER TABLE $t ALTER COLUMN point.y AFTER non_exist")) - assert(e2.getMessage.contains("AFTER column not found")) + assert(e2.getMessage.contains("Couldn't resolve positional argument")) // `AlterTable.resolved` checks column existence. intercept[AnalysisException]( @@ -849,6 +873,37 @@ trait AlterTableTests extends SharedSparkSession { } } + test("AlterTable: rename column - new name should not exist") { + val t = s"${catalogAndNamespace}table_name" + withTable(t) { + sql( + s"""CREATE TABLE $t ( + |id int, + |user_id int, + |point struct, + |arr array>, + |mk map, string>, + |mv map> + |) + |USING $v2Format""".stripMargin) + + Seq( + "id" -> "user_id", + "point.x" -> "y", + "arr.element.x" -> "y", + "mk.key.x" -> "y", + "mv.value.x" -> "y").foreach { case (field, newName) => + + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE $t RENAME COLUMN $field TO $newName") + } + assert(e.getMessage.contains("rename")) + assert(e.getMessage.contains((field.split("\\.").init :+ newName).mkString("."))) + assert(e.getMessage.contains("already exists")) + } + } + } + test("AlterTable: drop column") { val t = s"${catalogAndNamespace}table_name" withTable(t) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala new file mode 100644 index 0000000000000..289f9dc427795 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala @@ -0,0 +1,227 @@ +/* + * 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.connector + +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, TestRelation2} +import org.apache.spark.sql.catalyst.analysis.CreateTablePartitioningValidationSuite +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, CreateTableAsSelect, LogicalPlan, ReplaceTableAsSelect} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{Identifier, TableChange} +import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition +import org.apache.spark.sql.connector.expressions.Expressions +import org.apache.spark.sql.execution.datasources.PreprocessTableCreation +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{LongType, StringType} + +class V2CommandsCaseSensitivitySuite extends SharedSparkSession with AnalysisTest { + import CreateTablePartitioningValidationSuite._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override protected def extendedAnalysisRules: Seq[Rule[LogicalPlan]] = { + Seq(PreprocessTableCreation(spark)) + } + + test("CreateTableAsSelect: using top level field for partitioning") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + Seq("ID", "iD").foreach { ref => + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + Expressions.identity(ref) :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + if (caseSensitive) { + assertAnalysisError(plan, Seq("Couldn't find column", ref), caseSensitive) + } else { + assertAnalysisSuccess(plan, caseSensitive) + } + } + } + } + } + + test("CreateTableAsSelect: using nested column for partitioning") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + Seq("POINT.X", "point.X", "poInt.x", "poInt.X").foreach { ref => + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + Expressions.bucket(4, ref) :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + if (caseSensitive) { + val field = ref.split("\\.") + assertAnalysisError(plan, Seq("Couldn't find column", field.head), caseSensitive) + } else { + assertAnalysisSuccess(plan, caseSensitive) + } + } + } + } + } + + test("ReplaceTableAsSelect: using top level field for partitioning") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + Seq("ID", "iD").foreach { ref => + val plan = ReplaceTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + Expressions.identity(ref) :: Nil, + TestRelation2, + Map.empty, + Map.empty, + orCreate = true) + + if (caseSensitive) { + assertAnalysisError(plan, Seq("Couldn't find column", ref), caseSensitive) + } else { + assertAnalysisSuccess(plan, caseSensitive) + } + } + } + } + } + + test("ReplaceTableAsSelect: using nested column for partitioning") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + Seq("POINT.X", "point.X", "poInt.x", "poInt.X").foreach { ref => + val plan = ReplaceTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + Expressions.bucket(4, ref) :: Nil, + TestRelation2, + Map.empty, + Map.empty, + orCreate = true) + + if (caseSensitive) { + val field = ref.split("\\.") + assertAnalysisError(plan, Seq("Couldn't find column", field.head), caseSensitive) + } else { + assertAnalysisSuccess(plan, caseSensitive) + } + } + } + } + } + + test("AlterTable: add column - nested") { + Seq("POINT.Z", "poInt.z", "poInt.Z").foreach { ref => + val field = ref.split("\\.") + alterTableTest( + TableChange.addColumn(field, LongType), + Seq("add", field.head) + ) + } + } + + test("AlterTable: add column resolution - positional") { + Seq("ID", "iD").foreach { ref => + alterTableTest( + TableChange.addColumn( + Array("f"), LongType, true, null, ColumnPosition.after(ref)), + Seq("reference column", ref) + ) + } + } + + test("AlterTable: add column resolution - nested positional") { + Seq("X", "Y").foreach { ref => + alterTableTest( + TableChange.addColumn( + Array("point", "z"), LongType, true, null, ColumnPosition.after(ref)), + Seq("reference column", ref) + ) + } + } + + test("AlterTable: drop column resolution") { + Seq(Array("ID"), Array("point", "X"), Array("POINT", "X"), Array("POINT", "x")).foreach { ref => + alterTableTest( + TableChange.deleteColumn(ref), + Seq("Cannot delete missing field", ref.quoted) + ) + } + } + + test("AlterTable: rename column resolution") { + Seq(Array("ID"), Array("point", "X"), Array("POINT", "X"), Array("POINT", "x")).foreach { ref => + alterTableTest( + TableChange.renameColumn(ref, "newName"), + Seq("Cannot rename missing field", ref.quoted) + ) + } + } + + test("AlterTable: drop column nullability resolution") { + Seq(Array("ID"), Array("point", "X"), Array("POINT", "X"), Array("POINT", "x")).foreach { ref => + alterTableTest( + TableChange.updateColumnNullability(ref, true), + Seq("Cannot update missing field", ref.quoted) + ) + } + } + + test("AlterTable: change column type resolution") { + Seq(Array("ID"), Array("point", "X"), Array("POINT", "X"), Array("POINT", "x")).foreach { ref => + alterTableTest( + TableChange.updateColumnType(ref, StringType), + Seq("Cannot update missing field", ref.quoted) + ) + } + } + + test("AlterTable: change column comment resolution") { + Seq(Array("ID"), Array("point", "X"), Array("POINT", "X"), Array("POINT", "x")).foreach { ref => + alterTableTest( + TableChange.updateColumnComment(ref, "Here's a comment for ya"), + Seq("Cannot update missing field", ref.quoted) + ) + } + } + + private def alterTableTest(change: TableChange, error: Seq[String]): Unit = { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val plan = AlterTable( + catalog, + Identifier.of(Array(), "table_name"), + TestRelation2, + Seq(change) + ) + + if (caseSensitive) { + assertAnalysisError(plan, error, caseSensitive) + } else { + assertAnalysisSuccess(plan, caseSensitive) + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 30b7e93a4beb4..c0c3cd70fcc9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, Assignment, CreateTableAsSelect, CreateV2Table, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, Project, ShowTableProperties, SubqueryAlias, UpdateAction, UpdateTable} import org.apache.spark.sql.connector.FakeV2Provider import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, TableCapability, TableCatalog, TableChange, V1Table} +import org.apache.spark.sql.connector.catalog.TableChange.{UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf @@ -148,7 +149,8 @@ class PlanResolutionSuite extends AnalysisTest { analyzer.ResolveTables, analyzer.ResolveReferences, analyzer.ResolveSubqueryColumnAliases, - analyzer.ResolveReferences) + analyzer.ResolveReferences, + analyzer.ResolveAlterTableChanges) rules.foldLeft(parsePlan(query)) { case (plan, rule) => rule.apply(plan) } @@ -1075,6 +1077,28 @@ class PlanResolutionSuite extends AnalysisTest { } } + test("alter table: hive style change column") { + Seq("v2Table", "testcat.tab").foreach { tblName => + parseAndResolve(s"ALTER TABLE $tblName CHANGE COLUMN i i int COMMENT 'an index'") match { + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes.length == 1, "Should only have a comment change") + assert(changes.head.isInstanceOf[UpdateColumnComment], + s"Expected only a UpdateColumnComment change but got: ${changes.head}") + case _ => fail("expect AlterTable") + } + + parseAndResolve(s"ALTER TABLE $tblName CHANGE COLUMN i i long COMMENT 'an index'") match { + case AlterTable(_, _, _: DataSourceV2Relation, changes) => + assert(changes.length == 2, "Should have a comment change and type change") + assert(changes.exists(_.isInstanceOf[UpdateColumnComment]), + s"Expected UpdateColumnComment change but got: ${changes}") + assert(changes.exists(_.isInstanceOf[UpdateColumnType]), + s"Expected UpdateColumnType change but got: ${changes}") + case _ => fail("expect AlterTable") + } + } + } + val DSV2ResolutionTests = { val v2SessionCatalogTable = s"${CatalogManager.SESSION_CATALOG_NAME}.v2Table" Seq( From 6f4703e22e89a72eb3b0f8c9f84e9a782de82976 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 31 Jan 2020 22:50:01 +0900 Subject: [PATCH 1266/1387] [SPARK-30690][DOCS][BUILD] Add CalendarInterval into API documentation ### What changes were proposed in this pull request? We should also expose it in documentation as we marked it as unstable API as of SPARK-30547 Note that, seems Javadoc -> Scaladoc doesn't work but this PR does not target to fix. ### Why are the changes needed? To show the documentation of API. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually built the docs via `jykill serve` under `docs` directory: ![Screen Shot 2020-01-31 at 4 04 15 PM](https://user-images.githubusercontent.com/6477701/73519315-12143300-4444-11ea-9260-070c9f672dde.png) Closes #27412 from HyukjinKwon/SPARK-30547. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- .../java/org/apache/spark/unsafe/types/CalendarInterval.java | 2 ++ project/SparkBuild.scala | 4 +++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 884db6fe27c4c..f2d06e793f9dd 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -45,6 +45,8 @@ */ @Unstable public final class CalendarInterval implements Serializable { + // NOTE: If you're moving or renaming this file, you should also update Unidoc configuration + // specified in 'SparkBuild.scala'. public final int months; public final int days; public final long microseconds; diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 87e584bd28b68..707c31d2248eb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -825,7 +825,9 @@ object Unidoc { f.getCanonicalPath.contains("org/apache/spark/shuffle") && !f.getCanonicalPath.contains("org/apache/spark/shuffle/api"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/executor"))) - .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/unsafe"))) + .map(_.filterNot(f => + f.getCanonicalPath.contains("org/apache/spark/unsafe") && + !f.getCanonicalPath.contains("org/apache/spark/unsafe/types/CalendarInterval"))) .map(_.filterNot(_.getCanonicalPath.contains("python"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/collection"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/util/kvstore"))) From 3d2b8d8b13eff0faa02316542a343e7a64873b8a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 31 Jan 2020 08:25:32 -0600 Subject: [PATCH 1267/1387] [SPARK-30638][CORE] Add resources allocated to PluginContext ### What changes were proposed in this pull request? Add the allocated resources to parameters to the PluginContext so that any plugins in driver or executor could use this information to initialize devices or use this information in a useful manner. ### Why are the changes needed? To allow users to initialize/track devices once at the executor level before each task runs to use them. ### Does this PR introduce any user-facing change? Yes to the people using the Executor/Driver plugin interface. ### How was this patch tested? Unit tests and manually by writing a plugin that initialized GPU's using this interface. Closes #27367 from tgravescs/pluginWithResources. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../spark/api/plugin/PluginContext.java | 5 + .../scala/org/apache/spark/SparkContext.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 10 +- .../org/apache/spark/executor/Executor.scala | 7 +- .../internal/plugin/PluginContainer.scala | 36 ++++-- .../internal/plugin/PluginContextImpl.scala | 6 +- .../local/LocalSchedulerBackend.scala | 5 +- .../apache/spark/executor/ExecutorSuite.scala | 12 +- .../plugin/PluginContainerSuite.scala | 109 +++++++++++++++++- .../spark/executor/MesosExecutorBackend.scala | 4 +- 10 files changed, 167 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/plugin/PluginContext.java b/core/src/main/java/org/apache/spark/api/plugin/PluginContext.java index b9413cf828aa1..36d827598dfc5 100644 --- a/core/src/main/java/org/apache/spark/api/plugin/PluginContext.java +++ b/core/src/main/java/org/apache/spark/api/plugin/PluginContext.java @@ -18,11 +18,13 @@ package org.apache.spark.api.plugin; import java.io.IOException; +import java.util.Map; import com.codahale.metrics.MetricRegistry; import org.apache.spark.SparkConf; import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.resource.ResourceInformation; /** * :: DeveloperApi :: @@ -54,6 +56,9 @@ public interface PluginContext { /** The host name which is being used by the Spark process for communication. */ String hostname(); + /** The custom resources (GPUs, FPGAs, etc) allocated to driver or executor. */ + Map resources(); + /** * Send a message to the plugin's driver-side component. *

    diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3262631616279..6e0c7acf8b18d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -542,7 +542,7 @@ class SparkContext(config: SparkConf) extends Logging { HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) // Initialize any plugins before the task scheduler is initialized. - _plugins = PluginContainer(this) + _plugins = PluginContainer(this, _resources.asJava) // Create and start the scheduler val (sched, ts) = SparkContext.createTaskScheduler(this, master, deployMode) 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 511c63ad3d551..ce211ce8dd67a 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -69,6 +69,8 @@ private[spark] class CoarseGrainedExecutorBackend( // to be changed so that we don't share the serializer instance across threads private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() + private var _resources = Map.empty[String, ResourceInformation] + /** * Map each taskId to the information about the resource allocated to it, Please refer to * [[ResourceInformation]] for specifics. @@ -78,9 +80,8 @@ private[spark] class CoarseGrainedExecutorBackend( override def onStart(): Unit = { logInfo("Connecting to driver: " + driverUrl) - var resources = Map.empty[String, ResourceInformation] try { - resources = parseOrFindResources(resourcesFileOpt) + _resources = parseOrFindResources(resourcesFileOpt) } catch { case NonFatal(e) => exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) @@ -89,7 +90,7 @@ private[spark] class CoarseGrainedExecutorBackend( // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls, - extractAttributes, resources, resourceProfile.id)) + extractAttributes, _resources, resourceProfile.id)) }(ThreadUtils.sameThread).onComplete { case Success(_) => self.send(RegisteredExecutor) @@ -125,7 +126,8 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor => logInfo("Successfully registered with driver") try { - executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false, + resources = _resources) driver.get.send(LaunchedExecutor(executorId)) } catch { case NonFatal(e) => diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 0ea16d0dcb159..8aeb16fe5d8c8 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ +import scala.collection.immutable import scala.collection.mutable.{ArrayBuffer, HashMap, Map, WrappedArray} import scala.concurrent.duration._ import scala.util.control.NonFatal @@ -41,6 +42,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.plugin.PluginContainer import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.metrics.source.JVMCPUSource +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException @@ -61,7 +63,8 @@ private[spark] class Executor( env: SparkEnv, userClassPath: Seq[URL] = Nil, isLocal: Boolean = false, - uncaughtExceptionHandler: UncaughtExceptionHandler = new SparkUncaughtExceptionHandler) + uncaughtExceptionHandler: UncaughtExceptionHandler = new SparkUncaughtExceptionHandler, + resources: immutable.Map[String, ResourceInformation]) extends Logging { logInfo(s"Starting executor ID $executorId on host $executorHostname") @@ -152,7 +155,7 @@ private[spark] class Executor( // Plugins need to load using a class loader that includes the executor's user classpath private val plugins: Option[PluginContainer] = Utils.withContextClassLoader(replClassLoader) { - PluginContainer(env) + PluginContainer(env, resources.asJava) } // Max size of direct result. If task result is bigger than this, we use the block manager diff --git a/core/src/main/scala/org/apache/spark/internal/plugin/PluginContainer.scala b/core/src/main/scala/org/apache/spark/internal/plugin/PluginContainer.scala index fc7a9d85957c0..4eda4767094ad 100644 --- a/core/src/main/scala/org/apache/spark/internal/plugin/PluginContainer.scala +++ b/core/src/main/scala/org/apache/spark/internal/plugin/PluginContainer.scala @@ -24,6 +24,7 @@ import org.apache.spark.{SparkContext, SparkEnv} import org.apache.spark.api.plugin._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceInformation import org.apache.spark.util.Utils sealed abstract class PluginContainer { @@ -33,7 +34,10 @@ sealed abstract class PluginContainer { } -private class DriverPluginContainer(sc: SparkContext, plugins: Seq[SparkPlugin]) +private class DriverPluginContainer( + sc: SparkContext, + resources: java.util.Map[String, ResourceInformation], + plugins: Seq[SparkPlugin]) extends PluginContainer with Logging { private val driverPlugins: Seq[(String, DriverPlugin, PluginContextImpl)] = plugins.flatMap { p => @@ -41,7 +45,7 @@ private class DriverPluginContainer(sc: SparkContext, plugins: Seq[SparkPlugin]) if (driverPlugin != null) { val name = p.getClass().getName() val ctx = new PluginContextImpl(name, sc.env.rpcEnv, sc.env.metricsSystem, sc.conf, - sc.env.executorId) + sc.env.executorId, resources) val extraConf = driverPlugin.init(sc, ctx) if (extraConf != null) { @@ -83,7 +87,10 @@ private class DriverPluginContainer(sc: SparkContext, plugins: Seq[SparkPlugin]) } -private class ExecutorPluginContainer(env: SparkEnv, plugins: Seq[SparkPlugin]) +private class ExecutorPluginContainer( + env: SparkEnv, + resources: java.util.Map[String, ResourceInformation], + plugins: Seq[SparkPlugin]) extends PluginContainer with Logging { private val executorPlugins: Seq[(String, ExecutorPlugin)] = { @@ -100,7 +107,7 @@ private class ExecutorPluginContainer(env: SparkEnv, plugins: Seq[SparkPlugin]) .toMap .asJava val ctx = new PluginContextImpl(name, env.rpcEnv, env.metricsSystem, env.conf, - env.executorId) + env.executorId, resources) executorPlugin.init(ctx, extraConf) ctx.registerMetrics() @@ -133,17 +140,28 @@ object PluginContainer { val EXTRA_CONF_PREFIX = "spark.plugins.internal.conf." - def apply(sc: SparkContext): Option[PluginContainer] = PluginContainer(Left(sc)) + def apply( + sc: SparkContext, + resources: java.util.Map[String, ResourceInformation]): Option[PluginContainer] = { + PluginContainer(Left(sc), resources) + } + + def apply( + env: SparkEnv, + resources: java.util.Map[String, ResourceInformation]): Option[PluginContainer] = { + PluginContainer(Right(env), resources) + } - def apply(env: SparkEnv): Option[PluginContainer] = PluginContainer(Right(env)) - private def apply(ctx: Either[SparkContext, SparkEnv]): Option[PluginContainer] = { + private def apply( + ctx: Either[SparkContext, SparkEnv], + resources: java.util.Map[String, ResourceInformation]): Option[PluginContainer] = { val conf = ctx.fold(_.conf, _.conf) val plugins = Utils.loadExtensions(classOf[SparkPlugin], conf.get(PLUGINS).distinct, conf) if (plugins.nonEmpty) { ctx match { - case Left(sc) => Some(new DriverPluginContainer(sc, plugins)) - case Right(env) => Some(new ExecutorPluginContainer(env, plugins)) + case Left(sc) => Some(new DriverPluginContainer(sc, resources, plugins)) + case Right(env) => Some(new ExecutorPluginContainer(env, resources, plugins)) } } else { None diff --git a/core/src/main/scala/org/apache/spark/internal/plugin/PluginContextImpl.scala b/core/src/main/scala/org/apache/spark/internal/plugin/PluginContextImpl.scala index 279f3d388fb2e..ca9119409d4b9 100644 --- a/core/src/main/scala/org/apache/spark/internal/plugin/PluginContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/internal/plugin/PluginContextImpl.scala @@ -17,6 +17,8 @@ package org.apache.spark.internal.plugin +import java.util + import com.codahale.metrics.MetricRegistry import org.apache.spark.{SparkConf, SparkException} @@ -24,6 +26,7 @@ import org.apache.spark.api.plugin.PluginContext import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.metrics.source.Source +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.RpcUtils @@ -32,7 +35,8 @@ private class PluginContextImpl( rpcEnv: RpcEnv, metricsSystem: MetricsSystem, override val conf: SparkConf, - override val executorID: String) + override val executorID: String, + override val resources: util.Map[String, ResourceInformation]) extends PluginContext with Logging { override def hostname(): String = rpcEnv.address.hostPort.split(":")(0) diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index d2c0dc88d987e..42a5afe0b3f9d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -26,6 +26,7 @@ import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -57,8 +58,10 @@ private[spark] class LocalEndpoint( val localExecutorId = SparkContext.DRIVER_IDENTIFIER val localExecutorHostname = Utils.localCanonicalHostName() + // local mode doesn't support extra resources like GPUs right now private val executor = new Executor( - localExecutorId, localExecutorHostname, SparkEnv.get, userClassPath, isLocal = true) + localExecutorId, localExecutorHostname, SparkEnv.get, userClassPath, isLocal = true, + resources = Map.empty[String, ResourceInformation]) override def receive: PartialFunction[Any, Unit] = { case ReviveOffers => diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 7272a98c9770b..31049d104e63d 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -117,7 +117,8 @@ class ExecutorSuite extends SparkFunSuite var executor: Executor = null try { - executor = new Executor("id", "localhost", env, userClassPath = Nil, isLocal = true) + executor = new Executor("id", "localhost", env, userClassPath = Nil, isLocal = true, + resources = immutable.Map.empty[String, ResourceInformation]) // the task will be launched in a dedicated worker thread executor.launchTask(mockExecutorBackend, taskDescription) @@ -254,7 +255,8 @@ class ExecutorSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) val executor = - new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true) + new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true, + resources = immutable.Map.empty[String, ResourceInformation]) val executorClass = classOf[Executor] // Save all heartbeats sent into an ArrayBuffer for verification @@ -353,7 +355,8 @@ class ExecutorSuite extends SparkFunSuite val mockBackend = mock[ExecutorBackend] var executor: Executor = null try { - executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true) + executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true, + resources = immutable.Map.empty[String, ResourceInformation]) executor.launchTask(mockBackend, taskDescription) // Ensure that the executor's metricsPoller is polled so that values are recorded for @@ -466,7 +469,8 @@ class ExecutorSuite extends SparkFunSuite val timedOut = new AtomicBoolean(false) try { executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true, - uncaughtExceptionHandler = mockUncaughtExceptionHandler) + uncaughtExceptionHandler = mockUncaughtExceptionHandler, + resources = immutable.Map.empty[String, ResourceInformation]) // the task will be launched in a dedicated worker thread executor.launchTask(mockBackend, taskDescription) if (killTask) { diff --git a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala index b432253ad80de..ac57e29916b82 100644 --- a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala @@ -32,15 +32,20 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} import org.apache.spark._ +import org.apache.spark.TestUtils._ import org.apache.spark.api.plugin._ import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceUtils.GPU +import org.apache.spark.resource.TestResourceIDs.{DRIVER_GPU_ID, EXECUTOR_GPU_ID, WORKER_GPU_ID} import org.apache.spark.util.Utils class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with LocalSparkContext { override def afterEach(): Unit = { TestSparkPlugin.reset() + NonLocalModeSparkPlugin.reset() super.afterEach() } @@ -61,6 +66,7 @@ class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with Lo verify(TestSparkPlugin.executorPlugin).init(any(), meq(TestSparkPlugin.extraConf)) assert(TestSparkPlugin.executorContext != null) + assert(TestSparkPlugin.executorContext.resources.isEmpty) // One way messages don't block, so need to loop checking whether it arrives. TestSparkPlugin.executorContext.send("oneway") @@ -105,7 +111,8 @@ class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with Lo val conf = new SparkConf() val env = mock(classOf[SparkEnv]) when(env.conf).thenReturn(conf) - assert(PluginContainer(env) === None) + val container = PluginContainer(env, Map.empty[String, ResourceInformation].asJava) + assert(container === None) } test("merging of config options") { @@ -140,6 +147,53 @@ class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with Lo assert(children.length >= 3) } } + + test("plugin initialization in non-local mode with resources") { + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", + """{"name": "gpu","addresses":["5", "6"]}""") + + val workerScript = createTempScriptWithExpectedOutput(dir, "resourceDiscoveryScript", + """{"name": "gpu","addresses":["3", "4"]}""") + + val conf = new SparkConf() + .setAppName(getClass().getName()) + .set(SparkLauncher.SPARK_MASTER, "local-cluster[1,1,1024]") + .set(PLUGINS, Seq(classOf[NonLocalModeSparkPlugin].getName())) + .set(NonLocalModeSparkPlugin.TEST_PATH_CONF, dir.getAbsolutePath()) + .set(DRIVER_GPU_ID.amountConf, "2") + .set(DRIVER_GPU_ID.discoveryScriptConf, scriptPath) + .set(WORKER_GPU_ID.amountConf, "2") + .set(WORKER_GPU_ID.discoveryScriptConf, workerScript) + .set(EXECUTOR_GPU_ID.amountConf, "2") + sc = new SparkContext(conf) + + // Ensure all executors has started + TestUtils.waitUntilExecutorsUp(sc, 1, 10000) + + var children = Array.empty[File] + eventually(timeout(10.seconds), interval(100.millis)) { + children = dir.listFiles() + assert(children != null) + // we have 2 discovery scripts and then expect 1 driver and 1 executor file + assert(children.length >= 4) + } + val execFiles = + children.filter(_.getName.startsWith(NonLocalModeSparkPlugin.executorFileStr)) + assert(execFiles.size === 1) + val allLines = Files.readLines(execFiles(0), StandardCharsets.US_ASCII) + assert(allLines.size === 1) + val addrs = NonLocalModeSparkPlugin.extractGpuAddrs(allLines.get(0)) + assert(addrs.size === 2) + assert(addrs.sorted === Array("3", "4")) + + assert(NonLocalModeSparkPlugin.driverContext != null) + val driverResources = NonLocalModeSparkPlugin.driverContext.resources() + assert(driverResources.size === 1) + assert(driverResources.get(GPU).addresses === Array("5", "6")) + assert(driverResources.get(GPU).name === GPU) + } + } } class NonLocalModeSparkPlugin extends SparkPlugin { @@ -147,8 +201,10 @@ class NonLocalModeSparkPlugin extends SparkPlugin { override def driverPlugin(): DriverPlugin = { new DriverPlugin() { override def init(sc: SparkContext, ctx: PluginContext): JMap[String, String] = { - NonLocalModeSparkPlugin.writeFile(ctx.conf(), ctx.executorID()) - Map.empty.asJava + NonLocalModeSparkPlugin.writeDriverFile(NonLocalModeSparkPlugin.driverFileStr, ctx.conf(), + ctx.executorID()) + NonLocalModeSparkPlugin.driverContext = ctx + Map.empty[String, String].asJava } } } @@ -156,7 +212,8 @@ class NonLocalModeSparkPlugin extends SparkPlugin { override def executorPlugin(): ExecutorPlugin = { new ExecutorPlugin() { override def init(ctx: PluginContext, extraConf: JMap[String, String]): Unit = { - NonLocalModeSparkPlugin.writeFile(ctx.conf(), ctx.executorID()) + NonLocalModeSparkPlugin.writeFile(NonLocalModeSparkPlugin.executorFileStr, ctx.conf(), + ctx.executorID(), ctx.resources().asScala.toMap) } } } @@ -164,10 +221,50 @@ class NonLocalModeSparkPlugin extends SparkPlugin { object NonLocalModeSparkPlugin { val TEST_PATH_CONF = "spark.nonLocalPlugin.path" + var driverContext: PluginContext = _ + val executorFileStr = "EXECUTOR_FILE_" + val driverFileStr = "DRIVER_FILE_" + + private def createFileStringWithGpuAddrs( + id: String, + resources: Map[String, ResourceInformation]): String = { + // try to keep this simple and only write the gpus addresses, if we add more resources need to + // make more complex + val resourcesString = resources.filterKeys(_.equals(GPU)).map { + case (_, ri) => + s"${ri.addresses.mkString(",")}" + }.mkString(",") + s"$id&$resourcesString" + } - def writeFile(conf: SparkConf, id: String): Unit = { + def extractGpuAddrs(str: String): Array[String] = { + val idAndAddrs = str.split("&") + if (idAndAddrs.size > 1) { + idAndAddrs(1).split(",") + } else { + Array.empty[String] + } + } + + def writeDriverFile( + filePrefix: String, + conf: SparkConf, + id: String): Unit = { + writeFile(filePrefix, conf, id, Map.empty) + } + + def writeFile( + filePrefix: String, + conf: SparkConf, + id: String, + resources: Map[String, ResourceInformation]): Unit = { val path = conf.get(TEST_PATH_CONF) - Files.write(id, new File(path, id), StandardCharsets.UTF_8) + val strToWrite = createFileStringWithGpuAddrs(id, resources) + Files.write(strToWrite, new File(path, s"$filePrefix$id"), StandardCharsets.UTF_8) + } + + def reset(): Unit = { + driverContext = null } } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 213d33cb4f0f2..47243e83d1335 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -30,6 +30,7 @@ import org.apache.spark.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config.EXECUTOR_ID +import org.apache.spark.resource.ResourceInformation import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.mesos.MesosSchedulerUtils import org.apache.spark.util.Utils @@ -82,7 +83,8 @@ private[spark] class MesosExecutorBackend executor = new Executor( executorId, slaveInfo.getHostname, - env) + env, + resources = Map.empty[String, ResourceInformation]) } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo): Unit = { From 21bc0474bbb16c7648aed40f25a2945d98d2a167 Mon Sep 17 00:00:00 2001 From: "zebingl@fb.com" Date: Fri, 31 Jan 2020 08:49:34 -0600 Subject: [PATCH 1268/1387] [SPARK-30511][SPARK-28403][CORE] Don't treat failed/killed speculative tasks as pending in ExecutorAllocationManager MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Currently, when speculative tasks fail/get killed, they are still considered as pending and count towards the calculation of number of needed executors. To be more accurate: `stageAttemptToNumSpeculativeTasks(stageAttempt)` is incremented on onSpeculativeTaskSubmitted, but never decremented.  `stageAttemptToNumSpeculativeTasks -= stageAttempt` is performed on stage completion. **This means Spark is marking ended speculative tasks as pending, which leads to Spark to hold more executors that it actually needs!** This PR fixes this issue by updating `stageAttemptToSpeculativeTaskIndices` and `stageAttemptToNumSpeculativeTasks` on speculative tasks completion. This PR also addresses some other minor issues: scheduler behavior after receiving an intentionally killed task event; try to address [SPARK-28403](https://issues.apache.org/jira/browse/SPARK-28403). ### Why are the changes needed? This has caused resource wastage in our production with speculation enabled. With aggressive speculation, we found data skewed jobs can hold hundreds of idle executors with less than 10 tasks running. An easy repro of the issue (`--conf spark.speculation=true --conf spark.executor.cores=4 --conf spark.dynamicAllocation.maxExecutors=1000` in cluster mode): ``` val n = 4000 val someRDD = sc.parallelize(1 to n, n) someRDD.mapPartitionsWithIndex( (index: Int, it: Iterator[Int]) => { if (index < 300 && index >= 150) { Thread.sleep(index * 1000) // Fake running tasks } else if (index == 300) { Thread.sleep(1000 * 1000) // Fake long running tasks } it.toList.map(x => index + ", " + x).iterator }).collect ``` You will see when running the last task, we would be hold 38 executors (see below), which is exactly (152 + 3) / 4 = 38. ![image](https://user-images.githubusercontent.com/9404831/72469112-9a7fac00-3793-11ea-8f50-74d0ab7325a4.png) ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added a comprehensive unit test. Test with the above repro shows that we are holding 2 executors at the end ![image](https://user-images.githubusercontent.com/9404831/72469177-bbe09800-3793-11ea-850f-4a2c67142899.png) Closes #27223 from linzebing/speculation_fix. Authored-by: zebingl@fb.com Signed-off-by: Thomas Graves --- .../spark/ExecutorAllocationManager.scala | 61 ++++---- .../ExecutorAllocationManagerSuite.scala | 135 ++++++++++++++++++ 2 files changed, 172 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index bff854a003572..677386cc7a572 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -263,9 +263,16 @@ private[spark] class ExecutorAllocationManager( */ private def maxNumExecutorsNeeded(): Int = { val numRunningOrPendingTasks = listener.totalPendingTasks + listener.totalRunningTasks - math.ceil(numRunningOrPendingTasks * executorAllocationRatio / - tasksPerExecutorForFullParallelism) - .toInt + val maxNeeded = math.ceil(numRunningOrPendingTasks * executorAllocationRatio / + tasksPerExecutorForFullParallelism).toInt + if (tasksPerExecutorForFullParallelism > 1 && maxNeeded == 1 && + listener.pendingSpeculativeTasks > 0) { + // If we have pending speculative tasks and only need a single executor, allocate one more + // to satisfy the locality requirements of speculation + maxNeeded + 1 + } else { + maxNeeded + } } private def totalRunningTasks(): Int = synchronized { @@ -377,14 +384,8 @@ private[spark] class ExecutorAllocationManager( // If our target has not changed, do not send a message // to the cluster manager and reset our exponential growth if (delta == 0) { - // Check if there is any speculative jobs pending - if (listener.pendingTasks == 0 && listener.pendingSpeculativeTasks > 0) { - numExecutorsTarget = - math.max(math.min(maxNumExecutorsNeeded + 1, maxNumExecutors), minNumExecutors) - } else { - numExecutorsToAdd = 1 - return 0 - } + numExecutorsToAdd = 1 + return 0 } val addRequestAcknowledged = try { @@ -512,7 +513,7 @@ private[spark] class ExecutorAllocationManager( // Should be 0 when no stages are active. private val stageAttemptToNumRunningTask = new mutable.HashMap[StageAttempt, Int] private val stageAttemptToTaskIndices = new mutable.HashMap[StageAttempt, mutable.HashSet[Int]] - // Number of speculative tasks to be scheduled in each stageAttempt + // Number of speculative tasks pending/running in each stageAttempt private val stageAttemptToNumSpeculativeTasks = new mutable.HashMap[StageAttempt, Int] // The speculative tasks started in each stageAttempt private val stageAttemptToSpeculativeTaskIndices = @@ -614,18 +615,30 @@ private[spark] class ExecutorAllocationManager( stageAttemptToNumRunningTask -= stageAttempt } } - // If the task failed, we expect it to be resubmitted later. To ensure we have - // enough resources to run the resubmitted task, we need to mark the scheduler - // as backlogged again if it's not already marked as such (SPARK-8366) - if (taskEnd.reason != Success) { - if (totalPendingTasks() == 0) { - allocationManager.onSchedulerBacklogged() - } - if (taskEnd.taskInfo.speculative) { - stageAttemptToSpeculativeTaskIndices.get(stageAttempt).foreach {_.remove(taskIndex)} - } else { - stageAttemptToTaskIndices.get(stageAttempt).foreach {_.remove(taskIndex)} - } + + if (taskEnd.taskInfo.speculative) { + stageAttemptToSpeculativeTaskIndices.get(stageAttempt).foreach {_.remove{taskIndex}} + stageAttemptToNumSpeculativeTasks(stageAttempt) -= 1 + } + + taskEnd.reason match { + case Success | _: TaskKilled => + case _ => + if (totalPendingTasks() == 0) { + // If the task failed (not intentionally killed), we expect it to be resubmitted + // later. To ensure we have enough resources to run the resubmitted task, we need to + // mark the scheduler as backlogged again if it's not already marked as such + // (SPARK-8366) + allocationManager.onSchedulerBacklogged() + } + if (!taskEnd.taskInfo.speculative) { + // If a non-speculative task is intentionally killed, it means the speculative task + // has succeeded, and no further task of this task index will be resubmitted. In this + // case, the task index is completed and we shouldn't remove it from + // stageAttemptToTaskIndices. Otherwise, we will have a pending non-speculative task + // for the task index (SPARK-30511) + stageAttemptToTaskIndices.get(stageAttempt).foreach {_.remove(taskIndex)} + } } } } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 99f3e3b2e478d..8d958494d52be 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -264,6 +264,141 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(numExecutorsToAdd(manager) === 1) } + test("SPARK-30511 remove executors when speculative tasks end") { + val clock = new ManualClock() + val stage = createStageInfo(0, 40) + val manager = createManager(createConf(0, 10, 0).set(config.EXECUTOR_CORES, 4), clock = clock) + + post(SparkListenerStageSubmitted(stage)) + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 2) + assert(addExecutors(manager) === 4) + assert(addExecutors(manager) === 3) + + (0 to 9).foreach(execId => onExecutorAdded(manager, execId.toString)) + (0 to 39).map { i => createTaskInfo(i, i, executorId = s"${i / 4}")}.foreach { + info => post(SparkListenerTaskStart(0, 0, info)) + } + assert(numExecutorsTarget(manager) === 10) + assert(maxNumExecutorsNeeded(manager) == 10) + + // 30 tasks (0 - 29) finished + (0 to 29).map { i => createTaskInfo(i, i, executorId = s"${i / 4}")}.foreach { + info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) } + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager) === 3) + assert(maxNumExecutorsNeeded(manager) == 3) + (0 to 6).foreach { i => assert(removeExecutor(manager, i.toString))} + (0 to 6).foreach { i => onExecutorRemoved(manager, i.toString)} + + // 10 speculative tasks (30 - 39) launch for the remaining tasks + (30 to 39).foreach { _ => post(SparkListenerSpeculativeTaskSubmitted(0))} + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 1) + assert(numExecutorsTarget(manager) == 5) + assert(maxNumExecutorsNeeded(manager) == 5) + (10 to 12).foreach(execId => onExecutorAdded(manager, execId.toString)) + (40 to 49).map { i => + createTaskInfo(taskId = i, taskIndex = i - 10, executorId = s"${i / 4}", speculative = true)} + .foreach { info => post(SparkListenerTaskStart(0, 0, info))} + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager) == 5) // At this point, we still have 6 executors running + assert(maxNumExecutorsNeeded(manager) == 5) + + // 6 speculative tasks (40 - 45) finish before the original tasks, with 4 speculative remaining + (40 to 45).map { i => + createTaskInfo(taskId = i, taskIndex = i - 10, executorId = s"${i / 4}", speculative = true)} + .foreach { + info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null))} + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager) === 4) + assert(maxNumExecutorsNeeded(manager) == 4) + assert(removeExecutor(manager, "10")) + onExecutorRemoved(manager, "10") + // At this point, we still have 5 executors running: ["7", "8", "9", "11", "12"] + + // 6 original tasks (30 - 35) are intentionally killed + (30 to 35).map { i => + createTaskInfo(i, i, executorId = s"${i / 4}")} + .foreach { info => post( + SparkListenerTaskEnd(0, 0, null, TaskKilled("test"), info, new ExecutorMetrics, null))} + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager) === 2) + assert(maxNumExecutorsNeeded(manager) == 2) + (7 to 8).foreach { i => assert(removeExecutor(manager, i.toString))} + (7 to 8).foreach { i => onExecutorRemoved(manager, i.toString)} + // At this point, we still have 3 executors running: ["9", "11", "12"] + + // Task 36 finishes before the speculative task 46, task 46 killed + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(36, 36, executorId = "9"), new ExecutorMetrics, null)) + post(SparkListenerTaskEnd(0, 0, null, TaskKilled("test"), + createTaskInfo(46, 36, executorId = "11", speculative = true), new ExecutorMetrics, null)) + + // We should have 3 original tasks (index 37, 38, 39) running, with corresponding 3 speculative + // tasks running. Target lowers to 2, but still hold 3 executors ["9", "11", "12"] + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager) === 2) + assert(maxNumExecutorsNeeded(manager) == 2) + // At this point, we still have 3 executors running: ["9", "11", "12"] + + // Task 37 and 47 succeed at the same time + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(37, 37, executorId = "9"), new ExecutorMetrics, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(47, 37, executorId = "11", speculative = true), new ExecutorMetrics, null)) + + // We should have 2 original tasks (index 38, 39) running, with corresponding 2 speculative + // tasks running + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager) === 1) + assert(maxNumExecutorsNeeded(manager) == 1) + assert(removeExecutor(manager, "11")) + onExecutorRemoved(manager, "11") + // At this point, we still have 2 executors running: ["9", "12"] + + // Task 38 fails and task 49 fails, new speculative task 50 is submitted to speculate on task 39 + post(SparkListenerTaskEnd(0, 0, null, UnknownReason, + createTaskInfo(38, 38, executorId = "9"), new ExecutorMetrics, null)) + post(SparkListenerTaskEnd(0, 0, null, UnknownReason, + createTaskInfo(49, 39, executorId = "12", speculative = true), new ExecutorMetrics, null)) + post(SparkListenerSpeculativeTaskSubmitted(0)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + // maxNeeded = 1, allocate one more to satisfy speculation locality requirement + assert(numExecutorsTarget(manager) === 2) + assert(maxNumExecutorsNeeded(manager) == 2) + post(SparkListenerTaskStart(0, 0, + createTaskInfo(50, 39, executorId = "12", speculative = true))) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager) === 1) + assert(maxNumExecutorsNeeded(manager) == 1) + + // Task 39 and 48 succeed, task 50 killed + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(39, 39, executorId = "9"), new ExecutorMetrics, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, + createTaskInfo(48, 38, executorId = "12", speculative = true), new ExecutorMetrics, null)) + post(SparkListenerTaskEnd(0, 0, null, TaskKilled("test"), + createTaskInfo(50, 39, executorId = "12", speculative = true), new ExecutorMetrics, null)) + post(SparkListenerStageCompleted(stage)) + clock.advance(1000) + manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) + assert(numExecutorsTarget(manager) === 0) + assert(maxNumExecutorsNeeded(manager) == 0) + assert(removeExecutor(manager, "9")) + onExecutorRemoved(manager, "9") + assert(removeExecutor(manager, "12")) + onExecutorRemoved(manager, "12") + } + test("properly handle task end events from completed stages") { val manager = createManager(createConf(0, 10, 0)) From 5ccbb38a71890b114c707279e7395d1f6284ebfd Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Sat, 1 Feb 2020 01:03:00 +0800 Subject: [PATCH 1269/1387] [SPARK-29938][SQL][FOLLOW-UP] Improve AlterTableAddPartitionCommand All credit to Ngone51, Closes #27293. ### What changes were proposed in this pull request? This PR improves `AlterTableAddPartitionCommand` by: 1. adds an internal config for partitions batch size to avoid hard code 2. reuse `InMemoryFileIndex.bulkListLeafFiles` to perform parallel file listing to improve code reuse ### Why are the changes needed? Improve code quality. ### Does this PR introduce any user-facing change? Yes. We renamed `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `spark.sql.parallelFileListingInCommands.enabled` as a side effect of this change. ### How was this patch tested? Pass Jenkins. Closes #27413 from xuanyuanking/SPARK-29938. Lead-authored-by: yi.wu Co-authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 10 +++ .../command/AnalyzePartitionCommand.scala | 2 +- .../sql/execution/command/CommandUtils.scala | 64 ++++++++++++++----- .../spark/sql/execution/command/ddl.scala | 15 ++--- 4 files changed, 63 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e13d65bf81821..04572c38be8dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2148,6 +2148,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ADD_PARTITION_BATCH_SIZE = + buildConf("spark.sql.addPartitionInBatch.size") + .internal() + .doc("The number of partitions to be handled in one turn when use " + + "`AlterTableAddPartitionCommand` to add partitions into table. The smaller " + + "batch size is, the less memory is required for the real handler, e.g. Hive Metastore.") + .intConf + .checkValue(_ > 0, "The value of spark.sql.addPartitionInBatch.size must be positive") + .createWithDefault(100) + /** * Holds information about keys that have been deprecated. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala index 9ddc470fbd33c..c5db5ad1157fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala @@ -107,7 +107,7 @@ case class AnalyzePartitionCommand( // Update the metastore if newly computed statistics are different from those // recorded in the metastore. val newPartitions = partitions.flatMap { p => - val newTotalSize = CommandUtils.calculateLocationSize( + val newTotalSize = CommandUtils.calculateSingleLocationSize( sessionState, tableMeta.identifier, p.storage.locationUri) val newRowCount = rowCounts.get(p.spec) val newStats = CommandUtils.compareAndGetNewStats(p.stats, newTotalSize, newRowCount) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index 5cb77069b5514..b229b238238fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -37,6 +37,17 @@ import org.apache.spark.sql.execution.datasources.{DataSourceUtils, InMemoryFile import org.apache.spark.sql.internal.{SessionState, SQLConf} import org.apache.spark.sql.types._ +/** + * For the purpose of calculating total directory sizes, use this filter to + * ignore some irrelevant files. + * @param stagingDir hive staging dir + */ +class PathFilterIgnoreNonData(stagingDir: String) extends PathFilter with Serializable { + override def accept(path: Path): Boolean = { + val fileName = path.getName + !fileName.startsWith(stagingDir) && DataSourceUtils.isDataFile(fileName) + } +} object CommandUtils extends Logging { @@ -60,32 +71,21 @@ object CommandUtils extends Logging { val sessionState = spark.sessionState val startTime = System.nanoTime() val totalSize = if (catalogTable.partitionColumnNames.isEmpty) { - calculateLocationSize(sessionState, catalogTable.identifier, catalogTable.storage.locationUri) + calculateSingleLocationSize(sessionState, catalogTable.identifier, + catalogTable.storage.locationUri) } else { // Calculate table size as a sum of the visible partitions. See SPARK-21079 val partitions = sessionState.catalog.listPartitions(catalogTable.identifier) logInfo(s"Starting to calculate sizes for ${partitions.length} partitions.") - if (spark.sessionState.conf.parallelFileListingInStatsComputation) { - val paths = partitions.map(x => new Path(x.storage.locationUri.get)) - val stagingDir = sessionState.conf.getConfString("hive.exec.stagingdir", ".hive-staging") - val pathFilter = new PathFilter with Serializable { - override def accept(path: Path): Boolean = isDataPath(path, stagingDir) - } - val fileStatusSeq = InMemoryFileIndex.bulkListLeafFiles( - paths, sessionState.newHadoopConf(), pathFilter, spark, areRootPaths = true) - fileStatusSeq.flatMap(_._2.map(_.getLen)).sum - } else { - partitions.map { p => - calculateLocationSize(sessionState, catalogTable.identifier, p.storage.locationUri) - }.sum - } + val paths = partitions.map(_.storage.locationUri) + calculateTotalLocationSize(spark, catalogTable.identifier, paths) } logInfo(s"It took ${(System.nanoTime() - startTime) / (1000 * 1000)} ms to calculate" + s" the total size for table ${catalogTable.identifier}.") totalSize } - def calculateLocationSize( + def calculateSingleLocationSize( sessionState: SessionState, identifier: TableIdentifier, locationUri: Option[URI]): Long = { @@ -137,6 +137,38 @@ object CommandUtils extends Logging { size } + def calculateTotalLocationSize( + sparkSession: SparkSession, + tid: TableIdentifier, + paths: Seq[Option[URI]]): Long = { + if (sparkSession.sessionState.conf.parallelFileListingInStatsComputation) { + calculateLocationSizeParallel(sparkSession, paths.map(_.map(new Path(_)))) + } else { + paths.map(p => calculateSingleLocationSize(sparkSession.sessionState, tid, p)).sum + } + } + + /** + * Launch a Job to list all leaf files in `paths` and compute the total size + * for each path. + * @param sparkSession the [[SparkSession]] + * @param paths the Seq of [[Option[Path]]]s + * @return total size of all partitions + */ + def calculateLocationSizeParallel( + sparkSession: SparkSession, + paths: Seq[Option[Path]]): Long = { + val stagingDir = sparkSession.sessionState.conf + .getConfString("hive.exec.stagingdir", ".hive-staging") + val filter = new PathFilterIgnoreNonData(stagingDir) + val sizes = InMemoryFileIndex.bulkListLeafFiles(paths.flatten, + sparkSession.sessionState.newHadoopConf(), filter, sparkSession, areRootPaths = true).map { + case (_, files) => files.map(_.getLen).sum + } + // the size is 0 where paths(i) is not defined and sizes(i) where it is defined + paths.zipWithIndex.filter(_._1.isDefined).map(i => sizes(i._2)).sum + } + def compareAndGetNewStats( oldStats: Option[CatalogStatistics], newTotalSize: BigInt, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 9c0053c6ae0ff..f41c4eca203af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -42,7 +42,7 @@ import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter -import org.apache.spark.sql.internal.HiveSerDe +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types._ import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} @@ -479,22 +479,15 @@ case class AlterTableAddPartitionCommand( // Hive metastore may not have enough memory to handle millions of partitions in single RPC. // Also the request to metastore times out when adding lot of partitions in one shot. // we should split them into smaller batches - val batchSize = 100 + val batchSize = conf.getConf(SQLConf.ADD_PARTITION_BATCH_SIZE) parts.toIterator.grouped(batchSize).foreach { batch => catalog.createPartitions(table.identifier, batch, ignoreIfExists = ifNotExists) } if (table.stats.nonEmpty) { if (sparkSession.sessionState.conf.autoSizeUpdateEnabled) { - def calculatePartSize(part: CatalogTablePartition) = CommandUtils.calculateLocationSize( - sparkSession.sessionState, table.identifier, part.storage.locationUri) - val threshold = sparkSession.sparkContext.conf.get(RDD_PARALLEL_LISTING_THRESHOLD) - val partSizes = if (parts.length > threshold) { - ThreadUtils.parmap(parts, "gatheringNewPartitionStats", 8)(calculatePartSize) - } else { - parts.map(calculatePartSize) - } - val addedSize = partSizes.sum + val addedSize = CommandUtils.calculateTotalLocationSize(sparkSession, table.identifier, + parts.map(_.storage.locationUri)) if (addedSize > 0) { val newStats = CatalogStatistics(sizeInBytes = table.stats.get.sizeInBytes + addedSize) catalog.alterTableStats(table.identifier, Some(newStats)) From 481e5211d237173ea0fb7c0b292eb7abd2b8a3fe Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 31 Jan 2020 09:26:03 -0800 Subject: [PATCH 1270/1387] [SPARK-30657][SPARK-30658][SS] Fixed two bugs in streaming limits This PR solves two bugs related to streaming limits **Bug 1 (SPARK-30658)**: Limit before a streaming aggregate (i.e. `df.limit(5).groupBy().count()`) in complete mode was not being planned as a stateful streaming limit. The planner rule planned a logical limit with a stateful streaming limit plan only if the query is in append mode. As a result, instead of allowing max 5 rows across batches, the planned streaming query was allowing 5 rows in every batch thus producing incorrect results. **Solution**: Change the planner rule to plan the logical limit with a streaming limit plan even when the query is in complete mode if the logical limit has no stateful operator before it. **Bug 2 (SPARK-30657)**: `LocalLimitExec` does not consume the iterator of the child plan. So if there is a limit after a stateful operator like streaming dedup in append mode (e.g. `df.dropDuplicates().limit(5)`), the state changes of streaming duplicate may not be committed (most stateful ops commit state changes only after the generated iterator is fully consumed). **Solution**: Change the planner rule to always use a new `StreamingLocalLimitExec` which always fully consumes the iterator. This is the safest thing to do. However, this will introduce a performance regression as consuming the iterator is extra work. To minimize this performance impact, add an additional post-planner optimization rule to replace `StreamingLocalLimitExec` with `LocalLimitExec` when there is no stateful operator before the limit that could be affected by it. No Updated incorrect unit tests and added new ones Closes #27373 from tdas/SPARK-30657. Authored-by: Tathagata Das Signed-off-by: Shixiong Zhu --- .../spark/sql/execution/SparkStrategies.scala | 38 ++++-- .../streaming/IncrementalExecution.scala | 34 +++++- ...lLimitExec.scala => streamingLimits.scala} | 55 +++++++-- .../spark/sql/streaming/StreamSuite.scala | 112 +++++++++++++++++- 4 files changed, 211 insertions(+), 28 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/{StreamingGlobalLimitExec.scala => streamingLimits.scala} (68%) 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 00ad4e0fe0c11..bd2684d92a1d2 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 @@ -451,21 +451,35 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * Used to plan the streaming global limit operator for streams in append mode. * We need to check for either a direct Limit or a Limit wrapped in a ReturnAnswer operator, * following the example of the SpecialLimits Strategy above. - * Streams with limit in Append mode use the stateful StreamingGlobalLimitExec. - * Streams with limit in Complete mode use the stateless CollectLimitExec operator. - * Limit is unsupported for streams in Update mode. */ case class StreamingGlobalLimitStrategy(outputMode: OutputMode) extends Strategy { - override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ReturnAnswer(rootPlan) => rootPlan match { - case Limit(IntegerLiteral(limit), child) - if plan.isStreaming && outputMode == InternalOutputModes.Append => - StreamingGlobalLimitExec(limit, LocalLimitExec(limit, planLater(child))) :: Nil - case _ => Nil + + private def generatesStreamingAppends(plan: LogicalPlan): Boolean = { + + /** Ensures that this plan does not have a streaming aggregate in it. */ + def hasNoStreamingAgg: Boolean = { + plan.collectFirst { case a: Aggregate if a.isStreaming => a }.isEmpty } - case Limit(IntegerLiteral(limit), child) - if plan.isStreaming && outputMode == InternalOutputModes.Append => - StreamingGlobalLimitExec(limit, LocalLimitExec(limit, planLater(child))) :: Nil + + // The following cases of limits on a streaming plan has to be executed with a stateful + // streaming plan. + // 1. When the query is in append mode (that is, all logical plan operate on appended data). + // 2. When the plan does not contain any streaming aggregate (that is, plan has only + // operators that operate on appended data). This must be executed with a stateful + // streaming plan even if the query is in complete mode because of a later streaming + // aggregation (e.g., `streamingDf.limit(5).groupBy().count()`). + plan.isStreaming && ( + outputMode == InternalOutputModes.Append || + outputMode == InternalOutputModes.Complete && hasNoStreamingAgg) + } + + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ReturnAnswer(Limit(IntegerLiteral(limit), child)) if generatesStreamingAppends(child) => + StreamingGlobalLimitExec(limit, StreamingLocalLimitExec(limit, planLater(child))) :: Nil + + case Limit(IntegerLiteral(limit), child) if generatesStreamingAppends(child) => + StreamingGlobalLimitExec(limit, StreamingLocalLimitExec(limit, planLater(child))) :: Nil + case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index bf80a0b1c167a..09ae7692ec518 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{CurrentBatchTimestamp, Express import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, HashPartitioning, SinglePartition} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} +import org.apache.spark.sql.execution.{LeafExecNode, LocalLimitExec, QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode @@ -105,6 +105,32 @@ class IncrementalExecution( /** Locates save/restore pairs surrounding aggregation. */ val state = new Rule[SparkPlan] { + /** + * Ensures that this plan DOES NOT have any stateful operation in it whose pipelined execution + * depends on this plan. In other words, this function returns true if this plan does + * have a narrow dependency on a stateful subplan. + */ + private def hasNoStatefulOp(plan: SparkPlan): Boolean = { + var statefulOpFound = false + + def findStatefulOp(planToCheck: SparkPlan): Unit = { + planToCheck match { + case s: StatefulOperator => + statefulOpFound = true + + case e: ShuffleExchangeExec => + // Don't search recursively any further as any child stateful operator as we + // are only looking for stateful subplans that this plan has narrow dependencies on. + + case p: SparkPlan => + p.children.foreach(findStatefulOp) + } + } + + findStatefulOp(plan) + !statefulOpFound + } + override def apply(plan: SparkPlan): SparkPlan = plan transform { case StateStoreSaveExec(keys, None, None, None, stateFormatVersion, UnaryExecNode(agg, @@ -149,6 +175,12 @@ class IncrementalExecution( l.copy( stateInfo = Some(nextStatefulOperationStateInfo), outputMode = Some(outputMode)) + + case StreamingLocalLimitExec(limit, child) if hasNoStatefulOp(child) => + // Optimize limit execution by replacing StreamingLocalLimitExec (consumes the iterator + // completely) to LocalLimitExec (does not consume the iterator) when the child plan has + // no stateful operator (i.e., consuming the iterator is not needed). + LocalLimitExec(limit, child) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingGlobalLimitExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala similarity index 68% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingGlobalLimitExec.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala index bf4af60c8cf03..b19540253d7eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingGlobalLimitExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala @@ -20,21 +20,21 @@ import java.util.concurrent.TimeUnit.NANOSECONDS import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.catalyst.expressions.UnsafeProjection -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericInternalRow, SortOrder, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, Distribution, Partitioning} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes -import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{LimitExec, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.streaming.state.StateStoreOps import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{LongType, NullType, StructField, StructType} -import org.apache.spark.util.CompletionIterator +import org.apache.spark.util.{CompletionIterator, NextIterator} /** * A physical operator for executing a streaming limit, which makes sure no more than streamLimit - * rows are returned. This operator is meant for streams in Append mode only. + * rows are returned. This physical operator is only meant for logical limit operations that + * will get a input stream of rows that are effectively appends. For example, + * - limit on any query in append mode + * - limit before the aggregation in a streaming aggregation query complete mode */ case class StreamingGlobalLimitExec( streamLimit: Long, @@ -49,9 +49,6 @@ case class StreamingGlobalLimitExec( override protected def doExecute(): RDD[InternalRow] = { metrics // force lazy init at driver - assert(outputMode.isDefined && outputMode.get == InternalOutputModes.Append, - "StreamingGlobalLimitExec is only valid for streams in Append output mode") - child.execute().mapPartitionsWithStateStore( getStateInfo, keySchema, @@ -100,3 +97,41 @@ case class StreamingGlobalLimitExec( UnsafeProjection.create(valueSchema)(new GenericInternalRow(Array[Any](value))) } } + + +/** + * A physical operator for executing limits locally on each partition. The main difference from + * LocalLimitExec is that this will fully consume `child` plan's iterators to ensure that any + * stateful operation within `child` commits all the state changes (many stateful operations + * commit state changes only after the iterator is consumed). + */ +case class StreamingLocalLimitExec(limit: Int, child: SparkPlan) + extends LimitExec { + + override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter => + + var generatedCount = 0 + + new NextIterator[InternalRow]() { + override protected def getNext(): InternalRow = { + if (generatedCount < limit && iter.hasNext) { + generatedCount += 1 + iter.next() + } else { + finished = true + null + } + } + + override protected def close(): Unit = { + while (iter.hasNext) iter.next() // consume the iterator completely + } + } + } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def output: Seq[Attribute] = child.output +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index bf80962d62c65..b6618826487c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.SimpleMode +import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{ContinuousMemoryStream, MemorySink} @@ -976,24 +976,50 @@ class StreamSuite extends StreamTest { CheckAnswer(1 to 3: _*)) } - test("streaming limit in complete mode") { + test("SPARK-30658: streaming limit before agg in complete mode") { val inputData = MemoryStream[Int] val limited = inputData.toDF().limit(5).groupBy("value").count() testStream(limited, OutputMode.Complete())( AddData(inputData, 1 to 3: _*), CheckAnswer(Row(1, 1), Row(2, 1), Row(3, 1)), AddData(inputData, 1 to 9: _*), - CheckAnswer(Row(1, 2), Row(2, 2), Row(3, 2), Row(4, 1), Row(5, 1))) + CheckAnswer(Row(1, 2), Row(2, 2), Row(3, 1))) } - test("streaming limits in complete mode") { + test("SPARK-30658: streaming limits before and after agg in complete mode " + + "(after limit < before limit)") { val inputData = MemoryStream[Int] val limited = inputData.toDF().limit(4).groupBy("value").count().orderBy("value").limit(3) testStream(limited, OutputMode.Complete())( + StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "1")), AddData(inputData, 1 to 9: _*), + // only 1 to 4 should be allowed to aggregate, and counts for only 1 to 3 should be output CheckAnswer(Row(1, 1), Row(2, 1), Row(3, 1)), AddData(inputData, 2 to 6: _*), - CheckAnswer(Row(1, 1), Row(2, 2), Row(3, 2))) + // None of the new values should be allowed to aggregate, same 3 counts should be output + CheckAnswer(Row(1, 1), Row(2, 1), Row(3, 1))) + } + + test("SPARK-30658: streaming limits before and after agg in complete mode " + + "(before limit < after limit)") { + val inputData = MemoryStream[Int] + val limited = inputData.toDF().limit(2).groupBy("value").count().orderBy("value").limit(3) + testStream(limited, OutputMode.Complete())( + StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "1")), + AddData(inputData, 1 to 9: _*), + CheckAnswer(Row(1, 1), Row(2, 1)), + AddData(inputData, 2 to 6: _*), + CheckAnswer(Row(1, 1), Row(2, 1))) + } + + test("SPARK-30657: streaming limit after streaming dedup in append mode") { + val inputData = MemoryStream[Int] + val limited = inputData.toDF().dropDuplicates().limit(1) + testStream(limited)( + AddData(inputData, 1, 2), + CheckAnswer(Row(1)), + AddData(inputData, 3, 4), + CheckAnswer(Row(1))) } test("streaming limit in update mode") { @@ -1034,6 +1060,82 @@ class StreamSuite extends StreamTest { false)) } + test("SPARK-30657: streaming limit should not apply on limits on state subplans") { + val streanData = MemoryStream[Int] + val streamingDF = streanData.toDF().toDF("value") + val staticDF = spark.createDataset(Seq(1)).toDF("value").orderBy("value") + testStream(streamingDF.join(staticDF.limit(1), "value"))( + AddData(streanData, 1, 2, 3), + CheckAnswer(Row(1)), + AddData(streanData, 1, 3, 5), + CheckAnswer(Row(1), Row(1))) + } + + test("SPARK-30657: streaming limit optimization from StreamingLocalLimitExec to LocalLimitExec") { + val inputData = MemoryStream[Int] + val inputDF = inputData.toDF() + + /** Verify whether the local limit in the plan is a streaming limit or is a simple */ + def verifyLocalLimit( + df: DataFrame, + expectStreamingLimit: Boolean, + outputMode: OutputMode = OutputMode.Append): Unit = { + + var execPlan: SparkPlan = null + testStream(df, outputMode)( + AddData(inputData, 1), + AssertOnQuery { q => + q.processAllAvailable() + execPlan = q.lastExecution.executedPlan + true + } + ) + require(execPlan != null) + + val localLimits = execPlan.collect { + case l: LocalLimitExec => l + case l: StreamingLocalLimitExec => l + } + + require( + localLimits.size == 1, + s"Cant verify local limit optimization with this plan:\n$execPlan") + + if (expectStreamingLimit) { + assert( + localLimits.head.isInstanceOf[StreamingLocalLimitExec], + s"Local limit was not StreamingLocalLimitExec:\n$execPlan") + } else { + assert( + localLimits.head.isInstanceOf[LocalLimitExec], + s"Local limit was not LocalLimitExec:\n$execPlan") + } + } + + // Should not be optimized, so StreamingLocalLimitExec should be present + verifyLocalLimit(inputDF.dropDuplicates().limit(1), expectStreamingLimit = true) + + // Should be optimized from StreamingLocalLimitExec to LocalLimitExec + verifyLocalLimit(inputDF.limit(1), expectStreamingLimit = false) + verifyLocalLimit( + inputDF.limit(1).groupBy().count(), + expectStreamingLimit = false, + outputMode = OutputMode.Complete()) + + // Should be optimized as repartition is sufficient to ensure that the iterators of + // StreamingDeduplicationExec should be consumed completely by the repartition exchange. + verifyLocalLimit(inputDF.dropDuplicates().repartition(1).limit(1), expectStreamingLimit = false) + + // Should be LocalLimitExec in the first place, not from optimization of StreamingLocalLimitExec + val staticDF = spark.range(1).toDF("value").limit(1) + verifyLocalLimit(inputDF.toDF("value").join(staticDF, "value"), expectStreamingLimit = false) + + verifyLocalLimit( + inputDF.groupBy().count().limit(1), + expectStreamingLimit = false, + outputMode = OutputMode.Complete()) + } + test("is_continuous_processing property should be false for microbatch processing") { val input = MemoryStream[Int] val df = input.toDS() From ff0f63627901dcb6992d49655e4388bfe43e4755 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 31 Jan 2020 09:48:34 -0800 Subject: [PATCH 1271/1387] [SPARK-30638][CORE][FOLLOWUP] Fix a spacing issue and use UTF-8 instead of ASCII ### What changes were proposed in this pull request? Followup from https://github.com/apache/spark/pull/27367 to fix a couple of my minor issues with the Test. Fix an indentation and then use UTF-8 instead of ASCII. ### Why are the changes needed? followup ### Does this PR introduce any user-facing change? no ### How was this patch tested? compiled and ran unit test Closes #27420 from tgravescs/SPARK-30638-followup. Authored-by: Thomas Graves Signed-off-by: Dongjoon Hyun --- .../apache/spark/internal/plugin/PluginContainerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala index ac57e29916b82..cf2d9293ef822 100644 --- a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala @@ -181,7 +181,7 @@ class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with Lo val execFiles = children.filter(_.getName.startsWith(NonLocalModeSparkPlugin.executorFileStr)) assert(execFiles.size === 1) - val allLines = Files.readLines(execFiles(0), StandardCharsets.US_ASCII) + val allLines = Files.readLines(execFiles(0), StandardCharsets.UTF_8) assert(allLines.size === 1) val addrs = NonLocalModeSparkPlugin.extractGpuAddrs(allLines.get(0)) assert(addrs.size === 2) @@ -213,7 +213,7 @@ class NonLocalModeSparkPlugin extends SparkPlugin { new ExecutorPlugin() { override def init(ctx: PluginContext, extraConf: JMap[String, String]): Unit = { NonLocalModeSparkPlugin.writeFile(NonLocalModeSparkPlugin.executorFileStr, ctx.conf(), - ctx.executorID(), ctx.resources().asScala.toMap) + ctx.executorID(), ctx.resources().asScala.toMap) } } } From 5e0faf9a3d17bdedcc3541e43531cd903e968907 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 31 Jan 2020 10:17:07 -0800 Subject: [PATCH 1272/1387] [SPARK-29779][SPARK-30479][CORE][SQL][FOLLOWUP] Reflect review comments on post-hoc review ### What changes were proposed in this pull request? This PR reflects review comments on post-hoc review among PRs for SPARK-29779 (#27085), SPARK-30479 (#27164). The list of review comments this PR addresses are below: * https://github.com/apache/spark/pull/27085#discussion_r373304218 * https://github.com/apache/spark/pull/27164#discussion_r373300793 * https://github.com/apache/spark/pull/27164#discussion_r373301193 * https://github.com/apache/spark/pull/27164#discussion_r373301351 I also applied review comments to the CORE module (BasicEventFilterBuilder.scala) as well, as the review comments for SQL/core module (SQLEventFilterBuilder.scala) can be applied there as well. ### Why are the changes needed? There're post-hoc reviews on PRs for such issues, like links in above section. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing UTs. Closes #27414 from HeartSaVioR/SPARK-28869-SPARK-29779-SPARK-30479-FOLLOWUP-posthoc-reviews. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../history/BasicEventFilterBuilder.scala | 52 +++++++++---------- .../spark/internal/config/History.scala | 3 ++ .../history/SQLEventFilterBuilder.scala | 50 +++++++++--------- 3 files changed, 54 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala index 106da1675f71e..b18bf2665d6ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/BasicEventFilterBuilder.scala @@ -31,43 +31,43 @@ import org.apache.spark.storage.BlockManagerId * and dead executors. */ private[spark] class BasicEventFilterBuilder extends SparkListener with EventFilterBuilder { - private val _liveJobToStages = new mutable.HashMap[Int, Set[Int]] - private val _stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] - private val _stageToRDDs = new mutable.HashMap[Int, Set[Int]] + private val liveJobToStages = new mutable.HashMap[Int, Set[Int]] + private val stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] + private val stageToRDDs = new mutable.HashMap[Int, Set[Int]] private val _liveExecutors = new mutable.HashSet[String] private var totalJobs: Long = 0L private var totalStages: Long = 0L private var totalTasks: Long = 0L - def liveJobs: Set[Int] = _liveJobToStages.keySet.toSet - def liveStages: Set[Int] = _stageToRDDs.keySet.toSet - def liveTasks: Set[Long] = _stageToTasks.values.flatten.toSet - def liveRDDs: Set[Int] = _stageToRDDs.values.flatten.toSet - def liveExecutors: Set[String] = _liveExecutors.toSet + private[history] def liveJobs: Set[Int] = liveJobToStages.keySet.toSet + private[history] def liveStages: Set[Int] = stageToRDDs.keySet.toSet + private[history] def liveTasks: Set[Long] = stageToTasks.values.flatten.toSet + private[history] def liveRDDs: Set[Int] = stageToRDDs.values.flatten.toSet + private[history] def liveExecutors: Set[String] = _liveExecutors.toSet override def onJobStart(jobStart: SparkListenerJobStart): Unit = { totalJobs += 1 totalStages += jobStart.stageIds.length - _liveJobToStages += jobStart.jobId -> jobStart.stageIds.toSet + liveJobToStages += jobStart.jobId -> jobStart.stageIds.toSet } override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { - val stages = _liveJobToStages.getOrElse(jobEnd.jobId, Seq.empty[Int]) - _liveJobToStages -= jobEnd.jobId - _stageToTasks --= stages - _stageToRDDs --= stages + val stages = liveJobToStages.getOrElse(jobEnd.jobId, Seq.empty[Int]) + liveJobToStages -= jobEnd.jobId + stageToTasks --= stages + stageToRDDs --= stages } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { val stageId = stageSubmitted.stageInfo.stageId - _stageToRDDs.put(stageId, stageSubmitted.stageInfo.rddInfos.map(_.id).toSet) - _stageToTasks.getOrElseUpdate(stageId, new mutable.HashSet[Long]()) + stageToRDDs.put(stageId, stageSubmitted.stageInfo.rddInfos.map(_.id).toSet) + stageToTasks.getOrElseUpdate(stageId, new mutable.HashSet[Long]()) } override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { totalTasks += 1 - _stageToTasks.get(taskStart.stageId).foreach { tasks => + stageToTasks.get(taskStart.stageId).foreach { tasks => tasks += taskStart.taskInfo.taskId } } @@ -140,18 +140,18 @@ private[spark] abstract class JobEventFilter( * will be considered as "Don't mind". */ private[spark] class BasicEventFilter( - _stats: FilterStatistics, - _liveJobs: Set[Int], - _liveStages: Set[Int], - _liveTasks: Set[Long], - _liveRDDs: Set[Int], + stats: FilterStatistics, + liveJobs: Set[Int], + liveStages: Set[Int], + liveTasks: Set[Long], + liveRDDs: Set[Int], liveExecutors: Set[String]) extends JobEventFilter( - Some(_stats), - _liveJobs, - _liveStages, - _liveTasks, - _liveRDDs) with Logging { + Some(stats), + liveJobs, + liveStages, + liveTasks, + liveRDDs) with Logging { logDebug(s"live executors : $liveExecutors") diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index 17fb55d9db860..14fb5ff075472 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -96,6 +96,9 @@ private[spark] object History { private[spark] val EVENT_LOG_COMPACTION_SCORE_THRESHOLD = ConfigBuilder("spark.history.fs.eventLog.rolling.compaction.score.threshold") + .doc("The threshold score to determine whether it's good to do the compaction or not. " + + "The compaction score is calculated in analyzing, and being compared to this value. " + + "Compaction will proceed only when the score is higher than the threshold value.") .internal() .doubleConf .createWithDefault(0.7d) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala index fbd729b9d7c37..e1f42d7abe0fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/history/SQLEventFilterBuilder.scala @@ -35,17 +35,17 @@ import org.apache.spark.sql.streaming.StreamingQueryListener * between finished job and live job without relation of SQL execution. */ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilterBuilder { - private val _liveExecutionToJobs = new mutable.HashMap[Long, mutable.Set[Int]] - private val _jobToStages = new mutable.HashMap[Int, Set[Int]] - private val _stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] - private val _stageToRDDs = new mutable.HashMap[Int, Set[Int]] + private val liveExecutionToJobs = new mutable.HashMap[Long, mutable.Set[Int]] + private val jobToStages = new mutable.HashMap[Int, Set[Int]] + private val stageToTasks = new mutable.HashMap[Int, mutable.Set[Long]] + private val stageToRDDs = new mutable.HashMap[Int, Set[Int]] private val stages = new mutable.HashSet[Int] - def liveSQLExecutions: Set[Long] = _liveExecutionToJobs.keySet.toSet - def liveJobs: Set[Int] = _liveExecutionToJobs.values.flatten.toSet - def liveStages: Set[Int] = _stageToRDDs.keySet.toSet - def liveTasks: Set[Long] = _stageToTasks.values.flatten.toSet - def liveRDDs: Set[Int] = _stageToRDDs.values.flatten.toSet + private[history] def liveSQLExecutions: Set[Long] = liveExecutionToJobs.keySet.toSet + private[history] def liveJobs: Set[Int] = liveExecutionToJobs.values.flatten.toSet + private[history] def liveStages: Set[Int] = stageToRDDs.keySet.toSet + private[history] def liveTasks: Set[Long] = stageToTasks.values.flatten.toSet + private[history] def liveRDDs: Set[Int] = stageToRDDs.values.flatten.toSet override def onJobStart(jobStart: SparkListenerJobStart): Unit = { val executionIdString = jobStart.properties.getProperty(SQLExecution.EXECUTION_ID_KEY) @@ -57,24 +57,24 @@ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilte val executionId = executionIdString.toLong val jobId = jobStart.jobId - val jobsForExecution = _liveExecutionToJobs.getOrElseUpdate(executionId, + val jobsForExecution = liveExecutionToJobs.getOrElseUpdate(executionId, mutable.HashSet[Int]()) jobsForExecution += jobId - _jobToStages += jobStart.jobId -> jobStart.stageIds.toSet + jobToStages += jobStart.jobId -> jobStart.stageIds.toSet stages ++= jobStart.stageIds } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { val stageId = stageSubmitted.stageInfo.stageId if (stages.contains(stageId)) { - _stageToRDDs.put(stageId, stageSubmitted.stageInfo.rddInfos.map(_.id).toSet) - _stageToTasks.getOrElseUpdate(stageId, new mutable.HashSet[Long]()) + stageToRDDs.put(stageId, stageSubmitted.stageInfo.rddInfos.map(_.id).toSet) + stageToTasks.getOrElseUpdate(stageId, new mutable.HashSet[Long]()) } } override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { - _stageToTasks.get(taskStart.stageId).foreach { tasks => + stageToTasks.get(taskStart.stageId).foreach { tasks => tasks += taskStart.taskInfo.taskId } } @@ -86,16 +86,16 @@ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilte } private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { - _liveExecutionToJobs += event.executionId -> mutable.HashSet[Int]() + liveExecutionToJobs += event.executionId -> mutable.HashSet[Int]() } private def onExecutionEnd(event: SparkListenerSQLExecutionEnd): Unit = { - _liveExecutionToJobs.remove(event.executionId).foreach { jobs => - val stagesToDrop = _jobToStages.filter(kv => jobs.contains(kv._1)).values.flatten - _jobToStages --= jobs + liveExecutionToJobs.remove(event.executionId).foreach { jobs => + val stagesToDrop = jobToStages.filter(kv => jobs.contains(kv._1)).values.flatten + jobToStages --= jobs stages --= stagesToDrop - _stageToTasks --= stagesToDrop - _stageToRDDs --= stagesToDrop + stageToTasks --= stagesToDrop + stageToRDDs --= stagesToDrop } } @@ -115,11 +115,11 @@ private[spark] class SQLEventFilterBuilder extends SparkListener with EventFilte */ private[spark] class SQLLiveEntitiesEventFilter( liveSQLExecutions: Set[Long], - _liveJobs: Set[Int], - _liveStages: Set[Int], - _liveTasks: Set[Long], - _liveRDDs: Set[Int]) - extends JobEventFilter(None, _liveJobs, _liveStages, _liveTasks, _liveRDDs) with Logging { + liveJobs: Set[Int], + liveStages: Set[Int], + liveTasks: Set[Long], + liveRDDs: Set[Int]) + extends JobEventFilter(None, liveJobs, liveStages, liveTasks, liveRDDs) with Logging { logDebug(s"live SQL executions : $liveSQLExecutions") From 5eac2dcbcd41208bba2539c964f94adb64b15446 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 31 Jan 2020 12:52:22 -0600 Subject: [PATCH 1273/1387] [SPARK-30691][SQL][DOC] Add a few main pages to SQL Reference ### What changes were proposed in this pull request? Add a few main pages ### Why are the changes needed? To make SQL Reference complete. ### Does this PR introduce any user-facing change? Yes ![image](https://user-images.githubusercontent.com/13592258/73563358-f859f800-4411-11ea-8bd9-27d4db784957.png) ![image](https://user-images.githubusercontent.com/13592258/73530590-a55e5180-43cd-11ea-81b9-0192ff990b96.png) ![image](https://user-images.githubusercontent.com/13592258/73530629-b909b800-43cd-11ea-91a9-cfc71e213c7a.png) ![image](https://user-images.githubusercontent.com/13592258/73530812-0be36f80-43ce-11ea-9151-efa4ab7f2105.png) ![image](https://user-images.githubusercontent.com/13592258/73530908-3e8d6800-43ce-11ea-9943-10f2bd2bb408.png) ![image](https://user-images.githubusercontent.com/13592258/73530916-451bdf80-43ce-11ea-83c2-c7a9b063add7.png) ![image](https://user-images.githubusercontent.com/13592258/73530927-4baa5700-43ce-11ea-963c-951c8820ff54.png) ![image](https://user-images.githubusercontent.com/13592258/73530963-5cf36380-43ce-11ea-8cb1-6064ba2992f3.png) ### How was this patch tested? Manually build and check Closes #27416 from huaxingao/spark-doc. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- docs/_data/menu-sql.yaml | 17 ---------------- docs/sql-ref-functions-builtin.md | 2 +- docs/sql-ref-functions-udf.md | 2 +- docs/sql-ref-functions.md | 2 +- docs/sql-ref-syntax-aux-analyze.md | 4 ++-- docs/sql-ref-syntax-aux-cache.md | 4 ++-- docs/sql-ref-syntax-aux-conf-mgmt.md | 10 ++++------ docs/sql-ref-syntax-aux-describe.md | 12 ++++++------ docs/sql-ref-syntax-aux-resource-mgmt.md | 12 ++++++------ docs/sql-ref-syntax-aux-show.md | 17 +++++++++------- docs/sql-ref-syntax-aux.md | 16 +++++++++------ docs/sql-ref-syntax-ddl.md | 25 ++++++++++++++++++------ docs/sql-ref-syntax-dml.md | 8 ++++---- docs/sql-ref-syntax.md | 9 +++------ docs/sql-ref.md | 2 +- 15 files changed, 70 insertions(+), 72 deletions(-) diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 7673731778e3b..3e4db7107ec34 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -212,22 +212,5 @@ url: sql-ref-syntax-aux-resource-mgmt-list-file.html - text: LIST JAR url: sql-ref-syntax-aux-resource-mgmt-list-jar.html - - text: Functions - url: sql-ref-functions.html - subitems: - - text: Builtin Functions - url: sql-ref-functions-builtin.html - subitems: - - text: Scalar functions - url: sql-ref-functions-builtin-scalar.html - - text: Aggregate functions - url: sql-ref-functions-builtin-aggregate.html - - text: User defined Functions - url: sql-ref-functions-udf.html - subitems: - - text: Scalar functions - url: sql-ref-functions-udf-scalar.html - - text: Aggregate functions - url: sql-ref-functions-udf-aggregate.html - text: Arithmetic operations url: sql-ref-arithmetic-ops.html diff --git a/docs/sql-ref-functions-builtin.md b/docs/sql-ref-functions-builtin.md index eb0e73d00e848..6c57b0d6fdf6f 100644 --- a/docs/sql-ref-functions-builtin.md +++ b/docs/sql-ref-functions-builtin.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. +Spark SQL is Apache Spark's module for working with structured data. This guide is a reference for Structured Query Language (SQL) for Apache Spark. This document describes the SQL constructs supported by Spark in detail along with usage examples when applicable. diff --git a/docs/sql-ref-functions-udf.md b/docs/sql-ref-functions-udf.md index eb0e73d00e848..6c57b0d6fdf6f 100644 --- a/docs/sql-ref-functions-udf.md +++ b/docs/sql-ref-functions-udf.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. +Spark SQL is Apache Spark's module for working with structured data. This guide is a reference for Structured Query Language (SQL) for Apache Spark. This document describes the SQL constructs supported by Spark in detail along with usage examples when applicable. diff --git a/docs/sql-ref-functions.md b/docs/sql-ref-functions.md index eb0e73d00e848..6c57b0d6fdf6f 100644 --- a/docs/sql-ref-functions.md +++ b/docs/sql-ref-functions.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. +Spark SQL is Apache Spark's module for working with structured data. This guide is a reference for Structured Query Language (SQL) for Apache Spark. This document describes the SQL constructs supported by Spark in detail along with usage examples when applicable. diff --git a/docs/sql-ref-syntax-aux-analyze.md b/docs/sql-ref-syntax-aux-analyze.md index 2d19d18b42527..b1bdc73657724 100644 --- a/docs/sql-ref-syntax-aux-analyze.md +++ b/docs/sql-ref-syntax-aux-analyze.md @@ -1,7 +1,7 @@ --- layout: global -title: ANALYZE -displayTitle: ANALYZE +title: Analyze Statement +displayTitle: Analyze Statement license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with diff --git a/docs/sql-ref-syntax-aux-cache.md b/docs/sql-ref-syntax-aux-cache.md index e213bc3dff4e1..e4a640a6edbe5 100644 --- a/docs/sql-ref-syntax-aux-cache.md +++ b/docs/sql-ref-syntax-aux-cache.md @@ -1,7 +1,7 @@ --- layout: global -title: Cache -displayTitle: Cache +title: Cache Statements +displayTitle: Cache Statements license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with diff --git a/docs/sql-ref-syntax-aux-conf-mgmt.md b/docs/sql-ref-syntax-aux-conf-mgmt.md index eb0e73d00e848..7c5d9cc895c10 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Configuration Management +displayTitle: Configuration Management license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,7 +19,5 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +* [SET](sql-ref-syntax-aux-conf-mgmt-set.html) +* [UNSET](sql-ref-syntax-aux-conf-mgmt-reset.html) diff --git a/docs/sql-ref-syntax-aux-describe.md b/docs/sql-ref-syntax-aux-describe.md index eb0e73d00e848..9f17746316480 100644 --- a/docs/sql-ref-syntax-aux-describe.md +++ b/docs/sql-ref-syntax-aux-describe.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Describe Commands +displayTitle: Describe Commands license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,7 +19,7 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +* [DESCRIBE DATABASE](sql-ref-syntax-aux-describe-database.html) +* [DESCRIBE TABLE](sql-ref-syntax-aux-describe-table.html) +* [DESCRIBE FUNCTION](sql-ref-syntax-aux-describe-function.html) +* [DESCRIBE QUERY](sql-ref-syntax-aux-describe-query.html) diff --git a/docs/sql-ref-syntax-aux-resource-mgmt.md b/docs/sql-ref-syntax-aux-resource-mgmt.md index eb0e73d00e848..5eddb979388aa 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Resource Management +displayTitle: Resource Management license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,7 +19,7 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +* [ADD FILE](sql-ref-syntax-aux-resource-mgmt-add-file.html) +* [ADD JAR](sql-ref-syntax-aux-resource-mgmt-add-jar.html) +* [LIST FILE](sql-ref-syntax-aux-resource-mgmt-list-file.html) +* [LIST JAR](sql-ref-syntax-aux-resource-mgmt-list-jar.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux-show.md b/docs/sql-ref-syntax-aux-show.md index eb0e73d00e848..ad5c3954c23fa 100644 --- a/docs/sql-ref-syntax-aux-show.md +++ b/docs/sql-ref-syntax-aux-show.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Show Commands +displayTitle: Show Commands license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -18,8 +18,11 @@ license: | See the License for the specific language governing permissions and limitations under the License. --- - -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +* [SHOW COLUMNS](sql-ref-syntax-aux-show-columns.html) +* [SHOW DATABASES](sql-ref-syntax-aux-show-databases.html) +* [SHOW FUNCTIONS](sql-ref-syntax-aux-show-functions.html) +* [SHOW TABLE EXTENDED](sql-ref-syntax-aux-show-table.html) +* [SHOW TABLES](sql-ref-syntax-aux-show-tables.html) +* [SHOW TBLPROPERTIES](sql-ref-syntax-aux-show-tblproperties.html) +* [SHOW PARTITIONS](sql-ref-syntax-aux-show-partitions.html) +* [SHOW CREATE TABLE](sql-ref-syntax-aux-show-create-table.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax-aux.md b/docs/sql-ref-syntax-aux.md index eb0e73d00e848..ba09d70b437a9 100644 --- a/docs/sql-ref-syntax-aux.md +++ b/docs/sql-ref-syntax-aux.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Auxiliary Statements +displayTitle: Auxiliary Statements license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,7 +19,11 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +Besides the major SQL statements such as Data Definition Statements, Data Manipulation Statements and Data Retrieval Statements, Spark SQL also supports the following Auxiliary Statements: + +- [ANALYZE](sql-ref-syntax-aux-analyze.html) +- [CACHE](sql-ref-syntax-aux-cache.html) +- [DESCRIBE](sql-ref-syntax-aux-describe.html) +- [SHOW](sql-ref-syntax-aux-show.html) +- [CONFIGURATION MANAGEMENT](sql-ref-syntax-aux-conf-mgmt.html) +- [RESOURCE MANAGEMENT](sql-ref-syntax-aux-resource-mgmt.html) diff --git a/docs/sql-ref-syntax-ddl.md b/docs/sql-ref-syntax-ddl.md index eb0e73d00e848..954020a874455 100644 --- a/docs/sql-ref-syntax-ddl.md +++ b/docs/sql-ref-syntax-ddl.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: Data Definition Statements +displayTitle: Data Definition Statements license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,7 +19,20 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. + +Data Definition Statements are used to create or modify the structure of database objects in a database. Spark SQL supports the following Data Definition Statements: + + +- [ALTER DATABASE](sql-ref-syntax-ddl-alter-database.html) +- [ALTER TABLE](sql-ref-syntax-ddl-alter-table.html) +- [ALTER VIEW](sql-ref-syntax-ddl-alter-view.html) +- [CREATE DATABASE](sql-ref-syntax-ddl-create-database.html) +- [CREATE FUNCTION](sql-ref-syntax-ddl-create-function.html) +- [CREATE TABLE](sql-ref-syntax-ddl-create-table.html) +- [CREATE VIEW](sql-ref-syntax-ddl-create-view.html) +- [DROP DATABASE](sql-ref-syntax-ddl-drop-database.html) +- [DROP FUNCTION](sql-ref-syntax-ddl-drop-function.html) +- [DROP TABLE](sql-ref-syntax-ddl-drop-table.html) +- [DROP VIEW](sql-ref-syntax-ddl-drop-view.html) +- [TRUNCATE TABLE](sql-ref-syntax-ddl-truncate-table.html) +- [REPAIR TABLE](sql-ref-syntax-ddl-repair-table.html) diff --git a/docs/sql-ref-syntax-dml.md b/docs/sql-ref-syntax-dml.md index 7fd537e0a0f5c..af09e2054562e 100644 --- a/docs/sql-ref-syntax-dml.md +++ b/docs/sql-ref-syntax-dml.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +Data Manipulation Statements are used to add, change, or delete data. Spark SQL supports the following Data Manipulation Statements: + +- [INSERT](sql-ref-syntax-dml-insert.html) +- [LOAD](sql-ref-syntax-dml-load.html) \ No newline at end of file diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index eb0e73d00e848..2510278ab252c 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -1,7 +1,7 @@ --- layout: global -title: Reference -displayTitle: Reference +title: SQL Syntax +displayTitle: SQL Syntax license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,7 +19,4 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. -This guide is a reference for Structured Query Language (SQL) for Apache -Spark. This document describes the SQL constructs supported by Spark in detail -along with usage examples when applicable. +Spark SQL is Apache Spark's module for working with structured data. The SQL Syntax section describes the SQL syntax in detail along with usage examples when applicable. diff --git a/docs/sql-ref.md b/docs/sql-ref.md index eb0e73d00e848..6c57b0d6fdf6f 100644 --- a/docs/sql-ref.md +++ b/docs/sql-ref.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -Spark SQL is a Apache Spark's module for working with structured data. +Spark SQL is Apache Spark's module for working with structured data. This guide is a reference for Structured Query Language (SQL) for Apache Spark. This document describes the SQL constructs supported by Spark in detail along with usage examples when applicable. From 33546d637d49a2dcad52edcf9570c29819c2b589 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 1 Feb 2020 02:59:16 +0800 Subject: [PATCH 1274/1387] Revert "[SPARK-30036][SQL] Fix: REPARTITION hint does not work with order by" This reverts commit a2de20c0e6857653de63f46052935784be87d34f. --- .../exchange/EnsureRequirements.scala | 2 - .../spark/sql/ConfigBehaviorSuite.scala | 8 +-- .../spark/sql/execution/PlannerSuite.scala | 50 ------------------- 3 files changed, 5 insertions(+), 55 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index 68cf2200bf73a..ab4176cada527 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -55,8 +55,6 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { child case (child, BroadcastDistribution(mode)) => BroadcastExchangeExec(mode, child) - case (ShuffleExchangeExec(partitioning, child, _), distribution: OrderedDistribution) => - ShuffleExchangeExec(distribution.createPartitioning(partitioning.numPartitions), child) case (child, distribution) => val numPartitions = distribution.requiredNumPartitions .getOrElse(defaultNumPreShufflePartitions) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala index 982681f18bd98..c3dbbb325d842 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ConfigBehaviorSuite.scala @@ -39,7 +39,9 @@ class ConfigBehaviorSuite extends QueryTest with SharedSparkSession { def computeChiSquareTest(): Double = { val n = 10000 // Trigger a sort - val data = spark.range(0, n, 1, 10).sort($"id".desc) + // Range has range partitioning in its output now. To have a range shuffle, we + // need to run a repartition first. + val data = spark.range(0, n, 1, 1).repartition(10).sort($"id".desc) .selectExpr("SPARK_PARTITION_ID() pid", "id").as[(Int, Long)].collect() // Compute histogram for the number of records per partition post sort @@ -57,12 +59,12 @@ class ConfigBehaviorSuite extends QueryTest with SharedSparkSession { SQLConf.SHUFFLE_PARTITIONS.key -> numPartitions.toString, SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The default chi-sq value should be low - assert(computeChiSquareTest() < 10) + assert(computeChiSquareTest() < 100) withSQLConf(SQLConf.RANGE_EXCHANGE_SAMPLE_SIZE_PER_PARTITION.key -> "1") { // If we only sample one point, the range boundaries will be pretty bad and the // chi-sq value would be very high. - assert(computeChiSquareTest() > 100) + assert(computeChiSquareTest() > 300) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 94ce3559bb44b..0c5e2e3c7d1d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -425,56 +425,6 @@ class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { } } - test("SPARK-30036: Remove unnecessary RoundRobinPartitioning " + - "if SortExec is followed by RoundRobinPartitioning") { - val distribution = OrderedDistribution(SortOrder(Literal(1), Ascending) :: Nil) - val partitioning = RoundRobinPartitioning(5) - assert(!partitioning.satisfies(distribution)) - - val inputPlan = SortExec(SortOrder(Literal(1), Ascending) :: Nil, - global = true, - child = ShuffleExchangeExec( - partitioning, - DummySparkPlan(outputPartitioning = partitioning))) - val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) - assert(outputPlan.find { - case ShuffleExchangeExec(_: RoundRobinPartitioning, _, _) => true - case _ => false - }.isEmpty, - "RoundRobinPartitioning should be changed to RangePartitioning") - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - // when enable AQE, the post partiiton number is changed. - val query = testData.select('key, 'value).repartition(2).sort('key.asc) - assert(query.rdd.getNumPartitions == 2) - assert(query.rdd.collectPartitions()(0).map(_.get(0)).toSeq == (1 to 50)) - } - } - - test("SPARK-30036: Remove unnecessary HashPartitioning " + - "if SortExec is followed by HashPartitioning") { - val distribution = OrderedDistribution(SortOrder(Literal(1), Ascending) :: Nil) - val partitioning = HashPartitioning(Literal(1) :: Nil, 5) - assert(!partitioning.satisfies(distribution)) - - val inputPlan = SortExec(SortOrder(Literal(1), Ascending) :: Nil, - global = true, - child = ShuffleExchangeExec( - partitioning, - DummySparkPlan(outputPartitioning = partitioning))) - val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) - assert(outputPlan.find { - case ShuffleExchangeExec(_: HashPartitioning, _, _) => true - case _ => false - }.isEmpty, - "HashPartitioning should be changed to RangePartitioning") - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - // when enable AQE, the post partiiton number is changed. - val query = testData.select('key, 'value).repartition(5, 'key).sort('key.asc) - assert(query.rdd.getNumPartitions == 5) - assert(query.rdd.collectPartitions()(0).map(_.get(0)).toSeq == (1 to 20)) - } - } - test("EnsureRequirements does not eliminate Exchange with different partitioning") { val distribution = ClusteredDistribution(Literal(1) :: Nil) val partitioning = HashPartitioning(Literal(2) :: Nil, 5) From 18bc4e55eff3bab9d72792125b52a336c0356110 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 31 Jan 2020 11:58:52 -0800 Subject: [PATCH 1275/1387] [SPARK-30684][WEBUI] Show the descripton of metrics for WholeStageCodegen in DAG viz ### What changes were proposed in this pull request? Added description for metrics shown in the WholeStageCodegen-node in DAG viz. This is before the change is applied. ![before-changed](https://user-images.githubusercontent.com/4736016/73469870-5cf16480-43ca-11ea-9a13-714083508a3b.png) And following is after change. ![after-fixing-layout](https://user-images.githubusercontent.com/4736016/73469364-983f6380-43c9-11ea-8b7e-ddab030d0270.png) For this change, I also modify the layout of DAG viz. Actually, I noticed it's not enough to just added the description. Following is without changing the layout. ![layout-is-broken](https://user-images.githubusercontent.com/4736016/73470178-cffadb00-43ca-11ea-86d7-aed109b105e6.png) ### Why are the changes needed? Users can't understand what those metrics mean. ### Does this PR introduce any user-facing change? Yes. The layout is a little bit changed. ### How was this patch tested? I confirm the result of DAG viz with following 3 operations. `sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").show` `sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").write.format("json").mode("overwrite").save("/tmp/test_output")` `sc.parallelize(1 to 10).toDF.write.format("json").mode("append").save("/tmp/test_output")` Closes #27405 from sarutak/sql-dag-metrics. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- .../sql/execution/ui/static/spark-sql-viz.css | 7 ++++- .../sql/execution/ui/static/spark-sql-viz.js | 30 +++++++++++++++++++ .../sql/execution/ui/SparkPlanGraph.scala | 17 +++++++---- 3 files changed, 47 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css index b13850c301490..eff0142dc523f 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css @@ -32,11 +32,16 @@ stroke-width: 1px; } -/* Highlight the SparkPlan node name */ +/* This declaration is needed to define the width of rectangles */ #plan-viz-graph svg text :first-child { font-weight: bold; } +/* Highlight the SparkPlan node name */ +#plan-viz-graph svg text .operator-name { + font-weight: bold; +} + #plan-viz-graph svg path { stroke: #444; stroke-width: 1.5px; diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js index 754711bd5ad85..e6ce641a841b3 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js @@ -34,6 +34,7 @@ function renderPlanViz() { preprocessGraphLayout(g); var renderer = new dagreD3.render(); renderer(graph, g); + adjustPositionOfOperationName(); // Round corners on rectangles svg @@ -81,6 +82,7 @@ function setupTooltipForSparkPlanNode(nodeId) { * and sizes of graph elements, e.g. padding, font style, shape. */ function preprocessGraphLayout(g) { + g.graph().ranksep = "90"; var nodes = g.nodes(); for (var i = 0; i < nodes.length; i++) { var node = g.node(nodes[i]); @@ -127,6 +129,34 @@ function resizeSvg(svg) { .attr("height", height); } + +/* Helper function to adjust the position of operation name and mark as a operation-name class. */ +function adjustPositionOfOperationName() { + $("#plan-viz-graph svg text") + .each(function() { + var tspans = $(this).find("tspan"); + + if (tspans[0].textContent.trim() !== "") { + var isOperationNameOnly = + $(tspans).filter(function(i, n) { + return i !== 0 && n.textContent.trim() !== ""; + }).length === 0; + + if (isOperationNameOnly) { + // If the only text in a node is operation name, + // vertically centering the position of operation name. + var operationName = tspans[0].textContent; + var half = parseInt(tspans.length / 2); + tspans[0].textContent = tspans[half].textContent; + tspans[half].textContent = operationName; + $(tspans[half]).addClass("operator-name"); + } else { + tspans.first().addClass("operator-name"); + } + } + }); +} + /* Helper function to convert attributes to numeric values. */ function toFloat(f) { if (f) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index bd14be702a407..1e767c3c043c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -169,12 +169,17 @@ private[ui] class SparkPlanGraphNode( metric.name + ": " + value } + // If there are metrics, display each entry in a separate line. + // Note: whitespace between two "\n"s is to create an empty line between the name of + // SparkPlan and metrics. If removing it, it won't display the empty line in UI. + builder ++= "\n \n" + if (values.nonEmpty) { - // If there are metrics, display each entry in a separate line. - // Note: whitespace between two "\n"s is to create an empty line between the name of - // SparkPlan and metrics. If removing it, it won't display the empty line in UI. - builder ++= "\n \n" builder ++= values.mkString("\n") + } else { + // A certain level of height is needed for a rect as a node in a sub-graph + // to avoid layout collapse for sub-graphs. + builder ++= " " } s""" $id [label="${StringEscapeUtils.escapeJava(builder.toString())}"];""" @@ -197,8 +202,8 @@ private[ui] class SparkPlanGraphCluster( val labelStr = if (duration.nonEmpty) { require(duration.length == 1) val id = duration(0).accumulatorId - if (metricsValue.contains(duration(0).accumulatorId)) { - name + "\n\n" + metricsValue(id) + if (metricsValue.contains(id)) { + name + "\n \n" + duration(0).name + ": " + metricsValue(id) } else { name } From 387ce89a0631f1a4c6668b90ff2a7bbcf11919cd Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Fri, 31 Jan 2020 14:28:02 -0600 Subject: [PATCH 1276/1387] [SPARK-27324][DOC][CORE] Document configurations related to executor metrics and modify a configuration ### What changes were proposed in this pull request? Add a section to the Configuration page to document configurations for executor metrics. At the same time, rename spark.eventLog.logStageExecutorProcessTreeMetrics.enabled to spark.executor.processTreeMetrics.enabled and make it independent of spark.eventLog.logStageExecutorMetrics.enabled. ### Why are the changes needed? Executor metrics are new in Spark 3.0. They lack documentation. Memory metrics as a whole are always collected, but the ones obtained from the process tree have to be optionally enabled. Making this depend on a single configuration makes for more intuitive behavior. Given this, the configuration property is renamed to better reflect its meaning. ### Does this PR introduce any user-facing change? Yes, only in that the configurations are all new to 3.0. ### How was this patch tested? Not necessary. Closes #27329 from wypoon/SPARK-27324. Authored-by: Wing Yew Poon Signed-off-by: Imran Rashid --- .../executor/ExecutorMetricsSource.scala | 3 +- .../spark/executor/ProcfsMetricsGetter.scala | 8 ++-- .../spark/internal/config/package.scala | 17 ++++++--- .../deploy/history/HistoryServerSuite.scala | 2 +- docs/configuration.md | 37 +++++++++++++++++++ docs/monitoring.md | 20 +++++----- 6 files changed, 65 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala index b052e43d4665b..14645f73ef278 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsSource.scala @@ -32,8 +32,7 @@ import org.apache.spark.metrics.source.Source * spark.executor.metrics.pollingInterval=. * (2) Procfs metrics are gathered all in one-go and only conditionally: * if the /proc filesystem exists - * and spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true - * and spark.eventLog.logStageExecutorMetrics.enabled=true. + * and spark.executor.processTreeMetrics.enabled=true. */ private[spark] class ExecutorMetricsSource extends Source { diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala index 0d5dcfb43cbfd..80ef757332e43 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -58,11 +58,9 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L logWarning("Exception checking for procfs dir", ioe) false } - val shouldLogStageExecutorMetrics = - SparkEnv.get.conf.get(config.EVENT_LOG_STAGE_EXECUTOR_METRICS) - val shouldLogStageExecutorProcessTreeMetrics = - SparkEnv.get.conf.get(config.EVENT_LOG_PROCESS_TREE_METRICS) - procDirExists.get && shouldLogStageExecutorProcessTreeMetrics && shouldLogStageExecutorMetrics + val shouldPollProcessTreeMetrics = + SparkEnv.get.conf.get(config.EXECUTOR_PROCESS_TREE_METRICS_ENABLED) + procDirExists.get && shouldPollProcessTreeMetrics } } 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 40b05cf96d1e3..e68368f37ab6f 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 @@ -148,11 +148,8 @@ package object config { private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS = ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled") - .booleanConf - .createWithDefault(false) - - private[spark] val EVENT_LOG_PROCESS_TREE_METRICS = - ConfigBuilder("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled") + .doc("Whether to write per-stage peaks of executor metrics (for each executor) " + + "to the event log.") .booleanConf .createWithDefault(false) @@ -215,8 +212,18 @@ package object config { private[spark] val EXECUTOR_HEARTBEAT_MAX_FAILURES = ConfigBuilder("spark.executor.heartbeat.maxFailures").internal().intConf.createWithDefault(60) + private[spark] val EXECUTOR_PROCESS_TREE_METRICS_ENABLED = + ConfigBuilder("spark.executor.processTreeMetrics.enabled") + .doc("Whether to collect process tree metrics (from the /proc filesystem) when collecting " + + "executor metrics.") + .booleanConf + .createWithDefault(false) + private[spark] val EXECUTOR_METRICS_POLLING_INTERVAL = ConfigBuilder("spark.executor.metrics.pollingInterval") + .doc("How often to collect executor metrics (in milliseconds). " + + "If 0, the polling is done on executor heartbeats. " + + "If positive, the polling is done at this interval.") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("0") diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 06554e570a758..206db0feb5716 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -84,7 +84,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers .set(IS_TESTING, true) .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) .set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) - .set(EVENT_LOG_PROCESS_TREE_METRICS, true) + .set(EXECUTOR_PROCESS_TREE_METRICS_ENABLED, true) conf.setAll(extraConf) provider = new FsHistoryProvider(conf) provider.checkForLogs() diff --git a/docs/configuration.md b/docs/configuration.md index 559c5cdbff1d1..8164ed491d4c6 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1624,6 +1624,43 @@ Apart from these, the following properties are also available, and may be useful +### Executor Metrics + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.eventLog.logStageExecutorMetrics.enabledfalse + Whether to write per-stage peaks of executor metrics (for each executor) to the event log. +
    + Note: The metrics are polled (collected) and sent in the executor heartbeat, + and this is always done; this configuration is only to determine if aggregated metric peaks + are written to the event log. +
    spark.executor.processTreeMetrics.enabledfalse + Whether to collect process tree metrics (from the /proc filesystem) when collecting + executor metrics. +
    + Note: The process tree metrics are collected only if the /proc filesystem + exists. +
    spark.executor.metrics.pollingInterval0 + How often to collect executor metrics (in milliseconds). +
    + If 0, the polling is done on executor heartbeats (thus at the heartbeat interval, + specified by spark.executor.heartbeatInterval). + If positive, the polling is done at this interval. +
    + ### Networking diff --git a/docs/monitoring.md b/docs/monitoring.md index 31bf1ebdecad3..fc95f6a8d3191 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -659,7 +659,10 @@ A list of the available metrics, with a short description: ### Executor Metrics -Executor-level metrics are sent from each executor to the driver as part of the Heartbeat to describe the performance metrics of Executor itself like JVM heap memory, GC information. Metrics `peakExecutorMetrics.*` are only enabled if `spark.eventLog.logStageExecutorMetrics.enabled` is true. +Executor-level metrics are sent from each executor to the driver as part of the Heartbeat to describe the performance metrics of Executor itself like JVM heap memory, GC information. +Executor metric values and their measured peak values per executor are exposed via the REST API at the end point `/applications/[app-id]/executors`. +In addition, aggregated per-stage peak values of the executor metrics are written to the event log if `spark.eventLog.logStageExecutorMetrics.enabled` is true. +Executor metrics are also exposed via the Spark metrics system based on the Dropwizard metrics library. A list of the available metrics, with a short description:
    @@ -755,7 +758,7 @@ A list of the available metrics, with a short description: - + @@ -763,23 +766,23 @@ A list of the available metrics, with a short description: in real memory. This is just the pages which count toward text, data, or stack space. This does not include pages which have not been demand-loaded in, - or which are swapped out. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true. + or which are swapped out. Enabled if spark.executor.processTreeMetrics.enabled is true. - + - + - + - + @@ -1102,8 +1105,7 @@ when running in local mode. - ProcessTreeOtherRSSMemory - **note:** "ProcessTree*" metrics are collected only under certain conditions. The conditions are the logical AND of the following: `/proc` filesystem exists, - `spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true`, - `spark.eventLog.logStageExecutorMetrics.enabled=true`. + `spark.executor.processTreeMetrics.enabled=true`. "ProcessTree*" metrics report 0 when those conditions are not met. - namespace=JVMCPU From 2d4b5eaee4bdaea692a08e2edd1b3a63ce77d5b9 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 31 Jan 2020 15:03:16 -0600 Subject: [PATCH 1277/1387] [SPARK-30676][CORE][TESTS] Eliminate warnings from deprecated constructors of java.lang.Integer and java.lang.Double ### What changes were proposed in this pull request? - Replace `new Integer(0)` by a serializable instance in RDD.scala - Use `.valueOf()` instead of constructors of `java.lang.Integer` and `java.lang.Double` because constructors has been deprecated, see https://docs.oracle.com/javase/9/docs/api/java/lang/Integer.html ### Why are the changes needed? This fixes the following warnings: 1. RDD.scala:240: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information. 2. MutableProjectionSuite.scala:63: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information. 3. UDFSuite.scala:446: constructor Integer in class Integer is deprecated: see corresponding Javadoc for more information. 4. UDFSuite.scala:451: constructor Double in class Double is deprecated: see corresponding Javadoc for more information. 5. HiveUserDefinedTypeSuite.scala:71: constructor Double in class Double is deprecated: see corresponding Javadoc for more information. ### Does this PR introduce any user-facing change? No ### How was this patch tested? - By RDDSuite, MutableProjectionSuite, UDFSuite and HiveUserDefinedTypeSuite Closes #27399 from MaxGekk/eliminate-warning-part4. Authored-by: Maxim Gekk Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 6 +++--- .../sql/catalyst/expressions/MutableProjectionSuite.scala | 2 +- sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala | 4 ++-- .../apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 5788b70e75a7a..64d2032a12721 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -234,10 +234,10 @@ abstract class RDD[T: ClassTag]( * because DAGs are acyclic, and we only ever hold locks for one path in that DAG, there is no * chance of deadlock. * - * The use of Integer is simply so this is serializable -- executors may reference the shared - * fields (though they should never mutate them, that only happens on the driver). + * Executors may reference the shared fields (though they should never mutate them, + * that only happens on the driver). */ - private val stateLock = new Integer(0) + private val stateLock = new Serializable {} // Our dependencies and partitions will be gotten by calling subclass's methods below, and will // be overwritten when we're checkpointed diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala index 63700a1e94a3e..c31310bc54023 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -60,7 +60,7 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { val scalaValues = Seq("abc", BigDecimal(10), IntervalUtils.stringToInterval(UTF8String.fromString("interval 1 day")), Array[Byte](1, 2), Array("123", "456"), Map(1 -> "a", 2 -> "b"), Row(1, "a"), - new java.lang.Integer(5)) + Integer.valueOf(5)) val inputRow = InternalRow.fromSeq(scalaValues.zip(variableLengthTypes).map { case (v, dataType) => CatalystTypeConverters.createToCatalystConverter(dataType)(v) }) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 49e6e8f2cb8e9..cc3995516dcc2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -443,12 +443,12 @@ class UDFSuite extends QueryTest with SharedSparkSession { test("SPARK-25044 Verify null input handling for primitive types - with udf(Any, DataType)") { val f = udf((x: Int) => x, IntegerType) checkAnswer( - Seq(new Integer(1), null).toDF("x").select(f($"x")), + Seq(Integer.valueOf(1), null).toDF("x").select(f($"x")), Row(1) :: Row(0) :: Nil) val f2 = udf((x: Double) => x, DoubleType) checkAnswer( - Seq(new java.lang.Double(1.1), null).toDF("x").select(f2($"x")), + Seq(java.lang.Double.valueOf(1.1), null).toDF("x").select(f2($"x")), Row(1.1) :: Row(0.0) :: Nil) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala index c160ff206a62a..ca1af73b038a7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUserDefinedTypeSuite.scala @@ -68,6 +68,6 @@ class TestUDF extends GenericUDF { override def evaluate(arguments: Array[GenericUDF.DeferredObject]): AnyRef = { val point = data.getList(arguments(0).get()) - new java.lang.Double(point.get(0).asInstanceOf[Double]) + java.lang.Double.valueOf(point.get(0).asInstanceOf[Double]) } } From 82b4f753a088f229616a935c9f5b698fa303f343 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 31 Jan 2020 15:05:26 -0800 Subject: [PATCH 1278/1387] [SPARK-30508][SQL] Add SparkSession.executeCommand API for external datasource ### What changes were proposed in this pull request? This PR adds `SparkSession.executeCommand` API for external datasource to execute a random command like ``` val df = spark.executeCommand("xxxCommand", "xxxSource", "xxxOptions") ``` Note that the command doesn't execute in Spark, but inside an external execution engine depending on data source. And it will be eagerly executed after `executeCommand` called and the returned `DataFrame` will contain the output of the command(if any). ### Why are the changes needed? This can be useful when user wants to execute some commands out of Spark. For example, executing custom DDL/DML command for JDBC, creating index for ElasticSearch, creating cores for Solr and so on(as HyukjinKwon suggested). Previously, user needs to use an option to achieve the goal, e.g. `spark.read.format("xxxSource").option("command", "xxxCommand").load()`, which is kind of cumbersome. With this change, it can be more convenient for user to achieve the same goal. ### Does this PR introduce any user-facing change? Yes, new API from `SparkSession` and a new interface `ExternalCommandRunnableProvider`. ### How was this patch tested? Added a new test suite. Closes #27199 from Ngone51/dev-executeCommand. Lead-authored-by: yi.wu Co-authored-by: Xiao Li Co-authored-by: Wenchen Fan Signed-off-by: Xiao Li --- .../sql/connector/ExternalCommandRunner.java | 43 ++++++++++++++++ .../org/apache/spark/sql/SparkSession.scala | 31 +++++++++++- .../sql/execution/command/commands.scala | 30 ++++++++--- .../sources/ExternalCommandRunnerSuite.scala | 50 +++++++++++++++++++ 4 files changed, 147 insertions(+), 7 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/ExternalCommandRunner.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/ExternalCommandRunnerSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/ExternalCommandRunner.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/ExternalCommandRunner.java new file mode 100644 index 0000000000000..37534be31b6f6 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/ExternalCommandRunner.java @@ -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. + */ + +package org.apache.spark.sql.connector; + +import org.apache.spark.annotation.Unstable; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * An interface to execute an arbitrary string command inside an external execution engine rather + * than Spark. This could be useful when user wants to execute some commands out of Spark. For + * example, executing custom DDL/DML command for JDBC, creating index for ElasticSearch, creating + * cores for Solr and so on. + *

    + * This interface will be instantiated when end users call `SparkSession#executeCommand`. + */ +@Unstable +public interface ExternalCommandRunner { + + /** + * Execute the given command. + * + * @param command The command string provided by users. + * @param options The user-specified case-insensitive options. + * + * @return The output of the command. + */ + String[] executeCommand(String command, CaseInsensitiveStringMap options); +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 0ce514fb0f83c..abefb348cafc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -37,8 +37,10 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Range} +import org.apache.spark.sql.connector.ExternalCommandRunner import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.command.ExternalCommandExecutor +import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} import org.apache.spark.sql.internal._ import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.sources.BaseRelation @@ -605,6 +607,33 @@ class SparkSession private( Dataset.ofRows(self, plan, tracker) } + /** + * Execute an arbitrary string command inside an external execution engine rather than Spark. + * This could be useful when user wants to execute some commands out of Spark. For + * example, executing custom DDL/DML command for JDBC, creating index for ElasticSearch, + * creating cores for Solr and so on. + * + * The command will be eagerly executed after this method is called and the returned + * DataFrame will contain the output of the command(if any). + * + * @param runner The class name of the runner that implements `ExternalCommandRunner`. + * @param command The target command to be executed + * @param options The options for the runner. + * + * @since 3.0.0 + */ + @Unstable + def executeCommand(runner: String, command: String, options: Map[String, String]): DataFrame = { + DataSource.lookupDataSource(runner, sessionState.conf) match { + case source if classOf[ExternalCommandRunner].isAssignableFrom(source) => + Dataset.ofRows(self, ExternalCommandExecutor( + source.newInstance().asInstanceOf[ExternalCommandRunner], command, options)) + + case _ => + throw new AnalysisException(s"Command execution is not supported in runner $runner") + } + } + /** * Returns a [[DataFrameReader]] that can be used to read non-streaming data in as a * `DataFrame`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index ef7a8551fe811..18fd2a5ac2330 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -17,21 +17,21 @@ package org.apache.spark.sql.execution.command -import java.util.UUID +import scala.collection.JavaConverters._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} -import org.apache.spark.sql.execution.{ExplainMode, LeafExecNode, QueryExecution, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.debug._ +import org.apache.spark.sql.connector.ExternalCommandRunner +import org.apache.spark.sql.execution.{ExplainMode, LeafExecNode, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} -import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.execution.streaming.IncrementalExecution import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * A logical command that is executed for its side-effects. `RunnableCommand`s are @@ -183,3 +183,21 @@ case class StreamingExplainCommand( ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_)) } } + +/** + * Used to execute an arbitrary string command inside an external execution engine + * rather than Spark. Please check [[ExternalCommandRunner]] for more details. + */ +case class ExternalCommandExecutor( + runner: ExternalCommandRunner, + command: String, + options: Map[String, String]) extends RunnableCommand { + + override def output: Seq[Attribute] = + Seq(AttributeReference("command_output", StringType)()) + + override def run(sparkSession: SparkSession): Seq[Row] = { + val output = runner.executeCommand(command, new CaseInsensitiveStringMap(options.asJava)) + output.map(Row(_)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ExternalCommandRunnerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ExternalCommandRunnerSuite.scala new file mode 100644 index 0000000000000..55fb3eb8ade35 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ExternalCommandRunnerSuite.scala @@ -0,0 +1,50 @@ +/* + * 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.sources + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.connector.ExternalCommandRunner +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class ExternalCommandRunnerSuite extends QueryTest with SharedSparkSession { + test("execute command") { + try { + System.setProperty("command", "hello") + assert(System.getProperty("command") === "hello") + + val options = Map("one" -> "1", "two" -> "2") + val df = spark.executeCommand(classOf[FakeCommandRunner].getName, "world", options) + // executeCommand should execute the command eagerly + assert(System.getProperty("command") === "world") + checkAnswer(df, Seq(Row("one"), Row("two"))) + } finally { + System.clearProperty("command") + } + } +} + +class FakeCommandRunner extends ExternalCommandRunner { + + override def executeCommand(command: String, options: CaseInsensitiveStringMap): Array[String] = { + System.setProperty("command", command) + options.keySet().iterator().asScala.toSeq.sorted.toArray + } +} From 2fd15a26fbb41b55933752a6f95553b21f4dab42 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 31 Jan 2020 17:41:27 -0800 Subject: [PATCH 1279/1387] [SPARK-30695][BUILD] Upgrade Apache ORC to 1.5.9 ### What changes were proposed in this pull request? This PR aims to upgrade to Apache ORC 1.5.9. - For `hive-2.3` profile, we need to upgrade `hive-storage-api` from `2.6.0` to `2.7.1`. - For `hive-1.2` profile, ORC library with classifier `nohive` already shaded it. So, there is no change. ### Why are the changes needed? This will bring the latest bug fixes. The following is the full release note. - https://issues.apache.org/jira/projects/ORC/versions/12346546 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins with the existing tests. Here is the summary. 1. `Hive 1.2 + Hadoop 2.7` passed. ([here](https://github.com/apache/spark/pull/27421#issuecomment-580924552)) 2. `Hive 2.3 + Hadoop 2.7` passed. ([here](https://github.com/apache/spark/pull/27421#issuecomment-580973391)) Closes #27421 from dongjoon-hyun/SPARK-ORC-1.5.9. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- LICENSE-binary | 1 + dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 7 ++++--- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 9 +++++---- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 9 +++++---- pom.xml | 6 ++++-- 5 files changed, 19 insertions(+), 13 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index e19cc9784b63d..b50da6be4e697 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -469,6 +469,7 @@ xmlenc:xmlenc net.sf.py4j:py4j org.jpmml:pmml-model org.jpmml:pmml-schema +org.threeten:threeten-extra python/lib/py4j-*-src.zip python/pyspark/cloudpickle.py diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index e75d2cd800b34..62d5772ffc94a 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -160,9 +160,9 @@ objenesis/2.5.1//objenesis-2.5.1.jar okhttp/3.12.6//okhttp-3.12.6.jar okio/1.15.0//okio-1.15.0.jar opencsv/2.3//opencsv-2.3.jar -orc-core/1.5.8/nohive/orc-core-1.5.8-nohive.jar -orc-mapreduce/1.5.8/nohive/orc-mapreduce-1.5.8-nohive.jar -orc-shims/1.5.8//orc-shims-1.5.8.jar +orc-core/1.5.9/nohive/orc-core-1.5.9-nohive.jar +orc-mapreduce/1.5.9/nohive/orc-mapreduce-1.5.9-nohive.jar +orc-shims/1.5.9//orc-shims-1.5.9.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar @@ -198,6 +198,7 @@ stax-api/1.0.1//stax-api-1.0.1.jar stream/2.9.6//stream-2.9.6.jar stringtemplate/3.2.1//stringtemplate-3.2.1.jar super-csv/2.2.0//super-csv-2.2.0.jar +threeten-extra/1.5.0//threeten-extra-1.5.0.jar univocity-parsers/2.8.3//univocity-parsers-2.8.3.jar xbean-asm7-shaded/4.15//xbean-asm7-shaded-4.15.jar xercesImpl/2.9.1//xercesImpl-2.9.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index f27eb64bd121e..1b57250c1fb54 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -92,7 +92,7 @@ hive-shims-0.23/2.3.6//hive-shims-0.23-2.3.6.jar hive-shims-common/2.3.6//hive-shims-common-2.3.6.jar hive-shims-scheduler/2.3.6//hive-shims-scheduler-2.3.6.jar hive-shims/2.3.6//hive-shims-2.3.6.jar -hive-storage-api/2.6.0//hive-storage-api-2.6.0.jar +hive-storage-api/2.7.1//hive-storage-api-2.7.1.jar hive-vector-code-gen/2.3.6//hive-vector-code-gen-2.3.6.jar hk2-api/2.6.1//hk2-api-2.6.1.jar hk2-locator/2.6.1//hk2-locator-2.6.1.jar @@ -176,9 +176,9 @@ objenesis/2.5.1//objenesis-2.5.1.jar okhttp/3.12.6//okhttp-3.12.6.jar okio/1.15.0//okio-1.15.0.jar opencsv/2.3//opencsv-2.3.jar -orc-core/1.5.8//orc-core-1.5.8.jar -orc-mapreduce/1.5.8//orc-mapreduce-1.5.8.jar -orc-shims/1.5.8//orc-shims-1.5.8.jar +orc-core/1.5.9//orc-core-1.5.9.jar +orc-mapreduce/1.5.9//orc-mapreduce-1.5.9.jar +orc-shims/1.5.9//orc-shims-1.5.9.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar @@ -211,6 +211,7 @@ stax-api/1.0-2//stax-api-1.0-2.jar stax-api/1.0.1//stax-api-1.0.1.jar stream/2.9.6//stream-2.9.6.jar super-csv/2.2.0//super-csv-2.2.0.jar +threeten-extra/1.5.0//threeten-extra-1.5.0.jar transaction-api/1.1//transaction-api-1.1.jar univocity-parsers/2.8.3//univocity-parsers-2.8.3.jar velocity/1.5//velocity-1.5.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index b1d98fe0f62c9..ffd2364a51317 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -91,7 +91,7 @@ hive-shims-0.23/2.3.6//hive-shims-0.23-2.3.6.jar hive-shims-common/2.3.6//hive-shims-common-2.3.6.jar hive-shims-scheduler/2.3.6//hive-shims-scheduler-2.3.6.jar hive-shims/2.3.6//hive-shims-2.3.6.jar -hive-storage-api/2.6.0//hive-storage-api-2.6.0.jar +hive-storage-api/2.7.1//hive-storage-api-2.7.1.jar hive-vector-code-gen/2.3.6//hive-vector-code-gen-2.3.6.jar hk2-api/2.6.1//hk2-api-2.6.1.jar hk2-locator/2.6.1//hk2-locator-2.6.1.jar @@ -191,9 +191,9 @@ okhttp/2.7.5//okhttp-2.7.5.jar okhttp/3.12.6//okhttp-3.12.6.jar okio/1.15.0//okio-1.15.0.jar opencsv/2.3//opencsv-2.3.jar -orc-core/1.5.8//orc-core-1.5.8.jar -orc-mapreduce/1.5.8//orc-mapreduce-1.5.8.jar -orc-shims/1.5.8//orc-shims-1.5.8.jar +orc-core/1.5.9//orc-core-1.5.9.jar +orc-mapreduce/1.5.9//orc-mapreduce-1.5.9.jar +orc-shims/1.5.9//orc-shims-1.5.9.jar oro/2.0.8//oro-2.0.8.jar osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar paranamer/2.8//paranamer-2.8.jar @@ -227,6 +227,7 @@ stax-api/1.0.1//stax-api-1.0.1.jar stax2-api/3.1.4//stax2-api-3.1.4.jar stream/2.9.6//stream-2.9.6.jar super-csv/2.2.0//super-csv-2.2.0.jar +threeten-extra/1.5.0//threeten-extra-1.5.0.jar token-provider/1.0.1//token-provider-1.0.1.jar transaction-api/1.1//transaction-api-1.1.jar univocity-parsers/2.8.3//univocity-parsers-2.8.3.jar diff --git a/pom.xml b/pom.xml index 6a74b8b2e8e28..eccc620e67690 100644 --- a/pom.xml +++ b/pom.xml @@ -135,7 +135,7 @@ 2.4.0 10.12.1.1 1.10.1 - 1.5.8 + 1.5.9 com.twitter 1.6.0 @@ -227,6 +227,7 @@ compile compile provided + 2.7.1 compile compile compile @@ -2244,7 +2245,7 @@ org.apache.hive hive-storage-api - 2.6.0 + ${hive.storage.version} ${hive.storage.scope} @@ -3006,6 +3007,7 @@ 1.2 ${hive.deps.scope} + 2.6.0 provided provided provided From d0c3e9f1f70eefeaada9336d10848e618442196c Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Fri, 31 Jan 2020 21:04:26 -0600 Subject: [PATCH 1280/1387] [SPARK-30660][ML][PYSPARK] LinearRegression blockify input vectors ### What changes were proposed in this pull request? 1, use blocks instead of vectors for performance improvement 2, use Level-2 BLAS 3, move standardization of input vectors outside of gradient computation ### Why are the changes needed? 1, less RAM to persist training data; (save ~40%) 2, faster than existing impl; (30% ~ 102%) ### Does this PR introduce any user-facing change? add a new expert param `blockSize` ### How was this patch tested? updated testsuites Closes #27396 from zhengruifeng/blockify_lireg. Authored-by: zhengruifeng Signed-off-by: Sean Owen --- .../ml/optim/aggregator/HuberAggregator.scala | 103 ++++++++++++++---- .../aggregator/LeastSquaresAggregator.scala | 74 ++++++++++--- .../ml/regression/LinearRegression.scala | 49 ++++++--- .../aggregator/HuberAggregatorSuite.scala | 61 +++++++++-- .../LeastSquaresAggregatorSuite.scala | 62 +++++++++-- python/pyspark/ml/regression.py | 22 +++- 6 files changed, 289 insertions(+), 82 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala index 8a1a41b2950c1..f83621506500e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.linalg._ /** * HuberAggregator computes the gradient and loss for a huber loss function, @@ -62,19 +62,17 @@ import org.apache.spark.ml.linalg.Vector * * @param fitIntercept Whether to fit an intercept term. * @param epsilon The shape parameter to control the amount of robustness. - * @param bcFeaturesStd The broadcast standard deviation values of the features. * @param bcParameters including three parts: the regression coefficients corresponding * to the features, the intercept (if fitIntercept is ture) * and the scale parameter (sigma). */ private[ml] class HuberAggregator( + numFeatures: Int, fitIntercept: Boolean, - epsilon: Double, - bcFeaturesStd: Broadcast[Array[Double]])(bcParameters: Broadcast[Vector]) - extends DifferentiableLossAggregator[Instance, HuberAggregator] { + epsilon: Double)(bcParameters: Broadcast[Vector]) + extends DifferentiableLossAggregator[InstanceBlock, HuberAggregator] { protected override val dim: Int = bcParameters.value.size - private val numFeatures: Int = if (fitIntercept) dim - 2 else dim - 1 private val sigma: Double = bcParameters.value(dim - 1) private val intercept: Double = if (fitIntercept) { bcParameters.value(dim - 2) @@ -82,7 +80,8 @@ private[ml] class HuberAggregator( 0.0 } // make transient so we do not serialize between aggregation stages - @transient private lazy val coefficients = bcParameters.value.toArray.slice(0, numFeatures) + @transient private lazy val linear = + new DenseVector(bcParameters.value.toArray.take(numFeatures)) /** * Add a new training instance to this HuberAggregator, and update the loss and gradient @@ -98,16 +97,13 @@ private[ml] class HuberAggregator( require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") if (weight == 0.0) return this - val localFeaturesStd = bcFeaturesStd.value - val localCoefficients = coefficients + val localCoefficients = linear.values val localGradientSumArray = gradientSumArray val margin = { var sum = 0.0 features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - sum += localCoefficients(index) * (value / localFeaturesStd(index)) - } + sum += localCoefficients(index) * value } if (fitIntercept) sum += intercept sum @@ -119,10 +115,7 @@ private[ml] class HuberAggregator( val linearLossDivSigma = linearLoss / sigma features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - localGradientSumArray(index) += - -1.0 * weight * linearLossDivSigma * (value / localFeaturesStd(index)) - } + localGradientSumArray(index) -= weight * linearLossDivSigma * value } if (fitIntercept) { localGradientSumArray(dim - 2) += -1.0 * weight * linearLossDivSigma @@ -134,10 +127,7 @@ private[ml] class HuberAggregator( (sigma + 2.0 * epsilon * math.abs(linearLoss) - sigma * epsilon * epsilon) features.foreachNonZero { (index, value) => - if (localFeaturesStd(index) != 0.0) { - localGradientSumArray(index) += - weight * sign * epsilon * (value / localFeaturesStd(index)) - } + localGradientSumArray(index) += weight * sign * epsilon * value } if (fitIntercept) { localGradientSumArray(dim - 2) += weight * sign * epsilon @@ -149,4 +139,75 @@ private[ml] class HuberAggregator( this } } + + /** + * Add a new training instance block to this HuberAggregator, and update the loss and gradient + * of the objective function. + * + * @param block The instance block of data point to be added. + * @return This HuberAggregator object. + */ + def add(block: InstanceBlock): HuberAggregator = { + require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + + s"instance. Expecting $numFeatures but got ${block.numFeatures}.") + require(block.weightIter.forall(_ >= 0), + s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") + + if (block.weightIter.forall(_ == 0)) return this + val size = block.size + val localGradientSumArray = gradientSumArray + + // vec here represents margins or dotProducts + val vec = if (fitIntercept && intercept != 0) { + new DenseVector(Array.fill(size)(intercept)) + } else { + new DenseVector(Array.ofDim[Double](size)) + } + + if (fitIntercept) { + BLAS.gemv(1.0, block.matrix, linear, 1.0, vec) + } else { + BLAS.gemv(1.0, block.matrix, linear, 0.0, vec) + } + + // in-place convert margins to multipliers + // then, vec represents multipliers + var i = 0 + while (i < size) { + val weight = block.getWeight(i) + if (weight > 0) { + weightSum += weight + val label = block.getLabel(i) + val margin = vec(i) + val linearLoss = label - margin + + if (math.abs(linearLoss) <= sigma * epsilon) { + lossSum += 0.5 * weight * (sigma + math.pow(linearLoss, 2.0) / sigma) + val linearLossDivSigma = linearLoss / sigma + val multiplier = -1.0 * weight * linearLossDivSigma + vec.values(i) = multiplier + localGradientSumArray(dim - 1) += 0.5 * weight * (1.0 - math.pow(linearLossDivSigma, 2.0)) + } else { + lossSum += 0.5 * weight * + (sigma + 2.0 * epsilon * math.abs(linearLoss) - sigma * epsilon * epsilon) + val sign = if (linearLoss >= 0) -1.0 else 1.0 + val multiplier = weight * sign * epsilon + vec.values(i) = multiplier + localGradientSumArray(dim - 1) += 0.5 * weight * (1.0 - epsilon * epsilon) + } + } else { + vec.values(i) = 0.0 + } + i += 1 + } + + val linearGradSumVec = new DenseVector(Array.ofDim[Double](numFeatures)) + BLAS.gemv(1.0, block.matrix.transpose, vec, 0.0, linearGradSumVec) + linearGradSumVec.foreachNonZero { (i, v) => localGradientSumArray(i) += v } + if (fitIntercept) { + localGradientSumArray(dim - 2) += vec.values.sum + } + + this + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala index 7a5806dc24aee..a8bda9ca5d245 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.linalg._ /** * LeastSquaresAggregator computes the gradient and loss for a Least-squared loss function, @@ -157,26 +157,25 @@ private[ml] class LeastSquaresAggregator( labelStd: Double, labelMean: Double, fitIntercept: Boolean, - bcFeaturesStd: Broadcast[Array[Double]], - bcFeaturesMean: Broadcast[Array[Double]])(bcCoefficients: Broadcast[Vector]) - extends DifferentiableLossAggregator[Instance, LeastSquaresAggregator] { + bcFeaturesStd: Broadcast[Vector], + bcFeaturesMean: Broadcast[Vector])(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[InstanceBlock, LeastSquaresAggregator] { require(labelStd > 0.0, s"${this.getClass.getName} requires the label standard " + s"deviation to be positive.") - private val numFeatures = bcFeaturesStd.value.length + private val numFeatures = bcFeaturesStd.value.size protected override val dim: Int = numFeatures // make transient so we do not serialize between aggregation stages - @transient private lazy val featuresStd = bcFeaturesStd.value @transient private lazy val effectiveCoefAndOffset = { val coefficientsArray = bcCoefficients.value.toArray.clone() val featuresMean = bcFeaturesMean.value + val featuresStd = bcFeaturesStd.value var sum = 0.0 var i = 0 val len = coefficientsArray.length while (i < len) { if (featuresStd(i) != 0.0) { - coefficientsArray(i) /= featuresStd(i) - sum += coefficientsArray(i) * featuresMean(i) + sum += coefficientsArray(i) / featuresStd(i) * featuresMean(i) } else { coefficientsArray(i) = 0.0 } @@ -186,7 +185,7 @@ private[ml] class LeastSquaresAggregator( (Vectors.dense(coefficientsArray), offset) } // do not use tuple assignment above because it will circumvent the @transient tag - @transient private lazy val effectiveCoefficientsVector = effectiveCoefAndOffset._1 + @transient private lazy val effectiveCoefficientsVec = effectiveCoefAndOffset._1 @transient private lazy val offset = effectiveCoefAndOffset._2 /** @@ -204,16 +203,20 @@ private[ml] class LeastSquaresAggregator( if (weight == 0.0) return this - val diff = BLAS.dot(features, effectiveCoefficientsVector) - label / labelStd + offset + val localEffectiveCoefficientsVec = effectiveCoefficientsVec + + val diff = { + var dot = 0.0 + features.foreachNonZero { (index, value) => + dot += localEffectiveCoefficientsVec(index) * value + } + dot - label / labelStd + offset + } if (diff != 0) { val localGradientSumArray = gradientSumArray - val localFeaturesStd = featuresStd features.foreachNonZero { (index, value) => - val fStd = localFeaturesStd(index) - if (fStd != 0.0) { - localGradientSumArray(index) += weight * diff * value / fStd - } + localGradientSumArray(index) += weight * diff * value } lossSum += weight * diff * diff / 2.0 } @@ -221,4 +224,43 @@ private[ml] class LeastSquaresAggregator( this } } + + /** + * Add a new training instance block to this LeastSquaresAggregator, and update the loss + * and gradient of the objective function. + * + * @param block The instance block of data point to be added. + * @return This LeastSquaresAggregator object. + */ + def add(block: InstanceBlock): LeastSquaresAggregator = { + require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + + s"instance. Expecting $numFeatures but got ${block.numFeatures}.") + require(block.weightIter.forall(_ >= 0), + s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") + + if (block.weightIter.forall(_ == 0)) return this + val size = block.size + + // vec here represents diffs + val vec = new DenseVector(Array.tabulate(size)(i => offset - block.getLabel(i) / labelStd)) + BLAS.gemv(1.0, block.matrix, effectiveCoefficientsVec, 1.0, vec) + + // in-place convert diffs to multipliers + // then, vec represents multipliers + var i = 0 + while (i < size) { + val weight = block.getWeight(i) + val diff = vec(i) + lossSum += weight * diff * diff / 2 + weightSum += weight + val multiplier = weight * diff + vec.values(i) = multiplier + i += 1 + } + + val gradSumVec = new DenseVector(gradientSumArray) + BLAS.gemv(1.0, block.matrix.transpose, vec, 1.0, gradSumVec) + + this + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 64e5e191ffd17..fc59da8a9c165 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -28,7 +28,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{PipelineStage, PredictorParams} -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.optim.WeightedLeastSquares @@ -55,7 +55,7 @@ import org.apache.spark.util.VersionUtils.majorMinorVersion private[regression] trait LinearRegressionParams extends PredictorParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver - with HasAggregationDepth with HasLoss { + with HasAggregationDepth with HasLoss with HasBlockSize { import LinearRegression._ @@ -316,6 +316,15 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String def setEpsilon(value: Double): this.type = set(epsilon, value) setDefault(epsilon -> 1.35) + /** + * Set block size for stacking input data in matrices. + * Default is 1024. + * + * @group expertSetParam + */ + @Since("3.0.0") + def setBlockSize(value: Int): this.type = set(blockSize, value) + override protected def train(dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => // Extract the number of features before deciding optimization solver. val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) @@ -354,9 +363,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String return lrModel.setSummary(Some(trainingSummary)) } - val handlePersistence = dataset.storageLevel == StorageLevel.NONE - if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) - val (featuresSummarizer, ySummarizer) = instances.treeAggregate( (Summarizer.createSummarizerBuffer("mean", "std"), Summarizer.createSummarizerBuffer("mean", "std", "count")))( @@ -392,7 +398,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String s"will be zeros and the intercept will be the mean of the label; as a result, " + s"training is not needed.") } - if (handlePersistence) instances.unpersist() val coefficients = Vectors.sparse(numFeatures, Seq.empty) val intercept = yMean @@ -421,8 +426,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String // if y is constant (rawYStd is zero), then y cannot be scaled. In this case // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) - val featuresMean = featuresSummarizer.mean.toArray - val featuresStd = featuresSummarizer.std.toArray + val featuresMean = featuresSummarizer.mean.compressed + val featuresStd = featuresSummarizer.std.compressed val bcFeaturesMean = instances.context.broadcast(featuresMean) val bcFeaturesStd = instances.context.broadcast(featuresStd) @@ -442,23 +447,36 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val effectiveL1RegParam = $(elasticNetParam) * effectiveRegParam val effectiveL2RegParam = (1.0 - $(elasticNetParam)) * effectiveRegParam - val getFeaturesStd = (j: Int) => if (j >= 0 && j < numFeatures) featuresStd(j) else 0.0 val regularization = if (effectiveL2RegParam != 0.0) { val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures Some(new L2Regularization(effectiveL2RegParam, shouldApply, - if ($(standardization)) None else Some(getFeaturesStd))) + if ($(standardization)) None else Some(featuresStd.apply))) } else { None } + val standardized = instances.map { + case Instance(label, weight, features) => + val featuresStd = bcFeaturesStd.value + val array = Array.ofDim[Double](numFeatures) + features.foreachNonZero { (i, v) => + val std = featuresStd(i) + if (std != 0) array(i) = v / std + } + Instance(label, weight, Vectors.dense(array)) + } + val blocks = InstanceBlock.blokify(standardized, $(blockSize)) + .persist(StorageLevel.MEMORY_AND_DISK) + .setName(s"training dataset (blockSize=${$(blockSize)})") + val costFun = $(loss) match { case SquaredError => val getAggregatorFunc = new LeastSquaresAggregator(yStd, yMean, $(fitIntercept), bcFeaturesStd, bcFeaturesMean)(_) - new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) + new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) case Huber => - val getAggregatorFunc = new HuberAggregator($(fitIntercept), $(epsilon), bcFeaturesStd)(_) - new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) + val getAggregatorFunc = new HuberAggregator(numFeatures, $(fitIntercept), $(epsilon))(_) + new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) } val optimizer = $(loss) match { @@ -524,6 +542,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String throw new SparkException(msg) } + blocks.unpersist() bcFeaturesMean.destroy() bcFeaturesStd.destroy() @@ -557,7 +576,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String after the coefficients are converged. See the following discussion for detail. http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet */ - yMean - dot(Vectors.dense(rawCoefficients), Vectors.dense(featuresMean)) + yMean - dot(Vectors.dense(rawCoefficients), featuresMean) case Huber => parameters(numFeatures) } } else { @@ -572,8 +591,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String (Vectors.dense(rawCoefficients).compressed, interceptValue, scaleValue, arrayBuilder.result()) } - if (handlePersistence) instances.unpersist() - val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept, scale)) // Handle possible missing or invalid prediction columns val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberAggregatorSuite.scala index 718ffa230a749..7c544e99f88b5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberAggregatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberAggregatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,21 +32,21 @@ class HuberAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { override def beforeAll(): Unit = { super.beforeAll() - instances = Array( + instances = standardize(Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) - ) - instancesConstantFeature = Array( + )) + instancesConstantFeature = standardize(Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) - ) - instancesConstantFeatureFiltered = Array( + )) + instancesConstantFeatureFiltered = standardize(Array( Instance(0.0, 0.1, Vectors.dense(2.0)), Instance(1.0, 0.5, Vectors.dense(1.0)), Instance(2.0, 0.3, Vectors.dense(0.5)) - ) + )) } /** Get summary statistics for some data and create a new HuberAggregator. */ @@ -56,10 +56,28 @@ class HuberAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { fitIntercept: Boolean, epsilon: Double): HuberAggregator = { val (featuresSummarizer, _) = getRegressionSummarizers(instances) - val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) - val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) + val numFeatures = featuresSummarizer.variance.size val bcParameters = spark.sparkContext.broadcast(parameters) - new HuberAggregator(fitIntercept, epsilon, bcFeaturesStd)(bcParameters) + new HuberAggregator(numFeatures, fitIntercept, epsilon)(bcParameters) + } + + private def standardize( + instances: Array[Instance], + std: Array[Double] = null): Array[Instance] = { + val stdArray = if (std == null) { + getRegressionSummarizers(instances)._1.variance.toArray.map(math.sqrt) + } else { + std + } + val numFeatures = stdArray.length + instances.map { case Instance(label, weight, features) => + val standardized = Array.ofDim[Double](numFeatures) + features.foreachNonZero { (i, v) => + val std = stdArray(i) + if (std != 0) standardized(i) = v / std + } + Instance(label, weight, Vectors.dense(standardized).compressed) + } } test("aggregator add method should check input size") { @@ -155,9 +173,15 @@ class HuberAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { val parametersFiltered = Vectors.dense(2.0, 3.0, 4.0) val aggConstantFeature = getNewAggregator(instancesConstantFeature, parameters, fitIntercept = true, epsilon = 1.35) + // std of instancesConstantFeature + val stdConstantFeature = getRegressionSummarizers(instancesConstantFeature) + ._1.variance.toArray.map(math.sqrt) + // Since 3.0.0, we start to standardize input outside of gradient computation, + // so here we use std of instancesConstantFeature to standardize instances + standardize(instances, stdConstantFeature).foreach(aggConstantFeature.add) + val aggConstantFeatureFiltered = getNewAggregator(instancesConstantFeatureFiltered, parametersFiltered, fitIntercept = true, epsilon = 1.35) - instances.foreach(aggConstantFeature.add) instancesConstantFeatureFiltered.foreach(aggConstantFeatureFiltered.add) // constant features should not affect gradient def validateGradient(grad: Vector, gradFiltered: Vector): Unit = { @@ -167,4 +191,19 @@ class HuberAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { validateGradient(aggConstantFeature.gradient, aggConstantFeatureFiltered.gradient) } + + test("add instance block") { + val paramWithIntercept = Vectors.dense(1.0, 2.0, 3.0, 4.0) + val agg1 = getNewAggregator(instances, paramWithIntercept, + fitIntercept = true, epsilon = 1.35) + instances.foreach(agg1.add) + + val agg2 = getNewAggregator(instances, paramWithIntercept, + fitIntercept = true, epsilon = 1.35) + val block = InstanceBlock.fromInstances(instances) + agg2.add(block) + + assert(agg1.loss ~== agg2.loss relTol 1e-8) + assert(agg1.gradient ~== agg2.gradient relTol 1e-8) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala index 35b6944624707..5eb4e41c58262 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.feature.{Instance, InstanceBlock} import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,21 +32,21 @@ class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkConte override def beforeAll(): Unit = { super.beforeAll() - instances = Array( + instances = standardize(Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) - ) - instancesConstantFeature = Array( + )) + instancesConstantFeature = standardize(Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) - ) - instancesConstantLabel = Array( + )) + instancesConstantLabel = standardize(Array( Instance(1.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(1.0, 0.3, Vectors.dense(4.0, 0.5)) - ) + )) } /** Get summary statistics for some data and create a new LeastSquaresAggregator. */ @@ -57,15 +57,34 @@ class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkConte val (featuresSummarizer, ySummarizer) = getRegressionSummarizers(instances) val yStd = math.sqrt(ySummarizer.variance(0)) val yMean = ySummarizer.mean(0) - val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val featuresStd = Vectors.dense(featuresSummarizer.variance.toArray.map(math.sqrt)) val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) - val featuresMean = featuresSummarizer.mean - val bcFeaturesMean = spark.sparkContext.broadcast(featuresMean.toArray) - val bcCoefficients = spark.sparkContext.broadcast(coefficients) + val featuresMean = featuresSummarizer.mean.asML + val bcFeaturesMean = spark.sparkContext.broadcast(featuresMean.compressed) + val bcCoefficients = spark.sparkContext.broadcast(coefficients.compressed) new LeastSquaresAggregator(yStd, yMean, fitIntercept, bcFeaturesStd, bcFeaturesMean)(bcCoefficients) } + private def standardize( + instances: Array[Instance], + std: Array[Double] = null): Array[Instance] = { + val stdArray = if (std == null) { + getRegressionSummarizers(instances)._1.variance.toArray.map(math.sqrt) + } else { + std + } + val numFeatures = stdArray.length + instances.map { case Instance(label, weight, features) => + val standardized = Array.ofDim[Double](numFeatures) + features.foreachNonZero { (i, v) => + val std = stdArray(i) + if (std != 0) standardized(i) = v / std + } + Instance(label, weight, Vectors.dense(standardized).compressed) + } + } + test("aggregator add method input size") { val coefficients = Vectors.dense(1.0, 2.0) val agg = getNewAggregator(instances, coefficients, fitIntercept = true) @@ -145,9 +164,15 @@ class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkConte test("check with zero standard deviation") { val coefficients = Vectors.dense(1.0, 2.0) + // aggConstantFeature contains std of instancesConstantFeature, and the std of dim=0 is 0 val aggConstantFeature = getNewAggregator(instancesConstantFeature, coefficients, fitIntercept = true) - instances.foreach(aggConstantFeature.add) + // std of instancesConstantFeature + val stdConstantFeature = getRegressionSummarizers(instancesConstantFeature) + ._1.variance.toArray.map(math.sqrt) + // Since 3.0.0, we start to standardize input outside of gradient computation, + // so here we use std of instancesConstantFeature to standardize instances + standardize(instances, stdConstantFeature).foreach(aggConstantFeature.add) // constant features should not affect gradient assert(aggConstantFeature.gradient(0) === 0.0) @@ -157,4 +182,17 @@ class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkConte } } } + + test("add instance block") { + val coefficients = Vectors.dense(1.0, 2.0) + val agg1 = getNewAggregator(instances, coefficients, fitIntercept = true) + instances.foreach(agg1.add) + + val agg2 = getNewAggregator(instances, coefficients, fitIntercept = true) + val block = InstanceBlock.fromInstances(instances) + agg2.add(block) + + assert(agg1.loss ~== agg2.loss relTol 1e-8) + assert(agg1.gradient ~== agg2.gradient relTol 1e-8) + } } diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index a4c97827d55de..a74ba3485bcd9 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -62,7 +62,7 @@ class JavaRegressionModel(JavaPredictionModel, _JavaPredictorParams): class _LinearRegressionParams(_JavaPredictorParams, HasRegParam, HasElasticNetParam, HasMaxIter, HasTol, HasFitIntercept, HasStandardization, HasWeightCol, HasSolver, - HasAggregationDepth, HasLoss): + HasAggregationDepth, HasLoss, HasBlockSize): """ Params for :py:class:`LinearRegression` and :py:class:`LinearRegressionModel`. @@ -124,6 +124,8 @@ class LinearRegression(JavaRegressor, _LinearRegressionParams, JavaMLWritable, J >>> lr.setRegParam(0.0) LinearRegression... >>> model = lr.fit(df) + >>> model.getBlockSize() + 1024 >>> model.setFeaturesCol("features") LinearRegressionModel... >>> model.setPredictionCol("newPrediction") @@ -169,17 +171,18 @@ class LinearRegression(JavaRegressor, _LinearRegressionParams, JavaMLWritable, J def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, standardization=True, solver="auto", weightCol=None, aggregationDepth=2, - loss="squaredError", epsilon=1.35): + loss="squaredError", epsilon=1.35, blockSize=1024): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, \ standardization=True, solver="auto", weightCol=None, aggregationDepth=2, \ - loss="squaredError", epsilon=1.35) + loss="squaredError", epsilon=1.35, blockSize=1024) """ super(LinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.LinearRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35) + self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35, + blockSize=1024) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -188,12 +191,12 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, standardization=True, solver="auto", weightCol=None, aggregationDepth=2, - loss="squaredError", epsilon=1.35): + loss="squaredError", epsilon=1.35, blockSize=1024): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, \ standardization=True, solver="auto", weightCol=None, aggregationDepth=2, \ - loss="squaredError", epsilon=1.35) + loss="squaredError", epsilon=1.35, blockSize=1024) Sets params for linear regression. """ kwargs = self._input_kwargs @@ -269,6 +272,13 @@ def setLoss(self, value): """ return self._set(lossType=value) + @since("3.0.0") + def setBlockSize(self, value): + """ + Sets the value of :py:attr:`blockSize`. + """ + return self._set(blockSize=value) + class LinearRegressionModel(JavaRegressionModel, _LinearRegressionParams, GeneralJavaMLWritable, JavaMLReadable, HasTrainingSummary): From 8eecc20b1191fa98814e488fd654f6319da8f715 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 31 Jan 2020 19:55:25 -0800 Subject: [PATCH 1281/1387] [SPARK-27946][SQL] Hive DDL to Spark DDL conversion USING "show create table" ## What changes were proposed in this pull request? This patch adds a DDL command `SHOW CREATE TABLE AS SERDE`. It is used to generate Hive DDL for a Hive table. For original `SHOW CREATE TABLE`, it now shows Spark DDL always. If given a Hive table, it tries to generate Spark DDL. For Hive serde to data source conversion, this uses the existing mapping inside `HiveSerDe`. If can't find a mapping there, throws an analysis exception on unsupported serde configuration. It is arguably that some Hive fileformat + row serde might be mapped to Spark data source, e.g., CSV. It is not included in this PR. To be conservative, it may not be supported. For Hive serde properties, for now this doesn't save it to Spark DDL because it may not useful to keep Hive serde properties in Spark table. ## How was this patch tested? Added test. Closes #24938 from viirya/SPARK-27946. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Xiao Li --- docs/sql-migration-guide.md | 2 + .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../catalyst/plans/logical/statements.scala | 4 +- .../analysis/ResolveSessionCatalog.scala | 6 +- .../spark/sql/execution/command/tables.scala | 285 ++++++++++----- .../apache/spark/sql/internal/HiveSerDe.scala | 16 + .../sql-tests/inputs/show-create-table.sql | 11 +- .../results/show-create-table.sql.out | 34 +- .../spark/sql/ShowCreateTableSuite.scala | 16 +- .../sql/hive/HiveShowCreateTableSuite.scala | 327 +++++++++++++++++- 11 files changed, 581 insertions(+), 124 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 18e7df7aca5aa..533c96a0832de 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -328,6 +328,8 @@ license: | - Since Spark 3.0, `SHOW TBLPROPERTIES` will cause `AnalysisException` if the table does not exist. In Spark version 2.4 and earlier, this scenario caused `NoSuchTableException`. Also, `SHOW TBLPROPERTIES` on a temporary view will cause `AnalysisException`. In Spark version 2.4 and earlier, it returned an empty result. + - Since Spark 3.0, `SHOW CREATE TABLE` will always return Spark DDL, even when the given table is a Hive serde table. For Hive DDL, please use `SHOW CREATE TABLE AS SERDE` command instead. + ## Upgrading from Spark SQL 2.4.4 to 2.4.5 - Since Spark 2.4.5, `TRUNCATE TABLE` command tries to set back original permission and ACLs during re-creating the table/partition paths. To restore the behaviour of earlier versions, set `spark.sql.truncateTable.ignorePermissionAcl.enabled` to `true`. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 358943fc484ca..6f2bb7a9a7536 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -210,7 +210,7 @@ statement | SHOW PARTITIONS multipartIdentifier partitionSpec? #showPartitions | SHOW identifier? FUNCTIONS (LIKE? (multipartIdentifier | pattern=STRING))? #showFunctions - | SHOW CREATE TABLE multipartIdentifier #showCreateTable + | SHOW CREATE TABLE multipartIdentifier (AS SERDE)? #showCreateTable | SHOW CURRENT NAMESPACE #showCurrentNamespace | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction | (DESC | DESCRIBE) namespace EXTENDED? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 967eca77145e2..e9ad84472904d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3215,7 +3215,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Creates a [[ShowCreateTableStatement]] */ override def visitShowCreateTable(ctx: ShowCreateTableContext): LogicalPlan = withOrigin(ctx) { - ShowCreateTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier())) + ShowCreateTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier()), ctx.SERDE != null) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 44f7b4143926d..1e6b67bf78b70 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -389,7 +389,9 @@ case class LoadDataStatement( /** * A SHOW CREATE TABLE statement, as parsed from SQL. */ -case class ShowCreateTableStatement(tableName: Seq[String]) extends ParsedStatement +case class ShowCreateTableStatement( + tableName: Seq[String], + asSerde: Boolean = false) extends ParsedStatement /** * A CACHE TABLE statement, as parsed from SQL diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 12439fd94c679..486e7f1f84b46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -378,10 +378,14 @@ class ResolveSessionCatalog( isOverwrite, partition) - case ShowCreateTableStatement(tbl) => + case ShowCreateTableStatement(tbl, asSerde) if !asSerde => val v1TableName = parseV1Table(tbl, "SHOW CREATE TABLE") ShowCreateTableCommand(v1TableName.asTableIdentifier) + case ShowCreateTableStatement(tbl, asSerde) if asSerde => + val v1TableName = parseV1Table(tbl, "SHOW CREATE TABLE AS SERDE") + ShowCreateTableAsSerdeCommand(v1TableName.asTableIdentifier) + case CacheTableStatement(tbl, plan, isLazy, options) => val v1TableName = parseV1Table(tbl, "CACHE TABLE") CacheTableCommand(v1TableName.asTableIdentifier, plan, isLazy, options) 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 a92fbdf25975b..468ca505cce1f 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 @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.datasources.v2.csv.CSVDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.json.JsonDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -1002,7 +1002,57 @@ case class ShowPartitionsCommand( } } -case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableCommand { +/** + * Provides common utilities between `ShowCreateTableCommand` and `ShowCreateTableAsSparkCommand`. + */ +trait ShowCreateTableCommandBase { + + protected val table: TableIdentifier + + protected def showTableLocation(metadata: CatalogTable, builder: StringBuilder): Unit = { + if (metadata.tableType == EXTERNAL) { + metadata.storage.locationUri.foreach { location => + builder ++= s"LOCATION '${escapeSingleQuotedString(CatalogUtils.URIToString(location))}'\n" + } + } + } + + protected def showTableComment(metadata: CatalogTable, builder: StringBuilder): Unit = { + metadata + .comment + .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") + .foreach(builder.append) + } + + protected def showTableProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { + if (metadata.properties.nonEmpty) { + val props = metadata.properties.map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + + builder ++= "TBLPROPERTIES " + builder ++= concatByMultiLines(props) + } + } + + + protected def concatByMultiLines(iter: Iterable[String]): String = { + iter.mkString("(\n ", ",\n ", ")\n") + } +} + +/** + * A command that shows the Spark DDL syntax that can be used to create a given table. + * For Hive serde table, this command will generate Spark DDL that can be used to + * create corresponding Spark table. + * + * The syntax of using this command in SQL is: + * {{{ + * SHOW CREATE TABLE [db_name.]table_name + * }}} + */ +case class ShowCreateTableCommand(table: TableIdentifier) + extends RunnableCommand with ShowCreateTableCommandBase { override val output: Seq[Attribute] = Seq( AttributeReference("createtab_stmt", StringType, nullable = false)() ) @@ -1017,16 +1067,154 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman // TODO: [SPARK-28692] unify this after we unify the // CREATE TABLE syntax for hive serde and data source table. - val stmt = if (DDLUtils.isDatasourceTable(tableMetadata)) { - showCreateDataSourceTable(tableMetadata) + val metadata = if (DDLUtils.isDatasourceTable(tableMetadata)) { + tableMetadata } else { - showCreateHiveTable(tableMetadata) + // For a Hive serde table, we try to convert it to Spark DDL. + if (tableMetadata.unsupportedFeatures.nonEmpty) { + throw new AnalysisException( + "Failed to execute SHOW CREATE TABLE against table " + + s"${tableMetadata.identifier}, which is created by Hive and uses the " + + "following unsupported feature(s)\n" + + tableMetadata.unsupportedFeatures.map(" - " + _).mkString("\n") + ) + } + + if (tableMetadata.tableType == VIEW) { + throw new AnalysisException("Hive view isn't supported by SHOW CREATE TABLE") + } + + if ("true".equalsIgnoreCase(tableMetadata.properties.getOrElse("transactional", "false"))) { + throw new AnalysisException( + "SHOW CREATE TABLE doesn't support transactional Hive table") + } + + convertTableMetadata(tableMetadata) } + val stmt = showCreateDataSourceTable(metadata) + Seq(Row(stmt)) } } + private def convertTableMetadata(tableMetadata: CatalogTable): CatalogTable = { + val hiveSerde = HiveSerDe( + serde = tableMetadata.storage.serde, + inputFormat = tableMetadata.storage.inputFormat, + outputFormat = tableMetadata.storage.outputFormat) + + // Looking for Spark data source that maps to to the Hive serde. + // TODO: some Hive fileformat + row serde might be mapped to Spark data source, e.g. CSV. + val source = HiveSerDe.serdeToSource(hiveSerde) + if (source.isEmpty) { + val builder = StringBuilder.newBuilder + hiveSerde.serde.foreach { serde => + builder ++= s" SERDE: $serde" + } + hiveSerde.inputFormat.foreach { format => + builder ++= s" INPUTFORMAT: $format" + } + hiveSerde.outputFormat.foreach { format => + builder ++= s" OUTPUTFORMAT: $format" + } + throw new AnalysisException( + "Failed to execute SHOW CREATE TABLE against table " + + s"${tableMetadata.identifier}, which is created by Hive and uses the " + + "following unsupported serde configuration\n" + + builder.toString() + ) + } else { + // TODO: should we keep Hive serde properties? + val newStorage = tableMetadata.storage.copy(properties = Map.empty) + tableMetadata.copy(provider = source, storage = newStorage) + } + } + + private def showDataSourceTableDataColumns( + metadata: CatalogTable, builder: StringBuilder): Unit = { + val columns = metadata.schema.fields.map(_.toDDL) + builder ++= concatByMultiLines(columns) + } + + private def showDataSourceTableOptions(metadata: CatalogTable, builder: StringBuilder): Unit = { + // For datasource table, there is a provider there in the metadata. + // If it is a Hive table, we already convert its metadata and fill in a provider. + builder ++= s"USING ${metadata.provider.get}\n" + + val dataSourceOptions = SQLConf.get.redactOptions(metadata.storage.properties).map { + case (key, value) => s"${quoteIdentifier(key)} '${escapeSingleQuotedString(value)}'" + } + + if (dataSourceOptions.nonEmpty) { + builder ++= "OPTIONS " + builder ++= concatByMultiLines(dataSourceOptions) + } + } + + private def showDataSourceTableNonDataColumns( + metadata: CatalogTable, builder: StringBuilder): Unit = { + val partCols = metadata.partitionColumnNames + if (partCols.nonEmpty) { + builder ++= s"PARTITIONED BY ${partCols.mkString("(", ", ", ")")}\n" + } + + metadata.bucketSpec.foreach { spec => + if (spec.bucketColumnNames.nonEmpty) { + builder ++= s"CLUSTERED BY ${spec.bucketColumnNames.mkString("(", ", ", ")")}\n" + + if (spec.sortColumnNames.nonEmpty) { + builder ++= s"SORTED BY ${spec.sortColumnNames.mkString("(", ", ", ")")}\n" + } + + builder ++= s"INTO ${spec.numBuckets} BUCKETS\n" + } + } + } + + private def showCreateDataSourceTable(metadata: CatalogTable): String = { + val builder = StringBuilder.newBuilder + + builder ++= s"CREATE TABLE ${table.quotedString} " + showDataSourceTableDataColumns(metadata, builder) + showDataSourceTableOptions(metadata, builder) + showDataSourceTableNonDataColumns(metadata, builder) + showTableComment(metadata, builder) + showTableLocation(metadata, builder) + showTableProperties(metadata, builder) + + builder.toString() + } +} + +/** + * This commands generates the DDL for Hive serde table. + * + * The syntax of using this command in SQL is: + * {{{ + * SHOW CREATE TABLE table_identifier AS SERDE; + * }}} + */ +case class ShowCreateTableAsSerdeCommand(table: TableIdentifier) + extends RunnableCommand with ShowCreateTableCommandBase { + override val output: Seq[Attribute] = Seq( + AttributeReference("createtab_stmt", StringType, nullable = false)() + ) + + override def run(sparkSession: SparkSession): Seq[Row] = { + val catalog = sparkSession.sessionState.catalog + val tableMetadata = catalog.getTableMetadata(table) + + val stmt = if (DDLUtils.isDatasourceTable(tableMetadata)) { + throw new AnalysisException( + s"$table is a Spark data source table. Use `SHOW CREATE TABLE` without `AS SERDE` instead.") + } else { + showCreateHiveTable(tableMetadata) + } + + Seq(Row(stmt)) + } + private def showCreateHiveTable(metadata: CatalogTable): String = { def reportUnsupportedError(features: Seq[String]): Unit = { throw new AnalysisException( @@ -1055,12 +1243,12 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman if (metadata.tableType == VIEW) { showViewDataColumns(metadata, builder) - showComment(metadata, builder) + showTableComment(metadata, builder) showViewProperties(metadata, builder) showViewText(metadata, builder) } else { showHiveTableHeader(metadata, builder) - showComment(metadata, builder) + showTableComment(metadata, builder) showHiveTableNonDataColumns(metadata, builder) showHiveTableStorageInfo(metadata, builder) showTableLocation(metadata, builder) @@ -1084,10 +1272,6 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman } } - private def concatByMultiLines(iter: Iterable[String]): String = { - iter.mkString("(\n ", ",\n ", ")\n") - } - private def showViewProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { val viewProps = metadata.properties.filterKeys(!_.startsWith(CatalogTable.VIEW_PREFIX)) if (viewProps.nonEmpty) { @@ -1136,7 +1320,7 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman storage.serde.foreach { serde => builder ++= s"ROW FORMAT SERDE '$serde'\n" - val serdeProps = metadata.storage.properties.map { + val serdeProps = SQLConf.get.redactOptions(metadata.storage.properties).map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" } @@ -1156,81 +1340,4 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman } } } - - private def showTableLocation(metadata: CatalogTable, builder: StringBuilder): Unit = { - if (metadata.tableType == EXTERNAL) { - metadata.storage.locationUri.foreach { location => - builder ++= s"LOCATION '${escapeSingleQuotedString(CatalogUtils.URIToString(location))}'\n" - } - } - } - - private def showComment(metadata: CatalogTable, builder: StringBuilder): Unit = { - metadata - .comment - .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") - .foreach(builder.append) - } - - private def showTableProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { - if (metadata.properties.nonEmpty) { - val props = metadata.properties.map { case (key, value) => - s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" - } - - builder ++= s"TBLPROPERTIES ${concatByMultiLines(props)}" - } - } - - private def showCreateDataSourceTable(metadata: CatalogTable): String = { - val builder = StringBuilder.newBuilder - - builder ++= s"CREATE TABLE ${table.quotedString} " - showDataSourceTableDataColumns(metadata, builder) - showDataSourceTableOptions(metadata, builder) - showDataSourceTableNonDataColumns(metadata, builder) - showComment(metadata, builder) - showTableLocation(metadata, builder) - showTableProperties(metadata, builder) - - builder.toString() - } - - private def showDataSourceTableDataColumns( - metadata: CatalogTable, builder: StringBuilder): Unit = { - val columns = metadata.schema.fields.map(_.toDDL) - builder ++= concatByMultiLines(columns) - } - - private def showDataSourceTableOptions(metadata: CatalogTable, builder: StringBuilder): Unit = { - builder ++= s"USING ${metadata.provider.get}\n" - - val dataSourceOptions = SQLConf.get.redactOptions(metadata.storage.properties).map { - case (key, value) => s"${quoteIdentifier(key)} '${escapeSingleQuotedString(value)}'" - } - - if (dataSourceOptions.nonEmpty) { - builder ++= s"OPTIONS ${concatByMultiLines(dataSourceOptions)}" - } - } - - private def showDataSourceTableNonDataColumns( - metadata: CatalogTable, builder: StringBuilder): Unit = { - val partCols = metadata.partitionColumnNames - if (partCols.nonEmpty) { - builder ++= s"PARTITIONED BY ${partCols.mkString("(", ", ", ")")}\n" - } - - metadata.bucketSpec.foreach { spec => - if (spec.bucketColumnNames.nonEmpty) { - builder ++= s"CLUSTERED BY ${spec.bucketColumnNames.mkString("(", ", ", ")")}\n" - - if (spec.sortColumnNames.nonEmpty) { - builder ++= s"SORTED BY ${spec.sortColumnNames.mkString("(", ", ", ")")}\n" - } - - builder ++= s"INTO ${spec.numBuckets} BUCKETS\n" - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala index 4921e3ca903c4..64b7e7fe7923a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala @@ -65,6 +65,14 @@ object HiveSerDe { outputFormat = Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"), serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe"))) + // `HiveSerDe` in `serdeMap` should be dintinct. + val serdeInverseMap: Map[HiveSerDe, String] = serdeMap.flatMap { + case ("sequencefile", _) => None + case ("rcfile", _) => None + case ("textfile", serde) => Some((serde, "text")) + case pair => Some(pair.swap) + } + /** * Get the Hive SerDe information from the data source abbreviation string or classname. * @@ -88,6 +96,14 @@ object HiveSerDe { serdeMap.get(key) } + /** + * Get the Spark data source name from the Hive SerDe information. + * + * @param serde Hive SerDe information. + * @return Spark data source name associated with the specified Hive Serde. + */ + def serdeToSource(serde: HiveSerDe): Option[String] = serdeInverseMap.get(serde) + def getDefaultStorage(conf: SQLConf): CatalogStorageFormat = { // To respect hive-site.xml, it peeks Hadoop configuration from existing Spark session, // as an easy workaround. See SPARK-27555. diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql b/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql index ccb40f8d991b4..dc77f87d9743a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql @@ -73,7 +73,7 @@ CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet; CREATE VIEW view_SPARK_30302 (aaa, bbb) AS SELECT a, b FROM tbl; -SHOW CREATE TABLE view_SPARK_30302; +SHOW CREATE TABLE view_SPARK_30302 AS SERDE; DROP VIEW view_SPARK_30302; @@ -82,7 +82,7 @@ CREATE VIEW view_SPARK_30302 (aaa COMMENT 'comment with \'quoted text\' for aaa' COMMENT 'This is a comment with \'quoted text\' for view' AS SELECT a, b FROM tbl; -SHOW CREATE TABLE view_SPARK_30302; +SHOW CREATE TABLE view_SPARK_30302 AS SERDE; DROP VIEW view_SPARK_30302; @@ -91,6 +91,13 @@ CREATE VIEW view_SPARK_30302 (aaa, bbb) TBLPROPERTIES ('a' = '1', 'b' = '2') AS SELECT a, b FROM tbl; +SHOW CREATE TABLE view_SPARK_30302 AS SERDE; +DROP VIEW view_SPARK_30302; + +-- SHOW CREATE TABLE does not support view +CREATE VIEW view_SPARK_30302 (aaa, bbb) +AS SELECT a, b FROM tbl; + SHOW CREATE TABLE view_SPARK_30302; DROP VIEW view_SPARK_30302; diff --git a/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out index 5771f218e3b57..e8ee07171651d 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 38 +-- Number of queries: 41 -- !query @@ -291,7 +291,7 @@ struct<> -- !query -SHOW CREATE TABLE view_SPARK_30302 +SHOW CREATE TABLE view_SPARK_30302 AS SERDE -- !query schema struct -- !query output @@ -320,7 +320,7 @@ struct<> -- !query -SHOW CREATE TABLE view_SPARK_30302 +SHOW CREATE TABLE view_SPARK_30302 AS SERDE -- !query schema struct -- !query output @@ -350,7 +350,7 @@ struct<> -- !query -SHOW CREATE TABLE view_SPARK_30302 +SHOW CREATE TABLE view_SPARK_30302 AS SERDE -- !query schema struct -- !query output @@ -371,6 +371,32 @@ struct<> +-- !query +CREATE VIEW view_SPARK_30302 (aaa, bbb) +AS SELECT a, b FROM tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Hive view isn't supported by SHOW CREATE TABLE; + + +-- !query +DROP VIEW view_SPARK_30302 +-- !query schema +struct<> +-- !query output + + + -- !query DROP TABLE tbl -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala index d7b489e4fa07d..b3b94f8be0d17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala @@ -148,20 +148,6 @@ abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { } } - test("view") { - withView("v1") { - sql("CREATE VIEW v1 AS SELECT 1 AS a") - checkCreateView("v1") - } - } - - test("view with output columns") { - withView("v1") { - sql("CREATE VIEW v1 (b) AS SELECT 1 AS a") - checkCreateView("v1") - } - } - test("temp view") { val viewName = "spark_28383" withTempView(viewName) { @@ -225,7 +211,7 @@ abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { } } - private def checkCatalogTables(expected: CatalogTable, actual: CatalogTable): Unit = { + protected def checkCatalogTables(expected: CatalogTable, actual: CatalogTable): Unit = { def normalize(table: CatalogTable): CatalogTable = { val nondeterministicProps = Set( "CreateTime", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala index a551ab704b62a..e5d572c90af38 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.{AnalysisException, ShowCreateTableSuite} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSingleton { @@ -38,6 +40,20 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet super.afterAll() } + test("view") { + withView("v1") { + sql("CREATE VIEW v1 AS SELECT 1 AS a") + checkCreateHiveTableOrView("v1", "VIEW") + } + } + + test("view with output columns") { + withView("v1") { + sql("CREATE VIEW v1 (b) AS SELECT 1 AS a") + checkCreateHiveTableOrView("v1", "VIEW") + } + } + test("simple hive table") { withTable("t1") { sql( @@ -52,7 +68,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateTable("t1") + checkCreateHiveTableOrView("t1") } } @@ -72,7 +88,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateTable("t1") + checkCreateHiveTableOrView("t1") } } } @@ -92,7 +108,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateTable("t1") + checkCreateHiveTableOrView("t1") } } @@ -110,7 +126,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateTable("t1") + checkCreateHiveTableOrView("t1") } } @@ -125,7 +141,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateTable("t1") + checkCreateHiveTableOrView("t1") } } @@ -147,7 +163,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet """.stripMargin ) - checkCreateTable("t1") + checkCreateHiveTableOrView("t1") } } @@ -160,7 +176,7 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet |INTO 2 BUCKETS """.stripMargin ) - checkCreateTable("t1") + checkCreateHiveTableOrView("t1") } } @@ -188,18 +204,44 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet } assert(cause.getMessage.contains(" - partitioned view")) + + val causeForSpark = intercept[AnalysisException] { + sql("SHOW CREATE TABLE v1 AS SERDE") + } + + assert(causeForSpark.getMessage.contains(" - partitioned view")) } } } test("SPARK-24911: keep quotes for nested fields in hive") { withTable("t1") { - val createTable = "CREATE TABLE `t1`(`a` STRUCT<`b`: STRING>) USING hive" + val createTable = "CREATE TABLE `t1` (`a` STRUCT<`b`: STRING>) USING hive" sql(createTable) val shownDDL = getShowDDL("SHOW CREATE TABLE t1") assert(shownDDL == createTable.dropRight(" USING hive".length)) - checkCreateTable("t1") + checkCreateHiveTableOrView("t1") + } + } + + /** + * This method compares the given table with the table created by the DDL generated by + * `SHOW CREATE TABLE AS SERDE`. + */ + private def checkCreateHiveTableOrView(tableName: String, checkType: String = "TABLE"): Unit = { + val table = TableIdentifier(tableName, Some("default")) + val db = table.database.getOrElse("default") + val expected = spark.sharedState.externalCatalog.getTable(db, table.table) + val shownDDL = sql(s"SHOW CREATE TABLE ${table.quotedString} AS SERDE").head().getString(0) + sql(s"DROP $checkType ${table.quotedString}") + + try { + sql(shownDDL) + val actual = spark.sharedState.externalCatalog.getTable(db, table.table) + checkCatalogTables(expected, actual) + } finally { + sql(s"DROP $checkType IF EXISTS ${table.table}") } } @@ -207,4 +249,269 @@ class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSinglet hiveContext.sharedState.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog] .client.runSqlHive(ddl) } + + private def checkCreateSparkTableAsHive(tableName: String): Unit = { + val table = TableIdentifier(tableName, Some("default")) + val db = table.database.get + val hiveTable = spark.sharedState.externalCatalog.getTable(db, table.table) + val sparkDDL = sql(s"SHOW CREATE TABLE ${table.quotedString}").head().getString(0) + // Drops original Hive table. + sql(s"DROP TABLE ${table.quotedString}") + + try { + // Creates Spark datasource table using generated Spark DDL. + sql(sparkDDL) + val sparkTable = spark.sharedState.externalCatalog.getTable(db, table.table) + checkHiveCatalogTables(hiveTable, sparkTable) + } finally { + sql(s"DROP TABLE IF EXISTS ${table.table}") + } + } + + private def checkHiveCatalogTables(hiveTable: CatalogTable, sparkTable: CatalogTable): Unit = { + def normalize(table: CatalogTable): CatalogTable = { + val nondeterministicProps = Set( + "CreateTime", + "transient_lastDdlTime", + "grantTime", + "lastUpdateTime", + "last_modified_by", + "last_modified_time", + "Owner:", + // The following are hive specific schema parameters which we do not need to match exactly. + "totalNumberFiles", + "maxFileSize", + "minFileSize" + ) + + table.copy( + createTime = 0L, + lastAccessTime = 0L, + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), + stats = None, + ignoredProperties = Map.empty, + storage = table.storage.copy(properties = Map.empty), + provider = None, + tracksPartitionsInCatalog = false + ) + } + + def fillSerdeFromProvider(table: CatalogTable): CatalogTable = { + table.provider.flatMap(HiveSerDe.sourceToSerDe(_)).map { hiveSerde => + val newStorage = table.storage.copy( + inputFormat = hiveSerde.inputFormat, + outputFormat = hiveSerde.outputFormat, + serde = hiveSerde.serde + ) + table.copy(storage = newStorage) + }.getOrElse(table) + } + + assert(normalize(fillSerdeFromProvider(sparkTable)) == normalize(hiveTable)) + } + + test("simple hive table in Spark DDL") { + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | c1 STRING COMMENT 'bla', + | c2 STRING + |) + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + |STORED AS orc + """.stripMargin + ) + + checkCreateSparkTableAsHive("t1") + } + } + + test("show create table as serde can't work on data source table") { + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | c1 STRING COMMENT 'bla', + | c2 STRING + |) + |USING orc + """.stripMargin + ) + + val cause = intercept[AnalysisException] { + checkCreateHiveTableOrView("t1") + } + + assert(cause.getMessage.contains("Use `SHOW CREATE TABLE` without `AS SERDE` instead")) + } + } + + test("simple external hive table in Spark DDL") { + withTempDir { dir => + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | c1 STRING COMMENT 'bla', + | c2 STRING + |) + |LOCATION '${dir.toURI}' + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + |STORED AS orc + """.stripMargin + ) + + checkCreateSparkTableAsHive("t1") + } + } + } + + test("hive table with STORED AS clause in Spark DDL") { + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |STORED AS PARQUET + """.stripMargin + ) + + checkCreateSparkTableAsHive("t1") + } + } + + test("hive table with nested fields with STORED AS clause in Spark DDL") { + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING, + | c3 STRUCT + |) + |STORED AS PARQUET + """.stripMargin + ) + + checkCreateSparkTableAsHive("t1") + } + } + + test("hive table with unsupported fileformat in Spark DDL") { + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |STORED AS RCFILE + """.stripMargin + ) + + val cause = intercept[AnalysisException] { + checkCreateSparkTableAsHive("t1") + } + + assert(cause.getMessage.contains("unsupported serde configuration")) + } + } + + test("hive table with serde info in Spark DDL") { + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin + ) + + checkCreateSparkTableAsHive("t1") + } + } + + test("hive view is not supported by show create table without as serde") { + withTable("t1") { + withView("v1") { + sql("CREATE TABLE t1 (c1 STRING, c2 STRING)") + + createRawHiveTable( + s""" + |CREATE VIEW v1 + |AS SELECT * from t1 + """.stripMargin + ) + + val cause = intercept[AnalysisException] { + sql("SHOW CREATE TABLE v1") + } + + assert(cause.getMessage.contains("view isn't supported")) + } + } + } + + test("partitioned, bucketed hive table in Spark DDL") { + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | emp_id INT COMMENT 'employee id', emp_name STRING, + | emp_dob STRING COMMENT 'employee date of birth', emp_sex STRING COMMENT 'M/F' + |) + |COMMENT 'employee table' + |PARTITIONED BY ( + | emp_country STRING COMMENT '2-char code', emp_state STRING COMMENT '2-char code' + |) + |CLUSTERED BY (emp_sex) SORTED BY (emp_id ASC) INTO 10 BUCKETS + |STORED AS ORC + """.stripMargin + ) + + checkCreateSparkTableAsHive("t1") + } + } + + test("show create table for transactional hive table") { + withTable("t1") { + sql( + s""" + |CREATE TABLE t1 ( + | c1 STRING COMMENT 'bla', + | c2 STRING + |) + |TBLPROPERTIES ( + | 'transactional' = 'true', + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + |CLUSTERED BY (c1) INTO 10 BUCKETS + |STORED AS ORC + """.stripMargin + ) + + + val cause = intercept[AnalysisException] { + sql("SHOW CREATE TABLE t1") + } + + assert(cause.getMessage.contains( + "SHOW CREATE TABLE doesn't support transactional Hive table")) + } + } } From 878094f9720d3c1866cbc01fb24c9794fe34edd9 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 31 Jan 2020 22:20:28 -0600 Subject: [PATCH 1282/1387] [SPARK-30689][CORE][YARN] Add resource discovery plugin api to support YARN versions with resource scheduling ### What changes were proposed in this pull request? This change is to allow custom resource scheduler (GPUs,FPGAs,etc) resource discovery to be more flexible. Users are asking for it to work with hadoop 2.x versions that do not support resource scheduling in YARN and/or also they may not run in an isolated environment. This change creates a plugin api that users can write their own resource discovery class that allows a lot more flexibility. The user can chain plugins for different resource types. The user specified plugins execute in the order specified and will fall back to use the discovery script plugin if they don't return information for a particular resource. I had to open up a few of the classes to be public and change them to not be case classes and make them developer api in order for the the plugin to get enough information it needs. I also relaxed the yarn side so that if yarn isn't configured for resource scheduling we just warn and go on. This helps users that have yarn 3.1 but haven't configured the resource scheduling side on their cluster yet, or aren't running in isolated environment. The user would configured this like: --conf spark.resources.discovery.plugin="org.apache.spark.resource.ResourceDiscoveryFPGAPlugin, org.apache.spark.resource.ResourceDiscoveryGPUPlugin" Note the executor side had to be wrapped with a classloader to make sure we include the user classpath for jars they specified on submission. Note this is more flexible because the discovery script has limitations such as spawning it in a separate process. This means if you are trying to allocate resources in that process they might be released when the script returns. Other things are the class makes it more flexible to be able to integrate with existing systems and solutions for assigning resources. ### Why are the changes needed? to more easily use spark resource scheduling with older versions of hadoop or in non-isolated enivronments. ### Does this PR introduce any user-facing change? Yes a plugin api ### How was this patch tested? Unit tests added and manual testing done on yarn and standalone modes. Closes #27410 from tgravescs/hadoop27spark3. Lead-authored-by: Thomas Graves Co-authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../api/resource/ResourceDiscoveryPlugin.java | 63 ++++++ .../scala/org/apache/spark/SparkContext.scala | 8 +- .../deploy/StandaloneResourceUtils.scala | 4 +- .../CoarseGrainedExecutorBackend.scala | 36 +++- .../spark/internal/config/package.scala | 12 ++ .../ResourceDiscoveryScriptPlugin.scala | 62 ++++++ .../spark/resource/ResourceProfile.scala | 4 +- .../apache/spark/resource/ResourceUtils.scala | 136 +++++++----- .../org/apache/spark/SparkConfSuite.scala | 2 +- .../CoarseGrainedExecutorBackendSuite.scala | 3 +- .../ResourceDiscoveryPluginSuite.scala | 194 ++++++++++++++++++ .../spark/resource/ResourceUtilsSuite.scala | 65 ++++-- .../spark/resource/TestResourceIDs.scala | 16 +- docs/configuration.md | 12 ++ .../spark/deploy/k8s/KubernetesUtils.scala | 8 +- .../BasicDriverFeatureStepSuite.scala | 2 +- .../BasicExecutorFeatureStepSuite.scala | 4 +- .../deploy/yarn/ResourceRequestHelper.scala | 31 ++- .../spark/deploy/yarn/ClientSuite.scala | 6 +- 19 files changed, 560 insertions(+), 108 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/api/resource/ResourceDiscoveryPlugin.java create mode 100644 core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala create mode 100644 core/src/test/scala/org/apache/spark/resource/ResourceDiscoveryPluginSuite.scala diff --git a/core/src/main/java/org/apache/spark/api/resource/ResourceDiscoveryPlugin.java b/core/src/main/java/org/apache/spark/api/resource/ResourceDiscoveryPlugin.java new file mode 100644 index 0000000000000..ffd2f83552a63 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/resource/ResourceDiscoveryPlugin.java @@ -0,0 +1,63 @@ +/* + * 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.api.resource; + +import java.util.Optional; + +import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.SparkConf; +import org.apache.spark.resource.ResourceInformation; +import org.apache.spark.resource.ResourceRequest; + +/** + * :: DeveloperApi :: + * A plugin that can be dynamically loaded into a Spark application to control how custom + * resources are discovered. Plugins can be chained to allow different plugins to handle + * different resource types. + *

    + * Plugins must implement the function discoveryResource. + * + * @since 3.0.0 + */ +@DeveloperApi +public interface ResourceDiscoveryPlugin { + /** + * Discover the addresses of the requested resource. + *

    + * This method is called early in the initialization of the Spark Executor/Driver/Worker. + * This function is responsible for discovering the addresses of the resource which Spark will + * then use for scheduling and eventually providing to the user. + * Depending on the deployment mode and and configuration of custom resources, this could be + * called by the Spark Driver, the Spark Executors, in standalone mode the Workers, or all of + * them. The ResourceRequest has a ResourceID component that can be used to distinguish which + * component it is called from and what resource its being called for. + * This will get called once for each resource type requested and its the responsibility of + * this function to return enough addresses of that resource based on the request. If + * the addresses do not meet the requested amount, Spark will fail. + * If this plugin doesn't handle a particular resource, it should return an empty Optional + * and Spark will try other plugins and then last fall back to the default discovery script + * plugin. + * + * @param request The ResourceRequest that to be discovered. + * @param sparkConf SparkConf + * @return An {@link Optional} containing a {@link ResourceInformation} object containing + * the resource name and the addresses of the resource. If it returns {@link Optional#EMPTY} + * other plugins will be called. + */ + Optional discoverResource(ResourceRequest request, SparkConf sparkConf); +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6e0c7acf8b18d..91188d58f4201 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2806,17 +2806,17 @@ object SparkContext extends Logging { // Make sure the executor resources were specified through config. val execAmount = executorResourcesAndAmounts.getOrElse(taskReq.resourceName, throw new SparkException("The executor resource config: " + - ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + + new ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + " needs to be specified since a task requirement config: " + - ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + + new ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + " was specified") ) // Make sure the executor resources are large enough to launch at least one task. if (execAmount < taskReq.amount) { throw new SparkException("The executor resource config: " + - ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + + new ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + s" = $execAmount has to be >= the requested amount in task resource config: " + - ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + + new ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + s" = ${taskReq.amount}") } // Compare and update the max slots each executor can provide. diff --git a/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala b/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala index d6f9618af4aac..65bf4351ebfd9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/StandaloneResourceUtils.scala @@ -208,7 +208,7 @@ private[spark] object StandaloneResourceUtils extends Logging { } val newAllocation = { val allocations = newAssignments.map { case (rName, addresses) => - ResourceAllocation(ResourceID(componentName, rName), addresses) + ResourceAllocation(new ResourceID(componentName, rName), addresses) }.toSeq StandaloneResourceAllocation(pid, allocations) } @@ -348,7 +348,7 @@ private[spark] object StandaloneResourceUtils extends Logging { val compShortName = componentName.substring(componentName.lastIndexOf(".") + 1) val tmpFile = Utils.tempFileWith(dir) val allocations = resources.map { case (rName, rInfo) => - ResourceAllocation(ResourceID(componentName, rName), rInfo.addresses) + ResourceAllocation(new ResourceID(componentName, rName), rInfo.addresses) }.toSeq try { writeResourceAllocationJson(componentName, allocations, tmpFile) 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 ce211ce8dd67a..25c5b9812fa1a 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -17,6 +17,7 @@ package org.apache.spark.executor +import java.io.File import java.net.URL import java.nio.ByteBuffer import java.util.Locale @@ -42,7 +43,7 @@ import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, ThreadUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( override val rpcEnv: RpcEnv, @@ -99,15 +100,36 @@ private[spark] class CoarseGrainedExecutorBackend( }(ThreadUtils.sameThread) } + /** + * Create a classLoader for use for resource discovery. The user could provide a class + * as a substitute for the default one so we have to be able to load it from a user specified + * jar. + */ + private def createClassLoader(): MutableURLClassLoader = { + val currentLoader = Utils.getContextOrSparkClassLoader + val urls = userClassPath.toArray + if (env.conf.get(EXECUTOR_USER_CLASS_PATH_FIRST)) { + new ChildFirstURLClassLoader(urls, currentLoader) + } else { + new MutableURLClassLoader(urls, currentLoader) + } + } + // visible for testing def parseOrFindResources(resourcesFileOpt: Option[String]): Map[String, ResourceInformation] = { + // use a classloader that includes the user classpath in case they specified a class for + // resource discovery + val urlClassLoader = createClassLoader() logDebug(s"Resource profile id is: ${resourceProfile.id}") - val resources = getOrDiscoverAllResourcesForResourceProfile( - resourcesFileOpt, - SPARK_EXECUTOR_PREFIX, - resourceProfile) - logResourceInfo(SPARK_EXECUTOR_PREFIX, resources) - resources + Utils.withContextClassLoader(urlClassLoader) { + val resources = getOrDiscoverAllResourcesForResourceProfile( + resourcesFileOpt, + SPARK_EXECUTOR_PREFIX, + resourceProfile, + env.conf) + logResourceInfo(SPARK_EXECUTOR_PREFIX, resources) + resources + } } def extractLogUrls: Map[String, String] = { 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 e68368f37ab6f..f91f31be2f1ad 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 @@ -54,6 +54,18 @@ package object config { .stringConf .createOptional + private[spark] val RESOURCES_DISCOVERY_PLUGIN = + ConfigBuilder("spark.resources.discovery.plugin") + .doc("Comma-separated list of class names implementing" + + "org.apache.spark.api.resource.ResourceDiscoveryPlugin to load into the application." + + "This is for advanced users to replace the resource discovery class with a " + + "custom implementation. Spark will try each class specified until one of them " + + "returns the resource information for that resource. It tries the discovery " + + "script last if none of the plugins return information for that resource.") + .stringConf + .toSequence + .createWithDefault(Nil) + private[spark] val DRIVER_RESOURCES_FILE = ConfigBuilder("spark.driver.resourcesFile") .internal() diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala b/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala new file mode 100644 index 0000000000000..2ac6d3c500f9d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.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.resource + +import java.io.File +import java.util.Optional + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.api.resource.ResourceDiscoveryPlugin +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils.executeAndGetOutput + +/** + * The default plugin that is loaded into a Spark application to control how custom + * resources are discovered. This executes the discovery script specified by the user + * and gets the json output back and contructs ResourceInformation objects from that. + * If the user specifies custom plugins, this is the last one to be executed and + * throws if the resource isn't discovered. + */ +class ResourceDiscoveryScriptPlugin extends ResourceDiscoveryPlugin with Logging { + override def discoverResource( + request: ResourceRequest, + sparkConf: SparkConf): Optional[ResourceInformation] = { + val script = request.discoveryScript + val resourceName = request.id.resourceName + val result = if (script.isPresent) { + val scriptFile = new File(script.get) + logInfo(s"Discovering resources for $resourceName with script: $scriptFile") + // check that script exists and try to execute + if (scriptFile.exists()) { + val output = executeAndGetOutput(Seq(script.get), new File(".")) + ResourceInformation.parseJson(output) + } else { + throw new SparkException(s"Resource script: $scriptFile to discover $resourceName " + + "doesn't exist!") + } + } else { + throw new SparkException(s"User is expecting to use resource: $resourceName, but " + + "didn't specify a discovery script!") + } + if (!result.name.equals(resourceName)) { + throw new SparkException(s"Error running the resource discovery script ${script.get}: " + + s"script returned resource name ${result.name} and we were expecting $resourceName.") + } + Optional.of(result) + } +} diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index eb713a27be603..14019d27fc2e6 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -149,8 +149,8 @@ object ResourceProfile extends Logging { val execReq = ResourceUtils.parseAllResourceRequests(conf, SPARK_EXECUTOR_PREFIX) execReq.foreach { req => val name = req.id.resourceName - ereqs.resource(name, req.amount, req.discoveryScript.getOrElse(""), - req.vendor.getOrElse("")) + ereqs.resource(name, req.amount, req.discoveryScript.orElse(""), + req.vendor.orElse("")) } ereqs.requests } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index 190b0cdc8887a..7dd7fc1b99353 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -17,8 +17,8 @@ package org.apache.spark.resource -import java.io.File import java.nio.file.{Files, Paths} +import java.util.Optional import scala.util.control.NonFatal @@ -26,39 +26,75 @@ import org.json4s.DefaultFormats import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.resource.ResourceDiscoveryPlugin import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.SPARK_TASK_PREFIX -import org.apache.spark.util.Utils.executeAndGetOutput +import org.apache.spark.internal.config.{RESOURCES_DISCOVERY_PLUGIN, SPARK_TASK_PREFIX} +import org.apache.spark.util.Utils /** * Resource identifier. * @param componentName spark.driver / spark.executor / spark.task * @param resourceName gpu, fpga, etc + * + * @since 3.0.0 */ -private[spark] case class ResourceID(componentName: String, resourceName: String) { - def confPrefix: String = s"$componentName.${ResourceUtils.RESOURCE_PREFIX}.$resourceName." - def amountConf: String = s"$confPrefix${ResourceUtils.AMOUNT}" - def discoveryScriptConf: String = s"$confPrefix${ResourceUtils.DISCOVERY_SCRIPT}" - def vendorConf: String = s"$confPrefix${ResourceUtils.VENDOR}" +@DeveloperApi +class ResourceID(val componentName: String, val resourceName: String) { + private[spark] def confPrefix: String = { + s"$componentName.${ResourceUtils.RESOURCE_PREFIX}.$resourceName." + } + private[spark] def amountConf: String = s"$confPrefix${ResourceUtils.AMOUNT}" + private[spark] def discoveryScriptConf: String = s"$confPrefix${ResourceUtils.DISCOVERY_SCRIPT}" + private[spark] def vendorConf: String = s"$confPrefix${ResourceUtils.VENDOR}" + + override def equals(obj: Any): Boolean = { + obj match { + case that: ResourceID => + that.getClass == this.getClass && + that.componentName == componentName && that.resourceName == resourceName + case _ => + false + } + } + + override def hashCode(): Int = Seq(componentName, resourceName).hashCode() } /** - * Case class that represents a resource request at the executor level. + * Class that represents a resource request. * * The class used when discovering resources (using the discovery script), - * or via the context as it is parsing configuration, for SPARK_EXECUTOR_PREFIX. + * or via the context as it is parsing configuration for the ResourceID. * * @param id object identifying the resource * @param amount integer amount for the resource. Note that for a request (executor level), * fractional resources does not make sense, so amount is an integer. * @param discoveryScript optional discovery script file name * @param vendor optional vendor name + * + * @since 3.0.0 */ -private[spark] case class ResourceRequest( - id: ResourceID, - amount: Int, - discoveryScript: Option[String], - vendor: Option[String]) +@DeveloperApi +class ResourceRequest( + val id: ResourceID, + val amount: Long, + val discoveryScript: Optional[String], + val vendor: Optional[String]) { + + override def equals(obj: Any): Boolean = { + obj match { + case that: ResourceRequest => + that.getClass == this.getClass && + that.id == id && that.amount == amount && discoveryScript == discoveryScript && + vendor == vendor + case _ => + false + } + } + + override def hashCode(): Int = Seq(id, amount, discoveryScript, vendor).hashCode() +} /** * Case class that represents resource requirements for a component in a @@ -105,15 +141,15 @@ private[spark] object ResourceUtils extends Logging { val amount = settings.getOrElse(AMOUNT, throw new SparkException(s"You must specify an amount for ${resourceId.resourceName}") ).toInt - val discoveryScript = settings.get(DISCOVERY_SCRIPT) - val vendor = settings.get(VENDOR) - ResourceRequest(resourceId, amount, discoveryScript, vendor) + val discoveryScript = Optional.ofNullable(settings.get(DISCOVERY_SCRIPT).orNull) + val vendor = Optional.ofNullable(settings.get(VENDOR).orNull) + new ResourceRequest(resourceId, amount, discoveryScript, vendor) } def listResourceIds(sparkConf: SparkConf, componentName: String): Seq[ResourceID] = { sparkConf.getAllWithPrefix(s"$componentName.$RESOURCE_PREFIX.").map { case (key, _) => key.substring(0, key.indexOf('.')) - }.toSet.toSeq.map(name => ResourceID(componentName, name)) + }.toSet.toSeq.map(name => new ResourceID(componentName, name)) } def parseAllResourceRequests( @@ -218,7 +254,7 @@ private[spark] object ResourceUtils extends Logging { val otherResources = otherResourceIds.flatMap { id => val request = parseResourceRequest(sparkConf, id) if (request.amount > 0) { - Some(ResourceAllocation(id, discoverResource(request).addresses)) + Some(ResourceAllocation(id, discoverResource(sparkConf, request).addresses)) } else { None } @@ -274,6 +310,15 @@ private[spark] object ResourceUtils extends Logging { resourceInfoMap } + // create an empty Optional if the string is empty + private def emptyStringToOptional(optStr: String): Optional[String] = { + if (optStr.isEmpty) { + Optional.empty[String] + } else { + Optional.of(optStr) + } + } + /** * This function is similar to getOrDiscoverallResources, except for it uses the ResourceProfile * information instead of the application level configs. @@ -290,14 +335,19 @@ private[spark] object ResourceUtils extends Logging { def getOrDiscoverAllResourcesForResourceProfile( resourcesFileOpt: Option[String], componentName: String, - resourceProfile: ResourceProfile): Map[String, ResourceInformation] = { + resourceProfile: ResourceProfile, + sparkConf: SparkConf): Map[String, ResourceInformation] = { val fileAllocated = parseAllocated(resourcesFileOpt, componentName) val fileAllocResMap = fileAllocated.map(a => (a.id.resourceName, a.toResourceInformation)).toMap // only want to look at the ResourceProfile for resources not in the resources file val execReq = ResourceProfile.getCustomExecutorResources(resourceProfile) val filteredExecreq = execReq.filterNot { case (rname, _) => fileAllocResMap.contains(rname) } val rpAllocations = filteredExecreq.map { case (rName, execRequest) => - val addrs = discoverResource(rName, Option(execRequest.discoveryScript)).addresses + val resourceId = new ResourceID(componentName, rName) + val scriptOpt = emptyStringToOptional(execRequest.discoveryScript) + val vendorOpt = emptyStringToOptional(execRequest.vendor) + val resourceReq = new ResourceRequest(resourceId, execRequest.amount, scriptOpt, vendorOpt) + val addrs = discoverResource(sparkConf, resourceReq).addresses (rName, new ResourceInformation(rName, addrs)) } val allAllocations = fileAllocResMap ++ rpAllocations @@ -312,36 +362,24 @@ private[spark] object ResourceUtils extends Logging { logInfo("==============================================================") } - // visible for test private[spark] def discoverResource( - resourceName: String, - script: Option[String]): ResourceInformation = { - val result = if (script.nonEmpty) { - val scriptFile = new File(script.get) - // check that script exists and try to execute - if (scriptFile.exists()) { - val output = executeAndGetOutput(Seq(script.get), new File(".")) - ResourceInformation.parseJson(output) - } else { - throw new SparkException(s"Resource script: $scriptFile to discover $resourceName " + - "doesn't exist!") + sparkConf: SparkConf, + resourceRequest: ResourceRequest): ResourceInformation = { + // always put the discovery script plugin as last plugin + val discoveryScriptPlugin = "org.apache.spark.resource.ResourceDiscoveryScriptPlugin" + val pluginClasses = sparkConf.get(RESOURCES_DISCOVERY_PLUGIN) :+ discoveryScriptPlugin + val resourcePlugins = Utils.loadExtensions(classOf[ResourceDiscoveryPlugin], pluginClasses, + sparkConf) + // apply each plugin until one of them returns the information for this resource + var riOption: Optional[ResourceInformation] = Optional.empty() + resourcePlugins.foreach { plugin => + val riOption = plugin.discoverResource(resourceRequest, sparkConf) + if (riOption.isPresent()) { + return riOption.get() } - } else { - throw new SparkException(s"User is expecting to use resource: $resourceName, but " + - "didn't specify a discovery script!") - } - if (!result.name.equals(resourceName)) { - throw new SparkException(s"Error running the resource discovery script ${script.get}: " + - s"script returned resource name ${result.name} and we were expecting $resourceName.") } - result - } - - // visible for test - private[spark] def discoverResource(resourceRequest: ResourceRequest): ResourceInformation = { - val resourceName = resourceRequest.id.resourceName - val script = resourceRequest.discoveryScript - discoverResource(resourceName, script) + throw new SparkException(s"None of the discovery plugins returned ResourceInformation for " + + s"${resourceRequest.id.resourceName}") } // known types of resources diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index c210eb0d60d4b..3bc2061c4f2ad 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -449,7 +449,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst conf.remove(TASK_FPGA_ID.amountConf) // Ignore invalid prefix - conf.set(ResourceID("spark.invalid.prefix", FPGA).amountConf, "1") + conf.set(new ResourceID("spark.invalid.prefix", FPGA).amountConf, "1") taskResourceRequirement = parseResourceRequirements(conf, SPARK_TASK_PREFIX) .map(req => (req.resourceName, req.amount)).toMap diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index a996fc4a0b648..3134a738b33fa 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -164,7 +164,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val parsedResources = backend.parseOrFindResources(Some(f1)) }.getMessage() - assert(error.contains("Resource script: to discover gpu doesn't exist!")) + assert(error.contains("User is expecting to use resource: gpu, but didn't " + + "specify a discovery script!")) } } diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceDiscoveryPluginSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceDiscoveryPluginSuite.scala new file mode 100644 index 0000000000000..7a05daa2ad715 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/resource/ResourceDiscoveryPluginSuite.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import java.io.File +import java.nio.charset.StandardCharsets +import java.util.Optional +import java.util.UUID + +import scala.concurrent.duration._ + +import com.google.common.io.Files +import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} + +import org.apache.spark._ +import org.apache.spark.TestUtils.createTempScriptWithExpectedOutput +import org.apache.spark.api.resource.ResourceDiscoveryPlugin +import org.apache.spark.internal.config._ +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.resource.ResourceUtils.{FPGA, GPU} +import org.apache.spark.resource.TestResourceIDs._ +import org.apache.spark.util.Utils + +class ResourceDiscoveryPluginSuite extends SparkFunSuite with LocalSparkContext { + + test("plugin initialization in non-local mode fpga and gpu") { + assume(!(Utils.isWindows)) + withTempDir { dir => + val conf = new SparkConf() + .setAppName(getClass().getName()) + .set(SparkLauncher.SPARK_MASTER, "local-cluster[2,1,1024]") + .set(RESOURCES_DISCOVERY_PLUGIN, Seq(classOf[TestResourceDiscoveryPluginGPU].getName(), + classOf[TestResourceDiscoveryPluginFPGA].getName())) + .set(TestResourceDiscoveryPlugin.TEST_PATH_CONF, dir.getAbsolutePath()) + .set(WORKER_GPU_ID.amountConf, "2") + .set(TASK_GPU_ID.amountConf, "1") + .set(EXECUTOR_GPU_ID.amountConf, "1") + .set(SPARK_RESOURCES_DIR, dir.getName()) + .set(WORKER_FPGA_ID.amountConf, "2") + .set(TASK_FPGA_ID.amountConf, "1") + .set(EXECUTOR_FPGA_ID.amountConf, "1") + + sc = new SparkContext(conf) + TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + + eventually(timeout(10.seconds), interval(100.millis)) { + val children = dir.listFiles() + assert(children != null) + assert(children.length >= 4) + val gpuFiles = children.filter(f => f.getName().contains(GPU)) + val fpgaFiles = children.filter(f => f.getName().contains(FPGA)) + assert(gpuFiles.length == 2) + assert(fpgaFiles.length == 2) + } + } + } + + test("single plugin gpu") { + assume(!(Utils.isWindows)) + withTempDir { dir => + val conf = new SparkConf() + .setAppName(getClass().getName()) + .set(SparkLauncher.SPARK_MASTER, "local-cluster[2,1,1024]") + .set(RESOURCES_DISCOVERY_PLUGIN, Seq(classOf[TestResourceDiscoveryPluginGPU].getName())) + .set(TestResourceDiscoveryPlugin.TEST_PATH_CONF, dir.getAbsolutePath()) + .set(WORKER_GPU_ID.amountConf, "2") + .set(TASK_GPU_ID.amountConf, "1") + .set(EXECUTOR_GPU_ID.amountConf, "1") + .set(SPARK_RESOURCES_DIR, dir.getName()) + + sc = new SparkContext(conf) + TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + + eventually(timeout(10.seconds), interval(100.millis)) { + val children = dir.listFiles() + assert(children != null) + assert(children.length >= 2) + val gpuFiles = children.filter(f => f.getName().contains(GPU)) + assert(gpuFiles.length == 2) + } + } + } + + test("multiple plugins with one empty") { + assume(!(Utils.isWindows)) + withTempDir { dir => + val conf = new SparkConf() + .setAppName(getClass().getName()) + .set(SparkLauncher.SPARK_MASTER, "local-cluster[2,1,1024]") + .set(RESOURCES_DISCOVERY_PLUGIN, Seq(classOf[TestResourceDiscoveryPluginEmpty].getName(), + classOf[TestResourceDiscoveryPluginGPU].getName())) + .set(TestResourceDiscoveryPlugin.TEST_PATH_CONF, dir.getAbsolutePath()) + .set(WORKER_GPU_ID.amountConf, "2") + .set(TASK_GPU_ID.amountConf, "1") + .set(EXECUTOR_GPU_ID.amountConf, "1") + .set(SPARK_RESOURCES_DIR, dir.getName()) + + sc = new SparkContext(conf) + TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + + eventually(timeout(10.seconds), interval(100.millis)) { + val children = dir.listFiles() + assert(children != null) + assert(children.length >= 2) + val gpuFiles = children.filter(f => f.getName().contains(GPU)) + assert(gpuFiles.length == 2) + } + } + } + + test("empty plugin fallback to discovery script") { + assume(!(Utils.isWindows)) + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", + """{"name": "gpu","addresses":["5", "6"]}""") + val conf = new SparkConf() + .setAppName(getClass().getName()) + .set(SparkLauncher.SPARK_MASTER, "local-cluster[2,1,1024]") + .set(RESOURCES_DISCOVERY_PLUGIN, Seq(classOf[TestResourceDiscoveryPluginEmpty].getName())) + .set(DRIVER_GPU_ID.discoveryScriptConf, scriptPath) + .set(DRIVER_GPU_ID.amountConf, "2") + .set(SPARK_RESOURCES_DIR, dir.getName()) + + sc = new SparkContext(conf) + TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + + assert(sc.resources.size === 1) + assert(sc.resources.get(GPU).get.addresses === Array("5", "6")) + assert(sc.resources.get(GPU).get.name === "gpu") + } + } +} + +object TestResourceDiscoveryPlugin { + val TEST_PATH_CONF = "spark.nonLocalDiscoveryPlugin.path" + + def writeFile(conf: SparkConf, id: String): Unit = { + val path = conf.get(TEST_PATH_CONF) + val fileName = s"$id - ${UUID.randomUUID.toString}" + Files.write(id, new File(path, fileName), StandardCharsets.UTF_8) + } +} + +private class TestResourceDiscoveryPluginGPU extends ResourceDiscoveryPlugin { + + override def discoverResource( + request: ResourceRequest, + conf: SparkConf): Optional[ResourceInformation] = { + if (request.id.resourceName.equals(GPU)) { + TestResourceDiscoveryPlugin.writeFile(conf, request.id.resourceName) + Optional.of(new ResourceInformation(GPU, Array("0", "1", "2", "3"))) + } else { + Optional.empty() + } + } +} + +private class TestResourceDiscoveryPluginEmpty extends ResourceDiscoveryPlugin { + + override def discoverResource( + request: ResourceRequest, + conf: SparkConf): Optional[ResourceInformation] = { + Optional.empty() + } +} + +private class TestResourceDiscoveryPluginFPGA extends ResourceDiscoveryPlugin { + + override def discoverResource( + request: ResourceRequest, + conf: SparkConf): Optional[ResourceInformation] = { + if (request.id.resourceName.equals(FPGA)) { + TestResourceDiscoveryPlugin.writeFile(conf, request.id.resourceName) + Optional.of(new ResourceInformation(FPGA, Array("0", "1", "2", "3"))) + } else { + Optional.empty() + } + } +} diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala index b809469fd7163..dffe9a02e9aa4 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.resource import java.io.File import java.nio.file.{Files => JavaFiles} +import java.util.Optional import org.json4s.{DefaultFormats, Extraction} @@ -35,7 +36,7 @@ class ResourceUtilsSuite extends SparkFunSuite test("ResourceID") { val componentName = "spark.test" val resourceName = "p100" - val id = ResourceID(componentName, resourceName) + val id = new ResourceID(componentName, resourceName) val confPrefix = s"$componentName.resource.$resourceName." assert(id.confPrefix === confPrefix) assert(id.amountConf === s"${confPrefix}amount") @@ -91,7 +92,7 @@ class ResourceUtilsSuite extends SparkFunSuite // test one with amount 0 to make sure ignored val fooDiscovery = createTempScriptWithExpectedOutput(dir, "fooDiscoverScript", """{"name": "foo", "addresses": ["f1", "f2", "f3"]}""") - val fooId = ResourceID(SPARK_EXECUTOR_PREFIX, "foo") + val fooId = new ResourceID(SPARK_EXECUTOR_PREFIX, "foo") conf.set(fooId.amountConf, "0") conf.set(fooId.discoveryScriptConf, fooDiscovery) @@ -153,7 +154,8 @@ class ResourceUtilsSuite extends SparkFunSuite val resourcesFromFileOnly = getOrDiscoverAllResourcesForResourceProfile( Some(resourcesFile), SPARK_EXECUTOR_PREFIX, - ResourceProfile.getOrCreateDefaultProfile(conf)) + ResourceProfile.getOrCreateDefaultProfile(conf), + conf) val expectedFpgaInfo = new ResourceInformation(FPGA, fpgaAddrs.toArray) assert(resourcesFromFileOnly(FPGA) === expectedFpgaInfo) @@ -165,7 +167,7 @@ class ResourceUtilsSuite extends SparkFunSuite val treqs = new TaskResourceRequests().resource(GPU, 1) val rp = rpBuilder.require(ereqs).require(treqs).build val resourcesFromBoth = getOrDiscoverAllResourcesForResourceProfile( - Some(resourcesFile), SPARK_EXECUTOR_PREFIX, rp) + Some(resourcesFile), SPARK_EXECUTOR_PREFIX, rp, conf) val expectedGpuInfo = new ResourceInformation(GPU, Array("0", "1")) assert(resourcesFromBoth(FPGA) === expectedFpgaInfo) assert(resourcesFromBoth(GPU) === expectedGpuInfo) @@ -193,8 +195,8 @@ class ResourceUtilsSuite extends SparkFunSuite var request = parseResourceRequest(conf, DRIVER_GPU_ID) assert(request.id.resourceName === GPU, "should only have GPU for resource") assert(request.amount === 2, "GPU count should be 2") - assert(request.discoveryScript === None, "discovery script should be empty") - assert(request.vendor === None, "vendor should be empty") + assert(request.discoveryScript === Optional.empty(), "discovery script should be empty") + assert(request.vendor === Optional.empty(), "vendor should be empty") val vendor = "nvidia.com" val discoveryScript = "discoveryScriptGPU" @@ -240,14 +242,14 @@ class ResourceUtilsSuite extends SparkFunSuite val gpuDiscovery = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", """{"name": "fpga", "addresses": ["0", "1"]}""") val request = - ResourceRequest( + new ResourceRequest( DRIVER_GPU_ID, 2, - Some(gpuDiscovery), - None) + Optional.of(gpuDiscovery), + Optional.empty[String]) val error = intercept[SparkException] { - discoverResource(request) + discoverResource(conf, request) }.getMessage() assert(error.contains(s"Error running the resource discovery script $gpuDiscovery: " + @@ -255,6 +257,28 @@ class ResourceUtilsSuite extends SparkFunSuite } } + test("Resource discoverer with invalid class") { + val conf = new SparkConf() + .set(RESOURCES_DISCOVERY_PLUGIN, Seq("someinvalidclass")) + assume(!(Utils.isWindows)) + withTempDir { dir => + val gpuDiscovery = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", + """{"name": "fpga", "addresses": ["0", "1"]}""") + val request = + new ResourceRequest( + DRIVER_GPU_ID, + 2, + Optional.of(gpuDiscovery), + Optional.empty[String]) + + val error = intercept[ClassNotFoundException] { + discoverResource(conf, request) + }.getMessage() + + assert(error.contains(s"someinvalidclass")) + } + } + test("Resource discoverer script returns invalid format") { val conf = new SparkConf assume(!(Utils.isWindows)) @@ -263,14 +287,14 @@ class ResourceUtilsSuite extends SparkFunSuite """{"addresses": ["0", "1"]}""") val request = - ResourceRequest( + new ResourceRequest( EXECUTOR_GPU_ID, 2, - Some(gpuDiscovery), - None) + Optional.of(gpuDiscovery), + Optional.empty[String]) val error = intercept[SparkException] { - discoverResource(request) + discoverResource(conf, request) }.getMessage() assert(error.contains("Error parsing JSON into ResourceInformation")) @@ -283,14 +307,14 @@ class ResourceUtilsSuite extends SparkFunSuite val file1 = new File(dir, "bogusfilepath") try { val request = - ResourceRequest( + new ResourceRequest( EXECUTOR_GPU_ID, 2, - Some(file1.getPath()), - None) + Optional.of(file1.getPath()), + Optional.empty[String]) val error = intercept[SparkException] { - discoverResource(request) + discoverResource(conf, request) }.getMessage() assert(error.contains("doesn't exist")) @@ -301,10 +325,11 @@ class ResourceUtilsSuite extends SparkFunSuite } test("gpu's specified but not a discovery script") { - val request = ResourceRequest(EXECUTOR_GPU_ID, 2, None, None) + val request = new ResourceRequest(EXECUTOR_GPU_ID, 2, Optional.empty[String], + Optional.empty[String]) val error = intercept[SparkException] { - discoverResource(request) + discoverResource(new SparkConf(), request) }.getMessage() assert(error.contains("User is expecting to use resource: gpu, but " + diff --git a/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala b/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala index c4509e93104d5..60246f5fad9a8 100644 --- a/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala +++ b/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala @@ -22,14 +22,14 @@ import org.apache.spark.internal.config.Worker.SPARK_WORKER_PREFIX import org.apache.spark.resource.ResourceUtils.{FPGA, GPU} object TestResourceIDs { - val DRIVER_GPU_ID = ResourceID(SPARK_DRIVER_PREFIX, GPU) - val EXECUTOR_GPU_ID = ResourceID(SPARK_EXECUTOR_PREFIX, GPU) - val TASK_GPU_ID = ResourceID(SPARK_TASK_PREFIX, GPU) - val WORKER_GPU_ID = ResourceID(SPARK_WORKER_PREFIX, GPU) + val DRIVER_GPU_ID = new ResourceID(SPARK_DRIVER_PREFIX, GPU) + val EXECUTOR_GPU_ID = new ResourceID(SPARK_EXECUTOR_PREFIX, GPU) + val TASK_GPU_ID = new ResourceID(SPARK_TASK_PREFIX, GPU) + val WORKER_GPU_ID = new ResourceID(SPARK_WORKER_PREFIX, GPU) - val DRIVER_FPGA_ID = ResourceID(SPARK_DRIVER_PREFIX, FPGA) - val EXECUTOR_FPGA_ID = ResourceID(SPARK_EXECUTOR_PREFIX, FPGA) - val TASK_FPGA_ID = ResourceID(SPARK_TASK_PREFIX, FPGA) - val WORKER_FPGA_ID = ResourceID(SPARK_WORKER_PREFIX, FPGA) + val DRIVER_FPGA_ID = new ResourceID(SPARK_DRIVER_PREFIX, FPGA) + val EXECUTOR_FPGA_ID = new ResourceID(SPARK_EXECUTOR_PREFIX, FPGA) + val TASK_FPGA_ID = new ResourceID(SPARK_TASK_PREFIX, FPGA) + val WORKER_FPGA_ID = new ResourceID(SPARK_WORKER_PREFIX, FPGA) } diff --git a/docs/configuration.md b/docs/configuration.md index 8164ed491d4c6..2febfe9744d5c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -243,6 +243,18 @@ of the most common options to set are: this config would be set to nvidia.com or amd.com)

    + + + + + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index b1b7751b012e1..e234b1780a7d0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -228,8 +228,12 @@ private[spark] object KubernetesUtils extends Logging { sparkConf: SparkConf): Map[String, Quantity] = { val requests = ResourceUtils.parseAllResourceRequests(sparkConf, componentName) requests.map { request => - val vendorDomain = request.vendor.getOrElse(throw new SparkException("Resource: " + - s"${request.id.resourceName} was requested, but vendor was not specified.")) + val vendorDomain = if (request.vendor.isPresent()) { + request.vendor.get() + } else { + throw new SparkException(s"Resource: ${request.id.resourceName} was requested, " + + "but vendor was not specified.") + } val quantity = new QuantityBuilder(false) .withAmount(request.amount.toString) .build() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 710f28a859347..ce66afd9448a9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -47,7 +47,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { } test("Check the pod respects all configurations from the user.") { - val resourceID = ResourceID(SPARK_DRIVER_PREFIX, GPU) + val resourceID = new ResourceID(SPARK_DRIVER_PREFIX, GPU) val resources = Map(("nvidia.com/gpu" -> TestResourceInformation(resourceID, "2", "nvidia.com"))) val sparkConf = new SparkConf() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 51067bd889a58..f375b1fe6a5cd 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -115,8 +115,8 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("basic executor pod with resources") { - val fpgaResourceID = ResourceID(SPARK_EXECUTOR_PREFIX, FPGA) - val gpuExecutorResourceID = ResourceID(SPARK_EXECUTOR_PREFIX, GPU) + val fpgaResourceID = new ResourceID(SPARK_EXECUTOR_PREFIX, FPGA) + val gpuExecutorResourceID = new ResourceID(SPARK_EXECUTOR_PREFIX, GPU) val gpuResources = Map(("nvidia.com/gpu" -> TestResourceInformation(gpuExecutorResourceID, "2", "nvidia.com")), ("foo.com/fpga" -> TestResourceInformation(fpgaResourceID, "1", "foo.com"))) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala index f524962141455..ae316b02eefd8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -40,8 +40,10 @@ import org.apache.spark.util.{CausedBy, Utils} private object ResourceRequestHelper extends Logging { private val AMOUNT_AND_UNIT_REGEX = "([0-9]+)([A-Za-z]*)".r private val RESOURCE_INFO_CLASS = "org.apache.hadoop.yarn.api.records.ResourceInformation" + private val RESOURCE_NOT_FOUND = "org.apache.hadoop.yarn.exceptions.ResourceNotFoundException" val YARN_GPU_RESOURCE_CONFIG = "yarn.io/gpu" val YARN_FPGA_RESOURCE_CONFIG = "yarn.io/fpga" + @volatile private var numResourceErrors: Int = 0 private[yarn] def getYarnResourcesAndAmounts( sparkConf: SparkConf, @@ -76,7 +78,7 @@ private object ResourceRequestHelper extends Logging { ): Map[String, String] = { Map(GPU -> YARN_GPU_RESOURCE_CONFIG, FPGA -> YARN_FPGA_RESOURCE_CONFIG).map { case (rName, yarnName) => - (yarnName -> sparkConf.get(ResourceID(confPrefix, rName).amountConf, "0")) + (yarnName -> sparkConf.get(new ResourceID(confPrefix, rName).amountConf, "0")) }.filter { case (_, count) => count.toLong > 0 } } @@ -108,13 +110,13 @@ private object ResourceRequestHelper extends Logging { (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "cpu-vcores"), (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "cpu-vcores"), (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "cpu-vcores"), - (ResourceID(SPARK_EXECUTOR_PREFIX, "fpga").amountConf, + (new ResourceID(SPARK_EXECUTOR_PREFIX, "fpga").amountConf, s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${YARN_FPGA_RESOURCE_CONFIG}"), - (ResourceID(SPARK_DRIVER_PREFIX, "fpga").amountConf, + (new ResourceID(SPARK_DRIVER_PREFIX, "fpga").amountConf, s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${YARN_FPGA_RESOURCE_CONFIG}"), - (ResourceID(SPARK_EXECUTOR_PREFIX, "gpu").amountConf, + (new ResourceID(SPARK_EXECUTOR_PREFIX, "gpu").amountConf, s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${YARN_GPU_RESOURCE_CONFIG}"), - (ResourceID(SPARK_DRIVER_PREFIX, "gpu").amountConf, + (new ResourceID(SPARK_DRIVER_PREFIX, "gpu").amountConf, s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${YARN_GPU_RESOURCE_CONFIG}")) val errorMessage = new mutable.StringBuilder() @@ -185,7 +187,24 @@ private object ResourceRequestHelper extends Logging { s"does not match pattern $AMOUNT_AND_UNIT_REGEX.") case CausedBy(e: IllegalArgumentException) => throw new IllegalArgumentException(s"Invalid request for $name: ${e.getMessage}") - case e: InvocationTargetException if e.getCause != null => throw e.getCause + case e: InvocationTargetException => + if (e.getCause != null) { + if (Try(Utils.classForName(RESOURCE_NOT_FOUND)).isSuccess) { + if (e.getCause().getClass().getName().equals(RESOURCE_NOT_FOUND)) { + // warn a couple times and then stop so we don't spam the logs + if (numResourceErrors < 2) { + logWarning(s"YARN doesn't know about resource $name, your resource discovery " + + s"has to handle properly discovering and isolating the resource! Error: " + + s"${e.getCause().getMessage}") + numResourceErrors += 1 + } + } else { + throw e.getCause + } + } else { + throw e.getCause + } + } } } } 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 7cce908cd5fb7..b42c8b933dda0 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 @@ -403,7 +403,7 @@ class ClientSuite extends SparkFunSuite with Matchers { conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnName}.${AMOUNT}", "2") } resources.values.foreach { rName => - conf.set(ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") + conf.set(new ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") } val error = intercept[SparkException] { @@ -426,7 +426,7 @@ class ClientSuite extends SparkFunSuite with Matchers { conf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnName}.${AMOUNT}", "2") } resources.values.foreach { rName => - conf.set(ResourceID(SPARK_EXECUTOR_PREFIX, rName).amountConf, "3") + conf.set(new ResourceID(SPARK_EXECUTOR_PREFIX, rName).amountConf, "3") } val error = intercept[SparkException] { @@ -450,7 +450,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, "cluster") resources.values.foreach { rName => - conf.set(ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") + conf.set(new ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") } // also just set yarn one that we don't convert conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}.${AMOUNT}", "5") From 35380958b89b13c70d24132e8b39c42d02f4a3d8 Mon Sep 17 00:00:00 2001 From: jiaan geng Date: Fri, 31 Jan 2020 21:14:11 -0800 Subject: [PATCH 1283/1387] [SPARK-30698][BUILD] Bumps checkstyle from 8.25 to 8.29 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? I found checkstyle have a new release https://checkstyle.org/releasenotes.html#Release_8.29 Bumps checkstyle from 8.25 to 8.29. ### Why are the changes needed? I have bump checkstyle from 8.25 to 8.29 on my fork branch and test to build. It's OK 8.29 added some new features: - New Check: AvoidNoArgumentSuperConstructorCall. - New Check NoEnumTrailingComma. - ENUM_DEF token support in RightCurlyCheck. - FallThrough module does not support the spelling "fall-through" by default. 8.29 fix some bugs: - Java 8 Grammar: annotations on varargs parameters. - Sonar violation: Disable XML external entity (XXE) processing. - Disable instantiation of modules with private ctor. - Sonar violation: "ThreadLocal" variables should be cleaned up when no longer used. - Indentation incorrect level for chained method with bracket on new line. - InvalidJavadocPosition: false positive when comment is between javadoc and package. ### Does this PR introduce any user-facing change? No ### How was this patch tested? No UT Closes #27426 from beliefer/bump-checkstyle. Authored-by: jiaan geng Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index eccc620e67690..7c23444054efc 100644 --- a/pom.xml +++ b/pom.xml @@ -2837,7 +2837,7 @@ com.puppycrawl.tools checkstyle - 8.25 + 8.29 From 534f5d409ace63b0eafc3a0545aab24d0424f253 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 1 Feb 2020 15:38:16 +0900 Subject: [PATCH 1284/1387] [SPARK-29138][PYTHON][TEST] Increase timeout of StreamingLogisticRegressionWithSGDTests.test_parameter_accuracy ### What changes were proposed in this pull request? This PR aims to increase the timeout of `StreamingLogisticRegressionWithSGDTests.test_parameter_accuracy` from 30s (default) to 60s. In this PR, before increasing the timeout, 1. I verified that this is not a JDK11 environmental issue by repeating 3 times first. 2. I reproduced the accuracy failure by reducing the timeout in Jenkins (https://github.com/apache/spark/pull/27424#issuecomment-580981262) Then, the final commit passed the Jenkins. ### Why are the changes needed? This seems to happen when Jenkins environment has congestion and the jobs are slowdown. The streaming job seems to be unable to repeat the designed iteration `numIteration=25` in 30 seconds. Since the error is decreasing at each iteration, the failure occurs. By reducing the timeout, we can reproduce the similar issue locally like Jenkins. ```python - eventually(condition, catch_assertions=True) + eventually(condition, timeout=10.0, catch_assertions=True) ``` ``` $ python/run-tests --testname 'pyspark.mllib.tests.test_streaming_algorithms StreamingLogisticRegressionWithSGDTests.test_parameter_accuracy' --python-executables=python ... ====================================================================== FAIL: test_parameter_accuracy (pyspark.mllib.tests.test_streaming_algorithms.StreamingLogisticRegressionWithSGDTests) ---------------------------------------------------------------------- Traceback (most recent call last): File "/Users/dongjoon/PRS/SPARK-TEST/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 229, in test_parameter_accuracy eventually(condition, timeout=10.0, catch_assertions=True) File "/Users/dongjoon/PRS/SPARK-TEST/python/pyspark/testing/utils.py", line 86, in eventually raise lastValue Reproduce the error File "/Users/dongjoon/PRS/SPARK-TEST/python/pyspark/testing/utils.py", line 77, in eventually lastValue = condition() File "/Users/dongjoon/PRS/SPARK-TEST/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 226, in condition self.assertAlmostEqual(rel, 0.1, 1) AssertionError: 0.25749106949322637 != 0.1 within 1 places (0.15749106949322636 difference) ---------------------------------------------------------------------- Ran 1 test in 14.814s FAILED (failures=1) ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins (and manual check by reducing the timeout). Since this is a flakiness issue depending on the Jenkins job situation, it's difficult to reproduce there. Closes #27424 from dongjoon-hyun/SPARK-TEST. Authored-by: Dongjoon Hyun Signed-off-by: HyukjinKwon --- python/pyspark/mllib/tests/test_streaming_algorithms.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/mllib/tests/test_streaming_algorithms.py b/python/pyspark/mllib/tests/test_streaming_algorithms.py index 8056debb963b4..2077809a043f1 100644 --- a/python/pyspark/mllib/tests/test_streaming_algorithms.py +++ b/python/pyspark/mllib/tests/test_streaming_algorithms.py @@ -226,7 +226,7 @@ def condition(): self.assertAlmostEqual(rel, 0.1, 1) return True - eventually(condition, catch_assertions=True) + eventually(condition, timeout=60.0, catch_assertions=True) def test_convergence(self): """ From da32d1e6b5cc409f408384576002ccf63a83e9a1 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Sat, 1 Feb 2020 15:19:16 +0800 Subject: [PATCH 1285/1387] [SPARK-30700][ML] NaiveBayesModel predict optimization ### What changes were proposed in this pull request? var `negThetaSum` is always used together with `pi`, so we can add them at first ### Why are the changes needed? only need to add one var `piMinusThetaSum`, instead of `pi` and `negThetaSum` ### Does this PR introduce any user-facing change? No ### How was this patch tested? existing testsuites Closes #27427 from zhengruifeng/nb_predict. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../spark/ml/classification/NaiveBayes.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 5305cbfcba547..94681ae9ef796 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -405,14 +405,14 @@ class NaiveBayesModel private[ml] ( * This precomputes log(1.0 - exp(theta)) and its sum which are used for the linear algebra * application of this condition (in predict function). */ - @transient private lazy val (thetaMinusNegTheta, negThetaSum) = $(modelType) match { + @transient private lazy val (thetaMinusNegTheta, piMinusThetaSum) = $(modelType) match { case Bernoulli => + val thetaMinusNegTheta = theta.map(value => value - math.log1p(-math.exp(value))) val negTheta = theta.map(value => math.log1p(-math.exp(value))) val ones = new DenseVector(Array.fill(theta.numCols)(1.0)) - val thetaMinusNegTheta = theta.map { value => - value - math.log1p(-math.exp(value)) - } - (thetaMinusNegTheta, negTheta.multiply(ones)) + val piMinusThetaSum = pi.toDense.copy + BLAS.gemv(1.0, negTheta, ones, 1.0, piMinusThetaSum) + (thetaMinusNegTheta, piMinusThetaSum) case _ => // This should never happen. throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}. " + @@ -445,8 +445,8 @@ class NaiveBayesModel private[ml] ( private def multinomialCalculation(features: Vector) = { requireNonnegativeValues(features) - val prob = theta.multiply(features) - BLAS.axpy(1.0, pi, prob) + val prob = pi.toDense.copy + BLAS.gemv(1.0, theta, features, 1.0, prob) prob } @@ -477,9 +477,8 @@ class NaiveBayesModel private[ml] ( private def bernoulliCalculation(features: Vector) = { requireZeroOneBernoulliValues(features) - val prob = thetaMinusNegTheta.multiply(features) - BLAS.axpy(1.0, pi, prob) - BLAS.axpy(1.0, negThetaSum, prob) + val prob = piMinusThetaSum.copy + BLAS.gemv(1.0, thetaMinusNegTheta, features, 1.0, prob) prob } From cd5f03a3ba18ae455f93abc5e5d04f098fa8f046 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 1 Feb 2020 20:50:47 -0800 Subject: [PATCH 1286/1387] [SPARK-27686][DOC][SQL] Update migration guide for make Hive 2.3 dependency by default ### What changes were proposed in this pull request? We have upgraded the built-in Hive from 1.2 to 2.3. This may need to set `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars` according to the version of your Hive metastore. Example: ``` --conf spark.sql.hive.metastore.version=1.2.1 --conf spark.sql.hive.metastore.jars=/root/hive-1.2.1-lib/* ``` Otherwise: ``` org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: Unable to fetch table spark_27686. Invalid method name: 'get_table_req'; at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:110) at org.apache.spark.sql.hive.HiveExternalCatalog.tableExists(HiveExternalCatalog.scala:841) at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.tableExists(ExternalCatalogWithListener.scala:146) at org.apache.spark.sql.catalyst.catalog.SessionCatalog.tableExists(SessionCatalog.scala:431) at org.apache.spark.sql.execution.command.CreateDataSourceTableCommand.run(createDataSourceTables.scala:52) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70) at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68) at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79) at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:226) at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3487) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$4(SQLExecution.scala:100) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:87) at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3485) at org.apache.spark.sql.Dataset.(Dataset.scala:226) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:96) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:607) ... 47 elided Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Unable to fetch table spark_27686. Invalid method name: 'get_table_req' at org.apache.hadoop.hive.ql.metadata.Hive.getTable(Hive.java:1282) at org.apache.spark.sql.hive.client.HiveClientImpl.getRawTableOption(HiveClientImpl.scala:422) at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$tableExists$1(HiveClientImpl.scala:436) at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:322) at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:256) at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:255) at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:305) at org.apache.spark.sql.hive.client.HiveClientImpl.tableExists(HiveClientImpl.scala:436) at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$tableExists$1(HiveExternalCatalog.scala:841) at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23) at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:100) ... 63 more Caused by: org.apache.thrift.TApplicationException: Invalid method name: 'get_table_req' at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_get_table_req(ThriftHiveMetastore.java:1567) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_table_req(ThriftHiveMetastore.java:1554) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.getTable(HiveMetaStoreClient.java:1350) at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.getTable(SessionHiveMetaStoreClient.java:127) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:173) at com.sun.proxy.$Proxy38.getTable(Unknown Source) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler.invoke(HiveMetaStoreClient.java:2336) at com.sun.proxy.$Proxy38.getTable(Unknown Source) at org.apache.hadoop.hive.ql.metadata.Hive.getTable(Hive.java:1274) ... 74 more ``` ### Why are the changes needed? Improve documentation. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? ```SKIP_API=1 jekyll build```: ![image](https://user-images.githubusercontent.com/5399861/73531432-67a50b80-4455-11ea-9401-5cad12fd3d14.png) Closes #27161 from wangyum/SPARK-27686. Authored-by: Yuming Wang Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 533c96a0832de..e4d2358b5de09 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -330,6 +330,9 @@ license: | - Since Spark 3.0, `SHOW CREATE TABLE` will always return Spark DDL, even when the given table is a Hive serde table. For Hive DDL, please use `SHOW CREATE TABLE AS SERDE` command instead. + - Since Spark 3.0, we upgraded the built-in Hive from 1.2 to 2.3. This may need to set `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars` according to the version of the Hive metastore. + For example: set `spark.sql.hive.metastore.version` to `1.2.1` and `spark.sql.hive.metastore.jars` to `maven` if your Hive metastore version is 1.2.1. + ## Upgrading from Spark SQL 2.4.4 to 2.4.5 - Since Spark 2.4.5, `TRUNCATE TABLE` command tries to set back original permission and ACLs during re-creating the table/partition paths. To restore the behaviour of earlier versions, set `spark.sql.truncateTable.ignorePermissionAcl.enabled` to `true`. From 1adf3520e3c753e6df8dccb752e8239de682a09a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 2 Feb 2020 00:44:25 -0800 Subject: [PATCH 1287/1387] [SPARK-30704][INFRA] Use jekyll-redirect-from 0.15.0 instead of the latest ### What changes were proposed in this pull request? This PR aims to pin the version of `jekyll-redirect-from` to 0.15.0. This is a release blocker for both Apache Spark 3.0.0 and 2.4.5. ### Why are the changes needed? `jekyll-redirect-from` released 0.16.0 a few days ago and that requires Ruby 2.4.0. - https://github.com/jekyll/jekyll-redirect-from/releases/tag/v0.16.0 ``` $ cd dev/create-release/spark-rm/ $ docker build -t spark:test . ... ERROR: Error installing jekyll-redirect-from: jekyll-redirect-from requires Ruby version >= 2.4.0. ... ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manually do the above command to build `spark-rm` Docker image. ``` ... Successfully installed jekyll-redirect-from-0.15.0 Parsing documentation for jekyll-redirect-from-0.15.0 Installing ri documentation for jekyll-redirect-from-0.15.0 Done installing documentation for jekyll-redirect-from after 0 seconds 1 gem installed Successfully installed rouge-3.15.0 Parsing documentation for rouge-3.15.0 Installing ri documentation for rouge-3.15.0 Done installing documentation for rouge after 4 seconds 1 gem installed Removing intermediate container e0ec7c77b69f ---> 32dec37291c6 ``` Closes #27434 from dongjoon-hyun/SPARK-30704. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/create-release/spark-rm/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 3ba8e97929613..63451687ee8c2 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -78,7 +78,7 @@ RUN apt-get clean && apt-get update && $APT_INSTALL gnupg ca-certificates && \ # Install tools needed to build the documentation. $APT_INSTALL ruby2.3 ruby2.3-dev mkdocs && \ gem install jekyll --no-rdoc --no-ri -v 3.8.6 && \ - gem install jekyll-redirect-from && \ + gem install jekyll-redirect-from -v 0.15.0 && \ gem install rouge WORKDIR /opt/spark-rm/output From fb321b666dd4cea1e4d3b8387c79980700939a15 Mon Sep 17 00:00:00 2001 From: zero323 Date: Mon, 3 Feb 2020 11:09:25 +0900 Subject: [PATCH 1288/1387] [MINOR][SPARKR][DOCS] Remove duplicate @name tags from read.df and read.stream ### What changes were proposed in this pull request? Remove duplicate `name` tags from `read.df` and `read.stream`. ### Why are the changes needed? These tags are already present in https://github.com/apache/spark/blob/1adf3520e3c753e6df8dccb752e8239de682a09a/R/pkg/R/SQLContext.R#L546 and https://github.com/apache/spark/blob/1adf3520e3c753e6df8dccb752e8239de682a09a/R/pkg/R/SQLContext.R#L678 for `read.df` and `read.stream` respectively. As only one `name` tag per block is allowed, this causes build warnings with recent `roxygen2` versions: ``` Warning: [/path/to/spark/R/pkg/R/SQLContext.R:559] name May only use one name per block Warning: [/path/to/spark/R/pkg/R/SQLContext.R:690] name May only use one name per block ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #27437 from zero323/roxygen-warnings-names. Authored-by: zero323 Signed-off-by: HyukjinKwon --- R/pkg/R/SQLContext.R | 2 -- 1 file changed, 2 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index f48a334ed6766..c6842912706af 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -556,7 +556,6 @@ tableToDF <- function(tableName) { #' stringSchema <- "name STRING, info MAP" #' df4 <- read.df(mapTypeJsonPath, "json", stringSchema, multiLine = TRUE) #' } -#' @name read.df #' @note read.df since 1.4.0 read.df <- function(path = NULL, source = NULL, schema = NULL, na.strings = "NA", ...) { if (!is.null(path) && !is.character(path)) { @@ -687,7 +686,6 @@ read.jdbc <- function(url, tableName, #' stringSchema <- "name STRING, info MAP" #' df1 <- read.stream("json", path = jsonDir, schema = stringSchema, maxFilesPerTrigger = 1) #' } -#' @name read.stream #' @note read.stream since 2.2.0 #' @note experimental read.stream <- function(source = NULL, schema = NULL, ...) { From 2eccfd8a73c4afa30a6aa97c2afd38661f29e24b Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 3 Feb 2020 14:08:59 +0800 Subject: [PATCH 1289/1387] [SPARK-30697][SQL] Handle database and namespace exceptions in catalog.isView ### What changes were proposed in this pull request? Adds NoSuchDatabaseException and NoSuchNamespaceException to the `isView` method for SessionCatalog. ### Why are the changes needed? This method prevents specialized resolutions from kicking in within Analysis when using V2 Catalogs if the identifier is a specialized identifier. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added test to DataSourceV2SessionCatalogSuite Closes #27423 from brkyvz/isViewF. Authored-by: Burak Yavuz Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/SessionCatalog.scala | 2 ++ ...SourceV2DataFrameSessionCatalogSuite.scala | 22 +++++++++++++++++++ .../DataSourceV2SQLSessionCatalogSuite.scala | 14 ++++++++++++ 3 files changed, 38 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 45f0ef6c97e70..12f9a61fc2b65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -826,6 +826,8 @@ class SessionCatalog( getTempViewOrPermanentTableMetadata(ident).tableType == CatalogTableType.VIEW } catch { case _: NoSuchTableException => false + case _: NoSuchDatabaseException => false + case _: NoSuchNamespaceException => false } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala index 4c67888cbdc48..01caf8e2eb115 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala @@ -101,6 +101,13 @@ class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable new InMemoryTable(name, schema, partitions, properties) } + override def loadTable(ident: Identifier): Table = { + val identToUse = Option(InMemoryTableSessionCatalog.customIdentifierResolution) + .map(_(ident)) + .getOrElse(ident) + super.loadTable(identToUse) + } + override def alterTable(ident: Identifier, changes: TableChange*): Table = { val fullIdent = fullIdentifier(ident) Option(tables.get(fullIdent)) match { @@ -125,6 +132,21 @@ class InMemoryTableSessionCatalog extends TestV2SessionCatalogBase[InMemoryTable } } +object InMemoryTableSessionCatalog { + private var customIdentifierResolution: Identifier => Identifier = _ + + def withCustomIdentifierResolver( + resolver: Identifier => Identifier)( + f: => Unit): Unit = { + try { + customIdentifierResolution = resolver + f + } finally { + customIdentifierResolution = null + } + } +} + private [connector] trait SessionCatalogTest[T <: Table, Catalog <: TestV2SessionCatalogBase[T]] extends QueryTest with SharedSparkSession diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala index 27725bcadbcd5..b6997445013e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala @@ -49,4 +49,18 @@ class DataSourceV2SQLSessionCatalogSuite v2Catalog.asInstanceOf[TableCatalog] .loadTable(Identifier.of(Array.empty, nameParts.last)) } + + test("SPARK-30697: catalog.isView doesn't throw an error for specialized identifiers") { + val t1 = "tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") + + def idResolver(id: Identifier): Identifier = Identifier.of(Array.empty, id.name()) + + InMemoryTableSessionCatalog.withCustomIdentifierResolver(idResolver) { + // The following should not throw AnalysisException. + sql(s"DESCRIBE TABLE ignored.$t1") + } + } + } } From a4912cee615314e9578e6ab4eae25f147feacbd5 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sun, 2 Feb 2020 23:37:13 -0800 Subject: [PATCH 1290/1387] [SPARK-29543][SS][FOLLOWUP] Move `spark.sql.streaming.ui.*` configs to StaticSQLConf ### What changes were proposed in this pull request? Put the configs below needed by Structured Streaming UI into StaticSQLConf: - spark.sql.streaming.ui.enabled - spark.sql.streaming.ui.retainedProgressUpdates - spark.sql.streaming.ui.retainedQueries ### Why are the changes needed? Make all SS UI configs consistent with other similar configs in usage and naming. ### Does this PR introduce any user-facing change? Yes, add new static config `spark.sql.streaming.ui.retainedProgressUpdates`. ### How was this patch tested? Existing UT. Closes #27425 from xuanyuanking/SPARK-29543-follow. Authored-by: Yuanjian Li Signed-off-by: Shixiong Zhu --- .../apache/spark/sql/internal/SQLConf.scala | 16 --------------- .../spark/sql/internal/StaticSQLConf.scala | 20 +++++++++++++++++++ .../spark/sql/internal/SharedState.scala | 15 +++++++------- .../ui/StreamingQueryStatusListener.scala | 10 ++++++---- .../sql/streaming/ui/StreamingQueryTab.scala | 2 +- .../StreamingQueryStatusListenerSuite.scala | 4 ++-- 6 files changed, 37 insertions(+), 30 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 04572c38be8dd..3ad3416256c7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1150,18 +1150,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val STREAMING_UI_ENABLED = - buildConf("spark.sql.streaming.ui.enabled") - .doc("Whether to run the structured streaming UI for the Spark application.") - .booleanConf - .createWithDefault(true) - - val STREAMING_UI_INACTIVE_QUERY_RETENTION = - buildConf("spark.sql.streaming.ui.numInactiveQueries") - .doc("The number of inactive queries to retain for structured streaming ui.") - .intConf - .createWithDefault(100) - val VARIABLE_SUBSTITUTE_ENABLED = buildConf("spark.sql.variable.substitute") .doc("This enables substitution using syntax like ${var} ${system:var} and ${env:var}.") @@ -2284,10 +2272,6 @@ class SQLConf extends Serializable with Logging { def isUnsupportedOperationCheckEnabled: Boolean = getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED) - def isStreamingUIEnabled: Boolean = getConf(STREAMING_UI_ENABLED) - - def streamingUIInactiveQueryRetention: Int = getConf(STREAMING_UI_INACTIVE_QUERY_RETENTION) - def streamingFileCommitProtocolClass: String = getConf(STREAMING_FILE_COMMIT_PROTOCOL_CLASS) def fileSinkLogDeletion: Boolean = getConf(FILE_SINK_LOG_DELETION) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 66ac9ddb21aaa..6bc752260a893 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -176,4 +176,24 @@ object StaticSQLConf { .internal() .booleanConf .createWithDefault(true) + + val STREAMING_UI_ENABLED = + buildStaticConf("spark.sql.streaming.ui.enabled") + .doc("Whether to run the Structured Streaming Web UI for the Spark application when the " + + "Spark Web UI is enabled.") + .booleanConf + .createWithDefault(true) + + val STREAMING_UI_RETAINED_PROGRESS_UPDATES = + buildStaticConf("spark.sql.streaming.ui.retainedProgressUpdates") + .doc("The number of progress updates to retain for a streaming query for Structured " + + "Streaming UI.") + .intConf + .createWithDefault(100) + + val STREAMING_UI_RETAINED_QUERIES = + buildStaticConf("spark.sql.streaming.ui.retainedQueries") + .doc("The number of inactive queries to retain for Structured Streaming UI.") + .intConf + .createWithDefault(100) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index fefd72dcf1752..5347264d7c50a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -145,13 +145,14 @@ private[sql] class SharedState( * data to show. */ lazy val streamingQueryStatusListener: Option[StreamingQueryStatusListener] = { - val sqlConf = SQLConf.get - if (sqlConf.isStreamingUIEnabled) { - val statusListener = new StreamingQueryStatusListener(sqlConf) - sparkContext.ui.foreach(new StreamingQueryTab(statusListener, _)) - Some(statusListener) - } else { - None + sparkContext.ui.flatMap { ui => + if (conf.get(STREAMING_UI_ENABLED)) { + val statusListener = new StreamingQueryStatusListener(conf) + new StreamingQueryTab(statusListener, ui) + Some(statusListener) + } else { + None + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala index db085dbe87ec4..91815110e0d39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListener.scala @@ -24,8 +24,9 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import scala.collection.mutable +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryProgress} /** @@ -33,7 +34,7 @@ import org.apache.spark.sql.streaming.{StreamingQueryListener, StreamingQueryPro * UI data for both active and inactive query. * TODO: Add support for history server. */ -private[sql] class StreamingQueryStatusListener(sqlConf: SQLConf) extends StreamingQueryListener { +private[sql] class StreamingQueryStatusListener(conf: SparkConf) extends StreamingQueryListener { private val timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 timestampFormat.setTimeZone(DateTimeUtils.getTimeZone("UTC")) @@ -45,8 +46,9 @@ private[sql] class StreamingQueryStatusListener(sqlConf: SQLConf) extends Stream private[ui] val activeQueryStatus = new ConcurrentHashMap[UUID, StreamingQueryUIData]() private[ui] val inactiveQueryStatus = new mutable.Queue[StreamingQueryUIData]() - private val streamingProgressRetention = sqlConf.streamingProgressRetention - private val inactiveQueryStatusRetention = sqlConf.streamingUIInactiveQueryRetention + private val streamingProgressRetention = + conf.get(StaticSQLConf.STREAMING_UI_RETAINED_PROGRESS_UPDATES) + private val inactiveQueryStatusRetention = conf.get(StaticSQLConf.STREAMING_UI_RETAINED_QUERIES) override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = { activeQueryStatus.putIfAbsent(event.runId, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala index f909cfd97514e..bb097ffc06912 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryTab.scala @@ -34,6 +34,6 @@ private[sql] class StreamingQueryTab( parent.addStaticHandler(StreamingQueryTab.STATIC_RESOURCE_DIR, "/static/sql") } -object StreamingQueryTab { +private[sql] object StreamingQueryTab { private val STATIC_RESOURCE_DIR = "org/apache/spark/sql/execution/ui/static" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala index bd74ed340b408..adbb501f9842e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.streaming class StreamingQueryStatusListenerSuite extends StreamTest { test("onQueryStarted, onQueryProgress, onQueryTerminated") { - val listener = new StreamingQueryStatusListener(spark.sqlContext.conf) + val listener = new StreamingQueryStatusListener(spark.sparkContext.conf) // hanlde query started event val id = UUID.randomUUID() @@ -73,7 +73,7 @@ class StreamingQueryStatusListenerSuite extends StreamTest { } test("same query start multiple times") { - val listener = new StreamingQueryStatusListener(spark.sqlContext.conf) + val listener = new StreamingQueryStatusListener(spark.sparkContext.conf) // handle first time start val id = UUID.randomUUID() From 47659a0675e4bb0403d1ebe207ec697a78936d04 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Tue, 4 Feb 2020 01:24:53 +0800 Subject: [PATCH 1291/1387] [SPARK-30525][SQL] HiveTableScanExec do not need to prune partitions again after pushing down to SessionCatalog for partition pruning ### What changes were proposed in this pull request? HiveTableScanExec does not prune partitions again after SessionCatalog.listPartitionsByFilter called. ### Why are the changes needed? In HiveTableScanExec, it will push down to hive metastore for partition pruning if spark.sql.hive.metastorePartitionPruning is true, and then it will prune the returned partitions again using partition filters, because some predicates, eg. "b like 'xyz'", are not supported in hive metastore. But now this problem is already fixed in HiveExternalCatalog.listPartitionsByFilter, the HiveExternalCatalog.listPartitionsByFilter can return exactly what we want now. So it is not necessary any more to double prune in HiveTableScanExec. ### Does this PR introduce any user-facing change? no ### How was this patch tested? Existing unit tests. Closes #27232 from fuwhu/SPARK-30525. Authored-by: fuwhu Signed-off-by: Wenchen Fan --- .../hive/execution/HiveTableScanExec.scala | 32 +++++++++++++++---- .../sql/hive/execution/PruningSuite.scala | 2 +- 2 files changed, 26 insertions(+), 8 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index 9f0f05023bc24..4dccacef337e9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -146,7 +146,7 @@ case class HiveTableScanExec( * @param partitions All partitions of the relation. * @return Partitions that are involved in the query plan. */ - private[hive] def prunePartitions(partitions: Seq[HivePartition]) = { + private[hive] def prunePartitions(partitions: Seq[HivePartition]): Seq[HivePartition] = { boundPruningPred match { case None => partitions case Some(shouldKeep) => partitions.filter { part => @@ -162,18 +162,36 @@ case class HiveTableScanExec( } } + @transient lazy val prunedPartitions: Seq[HivePartition] = { + if (relation.prunedPartitions.nonEmpty) { + val hivePartitions = + relation.prunedPartitions.get.map(HiveClientImpl.toHivePartition(_, hiveQlTable)) + if (partitionPruningPred.forall(!ExecSubqueryExpression.hasSubquery(_))) { + hivePartitions + } else { + prunePartitions(hivePartitions) + } + } else { + if (sparkSession.sessionState.conf.metastorePartitionPruning && + partitionPruningPred.nonEmpty) { + rawPartitions + } else { + prunePartitions(rawPartitions) + } + } + } + // exposed for tests - @transient lazy val rawPartitions = { + @transient lazy val rawPartitions: Seq[HivePartition] = { val prunedPartitions = if (sparkSession.sessionState.conf.metastorePartitionPruning && - partitionPruningPred.nonEmpty) { + partitionPruningPred.nonEmpty) { // Retrieve the original attributes based on expression ID so that capitalization matches. val normalizedFilters = partitionPruningPred.map(_.transform { case a: AttributeReference => originalAttributes(a) }) - relation.prunedPartitions.getOrElse( - sparkSession.sessionState.catalog - .listPartitionsByFilter(relation.tableMeta.identifier, normalizedFilters)) + sparkSession.sessionState.catalog + .listPartitionsByFilter(relation.tableMeta.identifier, normalizedFilters) } else { sparkSession.sessionState.catalog.listPartitions(relation.tableMeta.identifier) } @@ -189,7 +207,7 @@ case class HiveTableScanExec( } } else { Utils.withDummyCallSite(sqlContext.sparkContext) { - hadoopReader.makeRDDForPartitionedTable(prunePartitions(rawPartitions)) + hadoopReader.makeRDDForPartitionedTable(prunedPartitions) } } val numOutputRows = longMetric("numOutputRows") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 390181ce5d46e..985281bce3036 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -161,7 +161,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { case p @ HiveTableScanExec(columns, relation, _) => val columnNames = columns.map(_.name) val partValues = if (relation.isPartitioned) { - p.prunePartitions(p.rawPartitions).map(_.getValues) + p.prunedPartitions.map(_.getValues) } else { Seq.empty } From 41bdb7ad3949d05542abe5ab2b440a51c3a18bce Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 3 Feb 2020 19:57:16 -0800 Subject: [PATCH 1292/1387] [SPARK-30718][BUILD] Exclude jdk.tools dependency from hadoop-yarn-api ### What changes were proposed in this pull request? This PR removes the `jdk.tools:jdk.tools` transitive dependency from `hadoop-yarn-api`. - This is only used in `hadoop-annotation` project in some `*Doclet.java`. ### Why are the changes needed? Although this is not used in Apache Spark, this can cause a resolve failure in JDK11 environment. jdk tools ### Does this PR introduce any user-facing change? No. This is a dev-only change. From developers, this will remove the `Cannot resolve` error in IDE environment. ### How was this patch tested? - Pass the Jenkins in JDK8 - Manually, import the project with JDK11. Closes #27445 from dongjoon-hyun/SPARK-30718. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index 7c23444054efc..a8d6ac932bac2 100644 --- a/pom.xml +++ b/pom.xml @@ -1200,6 +1200,10 @@ com.sun.jersey.contribs * + + jdk.tools + jdk.tools + From 6097b343baa8e4a8bc7159dc3d394f13b3c9959b Mon Sep 17 00:00:00 2001 From: maryannxue Date: Tue, 4 Feb 2020 12:31:44 +0800 Subject: [PATCH 1293/1387] [SPARK-30717][SQL] AQE subquery map should cache `SubqueryExec` instead of `ExecSubqueryExpression` ### What changes were proposed in this pull request? This PR is to fix a potential bug in AQE where an `ExecSubqueryExpression` could be mistakenly replaced with another `ExecSubqueryExpression` with the same `ListQuery` but a different `child` expression. This is because a ListQuery's id can only identify the ListQuery itself, not the parent expression `InSubquery`, but right now the `subqueryMap` in `InsertAdaptiveSparkPlan` uses the `ListQuery`'s id as key and the corresponding `InSubqueryExec` for the `ListQuery`'s parent expression as value. So the fix uses the corresponding `SubqueryExec` for the `ListQuery` itself as the map's value. ### Why are the changes needed? This logical bug could potentially cause a wrong query plan, which could throw an exception related to unresolved columns. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Passed existing UTs. Closes #27446 from maryannxue/spark-30717. Authored-by: maryannxue Signed-off-by: Wenchen Fan --- .../adaptive/InsertAdaptiveSparkPlan.scala | 30 ++++++------------- .../adaptive/PlanAdaptiveSubqueries.scala | 23 +++++++++----- 2 files changed, 25 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 04696209ce10e..9252827856af4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -20,11 +20,10 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, DynamicPruningSubquery, ListQuery, Literal, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{DynamicPruningSubquery, ListQuery, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.exchange.Exchange @@ -102,36 +101,25 @@ case class InsertAdaptiveSparkPlan( * For each sub-query, generate the adaptive execution plan for each sub-query by applying this * rule, or reuse the execution plan from another sub-query of the same semantics if possible. */ - private def buildSubqueryMap(plan: SparkPlan): mutable.HashMap[Long, ExecSubqueryExpression] = { - val subqueryMap = mutable.HashMap.empty[Long, ExecSubqueryExpression] + private def buildSubqueryMap(plan: SparkPlan): Map[Long, SubqueryExec] = { + val subqueryMap = mutable.HashMap.empty[Long, SubqueryExec] plan.foreach(_.expressions.foreach(_.foreach { case expressions.ScalarSubquery(p, _, exprId) if !subqueryMap.contains(exprId.id) => val executedPlan = compileSubquery(p) verifyAdaptivePlan(executedPlan, p) - val scalarSubquery = execution.ScalarSubquery( - SubqueryExec(s"subquery${exprId.id}", executedPlan), exprId) - subqueryMap.put(exprId.id, scalarSubquery) - case expressions.InSubquery(values, ListQuery(query, _, exprId, _)) + val subquery = SubqueryExec(s"subquery${exprId.id}", executedPlan) + subqueryMap.put(exprId.id, subquery) + case expressions.InSubquery(_, ListQuery(query, _, exprId, _)) if !subqueryMap.contains(exprId.id) => val executedPlan = compileSubquery(query) verifyAdaptivePlan(executedPlan, query) - val expr = if (values.length == 1) { - values.head - } else { - CreateNamedStruct( - values.zipWithIndex.flatMap { case (v, index) => - Seq(Literal(s"col_$index"), v) - } - ) - } - val inSubquery = InSubqueryExec(expr, - SubqueryExec(s"subquery#${exprId.id}", executedPlan), exprId) - subqueryMap.put(exprId.id, inSubquery) + val subquery = SubqueryExec(s"subquery#${exprId.id}", executedPlan) + subqueryMap.put(exprId.id, subquery) case _ => })) - subqueryMap + subqueryMap.toMap } def compileSubquery(plan: LogicalPlan): SparkPlan = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala index 91d4359224a6a..f845b6b16ee3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala @@ -18,19 +18,28 @@ package org.apache.spark.sql.execution.adaptive import org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions.ListQuery +import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, ListQuery, Literal} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{ExecSubqueryExpression, SparkPlan} +import org.apache.spark.sql.execution +import org.apache.spark.sql.execution.{InSubqueryExec, SparkPlan, SubqueryExec} -case class PlanAdaptiveSubqueries( - subqueryMap: scala.collection.Map[Long, ExecSubqueryExpression]) extends Rule[SparkPlan] { +case class PlanAdaptiveSubqueries(subqueryMap: Map[Long, SubqueryExec]) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = { plan.transformAllExpressions { case expressions.ScalarSubquery(_, _, exprId) => - subqueryMap(exprId.id) - case expressions.InSubquery(_, ListQuery(_, _, exprId, _)) => - subqueryMap(exprId.id) + execution.ScalarSubquery(subqueryMap(exprId.id), exprId) + case expressions.InSubquery(values, ListQuery(_, _, exprId, _)) => + val expr = if (values.length == 1) { + values.head + } else { + CreateNamedStruct( + values.zipWithIndex.flatMap { case (v, index) => + Seq(Literal(s"col_$index"), v) + } + ) + } + InSubqueryExec(expr, subqueryMap(exprId.id), exprId) } } } From 0202b675afca65c6615a06805a4d4d12f3f97bdb Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 4 Feb 2020 16:33:34 +0900 Subject: [PATCH 1294/1387] [SPARK-26618][SQL][FOLLOWUP] Describe the behavior change of typed `TIMESTAMP`/`DATE` literals ### What changes were proposed in this pull request? In the PR, I propose to update the SQL migration guide, and clarify behavior change of typed `TIMESTAMP` and `DATE` literals for input strings without time zone information - local timestamp and date strings. ### Why are the changes needed? To inform users that the typed literals may change their behavior in Spark 3.0 because of different sources of the default time zone - JVM system time zone in Spark 2.4 and earlier, and `spark.sql.session.timeZone` in Spark 3.0. ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #27435 from MaxGekk/timestamp-lit-migration-guide. Authored-by: Maxim Gekk Signed-off-by: HyukjinKwon --- docs/sql-migration-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index e4d2358b5de09..a5ef1c2b1d045 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -77,7 +77,7 @@ license: | - Formatting of `TIMESTAMP` and `DATE` literals. - - Creating of typed `TIMESTAMP` and `DATE` literals from strings. Since Spark 3.0, string conversion to typed `TIMESTAMP`/`DATE` literals is performed via casting to `TIMESTAMP`/`DATE` values. For example, `TIMESTAMP '2019-12-23 12:59:30'` is semantically equal to `CAST('2019-12-23 12:59:30' AS TIMESTAMP)`. In Spark version 2.4 and earlier, the `java.sql.Timestamp.valueOf()` and `java.sql.Date.valueOf()` functions are used for the same purpose. + - Creating of typed `TIMESTAMP` and `DATE` literals from strings. Since Spark 3.0, string conversion to typed `TIMESTAMP`/`DATE` literals is performed via casting to `TIMESTAMP`/`DATE` values. For example, `TIMESTAMP '2019-12-23 12:59:30'` is semantically equal to `CAST('2019-12-23 12:59:30' AS TIMESTAMP)`. When the input string does not contain information about time zone, the time zone from the SQL config `spark.sql.session.timeZone` is used in that case. In Spark version 2.4 and earlier, the conversion is based on JVM system time zone. The different sources of the default time zone may change the behavior of typed `TIMESTAMP` and `DATE` literals. - In Spark version 2.4 and earlier, invalid time zone ids are silently ignored and replaced by GMT time zone, for example, in the from_utc_timestamp function. Since Spark 3.0, such time zone ids are rejected, and Spark throws `java.time.DateTimeException`. From f2dd082544aeba5978d0c140d0194eedb969d132 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 4 Feb 2020 21:17:05 +0800 Subject: [PATCH 1295/1387] [SPARK-30725][SQL] Make legacy SQL configs as internal configs ### What changes were proposed in this pull request? All legacy SQL configs are marked as internal configs. In particular, the following configs are updated as internals: - spark.sql.legacy.sizeOfNull - spark.sql.legacy.replaceDatabricksSparkAvro.enabled - spark.sql.legacy.typeCoercion.datetimeToString.enabled - spark.sql.legacy.looseUpcast - spark.sql.legacy.arrayExistsFollowsThreeValuedLogic ### Why are the changes needed? In general case, users shouldn't change legacy configs, so, they can be marked as internals. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Should be tested by jenkins build and run tests. Closes #27448 from MaxGekk/legacy-internal-sql-conf. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3ad3416256c7d..b94ddbdc0fc9a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1916,6 +1916,7 @@ object SQLConf { .createWithDefault(Deflater.DEFAULT_COMPRESSION) val LEGACY_SIZE_OF_NULL = buildConf("spark.sql.legacy.sizeOfNull") + .internal() .doc("If it is set to true, size of null returns -1. This behavior was inherited from Hive. " + "The size function returns null for null input if the flag is disabled.") .booleanConf @@ -1923,6 +1924,7 @@ object SQLConf { val LEGACY_REPLACE_DATABRICKS_SPARK_AVRO_ENABLED = buildConf("spark.sql.legacy.replaceDatabricksSparkAvro.enabled") + .internal() .doc("If it is set to true, the data source provider com.databricks.spark.avro is mapped " + "to the built-in but external Avro data source module for backward compatibility.") .booleanConf @@ -2048,10 +2050,11 @@ object SQLConf { val LEGACY_CAST_DATETIME_TO_STRING = buildConf("spark.sql.legacy.typeCoercion.datetimeToString.enabled") + .internal() .doc("If it is set to true, date/timestamp will cast to string in binary comparisons " + "with String") - .booleanConf - .createWithDefault(false) + .booleanConf + .createWithDefault(false) val DEFAULT_CATALOG = buildConf("spark.sql.defaultCatalog") .doc("Name of the default catalog. This will be the current catalog if users have not " + @@ -2071,6 +2074,7 @@ object SQLConf { .createOptional val LEGACY_LOOSE_UPCAST = buildConf("spark.sql.legacy.looseUpcast") + .internal() .doc("When true, the upcast will be loose and allows string to atomic types.") .booleanConf .createWithDefault(false) @@ -2083,6 +2087,7 @@ object SQLConf { val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC = buildConf("spark.sql.legacy.arrayExistsFollowsThreeValuedLogic") + .internal() .doc("When true, the ArrayExists will follow the three-valued boolean logic.") .booleanConf .createWithDefault(true) From 7631275f974d2ecf68cd8394ed683e30be320e56 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 4 Feb 2020 17:22:23 -0800 Subject: [PATCH 1296/1387] [SPARK-25040][SQL][FOLLOWUP] Add legacy config for allowing empty strings for certain types in json parser ### What changes were proposed in this pull request? This is a follow-up for #22787. In #22787 we disallowed empty strings for json parser except for string and binary types. This follow-up adds a legacy config for restoring previous behavior of allowing empty string. ### Why are the changes needed? Adding a legacy config to make migration easy for Spark users. ### Does this PR introduce any user-facing change? Yes. If set this legacy config to true, the users can restore previous behavior prior to Spark 3.0.0. ### How was this patch tested? Unit test. Closes #27456 from viirya/SPARK-25040-followup. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 2 +- .../sql/catalyst/json/JacksonParser.scala | 14 ++++- .../apache/spark/sql/internal/SQLConf.scala | 8 +++ .../datasources/json/JsonSuite.scala | 61 ++++++++++++++----- 4 files changed, 68 insertions(+), 17 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index a5ef1c2b1d045..0c47370283736 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -37,7 +37,7 @@ license: | - Since Spark 3.0, the Dataset and DataFrame API `unionAll` is not deprecated any more. It is an alias for `union`. - - In Spark version 2.4 and earlier, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType` and `DoubleType`, it fails on empty strings and throws exceptions. Since Spark 3.0, we disallow empty strings and will throw exceptions for data types except for `StringType` and `BinaryType`. + - In Spark version 2.4 and earlier, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType`, `DoubleType`, `DateType` and `TimestampType`, it fails on empty strings and throws exceptions. Since Spark 3.0, we disallow empty strings and will throw exceptions for data types except for `StringType` and `BinaryType`. The previous behaviour of allowing empty string can be restored by setting `spark.sql.legacy.json.allowEmptyString.enabled` to `true`. - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index c44025ca8bcfd..76efa574a99ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.Utils @@ -307,6 +308,8 @@ class JacksonParser( } } + private val allowEmptyString = SQLConf.get.getConf(SQLConf.LEGACY_ALLOW_EMPTY_STRING_IN_JSON) + /** * This function throws an exception for failed conversion. For empty string on data types * except for string and binary types, this also throws an exception. @@ -315,7 +318,16 @@ class JacksonParser( parser: JsonParser, dataType: DataType): PartialFunction[JsonToken, R] = { - // SPARK-25040: Disallow empty strings for data types except for string and binary types. + // SPARK-25040: Disallows empty strings for data types except for string and binary types. + // But treats empty strings as null for certain types if the legacy config is enabled. + case VALUE_STRING if parser.getTextLength < 1 && allowEmptyString => + dataType match { + case FloatType | DoubleType | TimestampType | DateType => + throw new RuntimeException( + s"Failed to parse an empty string for data type ${dataType.catalogString}") + case _ => null + } + case VALUE_STRING if parser.getTextLength < 1 => throw new RuntimeException( s"Failed to parse an empty string for data type ${dataType.catalogString}") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b94ddbdc0fc9a..5ce5692123805 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1989,6 +1989,14 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_ALLOW_EMPTY_STRING_IN_JSON = + buildConf("spark.sql.legacy.json.allowEmptyString.enabled") + .internal() + .doc("When set to true, the parser of JSON data source treats empty strings as null for " + + "some data types such as `IntegerType`.") + .booleanConf + .createWithDefault(false) + val TRUNCATE_TABLE_IGNORE_PERMISSION_ACL = buildConf("spark.sql.truncateTable.ignorePermissionAcl.enabled") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index d0e2e001034fb..b20da2266b0f3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2436,23 +2436,24 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson } } - test("SPARK-25040: empty strings should be disallowed") { - def failedOnEmptyString(dataType: DataType): Unit = { - val df = spark.read.schema(s"a ${dataType.catalogString}") - .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS) - val errMessage = intercept[SparkException] { - df.collect() - }.getMessage - assert(errMessage.contains( - s"Failed to parse an empty string for data type ${dataType.catalogString}")) - } - def emptyString(dataType: DataType, expected: Any): Unit = { - val df = spark.read.schema(s"a ${dataType.catalogString}") - .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS) - checkAnswer(df, Row(expected) :: Nil) - } + private def failedOnEmptyString(dataType: DataType): Unit = { + val df = spark.read.schema(s"a ${dataType.catalogString}") + .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS) + val errMessage = intercept[SparkException] { + df.collect() + }.getMessage + assert(errMessage.contains( + s"Failed to parse an empty string for data type ${dataType.catalogString}")) + } + private def emptyString(dataType: DataType, expected: Any): Unit = { + val df = spark.read.schema(s"a ${dataType.catalogString}") + .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS) + checkAnswer(df, Row(expected) :: Nil) + } + + test("SPARK-25040: empty strings should be disallowed") { failedOnEmptyString(BooleanType) failedOnEmptyString(ByteType) failedOnEmptyString(ShortType) @@ -2471,6 +2472,36 @@ abstract class JsonSuite extends QueryTest with SharedSparkSession with TestJson emptyString(BinaryType, "".getBytes(StandardCharsets.UTF_8)) } + test("SPARK-25040: allowing empty strings when legacy config is enabled") { + def emptyStringAsNull(dataType: DataType): Unit = { + val df = spark.read.schema(s"a ${dataType.catalogString}") + .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS) + checkAnswer(df, Row(null) :: Nil) + } + + // Legacy mode prior to Spark 3.0.0 + withSQLConf(SQLConf.LEGACY_ALLOW_EMPTY_STRING_IN_JSON.key -> "true") { + emptyStringAsNull(BooleanType) + emptyStringAsNull(ByteType) + emptyStringAsNull(ShortType) + emptyStringAsNull(IntegerType) + emptyStringAsNull(LongType) + + failedOnEmptyString(FloatType) + failedOnEmptyString(DoubleType) + failedOnEmptyString(TimestampType) + failedOnEmptyString(DateType) + + emptyStringAsNull(DecimalType.SYSTEM_DEFAULT) + emptyStringAsNull(ArrayType(IntegerType)) + emptyStringAsNull(MapType(StringType, IntegerType, true)) + emptyStringAsNull(StructType(StructField("f1", IntegerType, true) :: Nil)) + + emptyString(StringType, "") + emptyString(BinaryType, "".getBytes(StandardCharsets.UTF_8)) + } + } + test("return partial result for bad records") { val schema = "a double, b array, c string, _corrupt_record string" val badRecords = Seq( From 898716980dce44a4cc09411e72d64c848698cad5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 4 Feb 2020 17:26:46 -0800 Subject: [PATCH 1297/1387] Revert "[SPARK-28310][SQL] Support (FIRST_VALUE|LAST_VALUE)(expr[ (IGNORE|RESPECT) NULLS]?) syntax" ### What changes were proposed in this pull request? This reverts commit b89c3de1a439ed7302dd8f44c49b89bb7da2eebe. ### Why are the changes needed? `FIRST_VALUE` is used only for window expression. Please see the discussion on https://github.com/apache/spark/pull/25082 . ### Does this PR introduce any user-facing change? Yes. ### How was this patch tested? Pass the Jenkins. Closes #27458 from dongjoon-hyun/SPARK-28310. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/sql-keywords.md | 3 --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 11 ++--------- .../sql/catalyst/parser/ExpressionParserSuite.scala | 9 --------- .../catalyst/parser/TableIdentifierParserSuite.scala | 5 ----- 4 files changed, 2 insertions(+), 26 deletions(-) diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index b18855366bb2b..9e4a3c54100c6 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -119,7 +119,6 @@ Below is a list of all the keywords in Spark SQL. - @@ -153,7 +152,6 @@ Below is a list of all the keywords in Spark SQL. - @@ -221,7 +219,6 @@ Below is a list of all the keywords in Spark SQL. - diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 6f2bb7a9a7536..08d5ff53bf2e2 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -771,8 +771,8 @@ primaryExpression | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase | CAST '(' expression AS dataType ')' #cast | STRUCT '(' (argument+=namedExpression (',' argument+=namedExpression)*)? ')' #struct - | (FIRST | FIRST_VALUE) '(' expression ((IGNORE | RESPECT) NULLS)? ')' #first - | (LAST | LAST_VALUE) '(' expression ((IGNORE | RESPECT) NULLS)? ')' #last + | FIRST '(' expression (IGNORE NULLS)? ')' #first + | LAST '(' expression (IGNORE NULLS)? ')' #last | POSITION '(' substr=valueExpression IN str=valueExpression ')' #position | constant #constantDefault | ASTERISK #star @@ -1120,7 +1120,6 @@ ansiNonReserved | REPAIR | REPLACE | RESET - | RESPECT | RESTRICT | REVOKE | RLIKE @@ -1280,7 +1279,6 @@ nonReserved | FIELDS | FILEFORMAT | FIRST - | FIRST_VALUE | FOLLOWING | FOR | FOREIGN @@ -1310,7 +1308,6 @@ nonReserved | ITEMS | KEYS | LAST - | LAST_VALUE | LATERAL | LAZY | LEADING @@ -1374,7 +1371,6 @@ nonReserved | REPAIR | REPLACE | RESET - | RESPECT | RESTRICT | REVOKE | RLIKE @@ -1531,7 +1527,6 @@ FIELDS: 'FIELDS'; FILTER: 'FILTER'; FILEFORMAT: 'FILEFORMAT'; FIRST: 'FIRST'; -FIRST_VALUE: 'FIRST_VALUE'; FOLLOWING: 'FOLLOWING'; FOR: 'FOR'; FOREIGN: 'FOREIGN'; @@ -1565,7 +1560,6 @@ ITEMS: 'ITEMS'; JOIN: 'JOIN'; KEYS: 'KEYS'; LAST: 'LAST'; -LAST_VALUE: 'LAST_VALUE'; LATERAL: 'LATERAL'; LAZY: 'LAZY'; LEADING: 'LEADING'; @@ -1632,7 +1626,6 @@ RENAME: 'RENAME'; REPAIR: 'REPAIR'; REPLACE: 'REPLACE'; RESET: 'RESET'; -RESPECT: 'RESPECT'; RESTRICT: 'RESTRICT'; REVOKE: 'REVOKE'; RIGHT: 'RIGHT'; diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 85efc6accf01f..df012ccf09620 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -771,15 +771,6 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("last(a)", Last('a, Literal(false)).toAggregateExpression()) } - test("Support respect nulls keywords for first_value and last_value") { - assertEqual("first_value(a ignore nulls)", First('a, Literal(true)).toAggregateExpression()) - assertEqual("first_value(a respect nulls)", First('a, Literal(false)).toAggregateExpression()) - assertEqual("first_value(a)", First('a, Literal(false)).toAggregateExpression()) - assertEqual("last_value(a ignore nulls)", Last('a, Literal(true)).toAggregateExpression()) - assertEqual("last_value(a respect nulls)", Last('a, Literal(false)).toAggregateExpression()) - assertEqual("last_value(a)", Last('a, Literal(false)).toAggregateExpression()) - } - test("timestamp literals") { DateTimeTestUtils.outstandingTimezones.foreach { timeZone => withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone.getID) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 43244b3c0a57d..053d57846ce8d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -369,7 +369,6 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "fields", "fileformat", "first", - "first_value", "following", "for", "foreign", @@ -403,7 +402,6 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "join", "keys", "last", - "last_value", "lateral", "lazy", "leading", @@ -467,7 +465,6 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "repair", "replace", "reset", - "respect", "restrict", "revoke", "right", @@ -562,7 +559,6 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "except", "false", "fetch", - "first_value", "for", "foreign", "from", @@ -577,7 +573,6 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { "into", "join", "is", - "last_value", "leading", "left", "minute", From 692e3ddb4e517638156f7427ade8b62fb37634a7 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 5 Feb 2020 11:38:33 +0900 Subject: [PATCH 1298/1387] [SPARK-27870][PYTHON][FOLLOW-UP] Rename spark.sql.pandas.udf.buffer.size to spark.sql.execution.pandas.udf.buffer.size ### What changes were proposed in this pull request? This PR renames `spark.sql.pandas.udf.buffer.size` to `spark.sql.execution.pandas.udf.buffer.size` to be more consistent with other pandas configuration prefixes, given: - `spark.sql.execution.pandas.arrowSafeTypeConversion` - `spark.sql.execution.pandas.respectSessionTimeZone` - `spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName` - other configurations like `spark.sql.execution.arrow.*`. ### Why are the changes needed? To make configuration names consistent. ### Does this PR introduce any user-facing change? No because this configuration was not released yet. ### How was this patch tested? Existing tests should cover. Closes #27450 from HyukjinKwon/SPARK-27870-followup. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/tests/test_pandas_udf_scalar.py | 2 +- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/test_pandas_udf_scalar.py b/python/pyspark/sql/tests/test_pandas_udf_scalar.py index 8e7e85ffe6b73..b07de3c0b3cc9 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_scalar.py +++ b/python/pyspark/sql/tests/test_pandas_udf_scalar.py @@ -868,7 +868,7 @@ def test_close(batch_iter): with QuietTest(self.sc): with self.sql_conf({"spark.sql.execution.arrow.maxRecordsPerBatch": 1, - "spark.sql.pandas.udf.buffer.size": 4}): + "spark.sql.execution.pandas.udf.buffer.size": 4}): self.spark.range(10).repartition(1) \ .select(test_close(col("id"))).limit(2).collect() # wait here because python udf worker will take some time to detect diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5ce5692123805..44f6cccc47015 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1600,7 +1600,7 @@ object SQLConf { .createWithDefault(10000) val PANDAS_UDF_BUFFER_SIZE = - buildConf("spark.sql.pandas.udf.buffer.size") + buildConf("spark.sql.execution.pandas.udf.buffer.size") .doc( s"Same as ${BUFFER_SIZE} but only applies to Pandas UDF executions. If it is not set, " + s"the fallback is ${BUFFER_SIZE}. Note that Pandas execution requires more than 4 bytes. " + From e2d984aa1c79eb389cc8d333f656196b17af1c32 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 5 Feb 2020 16:45:54 +0900 Subject: [PATCH 1299/1387] [SPARK-30733][R][HOTFIX] Fix SparkR tests per testthat and R version upgrade, and disable CRAN MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? There are currently the R test failures after upgrading `testthat` to 2.0.0, and R version 3.5.2 as of SPARK-23435. This PR targets to fix the tests and make the tests pass. See the explanations and causes below: ``` test_context.R:49: failure: Check masked functions length(maskedCompletely) not equal to length(namesOfMaskedCompletely). 1/1 mismatches [1] 6 - 4 == 2 test_context.R:53: failure: Check masked functions sort(maskedCompletely, na.last = TRUE) not equal to sort(namesOfMaskedCompletely, na.last = TRUE). 5/6 mismatches x[2]: "endsWith" y[2]: "filter" x[3]: "filter" y[3]: "not" x[4]: "not" y[4]: "sample" x[5]: "sample" y[5]: NA x[6]: "startsWith" y[6]: NA ``` From my cursory look, R base and R's version are mismatched. I fixed accordingly and Jenkins will test it out. ``` test_includePackage.R:31: error: include inside function package or namespace load failed for ���plyr���: package ���plyr��� was installed by an R version with different internals; it needs to be reinstalled for use with this R version Seems it's a package installation issue. Looks like plyr has to be re-installed. ``` From my cursory look, previously installed `plyr` remains and it's not compatible with the new R version. I fixed accordingly and Jenkins will test it out. ``` test_sparkSQL.R:499: warning: SPARK-17811: can create DataFrame containing NA as date and time Your system is mis-configured: ���/etc/localtime��� is not a symlink ``` Seems a env problem. I suppressed the warnings for now. ``` test_sparkSQL.R:499: warning: SPARK-17811: can create DataFrame containing NA as date and time It is strongly recommended to set envionment variable TZ to ���America/Los_Angeles��� (or equivalent) ``` Seems a env problem. I suppressed the warnings for now. ``` test_sparkSQL.R:1814: error: string operators unable to find an inherited method for function ���startsWith��� for signature ���"character"��� 1: expect_true(startsWith("Hello World", "Hello")) at /home/jenkins/workspace/SparkPullRequestBuilder2/R/pkg/tests/fulltests/test_sparkSQL.R:1814 2: quasi_label(enquo(object), label) 3: eval_bare(get_expr(quo), get_env(quo)) 4: startsWith("Hello World", "Hello") 5: (function (classes, fdef, mtable) { methods <- .findInheritedMethods(classes, fdef, mtable) if (length(methods) == 1L) return(methods[[1L]]) else if (length(methods) == 0L) { cnames <- paste0("\"", vapply(classes, as.character, ""), "\"", collapse = ", ") stop(gettextf("unable to find an inherited method for function %s for signature %s", sQuote(fdefgeneric), sQuote(cnames)), domain = NA) } else stop("Internal error in finding inherited methods; didn't return a unique method", domain = NA) })(list("character"), new("nonstandardGenericFunction", .Data = function (x, prefix) { standardGeneric("startsWith") }, generic = structure("startsWith", package = "SparkR"), package = "SparkR", group = list(), valueClass = character(0), signature = c("x", "prefix"), default = NULL, skeleton = (function (x, prefix) stop("invalid call in method dispatch to 'startsWith' (no default method)", domain = NA))(x, prefix)), ) 6: stop(gettextf("unable to find an inherited method for function %s for signature %s", sQuote(fdefgeneric), sQuote(cnames)), domain = NA) ``` From my cursory look, R base and R's version are mismatched. I fixed accordingly and Jenkins will test it out. Also, this PR causes a CRAN check failure as below: ``` * creating vignettes ... ERROR Error: processing vignette 'sparkr-vignettes.Rmd' failed with diagnostics: package ���htmltools��� was installed by an R version with different internals; it needs to be reinstalled for use with this R version ``` This PR disables it for now. ### Why are the changes needed? To unblock other PRs. ### Does this PR introduce any user-facing change? No. Test only and dev only. ### How was this patch tested? No. I am going to use Jenkins to test. Closes #27460 from HyukjinKwon/r-test-failure. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/pkg/tests/fulltests/test_context.R | 3 ++- R/pkg/tests/fulltests/test_includePackage.R | 8 ++++---- R/pkg/tests/fulltests/test_sparkSQL.R | 10 ++++++++-- R/run-tests.sh | 7 ++++--- 4 files changed, 18 insertions(+), 10 deletions(-) diff --git a/R/pkg/tests/fulltests/test_context.R b/R/pkg/tests/fulltests/test_context.R index b9139154bc165..6be04b321e985 100644 --- a/R/pkg/tests/fulltests/test_context.R +++ b/R/pkg/tests/fulltests/test_context.R @@ -25,7 +25,8 @@ test_that("Check masked functions", { namesOfMasked <- c("describe", "cov", "filter", "lag", "na.omit", "predict", "sd", "var", "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", "summary", "transform", "drop", "window", "as.data.frame", "union", "not") - if (as.numeric(R.version$major) >= 3 && as.numeric(R.version$minor) >= 3) { + version <- packageVersion("base") + if (as.numeric(version$major) >= 3 && as.numeric(version$minor) >= 3) { namesOfMasked <- c("endsWith", "startsWith", namesOfMasked) } masked <- conflicts(detail = TRUE)$`package:SparkR` diff --git a/R/pkg/tests/fulltests/test_includePackage.R b/R/pkg/tests/fulltests/test_includePackage.R index 916361ff4c797..1d16b260c4c52 100644 --- a/R/pkg/tests/fulltests/test_includePackage.R +++ b/R/pkg/tests/fulltests/test_includePackage.R @@ -27,8 +27,8 @@ rdd <- parallelize(sc, nums, 2L) test_that("include inside function", { # Only run the test if plyr is installed. - if ("plyr" %in% rownames(installed.packages())) { - suppressPackageStartupMessages(library(plyr)) + if ("plyr" %in% rownames(installed.packages()) && + suppressPackageStartupMessages(suppressWarnings(library(plyr, logical.return = TRUE)))) { generateData <- function(x) { suppressPackageStartupMessages(library(plyr)) attach(airquality) @@ -44,8 +44,8 @@ test_that("include inside function", { test_that("use include package", { # Only run the test if plyr is installed. - if ("plyr" %in% rownames(installed.packages())) { - suppressPackageStartupMessages(library(plyr)) + if ("plyr" %in% rownames(installed.packages()) && + suppressPackageStartupMessages(suppressWarnings(library(plyr, logical.return = TRUE)))) { generateData <- function(x) { attach(airquality) result <- transform(Ozone, logOzone = log(Ozone)) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 3b3768f7e2715..23fadc4373c3f 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -496,7 +496,12 @@ test_that("SPARK-17811: can create DataFrame containing NA as date and time", { expect_true(is.na(DF$date[2])) expect_equal(DF$date[1], as.Date("2016-10-01")) expect_true(is.na(DF$time[2])) - expect_equal(DF$time[1], as.POSIXlt("2016-01-10")) + # Warnings were suppressed due to Jenkins environment issues. + # It shows both warnings as below in Jenkins: + # - Your system is mis-configured: /etc/localtime is not a symlink + # - It is strongly recommended to set environment variable TZ to + # America/Los_Angeles (or equivalent) + suppressWarnings(expect_equal(DF$time[1], as.POSIXlt("2016-01-10"))) }) test_that("create DataFrame with complex types", { @@ -1810,7 +1815,8 @@ test_that("string operators", { expect_true(first(select(df, endsWith(df$name, "el")))[[1]]) expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi") expect_equal(first(select(df, substr(df$name, 4, 6)))[[1]], "hae") - if (as.numeric(R.version$major) >= 3 && as.numeric(R.version$minor) >= 3) { + version <- packageVersion("base") + if (as.numeric(version$major) >= 3 && as.numeric(version$minor) >= 3) { expect_true(startsWith("Hello World", "Hello")) expect_false(endsWith("Hello World", "a")) } diff --git a/R/run-tests.sh b/R/run-tests.sh index 51ca7d600caf0..782b5f5baebaf 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -31,9 +31,10 @@ NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" # Also run the documentation tests for CRAN CRAN_CHECK_LOG_FILE=$FWDIR/cran-check.out rm -f $CRAN_CHECK_LOG_FILE - -NO_TESTS=1 NO_MANUAL=1 $FWDIR/check-cran.sh 2>&1 | tee -a $CRAN_CHECK_LOG_FILE -FAILED=$((PIPESTATUS[0]||$FAILED)) +# TODO(SPARK-30737) reenable this once packages are correctly installed in Jenkins +# NO_TESTS=1 NO_MANUAL=1 $FWDIR/check-cran.sh 2>&1 | tee -a $CRAN_CHECK_LOG_FILE +# FAILED=$((PIPESTATUS[0]||$FAILED)) +touch $CRAN_CHECK_LOG_FILE NUM_CRAN_WARNING="$(grep -c WARNING$ $CRAN_CHECK_LOG_FILE)" NUM_CRAN_ERROR="$(grep -c ERROR$ $CRAN_CHECK_LOG_FILE)" From 30e418a6fe971b4a84c37ca0ae20f1a664b117d3 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 5 Feb 2020 16:15:44 +0800 Subject: [PATCH 1300/1387] [SPARK-30594][CORE] Do not post SparkListenerBlockUpdated when updateBlockInfo returns false ### What changes were proposed in this pull request? If `updateBlockInfo` returns false, which means the `BlockManager` will re-register and report all blocks later. So, we may report two times for the same block, which causes `AppStatusListener` to count used memory for two times, too. As a result, the used memory can exceed the total memory. So, this PR changes it to not post `SparkListenerBlockUpdated` when `updateBlockInfo` returns false. And, always clean up used memory whenever `AppStatusListener` receives `SparkListenerBlockManagerAdded`. ### Why are the changes needed? This PR tries to fix negative memory usage in UI (https://user-images.githubusercontent.com/3488126/72131225-95e37e00-33b6-11ea-8708-6e5ed328d1ca.png, see #27144 ). Though, I'm not very sure this is the root cause for #27144 since known information is limited here. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added new tests by xuanyuanking Closes #27306 from Ngone51/fix-possible-negative-memory. Lead-authored-by: yi.wu Co-authored-by: Yuanjian Li Co-authored-by: wuyi Signed-off-by: Wenchen Fan --- .../spark/status/AppStatusListener.scala | 9 +++++-- .../org/apache/spark/status/LiveEntity.scala | 2 +- .../storage/BlockManagerMasterEndpoint.scala | 9 +++++-- .../spark/status/AppStatusListenerSuite.scala | 24 +++++++++++++++++++ .../spark/storage/BlockManagerSuite.scala | 17 +++++++++++-- 5 files changed, 54 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index a5850fc2ac4b9..c3f22f32993a8 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -71,7 +71,7 @@ private[spark] class AppStatusListener( // causing too many writes to the underlying store, and other expensive operations). private val liveStages = new ConcurrentHashMap[(Int, Int), LiveStage]() private val liveJobs = new HashMap[Int, LiveJob]() - private val liveExecutors = new HashMap[String, LiveExecutor]() + private[spark] val liveExecutors = new HashMap[String, LiveExecutor]() private val deadExecutors = new HashMap[String, LiveExecutor]() private val liveTasks = new HashMap[Long, LiveTask]() private val liveRDDs = new HashMap[Int, LiveRDD]() @@ -772,6 +772,11 @@ private[spark] class AppStatusListener( event.maxOnHeapMem.foreach { _ => exec.totalOnHeap = event.maxOnHeapMem.get exec.totalOffHeap = event.maxOffHeapMem.get + // SPARK-30594: whenever(first time or re-register) a BlockManager added, all blocks + // from this BlockManager will be reported to driver later. So, we should clean up + // used memory to avoid overlapped count. + exec.usedOnHeap = 0 + exec.usedOffHeap = 0 } exec.isActive = true exec.maxMemory = event.maxMem @@ -1042,7 +1047,7 @@ private[spark] class AppStatusListener( } } - private def updateExecutorMemoryDiskInfo( + private[spark] def updateExecutorMemoryDiskInfo( exec: LiveExecutor, storageLevel: StorageLevel, memoryDelta: Long, diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index e3046dce34e67..2714f30de14f0 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -245,7 +245,7 @@ private class LiveTask( } -private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveEntity { +private[spark] class LiveExecutor(val executorId: String, _addTime: Long) extends LiveEntity { var hostPort: String = null var host: String = null diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 41ef1909cd4c2..d7f7eedc7f33b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -98,8 +98,13 @@ class BlockManagerMasterEndpoint( case _updateBlockInfo @ UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => - context.reply(updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size)) - listenerBus.post(SparkListenerBlockUpdated(BlockUpdatedInfo(_updateBlockInfo))) + val isSuccess = updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) + context.reply(isSuccess) + // SPARK-30594: we should not post `SparkListenerBlockUpdated` when updateBlockInfo + // returns false since the block info would be updated again later. + if (isSuccess) { + listenerBus.post(SparkListenerBlockUpdated(BlockUpdatedInfo(_updateBlockInfo))) + } case GetLocations(blockId) => context.reply(getLocations(blockId)) diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index e7eed7bf4c879..255f91866ef58 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1657,6 +1657,30 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } } + test("clean up used memory when BlockManager added") { + val listener = new AppStatusListener(store, conf, true) + // Add block manager at the first time + val driver = BlockManagerId(SparkContext.DRIVER_IDENTIFIER, "localhost", 42) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded( + time, driver, 42L, Some(43L), Some(44L))) + // Update the memory metrics + listener.updateExecutorMemoryDiskInfo( + listener.liveExecutors(SparkContext.DRIVER_IDENTIFIER), + StorageLevel.MEMORY_AND_DISK, + 10L, + 10L + ) + // Re-add the same block manager again + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded( + time, driver, 42L, Some(43L), Some(44L))) + + check[ExecutorSummaryWrapper](SparkContext.DRIVER_IDENTIFIER) { d => + val memoryMetrics = d.info.memoryMetrics.get + assert(memoryMetrics.usedOffHeapStorageMemory == 0) + assert(memoryMetrics.usedOnHeapStorageMemory == 0) + } + } + private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptNumber) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 89f00b5a9d902..8d06768a2b284 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -50,7 +50,7 @@ import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, Transpo import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExecutorDiskUtils, ExternalBlockStoreClient} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} import org.apache.spark.rpc.RpcEnv -import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerBlockUpdated} import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager} import org.apache.spark.shuffle.sort.SortShuffleManager @@ -71,6 +71,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val allStores = ArrayBuffer[BlockManager]() var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null + var liveListenerBus: LiveListenerBus = null val securityMgr = new SecurityManager(new SparkConf(false)) val bcastManager = new BroadcastManager(true, new SparkConf(false), securityMgr) val mapOutputTracker = new MapOutputTrackerMaster(new SparkConf(false), bcastManager, true) @@ -145,9 +146,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(sc.conf).thenReturn(conf) val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo]() + liveListenerBus = spy(new LiveListenerBus(conf)) master = spy(new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf), None, blockManagerInfo)), + liveListenerBus, None, blockManagerInfo)), rpcEnv.setupEndpoint("blockmanagerHeartbeat", new BlockManagerMasterHeartbeatEndpoint(rpcEnv, true, blockManagerInfo)), conf, true)) @@ -164,6 +166,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE rpcEnv.awaitTermination() rpcEnv = null master = null + liveListenerBus = null } finally { super.afterEach() } @@ -1693,6 +1696,16 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(locs(blockIds(0)) == expectedLocs) } + test("SPARK-30594: Do not post SparkListenerBlockUpdated when updateBlockInfo returns false") { + // update block info for non-existent block manager + val updateInfo = UpdateBlockInfo(BlockManagerId("1", "host1", 100), + BlockId("test_1"), StorageLevel.MEMORY_ONLY, 1, 1) + val result = master.driverEndpoint.askSync[Boolean](updateInfo) + + assert(!result) + verify(liveListenerBus, never()).post(SparkListenerBlockUpdated(BlockUpdatedInfo(updateInfo))) + } + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { var numCalls = 0 var tempFileManager: DownloadFileManager = null From 5983ad9cc4481e224a7e094de830ef2e816c1fe6 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 5 Feb 2020 17:16:38 +0800 Subject: [PATCH 1301/1387] [SPARK-30506][SQL][DOC] Document for generic file source options/configs ### What changes were proposed in this pull request? Add a new document page named *Generic File Source Options* for *Data Sources* menu and added following sub items: * spark.sql.files.ignoreCorruptFiles * spark.sql.files.ignoreMissingFiles * pathGlobFilter * recursiveFileLookup And here're snapshots of the generated document: doc-1 doc-2 doc-3 doc-4 ### Why are the changes needed? Better guidance for end-user. ### Does this PR introduce any user-facing change? No, added in Spark 3.0. ### How was this patch tested? Pass Jenkins. Closes #27302 from Ngone51/doc-generic-file-source-option. Lead-authored-by: yi.wu Co-authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- docs/_data/menu-sql.yaml | 2 + docs/sql-data-sources-avro.md | 2 +- docs/sql-data-sources-generic-options.md | 121 ++++++++++++++++++ docs/sql-data-sources-load-save-functions.md | 21 --- docs/sql-data-sources.md | 5 + .../sql/JavaSQLDataSourceExample.java | 48 ++++++- examples/src/main/python/sql/datasource.py | 48 ++++++- examples/src/main/r/RSparkSQLExample.R | 24 +++- .../main/resources/dir1/dir2/file2.parquet | Bin 0 -> 520 bytes .../src/main/resources/dir1/file1.parquet | Bin 0 -> 520 bytes examples/src/main/resources/dir1/file3.json | 1 + .../do_not_read_this.txt | 1 - .../users.orc | Bin 448 -> 0 bytes .../favorite_color=red/users.orc | Bin 402 -> 0 bytes .../examples/sql/SQLDataSourceExample.scala | 48 ++++++- 15 files changed, 282 insertions(+), 39 deletions(-) create mode 100644 docs/sql-data-sources-generic-options.md create mode 100644 examples/src/main/resources/dir1/dir2/file2.parquet create mode 100644 examples/src/main/resources/dir1/file1.parquet create mode 100644 examples/src/main/resources/dir1/file3.json delete mode 100644 examples/src/main/resources/partitioned_users.orc/do_not_read_this.txt delete mode 100644 examples/src/main/resources/partitioned_users.orc/favorite_color=__HIVE_DEFAULT_PARTITION__/users.orc delete mode 100644 examples/src/main/resources/partitioned_users.orc/favorite_color=red/users.orc diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 3e4db7107ec34..241ec399d7bd5 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -24,6 +24,8 @@ subitems: - text: "Generic Load/Save Functions" url: sql-data-sources-load-save-functions.html + - text: "Generic File Source Options" + url: sql-data-sources-generic-options.html - text: Parquet Files url: sql-data-sources-parquet.html - text: ORC Files diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md index b0076878e02da..8e6a4079cd5de 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/sql-data-sources-avro.md @@ -230,7 +230,7 @@ Data source options of Avro can be set via: - + diff --git a/docs/sql-data-sources-generic-options.md b/docs/sql-data-sources-generic-options.md new file mode 100644 index 0000000000000..0cfe2ed1aa891 --- /dev/null +++ b/docs/sql-data-sources-generic-options.md @@ -0,0 +1,121 @@ +--- +layout: global +title: Generic File Source Options +displayTitle: Generic File Source Options +license: | + 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. +--- + +* Table of contents +{:toc} + +These generic options/configurations are effective only when using file-based sources: parquet, orc, avro, json, csv, text. + +Please note that the hierarchy of directories used in examples below are: + +{% highlight text %} + +dir1/ + ├── dir2/ + │ └── file2.parquet (schema: , content: "file2.parquet") + └── file1.parquet (schema: , content: "file1.parquet") + └── file3.json (schema: , content: "{'file':'corrupt.json'}") + +{% endhighlight %} + +### Ignore Corrupt Files + +Spark allows you to use `spark.sql.files.ignoreCorruptFiles` to ignore corrupt files while reading data +from files. When set to true, the Spark jobs will continue to run when encountering corrupted files and +the contents that have been read will still be returned. + +To ignore corrupt files while reading data files, you can use: + +
    +
    +{% include_example ignore_corrupt_files scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example ignore_corrupt_files java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example ignore_corrupt_files python/sql/datasource.py %} +
    + +
    +{% include_example ignore_corrupt_files r/RSparkSQLExample.R %} +
    +
    + +### Ignore Missing Files + +Spark allows you to use `spark.sql.files.ignoreMissingFiles` to ignore missing files while reading data +from files. Here, missing file really means the deleted file under directory after you construct the +`DataFrame`. When set to true, the Spark jobs will continue to run when encountering missing files and +the contents that have been read will still be returned. + +### Path Global Filter + +`pathGlobFilter` is used to only include files with file names matching the pattern. +The syntax follows org.apache.hadoop.fs.GlobFilter. +It does not change the behavior of partition discovery. + +To load files with paths matching a given glob pattern while keeping the behavior of partition discovery, +you can use: + +
    +
    +{% include_example load_with_path_glob_filter scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example load_with_path_glob_filter java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example load_with_path_glob_filter python/sql/datasource.py %} +
    + +
    +{% include_example load_with_path_glob_filter r/RSparkSQLExample.R %} +
    +
    + +### Recursive File Lookup +`recursiveFileLookup` is used to recursively load files and it disables partition inferring. Its default value is `false`. +If data source explicitly specifies the `partitionSpec` when `recursiveFileLookup` is true, exception will be thrown. + +To load all files recursively, you can use: + +
    +
    +{% include_example recursive_file_lookup scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example recursive_file_lookup java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example recursive_file_lookup python/sql/datasource.py %} +
    + +
    +{% include_example recursive_file_lookup r/RSparkSQLExample.R %} +
    +
    \ No newline at end of file diff --git a/docs/sql-data-sources-load-save-functions.md b/docs/sql-data-sources-load-save-functions.md index 07482137a28a3..a7efb9347ac64 100644 --- a/docs/sql-data-sources-load-save-functions.md +++ b/docs/sql-data-sources-load-save-functions.md @@ -102,27 +102,6 @@ To load a CSV file you can use: -To load files with paths matching a given glob pattern while keeping the behavior of partition discovery, -you can use: - -
    -
    -{% include_example load_with_path_glob_filter scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example load_with_path_glob_filter java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example load_with_path_glob_filter python/sql/datasource.py %} -
    - -
    -{% include_example load_with_path_glob_filter r/RSparkSQLExample.R %} -
    -
    - The extra options are also used during write operation. For example, you can control bloom filters and dictionary encodings for ORC data sources. The following ORC example will create bloom filter and use dictionary encoding only for `favorite_color`. diff --git a/docs/sql-data-sources.md b/docs/sql-data-sources.md index 079c54060d15d..9396846041709 100644 --- a/docs/sql-data-sources.md +++ b/docs/sql-data-sources.md @@ -33,6 +33,11 @@ goes into specific options that are available for the built-in data sources. * [Save Modes](sql-data-sources-load-save-functions.html#save-modes) * [Saving to Persistent Tables](sql-data-sources-load-save-functions.html#saving-to-persistent-tables) * [Bucketing, Sorting and Partitioning](sql-data-sources-load-save-functions.html#bucketing-sorting-and-partitioning) +* [Generic File Source Options](sql-data-sources-generic-options.html) + * [Ignore Corrupt Files](sql-data-sources-generic-options.html#ignore-corrupt-iles) + * [Ignore Missing Files](sql-data-sources-generic-options.html#ignore-missing-iles) + * [Path Global Filter](sql-data-sources-generic-options.html#path-global-filter) + * [Recursive File Lookup](sql-data-sources-generic-options.html#recursive-file-lookup) * [Parquet Files](sql-data-sources-parquet.html) * [Loading Data Programmatically](sql-data-sources-parquet.html#loading-data-programmatically) * [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index b2ce0bc08642a..2295225387a33 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -98,6 +98,7 @@ public static void main(String[] args) { .getOrCreate(); runBasicDataSourceExample(spark); + runGenericFileSourceOptionsExample(spark); runBasicParquetExample(spark); runParquetSchemaMergingExample(spark); runJsonDatasetExample(spark); @@ -106,6 +107,48 @@ public static void main(String[] args) { spark.stop(); } + private static void runGenericFileSourceOptionsExample(SparkSession spark) { + // $example on:ignore_corrupt_files$ + // enable ignore corrupt files + spark.sql("set spark.sql.files.ignoreCorruptFiles=true"); + // dir1/file3.json is corrupt from parquet's view + Dataset testCorruptDF = spark.read().parquet( + "examples/src/main/resources/dir1/", + "examples/src/main/resources/dir1/dir2/"); + testCorruptDF.show(); + // +-------------+ + // | file| + // +-------------+ + // |file1.parquet| + // |file2.parquet| + // +-------------+ + // $example off:ignore_corrupt_files$ + // $example on:recursive_file_lookup$ + Dataset recursiveLoadedDF = spark.read().format("parquet") + .option("recursiveFileLookup", "true") + .load("examples/src/main/resources/dir1"); + recursiveLoadedDF.show(); + // +-------------+ + // | file| + // +-------------+ + // |file1.parquet| + // |file2.parquet| + // +-------------+ + // $example off:recursive_file_lookup$ + spark.sql("set spark.sql.files.ignoreCorruptFiles=false"); + // $example on:load_with_path_glob_filter$ + Dataset testGlobFilterDF = spark.read().format("parquet") + .option("pathGlobFilter", "*.parquet") // json file should be filtered out + .load("examples/src/main/resources/dir1"); + testGlobFilterDF.show(); + // +-------------+ + // | file| + // +-------------+ + // |file1.parquet| + // +-------------+ + // $example off:load_with_path_glob_filter$ + } + private static void runBasicDataSourceExample(SparkSession spark) { // $example on:generic_load_save_functions$ Dataset usersDF = spark.read().load("examples/src/main/resources/users.parquet"); @@ -123,11 +166,6 @@ private static void runBasicDataSourceExample(SparkSession spark) { .option("header", "true") .load("examples/src/main/resources/people.csv"); // $example off:manual_load_options_csv$ - // $example on:load_with_path_glob_filter$ - Dataset partitionedUsersDF = spark.read().format("orc") - .option("pathGlobFilter", "*.orc") - .load("examples/src/main/resources/partitioned_users.orc"); - // $example off:load_with_path_glob_filter$ // $example on:manual_save_options_orc$ usersDF.write().format("orc") .option("orc.bloom.filter.columns", "favorite_color") diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index 0d78097ea975e..265f135e1e5f2 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -28,6 +28,48 @@ # $example off:schema_merging$ +def generic_file_source_options_example(spark): + # $example on:ignore_corrupt_files$ + # enable ignore corrupt files + spark.sql("set spark.sql.files.ignoreCorruptFiles=true") + # dir1/file3.json is corrupt from parquet's view + test_corrupt_df = spark.read.parquet("examples/src/main/resources/dir1/", + "examples/src/main/resources/dir1/dir2/") + test_corrupt_df.show() + # +-------------+ + # | file| + # +-------------+ + # |file1.parquet| + # |file2.parquet| + # +-------------+ + # $example off:ignore_corrupt_files$ + + # $example on:recursive_file_lookup$ + recursive_loaded_df = spark.read.format("parquet")\ + .option("recursiveFileLookup", "true")\ + .load("examples/src/main/resources/dir1") + recursive_loaded_df.show() + # +-------------+ + # | file| + # +-------------+ + # |file1.parquet| + # |file2.parquet| + # +-------------+ + # $example off:recursive_file_lookup$ + spark.sql("set spark.sql.files.ignoreCorruptFiles=false") + + # $example on:load_with_path_glob_filter$ + df = spark.read.load("examples/src/main/resources/dir1", + format="parquet", pathGlobFilter="*.parquet") + df.show() + # +-------------+ + # | file| + # +-------------+ + # |file1.parquet| + # +-------------+ + # $example off:load_with_path_glob_filter$ + + def basic_datasource_example(spark): # $example on:generic_load_save_functions$ df = spark.read.load("examples/src/main/resources/users.parquet") @@ -57,11 +99,6 @@ def basic_datasource_example(spark): format="csv", sep=":", inferSchema="true", header="true") # $example off:manual_load_options_csv$ - # $example on:load_with_path_glob_filter$ - df = spark.read.load("examples/src/main/resources/partitioned_users.orc", - format="orc", pathGlobFilter="*.orc") - # $example off:load_with_path_glob_filter$ - # $example on:manual_save_options_orc$ df = spark.read.orc("examples/src/main/resources/users.orc") (df.write.format("orc") @@ -233,6 +270,7 @@ def jdbc_dataset_example(spark): .getOrCreate() basic_datasource_example(spark) + generic_file_source_options_example(spark) parquet_example(spark) parquet_schema_merging_example(spark) json_dataset_example(spark) diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R index fa083d5542fae..8685cfb5c05f2 100644 --- a/examples/src/main/r/RSparkSQLExample.R +++ b/examples/src/main/r/RSparkSQLExample.R @@ -99,6 +99,26 @@ createOrReplaceTempView(df, "table") df <- sql("SELECT * FROM table") # $example off:run_sql$ +# Ignore corrupt files +# $example on:ignore_corrupt_files$ +# enable ignore corrupt files +sql("set spark.sql.files.ignoreCorruptFiles=true") +# dir1/file3.json is corrupt from parquet's view +testCorruptDF <- read.parquet(c("examples/src/main/resources/dir1/", "examples/src/main/resources/dir1/dir2/")) +head(testCorruptDF) +# file +# 1 file1.parquet +# 2 file2.parquet +# $example off:ignore_corrupt_files$ + +# $example on:recursive_file_lookup$ +recursiveLoadedDF <- read.df("examples/src/main/resources/dir1", "parquet", recursiveFileLookup = "true") +head(recursiveLoadedDF) +# file +# 1 file1.parquet +# 2 file2.parquet +# $example off:recursive_file_lookup$ +sql("set spark.sql.files.ignoreCorruptFiles=false") # $example on:generic_load_save_functions$ df <- read.df("examples/src/main/resources/users.parquet") @@ -119,7 +139,9 @@ namesAndAges <- select(df, "name", "age") # $example off:manual_load_options_csv$ # $example on:load_with_path_glob_filter$ -df <- read.df("examples/src/main/resources/partitioned_users.orc", "orc", pathGlobFilter = "*.orc") +df <- read.df("examples/src/main/resources/dir1", "parquet", pathGlobFilter = "*.parquet") +# file +# 1 file1.parquet # $example off:load_with_path_glob_filter$ # $example on:manual_save_options_orc$ diff --git a/examples/src/main/resources/dir1/dir2/file2.parquet b/examples/src/main/resources/dir1/dir2/file2.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d1895bf29b75ce0d6c70009562975645b7699fb9 GIT binary patch literal 520 zcmah{O-sW-5S>OXIi$xeyO2XzXlWrXNkh{nc=0Blibq9c(`;%mN!xrVQu;6aVg4ki z+9Fm^*j@JVX5PG+-Pz@hOMnO>Y@*?%O>~oXk~C8zv6AJwQS}k*!r)IH05seutqz_) zgowuME2Bc$r-y3(sB%d(AVyE4r@OcwKv!cXGyA$p3^s0q&b}CeMEAXgtFK=i**Sv$ zx??7G30N4bp`-@PrgT{@gj`AVBtGqlXH{0|vY=<4aD)SN_$#7XXNLiaa`_^1Rm)h` zlHP83{kl>-=0P+GCQhWH R*c~#{#2LJ`0A~0HJ^)`pl6wFE literal 0 HcmV?d00001 diff --git a/examples/src/main/resources/dir1/file1.parquet b/examples/src/main/resources/dir1/file1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..ad360b16fd898ea301cbbc00de07a2fd40bbbb44 GIT binary patch literal 520 zcmah{O-sW-5S>OXIkd+uyRe6_&|)DjiJ@r|ym%8&#iJs!X*M;Oq-{PFDg77zFn^L$ zZ4oOd>@NFwGjHC^?)37;B|s#RHZkzhCMKudB+axdm#SPfx;}tCIQ^3c00Z|?tBdCk zA!0E5s#Brt(?d0T)VU%CkWSyA$J@9*K+j;D@qx)nhmG1`vM+`p(R**<8kiRcb`Idb z?ij}|0SjX_lr(_ZlrGA-R7-`PYgRtsAssm|g`gm-IEsC-Lp`$ghAFfHbYizdrdgj9&B+7^3J5w16z z`wh_R-J6`{rM2^Dxzcr{Yn?_-n#Goy1aUG+g1|eEe4Y5d=f#fi_<<$^{ID@wqz!d+$2T@;=asU7T literal 0 HcmV?d00001 diff --git a/examples/src/main/resources/dir1/file3.json b/examples/src/main/resources/dir1/file3.json new file mode 100644 index 0000000000000..0490f92d7f317 --- /dev/null +++ b/examples/src/main/resources/dir1/file3.json @@ -0,0 +1 @@ +{"file":"corrupt.json"} diff --git a/examples/src/main/resources/partitioned_users.orc/do_not_read_this.txt b/examples/src/main/resources/partitioned_users.orc/do_not_read_this.txt deleted file mode 100644 index 9c19f2a0449eb..0000000000000 --- a/examples/src/main/resources/partitioned_users.orc/do_not_read_this.txt +++ /dev/null @@ -1 +0,0 @@ -do not read this diff --git a/examples/src/main/resources/partitioned_users.orc/favorite_color=__HIVE_DEFAULT_PARTITION__/users.orc b/examples/src/main/resources/partitioned_users.orc/favorite_color=__HIVE_DEFAULT_PARTITION__/users.orc deleted file mode 100644 index 890395a9281abb71a8444a3c9041155fae6c0f9f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 448 zcmZ9I&q~8U5XNVBo9(zw?Ydr;SjFZbMIi*dg&ryutq3;s6fbI&s73pSL<`=EPtdpU zMZ_oa0on0VPFPM^G~@eRf!WgU9H=ldim;DS}SrJF{zz-(+ldGL?0J-_Gzeh^9ZY$ja_ PcC+R4_ix5}{f_k!}tAl=uBlmXZBh|u*|g)<(VO4sHM8gz=Q*dq|0Nzq8h zaaAOV-=FcZP}g^L`!CzXuYyvTD7|GlK_F?~v7@;p&h{?)7#xsq1&U6i$Yy!R+-2)| zIep5ni`DEVFPDL!X5f*wD7lr#huY1{`!HK%w-0%^Tx8{AxMtsUURZAsMqw0TTvNsW iQpG=1@s?`%dUXl(hDNx}#smWV{vcA%pHe2{k^2I!Wi|=` diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index d4c05e5ad9944..2c7abfcd335d1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -32,6 +32,7 @@ object SQLDataSourceExample { .getOrCreate() runBasicDataSourceExample(spark) + runGenericFileSourceOptionsExample(spark) runBasicParquetExample(spark) runParquetSchemaMergingExample(spark) runJsonDatasetExample(spark) @@ -40,6 +41,48 @@ object SQLDataSourceExample { spark.stop() } + private def runGenericFileSourceOptionsExample(spark: SparkSession): Unit = { + // $example on:ignore_corrupt_files$ + // enable ignore corrupt files + spark.sql("set spark.sql.files.ignoreCorruptFiles=true") + // dir1/file3.json is corrupt from parquet's view + val testCorruptDF = spark.read.parquet( + "examples/src/main/resources/dir1/", + "examples/src/main/resources/dir1/dir2/") + testCorruptDF.show() + // +-------------+ + // | file| + // +-------------+ + // |file1.parquet| + // |file2.parquet| + // +-------------+ + // $example off:ignore_corrupt_files$ + // $example on:recursive_file_lookup$ + val recursiveLoadedDF = spark.read.format("parquet") + .option("recursiveFileLookup", "true") + .load("examples/src/main/resources/dir1") + recursiveLoadedDF.show() + // +-------------+ + // | file| + // +-------------+ + // |file1.parquet| + // |file2.parquet| + // +-------------+ + // $example off:recursive_file_lookup$ + spark.sql("set spark.sql.files.ignoreCorruptFiles=false") + // $example on:load_with_path_glob_filter$ + val testGlobFilterDF = spark.read.format("parquet") + .option("pathGlobFilter", "*.parquet") // json file should be filtered out + .load("examples/src/main/resources/dir1") + testGlobFilterDF.show() + // +-------------+ + // | file| + // +-------------+ + // |file1.parquet| + // +-------------+ + // $example off:load_with_path_glob_filter$ + } + private def runBasicDataSourceExample(spark: SparkSession): Unit = { // $example on:generic_load_save_functions$ val usersDF = spark.read.load("examples/src/main/resources/users.parquet") @@ -56,11 +99,6 @@ object SQLDataSourceExample { .option("header", "true") .load("examples/src/main/resources/people.csv") // $example off:manual_load_options_csv$ - // $example on:load_with_path_glob_filter$ - val partitionedUsersDF = spark.read.format("orc") - .option("pathGlobFilter", "*.orc") - .load("examples/src/main/resources/partitioned_users.orc") - // $example off:load_with_path_glob_filter$ // $example on:manual_save_options_orc$ usersDF.write.format("orc") .option("orc.bloom.filter.columns", "favorite_color") From 86fdb818bf5dfde7744bf2b358876af361ec9a68 Mon Sep 17 00:00:00 2001 From: Onur Satici Date: Wed, 5 Feb 2020 01:17:30 -0800 Subject: [PATCH 1302/1387] [SPARK-30715][K8S] Bump fabric8 to 4.7.1 ### What changes were proposed in this pull request? Bump fabric8 kubernetes-client to 4.7.1 ### Why are the changes needed? New fabric8 version brings support for Kubernetes 1.17 clusters. Full release notes: - https://github.com/fabric8io/kubernetes-client/releases/tag/v4.7.0 - https://github.com/fabric8io/kubernetes-client/releases/tag/v4.7.1 ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing unit and integration tests cover creation of K8S objects. Adjusted them to work with the new fabric8 version Closes #27443 from onursatici/os/bump-fabric8. Authored-by: Onur Satici Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-1.2 | 6 ++--- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 6 ++--- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 6 ++--- resource-managers/kubernetes/core/pom.xml | 2 +- .../spark/deploy/k8s/KubernetesUtils.scala | 6 ++--- .../k8s/features/BasicDriverFeatureStep.scala | 10 +++------ .../features/BasicExecutorFeatureStep.scala | 12 +++------- .../features/MountVolumesFeatureStep.scala | 2 +- .../spark/deploy/k8s/PodBuilderSuite.scala | 5 ++--- .../BasicDriverFeatureStepSuite.scala | 22 ++++++++++--------- .../BasicExecutorFeatureStepSuite.scala | 15 ++++++++----- .../MountVolumesFeatureStepSuite.scala | 5 +++-- .../k8s/integrationtest/DepsTestsSuite.scala | 8 ++----- .../k8s/integrationtest/PVTestsSuite.scala | 5 ++--- 14 files changed, 49 insertions(+), 61 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 62d5772ffc94a..534ac39e0c46e 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -137,9 +137,9 @@ jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar jul-to-slf4j/1.7.16//jul-to-slf4j-1.7.16.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.6.4//kubernetes-client-4.6.4.jar -kubernetes-model-common/4.6.4//kubernetes-model-common-4.6.4.jar -kubernetes-model/4.6.4//kubernetes-model-4.6.4.jar +kubernetes-client/4.7.1//kubernetes-client-4.7.1.jar +kubernetes-model-common/4.7.1//kubernetes-model-common-4.7.1.jar +kubernetes-model/4.7.1//kubernetes-model-4.7.1.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 1b57250c1fb54..42bdf112efccb 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -153,9 +153,9 @@ jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar jul-to-slf4j/1.7.16//jul-to-slf4j-1.7.16.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.6.4//kubernetes-client-4.6.4.jar -kubernetes-model-common/4.6.4//kubernetes-model-common-4.6.4.jar -kubernetes-model/4.6.4//kubernetes-model-4.6.4.jar +kubernetes-client/4.7.1//kubernetes-client-4.7.1.jar +kubernetes-model-common/4.7.1//kubernetes-model-common-4.7.1.jar +kubernetes-model/4.7.1//kubernetes-model-4.7.1.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index ffd2364a51317..6006fa4b43f42 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -165,9 +165,9 @@ kerby-pkix/1.0.1//kerby-pkix-1.0.1.jar kerby-util/1.0.1//kerby-util-1.0.1.jar kerby-xdr/1.0.1//kerby-xdr-1.0.1.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client/4.6.4//kubernetes-client-4.6.4.jar -kubernetes-model-common/4.6.4//kubernetes-model-common-4.6.4.jar -kubernetes-model/4.6.4//kubernetes-model-4.6.4.jar +kubernetes-client/4.7.1//kubernetes-client-4.7.1.jar +kubernetes-model-common/4.7.1//kubernetes-model-common-4.7.1.jar +kubernetes-model/4.7.1//kubernetes-model-4.7.1.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar libthrift/0.12.0//libthrift-0.12.0.jar diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 8c46738b259bc..f8c6b38225559 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -29,7 +29,7 @@ Spark Project Kubernetes kubernetes - 4.6.4 + 4.7.1 diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index e234b1780a7d0..c49f4a15de974 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -23,7 +23,7 @@ import java.util.UUID import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, PodBuilder, Quantity, QuantityBuilder} +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, PodBuilder, Quantity} import io.fabric8.kubernetes.client.KubernetesClient import org.apache.commons.codec.binary.Hex import org.apache.hadoop.fs.{FileSystem, Path} @@ -234,9 +234,7 @@ private[spark] object KubernetesUtils extends Logging { throw new SparkException(s"Resource: ${request.id.resourceName} was requested, " + "but vendor was not specified.") } - val quantity = new QuantityBuilder(false) - .withAmount(request.amount.toString) - .build() + val quantity = new Quantity(request.amount.toString) (KubernetesConf.buildKubernetesResourceName(vendorDomain, request.id.resourceName), quantity) }.toMap } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index e2d70d7692ef3..eec275e6e6f23 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -80,14 +80,10 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) .build() } - val driverCpuQuantity = new QuantityBuilder(false) - .withAmount(driverCoresRequest) - .build() - val driverMemoryQuantity = new QuantityBuilder(false) - .withAmount(s"${driverMemoryWithOverheadMiB}Mi") - .build() + val driverCpuQuantity = new Quantity(driverCoresRequest) + val driverMemoryQuantity = new Quantity(s"${driverMemoryWithOverheadMiB}Mi") val maybeCpuLimitQuantity = driverLimitCores.map { limitCores => - ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) + ("cpu", new Quantity(limitCores)) } val driverResourceQuantities = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index d88bd5858bc94..6a26df2997fd2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -88,12 +88,8 @@ private[spark] class BasicExecutorFeatureStep( // Replace dangerous characters in the remaining string with a safe alternative. .replaceAll("[^\\w-]+", "_") - val executorMemoryQuantity = new QuantityBuilder(false) - .withAmount(s"${executorMemoryTotal}Mi") - .build() - val executorCpuQuantity = new QuantityBuilder(false) - .withAmount(executorCoresRequest) - .build() + val executorMemoryQuantity = new Quantity(s"${executorMemoryTotal}Mi") + val executorCpuQuantity = new Quantity(executorCoresRequest) val executorResourceQuantities = KubernetesUtils.buildResourcesQuantities(SPARK_EXECUTOR_PREFIX, @@ -183,9 +179,7 @@ private[spark] class BasicExecutorFeatureStep( .addToArgs("executor") .build() val containerWithLimitCores = executorLimitCores.map { limitCores => - val executorCpuLimitQuantity = new QuantityBuilder(false) - .withAmount(limitCores) - .build() + val executorCpuLimitQuantity = new Quantity(limitCores) new ContainerBuilder(executorContainer) .editResources() .addToLimits("cpu", executorCpuLimitQuantity) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala index 8548e7057cdf0..4599df99b3c61 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala @@ -65,7 +65,7 @@ private[spark] class MountVolumesFeatureStep(conf: KubernetesConf) new VolumeBuilder() .withEmptyDir( new EmptyDirVolumeSource(medium.getOrElse(""), - new Quantity(sizeLimit.orNull))) + sizeLimit.map(new Quantity(_)).orNull)) } val volume = volumeBuilder.withName(spec.volumeName).build() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala index 707c823d69cf0..26bd317de8ec6 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala @@ -101,8 +101,7 @@ abstract class PodBuilderSuite extends SparkFunSuite { assert(container.getArgs.contains("arg")) assert(container.getCommand.equals(List("command").asJava)) assert(container.getEnv.asScala.exists(_.getName == "env-key")) - assert(container.getResources.getLimits.get("gpu") === - new QuantityBuilder().withAmount("1").build()) + assert(container.getResources.getLimits.get("gpu") === new Quantity("1")) assert(container.getSecurityContext.getRunAsNonRoot) assert(container.getStdin) assert(container.getTerminationMessagePath === "termination-message-path") @@ -156,7 +155,7 @@ abstract class PodBuilderSuite extends SparkFunSuite { .withImagePullPolicy("Always") .withName("executor-container") .withNewResources() - .withLimits(Map("gpu" -> new QuantityBuilder().withAmount("1").build()).asJava) + .withLimits(Map("gpu" -> new Quantity("1")).asJava) .endResources() .withNewSecurityContext() .withRunAsNonRoot(true) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index ce66afd9448a9..ef69600ea88ab 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.features import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{ContainerPort, ContainerPortBuilder, LocalObjectReferenceBuilder} +import io.fabric8.kubernetes.api.model.{ContainerPort, ContainerPortBuilder, LocalObjectReferenceBuilder, Quantity} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} @@ -105,13 +105,13 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val resourceRequirements = configuredPod.container.getResources val requests = resourceRequirements.getRequests.asScala - assert(requests("cpu").getAmount === "2") - assert(requests("memory").getAmount === "456Mi") + assert(amountAndFormat(requests("cpu")) === "2") + assert(amountAndFormat(requests("memory")) === "456Mi") val limits = resourceRequirements.getLimits.asScala - assert(limits("memory").getAmount === "456Mi") - assert(limits("cpu").getAmount === "4") + assert(amountAndFormat(limits("memory")) === "456Mi") + assert(amountAndFormat(limits("cpu")) === "4") resources.foreach { case (k8sName, testRInfo) => - assert(limits(k8sName).getAmount === testRInfo.count) + assert(amountAndFormat(limits(k8sName)) === testRInfo.count) } val driverPodMetadata = configuredPod.pod.getMetadata @@ -141,7 +141,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .configurePod(basePod) .container.getResources .getRequests.asScala - assert(requests1("cpu").getAmount === "1") + assert(amountAndFormat(requests1("cpu")) === "1") // if spark.driver.cores is set it should be used sparkConf.set(DRIVER_CORES, 10) @@ -149,7 +149,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .configurePod(basePod) .container.getResources .getRequests.asScala - assert(requests2("cpu").getAmount === "10") + assert(amountAndFormat(requests2("cpu")) === "10") // spark.kubernetes.driver.request.cores should be preferred over spark.driver.cores Seq("0.1", "100m").foreach { value => @@ -158,7 +158,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .configurePod(basePod) .container.getResources .getRequests.asScala - assert(requests3("cpu").getAmount === value) + assert(amountAndFormat(requests3("cpu")) === value) } } @@ -204,7 +204,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { mainAppResource = resource) val step = new BasicDriverFeatureStep(conf) val pod = step.configurePod(SparkPod.initialPod()) - val mem = pod.container.getResources.getRequests.get("memory").getAmount() + val mem = amountAndFormat(pod.container.getResources.getRequests.get("memory")) val expected = (driverMem + driverMem * expectedFactor).toInt assert(mem === s"${expected}Mi") @@ -219,4 +219,6 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .withContainerPort(portNumber) .withProtocol("TCP") .build() + + private def amountAndFormat(quantity: Quantity): String = quantity.getAmount + quantity.getFormat } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index f375b1fe6a5cd..da50372d04c73 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -128,10 +128,11 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { val executor = step.configurePod(SparkPod.initialPod()) assert(executor.container.getResources.getLimits.size() === 3) - assert(executor.container.getResources - .getLimits.get("memory").getAmount === "1408Mi") + assert(amountAndFormat(executor.container.getResources + .getLimits.get("memory")) === "1408Mi") gpuResources.foreach { case (k8sName, testRInfo) => - assert(executor.container.getResources.getLimits.get(k8sName).getAmount === testRInfo.count) + assert(amountAndFormat( + executor.container.getResources.getLimits.get(k8sName)) === testRInfo.count) } } @@ -151,8 +152,8 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { assert(executor.container.getImage === EXECUTOR_IMAGE) assert(executor.container.getVolumeMounts.isEmpty) assert(executor.container.getResources.getLimits.size() === 1) - assert(executor.container.getResources - .getLimits.get("memory").getAmount === "1408Mi") + assert(amountAndFormat(executor.container.getResources + .getLimits.get("memory")) === "1408Mi") // The pod has no node selector, volumes. assert(executor.pod.getSpec.getNodeSelector.isEmpty) @@ -201,7 +202,7 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) val executor = step.configurePod(SparkPod.initialPod()) // This is checking that basic executor + executorMemory = 1408 + 42 = 1450 - assert(executor.container.getResources.getRequests.get("memory").getAmount === "1450Mi") + assert(amountAndFormat(executor.container.getResources.getRequests.get("memory")) === "1450Mi") } test("auth secret propagation") { @@ -273,4 +274,6 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { val expectedEnvs = defaultEnvs ++ additionalEnvVars ++ extraJavaOptsEnvs assert(containerEnvs === expectedEnvs) } + + private def amountAndFormat(quantity: Quantity): String = quantity.getAmount + quantity.getFormat } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala index 8c430eeb3fa71..3888062785324 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala @@ -79,7 +79,8 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { assert(configuredPod.pod.getSpec.getVolumes.size() === 1) val emptyDir = configuredPod.pod.getSpec.getVolumes.get(0).getEmptyDir assert(emptyDir.getMedium === "Memory") - assert(emptyDir.getSizeLimit.getAmount === "6G") + assert(emptyDir.getSizeLimit.getAmount === "6") + assert(emptyDir.getSizeLimit.getFormat === "G") assert(configuredPod.container.getVolumeMounts.size() === 1) assert(configuredPod.container.getVolumeMounts.get(0).getMountPath === "/tmp") assert(configuredPod.container.getVolumeMounts.get(0).getName === "testVolume") @@ -101,7 +102,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { assert(configuredPod.pod.getSpec.getVolumes.size() === 1) val emptyDir = configuredPod.pod.getSpec.getVolumes.get(0).getEmptyDir assert(emptyDir.getMedium === "") - assert(emptyDir.getSizeLimit.getAmount === null) + assert(emptyDir.getSizeLimit === null) assert(configuredPod.container.getVolumeMounts.size() === 1) assert(configuredPod.container.getVolumeMounts.get(0).getMountPath === "/tmp") assert(configuredPod.container.getVolumeMounts.get(0).getName === "testVolume") diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala index 7181774b9f17e..289fb9641295d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala @@ -54,12 +54,8 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => ).toArray val resources = Map( - "cpu" -> new QuantityBuilder() - .withAmount("1") - .build(), - "memory" -> new QuantityBuilder() - .withAmount("512M") - .build() + "cpu" -> new Quantity("1"), + "memory" -> new Quantity("512M") ).asJava new ContainerBuilder() diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala index f0218217e6afb..4b4dff93f5742 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala @@ -45,7 +45,7 @@ private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => .withName("test-local-pv") .endMetadata() .withNewSpec() - .withCapacity(Map("storage" -> new QuantityBuilder().withAmount("1Gi").build()).asJava) + .withCapacity(Map("storage" -> new Quantity("1Gi")).asJava) .withAccessModes("ReadWriteOnce") .withPersistentVolumeReclaimPolicy("Retain") .withStorageClassName("test-local-storage") @@ -71,8 +71,7 @@ private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => .withAccessModes("ReadWriteOnce") .withStorageClassName("test-local-storage") .withResources(new ResourceRequirementsBuilder() - .withRequests(Map("storage" -> new QuantityBuilder() - .withAmount("1Gi").build()).asJava).build()) + .withRequests(Map("storage" -> new Quantity("1Gi")).asJava).build()) .endSpec() kubernetesTestComponents From 459e757ed40fd1cdd37911d3f57b48b54ca2fff7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 5 Feb 2020 18:48:45 +0800 Subject: [PATCH 1303/1387] [SPARK-30668][SQL] Support `SimpleDateFormat` patterns in parsing timestamps/dates strings ### What changes were proposed in this pull request? In the PR, I propose to partially revert the commit https://github.com/apache/spark/commit/51a6ba0181a013f2b62b47184785a8b6f6a78f12, and provide a legacy parser based on `FastDateFormat` which is compatible to `SimpleDateFormat`. To enable the legacy parser, set `spark.sql.legacy.timeParser.enabled` to `true`. ### Why are the changes needed? To allow users to restore old behavior in parsing timestamps/dates using `SimpleDateFormat` patterns. The main reason for restoring is `DateTimeFormatter`'s patterns are not fully compatible to `SimpleDateFormat` patterns, see https://issues.apache.org/jira/browse/SPARK-30668 ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? - Added new test to `DateFunctionsSuite` - Restored additional test cases in `JsonInferSchemaSuite`. Closes #27441 from MaxGekk/support-simpledateformat. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 4 +- .../sql/catalyst/util/DateFormatter.scala | 35 +++++++- .../catalyst/util/TimestampFormatter.scala | 38 +++++++-- .../apache/spark/sql/internal/SQLConf.scala | 10 +++ .../catalyst/json/JsonInferSchemaSuite.scala | 79 +++++++++++-------- .../apache/spark/sql/DateFunctionsSuite.scala | 14 ++++ 6 files changed, 136 insertions(+), 44 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 0c47370283736..5a5e802f6a900 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -67,9 +67,7 @@ license: | - Since Spark 3.0, Proleptic Gregorian calendar is used in parsing, formatting, and converting dates and timestamps as well as in extracting sub-components like years, days and etc. Spark 3.0 uses Java 8 API classes from the java.time packages that based on ISO chronology (https://docs.oracle.com/javase/8/docs/api/java/time/chrono/IsoChronology.html). In Spark version 2.4 and earlier, those operations are performed by using the hybrid calendar (Julian + Gregorian, see https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html). The changes impact on the results for dates before October 15, 1582 (Gregorian) and affect on the following Spark 3.0 API: - - CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. - - - The `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. + - Parsing/formatting of timestamp/date strings. This effects on CSV/JSON datasources and on the `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions when patterns specified by users is used for parsing and formatting. Since Spark 3.0, the conversions are based on `java.time.format.DateTimeFormatter`, see https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html. New implementation performs strict checking of its input. For example, the `2015-07-22 10:00:00` timestamp cannot be parse if pattern is `yyyy-MM-dd` because the parser does not consume whole input. Another example is the `31/01/2015 00:00` input cannot be parsed by the `dd/MM/yyyy hh:mm` pattern because `hh` supposes hours in the range `1-12`. In Spark version 2.4 and earlier, `java.text.SimpleDateFormat` is used for timestamp/date string conversions, and the supported patterns are described in https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html. The old behavior can be restored by setting `spark.sql.legacy.timeParser.enabled` to `true`. - The `weekofyear`, `weekday`, `dayofweek`, `date_trunc`, `from_utc_timestamp`, `to_utc_timestamp`, and `unix_timestamp` functions use java.time API for calculation week number of year, day number of week as well for conversion from/to TimestampType values in UTC time zone. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 7f982b019c8d1..28189b65dee9a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -20,7 +20,10 @@ package org.apache.spark.sql.catalyst.util import java.time.{LocalDate, ZoneId} import java.util.Locale -import DateTimeUtils.{convertSpecialDate, localDateToDays} +import org.apache.commons.lang3.time.FastDateFormat + +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, localDateToDays} +import org.apache.spark.sql.internal.SQLConf sealed trait DateFormatter extends Serializable { def parse(s: String): Int // returns days since epoch @@ -48,17 +51,41 @@ class Iso8601DateFormatter( } } +class LegacyDateFormatter(pattern: String, locale: Locale) extends DateFormatter { + @transient + private lazy val format = FastDateFormat.getInstance(pattern, locale) + + override def parse(s: String): Int = { + val milliseconds = format.parse(s).getTime + DateTimeUtils.millisToDays(milliseconds) + } + + override def format(days: Int): String = { + val date = DateTimeUtils.toJavaDate(days) + format.format(date) + } +} + object DateFormatter { - val defaultPattern: String = "uuuu-MM-dd" val defaultLocale: Locale = Locale.US def apply(format: String, zoneId: ZoneId, locale: Locale): DateFormatter = { - new Iso8601DateFormatter(format, zoneId, locale) + if (SQLConf.get.legacyTimeParserEnabled) { + new LegacyDateFormatter(format, locale) + } else { + new Iso8601DateFormatter(format, zoneId, locale) + } } def apply(format: String, zoneId: ZoneId): DateFormatter = { apply(format, zoneId, defaultLocale) } - def apply(zoneId: ZoneId): DateFormatter = apply(defaultPattern, zoneId) + def apply(zoneId: ZoneId): DateFormatter = { + if (SQLConf.get.legacyTimeParserEnabled) { + new LegacyDateFormatter("yyyy-MM-dd", defaultLocale) + } else { + new Iso8601DateFormatter("uuuu-MM-dd", zoneId, defaultLocale) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index 5be4807083fa3..fe1a4fe710c20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -22,10 +22,14 @@ import java.time._ import java.time.format.DateTimeParseException import java.time.temporal.ChronoField.MICRO_OF_SECOND import java.time.temporal.TemporalQueries -import java.util.Locale +import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit.SECONDS -import DateTimeUtils.convertSpecialTimestamp +import org.apache.commons.lang3.time.FastDateFormat + +import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_MILLIS +import org.apache.spark.sql.catalyst.util.DateTimeUtils.convertSpecialTimestamp +import org.apache.spark.sql.internal.SQLConf sealed trait TimestampFormatter extends Serializable { /** @@ -86,12 +90,32 @@ class FractionTimestampFormatter(zoneId: ZoneId) override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter } +class LegacyTimestampFormatter( + pattern: String, + zoneId: ZoneId, + locale: Locale) extends TimestampFormatter { + + @transient private lazy val format = + FastDateFormat.getInstance(pattern, TimeZone.getTimeZone(zoneId), locale) + + protected def toMillis(s: String): Long = format.parse(s).getTime + + override def parse(s: String): Long = toMillis(s) * MICROS_PER_MILLIS + + override def format(us: Long): String = { + format.format(DateTimeUtils.toJavaTimestamp(us)) + } +} + object TimestampFormatter { - val defaultPattern: String = "uuuu-MM-dd HH:mm:ss" val defaultLocale: Locale = Locale.US def apply(format: String, zoneId: ZoneId, locale: Locale): TimestampFormatter = { - new Iso8601TimestampFormatter(format, zoneId, locale) + if (SQLConf.get.legacyTimeParserEnabled) { + new LegacyTimestampFormatter(format, zoneId, locale) + } else { + new Iso8601TimestampFormatter(format, zoneId, locale) + } } def apply(format: String, zoneId: ZoneId): TimestampFormatter = { @@ -99,7 +123,11 @@ object TimestampFormatter { } def apply(zoneId: ZoneId): TimestampFormatter = { - apply(defaultPattern, zoneId, defaultLocale) + if (SQLConf.get.legacyTimeParserEnabled) { + new LegacyTimestampFormatter("yyyy-MM-dd HH:mm:ss", zoneId, defaultLocale) + } else { + new Iso8601TimestampFormatter("uuuu-MM-dd HH:mm:ss", zoneId, defaultLocale) + } } def getFractionFormatter(zoneId: ZoneId): TimestampFormatter = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 44f6cccc47015..b0be37d2b2ee5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2159,6 +2159,14 @@ object SQLConf { .checkValue(_ > 0, "The value of spark.sql.addPartitionInBatch.size must be positive") .createWithDefault(100) + val LEGACY_TIME_PARSER_ENABLED = buildConf("spark.sql.legacy.timeParser.enabled") + .internal() + .doc("When set to true, java.text.SimpleDateFormat is used for formatting and parsing " + + "dates/timestamps in a locale-sensitive manner. When set to false, classes from " + + "java.time.* packages are used for the same purpose.") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * @@ -2447,6 +2455,8 @@ class SQLConf extends Serializable with Logging { def legacyMsSqlServerNumericMappingEnabled: Boolean = getConf(LEGACY_MSSQLSERVER_NUMERIC_MAPPING_ENABLED) + def legacyTimeParserEnabled: Boolean = getConf(SQLConf.LEGACY_TIME_PARSER_ENABLED) + /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two * identifiers are equal. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala index a48e61861c158..c2e03bd2c3609 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.catalyst.json -import com.fasterxml.jackson.core.JsonFactory - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { @@ -41,45 +40,61 @@ class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { } test("inferring timestamp type") { - checkTimestampType("yyyy", """{"a": "2018"}""") - checkTimestampType("yyyy=MM", """{"a": "2018=12"}""") - checkTimestampType("yyyy MM dd", """{"a": "2018 12 02"}""") - checkTimestampType( - "yyyy-MM-dd'T'HH:mm:ss.SSS", - """{"a": "2018-12-02T21:04:00.123"}""") - checkTimestampType( - "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXX", - """{"a": "2018-12-02T21:04:00.123567+01:00"}""") + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + checkTimestampType("yyyy", """{"a": "2018"}""") + checkTimestampType("yyyy=MM", """{"a": "2018=12"}""") + checkTimestampType("yyyy MM dd", """{"a": "2018 12 02"}""") + checkTimestampType( + "yyyy-MM-dd'T'HH:mm:ss.SSS", + """{"a": "2018-12-02T21:04:00.123"}""") + checkTimestampType( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXX", + """{"a": "2018-12-02T21:04:00.123567+01:00"}""") + } + } } test("prefer decimals over timestamps") { - checkType( - options = Map( - "prefersDecimal" -> "true", - "timestampFormat" -> "yyyyMMdd.HHmmssSSS" - ), - json = """{"a": "20181202.210400123"}""", - dt = DecimalType(17, 9) - ) + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + checkType( + options = Map( + "prefersDecimal" -> "true", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS" + ), + json = """{"a": "20181202.210400123"}""", + dt = DecimalType(17, 9) + ) + } + } } test("skip decimal type inferring") { - checkType( - options = Map( - "prefersDecimal" -> "false", - "timestampFormat" -> "yyyyMMdd.HHmmssSSS" - ), - json = """{"a": "20181202.210400123"}""", - dt = TimestampType - ) + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + checkType( + options = Map( + "prefersDecimal" -> "false", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS" + ), + json = """{"a": "20181202.210400123"}""", + dt = TimestampType + ) + } + } } test("fallback to string type") { - checkType( - options = Map("timestampFormat" -> "yyyy,MM,dd.HHmmssSSS"), - json = """{"a": "20181202.210400123"}""", - dt = StringType - ) + Seq(true, false).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + checkType( + options = Map("timestampFormat" -> "yyyy,MM,dd.HHmmssSSS"), + json = """{"a": "20181202.210400123"}""", + dt = StringType + ) + } + } } test("disable timestamp inferring") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index d7d8c2c52d12b..3b3d3cc3d7a17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -789,4 +789,18 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { Row(Timestamp.valueOf("2015-07-24 07:00:00")), Row(Timestamp.valueOf("2015-07-24 22:00:00")))) } + + test("SPARK-30668: use legacy timestamp parser in to_timestamp") { + def checkTimeZoneParsing(expected: Any): Unit = { + val df = Seq("2020-01-27T20:06:11.847-0800").toDF("ts") + checkAnswer(df.select(to_timestamp(col("ts"), "yyyy-MM-dd'T'HH:mm:ss.SSSz")), + Row(expected)) + } + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> "true") { + checkTimeZoneParsing(Timestamp.valueOf("2020-01-27 20:06:11.847")) + } + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> "false") { + checkTimeZoneParsing(null) + } + } } From 6d507b4a31feb965bf31d104f1a6a2c359b166dc Mon Sep 17 00:00:00 2001 From: turbofei Date: Wed, 5 Feb 2020 21:24:02 +0800 Subject: [PATCH 1304/1387] [SPARK-26218][SQL][FOLLOW UP] Fix the corner case when casting float to Integer ### What changes were proposed in this pull request? When spark.sql.ansi.enabled is true, for the statement: ``` select cast(cast(2147483648 as Float) as Integer) //result is 2147483647 ``` Its result is 2147483647 and does not throw `ArithmeticException`. The root cause is that, the below code does not work for some corner cases. https://github.com/apache/spark/blob/94fc0e3235162afc6038019eed6ec546e3d1983e/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala#L129-L141 For example: ![image](https://user-images.githubusercontent.com/6757692/72074911-badfde80-332d-11ea-963e-2db0e43c33e8.png) In this PR, I fix it by comparing Math.floor(x) with Int.MaxValue directly. ### Why are the changes needed? Result corrupt. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added Unit test. Closes #27151 from turboFei/SPARK-26218-follow-up-int-overflow. Authored-by: turbofei Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/types/numerics.scala | 16 ++++++++-------- .../sql-tests/results/postgreSQL/float4.sql.out | 5 +++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 11 +++++++++++ 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala index 1ac85360f944f..b5226213effc4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala @@ -121,10 +121,10 @@ object FloatExactNumeric extends FloatIsFractional { private def overflowException(x: Float, dataType: String) = throw new ArithmeticException(s"Casting $x to $dataType causes overflow") - private val intUpperBound = Int.MaxValue.toFloat - private val intLowerBound = Int.MinValue.toFloat - private val longUpperBound = Long.MaxValue.toFloat - private val longLowerBound = Long.MinValue.toFloat + private val intUpperBound = Int.MaxValue + private val intLowerBound = Int.MinValue + private val longUpperBound = Long.MaxValue + private val longLowerBound = Long.MinValue override def toInt(x: Float): Int = { // When casting floating values to integral types, Spark uses the method `Numeric.toInt` @@ -155,10 +155,10 @@ object DoubleExactNumeric extends DoubleIsFractional { private def overflowException(x: Double, dataType: String) = throw new ArithmeticException(s"Casting $x to $dataType causes overflow") - private val intUpperBound = Int.MaxValue.toDouble - private val intLowerBound = Int.MinValue.toDouble - private val longUpperBound = Long.MaxValue.toDouble - private val longLowerBound = Long.MinValue.toDouble + private val intUpperBound = Int.MaxValue + private val intLowerBound = Int.MinValue + private val longUpperBound = Long.MaxValue + private val longLowerBound = Long.MinValue override def toInt(x: Double): Int = { if (Math.floor(x) <= intUpperBound && Math.ceil(x) >= intLowerBound) { diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out index ba913789d5623..fe8375c5eab8f 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out @@ -322,9 +322,10 @@ struct -- !query SELECT int(float('2147483647')) -- !query schema -struct +struct<> -- !query output -2147483647 +java.lang.ArithmeticException +Casting 2.14748365E9 to int causes overflow -- !query 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 a6dae9a269740..11f9724e587f2 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 @@ -3383,6 +3383,17 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark checkAnswer(df, Row(1)) } } + + test("SPARK-26218: Fix the corner case when casting float to Integer") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + intercept[ArithmeticException]( + sql("SELECT CAST(CAST(2147483648 as FLOAT) as Integer)").collect() + ) + intercept[ArithmeticException]( + sql("SELECT CAST(CAST(2147483648 as DOUBLE) as Integer)").collect() + ) + } + } } case class Foo(bar: Option[String]) From ec70e0708f953f3b22ec17d931ff388d007ac1f6 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 5 Feb 2020 07:54:16 -0800 Subject: [PATCH 1305/1387] [MINOR][DOC] Add migration note for removing `org.apache.spark.ml.image.ImageSchema.readImages` ### What changes were proposed in this pull request? Add migration note for removing `org.apache.spark.ml.image.ImageSchema.readImages` ### Why are the changes needed? ### Does this PR introduce any user-facing change? ### How was this patch tested? Closes #27467 from WeichenXu123/SC-26286. Authored-by: WeichenXu Signed-off-by: Dongjoon Hyun --- docs/ml-migration-guide.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ml-migration-guide.md b/docs/ml-migration-guide.md index 49f701b2156b3..860c941e6b44b 100644 --- a/docs/ml-migration-guide.md +++ b/docs/ml-migration-guide.md @@ -32,6 +32,7 @@ Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide. {:.no_toc} * `OneHotEncoder` which is deprecated in 2.3, is removed in 3.0 and `OneHotEncoderEstimator` is now renamed to `OneHotEncoder`. +* `org.apache.spark.ml.image.ImageSchema.readImages` which is deprecated in 2.3, is removed in 3.0, use `spark.read.format('image')` instead. ### Changes of behavior {:.no_toc} From 9d90c8b898d0f043afbcebd901ec866c2883c6ca Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 5 Feb 2020 11:01:53 -0800 Subject: [PATCH 1306/1387] [SPARK-30738][K8S] Use specific image version in "Launcher client dependencies" test ### What changes were proposed in this pull request? This PR use a specific version of docker image instead of `latest`. As of today, when I run K8s integration test locally, this test case fails always. Also, in this PR, I shows two consecutive failures with a dummy change. - https://github.com/apache/spark/pull/27465#issuecomment-582326614 - https://github.com/apache/spark/pull/27465#issuecomment-582329114 ``` - Launcher client dependencies *** FAILED *** ``` After that, I added the patch and K8s Integration test passed. - https://github.com/apache/spark/pull/27465#issuecomment-582361696 ### Why are the changes needed? [SPARK-28465](https://github.com/apache/spark/pull/25222) switched from `v4.0.0-stable-4.0-master-centos-7-x86_64` to `latest` to catch up the API change. However, the API change seems to occur again. We had better use a specific version to prevent accidental failures. ```scala - .withImage("ceph/daemon:v4.0.0-stable-4.0-master-centos-7-x86_64") + .withImage("ceph/daemon:latest") ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass `Launcher client dependencies` test in Jenkins K8s Integration Suite. Or, run K8s Integration test locally. Closes #27465 from dongjoon-hyun/SPARK-K8S-IT. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../spark/deploy/k8s/integrationtest/DepsTestsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala index 289fb9641295d..367cff62cd493 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala @@ -59,7 +59,7 @@ private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => ).asJava new ContainerBuilder() - .withImage("ceph/daemon:latest") + .withImage("ceph/daemon:v4.0.3-stable-4.0-nautilus-centos-7-x86_64") .withImagePullPolicy("Always") .withName(cName) .withPorts(new ContainerPortBuilder() From 4938905a1c047e367c066e39dce8232bfcff14f1 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 5 Feb 2020 11:19:42 -0800 Subject: [PATCH 1307/1387] [SPARK-29864][SQL][FOLLOWUP] Reference the config for the old behavior in error message ### What changes were proposed in this pull request? Follow up work for SPARK-29864, reference the config `spark.sql.legacy.fromDayTimeString.enabled` in error message. ### Why are the changes needed? For better usability. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #27464 from xuanyuanking/SPARK-29864-follow. Authored-by: Yuanjian Li Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/util/IntervalUtils.scala | 9 +++++++-- .../sql-tests/results/ansi/interval.sql.out | 12 +++++------ .../sql-tests/results/interval.sql.out | 12 +++++------ .../results/postgreSQL/interval.sql.out | 20 +++++++++---------- 4 files changed, 29 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 7692299a46ef5..2d98384363323 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -176,6 +176,9 @@ object IntervalUtils { private val dayTimePatternLegacy = "^([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?$".r + private val fallbackNotice = s"set ${SQLConf.LEGACY_FROM_DAYTIME_STRING.key} to true " + + "to restore the behavior before Spark 3.0." + /** * Legacy method of parsing a string in a day-time format. It ignores the `from` bound, * and takes into account only the `to` bound by truncating the result. For example, @@ -195,7 +198,8 @@ object IntervalUtils { require(input != null, "Interval day-time string must be not null") assert(input.length == input.trim.length) val m = dayTimePatternLegacy.pattern.matcher(input) - require(m.matches, s"Interval string must match day-time format of 'd h:m:s.n': $input") + require(m.matches, s"Interval string must match day-time format of 'd h:m:s.n': $input, " + + s"$fallbackNotice") try { val sign = if (m.group(1) != null && m.group(1) == "-") -1 else 1 @@ -296,7 +300,8 @@ object IntervalUtils { require(regexp.isDefined, s"Cannot support (interval '$input' $from to $to) expression") val pattern = regexp.get.pattern val m = pattern.matcher(input) - require(m.matches, s"Interval string must match day-time format of '$pattern': $input") + require(m.matches, s"Interval string must match day-time format of '$pattern': $input, " + + s"$fallbackNotice") var micros: Long = 0L var days: Int = 0 unitsRange(to, from).foreach { diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index ab6130da869c4..f37049064d989 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -320,7 +320,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '20 15:40:32.99899999' day to hour @@ -334,7 +334,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '20 15:40:32.99899999' day to minute @@ -348,7 +348,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 15:40:32.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '15:40:32.99899999' hour to minute @@ -362,7 +362,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '15:40.99899999' hour to second @@ -376,7 +376,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '15:40' hour to second @@ -390,7 +390,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 20 40:32.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '20 40:32.99899999' minute to second diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 8f523a35f3c19..94b4f15815ca5 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -314,7 +314,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '20 15:40:32.99899999' day to hour @@ -328,7 +328,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 20 15:40:32.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '20 15:40:32.99899999' day to minute @@ -342,7 +342,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 15:40:32.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '15:40:32.99899999' hour to minute @@ -356,7 +356,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '15:40.99899999' hour to second @@ -370,7 +370,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '15:40' hour to second @@ -384,7 +384,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 20 40:32.99899999(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == select interval '20 40:32.99899999' minute to second diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out index 4bd846d3ff923..62d47410aab65 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/interval.sql.out @@ -105,7 +105,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 1 2:03(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03' day to hour @@ -119,7 +119,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 1 2:03:04(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2})$': 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03:04' day to hour @@ -141,7 +141,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 1 2:03:04(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2})$': 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03:04' day to minute @@ -155,7 +155,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d+) (?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03' day to second @@ -177,7 +177,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 1 2:03(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03' hour to minute @@ -191,7 +191,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 1 2:03:04(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2})$': 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03:04' hour to minute @@ -205,7 +205,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03' hour to second @@ -219,7 +219,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03:04(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03:04' hour to second @@ -233,7 +233,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03' minute to second @@ -247,7 +247,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03:04(line 1, pos 16) +requirement failed: Interval string must match day-time format of '^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 1 2:03:04, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16) == SQL == SELECT interval '1 2:03:04' minute to second From 3b26f807a0eb0e59c5123c3f1e2262b712800c0f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 5 Feb 2020 12:36:51 -0800 Subject: [PATCH 1308/1387] [SPARK-30721][SQL][TESTS] Fix DataFrameAggregateSuite when enabling AQE ### What changes were proposed in this pull request? update `DataFrameAggregateSuite` to make it pass with AQE ### Why are the changes needed? We don't need to turn off AQE in `DataFrameAggregateSuite` ### Does this PR introduce any user-facing change? no ### How was this patch tested? run `DataFrameAggregateSuite` locally with AQE on. Closes #27451 from cloud-fan/aqe-test. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../spark/sql/DataFrameAggregateSuite.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index dc1767a6852f6..d7df75fd0e2c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -615,34 +615,33 @@ class DataFrameAggregateSuite extends QueryTest Seq((true, true), (true, false), (false, true), (false, false))) { withSQLConf( (SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, wholeStage.toString), - (SQLConf.USE_OBJECT_HASH_AGG.key, useObjectHashAgg.toString), - (SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false")) { - // When enable AQE, the WholeStageCodegenExec is added during QueryStageExec. + (SQLConf.USE_OBJECT_HASH_AGG.key, useObjectHashAgg.toString)) { val df = Seq(("1", 1), ("1", 2), ("2", 3), ("2", 4)).toDF("x", "y") // test case for HashAggregate val hashAggDF = df.groupBy("x").agg(c, sum("y")) + hashAggDF.collect() val hashAggPlan = hashAggDF.queryExecution.executedPlan if (wholeStage) { - assert(hashAggPlan.find { + assert(find(hashAggPlan) { case WholeStageCodegenExec(_: HashAggregateExec) => true case _ => false }.isDefined) } else { - assert(hashAggPlan.isInstanceOf[HashAggregateExec]) + assert(stripAQEPlan(hashAggPlan).isInstanceOf[HashAggregateExec]) } - hashAggDF.collect() // test case for ObjectHashAggregate and SortAggregate val objHashAggOrSortAggDF = df.groupBy("x").agg(c, collect_list("y")) - val objHashAggOrSortAggPlan = objHashAggOrSortAggDF.queryExecution.executedPlan + objHashAggOrSortAggDF.collect() + val objHashAggOrSortAggPlan = + stripAQEPlan(objHashAggOrSortAggDF.queryExecution.executedPlan) if (useObjectHashAgg) { assert(objHashAggOrSortAggPlan.isInstanceOf[ObjectHashAggregateExec]) } else { assert(objHashAggOrSortAggPlan.isInstanceOf[SortAggregateExec]) } - objHashAggOrSortAggDF.collect() } } } From f9f06eee9853ad4b6458ac9d31233e729a1ca226 Mon Sep 17 00:00:00 2001 From: yudovin Date: Wed, 5 Feb 2020 14:16:59 -0800 Subject: [PATCH 1309/1387] [SPARK-30122][K8S] Support spark.kubernetes.authenticate.executor.serviceAccountName ### What changes were proposed in this pull request? Currently, it doesn't seem to be possible to have Spark Driver set the serviceAccountName for executor pods it launches. ### Why are the changes needed? it will allow settings serviceAccountName for executors pods. ### Does this PR introduce any user-facing change? No ### How was this patch tested? It was covered by unit test. Closes #27034 from ayudovin/srevice-account-name-for-executor-pods. Authored-by: yudovin Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/deploy/k8s/Config.scala | 9 +- ...iverKubernetesCredentialsFeatureStep.scala | 2 +- ...utorKubernetesCredentialsFeatureStep.scala | 14 +-- ...ubernetesCredentialsFeatureStepSuite.scala | 87 +++++++++++++++++++ 4 files changed, 105 insertions(+), 7 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.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 b326650476add..46fd8e7be2f1d 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 @@ -116,7 +116,7 @@ private[spark] object Config extends Logging { .intConf .createWithDefault(10000) - val KUBERNETES_SERVICE_ACCOUNT_NAME = + val KUBERNETES_DRIVER_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 " + @@ -125,6 +125,13 @@ private[spark] object Config extends Logging { .stringConf .createOptional + val KUBERNETES_EXECUTOR_SERVICE_ACCOUNT_NAME = + ConfigBuilder(s"$KUBERNETES_AUTH_EXECUTOR_CONF_PREFIX.serviceAccountName") + .doc("Service account that is used when running the executor pod." + + "If this parameter is not setup, the fallback logic will use the driver's service account.") + .stringConf + .createOptional + val KUBERNETES_DRIVER_LIMIT_CORES = ConfigBuilder("spark.kubernetes.driver.limit.cores") .doc("Specify the hard cpu limit for the driver pod") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala index b94fc9d2ffe9c..7233ddd4c0d2e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala @@ -42,7 +42,7 @@ private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: Kube s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") private val maybeMountedCaCertFile = kubernetesConf.getOption( s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") - private val driverServiceAccount = kubernetesConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) + private val driverServiceAccount = kubernetesConf.get(KUBERNETES_DRIVER_SERVICE_ACCOUNT_NAME) private val oauthTokenBase64 = kubernetesConf .getOption(s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala index 8b84aad553855..9514a2cb8e0ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala @@ -17,19 +17,23 @@ package org.apache.spark.deploy.k8s.features import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -import org.apache.spark.deploy.k8s.Config.KUBERNETES_SERVICE_ACCOUNT_NAME +import org.apache.spark.deploy.k8s.Config.{KUBERNETES_DRIVER_SERVICE_ACCOUNT_NAME, KUBERNETES_EXECUTOR_SERVICE_ACCOUNT_NAME} import org.apache.spark.deploy.k8s.KubernetesUtils.buildPodWithServiceAccount private[spark] class ExecutorKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { - private lazy val driverServiceAccount = kubernetesConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) + + private lazy val driverServiceAccount = kubernetesConf.get(KUBERNETES_DRIVER_SERVICE_ACCOUNT_NAME) + private lazy val executorServiceAccount = + kubernetesConf.get(KUBERNETES_EXECUTOR_SERVICE_ACCOUNT_NAME) override def configurePod(pod: SparkPod): SparkPod = { pod.copy( - // if not setup by the pod template fallback to the driver's sa, - // last option is the default sa. + // if not setup by the pod template, fallback to the executor's sa, + // if executor's sa is not setup, the last option is driver's sa. pod = if (Option(pod.pod.getSpec.getServiceAccount).isEmpty) { - buildPodWithServiceAccount(driverServiceAccount, pod).getOrElse(pod.pod) + buildPodWithServiceAccount(executorServiceAccount + .orElse(driverServiceAccount), pod).getOrElse(pod.pod) } else { pod.pod }) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala new file mode 100644 index 0000000000000..59cc7ac91d1ab --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStepSuite.scala @@ -0,0 +1,87 @@ +/* + * 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.features + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} +import org.apache.spark.deploy.k8s.Config._ + +class ExecutorKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { + + private var baseConf: SparkConf = _ + + before { + baseConf = new SparkConf(false) + } + + private def newExecutorConf(environment: Map[String, String] = Map.empty): + KubernetesExecutorConf = { + KubernetesTestConf.createExecutorConf( + sparkConf = baseConf, + environment = environment) + } + + test("configure spark pod with executor service account") { + baseConf.set(KUBERNETES_EXECUTOR_SERVICE_ACCOUNT_NAME, "executor-name") + val step = new ExecutorKubernetesCredentialsFeatureStep(newExecutorConf()) + val spec = step + .configurePod(SparkPod.initialPod()) + .pod + .getSpec + + val serviceAccountName = spec.getServiceAccountName + val accountName = spec.getServiceAccount + assertSAName(serviceAccountName, accountName) + } + + test("configure spark pod with with driver service account " + + "and without executor service account") { + baseConf.set(KUBERNETES_DRIVER_SERVICE_ACCOUNT_NAME, "driver-name") + val step = new ExecutorKubernetesCredentialsFeatureStep(newExecutorConf()) + val spec = step + .configurePod(SparkPod.initialPod()) + .pod + .getSpec + + val serviceAccountName = spec.getServiceAccountName + val accountName = spec.getServiceAccount + assertSAName(serviceAccountName, accountName) + } + + test("configure spark pod with with driver service account " + + "and with executor service account") { + baseConf.set(KUBERNETES_DRIVER_SERVICE_ACCOUNT_NAME, "driver-name") + baseConf.set(KUBERNETES_EXECUTOR_SERVICE_ACCOUNT_NAME, "executor-name") + + val step = new ExecutorKubernetesCredentialsFeatureStep(newExecutorConf()) + val spec = step + .configurePod(SparkPod.initialPod()) + .pod + .getSpec + + val serviceAccountName = spec.getServiceAccountName + val accountName = spec.getServiceAccount + assertSAName(serviceAccountName, accountName) + } + + def assertSAName(serviceAccountName: String, accountName: String): Unit = { + assert(serviceAccountName.equals(serviceAccountName)) + assert(accountName.equals(accountName)) + } +} From 9d907bc84df2f6f7e1abdb810b761a65ac6ce064 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 5 Feb 2020 16:55:45 -0800 Subject: [PATCH 1310/1387] [SPARK-30743][K8S][TESTS] Use JRE instead of JDK in K8S test docker image ### What changes were proposed in this pull request? This PR aims to replace JDK to JRE in K8S integration test docker images. ### Why are the changes needed? This will save some resources and make it sure that we only need JRE at runtime and testing. - https://lists.apache.org/thread.html/3145150b711d7806a86bcd3ab43e18bcd0e4892ab5f11600689ba087%40%3Cdev.spark.apache.org%3E ### Does this PR introduce any user-facing change? No. This is a dev-only test environment. ### How was this patch tested? Pass the Jenkins K8s Integration Test. - https://github.com/apache/spark/pull/27469#issuecomment-582681125 Closes #27469 from dongjoon-hyun/SPARK-30743. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../kubernetes/docker/src/main/dockerfiles/spark/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index b6eeff1cd18a9..a1fc63789bc61 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -15,7 +15,7 @@ # limitations under the License. # -FROM openjdk:8-jdk-slim +FROM openjdk:8-jre-slim ARG spark_uid=185 From 20c60a43ccebd6fc05e18b4a0bd4ddbd0ed658f9 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 5 Feb 2020 17:01:33 -0800 Subject: [PATCH 1311/1387] [MINOR][INFRA] Factor Python executable out as a variable in 'lint-python' script ### What changes were proposed in this pull request? This PR proposes to make hardcoded `python3` to a variable `PYTHON_EXECUTABLE` in ' lint-python' script. ### Why are the changes needed? To make changes easier. See https://github.com/apache/spark/commit/561e9b968821ca3e501aa5cba7ba5ceaa45796ea as an example. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually by running `dev/lint-python`. Closes #27470 from HyukjinKwon/minor-PYTHON_EXECUTABLE. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- dev/lint-python | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/dev/lint-python b/dev/lint-python index e9ed83dec5fce..24f0d8fb6ea36 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -27,6 +27,8 @@ MINIMUM_PYCODESTYLE="2.4.0" SPHINX_BUILD="sphinx-build" +PYTHON_EXECUTABLE="python3" + function compile_python_test { local COMPILE_STATUS= local COMPILE_REPORT= @@ -38,7 +40,7 @@ function compile_python_test { # compileall: https://docs.python.org/3/library/compileall.html echo "starting python compilation test..." - COMPILE_REPORT=$( (python3 -B -mcompileall -q -l -x "[/\\\\][.]git" $1) 2>&1) + COMPILE_REPORT=$( ("$PYTHON_EXECUTABLE" -B -mcompileall -q -l -x "[/\\\\][.]git" $1) 2>&1) COMPILE_STATUS=$? if [ $COMPILE_STATUS -ne 0 ]; then @@ -70,7 +72,7 @@ function pycodestyle_test { RUN_LOCAL_PYCODESTYLE="False" if hash "$PYCODESTYLE_BUILD" 2> /dev/null; then VERSION=$( $PYCODESTYLE_BUILD --version 2> /dev/null) - EXPECTED_PYCODESTYLE=$( (python3 -c 'from distutils.version import LooseVersion; + EXPECTED_PYCODESTYLE=$( ("$PYTHON_EXECUTABLE" -c 'from distutils.version import LooseVersion; print(LooseVersion("""'${VERSION[0]}'""") >= LooseVersion("""'$MINIMUM_PYCODESTYLE'"""))')\ 2> /dev/null) @@ -96,7 +98,7 @@ function pycodestyle_test { fi echo "starting pycodestyle test..." - PYCODESTYLE_REPORT=$( (python3 "$PYCODESTYLE_SCRIPT_PATH" --config=dev/tox.ini $1) 2>&1) + PYCODESTYLE_REPORT=$( ("$PYTHON_EXECUTABLE" "$PYCODESTYLE_SCRIPT_PATH" --config=dev/tox.ini $1) 2>&1) PYCODESTYLE_STATUS=$? else # we have the right version installed, so run locally @@ -130,7 +132,7 @@ function flake8_test { FLAKE8_VERSION="$($FLAKE8_BUILD --version 2> /dev/null)" VERSION=($FLAKE8_VERSION) - EXPECTED_FLAKE8=$( (python3 -c 'from distutils.version import LooseVersion; + EXPECTED_FLAKE8=$( ("$PYTHON_EXECUTABLE" -c 'from distutils.version import LooseVersion; print(LooseVersion("""'${VERSION[0]}'""") >= LooseVersion("""'$MINIMUM_FLAKE8'"""))') \ 2> /dev/null) @@ -175,7 +177,7 @@ function pydocstyle_test { fi PYDOCSTYLE_VERSION="$($PYDOCSTYLEBUILD --version 2> /dev/null)" - EXPECTED_PYDOCSTYLE=$(python3 -c 'from distutils.version import LooseVersion; \ + EXPECTED_PYDOCSTYLE=$("$PYTHON_EXECUTABLE" -c 'from distutils.version import LooseVersion; \ print(LooseVersion("""'$PYDOCSTYLE_VERSION'""") >= LooseVersion("""'$MINIMUM_PYDOCSTYLE'"""))' \ 2> /dev/null) From b95ccb1d8b726b11435789cdb5882df6643430ed Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Thu, 6 Feb 2020 13:01:08 +0900 Subject: [PATCH 1312/1387] [SPARK-30737][SPARK-27262][R][BUILD] Reenable CRAN check with UTF-8 encoding to DESCRIPTION ### What changes were proposed in this pull request? This PR proposes to reenable CRAN check disabled at https://github.com/apache/spark/pull/27460. Given the tests https://github.com/apache/spark/pull/27468, seems we should also port https://github.com/apache/spark/pull/23823 together. ### Why are the changes needed? To check CRAN back. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? It was tested at https://github.com/apache/spark/pull/27468 and Jenkins should test it out. Closes #27472 from HyukjinKwon/SPARK-30737. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- R/pkg/DESCRIPTION | 1 + R/pkg/tests/fulltests/test_sparkSQL.R | 7 +------ R/run-tests.sh | 7 +++---- 3 files changed, 5 insertions(+), 10 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 95d3e52bef3a9..c8cb1c3a992ad 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -62,3 +62,4 @@ Collate: RoxygenNote: 5.0.1 VignetteBuilder: knitr NeedsCompilation: no +Encoding: UTF-8 diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 23fadc4373c3f..c1d277ac84be1 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -496,12 +496,7 @@ test_that("SPARK-17811: can create DataFrame containing NA as date and time", { expect_true(is.na(DF$date[2])) expect_equal(DF$date[1], as.Date("2016-10-01")) expect_true(is.na(DF$time[2])) - # Warnings were suppressed due to Jenkins environment issues. - # It shows both warnings as below in Jenkins: - # - Your system is mis-configured: /etc/localtime is not a symlink - # - It is strongly recommended to set environment variable TZ to - # America/Los_Angeles (or equivalent) - suppressWarnings(expect_equal(DF$time[1], as.POSIXlt("2016-01-10"))) + expect_equal(DF$time[1], as.POSIXlt("2016-01-10")) }) test_that("create DataFrame with complex types", { diff --git a/R/run-tests.sh b/R/run-tests.sh index 782b5f5baebaf..51ca7d600caf0 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -31,10 +31,9 @@ NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" # Also run the documentation tests for CRAN CRAN_CHECK_LOG_FILE=$FWDIR/cran-check.out rm -f $CRAN_CHECK_LOG_FILE -# TODO(SPARK-30737) reenable this once packages are correctly installed in Jenkins -# NO_TESTS=1 NO_MANUAL=1 $FWDIR/check-cran.sh 2>&1 | tee -a $CRAN_CHECK_LOG_FILE -# FAILED=$((PIPESTATUS[0]||$FAILED)) -touch $CRAN_CHECK_LOG_FILE + +NO_TESTS=1 NO_MANUAL=1 $FWDIR/check-cran.sh 2>&1 | tee -a $CRAN_CHECK_LOG_FILE +FAILED=$((PIPESTATUS[0]||$FAILED)) NUM_CRAN_WARNING="$(grep -c WARNING$ $CRAN_CHECK_LOG_FILE)" NUM_CRAN_ERROR="$(grep -c ERROR$ $CRAN_CHECK_LOG_FILE)" From aebabf0bed712511eaa8844cab3a0c562219b2d0 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 6 Feb 2020 12:48:27 +0800 Subject: [PATCH 1313/1387] [SPARK-30729][CORE] Eagerly filter out zombie TaskSetManager before offering resources ### What changes were proposed in this pull request? Eagerly filter out zombie `TaskSetManager` before offering resources to reduce any overhead as possible. And this PR also avoid doing `recomputeLocality` and `addPendingTask` when `TaskSetManager` is zombie. ### Why are the changes needed? Zombie `TaskSetManager` could still exist in Pool's `schedulableQueue` when it has running tasks. Offering resources on a zombie `TaskSetManager` could bring unnecessary overhead and is meaningless. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass Jenkins. Closes #27455 from Ngone51/exclude-zombie-tsm. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index f25a36c7af22a..6a1d460e6a9d9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -430,7 +430,7 @@ private[spark] class TaskSchedulerImpl( val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK)) val availableResources = shuffledOffers.map(_.resources).toArray val availableCpus = shuffledOffers.map(o => o.cores).toArray - val sortedTaskSets = rootPool.getSortedTaskSetQueue + val sortedTaskSets = rootPool.getSortedTaskSetQueue.filterNot(_.isZombie) for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 3b620ec69a9ab..2ce11347ade39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -229,6 +229,8 @@ private[spark] class TaskSetManager( index: Int, resolveRacks: Boolean = true, speculatable: Boolean = false): Unit = { + // A zombie TaskSetManager may reach here while handling failed task. + if (isZombie) return val pendingTaskSetToAddTo = if (speculatable) pendingSpeculatableTasks else pendingTasks for (loc <- tasks(index).preferredLocations) { loc match { @@ -1082,6 +1084,8 @@ private[spark] class TaskSetManager( } def recomputeLocality(): Unit = { + // A zombie TaskSetManager may reach here while executorLost happens + if (isZombie) return val previousLocalityLevel = myLocalityLevels(currentLocalityIndex) myLocalityLevels = computeValidLocalityLevels() localityWaits = myLocalityLevels.map(getLocalityWait) From c27a616450959a5e984d10bf93b12ac0ced6c94d Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 6 Feb 2020 13:54:17 +0800 Subject: [PATCH 1314/1387] [SPARK-30612][SQL] Resolve qualified column name with v2 tables ### What changes were proposed in this pull request? This PR fixes the issue where queries with qualified columns like `SELECT t.a FROM t` would fail to resolve for v2 tables. This PR would allow qualified column names in query as following: ```SQL SELECT testcat.ns1.ns2.tbl.foo FROM testcat.ns1.ns2.tbl SELECT ns1.ns2.tbl.foo FROM testcat.ns1.ns2.tbl SELECT ns2.tbl.foo FROM testcat.ns1.ns2.tbl SELECT tbl.foo FROM testcat.ns1.ns2.tbl ``` ### Why are the changes needed? This is a bug because you cannot qualify column names in queries. ### Does this PR introduce any user-facing change? Yes, now users can qualify column names for v2 tables. ### How was this patch tested? Added new tests. Closes #27391 from imback82/qualified_col. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 5 +- .../expressions/namedExpressions.scala | 2 - .../sql/catalyst/expressions/package.scala | 83 ++++++++++- .../spark/sql/catalyst/identifiers.scala | 16 +- .../plans/logical/basicLogicalOperators.scala | 17 ++- .../spark/sql/catalyst/trees/TreeNode.scala | 3 +- .../AttributeResolutionSuite.scala | 137 ++++++++++++++++++ .../sql/catalyst/trees/TreeNodeSuite.scala | 10 +- .../sql-tests/results/group-by-filter.sql.out | 32 ++-- .../invalid-correlation.sql.out | 4 +- .../sql/connector/DataSourceV2SQLSuite.scala | 38 +++++ .../benchmark/TPCDSQueryBenchmark.scala | 2 +- .../command/PlanResolutionSuite.scala | 65 +++++---- .../sql/hive/HiveMetastoreCatalogSuite.scala | 2 +- 14 files changed, 342 insertions(+), 74 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeResolutionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3fd5039a4f116..56cc2a274bb7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -799,6 +799,7 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = ResolveTempViews(plan).resolveOperatorsUp { case u: UnresolvedRelation => lookupV2Relation(u.multipartIdentifier) + .map(SubqueryAlias(u.multipartIdentifier, _)) .getOrElse(u) case u @ UnresolvedTable(NonSessionCatalogAndIdentifier(catalog, ident)) => @@ -923,7 +924,9 @@ class Analyzer( case v1Table: V1Table => v1SessionCatalog.getRelation(v1Table.v1Table) case table => - DataSourceV2Relation.create(table, Some(catalog), Some(ident)) + SubqueryAlias( + identifier, + DataSourceV2Relation.create(table, Some(catalog), Some(ident))) } val key = catalog.name +: ident.namespace :+ ident.name Option(AnalysisContext.get.relationCache.getOrElseUpdate(key, loaded.orNull)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 3362353e2662a..02e90f8458c3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -236,8 +236,6 @@ case class AttributeReference( val qualifier: Seq[String] = Seq.empty[String]) extends Attribute with Unevaluable { - // currently can only handle qualifier of length 2 - require(qualifier.length <= 2) /** * Returns true iff the expression id is the same for both attributes. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 7164b6b82adbc..9f42e643e4cb2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -23,7 +23,6 @@ import com.google.common.collect.Maps import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StructField, StructType} /** @@ -153,13 +152,19 @@ package object expressions { unique(grouped) } - /** Perform attribute resolution given a name and a resolver. */ - def resolve(nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = { + /** Returns true if all qualifiers in `attrs` have 2 or less parts. */ + @transient private val hasTwoOrLessQualifierParts: Boolean = + attrs.forall(_.qualifier.length <= 2) + + /** Match attributes for the case where all qualifiers in `attrs` have 2 or less parts. */ + private def matchWithTwoOrLessQualifierParts( + nameParts: Seq[String], + resolver: Resolver): (Seq[Attribute], Seq[String]) = { // Collect matching attributes given a name and a lookup. def collectMatches(name: String, candidates: Option[Seq[Attribute]]): Seq[Attribute] = { - candidates.toSeq.flatMap(_.collect { + candidates.getOrElse(Nil).collect { case a if resolver(a.name, name) => a.withName(name) - }) + } } // Find matches for the given name assuming that the 1st two parts are qualifier @@ -204,13 +209,79 @@ package object expressions { // If none of attributes match database.table.column pattern or // `table.column` pattern, we try to resolve it as a column. - val (candidates, nestedFields) = matches match { + matches match { case (Seq(), _) => val name = nameParts.head val attributes = collectMatches(name, direct.get(name.toLowerCase(Locale.ROOT))) (attributes, nameParts.tail) case _ => matches } + } + + /** + * Match attributes for the case where at least one qualifier in `attrs` has more than 2 parts. + */ + private def matchWithThreeOrMoreQualifierParts( + nameParts: Seq[String], + resolver: Resolver): (Seq[Attribute], Seq[String]) = { + // Returns true if the `short` qualifier is a subset of the last elements of + // `long` qualifier. For example, Seq("a", "b") is a subset of Seq("a", "a", "b"), + // but not a subset of Seq("a", "b", "b"). + def matchQualifier(short: Seq[String], long: Seq[String]): Boolean = { + (long.length >= short.length) && + long.takeRight(short.length) + .zip(short) + .forall(x => resolver(x._1, x._2)) + } + + // Collect attributes that match the given name and qualifier. + // A match occurs if + // 1) the given name matches the attribute's name according to the resolver. + // 2) the given qualifier is a subset of the attribute's qualifier. + def collectMatches( + name: String, + qualifier: Seq[String], + candidates: Option[Seq[Attribute]]): Seq[Attribute] = { + candidates.getOrElse(Nil).collect { + case a if resolver(name, a.name) && matchQualifier(qualifier, a.qualifier) => + a.withName(name) + } + } + + // Iterate each string in `nameParts` in a reverse order and try to match the attributes + // considering the current string as the attribute name. For example, if `nameParts` is + // Seq("a", "b", "c"), the match will be performed in the following order: + // 1) name = "c", qualifier = Seq("a", "b") + // 2) name = "b", qualifier = Seq("a") + // 3) name = "a", qualifier = Seq() + // Note that the match is performed in the reverse order in order to match the longest + // qualifier as possible. If a match is found, the remaining portion of `nameParts` + // is also returned as nested fields. + var candidates: Seq[Attribute] = Nil + var nestedFields: Seq[String] = Nil + var i = nameParts.length - 1 + while (i >= 0 && candidates.isEmpty) { + val name = nameParts(i) + candidates = collectMatches( + name, + nameParts.take(i), + direct.get(name.toLowerCase(Locale.ROOT))) + if (candidates.nonEmpty) { + nestedFields = nameParts.takeRight(nameParts.length - i - 1) + } + i -= 1 + } + + (candidates, nestedFields) + } + + /** Perform attribute resolution given a name and a resolver. */ + def resolve(nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = { + val (candidates, nestedFields) = if (hasTwoOrLessQualifierParts) { + matchWithTwoOrLessQualifierParts(nameParts, resolver) + } else { + matchWithThreeOrMoreQualifierParts(nameParts, resolver) + } def name = UnresolvedAttribute(nameParts).name candidates match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala index deceec73dda30..c574a20da0b5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala @@ -49,19 +49,21 @@ sealed trait IdentifierWithDatabase { /** * Encapsulates an identifier that is either a alias name or an identifier that has table - * name and optionally a database name. + * name and a qualifier. * The SubqueryAlias node keeps track of the qualifier using the information in this structure - * @param identifier - Is an alias name or a table name - * @param database - Is a database name and is optional + * @param name - Is an alias name or a table name + * @param qualifier - Is a qualifier */ -case class AliasIdentifier(identifier: String, database: Option[String]) - extends IdentifierWithDatabase { +case class AliasIdentifier(name: String, qualifier: Seq[String]) { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + def this(identifier: String) = this(identifier, Seq()) - def this(identifier: String) = this(identifier, None) + override def toString: String = (qualifier :+ name).quoted } object AliasIdentifier { - def apply(identifier: String): AliasIdentifier = new AliasIdentifier(identifier) + def apply(name: String): AliasIdentifier = new AliasIdentifier(name) } /** 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 40db8b6f49dc4..54e5ff7aeb754 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.types._ import org.apache.spark.util.random.RandomSampler @@ -849,18 +850,18 @@ case class Tail(limitExpr: Expression, child: LogicalPlan) extends OrderPreservi /** * Aliased subquery. * - * @param name the alias identifier for this subquery. + * @param identifier the alias identifier for this subquery. * @param child the logical plan of this subquery. */ case class SubqueryAlias( - name: AliasIdentifier, + identifier: AliasIdentifier, child: LogicalPlan) extends OrderPreservingUnaryNode { - def alias: String = name.identifier + def alias: String = identifier.name override def output: Seq[Attribute] = { - val qualifierList = name.database.map(Seq(_, alias)).getOrElse(Seq(alias)) + val qualifierList = identifier.qualifier :+ alias child.output.map(_.withQualifier(qualifierList)) } override def doCanonicalize(): LogicalPlan = child.canonicalized @@ -877,7 +878,13 @@ object SubqueryAlias { identifier: String, database: String, child: LogicalPlan): SubqueryAlias = { - SubqueryAlias(AliasIdentifier(identifier, Some(database)), child) + SubqueryAlias(AliasIdentifier(identifier, Seq(database)), child) + } + + def apply( + multipartIdentifier: Seq[String], + child: LogicalPlan): SubqueryAlias = { + SubqueryAlias(AliasIdentifier(multipartIdentifier.last, multipartIdentifier.init), child) } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index ba1eeb38e247e..56a198763b4e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -27,7 +27,7 @@ import org.json4s.JsonAST._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.sql.catalyst.IdentifierWithDatabase +import org.apache.spark.sql.catalyst.{AliasIdentifier, IdentifierWithDatabase} import org.apache.spark.sql.catalyst.ScalaReflection._ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, FunctionResource} import org.apache.spark.sql.catalyst.errors._ @@ -780,6 +780,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case exprId: ExprId => true case field: StructField => true case id: IdentifierWithDatabase => true + case alias: AliasIdentifier => true case join: JoinType => true case spec: BucketSpec => true case catalog: CatalogTable => true diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeResolutionSuite.scala new file mode 100644 index 0000000000000..813a68f68451c --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeResolutionSuite.scala @@ -0,0 +1,137 @@ +/* + * 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.AnalysisException +import org.apache.spark.sql.catalyst.analysis.{caseInsensitiveResolution, caseSensitiveResolution} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} + +class AttributeResolutionSuite extends SparkFunSuite { + val resolver = caseInsensitiveResolution + + test("basic attribute resolution with namespaces") { + val attrs = Seq( + AttributeReference("a", IntegerType)(qualifier = Seq("ns1", "ns2", "t1")), + AttributeReference("a", IntegerType)(qualifier = Seq("ns1", "ns2", "ns3", "t2"))) + + // Try to match attribute reference with name "a" with qualifier "ns1.ns2.t1". + Seq(Seq("t1", "a"), Seq("ns2", "t1", "a"), Seq("ns1", "ns2", "t1", "a")).foreach { nameParts => + attrs.resolve(nameParts, resolver) match { + case Some(attr) => assert(attr.semanticEquals(attrs(0))) + case _ => fail() + } + } + + // Non-matching cases + Seq(Seq("ns1", "ns2", "t1"), Seq("ns2", "a")).foreach { nameParts => + assert(attrs.resolve(nameParts, resolver).isEmpty) + } + } + + test("attribute resolution where table and attribute names are the same") { + val attrs = Seq(AttributeReference("t", IntegerType)(qualifier = Seq("ns1", "ns2", "t"))) + // Matching cases + Seq( + Seq("t"), Seq("t", "t"), Seq("ns2", "t", "t"), Seq("ns1", "ns2", "t", "t") + ).foreach { nameParts => + attrs.resolve(nameParts, resolver) match { + case Some(attr) => assert(attr.semanticEquals(attrs(0))) + case _ => fail() + } + } + + // Non-matching case + assert(attrs.resolve(Seq("ns1", "ns2", "t"), resolver).isEmpty) + } + + test("attribute resolution ambiguity at the attribute name level") { + val attrs = Seq( + AttributeReference("a", IntegerType)(qualifier = Seq("ns1", "t1")), + AttributeReference("a", IntegerType)(qualifier = Seq("ns1", "ns2", "t2"))) + + val ex = intercept[AnalysisException] { + attrs.resolve(Seq("a"), resolver) + } + assert(ex.getMessage.contains( + "Reference 'a' is ambiguous, could be: ns1.t1.a, ns1.ns2.t2.a.")) + } + + test("attribute resolution ambiguity at the qualifier level") { + val attrs = Seq( + AttributeReference("a", IntegerType)(qualifier = Seq("ns1", "t")), + AttributeReference("a", IntegerType)(qualifier = Seq("ns2", "ns1", "t"))) + + val ex = intercept[AnalysisException] { + attrs.resolve(Seq("ns1", "t", "a"), resolver) + } + assert(ex.getMessage.contains( + "Reference 'ns1.t.a' is ambiguous, could be: ns1.t.a, ns2.ns1.t.a.")) + } + + test("attribute resolution with nested fields") { + val attrType = StructType(Seq(StructField("aa", IntegerType), StructField("bb", IntegerType))) + val attrs = Seq(AttributeReference("a", attrType)(qualifier = Seq("ns1", "t"))) + + val resolved = attrs.resolve(Seq("ns1", "t", "a", "aa"), resolver) + resolved match { + case Some(Alias(_, name)) => assert(name == "aa") + case _ => fail() + } + + val ex = intercept[AnalysisException] { + attrs.resolve(Seq("ns1", "t", "a", "cc"), resolver) + } + assert(ex.getMessage.contains("No such struct field cc in aa, bb")) + } + + test("attribute resolution with case insensitive resolver") { + val attrs = Seq(AttributeReference("a", IntegerType)(qualifier = Seq("ns1", "t"))) + attrs.resolve(Seq("Ns1", "T", "A"), caseInsensitiveResolution) match { + case Some(attr) => assert(attr.semanticEquals(attrs(0)) && attr.name == "A") + case _ => fail() + } + } + + test("attribute resolution with case sensitive resolver") { + val attrs = Seq(AttributeReference("a", IntegerType)(qualifier = Seq("ns1", "t"))) + assert(attrs.resolve(Seq("Ns1", "T", "A"), caseSensitiveResolution).isEmpty) + assert(attrs.resolve(Seq("ns1", "t", "A"), caseSensitiveResolution).isEmpty) + attrs.resolve(Seq("ns1", "t", "a"), caseSensitiveResolution) match { + case Some(attr) => assert(attr.semanticEquals(attrs(0))) + case _ => fail() + } + } + + test("attribute resolution should try to match the longest qualifier") { + // We have two attributes: + // 1) "a.b" where "a" is the name and "b" is the nested field. + // 2) "a.b.a" where "b" is the name, left-side "a" is the qualifier and the right-side "a" + // is the nested field. + // When "a.b" is resolved, "b" is tried first as the name, so it is resolved to #2 attribute. + val a1Type = StructType(Seq(StructField("b", IntegerType))) + val a2Type = StructType(Seq(StructField("a", IntegerType))) + val attrs = Seq( + AttributeReference("a", a1Type)(), + AttributeReference("b", a2Type)(qualifier = Seq("a"))) + attrs.resolve(Seq("a", "b"), resolver) match { + case Some(attr) => assert(attr.semanticEquals(attrs(1))) + case _ => fail() + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 0e094bc06b05f..e72b2e9b1b214 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -433,10 +433,11 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { // Converts AliasIdentifier to JSON assertJSON( - AliasIdentifier("alias"), + AliasIdentifier("alias", Seq("ns1", "ns2")), JObject( "product-class" -> JString(classOf[AliasIdentifier].getName), - "identifier" -> "alias")) + "name" -> "alias", + "qualifier" -> "[ns1, ns2]")) // Converts SubqueryAlias to JSON assertJSON( @@ -445,8 +446,9 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { JObject( "class" -> classOf[SubqueryAlias].getName, "num-children" -> 1, - "name" -> JObject("product-class" -> JString(classOf[AliasIdentifier].getName), - "identifier" -> "t1"), + "identifier" -> JObject("product-class" -> JString(classOf[AliasIdentifier].getName), + "name" -> "t1", + "qualifier" -> JArray(Nil)), "child" -> 0), JObject( "class" -> classOf[JsonTestTreeNode].getName, diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out index a032678e90fe8..a4c7c2cf90cd7 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out @@ -369,13 +369,13 @@ org.apache.spark.sql.AnalysisException IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE exists#x [dept_id#x]) AS avg(salary) FILTER (WHERE exists(dept_id))#x] : +- Project [state#x] : +- Filter (dept_id#x = outer(dept_id#x)) -: +- SubqueryAlias `dept` +: +- SubqueryAlias dept : +- Project [dept_id#x, dept_name#x, state#x] -: +- SubqueryAlias `DEPT` +: +- SubqueryAlias DEPT : +- LocalRelation [dept_id#x, dept_name#x, state#x] -+- SubqueryAlias `emp` ++- SubqueryAlias emp +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] - +- SubqueryAlias `EMP` + +- SubqueryAlias EMP +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ; @@ -395,13 +395,13 @@ org.apache.spark.sql.AnalysisException IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT exists#x [dept_id#x]) AS sum(salary) FILTER (WHERE (NOT exists(dept_id)))#x] : +- Project [state#x] : +- Filter (dept_id#x = outer(dept_id#x)) -: +- SubqueryAlias `dept` +: +- SubqueryAlias dept : +- Project [dept_id#x, dept_name#x, state#x] -: +- SubqueryAlias `DEPT` +: +- SubqueryAlias DEPT : +- LocalRelation [dept_id#x, dept_name#x, state#x] -+- SubqueryAlias `emp` ++- SubqueryAlias emp +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] - +- SubqueryAlias `EMP` + +- SubqueryAlias EMP +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ; @@ -420,13 +420,13 @@ org.apache.spark.sql.AnalysisException IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE dept_id#x IN (list#x [])) AS avg(salary) FILTER (WHERE (dept_id IN (listquery())))#x] : +- Distinct : +- Project [dept_id#x] -: +- SubqueryAlias `dept` +: +- SubqueryAlias dept : +- Project [dept_id#x, dept_name#x, state#x] -: +- SubqueryAlias `DEPT` +: +- SubqueryAlias DEPT : +- LocalRelation [dept_id#x, dept_name#x, state#x] -+- SubqueryAlias `emp` ++- SubqueryAlias emp +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] - +- SubqueryAlias `EMP` + +- SubqueryAlias EMP +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ; @@ -445,13 +445,13 @@ org.apache.spark.sql.AnalysisException IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT dept_id#x IN (list#x [])) AS sum(salary) FILTER (WHERE (NOT (dept_id IN (listquery()))))#x] : +- Distinct : +- Project [dept_id#x] -: +- SubqueryAlias `dept` +: +- SubqueryAlias dept : +- Project [dept_id#x, dept_name#x, state#x] -: +- SubqueryAlias `DEPT` +: +- SubqueryAlias DEPT : +- LocalRelation [dept_id#x, dept_name#x, state#x] -+- SubqueryAlias `emp` ++- SubqueryAlias emp +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] - +- SubqueryAlias `EMP` + +- SubqueryAlias EMP +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] ; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out index 1599634ff9efb..ec7ecf28754ef 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -110,8 +110,8 @@ struct<> org.apache.spark.sql.AnalysisException Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses: Aggregate [min(outer(t2a#x)) AS min(outer())#x] -+- SubqueryAlias `t3` ++- SubqueryAlias t3 +- Project [t3a#x, t3b#x, t3c#x] - +- SubqueryAlias `t3` + +- SubqueryAlias t3 +- LocalRelation [t3a#x, t3b#x, t3c#x] ; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 2c8349a0e6a75..eabcb81c50646 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -679,6 +679,44 @@ class DataSourceV2SQLSuite } } + test("qualified column names for v2 tables") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, point struct) USING foo") + sql(s"INSERT INTO $t VALUES (1, (10, 20))") + + checkAnswer( + sql(s"SELECT testcat.ns1.ns2.tbl.id, testcat.ns1.ns2.tbl.point.x FROM $t"), + Row(1, 10)) + checkAnswer(sql(s"SELECT ns1.ns2.tbl.id, ns1.ns2.tbl.point.x FROM $t"), Row(1, 10)) + checkAnswer(sql(s"SELECT ns2.tbl.id, ns2.tbl.point.x FROM $t"), Row(1, 10)) + checkAnswer(sql(s"SELECT tbl.id, tbl.point.x FROM $t"), Row(1, 10)) + + val ex = intercept[AnalysisException] { + sql(s"SELECT ns1.ns2.ns3.tbl.id from $t") + } + assert(ex.getMessage.contains("cannot resolve '`ns1.ns2.ns3.tbl.id`")) + } + } + + test("qualified column names for v1 tables") { + // unset this config to use the default v2 session catalog. + spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + + withTable("t") { + sql("CREATE TABLE t USING json AS SELECT 1 AS i") + checkAnswer(sql("select default.t.i from spark_catalog.t"), Row(1)) + checkAnswer(sql("select t.i from spark_catalog.default.t"), Row(1)) + checkAnswer(sql("select default.t.i from spark_catalog.default.t"), Row(1)) + + // catalog name cannot be used for v1 tables. + val ex = intercept[AnalysisException] { + sql(s"select spark_catalog.default.t.i from spark_catalog.default.t") + } + assert(ex.getMessage.contains("cannot resolve '`spark_catalog.default.t.i`")) + } + } + test("InsertInto: append - across catalog") { val t1 = "testcat.ns1.ns2.tbl" val t2 = "testcat2.db.tbl" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala index c93d27f02c686..ad3d79760adf0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -81,7 +81,7 @@ object TPCDSQueryBenchmark extends SqlBasedBenchmark { val queryRelations = scala.collection.mutable.HashSet[String]() spark.sql(queryString).queryExecution.analyzed.foreach { case SubqueryAlias(alias, _: LogicalRelation) => - queryRelations.add(alias.identifier) + queryRelations.add(alias.name) case LogicalRelation(_, _, Some(catalogTable), _) => queryRelations.add(catalogTable.identifier.table) case HiveTableRelation(tableMeta, _, _, _, _) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index c0c3cd70fcc9e..88f30353cce94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -885,33 +885,34 @@ class PlanResolutionSuite extends AnalysisTest { val parsed4 = parseAndResolve(sql4) parsed1 match { - case DeleteFromTable(_: DataSourceV2Relation, None) => - case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed1.treeString) + case DeleteFromTable(AsDataSourceV2Relation(_), None) => + case _ => fail("Expect DeleteFromTable, but got:\n" + parsed1.treeString) } parsed2 match { case DeleteFromTable( - _: DataSourceV2Relation, + AsDataSourceV2Relation(_), Some(EqualTo(name: UnresolvedAttribute, StringLiteral("Robert")))) => assert(name.name == "name") - case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed2.treeString) + case _ => fail("Expect DeleteFromTable, but got:\n" + parsed2.treeString) } parsed3 match { case DeleteFromTable( - SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("t", Seq()), AsDataSourceV2Relation(_)), Some(EqualTo(name: UnresolvedAttribute, StringLiteral("Robert")))) => assert(name.name == "t.name") - case _ => fail("Expect DeleteFromTable, bug got:\n" + parsed3.treeString) + case _ => fail("Expect DeleteFromTable, but got:\n" + parsed3.treeString) } parsed4 match { - case DeleteFromTable(SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + case DeleteFromTable( + SubqueryAlias(AliasIdentifier("t", Seq()), AsDataSourceV2Relation(_)), Some(InSubquery(values, query))) => assert(values.size == 1 && values.head.isInstanceOf[UnresolvedAttribute]) assert(values.head.asInstanceOf[UnresolvedAttribute].name == "t.name") query match { - case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", None), + case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", Seq()), UnresolvedSubqueryColumnAliases(outputColumnNames, Project(_, _: OneRowRelation)))), _, _, _) => assert(projects.size == 1 && projects.head.name == "s.name") @@ -944,7 +945,7 @@ class PlanResolutionSuite extends AnalysisTest { parsed1 match { case UpdateTable( - _: DataSourceV2Relation, + AsDataSourceV2Relation(_), Seq(Assignment(name: UnresolvedAttribute, StringLiteral("Robert")), Assignment(age: UnresolvedAttribute, IntegerLiteral(32))), None) => @@ -956,7 +957,7 @@ class PlanResolutionSuite extends AnalysisTest { parsed2 match { case UpdateTable( - SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("t", Seq()), AsDataSourceV2Relation(_)), Seq(Assignment(name: UnresolvedAttribute, StringLiteral("Robert")), Assignment(age: UnresolvedAttribute, IntegerLiteral(32))), None) => @@ -968,7 +969,7 @@ class PlanResolutionSuite extends AnalysisTest { parsed3 match { case UpdateTable( - SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("t", Seq()), AsDataSourceV2Relation(_)), Seq(Assignment(name: UnresolvedAttribute, StringLiteral("Robert")), Assignment(age: UnresolvedAttribute, IntegerLiteral(32))), Some(EqualTo(p: UnresolvedAttribute, IntegerLiteral(1)))) => @@ -980,14 +981,14 @@ class PlanResolutionSuite extends AnalysisTest { } parsed4 match { - case UpdateTable(SubqueryAlias(AliasIdentifier("t", None), _: DataSourceV2Relation), + case UpdateTable(SubqueryAlias(AliasIdentifier("t", Seq()), AsDataSourceV2Relation(_)), Seq(Assignment(key: UnresolvedAttribute, IntegerLiteral(32))), Some(InSubquery(values, query))) => assert(key.name == "t.age") assert(values.size == 1 && values.head.isInstanceOf[UnresolvedAttribute]) assert(values.head.asInstanceOf[UnresolvedAttribute].name == "t.name") query match { - case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", None), + case ListQuery(Project(projects, SubqueryAlias(AliasIdentifier("s", Seq()), UnresolvedSubqueryColumnAliases(outputColumnNames, Project(_, _: OneRowRelation)))), _, _, _) => assert(projects.size == 1 && projects.head.name == "s.name") @@ -1129,7 +1130,7 @@ class PlanResolutionSuite extends AnalysisTest { case AlterTable(_, _, r: DataSourceV2Relation, _) => assert(r.catalog.exists(_ == catlogIdent)) assert(r.identifier.exists(_.name() == tableIdent)) - case Project(_, r: DataSourceV2Relation) => + case Project(_, AsDataSourceV2Relation(r)) => assert(r.catalog.exists(_ == catlogIdent)) assert(r.identifier.exists(_.name() == tableIdent)) case InsertIntoStatement(r: DataSourceV2Relation, _, _, _, _) => @@ -1206,8 +1207,8 @@ class PlanResolutionSuite extends AnalysisTest { """.stripMargin parseAndResolve(sql1) match { case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), source: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("target", Seq()), AsDataSourceV2Relation(target)), + SubqueryAlias(AliasIdentifier("source", Seq()), AsDataSourceV2Relation(source)), mergeCondition, Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), @@ -1232,8 +1233,8 @@ class PlanResolutionSuite extends AnalysisTest { """.stripMargin parseAndResolve(sql2) match { case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), source: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("target", Seq()), AsDataSourceV2Relation(target)), + SubqueryAlias(AliasIdentifier("source", Seq()), AsDataSourceV2Relation(source)), mergeCondition, Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(ul: AttributeReference, @@ -1258,8 +1259,8 @@ class PlanResolutionSuite extends AnalysisTest { """.stripMargin parseAndResolve(sql3) match { case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), source: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("target", Seq()), AsDataSourceV2Relation(target)), + SubqueryAlias(AliasIdentifier("source", Seq()), AsDataSourceV2Relation(source)), mergeCondition, Seq(DeleteAction(None), UpdateAction(None, updateAssigns)), Seq(InsertAction(None, insertAssigns))) => @@ -1282,8 +1283,8 @@ class PlanResolutionSuite extends AnalysisTest { """.stripMargin parseAndResolve(sql4) match { case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), source: Project), + SubqueryAlias(AliasIdentifier("target", Seq()), AsDataSourceV2Relation(target)), + SubqueryAlias(AliasIdentifier("source", Seq()), source: Project), mergeCondition, Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), @@ -1311,8 +1312,8 @@ class PlanResolutionSuite extends AnalysisTest { """.stripMargin parseAndResolve(sql5) match { case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), target: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), source: Project), + SubqueryAlias(AliasIdentifier("target", Seq()), AsDataSourceV2Relation(target)), + SubqueryAlias(AliasIdentifier("source", Seq()), source: Project), mergeCondition, Seq(DeleteAction(Some(EqualTo(dl: AttributeReference, StringLiteral("delete")))), UpdateAction(Some(EqualTo(ul: AttributeReference, StringLiteral("update"))), @@ -1346,8 +1347,8 @@ class PlanResolutionSuite extends AnalysisTest { parseAndResolve(sql1) match { case MergeIntoTable( - target: DataSourceV2Relation, - source: DataSourceV2Relation, + AsDataSourceV2Relation(target), + AsDataSourceV2Relation(source), _, Seq(DeleteAction(None), UpdateAction(None, updateAssigns)), Seq(InsertAction( @@ -1453,8 +1454,8 @@ class PlanResolutionSuite extends AnalysisTest { parseAndResolve(sql) match { case MergeIntoTable( - SubqueryAlias(AliasIdentifier("target", None), _: DataSourceV2Relation), - SubqueryAlias(AliasIdentifier("source", None), _: DataSourceV2Relation), + SubqueryAlias(AliasIdentifier("target", Seq()), AsDataSourceV2Relation(_)), + SubqueryAlias(AliasIdentifier("source", Seq()), AsDataSourceV2Relation(_)), EqualTo(l: UnresolvedAttribute, r: UnresolvedAttribute), Seq( DeleteAction(Some(EqualTo(dl: UnresolvedAttribute, StringLiteral("delete")))), @@ -1481,3 +1482,11 @@ class PlanResolutionSuite extends AnalysisTest { } // TODO: add tests for more commands. } + +object AsDataSourceV2Relation { + def unapply(plan: LogicalPlan): Option[DataSourceV2Relation] = plan match { + case SubqueryAlias(_, r: DataSourceV2Relation) => Some(r) + case _ => None + } +} + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 20bafd832d0da..b8ef44b096eed 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -62,7 +62,7 @@ class HiveMetastoreCatalogSuite extends TestHiveSingleton with SQLTestUtils { spark.sql("create view vw1 as select 1 as id") val plan = spark.sql("select id from vw1").queryExecution.analyzed val aliases = plan.collect { - case x @ SubqueryAlias(AliasIdentifier("vw1", Some("default")), _) => x + case x @ SubqueryAlias(AliasIdentifier("vw1", Seq("default")), _) => x } assert(aliases.size == 1) } From c8ef1dee90e939e004a024cf98b7ea7e1eaa9ffe Mon Sep 17 00:00:00 2001 From: beliefer Date: Thu, 6 Feb 2020 15:24:26 +0900 Subject: [PATCH 1315/1387] [SPARK-29108][SQL][TESTS][FOLLOWUP] Comment out no use test case and add 'insert into' statement of window.sql (Part 2) ### What changes were proposed in this pull request? When I running the `window_part2.sql` tests find it lack insert sql. Therefore, the output is empty. I checked the postgresql and reference https://github.com/postgres/postgres/blob/master/src/test/regress/sql/window.sql Although `window_part1.sql` and `window_part3.sql` exists the insert sql, I think should also add it into `window_part2.sql`. Because only one case reference the table `empsalary` and it throws `AnalysisException`. ``` -- !query select last(salary) over(order by salary range between 1000 preceding and 1000 following), lag(salary) over(order by salary range between 1000 preceding and 1000 following), salary from empsalary -- !query schema struct<> -- !query output org.apache.spark.sql.AnalysisException Window Frame specifiedwindowframe(RangeFrame, -1000, 1000) must match the required frame specifiedwindowframe(RowFrame, -1, -1); ``` So we should do four work: 1. comment out the only one case and create a new ticket. 2. Add `INSERT INTO empsalary`. Note: window_part4.sql not use the table `empsalary`. ### Why are the changes needed? Supplementary test data. ### Does this PR introduce any user-facing change? No ### How was this patch tested? New test case Closes #27439 from beliefer/add-insert-to-window. Authored-by: beliefer Signed-off-by: HyukjinKwon --- .../inputs/postgreSQL/window_part2.sql | 19 ++++++++++-- .../results/postgreSQL/window_part2.sql.out | 29 ++++++++++++------- 2 files changed, 34 insertions(+), 14 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql index 395149e48d5c8..ba1acc9f56b4a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/window_part2.sql @@ -15,6 +15,18 @@ CREATE TABLE empsalary ( enroll_date date ) USING parquet; +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15'); + -- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp -- CREATE TEMP VIEW v_window AS -- SELECT i, min(i) over (order by i range between '1 day' preceding and '10 days' following) as min_i @@ -99,9 +111,10 @@ FROM tenk1 WHERE unique1 < 10; -- nth_value(salary, 1) over(order by salary range between 1000 preceding and 1000 following), -- salary from empsalary; -select last(salary) over(order by salary range between 1000 preceding and 1000 following), -lag(salary) over(order by salary range between 1000 preceding and 1000 following), -salary from empsalary; +-- [SPARK-30734] AnalysisException that window RangeFrame not match RowFrame +-- select last(salary) over(order by salary range between 1000 preceding and 1000 following), +-- lag(salary) over(order by salary range between 1000 preceding and 1000 following), +-- salary from empsalary; -- [SPARK-27951] ANSI SQL: NTH_VALUE function -- select first_value(salary) over(order by salary range between 1000 following and 3000 following diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out index 0015740a0638e..f41659a196ae1 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part2.sql.out @@ -15,6 +15,24 @@ struct<> +-- !query +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15') +-- !query schema +struct<> +-- !query output + + + -- !query SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), unique1, four @@ -72,17 +90,6 @@ struct --- !query output -org.apache.spark.sql.AnalysisException -Window Frame specifiedwindowframe(RangeFrame, -1000, 1000) must match the required frame specifiedwindowframe(RowFrame, -1, -1); - - -- !query select ss.id, ss.y, first(ss.y) over w, From 3f5b23340e16a855f667d353447655c1345b48a0 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 6 Feb 2020 20:20:44 +0800 Subject: [PATCH 1316/1387] [SPARK-30744][SQL] Optimize AnalyzePartitionCommand by calculating location sizes in parallel ### What changes were proposed in this pull request? Use `CommandUtils.calculateTotalLocationSize` for `AnalyzePartitionCommand` in order to calculate location sizes in parallel. ### Why are the changes needed? For better performance. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass Jenkins. Closes #27471 from Ngone51/dev_calculate_in_parallel. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../command/AnalyzePartitionCommand.scala | 9 +++++---- .../sql/execution/command/CommandUtils.scala | 19 ++++++++++--------- .../spark/sql/execution/command/ddl.scala | 4 ++-- 3 files changed, 17 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala index c5db5ad1157fe..33b29bde93ee5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzePartitionCommand.scala @@ -106,11 +106,12 @@ case class AnalyzePartitionCommand( // Update the metastore if newly computed statistics are different from those // recorded in the metastore. - val newPartitions = partitions.flatMap { p => - val newTotalSize = CommandUtils.calculateSingleLocationSize( - sessionState, tableMeta.identifier, p.storage.locationUri) + + val sizes = CommandUtils.calculateMultipleLocationSizes(sparkSession, tableMeta.identifier, + partitions.map(_.storage.locationUri)) + val newPartitions = partitions.zipWithIndex.flatMap { case (p, idx) => val newRowCount = rowCounts.get(p.spec) - val newStats = CommandUtils.compareAndGetNewStats(p.stats, newTotalSize, newRowCount) + val newStats = CommandUtils.compareAndGetNewStats(p.stats, sizes(idx), newRowCount) newStats.map(_ => p.copy(stats = newStats)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala index b229b238238fd..81157ca0bfe9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala @@ -78,7 +78,7 @@ object CommandUtils extends Logging { val partitions = sessionState.catalog.listPartitions(catalogTable.identifier) logInfo(s"Starting to calculate sizes for ${partitions.length} partitions.") val paths = partitions.map(_.storage.locationUri) - calculateTotalLocationSize(spark, catalogTable.identifier, paths) + calculateMultipleLocationSizes(spark, catalogTable.identifier, paths).sum } logInfo(s"It took ${(System.nanoTime() - startTime) / (1000 * 1000)} ms to calculate" + s" the total size for table ${catalogTable.identifier}.") @@ -137,14 +137,14 @@ object CommandUtils extends Logging { size } - def calculateTotalLocationSize( + def calculateMultipleLocationSizes( sparkSession: SparkSession, tid: TableIdentifier, - paths: Seq[Option[URI]]): Long = { + paths: Seq[Option[URI]]): Seq[Long] = { if (sparkSession.sessionState.conf.parallelFileListingInStatsComputation) { - calculateLocationSizeParallel(sparkSession, paths.map(_.map(new Path(_)))) + calculateMultipleLocationSizesInParallel(sparkSession, paths.map(_.map(new Path(_)))) } else { - paths.map(p => calculateSingleLocationSize(sparkSession.sessionState, tid, p)).sum + paths.map(p => calculateSingleLocationSize(sparkSession.sessionState, tid, p)) } } @@ -153,11 +153,12 @@ object CommandUtils extends Logging { * for each path. * @param sparkSession the [[SparkSession]] * @param paths the Seq of [[Option[Path]]]s - * @return total size of all partitions + * @return a Seq of same size as `paths` where i-th element is total size of `paths(i)` or 0 + * if `paths(i)` is None */ - def calculateLocationSizeParallel( + def calculateMultipleLocationSizesInParallel( sparkSession: SparkSession, - paths: Seq[Option[Path]]): Long = { + paths: Seq[Option[Path]]): Seq[Long] = { val stagingDir = sparkSession.sessionState.conf .getConfString("hive.exec.stagingdir", ".hive-staging") val filter = new PathFilterIgnoreNonData(stagingDir) @@ -166,7 +167,7 @@ object CommandUtils extends Logging { case (_, files) => files.map(_.getLen).sum } // the size is 0 where paths(i) is not defined and sizes(i) where it is defined - paths.zipWithIndex.filter(_._1.isDefined).map(i => sizes(i._2)).sum + paths.zipWithIndex.map { case (p, idx) => p.map(_ => sizes(idx)).getOrElse(0L) } } def compareAndGetNewStats( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index f41c4eca203af..47b213fc2d83b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -486,8 +486,8 @@ case class AlterTableAddPartitionCommand( if (table.stats.nonEmpty) { if (sparkSession.sessionState.conf.autoSizeUpdateEnabled) { - val addedSize = CommandUtils.calculateTotalLocationSize(sparkSession, table.identifier, - parts.map(_.storage.locationUri)) + val addedSize = CommandUtils.calculateMultipleLocationSizes(sparkSession, table.identifier, + parts.map(_.storage.locationUri)).sum if (addedSize > 0) { val newStats = CatalogStatistics(sizeInBytes = table.stats.get.sizeInBytes + addedSize) catalog.alterTableStats(table.identifier, Some(newStats)) From 368ee62a5dce83682ccaec92feeea8428af5a8cf Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Thu, 6 Feb 2020 20:34:29 +0800 Subject: [PATCH 1317/1387] [SPARK-27297][DOC][FOLLOW-UP] Improve documentation for various Scala functions ### What changes were proposed in this pull request? Add examples and parameter description for these Scala functions: * transform * exists * forall * aggregate * zip_with * transform_keys * transform_values * map_filter * map_zip_with ### Why are the changes needed? Better documentation for UX. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass Jenkins. Closes #27449 from Ngone51/doc-funcs. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/functions.scala | 93 +++++++++++++++++-- 1 file changed, 83 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index da26c5a2f4625..d125581857e0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3410,6 +3410,12 @@ object functions { /** * Returns an array of elements after applying a transformation to each element * in the input array. + * {{{ + * df.select(transform(col("i"), x => x + 1)) + * }}} + * + * @param column the input array column + * @param f col => transformed_col, the lambda function to transform the input column * * @group collection_funcs * @since 3.0.0 @@ -3421,6 +3427,13 @@ object functions { /** * Returns an array of elements after applying a transformation to each element * in the input array. + * {{{ + * df.select(transform(col("i"), (x, i) => x + i)) + * }}} + * + * @param column the input array column + * @param f (col, index) => transformed_col, the lambda function to filter the input column + * given the index. Indices start at 0. * * @group collection_funcs * @since 3.0.0 @@ -3431,6 +3444,12 @@ object functions { /** * Returns whether a predicate holds for one or more elements in the array. + * {{{ + * df.select(exists(col("i"), _ % 2 === 0)) + * }}} + * + * @param column the input array column + * @param f col => predicate, the Boolean predicate to check the input column * * @group collection_funcs * @since 3.0.0 @@ -3441,6 +3460,12 @@ object functions { /** * Returns whether a predicate holds for every element in the array. + * {{{ + * df.select(forall(col("i"), x => x % 2 === 0)) + * }}} + * + * @param column the input array column + * @param f col => predicate, the Boolean predicate to check the input column * * @group collection_funcs * @since 3.0.0 @@ -3453,11 +3478,10 @@ object functions { * Returns an array of elements for which a predicate holds in a given array. * {{{ * df.select(filter(col("s"), x => x % 2 === 0)) - * df.selectExpr("filter(col, x -> x % 2 == 0)") * }}} * - * @param column: the input array column - * @param f: col => predicate, the Boolean predicate to filter the input column + * @param column the input array column + * @param f col => predicate, the Boolean predicate to filter the input column * * @group collection_funcs * @since 3.0.0 @@ -3470,11 +3494,10 @@ object functions { * Returns an array of elements for which a predicate holds in a given array. * {{{ * df.select(filter(col("s"), (x, i) => i % 2 === 0)) - * df.selectExpr("filter(col, (x, i) -> i % 2 == 0)") * }}} * - * @param column: the input array column - * @param f: (col, index) => predicate, the Boolean predicate to filter the input column + * @param column the input array column + * @param f (col, index) => predicate, the Boolean predicate to filter the input column * given the index. Indices start at 0. * * @group collection_funcs @@ -3488,18 +3511,28 @@ object functions { * Applies a binary operator to an initial state and all elements in the array, * and reduces this to a single state. The final state is converted into the final result * by applying a finish function. + * {{{ + * df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x, _ * 10)) + * }}} + * + * @param expr the input array column + * @param initialValue the initial value + * @param merge (combined_value, input_value) => combined_value, the merge function to merge + * an input value to the combined_value + * @param finish combined_value => final_value, the lambda function to convert the combined value + * of all inputs to final result * * @group collection_funcs * @since 3.0.0 */ def aggregate( expr: Column, - zero: Column, + initialValue: Column, merge: (Column, Column) => Column, finish: Column => Column): Column = withExpr { ArrayAggregate( expr.expr, - zero.expr, + initialValue.expr, createLambda(merge), createLambda(finish) ) @@ -3508,17 +3541,31 @@ object functions { /** * Applies a binary operator to an initial state and all elements in the array, * and reduces this to a single state. + * {{{ + * df.select(aggregate(col("i"), lit(0), (acc, x) => acc + x)) + * }}} * + * @param expr the input array column + * @param initialValue the initial value + * @param merge (combined_value, input_value) => combined_value, the merge function to merge + * an input value to the combined_value * @group collection_funcs * @since 3.0.0 */ - def aggregate(expr: Column, zero: Column, merge: (Column, Column) => Column): Column = - aggregate(expr, zero, merge, c => c) + def aggregate(expr: Column, initialValue: Column, merge: (Column, Column) => Column): Column = + aggregate(expr, initialValue, merge, c => c) /** * Merge two given arrays, element-wise, into a single array using a function. * If one array is shorter, nulls are appended at the end to match the length of the longer * array, before applying the function. + * {{{ + * df.select(zip_with(df1("val1"), df1("val2"), (x, y) => x + y)) + * }}} + * + * @param left the left input array column + * @param right the right input array column + * @param f (lCol, rCol) => col, the lambda function to merge two input columns into one column * * @group collection_funcs * @since 3.0.0 @@ -3530,6 +3577,12 @@ object functions { /** * Applies a function to every key-value pair in a map and returns * a map with the results of those applications as the new keys for the pairs. + * {{{ + * df.select(transform_keys(col("i"), (k, v) => k + v)) + * }}} + * + * @param expr the input map column + * @param f (key, value) => new_key, the lambda function to transform the key of input map column * * @group collection_funcs * @since 3.0.0 @@ -3541,6 +3594,13 @@ object functions { /** * Applies a function to every key-value pair in a map and returns * a map with the results of those applications as the new values for the pairs. + * {{{ + * df.select(transform_values(col("i"), (k, v) => k + v)) + * }}} + * + * @param expr the input map column + * @param f (key, value) => new_value, the lambda function to transform the value of input map + * column * * @group collection_funcs * @since 3.0.0 @@ -3551,6 +3611,12 @@ object functions { /** * Returns a map whose key-value pairs satisfy a predicate. + * {{{ + * df.select(map_filter(col("m"), (k, v) => k * 10 === v)) + * }}} + * + * @param expr the input map column + * @param f (key, value) => predicate, the Boolean predicate to filter the input map column * * @group collection_funcs * @since 3.0.0 @@ -3561,6 +3627,13 @@ object functions { /** * Merge two given maps, key-wise into a single map using a function. + * {{{ + * df.select(map_zip_with(df("m1"), df("m2"), (k, v1, v2) => k === v1 + v2)) + * }}} + * + * @param left the left input map column + * @param right the right input map column + * @param f (key, value1, value2) => new_value, the lambda function to merge the map values * * @group collection_funcs * @since 3.0.0 From d8613571bc1847775dd5c1945757279234cb388c Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 6 Feb 2020 20:53:44 +0800 Subject: [PATCH 1318/1387] [SPARK-26700][CORE][FOLLOWUP] Add config `spark.network.maxRemoteBlockSizeFetchToMem` ### What changes were proposed in this pull request? Add new config `spark.network.maxRemoteBlockSizeFetchToMem` fallback to the old config `spark.maxRemoteBlockSizeFetchToMem`. ### Why are the changes needed? For naming consistency. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests. Closes #27463 from xuanyuanking/SPARK-26700-follow. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- core/src/main/scala/org/apache/spark/SparkConf.scala | 3 ++- .../main/scala/org/apache/spark/internal/config/package.scala | 2 +- docs/configuration.md | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0e0291d2407d1..40915e3904f7e 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -684,7 +684,8 @@ private[spark] object SparkConf extends Logging { "spark.yarn.jars" -> Seq( AlternateConfig("spark.yarn.jar", "2.0")), MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM.key -> Seq( - AlternateConfig("spark.reducer.maxReqSizeShuffleToMem", "2.3")), + AlternateConfig("spark.reducer.maxReqSizeShuffleToMem", "2.3"), + AlternateConfig("spark.maxRemoteBlockSizeFetchToMem", "3.0")), LISTENER_BUS_EVENT_QUEUE_CAPACITY.key -> Seq( AlternateConfig("spark.scheduler.listenerbus.eventqueue.size", "2.3")), DRIVER_MEMORY_OVERHEAD.key -> Seq( 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 f91f31be2f1ad..02acb6b530737 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 @@ -895,7 +895,7 @@ package object config { .createWithDefault(Int.MaxValue) private[spark] val MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM = - ConfigBuilder("spark.maxRemoteBlockSizeFetchToMem") + ConfigBuilder("spark.network.maxRemoteBlockSizeFetchToMem") .doc("Remote block will be fetched to disk when size of the block is above this threshold " + "in bytes. This is to avoid a giant request takes too much memory. Note this " + "configuration will affect both shuffle fetch and block manager remote block fetch. " + diff --git a/docs/configuration.md b/docs/configuration.md index 2febfe9744d5c..5bd3f3e80cf71 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1810,7 +1810,7 @@ Apart from these, the following properties are also available, and may be useful
    - +
        .ProcessTreeJVMVMemoryVirtual memory size in bytes. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.Virtual memory size in bytes. Enabled if spark.executor.processTreeMetrics.enabled is true.
        .ProcessTreeJVMRSSMemory
        .ProcessTreePythonVMemoryVirtual memory size for Python in bytes. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.Virtual memory size for Python in bytes. Enabled if spark.executor.processTreeMetrics.enabled is true.
        .ProcessTreePythonRSSMemoryResident Set Size for Python. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.Resident Set Size for Python. Enabled if spark.executor.processTreeMetrics.enabled is true.
        .ProcessTreeOtherVMemoryVirtual memory size for other kind of process in bytes. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.Virtual memory size for other kind of process in bytes. Enabled if spark.executor.processTreeMetrics.enabled is true.
        .ProcessTreeOtherRSSMemoryResident Set Size for other kind of process. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.Resident Set Size for other kind of process. Enabled if spark.executor.processTreeMetrics.enabled is true.
        .MinorGCCount
    spark.resources.discovery.pluginorg.apache.spark.resource.ResourceDiscoveryScriptPlugin + Comma-separated list of class names implementing + org.apache.spark.api.resource.ResourceDiscoveryPlugin to load into the application. + This is for advanced users to replace the resource discovery class with a + custom implementation. Spark will try each class specified until one of them + returns the resource information for that resource. It tries the discovery + script last if none of the plugins return information for that resource. +
    spark.executor.memory 1g
    FILTERreservednon-reservedreserved
    FILEFORMATnon-reservednon-reservednon-reserved
    FIRSTnon-reservednon-reservednon-reserved
    FIRST_VALUEreservednon-reservedreserved
    FOLLOWINGnon-reservednon-reservednon-reserved
    FORreservednon-reservedreserved
    FOREIGNreservednon-reservedreserved
    JOINreservedstrict-non-reservedreserved
    KEYSnon-reservednon-reservednon-reserved
    LASTnon-reservednon-reservednon-reserved
    LAST_VALUEreservednon-reservedreserved
    LATERALnon-reservednon-reservedreserved
    LAZYnon-reservednon-reservednon-reserved
    LEADINGreservednon-reservedreserved
    REPAIRnon-reservednon-reservednon-reserved
    REPLACEnon-reservednon-reservednon-reserved
    RESETnon-reservednon-reservednon-reserved
    RESPECTnon-reservednon-reservednon-reserved
    RESTRICTnon-reservednon-reservednon-reserved
    REVOKEnon-reservednon-reservedreserved
    RIGHTreservedstrict-non-reservedreserved
    ignoreExtension trueThe option controls ignoring of files without .avro extensions in read.
    If the option is enabled, all files (with and without .avro extension) are loaded.
    The option has been deprecated, and it will be removed in the future releases. Please use the general data source option pathGlobFilter for filtering file names.
    The option controls ignoring of files without .avro extensions in read.
    If the option is enabled, all files (with and without .avro extension) are loaded.
    The option has been deprecated, and it will be removed in the future releases. Please use the general data source option pathGlobFilter for filtering file names.
    read
    spark.maxRemoteBlockSizeFetchToMemspark.network.maxRemoteBlockSizeFetchToMem 200m Remote block will be fetched to disk when size of the block is above this threshold From 8ce58627ebe4f0372fba9a30d8cd4213611acd9b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 6 Feb 2020 09:16:14 -0800 Subject: [PATCH 1319/1387] [SPARK-30719][SQL] do not log warning if AQE is intentionally skipped and add a config to force apply ### What changes were proposed in this pull request? Update `InsertAdaptiveSparkPlan` to not log warning if AQE is skipped intentionally. This PR also add a config to not skip AQE. ### Why are the changes needed? It's not a warning at all if we intentionally skip AQE. ### Does this PR introduce any user-facing change? no ### How was this patch tested? run `AdaptiveQueryExecSuite` locally and verify that there is no warning logs. Closes #27452 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: Xiao Li --- .../apache/spark/sql/internal/SQLConf.scala | 9 ++ .../adaptive/InsertAdaptiveSparkPlan.scala | 83 +++++++++++-------- .../adaptive/AdaptiveQueryExecSuite.scala | 9 ++ 3 files changed, 65 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index b0be37d2b2ee5..e17a038586c33 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -358,6 +358,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val ADAPTIVE_EXECUTION_FORCE_APPLY = buildConf("spark.sql.adaptive.forceApply") + .internal() + .doc("Adaptive query execution is skipped when the query does not have exchanges or " + + "sub-queries. By setting this config to true (together with " + + s"'${ADAPTIVE_EXECUTION_ENABLED.key}' enabled), Spark will force apply adaptive query " + + "execution for all supported queries.") + .booleanConf + .createWithDefault(false) + val REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED = buildConf("spark.sql.adaptive.shuffle.reducePostShufflePartitions.enabled") .doc(s"When true and '${ADAPTIVE_EXECUTION_ENABLED.key}' is enabled, this enables reducing " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala index 9252827856af4..621c063e5a7d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -40,49 +40,60 @@ case class InsertAdaptiveSparkPlan( private val conf = adaptiveExecutionContext.session.sessionState.conf - def containShuffle(plan: SparkPlan): Boolean = { - plan.find { - case _: Exchange => true - case s: SparkPlan => !s.requiredChildDistribution.forall(_ == UnspecifiedDistribution) - }.isDefined - } - - def containSubQuery(plan: SparkPlan): Boolean = { - plan.find(_.expressions.exists(_.find { - case _: SubqueryExpression => true - case _ => false - }.isDefined)).isDefined - } - override def apply(plan: SparkPlan): SparkPlan = applyInternal(plan, false) private def applyInternal(plan: SparkPlan, isSubquery: Boolean): SparkPlan = plan match { + case _ if !conf.adaptiveExecutionEnabled => plan case _: ExecutedCommandExec => plan - case _ if conf.adaptiveExecutionEnabled && supportAdaptive(plan) - && (isSubquery || containShuffle(plan) || containSubQuery(plan)) => - try { - // Plan sub-queries recursively and pass in the shared stage cache for exchange reuse. Fall - // back to non-adaptive mode if adaptive execution is supported in any of the sub-queries. - val subqueryMap = buildSubqueryMap(plan) - val planSubqueriesRule = PlanAdaptiveSubqueries(subqueryMap) - val preprocessingRules = Seq( - planSubqueriesRule) - // Run pre-processing rules. - val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preprocessingRules) - logDebug(s"Adaptive execution enabled for plan: $plan") - AdaptiveSparkPlanExec(newPlan, adaptiveExecutionContext, preprocessingRules, isSubquery) - } catch { - case SubqueryAdaptiveNotSupportedException(subquery) => - logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + - s"but is not supported for sub-query: $subquery.") - plan - } - case _ => - if (conf.adaptiveExecutionEnabled) { + case _ if shouldApplyAQE(plan, isSubquery) => + if (supportAdaptive(plan)) { + try { + // Plan sub-queries recursively and pass in the shared stage cache for exchange reuse. + // Fall back to non-AQE mode if AQE is not supported in any of the sub-queries. + val subqueryMap = buildSubqueryMap(plan) + val planSubqueriesRule = PlanAdaptiveSubqueries(subqueryMap) + val preprocessingRules = Seq( + planSubqueriesRule) + // Run pre-processing rules. + val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preprocessingRules) + logDebug(s"Adaptive execution enabled for plan: $plan") + AdaptiveSparkPlanExec(newPlan, adaptiveExecutionContext, preprocessingRules, isSubquery) + } catch { + case SubqueryAdaptiveNotSupportedException(subquery) => + logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + + s"but is not supported for sub-query: $subquery.") + plan + } + } else { logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + s"but is not supported for query: $plan.") + plan } - plan + + case _ => plan + } + + // AQE is only useful when the query has exchanges or sub-queries. This method returns true if + // one of the following conditions is satisfied: + // - The config ADAPTIVE_EXECUTION_FORCE_APPLY is true. + // - The input query is from a sub-query. When this happens, it means we've already decided to + // apply AQE for the main query and we must continue to do it. + // - The query contains exchanges. + // - The query may need to add exchanges. It's an overkill to run `EnsureRequirements` here, so + // we just check `SparkPlan.requiredChildDistribution` and see if it's possible that the + // the query needs to add exchanges later. + // - The query contains sub-query. + private def shouldApplyAQE(plan: SparkPlan, isSubquery: Boolean): Boolean = { + conf.getConf(SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY) || isSubquery || { + plan.find { + case _: Exchange => true + case p if !p.requiredChildDistribution.forall(_ == UnspecifiedDistribution) => true + case p => p.expressions.exists(_.find { + case _: SubqueryExpression => true + case _ => false + }.isDefined) + }.isDefined + } } private def supportAdaptive(plan: SparkPlan): Boolean = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 78a1183664749..96e977221e512 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -780,4 +780,13 @@ class AdaptiveQueryExecSuite ) } } + + test("force apply AQE") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + val plan = sql("SELECT * FROM testData").queryExecution.executedPlan + assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) + } + } } From 5a4c70b4e2367441ce4260f02d39d3345078f411 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 6 Feb 2020 13:33:39 -0800 Subject: [PATCH 1320/1387] [SPARK-27986][SQL][FOLLOWUP] window aggregate function with filter predicate is not supported ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/26656. We don't support window aggregate function with filter predicate yet and we should fail explicitly. Observable metrics has the same issue. This PR fixes it as well. ### Why are the changes needed? If we simply ignore filter predicate when we don't support it, the result is wrong. ### Does this PR introduce any user-facing change? yea, fix the query result. ### How was this patch tested? new tests Closes #27476 from cloud-fan/filter. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/analysis/Analyzer.scala | 4 ++++ .../sql/catalyst/analysis/CheckAnalysis.scala | 3 +++ .../analysis/AnalysisErrorSuite.scala | 20 +++++++++++++++++-- .../sql/catalyst/analysis/AnalysisSuite.scala | 10 +++++++++- .../resources/sql-tests/inputs/window.sql | 5 +++++ .../sql-tests/results/window.sql.out | 13 +++++++++++- 6 files changed, 51 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 56cc2a274bb7a..75f1aa7185ef3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2428,6 +2428,10 @@ class Analyzer( } wsc.copy(partitionSpec = newPartitionSpec, orderSpec = newOrderSpec) + case WindowExpression(ae: AggregateExpression, _) if ae.filter.isDefined => + failAnalysis( + "window aggregate function with filter predicate is not supported yet.") + // Extract Windowed AggregateExpression case we @ WindowExpression( ae @ AggregateExpression(function, _, _, _, _), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 4ec737fd9b70d..e769e038c960f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -308,6 +308,9 @@ trait CheckAnalysis extends PredicateHelper { case a: AggregateExpression if a.isDistinct => e.failAnalysis( "distinct aggregates are not allowed in observed metrics, but found: " + s.sql) + case a: AggregateExpression if a.filter.isDefined => + e.failAnalysis("aggregates with filter predicate are not allowed in " + + "observed metrics, but found: " + s.sql) case _: Attribute if !seenAggregate => e.failAnalysis (s"attribute ${s.sql} can only be used as an argument to an " + "aggregate function.") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 7023dbe2a3672..5cc0453135c07 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -164,6 +164,22 @@ class AnalysisErrorSuite extends AnalysisTest { UnspecifiedFrame)).as("window")), "Distinct window functions are not supported" :: Nil) + errorTest( + "window aggregate function with filter predicate", + testRelation2.select( + WindowExpression( + AggregateExpression( + Count(UnresolvedAttribute("b")), + Complete, + isDistinct = false, + filter = Some(UnresolvedAttribute("b") > 1)), + WindowSpecDefinition( + UnresolvedAttribute("a") :: Nil, + SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, + UnspecifiedFrame)).as("window")), + "window aggregate function with filter predicate is not supported" :: Nil + ) + errorTest( "distinct function", CatalystSqlParser.parsePlan("SELECT hex(DISTINCT a) FROM TaBlE"), @@ -191,12 +207,12 @@ class AnalysisErrorSuite extends AnalysisTest { "FILTER predicate specified, but aggregate is not an aggregate function" :: Nil) errorTest( - "DISTINCT and FILTER cannot be used in aggregate functions at the same time", + "DISTINCT aggregate function with filter predicate", CatalystSqlParser.parsePlan("SELECT count(DISTINCT a) FILTER (WHERE c > 1) FROM TaBlE2"), "DISTINCT and FILTER cannot be used in aggregate functions at the same time" :: Nil) errorTest( - "FILTER expression is non-deterministic, it cannot be used in aggregate functions", + "non-deterministic filter predicate in aggregate functions", CatalystSqlParser.parsePlan("SELECT count(a) FILTER (WHERE rand(int(c)) > 1) FROM TaBlE2"), "FILTER expression is non-deterministic, it cannot be used in aggregate functions" :: Nil) 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 5405009c9e208..c747d394b1bc2 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable 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.aggregate.{Count, Sum} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Count, Sum} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical._ @@ -736,5 +736,13 @@ class AnalysisSuite extends AnalysisTest with Matchers { b :: ScalarSubquery(subquery, Nil).as("sum") :: Nil, CollectMetrics("evt1", count :: Nil, tblB)) assertAnalysisError(query, "Multiple definitions of observed metrics" :: "evt1" :: Nil) + + // Aggregate with filter predicate - fail + val sumWithFilter = sum.transform { + case a: AggregateExpression => a.copy(filter = Some(true)) + }.asInstanceOf[NamedExpression] + assertAnalysisError( + CollectMetrics("evt1", sumWithFilter :: Nil, testRelation), + "aggregates with filter predicate are not allowed" :: Nil) } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index e25a252418301..3d05dfda6c3fa 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -120,3 +120,8 @@ SELECT cate, sum(val) OVER (w) FROM testData WHERE val is not null WINDOW w AS (PARTITION BY cate ORDER BY val); + +-- with filter predicate +SELECT val, cate, +count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate) +FROM testData ORDER BY cate, val; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index f795374735f59..625088f90ced9 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 23 +-- Number of queries: 24 -- !query @@ -380,3 +380,14 @@ a 4 b 1 b 3 b 6 + + +-- !query +SELECT val, cate, +count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate) +FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +window aggregate function with filter predicate is not supported yet.; From 4804445327f06ae3a26365d8f110f06ea07eb637 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 6 Feb 2020 14:58:53 -0800 Subject: [PATCH 1321/1387] [MINOR][DOC] Fix document UI left menu broken ### What changes were proposed in this pull request? Fix the left menu broken introduced in #25459. ### Why are the changes needed? The `left-menu-wrapper` CSS reused for both ml-guide and sql-programming-guide, the before changes will break the UI. Before: ![image](https://user-images.githubusercontent.com/4833765/73952563-1061d800-493a-11ea-8a75-d802a1534a44.png) ![image](https://user-images.githubusercontent.com/4833765/73952584-18217c80-493a-11ea-85a3-ce5f9875545f.png) ![image](https://user-images.githubusercontent.com/4833765/73952605-21124e00-493a-11ea-8d79-24f4dfec73d9.png) After: ![image](https://user-images.githubusercontent.com/4833765/73952630-2a031f80-493a-11ea-80ff-4630801cfaf4.png) ![image](https://user-images.githubusercontent.com/4833765/73952652-30919700-493a-11ea-9db1-8bb4a3f913b4.png) ![image](https://user-images.githubusercontent.com/4833765/73952671-35eee180-493a-11ea-801b-d50c4397adf2.png) ### Does this PR introduce any user-facing change? Document UI change only. ### How was this patch tested? Local test, screenshot attached below. Closes #27479 from xuanyuanking/doc-ui. Authored-by: Yuanjian Li Signed-off-by: Dongjoon Hyun --- docs/css/main.css | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/css/main.css b/docs/css/main.css index e24dff8531f24..dc05d287996be 100755 --- a/docs/css/main.css +++ b/docs/css/main.css @@ -211,8 +211,6 @@ a.anchorjs-link:hover { text-decoration: none; } float: left; position: fixed; overflow-y: scroll; - top: 0; - bottom: 0; } .left-menu { From dd2f4431f56e02cd06848b02b93b4cf34c97a5d5 Mon Sep 17 00:00:00 2001 From: sharif ahmad Date: Fri, 7 Feb 2020 18:42:16 +0900 Subject: [PATCH 1322/1387] [MINOR][DOCS] Fix typos at python/pyspark/sql/types.py ### What changes were proposed in this pull request? This PR fixes some typos in `python/pyspark/sql/types.py` file. ### Why are the changes needed? To deliver correct wording in documentation and codes. ### Does this PR introduce any user-facing change? Yes, it fixes some typos in user-facing API documentation. ### How was this patch tested? Locally tested the linter. Closes #27475 from sharifahmad2061/master. Lead-authored-by: sharif ahmad Co-authored-by: Sharif ahmad Signed-off-by: HyukjinKwon --- python/pyspark/sql/types.py | 40 ++++++++++++++++++------------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 8afff77b723a8..a5302e7bfd5ab 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -76,7 +76,7 @@ def json(self): def needConversion(self): """ - Does this type need to conversion between Python object and internal SQL object. + Does this type needs conversion between Python object and internal SQL object. This is used to avoid the unnecessary conversion for ArrayType/MapType/StructType. """ @@ -210,17 +210,17 @@ class DecimalType(FractionalType): The precision can be up to 38, the scale must be less or equal to precision. - When create a DecimalType, the default precision and scale is (10, 0). When infer + When creating a DecimalType, the default precision and scale is (10, 0). When inferring schema from decimal.Decimal objects, it will be DecimalType(38, 18). - :param precision: the maximum total number of digits (default: 10) + :param precision: the maximum (i.e. total) number of digits (default: 10) :param scale: the number of digits on right side of dot. (default: 0) """ def __init__(self, precision=10, scale=0): self.precision = precision self.scale = scale - self.hasPrecisionInfo = True # this is public API + self.hasPrecisionInfo = True # this is a public API def simpleString(self): return "decimal(%d,%d)" % (self.precision, self.scale) @@ -457,8 +457,8 @@ class StructType(DataType): This is the data type representing a :class:`Row`. - Iterating a :class:`StructType` will iterate its :class:`StructField`\\s. - A contained :class:`StructField` can be accessed by name or position. + Iterating a :class:`StructType` will iterate over its :class:`StructField`\\s. + A contained :class:`StructField` can be accessed by its name or position. >>> struct1 = StructType([StructField("f1", StringType(), True)]) >>> struct1["f1"] @@ -492,8 +492,8 @@ def __init__(self, fields=None): def add(self, field, data_type=None, nullable=True, metadata=None): """ - Construct a StructType by adding new elements to it to define the schema. The method accepts - either: + Construct a StructType by adding new elements to it, to define the schema. + The method accepts either: a) A single parameter which is a StructField object. b) Between 2 and 4 parameters as (name, data_type, nullable (optional), @@ -676,7 +676,7 @@ def needConversion(self): @classmethod def _cachedSqlType(cls): """ - Cache the sqlType() into class, because it's heavy used in `toInternal`. + Cache the sqlType() into class, because it's heavily used in `toInternal`. """ if not hasattr(cls, "_cached_sql_type"): cls._cached_sql_type = cls.sqlType() @@ -693,7 +693,7 @@ def fromInternal(self, obj): def serialize(self, obj): """ - Converts the a user-type object into a SQL datum. + Converts a user-type object into a SQL datum. """ raise NotImplementedError("UDT must implement toInternal().") @@ -760,7 +760,7 @@ def __eq__(self, other): def _parse_datatype_string(s): """ Parses the given data type string to a :class:`DataType`. The data type string format equals - to :class:`DataType.simpleString`, except that top level struct type can omit + :class:`DataType.simpleString`, except that the top level struct type can omit the ``struct<>`` and atomic types use ``typeName()`` as their format, e.g. use ``byte`` instead of ``tinyint`` for :class:`ByteType`. We can also use ``int`` as a short name for :class:`IntegerType`. Since Spark 2.3, this also supports a schema in a DDL-formatted @@ -921,7 +921,7 @@ def _parse_datatype_json_value(json_value): # We should be careful here. The size of these types in python depends on C # implementation. We need to make sure that this conversion does not lose any # precision. Also, JVM only support signed types, when converting unsigned types, -# keep in mind that it required 1 more bit when stored as singed types. +# keep in mind that it require 1 more bit when stored as signed types. # # Reference for C integer size, see: # ISO/IEC 9899:201x specification, chapter 5.2.4.2.1 Sizes of integer types . @@ -959,7 +959,7 @@ def _int_size_to_type(size): if size <= 64: return LongType -# The list of all supported array typecodes is stored here +# The list of all supported array typecodes, is stored here _array_type_mappings = { # Warning: Actual properties for float and double in C is not specified in C. # On almost every system supported by both python and JVM, they are IEEE 754 @@ -995,9 +995,9 @@ def _int_size_to_type(size): _array_type_mappings['c'] = StringType # SPARK-21465: -# In python2, array of 'L' happened to be mistakenly partially supported. To +# In python2, array of 'L' happened to be mistakenly, just partially supported. To # avoid breaking user's code, we should keep this partial support. Below is a -# dirty hacking to keep this partial support and make the unit test passes +# dirty hacking to keep this partial support and pass the unit test. import platform if sys.version_info[0] < 3 and platform.python_implementation() != 'PyPy': if 'L' not in _array_type_mappings.keys(): @@ -1071,7 +1071,7 @@ def _infer_schema(row, names=None): def _has_nulltype(dt): - """ Return whether there is NullType in `dt` or not """ + """ Return whether there is a NullType in `dt` or not """ if isinstance(dt, StructType): return any(_has_nulltype(f.dataType) for f in dt.fields) elif isinstance(dt, ArrayType): @@ -1211,7 +1211,7 @@ def _make_type_verifier(dataType, nullable=True, name=None): This verifier also checks the value of obj against datatype and raises a ValueError if it's not within the allowed range, e.g. using 128 as ByteType will overflow. Note that, Python float is - not checked, so it will become infinity when cast to Java float if it overflows. + not checked, so it will become infinity when cast to Java float, if it overflows. >>> _make_type_verifier(StructType([]))(None) >>> _make_type_verifier(StringType())("") @@ -1433,7 +1433,7 @@ class Row(tuple): ``key in row`` will search through row keys. Row can be used to create a row object by using named arguments. - It is not allowed to omit a named argument to represent the value is + It is not allowed to omit a named argument to represent that the value is None or missing. This should be explicitly set to None in this case. NOTE: As of Spark 3.0.0, Rows created from named arguments no longer have @@ -1524,9 +1524,9 @@ def __new__(cls, *args, **kwargs): def asDict(self, recursive=False): """ - Return as an dict + Return as a dict - :param recursive: turns the nested Row as dict (default: False). + :param recursive: turns the nested Rows to dict (default: False). >>> Row(name="Alice", age=11).asDict() == {'name': 'Alice', 'age': 11} True From a3e77773cfa03a18d31370acd9a10562ff5312bb Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 8 Feb 2020 02:32:07 +0800 Subject: [PATCH 1323/1387] [SPARK-30752][SQL] Fix `to_utc_timestamp` on daylight saving day ### What changes were proposed in this pull request? - Rewrite the `convertTz` method of `DateTimeUtils` using Java 8 time API - Change types of `convertTz` parameters from `TimeZone` to `ZoneId`. This allows to avoid unnecessary conversions `TimeZone` -> `ZoneId` and performance regressions as a consequence. ### Why are the changes needed? - Fixes incorrect behavior of `to_utc_timestamp` on daylight saving day. For example: ```scala scala> df.select(to_utc_timestamp(lit("2019-11-03T12:00:00"), "Asia/Hong_Kong").as("local UTC")).show +-------------------+ | local UTC| +-------------------+ |2019-11-03 03:00:00| +-------------------+ ``` but the result must be 2019-11-03 04:00:00: Screen Shot 2020-02-06 at 20 09 36 - Simplifies the code, and make it more maintainable - Switches `convertTz` on Proleptic Gregorian calendar used by Java 8 time classes by default. That makes the function consistent to other date-time functions. ### Does this PR introduce any user-facing change? Yes, after the changes `to_utc_timestamp` returns the correct result `2019-11-03 04:00:00`. ### How was this patch tested? - By existing test suite `DateTimeUtilsSuite`, `DateFunctionsSuite` and `DateExpressionsSuite`. - Added `convert time zones on a daylight saving day` to DateFunctionsSuite Closes #27474 from MaxGekk/port-convertTz-on-Java8-api. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../expressions/datetimeExpressions.scala | 16 ++++------ .../sql/catalyst/util/DateTimeUtils.scala | 28 ++++-------------- .../parquet/VectorizedColumnReader.java | 9 +++--- .../VectorizedParquetRecordReader.java | 6 ++-- .../parquet/ParquetFileFormat.scala | 2 +- .../parquet/ParquetReadSupport.scala | 5 ++-- .../parquet/ParquetRecordMaterializer.scala | 4 +-- .../parquet/ParquetRowConverter.scala | 9 +++--- .../ParquetPartitionReaderFactory.scala | 10 +++---- .../apache/spark/sql/DateFunctionsSuite.scala | 29 +++++++++++++++++-- .../ParquetInteroperabilitySuite.scala | 5 ++-- 11 files changed, 64 insertions(+), 59 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 28f1d34267224..aa2bd5a1273e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1176,14 +1176,12 @@ case class FromUTCTimestamp(left: Expression, right: Expression) |long ${ev.value} = 0; """.stripMargin) } else { - val tzClass = classOf[TimeZone].getName + val tzClass = classOf[ZoneId].getName val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") val escapedTz = StringEscapeUtils.escapeJava(tz.toString) val tzTerm = ctx.addMutableState(tzClass, "tz", - v => s"""$v = $dtu.getTimeZone("$escapedTz");""") - val utcTerm = "tzUTC" - ctx.addImmutableStateIfNotExists(tzClass, utcTerm, - v => s"""$v = $dtu.getTimeZone("UTC");""") + v => s"""$v = $dtu.getZoneId("$escapedTz");""") + val utcTerm = "java.time.ZoneOffset.UTC" val eval = left.genCode(ctx) ev.copy(code = code""" |${eval.code} @@ -1382,14 +1380,12 @@ case class ToUTCTimestamp(left: Expression, right: Expression) |long ${ev.value} = 0; """.stripMargin) } else { - val tzClass = classOf[TimeZone].getName + val tzClass = classOf[ZoneId].getName val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") val escapedTz = StringEscapeUtils.escapeJava(tz.toString) val tzTerm = ctx.addMutableState(tzClass, "tz", - v => s"""$v = $dtu.getTimeZone("$escapedTz");""") - val utcTerm = "tzUTC" - ctx.addImmutableStateIfNotExists(tzClass, utcTerm, - v => s"""$v = $dtu.getTimeZone("UTC");""") + v => s"""$v = $dtu.getZoneId("$escapedTz");""") + val utcTerm = "java.time.ZoneOffset.UTC" val eval = left.genCode(ctx) ev.copy(code = code""" |${eval.code} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 6800abb2ae109..8eb560944d4cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -801,27 +801,9 @@ object DateTimeUtils { * mapping, the conversion here may return wrong result, we should make the timestamp * timezone-aware. */ - def convertTz(ts: SQLTimestamp, fromZone: TimeZone, toZone: TimeZone): SQLTimestamp = { - // We always use local timezone to parse or format a timestamp - val localZone = defaultTimeZone() - val utcTs = if (fromZone.getID == localZone.getID) { - ts - } else { - // get the human time using local time zone, that actually is in fromZone. - val localZoneOffsetMs = localZone.getOffset(MICROSECONDS.toMillis(ts)) - val localTsUs = ts + MILLISECONDS.toMicros(localZoneOffsetMs) // in fromZone - val offsetFromLocalMs = getOffsetFromLocalMillis(MICROSECONDS.toMillis(localTsUs), fromZone) - localTsUs - MILLISECONDS.toMicros(offsetFromLocalMs) - } - if (toZone.getID == localZone.getID) { - utcTs - } else { - val toZoneOffsetMs = toZone.getOffset(MICROSECONDS.toMillis(utcTs)) - val localTsUs = utcTs + MILLISECONDS.toMicros(toZoneOffsetMs) // in toZone - // treat it as local timezone, convert to UTC (we could get the expected human time back) - val offsetFromLocalMs = getOffsetFromLocalMillis(MICROSECONDS.toMillis(localTsUs), localZone) - localTsUs - MILLISECONDS.toMicros(offsetFromLocalMs) - } + def convertTz(ts: SQLTimestamp, fromZone: ZoneId, toZone: ZoneId): SQLTimestamp = { + val rebasedDateTime = microsToInstant(ts).atZone(toZone).toLocalDateTime.atZone(fromZone) + instantToMicros(rebasedDateTime.toInstant) } /** @@ -829,7 +811,7 @@ object DateTimeUtils { * representation in their timezone. */ def fromUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { - convertTz(time, TimeZoneGMT, getTimeZone(timeZone)) + convertTz(time, ZoneOffset.UTC, getZoneId(timeZone)) } /** @@ -837,7 +819,7 @@ object DateTimeUtils { * string representation in their timezone. */ def toUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { - convertTz(time, getTimeZone(timeZone), TimeZoneGMT) + convertTz(time, getZoneId(timeZone), ZoneOffset.UTC) } /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index ba26b57567e64..329465544979d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -18,8 +18,9 @@ package org.apache.spark.sql.execution.datasources.parquet; import java.io.IOException; +import java.time.ZoneId; +import java.time.ZoneOffset; import java.util.Arrays; -import java.util.TimeZone; import org.apache.parquet.bytes.ByteBufferInputStream; import org.apache.parquet.bytes.BytesInput; @@ -98,14 +99,14 @@ public class VectorizedColumnReader { private final ColumnDescriptor descriptor; private final OriginalType originalType; // The timezone conversion to apply to int96 timestamps. Null if no conversion. - private final TimeZone convertTz; - private static final TimeZone UTC = DateTimeUtils.TimeZoneUTC(); + private final ZoneId convertTz; + private static final ZoneId UTC = ZoneOffset.UTC; public VectorizedColumnReader( ColumnDescriptor descriptor, OriginalType originalType, PageReader pageReader, - TimeZone convertTz) throws IOException { + ZoneId convertTz) throws IOException { this.descriptor = descriptor; this.pageReader = pageReader; this.convertTz = convertTz; diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java index f02861355c404..7306709a79c34 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources.parquet; import java.io.IOException; +import java.time.ZoneId; import java.util.Arrays; import java.util.List; -import java.util.TimeZone; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -86,7 +86,7 @@ public class VectorizedParquetRecordReader extends SpecificParquetRecordReaderBa * The timezone that timestamp INT96 values should be converted to. Null if no conversion. Here to * workaround incompatibilities between different engines when writing timestamp values. */ - private TimeZone convertTz = null; + private ZoneId convertTz = null; /** * columnBatch object that is used for batch decoding. This is created on first use and triggers @@ -116,7 +116,7 @@ public class VectorizedParquetRecordReader extends SpecificParquetRecordReaderBa */ private final MemoryMode MEMORY_MODE; - public VectorizedParquetRecordReader(TimeZone convertTz, boolean useOffHeap, int capacity) { + public VectorizedParquetRecordReader(ZoneId convertTz, boolean useOffHeap, int capacity) { this.convertTz = convertTz; MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP; this.capacity = capacity; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index f52aaf0140e1d..29dbd8dfbca8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -295,7 +295,7 @@ class ParquetFileFormat val convertTz = if (timestampConversion && !isCreatedByParquetMr) { - Some(DateTimeUtils.getTimeZone(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + Some(DateTimeUtils.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) } else { None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 69c8bad5f1c83..c05ecf16311ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.util.{Locale, Map => JMap, TimeZone} +import java.time.ZoneId +import java.util.{Locale, Map => JMap} import scala.collection.JavaConverters._ @@ -49,7 +50,7 @@ import org.apache.spark.sql.types._ * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]] * to [[prepareForRead()]], but use a private `var` for simplicity. */ -class ParquetReadSupport(val convertTz: Option[TimeZone], +class ParquetReadSupport(val convertTz: Option[ZoneId], enableVectorizedReader: Boolean) extends ReadSupport[InternalRow] with Logging { private var catalystRequestedSchema: StructType = _ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala index 3098a332d3027..5622169df1281 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.util.TimeZone +import java.time.ZoneId import org.apache.parquet.io.api.{GroupConverter, RecordMaterializer} import org.apache.parquet.schema.MessageType @@ -36,7 +36,7 @@ private[parquet] class ParquetRecordMaterializer( parquetSchema: MessageType, catalystSchema: StructType, schemaConverter: ParquetToSparkSchemaConverter, - convertTz: Option[TimeZone]) + convertTz: Option[ZoneId]) extends RecordMaterializer[InternalRow] { private val rootConverter = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 98ac2ecd2955c..850adae8a6b95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.math.{BigDecimal, BigInteger} import java.nio.ByteOrder -import java.util.TimeZone +import java.time.{ZoneId, ZoneOffset} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -125,7 +125,7 @@ private[parquet] class ParquetRowConverter( schemaConverter: ParquetToSparkSchemaConverter, parquetType: GroupType, catalystType: StructType, - convertTz: Option[TimeZone], + convertTz: Option[ZoneId], updater: ParentContainerUpdater) extends ParquetGroupConverter(updater) with Logging { @@ -154,8 +154,6 @@ private[parquet] class ParquetRowConverter( |${catalystType.prettyJson} """.stripMargin) - private[this] val UTC = DateTimeUtils.TimeZoneUTC - /** * Updater used together with field converters within a [[ParquetRowConverter]]. It propagates * converted filed values to the `ordinal`-th cell in `currentRow`. @@ -292,7 +290,8 @@ private[parquet] class ParquetRowConverter( val timeOfDayNanos = buf.getLong val julianDay = buf.getInt val rawTime = DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) - val adjTime = convertTz.map(DateTimeUtils.convertTz(rawTime, _, UTC)).getOrElse(rawTime) + val adjTime = convertTz.map(DateTimeUtils.convertTz(rawTime, _, ZoneOffset.UTC)) + .getOrElse(rawTime) updater.setLong(adjTime) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala index b2fc724057eba..047bc74a8d81e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.v2.parquet import java.net.URI -import java.util.TimeZone +import java.time.ZoneId import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ @@ -117,7 +117,7 @@ case class ParquetPartitionReaderFactory( file: PartitionedFile, buildReaderFunc: ( ParquetInputSplit, InternalRow, TaskAttemptContextImpl, Option[FilterPredicate], - Option[TimeZone]) => RecordReader[Void, T]): RecordReader[Void, T] = { + Option[ZoneId]) => RecordReader[Void, T]): RecordReader[Void, T] = { val conf = broadcastedConf.value.value val filePath = new Path(new URI(file.filePath)) @@ -156,7 +156,7 @@ case class ParquetPartitionReaderFactory( val convertTz = if (timestampConversion && !isCreatedByParquetMr) { - Some(DateTimeUtils.getTimeZone(conf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + Some(DateTimeUtils.getZoneId(conf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) } else { None } @@ -184,7 +184,7 @@ case class ParquetPartitionReaderFactory( partitionValues: InternalRow, hadoopAttemptContext: TaskAttemptContextImpl, pushed: Option[FilterPredicate], - convertTz: Option[TimeZone]): RecordReader[Void, InternalRow] = { + convertTz: Option[ZoneId]): RecordReader[Void, InternalRow] = { logDebug(s"Falling back to parquet-mr") val taskContext = Option(TaskContext.get()) // ParquetRecordReader returns InternalRow @@ -213,7 +213,7 @@ case class ParquetPartitionReaderFactory( partitionValues: InternalRow, hadoopAttemptContext: TaskAttemptContextImpl, pushed: Option[FilterPredicate], - convertTz: Option[TimeZone]): VectorizedParquetRecordReader = { + convertTz: Option[ZoneId]): VectorizedParquetRecordReader = { val taskContext = Option(TaskContext.get()) val vectorizedReader = new VectorizedParquetRecordReader( convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 3b3d3cc3d7a17..bb8cdf3cb6de1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.Instant -import java.util.Locale +import java.time.{Instant, LocalDateTime} +import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils} @@ -803,4 +803,29 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { checkTimeZoneParsing(null) } } + + test("SPARK-30752: convert time zones on a daylight saving day") { + val systemTz = "PST" + val sessionTz = "UTC" + val fromTz = "Asia/Hong_Kong" + val fromTs = "2019-11-03T12:00:00" // daylight saving date in PST + val utsTs = "2019-11-03T04:00:00" + val defaultTz = TimeZone.getDefault + try { + TimeZone.setDefault(DateTimeUtils.getTimeZone(systemTz)) + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true", + SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionTz) { + val expected = LocalDateTime.parse(utsTs) + .atZone(DateTimeUtils.getZoneId(sessionTz)) + .toInstant + val df = Seq(fromTs).toDF("localTs") + checkAnswer( + df.select(to_utc_timestamp(col("localTs"), fromTz)), + Row(expected)) + } + } finally { + TimeZone.setDefault(defaultTz) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index 1ded34f24e436..649a46f190580 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File +import java.time.ZoneOffset import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} @@ -145,8 +146,8 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS impalaFileData.map { ts => DateTimeUtils.toJavaTimestamp(DateTimeUtils.convertTz( DateTimeUtils.fromJavaTimestamp(ts), - DateTimeUtils.TimeZoneUTC, - DateTimeUtils.getTimeZone(conf.sessionLocalTimeZone))) + ZoneOffset.UTC, + DateTimeUtils.getZoneId(conf.sessionLocalTimeZone))) } } val fullExpectations = (ts ++ impalaExpectations).map(_.toString).sorted.toArray From a7451f44d234a946668deb99bed8fdbd4b8ebf8e Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Sat, 8 Feb 2020 02:47:44 +0800 Subject: [PATCH 1324/1387] [SPARK-30614][SQL] The native ALTER COLUMN syntax should change one property at a time ### What changes were proposed in this pull request? The current ALTER COLUMN syntax allows to change multiple properties at a time: ``` ALTER TABLE table=multipartIdentifier (ALTER | CHANGE) COLUMN? column=multipartIdentifier (TYPE dataType)? (COMMENT comment=STRING)? colPosition? ``` The SQL standard (section 11.12) only allows changing one property at a time. This is also true on other recent SQL systems like [snowflake](https://docs.snowflake.net/manuals/sql-reference/sql/alter-table-column.html) and [redshift](https://docs.aws.amazon.com/redshift/latest/dg/r_ALTER_TABLE.html). (credit to cloud-fan) This PR proposes to change ALTER COLUMN to follow SQL standard, thus allows altering only one column property at a time. Note that ALTER COLUMN syntax being changed here is newly added in Spark 3.0, so it doesn't affect Spark 2.4 behavior. ### Why are the changes needed? To follow SQL standard (and other recent SQL systems) behavior. ### Does this PR introduce any user-facing change? Yes, now the user can update the column properties only one at a time. For example, ``` ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment' ``` should be broken into ``` ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment' ``` ### How was this patch tested? Updated existing tests. Closes #27444 from imback82/alter_column_one_at_a_time. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 14 ++-- .../sql/catalyst/parser/AstBuilder.scala | 78 ++++++++++--------- .../sql/catalyst/parser/DDLParserSuite.scala | 23 +++--- .../analysis/ResolveSessionCatalog.scala | 24 +++--- .../sql-tests/inputs/change-column.sql | 21 +++-- .../sql-tests/results/change-column.sql.out | 46 ++++++++--- .../spark/sql/connector/AlterTableTests.scala | 13 ---- .../sql/execution/command/DDLSuite.scala | 5 +- .../command/PlanResolutionSuite.scala | 47 ++++++++--- 9 files changed, 165 insertions(+), 106 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 08d5ff53bf2e2..563ef69b3b8ae 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -158,12 +158,9 @@ statement SET TBLPROPERTIES tablePropertyList #setTableProperties | ALTER (TABLE | VIEW) multipartIdentifier UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties - | ALTER TABLE table=multipartIdentifier + |ALTER TABLE table=multipartIdentifier (ALTER | CHANGE) COLUMN? column=multipartIdentifier - (TYPE dataType)? commentSpec? colPosition? #alterTableColumn - | ALTER TABLE table=multipartIdentifier - ALTER COLUMN? column=multipartIdentifier - setOrDrop=(SET | DROP) NOT NULL #alterColumnNullability + alterColumnAction? #alterTableAlterColumn | ALTER TABLE table=multipartIdentifier partitionSpec? CHANGE COLUMN? colName=multipartIdentifier colType colPosition? #hiveChangeColumn @@ -983,6 +980,13 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; +alterColumnAction + : TYPE dataType + | commentSpec + | colPosition + | setOrDrop=(SET | DROP) NOT NULL + ; + // When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index e9ad84472904d..6fc65e14868e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2940,55 +2940,61 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parse a [[AlterTableAlterColumnStatement]] command. + * Parse a [[AlterTableAlterColumnStatement]] command to alter a column's property. * * For example: * {{{ * ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint - * ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment' + * ALTER TABLE table1 ALTER COLUMN a.b.c SET NOT NULL + * ALTER TABLE table1 ALTER COLUMN a.b.c DROP NOT NULL * ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment' + * ALTER TABLE table1 ALTER COLUMN a.b.c FIRST + * ALTER TABLE table1 ALTER COLUMN a.b.c AFTER x * }}} */ - override def visitAlterTableColumn( - ctx: AlterTableColumnContext): LogicalPlan = withOrigin(ctx) { - val verb = if (ctx.CHANGE != null) "CHANGE" else "ALTER" - if (ctx.dataType == null && ctx.commentSpec() == null && ctx.colPosition == null) { + override def visitAlterTableAlterColumn( + ctx: AlterTableAlterColumnContext): LogicalPlan = withOrigin(ctx) { + val action = ctx.alterColumnAction + if (action == null) { + val verb = if (ctx.CHANGE != null) "CHANGE" else "ALTER" operationNotAllowed( - s"ALTER TABLE table $verb COLUMN requires a TYPE or a COMMENT or a FIRST/AFTER", ctx) + s"ALTER TABLE table $verb COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER", + ctx) + } + + val dataType = if (action.dataType != null) { + Some(typedVisit[DataType](action.dataType)) + } else { + None + } + val nullable = if (action.setOrDrop != null) { + action.setOrDrop.getType match { + case SqlBaseParser.SET => Some(false) + case SqlBaseParser.DROP => Some(true) + } + } else { + None + } + val comment = if (action.commentSpec != null) { + Some(visitCommentSpec(action.commentSpec())) + } else { + None + } + val position = if (action.colPosition != null) { + Some(typedVisit[ColumnPosition](action.colPosition)) + } else { + None } + assert(Seq(dataType, nullable, comment, position).count(_.nonEmpty) == 1) + AlterTableAlterColumnStatement( visitMultipartIdentifier(ctx.table), typedVisit[Seq[String]](ctx.column), - dataType = Option(ctx.dataType).map(typedVisit[DataType]), - nullable = None, - comment = Option(ctx.commentSpec()).map(visitCommentSpec), - position = Option(ctx.colPosition).map(typedVisit[ColumnPosition])) - } - - /** - * Parse a [[AlterTableAlterColumnStatement]] command to change column nullability. - * - * For example: - * {{{ - * ALTER TABLE table1 ALTER COLUMN a.b.c SET NOT NULL - * ALTER TABLE table1 ALTER COLUMN a.b.c DROP NOT NULL - * }}} - */ - override def visitAlterColumnNullability(ctx: AlterColumnNullabilityContext): LogicalPlan = { - withOrigin(ctx) { - val nullable = ctx.setOrDrop.getType match { - case SqlBaseParser.SET => false - case SqlBaseParser.DROP => true - } - AlterTableAlterColumnStatement( - visitMultipartIdentifier(ctx.table), - typedVisit[Seq[String]](ctx.column), - dataType = None, - nullable = Some(nullable), - comment = None, - position = None) - } + dataType = dataType, + nullable = nullable, + comment = comment, + position = position) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 56d52571d1cc3..bc7b51f25b20d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -646,17 +646,18 @@ class DDLParserSuite extends AnalysisTest { Some(first()))) } - test("alter table: update column type, comment and position") { - comparePlans( - parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c " + - "TYPE bigint COMMENT 'new comment' AFTER d"), - AlterTableAlterColumnStatement( - Seq("table_name"), - Seq("a", "b", "c"), - Some(LongType), - None, - Some("new comment"), - Some(after("d")))) + test("alter table: mutiple property changes are not allowed") { + intercept[ParseException] { + parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c " + + "TYPE bigint COMMENT 'new comment'")} + + intercept[ParseException] { + parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c " + + "TYPE bigint COMMENT AFTER d")} + + intercept[ParseException] { + parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c " + + "TYPE bigint COMMENT 'new comment' AFTER d")} } test("alter table: SET/DROP NOT NULL") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 486e7f1f84b46..77d549c28aae5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -77,10 +77,6 @@ class ResolveSessionCatalog( throw new AnalysisException( "ALTER COLUMN with qualified column is only supported with v2 tables.") } - if (a.dataType.isEmpty) { - throw new AnalysisException( - "ALTER COLUMN with v1 tables must specify new data type.") - } if (a.nullable.isDefined) { throw new AnalysisException( "ALTER COLUMN with v1 tables cannot specify NOT NULL.") @@ -92,17 +88,27 @@ class ResolveSessionCatalog( val builder = new MetadataBuilder // Add comment to metadata a.comment.map(c => builder.putString("comment", c)) + val colName = a.column(0) + val dataType = a.dataType.getOrElse { + v1Table.schema.findNestedField(Seq(colName), resolver = conf.resolver) + .map(_._2.dataType) + .getOrElse { + throw new AnalysisException( + s"ALTER COLUMN cannot find column ${quote(colName)} in v1 table. " + + s"Available: ${v1Table.schema.fieldNames.mkString(", ")}") + } + } // Add Hive type string to metadata. - val cleanedDataType = HiveStringType.replaceCharType(a.dataType.get) - if (a.dataType.get != cleanedDataType) { - builder.putString(HIVE_TYPE_STRING, a.dataType.get.catalogString) + val cleanedDataType = HiveStringType.replaceCharType(dataType) + if (dataType != cleanedDataType) { + builder.putString(HIVE_TYPE_STRING, dataType.catalogString) } val newColumn = StructField( - a.column(0), + colName, cleanedDataType, nullable = true, builder.build()) - AlterTableChangeColumnCommand(tbl.asTableIdentifier, a.column(0), newColumn) + AlterTableChangeColumnCommand(tbl.asTableIdentifier, colName, newColumn) }.getOrElse { val colName = a.column.toArray val typeChange = a.dataType.map { newDataType => diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index dd2fc660b53e3..2b57891cfcbc5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -15,29 +15,34 @@ ALTER TABLE test_change CHANGE a TYPE STRING; DESC test_change; -- Change column position (not supported yet) -ALTER TABLE test_change CHANGE a TYPE INT AFTER b; -ALTER TABLE test_change CHANGE b TYPE STRING FIRST; +ALTER TABLE test_change CHANGE a AFTER b; +ALTER TABLE test_change CHANGE b FIRST; DESC test_change; -- Change column comment -ALTER TABLE test_change CHANGE a TYPE INT COMMENT 'this is column a'; -ALTER TABLE test_change CHANGE b TYPE STRING COMMENT '#*02?`'; -ALTER TABLE test_change CHANGE c TYPE INT COMMENT ''; +ALTER TABLE test_change CHANGE a COMMENT 'this is column a'; +ALTER TABLE test_change CHANGE b COMMENT '#*02?`'; +ALTER TABLE test_change CHANGE c COMMENT ''; DESC test_change; -- Don't change anything. -ALTER TABLE test_change CHANGE a TYPE INT COMMENT 'this is column a'; +ALTER TABLE test_change CHANGE a TYPE INT; +ALTER TABLE test_change CHANGE a COMMENT 'this is column a'; DESC test_change; -- Change a invalid column ALTER TABLE test_change CHANGE invalid_col TYPE INT; DESC test_change; +-- Check case insensitivity. +ALTER TABLE test_change CHANGE A COMMENT 'case insensitivity'; +DESC test_change; + -- Change column can't apply to a temporary/global_temporary view CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one"; -ALTER TABLE temp_view CHANGE a TYPE INT COMMENT 'this is column a'; +ALTER TABLE temp_view CHANGE a TYPE INT; CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"; -ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT COMMENT 'this is column a'; +ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT; -- DROP TEST TABLE DROP TABLE test_change; diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index 5bb00e028c4b7..b1a32ad1f63e9 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 25 +-- Number of queries: 28 -- !query @@ -27,7 +27,7 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -Operation not allowed: ALTER TABLE table CHANGE COLUMN requires a TYPE or a COMMENT or a FIRST/AFTER(line 1, pos 0) +Operation not allowed: ALTER TABLE table CHANGE COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER(line 1, pos 0) == SQL == ALTER TABLE test_change CHANGE a @@ -83,7 +83,7 @@ c int -- !query -ALTER TABLE test_change CHANGE a TYPE INT AFTER b +ALTER TABLE test_change CHANGE a AFTER b -- !query schema struct<> -- !query output @@ -92,7 +92,7 @@ ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.; -- !query -ALTER TABLE test_change CHANGE b TYPE STRING FIRST +ALTER TABLE test_change CHANGE b FIRST -- !query schema struct<> -- !query output @@ -111,7 +111,7 @@ c int -- !query -ALTER TABLE test_change CHANGE a TYPE INT COMMENT 'this is column a' +ALTER TABLE test_change CHANGE a COMMENT 'this is column a' -- !query schema struct<> -- !query output @@ -119,7 +119,7 @@ struct<> -- !query -ALTER TABLE test_change CHANGE b TYPE STRING COMMENT '#*02?`' +ALTER TABLE test_change CHANGE b COMMENT '#*02?`' -- !query schema struct<> -- !query output @@ -127,7 +127,7 @@ struct<> -- !query -ALTER TABLE test_change CHANGE c TYPE INT COMMENT '' +ALTER TABLE test_change CHANGE c COMMENT '' -- !query schema struct<> -- !query output @@ -145,7 +145,15 @@ c int -- !query -ALTER TABLE test_change CHANGE a TYPE INT COMMENT 'this is column a' +ALTER TABLE test_change CHANGE a TYPE INT +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE test_change CHANGE a COMMENT 'this is column a' -- !query schema struct<> -- !query output @@ -181,6 +189,24 @@ b string #*02?` c int +-- !query +ALTER TABLE test_change CHANGE A COMMENT 'case insensitivity' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int case insensitivity +b string #*02?` +c int + + -- !query CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" -- !query schema @@ -190,7 +216,7 @@ struct<> -- !query -ALTER TABLE temp_view CHANGE a TYPE INT COMMENT 'this is column a' +ALTER TABLE temp_view CHANGE a TYPE INT -- !query schema struct<> -- !query output @@ -207,7 +233,7 @@ struct<> -- !query -ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT COMMENT 'this is column a' +ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT -- !query schema struct<> -- !query output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 3cdac59c20fc9..420cb01d766a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -651,19 +651,6 @@ trait AlterTableTests extends SharedSparkSession { } } - test("AlterTable: update column type and comment") { - val t = s"${catalogAndNamespace}table_name" - withTable(t) { - sql(s"CREATE TABLE $t (id int) USING $v2Format") - sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint COMMENT 'doc'") - - val table = getTableMetadata(t) - - assert(table.name === fullTableName(t)) - assert(table.schema === StructType(Seq(StructField("id", LongType).withComment("doc")))) - } - } - test("AlterTable: update nested column comment") { val t = s"${catalogAndNamespace}table_name" withTable(t) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 913cd80a24c6e..31e00781ae6b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -188,7 +188,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession { withTable("t") { sql("CREATE TABLE t(i INT) USING parquet") val e = intercept[AnalysisException] { - sql("ALTER TABLE t ALTER COLUMN i TYPE INT FIRST") + sql("ALTER TABLE t ALTER COLUMN i FIRST") } assert(e.message.contains("ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables")) } @@ -1786,7 +1786,8 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { column.map(_.metadata).getOrElse(Metadata.empty) } // Ensure that change column will preserve other metadata fields. - sql("ALTER TABLE dbx.tab1 CHANGE COLUMN col1 TYPE INT COMMENT 'this is col1'") + sql("ALTER TABLE dbx.tab1 CHANGE COLUMN col1 TYPE INT") + sql("ALTER TABLE dbx.tab1 CHANGE COLUMN col1 COMMENT 'this is col1'") assert(getMetadata("col1").getString("key") == "value") assert(getMetadata("col1").getString("comment") == "this is col1") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 88f30353cce94..d439e5b1cd651 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -1013,27 +1013,29 @@ class PlanResolutionSuite extends AnalysisTest { Seq("v1Table" -> true, "v2Table" -> false, "testcat.tab" -> false).foreach { case (tblName, useV1Command) => val sql1 = s"ALTER TABLE $tblName ALTER COLUMN i TYPE bigint" - val sql2 = s"ALTER TABLE $tblName ALTER COLUMN i TYPE bigint COMMENT 'new comment'" - val sql3 = s"ALTER TABLE $tblName ALTER COLUMN i COMMENT 'new comment'" + val sql2 = s"ALTER TABLE $tblName ALTER COLUMN i COMMENT 'new comment'" val parsed1 = parseAndResolve(sql1) val parsed2 = parseAndResolve(sql2) val tableIdent = TableIdentifier(tblName, None) if (useV1Command) { + val oldColumn = StructField("i", IntegerType) val newColumn = StructField("i", LongType) val expected1 = AlterTableChangeColumnCommand( tableIdent, "i", newColumn) val expected2 = AlterTableChangeColumnCommand( - tableIdent, "i", newColumn.withComment("new comment")) + tableIdent, "i", oldColumn.withComment("new comment")) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) + val sql3 = s"ALTER TABLE $tblName ALTER COLUMN j COMMENT 'new comment'" val e1 = intercept[AnalysisException] { parseAndResolve(sql3) } - assert(e1.getMessage.contains("ALTER COLUMN with v1 tables must specify new data type")) + assert(e1.getMessage.contains( + "ALTER COLUMN cannot find column j in v1 table. Available: i, s")) val sql4 = s"ALTER TABLE $tblName ALTER COLUMN a.b.c TYPE bigint" val e2 = intercept[AnalysisException] { @@ -1051,8 +1053,6 @@ class PlanResolutionSuite extends AnalysisTest { val parsed5 = parseAndResolve(sql5) comparePlans(parsed5, expected5) } else { - val parsed3 = parseAndResolve(sql3) - parsed1 match { case AlterTable(_, _, _: DataSourceV2Relation, changes) => assert(changes == Seq( @@ -1063,18 +1063,41 @@ class PlanResolutionSuite extends AnalysisTest { parsed2 match { case AlterTable(_, _, _: DataSourceV2Relation, changes) => assert(changes == Seq( - TableChange.updateColumnType(Array("i"), LongType), TableChange.updateColumnComment(Array("i"), "new comment"))) case _ => fail("expect AlterTable") } + } + } + } - parsed3 match { - case AlterTable(_, _, _: DataSourceV2Relation, changes) => - assert(changes == Seq( - TableChange.updateColumnComment(Array("i"), "new comment"))) - case _ => fail("expect AlterTable") + test("alter table: alter column action is not specified") { + val e = intercept[AnalysisException] { + parseAndResolve("ALTER TABLE v1Table ALTER COLUMN i") + } + assert(e.getMessage.contains( + "ALTER TABLE table ALTER COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER")) + } + + test("alter table: alter column case sensitivity for v1 table") { + val tblName = "v1Table" + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val sql = s"ALTER TABLE $tblName ALTER COLUMN I COMMENT 'new comment'" + if (caseSensitive) { + val e = intercept[AnalysisException] { + parseAndResolve(sql) } + assert(e.getMessage.contains( + "ALTER COLUMN cannot find column I in v1 table. Available: i, s")) + } else { + val actual = parseAndResolve(sql) + val expected = AlterTableChangeColumnCommand( + TableIdentifier(tblName, None), + "I", + StructField("I", IntegerType).withComment("new comment")) + comparePlans(actual, expected) } + } } } From 12e1bbaddbb2ef304b5880a62df6683fcc94ea54 Mon Sep 17 00:00:00 2001 From: zhengruifeng Date: Sat, 8 Feb 2020 08:46:16 +0800 Subject: [PATCH 1325/1387] Revert "[SPARK-30642][SPARK-30659][SPARK-30660][SPARK-30662]" ### What changes were proposed in this pull request? Revert #27360 #27396 #27374 #27389 ### Why are the changes needed? BLAS need more performace tests, specially on sparse datasets. Perfermance test of LogisticRegression (https://github.com/apache/spark/pull/27374) on sparse dataset shows that blockify vectors to matrices and use BLAS will cause performance regression. LinearSVC and LinearRegression were also updated in the same way as LogisticRegression, so we need to revert them to make sure no regression. ### Does this PR introduce any user-facing change? remove newly added param blockSize ### How was this patch tested? reverted testsuites Closes #27487 from zhengruifeng/revert_blockify_ii. Authored-by: zhengruifeng Signed-off-by: zhengruifeng --- .../spark/serializer/KryoSerializer.scala | 1 - .../org/apache/spark/ml/linalg/BLAS.scala | 1 + .../spark/ml/classification/LinearSVC.scala | 51 ++-- .../classification/LogisticRegression.scala | 58 ++-- .../MultilayerPerceptronClassifier.scala | 22 +- .../apache/spark/ml/feature/Instance.scala | 130 +-------- .../ml/optim/aggregator/HingeAggregator.scala | 102 +------ .../ml/optim/aggregator/HuberAggregator.scala | 103 ++----- .../aggregator/LeastSquaresAggregator.scala | 74 ++--- .../optim/aggregator/LogisticAggregator.scala | 252 ++---------------- .../ml/param/shared/SharedParamsCodeGen.scala | 6 +- .../spark/ml/param/shared/sharedParams.scala | 23 +- .../apache/spark/ml/recommendation/ALS.scala | 46 +--- .../ml/regression/LinearRegression.scala | 49 ++-- .../classification/LogisticRegression.scala | 4 +- .../ml/classification/LinearSVCSuite.scala | 2 +- .../LogisticRegressionSuite.scala | 4 +- .../spark/ml/feature/InstanceSuite.scala | 31 --- .../aggregator/HingeAggregatorSuite.scala | 52 +--- .../aggregator/HuberAggregatorSuite.scala | 61 +---- .../LeastSquaresAggregatorSuite.scala | 62 +---- .../aggregator/LogisticAggregatorSuite.scala | 56 +--- python/pyspark/ml/classification.py | 67 ++--- .../ml/param/_shared_params_code_gen.py | 5 +- python/pyspark/ml/param/shared.py | 18 -- python/pyspark/ml/recommendation.py | 29 +- python/pyspark/ml/regression.py | 22 +- 27 files changed, 260 insertions(+), 1071 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 55ac2c410953b..cdaab599e2a0b 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -502,7 +502,6 @@ private[serializer] object KryoSerializer { "org.apache.spark.ml.attribute.NumericAttribute", "org.apache.spark.ml.feature.Instance", - "org.apache.spark.ml.feature.InstanceBlock", "org.apache.spark.ml.feature.LabeledPoint", "org.apache.spark.ml.feature.OffsetInstance", "org.apache.spark.ml.linalg.DenseMatrix", diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala index 00e5b61dbdc18..e054a15fc9b75 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala @@ -682,6 +682,7 @@ private[spark] object BLAS extends Serializable { val xTemp = xValues(k) * alpha while (i < indEnd) { + val rowIndex = Arows(i) yValues(Arows(i)) += Avals(i) * xTemp i += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index f16648d2abee6..905789090d625 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ import org.apache.spark.ml.optim.aggregator.HingeAggregator import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} @@ -41,7 +41,7 @@ import org.apache.spark.storage.StorageLevel /** Params for linear SVM Classifier. */ private[classification] trait LinearSVCParams extends ClassifierParams with HasRegParam with HasMaxIter with HasFitIntercept with HasTol with HasStandardization with HasWeightCol - with HasAggregationDepth with HasThreshold with HasBlockSize { + with HasAggregationDepth with HasThreshold { /** * Param for threshold in binary classification prediction. @@ -155,26 +155,19 @@ class LinearSVC @Since("2.2.0") ( def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) setDefault(aggregationDepth -> 2) - /** - * Set block size for stacking input data in matrices. - * Default is 1024. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - @Since("2.2.0") override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) override protected def train(dataset: Dataset[_]): LinearSVCModel = instrumented { instr => + val handlePersistence = dataset.storageLevel == StorageLevel.NONE + + val instances = extractInstances(dataset) + if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) + instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, - regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth, blockSize) - - val sc = dataset.sparkSession.sparkContext - val instances = extractInstances(dataset) + regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth) val (summarizer, labelSummarizer) = instances.treeAggregate( (Summarizer.createSummarizerBuffer("mean", "std", "count"), new MultiClassSummarizer))( @@ -215,33 +208,20 @@ class LinearSVC @Since("2.2.0") ( throw new SparkException(msg) } - val featuresStd = summarizer.std.compressed - val bcFeaturesStd = sc.broadcast(featuresStd) + val featuresStd = summarizer.std.toArray + val getFeaturesStd = (j: Int) => featuresStd(j) val regParamL2 = $(regParam) + val bcFeaturesStd = instances.context.broadcast(featuresStd) val regularization = if (regParamL2 != 0.0) { val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures Some(new L2Regularization(regParamL2, shouldApply, - if ($(standardization)) None else Some(featuresStd.apply))) + if ($(standardization)) None else Some(getFeaturesStd))) } else { None } - val standardized = instances.map { - case Instance(label, weight, features) => - val featuresStd = bcFeaturesStd.value - val array = Array.ofDim[Double](numFeatures) - features.foreachNonZero { (i, v) => - val std = featuresStd(i) - if (std != 0) array(i) = v / std - } - Instance(label, weight, Vectors.dense(array)) - } - val blocks = InstanceBlock.blokify(standardized, $(blockSize)) - .persist(StorageLevel.MEMORY_AND_DISK) - .setName(s"training dataset (blockSize=${$(blockSize)})") - - val getAggregatorFunc = new HingeAggregator(numFeatures, $(fitIntercept))(_) - val costFun = new RDDLossFunction(blocks, getAggregatorFunc, regularization, + val getAggregatorFunc = new HingeAggregator(bcFeaturesStd, $(fitIntercept))(_) + val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) def regParamL1Fun = (index: Int) => 0D @@ -258,7 +238,6 @@ class LinearSVC @Since("2.2.0") ( scaledObjectiveHistory += state.adjustedValue } - blocks.unpersist() bcFeaturesStd.destroy() if (state == null) { val msg = s"${optimizer.getClass.getName} failed." @@ -289,6 +268,8 @@ class LinearSVC @Since("2.2.0") ( (Vectors.dense(coefficientArray), intercept, scaledObjectiveHistory.result()) } + if (handlePersistence) instances.unpersist() + copyValues(new LinearSVCModel(uid, coefficientVector, interceptVector)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 9b5b36257a584..50c14d086957f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ import org.apache.spark.ml.optim.aggregator.LogisticAggregator import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} @@ -50,8 +50,7 @@ import org.apache.spark.util.VersionUtils */ private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasFitIntercept with HasTol - with HasStandardization with HasWeightCol with HasThreshold with HasAggregationDepth - with HasBlockSize { + with HasStandardization with HasWeightCol with HasThreshold with HasAggregationDepth { import org.apache.spark.ml.classification.LogisticRegression.supportedFamilyNames @@ -431,15 +430,6 @@ class LogisticRegression @Since("1.2.0") ( @Since("2.2.0") def setUpperBoundsOnIntercepts(value: Vector): this.type = set(upperBoundsOnIntercepts, value) - /** - * Set block size for stacking input data in matrices. - * Default is 1024. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - private def assertBoundConstrainedOptimizationParamsValid( numCoefficientSets: Int, numFeatures: Int): Unit = { @@ -492,17 +482,24 @@ class LogisticRegression @Since("1.2.0") ( this } - override protected[spark] def train( - dataset: Dataset[_]): LogisticRegressionModel = instrumented { instr => + override protected[spark] def train(dataset: Dataset[_]): LogisticRegressionModel = { + val handlePersistence = dataset.storageLevel == StorageLevel.NONE + train(dataset, handlePersistence) + } + + protected[spark] def train( + dataset: Dataset[_], + handlePersistence: Boolean): LogisticRegressionModel = instrumented { instr => + val instances = extractInstances(dataset) + + if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) + instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, probabilityCol, regParam, elasticNetParam, standardization, threshold, maxIter, tol, fitIntercept) - val sc = dataset.sparkSession.sparkContext - val instances = extractInstances(dataset) - val (summarizer, labelSummarizer) = instances.treeAggregate( (Summarizer.createSummarizerBuffer("mean", "std", "count"), new MultiClassSummarizer))( seqOp = (c: (SummarizerBuffer, MultiClassSummarizer), instance: Instance) => @@ -585,9 +582,8 @@ class LogisticRegression @Since("1.2.0") ( s"dangerous ground, so the algorithm may not converge.") } - val featuresMean = summarizer.mean.compressed - val featuresStd = summarizer.std.compressed - val bcFeaturesStd = sc.broadcast(featuresStd) + val featuresMean = summarizer.mean.toArray + val featuresStd = summarizer.std.toArray if (!$(fitIntercept) && (0 until numFeatures).exists { i => featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { @@ -599,7 +595,8 @@ class LogisticRegression @Since("1.2.0") ( val regParamL1 = $(elasticNetParam) * $(regParam) val regParamL2 = (1.0 - $(elasticNetParam)) * $(regParam) - val getAggregatorFunc = new LogisticAggregator(numFeatures, numClasses, $(fitIntercept), + val bcFeaturesStd = instances.context.broadcast(featuresStd) + val getAggregatorFunc = new LogisticAggregator(bcFeaturesStd, numClasses, $(fitIntercept), multinomial = isMultinomial)(_) val getFeaturesStd = (j: Int) => if (j >= 0 && j < numCoefficientSets * numFeatures) { featuresStd(j / numCoefficientSets) @@ -615,21 +612,7 @@ class LogisticRegression @Since("1.2.0") ( None } - val standardized = instances.map { - case Instance(label, weight, features) => - val featuresStd = bcFeaturesStd.value - val array = Array.ofDim[Double](numFeatures) - features.foreachNonZero { (i, v) => - val std = featuresStd(i) - if (std != 0) array(i) = v / std - } - Instance(label, weight, Vectors.dense(array)) - } - val blocks = InstanceBlock.blokify(standardized, $(blockSize)) - .persist(StorageLevel.MEMORY_AND_DISK) - .setName(s"training dataset (blockSize=${$(blockSize)})") - - val costFun = new RDDLossFunction(blocks, getAggregatorFunc, regularization, + val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) val numCoeffsPlusIntercepts = numFeaturesPlusIntercept * numCoefficientSets @@ -823,7 +806,6 @@ class LogisticRegression @Since("1.2.0") ( state = states.next() arrayBuilder += state.adjustedValue } - blocks.unpersist() bcFeaturesStd.destroy() if (state == null) { @@ -893,6 +875,8 @@ class LogisticRegression @Since("1.2.0") ( } } + if (handlePersistence) instances.unpersist() + val model = copyValues(new LogisticRegressionModel(uid, coefficientMatrix, interceptVector, numClasses, isMultinomial)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 6e8f92b9b1e64..c7a8237849b5b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.VersionUtils.majorMinorVersion /** Params for Multilayer Perceptron. */ private[classification] trait MultilayerPerceptronParams extends ProbabilisticClassifierParams - with HasSeed with HasMaxIter with HasTol with HasStepSize with HasSolver with HasBlockSize { + with HasSeed with HasMaxIter with HasTol with HasStepSize with HasSolver { import MultilayerPerceptronClassifier._ @@ -54,6 +54,26 @@ private[classification] trait MultilayerPerceptronParams extends ProbabilisticCl @Since("1.5.0") final def getLayers: Array[Int] = $(layers) + /** + * Block size for stacking input data in matrices to speed up the computation. + * Data is stacked within partitions. If block size is more than remaining data in + * a partition then it is adjusted to the size of this data. + * Recommended size is between 10 and 1000. + * Default: 128 + * + * @group expertParam + */ + @Since("1.5.0") + final val blockSize: IntParam = new IntParam(this, "blockSize", + "Block size for stacking input data in matrices. Data is stacked within partitions." + + " If block size is more than remaining data in a partition then " + + "it is adjusted to the size of this data. Recommended size is between 10 and 1000", + ParamValidators.gt(0)) + + /** @group expertGetParam */ + @Since("1.5.0") + final def getBlockSize: Int = $(blockSize) + /** * The solver algorithm for optimization. * Supported options: "gd" (minibatch gradient descent) or "l-bfgs". diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index 5476a86eb9d76..11d0c4689cbba 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -17,10 +17,7 @@ package org.apache.spark.ml.feature -import scala.collection.mutable - -import org.apache.spark.ml.linalg._ -import org.apache.spark.rdd.RDD +import org.apache.spark.ml.linalg.Vector /** * Class that represents an instance of weighted data point with label and features. @@ -31,131 +28,6 @@ import org.apache.spark.rdd.RDD */ private[spark] case class Instance(label: Double, weight: Double, features: Vector) - -/** - * Class that represents an block of instance. - * If all weights are 1, then an empty array is stored. - */ -private[spark] case class InstanceBlock( - labels: Array[Double], - weights: Array[Double], - matrix: Matrix) { - require(labels.length == matrix.numRows) - require(matrix.isTransposed) - if (weights.nonEmpty) { - require(labels.length == weights.length) - } - - def size: Int = labels.length - - def numFeatures: Int = matrix.numCols - - def instanceIterator: Iterator[Instance] = { - if (weights.nonEmpty) { - labels.iterator.zip(weights.iterator).zip(matrix.rowIter) - .map { case ((label, weight), vec) => Instance(label, weight, vec) } - } else { - labels.iterator.zip(matrix.rowIter) - .map { case (label, vec) => Instance(label, 1.0, vec) } - } - } - - def getLabel(i: Int): Double = labels(i) - - def labelIter: Iterator[Double] = labels.iterator - - @transient lazy val getWeight: Int => Double = { - if (weights.nonEmpty) { - (i: Int) => weights(i) - } else { - (i: Int) => 1.0 - } - } - - def weightIter: Iterator[Double] = { - if (weights.nonEmpty) { - weights.iterator - } else { - Iterator.fill(size)(1.0) - } - } - - // directly get the non-zero iterator of i-th row vector without array copy or slice - @transient lazy val getNonZeroIter: Int => Iterator[(Int, Double)] = { - matrix match { - case dm: DenseMatrix => - (i: Int) => - val start = numFeatures * i - Iterator.tabulate(numFeatures)(j => - (j, dm.values(start + j)) - ).filter(_._2 != 0) - case sm: SparseMatrix => - (i: Int) => - val start = sm.colPtrs(i) - val end = sm.colPtrs(i + 1) - Iterator.tabulate(end - start)(j => - (sm.rowIndices(start + j), sm.values(start + j)) - ).filter(_._2 != 0) - } - } -} - -private[spark] object InstanceBlock { - - def fromInstances(instances: Seq[Instance]): InstanceBlock = { - val labels = instances.map(_.label).toArray - val weights = if (instances.exists(_.weight != 1)) { - instances.map(_.weight).toArray - } else { - Array.emptyDoubleArray - } - val numRows = instances.length - val numCols = instances.head.features.size - val denseSize = Matrices.getDenseSize(numCols, numRows) - val nnz = instances.iterator.map(_.features.numNonzeros).sum - val sparseSize = Matrices.getSparseSize(nnz, numRows + 1) - val matrix = if (denseSize < sparseSize) { - val values = Array.ofDim[Double](numRows * numCols) - var offset = 0 - var j = 0 - while (j < numRows) { - instances(j).features.foreachNonZero { (i, v) => - values(offset + i) = v - } - offset += numCols - j += 1 - } - new DenseMatrix(numRows, numCols, values, true) - } else { - val colIndices = mutable.ArrayBuilder.make[Int] - val values = mutable.ArrayBuilder.make[Double] - val rowPtrs = mutable.ArrayBuilder.make[Int] - var rowPtr = 0 - rowPtrs += 0 - var j = 0 - while (j < numRows) { - var nnz = 0 - instances(j).features.foreachNonZero { (i, v) => - colIndices += i - values += v - nnz += 1 - } - rowPtr += nnz - rowPtrs += rowPtr - j += 1 - } - new SparseMatrix(numRows, numCols, rowPtrs.result(), - colIndices.result(), values.result(), true) - } - InstanceBlock(labels, weights, matrix) - } - - def blokify(instances: RDD[Instance], blockSize: Int): RDD[InstanceBlock] = { - instances.mapPartitions(_.grouped(blockSize).map(InstanceBlock.fromInstances)) - } -} - - /** * Case class that represents an instance of data point with * label, weight, offset and features. diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala index 292187b3e146e..b0906f1b06511 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg._ /** @@ -32,28 +32,21 @@ import org.apache.spark.ml.linalg._ * * @param bcCoefficients The coefficients corresponding to the features. * @param fitIntercept Whether to fit an intercept term. + * @param bcFeaturesStd The standard deviation values of the features. */ private[ml] class HingeAggregator( - numFeatures: Int, + bcFeaturesStd: Broadcast[Array[Double]], fitIntercept: Boolean)(bcCoefficients: Broadcast[Vector]) - extends DifferentiableLossAggregator[InstanceBlock, HingeAggregator] { + extends DifferentiableLossAggregator[Instance, HingeAggregator] { + private val numFeatures: Int = bcFeaturesStd.value.length private val numFeaturesPlusIntercept: Int = if (fitIntercept) numFeatures + 1 else numFeatures - protected override val dim: Int = numFeaturesPlusIntercept @transient private lazy val coefficientsArray = bcCoefficients.value match { case DenseVector(values) => values case _ => throw new IllegalArgumentException(s"coefficients only supports dense vector" + s" but got type ${bcCoefficients.value.getClass}.") } - - @transient private lazy val linear = { - if (fitIntercept) { - new DenseVector(coefficientsArray.take(numFeatures)) - } else { - new DenseVector(coefficientsArray) - } - } - + protected override val dim: Int = numFeaturesPlusIntercept /** * Add a new training instance to this HingeAggregator, and update the loss and gradient @@ -69,13 +62,16 @@ private[ml] class HingeAggregator( require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") if (weight == 0.0) return this + val localFeaturesStd = bcFeaturesStd.value val localCoefficients = coefficientsArray val localGradientSumArray = gradientSumArray val dotProduct = { var sum = 0.0 features.foreachNonZero { (index, value) => - sum += localCoefficients(index) * value + if (localFeaturesStd(index) != 0.0) { + sum += localCoefficients(index) * value / localFeaturesStd(index) + } } if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1) sum @@ -92,7 +88,9 @@ private[ml] class HingeAggregator( if (1.0 > labelScaled * dotProduct) { val gradientScale = -labelScaled * weight features.foreachNonZero { (index, value) => - localGradientSumArray(index) += value * gradientScale + if (localFeaturesStd(index) != 0.0) { + localGradientSumArray(index) += value * gradientScale / localFeaturesStd(index) + } } if (fitIntercept) { localGradientSumArray(localGradientSumArray.length - 1) += gradientScale @@ -104,78 +102,4 @@ private[ml] class HingeAggregator( this } } - - /** - * Add a new training instance block to this HingeAggregator, and update the loss and gradient - * of the objective function. - * - * @param block The InstanceBlock to be added. - * @return This HingeAggregator object. - */ - def add(block: InstanceBlock): this.type = { - require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + - s"instance. Expecting $numFeatures but got ${block.numFeatures}.") - require(block.weightIter.forall(_ >= 0), - s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") - - if (block.weightIter.forall(_ == 0)) return this - val size = block.size - val localGradientSumArray = gradientSumArray - - // vec here represents dotProducts - val vec = if (fitIntercept && coefficientsArray.last != 0) { - val intercept = coefficientsArray.last - new DenseVector(Array.fill(size)(intercept)) - } else { - new DenseVector(Array.ofDim[Double](size)) - } - - if (fitIntercept) { - BLAS.gemv(1.0, block.matrix, linear, 1.0, vec) - } else { - BLAS.gemv(1.0, block.matrix, linear, 0.0, vec) - } - - // in-place convert dotProducts to gradient scales - // then, vec represents gradient scales - var i = 0 - while (i < size) { - val weight = block.getWeight(i) - if (weight > 0) { - weightSum += weight - // Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x))) - // Therefore the gradient is -(2y - 1)*x - val label = block.getLabel(i) - val labelScaled = 2 * label - 1.0 - val loss = (1.0 - labelScaled * vec(i)) * weight - if (loss > 0) { - lossSum += loss - val gradScale = -labelScaled * weight - vec.values(i) = gradScale - } else { - vec.values(i) = 0.0 - } - } else { - vec.values(i) = 0.0 - } - i += 1 - } - - // predictions are all correct, no gradient signal - if (vec.values.forall(_ == 0)) return this - - if (fitIntercept) { - // localGradientSumArray is of size numFeatures+1, so can not - // be directly used as the output of BLAS.gemv - val linearGradSumVec = new DenseVector(Array.ofDim[Double](numFeatures)) - BLAS.gemv(1.0, block.matrix.transpose, vec, 0.0, linearGradSumVec) - linearGradSumVec.foreachNonZero { (i, v) => localGradientSumArray(i) += v } - localGradientSumArray(numFeatures) += vec.values.sum - } else { - val gradSumVec = new DenseVector(localGradientSumArray) - BLAS.gemv(1.0, block.matrix.transpose, vec, 1.0, gradSumVec) - } - - this - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala index f83621506500e..8a1a41b2950c1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast -import org.apache.spark.ml.feature.{Instance, InstanceBlock} -import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.Vector /** * HuberAggregator computes the gradient and loss for a huber loss function, @@ -62,17 +62,19 @@ import org.apache.spark.ml.linalg._ * * @param fitIntercept Whether to fit an intercept term. * @param epsilon The shape parameter to control the amount of robustness. + * @param bcFeaturesStd The broadcast standard deviation values of the features. * @param bcParameters including three parts: the regression coefficients corresponding * to the features, the intercept (if fitIntercept is ture) * and the scale parameter (sigma). */ private[ml] class HuberAggregator( - numFeatures: Int, fitIntercept: Boolean, - epsilon: Double)(bcParameters: Broadcast[Vector]) - extends DifferentiableLossAggregator[InstanceBlock, HuberAggregator] { + epsilon: Double, + bcFeaturesStd: Broadcast[Array[Double]])(bcParameters: Broadcast[Vector]) + extends DifferentiableLossAggregator[Instance, HuberAggregator] { protected override val dim: Int = bcParameters.value.size + private val numFeatures: Int = if (fitIntercept) dim - 2 else dim - 1 private val sigma: Double = bcParameters.value(dim - 1) private val intercept: Double = if (fitIntercept) { bcParameters.value(dim - 2) @@ -80,8 +82,7 @@ private[ml] class HuberAggregator( 0.0 } // make transient so we do not serialize between aggregation stages - @transient private lazy val linear = - new DenseVector(bcParameters.value.toArray.take(numFeatures)) + @transient private lazy val coefficients = bcParameters.value.toArray.slice(0, numFeatures) /** * Add a new training instance to this HuberAggregator, and update the loss and gradient @@ -97,13 +98,16 @@ private[ml] class HuberAggregator( require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") if (weight == 0.0) return this - val localCoefficients = linear.values + val localFeaturesStd = bcFeaturesStd.value + val localCoefficients = coefficients val localGradientSumArray = gradientSumArray val margin = { var sum = 0.0 features.foreachNonZero { (index, value) => - sum += localCoefficients(index) * value + if (localFeaturesStd(index) != 0.0) { + sum += localCoefficients(index) * (value / localFeaturesStd(index)) + } } if (fitIntercept) sum += intercept sum @@ -115,7 +119,10 @@ private[ml] class HuberAggregator( val linearLossDivSigma = linearLoss / sigma features.foreachNonZero { (index, value) => - localGradientSumArray(index) -= weight * linearLossDivSigma * value + if (localFeaturesStd(index) != 0.0) { + localGradientSumArray(index) += + -1.0 * weight * linearLossDivSigma * (value / localFeaturesStd(index)) + } } if (fitIntercept) { localGradientSumArray(dim - 2) += -1.0 * weight * linearLossDivSigma @@ -127,7 +134,10 @@ private[ml] class HuberAggregator( (sigma + 2.0 * epsilon * math.abs(linearLoss) - sigma * epsilon * epsilon) features.foreachNonZero { (index, value) => - localGradientSumArray(index) += weight * sign * epsilon * value + if (localFeaturesStd(index) != 0.0) { + localGradientSumArray(index) += + weight * sign * epsilon * (value / localFeaturesStd(index)) + } } if (fitIntercept) { localGradientSumArray(dim - 2) += weight * sign * epsilon @@ -139,75 +149,4 @@ private[ml] class HuberAggregator( this } } - - /** - * Add a new training instance block to this HuberAggregator, and update the loss and gradient - * of the objective function. - * - * @param block The instance block of data point to be added. - * @return This HuberAggregator object. - */ - def add(block: InstanceBlock): HuberAggregator = { - require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + - s"instance. Expecting $numFeatures but got ${block.numFeatures}.") - require(block.weightIter.forall(_ >= 0), - s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") - - if (block.weightIter.forall(_ == 0)) return this - val size = block.size - val localGradientSumArray = gradientSumArray - - // vec here represents margins or dotProducts - val vec = if (fitIntercept && intercept != 0) { - new DenseVector(Array.fill(size)(intercept)) - } else { - new DenseVector(Array.ofDim[Double](size)) - } - - if (fitIntercept) { - BLAS.gemv(1.0, block.matrix, linear, 1.0, vec) - } else { - BLAS.gemv(1.0, block.matrix, linear, 0.0, vec) - } - - // in-place convert margins to multipliers - // then, vec represents multipliers - var i = 0 - while (i < size) { - val weight = block.getWeight(i) - if (weight > 0) { - weightSum += weight - val label = block.getLabel(i) - val margin = vec(i) - val linearLoss = label - margin - - if (math.abs(linearLoss) <= sigma * epsilon) { - lossSum += 0.5 * weight * (sigma + math.pow(linearLoss, 2.0) / sigma) - val linearLossDivSigma = linearLoss / sigma - val multiplier = -1.0 * weight * linearLossDivSigma - vec.values(i) = multiplier - localGradientSumArray(dim - 1) += 0.5 * weight * (1.0 - math.pow(linearLossDivSigma, 2.0)) - } else { - lossSum += 0.5 * weight * - (sigma + 2.0 * epsilon * math.abs(linearLoss) - sigma * epsilon * epsilon) - val sign = if (linearLoss >= 0) -1.0 else 1.0 - val multiplier = weight * sign * epsilon - vec.values(i) = multiplier - localGradientSumArray(dim - 1) += 0.5 * weight * (1.0 - epsilon * epsilon) - } - } else { - vec.values(i) = 0.0 - } - i += 1 - } - - val linearGradSumVec = new DenseVector(Array.ofDim[Double](numFeatures)) - BLAS.gemv(1.0, block.matrix.transpose, vec, 0.0, linearGradSumVec) - linearGradSumVec.foreachNonZero { (i, v) => localGradientSumArray(i) += v } - if (fitIntercept) { - localGradientSumArray(dim - 2) += vec.values.sum - } - - this - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala index a8bda9ca5d245..7a5806dc24aee 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala @@ -17,8 +17,8 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast -import org.apache.spark.ml.feature.{Instance, InstanceBlock} -import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} /** * LeastSquaresAggregator computes the gradient and loss for a Least-squared loss function, @@ -157,25 +157,26 @@ private[ml] class LeastSquaresAggregator( labelStd: Double, labelMean: Double, fitIntercept: Boolean, - bcFeaturesStd: Broadcast[Vector], - bcFeaturesMean: Broadcast[Vector])(bcCoefficients: Broadcast[Vector]) - extends DifferentiableLossAggregator[InstanceBlock, LeastSquaresAggregator] { + bcFeaturesStd: Broadcast[Array[Double]], + bcFeaturesMean: Broadcast[Array[Double]])(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[Instance, LeastSquaresAggregator] { require(labelStd > 0.0, s"${this.getClass.getName} requires the label standard " + s"deviation to be positive.") - private val numFeatures = bcFeaturesStd.value.size + private val numFeatures = bcFeaturesStd.value.length protected override val dim: Int = numFeatures // make transient so we do not serialize between aggregation stages + @transient private lazy val featuresStd = bcFeaturesStd.value @transient private lazy val effectiveCoefAndOffset = { val coefficientsArray = bcCoefficients.value.toArray.clone() val featuresMean = bcFeaturesMean.value - val featuresStd = bcFeaturesStd.value var sum = 0.0 var i = 0 val len = coefficientsArray.length while (i < len) { if (featuresStd(i) != 0.0) { - sum += coefficientsArray(i) / featuresStd(i) * featuresMean(i) + coefficientsArray(i) /= featuresStd(i) + sum += coefficientsArray(i) * featuresMean(i) } else { coefficientsArray(i) = 0.0 } @@ -185,7 +186,7 @@ private[ml] class LeastSquaresAggregator( (Vectors.dense(coefficientsArray), offset) } // do not use tuple assignment above because it will circumvent the @transient tag - @transient private lazy val effectiveCoefficientsVec = effectiveCoefAndOffset._1 + @transient private lazy val effectiveCoefficientsVector = effectiveCoefAndOffset._1 @transient private lazy val offset = effectiveCoefAndOffset._2 /** @@ -203,20 +204,16 @@ private[ml] class LeastSquaresAggregator( if (weight == 0.0) return this - val localEffectiveCoefficientsVec = effectiveCoefficientsVec - - val diff = { - var dot = 0.0 - features.foreachNonZero { (index, value) => - dot += localEffectiveCoefficientsVec(index) * value - } - dot - label / labelStd + offset - } + val diff = BLAS.dot(features, effectiveCoefficientsVector) - label / labelStd + offset if (diff != 0) { val localGradientSumArray = gradientSumArray + val localFeaturesStd = featuresStd features.foreachNonZero { (index, value) => - localGradientSumArray(index) += weight * diff * value + val fStd = localFeaturesStd(index) + if (fStd != 0.0) { + localGradientSumArray(index) += weight * diff * value / fStd + } } lossSum += weight * diff * diff / 2.0 } @@ -224,43 +221,4 @@ private[ml] class LeastSquaresAggregator( this } } - - /** - * Add a new training instance block to this LeastSquaresAggregator, and update the loss - * and gradient of the objective function. - * - * @param block The instance block of data point to be added. - * @return This LeastSquaresAggregator object. - */ - def add(block: InstanceBlock): LeastSquaresAggregator = { - require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + - s"instance. Expecting $numFeatures but got ${block.numFeatures}.") - require(block.weightIter.forall(_ >= 0), - s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") - - if (block.weightIter.forall(_ == 0)) return this - val size = block.size - - // vec here represents diffs - val vec = new DenseVector(Array.tabulate(size)(i => offset - block.getLabel(i) / labelStd)) - BLAS.gemv(1.0, block.matrix, effectiveCoefficientsVec, 1.0, vec) - - // in-place convert diffs to multipliers - // then, vec represents multipliers - var i = 0 - while (i < size) { - val weight = block.getWeight(i) - val diff = vec(i) - lossSum += weight * diff * diff / 2 - weightSum += weight - val multiplier = weight * diff - vec.values(i) = multiplier - i += 1 - } - - val gradSumVec = new DenseVector(gradientSumArray) - BLAS.gemv(1.0, block.matrix.transpose, vec, 1.0, gradSumVec) - - this - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala index 76d21995a2c50..f2b3566f8f09e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.{Instance, InstanceBlock} -import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{DenseVector, Vector} import org.apache.spark.mllib.util.MLUtils /** @@ -171,6 +171,7 @@ import org.apache.spark.mllib.util.MLUtils * * * @param bcCoefficients The broadcast coefficients corresponding to the features. + * @param bcFeaturesStd The broadcast standard deviation values of the features. * @param numClasses the number of possible outcomes for k classes classification problem in * Multinomial Logistic Regression. * @param fitIntercept Whether to fit an intercept term. @@ -182,12 +183,13 @@ import org.apache.spark.mllib.util.MLUtils * since this form is optimal for the matrix operations used for prediction. */ private[ml] class LogisticAggregator( - numFeatures: Int, + bcFeaturesStd: Broadcast[Array[Double]], numClasses: Int, fitIntercept: Boolean, multinomial: Boolean)(bcCoefficients: Broadcast[Vector]) - extends DifferentiableLossAggregator[InstanceBlock, LogisticAggregator] with Logging { + extends DifferentiableLossAggregator[Instance, LogisticAggregator] with Logging { + private val numFeatures = bcFeaturesStd.value.length private val numFeaturesPlusIntercept = if (fitIntercept) numFeatures + 1 else numFeatures private val coefficientSize = bcCoefficients.value.size protected override val dim: Int = coefficientSize @@ -207,31 +209,6 @@ private[ml] class LogisticAggregator( s"got type ${bcCoefficients.value.getClass}.)") } - @transient private lazy val binaryLinear = { - if (!multinomial) { - if (fitIntercept) { - new DenseVector(coefficientsArray.take(numFeatures)) - } else { - new DenseVector(coefficientsArray) - } - } else { - null - } - } - - @transient private lazy val multinomialLinear = { - if (multinomial) { - if (fitIntercept) { - new DenseMatrix(numClasses, numFeatures, coefficientsArray.take(numClasses * numFeatures)) - } else { - new DenseMatrix(numClasses, numFeatures, coefficientsArray) - } - } else { - null - } - } - - if (multinomial && numClasses <= 2) { logInfo(s"Multinomial logistic regression for binary classification yields separate " + s"coefficients for positive and negative classes. When no regularization is applied, the" + @@ -242,12 +219,15 @@ private[ml] class LogisticAggregator( /** Update gradient and loss using binary loss function. */ private def binaryUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = { + val localFeaturesStd = bcFeaturesStd.value val localCoefficients = coefficientsArray val localGradientArray = gradientSumArray val margin = - { var sum = 0.0 features.foreachNonZero { (index, value) => - sum += localCoefficients(index) * value + if (localFeaturesStd(index) != 0.0) { + sum += localCoefficients(index) * value / localFeaturesStd(index) + } } if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1) sum @@ -256,7 +236,9 @@ private[ml] class LogisticAggregator( val multiplier = weight * (1.0 / (1.0 + math.exp(margin)) - label) features.foreachNonZero { (index, value) => - localGradientArray(index) += multiplier * value + if (localFeaturesStd(index) != 0.0) { + localGradientArray(index) += multiplier * value / localFeaturesStd(index) + } } if (fitIntercept) { @@ -271,61 +253,6 @@ private[ml] class LogisticAggregator( } } - /** Update gradient and loss using binary loss function. */ - private def binaryUpdateInPlace(block: InstanceBlock): Unit = { - val size = block.size - val localGradientSumArray = gradientSumArray - - // vec here represents margins or negative dotProducts - val vec = if (fitIntercept && coefficientsArray.last != 0) { - val intercept = coefficientsArray.last - new DenseVector(Array.fill(size)(intercept)) - } else { - new DenseVector(Array.ofDim[Double](size)) - } - - if (fitIntercept) { - BLAS.gemv(-1.0, block.matrix, binaryLinear, -1.0, vec) - } else { - BLAS.gemv(-1.0, block.matrix, binaryLinear, 0.0, vec) - } - - // in-place convert margins to multiplier - // then, vec represents multiplier - var i = 0 - while (i < size) { - val weight = block.getWeight(i) - if (weight > 0) { - weightSum += weight - val label = block.getLabel(i) - val margin = vec(i) - if (label > 0) { - // The following is equivalent to log(1 + exp(margin)) but more numerically stable. - lossSum += weight * MLUtils.log1pExp(margin) - } else { - lossSum += weight * (MLUtils.log1pExp(margin) - margin) - } - val multiplier = weight * (1.0 / (1.0 + math.exp(margin)) - label) - vec.values(i) = multiplier - } else { - vec.values(i) = 0.0 - } - i += 1 - } - - if (fitIntercept) { - // localGradientSumArray is of size numFeatures+1, so can not - // be directly used as the output of BLAS.gemv - val linearGradSumVec = new DenseVector(Array.ofDim[Double](numFeatures)) - BLAS.gemv(1.0, block.matrix.transpose, vec, 0.0, linearGradSumVec) - linearGradSumVec.foreachNonZero { (i, v) => localGradientSumArray(i) += v } - localGradientSumArray(numFeatures) += vec.values.sum - } else { - val gradSumVec = new DenseVector(localGradientSumArray) - BLAS.gemv(1.0, block.matrix.transpose, vec, 1.0, gradSumVec) - } - } - /** Update gradient and loss using multinomial (softmax) loss function. */ private def multinomialUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = { // TODO: use level 2 BLAS operations @@ -333,6 +260,7 @@ private[ml] class LogisticAggregator( Note: this can still be used when numClasses = 2 for binary logistic regression without pivoting. */ + val localFeaturesStd = bcFeaturesStd.value val localCoefficients = coefficientsArray val localGradientArray = gradientSumArray @@ -342,10 +270,13 @@ private[ml] class LogisticAggregator( val margins = new Array[Double](numClasses) features.foreachNonZero { (index, value) => - var j = 0 - while (j < numClasses) { - margins(j) += localCoefficients(index * numClasses + j) * value - j += 1 + if (localFeaturesStd(index) != 0.0) { + val stdValue = value / localFeaturesStd(index) + var j = 0 + while (j < numClasses) { + margins(j) += localCoefficients(index * numClasses + j) * stdValue + j += 1 + } } } var i = 0 @@ -383,10 +314,13 @@ private[ml] class LogisticAggregator( multipliers(i) = multipliers(i) / sum - (if (label == i) 1.0 else 0.0) } features.foreachNonZero { (index, value) => - var j = 0 - while (j < numClasses) { - localGradientArray(index * numClasses + j) += weight * multipliers(j) * value - j += 1 + if (localFeaturesStd(index) != 0.0) { + val stdValue = value / localFeaturesStd(index) + var j = 0 + while (j < numClasses) { + localGradientArray(index * numClasses + j) += weight * multipliers(j) * stdValue + j += 1 + } } } if (fitIntercept) { @@ -405,112 +339,6 @@ private[ml] class LogisticAggregator( lossSum += weight * loss } - /** Update gradient and loss using multinomial (softmax) loss function. */ - private def multinomialUpdateInPlace(block: InstanceBlock): Unit = { - val size = block.size - val localGradientSumArray = gradientSumArray - - // mat here represents margins, shape: S X C - val mat = new DenseMatrix(size, numClasses, Array.ofDim[Double](size * numClasses)) - - if (fitIntercept) { - val intercept = coefficientsArray.takeRight(numClasses) - var i = 0 - while (i < size) { - var j = 0 - while (j < numClasses) { - mat.update(i, j, intercept(j)) - j += 1 - } - i += 1 - } - BLAS.gemm(1.0, block.matrix, multinomialLinear.transpose, 1.0, mat) - } else { - BLAS.gemm(1.0, block.matrix, multinomialLinear.transpose, 0.0, mat) - } - - // in-place convert margins to multipliers - // then, mat represents multipliers - var i = 0 - val tmp = Array.ofDim[Double](numClasses) - while (i < size) { - val weight = block.getWeight(i) - if (weight > 0) { - weightSum += weight - val label = block.getLabel(i) - - var maxMargin = Double.NegativeInfinity - var j = 0 - while (j < numClasses) { - tmp(j) = mat(i, j) - maxMargin = math.max(maxMargin, tmp(j)) - j += 1 - } - - // marginOfLabel is margins(label) in the formula - val marginOfLabel = tmp(label.toInt) - - var sum = 0.0 - j = 0 - while (j < numClasses) { - if (maxMargin > 0) tmp(j) -= maxMargin - val exp = math.exp(tmp(j)) - sum += exp - tmp(j) = exp - j += 1 - } - - j = 0 - while (j < numClasses) { - val multiplier = weight * (tmp(j) / sum - (if (label == j) 1.0 else 0.0)) - mat.update(i, j, multiplier) - j += 1 - } - - if (maxMargin > 0) { - lossSum += weight * (math.log(sum) - marginOfLabel + maxMargin) - } else { - lossSum += weight * (math.log(sum) - marginOfLabel) - } - } else { - var j = 0 - while (j < numClasses) { - mat.update(i, j, 0.0) - j += 1 - } - } - i += 1 - } - - // block.matrix: S X F, unknown type - // mat (multipliers): S X C, dense - // gradSumMat(gradientSumArray): C X FPI (numFeaturesPlusIntercept), dense - block.matrix match { - case dm: DenseMatrix if !fitIntercept => - // If fitIntercept==false, gradientSumArray += mat.T X matrix - // GEMM requires block.matrix is dense - val gradSumMat = new DenseMatrix(numClasses, numFeatures, localGradientSumArray) - BLAS.gemm(1.0, mat.transpose, dm, 1.0, gradSumMat) - - case _ => - // Otherwise, use linearGradSumMat (F X C) as a temp matrix: - // linearGradSumMat = matrix.T X mat - val linearGradSumMat = new DenseMatrix(numFeatures, numClasses, - Array.ofDim[Double](numFeatures * numClasses)) - BLAS.gemm(1.0, block.matrix.transpose, mat, 0.0, linearGradSumMat) - linearGradSumMat.foreachActive { (i, j, v) => - if (v != 0) localGradientSumArray(i * numClasses + j) += v - } - - if (fitIntercept) { - val start = numClasses * numFeatures - mat.foreachActive { (i, j, v) => - if (v != 0) localGradientSumArray(start + j) += v - } - } - } - } - /** * Add a new training instance to this LogisticAggregator, and update the loss and gradient * of the objective function. @@ -535,28 +363,4 @@ private[ml] class LogisticAggregator( this } } - - /** - * Add a new training instance block to this LogisticAggregator, and update the loss and gradient - * of the objective function. - * - * @param block The instance block of data point to be added. - * @return This LogisticAggregator object. - */ - def add(block: InstanceBlock): this.type = { - require(numFeatures == block.numFeatures, s"Dimensions mismatch when adding new " + - s"instance. Expecting $numFeatures but got ${block.numFeatures}.") - require(block.weightIter.forall(_ >= 0), - s"instance weights ${block.weightIter.mkString("[", ",", "]")} has to be >= 0.0") - - if (block.weightIter.forall(_ == 0)) return this - - if (multinomial) { - multinomialUpdateInPlace(block) - } else { - binaryUpdateInPlace(block) - } - - this - } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index 3d1fab8692af7..7ac680ec1183a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -104,11 +104,7 @@ private[shared] object SharedParamsCodeGen { isValid = "ParamValidators.inArray(Array(\"euclidean\", \"cosine\"))"), ParamDesc[String]("validationIndicatorCol", "name of the column that indicates whether " + "each row is for training or for validation. False indicates training; true indicates " + - "validation"), - ParamDesc[Int]("blockSize", "block size for stacking input data in matrices. Data is " + - "stacked within partitions. If block size is more than remaining data in a partition " + - "then it is adjusted to the size of this data", Some("1024"), - isValid = "ParamValidators.gt(0)", isExpertParam = true) + "validation.") ) val code = genSharedParams(params) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 7fe8ccd973a72..44c993eeafddc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -570,31 +570,12 @@ trait HasDistanceMeasure extends Params { trait HasValidationIndicatorCol extends Params { /** - * Param for name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation. + * Param for name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation.. * @group param */ - final val validationIndicatorCol: Param[String] = new Param[String](this, "validationIndicatorCol", "name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation") + final val validationIndicatorCol: Param[String] = new Param[String](this, "validationIndicatorCol", "name of the column that indicates whether each row is for training or for validation. False indicates training; true indicates validation.") /** @group getParam */ final def getValidationIndicatorCol: String = $(validationIndicatorCol) } - -/** - * Trait for shared param blockSize (default: 1024). This trait may be changed or - * removed between minor versions. - */ -@DeveloperApi -trait HasBlockSize extends Params { - - /** - * Param for block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data. - * @group expertParam - */ - final val blockSize: IntParam = new IntParam(this, "blockSize", "block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data", ParamValidators.gt(0)) - - setDefault(blockSize, 1024) - - /** @group expertGetParam */ - final def getBlockSize: Int = $(blockSize) -} // scalastyle:on diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 002146f89e79a..2fb9a276be887 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -54,8 +54,7 @@ import org.apache.spark.util.random.XORShiftRandom /** * Common params for ALS and ALSModel. */ -private[recommendation] trait ALSModelParams extends Params with HasPredictionCol - with HasBlockSize { +private[recommendation] trait ALSModelParams extends Params with HasPredictionCol { /** * Param for the column name for user ids. Ids must be integers. Other * numeric types are supported for this column, but will be cast to integers as long as they @@ -126,8 +125,6 @@ private[recommendation] trait ALSModelParams extends Params with HasPredictionCo /** @group expertGetParam */ def getColdStartStrategy: String = $(coldStartStrategy).toLowerCase(Locale.ROOT) - - setDefault(blockSize -> 4096) } /** @@ -291,15 +288,6 @@ class ALSModel private[ml] ( @Since("2.2.0") def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - private val predict = udf { (featuresA: Seq[Float], featuresB: Seq[Float]) => if (featuresA != null && featuresB != null) { var dotProduct = 0.0f @@ -363,7 +351,7 @@ class ALSModel private[ml] ( */ @Since("2.2.0") def recommendForAllUsers(numItems: Int): DataFrame = { - recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) + recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems) } /** @@ -378,7 +366,7 @@ class ALSModel private[ml] ( @Since("2.3.0") def recommendForUserSubset(dataset: Dataset[_], numItems: Int): DataFrame = { val srcFactorSubset = getSourceFactorSubset(dataset, userFactors, $(userCol)) - recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) + recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems) } /** @@ -389,7 +377,7 @@ class ALSModel private[ml] ( */ @Since("2.2.0") def recommendForAllItems(numUsers: Int): DataFrame = { - recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) + recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers) } /** @@ -404,7 +392,7 @@ class ALSModel private[ml] ( @Since("2.3.0") def recommendForItemSubset(dataset: Dataset[_], numUsers: Int): DataFrame = { val srcFactorSubset = getSourceFactorSubset(dataset, itemFactors, $(itemCol)) - recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) + recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers) } /** @@ -453,12 +441,11 @@ class ALSModel private[ml] ( dstFactors: DataFrame, srcOutputColumn: String, dstOutputColumn: String, - num: Int, - blockSize: Int): DataFrame = { + num: Int): DataFrame = { import srcFactors.sparkSession.implicits._ - val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])], blockSize) - val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])], blockSize) + val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])]) + val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])]) val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked) .as[(Seq[(Int, Array[Float])], Seq[(Int, Array[Float])])] .flatMap { case (srcIter, dstIter) => @@ -496,10 +483,11 @@ class ALSModel private[ml] ( /** * Blockifies factors to improve the efficiency of cross join + * TODO: SPARK-20443 - expose blockSize as a param? */ private def blockify( factors: Dataset[(Int, Array[Float])], - blockSize: Int): Dataset[Seq[(Int, Array[Float])]] = { + blockSize: Int = 4096): Dataset[Seq[(Int, Array[Float])]] = { import factors.sparkSession.implicits._ factors.mapPartitions(_.grouped(blockSize)) } @@ -666,15 +654,6 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] @Since("2.2.0") def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) - /** - * Set block size for stacking input data in matrices. - * Default is 4096. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - /** * Sets both numUserBlocks and numItemBlocks to the specific value. * @@ -704,7 +683,7 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] instr.logDataset(dataset) instr.logParams(this, rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, userCol, itemCol, ratingCol, predictionCol, maxIter, regParam, nonnegative, checkpointInterval, - seed, intermediateStorageLevel, finalStorageLevel, blockSize) + seed, intermediateStorageLevel, finalStorageLevel) val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), @@ -715,8 +694,7 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] checkpointInterval = $(checkpointInterval), seed = $(seed)) val userDF = userFactors.toDF("id", "features") val itemDF = itemFactors.toDF("id", "features") - val model = new ALSModel(uid, $(rank), userDF, itemDF).setBlockSize($(blockSize)) - .setParent(this) + val model = new ALSModel(uid, $(rank), userDF, itemDF).setParent(this) copyValues(model) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index fc59da8a9c165..64e5e191ffd17 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -28,7 +28,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{PipelineStage, PredictorParams} -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.linalg.BLAS._ import org.apache.spark.ml.optim.WeightedLeastSquares @@ -55,7 +55,7 @@ import org.apache.spark.util.VersionUtils.majorMinorVersion private[regression] trait LinearRegressionParams extends PredictorParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver - with HasAggregationDepth with HasLoss with HasBlockSize { + with HasAggregationDepth with HasLoss { import LinearRegression._ @@ -316,15 +316,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String def setEpsilon(value: Double): this.type = set(epsilon, value) setDefault(epsilon -> 1.35) - /** - * Set block size for stacking input data in matrices. - * Default is 1024. - * - * @group expertSetParam - */ - @Since("3.0.0") - def setBlockSize(value: Int): this.type = set(blockSize, value) - override protected def train(dataset: Dataset[_]): LinearRegressionModel = instrumented { instr => // Extract the number of features before deciding optimization solver. val numFeatures = MetadataUtils.getNumFeatures(dataset, $(featuresCol)) @@ -363,6 +354,9 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String return lrModel.setSummary(Some(trainingSummary)) } + val handlePersistence = dataset.storageLevel == StorageLevel.NONE + if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) + val (featuresSummarizer, ySummarizer) = instances.treeAggregate( (Summarizer.createSummarizerBuffer("mean", "std"), Summarizer.createSummarizerBuffer("mean", "std", "count")))( @@ -398,6 +392,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String s"will be zeros and the intercept will be the mean of the label; as a result, " + s"training is not needed.") } + if (handlePersistence) instances.unpersist() val coefficients = Vectors.sparse(numFeatures, Seq.empty) val intercept = yMean @@ -426,8 +421,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String // if y is constant (rawYStd is zero), then y cannot be scaled. In this case // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) - val featuresMean = featuresSummarizer.mean.compressed - val featuresStd = featuresSummarizer.std.compressed + val featuresMean = featuresSummarizer.mean.toArray + val featuresStd = featuresSummarizer.std.toArray val bcFeaturesMean = instances.context.broadcast(featuresMean) val bcFeaturesStd = instances.context.broadcast(featuresStd) @@ -447,36 +442,23 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val effectiveL1RegParam = $(elasticNetParam) * effectiveRegParam val effectiveL2RegParam = (1.0 - $(elasticNetParam)) * effectiveRegParam + val getFeaturesStd = (j: Int) => if (j >= 0 && j < numFeatures) featuresStd(j) else 0.0 val regularization = if (effectiveL2RegParam != 0.0) { val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures Some(new L2Regularization(effectiveL2RegParam, shouldApply, - if ($(standardization)) None else Some(featuresStd.apply))) + if ($(standardization)) None else Some(getFeaturesStd))) } else { None } - val standardized = instances.map { - case Instance(label, weight, features) => - val featuresStd = bcFeaturesStd.value - val array = Array.ofDim[Double](numFeatures) - features.foreachNonZero { (i, v) => - val std = featuresStd(i) - if (std != 0) array(i) = v / std - } - Instance(label, weight, Vectors.dense(array)) - } - val blocks = InstanceBlock.blokify(standardized, $(blockSize)) - .persist(StorageLevel.MEMORY_AND_DISK) - .setName(s"training dataset (blockSize=${$(blockSize)})") - val costFun = $(loss) match { case SquaredError => val getAggregatorFunc = new LeastSquaresAggregator(yStd, yMean, $(fitIntercept), bcFeaturesStd, bcFeaturesMean)(_) - new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) + new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) case Huber => - val getAggregatorFunc = new HuberAggregator(numFeatures, $(fitIntercept), $(epsilon))(_) - new RDDLossFunction(blocks, getAggregatorFunc, regularization, $(aggregationDepth)) + val getAggregatorFunc = new HuberAggregator($(fitIntercept), $(epsilon), bcFeaturesStd)(_) + new RDDLossFunction(instances, getAggregatorFunc, regularization, $(aggregationDepth)) } val optimizer = $(loss) match { @@ -542,7 +524,6 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String throw new SparkException(msg) } - blocks.unpersist() bcFeaturesMean.destroy() bcFeaturesStd.destroy() @@ -576,7 +557,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String after the coefficients are converged. See the following discussion for detail. http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet */ - yMean - dot(Vectors.dense(rawCoefficients), featuresMean) + yMean - dot(Vectors.dense(rawCoefficients), Vectors.dense(featuresMean)) case Huber => parameters(numFeatures) } } else { @@ -591,6 +572,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String (Vectors.dense(rawCoefficients).compressed, interceptValue, scaleValue, arrayBuilder.result()) } + if (handlePersistence) instances.unpersist() + val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept, scale)) // Handle possible missing or invalid prediction columns val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index f88f3fce61b33..21eb17dfaacb3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -339,8 +339,10 @@ class LogisticRegressionWithLBFGS // Convert our input into a DataFrame val spark = SparkSession.builder().sparkContext(input.context).getOrCreate() val df = spark.createDataFrame(input.map(_.asML)) + // Determine if we should cache the DF + val handlePersistence = input.getStorageLevel == StorageLevel.NONE // Train our model - val mlLogisticRegressionModel = lr.train(df) + val mlLogisticRegressionModel = lr.train(df, handlePersistence) // convert the model val weights = Vectors.dense(mlLogisticRegressionModel.coefficients.toArray) createModel(weights, mlLogisticRegressionModel.intercept) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala index 2b63dc259a14f..c2072cea11859 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -179,7 +179,7 @@ class LinearSVCSuite extends MLTest with DefaultReadWriteTest { test("sparse coefficients in HingeAggregator") { val bcCoefficients = spark.sparkContext.broadcast(Vectors.sparse(2, Array(0), Array(1.0))) val bcFeaturesStd = spark.sparkContext.broadcast(Array(1.0)) - val agg = new HingeAggregator(1, true)(bcCoefficients) + val agg = new HingeAggregator(bcFeaturesStd, true)(bcCoefficients) val thrown = withClue("LinearSVCAggregator cannot handle sparse coefficients") { intercept[IllegalArgumentException] { agg.add(Instance(1.0, 1.0, Vectors.dense(1.0))) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 9e359ba098bfb..6d31e6efc7e1c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -542,7 +542,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { test("sparse coefficients in LogisticAggregator") { val bcCoefficientsBinary = spark.sparkContext.broadcast(Vectors.sparse(2, Array(0), Array(1.0))) val bcFeaturesStd = spark.sparkContext.broadcast(Array(1.0)) - val binaryAgg = new LogisticAggregator(1, 2, + val binaryAgg = new LogisticAggregator(bcFeaturesStd, 2, fitIntercept = true, multinomial = false)(bcCoefficientsBinary) val thrownBinary = withClue("binary logistic aggregator cannot handle sparse coefficients") { intercept[IllegalArgumentException] { @@ -552,7 +552,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(thrownBinary.getMessage.contains("coefficients only supports dense")) val bcCoefficientsMulti = spark.sparkContext.broadcast(Vectors.sparse(6, Array(0), Array(1.0))) - val multinomialAgg = new LogisticAggregator(1, 3, + val multinomialAgg = new LogisticAggregator(bcFeaturesStd, 3, fitIntercept = true, multinomial = true)(bcCoefficientsMulti) val thrown = withClue("multinomial logistic aggregator cannot handle sparse coefficients") { intercept[IllegalArgumentException] { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala index d780bdf5f5dc8..5a74490058398 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala @@ -42,36 +42,5 @@ class InstanceSuite extends SparkFunSuite{ val o2 = ser.deserialize[OffsetInstance](ser.serialize(o)) assert(o === o2) } - - val block1 = InstanceBlock.fromInstances(Seq(instance1)) - val block2 = InstanceBlock.fromInstances(Seq(instance1, instance2)) - Seq(block1, block2).foreach { o => - val o2 = ser.deserialize[InstanceBlock](ser.serialize(o)) - assert(o.labels === o2.labels) - assert(o.weights === o2.weights) - assert(o.matrix === o2.matrix) - } - } - - test("InstanceBlock: check correctness") { - val instance1 = Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)) - val instance2 = Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse) - val instances = Seq(instance1, instance2) - - val block = InstanceBlock.fromInstances(instances) - assert(block.size === 2) - assert(block.numFeatures === 2) - block.instanceIterator.zipWithIndex.foreach { - case (instance, i) => - assert(instance.label === instances(i).label) - assert(instance.weight === instances(i).weight) - assert(instance.features.toArray === instances(i).features.toArray) - } - Seq(0, 1).foreach { i => - val nzIter = block.getNonZeroIter(i) - val vec = Vectors.sparse(2, nzIter.toSeq) - assert(vec.toArray === instances(i).features.toArray) - } } - } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala index c02a0a5e5e7d0..61b48ffa10944 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,21 +32,21 @@ class HingeAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { override def beforeAll(): Unit = { super.beforeAll() - instances = standardize(Array( + instances = Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(0.0, 0.3, Vectors.dense(4.0, 0.5)) - )) - instancesConstantFeature = standardize(Array( + ) + instancesConstantFeature = Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), Instance(1.0, 0.3, Vectors.dense(1.0, 0.5)) - )) - instancesConstantFeatureFiltered = standardize(Array( + ) + instancesConstantFeatureFiltered = Array( Instance(0.0, 0.1, Vectors.dense(2.0)), Instance(1.0, 0.5, Vectors.dense(1.0)), Instance(2.0, 0.3, Vectors.dense(0.5)) - )) + ) } /** Get summary statistics for some data and create a new HingeAggregator. */ @@ -54,23 +54,12 @@ class HingeAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { instances: Array[Instance], coefficients: Vector, fitIntercept: Boolean): HingeAggregator = { - val bcCoefficients = spark.sparkContext.broadcast(coefficients) - new HingeAggregator(instances.head.features.size, fitIntercept)(bcCoefficients) - } - - private def standardize(instances: Array[Instance]): Array[Instance] = { - val (featuresSummarizer, _) = + val (featuresSummarizer, ySummarizer) = DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) - val stdArray = featuresSummarizer.variance.toArray.map(math.sqrt) - val numFeatures = stdArray.length - instances.map { case Instance(label, weight, features) => - val standardized = Array.ofDim[Double](numFeatures) - features.foreachNonZero { (i, v) => - val std = stdArray(i) - if (std != 0) standardized(i) = v / std - } - Instance(label, weight, Vectors.dense(standardized).compressed) - } + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) + val bcCoefficients = spark.sparkContext.broadcast(coefficients) + new HingeAggregator(bcFeaturesStd, fitIntercept)(bcCoefficients) } test("aggregator add method input size") { @@ -171,21 +160,4 @@ class HingeAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { assert(aggConstantFeatureBinary.gradient(1) == aggConstantFeatureBinaryFiltered.gradient(0)) } - test("add instance block") { - val coefArray = Array(1.0, 2.0) - val intercept = 1.0 - - val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ Array(intercept)), - fitIntercept = true) - instances.foreach(agg.add) - - val agg2 = getNewAggregator(instances, Vectors.dense(coefArray ++ Array(intercept)), - fitIntercept = true) - val block = InstanceBlock.fromInstances(instances) - agg2.add(block) - - assert(agg.loss ~== agg2.loss relTol 1e-8) - assert(agg.gradient ~== agg2.gradient relTol 1e-8) - } - } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberAggregatorSuite.scala index 7c544e99f88b5..718ffa230a749 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberAggregatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HuberAggregatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,21 +32,21 @@ class HuberAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { override def beforeAll(): Unit = { super.beforeAll() - instances = standardize(Array( + instances = Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) - )) - instancesConstantFeature = standardize(Array( + ) + instancesConstantFeature = Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) - )) - instancesConstantFeatureFiltered = standardize(Array( + ) + instancesConstantFeatureFiltered = Array( Instance(0.0, 0.1, Vectors.dense(2.0)), Instance(1.0, 0.5, Vectors.dense(1.0)), Instance(2.0, 0.3, Vectors.dense(0.5)) - )) + ) } /** Get summary statistics for some data and create a new HuberAggregator. */ @@ -56,28 +56,10 @@ class HuberAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { fitIntercept: Boolean, epsilon: Double): HuberAggregator = { val (featuresSummarizer, _) = getRegressionSummarizers(instances) - val numFeatures = featuresSummarizer.variance.size + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) val bcParameters = spark.sparkContext.broadcast(parameters) - new HuberAggregator(numFeatures, fitIntercept, epsilon)(bcParameters) - } - - private def standardize( - instances: Array[Instance], - std: Array[Double] = null): Array[Instance] = { - val stdArray = if (std == null) { - getRegressionSummarizers(instances)._1.variance.toArray.map(math.sqrt) - } else { - std - } - val numFeatures = stdArray.length - instances.map { case Instance(label, weight, features) => - val standardized = Array.ofDim[Double](numFeatures) - features.foreachNonZero { (i, v) => - val std = stdArray(i) - if (std != 0) standardized(i) = v / std - } - Instance(label, weight, Vectors.dense(standardized).compressed) - } + new HuberAggregator(fitIntercept, epsilon, bcFeaturesStd)(bcParameters) } test("aggregator add method should check input size") { @@ -173,15 +155,9 @@ class HuberAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { val parametersFiltered = Vectors.dense(2.0, 3.0, 4.0) val aggConstantFeature = getNewAggregator(instancesConstantFeature, parameters, fitIntercept = true, epsilon = 1.35) - // std of instancesConstantFeature - val stdConstantFeature = getRegressionSummarizers(instancesConstantFeature) - ._1.variance.toArray.map(math.sqrt) - // Since 3.0.0, we start to standardize input outside of gradient computation, - // so here we use std of instancesConstantFeature to standardize instances - standardize(instances, stdConstantFeature).foreach(aggConstantFeature.add) - val aggConstantFeatureFiltered = getNewAggregator(instancesConstantFeatureFiltered, parametersFiltered, fitIntercept = true, epsilon = 1.35) + instances.foreach(aggConstantFeature.add) instancesConstantFeatureFiltered.foreach(aggConstantFeatureFiltered.add) // constant features should not affect gradient def validateGradient(grad: Vector, gradFiltered: Vector): Unit = { @@ -191,19 +167,4 @@ class HuberAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { validateGradient(aggConstantFeature.gradient, aggConstantFeatureFiltered.gradient) } - - test("add instance block") { - val paramWithIntercept = Vectors.dense(1.0, 2.0, 3.0, 4.0) - val agg1 = getNewAggregator(instances, paramWithIntercept, - fitIntercept = true, epsilon = 1.35) - instances.foreach(agg1.add) - - val agg2 = getNewAggregator(instances, paramWithIntercept, - fitIntercept = true, epsilon = 1.35) - val block = InstanceBlock.fromInstances(instances) - agg2.add(block) - - assert(agg1.loss ~== agg2.loss relTol 1e-8) - assert(agg1.gradient ~== agg2.gradient relTol 1e-8) - } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala index 5eb4e41c58262..35b6944624707 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,21 +32,21 @@ class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkConte override def beforeAll(): Unit = { super.beforeAll() - instances = standardize(Array( + instances = Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) - )) - instancesConstantFeature = standardize(Array( + ) + instancesConstantFeature = Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) - )) - instancesConstantLabel = standardize(Array( + ) + instancesConstantLabel = Array( Instance(1.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(1.0, 0.3, Vectors.dense(4.0, 0.5)) - )) + ) } /** Get summary statistics for some data and create a new LeastSquaresAggregator. */ @@ -57,34 +57,15 @@ class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkConte val (featuresSummarizer, ySummarizer) = getRegressionSummarizers(instances) val yStd = math.sqrt(ySummarizer.variance(0)) val yMean = ySummarizer.mean(0) - val featuresStd = Vectors.dense(featuresSummarizer.variance.toArray.map(math.sqrt)) + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) - val featuresMean = featuresSummarizer.mean.asML - val bcFeaturesMean = spark.sparkContext.broadcast(featuresMean.compressed) - val bcCoefficients = spark.sparkContext.broadcast(coefficients.compressed) + val featuresMean = featuresSummarizer.mean + val bcFeaturesMean = spark.sparkContext.broadcast(featuresMean.toArray) + val bcCoefficients = spark.sparkContext.broadcast(coefficients) new LeastSquaresAggregator(yStd, yMean, fitIntercept, bcFeaturesStd, bcFeaturesMean)(bcCoefficients) } - private def standardize( - instances: Array[Instance], - std: Array[Double] = null): Array[Instance] = { - val stdArray = if (std == null) { - getRegressionSummarizers(instances)._1.variance.toArray.map(math.sqrt) - } else { - std - } - val numFeatures = stdArray.length - instances.map { case Instance(label, weight, features) => - val standardized = Array.ofDim[Double](numFeatures) - features.foreachNonZero { (i, v) => - val std = stdArray(i) - if (std != 0) standardized(i) = v / std - } - Instance(label, weight, Vectors.dense(standardized).compressed) - } - } - test("aggregator add method input size") { val coefficients = Vectors.dense(1.0, 2.0) val agg = getNewAggregator(instances, coefficients, fitIntercept = true) @@ -164,15 +145,9 @@ class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkConte test("check with zero standard deviation") { val coefficients = Vectors.dense(1.0, 2.0) - // aggConstantFeature contains std of instancesConstantFeature, and the std of dim=0 is 0 val aggConstantFeature = getNewAggregator(instancesConstantFeature, coefficients, fitIntercept = true) - // std of instancesConstantFeature - val stdConstantFeature = getRegressionSummarizers(instancesConstantFeature) - ._1.variance.toArray.map(math.sqrt) - // Since 3.0.0, we start to standardize input outside of gradient computation, - // so here we use std of instancesConstantFeature to standardize instances - standardize(instances, stdConstantFeature).foreach(aggConstantFeature.add) + instances.foreach(aggConstantFeature.add) // constant features should not affect gradient assert(aggConstantFeature.gradient(0) === 0.0) @@ -182,17 +157,4 @@ class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkConte } } } - - test("add instance block") { - val coefficients = Vectors.dense(1.0, 2.0) - val agg1 = getNewAggregator(instances, coefficients, fitIntercept = true) - instances.foreach(agg1.add) - - val agg2 = getNewAggregator(instances, coefficients, fitIntercept = true) - val block = InstanceBlock.fromInstances(instances) - agg2.add(block) - - assert(agg1.loss ~== agg2.loss relTol 1e-8) - assert(agg1.gradient ~== agg2.gradient relTol 1e-8) - } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala index 83718076dde7b..e699adcc14c03 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.optim.aggregator import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.{Instance, InstanceBlock} +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{BLAS, Matrices, Vector, Vectors} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -32,21 +32,21 @@ class LogisticAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { override def beforeAll(): Unit = { super.beforeAll() - instances = standardize(Array( + instances = Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) - )) - instancesConstantFeature = standardize(Array( + ) + instancesConstantFeature = Array( Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) - )) - instancesConstantFeatureFiltered = standardize(Array( + ) + instancesConstantFeatureFiltered = Array( Instance(0.0, 0.1, Vectors.dense(2.0)), Instance(1.0, 0.5, Vectors.dense(1.0)), Instance(2.0, 0.3, Vectors.dense(0.5)) - )) + ) } /** Get summary statistics for some data and create a new LogisticAggregator. */ @@ -55,27 +55,13 @@ class LogisticAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { coefficients: Vector, fitIntercept: Boolean, isMultinomial: Boolean): LogisticAggregator = { - val (_, ySummarizer) = + val (featuresSummarizer, ySummarizer) = DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) val numClasses = ySummarizer.histogram.length - val numFeatures = instances.head.features.size + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) val bcCoefficients = spark.sparkContext.broadcast(coefficients) - new LogisticAggregator(numFeatures, numClasses, fitIntercept, isMultinomial)(bcCoefficients) - } - - private def standardize(instances: Array[Instance]): Array[Instance] = { - val (featuresSummarizer, _) = - DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) - val stdArray = featuresSummarizer.variance.toArray.map(math.sqrt) - val numFeatures = stdArray.length - instances.map { case Instance(label, weight, features) => - val standardized = Array.ofDim[Double](numFeatures) - features.foreachNonZero { (i, v) => - val std = stdArray(i) - if (std != 0) standardized(i) = v / std - } - Instance(label, weight, Vectors.dense(standardized).compressed) - } + new LogisticAggregator(bcFeaturesStd, numClasses, fitIntercept, isMultinomial)(bcCoefficients) } test("aggregator add method input size") { @@ -291,24 +277,4 @@ class LogisticAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { validateGradient(aggConstantFeatureBinary.gradient, aggConstantFeatureBinaryFiltered.gradient, 1) } - - test("add instance block") { - val binaryInstances = instances.map { instance => - if (instance.label <= 1.0) instance else Instance(0.0, instance.weight, instance.features) - } - val coefArray = Array(1.0, 2.0) - val intercept = 1.0 - - val agg = getNewAggregator(binaryInstances, Vectors.dense(coefArray ++ Array(intercept)), - fitIntercept = true, isMultinomial = false) - binaryInstances.foreach(agg.add) - - val agg2 = getNewAggregator(binaryInstances, Vectors.dense(coefArray ++ Array(intercept)), - fitIntercept = true, isMultinomial = false) - val block = InstanceBlock.fromInstances(binaryInstances) - agg2.add(block) - - assert(agg.loss ~== agg2.loss relTol 1e-8) - assert(agg.gradient ~== agg2.gradient relTol 1e-8) - } } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 5ee42318afd45..5ab8e606bda03 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -165,8 +165,7 @@ def predictProbability(self, value): class _LinearSVCParams(_JavaClassifierParams, HasRegParam, HasMaxIter, HasFitIntercept, HasTol, - HasStandardization, HasWeightCol, HasAggregationDepth, HasThreshold, - HasBlockSize): + HasStandardization, HasWeightCol, HasAggregationDepth, HasThreshold): """ Params for :py:class:`LinearSVC` and :py:class:`LinearSVCModel`. @@ -215,8 +214,6 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable LinearSVCModel... >>> model.getThreshold() 0.5 - >>> model.getBlockSize() - 1024 >>> model.coefficients DenseVector([0.0, -0.2792, -0.1833]) >>> model.intercept @@ -255,19 +252,18 @@ class LinearSVC(JavaClassifier, _LinearSVCParams, JavaMLWritable, JavaMLReadable def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, - aggregationDepth=2, blockSize=1024): + aggregationDepth=2): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", \ fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, \ - aggregationDepth=2, blockSize=1024): + aggregationDepth=2): """ super(LinearSVC, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LinearSVC", self.uid) self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, fitIntercept=True, - standardization=True, threshold=0.0, aggregationDepth=2, - blockSize=1024) + standardization=True, threshold=0.0, aggregationDepth=2) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -276,12 +272,12 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, - aggregationDepth=2, blockSize=1024): + aggregationDepth=2): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", \ fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, \ - aggregationDepth=2, blockSize=1024): + aggregationDepth=2): Sets params for Linear SVM Classifier. """ kwargs = self._input_kwargs @@ -346,13 +342,6 @@ def setAggregationDepth(self, value): """ return self._set(aggregationDepth=value) - @since("3.0.0") - def setBlockSize(self, value): - """ - Sets the value of :py:attr:`blockSize`. - """ - return self._set(blockSize=value) - class LinearSVCModel(JavaClassificationModel, _LinearSVCParams, JavaMLWritable, JavaMLReadable): """ @@ -388,7 +377,7 @@ def intercept(self): class _LogisticRegressionParams(_JavaProbabilisticClassifierParams, HasRegParam, HasElasticNetParam, HasMaxIter, HasFitIntercept, HasTol, HasStandardization, HasWeightCol, HasAggregationDepth, - HasThreshold, HasBlockSize): + HasThreshold): """ Params for :py:class:`LogisticRegression` and :py:class:`LogisticRegressionModel`. @@ -570,8 +559,6 @@ class LogisticRegression(JavaProbabilisticClassifier, _LogisticRegressionParams, 10 >>> blor.clear(blor.maxIter) >>> blorModel = blor.fit(bdf) - >>> blorModel.getBlockSize() - 1024 >>> blorModel.setFeaturesCol("features") LogisticRegressionModel... >>> blorModel.setProbabilityCol("newProbability") @@ -640,7 +627,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred rawPredictionCol="rawPrediction", standardization=True, weightCol=None, aggregationDepth=2, family="auto", lowerBoundsOnCoefficients=None, upperBoundsOnCoefficients=None, - lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None, blockSize=1024): + lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ @@ -649,14 +636,13 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred rawPredictionCol="rawPrediction", standardization=True, weightCol=None, \ aggregationDepth=2, family="auto", \ lowerBoundsOnCoefficients=None, upperBoundsOnCoefficients=None, \ - lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None, blockSize=1024): + lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None): If the threshold and thresholds Params are both set, they must be equivalent. """ super(LogisticRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.LogisticRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto", - blockSize=1024) + self._setDefault(maxIter=100, regParam=0.0, tol=1E-6, threshold=0.5, family="auto") kwargs = self._input_kwargs self.setParams(**kwargs) self._checkThresholdConsistency() @@ -669,7 +655,7 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre rawPredictionCol="rawPrediction", standardization=True, weightCol=None, aggregationDepth=2, family="auto", lowerBoundsOnCoefficients=None, upperBoundsOnCoefficients=None, - lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None, blockSize=1024): + lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, \ @@ -677,7 +663,7 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre rawPredictionCol="rawPrediction", standardization=True, weightCol=None, \ aggregationDepth=2, family="auto", \ lowerBoundsOnCoefficients=None, upperBoundsOnCoefficients=None, \ - lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None, blockSize=1024): + lowerBoundsOnIntercepts=None, upperBoundsOnIntercepts=None): Sets params for logistic regression. If the threshold and thresholds Params are both set, they must be equivalent. """ @@ -772,13 +758,6 @@ def setAggregationDepth(self, value): """ return self._set(aggregationDepth=value) - @since("3.0.0") - def setBlockSize(self, value): - """ - Sets the value of :py:attr:`blockSize`. - """ - return self._set(blockSize=value) - class LogisticRegressionModel(JavaProbabilisticClassificationModel, _LogisticRegressionParams, JavaMLWritable, JavaMLReadable, HasTrainingSummary): @@ -2174,7 +2153,7 @@ def sigma(self): class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, HasMaxIter, - HasTol, HasStepSize, HasSolver, HasBlockSize): + HasTol, HasStepSize, HasSolver): """ Params for :py:class:`MultilayerPerceptronClassifier`. @@ -2185,6 +2164,11 @@ class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, H "E.g., Array(780, 100, 10) means 780 inputs, one hidden layer with 100 " + "neurons and output layer of 10 neurons.", typeConverter=TypeConverters.toListInt) + blockSize = Param(Params._dummy(), "blockSize", "Block size for stacking input data in " + + "matrices. Data is stacked within partitions. If block size is more than " + + "remaining data in a partition then it is adjusted to the size of this " + + "data. Recommended size is between 10 and 1000, default is 128.", + typeConverter=TypeConverters.toInt) solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: l-bfgs, gd.", typeConverter=TypeConverters.toString) initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", @@ -2197,6 +2181,13 @@ def getLayers(self): """ return self.getOrDefault(self.layers) + @since("1.6.0") + def getBlockSize(self): + """ + Gets the value of blockSize or its default value. + """ + return self.getOrDefault(self.blockSize) + @since("2.0.0") def getInitialWeights(self): """ @@ -2220,17 +2211,11 @@ class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, _MultilayerPer ... (1.0, Vectors.dense([0.0, 1.0])), ... (1.0, Vectors.dense([1.0, 0.0])), ... (0.0, Vectors.dense([1.0, 1.0]))], ["label", "features"]) - >>> mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], seed=123) + >>> mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], blockSize=1, seed=123) >>> mlp.setMaxIter(100) MultilayerPerceptronClassifier... >>> mlp.getMaxIter() 100 - >>> mlp.getBlockSize() - 128 - >>> mlp.setBlockSize(1) - MultilayerPerceptronClassifier... - >>> mlp.getBlockSize() - 1 >>> model = mlp.fit(df) >>> model.setFeaturesCol("features") MultilayerPerceptronClassificationModel... diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index fb4d55d57a2df..ded3ca84b30f2 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -164,10 +164,7 @@ def get$Name(self): "'euclidean'", "TypeConverters.toString"), ("validationIndicatorCol", "name of the column that indicates whether each row is for " + "training or for validation. False indicates training; true indicates validation.", - None, "TypeConverters.toString"), - ("blockSize", "block size for stacking input data in matrices. Data is stacked within " - "partitions. If block size is more than remaining data in a partition then it is " - "adjusted to the size of this data.", "1024", "TypeConverters.toInt")] + None, "TypeConverters.toString")] code = [] for name, doc, defaultValueStr, typeConverter in shared: diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 456463580878c..8fc115691f1ab 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -580,21 +580,3 @@ def getValidationIndicatorCol(self): Gets the value of validationIndicatorCol or its default value. """ return self.getOrDefault(self.validationIndicatorCol) - - -class HasBlockSize(Params): - """ - Mixin for param blockSize: block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data. - """ - - blockSize = Param(Params._dummy(), "blockSize", "block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.", typeConverter=TypeConverters.toInt) - - def __init__(self): - super(HasBlockSize, self).__init__() - self._setDefault(blockSize=1024) - - def getBlockSize(self): - """ - Gets the value of blockSize or its default value. - """ - return self.getOrDefault(self.blockSize) diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index fe571e25c05f5..ee276962c898b 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -28,7 +28,7 @@ @inherit_doc -class _ALSModelParams(HasPredictionCol, HasBlockSize): +class _ALSModelParams(HasPredictionCol): """ Params for :py:class:`ALS` and :py:class:`ALSModel`. @@ -223,8 +223,6 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): 0.1 >>> als.clear(als.regParam) >>> model = als.fit(df) - >>> model.getBlockSize() - 4096 >>> model.getUserCol() 'user' >>> model.setUserCol("user") @@ -284,13 +282,13 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096): + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan"): """ __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, \ implicitPrefs=false, alpha=1.0, userCol="user", itemCol="item", seed=None, \ ratingCol="rating", nonnegative=false, checkpointInterval=10, \ intermediateStorageLevel="MEMORY_AND_DISK", \ - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", lockSize=4096) + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") """ super(ALS, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.recommendation.ALS", self.uid) @@ -298,8 +296,7 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", - blockSize=4096) + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") kwargs = self._input_kwargs self.setParams(**kwargs) @@ -309,13 +306,13 @@ def setParams(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItem implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096): + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan"): """ setParams(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, \ implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, \ ratingCol="rating", nonnegative=False, checkpointInterval=10, \ intermediateStorageLevel="MEMORY_AND_DISK", \ - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096) + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") Sets params for ALS. """ kwargs = self._input_kwargs @@ -446,13 +443,6 @@ def setSeed(self, value): """ return self._set(seed=value) - @since("3.0.0") - def setBlockSize(self, value): - """ - Sets the value of :py:attr:`blockSize`. - """ - return self._set(blockSize=value) - class ALSModel(JavaModel, _ALSModelParams, JavaMLWritable, JavaMLReadable): """ @@ -489,13 +479,6 @@ def setPredictionCol(self, value): """ return self._set(predictionCol=value) - @since("3.0.0") - def setBlockSize(self, value): - """ - Sets the value of :py:attr:`blockSize`. - """ - return self._set(blockSize=value) - @property @since("1.4.0") def rank(self): diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index a74ba3485bcd9..a4c97827d55de 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -62,7 +62,7 @@ class JavaRegressionModel(JavaPredictionModel, _JavaPredictorParams): class _LinearRegressionParams(_JavaPredictorParams, HasRegParam, HasElasticNetParam, HasMaxIter, HasTol, HasFitIntercept, HasStandardization, HasWeightCol, HasSolver, - HasAggregationDepth, HasLoss, HasBlockSize): + HasAggregationDepth, HasLoss): """ Params for :py:class:`LinearRegression` and :py:class:`LinearRegressionModel`. @@ -124,8 +124,6 @@ class LinearRegression(JavaRegressor, _LinearRegressionParams, JavaMLWritable, J >>> lr.setRegParam(0.0) LinearRegression... >>> model = lr.fit(df) - >>> model.getBlockSize() - 1024 >>> model.setFeaturesCol("features") LinearRegressionModel... >>> model.setPredictionCol("newPrediction") @@ -171,18 +169,17 @@ class LinearRegression(JavaRegressor, _LinearRegressionParams, JavaMLWritable, J def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, standardization=True, solver="auto", weightCol=None, aggregationDepth=2, - loss="squaredError", epsilon=1.35, blockSize=1024): + loss="squaredError", epsilon=1.35): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, \ standardization=True, solver="auto", weightCol=None, aggregationDepth=2, \ - loss="squaredError", epsilon=1.35, blockSize=1024) + loss="squaredError", epsilon=1.35) """ super(LinearRegression, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.regression.LinearRegression", self.uid) - self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35, - blockSize=1024) + self._setDefault(maxIter=100, regParam=0.0, tol=1e-6, loss="squaredError", epsilon=1.35) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -191,12 +188,12 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred def setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, standardization=True, solver="auto", weightCol=None, aggregationDepth=2, - loss="squaredError", epsilon=1.35, blockSize=1024): + loss="squaredError", epsilon=1.35): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxIter=100, regParam=0.0, elasticNetParam=0.0, tol=1e-6, fitIntercept=True, \ standardization=True, solver="auto", weightCol=None, aggregationDepth=2, \ - loss="squaredError", epsilon=1.35, blockSize=1024) + loss="squaredError", epsilon=1.35) Sets params for linear regression. """ kwargs = self._input_kwargs @@ -272,13 +269,6 @@ def setLoss(self, value): """ return self._set(lossType=value) - @since("3.0.0") - def setBlockSize(self, value): - """ - Sets the value of :py:attr:`blockSize`. - """ - return self._set(blockSize=value) - class LinearRegressionModel(JavaRegressionModel, _LinearRegressionParams, GeneralJavaMLWritable, JavaMLReadable, HasTrainingSummary): From 3db3e39f1122350f55f305bee049363621c5894d Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sat, 8 Feb 2020 14:10:28 -0800 Subject: [PATCH 1326/1387] [SPARK-28228][SQL] Change the default behavior for name conflict in nested WITH clause ### What changes were proposed in this pull request? This is a follow-up for #25029, in this PR we throw an AnalysisException when name conflict is detected in nested WITH clause. In this way, the config `spark.sql.legacy.ctePrecedence.enabled` should be set explicitly for the expected behavior. ### Why are the changes needed? The original change might risky to end-users, it changes behavior silently. ### Does this PR introduce any user-facing change? Yes, change the config `spark.sql.legacy.ctePrecedence.enabled` as optional. ### How was this patch tested? New UT. Closes #27454 from xuanyuanking/SPARK-28228-follow. Authored-by: Yuanjian Li Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 2 +- .../catalyst/analysis/CTESubstitution.scala | 49 ++- .../apache/spark/sql/internal/SQLConf.scala | 6 +- .../sql-tests/inputs/cte-nonlegacy.sql | 2 + .../sql-tests/results/cte-nonlegacy.sql.out | 343 ++++++++++++++++++ .../resources/sql-tests/results/cte.sql.out | 30 +- 6 files changed, 415 insertions(+), 17 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/cte-nonlegacy.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 5a5e802f6a900..be0fe32ded99b 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -101,7 +101,7 @@ license: | - Since Spark 3.0, if files or subdirectories disappear during recursive directory listing (i.e. they appear in an intermediate listing but then cannot be read or listed during later phases of the recursive directory listing, due to either concurrent file deletions or object store consistency issues) then the listing will fail with an exception unless `spark.sql.files.ignoreMissingFiles` is `true` (default `false`). In previous versions, these missing files or subdirectories would be ignored. Note that this change of behavior only applies during initial table file listing (or during `REFRESH TABLE`), not during query execution: the net change is that `spark.sql.files.ignoreMissingFiles` is now obeyed during table file listing / query planning, not only at query execution time. - - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. + - Since Spark 3.0, Spark throws an AnalysisException if name conflict is detected in the nested WITH clause by default. It forces the users to choose the specific substitution order they wanted, which is controlled by `spark.sql.legacy.ctePrecedence.enabled`. If set to false (which is recommended), inner CTE definitions take precedence over outer definitions. For example, set the config to `false`, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `2`, while setting it to `true`, the result is `1` which is the behavior in version 2.4 and earlier. - Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index 60e6bf8db06d7..d2be15d87d023 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, With} import org.apache.spark.sql.catalyst.rules.Rule @@ -28,10 +29,54 @@ import org.apache.spark.sql.internal.SQLConf.LEGACY_CTE_PRECEDENCE_ENABLED */ object CTESubstitution extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { - if (SQLConf.get.getConf(LEGACY_CTE_PRECEDENCE_ENABLED)) { + val isLegacy = SQLConf.get.getConf(LEGACY_CTE_PRECEDENCE_ENABLED) + if (isLegacy.isEmpty) { + assertNoNameConflictsInCTE(plan, inTraverse = false) + traverseAndSubstituteCTE(plan, inTraverse = false) + } else if (isLegacy.get) { legacyTraverseAndSubstituteCTE(plan) } else { - traverseAndSubstituteCTE(plan, false) + traverseAndSubstituteCTE(plan, inTraverse = false) + } + } + + /** + * Check the plan to be traversed has naming conflicts in nested CTE or not, traverse through + * child, innerChildren and subquery for the current plan. + */ + private def assertNoNameConflictsInCTE( + plan: LogicalPlan, + inTraverse: Boolean, + cteNames: Set[String] = Set.empty): Unit = { + plan.foreach { + case w @ With(child, relations) => + val newNames = relations.map { + case (cteName, _) => + if (cteNames.contains(cteName)) { + throw new AnalysisException(s"Name $cteName is ambiguous in nested CTE. " + + s"Please set ${LEGACY_CTE_PRECEDENCE_ENABLED.key} to false so that name defined " + + "in inner CTE takes precedence. See more details in SPARK-28228.") + } else { + cteName + } + }.toSet + child.transformExpressions { + case e: SubqueryExpression => + assertNoNameConflictsInCTE(e.plan, inTraverse = true, cteNames ++ newNames) + e + } + w.innerChildren.foreach { p => + assertNoNameConflictsInCTE(p, inTraverse = true, cteNames ++ newNames) + } + + case other if inTraverse => + other.transformExpressions { + case e: SubqueryExpression => + assertNoNameConflictsInCTE(e.plan, inTraverse = true, cteNames) + e + } + + case _ => } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e17a038586c33..9c736df41dc04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2098,9 +2098,11 @@ object SQLConf { val LEGACY_CTE_PRECEDENCE_ENABLED = buildConf("spark.sql.legacy.ctePrecedence.enabled") .internal() - .doc("When true, outer CTE definitions takes precedence over inner definitions.") + .doc("When true, outer CTE definitions takes precedence over inner definitions. If set to " + + "false, inner CTE definitions take precedence. The default value is empty, " + + "AnalysisException is thrown while name conflict is detected in nested CTE.") .booleanConf - .createWithDefault(false) + .createOptional val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC = buildConf("spark.sql.legacy.arrayExistsFollowsThreeValuedLogic") diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-nonlegacy.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-nonlegacy.sql new file mode 100644 index 0000000000000..b711bf338ab08 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-nonlegacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.ctePrecedence.enabled = false +--IMPORT cte.sql diff --git a/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out new file mode 100644 index 0000000000000..2d87781193c25 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out @@ -0,0 +1,343 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 27 + + +-- !query +create temporary view t as select * from values 0, 1, 2 as t(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values 0, 1 as t(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH s AS (SELECT 1 FROM s) SELECT * FROM s +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Table or view not found: s; line 1 pos 25 + + +-- !query +WITH r AS (SELECT (SELECT * FROM r)) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Table or view not found: r; line 1 pos 33 + + +-- !query +WITH t AS (SELECT 1 FROM t) SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 + + +-- !query +WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +Table or view not found: s2; line 1 pos 26 + + +-- !query +WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 +-- !query schema +struct +-- !query output +0 2 +0 2 +1 2 +1 2 + + +-- !query +WITH CTE1 AS ( + SELECT b.id AS id + FROM T2 a + CROSS JOIN (SELECT id AS id FROM T2) b +) +SELECT t1.id AS c1, + t2.id AS c2 +FROM CTE1 t1 + CROSS JOIN CTE1 t2 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 1 +0 1 +0 1 +0 1 +1 0 +1 0 +1 0 +1 0 +1 1 +1 1 +1 1 +1 1 + + +-- !query +WITH t(x) AS (SELECT 1) +SELECT * FROM t WHERE x = 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH t(x, y) AS (SELECT 1, 2) +SELECT * FROM t WHERE x = 1 AND y = 2 +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH t(x, x) AS (SELECT 1, 2) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH t() AS (SELECT 1) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'WITH t()'(line 1, pos 7) + +== SQL == +WITH t() AS (SELECT 1) +-------^^^ +SELECT * FROM t + + +-- !query +WITH + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException + +CTE definition can't have duplicate names: 't'.(line 1, pos 0) + +== SQL == +WITH +^^^ + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t + + +-- !query +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query schema +struct<2:int> +-- !query output +2 + + +-- !query +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query schema +struct<3:int> +-- !query output +3 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +3 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +3 + + +-- !query +DROP VIEW IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index 2d87781193c25..1d50aa8f57505 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -204,9 +204,10 @@ WITH ) SELECT * FROM t2 -- !query schema -struct<2:int> +struct<> -- !query output -2 +org.apache.spark.sql.AnalysisException +Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedence.enabled to false so that name defined in inner CTE takes precedence. See more details in SPARK-28228.; -- !query @@ -222,9 +223,10 @@ WITH ) SELECT * FROM t2 -- !query schema -struct +struct<> -- !query output -2 +org.apache.spark.sql.AnalysisException +Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedence.enabled to false so that name defined in inner CTE takes precedence. See more details in SPARK-28228.; -- !query @@ -240,9 +242,10 @@ WITH ) SELECT * FROM t2 -- !query schema -struct<3:int> +struct<> -- !query output -3 +org.apache.spark.sql.AnalysisException +Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedence.enabled to false so that name defined in inner CTE takes precedence. See more details in SPARK-28228.; -- !query @@ -293,9 +296,10 @@ SELECT ( SELECT * FROM t ) -- !query schema -struct +struct<> -- !query output -2 +org.apache.spark.sql.AnalysisException +Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedence.enabled to false so that name defined in inner CTE takes precedence. See more details in SPARK-28228.; -- !query @@ -307,9 +311,10 @@ SELECT ( ) ) -- !query schema -struct +struct<> -- !query output -2 +org.apache.spark.sql.AnalysisException +Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedence.enabled to false so that name defined in inner CTE takes precedence. See more details in SPARK-28228.; -- !query @@ -322,9 +327,10 @@ SELECT ( ) ) -- !query schema -struct +struct<> -- !query output -3 +org.apache.spark.sql.AnalysisException +Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedence.enabled to false so that name defined in inner CTE takes precedence. See more details in SPARK-28228.; -- !query From e1cd4d9dc25ac3abe33c07686fc2a7d1f2b5c122 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sat, 8 Feb 2020 14:28:15 -0800 Subject: [PATCH 1327/1387] [SPARK-29587][DOC][FOLLOWUP] Add `SQL` tab in the `Data Types` page ### What changes were proposed in this pull request? Add the new tab `SQL` in the `Data Types` page. ### Why are the changes needed? New type added in SPARK-29587. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Locally test by Jekyll. ![image](https://user-images.githubusercontent.com/4833765/73908593-2e511d80-48e5-11ea-85a7-6ee451e6b727.png) Closes #27447 from xuanyuanking/SPARK-29587-follow. Authored-by: Yuanjian Li Signed-off-by: Dongjoon Hyun --- docs/sql-ref-datatypes.md | 75 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 75 insertions(+) diff --git a/docs/sql-ref-datatypes.md b/docs/sql-ref-datatypes.md index 0add62b10ed6b..9700608fe8a34 100644 --- a/docs/sql-ref-datatypes.md +++ b/docs/sql-ref-datatypes.md @@ -631,4 +631,79 @@ from pyspark.sql.types import *
    + +

    + +The following table shows the type names as well as aliases used in Spark SQL parser for each data type. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeSQL name
    BooleanType BOOLEAN
    ByteType BYTE, TINYINT
    ShortType SHORT, SMALLINT
    IntegerType INT, INTEGER
    LongType LONG, BIGINT
    FloatType FLOAT, REAL
    DoubleType DOUBLE
    DateType DATE
    TimestampType TIMESTAMP
    StringType STRING
    BinaryType BINARY
    DecimalType DECIMAL, DEC, NUMERIC
    CalendarIntervalType INTERVAL
    ArrayType ARRAY<element_type>
    StructType STRUCT<field1_name: field1_type, field2_name: field2_type, ...>
    MapType MAP<key_type, value_type>
    +
    From a7ae77a8d83bfbb8de5bb0dc2a8a0485c1486614 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sun, 9 Feb 2020 13:14:30 +0800 Subject: [PATCH 1328/1387] [SPARK-30662][ML][PYSPARK] Put back the API changes for HasBlockSize in ALS/MLP ### What changes were proposed in this pull request? Add ```HasBlockSize``` in shared Params in both Scala and Python. Make ALS/MLP extend ```HasBlockSize``` ### Why are the changes needed? Add ```HasBlockSize ``` in ALS, so user can specify the blockSize. Make ```HasBlockSize``` a shared param so both ALS and MLP can use it. ### Does this PR introduce any user-facing change? Yes ```ALS.setBlockSize/getBlockSize``` ```ALSModel.setBlockSize/getBlockSize``` ### How was this patch tested? Manually tested. Also added doctest. Closes #27501 from huaxingao/spark_30662. Authored-by: Huaxin Gao Signed-off-by: zhengruifeng --- .../MultilayerPerceptronClassifier.scala | 22 +-------- .../ml/param/shared/SharedParamsCodeGen.scala | 6 ++- .../spark/ml/param/shared/sharedParams.scala | 17 +++++++ .../apache/spark/ml/recommendation/ALS.scala | 46 ++++++++++++++----- python/pyspark/ml/classification.py | 22 ++++----- .../ml/param/_shared_params_code_gen.py | 5 +- python/pyspark/ml/param/shared.py | 17 +++++++ python/pyspark/ml/recommendation.py | 29 +++++++++--- 8 files changed, 109 insertions(+), 55 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index c7a8237849b5b..6e8f92b9b1e64 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.VersionUtils.majorMinorVersion /** Params for Multilayer Perceptron. */ private[classification] trait MultilayerPerceptronParams extends ProbabilisticClassifierParams - with HasSeed with HasMaxIter with HasTol with HasStepSize with HasSolver { + with HasSeed with HasMaxIter with HasTol with HasStepSize with HasSolver with HasBlockSize { import MultilayerPerceptronClassifier._ @@ -54,26 +54,6 @@ private[classification] trait MultilayerPerceptronParams extends ProbabilisticCl @Since("1.5.0") final def getLayers: Array[Int] = $(layers) - /** - * Block size for stacking input data in matrices to speed up the computation. - * Data is stacked within partitions. If block size is more than remaining data in - * a partition then it is adjusted to the size of this data. - * Recommended size is between 10 and 1000. - * Default: 128 - * - * @group expertParam - */ - @Since("1.5.0") - final val blockSize: IntParam = new IntParam(this, "blockSize", - "Block size for stacking input data in matrices. Data is stacked within partitions." + - " If block size is more than remaining data in a partition then " + - "it is adjusted to the size of this data. Recommended size is between 10 and 1000", - ParamValidators.gt(0)) - - /** @group expertGetParam */ - @Since("1.5.0") - final def getBlockSize: Int = $(blockSize) - /** * The solver algorithm for optimization. * Supported options: "gd" (minibatch gradient descent) or "l-bfgs". diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index 7ac680ec1183a..6194dfacf01fc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -104,7 +104,11 @@ private[shared] object SharedParamsCodeGen { isValid = "ParamValidators.inArray(Array(\"euclidean\", \"cosine\"))"), ParamDesc[String]("validationIndicatorCol", "name of the column that indicates whether " + "each row is for training or for validation. False indicates training; true indicates " + - "validation.") + "validation."), + ParamDesc[Int]("blockSize", "block size for stacking input data in matrices. Data is " + + "stacked within partitions. If block size is more than remaining data in a partition " + + "then it is adjusted to the size of this data.", + isValid = "ParamValidators.gt(0)", isExpertParam = true) ) val code = genSharedParams(params) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index 44c993eeafddc..0c0d2b5981dbe 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -578,4 +578,21 @@ trait HasValidationIndicatorCol extends Params { /** @group getParam */ final def getValidationIndicatorCol: String = $(validationIndicatorCol) } + +/** + * Trait for shared param blockSize. This trait may be changed or + * removed between minor versions. + */ +@DeveloperApi +trait HasBlockSize extends Params { + + /** + * Param for block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.. + * @group expertParam + */ + final val blockSize: IntParam = new IntParam(this, "blockSize", "block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.", ParamValidators.gt(0)) + + /** @group expertGetParam */ + final def getBlockSize: Int = $(blockSize) +} // scalastyle:on diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 2fb9a276be887..002146f89e79a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -54,7 +54,8 @@ import org.apache.spark.util.random.XORShiftRandom /** * Common params for ALS and ALSModel. */ -private[recommendation] trait ALSModelParams extends Params with HasPredictionCol { +private[recommendation] trait ALSModelParams extends Params with HasPredictionCol + with HasBlockSize { /** * Param for the column name for user ids. Ids must be integers. Other * numeric types are supported for this column, but will be cast to integers as long as they @@ -125,6 +126,8 @@ private[recommendation] trait ALSModelParams extends Params with HasPredictionCo /** @group expertGetParam */ def getColdStartStrategy: String = $(coldStartStrategy).toLowerCase(Locale.ROOT) + + setDefault(blockSize -> 4096) } /** @@ -288,6 +291,15 @@ class ALSModel private[ml] ( @Since("2.2.0") def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) + /** + * Set block size for stacking input data in matrices. + * Default is 4096. + * + * @group expertSetParam + */ + @Since("3.0.0") + def setBlockSize(value: Int): this.type = set(blockSize, value) + private val predict = udf { (featuresA: Seq[Float], featuresB: Seq[Float]) => if (featuresA != null && featuresB != null) { var dotProduct = 0.0f @@ -351,7 +363,7 @@ class ALSModel private[ml] ( */ @Since("2.2.0") def recommendForAllUsers(numItems: Int): DataFrame = { - recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems) + recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) } /** @@ -366,7 +378,7 @@ class ALSModel private[ml] ( @Since("2.3.0") def recommendForUserSubset(dataset: Dataset[_], numItems: Int): DataFrame = { val srcFactorSubset = getSourceFactorSubset(dataset, userFactors, $(userCol)) - recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems) + recommendForAll(srcFactorSubset, itemFactors, $(userCol), $(itemCol), numItems, $(blockSize)) } /** @@ -377,7 +389,7 @@ class ALSModel private[ml] ( */ @Since("2.2.0") def recommendForAllItems(numUsers: Int): DataFrame = { - recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers) + recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) } /** @@ -392,7 +404,7 @@ class ALSModel private[ml] ( @Since("2.3.0") def recommendForItemSubset(dataset: Dataset[_], numUsers: Int): DataFrame = { val srcFactorSubset = getSourceFactorSubset(dataset, itemFactors, $(itemCol)) - recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers) + recommendForAll(srcFactorSubset, userFactors, $(itemCol), $(userCol), numUsers, $(blockSize)) } /** @@ -441,11 +453,12 @@ class ALSModel private[ml] ( dstFactors: DataFrame, srcOutputColumn: String, dstOutputColumn: String, - num: Int): DataFrame = { + num: Int, + blockSize: Int): DataFrame = { import srcFactors.sparkSession.implicits._ - val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])]) - val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])]) + val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])], blockSize) + val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])], blockSize) val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked) .as[(Seq[(Int, Array[Float])], Seq[(Int, Array[Float])])] .flatMap { case (srcIter, dstIter) => @@ -483,11 +496,10 @@ class ALSModel private[ml] ( /** * Blockifies factors to improve the efficiency of cross join - * TODO: SPARK-20443 - expose blockSize as a param? */ private def blockify( factors: Dataset[(Int, Array[Float])], - blockSize: Int = 4096): Dataset[Seq[(Int, Array[Float])]] = { + blockSize: Int): Dataset[Seq[(Int, Array[Float])]] = { import factors.sparkSession.implicits._ factors.mapPartitions(_.grouped(blockSize)) } @@ -654,6 +666,15 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] @Since("2.2.0") def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) + /** + * Set block size for stacking input data in matrices. + * Default is 4096. + * + * @group expertSetParam + */ + @Since("3.0.0") + def setBlockSize(value: Int): this.type = set(blockSize, value) + /** * Sets both numUserBlocks and numItemBlocks to the specific value. * @@ -683,7 +704,7 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] instr.logDataset(dataset) instr.logParams(this, rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, userCol, itemCol, ratingCol, predictionCol, maxIter, regParam, nonnegative, checkpointInterval, - seed, intermediateStorageLevel, finalStorageLevel) + seed, intermediateStorageLevel, finalStorageLevel, blockSize) val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), @@ -694,7 +715,8 @@ class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] checkpointInterval = $(checkpointInterval), seed = $(seed)) val userDF = userFactors.toDF("id", "features") val itemDF = itemFactors.toDF("id", "features") - val model = new ALSModel(uid, $(rank), userDF, itemDF).setParent(this) + val model = new ALSModel(uid, $(rank), userDF, itemDF).setBlockSize($(blockSize)) + .setParent(this) copyValues(model) } diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 5ab8e606bda03..1436b78b11202 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -2153,7 +2153,7 @@ def sigma(self): class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, HasMaxIter, - HasTol, HasStepSize, HasSolver): + HasTol, HasStepSize, HasSolver, HasBlockSize): """ Params for :py:class:`MultilayerPerceptronClassifier`. @@ -2164,11 +2164,6 @@ class _MultilayerPerceptronParams(_JavaProbabilisticClassifierParams, HasSeed, H "E.g., Array(780, 100, 10) means 780 inputs, one hidden layer with 100 " + "neurons and output layer of 10 neurons.", typeConverter=TypeConverters.toListInt) - blockSize = Param(Params._dummy(), "blockSize", "Block size for stacking input data in " + - "matrices. Data is stacked within partitions. If block size is more than " + - "remaining data in a partition then it is adjusted to the size of this " + - "data. Recommended size is between 10 and 1000, default is 128.", - typeConverter=TypeConverters.toInt) solver = Param(Params._dummy(), "solver", "The solver algorithm for optimization. Supported " + "options: l-bfgs, gd.", typeConverter=TypeConverters.toString) initialWeights = Param(Params._dummy(), "initialWeights", "The initial weights of the model.", @@ -2181,13 +2176,6 @@ def getLayers(self): """ return self.getOrDefault(self.layers) - @since("1.6.0") - def getBlockSize(self): - """ - Gets the value of blockSize or its default value. - """ - return self.getOrDefault(self.blockSize) - @since("2.0.0") def getInitialWeights(self): """ @@ -2211,11 +2199,17 @@ class MultilayerPerceptronClassifier(JavaProbabilisticClassifier, _MultilayerPer ... (1.0, Vectors.dense([0.0, 1.0])), ... (1.0, Vectors.dense([1.0, 0.0])), ... (0.0, Vectors.dense([1.0, 1.0]))], ["label", "features"]) - >>> mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], blockSize=1, seed=123) + >>> mlp = MultilayerPerceptronClassifier(layers=[2, 2, 2], seed=123) >>> mlp.setMaxIter(100) MultilayerPerceptronClassifier... >>> mlp.getMaxIter() 100 + >>> mlp.getBlockSize() + 128 + >>> mlp.setBlockSize(1) + MultilayerPerceptronClassifier... + >>> mlp.getBlockSize() + 1 >>> model = mlp.fit(df) >>> model.setFeaturesCol("features") MultilayerPerceptronClassificationModel... diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index ded3ca84b30f2..2086e831f4282 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -164,7 +164,10 @@ def get$Name(self): "'euclidean'", "TypeConverters.toString"), ("validationIndicatorCol", "name of the column that indicates whether each row is for " + "training or for validation. False indicates training; true indicates validation.", - None, "TypeConverters.toString")] + None, "TypeConverters.toString"), + ("blockSize", "block size for stacking input data in matrices. Data is stacked within " + "partitions. If block size is more than remaining data in a partition then it is " + "adjusted to the size of this data.", None, "TypeConverters.toInt")] code = [] for name, doc, defaultValueStr, typeConverter in shared: diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 8fc115691f1ab..24fb0d3e2554d 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -580,3 +580,20 @@ def getValidationIndicatorCol(self): Gets the value of validationIndicatorCol or its default value. """ return self.getOrDefault(self.validationIndicatorCol) + + +class HasBlockSize(Params): + """ + Mixin for param blockSize: block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data. + """ + + blockSize = Param(Params._dummy(), "blockSize", "block size for stacking input data in matrices. Data is stacked within partitions. If block size is more than remaining data in a partition then it is adjusted to the size of this data.", typeConverter=TypeConverters.toInt) + + def __init__(self): + super(HasBlockSize, self).__init__() + + def getBlockSize(self): + """ + Gets the value of blockSize or its default value. + """ + return self.getOrDefault(self.blockSize) diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index ee276962c898b..99d80aa867bda 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -28,7 +28,7 @@ @inherit_doc -class _ALSModelParams(HasPredictionCol): +class _ALSModelParams(HasPredictionCol, HasBlockSize): """ Params for :py:class:`ALS` and :py:class:`ALSModel`. @@ -223,6 +223,8 @@ class ALS(JavaEstimator, _ALSParams, JavaMLWritable, JavaMLReadable): 0.1 >>> als.clear(als.regParam) >>> model = als.fit(df) + >>> model.getBlockSize() + 4096 >>> model.getUserCol() 'user' >>> model.setUserCol("user") @@ -282,13 +284,13 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan"): + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096): """ __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, \ implicitPrefs=false, alpha=1.0, userCol="user", itemCol="item", seed=None, \ ratingCol="rating", nonnegative=false, checkpointInterval=10, \ intermediateStorageLevel="MEMORY_AND_DISK", \ - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096) """ super(ALS, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.recommendation.ALS", self.uid) @@ -296,7 +298,8 @@ def __init__(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemB implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", + blockSize=4096) kwargs = self._input_kwargs self.setParams(**kwargs) @@ -306,13 +309,13 @@ def setParams(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItem implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, ratingCol="rating", nonnegative=False, checkpointInterval=10, intermediateStorageLevel="MEMORY_AND_DISK", - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan"): + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096): """ setParams(self, rank=10, maxIter=10, regParam=0.1, numUserBlocks=10, numItemBlocks=10, \ implicitPrefs=False, alpha=1.0, userCol="user", itemCol="item", seed=None, \ ratingCol="rating", nonnegative=False, checkpointInterval=10, \ intermediateStorageLevel="MEMORY_AND_DISK", \ - finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan") + finalStorageLevel="MEMORY_AND_DISK", coldStartStrategy="nan", blockSize=4096) Sets params for ALS. """ kwargs = self._input_kwargs @@ -443,6 +446,13 @@ def setSeed(self, value): """ return self._set(seed=value) + @since("3.0.0") + def setBlockSize(self, value): + """ + Sets the value of :py:attr:`blockSize`. + """ + return self._set(blockSize=value) + class ALSModel(JavaModel, _ALSModelParams, JavaMLWritable, JavaMLReadable): """ @@ -479,6 +489,13 @@ def setPredictionCol(self, value): """ return self._set(predictionCol=value) + @since("3.0.0") + def setBlockSize(self, value): + """ + Sets the value of :py:attr:`blockSize`. + """ + return self._set(blockSize=value) + @property @since("1.4.0") def rank(self): From 339c0f9a623521acd4d66292b3fe3e6c4ec3b108 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sun, 9 Feb 2020 19:20:47 +0900 Subject: [PATCH 1329/1387] [SPARK-30510][SQL][DOCS] Publicly document Spark SQL configuration options ### What changes were proposed in this pull request? This PR adds a doc builder for Spark SQL's configuration options. Here's what the new Spark SQL config docs look like ([configuration.html.zip](https://github.com/apache/spark/files/4172109/configuration.html.zip)): ![Screen Shot 2020-02-07 at 12 13 23 PM](https://user-images.githubusercontent.com/1039369/74050007-425b5480-49a3-11ea-818c-42700c54d1fb.png) Compare this to the [current docs](http://spark.apache.org/docs/3.0.0-preview2/configuration.html#spark-sql): ![Screen Shot 2020-02-04 at 4 55 10 PM](https://user-images.githubusercontent.com/1039369/73790828-24a5a980-476f-11ea-998c-12cd613883e8.png) ### Why are the changes needed? There is no visibility into the various Spark SQL configs on [the config docs page](http://spark.apache.org/docs/3.0.0-preview2/configuration.html#spark-sql). ### Does this PR introduce any user-facing change? No, apart from new documentation. ### How was this patch tested? I tested this manually by building the docs and reviewing them in my browser. Closes #27459 from nchammas/SPARK-30510-spark-sql-options. Authored-by: Nicholas Chammas Signed-off-by: HyukjinKwon --- docs/.gitignore | 1 + docs/configuration.md | 46 ++----- sql/README.md | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 35 +++--- .../spark/sql/api/python/PythonSQLUtils.scala | 7 ++ sql/create-docs.sh | 14 ++- ...en-sql-markdown.py => gen-sql-api-docs.py} | 8 +- sql/gen-sql-config-docs.py | 117 ++++++++++++++++++ 8 files changed, 163 insertions(+), 67 deletions(-) create mode 100644 docs/.gitignore rename sql/{gen-sql-markdown.py => gen-sql-api-docs.py} (96%) create mode 100644 sql/gen-sql-config-docs.py diff --git a/docs/.gitignore b/docs/.gitignore new file mode 100644 index 0000000000000..2260493b46ab3 --- /dev/null +++ b/docs/.gitignore @@ -0,0 +1 @@ +sql-configs.html diff --git a/docs/configuration.md b/docs/configuration.md index 5bd3f3e80cf71..1343755f9d87f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2399,47 +2399,15 @@ the driver or executor, or, in the absence of that value, the number of cores av Please refer to the [Security](security.html) page for available options on how to secure different Spark subsystems. -### Spark SQL - -Running the SET -v command will show the entire list of the SQL configuration. - -
    -
    -{% highlight scala %} -// spark is an existing SparkSession -spark.sql("SET -v").show(numRows = 200, truncate = false) -{% endhighlight %} - -
    - -
    - -{% highlight java %} -// spark is an existing SparkSession -spark.sql("SET -v").show(200, false); -{% endhighlight %} -
    - -
    - -{% highlight python %} -# spark is an existing SparkSession -spark.sql("SET -v").show(n=200, truncate=False) -{% endhighlight %} - -
    - -
    - -{% highlight r %} -sparkR.session() -properties <- sql("SET -v") -showDF(properties, numRows = 200, truncate = FALSE) -{% endhighlight %} +{% for static_file in site.static_files %} + {% if static_file.name == 'sql-configs.html' %} +### Spark SQL -
    -
    + {% include_relative sql-configs.html %} + {% break %} + {% endif %} +{% endfor %} ### Spark Streaming diff --git a/sql/README.md b/sql/README.md index 67e3225e2c275..ae5ebd1d75370 100644 --- a/sql/README.md +++ b/sql/README.md @@ -9,4 +9,4 @@ Spark SQL is broken up into four subprojects: - Hive Support (sql/hive) - Includes extensions that allow users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allow users to run queries that include Hive UDFs, UDAFs, and UDTFs. - HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server. -Running `./sql/create-docs.sh` generates SQL documentation for built-in functions under `sql/site`. +Running `./sql/create-docs.sh` generates SQL documentation for built-in functions under `sql/site`, and SQL configuration documentation that gets included as part of `configuration.md` in the main `docs` directory. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9c736df41dc04..64c613611c861 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -324,11 +324,11 @@ object SQLConf { .doc("Configures the maximum size in bytes for a table that will be broadcast to all worker " + "nodes when performing a join. By setting this value to -1 broadcasting can be disabled. " + "Note that currently statistics are only supported for Hive Metastore tables where the " + - "command ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been " + + "command `ANALYZE TABLE COMPUTE STATISTICS noscan` has been " + "run, and file-based data source tables where the statistics are computed directly on " + "the files of data.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(10L * 1024 * 1024) + .createWithDefaultString("10MB") val LIMIT_SCALE_UP_FACTOR = buildConf("spark.sql.limit.scaleUpFactor") .internal() @@ -402,7 +402,7 @@ object SQLConf { s"an effect when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " + s"'${REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key}' is enabled.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(64 * 1024 * 1024) + .createWithDefaultString("64MB") val SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.shuffle.maxNumPostShufflePartitions") @@ -436,7 +436,7 @@ object SQLConf { .doc("Configures the minimum size in bytes for a partition that is considered as a skewed " + "partition in adaptive skewed join.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(64 * 1024 * 1024) + .createWithDefaultString("64MB") val ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR = buildConf("spark.sql.adaptive.optimizeSkewedJoin.skewedPartitionFactor") @@ -770,7 +770,7 @@ object SQLConf { val BROADCAST_TIMEOUT = buildConf("spark.sql.broadcastTimeout") .doc("Timeout in seconds for the broadcast wait time in broadcast joins.") .timeConf(TimeUnit.SECONDS) - .createWithDefault(5 * 60) + .createWithDefaultString(s"${5 * 60}") // This is only used for the thriftserver val THRIFTSERVER_POOL = buildConf("spark.sql.thriftserver.scheduler.pool") @@ -830,7 +830,7 @@ object SQLConf { .createWithDefault(true) val BUCKETING_MAX_BUCKETS = buildConf("spark.sql.sources.bucketing.maxBuckets") - .doc("The maximum number of buckets allowed. Defaults to 100000") + .doc("The maximum number of buckets allowed.") .intConf .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be greater than 0") .createWithDefault(100000) @@ -1022,7 +1022,7 @@ object SQLConf { "This configuration is effective only when using file-based sources such as Parquet, JSON " + "and ORC.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(128 * 1024 * 1024) // parquet.block.size + .createWithDefaultString("128MB") // parquet.block.size val FILES_OPEN_COST_IN_BYTES = buildConf("spark.sql.files.openCostInBytes") .internal() @@ -1161,7 +1161,8 @@ object SQLConf { val VARIABLE_SUBSTITUTE_ENABLED = buildConf("spark.sql.variable.substitute") - .doc("This enables substitution using syntax like ${var} ${system:var} and ${env:var}.") + .doc("This enables substitution using syntax like `${var}`, `${system:var}`, " + + "and `${env:var}`.") .booleanConf .createWithDefault(true) @@ -1171,7 +1172,7 @@ object SQLConf { .doc("Enable two-level aggregate hash map. When enabled, records will first be " + "inserted/looked-up at a 1st-level, small, fast map, and then fallback to a " + "2nd-level, larger, slower map when 1st level is full or keys cannot be found. " + - "When disabled, records go directly to the 2nd level. Defaults to true.") + "When disabled, records go directly to the 2nd level.") .booleanConf .createWithDefault(true) @@ -1325,10 +1326,10 @@ object SQLConf { val STREAMING_STOP_TIMEOUT = buildConf("spark.sql.streaming.stopTimeout") - .doc("How long to wait for the streaming execution thread to stop when calling the " + - "streaming query's stop() method in milliseconds. 0 or negative values wait indefinitely.") + .doc("How long to wait in milliseconds for the streaming execution thread to stop when " + + "calling the streaming query's stop() method. 0 or negative values wait indefinitely.") .timeConf(TimeUnit.MILLISECONDS) - .createWithDefault(0L) + .createWithDefaultString("0") val STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL = buildConf("spark.sql.streaming.noDataProgressEventInterval") @@ -1611,10 +1612,10 @@ object SQLConf { val PANDAS_UDF_BUFFER_SIZE = buildConf("spark.sql.execution.pandas.udf.buffer.size") .doc( - s"Same as ${BUFFER_SIZE} but only applies to Pandas UDF executions. If it is not set, " + - s"the fallback is ${BUFFER_SIZE}. Note that Pandas execution requires more than 4 bytes. " + - "Lowering this value could make small Pandas UDF batch iterated and pipelined; however, " + - "it might degrade performance. See SPARK-27870.") + s"Same as `${BUFFER_SIZE.key}` but only applies to Pandas UDF executions. If it is not " + + s"set, the fallback is `${BUFFER_SIZE.key}`. Note that Pandas execution requires more " + + "than 4 bytes. Lowering this value could make small Pandas UDF batch iterated and " + + "pipelined; however, it might degrade performance. See SPARK-27870.") .fallbackConf(BUFFER_SIZE) val PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME = @@ -2039,7 +2040,7 @@ object SQLConf { .checkValue(i => i >= 0 && i <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, "Invalid " + "value for 'spark.sql.maxPlanStringLength'. Length must be a valid string length " + "(nonnegative and shorter than the maximum size).") - .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) + .createWithDefaultString(s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}") val SET_COMMAND_REJECTS_SPARK_CORE_CONFS = buildConf("spark.sql.legacy.setCommandRejectsSparkCoreConfs") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index b232aa18c816e..bf3055d5e3e09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.{ExplainMode, QueryExecution} import org.apache.spark.sql.execution.arrow.ArrowConverters +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.DataType private[sql] object PythonSQLUtils { @@ -39,6 +40,12 @@ private[sql] object PythonSQLUtils { FunctionRegistry.functionSet.flatMap(f => FunctionRegistry.builtin.lookupFunction(f)).toArray } + def listSQLConfigs(): Array[(String, String, String)] = { + val conf = new SQLConf() + // Py4J doesn't seem to translate Seq well, so we convert to an Array. + conf.getAllDefinedConfs.toArray + } + /** * Python callable function to read a file in Arrow stream format and create a [[RDD]] * using each serialized ArrowRecordBatch as a partition. diff --git a/sql/create-docs.sh b/sql/create-docs.sh index 4353708d22f7b..44aa877332fd5 100755 --- a/sql/create-docs.sh +++ b/sql/create-docs.sh @@ -17,7 +17,7 @@ # limitations under the License. # -# Script to create SQL API docs. This requires `mkdocs` and to build +# Script to create SQL API and config docs. This requires `mkdocs` and to build # Spark first. After running this script the html docs can be found in # $SPARK_HOME/sql/site @@ -39,14 +39,16 @@ fi pushd "$FWDIR" > /dev/null -# Now create the markdown file rm -fr docs mkdir docs -echo "Generating markdown files for SQL documentation." -"$SPARK_HOME/bin/spark-submit" gen-sql-markdown.py -# Now create the HTML files -echo "Generating HTML files for SQL documentation." +echo "Generating SQL API Markdown files." +"$SPARK_HOME/bin/spark-submit" gen-sql-api-docs.py + +echo "Generating SQL configuration table HTML file." +"$SPARK_HOME/bin/spark-submit" gen-sql-config-docs.py + +echo "Generating HTML files for SQL API documentation." mkdocs build --clean rm -fr docs diff --git a/sql/gen-sql-markdown.py b/sql/gen-sql-api-docs.py similarity index 96% rename from sql/gen-sql-markdown.py rename to sql/gen-sql-api-docs.py index e0529f8310613..4feee7ad52570 100644 --- a/sql/gen-sql-markdown.py +++ b/sql/gen-sql-api-docs.py @@ -15,10 +15,11 @@ # limitations under the License. # -import sys import os from collections import namedtuple +from pyspark.java_gateway import launch_gateway + ExpressionInfo = namedtuple( "ExpressionInfo", "className name usage arguments examples note since deprecated") @@ -219,8 +220,7 @@ def generate_sql_markdown(jvm, path): if __name__ == "__main__": - from pyspark.java_gateway import launch_gateway - jvm = launch_gateway().jvm - markdown_file_path = "%s/docs/index.md" % os.path.dirname(sys.argv[0]) + spark_root_dir = os.path.dirname(os.path.dirname(__file__)) + markdown_file_path = os.path.join(spark_root_dir, "sql/docs/index.md") generate_sql_markdown(jvm, markdown_file_path) diff --git a/sql/gen-sql-config-docs.py b/sql/gen-sql-config-docs.py new file mode 100644 index 0000000000000..04f5a850c9980 --- /dev/null +++ b/sql/gen-sql-config-docs.py @@ -0,0 +1,117 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import os +import re +from collections import namedtuple +from textwrap import dedent + +# To avoid adding a new direct dependency, we import markdown from within mkdocs. +from mkdocs.structure.pages import markdown +from pyspark.java_gateway import launch_gateway + +SQLConfEntry = namedtuple( + "SQLConfEntry", ["name", "default", "description"]) + + +def get_public_sql_configs(jvm): + sql_configs = [ + SQLConfEntry( + name=_sql_config._1(), + default=_sql_config._2(), + description=_sql_config._3(), + ) + for _sql_config in jvm.org.apache.spark.sql.api.python.PythonSQLUtils.listSQLConfigs() + ] + return sql_configs + + +def generate_sql_configs_table(sql_configs, path): + """ + Generates an HTML table at `path` that lists all public SQL + configuration options. + + The table will look something like this: + + ```html + + + + + + + + + + ... + +
    Property NameDefaultMeaning
    spark.sql.adaptive.enabledfalse

    When true, enable adaptive query execution.

    + ``` + """ + value_reference_pattern = re.compile(r"^$") + + with open(path, 'w') as f: + f.write(dedent( + """ + + + """ + )) + for config in sorted(sql_configs, key=lambda x: x.name): + if config.default == "": + default = "(none)" + elif config.default.startswith(" + + + + + """ + .format( + name=config.name, + default=default, + description=markdown.markdown(config.description), + ) + )) + f.write("
    Property NameDefaultMeaning
    {name}{default}{description}
    \n") + + +if __name__ == "__main__": + jvm = launch_gateway().jvm + sql_configs = get_public_sql_configs(jvm) + + spark_root_dir = os.path.dirname(os.path.dirname(__file__)) + sql_configs_table_path = os.path.join(spark_root_dir, "docs/sql-configs.html") + + generate_sql_configs_table(sql_configs, path=sql_configs_table_path) From b877aac14657832d1b896ea57e06b0d0fd15ee01 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sun, 9 Feb 2020 14:18:51 -0800 Subject: [PATCH 1330/1387] [SPARK-30684 ][WEBUI][FollowUp] A new approach for SPARK-30684 ### What changes were proposed in this pull request? Simplify the changes for adding metrics description for WholeStageCodegen in https://github.com/apache/spark/pull/27405 ### Why are the changes needed? In https://github.com/apache/spark/pull/27405, the UI changes can be made without using the function `adjustPositionOfOperationName` to adjust the position of operation name and mark as an operation-name class. I suggest we make simpler changes so that it would be easier for future development. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manual test with the queries provided in https://github.com/apache/spark/pull/27405 ``` sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").show sc.parallelize(1 to 10).toDF.sort("value").filter("value > 1").selectExpr("value * 2").write.format("json").mode("overwrite").save("/tmp/test_output") sc.parallelize(1 to 10).toDF.write.format("json").mode("append").save("/tmp/test_output") ``` ![image](https://user-images.githubusercontent.com/1097932/74073629-e3f09f00-49bf-11ea-90dc-1edb5ca29e5e.png) Closes #27490 from gengliangwang/wholeCodegenUI. Authored-by: Gengliang Wang Signed-off-by: Gengliang Wang --- .../sql/execution/ui/static/spark-sql-viz.css | 8 ++--- .../sql/execution/ui/static/spark-sql-viz.js | 31 +------------------ .../sql/execution/ui/SparkPlanGraph.scala | 18 +++++------ 3 files changed, 11 insertions(+), 46 deletions(-) diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css index eff0142dc523f..20188387c9ba4 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css @@ -18,6 +18,7 @@ #plan-viz-graph .label { font-weight: normal; text-shadow: none; + color: black; } #plan-viz-graph svg g.cluster rect { @@ -32,13 +33,8 @@ stroke-width: 1px; } -/* This declaration is needed to define the width of rectangles */ -#plan-viz-graph svg text :first-child { - font-weight: bold; -} - /* Highlight the SparkPlan node name */ -#plan-viz-graph svg text .operator-name { +#plan-viz-graph svg text :first-child { font-weight: bold; } diff --git a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js index e6ce641a841b3..c8349149439c8 100644 --- a/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js +++ b/sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js @@ -34,7 +34,6 @@ function renderPlanViz() { preprocessGraphLayout(g); var renderer = new dagreD3.render(); renderer(graph, g); - adjustPositionOfOperationName(); // Round corners on rectangles svg @@ -82,7 +81,7 @@ function setupTooltipForSparkPlanNode(nodeId) { * and sizes of graph elements, e.g. padding, font style, shape. */ function preprocessGraphLayout(g) { - g.graph().ranksep = "90"; + g.graph().ranksep = "70"; var nodes = g.nodes(); for (var i = 0; i < nodes.length; i++) { var node = g.node(nodes[i]); @@ -129,34 +128,6 @@ function resizeSvg(svg) { .attr("height", height); } - -/* Helper function to adjust the position of operation name and mark as a operation-name class. */ -function adjustPositionOfOperationName() { - $("#plan-viz-graph svg text") - .each(function() { - var tspans = $(this).find("tspan"); - - if (tspans[0].textContent.trim() !== "") { - var isOperationNameOnly = - $(tspans).filter(function(i, n) { - return i !== 0 && n.textContent.trim() !== ""; - }).length === 0; - - if (isOperationNameOnly) { - // If the only text in a node is operation name, - // vertically centering the position of operation name. - var operationName = tspans[0].textContent; - var half = parseInt(tspans.length / 2); - tspans[0].textContent = tspans[half].textContent; - tspans[half].textContent = operationName; - $(tspans[half]).addClass("operator-name"); - } else { - tspans.first().addClass("operator-name"); - } - } - }); -} - /* Helper function to convert attributes to numeric values. */ function toFloat(f) { if (f) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index 1e767c3c043c3..d31d77840b802 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -169,20 +169,18 @@ private[ui] class SparkPlanGraphNode( metric.name + ": " + value } - // If there are metrics, display each entry in a separate line. - // Note: whitespace between two "\n"s is to create an empty line between the name of - // SparkPlan and metrics. If removing it, it won't display the empty line in UI. - builder ++= "\n \n" - if (values.nonEmpty) { + // If there are metrics, display each entry in a separate line. + // Note: whitespace between two "\n"s is to create an empty line between the name of + // SparkPlan and metrics. If removing it, it won't display the empty line in UI. + builder ++= "\n \n" builder ++= values.mkString("\n") + s""" $id [label="${StringEscapeUtils.escapeJava(builder.toString())}"];""" } else { - // A certain level of height is needed for a rect as a node in a sub-graph - // to avoid layout collapse for sub-graphs. - builder ++= " " + // SPARK-30684: when there is no metrics, add empty lines to increase the height of the node, + // so that there won't be gaps between an edge and a small node. + s""" $id [labelType="html" label="
    $name

    "];""" } - - s""" $id [label="${StringEscapeUtils.escapeJava(builder.toString())}"];""" } } From 9f8172e96a8ee60cd42545778c01d98b6902161f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 9 Feb 2020 19:45:16 -0800 Subject: [PATCH 1331/1387] Revert "[SPARK-29721][SQL] Prune unnecessary nested fields from Generate without Project This reverts commit a0e63b61e7c5d55ae2a9213b95ab1e87ac7c203c. ### What changes were proposed in this pull request? This reverts the patch at #26978 based on gatorsmile's suggestion. ### Why are the changes needed? Original patch #26978 has not considered a corner case. We may need to put more time on ensuring we can cover all cases. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test. Closes #27504 from viirya/revert-SPARK-29721. Authored-by: Liang-Chi Hsieh Signed-off-by: Xiao Li --- .../optimizer/NestedColumnAliasing.scala | 47 ------------------- .../sql/catalyst/optimizer/Optimizer.scala | 43 ++++++++++------- .../datasources/SchemaPruningSuite.scala | 32 ------------- 3 files changed, 25 insertions(+), 97 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index ea85014a37bd8..43a6006f9b5c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -155,53 +155,6 @@ object NestedColumnAliasing { case MapType(keyType, valueType, _) => totalFieldNum(keyType) + totalFieldNum(valueType) case _ => 1 // UDT and others } -} - -/** - * This prunes unnessary nested columns from `Generate` and optional `Project` on top - * of it. - */ -object GeneratorNestedColumnAliasing { - def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { - // Either `nestedPruningOnExpressions` or `nestedSchemaPruningEnabled` is enabled, we - // need to prune nested columns through Project and under Generate. The difference is - // when `nestedSchemaPruningEnabled` is on, nested columns will be pruned further at - // file format readers if it is supported. - case Project(projectList, g: Generate) if (SQLConf.get.nestedPruningOnExpressions || - SQLConf.get.nestedSchemaPruningEnabled) && canPruneGenerator(g.generator) => - // On top on `Generate`, a `Project` that might have nested column accessors. - // We try to get alias maps for both project list and generator's children expressions. - NestedColumnAliasing.getAliasSubMap(projectList ++ g.generator.children).map { - case (nestedFieldToAlias, attrToAliases) => - val newChild = pruneGenerate(g, nestedFieldToAlias, attrToAliases) - Project(NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias), newChild) - } - - case g: Generate if SQLConf.get.nestedSchemaPruningEnabled && - canPruneGenerator(g.generator) => - NestedColumnAliasing.getAliasSubMap(g.generator.children).map { - case (nestedFieldToAlias, attrToAliases) => - pruneGenerate(g, nestedFieldToAlias, attrToAliases) - } - - case _ => - None - } - - private def pruneGenerate( - g: Generate, - nestedFieldToAlias: Map[ExtractValue, Alias], - attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = { - val newGenerator = g.generator.transform { - case f: ExtractValue if nestedFieldToAlias.contains(f) => - nestedFieldToAlias(f).toAttribute - }.asInstanceOf[Generator] - - // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. - val newGenerate = g.copy(generator = newGenerator) - - NestedColumnAliasing.replaceChildrenWithAliases(newGenerate, attrToAliases) - } /** * This is a while-list for pruning nested fields at `Generator`. 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 0fdf6b022d885..935d62015afa1 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 @@ -598,24 +598,31 @@ object ColumnPruning extends Rule[LogicalPlan] { s.copy(child = prunedChild(child, s.references)) // prune unrequired references - case p @ Project(_, g: Generate) => - val currP = if (p.references != g.outputSet) { - val requiredAttrs = p.references -- g.producedAttributes ++ g.generator.references - val newChild = prunedChild(g.child, requiredAttrs) - val unrequired = g.generator.references -- p.references - val unrequiredIndices = newChild.output.zipWithIndex.filter(t => unrequired.contains(t._1)) - .map(_._2) - p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices)) - } else { - p - } - // If we can prune nested column on Project + Generate, do it now. - // Otherwise by transforming down to Generate, it could be pruned individually, - // and causes nested column on top Project unable to resolve. - GeneratorNestedColumnAliasing.unapply(currP).getOrElse(currP) - - // prune unrequired nested fields from `Generate`. - case GeneratorNestedColumnAliasing(p) => p + case p @ Project(_, g: Generate) if p.references != g.outputSet => + val requiredAttrs = p.references -- g.producedAttributes ++ g.generator.references + val newChild = prunedChild(g.child, requiredAttrs) + val unrequired = g.generator.references -- p.references + val unrequiredIndices = newChild.output.zipWithIndex.filter(t => unrequired.contains(t._1)) + .map(_._2) + p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices)) + + // prune unrequired nested fields + case p @ Project(projectList, g: Generate) if SQLConf.get.nestedPruningOnExpressions && + NestedColumnAliasing.canPruneGenerator(g.generator) => + NestedColumnAliasing.getAliasSubMap(projectList ++ g.generator.children).map { + case (nestedFieldToAlias, attrToAliases) => + val newGenerator = g.generator.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f) => + nestedFieldToAlias(f).toAttribute + }.asInstanceOf[Generator] + + // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. + val newGenerate = g.copy(generator = newGenerator) + + val newChild = NestedColumnAliasing.replaceChildrenWithAliases(newGenerate, attrToAliases) + + Project(NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias), newChild) + }.getOrElse(p) // Eliminate unneeded attributes from right side of a Left Existence Join. case j @ Join(_, right, LeftExistence(_), _, _) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 5977e867f788a..a3d4905e82cee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -301,38 +301,6 @@ abstract class SchemaPruningSuite checkAnswer(query, Row("Y.", 1) :: Row("X.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) } - testSchemaPruning("select explode of nested field of array of struct") { - // Config combinations - val configs = Seq((true, true), (true, false), (false, true), (false, false)) - - configs.foreach { case (nestedPruning, nestedPruningOnExpr) => - withSQLConf( - SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> nestedPruning.toString, - SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> nestedPruningOnExpr.toString) { - val query1 = spark.table("contacts") - .select(explode(col("friends.first"))) - if (nestedPruning) { - // If `NESTED_SCHEMA_PRUNING_ENABLED` is enabled, - // even disabling `NESTED_PRUNING_ON_EXPRESSIONS`, - // nested schema is still pruned at scan node. - checkScan(query1, "struct>>") - } else { - checkScan(query1, "struct>>") - } - checkAnswer(query1, Row("Susan") :: Nil) - - val query2 = spark.table("contacts") - .select(explode(col("friends.first")), col("friends.middle")) - if (nestedPruning) { - checkScan(query2, "struct>>") - } else { - checkScan(query2, "struct>>") - } - checkAnswer(query2, Row("Susan", Array("Z.")) :: Nil) - } - } - } - protected def testSchemaPruning(testName: String)(testThunk: => Unit): Unit = { test(s"Spark vectorized reader - without partition data column - $testName") { withSQLConf(vectorizedReaderEnabledKey -> "true") { From 58b9ca1e6f7768b23e752dabc30468c06d0e1c57 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 10 Feb 2020 16:23:44 +0900 Subject: [PATCH 1332/1387] [SPARK-30592][SQL][FOLLOWUP] Add some round-trip test cases ### What changes were proposed in this pull request? Add round-trip tests for CSV and JSON functions as https://github.com/apache/spark/pull/27317#discussion_r376745135 asked. ### Why are the changes needed? improve test coverage ### Does this PR introduce any user-facing change? no ### How was this patch tested? add uts Closes #27510 from yaooqinn/SPARK-30592-F. Authored-by: Kent Yao Signed-off-by: HyukjinKwon --- .../resources/sql-tests/inputs/interval.sql | 14 +++++-- .../sql-tests/results/ansi/interval.sql.out | 38 ++++++++----------- .../sql-tests/results/interval.sql.out | 38 ++++++++----------- 3 files changed, 40 insertions(+), 50 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/interval.sql index fb6c485f619ae..a4e621e9639d4 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql @@ -222,7 +222,13 @@ select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); -- interval support for csv and json functions -SELECT from_csv('1, 1 day', 'a INT, b interval'); -SELECT to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)); -SELECT from_json('{"a":"1 days"}', 'a interval'); -SELECT to_json(map('a', interval 25 month 100 day 130 minute)); +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)), 'a interval, b interval'); +SELECT + from_json('{"a":"1 days"}', 'a interval'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_json(map('a', interval 25 month 100 day 130 minute)), + from_json(to_json(map('a', interval 25 month 100 day 130 minute)), 'a interval'); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index f37049064d989..7fdb4c53d1dcb 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 101 +-- Number of queries: 99 -- !query @@ -988,32 +988,24 @@ integer overflow -- !query -SELECT from_csv('1, 1 day', 'a INT, b interval') --- !query schema -struct> --- !query output -{"a":1,"b":1 days} - - --- !query -SELECT to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)) --- !query schema -struct --- !query output -2 years 8 months,1 hours 10 minutes - - --- !query -SELECT from_json('{"a":"1 days"}', 'a interval') +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)), 'a interval, b interval') -- !query schema -struct> +struct,to_csv(from_csv(1, 1 day)):string,to_csv(named_struct(a, INTERVAL '2 years 8 months', b, INTERVAL '1 hours 10 minutes')):string,from_csv(to_csv(named_struct(a, INTERVAL '2 years 8 months', b, INTERVAL '1 hours 10 minutes'))):struct> -- !query output -{"a":1 days} +{"a":1,"b":1 days} 1,1 days 2 years 8 months,1 hours 10 minutes {"a":2 years 8 months,"b":1 hours 10 minutes} -- !query -SELECT to_json(map('a', interval 25 month 100 day 130 minute)) +SELECT + from_json('{"a":"1 days"}', 'a interval'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_json(map('a', interval 25 month 100 day 130 minute)), + from_json(to_json(map('a', interval 25 month 100 day 130 minute)), 'a interval') -- !query schema -struct +struct,to_json(from_json({"a":"1 days"})):string,to_json(map(a, INTERVAL '2 years 1 months 100 days 2 hours 10 minutes')):string,from_json(to_json(map(a, INTERVAL '2 years 1 months 100 days 2 hours 10 minutes'))):struct> -- !query output -{"a":"2 years 1 months 100 days 2 hours 10 minutes"} +{"a":1 days} {"a":"1 days"} {"a":"2 years 1 months 100 days 2 hours 10 minutes"} {"a":2 years 1 months 100 days 2 hours 10 minutes} diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 94b4f15815ca5..3c4b4301d0025 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 101 +-- Number of queries: 99 -- !query @@ -969,32 +969,24 @@ integer overflow -- !query -SELECT from_csv('1, 1 day', 'a INT, b interval') --- !query schema -struct> --- !query output -{"a":1,"b":1 days} - - --- !query -SELECT to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)) --- !query schema -struct --- !query output -2 years 8 months,1 hours 10 minutes - - --- !query -SELECT from_json('{"a":"1 days"}', 'a interval') +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 month, 'b', interval 70 minute)), 'a interval, b interval') -- !query schema -struct> +struct,to_csv(from_csv(1, 1 day)):string,to_csv(named_struct(a, INTERVAL '2 years 8 months', b, INTERVAL '1 hours 10 minutes')):string,from_csv(to_csv(named_struct(a, INTERVAL '2 years 8 months', b, INTERVAL '1 hours 10 minutes'))):struct> -- !query output -{"a":1 days} +{"a":1,"b":1 days} 1,1 days 2 years 8 months,1 hours 10 minutes {"a":2 years 8 months,"b":1 hours 10 minutes} -- !query -SELECT to_json(map('a', interval 25 month 100 day 130 minute)) +SELECT + from_json('{"a":"1 days"}', 'a interval'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_json(map('a', interval 25 month 100 day 130 minute)), + from_json(to_json(map('a', interval 25 month 100 day 130 minute)), 'a interval') -- !query schema -struct +struct,to_json(from_json({"a":"1 days"})):string,to_json(map(a, INTERVAL '2 years 1 months 100 days 2 hours 10 minutes')):string,from_json(to_json(map(a, INTERVAL '2 years 1 months 100 days 2 hours 10 minutes'))):struct> -- !query output -{"a":"2 years 1 months 100 days 2 hours 10 minutes"} +{"a":1 days} {"a":"1 days"} {"a":"2 years 1 months 100 days 2 hours 10 minutes"} {"a":2 years 1 months 100 days 2 hours 10 minutes} From 70e545a94d47afb2848c24e81c908d28d41016da Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Mon, 10 Feb 2020 19:04:49 +0800 Subject: [PATCH 1333/1387] [SPARK-30757][SQL][DOC] Update the doc on TableCatalog.alterTable's behavior ### What changes were proposed in this pull request? This PR updates the documentation on `TableCatalog.alterTable`s behavior on the order by which the requested changes are applied. It now explicitly mentions that the changes are applied in the order given. ### Why are the changes needed? The current documentation on `TableCatalog.alterTable` doesn't mention which order the requested changes are applied. It will be useful to explicitly document this behavior so that the user can expect the behavior. For example, `REPLACE COLUMNS` needs to delete columns before adding new columns, and if the order is guaranteed by `alterTable`, it's much easier to work with the catalog API. ### Does this PR introduce any user-facing change? Yes, document change. ### How was this patch tested? Not added (doc changes). Closes #27496 from imback82/catalog_table_alter_table. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/connector/catalog/TableCatalog.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java index a69b23bf84d0c..2f102348ec517 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java @@ -134,6 +134,8 @@ Table createTable( * Implementations may reject the requested changes. If any change is rejected, none of the * changes should be applied to the table. *

    + * The requested changes must be applied in the order given. + *

    * If the catalog supports views and contains a view for the identifier and not a table, this * must throw {@link NoSuchTableException}. * From 5a240603fd920e3cb5d9ef49c31d46df8a630d8c Mon Sep 17 00:00:00 2001 From: jiake Date: Mon, 10 Feb 2020 21:48:00 +0800 Subject: [PATCH 1334/1387] [SPARK-30719][SQL] Add unit test to verify the log warning print when intentionally skip AQE ### What changes were proposed in this pull request? This is a follow up in [#27452](https://github.com/apache/spark/pull/27452). Add a unit test to verify whether the log warning is print when intentionally skip AQE. ### Why are the changes needed? Add unit test ### Does this PR introduce any user-facing change? No ### How was this patch tested? adding unit test Closes #27515 from JkSelf/aqeLoggingWarningTest. Authored-by: jiake Signed-off-by: Wenchen Fan --- .../adaptive/AdaptiveQueryExecSuite.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 96e977221e512..a2071903bea7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -789,4 +789,19 @@ class AdaptiveQueryExecSuite assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) } } + + test("SPARK-30719: do not log warning if intentionally skip AQE") { + val testAppender = new LogAppender("aqe logging warning test when skip") + withLogAppender(testAppender) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val plan = sql("SELECT * FROM testData").queryExecution.executedPlan + assert(!plan.isInstanceOf[AdaptiveSparkPlanExec]) + } + } + assert(!testAppender.loggingEvents + .exists(msg => msg.getRenderedMessage.contains( + s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is" + + s" enabled but is not supported for"))) + } } From b2011a295bd78b3693a516e049e90250366b8f52 Mon Sep 17 00:00:00 2001 From: Eric Wu <492960551@qq.com> Date: Mon, 10 Feb 2020 23:41:39 +0800 Subject: [PATCH 1335/1387] [SPARK-30326][SQL] Raise exception if analyzer exceed max iterations ### What changes were proposed in this pull request? Enhance RuleExecutor strategy to take different actions when exceeding max iterations. And raise exception if analyzer exceed max iterations. ### Why are the changes needed? Currently, both analyzer and optimizer just log warning message if rule execution exceed max iterations. They should have different behavior. Analyzer should raise exception to indicates the plan is not fixed after max iterations, while optimizer just log warning to keep the current plan. This is more feasible after SPARK-30138 was introduced. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Add test in AnalysisSuite Closes #26977 from Eric5553/EnhanceMaxIterations. Authored-by: Eric Wu <492960551@qq.com> Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 10 ++++++- .../sql/catalyst/optimizer/Optimizer.scala | 5 +++- .../sql/catalyst/rules/RuleExecutor.scala | 27 ++++++++++++++++--- .../sql/catalyst/analysis/AnalysisSuite.scala | 25 ++++++++++++++++- 4 files changed, 60 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 75f1aa7185ef3..ce82b3b567b54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -176,7 +176,15 @@ class Analyzer( def resolver: Resolver = conf.resolver - protected val fixedPoint = FixedPoint(maxIterations) + /** + * If the plan cannot be resolved within maxIterations, analyzer will throw exception to inform + * user to increase the value of SQLConf.ANALYZER_MAX_ITERATIONS. + */ + protected val fixedPoint = + FixedPoint( + maxIterations, + errorOnExceed = true, + maxIterationsSetting = SQLConf.ANALYZER_MAX_ITERATIONS.key) /** * Override to provide additional rules for the "Resolution" batch. 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 935d62015afa1..08acac18f48bb 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 @@ -53,7 +53,10 @@ abstract class Optimizer(catalogManager: CatalogManager) "PartitionPruning", "Extract Python UDFs") - protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations) + protected def fixedPoint = + FixedPoint( + SQLConf.get.optimizerMaxIterations, + maxIterationsSetting = SQLConf.OPTIMIZER_MAX_ITERATIONS.key) /** * Defines the default rule batches in the Optimizer. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 287ae0e8e9f67..da5242bee28e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -45,7 +45,17 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { * An execution strategy for rules that indicates the maximum number of executions. If the * execution reaches fix point (i.e. converge) before maxIterations, it will stop. */ - abstract class Strategy { def maxIterations: Int } + abstract class Strategy { + + /** The maximum number of executions. */ + def maxIterations: Int + + /** Whether to throw exception when exceeding the maximum number. */ + def errorOnExceed: Boolean = false + + /** The key of SQLConf setting to tune maxIterations */ + def maxIterationsSetting: String = null + } /** A strategy that is run once and idempotent. */ case object Once extends Strategy { val maxIterations = 1 } @@ -54,7 +64,10 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { * A strategy that runs until fix point or maxIterations times, whichever comes first. * Especially, a FixedPoint(1) batch is supposed to run only once. */ - case class FixedPoint(maxIterations: Int) extends Strategy + case class FixedPoint( + override val maxIterations: Int, + override val errorOnExceed: Boolean = false, + override val maxIterationsSetting: String = null) extends Strategy /** A batch of rules. */ protected case class Batch(name: String, strategy: Strategy, rules: Rule[TreeType]*) @@ -155,8 +168,14 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { if (iteration > batch.strategy.maxIterations) { // Only log if this is a rule that is supposed to run more than once. if (iteration != 2) { - val message = s"Max iterations (${iteration - 1}) reached for batch ${batch.name}" - if (Utils.isTesting) { + val endingMsg = if (batch.strategy.maxIterationsSetting == null) { + "." + } else { + s", please set '${batch.strategy.maxIterationsSetting}' to a larger value." + } + val message = s"Max iterations (${iteration - 1}) reached for batch ${batch.name}" + + s"$endingMsg" + if (Utils.isTesting || batch.strategy.errorOnExceed) { throw new TreeNodeException(curPlan, message, null) } else { logWarning(message) 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 c747d394b1bc2..d38513319388b 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 @@ -25,9 +25,10 @@ import org.scalatest.Matchers import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Count, Sum} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan @@ -745,4 +746,26 @@ class AnalysisSuite extends AnalysisTest with Matchers { CollectMetrics("evt1", sumWithFilter :: Nil, testRelation), "aggregates with filter predicate are not allowed" :: Nil) } + + test("Analysis exceed max iterations") { + // RuleExecutor only throw exception or log warning when the rule is supposed to run + // more than once. + val maxIterations = 2 + val conf = new SQLConf().copy(SQLConf.ANALYZER_MAX_ITERATIONS -> maxIterations) + val testAnalyzer = new Analyzer( + new SessionCatalog(new InMemoryCatalog, FunctionRegistry.builtin, conf), conf) + + val plan = testRelation2.select( + $"a" / Literal(2) as "div1", + $"a" / $"b" as "div2", + $"a" / $"c" as "div3", + $"a" / $"d" as "div4", + $"e" / $"e" as "div5") + + val message = intercept[TreeNodeException[LogicalPlan]] { + testAnalyzer.execute(plan) + }.getMessage + assert(message.startsWith(s"Max iterations ($maxIterations) reached for batch Resolution, " + + s"please set '${SQLConf.ANALYZER_MAX_ITERATIONS.key}' to a larger value.")) + } } From acfdb46a60fc06dac0af55951492d74b7073f546 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 10 Feb 2020 10:45:00 -0800 Subject: [PATCH 1336/1387] [SPARK-27946][SQL][FOLLOW-UP] Change doc and error message for SHOW CREATE TABLE ### What changes were proposed in this pull request? This is a follow-up for #24938 to tweak error message and migration doc. ### Why are the changes needed? Making user know workaround if SHOW CREATE TABLE doesn't work for some Hive tables. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing unit tests. Closes #27505 from viirya/SPARK-27946-followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- docs/sql-migration-guide.md | 2 +- .../org/apache/spark/sql/execution/command/tables.scala | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index be0fe32ded99b..26eb5838892b4 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -326,7 +326,7 @@ license: | - Since Spark 3.0, `SHOW TBLPROPERTIES` will cause `AnalysisException` if the table does not exist. In Spark version 2.4 and earlier, this scenario caused `NoSuchTableException`. Also, `SHOW TBLPROPERTIES` on a temporary view will cause `AnalysisException`. In Spark version 2.4 and earlier, it returned an empty result. - - Since Spark 3.0, `SHOW CREATE TABLE` will always return Spark DDL, even when the given table is a Hive serde table. For Hive DDL, please use `SHOW CREATE TABLE AS SERDE` command instead. + - Since Spark 3.0, `SHOW CREATE TABLE` will always return Spark DDL, even when the given table is a Hive serde table. For generating Hive DDL, please use `SHOW CREATE TABLE AS SERDE` command instead. - Since Spark 3.0, we upgraded the built-in Hive from 1.2 to 2.3. This may need to set `spark.sql.hive.metastore.version` and `spark.sql.hive.metastore.jars` according to the version of the Hive metastore. For example: set `spark.sql.hive.metastore.version` to `1.2.1` and `spark.sql.hive.metastore.jars` to `maven` if your Hive metastore version is 1.2.1. 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 468ca505cce1f..90dbdf5515d4d 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 @@ -1076,7 +1076,9 @@ case class ShowCreateTableCommand(table: TableIdentifier) "Failed to execute SHOW CREATE TABLE against table " + s"${tableMetadata.identifier}, which is created by Hive and uses the " + "following unsupported feature(s)\n" + - tableMetadata.unsupportedFeatures.map(" - " + _).mkString("\n") + tableMetadata.unsupportedFeatures.map(" - " + _).mkString("\n") + ". " + + s"Please use `SHOW CREATE TABLE ${tableMetadata.identifier} AS SERDE` " + + "to show Hive DDL instead." ) } @@ -1086,7 +1088,9 @@ case class ShowCreateTableCommand(table: TableIdentifier) if ("true".equalsIgnoreCase(tableMetadata.properties.getOrElse("transactional", "false"))) { throw new AnalysisException( - "SHOW CREATE TABLE doesn't support transactional Hive table") + "SHOW CREATE TABLE doesn't support transactional Hive table. " + + s"Please use `SHOW CREATE TABLE ${tableMetadata.identifier} AS SERDE` " + + "to show Hive DDL instead.") } convertTableMetadata(tableMetadata) From 4439b29bd2ac0c3cc4c6ceea825fc797ff0029a3 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Mon, 10 Feb 2020 10:56:43 -0800 Subject: [PATCH 1337/1387] Revert "[SPARK-30245][SQL] Add cache for Like and RLike when pattern is not static" ### What changes were proposed in this pull request? This reverts commit 8ce7962931680c204e84dd75783b1c943ea9c525. There's variable name conflicts with https://github.com/apache/spark/commit/8aebc80e0e67bcb1aa300b8c8b1a209159237632#diff-39298b470865a4cbc67398a4ea11e767. This can be cleanly ported back to branch-3.0. ### Why are the changes needed? Performance investigation were not made enough and it's not clear if it really beneficial or now. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Jenkins tests. Closes #27514 from HyukjinKwon/revert-cache-PR. Authored-by: HyukjinKwon Signed-off-by: Xiao Li --- .../expressions/regexpExpressions.scala | 21 ++++++------------- 1 file changed, 6 insertions(+), 15 deletions(-) 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 c9ddc70bf5bc6..f84c476ea5807 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 @@ -177,6 +177,8 @@ case class Like(str: Expression, pattern: Expression, escape: Expression) """) } } else { + val patternStr = ctx.freshName("patternStr") + val compiledPattern = ctx.freshName("compiledPattern") // We need double escape to avoid org.codehaus.commons.compiler.CompileException. // '\\' will cause exception 'Single quote must be backslash-escaped in character literal'. // '\"' will cause exception 'Line break in literal not allowed'. @@ -185,17 +187,11 @@ case class Like(str: Expression, pattern: Expression, escape: Expression) } else { escapeChar } - val patternStr = ctx.freshName("patternStr") - val compiledPattern = ctx.addMutableState(patternClass, "compiledPattern") - val lastPatternStr = ctx.addMutableState(classOf[String].getName, "lastPatternStr") - nullSafeCodeGen(ctx, ev, (eval1, eval2, _) => { s""" String $patternStr = $eval2.toString(); - if (!$patternStr.equals($lastPatternStr)) { - $compiledPattern = $patternClass.compile($escapeFunc($patternStr, '$newEscapeChar')); - $lastPatternStr = $patternStr; - } + $patternClass $compiledPattern = $patternClass.compile( + $escapeFunc($patternStr, '$newEscapeChar')); ${ev.value} = $compiledPattern.matcher($eval1.toString()).matches(); """ }) @@ -278,16 +274,11 @@ case class RLike(left: Expression, right: Expression) } } else { val rightStr = ctx.freshName("rightStr") - val pattern = ctx.addMutableState(patternClass, "pattern") - val lastRightStr = ctx.addMutableState(classOf[String].getName, "lastRightStr") - + val pattern = ctx.freshName("pattern") nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" String $rightStr = $eval2.toString(); - if (!$rightStr.equals($lastRightStr)) { - $pattern = $patternClass.compile($rightStr); - $lastRightStr = $rightStr; - } + $patternClass $pattern = $patternClass.compile($rightStr); ${ev.value} = $pattern.matcher($eval1.toString()).find(0); """ }) From 3c1c9b48fcca1a714e6c2a3045b512598438d672 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 10 Feb 2020 12:51:37 -0800 Subject: [PATCH 1338/1387] [SPARK-30759][SQL] Initialize cache for foldable patterns in StringRegexExpression ### What changes were proposed in this pull request? In the PR, I propose to fix `cache` initialization in `StringRegexExpression` by changing `case Literal(value: String, StringType)` to `case p: Expression if p.foldable` ### Why are the changes needed? Actually, the case doesn't work at all because of: 1. Literals value has type `UTF8String` 2. It doesn't work for foldable expressions like in the example: ```sql SELECT '%SystemDrive%\Users\John' _FUNC_ '%SystemDrive%\\Users.*'; ``` Screen Shot 2020-02-08 at 22 45 50 ### Does this PR introduce any user-facing change? No ### How was this patch tested? By the `check outputs of expression examples` test from `SQLQuerySuite`. Closes #27502 from MaxGekk/str-regexp-foldable-pattern. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/expressions/regexpExpressions.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 f84c476ea5807..f8d328bf601e4 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 @@ -41,9 +41,10 @@ trait StringRegexExpression extends Expression override def dataType: DataType = BooleanType - // try cache the pattern for Literal + // try cache foldable pattern private lazy val cache: Pattern = pattern match { - case Literal(value: String, StringType) => compile(value) + case p: Expression if p.foldable => + compile(p.eval().asInstanceOf[UTF8String].toString) case _ => null } From a6b91d2bf727e175d0e175295001db85647539b1 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 10 Feb 2020 22:16:25 +0100 Subject: [PATCH 1339/1387] [SPARK-30556][SQL][FOLLOWUP] Reset the status changed in SQLExecution.withThreadLocalCaptured ### What changes were proposed in this pull request? Follow up for #27267, reset the status changed in SQLExecution.withThreadLocalCaptured. ### Why are the changes needed? For code safety. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing UT. Closes #27516 from xuanyuanking/SPARK-30556-follow. Authored-by: Yuanjian Li Signed-off-by: herman --- .../apache/spark/sql/execution/SQLExecution.scala | 12 +++++++++++- .../sql/internal/ExecutorSideSQLConfSuite.scala | 10 ++++++---- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 995d94ef5eac7..9f177819f6ea7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -177,9 +177,19 @@ object SQLExecution { val sc = sparkSession.sparkContext val localProps = Utils.cloneProperties(sc.getLocalProperties) Future { + val originalSession = SparkSession.getActiveSession + val originalLocalProps = sc.getLocalProperties SparkSession.setActiveSession(activeSession) sc.setLocalProperties(localProps) - body + val res = body + // reset active session and local props. + sc.setLocalProperties(originalLocalProps) + if (originalSession.nonEmpty) { + SparkSession.setActiveSession(originalSession.get) + } else { + SparkSession.clearActiveSession() + } + res }(exec) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index 0cc658c499615..46d0c64592a00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.internal +import java.util.UUID + import org.scalatest.Assertions._ import org.apache.spark.{SparkException, SparkFunSuite, TaskContext} @@ -144,16 +146,16 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { } // set local configuration and assert - val confValue1 = "e" + val confValue1 = UUID.randomUUID().toString() createDataframe(confKey, confValue1).createOrReplaceTempView("m") spark.sparkContext.setLocalProperty(confKey, confValue1) - assert(sql("SELECT * FROM l WHERE EXISTS (SELECT * FROM m)").collect.size == 1) + assert(sql("SELECT * FROM l WHERE EXISTS (SELECT * FROM m)").collect().length == 1) // change the conf value and assert again - val confValue2 = "f" + val confValue2 = UUID.randomUUID().toString() createDataframe(confKey, confValue2).createOrReplaceTempView("n") spark.sparkContext.setLocalProperty(confKey, confValue2) - assert(sql("SELECT * FROM l WHERE EXISTS (SELECT * FROM n)").collect().size == 1) + assert(sql("SELECT * FROM l WHERE EXISTS (SELECT * FROM n)").collect().length == 1) } } } From e2ebca733ce4366349a5a25fe94a8e31b67d410e Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 10 Feb 2020 14:26:14 -0800 Subject: [PATCH 1340/1387] [SPARK-30779][SS] Fix some API issues found when reviewing Structured Streaming API docs ### What changes were proposed in this pull request? - Fix the scope of `Logging.initializeForcefully` so that it doesn't appear in subclasses' public methods. Right now, `sc.initializeForcefully(false, false)` is allowed to called. - Don't show classes under `org.apache.spark.internal` package in API docs. - Add missing `since` annotation. - Fix the scope of `ArrowUtils` to remove it from the API docs. ### Why are the changes needed? Avoid leaking APIs unintentionally in Spark 3.0.0. ### Does this PR introduce any user-facing change? No. All these changes are to avoid leaking APIs unintentionally in Spark 3.0.0. ### How was this patch tested? Manually generated the API docs and verified the above issues have been fixed. Closes #27528 from zsxwing/audit-ss-apis. Authored-by: Shixiong Zhu Signed-off-by: Xiao Li --- core/src/main/scala/org/apache/spark/internal/Logging.scala | 2 +- project/SparkBuild.scala | 1 + .../sql/connector/read/streaming/ContinuousPartitionReader.java | 2 ++ .../read/streaming/ContinuousPartitionReaderFactory.java | 2 ++ .../spark/sql/connector/read/streaming/ContinuousStream.java | 2 ++ .../spark/sql/connector/read/streaming/MicroBatchStream.java | 2 ++ .../org/apache/spark/sql/connector/read/streaming/Offset.java | 2 ++ .../spark/sql/connector/read/streaming/PartitionOffset.java | 2 ++ .../apache/spark/sql/connector/read/streaming/ReadLimit.java | 1 + .../spark/sql/connector/read/streaming/SparkDataStream.java | 2 ++ .../connector/write/streaming/StreamingDataWriterFactory.java | 2 ++ .../spark/sql/connector/write/streaming/StreamingWrite.java | 2 ++ .../src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala | 2 +- 13 files changed, 22 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala index 2e4846bec2db4..0c1d9635b6535 100644 --- a/core/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -117,7 +117,7 @@ trait Logging { } // For testing - def initializeForcefully(isInterpreter: Boolean, silent: Boolean): Unit = { + private[spark] def initializeForcefully(isInterpreter: Boolean, silent: Boolean): Unit = { initializeLogging(isInterpreter, silent) } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 707c31d2248eb..9d0af3aa8c1b6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -819,6 +819,7 @@ object Unidoc { .map(_.filterNot(_.getName.contains("$"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/deploy"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/examples"))) + .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/internal"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/memory"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/network"))) .map(_.filterNot(f => diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java index 8bd5273bb7d8e..c2ad9ec244a0d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReader.java @@ -22,6 +22,8 @@ /** * A variation on {@link PartitionReader} for use with continuous streaming processing. + * + * @since 3.0.0 */ @Evolving public interface ContinuousPartitionReader extends PartitionReader { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java index 962864da4aad8..385c6f655440f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousPartitionReaderFactory.java @@ -27,6 +27,8 @@ /** * A variation on {@link PartitionReaderFactory} that returns {@link ContinuousPartitionReader} * instead of {@link PartitionReader}. It's used for continuous streaming processing. + * + * @since 3.0.0 */ @Evolving public interface ContinuousPartitionReaderFactory extends PartitionReaderFactory { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java index ee01a2553ae7a..a84578fe461a3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ContinuousStream.java @@ -23,6 +23,8 @@ /** * A {@link SparkDataStream} for streaming queries with continuous mode. + * + * @since 3.0.0 */ @Evolving public interface ContinuousStream extends SparkDataStream { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java index ceab0f75734d3..40ecbf0578ee5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/MicroBatchStream.java @@ -25,6 +25,8 @@ /** * A {@link SparkDataStream} for streaming queries with micro-batch mode. + * + * @since 3.0.0 */ @Evolving public interface MicroBatchStream extends SparkDataStream { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java index 400de2a659746..efb8ebb684f06 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/Offset.java @@ -25,6 +25,8 @@ * During execution, offsets provided by the data source implementation will be logged and used as * restart checkpoints. Each source should provide an offset implementation which the source can use * to reconstruct a position in the stream up to which data has been seen/processed. + * + * @since 3.0.0 */ @Evolving public abstract class Offset { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java index 35ad3bbde5cbf..faee230467bea 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/PartitionOffset.java @@ -26,6 +26,8 @@ * provide a method to merge these into a global Offset. * * These offsets must be serializable. + * + * @since 3.0.0 */ @Evolving public interface PartitionOffset extends Serializable { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadLimit.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadLimit.java index 121ed1ad116f9..36f6e05e365d9 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadLimit.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/ReadLimit.java @@ -27,6 +27,7 @@ * @see SupportsAdmissionControl#latestOffset(Offset, ReadLimit) * @see ReadAllAvailable * @see ReadMaxRows + * @since 3.0.0 */ @Evolving public interface ReadLimit { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java index 1ba0c25ef4466..95703e255ea4e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SparkDataStream.java @@ -25,6 +25,8 @@ * * Data sources should implement concrete data stream interfaces: * {@link MicroBatchStream} and {@link ContinuousStream}. + * + * @since 3.0.0 */ @Evolving public interface SparkDataStream { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java index 9946867e8ea65..0923d07e7e5a3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingDataWriterFactory.java @@ -33,6 +33,8 @@ * Note that, the writer factory will be serialized and sent to executors, then the data writer * will be created on executors and do the actual writing. So this interface must be * serializable and {@link DataWriter} doesn't need to be. + * + * @since 3.0.0 */ @Evolving public interface StreamingDataWriterFactory extends Serializable { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java index 4f930e1c158e5..e3dec3b2ff55e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/streaming/StreamingWrite.java @@ -40,6 +40,8 @@ * do it manually in their Spark applications if they want to retry. * * Please refer to the documentation of commit/abort methods for detailed specifications. + * + * @since 3.0.0 */ @Evolving public interface StreamingWrite { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index 2da0d1a51cb29..003ce850c926e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -27,7 +27,7 @@ import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -object ArrowUtils { +private[sql] object ArrowUtils { val rootAllocator = new RootAllocator(Long.MaxValue) From 07a9885f2792be1353f4a923d649e90bc431cb38 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Tue, 11 Feb 2020 10:03:01 +0900 Subject: [PATCH 1341/1387] [SPARK-30777][PYTHON][TESTS] Fix test failures for Pandas >= 1.0.0 ### What changes were proposed in this pull request? Fix PySpark test failures for using Pandas >= 1.0.0. ### Why are the changes needed? Pandas 1.0.0 has recently been released and has API changes that result in PySpark test failures, this PR fixes the broken tests. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Manually tested with Pandas 1.0.1 and PyArrow 0.16.0 Closes #27529 from BryanCutler/pandas-fix-tests-1.0-SPARK-30777. Authored-by: Bryan Cutler Signed-off-by: HyukjinKwon --- python/pyspark/sql/tests/test_arrow.py | 4 ++-- python/pyspark/sql/tests/test_pandas_grouped_map.py | 6 +++--- python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index 98f44dfd29da5..004c79f290213 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -297,9 +297,9 @@ def test_createDataFrame_does_not_modify_input(self): # Some series get converted for Spark to consume, this makes sure input is unchanged pdf = self.create_pandas_data_frame() # Use a nanosecond value to make sure it is not truncated - pdf.ix[0, '8_timestamp_t'] = pd.Timestamp(1) + pdf.iloc[0, 7] = pd.Timestamp(1) # Integers with nulls will get NaNs filled with 0 and will be casted - pdf.ix[1, '2_int_t'] = None + pdf.iloc[1, 1] = None pdf_copy = pdf.copy(deep=True) self.spark.createDataFrame(pdf, schema=self.schema) self.assertTrue(pdf.equals(pdf_copy)) diff --git a/python/pyspark/sql/tests/test_pandas_grouped_map.py b/python/pyspark/sql/tests/test_pandas_grouped_map.py index 51dd07fd7d70c..ff53a0c6f2cf2 100644 --- a/python/pyspark/sql/tests/test_pandas_grouped_map.py +++ b/python/pyspark/sql/tests/test_pandas_grouped_map.py @@ -390,11 +390,11 @@ def rename_pdf(pdf, names): # Function returns a pdf with required column names, but order could be arbitrary using dict def change_col_order(pdf): # Constructing a DataFrame from a dict should result in the same order, - # but use from_items to ensure the pdf column order is different than schema - return pd.DataFrame.from_items([ + # but use OrderedDict to ensure the pdf column order is different than schema + return pd.DataFrame.from_dict(OrderedDict([ ('id', pdf.id), ('u', pdf.v * 2), - ('v', pdf.v)]) + ('v', pdf.v)])) ordered_udf = pandas_udf( change_col_order, diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py index 974ad560daebf..21679785a769e 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py @@ -357,7 +357,7 @@ def test_complex_expressions(self): plus_one(sum_udf(col('v1'))), sum_udf(plus_one(col('v2')))) .sort(['id', '(v % 2)']) - .toPandas().sort_index(by=['id', '(v % 2)'])) + .toPandas().sort_values(by=['id', '(v % 2)'])) expected1 = (df.withColumn('v1', df.v + 1) .withColumn('v2', df.v + 2) @@ -368,7 +368,7 @@ def test_complex_expressions(self): plus_one(sum(col('v1'))), sum(plus_one(col('v2')))) .sort(['id', '(v % 2)']) - .toPandas().sort_index(by=['id', '(v % 2)'])) + .toPandas().sort_values(by=['id', '(v % 2)'])) # Test complex expressions with sql expression, scala pandas UDF and # group aggregate pandas UDF @@ -381,7 +381,7 @@ def test_complex_expressions(self): plus_two(sum_udf(col('v1'))), sum_udf(plus_two(col('v2')))) .sort(['id', '(v % 2)']) - .toPandas().sort_index(by=['id', '(v % 2)'])) + .toPandas().sort_values(by=['id', '(v % 2)'])) expected2 = (df.withColumn('v1', df.v + 1) .withColumn('v2', df.v + 2) @@ -392,7 +392,7 @@ def test_complex_expressions(self): plus_two(sum(col('v1'))), sum(plus_two(col('v2')))) .sort(['id', '(v % 2)']) - .toPandas().sort_index(by=['id', '(v % 2)'])) + .toPandas().sort_values(by=['id', '(v % 2)'])) # Test sequential groupby aggregate result3 = (df.groupby('id') From 2bc765a831d7f15c7971d41c36cfbec1fd898dfd Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 11 Feb 2020 15:50:03 +0900 Subject: [PATCH 1342/1387] [SPARK-30756][SQL] Fix `ThriftServerWithSparkContextSuite` on spark-branch-3.0-test-sbt-hadoop-2.7-hive-2.3 ### What changes were proposed in this pull request? This PR tries #26710 (comment) way to fix the test. ### Why are the changes needed? To make the tests pass. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Jenkins will test first, and then `on spark-branch-3.0-test-sbt-hadoop-2.7-hive-2.3` will test it out. Closes #27513 from HyukjinKwon/test-SPARK-30756. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon (cherry picked from commit 8efe367a4ee862b8a85aee8881b0335b34cbba70) Signed-off-by: HyukjinKwon --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9d0af3aa8c1b6..1c5c36ea8eae2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -478,7 +478,8 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite", "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", - "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite" + "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", + "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite" ) private val DEFAULT_TEST_GROUP = "default_test_group" From 0045be766b949dff23ed72bd559568f17f645ffe Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 11 Feb 2020 17:22:08 +0900 Subject: [PATCH 1343/1387] [SPARK-29462][SQL] The data type of "array()" should be array ### What changes were proposed in this pull request? This brings https://github.com/apache/spark/pull/26324 back. It was reverted basically because, firstly Hive compatibility, and the lack of investigations in other DBMSes and ANSI. - In case of PostgreSQL seems coercing NULL literal to TEXT type. - Presto seems coercing `array() + array(1)` -> array of int. - Hive seems `array() + array(1)` -> array of strings Given that, the design choices have been differently made for some reasons. If we pick one of both, seems coercing to array of int makes much more sense. Another investigation was made offline internally. Seems ANSI SQL 2011, section 6.5 "" states: > If ES is specified, then let ET be the element type determined by the context in which ES appears. The declared type DT of ES is Case: > > a) If ES simply contains ARRAY, then ET ARRAY[0]. > > b) If ES simply contains MULTISET, then ET MULTISET. > > ES is effectively replaced by CAST ( ES AS DT ) From reading other related context, doing it to `NullType`. Given the investigation made, choosing to `null` seems correct, and we have a reference Presto now. Therefore, this PR proposes to bring it back. ### Why are the changes needed? When empty array is created, it should be declared as array. ### Does this PR introduce any user-facing change? Yes, `array()` creates `array`. Now `array(1) + array()` can correctly create `array(1)` instead of `array("1")`. ### How was this patch tested? Tested manually Closes #27521 from HyukjinKwon/SPARK-29462. Lead-authored-by: HyukjinKwon Co-authored-by: Aman Omer Signed-off-by: HyukjinKwon --- docs/sql-migration-guide.md | 2 ++ .../expressions/complexTypeCreator.scala | 11 ++++++++++- .../org/apache/spark/sql/internal/SQLConf.scala | 9 +++++++++ .../spark/sql/DataFrameFunctionsSuite.scala | 17 +++++++++++++---- 4 files changed, 34 insertions(+), 5 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 26eb5838892b4..f98fab5b4c56b 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -215,6 +215,8 @@ license: | For example `SELECT timestamp 'tomorrow';`. - Since Spark 3.0, the `size` function returns `NULL` for the `NULL` input. In Spark version 2.4 and earlier, this function gives `-1` for the same input. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.sizeOfNull` to `true`. + + - Since Spark 3.0, when the `array` function is called without any parameters, it returns an empty array of `NullType`. In Spark version 2.4 and earlier, it returns an empty array of string type. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.arrayDefaultToStringType.enabled` to `true`. - Since Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception. 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 9ce87a4922c01..7335e305bfe55 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -44,10 +45,18 @@ case class CreateArray(children: Seq[Expression]) extends Expression { TypeUtils.checkForSameTypeInputExpr(children.map(_.dataType), s"function $prettyName") } + private val defaultElementType: DataType = { + if (SQLConf.get.getConf(SQLConf.LEGACY_ARRAY_DEFAULT_TO_STRING)) { + StringType + } else { + NullType + } + } + override def dataType: ArrayType = { ArrayType( TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(children.map(_.dataType)) - .getOrElse(StringType), + .getOrElse(defaultElementType), containsNull = children.exists(_.nullable)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 64c613611c861..d86f8693e0655 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2007,6 +2007,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val LEGACY_ARRAY_DEFAULT_TO_STRING = + buildConf("spark.sql.legacy.arrayDefaultToStringType.enabled") + .internal() + .doc("When set to true, it returns an empty array of string type when the `array` " + + "function is called without any parameters. Otherwise, it returns an empty " + + "array of `NullType`") + .booleanConf + .createWithDefault(false) + val TRUNCATE_TABLE_IGNORE_PERMISSION_ACL = buildConf("spark.sql.truncateTable.ignorePermissionAcl.enabled") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 7fce03658fc16..9e9d8c3e9a7c5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -3499,12 +3499,9 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ).foreach(assertValuesDoNotChangeAfterCoalesceOrUnion(_)) } - test("SPARK-21281 use string types by default if array and map have no argument") { + test("SPARK-21281 use string types by default if map have no argument") { val ds = spark.range(1) var expectedSchema = new StructType() - .add("x", ArrayType(StringType, containsNull = false), nullable = false) - assert(ds.select(array().as("x")).schema == expectedSchema) - expectedSchema = new StructType() .add("x", MapType(StringType, StringType, valueContainsNull = false), nullable = false) assert(ds.select(map().as("x")).schema == expectedSchema) } @@ -3577,6 +3574,18 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { }.getMessage assert(nonFoldableError.contains("The 'escape' parameter must be a string literal")) } + + test("SPARK-29462: Empty array of NullType for array function with no arguments") { + Seq((true, StringType), (false, NullType)).foreach { + case (arrayDefaultToString, expectedType) => + withSQLConf(SQLConf.LEGACY_ARRAY_DEFAULT_TO_STRING.key -> arrayDefaultToString.toString) { + val schema = spark.range(1).select(array()).schema + assert(schema.nonEmpty && schema.head.dataType.isInstanceOf[ArrayType]) + val actualType = schema.head.dataType.asInstanceOf[ArrayType].elementType + assert(actualType === expectedType) + } + } + } } object DataFrameFunctionsSuite { From b20754d9ee033091e2ef4d5bfa2576f946c9df50 Mon Sep 17 00:00:00 2001 From: root1 Date: Tue, 11 Feb 2020 20:42:02 +0800 Subject: [PATCH 1344/1387] [SPARK-27545][SQL][DOC] Update the Documentation for CACHE TABLE and UNCACHE TABLE ### What changes were proposed in this pull request? Document updated for `CACHE TABLE` & `UNCACHE TABLE` ### Why are the changes needed? Cache table creates a temp view while caching data using `CACHE TABLE name AS query`. `UNCACHE TABLE` does not remove this temp view. These things were not mentioned in the existing doc for `CACHE TABLE` & `UNCACHE TABLE`. ### Does this PR introduce any user-facing change? Document updated for `CACHE TABLE` & `UNCACHE TABLE` command. ### How was this patch tested? Manually Closes #27090 from iRakson/SPARK-27545. Lead-authored-by: root1 Co-authored-by: iRakson Signed-off-by: Wenchen Fan --- docs/sql-ref-syntax-aux-cache-cache-table.md | 3 ++- docs/sql-ref-syntax-aux-cache-uncache-table.md | 6 +++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-aux-cache-cache-table.md b/docs/sql-ref-syntax-aux-cache-cache-table.md index ed6ef973466dd..20ade1961ab0b 100644 --- a/docs/sql-ref-syntax-aux-cache-cache-table.md +++ b/docs/sql-ref-syntax-aux-cache-cache-table.md @@ -20,7 +20,8 @@ license: | --- ### Description -`CACHE TABLE` statement caches contents of a table or output of a query with the given storage level. This reduces scanning of the original files in future queries. +`CACHE TABLE` statement caches contents of a table or output of a query with the given storage level. If a query is cached, then a temp view will be created for this query. +This reduces scanning of the original files in future queries. ### Syntax {% highlight sql %} diff --git a/docs/sql-ref-syntax-aux-cache-uncache-table.md b/docs/sql-ref-syntax-aux-cache-uncache-table.md index e0581d0d213df..69e21c258a333 100644 --- a/docs/sql-ref-syntax-aux-cache-uncache-table.md +++ b/docs/sql-ref-syntax-aux-cache-uncache-table.md @@ -21,11 +21,13 @@ license: | ### Description `UNCACHE TABLE` removes the entries and associated data from the in-memory and/or on-disk cache for a given table or view. The -underlying entries should already have been brought to cache by previous `CACHE TABLE` operation. `UNCACHE TABLE` on a non-existent table throws Exception if `IF EXISTS` is not specified. +underlying entries should already have been brought to cache by previous `CACHE TABLE` operation. `UNCACHE TABLE` on a non-existent table throws an exception if `IF EXISTS` is not specified. + ### Syntax {% highlight sql %} UNCACHE TABLE [ IF EXISTS ] table_identifier {% endhighlight %} + ### Parameters

    + ### Examples {% highlight sql %} UNCACHE TABLE t1; {% endhighlight %} + ### Related Statements * [CACHE TABLE](sql-ref-syntax-aux-cache-cache-table.html) * [CLEAR CACHE](sql-ref-syntax-aux-cache-clear-cache.html) From f1d0dce4848a53831268c80bf7e1e0f47a1f7612 Mon Sep 17 00:00:00 2001 From: fuwhu Date: Tue, 11 Feb 2020 22:16:44 +0800 Subject: [PATCH 1345/1387] [MINOR][DOC] Add class document for PruneFileSourcePartitions and PruneHiveTablePartitions ### What changes were proposed in this pull request? Add class document for PruneFileSourcePartitions and PruneHiveTablePartitions. ### Why are the changes needed? To describe these two classes. ### Does this PR introduce any user-facing change? no ### How was this patch tested? no Closes #27535 from fuwhu/SPARK-15616-FOLLOW-UP. Authored-by: fuwhu Signed-off-by: Wenchen Fan --- .../datasources/PruneFileSourcePartitions.scala | 13 +++++++++++++ .../hive/execution/PruneHiveTablePartitions.scala | 8 ++++++++ 2 files changed, 21 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 1ea19c187e51a..a7129fb14d1a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -26,6 +26,19 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, FileScan} import org.apache.spark.sql.types.StructType +/** + * Prune the partitions of file source based table using partition filters. Currently, this rule + * is applied to [[HadoopFsRelation]] with [[CatalogFileIndex]] and [[DataSourceV2ScanRelation]] + * with [[FileScan]]. + * + * For [[HadoopFsRelation]], the location will be replaced by pruned file index, and corresponding + * statistics will be updated. And the partition filters will be kept in the filters of returned + * logical plan. + * + * For [[DataSourceV2ScanRelation]], both partition filters and data filters will be added to + * its underlying [[FileScan]]. And the partition filters will be removed in the filters of + * returned logical plan. + */ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { private def getPartitionKeyFiltersAndDataFilters( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala index a0349f627d107..da6e4c52cf3a7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala @@ -30,6 +30,14 @@ import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.internal.SQLConf /** + * Prune hive table partitions using partition filters on [[HiveTableRelation]]. The pruned + * partitions will be kept in [[HiveTableRelation.prunedPartitions]], and the statistics of + * the hive table relation will be updated based on pruned partitions. + * + * This rule is executed in optimization phase, so the statistics can be updated before physical + * planning, which is useful for some spark strategy, eg. + * [[org.apache.spark.sql.execution.SparkStrategies.JoinSelection]]. + * * TODO: merge this with PruneFileSourcePartitions after we completely make hive as a data source. */ private[sql] class PruneHiveTablePartitions(session: SparkSession) From dc66d57e981ac5108e097d4298fa467f0843ffcf Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 11 Feb 2020 09:07:40 -0600 Subject: [PATCH 1346/1387] [SPARK-30754][SQL] Reuse results of floorDiv in calculations of floorMod in DateTimeUtils ### What changes were proposed in this pull request? In the case of back-to-back calculation of `floorDiv` and `floorMod` with the same arguments, the result of `foorDiv` can be reused in calculation of `floorMod`. The `floorMod` method is defined as the following in Java standard library: ```java public static int floorMod(int x, int y) { int r = x - floorDiv(x, y) * y; return r; } ``` If `floorDiv(x, y)` has been already calculated, it can be reused in `x - floorDiv(x, y) * y`. I propose to modify 2 places in `DateTimeUtils`: 1. `microsToInstant` which is widely used in many date-time functions. `Math.floorMod(us, MICROS_PER_SECOND)` is just replaced by its definition from Java Math library. 2. `truncDate`: `Math.floorMod(oldYear, divider) == 0` is replaced by `Math.floorDiv(oldYear, divider) * divider == oldYear` where `floorDiv(...) * divider` is pre-calculated. ### Why are the changes needed? This reduces the number of arithmetic operations, and can slightly improve performance of date-time functions. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite` and `DateExpressionsSuite`. Closes #27491 from MaxGekk/opt-microsToInstant. Authored-by: Maxim Gekk Signed-off-by: Sean Owen --- .../spark/sql/catalyst/util/DateTimeUtils.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 8eb560944d4cb..eeae0674166bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -337,7 +337,9 @@ object DateTimeUtils { def microsToInstant(us: Long): Instant = { val secs = Math.floorDiv(us, MICROS_PER_SECOND) - val mos = Math.floorMod(us, MICROS_PER_SECOND) + // Unfolded Math.floorMod(us, MICROS_PER_SECOND) to reuse the result of + // the above calculation of `secs` via `floorDiv`. + val mos = us - secs * MICROS_PER_SECOND Instant.ofEpochSecond(secs, mos * NANOS_PER_MICROS) } @@ -691,11 +693,11 @@ object DateTimeUtils { def truncDate(d: SQLDate, level: Int): SQLDate = { def truncToYearLevel(divider: Int, adjust: Int): SQLDate = { val oldYear = getYear(d) - var newYear = Math.floorDiv(oldYear, divider) - if (adjust > 0 && Math.floorMod(oldYear, divider) == 0) { - newYear -= 1 + var newYear = Math.floorDiv(oldYear, divider) * divider + if (adjust > 0 && newYear == oldYear) { + newYear -= divider } - newYear = newYear * divider + adjust + newYear += adjust localDateToDays(LocalDate.of(newYear, 1, 1)) } level match { From ea626b6acf0de0ff3b0678372f30ba6f84ae2b09 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 12 Feb 2020 00:12:45 +0800 Subject: [PATCH 1347/1387] [SPARK-30783] Exclude hive-service-rpc ### What changes were proposed in this pull request? Exclude hive-service-rpc from build. ### Why are the changes needed? hive-service-rpc 2.3.6 and spark sql's thrift server module have duplicate classes. Leaving hive-service-rpc 2.3.6 in the class path means that spark can pick up classes defined in hive instead of its thrift server module, which can cause hard to debug runtime errors due to class loading order and compilation errors for applications depend on spark. If you compare hive-service-rpc 2.3.6's jar (https://search.maven.org/remotecontent?filepath=org/apache/hive/hive-service-rpc/2.3.6/hive-service-rpc-2.3.6.jar) and spark thrift server's jar (e.g. https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-hive-thriftserver_2.12/3.0.0-SNAPSHOT/spark-hive-thriftserver_2.12-3.0.0-20200207.021914-364.jar), you will see that all of classes provided by hive-service-rpc-2.3.6.jar are covered by spark thrift server's jar. https://issues.apache.org/jira/browse/SPARK-30783 has output of jar tf for both jars. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #27533 from yhuai/SPARK-30783. Authored-by: Yin Huai Signed-off-by: Wenchen Fan --- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 1 - dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 1 - pom.xml | 20 ++++++++++++++++++++ 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 42bdf112efccb..c50cf96dc9065 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -87,7 +87,6 @@ hive-jdbc/2.3.6//hive-jdbc-2.3.6.jar hive-llap-common/2.3.6//hive-llap-common-2.3.6.jar hive-metastore/2.3.6//hive-metastore-2.3.6.jar hive-serde/2.3.6//hive-serde-2.3.6.jar -hive-service-rpc/2.3.6//hive-service-rpc-2.3.6.jar hive-shims-0.23/2.3.6//hive-shims-0.23-2.3.6.jar hive-shims-common/2.3.6//hive-shims-common-2.3.6.jar hive-shims-scheduler/2.3.6//hive-shims-scheduler-2.3.6.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 6006fa4b43f42..c37ce7fab36f6 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -86,7 +86,6 @@ hive-jdbc/2.3.6//hive-jdbc-2.3.6.jar hive-llap-common/2.3.6//hive-llap-common-2.3.6.jar hive-metastore/2.3.6//hive-metastore-2.3.6.jar hive-serde/2.3.6//hive-serde-2.3.6.jar -hive-service-rpc/2.3.6//hive-service-rpc-2.3.6.jar hive-shims-0.23/2.3.6//hive-shims-0.23-2.3.6.jar hive-shims-common/2.3.6//hive-shims-common-2.3.6.jar hive-shims-scheduler/2.3.6//hive-shims-scheduler-2.3.6.jar diff --git a/pom.xml b/pom.xml index a8d6ac932bac2..925fa28a291a4 100644 --- a/pom.xml +++ b/pom.xml @@ -1452,6 +1452,11 @@ ${hive.group} hive-service + + + ${hive.group} + hive-service-rpc + ${hive.group} hive-shims @@ -1508,6 +1513,11 @@ ${hive.group} hive-service + + + ${hive.group} + hive-service-rpc + ${hive.group} hive-shims @@ -1761,6 +1771,11 @@ ${hive.group} hive-service + + + ${hive.group} + hive-service-rpc + ${hive.group} hive-shims @@ -1911,6 +1926,11 @@ groovy-all + + + ${hive.group} + hive-service-rpc + org.apache.parquet From 99bd59fe29a87bb70485db536b0ae676e7a9d42e Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 11 Feb 2020 09:55:02 -0800 Subject: [PATCH 1348/1387] [SPARK-29462][SQL][DOCS] Add some more context and details in 'spark.sql.defaultUrlStreamHandlerFactory.enabled' documentation ### What changes were proposed in this pull request? This PR adds some more information and context to `spark.sql.defaultUrlStreamHandlerFactory.enabled`. ### Why are the changes needed? It is a bit difficult to understand the documentation of `spark.sql.defaultUrlStreamHandlerFactory.enabled`. ### Does this PR introduce any user-facing change? Nope, internal doc only fix. ### How was this patch tested? Nope. I only tested linter. Closes #27541 from HyukjinKwon/SPARK-29462-followup. Authored-by: HyukjinKwon Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/internal/StaticSQLConf.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala index 6bc752260a893..563e51ed597b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala @@ -172,7 +172,13 @@ object StaticSQLConf { val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED = buildStaticConf("spark.sql.defaultUrlStreamHandlerFactory.enabled") - .doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations") + .doc( + "When true, register Hadoop's FsUrlStreamHandlerFactory to support " + + "ADD JAR against HDFS locations. " + + "It should be disabled when a different stream protocol handler should be registered " + + "to support a particular protocol type, or if Hadoop's FsUrlStreamHandlerFactory " + + "conflicts with other protocol types such as `http` or `https`. See also SPARK-25694 " + + "and HADOOP-14598.") .internal() .booleanConf .createWithDefault(true) From 45db48e2d29359591a4ebc3db4625dd2158e446e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 11 Feb 2020 10:15:34 -0800 Subject: [PATCH 1349/1387] Revert "[SPARK-30625][SQL] Support `escape` as third parameter of the `like` function ### What changes were proposed in this pull request? In the PR, I propose to revert the commit 8aebc80e0e67bcb1aa300b8c8b1a209159237632. ### Why are the changes needed? See the concerns https://github.com/apache/spark/pull/27355#issuecomment-584344438 ### Does this PR introduce any user-facing change? No ### How was this patch tested? By existing test suites. Closes #27531 from MaxGekk/revert-like-3-args. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../expressions/regexpExpressions.scala | 85 ++++++------------- .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../spark/sql/DataFrameFunctionsSuite.scala | 15 ---- 4 files changed, 31 insertions(+), 75 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 40998080bc4e3..b4a8bafe22dfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -99,7 +99,7 @@ package object dsl { } def like(other: Expression, escapeChar: Char = '\\'): Expression = - Like(expr, other, Literal(escapeChar.toString)) + Like(expr, other, escapeChar) def rlike(other: Expression): Expression = RLike(expr, other) def contains(other: Expression): Expression = Contains(expr, other) def startsWith(other: Expression): Expression = StartsWith(expr, other) 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 f8d328bf601e4..e5ee0edfcf79b 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 @@ -22,7 +22,6 @@ import java.util.regex.{MatchResult, Pattern} import org.apache.commons.text.StringEscapeUtils -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{GenericArrayData, StringUtils} @@ -30,19 +29,17 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -trait StringRegexExpression extends Expression +abstract class StringRegexExpression extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { - def str: Expression - def pattern: Expression - def escape(v: String): String def matches(regex: Pattern, str: String): Boolean override def dataType: DataType = BooleanType + override def inputTypes: Seq[DataType] = Seq(StringType, StringType) // try cache foldable pattern - private lazy val cache: Pattern = pattern match { + private lazy val cache: Pattern = right match { case p: Expression if p.foldable => compile(p.eval().asInstanceOf[UTF8String].toString) case _ => null @@ -55,9 +52,10 @@ trait StringRegexExpression extends Expression Pattern.compile(escape(str)) } - def nullSafeMatch(input1: Any, input2: Any): Any = { - val s = input2.asInstanceOf[UTF8String].toString - val regex = if (cache == null) compile(s) else cache + protected def pattern(str: String) = if (cache == null) compile(str) else cache + + protected override def nullSafeEval(input1: Any, input2: Any): Any = { + val regex = pattern(input2.asInstanceOf[UTF8String].toString) if(regex == null) { null } else { @@ -65,7 +63,7 @@ trait StringRegexExpression extends Expression } } - override def sql: String = s"${str.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${pattern.sql}" + override def sql: String = s"${left.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}" } // scalastyle:off line.contains.tab @@ -110,65 +108,46 @@ trait StringRegexExpression extends Expression true > SELECT '%SystemDrive%/Users/John' _FUNC_ '/%SystemDrive/%//Users%' ESCAPE '/'; true - > SELECT _FUNC_('_Apache Spark_', '__%Spark__', '_'); - true """, note = """ Use RLIKE to match with standard regular expressions. """, since = "1.0.0") // scalastyle:on line.contains.tab -case class Like(str: Expression, pattern: Expression, escape: Expression) - extends TernaryExpression with StringRegexExpression { - - def this(str: Expression, pattern: Expression) = this(str, pattern, Literal("\\")) - - override def inputTypes: Seq[DataType] = Seq(StringType, StringType, StringType) - override def children: Seq[Expression] = Seq(str, pattern, escape) +case class Like(left: Expression, right: Expression, escapeChar: Char) + extends StringRegexExpression { - private lazy val escapeChar: Char = if (escape.foldable) { - escape.eval() match { - case s: UTF8String if s != null && s.numChars() == 1 => s.toString.charAt(0) - case s => throw new AnalysisException( - s"The 'escape' parameter must be a string literal of one char but it is $s.") - } - } else { - throw new AnalysisException("The 'escape' parameter must be a string literal.") - } + def this(left: Expression, right: Expression) = this(left, right, '\\') override def escape(v: String): String = StringUtils.escapeLikeRegex(v, escapeChar) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() override def toString: String = escapeChar match { - case '\\' => s"$str LIKE $pattern" - case c => s"$str LIKE $pattern ESCAPE '$c'" - } - - protected override def nullSafeEval(input1: Any, input2: Any, input3: Any): Any = { - nullSafeMatch(input1, input2) + case '\\' => s"$left LIKE $right" + case c => s"$left LIKE $right ESCAPE '$c'" } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val patternClass = classOf[Pattern].getName val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + ".escapeLikeRegex" - if (pattern.foldable) { - val patternVal = pattern.eval() - if (patternVal != null) { + if (right.foldable) { + val rVal = right.eval() + if (rVal != null) { val regexStr = - StringEscapeUtils.escapeJava(escape(patternVal.asInstanceOf[UTF8String].toString())) - val compiledPattern = ctx.addMutableState(patternClass, "compiledPattern", + StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString())) + val pattern = ctx.addMutableState(patternClass, "patternLike", v => s"""$v = $patternClass.compile("$regexStr");""") // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. - val eval = str.genCode(ctx) + val eval = left.genCode(ctx) ev.copy(code = code""" ${eval.code} boolean ${ev.isNull} = ${eval.isNull}; ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { - ${ev.value} = $compiledPattern.matcher(${eval.value}.toString()).matches(); + ${ev.value} = $pattern.matcher(${eval.value}.toString()).matches(); } """) } else { @@ -178,8 +157,8 @@ case class Like(str: Expression, pattern: Expression, escape: Expression) """) } } else { - val patternStr = ctx.freshName("patternStr") - val compiledPattern = ctx.freshName("compiledPattern") + val pattern = ctx.freshName("pattern") + val rightStr = ctx.freshName("rightStr") // We need double escape to avoid org.codehaus.commons.compiler.CompileException. // '\\' will cause exception 'Single quote must be backslash-escaped in character literal'. // '\"' will cause exception 'Line break in literal not allowed'. @@ -188,12 +167,12 @@ case class Like(str: Expression, pattern: Expression, escape: Expression) } else { escapeChar } - nullSafeCodeGen(ctx, ev, (eval1, eval2, _) => { + nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" - String $patternStr = $eval2.toString(); - $patternClass $compiledPattern = $patternClass.compile( - $escapeFunc($patternStr, '$newEscapeChar')); - ${ev.value} = $compiledPattern.matcher($eval1.toString()).matches(); + String $rightStr = $eval2.toString(); + $patternClass $pattern = $patternClass.compile( + $escapeFunc($rightStr, '$newEscapeChar')); + ${ev.value} = $pattern.matcher($eval1.toString()).matches(); """ }) } @@ -232,20 +211,12 @@ case class Like(str: Expression, pattern: Expression, escape: Expression) """, since = "1.0.0") // scalastyle:on line.contains.tab -case class RLike(left: Expression, right: Expression) - extends BinaryExpression with StringRegexExpression { - - override def inputTypes: Seq[DataType] = Seq(StringType, StringType) - - override def str: Expression = left - override def pattern: Expression = right +case class RLike(left: Expression, right: Expression) extends StringRegexExpression { override def escape(v: String): String = v override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) override def toString: String = s"$left RLIKE $right" - protected override def nullSafeEval(input1: Any, input2: Any): Any = nullSafeMatch(input1, input2) - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val patternClass = classOf[Pattern].getName diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 6fc65e14868e0..62e568587fcc6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1392,9 +1392,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging throw new ParseException("Invalid escape string." + "Escape string must contains only one character.", ctx) } - str + str.charAt(0) }.getOrElse('\\') - invertIfNotDefined(Like(e, expression(ctx.pattern), Literal(escapeChar))) + invertIfNotDefined(Like(e, expression(ctx.pattern), escapeChar)) case SqlBaseParser.RLIKE => invertIfNotDefined(RLike(e, expression(ctx.pattern))) case SqlBaseParser.NULL if ctx.NOT != null => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 9e9d8c3e9a7c5..6012678341ccc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -3560,21 +3560,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { Seq(Row(1))) } - test("the like function with the escape parameter") { - val df = Seq(("abc", "a_c", "!")).toDF("str", "pattern", "escape") - checkAnswer(df.selectExpr("like(str, pattern, '@')"), Row(true)) - - val longEscapeError = intercept[AnalysisException] { - df.selectExpr("like(str, pattern, '@%')").collect() - }.getMessage - assert(longEscapeError.contains("The 'escape' parameter must be a string literal of one char")) - - val nonFoldableError = intercept[AnalysisException] { - df.selectExpr("like(str, pattern, escape)").collect() - }.getMessage - assert(nonFoldableError.contains("The 'escape' parameter must be a string literal")) - } - test("SPARK-29462: Empty array of NullType for array function with no arguments") { Seq((true, StringType), (false, NullType)).foreach { case (arrayDefaultToString, expectedType) => From b25359cca3190f6a34dce3c3e49c4d2a80e88bdc Mon Sep 17 00:00:00 2001 From: herman Date: Wed, 12 Feb 2020 10:48:29 +0900 Subject: [PATCH 1350/1387] [SPARK-30780][SQL] Empty LocalTableScan should use RDD without partitions ### What changes were proposed in this pull request? This is a small follow-up for https://github.com/apache/spark/pull/27400. This PR makes an empty `LocalTableScanExec` return an `RDD` without partitions. ### Why are the changes needed? It is a bit unexpected that the RDD contains partitions if there is not work to do. It also can save a bit of work when this is used in a more complex plan. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Added test to `SparkPlanSuite`. Closes #27530 from hvanhovell/SPARK-30780. Authored-by: herman Signed-off-by: HyukjinKwon --- .../spark/sql/execution/LocalTableScanExec.scala | 12 ++++++++---- .../scala/org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../apache/spark/sql/execution/SparkPlanSuite.scala | 4 ++++ 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index 1b5115f2e29a3..b452213cd6cc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -45,10 +45,14 @@ case class LocalTableScanExec( } } - private lazy val numParallelism: Int = math.min(math.max(unsafeRows.length, 1), - sqlContext.sparkContext.defaultParallelism) - - private lazy val rdd = sqlContext.sparkContext.parallelize(unsafeRows, numParallelism) + @transient private lazy val rdd: RDD[InternalRow] = { + if (rows.isEmpty) { + sqlContext.sparkContext.emptyRDD + } else { + val numSlices = math.min(unsafeRows.length, sqlContext.sparkContext.defaultParallelism) + sqlContext.sparkContext.parallelize(unsafeRows, numSlices) + } + } protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") 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 d2d58a83ded5d..694e576fcded4 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 @@ -330,7 +330,7 @@ class DataFrameSuite extends QueryTest testData.select("key").coalesce(1).select("key"), testData.select("key").collect().toSeq) - assert(spark.emptyDataFrame.coalesce(1).rdd.partitions.size === 1) + assert(spark.emptyDataFrame.coalesce(1).rdd.partitions.size === 0) } test("convert $\"attribute name\" into unresolved attribute") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index e3bc414516c04..56fff1107ae39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala @@ -84,4 +84,8 @@ class SparkPlanSuite extends QueryTest with SharedSparkSession { } } } + + test("SPARK-30780 empty LocalTableScan should use RDD without partitions") { + assert(LocalTableScanExec(Nil, Nil).execute().getNumPartitions == 0) + } } From aa6a60530e63ab3bb8b117f8738973d1b26a2cc7 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 12 Feb 2020 10:49:46 +0900 Subject: [PATCH 1351/1387] [SPARK-30722][PYTHON][DOCS] Update documentation for Pandas UDF with Python type hints ### What changes were proposed in this pull request? This PR targets to document the Pandas UDF redesign with type hints introduced at SPARK-28264. Mostly self-describing; however, there are few things to note for reviewers. 1. This PR replace the existing documentation of pandas UDFs to the newer redesign to promote the Python type hints. I added some words that Spark 3.0 still keeps the compatibility though. 2. This PR proposes to name non-pandas UDFs as "Pandas Function API" 3. SCALAR_ITER become two separate sections to reduce confusion: - `Iterator[pd.Series]` -> `Iterator[pd.Series]` - `Iterator[Tuple[pd.Series, ...]]` -> `Iterator[pd.Series]` 4. I removed some examples that look overkill to me. 5. I also removed some information in the doc, that seems duplicating or too much. ### Why are the changes needed? To document new redesign in pandas UDF. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing tests should cover. Closes #27466 from HyukjinKwon/SPARK-30722. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- dev/sparktestsupport/modules.py | 1 - docs/sql-pyspark-pandas-with-arrow.md | 233 +++++++---- examples/src/main/python/sql/arrow.py | 258 ++++++------ python/pyspark/sql/pandas/functions.py | 538 +++++++++++-------------- python/pyspark/sql/pandas/group_ops.py | 99 ++++- python/pyspark/sql/pandas/map_ops.py | 6 +- python/pyspark/sql/udf.py | 16 +- 7 files changed, 609 insertions(+), 542 deletions(-) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 40f2ca288d694..391e4bbe1b1f0 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -364,7 +364,6 @@ def __hash__(self): "pyspark.sql.avro.functions", "pyspark.sql.pandas.conversion", "pyspark.sql.pandas.map_ops", - "pyspark.sql.pandas.functions", "pyspark.sql.pandas.group_ops", "pyspark.sql.pandas.types", "pyspark.sql.pandas.serializers", diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md index 7eb8a74547f70..92a515746b607 100644 --- a/docs/sql-pyspark-pandas-with-arrow.md +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -35,7 +35,7 @@ working with Arrow-enabled data. If you install PySpark using pip, then PyArrow can be brought in as an extra dependency of the SQL module with the command `pip install pyspark[sql]`. Otherwise, you must ensure that PyArrow -is installed and available on all cluster nodes. The current supported version is 0.12.1. +is installed and available on all cluster nodes. The current supported version is 0.15.1+. You can install using pip or conda from the conda-forge channel. See PyArrow [installation](https://arrow.apache.org/docs/python/install.html) for details. @@ -65,132 +65,216 @@ Spark will fall back to create the DataFrame without Arrow. ## Pandas UDFs (a.k.a. Vectorized UDFs) -Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and -Pandas to work with the data. A Pandas UDF is defined using the keyword `pandas_udf` as a decorator -or to wrap the function, no additional configuration is required. Currently, there are two types of -Pandas UDF: Scalar and Grouped Map. +Pandas UDFs are user defined functions that are executed by Spark using +Arrow to transfer data and Pandas to work with the data, which allows vectorized operations. A Pandas +UDF is defined using the `pandas_udf` as a decorator or to wrap the function, and no additional +configuration is required. A Pandas UDF behaves as a regular PySpark function API in general. -### Scalar +Before Spark 3.0, Pandas UDFs used to be defined with `PandasUDFType`. From Spark 3.0 +with Python 3.6+, you can also use [Python type hints](https://www.python.org/dev/peps/pep-0484). +Using Python type hints are preferred and using `PandasUDFType` will be deprecated in +the future release. -Scalar Pandas UDFs are used for vectorizing scalar operations. They can be used with functions such -as `select` and `withColumn`. The Python function should take `pandas.Series` as inputs and return -a `pandas.Series` of the same length. Internally, Spark will execute a Pandas UDF by splitting -columns into batches and calling the function for each batch as a subset of the data, then -concatenating the results together. +Note that the type hint should use `pandas.Series` in all cases but there is one variant +that `pandas.DataFrame` should be used for its input or output type hint instead when the input +or output column is of `StructType`. The following example shows a Pandas UDF which takes long +column, string column and struct column, and outputs a struct column. It requires the function to +specify the type hints of `pandas.Series` and `pandas.DataFrame` as below: -The following example shows how to create a scalar Pandas UDF that computes the product of 2 columns. +

    +

    +
    +{% include_example ser_to_frame_pandas_udf python/sql/arrow.py %} +
    +
    +

    + +In the following sections, it describes the cominations of the supported type hints. For simplicity, +`pandas.DataFrame` variant is omitted. + +### Series to Series + +The type hint can be expressed as `pandas.Series`, ... -> `pandas.Series`. + +By using `pandas_udf` with the function having such type hints, it creates a Pandas UDF where the given +function takes one or more `pandas.Series` and outputs one `pandas.Series`. The output of the function should +always be of the same length as the input. Internally, PySpark will execute a Pandas UDF by splitting +columns into batches and calling the function for each batch as a subset of the data, then concatenating +the results together. + +The following example shows how to create this Pandas UDF that computes the product of 2 columns.
    -{% include_example scalar_pandas_udf python/sql/arrow.py %} +{% include_example ser_to_ser_pandas_udf python/sql/arrow.py %}
    -### Scalar Iterator +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) + +### Iterator of Series to Iterator of Series + +The type hint can be expressed as `Iterator[pandas.Series]` -> `Iterator[pandas.Series]`. + +By using `pandas_udf` with the function having such type hints, it creates a Pandas UDF where the given +function takes an iterator of `pandas.Series` and outputs an iterator of `pandas.Series`. The output of each +series from the function should always be of the same length as the input. In this case, the created +Pandas UDF requires one input column when the Pandas UDF is called. To use multiple input columns, +a different type hint is required. See Iterator of Multiple Series to Iterator of Series. + +It is useful when the UDF execution requires initializing some states although internally it works +identically as Series to Series case. The pseudocode below illustrates the example. + +{% highlight python %} +@pandas_udf("long") +def calculate(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]: + # Do some expensive initialization with a state + state = very_expensive_initialization() + for x in iterator: + # Use that state for whole iterator. + yield calculate_with_state(x, state) -Scalar iterator (`SCALAR_ITER`) Pandas UDF is the same as scalar Pandas UDF above except that the -underlying Python function takes an iterator of batches as input instead of a single batch and, -instead of returning a single output batch, it yields output batches or returns an iterator of -output batches. -It is useful when the UDF execution requires initializing some states, e.g., loading an machine -learning model file to apply inference to every input batch. +df.select(calculate("value")).show() +{% endhighlight %} -The following example shows how to create scalar iterator Pandas UDFs: +The following example shows how to create this Pandas UDF:
    -{% include_example scalar_iter_pandas_udf python/sql/arrow.py %} +{% include_example iter_ser_to_iter_ser_pandas_udf python/sql/arrow.py %}
    -### Grouped Map -Grouped map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern. -Split-apply-combine consists of three steps: -* Split the data into groups by using `DataFrame.groupBy`. -* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The - input data contains all the rows and columns for each group. -* Combine the results into a new `DataFrame`. +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) -To use `groupBy().apply()`, the user needs to define the following: -* A Python function that defines the computation for each group. -* A `StructType` object or a string that defines the schema of the output `DataFrame`. +### Iterator of Multiple Series to Iterator of Series -The column labels of the returned `pandas.DataFrame` must either match the field names in the -defined output schema if specified as strings, or match the field data types by position if not -strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) -on how to label columns when constructing a `pandas.DataFrame`. +The type hint can be expressed as `Iterator[Tuple[pandas.Series, ...]]` -> `Iterator[pandas.Series]`. -Note that all data for a group will be loaded into memory before the function is applied. This can -lead to out of memory exceptions, especially if the group sizes are skewed. The configuration for -[maxRecordsPerBatch](#setting-arrow-batch-size) is not applied on groups and it is up to the user -to ensure that the grouped data will fit into the available memory. +By using `pandas_udf` with the function having such type hints, it creates a Pandas UDF where the +given function takes an iterator of a tuple of multiple `pandas.Series` and outputs an iterator of `pandas.Series`. +In this case, the created pandas UDF requires multiple input columns as many as the series in the tuple +when the Pandas UDF is called. It works identically as Iterator of Series to Iterator of Series case except the parameter difference. -The following example shows how to use `groupby().apply()` to subtract the mean from each value in the group. +The following example shows how to create this Pandas UDF:
    -{% include_example grouped_map_pandas_udf python/sql/arrow.py %} +{% include_example iter_sers_to_iter_ser_pandas_udf python/sql/arrow.py %}
    -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and -[`pyspark.sql.GroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.GroupedData.apply). +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) + +### Series to Scalar -### Grouped Aggregate +The type hint can be expressed as `pandas.Series`, ... -> `Any`. -Grouped aggregate Pandas UDFs are similar to Spark aggregate functions. Grouped aggregate Pandas UDFs are used with `groupBy().agg()` and -[`pyspark.sql.Window`](api/python/pyspark.sql.html#pyspark.sql.Window). It defines an aggregation from one or more `pandas.Series` -to a scalar value, where each `pandas.Series` represents a column within the group or window. +By using `pandas_udf` with the function having such type hints, it creates a Pandas UDF similar +to PySpark's aggregate functions. The given function takes `pandas.Series` and returns a scalar value. +The return type should be a primitive data type, and the returned scalar can be either a python +primitive type, e.g., `int` or `float` or a numpy data type, e.g., `numpy.int64` or `numpy.float64`. +`Any` should ideally be a specific scalar type accordingly. -Note that this type of UDF does not support partial aggregation and all data for a group or window will be loaded into memory. Also, -only unbounded window is supported with Grouped aggregate Pandas UDFs currently. +This UDF can be also used with `groupBy().agg()` and [`pyspark.sql.Window`](api/python/pyspark.sql.html#pyspark.sql.Window). +It defines an aggregation from one or more `pandas.Series` to a scalar value, where each `pandas.Series` +represents a column within the group or window. -The following example shows how to use this type of UDF to compute mean with groupBy and window operations: +Note that this type of UDF does not support partial aggregation and all data for a group or window +will be loaded into memory. Also, only unbounded window is supported with Grouped aggregate Pandas +UDFs currently. The following example shows how to use this type of UDF to compute mean with a group-by +and window operations:
    -{% include_example grouped_agg_pandas_udf python/sql/arrow.py %} +{% include_example ser_to_scalar_pandas_udf python/sql/arrow.py %}
    For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) -### Map Iterator +## Pandas Function APIs + +Pandas function APIs can directly apply a Python native function against the whole DataFrame by +using Pandas instances. Internally it works similarly with Pandas UDFs by Spark using Arrow to transfer +data and Pandas to work with the data, which allows vectorized operations. A Pandas function API behaves +as a regular API under PySpark `DataFrame` in general. + +From Spark 3.0, Grouped map pandas UDF is now categorized as a separate Pandas Function API, +`DataFrame.groupby().applyInPandas()`. It is still possible to use it with `PandasUDFType` +and `DataFrame.groupby().apply()` as it was; however, it is preferred to use +`DataFrame.groupby().applyInPandas()` directly. Using `PandasUDFType` will be deprecated +in the future. + +### Grouped Map + +Grouped map operations with Pandas instances are supported by `DataFrame.groupby().applyInPandas()` +which requires a Python function that takes a `pandas.DataFrame` and return another `pandas.DataFrame`. +It maps each group to each `pandas.DataFrame` in the Python function. + +This API implements the "split-apply-combine" pattern which consists of three steps: +* Split the data into groups by using `DataFrame.groupBy`. +* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The + input data contains all the rows and columns for each group. +* Combine the results into a new PySpark `DataFrame`. -Map iterator Pandas UDFs are used to transform data with an iterator of batches. Map iterator -Pandas UDFs can be used with -[`pyspark.sql.DataFrame.mapInPandas`](api/python/pyspark.sql.html#pyspark.sql.DataFrame.mapInPandas). -It defines a map function that transforms an iterator of `pandas.DataFrame` to another. +To use `groupBy().applyInPandas()`, the user needs to define the following: +* A Python function that defines the computation for each group. +* A `StructType` object or a string that defines the schema of the output PySpark `DataFrame`. + +The column labels of the returned `pandas.DataFrame` must either match the field names in the +defined output schema if specified as strings, or match the field data types by position if not +strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) +on how to label columns when constructing a `pandas.DataFrame`. -It can return the output of arbitrary length in contrast to the scalar Pandas UDF. It maps an iterator of `pandas.DataFrame`s, -that represents the current `DataFrame`, using the map iterator UDF and returns the result as a `DataFrame`. +Note that all data for a group will be loaded into memory before the function is applied. This can +lead to out of memory exceptions, especially if the group sizes are skewed. The configuration for +[maxRecordsPerBatch](#setting-arrow-batch-size) is not applied on groups and it is up to the user +to ensure that the grouped data will fit into the available memory. -The following example shows how to create map iterator Pandas UDFs: +The following example shows how to use `groupby().applyInPandas()` to subtract the mean from each value +in the group.
    -{% include_example map_iter_pandas_udf python/sql/arrow.py %} +{% include_example grouped_apply_in_pandas python/sql/arrow.py %}
    -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and -[`pyspark.sql.DataFrame.mapsInPandas`](api/python/pyspark.sql.html#pyspark.sql.DataFrame.mapInPandas). +For detailed usage, please see [`pyspark.sql.GroupedData.applyInPandas`](api/python/pyspark.sql.html#pyspark.sql.GroupedData.applyInPandas). +### Map + +Map operations with Pandas instances are supported by `DataFrame.mapInPandas()` which maps an iterator +of `pandas.DataFrame`s to another iterator of `pandas.DataFrame`s that represents the current +PySpark `DataFrame` and returns the result as a PySpark `DataFrame`. The functions takes and outputs +an iterator of `pandas.DataFrame`. It can return the output of arbitrary length in contrast to some +Pandas UDFs although internally it works similarly with Series to Series Pandas UDF. + +The following example shows how to use `mapInPandas()`: + +
    +
    +{% include_example map_in_pandas python/sql/arrow.py %} +
    +
    -### Cogrouped Map +For detailed usage, please see [`pyspark.sql.DataFrame.mapsInPandas`](api/python/pyspark.sql.html#pyspark.sql.DataFrame.mapInPandas). -Cogrouped map Pandas UDFs allow two DataFrames to be cogrouped by a common key and then a python function applied to -each cogroup. They are used with `groupBy().cogroup().apply()` which consists of the following steps: +### Co-grouped Map +Co-grouped map operations with Pandas instances are supported by `DataFrame.groupby().cogroup().applyInPandas()` which +allows two PySpark `DataFrame`s to be cogrouped by a common key and then a Python function applied to each +cogroup. It consists of the following steps: * Shuffle the data such that the groups of each dataframe which share a key are cogrouped together. -* Apply a function to each cogroup. The input of the function is two `pandas.DataFrame` (with an optional Tuple -representing the key). The output of the function is a `pandas.DataFrame`. -* Combine the pandas.DataFrames from all groups into a new `DataFrame`. +* Apply a function to each cogroup. The input of the function is two `pandas.DataFrame` (with an optional tuple +representing the key). The output of the function is a `pandas.DataFrame`. +* Combine the `pandas.DataFrame`s from all groups into a new PySpark `DataFrame`. -To use `groupBy().cogroup().apply()`, the user needs to define the following: +To use `groupBy().cogroup().applyInPandas()`, the user needs to define the following: * A Python function that defines the computation for each cogroup. -* A `StructType` object or a string that defines the schema of the output `DataFrame`. +* A `StructType` object or a string that defines the schema of the output PySpark `DataFrame`. The column labels of the returned `pandas.DataFrame` must either match the field names in the defined output schema if specified as strings, or match the field data types by position if not @@ -201,16 +285,15 @@ Note that all data for a cogroup will be loaded into memory before the function memory exceptions, especially if the group sizes are skewed. The configuration for [maxRecordsPerBatch](#setting-arrow-batch-size) is not applied and it is up to the user to ensure that the cogrouped data will fit into the available memory. -The following example shows how to use `groupby().cogroup().apply()` to perform an asof join between two datasets. +The following example shows how to use `groupby().cogroup().applyInPandas()` to perform an asof join between two datasets.
    -{% include_example cogrouped_map_pandas_udf python/sql/arrow.py %} +{% include_example cogrouped_apply_in_pandas python/sql/arrow.py %}
    -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and -[`pyspark.sql.CoGroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.CoGroupedData.apply). +For detailed usage, please see [`pyspark.sql.PandasCogroupedOps.applyInPandas()`](api/python/pyspark.sql.html#pyspark.sql.PandasCogroupedOps.applyInPandas). ## Usage Notes diff --git a/examples/src/main/python/sql/arrow.py b/examples/src/main/python/sql/arrow.py index 1c983172d36ef..b7d8467172fab 100644 --- a/examples/src/main/python/sql/arrow.py +++ b/examples/src/main/python/sql/arrow.py @@ -23,12 +23,19 @@ from __future__ import print_function +import sys + from pyspark.sql import SparkSession from pyspark.sql.pandas.utils import require_minimum_pandas_version, require_minimum_pyarrow_version require_minimum_pandas_version() require_minimum_pyarrow_version() +if sys.version_info < (3, 6): + raise Exception( + "Running this example file requires Python 3.6+; however, " + "your Python version was:\n %s" % sys.version) + def dataframe_with_arrow_example(spark): # $example on:dataframe_with_arrow$ @@ -50,15 +57,45 @@ def dataframe_with_arrow_example(spark): print("Pandas DataFrame result statistics:\n%s\n" % str(result_pdf.describe())) -def scalar_pandas_udf_example(spark): - # $example on:scalar_pandas_udf$ +def ser_to_frame_pandas_udf_example(spark): + # $example on:ser_to_frame_pandas_udf$ + import pandas as pd + + from pyspark.sql.functions import pandas_udf + + @pandas_udf("col1 string, col2 long") + def func(s1: pd.Series, s2: pd.Series, s3: pd.DataFrame) -> pd.DataFrame: + s3['col2'] = s1 + s2.str.len() + return s3 + + # Create a Spark DataFrame that has three columns including a sturct column. + df = spark.createDataFrame( + [[1, "a string", ("a nested string",)]], + "long_col long, string_col string, struct_col struct") + + df.printSchema() + # root + # |-- long_column: long (nullable = true) + # |-- string_column: string (nullable = true) + # |-- struct_column: struct (nullable = true) + # | |-- col1: string (nullable = true) + + df.select(func("long_col", "string_col", "struct_col")).printSchema() + # |-- func(long_col, string_col, struct_col): struct (nullable = true) + # | |-- col1: string (nullable = true) + # | |-- col2: long (nullable = true) + # $example off:ser_to_frame_pandas_udf$$ + + +def ser_to_ser_pandas_udf_example(spark): + # $example on:ser_to_ser_pandas_udf$ import pandas as pd from pyspark.sql.functions import col, pandas_udf from pyspark.sql.types import LongType # Declare the function and create the UDF - def multiply_func(a, b): + def multiply_func(a: pd.Series, b: pd.Series) -> pd.Series: return a * b multiply = pandas_udf(multiply_func, returnType=LongType()) @@ -83,26 +120,27 @@ def multiply_func(a, b): # | 4| # | 9| # +-------------------+ - # $example off:scalar_pandas_udf$ + # $example off:ser_to_ser_pandas_udf$ -def scalar_iter_pandas_udf_example(spark): - # $example on:scalar_iter_pandas_udf$ +def iter_ser_to_iter_ser_pandas_udf_example(spark): + # $example on:iter_ser_to_iter_ser_pandas_udf$ + from typing import Iterator + import pandas as pd - from pyspark.sql.functions import col, pandas_udf, struct, PandasUDFType + from pyspark.sql.functions import pandas_udf pdf = pd.DataFrame([1, 2, 3], columns=["x"]) df = spark.createDataFrame(pdf) - # When the UDF is called with a single column that is not StructType, - # the input to the underlying function is an iterator of pd.Series. - @pandas_udf("long", PandasUDFType.SCALAR_ITER) - def plus_one(batch_iter): - for x in batch_iter: + # Declare the function and create the UDF + @pandas_udf("long") + def plus_one(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]: + for x in iterator: yield x + 1 - df.select(plus_one(col("x"))).show() + df.select(plus_one("x")).show() # +-----------+ # |plus_one(x)| # +-----------+ @@ -110,15 +148,28 @@ def plus_one(batch_iter): # | 3| # | 4| # +-----------+ + # $example off:iter_ser_to_iter_ser_pandas_udf$ + + +def iter_sers_to_iter_ser_pandas_udf_example(spark): + # $example on:iter_sers_to_iter_ser_pandas_udf$ + from typing import Iterator, Tuple + + import pandas as pd - # When the UDF is called with more than one columns, - # the input to the underlying function is an iterator of pd.Series tuple. - @pandas_udf("long", PandasUDFType.SCALAR_ITER) - def multiply_two_cols(batch_iter): - for a, b in batch_iter: + from pyspark.sql.functions import pandas_udf + + pdf = pd.DataFrame([1, 2, 3], columns=["x"]) + df = spark.createDataFrame(pdf) + + # Declare the function and create the UDF + @pandas_udf("long") + def multiply_two_cols( + iterator: Iterator[Tuple[pd.Series, pd.Series]]) -> Iterator[pd.Series]: + for a, b in iterator: yield a * b - df.select(multiply_two_cols(col("x"), col("x"))).show() + df.select(multiply_two_cols("x", "x")).show() # +-----------------------+ # |multiply_two_cols(x, x)| # +-----------------------+ @@ -126,92 +177,32 @@ def multiply_two_cols(batch_iter): # | 4| # | 9| # +-----------------------+ + # $example off:iter_sers_to_iter_ser_pandas_udf$ - # When the UDF is called with a single column that is StructType, - # the input to the underlying function is an iterator of pd.DataFrame. - @pandas_udf("long", PandasUDFType.SCALAR_ITER) - def multiply_two_nested_cols(pdf_iter): - for pdf in pdf_iter: - yield pdf["a"] * pdf["b"] - - df.select( - multiply_two_nested_cols( - struct(col("x").alias("a"), col("x").alias("b")) - ).alias("y") - ).show() - # +---+ - # | y| - # +---+ - # | 1| - # | 4| - # | 9| - # +---+ - - # In the UDF, you can initialize some states before processing batches. - # Wrap your code with try/finally or use context managers to ensure - # the release of resources at the end. - y_bc = spark.sparkContext.broadcast(1) - - @pandas_udf("long", PandasUDFType.SCALAR_ITER) - def plus_y(batch_iter): - y = y_bc.value # initialize states - try: - for x in batch_iter: - yield x + y - finally: - pass # release resources here, if any - - df.select(plus_y(col("x"))).show() - # +---------+ - # |plus_y(x)| - # +---------+ - # | 2| - # | 3| - # | 4| - # +---------+ - # $example off:scalar_iter_pandas_udf$ - - -def grouped_map_pandas_udf_example(spark): - # $example on:grouped_map_pandas_udf$ - from pyspark.sql.functions import pandas_udf, PandasUDFType - - df = spark.createDataFrame( - [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], - ("id", "v")) - - @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) - def subtract_mean(pdf): - # pdf is a pandas.DataFrame - v = pdf.v - return pdf.assign(v=v - v.mean()) - - df.groupby("id").apply(subtract_mean).show() - # +---+----+ - # | id| v| - # +---+----+ - # | 1|-0.5| - # | 1| 0.5| - # | 2|-3.0| - # | 2|-1.0| - # | 2| 4.0| - # +---+----+ - # $example off:grouped_map_pandas_udf$ +def ser_to_scalar_pandas_udf_example(spark): + # $example on:ser_to_scalar_pandas_udf$ + import pandas as pd -def grouped_agg_pandas_udf_example(spark): - # $example on:grouped_agg_pandas_udf$ - from pyspark.sql.functions import pandas_udf, PandasUDFType + from pyspark.sql.functions import pandas_udf from pyspark.sql import Window df = spark.createDataFrame( [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v")) - @pandas_udf("double", PandasUDFType.GROUPED_AGG) - def mean_udf(v): + # Declare the function and create the UDF + @pandas_udf("double") + def mean_udf(v: pd.Series) -> float: return v.mean() + df.select(mean_udf(df['v'])).show() + # +-----------+ + # |mean_udf(v)| + # +-----------+ + # | 4.2| + # +-----------+ + df.groupby("id").agg(mean_udf(df['v'])).show() # +---+-----------+ # | id|mean_udf(v)| @@ -233,37 +224,54 @@ def mean_udf(v): # | 2| 5.0| 6.0| # | 2|10.0| 6.0| # +---+----+------+ - # $example off:grouped_agg_pandas_udf$ + # $example off:ser_to_scalar_pandas_udf$ -def map_iter_pandas_udf_example(spark): - # $example on:map_iter_pandas_udf$ - import pandas as pd +def grouped_apply_in_pandas_example(spark): + # $example on:grouped_apply_in_pandas$ + df = spark.createDataFrame( + [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], + ("id", "v")) - from pyspark.sql.functions import pandas_udf, PandasUDFType + def subtract_mean(pdf): + # pdf is a pandas.DataFrame + v = pdf.v + return pdf.assign(v=v - v.mean()) + + df.groupby("id").applyInPandas(subtract_mean, schema="id long, v double").show() + # +---+----+ + # | id| v| + # +---+----+ + # | 1|-0.5| + # | 1| 0.5| + # | 2|-3.0| + # | 2|-1.0| + # | 2| 4.0| + # +---+----+ + # $example off:grouped_apply_in_pandas$ + +def map_in_pandas_example(spark): + # $example on:map_in_pandas$ df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age")) - @pandas_udf(df.schema, PandasUDFType.MAP_ITER) - def filter_func(batch_iter): - for pdf in batch_iter: + def filter_func(iterator): + for pdf in iterator: yield pdf[pdf.id == 1] - df.mapInPandas(filter_func).show() + df.mapInPandas(filter_func, schema=df.schema).show() # +---+---+ # | id|age| # +---+---+ # | 1| 21| # +---+---+ - # $example off:map_iter_pandas_udf$ + # $example off:map_in_pandas$ -def cogrouped_map_pandas_udf_example(spark): - # $example on:cogrouped_map_pandas_udf$ +def cogrouped_apply_in_pandas_example(spark): + # $example on:cogrouped_apply_in_pandas$ import pandas as pd - from pyspark.sql.functions import pandas_udf, PandasUDFType - df1 = spark.createDataFrame( [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], ("time", "id", "v1")) @@ -272,11 +280,11 @@ def cogrouped_map_pandas_udf_example(spark): [(20000101, 1, "x"), (20000101, 2, "y")], ("time", "id", "v2")) - @pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP) def asof_join(l, r): return pd.merge_asof(l, r, on="time", by="id") - df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show() + df1.groupby("id").cogroup(df2.groupby("id")).applyInPandas( + asof_join, schema="time int, id int, v1 double, v2 string").show() # +--------+---+---+---+ # | time| id| v1| v2| # +--------+---+---+---+ @@ -285,7 +293,7 @@ def asof_join(l, r): # |20000101| 2|2.0| y| # |20000102| 2|4.0| y| # +--------+---+---+---+ - # $example off:cogrouped_map_pandas_udf$ + # $example off:cogrouped_apply_in_pandas$ if __name__ == "__main__": @@ -296,17 +304,21 @@ def asof_join(l, r): print("Running Pandas to/from conversion example") dataframe_with_arrow_example(spark) - print("Running pandas_udf scalar example") - scalar_pandas_udf_example(spark) - print("Running pandas_udf scalar iterator example") - scalar_iter_pandas_udf_example(spark) - print("Running pandas_udf grouped map example") - grouped_map_pandas_udf_example(spark) - print("Running pandas_udf grouped agg example") - grouped_agg_pandas_udf_example(spark) - print("Running pandas_udf map iterator example") - map_iter_pandas_udf_example(spark) - print("Running pandas_udf cogrouped map example") - cogrouped_map_pandas_udf_example(spark) + print("Running pandas_udf example: Series to Frame") + ser_to_frame_pandas_udf_example(spark) + print("Running pandas_udf example: Series to Series") + ser_to_ser_pandas_udf_example(spark) + print("Running pandas_udf example: Iterator of Series to Iterator of Seires") + iter_ser_to_iter_ser_pandas_udf_example(spark) + print("Running pandas_udf example: Iterator of Multiple Series to Iterator of Series") + iter_sers_to_iter_ser_pandas_udf_example(spark) + print("Running pandas_udf example: Series to Scalar") + ser_to_scalar_pandas_udf_example(spark) + print("Running pandas function example: Grouped Map") + grouped_apply_in_pandas_example(spark) + print("Running pandas function example: Map") + map_in_pandas_example(spark) + print("Running pandas function example: Co-grouped Map") + cogrouped_apply_in_pandas_example(spark) spark.stop() diff --git a/python/pyspark/sql/pandas/functions.py b/python/pyspark/sql/pandas/functions.py index 30602789a33a9..31aa321bf5826 100644 --- a/python/pyspark/sql/pandas/functions.py +++ b/python/pyspark/sql/pandas/functions.py @@ -43,303 +43,228 @@ class PandasUDFType(object): @since(2.3) def pandas_udf(f=None, returnType=None, functionType=None): """ - Creates a vectorized user defined function (UDF). + Creates a pandas user defined function (a.k.a. vectorized user defined function). + + Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer + data and Pandas to work with the data, which allows vectorized operations. A Pandas UDF + is defined using the `pandas_udf` as a decorator or to wrap the function, and no + additional configuration is required. A Pandas UDF behaves as a regular PySpark function + API in general. :param f: user-defined function. A python function if used as a standalone function :param returnType: the return type of the user-defined function. The value can be either a :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. :param functionType: an enum value in :class:`pyspark.sql.functions.PandasUDFType`. - Default: SCALAR. - - .. seealso:: :meth:`pyspark.sql.DataFrame.mapInPandas` - .. seealso:: :meth:`pyspark.sql.GroupedData.applyInPandas` - .. seealso:: :meth:`pyspark.sql.PandasCogroupedOps.applyInPandas` - - The function type of the UDF can be one of the following: - - 1. SCALAR - - A scalar UDF defines a transformation: One or more `pandas.Series` -> A `pandas.Series`. - The length of the returned `pandas.Series` must be of the same as the input `pandas.Series`. - If the return type is :class:`StructType`, the returned value should be a `pandas.DataFrame`. - - :class:`MapType`, nested :class:`StructType` are currently not supported as output types. - - Scalar UDFs can be used with :meth:`pyspark.sql.DataFrame.withColumn` and - :meth:`pyspark.sql.DataFrame.select`. - - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType - >>> from pyspark.sql.types import IntegerType, StringType - >>> slen = pandas_udf(lambda s: s.str.len(), IntegerType()) # doctest: +SKIP - >>> @pandas_udf(StringType()) # doctest: +SKIP - ... def to_upper(s): - ... return s.str.upper() - ... - >>> @pandas_udf("integer", PandasUDFType.SCALAR) # doctest: +SKIP - ... def add_one(x): - ... return x + 1 - ... - >>> df = spark.createDataFrame([(1, "John Doe", 21)], - ... ("id", "name", "age")) # doctest: +SKIP - >>> df.select(slen("name").alias("slen(name)"), to_upper("name"), add_one("age")) \\ - ... .show() # doctest: +SKIP - +----------+--------------+------------+ - |slen(name)|to_upper(name)|add_one(age)| - +----------+--------------+------------+ - | 8| JOHN DOE| 22| - +----------+--------------+------------+ - >>> @pandas_udf("first string, last string") # doctest: +SKIP - ... def split_expand(n): - ... return n.str.split(expand=True) - >>> df.select(split_expand("name")).show() # doctest: +SKIP - +------------------+ - |split_expand(name)| - +------------------+ - | [John, Doe]| - +------------------+ - - .. note:: The length of `pandas.Series` within a scalar UDF is not that of the whole input - column, but is the length of an internal batch used for each call to the function. - Therefore, this can be used, for example, to ensure the length of each returned - `pandas.Series`, and can not be used as the column length. - - 2. SCALAR_ITER - - A scalar iterator UDF is semantically the same as the scalar Pandas UDF above except that the - wrapped Python function takes an iterator of batches as input instead of a single batch and, - instead of returning a single output batch, it yields output batches or explicitly returns an - generator or an iterator of output batches. - It is useful when the UDF execution requires initializing some state, e.g., loading a machine - learning model file to apply inference to every input batch. - - .. note:: It is not guaranteed that one invocation of a scalar iterator UDF will process all - batches from one partition, although it is currently implemented this way. - Your code shall not rely on this behavior because it might change in the future for - further optimization, e.g., one invocation processes multiple partitions. - - Scalar iterator UDFs are used with :meth:`pyspark.sql.DataFrame.withColumn` and - :meth:`pyspark.sql.DataFrame.select`. - - >>> import pandas as pd # doctest: +SKIP - >>> from pyspark.sql.functions import col, pandas_udf, struct, PandasUDFType - >>> pdf = pd.DataFrame([1, 2, 3], columns=["x"]) # doctest: +SKIP - >>> df = spark.createDataFrame(pdf) # doctest: +SKIP - - When the UDF is called with a single column that is not `StructType`, the input to the - underlying function is an iterator of `pd.Series`. - - >>> @pandas_udf("long", PandasUDFType.SCALAR_ITER) # doctest: +SKIP - ... def plus_one(batch_iter): - ... for x in batch_iter: - ... yield x + 1 - ... - >>> df.select(plus_one(col("x"))).show() # doctest: +SKIP - +-----------+ - |plus_one(x)| - +-----------+ - | 2| - | 3| - | 4| - +-----------+ - - When the UDF is called with more than one columns, the input to the underlying function is an - iterator of `pd.Series` tuple. - - >>> @pandas_udf("long", PandasUDFType.SCALAR_ITER) # doctest: +SKIP - ... def multiply_two_cols(batch_iter): - ... for a, b in batch_iter: - ... yield a * b - ... - >>> df.select(multiply_two_cols(col("x"), col("x"))).show() # doctest: +SKIP - +-----------------------+ - |multiply_two_cols(x, x)| - +-----------------------+ - | 1| - | 4| - | 9| - +-----------------------+ - - When the UDF is called with a single column that is `StructType`, the input to the underlying - function is an iterator of `pd.DataFrame`. - - >>> @pandas_udf("long", PandasUDFType.SCALAR_ITER) # doctest: +SKIP - ... def multiply_two_nested_cols(pdf_iter): - ... for pdf in pdf_iter: - ... yield pdf["a"] * pdf["b"] - ... - >>> df.select( - ... multiply_two_nested_cols( - ... struct(col("x").alias("a"), col("x").alias("b")) - ... ).alias("y") - ... ).show() # doctest: +SKIP - +---+ - | y| - +---+ - | 1| - | 4| - | 9| - +---+ - - In the UDF, you can initialize some states before processing batches, wrap your code with - `try ... finally ...` or use context managers to ensure the release of resources at the end - or in case of early termination. - - >>> y_bc = spark.sparkContext.broadcast(1) # doctest: +SKIP - >>> @pandas_udf("long", PandasUDFType.SCALAR_ITER) # doctest: +SKIP - ... def plus_y(batch_iter): - ... y = y_bc.value # initialize some state - ... try: - ... for x in batch_iter: - ... yield x + y - ... finally: - ... pass # release resources here, if any - ... - >>> df.select(plus_y(col("x"))).show() # doctest: +SKIP - +---------+ - |plus_y(x)| - +---------+ - | 2| - | 3| - | 4| - +---------+ - - 3. GROUPED_MAP - - A grouped map UDF defines transformation: A `pandas.DataFrame` -> A `pandas.DataFrame` - The returnType should be a :class:`StructType` describing the schema of the returned - `pandas.DataFrame`. The column labels of the returned `pandas.DataFrame` must either match - the field names in the defined returnType schema if specified as strings, or match the - field data types by position if not strings, e.g. integer indices. - The length of the returned `pandas.DataFrame` can be arbitrary. - - Grouped map UDFs are used with :meth:`pyspark.sql.GroupedData.apply`. - - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType - >>> df = spark.createDataFrame( - ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], - ... ("id", "v")) # doctest: +SKIP - >>> @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) # doctest: +SKIP - ... def normalize(pdf): - ... v = pdf.v - ... return pdf.assign(v=(v - v.mean()) / v.std()) - >>> df.groupby("id").apply(normalize).show() # doctest: +SKIP - +---+-------------------+ - | id| v| - +---+-------------------+ - | 1|-0.7071067811865475| - | 1| 0.7071067811865475| - | 2|-0.8320502943378437| - | 2|-0.2773500981126146| - | 2| 1.1094003924504583| - +---+-------------------+ - - Alternatively, the user can define a function that takes two arguments. - In this case, the grouping key(s) will be passed as the first argument and the data will - be passed as the second argument. The grouping key(s) will be passed as a tuple of numpy - data types, e.g., `numpy.int32` and `numpy.float64`. The data will still be passed in - as a `pandas.DataFrame` containing all columns from the original Spark DataFrame. - This is useful when the user does not want to hardcode grouping key(s) in the function. - - >>> import pandas as pd # doctest: +SKIP - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType - >>> df = spark.createDataFrame( - ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], - ... ("id", "v")) # doctest: +SKIP - >>> @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) # doctest: +SKIP - ... def mean_udf(key, pdf): - ... # key is a tuple of one numpy.int64, which is the value - ... # of 'id' for the current group - ... return pd.DataFrame([key + (pdf.v.mean(),)]) - >>> df.groupby('id').apply(mean_udf).show() # doctest: +SKIP - +---+---+ - | id| v| - +---+---+ - | 1|1.5| - | 2|6.0| - +---+---+ - >>> @pandas_udf( - ... "id long, `ceil(v / 2)` long, v double", - ... PandasUDFType.GROUPED_MAP) # doctest: +SKIP - >>> def sum_udf(key, pdf): - ... # key is a tuple of two numpy.int64s, which is the values - ... # of 'id' and 'ceil(df.v / 2)' for the current group - ... return pd.DataFrame([key + (pdf.v.sum(),)]) - >>> df.groupby(df.id, ceil(df.v / 2)).apply(sum_udf).show() # doctest: +SKIP - +---+-----------+----+ - | id|ceil(v / 2)| v| - +---+-----------+----+ - | 2| 5|10.0| - | 1| 1| 3.0| - | 2| 3| 5.0| - | 2| 2| 3.0| - +---+-----------+----+ - - .. note:: If returning a new `pandas.DataFrame` constructed with a dictionary, it is - recommended to explicitly index the columns by name to ensure the positions are correct, - or alternatively use an `OrderedDict`. - For example, `pd.DataFrame({'id': ids, 'a': data}, columns=['id', 'a'])` or - `pd.DataFrame(OrderedDict([('id', ids), ('a', data)]))`. - - .. seealso:: :meth:`pyspark.sql.GroupedData.apply` - - 4. GROUPED_AGG - - A grouped aggregate UDF defines a transformation: One or more `pandas.Series` -> A scalar - The `returnType` should be a primitive data type, e.g., :class:`DoubleType`. - The returned scalar can be either a python primitive type, e.g., `int` or `float` - or a numpy data type, e.g., `numpy.int64` or `numpy.float64`. - - :class:`MapType` and :class:`StructType` are currently not supported as output types. - - Group aggregate UDFs are used with :meth:`pyspark.sql.GroupedData.agg` and - :class:`pyspark.sql.Window` - - This example shows using grouped aggregated UDFs with groupby: - - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType - >>> df = spark.createDataFrame( - ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], - ... ("id", "v")) - >>> @pandas_udf("double", PandasUDFType.GROUPED_AGG) # doctest: +SKIP - ... def mean_udf(v): - ... return v.mean() - >>> df.groupby("id").agg(mean_udf(df['v'])).show() # doctest: +SKIP - +---+-----------+ - | id|mean_udf(v)| - +---+-----------+ - | 1| 1.5| - | 2| 6.0| - +---+-----------+ - - This example shows using grouped aggregated UDFs as window functions. - - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType - >>> from pyspark.sql import Window - >>> df = spark.createDataFrame( - ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], - ... ("id", "v")) - >>> @pandas_udf("double", PandasUDFType.GROUPED_AGG) # doctest: +SKIP - ... def mean_udf(v): - ... return v.mean() - >>> w = (Window.partitionBy('id') - ... .orderBy('v') - ... .rowsBetween(-1, 0)) - >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show() # doctest: +SKIP - +---+----+------+ - | id| v|mean_v| - +---+----+------+ - | 1| 1.0| 1.0| - | 1| 2.0| 1.5| - | 2| 3.0| 3.0| - | 2| 5.0| 4.0| - | 2|10.0| 7.5| - +---+----+------+ - - .. note:: For performance reasons, the input series to window functions are not copied. + Default: SCALAR. + + .. note:: This parameter exists for compatibility. Using Python type hints is encouraged. + + In order to use this API, customarily the below are imported: + + >>> import pandas as pd + >>> from pyspark.sql.functions import pandas_udf + + From Spark 3.0 with Python 3.6+, `Python type hints `_ + detect the function types as below: + + >>> @pandas_udf(IntegerType()) + ... def slen(s: pd.Series) -> pd.Series: + ... return s.str.len() + + Prior to Spark 3.0, the pandas UDF used `functionType` to decide the execution type as below: + + >>> from pyspark.sql.functions import PandasUDFType + >>> from pyspark.sql.types import IntegerType + >>> @pandas_udf(IntegerType(), PandasUDFType.SCALAR) + ... def slen(s): + ... return s.str.len() + + It is preferred to specify type hints for the pandas UDF instead of specifying pandas UDF + type via `functionType` which will be deprecated in the future releases. + + Note that the type hint should use `pandas.Series` in all cases but there is one variant + that `pandas.DataFrame` should be used for its input or output type hint instead when the input + or output column is of :class:`pyspark.sql.types.StructType`. The following example shows + a Pandas UDF which takes long column, string column and struct column, and outputs a struct + column. It requires the function to specify the type hints of `pandas.Series` and + `pandas.DataFrame` as below: + + >>> @pandas_udf("col1 string, col2 long") + >>> def func(s1: pd.Series, s2: pd.Series, s3: pd.DataFrame) -> pd.DataFrame: + ... s3['col2'] = s1 + s2.str.len() + ... return s3 + ... + >>> # Create a Spark DataFrame that has three columns including a sturct column. + ... df = spark.createDataFrame( + ... [[1, "a string", ("a nested string",)]], + ... "long_col long, string_col string, struct_col struct") + >>> df.printSchema() + root + |-- long_column: long (nullable = true) + |-- string_column: string (nullable = true) + |-- struct_column: struct (nullable = true) + | |-- col1: string (nullable = true) + >>> df.select(func("long_col", "string_col", "struct_col")).printSchema() + |-- func(long_col, string_col, struct_col): struct (nullable = true) + | |-- col1: string (nullable = true) + | |-- col2: long (nullable = true) + + In the following sections, it describes the cominations of the supported type hints. For + simplicity, `pandas.DataFrame` variant is omitted. + + * Series to Series + `pandas.Series`, ... -> `pandas.Series` + + The function takes one or more `pandas.Series` and outputs one `pandas.Series`. + The output of the function should always be of the same length as the input. + + >>> @pandas_udf("string") + ... def to_upper(s: pd.Series) -> pd.Series: + ... return s.str.upper() + ... + >>> df = spark.createDataFrame([("John Doe",)], ("name",)) + >>> df.select(to_upper("name")).show() + +--------------+ + |to_upper(name)| + +--------------+ + | JOHN DOE| + +--------------+ + + >>> @pandas_udf("first string, last string") + ... def split_expand(s: pd.Series) -> pd.DataFrame: + ... return s.str.split(expand=True) + ... + >>> df = spark.createDataFrame([("John Doe",)], ("name",)) + >>> df.select(split_expand("name")).show() + +------------------+ + |split_expand(name)| + +------------------+ + | [John, Doe]| + +------------------+ + + .. note:: The length of the input is not that of the whole input column, but is the + length of an internal batch used for each call to the function. + + * Iterator of Series to Iterator of Series + `Iterator[pandas.Series]` -> `Iterator[pandas.Series]` + + The function takes an iterator of `pandas.Series` and outputs an iterator of + `pandas.Series`. In this case, the created pandas UDF instance requires one input + column when this is called as a PySpark column. The output of each series from + the function should always be of the same length as the input. + + It is useful when the UDF execution + requires initializing some states although internally it works identically as + Series to Series case. The pseudocode below illustrates the example. + + .. highlight:: python + .. code-block:: python + + @pandas_udf("long") + def calculate(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]: + # Do some expensive initialization with a state + state = very_expensive_initialization() + for x in iterator: + # Use that state for whole iterator. + yield calculate_with_state(x, state) + + df.select(calculate("value")).show() + + >>> from typing import Iterator + >>> @pandas_udf("long") + ... def plus_one(iterator: Iterator[pd.Series]) -> Iterator[pd.Series]: + ... for s in iterator: + ... yield s + 1 + ... + >>> df = spark.createDataFrame(pd.DataFrame([1, 2, 3], columns=["v"])) + >>> df.select(plus_one(df.v)).show() + +-----------+ + |plus_one(v)| + +-----------+ + | 2| + | 3| + | 4| + +-----------+ + + .. note:: The length of each series is the length of a batch internally used. + + * Iterator of Multiple Series to Iterator of Series + `Iterator[Tuple[pandas.Series, ...]]` -> `Iterator[pandas.Series]` + + The function takes an iterator of a tuple of multiple `pandas.Series` and outputs an + iterator of `pandas.Series`. In this case, the created pandas UDF instance requires + input columns as many as the series when this is called as a PySpark column. + It works identically as Iterator of Series to Iterator of Series case except + the parameter difference. The output of each series from the function should always + be of the same length as the input. + + >>> from typing import Iterator, Tuple + >>> from pyspark.sql.functions import struct, col + >>> @pandas_udf("long") + ... def multiply(iterator: Iterator[Tuple[pd.Series, pd.DataFrame]]) -> Iterator[pd.Series]: + ... for s1, df in iterator: + ... yield s1 * df.v + ... + >>> df = spark.createDataFrame(pd.DataFrame([1, 2, 3], columns=["v"])) + >>> df.withColumn('output', multiply(col("v"), struct(col("v")))).show() + +---+------+ + | v|output| + +---+------+ + | 1| 1| + | 2| 4| + | 3| 9| + +---+------+ + + .. note:: The length of each series is the length of a batch internally used. + + * Series to Scalar + `pandas.Series`, ... -> `Any` + + The function takes `pandas.Series` and returns a scalar value. The `returnType` + should be a primitive data type, and the returned scalar can be either a python primitive + type, e.g., int or float or a numpy data type, e.g., numpy.int64 or numpy.float64. + `Any` should ideally be a specific scalar type accordingly. + + >>> @pandas_udf("double") + ... def mean_udf(v: pd.Series) -> float: + ... return v.mean() + ... + >>> df = spark.createDataFrame( + ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v")) + >>> df.groupby("id").agg(mean_udf(df['v'])).show() + +---+-----------+ + | id|mean_udf(v)| + +---+-----------+ + | 1| 1.5| + | 2| 6.0| + +---+-----------+ + + This UDF can also be used as window functions as below: + + >>> from pyspark.sql import Window + >>> @pandas_udf("double") + ... def mean_udf(v: pd.Series) -> float: + ... return v.mean() + ... + >>> df = spark.createDataFrame( + ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ("id", "v")) + >>> w = Window.partitionBy('id').orderBy('v').rowsBetween(-1, 0) + >>> df.withColumn('mean_v', mean_udf("v").over(w)).show() + +---+----+------+ + | id| v|mean_v| + +---+----+------+ + | 1| 1.0| 1.0| + | 1| 2.0| 1.5| + | 2| 3.0| 3.0| + | 2| 5.0| 4.0| + | 2|10.0| 7.5| + +---+----+------+ + + .. note:: For performance reasons, the input series to window functions are not copied. Therefore, mutating the input series is not allowed and will cause incorrect results. For the same reason, users should also not rely on the index of the input series. - .. seealso:: :meth:`pyspark.sql.GroupedData.agg` and :class:`pyspark.sql.Window` + .. seealso:: :meth:`pyspark.sql.GroupedData.agg` and :class:`pyspark.sql.Window` .. note:: The user-defined functions do not support conditional expressions or short circuiting in boolean expressions and it ends up with being executed all internally. If the functions @@ -348,10 +273,21 @@ def pandas_udf(f=None, returnType=None, functionType=None): .. note:: The user-defined functions do not take keyword arguments on the calling side. .. note:: The data type of returned `pandas.Series` from the user-defined functions should be - matched with defined returnType (see :meth:`types.to_arrow_type` and + matched with defined `returnType` (see :meth:`types.to_arrow_type` and :meth:`types.from_arrow_type`). When there is mismatch between them, Spark might do conversion on returned data. The conversion is not guaranteed to be correct and results should be checked for accuracy by users. + + .. note:: Currently, + :class:`pyspark.sql.types.MapType`, + :class:`pyspark.sql.types.ArrayType` of :class:`pyspark.sql.types.TimestampType` and + nested :class:`pyspark.sql.types.StructType` + are currently not supported as output types. + + .. seealso:: :meth:`pyspark.sql.DataFrame.mapInPandas` + .. seealso:: :meth:`pyspark.sql.GroupedData.applyInPandas` + .. seealso:: :meth:`pyspark.sql.PandasCogroupedOps.applyInPandas` + .. seealso:: :meth:`pyspark.sql.UDFRegistration.register` """ # The following table shows most of Pandas data and SQL type conversions in Pandas UDFs that @@ -480,25 +416,3 @@ def _create_pandas_udf(f, returnType, evalType): "or three arguments (key, left, right).") return _create_udf(f, returnType, evalType) - - -def _test(): - import doctest - from pyspark.sql import SparkSession - import pyspark.sql.pandas.functions - globs = pyspark.sql.pandas.functions.__dict__.copy() - spark = SparkSession.builder\ - .master("local[4]")\ - .appName("sql.pandas.functions tests")\ - .getOrCreate() - globs['spark'] = spark - (failure_count, test_count) = doctest.testmod( - pyspark.sql.pandas.functions, globs=globs, - optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) - spark.stop() - if failure_count: - sys.exit(-1) - - -if __name__ == "__main__": - _test() diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index 3152271ba9df8..b93f0516cadb1 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -88,29 +88,27 @@ def applyInPandas(self, func, schema): to the user-function and the returned `pandas.DataFrame` are combined as a :class:`DataFrame`. - The returned `pandas.DataFrame` can be of arbitrary length and its schema must match the - returnType of the pandas udf. - - .. note:: This function requires a full shuffle. All the data of a group will be loaded - into memory, so the user should be aware of the potential OOM risk if data is skewed - and certain groups are too large to fit in memory. + The `schema` should be a :class:`StructType` describing the schema of the returned + `pandas.DataFrame`. The column labels of the returned `pandas.DataFrame` must either match + the field names in the defined schema if specified as strings, or match the + field data types by position if not strings, e.g. integer indices. + The length of the returned `pandas.DataFrame` can be arbitrary. :param func: a Python native function that takes a `pandas.DataFrame`, and outputs a `pandas.DataFrame`. :param schema: the return type of the `func` in PySpark. The value can be either a :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. - .. note:: Experimental - - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType + >>> import pandas as pd # doctest: +SKIP + >>> from pyspark.sql.functions import pandas_udf, ceil >>> df = spark.createDataFrame( ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], - ... ("id", "v")) + ... ("id", "v")) # doctest: +SKIP >>> def normalize(pdf): ... v = pdf.v ... return pdf.assign(v=(v - v.mean()) / v.std()) - >>> df.groupby("id").applyInPandas(normalize, schema="id long, v double").show() - ... # doctest: +SKIP + >>> df.groupby("id").applyInPandas( + ... normalize, schema="id long, v double").show() # doctest: +SKIP +---+-------------------+ | id| v| +---+-------------------+ @@ -121,8 +119,56 @@ def applyInPandas(self, func, schema): | 2| 1.1094003924504583| +---+-------------------+ - .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` + Alternatively, the user can pass a function that takes two arguments. + In this case, the grouping key(s) will be passed as the first argument and the data will + be passed as the second argument. The grouping key(s) will be passed as a tuple of numpy + data types, e.g., `numpy.int32` and `numpy.float64`. The data will still be passed in + as a `pandas.DataFrame` containing all columns from the original Spark DataFrame. + This is useful when the user does not want to hardcode grouping key(s) in the function. + + >>> df = spark.createDataFrame( + ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], + ... ("id", "v")) # doctest: +SKIP + >>> def mean_func(key, pdf): + ... # key is a tuple of one numpy.int64, which is the value + ... # of 'id' for the current group + ... return pd.DataFrame([key + (pdf.v.mean(),)]) + >>> df.groupby('id').applyInPandas( + ... mean_func, schema="id long, v double").show() # doctest: +SKIP + +---+---+ + | id| v| + +---+---+ + | 1|1.5| + | 2|6.0| + +---+---+ + >>> def sum_func(key, pdf): + ... # key is a tuple of two numpy.int64s, which is the values + ... # of 'id' and 'ceil(df.v / 2)' for the current group + ... return pd.DataFrame([key + (pdf.v.sum(),)]) + >>> df.groupby(df.id, ceil(df.v / 2)).applyInPandas( + ... sum_func, schema="id long, `ceil(v / 2)` long, v double").show() # doctest: +SKIP + +---+-----------+----+ + | id|ceil(v / 2)| v| + +---+-----------+----+ + | 2| 5|10.0| + | 1| 1| 3.0| + | 2| 3| 5.0| + | 2| 2| 3.0| + +---+-----------+----+ + + .. note:: This function requires a full shuffle. All the data of a group will be loaded + into memory, so the user should be aware of the potential OOM risk if data is skewed + and certain groups are too large to fit in memory. + + .. note:: If returning a new `pandas.DataFrame` constructed with a dictionary, it is + recommended to explicitly index the columns by name to ensure the positions are correct, + or alternatively use an `OrderedDict`. + For example, `pd.DataFrame({'id': ids, 'a': data}, columns=['id', 'a'])` or + `pd.DataFrame(OrderedDict([('id', ids), ('a', data)]))`. + .. note:: Experimental + + .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` """ from pyspark.sql import GroupedData from pyspark.sql.functions import pandas_udf, PandasUDFType @@ -176,14 +222,11 @@ def applyInPandas(self, func, schema): `pandas.DataFrame` to the user-function and the returned `pandas.DataFrame` are combined as a :class:`DataFrame`. - The returned `pandas.DataFrame` can be of arbitrary length and its schema must match the - returnType of the pandas udf. - - .. note:: This function requires a full shuffle. All the data of a cogroup will be loaded - into memory, so the user should be aware of the potential OOM risk if data is skewed - and certain groups are too large to fit in memory. - - .. note:: Experimental + The `schema` should be a :class:`StructType` describing the schema of the returned + `pandas.DataFrame`. The column labels of the returned `pandas.DataFrame` must either match + the field names in the defined schema if specified as strings, or match the + field data types by position if not strings, e.g. integer indices. + The length of the returned `pandas.DataFrame` can be arbitrary. :param func: a Python native function that takes two `pandas.DataFrame`\\s, and outputs a `pandas.DataFrame`, or that takes one tuple (grouping keys) and two @@ -191,7 +234,7 @@ def applyInPandas(self, func, schema): :param schema: the return type of the `func` in PySpark. The value can be either a :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType + >>> from pyspark.sql.functions import pandas_udf >>> df1 = spark.createDataFrame( ... [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)], ... ("time", "id", "v1")) @@ -232,6 +275,18 @@ def applyInPandas(self, func, schema): |20000102| 1|3.0| x| +--------+---+---+---+ + .. note:: This function requires a full shuffle. All the data of a cogroup will be loaded + into memory, so the user should be aware of the potential OOM risk if data is skewed + and certain groups are too large to fit in memory. + + .. note:: If returning a new `pandas.DataFrame` constructed with a dictionary, it is + recommended to explicitly index the columns by name to ensure the positions are correct, + or alternatively use an `OrderedDict`. + For example, `pd.DataFrame({'id': ids, 'a': data}, columns=['id', 'a'])` or + `pd.DataFrame(OrderedDict([('id', ids), ('a', data)]))`. + + .. note:: Experimental + .. seealso:: :meth:`pyspark.sql.functions.pandas_udf` """ diff --git a/python/pyspark/sql/pandas/map_ops.py b/python/pyspark/sql/pandas/map_ops.py index 75cacd797f9dd..9835e88c6ac21 100644 --- a/python/pyspark/sql/pandas/map_ops.py +++ b/python/pyspark/sql/pandas/map_ops.py @@ -45,10 +45,10 @@ def mapInPandas(self, func, schema): :param schema: the return type of the `func` in PySpark. The value can be either a :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string. - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType + >>> from pyspark.sql.functions import pandas_udf >>> df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age")) - >>> def filter_func(batch_iter): - ... for pdf in batch_iter: + >>> def filter_func(iterator): + ... for pdf in iterator: ... yield pdf[pdf.id == 1] >>> df.mapInPandas(filter_func, df.schema).show() # doctest: +SKIP +---+---+ diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 433c5fc845c59..10546ecacc57f 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -297,17 +297,18 @@ def register(self, name, f, returnType=None): >>> spark.sql("SELECT random_udf()").collect() # doctest: +SKIP [Row(random_udf()=82)] - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType - >>> @pandas_udf("integer", PandasUDFType.SCALAR) # doctest: +SKIP - ... def add_one(x): - ... return x + 1 + >>> import pandas as pd # doctest: +SKIP + >>> from pyspark.sql.functions import pandas_udf + >>> @pandas_udf("integer") # doctest: +SKIP + ... def add_one(s: pd.Series) -> pd.Series: + ... return s + 1 ... >>> _ = spark.udf.register("add_one", add_one) # doctest: +SKIP >>> spark.sql("SELECT add_one(id) FROM range(3)").collect() # doctest: +SKIP [Row(add_one(id)=1), Row(add_one(id)=2), Row(add_one(id)=3)] - >>> @pandas_udf("integer", PandasUDFType.GROUPED_AGG) # doctest: +SKIP - ... def sum_udf(v): + >>> @pandas_udf("integer") # doctest: +SKIP + ... def sum_udf(v: pd.Series) -> int: ... return v.sum() ... >>> _ = spark.udf.register("sum_udf", sum_udf) # doctest: +SKIP @@ -414,6 +415,9 @@ def _test(): .appName("sql.udf tests")\ .getOrCreate() globs['spark'] = spark + # Hack to skip the unit tests in register. These are currently being tested in proper tests. + # We should reenable this test once we completely drop Python 2. + del pyspark.sql.udf.UDFRegistration.register (failure_count, test_count) = doctest.testmod( pyspark.sql.udf, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE) From b4769998efee0f5998104b689b710c11ee0dbd14 Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Wed, 12 Feb 2020 15:19:16 +0900 Subject: [PATCH 1352/1387] [SPARK-30795][SQL] Spark SQL codegen's code() interpolator should treat escapes like Scala's StringContext.s() ### What changes were proposed in this pull request? This PR proposes to make the `code` string interpolator treat escapes the same way as Scala's builtin `StringContext.s()` string interpolator. This will remove the need for an ugly workaround in `Like` expression's codegen. ### Why are the changes needed? The `code()` string interpolator in Spark SQL's code generator should treat escapes like Scala's builtin `StringContext.s()` interpolator, i.e. it should treat escapes in the code parts, and should not treat escapes in the input arguments. For example, ```scala val arg = "This is an argument." val str = s"This is string part 1. $arg This is string part 2." val code = code"This is string part 1. $arg This is string part 2." assert(code.toString == str) ``` We should expect the `code()` interpolator to produce the same result as the `StringContext.s()` interpolator, where only escapes in the string parts should be treated, while the args should be kept verbatim. But in the current implementation, due to the eager folding of code parts and literal input args, the escape treatment is incorrectly done on both code parts and literal args. That causes a problem when an arg contains escape sequences and wants to preserve that in the final produced code string. For example, in `Like` expression's codegen, there's an ugly workaround for this bug: ```scala // We need double escape to avoid org.codehaus.commons.compiler.CompileException. // '\\' will cause exception 'Single quote must be backslash-escaped in character literal'. // '\"' will cause exception 'Line break in literal not allowed'. val newEscapeChar = if (escapeChar == '\"' || escapeChar == '\\') { s"""\\\\\\$escapeChar""" } else { escapeChar } ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added a new unit test case in `CodeBlockSuite`. Closes #27544 from rednaxelafx/fix-code-string-interpolator. Authored-by: Kris Mok Signed-off-by: HyukjinKwon --- .../sql/catalyst/expressions/codegen/javaCode.scala | 13 +++++++++---- .../catalyst/expressions/regexpExpressions.scala | 13 ++++--------- .../expressions/codegen/CodeBlockSuite.scala | 12 ++++++++++++ 3 files changed, 25 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala index d9393b9df6bbd..dff258902a0b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala @@ -223,6 +223,11 @@ object Block { implicit def blocksToBlock(blocks: Seq[Block]): Block = blocks.reduceLeft(_ + _) implicit class BlockHelper(val sc: StringContext) extends AnyVal { + /** + * A string interpolator that retains references to the `JavaCode` inputs, and behaves like + * the Scala builtin StringContext.s() interpolator otherwise, i.e. it will treat escapes in + * the code parts, and will not treat escapes in the input arguments. + */ def code(args: Any*): Block = { sc.checkLengths(args) if (sc.parts.length == 0) { @@ -250,7 +255,7 @@ object Block { val inputs = args.iterator val buf = new StringBuilder(Block.CODE_BLOCK_BUFFER_LENGTH) - buf.append(strings.next) + buf.append(StringContext.treatEscapes(strings.next)) while (strings.hasNext) { val input = inputs.next input match { @@ -262,7 +267,7 @@ object Block { case _ => buf.append(input) } - buf.append(strings.next) + buf.append(StringContext.treatEscapes(strings.next)) } codeParts += buf.toString @@ -286,10 +291,10 @@ case class CodeBlock(codeParts: Seq[String], blockInputs: Seq[JavaCode]) extends val strings = codeParts.iterator val inputs = blockInputs.iterator val buf = new StringBuilder(Block.CODE_BLOCK_BUFFER_LENGTH) - buf.append(StringContext.treatEscapes(strings.next)) + buf.append(strings.next) while (strings.hasNext) { buf.append(inputs.next) - buf.append(StringContext.treatEscapes(strings.next)) + buf.append(strings.next) } buf.toString } 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 e5ee0edfcf79b..6a4d813d345b3 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 @@ -159,19 +159,14 @@ case class Like(left: Expression, right: Expression, escapeChar: Char) } else { val pattern = ctx.freshName("pattern") val rightStr = ctx.freshName("rightStr") - // We need double escape to avoid org.codehaus.commons.compiler.CompileException. - // '\\' will cause exception 'Single quote must be backslash-escaped in character literal'. - // '\"' will cause exception 'Line break in literal not allowed'. - val newEscapeChar = if (escapeChar == '\"' || escapeChar == '\\') { - s"""\\\\\\$escapeChar""" - } else { - escapeChar - } + // We need to escape the escapeChar to make sure the generated code is valid. + // Otherwise we'll hit org.codehaus.commons.compiler.CompileException. + val escapedEscapeChar = StringEscapeUtils.escapeJava(escapeChar.toString) nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" String $rightStr = $eval2.toString(); $patternClass $pattern = $patternClass.compile( - $escapeFunc($rightStr, '$newEscapeChar')); + $escapeFunc($rightStr, '$escapedEscapeChar')); ${ev.value} = $pattern.matcher($eval1.toString()).matches(); """ }) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeBlockSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeBlockSuite.scala index 55569b6f2933e..67e3bc69543e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeBlockSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeBlockSuite.scala @@ -37,6 +37,18 @@ class CodeBlockSuite extends SparkFunSuite { assert(code.asInstanceOf[CodeBlock].blockInputs === Seq(value)) } + test("Code parts should be treated for escapes, but string inputs shouldn't be") { + val strlit = raw"\\" + val code = code"""String s = "foo\\bar" + "$strlit";""" + + val builtin = s"""String s = "foo\\bar" + "$strlit";""" + + val expected = raw"""String s = "foo\bar" + "\\";""" + + assert(builtin == expected) + assert(code.asInstanceOf[CodeBlock].toString == expected) + } + test("Block.stripMargin") { val isNull = JavaCode.isNullVariable("expr1_isNull") val value = JavaCode.variable("expr1", IntegerType) From f5026b1ba7c05548d5f271d6d3edf7dfd4c3f9ce Mon Sep 17 00:00:00 2001 From: beliefer Date: Wed, 12 Feb 2020 14:49:22 +0800 Subject: [PATCH 1353/1387] [SPARK-30763][SQL] Fix java.lang.IndexOutOfBoundsException No group 1 for regexp_extract ### What changes were proposed in this pull request? The current implement of `regexp_extract` will throws a unprocessed exception show below: `SELECT regexp_extract('1a 2b 14m', 'd+')` ``` java.lang.IndexOutOfBoundsException: No group 1 [info] at java.util.regex.Matcher.group(Matcher.java:538) [info] at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source) [info] at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) [info] at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:729) ``` I think should treat this exception well. ### Why are the changes needed? Fix a bug `java.lang.IndexOutOfBoundsException No group 1 ` ### Does this PR introduce any user-facing change? Yes ### How was this patch tested? New UT Closes #27508 from beliefer/fix-regexp_extract-bug. Authored-by: beliefer Signed-off-by: Wenchen Fan --- .../expressions/regexpExpressions.scala | 15 +++- .../expressions/RegexpExpressionsSuite.scala | 12 ++++ .../sql-tests/inputs/regexp-functions.sql | 9 +++ .../results/regexp-functions.sql.out | 69 +++++++++++++++++++ 4 files changed, 104 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out 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 6a4d813d345b3..3f60ca388a807 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 @@ -410,6 +410,15 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio } } +object RegExpExtract { + def checkGroupIndex(groupCount: Int, groupIndex: Int): Unit = { + if (groupCount < groupIndex) { + throw new IllegalArgumentException( + s"Regex group count is $groupCount, but the specified group index is $groupIndex") + } + } +} + /** * Extract a specific(idx) group identified by a Java regex. * @@ -441,7 +450,9 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio val m = pattern.matcher(s.toString) if (m.find) { val mr: MatchResult = m.toMatchResult - val group = mr.group(r.asInstanceOf[Int]) + val index = r.asInstanceOf[Int] + RegExpExtract.checkGroupIndex(mr.groupCount, index) + val group = mr.group(index) if (group == null) { // Pattern matched, but not optional group UTF8String.EMPTY_UTF8 } else { @@ -459,6 +470,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val classNamePattern = classOf[Pattern].getCanonicalName + val classNameRegExpExtract = classOf[RegExpExtract].getCanonicalName val matcher = ctx.freshName("matcher") val matchResult = ctx.freshName("matchResult") @@ -482,6 +494,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio $termPattern.matcher($subject.toString()); if ($matcher.find()) { java.util.regex.MatchResult $matchResult = $matcher.toMatchResult(); + $classNameRegExpExtract.checkGroupIndex($matchResult.groupCount(), $idx); if ($matchResult.group($idx) == null) { ${ev.value} = UTF8String.EMPTY_UTF8; } else { 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 2c8794f083dbb..86da62bc74940 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 @@ -293,6 +293,18 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val nonNullExpr = RegExpExtract(Literal("100-200"), Literal("(\\d+)-(\\d+)"), Literal(1)) checkEvaluation(nonNullExpr, "100", row1) + + // invalid group index + val row8 = create_row("100-200", "(\\d+)-(\\d+)", 3) + val row9 = create_row("100-200", "(\\d+).*", 2) + val row10 = create_row("100-200", "\\d+", 1) + + checkExceptionInExpression[IllegalArgumentException]( + expr, row8, "Regex group count is 2, but the specified group index is 3") + checkExceptionInExpression[IllegalArgumentException]( + expr, row9, "Regex group count is 1, but the specified group index is 2") + checkExceptionInExpression[IllegalArgumentException]( + expr, row10, "Regex group count is 0, but the specified group index is 1") } test("SPLIT") { diff --git a/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql new file mode 100644 index 0000000000000..c0827a3cba39b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql @@ -0,0 +1,9 @@ +-- regexp_extract +SELECT regexp_extract('1a 2b 14m', '\\d+'); +SELECT regexp_extract('1a 2b 14m', '\\d+', 0); +SELECT regexp_extract('1a 2b 14m', '\\d+', 1); +SELECT regexp_extract('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)'); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2); diff --git a/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out new file mode 100644 index 0000000000000..c92c1ddca774f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/regexp-functions.sql.out @@ -0,0 +1,69 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 0) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 1) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 2) +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Regex group count is 0, but the specified group index is 2 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query schema +struct +-- !query output +1a + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query schema +struct +-- !query output +a From 8b1839728acaa5e61f542a7332505289726d3162 Mon Sep 17 00:00:00 2001 From: turbofei Date: Wed, 12 Feb 2020 20:21:52 +0900 Subject: [PATCH 1354/1387] [SPARK-29542][FOLLOW-UP] Keep the description of spark.sql.files.* in tuning guide be consistent with that in SQLConf ### What changes were proposed in this pull request? This pr is a follow up of https://github.com/apache/spark/pull/26200. In this PR, I modify the description of spark.sql.files.* in sql-performance-tuning.md to keep consistent with that in SQLConf. ### Why are the changes needed? To keep consistent with the description in SQLConf. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existed UT. Closes #27545 from turboFei/SPARK-29542-follow-up. Authored-by: turbofei Signed-off-by: HyukjinKwon --- docs/sql-performance-tuning.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/sql-performance-tuning.md b/docs/sql-performance-tuning.md index e289854c7acc7..5a86c0cc31e12 100644 --- a/docs/sql-performance-tuning.md +++ b/docs/sql-performance-tuning.md @@ -67,6 +67,7 @@ that these options will be deprecated in future release as more optimizations ar 134217728 (128 MB) The maximum number of bytes to pack into a single partition when reading files. + This configuration is effective only when using file-based sources such as Parquet, JSON and ORC. @@ -76,7 +77,8 @@ that these options will be deprecated in future release as more optimizations ar The estimated cost to open a file, measured by the number of bytes could be scanned in the same time. This is used when putting multiple files into a partition. It is better to over-estimated, then the partitions with small files will be faster than partitions with bigger files (which is - scheduled first). + scheduled first). This configuration is effective only when using file-based sources such as Parquet, + JSON and ORC. From c1986204e59f1e8cc4b611d5a578cb248cb74c28 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 12 Feb 2020 20:12:38 +0800 Subject: [PATCH 1355/1387] [SPARK-30788][SQL] Support `SimpleDateFormat` and `FastDateFormat` as legacy date/timestamp formatters ### What changes were proposed in this pull request? In the PR, I propose to add legacy date/timestamp formatters based on `SimpleDateFormat` and `FastDateFormat`: - `LegacyFastTimestampFormatter` - uses `FastDateFormat` and supports parsing/formatting in microsecond precision. The code was borrowed from Spark 2.4, see https://github.com/apache/spark/pull/26507 & https://github.com/apache/spark/pull/26582 - `LegacySimpleTimestampFormatter` uses `SimpleDateFormat`, and support the `lenient` mode. When the `lenient` parameter is set to `false`, the parser become much stronger in checking its input. ### Why are the changes needed? Spark 2.4.x uses the following parsers for parsing/formatting date/timestamp strings: - `DateTimeFormat` in CSV/JSON datasource - `SimpleDateFormat` - is used in JDBC datasource, in partitions parsing. - `SimpleDateFormat` in strong mode (`lenient = false`), see https://github.com/apache/spark/blob/branch-2.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala#L124. It is used by the `date_format`, `from_unixtime`, `unix_timestamp` and `to_unix_timestamp` functions. The PR aims to make Spark 3.0 compatible with Spark 2.4.x in all those cases when `spark.sql.legacy.timeParser.enabled` is set to `true`. ### Does this PR introduce any user-facing change? This shouldn't change behavior with default settings. If `spark.sql.legacy.timeParser.enabled` is set to `true`, users should observe behavior of Spark 2.4. ### How was this patch tested? - Modified tests in `DateExpressionsSuite` to check the legacy parser - `SimpleDateFormat`. - Added `CSVLegacyTimeParserSuite` and `JsonLegacyTimeParserSuite` to run `CSVSuite` and `JsonSuite` with the legacy parser - `FastDateFormat`. Closes #27524 from MaxGekk/timestamp-formatter-legacy-fallback. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/csv/CSVInferSchema.scala | 4 +- .../spark/sql/catalyst/csv/CSVOptions.scala | 4 +- .../sql/catalyst/csv/UnivocityGenerator.scala | 7 +- .../sql/catalyst/csv/UnivocityParser.scala | 7 +- .../expressions/datetimeExpressions.scala | 52 ++- .../spark/sql/catalyst/json/JSONOptions.scala | 4 +- .../sql/catalyst/json/JacksonGenerator.scala | 7 +- .../sql/catalyst/json/JacksonParser.scala | 7 +- .../sql/catalyst/json/JsonInferSchema.scala | 4 +- .../sql/catalyst/util/DateFormatter.scala | 66 ++- .../catalyst/util/TimestampFormatter.scala | 132 +++++- .../org/apache/spark/sql/types/Decimal.scala | 2 +- .../expressions/DateExpressionsSuite.scala | 390 ++++++++++-------- .../org/apache/spark/sql/functions.scala | 7 +- .../resources/test-data/bad_after_good.csv | 2 +- .../resources/test-data/value-malformed.csv | 2 +- .../apache/spark/sql/DateFunctionsSuite.scala | 346 +++++++++------- .../execution/datasources/csv/CSVSuite.scala | 23 +- .../datasources/json/JsonSuite.scala | 7 + 19 files changed, 654 insertions(+), 419 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala index 03cc3cbdf790a..c6a03183ab45e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala @@ -24,6 +24,7 @@ import scala.util.control.Exception.allCatch import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.catalyst.util.TimestampFormatter import org.apache.spark.sql.types._ @@ -32,7 +33,8 @@ class CSVInferSchema(val options: CSVOptions) extends Serializable { private val timestampParser = TimestampFormatter( options.timestampFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) private val decimalParser = if (options.locale == Locale.US) { // Special handling the default locale for backward compatibility diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 5e40d74e54f11..8892037e03a7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -146,10 +146,10 @@ class CSVOptions( // A language tag in IETF BCP 47 format val locale: Locale = parameters.get("locale").map(Locale.forLanguageTag).getOrElse(Locale.US) - val dateFormat: String = parameters.getOrElse("dateFormat", "uuuu-MM-dd") + val dateFormat: String = parameters.getOrElse("dateFormat", DateFormatter.defaultPattern) val timestampFormat: String = - parameters.getOrElse("timestampFormat", "uuuu-MM-dd'T'HH:mm:ss.SSSXXX") + parameters.getOrElse("timestampFormat", s"${DateFormatter.defaultPattern}'T'HH:mm:ss.SSSXXX") val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala index 05cb91d10868e..00e3d49787db1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala @@ -23,6 +23,7 @@ import com.univocity.parsers.csv.CsvWriter import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.types._ class UnivocityGenerator( @@ -44,11 +45,13 @@ class UnivocityGenerator( private val timestampFormatter = TimestampFormatter( options.timestampFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) private val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) private def makeConverter(dataType: DataType): ValueConverter = dataType match { case DateType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 5510953804025..cd69c21a01976 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -27,6 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -86,11 +87,13 @@ class UnivocityParser( private val timestampFormatter = TimestampFormatter( options.timestampFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) private val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) private val csvFilters = new CSVFilters(filters, requiredSchema) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index aa2bd5a1273e0..1f4c8c041f8bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -30,9 +30,10 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, LegacyDateFormats, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.catalyst.util.LegacyDateFormats.SIMPLE_DATE_FORMAT import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -622,13 +623,15 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti @transient private lazy val formatter: Option[TimestampFormatter] = { if (right.foldable) { - Option(right.eval()).map(format => TimestampFormatter(format.toString, zoneId)) + Option(right.eval()).map { format => + TimestampFormatter(format.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + } } else None } override protected def nullSafeEval(timestamp: Any, format: Any): Any = { val tf = if (formatter.isEmpty) { - TimestampFormatter(format.toString, zoneId) + TimestampFormatter(format.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) } else { formatter.get } @@ -643,10 +646,14 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti }) }.getOrElse { val tf = TimestampFormatter.getClass.getName.stripSuffix("$") + val ldf = LegacyDateFormats.getClass.getName.stripSuffix("$") val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) defineCodeGen(ctx, ev, (timestamp, format) => { - s"""UTF8String.fromString($tf$$.MODULE$$.apply($format.toString(), $zid) - .format($timestamp))""" + s"""|UTF8String.fromString($tf$$.MODULE$$.apply( + | $format.toString(), + | $zid, + | $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT()) + |.format($timestamp))""".stripMargin }) } } @@ -688,7 +695,7 @@ case class ToUnixTimestamp( copy(timeZoneId = Option(timeZoneId)) def this(time: Expression) = { - this(time, Literal("uuuu-MM-dd HH:mm:ss")) + this(time, Literal(TimestampFormatter.defaultPattern)) } override def prettyName: String = "to_unix_timestamp" @@ -732,7 +739,7 @@ case class UnixTimestamp(timeExp: Expression, format: Expression, timeZoneId: Op copy(timeZoneId = Option(timeZoneId)) def this(time: Expression) = { - this(time, Literal("uuuu-MM-dd HH:mm:ss")) + this(time, Literal(TimestampFormatter.defaultPattern)) } def this() = { @@ -758,7 +765,7 @@ abstract class ToTimestamp private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] private lazy val formatter: TimestampFormatter = try { - TimestampFormatter(constFormat.toString, zoneId) + TimestampFormatter(constFormat.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) } catch { case NonFatal(_) => null } @@ -791,8 +798,8 @@ abstract class ToTimestamp } else { val formatString = f.asInstanceOf[UTF8String].toString try { - TimestampFormatter(formatString, zoneId).parse( - t.asInstanceOf[UTF8String].toString) / downScaleFactor + TimestampFormatter(formatString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + .parse(t.asInstanceOf[UTF8String].toString) / downScaleFactor } catch { case NonFatal(_) => null } @@ -831,13 +838,16 @@ abstract class ToTimestamp } case StringType => val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) - val locale = ctx.addReferenceObj("locale", Locale.US) val tf = TimestampFormatter.getClass.getName.stripSuffix("$") + val ldf = LegacyDateFormats.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (string, format) => { s""" try { - ${ev.value} = $tf$$.MODULE$$.apply($format.toString(), $zid, $locale) - .parse($string.toString()) / $downScaleFactor; + ${ev.value} = $tf$$.MODULE$$.apply( + $format.toString(), + $zid, + $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT()) + .parse($string.toString()) / $downScaleFactor; } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; } catch (java.text.ParseException e) { @@ -908,7 +918,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ override def prettyName: String = "from_unixtime" def this(unix: Expression) = { - this(unix, Literal("uuuu-MM-dd HH:mm:ss")) + this(unix, Literal(TimestampFormatter.defaultPattern)) } override def dataType: DataType = StringType @@ -922,7 +932,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] private lazy val formatter: TimestampFormatter = try { - TimestampFormatter(constFormat.toString, zoneId) + TimestampFormatter(constFormat.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) } catch { case NonFatal(_) => null } @@ -948,8 +958,9 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ null } else { try { - UTF8String.fromString(TimestampFormatter(f.toString, zoneId) - .format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) + UTF8String.fromString( + TimestampFormatter(f.toString, zoneId, legacyFormat = SIMPLE_DATE_FORMAT) + .format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { case NonFatal(_) => null } @@ -980,13 +991,14 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ } } else { val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) - val locale = ctx.addReferenceObj("locale", Locale.US) val tf = TimestampFormatter.getClass.getName.stripSuffix("$") + val ldf = LegacyDateFormats.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (seconds, f) => { s""" try { - ${ev.value} = UTF8String.fromString($tf$$.MODULE$$.apply($f.toString(), $zid, $locale). - format($seconds * 1000000L)); + ${ev.value} = UTF8String.fromString( + $tf$$.MODULE$$.apply($f.toString(), $zid, $ldf$$.MODULE$$.SIMPLE_DATE_FORMAT()) + .format($seconds * 1000000L)); } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; }""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index cdf4b4689e821..45c4edff47070 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -88,10 +88,10 @@ private[sql] class JSONOptions( val zoneId: ZoneId = DateTimeUtils.getZoneId( parameters.getOrElse(DateTimeUtils.TIMEZONE_OPTION, defaultTimeZoneId)) - val dateFormat: String = parameters.getOrElse("dateFormat", "uuuu-MM-dd") + val dateFormat: String = parameters.getOrElse("dateFormat", DateFormatter.defaultPattern) val timestampFormat: String = - parameters.getOrElse("timestampFormat", "uuuu-MM-dd'T'HH:mm:ss.SSSXXX") + parameters.getOrElse("timestampFormat", s"${DateFormatter.defaultPattern}'T'HH:mm:ss.SSSXXX") val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index 9c63593ea1752..141360ff02117 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -24,6 +24,7 @@ import com.fasterxml.jackson.core._ 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.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.types._ /** @@ -80,11 +81,13 @@ private[sql] class JacksonGenerator( private val timestampFormatter = TimestampFormatter( options.timestampFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) private val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) private def makeWriter(dataType: DataType): ValueWriter = dataType match { case NullType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 76efa574a99ff..1e408cdb126b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -58,11 +59,13 @@ class JacksonParser( private val timestampFormatter = TimestampFormatter( options.timestampFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) private val dateFormatter = DateFormatter( options.dateFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) /** * Create a converter which converts the JSON documents held by the `JsonParser` diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index f030955ee6e7f..82dd6d0da2632 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.JacksonUtils.nextUntil import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -40,7 +41,8 @@ private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { private val timestampFormatter = TimestampFormatter( options.timestampFormat, options.zoneId, - options.locale) + options.locale, + legacyFormat = FAST_DATE_FORMAT) /** * Infer the type of a collection of json records in three stages: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala index 28189b65dee9a..2cf82d1cfa177 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.catalyst.util +import java.text.SimpleDateFormat import java.time.{LocalDate, ZoneId} -import java.util.Locale +import java.util.{Date, Locale} import org.apache.commons.lang3.time.FastDateFormat @@ -51,41 +52,76 @@ class Iso8601DateFormatter( } } -class LegacyDateFormatter(pattern: String, locale: Locale) extends DateFormatter { - @transient - private lazy val format = FastDateFormat.getInstance(pattern, locale) +trait LegacyDateFormatter extends DateFormatter { + def parseToDate(s: String): Date + def formatDate(d: Date): String override def parse(s: String): Int = { - val milliseconds = format.parse(s).getTime + val milliseconds = parseToDate(s).getTime DateTimeUtils.millisToDays(milliseconds) } override def format(days: Int): String = { val date = DateTimeUtils.toJavaDate(days) - format.format(date) + formatDate(date) } } +class LegacyFastDateFormatter(pattern: String, locale: Locale) extends LegacyDateFormatter { + @transient + private lazy val fdf = FastDateFormat.getInstance(pattern, locale) + override def parseToDate(s: String): Date = fdf.parse(s) + override def formatDate(d: Date): String = fdf.format(d) +} + +class LegacySimpleDateFormatter(pattern: String, locale: Locale) extends LegacyDateFormatter { + @transient + private lazy val sdf = new SimpleDateFormat(pattern, locale) + override def parseToDate(s: String): Date = sdf.parse(s) + override def formatDate(d: Date): String = sdf.format(d) +} + object DateFormatter { + import LegacyDateFormats._ + val defaultLocale: Locale = Locale.US - def apply(format: String, zoneId: ZoneId, locale: Locale): DateFormatter = { + def defaultPattern(): String = { + if (SQLConf.get.legacyTimeParserEnabled) "yyyy-MM-dd" else "uuuu-MM-dd" + } + + private def getFormatter( + format: Option[String], + zoneId: ZoneId, + locale: Locale = defaultLocale, + legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT): DateFormatter = { + + val pattern = format.getOrElse(defaultPattern) if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyDateFormatter(format, locale) + legacyFormat match { + case FAST_DATE_FORMAT => + new LegacyFastDateFormatter(pattern, locale) + case SIMPLE_DATE_FORMAT | LENIENT_SIMPLE_DATE_FORMAT => + new LegacySimpleDateFormatter(pattern, locale) + } } else { - new Iso8601DateFormatter(format, zoneId, locale) + new Iso8601DateFormatter(pattern, zoneId, locale) } } + def apply( + format: String, + zoneId: ZoneId, + locale: Locale, + legacyFormat: LegacyDateFormat): DateFormatter = { + getFormatter(Some(format), zoneId, locale, legacyFormat) + } + def apply(format: String, zoneId: ZoneId): DateFormatter = { - apply(format, zoneId, defaultLocale) + getFormatter(Some(format), zoneId) } def apply(zoneId: ZoneId): DateFormatter = { - if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyDateFormatter("yyyy-MM-dd", defaultLocale) - } else { - new Iso8601DateFormatter("uuuu-MM-dd", zoneId, defaultLocale) - } + getFormatter(None, zoneId) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala index fe1a4fe710c20..4893a7ec91cbb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -17,19 +17,20 @@ package org.apache.spark.sql.catalyst.util -import java.text.ParseException +import java.text.{ParseException, ParsePosition, SimpleDateFormat} import java.time._ import java.time.format.DateTimeParseException import java.time.temporal.ChronoField.MICRO_OF_SECOND import java.time.temporal.TemporalQueries -import java.util.{Locale, TimeZone} +import java.util.{Calendar, GregorianCalendar, Locale, TimeZone} import java.util.concurrent.TimeUnit.SECONDS import org.apache.commons.lang3.time.FastDateFormat -import org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_MILLIS -import org.apache.spark.sql.catalyst.util.DateTimeUtils.convertSpecialTimestamp +import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{ convertSpecialTimestamp, SQLTimestamp} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.Decimal sealed trait TimestampFormatter extends Serializable { /** @@ -90,44 +91,139 @@ class FractionTimestampFormatter(zoneId: ZoneId) override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter } -class LegacyTimestampFormatter( +/** + * The custom sub-class of `GregorianCalendar` is needed to get access to + * protected `fields` immediately after parsing. We cannot use + * the `get()` method because it performs normalization of the fraction + * part. Accordingly, the `MILLISECOND` field doesn't contain original value. + * + * Also this class allows to set raw value to the `MILLISECOND` field + * directly before formatting. + */ +class MicrosCalendar(tz: TimeZone, digitsInFraction: Int) + extends GregorianCalendar(tz, Locale.US) { + // Converts parsed `MILLISECOND` field to seconds fraction in microsecond precision. + // For example if the fraction pattern is `SSSS` then `digitsInFraction` = 4, and + // if the `MILLISECOND` field was parsed to `1234`. + def getMicros(): SQLTimestamp = { + // Append 6 zeros to the field: 1234 -> 1234000000 + val d = fields(Calendar.MILLISECOND) * MICROS_PER_SECOND + // Take the first 6 digits from `d`: 1234000000 -> 123400 + // The rest contains exactly `digitsInFraction`: `0000` = 10 ^ digitsInFraction + // So, the result is `(1234 * 1000000) / (10 ^ digitsInFraction) + d / Decimal.POW_10(digitsInFraction) + } + + // Converts the seconds fraction in microsecond precision to a value + // that can be correctly formatted according to the specified fraction pattern. + // The method performs operations opposite to `getMicros()`. + def setMicros(micros: Long): Unit = { + val d = micros * Decimal.POW_10(digitsInFraction) + fields(Calendar.MILLISECOND) = (d / MICROS_PER_SECOND).toInt + } +} + +class LegacyFastTimestampFormatter( pattern: String, zoneId: ZoneId, locale: Locale) extends TimestampFormatter { - @transient private lazy val format = + @transient private lazy val fastDateFormat = FastDateFormat.getInstance(pattern, TimeZone.getTimeZone(zoneId), locale) + @transient private lazy val cal = new MicrosCalendar( + fastDateFormat.getTimeZone, + fastDateFormat.getPattern.count(_ == 'S')) + + def parse(s: String): SQLTimestamp = { + cal.clear() // Clear the calendar because it can be re-used many times + if (!fastDateFormat.parse(s, new ParsePosition(0), cal)) { + throw new IllegalArgumentException(s"'$s' is an invalid timestamp") + } + val micros = cal.getMicros() + cal.set(Calendar.MILLISECOND, 0) + cal.getTimeInMillis * MICROS_PER_MILLIS + micros + } + + def format(timestamp: SQLTimestamp): String = { + cal.setTimeInMillis(Math.floorDiv(timestamp, MICROS_PER_SECOND) * MILLIS_PER_SECOND) + cal.setMicros(Math.floorMod(timestamp, MICROS_PER_SECOND)) + fastDateFormat.format(cal) + } +} - protected def toMillis(s: String): Long = format.parse(s).getTime +class LegacySimpleTimestampFormatter( + pattern: String, + zoneId: ZoneId, + locale: Locale, + lenient: Boolean = true) extends TimestampFormatter { + @transient private lazy val sdf = { + val formatter = new SimpleDateFormat(pattern, locale) + formatter.setTimeZone(TimeZone.getTimeZone(zoneId)) + formatter.setLenient(lenient) + formatter + } - override def parse(s: String): Long = toMillis(s) * MICROS_PER_MILLIS + override def parse(s: String): Long = { + sdf.parse(s).getTime * MICROS_PER_MILLIS + } override def format(us: Long): String = { - format.format(DateTimeUtils.toJavaTimestamp(us)) + val timestamp = DateTimeUtils.toJavaTimestamp(us) + sdf.format(timestamp) } } +object LegacyDateFormats extends Enumeration { + type LegacyDateFormat = Value + val FAST_DATE_FORMAT, SIMPLE_DATE_FORMAT, LENIENT_SIMPLE_DATE_FORMAT = Value +} + object TimestampFormatter { + import LegacyDateFormats._ + val defaultLocale: Locale = Locale.US - def apply(format: String, zoneId: ZoneId, locale: Locale): TimestampFormatter = { + def defaultPattern(): String = s"${DateFormatter.defaultPattern()} HH:mm:ss" + + private def getFormatter( + format: Option[String], + zoneId: ZoneId, + locale: Locale = defaultLocale, + legacyFormat: LegacyDateFormat = LENIENT_SIMPLE_DATE_FORMAT): TimestampFormatter = { + + val pattern = format.getOrElse(defaultPattern) if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyTimestampFormatter(format, zoneId, locale) + legacyFormat match { + case FAST_DATE_FORMAT => + new LegacyFastTimestampFormatter(pattern, zoneId, locale) + case SIMPLE_DATE_FORMAT => + new LegacySimpleTimestampFormatter(pattern, zoneId, locale, lenient = false) + case LENIENT_SIMPLE_DATE_FORMAT => + new LegacySimpleTimestampFormatter(pattern, zoneId, locale, lenient = true) + } } else { - new Iso8601TimestampFormatter(format, zoneId, locale) + new Iso8601TimestampFormatter(pattern, zoneId, locale) } } + def apply( + format: String, + zoneId: ZoneId, + locale: Locale, + legacyFormat: LegacyDateFormat): TimestampFormatter = { + getFormatter(Some(format), zoneId, locale, legacyFormat) + } + + def apply(format: String, zoneId: ZoneId, legacyFormat: LegacyDateFormat): TimestampFormatter = { + getFormatter(Some(format), zoneId, defaultLocale, legacyFormat) + } + def apply(format: String, zoneId: ZoneId): TimestampFormatter = { - apply(format, zoneId, defaultLocale) + getFormatter(Some(format), zoneId) } def apply(zoneId: ZoneId): TimestampFormatter = { - if (SQLConf.get.legacyTimeParserEnabled) { - new LegacyTimestampFormatter("yyyy-MM-dd HH:mm:ss", zoneId, defaultLocale) - } else { - new Iso8601TimestampFormatter("uuuu-MM-dd HH:mm:ss", zoneId, defaultLocale) - } + getFormatter(None, zoneId) } def getFractionFormatter(zoneId: ZoneId): TimestampFormatter = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 9ce64b09f7870..f32e48e1cc128 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -541,7 +541,7 @@ object Decimal { /** Maximum number of decimal digits a Long can represent */ val MAX_LONG_DIGITS = 18 - private val POW_10 = Array.tabulate[Long](MAX_LONG_DIGITS + 1)(i => math.pow(10, i).toLong) + val POW_10 = Array.tabulate[Long](MAX_LONG_DIGITS + 1)(i => math.pow(10, i).toLong) private val BIG_DEC_ZERO = BigDecimal(0) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 274d0beebd300..f04149ab7eb29 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.util.{DateTimeUtils, IntervalUtils, Timesta import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -241,41 +242,45 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("DateFormat") { - checkEvaluation( - DateFormatClass(Literal.create(null, TimestampType), Literal("y"), gmtId), - null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), - Literal.create(null, StringType), gmtId), null) - - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), - Literal("y"), gmtId), "2015") - checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), gmtId), "2013") - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), - Literal("H"), gmtId), "0") - checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), gmtId), "13") - - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, pstId), - Literal("y"), pstId), "2015") - checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), pstId), "2013") - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, pstId), - Literal("H"), pstId), "0") - checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), pstId), "5") - - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId), - Literal("y"), jstId), "2015") - checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), jstId), "2013") - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId), - Literal("H"), jstId), "0") - checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), jstId), "22") - - // SPARK-28072 The codegen path should work - checkEvaluation( - expression = DateFormatClass( - BoundReference(ordinal = 0, dataType = TimestampType, nullable = true), - BoundReference(ordinal = 1, dataType = StringType, nullable = true), - jstId), - expected = "22", - inputRow = InternalRow(DateTimeUtils.fromJavaTimestamp(ts), UTF8String.fromString("H"))) + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + checkEvaluation( + DateFormatClass(Literal.create(null, TimestampType), Literal("y"), gmtId), + null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), + Literal.create(null, StringType), gmtId), null) + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), + Literal("y"), gmtId), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), gmtId), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, gmtId), + Literal("H"), gmtId), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), gmtId), "13") + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, pstId), + Literal("y"), pstId), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), pstId), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, pstId), + Literal("H"), pstId), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), pstId), "5") + + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId), + Literal("y"), jstId), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), jstId), "2013") + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId), + Literal("H"), jstId), "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), jstId), "22") + + // SPARK-28072 The codegen path should work + checkEvaluation( + expression = DateFormatClass( + BoundReference(ordinal = 0, dataType = TimestampType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + jstId), + expected = "22", + inputRow = InternalRow(DateTimeUtils.fromJavaTimestamp(ts), UTF8String.fromString("H"))) + } + } } test("Hour") { @@ -705,162 +710,189 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("from_unixtime") { - val fmt1 = "yyyy-MM-dd HH:mm:ss" - val sdf1 = new SimpleDateFormat(fmt1, Locale.US) - val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2, Locale.US) - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) - sdf1.setTimeZone(tz) - sdf2.setTimeZone(tz) + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val fmt1 = "yyyy-MM-dd HH:mm:ss" + val sdf1 = new SimpleDateFormat(fmt1, Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) - checkEvaluation( - FromUnixTime(Literal(0L), Literal(fmt1), timeZoneId), - sdf1.format(new Timestamp(0))) - checkEvaluation(FromUnixTime( - Literal(1000L), Literal(fmt1), timeZoneId), - sdf1.format(new Timestamp(1000000))) - checkEvaluation( - FromUnixTime(Literal(-1000L), Literal(fmt2), timeZoneId), - sdf2.format(new Timestamp(-1000000))) - checkEvaluation( - FromUnixTime(Literal.create(null, LongType), Literal.create(null, StringType), timeZoneId), - null) - checkEvaluation( - FromUnixTime(Literal.create(null, LongType), Literal(fmt1), timeZoneId), - null) - checkEvaluation( - FromUnixTime(Literal(1000L), Literal.create(null, StringType), timeZoneId), - null) - checkEvaluation( - FromUnixTime(Literal(0L), Literal("not a valid format"), timeZoneId), null) + checkEvaluation( + FromUnixTime(Literal(0L), Literal(fmt1), timeZoneId), + sdf1.format(new Timestamp(0))) + checkEvaluation(FromUnixTime( + Literal(1000L), Literal(fmt1), timeZoneId), + sdf1.format(new Timestamp(1000000))) + checkEvaluation( + FromUnixTime(Literal(-1000L), Literal(fmt2), timeZoneId), + sdf2.format(new Timestamp(-1000000))) + checkEvaluation( + FromUnixTime( + Literal.create(null, LongType), + Literal.create(null, StringType), timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal.create(null, LongType), Literal(fmt1), timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal(1000L), Literal.create(null, StringType), timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal(0L), Literal("not a valid format"), timeZoneId), null) - // SPARK-28072 The codegen path for non-literal input should also work - checkEvaluation( - expression = FromUnixTime( - BoundReference(ordinal = 0, dataType = LongType, nullable = true), - BoundReference(ordinal = 1, dataType = StringType, nullable = true), - timeZoneId), - expected = UTF8String.fromString(sdf1.format(new Timestamp(0))), - inputRow = InternalRow(0L, UTF8String.fromString(fmt1))) + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = FromUnixTime( + BoundReference(ordinal = 0, dataType = LongType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = UTF8String.fromString(sdf1.format(new Timestamp(0))), + inputRow = InternalRow(0L, UTF8String.fromString(fmt1))) + } + } } } test("unix_timestamp") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) - val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2, Locale.US) - val fmt3 = "yy-MM-dd" - val sdf3 = new SimpleDateFormat(fmt3, Locale.US) - sdf3.setTimeZone(TimeZoneGMT) - - withDefaultTimeZone(TimeZoneGMT) { - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) - sdf1.setTimeZone(tz) - sdf2.setTimeZone(tz) - - val date1 = Date.valueOf("2015-07-24") - checkEvaluation(UnixTimestamp( - Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), 0L) - checkEvaluation(UnixTimestamp( - Literal(sdf1.format(new Timestamp(1000000))), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), - 1000L) - checkEvaluation( - UnixTimestamp( - Literal(new Timestamp(1000000)), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), - 1000L) - checkEvaluation( - UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) - checkEvaluation( - UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), - -1000L) - checkEvaluation(UnixTimestamp( - Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) - val t1 = UnixTimestamp( - CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] - val t2 = UnixTimestamp( - CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] - assert(t2 - t1 <= 1) - checkEvaluation( - UnixTimestamp( - Literal.create(null, DateType), Literal.create(null, StringType), timeZoneId), - null) - checkEvaluation( - UnixTimestamp(Literal.create(null, DateType), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), - null) - checkEvaluation( - UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) - checkEvaluation( - UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + sdf3.setTimeZone(TimeZoneGMT) + + withDefaultTimeZone(TimeZoneGMT) { + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation(UnixTimestamp( + Literal(sdf1.format(new Timestamp(0))), + Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), 0L) + checkEvaluation(UnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), + Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp( + Literal(new Timestamp(1000000)), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + MILLISECONDS.toSeconds( + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + checkEvaluation( + UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), + Literal(fmt2), timeZoneId), + -1000L) + checkEvaluation(UnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), + MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) + val t1 = UnixTimestamp( + CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] + val t2 = UnixTimestamp( + CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] + assert(t2 - t1 <= 1) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), Literal.create(null, StringType), timeZoneId), + null) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), + Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + null) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), + MILLISECONDS.toSeconds( + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + checkEvaluation( + UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) + } + } } } } test("to_unix_timestamp") { - val fmt1 = "yyyy-MM-dd HH:mm:ss" - val sdf1 = new SimpleDateFormat(fmt1, Locale.US) - val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2, Locale.US) - val fmt3 = "yy-MM-dd" - val sdf3 = new SimpleDateFormat(fmt3, Locale.US) - sdf3.setTimeZone(TimeZoneGMT) - - withDefaultTimeZone(TimeZoneGMT) { - for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { - val timeZoneId = Option(tz.getID) - sdf1.setTimeZone(tz) - sdf2.setTimeZone(tz) - - val date1 = Date.valueOf("2015-07-24") - checkEvaluation(ToUnixTimestamp( - Literal(sdf1.format(new Timestamp(0))), Literal(fmt1), timeZoneId), 0L) - checkEvaluation(ToUnixTimestamp( - Literal(sdf1.format(new Timestamp(1000000))), Literal(fmt1), timeZoneId), - 1000L) - checkEvaluation(ToUnixTimestamp( - Literal(new Timestamp(1000000)), Literal(fmt1)), - 1000L) - checkEvaluation( - ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) - checkEvaluation( - ToUnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), - -1000L) - checkEvaluation(ToUnixTimestamp( - Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) - val t1 = ToUnixTimestamp( - CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] - val t2 = ToUnixTimestamp( - CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] - assert(t2 - t1 <= 1) - checkEvaluation(ToUnixTimestamp( - Literal.create(null, DateType), Literal.create(null, StringType), timeZoneId), null) - checkEvaluation( - ToUnixTimestamp( - Literal.create(null, DateType), Literal(fmt1), timeZoneId), - null) - checkEvaluation(ToUnixTimestamp( - Literal(date1), Literal.create(null, StringType), timeZoneId), - MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) - checkEvaluation( - ToUnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val fmt1 = "yyyy-MM-dd HH:mm:ss" + val sdf1 = new SimpleDateFormat(fmt1, Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + sdf3.setTimeZone(TimeZoneGMT) + + withDefaultTimeZone(TimeZoneGMT) { + for (tz <- Seq(TimeZoneGMT, TimeZonePST, TimeZoneJST)) { + val timeZoneId = Option(tz.getID) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation(ToUnixTimestamp( + Literal(sdf1.format(new Timestamp(0))), Literal(fmt1), timeZoneId), 0L) + checkEvaluation(ToUnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), Literal(fmt1), timeZoneId), + 1000L) + checkEvaluation(ToUnixTimestamp( + Literal(new Timestamp(1000000)), Literal(fmt1)), + 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), + MILLISECONDS.toSeconds( + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf2.format(new Timestamp(-1000000))), + Literal(fmt2), timeZoneId), + -1000L) + checkEvaluation(ToUnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), + MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) + val t1 = ToUnixTimestamp( + CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] + val t2 = ToUnixTimestamp( + CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] + assert(t2 - t1 <= 1) + checkEvaluation(ToUnixTimestamp( + Literal.create(null, DateType), Literal.create(null, StringType), timeZoneId), null) + checkEvaluation( + ToUnixTimestamp( + Literal.create(null, DateType), Literal(fmt1), timeZoneId), + null) + checkEvaluation(ToUnixTimestamp( + Literal(date1), Literal.create(null, StringType), timeZoneId), + MILLISECONDS.toSeconds( + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + checkEvaluation( + ToUnixTimestamp( + Literal("2015-07-24"), + Literal("not a valid format"), timeZoneId), null) - // SPARK-28072 The codegen path for non-literal input should also work - checkEvaluation( - expression = ToUnixTimestamp( - BoundReference(ordinal = 0, dataType = StringType, nullable = true), - BoundReference(ordinal = 1, dataType = StringType, nullable = true), - timeZoneId), - expected = 0L, - inputRow = InternalRow( - UTF8String.fromString(sdf1.format(new Timestamp(0))), UTF8String.fromString(fmt1))) + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = ToUnixTimestamp( + BoundReference(ordinal = 0, dataType = StringType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = 0L, + inputRow = InternalRow( + UTF8String.fromString(sdf1.format(new Timestamp(0))), UTF8String.fromString(fmt1))) + } + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index d125581857e0b..2d5504ac00ffa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, HintInfo, ResolvedHint} +import org.apache.spark.sql.catalyst.util.TimestampFormatter import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.expressions.{Aggregator, SparkUserDefinedFunction, UserDefinedAggregator, UserDefinedFunction} import org.apache.spark.sql.internal.SQLConf @@ -2881,7 +2882,7 @@ object functions { * @since 1.5.0 */ def from_unixtime(ut: Column): Column = withExpr { - FromUnixTime(ut.expr, Literal("uuuu-MM-dd HH:mm:ss")) + FromUnixTime(ut.expr, Literal(TimestampFormatter.defaultPattern)) } /** @@ -2913,7 +2914,7 @@ object functions { * @since 1.5.0 */ def unix_timestamp(): Column = withExpr { - UnixTimestamp(CurrentTimestamp(), Literal("uuuu-MM-dd HH:mm:ss")) + UnixTimestamp(CurrentTimestamp(), Literal(TimestampFormatter.defaultPattern)) } /** @@ -2927,7 +2928,7 @@ object functions { * @since 1.5.0 */ def unix_timestamp(s: Column): Column = withExpr { - UnixTimestamp(s.expr, Literal("uuuu-MM-dd HH:mm:ss")) + UnixTimestamp(s.expr, Literal(TimestampFormatter.defaultPattern)) } /** diff --git a/sql/core/src/test/resources/test-data/bad_after_good.csv b/sql/core/src/test/resources/test-data/bad_after_good.csv index 4621a7d23714d..1a7c2651a11a7 100644 --- a/sql/core/src/test/resources/test-data/bad_after_good.csv +++ b/sql/core/src/test/resources/test-data/bad_after_good.csv @@ -1,2 +1,2 @@ "good record",1999-08-01 -"bad record",1999-088-01 +"bad record",1999-088_01 diff --git a/sql/core/src/test/resources/test-data/value-malformed.csv b/sql/core/src/test/resources/test-data/value-malformed.csv index 8945ed73d2e83..6e6f08fca6df8 100644 --- a/sql/core/src/test/resources/test-data/value-malformed.csv +++ b/sql/core/src/test/resources/test-data/value-malformed.csv @@ -1,2 +1,2 @@ -0,2013-111-11 12:13:14 +0,2013-111_11 12:13:14 1,1983-08-04 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index bb8cdf3cb6de1..41d53c959ef99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -96,15 +96,19 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { } test("date format") { - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - checkAnswer( - df.select(date_format($"a", "y"), date_format($"b", "y"), date_format($"c", "y")), - Row("2015", "2015", "2013")) + checkAnswer( + df.select(date_format($"a", "y"), date_format($"b", "y"), date_format($"c", "y")), + Row("2015", "2015", "2013")) - checkAnswer( - df.selectExpr("date_format(a, 'y')", "date_format(b, 'y')", "date_format(c, 'y')"), - Row("2015", "2015", "2013")) + checkAnswer( + df.selectExpr("date_format(a, 'y')", "date_format(b, 'y')", "date_format(c, 'y')"), + Row("2015", "2015", "2013")) + } + } } test("year") { @@ -525,170 +529,194 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { } test("from_unixtime") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) - val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2, Locale.US) - val fmt3 = "yy-MM-dd HH-mm-ss" - val sdf3 = new SimpleDateFormat(fmt3, Locale.US) - val df = Seq((1000, "yyyy-MM-dd HH:mm:ss.SSS"), (-1000, "yy-MM-dd HH-mm-ss")).toDF("a", "b") - checkAnswer( - df.select(from_unixtime(col("a"))), - Seq(Row(sdf1.format(new Timestamp(1000000))), Row(sdf1.format(new Timestamp(-1000000))))) - checkAnswer( - df.select(from_unixtime(col("a"), fmt2)), - Seq(Row(sdf2.format(new Timestamp(1000000))), Row(sdf2.format(new Timestamp(-1000000))))) - checkAnswer( - df.select(from_unixtime(col("a"), fmt3)), - Seq(Row(sdf3.format(new Timestamp(1000000))), Row(sdf3.format(new Timestamp(-1000000))))) - checkAnswer( - df.selectExpr("from_unixtime(a)"), - Seq(Row(sdf1.format(new Timestamp(1000000))), Row(sdf1.format(new Timestamp(-1000000))))) - checkAnswer( - df.selectExpr(s"from_unixtime(a, '$fmt2')"), - Seq(Row(sdf2.format(new Timestamp(1000000))), Row(sdf2.format(new Timestamp(-1000000))))) - checkAnswer( - df.selectExpr(s"from_unixtime(a, '$fmt3')"), - Seq(Row(sdf3.format(new Timestamp(1000000))), Row(sdf3.format(new Timestamp(-1000000))))) + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd HH-mm-ss" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + val df = Seq((1000, "yyyy-MM-dd HH:mm:ss.SSS"), (-1000, "yy-MM-dd HH-mm-ss")).toDF("a", "b") + checkAnswer( + df.select(from_unixtime(col("a"))), + Seq(Row(sdf1.format(new Timestamp(1000000))), Row(sdf1.format(new Timestamp(-1000000))))) + checkAnswer( + df.select(from_unixtime(col("a"), fmt2)), + Seq(Row(sdf2.format(new Timestamp(1000000))), Row(sdf2.format(new Timestamp(-1000000))))) + checkAnswer( + df.select(from_unixtime(col("a"), fmt3)), + Seq(Row(sdf3.format(new Timestamp(1000000))), Row(sdf3.format(new Timestamp(-1000000))))) + checkAnswer( + df.selectExpr("from_unixtime(a)"), + Seq(Row(sdf1.format(new Timestamp(1000000))), Row(sdf1.format(new Timestamp(-1000000))))) + checkAnswer( + df.selectExpr(s"from_unixtime(a, '$fmt2')"), + Seq(Row(sdf2.format(new Timestamp(1000000))), Row(sdf2.format(new Timestamp(-1000000))))) + checkAnswer( + df.selectExpr(s"from_unixtime(a, '$fmt3')"), + Seq(Row(sdf3.format(new Timestamp(1000000))), Row(sdf3.format(new Timestamp(-1000000))))) + } + } } private def secs(millis: Long): Long = TimeUnit.MILLISECONDS.toSeconds(millis) test("unix_timestamp") { - val date1 = Date.valueOf("2015-07-24") - val date2 = Date.valueOf("2015-07-25") - val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") - val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") - val s1 = "2015/07/24 10:00:00.5" - val s2 = "2015/07/25 02:02:02.6" - val ss1 = "2015-07-24 10:00:00" - val ss2 = "2015-07-25 02:02:02" - val fmt = "yyyy/MM/dd HH:mm:ss.S" - val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") - checkAnswer(df.select(unix_timestamp(col("ts"))), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - checkAnswer(df.select(unix_timestamp(col("ss"))), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - checkAnswer(df.select(unix_timestamp(col("d"), fmt)), Seq( - Row(secs(date1.getTime)), Row(secs(date2.getTime)))) - checkAnswer(df.select(unix_timestamp(col("s"), fmt)), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - checkAnswer(df.selectExpr("unix_timestamp(ts)"), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - checkAnswer(df.selectExpr("unix_timestamp(ss)"), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - checkAnswer(df.selectExpr(s"unix_timestamp(d, '$fmt')"), Seq( - Row(secs(date1.getTime)), Row(secs(date2.getTime)))) - checkAnswer(df.selectExpr(s"unix_timestamp(s, '$fmt')"), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - - val x1 = "2015-07-24 10:00:00" - val x2 = "2015-25-07 02:02:02" - val x3 = "2015-07-24 25:02:02" - val x4 = "2015-24-07 26:02:02" - val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") - val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") - - val df1 = Seq(x1, x2, x3, x4).toDF("x") - checkAnswer(df1.select(unix_timestamp(col("x"))), Seq( - Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) - checkAnswer(df1.selectExpr("unix_timestamp(x)"), Seq( - Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) - checkAnswer(df1.select(unix_timestamp(col("x"), "yyyy-dd-MM HH:mm:ss")), Seq( - Row(null), Row(secs(ts2.getTime)), Row(null), Row(null))) - checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), Seq( - Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) - - // invalid format - checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')"), Seq( - Row(null), Row(null), Row(null), Row(null))) - - // february - val y1 = "2016-02-29" - val y2 = "2017-02-29" - val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") - val df2 = Seq(y1, y2).toDF("y") - checkAnswer(df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), Seq( - Row(secs(ts5.getTime)), Row(null))) - - val now = sql("select unix_timestamp()").collect().head.getLong(0) - checkAnswer( - sql(s"select cast ($now as timestamp)"), - Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") + checkAnswer(df.select(unix_timestamp(col("ts"))), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer(df.select(unix_timestamp(col("ss"))), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer(df.select(unix_timestamp(col("d"), fmt)), Seq( + Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer(df.select(unix_timestamp(col("s"), fmt)), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer(df.selectExpr("unix_timestamp(ts)"), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer(df.selectExpr("unix_timestamp(ss)"), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer(df.selectExpr(s"unix_timestamp(d, '$fmt')"), Seq( + Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer(df.selectExpr(s"unix_timestamp(s, '$fmt')"), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer(df1.select(unix_timestamp(col("x"))), Seq( + Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer(df1.selectExpr("unix_timestamp(x)"), Seq( + Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer(df1.select(unix_timestamp(col("x"), "yyyy-dd-MM HH:mm:ss")), Seq( + Row(null), Row(secs(ts2.getTime)), Row(null), Row(null))) + checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), Seq( + Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) + + // invalid format + checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')"), Seq( + Row(null), Row(null), Row(null), Row(null))) + + // february + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer(df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), Seq( + Row(secs(ts5.getTime)), Row(null))) + + val now = sql("select unix_timestamp()").collect().head.getLong(0) + checkAnswer( + sql(s"select cast ($now as timestamp)"), + Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) + } + } } test("to_unix_timestamp") { - val date1 = Date.valueOf("2015-07-24") - val date2 = Date.valueOf("2015-07-25") - val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") - val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") - val s1 = "2015/07/24 10:00:00.5" - val s2 = "2015/07/25 02:02:02.6" - val ss1 = "2015-07-24 10:00:00" - val ss2 = "2015-07-25 02:02:02" - val fmt = "yyyy/MM/dd HH:mm:ss.S" - val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") - checkAnswer(df.selectExpr("to_unix_timestamp(ts)"), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - checkAnswer(df.selectExpr("to_unix_timestamp(ss)"), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - checkAnswer(df.selectExpr(s"to_unix_timestamp(d, '$fmt')"), Seq( - Row(secs(date1.getTime)), Row(secs(date2.getTime)))) - checkAnswer(df.selectExpr(s"to_unix_timestamp(s, '$fmt')"), Seq( - Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) - - val x1 = "2015-07-24 10:00:00" - val x2 = "2015-25-07 02:02:02" - val x3 = "2015-07-24 25:02:02" - val x4 = "2015-24-07 26:02:02" - val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") - val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") - - val df1 = Seq(x1, x2, x3, x4).toDF("x") - checkAnswer(df1.selectExpr("to_unix_timestamp(x)"), Seq( - Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) - checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), Seq( - Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) - - // february - val y1 = "2016-02-29" - val y2 = "2017-02-29" - val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") - val df2 = Seq(y1, y2).toDF("y") - checkAnswer(df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), Seq( - Row(secs(ts5.getTime)), Row(null))) - - // invalid format - checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')"), Seq( - Row(null), Row(null), Row(null), Row(null))) + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") + checkAnswer(df.selectExpr("to_unix_timestamp(ts)"), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer(df.selectExpr("to_unix_timestamp(ss)"), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer(df.selectExpr(s"to_unix_timestamp(d, '$fmt')"), Seq( + Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer(df.selectExpr(s"to_unix_timestamp(s, '$fmt')"), Seq( + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer(df1.selectExpr("to_unix_timestamp(x)"), Seq( + Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), Seq( + Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) + + // february + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer(df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), Seq( + Row(secs(ts5.getTime)), Row(null))) + + // invalid format + checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')"), Seq( + Row(null), Row(null), Row(null), Row(null))) + } + } } test("to_timestamp") { - val date1 = Date.valueOf("2015-07-24") - val date2 = Date.valueOf("2015-07-25") - val ts_date1 = Timestamp.valueOf("2015-07-24 00:00:00") - val ts_date2 = Timestamp.valueOf("2015-07-25 00:00:00") - val ts1 = Timestamp.valueOf("2015-07-24 10:00:00") - val ts2 = Timestamp.valueOf("2015-07-25 02:02:02") - val s1 = "2015/07/24 10:00:00.5" - val s2 = "2015/07/25 02:02:02.6" - val ts1m = Timestamp.valueOf("2015-07-24 10:00:00.5") - val ts2m = Timestamp.valueOf("2015-07-25 02:02:02.6") - val ss1 = "2015-07-24 10:00:00" - val ss2 = "2015-07-25 02:02:02" - val fmt = "yyyy/MM/dd HH:mm:ss.S" - val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") - - checkAnswer(df.select(to_timestamp(col("ss"))), - df.select(unix_timestamp(col("ss")).cast("timestamp"))) - checkAnswer(df.select(to_timestamp(col("ss"))), Seq( - Row(ts1), Row(ts2))) - checkAnswer(df.select(to_timestamp(col("s"), fmt)), Seq( - Row(ts1m), Row(ts2m))) - checkAnswer(df.select(to_timestamp(col("ts"), fmt)), Seq( - Row(ts1), Row(ts2))) - checkAnswer(df.select(to_timestamp(col("d"), "yyyy-MM-dd")), Seq( - Row(ts_date1), Row(ts_date2))) + Seq(false, true).foreach { legacyParser => + withSQLConf(SQLConf.LEGACY_TIME_PARSER_ENABLED.key -> legacyParser.toString) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts_date1 = Timestamp.valueOf("2015-07-24 00:00:00") + val ts_date2 = Timestamp.valueOf("2015-07-25 00:00:00") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ts1m = Timestamp.valueOf("2015-07-24 10:00:00.5") + val ts2m = Timestamp.valueOf("2015-07-25 02:02:02.6") + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") + + checkAnswer(df.select(to_timestamp(col("ss"))), + df.select(unix_timestamp(col("ss")).cast("timestamp"))) + checkAnswer(df.select(to_timestamp(col("ss"))), Seq( + Row(ts1), Row(ts2))) + if (legacyParser) { + // In Spark 2.4 and earlier, to_timestamp() parses in seconds precision and cuts off + // the fractional part of seconds. The behavior was changed by SPARK-27438. + val legacyFmt = "yyyy/MM/dd HH:mm:ss" + checkAnswer(df.select(to_timestamp(col("s"), legacyFmt)), Seq( + Row(ts1), Row(ts2))) + } else { + checkAnswer(df.select(to_timestamp(col("s"), fmt)), Seq( + Row(ts1m), Row(ts2m))) + } + checkAnswer(df.select(to_timestamp(col("ts"), fmt)), Seq( + Row(ts1), Row(ts2))) + checkAnswer(df.select(to_timestamp(col("d"), "yyyy-MM-dd")), Seq( + Row(ts_date1), Row(ts_date2))) + } + } } test("datediff") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 97dfbbdb7fd2f..b1105b4a63bba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -1182,7 +1182,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa .schema(schemaWithCorrField1) .csv(testFile(valueMalformedFile)) checkAnswer(df2, - Row(0, null, "0,2013-111-11 12:13:14") :: + Row(0, null, "0,2013-111_11 12:13:14") :: Row(1, java.sql.Date.valueOf("1983-08-04"), null) :: Nil) @@ -1199,7 +1199,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa .schema(schemaWithCorrField2) .csv(testFile(valueMalformedFile)) checkAnswer(df3, - Row(0, "0,2013-111-11 12:13:14", null) :: + Row(0, "0,2013-111_11 12:13:14", null) :: Row(1, null, java.sql.Date.valueOf("1983-08-04")) :: Nil) @@ -1435,7 +1435,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa assert(df.filter($"_corrupt_record".isNull).count() == 1) checkAnswer( df.select(columnNameOfCorruptRecord), - Row("0,2013-111-11 12:13:14") :: Row(null) :: Nil + Row("0,2013-111_11 12:13:14") :: Row(null) :: Nil ) } @@ -2093,7 +2093,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa Seq("csv", "").foreach { reader => withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> reader) { withTempPath { path => - val df = Seq(("0", "2013-111-11")).toDF("a", "b") + val df = Seq(("0", "2013-111_11")).toDF("a", "b") df.write .option("header", "true") .csv(path.getAbsolutePath) @@ -2109,7 +2109,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) .schema(schemaWithCorrField) .csv(path.getAbsoluteFile.toString) - checkAnswer(readDF, Row(0, null, "0,2013-111-11") :: Nil) + checkAnswer(readDF, Row(0, null, "0,2013-111_11") :: Nil) } } } @@ -2216,7 +2216,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa val readback = spark.read .option("mode", mode) .option("header", true) - .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") .option("multiLine", multiLine) .schema("c0 string, c1 integer, c2 timestamp") .csv(path.getAbsolutePath) @@ -2235,7 +2235,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa } test("filters push down - malformed input in PERMISSIVE mode") { - val invalidTs = "2019-123-14 20:35:30" + val invalidTs = "2019-123_14 20:35:30" val invalidRow = s"0,$invalidTs,999" val validTs = "2019-12-14 20:35:30" Seq(true, false).foreach { filterPushdown => @@ -2252,7 +2252,7 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa .option("mode", "PERMISSIVE") .option("columnNameOfCorruptRecord", "c3") .option("header", true) - .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") .schema("c0 integer, c1 timestamp, c2 integer, c3 string") .csv(path.getAbsolutePath) .where(condition) @@ -2309,3 +2309,10 @@ class CSVv2Suite extends CSVSuite { .sparkConf .set(SQLConf.USE_V1_SOURCE_LIST, "") } + +class CSVLegacyTimeParserSuite extends CSVSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.LEGACY_TIME_PARSER_ENABLED, true) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index b20da2266b0f3..7abe818a29d9f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -2572,3 +2572,10 @@ class JsonV2Suite extends JsonSuite { .sparkConf .set(SQLConf.USE_V1_SOURCE_LIST, "") } + +class JsonLegacyTimeParserSuite extends JsonSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.LEGACY_TIME_PARSER_ENABLED, true) +} From 61b1e608f07afd965028313c13bf89c19b006312 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 12 Feb 2020 23:50:34 +0800 Subject: [PATCH 1356/1387] [SPARK-30759][SQL][TESTS][FOLLOWUP] Check cache initialization in StringRegexExpression ### What changes were proposed in this pull request? Added new test to `RegexpExpressionsSuite` which checks that `cache` of compiled pattern is set when the `right` expression (pattern in `LIKE`) is a foldable expression. ### Why are the changes needed? To be sure that `cache` in `StringRegexExpression` is initialized for foldable patterns. ### Does this PR introduce any user-facing change? No ### How was this patch tested? By running the added test in `RegexpExpressionsSuite`. Closes #27547 from MaxGekk/regexp-cache-test. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/RegexpExpressionsSuite.scala | 8 ++++++++ 1 file changed, 8 insertions(+) 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 86da62bc74940..712d2bc4c4736 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 @@ -329,4 +329,12 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringSplit(s1, s2, -1), null, row3) } + test("SPARK-30759: cache initialization for literal patterns") { + val expr = "A" like Literal.create("a", StringType) + expr.eval() + val cache = expr.getClass.getSuperclass + .getDeclaredFields.filter(_.getName.endsWith("cache")).head + cache.setAccessible(true) + assert(cache.get(expr).asInstanceOf[java.util.regex.Pattern].pattern().contains("a")) + } } From 5919bd3b8d3ef3c3e957d8e3e245e00383b979bf Mon Sep 17 00:00:00 2001 From: Eric Wu <492960551@qq.com> Date: Thu, 13 Feb 2020 02:00:23 +0800 Subject: [PATCH 1357/1387] [SPARK-30651][SQL] Add detailed information for Aggregate operators in EXPLAIN FORMATTED ### What changes were proposed in this pull request? Currently `EXPLAIN FORMATTED` only report input attributes of HashAggregate/ObjectHashAggregate/SortAggregate, while `EXPLAIN EXTENDED` provides more information of Keys, Functions, etc. This PR enhanced `EXPLAIN FORMATTED` to sync with original explain behavior. ### Why are the changes needed? The newly added `EXPLAIN FORMATTED` got less information comparing to the original `EXPLAIN EXTENDED` ### Does this PR introduce any user-facing change? Yes, taking HashAggregate explain result as example. **SQL** ``` EXPLAIN FORMATTED SELECT COUNT(val) + SUM(key) as TOTAL, COUNT(key) FILTER (WHERE val > 1) FROM explain_temp1; ``` **EXPLAIN EXTENDED** ``` == Physical Plan == *(2) HashAggregate(keys=[], functions=[count(val#6), sum(cast(key#5 as bigint)), count(key#5)], output=[TOTAL#62L, count(key) FILTER (WHERE (val > 1))#71L]) +- Exchange SinglePartition, true, [id=#89] +- HashAggregate(keys=[], functions=[partial_count(val#6), partial_sum(cast(key#5 as bigint)), partial_count(key#5) FILTER (WHERE (val#6 > 1))], output=[count#75L, sum#76L, count#77L]) +- *(1) ColumnarToRow +- FileScan parquet default.explain_temp1[key#5,val#6] Batched: true, DataFilters: [], Format: Parquet, Location: InMemoryFileIndex[file:/Users/XXX/spark-dev/spark/spark-warehouse/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct ``` **EXPLAIN FORMATTED - BEFORE** ``` == Physical Plan == * HashAggregate (5) +- Exchange (4) +- HashAggregate (3) +- * ColumnarToRow (2) +- Scan parquet default.explain_temp1 (1) ... ... (5) HashAggregate [codegen id : 2] Input: [count#91L, sum#92L, count#93L] ... ... ``` **EXPLAIN FORMATTED - AFTER** ``` == Physical Plan == * HashAggregate (5) +- Exchange (4) +- HashAggregate (3) +- * ColumnarToRow (2) +- Scan parquet default.explain_temp1 (1) ... ... (5) HashAggregate [codegen id : 2] Input: [count#91L, sum#92L, count#93L] Keys: [] Functions: [count(val#6), sum(cast(key#5 as bigint)), count(key#5)] Results: [(count(val#6)#84L + sum(cast(key#5 as bigint))#85L) AS TOTAL#78L, count(key#5)#86L AS count(key) FILTER (WHERE (val > 1))#87L] Output: [TOTAL#78L, count(key) FILTER (WHERE (val > 1))#87L] ... ... ``` ### How was this patch tested? Three tests added in explain.sql for HashAggregate/ObjectHashAggregate/SortAggregate. Closes #27368 from Eric5553/ExplainFormattedAgg. Authored-by: Eric Wu <492960551@qq.com> Signed-off-by: Wenchen Fan --- .../aggregate/BaseAggregateExec.scala | 48 ++++ .../aggregate/HashAggregateExec.scala | 2 +- .../aggregate/ObjectHashAggregateExec.scala | 2 +- .../aggregate/SortAggregateExec.scala | 4 +- .../resources/sql-tests/inputs/explain.sql | 22 +- .../sql-tests/results/explain.sql.out | 232 +++++++++++++++++- 6 files changed, 300 insertions(+), 10 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala new file mode 100644 index 0000000000000..0eaa0f53fdacd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala @@ -0,0 +1,48 @@ +/* + * 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.aggregate + +import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.execution.{ExplainUtils, UnaryExecNode} + +/** + * Holds common logic for aggregate operators + */ +trait BaseAggregateExec extends UnaryExecNode { + def groupingExpressions: Seq[NamedExpression] + def aggregateExpressions: Seq[AggregateExpression] + def aggregateAttributes: Seq[Attribute] + def resultExpressions: Seq[NamedExpression] + + override def verboseStringWithOperatorId(): String = { + val inputString = child.output.mkString("[", ", ", "]") + val keyString = groupingExpressions.mkString("[", ", ", "]") + val functionString = aggregateExpressions.mkString("[", ", ", "]") + val aggregateAttributeString = aggregateAttributes.mkString("[", ", ", "]") + val resultString = resultExpressions.mkString("[", ", ", "]") + s""" + |(${ExplainUtils.getOpId(this)}) $nodeName ${ExplainUtils.getCodegenId(this)} + |Input: $inputString + |Keys: $keyString + |Functions: $functionString + |Aggregate Attributes: $aggregateAttributeString + |Results: $resultString + """.stripMargin + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index f73e214a6b41f..7a26fd7a8541a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -53,7 +53,7 @@ case class HashAggregateExec( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode with BlockingOperatorWithCodegen with AliasAwareOutputPartitioning { + extends BaseAggregateExec with BlockingOperatorWithCodegen with AliasAwareOutputPartitioning { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index 4376f6b6edd57..3fb58eb2cc8ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -67,7 +67,7 @@ case class ObjectHashAggregateExec( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode with AliasAwareOutputPartitioning { + extends BaseAggregateExec with AliasAwareOutputPartitioning { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index b6e684e62ea5c..77ed469016fa3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.{AliasAwareOutputPartitioning, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{AliasAwareOutputPartitioning, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -38,7 +38,7 @@ case class SortAggregateExec( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode with AliasAwareOutputPartitioning { + extends BaseAggregateExec with AliasAwareOutputPartitioning { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain.sql b/sql/core/src/test/resources/sql-tests/inputs/explain.sql index d5253e3daddb0..497b61c6134a2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/explain.sql @@ -5,6 +5,7 @@ CREATE table explain_temp1 (key int, val int) USING PARQUET; CREATE table explain_temp2 (key int, val int) USING PARQUET; CREATE table explain_temp3 (key int, val int) USING PARQUET; +CREATE table explain_temp4 (key int, val string) USING PARQUET; SET spark.sql.codegen.wholeStage = true; @@ -61,7 +62,7 @@ EXPLAIN FORMATTED FROM explain_temp2 WHERE val > 0) OR - key = (SELECT max(key) + key = (SELECT avg(key) FROM explain_temp3 WHERE val > 0); @@ -93,6 +94,25 @@ EXPLAIN FORMATTED CREATE VIEW explain_view AS SELECT key, val FROM explain_temp1; +-- HashAggregate +EXPLAIN FORMATTED + SELECT + COUNT(val) + SUM(key) as TOTAL, + COUNT(key) FILTER (WHERE val > 1) + FROM explain_temp1; + +-- ObjectHashAggregate +EXPLAIN FORMATTED + SELECT key, sort_array(collect_set(val))[0] + FROM explain_temp4 + GROUP BY key; + +-- SortAggregate +EXPLAIN FORMATTED + SELECT key, MIN(val) + FROM explain_temp4 + GROUP BY key; + -- cleanup DROP TABLE explain_temp1; DROP TABLE explain_temp2; diff --git a/sql/core/src/test/resources/sql-tests/results/explain.sql.out b/sql/core/src/test/resources/sql-tests/results/explain.sql.out index 756c14f28a657..bc28d7f87bf00 100644 --- a/sql/core/src/test/resources/sql-tests/results/explain.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/explain.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 18 +-- Number of queries: 22 -- !query @@ -26,6 +26,14 @@ struct<> +-- !query +CREATE table explain_temp4 (key int, val string) USING PARQUET +-- !query schema +struct<> +-- !query output + + + -- !query SET spark.sql.codegen.wholeStage = true -- !query schema @@ -76,12 +84,20 @@ Input : [key#x, val#x] (5) HashAggregate [codegen id : 1] Input: [key#x, val#x] +Keys: [key#x] +Functions: [partial_max(val#x)] +Aggregate Attributes: [max#x] +Results: [key#x, max#x] (6) Exchange Input: [key#x, max#x] (7) HashAggregate [codegen id : 2] Input: [key#x, max#x] +Keys: [key#x] +Functions: [max(val#x)] +Aggregate Attributes: [max(val#x)#x] +Results: [key#x, max(val#x)#x AS max(val)#x] (8) Exchange Input: [key#x, max(val)#x] @@ -132,12 +148,20 @@ Input : [key#x, val#x] (5) HashAggregate [codegen id : 1] Input: [key#x, val#x] +Keys: [key#x] +Functions: [partial_max(val#x)] +Aggregate Attributes: [max#x] +Results: [key#x, max#x] (6) Exchange Input: [key#x, max#x] (7) HashAggregate [codegen id : 2] Input: [key#x, max#x] +Keys: [key#x] +Functions: [max(val#x)] +Aggregate Attributes: [max(val#x)#x] +Results: [key#x, max(val#x)#x AS max(val)#x, max(val#x)#x AS max(val#x)#x] (8) Filter [codegen id : 2] Input : [key#x, max(val)#x, max(val#x)#x] @@ -211,12 +235,20 @@ Input : [key#x, val#x] (10) HashAggregate [codegen id : 3] Input: [key#x, val#x] +Keys: [key#x, val#x] +Functions: [] +Aggregate Attributes: [] +Results: [key#x, val#x] (11) Exchange Input: [key#x, val#x] (12) HashAggregate [codegen id : 4] Input: [key#x, val#x] +Keys: [key#x, val#x] +Functions: [] +Aggregate Attributes: [] +Results: [key#x, val#x] -- !query @@ -413,12 +445,20 @@ Input : [key#x, val#x] (9) HashAggregate [codegen id : 1] Input: [key#x] +Keys: [] +Functions: [partial_max(key#x)] +Aggregate Attributes: [max#x] +Results: [max#x] (10) Exchange Input: [max#x] (11) HashAggregate [codegen id : 2] Input: [max#x] +Keys: [] +Functions: [max(key#x)] +Aggregate Attributes: [max(key#x)#x] +Results: [max(key#x)#x AS max(key)#x] Subquery:2 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#x, [id=#x] * HashAggregate (18) @@ -450,12 +490,20 @@ Input : [key#x, val#x] (16) HashAggregate [codegen id : 1] Input: [key#x] +Keys: [] +Functions: [partial_max(key#x)] +Aggregate Attributes: [max#x] +Results: [max#x] (17) Exchange Input: [max#x] (18) HashAggregate [codegen id : 2] Input: [max#x] +Keys: [] +Functions: [max(key#x)] +Aggregate Attributes: [max(key#x)#x] +Results: [max(key#x)#x AS max(key)#x] -- !query @@ -466,7 +514,7 @@ EXPLAIN FORMATTED FROM explain_temp2 WHERE val > 0) OR - key = (SELECT max(key) + key = (SELECT avg(key) FROM explain_temp3 WHERE val > 0) -- !query schema @@ -489,7 +537,7 @@ Input: [key#x, val#x] (3) Filter [codegen id : 1] Input : [key#x, val#x] -Condition : ((key#x = Subquery scalar-subquery#x, [id=#x]) OR (key#x = Subquery scalar-subquery#x, [id=#x])) +Condition : ((key#x = Subquery scalar-subquery#x, [id=#x]) OR (cast(key#x as double) = Subquery scalar-subquery#x, [id=#x])) ===== Subqueries ===== @@ -523,12 +571,20 @@ Input : [key#x, val#x] (8) HashAggregate [codegen id : 1] Input: [key#x] +Keys: [] +Functions: [partial_max(key#x)] +Aggregate Attributes: [max#x] +Results: [max#x] (9) Exchange Input: [max#x] (10) HashAggregate [codegen id : 2] Input: [max#x] +Keys: [] +Functions: [max(key#x)] +Aggregate Attributes: [max(key#x)#x] +Results: [max(key#x)#x AS max(key)#x] Subquery:2 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#x, [id=#x] * HashAggregate (17) @@ -560,12 +616,20 @@ Input : [key#x, val#x] (15) HashAggregate [codegen id : 1] Input: [key#x] +Keys: [] +Functions: [partial_avg(cast(key#x as bigint))] +Aggregate Attributes: [sum#x, count#xL] +Results: [sum#x, count#xL] (16) Exchange -Input: [max#x] +Input: [sum#x, count#xL] (17) HashAggregate [codegen id : 2] -Input: [max#x] +Input: [sum#x, count#xL] +Keys: [] +Functions: [avg(cast(key#x as bigint))] +Aggregate Attributes: [avg(cast(key#x as bigint))#x] +Results: [avg(cast(key#x as bigint))#x AS avg(key)#x] -- !query @@ -615,12 +679,20 @@ Input: [key#x] (6) HashAggregate [codegen id : 1] Input: [key#x] +Keys: [] +Functions: [partial_avg(cast(key#x as bigint))] +Aggregate Attributes: [sum#x, count#xL] +Results: [sum#x, count#xL] (7) Exchange Input: [sum#x, count#xL] (8) HashAggregate [codegen id : 2] Input: [sum#x, count#xL] +Keys: [] +Functions: [avg(cast(key#x as bigint))] +Aggregate Attributes: [avg(cast(key#x as bigint))#x] +Results: [avg(cast(key#x as bigint))#x AS avg(key)#x] Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#x, [id=#x] @@ -740,18 +812,30 @@ Input : [key#x, val#x] (5) HashAggregate [codegen id : 1] Input: [key#x, val#x] +Keys: [key#x] +Functions: [partial_max(val#x)] +Aggregate Attributes: [max#x] +Results: [key#x, max#x] (6) Exchange Input: [key#x, max#x] (7) HashAggregate [codegen id : 4] Input: [key#x, max#x] +Keys: [key#x] +Functions: [max(val#x)] +Aggregate Attributes: [max(val#x)#x] +Results: [key#x, max(val#x)#x AS max(val)#x] (8) ReusedExchange [Reuses operator id: 6] Output : ArrayBuffer(key#x, max#x) (9) HashAggregate [codegen id : 3] Input: [key#x, max#x] +Keys: [key#x] +Functions: [max(val#x)] +Aggregate Attributes: [max(val#x)#x] +Results: [key#x, max(val#x)#x AS max(val)#x] (10) BroadcastExchange Input: [key#x, max(val)#x] @@ -786,6 +870,144 @@ Output: [] (4) Project +-- !query +EXPLAIN FORMATTED + SELECT + COUNT(val) + SUM(key) as TOTAL, + COUNT(key) FILTER (WHERE val > 1) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Physical Plan == +* HashAggregate (5) ++- Exchange (4) + +- HashAggregate (3) + +- * ColumnarToRow (2) + +- Scan parquet default.explain_temp1 (1) + + +(1) Scan parquet default.explain_temp1 +Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input: [key#x, val#x] + +(3) HashAggregate +Input: [key#x, val#x] +Keys: [] +Functions: [partial_count(val#x), partial_sum(cast(key#x as bigint)), partial_count(key#x) FILTER (WHERE (val#x > 1))] +Aggregate Attributes: [count#xL, sum#xL, count#xL] +Results: [count#xL, sum#xL, count#xL] + +(4) Exchange +Input: [count#xL, sum#xL, count#xL] + +(5) HashAggregate [codegen id : 2] +Input: [count#xL, sum#xL, count#xL] +Keys: [] +Functions: [count(val#x), sum(cast(key#x as bigint)), count(key#x)] +Aggregate Attributes: [count(val#x)#xL, sum(cast(key#x as bigint))#xL, count(key#x)#xL] +Results: [(count(val#x)#xL + sum(cast(key#x as bigint))#xL) AS TOTAL#xL, count(key#x)#xL AS count(key) FILTER (WHERE (val > 1))#xL] + + +-- !query +EXPLAIN FORMATTED + SELECT key, sort_array(collect_set(val))[0] + FROM explain_temp4 + GROUP BY key +-- !query schema +struct +-- !query output +== Physical Plan == +ObjectHashAggregate (5) ++- Exchange (4) + +- ObjectHashAggregate (3) + +- * ColumnarToRow (2) + +- Scan parquet default.explain_temp4 (1) + + +(1) Scan parquet default.explain_temp4 +Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp4] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input: [key#x, val#x] + +(3) ObjectHashAggregate +Input: [key#x, val#x] +Keys: [key#x] +Functions: [partial_collect_set(val#x, 0, 0)] +Aggregate Attributes: [buf#x] +Results: [key#x, buf#x] + +(4) Exchange +Input: [key#x, buf#x] + +(5) ObjectHashAggregate +Input: [key#x, buf#x] +Keys: [key#x] +Functions: [collect_set(val#x, 0, 0)] +Aggregate Attributes: [collect_set(val#x, 0, 0)#x] +Results: [key#x, sort_array(collect_set(val#x, 0, 0)#x, true)[0] AS sort_array(collect_set(val), true)[0]#x] + + +-- !query +EXPLAIN FORMATTED + SELECT key, MIN(val) + FROM explain_temp4 + GROUP BY key +-- !query schema +struct +-- !query output +== Physical Plan == +SortAggregate (7) ++- * Sort (6) + +- Exchange (5) + +- SortAggregate (4) + +- * Sort (3) + +- * ColumnarToRow (2) + +- Scan parquet default.explain_temp4 (1) + + +(1) Scan parquet default.explain_temp4 +Output: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp4] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input: [key#x, val#x] + +(3) Sort [codegen id : 1] +Input: [key#x, val#x] + +(4) SortAggregate +Input: [key#x, val#x] +Keys: [key#x] +Functions: [partial_min(val#x)] +Aggregate Attributes: [min#x] +Results: [key#x, min#x] + +(5) Exchange +Input: [key#x, min#x] + +(6) Sort [codegen id : 2] +Input: [key#x, min#x] + +(7) SortAggregate +Input: [key#x, min#x] +Keys: [key#x] +Functions: [min(val#x)] +Aggregate Attributes: [min(val#x)#x] +Results: [key#x, min(val#x)#x AS min(val)#x] + + -- !query DROP TABLE explain_temp1 -- !query schema From aa0d13683cdf9f38f04cc0e73dc8cf63eed29bf4 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 13 Feb 2020 02:31:48 +0800 Subject: [PATCH 1358/1387] [SPARK-30760][SQL] Port `millisToDays` and `daysToMillis` on Java 8 time API ### What changes were proposed in this pull request? In the PR, I propose to rewrite the `millisToDays` and `daysToMillis` of `DateTimeUtils` using Java 8 time API. I removed `getOffsetFromLocalMillis` from `DateTimeUtils` because it is a private methods, and is not used anymore in Spark SQL. ### Why are the changes needed? New implementation is based on Proleptic Gregorian calendar which has been already used by other date-time functions. This changes make `millisToDays` and `daysToMillis` consistent to rest Spark SQL API related to date & time operations. ### Does this PR introduce any user-facing change? Yes, this might effect behavior for old dates before 1582 year. ### How was this patch tested? By existing test suites `DateTimeUtilsSuite`, `DateFunctionsSuite`, DateExpressionsSuite`, `SQLQuerySuite` and `HiveResultSuite`. Closes #27494 from MaxGekk/millis-2-days-java8-api. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../expressions/datetimeExpressions.scala | 8 +-- .../sql/catalyst/util/DateTimeUtils.scala | 58 +++++-------------- .../catalyst/csv/UnivocityParserSuite.scala | 3 +- .../expressions/DateExpressionsSuite.scala | 19 +++--- .../catalyst/util/DateTimeUtilsSuite.scala | 34 ++++++----- .../spark/sql/execution/HiveResult.scala | 5 ++ .../sql-tests/results/postgreSQL/date.sql.out | 12 ++-- .../apache/spark/sql/SQLQueryTestSuite.scala | 1 + 8 files changed, 62 insertions(+), 78 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 1f4c8c041f8bf..cf91489d8e6b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -135,7 +135,7 @@ case class CurrentBatchTimestamp( def toLiteral: Literal = dataType match { case _: TimestampType => Literal(DateTimeUtils.fromJavaTimestamp(new Timestamp(timestampMs)), TimestampType) - case _: DateType => Literal(DateTimeUtils.millisToDays(timestampMs, timeZone), DateType) + case _: DateType => Literal(DateTimeUtils.millisToDays(timestampMs, zoneId), DateType) } } @@ -1332,14 +1332,14 @@ case class MonthsBetween( override def nullSafeEval(t1: Any, t2: Any, roundOff: Any): Any = { DateTimeUtils.monthsBetween( - t1.asInstanceOf[Long], t2.asInstanceOf[Long], roundOff.asInstanceOf[Boolean], timeZone) + t1.asInstanceOf[Long], t2.asInstanceOf[Long], roundOff.asInstanceOf[Boolean], zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (d1, d2, roundOff) => { - s"""$dtu.monthsBetween($d1, $d2, $roundOff, $tz)""" + s"""$dtu.monthsBetween($d1, $d2, $roundOff, $zid)""" }) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index eeae0674166bc..5976bcbb52fd7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -67,24 +67,22 @@ object DateTimeUtils { // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisUtc: Long): SQLDate = { - millisToDays(millisUtc, defaultTimeZone()) + millisToDays(millisUtc, defaultTimeZone().toZoneId) } - def millisToDays(millisUtc: Long, timeZone: TimeZone): SQLDate = { - // SPARK-6785: use Math.floorDiv so negative number of days (dates before 1970) - // will correctly work as input for function toJavaDate(Int) - val millisLocal = millisUtc + timeZone.getOffset(millisUtc) - Math.floorDiv(millisLocal, MILLIS_PER_DAY).toInt + def millisToDays(millisUtc: Long, zoneId: ZoneId): SQLDate = { + val instant = microsToInstant(Math.multiplyExact(millisUtc, MICROS_PER_MILLIS)) + localDateToDays(LocalDateTime.ofInstant(instant, zoneId).toLocalDate) } // reverse of millisToDays def daysToMillis(days: SQLDate): Long = { - daysToMillis(days, defaultTimeZone()) + daysToMillis(days, defaultTimeZone().toZoneId) } - def daysToMillis(days: SQLDate, timeZone: TimeZone): Long = { - val millisLocal = days.toLong * MILLIS_PER_DAY - millisLocal - getOffsetFromLocalMillis(millisLocal, timeZone) + def daysToMillis(days: SQLDate, zoneId: ZoneId): Long = { + val instant = daysToLocalDate(days).atStartOfDay(zoneId).toInstant + instantToMicros(instant) / MICROS_PER_MILLIS } // Converts Timestamp to string according to Hive TimestampWritable convention. @@ -589,11 +587,11 @@ object DateTimeUtils { time1: SQLTimestamp, time2: SQLTimestamp, roundOff: Boolean, - timeZone: TimeZone): Double = { + zoneId: ZoneId): Double = { val millis1 = MICROSECONDS.toMillis(time1) val millis2 = MICROSECONDS.toMillis(time2) - val date1 = millisToDays(millis1, timeZone) - val date2 = millisToDays(millis2, timeZone) + val date1 = millisToDays(millis1, zoneId) + val date2 = millisToDays(millis2, zoneId) val (year1, monthInYear1, dayInMonth1, daysToMonthEnd1) = splitDate(date1) val (year2, monthInYear2, dayInMonth2, daysToMonthEnd2) = splitDate(date2) @@ -607,8 +605,8 @@ object DateTimeUtils { } // using milliseconds can cause precision loss with more than 8 digits // we follow Hive's implementation which uses seconds - val secondsInDay1 = MILLISECONDS.toSeconds(millis1 - daysToMillis(date1, timeZone)) - val secondsInDay2 = MILLISECONDS.toSeconds(millis2 - daysToMillis(date2, timeZone)) + val secondsInDay1 = MILLISECONDS.toSeconds(millis1 - daysToMillis(date1, zoneId)) + val secondsInDay2 = MILLISECONDS.toSeconds(millis2 - daysToMillis(date2, zoneId)) val secondsDiff = (dayInMonth1 - dayInMonth2) * SECONDS_PER_DAY + secondsInDay1 - secondsInDay2 val secondsInMonth = DAYS.toSeconds(31) val diff = monthDiff + secondsDiff / secondsInMonth.toDouble @@ -737,8 +735,8 @@ object DateTimeUtils { millis += offset millis - millis % MILLIS_PER_DAY - offset case _ => // Try to truncate date levels - val dDays = millisToDays(millis, timeZone) - daysToMillis(truncDate(dDays, level), timeZone) + val dDays = millisToDays(millis, timeZone.toZoneId) + daysToMillis(truncDate(dDays, level), timeZone.toZoneId) } truncated * MICROS_PER_MILLIS } @@ -770,32 +768,6 @@ object DateTimeUtils { } } - /** - * Lookup the offset for given millis seconds since 1970-01-01 00:00:00 in given timezone. - * TODO: Improve handling of normalization differences. - * TODO: Replace with JSR-310 or similar system - see SPARK-16788 - */ - private[sql] def getOffsetFromLocalMillis(millisLocal: Long, tz: TimeZone): Long = { - var guess = tz.getRawOffset - // the actual offset should be calculated based on milliseconds in UTC - val offset = tz.getOffset(millisLocal - guess) - if (offset != guess) { - guess = tz.getOffset(millisLocal - offset) - if (guess != offset) { - // fallback to do the reverse lookup using java.time.LocalDateTime - // this should only happen near the start or end of DST - val localDate = LocalDate.ofEpochDay(MILLISECONDS.toDays(millisLocal)) - val localTime = LocalTime.ofNanoOfDay(MILLISECONDS.toNanos( - Math.floorMod(millisLocal, MILLIS_PER_DAY))) - val localDateTime = LocalDateTime.of(localDate, localTime) - val millisEpoch = localDateTime.atZone(tz.toZoneId).toInstant.toEpochMilli - - guess = (millisLocal - millisEpoch).toInt - } - } - guess - } - /** * Convert the timestamp `ts` from one timezone to another. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index 77a2ca7e4a828..536c76f042d23 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.csv import java.math.BigDecimal import java.text.{DecimalFormat, DecimalFormatSymbols} +import java.time.ZoneOffset import java.util.{Locale, TimeZone} import org.apache.commons.lang3.time.FastDateFormat @@ -137,7 +138,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { val expectedDate = format.parse(customDate).getTime val castedDate = parser.makeConverter("_1", DateType, nullable = true) .apply(customDate) - assert(castedDate == DateTimeUtils.millisToDays(expectedDate, TimeZone.getTimeZone("GMT"))) + assert(castedDate == DateTimeUtils.millisToDays(expectedDate, ZoneOffset.UTC)) val timestamp = "2015-01-01 00:00:00" timestampsOptions = new CSVOptions(Map( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index f04149ab7eb29..39b859af47ca9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -56,9 +56,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val ts = new Timestamp(toMillis(time)) test("datetime function current_date") { - val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT) + val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), ZoneOffset.UTC) val cd = CurrentDate(gmtId).eval(EmptyRow).asInstanceOf[Int] - val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT) + val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis(), ZoneOffset.UTC) assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1) val cdjst = CurrentDate(jstId).eval(EmptyRow).asInstanceOf[Int] @@ -499,7 +499,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // Valid range of DateType is [0001-01-01, 9999-12-31] val maxMonthInterval = 10000 * 12 checkEvaluation( - AddMonths(Literal(Date.valueOf("0001-01-01")), Literal(maxMonthInterval)), 2933261) + AddMonths(Literal(LocalDate.parse("0001-01-01")), Literal(maxMonthInterval)), + LocalDate.of(10001, 1, 1).toEpochDay.toInt) checkEvaluation( AddMonths(Literal(Date.valueOf("9999-12-31")), Literal(-1 * maxMonthInterval)), -719529) // Test evaluation results between Interpreted mode and Codegen mode @@ -788,7 +789,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), MILLISECONDS.toSeconds( - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), @@ -796,7 +797,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(UnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId))) val t1 = UnixTimestamp( CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] val t2 = UnixTimestamp( @@ -814,7 +815,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), MILLISECONDS.toSeconds( - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) } @@ -852,7 +853,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), MILLISECONDS.toSeconds( - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( ToUnixTimestamp( Literal(sdf2.format(new Timestamp(-1000000))), @@ -861,7 +862,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ToUnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz.toZoneId))) val t1 = ToUnixTimestamp( CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] val t2 = ToUnixTimestamp( @@ -876,7 +877,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(ToUnixTimestamp( Literal(date1), Literal.create(null, StringType), timeZoneId), MILLISECONDS.toSeconds( - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) + DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz.toZoneId))) checkEvaluation( ToUnixTimestamp( Literal("2015-07-24"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index cabcd3007d1c0..cd0594c775a47 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -86,9 +86,13 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { } test("SPARK-6785: java date conversion before and after epoch") { + def format(d: Date): String = { + TimestampFormatter("uuuu-MM-dd", defaultTimeZone().toZoneId) + .format(d.getTime * MICROS_PER_MILLIS) + } def checkFromToJavaDate(d1: Date): Unit = { val d2 = toJavaDate(fromJavaDate(d1)) - assert(d2.toString === d1.toString) + assert(format(d2) === format(d1)) } val df1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) @@ -413,22 +417,22 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { test("monthsBetween") { val date1 = date(1997, 2, 28, 10, 30, 0) var date2 = date(1996, 10, 30) - assert(monthsBetween(date1, date2, true, TimeZoneUTC) === 3.94959677) - assert(monthsBetween(date1, date2, false, TimeZoneUTC) === 3.9495967741935485) + assert(monthsBetween(date1, date2, true, ZoneOffset.UTC) === 3.94959677) + assert(monthsBetween(date1, date2, false, ZoneOffset.UTC) === 3.9495967741935485) Seq(true, false).foreach { roundOff => date2 = date(2000, 2, 28) - assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === -36) + assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36) date2 = date(2000, 2, 29) - assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === -36) + assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === -36) date2 = date(1996, 3, 31) - assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === 11) + assert(monthsBetween(date1, date2, roundOff, ZoneOffset.UTC) === 11) } val date3 = date(2000, 2, 28, 16, tz = TimeZonePST) val date4 = date(1997, 2, 28, 16, tz = TimeZonePST) - assert(monthsBetween(date3, date4, true, TimeZonePST) === 36.0) - assert(monthsBetween(date3, date4, true, TimeZoneGMT) === 35.90322581) - assert(monthsBetween(date3, date4, false, TimeZoneGMT) === 35.903225806451616) + assert(monthsBetween(date3, date4, true, TimeZonePST.toZoneId) === 36.0) + assert(monthsBetween(date3, date4, true, ZoneOffset.UTC) === 35.90322581) + assert(monthsBetween(date3, date4, false, ZoneOffset.UTC) === 35.903225806451616) } test("from UTC timestamp") { @@ -571,15 +575,15 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { test("daysToMillis and millisToDays") { val input = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, 16, tz = TimeZonePST)) - assert(millisToDays(input, TimeZonePST) === 16800) - assert(millisToDays(input, TimeZoneGMT) === 16801) - assert(millisToDays(-1 * MILLIS_PER_DAY + 1, TimeZoneGMT) == -1) + assert(millisToDays(input, TimeZonePST.toZoneId) === 16800) + assert(millisToDays(input, ZoneOffset.UTC) === 16801) + assert(millisToDays(-1 * MILLIS_PER_DAY + 1, ZoneOffset.UTC) == -1) var expected = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, tz = TimeZonePST)) - assert(daysToMillis(16800, TimeZonePST) === expected) + assert(daysToMillis(16800, TimeZonePST.toZoneId) === expected) expected = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, tz = TimeZoneGMT)) - assert(daysToMillis(16800, TimeZoneGMT) === expected) + assert(daysToMillis(16800, ZoneOffset.UTC) === expected) // There are some days are skipped entirely in some timezone, skip them here. val skipped_days = Map[String, Set[Int]]( @@ -594,7 +598,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { val skipped = skipped_days.getOrElse(tz.getID, Set.empty) (-20000 to 20000).foreach { d => if (!skipped.contains(d)) { - assert(millisToDays(daysToMillis(d, tz), tz) === d, + assert(millisToDays(daysToMillis(d, tz.toZoneId), tz.toZoneId) === d, s"Round trip of ${d} did not work in tz ${tz}") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index bbe47a63f4d61..5a2f16d8e1526 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} @@ -67,8 +68,12 @@ object HiveResult { case (null, _) => if (nested) "null" else "NULL" case (b, BooleanType) => b.toString case (d: Date, DateType) => dateFormatter.format(DateTimeUtils.fromJavaDate(d)) + case (ld: LocalDate, DateType) => + dateFormatter.format(DateTimeUtils.localDateToDays(ld)) case (t: Timestamp, TimestampType) => timestampFormatter.format(DateTimeUtils.fromJavaTimestamp(t)) + case (i: Instant, TimestampType) => + timestampFormatter.format(DateTimeUtils.instantToMicros(i)) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString case (n, _: NumericType) => n.toString diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out index fd5dc42632176..ed27317121623 100755 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -800,7 +800,7 @@ SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000') -- !query schema struct -- !query output -1001-01-01 00:07:02 +1001-01-01 00:00:00 -- !query @@ -808,7 +808,7 @@ SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20') -- !query schema struct -- !query output -1001-01-01 00:07:02 +1001-01-01 00:00:00 -- !query @@ -840,7 +840,7 @@ SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04') -- !query schema struct -- !query output -0001-01-01 00:07:02 +0001-01-01 00:00:00 -- !query @@ -848,7 +848,7 @@ SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')) -- !query schema struct -- !query output --0099-01-01 00:07:02 +-0099-01-01 00:00:00 -- !query @@ -864,7 +864,7 @@ SELECT DATE_TRUNC('DECADE', DATE '0004-12-25') -- !query schema struct -- !query output -0000-01-01 00:07:02 +0000-01-01 00:00:00 -- !query @@ -872,7 +872,7 @@ SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')) -- !query schema struct -- !query output --0010-01-01 00:07:02 +-0010-01-01 00:00:00 -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 2e5a9e0b4d45d..6b9e5bbd3c961 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -337,6 +337,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } + localSparkSession.conf.set(SQLConf.DATETIME_JAVA8API_ENABLED.key, true) if (configSet.nonEmpty) { // Execute the list of set operation in order to add the desired configs From 5b76367a9d0aaca53ce96ab7e555a596567e8335 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 12 Feb 2020 14:27:18 -0800 Subject: [PATCH 1359/1387] [SPARK-30797][SQL] Set tradition user/group/other permission to ACL entries when setting up ACLs in truncate table ### What changes were proposed in this pull request? This is a follow-up to the PR #26956. In #26956, the patch proposed to preserve path permission when truncating table. When setting up original ACLs, we need to set user/group/other permission as ACL entries too, otherwise if the path doesn't have default user/group/other ACL entries, ACL API will complain an error `Invalid ACL: the user, group and other entries are required.`. In short this change makes sure: 1. Permissions for user/group/other are always kept into ACLs to work with ACL API. 2. Other custom ACLs are still kept after TRUNCATE TABLE (#26956 did this). ### Why are the changes needed? Without this fix, `TRUNCATE TABLE` will get an error when setting up ACLs if there is no default default user/group/other ACL entries. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Update unit test. Manual test on dev Spark cluster. Set ACLs for a table path without default user/group/other ACL entries: ``` hdfs dfs -setfacl --set 'user:liangchi:rwx,user::rwx,group::r--,other::r--' /user/hive/warehouse/test.db/test_truncate_table hdfs dfs -getfacl /user/hive/warehouse/test.db/test_truncate_table # file: /user/hive/warehouse/test.db/test_truncate_table # owner: liangchi # group: supergroup user::rwx user:liangchi:rwx group::r-- mask::rwx other::r-- ``` Then run `sql("truncate table test.test_truncate_table")`, it works by normally truncating the table and preserve ACLs. Closes #27548 from viirya/fix-truncate-table-permission. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/command/tables.scala | 32 +++++++++++++++++-- .../sql/execution/command/DDLSuite.scala | 21 +++++++++++- 2 files changed, 49 insertions(+), 4 deletions(-) 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 90dbdf5515d4d..61500b773cd7e 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 @@ -19,12 +19,13 @@ package org.apache.spark.sql.execution.command import java.net.{URI, URISyntaxException} +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.util.Try import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileContext, FsConstants, Path} -import org.apache.hadoop.fs.permission.{AclEntry, FsPermission} +import org.apache.hadoop.fs.permission.{AclEntry, AclEntryScope, AclEntryType, FsAction, FsPermission} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier @@ -538,12 +539,27 @@ case class TruncateTableCommand( } } optAcls.foreach { acls => + val aclEntries = acls.asScala.filter(_.getName != null).asJava + + // If the path doesn't have default ACLs, `setAcl` API will throw an error + // as it expects user/group/other permissions must be in ACL entries. + // So we need to add tradition user/group/other permission + // in the form of ACL. + optPermission.map { permission => + aclEntries.add(newAclEntry(AclEntryScope.ACCESS, + AclEntryType.USER, permission.getUserAction())) + aclEntries.add(newAclEntry(AclEntryScope.ACCESS, + AclEntryType.GROUP, permission.getGroupAction())) + aclEntries.add(newAclEntry(AclEntryScope.ACCESS, + AclEntryType.OTHER, permission.getOtherAction())) + } + try { - fs.setAcl(path, acls) + fs.setAcl(path, aclEntries) } catch { case NonFatal(e) => throw new SecurityException( - s"Failed to set original ACL $acls back to " + + s"Failed to set original ACL $aclEntries back to " + s"the created path: $path. Exception: ${e.getMessage}") } } @@ -574,6 +590,16 @@ case class TruncateTableCommand( } Seq.empty[Row] } + + private def newAclEntry( + scope: AclEntryScope, + aclType: AclEntryType, + permission: FsAction): AclEntry = { + new AclEntry.Builder() + .setScope(scope) + .setType(aclType) + .setPermission(permission).build() + } } abstract class DescribeCommandBase extends RunnableCommand { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 31e00781ae6b4..dbf4b09403423 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2042,6 +2042,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { // Set ACL to table path. val customAcl = new java.util.ArrayList[AclEntry]() customAcl.add(new AclEntry.Builder() + .setName("test") .setType(AclEntryType.USER) .setScope(AclEntryScope.ACCESS) .setPermission(FsAction.READ).build()) @@ -2061,8 +2062,26 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { if (ignore) { assert(aclEntries.size() == 0) } else { - assert(aclEntries.size() == 1) + assert(aclEntries.size() == 4) assert(aclEntries.get(0) == customAcl.get(0)) + + // Setting ACLs will also set user/group/other permissions + // as ACL entries. + val user = new AclEntry.Builder() + .setType(AclEntryType.USER) + .setScope(AclEntryScope.ACCESS) + .setPermission(FsAction.ALL).build() + val group = new AclEntry.Builder() + .setType(AclEntryType.GROUP) + .setScope(AclEntryScope.ACCESS) + .setPermission(FsAction.ALL).build() + val other = new AclEntry.Builder() + .setType(AclEntryType.OTHER) + .setScope(AclEntryScope.ACCESS) + .setPermission(FsAction.ALL).build() + assert(aclEntries.get(1) == user) + assert(aclEntries.get(2) == group) + assert(aclEntries.get(3) == other) } } } From 496f6ac86001d284cbfb7488a63dd3a168919c0f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 12 Feb 2020 16:45:42 -0600 Subject: [PATCH 1360/1387] [SPARK-29148][CORE] Add stage level scheduling dynamic allocation and scheduler backend changes ### What changes were proposed in this pull request? This is another PR for stage level scheduling. In particular this adds changes to the dynamic allocation manager and the scheduler backend to be able to track what executors are needed per ResourceProfile. Note the api is still private to Spark until the entire feature gets in, so this functionality will be there but only usable by tests for profiles other then the DefaultProfile. The main changes here are simply tracking things on a ResourceProfile basis as well as sending the executor requests to the scheduler backend for all ResourceProfiles. I introduce a ResourceProfileManager in this PR that will track all the actual ResourceProfile objects so that we can keep them all in a single place and just pass around and use in datastructures the resource profile id. The resource profile id can be used with the ResourceProfileManager to get the actual ResourceProfile contents. There are various places in the code that use executor "slots" for things. The ResourceProfile adds functionality to keep that calculation in it. This logic is more complex then it should due to standalone mode and mesos coarse grained not setting the executor cores config. They default to all cores on the worker, so calculating slots is harder there. This PR keeps the functionality to make the cores the limiting resource because the scheduler still uses that for "slots" for a few things. This PR does also add the resource profile id to the Stage and stage info classes to be able to test things easier. That full set of changes will come with the scheduler PR that will be after this one. The PR stops at the scheduler backend pieces for the cluster manager and the real YARN support hasn't been added in this PR, that again will be in a separate PR, so this has a few of the API changes up to the cluster manager and then just uses the default profile requests to continue. The code for the entire feature is here for reference: https://github.com/apache/spark/pull/27053/files although it needs to be upmerged again as well. ### Why are the changes needed? Needed for stage level scheduling feature. ### Does this PR introduce any user-facing change? No user facing api changes added here. ### How was this patch tested? Lots of unit tests and manually testing. I tested on yarn, k8s, standalone, local modes. Ran both failure and success cases. Closes #27313 from tgravescs/SPARK-29148. Authored-by: Thomas Graves Signed-off-by: Thomas Graves --- .../spark/ExecutorAllocationClient.scala | 31 +- .../spark/ExecutorAllocationManager.scala | 473 +++++--- .../scala/org/apache/spark/SparkContext.scala | 150 +-- .../apache/spark/internal/config/Tests.scala | 9 + .../resource/ExecutorResourceRequests.scala | 2 +- .../spark/resource/ResourceProfile.scala | 150 ++- .../resource/ResourceProfileBuilder.scala | 2 +- .../resource/ResourceProfileManager.scala | 86 ++ .../apache/spark/resource/ResourceUtils.scala | 109 +- .../apache/spark/scheduler/DAGScheduler.scala | 9 +- .../apache/spark/scheduler/ResultStage.scala | 5 +- .../spark/scheduler/ShuffleMapStage.scala | 5 +- .../org/apache/spark/scheduler/Stage.scala | 9 +- .../apache/spark/scheduler/StageInfo.scala | 9 +- .../spark/scheduler/TaskSchedulerImpl.scala | 4 +- .../CoarseGrainedSchedulerBackend.scala | 150 +-- .../cluster/StandaloneSchedulerBackend.scala | 11 +- .../scheduler/dynalloc/ExecutorMonitor.scala | 11 +- .../org/apache/spark/util/JsonProtocol.scala | 8 +- .../ExecutorAllocationManagerSuite.scala | 1049 +++++++++++------ .../apache/spark/HeartbeatReceiverSuite.scala | 9 +- .../org/apache/spark/LocalSparkContext.scala | 2 +- .../org/apache/spark/SparkContextSuite.scala | 36 +- .../BasicEventFilterBuilderSuite.scala | 4 +- .../ResourceProfileManagerSuite.scala | 103 ++ .../spark/resource/ResourceProfileSuite.scala | 79 +- .../spark/resource/ResourceUtilsSuite.scala | 3 + .../CoarseGrainedSchedulerBackendSuite.scala | 13 +- .../scheduler/EventLoggingListenerSuite.scala | 7 +- .../dynalloc/ExecutorMonitorSuite.scala | 19 +- .../spark/status/AppStatusListenerSuite.scala | 76 +- .../status/ListenerEventsTestHelper.scala | 10 +- .../org/apache/spark/ui/StagePageSuite.scala | 4 +- .../apache/spark/util/JsonProtocolSuite.scala | 15 +- python/pyspark/tests/test_context.py | 5 + python/pyspark/tests/test_taskcontext.py | 6 + .../KubernetesClusterSchedulerBackend.scala | 8 +- ...bernetesClusterSchedulerBackendSuite.scala | 4 + .../MesosCoarseGrainedSchedulerBackend.scala | 18 +- ...osCoarseGrainedSchedulerBackendSuite.scala | 26 +- .../cluster/YarnSchedulerBackend.scala | 25 +- .../cluster/YarnSchedulerBackendSuite.scala | 7 +- .../ui/MetricsAggregationBenchmark.scala | 4 +- .../ui/SQLAppStatusListenerSuite.scala | 4 +- .../scheduler/ExecutorAllocationManager.scala | 7 +- .../ExecutorAllocationManagerSuite.scala | 19 +- 46 files changed, 1935 insertions(+), 860 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala create mode 100644 core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala index cb965cb180207..00bd0063c9e3a 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -37,24 +37,29 @@ private[spark] trait ExecutorAllocationClient { /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. - * @param numExecutors The total number of executors we'd like to have. The cluster manager - * shouldn't kill any running executor to reach this number, but, - * if all existing executors were to die, this is the number of executors - * we'd want to be allocated. - * @param localityAwareTasks The number of tasks in all active stages that have a locality - * preferences. This includes running, pending, and completed tasks. - * @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages - * that would like to like to run on that host. - * This includes running, pending, and completed tasks. + * + * @param resourceProfileIdToNumExecutors The total number of executors we'd like to have per + * ResourceProfile id. The cluster manager shouldn't kill + * any running executor to reach this number, but, if all + * existing executors were to die, this is the number + * of executors we'd want to be allocated. + * @param numLocalityAwareTasksPerResourceProfileId The number of tasks in all active stages that + * have a locality preferences per + * ResourceProfile id. This includes running, + * pending, and completed tasks. + * @param hostToLocalTaskCount A map of ResourceProfile id to a map of hosts to the number of + * tasks from all active stages that would like to like to run on + * that host. This includes running, pending, and completed tasks. * @return whether the request is acknowledged by the cluster manager. */ private[spark] def requestTotalExecutors( - numExecutors: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int]): Boolean + resourceProfileIdToNumExecutors: Map[Int, Int], + numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], + hostToLocalTaskCount: Map[Int, Map[String, Int]]): Boolean /** - * Request an additional number of executors from the cluster manager. + * Request an additional number of executors from the cluster manager for the default + * ResourceProfile. * @return whether the request is acknowledged by the cluster manager. */ def requestExecutors(numAdditionalExecutors: Int): Boolean diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 677386cc7a572..5cb3160711a90 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -29,6 +29,8 @@ import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.source.Source +import org.apache.spark.resource.ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID +import org.apache.spark.resource.ResourceProfileManager import org.apache.spark.scheduler._ import org.apache.spark.scheduler.dynalloc.ExecutorMonitor import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} @@ -36,9 +38,9 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} /** * An agent that dynamically allocates and removes executors based on the workload. * - * The ExecutorAllocationManager maintains a moving target number of executors which is periodically - * synced to the cluster manager. The target starts at a configured initial value and changes with - * the number of pending and running tasks. + * The ExecutorAllocationManager maintains a moving target number of executors, for each + * ResourceProfile, which is periodically synced to the cluster manager. The target starts + * at a configured initial value and changes with the number of pending and running tasks. * * Decreasing the target number of executors happens when the current target is more than needed to * handle the current load. The target number of executors is always truncated to the number of @@ -57,14 +59,18 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} * quickly over time in case the maximum number of executors is very high. Otherwise, it will take * a long time to ramp up under heavy workloads. * - * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not - * been scheduled to run any tasks, then it is removed. Note that an executor caching any data + * The remove policy is simpler and is applied on each ResourceProfile separately. If an executor + * for that ResourceProfile has been idle for K seconds and the number of executors is more + * then what is needed for that ResourceProfile, meaning there are not enough tasks that could use + * the executor, then it is removed. Note that an executor caching any data * blocks will be removed if it has been idle for more than L seconds. * * There is no retry logic in either case because we make the assumption that the cluster manager * will eventually fulfill all requests it receives asynchronously. * - * The relevant Spark properties include the following: + * The relevant Spark properties are below. Each of these properties applies separately to + * every ResourceProfile. So if you set a minimum number of executors, that is a minimum + * for each ResourceProfile. * * spark.dynamicAllocation.enabled - Whether this feature is enabled * spark.dynamicAllocation.minExecutors - Lower bound on the number of executors @@ -95,7 +101,8 @@ private[spark] class ExecutorAllocationManager( listenerBus: LiveListenerBus, conf: SparkConf, cleaner: Option[ContextCleaner] = None, - clock: Clock = new SystemClock()) + clock: Clock = new SystemClock(), + resourceProfileManager: ResourceProfileManager) extends Logging { allocationManager => @@ -117,23 +124,23 @@ private[spark] class ExecutorAllocationManager( // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.get(DYN_ALLOCATION_TESTING) - // TODO: The default value of 1 for spark.executor.cores works right now because dynamic - // allocation is only supported for YARN and the default number of cores per executor in YARN is - // 1, but it might need to be attained differently for different cluster managers - private val tasksPerExecutorForFullParallelism = - conf.get(EXECUTOR_CORES) / conf.get(CPUS_PER_TASK) - private val executorAllocationRatio = conf.get(DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO) + private val defaultProfileId = resourceProfileManager.defaultResourceProfile.id + validateSettings() - // Number of executors to add in the next round - private var numExecutorsToAdd = 1 + // Number of executors to add for each ResourceProfile in the next round + private val numExecutorsToAddPerResourceProfileId = new mutable.HashMap[Int, Int] + numExecutorsToAddPerResourceProfileId(defaultProfileId) = 1 // The desired number of executors at this moment in time. If all our executors were to die, this // is the number of executors we would immediately want from the cluster manager. - private var numExecutorsTarget = initialNumExecutors + // Note every profile will be allowed to have initial number, + // we may want to make this configurable per Profile in the future + private val numExecutorsTargetPerResourceProfileId = new mutable.HashMap[Int, Int] + numExecutorsTargetPerResourceProfileId(defaultProfileId) = initialNumExecutors // A timestamp of when an addition should be triggered, or NOT_SET if it is not set // This is set when pending tasks are added but not scheduled yet @@ -165,11 +172,12 @@ private[spark] class ExecutorAllocationManager( // (2) an executor idle timeout has elapsed. @volatile private var initializing: Boolean = true - // Number of locality aware tasks, used for executor placement. - private var localityAwareTasks = 0 + // Number of locality aware tasks for each ResourceProfile, used for executor placement. + private var numLocalityAwareTasksPerResourceProfileId = new mutable.HashMap[Int, Int] + numLocalityAwareTasksPerResourceProfileId(defaultProfileId) = 0 - // Host to possible task running on it, used for executor placement. - private var hostToLocalTaskCount: Map[String, Int] = Map.empty + // ResourceProfile id to Host to possible task running on it, used for executor placement. + private var rpIdToHostToLocalTaskCount: Map[Int, Map[String, Int]] = Map.empty /** * Verify that the settings specified through the config are valid. @@ -233,7 +241,14 @@ private[spark] class ExecutorAllocationManager( } executor.scheduleWithFixedDelay(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS) - client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) + // copy the maps inside synchonize to ensure not being modified + val (numExecutorsTarget, numLocalityAware) = synchronized { + val numTarget = numExecutorsTargetPerResourceProfileId.toMap + val numLocality = numLocalityAwareTasksPerResourceProfileId.toMap + (numTarget, numLocality) + } + + client.requestTotalExecutors(numExecutorsTarget, numLocalityAware, rpIdToHostToLocalTaskCount) } /** @@ -253,20 +268,28 @@ private[spark] class ExecutorAllocationManager( */ def reset(): Unit = synchronized { addTime = 0L - numExecutorsTarget = initialNumExecutors + numExecutorsTargetPerResourceProfileId.keys.foreach { rpId => + numExecutorsTargetPerResourceProfileId(rpId) = initialNumExecutors + } executorMonitor.reset() } /** - * The maximum number of executors we would need under the current load to satisfy all running - * and pending tasks, rounded up. + * The maximum number of executors, for the ResourceProfile id passed in, that we would need + * under the current load to satisfy all running and pending tasks, rounded up. */ - private def maxNumExecutorsNeeded(): Int = { - val numRunningOrPendingTasks = listener.totalPendingTasks + listener.totalRunningTasks + private def maxNumExecutorsNeededPerResourceProfile(rpId: Int): Int = { + val pending = listener.totalPendingTasksPerResourceProfile(rpId) + val pendingSpeculative = listener.pendingSpeculativeTasksPerResourceProfile(rpId) + val running = listener.totalRunningTasksPerResourceProfile(rpId) + val numRunningOrPendingTasks = pending + running + val rp = resourceProfileManager.resourceProfileFromId(rpId) + val tasksPerExecutor = rp.maxTasksPerExecutor(conf) + logDebug(s"max needed for rpId: $rpId numpending: $numRunningOrPendingTasks," + + s" tasksperexecutor: $tasksPerExecutor") val maxNeeded = math.ceil(numRunningOrPendingTasks * executorAllocationRatio / - tasksPerExecutorForFullParallelism).toInt - if (tasksPerExecutorForFullParallelism > 1 && maxNeeded == 1 && - listener.pendingSpeculativeTasks > 0) { + tasksPerExecutor).toInt + if (tasksPerExecutor > 1 && maxNeeded == 1 && pendingSpeculative > 0) { // If we have pending speculative tasks and only need a single executor, allocate one more // to satisfy the locality requirements of speculation maxNeeded + 1 @@ -275,8 +298,8 @@ private[spark] class ExecutorAllocationManager( } } - private def totalRunningTasks(): Int = synchronized { - listener.totalRunningTasks + private def totalRunningTasksPerResourceProfile(id: Int): Int = synchronized { + listener.totalRunningTasksPerResourceProfile(id) } /** @@ -302,7 +325,8 @@ private[spark] class ExecutorAllocationManager( } /** - * Updates our target number of executors and syncs the result with the cluster manager. + * Updates our target number of executors for each ResourceProfile and then syncs the result + * with the cluster manager. * * Check to see whether our existing allocation and the requests we've made previously exceed our * current needs. If so, truncate our target and let the cluster manager know so that it can @@ -314,130 +338,205 @@ private[spark] class ExecutorAllocationManager( * @return the delta in the target number of executors. */ private def updateAndSyncNumExecutorsTarget(now: Long): Int = synchronized { - val maxNeeded = maxNumExecutorsNeeded - if (initializing) { // Do not change our target while we are still initializing, // Otherwise the first job may have to ramp up unnecessarily 0 - } else if (maxNeeded < numExecutorsTarget) { - // The target number exceeds the number we actually need, so stop adding new - // executors and inform the cluster manager to cancel the extra pending requests - val oldNumExecutorsTarget = numExecutorsTarget - numExecutorsTarget = math.max(maxNeeded, minNumExecutors) - numExecutorsToAdd = 1 - - // If the new target has not changed, avoid sending a message to the cluster manager - if (numExecutorsTarget < oldNumExecutorsTarget) { - // We lower the target number of executors but don't actively kill any yet. Killing is - // controlled separately by an idle timeout. It's still helpful to reduce the target number - // in case an executor just happens to get lost (eg., bad hardware, or the cluster manager - // preempts it) -- in that case, there is no point in trying to immediately get a new - // executor, since we wouldn't even use it yet. - client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) - logDebug(s"Lowering target number of executors to $numExecutorsTarget (previously " + - s"$oldNumExecutorsTarget) because not all requested executors are actually needed") + } else { + val updatesNeeded = new mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates] + + // Update targets for all ResourceProfiles then do a single request to the cluster manager + numExecutorsTargetPerResourceProfileId.foreach { case (rpId, targetExecs) => + val maxNeeded = maxNumExecutorsNeededPerResourceProfile(rpId) + if (maxNeeded < targetExecs) { + // The target number exceeds the number we actually need, so stop adding new + // executors and inform the cluster manager to cancel the extra pending requests + + // We lower the target number of executors but don't actively kill any yet. Killing is + // controlled separately by an idle timeout. It's still helpful to reduce + // the target number in case an executor just happens to get lost (eg., bad hardware, + // or the cluster manager preempts it) -- in that case, there is no point in trying + // to immediately get a new executor, since we wouldn't even use it yet. + decrementExecutorsFromTarget(maxNeeded, rpId, updatesNeeded) + } else if (addTime != NOT_SET && now >= addTime) { + addExecutorsToTarget(maxNeeded, rpId, updatesNeeded) + } + } + doUpdateRequest(updatesNeeded.toMap, now) + } + } + + private def addExecutorsToTarget( + maxNeeded: Int, + rpId: Int, + updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]): Int = { + updateTargetExecs(addExecutors, maxNeeded, rpId, updatesNeeded) + } + + private def decrementExecutorsFromTarget( + maxNeeded: Int, + rpId: Int, + updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]): Int = { + updateTargetExecs(decrementExecutors, maxNeeded, rpId, updatesNeeded) + } + + private def updateTargetExecs( + updateTargetFn: (Int, Int) => Int, + maxNeeded: Int, + rpId: Int, + updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]): Int = { + val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId) + // update the target number (add or remove) + val delta = updateTargetFn(maxNeeded, rpId) + if (delta != 0) { + updatesNeeded(rpId) = ExecutorAllocationManager.TargetNumUpdates(delta, oldNumExecutorsTarget) + } + delta + } + + private def doUpdateRequest( + updates: Map[Int, ExecutorAllocationManager.TargetNumUpdates], + now: Long): Int = { + // Only call cluster manager if target has changed. + if (updates.size > 0) { + val requestAcknowledged = try { + logDebug("requesting updates: " + updates) + testing || + client.requestTotalExecutors( + numExecutorsTargetPerResourceProfileId.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, + rpIdToHostToLocalTaskCount) + } catch { + case NonFatal(e) => + // Use INFO level so the error it doesn't show up by default in shells. + // Errors here are more commonly caused by YARN AM restarts, which is a recoverable + // issue, and generate a lot of noisy output. + logInfo("Error reaching cluster manager.", e) + false + } + if (requestAcknowledged) { + // have to go through all resource profiles that changed + var totalDelta = 0 + updates.foreach { case (rpId, targetNum) => + val delta = targetNum.delta + totalDelta += delta + if (delta > 0) { + val executorsString = "executor" + { if (delta > 1) "s" else "" } + logInfo(s"Requesting $delta new $executorsString because tasks are backlogged " + + s"(new desired total will be ${numExecutorsTargetPerResourceProfileId(rpId)} " + + s"for resource profile id: ${rpId})") + numExecutorsToAddPerResourceProfileId(rpId) = + if (delta == numExecutorsToAddPerResourceProfileId(rpId)) { + numExecutorsToAddPerResourceProfileId(rpId) * 2 + } else { + 1 + } + logDebug(s"Starting timer to add more executors (to " + + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") + addTime = now + TimeUnit.SECONDS.toNanos(sustainedSchedulerBacklogTimeoutS) + } else { + logDebug(s"Lowering target number of executors to" + + s" ${numExecutorsTargetPerResourceProfileId(rpId)} (previously " + + s"$targetNum.oldNumExecutorsTarget for resource profile id: ${rpId}) " + + "because not all requested executors " + + "are actually needed") + } + } + totalDelta + } else { + // request was for all profiles so we have to go through all to reset to old num + updates.foreach { case (rpId, targetNum) => + logWarning("Unable to reach the cluster manager to request more executors!") + numExecutorsTargetPerResourceProfileId(rpId) = targetNum.oldNumExecutorsTarget + } + 0 } - numExecutorsTarget - oldNumExecutorsTarget - } else if (addTime != NOT_SET && now >= addTime) { - val delta = addExecutors(maxNeeded) - logDebug(s"Starting timer to add more executors (to " + - s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") - addTime = now + TimeUnit.SECONDS.toNanos(sustainedSchedulerBacklogTimeoutS) - delta } else { + logDebug("No change in number of executors") 0 } } + private def decrementExecutors(maxNeeded: Int, rpId: Int): Int = { + val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId) + numExecutorsTargetPerResourceProfileId(rpId) = math.max(maxNeeded, minNumExecutors) + numExecutorsToAddPerResourceProfileId(rpId) = 1 + numExecutorsTargetPerResourceProfileId(rpId) - oldNumExecutorsTarget + } + /** - * Request a number of executors from the cluster manager. + * Update the target number of executors and figure out how many to add. * If the cap on the number of executors is reached, give up and reset the * number of executors to add next round instead of continuing to double it. * * @param maxNumExecutorsNeeded the maximum number of executors all currently running or pending * tasks could fill + * @param rpId the ResourceProfile id of the executors * @return the number of additional executors actually requested. */ - private def addExecutors(maxNumExecutorsNeeded: Int): Int = { + private def addExecutors(maxNumExecutorsNeeded: Int, rpId: Int): Int = { + val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId) // Do not request more executors if it would put our target over the upper bound - if (numExecutorsTarget >= maxNumExecutors) { - logDebug(s"Not adding executors because our current target total " + - s"is already $numExecutorsTarget (limit $maxNumExecutors)") - numExecutorsToAdd = 1 + // this is doing a max check per ResourceProfile + if (oldNumExecutorsTarget >= maxNumExecutors) { + logDebug("Not adding executors because our current target total " + + s"is already ${oldNumExecutorsTarget} (limit $maxNumExecutors)") + numExecutorsToAddPerResourceProfileId(rpId) = 1 return 0 } - - val oldNumExecutorsTarget = numExecutorsTarget // There's no point in wasting time ramping up to the number of executors we already have, so // make sure our target is at least as much as our current allocation: - numExecutorsTarget = math.max(numExecutorsTarget, executorMonitor.executorCount) + var numExecutorsTarget = math.max(numExecutorsTargetPerResourceProfileId(rpId), + executorMonitor.executorCountWithResourceProfile(rpId)) // Boost our target with the number to add for this round: - numExecutorsTarget += numExecutorsToAdd + numExecutorsTarget += numExecutorsToAddPerResourceProfileId(rpId) // Ensure that our target doesn't exceed what we need at the present moment: numExecutorsTarget = math.min(numExecutorsTarget, maxNumExecutorsNeeded) // Ensure that our target fits within configured bounds: numExecutorsTarget = math.max(math.min(numExecutorsTarget, maxNumExecutors), minNumExecutors) - val delta = numExecutorsTarget - oldNumExecutorsTarget + numExecutorsTargetPerResourceProfileId(rpId) = numExecutorsTarget // If our target has not changed, do not send a message // to the cluster manager and reset our exponential growth if (delta == 0) { - numExecutorsToAdd = 1 - return 0 - } - - val addRequestAcknowledged = try { - testing || - client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) - } catch { - case NonFatal(e) => - // Use INFO level so the error it doesn't show up by default in shells. Errors here are more - // commonly caused by YARN AM restarts, which is a recoverable issue, and generate a lot of - // noisy output. - logInfo("Error reaching cluster manager.", e) - false - } - if (addRequestAcknowledged) { - val executorsString = "executor" + { if (delta > 1) "s" else "" } - logInfo(s"Requesting $delta new $executorsString because tasks are backlogged" + - s" (new desired total will be $numExecutorsTarget)") - numExecutorsToAdd = if (delta == numExecutorsToAdd) { - numExecutorsToAdd * 2 - } else { - 1 - } - delta - } else { - logWarning( - s"Unable to reach the cluster manager to request $numExecutorsTarget total executors!") - numExecutorsTarget = oldNumExecutorsTarget - 0 + numExecutorsToAddPerResourceProfileId(rpId) = 1 } + delta } /** * Request the cluster manager to remove the given executors. * Returns the list of executors which are removed. */ - private def removeExecutors(executors: Seq[String]): Seq[String] = synchronized { + private def removeExecutors(executors: Seq[(String, Int)]): Seq[String] = synchronized { val executorIdsToBeRemoved = new ArrayBuffer[String] - logDebug(s"Request to remove executorIds: ${executors.mkString(", ")}") - val numExistingExecutors = executorMonitor.executorCount - executorMonitor.pendingRemovalCount - - var newExecutorTotal = numExistingExecutors - executors.foreach { executorIdToBeRemoved => - if (newExecutorTotal - 1 < minNumExecutors) { - logDebug(s"Not removing idle executor $executorIdToBeRemoved because there are only " + - s"$newExecutorTotal executor(s) left (minimum number of executor limit $minNumExecutors)") - } else if (newExecutorTotal - 1 < numExecutorsTarget) { - logDebug(s"Not removing idle executor $executorIdToBeRemoved because there are only " + - s"$newExecutorTotal executor(s) left (number of executor target $numExecutorsTarget)") + val numExecutorsTotalPerRpId = mutable.Map[Int, Int]() + executors.foreach { case (executorIdToBeRemoved, rpId) => + if (rpId == UNKNOWN_RESOURCE_PROFILE_ID) { + if (testing) { + throw new SparkException("ResourceProfile Id was UNKNOWN, this is not expected") + } + logWarning(s"Not removing executor $executorIdsToBeRemoved because the " + + "ResourceProfile was UNKNOWN!") } else { - executorIdsToBeRemoved += executorIdToBeRemoved - newExecutorTotal -= 1 + // get the running total as we remove or initialize it to the count - pendingRemoval + val newExecutorTotal = numExecutorsTotalPerRpId.getOrElseUpdate(rpId, + (executorMonitor.executorCountWithResourceProfile(rpId) - + executorMonitor.pendingRemovalCountPerResourceProfileId(rpId))) + if (newExecutorTotal - 1 < minNumExecutors) { + logDebug(s"Not removing idle executor $executorIdToBeRemoved because there " + + s"are only $newExecutorTotal executor(s) left (minimum number of executor limit " + + s"$minNumExecutors)") + } else if (newExecutorTotal - 1 < numExecutorsTargetPerResourceProfileId(rpId)) { + logDebug(s"Not removing idle executor $executorIdToBeRemoved because there " + + s"are only $newExecutorTotal executor(s) left (number of executor " + + s"target ${numExecutorsTargetPerResourceProfileId(rpId)})") + } else { + executorIdsToBeRemoved += executorIdToBeRemoved + numExecutorsTotalPerRpId(rpId) -= 1 + } } } @@ -457,14 +556,15 @@ private[spark] class ExecutorAllocationManager( // [SPARK-21834] killExecutors api reduces the target number of executors. // So we need to update the target with desired value. - client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) + client.requestTotalExecutors( + numExecutorsTargetPerResourceProfileId.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, + rpIdToHostToLocalTaskCount) + // reset the newExecutorTotal to the existing number of executors - newExecutorTotal = numExistingExecutors if (testing || executorsRemoved.nonEmpty) { - newExecutorTotal -= executorsRemoved.size executorMonitor.executorsKilled(executorsRemoved) - logInfo(s"Executors ${executorsRemoved.mkString(",")} removed due to idle timeout." + - s"(new desired total will be $newExecutorTotal)") + logInfo(s"Executors ${executorsRemoved.mkString(",")} removed due to idle timeout.") executorsRemoved } else { logWarning(s"Unable to reach the cluster manager to kill executor/s " + @@ -493,7 +593,7 @@ private[spark] class ExecutorAllocationManager( private def onSchedulerQueueEmpty(): Unit = synchronized { logDebug("Clearing timer to add executors because there are no more pending tasks") addTime = NOT_SET - numExecutorsToAdd = 1 + numExecutorsToAddPerResourceProfileId.transform { case (_, _) => 1 } } private case class StageAttempt(stageId: Int, stageAttemptId: Int) { @@ -519,12 +619,16 @@ private[spark] class ExecutorAllocationManager( private val stageAttemptToSpeculativeTaskIndices = new mutable.HashMap[StageAttempt, mutable.HashSet[Int]] + private val resourceProfileIdToStageAttempt = + new mutable.HashMap[Int, mutable.Set[StageAttempt]] + // stageAttempt to tuple (the number of task with locality preferences, a map where each pair - // is a node and the number of tasks that would like to be scheduled on that node) map, + // is a node and the number of tasks that would like to be scheduled on that node, and + // the resource profile id) map, // maintain the executor placement hints for each stageAttempt used by resource framework // to better place the executors. private val stageAttemptToExecutorPlacementHints = - new mutable.HashMap[StageAttempt, (Int, Map[String, Int])] + new mutable.HashMap[StageAttempt, (Int, Map[String, Int], Int)] override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { initializing = false @@ -535,6 +639,13 @@ private[spark] class ExecutorAllocationManager( allocationManager.synchronized { stageAttemptToNumTasks(stageAttempt) = numTasks allocationManager.onSchedulerBacklogged() + // need to keep stage task requirements to ask for the right containers + val profId = stageSubmitted.stageInfo.resourceProfileId + logDebug(s"Stage resource profile id is: $profId with numTasks: $numTasks") + resourceProfileIdToStageAttempt.getOrElseUpdate( + profId, new mutable.HashSet[StageAttempt]) += stageAttempt + numExecutorsToAddPerResourceProfileId.getOrElseUpdate(profId, 1) + numExecutorsTargetPerResourceProfileId.getOrElseUpdate(profId, initialNumExecutors) // Compute the number of tasks requested by the stage on each host var numTasksPending = 0 @@ -549,7 +660,7 @@ private[spark] class ExecutorAllocationManager( } } stageAttemptToExecutorPlacementHints.put(stageAttempt, - (numTasksPending, hostToLocalTaskCountPerStage.toMap)) + (numTasksPending, hostToLocalTaskCountPerStage.toMap, profId)) // Update the executor placement hints updateExecutorPlacementHints() @@ -561,7 +672,7 @@ private[spark] class ExecutorAllocationManager( val stageAttemptId = stageCompleted.stageInfo.attemptNumber() val stageAttempt = StageAttempt(stageId, stageAttemptId) allocationManager.synchronized { - // do NOT remove stageAttempt from stageAttemptToNumRunningTasks, + // do NOT remove stageAttempt from stageAttemptToNumRunningTask // because the attempt may still have running tasks, // even after another attempt for the stage is submitted. stageAttemptToNumTasks -= stageAttempt @@ -597,7 +708,7 @@ private[spark] class ExecutorAllocationManager( stageAttemptToTaskIndices.getOrElseUpdate(stageAttempt, new mutable.HashSet[Int]) += taskIndex } - if (totalPendingTasks() == 0) { + if (!hasPendingTasks) { allocationManager.onSchedulerQueueEmpty() } } @@ -613,9 +724,22 @@ private[spark] class ExecutorAllocationManager( stageAttemptToNumRunningTask(stageAttempt) -= 1 if (stageAttemptToNumRunningTask(stageAttempt) == 0) { stageAttemptToNumRunningTask -= stageAttempt + if (!stageAttemptToNumTasks.contains(stageAttempt)) { + val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) => + v.contains(stageAttempt) + }.keys + if (rpForStage.size == 1) { + // be careful about the removal from here due to late tasks, make sure stage is + // really complete and no tasks left + resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt + } else { + logWarning(s"Should have exactly one resource profile for stage $stageAttempt," + + s" but have $rpForStage") + } + } + } } - if (taskEnd.taskInfo.speculative) { stageAttemptToSpeculativeTaskIndices.get(stageAttempt).foreach {_.remove{taskIndex}} stageAttemptToNumSpeculativeTasks(stageAttempt) -= 1 @@ -624,7 +748,7 @@ private[spark] class ExecutorAllocationManager( taskEnd.reason match { case Success | _: TaskKilled => case _ => - if (totalPendingTasks() == 0) { + if (!hasPendingTasks) { // If the task failed (not intentionally killed), we expect it to be resubmitted // later. To ensure we have enough resources to run the resubmitted task, we need to // mark the scheduler as backlogged again if it's not already marked as such @@ -661,20 +785,46 @@ private[spark] class ExecutorAllocationManager( * * Note: This is not thread-safe without the caller owning the `allocationManager` lock. */ - def pendingTasks(): Int = { - stageAttemptToNumTasks.map { case (stageAttempt, numTasks) => - numTasks - stageAttemptToTaskIndices.get(stageAttempt).map(_.size).getOrElse(0) - }.sum + def pendingTasksPerResourceProfile(rpId: Int): Int = { + val attempts = resourceProfileIdToStageAttempt.getOrElse(rpId, Set.empty).toSeq + attempts.map(attempt => getPendingTaskSum(attempt)).sum } - def pendingSpeculativeTasks(): Int = { - stageAttemptToNumSpeculativeTasks.map { case (stageAttempt, numTasks) => - numTasks - stageAttemptToSpeculativeTaskIndices.get(stageAttempt).map(_.size).getOrElse(0) - }.sum + def hasPendingRegularTasks: Boolean = { + val attemptSets = resourceProfileIdToStageAttempt.values + attemptSets.exists(attempts => attempts.exists(getPendingTaskSum(_) > 0)) + } + + private def getPendingTaskSum(attempt: StageAttempt): Int = { + val numTotalTasks = stageAttemptToNumTasks.getOrElse(attempt, 0) + val numRunning = stageAttemptToTaskIndices.get(attempt).map(_.size).getOrElse(0) + numTotalTasks - numRunning } - def totalPendingTasks(): Int = { - pendingTasks + pendingSpeculativeTasks + def pendingSpeculativeTasksPerResourceProfile(rp: Int): Int = { + val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq + attempts.map(attempt => getPendingSpeculativeTaskSum(attempt)).sum + } + + def hasPendingSpeculativeTasks: Boolean = { + val attemptSets = resourceProfileIdToStageAttempt.values + attemptSets.exists { attempts => + attempts.exists(getPendingSpeculativeTaskSum(_) > 0) + } + } + + private def getPendingSpeculativeTaskSum(attempt: StageAttempt): Int = { + val numTotalTasks = stageAttemptToNumSpeculativeTasks.getOrElse(attempt, 0) + val numRunning = stageAttemptToSpeculativeTaskIndices.get(attempt).map(_.size).getOrElse(0) + numTotalTasks - numRunning + } + + def hasPendingTasks: Boolean = { + hasPendingSpeculativeTasks || hasPendingRegularTasks + } + + def totalPendingTasksPerResourceProfile(rp: Int): Int = { + pendingTasksPerResourceProfile(rp) + pendingSpeculativeTasksPerResourceProfile(rp) } /** @@ -685,6 +835,14 @@ private[spark] class ExecutorAllocationManager( stageAttemptToNumRunningTask.values.sum } + def totalRunningTasksPerResourceProfile(rp: Int): Int = { + val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq + // attempts is a Set, change to Seq so we keep all values + attempts.map { attempt => + stageAttemptToNumRunningTask.getOrElseUpdate(attempt, 0) + }.sum + } + /** * Update the Executor placement hints (the number of tasks with locality preferences, * a map where each pair is a node and the number of tasks that would like to be scheduled @@ -694,18 +852,27 @@ private[spark] class ExecutorAllocationManager( * granularity within stages. */ def updateExecutorPlacementHints(): Unit = { - var localityAwareTasks = 0 - val localityToCount = new mutable.HashMap[String, Int]() - stageAttemptToExecutorPlacementHints.values.foreach { case (numTasksPending, localities) => - localityAwareTasks += numTasksPending - localities.foreach { case (hostname, count) => - val updatedCount = localityToCount.getOrElse(hostname, 0) + count - localityToCount(hostname) = updatedCount - } + val localityAwareTasksPerResourceProfileId = new mutable.HashMap[Int, Int] + + // ResourceProfile id => map[host, count] + val rplocalityToCount = new mutable.HashMap[Int, mutable.HashMap[String, Int]]() + stageAttemptToExecutorPlacementHints.values.foreach { + case (numTasksPending, localities, rpId) => + val rpNumPending = + localityAwareTasksPerResourceProfileId.getOrElse(rpId, 0) + localityAwareTasksPerResourceProfileId(rpId) = rpNumPending + numTasksPending + localities.foreach { case (hostname, count) => + val rpBasedHostToCount = + rplocalityToCount.getOrElseUpdate(rpId, new mutable.HashMap[String, Int]) + val newUpdated = rpBasedHostToCount.getOrElse(hostname, 0) + count + rpBasedHostToCount(hostname) = newUpdated + } } - allocationManager.localityAwareTasks = localityAwareTasks - allocationManager.hostToLocalTaskCount = localityToCount.toMap + allocationManager.numLocalityAwareTasksPerResourceProfileId = + localityAwareTasksPerResourceProfileId + allocationManager.rpIdToHostToLocalTaskCount = + rplocalityToCount.map { case (k, v) => (k, v.toMap)}.toMap } } @@ -726,14 +893,22 @@ private[spark] class ExecutorAllocationManager( }) } - registerGauge("numberExecutorsToAdd", numExecutorsToAdd, 0) + // The metrics are going to return the sum for all the different ResourceProfiles. + registerGauge("numberExecutorsToAdd", + numExecutorsToAddPerResourceProfileId.values.sum, 0) registerGauge("numberExecutorsPendingToRemove", executorMonitor.pendingRemovalCount, 0) registerGauge("numberAllExecutors", executorMonitor.executorCount, 0) - registerGauge("numberTargetExecutors", numExecutorsTarget, 0) - registerGauge("numberMaxNeededExecutors", maxNumExecutorsNeeded(), 0) + registerGauge("numberTargetExecutors", + numExecutorsTargetPerResourceProfileId.values.sum, 0) + registerGauge("numberMaxNeededExecutors", numExecutorsTargetPerResourceProfileId.keys + .map(maxNumExecutorsNeededPerResourceProfile(_)).sum, 0) } } private object ExecutorAllocationManager { val NOT_SET = Long.MaxValue + + // helper case class for requesting executors, here to be visible for testing + private[spark] case class TargetNumUpdates(delta: Int, oldNumExecutorsTarget: Int) + } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 91188d58f4201..a47136ea36736 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReferenc import scala.collection.JavaConverters._ import scala.collection.Map +import scala.collection.immutable import scala.collection.mutable.HashMap import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} @@ -53,7 +54,7 @@ import org.apache.spark.io.CompressionCodec import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ -import org.apache.spark.resource.{ResourceID, ResourceInformation} +import org.apache.spark.resource._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ @@ -219,9 +220,10 @@ class SparkContext(config: SparkConf) extends Logging { private var _shutdownHookRef: AnyRef = _ private var _statusStore: AppStatusStore = _ private var _heartbeater: Heartbeater = _ - private var _resources: scala.collection.immutable.Map[String, ResourceInformation] = _ + private var _resources: immutable.Map[String, ResourceInformation] = _ private var _shuffleDriverComponents: ShuffleDriverComponents = _ private var _plugins: Option[PluginContainer] = None + private var _resourceProfileManager: ResourceProfileManager = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -343,6 +345,8 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] def executorAllocationManager: Option[ExecutorAllocationManager] = _executorAllocationManager + private[spark] def resourceProfileManager: ResourceProfileManager = _resourceProfileManager + private[spark] def cleaner: Option[ContextCleaner] = _cleaner private[spark] var checkpointDir: Option[String] = None @@ -451,6 +455,7 @@ class SparkContext(config: SparkConf) extends Logging { } _listenerBus = new LiveListenerBus(_conf) + _resourceProfileManager = new ResourceProfileManager(_conf) // Initialize the app status store and listener before SparkEnv is created so that it gets // all events. @@ -611,7 +616,7 @@ class SparkContext(config: SparkConf) extends Logging { case b: ExecutorAllocationClient => Some(new ExecutorAllocationManager( schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf, - cleaner = cleaner)) + cleaner = cleaner, resourceProfileManager = resourceProfileManager)) case _ => None } @@ -1622,7 +1627,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Update the cluster manager on our scheduling needs. Three bits of information are included - * to help it make decisions. + * to help it make decisions. This applies to the default ResourceProfile. * @param numExecutors The total number of executors we'd like to have. The cluster manager * shouldn't kill any running executor to reach this number, but, * if all existing executors were to die, this is the number of executors @@ -1638,11 +1643,16 @@ class SparkContext(config: SparkConf) extends Logging { def requestTotalExecutors( numExecutors: Int, localityAwareTasks: Int, - hostToLocalTaskCount: scala.collection.immutable.Map[String, Int] + hostToLocalTaskCount: immutable.Map[String, Int] ): Boolean = { schedulerBackend match { case b: ExecutorAllocationClient => - b.requestTotalExecutors(numExecutors, localityAwareTasks, hostToLocalTaskCount) + // this is being applied to the default resource profile, would need to add api to support + // others + val defaultProfId = resourceProfileManager.defaultResourceProfile.id + b.requestTotalExecutors(immutable.Map(defaultProfId-> numExecutors), + immutable.Map(localityAwareTasks -> defaultProfId), + immutable.Map(defaultProfId -> hostToLocalTaskCount)) case _ => logWarning("Requesting executors is not supported by current scheduler.") false @@ -2036,6 +2046,7 @@ class SparkContext(config: SparkConf) extends Logging { // Clear this `InheritableThreadLocal`, or it will still be inherited in child threads even this // `SparkContext` is stopped. localProperties.remove() + ResourceProfile.clearDefaultProfile() // Unset YARN mode system env variable, to allow switching between cluster types. SparkContext.clearActiveContext() logInfo("Successfully stopped SparkContext") @@ -2771,109 +2782,34 @@ object SparkContext extends Logging { // When running locally, don't try to re-execute tasks on failure. val MAX_LOCAL_TASK_FAILURES = 1 - // Ensure that executor's resources satisfies one or more tasks requirement. - def checkResourcesPerTask(clusterMode: Boolean, executorCores: Option[Int]): Unit = { + // Ensure that default executor's resources satisfies one or more tasks requirement. + // This function is for cluster managers that don't set the executor cores config, for + // others its checked in ResourceProfile. + def checkResourcesPerTask(executorCores: Int): Unit = { val taskCores = sc.conf.get(CPUS_PER_TASK) - val execCores = if (clusterMode) { - executorCores.getOrElse(sc.conf.get(EXECUTOR_CORES)) - } else { - executorCores.get - } - // some cluster managers don't set the EXECUTOR_CORES config by default (standalone - // and mesos coarse grained), so we can't rely on that config for those. - val shouldCheckExecCores = executorCores.isDefined || sc.conf.contains(EXECUTOR_CORES) || - (master.equalsIgnoreCase("yarn") || master.startsWith("k8s")) - - // Number of cores per executor must meet at least one task requirement. - if (shouldCheckExecCores && execCores < taskCores) { - throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " + - s"the task config: ${CPUS_PER_TASK.key} = $taskCores when run on $master.") - } - - // Calculate the max slots each executor can provide based on resources available on each - // executor and resources required by each task. - val taskResourceRequirements = parseResourceRequirements(sc.conf, SPARK_TASK_PREFIX) - val executorResourcesAndAmounts = parseAllResourceRequests(sc.conf, SPARK_EXECUTOR_PREFIX) - .map(request => (request.id.resourceName, request.amount)).toMap - - var (numSlots, limitingResourceName) = if (shouldCheckExecCores) { - (execCores / taskCores, "CPU") - } else { - (-1, "") - } - - taskResourceRequirements.foreach { taskReq => - // Make sure the executor resources were specified through config. - val execAmount = executorResourcesAndAmounts.getOrElse(taskReq.resourceName, - throw new SparkException("The executor resource config: " + - new ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + - " needs to be specified since a task requirement config: " + - new ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + - " was specified") - ) - // Make sure the executor resources are large enough to launch at least one task. - if (execAmount < taskReq.amount) { - throw new SparkException("The executor resource config: " + - new ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + - s" = $execAmount has to be >= the requested amount in task resource config: " + - new ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + - s" = ${taskReq.amount}") - } - // Compare and update the max slots each executor can provide. - // If the configured amount per task was < 1.0, a task is subdividing - // executor resources. If the amount per task was > 1.0, the task wants - // multiple executor resources. - val resourceNumSlots = Math.floor(execAmount * taskReq.numParts / taskReq.amount).toInt - if (resourceNumSlots < numSlots) { - if (shouldCheckExecCores) { - throw new IllegalArgumentException("The number of slots on an executor has to be " + - "limited by the number of cores, otherwise you waste resources and " + - "dynamic allocation doesn't work properly. Your configuration has " + - s"core/task cpu slots = ${numSlots} and " + - s"${taskReq.resourceName} = ${resourceNumSlots}. " + - "Please adjust your configuration so that all resources require same number " + - "of executor slots.") - } - numSlots = resourceNumSlots - limitingResourceName = taskReq.resourceName - } - } - if(!shouldCheckExecCores && Utils.isDynamicAllocationEnabled(sc.conf)) { - // if we can't rely on the executor cores config throw a warning for user - logWarning("Please ensure that the number of slots available on your " + - "executors is limited by the number of cores to task cpus and not another " + - "custom resource. If cores is not the limiting resource then dynamic " + - "allocation will not work properly!") - } - // warn if we would waste any resources due to another resource limiting the number of - // slots on an executor - taskResourceRequirements.foreach { taskReq => - val execAmount = executorResourcesAndAmounts(taskReq.resourceName) - if ((numSlots * taskReq.amount / taskReq.numParts) < execAmount) { - val taskReqStr = if (taskReq.numParts > 1) { - s"${taskReq.amount}/${taskReq.numParts}" - } else { - s"${taskReq.amount}" - } - val resourceNumSlots = Math.floor(execAmount * taskReq.numParts / taskReq.amount).toInt - val message = s"The configuration of resource: ${taskReq.resourceName} " + - s"(exec = ${execAmount}, task = ${taskReqStr}, " + - s"runnable tasks = ${resourceNumSlots}) will " + - s"result in wasted resources due to resource ${limitingResourceName} limiting the " + - s"number of runnable tasks per executor to: ${numSlots}. Please adjust " + - s"your configuration." - if (Utils.isTesting) { - throw new SparkException(message) - } else { - logWarning(message) - } - } + validateTaskCpusLargeEnough(executorCores, taskCores) + val defaultProf = sc.resourceProfileManager.defaultResourceProfile + // TODO - this is temporary until all of stage level scheduling feature is integrated, + // fail if any other resource limiting due to dynamic allocation and scheduler using + // slots based on cores + val cpuSlots = executorCores/taskCores + val limitingResource = defaultProf.limitingResource(sc.conf) + if (limitingResource.nonEmpty && !limitingResource.equals(ResourceProfile.CPUS) && + defaultProf.maxTasksPerExecutor(sc.conf) < cpuSlots) { + throw new IllegalArgumentException("The number of slots on an executor has to be " + + "limited by the number of cores, otherwise you waste resources and " + + "dynamic allocation doesn't work properly. Your configuration has " + + s"core/task cpu slots = ${cpuSlots} and " + + s"${limitingResource} = " + + s"${defaultProf.maxTasksPerExecutor(sc.conf)}. Please adjust your configuration " + + "so that all resources require same number of executor slots.") } + ResourceUtils.warnOnWastedResources(defaultProf, sc.conf, Some(executorCores)) } master match { case "local" => - checkResourcesPerTask(clusterMode = false, Some(1)) + checkResourcesPerTask(1) val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) @@ -2886,7 +2822,7 @@ object SparkContext extends Logging { if (threadCount <= 0) { throw new SparkException(s"Asked to run locally with $threadCount threads") } - checkResourcesPerTask(clusterMode = false, Some(threadCount)) + checkResourcesPerTask(threadCount) val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) @@ -2897,14 +2833,13 @@ object SparkContext extends Logging { // local[*, M] means the number of cores on the computer with M failures // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt - checkResourcesPerTask(clusterMode = false, Some(threadCount)) + checkResourcesPerTask(threadCount) val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) case SPARK_REGEX(sparkUrl) => - checkResourcesPerTask(clusterMode = true, None) val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) @@ -2912,7 +2847,7 @@ object SparkContext extends Logging { (backend, scheduler) case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - checkResourcesPerTask(clusterMode = true, Some(coresPerSlave.toInt)) + checkResourcesPerTask(coresPerSlave.toInt) // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt if (sc.executorMemory > memoryPerSlaveInt) { @@ -2941,7 +2876,6 @@ object SparkContext extends Logging { (backend, scheduler) case masterUrl => - checkResourcesPerTask(clusterMode = true, None) val cm = getClusterManager(masterUrl) match { case Some(clusterMgr) => clusterMgr case None => throw new SparkException("Could not parse Master URL: '" + master + "'") diff --git a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala index 21660ab3a9512..51df73ebde07d 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala @@ -53,4 +53,13 @@ private[spark] object Tests { val TEST_N_CORES_EXECUTOR = ConfigBuilder("spark.testing.nCoresPerExecutor") .intConf .createWithDefault(2) + + val RESOURCES_WARNING_TESTING = + ConfigBuilder("spark.resources.warnings.testing").booleanConf.createWithDefault(false) + + val RESOURCE_PROFILE_MANAGER_TESTING = + ConfigBuilder("spark.testing.resourceProfileManager") + .booleanConf + .createWithDefault(false) + } diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index d345674d6635c..d4c29f9a70c44 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -109,7 +109,7 @@ private[spark] class ExecutorResourceRequests() extends Serializable { discoveryScript: String = "", vendor: String = ""): this.type = { // a bit weird but for Java api use empty string as meaning None because empty - // string is otherwise invalid for those paramters anyway + // string is otherwise invalid for those parameters anyway val req = new ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor) _executorResources.put(resourceName, req) this diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 14019d27fc2e6..03dcf5e317798 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -22,12 +22,14 @@ import java.util.concurrent.atomic.AtomicInteger import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ +import scala.collection.mutable -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.util.Utils /** * Resource profile to associate with an RDD. A ResourceProfile allows the user to @@ -42,6 +44,13 @@ class ResourceProfile( // _id is only a var for testing purposes private var _id = ResourceProfile.getNextProfileId + // This is used for any resources that use fractional amounts, the key is the resource name + // and the value is the number of tasks that can share a resource address. For example, + // if the user says task gpu amount is 0.5, that results in 2 tasks per resource address. + private var _executorResourceSlotsPerAddr: Option[Map[String, Int]] = None + private var _limitingResource: Option[String] = None + private var _maxTasksPerExecutor: Option[Int] = None + private var _coresLimitKnown: Boolean = false def id: Int = _id @@ -67,6 +76,138 @@ class ResourceProfile( taskResources.get(ResourceProfile.CPUS).map(_.amount.toInt) } + private[spark] def getNumSlotsPerAddress(resource: String, sparkConf: SparkConf): Int = { + _executorResourceSlotsPerAddr.getOrElse { + calculateTasksAndLimitingResource(sparkConf) + } + _executorResourceSlotsPerAddr.get.getOrElse(resource, + throw new SparkException(s"Resource $resource doesn't exist in profile id: $id")) + } + + // Maximum tasks you could put on an executor with this profile based on the limiting resource. + // If the executor cores config is not present this value is based on the other resources + // available or 1 if no other resources. You need to check the isCoresLimitKnown to + // calculate proper value. + private[spark] def maxTasksPerExecutor(sparkConf: SparkConf): Int = { + _maxTasksPerExecutor.getOrElse { + calculateTasksAndLimitingResource(sparkConf) + _maxTasksPerExecutor.get + } + } + + // Returns whether the executor cores was available to use to calculate the max tasks + // per executor and limiting resource. Some cluster managers (like standalone and coarse + // grained mesos) don't use the cores config by default so we can't use it to calculate slots. + private[spark] def isCoresLimitKnown: Boolean = _coresLimitKnown + + // The resource that has the least amount of slots per executor. Its possible multiple or all + // resources result in same number of slots and this could be any of those. + // If the executor cores config is not present this value is based on the other resources + // available or empty string if no other resources. You need to check the isCoresLimitKnown to + // calculate proper value. + private[spark] def limitingResource(sparkConf: SparkConf): String = { + _limitingResource.getOrElse { + calculateTasksAndLimitingResource(sparkConf) + _limitingResource.get + } + } + + // executor cores config is not set for some masters by default and the default value + // only applies to yarn/k8s + private def shouldCheckExecutorCores(sparkConf: SparkConf): Boolean = { + val master = sparkConf.getOption("spark.master") + sparkConf.contains(EXECUTOR_CORES) || + (master.isDefined && (master.get.equalsIgnoreCase("yarn") || master.get.startsWith("k8s"))) + } + + /** + * Utility function to calculate the number of tasks you can run on a single Executor based + * on the task and executor resource requests in the ResourceProfile. This will be based + * off the resource that is most restrictive. For instance, if the executor + * request is for 4 cpus and 2 gpus and your task request is for 1 cpu and 1 gpu each, the + * limiting resource is gpu and the number of tasks you can run on a single executor is 2. + * This function also sets the limiting resource, isCoresLimitKnown and number of slots per + * resource address. + */ + private def calculateTasksAndLimitingResource(sparkConf: SparkConf): Unit = synchronized { + val shouldCheckExecCores = shouldCheckExecutorCores(sparkConf) + var (taskLimit, limitingResource) = if (shouldCheckExecCores) { + val cpusPerTask = taskResources.get(ResourceProfile.CPUS) + .map(_.amount).getOrElse(sparkConf.get(CPUS_PER_TASK).toDouble).toInt + assert(cpusPerTask > 0, "CPUs per task configuration has to be > 0") + val coresPerExecutor = getExecutorCores.getOrElse(sparkConf.get(EXECUTOR_CORES)) + _coresLimitKnown = true + ResourceUtils.validateTaskCpusLargeEnough(coresPerExecutor, cpusPerTask) + val tasksBasedOnCores = coresPerExecutor / cpusPerTask + // Note that if the cores per executor aren't set properly this calculation could be off, + // we default it to just be 1 in order to allow checking of the rest of the custom + // resources. We set the limit based on the other resources available. + (tasksBasedOnCores, ResourceProfile.CPUS) + } else { + (-1, "") + } + val numPartsPerResourceMap = new mutable.HashMap[String, Int] + numPartsPerResourceMap(ResourceProfile.CORES) = 1 + val taskResourcesToCheck = new mutable.HashMap[String, TaskResourceRequest] + taskResourcesToCheck ++= ResourceProfile.getCustomTaskResources(this) + val execResourceToCheck = ResourceProfile.getCustomExecutorResources(this) + execResourceToCheck.foreach { case (rName, execReq) => + val taskReq = taskResources.get(rName).map(_.amount).getOrElse(0.0) + numPartsPerResourceMap(rName) = 1 + if (taskReq > 0.0) { + if (taskReq > execReq.amount) { + throw new SparkException(s"The executor resource: $rName, amount: ${execReq.amount} " + + s"needs to be >= the task resource request amount of $taskReq") + } + val (numPerTask, parts) = ResourceUtils.calculateAmountAndPartsForFraction(taskReq) + numPartsPerResourceMap(rName) = parts + val numTasks = ((execReq.amount * parts) / numPerTask).toInt + if (taskLimit == -1 || numTasks < taskLimit) { + if (shouldCheckExecCores) { + // TODO - until resource profiles full implemented we need to error if cores not + // limiting resource because the scheduler code uses that for slots + throw new IllegalArgumentException("The number of slots on an executor has to be " + + "limited by the number of cores, otherwise you waste resources and " + + "dynamic allocation doesn't work properly. Your configuration has " + + s"core/task cpu slots = ${taskLimit} and " + + s"${execReq.resourceName} = ${numTasks}. " + + "Please adjust your configuration so that all resources require same number " + + "of executor slots.") + } + limitingResource = rName + taskLimit = numTasks + } + taskResourcesToCheck -= rName + } else { + logWarning(s"The executor resource config for resource: $rName was specified but " + + "no corresponding task resource request was specified.") + } + } + if(!shouldCheckExecCores) { + // if we can't rely on the executor cores config throw a warning for user + logWarning("Please ensure that the number of slots available on your " + + "executors is limited by the number of cores to task cpus and not another " + + "custom resource. If cores is not the limiting resource then dynamic " + + "allocation will not work properly!") + } + if (taskResourcesToCheck.nonEmpty) { + throw new SparkException("No executor resource configs were not specified for the " + + s"following task configs: ${taskResourcesToCheck.keys.mkString(",")}") + } + logInfo(s"Limiting resource is $limitingResource at $taskLimit tasks per executor") + _executorResourceSlotsPerAddr = Some(numPartsPerResourceMap.toMap) + _maxTasksPerExecutor = if (taskLimit == -1) Some(1) else Some(taskLimit) + _limitingResource = Some(limitingResource) + if (shouldCheckExecCores) { + ResourceUtils.warnOnWastedResources(this, sparkConf) + } + } + + // to be used only by history server for reconstruction from events + private[spark] def setResourceProfileId(id: Int): Unit = { + _id = id + } + // testing only private[spark] def setToDefaultProfile(): Unit = { _id = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID @@ -123,7 +264,7 @@ object ResourceProfile extends Logging { val taskResources = getDefaultTaskResources(conf) val executorResources = getDefaultExecutorResources(conf) val defProf = new ResourceProfile(executorResources, taskResources) - defProf.setToDefaultProfile + defProf.setToDefaultProfile() defaultProfile = Some(defProf) logInfo("Default ResourceProfile created, executor resources: " + s"${defProf.executorResources}, task resources: " + @@ -157,13 +298,12 @@ object ResourceProfile extends Logging { // for testing only private[spark] def reInitDefaultProfile(conf: SparkConf): Unit = { - clearDefaultProfile + clearDefaultProfile() // force recreate it after clearing getOrCreateDefaultProfile(conf) } - // for testing only - private[spark] def clearDefaultProfile: Unit = { + private[spark] def clearDefaultProfile(): Unit = { DEFAULT_PROFILE_LOCK.synchronized { defaultProfile = None } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala index 0d55c176eeb65..26f23f4bf0476 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala @@ -31,7 +31,7 @@ import org.apache.spark.annotation.Evolving * requirements between stages. */ @Evolving -class ResourceProfileBuilder() { +private[spark] class ResourceProfileBuilder() { private val _taskResources = new ConcurrentHashMap[String, TaskResourceRequest]() private val _executorResources = new ConcurrentHashMap[String, ExecutorResourceRequest]() diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala new file mode 100644 index 0000000000000..06db9468c451e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import java.util.concurrent.ConcurrentHashMap + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.annotation.Evolving +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Tests._ +import org.apache.spark.util.Utils +import org.apache.spark.util.Utils.isTesting + +/** + * Manager of resource profiles. The manager allows one place to keep the actual ResourceProfiles + * and everywhere else we can use the ResourceProfile Id to save on space. + * Note we never remove a resource profile at this point. Its expected this number if small + * so this shouldn't be much overhead. + */ +@Evolving +private[spark] class ResourceProfileManager(sparkConf: SparkConf) extends Logging { + private val resourceProfileIdToResourceProfile = new ConcurrentHashMap[Int, ResourceProfile]() + + private val defaultProfile = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + addResourceProfile(defaultProfile) + + def defaultResourceProfile: ResourceProfile = defaultProfile + + private val taskCpusDefaultProfile = defaultProfile.getTaskCpus.get + private val dynamicEnabled = Utils.isDynamicAllocationEnabled(sparkConf) + private val master = sparkConf.getOption("spark.master") + private val isNotYarn = master.isDefined && !master.get.equals("yarn") + private val errorForTesting = !isTesting || sparkConf.get(RESOURCE_PROFILE_MANAGER_TESTING) + + // If we use anything except the default profile, its only supported on YARN right now. + // Throw an exception if not supported. + private[spark] def isSupported(rp: ResourceProfile): Boolean = { + val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + val notYarnAndNotDefaultProfile = isNotDefaultProfile && isNotYarn + val YarnNotDynAllocAndNotDefaultProfile = isNotDefaultProfile && !isNotYarn && !dynamicEnabled + if (errorForTesting && (notYarnAndNotDefaultProfile || YarnNotDynAllocAndNotDefaultProfile)) { + throw new SparkException("ResourceProfiles are only supported on YARN with dynamic " + + "allocation enabled.") + } + true + } + + def addResourceProfile(rp: ResourceProfile): Unit = { + isSupported(rp) + // force the computation of maxTasks and limitingResource now so we don't have cost later + rp.limitingResource(sparkConf) + logInfo(s"Adding ResourceProfile id: ${rp.id}") + resourceProfileIdToResourceProfile.putIfAbsent(rp.id, rp) + } + + /* + * Gets the ResourceProfile associated with the id, if a profile doesn't exist + * it returns the default ResourceProfile created from the application level configs. + */ + def resourceProfileFromId(rpId: Int): ResourceProfile = { + val rp = resourceProfileIdToResourceProfile.get(rpId) + if (rp == null) { + throw new SparkException(s"ResourceProfileId $rpId not found!") + } + rp + } + + def taskCpusForProfileId(rpId: Int): Int = { + resourceProfileFromId(rpId).getTaskCpus.getOrElse(taskCpusDefaultProfile) + } +} diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index 7dd7fc1b99353..cdb761c7566e7 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -29,7 +29,8 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.resource.ResourceDiscoveryPlugin import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{RESOURCES_DISCOVERY_PLUGIN, SPARK_TASK_PREFIX} +import org.apache.spark.internal.config.{CPUS_PER_TASK, EXECUTOR_CORES, RESOURCES_DISCOVERY_PLUGIN, SPARK_TASK_PREFIX} +import org.apache.spark.internal.config.Tests.{RESOURCES_WARNING_TESTING} import org.apache.spark.util.Utils /** @@ -161,19 +162,23 @@ private[spark] object ResourceUtils extends Logging { } // Used to take a fraction amount from a task resource requirement and split into a real - // integer amount and the number of parts expected. For instance, if the amount is 0.5, - // the we get (1, 2) back out. - // Returns tuple of (amount, numParts) - def calculateAmountAndPartsForFraction(amount: Double): (Int, Int) = { - val parts = if (amount <= 0.5) { - Math.floor(1.0 / amount).toInt - } else if (amount % 1 != 0) { + // integer amount and the number of slots per address. For instance, if the amount is 0.5, + // the we get (1, 2) back out. This indicates that for each 1 address, it has 2 slots per + // address, which allows you to put 2 tasks on that address. Note if amount is greater + // than 1, then the number of slots per address has to be 1. This would indicate that a + // would have multiple addresses assigned per task. This can be used for calculating + // the number of tasks per executor -> (executorAmount * numParts) / (integer amount). + // Returns tuple of (integer amount, numParts) + def calculateAmountAndPartsForFraction(doubleAmount: Double): (Int, Int) = { + val parts = if (doubleAmount <= 0.5) { + Math.floor(1.0 / doubleAmount).toInt + } else if (doubleAmount % 1 != 0) { throw new SparkException( - s"The resource amount ${amount} must be either <= 0.5, or a whole number.") + s"The resource amount ${doubleAmount} must be either <= 0.5, or a whole number.") } else { 1 } - (Math.ceil(amount).toInt, parts) + (Math.ceil(doubleAmount).toInt, parts) } // Add any task resource requests from the spark conf to the TaskResourceRequests passed in @@ -382,6 +387,90 @@ private[spark] object ResourceUtils extends Logging { s"${resourceRequest.id.resourceName}") } + def validateTaskCpusLargeEnough(execCores: Int, taskCpus: Int): Boolean = { + // Number of cores per executor must meet at least one task requirement. + if (execCores < taskCpus) { + throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " + + s"the number of cpus per task = $taskCpus.") + } + true + } + + // the option executor cores parameter is by the different local modes since it not configured + // via the config + def warnOnWastedResources( + rp: ResourceProfile, + sparkConf: SparkConf, + execCores: Option[Int] = None): Unit = { + // There have been checks on the ResourceProfile to make sure the executor resources were + // specified and are large enough if any task resources were specified. + // Now just do some sanity test and log warnings when it looks like the user will + // waste some resources. + val coresKnown = rp.isCoresLimitKnown + var limitingResource = rp.limitingResource(sparkConf) + var maxTaskPerExec = rp.maxTasksPerExecutor(sparkConf) + val taskCpus = rp.getTaskCpus.getOrElse(sparkConf.get(CPUS_PER_TASK)) + val cores = if (execCores.isDefined) { + execCores.get + } else if (coresKnown) { + rp.getExecutorCores.getOrElse(sparkConf.get(EXECUTOR_CORES)) + } else { + // can't calculate cores limit + return + } + // when executor cores config isn't set, we can't calculate the real limiting resource + // and number of tasks per executor ahead of time, so calculate it now. + if (!coresKnown) { + val numTasksPerExecCores = cores / taskCpus + val numTasksPerExecCustomResource = rp.maxTasksPerExecutor(sparkConf) + if (limitingResource.isEmpty || + (limitingResource.nonEmpty && numTasksPerExecCores < numTasksPerExecCustomResource)) { + limitingResource = ResourceProfile.CPUS + maxTaskPerExec = numTasksPerExecCores + } + } + val taskReq = ResourceProfile.getCustomTaskResources(rp) + val execReq = ResourceProfile.getCustomExecutorResources(rp) + + if (limitingResource.nonEmpty && !limitingResource.equals(ResourceProfile.CPUS)) { + if ((taskCpus * maxTaskPerExec) < cores) { + val resourceNumSlots = Math.floor(cores/taskCpus).toInt + val message = s"The configuration of cores (exec = ${cores} " + + s"task = ${taskCpus}, runnable tasks = ${resourceNumSlots}) will " + + s"result in wasted resources due to resource ${limitingResource} limiting the " + + s"number of runnable tasks per executor to: ${maxTaskPerExec}. Please adjust " + + "your configuration." + if (sparkConf.get(RESOURCES_WARNING_TESTING)) { + throw new SparkException(message) + } else { + logWarning(message) + } + } + } + + taskReq.foreach { case (rName, treq) => + val execAmount = execReq(rName).amount + val numParts = rp.getNumSlotsPerAddress(rName, sparkConf) + // handle fractional + val taskAmount = if (numParts > 1) 1 else treq.amount + if (maxTaskPerExec < (execAmount * numParts / taskAmount)) { + val taskReqStr = s"${taskAmount}/${numParts}" + val resourceNumSlots = Math.floor(execAmount * numParts / taskAmount).toInt + val message = s"The configuration of resource: ${treq.resourceName} " + + s"(exec = ${execAmount}, task = ${taskReqStr}, " + + s"runnable tasks = ${resourceNumSlots}) will " + + s"result in wasted resources due to resource ${limitingResource} limiting the " + + s"number of runnable tasks per executor to: ${maxTaskPerExec}. Please adjust " + + "your configuration." + if (sparkConf.get(RESOURCES_WARNING_TESTING)) { + throw new SparkException(message) + } else { + logWarning(message) + } + } + } + } + // known types of resources final val GPU: String = "gpu" final val FPGA: String = "fpga" diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7bf363dd71c1b..fd5c3e0827bf9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -37,7 +37,8 @@ import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData} +import org.apache.spark.rdd.{RDD, RDDCheckpointData} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -391,7 +392,8 @@ private[spark] class DAGScheduler( val parents = getOrCreateParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() val stage = new ShuffleMapStage( - id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep, mapOutputTracker) + id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep, mapOutputTracker, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stageIdToStage(id) = stage shuffleIdToMapStage(shuffleDep.shuffleId) = stage @@ -453,7 +455,8 @@ private[spark] class DAGScheduler( checkBarrierStageWithRDDChainPattern(rdd, partitions.toSet.size) val parents = getOrCreateParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() - val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite) + val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala index d1687830ff7bf..7fdc3186e86bd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -34,8 +34,9 @@ private[spark] class ResultStage( val partitions: Array[Int], parents: List[Stage], firstJobId: Int, - callSite: CallSite) - extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite) { + callSite: CallSite, + resourceProfileId: Int) + extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite, resourceProfileId) { /** * The active job for this result stage. Will be empty if the job has already finished diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 1b44d0aee3195..be1984de9837f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -42,8 +42,9 @@ private[spark] class ShuffleMapStage( firstJobId: Int, callSite: CallSite, val shuffleDep: ShuffleDependency[_, _, _], - mapOutputTrackerMaster: MapOutputTrackerMaster) - extends Stage(id, rdd, numTasks, parents, firstJobId, callSite) { + mapOutputTrackerMaster: MapOutputTrackerMaster, + resourceProfileId: Int) + extends Stage(id, rdd, numTasks, parents, firstJobId, callSite, resourceProfileId) { private[this] var _mapStageJobs: List[ActiveJob] = Nil diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index a9f72eae71368..ae7924d66a301 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -59,7 +59,8 @@ private[scheduler] abstract class Stage( val numTasks: Int, val parents: List[Stage], val firstJobId: Int, - val callSite: CallSite) + val callSite: CallSite, + val resourceProfileId: Int) extends Logging { val numPartitions = rdd.partitions.length @@ -79,7 +80,8 @@ private[scheduler] abstract class Stage( * StageInfo to tell SparkListeners when a job starts (which happens before any stage attempts * have been created). */ - private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId) + private var _latestInfo: StageInfo = + StageInfo.fromStage(this, nextAttemptId, resourceProfileId = resourceProfileId) /** * Set of stage attempt IDs that have failed. We keep track of these failures in order to avoid @@ -100,7 +102,8 @@ private[scheduler] abstract class Stage( val metrics = new TaskMetrics metrics.register(rdd.sparkContext) _latestInfo = StageInfo.fromStage( - this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences) + this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences, + resourceProfileId = resourceProfileId) nextAttemptId += 1 } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index fdc50328b43d8..556478d83cf39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -38,7 +38,8 @@ class StageInfo( val details: String, val taskMetrics: TaskMetrics = null, private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, - private[spark] val shuffleDepId: Option[Int] = None) { + private[spark] val shuffleDepId: Option[Int] = None, + val resourceProfileId: Int) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -87,7 +88,8 @@ private[spark] object StageInfo { attemptId: Int, numTasks: Option[Int] = None, taskMetrics: TaskMetrics = null, - taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty + taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, + resourceProfileId: Int ): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos @@ -105,6 +107,7 @@ private[spark] object StageInfo { stage.details, taskMetrics, taskLocalityPreferences, - shuffleDepId) + shuffleDepId, + resourceProfileId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 6a1d460e6a9d9..bf92081d13907 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -384,7 +384,9 @@ private[spark] class TaskSchedulerImpl( */ private def resourcesMeetTaskRequirements(resources: Map[String, Buffer[String]]): Boolean = { val resourcesFree = resources.map(r => r._1 -> r._2.length) - ResourceUtils.resourcesMeetRequirements(resourcesFree, resourcesReqsPerTask) + val meetsReqs = ResourceUtils.resourcesMeetRequirements(resourcesFree, resourcesReqsPerTask) + logDebug(s"Resources meet task requirements is: $meetsReqs") + meetsReqs } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 55f4005ef1b45..63aa04986b073 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -69,13 +69,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp conf.get(SCHEDULER_MAX_REGISTERED_RESOURCE_WAITING_TIME)) private val createTimeNs = System.nanoTime() - private val taskResourceNumParts: Map[String, Int] = - if (scheduler.resourcesReqsPerTask != null) { - scheduler.resourcesReqsPerTask.map(req => req.resourceName -> req.numParts).toMap - } else { - Map.empty - } - // Accessing `executorDataMap` in the inherited methods from ThreadSafeRpcEndpoint doesn't need // any protection. But accessing `executorDataMap` out of the inherited methods must be // protected by `CoarseGrainedSchedulerBackend.this`. Besides, `executorDataMap` should only @@ -83,13 +76,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // `CoarseGrainedSchedulerBackend.this`. private val executorDataMap = new HashMap[String, ExecutorData] - // Number of executors requested by the cluster manager, [[ExecutorAllocationManager]] - @GuardedBy("CoarseGrainedSchedulerBackend.this") - private var requestedTotalExecutors = 0 - - // Number of executors requested from the cluster manager that have not registered yet + // Number of executors for each ResourceProfile requested by the cluster + // manager, [[ExecutorAllocationManager]] @GuardedBy("CoarseGrainedSchedulerBackend.this") - private var numPendingExecutors = 0 + private val requestedTotalExecutorsPerResourceProfile = new HashMap[ResourceProfile, Int] private val listenerBus = scheduler.sc.listenerBus @@ -102,13 +92,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Executors that have been lost, but for which we don't yet know the real exit reason. private val executorsPendingLossReason = new HashSet[String] - // A map to store hostname with its possible task number running on it + // A map of ResourceProfile id to map of hostname with its possible task number running on it @GuardedBy("CoarseGrainedSchedulerBackend.this") - protected var hostToLocalTaskCount: Map[String, Int] = Map.empty + protected var rpHostToLocalTaskCount: Map[Int, Map[String, Int]] = Map.empty - // The number of pending tasks which is locality required + // The number of pending tasks per ResourceProfile id which is locality required @GuardedBy("CoarseGrainedSchedulerBackend.this") - protected var localityAwareTasks = 0 + protected var numLocalityAwareTasksPerResourceProfileId = Map.empty[Int, Int] // The num of current max ExecutorId used to re-register appMaster @volatile protected var currentExecutorIdCounter = 0 @@ -223,16 +213,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } else { context.senderAddress } - logInfo(s"Registered executor $executorRef ($executorAddress) with ID $executorId") + logInfo(s"Registered executor $executorRef ($executorAddress) with ID $executorId, " + + s" ResourceProfileId $resourceProfileId") addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) - val resourcesInfo = resources.map{ case (k, v) => - (v.name, - new ExecutorResourceInfo(v.name, v.addresses, - // tell the executor it can schedule resources up to numParts times, - // as configured by the user, or set to 1 as that is the default (1 task/resource) - taskResourceNumParts.getOrElse(v.name, 1))) + val resourcesInfo = resources.map { case (rName, info) => + // tell the executor it can schedule resources up to numParts times, + // as configured by the user, or set to 1 as that is the default (1 task/resource) + val numParts = scheduler.sc.resourceProfileManager + .resourceProfileFromId(resourceProfileId).getNumSlotsPerAddress(rName, conf) + (info.name, new ExecutorResourceInfo(info.name, info.addresses, numParts)) } val data = new ExecutorData(executorRef, executorAddress, hostname, 0, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, @@ -244,10 +235,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp if (currentExecutorIdCounter < executorId.toInt) { currentExecutorIdCounter = executorId.toInt } - if (numPendingExecutors > 0) { - numPendingExecutors -= 1 - logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") - } } // Note: some tests expect the reply to come after we put the executor in the map context.reply(true) @@ -271,10 +258,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(true) case RetrieveSparkAppConfig(resourceProfileId) => - // note this will be updated in later prs to get the ResourceProfile from a - // ResourceProfileManager that matches the resource profile id - // for now just use default profile - val rp = ResourceProfile.getOrCreateDefaultProfile(conf) + val rp = scheduler.sc.resourceProfileManager.resourceProfileFromId(resourceProfileId) val reply = SparkAppConfig( sparkProperties, SparkEnv.get.securityManager.getIOEncryptionKey(), @@ -494,8 +478,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * */ protected[scheduler] def reset(): Unit = { val executors: Set[String] = synchronized { - requestedTotalExecutors = 0 - numPendingExecutors = 0 + requestedTotalExecutorsPerResourceProfile.clear() executorDataMap.keys.toSet } @@ -577,12 +560,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // this function is for testing only def getExecutorResourceProfileId(executorId: String): Int = synchronized { - val res = executorDataMap.get(executorId) - res.map(_.resourceProfileId).getOrElse(ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) + val execDataOption = executorDataMap.get(executorId) + execDataOption.map(_.resourceProfileId).getOrElse(ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) } /** - * Request an additional number of executors from the cluster manager. + * Request an additional number of executors from the cluster manager. This is + * requesting against the default ResourceProfile, we will need an API change to + * allow against other profiles. * @return whether the request is acknowledged. */ final override def requestExecutors(numAdditionalExecutors: Int): Boolean = { @@ -594,21 +579,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") val response = synchronized { - requestedTotalExecutors += numAdditionalExecutors - numPendingExecutors += numAdditionalExecutors - logDebug(s"Number of pending executors is now $numPendingExecutors") - if (requestedTotalExecutors != - (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) { - logDebug( - s"""requestExecutors($numAdditionalExecutors): Executor request doesn't match: - |requestedTotalExecutors = $requestedTotalExecutors - |numExistingExecutors = $numExistingExecutors - |numPendingExecutors = $numPendingExecutors - |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin) - } - + val defaultProf = scheduler.sc.resourceProfileManager.defaultResourceProfile + val numExisting = requestedTotalExecutorsPerResourceProfile.getOrElse(defaultProf, 0) + requestedTotalExecutorsPerResourceProfile(defaultProf) = numExisting + numAdditionalExecutors // Account for executors pending to be added or removed - doRequestTotalExecutors(requestedTotalExecutors) + doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap) } defaultAskTimeout.awaitResult(response) @@ -617,39 +592,41 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. - * @param numExecutors The total number of executors we'd like to have. The cluster manager - * shouldn't kill any running executor to reach this number, but, - * if all existing executors were to die, this is the number of executors - * we'd want to be allocated. - * @param localityAwareTasks The number of tasks in all active stages that have a locality - * preferences. This includes running, pending, and completed tasks. + * @param resourceProfileToNumExecutors The total number of executors we'd like to have per + * ResourceProfile. The cluster manager shouldn't kill any + * running executor to reach this number, but, if all + * existing executors were to die, this is the number + * of executors we'd want to be allocated. + * @param numLocalityAwareTasksPerResourceProfileId The number of tasks in all active stages that + * have a locality preferences per + * ResourceProfile. This includes running, + * pending, and completed tasks. * @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages * that would like to like to run on that host. * This includes running, pending, and completed tasks. * @return whether the request is acknowledged by the cluster manager. */ final override def requestTotalExecutors( - numExecutors: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int] - ): Boolean = { - if (numExecutors < 0) { + resourceProfileIdToNumExecutors: Map[Int, Int], + numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], + hostToLocalTaskCount: Map[Int, Map[String, Int]] + ): Boolean = { + val totalExecs = resourceProfileIdToNumExecutors.values.sum + if (totalExecs < 0) { throw new IllegalArgumentException( "Attempted to request a negative number of executor(s) " + - s"$numExecutors from the cluster manager. Please specify a positive number!") + s"$totalExecs from the cluster manager. Please specify a positive number!") + } + val resourceProfileToNumExecutors = resourceProfileIdToNumExecutors.map { case (rpid, num) => + (scheduler.sc.resourceProfileManager.resourceProfileFromId(rpid), num) } - val response = synchronized { - this.requestedTotalExecutors = numExecutors - this.localityAwareTasks = localityAwareTasks - this.hostToLocalTaskCount = hostToLocalTaskCount - - numPendingExecutors = - math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0) - - doRequestTotalExecutors(numExecutors) + this.requestedTotalExecutorsPerResourceProfile.clear() + this.requestedTotalExecutorsPerResourceProfile ++= resourceProfileToNumExecutors + this.numLocalityAwareTasksPerResourceProfileId = numLocalityAwareTasksPerResourceProfileId + this.rpHostToLocalTaskCount = hostToLocalTaskCount + doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap) } - defaultAskTimeout.awaitResult(response) } @@ -665,7 +642,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * * @return a future whose evaluation indicates whether the request is acknowledged. */ - protected def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = + protected def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = Future.successful(false) /** @@ -706,20 +684,20 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // take into account executors that are pending to be added or removed. val adjustTotalExecutors = if (adjustTargetNumExecutors) { - requestedTotalExecutors = math.max(requestedTotalExecutors - executorsToKill.size, 0) - if (requestedTotalExecutors != - (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) { - logDebug( - s"""killExecutors($executorIds, $adjustTargetNumExecutors, $countFailures, $force): - |Executor counts do not match: - |requestedTotalExecutors = $requestedTotalExecutors - |numExistingExecutors = $numExistingExecutors - |numPendingExecutors = $numPendingExecutors - |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin) + executorsToKill.foreach { exec => + val rpId = executorDataMap(exec).resourceProfileId + val rp = scheduler.sc.resourceProfileManager.resourceProfileFromId(rpId) + if (requestedTotalExecutorsPerResourceProfile.isEmpty) { + // Assume that we are killing an executor that was started by default and + // not through the request api + requestedTotalExecutorsPerResourceProfile(rp) = 0 + } else { + val requestedTotalForRp = requestedTotalExecutorsPerResourceProfile(rp) + requestedTotalExecutorsPerResourceProfile(rp) = math.max(requestedTotalForRp - 1, 0) + } } - doRequestTotalExecutors(requestedTotalExecutors) + doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap) } else { - numPendingExecutors += executorsToKill.size Future.successful(true) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index a9b607d8cc38c..d91d78b29f98d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientL import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} -import org.apache.spark.resource.ResourceUtils +import org.apache.spark.resource.{ResourceProfile, ResourceUtils} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -58,6 +58,7 @@ private[spark] class StandaloneSchedulerBackend( private val maxCores = conf.get(config.CORES_MAX) private val totalExpectedCores = maxCores.getOrElse(0) + private val defaultProf = sc.resourceProfileManager.defaultResourceProfile override def start(): Unit = { super.start() @@ -194,9 +195,13 @@ private[spark] class StandaloneSchedulerBackend( * * @return whether the request is acknowledged. */ - protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + protected override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { + // resources profiles not supported Option(client) match { - case Some(c) => c.requestTotalExecutors(requestedTotal) + case Some(c) => + val numExecs = resourceProfileToTotalExecs.getOrElse(defaultProf, 0) + c.requestTotalExecutors(numExecs) case None => logWarning("Attempted to request executors before driver fully initialized.") Future.successful(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index a24f1902faa31..c29546b7577fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -70,7 +70,7 @@ private[spark] class ExecutorMonitor( // this listener. There are safeguards in other parts of the code that would prevent that executor // from being removed. private val nextTimeout = new AtomicLong(Long.MaxValue) - private var timedOutExecs = Seq.empty[String] + private var timedOutExecs = Seq.empty[(String, Int)] // Active job tracking. // @@ -100,10 +100,10 @@ private[spark] class ExecutorMonitor( } /** - * Returns the list of executors that are currently considered to be timed out. - * Should only be called from the EAM thread. + * Returns the list of executors and their ResourceProfile id that are currently considered to + * be timed out. Should only be called from the EAM thread. */ - def timedOutExecutors(): Seq[String] = { + def timedOutExecutors(): Seq[(String, Int)] = { val now = clock.nanoTime() if (now >= nextTimeout.get()) { // Temporarily set the next timeout at Long.MaxValue. This ensures that after @@ -126,7 +126,7 @@ private[spark] class ExecutorMonitor( true } } - .keys + .map { case (name, exec) => (name, exec.resourceProfileId)} .toSeq updateNextTimeout(newNextTimeout) } @@ -155,6 +155,7 @@ private[spark] class ExecutorMonitor( execResourceProfileCount.getOrDefault(id, 0) } + // for testing def getResourceProfileId(executorId: String): Int = { val execTrackingInfo = executors.get(executorId) if (execTrackingInfo != null) { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 4d89c4f079f29..53824735d2fc5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -33,7 +33,7 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope -import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ @@ -662,7 +662,8 @@ private[spark] object JsonProtocol { val stageInfos = jsonOption(json \ "Stage Infos") .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { stageIds.map { id => - new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown") + new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } } SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) @@ -803,7 +804,8 @@ private[spark] object JsonProtocol { } val stageInfo = new StageInfo( - stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details) + stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 8d958494d52be..8fa33f4915ea4 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, ResourceProfileManager, TaskResourceRequests} import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -45,6 +46,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private val managers = new mutable.ListBuffer[ExecutorAllocationManager]() private var listenerBus: LiveListenerBus = _ private var client: ExecutorAllocationClient = _ + private val clock = new SystemClock() + private var rpManager: ResourceProfileManager = _ + override def beforeEach(): Unit = { super.beforeEach() @@ -108,65 +112,257 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("starting state") { val manager = createManager(createConf()) - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) assert(executorsPendingToRemove(manager).isEmpty) assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) } - test("add executors") { + test("add executors default profile") { val manager = createManager(createConf(1, 10, 1)) post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + // Keep adding until the limit is reached + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 8) + // reached the limit of 10 + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + + // Register previously requested executors + onExecutorAddedDefaultProfile(manager, "first") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + onExecutorAddedDefaultProfile(manager, "first") // duplicates should not count + onExecutorAddedDefaultProfile(manager, "second") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + + // Try adding again + // This should still fail because the number pending + running is still at the limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + } + + test("add executors multiple profiles") { + val manager = createManager(createConf(1, 10, 1)) + post(SparkListenerStageSubmitted(createStageInfo(0, 1000, rp = defaultProfile))) + val rp1 = new ResourceProfileBuilder() + val execReqs = new ExecutorResourceRequests().cores(4).resource("gpu", 4) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + rp1.require(execReqs).require(taskReqs) + val rprof1 = rp1.build + rpManager.addResourceProfile(rprof1) + post(SparkListenerStageSubmitted(createStageInfo(1, 1000, rp = rprof1))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Keep adding until the limit is reached - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 2) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 4) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 4) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 8) - assert(addExecutors(manager) === 2) // reached the limit of 10 - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 1) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + assert(numExecutorsToAdd(manager, rprof1) === 2) + assert(numExecutorsTarget(manager, rprof1.id) === 2) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + assert(numExecutorsToAdd(manager, rprof1) === 4) + assert(numExecutorsTarget(manager, rprof1.id) === 4) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 8) + // reached the limit of 10 + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + assert(numExecutorsToAdd(manager, rprof1) === 8) + assert(numExecutorsTarget(manager, rprof1.id) === 8) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) // Register previously requested executors - onExecutorAdded(manager, "first") - assert(numExecutorsTarget(manager) === 10) - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - assert(numExecutorsTarget(manager) === 10) - onExecutorAdded(manager, "first") // duplicates should not count - onExecutorAdded(manager, "second") - assert(numExecutorsTarget(manager) === 10) + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAdded(manager, "firstrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAdded(manager, "secondrp1", rprof1) + onExecutorAdded(manager, "thirdrp1", rprof1) + onExecutorAdded(manager, "fourthrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + onExecutorAddedDefaultProfile(manager, "first") // duplicates should not count + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAdded(manager, "firstrp1", rprof1) + onExecutorAdded(manager, "secondrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) // Try adding again // This should still fail because the number pending + running is still at the limit - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + } + + test("remove executors multiple profiles") { + val manager = createManager(createConf(5, 10, 5)) + val rp1 = new ResourceProfileBuilder() + val execReqs = new ExecutorResourceRequests().cores(4).resource("gpu", 4) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + rp1.require(execReqs).require(taskReqs) + val rprof1 = rp1.build + val rp2 = new ResourceProfileBuilder() + val execReqs2 = new ExecutorResourceRequests().cores(1) + val taskReqs2 = new TaskResourceRequests().cpus(1) + rp2.require(execReqs2).require(taskReqs2) + val rprof2 = rp2.build + rpManager.addResourceProfile(rprof1) + rpManager.addResourceProfile(rprof2) + post(SparkListenerStageSubmitted(createStageInfo(1, 10, rp = rprof1))) + post(SparkListenerStageSubmitted(createStageInfo(2, 10, rp = rprof2))) + + (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id, rprof1) } + (11 to 20).map(_.toString).foreach { id => onExecutorAdded(manager, id, rprof2) } + (21 to 30).map(_.toString).foreach { id => onExecutorAdded(manager, id, defaultProfile) } + + // Keep removing until the limit is reached + assert(executorsPendingToRemove(manager).isEmpty) + assert(removeExecutor(manager, "1", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 1) + assert(executorsPendingToRemove(manager).contains("1")) + assert(removeExecutor(manager, "11", rprof2.id)) + assert(removeExecutor(manager, "2", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 3) + assert(executorsPendingToRemove(manager).contains("2")) + assert(executorsPendingToRemove(manager).contains("11")) + assert(removeExecutor(manager, "21", defaultProfile.id)) + assert(removeExecutor(manager, "3", rprof1.id)) + assert(removeExecutor(manager, "4", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 6) + assert(executorsPendingToRemove(manager).contains("21")) + assert(executorsPendingToRemove(manager).contains("3")) + assert(executorsPendingToRemove(manager).contains("4")) + assert(removeExecutor(manager, "5", rprof1.id)) + assert(!removeExecutor(manager, "6", rprof1.id)) // reached the limit of 5 + assert(executorsPendingToRemove(manager).size === 7) + assert(executorsPendingToRemove(manager).contains("5")) + assert(!executorsPendingToRemove(manager).contains("6")) + + // Kill executors previously requested to remove + onExecutorRemoved(manager, "1") + assert(executorsPendingToRemove(manager).size === 6) + assert(!executorsPendingToRemove(manager).contains("1")) + onExecutorRemoved(manager, "2") + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 4) + assert(!executorsPendingToRemove(manager).contains("2")) + assert(!executorsPendingToRemove(manager).contains("3")) + onExecutorRemoved(manager, "2") // duplicates should not count + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 4) + onExecutorRemoved(manager, "4") + onExecutorRemoved(manager, "5") + assert(executorsPendingToRemove(manager).size === 2) + assert(executorsPendingToRemove(manager).contains("11")) + assert(executorsPendingToRemove(manager).contains("21")) + + // Try removing again + // This should still fail because the number pending + running is still at the limit + assert(!removeExecutor(manager, "7", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 2) + assert(!removeExecutor(manager, "8", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 2) + + // make sure rprof2 has the same min limit or 5 + assert(removeExecutor(manager, "12", rprof2.id)) + assert(removeExecutor(manager, "13", rprof2.id)) + assert(removeExecutor(manager, "14", rprof2.id)) + assert(removeExecutor(manager, "15", rprof2.id)) + assert(!removeExecutor(manager, "16", rprof2.id)) // reached the limit of 5 + assert(executorsPendingToRemove(manager).size === 6) + assert(!executorsPendingToRemove(manager).contains("16")) + onExecutorRemoved(manager, "11") + onExecutorRemoved(manager, "12") + onExecutorRemoved(manager, "13") + onExecutorRemoved(manager, "14") + onExecutorRemoved(manager, "15") + assert(executorsPendingToRemove(manager).size === 1) } def testAllocationRatio(cores: Int, divisor: Double, expected: Int): Unit = { + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] val conf = createConf(3, 15) .set(config.DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO, divisor) .set(config.EXECUTOR_CORES, cores) val manager = createManager(conf) post(SparkListenerStageSubmitted(createStageInfo(0, 20))) for (i <- 0 to 5) { - addExecutors(manager) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) } - assert(numExecutorsTarget(manager) === expected) + assert(numExecutorsTargetForDefaultProfileId(manager) === expected) } test("executionAllocationRatio is correctly handled") { @@ -185,127 +381,158 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(0, 10, 0)) post(SparkListenerStageSubmitted(createStageInfo(0, 5))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Verify that we're capped at number of tasks in the stage - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 5) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task doesn't affect the target post(SparkListenerStageSubmitted(createStageInfo(1, 3))) post(SparkListenerExecutorAdded( 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 6) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 6) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that re-running a task doesn't blow things up post(SparkListenerStageSubmitted(createStageInfo(2, 3))) post(SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1"))) post(SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1"))) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 9) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 9) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task once we're at our limit doesn't blow things up post(SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1"))) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) } test("add executors when speculative tasks added") { val manager = createManager(createConf(0, 10, 0)) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(createStageInfo(1, 2))) // Verify that we're capped at number of tasks including the speculative ones in the stage post(SparkListenerSpeculativeTaskSubmitted(1)) - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) post(SparkListenerSpeculativeTaskSubmitted(1)) post(SparkListenerSpeculativeTaskSubmitted(1)) - post(SparkListenerStageSubmitted(createStageInfo(1, 2))) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 5) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task doesn't affect the target post(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a speculative task doesn't affect the target post(SparkListenerTaskStart(1, 0, createTaskInfo(1, 0, "executor-2", true))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) } test("SPARK-30511 remove executors when speculative tasks end") { val clock = new ManualClock() val stage = createStageInfo(0, 40) - val manager = createManager(createConf(0, 10, 0).set(config.EXECUTOR_CORES, 4), clock = clock) + val conf = createConf(0, 10, 0).set(config.EXECUTOR_CORES, 4) + val manager = createManager(conf, clock = clock) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] post(SparkListenerStageSubmitted(stage)) - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - assert(addExecutors(manager) === 4) - assert(addExecutors(manager) === 3) - - (0 to 9).foreach(execId => onExecutorAdded(manager, execId.toString)) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 3) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + + (0 to 9).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) (0 to 39).map { i => createTaskInfo(i, i, executorId = s"${i / 4}")}.foreach { info => post(SparkListenerTaskStart(0, 0, info)) } - assert(numExecutorsTarget(manager) === 10) - assert(maxNumExecutorsNeeded(manager) == 10) + assert(numExecutorsTarget(manager, defaultProfile.id) === 10) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 10) // 30 tasks (0 - 29) finished (0 to 29).map { i => createTaskInfo(i, i, executorId = s"${i / 4}")}.foreach { info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) } clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 3) - assert(maxNumExecutorsNeeded(manager) == 3) - (0 to 6).foreach { i => assert(removeExecutor(manager, i.toString))} + assert(numExecutorsTarget(manager, defaultProfile.id) === 3) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 3) + (0 to 6).foreach { i => assert(removeExecutorDefaultProfile(manager, i.toString))} (0 to 6).foreach { i => onExecutorRemoved(manager, i.toString)} // 10 speculative tasks (30 - 39) launch for the remaining tasks (30 to 39).foreach { _ => post(SparkListenerSpeculativeTaskSubmitted(0))} - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) == 5) - assert(maxNumExecutorsNeeded(manager) == 5) - (10 to 12).foreach(execId => onExecutorAdded(manager, execId.toString)) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTarget(manager, defaultProfile.id) == 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) + (10 to 12).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) (40 to 49).map { i => createTaskInfo(taskId = i, taskIndex = i - 10, executorId = s"${i / 4}", speculative = true)} .foreach { info => post(SparkListenerTaskStart(0, 0, info))} clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) == 5) // At this point, we still have 6 executors running - assert(maxNumExecutorsNeeded(manager) == 5) + // At this point, we still have 6 executors running + assert(numExecutorsTarget(manager, defaultProfile.id) == 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) // 6 speculative tasks (40 - 45) finish before the original tasks, with 4 speculative remaining (40 to 45).map { i => @@ -314,9 +541,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null))} clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 4) - assert(maxNumExecutorsNeeded(manager) == 4) - assert(removeExecutor(manager, "10")) + assert(numExecutorsTarget(manager, defaultProfile.id) === 4) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 4) + assert(removeExecutorDefaultProfile(manager, "10")) onExecutorRemoved(manager, "10") // At this point, we still have 5 executors running: ["7", "8", "9", "11", "12"] @@ -327,9 +554,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { SparkListenerTaskEnd(0, 0, null, TaskKilled("test"), info, new ExecutorMetrics, null))} clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 2) - assert(maxNumExecutorsNeeded(manager) == 2) - (7 to 8).foreach { i => assert(removeExecutor(manager, i.toString))} + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) + (7 to 8).foreach { i => assert(removeExecutorDefaultProfile(manager, i.toString))} (7 to 8).foreach { i => onExecutorRemoved(manager, i.toString)} // At this point, we still have 3 executors running: ["9", "11", "12"] @@ -343,8 +570,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // tasks running. Target lowers to 2, but still hold 3 executors ["9", "11", "12"] clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 2) - assert(maxNumExecutorsNeeded(manager) == 2) + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) // At this point, we still have 3 executors running: ["9", "11", "12"] // Task 37 and 47 succeed at the same time @@ -357,9 +584,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // tasks running clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 1) - assert(maxNumExecutorsNeeded(manager) == 1) - assert(removeExecutor(manager, "11")) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + assert(removeExecutorDefaultProfile(manager, "11")) onExecutorRemoved(manager, "11") // At this point, we still have 2 executors running: ["9", "12"] @@ -372,14 +599,14 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) // maxNeeded = 1, allocate one more to satisfy speculation locality requirement - assert(numExecutorsTarget(manager) === 2) - assert(maxNumExecutorsNeeded(manager) == 2) + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) post(SparkListenerTaskStart(0, 0, createTaskInfo(50, 39, executorId = "12", speculative = true))) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 1) - assert(maxNumExecutorsNeeded(manager) == 1) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) // Task 39 and 48 succeed, task 50 killed post(SparkListenerTaskEnd(0, 0, null, Success, @@ -391,11 +618,11 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerStageCompleted(stage)) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 0) - assert(maxNumExecutorsNeeded(manager) == 0) - assert(removeExecutor(manager, "9")) + assert(numExecutorsTarget(manager, defaultProfile.id) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 0) + assert(removeExecutorDefaultProfile(manager, "9")) onExecutorRemoved(manager, "9") - assert(removeExecutor(manager, "12")) + assert(removeExecutorDefaultProfile(manager, "12")) onExecutorRemoved(manager, "12") } @@ -417,43 +644,49 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerStageCompleted(stage)) // There are still two tasks that belong to the zombie stage running. - assert(totalRunningTasks(manager) === 2) + assert(totalRunningTasksPerResourceProfile(manager) === 2) // submit another attempt for the stage. We count completions from the first zombie attempt val stageAttempt1 = createStageInfo(stage.stageId, 5, attemptId = 1) post(SparkListenerStageSubmitted(stageAttempt1)) post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo1, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 1) + assert(totalRunningTasksPerResourceProfile(manager) === 1) val attemptTaskInfo1 = createTaskInfo(3, 0, "executor-1") val attemptTaskInfo2 = createTaskInfo(4, 1, "executor-1") post(SparkListenerTaskStart(0, 1, attemptTaskInfo1)) post(SparkListenerTaskStart(0, 1, attemptTaskInfo2)) - assert(totalRunningTasks(manager) === 3) + assert(totalRunningTasksPerResourceProfile(manager) === 3) post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo1, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 2) + assert(totalRunningTasksPerResourceProfile(manager) === 2) post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo2, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 1) + assert(totalRunningTasksPerResourceProfile(manager) === 1) post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo2, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 0) + assert(totalRunningTasksPerResourceProfile(manager) === 0) } testRetry("cancel pending executors when no longer needed") { val manager = createManager(createConf(0, 10, 0)) post(SparkListenerStageSubmitted(createStageInfo(2, 5))) - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) val task1Info = createTaskInfo(0, 0, "executor-1") post(SparkListenerTaskStart(2, 0, task1Info)) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) val task2Info = createTaskInfo(1, 0, "executor-1") post(SparkListenerTaskStart(2, 0, task2Info)) @@ -469,22 +702,21 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("remove executors") { val manager = createManager(createConf(5, 10, 5)) - (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + (1 to 10).map(_.toString).foreach { id => onExecutorAddedDefaultProfile(manager, id) } // Keep removing until the limit is reached assert(executorsPendingToRemove(manager).isEmpty) - assert(removeExecutor(manager, "1")) + assert(removeExecutorDefaultProfile(manager, "1")) assert(executorsPendingToRemove(manager).size === 1) assert(executorsPendingToRemove(manager).contains("1")) - assert(removeExecutor(manager, "2")) - assert(removeExecutor(manager, "3")) + assert(removeExecutorDefaultProfile(manager, "2")) + assert(removeExecutorDefaultProfile(manager, "3")) assert(executorsPendingToRemove(manager).size === 3) assert(executorsPendingToRemove(manager).contains("2")) assert(executorsPendingToRemove(manager).contains("3")) - assert(executorsPendingToRemove(manager).size === 3) - assert(removeExecutor(manager, "4")) - assert(removeExecutor(manager, "5")) - assert(!removeExecutor(manager, "6")) // reached the limit of 5 + assert(removeExecutorDefaultProfile(manager, "4")) + assert(removeExecutorDefaultProfile(manager, "5")) + assert(!removeExecutorDefaultProfile(manager, "6")) // reached the limit of 5 assert(executorsPendingToRemove(manager).size === 5) assert(executorsPendingToRemove(manager).contains("4")) assert(executorsPendingToRemove(manager).contains("5")) @@ -508,29 +740,29 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // Try removing again // This should still fail because the number pending + running is still at the limit - assert(!removeExecutor(manager, "7")) + assert(!removeExecutorDefaultProfile(manager, "7")) assert(executorsPendingToRemove(manager).isEmpty) - assert(!removeExecutor(manager, "8")) + assert(!removeExecutorDefaultProfile(manager, "8")) assert(executorsPendingToRemove(manager).isEmpty) } test("remove multiple executors") { val manager = createManager(createConf(5, 10, 5)) - (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + (1 to 10).map(_.toString).foreach { id => onExecutorAddedDefaultProfile(manager, id) } // Keep removing until the limit is reached assert(executorsPendingToRemove(manager).isEmpty) - assert(removeExecutors(manager, Seq("1")) === Seq("1")) + assert(removeExecutorsDefaultProfile(manager, Seq("1")) === Seq("1")) assert(executorsPendingToRemove(manager).size === 1) assert(executorsPendingToRemove(manager).contains("1")) - assert(removeExecutors(manager, Seq("2", "3")) === Seq("2", "3")) + assert(removeExecutorsDefaultProfile(manager, Seq("2", "3")) === Seq("2", "3")) assert(executorsPendingToRemove(manager).size === 3) assert(executorsPendingToRemove(manager).contains("2")) assert(executorsPendingToRemove(manager).contains("3")) assert(executorsPendingToRemove(manager).size === 3) - assert(removeExecutor(manager, "4")) - assert(removeExecutors(manager, Seq("5")) === Seq("5")) - assert(!removeExecutor(manager, "6")) // reached the limit of 5 + assert(removeExecutorDefaultProfile(manager, "4")) + assert(removeExecutorsDefaultProfile(manager, Seq("5")) === Seq("5")) + assert(!removeExecutorDefaultProfile(manager, "6")) // reached the limit of 5 assert(executorsPendingToRemove(manager).size === 5) assert(executorsPendingToRemove(manager).contains("4")) assert(executorsPendingToRemove(manager).contains("5")) @@ -554,87 +786,100 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // Try removing again // This should still fail because the number pending + running is still at the limit - assert(!removeExecutor(manager, "7")) + assert(!removeExecutorDefaultProfile(manager, "7")) assert(executorsPendingToRemove(manager).isEmpty) - assert(removeExecutors(manager, Seq("8")) !== Seq("8")) + assert(removeExecutorsDefaultProfile(manager, Seq("8")) !== Seq("8")) assert(executorsPendingToRemove(manager).isEmpty) } - test ("Removing with various numExecutorsTarget condition") { + test ("Removing with various numExecutorsTargetForDefaultProfileId condition") { val manager = createManager(createConf(5, 12, 5)) post(SparkListenerStageSubmitted(createStageInfo(0, 8))) - // Remove when numExecutorsTarget is the same as the current number of executors - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - (1 to 8).foreach(execId => onExecutorAdded(manager, execId.toString)) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + // Remove when numExecutorsTargetForDefaultProfileId is the same as the current + // number of executors + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + (1 to 8).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) (1 to 8).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => post(SparkListenerTaskStart(0, 0, info)) } assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 8) - assert(maxNumExecutorsNeeded(manager) == 8) - assert(!removeExecutor(manager, "1")) // won't work since numExecutorsTarget == numExecutors + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 8) + // won't work since numExecutorsTargetForDefaultProfileId == numExecutors + assert(!removeExecutorDefaultProfile(manager, "1")) - // Remove executors when numExecutorsTarget is lower than current number of executors + // Remove executors when numExecutorsTargetForDefaultProfileId is lower than + // current number of executors (1 to 3).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) } adjustRequestedExecutors(manager) assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 5) - assert(maxNumExecutorsNeeded(manager) == 5) - assert(removeExecutor(manager, "1")) - assert(removeExecutors(manager, Seq("2", "3"))=== Seq("2", "3")) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) + assert(removeExecutorDefaultProfile(manager, "1")) + assert(removeExecutorsDefaultProfile(manager, Seq("2", "3"))=== Seq("2", "3")) onExecutorRemoved(manager, "1") onExecutorRemoved(manager, "2") onExecutorRemoved(manager, "3") - // numExecutorsTarget is lower than minNumExecutors + // numExecutorsTargetForDefaultProfileId is lower than minNumExecutors post(SparkListenerTaskEnd(0, 0, null, Success, createTaskInfo(4, 4, "4"), new ExecutorMetrics, null)) assert(manager.executorMonitor.executorCount === 5) - assert(numExecutorsTarget(manager) === 5) - assert(maxNumExecutorsNeeded(manager) == 4) - assert(!removeExecutor(manager, "4")) // lower limit - assert(addExecutors(manager) === 0) // upper limit + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 4) + assert(!removeExecutorDefaultProfile(manager, "4")) // lower limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) // upper limit } test ("interleaving add and remove") { val manager = createManager(createConf(5, 12, 5)) post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Add a few executors - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - onExecutorAdded(manager, "1") - onExecutorAdded(manager, "2") - onExecutorAdded(manager, "3") - onExecutorAdded(manager, "4") - onExecutorAdded(manager, "5") - onExecutorAdded(manager, "6") - onExecutorAdded(manager, "7") - onExecutorAdded(manager, "8") + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "1") + onExecutorAddedDefaultProfile(manager, "2") + onExecutorAddedDefaultProfile(manager, "3") + onExecutorAddedDefaultProfile(manager, "4") + onExecutorAddedDefaultProfile(manager, "5") + onExecutorAddedDefaultProfile(manager, "6") + onExecutorAddedDefaultProfile(manager, "7") + onExecutorAddedDefaultProfile(manager, "8") assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 8) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) // Remove when numTargetExecutors is equal to the current number of executors - assert(!removeExecutor(manager, "1")) - assert(removeExecutors(manager, Seq("2", "3")) !== Seq("2", "3")) + assert(!removeExecutorDefaultProfile(manager, "1")) + assert(removeExecutorsDefaultProfile(manager, Seq("2", "3")) !== Seq("2", "3")) // Remove until limit - onExecutorAdded(manager, "9") - onExecutorAdded(manager, "10") - onExecutorAdded(manager, "11") - onExecutorAdded(manager, "12") + onExecutorAddedDefaultProfile(manager, "9") + onExecutorAddedDefaultProfile(manager, "10") + onExecutorAddedDefaultProfile(manager, "11") + onExecutorAddedDefaultProfile(manager, "12") assert(manager.executorMonitor.executorCount === 12) - assert(numExecutorsTarget(manager) === 8) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) - assert(removeExecutor(manager, "1")) - assert(removeExecutors(manager, Seq("2", "3", "4")) === Seq("2", "3", "4")) - assert(!removeExecutor(manager, "5")) // lower limit reached - assert(!removeExecutor(manager, "6")) + assert(removeExecutorDefaultProfile(manager, "1")) + assert(removeExecutorsDefaultProfile(manager, Seq("2", "3", "4")) === Seq("2", "3", "4")) + assert(!removeExecutorDefaultProfile(manager, "5")) // lower limit reached + assert(!removeExecutorDefaultProfile(manager, "6")) onExecutorRemoved(manager, "1") onExecutorRemoved(manager, "2") onExecutorRemoved(manager, "3") @@ -642,33 +887,36 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(manager.executorMonitor.executorCount === 8) // Add until limit - assert(!removeExecutor(manager, "7")) // still at lower limit + assert(!removeExecutorDefaultProfile(manager, "7")) // still at lower limit assert((manager, Seq("8")) !== Seq("8")) - onExecutorAdded(manager, "13") - onExecutorAdded(manager, "14") - onExecutorAdded(manager, "15") - onExecutorAdded(manager, "16") + onExecutorAddedDefaultProfile(manager, "13") + onExecutorAddedDefaultProfile(manager, "14") + onExecutorAddedDefaultProfile(manager, "15") + onExecutorAddedDefaultProfile(manager, "16") assert(manager.executorMonitor.executorCount === 12) // Remove succeeds again, now that we are no longer at the lower limit - assert(removeExecutors(manager, Seq("5", "6", "7")) === Seq("5", "6", "7")) - assert(removeExecutor(manager, "8")) + assert(removeExecutorsDefaultProfile(manager, Seq("5", "6", "7")) === Seq("5", "6", "7")) + assert(removeExecutorDefaultProfile(manager, "8")) assert(manager.executorMonitor.executorCount === 12) onExecutorRemoved(manager, "5") onExecutorRemoved(manager, "6") assert(manager.executorMonitor.executorCount === 10) - assert(numExecutorsToAdd(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) onExecutorRemoved(manager, "9") onExecutorRemoved(manager, "10") - assert(addExecutors(manager) === 4) // at upper limit - onExecutorAdded(manager, "17") - onExecutorAdded(manager, "18") + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) // at upper limit + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "17") + onExecutorAddedDefaultProfile(manager, "18") assert(manager.executorMonitor.executorCount === 10) - assert(addExecutors(manager) === 0) // still at upper limit - onExecutorAdded(manager, "19") - onExecutorAdded(manager, "20") + // still at upper limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "19") + onExecutorAddedDefaultProfile(manager, "20") assert(manager.executorMonitor.executorCount === 12) - assert(numExecutorsTarget(manager) === 12) + assert(numExecutorsTargetForDefaultProfileId(manager) === 12) } test("starting/canceling add timer") { @@ -706,22 +954,22 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(0, 20, 0), clock = clock) // No events - we should not be adding or removing - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(100L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(1000L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(10000L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) } @@ -734,43 +982,43 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { onSchedulerBacklogged(manager) clock.advance(schedulerBacklogTimeout * 1000 / 2) schedule(manager) - assert(numExecutorsTarget(manager) === 0) // timer not exceeded yet + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) // timer not exceeded yet clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1) // first timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // first timer exceeded clock.advance(sustainedSchedulerBacklogTimeout * 1000 / 2) schedule(manager) - assert(numExecutorsTarget(manager) === 1) // second timer not exceeded yet + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // second timer not exceeded yet clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1 + 2) // second timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1 + 2) // second timer exceeded clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1 + 2 + 4) // third timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1 + 2 + 4) // third timer exceeded // Scheduler queue drained onSchedulerQueueEmpty(manager) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7) // timer is canceled + assert(numExecutorsTargetForDefaultProfileId(manager) === 7) // timer is canceled clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7) // Scheduler queue backlogged again onSchedulerBacklogged(manager) clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1) // timer restarted + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1) // timer restarted clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1 + 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1 + 2) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1 + 2 + 4) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1 + 2 + 4) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 20) // limit reached + assert(numExecutorsTargetForDefaultProfileId(manager) === 20) // limit reached } test("mock polling loop remove behavior") { @@ -778,9 +1026,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(1, 20, 1), clock = clock) // Remove idle executors on timeout - onExecutorAdded(manager, "executor-1") - onExecutorAdded(manager, "executor-2") - onExecutorAdded(manager, "executor-3") + onExecutorAddedDefaultProfile(manager, "executor-1") + onExecutorAddedDefaultProfile(manager, "executor-2") + onExecutorAddedDefaultProfile(manager, "executor-3") assert(executorsPendingToRemove(manager).isEmpty) // idle threshold not reached yet @@ -796,10 +1044,10 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining) // Mark a subset as busy - only idle executors should be removed - onExecutorAdded(manager, "executor-4") - onExecutorAdded(manager, "executor-5") - onExecutorAdded(manager, "executor-6") - onExecutorAdded(manager, "executor-7") + onExecutorAddedDefaultProfile(manager, "executor-4") + onExecutorAddedDefaultProfile(manager, "executor-5") + onExecutorAddedDefaultProfile(manager, "executor-6") + onExecutorAddedDefaultProfile(manager, "executor-7") assert(manager.executorMonitor.executorCount === 7) assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed onExecutorBusy(manager, "executor-4") @@ -864,23 +1112,31 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stage1 = createStageInfo(0, 1000) post(SparkListenerStageSubmitted(stage1)) - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - assert(addExecutors(manager) === 4) - assert(addExecutors(manager) === 8) - assert(numExecutorsTarget(manager) === 15) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 8) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 15) (0 until 15).foreach { i => - onExecutorAdded(manager, s"executor-$i") + onExecutorAddedDefaultProfile(manager, s"executor-$i") } assert(manager.executorMonitor.executorCount === 15) post(SparkListenerStageCompleted(stage1)) adjustRequestedExecutors(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) post(SparkListenerStageSubmitted(createStageInfo(1, 1000))) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 16) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 16) } test("avoid ramp down initial executors until first job is submitted") { @@ -888,19 +1144,19 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(2, 5, 3), clock = clock) // Verify the initial number of executors - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) schedule(manager) // Verify whether the initial number of executors is kept with no pending tasks - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) post(SparkListenerStageSubmitted(createStageInfo(1, 2))) clock.advance(100L) - assert(maxNumExecutorsNeeded(manager) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 2) schedule(manager) // Verify that current number of executors should be ramp down when first job is submitted - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) } test("avoid ramp down initial executors until idle executor is timeout") { @@ -908,20 +1164,20 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(2, 5, 3), clock = clock) // Verify the initial number of executors - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) schedule(manager) // Verify the initial number of executors is kept when no pending tasks - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) (0 until 3).foreach { i => - onExecutorAdded(manager, s"executor-$i") + onExecutorAddedDefaultProfile(manager, s"executor-$i") } clock.advance(executorIdleTimeout * 1000) - assert(maxNumExecutorsNeeded(manager) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 0) schedule(manager) - // Verify executor is timeout,numExecutorsTarget is recalculated - assert(numExecutorsTarget(manager) === 2) + // Verify executor is timeout,numExecutorsTargetForDefaultProfileId is recalculated + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) } test("get pending task number and related locality preference") { @@ -937,7 +1193,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stageInfo1 = createStageInfo(1, 5, localityPreferences1) post(SparkListenerStageSubmitted(stageInfo1)) - assert(localityAwareTasks(manager) === 3) + assert(localityAwareTasksForDefaultProfile(manager) === 3) + val hostToLocal = hostToLocalTaskCount(manager) assert(hostToLocalTaskCount(manager) === Map("host1" -> 2, "host2" -> 3, "host3" -> 2, "host4" -> 2)) @@ -949,67 +1206,76 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stageInfo2 = createStageInfo(2, 3, localityPreferences2) post(SparkListenerStageSubmitted(stageInfo2)) - assert(localityAwareTasks(manager) === 5) + assert(localityAwareTasksForDefaultProfile(manager) === 5) assert(hostToLocalTaskCount(manager) === Map("host1" -> 2, "host2" -> 4, "host3" -> 4, "host4" -> 3, "host5" -> 2)) post(SparkListenerStageCompleted(stageInfo1)) - assert(localityAwareTasks(manager) === 2) + assert(localityAwareTasksForDefaultProfile(manager) === 2) assert(hostToLocalTaskCount(manager) === Map("host2" -> 1, "host3" -> 2, "host4" -> 1, "host5" -> 2)) } - test("SPARK-8366: maxNumExecutorsNeeded should properly handle failed tasks") { + test("SPARK-8366: maxNumExecutorsNeededPerResourceProfile should properly handle failed tasks") { val manager = createManager(createConf()) - assert(maxNumExecutorsNeeded(manager) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 0) post(SparkListenerStageSubmitted(createStageInfo(0, 1))) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) val taskInfo = createTaskInfo(1, 1, "executor-1") post(SparkListenerTaskStart(0, 0, taskInfo)) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) // If the task is failed, we expect it to be resubmitted later. val taskEndReason = ExceptionFailure(null, null, null, null, None) post(SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, new ExecutorMetrics, null)) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) } test("reset the state of allocation manager") { val manager = createManager(createConf()) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] // Allocation manager is reset when adding executor requests are sent without reporting back // executor added. post(SparkListenerStageSubmitted(createStageInfo(0, 10))) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 4) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(manager.executorMonitor.executorCount === 0) // Allocation manager is reset when executors are added. post(SparkListenerStageSubmitted(createStageInfo(0, 10))) - addExecutors(manager) - addExecutors(manager) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 5) - - onExecutorAdded(manager, "first") - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - onExecutorAdded(manager, "fifth") + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAddedDefaultProfile(manager, "fifth") assert(manager.executorMonitor.executorCount === 5) // Cluster manager lost will make all the live executors lost, so here simulate this behavior @@ -1020,28 +1286,31 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { onExecutorRemoved(manager, "fifth") manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(manager.executorMonitor.executorCount === 0) // Allocation manager is reset when executors are pending to remove - addExecutors(manager) - addExecutors(manager) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 5) - - onExecutorAdded(manager, "first") - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - onExecutorAdded(manager, "fifth") - onExecutorAdded(manager, "sixth") - onExecutorAdded(manager, "seventh") - onExecutorAdded(manager, "eighth") + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAddedDefaultProfile(manager, "fifth") + onExecutorAddedDefaultProfile(manager, "sixth") + onExecutorAddedDefaultProfile(manager, "seventh") + onExecutorAddedDefaultProfile(manager, "eighth") assert(manager.executorMonitor.executorCount === 8) - removeExecutor(manager, "first") - removeExecutors(manager, Seq("second", "third")) + removeExecutorDefaultProfile(manager, "first") + removeExecutorsDefaultProfile(manager, Seq("second", "third")) assert(executorsPendingToRemove(manager) === Set("first", "second", "third")) assert(manager.executorMonitor.executorCount === 8) @@ -1055,8 +1324,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(executorsPendingToRemove(manager) === Set.empty) assert(manager.executorMonitor.executorCount === 0) } @@ -1067,31 +1336,31 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { createConf(1, 2, 1).set(config.DYN_ALLOCATION_TESTING, false), clock = clock) - when(client.requestTotalExecutors(meq(2), any(), any())).thenReturn(true) + when(client.requestTotalExecutors(any(), any(), any())).thenReturn(true) // test setup -- job with 2 tasks, scale up to two executors - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(SparkListenerStageSubmitted(createStageInfo(0, 2))) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) val taskInfo0 = createTaskInfo(0, 0, "executor-1") post(SparkListenerTaskStart(0, 0, taskInfo0)) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) val taskInfo1 = createTaskInfo(1, 1, "executor-2") post(SparkListenerTaskStart(0, 0, taskInfo1)) - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) // have one task finish -- we should adjust the target number of executors down // but we should *not* kill any executors yet post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo0, new ExecutorMetrics, null)) - assert(maxNumExecutorsNeeded(manager) === 1) - assert(numExecutorsTarget(manager) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) verify(client, never).killExecutors(any(), any(), any(), any()) // now we cross the idle timeout for executor-1, so we kill it. the really important @@ -1101,8 +1370,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { .thenReturn(Seq("executor-1")) clock.advance(3000) schedule(manager) - assert(maxNumExecutorsNeeded(manager) === 1) - assert(numExecutorsTarget(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // here's the important verify -- we did kill the executors, but did not adjust the target count verify(client).killExecutors(Seq("executor-1"), false, false, false) } @@ -1110,7 +1379,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("SPARK-26758 check executor target number after idle time out ") { val clock = new ManualClock(10000L) val manager = createManager(createConf(1, 5, 3), clock = clock) - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty))) post(SparkListenerExecutorAdded( @@ -1121,14 +1390,14 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { clock.advance(executorIdleTimeout * 1000) schedule(manager) // once the schedule is run target executor number should be 1 - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) } private def createConf( minExecutors: Int = 1, maxExecutors: Int = 5, initialExecutors: Int = 1): SparkConf = { - new SparkConf() + val sparkConf = new SparkConf() .set(config.DYN_ALLOCATION_ENABLED, true) .set(config.DYN_ALLOCATION_MIN_EXECUTORS, minExecutors) .set(config.DYN_ALLOCATION_MAX_EXECUTORS, maxExecutors) @@ -1143,12 +1412,16 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // SPARK-22864: effectively disable the allocation schedule by setting the period to a // really long value. .set(TEST_SCHEDULE_INTERVAL, 10000L) + sparkConf } private def createManager( conf: SparkConf, clock: Clock = new SystemClock()): ExecutorAllocationManager = { - val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock) + ResourceProfile.reInitDefaultProfile(conf) + rpManager = new ResourceProfileManager(conf) + val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock, + resourceProfileManager = rpManager) managers += manager manager.start() manager @@ -1157,7 +1430,18 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private val execInfo = new ExecutorInfo("host1", 1, Map.empty, Map.empty, Map.empty, DEFAULT_RESOURCE_PROFILE_ID) - private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { + private def onExecutorAddedDefaultProfile( + manager: ExecutorAllocationManager, + id: String): Unit = { + post(SparkListenerExecutorAdded(0L, id, execInfo)) + } + + private def onExecutorAdded( + manager: ExecutorAllocationManager, + id: String, + rp: ResourceProfile): Unit = { + val cores = rp.getExecutorCores.getOrElse(1) + val execInfo = new ExecutorInfo("host1", cores, Map.empty, Map.empty, Map.empty, rp.id) post(SparkListenerExecutorAdded(0L, id, execInfo)) } @@ -1176,8 +1460,18 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerTaskEnd(1, 1, "foo", Success, info, new ExecutorMetrics, null)) } - private def removeExecutor(manager: ExecutorAllocationManager, executorId: String): Boolean = { - val executorsRemoved = removeExecutors(manager, Seq(executorId)) + private def removeExecutorDefaultProfile( + manager: ExecutorAllocationManager, + executorId: String): Boolean = { + val executorsRemoved = removeExecutorsDefaultProfile(manager, Seq(executorId)) + executorsRemoved.nonEmpty && executorsRemoved(0) == executorId + } + + private def removeExecutor( + manager: ExecutorAllocationManager, + executorId: String, + rpId: Int): Boolean = { + val executorsRemoved = removeExecutors(manager, Seq((executorId, rpId))) executorsRemoved.nonEmpty && executorsRemoved(0) == executorId } @@ -1199,10 +1493,11 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { stageId: Int, numTasks: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, - attemptId: Int = 0 + attemptId: Int = 0, + rp: ResourceProfile = defaultProfile ): StageInfo = { new StageInfo(stageId, attemptId, "name", numTasks, Seq.empty, Seq.empty, "no details", - taskLocalityPreferences = taskLocalityPreferences) + taskLocalityPreferences = taskLocalityPreferences, resourceProfileId = rp.id) } private def createTaskInfo( @@ -1217,54 +1512,117 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { | Helper methods for accessing private methods and fields | * ------------------------------------------------------- */ - private val _numExecutorsToAdd = PrivateMethod[Int](Symbol("numExecutorsToAdd")) - private val _numExecutorsTarget = PrivateMethod[Int](Symbol("numExecutorsTarget")) - private val _maxNumExecutorsNeeded = PrivateMethod[Int](Symbol("maxNumExecutorsNeeded")) + private val _numExecutorsToAddPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]]( + Symbol("numExecutorsToAddPerResourceProfileId")) + private val _numExecutorsTargetPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]]( + Symbol("numExecutorsTargetPerResourceProfileId")) + private val _maxNumExecutorsNeededPerResourceProfile = + PrivateMethod[Int](Symbol("maxNumExecutorsNeededPerResourceProfile")) private val _addTime = PrivateMethod[Long](Symbol("addTime")) private val _schedule = PrivateMethod[Unit](Symbol("schedule")) - private val _addExecutors = PrivateMethod[Int](Symbol("addExecutors")) + private val _doUpdateRequest = PrivateMethod[Unit](Symbol("doUpdateRequest")) private val _updateAndSyncNumExecutorsTarget = PrivateMethod[Int](Symbol("updateAndSyncNumExecutorsTarget")) + private val _addExecutorsToTarget = PrivateMethod[Int](Symbol("addExecutorsToTarget")) private val _removeExecutors = PrivateMethod[Seq[String]](Symbol("removeExecutors")) private val _onSchedulerBacklogged = PrivateMethod[Unit](Symbol("onSchedulerBacklogged")) private val _onSchedulerQueueEmpty = PrivateMethod[Unit](Symbol("onSchedulerQueueEmpty")) - private val _localityAwareTasks = PrivateMethod[Int](Symbol("localityAwareTasks")) - private val _hostToLocalTaskCount = - PrivateMethod[Map[String, Int]](Symbol("hostToLocalTaskCount")) + private val _localityAwareTasksPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]](Symbol("numLocalityAwareTasksPerResourceProfileId")) + private val _rpIdToHostToLocalTaskCount = + PrivateMethod[Map[Int, Map[String, Int]]](Symbol("rpIdToHostToLocalTaskCount")) private val _onSpeculativeTaskSubmitted = PrivateMethod[Unit](Symbol("onSpeculativeTaskSubmitted")) - private val _totalRunningTasks = PrivateMethod[Int](Symbol("totalRunningTasks")) + private val _totalRunningTasksPerResourceProfile = + PrivateMethod[Int](Symbol("totalRunningTasksPerResourceProfile")) + + private val defaultProfile = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) + + private def numExecutorsToAddForDefaultProfile(manager: ExecutorAllocationManager): Int = { + numExecutorsToAdd(manager, defaultProfile) + } + + private def numExecutorsToAdd( + manager: ExecutorAllocationManager, + rp: ResourceProfile): Int = { + val nmap = manager invokePrivate _numExecutorsToAddPerResourceProfileId() + nmap(rp.id) + } + + private def updateAndSyncNumExecutorsTarget( + manager: ExecutorAllocationManager, + now: Long): Unit = { + manager invokePrivate _updateAndSyncNumExecutorsTarget(now) + } + + private def numExecutorsTargetForDefaultProfileId(manager: ExecutorAllocationManager): Int = { + numExecutorsTarget(manager, defaultProfile.id) + } - private def numExecutorsToAdd(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _numExecutorsToAdd() + private def numExecutorsTarget( + manager: ExecutorAllocationManager, + rpId: Int): Int = { + val numMap = manager invokePrivate _numExecutorsTargetPerResourceProfileId() + numMap(rpId) } - private def numExecutorsTarget(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _numExecutorsTarget() + private def addExecutorsToTargetForDefaultProfile( + manager: ExecutorAllocationManager, + updatesNeeded: mutable.HashMap[ResourceProfile, + ExecutorAllocationManager.TargetNumUpdates] + ): Int = { + addExecutorsToTarget(manager, updatesNeeded, defaultProfile) + } + + private def addExecutorsToTarget( + manager: ExecutorAllocationManager, + updatesNeeded: mutable.HashMap[ResourceProfile, + ExecutorAllocationManager.TargetNumUpdates], + rp: ResourceProfile + ): Int = { + val maxNumExecutorsNeeded = + manager invokePrivate _maxNumExecutorsNeededPerResourceProfile(rp.id) + manager invokePrivate + _addExecutorsToTarget(maxNumExecutorsNeeded, rp.id, updatesNeeded) } private def addTime(manager: ExecutorAllocationManager): Long = { manager invokePrivate _addTime() } - private def schedule(manager: ExecutorAllocationManager): Unit = { - manager invokePrivate _schedule() + private def doUpdateRequest( + manager: ExecutorAllocationManager, + updates: Map[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates], + now: Long): Unit = { + manager invokePrivate _doUpdateRequest(updates, now) } - private def maxNumExecutorsNeeded(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _maxNumExecutorsNeeded() + private def schedule(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _schedule() } - private def addExecutors(manager: ExecutorAllocationManager): Int = { - val maxNumExecutorsNeeded = manager invokePrivate _maxNumExecutorsNeeded() - manager invokePrivate _addExecutors(maxNumExecutorsNeeded) + private def maxNumExecutorsNeededPerResourceProfile( + manager: ExecutorAllocationManager, + rp: ResourceProfile): Int = { + manager invokePrivate _maxNumExecutorsNeededPerResourceProfile(rp.id) } private def adjustRequestedExecutors(manager: ExecutorAllocationManager): Int = { manager invokePrivate _updateAndSyncNumExecutorsTarget(0L) } - private def removeExecutors(manager: ExecutorAllocationManager, ids: Seq[String]): Seq[String] = { + private def removeExecutorsDefaultProfile( + manager: ExecutorAllocationManager, + ids: Seq[String]): Seq[String] = { + val idsAndProfileIds = ids.map((_, defaultProfile.id)) + manager invokePrivate _removeExecutors(idsAndProfileIds) + } + + private def removeExecutors( + manager: ExecutorAllocationManager, + ids: Seq[(String, Int)]): Seq[String] = { manager invokePrivate _removeExecutors(ids) } @@ -1280,15 +1638,22 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _onSpeculativeTaskSubmitted(id) } - private def localityAwareTasks(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _localityAwareTasks() + private def localityAwareTasksForDefaultProfile(manager: ExecutorAllocationManager): Int = { + val localMap = manager invokePrivate _localityAwareTasksPerResourceProfileId() + localMap(defaultProfile.id) + } + + private def totalRunningTasksPerResourceProfile(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _totalRunningTasksPerResourceProfile(defaultProfile.id) } - private def totalRunningTasks(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _totalRunningTasks() + private def hostToLocalTaskCount( + manager: ExecutorAllocationManager): Map[String, Int] = { + val rpIdToHostLocal = manager invokePrivate _rpIdToHostToLocalTaskCount() + rpIdToHostLocal(defaultProfile.id) } - private def hostToLocalTaskCount(manager: ExecutorAllocationManager): Map[String, Int] = { - manager invokePrivate _hostToLocalTaskCount() + private def getResourceProfileIdOfExecutor(manager: ExecutorAllocationManager): Int = { + defaultProfile.id } } diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index ff0f2f9134ed3..a9296955d18b4 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -285,9 +285,14 @@ private class FakeSchedulerBackend( clusterManagerEndpoint: RpcEndpointRef) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { - protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + protected override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { clusterManagerEndpoint.ask[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, Set.empty)) + RequestExecutors( + resourceProfileToTotalExecs(ResourceProfile.getOrCreateDefaultProfile(conf)), + numLocalityAwareTasksPerResourceProfileId(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + rpHostToLocalTaskCount(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + Set.empty)) } protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 1fe12e116d96e..599ea8955491f 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -44,7 +44,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self def resetSparkContext(): Unit = { LocalSparkContext.stop(sc) - ResourceProfile.clearDefaultProfile + ResourceProfile.clearDefaultProfile() sc = null } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index df9c7c5eaa368..b6dfa69015c28 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -36,6 +36,7 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.internal.config.UI._ import org.apache.spark.resource.ResourceAllocation import org.apache.spark.resource.ResourceUtils._ @@ -784,7 +785,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test(s"Avoid setting ${CPUS_PER_TASK.key} unreasonably (SPARK-27192)") { - val FAIL_REASON = s"has to be >= the task config: ${CPUS_PER_TASK.key}" + val FAIL_REASON = " has to be >= the number of cpus per task" Seq( ("local", 2, None), ("local[2]", 3, None), @@ -864,9 +865,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) }.getMessage() - assert(error.contains("The executor resource config: spark.executor.resource.gpu.amount " + - "needs to be specified since a task requirement config: spark.task.resource.gpu.amount " + - "was specified")) + assert(error.contains("No executor resource configs were not specified for the following " + + "task configs: gpu")) } test("Test parsing resources executor config < task requirements") { @@ -880,15 +880,15 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) }.getMessage() - assert(error.contains("The executor resource config: spark.executor.resource.gpu.amount = 1 " + - "has to be >= the requested amount in task resource config: " + - "spark.task.resource.gpu.amount = 2")) + assert(error.contains("The executor resource: gpu, amount: 1 needs to be >= the task " + + "resource request amount of 2.0")) } test("Parse resources executor config not the same multiple numbers of the task requirements") { val conf = new SparkConf() .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") + conf.set(RESOURCES_WARNING_TESTING, true) conf.set(TASK_GPU_ID.amountConf, "2") conf.set(EXECUTOR_GPU_ID.amountConf, "4") @@ -897,25 +897,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu }.getMessage() assert(error.contains( - "The configuration of resource: gpu (exec = 4, task = 2, runnable tasks = 2) will result " + - "in wasted resources due to resource CPU limiting the number of runnable tasks per " + - "executor to: 1. Please adjust your configuration.")) - } - - test("Parse resources executor config cpus not limiting resource") { - val conf = new SparkConf() - .setMaster("local-cluster[1, 8, 1024]") - .setAppName("test-cluster") - conf.set(TASK_GPU_ID.amountConf, "2") - conf.set(EXECUTOR_GPU_ID.amountConf, "4") - - var error = intercept[IllegalArgumentException] { - sc = new SparkContext(conf) - }.getMessage() - - assert(error.contains("The number of slots on an executor has to be " + - "limited by the number of cores, otherwise you waste resources and " + - "dynamic allocation doesn't work properly")) + "The configuration of resource: gpu (exec = 4, task = 2.0/1, runnable tasks = 2) will " + + "result in wasted resources due to resource cpus limiting the number of runnable " + + "tasks per executor to: 1. Please adjust your configuration.")) } test("test resource scheduling under local-cluster mode") { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala index 86511ae08784a..c905797bf1287 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.history import org.apache.spark.{SparkFunSuite, Success, TaskResultLost, TaskState} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.status.ListenerEventsTestHelper @@ -141,7 +142,8 @@ class BasicEventFilterBuilderSuite extends SparkFunSuite { // - Re-submit stage 1, all tasks, and succeed them and the stage. val oldS1 = stages.last val newS1 = new StageInfo(oldS1.stageId, oldS1.attemptNumber + 1, oldS1.name, oldS1.numTasks, - oldS1.rddInfos, oldS1.parentIds, oldS1.details, oldS1.taskMetrics) + oldS1.rddInfos, oldS1.parentIds, oldS1.details, oldS1.taskMetrics, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 newS1.submissionTime = Some(time) diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala new file mode 100644 index 0000000000000..075260317284d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ + +class ResourceProfileManagerSuite extends SparkFunSuite { + + override def beforeAll() { + try { + ResourceProfile.clearDefaultProfile() + } finally { + super.beforeAll() + } + } + + override def afterEach() { + try { + ResourceProfile.clearDefaultProfile() + } finally { + super.afterEach() + } + } + + test("ResourceProfileManager") { + val conf = new SparkConf().set(EXECUTOR_CORES, 4) + val rpmanager = new ResourceProfileManager(conf) + val defaultProf = rpmanager.defaultResourceProfile + assert(defaultProf.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + assert(defaultProf.executorResources.size === 2, + "Executor resources should contain cores and memory by default") + assert(defaultProf.executorResources(ResourceProfile.CORES).amount === 4, + s"Executor resources should have 4 cores") + } + + test("isSupported yarn no dynamic allocation") { + val conf = new SparkConf().setMaster("yarn").set(EXECUTOR_CORES, 4) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + val error = intercept[SparkException] { + rpmanager.isSupported(immrprof) + }.getMessage() + + assert(error.contains("ResourceProfiles are only supported on YARN with dynamic allocation")) + } + + test("isSupported yarn with dynamic allocation") { + val conf = new SparkConf().setMaster("yarn").set(EXECUTOR_CORES, 4) + conf.set(DYN_ALLOCATION_ENABLED, true) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + assert(rpmanager.isSupported(immrprof) == true) + } + + test("isSupported yarn with local mode") { + val conf = new SparkConf().setMaster("local").set(EXECUTOR_CORES, 4) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + var error = intercept[SparkException] { + rpmanager.isSupported(immrprof) + }.getMessage() + + assert(error.contains("ResourceProfiles are only supported on YARN with dynamic allocation")) + } + + + +} diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index c0637eeeacaba..b2f2c3632e454 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -18,18 +18,28 @@ package org.apache.spark.resource import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD, SPARK_EXECUTOR_PREFIX} +import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.resource.TestResourceIDs._ class ResourceProfileSuite extends SparkFunSuite { + override def beforeAll() { + try { + ResourceProfile.clearDefaultProfile() + } finally { + super.beforeAll() + } + } + override def afterEach() { try { - ResourceProfile.clearDefaultProfile + ResourceProfile.clearDefaultProfile() } finally { super.afterEach() } } + test("Default ResourceProfile") { val rprof = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) @@ -59,18 +69,19 @@ class ResourceProfileSuite extends SparkFunSuite { conf.set(EXECUTOR_MEMORY_OVERHEAD.key, "1g") conf.set(EXECUTOR_MEMORY.key, "4g") conf.set(EXECUTOR_CORES.key, "4") - conf.set("spark.task.resource.gpu.amount", "1") - conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.amount", "1") - conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.discoveryScript", "nameOfScript") + conf.set(TASK_GPU_ID.amountConf, "1") + conf.set(EXECUTOR_GPU_ID.amountConf, "1") + conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, "nameOfScript") val rprof = ResourceProfile.getOrCreateDefaultProfile(conf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val execResources = rprof.executorResources - assert(execResources.size === 5, - "Executor resources should contain cores, memory, and gpu " + execResources) + assert(execResources.size === 5, s"Executor resources should contain cores, pyspark " + + s"memory, memory overhead, memory, and gpu $execResources") assert(execResources.contains("gpu"), "Executor resources should have gpu") assert(rprof.executorResources(ResourceProfile.CORES).amount === 4, "Executor resources should have 4 core") - assert(rprof.getExecutorCores.get === 4, "Executor resources should have 4 core") + assert(rprof.getExecutorCores.get === 4, + "Executor resources should have 4 core") assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, "Executor resources should have 1024 memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount == 2048, @@ -84,12 +95,60 @@ class ResourceProfileSuite extends SparkFunSuite { test("test default profile task gpus fractional") { val sparkConf = new SparkConf() - .set("spark.executor.resource.gpu.amount", "2") - .set("spark.task.resource.gpu.amount", "0.33") + .set(EXECUTOR_GPU_ID.amountConf, "2") + .set(TASK_GPU_ID.amountConf, "0.33") val immrprof = ResourceProfile.getOrCreateDefaultProfile(sparkConf) assert(immrprof.taskResources.get("gpu").get.amount == 0.33) } + test("maxTasksPerExecutor cpus") { + val sparkConf = new SparkConf() + .set(EXECUTOR_CORES, 1) + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "cpus") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 1) + } + + test("maxTasksPerExecutor/limiting no executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing") + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "gpu") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 2) + assert(immrprof.isCoresLimitKnown == false) + } + + test("maxTasksPerExecutor/limiting no other resource no executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing") + val immrprof = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + assert(immrprof.limitingResource(sparkConf) == "") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 1) + assert(immrprof.isCoresLimitKnown == false) + } + + test("maxTasksPerExecutor/limiting executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing").set(EXECUTOR_CORES, 2) + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == ResourceProfile.CPUS) + assert(immrprof.maxTasksPerExecutor(sparkConf) == 2) + assert(immrprof.isCoresLimitKnown == true) + } + + test("Create ResourceProfile") { val rprof = new ResourceProfileBuilder() val taskReq = new TaskResourceRequests().resource("gpu", 1) diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala index dffe9a02e9aa4..278a72a7192d8 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -26,8 +26,10 @@ import org.json4s.{DefaultFormats, Extraction} import org.apache.spark.{LocalSparkContext, SparkConf, SparkException, SparkFunSuite} import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.util.Utils class ResourceUtilsSuite extends SparkFunSuite @@ -165,6 +167,7 @@ class ResourceUtilsSuite extends SparkFunSuite val rpBuilder = new ResourceProfileBuilder() val ereqs = new ExecutorResourceRequests().resource(GPU, 2, gpuDiscovery) val treqs = new TaskResourceRequests().resource(GPU, 1) + val rp = rpBuilder.require(ereqs).require(treqs).build val resourcesFromBoth = getOrDiscoverAllResourcesForResourceProfile( Some(resourcesFile), SPARK_EXECUTOR_PREFIX, rp, conf) diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index c063301673598..7666c6c7810cc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.rdd.RDD -import org.apache.spark.resource.{ResourceInformation, ResourceProfile} +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceInformation, ResourceProfile, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} @@ -187,8 +187,6 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo } test("extra resources from executor") { - import TestUtils._ - val conf = new SparkConf() .set(EXECUTOR_CORES, 1) .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test @@ -200,6 +198,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo conf.set(EXECUTOR_GPU_ID.amountConf, "1") sc = new SparkContext(conf) + val execGpu = new ExecutorResourceRequests().cores(1).resource(GPU, 3) + val taskGpu = new TaskResourceRequests().cpus(1).resource(GPU, 1) + val rp = new ResourceProfile(execGpu.requests, taskGpu.requests) + sc.resourceProfileManager.addResourceProfile(rp) + assert(rp.id > ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] @@ -224,7 +227,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) backend.driverEndpoint.askSync[Boolean]( RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources, - 5)) + rp.id)) val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) val bytebuffer = java.nio.ByteBuffer.allocate(frameSize - 100) @@ -234,7 +237,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")) var exec3ResourceProfileId = backend.getExecutorResourceProfileId("3") - assert(exec3ResourceProfileId === 5) + assert(exec3ResourceProfileId === rp.id) val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 286924001e920..61ea21fa86c5a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -38,6 +38,7 @@ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} @@ -438,12 +439,14 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit private def createStageSubmittedEvent(stageId: Int) = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } private def createStageCompletedEvent(stageId: Int) = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } private def createExecutorAddedEvent(executorId: Int) = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index 615389ae5c2d4..3596a9ebb1f5a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark._ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceProfile.{DEFAULT_RESOURCE_PROFILE_ID, UNKNOWN_RESOURCE_PROFILE_ID} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ @@ -255,25 +256,28 @@ class ExecutorMonitorSuite extends SparkFunSuite { test("track executors pending for removal") { knownExecs ++= Set("1", "2", "3") + val execInfoRp1 = new ExecutorInfo("host1", 1, Map.empty, + Map.empty, Map.empty, 1) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", execInfo)) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", execInfo)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", execInfoRp1)) clock.setTime(idleDeadline) - assert(monitor.timedOutExecutors().toSet === Set("1", "2", "3")) + assert(monitor.timedOutExecutors().toSet === Set(("1", 0), ("2", 0), ("3", 1))) assert(monitor.pendingRemovalCount === 0) // Notify that only a subset of executors was killed, to mimic the case where the scheduler // refuses to kill an executor that is busy for whatever reason the monitor hasn't detected yet. monitor.executorsKilled(Seq("1")) - assert(monitor.timedOutExecutors().toSet === Set("2", "3")) + assert(monitor.timedOutExecutors().toSet === Set(("2", 0), ("3", 1))) assert(monitor.pendingRemovalCount === 1) // Check the timed out executors again so that we're sure they're still timed out when no // events happen. This ensures that the monitor doesn't lose track of them. - assert(monitor.timedOutExecutors().toSet === Set("2", "3")) + assert(monitor.timedOutExecutors().toSet === Set(("2", 0), ("3", 1))) monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("2", 1))) - assert(monitor.timedOutExecutors().toSet === Set("3")) + assert(monitor.timedOutExecutors().toSet === Set(("3", 1))) monitor.executorsKilled(Seq("3")) assert(monitor.pendingRemovalCount === 2) @@ -282,7 +286,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { new ExecutorMetrics, null)) assert(monitor.timedOutExecutors().isEmpty) clock.advance(idleDeadline) - assert(monitor.timedOutExecutors().toSet === Set("2")) + assert(monitor.timedOutExecutors().toSet === Set(("2", 0))) } test("shuffle block tracking") { @@ -435,7 +439,8 @@ class ExecutorMonitorSuite extends SparkFunSuite { private def stageInfo(id: Int, shuffleId: Int = -1): StageInfo = { new StageInfo(id, 0, s"stage$id", 1, Nil, Nil, "", - shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None) + shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } private def taskInfo( diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 255f91866ef58..24eb1685f577a 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.Status._ import org.apache.spark.metrics.ExecutorMetricType +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.ListenerEventsTestHelper._ @@ -151,8 +152,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Start a job with 2 stages / 4 tasks each time += 1 val stages = Seq( - new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2")) + new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) val jobProps = new Properties() jobProps.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, "jobDescription") @@ -524,7 +527,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // - Re-submit stage 2, all tasks, and succeed them and the stage. val oldS2 = stages.last val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptNumber + 1, oldS2.name, oldS2.numTasks, - oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics) + oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 newS2.submissionTime = Some(time) @@ -575,8 +579,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // change the stats of the already finished job. time += 1 val j2Stages = Seq( - new StageInfo(3, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(4, 0, "stage2", 4, Nil, Seq(3), "details2")) + new StageInfo(3, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(4, 0, "stage2", 4, Nil, Seq(3), "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) j2Stages.last.submissionTime = Some(time) listener.onJobStart(SparkListenerJobStart(2, time, j2Stages, null)) assert(store.count(classOf[JobDataWrapper]) === 2) @@ -703,7 +709,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage for the first RDD before it's marked for caching, to make sure later // the listener picks up the correct storage level. val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, StorageLevel.NONE, false, Nil) - val stage0 = new StageInfo(0, 0, "stage0", 4, Seq(rdd1Info), Nil, "details0") + val stage0 = new StageInfo(0, 0, "stage0", 4, Seq(rdd1Info), Nil, "details0", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage0, new Properties())) listener.onStageCompleted(SparkListenerStageCompleted(stage0)) assert(store.count(classOf[RDDStorageInfoWrapper]) === 0) @@ -711,7 +718,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage and make sure the RDDs are recorded. rdd1Info.storageLevel = level val rdd2Info = new RDDInfo(rdd2b1.rddId, "rdd2", 1, level, false, Nil) - val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info, rdd2Info), Nil, "details1") + val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info, rdd2Info), Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => @@ -1018,9 +1026,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // data is not deleted. time += 1 val stages = Seq( - new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2"), - new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3")) + new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) // Graph data is generated by the job start event, so fire it. listener.onJobStart(SparkListenerJobStart(4, time, stages, null)) @@ -1068,7 +1079,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } assert(store.count(classOf[CachedQuantile], "stage", key(dropped)) === 0) - val attempt2 = new StageInfo(3, 1, "stage3", 4, Nil, Nil, "details3") + val attempt2 = new StageInfo(3, 1, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 attempt2.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(attempt2, new Properties())) @@ -1139,9 +1151,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Start stage 1 and stage 2 time += 1 @@ -1172,8 +1187,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Sart job 1 time += 1 @@ -1193,7 +1210,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded)) // Submit stage 3 and verify stage 2 is evicted - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 stage3.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(stage3, new Properties())) @@ -1208,7 +1226,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_TASKS_PER_STAGE, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stage1.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties())) @@ -1243,9 +1262,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val listener = new AppStatusListener(store, testConf, true) val appStore = new AppStatusStore(store) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 stage1.submissionTime = Some(time) @@ -1274,8 +1296,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { test("SPARK-24415: update metrics for tasks that finish late") { val listener = new AppStatusListener(store, conf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Start job listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage1, stage2), null)) @@ -1340,7 +1364,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onExecutorAdded(createExecutorAddedEvent(1)) listener.onExecutorAdded(createExecutorAddedEvent(2)) - val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") + val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) @@ -1577,7 +1602,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage and make sure the RDDs are recorded. val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, false, Nil) - val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info), Nil, "details1") + val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info), Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) // Add partition 1 replicated on two block managers. diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala index 4b3fbacc47f9c..99c0d9593ccae 100644 --- a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala @@ -23,6 +23,7 @@ import scala.collection.immutable.Map import org.apache.spark.{AccumulatorSuite, SparkContext, Success, TaskState} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskStart, StageInfo, TaskInfo, TaskLocality} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{RDDInfo, StorageLevel} @@ -61,7 +62,8 @@ object ListenerEventsTestHelper { } def createStage(id: Int, rdds: Seq[RDDInfo], parentIds: Seq[Int]): StageInfo = { - new StageInfo(id, 0, s"stage${id}", 4, rdds, parentIds, s"details${id}") + new StageInfo(id, 0, s"stage${id}", 4, rdds, parentIds, s"details${id}", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } def createStage(rdds: Seq[RDDInfo], parentIds: Seq[Int]): StageInfo = { @@ -96,13 +98,15 @@ object ListenerEventsTestHelper { /** Create a stage submitted event for the specified stage Id. */ def createStageSubmittedEvent(stageId: Int): SparkListenerStageSubmitted = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } /** Create a stage completed event for the specified stage Id. */ def createStageCompletedEvent(stageId: Int): SparkListenerStageCompleted = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } def createExecutorAddedEvent(executorId: Int): SparkListenerExecutorAdded = { diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index bd18e9e628da8..7711934cbe8a6 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.Status._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1.{AccumulableInfo => UIAccumulableInfo, StageData, StageStatus} @@ -131,7 +132,8 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { val page = new StagePage(tab, statusStore) // Simulate a stage in job progress listener - val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") + val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness (1 to 2).foreach { taskId => diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index a2a4b3aa974fc..edc0662a0f73e 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -32,8 +32,7 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope -import org.apache.spark.resource.ResourceInformation -import org.apache.spark.resource.ResourceUtils +import org.apache.spark.resource.{ResourceInformation, ResourceProfile, ResourceUtils} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.shuffle.MetadataFetchFailedException @@ -341,7 +340,8 @@ class JsonProtocolSuite extends SparkFunSuite { val stageIds = Seq[Int](1, 2, 3, 4) val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) val dummyStageInfos = - stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown")) + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) val jobStart = SparkListenerJobStart(10, jobSubmissionTime, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) val expectedJobStart = @@ -383,9 +383,11 @@ class JsonProtocolSuite extends SparkFunSuite { test("StageInfo backward compatibility (parent IDs)") { // Prior to Spark 1.4.0, StageInfo did not have the "Parent IDs" property - val stageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq(1, 2, 3), "details") + val stageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq(1, 2, 3), "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val oldStageInfo = JsonProtocol.stageInfoToJson(stageInfo).removeField({ _._1 == "Parent IDs"}) - val expectedStageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq.empty, "details") + val expectedStageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) assertEquals(expectedStageInfo, JsonProtocol.stageInfoFromJson(oldStageInfo)) } @@ -873,7 +875,8 @@ private[spark] object JsonProtocolSuite extends Assertions { private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) } - val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, Seq(100, 200, 300), "details") + val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, Seq(100, 200, 300), "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val (acc1, acc2) = (makeAccumulableInfo(1), makeAccumulableInfo(2)) stageInfo.accumulables(acc1.id) = acc1 stageInfo.accumulables(acc2.id) = acc2 diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index c7f435a582210..edfea42bed71d 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -275,9 +275,13 @@ def setUp(self): self.tempFile = tempfile.NamedTemporaryFile(delete=False) self.tempFile.write(b'echo {\\"name\\": \\"gpu\\", \\"addresses\\": [\\"0\\"]}') self.tempFile.close() + # create temporary directory for Worker resources coordination + self.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(self.tempdir.name) os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP | stat.S_IROTH | stat.S_IXOTH) conf = SparkConf().set("spark.test.home", SPARK_HOME) + conf = conf.set("spark.resources.dir", self.tempdir.name) conf = conf.set("spark.driver.resource.gpu.amount", "1") conf = conf.set("spark.driver.resource.gpu.discoveryScript", self.tempFile.name) self.sc = SparkContext('local-cluster[2,1,1024]', class_name, conf=conf) @@ -292,6 +296,7 @@ def test_resources(self): def tearDown(self): os.unlink(self.tempFile.name) + shutil.rmtree(self.tempdir.name) self.sc.stop() diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index 6095a384679af..68cfe814762e0 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -16,6 +16,7 @@ # import os import random +import shutil import stat import sys import tempfile @@ -277,9 +278,13 @@ def setUp(self): self.tempFile = tempfile.NamedTemporaryFile(delete=False) self.tempFile.write(b'echo {\\"name\\": \\"gpu\\", \\"addresses\\": [\\"0\\"]}') self.tempFile.close() + # create temporary directory for Worker resources coordination + self.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(self.tempdir.name) os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP | stat.S_IROTH | stat.S_IXOTH) conf = SparkConf().set("spark.test.home", SPARK_HOME) + conf = conf.set("spark.resources.dir", self.tempdir.name) conf = conf.set("spark.worker.resource.gpu.discoveryScript", self.tempFile.name) conf = conf.set("spark.worker.resource.gpu.amount", 1) conf = conf.set("spark.task.resource.gpu.amount", "1") @@ -297,6 +302,7 @@ def test_resources(self): def tearDown(self): os.unlink(self.tempFile.name) + shutil.rmtree(self.tempdir.name) self.sc.stop() if __name__ == "__main__": diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 105841ac834b3..5655ef50d214f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -27,6 +27,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.RpcAddress import org.apache.spark.scheduler.{ExecutorKilled, ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} @@ -55,6 +56,8 @@ private[spark] class KubernetesClusterSchedulerBackend( private val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) + private val defaultProfile = scheduler.sc.resourceProfileManager.defaultResourceProfile + // Allow removeExecutor to be accessible by ExecutorPodsLifecycleEventHandler private[k8s] def doRemoveExecutor(executorId: String, reason: ExecutorLossReason): Unit = { if (isExecutorActive(executorId)) { @@ -116,8 +119,9 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { - podAllocator.setTotalExpectedExecutors(requestedTotal) + override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { + podAllocator.setTotalExpectedExecutors(resourceProfileToTotalExecs(defaultProfile)) Future.successful(true) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 7e1e39c85a183..8c683e85dd5e2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ +import org.apache.spark.resource.ResourceProfileManager import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{ExecutorKilled, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor @@ -86,10 +87,13 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _ private var schedulerBackendUnderTest: KubernetesClusterSchedulerBackend = _ + private val resourceProfileManager = new ResourceProfileManager(sparkConf) + before { MockitoAnnotations.initMocks(this) when(taskScheduler.sc).thenReturn(sc) when(sc.conf).thenReturn(sparkConf) + when(sc.resourceProfileManager).thenReturn(resourceProfileManager) when(sc.env).thenReturn(env) when(env.rpcEnv).thenReturn(rpcEnv) driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index e916125ffdb67..0b447025c8a7a 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -38,6 +38,7 @@ import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalBlockStoreClient +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.{RpcEndpointAddress, RpcEndpointRef} import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -181,6 +182,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( private var schedulerDriver: SchedulerDriver = _ + private val defaultProfile = sc.resourceProfileManager.defaultResourceProfile + + def newMesosTaskId(): String = { val id = nextMesosTaskId nextMesosTaskId += 1 @@ -595,13 +599,16 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } private def satisfiesLocality(offerHostname: String): Boolean = { + val hostToLocalTaskCount = + rpHostToLocalTaskCount.getOrElse(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, Map.empty) if (!Utils.isDynamicAllocationEnabled(conf) || hostToLocalTaskCount.isEmpty) { return true } // Check the locality information val currentHosts = slaves.values.filter(_.taskIDs.nonEmpty).map(_.hostname).toSet - val allDesiredHosts = hostToLocalTaskCount.keys.toSet + val allDesiredHosts = hostToLocalTaskCount.map { case (k, v) => k }.toSet + // Try to match locality for hosts which do not have executors yet, to potentially // increase coverage. val remainingHosts = allDesiredHosts -- currentHosts @@ -759,11 +766,14 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( super.applicationId } - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future.successful { + override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int] + ): Future[Boolean] = Future.successful { // We don't truly know if we can fulfill the full amount of executors // since at coarse grain it depends on the amount of slaves available. - logInfo("Capping the total amount of executors to " + requestedTotal) - executorLimitOption = Some(requestedTotal) + val numExecs = resourceProfileToTotalExecs.getOrElse(defaultProfile, 0) + logInfo("Capping the total amount of executors to " + numExecs) + executorLimitOption = Some(numExecs) // Update the locality wait start time to continue trying for locality. localityWaitStartTimeNs = System.nanoTime() true diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 1876861700fc0..5ab277ed87a72 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -71,8 +71,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite offerResources(offers) verifyTaskLaunched(driver, "o1") + val totalExecs = Map(ResourceProfile.getOrCreateDefaultProfile(sparkConf) -> 0) // kills executors - assert(backend.doRequestTotalExecutors(0).futureValue) + val defaultResourceProfile = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + assert(backend.doRequestTotalExecutors(Map(defaultResourceProfile -> 0)).futureValue) assert(backend.doKillExecutors(Seq("0")).futureValue) val taskID0 = createTaskId("0") verify(driver, times(1)).killTask(taskID0) @@ -82,7 +84,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite verifyDeclinedOffer(driver, createOfferId("o2")) // Launches a new task when requested executors is positive - backend.doRequestTotalExecutors(2) + backend.doRequestTotalExecutors(Map(defaultResourceProfile -> 2)) offerResources(offers, 2) verifyTaskLaunched(driver, "o2") } @@ -635,7 +637,12 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite assert(backend.getExecutorIds().isEmpty) - backend.requestTotalExecutors(2, 2, Map("hosts10" -> 1, "hosts11" -> 1)) + val defaultProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + val defaultProf = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + backend.requestTotalExecutors( + Map(defaultProfileId -> 2), + Map(defaultProfileId -> 2), + Map(defaultProfileId -> Map("hosts10" -> 1, "hosts11" -> 1))) // Offer non-local resources, which should be rejected offerResourcesAndVerify(1, false) @@ -651,7 +658,11 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite offerResourcesAndVerify(1, true) // Update total executors - backend.requestTotalExecutors(3, 3, Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1)) + backend.requestTotalExecutors( + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 3), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 2), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> + Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1))) // Offer non-local resources, which should be rejected offerResourcesAndVerify(3, false) @@ -660,8 +671,11 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite Thread.sleep(2000) // Update total executors - backend.requestTotalExecutors(4, 4, Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1, - "hosts13" -> 1)) + backend.requestTotalExecutors( + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 4), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 4), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> + Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1, "hosts13" -> 1))) // Offer non-local resources, which should be rejected offerResourcesAndVerify(3, false) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 471ee58d05cb8..f8bbc39c8bcc5 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -27,13 +27,13 @@ import scala.util.{Failure, Success} import scala.util.control.NonFatal import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} -import org.eclipse.jetty.servlet.{FilterHolder, FilterMapping} import org.apache.spark.SparkContext import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.internal.config.UI._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -92,7 +92,7 @@ private[spark] abstract class YarnSchedulerBackend( try { // SPARK-12009: To prevent Yarn allocator from requesting backup for the executors which // was Stopped by SchedulerBackend. - requestTotalExecutors(0, 0, Map.empty) + requestTotalExecutors(Map.empty, Map.empty, Map.empty) super.stop() } finally { stopped.set(true) @@ -123,21 +123,28 @@ private[spark] abstract class YarnSchedulerBackend( } } - private[cluster] def prepareRequestExecutors(requestedTotal: Int): RequestExecutors = { + private[cluster] def prepareRequestExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): RequestExecutors = { val nodeBlacklist: Set[String] = scheduler.nodeBlacklist() // For locality preferences, ignore preferences for nodes that are blacklisted - val filteredHostToLocalTaskCount = - hostToLocalTaskCount.filter { case (k, v) => !nodeBlacklist.contains(k) } - RequestExecutors(requestedTotal, localityAwareTasks, filteredHostToLocalTaskCount, - nodeBlacklist) + val filteredRPHostToLocalTaskCount = rpHostToLocalTaskCount.map { case (rpid, v) => + (rpid, v.filter { case (host, count) => !nodeBlacklist.contains(host) }) + } + // TODO - default everything to default profile until YARN pieces + val defaultProf = ResourceProfile.getOrCreateDefaultProfile(conf) + val hostToLocalTaskCount = filteredRPHostToLocalTaskCount.getOrElse(defaultProf.id, Map.empty) + val localityAwareTasks = numLocalityAwareTasksPerResourceProfileId.getOrElse(defaultProf.id, 0) + val numExecutors = resourceProfileToTotalExecs.getOrElse(defaultProf, 0) + RequestExecutors(numExecutors, localityAwareTasks, hostToLocalTaskCount, nodeBlacklist) } /** * Request executors from the ApplicationMaster by specifying the total number desired. * This includes executors already pending or running. */ - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { - yarnSchedulerEndpointRef.ask[Boolean](prepareRequestExecutors(requestedTotal)) + override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { + yarnSchedulerEndpointRef.ask[Boolean](prepareRequestExecutors(resourceProfileToTotalExecs)) } /** diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala index a87820b1528ad..c0c6fff5130bb 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala @@ -24,6 +24,7 @@ import org.mockito.Mockito.when import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.serializer.JavaSerializer import org.apache.spark.ui.TestFilter @@ -51,7 +52,8 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc private class TestYarnSchedulerBackend(scheduler: TaskSchedulerImpl, sc: SparkContext) extends YarnSchedulerBackend(scheduler, sc) { def setHostToLocalTaskCount(hostToLocalTaskCount: Map[String, Int]): Unit = { - this.hostToLocalTaskCount = hostToLocalTaskCount + this.rpHostToLocalTaskCount = Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> + hostToLocalTaskCount) } } @@ -72,7 +74,8 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc } { yarnSchedulerBackendExtended.setHostToLocalTaskCount(hostToLocalCount) sched.setNodeBlacklist(blacklist) - val req = yarnSchedulerBackendExtended.prepareRequestExecutors(numRequested) + val numReq = Map(ResourceProfile.getOrCreateDefaultProfile(sc.getConf) -> numRequested) + val req = yarnSchedulerBackendExtended.prepareRequestExecutors(numReq) assert(req.requestedTotal === numRequested) assert(req.nodeBlacklist === blacklist) assert(req.hostToLocalTaskCount.keySet.intersect(blacklist).isEmpty) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala index a88abc8209a88..c09ff51ecaff2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala @@ -27,6 +27,7 @@ import org.apache.spark.{SparkConf, TaskState} import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config.Status._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetricInfo @@ -89,7 +90,8 @@ object MetricsAggregationBenchmark extends BenchmarkBase { val taskEventsTime = (0 until numStages).map { _ => val stageInfo = new StageInfo(idgen.incrementAndGet(), 0, getClass().getName(), - numTasks, Nil, Nil, getClass().getName()) + numTasks, Nil, Nil, getClass().getName(), + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val jobId = idgen.incrementAndGet() val jobStart = SparkListenerJobStart( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 55b551d0af078..fdfd392a224cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Status._ import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -86,7 +87,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils name = "", rddInfos = Nil, parentIds = Nil, - details = "") + details = "", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } private def createTaskInfo( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala index e85a3b9009c32..58bd56c591d04 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala @@ -23,6 +23,7 @@ import scala.util.Random import org.apache.spark.{ExecutorAllocationClient, SparkConf} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Streaming._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.streaming.util.RecurringTimer import org.apache.spark.util.{Clock, Utils} @@ -111,7 +112,11 @@ private[streaming] class ExecutorAllocationManager( logDebug(s"Executors (${allExecIds.size}) = ${allExecIds}") val targetTotalExecutors = math.max(math.min(maxNumExecutors, allExecIds.size + numNewExecutors), minNumExecutors) - client.requestTotalExecutors(targetTotalExecutors, 0, Map.empty) + // Just map the targetTotalExecutors to the default ResourceProfile + client.requestTotalExecutors( + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> targetTotalExecutors), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0), + Map.empty) logInfo(s"Requested total $targetTotalExecutors executors") } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala index 9121da4b9b673..65efa10bfcf92 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala @@ -27,6 +27,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{ExecutorAllocationClient, SparkConf} import org.apache.spark.internal.config.{DYN_ALLOCATION_ENABLED, DYN_ALLOCATION_TESTING} import org.apache.spark.internal.config.Streaming._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.streaming.{DummyInputDStream, Seconds, StreamingContext, TestSuiteBase} import org.apache.spark.util.{ManualClock, Utils} @@ -71,10 +72,15 @@ class ExecutorAllocationManagerSuite extends TestSuiteBase if (expectedRequestedTotalExecs.nonEmpty) { require(expectedRequestedTotalExecs.get > 0) verify(allocationClient, times(1)).requestTotalExecutors( - meq(expectedRequestedTotalExecs.get), meq(0), meq(Map.empty)) + meq(Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> + expectedRequestedTotalExecs.get)), + meq(Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0)), + meq(Map.empty)) } else { - verify(allocationClient, never).requestTotalExecutors(0, 0, Map.empty) - } + verify(allocationClient, never).requestTotalExecutors( + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0), + Map.empty)} } /** Verify that a particular executor was killed */ @@ -139,8 +145,11 @@ class ExecutorAllocationManagerSuite extends TestSuiteBase reset(allocationClient) when(allocationClient.getExecutorIds()).thenReturn((1 to numExecs).map(_.toString)) requestExecutors(allocationManager, numNewExecs) - verify(allocationClient, times(1)).requestTotalExecutors( - meq(expectedRequestedTotalExecs), meq(0), meq(Map.empty)) + val defaultProfId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + verify(allocationClient, times(1)). + requestTotalExecutors( + meq(Map(defaultProfId -> expectedRequestedTotalExecs)), + meq(Map(defaultProfId -> 0)), meq(Map.empty)) } withAllocationManager(numReceivers = 1) { case (_, allocationManager) => From 926e3a1efe9e142804fcbf52146b22700640ae1b Mon Sep 17 00:00:00 2001 From: iRakson Date: Thu, 13 Feb 2020 12:23:40 +0800 Subject: [PATCH 1361/1387] [SPARK-30790] The dataType of map() should be map ### What changes were proposed in this pull request? `spark.sql("select map()")` returns {}. After these changes it will return map ### Why are the changes needed? After changes introduced due to #27521, it is important to maintain consistency while using map(). ### Does this PR introduce any user-facing change? Yes. Now map() will give map instead of {}. ### How was this patch tested? UT added. Migration guide updated as well Closes #27542 from iRakson/SPARK-30790. Authored-by: iRakson Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 +- .../expressions/complexTypeCreator.scala | 14 ++++++++--- .../catalyst/util/ArrayBasedMapBuilder.scala | 5 ++-- .../apache/spark/sql/internal/SQLConf.scala | 10 ++++---- .../spark/sql/DataFrameFunctionsSuite.scala | 25 +++++++++++++------ 5 files changed, 36 insertions(+), 20 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index f98fab5b4c56b..46b741687363f 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -216,7 +216,7 @@ license: | - Since Spark 3.0, the `size` function returns `NULL` for the `NULL` input. In Spark version 2.4 and earlier, this function gives `-1` for the same input. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.sizeOfNull` to `true`. - - Since Spark 3.0, when the `array` function is called without any parameters, it returns an empty array of `NullType`. In Spark version 2.4 and earlier, it returns an empty array of string type. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.arrayDefaultToStringType.enabled` to `true`. + - Since Spark 3.0, when the `array`/`map` function is called without any parameters, it returns an empty collection with `NullType` as element type. In Spark version 2.4 and earlier, it returns an empty collection with `StringType` as element type. To restore the behavior before Spark 3.0, you can set `spark.sql.legacy.createEmptyCollectionUsingStringType` to `true`. - Since Spark 3.0, the interval literal syntax does not allow multiple from-to units anymore. For example, `SELECT INTERVAL '1-1' YEAR TO MONTH '2-2' YEAR TO MONTH'` throws parser exception. 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 7335e305bfe55..4bd85d304ded2 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 @@ -46,7 +46,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { } private val defaultElementType: DataType = { - if (SQLConf.get.getConf(SQLConf.LEGACY_ARRAY_DEFAULT_TO_STRING)) { + if (SQLConf.get.getConf(SQLConf.LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE)) { StringType } else { NullType @@ -145,6 +145,14 @@ case class CreateMap(children: Seq[Expression]) extends Expression { lazy val keys = children.indices.filter(_ % 2 == 0).map(children) lazy val values = children.indices.filter(_ % 2 != 0).map(children) + private val defaultElementType: DataType = { + if (SQLConf.get.getConf(SQLConf.LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE)) { + StringType + } else { + NullType + } + } + override def foldable: Boolean = children.forall(_.foldable) override def checkInputDataTypes(): TypeCheckResult = { @@ -167,9 +175,9 @@ case class CreateMap(children: Seq[Expression]) extends Expression { override lazy val dataType: MapType = { MapType( keyType = TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(keys.map(_.dataType)) - .getOrElse(StringType), + .getOrElse(defaultElementType), valueType = TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(values.map(_.dataType)) - .getOrElse(StringType), + .getOrElse(defaultElementType), valueContainsNull = values.exists(_.nullable)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala index 98934368205ec..37d65309e2b89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala @@ -29,12 +29,11 @@ import org.apache.spark.unsafe.array.ByteArrayMethods */ class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Serializable { assert(!keyType.existsRecursively(_.isInstanceOf[MapType]), "key of map cannot be/contain map") - assert(keyType != NullType, "map key cannot be null type.") private lazy val keyToIndex = keyType match { // Binary type data is `byte[]`, which can't use `==` to check equality. - case _: AtomicType | _: CalendarIntervalType if !keyType.isInstanceOf[BinaryType] => - new java.util.HashMap[Any, Int]() + case _: AtomicType | _: CalendarIntervalType | _: NullType + if !keyType.isInstanceOf[BinaryType] => new java.util.HashMap[Any, Int]() case _ => // for complex types, use interpreted ordering to be able to compare unsafe data with safe // data, e.g. UnsafeRow vs GenericInternalRow. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d86f8693e0655..95b5b3afc3933 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2007,12 +2007,12 @@ object SQLConf { .booleanConf .createWithDefault(false) - val LEGACY_ARRAY_DEFAULT_TO_STRING = - buildConf("spark.sql.legacy.arrayDefaultToStringType.enabled") + val LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE = + buildConf("spark.sql.legacy.createEmptyCollectionUsingStringType") .internal() - .doc("When set to true, it returns an empty array of string type when the `array` " + - "function is called without any parameters. Otherwise, it returns an empty " + - "array of `NullType`") + .doc("When set to true, Spark returns an empty collection with `StringType` as element " + + "type if the `array`/`map` function is called without any parameters. Otherwise, Spark " + + "returns an empty collection with `NullType` as element type.") .booleanConf .createWithDefault(false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 6012678341ccc..f7531ea446015 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -3499,13 +3499,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { ).foreach(assertValuesDoNotChangeAfterCoalesceOrUnion(_)) } - test("SPARK-21281 use string types by default if map have no argument") { - val ds = spark.range(1) - var expectedSchema = new StructType() - .add("x", MapType(StringType, StringType, valueContainsNull = false), nullable = false) - assert(ds.select(map().as("x")).schema == expectedSchema) - } - test("SPARK-21281 fails if functions have no argument") { val df = Seq(1).toDF("a") @@ -3563,7 +3556,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { test("SPARK-29462: Empty array of NullType for array function with no arguments") { Seq((true, StringType), (false, NullType)).foreach { case (arrayDefaultToString, expectedType) => - withSQLConf(SQLConf.LEGACY_ARRAY_DEFAULT_TO_STRING.key -> arrayDefaultToString.toString) { + withSQLConf(SQLConf.LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE.key -> + arrayDefaultToString.toString) { val schema = spark.range(1).select(array()).schema assert(schema.nonEmpty && schema.head.dataType.isInstanceOf[ArrayType]) val actualType = schema.head.dataType.asInstanceOf[ArrayType].elementType @@ -3571,6 +3565,21 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { } } } + + test("SPARK-30790: Empty map with NullType as key/value type for map function with no argument") { + Seq((true, StringType), (false, NullType)).foreach { + case (mapDefaultToString, expectedType) => + withSQLConf(SQLConf.LEGACY_CREATE_EMPTY_COLLECTION_USING_STRING_TYPE.key -> + mapDefaultToString.toString) { + val schema = spark.range(1).select(map()).schema + assert(schema.nonEmpty && schema.head.dataType.isInstanceOf[MapType]) + val actualKeyType = schema.head.dataType.asInstanceOf[MapType].keyType + val actualValueType = schema.head.dataType.asInstanceOf[MapType].valueType + assert(actualKeyType === expectedType) + assert(actualValueType === expectedType) + } + } + } } object DataFrameFunctionsSuite { From 453d5261b22ebcdd5886e65ab9d0d9857051e76a Mon Sep 17 00:00:00 2001 From: maryannxue Date: Thu, 13 Feb 2020 19:32:38 +0800 Subject: [PATCH 1362/1387] [SPARK-30528][SQL] Turn off DPP subquery duplication by default ### What changes were proposed in this pull request? This PR adds a config for Dynamic Partition Pruning subquery duplication and turns it off by default due to its potential performance regression. When planning a DPP filter, it seeks to reuse the broadcast exchange relation if the corresponding join is a BHJ with the filter relation being on the build side, otherwise it will either opt out or plan the filter as an un-reusable subquery duplication based on the cost estimate. However, the cost estimate is not accurate and only takes into account the table scan overhead, thus adding an un-reusable subquery duplication DPP filter can sometimes cause perf regression. This PR turns off the subquery duplication DPP filter by: 1. adding a config `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly` and setting it `true` by default. 2. removing the existing meaningless config `spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcast` since we always want to reuse broadcast results if possible. ### Why are the changes needed? This is to fix a potential performance regression caused by DPP. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Updated DynamicPartitionPruningSuite to test the new configuration. Closes #27551 from maryannxue/spark-30528. Authored-by: maryannxue Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 12 +- .../sql/dynamicpruning/PartitionPruning.scala | 4 +- .../PlanDynamicPruningFilters.scala | 5 +- .../sql/DynamicPartitionPruningSuite.scala | 183 +++++++----------- .../org/apache/spark/sql/ExplainSuite.scala | 3 +- 5 files changed, 82 insertions(+), 125 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 95b5b3afc3933..2214e03f34f0b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -259,11 +259,11 @@ object SQLConf { .doubleConf .createWithDefault(0.5) - val DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST = - buildConf("spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcast") + val DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY = + buildConf("spark.sql.optimizer.dynamicPartitionPruning.reuseBroadcastOnly") .internal() - .doc("When true, dynamic partition pruning will seek to reuse the broadcast results from " + - "a broadcast hash join operation.") + .doc("When true, dynamic partition pruning will only apply when the broadcast exchange of " + + "a broadcast hash join operation can be reused as the dynamic pruning filter.") .booleanConf .createWithDefault(true) @@ -2303,8 +2303,8 @@ class SQLConf extends Serializable with Logging { def dynamicPartitionPruningFallbackFilterRatio: Double = getConf(DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO) - def dynamicPartitionPruningReuseBroadcast: Boolean = - getConf(DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST) + def dynamicPartitionPruningReuseBroadcastOnly: Boolean = + getConf(DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY) def stateStoreProviderClass: String = getConf(STATE_STORE_PROVIDER_CLASS) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala index 48ba8618f272e..28f8f49d2ce44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PartitionPruning.scala @@ -86,7 +86,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { filteringPlan: LogicalPlan, joinKeys: Seq[Expression], hasBenefit: Boolean): LogicalPlan = { - val reuseEnabled = SQLConf.get.dynamicPartitionPruningReuseBroadcast + val reuseEnabled = SQLConf.get.exchangeReuseEnabled val index = joinKeys.indexOf(filteringKey) if (hasBenefit || reuseEnabled) { // insert a DynamicPruning wrapper to identify the subquery during query planning @@ -96,7 +96,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { filteringPlan, joinKeys, index, - !hasBenefit), + !hasBenefit || SQLConf.get.dynamicPartitionPruningReuseBroadcastOnly), pruningPlan) } else { // abort dynamic partition pruning diff --git a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PlanDynamicPruningFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PlanDynamicPruningFilters.scala index 1398dc049dd99..be00f728aa3ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PlanDynamicPruningFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/dynamicpruning/PlanDynamicPruningFilters.scala @@ -36,9 +36,6 @@ import org.apache.spark.sql.internal.SQLConf case class PlanDynamicPruningFilters(sparkSession: SparkSession) extends Rule[SparkPlan] with PredicateHelper { - private def reuseBroadcast: Boolean = - SQLConf.get.dynamicPartitionPruningReuseBroadcast && SQLConf.get.exchangeReuseEnabled - /** * Identify the shape in which keys of a given plan are broadcasted. */ @@ -59,7 +56,7 @@ case class PlanDynamicPruningFilters(sparkSession: SparkSession) sparkSession, sparkSession.sessionState.planner, buildPlan) // Using `sparkPlan` is a little hacky as it is based on the assumption that this rule is // the first to be applied (apart from `InsertAdaptiveSparkPlan`). - val canReuseExchange = reuseBroadcast && buildKeys.nonEmpty && + val canReuseExchange = SQLConf.get.exchangeReuseEnabled && buildKeys.nonEmpty && plan.find { case BroadcastHashJoinExec(_, _, _, BuildLeft, _, left, _) => left.sameResult(sparkPlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index e1f9bcc4e008d..f7b51d6f4c8ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -239,7 +239,8 @@ class DynamicPartitionPruningSuite */ test("simple inner join triggers DPP with mock-up tables") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { withTable("df1", "df2") { spark.range(1000) .select(col("id"), col("id").as("k")) @@ -271,7 +272,8 @@ class DynamicPartitionPruningSuite */ test("self-join on a partitioned table should not trigger DPP") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { withTable("fact") { sql( s""" @@ -302,7 +304,8 @@ class DynamicPartitionPruningSuite */ test("static scan metrics") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { withTable("fact", "dim") { spark.range(10) .map { x => Tuple3(x, x + 1, 0) } @@ -370,7 +373,8 @@ class DynamicPartitionPruningSuite test("DPP should not be rewritten as an existential join") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "1.5", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { val df = sql( s""" |SELECT * FROM product p WHERE p.store_id NOT IN @@ -395,7 +399,7 @@ class DynamicPartitionPruningSuite */ test("DPP triggers only for certain types of query") { withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false") { Given("dynamic partition pruning disabled") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "false") { val df = sql( @@ -433,7 +437,8 @@ class DynamicPartitionPruningSuite } Given("left-semi join with partition column on the left side") - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { val df = sql( """ |SELECT * FROM fact_sk f @@ -457,7 +462,8 @@ class DynamicPartitionPruningSuite } Given("right outer join with partition column on the left side") - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { val df = sql( """ |SELECT * FROM fact_sk f RIGHT OUTER JOIN dim_store s @@ -474,7 +480,8 @@ class DynamicPartitionPruningSuite */ test("filtering ratio policy fallback") { withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { Given("no stats and selective predicate") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "true") { @@ -543,7 +550,8 @@ class DynamicPartitionPruningSuite */ test("filtering ratio policy with stats when the broadcast pruning is disabled") { withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { Given("disabling the use of stats in the DPP heuristic") withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false") { @@ -613,10 +621,7 @@ class DynamicPartitionPruningSuite test("partition pruning in broadcast hash joins with non-deterministic probe part") { Given("alias with simple join condition, and non-deterministic query") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT f.date_id, f.pid, f.sid FROM @@ -630,10 +635,7 @@ class DynamicPartitionPruningSuite } Given("alias over multiple sub-queries with simple join condition") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT f.date_id, f.pid, f.sid FROM @@ -651,10 +653,7 @@ class DynamicPartitionPruningSuite test("partition pruning in broadcast hash joins with aliases") { Given("alias with simple join condition, using attribute names only") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT f.date_id, f.pid, f.sid FROM @@ -674,10 +673,7 @@ class DynamicPartitionPruningSuite } Given("alias with expr as join condition") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT f.date_id, f.pid, f.sid FROM @@ -697,10 +693,7 @@ class DynamicPartitionPruningSuite } Given("alias over multiple sub-queries with simple join condition") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT f.date_id, f.pid, f.sid FROM @@ -722,10 +715,7 @@ class DynamicPartitionPruningSuite } Given("alias over multiple sub-queries with simple join condition") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT f.date_id, f.pid_d as pid, f.sid_d as sid FROM @@ -754,10 +744,8 @@ class DynamicPartitionPruningSuite test("partition pruning in broadcast hash joins") { Given("disable broadcast pruning and disable subquery duplication") withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { val df = sql( """ |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f @@ -777,9 +765,10 @@ class DynamicPartitionPruningSuite Given("disable reuse broadcast results and enable subquery duplication") withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0.5") { + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0.5", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { val df = sql( """ |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f @@ -798,52 +787,47 @@ class DynamicPartitionPruningSuite } Given("enable reuse broadcast results and disable query duplication") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { - val df = sql( - """ - |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f - |JOIN dim_stats s - |ON f.store_id = s.store_id WHERE s.country = 'DE' - """.stripMargin) + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ + |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f + |JOIN dim_stats s + |ON f.store_id = s.store_id WHERE s.country = 'DE' + """.stripMargin) - checkPartitionPruningPredicate(df, false, true) + checkPartitionPruningPredicate(df, false, true) - checkAnswer(df, - Row(1030, 2, 10, 3) :: - Row(1040, 2, 50, 3) :: - Row(1050, 2, 50, 3) :: - Row(1060, 2, 50, 3) :: Nil - ) + checkAnswer(df, + Row(1030, 2, 10, 3) :: + Row(1040, 2, 50, 3) :: + Row(1050, 2, 50, 3) :: + Row(1060, 2, 50, 3) :: Nil + ) } Given("disable broadcast hash join and disable query duplication") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { - val df = sql( - """ - |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f - |JOIN dim_stats s - |ON f.store_id = s.store_id WHERE s.country = 'DE' - """.stripMargin) + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = sql( + """ + |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f + |JOIN dim_stats s + |ON f.store_id = s.store_id WHERE s.country = 'DE' + """.stripMargin) - checkPartitionPruningPredicate(df, false, false) + checkPartitionPruningPredicate(df, false, false) - checkAnswer(df, - Row(1030, 2, 10, 3) :: - Row(1040, 2, 50, 3) :: - Row(1050, 2, 50, 3) :: - Row(1060, 2, 50, 3) :: Nil - ) + checkAnswer(df, + Row(1030, 2, 10, 3) :: + Row(1040, 2, 50, 3) :: + Row(1050, 2, 50, 3) :: + Row(1060, 2, 50, 3) :: Nil + ) } Given("disable broadcast hash join and enable query duplication") - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "true") { val df = sql( @@ -865,9 +849,7 @@ class DynamicPartitionPruningSuite } test("broadcast a single key in a HashedRelation") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( $"id", @@ -925,9 +907,7 @@ class DynamicPartitionPruningSuite } test("broadcast multiple keys in a LongHashedRelation") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( $"id", @@ -962,9 +942,7 @@ class DynamicPartitionPruningSuite } test("broadcast multiple keys in an UnsafeHashedRelation") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( $"id", @@ -999,9 +977,7 @@ class DynamicPartitionPruningSuite } test("different broadcast subqueries with identical children") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(100).select( $"id", @@ -1073,7 +1049,7 @@ class DynamicPartitionPruningSuite } test("avoid reordering broadcast join keys to match input hash partitioning") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { spark.range(100).select( @@ -1123,9 +1099,7 @@ class DynamicPartitionPruningSuite * duplicated partitioning keys, also used to uniquely identify the dynamic pruning filters. */ test("dynamic partition pruning ambiguity issue across nested joins") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("store", "date", "item") { spark.range(500) .select((($"id" + 30) % 50).as("ss_item_sk"), @@ -1163,9 +1137,7 @@ class DynamicPartitionPruningSuite } test("cleanup any DPP filter that isn't pushed down due to expression id clashes") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withTable("fact", "dim") { spark.range(1000).select($"id".as("A"), $"id".as("AA")) .write.partitionBy("A").format(tableFormat).mode("overwrite").saveAsTable("fact") @@ -1186,10 +1158,7 @@ class DynamicPartitionPruningSuite } test("cleanup any DPP filter that isn't pushed down due to non-determinism") { - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT f.date_id, f.pid, f.sid FROM @@ -1204,9 +1173,7 @@ class DynamicPartitionPruningSuite } test("join key with multiple references on the filtering plan") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0", + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // when enable AQE, the reusedExchange is inserted when executed. withTable("fact", "dim") { @@ -1240,9 +1207,7 @@ class DynamicPartitionPruningSuite } test("Make sure dynamic pruning works on uncorrelated queries") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT d.store_id, @@ -1266,10 +1231,7 @@ class DynamicPartitionPruningSuite test("Plan broadcast pruning only when the broadcast can be reused") { Given("dynamic pruning filter on the build side") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT f.date_id, f.store_id, f.product_id, f.units_sold FROM fact_np f @@ -1288,10 +1250,7 @@ class DynamicPartitionPruningSuite } Given("dynamic pruning filter on the probe side") - withSQLConf( - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_USE_STATS.key -> "false", - SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "0") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ |SELECT /*+ BROADCAST(f)*/ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index d9f4d6d5132ae..b591705274110 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -239,7 +239,8 @@ class ExplainSuite extends QueryTest with SharedSparkSession { test("explain formatted - check presence of subquery in case of DPP") { withTable("df1", "df2") { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST.key -> "false") { + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { withTable("df1", "df2") { spark.range(1000).select(col("id"), col("id").as("k")) .write From a6b4b914f2d2b873b0e9b9d446fda69dc74c3cf8 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Thu, 13 Feb 2020 20:13:36 +0800 Subject: [PATCH 1363/1387] [SPARK-30613][SQL] Support Hive style REPLACE COLUMNS syntax ### What changes were proposed in this pull request? This PR proposes to support Hive-style `ALTER TABLE ... REPLACE COLUMNS ...` as described in https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Add/ReplaceColumns The user now can do the following: ```SQL CREATE TABLE t (col1 int, col2 int) USING Foo; ALTER TABLE t REPLACE COLUMNS (col2 string COMMENT 'comment2', col3 int COMMENT 'comment3'); ``` , which drops the existing columns `col1` and `col2`, and add new columns `col2` and `col3`. ### Why are the changes needed? This is a new DDL statement. Spark currently supports the Hive-style `ALTER TABLE ... CHANGE COLUMN ...`, so this new addition can be useful. ### Does this PR introduce any user-facing change? Yes, adding a new DDL statement. ### How was this patch tested? More tests to be added. Closes #27482 from imback82/replace_cols. Authored-by: Terry Kim Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 ++ .../sql/catalyst/analysis/CheckAnalysis.scala | 12 ++++- .../catalyst/analysis/ResolveCatalogs.scala | 21 ++++++++ .../sql/catalyst/parser/AstBuilder.scala | 21 ++++++++ .../catalyst/plans/logical/statements.scala | 4 ++ .../sql/catalyst/parser/DDLParserSuite.scala | 48 ++++++++++++++++++- .../analysis/ResolveSessionCatalog.scala | 23 +++++++++ .../spark/sql/connector/AlterTableTests.scala | 15 ++++++ 8 files changed, 145 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 563ef69b3b8ae..2bc71476aba02 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -164,6 +164,9 @@ statement | ALTER TABLE table=multipartIdentifier partitionSpec? CHANGE COLUMN? colName=multipartIdentifier colType colPosition? #hiveChangeColumn + | ALTER TABLE table=multipartIdentifier partitionSpec? + REPLACE COLUMNS + '(' columns=qualifiedColTypeWithPositionList ')' #hiveReplaceColumns | ALTER TABLE multipartIdentifier (partitionSpec)? SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe | ALTER TABLE multipartIdentifier (partitionSpec)? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index e769e038c960f..67c509ed98245 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -473,9 +473,15 @@ trait CheckAnalysis extends PredicateHelper { } } + val colsToDelete = mutable.Set.empty[Seq[String]] + alter.changes.foreach { case add: AddColumn => - checkColumnNotExists("add", add.fieldNames(), table.schema) + // If a column to add is a part of columns to delete, we don't need to check + // if column already exists - applies to REPLACE COLUMNS scenario. + if (!colsToDelete.contains(add.fieldNames())) { + checkColumnNotExists("add", add.fieldNames(), table.schema) + } val parent = findParentStruct("add", add.fieldNames()) positionArgumentExists(add.position(), parent) TypeUtils.failWithIntervalType(add.dataType()) @@ -526,6 +532,10 @@ trait CheckAnalysis extends PredicateHelper { findField("update", update.fieldNames) case delete: DeleteColumn => findField("delete", delete.fieldNames) + // REPLACE COLUMNS has deletes followed by adds. Remember the deleted columns + // so that add operations do not fail when the columns to add exist and they + // are to be deleted. + colsToDelete += delete.fieldNames case _ => // no validation needed for set and remove property } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 88a3c0a73a10b..96558410d4004 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -44,6 +44,27 @@ class ResolveCatalogs(val catalogManager: CatalogManager) } createAlterTable(nameParts, catalog, tbl, changes) + case AlterTableReplaceColumnsStatement( + nameParts @ NonSessionCatalogAndTable(catalog, tbl), cols) => + val changes: Seq[TableChange] = loadTable(catalog, tbl.asIdentifier) match { + case Some(table) => + // REPLACE COLUMNS deletes all the existing columns and adds new columns specified. + val deleteChanges = table.schema.fieldNames.map { name => + TableChange.deleteColumn(Array(name)) + } + val addChanges = cols.map { col => + TableChange.addColumn( + col.name.toArray, + col.dataType, + col.nullable, + col.comment.orNull, + col.position.orNull) + } + deleteChanges ++ addChanges + case None => Seq() + } + createAlterTable(nameParts, catalog, tbl, changes) + case a @ AlterTableAlterColumnStatement( nameParts @ NonSessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => val colName = a.column.toArray diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 62e568587fcc6..b3541a7f7374d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3030,6 +3030,27 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging position = Option(ctx.colPosition).map(typedVisit[ColumnPosition])) } + override def visitHiveReplaceColumns( + ctx: HiveReplaceColumnsContext): LogicalPlan = withOrigin(ctx) { + if (ctx.partitionSpec != null) { + operationNotAllowed("ALTER TABLE table PARTITION partition_spec REPLACE COLUMNS", ctx) + } + AlterTableReplaceColumnsStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + ctx.columns.qualifiedColTypeWithPosition.asScala.map { colType => + if (colType.NULL != null) { + throw new AnalysisException( + "NOT NULL is not supported in Hive-style REPLACE COLUMNS") + } + if (colType.colPosition != null) { + throw new AnalysisException( + "Column position is not supported in Hive-style REPLACE COLUMNS") + } + typedVisit[QualifiedColType](colType) + } + ) + } + /** * Parse a [[AlterTableDropColumnsStatement]] command. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 1e6b67bf78b70..6731214d3842d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -156,6 +156,10 @@ case class AlterTableAddColumnsStatement( tableName: Seq[String], columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement +case class AlterTableReplaceColumnsStatement( + tableName: Seq[String], + columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement + /** * ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index bc7b51f25b20d..049f56c8c9ce1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -699,7 +699,7 @@ class DDLParserSuite extends AnalysisTest { } } - test("alter table: hive style") { + test("alter table: hive style change column") { val sql1 = "ALTER TABLE table_name CHANGE COLUMN a.b.c c INT" val sql2 = "ALTER TABLE table_name CHANGE COLUMN a.b.c c INT COMMENT 'new_comment'" val sql3 = "ALTER TABLE table_name CHANGE COLUMN a.b.c c INT AFTER other_col" @@ -742,6 +742,52 @@ class DDLParserSuite extends AnalysisTest { intercept("ALTER TABLE table_name PARTITION (a='1') CHANGE COLUMN a.b.c c INT") } + test("alter table: hive style replace columns") { + val sql1 = "ALTER TABLE table_name REPLACE COLUMNS (x string)" + val sql2 = "ALTER TABLE table_name REPLACE COLUMNS (x string COMMENT 'x1')" + val sql3 = "ALTER TABLE table_name REPLACE COLUMNS (x string COMMENT 'x1', y int)" + val sql4 = "ALTER TABLE table_name REPLACE COLUMNS (x string COMMENT 'x1', y int COMMENT 'y1')" + + comparePlans( + parsePlan(sql1), + AlterTableReplaceColumnsStatement( + Seq("table_name"), + Seq(QualifiedColType(Seq("x"), StringType, true, None, None)))) + + comparePlans( + parsePlan(sql2), + AlterTableReplaceColumnsStatement( + Seq("table_name"), + Seq(QualifiedColType(Seq("x"), StringType, true, Some("x1"), None)))) + + comparePlans( + parsePlan(sql3), + AlterTableReplaceColumnsStatement( + Seq("table_name"), + Seq( + QualifiedColType(Seq("x"), StringType, true, Some("x1"), None), + QualifiedColType(Seq("y"), IntegerType, true, None, None) + ))) + + comparePlans( + parsePlan(sql4), + AlterTableReplaceColumnsStatement( + Seq("table_name"), + Seq( + QualifiedColType(Seq("x"), StringType, true, Some("x1"), None), + QualifiedColType(Seq("y"), IntegerType, true, Some("y1"), None) + ))) + + intercept("ALTER TABLE table_name PARTITION (a='1') REPLACE COLUMNS (x string)", + "Operation not allowed: ALTER TABLE table PARTITION partition_spec REPLACE COLUMNS") + + intercept("ALTER TABLE table_name REPLACE COLUMNS (x string NOT NULL)", + "NOT NULL is not supported in Hive-style REPLACE COLUMNS") + + intercept("ALTER TABLE table_name REPLACE COLUMNS (x string FIRST)", + "Column position is not supported in Hive-style REPLACE COLUMNS") + } + test("alter table/view: rename table/view") { comparePlans( parsePlan("ALTER TABLE a.b.c RENAME TO x.y.z"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 77d549c28aae5..adeb2164eff63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -69,6 +69,29 @@ class ResolveSessionCatalog( createAlterTable(nameParts, catalog, tbl, changes) } + case AlterTableReplaceColumnsStatement( + nameParts @ SessionCatalogAndTable(catalog, tbl), cols) => + val changes: Seq[TableChange] = loadTable(catalog, tbl.asIdentifier) match { + case Some(_: V1Table) => + throw new AnalysisException("REPLACE COLUMNS is only supported with v2 tables.") + case Some(table) => + // REPLACE COLUMNS deletes all the existing columns and adds new columns specified. + val deleteChanges = table.schema.fieldNames.map { name => + TableChange.deleteColumn(Array(name)) + } + val addChanges = cols.map { col => + TableChange.addColumn( + col.name.toArray, + col.dataType, + col.nullable, + col.comment.orNull, + col.position.orNull) + } + deleteChanges ++ addChanges + case None => Seq() // Unresolved table will be handled in CheckAnalysis. + } + createAlterTable(nameParts, catalog, tbl, changes) + case a @ AlterTableAlterColumnStatement( nameParts @ SessionCatalogAndTable(catalog, tbl), _, _, _, _, _) => loadTable(catalog, tbl.asIdentifier).collect { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala index 420cb01d766a0..96fe301b512ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala @@ -1060,4 +1060,19 @@ trait AlterTableTests extends SharedSparkSession { assert(updated.properties === withDefaultOwnership(Map("provider" -> v2Format)).asJava) } } + + test("AlterTable: replace columns") { + val t = s"${catalogAndNamespace}table_name" + withTable(t) { + sql(s"CREATE TABLE $t (col1 int, col2 int COMMENT 'c2') USING $v2Format") + sql(s"ALTER TABLE $t REPLACE COLUMNS (col2 string, col3 int COMMENT 'c3')") + + val table = getTableMetadata(t) + + assert(table.name === fullTableName(t)) + assert(table.schema === StructType(Seq( + StructField("col2", StringType), + StructField("col3", IntegerType).withComment("c3")))) + } + } } From 04604b9899cc43a9726d671061ff305912fdb85f Mon Sep 17 00:00:00 2001 From: beliefer Date: Thu, 13 Feb 2020 22:06:24 +0800 Subject: [PATCH 1364/1387] [SPARK-30758][SQL][TESTS] Improve bracketed comments tests ### What changes were proposed in this pull request? Although Spark SQL support bracketed comments, but `SQLQueryTestSuite` can't treat bracketed comments well and lead to generated golden files can't display bracketed comments well. This PR will improve the treatment of bracketed comments and add three test case in `PlanParserSuite`. Spark SQL can't support nested bracketed comments and https://github.com/apache/spark/pull/27495 used to support it. ### Why are the changes needed? Golden files can't display well. ### Does this PR introduce any user-facing change? No ### How was this patch tested? New UT. Closes #27481 from beliefer/ansi-brancket-comments. Authored-by: beliefer Signed-off-by: Wenchen Fan --- .../sql-tests/inputs/postgreSQL/comments.sql | 6 +- .../results/postgreSQL/comments.sql.out | 137 ++++-------------- .../apache/spark/sql/SQLQueryTestSuite.scala | 51 ++++++- 3 files changed, 78 insertions(+), 116 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/comments.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/comments.sql index 6725ce45e72a5..1a454179ef79f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/comments.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/comments.sql @@ -11,17 +11,19 @@ SELECT /* embedded single line */ 'embedded' AS `second`; SELECT /* both embedded and trailing single line */ 'both' AS third; -- trailing single line SELECT 'before multi-line' AS fourth; +--QUERY-DELIMITER-START -- [SPARK-28880] ANSI SQL: Bracketed comments /* This is an example of SQL which should not execute: * select 'multi-line'; */ SELECT 'after multi-line' AS fifth; +--QUERY-DELIMITER-END -- [SPARK-28880] ANSI SQL: Bracketed comments -- -- Nested comments -- - +--QUERY-DELIMITER-START /* SELECT 'trailing' as x1; -- inside block comment */ @@ -44,5 +46,5 @@ Hoo boy. Still two deep... Now just one deep... */ 'deeply nested example' AS sixth; - +--QUERY-DELIMITER-END /* and this is the end of the file */ diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out index 4ea49013a62d1..637c5561bd940 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/comments.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 13 +-- Number of queries: 7 -- !query @@ -36,129 +36,32 @@ before multi-line -- !query /* This is an example of SQL which should not execute: - * select 'multi-line' --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException - -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) - -== SQL == -/* This is an example of SQL which should not execute: -^^^ - * select 'multi-line' - - --- !query -*/ + * select 'multi-line'; + */ SELECT 'after multi-line' AS fifth -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException - -extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) - -== SQL == -*/ -^^^ -SELECT 'after multi-line' AS fifth +after multi-line -- !query /* -SELECT 'trailing' as x1 --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException - -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) - -== SQL == -/* -^^^ -SELECT 'trailing' as x1 - - --- !query +SELECT 'trailing' as x1; -- inside block comment */ /* This block comment surrounds a query which itself has a block comment... -SELECT /* embedded single line */ 'embedded' AS x2 --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException - -mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) - -== SQL == -*/ -^^^ - -/* This block comment surrounds a query which itself has a block comment... -SELECT /* embedded single line */ 'embedded' AS x2 - - --- !query +SELECT /* embedded single line */ 'embedded' AS x2; */ SELECT -- continued after the following block comments... /* Deeply nested comment. This includes a single apostrophe to make sure we aren't decoding this part as a string. -SELECT 'deep nest' AS n1 --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException - -extraneous input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) - -== SQL == -*/ -^^^ - -SELECT -- continued after the following block comments... -/* Deeply nested comment. - This includes a single apostrophe to make sure we aren't decoding this part as a string. -SELECT 'deep nest' AS n1 - - --- !query +SELECT 'deep nest' AS n1; /* Second level of nesting... -SELECT 'deeper nest' as n2 --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException - -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) - -== SQL == -/* Second level of nesting... -^^^ -SELECT 'deeper nest' as n2 - - --- !query +SELECT 'deeper nest' as n2; /* Third level of nesting... -SELECT 'deepest nest' as n3 --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException - -mismatched input '/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) - -== SQL == -/* Third level of nesting... -^^^ -SELECT 'deepest nest' as n3 - - --- !query +SELECT 'deepest nest' as n3; */ Hoo boy. Still two deep... */ @@ -170,11 +73,27 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException -mismatched input '*/' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0) +mismatched input ''embedded'' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 6, pos 34) == SQL == +/* +SELECT 'trailing' as x1; -- inside block comment +*/ + +/* This block comment surrounds a query which itself has a block comment... +SELECT /* embedded single line */ 'embedded' AS x2; +----------------------------------^^^ +*/ + +SELECT -- continued after the following block comments... +/* Deeply nested comment. + This includes a single apostrophe to make sure we aren't decoding this part as a string. +SELECT 'deep nest' AS n1; +/* Second level of nesting... +SELECT 'deeper nest' as n2; +/* Third level of nesting... +SELECT 'deepest nest' as n3; */ -^^^ Hoo boy. Still two deep... */ Now just one deep... diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 6b9e5bbd3c961..da4727f6a98cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql import java.io.File import java.util.{Locale, TimeZone} +import java.util.regex.Pattern +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal import org.apache.spark.{SparkConf, SparkException} @@ -62,7 +64,12 @@ import org.apache.spark.tags.ExtendedSQLTest * }}} * * The format for input files is simple: - * 1. A list of SQL queries separated by semicolon. + * 1. A list of SQL queries separated by semicolons by default. If the semicolon cannot effectively + * separate the SQL queries in the test file(e.g. bracketed comments), please use + * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with + * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END represent the beginning and end of a query, + * respectively. Code that is not surrounded by lines that begin with --QUERY-DELIMITER-START + * and --QUERY-DELIMITER-END is still separated by semicolons. * 2. Lines starting with -- are treated as comments and ignored. * 3. Lines starting with --SET are used to specify the configs when running this testing file. You * can set multiple configs in one --SET, using comma to separate them. Or you can use multiple @@ -246,9 +253,15 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { /** Run a test case. */ protected def runTest(testCase: TestCase): Unit = { + def splitWithSemicolon(seq: Seq[String]) = { + seq.mkString("\n").split("(?<=[^\\\\]);") + } val input = fileToString(new File(testCase.inputFile)) - val (comments, code) = input.split("\n").partition(_.trim.startsWith("--")) + val (comments, code) = input.split("\n").partition { line => + val newLine = line.trim + newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER") + } // If `--IMPORT` found, load code from another test case file, then insert them // into the head in this test. @@ -261,10 +274,38 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { } }.flatten + val allCode = importedCode ++ code + val tempQueries = if (allCode.exists(_.trim.startsWith("--QUERY-DELIMITER"))) { + // Although the loop is heavy, only used for bracketed comments test. + val querys = new ArrayBuffer[String] + val otherCodes = new ArrayBuffer[String] + var tempStr = "" + var start = false + for (c <- allCode) { + if (c.trim.startsWith("--QUERY-DELIMITER-START")) { + start = true + querys ++= splitWithSemicolon(otherCodes.toSeq) + otherCodes.clear() + } else if (c.trim.startsWith("--QUERY-DELIMITER-END")) { + start = false + querys += s"\n${tempStr.stripSuffix(";")}" + tempStr = "" + } else if (start) { + tempStr += s"\n$c" + } else { + otherCodes += c + } + } + if (otherCodes.nonEmpty) { + querys ++= splitWithSemicolon(otherCodes.toSeq) + } + querys.toSeq + } else { + splitWithSemicolon(allCode).toSeq + } + // List of SQL queries to run - // note: this is not a robust way to split queries using semicolon, but works for now. - val queries = (importedCode ++ code).mkString("\n").split("(?<=[^\\\\]);") - .map(_.trim).filter(_ != "").toSeq + val queries = tempQueries.map(_.trim).filter(_ != "").toSeq // Fix misplacement when comment is at the end of the query. .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")).map(_.trim).filter(_ != "") From fb0e07b08ccaeda50a5121bcb1fab69a1ff749c4 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 13 Feb 2020 22:48:27 +0800 Subject: [PATCH 1365/1387] [SPARK-29231][SQL] Constraints should be inferred from cast equality constraint ### What changes were proposed in this pull request? This PR add support infer constraints from cast equality constraint. For example: ```scala scala> spark.sql("create table spark_29231_1(c1 bigint, c2 bigint)") res0: org.apache.spark.sql.DataFrame = [] scala> spark.sql("create table spark_29231_2(c1 int, c2 bigint)") res1: org.apache.spark.sql.DataFrame = [] scala> spark.sql("select t1.* from spark_29231_1 t1 join spark_29231_2 t2 on (t1.c1 = t2.c1 and t1.c1 = 1)").explain == Physical Plan == *(2) Project [c1#5L, c2#6L] +- *(2) BroadcastHashJoin [c1#5L], [cast(c1#7 as bigint)], Inner, BuildRight :- *(2) Project [c1#5L, c2#6L] : +- *(2) Filter (isnotnull(c1#5L) AND (c1#5L = 1)) : +- *(2) ColumnarToRow : +- FileScan parquet default.spark_29231_1[c1#5L,c2#6L] Batched: true, DataFilters: [isnotnull(c1#5L), (c1#5L = 1)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-preview2-bin-hadoop2.7/spark-warehouse/spark_29231_1], PartitionFilters: [], PushedFilters: [IsNotNull(c1), EqualTo(c1,1)], ReadSchema: struct +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#209] +- *(1) Project [c1#7] +- *(1) Filter isnotnull(c1#7) +- *(1) ColumnarToRow +- FileScan parquet default.spark_29231_2[c1#7] Batched: true, DataFilters: [isnotnull(c1#7)], Format: Parquet, Location: InMemoryFileIndex[file:/root/spark-3.0.0-preview2-bin-hadoop2.7/spark-warehouse/spark_29231_2], PartitionFilters: [], PushedFilters: [IsNotNull(c1)], ReadSchema: struct ``` After this PR: ```scala scala> spark.sql("select t1.* from spark_29231_1 t1 join spark_29231_2 t2 on (t1.c1 = t2.c1 and t1.c1 = 1)").explain == Physical Plan == *(2) Project [c1#0L, c2#1L] +- *(2) BroadcastHashJoin [c1#0L], [cast(c1#2 as bigint)], Inner, BuildRight :- *(2) Project [c1#0L, c2#1L] : +- *(2) Filter (isnotnull(c1#0L) AND (c1#0L = 1)) : +- *(2) ColumnarToRow : +- FileScan parquet default.spark_29231_1[c1#0L,c2#1L] Batched: true, DataFilters: [isnotnull(c1#0L), (c1#0L = 1)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/spark/spark-warehouse/spark_29231_1], PartitionFilters: [], PushedFilters: [IsNotNull(c1), EqualTo(c1,1)], ReadSchema: struct +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint))), [id=#99] +- *(1) Project [c1#2] +- *(1) Filter ((cast(c1#2 as bigint) = 1) AND isnotnull(c1#2)) +- *(1) ColumnarToRow +- FileScan parquet default.spark_29231_2[c1#2] Batched: true, DataFilters: [(cast(c1#2 as bigint) = 1), isnotnull(c1#2)], Format: Parquet, Location: InMemoryFileIndex[file:/root/opensource/spark/spark-warehouse/spark_29231_2], PartitionFilters: [], PushedFilters: [IsNotNull(c1)], ReadSchema: struct ``` ### Why are the changes needed? Improve query performance. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Unit test. Closes #27252 from wangyum/SPARK-29231. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan --- .../plans/logical/QueryPlanConstraints.scala | 12 +++- .../InferFiltersFromConstraintsSuite.scala | 57 ++++++++++++++++++- 2 files changed, 64 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala index 1355003358b9f..4c4ec000d0930 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala @@ -62,11 +62,17 @@ trait ConstraintHelper { */ def inferAdditionalConstraints(constraints: Set[Expression]): Set[Expression] = { var inferredConstraints = Set.empty[Expression] - constraints.foreach { + // IsNotNull should be constructed by `constructIsNotNullConstraints`. + val predicates = constraints.filterNot(_.isInstanceOf[IsNotNull]) + predicates.foreach { case eq @ EqualTo(l: Attribute, r: Attribute) => - val candidateConstraints = constraints - eq + val candidateConstraints = predicates - eq inferredConstraints ++= replaceConstraints(candidateConstraints, l, r) inferredConstraints ++= replaceConstraints(candidateConstraints, r, l) + case eq @ EqualTo(l @ Cast(_: Attribute, _, _), r: Attribute) => + inferredConstraints ++= replaceConstraints(predicates - eq, r, l) + case eq @ EqualTo(l: Attribute, r @ Cast(_: Attribute, _, _)) => + inferredConstraints ++= replaceConstraints(predicates - eq, l, r) case _ => // No inference } inferredConstraints -- constraints @@ -75,7 +81,7 @@ trait ConstraintHelper { private def replaceConstraints( constraints: Set[Expression], source: Expression, - destination: Attribute): Set[Expression] = constraints.map(_ transform { + destination: Expression): Set[Expression] = constraints.map(_ transform { case e: Expression if e.semanticEquals(source) => destination }) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index 974bc781d36ab..79bd573f1d84a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, LongType} class InferFiltersFromConstraintsSuite extends PlanTest { @@ -46,8 +47,8 @@ class InferFiltersFromConstraintsSuite extends PlanTest { y: LogicalPlan, expectedLeft: LogicalPlan, expectedRight: LogicalPlan, - joinType: JoinType) = { - val condition = Some("x.a".attr === "y.a".attr) + joinType: JoinType, + condition: Option[Expression] = Some("x.a".attr === "y.a".attr)) = { val originalQuery = x.join(y, joinType, condition).analyze val correctAnswer = expectedLeft.join(expectedRight, joinType, condition).analyze val optimized = Optimize.execute(originalQuery) @@ -263,4 +264,56 @@ class InferFiltersFromConstraintsSuite extends PlanTest { val y = testRelation.subquery('y) testConstraintsAfterJoin(x, y, x.where(IsNotNull('a)), y, RightOuter) } + + test("Constraints should be inferred from cast equality constraint(filter higher data type)") { + val testRelation1 = LocalRelation('a.int) + val testRelation2 = LocalRelation('b.long) + val originalLeft = testRelation1.subquery('left) + val originalRight = testRelation2.where('b === 1L).subquery('right) + + val left = testRelation1.where(IsNotNull('a) && 'a.cast(LongType) === 1L).subquery('left) + val right = testRelation2.where(IsNotNull('b) && 'b === 1L).subquery('right) + + Seq(Some("left.a".attr.cast(LongType) === "right.b".attr), + Some("right.b".attr === "left.a".attr.cast(LongType))).foreach { condition => + testConstraintsAfterJoin(originalLeft, originalRight, left, right, Inner, condition) + } + + Seq(Some("left.a".attr === "right.b".attr.cast(IntegerType)), + Some("right.b".attr.cast(IntegerType) === "left.a".attr)).foreach { condition => + testConstraintsAfterJoin( + originalLeft, + originalRight, + testRelation1.where(IsNotNull('a)).subquery('left), + right, + Inner, + condition) + } + } + + test("Constraints shouldn't be inferred from cast equality constraint(filter lower data type)") { + val testRelation1 = LocalRelation('a.int) + val testRelation2 = LocalRelation('b.long) + val originalLeft = testRelation1.where('a === 1).subquery('left) + val originalRight = testRelation2.subquery('right) + + val left = testRelation1.where(IsNotNull('a) && 'a === 1).subquery('left) + val right = testRelation2.where(IsNotNull('b)).subquery('right) + + Seq(Some("left.a".attr.cast(LongType) === "right.b".attr), + Some("right.b".attr === "left.a".attr.cast(LongType))).foreach { condition => + testConstraintsAfterJoin(originalLeft, originalRight, left, right, Inner, condition) + } + + Seq(Some("left.a".attr === "right.b".attr.cast(IntegerType)), + Some("right.b".attr.cast(IntegerType) === "left.a".attr)).foreach { condition => + testConstraintsAfterJoin( + originalLeft, + originalRight, + left, + testRelation2.where(IsNotNull('b) && 'b.attr.cast(IntegerType) === 1).subquery('right), + Inner, + condition) + } + } } From 82d0aa37ae521231d8067e473c6ea79a118a115a Mon Sep 17 00:00:00 2001 From: Liang Zhang Date: Thu, 13 Feb 2020 23:55:13 +0800 Subject: [PATCH 1366/1387] [SPARK-30762] Add dtype=float32 support to vector_to_array UDF ### What changes were proposed in this pull request? In this PR, we add a parameter in the python function vector_to_array(col) that allows converting to a column of arrays of Float (32bits) in scala, which would be mapped to a numpy array of dtype=float32. ### Why are the changes needed? In the downstream ML training, using float32 instead of float64 (default) would allow a larger batch size, i.e., allow more data to fit in the memory. ### Does this PR introduce any user-facing change? Yes. Old: `vector_to_array()` only take one param ``` df.select(vector_to_array("colA"), ...) ``` New: `vector_to_array()` can take an additional optional param: `dtype` = "float32" (or "float64") ``` df.select(vector_to_array("colA", "float32"), ...) ``` ### How was this patch tested? Unit test in scala. doctest in python. Closes #27522 from liangz1/udf-float32. Authored-by: Liang Zhang Signed-off-by: WeichenXu --- .../scala/org/apache/spark/ml/functions.scala | 34 ++++++++++++++++--- .../org/apache/spark/ml/FunctionsSuite.scala | 33 +++++++++++++++--- python/pyspark/ml/functions.py | 27 ++++++++++++--- 3 files changed, 81 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/functions.scala b/mllib/src/main/scala/org/apache/spark/ml/functions.scala index 1faf562c4d896..0f03231079866 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/functions.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/functions.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml import org.apache.spark.annotation.Since -import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.linalg.{SparseVector, Vector} import org.apache.spark.mllib.linalg.{Vector => OldVector} import org.apache.spark.sql.Column import org.apache.spark.sql.functions.udf @@ -27,7 +27,6 @@ import org.apache.spark.sql.functions.udf @Since("3.0.0") object functions { // scalastyle:on - private val vectorToArrayUdf = udf { vec: Any => vec match { case v: Vector => v.toArray @@ -39,10 +38,37 @@ object functions { } }.asNonNullable() + private val vectorToArrayFloatUdf = udf { vec: Any => + vec match { + case v: SparseVector => + val data = new Array[Float](v.size) + v.foreachActive { (index, value) => data(index) = value.toFloat } + data + case v: Vector => v.toArray.map(_.toFloat) + case v: OldVector => v.toArray.map(_.toFloat) + case v => throw new IllegalArgumentException( + "function vector_to_array requires a non-null input argument and input type must be " + + "`org.apache.spark.ml.linalg.Vector` or `org.apache.spark.mllib.linalg.Vector`, " + + s"but got ${ if (v == null) "null" else v.getClass.getName }.") + } + }.asNonNullable() + /** * Converts a column of MLlib sparse/dense vectors into a column of dense arrays. - * + * @param v: the column of MLlib sparse/dense vectors + * @param dtype: the desired underlying data type in the returned array + * @return an array<float> if dtype is float32, or array<double> if dtype is float64 * @since 3.0.0 */ - def vector_to_array(v: Column): Column = vectorToArrayUdf(v) + def vector_to_array(v: Column, dtype: String = "float64"): Column = { + if (dtype == "float64") { + vectorToArrayUdf(v) + } else if (dtype == "float32") { + vectorToArrayFloatUdf(v) + } else { + throw new IllegalArgumentException( + s"Unsupported dtype: $dtype. Valid values: float64, float32." + ) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala index 2f5062c689fc7..3dd9a7d8ec85d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/FunctionsSuite.scala @@ -34,9 +34,8 @@ class FunctionsSuite extends MLTest { (Vectors.sparse(3, Seq((0, 2.0), (2, 3.0))), OldVectors.sparse(3, Seq((0, 20.0), (2, 30.0)))) ).toDF("vec", "oldVec") - val result = df.select(vector_to_array('vec), vector_to_array('oldVec)) - .as[(Seq[Double], Seq[Double])] - .collect().toSeq + val result = df.select(vector_to_array('vec), vector_to_array('oldVec)) + .as[(Seq[Double], Seq[Double])].collect().toSeq val expected = Seq( (Seq(1.0, 2.0, 3.0), Seq(10.0, 20.0, 30.0)), @@ -50,7 +49,6 @@ class FunctionsSuite extends MLTest { (null, null, 0) ).toDF("vec", "oldVec", "label") - for ((colName, valType) <- Seq( ("vec", "null"), ("oldVec", "null"), ("label", "java.lang.Integer"))) { val thrown1 = intercept[SparkException] { @@ -61,5 +59,32 @@ class FunctionsSuite extends MLTest { "`org.apache.spark.ml.linalg.Vector` or `org.apache.spark.mllib.linalg.Vector`, " + s"but got ${valType}")) } + + val df3 = Seq( + (Vectors.dense(1.0, 2.0, 3.0), OldVectors.dense(10.0, 20.0, 30.0)), + (Vectors.sparse(3, Seq((0, 2.0), (2, 3.0))), OldVectors.sparse(3, Seq((0, 20.0), (2, 30.0)))) + ).toDF("vec", "oldVec") + val dfArrayFloat = df3.select( + vector_to_array('vec, dtype = "float32"), vector_to_array('oldVec, dtype = "float32")) + + // Check values are correct + val result3 = dfArrayFloat.as[(Seq[Float], Seq[Float])].collect().toSeq + + val expected3 = Seq( + (Seq(1.0, 2.0, 3.0), Seq(10.0, 20.0, 30.0)), + (Seq(2.0, 0.0, 3.0), Seq(20.0, 0.0, 30.0)) + ) + assert(result3 === expected3) + + // Check data types are correct + assert(dfArrayFloat.schema.simpleString === + "struct,UDF(oldVec):array>") + + val thrown2 = intercept[IllegalArgumentException] { + df3.select( + vector_to_array('vec, dtype = "float16"), vector_to_array('oldVec, dtype = "float16")) + } + assert(thrown2.getMessage.contains( + s"Unsupported dtype: float16. Valid values: float64, float32.")) } } diff --git a/python/pyspark/ml/functions.py b/python/pyspark/ml/functions.py index 2b4d8ddcd00a8..ec164f34bc4db 100644 --- a/python/pyspark/ml/functions.py +++ b/python/pyspark/ml/functions.py @@ -19,10 +19,15 @@ from pyspark.sql.column import Column, _to_java_column -@since(3.0) -def vector_to_array(col): +@since("3.0.0") +def vector_to_array(col, dtype="float64"): """ Converts a column of MLlib sparse/dense vectors into a column of dense arrays. + :param col: A string of the column name or a Column + :param dtype: The data type of the output array. Valid values: "float64" or "float32". + :return: The converted column of dense arrays. + + .. versionadded:: 3.0.0 >>> from pyspark.ml.linalg import Vectors >>> from pyspark.ml.functions import vector_to_array @@ -32,14 +37,26 @@ def vector_to_array(col): ... (Vectors.sparse(3, [(0, 2.0), (2, 3.0)]), ... OldVectors.sparse(3, [(0, 20.0), (2, 30.0)]))], ... ["vec", "oldVec"]) - >>> df.select(vector_to_array("vec").alias("vec"), - ... vector_to_array("oldVec").alias("oldVec")).collect() + >>> df1 = df.select(vector_to_array("vec").alias("vec"), + ... vector_to_array("oldVec").alias("oldVec")) + >>> df1.collect() + [Row(vec=[1.0, 2.0, 3.0], oldVec=[10.0, 20.0, 30.0]), + Row(vec=[2.0, 0.0, 3.0], oldVec=[20.0, 0.0, 30.0])] + >>> df2 = df.select(vector_to_array("vec", "float32").alias("vec"), + ... vector_to_array("oldVec", "float32").alias("oldVec")) + >>> df2.collect() [Row(vec=[1.0, 2.0, 3.0], oldVec=[10.0, 20.0, 30.0]), Row(vec=[2.0, 0.0, 3.0], oldVec=[20.0, 0.0, 30.0])] + >>> df1.schema.fields + [StructField(vec,ArrayType(DoubleType,false),false), + StructField(oldVec,ArrayType(DoubleType,false),false)] + >>> df2.schema.fields + [StructField(vec,ArrayType(FloatType,false),false), + StructField(oldVec,ArrayType(FloatType,false),false)] """ sc = SparkContext._active_spark_context return Column( - sc._jvm.org.apache.spark.ml.functions.vector_to_array(_to_java_column(col))) + sc._jvm.org.apache.spark.ml.functions.vector_to_array(_to_java_column(col), dtype)) def _test(): From 3c4044ea77fe3b1268b52744cd4f1ae61f17a9a8 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 13 Feb 2020 10:53:55 -0800 Subject: [PATCH 1367/1387] [SPARK-30703][SQL][DOCS] Add a document for the ANSI mode ### What changes were proposed in this pull request? This pr intends to add a document for the ANSI mode; Screen Shot 2020-02-13 at 8 08 52 Screen Shot 2020-02-13 at 8 09 13 Screen Shot 2020-02-13 at 8 09 26 Screen Shot 2020-02-13 at 8 09 38 ### Why are the changes needed? For better document coverage and usability. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? N/A Closes #27489 from maropu/SPARK-30703. Authored-by: Takeshi Yamamuro Signed-off-by: Gengliang Wang --- docs/_data/menu-sql.yaml | 11 +- ...keywords.md => sql-ref-ansi-compliance.md} | 125 +++++++++++++++++- docs/sql-ref-arithmetic-ops.md | 22 --- 3 files changed, 132 insertions(+), 26 deletions(-) rename docs/{sql-keywords.md => sql-ref-ansi-compliance.md} (82%) delete mode 100644 docs/sql-ref-arithmetic-ops.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 241ec399d7bd5..1e343f630f88e 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -80,6 +80,15 @@ url: sql-ref-null-semantics.html - text: NaN Semantics url: sql-ref-nan-semantics.html + - text: ANSI Compliance + url: sql-ref-ansi-compliance.html + subitems: + - text: Arithmetic Operations + url: sql-ref-ansi-compliance.html#arithmetic-operations + - text: Type Conversion + url: sql-ref-ansi-compliance.html#type-conversion + - text: SQL Keywords + url: sql-ref-ansi-compliance.html#sql-keywords - text: SQL Syntax url: sql-ref-syntax.html subitems: @@ -214,5 +223,3 @@ url: sql-ref-syntax-aux-resource-mgmt-list-file.html - text: LIST JAR url: sql-ref-syntax-aux-resource-mgmt-list-jar.html - - text: Arithmetic operations - url: sql-ref-arithmetic-ops.html diff --git a/docs/sql-keywords.md b/docs/sql-ref-ansi-compliance.md similarity index 82% rename from docs/sql-keywords.md rename to docs/sql-ref-ansi-compliance.md index 9e4a3c54100c6..d02383518b967 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-ref-ansi-compliance.md @@ -1,7 +1,7 @@ --- layout: global -title: Spark SQL Keywords -displayTitle: Spark SQL Keywords +title: ANSI Compliance +displayTitle: ANSI Compliance license: | Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file distributed with @@ -19,6 +19,127 @@ license: | limitations under the License. --- +Spark SQL has two options to comply with the SQL standard: `spark.sql.ansi.enabled` and `spark.sql.storeAssignmentPolicy` (See a table below for details). +When `spark.sql.ansi.enabled` is set to `true`, Spark SQL follows the standard in basic behaviours (e.g., arithmetic operations, type conversion, and SQL parsing). +Moreover, Spark SQL has an independent option to control implicit casting behaviours when inserting rows in a table. +The casting behaviours are defined as store assignment rules in the standard. +When `spark.sql.storeAssignmentPolicy` is set to `ANSI`, Spark SQL complies with the ANSI store assignment rules. + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.ansi.enabledfalse + When true, Spark tries to conform to the ANSI SQL specification: + 1. Spark will throw a runtime exception if an overflow occurs in any operation on integral/decimal field. + 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in the SQL parser. +
    spark.sql.storeAssignmentPolicyANSI + When inserting a value into a column with different data type, Spark will perform type coercion. + Currently, we support 3 policies for the type coercion rules: ANSI, legacy and strict. With ANSI policy, + Spark performs the type coercion as per ANSI SQL. In practice, the behavior is mostly the same as PostgreSQL. + It disallows certain unreasonable type conversions such as converting string to int or double to boolean. + With legacy policy, Spark allows the type coercion as long as it is a valid Cast, which is very loose. + e.g. converting string to int or double to boolean is allowed. + It is also the only behavior in Spark 2.x and it is compatible with Hive. + With strict policy, Spark doesn't allow any possible precision loss or data truncation in type coercion, + e.g. converting double to int or decimal to double is not allowed. +
    + +The following subsections present behaviour changes in arithmetic operations, type conversions, and SQL parsing when the ANSI mode enabled. + +### Arithmetic Operations + +In Spark SQL, arithmetic operations performed on numeric types (with the exception of decimal) are not checked for overflows by default. +This means that in case an operation causes overflows, the result is the same that the same operation returns in a Java/Scala program (e.g., if the sum of 2 integers is higher than the maximum value representable, the result is a negative number). +On the other hand, Spark SQL returns null for decimal overflows. +When `spark.sql.ansi.enabled` is set to `true` and an overflow occurs in numeric and interval arithmetic operations, it throws an arithmetic exception at runtime. + +{% highlight sql %} +-- `spark.sql.ansi.enabled=true` +SELECT 2147483647 + 1; + + java.lang.ArithmeticException: integer overflow + +-- `spark.sql.ansi.enabled=false` +SELECT 2147483647 + 1; + + +----------------+ + |(2147483647 + 1)| + +----------------+ + | -2147483648| + +----------------+ + +{% endhighlight %} + +### Type Conversion + +Spark SQL has three kinds of type conversions: explicit casting, type coercion, and store assignment casting. +When `spark.sql.ansi.enabled` is set to `true`, explicit casting by `CAST` syntax throws a runtime exception for illegal cast patterns defined in the standard, e.g. casts from a string to an integer. +On the other hand, `INSERT INTO` syntax throws an analysis exception when the ANSI mode enabled via `spark.sql.storeAssignmentPolicy=ANSI`. + +Currently, the ANSI mode affects explicit casting and assignment casting only. +In future releases, the behaviour of type coercion might change along with the other two type conversion rules. + +{% highlight sql %} +-- Examples of explicit casting + +-- `spark.sql.ansi.enabled=true` +SELECT CAST('a' AS INT); + + java.lang.NumberFormatException: invalid input syntax for type numeric: a + +SELECT CAST(2147483648L AS INT); + + java.lang.ArithmeticException: Casting 2147483648 to int causes overflow + +-- `spark.sql.ansi.enabled=false` (This is a default behaviour) +SELECT CAST('a' AS INT); + + +--------------+ + |CAST(a AS INT)| + +--------------+ + | null| + +--------------+ + +SELECT CAST(2147483648L AS INT); + + +-----------------------+ + |CAST(2147483648 AS INT)| + +-----------------------+ + | -2147483648| + +-----------------------+ + +-- Examples of store assignment rules +CREATE TABLE t (v INT); + +-- `spark.sql.storeAssignmentPolicy=ANSI` +INSERT INTO t VALUES ('1'); + + org.apache.spark.sql.AnalysisException: Cannot write incompatible data to table '`default`.`t`': + - Cannot safely cast 'v': StringType to IntegerType; + +-- `spark.sql.storeAssignmentPolicy=LEGACY` (This is a legacy behaviour until Spark 2.x) +INSERT INTO t VALUES ('1'); +SELECT * FROM t; + + +---+ + | v| + +---+ + | 1| + +---+ + +{% endhighlight %} + +### SQL Keywords + When `spark.sql.ansi.enabled` is true, Spark SQL will use the ANSI mode parser. In this mode, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. diff --git a/docs/sql-ref-arithmetic-ops.md b/docs/sql-ref-arithmetic-ops.md deleted file mode 100644 index 7bc8ffe31c990..0000000000000 --- a/docs/sql-ref-arithmetic-ops.md +++ /dev/null @@ -1,22 +0,0 @@ ---- -layout: global -title: Arithmetic Operations -displayTitle: Arithmetic Operations -license: | - 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. ---- - -Operations performed on numeric types (with the exception of decimal) are not checked for overflow. This means that in case an operation causes an overflow, the result is the same that the same operation returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, the result is a negative number). From a4ceea6868002b88161517b14b94a2006be8af1b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 13 Feb 2020 20:09:24 +0100 Subject: [PATCH 1368/1387] [SPARK-30751][SQL] Combine the skewed readers into one in AQE skew join optimizations ### What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/26434 This PR use one special shuffle reader for skew join, so that we only have one join after optimization. In order to do that, this PR 1. add a very general `CustomShuffledRowRDD` which support all kind of partition arrangement. 2. move the logic of coalescing shuffle partitions to a util function, and call it during skew join optimization, to totally decouple with the `ReduceNumShufflePartitions` rule. It's too complicated to interfere skew join with `ReduceNumShufflePartitions`, as you need to consider the size of split partitions which don't respect target size already. ### Why are the changes needed? The current skew join optimization has a serious performance issue: the size of the query plan depends on the number and size of skewed partitions. ### Does this PR introduce any user-facing change? no ### How was this patch tested? existing tests test UI manually: ![image](https://user-images.githubusercontent.com/3182036/74357390-cfb30480-4dfa-11ea-83f6-825d1b9379ca.png) explain output ``` AdaptiveSparkPlan(isFinalPlan=true) +- OverwriteByExpression org.apache.spark.sql.execution.datasources.noop.NoopTable$403a2ed5, [AlwaysTrue()], org.apache.spark.sql.util.CaseInsensitiveStringMap1f +- *(5) SortMergeJoin(skew=true) [key1#2L], [key2#6L], Inner :- *(3) Sort [key1#2L ASC NULLS FIRST], false, 0 : +- SkewJoinShuffleReader 2 skewed partitions with size(max=5 KB, min=5 KB, avg=5 KB) : +- ShuffleQueryStage 0 : +- Exchange hashpartitioning(key1#2L, 200), true, [id=#53] : +- *(1) Project [(id#0L % 2) AS key1#2L] : +- *(1) Filter isnotnull((id#0L % 2)) : +- *(1) Range (0, 100000, step=1, splits=6) +- *(4) Sort [key2#6L ASC NULLS FIRST], false, 0 +- SkewJoinShuffleReader 2 skewed partitions with size(max=5 KB, min=5 KB, avg=5 KB) +- ShuffleQueryStage 1 +- Exchange hashpartitioning(key2#6L, 200), true, [id=#64] +- *(2) Project [((id#4L % 2) + 1) AS key2#6L] +- *(2) Filter isnotnull(((id#4L % 2) + 1)) +- *(2) Range (0, 100000, step=1, splits=6) ``` Closes #27493 from cloud-fan/aqe. Authored-by: Wenchen Fan Signed-off-by: herman --- .../spark/sql/execution/ShuffledRowRDD.scala | 23 +- .../adaptive/CustomShuffledRowRDD.scala | 113 +++++++ .../adaptive/OptimizeLocalShuffleReader.scala | 2 +- .../adaptive/OptimizeSkewedJoin.scala | 276 +++++++++++------- .../adaptive/ReduceNumShufflePartitions.scala | 157 ++-------- .../adaptive/ShufflePartitionsCoalescer.scala | 112 +++++++ .../adaptive/SkewedShuffledRowRDD.scala | 78 ----- .../exchange/ShuffleExchangeExec.scala | 21 +- .../execution/joins/SortMergeJoinExec.scala | 13 +- .../ReduceNumShufflePartitionsSuite.scala | 210 +------------ .../ShufflePartitionsCoalescerSuite.scala | 220 ++++++++++++++ .../adaptive/AdaptiveQueryExecSuite.scala | 219 +++++--------- 12 files changed, 741 insertions(+), 703 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffledRowRDD.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsCoalescer.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsCoalescerSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala index efa493923ccc1..4c19f95796d04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ShuffledRowRDD.scala @@ -116,7 +116,7 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A class ShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], metrics: Map[String, SQLMetric], - specifiedPartitionIndices: Option[Array[(Int, Int)]] = None) + specifiedPartitionStartIndices: Option[Array[Int]] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { if (SQLConf.get.fetchShuffleBlocksInBatchEnabled) { @@ -126,8 +126,8 @@ class ShuffledRowRDD( private[this] val numPreShufflePartitions = dependency.partitioner.numPartitions - private[this] val partitionStartIndices: Array[Int] = specifiedPartitionIndices match { - case Some(indices) => indices.map(_._1) + private[this] val partitionStartIndices: Array[Int] = specifiedPartitionStartIndices match { + case Some(indices) => indices case None => // When specifiedPartitionStartIndices is not defined, every post-shuffle partition // corresponds to a pre-shuffle partition. @@ -142,15 +142,16 @@ class ShuffledRowRDD( override val partitioner: Option[Partitioner] = Some(part) override def getPartitions: Array[Partition] = { - specifiedPartitionIndices match { - case Some(indices) => - Array.tabulate[Partition](indices.length) { i => - new ShuffledRowRDDPartition(i, indices(i)._1, indices(i)._2) - } - case None => - Array.tabulate[Partition](numPreShufflePartitions) { i => - new ShuffledRowRDDPartition(i, i, i + 1) + assert(partitionStartIndices.length == part.numPartitions) + Array.tabulate[Partition](partitionStartIndices.length) { i => + val startIndex = partitionStartIndices(i) + val endIndex = + if (i < partitionStartIndices.length - 1) { + partitionStartIndices(i + 1) + } else { + numPreShufflePartitions } + new ShuffledRowRDDPartition(i, startIndex, endIndex) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffledRowRDD.scala new file mode 100644 index 0000000000000..5aba57443d632 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffledRowRDD.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.adaptive + +import org.apache.spark.{Dependency, MapOutputTrackerMaster, Partition, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} + +sealed trait ShufflePartitionSpec + +// A partition that reads data of one reducer. +case class SinglePartitionSpec(reducerIndex: Int) extends ShufflePartitionSpec + +// A partition that reads data of multiple reducers, from `startReducerIndex` (inclusive) to +// `endReducerIndex` (exclusive). +case class CoalescedPartitionSpec( + startReducerIndex: Int, endReducerIndex: Int) extends ShufflePartitionSpec + +// A partition that reads partial data of one reducer, from `startMapIndex` (inclusive) to +// `endMapIndex` (exclusive). +case class PartialPartitionSpec( + reducerIndex: Int, startMapIndex: Int, endMapIndex: Int) extends ShufflePartitionSpec + +private final case class CustomShufflePartition( + index: Int, spec: ShufflePartitionSpec) extends Partition + +// TODO: merge this with `ShuffledRowRDD`, and replace `LocalShuffledRowRDD` with this RDD. +class CustomShuffledRowRDD( + var dependency: ShuffleDependency[Int, InternalRow, InternalRow], + metrics: Map[String, SQLMetric], + partitionSpecs: Array[ShufflePartitionSpec]) + extends RDD[InternalRow](dependency.rdd.context, Nil) { + + override def getDependencies: Seq[Dependency[_]] = List(dependency) + + override def clearDependencies() { + super.clearDependencies() + dependency = null + } + + override def getPartitions: Array[Partition] = { + Array.tabulate[Partition](partitionSpecs.length) { i => + CustomShufflePartition(i, partitionSpecs(i)) + } + } + + override def getPreferredLocations(partition: Partition): Seq[String] = { + val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + partition.asInstanceOf[CustomShufflePartition].spec match { + case SinglePartitionSpec(reducerIndex) => + tracker.getPreferredLocationsForShuffle(dependency, reducerIndex) + + case CoalescedPartitionSpec(startReducerIndex, endReducerIndex) => + startReducerIndex.until(endReducerIndex).flatMap { reducerIndex => + tracker.getPreferredLocationsForShuffle(dependency, reducerIndex) + } + + case PartialPartitionSpec(_, startMapIndex, endMapIndex) => + tracker.getMapLocation(dependency, startMapIndex, endMapIndex) + } + } + + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() + // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, + // as well as the `tempMetrics` for basic shuffle metrics. + val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) + val reader = split.asInstanceOf[CustomShufflePartition].spec match { + case SinglePartitionSpec(reducerIndex) => + SparkEnv.get.shuffleManager.getReader( + dependency.shuffleHandle, + reducerIndex, + reducerIndex + 1, + context, + sqlMetricsReporter) + + case CoalescedPartitionSpec(startReducerIndex, endReducerIndex) => + SparkEnv.get.shuffleManager.getReader( + dependency.shuffleHandle, + startReducerIndex, + endReducerIndex, + context, + sqlMetricsReporter) + + case PartialPartitionSpec(reducerIndex, startMapIndex, endMapIndex) => + SparkEnv.get.shuffleManager.getReaderForRange( + dependency.shuffleHandle, + startMapIndex, + endMapIndex, + reducerIndex, + reducerIndex + 1, + context, + sqlMetricsReporter) + } + reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala index a8d8f358ab660..e95441e28aafe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeLocalShuffleReader.scala @@ -71,7 +71,7 @@ case class OptimizeLocalShuffleReader(conf: SQLConf) extends Rule[SparkPlan] { plan match { case c @ CoalescedShuffleReaderExec(s: ShuffleQueryStageExec, _) => LocalShuffleReaderExec( - s, getPartitionStartIndices(s, Some(c.partitionIndices.length))) + s, getPartitionStartIndices(s, Some(c.partitionStartIndices.length))) case s: ShuffleQueryStageExec => LocalShuffleReaderExec(s, getPartitionStartIndices(s, None)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index 74b7fbd317fc8..a716497c274b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.adaptive import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.commons.io.FileUtils + import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkEnv} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -44,11 +46,7 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { * partition size * spark.sql.adaptive.skewedPartitionFactor and also larger than * spark.sql.adaptive.skewedPartitionSizeThreshold. */ - private def isSkewed( - stats: MapOutputStatistics, - partitionId: Int, - medianSize: Long): Boolean = { - val size = stats.bytesByPartitionId(partitionId) + private def isSkewed(size: Long, medianSize: Long): Boolean = { size > medianSize * conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR) && size > conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD) } @@ -108,12 +106,12 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { stage.resultOption.get.asInstanceOf[MapOutputStatistics] } - private def supportSplitOnLeftPartition(joinType: JoinType) = { + private def canSplitLeftSide(joinType: JoinType) = { joinType == Inner || joinType == Cross || joinType == LeftSemi || joinType == LeftAnti || joinType == LeftOuter } - private def supportSplitOnRightPartition(joinType: JoinType) = { + private def canSplitRightSide(joinType: JoinType) = { joinType == Inner || joinType == Cross || joinType == RightOuter } @@ -130,17 +128,18 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { * 1. Check whether the shuffle partition is skewed based on the median size * and the skewed partition threshold in origin smj. * 2. Assuming partition0 is skewed in left side, and it has 5 mappers (Map0, Map1...Map4). - * And we will split the 5 Mappers into 3 mapper ranges [(Map0, Map1), (Map2, Map3), (Map4)] + * And we may split the 5 Mappers into 3 mapper ranges [(Map0, Map1), (Map2, Map3), (Map4)] * based on the map size and the max split number. - * 3. Create the 3 smjs with separately reading the above mapper ranges and then join with - * the Partition0 in right side. - * 4. Finally union the above 3 split smjs and the origin smj. + * 3. Wrap the join left child with a special shuffle reader that reads each mapper range with one + * task, so total 3 tasks. + * 4. Wrap the join right child with a special shuffle reader that reads partition0 3 times by + * 3 tasks separately. */ def optimizeSkewJoin(plan: SparkPlan): SparkPlan = plan.transformUp { - case smj @ SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, + case smj @ SortMergeJoinExec(_, _, joinType, _, s1 @ SortExec(_, _, left: ShuffleQueryStageExec, _), s2 @ SortExec(_, _, right: ShuffleQueryStageExec, _), _) - if (supportedJoinTypes.contains(joinType)) => + if supportedJoinTypes.contains(joinType) => val leftStats = getStatistics(left) val rightStats = getStatistics(right) val numPartitions = leftStats.bytesByPartitionId.length @@ -155,61 +154,134 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { |Right side partition size: |${getSizeInfo(rightMedSize, rightStats.bytesByPartitionId.max)} """.stripMargin) + val canSplitLeft = canSplitLeftSide(joinType) + val canSplitRight = canSplitRightSide(joinType) + + val leftSidePartitions = mutable.ArrayBuffer.empty[ShufflePartitionSpec] + val rightSidePartitions = mutable.ArrayBuffer.empty[ShufflePartitionSpec] + // This is used to delay the creation of non-skew partitions so that we can potentially + // coalesce them like `ReduceNumShufflePartitions` does. + val nonSkewPartitionIndices = mutable.ArrayBuffer.empty[Int] + val leftSkewDesc = new SkewDesc + val rightSkewDesc = new SkewDesc + for (partitionIndex <- 0 until numPartitions) { + val leftSize = leftStats.bytesByPartitionId(partitionIndex) + val isLeftSkew = isSkewed(leftSize, leftMedSize) && canSplitLeft + val rightSize = rightStats.bytesByPartitionId(partitionIndex) + val isRightSkew = isSkewed(rightSize, rightMedSize) && canSplitRight + if (isLeftSkew || isRightSkew) { + if (nonSkewPartitionIndices.nonEmpty) { + // As soon as we see a skew, we'll "flush" out unhandled non-skew partitions. + createNonSkewPartitions(leftStats, rightStats, nonSkewPartitionIndices).foreach { p => + leftSidePartitions += p + rightSidePartitions += p + } + nonSkewPartitionIndices.clear() + } - val skewedPartitions = mutable.HashSet[Int]() - val subJoins = mutable.ArrayBuffer[SparkPlan]() - for (partitionId <- 0 until numPartitions) { - val isLeftSkew = isSkewed(leftStats, partitionId, leftMedSize) - val isRightSkew = isSkewed(rightStats, partitionId, rightMedSize) - val leftMapIdStartIndices = if (isLeftSkew && supportSplitOnLeftPartition(joinType)) { - getMapStartIndices(left, partitionId) - } else { - Array(0) - } - val rightMapIdStartIndices = if (isRightSkew && supportSplitOnRightPartition(joinType)) { - getMapStartIndices(right, partitionId) - } else { - Array(0) - } + val leftParts = if (isLeftSkew) { + leftSkewDesc.addPartitionSize(leftSize) + createSkewPartitions( + partitionIndex, + getMapStartIndices(left, partitionIndex), + getNumMappers(left)) + } else { + Seq(SinglePartitionSpec(partitionIndex)) + } - if (leftMapIdStartIndices.length > 1 || rightMapIdStartIndices.length > 1) { - skewedPartitions += partitionId - for (i <- 0 until leftMapIdStartIndices.length; - j <- 0 until rightMapIdStartIndices.length) { - val leftEndMapId = if (i == leftMapIdStartIndices.length - 1) { - getNumMappers(left) - } else { - leftMapIdStartIndices(i + 1) - } - val rightEndMapId = if (j == rightMapIdStartIndices.length - 1) { - getNumMappers(right) - } else { - rightMapIdStartIndices(j + 1) + val rightParts = if (isRightSkew) { + rightSkewDesc.addPartitionSize(rightSize) + createSkewPartitions( + partitionIndex, + getMapStartIndices(right, partitionIndex), + getNumMappers(right)) + } else { + Seq(SinglePartitionSpec(partitionIndex)) + } + + for { + leftSidePartition <- leftParts + rightSidePartition <- rightParts + } { + leftSidePartitions += leftSidePartition + rightSidePartitions += rightSidePartition + } + } else { + // Add to `nonSkewPartitionIndices` first, and add real partitions later, in case we can + // coalesce the non-skew partitions. + nonSkewPartitionIndices += partitionIndex + // If this is the last partition, add real partition immediately. + if (partitionIndex == numPartitions - 1) { + createNonSkewPartitions(leftStats, rightStats, nonSkewPartitionIndices).foreach { p => + leftSidePartitions += p + rightSidePartitions += p } - // TODO: we may can optimize the sort merge join to broad cast join after - // obtaining the raw data size of per partition, - val leftSkewedReader = SkewedPartitionReaderExec( - left, partitionId, leftMapIdStartIndices(i), leftEndMapId) - val rightSkewedReader = SkewedPartitionReaderExec(right, partitionId, - rightMapIdStartIndices(j), rightEndMapId) - subJoins += SortMergeJoinExec(leftKeys, rightKeys, joinType, condition, - s1.copy(child = leftSkewedReader), s2.copy(child = rightSkewedReader), true) + nonSkewPartitionIndices.clear() } } } - logDebug(s"number of skewed partitions is ${skewedPartitions.size}") - if (skewedPartitions.nonEmpty) { - val optimizedSmj = smj.copy( - left = s1.copy(child = PartialShuffleReaderExec(left, skewedPartitions.toSet)), - right = s2.copy(child = PartialShuffleReaderExec(right, skewedPartitions.toSet)), - isPartial = true) - subJoins += optimizedSmj - UnionExec(subJoins) + + logDebug("number of skewed partitions: " + + s"left ${leftSkewDesc.numPartitions}, right ${rightSkewDesc.numPartitions}") + if (leftSkewDesc.numPartitions > 0 || rightSkewDesc.numPartitions > 0) { + val newLeft = SkewJoinShuffleReaderExec( + left, leftSidePartitions.toArray, leftSkewDesc.toString) + val newRight = SkewJoinShuffleReaderExec( + right, rightSidePartitions.toArray, rightSkewDesc.toString) + smj.copy( + left = s1.copy(child = newLeft), right = s2.copy(child = newRight), isSkewJoin = true) } else { smj } } + private def createNonSkewPartitions( + leftStats: MapOutputStatistics, + rightStats: MapOutputStatistics, + nonSkewPartitionIndices: Seq[Int]): Seq[ShufflePartitionSpec] = { + assert(nonSkewPartitionIndices.nonEmpty) + if (nonSkewPartitionIndices.length == 1) { + Seq(SinglePartitionSpec(nonSkewPartitionIndices.head)) + } else { + val startIndices = ShufflePartitionsCoalescer.coalescePartitions( + Array(leftStats, rightStats), + firstPartitionIndex = nonSkewPartitionIndices.head, + // `lastPartitionIndex` is exclusive. + lastPartitionIndex = nonSkewPartitionIndices.last + 1, + advisoryTargetSize = conf.targetPostShuffleInputSize) + startIndices.indices.map { i => + val startIndex = startIndices(i) + val endIndex = if (i == startIndices.length - 1) { + // `endIndex` is exclusive. + nonSkewPartitionIndices.last + 1 + } else { + startIndices(i + 1) + } + // Do not create `CoalescedPartitionSpec` if only need to read a singe partition. + if (startIndex + 1 == endIndex) { + SinglePartitionSpec(startIndex) + } else { + CoalescedPartitionSpec(startIndex, endIndex) + } + } + } + } + + private def createSkewPartitions( + reducerIndex: Int, + mapStartIndices: Array[Int], + numMappers: Int): Seq[PartialPartitionSpec] = { + mapStartIndices.indices.map { i => + val startMapIndex = mapStartIndices(i) + val endMapIndex = if (i == mapStartIndices.length - 1) { + numMappers + } else { + mapStartIndices(i + 1) + } + PartialPartitionSpec(reducerIndex, startMapIndex, endMapIndex) + } + } + override def apply(plan: SparkPlan): SparkPlan = { if (!conf.getConf(SQLConf.ADAPTIVE_EXECUTION_SKEWED_JOIN_ENABLED)) { return plan @@ -248,79 +320,69 @@ case class OptimizeSkewedJoin(conf: SQLConf) extends Rule[SparkPlan] { } } -/** - * A wrapper of shuffle query stage, which submits one reduce task to read a single - * shuffle partition 'partitionIndex' produced by the mappers in range [startMapIndex, endMapIndex). - * This is used to increase the parallelism when reading skewed partitions. - * - * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange - * node during canonicalization. - * @param partitionIndex The pre shuffle partition index. - * @param startMapIndex The start map index. - * @param endMapIndex The end map index. - */ -case class SkewedPartitionReaderExec( - child: QueryStageExec, - partitionIndex: Int, - startMapIndex: Int, - endMapIndex: Int) extends LeafExecNode { +private class SkewDesc { + private[this] var numSkewedPartitions: Int = 0 + private[this] var totalSize: Long = 0 + private[this] var maxSize: Long = 0 + private[this] var minSize: Long = 0 - override def output: Seq[Attribute] = child.output + def numPartitions: Int = numSkewedPartitions - override def outputPartitioning: Partitioning = { - UnknownPartitioning(1) + def addPartitionSize(size: Long): Unit = { + if (numSkewedPartitions == 0) { + maxSize = size + minSize = size + } + numSkewedPartitions += 1 + totalSize += size + if (size > maxSize) maxSize = size + if (size < minSize) minSize = size } - private var cachedSkewedShuffleRDD: SkewedShuffledRowRDD = null - override def doExecute(): RDD[InternalRow] = { - if (cachedSkewedShuffleRDD == null) { - cachedSkewedShuffleRDD = child match { - case stage: ShuffleQueryStageExec => - stage.shuffle.createSkewedShuffleRDD(partitionIndex, startMapIndex, endMapIndex) - case _ => - throw new IllegalStateException("operating on canonicalization plan") - } + override def toString: String = { + if (numSkewedPartitions == 0) { + "no skewed partition" + } else { + val maxSizeStr = FileUtils.byteCountToDisplaySize(maxSize) + val minSizeStr = FileUtils.byteCountToDisplaySize(minSize) + val avgSizeStr = FileUtils.byteCountToDisplaySize(totalSize / numSkewedPartitions) + s"$numSkewedPartitions skewed partitions with " + + s"size(max=$maxSizeStr, min=$minSizeStr, avg=$avgSizeStr)" } - cachedSkewedShuffleRDD } } /** - * A wrapper of shuffle query stage, which skips some partitions when reading the shuffle blocks. + * A wrapper of shuffle query stage, which follows the given partition arrangement. * * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange node during * canonicalization. - * @param excludedPartitions The partitions to skip when reading. + * @param partitionSpecs The partition specs that defines the arrangement. + * @param skewDesc The description of the skewed partitions. */ -case class PartialShuffleReaderExec( - child: QueryStageExec, - excludedPartitions: Set[Int]) extends UnaryExecNode { +case class SkewJoinShuffleReaderExec( + child: SparkPlan, + partitionSpecs: Array[ShufflePartitionSpec], + skewDesc: String) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = { - UnknownPartitioning(1) + UnknownPartitioning(partitionSpecs.length) } - private def shuffleExchange(): ShuffleExchangeExec = child match { - case stage: ShuffleQueryStageExec => stage.shuffle - case _ => - throw new IllegalStateException("operating on canonicalization plan") - } - - private def getPartitionIndexRanges(): Array[(Int, Int)] = { - val length = shuffleExchange().shuffleDependency.partitioner.numPartitions - (0 until length).filterNot(excludedPartitions.contains).map(i => (i, i + 1)).toArray - } + override def stringArgs: Iterator[Any] = Iterator(skewDesc) private var cachedShuffleRDD: RDD[InternalRow] = null - override def doExecute(): RDD[InternalRow] = { + override protected def doExecute(): RDD[InternalRow] = { if (cachedShuffleRDD == null) { - cachedShuffleRDD = if (excludedPartitions.isEmpty) { - child.execute() - } else { - shuffleExchange().createShuffledRDD(Some(getPartitionIndexRanges())) + cachedShuffleRDD = child match { + case stage: ShuffleQueryStageExec => + new CustomShuffledRowRDD( + stage.shuffle.shuffleDependency, stage.shuffle.readMetrics, partitionSpecs) + case _ => + throw new IllegalStateException("operating on canonicalization plan") } } cachedShuffleRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala index 2c50b638b4d45..5bbcb14e008d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.adaptive -import scala.collection.mutable.{ArrayBuffer, HashSet} - import org.apache.spark.MapOutputStatistics import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -29,24 +27,8 @@ import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} import org.apache.spark.sql.internal.SQLConf /** - * A rule to adjust the post shuffle partitions based on the map output statistics. - * - * The strategy used to determine the number of post-shuffle partitions is described as follows. - * To determine the number of post-shuffle partitions, we have a target input size for a - * post-shuffle partition. Once we have size statistics of all pre-shuffle partitions, we will do - * a pass of those statistics and pack pre-shuffle partitions with continuous indices to a single - * post-shuffle partition until adding another pre-shuffle partition would cause the size of a - * post-shuffle partition to be greater than the target size. - * - * For example, we have two stages with the following pre-shuffle partition size statistics: - * stage 1: [100 MiB, 20 MiB, 100 MiB, 10MiB, 30 MiB] - * stage 2: [10 MiB, 10 MiB, 70 MiB, 5 MiB, 5 MiB] - * assuming the target input size is 128 MiB, we will have four post-shuffle partitions, - * which are: - * - post-shuffle partition 0: pre-shuffle partition 0 (size 110 MiB) - * - post-shuffle partition 1: pre-shuffle partition 1 (size 30 MiB) - * - post-shuffle partition 2: pre-shuffle partition 2 (size 170 MiB) - * - post-shuffle partition 3: pre-shuffle partition 3 and 4 (size 50 MiB) + * A rule to reduce the post shuffle partitions based on the map output statistics, which can + * avoid many small reduce tasks that hurt performance. */ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { @@ -54,28 +36,21 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { if (!conf.reducePostShufflePartitionsEnabled) { return plan } - // 'SkewedShufflePartitionReader' is added by us, so it's safe to ignore it when changing - // number of reducers. - val leafNodes = plan.collectLeaves().filter(!_.isInstanceOf[SkewedPartitionReaderExec]) - if (!leafNodes.forall(_.isInstanceOf[QueryStageExec])) { + if (!plan.collectLeaves().forall(_.isInstanceOf[QueryStageExec])) { // If not all leaf nodes are query stages, it's not safe to reduce the number of // shuffle partitions, because we may break the assumption that all children of a spark plan // have same number of output partitions. return plan } - def collectShuffles(plan: SparkPlan): Seq[SparkPlan] = plan match { + def collectShuffleStages(plan: SparkPlan): Seq[ShuffleQueryStageExec] = plan match { case _: LocalShuffleReaderExec => Nil - case p: PartialShuffleReaderExec => Seq(p) + case _: SkewJoinShuffleReaderExec => Nil case stage: ShuffleQueryStageExec => Seq(stage) - case _ => plan.children.flatMap(collectShuffles) + case _ => plan.children.flatMap(collectShuffleStages) } - val shuffles = collectShuffles(plan) - val shuffleStages = shuffles.map { - case PartialShuffleReaderExec(s: ShuffleQueryStageExec, _) => s - case s: ShuffleQueryStageExec => s - } + val shuffleStages = collectShuffleStages(plan) // ShuffleExchanges introduced by repartition do not support changing the number of partitions. // We change the number of partitions in the stage only if all the ShuffleExchanges support it. if (!shuffleStages.forall(_.shuffle.canChangeNumPartitions)) { @@ -94,110 +69,27 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { // partition) and a result of a SortMergeJoin (multiple partitions). val distinctNumPreShufflePartitions = validMetrics.map(stats => stats.bytesByPartitionId.length).distinct - val distinctExcludedPartitions = shuffles.map { - case PartialShuffleReaderExec(_, excludedPartitions) => excludedPartitions - case _: ShuffleQueryStageExec => Set.empty[Int] - }.distinct - if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1 - && distinctExcludedPartitions.length == 1) { - val excludedPartitions = distinctExcludedPartitions.head - val partitionIndices = estimatePartitionStartAndEndIndices( - validMetrics.toArray, excludedPartitions) + if (validMetrics.nonEmpty && distinctNumPreShufflePartitions.length == 1) { + val partitionStartIndices = ShufflePartitionsCoalescer.coalescePartitions( + validMetrics.toArray, + firstPartitionIndex = 0, + lastPartitionIndex = distinctNumPreShufflePartitions.head, + advisoryTargetSize = conf.targetPostShuffleInputSize, + minNumPartitions = conf.minNumPostShufflePartitions) // This transformation adds new nodes, so we must use `transformUp` here. - // Even for shuffle exchange whose input RDD has 0 partition, we should still update its - // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same - // number of output partitions. - val visitedStages = HashSet.empty[Int] - plan.transformDown { - // Replace `PartialShuffleReaderExec` with `CoalescedShuffleReaderExec`, which keeps the - // "excludedPartition" requirement and also merges some partitions. - case PartialShuffleReaderExec(stage: ShuffleQueryStageExec, _) => - visitedStages.add(stage.id) - CoalescedShuffleReaderExec(stage, partitionIndices) - - // We are doing `transformDown`, so the `ShuffleQueryStageExec` may already be optimized - // and wrapped by `CoalescedShuffleReaderExec`. - case stage: ShuffleQueryStageExec if !visitedStages.contains(stage.id) => - visitedStages.add(stage.id) - CoalescedShuffleReaderExec(stage, partitionIndices) + val stageIds = shuffleStages.map(_.id).toSet + plan.transformUp { + // even for shuffle exchange whose input RDD has 0 partition, we should still update its + // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same + // number of output partitions. + case stage: ShuffleQueryStageExec if stageIds.contains(stage.id) => + CoalescedShuffleReaderExec(stage, partitionStartIndices) } } else { plan } } } - - /** - * Estimates partition start and end indices for post-shuffle partitions based on - * mapOutputStatistics provided by all pre-shuffle stages and skip the omittedPartitions - * already handled in skewed partition optimization. - */ - // visible for testing. - private[sql] def estimatePartitionStartAndEndIndices( - mapOutputStatistics: Array[MapOutputStatistics], - excludedPartitions: Set[Int] = Set.empty): Array[(Int, Int)] = { - val minNumPostShufflePartitions = conf.minNumPostShufflePartitions - excludedPartitions.size - val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize - // If minNumPostShufflePartitions is defined, it is possible that we need to use a - // value less than advisoryTargetPostShuffleInputSize as the target input size of - // a post shuffle task. - val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum - // The max at here is to make sure that when we have an empty table, we - // only have a single post-shuffle partition. - // There is no particular reason that we pick 16. We just need a number to - // prevent maxPostShuffleInputSize from being set to 0. - val maxPostShuffleInputSize = math.max( - math.ceil(totalPostShuffleInputSize / minNumPostShufflePartitions.toDouble).toLong, 16) - val targetPostShuffleInputSize = - math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize) - - logInfo( - s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " + - s"targetPostShuffleInputSize $targetPostShuffleInputSize.") - - // Make sure we do get the same number of pre-shuffle partitions for those stages. - val distinctNumPreShufflePartitions = - mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct - // The reason that we are expecting a single value of the number of pre-shuffle partitions - // is that when we add Exchanges, we set the number of pre-shuffle partitions - // (i.e. map output partitions) using a static setting, which is the value of - // spark.sql.shuffle.partitions. Even if two input RDDs are having different - // number of partitions, they will have the same number of pre-shuffle partitions - // (i.e. map output partitions). - assert( - distinctNumPreShufflePartitions.length == 1, - "There should be only one distinct value of the number pre-shuffle partitions " + - "among registered Exchange operator.") - - val partitionStartIndices = ArrayBuffer[Int]() - val partitionEndIndices = ArrayBuffer[Int]() - val numPartitions = distinctNumPreShufflePartitions.head - val includedPartitions = (0 until numPartitions).filter(!excludedPartitions.contains(_)) - val firstStartIndex = includedPartitions(0) - partitionStartIndices += firstStartIndex - var postShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId(firstStartIndex)).sum - var i = firstStartIndex - includedPartitions.drop(1).foreach { nextPartitionIndex => - val nextShuffleInputSize = - mapOutputStatistics.map(_.bytesByPartitionId(nextPartitionIndex)).sum - // If nextPartitionIndices is skewed and omitted, or including - // the nextShuffleInputSize would exceed the target partition size, - // then start a new partition. - if (nextPartitionIndex != i + 1 || - (postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize)) { - partitionEndIndices += i + 1 - partitionStartIndices += nextPartitionIndex - // reset postShuffleInputSize. - postShuffleInputSize = nextShuffleInputSize - i = nextPartitionIndex - } else { - postShuffleInputSize += nextShuffleInputSize - i += 1 - } - } - partitionEndIndices += i + 1 - partitionStartIndices.zip(partitionEndIndices).toArray - } } /** @@ -206,15 +98,16 @@ case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { * * @param child It's usually `ShuffleQueryStageExec`, but can be the shuffle exchange node during * canonicalization. + * @param partitionStartIndices The start partition indices for the coalesced partitions. */ case class CoalescedShuffleReaderExec( child: SparkPlan, - partitionIndices: Array[(Int, Int)]) extends UnaryExecNode { + partitionStartIndices: Array[Int]) extends UnaryExecNode { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = { - UnknownPartitioning(partitionIndices.length) + UnknownPartitioning(partitionStartIndices.length) } private var cachedShuffleRDD: ShuffledRowRDD = null @@ -223,7 +116,7 @@ case class CoalescedShuffleReaderExec( if (cachedShuffleRDD == null) { cachedShuffleRDD = child match { case stage: ShuffleQueryStageExec => - stage.shuffle.createShuffledRDD(Some(partitionIndices)) + stage.shuffle.createShuffledRDD(Some(partitionStartIndices)) case _ => throw new IllegalStateException("operating on canonicalization plan") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsCoalescer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsCoalescer.scala new file mode 100644 index 0000000000000..18f0585524aa2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ShufflePartitionsCoalescer.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.adaptive + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.MapOutputStatistics +import org.apache.spark.internal.Logging + +object ShufflePartitionsCoalescer extends Logging { + + /** + * Coalesce the same range of partitions (`firstPartitionIndex`` to `lastPartitionIndex`, the + * start is inclusive and the end is exclusive) from multiple shuffles. This method assumes that + * all the shuffles have the same number of partitions, and the partitions of same index will be + * read together by one task. + * + * The strategy used to determine the number of coalesced partitions is described as follows. + * To determine the number of coalesced partitions, we have a target size for a coalesced + * partition. Once we have size statistics of all shuffle partitions, we will do + * a pass of those statistics and pack shuffle partitions with continuous indices to a single + * coalesced partition until adding another shuffle partition would cause the size of a + * coalesced partition to be greater than the target size. + * + * For example, we have two shuffles with the following partition size statistics: + * - shuffle 1 (5 partitions): [100 MiB, 20 MiB, 100 MiB, 10MiB, 30 MiB] + * - shuffle 2 (5 partitions): [10 MiB, 10 MiB, 70 MiB, 5 MiB, 5 MiB] + * Assuming the target size is 128 MiB, we will have 4 coalesced partitions, which are: + * - coalesced partition 0: shuffle partition 0 (size 110 MiB) + * - coalesced partition 1: shuffle partition 1 (size 30 MiB) + * - coalesced partition 2: shuffle partition 2 (size 170 MiB) + * - coalesced partition 3: shuffle partition 3 and 4 (size 50 MiB) + * + * @return An array of partition indices which represents the coalesced partitions. For example, + * [0, 2, 3] means 3 coalesced partitions: [0, 2), [2, 3), [3, lastPartitionIndex] + */ + def coalescePartitions( + mapOutputStatistics: Array[MapOutputStatistics], + firstPartitionIndex: Int, + lastPartitionIndex: Int, + advisoryTargetSize: Long, + minNumPartitions: Int = 1): Array[Int] = { + // If `minNumPartitions` is very large, it is possible that we need to use a value less than + // `advisoryTargetSize` as the target size of a coalesced task. + val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum + // The max at here is to make sure that when we have an empty table, we only have a single + // coalesced partition. + // There is no particular reason that we pick 16. We just need a number to prevent + // `maxTargetSize` from being set to 0. + val maxTargetSize = math.max( + math.ceil(totalPostShuffleInputSize / minNumPartitions.toDouble).toLong, 16) + val targetSize = math.min(maxTargetSize, advisoryTargetSize) + + logInfo(s"advisory target size: $advisoryTargetSize, actual target size $targetSize.") + + // Make sure these shuffles have the same number of partitions. + val distinctNumShufflePartitions = + mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct + // The reason that we are expecting a single value of the number of shuffle partitions + // is that when we add Exchanges, we set the number of shuffle partitions + // (i.e. map output partitions) using a static setting, which is the value of + // `spark.sql.shuffle.partitions`. Even if two input RDDs are having different + // number of partitions, they will have the same number of shuffle partitions + // (i.e. map output partitions). + assert( + distinctNumShufflePartitions.length == 1, + "There should be only one distinct value of the number of shuffle partitions " + + "among registered Exchange operators.") + + val splitPoints = ArrayBuffer[Int]() + splitPoints += firstPartitionIndex + var coalescedSize = 0L + var i = firstPartitionIndex + while (i < lastPartitionIndex) { + // We calculate the total size of i-th shuffle partitions from all shuffles. + var totalSizeOfCurrentPartition = 0L + var j = 0 + while (j < mapOutputStatistics.length) { + totalSizeOfCurrentPartition += mapOutputStatistics(j).bytesByPartitionId(i) + j += 1 + } + + // If including the `totalSizeOfCurrentPartition` would exceed the target size, then start a + // new coalesced partition. + if (i > firstPartitionIndex && coalescedSize + totalSizeOfCurrentPartition > targetSize) { + splitPoints += i + // reset postShuffleInputSize. + coalescedSize = totalSizeOfCurrentPartition + } else { + coalescedSize += totalSizeOfCurrentPartition + } + i += 1 + } + + splitPoints.toArray + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala deleted file mode 100644 index 52f793b24aa17..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/SkewedShuffledRowRDD.scala +++ /dev/null @@ -1,78 +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.sql.execution.adaptive - -import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} - -/** - * The [[Partition]] used by [[SkewedShuffledRowRDD]]. - */ -class SkewedShuffledRowRDDPartition(override val index: Int) extends Partition - -/** - * This is a specialized version of [[org.apache.spark.sql.execution.ShuffledRowRDD]]. This is used - * in Spark SQL adaptive execution to solve data skew issues. This RDD includes rearranged - * partitions from mappers. - * - * This RDD takes a [[ShuffleDependency]] (`dependency`), a partitionIndex - * and the range of startMapIndex to endMapIndex. - */ -class SkewedShuffledRowRDD( - var dependency: ShuffleDependency[Int, InternalRow, InternalRow], - partitionIndex: Int, - startMapIndex: Int, - endMapIndex: Int, - metrics: Map[String, SQLMetric]) - extends RDD[InternalRow](dependency.rdd.context, Nil) { - - override def getDependencies: Seq[Dependency[_]] = List(dependency) - - override def getPartitions: Array[Partition] = { - Array(new SkewedShuffledRowRDDPartition(0)) - } - - override def getPreferredLocations(partition: Partition): Seq[String] = { - val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - tracker.getMapLocation(dependency, startMapIndex, endMapIndex) - } - - override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { - val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() - // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, - // as well as the `tempMetrics` for basic shuffle metrics. - val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) - - val reader = SparkEnv.get.shuffleManager.getReaderForRange( - dependency.shuffleHandle, - startMapIndex, - endMapIndex, - partitionIndex, - partitionIndex + 1, - context, - sqlMetricsReporter) - reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) - } - - override def clearDependencies() { - super.clearDependencies() - dependency = null - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index ffcd6c7783354..4b08da043b83e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -30,11 +30,11 @@ import org.apache.spark.shuffle.{ShuffleWriteMetricsReporter, ShuffleWriteProces import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Divide, Literal, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.{LocalShuffledRowRDD, SkewedShuffledRowRDD} +import org.apache.spark.sql.execution.adaptive.LocalShuffledRowRDD import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -49,11 +49,9 @@ case class ShuffleExchangeExec( child: SparkPlan, canChangeNumPartitions: Boolean = true) extends Exchange { - // NOTE: coordinator can be null after serialization/deserialization, - // e.g. it can be null on the Executor side private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) - private lazy val readMetrics = + private[sql] lazy val readMetrics = SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size") @@ -90,9 +88,8 @@ case class ShuffleExchangeExec( writeMetrics) } - def createShuffledRDD( - partitionRanges: Option[Array[(Int, Int)]]): ShuffledRowRDD = { - new ShuffledRowRDD(shuffleDependency, readMetrics, partitionRanges) + def createShuffledRDD(partitionStartIndices: Option[Array[Int]]): ShuffledRowRDD = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) } def createLocalShuffleRDD( @@ -100,14 +97,6 @@ case class ShuffleExchangeExec( new LocalShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndicesPerMapper) } - def createSkewedShuffleRDD( - partitionIndex: Int, - startMapIndex: Int, - endMapIndex: Int): SkewedShuffledRowRDD = { - new SkewedShuffledRowRDD(shuffleDependency, - partitionIndex, startMapIndex, endMapIndex, readMetrics) - } - /** * Caches the created ShuffleRowRDD so we can reuse that. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 6384aed6a78e0..62eea611556ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.{PartialShuffleReaderExec, SkewedPartitionReaderExec} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.util.collection.BitSet @@ -42,11 +41,17 @@ case class SortMergeJoinExec( condition: Option[Expression], left: SparkPlan, right: SparkPlan, - isPartial: Boolean = false) extends BinaryExecNode with CodegenSupport { + isSkewJoin: Boolean = false) extends BinaryExecNode with CodegenSupport { override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + override def nodeName: String = { + if (isSkewJoin) super.nodeName + "(skew=true)" else super.nodeName + } + + override def stringArgs: Iterator[Any] = super.stringArgs.toSeq.dropRight(1).iterator + override def simpleStringWithNodeId(): String = { val opId = ExplainUtils.getOpId(this) s"$nodeName $joinType ($opId)".trim @@ -98,7 +103,9 @@ case class SortMergeJoinExec( } override def requiredChildDistribution: Seq[Distribution] = { - if (isPartial) { + if (isSkewJoin) { + // We re-arrange the shuffle partitions to deal with skew join, and the new children + // partitioning doesn't satisfy `HashClusteredDistribution`. UnspecifiedDistribution :: UnspecifiedDistribution :: Nil } else { HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 04b4d4f29f850..5565a0dd01840 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.execution import org.scalatest.BeforeAndAfterAll -import org.apache.spark.{MapOutputStatistics, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.adaptive._ -import org.apache.spark.sql.execution.adaptive.{CoalescedShuffleReaderExec, ReduceNumShufflePartitions} +import org.apache.spark.sql.execution.adaptive.CoalescedShuffleReaderExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -52,212 +52,6 @@ class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterA } } - private def checkEstimation( - rule: ReduceNumShufflePartitions, - bytesByPartitionIdArray: Array[Array[Long]], - expectedPartitionStartIndices: Array[Int]): Unit = { - val mapOutputStatistics = bytesByPartitionIdArray.zipWithIndex.map { - case (bytesByPartitionId, index) => - new MapOutputStatistics(index, bytesByPartitionId) - } - val estimatedPartitionStartIndices = - rule.estimatePartitionStartAndEndIndices(mapOutputStatistics).map(_._1) - assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) - } - - private def createReduceNumShufflePartitionsRule( - advisoryTargetPostShuffleInputSize: Long, - minNumPostShufflePartitions: Int = 1): ReduceNumShufflePartitions = { - val conf = new SQLConf().copy( - SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE -> advisoryTargetPostShuffleInputSize, - SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS -> minNumPostShufflePartitions) - ReduceNumShufflePartitions(conf) - } - - test("test estimatePartitionStartIndices - 1 Exchange") { - val rule = createReduceNumShufflePartitionsRule(100L) - - { - // All bytes per partition are 0. - val bytesByPartitionId = Array[Long](0, 0, 0, 0, 0) - val expectedPartitionStartIndices = Array[Int](0) - checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) - } - - { - // Some bytes per partition are 0 and total size is less than the target size. - // 1 post-shuffle partition is needed. - val bytesByPartitionId = Array[Long](10, 0, 20, 0, 0) - val expectedPartitionStartIndices = Array[Int](0) - checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) - } - - { - // 2 post-shuffle partitions are needed. - val bytesByPartitionId = Array[Long](10, 0, 90, 20, 0) - val expectedPartitionStartIndices = Array[Int](0, 3) - checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) - } - - { - // There are a few large pre-shuffle partitions. - val bytesByPartitionId = Array[Long](110, 10, 100, 110, 0) - val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) - checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) - } - - { - // All pre-shuffle partitions are larger than the targeted size. - val bytesByPartitionId = Array[Long](100, 110, 100, 110, 110) - val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) - checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) - } - - { - // The last pre-shuffle partition is in a single post-shuffle partition. - val bytesByPartitionId = Array[Long](30, 30, 0, 40, 110) - val expectedPartitionStartIndices = Array[Int](0, 4) - checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) - } - } - - test("test estimatePartitionStartIndices - 2 Exchanges") { - val rule = createReduceNumShufflePartitionsRule(100L) - - { - // If there are multiple values of the number of pre-shuffle partitions, - // we should see an assertion error. - val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) - val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0, 0) - val mapOutputStatistics = - Array( - new MapOutputStatistics(0, bytesByPartitionId1), - new MapOutputStatistics(1, bytesByPartitionId2)) - intercept[AssertionError](rule.estimatePartitionStartAndEndIndices( - mapOutputStatistics)) - } - - { - // All bytes per partition are 0. - val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) - val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) - val expectedPartitionStartIndices = Array[Int](0) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - - { - // Some bytes per partition are 0. - // 1 post-shuffle partition is needed. - val bytesByPartitionId1 = Array[Long](0, 10, 0, 20, 0) - val bytesByPartitionId2 = Array[Long](30, 0, 20, 0, 20) - val expectedPartitionStartIndices = Array[Int](0) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - - { - // 2 post-shuffle partition are needed. - val bytesByPartitionId1 = Array[Long](0, 10, 0, 20, 0) - val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) - val expectedPartitionStartIndices = Array[Int](0, 2, 4) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - - { - // 4 post-shuffle partition are needed. - val bytesByPartitionId1 = Array[Long](0, 99, 0, 20, 0) - val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) - val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - - { - // 2 post-shuffle partition are needed. - val bytesByPartitionId1 = Array[Long](0, 100, 0, 30, 0) - val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) - val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - - { - // There are a few large pre-shuffle partitions. - val bytesByPartitionId1 = Array[Long](0, 100, 40, 30, 0) - val bytesByPartitionId2 = Array[Long](30, 0, 60, 0, 110) - val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - - { - // All pairs of pre-shuffle partitions are larger than the targeted size. - val bytesByPartitionId1 = Array[Long](100, 100, 40, 30, 0) - val bytesByPartitionId2 = Array[Long](30, 0, 60, 70, 110) - val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - } - - test("test estimatePartitionStartIndices and enforce minimal number of reducers") { - val rule = createReduceNumShufflePartitionsRule(100L, 2) - - { - // The minimal number of post-shuffle partitions is not enforced because - // the size of data is 0. - val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) - val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) - val expectedPartitionStartIndices = Array[Int](0) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - - { - // The minimal number of post-shuffle partitions is enforced. - val bytesByPartitionId1 = Array[Long](10, 5, 5, 0, 20) - val bytesByPartitionId2 = Array[Long](5, 10, 0, 10, 5) - val expectedPartitionStartIndices = Array[Int](0, 3) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - - { - // The number of post-shuffle partitions is determined by the coordinator. - val bytesByPartitionId1 = Array[Long](10, 50, 20, 80, 20) - val bytesByPartitionId2 = Array[Long](40, 10, 0, 10, 30) - val expectedPartitionStartIndices = Array[Int](0, 1, 3, 4) - checkEstimation( - rule, - Array(bytesByPartitionId1, bytesByPartitionId2), - expectedPartitionStartIndices) - } - } - - /////////////////////////////////////////////////////////////////////////// - // Query tests - /////////////////////////////////////////////////////////////////////////// - val numInputPartitions: Int = 10 def withSparkSession( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsCoalescerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsCoalescerSuite.scala new file mode 100644 index 0000000000000..fcfde83b2ffd5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ShufflePartitionsCoalescerSuite.scala @@ -0,0 +1,220 @@ +/* + * 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 + +import org.apache.spark.{MapOutputStatistics, SparkFunSuite} +import org.apache.spark.sql.execution.adaptive.ShufflePartitionsCoalescer + +class ShufflePartitionsCoalescerSuite extends SparkFunSuite { + + private def checkEstimation( + bytesByPartitionIdArray: Array[Array[Long]], + expectedPartitionStartIndices: Array[Int], + targetSize: Long, + minNumPartitions: Int = 1): Unit = { + val mapOutputStatistics = bytesByPartitionIdArray.zipWithIndex.map { + case (bytesByPartitionId, index) => + new MapOutputStatistics(index, bytesByPartitionId) + } + val estimatedPartitionStartIndices = ShufflePartitionsCoalescer.coalescePartitions( + mapOutputStatistics, + 0, + bytesByPartitionIdArray.head.length, + targetSize, + minNumPartitions) + assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) + } + + test("1 shuffle") { + val targetSize = 100 + + { + // All bytes per partition are 0. + val bytesByPartitionId = Array[Long](0, 0, 0, 0, 0) + val expectedPartitionStartIndices = Array[Int](0) + checkEstimation(Array(bytesByPartitionId), expectedPartitionStartIndices, targetSize) + } + + { + // Some bytes per partition are 0 and total size is less than the target size. + // 1 coalesced partition is expected. + val bytesByPartitionId = Array[Long](10, 0, 20, 0, 0) + val expectedPartitionStartIndices = Array[Int](0) + checkEstimation(Array(bytesByPartitionId), expectedPartitionStartIndices, targetSize) + } + + { + // 2 coalesced partitions are expected. + val bytesByPartitionId = Array[Long](10, 0, 90, 20, 0) + val expectedPartitionStartIndices = Array[Int](0, 3) + checkEstimation(Array(bytesByPartitionId), expectedPartitionStartIndices, targetSize) + } + + { + // There are a few large shuffle partitions. + val bytesByPartitionId = Array[Long](110, 10, 100, 110, 0) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) + checkEstimation(Array(bytesByPartitionId), expectedPartitionStartIndices, targetSize) + } + + { + // All shuffle partitions are larger than the targeted size. + val bytesByPartitionId = Array[Long](100, 110, 100, 110, 110) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) + checkEstimation(Array(bytesByPartitionId), expectedPartitionStartIndices, targetSize) + } + + { + // The last shuffle partition is in a single coalesced partition. + val bytesByPartitionId = Array[Long](30, 30, 0, 40, 110) + val expectedPartitionStartIndices = Array[Int](0, 4) + checkEstimation(Array(bytesByPartitionId), expectedPartitionStartIndices, targetSize) + } + } + + test("2 shuffles") { + val targetSize = 100 + + { + // If there are multiple values of the number of shuffle partitions, + // we should see an assertion error. + val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) + val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0, 0) + intercept[AssertionError] { + checkEstimation(Array(bytesByPartitionId1, bytesByPartitionId2), Array.empty, targetSize) + } + } + + { + // All bytes per partition are 0. + val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) + val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) + val expectedPartitionStartIndices = Array[Int](0) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize) + } + + { + // Some bytes per partition are 0. + // 1 coalesced partition is expected. + val bytesByPartitionId1 = Array[Long](0, 10, 0, 20, 0) + val bytesByPartitionId2 = Array[Long](30, 0, 20, 0, 20) + val expectedPartitionStartIndices = Array[Int](0) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize) + } + + { + // 2 coalesced partition are expected. + val bytesByPartitionId1 = Array[Long](0, 10, 0, 20, 0) + val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) + val expectedPartitionStartIndices = Array[Int](0, 2, 4) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize) + } + + { + // 4 coalesced partition are expected. + val bytesByPartitionId1 = Array[Long](0, 99, 0, 20, 0) + val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize) + } + + { + // 2 coalesced partition are needed. + val bytesByPartitionId1 = Array[Long](0, 100, 0, 30, 0) + val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize) + } + + { + // There are a few large shuffle partitions. + val bytesByPartitionId1 = Array[Long](0, 100, 40, 30, 0) + val bytesByPartitionId2 = Array[Long](30, 0, 60, 0, 110) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize) + } + + { + // All pairs of shuffle partitions are larger than the targeted size. + val bytesByPartitionId1 = Array[Long](100, 100, 40, 30, 0) + val bytesByPartitionId2 = Array[Long](30, 0, 60, 70, 110) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize) + } + } + + test("enforce minimal number of coalesced partitions") { + val targetSize = 100 + val minNumPartitions = 2 + + { + // The minimal number of coalesced partitions is not enforced because + // the size of data is 0. + val bytesByPartitionId1 = Array[Long](0, 0, 0, 0, 0) + val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) + val expectedPartitionStartIndices = Array[Int](0) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize, minNumPartitions) + } + + { + // The minimal number of coalesced partitions is enforced. + val bytesByPartitionId1 = Array[Long](10, 5, 5, 0, 20) + val bytesByPartitionId2 = Array[Long](5, 10, 0, 10, 5) + val expectedPartitionStartIndices = Array[Int](0, 3) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize, minNumPartitions) + } + + { + // The number of coalesced partitions is determined by the algorithm. + val bytesByPartitionId1 = Array[Long](10, 50, 20, 80, 20) + val bytesByPartitionId2 = Array[Long](40, 10, 0, 10, 30) + val expectedPartitionStartIndices = Array[Int](0, 1, 3, 4) + checkEstimation( + Array(bytesByPartitionId1, bytesByPartitionId2), + expectedPartitionStartIndices, + targetSize, minNumPartitions) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index a2071903bea7e..4edb35ea30fde 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -23,7 +23,7 @@ import java.net.URI import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} -import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, Exchange, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BuildRight, SortMergeJoinExec} import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate import org.apache.spark.sql.internal.SQLConf @@ -594,160 +594,84 @@ class AdaptiveQueryExecSuite .range(0, 1000, 1, 10) .selectExpr("id % 1 as key2", "id as value2") .createOrReplaceTempView("skewData2") - val (innerPlan, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult( "SELECT key1 FROM skewData1 join skewData2 ON key1 = key2 group by key1") - val innerSmj = findTopLevelSortMergeJoin(innerPlan) - assert(innerSmj.size == 1) // Additional shuffle introduced, so disable the "OptimizeSkewedJoin" optimization - val innerSmjAfter = findTopLevelSortMergeJoin(innerAdaptivePlan) - assert(innerSmjAfter.size == 1) + val innerSmj = findTopLevelSortMergeJoin(innerAdaptivePlan) + assert(innerSmj.size == 1 && !innerSmj.head.isSkewJoin) } } } + // TODO: we need a way to customize data distribution after shuffle, to improve test coverage + // of this case. test("SPARK-29544: adaptive skew join with different join types") { - Seq("false", "true").foreach { reducePostShufflePartitionsEnabled => - withSQLConf( - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", - SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> reducePostShufflePartitionsEnabled, - SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "700") { - withTempView("skewData1", "skewData2") { - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 2 as key1", "id as value1") - .createOrReplaceTempView("skewData1") - spark - .range(0, 1000, 1, 10) - .selectExpr("id % 1 as key2", "id as value2") - .createOrReplaceTempView("skewData2") - // skewed inner join optimization - val (innerPlan, innerAdaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 join skewData2 ON key1 = key2") - val innerSmj = findTopLevelSortMergeJoin(innerPlan) - assert(innerSmj.size == 1) - // left stats: [3496, 0, 0, 0, 4014] - // right stats:[6292, 0, 0, 0, 0] - // the partition 0 in both left and right side are all skewed. - // And divide into 5 splits both in left and right (the max splits number). - // So there are 5 x 5 smjs for partition 0. - // Partition 4 in left side is skewed and is divided into 5 splits. - // The right side of partition 4 is not skewed. - // So there are 5 smjs for partition 4. - // So total (25 + 5 + 1) smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val innerSmjAfter = findTopLevelSortMergeJoin(innerAdaptivePlan) - assert(innerSmjAfter.size == 31) - - // skewed left outer join optimization - val (leftPlan, leftAdaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") - val leftSmj = findTopLevelSortMergeJoin(leftPlan) - assert(leftSmj.size == 1) - // left stats: [3496, 0, 0, 0, 4014] - // right stats:[6292, 0, 0, 0, 0] - // The partition 0 in both left and right are all skewed. - // The partition 4 in left side is skewed. - // But for left outer join, we don't split the right partition even skewed. - // So the partition 0 in left side is divided into 5 splits(the max split number). - // the partition 4 in left side is divided into 5 splits(the max split number). - // So total (5 + 5 + 1) smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val leftSmjAfter = findTopLevelSortMergeJoin(leftAdaptivePlan) - assert(leftSmjAfter.size == 11) - - // skewed right outer join optimization - val (rightPlan, rightAdaptivePlan) = runAdaptiveAndVerifyResult( - "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") - val rightSmj = findTopLevelSortMergeJoin(rightPlan) - assert(rightSmj.size == 1) - // left stats: [3496, 0, 0, 0, 4014] - // right stats:[6292, 0, 0, 0, 0] - // The partition 0 in both left and right side are all skewed. - // And the partition 4 in left side is skewed. - // But for right outer join, we don't split the left partition even skewed. - // And divide right side into 5 splits(the max split number) - // So total 6 smjs. - // Union - // +- SortMergeJoin - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- CoalescedShuffleReader - // +- ShuffleQueryStage - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // . - // . - // . - // +- SortMergeJoin - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - // +- Sort - // +- SkewedShuffleReader - // +- ShuffleQueryStage - - val rightSmjAfter = findTopLevelSortMergeJoin(rightAdaptivePlan) - assert(rightSmjAfter.size == 6) + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_SKEWED_PARTITION_SIZE_THRESHOLD.key -> "100", + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "700") { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 2 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + + def checkSkewJoin(joins: Seq[SortMergeJoinExec], expectedNumPartitions: Int): Unit = { + assert(joins.size == 1 && joins.head.isSkewJoin) + assert(joins.head.left.collect { + case r: SkewJoinShuffleReaderExec => r + }.head.partitionSpecs.length == expectedNumPartitions) + assert(joins.head.right.collect { + case r: SkewJoinShuffleReaderExec => r + }.head.partitionSpecs.length == expectedNumPartitions) } + + // skewed inner join optimization + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 join skewData2 ON key1 = key2") + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // Partition 0: both left and right sides are skewed, and divide into 5 splits, so + // 5 x 5 sub-partitions. + // Partition 1, 2, 3: not skewed, and coalesced into 1 partition. + // Partition 4: only left side is skewed, and divide into 5 splits, so + // 5 sub-partitions. + // So total (25 + 1 + 5) partitions. + val innerSmj = findTopLevelSortMergeJoin(innerAdaptivePlan) + checkSkewJoin(innerSmj, 25 + 1 + 5) + + // skewed left outer join optimization + val (_, leftAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 left outer join skewData2 ON key1 = key2") + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // Partition 0: both left and right sides are skewed, but left join can't split right side, + // so only left side is divided into 5 splits, and thus 5 sub-partitions. + // Partition 1, 2, 3: not skewed, and coalesced into 1 partition. + // Partition 4: only left side is skewed, and divide into 5 splits, so + // 5 sub-partitions. + // So total (5 + 1 + 5) partitions. + val leftSmj = findTopLevelSortMergeJoin(leftAdaptivePlan) + checkSkewJoin(leftSmj, 5 + 1 + 5) + + // skewed right outer join optimization + val (_, rightAdaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM skewData1 right outer join skewData2 ON key1 = key2") + // left stats: [3496, 0, 0, 0, 4014] + // right stats:[6292, 0, 0, 0, 0] + // Partition 0: both left and right sides are skewed, but right join can't split left side, + // so only right side is divided into 5 splits, and thus 5 sub-partitions. + // Partition 1, 2, 3: not skewed, and coalesced into 1 partition. + // Partition 4: only left side is skewed, but right join can't split left side, so just + // 1 partition. + // So total (5 + 1 + 1) partitions. + val rightSmj = findTopLevelSortMergeJoin(rightAdaptivePlan) + checkSkewJoin(rightSmj, 5 + 1 + 1) } } } @@ -805,3 +729,4 @@ class AdaptiveQueryExecSuite s" enabled but is not supported for"))) } } + From 859699135cb63b57f5d844e762070065cedb4408 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 13 Feb 2020 11:17:27 -0800 Subject: [PATCH 1369/1387] [SPARK-30807][K8S][TESTS] Support Java 11 in K8S integration tests ### What changes were proposed in this pull request? This PR aims to support JDK11 test in K8S integration tests. - This is an update in testing framework instead of individual tests. - This will enable JDK11 runtime test when you didn't installed JDK11 on your local system. ### Why are the changes needed? Apache Spark 3.0.0 adds JDK11 support, but K8s integration tests use JDK8 until now. ### Does this PR introduce any user-facing change? No. This is a dev-only test-related PR. ### How was this patch tested? This is irrelevant to Jenkins UT, but Jenkins K8S IT (JDK8) should pass. - https://github.com/apache/spark/pull/27559#issuecomment-585903489 (JDK8 Passed) And, manually do the following for JDK11 test. ``` $ NO_MANUAL=1 ./dev/make-distribution.sh --r --pip --tgz -Phadoop-3.2 -Pkubernetes $ resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh --java-image-tag 11-jre-slim --spark-tgz $PWD/spark-*.tgz ``` ``` $ docker run -it --rm kubespark/spark:1318DD8A-2B15-4A00-BC69-D0E90CED235B /usr/local/openjdk-11/bin/java --version | tail -n1 OpenJDK 64-Bit Server VM 18.9 (build 11.0.6+10, mixed mode) ``` Closes #27559 from dongjoon-hyun/SPARK-30807. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../docker/src/main/dockerfiles/spark/Dockerfile | 3 ++- .../kubernetes/integration-tests/README.md | 15 +++++++++++++-- .../dev/dev-run-integration-tests.sh | 10 ++++++++++ .../kubernetes/integration-tests/pom.xml | 4 ++++ .../scripts/setup-integration-test-env.sh | 14 +++++++++++--- 5 files changed, 40 insertions(+), 6 deletions(-) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index a1fc63789bc61..6ed37fc637b31 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -14,8 +14,9 @@ # See the License for the specific language governing permissions and # limitations under the License. # +ARG java_image_tag=8-jre-slim -FROM openjdk:8-jre-slim +FROM openjdk:${java_image_tag} ARG spark_uid=185 diff --git a/resource-managers/kubernetes/integration-tests/README.md b/resource-managers/kubernetes/integration-tests/README.md index d7ad35a175a61..18b91916208d6 100644 --- a/resource-managers/kubernetes/integration-tests/README.md +++ b/resource-managers/kubernetes/integration-tests/README.md @@ -6,13 +6,17 @@ title: Spark on Kubernetes Integration Tests # Running the Kubernetes Integration Tests Note that the integration test framework is currently being heavily revised and -is subject to change. Note that currently the integration tests only run with Java 8. +is subject to change. The simplest way to run the integration tests is to install and run Minikube, then run the following from this directory: ./dev/dev-run-integration-tests.sh +To run tests with Java 11 instead of Java 8, use `--java-image-tag` to specify the base image. + + ./dev/dev-run-integration-tests.sh --java-image-tag 11-jre-slim + The minimum tested version of Minikube is 0.23.0. The kube-dns addon must be enabled. Minikube should run with a minimum of 4 CPUs and 6G of memory: @@ -183,7 +187,14 @@ to the wrapper scripts and using the wrapper scripts will simply set these appro A specific image tag to use, when set assumes images with those tags are already built and available in the specified image repository. When set to N/A (the default) fresh images will be built. - N/A + N/A + + + spark.kubernetes.test.javaImageTag + + A specific OpenJDK base image tag to use, when set uses it instead of 8-jre-slim. + + 8-jre-slim spark.kubernetes.test.imageTagFile diff --git a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh index 1f0a8035cea7b..76d6e1c1e8499 100755 --- a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh +++ b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh @@ -23,6 +23,7 @@ DEPLOY_MODE="minikube" IMAGE_REPO="docker.io/kubespark" SPARK_TGZ="N/A" IMAGE_TAG="N/A" +JAVA_IMAGE_TAG= BASE_IMAGE_NAME= JVM_IMAGE_NAME= PYTHON_IMAGE_NAME= @@ -52,6 +53,10 @@ while (( "$#" )); do IMAGE_TAG="$2" shift ;; + --java-image-tag) + JAVA_IMAGE_TAG="$2" + shift + ;; --deploy-mode) DEPLOY_MODE="$2" shift @@ -120,6 +125,11 @@ properties=( -Dtest.include.tags=$INCLUDE_TAGS ) +if [ -n "$JAVA_IMAGE_TAG" ]; +then + properties=( ${properties[@]} -Dspark.kubernetes.test.javaImageTag=$JAVA_IMAGE_TAG ) +fi + if [ -n $NAMESPACE ]; then properties=( ${properties[@]} -Dspark.kubernetes.test.namespace=$NAMESPACE ) diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 8e1043f77db6d..369dfd491826c 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -39,6 +39,7 @@ ${project.build.directory}/spark-dist-unpacked N/A + 8-jre-slim ${project.build.directory}/imageTag.txt minikube docker.io/kubespark @@ -109,6 +110,9 @@ --image-tag ${spark.kubernetes.test.imageTag} + --java-image-tag + ${spark.kubernetes.test.javaImageTag} + --image-tag-output-file ${spark.kubernetes.test.imageTagFile} diff --git a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh b/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh index 9e04b963fc40e..ab906604fce06 100755 --- a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh +++ b/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh @@ -23,6 +23,7 @@ IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt" DEPLOY_MODE="minikube" IMAGE_REPO="docker.io/kubespark" IMAGE_TAG="N/A" +JAVA_IMAGE_TAG="8-jre-slim" SPARK_TGZ="N/A" # Parse arguments @@ -40,6 +41,10 @@ while (( "$#" )); do IMAGE_TAG="$2" shift ;; + --java-image-tag) + JAVA_IMAGE_TAG="$2" + shift + ;; --image-tag-output-file) IMAGE_TAG_OUTPUT_FILE="$2" shift @@ -82,6 +87,9 @@ then IMAGE_TAG=$(uuidgen); cd $SPARK_INPUT_DIR + # OpenJDK base-image tag (e.g. 8-jre-slim, 11-jre-slim) + JAVA_IMAGE_TAG_BUILD_ARG="-b java_image_tag=$JAVA_IMAGE_TAG" + # Build PySpark image LANGUAGE_BINDING_BUILD_ARGS="-p $DOCKER_FILE_BASE_PATH/bindings/python/Dockerfile" @@ -95,7 +103,7 @@ then case $DEPLOY_MODE in cloud) # Build images - $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG $JAVA_IMAGE_TAG_BUILD_ARG $LANGUAGE_BINDING_BUILD_ARGS build # Push images appropriately if [[ $IMAGE_REPO == gcr.io* ]] ; @@ -109,13 +117,13 @@ then docker-for-desktop) # Only need to build as this will place it in our local Docker repo which is all # we need for Docker for Desktop to work so no need to also push - $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG $JAVA_IMAGE_TAG_BUILD_ARG $LANGUAGE_BINDING_BUILD_ARGS build ;; minikube) # Only need to build and if we do this with the -m option for minikube we will # build the images directly using the minikube Docker daemon so no need to push - $SPARK_INPUT_DIR/bin/docker-image-tool.sh -m -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -m -r $IMAGE_REPO -t $IMAGE_TAG $JAVA_IMAGE_TAG_BUILD_ARG $LANGUAGE_BINDING_BUILD_ARGS build ;; *) echo "Unrecognized deploy mode $DEPLOY_MODE" && exit 1 From 74cd46eb691be5dc1cb1c496eeeaa2614945bd98 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 13 Feb 2020 11:42:00 -0800 Subject: [PATCH 1370/1387] [SPARK-30816][K8S][TESTS] Fix dev-run-integration-tests.sh to ignore empty params ### What changes were proposed in this pull request? This PR aims to fix `dev-run-integration-tests.sh` to ignore empty params correctly. ### Why are the changes needed? The following script runs `mvn` integration test like the following. ``` $ resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh ... build/mvn integration-test -f /Users/dongjoon/APACHE/spark/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-2.12 -Pkubernetes -Pkubernetes-integration-tests -Djava.version=8 -Dspark.kubernetes.test.sparkTgz=N/A -Dspark.kubernetes.test.imageTag=N/A -Dspark.kubernetes.test.imageRepo=docker.io/kubespark -Dspark.kubernetes.test.deployMode=minikube -Dtest.include.tags=k8s -Dspark.kubernetes.test.namespace= -Dspark.kubernetes.test.serviceAccountName= -Dspark.kubernetes.test.kubeConfigContext= -Dspark.kubernetes.test.master= -Dtest.exclude.tags= -Dspark.kubernetes.test.jvmImage=spark -Dspark.kubernetes.test.pythonImage=spark-py -Dspark.kubernetes.test.rImage=spark-r ``` After this PR, the empty parameters like the followings will be skipped like the original design. ``` -Dspark.kubernetes.test.namespace= -Dspark.kubernetes.test.serviceAccountName= -Dspark.kubernetes.test.kubeConfigContext= -Dspark.kubernetes.test.master= -Dtest.exclude.tags= ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Pass the Jenkins K8S integration test. Closes #27566 from dongjoon-hyun/SPARK-30816. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../integration-tests/dev/dev-run-integration-tests.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh index 76d6e1c1e8499..607bb243458a6 100755 --- a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh +++ b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh @@ -130,27 +130,27 @@ then properties=( ${properties[@]} -Dspark.kubernetes.test.javaImageTag=$JAVA_IMAGE_TAG ) fi -if [ -n $NAMESPACE ]; +if [ -n "$NAMESPACE" ]; then properties=( ${properties[@]} -Dspark.kubernetes.test.namespace=$NAMESPACE ) fi -if [ -n $SERVICE_ACCOUNT ]; +if [ -n "$SERVICE_ACCOUNT" ]; then properties=( ${properties[@]} -Dspark.kubernetes.test.serviceAccountName=$SERVICE_ACCOUNT ) fi -if [ -n $CONTEXT ]; +if [ -n "$CONTEXT" ]; then properties=( ${properties[@]} -Dspark.kubernetes.test.kubeConfigContext=$CONTEXT ) fi -if [ -n $SPARK_MASTER ]; +if [ -n "$SPARK_MASTER" ]; then properties=( ${properties[@]} -Dspark.kubernetes.test.master=$SPARK_MASTER ) fi -if [ -n $EXCLUDE_TAGS ]; +if [ -n "$EXCLUDE_TAGS" ]; then properties=( ${properties[@]} -Dtest.exclude.tags=$EXCLUDE_TAGS ) fi From e2d3983de78f5c80fac066b7ee8bedd0987110dd Mon Sep 17 00:00:00 2001 From: Ali Afroozeh Date: Thu, 13 Feb 2020 23:58:55 +0100 Subject: [PATCH 1371/1387] [SPARK-30798][SQL] Scope Session.active in QueryExecution ### What changes were proposed in this pull request? This PR scopes `SparkSession.active` to prevent problems with processing queries with possibly different spark sessions (and different configs). A new method, `withActive` is introduced on `SparkSession` that restores the previous spark session after the block of code is executed. ### Why are the changes needed? `SparkSession.active` is a thread local variable that points to the current thread's spark session. It is important to note that the `SQLConf.get` method depends on `SparkSession.active`. In the current implementation it is possible that `SparkSession.active` points to a different session which causes various problems. Most of these problems arise because part of the query processing is done using the configurations of a different session. For example, when creating a data frame using a new session, i.e., `session.sql("...")`, part of the data frame is constructed using the currently active spark session, which can be a different session from the one used later for processing the query. ### Does this PR introduce any user-facing change? The `withActive` method is introduced on `SparkSession`. ### How was this patch tested? Unit tests (to be added) Closes #27387 from dbaliafroozeh/UseWithActiveSessionInQueryExecution. Authored-by: Ali Afroozeh Signed-off-by: herman --- .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../apache/spark/sql/DataFrameWriterV2.scala | 2 +- .../scala/org/apache/spark/sql/Dataset.scala | 36 ++++++++++--------- .../spark/sql/KeyValueGroupedDataset.scala | 5 +-- .../org/apache/spark/sql/SparkSession.scala | 30 +++++++++++----- .../spark/sql/execution/QueryExecution.scala | 16 +++++---- .../spark/sql/execution/SQLExecution.scala | 4 +-- .../streaming/MicroBatchExecution.scala | 4 +-- .../continuous/ContinuousExecution.scala | 2 +- .../spark/sql/internal/CatalogImpl.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 10 ++++++ .../apache/spark/sql/SQLQueryTestSuite.scala | 2 +- .../ui/SQLAppStatusListenerSuite.scala | 2 +- .../SparkExecuteStatementOperation.scala | 2 +- .../hive/thriftserver/SparkSQLDriver.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 3 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- 17 files changed, 74 insertions(+), 52 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 4557219abeb15..fff1f4b636dea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -896,7 +896,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private def runCommand(session: SparkSession, name: String)(command: LogicalPlan): Unit = { val qe = session.sessionState.executePlan(command) // call `QueryExecution.toRDD` to trigger the execution of commands. - SQLExecution.withNewExecutionId(session, qe, Some(name))(qe.toRdd) + SQLExecution.withNewExecutionId(qe, Some(name))(qe.toRdd) } private def lookupV2Provider(): Option[TableProvider] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala index f5dd7613d4103..cf6bde5a2bcb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala @@ -226,7 +226,7 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T]) private def runCommand(name: String)(command: LogicalPlan): Unit = { val qe = sparkSession.sessionState.executePlan(command) // call `QueryExecution.toRDD` to trigger the execution of commands. - SQLExecution.withNewExecutionId(sparkSession, qe, Some(name))(qe.toRdd) + SQLExecution.withNewExecutionId(qe, Some(name))(qe.toRdd) } private def internalReplace(orCreate: Boolean): Unit = { 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 a1c33f92d17b4..42f35354e864f 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 @@ -82,18 +82,19 @@ private[sql] object Dataset { dataset } - def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = { - val qe = sparkSession.sessionState.executePlan(logicalPlan) - qe.assertAnalyzed() - new Dataset[Row](sparkSession, qe, RowEncoder(qe.analyzed.schema)) + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = + sparkSession.withActive { + val qe = sparkSession.sessionState.executePlan(logicalPlan) + qe.assertAnalyzed() + new Dataset[Row](qe, RowEncoder(qe.analyzed.schema)) } /** A variant of ofRows that allows passing in a tracker so we can track query parsing time. */ def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan, tracker: QueryPlanningTracker) - : DataFrame = { + : DataFrame = sparkSession.withActive { val qe = new QueryExecution(sparkSession, logicalPlan, tracker) qe.assertAnalyzed() - new Dataset[Row](sparkSession, qe, RowEncoder(qe.analyzed.schema)) + new Dataset[Row](qe, RowEncoder(qe.analyzed.schema)) } } @@ -185,13 +186,12 @@ private[sql] object Dataset { */ @Stable class Dataset[T] private[sql]( - @transient private val _sparkSession: SparkSession, @DeveloperApi @Unstable @transient val queryExecution: QueryExecution, @DeveloperApi @Unstable @transient val encoder: Encoder[T]) extends Serializable { @transient lazy val sparkSession: SparkSession = { - if (_sparkSession == null) { + if (queryExecution == null || queryExecution.sparkSession == null) { throw new SparkException( "Dataset transformations and actions can only be invoked by the driver, not inside of" + " other Dataset transformations; for example, dataset1.map(x => dataset2.values.count()" + @@ -199,7 +199,7 @@ class Dataset[T] private[sql]( "performed inside of the dataset1.map transformation. For more information," + " see SPARK-28702.") } - _sparkSession + queryExecution.sparkSession } // A globally unique id of this Dataset. @@ -211,7 +211,7 @@ class Dataset[T] private[sql]( // you wrap it with `withNewExecutionId` if this actions doesn't call other action. def this(sparkSession: SparkSession, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { - this(sparkSession, sparkSession.sessionState.executePlan(logicalPlan), encoder) + this(sparkSession.sessionState.executePlan(logicalPlan), encoder) } def this(sqlContext: SQLContext, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { @@ -445,7 +445,7 @@ class Dataset[T] private[sql]( */ // This is declared with parentheses to prevent the Scala compiler from treating // `ds.toDF("1")` as invoking this toDF and then apply on the returned DataFrame. - def toDF(): DataFrame = new Dataset[Row](sparkSession, queryExecution, RowEncoder(schema)) + def toDF(): DataFrame = new Dataset[Row](queryExecution, RowEncoder(schema)) /** * Returns a new Dataset where each record has been mapped on to the specified type. The @@ -503,7 +503,9 @@ class Dataset[T] private[sql]( * @group basic * @since 1.6.0 */ - def schema: StructType = queryExecution.analyzed.schema + def schema: StructType = sparkSession.withActive { + queryExecution.analyzed.schema + } /** * Prints the schema to the console in a nice tree format. @@ -539,7 +541,7 @@ class Dataset[T] private[sql]( * @group basic * @since 3.0.0 */ - def explain(mode: String): Unit = { + def explain(mode: String): Unit = sparkSession.withActive { // Because temporary views are resolved during analysis when we create a Dataset, and // `ExplainCommand` analyzes input query plan and resolves temporary views again. Using // `ExplainCommand` here will probably output different query plans, compared to the results @@ -1502,7 +1504,7 @@ class Dataset[T] private[sql]( val namedColumns = columns.map(_.withInputType(exprEnc, logicalPlan.output).named) val execution = new QueryExecution(sparkSession, Project(namedColumns, logicalPlan)) - new Dataset(sparkSession, execution, ExpressionEncoder.tuple(encoders)) + new Dataset(execution, ExpressionEncoder.tuple(encoders)) } /** @@ -3472,7 +3474,7 @@ class Dataset[T] private[sql]( * an execution. */ private def withNewExecutionId[U](body: => U): U = { - SQLExecution.withNewExecutionId(sparkSession, queryExecution)(body) + SQLExecution.withNewExecutionId(queryExecution)(body) } /** @@ -3481,7 +3483,7 @@ class Dataset[T] private[sql]( * reset. */ private def withNewRDDExecutionId[U](body: => U): U = { - SQLExecution.withNewExecutionId(sparkSession, rddQueryExecution) { + SQLExecution.withNewExecutionId(rddQueryExecution) { rddQueryExecution.executedPlan.resetMetrics() body } @@ -3492,7 +3494,7 @@ class Dataset[T] private[sql]( * user-registered callback functions. */ private def withAction[U](name: String, qe: QueryExecution)(action: SparkPlan => U) = { - SQLExecution.withNewExecutionId(sparkSession, qe, Some(name)) { + SQLExecution.withNewExecutionId(qe, Some(name)) { qe.executedPlan.resetMetrics() action(qe.executedPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 89cc9735e4f6a..76ee297dfca79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -449,10 +449,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( val aggregate = Aggregate(groupingAttributes, keyColumn +: namedColumns, logicalPlan) val execution = new QueryExecution(sparkSession, aggregate) - new Dataset( - sparkSession, - execution, - ExpressionEncoder.tuple(kExprEnc +: encoders)) + new Dataset(execution, ExpressionEncoder.tuple(kExprEnc +: encoders)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index abefb348cafc7..1fb97fb4b4cf1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -293,8 +293,7 @@ class SparkSession private( * * @since 2.0.0 */ - def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { - SparkSession.setActiveSession(this) + def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = withActive { val encoder = Encoders.product[A] Dataset.ofRows(self, ExternalRDD(rdd, self)(encoder)) } @@ -304,8 +303,7 @@ class SparkSession private( * * @since 2.0.0 */ - def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { - SparkSession.setActiveSession(this) + def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = withActive { val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes Dataset.ofRows(self, LocalRelation.fromProduct(attributeSeq, data)) @@ -343,7 +341,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi - def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { + def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = withActive { // TODO: use MutableProjection when rowRDD is another DataFrame and the applied // schema differs from the existing schema on any field data type. val encoder = RowEncoder(schema) @@ -373,7 +371,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi - def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = { + def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = withActive { Dataset.ofRows(self, LocalRelation.fromExternalRows(schema.toAttributes, rows.asScala)) } @@ -385,7 +383,7 @@ class SparkSession private( * * @since 2.0.0 */ - def createDataFrame(rdd: RDD[_], beanClass: Class[_]): DataFrame = { + def createDataFrame(rdd: RDD[_], beanClass: Class[_]): DataFrame = withActive { val attributeSeq: Seq[AttributeReference] = getSchema(beanClass) val className = beanClass.getName val rowRdd = rdd.mapPartitions { iter => @@ -414,7 +412,7 @@ class SparkSession private( * SELECT * queries will return the columns in an undefined order. * @since 1.6.0 */ - def createDataFrame(data: java.util.List[_], beanClass: Class[_]): DataFrame = { + def createDataFrame(data: java.util.List[_], beanClass: Class[_]): DataFrame = withActive { val attrSeq = getSchema(beanClass) val rows = SQLContext.beansToRows(data.asScala.iterator, beanClass, attrSeq) Dataset.ofRows(self, LocalRelation(attrSeq, rows.toSeq)) @@ -599,7 +597,7 @@ class SparkSession private( * * @since 2.0.0 */ - def sql(sqlText: String): DataFrame = { + def sql(sqlText: String): DataFrame = withActive { val tracker = new QueryPlanningTracker val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { sessionState.sqlParser.parsePlan(sqlText) @@ -751,6 +749,20 @@ class SparkSession private( } } + /** + * Execute a block of code with the this session set as the active session, and restore the + * previous session on completion. + */ + private[sql] def withActive[T](block: => T): T = { + // Use the active session thread local directly to make sure we get the session that is actually + // set and not the default session. This to prevent that we promote the default session to the + // active session once we are done. + val old = SparkSession.activeThreadSession.get() + SparkSession.setActiveSession(this) + try block finally { + SparkSession.setActiveSession(old) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 38ef66682c413..53b6b5d82c021 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -63,13 +63,12 @@ class QueryExecution( } } - lazy val analyzed: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.ANALYSIS) { - SparkSession.setActiveSession(sparkSession) + lazy val analyzed: LogicalPlan = executePhase(QueryPlanningTracker.ANALYSIS) { // We can't clone `logical` here, which will reset the `_analyzed` flag. sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker) } - lazy val withCachedData: LogicalPlan = { + lazy val withCachedData: LogicalPlan = sparkSession.withActive { assertAnalyzed() assertSupported() // clone the plan to avoid sharing the plan instance between different stages like analyzing, @@ -77,20 +76,20 @@ class QueryExecution( sparkSession.sharedState.cacheManager.useCachedData(analyzed.clone()) } - lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { + lazy val optimizedPlan: LogicalPlan = executePhase(QueryPlanningTracker.OPTIMIZATION) { // clone the plan to avoid sharing the plan instance between different stages like analyzing, // optimizing and planning. sparkSession.sessionState.optimizer.executeAndTrack(withCachedData.clone(), tracker) } - lazy val sparkPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { + lazy val sparkPlan: SparkPlan = executePhase(QueryPlanningTracker.PLANNING) { // Clone the logical plan here, in case the planner rules change the states of the logical plan. QueryExecution.createSparkPlan(sparkSession, planner, optimizedPlan.clone()) } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. - lazy val executedPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { + lazy val executedPlan: SparkPlan = executePhase(QueryPlanningTracker.PLANNING) { // clone the plan to avoid sharing the plan instance between different stages like analyzing, // optimizing and planning. QueryExecution.prepareForExecution(preparations, sparkPlan.clone()) @@ -116,6 +115,10 @@ class QueryExecution( QueryExecution.preparations(sparkSession) } + private def executePhase[T](phase: String)(block: => T): T = sparkSession.withActive { + tracker.measurePhase(phase)(block) + } + def simpleString: String = simpleString(false) def simpleString(formatted: Boolean): String = withRedaction { @@ -305,7 +308,6 @@ object QueryExecution { sparkSession: SparkSession, planner: SparkPlanner, plan: LogicalPlan): SparkPlan = { - SparkSession.setActiveSession(sparkSession) // TODO: We use next(), i.e. take the first plan returned by the planner, here for now, // but we will implement to choose the best plan. planner.plan(ReturnAnswer(plan)).next() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index 9f177819f6ea7..59c503e372535 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -62,9 +62,9 @@ object SQLExecution { * we can connect them with an execution. */ def withNewExecutionId[T]( - sparkSession: SparkSession, queryExecution: QueryExecution, - name: Option[String] = None)(body: => T): T = { + name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive { + val sparkSession = queryExecution.sparkSession val sc = sparkSession.sparkContext val oldExecutionId = sc.getLocalProperty(EXECUTION_ID_KEY) val executionId = SQLExecution.nextExecutionId diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 83bc347e23ed4..45a2ce16183a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -563,11 +563,11 @@ class MicroBatchExecution( } val nextBatch = - new Dataset(sparkSessionToRunBatch, lastExecution, RowEncoder(lastExecution.analyzed.schema)) + new Dataset(lastExecution, RowEncoder(lastExecution.analyzed.schema)) val batchSinkProgress: Option[StreamWriterCommitProgress] = reportTimeTaken("addBatch") { - SQLExecution.withNewExecutionId(sparkSessionToRunBatch, lastExecution) { + SQLExecution.withNewExecutionId(lastExecution) { sink match { case s: Sink => s.addBatch(currentBatchId, nextBatch) case _: SupportsWrite => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index a9b724a73a18e..a109c2171f3d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -252,7 +252,7 @@ class ContinuousExecution( updateStatusMessage("Running") reportTimeTaken("runContinuous") { - SQLExecution.withNewExecutionId(sparkSessionForQuery, lastExecution) { + SQLExecution.withNewExecutionId(lastExecution) { lastExecution.executedPlan.execute() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 3740b56cb9cbb..d3ef03e9b3b74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -520,7 +520,7 @@ private[sql] object CatalogImpl { val encoded = data.map(d => enc.toRow(d).copy()) val plan = new LocalRelation(enc.schema.toAttributes, encoded) val queryExecution = sparkSession.sessionState.executePlan(plan) - new Dataset[T](sparkSession, queryExecution, enc) + new Dataset[T](queryExecution, enc) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 233d67898f909..b0bd612e88d98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1899,6 +1899,16 @@ class DatasetSuite extends QueryTest val e = intercept[AnalysisException](spark.range(1).tail(-1)) e.getMessage.contains("tail expression must be equal to or greater than 0") } + + test("SparkSession.active should be the same instance after dataset operations") { + val active = SparkSession.getActiveSession.get + val clone = active.cloneSession() + val ds = new Dataset(clone, spark.range(10).queryExecution.logical, Encoders.INT) + + ds.queryExecution.analyzed + + assert(active eq SparkSession.getActiveSession.get) + } } object AssertExecutionId { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index da4727f6a98cb..83285911b3948 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -511,7 +511,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { val df = session.sql(sql) val schema = df.schema.catalogString // Get answer, but also get rid of the #1234 expression ids that show up in explain plans - val answer = SQLExecution.withNewExecutionId(session, df.queryExecution, Some(sql)) { + val answer = SQLExecution.withNewExecutionId(df.queryExecution, Some(sql)) { hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index fdfd392a224cb..d18a35c3110f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -508,7 +508,7 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils override lazy val executedPlan = physicalPlan } - SQLExecution.withNewExecutionId(spark, dummyQueryExecution) { + SQLExecution.withNewExecutionId(dummyQueryExecution) { physicalPlan.execute().collect() } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 76d07848f79a9..cf0e5ebf3a2b1 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -295,7 +295,7 @@ private[hive] class SparkExecuteStatementOperation( resultList.get.iterator } } - dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + dataTypes = result.schema.fields.map(_.dataType) } catch { // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 362ac362e9718..12fba0eae6dce 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -61,7 +61,7 @@ private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlCont try { context.sparkContext.setJobDescription(command) val execution = context.sessionState.executePlan(context.sql(command).logicalPlan) - hiveResponse = SQLExecution.withNewExecutionId(context.sparkSession, execution) { + hiveResponse = SQLExecution.withNewExecutionId(execution) { hiveResultString(execution.executedPlan) } tableSchema = getResultSetSchema(execution) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 28e1db961f611..8b1f4c92755b9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -346,8 +346,7 @@ abstract class HiveComparisonTest val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) def getResult(): Seq[String] = { - SQLExecution.withNewExecutionId( - query.sparkSession, query)(hiveResultString(query.executedPlan)) + SQLExecution.withNewExecutionId(query)(hiveResultString(query.executedPlan)) } try { (query, prepareAnswer(query, getResult())) } catch { case e: Throwable => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index cc4592a5caf68..222244a04f5f5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -501,7 +501,7 @@ private[hive] class TestHiveSparkSession( // has already set the execution id. if (sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) == null) { // We don't actually have a `QueryExecution` here, use a fake one instead. - SQLExecution.withNewExecutionId(this, new QueryExecution(this, OneRowRelation())) { + SQLExecution.withNewExecutionId(new QueryExecution(this, OneRowRelation())) { createCmds.foreach(_()) } } else { From 57254c9719f9af9ad985596ed7fbbaafa4052002 Mon Sep 17 00:00:00 2001 From: sarthfrey-db Date: Thu, 13 Feb 2020 16:15:00 -0800 Subject: [PATCH 1372/1387] [SPARK-30667][CORE] Add allGather method to BarrierTaskContext ### What changes were proposed in this pull request? The `allGather` method is added to the `BarrierTaskContext`. This method contains the same functionality as the `BarrierTaskContext.barrier` method; it blocks the task until all tasks make the call, at which time they may continue execution. In addition, the `allGather` method takes an input message. Upon returning from the `allGather` the task receives a list of all the messages sent by all the tasks that made the `allGather` call. ### Why are the changes needed? There are many situations where having the tasks communicate in a synchronized way is useful. One simple example is if each task needs to start a server to serve requests from one another; first the tasks must find a free port (the result of which is undetermined beforehand) and then start making requests, but to do so they each must know the port chosen by the other task. An `allGather` method would allow them to inform each other of the port they will run on. ### Does this PR introduce any user-facing change? Yes, an `BarrierTaskContext.allGather` method will be available through the Scala, Java, and Python APIs. ### How was this patch tested? Most of the code path is already covered by tests to the `barrier` method, since this PR includes a refactor so that much code is shared by the `barrier` and `allGather` methods. However, a test is added to assert that an all gather on each tasks partition ID will return a list of every partition ID. An example through the Python API: ```python >>> from pyspark import BarrierTaskContext >>> >>> def f(iterator): ... context = BarrierTaskContext.get() ... return [context.allGather('{}'.format(context.partitionId()))] ... >>> sc.parallelize(range(4), 4).barrier().mapPartitions(f).collect()[0] [u'3', u'1', u'0', u'2'] ``` Closes #27395 from sarthfrey/master. Lead-authored-by: sarthfrey-db Co-authored-by: sarthfrey Signed-off-by: Xiangrui Meng --- .../org/apache/spark/BarrierCoordinator.scala | 113 +++++++++++-- .../org/apache/spark/BarrierTaskContext.scala | 153 ++++++++++++------ .../spark/api/python/PythonRunner.scala | 51 ++++-- .../scheduler/BarrierTaskContextSuite.scala | 74 +++++++++ python/pyspark/taskcontext.py | 49 +++++- python/pyspark/tests/test_taskcontext.py | 20 +++ 6 files changed, 381 insertions(+), 79 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala index 4e417679ca663..042a2664a0e27 100644 --- a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala @@ -17,12 +17,17 @@ package org.apache.spark +import java.nio.charset.StandardCharsets.UTF_8 import java.util.{Timer, TimerTask} import java.util.concurrent.ConcurrentHashMap import java.util.function.Consumer import scala.collection.mutable.ArrayBuffer +import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, render} + import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerStageCompleted} @@ -99,10 +104,15 @@ private[spark] class BarrierCoordinator( // reset when a barrier() call fails due to timeout. private var barrierEpoch: Int = 0 - // An array of RPCCallContexts for barrier tasks that are waiting for reply of a barrier() - // call. + // An Array of RPCCallContexts for barrier tasks that have made a blocking runBarrier() call private val requesters: ArrayBuffer[RpcCallContext] = new ArrayBuffer[RpcCallContext](numTasks) + // An Array of allGather messages for barrier tasks that have made a blocking runBarrier() call + private val allGatherMessages: ArrayBuffer[String] = new Array[String](numTasks).to[ArrayBuffer] + + // The blocking requestMethod called by tasks to sync up for this stage attempt + private var requestMethodToSync: RequestMethod.Value = RequestMethod.BARRIER + // A timer task that ensures we may timeout for a barrier() call. private var timerTask: TimerTask = null @@ -130,9 +140,32 @@ private[spark] class BarrierCoordinator( // Process the global sync request. The barrier() call succeed if collected enough requests // within a configured time, otherwise fail all the pending requests. - def handleRequest(requester: RpcCallContext, request: RequestToSync): Unit = synchronized { + def handleRequest( + requester: RpcCallContext, + request: RequestToSync + ): Unit = synchronized { val taskId = request.taskAttemptId val epoch = request.barrierEpoch + val requestMethod = request.requestMethod + val partitionId = request.partitionId + val allGatherMessage = request match { + case ag: AllGatherRequestToSync => ag.allGatherMessage + case _ => "" + } + + if (requesters.size == 0) { + requestMethodToSync = requestMethod + } + + if (requestMethodToSync != requestMethod) { + requesters.foreach( + _.sendFailure(new SparkException(s"$barrierId tried to use requestMethod " + + s"`$requestMethod` during barrier epoch $barrierEpoch, which does not match " + + s"the current synchronized requestMethod `$requestMethodToSync`" + )) + ) + cleanupBarrierStage(barrierId) + } // Require the number of tasks is correctly set from the BarrierTaskContext. require(request.numTasks == numTasks, s"Number of tasks of $barrierId is " + @@ -153,6 +186,7 @@ private[spark] class BarrierCoordinator( } // Add the requester to array of RPCCallContexts pending for reply. requesters += requester + allGatherMessages(partitionId) = allGatherMessage logInfo(s"Barrier sync epoch $barrierEpoch from $barrierId received update from Task " + s"$taskId, current progress: ${requesters.size}/$numTasks.") if (maybeFinishAllRequesters(requesters, numTasks)) { @@ -162,6 +196,7 @@ private[spark] class BarrierCoordinator( s"tasks, finished successfully.") barrierEpoch += 1 requesters.clear() + allGatherMessages.clear() cancelTimerTask() } } @@ -173,7 +208,13 @@ private[spark] class BarrierCoordinator( requesters: ArrayBuffer[RpcCallContext], numTasks: Int): Boolean = { if (requesters.size == numTasks) { - requesters.foreach(_.reply(())) + requestMethodToSync match { + case RequestMethod.BARRIER => + requesters.foreach(_.reply("")) + case RequestMethod.ALL_GATHER => + val json: String = compact(render(allGatherMessages)) + requesters.foreach(_.reply(json)) + } true } else { false @@ -186,6 +227,7 @@ private[spark] class BarrierCoordinator( // messages come from current stage attempt shall fail. barrierEpoch = -1 requesters.clear() + allGatherMessages.clear() cancelTimerTask() } } @@ -199,11 +241,11 @@ private[spark] class BarrierCoordinator( } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case request @ RequestToSync(numTasks, stageId, stageAttemptId, _, _) => + case request: RequestToSync => // Get or init the ContextBarrierState correspond to the stage attempt. - val barrierId = ContextBarrierId(stageId, stageAttemptId) + val barrierId = ContextBarrierId(request.stageId, request.stageAttemptId) states.computeIfAbsent(barrierId, - (key: ContextBarrierId) => new ContextBarrierState(key, numTasks)) + (key: ContextBarrierId) => new ContextBarrierState(key, request.numTasks)) val barrierState = states.get(barrierId) barrierState.handleRequest(context, request) @@ -216,6 +258,16 @@ private[spark] class BarrierCoordinator( private[spark] sealed trait BarrierCoordinatorMessage extends Serializable +private[spark] sealed trait RequestToSync extends BarrierCoordinatorMessage { + def numTasks: Int + def stageId: Int + def stageAttemptId: Int + def taskAttemptId: Long + def barrierEpoch: Int + def partitionId: Int + def requestMethod: RequestMethod.Value +} + /** * A global sync request message from BarrierTaskContext, by `barrier()` call. Each request is * identified by stageId + stageAttemptId + barrierEpoch. @@ -224,11 +276,44 @@ private[spark] sealed trait BarrierCoordinatorMessage extends Serializable * @param stageId ID of current stage * @param stageAttemptId ID of current stage attempt * @param taskAttemptId Unique ID of current task - * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls. + * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls + * @param partitionId ID of the current partition the task is assigned to + * @param requestMethod The BarrierTaskContext method that was called to trigger BarrierCoordinator */ -private[spark] case class RequestToSync( - numTasks: Int, - stageId: Int, - stageAttemptId: Int, - taskAttemptId: Long, - barrierEpoch: Int) extends BarrierCoordinatorMessage +private[spark] case class BarrierRequestToSync( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + taskAttemptId: Long, + barrierEpoch: Int, + partitionId: Int, + requestMethod: RequestMethod.Value +) extends RequestToSync + +/** + * A global sync request message from BarrierTaskContext, by `allGather()` call. Each request is + * identified by stageId + stageAttemptId + barrierEpoch. + * + * @param numTasks The number of global sync requests the BarrierCoordinator shall receive + * @param stageId ID of current stage + * @param stageAttemptId ID of current stage attempt + * @param taskAttemptId Unique ID of current task + * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls + * @param partitionId ID of the current partition the task is assigned to + * @param requestMethod The BarrierTaskContext method that was called to trigger BarrierCoordinator + * @param allGatherMessage Message sent from the BarrierTaskContext if requestMethod is ALL_GATHER + */ +private[spark] case class AllGatherRequestToSync( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + taskAttemptId: Long, + barrierEpoch: Int, + partitionId: Int, + requestMethod: RequestMethod.Value, + allGatherMessage: String +) extends RequestToSync + +private[spark] object RequestMethod extends Enumeration { + val BARRIER, ALL_GATHER = Value +} diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index 3d369802f3023..2263538a11676 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -17,11 +17,19 @@ package org.apache.spark +import java.nio.charset.StandardCharsets.UTF_8 import java.util.{Properties, Timer, TimerTask} import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.TimeoutException import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.json4s.DefaultFormats +import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.parse import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.executor.TaskMetrics @@ -59,49 +67,31 @@ class BarrierTaskContext private[spark] ( // from different tasks within the same barrier stage attempt to succeed. private lazy val numTasks = getTaskInfos().size - /** - * :: Experimental :: - * Sets a global barrier and waits until all tasks in this stage hit this barrier. Similar to - * MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same - * stage have reached this routine. - * - * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all - * possible code branches. Otherwise, you may get the job hanging or a SparkException after - * timeout. Some examples of '''misuses''' are listed below: - * 1. Only call barrier() function on a subset of all the tasks in the same barrier stage, it - * shall lead to timeout of the function call. - * {{{ - * rdd.barrier().mapPartitions { iter => - * val context = BarrierTaskContext.get() - * if (context.partitionId() == 0) { - * // Do nothing. - * } else { - * context.barrier() - * } - * iter - * } - * }}} - * - * 2. Include barrier() function in a try-catch code block, this may lead to timeout of the - * second function call. - * {{{ - * rdd.barrier().mapPartitions { iter => - * val context = BarrierTaskContext.get() - * try { - * // Do something that might throw an Exception. - * doSomething() - * context.barrier() - * } catch { - * case e: Exception => logWarning("...", e) - * } - * context.barrier() - * iter - * } - * }}} - */ - @Experimental - @Since("2.4.0") - def barrier(): Unit = { + private def getRequestToSync( + numTasks: Int, + stageId: Int, + stageAttemptNumber: Int, + taskAttemptId: Long, + barrierEpoch: Int, + partitionId: Int, + requestMethod: RequestMethod.Value, + allGatherMessage: String + ): RequestToSync = { + requestMethod match { + case RequestMethod.BARRIER => + BarrierRequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, + barrierEpoch, partitionId, requestMethod) + case RequestMethod.ALL_GATHER => + AllGatherRequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, + barrierEpoch, partitionId, requestMethod, allGatherMessage) + } + } + + private def runBarrier( + requestMethod: RequestMethod.Value, + allGatherMessage: String = "" + ): String = { + logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) has entered " + s"the global sync, current barrier epoch is $barrierEpoch.") logTrace("Current callSite: " + Utils.getCallSite()) @@ -118,10 +108,12 @@ class BarrierTaskContext private[spark] ( // Log the update of global sync every 60 seconds. timer.schedule(timerTask, 60000, 60000) + var json: String = "" + try { - val abortableRpcFuture = barrierCoordinator.askAbortable[Unit]( - message = RequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, - barrierEpoch), + val abortableRpcFuture = barrierCoordinator.askAbortable[String]( + message = getRequestToSync(numTasks, stageId, stageAttemptNumber, + taskAttemptId, barrierEpoch, partitionId, requestMethod, allGatherMessage), // Set a fixed timeout for RPC here, so users shall get a SparkException thrown by // BarrierCoordinator on timeout, instead of RPCTimeoutException from the RPC framework. timeout = new RpcTimeout(365.days, "barrierTimeout")) @@ -133,7 +125,7 @@ class BarrierTaskContext private[spark] ( while (!abortableRpcFuture.toFuture.isCompleted) { // wait RPC future for at most 1 second try { - ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second) + json = ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second) } catch { case _: TimeoutException | _: InterruptedException => // If `TimeoutException` thrown, waiting RPC future reach 1 second. @@ -163,6 +155,73 @@ class BarrierTaskContext private[spark] ( timerTask.cancel() timer.purge() } + json + } + + /** + * :: Experimental :: + * Sets a global barrier and waits until all tasks in this stage hit this barrier. Similar to + * MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same + * stage have reached this routine. + * + * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all + * possible code branches. Otherwise, you may get the job hanging or a SparkException after + * timeout. Some examples of '''misuses''' are listed below: + * 1. Only call barrier() function on a subset of all the tasks in the same barrier stage, it + * shall lead to timeout of the function call. + * {{{ + * rdd.barrier().mapPartitions { iter => + * val context = BarrierTaskContext.get() + * if (context.partitionId() == 0) { + * // Do nothing. + * } else { + * context.barrier() + * } + * iter + * } + * }}} + * + * 2. Include barrier() function in a try-catch code block, this may lead to timeout of the + * second function call. + * {{{ + * rdd.barrier().mapPartitions { iter => + * val context = BarrierTaskContext.get() + * try { + * // Do something that might throw an Exception. + * doSomething() + * context.barrier() + * } catch { + * case e: Exception => logWarning("...", e) + * } + * context.barrier() + * iter + * } + * }}} + */ + @Experimental + @Since("2.4.0") + def barrier(): Unit = { + runBarrier(RequestMethod.BARRIER) + () + } + + /** + * :: Experimental :: + * Blocks until all tasks in the same stage have reached this routine. Each task passes in + * a message and returns with a list of all the messages passed in by each of those tasks. + * + * CAUTION! The allGather method requires the same precautions as the barrier method + * + * The message is type String rather than Array[Byte] because it is more convenient for + * the user at the cost of worse performance. + */ + @Experimental + @Since("3.0.0") + def allGather(message: String): ArrayBuffer[String] = { + val json = runBarrier(RequestMethod.ALL_GATHER, message) + val jsonArray = parse(json) + implicit val formats = DefaultFormats + ArrayBuffer(jsonArray.extract[Array[String]]: _*) } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 658e0d593a167..fa8bf0fc06358 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -24,8 +24,13 @@ import java.nio.charset.StandardCharsets.UTF_8 import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal +import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, render} + import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} @@ -238,13 +243,18 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( sock.setSoTimeout(10000) authHelper.authClient(sock) val input = new DataInputStream(sock.getInputStream()) - input.readInt() match { + val requestMethod = input.readInt() + // The BarrierTaskContext function may wait infinitely, socket shall not timeout + // before the function finishes. + sock.setSoTimeout(0) + requestMethod match { case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION => - // The barrier() function may wait infinitely, socket shall not timeout - // before the function finishes. - sock.setSoTimeout(0) - barrierAndServe(sock) - + barrierAndServe(requestMethod, sock) + case BarrierTaskContextMessageProtocol.ALL_GATHER_FUNCTION => + val length = input.readInt() + val message = new Array[Byte](length) + input.readFully(message) + barrierAndServe(requestMethod, sock, new String(message, UTF_8)) case _ => val out = new DataOutputStream(new BufferedOutputStream( sock.getOutputStream)) @@ -395,15 +405,31 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( } /** - * Gateway to call BarrierTaskContext.barrier(). + * Gateway to call BarrierTaskContext methods. */ - def barrierAndServe(sock: Socket): Unit = { - require(serverSocket.isDefined, "No available ServerSocket to redirect the barrier() call.") - + def barrierAndServe(requestMethod: Int, sock: Socket, message: String = ""): Unit = { + require( + serverSocket.isDefined, + "No available ServerSocket to redirect the BarrierTaskContext method call." + ) val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) try { - context.asInstanceOf[BarrierTaskContext].barrier() - writeUTF(BarrierTaskContextMessageProtocol.BARRIER_RESULT_SUCCESS, out) + var result: String = "" + requestMethod match { + case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION => + context.asInstanceOf[BarrierTaskContext].barrier() + result = BarrierTaskContextMessageProtocol.BARRIER_RESULT_SUCCESS + case BarrierTaskContextMessageProtocol.ALL_GATHER_FUNCTION => + val messages: ArrayBuffer[String] = context.asInstanceOf[BarrierTaskContext].allGather( + message + ) + result = compact(render(JArray( + messages.map( + (message) => JString(message) + ).toList + ))) + } + writeUTF(result, out) } catch { case e: SparkException => writeUTF(e.getMessage, out) @@ -638,6 +664,7 @@ private[spark] object SpecialLengths { private[spark] object BarrierTaskContextMessageProtocol { val BARRIER_FUNCTION = 1 + val ALL_GATHER_FUNCTION = 2 val BARRIER_RESULT_SUCCESS = "success" val ERROR_UNRECOGNIZED_FUNCTION = "Not recognized function call from python side." } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index fc8ac38479932..ed38b7f0ecac1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.io.File +import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark._ @@ -52,6 +53,79 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { assert(times.max - times.min <= 1000) } + test("share messages with allGather() call") { + val conf = new SparkConf() + .setMaster("local-cluster[4, 1, 1024]") + .setAppName("test-cluster") + sc = new SparkContext(conf) + val rdd = sc.makeRDD(1 to 10, 4) + val rdd2 = rdd.barrier().mapPartitions { it => + val context = BarrierTaskContext.get() + // Sleep for a random time before global sync. + Thread.sleep(Random.nextInt(1000)) + // Pass partitionId message in + val message = context.partitionId().toString + val messages = context.allGather(message) + messages.toList.iterator + } + // Take a sorted list of all the partitionId messages + val messages = rdd2.collect().head + // All the task partitionIds are shared + for((x, i) <- messages.view.zipWithIndex) assert(x == i.toString) + } + + test("throw exception if we attempt to synchronize with different blocking calls") { + val conf = new SparkConf() + .setMaster("local-cluster[4, 1, 1024]") + .setAppName("test-cluster") + sc = new SparkContext(conf) + val rdd = sc.makeRDD(1 to 10, 4) + val rdd2 = rdd.barrier().mapPartitions { it => + val context = BarrierTaskContext.get() + val partitionId = context.partitionId + if (partitionId == 0) { + context.barrier() + } else { + context.allGather(partitionId.toString) + } + Seq(null).iterator + } + val error = intercept[SparkException] { + rdd2.collect() + }.getMessage + assert(error.contains("does not match the current synchronized requestMethod")) + } + + test("successively sync with allGather and barrier") { + val conf = new SparkConf() + .setMaster("local-cluster[4, 1, 1024]") + .setAppName("test-cluster") + sc = new SparkContext(conf) + val rdd = sc.makeRDD(1 to 10, 4) + val rdd2 = rdd.barrier().mapPartitions { it => + val context = BarrierTaskContext.get() + // Sleep for a random time before global sync. + Thread.sleep(Random.nextInt(1000)) + context.barrier() + val time1 = System.currentTimeMillis() + // Sleep for a random time before global sync. + Thread.sleep(Random.nextInt(1000)) + // Pass partitionId message in + val message = context.partitionId().toString + val messages = context.allGather(message) + val time2 = System.currentTimeMillis() + Seq((time1, time2)).iterator + } + val times = rdd2.collect() + // All the tasks shall finish the first round of global sync within a short time slot. + val times1 = times.map(_._1) + assert(times1.max - times1.min <= 1000) + + // All the tasks shall finish the second round of global sync within a short time slot. + val times2 = times.map(_._2) + assert(times2.max - times2.min <= 1000) + } + test("support multiple barrier() call within a single task") { initLocalClusterSparkContext() val rdd = sc.makeRDD(1 to 10, 4) diff --git a/python/pyspark/taskcontext.py b/python/pyspark/taskcontext.py index d648f63338514..90bd2345ac525 100644 --- a/python/pyspark/taskcontext.py +++ b/python/pyspark/taskcontext.py @@ -16,9 +16,10 @@ # from __future__ import print_function +import json from pyspark.java_gateway import local_connect_and_auth -from pyspark.serializers import write_int, UTF8Deserializer +from pyspark.serializers import write_int, write_with_length, UTF8Deserializer class TaskContext(object): @@ -107,18 +108,28 @@ def resources(self): BARRIER_FUNCTION = 1 +ALL_GATHER_FUNCTION = 2 -def _load_from_socket(port, auth_secret): +def _load_from_socket(port, auth_secret, function, all_gather_message=None): """ Load data from a given socket, this is a blocking method thus only return when the socket connection has been closed. """ (sockfile, sock) = local_connect_and_auth(port, auth_secret) - # The barrier() call may block forever, so no timeout + + # The call may block forever, so no timeout sock.settimeout(None) - # Make a barrier() function call. - write_int(BARRIER_FUNCTION, sockfile) + + if function == BARRIER_FUNCTION: + # Make a barrier() function call. + write_int(function, sockfile) + elif function == ALL_GATHER_FUNCTION: + # Make a all_gather() function call. + write_int(function, sockfile) + write_with_length(all_gather_message.encode("utf-8"), sockfile) + else: + raise ValueError("Unrecognized function type") sockfile.flush() # Collect result. @@ -199,7 +210,33 @@ def barrier(self): raise Exception("Not supported to call barrier() before initialize " + "BarrierTaskContext.") else: - _load_from_socket(self._port, self._secret) + _load_from_socket(self._port, self._secret, BARRIER_FUNCTION) + + def allGather(self, message=""): + """ + .. note:: Experimental + + This function blocks until all tasks in the same stage have reached this routine. + Each task passes in a message and returns with a list of all the messages passed in + by each of those tasks. + + .. warning:: In a barrier stage, each task much have the same number of `allGather()` + calls, in all possible code branches. + Otherwise, you may get the job hanging or a SparkException after timeout. + """ + if not isinstance(message, str): + raise ValueError("Argument `message` must be of type `str`") + elif self._port is None or self._secret is None: + raise Exception("Not supported to call barrier() before initialize " + + "BarrierTaskContext.") + else: + gathered_items = _load_from_socket( + self._port, + self._secret, + ALL_GATHER_FUNCTION, + message, + ) + return [e for e in json.loads(gathered_items)] def getTaskInfos(self): """ diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index 68cfe814762e0..0053aadd9c3ed 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -135,6 +135,26 @@ def context_barrier(x): times = rdd.barrier().mapPartitions(f).map(context_barrier).collect() self.assertTrue(max(times) - min(times) < 1) + def test_all_gather(self): + """ + Verify that BarrierTaskContext.allGather() performs global sync among all barrier tasks + within a stage and passes messages properly. + """ + rdd = self.sc.parallelize(range(10), 4) + + def f(iterator): + yield sum(iterator) + + def context_barrier(x): + tc = BarrierTaskContext.get() + time.sleep(random.randint(1, 10)) + out = tc.allGather(str(context.partitionId())) + pids = [int(e) for e in out] + return [pids] + + pids = rdd.barrier().mapPartitions(f).map(context_barrier).collect()[0] + self.assertTrue(pids == [0, 1, 2, 3]) + def test_barrier_infos(self): """ Verify that BarrierTaskContext.getTaskInfos() returns a list of all task infos in the From fa3517cdb163b0589dc02c7d1fefb65be811f65f Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Thu, 13 Feb 2020 17:43:55 -0800 Subject: [PATCH 1373/1387] Revert "[SPARK-30667][CORE] Add allGather method to BarrierTaskContext" This reverts commit 57254c9719f9af9ad985596ed7fbbaafa4052002. --- .../org/apache/spark/BarrierCoordinator.scala | 113 ++----------- .../org/apache/spark/BarrierTaskContext.scala | 153 ++++++------------ .../spark/api/python/PythonRunner.scala | 51 ++---- .../scheduler/BarrierTaskContextSuite.scala | 74 --------- python/pyspark/taskcontext.py | 49 +----- python/pyspark/tests/test_taskcontext.py | 20 --- 6 files changed, 79 insertions(+), 381 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala index 042a2664a0e27..4e417679ca663 100644 --- a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala @@ -17,17 +17,12 @@ package org.apache.spark -import java.nio.charset.StandardCharsets.UTF_8 import java.util.{Timer, TimerTask} import java.util.concurrent.ConcurrentHashMap import java.util.function.Consumer import scala.collection.mutable.ArrayBuffer -import org.json4s.JsonAST._ -import org.json4s.JsonDSL._ -import org.json4s.jackson.JsonMethods.{compact, render} - import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerStageCompleted} @@ -104,15 +99,10 @@ private[spark] class BarrierCoordinator( // reset when a barrier() call fails due to timeout. private var barrierEpoch: Int = 0 - // An Array of RPCCallContexts for barrier tasks that have made a blocking runBarrier() call + // An array of RPCCallContexts for barrier tasks that are waiting for reply of a barrier() + // call. private val requesters: ArrayBuffer[RpcCallContext] = new ArrayBuffer[RpcCallContext](numTasks) - // An Array of allGather messages for barrier tasks that have made a blocking runBarrier() call - private val allGatherMessages: ArrayBuffer[String] = new Array[String](numTasks).to[ArrayBuffer] - - // The blocking requestMethod called by tasks to sync up for this stage attempt - private var requestMethodToSync: RequestMethod.Value = RequestMethod.BARRIER - // A timer task that ensures we may timeout for a barrier() call. private var timerTask: TimerTask = null @@ -140,32 +130,9 @@ private[spark] class BarrierCoordinator( // Process the global sync request. The barrier() call succeed if collected enough requests // within a configured time, otherwise fail all the pending requests. - def handleRequest( - requester: RpcCallContext, - request: RequestToSync - ): Unit = synchronized { + def handleRequest(requester: RpcCallContext, request: RequestToSync): Unit = synchronized { val taskId = request.taskAttemptId val epoch = request.barrierEpoch - val requestMethod = request.requestMethod - val partitionId = request.partitionId - val allGatherMessage = request match { - case ag: AllGatherRequestToSync => ag.allGatherMessage - case _ => "" - } - - if (requesters.size == 0) { - requestMethodToSync = requestMethod - } - - if (requestMethodToSync != requestMethod) { - requesters.foreach( - _.sendFailure(new SparkException(s"$barrierId tried to use requestMethod " + - s"`$requestMethod` during barrier epoch $barrierEpoch, which does not match " + - s"the current synchronized requestMethod `$requestMethodToSync`" - )) - ) - cleanupBarrierStage(barrierId) - } // Require the number of tasks is correctly set from the BarrierTaskContext. require(request.numTasks == numTasks, s"Number of tasks of $barrierId is " + @@ -186,7 +153,6 @@ private[spark] class BarrierCoordinator( } // Add the requester to array of RPCCallContexts pending for reply. requesters += requester - allGatherMessages(partitionId) = allGatherMessage logInfo(s"Barrier sync epoch $barrierEpoch from $barrierId received update from Task " + s"$taskId, current progress: ${requesters.size}/$numTasks.") if (maybeFinishAllRequesters(requesters, numTasks)) { @@ -196,7 +162,6 @@ private[spark] class BarrierCoordinator( s"tasks, finished successfully.") barrierEpoch += 1 requesters.clear() - allGatherMessages.clear() cancelTimerTask() } } @@ -208,13 +173,7 @@ private[spark] class BarrierCoordinator( requesters: ArrayBuffer[RpcCallContext], numTasks: Int): Boolean = { if (requesters.size == numTasks) { - requestMethodToSync match { - case RequestMethod.BARRIER => - requesters.foreach(_.reply("")) - case RequestMethod.ALL_GATHER => - val json: String = compact(render(allGatherMessages)) - requesters.foreach(_.reply(json)) - } + requesters.foreach(_.reply(())) true } else { false @@ -227,7 +186,6 @@ private[spark] class BarrierCoordinator( // messages come from current stage attempt shall fail. barrierEpoch = -1 requesters.clear() - allGatherMessages.clear() cancelTimerTask() } } @@ -241,11 +199,11 @@ private[spark] class BarrierCoordinator( } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case request: RequestToSync => + case request @ RequestToSync(numTasks, stageId, stageAttemptId, _, _) => // Get or init the ContextBarrierState correspond to the stage attempt. - val barrierId = ContextBarrierId(request.stageId, request.stageAttemptId) + val barrierId = ContextBarrierId(stageId, stageAttemptId) states.computeIfAbsent(barrierId, - (key: ContextBarrierId) => new ContextBarrierState(key, request.numTasks)) + (key: ContextBarrierId) => new ContextBarrierState(key, numTasks)) val barrierState = states.get(barrierId) barrierState.handleRequest(context, request) @@ -258,16 +216,6 @@ private[spark] class BarrierCoordinator( private[spark] sealed trait BarrierCoordinatorMessage extends Serializable -private[spark] sealed trait RequestToSync extends BarrierCoordinatorMessage { - def numTasks: Int - def stageId: Int - def stageAttemptId: Int - def taskAttemptId: Long - def barrierEpoch: Int - def partitionId: Int - def requestMethod: RequestMethod.Value -} - /** * A global sync request message from BarrierTaskContext, by `barrier()` call. Each request is * identified by stageId + stageAttemptId + barrierEpoch. @@ -276,44 +224,11 @@ private[spark] sealed trait RequestToSync extends BarrierCoordinatorMessage { * @param stageId ID of current stage * @param stageAttemptId ID of current stage attempt * @param taskAttemptId Unique ID of current task - * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls - * @param partitionId ID of the current partition the task is assigned to - * @param requestMethod The BarrierTaskContext method that was called to trigger BarrierCoordinator + * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls. */ -private[spark] case class BarrierRequestToSync( - numTasks: Int, - stageId: Int, - stageAttemptId: Int, - taskAttemptId: Long, - barrierEpoch: Int, - partitionId: Int, - requestMethod: RequestMethod.Value -) extends RequestToSync - -/** - * A global sync request message from BarrierTaskContext, by `allGather()` call. Each request is - * identified by stageId + stageAttemptId + barrierEpoch. - * - * @param numTasks The number of global sync requests the BarrierCoordinator shall receive - * @param stageId ID of current stage - * @param stageAttemptId ID of current stage attempt - * @param taskAttemptId Unique ID of current task - * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls - * @param partitionId ID of the current partition the task is assigned to - * @param requestMethod The BarrierTaskContext method that was called to trigger BarrierCoordinator - * @param allGatherMessage Message sent from the BarrierTaskContext if requestMethod is ALL_GATHER - */ -private[spark] case class AllGatherRequestToSync( - numTasks: Int, - stageId: Int, - stageAttemptId: Int, - taskAttemptId: Long, - barrierEpoch: Int, - partitionId: Int, - requestMethod: RequestMethod.Value, - allGatherMessage: String -) extends RequestToSync - -private[spark] object RequestMethod extends Enumeration { - val BARRIER, ALL_GATHER = Value -} +private[spark] case class RequestToSync( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + taskAttemptId: Long, + barrierEpoch: Int) extends BarrierCoordinatorMessage diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index 2263538a11676..3d369802f3023 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -17,19 +17,11 @@ package org.apache.spark -import java.nio.charset.StandardCharsets.UTF_8 import java.util.{Properties, Timer, TimerTask} import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer import scala.concurrent.TimeoutException import scala.concurrent.duration._ -import scala.language.postfixOps - -import org.json4s.DefaultFormats -import org.json4s.JsonAST._ -import org.json4s.JsonDSL._ -import org.json4s.jackson.JsonMethods.parse import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.executor.TaskMetrics @@ -67,31 +59,49 @@ class BarrierTaskContext private[spark] ( // from different tasks within the same barrier stage attempt to succeed. private lazy val numTasks = getTaskInfos().size - private def getRequestToSync( - numTasks: Int, - stageId: Int, - stageAttemptNumber: Int, - taskAttemptId: Long, - barrierEpoch: Int, - partitionId: Int, - requestMethod: RequestMethod.Value, - allGatherMessage: String - ): RequestToSync = { - requestMethod match { - case RequestMethod.BARRIER => - BarrierRequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, - barrierEpoch, partitionId, requestMethod) - case RequestMethod.ALL_GATHER => - AllGatherRequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, - barrierEpoch, partitionId, requestMethod, allGatherMessage) - } - } - - private def runBarrier( - requestMethod: RequestMethod.Value, - allGatherMessage: String = "" - ): String = { - + /** + * :: Experimental :: + * Sets a global barrier and waits until all tasks in this stage hit this barrier. Similar to + * MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same + * stage have reached this routine. + * + * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all + * possible code branches. Otherwise, you may get the job hanging or a SparkException after + * timeout. Some examples of '''misuses''' are listed below: + * 1. Only call barrier() function on a subset of all the tasks in the same barrier stage, it + * shall lead to timeout of the function call. + * {{{ + * rdd.barrier().mapPartitions { iter => + * val context = BarrierTaskContext.get() + * if (context.partitionId() == 0) { + * // Do nothing. + * } else { + * context.barrier() + * } + * iter + * } + * }}} + * + * 2. Include barrier() function in a try-catch code block, this may lead to timeout of the + * second function call. + * {{{ + * rdd.barrier().mapPartitions { iter => + * val context = BarrierTaskContext.get() + * try { + * // Do something that might throw an Exception. + * doSomething() + * context.barrier() + * } catch { + * case e: Exception => logWarning("...", e) + * } + * context.barrier() + * iter + * } + * }}} + */ + @Experimental + @Since("2.4.0") + def barrier(): Unit = { logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) has entered " + s"the global sync, current barrier epoch is $barrierEpoch.") logTrace("Current callSite: " + Utils.getCallSite()) @@ -108,12 +118,10 @@ class BarrierTaskContext private[spark] ( // Log the update of global sync every 60 seconds. timer.schedule(timerTask, 60000, 60000) - var json: String = "" - try { - val abortableRpcFuture = barrierCoordinator.askAbortable[String]( - message = getRequestToSync(numTasks, stageId, stageAttemptNumber, - taskAttemptId, barrierEpoch, partitionId, requestMethod, allGatherMessage), + val abortableRpcFuture = barrierCoordinator.askAbortable[Unit]( + message = RequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, + barrierEpoch), // Set a fixed timeout for RPC here, so users shall get a SparkException thrown by // BarrierCoordinator on timeout, instead of RPCTimeoutException from the RPC framework. timeout = new RpcTimeout(365.days, "barrierTimeout")) @@ -125,7 +133,7 @@ class BarrierTaskContext private[spark] ( while (!abortableRpcFuture.toFuture.isCompleted) { // wait RPC future for at most 1 second try { - json = ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second) + ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second) } catch { case _: TimeoutException | _: InterruptedException => // If `TimeoutException` thrown, waiting RPC future reach 1 second. @@ -155,73 +163,6 @@ class BarrierTaskContext private[spark] ( timerTask.cancel() timer.purge() } - json - } - - /** - * :: Experimental :: - * Sets a global barrier and waits until all tasks in this stage hit this barrier. Similar to - * MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same - * stage have reached this routine. - * - * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all - * possible code branches. Otherwise, you may get the job hanging or a SparkException after - * timeout. Some examples of '''misuses''' are listed below: - * 1. Only call barrier() function on a subset of all the tasks in the same barrier stage, it - * shall lead to timeout of the function call. - * {{{ - * rdd.barrier().mapPartitions { iter => - * val context = BarrierTaskContext.get() - * if (context.partitionId() == 0) { - * // Do nothing. - * } else { - * context.barrier() - * } - * iter - * } - * }}} - * - * 2. Include barrier() function in a try-catch code block, this may lead to timeout of the - * second function call. - * {{{ - * rdd.barrier().mapPartitions { iter => - * val context = BarrierTaskContext.get() - * try { - * // Do something that might throw an Exception. - * doSomething() - * context.barrier() - * } catch { - * case e: Exception => logWarning("...", e) - * } - * context.barrier() - * iter - * } - * }}} - */ - @Experimental - @Since("2.4.0") - def barrier(): Unit = { - runBarrier(RequestMethod.BARRIER) - () - } - - /** - * :: Experimental :: - * Blocks until all tasks in the same stage have reached this routine. Each task passes in - * a message and returns with a list of all the messages passed in by each of those tasks. - * - * CAUTION! The allGather method requires the same precautions as the barrier method - * - * The message is type String rather than Array[Byte] because it is more convenient for - * the user at the cost of worse performance. - */ - @Experimental - @Since("3.0.0") - def allGather(message: String): ArrayBuffer[String] = { - val json = runBarrier(RequestMethod.ALL_GATHER, message) - val jsonArray = parse(json) - implicit val formats = DefaultFormats - ArrayBuffer(jsonArray.extract[Array[String]]: _*) } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index fa8bf0fc06358..658e0d593a167 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -24,13 +24,8 @@ import java.nio.charset.StandardCharsets.UTF_8 import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal -import org.json4s.JsonAST._ -import org.json4s.JsonDSL._ -import org.json4s.jackson.JsonMethods.{compact, render} - import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} @@ -243,18 +238,13 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( sock.setSoTimeout(10000) authHelper.authClient(sock) val input = new DataInputStream(sock.getInputStream()) - val requestMethod = input.readInt() - // The BarrierTaskContext function may wait infinitely, socket shall not timeout - // before the function finishes. - sock.setSoTimeout(0) - requestMethod match { + input.readInt() match { case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION => - barrierAndServe(requestMethod, sock) - case BarrierTaskContextMessageProtocol.ALL_GATHER_FUNCTION => - val length = input.readInt() - val message = new Array[Byte](length) - input.readFully(message) - barrierAndServe(requestMethod, sock, new String(message, UTF_8)) + // The barrier() function may wait infinitely, socket shall not timeout + // before the function finishes. + sock.setSoTimeout(0) + barrierAndServe(sock) + case _ => val out = new DataOutputStream(new BufferedOutputStream( sock.getOutputStream)) @@ -405,31 +395,15 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( } /** - * Gateway to call BarrierTaskContext methods. + * Gateway to call BarrierTaskContext.barrier(). */ - def barrierAndServe(requestMethod: Int, sock: Socket, message: String = ""): Unit = { - require( - serverSocket.isDefined, - "No available ServerSocket to redirect the BarrierTaskContext method call." - ) + def barrierAndServe(sock: Socket): Unit = { + require(serverSocket.isDefined, "No available ServerSocket to redirect the barrier() call.") + val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) try { - var result: String = "" - requestMethod match { - case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION => - context.asInstanceOf[BarrierTaskContext].barrier() - result = BarrierTaskContextMessageProtocol.BARRIER_RESULT_SUCCESS - case BarrierTaskContextMessageProtocol.ALL_GATHER_FUNCTION => - val messages: ArrayBuffer[String] = context.asInstanceOf[BarrierTaskContext].allGather( - message - ) - result = compact(render(JArray( - messages.map( - (message) => JString(message) - ).toList - ))) - } - writeUTF(result, out) + context.asInstanceOf[BarrierTaskContext].barrier() + writeUTF(BarrierTaskContextMessageProtocol.BARRIER_RESULT_SUCCESS, out) } catch { case e: SparkException => writeUTF(e.getMessage, out) @@ -664,7 +638,6 @@ private[spark] object SpecialLengths { private[spark] object BarrierTaskContextMessageProtocol { val BARRIER_FUNCTION = 1 - val ALL_GATHER_FUNCTION = 2 val BARRIER_RESULT_SUCCESS = "success" val ERROR_UNRECOGNIZED_FUNCTION = "Not recognized function call from python side." } diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index ed38b7f0ecac1..fc8ac38479932 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import java.io.File -import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark._ @@ -53,79 +52,6 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { assert(times.max - times.min <= 1000) } - test("share messages with allGather() call") { - val conf = new SparkConf() - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) - val rdd = sc.makeRDD(1 to 10, 4) - val rdd2 = rdd.barrier().mapPartitions { it => - val context = BarrierTaskContext.get() - // Sleep for a random time before global sync. - Thread.sleep(Random.nextInt(1000)) - // Pass partitionId message in - val message = context.partitionId().toString - val messages = context.allGather(message) - messages.toList.iterator - } - // Take a sorted list of all the partitionId messages - val messages = rdd2.collect().head - // All the task partitionIds are shared - for((x, i) <- messages.view.zipWithIndex) assert(x == i.toString) - } - - test("throw exception if we attempt to synchronize with different blocking calls") { - val conf = new SparkConf() - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) - val rdd = sc.makeRDD(1 to 10, 4) - val rdd2 = rdd.barrier().mapPartitions { it => - val context = BarrierTaskContext.get() - val partitionId = context.partitionId - if (partitionId == 0) { - context.barrier() - } else { - context.allGather(partitionId.toString) - } - Seq(null).iterator - } - val error = intercept[SparkException] { - rdd2.collect() - }.getMessage - assert(error.contains("does not match the current synchronized requestMethod")) - } - - test("successively sync with allGather and barrier") { - val conf = new SparkConf() - .setMaster("local-cluster[4, 1, 1024]") - .setAppName("test-cluster") - sc = new SparkContext(conf) - val rdd = sc.makeRDD(1 to 10, 4) - val rdd2 = rdd.barrier().mapPartitions { it => - val context = BarrierTaskContext.get() - // Sleep for a random time before global sync. - Thread.sleep(Random.nextInt(1000)) - context.barrier() - val time1 = System.currentTimeMillis() - // Sleep for a random time before global sync. - Thread.sleep(Random.nextInt(1000)) - // Pass partitionId message in - val message = context.partitionId().toString - val messages = context.allGather(message) - val time2 = System.currentTimeMillis() - Seq((time1, time2)).iterator - } - val times = rdd2.collect() - // All the tasks shall finish the first round of global sync within a short time slot. - val times1 = times.map(_._1) - assert(times1.max - times1.min <= 1000) - - // All the tasks shall finish the second round of global sync within a short time slot. - val times2 = times.map(_._2) - assert(times2.max - times2.min <= 1000) - } - test("support multiple barrier() call within a single task") { initLocalClusterSparkContext() val rdd = sc.makeRDD(1 to 10, 4) diff --git a/python/pyspark/taskcontext.py b/python/pyspark/taskcontext.py index 90bd2345ac525..d648f63338514 100644 --- a/python/pyspark/taskcontext.py +++ b/python/pyspark/taskcontext.py @@ -16,10 +16,9 @@ # from __future__ import print_function -import json from pyspark.java_gateway import local_connect_and_auth -from pyspark.serializers import write_int, write_with_length, UTF8Deserializer +from pyspark.serializers import write_int, UTF8Deserializer class TaskContext(object): @@ -108,28 +107,18 @@ def resources(self): BARRIER_FUNCTION = 1 -ALL_GATHER_FUNCTION = 2 -def _load_from_socket(port, auth_secret, function, all_gather_message=None): +def _load_from_socket(port, auth_secret): """ Load data from a given socket, this is a blocking method thus only return when the socket connection has been closed. """ (sockfile, sock) = local_connect_and_auth(port, auth_secret) - - # The call may block forever, so no timeout + # The barrier() call may block forever, so no timeout sock.settimeout(None) - - if function == BARRIER_FUNCTION: - # Make a barrier() function call. - write_int(function, sockfile) - elif function == ALL_GATHER_FUNCTION: - # Make a all_gather() function call. - write_int(function, sockfile) - write_with_length(all_gather_message.encode("utf-8"), sockfile) - else: - raise ValueError("Unrecognized function type") + # Make a barrier() function call. + write_int(BARRIER_FUNCTION, sockfile) sockfile.flush() # Collect result. @@ -210,33 +199,7 @@ def barrier(self): raise Exception("Not supported to call barrier() before initialize " + "BarrierTaskContext.") else: - _load_from_socket(self._port, self._secret, BARRIER_FUNCTION) - - def allGather(self, message=""): - """ - .. note:: Experimental - - This function blocks until all tasks in the same stage have reached this routine. - Each task passes in a message and returns with a list of all the messages passed in - by each of those tasks. - - .. warning:: In a barrier stage, each task much have the same number of `allGather()` - calls, in all possible code branches. - Otherwise, you may get the job hanging or a SparkException after timeout. - """ - if not isinstance(message, str): - raise ValueError("Argument `message` must be of type `str`") - elif self._port is None or self._secret is None: - raise Exception("Not supported to call barrier() before initialize " + - "BarrierTaskContext.") - else: - gathered_items = _load_from_socket( - self._port, - self._secret, - ALL_GATHER_FUNCTION, - message, - ) - return [e for e in json.loads(gathered_items)] + _load_from_socket(self._port, self._secret) def getTaskInfos(self): """ diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index 0053aadd9c3ed..68cfe814762e0 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -135,26 +135,6 @@ def context_barrier(x): times = rdd.barrier().mapPartitions(f).map(context_barrier).collect() self.assertTrue(max(times) - min(times) < 1) - def test_all_gather(self): - """ - Verify that BarrierTaskContext.allGather() performs global sync among all barrier tasks - within a stage and passes messages properly. - """ - rdd = self.sc.parallelize(range(10), 4) - - def f(iterator): - yield sum(iterator) - - def context_barrier(x): - tc = BarrierTaskContext.get() - time.sleep(random.randint(1, 10)) - out = tc.allGather(str(context.partitionId())) - pids = [int(e) for e in out] - return [pids] - - pids = rdd.barrier().mapPartitions(f).map(context_barrier).collect()[0] - self.assertTrue(pids == [0, 1, 2, 3]) - def test_barrier_infos(self): """ Verify that BarrierTaskContext.getTaskInfos() returns a list of all task infos in the From 25db8c71a2100c167b8c2d7a6c540ebc61db9b73 Mon Sep 17 00:00:00 2001 From: David Toneian Date: Fri, 14 Feb 2020 11:00:35 +0900 Subject: [PATCH 1374/1387] [PYSPARK][DOCS][MINOR] Changed `:func:` to `:attr:` Sphinx roles, fixed links in documentation of `Data{Frame,Stream}{Reader,Writer}` This commit is published into the public domain. ### What changes were proposed in this pull request? This PR fixes the documentation of `DataFrameReader`, `DataFrameWriter`, `DataStreamReader`, and `DataStreamWriter`, where attributes of other classes were misrepresented as functions. Additionally, creation of hyperlinks across modules was fixed in these instances. ### Why are the changes needed? The old state produced documentation that suggested invalid usage of PySpark objects (accessing attributes as though they were callable.) ### Does this PR introduce any user-facing change? No, except for improved documentation. ### How was this patch tested? No test added; documentation build runs through. Closes #27553 from DavidToneian/docfix-DataFrameReader-DataFrameWriter-DataStreamReader-DataStreamWriter. Authored-by: David Toneian Signed-off-by: HyukjinKwon --- python/pyspark/sql/readwriter.py | 4 ++-- python/pyspark/sql/streaming.py | 11 ++++++----- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 3d3280dbd9943..69660395ad823 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -48,7 +48,7 @@ def _set_opts(self, schema=None, **options): class DataFrameReader(OptionUtils): """ Interface used to load a :class:`DataFrame` from external storage systems - (e.g. file systems, key-value stores, etc). Use :func:`spark.read` + (e.g. file systems, key-value stores, etc). Use :attr:`SparkSession.read` to access this. .. versionadded:: 1.4 @@ -616,7 +616,7 @@ def jdbc(self, url, table, column=None, lowerBound=None, upperBound=None, numPar class DataFrameWriter(OptionUtils): """ Interface used to write a :class:`DataFrame` to external storage systems - (e.g. file systems, key-value stores, etc). Use :func:`DataFrame.write` + (e.g. file systems, key-value stores, etc). Use :attr:`DataFrame.write` to access this. .. versionadded:: 1.4 diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index f17a52f6b3dc8..5fced8aca9bdf 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -276,9 +276,9 @@ def resetTerminated(self): class DataStreamReader(OptionUtils): """ - Interface used to load a streaming :class:`DataFrame` from external storage systems - (e.g. file systems, key-value stores, etc). Use :func:`spark.readStream` - to access this. + Interface used to load a streaming :class:`DataFrame ` from external + storage systems (e.g. file systems, key-value stores, etc). + Use :attr:`SparkSession.readStream ` to access this. .. note:: Evolving. @@ -750,8 +750,9 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non class DataStreamWriter(object): """ - Interface used to write a streaming :class:`DataFrame` to external storage systems - (e.g. file systems, key-value stores, etc). Use :func:`DataFrame.writeStream` + Interface used to write a streaming :class:`DataFrame ` to external + storage systems (e.g. file systems, key-value stores, etc). + Use :attr:`DataFrame.writeStream ` to access this. .. note:: Evolving. From 0aed77a0155b404e39bc5dbc0579e29e4c7bf887 Mon Sep 17 00:00:00 2001 From: maryannxue Date: Fri, 14 Feb 2020 11:20:55 +0800 Subject: [PATCH 1375/1387] [SPARK-30801][SQL] Subqueries should not be AQE-ed if main query is not ### What changes were proposed in this pull request? This PR makes sure AQE is either enabled or disabled for the entire query, including the main query and all subqueries. Currently there are unsupported queries by AQE, e.g., queries that contain DPP filters. We need to make sure that if the main query is unsupported, none of the sub-queries should apply AQE, otherwise it can lead to performance regressions due to missed opportunity of sub-query reuse. ### Why are the changes needed? To get rid of potential perf regressions when AQE is turned on. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Updated DynamicPartitionPruningSuite: 1. Removed the existing workaround `withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false")` 2. Added `DynamicPartitionPruningSuiteAEOn` and `DynamicPartitionPruningSuiteAEOff` to enable testing this suite with AQE on and off options 3. Added a check in `checkPartitionPruningPredicate` to verify that the subqueries are always in sync with the main query in terms of whether AQE is applied. Closes #27554 from maryannxue/spark-30801. Authored-by: maryannxue Signed-off-by: Wenchen Fan --- .../spark/sql/execution/QueryExecution.scala | 19 ++++++++++--- .../sql/DynamicPartitionPruningSuite.scala | 27 ++++++++++++++++--- 2 files changed, 39 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 53b6b5d82c021..9109c05e75853 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -274,13 +274,25 @@ object QueryExecution { * are correct, insert whole stage code gen, and try to reduce the work done by reusing exchanges * and subqueries. */ - private[execution] def preparations(sparkSession: SparkSession): Seq[Rule[SparkPlan]] = + private[execution] def preparations(sparkSession: SparkSession): Seq[Rule[SparkPlan]] = { + + val sparkSessionWithAdaptiveExecutionOff = + if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { + val session = sparkSession.cloneSession() + session.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, false) + session + } else { + sparkSession + } + Seq( // `AdaptiveSparkPlanExec` is a leaf node. If inserted, all the following rules will be no-op // as the original plan is hidden behind `AdaptiveSparkPlanExec`. InsertAdaptiveSparkPlan(AdaptiveExecutionContext(sparkSession)), - PlanDynamicPruningFilters(sparkSession), - PlanSubqueries(sparkSession), + // If the following rules apply, it means the main query is not AQE-ed, so we make sure the + // subqueries are not AQE-ed either. + PlanDynamicPruningFilters(sparkSessionWithAdaptiveExecutionOff), + PlanSubqueries(sparkSessionWithAdaptiveExecutionOff), EnsureRequirements(sparkSession.sessionState.conf), ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, sparkSession.sessionState.columnarRules), @@ -288,6 +300,7 @@ object QueryExecution { ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf) ) + } /** * Prepares a planned [[SparkPlan]] for execution by inserting shuffle operations and internal diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index f7b51d6f4c8ef..baa9f5ecafc68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.plans.ExistenceJoin import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec} import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} @@ -33,7 +33,7 @@ import org.apache.spark.sql.test.SharedSparkSession /** * Test suite for the filtering ratio policy used to trigger dynamic partition pruning (DPP). */ -class DynamicPartitionPruningSuite +abstract class DynamicPartitionPruningSuiteBase extends QueryTest with SharedSparkSession with GivenWhenThen @@ -43,9 +43,14 @@ class DynamicPartitionPruningSuite import testImplicits._ + val adaptiveExecutionOn: Boolean + override def beforeAll(): Unit = { super.beforeAll() + spark.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED, adaptiveExecutionOn) + spark.sessionState.conf.setConf(SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY, true) + val factData = Seq[(Int, Int, Int, Int)]( (1000, 1, 1, 10), (1010, 2, 1, 10), @@ -153,6 +158,8 @@ class DynamicPartitionPruningSuite sql("DROP TABLE IF EXISTS fact_stats") sql("DROP TABLE IF EXISTS dim_stats") } finally { + spark.sessionState.conf.unsetConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED) + spark.sessionState.conf.unsetConf(SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY) super.afterAll() } } @@ -195,6 +202,11 @@ class DynamicPartitionPruningSuite fail(s"Invalid child node found in\n$s") } } + + val isMainQueryAdaptive = plan.isInstanceOf[AdaptiveSparkPlanExec] + subqueriesAll(plan).filterNot(subqueryBroadcast.contains).foreach { s => + assert(s.find(_.isInstanceOf[AdaptiveSparkPlanExec]).isDefined == isMainQueryAdaptive) + } } /** @@ -1173,8 +1185,7 @@ class DynamicPartitionPruningSuite } test("join key with multiple references on the filtering plan") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { // when enable AQE, the reusedExchange is inserted when executed. withTable("fact", "dim") { spark.range(100).select( @@ -1270,3 +1281,11 @@ class DynamicPartitionPruningSuite } } } + +class DynamicPartitionPruningSuiteAEOff extends DynamicPartitionPruningSuiteBase { + override val adaptiveExecutionOn: Boolean = false +} + +class DynamicPartitionPruningSuiteAEOn extends DynamicPartitionPruningSuiteBase { + override val adaptiveExecutionOn: Boolean = true +} From b2134ee73cfad4d78aaf8f0a2898011ac0308e48 Mon Sep 17 00:00:00 2001 From: David Toneian Date: Fri, 14 Feb 2020 13:49:11 +0900 Subject: [PATCH 1376/1387] [SPARK-30823][PYTHON][DOCS] Set `%PYTHONPATH%` when building PySpark documentation on Windows MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit is published into the public domain. ### What changes were proposed in this pull request? In analogy to `python/docs/Makefile`, which has > export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.8.1-src.zip) on line 10, this PR adds > set PYTHONPATH=..;..\lib\py4j-0.10.8.1-src.zip to `make2.bat`. Since there is no `realpath` in default installations of Windows, I left the relative paths unresolved. Per the instructions on how to build docs, `make.bat` is supposed to be run from `python/docs` as the working directory, so this should probably not cause issues (`%BUILDDIR%` is a relative path as well.) ### Why are the changes needed? When building the PySpark documentation on Windows, by changing directory to `python/docs` and running `make.bat` (which runs `make2.bat`), the majority of the documentation may not be built if pyspark is not in the default `%PYTHONPATH%`. Sphinx then reports that `pyspark` (and possibly dependencies) cannot be imported. If `pyspark` is in the default `%PYTHONPATH%`, I suppose it is that version of `pyspark` – as opposed to the version found above the `python/docs` directory – that is considered when building the documentation, which may result in documentation that does not correspond to the development version one is trying to build. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Manual tests on my Windows 10 machine. Additional tests with other environments very welcome! Closes #27569 from DavidToneian/SPARK-30823. Authored-by: David Toneian Signed-off-by: HyukjinKwon --- python/docs/make2.bat | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/docs/make2.bat b/python/docs/make2.bat index 05d22eb5cdd23..742df373166da 100644 --- a/python/docs/make2.bat +++ b/python/docs/make2.bat @@ -2,6 +2,8 @@ REM Command file for Sphinx documentation +set PYTHONPATH=..;..\lib\py4j-0.10.8.1-src.zip + if "%SPHINXBUILD%" == "" ( set SPHINXBUILD=sphinx-build From 99b8136a86030411e6bcbd312f40eb2a901ab0f0 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Fri, 14 Feb 2020 16:52:28 +0800 Subject: [PATCH 1377/1387] [SPARK-25990][SQL] ScriptTransformation should handle different data types correctly ### What changes were proposed in this pull request? We should convert Spark InternalRows to hive data via `HiveInspectors.wrapperFor`. ### Why are the changes needed? We may hit below exception without this change: ``` [info] org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1, 192.168.1.6, executor driver): java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to org.apache.hadoop.hive.common.type.HiveDecimal [info] at org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector.getPrimitiveJavaObject(JavaHiveDecimalObjectInspector.java:55) [info] at org.apache.hadoop.hive.serde2.lazy.LazyUtils.writePrimitiveUTF8(LazyUtils.java:321) [info] at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.serialize(LazySimpleSerDe.java:292) [info] at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.serializeField(LazySimpleSerDe.java:247) [info] at org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.doSerialize(LazySimpleSerDe.java:231) [info] at org.apache.hadoop.hive.serde2.AbstractEncodingAwareSerDe.serialize(AbstractEncodingAwareSerDe.java:55) [info] at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$2(ScriptTransformationExec.scala:300) [info] at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$2$adapted(ScriptTransformationExec.scala:281) [info] at scala.collection.Iterator.foreach(Iterator.scala:941) [info] at scala.collection.Iterator.foreach$(Iterator.scala:941) [info] at scala.collection.AbstractIterator.foreach(Iterator.scala:1429) [info] at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.$anonfun$run$1(ScriptTransformationExec.scala:281) [info] at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) [info] at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1932) [info] at org.apache.spark.sql.hive.execution.ScriptTransformationWriterThread.run(ScriptTransformationExec.scala:270) ``` ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Added new test. But please note that this test returns different result between Hive1.2 and Hive2.3 due to `HiveDecimal` or `SerDe` difference(don't know the root cause yet). Closes #27556 from Ngone51/script_transform. Lead-authored-by: yi.wu Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../execution/ScriptTransformationExec.scala | 32 ++++++++----- sql/hive/src/test/resources/test_script.py | 21 +++++++++ .../execution/ScriptTransformationSuite.scala | 46 ++++++++++++++++++- 3 files changed, 85 insertions(+), 14 deletions(-) create mode 100644 sql/hive/src/test/resources/test_script.py diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala index e12f663304e7a..40f7b4e8db7c5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala @@ -94,9 +94,8 @@ case class ScriptTransformationExec( // This new thread will consume the ScriptTransformation's input rows and write them to the // external process. That process's output will be read by this current thread. val writerThread = new ScriptTransformationWriterThread( - inputIterator, + inputIterator.map(outputProjection), input.map(_.dataType), - outputProjection, inputSerde, inputSoi, ioschema, @@ -249,16 +248,15 @@ case class ScriptTransformationExec( private class ScriptTransformationWriterThread( iter: Iterator[InternalRow], inputSchema: Seq[DataType], - outputProjection: Projection, @Nullable inputSerde: AbstractSerDe, - @Nullable inputSoi: ObjectInspector, + @Nullable inputSoi: StructObjectInspector, ioschema: HiveScriptIOSchema, outputStream: OutputStream, proc: Process, stderrBuffer: CircularBuffer, taskContext: TaskContext, conf: Configuration - ) extends Thread("Thread-ScriptTransformation-Feed") with Logging { + ) extends Thread("Thread-ScriptTransformation-Feed") with HiveInspectors with Logging { setDaemon(true) @@ -278,8 +276,8 @@ private class ScriptTransformationWriterThread( var threwException: Boolean = true val len = inputSchema.length try { - iter.map(outputProjection).foreach { row => - if (inputSerde == null) { + if (inputSerde == null) { + iter.foreach { row => val data = if (len == 0) { ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES") } else { @@ -295,10 +293,21 @@ private class ScriptTransformationWriterThread( sb.toString() } outputStream.write(data.getBytes(StandardCharsets.UTF_8)) - } else { - val writable = inputSerde.serialize( - row.asInstanceOf[GenericInternalRow].values, inputSoi) + } + } else { + // Convert Spark InternalRows to hive data via `HiveInspectors.wrapperFor`. + val hiveData = new Array[Any](inputSchema.length) + val fieldOIs = inputSoi.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray + val wrappers = fieldOIs.zip(inputSchema).map { case (f, dt) => wrapperFor(f, dt) } + + iter.foreach { row => + var i = 0 + while (i < fieldOIs.length) { + hiveData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, inputSchema(i))) + i += 1 + } + val writable = inputSerde.serialize(hiveData, inputSoi) if (scriptInputWriter != null) { scriptInputWriter.write(writable) } else { @@ -374,14 +383,13 @@ case class HiveScriptIOSchema ( val outputRowFormatMap = outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) - def initInputSerDe(input: Seq[Expression]): Option[(AbstractSerDe, ObjectInspector)] = { + def initInputSerDe(input: Seq[Expression]): Option[(AbstractSerDe, StructObjectInspector)] = { inputSerdeClass.map { serdeClass => val (columns, columnTypes) = parseAttrs(input) val serde = initSerDe(serdeClass, columns, columnTypes, inputSerdeProps) val fieldObjectInspectors = columnTypes.map(toInspector) val objectInspector = ObjectInspectorFactory .getStandardStructObjectInspector(columns.asJava, fieldObjectInspectors.asJava) - .asInstanceOf[ObjectInspector] (serde, objectInspector) } } diff --git a/sql/hive/src/test/resources/test_script.py b/sql/hive/src/test/resources/test_script.py new file mode 100644 index 0000000000000..82ef7b38f0c1b --- /dev/null +++ b/sql/hive/src/test/resources/test_script.py @@ -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. +# + +import sys +for line in sys.stdin: + (a, b, c, d, e) = line.split('\t') + sys.stdout.write('\t'.join([a, b, c, d, e])) + sys.stdout.flush() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 80a50c18bcb93..7d01fc53a4099 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import java.sql.Timestamp + import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.scalatest.Assertions._ import org.scalatest.BeforeAndAfterEach @@ -24,15 +26,18 @@ import org.scalatest.exceptions.TestFailedException import org.apache.spark.{SparkException, TaskContext, TestUtils} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode} +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StringType -class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton with - BeforeAndAfterEach { +class ScriptTransformationSuite extends SparkPlanTest with SQLTestUtils with TestHiveSingleton + with BeforeAndAfterEach { import spark.implicits._ private val noSerdeIOSchema = HiveScriptIOSchema( @@ -186,6 +191,43 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton wit rowsDf.select("name").collect()) assert(uncaughtExceptionHandler.exception.isEmpty) } + + test("SPARK-25990: TRANSFORM should handle different data types correctly") { + assume(TestUtils.testCommandAvailable("python")) + val scriptFilePath = getTestResourcePath("test_script.py") + + withTempView("v") { + val df = Seq( + (1, "1", 1.0, BigDecimal(1.0), new Timestamp(1)), + (2, "2", 2.0, BigDecimal(2.0), new Timestamp(2)), + (3, "3", 3.0, BigDecimal(3.0), new Timestamp(3)) + ).toDF("a", "b", "c", "d", "e") // Note column d's data type is Decimal(38, 18) + df.createTempView("v") + + val query = sql( + s""" + |SELECT + |TRANSFORM(a, b, c, d, e) + |USING 'python $scriptFilePath' AS (a, b, c, d, e) + |FROM v + """.stripMargin) + + // In Hive1.2, it does not do well on Decimal conversion. For example, in this case, + // it converts a decimal value's type from Decimal(38, 18) to Decimal(1, 0). So we need + // do extra cast here for Hive1.2. But in Hive2.3, it still keeps the original Decimal type. + val decimalToString: Column => Column = if (HiveUtils.isHive23) { + c => c.cast("string") + } else { + c => c.cast("decimal(1, 0)").cast("string") + } + checkAnswer(query, identity, df.select( + 'a.cast("string"), + 'b.cast("string"), + 'c.cast("string"), + decimalToString('d), + 'e.cast("string")).collect()) + } + } } private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExecNode { From 2a270a731a3b1da9a0fb036d648dd522e5c4d5ad Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 14 Feb 2020 18:20:18 +0800 Subject: [PATCH 1378/1387] [SPARK-30810][SQL] Parses and convert a CSV Dataset having different column from 'value' in csv(dataset) API ### What changes were proposed in this pull request? This PR fixes `DataFrameReader.csv(dataset: Dataset[String])` API to take a `Dataset[String]` originated from a column name different from `value`. This is a long-standing bug started from the very first place. `CSVUtils.filterCommentAndEmpty` assumed the `Dataset[String]` to be originated with `value` column. This PR changes to use the first column name in the schema. ### Why are the changes needed? For `DataFrameReader.csv(dataset: Dataset[String])` to support any `Dataset[String]` as the signature indicates. ### Does this PR introduce any user-facing change? Yes, ```scala val ds = spark.range(2).selectExpr("concat('a,b,', id) AS text").as[String] spark.read.option("header", true).option("inferSchema", true).csv(ds).show() ``` Before: ``` org.apache.spark.sql.AnalysisException: cannot resolve '`value`' given input columns: [text];; 'Filter (length(trim('value, None)) > 0) +- Project [concat(a,b,, cast(id#0L as string)) AS text#2] +- Range (0, 2, step=1, splits=Some(2)) ``` After: ``` +---+---+---+ | a| b| 0| +---+---+---+ | a| b| 1| +---+---+---+ ``` ### How was this patch tested? Unittest was added. Closes #27561 from HyukjinKwon/SPARK-30810. Authored-by: HyukjinKwon Signed-off-by: Wenchen Fan --- .../spark/sql/execution/datasources/csv/CSVUtils.scala | 7 ++++--- .../spark/sql/execution/datasources/csv/CSVSuite.scala | 7 +++++++ 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala index 21fabac472f4b..d8b52c503ad34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala @@ -33,11 +33,12 @@ object CSVUtils { // with the one below, `filterCommentAndEmpty` but execution path is different. One of them // might have to be removed in the near future if possible. import lines.sqlContext.implicits._ - val nonEmptyLines = lines.filter(length(trim($"value")) > 0) + val aliased = lines.toDF("value") + val nonEmptyLines = aliased.filter(length(trim($"value")) > 0) if (options.isCommentSet) { - nonEmptyLines.filter(!$"value".startsWith(options.comment.toString)) + nonEmptyLines.filter(!$"value".startsWith(options.comment.toString)).as[String] } else { - nonEmptyLines + nonEmptyLines.as[String] } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index b1105b4a63bba..0be0e1e3da3dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2294,6 +2294,13 @@ abstract class CSVSuite extends QueryTest with SharedSparkSession with TestCsvDa } } } + + test("SPARK-30810: parses and convert a CSV Dataset having different column from 'value'") { + val ds = spark.range(2).selectExpr("concat('a,b,', id) AS `a.text`").as[String] + val csv = spark.read.option("header", true).option("inferSchema", true).csv(ds) + assert(csv.schema.fieldNames === Seq("a", "b", "0")) + checkAnswer(csv, Row("a", "b", 1)) + } } class CSVv1Suite extends CSVSuite { From 7137a6d065edeaab97bf5bf49ffaca3d060a14fe Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 14 Feb 2020 22:16:57 +0800 Subject: [PATCH 1379/1387] [SPARK-30766][SQL] Fix the timestamp truncation to the `HOUR` and `DAY` levels ### What changes were proposed in this pull request? In the PR, I propose to use Java 8 time API in timestamp truncations to the levels of `HOUR` and `DAY`. The problem is in the usage of `timeZone.getOffset(millis)` in days/hours truncations where the combined calendar (Julian + Gregorian) is used underneath. ### Why are the changes needed? The change fix wrong truncations. For example, the following truncation to hours should print `0010-01-01 01:00:00` but it outputs wrong timestamp: ```scala Seq("0010-01-01 01:02:03.123456").toDF() .select($"value".cast("timestamp").as("ts")) .select(date_trunc("HOUR", $"ts").cast("string")) .show(false) +------------------------------------+ |CAST(date_trunc(HOUR, ts) AS STRING)| +------------------------------------+ |0010-01-01 01:30:17 | +------------------------------------+ ``` ### Does this PR introduce any user-facing change? Yes. After the changes, the result of the example above is: ```scala +------------------------------------+ |CAST(date_trunc(HOUR, ts) AS STRING)| +------------------------------------+ |0010-01-01 01:00:00 | +------------------------------------+ ``` ### How was this patch tested? - Added new test to `DateFunctionsSuite` - By `DateExpressionsSuite` and `DateTimeUtilsSuite` Closes #27512 from MaxGekk/fix-trunc-old-timestamp. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../expressions/datetimeExpressions.scala | 6 +-- .../sql/catalyst/util/DateTimeUtils.scala | 44 ++++++++++--------- .../catalyst/util/DateTimeUtilsSuite.scala | 39 ++++++++-------- .../apache/spark/sql/DateFunctionsSuite.scala | 13 ++++++ 4 files changed, 59 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index cf91489d8e6b7..adf7251256041 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1690,15 +1690,15 @@ case class TruncTimestamp( override def eval(input: InternalRow): Any = { evalHelper(input, minLevel = MIN_LEVEL_OF_TIMESTAMP_TRUNC) { (t: Any, level: Int) => - DateTimeUtils.truncTimestamp(t.asInstanceOf[Long], level, timeZone) + DateTimeUtils.truncTimestamp(t.asInstanceOf[Long], level, zoneId) } } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) codeGenHelper(ctx, ev, minLevel = MIN_LEVEL_OF_TIMESTAMP_TRUNC, true) { (date: String, fmt: String) => - s"truncTimestamp($date, $fmt, $tz);" + s"truncTimestamp($date, $fmt, $zid);" } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 5976bcbb52fd7..dcc7337116777 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -713,32 +713,34 @@ object DateTimeUtils { } } + private def truncToUnit(t: SQLTimestamp, zoneId: ZoneId, unit: ChronoUnit): SQLTimestamp = { + val truncated = microsToInstant(t).atZone(zoneId).truncatedTo(unit) + instantToMicros(truncated.toInstant) + } + /** * Returns the trunc date time from original date time and trunc level. * Trunc level should be generated using `parseTruncLevel()`, should be between 0 and 12. */ - def truncTimestamp(t: SQLTimestamp, level: Int, timeZone: TimeZone): SQLTimestamp = { - if (level == TRUNC_TO_MICROSECOND) return t - var millis = MICROSECONDS.toMillis(t) - val truncated = level match { - case TRUNC_TO_MILLISECOND => millis - case TRUNC_TO_SECOND => - millis - millis % MILLIS_PER_SECOND - case TRUNC_TO_MINUTE => - millis - millis % MILLIS_PER_MINUTE - case TRUNC_TO_HOUR => - val offset = timeZone.getOffset(millis) - millis += offset - millis - millis % MILLIS_PER_HOUR - offset - case TRUNC_TO_DAY => - val offset = timeZone.getOffset(millis) - millis += offset - millis - millis % MILLIS_PER_DAY - offset - case _ => // Try to truncate date levels - val dDays = millisToDays(millis, timeZone.toZoneId) - daysToMillis(truncDate(dDays, level), timeZone.toZoneId) + def truncTimestamp(t: SQLTimestamp, level: Int, zoneId: ZoneId): SQLTimestamp = { + level match { + case TRUNC_TO_MICROSECOND => t + case TRUNC_TO_HOUR => truncToUnit(t, zoneId, ChronoUnit.HOURS) + case TRUNC_TO_DAY => truncToUnit(t, zoneId, ChronoUnit.DAYS) + case _ => + val millis = MICROSECONDS.toMillis(t) + val truncated = level match { + case TRUNC_TO_MILLISECOND => millis + case TRUNC_TO_SECOND => + millis - millis % MILLIS_PER_SECOND + case TRUNC_TO_MINUTE => + millis - millis % MILLIS_PER_MINUTE + case _ => // Try to truncate date levels + val dDays = millisToDays(millis, zoneId) + daysToMillis(truncDate(dDays, level), zoneId) + } + truncated * MICROS_PER_MILLIS } - truncated * MICROS_PER_MILLIS } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index cd0594c775a47..ff4d8a2457922 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -499,9 +499,9 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { level: Int, expected: String, inputTS: SQLTimestamp, - timezone: TimeZone = DateTimeUtils.defaultTimeZone()): Unit = { + zoneId: ZoneId = defaultZoneId): Unit = { val truncated = - DateTimeUtils.truncTimestamp(inputTS, level, timezone) + DateTimeUtils.truncTimestamp(inputTS, level, zoneId) val expectedTS = toTimestamp(expected, defaultZoneId) assert(truncated === expectedTS.get) } @@ -539,6 +539,7 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { for (tz <- ALL_TIMEZONES) { withDefaultTimeZone(tz) { + val zid = tz.toZoneId val inputTS = DateTimeUtils.stringToTimestamp( UTF8String.fromString("2015-03-05T09:32:05.359"), defaultZoneId) val inputTS1 = DateTimeUtils.stringToTimestamp( @@ -552,23 +553,23 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { val inputTS5 = DateTimeUtils.stringToTimestamp( UTF8String.fromString("1999-03-29T01:02:03.456789"), defaultZoneId) - testTrunc(DateTimeUtils.TRUNC_TO_YEAR, "2015-01-01T00:00:00", inputTS.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_MONTH, "2015-03-01T00:00:00", inputTS.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_DAY, "2015-03-05T00:00:00", inputTS.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_HOUR, "2015-03-05T09:00:00", inputTS.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_MINUTE, "2015-03-05T09:32:00", inputTS.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_SECOND, "2015-03-05T09:32:05", inputTS.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-02T00:00:00", inputTS.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS1.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS2.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS3.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-23T00:00:00", inputTS4.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", inputTS.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", inputTS1.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", inputTS2.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_DECADE, "1990-01-01", inputTS5.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_CENTURY, "1901-01-01", inputTS5.get, tz) - testTrunc(DateTimeUtils.TRUNC_TO_MILLENNIUM, "2001-01-01", inputTS.get, tz) + testTrunc(DateTimeUtils.TRUNC_TO_YEAR, "2015-01-01T00:00:00", inputTS.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_MONTH, "2015-03-01T00:00:00", inputTS.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_DAY, "2015-03-05T00:00:00", inputTS.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_HOUR, "2015-03-05T09:00:00", inputTS.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_MINUTE, "2015-03-05T09:32:00", inputTS.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_SECOND, "2015-03-05T09:32:05", inputTS.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-02T00:00:00", inputTS.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS1.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS2.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-30T00:00:00", inputTS3.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_WEEK, "2015-03-23T00:00:00", inputTS4.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", inputTS.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", inputTS1.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", inputTS2.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_DECADE, "1990-01-01", inputTS5.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_CENTURY, "1901-01-01", inputTS5.get, zid) + testTrunc(DateTimeUtils.TRUNC_TO_MILLENNIUM, "2001-01-01", inputTS.get, zid) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 41d53c959ef99..ba45b9f9b62df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -856,4 +856,17 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { TimeZone.setDefault(defaultTz) } } + + test("SPARK-30766: date_trunc of old timestamps to hours and days") { + def checkTrunc(level: String, expected: String): Unit = { + val df = Seq("0010-01-01 01:02:03.123456") + .toDF() + .select($"value".cast("timestamp").as("ts")) + .select(date_trunc(level, $"ts").cast("string")) + checkAnswer(df, Row(expected)) + } + + checkTrunc("HOUR", "0010-01-01 01:00:00") + checkTrunc("DAY", "0010-01-01 00:00:00") + } } From b343757b1bd5d0344b82f36aa4d65ed34f840606 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Fri, 14 Feb 2020 10:18:08 -0800 Subject: [PATCH 1380/1387] [SPARK-29748][DOCS][FOLLOW-UP] Add a note that the legacy environment variable to set in both executor and driver ### What changes were proposed in this pull request? This PR address the comment at https://github.com/apache/spark/pull/26496#discussion_r379194091 and improves the migration guide to explicitly note that the legacy environment variable to set in both executor and driver. ### Why are the changes needed? To clarify this env should be set both in driver and executors. ### Does this PR introduce any user-facing change? Nope. ### How was this patch tested? I checked it via md editor. Closes #27573 from HyukjinKwon/SPARK-29748. Authored-by: HyukjinKwon Signed-off-by: Shixiong Zhu --- docs/pyspark-migration-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/pyspark-migration-guide.md b/docs/pyspark-migration-guide.md index 8ea4fec75edf8..f7f20389aa694 100644 --- a/docs/pyspark-migration-guide.md +++ b/docs/pyspark-migration-guide.md @@ -87,7 +87,7 @@ Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide. - Since Spark 3.0, `Column.getItem` is fixed such that it does not call `Column.apply`. Consequently, if `Column` is used as an argument to `getItem`, the indexing operator should be used. For example, `map_col.getItem(col('id'))` should be replaced with `map_col[col('id')]`. - - As of Spark 3.0 `Row` field names are no longer sorted alphabetically when constructing with named arguments for Python versions 3.6 and above, and the order of fields will match that as entered. To enable sorted fields by default, as in Spark 2.4, set the environment variable `PYSPARK_ROW_FIELD_SORTING_ENABLED` to "true". For Python versions less than 3.6, the field names will be sorted alphabetically as the only option. + - As of Spark 3.0 `Row` field names are no longer sorted alphabetically when constructing with named arguments for Python versions 3.6 and above, and the order of fields will match that as entered. To enable sorted fields by default, as in Spark 2.4, set the environment variable `PYSPARK_ROW_FIELD_SORTING_ENABLED` to "true" for both executors and driver - this environment variable must be consistent on all executors and driver; otherwise, it may cause failures or incorrect answers. For Python versions less than 3.6, the field names will be sorted alphabetically as the only option. ## Upgrading from PySpark 2.3 to 2.4 From d273a2bb0fac452a97f5670edd69d3e452e3e57e Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Fri, 14 Feb 2020 12:36:52 -0800 Subject: [PATCH 1381/1387] [SPARK-20628][CORE][K8S] Start to improve Spark decommissioning & preemption support This PR is based on an existing/previou PR - https://github.com/apache/spark/pull/19045 ### What changes were proposed in this pull request? This changes adds a decommissioning state that we can enter when the cloud provider/scheduler lets us know we aren't going to be removed immediately but instead will be removed soon. This concept fits nicely in K8s and also with spot-instances on AWS / preemptible instances all of which we can get a notice that our host is going away. For now we simply stop scheduling jobs, in the future we could perform some kind of migration of data during scale-down, or at least stop accepting new blocks to cache. There is a design document at https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE/edit?usp=sharing ### Why are the changes needed? With more move to preemptible multi-tenancy, serverless environments, and spot-instances better handling of node scale down is required. ### Does this PR introduce any user-facing change? There is no API change, however an additional configuration flag is added to enable/disable this behaviour. ### How was this patch tested? New integration tests in the Spark K8s integration testing. Extension of the AppClientSuite to test decommissioning seperate from the K8s. Closes #26440 from holdenk/SPARK-20628-keep-track-of-nodes-which-are-going-to-be-shutdown-r4. Lead-authored-by: Holden Karau Co-authored-by: Holden Karau Signed-off-by: Holden Karau --- .../apache/spark/deploy/DeployMessage.scala | 11 ++ .../apache/spark/deploy/ExecutorState.scala | 8 +- .../deploy/client/StandaloneAppClient.scala | 2 + .../client/StandaloneAppClientListener.scala | 2 + .../apache/spark/deploy/master/Master.scala | 31 +++++ .../apache/spark/deploy/worker/Worker.scala | 26 ++++ .../CoarseGrainedExecutorBackend.scala | 39 +++++- .../org/apache/spark/executor/Executor.scala | 16 +++ .../apache/spark/internal/config/Worker.scala | 5 + .../main/scala/org/apache/spark/rdd/RDD.scala | 2 + .../spark/scheduler/ExecutorLossReason.scala | 8 ++ .../org/apache/spark/scheduler/Pool.scala | 4 + .../apache/spark/scheduler/Schedulable.scala | 1 + .../spark/scheduler/SchedulerBackend.scala | 3 + .../spark/scheduler/TaskScheduler.scala | 5 + .../spark/scheduler/TaskSchedulerImpl.scala | 5 + .../spark/scheduler/TaskSetManager.scala | 6 + .../cluster/CoarseGrainedClusterMessage.scala | 2 + .../CoarseGrainedSchedulerBackend.scala | 66 +++++++++- .../cluster/StandaloneSchedulerBackend.scala | 6 + .../org/apache/spark/util/SignalUtils.scala | 2 +- .../spark/deploy/client/AppClientSuite.scala | 39 +++++- .../spark/scheduler/DAGSchedulerSuite.scala | 2 + .../ExternalClusterManagerSuite.scala | 1 + .../scheduler/WorkerDecommissionSuite.scala | 84 +++++++++++++ .../spark/deploy/k8s/KubernetesConf.scala | 3 + .../features/BasicExecutorFeatureStep.scala | 20 ++- .../src/main/dockerfiles/spark/Dockerfile | 4 +- .../src/main/dockerfiles/spark/decom.sh | 35 ++++++ .../src/main/dockerfiles/spark/entrypoint.sh | 6 +- .../dev/dev-run-integration-tests.sh | 9 +- .../integrationtest/DecommissionSuite.scala | 49 ++++++++ .../k8s/integrationtest/KubernetesSuite.scala | 117 ++++++++++++++---- .../tests/decommissioning.py | 45 +++++++ sbin/decommission-slave.sh | 57 +++++++++ sbin/spark-daemon.sh | 15 +++ 36 files changed, 690 insertions(+), 46 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala create mode 100755 resource-managers/kubernetes/docker/src/main/dockerfiles/spark/decom.sh create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala create mode 100644 resource-managers/kubernetes/integration-tests/tests/decommissioning.py create mode 100644 sbin/decommission-slave.sh diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index fba371dcfb761..18305ad3746a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -60,6 +60,15 @@ private[deploy] object DeployMessages { assert (port > 0) } + /** + * @param id the worker id + * @param worker the worker endpoint ref + */ + case class WorkerDecommission( + id: String, + worker: RpcEndpointRef) + extends DeployMessage + case class ExecutorStateChanged( appId: String, execId: Int, @@ -149,6 +158,8 @@ private[deploy] object DeployMessages { case object ReregisterWithMaster // used when a worker attempts to reconnect to a master + case object DecommissionSelf // Mark as decommissioned. May be Master to Worker in the future. + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription, driver: RpcEndpointRef) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 69c98e28931d7..0751bcf221f86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -19,9 +19,13 @@ package org.apache.spark.deploy private[deploy] object ExecutorState extends Enumeration { - val LAUNCHING, RUNNING, KILLED, FAILED, LOST, EXITED = Value + val LAUNCHING, RUNNING, KILLED, FAILED, LOST, EXITED, DECOMMISSIONED = Value type ExecutorState = Value - def isFinished(state: ExecutorState): Boolean = Seq(KILLED, FAILED, LOST, EXITED).contains(state) + // DECOMMISSIONED isn't listed as finished since we don't want to remove the executor from + // the worker and the executor still exists - but we do want to avoid scheduling new tasks on it. + private val finishedStates = Seq(KILLED, FAILED, LOST, EXITED) + + def isFinished(state: ExecutorState): Boolean = finishedStates.contains(state) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index 8f17159228f8b..eedf5e969e291 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -180,6 +180,8 @@ private[spark] class StandaloneAppClient( logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText)) if (ExecutorState.isFinished(state)) { listener.executorRemoved(fullId, message.getOrElse(""), exitStatus, workerLost) + } else if (state == ExecutorState.DECOMMISSIONED) { + listener.executorDecommissioned(fullId, message.getOrElse("")) } case WorkerRemoved(id, host, message) => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala index d8bc1a883def1..2e38a6847891d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala @@ -39,5 +39,7 @@ private[spark] trait StandaloneAppClientListener { def executorRemoved( fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit + def executorDecommissioned(fullId: String, message: String): Unit + def workerRemoved(workerId: String, host: String, message: String): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 8d3795cae707a..71df5dfa423a9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -243,6 +243,15 @@ private[deploy] class Master( logError("Leadership has been revoked -- master shutting down.") System.exit(0) + case WorkerDecommission(id, workerRef) => + logInfo("Recording worker %s decommissioning".format(id)) + if (state == RecoveryState.STANDBY) { + workerRef.send(MasterInStandby) + } else { + // We use foreach since get gives us an option and we can skip the failures. + idToWorker.get(id).foreach(decommissionWorker) + } + case RegisterWorker( id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl, masterAddress, resources) => @@ -313,7 +322,9 @@ private[deploy] class Master( // Only retry certain number of times so we don't go into an infinite loop. // Important note: this code path is not exercised by tests, so be very careful when // changing this `if` condition. + // We also don't count failures from decommissioned workers since they are "expected." if (!normalExit + && oldState != ExecutorState.DECOMMISSIONED && appInfo.incrementRetryCount() >= maxExecutorRetries && maxExecutorRetries >= 0) { // < 0 disables this application-killing path val execs = appInfo.executors.values @@ -850,6 +861,26 @@ private[deploy] class Master( true } + private def decommissionWorker(worker: WorkerInfo): Unit = { + if (worker.state != WorkerState.DECOMMISSIONED) { + logInfo("Decommissioning worker %s on %s:%d".format(worker.id, worker.host, worker.port)) + worker.setState(WorkerState.DECOMMISSIONED) + for (exec <- worker.executors.values) { + logInfo("Telling app of decommission executors") + exec.application.driver.send(ExecutorUpdated( + exec.id, ExecutorState.DECOMMISSIONED, + Some("worker decommissioned"), None, workerLost = false)) + exec.state = ExecutorState.DECOMMISSIONED + exec.application.removeExecutor(exec) + } + // On recovery do not add a decommissioned executor + persistenceEngine.removeWorker(worker) + } else { + logWarning("Skipping decommissioning worker %s on %s:%d as worker is already decommissioned". + format(worker.id, worker.host, worker.port)) + } + } + private def removeWorker(worker: WorkerInfo, msg: String): Unit = { logInfo("Removing worker " + worker.id + " on " + worker.host + ":" + worker.port) worker.setState(WorkerState.DEAD) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 4be495ac4f13f..d988bcedb47f0 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -67,6 +67,14 @@ private[deploy] class Worker( Utils.checkHost(host) assert (port > 0) + // If worker decommissioning is enabled register a handler on PWR to shutdown. + if (conf.get(WORKER_DECOMMISSION_ENABLED)) { + logInfo("Registering SIGPWR handler to trigger decommissioning.") + SignalUtils.register("PWR")(decommissionSelf) + } else { + logInfo("Worker decommissioning not enabled, SIGPWR will result in exiting.") + } + // A scheduled executor used to send messages at the specified time. private val forwardMessageScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("worker-forward-message-scheduler") @@ -128,6 +136,7 @@ private[deploy] class Worker( private val workerUri = RpcEndpointAddress(rpcEnv.address, endpointName).toString private var registered = false private var connected = false + private var decommissioned = false private val workerId = generateWorkerId() private val sparkHome = if (sys.props.contains(IS_TESTING.key)) { @@ -549,6 +558,8 @@ private[deploy] class Worker( case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_, resources_) => if (masterUrl != activeMasterUrl) { logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.") + } else if (decommissioned) { + logWarning("Asked to launch an executor while decommissioned. Not launching executor.") } else { try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) @@ -672,6 +683,9 @@ private[deploy] class Worker( case ApplicationFinished(id) => finishedApps += id maybeCleanupApplication(id) + + case DecommissionSelf => + decommissionSelf() } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -771,6 +785,18 @@ private[deploy] class Worker( } } + private[deploy] def decommissionSelf(): Boolean = { + if (conf.get(WORKER_DECOMMISSION_ENABLED)) { + logDebug("Decommissioning self") + decommissioned = true + sendToMaster(WorkerDecommission(workerId, self)) + } else { + logWarning("Asked to decommission self, but decommissioning not enabled") + } + // Return true since can be called as a signal handler + true + } + private[worker] def handleDriverStateChanged(driverStateChanged: DriverStateChanged): Unit = { val driverId = driverStateChanged.driverId val exception = driverStateChanged.exception diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 25c5b9812fa1a..faf03a64ae8b2 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -43,7 +43,7 @@ import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, ThreadUtils, Utils} +import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, SignalUtils, ThreadUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( override val rpcEnv: RpcEnv, @@ -64,6 +64,7 @@ private[spark] class CoarseGrainedExecutorBackend( private[this] val stopping = new AtomicBoolean(false) var executor: Executor = null + @volatile private var decommissioned = false @volatile var driver: Option[RpcEndpointRef] = None // If this CoarseGrainedExecutorBackend is changed to support multiple threads, then this may need @@ -80,6 +81,9 @@ private[spark] class CoarseGrainedExecutorBackend( private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] override def onStart(): Unit = { + logInfo("Registering PWR handler.") + SignalUtils.register("PWR")(decommissionSelf) + logInfo("Connecting to driver: " + driverUrl) try { _resources = parseOrFindResources(resourcesFileOpt) @@ -160,6 +164,16 @@ private[spark] class CoarseGrainedExecutorBackend( if (executor == null) { exitExecutor(1, "Received LaunchTask command but executor was null") } else { + if (decommissioned) { + logError("Asked to launch a task while decommissioned.") + driver match { + case Some(endpoint) => + logInfo("Sending DecommissionExecutor to driver.") + endpoint.send(DecommissionExecutor(executorId)) + case _ => + logError("No registered driver to send Decommission to.") + } + } val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) taskResources(taskDesc.taskId) = taskDesc.resources @@ -242,6 +256,29 @@ private[spark] class CoarseGrainedExecutorBackend( System.exit(code) } + + private def decommissionSelf(): Boolean = { + logInfo("Decommissioning self w/sync") + try { + decommissioned = true + // Tell master we are are decommissioned so it stops trying to schedule us + if (driver.nonEmpty) { + driver.get.askSync[Boolean](DecommissionExecutor(executorId)) + } else { + logError("No driver to message decommissioning.") + } + if (executor != null) { + executor.decommission() + } + logInfo("Done decommissioning self.") + // Return true since we are handling a signal + true + } catch { + case e: Exception => + logError(s"Error ${e} during attempt to decommission self") + false + } + } } private[spark] object CoarseGrainedExecutorBackend extends Logging { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 8aeb16fe5d8c8..2bfa1cea4b26f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -216,16 +216,32 @@ private[spark] class Executor( */ private var heartbeatFailures = 0 + /** + * Flag to prevent launching new tasks while decommissioned. There could be a race condition + * accessing this, but decommissioning is only intended to help not be a hard stop. + */ + private var decommissioned = false + heartbeater.start() metricsPoller.start() private[executor] def numRunningTasks: Int = runningTasks.size() + /** + * Mark an executor for decommissioning and avoid launching new tasks. + */ + private[spark] def decommission(): Unit = { + decommissioned = true + } + def launchTask(context: ExecutorBackend, taskDescription: TaskDescription): Unit = { val tr = new TaskRunner(context, taskDescription) runningTasks.put(taskDescription.taskId, tr) threadPool.execute(tr) + if (decommissioned) { + log.error(s"Launching a task while in decommissioned state.") + } } def killTask(taskId: Long, interruptThread: Boolean, reason: String): Unit = { diff --git a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala index f1eaae29f18df..2b175c1e14ee5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala @@ -71,4 +71,9 @@ private[spark] object Worker { ConfigBuilder("spark.worker.ui.compressedLogFileLengthCacheSize") .intConf .createWithDefault(100) + + private[spark] val WORKER_DECOMMISSION_ENABLED = + ConfigBuilder("spark.worker.decommission.enabled") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 64d2032a12721..a26b5791fa08b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -361,6 +361,7 @@ abstract class RDD[T: ClassTag]( readCachedBlock = false computeOrReadCheckpoint(partition, context) }) match { + // Block hit. case Left(blockResult) => if (readCachedBlock) { val existingMetrics = context.taskMetrics().inputMetrics @@ -374,6 +375,7 @@ abstract class RDD[T: ClassTag]( } else { new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]]) } + // Need to compute the block. case Right(iter) => new InterruptibleIterator(context, iter.asInstanceOf[Iterator[T]]) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index 46a35b6a2eaf9..ee31093ec0652 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -58,3 +58,11 @@ private [spark] object LossReasonPending extends ExecutorLossReason("Pending los private[spark] case class SlaveLost(_message: String = "Slave lost", workerLost: Boolean = false) extends ExecutorLossReason(_message) + +/** + * A loss reason that means the executor is marked for decommissioning. + * + * This is used by the task scheduler to remove state associated with the executor, but + * not yet fail any tasks that were running in the executor before the executor is "fully" lost. + */ +private [spark] object ExecutorDecommission extends ExecutorLossReason("Executor decommission.") diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 80805df256a15..2e2851eb9070b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -88,6 +88,10 @@ private[spark] class Pool( schedulableQueue.asScala.foreach(_.executorLost(executorId, host, reason)) } + override def executorDecommission(executorId: String): Unit = { + schedulableQueue.asScala.foreach(_.executorDecommission(executorId)) + } + override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = { var shouldRevive = false for (schedulable <- schedulableQueue.asScala) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index b6f88ed0a93aa..8cc239c81d11a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -43,6 +43,7 @@ private[spark] trait Schedulable { def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String, reason: ExecutorLossReason): Unit + def executorDecommission(executorId: String): Unit def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 9159d2a0158d5..4752353046c19 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -27,6 +27,9 @@ private[spark] trait SchedulerBackend { def start(): Unit def stop(): Unit + /** + * Update the current offers and schedule tasks + */ def reviveOffers(): Unit def defaultParallelism(): Int diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 15f5d20e9be75..e9e638a3645ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -98,6 +98,11 @@ private[spark] trait TaskScheduler { */ def applicationId(): String = appId + /** + * Process a decommissioning executor. + */ + def executorDecommission(executorId: String): Unit + /** * Process a lost executor */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index bf92081d13907..1b197c4cca53e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -734,6 +734,11 @@ private[spark] class TaskSchedulerImpl( } } + override def executorDecommission(executorId: String): Unit = { + rootPool.executorDecommission(executorId) + backend.reviveOffers() + } + override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = { var failedExecutor: Option[String] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 2ce11347ade39..18684ee8ebbc2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -1083,6 +1083,12 @@ private[spark] class TaskSetManager( levels.toArray } + def executorDecommission(execId: String): Unit = { + recomputeLocality() + // Future consideration: if an executor is decommissioned it may make sense to add the current + // tasks to the spec exec queue. + } + def recomputeLocality(): Unit = { // A zombie TaskSetManager may reach here while executorLost happens if (isZombie) return diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 283390814a6c0..8db0122f17ab4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -94,6 +94,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: ExecutorLossReason) extends CoarseGrainedClusterMessage + case class DecommissionExecutor(executorId: String) extends CoarseGrainedClusterMessage + case class RemoveWorker(workerId: String, host: String, message: String) extends CoarseGrainedClusterMessage 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 63aa04986b073..6e1efdaf5beb2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -92,6 +92,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Executors that have been lost, but for which we don't yet know the real exit reason. private val executorsPendingLossReason = new HashSet[String] + // Executors which are being decommissioned + protected val executorsPendingDecommission = new HashSet[String] + // A map of ResourceProfile id to map of hostname with its possible task number running on it @GuardedBy("CoarseGrainedSchedulerBackend.this") protected var rpHostToLocalTaskCount: Map[Int, Map[String, Int]] = Map.empty @@ -185,11 +188,20 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) + case DecommissionExecutor(executorId) => + logError(s"Received decommission executor message ${executorId}.") + decommissionExecutor(executorId) + + case RemoveWorker(workerId, host, message) => + removeWorker(workerId, host, message) + case LaunchedExecutor(executorId) => executorDataMap.get(executorId).foreach { data => data.freeCores = data.totalCores } makeOffers(executorId) + case e => + logError(s"Received unexpected message. ${e}") } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -257,6 +269,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp removeWorker(workerId, host, message) context.reply(true) + case DecommissionExecutor(executorId) => + logError(s"Received decommission executor message ${executorId}.") + decommissionExecutor(executorId) + context.reply(true) + case RetrieveSparkAppConfig(resourceProfileId) => val rp = scheduler.sc.resourceProfileManager.resourceProfileFromId(resourceProfileId) val reply = SparkAppConfig( @@ -265,6 +282,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp Option(delegationTokens.get()), rp) context.reply(reply) + case e => + logError(s"Received unexpected ask ${e}") } // Make fake resource offers on all executors @@ -365,6 +384,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId executorsPendingLossReason -= executorId + executorsPendingDecommission -= executorId executorsPendingToRemove.remove(executorId).getOrElse(false) } totalCoreCount.addAndGet(-executorInfo.totalCores) @@ -389,6 +409,35 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp scheduler.workerRemoved(workerId, host, message) } + /** + * Mark a given executor as decommissioned and stop making resource offers for it. + */ + private def decommissionExecutor(executorId: String): Boolean = { + val shouldDisable = CoarseGrainedSchedulerBackend.this.synchronized { + // Only bother decommissioning executors which are alive. + if (isExecutorActive(executorId)) { + executorsPendingDecommission += executorId + true + } else { + false + } + } + + if (shouldDisable) { + logInfo(s"Starting decommissioning executor $executorId.") + try { + scheduler.executorDecommission(executorId) + } catch { + case e: Exception => + logError(s"Unexpected error during decommissioning ${e.toString}", e) + } + logInfo(s"Finished decommissioning executor $executorId.") + } else { + logInfo(s"Skipping decommissioning of executor $executorId.") + } + shouldDisable + } + /** * Stop making resource offers for the given executor. The executor is marked as lost with * the loss reason still pending. @@ -511,8 +560,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } protected def removeWorker(workerId: String, host: String, message: String): Unit = { - driverEndpoint.ask[Boolean](RemoveWorker(workerId, host, message)).failed.foreach(t => - logError(t.getMessage, t))(ThreadUtils.sameThread) + driverEndpoint.send(RemoveWorker(workerId, host, message)) + } + + /** + * Called by subclasses when notified of a decommissioning executor. + */ + private[spark] def decommissionExecutor(executorId: String): Unit = { + if (driverEndpoint != null) { + logInfo("Propegating executor decommission to driver.") + driverEndpoint.send(DecommissionExecutor(executorId)) + } } def sufficientResourcesRegistered(): Boolean = true @@ -543,7 +601,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def isExecutorActive(id: String): Boolean = synchronized { executorDataMap.contains(id) && !executorsPendingToRemove.contains(id) && - !executorsPendingLossReason.contains(id) + !executorsPendingLossReason.contains(id) && + !executorsPendingDecommission.contains(id) + } override def maxNumConcurrentTasks(): Int = synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index d91d78b29f98d..42c46464d79e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -174,6 +174,12 @@ private[spark] class StandaloneSchedulerBackend( removeExecutor(fullId.split("/")(1), reason) } + override def executorDecommissioned(fullId: String, message: String) { + logInfo("Asked to decommission executor") + decommissionExecutor(fullId.split("/")(1)) + logInfo("Executor %s decommissioned: %s".format(fullId, message)) + } + override def workerRemoved(workerId: String, host: String, message: String): Unit = { logInfo("Worker %s removed: %s".format(workerId, message)) removeWorker(workerId, host, message) diff --git a/core/src/main/scala/org/apache/spark/util/SignalUtils.scala b/core/src/main/scala/org/apache/spark/util/SignalUtils.scala index 5a24965170cef..230195da2a121 100644 --- a/core/src/main/scala/org/apache/spark/util/SignalUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/SignalUtils.scala @@ -60,7 +60,7 @@ private[spark] object SignalUtils extends Logging { if (SystemUtils.IS_OS_UNIX) { try { val handler = handlers.getOrElseUpdate(signal, { - logInfo("Registered signal handler for " + signal) + logInfo("Registering signal handler for " + signal) new ActionHandler(new Signal(signal)) }) handler.register(action) diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index a1d3077b8fc87..a3e39d7f53728 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, Master} import org.apache.spark.deploy.worker.Worker -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.Utils @@ -44,13 +44,13 @@ class AppClientSuite with Eventually with ScalaFutures { private val numWorkers = 2 - private val conf = new SparkConf() - private val securityManager = new SecurityManager(conf) + private var conf: SparkConf = null private var masterRpcEnv: RpcEnv = null private var workerRpcEnvs: Seq[RpcEnv] = null private var master: Master = null private var workers: Seq[Worker] = null + private var securityManager: SecurityManager = null /** * Start the local cluster. @@ -58,6 +58,8 @@ class AppClientSuite */ override def beforeAll(): Unit = { super.beforeAll() + conf = new SparkConf().set(config.Worker.WORKER_DECOMMISSION_ENABLED.key, "true") + securityManager = new SecurityManager(conf) masterRpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 0, conf, securityManager) workerRpcEnvs = (0 until numWorkers).map { i => RpcEnv.create(Worker.SYSTEM_NAME + i, "localhost", 0, conf, securityManager) @@ -111,8 +113,23 @@ class AppClientSuite assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") } + + // Save the executor id before decommissioning so we can kill it + val application = getApplications().head + val executors = application.executors + val executorId: String = executors.head._2.fullId + + // Send a decommission self to all the workers + // Note: normally the worker would send this on their own. + workers.foreach(worker => worker.decommissionSelf()) + + // Decommissioning is async. + eventually(timeout(1.seconds), interval(10.millis)) { + // We only record decommissioning for the executor we've requested + assert(ci.listener.execDecommissionedList.size === 1) + } + // Send request to kill executor, verify request was made - val executorId: String = getApplications().head.executors.head._2.fullId whenReady( ci.client.killExecutors(Seq(executorId)), timeout(10.seconds), @@ -120,6 +137,15 @@ class AppClientSuite assert(acknowledged) } + // Verify that asking for executors on the decommissioned workers fails + whenReady( + ci.client.requestTotalExecutors(numExecutorsRequested), + timeout(10.seconds), + interval(10.millis)) { acknowledged => + assert(acknowledged) + } + assert(getApplications().head.executors.size === 0) + // Issue stop command for Client to disconnect from Master ci.client.stop() @@ -189,6 +215,7 @@ class AppClientSuite val deadReasonList = new ConcurrentLinkedQueue[String]() val execAddedList = new ConcurrentLinkedQueue[String]() val execRemovedList = new ConcurrentLinkedQueue[String]() + val execDecommissionedList = new ConcurrentLinkedQueue[String]() def connected(id: String): Unit = { connectedIdList.add(id) @@ -218,6 +245,10 @@ class AppClientSuite execRemovedList.add(id) } + def executorDecommissioned(id: String, message: String): Unit = { + execDecommissionedList.add(id) + } + def workerRemoved(workerId: String, host: String, message: String): Unit = {} } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 101e60c73e9f8..e40b63fe13cb1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -167,6 +167,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 + override def executorDecommission(executorId: String) = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None @@ -707,6 +708,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, executorUpdates: Map[(Int, Int), ExecutorMetrics]): Boolean = true + override def executorDecommission(executorId: String): Unit = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 4e71ec1ea7b37..9f593e0039adc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -89,6 +89,7 @@ private class DummyTaskScheduler extends TaskScheduler { override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = {} override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} override def defaultParallelism(): Int = 2 + override def executorDecommission(executorId: String): Unit = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala new file mode 100644 index 0000000000000..15733b0d932ec --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.util.concurrent.Semaphore + +import scala.concurrent.TimeoutException +import scala.concurrent.duration._ + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark.internal.config +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils} + +class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { + + override def beforeEach(): Unit = { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) + + sc = new SparkContext("local-cluster[2, 1, 1024]", "test", conf) + } + + test("verify task with no decommissioning works as expected") { + val input = sc.parallelize(1 to 10) + input.count() + val sleepyRdd = input.mapPartitions{ x => + Thread.sleep(100) + x + } + assert(sleepyRdd.count() === 10) + } + + test("verify a task with all workers decommissioned succeeds") { + val input = sc.parallelize(1 to 10) + // Do a count to wait for the executors to be registered. + input.count() + val sleepyRdd = input.mapPartitions{ x => + Thread.sleep(50) + x + } + // Listen for the job + val sem = new Semaphore(0) + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + sem.release() + } + }) + // Start the task. + val asyncCount = sleepyRdd.countAsync() + // Wait for the job to have started + sem.acquire(1) + // Decommission all the executors, this should not halt the current task. + // decom.sh message passing is tested manually. + val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] + val execs = sched.getExecutorIds() + execs.foreach(execId => sched.decommissionExecutor(execId)) + val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 2.seconds) + assert(asyncCountResult === 10) + // Try and launch task after decommissioning, this should fail + val postDecommissioned = input.map(x => x) + val postDecomAsyncCount = postDecommissioned.countAsync() + val thrown = intercept[java.util.concurrent.TimeoutException]{ + val result = ThreadUtils.awaitResult(postDecomAsyncCount, 2.seconds) + } + assert(postDecomAsyncCount.isCompleted === false, + "After exec decommission new task could not launch") + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index 09943b7974ed9..f42f3415baa15 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -55,6 +55,9 @@ private[spark] abstract class KubernetesConf(val sparkConf: SparkConf) { } } + def workerDecommissioning: Boolean = + sparkConf.get(org.apache.spark.internal.config.Worker.WORKER_DECOMMISSION_ENABLED) + def nodeSelector: Map[String, String] = KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 6a26df2997fd2..f575241de9540 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -24,6 +24,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python._ import org.apache.spark.rpc.RpcEndpointAddress @@ -33,7 +34,7 @@ import org.apache.spark.util.Utils private[spark] class BasicExecutorFeatureStep( kubernetesConf: KubernetesExecutorConf, secMgr: SecurityManager) - extends KubernetesFeatureConfigStep { + extends KubernetesFeatureConfigStep with Logging { // Consider moving some of these fields to KubernetesConf or KubernetesExecutorSpecificConf private val executorContainerImage = kubernetesConf @@ -186,6 +187,21 @@ private[spark] class BasicExecutorFeatureStep( .endResources() .build() }.getOrElse(executorContainer) + val containerWithLifecycle = + if (!kubernetesConf.workerDecommissioning) { + logInfo("Decommissioning not enabled, skipping shutdown script") + containerWithLimitCores + } else { + logInfo("Adding decommission script to lifecycle") + new ContainerBuilder(containerWithLimitCores).withNewLifecycle() + .withNewPreStop() + .withNewExec() + .addToCommand("/opt/decom.sh") + .endExec() + .endPreStop() + .endLifecycle() + .build() + } val ownerReference = kubernetesConf.driverPod.map { pod => new OwnerReferenceBuilder() .withController(true) @@ -213,6 +229,6 @@ private[spark] class BasicExecutorFeatureStep( kubernetesConf.get(KUBERNETES_EXECUTOR_SCHEDULER_NAME) .foreach(executorPod.getSpec.setSchedulerName) - SparkPod(executorPod, containerWithLimitCores) + SparkPod(executorPod, containerWithLifecycle) } } diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 6ed37fc637b31..cc65a7da12eef 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -30,7 +30,7 @@ ARG spark_uid=185 RUN set -ex && \ apt-get update && \ ln -s /lib /lib64 && \ - apt install -y bash tini libc6 libpam-modules krb5-user libnss3 && \ + apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps && \ mkdir -p /opt/spark && \ mkdir -p /opt/spark/examples && \ mkdir -p /opt/spark/work-dir && \ @@ -45,6 +45,7 @@ COPY jars /opt/spark/jars COPY bin /opt/spark/bin COPY sbin /opt/spark/sbin COPY kubernetes/dockerfiles/spark/entrypoint.sh /opt/ +COPY kubernetes/dockerfiles/spark/decom.sh /opt/ COPY examples /opt/spark/examples COPY kubernetes/tests /opt/spark/tests COPY data /opt/spark/data @@ -53,6 +54,7 @@ ENV SPARK_HOME /opt/spark WORKDIR /opt/spark/work-dir RUN chmod g+w /opt/spark/work-dir +RUN chmod a+x /opt/decom.sh ENTRYPOINT [ "/opt/entrypoint.sh" ] diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/decom.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/decom.sh new file mode 100755 index 0000000000000..8a5208d49a70f --- /dev/null +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/decom.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env 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. +# + + +set -ex +echo "Asked to decommission" +# Find the pid to signal +date | tee -a ${LOG} +WORKER_PID=$(ps -o pid -C java | tail -n 1| awk '{ sub(/^[ \t]+/, ""); print }') +echo "Using worker pid $WORKER_PID" +kill -s SIGPWR ${WORKER_PID} +# For now we expect this to timeout, since we don't start exiting the backend. +echo "Waiting for worker pid to exit" +# If the worker does exit stop blocking the cleanup. +timeout 60 tail --pid=${WORKER_PID} -f /dev/null +date +echo "Done" +date +sleep 30 diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 6ee3523c8edab..05ab782caecae 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -30,9 +30,9 @@ set -e # 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:${SPARK_USER_NAME:-anonymous uid}:$SPARK_HOME:/bin/false" >> /etc/passwd + echo "$myuid:x:$myuid:$mygid:${SPARK_USER_NAME:-anonymous uid}:$SPARK_HOME:/bin/false" >> /etc/passwd else - echo "Container ENTRYPOINT failed to add passwd entry for anonymous UID" + echo "Container ENTRYPOINT failed to add passwd entry for anonymous UID" fi fi @@ -59,7 +59,7 @@ fi # If HADOOP_HOME is set and SPARK_DIST_CLASSPATH is not set, set it here so Hadoop jars are available to the executor. # It does not set SPARK_DIST_CLASSPATH if already set, to avoid overriding customizations of this value from elsewhere e.g. Docker/K8s. if [ -n ${HADOOP_HOME} ] && [ -z ${SPARK_DIST_CLASSPATH} ]; then - export SPARK_DIST_CLASSPATH=$($HADOOP_HOME/bin/hadoop classpath) + export SPARK_DIST_CLASSPATH=$($HADOOP_HOME/bin/hadoop classpath) fi if ! [ -z ${HADOOP_CONF_DIR+x} ]; then diff --git a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh index 607bb243458a6..292abe91d35b6 100755 --- a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh +++ b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh @@ -16,7 +16,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # -set -xo errexit +set -exo errexit TEST_ROOT_DIR=$(git rev-parse --show-toplevel) DEPLOY_MODE="minikube" @@ -42,6 +42,9 @@ SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version 2>/dev/nu | grep -v "WARNING"\ | tail -n 1) +export SCALA_VERSION +echo $SCALA_VERSION + # Parse arguments while (( "$#" )); do case $1 in @@ -110,7 +113,8 @@ while (( "$#" )); do shift ;; *) - break + echo "Unexpected command line flag $2 $1." + exit 1 ;; esac shift @@ -164,6 +168,7 @@ properties+=( -Dspark.kubernetes.test.jvmImage=$JVM_IMAGE_NAME -Dspark.kubernetes.test.pythonImage=$PYTHON_IMAGE_NAME -Dspark.kubernetes.test.rImage=$R_IMAGE_NAME + -Dlog4j.logger.org.apache.spark=DEBUG ) $TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-$SCALA_VERSION -Pkubernetes -Pkubernetes-integration-tests ${properties[@]} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala new file mode 100644 index 0000000000000..f5eab6e4bbad6 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DecommissionSuite.scala @@ -0,0 +1,49 @@ +/* + * 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.integrationtest + +import org.apache.spark.internal.config.Worker + +private[spark] trait DecommissionSuite { k8sSuite: KubernetesSuite => + + import DecommissionSuite._ + import KubernetesSuite.k8sTestTag + + test("Test basic decommissioning", k8sTestTag) { + sparkAppConf + .set(Worker.WORKER_DECOMMISSION_ENABLED.key, "true") + .set("spark.kubernetes.pyspark.pythonVersion", "3") + .set("spark.kubernetes.container.image", pyImage) + + runSparkApplicationAndVerifyCompletion( + appResource = PYSPARK_DECOMISSIONING, + mainClass = "", + expectedLogOnCompletion = Seq("decommissioning executor", + "Finished waiting, stopping Spark"), + appArgs = Array.empty[String], + driverPodChecker = doBasicDriverPyPodCheck, + executorPodChecker = doBasicExecutorPyPodCheck, + appLocator = appLocator, + isJVM = false, + decommissioningTest = true) + } +} + +private[spark] object DecommissionSuite { + val TEST_LOCAL_PYSPARK: String = "local:///opt/spark/tests/" + val PYSPARK_DECOMISSIONING: String = TEST_LOCAL_PYSPARK + "decommissioning.py" +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 0d4fcccc35cf9..61e1f27b55462 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -42,7 +42,9 @@ import org.apache.spark.internal.config._ class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite - with DepsTestsSuite with RTestsSuite with Logging with Eventually with Matchers { + with DepsTestsSuite with DecommissionSuite with RTestsSuite with Logging with Eventually + with Matchers { + import KubernetesSuite._ @@ -254,6 +256,7 @@ class KubernetesSuite extends SparkFunSuite } } + // scalastyle:off argcount protected def runSparkApplicationAndVerifyCompletion( appResource: String, mainClass: String, @@ -264,60 +267,120 @@ class KubernetesSuite extends SparkFunSuite appLocator: String, isJVM: Boolean, pyFiles: Option[String] = None, - executorPatience: Option[(Option[Interval], Option[Timeout])] = None): Unit = { + executorPatience: Option[(Option[Interval], Option[Timeout])] = None, + decommissioningTest: Boolean = false): Unit = { + + // scalastyle:on argcount val appArguments = SparkAppArguments( mainAppResource = appResource, mainClass = mainClass, appArgs = appArgs) - SparkAppLauncher.launch( - appArguments, - sparkAppConf, - TIMEOUT.value.toSeconds.toInt, - sparkHomeDir, - isJVM, - pyFiles) - val driverPod = kubernetesTestComponents.kubernetesClient - .pods() - .withLabel("spark-app-locator", appLocator) - .withLabel("spark-role", "driver") - .list() - .getItems - .get(0) - driverPodChecker(driverPod) val execPods = scala.collection.mutable.Map[String, Pod]() + val (patienceInterval, patienceTimeout) = { + executorPatience match { + case Some(patience) => (patience._1.getOrElse(INTERVAL), patience._2.getOrElse(TIMEOUT)) + case _ => (INTERVAL, TIMEOUT) + } + } + def checkPodReady(namespace: String, name: String) = { + val execPod = kubernetesTestComponents.kubernetesClient + .pods() + .inNamespace(namespace) + .withName(name) + .get() + val resourceStatus = execPod.getStatus + val conditions = resourceStatus.getConditions().asScala + val conditionTypes = conditions.map(_.getType()) + val readyConditions = conditions.filter{cond => cond.getType() == "Ready"} + val result = readyConditions + .map(cond => cond.getStatus() == "True") + .headOption.getOrElse(false) + result + } val execWatcher = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", appLocator) .withLabel("spark-role", "executor") .watch(new Watcher[Pod] { - logInfo("Beginning watch of executors") + logDebug("Beginning watch of executors") override def onClose(cause: KubernetesClientException): Unit = logInfo("Ending watch of executors") override def eventReceived(action: Watcher.Action, resource: Pod): Unit = { val name = resource.getMetadata.getName + val namespace = resource.getMetadata().getNamespace() action match { - case Action.ADDED | Action.MODIFIED => + case Action.MODIFIED => + execPods(name) = resource + case Action.ADDED => + logDebug(s"Add event received for $name.") execPods(name) = resource + // If testing decommissioning start a thread to simulate + // decommissioning. + if (decommissioningTest && execPods.size == 1) { + // Wait for all the containers in the pod to be running + logDebug("Waiting for first pod to become OK prior to deletion") + Eventually.eventually(patienceTimeout, patienceInterval) { + val result = checkPodReady(namespace, name) + result shouldBe (true) + } + // Sleep a small interval to allow execution of job + logDebug("Sleeping before killing pod.") + Thread.sleep(2000) + // Delete the pod to simulate cluster scale down/migration. + val pod = kubernetesTestComponents.kubernetesClient.pods().withName(name) + pod.delete() + logDebug(s"Triggered pod decom/delete: $name deleted") + } case Action.DELETED | Action.ERROR => execPods.remove(name) } } }) - val (patienceInterval, patienceTimeout) = { - executorPatience match { - case Some(patience) => (patience._1.getOrElse(INTERVAL), patience._2.getOrElse(TIMEOUT)) - case _ => (INTERVAL, TIMEOUT) - } - } + logDebug("Starting Spark K8s job") + SparkAppLauncher.launch( + appArguments, + sparkAppConf, + TIMEOUT.value.toSeconds.toInt, + sparkHomeDir, + isJVM, + pyFiles) + val driverPod = kubernetesTestComponents.kubernetesClient + .pods() + .withLabel("spark-app-locator", appLocator) + .withLabel("spark-role", "driver") + .list() + .getItems + .get(0) + + driverPodChecker(driverPod) + // If we're testing decommissioning we delete all the executors, but we should have + // an executor at some point. Eventually.eventually(patienceTimeout, patienceInterval) { execPods.values.nonEmpty should be (true) } + // If decommissioning we need to wait and check the executors were removed + if (decommissioningTest) { + // Sleep a small interval to ensure everything is registered. + Thread.sleep(100) + // Wait for the executors to become ready + Eventually.eventually(patienceTimeout, patienceInterval) { + val anyReadyPods = ! execPods.map{ + case (name, resource) => + (name, resource.getMetadata().getNamespace()) + }.filter{ + case (name, namespace) => checkPodReady(namespace, name) + }.isEmpty + val podsEmpty = execPods.values.isEmpty + val podsReadyOrDead = anyReadyPods || podsEmpty + podsReadyOrDead shouldBe (true) + } + } execWatcher.close() execPods.values.foreach(executorPodChecker(_)) - Eventually.eventually(TIMEOUT, patienceInterval) { + Eventually.eventually(patienceTimeout, patienceInterval) { expectedLogOnCompletion.foreach { e => assert(kubernetesTestComponents.kubernetesClient .pods() @@ -425,5 +488,5 @@ private[spark] object KubernetesSuite { val SPARK_REMOTE_MAIN_CLASS: String = "org.apache.spark.examples.SparkRemoteFileTest" val SPARK_DRIVER_MAIN_CLASS: String = "org.apache.spark.examples.DriverSubmissionTest" val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) - val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) + val INTERVAL = PatienceConfiguration.Interval(Span(1, Seconds)) } diff --git a/resource-managers/kubernetes/integration-tests/tests/decommissioning.py b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py new file mode 100644 index 0000000000000..f68f24d49763d --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/tests/decommissioning.py @@ -0,0 +1,45 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import time + +from pyspark.sql import SparkSession + + +if __name__ == "__main__": + """ + Usage: decommissioning + """ + print("Starting decom test") + spark = SparkSession \ + .builder \ + .appName("PyMemoryTest") \ + .getOrCreate() + sc = spark._sc + rdd = sc.parallelize(range(10)) + rdd.collect() + print("Waiting to give nodes time to finish.") + time.sleep(5) + rdd.collect() + print("Waiting some more....") + time.sleep(10) + rdd.collect() + print("Finished waiting, stopping Spark.") + spark.stop() + print("Done, exiting Python") + sys.exit(0) diff --git a/sbin/decommission-slave.sh b/sbin/decommission-slave.sh new file mode 100644 index 0000000000000..4bbf257ff1d3a --- /dev/null +++ b/sbin/decommission-slave.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env 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. +# + +# A shell script to decommission all workers on a single slave +# +# Environment variables +# +# SPARK_WORKER_INSTANCES The number of worker instances that should be +# running on this slave. Default is 1. + +# Usage: decommission-slave.sh [--block-until-exit] +# Decommissions all slaves on this worker machine + +set -ex + +if [ -z "${SPARK_HOME}" ]; then + export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${SPARK_HOME}/sbin/spark-config.sh" + +. "${SPARK_HOME}/bin/load-spark-env.sh" + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + "${SPARK_HOME}/sbin"/spark-daemon.sh decommission org.apache.spark.deploy.worker.Worker 1 +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "${SPARK_HOME}/sbin"/spark-daemon.sh decommission org.apache.spark.deploy.worker.Worker $(( $i + 1 )) + done +fi + +# Check if --block-until-exit is set. +# This is done for systems which block on the decomissioning script and on exit +# shut down the entire system (e.g. K8s). +if [ "$1" == "--block-until-exit" ]; then + shift + # For now we only block on the 0th instance if there multiple instances. + instance=$1 + pid="$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid" + wait $pid +fi diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 6de67e039b48f..81f2fd40a706f 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -215,6 +215,21 @@ case $option in fi ;; + (decommission) + + if [ -f $pid ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then + echo "decommissioning $command" + kill -s SIGPWR "$TARGET_ID" + else + echo "no $command to decommission" + fi + else + echo "no $command to decommission" + fi + ;; + (status) if [ -f $pid ]; then From d0f961476031b62bda0d4d41f7248295d651ea92 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Fri, 14 Feb 2020 21:46:01 +0000 Subject: [PATCH 1382/1387] [SPARK-30289][SQL] Partitioned by Nested Column for `InMemoryTable` ### What changes were proposed in this pull request? 1. `InMemoryTable` was flatting the nested columns, and then the flatten columns was used to look up the indices which is not correct. This PR implements partitioned by nested column for `InMemoryTable`. ### Why are the changes needed? This PR implements partitioned by nested column for `InMemoryTable`, so we can test this features in DSv2 ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing unit tests and new tests. Closes #26929 from dbtsai/addTests. Authored-by: DB Tsai Signed-off-by: DB Tsai --- .../spark/sql/connector/InMemoryTable.scala | 35 ++++++-- .../spark/sql/DataFrameWriterV2Suite.scala | 86 ++++++++++++++++++- 2 files changed, 114 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index c9e4e0aad5704..0187ae31e2d1c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -26,7 +26,7 @@ import org.scalatest.Assertions._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.catalog._ -import org.apache.spark.sql.connector.expressions.{IdentityTransform, Transform} +import org.apache.spark.sql.connector.expressions.{IdentityTransform, NamedReference, Transform} import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.connector.write._ import org.apache.spark.sql.sources.{And, EqualTo, Filter, IsNotNull} @@ -59,10 +59,30 @@ class InMemoryTable( def rows: Seq[InternalRow] = dataMap.values.flatMap(_.rows).toSeq - private val partFieldNames = partitioning.flatMap(_.references).toSeq.flatMap(_.fieldNames) - private val partIndexes = partFieldNames.map(schema.fieldIndex) + private val partCols: Array[Array[String]] = partitioning.flatMap(_.references).map { ref => + schema.findNestedField(ref.fieldNames(), includeCollections = false) match { + case Some(_) => ref.fieldNames() + case None => throw new IllegalArgumentException(s"${ref.describe()} does not exist.") + } + } - private def getKey(row: InternalRow): Seq[Any] = partIndexes.map(row.toSeq(schema)(_)) + private def getKey(row: InternalRow): Seq[Any] = { + def extractor(fieldNames: Array[String], schema: StructType, row: InternalRow): Any = { + val index = schema.fieldIndex(fieldNames(0)) + val value = row.toSeq(schema).apply(index) + if (fieldNames.length > 1) { + (value, schema(index).dataType) match { + case (row: InternalRow, nestedSchema: StructType) => + extractor(fieldNames.drop(1), nestedSchema, row) + case (_, dataType) => + throw new IllegalArgumentException(s"Unsupported type, ${dataType.simpleString}") + } + } else { + value + } + } + partCols.map(fieldNames => extractor(fieldNames, schema, row)) + } def withData(data: Array[BufferedRows]): InMemoryTable = dataMap.synchronized { data.foreach(_.rows.foreach { row => @@ -146,8 +166,10 @@ class InMemoryTable( } private class Overwrite(filters: Array[Filter]) extends TestBatchWrite { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized { - val deleteKeys = InMemoryTable.filtersToKeys(dataMap.keys, partFieldNames, filters) + val deleteKeys = InMemoryTable.filtersToKeys( + dataMap.keys, partCols.map(_.toSeq.quoted), filters) dataMap --= deleteKeys withData(messages.map(_.asInstanceOf[BufferedRows])) } @@ -161,7 +183,8 @@ class InMemoryTable( } override def deleteWhere(filters: Array[Filter]): Unit = dataMap.synchronized { - dataMap --= InMemoryTable.filtersToKeys(dataMap.keys, partFieldNames, filters) + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper + dataMap --= InMemoryTable.filtersToKeys(dataMap.keys, partCols.map(_.toSeq.quoted), filters) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala index d49dc58e93ddb..cd157086a8b8e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWriterV2Suite.scala @@ -17,20 +17,24 @@ package org.apache.spark.sql +import java.sql.Timestamp + import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} -import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog} import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} +import org.apache.spark.sql.types.TimestampType import org.apache.spark.sql.util.QueryExecutionListener +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with BeforeAndAfter { @@ -550,4 +554,84 @@ class DataFrameWriterV2Suite extends QueryTest with SharedSparkSession with Befo assert(replaced.partitioning.isEmpty) assert(replaced.properties === defaultOwnership.asJava) } + + test("SPARK-30289 Create: partitioned by nested column") { + val schema = new StructType().add("ts", new StructType() + .add("created", TimestampType) + .add("modified", TimestampType) + .add("timezone", StringType)) + + val data = Seq( + Row(Row(Timestamp.valueOf("2019-06-01 10:00:00"), Timestamp.valueOf("2019-09-02 07:00:00"), + "America/Los_Angeles")), + Row(Row(Timestamp.valueOf("2019-08-26 18:00:00"), Timestamp.valueOf("2019-09-26 18:00:00"), + "America/Los_Angeles")), + Row(Row(Timestamp.valueOf("2018-11-23 18:00:00"), Timestamp.valueOf("2018-12-22 18:00:00"), + "America/New_York"))) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1), schema) + + df.writeTo("testcat.table_name") + .partitionedBy($"ts.timezone") + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + .asInstanceOf[InMemoryTable] + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(IdentityTransform(FieldReference(Array("ts", "timezone"))))) + checkAnswer(spark.table(table.name), data) + assert(table.dataMap.toArray.length == 2) + assert(table.dataMap(Seq(UTF8String.fromString("America/Los_Angeles"))).rows.size == 2) + assert(table.dataMap(Seq(UTF8String.fromString("America/New_York"))).rows.size == 1) + + // TODO: `DataSourceV2Strategy` can not translate nested fields into source filter yet + // so the following sql will fail. + // sql("DELETE FROM testcat.table_name WHERE ts.timezone = \"America/Los_Angeles\"") + } + + test("SPARK-30289 Create: partitioned by multiple transforms on nested columns") { + spark.table("source") + .withColumn("ts", struct( + lit("2019-06-01 10:00:00.000000").cast("timestamp") as "created", + lit("2019-09-02 07:00:00.000000").cast("timestamp") as "modified", + lit("America/Los_Angeles") as "timezone")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy( + years($"ts.created"), months($"ts.created"), days($"ts.created"), hours($"ts.created"), + years($"ts.modified"), months($"ts.modified"), days($"ts.modified"), hours($"ts.modified") + ) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq( + YearsTransform(FieldReference(Array("ts", "created"))), + MonthsTransform(FieldReference(Array("ts", "created"))), + DaysTransform(FieldReference(Array("ts", "created"))), + HoursTransform(FieldReference(Array("ts", "created"))), + YearsTransform(FieldReference(Array("ts", "modified"))), + MonthsTransform(FieldReference(Array("ts", "modified"))), + DaysTransform(FieldReference(Array("ts", "modified"))), + HoursTransform(FieldReference(Array("ts", "modified"))))) + } + + test("SPARK-30289 Create: partitioned by bucket(4, ts.timezone)") { + spark.table("source") + .withColumn("ts", struct( + lit("2019-06-01 10:00:00.000000").cast("timestamp") as "created", + lit("2019-09-02 07:00:00.000000").cast("timestamp") as "modified", + lit("America/Los_Angeles") as "timezone")) + .writeTo("testcat.table_name") + .tableProperty("allow-unsupported-transforms", "true") + .partitionedBy(bucket(4, $"ts.timezone")) + .create() + + val table = catalog("testcat").loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name === "testcat.table_name") + assert(table.partitioning === Seq(BucketTransform(LiteralValue(4, IntegerType), + Seq(FieldReference(Seq("ts", "timezone")))))) + } } From 8b73b92aadd685b29ef3d9b33366f5e1fd3dae99 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Feb 2020 19:49:58 +0800 Subject: [PATCH 1383/1387] [SPARK-30826][SQL] Respect reference case in `StringStartsWith` pushed down to parquet ### What changes were proposed in this pull request? In the PR, I propose to convert the attribute name of `StringStartsWith` pushed down to the Parquet datasource to column reference via the `nameToParquetField` map. Similar conversions are performed for other source filters pushed down to parquet. ### Why are the changes needed? This fixes the bug described in [SPARK-30826](https://issues.apache.org/jira/browse/SPARK-30826). The query from an external table: ```sql CREATE TABLE t1 (col STRING) USING parquet OPTIONS (path '$path') ``` created on top of written parquet files by `Seq("42").toDF("COL").write.parquet(path)` returns wrong empty result: ```scala spark.sql("SELECT * FROM t1 WHERE col LIKE '4%'").show +---+ |col| +---+ +---+ ``` ### Does this PR introduce any user-facing change? Yes. After the changes the result is correct for the example above: ```scala spark.sql("SELECT * FROM t1 WHERE col LIKE '4%'").show +---+ |col| +---+ | 42| +---+ ``` ### How was this patch tested? Added a test to `ParquetFilterSuite` Closes #27574 from MaxGekk/parquet-StringStartsWith-case-sens. Authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../datasources/parquet/ParquetFilters.scala | 2 +- .../parquet/ParquetFilterSuite.scala | 21 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index b9b86adb438e6..948a120e0d6e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -591,7 +591,7 @@ class ParquetFilters( case sources.StringStartsWith(name, prefix) if pushDownStartWith && canMakeFilterOn(name, prefix) => Option(prefix).map { v => - FilterApi.userDefined(binaryColumn(name), + FilterApi.userDefined(binaryColumn(nameToParquetField(name).fieldName), new UserDefinedPredicate[Binary] with Serializable { private val strToBinary = Binary.fromReusedByteArray(v.getBytes) private val size = strToBinary.length diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 286bb1e920266..4e0c1c2dbe601 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1390,6 +1390,27 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } } + + test("SPARK-30826: case insensitivity of StringStartsWith attribute") { + import testImplicits._ + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTable("t1") { + withTempPath { dir => + val path = dir.toURI.toString + Seq("42").toDF("COL").write.parquet(path) + spark.sql( + s""" + |CREATE TABLE t1 (col STRING) + |USING parquet + |OPTIONS (path '$path') + """.stripMargin) + checkAnswer( + spark.sql("SELECT * FROM t1 WHERE col LIKE '4%'"), + Row("42")) + } + } + } + } } class ParquetV1FilterSuite extends ParquetFilterSuite { From f5238ea6cb0d2cfa69ae0488df94b29cc50065e0 Mon Sep 17 00:00:00 2001 From: "Wu, Xiaochang" Date: Sun, 16 Feb 2020 09:51:02 -0600 Subject: [PATCH 1384/1387] [GRAPHX][MINOR] Fix typo setRest => setDest ### What changes were proposed in this pull request? Fix typo def setRest(dstId: VertexId, localDstId: Int, dstAttr: VD, attr: ED) to def setDest(dstId: VertexId, localDstId: Int, dstAttr: VD, attr: ED) ### Why are the changes needed? Typo ### Does this PR introduce any user-facing change? No ### How was this patch tested? N/A Closes #27594 from xwu99/fix-graphx-setDest. Authored-by: Wu, Xiaochang Signed-off-by: Sean Owen --- .../scala/org/apache/spark/graphx/impl/EdgePartition.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 8d03112a1c3dc..c0a2ba67d2942 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -465,7 +465,7 @@ class EdgePartition[ if (edgeIsActive) { val dstAttr = if (tripletFields.useDst) vertexAttrs(localDstId) else null.asInstanceOf[VD] - ctx.setRest(dstId, localDstId, dstAttr, data(pos)) + ctx.setDest(dstId, localDstId, dstAttr, data(pos)) sendMsg(ctx) } pos += 1 @@ -511,7 +511,7 @@ private class AggregatingEdgeContext[VD, ED, A]( _srcAttr = srcAttr } - def setRest(dstId: VertexId, localDstId: Int, dstAttr: VD, attr: ED): Unit = { + def setDest(dstId: VertexId, localDstId: Int, dstAttr: VD, attr: ED): Unit = { _dstId = dstId _localDstId = localDstId _dstAttr = dstAttr From 0a03e7e679771da8556fae72b35edf21ae71ac44 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sun, 16 Feb 2020 09:53:12 -0600 Subject: [PATCH 1385/1387] [SPARK-30691][SQL][DOC][FOLLOW-UP] Make link names exactly the same as the side bar names ### What changes were proposed in this pull request? Make link names exactly the same as the side bar names ### Why are the changes needed? Make doc look better ### Does this PR introduce any user-facing change? before: ![image](https://user-images.githubusercontent.com/13592258/74578603-ad300100-4f4a-11ea-8430-11fccf31eab4.png) after: ![image](https://user-images.githubusercontent.com/13592258/74578670-eff1d900-4f4a-11ea-97d8-5908c0e50e95.png) ### How was this patch tested? Manually build and check the docs Closes #27591 from huaxingao/spark-doc-followup. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- docs/_data/menu-sql.yaml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 1e343f630f88e..38a5cf61245a6 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -157,12 +157,12 @@ - text: Auxiliary Statements url: sql-ref-syntax-aux.html subitems: - - text: Analyze statement + - text: ANALYZE url: sql-ref-syntax-aux-analyze.html subitems: - text: ANALYZE TABLE url: sql-ref-syntax-aux-analyze-table.html - - text: Caching statements + - text: CACHE url: sql-ref-syntax-aux-cache.html subitems: - text: CACHE TABLE @@ -175,7 +175,7 @@ url: sql-ref-syntax-aux-refresh-table.html - text: REFRESH url: sql-ref-syntax-aux-cache-refresh.md - - text: Describe Commands + - text: DESCRIBE url: sql-ref-syntax-aux-describe.html subitems: - text: DESCRIBE DATABASE @@ -186,7 +186,7 @@ url: sql-ref-syntax-aux-describe-function.html - text: DESCRIBE QUERY url: sql-ref-syntax-aux-describe-query.html - - text: Show commands + - text: SHOW url: sql-ref-syntax-aux-show.html subitems: - text: SHOW COLUMNS @@ -205,14 +205,14 @@ url: sql-ref-syntax-aux-show-partitions.html - text: SHOW CREATE TABLE url: sql-ref-syntax-aux-show-create-table.html - - text: Configuration Management Commands + - text: CONFIGURATION MANAGEMENT url: sql-ref-syntax-aux-conf-mgmt.html subitems: - text: SET url: sql-ref-syntax-aux-conf-mgmt-set.html - text: RESET url: sql-ref-syntax-aux-conf-mgmt-reset.html - - text: Resource Management Commands + - text: RESOURCE MANAGEMENT url: sql-ref-syntax-aux-resource-mgmt.html subitems: - text: ADD FILE From 01cc852982cd065e08f9a652c14a0514f49fb662 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sun, 16 Feb 2020 09:55:03 -0600 Subject: [PATCH 1386/1387] [SPARK-30803][DOCS] Fix the home page link for Scala API document ### What changes were proposed in this pull request? Change the link to the Scala API document. ``` $ git grep "#org.apache.spark.package" docs/_layouts/global.html:
  • Scala
  • docs/index.md:* [Spark Scala API (Scaladoc)](api/scala/index.html#org.apache.spark.package) docs/rdd-programming-guide.md:[Scala](api/scala/#org.apache.spark.package), [Java](api/java/), [Python](api/python/) and [R](api/R/). ``` ### Why are the changes needed? The home page link for Scala API document is incorrect after upgrade to 3.0 ### Does this PR introduce any user-facing change? Document UI change only. ### How was this patch tested? Local test, attach screenshots below: Before: ![image](https://user-images.githubusercontent.com/4833765/74335713-c2385300-4dd7-11ea-95d8-f5a3639d2578.png) After: ![image](https://user-images.githubusercontent.com/4833765/74335727-cbc1bb00-4dd7-11ea-89d9-4dcc1310e679.png) Closes #27549 from xuanyuanking/scala-doc. Authored-by: Yuanjian Li Signed-off-by: Sean Owen --- docs/_layouts/global.html | 2 +- docs/configuration.md | 8 +- docs/graphx-programming-guide.md | 68 +++++++------- docs/index.md | 2 +- docs/ml-advanced.md | 10 +- docs/ml-classification-regression.md | 40 ++++---- docs/ml-clustering.md | 10 +- docs/ml-collaborative-filtering.md | 2 +- docs/ml-datasource.md | 4 +- docs/ml-features.md | 92 +++++++++---------- docs/ml-frequent-pattern-mining.md | 4 +- docs/ml-migration-guide.md | 36 ++++---- docs/ml-pipeline.md | 10 +- docs/ml-statistics.md | 8 +- docs/ml-tuning.md | 18 ++-- docs/mllib-clustering.md | 26 +++--- docs/mllib-collaborative-filtering.md | 4 +- docs/mllib-data-types.md | 48 +++++----- docs/mllib-decision-tree.md | 10 +- docs/mllib-dimensionality-reduction.md | 6 +- docs/mllib-ensembles.md | 10 +- docs/mllib-evaluation-metrics.md | 8 +- docs/mllib-feature-extraction.md | 34 +++---- docs/mllib-frequent-pattern-mining.md | 14 +-- docs/mllib-isotonic-regression.md | 2 +- docs/mllib-linear-methods.md | 22 ++--- docs/mllib-naive-bayes.md | 8 +- docs/mllib-optimization.md | 14 +-- docs/mllib-pmml-model-export.md | 2 +- docs/mllib-statistics.md | 28 +++--- docs/quick-start.md | 2 +- docs/rdd-programming-guide.md | 28 +++--- docs/sql-data-sources-generic-options.md | 2 +- docs/sql-data-sources-jdbc.md | 2 +- docs/sql-data-sources-json.md | 2 +- docs/sql-getting-started.md | 16 ++-- docs/sql-migration-guide.md | 4 +- docs/sql-programming-guide.md | 2 +- docs/sql-ref-syntax-aux-analyze-table.md | 2 +- docs/sql-ref-syntax-aux-cache-refresh.md | 2 +- docs/sql-ref-syntax-aux-refresh-table.md | 2 +- docs/sql-ref-syntax-aux-resource-mgmt.md | 2 +- docs/sql-ref-syntax-aux-show-tables.md | 2 +- docs/sql-ref-syntax-aux-show.md | 2 +- docs/sql-ref-syntax-ddl-drop-database.md | 2 +- docs/sql-ref-syntax-ddl-drop-function.md | 2 +- ...tax-dml-insert-overwrite-directory-hive.md | 2 +- ...f-syntax-dml-insert-overwrite-directory.md | 2 +- docs/sql-ref-syntax-dml.md | 2 +- docs/sql-ref-syntax-qry-select-clusterby.md | 2 +- ...sql-ref-syntax-qry-select-distribute-by.md | 2 +- docs/sql-ref-syntax-qry-select-sortby.md | 2 +- docs/sql-ref-syntax-qry-select.md | 2 +- docs/streaming-custom-receivers.md | 2 +- docs/streaming-kafka-integration.md | 2 +- docs/streaming-kinesis-integration.md | 2 +- docs/streaming-programming-guide.md | 42 ++++----- .../structured-streaming-programming-guide.md | 22 ++--- docs/tuning.md | 2 +- 59 files changed, 355 insertions(+), 355 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index d5fb18bfb06c0..d05ac6bbe129d 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -82,7 +82,7 @@
    table_identifier
    @@ -37,10 +39,12 @@ UNCACHE TABLE [ IF EXISTS ] table_identifier